From e4b5d9818f003be2b9091c48f8435d29202ffe2d Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Thu, 2 May 2019 17:24:43 +0100 Subject: [PATCH 01/78] Update token metadata for non-normal state changes Patch by Benedict Elliot Smith; reviewed by Sam Tunnicliffe for CASSANDRA-15120 --- CHANGES.txt | 1 + .../cassandra/concurrent/SEPExecutor.java | 10 +- .../cassandra/concurrent/SEPWorker.java | 33 ++-- .../concurrent/SharedExecutorPool.java | 8 +- .../cassandra/concurrent/StageManager.java | 2 +- .../cassandra/net/MessagingService.java | 10 ++ .../cassandra/service/StorageService.java | 163 ++++++++++-------- .../apache/cassandra/utils/ExpiringMap.java | 5 + .../apache/cassandra/distributed/Cluster.java | 18 +- .../distributed/UpgradeableCluster.java | 12 +- .../cassandra/distributed/api/Feature.java | 24 +++ .../cassandra/distributed/api/IInstance.java | 5 +- .../distributed/impl/AbstractCluster.java | 58 +++++-- .../impl/DelegatingInvokableInstance.java | 6 +- .../cassandra/distributed/impl/Instance.java | 46 +++-- .../distributed/impl/InstanceClassLoader.java | 2 +- .../distributed/test/DistributedTestBase.java | 1 + .../distributed/test/GossipTest.java | 113 ++++++++++++ .../cassandra/concurrent/SEPExecutorTest.java | 2 +- .../apache/cassandra/service/MoveTest.java | 11 +- 20 files changed, 393 insertions(+), 137 deletions(-) create mode 100644 test/distributed/org/apache/cassandra/distributed/api/Feature.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/GossipTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 6f6bd70a5dc1..36eb9c20a0c1 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Update token metadata when handling MOVING/REMOVING_TOKEN events (CASSANDRA-15120) * Add ability to customize cassandra log directory using $CASSANDRA_LOG_DIR (CASSANDRA-15090) * Skip cells with illegal column names when reading legacy sstables (CASSANDRA-15086) * Fix assorted gossip races and add related runtime checks (CASSANDRA-15059) diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java index 8b12b82eb52b..d5c7b1421994 100644 --- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java +++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java @@ -174,7 +174,11 @@ void returnWorkPermit() long current = permits.get(); int workPermits = workPermits(current); if (permits.compareAndSet(current, updateWorkPermits(current, workPermits + 1))) - return; + { + if (shuttingDown && workPermits + 1 == maxWorkers) + shutdown.signalAll(); + break; + } } } @@ -206,7 +210,7 @@ public synchronized void shutdown() { shuttingDown = true; pool.executors.remove(this); - if (getActiveCount() == 0) + if (getActiveCount() == 0 && getPendingTasks() == 0) shutdown.signalAll(); // release metrics @@ -219,6 +223,8 @@ public synchronized List shutdownNow() List aborted = new ArrayList<>(); while (takeTaskPermit()) aborted.add(tasks.poll()); + if (getActiveCount() == 0) + shutdown.signalAll(); return aborted; } diff --git a/src/java/org/apache/cassandra/concurrent/SEPWorker.java b/src/java/org/apache/cassandra/concurrent/SEPWorker.java index edc31daa8cdc..f7eb47aa0d7c 100644 --- a/src/java/org/apache/cassandra/concurrent/SEPWorker.java +++ b/src/java/org/apache/cassandra/concurrent/SEPWorker.java @@ -98,7 +98,6 @@ public void run() // if we do have tasks assigned, nobody will change our state so we can simply set it to WORKING // (which is also a state that will never be interrupted externally) set(Work.WORKING); - boolean shutdown; while (true) { // before we process any task, we maybe schedule a new worker _to our executor only_; this @@ -111,19 +110,13 @@ public void run() task = null; // if we're shutting down, or we fail to take a permit, we don't perform any more work - if ((shutdown = assigned.shuttingDown) || !assigned.takeTaskPermit()) + if (!assigned.takeTaskPermit()) break; task = assigned.tasks.poll(); } // return our work permit, and maybe signal shutdown assigned.returnWorkPermit(); - if (shutdown) - { - if (assigned.getActiveCount() == 0) - assigned.shutdown.signalAll(); - return; - } assigned = null; // try to immediately reassign ourselves some work; if we fail, start spinning @@ -134,22 +127,24 @@ public void run() catch (Throwable t) { JVMStabilityInspector.inspectThrowable(t); - while (true) + if (task != null) + logger.error("Failed to execute task, unexpected exception killed worker: {}", t); + else + logger.error("Unexpected exception killed worker: {}", t); + } + finally + { + if (assigned != null) + assigned.returnWorkPermit(); + + do { if (get().assigned != null) { - assigned = get().assigned; + get().assigned.returnWorkPermit(); set(Work.WORKING); } - if (assign(Work.STOPPED, true)) - break; - } - if (assigned != null) - assigned.returnWorkPermit(); - if (task != null) - logger.error("Failed to execute task, unexpected exception killed worker: {}", t); - else - logger.error("Unexpected exception killed worker: {}", t); + } while (!assign(Work.STOPPED, true)); } } diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java index d355d77b4b7a..3997c1afec74 100644 --- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java +++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.concurrent; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentSkipListMap; @@ -107,16 +108,17 @@ void maybeStartSpinningWorker() schedule(Work.SPINNING); } - public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name) + public synchronized LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name) { SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maxQueuedTasks, jmxPath, name); executors.add(executor); return executor; } - public void shutdown() throws InterruptedException + public synchronized void shutdownAndWait() throws InterruptedException { shuttingDown = true; + List executors = new ArrayList<>(this.executors); for (SEPExecutor executor : executors) executor.shutdownNow(); @@ -127,7 +129,7 @@ public void shutdown() throws InterruptedException executor.shutdown.await(until - System.nanoTime(), TimeUnit.NANOSECONDS); } - void terminateWorkers() + private void terminateWorkers() { assert shuttingDown; diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java index 860377810bae..2f90a2951080 100644 --- a/src/java/org/apache/cassandra/concurrent/StageManager.java +++ b/src/java/org/apache/cassandra/concurrent/StageManager.java @@ -125,7 +125,7 @@ public void run() public static void shutdownAndWait() throws InterruptedException { for (Stage stage : Stage.values()) - StageManager.stages.get(stage).shutdown(); + StageManager.stages.get(stage).shutdownNow(); for (Stage stage : Stage.values()) StageManager.stages.get(stage).awaitTermination(60, TimeUnit.SECONDS); } diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java index a76df0d306d5..82b26ea7cae1 100644 --- a/src/java/org/apache/cassandra/net/MessagingService.java +++ b/src/java/org/apache/cassandra/net/MessagingService.java @@ -815,12 +815,19 @@ public void clearCallbacksUnsafe() * Wait for callbacks and don't allow any more to be created (since they could require writing hints) */ public void shutdown() + { + shutdown(true); + } + public void shutdown(boolean gracefully) { logger.info("Waiting for messaging service to quiesce"); // We may need to schedule hints on the mutation stage, so it's erroneous to shut down the mutation stage first assert !StageManager.getStage(Stage.MUTATION).isShutdown(); // the important part + if (!gracefully) + callbacks.reset(); + if (!callbacks.shutdownBlocking()) logger.warn("Failed to wait for messaging service callbacks shutdown"); @@ -829,6 +836,7 @@ public void shutdown() { clearMessageSinks(); for (SocketThread th : socketThreads) + { try { th.close(); @@ -838,6 +846,8 @@ public void shutdown() // see https://issues.apache.org/jira/browse/CASSANDRA-10545 handleIOExceptionOnClose(e); } + } + connectionManagers.values().forEach(OutboundTcpConnectionPool::close); } catch (IOException e) { diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index a1f361daf0fb..4769b226ff55 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -718,7 +718,8 @@ private boolean shouldBootstrap() return DatabaseDescriptor.isAutoBootstrap() && !SystemKeyspace.bootstrapComplete() && !DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()); } - private void prepareToJoin() throws ConfigurationException + @VisibleForTesting + public void prepareToJoin() throws ConfigurationException { if (!joined) { @@ -787,7 +788,8 @@ else if (shouldBootstrap()) } } - private void joinTokenRing(int delay) throws ConfigurationException + @VisibleForTesting + public void joinTokenRing(int delay) throws ConfigurationException { joined = true; @@ -2049,6 +2051,85 @@ private void handleStateBootreplacing(InetAddress newNode, String[] pieces) tokenMetadata.updateHostId(Gossiper.instance.getHostId(newNode), newNode); } + private void ensureUpToDateTokenMetadata(String status, InetAddress endpoint) + { + Set tokens = new TreeSet<>(getTokensFor(endpoint)); + + if (logger.isDebugEnabled()) + logger.debug("Node {} state {}, tokens {}", endpoint, status, tokens); + + // If the node is previously unknown or tokens do not match, update tokenmetadata to + // have this node as 'normal' (it must have been using this token before the + // leave). This way we'll get pending ranges right. + if (!tokenMetadata.isMember(endpoint)) + { + logger.info("Node {} state jump to {}", endpoint, status); + updateTokenMetadata(endpoint, tokens); + } + else if (!tokens.equals(new TreeSet<>(tokenMetadata.getTokens(endpoint)))) + { + logger.warn("Node {} '{}' token mismatch. Long network partition?", endpoint, status); + updateTokenMetadata(endpoint, tokens); + } + } + + private void updateTokenMetadata(InetAddress endpoint, Iterable tokens) + { + updateTokenMetadata(endpoint, tokens, new HashSet<>()); + } + + private void updateTokenMetadata(InetAddress endpoint, Iterable tokens, Set endpointsToRemove) + { + Set tokensToUpdateInMetadata = new HashSet<>(); + Set tokensToUpdateInSystemKeyspace = new HashSet<>(); + + for (final Token token : tokens) + { + // we don't want to update if this node is responsible for the token and it has a later startup time than endpoint. + InetAddress currentOwner = tokenMetadata.getEndpoint(token); + if (currentOwner == null) + { + logger.debug("New node {} at token {}", endpoint, token); + tokensToUpdateInMetadata.add(token); + tokensToUpdateInSystemKeyspace.add(token); + } + else if (endpoint.equals(currentOwner)) + { + // set state back to normal, since the node may have tried to leave, but failed and is now back up + tokensToUpdateInMetadata.add(token); + tokensToUpdateInSystemKeyspace.add(token); + } + else if (Gossiper.instance.compareEndpointStartup(endpoint, currentOwner) > 0) + { + tokensToUpdateInMetadata.add(token); + tokensToUpdateInSystemKeyspace.add(token); + + // currentOwner is no longer current, endpoint is. Keep track of these moves, because when + // a host no longer has any tokens, we'll want to remove it. + Multimap epToTokenCopy = getTokenMetadata().getEndpointToTokenMapForReading(); + epToTokenCopy.get(currentOwner).remove(token); + if (epToTokenCopy.get(currentOwner).isEmpty()) + endpointsToRemove.add(currentOwner); + + logger.info("Nodes {} and {} have the same token {}. {} is the new owner", endpoint, currentOwner, token, endpoint); + } + else + { + logger.info("Nodes () and {} have the same token {}. Ignoring {}", endpoint, currentOwner, token, endpoint); + } + } + + tokenMetadata.updateNormalTokens(tokensToUpdateInMetadata, endpoint); + for (InetAddress ep : endpointsToRemove) + { + removeEndpoint(ep); + if (replacing && ep.equals(DatabaseDescriptor.getReplaceAddress())) + Gossiper.instance.replacementQuarantine(ep); // quarantine locally longer than normally; see CASSANDRA-8260 + } + if (!tokensToUpdateInSystemKeyspace.isEmpty()) + SystemKeyspace.updateTokens(endpoint, tokensToUpdateInSystemKeyspace, StageManager.getStage(Stage.MUTATION)); + } + /** * Handle node move to normal state. That is, node is entering token ring and participating * in reads. @@ -2058,8 +2139,6 @@ private void handleStateBootreplacing(InetAddress newNode, String[] pieces) private void handleStateNormal(final InetAddress endpoint, final String status) { Collection tokens = getTokensFor(endpoint); - Set tokensToUpdateInMetadata = new HashSet<>(); - Set tokensToUpdateInSystemKeyspace = new HashSet<>(); Set endpointsToRemove = new HashSet<>(); if (logger.isDebugEnabled()) @@ -2127,62 +2206,11 @@ else if (Gossiper.instance.compareEndpointStartup(endpoint, existing) > 0) tokenMetadata.updateHostId(hostId, endpoint); } - for (final Token token : tokens) - { - // we don't want to update if this node is responsible for the token and it has a later startup time than endpoint. - InetAddress currentOwner = tokenMetadata.getEndpoint(token); - if (currentOwner == null) - { - logger.debug("New node {} at token {}", endpoint, token); - tokensToUpdateInMetadata.add(token); - tokensToUpdateInSystemKeyspace.add(token); - } - else if (endpoint.equals(currentOwner)) - { - // set state back to normal, since the node may have tried to leave, but failed and is now back up - tokensToUpdateInMetadata.add(token); - tokensToUpdateInSystemKeyspace.add(token); - } - else if (Gossiper.instance.compareEndpointStartup(endpoint, currentOwner) > 0) - { - tokensToUpdateInMetadata.add(token); - tokensToUpdateInSystemKeyspace.add(token); - - // currentOwner is no longer current, endpoint is. Keep track of these moves, because when - // a host no longer has any tokens, we'll want to remove it. - Multimap epToTokenCopy = getTokenMetadata().getEndpointToTokenMapForReading(); - epToTokenCopy.get(currentOwner).remove(token); - if (epToTokenCopy.get(currentOwner).size() < 1) - endpointsToRemove.add(currentOwner); - - logger.info(String.format("Nodes %s and %s have the same token %s. %s is the new owner", - endpoint, - currentOwner, - token, - endpoint)); - } - else - { - logger.info(String.format("Nodes %s and %s have the same token %s. Ignoring %s", - endpoint, - currentOwner, - token, - endpoint)); - } - } - // capture because updateNormalTokens clears moving and member status boolean isMember = tokenMetadata.isMember(endpoint); boolean isMoving = tokenMetadata.isMoving(endpoint); - tokenMetadata.updateNormalTokens(tokensToUpdateInMetadata, endpoint); - for (InetAddress ep : endpointsToRemove) - { - removeEndpoint(ep); - if (replacing && DatabaseDescriptor.getReplaceAddress().equals(ep)) - Gossiper.instance.replacementQuarantine(ep); // quarantine locally longer than normally; see CASSANDRA-8260 - } - if (!tokensToUpdateInSystemKeyspace.isEmpty()) - SystemKeyspace.updateTokens(endpoint, tokensToUpdateInSystemKeyspace, StageManager.getStage(Stage.MUTATION)); + + updateTokenMetadata(endpoint, tokens, endpointsToRemove); if (isMoving || operationMode == Mode.MOVING) { @@ -2204,24 +2232,11 @@ else if (!isMember) // prior to this, the node was not a member */ private void handleStateLeaving(InetAddress endpoint) { - Collection tokens = getTokensFor(endpoint); - - if (logger.isDebugEnabled()) - logger.debug("Node {} state leaving, tokens {}", endpoint, tokens); - // If the node is previously unknown or tokens do not match, update tokenmetadata to // have this node as 'normal' (it must have been using this token before the // leave). This way we'll get pending ranges right. - if (!tokenMetadata.isMember(endpoint)) - { - logger.info("Node {} state jump to leaving", endpoint); - tokenMetadata.updateNormalTokens(tokens, endpoint); - } - else if (!tokenMetadata.getTokens(endpoint).containsAll(tokens)) - { - logger.warn("Node {} 'leaving' token mismatch. Long network partition?", endpoint); - tokenMetadata.updateNormalTokens(tokens, endpoint); - } + + ensureUpToDateTokenMetadata(VersionedValue.STATUS_LEAVING, endpoint); // at this point the endpoint is certainly a member with this token, so let's proceed // normally @@ -2254,6 +2269,8 @@ private void handleStateLeft(InetAddress endpoint, String[] pieces) */ private void handleStateMoving(InetAddress endpoint, String[] pieces) { + ensureUpToDateTokenMetadata(VersionedValue.STATUS_MOVING, endpoint); + assert pieces.length >= 2; Token token = getTokenFactory().fromString(pieces[1]); @@ -2299,6 +2316,8 @@ private void handleStateRemoving(InetAddress endpoint, String[] pieces) } else if (VersionedValue.REMOVING_TOKEN.equals(state)) { + ensureUpToDateTokenMetadata(state, endpoint); + if (logger.isDebugEnabled()) logger.debug("Tokens {} removed manually (endpoint was {})", removeTokens, endpoint); diff --git a/src/java/org/apache/cassandra/utils/ExpiringMap.java b/src/java/org/apache/cassandra/utils/ExpiringMap.java index ef013f57fdd6..a6895c5d1981 100644 --- a/src/java/org/apache/cassandra/utils/ExpiringMap.java +++ b/src/java/org/apache/cassandra/utils/ExpiringMap.java @@ -121,6 +121,11 @@ public boolean shutdownBlocking() public void reset() { shutdown = false; + clear(); + } + + public void clear() + { cache.clear(); } diff --git a/test/distributed/org/apache/cassandra/distributed/Cluster.java b/test/distributed/org/apache/cassandra/distributed/Cluster.java index c7f7675cb847..4ae4e5d340d2 100644 --- a/test/distributed/org/apache/cassandra/distributed/Cluster.java +++ b/test/distributed/org/apache/cassandra/distributed/Cluster.java @@ -19,10 +19,10 @@ package org.apache.cassandra.distributed; import java.io.File; -import java.io.IOException; -import java.nio.file.Files; import java.util.List; +import java.util.Set; +import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.impl.AbstractCluster; import org.apache.cassandra.distributed.impl.IInvokableInstance; @@ -35,9 +35,9 @@ */ public class Cluster extends AbstractCluster implements ICluster, AutoCloseable { - private Cluster(File root, Versions.Version version, List configs, ClassLoader sharedClassLoader) + private Cluster(File root, Versions.Version version, List configs, Set features, ClassLoader sharedClassLoader) { - super(root, version, configs, sharedClassLoader); + super(root, version, configs, features, sharedClassLoader); } protected IInvokableInstance newInstanceWrapper(Versions.Version version, InstanceConfig config) @@ -49,9 +49,17 @@ public static Cluster create(int nodeCount) throws Throwable { return create(nodeCount, Cluster::new); } + public static Cluster create(int nodeCount, Set with) throws Throwable + { + return create(nodeCount, with, Cluster::new); + } public static Cluster create(int nodeCount, File root) { - return create(nodeCount, Versions.CURRENT, root, Cluster::new); + return create(nodeCount, root, Cluster::new); + } + public static Cluster create(int nodeCount, File root, Set with) + { + return create(nodeCount, Versions.CURRENT, root, with, Cluster::new); } } diff --git a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java index 0c8e63ae9101..d0613b10a978 100644 --- a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java @@ -22,7 +22,9 @@ import java.io.IOException; import java.nio.file.Files; import java.util.List; +import java.util.Set; +import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.impl.AbstractCluster; import org.apache.cassandra.distributed.impl.IUpgradeableInstance; @@ -38,9 +40,9 @@ */ public class UpgradeableCluster extends AbstractCluster implements ICluster, AutoCloseable { - private UpgradeableCluster(File root, Versions.Version version, List configs, ClassLoader sharedClassLoader) + private UpgradeableCluster(File root, Versions.Version version, List configs, Set features, ClassLoader sharedClassLoader) { - super(root, version, configs, sharedClassLoader); + super(root, version, configs, features, sharedClassLoader); } protected IUpgradeableInstance newInstanceWrapper(Versions.Version version, InstanceConfig config) @@ -52,15 +54,17 @@ public static UpgradeableCluster create(int nodeCount) throws Throwable { return create(nodeCount, UpgradeableCluster::new); } + public static UpgradeableCluster create(int nodeCount, File root) { - return create(nodeCount, Versions.CURRENT, root, UpgradeableCluster::new); + return create(nodeCount, root, UpgradeableCluster::new); } public static UpgradeableCluster create(int nodeCount, Versions.Version version) throws IOException { - return create(nodeCount, version, Files.createTempDirectory("dtests").toFile(), UpgradeableCluster::new); + return create(nodeCount, version, UpgradeableCluster::new); } + public static UpgradeableCluster create(int nodeCount, Versions.Version version, File root) { return create(nodeCount, version, root, UpgradeableCluster::new); diff --git a/test/distributed/org/apache/cassandra/distributed/api/Feature.java b/test/distributed/org/apache/cassandra/distributed/api/Feature.java new file mode 100644 index 000000000000..a5c9316930e3 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/api/Feature.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.api; + +public enum Feature +{ + NETWORK, GOSSIP +} diff --git a/test/distributed/org/apache/cassandra/distributed/api/IInstance.java b/test/distributed/org/apache/cassandra/distributed/api/IInstance.java index 3834093fbbec..25e2c94e4dbb 100644 --- a/test/distributed/org/apache/cassandra/distributed/api/IInstance.java +++ b/test/distributed/org/apache/cassandra/distributed/api/IInstance.java @@ -20,6 +20,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; +import java.util.Set; import java.util.UUID; import java.util.concurrent.Future; @@ -37,10 +38,12 @@ public interface IInstance extends IIsolatedExecutor UUID schemaVersion(); void startup(); + boolean isShutdown(); Future shutdown(); + Future shutdown(boolean graceful); // these methods are not for external use, but for simplicity we leave them public and on the normal IInstance interface - void startup(ICluster cluster); + void startup(ICluster cluster, Set with); void receiveMessage(IMessage message); int getMessagingVersion(); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java index c27d9bf2d892..67c844f7a272 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java @@ -30,9 +30,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -47,6 +44,7 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICoordinator; import org.apache.cassandra.distributed.api.IInstance; import org.apache.cassandra.distributed.api.IInstanceConfig; @@ -94,6 +92,7 @@ public abstract class AbstractCluster implements ICluster, private final File root; private final ClassLoader sharedClassLoader; + private final Set features; // mutated by starting/stopping a node private final List instances; @@ -136,23 +135,28 @@ public IInstanceConfig config() return config; } + public boolean isShutdown() + { + return isShutdown; + } + @Override public synchronized void startup() { if (!isShutdown) throw new IllegalStateException(); - delegate().startup(AbstractCluster.this); + delegate().startup(AbstractCluster.this, features); isShutdown = false; updateMessagingVersions(); } @Override - public synchronized Future shutdown() + public synchronized Future shutdown(boolean graceful) { if (isShutdown) throw new IllegalStateException(); isShutdown = true; - Future future = delegate.shutdown(); + Future future = delegate.shutdown(graceful); delegate = null; return future; } @@ -181,9 +185,10 @@ public synchronized void setVersion(Versions.Version version) } } - protected AbstractCluster(File root, Versions.Version version, List configs, ClassLoader sharedClassLoader) + protected AbstractCluster(File root, Versions.Version version, List configs, Set features, ClassLoader sharedClassLoader) { this.root = root; + this.features = features; this.sharedClassLoader = sharedClassLoader; this.instances = new ArrayList<>(); this.instanceMap = new HashMap<>(); @@ -325,36 +330,59 @@ public void schemaChange(String statement, int instance) get(instance).schemaChangeInternal(statement); } - void startup() + public void startup() { - parallelForEach(I::startup, 0, null); + forEach(I::startup); } protected interface Factory> { - C newCluster(File root, Versions.Version version, List configs, ClassLoader sharedClassLoader); + C newCluster(File root, Versions.Version version, List configs, Set features, ClassLoader sharedClassLoader); } protected static > C create(int nodeCount, Factory factory) throws Throwable { - return create(nodeCount, Files.createTempDirectory("dtests").toFile(), factory); + return create(nodeCount, Collections.emptySet(), factory); + } + + protected static > C + create(int nodeCount, Set features, Factory factory) throws Throwable + { + return create(nodeCount, Files.createTempDirectory("dtests").toFile(), features, factory); } protected static > C create(int nodeCount, File root, Factory factory) { - return create(nodeCount, Versions.CURRENT, root, factory); + return create(nodeCount, root, Collections.emptySet(), factory); + } + + protected static > C + create(int nodeCount, File root, Set features, Factory factory) + { + return create(nodeCount, Versions.CURRENT, root, features, factory); } protected static > C create(int nodeCount, Versions.Version version, Factory factory) throws IOException { - return create(nodeCount, version, Files.createTempDirectory("dtests").toFile(), factory); + return create(nodeCount, version, Collections.emptySet(), factory); + } + + protected static > C + create(int nodeCount, Versions.Version version, Set features, Factory factory) throws IOException + { + return create(nodeCount, version, Files.createTempDirectory("dtests").toFile(), features, factory); } protected static > C create(int nodeCount, Versions.Version version, File root, Factory factory) + { + return create(nodeCount, version, root, Collections.emptySet(), factory); + } + protected static > C + create(int nodeCount, Versions.Version version, File root, Set features, Factory factory) { root.mkdirs(); setupLogging(root); @@ -370,8 +398,7 @@ protected interface Factory> token += increment; } - C cluster = factory.newCluster(root, version, configs, sharedClassLoader); - cluster.startup(); + C cluster = factory.newCluster(root, version, configs, features, sharedClassLoader); return cluster; } @@ -398,6 +425,7 @@ private static void setupLogging(File root) public void close() { FBUtilities.waitOnFutures(instances.stream() + .filter(i -> !i.isShutdown()) .map(IInstance::shutdown) .collect(Collectors.toList()), 1L, TimeUnit.MINUTES); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java b/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java index e9e684463d1c..94df6cd11d25 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/DelegatingInvokableInstance.java @@ -19,6 +19,7 @@ package org.apache.cassandra.distributed.impl; import java.io.Serializable; +import java.util.Set; import java.util.UUID; import java.util.concurrent.Future; import java.util.function.BiConsumer; @@ -26,6 +27,7 @@ import java.util.function.Consumer; import java.util.function.Function; +import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.api.ICoordinator; import org.apache.cassandra.distributed.api.IInstanceConfig; @@ -110,9 +112,9 @@ public Future shutdown() } @Override - public void startup(ICluster cluster) + public void startup(ICluster cluster, Set with) { - delegate().startup(cluster); + delegate().startup(cluster, with); } @Override diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 94dbc9684ffe..1b385fbc9aef 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -26,6 +26,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -36,7 +37,6 @@ import org.slf4j.LoggerFactory; import ch.qos.logback.classic.LoggerContext; -import com.codahale.metrics.MetricFilter; import org.apache.cassandra.batchlog.BatchlogManager; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.concurrent.SharedExecutorPool; @@ -56,6 +56,7 @@ import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.api.ICoordinator; import org.apache.cassandra.distributed.api.IInstanceConfig; @@ -98,6 +99,10 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance this.config = config; InstanceIDDefiner.setInstanceId(config.num()); FBUtilities.setBroadcastInetAddress(config.broadcastAddressAndPort().address); + acceptsOnInstance((IInstanceConfig override) -> { + Config.setOverrideLoadConfig(() -> loadConfig(override)); + DatabaseDescriptor.setDaemonInitialized(); + }).accept(config); } public IInstanceConfig config() @@ -145,6 +150,11 @@ public void startup() throw new UnsupportedOperationException(); } + public boolean isShutdown() + { + throw new UnsupportedOperationException(); + } + @Override public void schemaChangeInternal(String query) { @@ -249,18 +259,15 @@ public void setMessagingVersion(InetAddressAndPort endpoint, int version) } @Override - public void startup(ICluster cluster) + public void startup(ICluster cluster, Set with) { sync(() -> { try { mkdirs(); - - Config.setOverrideLoadConfig(() -> loadConfig(config)); - DatabaseDescriptor.setDaemonInitialized(); DatabaseDescriptor.createAllDirectories(); - // We need to persist this as soon as possible after startup checks. + // We need to persist this as soon as possible after startup checks. // This should be the first write to SystemKeyspace (CASSANDRA-11742) SystemKeyspace.persistLocalMetadata(); LegacySchemaMigrator.migrate(); @@ -287,8 +294,17 @@ public void startup(ICluster cluster) throw new RuntimeException(e); } - initializeRing(cluster); - registerMockMessaging(cluster); + // TODO: support each separately + if (with.contains(Feature.GOSSIP) || with.contains(Feature.NETWORK)) + { + StorageService.instance.prepareToJoin(); + StorageService.instance.joinTokenRing(1000); + } + else + { + initializeRing(cluster); + registerMockMessaging(cluster); + } SystemKeyspace.finishStartup(); @@ -376,6 +392,14 @@ private void initializeRing(ICluster cluster) public Future shutdown() { + return shutdown(true); + } + + public Future shutdown(boolean graceful) + { + if (!graceful) + MessagingService.instance().shutdown(false); + Future future = async((ExecutorService executor) -> { Throwable error = null; error = parallelRun(error, executor, @@ -383,7 +407,6 @@ public Future shutdown() CompactionManager.instance::forceShutdown, BatchlogManager.instance::shutdown, HintsService.instance::shutdownBlocking, - CommitLog.instance::shutdownBlocking, SecondaryIndexManager::shutdownExecutors, ColumnFamilyStore::shutdownFlushExecutor, ColumnFamilyStore::shutdownPostFlushExecutor, @@ -401,7 +424,10 @@ public Future shutdown() ); error = parallelRun(error, executor, StageManager::shutdownAndWait, - SharedExecutorPool.SHARED::shutdown + SharedExecutorPool.SHARED::shutdownAndWait + ); + error = parallelRun(error, executor, + CommitLog.instance::shutdownBlocking ); LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java index 56c80740d9d3..57530e093462 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java @@ -47,7 +47,7 @@ public class InstanceClassLoader extends URLClassLoader || name.startsWith("sun.") || name.startsWith("oracle.") || name.startsWith("com.sun.") - || name.startsWith("com.oracle.") + || name.startsWith("com.sun.") || name.startsWith("java.") || name.startsWith("javax.") || name.startsWith("jdk.") diff --git a/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java index 18ca17f04d8b..3945ec574627 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java @@ -49,6 +49,7 @@ public static void setup() protected static > C init(C cluster) { + cluster.startup(); cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + cluster.size() + "};"); return cluster; } diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java new file mode 100644 index 000000000000..11e9985a6c87 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/GossipTest.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.net.InetAddress; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; +import java.util.stream.Collectors; + +import com.google.common.collect.Iterables; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.EndpointState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.utils.FBUtilities; + +public class GossipTest extends DistributedTestBase +{ + + @Test + public void nodeDownDuringMove() throws Throwable + { + int liveCount = 1; + System.setProperty("cassandra.consistent.rangemovement", "false"); + try (Cluster cluster = Cluster.create(2 + liveCount, EnumSet.of(Feature.GOSSIP))) + { + int fail = liveCount + 1; + int late = fail + 1; + for (int i = 1 ; i <= liveCount ; ++i) + cluster.get(i).startup(); + cluster.get(fail).startup(); + Collection expectTokens = cluster.get(fail).callsOnInstance(() -> + StorageService.instance.getTokenMetadata().getTokens(FBUtilities.getBroadcastAddress()) + .stream().map(Object::toString).collect(Collectors.toList()) + ).call(); + + InetAddress failAddress = cluster.get(fail).broadcastAddressAndPort().address; + // wait for NORMAL state + for (int i = 1 ; i <= liveCount ; ++i) + { + cluster.get(i).acceptsOnInstance((InetAddress endpoint) -> { + EndpointState ep; + while (null == (ep = Gossiper.instance.getEndpointStateForEndpoint(endpoint)) + || ep.getApplicationState(ApplicationState.STATUS) == null + || !ep.getApplicationState(ApplicationState.STATUS).value.startsWith("NORMAL")) + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10L)); + }).accept(failAddress); + } + + // set ourselves to MOVING, and wait for it to propagate + cluster.get(fail).runOnInstance(() -> { + + Token token = Iterables.getFirst(StorageService.instance.getTokenMetadata().getTokens(FBUtilities.getBroadcastAddress()), null); + Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.moving(token)); + }); + + for (int i = 1 ; i <= liveCount ; ++i) + { + cluster.get(i).acceptsOnInstance((InetAddress endpoint) -> { + EndpointState ep; + while (null == (ep = Gossiper.instance.getEndpointStateForEndpoint(endpoint)) + || (ep.getApplicationState(ApplicationState.STATUS) == null + || !ep.getApplicationState(ApplicationState.STATUS).value.startsWith("MOVING"))) + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10L)); + }).accept(failAddress); + } + + cluster.get(fail).shutdown(false).get(); + cluster.get(late).startup(); + cluster.get(late).acceptsOnInstance((InetAddress endpoint) -> { + EndpointState ep; + while (null == (ep = Gossiper.instance.getEndpointStateForEndpoint(endpoint)) + || !ep.getApplicationState(ApplicationState.STATUS).value.startsWith("MOVING")) + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10L)); + }).accept(failAddress); + + Collection tokens = cluster.get(late).appliesOnInstance((InetAddress endpoint) -> + StorageService.instance.getTokenMetadata().getTokens(failAddress) + .stream().map(Object::toString).collect(Collectors.toList()) + ).apply(failAddress); + + Assert.assertEquals(expectTokens, tokens); + } + } + +} diff --git a/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java b/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java index 011a8bac1746..e4c695cdf07a 100644 --- a/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java +++ b/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java @@ -56,7 +56,7 @@ public void write(int b) { } } // shutdown does not guarantee that threads are actually dead once it exits, only that they will stop promptly afterwards - sharedPool.shutdown(); + sharedPool.shutdownAndWait(); for (Thread thread : Thread.getAllStackTraces().keySet()) { if (thread.getName().contains(MAGIC)) diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java index 53365aa5aa74..bc6c6d28cc67 100644 --- a/test/unit/org/apache/cassandra/service/MoveTest.java +++ b/test/unit/org/apache/cassandra/service/MoveTest.java @@ -479,7 +479,16 @@ private void finishMove(InetAddress host, int token, TokenMetadata tmd) { tmd.removeFromMoving(host); assertTrue(!tmd.isMoving(host)); - tmd.updateNormalToken(new BigIntegerToken(String.valueOf(token)), host); + Token newToken = new BigIntegerToken(String.valueOf(token)); + tmd.updateNormalToken(newToken, host); + // As well as upating TMD, update the host's tokens in gossip. Since CASSANDRA-15120, status changing to MOVING + // ensures that TMD is up to date with token assignments according to gossip. So we need to make sure gossip has + // the correct new token, as the moving node itself would do upon successful completion of the move operation. + // Without this, the next movement for that host will set the token in TMD's back to the old value from gossip + // and incorrect range movements will follow + Gossiper.instance.injectApplicationState(host, + ApplicationState.TOKENS, + new VersionedValue.VersionedValueFactory(partitioner).tokens(Collections.singleton(newToken))); } private Map.Entry, Collection> generatePendingMapEntry(int start, int end, String... endpoints) throws UnknownHostException From 7f4ecb01247088e27ff2ae15aa5375805bada53a Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe Date: Mon, 3 Jun 2019 16:32:45 +0100 Subject: [PATCH 02/78] Ninja fix RowFilterTest Unit test added in CASSANDRA-15126 is broken on 3.11 --- test/unit/org/apache/cassandra/db/filter/RowFilterTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test/unit/org/apache/cassandra/db/filter/RowFilterTest.java b/test/unit/org/apache/cassandra/db/filter/RowFilterTest.java index 0e15013188e1..9313c3a82434 100644 --- a/test/unit/org/apache/cassandra/db/filter/RowFilterTest.java +++ b/test/unit/org/apache/cassandra/db/filter/RowFilterTest.java @@ -25,6 +25,7 @@ import org.junit.Assert; import org.junit.Test; +import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.config.ColumnDefinition; import org.apache.cassandra.cql3.ColumnIdentifier; @@ -54,6 +55,7 @@ public class RowFilterTest public void testCQLFilterClose() { // CASSANDRA-15126 + SchemaLoader.prepareServer(); CFMetaData metadata = CFMetaData.Builder.create("testks", "testcf") .addPartitionKey("pk", Int32Type.instance) .addStaticColumn("s", Int32Type.instance) From a9a4f171be6c7d33ce43d04a9bc2dd20f3fd82a5 Mon Sep 17 00:00:00 2001 From: Ted Petersson Date: Thu, 6 Jun 2019 11:12:52 -0500 Subject: [PATCH 03/78] Fix JDK7 compatibility broken in cassandra-2.2 Patch by Ted Petersson; Reviewed by Michael Shuler for CASSANDRA-15050 --- CHANGES.txt | 1 + build.xml | 4 ++-- src/java/org/apache/cassandra/utils/MBeanWrapper.java | 7 ++++++- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1cc415312d06..48bf14f5d411 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.2.15 + * Fix JDK7 compatibility broken in cassandra-2.2 (CASSANDRA-15050) * Support cross version messaging in in-jvm upgrade dtests (CASSANDRA-15078) * Fix index summary redistribution cancellation (CASSANDRA-15045) * Refactor Circle CI configuration (CASSANDRA-14806) diff --git a/build.xml b/build.xml index ca06b41e17a6..d522b595a451 100644 --- a/build.xml +++ b/build.xml @@ -71,9 +71,9 @@ - + - + diff --git a/src/java/org/apache/cassandra/utils/MBeanWrapper.java b/src/java/org/apache/cassandra/utils/MBeanWrapper.java index 3b5c7cb70c57..1ee787d8d849 100644 --- a/src/java/org/apache/cassandra/utils/MBeanWrapper.java +++ b/src/java/org/apache/cassandra/utils/MBeanWrapper.java @@ -19,7 +19,6 @@ package org.apache.cassandra.utils; import java.lang.management.ManagementFactory; -import java.util.function.Consumer; import javax.management.MBeanServer; import javax.management.MalformedObjectNameException; import javax.management.ObjectName; @@ -206,4 +205,10 @@ public void accept(Exception e) this.handler = handler; } } + + // Locally defined Consumer interface, to be compatible with Java 7. Only needed for cassandra-2.2 + interface Consumer + { + void accept(T e); + } } From ed9083f8a4d7b5d5631b367187f2e104b448f872 Mon Sep 17 00:00:00 2001 From: Jeremy Hanna Date: Fri, 7 Jun 2019 09:23:28 -0500 Subject: [PATCH 04/78] Update contact page to point to Slack rather than IRC Patch by Jeremy Hanna, Reviewed by Michael Shuler for CASSANDRA-15150 --- doc/source/contactus.rst | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/doc/source/contactus.rst b/doc/source/contactus.rst index 8d0f5dd04663..8cecd901b0cf 100644 --- a/doc/source/contactus.rst +++ b/doc/source/contactus.rst @@ -39,15 +39,10 @@ Subscribe by sending an email to the email address in the Subscribe links above. email to confirm your subscription. Make sure to keep the welcome email as it contains instructions on how to unsubscribe. -.. _irc-channels: - -IRC ---- - -To chat with developers or users in real-time, join our channels on `IRC freenode `__. The -following channels are available: - +.. _slack: +Slack +_____ +To chat with developers or users in real-time, join our rooms on `ASF Slack `__: - ``#cassandra`` - for user questions and general discussions. - ``#cassandra-dev`` - strictly for questions or discussions related to Cassandra development. -- ``#cassandra-builds`` - results of automated test builds. From 6b9575af7e244efe98943e5e7f92b33e252218a3 Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe Date: Wed, 15 May 2019 11:43:03 +0100 Subject: [PATCH 05/78] Add note regarding DROP ROLE and connected sessions patch by Sam Tunnicliffe; reviewed by Michael Shuler --- doc/source/cql/security.rst | 8 ++++++++ doc/source/operating/security.rst | 4 ++++ 2 files changed, 12 insertions(+) diff --git a/doc/source/cql/security.rst b/doc/source/cql/security.rst index 4abeb2d1a603..429a1ef0d67d 100644 --- a/doc/source/cql/security.rst +++ b/doc/source/cql/security.rst @@ -148,6 +148,14 @@ status may ``DROP`` another ``SUPERUSER`` role. Attempting to drop a role which does not exist results in an invalid query condition unless the ``IF EXISTS`` option is used. If the option is used and the role does not exist the statement is a no-op. +.. note:: DROP ROLE intentionally does not terminate any open user sessions. Currently connected sessions will remain + connected and will retain the ability to perform any database actions which do not require :ref:`authorization`. + However, if authorization is enabled, :ref:`permissions` of the dropped role are also revoked, + subject to the :ref:`caching options` configured in :ref:`cassandra.yaml`. + Should a dropped role be subsequently recreated and have new :ref:`permissions` or + :ref:`roles` granted to it, any client sessions still connected will acquire the newly granted + permissions and roles. + .. _grant-role-statement: GRANT ROLE diff --git a/doc/source/operating/security.rst b/doc/source/operating/security.rst index e229c7fa3e68..c2d8b79b0798 100644 --- a/doc/source/operating/security.rst +++ b/doc/source/operating/security.rst @@ -182,6 +182,8 @@ See also: :ref:`setting-credentials-for-internal-authentication`, :ref:`CREATE R :ref:`ALTER ROLE `, :ref:`ALTER KEYSPACE ` and :ref:`GRANT PERMISSION `, +.. _authorization: + Authorization ^^^^^^^^^^^^^ @@ -233,6 +235,8 @@ The following assumes that authentication has already been enabled via the proce See also: :ref:`GRANT PERMISSION `, `GRANT ALL ` and :ref:`REVOKE PERMISSION ` +.. _auth-caching: + Caching ^^^^^^^ From 47dfb6642e7fc3e1e3fef57d8c56c3db577e193a Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Thu, 2 May 2019 15:51:12 +0100 Subject: [PATCH 06/78] Throw IOE instead of asserting on unsupporter peer versions patch by Aleksey Yeschenko; reviewed by Benedict Elliott Smith for CASSANDRA-15066 --- CHANGES.txt | 1 + .../org/apache/cassandra/net/IncomingTcpConnection.java | 7 +++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 36eb9c20a0c1..652833ee76a1 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Throw IOE instead of asserting on unsupporter peer versions (CASSANDRA-15066) * Update token metadata when handling MOVING/REMOVING_TOKEN events (CASSANDRA-15120) * Add ability to customize cassandra log directory using $CASSANDRA_LOG_DIR (CASSANDRA-15090) * Skip cells with illegal column names when reading legacy sstables (CASSANDRA-15086) diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java index de644442f1a6..e79da313ab6a 100644 --- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java +++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java @@ -141,10 +141,13 @@ private void receiveMessages() throws IOException // to connect with, the other node will disconnect out.writeInt(MessagingService.current_version); out.flush(); + + // outbound side will reconnect if necessary to upgrade version + if (version > MessagingService.current_version) + throw new IOException("Peer-used messaging version " + version + " is larger than max supported " + MessagingService.current_version); + DataInputPlus in = new DataInputStreamPlus(socket.getInputStream()); int maxVersion = in.readInt(); - // outbound side will reconnect if necessary to upgrade version - assert version <= MessagingService.current_version; from = CompactEndpointSerializationHelper.deserialize(in); // record the (true) version of the endpoint MessagingService.instance().setVersion(from, maxVersion); From 310a48efcfdc0fb77fbd5fd6e734bf17667f617d Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Wed, 12 Jun 2019 16:26:54 +0100 Subject: [PATCH 07/78] Rewrite the internode messaging subsystem patch by Aleksey Yeschenko and Benedict Elliott Smith; reviewed by Alex Petrov, Aleksey Yeschenko, and Benedict Elliott Smith for CASSANDRA-15066 Co-authored-by: Aleksey Yeschenko Co-authored-by: Benedict Elliott Smith --- build.xml | 15 + conf/cassandra.yaml | 26 + .../apache/cassandra/auth/IAuthenticator.java | 2 +- .../batchlog/BatchRemoveVerbHandler.java | 6 +- .../batchlog/BatchStoreVerbHandler.java | 9 +- .../cassandra/batchlog/BatchlogManager.java | 26 +- .../concurrent/ImmediateExecutor.java | 56 + .../concurrent/InfiniteLoopExecutor.java | 2 +- .../concurrent/NamedThreadFactory.java | 2 + .../concurrent/ScheduledExecutors.java | 15 +- .../concurrent/SharedExecutorPool.java | 9 +- .../apache/cassandra/concurrent/Stage.java | 20 +- .../cassandra/concurrent/StageManager.java | 25 +- .../org/apache/cassandra/config/Config.java | 26 +- .../cassandra/config/DatabaseDescriptor.java | 170 +- .../cassandra/config/EncryptionOptions.java | 312 ++- .../cassandra/db/ColumnFamilyStore.java | 34 +- src/java/org/apache/cassandra/db/Columns.java | 3 +- .../apache/cassandra/db/CounterMutation.java | 18 +- .../db/CounterMutationVerbHandler.java | 18 +- .../org/apache/cassandra/db/IMutation.java | 3 +- .../org/apache/cassandra/db/Keyspace.java | 7 +- .../org/apache/cassandra/db/Mutation.java | 37 +- .../cassandra/db/MutationVerbHandler.java | 52 +- .../db/PartitionRangeReadCommand.java | 20 +- .../org/apache/cassandra/db/ReadCommand.java | 33 +- .../cassandra/db/ReadCommandVerbHandler.java | 38 +- .../cassandra/db/ReadExecutionController.java | 10 +- .../cassandra/db/ReadRepairVerbHandler.java | 8 +- .../cassandra/db/SerializationHeader.java | 5 +- .../db/SinglePartitionReadCommand.java | 24 +- .../apache/cassandra/db/SnapshotCommand.java | 9 +- .../apache/cassandra/db/SystemKeyspace.java | 7 +- .../apache/cassandra/db/TruncateRequest.java | 68 + .../apache/cassandra/db/TruncateResponse.java | 7 - .../cassandra/db/TruncateVerbHandler.java | 28 +- .../org/apache/cassandra/db/Truncation.java | 74 - .../commitlog/AbstractCommitLogService.java | 12 +- .../cassandra/db/context/CounterContext.java | 4 +- .../db/filter/ColumnSubselection.java | 3 +- .../cassandra/db/marshal/CompositeType.java | 2 +- .../db/monitoring/ApproximateTime.java | 61 - .../cassandra/db/monitoring/Monitorable.java | 6 +- .../db/monitoring/MonitorableImpl.java | 46 +- .../db/monitoring/MonitoringTask.java | 76 +- .../CassandraCompressedStreamReader.java | 3 +- .../CassandraCompressedStreamWriter.java | 40 +- .../CassandraEntireSSTableStreamWriter.java | 10 +- .../db/streaming/CassandraOutgoingFile.java | 7 +- .../db/streaming/CassandraStreamReader.java | 9 +- .../db/streaming/CassandraStreamWriter.java | 75 +- .../db/virtual/InternodeInboundTable.java | 135 ++ .../db/virtual/InternodeOutboundTable.java | 140 ++ .../cassandra/db/virtual/SettingsTable.java | 2 +- .../db/virtual/SystemViewsKeyspace.java | 4 +- .../cassandra/db/virtual/VirtualMutation.java | 5 +- .../apache/cassandra/dht/BootStrapper.java | 26 - .../apache/cassandra/dht/IPartitioner.java | 21 + .../IncompatibleSchemaException.java} | 14 +- .../exceptions/RequestFailureException.java | 13 +- .../exceptions/RequestFailureReason.java | 98 +- .../exceptions/UnknownColumnException.java | 26 + .../exceptions/UnknownTableException.java | 4 +- .../org/apache/cassandra/gms/EchoMessage.java | 56 - .../apache/cassandra/gms/FailureDetector.java | 11 +- .../apache/cassandra/gms/GossipDigest.java | 9 +- .../apache/cassandra/gms/GossipDigestAck.java | 9 +- .../cassandra/gms/GossipDigestAck2.java | 9 +- .../gms/GossipDigestAck2VerbHandler.java | 13 +- .../gms/GossipDigestAckVerbHandler.java | 22 +- .../gms/GossipDigestSynVerbHandler.java | 28 +- .../gms/GossipShutdownVerbHandler.java | 10 +- .../cassandra/gms/GossipVerbHandler.java | 30 + .../org/apache/cassandra/gms/Gossiper.java | 67 +- .../cassandra/hints/EncodedHintMessage.java | 94 - .../apache/cassandra/hints/HintMessage.java | 114 +- .../apache/cassandra/hints/HintResponse.java | 58 - .../cassandra/hints/HintVerbHandler.java | 20 +- .../cassandra/hints/HintsDispatcher.java | 52 +- .../SerializableHintMessage.java} | 6 +- .../index/SecondaryIndexManager.java | 13 +- .../cassandra/index/sasi/SASIIndex.java | 5 +- .../io/DummyByteVersionedSerializer.java | 55 - .../io/IVersionedAsymmetricSerializer.java | 53 + .../cassandra/io/IVersionedSerializer.java | 33 +- .../io/sstable/format/SSTableReader.java | 53 +- .../format/big/BigTableZeroCopyWriter.java | 21 +- .../io/util/BufferedDataOutputStreamPlus.java | 100 +- .../cassandra/io/util/DataOutputPlus.java | 13 +- .../io/util/DataOutputStreamPlus.java | 2 +- .../io/util/FastByteArrayInputStream.java | 249 --- .../org/apache/cassandra/io/util/Memory.java | 2 +- .../cassandra/io/util/SequentialWriter.java | 19 + .../util/UnbufferedDataOutputStreamPlus.java | 11 - .../locator/DynamicEndpointSnitch.java | 9 +- .../apache/cassandra/locator/Endpoints.java | 20 + .../cassandra/locator/InetAddressAndPort.java | 135 +- .../locator/ReconnectableSnitchHelper.java | 12 +- .../cassandra/metrics/ConnectionMetrics.java | 154 -- .../metrics/DroppedMessageMetrics.java | 3 +- .../cassandra/metrics/FrequencySampler.java | 18 +- .../metrics/InternodeInboundMetrics.java | 98 + .../metrics/InternodeOutboundMetrics.java | 205 ++ .../apache/cassandra/metrics/MaxSampler.java | 16 +- .../cassandra/metrics/MessagingMetrics.java | 204 +- .../org/apache/cassandra/metrics/Sampler.java | 8 +- .../cassandra/metrics/TableMetrics.java | 2 +- ...{IMessageSink.java => AcceptVersions.java} | 35 +- .../cassandra/net/AsyncChannelOutputPlus.java | 268 +++ .../cassandra/net/AsyncChannelPromise.java | 164 ++ .../cassandra/net/AsyncMessageOutputPlus.java | 131 ++ .../cassandra/net/AsyncOneResponse.java | 39 +- .../apache/cassandra/net/AsyncPromise.java | 488 +++++ .../net/AsyncStreamingInputPlus.java | 251 +++ .../net/AsyncStreamingOutputPlus.java | 220 ++ .../cassandra/net/BackPressureState.java | 2 +- .../cassandra/net/BufferPoolAllocator.java | 116 + .../apache/cassandra/net/CallbackInfo.java | 70 - .../cassandra/net/ChunkedInputPlus.java | 97 + .../CompactEndpointSerializationHelper.java | 128 -- ...thFailure.java => ConnectionCategory.java} | 20 +- .../apache/cassandra/net/ConnectionType.java | 69 + src/java/org/apache/cassandra/net/Crc.java | 136 ++ .../net/EndpointMessagingVersions.java | 94 + .../cassandra/net/ForwardToSerializer.java | 86 - .../apache/cassandra/net/ForwardingInfo.java | 139 ++ .../apache/cassandra/net/FrameDecoder.java | 400 ++++ .../apache/cassandra/net/FrameDecoderCrc.java | 158 ++ .../apache/cassandra/net/FrameDecoderLZ4.java | 165 ++ .../cassandra/net/FrameDecoderLegacy.java | 184 ++ .../cassandra/net/FrameDecoderLegacyLZ4.java | 377 ++++ .../net/FrameDecoderUnprotected.java | 100 + .../net/FrameDecoderWith8bHeader.java | 144 ++ .../apache/cassandra/net/FrameEncoder.java | 140 ++ .../apache/cassandra/net/FrameEncoderCrc.java | 98 + .../apache/cassandra/net/FrameEncoderLZ4.java | 118 + .../cassandra/net/FrameEncoderLegacy.java | 38 + .../cassandra/net/FrameEncoderLegacyLZ4.java | 137 ++ .../net/FrameEncoderUnprotected.java | 66 + .../apache/cassandra/net/FutureCombiner.java | 85 + .../apache/cassandra/net/FutureDelegate.java | 145 ++ ...ufDataInputPlus.java => FutureResult.java} | 40 +- .../net/GlobalBufferPoolAllocator.java | 27 +- .../cassandra/net/HandshakeProtocol.java | 414 ++++ .../apache/cassandra/net/IVerbHandler.java | 4 +- .../net/InboundConnectionInitiator.java | 495 +++++ .../net/InboundConnectionSettings.java | 213 ++ .../apache/cassandra/net/InboundCounters.java | 130 ++ .../net/InboundMessageCallbacks.java | 99 + .../cassandra/net/InboundMessageHandler.java | 1194 +++++++++++ .../cassandra/net/InboundMessageHandlers.java | 447 ++++ .../org/apache/cassandra/net/InboundSink.java | 161 ++ .../apache/cassandra/net/InboundSockets.java | 235 ++ .../net/InvalidSerializedSizeException.java | 45 + .../apache/cassandra/net/LatencyConsumer.java | 25 + .../cassandra/net/LatencySubscribers.java | 75 + .../LegacyFlag.java} | 45 +- .../cassandra/net/LegacyLZ4Constants.java | 54 + .../net/LocalBufferPoolAllocator.java | 64 + .../net/ManyToOneConcurrentLinkedQueue.java | 350 +++ .../org/apache/cassandra/net/Message.java | 1338 ++++++++++++ .../cassandra/net/MessageDeliveryTask.java | 134 -- .../org/apache/cassandra/net/MessageFlag.java | 83 + .../org/apache/cassandra/net/MessageIn.java | 234 -- .../org/apache/cassandra/net/MessageOut.java | 406 ---- .../cassandra/net/MessagingService.java | 1897 ++++------------- .../net/MessagingServiceMBeanImpl.java | 304 +++ .../net/{PongMessage.java => NoPayload.java} | 33 +- .../apache/cassandra/net/NoSizeEstimator.java | 31 + .../cassandra/net/OutboundConnection.java | 1729 +++++++++++++++ .../net/OutboundConnectionInitiator.java | 469 ++++ .../net/OutboundConnectionSettings.java | 517 +++++ .../cassandra/net/OutboundConnections.java | 323 +++ .../cassandra/net/OutboundDebugCallbacks.java | 40 + ...ner.java => OutboundMessageCallbacks.java} | 31 +- .../cassandra/net/OutboundMessageQueue.java | 484 +++++ .../apache/cassandra/net/OutboundSink.java | 108 + .../org/apache/cassandra/net/ParamType.java | 116 + .../apache/cassandra/net/ParameterType.java | 70 - .../org/apache/cassandra/net/PingMessage.java | 82 - .../org/apache/cassandra/net/PingRequest.java | 72 + .../apache/cassandra/net/PingVerbHandler.java | 12 +- .../cassandra/net/PrunableArrayQueue.java | 172 ++ .../cassandra/net/RateBasedBackPressure.java | 4 +- .../net/RateBasedBackPressureState.java | 2 +- ...syncCallback.java => RequestCallback.java} | 31 +- .../cassandra/net/RequestCallbacks.java | 382 ++++ .../apache/cassandra/net/ResourceLimits.java | 201 ++ .../cassandra/net/ResponseVerbHandler.java | 37 +- .../apache/cassandra/net/ShareableBytes.java | 174 ++ .../net/SharedDefaultFileRegion.java | 93 + .../apache/cassandra/net/SocketFactory.java | 327 +++ .../StartupClusterConnectivityChecker.java | 44 +- src/java/org/apache/cassandra/net/Verb.java | 331 +++ .../cassandra/net/WriteCallbackInfo.java | 80 - .../net/async/BaseMessageInHandler.java | 197 -- .../net/async/ByteBufDataOutputPlus.java | 139 -- .../async/ByteBufDataOutputStreamPlus.java | 243 --- .../cassandra/net/async/ChannelWriter.java | 418 ---- .../net/async/HandshakeProtocol.java | 314 --- .../net/async/InboundHandshakeHandler.java | 321 --- .../cassandra/net/async/MessageInHandler.java | 211 -- .../net/async/MessageInHandlerPre40.java | 269 --- .../net/async/MessageOutHandler.java | 324 --- .../cassandra/net/async/MessageResult.java | 51 - .../cassandra/net/async/NettyFactory.java | 418 ---- .../async/NonClosingDefaultFileRegion.java | 51 - .../net/async/OptionalSslHandler.java | 67 - .../async/OutboundConnectionIdentifier.java | 194 -- .../net/async/OutboundConnectionParams.java | 230 -- .../net/async/OutboundHandshakeHandler.java | 262 --- .../async/OutboundMessagingConnection.java | 747 ------- .../net/async/OutboundMessagingPool.java | 177 -- .../cassandra/net/async/QueuedMessage.java | 75 - .../RebufferingByteBufDataInputPlus.java | 291 --- .../repair/AsymmetricRemoteSyncTask.java | 7 +- .../cassandra/repair/RepairJobDesc.java | 5 +- .../repair/RepairMessageVerbHandler.java | 51 +- .../apache/cassandra/repair/SnapshotTask.java | 27 +- .../cassandra/repair/StreamingRepairTask.java | 11 +- .../repair/SymmetricRemoteSyncTask.java | 7 +- .../apache/cassandra/repair/SyncNodePair.java | 15 +- .../cassandra/repair/ValidationTask.java | 5 +- .../apache/cassandra/repair/Validator.java | 8 +- .../repair/consistent/CoordinatorSession.java | 7 +- .../repair/consistent/LocalSessions.java | 9 +- .../messages/AsymmetricSyncRequest.java | 28 +- .../repair/messages/FinalizePromise.java | 9 +- .../messages/PrepareConsistentRequest.java | 15 +- .../messages/PrepareConsistentResponse.java | 9 +- .../repair/messages/PrepareMessage.java | 5 +- .../repair/messages/RepairMessage.java | 9 +- .../repair/messages/SyncRequest.java | 23 +- .../cassandra/schema/CompressionParams.java | 7 +- .../cassandra/schema/MigrationManager.java | 26 +- .../cassandra/schema/MigrationTask.java | 39 +- .../schema/SchemaMigrationEvent.java | 4 +- .../schema/SchemaPullVerbHandler.java | 23 +- .../schema/SchemaPushVerbHandler.java | 10 +- .../schema/SchemaVersionVerbHandler.java | 22 +- .../apache/cassandra/security/SSLFactory.java | 4 +- .../serializers/CollectionSerializer.java | 2 +- .../service/AbstractWriteResponseHandler.java | 39 +- .../service/ActiveRepairService.java | 31 +- .../service/BatchlogResponseHandler.java | 14 +- .../cassandra/service/CassandraDaemon.java | 8 +- .../DatacenterSyncWriteResponseHandler.java | 12 +- .../DatacenterWriteResponseHandler.java | 8 +- .../cassandra/service/EchoVerbHandler.java | 21 +- .../PendingRangeCalculatorService.java | 6 +- .../service/SnapshotVerbHandler.java | 14 +- .../cassandra/service/StorageProxy.java | 307 ++- .../cassandra/service/StorageProxyMBean.java | 2 + .../cassandra/service/StorageService.java | 132 +- .../service/TruncateResponseHandler.java | 20 +- .../service/WriteResponseHandler.java | 9 +- .../service/paxos/AbstractPaxosCallback.java | 19 +- .../service/paxos/CommitVerbHandler.java | 11 +- .../service/paxos/PrepareCallback.java | 10 +- .../service/paxos/PrepareVerbHandler.java | 13 +- .../service/paxos/ProposeCallback.java | 8 +- .../service/paxos/ProposeVerbHandler.java | 14 +- .../service/reads/AbstractReadExecutor.java | 23 +- .../cassandra/service/reads/DataResolver.java | 13 +- .../service/reads/DigestResolver.java | 32 +- .../cassandra/service/reads/ReadCallback.java | 37 +- .../service/reads/ResponseResolver.java | 10 +- .../reads/ShortReadPartitionsProtection.java | 2 +- .../reads/repair/AbstractReadRepair.java | 18 +- .../reads/repair/BlockingPartitionRepair.java | 29 +- .../reads/repair/BlockingReadRepair.java | 5 +- .../reads/repair/BlockingReadRepairs.java | 2 +- .../streaming/DefaultConnectionFactory.java | 93 +- ...r.java => ReplicationDoneVerbHandler.java} | 19 +- .../cassandra/streaming/SessionSummary.java | 16 +- .../streaming/StreamConnectionFactory.java | 4 +- .../cassandra/streaming/StreamRequest.java | 17 +- .../cassandra/streaming/StreamSession.java | 47 +- .../streaming/StreamingMessageSender.java | 6 +- .../async/NettyStreamingMessageSender.java | 60 +- .../async/StreamCompressionSerializer.java | 38 +- .../async/StreamingInboundHandler.java | 35 +- ...yteBufCompressionDataOutputStreamPlus.java | 76 - .../StreamCompressionInputStream.java | 6 +- .../streaming/messages/StreamInitMessage.java | 9 +- .../streaming/messages/StreamMessage.java | 78 +- .../messages/StreamMessageHeader.java | 9 +- .../tools/BulkLoadConnectionFactory.java | 20 +- .../apache/cassandra/tools/BulkLoader.java | 2 +- .../apache/cassandra/tools/LoaderOptions.java | 21 +- .../org/apache/cassandra/tracing/Tracing.java | 65 +- .../cassandra/transport/ProtocolVersion.java | 4 +- .../transport/ProtocolVersionTracker.java | 5 +- .../transport/messages/ErrorMessage.java | 2 +- .../cassandra/utils/ApproximateTime.java | 192 ++ .../cassandra/utils/ByteBufferUtil.java | 8 +- .../org/apache/cassandra/utils/Clock.java | 80 - .../cassandra/utils/CoalescingStrategies.java | 444 ---- .../apache/cassandra/utils/ExecutorUtils.java | 109 + .../apache/cassandra/utils/ExpiringMap.java | 186 -- .../apache/cassandra/utils/FBUtilities.java | 6 - .../utils/JVMStabilityInspector.java | 10 +- .../cassandra/utils/MonotonicClock.java | 346 +++ .../utils/MonotonicClockTranslation.java | 29 + .../utils/NanoTimeToCurrentTimeMillis.java | 72 - .../apache/cassandra/utils/Throwables.java | 14 +- .../cassandra/utils/concurrent/Ref.java | 14 +- .../utils/concurrent/SimpleCondition.java | 11 +- .../cassandra/utils/concurrent/WaitQueue.java | 20 + .../cassandra/utils/memory/BufferPool.java | 878 +++++--- .../cassandra/utils/memory/MemoryUtil.java | 29 +- .../cassandra/utils/memory/MemtablePool.java | 8 +- .../cassandra/utils/vint/VIntCoding.java | 87 +- .../net/BytesInFlightController.java | 174 ++ .../org/apache/cassandra/net/Connection.java | 397 ++++ .../cassandra/net/ConnectionBurnTest.java | 656 ++++++ .../cassandra/net/GlobalInboundSettings.java | 57 + .../apache/cassandra/net/LogbackFilter.java | 80 + .../cassandra/net/MessageGenerator.java | 190 ++ .../cassandra/net/MessageGenerators.java | 45 + .../org/apache/cassandra/net/Reporters.java | 322 +++ .../org/apache/cassandra/net/Verifier.java | 1637 ++++++++++++++ .../utils/memory/LongBufferPoolTest.java | 73 +- test/conf/cassandra-murmur.yaml | 4 +- test/conf/cassandra-seeds.yaml | 2 +- test/conf/cassandra.yaml | 4 +- test/conf/logback-burntest.xml | 66 + test/conf/logback-dtest.xml | 9 - .../4.0/service.SyncComplete.bin | Bin 554 -> 258 bytes .../serialization/4.0/service.SyncRequest.bin | Bin 241 -> 111 bytes .../4.0/service.ValidationComplete.bin | Bin 1257 -> 600 bytes .../4.0/service.ValidationRequest.bin | Bin 169 -> 75 bytes .../apache/cassandra/distributed/Cluster.java | 19 +- .../distributed/UpgradeableCluster.java | 23 +- .../distributed/api/IInstanceConfig.java | 1 + .../cassandra/distributed/api/IMessage.java | 1 + .../distributed/api/IMessageFilters.java | 6 +- .../distributed/impl/AbstractCluster.java | 118 +- .../cassandra/distributed/impl/ExecUtil.java | 38 +- .../cassandra/distributed/impl/Instance.java | 134 +- .../distributed/impl/InstanceClassLoader.java | 10 +- .../distributed/impl/InstanceConfig.java | 31 +- .../distributed/impl/MessageFilters.java | 42 +- .../impl/{Message.java => MessageImpl.java} | 21 +- .../test/DistributedReadWritePathTest.java | 41 +- .../distributed/test/DistributedTestBase.java | 7 +- .../distributed/test/GossipSettlesTest.java | 45 + .../distributed/test/LargeColumnTest.java | 96 + .../distributed/test/RepairTest.java | 134 ++ .../distributed/test/StreamingTest.java | 75 + .../cassandra/distributed/util/PyDtest.java | 186 ++ .../DynamicEndpointSnitchLongTest.java | 4 +- .../test/microbench/MessageOutBench.java | 67 +- .../test/microbench/MutationBench.java | 14 +- .../PreaggregatedByteBufsBench.java | 107 + .../ZeroCopyStreamingBenchmark.java | 16 +- .../batchlog/BatchlogManagerTest.java | 7 +- .../cassandra/concurrent/SEPExecutorTest.java | 4 +- .../apache/cassandra/db/ReadCommandTest.java | 7 +- .../db/ReadCommandVerbHandlerTest.java | 84 +- .../AbstractCommitLogServiceTest.java | 1 - .../compaction/AbstractPendingRepairTest.java | 19 +- .../db/monitoring/MonitoringTaskTest.java | 55 +- ...assandraEntireSSTableStreamWriterTest.java | 14 +- .../db/streaming/ComponentManifestTest.java | 21 +- .../db/virtual/SettingsTableTest.java | 18 +- .../apache/cassandra/gms/ShadowRoundTest.java | 20 +- .../org/apache/cassandra/hints/HintTest.java | 13 +- .../cassandra/hints/HintsServiceTest.java | 20 +- .../cassandra/index/sasi/SASIIndexTest.java | 4 +- .../cassandra/io/compress/CompressorTest.java | 2 +- .../big/BigTableZeroCopyWriterTest.java | 18 +- .../io/util/BufferedDataOutputStreamTest.java | 44 - .../locator/DynamicEndpointSnitchTest.java | 5 +- .../InetAddressAndPortSerializerTest.java} | 18 +- .../net/AsyncChannelPromiseTest.java | 72 + .../net/AsyncMessageOutputPlusTest.java | 100 + .../cassandra/net/AsyncOneResponseTest.java | 30 +- .../cassandra/net/AsyncPromiseTest.java | 73 + ....java => AsyncStreamingInputPlusTest.java} | 148 +- .../net/AsyncStreamingOutputPlusTest.java | 114 + .../cassandra/net/ChunkedInputPlusTest.java | 159 ++ .../apache/cassandra/net/ConnectionTest.java | 811 +++++++ .../apache/cassandra/net/ConnectionUtils.java | 253 +++ ...ainerTest.java => ForwardingInfoTest.java} | 12 +- .../org/apache/cassandra/net/FramingTest.java | 432 ++++ .../apache/cassandra/net/HandshakeTest.java | 219 ++ .../ManyToOneConcurrentLinkedQueueTest.java | 301 +++ .../org/apache/cassandra/net/Matcher.java | 2 +- .../apache/cassandra/net/MatcherResponse.java | 112 +- .../net/MessageDeliveryTaskTest.java | 121 -- .../apache/cassandra/net/MessageInTest.java | 66 - .../org/apache/cassandra/net/MessageTest.java | 254 +++ .../cassandra/net/MessagingServiceTest.java | 470 ++-- .../cassandra/net/MockMessagingService.java | 17 +- .../net/MockMessagingServiceTest.java | 36 +- .../cassandra/net/MockMessagingSpy.java | 36 +- .../net/OutboundConnectionSettingsTest.java | 145 ++ .../net/OutboundConnectionsTest.java | 162 ++ .../net/OutboundMessageQueueTest.java | 94 + .../cassandra/net/PrunableArrayQueueTest.java | 130 ++ .../cassandra/net/ResourceLimitsTest.java | 152 ++ .../org/apache/cassandra/net/SocketUtils.java | 57 + ...StartupClusterConnectivityCheckerTest.java | 91 +- .../net/TestAbstractAsyncPromise.java | 234 ++ .../cassandra/net/TestAbstractPromise.java | 112 + .../org/apache/cassandra/net/TestChannel.java | 127 ++ .../net/{async => }/TestScheduledFuture.java | 2 +- .../cassandra/net/WriteCallbackInfoTest.java | 10 +- .../net/async/ByteBufDataOutputPlusTest.java | 178 -- .../net/async/ChannelWriterTest.java | 314 --- .../net/async/HandshakeHandlersTest.java | 227 -- .../net/async/HandshakeProtocolTest.java | 95 - .../async/InboundHandshakeHandlerTest.java | 291 --- .../net/async/MessageInHandlerTest.java | 339 --- .../net/async/MessageOutHandlerTest.java | 288 --- .../cassandra/net/async/NettyFactoryTest.java | 336 --- ...NonSendingOutboundMessagingConnection.java | 42 - .../async/OutboundConnectionParamsTest.java | 67 - .../async/OutboundHandshakeHandlerTest.java | 258 --- .../OutboundMessagingConnectionTest.java | 521 ----- .../net/async/OutboundMessagingPoolTest.java | 150 -- .../cassandra/repair/LocalSyncTaskTest.java | 2 - .../cassandra/repair/RepairJobTest.java | 75 +- .../cassandra/repair/ValidatorTest.java | 51 +- .../consistent/CoordinatorMessagingTest.java | 26 +- .../cassandra/security/SSLFactoryTest.java | 87 +- .../service/NativeTransportServiceTest.java | 12 +- .../apache/cassandra/service/RemoveTest.java | 15 +- .../cassandra/service/SerializationsTest.java | 17 +- .../service/WriteResponseHandlerTest.java | 48 +- .../reads/AbstractReadResponseTest.java | 16 +- .../service/reads/ReadExecutorTest.java | 26 +- .../reads/repair/AbstractReadRepairTest.java | 20 +- .../reads/repair/BlockingReadRepairTest.java | 4 +- .../DiagEventsBlockingReadRepairTest.java | 4 +- .../service/reads/repair/ReadRepairTest.java | 4 +- .../streaming/StreamSessionTest.java | 67 - .../streaming/StreamTransferTaskTest.java | 2 +- .../NettyStreamingMessageSenderTest.java | 10 +- .../StreamCompressionSerializerTest.java | 25 +- .../async/StreamingInboundHandlerTest.java | 43 +- .../cassandra/utils/ByteBufferUtilTest.java | 4 +- .../utils/CoalescingStrategiesTest.java | 124 -- .../cassandra/utils/FreeRunningClock.java | 28 +- ...illisTest.java => MonotonicClockTest.java} | 7 +- .../utils/memory/BufferPoolTest.java | 84 +- .../cassandra/utils/vint/VIntCodingTest.java | 12 +- .../stress/generate/SeedManager.java | 1 - .../stress/settings/SettingsTransport.java | 23 +- .../stress/util/JavaDriverClient.java | 2 +- 451 files changed, 30729 insertions(+), 16729 deletions(-) create mode 100644 src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java create mode 100644 src/java/org/apache/cassandra/db/TruncateRequest.java delete mode 100644 src/java/org/apache/cassandra/db/Truncation.java delete mode 100644 src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java create mode 100644 src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java create mode 100644 src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java rename src/java/org/apache/cassandra/{net/async/ExpiredException.java => exceptions/IncompatibleSchemaException.java} (71%) create mode 100644 src/java/org/apache/cassandra/exceptions/UnknownColumnException.java delete mode 100644 src/java/org/apache/cassandra/gms/EchoMessage.java create mode 100644 src/java/org/apache/cassandra/gms/GossipVerbHandler.java delete mode 100644 src/java/org/apache/cassandra/hints/EncodedHintMessage.java delete mode 100644 src/java/org/apache/cassandra/hints/HintResponse.java rename src/java/org/apache/cassandra/{locator/ILatencySubscriber.java => hints/SerializableHintMessage.java} (84%) delete mode 100644 src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java create mode 100644 src/java/org/apache/cassandra/io/IVersionedAsymmetricSerializer.java delete mode 100644 src/java/org/apache/cassandra/io/util/FastByteArrayInputStream.java delete mode 100644 src/java/org/apache/cassandra/metrics/ConnectionMetrics.java create mode 100644 src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java create mode 100644 src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java rename src/java/org/apache/cassandra/net/{IMessageSink.java => AcceptVersions.java} (62%) create mode 100644 src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java create mode 100644 src/java/org/apache/cassandra/net/AsyncChannelPromise.java create mode 100644 src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java create mode 100644 src/java/org/apache/cassandra/net/AsyncPromise.java create mode 100644 src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java create mode 100644 src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java create mode 100644 src/java/org/apache/cassandra/net/BufferPoolAllocator.java delete mode 100644 src/java/org/apache/cassandra/net/CallbackInfo.java create mode 100644 src/java/org/apache/cassandra/net/ChunkedInputPlus.java delete mode 100644 src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java rename src/java/org/apache/cassandra/net/{IAsyncCallbackWithFailure.java => ConnectionCategory.java} (69%) create mode 100644 src/java/org/apache/cassandra/net/ConnectionType.java create mode 100644 src/java/org/apache/cassandra/net/Crc.java create mode 100644 src/java/org/apache/cassandra/net/EndpointMessagingVersions.java delete mode 100644 src/java/org/apache/cassandra/net/ForwardToSerializer.java create mode 100644 src/java/org/apache/cassandra/net/ForwardingInfo.java create mode 100644 src/java/org/apache/cassandra/net/FrameDecoder.java create mode 100644 src/java/org/apache/cassandra/net/FrameDecoderCrc.java create mode 100644 src/java/org/apache/cassandra/net/FrameDecoderLZ4.java create mode 100644 src/java/org/apache/cassandra/net/FrameDecoderLegacy.java create mode 100644 src/java/org/apache/cassandra/net/FrameDecoderLegacyLZ4.java create mode 100644 src/java/org/apache/cassandra/net/FrameDecoderUnprotected.java create mode 100644 src/java/org/apache/cassandra/net/FrameDecoderWith8bHeader.java create mode 100644 src/java/org/apache/cassandra/net/FrameEncoder.java create mode 100644 src/java/org/apache/cassandra/net/FrameEncoderCrc.java create mode 100644 src/java/org/apache/cassandra/net/FrameEncoderLZ4.java create mode 100644 src/java/org/apache/cassandra/net/FrameEncoderLegacy.java create mode 100644 src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java create mode 100644 src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java create mode 100644 src/java/org/apache/cassandra/net/FutureCombiner.java create mode 100644 src/java/org/apache/cassandra/net/FutureDelegate.java rename src/java/org/apache/cassandra/net/{async/ByteBufDataInputPlus.java => FutureResult.java} (54%) rename test/unit/org/apache/cassandra/net/async/TestAuthenticator.java => src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java (60%) create mode 100644 src/java/org/apache/cassandra/net/HandshakeProtocol.java create mode 100644 src/java/org/apache/cassandra/net/InboundConnectionInitiator.java create mode 100644 src/java/org/apache/cassandra/net/InboundConnectionSettings.java create mode 100644 src/java/org/apache/cassandra/net/InboundCounters.java create mode 100644 src/java/org/apache/cassandra/net/InboundMessageCallbacks.java create mode 100644 src/java/org/apache/cassandra/net/InboundMessageHandler.java create mode 100644 src/java/org/apache/cassandra/net/InboundMessageHandlers.java create mode 100644 src/java/org/apache/cassandra/net/InboundSink.java create mode 100644 src/java/org/apache/cassandra/net/InboundSockets.java create mode 100644 src/java/org/apache/cassandra/net/InvalidSerializedSizeException.java create mode 100644 src/java/org/apache/cassandra/net/LatencyConsumer.java create mode 100644 src/java/org/apache/cassandra/net/LatencySubscribers.java rename src/java/org/apache/cassandra/{db/WriteResponse.java => net/LegacyFlag.java} (52%) create mode 100644 src/java/org/apache/cassandra/net/LegacyLZ4Constants.java create mode 100644 src/java/org/apache/cassandra/net/LocalBufferPoolAllocator.java create mode 100644 src/java/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueue.java create mode 100644 src/java/org/apache/cassandra/net/Message.java delete mode 100644 src/java/org/apache/cassandra/net/MessageDeliveryTask.java create mode 100644 src/java/org/apache/cassandra/net/MessageFlag.java delete mode 100644 src/java/org/apache/cassandra/net/MessageIn.java delete mode 100644 src/java/org/apache/cassandra/net/MessageOut.java create mode 100644 src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java rename src/java/org/apache/cassandra/net/{PongMessage.java => NoPayload.java} (59%) create mode 100644 src/java/org/apache/cassandra/net/NoSizeEstimator.java create mode 100644 src/java/org/apache/cassandra/net/OutboundConnection.java create mode 100644 src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java create mode 100644 src/java/org/apache/cassandra/net/OutboundConnectionSettings.java create mode 100644 src/java/org/apache/cassandra/net/OutboundConnections.java create mode 100644 src/java/org/apache/cassandra/net/OutboundDebugCallbacks.java rename src/java/org/apache/cassandra/net/{ForwardToContainer.java => OutboundMessageCallbacks.java} (55%) create mode 100644 src/java/org/apache/cassandra/net/OutboundMessageQueue.java create mode 100644 src/java/org/apache/cassandra/net/OutboundSink.java create mode 100644 src/java/org/apache/cassandra/net/ParamType.java delete mode 100644 src/java/org/apache/cassandra/net/ParameterType.java delete mode 100644 src/java/org/apache/cassandra/net/PingMessage.java create mode 100644 src/java/org/apache/cassandra/net/PingRequest.java create mode 100644 src/java/org/apache/cassandra/net/PrunableArrayQueue.java rename src/java/org/apache/cassandra/net/{IAsyncCallback.java => RequestCallback.java} (61%) create mode 100644 src/java/org/apache/cassandra/net/RequestCallbacks.java create mode 100644 src/java/org/apache/cassandra/net/ResourceLimits.java create mode 100644 src/java/org/apache/cassandra/net/ShareableBytes.java create mode 100644 src/java/org/apache/cassandra/net/SharedDefaultFileRegion.java create mode 100644 src/java/org/apache/cassandra/net/SocketFactory.java create mode 100644 src/java/org/apache/cassandra/net/Verb.java delete mode 100644 src/java/org/apache/cassandra/net/WriteCallbackInfo.java delete mode 100644 src/java/org/apache/cassandra/net/async/BaseMessageInHandler.java delete mode 100644 src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java delete mode 100644 src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java delete mode 100644 src/java/org/apache/cassandra/net/async/ChannelWriter.java delete mode 100644 src/java/org/apache/cassandra/net/async/HandshakeProtocol.java delete mode 100644 src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java delete mode 100644 src/java/org/apache/cassandra/net/async/MessageInHandler.java delete mode 100644 src/java/org/apache/cassandra/net/async/MessageInHandlerPre40.java delete mode 100644 src/java/org/apache/cassandra/net/async/MessageOutHandler.java delete mode 100644 src/java/org/apache/cassandra/net/async/MessageResult.java delete mode 100644 src/java/org/apache/cassandra/net/async/NettyFactory.java delete mode 100644 src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java delete mode 100644 src/java/org/apache/cassandra/net/async/OptionalSslHandler.java delete mode 100644 src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java delete mode 100644 src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java delete mode 100644 src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java delete mode 100644 src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java delete mode 100644 src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java delete mode 100644 src/java/org/apache/cassandra/net/async/QueuedMessage.java delete mode 100644 src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java rename src/java/org/apache/cassandra/streaming/{ReplicationFinishedVerbHandler.java => ReplicationDoneVerbHandler.java} (67%) delete mode 100644 src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.java create mode 100644 src/java/org/apache/cassandra/utils/ApproximateTime.java delete mode 100644 src/java/org/apache/cassandra/utils/Clock.java delete mode 100644 src/java/org/apache/cassandra/utils/CoalescingStrategies.java create mode 100644 src/java/org/apache/cassandra/utils/ExecutorUtils.java delete mode 100644 src/java/org/apache/cassandra/utils/ExpiringMap.java create mode 100644 src/java/org/apache/cassandra/utils/MonotonicClock.java create mode 100644 src/java/org/apache/cassandra/utils/MonotonicClockTranslation.java delete mode 100644 src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java create mode 100644 test/burn/org/apache/cassandra/net/BytesInFlightController.java create mode 100644 test/burn/org/apache/cassandra/net/Connection.java create mode 100644 test/burn/org/apache/cassandra/net/ConnectionBurnTest.java create mode 100644 test/burn/org/apache/cassandra/net/GlobalInboundSettings.java create mode 100644 test/burn/org/apache/cassandra/net/LogbackFilter.java create mode 100644 test/burn/org/apache/cassandra/net/MessageGenerator.java create mode 100644 test/burn/org/apache/cassandra/net/MessageGenerators.java create mode 100644 test/burn/org/apache/cassandra/net/Reporters.java create mode 100644 test/burn/org/apache/cassandra/net/Verifier.java create mode 100644 test/conf/logback-burntest.xml rename src/java/org/apache/cassandra/io/ShortVersionedSerializer.java => test/distributed/org/apache/cassandra/distributed/impl/ExecUtil.java (53%) rename test/distributed/org/apache/cassandra/distributed/impl/{Message.java => MessageImpl.java} (79%) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/GossipSettlesTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/RepairTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java create mode 100644 test/distributed/org/apache/cassandra/distributed/util/PyDtest.java create mode 100644 test/microbench/org/apache/cassandra/test/microbench/PreaggregatedByteBufsBench.java rename test/unit/org/apache/cassandra/{net/CompactEndpointSerializationHelperTest.java => locator/InetAddressAndPortSerializerTest.java} (75%) create mode 100644 test/unit/org/apache/cassandra/net/AsyncChannelPromiseTest.java create mode 100644 test/unit/org/apache/cassandra/net/AsyncMessageOutputPlusTest.java create mode 100644 test/unit/org/apache/cassandra/net/AsyncPromiseTest.java rename test/unit/org/apache/cassandra/net/{async/RebufferingByteBufDataInputPlusTest.java => AsyncStreamingInputPlusTest.java} (60%) create mode 100644 test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java create mode 100644 test/unit/org/apache/cassandra/net/ChunkedInputPlusTest.java create mode 100644 test/unit/org/apache/cassandra/net/ConnectionTest.java create mode 100644 test/unit/org/apache/cassandra/net/ConnectionUtils.java rename test/unit/org/apache/cassandra/net/{ForwardToContainerTest.java => ForwardingInfoTest.java} (89%) create mode 100644 test/unit/org/apache/cassandra/net/FramingTest.java create mode 100644 test/unit/org/apache/cassandra/net/HandshakeTest.java create mode 100644 test/unit/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueueTest.java delete mode 100644 test/unit/org/apache/cassandra/net/MessageDeliveryTaskTest.java delete mode 100644 test/unit/org/apache/cassandra/net/MessageInTest.java create mode 100644 test/unit/org/apache/cassandra/net/MessageTest.java create mode 100644 test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java create mode 100644 test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java create mode 100644 test/unit/org/apache/cassandra/net/OutboundMessageQueueTest.java create mode 100644 test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java create mode 100644 test/unit/org/apache/cassandra/net/ResourceLimitsTest.java create mode 100644 test/unit/org/apache/cassandra/net/SocketUtils.java create mode 100644 test/unit/org/apache/cassandra/net/TestAbstractAsyncPromise.java create mode 100644 test/unit/org/apache/cassandra/net/TestAbstractPromise.java create mode 100644 test/unit/org/apache/cassandra/net/TestChannel.java rename test/unit/org/apache/cassandra/net/{async => }/TestScheduledFuture.java (97%) delete mode 100644 test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.java delete mode 100644 test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java delete mode 100644 test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java delete mode 100644 test/unit/org/apache/cassandra/streaming/StreamSessionTest.java delete mode 100644 test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java rename test/unit/org/apache/cassandra/utils/{NanoTimeToCurrentTimeMillisTest.java => MonotonicClockTest.java} (87%) diff --git a/build.xml b/build.xml index acd3b7adb737..bdf5ae2ab08c 100644 --- a/build.xml +++ b/build.xml @@ -1522,6 +1522,7 @@ + @@ -1945,6 +1946,20 @@ + + + + + + + + + + + + + + diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index ca854ca65e68..300285710946 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -870,6 +870,32 @@ request_timeout_in_ms: 10000 # which picks up the OS default and configure the net.ipv4.tcp_retries2 sysctl to be ~8. # internode_tcp_user_timeout_in_ms = 30000 +# The maximum continuous period a connection may be unwritable in application space +# internode_application_timeout_in_ms = 30000 + +# Global, per-endpoint and per-connection limits imposed on messages queued for delivery to other nodes +# and waiting to be processed on arrival from other nodes in the cluster. These limits are applied to the on-wire +# size of the message being sent or received. +# +# The basic per-link limit is consumed in isolation before any endpoint or global limit is imposed. +# Each node-pair has three links: urgent, small and large. So any given node may have a maximum of +# N*3*(internode_application_send_queue_capacity_in_bytes+internode_application_receive_queue_capacity_in_bytes) +# messages queued without any coordination between them although in practice, with token-aware routing, only RF*tokens +# nodes should need to communicate with significant bandwidth. +# +# The per-endpoint limit is imposed on all messages exceeding the per-link limit, simultaneously with the global limit, +# on all links to or from a single node in the cluster. +# The global limit is imposed on all messages exceeding the per-link limit, simultaneously with the per-endpoint limit, +# on all links to or from any node in the cluster. +# +# internode_application_send_queue_capacity_in_bytes: 4194304 #4MiB +# internode_application_send_queue_reserve_endpoint_capacity_in_bytes: 134217728 #128MiB +# internode_application_send_queue_reserve_global_capacity_in_bytes: 536870912 #512MiB +# internode_application_receive_queue_capacity_in_bytes: 4194304 #4MiB +# internode_application_receive_queue_reserve_endpoint_capacity_in_bytes: 134217728 #128MiB +# internode_application_receive_queue_reserve_global_capacity_in_bytes: 536870912 #512MiB + + # How long before a node logs slow queries. Select queries that take longer than # this timeout to execute, will generate an aggregated log message, so that slow queries # can be identified. Set this value to zero to disable slow query logging. diff --git a/src/java/org/apache/cassandra/auth/IAuthenticator.java b/src/java/org/apache/cassandra/auth/IAuthenticator.java index 212e77495052..80ea719237b9 100644 --- a/src/java/org/apache/cassandra/auth/IAuthenticator.java +++ b/src/java/org/apache/cassandra/auth/IAuthenticator.java @@ -105,7 +105,7 @@ default SaslNegotiator newSaslNegotiator(InetAddress clientAddress, X509Certific public interface SaslNegotiator { /** - * Evaluates the client response data and generates a byte[] reply which may be a further challenge or purely + * Evaluates the client response data and generates a byte[] response which may be a further challenge or purely * informational in the case that the negotiation is completed on this round. * * This method is called each time a {@link org.apache.cassandra.transport.messages.AuthResponse} is received diff --git a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java b/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java index 3c3fcec49096..3443cab78393 100644 --- a/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java +++ b/src/java/org/apache/cassandra/batchlog/BatchRemoveVerbHandler.java @@ -20,11 +20,13 @@ import java.util.UUID; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; public final class BatchRemoveVerbHandler implements IVerbHandler { - public void doVerb(MessageIn message, int id) + public static final BatchRemoveVerbHandler instance = new BatchRemoveVerbHandler(); + + public void doVerb(Message message) { BatchlogManager.remove(message.payload); } diff --git a/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java b/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java index 4bc878cbf592..77335cb44389 100644 --- a/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java +++ b/src/java/org/apache/cassandra/batchlog/BatchStoreVerbHandler.java @@ -17,16 +17,17 @@ */ package org.apache.cassandra.batchlog; -import org.apache.cassandra.db.WriteResponse; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; public final class BatchStoreVerbHandler implements IVerbHandler { - public void doVerb(MessageIn message, int id) + public static final BatchStoreVerbHandler instance = new BatchStoreVerbHandler(); + + public void doVerb(Message message) { BatchlogManager.store(message.payload); - MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from); + MessagingService.instance().send(message.emptyResponse(), message.from()); } } diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java index b2b851df89d0..d0d34111dc24 100644 --- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java +++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java @@ -65,8 +65,8 @@ import org.apache.cassandra.locator.ReplicaLayout; import org.apache.cassandra.locator.ReplicaPlan; import org.apache.cassandra.locator.Replicas; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessageFlag; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableId; @@ -77,8 +77,10 @@ import org.apache.cassandra.utils.UUIDGen; import static com.google.common.collect.Iterables.transform; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; import static org.apache.cassandra.cql3.QueryProcessor.executeInternalWithPaging; +import static org.apache.cassandra.net.Verb.MUTATION_REQ; public class BatchlogManager implements BatchlogManagerMBean { @@ -88,7 +90,7 @@ public class BatchlogManager implements BatchlogManagerMBean private static final Logger logger = LoggerFactory.getLogger(BatchlogManager.class); public static final BatchlogManager instance = new BatchlogManager(); - public static final long BATCHLOG_REPLAY_TIMEOUT = Long.getLong("cassandra.batchlog.replay_timeout_in_ms", DatabaseDescriptor.getWriteRpcTimeout() * 2); + public static final long BATCHLOG_REPLAY_TIMEOUT = Long.getLong("cassandra.batchlog.replay_timeout_in_ms", DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS) * 2); private volatile long totalBatchesReplayed = 0; // no concurrency protection necessary as only written by replay thread. private volatile UUID lastReplayedUuid = UUIDGen.minTimeUUID(0); @@ -112,7 +114,7 @@ public void start() batchlogTasks.scheduleWithFixedDelay(this::replayFailedBatches, StorageService.RING_DELAY, REPLAY_INTERVAL, - TimeUnit.MILLISECONDS); + MILLISECONDS); } public void shutdown() throws InterruptedException @@ -356,7 +358,7 @@ public int replay(RateLimiter rateLimiter, Set hintedNodes) return 0; int gcgs = gcgs(mutations); - if (TimeUnit.MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds()) + if (MILLISECONDS.toSeconds(writtenAt) + gcgs <= FBUtilities.nowInSeconds()) return 0; replayHandlers = sendReplays(mutations, writtenAt, hintedNodes); @@ -419,7 +421,7 @@ private void writeHintsForUndeliveredEndpoints(int startFrom, Set sendSingleReplayMutation(fin ReplicaPlan.ForTokenWrite replicaPlan = new ReplicaPlan.ForTokenWrite(keyspace, ConsistencyLevel.ONE, liveRemoteOnly.pending(), liveRemoteOnly.all(), liveRemoteOnly.all(), liveRemoteOnly.all()); ReplayWriteResponseHandler handler = new ReplayWriteResponseHandler<>(replicaPlan, System.nanoTime()); - MessageOut message = mutation.createMessage(); + Message message = Message.outWithFlag(MUTATION_REQ, mutation, MessageFlag.CALL_BACK_ON_FAILURE); for (Replica replica : liveRemoteOnly.all()) - MessagingService.instance().sendWriteRR(message, replica, handler, false); + MessagingService.instance().sendWriteWithCallback(message, replica, handler, false); return handler; } @@ -506,7 +508,7 @@ private static int gcgs(Collection mutations) /** * A wrapper of WriteResponseHandler that stores the addresses of the endpoints from - * which we did not receive a successful reply. + * which we did not receive a successful response. */ private static class ReplayWriteResponseHandler extends WriteResponseHandler { @@ -525,11 +527,11 @@ protected int blockFor() } @Override - public void response(MessageIn m) + public void onResponse(Message m) { - boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddressAndPort() : m.from); + boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddressAndPort() : m.from()); assert removed; - super.response(m); + super.onResponse(m); } } } diff --git a/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java new file mode 100644 index 000000000000..1a00e4f3beda --- /dev/null +++ b/src/java/org/apache/cassandra/concurrent/ImmediateExecutor.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.concurrent; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.AbstractExecutorService; +import java.util.concurrent.TimeUnit; + +public class ImmediateExecutor extends AbstractExecutorService implements LocalAwareExecutorService +{ + public static final ImmediateExecutor INSTANCE = new ImmediateExecutor(); + + private ImmediateExecutor() {} + + public void execute(Runnable command, ExecutorLocals locals) + { + command.run(); + } + + public void maybeExecuteImmediately(Runnable command) + { + command.run(); + } + + public void execute(Runnable command) + { + command.run(); + } + + public int getActiveTaskCount() { return 0; } + public long getCompletedTaskCount() { return 0; } + public int getPendingTaskCount() { return 0; } + public int getMaximumPoolSize() { return 0; } + public void shutdown() { } + public List shutdownNow() { return Collections.emptyList(); } + public boolean isShutdown() { return false; } + public boolean isTerminated() { return false; } + public boolean awaitTermination(long timeout, TimeUnit unit) { return true; } +} diff --git a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java index 199803f04489..b54fa3fca51f 100644 --- a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java +++ b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java @@ -70,7 +70,7 @@ public InfiniteLoopExecutor start() return this; } - public void shutdown() + public void shutdownNow() { isShutdown = true; thread.interrupt(); diff --git a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java index 33f1312b9f8c..7cc73bd1c3c4 100644 --- a/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java +++ b/src/java/org/apache/cassandra/concurrent/NamedThreadFactory.java @@ -24,6 +24,7 @@ import io.netty.util.concurrent.FastThreadLocal; import io.netty.util.concurrent.FastThreadLocalThread; +import org.apache.cassandra.utils.memory.BufferPool; /** * This class is an implementation of the ThreadFactory interface. This @@ -35,6 +36,7 @@ public class NamedThreadFactory implements ThreadFactory { private static volatile String globalPrefix; public static void setGlobalPrefix(String prefix) { globalPrefix = prefix; } + public static String globalPrefix() { return globalPrefix == null ? "" : globalPrefix; } public final String id; private final int priority; diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java index 5e3e5cf3964c..90ceca5373f4 100644 --- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java +++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java @@ -17,10 +17,15 @@ */ package org.apache.cassandra.concurrent; +import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.utils.ExecutorUtils; /** * Centralized location for shared executors @@ -48,12 +53,10 @@ public class ScheduledExecutors public static final DebuggableScheduledThreadPoolExecutor optionalTasks = new DebuggableScheduledThreadPoolExecutor("OptionalTasks"); @VisibleForTesting - public static void shutdownAndWait() throws InterruptedException + public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - ExecutorService[] executors = new ExecutorService[] { scheduledFastTasks, scheduledTasks, nonPeriodicTasks, optionalTasks }; - for (ExecutorService executor : executors) - executor.shutdownNow(); - for (ExecutorService executor : executors) - executor.awaitTermination(60, TimeUnit.SECONDS); + List executors = ImmutableList.of(scheduledFastTasks, scheduledTasks, nonPeriodicTasks, optionalTasks); + ExecutorUtils.shutdownNow(executors); + ExecutorUtils.awaitTermination(timeout, unit, executors); } } diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java index 62bede9add1a..53792eca481e 100644 --- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java +++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java @@ -22,6 +22,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.LockSupport; @@ -114,7 +115,7 @@ public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTa return executor; } - public void shutdown() throws InterruptedException + public void shutdown(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { shuttingDown = true; for (SEPExecutor executor : executors) @@ -122,9 +123,13 @@ public void shutdown() throws InterruptedException terminateWorkers(); - long until = System.nanoTime() + TimeUnit.MINUTES.toNanos(1L); + long until = System.nanoTime() + unit.toNanos(timeout); for (SEPExecutor executor : executors) + { executor.shutdown.await(until - System.nanoTime(), TimeUnit.NANOSECONDS); + if (!executor.isTerminated()) + throw new TimeoutException(executor.name + " not terminated"); + } } void terminateWorkers() diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java index ccb156501e40..ed13eebe2b47 100644 --- a/src/java/org/apache/cassandra/concurrent/Stage.java +++ b/src/java/org/apache/cassandra/concurrent/Stage.java @@ -17,11 +17,6 @@ */ package org.apache.cassandra.concurrent; -import java.util.Arrays; - -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; - public enum Stage { READ, @@ -35,18 +30,7 @@ public enum Stage MISC, TRACING, INTERNAL_RESPONSE, - READ_REPAIR; - - public static Iterable jmxEnabledStages() - { - return Iterables.filter(Arrays.asList(values()), new Predicate() - { - public boolean apply(Stage stage) - { - return stage != TRACING; - } - }); - } + IMMEDIATE; public String getJmxType() { @@ -58,13 +42,13 @@ public String getJmxType() case MISC: case TRACING: case INTERNAL_RESPONSE: + case IMMEDIATE: return "internal"; case MUTATION: case COUNTER_MUTATION: case VIEW_MUTATION: case READ: case REQUEST_RESPONSE: - case READ_REPAIR: return "request"; default: throw new AssertionError("Unknown stage " + this); diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java index 608a00595216..46e8ceaa680e 100644 --- a/src/java/org/apache/cassandra/concurrent/StageManager.java +++ b/src/java/org/apache/cassandra/concurrent/StageManager.java @@ -17,7 +17,9 @@ */ package org.apache.cassandra.concurrent; +import java.util.Collections; import java.util.EnumMap; +import java.util.List; import java.util.concurrent.*; import com.google.common.annotations.VisibleForTesting; @@ -25,9 +27,11 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.config.DatabaseDescriptor.*; +import static org.apache.cassandra.utils.ExecutorUtils.*; /** @@ -56,24 +60,18 @@ public class StageManager stages.put(Stage.ANTI_ENTROPY, new JMXEnabledThreadPoolExecutor(Stage.ANTI_ENTROPY)); stages.put(Stage.MIGRATION, new JMXEnabledThreadPoolExecutor(Stage.MIGRATION)); stages.put(Stage.MISC, new JMXEnabledThreadPoolExecutor(Stage.MISC)); - stages.put(Stage.READ_REPAIR, multiThreadedStage(Stage.READ_REPAIR, FBUtilities.getAvailableProcessors())); stages.put(Stage.TRACING, tracingExecutor()); + stages.put(Stage.IMMEDIATE, ImmediateExecutor.INSTANCE); } private static LocalAwareExecutorService tracingExecutor() { - RejectedExecutionHandler reh = new RejectedExecutionHandler() - { - public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) - { - MessagingService.instance().incrementDroppedMessages(MessagingService.Verb._TRACE); - } - }; + RejectedExecutionHandler reh = (r, executor) -> MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._TRACE); return new TracingExecutor(1, 1, KEEPALIVE, TimeUnit.SECONDS, - new ArrayBlockingQueue(1000), + new ArrayBlockingQueue<>(1000), new NamedThreadFactory(Stage.TRACING.getJmxName()), reh); } @@ -114,12 +112,10 @@ public static void shutdownNow() } @VisibleForTesting - public static void shutdownAndWait() throws InterruptedException + public static void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException { - for (Stage stage : Stage.values()) - StageManager.stages.get(stage).shutdown(); - for (Stage stage : Stage.values()) - StageManager.stages.get(stage).awaitTermination(60, TimeUnit.SECONDS); + shutdown(StageManager.stages.values()); + awaitTermination(timeout, units, StageManager.stages.values()); } /** @@ -155,4 +151,5 @@ public int getPendingTaskCount() return getQueue().size(); } } + } diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index a6050bea8baa..9713ea2109ff 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -149,8 +149,21 @@ public class Config public boolean rpc_interface_prefer_ipv6 = false; public String broadcast_rpc_address; public boolean rpc_keepalive = true; - public int internode_send_buff_size_in_bytes = 0; - public int internode_recv_buff_size_in_bytes = 0; + + public Integer internode_max_message_size_in_bytes; + + public int internode_socket_send_buffer_size_in_bytes = 0; + public int internode_socket_receive_buffer_size_in_bytes = 0; + + // TODO: derive defaults from system memory settings? + public int internode_application_send_queue_capacity_in_bytes = 1 << 22; // 4MiB + public int internode_application_send_queue_reserve_endpoint_capacity_in_bytes = 1 << 27; // 128MiB + public int internode_application_send_queue_reserve_global_capacity_in_bytes = 1 << 29; // 512MiB + + public int internode_application_receive_queue_capacity_in_bytes = 1 << 22; // 4MiB + public int internode_application_receive_queue_reserve_endpoint_capacity_in_bytes = 1 << 27; // 128MiB + public int internode_application_receive_queue_reserve_global_capacity_in_bytes = 1 << 29; // 512MiB + // Defensive settings for protecting Cassandra from true network partitions. See (CASSANDRA-14358) for details. // The amount of time to wait for internode tcp connections to establish. public int internode_tcp_connect_timeout_in_ms = 2000; @@ -171,6 +184,7 @@ public class Config public volatile boolean native_transport_allow_older_protocols = true; public int native_transport_frame_block_size_in_kb = 32; + /** * Max size of values in SSTables, in MegaBytes. * Default is the same as the native protocol frame limit: 256Mb. @@ -322,7 +336,7 @@ public class Config public volatile ConsistencyLevel ideal_consistency_level = null; /* - * Strategy to use for coalescing messages in {@link OutboundMessagingPool}. + * Strategy to use for coalescing messages in {@link OutboundConnections}. * Can be fixed, movingaverage, timehorizon, disabled. Setting is case and leading/trailing * whitespace insensitive. You can also specify a subclass of * {@link org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy} by name. @@ -339,12 +353,6 @@ public class Config public int otc_coalescing_window_us = otc_coalescing_window_us_default; public int otc_coalescing_enough_coalesced_messages = 8; - /** - * Backlog expiration interval in milliseconds for the OutboundTcpConnection. - */ - public static final int otc_backlog_expiration_interval_ms_default = 200; - public volatile int otc_backlog_expiration_interval_ms = otc_backlog_expiration_interval_ms_default; - public int windows_timer_interval = 0; /** diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index b3ab054ce95a..3b7009b113b2 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -27,6 +27,7 @@ import java.nio.file.Paths; import java.util.*; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; @@ -72,6 +73,7 @@ import org.apache.commons.lang3.StringUtils; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.cassandra.io.util.FileUtils.ONE_GB; public class DatabaseDescriptor @@ -80,6 +82,7 @@ public class DatabaseDescriptor { // This static block covers most usages FBUtilities.preventIllegalAccessWarnings(); + System.setProperty("io.netty.transport.estimateSizeOnSubmit", "false"); } private static final Logger logger = LoggerFactory.getLogger(DatabaseDescriptor.class); @@ -801,6 +804,28 @@ else if (conf.max_value_size_in_mb >= 2048) if (conf.otc_coalescing_enough_coalesced_messages <= 0) throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false); + Integer maxMessageSize = conf.internode_max_message_size_in_bytes; + if (maxMessageSize != null) + { + if (maxMessageSize > conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes) + throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_receive_queue_reserve_endpoint_capacity_in_bytes", false); + + if (maxMessageSize > conf.internode_application_receive_queue_reserve_global_capacity_in_bytes) + throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_receive_queue_reserve_global_capacity_in_bytes", false); + + if (maxMessageSize > conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes) + throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_send_queue_reserve_endpoint_capacity_in_bytes", false); + + if (maxMessageSize > conf.internode_application_send_queue_reserve_global_capacity_in_bytes) + throw new ConfigurationException("internode_max_message_size_in_mb must no exceed internode_application_send_queue_reserve_global_capacity_in_bytes", false); + } + else + { + conf.internode_max_message_size_in_bytes = + Math.min(conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes, + conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes); + } + validateMaxConcurrentAutoUpgradeTasksConf(conf.max_concurrent_automatic_sstable_upgrades); } @@ -1448,9 +1473,9 @@ public static int getSSLStoragePort() return Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "ssl_storage_port", Integer.toString(conf.ssl_storage_port))); } - public static long getRpcTimeout() + public static long getRpcTimeout(TimeUnit unit) { - return conf.request_timeout_in_ms; + return unit.convert(conf.request_timeout_in_ms, MILLISECONDS); } public static void setRpcTimeout(long timeOutInMillis) @@ -1458,9 +1483,9 @@ public static void setRpcTimeout(long timeOutInMillis) conf.request_timeout_in_ms = timeOutInMillis; } - public static long getReadRpcTimeout() + public static long getReadRpcTimeout(TimeUnit unit) { - return conf.read_request_timeout_in_ms; + return unit.convert(conf.read_request_timeout_in_ms, MILLISECONDS); } public static void setReadRpcTimeout(long timeOutInMillis) @@ -1468,9 +1493,9 @@ public static void setReadRpcTimeout(long timeOutInMillis) conf.read_request_timeout_in_ms = timeOutInMillis; } - public static long getRangeRpcTimeout() + public static long getRangeRpcTimeout(TimeUnit unit) { - return conf.range_request_timeout_in_ms; + return unit.convert(conf.range_request_timeout_in_ms, MILLISECONDS); } public static void setRangeRpcTimeout(long timeOutInMillis) @@ -1478,9 +1503,9 @@ public static void setRangeRpcTimeout(long timeOutInMillis) conf.range_request_timeout_in_ms = timeOutInMillis; } - public static long getWriteRpcTimeout() + public static long getWriteRpcTimeout(TimeUnit unit) { - return conf.write_request_timeout_in_ms; + return unit.convert(conf.write_request_timeout_in_ms, MILLISECONDS); } public static void setWriteRpcTimeout(long timeOutInMillis) @@ -1488,9 +1513,9 @@ public static void setWriteRpcTimeout(long timeOutInMillis) conf.write_request_timeout_in_ms = timeOutInMillis; } - public static long getCounterWriteRpcTimeout() + public static long getCounterWriteRpcTimeout(TimeUnit unit) { - return conf.counter_write_request_timeout_in_ms; + return unit.convert(conf.counter_write_request_timeout_in_ms, MILLISECONDS); } public static void setCounterWriteRpcTimeout(long timeOutInMillis) @@ -1498,9 +1523,9 @@ public static void setCounterWriteRpcTimeout(long timeOutInMillis) conf.counter_write_request_timeout_in_ms = timeOutInMillis; } - public static long getCasContentionTimeout() + public static long getCasContentionTimeout(TimeUnit unit) { - return conf.cas_contention_timeout_in_ms; + return unit.convert(conf.cas_contention_timeout_in_ms, MILLISECONDS); } public static void setCasContentionTimeout(long timeOutInMillis) @@ -1508,9 +1533,9 @@ public static void setCasContentionTimeout(long timeOutInMillis) conf.cas_contention_timeout_in_ms = timeOutInMillis; } - public static long getTruncateRpcTimeout() + public static long getTruncateRpcTimeout(TimeUnit unit) { - return conf.truncate_request_timeout_in_ms; + return unit.convert(conf.truncate_request_timeout_in_ms, MILLISECONDS); } public static void setTruncateRpcTimeout(long timeOutInMillis) @@ -1523,27 +1548,32 @@ public static boolean hasCrossNodeTimeout() return conf.cross_node_timeout; } - public static long getSlowQueryTimeout() + public static void setCrossNodeTimeout(boolean crossNodeTimeout) + { + conf.cross_node_timeout = crossNodeTimeout; + } + + public static long getSlowQueryTimeout(TimeUnit units) { - return conf.slow_query_log_timeout_in_ms; + return units.convert(conf.slow_query_log_timeout_in_ms, MILLISECONDS); } /** * @return the minimum configured {read, write, range, truncate, misc} timeout */ - public static long getMinRpcTimeout() + public static long getMinRpcTimeout(TimeUnit unit) { - return Longs.min(getRpcTimeout(), - getReadRpcTimeout(), - getRangeRpcTimeout(), - getWriteRpcTimeout(), - getCounterWriteRpcTimeout(), - getTruncateRpcTimeout()); + return Longs.min(getRpcTimeout(unit), + getReadRpcTimeout(unit), + getRangeRpcTimeout(unit), + getWriteRpcTimeout(unit), + getCounterWriteRpcTimeout(unit), + getTruncateRpcTimeout(unit)); } - public static long getPingTimeout() + public static long getPingTimeout(TimeUnit unit) { - return TimeUnit.SECONDS.toMillis(getBlockForPeersTimeoutInSeconds()); + return unit.convert(getBlockForPeersTimeoutInSeconds(), TimeUnit.SECONDS); } public static double getPhiConvictThreshold() @@ -1833,14 +1863,44 @@ public static boolean getRpcKeepAlive() return conf.rpc_keepalive; } - public static int getInternodeSendBufferSize() + public static int getInternodeSocketSendBufferSizeInBytes() { - return conf.internode_send_buff_size_in_bytes; + return conf.internode_socket_send_buffer_size_in_bytes; } - public static int getInternodeRecvBufferSize() + public static int getInternodeSocketReceiveBufferSizeInBytes() { - return conf.internode_recv_buff_size_in_bytes; + return conf.internode_socket_receive_buffer_size_in_bytes; + } + + public static int getInternodeApplicationSendQueueCapacityInBytes() + { + return conf.internode_application_send_queue_capacity_in_bytes; + } + + public static int getInternodeApplicationSendQueueReserveEndpointCapacityInBytes() + { + return conf.internode_application_send_queue_reserve_endpoint_capacity_in_bytes; + } + + public static int getInternodeApplicationSendQueueReserveGlobalCapacityInBytes() + { + return conf.internode_application_send_queue_reserve_global_capacity_in_bytes; + } + + public static int getInternodeApplicationReceiveQueueCapacityInBytes() + { + return conf.internode_application_receive_queue_capacity_in_bytes; + } + + public static int getInternodeApplicationReceiveQueueReserveEndpointCapacityInBytes() + { + return conf.internode_application_receive_queue_reserve_endpoint_capacity_in_bytes; + } + + public static int getInternodeApplicationReceiveQueueReserveGlobalCapacityInBytes() + { + return conf.internode_application_receive_queue_reserve_global_capacity_in_bytes; } public static int getInternodeTcpConnectTimeoutInMS() @@ -1863,6 +1923,17 @@ public static void setInternodeTcpUserTimeoutInMS(int value) conf.internode_tcp_user_timeout_in_ms = value; } + public static int getInternodeMaxMessageSizeInBytes() + { + return conf.internode_max_message_size_in_bytes; + } + + @VisibleForTesting + public static void setInternodeMaxMessageSizeInBytes(int value) + { + conf.internode_max_message_size_in_bytes = value; + } + public static boolean startNativeTransport() { return conf.start_native_transport; @@ -2142,6 +2213,12 @@ public static EncryptionOptions getNativeProtocolEncryptionOptions() return conf.client_encryption_options; } + @VisibleForTesting + public static void updateNativeProtocolEncryptionOptions(Function update) + { + conf.client_encryption_options = update.apply(conf.client_encryption_options); + } + public static int getHintedHandoffThrottleInKB() { return conf.hinted_handoff_throttle_in_kb; @@ -2485,41 +2562,6 @@ public static int getTracetypeQueryTTL() return conf.tracetype_query_ttl; } - public static String getOtcCoalescingStrategy() - { - return conf.otc_coalescing_strategy; - } - - public static void setOtcCoalescingStrategy(String strategy) - { - conf.otc_coalescing_strategy = strategy; - } - - public static int getOtcCoalescingWindow() - { - return conf.otc_coalescing_window_us; - } - - public static int getOtcCoalescingEnoughCoalescedMessages() - { - return conf.otc_coalescing_enough_coalesced_messages; - } - - public static void setOtcCoalescingEnoughCoalescedMessages(int otc_coalescing_enough_coalesced_messages) - { - conf.otc_coalescing_enough_coalesced_messages = otc_coalescing_enough_coalesced_messages; - } - - public static int getOtcBacklogExpirationInterval() - { - return conf.otc_backlog_expiration_interval_ms; - } - - public static void setOtcBacklogExpirationInterval(int intervalInMillis) - { - conf.otc_backlog_expiration_interval_ms = intervalInMillis; - } - public static int getWindowsTimerInterval() { return conf.windows_timer_interval; diff --git a/src/java/org/apache/cassandra/config/EncryptionOptions.java b/src/java/org/apache/cassandra/config/EncryptionOptions.java index 9524cec36ec0..0a33dcc66778 100644 --- a/src/java/org/apache/cassandra/config/EncryptionOptions.java +++ b/src/java/org/apache/cassandra/config/EncryptionOptions.java @@ -17,30 +17,61 @@ */ package org.apache.cassandra.config; -import java.util.Arrays; +import java.util.List; import java.util.Objects; +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.locator.IEndpointSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; + public class EncryptionOptions { - public String keystore = "conf/.keystore"; - public String keystore_password = "cassandra"; - public String truststore = "conf/.truststore"; - public String truststore_password = "cassandra"; - public String[] cipher_suites = {}; - public String protocol = "TLS"; - public String algorithm = null; - public String store_type = "JKS"; - public boolean require_client_auth = false; - public boolean require_endpoint_verification = false; - public boolean enabled = false; - public boolean optional = false; + public final String keystore; + public final String keystore_password; + public final String truststore; + public final String truststore_password; + public final List cipher_suites; + public final String protocol; + public final String algorithm; + public final String store_type; + public final boolean require_client_auth; + public final boolean require_endpoint_verification; + public final boolean enabled; + public final boolean optional; public EncryptionOptions() - { } + { + keystore = "conf/.keystore"; + keystore_password = "cassandra"; + truststore = "conf/.truststore"; + truststore_password = "cassandra"; + cipher_suites = ImmutableList.of(); + protocol = "TLS"; + algorithm = null; + store_type = "JKS"; + require_client_auth = false; + require_endpoint_verification = false; + enabled = false; + optional = false; + } + + public EncryptionOptions(String keystore, String keystore_password, String truststore, String truststore_password, List cipher_suites, String protocol, String algorithm, String store_type, boolean require_client_auth, boolean require_endpoint_verification, boolean enabled, boolean optional) + { + this.keystore = keystore; + this.keystore_password = keystore_password; + this.truststore = truststore; + this.truststore_password = truststore_password; + this.cipher_suites = cipher_suites; + this.protocol = protocol; + this.algorithm = algorithm; + this.store_type = store_type; + this.require_client_auth = require_client_auth; + this.require_endpoint_verification = require_endpoint_verification; + this.enabled = enabled; + this.optional = optional; + } - /** - * Copy constructor - */ public EncryptionOptions(EncryptionOptions options) { keystore = options.keystore; @@ -57,6 +88,97 @@ public EncryptionOptions(EncryptionOptions options) optional = options.optional; } + public EncryptionOptions withKeyStore(String keystore) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withKeyStorePassword(String keystore_password) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withTrustStore(String truststore) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withTrustStorePassword(String truststore_password) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withCipherSuites(List cipher_suites) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withCipherSuites(String ... cipher_suites) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, ImmutableList.copyOf(cipher_suites), + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withProtocol(String protocol) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withAlgorithm(String algorithm) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withStoreType(String store_type) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withRequireClientAuth(boolean require_client_auth) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withEnabled(boolean enabled) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + + public EncryptionOptions withOptional(boolean optional) + { + return new EncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional); + } + /** * The method is being mainly used to cache SslContexts therefore, we only consider * fields that would make a difference when the TrustStore or KeyStore files are updated @@ -81,7 +203,7 @@ public boolean equals(Object o) Objects.equals(protocol, opt.protocol) && Objects.equals(algorithm, opt.algorithm) && Objects.equals(store_type, opt.store_type) && - Arrays.equals(cipher_suites, opt.cipher_suites); + Objects.equals(cipher_suites, opt.cipher_suites); } /** @@ -101,7 +223,7 @@ public int hashCode() result += 31 * (store_type == null ? 0 : store_type.hashCode()); result += 31 * Boolean.hashCode(enabled); result += 31 * Boolean.hashCode(optional); - result += 31 * Arrays.hashCode(cipher_suites); + result += 31 * (cipher_suites == null ? 0 : cipher_suites.hashCode()); result += 31 * Boolean.hashCode(require_client_auth); result += 31 * Boolean.hashCode(require_endpoint_verification); return result; @@ -114,20 +236,156 @@ public enum InternodeEncryption all, none, dc, rack } - public InternodeEncryption internode_encryption = InternodeEncryption.none; - public boolean enable_legacy_ssl_storage_port = false; + public final InternodeEncryption internode_encryption; + public final boolean enable_legacy_ssl_storage_port; public ServerEncryptionOptions() - { } + { + this.internode_encryption = InternodeEncryption.none; + this.enable_legacy_ssl_storage_port = false; + } + public ServerEncryptionOptions(String keystore, String keystore_password, String truststore, String truststore_password, List cipher_suites, String protocol, String algorithm, String store_type, boolean require_client_auth, boolean require_endpoint_verification, boolean enabled, boolean optional, InternodeEncryption internode_encryption, boolean enable_legacy_ssl_storage_port) + { + super(keystore, keystore_password, truststore, truststore_password, cipher_suites, protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, enabled, optional); + this.internode_encryption = internode_encryption; + this.enable_legacy_ssl_storage_port = enable_legacy_ssl_storage_port; + } - /** - * Copy constructor - */ public ServerEncryptionOptions(ServerEncryptionOptions options) { super(options); - internode_encryption = options.internode_encryption; - enable_legacy_ssl_storage_port = options.enable_legacy_ssl_storage_port; + this.internode_encryption = options.internode_encryption; + this.enable_legacy_ssl_storage_port = options.enable_legacy_ssl_storage_port; } + + public boolean shouldEncrypt(InetAddressAndPort endpoint) + { + IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); + switch (internode_encryption) + { + case none: + return false; // if nothing needs to be encrypted then return immediately. + case all: + break; + case dc: + if (snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter())) + return false; + break; + case rack: + // for rack then check if the DC's are the same. + if (snitch.getRack(endpoint).equals(snitch.getLocalRack()) + && snitch.getDatacenter(endpoint).equals(snitch.getLocalDatacenter())) + return false; + break; + } + return true; + } + + + public ServerEncryptionOptions withKeyStore(String keystore) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withKeyStorePassword(String keystore_password) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withTrustStore(String truststore) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withTrustStorePassword(String truststore_password) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withCipherSuites(List cipher_suites) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withCipherSuites(String ... cipher_suites) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, ImmutableList.copyOf(cipher_suites), + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withProtocol(String protocol) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withAlgorithm(String algorithm) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withStoreType(String store_type) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withRequireClientAuth(boolean require_client_auth) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withRequireEndpointVerification(boolean require_endpoint_verification) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withEnabled(boolean enabled) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withOptional(boolean optional) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withInternodeEncryption(InternodeEncryption internode_encryption) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + + public ServerEncryptionOptions withLegacySslStoragePort(boolean enable_legacy_ssl_storage_port) + { + return new ServerEncryptionOptions(keystore, keystore_password, truststore, truststore_password, cipher_suites, + protocol, algorithm, store_type, require_client_auth, require_endpoint_verification, + enabled, optional, internode_encryption, enable_legacy_ssl_storage_port); + } + } } diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index c09b88477f43..5414f23f6687 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -87,6 +87,9 @@ import org.json.simple.JSONArray; import org.json.simple.JSONObject; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.utils.ExecutorUtils.*; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; import static org.apache.cassandra.utils.Throwables.maybeFail; public class ColumnFamilyStore implements ColumnFamilyStoreMBean @@ -217,31 +220,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean private volatile boolean neverPurgeTombstones = false; - public static void shutdownFlushExecutor() throws InterruptedException - { - flushExecutor.shutdown(); - flushExecutor.awaitTermination(60, TimeUnit.SECONDS); - } - - public static void shutdownPostFlushExecutor() throws InterruptedException { postFlushExecutor.shutdown(); postFlushExecutor.awaitTermination(60, TimeUnit.SECONDS); } - public static void shutdownReclaimExecutor() throws InterruptedException - { - reclaimExecutor.shutdown(); - reclaimExecutor.awaitTermination(60, TimeUnit.SECONDS); - } - - public static void shutdownPerDiskFlushExecutors() throws InterruptedException + public static void shutdownExecutorsAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException { - for (ExecutorService executorService : perDiskflushExecutors) - executorService.shutdown(); - for (ExecutorService executorService : perDiskflushExecutors) - executorService.awaitTermination(60, TimeUnit.SECONDS); + List executors = ImmutableList.builder() + .add(perDiskflushExecutors) + .add(reclaimExecutor) + .add(postFlushExecutor) + .add(flushExecutor) + .build(); + shutdown(executors); + awaitTermination(timeout, units, executors); } public void reload() @@ -401,8 +395,8 @@ public ColumnFamilyStore(Keyspace keyspace, viewManager = keyspace.viewManager.forTable(metadata.id); metric = new TableMetrics(this); fileIndexGenerator.set(generation); - sampleReadLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getReadRpcTimeout() / 2); - additionalWriteLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getWriteRpcTimeout() / 2); + sampleReadLatencyNanos = DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS) / 2; + additionalWriteLatencyNanos = DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS) / 2; logger.info("Initializing {}.{}", keyspace.getName(), name); diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java index 50817399ca93..bf9e17492dda 100644 --- a/src/java/org/apache/cassandra/db/Columns.java +++ b/src/java/org/apache/cassandra/db/Columns.java @@ -28,6 +28,7 @@ import com.google.common.hash.Hasher; import net.nicoulaj.compilecommand.annotations.DontInline; +import org.apache.cassandra.exceptions.UnknownColumnException; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.ColumnIdentifier; @@ -454,7 +455,7 @@ public Columns deserialize(DataInputPlus in, TableMetadata metadata) throws IOEx // deserialization. The column will be ignore later on anyway. column = metadata.getDroppedColumn(name); if (column == null) - throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization"); + throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization"); } builder.add(column); } diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java index d04ddd8b909f..bb10a6a7a956 100644 --- a/src/java/org/apache/cassandra/db/CounterMutation.java +++ b/src/java/org/apache/cassandra/db/CounterMutation.java @@ -24,7 +24,6 @@ import com.google.common.base.Function; import com.google.common.base.Objects; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; @@ -39,14 +38,14 @@ import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.*; import org.apache.cassandra.utils.btree.BTreeSet; +import static java.util.concurrent.TimeUnit.*; + public class CounterMutation implements IMutation { public static final CounterMutationSerializer serializer = new CounterMutationSerializer(); @@ -92,11 +91,6 @@ public ConsistencyLevel consistency() return consistency; } - public MessageOut makeMutationMessage() - { - return new MessageOut<>(MessagingService.Verb.COUNTER_MUTATION, this, serializer); - } - /** * Applies the counter mutation, returns the result Mutation (for replication to other nodes). * @@ -146,10 +140,10 @@ private void grabCounterLocks(Keyspace keyspace, List locks) throws WriteT for (Lock lock : LOCKS.bulkGet(getCounterLockKeys())) { - long timeout = TimeUnit.MILLISECONDS.toNanos(getTimeout()) - (System.nanoTime() - startTime); + long timeout = getTimeout(NANOSECONDS) - (System.nanoTime() - startTime); try { - if (!lock.tryLock(timeout, TimeUnit.NANOSECONDS)) + if (!lock.tryLock(timeout, NANOSECONDS)) throw new WriteTimeoutException(WriteType.COUNTER, consistency(), 0, consistency().blockFor(keyspace)); locks.add(lock); } @@ -309,9 +303,9 @@ private void updateForRow(PeekingIterator markIter, } } - public long getTimeout() + public long getTimeout(TimeUnit unit) { - return DatabaseDescriptor.getCounterWriteRpcTimeout(); + return DatabaseDescriptor.getCounterWriteRpcTimeout(unit); } @Override diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java index c946ea595fef..a30ce665beba 100644 --- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java +++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java @@ -22,16 +22,17 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.StorageProxy; -import org.apache.cassandra.utils.FBUtilities; public class CounterMutationVerbHandler implements IVerbHandler { + public static final CounterMutationVerbHandler instance = new CounterMutationVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(CounterMutationVerbHandler.class); - public void doVerb(final MessageIn message, final int id) + public void doVerb(final Message message) { long queryStartNanoTime = System.nanoTime(); final CounterMutation cm = message.payload; @@ -45,12 +46,9 @@ public void doVerb(final MessageIn message, final int id) // will not be called if the request timeout, but this is ok // because the coordinator of the counter mutation will timeout on // it's own in that case. - StorageProxy.applyCounterMutationOnLeader(cm, localDataCenter, new Runnable() - { - public void run() - { - MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from); - } - }, queryStartNanoTime); + StorageProxy.applyCounterMutationOnLeader(cm, + localDataCenter, + () -> MessagingService.instance().send(message.emptyResponse(), message.from()), + queryStartNanoTime); } } diff --git a/src/java/org/apache/cassandra/db/IMutation.java b/src/java/org/apache/cassandra/db/IMutation.java index 9eaf19b4922f..1710cfd12379 100644 --- a/src/java/org/apache/cassandra/db/IMutation.java +++ b/src/java/org/apache/cassandra/db/IMutation.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db; import java.util.Collection; +import java.util.concurrent.TimeUnit; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.schema.TableId; @@ -28,7 +29,7 @@ public interface IMutation public String getKeyspaceName(); public Collection getTableIds(); public DecoratedKey key(); - public long getTimeout(); + public long getTimeout(TimeUnit unit); public String toString(boolean shallow); public Collection getPartitionUpdates(); diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index bc382eef82b8..4c1c2cf5448d 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -56,6 +56,9 @@ import org.apache.cassandra.utils.*; import org.apache.cassandra.utils.concurrent.OpOrder; +import static java.util.concurrent.TimeUnit.*; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + /** * It represents a Keyspace. */ @@ -544,7 +547,7 @@ private CompletableFuture applyInternal(final Mutation mutation, if (lock == null) { //throw WTE only if request is droppable - if (isDroppable && (System.currentTimeMillis() - mutation.createdAt) > DatabaseDescriptor.getWriteRpcTimeout()) + if (isDroppable && (approxTime.isAfter(mutation.approxCreatedAtNanos + DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS)))) { for (int j = 0; j < i; j++) locks[j].unlock(); @@ -605,7 +608,7 @@ else if (isDeferrable) if (isDroppable) { for(TableId tableId : tableIds) - columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, TimeUnit.MILLISECONDS); + columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, MILLISECONDS); } } int nowInSec = FBUtilities.nowInSeconds(); diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java index 6195fe4c87a5..22c4ed83766f 100644 --- a/src/java/org/apache/cassandra/db/Mutation.java +++ b/src/java/org/apache/cassandra/db/Mutation.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.*; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import com.google.common.collect.ImmutableCollection; @@ -32,13 +33,13 @@ import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.ByteBufferUtil; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + public class Mutation implements IMutation { public static final MutationSerializer serializer = new MutationSerializer(); @@ -52,7 +53,7 @@ public class Mutation implements IMutation private final ImmutableMap modifications; // Time at which this mutation or the builder that built it was instantiated - final long createdAt; + final long approxCreatedAtNanos; // keep track of when mutation has started waiting for a MV partition lock final AtomicLong viewLockAcquireStart = new AtomicLong(0); @@ -60,10 +61,10 @@ public class Mutation implements IMutation public Mutation(PartitionUpdate update) { - this(update.metadata().keyspace, update.partitionKey(), ImmutableMap.of(update.metadata().id, update), System.currentTimeMillis()); + this(update.metadata().keyspace, update.partitionKey(), ImmutableMap.of(update.metadata().id, update), approxTime.now()); } - public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap modifications, long createdAt) + public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap modifications, long approxCreatedAtNanos) { this.keyspaceName = keyspaceName; this.key = key; @@ -73,7 +74,7 @@ public Mutation(String keyspaceName, DecoratedKey key, ImmutableMap tableIds) @@ -90,7 +91,7 @@ public Mutation without(Set tableIds) } } - return new Mutation(keyspaceName, key, builder.build(), createdAt); + return new Mutation(keyspaceName, key, builder.build(), approxCreatedAtNanos); } public Mutation without(TableId tableId) @@ -177,7 +178,7 @@ public static Mutation merge(List mutations) modifications.put(table, updates.size() == 1 ? updates.get(0) : PartitionUpdate.merge(updates)); updates.clear(); } - return new Mutation(ks, key, modifications.build(), System.currentTimeMillis()); + return new Mutation(ks, key, modifications.build(), approxTime.now()); } public CompletableFuture applyFuture() @@ -210,19 +211,9 @@ public void applyUnsafe() apply(false); } - public MessageOut createMessage() - { - return createMessage(MessagingService.Verb.MUTATION); - } - - public MessageOut createMessage(MessagingService.Verb verb) - { - return new MessageOut<>(verb, this, serializer); - } - - public long getTimeout() + public long getTimeout(TimeUnit unit) { - return DatabaseDescriptor.getWriteRpcTimeout(); + return DatabaseDescriptor.getWriteRpcTimeout(unit); } public int smallestGCGS() @@ -363,7 +354,7 @@ public Mutation deserialize(DataInputPlus in, int version, SerializationHelper.F update = PartitionUpdate.serializer.deserialize(in, version, flag); modifications.put(update.metadata().id, update); } - return new Mutation(update.metadata().keyspace, dk, modifications.build(), System.currentTimeMillis()); + return new Mutation(update.metadata().keyspace, dk, modifications.build(), approxTime.now()); } public Mutation deserialize(DataInputPlus in, int version) throws IOException @@ -389,7 +380,7 @@ public static class PartitionUpdateCollector private final ImmutableMap.Builder modifications = new ImmutableMap.Builder<>(); private final String keyspaceName; private final DecoratedKey key; - private final long createdAt = System.currentTimeMillis(); + private final long approxCreatedAtNanos = approxTime.now(); private boolean empty = true; public PartitionUpdateCollector(String keyspaceName, DecoratedKey key) @@ -425,7 +416,7 @@ public boolean isEmpty() public Mutation build() { - return new Mutation(keyspaceName, key, modifications.build(), createdAt); + return new Mutation(keyspaceName, key, modifications.build(), approxCreatedAtNanos); } } } diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java index 9660f658dd35..bcb9cc7aaee3 100644 --- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java +++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java @@ -17,8 +17,6 @@ */ package org.apache.cassandra.db; -import java.util.Iterator; - import org.apache.cassandra.exceptions.WriteTimeoutException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.*; @@ -26,10 +24,12 @@ public class MutationVerbHandler implements IVerbHandler { - private void reply(int id, InetAddressAndPort replyTo) + public static final MutationVerbHandler instance = new MutationVerbHandler(); + + private void respond(Message respondTo, InetAddressAndPort respondToAddress) { - Tracing.trace("Enqueuing response to {}", replyTo); - MessagingService.instance().sendReply(WriteResponse.createMessage(), id, replyTo); + Tracing.trace("Enqueuing response to {}", respondToAddress); + MessagingService.instance().send(respondTo.emptyResponse(), respondToAddress); } private void failed() @@ -37,27 +37,25 @@ private void failed() Tracing.trace("Payload application resulted in WriteTimeout, not replying"); } - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { // Check if there were any forwarding headers in this message - InetAddressAndPort from = (InetAddressAndPort)message.parameters.get(ParameterType.FORWARD_FROM); - InetAddressAndPort replyTo; + InetAddressAndPort from = message.respondTo(); + InetAddressAndPort respondToAddress; if (from == null) { - replyTo = message.from; - ForwardToContainer forwardTo = (ForwardToContainer)message.parameters.get(ParameterType.FORWARD_TO); - if (forwardTo != null) - forwardToLocalNodes(message.payload, message.verb, forwardTo, message.from); + respondToAddress = message.from(); + ForwardingInfo forwardTo = message.forwardTo(); + if (forwardTo != null) forwardToLocalNodes(message, forwardTo); } else { - - replyTo = from; + respondToAddress = from; } try { - message.payload.applyFuture().thenAccept(o -> reply(id, replyTo)).exceptionally(wto -> { + message.payload.applyFuture().thenAccept(o -> respond(message, respondToAddress)).exceptionally(wto -> { failed(); return null; }); @@ -68,17 +66,21 @@ public void doVerb(MessageIn message, int id) } } - private static void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, ForwardToContainer forwardTo, InetAddressAndPort from) + private static void forwardToLocalNodes(Message originalMessage, ForwardingInfo forwardTo) { - // tell the recipients who to send their ack to - MessageOut message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(ParameterType.FORWARD_FROM, from); - Iterator iterator = forwardTo.targets.iterator(); - // Send a message to each of the addresses on our Forward List - for (int i = 0; i < forwardTo.targets.size(); i++) + Message.Builder builder = + Message.builder(originalMessage) + .withParam(ParamType.RESPOND_TO, originalMessage.from()) + .withoutParam(ParamType.FORWARD_TO); + + boolean useSameMessageID = forwardTo.useSameMessageID(); + // reuse the same Message if all ids are identical (as they will be for 4.0+ node originated messages) + Message message = useSameMessageID ? builder.build() : null; + + forwardTo.forEach((id, target) -> { - InetAddressAndPort address = iterator.next(); - Tracing.trace("Enqueuing forwarded write to {}", address); - MessagingService.instance().sendOneWay(message, forwardTo.messageIds[i], address); - } + Tracing.trace("Enqueuing forwarded write to {}", target); + MessagingService.instance().send(useSameMessageID ? message : builder.withId(id).build(), target); + }); } } diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java index b5f6fb534214..2145389607a6 100644 --- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java +++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java @@ -18,9 +18,12 @@ package org.apache.cassandra.db; import java.io.IOException; +import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; +import org.apache.cassandra.net.MessageFlag; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.filter.*; @@ -38,8 +41,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.metrics.TableMetrics; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Message; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.StorageProxy; @@ -233,9 +235,9 @@ public PartitionRangeReadCommand withUpdatedLimitsAndDataRange(DataLimits newLim indexMetadata()); } - public long getTimeout() + public long getTimeout(TimeUnit unit) { - return DatabaseDescriptor.getRangeRpcTimeout(); + return DatabaseDescriptor.getRangeRpcTimeout(unit); } public PartitionIterator execute(ConsistencyLevel consistency, ClientState clientState, long queryStartNanoTime) throws RequestExecutionException @@ -345,9 +347,10 @@ public BaseRowIterator applyToPartition(BaseRowIterator iter) return Transformation.apply(iter, new CacheFilter()); } - public MessageOut createMessage() + @Override + public Verb verb() { - return new MessageOut<>(MessagingService.Verb.RANGE_SLICE, this, serializer); + return Verb.RANGE_REQ; } protected void appendCQLWhereClause(StringBuilder sb) @@ -414,6 +417,11 @@ public boolean isLimitedToOnePartition() && dataRange.startKey().equals(dataRange.stopKey()); } + public boolean isRangeRequest() + { + return true; + } + private static class Deserializer extends SelectionDeserializer { public ReadCommand deserialize(DataInputPlus in, diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 32b91ad8905e..68ce2eacbf9b 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; +import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.function.LongPredicate; @@ -35,7 +36,9 @@ import org.apache.cassandra.config.*; import org.apache.cassandra.db.filter.*; -import org.apache.cassandra.db.monitoring.ApproximateTime; +import org.apache.cassandra.net.MessageFlag; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.utils.ApproximateTime; import org.apache.cassandra.db.partitions.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.db.transform.RTBoundCloser; @@ -52,9 +55,8 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.locator.ReplicaCollection; import org.apache.cassandra.metrics.TableMetrics; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; @@ -69,6 +71,7 @@ import static com.google.common.collect.Iterables.any; import static com.google.common.collect.Iterables.filter; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; /** * General interface for storage-engine read commands (common to both range and @@ -164,6 +167,8 @@ protected ReadCommand(Kind kind, public abstract boolean isLimitedToOnePartition(); + public abstract boolean isRangeRequest(); + /** * Creates a new ReadCommand instance with new limits. * @@ -177,7 +182,7 @@ protected ReadCommand(Kind kind, * * @return the configured timeout for this command. */ - public abstract long getTimeout(); + public abstract long getTimeout(TimeUnit unit); /** * Whether this query is a digest one or not. @@ -628,14 +633,15 @@ protected Row applyToRow(Row row) private boolean maybeAbort() { /** - * The value returned by ApproximateTime.currentTimeMillis() is updated only every - * {@link ApproximateTime.CHECK_INTERVAL_MS}, by default 10 millis. Since MonitorableImpl - * relies on ApproximateTime, we don't need to check unless the approximate time has elapsed. + * TODO: this is not a great way to abort early; why not expressly limit checks to 10ms intervals? + * The value returned by approxTime.now() is updated only every + * {@link org.apache.cassandra.utils.MonotonicClock.SampledClock.CHECK_INTERVAL_MS}, by default 2 millis. Since MonitorableImpl + * relies on approxTime, we don't need to check unless the approximate time has elapsed. */ - if (lastChecked == ApproximateTime.currentTimeMillis()) + if (lastChecked == approxTime.now()) return false; - lastChecked = ApproximateTime.currentTimeMillis(); + lastChecked = approxTime.now(); if (isAborted()) { @@ -661,7 +667,14 @@ protected UnfilteredPartitionIterator withStateTracking(UnfilteredPartitionItera /** * Creates a message for this command. */ - public abstract MessageOut createMessage(); + public Message createMessage(boolean trackRepairedData) + { + return trackRepairedData + ? Message.outWithFlags(verb(), this, MessageFlag.CALL_BACK_ON_FAILURE, MessageFlag.TRACK_REPAIRED_DATA) + : Message.outWithFlag (verb(), this, MessageFlag.CALL_BACK_ON_FAILURE); + } + + public abstract Verb verb(); protected abstract void appendCQLWhereClause(StringBuilder sb); diff --git a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java index e39e8a855205..2c28ed9d4b8f 100644 --- a/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java +++ b/src/java/org/apache/cassandra/db/ReadCommandVerbHandler.java @@ -20,29 +20,26 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.InvalidRequestException; -import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tracing.Tracing; +import static java.util.concurrent.TimeUnit.NANOSECONDS; + public class ReadCommandVerbHandler implements IVerbHandler { - private static final Logger logger = LoggerFactory.getLogger(ReadCommandVerbHandler.class); + public static final ReadCommandVerbHandler instance = new ReadCommandVerbHandler(); - protected IVersionedSerializer serializer() - { - return ReadResponse.serializer; - } + private static final Logger logger = LoggerFactory.getLogger(ReadCommandVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { if (StorageService.instance.isBootstrapMode()) { @@ -52,9 +49,10 @@ public void doVerb(MessageIn message, int id) ReadCommand command = message.payload; validateTransientStatus(message); - command.setMonitoringTime(message.constructionTime, message.isCrossNode(), message.getTimeout(), message.getSlowQueryTimeout()); + long timeout = message.expiresAtNanos() - message.createdAtNanos(); + command.setMonitoringTime(message.createdAtNanos(), message.isCrossNode(), timeout, DatabaseDescriptor.getSlowQueryTimeout(NANOSECONDS)); - if (message.parameters.containsKey(ParameterType.TRACK_REPAIRED_DATA)) + if (message.trackRepairedData()) command.trackRepairedStatus(); ReadResponse response; @@ -66,17 +64,17 @@ public void doVerb(MessageIn message, int id) if (!command.complete()) { - Tracing.trace("Discarding partial response to {} (timed out)", message.from); - MessagingService.instance().incrementDroppedMessages(message, message.getLifetimeInMS()); + Tracing.trace("Discarding partial response to {} (timed out)", message.from()); + MessagingService.instance().metrics.recordDroppedMessage(message, message.elapsedSinceCreated(NANOSECONDS), NANOSECONDS); return; } - Tracing.trace("Enqueuing response to {}", message.from); - MessageOut reply = new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, response, serializer()); - MessagingService.instance().sendReply(reply, id, message.from); + Tracing.trace("Enqueuing response to {}", message.from()); + Message reply = message.responseWith(response); + MessagingService.instance().send(reply, message.from()); } - private void validateTransientStatus(MessageIn message) + private void validateTransientStatus(Message message) { ReadCommand command = message.payload; Token token; @@ -93,14 +91,14 @@ private void validateTransientStatus(MessageIn message) if (replica == null) { logger.warn("Received a read request from {} for a range that is not owned by the current replica {}.", - message.from, + message.from(), command); return; } if (!command.acceptsTransient() && replica.isTransient()) { - MessagingService.instance().incrementDroppedMessages(message, message.getLifetimeInMS()); + MessagingService.instance().metrics.recordDroppedMessage(message, message.elapsedSinceCreated(NANOSECONDS), NANOSECONDS); throw new InvalidRequestException(String.format("Attempted to serve %s data request from %s node in %s", command.acceptsTransient() ? "transient" : "full", replica.isTransient() ? "transient" : "full", diff --git a/src/java/org/apache/cassandra/db/ReadExecutionController.java b/src/java/org/apache/cassandra/db/ReadExecutionController.java index 29b6fa7b0484..73ddad8022d8 100644 --- a/src/java/org/apache/cassandra/db/ReadExecutionController.java +++ b/src/java/org/apache/cassandra/db/ReadExecutionController.java @@ -21,9 +21,11 @@ import org.apache.cassandra.index.Index; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.concurrent.OpOrder; +import static org.apache.cassandra.utils.MonotonicClock.preciseTime; + public class ReadExecutionController implements AutoCloseable { private static final long NO_SAMPLING = Long.MIN_VALUE; @@ -36,7 +38,7 @@ public class ReadExecutionController implements AutoCloseable private final ReadExecutionController indexController; private final WriteContext writeContext; private final ReadCommand command; - static Clock clock = Clock.instance; + static MonotonicClock clock = preciseTime; private final long createdAtNanos; // Only used while sampling @@ -93,7 +95,7 @@ static ReadExecutionController forCommand(ReadCommand command) ColumnFamilyStore baseCfs = Keyspace.openAndGetStore(command.metadata()); ColumnFamilyStore indexCfs = maybeGetIndexCfs(baseCfs, command); - long createdAtNanos = baseCfs.metric.topLocalReadQueryTime.isEnabled() ? clock.nanoTime() : NO_SAMPLING; + long createdAtNanos = baseCfs.metric.topLocalReadQueryTime.isEnabled() ? clock.now() : NO_SAMPLING; if (indexCfs == null) return new ReadExecutionController(command, baseCfs.readOrdering.start(), baseCfs.metadata(), null, null, createdAtNanos); @@ -172,7 +174,7 @@ public void close() private void addSample() { String cql = command.toCQLString(); - int timeMicros = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(clock.nanoTime() - createdAtNanos), Integer.MAX_VALUE); + int timeMicros = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(clock.now() - createdAtNanos), Integer.MAX_VALUE); ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(baseMetadata.id); if (cfs != null) cfs.metric.topLocalReadQueryTime.addSample(cql, timeMicros); diff --git a/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java b/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java index 2e499e7935c5..903b3d43bdf3 100644 --- a/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java +++ b/src/java/org/apache/cassandra/db/ReadRepairVerbHandler.java @@ -18,14 +18,16 @@ package org.apache.cassandra.db; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; public class ReadRepairVerbHandler implements IVerbHandler { - public void doVerb(MessageIn message, int id) + public static final ReadRepairVerbHandler instance = new ReadRepairVerbHandler(); + + public void doVerb(Message message) { message.payload.apply(); - MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from); + MessagingService.instance().send(message.emptyResponse(), message.from()); } } diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java index deadf68785b1..2e5211c087e8 100644 --- a/src/java/org/apache/cassandra/db/SerializationHeader.java +++ b/src/java/org/apache/cassandra/db/SerializationHeader.java @@ -27,6 +27,7 @@ import org.apache.cassandra.db.marshal.TypeParser; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.rows.*; +import org.apache.cassandra.exceptions.UnknownColumnException; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.sstable.metadata.IMetadataComponentSerializer; @@ -292,7 +293,7 @@ public MetadataType getType() return MetadataType.HEADER; } - public SerializationHeader toHeader(TableMetadata metadata) + public SerializationHeader toHeader(TableMetadata metadata) throws UnknownColumnException { Map> typeMap = new HashMap<>(staticColumns.size() + regularColumns.size()); @@ -320,7 +321,7 @@ public SerializationHeader toHeader(TableMetadata metadata) // deserialization. The column will be ignore later on anyway. column = metadata.getDroppedColumn(name, isStatic); if (column == null) - throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization"); + throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization"); } builder.add(column); } diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index aec1a54cedb7..8c983aa164f8 100644 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@ -20,10 +20,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; +import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import org.apache.cassandra.cache.IRowCacheEntry; @@ -43,12 +42,11 @@ import org.apache.cassandra.io.sstable.format.SSTableReadsListener; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.locator.ReplicaCollection; import org.apache.cassandra.metrics.TableMetrics; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessageFlag; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.TableMetadata; @@ -364,9 +362,9 @@ public ClusteringIndexFilter clusteringIndexFilter(DecoratedKey key) return clusteringIndexFilter; } - public long getTimeout() + public long getTimeout(TimeUnit unit) { - return DatabaseDescriptor.getReadRpcTimeout(); + return DatabaseDescriptor.getReadRpcTimeout(unit); } @Override @@ -1040,9 +1038,10 @@ public String toString() nowInSec()); } - public MessageOut createMessage() + @Override + public Verb verb() { - return new MessageOut<>(MessagingService.Verb.READ, this, serializer); + return Verb.READ_REQ; } protected void appendCQLWhereClause(StringBuilder sb) @@ -1078,6 +1077,11 @@ public boolean isLimitedToOnePartition() return true; } + public boolean isRangeRequest() + { + return false; + } + /** * Groups multiple single partition read commands. */ diff --git a/src/java/org/apache/cassandra/db/SnapshotCommand.java b/src/java/org/apache/cassandra/db/SnapshotCommand.java index eb6f67a028e7..484db2fd7178 100644 --- a/src/java/org/apache/cassandra/db/SnapshotCommand.java +++ b/src/java/org/apache/cassandra/db/SnapshotCommand.java @@ -22,8 +22,8 @@ import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.Verb; public class SnapshotCommand { @@ -42,11 +42,6 @@ public SnapshotCommand(String keyspace, String columnFamily, String snapshotName this.clear_snapshot = clearSnapshot; } - public MessageOut createMessage() - { - return new MessageOut(MessagingService.Verb.SNAPSHOT, this, serializer); - } - @Override public String toString() { diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java index d48f84fc3f0e..0d79ae95e22d 100644 --- a/src/java/org/apache/cassandra/db/SystemKeyspace.java +++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java @@ -688,16 +688,17 @@ public static synchronized void updateTokens(InetAddressAndPort ep, Collection serializer = new Serializer(); + + public final String keyspace; + public final String table; + + public TruncateRequest(String keyspace, String table) + { + this.keyspace = keyspace; + this.table = table; + } + + @Override + public String toString() + { + return String.format("TruncateRequest(keyspace='%s', table='%s')'", keyspace, table); + } + + private static class Serializer implements IVersionedSerializer + { + public void serialize(TruncateRequest request, DataOutputPlus out, int version) throws IOException + { + out.writeUTF(request.keyspace); + out.writeUTF(request.table); + } + + public TruncateRequest deserialize(DataInputPlus in, int version) throws IOException + { + String keyspace = in.readUTF(); + String table = in.readUTF(); + return new TruncateRequest(keyspace, table); + } + + public long serializedSize(TruncateRequest request, int version) + { + return TypeSizes.sizeof(request.keyspace) + TypeSizes.sizeof(request.table); + } + } +} diff --git a/src/java/org/apache/cassandra/db/TruncateResponse.java b/src/java/org/apache/cassandra/db/TruncateResponse.java index af4ed8f2f929..822c9ccea30f 100644 --- a/src/java/org/apache/cassandra/db/TruncateResponse.java +++ b/src/java/org/apache/cassandra/db/TruncateResponse.java @@ -22,8 +22,6 @@ import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; /** * This message is sent back the truncate operation and basically specifies if @@ -44,11 +42,6 @@ public TruncateResponse(String keyspace, String columnFamily, boolean success) this.success = success; } - public MessageOut createMessage() - { - return new MessageOut(MessagingService.Verb.REQUEST_RESPONSE, this, serializer); - } - public static class TruncateResponseSerializer implements IVersionedSerializer { public void serialize(TruncateResponse tr, DataOutputPlus out, int version) throws IOException diff --git a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java index c2fac6561c87..c605d1f20e78 100644 --- a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java +++ b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java @@ -22,21 +22,23 @@ import org.apache.cassandra.io.FSError; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.tracing.Tracing; -public class TruncateVerbHandler implements IVerbHandler +public class TruncateVerbHandler implements IVerbHandler { + public static final TruncateVerbHandler instance = new TruncateVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(TruncateVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - Truncation t = message.payload; - Tracing.trace("Applying truncation of {}.{}", t.keyspace, t.columnFamily); + TruncateRequest t = message.payload; + Tracing.trace("Applying truncation of {}.{}", t.keyspace, t.table); try { - ColumnFamilyStore cfs = Keyspace.open(t.keyspace).getColumnFamilyStore(t.columnFamily); + ColumnFamilyStore cfs = Keyspace.open(t.keyspace).getColumnFamilyStore(t.table); cfs.truncateBlocking(); } catch (Exception e) @@ -47,16 +49,16 @@ public void doVerb(MessageIn message, int id) if (FSError.findNested(e) != null) throw FSError.findNested(e); } - Tracing.trace("Enqueuing response to truncate operation to {}", message.from); + Tracing.trace("Enqueuing response to truncate operation to {}", message.from()); - TruncateResponse response = new TruncateResponse(t.keyspace, t.columnFamily, true); - logger.trace("{} applied. Enqueuing response to {}@{} ", t, id, message.from ); - MessagingService.instance().sendReply(response.createMessage(), id, message.from); + TruncateResponse response = new TruncateResponse(t.keyspace, t.table, true); + logger.trace("{} applied. Enqueuing response to {}@{} ", t, message.id(), message.from()); + MessagingService.instance().send(message.responseWith(response), message.from()); } - private static void respondError(Truncation t, MessageIn truncateRequestMessage) + private static void respondError(TruncateRequest t, Message truncateRequestMessage) { - TruncateResponse response = new TruncateResponse(t.keyspace, t.columnFamily, false); - MessagingService.instance().sendOneWay(response.createMessage(), truncateRequestMessage.from); + TruncateResponse response = new TruncateResponse(t.keyspace, t.table, false); + MessagingService.instance().send(truncateRequestMessage.responseWith(response), truncateRequestMessage.from()); } } diff --git a/src/java/org/apache/cassandra/db/Truncation.java b/src/java/org/apache/cassandra/db/Truncation.java deleted file mode 100644 index 39a2ec6b98a4..000000000000 --- a/src/java/org/apache/cassandra/db/Truncation.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.db; - -import java.io.IOException; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; - -/** - * A truncate operation descriptor - */ -public class Truncation -{ - public static final IVersionedSerializer serializer = new TruncationSerializer(); - - public final String keyspace; - public final String columnFamily; - - public Truncation(String keyspace, String columnFamily) - { - this.keyspace = keyspace; - this.columnFamily = columnFamily; - } - - public MessageOut createMessage() - { - return new MessageOut(MessagingService.Verb.TRUNCATE, this, serializer); - } - - public String toString() - { - return "Truncation(" + "keyspace='" + keyspace + '\'' + ", cf='" + columnFamily + "\')"; - } -} - -class TruncationSerializer implements IVersionedSerializer -{ - public void serialize(Truncation t, DataOutputPlus out, int version) throws IOException - { - out.writeUTF(t.keyspace); - out.writeUTF(t.columnFamily); - } - - public Truncation deserialize(DataInputPlus in, int version) throws IOException - { - String keyspace = in.readUTF(); - String columnFamily = in.readUTF(); - return new Truncation(keyspace, columnFamily); - } - - public long serializedSize(Truncation truncation, int version) - { - return TypeSizes.sizeof(truncation.keyspace) + TypeSizes.sizeof(truncation.columnFamily); - } -} diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java index b7ab70592c06..45df4eb46733 100644 --- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java +++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java @@ -30,7 +30,7 @@ import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.config.Config; import org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation; -import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.NoSpamLogger; import org.apache.cassandra.utils.concurrent.WaitQueue; @@ -133,21 +133,21 @@ void start() throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %fms", syncIntervalNanos * 1e-6)); shutdown = false; - Runnable runnable = new SyncRunnable(new Clock()); + Runnable runnable = new SyncRunnable(MonotonicClock.preciseTime); thread = NamedThreadFactory.createThread(runnable, name); thread.start(); } class SyncRunnable implements Runnable { - private final Clock clock; + private final MonotonicClock clock; private long firstLagAt = 0; private long totalSyncDuration = 0; // total time spent syncing since firstLagAt private long syncExceededIntervalBy = 0; // time that syncs exceeded pollInterval since firstLagAt private int lagCount = 0; private int syncCount = 0; - SyncRunnable(Clock clock) + SyncRunnable(MonotonicClock clock) { this.clock = clock; } @@ -169,7 +169,7 @@ boolean sync() try { // sync and signal - long pollStarted = clock.nanoTime(); + long pollStarted = clock.now(); boolean flushToDisk = lastSyncedAt + syncIntervalNanos <= pollStarted || shutdownRequested || syncRequested; if (flushToDisk) { @@ -186,7 +186,7 @@ boolean sync() commitLog.sync(false); } - long now = clock.nanoTime(); + long now = clock.now(); if (flushToDisk) maybeLogFlushLag(pollStarted, now); diff --git a/src/java/org/apache/cassandra/db/context/CounterContext.java b/src/java/org/apache/cassandra/db/context/CounterContext.java index 29dc3f087cd4..01c2f1d65dd0 100644 --- a/src/java/org/apache/cassandra/db/context/CounterContext.java +++ b/src/java/org/apache/cassandra/db/context/CounterContext.java @@ -629,7 +629,7 @@ public ByteBuffer markLocalToBeCleared(ByteBuffer context) ByteBuffer marked = ByteBuffer.allocate(context.remaining()); marked.putShort(marked.position(), (short) (count * -1)); - ByteBufferUtil.arrayCopy(context, + ByteBufferUtil.copyBytes(context, context.position() + HEADER_SIZE_LENGTH, marked, marked.position() + HEADER_SIZE_LENGTH, @@ -668,7 +668,7 @@ public ByteBuffer clearAllLocal(ByteBuffer context) cleared.putShort(cleared.position() + HEADER_SIZE_LENGTH + i * HEADER_ELT_LENGTH, globalShardIndexes.get(i)); int origHeaderLength = headerLength(context); - ByteBufferUtil.arrayCopy(context, + ByteBufferUtil.copyBytes(context, context.position() + origHeaderLength, cleared, cleared.position() + headerLength(cleared), diff --git a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java index ddc7b1c2cc21..d0cc5143a5df 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java @@ -26,6 +26,7 @@ import org.apache.cassandra.db.marshal.CollectionType; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.rows.CellPath; +import org.apache.cassandra.exceptions.UnknownColumnException; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; @@ -196,7 +197,7 @@ public ColumnSubselection deserialize(DataInputPlus in, int version, TableMetada // deserialization. The column will be ignore later on anyway. column = metadata.getDroppedColumn(name); if (column == null) - throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization"); + throw new UnknownColumnException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization"); } Kind kind = Kind.values()[in.readUnsignedByte()]; diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java index ac4c69f7a6d2..e3423ff58008 100644 --- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java +++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java @@ -360,7 +360,7 @@ public static ByteBuffer build(boolean isStatic, ByteBuffer... buffers) { ByteBufferUtil.writeShortLength(out, bb.remaining()); int toCopy = bb.remaining(); - ByteBufferUtil.arrayCopy(bb, bb.position(), out, out.position(), toCopy); + ByteBufferUtil.copyBytes(bb, bb.position(), out, out.position(), toCopy); out.position(out.position() + toCopy); out.put((byte) 0); } diff --git a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java b/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java deleted file mode 100644 index cc4b41041ce3..000000000000 --- a/src/java/org/apache/cassandra/db/monitoring/ApproximateTime.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.db.monitoring; - -import java.util.concurrent.TimeUnit; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.concurrent.ScheduledExecutors; -import org.apache.cassandra.config.Config; - -/** - * This is an approximation of System.currentTimeInMillis(). It updates its - * time value at periodic intervals of CHECK_INTERVAL_MS milliseconds - * (currently 10 milliseconds by default). It can be used as a faster alternative - * to System.currentTimeInMillis() every time an imprecision of a few milliseconds - * can be accepted. - */ -public class ApproximateTime -{ - private static final Logger logger = LoggerFactory.getLogger(ApproximateTime.class); - private static final int CHECK_INTERVAL_MS = Math.max(5, Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "approximate_time_precision_ms", "10"))); - - private static volatile long time = System.currentTimeMillis(); - static - { - logger.info("Scheduling approximate time-check task with a precision of {} milliseconds", CHECK_INTERVAL_MS); - ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(() -> time = System.currentTimeMillis(), - CHECK_INTERVAL_MS, - CHECK_INTERVAL_MS, - TimeUnit.MILLISECONDS); - } - - public static long currentTimeMillis() - { - return time; - } - - public static long precision() - { - return 2 * CHECK_INTERVAL_MS; - } - -} diff --git a/src/java/org/apache/cassandra/db/monitoring/Monitorable.java b/src/java/org/apache/cassandra/db/monitoring/Monitorable.java index c9bf94e08f46..10bd10438aa5 100644 --- a/src/java/org/apache/cassandra/db/monitoring/Monitorable.java +++ b/src/java/org/apache/cassandra/db/monitoring/Monitorable.java @@ -21,9 +21,9 @@ public interface Monitorable { String name(); - long constructionTime(); - long timeout(); - long slowTimeout(); + long creationTimeNanos(); + long timeoutNanos(); + long slowTimeoutNanos(); boolean isInProgress(); boolean isAborted(); diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java index 48c815270498..a6e7947b23f1 100644 --- a/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java +++ b/src/java/org/apache/cassandra/db/monitoring/MonitorableImpl.java @@ -18,13 +18,15 @@ package org.apache.cassandra.db.monitoring; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + public abstract class MonitorableImpl implements Monitorable { private MonitoringState state; private boolean isSlow; - private long constructionTime = -1; - private long timeout; - private long slowTimeout; + private long approxCreationTimeNanos = -1; + private long timeoutNanos; + private long slowTimeoutNanos; private boolean isCrossNode; protected MonitorableImpl() @@ -38,23 +40,23 @@ protected MonitorableImpl() * is too complex, it would require passing new parameters to all serializers * or specializing the serializers to accept these message properties. */ - public void setMonitoringTime(long constructionTime, boolean isCrossNode, long timeout, long slowTimeout) + public void setMonitoringTime(long approxCreationTimeNanos, boolean isCrossNode, long timeoutNanos, long slowTimeoutNanos) { - assert constructionTime >= 0; - this.constructionTime = constructionTime; + assert approxCreationTimeNanos >= 0; + this.approxCreationTimeNanos = approxCreationTimeNanos; this.isCrossNode = isCrossNode; - this.timeout = timeout; - this.slowTimeout = slowTimeout; + this.timeoutNanos = timeoutNanos; + this.slowTimeoutNanos = slowTimeoutNanos; } - public long constructionTime() + public long creationTimeNanos() { - return constructionTime; + return approxCreationTimeNanos; } - public long timeout() + public long timeoutNanos() { - return timeout; + return timeoutNanos; } public boolean isCrossNode() @@ -62,9 +64,9 @@ public boolean isCrossNode() return isCrossNode; } - public long slowTimeout() + public long slowTimeoutNanos() { - return slowTimeout; + return slowTimeoutNanos; } public boolean isInProgress() @@ -95,8 +97,8 @@ public boolean abort() { if (state == MonitoringState.IN_PROGRESS) { - if (constructionTime >= 0) - MonitoringTask.addFailedOperation(this, ApproximateTime.currentTimeMillis()); + if (approxCreationTimeNanos >= 0) + MonitoringTask.addFailedOperation(this, approxTime.now()); state = MonitoringState.ABORTED; return true; @@ -109,8 +111,8 @@ public boolean complete() { if (state == MonitoringState.IN_PROGRESS) { - if (isSlow && slowTimeout > 0 && constructionTime >= 0) - MonitoringTask.addSlowOperation(this, ApproximateTime.currentTimeMillis()); + if (isSlow && slowTimeoutNanos > 0 && approxCreationTimeNanos >= 0) + MonitoringTask.addSlowOperation(this, approxTime.now()); state = MonitoringState.COMPLETED; return true; @@ -121,15 +123,15 @@ public boolean complete() private void check() { - if (constructionTime < 0 || state != MonitoringState.IN_PROGRESS) + if (approxCreationTimeNanos < 0 || state != MonitoringState.IN_PROGRESS) return; - long elapsed = ApproximateTime.currentTimeMillis() - constructionTime; + long minElapsedNanos = (approxTime.now() - approxCreationTimeNanos) - approxTime.error(); - if (elapsed >= slowTimeout && !isSlow) + if (minElapsedNanos >= slowTimeoutNanos && !isSlow) isSlow = true; - if (elapsed >= timeout) + if (minElapsedNanos >= timeoutNanos) abort(); } } diff --git a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java index 94260422798e..0f8555f17aa3 100644 --- a/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java +++ b/src/java/org/apache/cassandra/db/monitoring/MonitoringTask.java @@ -39,6 +39,8 @@ import org.apache.cassandra.utils.NoSpamLogger; import static java.lang.System.getProperty; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; /** * A task for monitoring in progress operations, currently only read queries, and aborting them if they time out. @@ -68,7 +70,7 @@ class MonitoringTask private final ScheduledFuture reportingTask; private final OperationsQueue failedOperationsQueue; private final OperationsQueue slowOperationsQueue; - private long lastLogTime; + private long approxLastLogTimeNanos; @VisibleForTesting @@ -88,10 +90,10 @@ private MonitoringTask(int reportIntervalMillis, int maxOperations) this.failedOperationsQueue = new OperationsQueue(maxOperations); this.slowOperationsQueue = new OperationsQueue(maxOperations); - this.lastLogTime = ApproximateTime.currentTimeMillis(); + this.approxLastLogTimeNanos = approxTime.now(); logger.info("Scheduling monitoring task with report interval of {} ms, max operations {}", reportIntervalMillis, maxOperations); - this.reportingTask = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(() -> logOperations(ApproximateTime.currentTimeMillis()), + this.reportingTask = ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(() -> logOperations(approxTime.now()), reportIntervalMillis, reportIntervalMillis, TimeUnit.MILLISECONDS); @@ -102,14 +104,14 @@ public void cancel() reportingTask.cancel(false); } - static void addFailedOperation(Monitorable operation, long now) + static void addFailedOperation(Monitorable operation, long nowNanos) { - instance.failedOperationsQueue.offer(new FailedOperation(operation, now)); + instance.failedOperationsQueue.offer(new FailedOperation(operation, nowNanos)); } - static void addSlowOperation(Monitorable operation, long now) + static void addSlowOperation(Monitorable operation, long nowNanos) { - instance.slowOperationsQueue.offer(new SlowOperation(operation, now)); + instance.slowOperationsQueue.offer(new SlowOperation(operation, nowNanos)); } @VisibleForTesting @@ -131,27 +133,27 @@ private List getLogMessages(AggregatedOperations operations) } @VisibleForTesting - private void logOperations(long now) + private void logOperations(long approxCurrentTimeNanos) { - logSlowOperations(now); - logFailedOperations(now); + logSlowOperations(approxCurrentTimeNanos); + logFailedOperations(approxCurrentTimeNanos); - lastLogTime = now; + approxLastLogTimeNanos = approxCurrentTimeNanos; } @VisibleForTesting - boolean logFailedOperations(long now) + boolean logFailedOperations(long nowNanos) { AggregatedOperations failedOperations = failedOperationsQueue.popOperations(); if (!failedOperations.isEmpty()) { - long elapsed = now - lastLogTime; + long elapsedNanos = nowNanos - approxLastLogTimeNanos; noSpamLogger.warn("Some operations timed out, details available at debug level (debug.log)"); if (logger.isDebugEnabled()) logger.debug("{} operations timed out in the last {} msecs:{}{}", failedOperations.num(), - elapsed, + NANOSECONDS.toMillis(elapsedNanos), LINE_SEPARATOR, failedOperations.getLogMessage()); return true; @@ -161,18 +163,18 @@ boolean logFailedOperations(long now) } @VisibleForTesting - boolean logSlowOperations(long now) + boolean logSlowOperations(long approxCurrentTimeNanos) { AggregatedOperations slowOperations = slowOperationsQueue.popOperations(); if (!slowOperations.isEmpty()) { - long elapsed = now - lastLogTime; + long approxElapsedNanos = approxCurrentTimeNanos - approxLastLogTimeNanos; noSpamLogger.info("Some operations were slow, details available at debug level (debug.log)"); if (logger.isDebugEnabled()) logger.debug("{} operations were slow in the last {} msecs:{}{}", slowOperations.num(), - elapsed, + NANOSECONDS.toMillis(approxElapsedNanos), LINE_SEPARATOR, slowOperations.getLogMessage()); return true; @@ -314,7 +316,7 @@ protected abstract static class Operation int numTimesReported; /** The total time spent by this operation */ - long totalTime; + long totalTimeNanos; /** The maximum time spent by this operation */ long maxTime; @@ -326,13 +328,13 @@ protected abstract static class Operation * this is set lazily as it takes time to build the query CQL */ private String name; - Operation(Monitorable operation, long failedAt) + Operation(Monitorable operation, long failedAtNanos) { this.operation = operation; numTimesReported = 1; - totalTime = failedAt - operation.constructionTime(); - minTime = totalTime; - maxTime = totalTime; + totalTimeNanos = failedAtNanos - operation.creationTimeNanos(); + minTime = totalTimeNanos; + maxTime = totalTimeNanos; } public String name() @@ -345,7 +347,7 @@ public String name() void add(Operation operation) { numTimesReported++; - totalTime += operation.totalTime; + totalTimeNanos += operation.totalTimeNanos; maxTime = Math.max(maxTime, operation.maxTime); minTime = Math.min(minTime, operation.minTime); } @@ -358,9 +360,9 @@ void add(Operation operation) */ private final static class FailedOperation extends Operation { - FailedOperation(Monitorable operation, long failedAt) + FailedOperation(Monitorable operation, long failedAtNanos) { - super(operation, failedAt); + super(operation, failedAtNanos); } public String getLogMessage() @@ -368,17 +370,17 @@ public String getLogMessage() if (numTimesReported == 1) return String.format("<%s>, total time %d msec, timeout %d %s", name(), - totalTime, - operation.timeout(), + NANOSECONDS.toMillis(totalTimeNanos), + NANOSECONDS.toMillis(operation.timeoutNanos()), operation.isCrossNode() ? "msec/cross-node" : "msec"); else return String.format("<%s> timed out %d times, avg/min/max %d/%d/%d msec, timeout %d %s", name(), numTimesReported, - totalTime / numTimesReported, - minTime, - maxTime, - operation.timeout(), + NANOSECONDS.toMillis(totalTimeNanos / numTimesReported), + NANOSECONDS.toMillis(minTime), + NANOSECONDS.toMillis(maxTime), + NANOSECONDS.toMillis(operation.timeoutNanos()), operation.isCrossNode() ? "msec/cross-node" : "msec"); } } @@ -398,17 +400,17 @@ public String getLogMessage() if (numTimesReported == 1) return String.format("<%s>, time %d msec - slow timeout %d %s", name(), - totalTime, - operation.slowTimeout(), + NANOSECONDS.toMillis(totalTimeNanos), + NANOSECONDS.toMillis(operation.slowTimeoutNanos()), operation.isCrossNode() ? "msec/cross-node" : "msec"); else return String.format("<%s>, was slow %d times: avg/min/max %d/%d/%d msec - slow timeout %d %s", name(), numTimesReported, - totalTime / numTimesReported, - minTime, - maxTime, - operation.slowTimeout(), + NANOSECONDS.toMillis(totalTimeNanos/ numTimesReported), + NANOSECONDS.toMillis(minTime), + NANOSECONDS.toMillis(maxTime), + NANOSECONDS.toMillis(operation.slowTimeoutNanos()), operation.isCrossNode() ? "msec/cross-node" : "msec"); } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java index eb993ff0f8d1..c362d1174329 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamReader.java @@ -99,8 +99,9 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException // when compressed, report total bytes of compressed chunks read since remoteFile.size is the sum of chunks transferred session.progress(filename, ProgressInfo.Direction.IN, cis.getTotalCompressedBytesRead(), totalSize); } + assert in.getBytesRead() == sectionLength; } - logger.debug("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum, + logger.trace("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}", session.planId(), fileSeqNum, session.peer, FBUtilities.prettyPrintMemory(cis.getTotalCompressedBytesRead()), FBUtilities.prettyPrintMemory(totalSize)); return writer; } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java index 3b971f885942..efbccdcf25b3 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraCompressedStreamWriter.java @@ -31,8 +31,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.ChannelProxy; import org.apache.cassandra.io.util.DataOutputStreamPlus; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; import org.apache.cassandra.streaming.ProgressInfo; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.utils.FBUtilities; @@ -55,10 +54,9 @@ public CassandraCompressedStreamWriter(SSTableReader sstable, Collection { + ByteBuffer outBuffer = bufferSupplier.get(toTransfer); + long read = fc.read(outBuffer, position); + assert read == toTransfer : String.format("could not read required number of bytes from file to be streamed: read %d bytes, wanted %d bytes", read, toTransfer); outBuffer.flip(); - output.writeToChannel(outBuffer); - } - catch (IOException e) - { - FileUtils.clean(outBuffer); - throw e; - } - - bytesTransferred += lastWrite; - progress += lastWrite; + }, limiter); + + bytesTransferred += toTransfer; + progress += toTransfer; session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize); } } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java index 7a20110d0bf3..401b20ed56ad 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriter.java @@ -27,7 +27,7 @@ import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; import org.apache.cassandra.streaming.ProgressInfo; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamSession; @@ -58,11 +58,11 @@ public CassandraEntireSSTableStreamWriter(SSTableReader sstable, StreamSession s /** * Stream the entire file to given channel. *

- * + * TODO: this currently requires a companion thread, but could be performed entirely asynchronously * @param out where this writes data to * @throws IOException on any I/O error */ - public void write(ByteBufDataOutputStreamPlus out) throws IOException + public void write(AsyncStreamingOutputPlus out) throws IOException { long totalSize = manifest.totalSize(); logger.debug("[Stream #{}] Start streaming sstable {} to {}, repairedAt = {}, totalSize = {}", @@ -76,7 +76,7 @@ public void write(ByteBufDataOutputStreamPlus out) throws IOException for (Component component : manifest.components()) { - @SuppressWarnings("resource") // this is closed after the file is transferred by ByteBufDataOutputStreamPlus + @SuppressWarnings("resource") // this is closed after the file is transferred by AsyncChannelOutputPlus FileChannel in = new RandomAccessFile(sstable.descriptor.filenameFor(component), "r").getChannel(); // Total Length to transmit for this file @@ -90,7 +90,7 @@ public void write(ByteBufDataOutputStreamPlus out) throws IOException component, prettyPrintMemory(length)); - long bytesWritten = out.writeToChannel(in, limiter); + long bytesWritten = out.writeFileToChannel(in, limiter); progress += bytesWritten; session.progress(sstable.descriptor.filenameFor(component), ProgressInfo.Direction.OUT, bytesWritten, length); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java index c688fdf7f3e4..e8f5485844c5 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraOutgoingFile.java @@ -20,7 +20,6 @@ import java.io.File; import java.io.IOException; -import java.util.Collection; import java.util.LinkedHashMap; import java.util.List; import java.util.Objects; @@ -41,7 +40,7 @@ import org.apache.cassandra.io.sstable.KeyIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataOutputStreamPlus; -import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.streaming.OutgoingStream; import org.apache.cassandra.streaming.StreamOperation; @@ -164,10 +163,10 @@ public void write(StreamSession session, DataOutputStreamPlus out, int version) CassandraStreamHeader.serializer.serialize(header, out, version); out.flush(); - if (shouldStreamEntireSSTable() && out instanceof ByteBufDataOutputStreamPlus) + if (shouldStreamEntireSSTable() && out instanceof AsyncStreamingOutputPlus) { CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, manifest); - writer.write((ByteBufDataOutputStreamPlus) out); + writer.write((AsyncStreamingOutputPlus) out); } else { diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java index 43371a95cd90..190f1360bbc2 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReader.java @@ -28,7 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; +import org.apache.cassandra.exceptions.UnknownColumnException; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.TrackedDataInputPlus; import org.apache.cassandra.schema.TableId; @@ -47,10 +47,11 @@ import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.compress.StreamCompressionInputStream; import org.apache.cassandra.streaming.messages.StreamMessageHeader; -import org.apache.cassandra.streaming.messages.StreamMessage; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.net.MessagingService.current_version; + /** * CassandraStreamReader reads from stream and writes to SSTable. */ @@ -114,7 +115,7 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException StreamDeserializer deserializer = null; SSTableMultiWriter writer = null; - try (StreamCompressionInputStream streamCompressionInputStream = new StreamCompressionInputStream(inputPlus, StreamMessage.CURRENT_VERSION)) + try (StreamCompressionInputStream streamCompressionInputStream = new StreamCompressionInputStream(inputPlus, current_version)) { TrackedDataInputPlus in = new TrackedDataInputPlus(streamCompressionInputStream); deserializer = new StreamDeserializer(cfs.metadata(), in, inputVersion, getHeader(cfs.metadata())); @@ -142,7 +143,7 @@ public SSTableMultiWriter read(DataInputPlus inputPlus) throws IOException } } - protected SerializationHeader getHeader(TableMetadata metadata) + protected SerializationHeader getHeader(TableMetadata metadata) throws UnknownColumnException { return header != null? header.toHeader(metadata) : null; //pre-3.0 sstable have no SerializationHeader } diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java index c6dd9a91e6d0..ffc663dd18cb 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamWriter.java @@ -25,19 +25,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.ChannelProxy; import org.apache.cassandra.io.util.DataIntegrityMetadata; import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator; import org.apache.cassandra.io.util.DataOutputStreamPlus; -import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; import org.apache.cassandra.streaming.ProgressInfo; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter; import org.apache.cassandra.streaming.StreamSession; -import org.apache.cassandra.streaming.compress.ByteBufCompressionDataOutputStreamPlus; +import org.apache.cassandra.streaming.async.StreamCompressionSerializer; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.memory.BufferPool; + +import static org.apache.cassandra.net.MessagingService.current_version; /** * CassandraStreamWriter writes given section of the SSTable to given channel. @@ -49,6 +55,7 @@ public class CassandraStreamWriter private static final Logger logger = LoggerFactory.getLogger(CassandraStreamWriter.class); protected final SSTableReader sstable; + private final LZ4Compressor compressor = LZ4Factory.fastestInstance().fastCompressor(); protected final Collection sections; protected final StreamRateLimiter limiter; protected final StreamSession session; @@ -75,6 +82,7 @@ public void write(DataOutputStreamPlus output) throws IOException logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(), sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize); + AsyncStreamingOutputPlus out = (AsyncStreamingOutputPlus) output; try(ChannelProxy proxy = sstable.getDataChannel().sharedCopy(); ChecksumValidator validator = new File(sstable.descriptor.filenameFor(Component.CRC)).exists() ? DataIntegrityMetadata.checksumValidator(sstable.descriptor) @@ -85,38 +93,35 @@ public void write(DataOutputStreamPlus output) throws IOException // setting up data compression stream long progress = 0L; - try (DataOutputStreamPlus compressedOutput = new ByteBufCompressionDataOutputStreamPlus(output, limiter)) + // stream each of the required sections of the file + for (SSTableReader.PartitionPositionBounds section : sections) { - // stream each of the required sections of the file - for (SSTableReader.PartitionPositionBounds section : sections) + long start = validator == null ? section.lowerPosition : validator.chunkStart(section.lowerPosition); + // if the transfer does not start on the valididator's chunk boundary, this is the number of bytes to offset by + int transferOffset = (int) (section.lowerPosition - start); + if (validator != null) + validator.seek(start); + + // length of the section to read + long length = section.upperPosition - start; + // tracks write progress + long bytesRead = 0; + while (bytesRead < length) { - long start = validator == null ? section.lowerPosition : validator.chunkStart(section.lowerPosition); - // if the transfer does not start on the valididator's chunk boundary, this is the number of bytes to offset by - int transferOffset = (int) (section.lowerPosition - start); - if (validator != null) - validator.seek(start); - - // length of the section to read - long length = section.upperPosition - start; - // tracks write progress - long bytesRead = 0; - while (bytesRead < length) - { - int toTransfer = (int) Math.min(bufferSize, length - bytesRead); - long lastBytesRead = write(proxy, validator, compressedOutput, start, transferOffset, toTransfer, bufferSize); - start += lastBytesRead; - bytesRead += lastBytesRead; - progress += (lastBytesRead - transferOffset); - session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize); - transferOffset = 0; - } - - // make sure that current section is sent - output.flush(); + int toTransfer = (int) Math.min(bufferSize, length - bytesRead); + long lastBytesRead = write(proxy, validator, out, start, transferOffset, toTransfer, bufferSize); + start += lastBytesRead; + bytesRead += lastBytesRead; + progress += (lastBytesRead - transferOffset); + session.progress(sstable.descriptor.filenameFor(Component.DATA), ProgressInfo.Direction.OUT, progress, totalSize); + transferOffset = 0; } - logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}", - session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), FBUtilities.prettyPrintMemory(totalSize)); + + // make sure that current section is sent + out.flush(); } + logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}", + session.planId(), sstable.getFilename(), session.peer, FBUtilities.prettyPrintMemory(progress), FBUtilities.prettyPrintMemory(totalSize)); } } @@ -141,14 +146,14 @@ protected long totalSize() * * @throws java.io.IOException on any I/O error */ - protected long write(ChannelProxy proxy, ChecksumValidator validator, DataOutputStreamPlus output, long start, int transferOffset, int toTransfer, int bufferSize) throws IOException + protected long write(ChannelProxy proxy, ChecksumValidator validator, AsyncStreamingOutputPlus output, long start, int transferOffset, int toTransfer, int bufferSize) throws IOException { // the count of bytes to read off disk int minReadable = (int) Math.min(bufferSize, proxy.size() - start); // this buffer will hold the data from disk. as it will be compressed on the fly by - // ByteBufCompressionDataOutputStreamPlus.write(ByteBuffer), we can release this buffer as soon as we can. - ByteBuffer buffer = ByteBuffer.allocateDirect(minReadable); + // AsyncChannelCompressedStreamWriter.write(ByteBuffer), we can release this buffer as soon as we can. + ByteBuffer buffer = BufferPool.get(minReadable, BufferType.OFF_HEAP); try { int readCount = proxy.read(buffer, start); @@ -163,11 +168,11 @@ protected long write(ChannelProxy proxy, ChecksumValidator validator, DataOutput buffer.position(transferOffset); buffer.limit(transferOffset + (toTransfer - transferOffset)); - output.write(buffer); + output.writeToChannel(StreamCompressionSerializer.serialize(compressor, buffer, current_version), limiter); } finally { - FileUtils.clean(buffer); + BufferPool.put(buffer); } return toTransfer; diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java new file mode 100644 index 000000000000..b0afe8f699a9 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/InternodeInboundTable.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.virtual; + +import java.net.InetAddress; +import java.nio.ByteBuffer; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.InboundMessageHandlers; +import org.apache.cassandra.schema.TableMetadata; + +public final class InternodeInboundTable extends AbstractVirtualTable +{ + private static final String ADDRESS = "address"; + private static final String PORT = "port"; + private static final String DC = "dc"; + private static final String RACK = "rack"; + + private static final String USING_BYTES = "using_bytes"; + private static final String USING_RESERVE_BYTES = "using_reserve_bytes"; + private static final String CORRUPT_FRAMES_RECOVERED = "corrupt_frames_recovered"; + private static final String CORRUPT_FRAMES_UNRECOVERED = "corrupt_frames_unrecovered"; + private static final String ERROR_BYTES = "error_bytes"; + private static final String ERROR_COUNT = "error_count"; + private static final String EXPIRED_BYTES = "expired_bytes"; + private static final String EXPIRED_COUNT = "expired_count"; + private static final String SCHEDULED_BYTES = "scheduled_bytes"; + private static final String SCHEDULED_COUNT = "scheduled_count"; + private static final String PROCESSED_BYTES = "processed_bytes"; + private static final String PROCESSED_COUNT = "processed_count"; + private static final String RECEIVED_BYTES = "received_bytes"; + private static final String RECEIVED_COUNT = "received_count"; + private static final String THROTTLED_COUNT = "throttled_count"; + private static final String THROTTLED_NANOS = "throttled_nanos"; + + InternodeInboundTable(String keyspace) + { + super(TableMetadata.builder(keyspace, "internode_inbound") + .kind(TableMetadata.Kind.VIRTUAL) + .partitioner(new LocalPartitioner(CompositeType.getInstance(InetAddressType.instance, Int32Type.instance))) + .addPartitionKeyColumn(ADDRESS, InetAddressType.instance) + .addPartitionKeyColumn(PORT, Int32Type.instance) + .addClusteringColumn(DC, UTF8Type.instance) + .addClusteringColumn(RACK, UTF8Type.instance) + .addRegularColumn(USING_BYTES, LongType.instance) + .addRegularColumn(USING_RESERVE_BYTES, LongType.instance) + .addRegularColumn(CORRUPT_FRAMES_RECOVERED, LongType.instance) + .addRegularColumn(CORRUPT_FRAMES_UNRECOVERED, LongType.instance) + .addRegularColumn(ERROR_BYTES, LongType.instance) + .addRegularColumn(ERROR_COUNT, LongType.instance) + .addRegularColumn(EXPIRED_BYTES, LongType.instance) + .addRegularColumn(EXPIRED_COUNT, LongType.instance) + .addRegularColumn(SCHEDULED_BYTES, LongType.instance) + .addRegularColumn(SCHEDULED_COUNT, LongType.instance) + .addRegularColumn(PROCESSED_BYTES, LongType.instance) + .addRegularColumn(PROCESSED_COUNT, LongType.instance) + .addRegularColumn(RECEIVED_BYTES, LongType.instance) + .addRegularColumn(RECEIVED_COUNT, LongType.instance) + .addRegularColumn(THROTTLED_COUNT, LongType.instance) + .addRegularColumn(THROTTLED_NANOS, LongType.instance) + .build()); + } + + @Override + public DataSet data(DecoratedKey partitionKey) + { + ByteBuffer[] addressAndPortBytes = ((CompositeType) metadata().partitionKeyType).split(partitionKey.getKey()); + InetAddress address = InetAddressType.instance.compose(addressAndPortBytes[0]); + int port = Int32Type.instance.compose(addressAndPortBytes[1]); + InetAddressAndPort addressAndPort = InetAddressAndPort.getByAddressOverrideDefaults(address, port); + + SimpleDataSet result = new SimpleDataSet(metadata()); + InboundMessageHandlers handlers = MessagingService.instance().messageHandlers.get(addressAndPort); + if (null != handlers) + addRow(result, addressAndPort, handlers); + return result; + } + + @Override + public DataSet data() + { + SimpleDataSet result = new SimpleDataSet(metadata()); + MessagingService.instance() + .messageHandlers + .forEach((addressAndPort, handlers) -> addRow(result, addressAndPort, handlers)); + return result; + } + + private void addRow(SimpleDataSet dataSet, InetAddressAndPort addressAndPort, InboundMessageHandlers handlers) + { + String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort); + String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort); + dataSet.row(addressAndPort.address, addressAndPort.port, dc, rack) + .column(USING_BYTES, handlers.usingCapacity()) + .column(USING_RESERVE_BYTES, handlers.usingEndpointReserveCapacity()) + .column(CORRUPT_FRAMES_RECOVERED, handlers.corruptFramesRecovered()) + .column(CORRUPT_FRAMES_UNRECOVERED, handlers.corruptFramesUnrecovered()) + .column(ERROR_BYTES, handlers.errorBytes()) + .column(ERROR_COUNT, handlers.errorCount()) + .column(EXPIRED_BYTES, handlers.expiredBytes()) + .column(EXPIRED_COUNT, handlers.expiredCount()) + .column(SCHEDULED_BYTES, handlers.scheduledBytes()) + .column(SCHEDULED_COUNT, handlers.scheduledCount()) + .column(PROCESSED_BYTES, handlers.processedBytes()) + .column(PROCESSED_COUNT, handlers.processedCount()) + .column(RECEIVED_BYTES, handlers.receivedBytes()) + .column(RECEIVED_COUNT, handlers.receivedCount()) + .column(THROTTLED_COUNT, handlers.throttledCount()) + .column(THROTTLED_NANOS, handlers.throttledNanos()); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java new file mode 100644 index 000000000000..87b38235e586 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/InternodeOutboundTable.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.virtual; + +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.util.function.ToLongFunction; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.OutboundConnection; +import org.apache.cassandra.net.OutboundConnections; +import org.apache.cassandra.schema.TableMetadata; + +public final class InternodeOutboundTable extends AbstractVirtualTable +{ + private static final String ADDRESS = "address"; + private static final String PORT = "port"; + private static final String DC = "dc"; + private static final String RACK = "rack"; + + private static final String USING_BYTES = "using_bytes"; + private static final String USING_RESERVE_BYTES = "using_reserve_bytes"; + private static final String PENDING_COUNT = "pending_count"; + private static final String PENDING_BYTES = "pending_bytes"; + private static final String SENT_COUNT = "sent_count"; + private static final String SENT_BYTES = "sent_bytes"; + private static final String EXPIRED_COUNT = "expired_count"; + private static final String EXPIRED_BYTES = "expired_bytes"; + private static final String ERROR_COUNT = "error_count"; + private static final String ERROR_BYTES = "error_bytes"; + private static final String OVERLOAD_COUNT = "overload_count"; + private static final String OVERLOAD_BYTES = "overload_bytes"; + private static final String ACTIVE_CONNECTION_COUNT = "active_connections"; + private static final String CONNECTION_ATTEMPTS = "connection_attempts"; + private static final String SUCCESSFUL_CONNECTION_ATTEMPTS = "successful_connection_attempts"; + + InternodeOutboundTable(String keyspace) + { + super(TableMetadata.builder(keyspace, "internode_outbound") + .kind(TableMetadata.Kind.VIRTUAL) + .partitioner(new LocalPartitioner(CompositeType.getInstance(InetAddressType.instance, Int32Type.instance))) + .addPartitionKeyColumn(ADDRESS, InetAddressType.instance) + .addPartitionKeyColumn(PORT, Int32Type.instance) + .addClusteringColumn(DC, UTF8Type.instance) + .addClusteringColumn(RACK, UTF8Type.instance) + .addRegularColumn(USING_BYTES, LongType.instance) + .addRegularColumn(USING_RESERVE_BYTES, LongType.instance) + .addRegularColumn(PENDING_COUNT, LongType.instance) + .addRegularColumn(PENDING_BYTES, LongType.instance) + .addRegularColumn(SENT_COUNT, LongType.instance) + .addRegularColumn(SENT_BYTES, LongType.instance) + .addRegularColumn(EXPIRED_COUNT, LongType.instance) + .addRegularColumn(EXPIRED_BYTES, LongType.instance) + .addRegularColumn(ERROR_COUNT, LongType.instance) + .addRegularColumn(ERROR_BYTES, LongType.instance) + .addRegularColumn(OVERLOAD_COUNT, LongType.instance) + .addRegularColumn(OVERLOAD_BYTES, LongType.instance) + .addRegularColumn(ACTIVE_CONNECTION_COUNT, LongType.instance) + .addRegularColumn(CONNECTION_ATTEMPTS, LongType.instance) + .addRegularColumn(SUCCESSFUL_CONNECTION_ATTEMPTS, LongType.instance) + .build()); + } + + @Override + public DataSet data(DecoratedKey partitionKey) + { + ByteBuffer[] addressAndPortBytes = ((CompositeType) metadata().partitionKeyType).split(partitionKey.getKey()); + InetAddress address = InetAddressType.instance.compose(addressAndPortBytes[0]); + int port = Int32Type.instance.compose(addressAndPortBytes[1]); + InetAddressAndPort addressAndPort = InetAddressAndPort.getByAddressOverrideDefaults(address, port); + + SimpleDataSet result = new SimpleDataSet(metadata()); + OutboundConnections connections = MessagingService.instance().channelManagers.get(addressAndPort); + if (null != connections) + addRow(result, addressAndPort, connections); + return result; + } + + @Override + public DataSet data() + { + SimpleDataSet result = new SimpleDataSet(metadata()); + MessagingService.instance() + .channelManagers + .forEach((addressAndPort, connections) -> addRow(result, addressAndPort, connections)); + return result; + } + + private void addRow(SimpleDataSet dataSet, InetAddressAndPort addressAndPort, OutboundConnections connections) + { + String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(addressAndPort); + String rack = DatabaseDescriptor.getEndpointSnitch().getRack(addressAndPort); + long pendingBytes = sum(connections, OutboundConnection::pendingBytes); + dataSet.row(addressAndPort.address, addressAndPort.port, dc, rack) + .column(USING_BYTES, pendingBytes) + .column(USING_RESERVE_BYTES, connections.usingReserveBytes()) + .column(PENDING_COUNT, sum(connections, OutboundConnection::pendingCount)) + .column(PENDING_BYTES, pendingBytes) + .column(SENT_COUNT, sum(connections, OutboundConnection::sentCount)) + .column(SENT_BYTES, sum(connections, OutboundConnection::sentBytes)) + .column(EXPIRED_COUNT, sum(connections, OutboundConnection::expiredCount)) + .column(EXPIRED_BYTES, sum(connections, OutboundConnection::expiredBytes)) + .column(ERROR_COUNT, sum(connections, OutboundConnection::errorCount)) + .column(ERROR_BYTES, sum(connections, OutboundConnection::errorBytes)) + .column(OVERLOAD_COUNT, sum(connections, OutboundConnection::overloadedCount)) + .column(OVERLOAD_BYTES, sum(connections, OutboundConnection::overloadedBytes)) + .column(ACTIVE_CONNECTION_COUNT, sum(connections, c -> c.isConnected() ? 1 : 0)) + .column(CONNECTION_ATTEMPTS, sum(connections, OutboundConnection::connectionAttempts)) + .column(SUCCESSFUL_CONNECTION_ATTEMPTS, sum(connections, OutboundConnection::successfulConnections)); + } + + private static long sum(OutboundConnections connections, ToLongFunction f) + { + return f.applyAsLong(connections.small) + f.applyAsLong(connections.large) + f.applyAsLong(connections.urgent); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/SettingsTable.java b/src/java/org/apache/cassandra/db/virtual/SettingsTable.java index 34debc6b09ee..048d4ba35da3 100644 --- a/src/java/org/apache/cassandra/db/virtual/SettingsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/SettingsTable.java @@ -163,7 +163,7 @@ private void addEncryptionOptions(SimpleDataSet result, Field f) result.row(f.getName() + "_enabled").column(VALUE, Boolean.toString(value.enabled)); result.row(f.getName() + "_algorithm").column(VALUE, value.algorithm); result.row(f.getName() + "_protocol").column(VALUE, value.protocol); - result.row(f.getName() + "_cipher_suites").column(VALUE, Arrays.toString(value.cipher_suites)); + result.row(f.getName() + "_cipher_suites").column(VALUE, value.cipher_suites.toString()); result.row(f.getName() + "_client_auth").column(VALUE, Boolean.toString(value.require_client_auth)); result.row(f.getName() + "_endpoint_verification").column(VALUE, Boolean.toString(value.require_endpoint_verification)); result.row(f.getName() + "_optional").column(VALUE, Boolean.toString(value.optional)); diff --git a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java index f85991acc159..fa1ddae36632 100644 --- a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java @@ -31,6 +31,8 @@ private SystemViewsKeyspace() new ClientsTable(NAME), new SettingsTable(NAME), new SSTableTasksTable(NAME), - new ThreadPoolsTable(NAME))); + new ThreadPoolsTable(NAME), + new InternodeOutboundTable(NAME), + new InternodeInboundTable(NAME))); } } diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java index dc32c8cca9ce..6db0acdf2613 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.virtual; import java.util.Collection; +import java.util.concurrent.TimeUnit; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; @@ -76,9 +77,9 @@ public DecoratedKey key() } @Override - public long getTimeout() + public long getTimeout(TimeUnit unit) { - return DatabaseDescriptor.getWriteRpcTimeout(); + return DatabaseDescriptor.getWriteRpcTimeout(unit); } @Override diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java index cef605eb87b7..f5c455ccc14a 100644 --- a/src/java/org/apache/cassandra/dht/BootStrapper.java +++ b/src/java/org/apache/cassandra/dht/BootStrapper.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.dht; -import java.io.IOException; import java.util.*; import java.util.concurrent.atomic.AtomicInteger; @@ -28,14 +27,9 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.dht.tokenallocator.TokenAllocation; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.TokenMetadata; @@ -234,24 +228,4 @@ public static Collection getRandomTokens(TokenMetadata metadata, int numT logger.info("Generated random tokens. tokens are {}", tokens); return tokens; } - - public static class StringSerializer implements IVersionedSerializer - { - public static final StringSerializer instance = new StringSerializer(); - - public void serialize(String s, DataOutputPlus out, int version) throws IOException - { - out.writeUTF(s); - } - - public String deserialize(DataInputPlus in, int version) throws IOException - { - return in.readUTF(); - } - - public long serializedSize(String s, int version) - { - return TypeSizes.sizeof(s); - } - } } diff --git a/src/java/org/apache/cassandra/dht/IPartitioner.java b/src/java/org/apache/cassandra/dht/IPartitioner.java index 5475f3c9c473..f433f209e73d 100644 --- a/src/java/org/apache/cassandra/dht/IPartitioner.java +++ b/src/java/org/apache/cassandra/dht/IPartitioner.java @@ -18,6 +18,7 @@ package org.apache.cassandra.dht; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; @@ -25,9 +26,29 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.service.StorageService; public interface IPartitioner { + static IPartitioner global() + { + return StorageService.instance.getTokenMetadata().partitioner; + } + + static void validate(Collection> allBounds) + { + for (AbstractBounds bounds : allBounds) + validate(bounds); + } + + static void validate(AbstractBounds bounds) + { + if (global() != bounds.left.getPartitioner()) + throw new AssertionError(String.format("Partitioner in bounds serialization. Expected %s, was %s.", + global().getClass().getName(), + bounds.left.getPartitioner().getClass().getName())); + } + /** * Transform key to object representation of the on-disk format. * diff --git a/src/java/org/apache/cassandra/net/async/ExpiredException.java b/src/java/org/apache/cassandra/exceptions/IncompatibleSchemaException.java similarity index 71% rename from src/java/org/apache/cassandra/net/async/ExpiredException.java rename to src/java/org/apache/cassandra/exceptions/IncompatibleSchemaException.java index 191900c4b4f3..fe3a167b6f72 100644 --- a/src/java/org/apache/cassandra/net/async/ExpiredException.java +++ b/src/java/org/apache/cassandra/exceptions/IncompatibleSchemaException.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.cassandra.exceptions; -package org.apache.cassandra.net.async; +import java.io.IOException; -/** - * Thrown when a {@link QueuedMessage} has timed out (has sat in the netty outbound channel for too long). - */ -class ExpiredException extends Exception +public class IncompatibleSchemaException extends IOException { - @SuppressWarnings("ThrowableInstanceNeverThrown") - static final ExpiredException INSTANCE = new ExpiredException(); + public IncompatibleSchemaException(String msg) + { + super(msg); + } } diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureException.java b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java index 2b57a7517421..e982b44f02e7 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailureException.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java @@ -19,6 +19,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.stream.Collectors; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.locator.InetAddressAndPort; @@ -32,7 +33,10 @@ public class RequestFailureException extends RequestExecutionException protected RequestFailureException(ExceptionCode code, ConsistencyLevel consistency, int received, int blockFor, Map failureReasonByEndpoint) { - super(code, String.format("Operation failed - received %d responses and %d failures", received, failureReasonByEndpoint.size())); + super(code, String.format("Operation failed - received %d responses and %d failures: %s", + received, + failureReasonByEndpoint.size(), + buildFailureString(failureReasonByEndpoint))); this.consistency = consistency; this.received = received; this.blockFor = blockFor; @@ -45,4 +49,11 @@ protected RequestFailureException(ExceptionCode code, ConsistencyLevel consisten // we encode this map for transport. this.failureReasonByEndpoint = new HashMap<>(failureReasonByEndpoint); } + + private static String buildFailureString(Map failures) + { + return failures.entrySet().stream() + .map(e -> String.format("%s from %s", e.getValue(), e.getKey())) + .collect(Collectors.joining(", ")); + } } diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java index 96ab7b5d0cfc..1cdbdb544d28 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java @@ -15,37 +15,101 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.exceptions; +import java.io.IOException; + +import com.google.common.primitives.Ints; + +import org.apache.cassandra.db.filter.TombstoneOverwhelmingException; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.utils.vint.VIntCoding; + +import static java.lang.Math.max; +import static org.apache.cassandra.net.MessagingService.VERSION_40; + public enum RequestFailureReason { - /** - * The reason for the failure was none of the below reasons or was not recorded by the data node. - */ - UNKNOWN (0x0000), + UNKNOWN (0), + READ_TOO_MANY_TOMBSTONES (1), + TIMEOUT (2), + INCOMPATIBLE_SCHEMA (3); - /** - * The data node read too many tombstones when attempting to execute a read query (see tombstone_failure_threshold). - */ - READ_TOO_MANY_TOMBSTONES (0x0001); + public static final Serializer serializer = new Serializer(); - /** The code to be serialized as an unsigned 16 bit integer */ public final int code; - public static final RequestFailureReason[] VALUES = values(); - RequestFailureReason(final int code) + RequestFailureReason(int code) { this.code = code; } - public static RequestFailureReason fromCode(final int code) + private static final RequestFailureReason[] codeToReasonMap; + + static { - for (RequestFailureReason reasonCode : VALUES) + RequestFailureReason[] reasons = values(); + + int max = -1; + for (RequestFailureReason r : reasons) + max = max(r.code, max); + + RequestFailureReason[] codeMap = new RequestFailureReason[max + 1]; + + for (RequestFailureReason reason : reasons) + { + if (codeMap[reason.code] != null) + throw new RuntimeException("Two RequestFailureReason-s that map to the same code: " + reason.code); + codeMap[reason.code] = reason; + } + + codeToReasonMap = codeMap; + } + + public static RequestFailureReason fromCode(int code) + { + if (code < 0) + throw new IllegalArgumentException("RequestFailureReason code must be non-negative (got " + code + ')'); + + // be forgiving and return UNKNOWN if we aren't aware of the code - for forward compatibility + return code < codeToReasonMap.length ? codeToReasonMap[code] : UNKNOWN; + } + + public static RequestFailureReason forException(Throwable t) + { + if (t instanceof TombstoneOverwhelmingException) + return READ_TOO_MANY_TOMBSTONES; + + if (t instanceof IncompatibleSchemaException) + return INCOMPATIBLE_SCHEMA; + + return UNKNOWN; + } + + public static final class Serializer implements IVersionedSerializer + { + private Serializer() + { + } + + public void serialize(RequestFailureReason reason, DataOutputPlus out, int version) throws IOException + { + if (version < VERSION_40) + out.writeShort(reason.code); + else + out.writeUnsignedVInt(reason.code); + } + + public RequestFailureReason deserialize(DataInputPlus in, int version) throws IOException + { + return fromCode(version < VERSION_40 ? in.readUnsignedShort() : Ints.checkedCast(in.readUnsignedVInt())); + } + + public long serializedSize(RequestFailureReason reason, int version) { - if (reasonCode.code == code) - return reasonCode; + return version < VERSION_40 ? 2 : VIntCoding.computeVIntSize(reason.code); } - throw new IllegalArgumentException("Unknown request failure reason error code: " + code); } } diff --git a/src/java/org/apache/cassandra/exceptions/UnknownColumnException.java b/src/java/org/apache/cassandra/exceptions/UnknownColumnException.java new file mode 100644 index 000000000000..93a464e77e02 --- /dev/null +++ b/src/java/org/apache/cassandra/exceptions/UnknownColumnException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.exceptions; + +public final class UnknownColumnException extends IncompatibleSchemaException +{ + public UnknownColumnException(String msg) + { + super(msg); + } +} diff --git a/src/java/org/apache/cassandra/exceptions/UnknownTableException.java b/src/java/org/apache/cassandra/exceptions/UnknownTableException.java index 2cd7aab21121..3e9c77537061 100644 --- a/src/java/org/apache/cassandra/exceptions/UnknownTableException.java +++ b/src/java/org/apache/cassandra/exceptions/UnknownTableException.java @@ -17,11 +17,9 @@ */ package org.apache.cassandra.exceptions; -import java.io.IOException; - import org.apache.cassandra.schema.TableId; -public class UnknownTableException extends IOException +public class UnknownTableException extends IncompatibleSchemaException { public final TableId id; diff --git a/src/java/org/apache/cassandra/gms/EchoMessage.java b/src/java/org/apache/cassandra/gms/EchoMessage.java deleted file mode 100644 index 2fee889f6b72..000000000000 --- a/src/java/org/apache/cassandra/gms/EchoMessage.java +++ /dev/null @@ -1,56 +0,0 @@ -package org.apache.cassandra.gms; -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - * - */ - - -import java.io.IOException; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; - -public final class EchoMessage -{ - public static final EchoMessage instance = new EchoMessage(); - - public static final IVersionedSerializer serializer = new EchoMessageSerializer(); - - private EchoMessage() - { - } - - public static class EchoMessageSerializer implements IVersionedSerializer - { - public void serialize(EchoMessage t, DataOutputPlus out, int version) throws IOException - { - } - - public EchoMessage deserialize(DataInputPlus in, int version) throws IOException - { - return EchoMessage.instance; - } - - public long serializedSize(EchoMessage t, int version) - { - return 0; - } - } -} diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java index 4a16f2a9e8c2..d3a5f340080b 100644 --- a/src/java/org/apache/cassandra/gms/FailureDetector.java +++ b/src/java/org/apache/cassandra/gms/FailureDetector.java @@ -37,10 +37,11 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; +import static org.apache.cassandra.utils.MonotonicClock.preciseTime; + /** * This FailureDetector is an implementation of the paper titled * "The Phi Accrual Failure Detector" by Hayashibara. @@ -55,7 +56,7 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean private static final int DEBUG_PERCENTAGE = 80; // if the phi is larger than this percentage of the max, log a debug message private static final long DEFAULT_MAX_PAUSE = 5000L * 1000000L; // 5 seconds private static final long MAX_LOCAL_PAUSE_IN_NANOS = getMaxLocalPause(); - private long lastInterpret = Clock.instance.nanoTime(); + private long lastInterpret = preciseTime.now(); private long lastPause = 0L; private static long getMaxLocalPause() @@ -283,7 +284,7 @@ public boolean isAlive(InetAddressAndPort ep) public void report(InetAddressAndPort ep) { - long now = Clock.instance.nanoTime(); + long now = preciseTime.now(); ArrivalWindow heartbeatWindow = arrivalSamples.get(ep); if (heartbeatWindow == null) { @@ -310,7 +311,7 @@ public void interpret(InetAddressAndPort ep) { return; } - long now = Clock.instance.nanoTime(); + long now = preciseTime.now(); long diff = now - lastInterpret; lastInterpret = now; if (diff > MAX_LOCAL_PAUSE_IN_NANOS) @@ -319,7 +320,7 @@ public void interpret(InetAddressAndPort ep) lastPause = now; return; } - if (Clock.instance.nanoTime() - lastPause < MAX_LOCAL_PAUSE_IN_NANOS) + if (preciseTime.now() - lastPause < MAX_LOCAL_PAUSE_IN_NANOS) { logger.debug("Still not marking nodes down due to local pause"); return; diff --git a/src/java/org/apache/cassandra/gms/GossipDigest.java b/src/java/org/apache/cassandra/gms/GossipDigest.java index c7e60c4c3443..53f6c5c52c59 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigest.java +++ b/src/java/org/apache/cassandra/gms/GossipDigest.java @@ -24,7 +24,8 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; /** * Contains information about a specified list of Endpoints and the largest version @@ -83,14 +84,14 @@ class GossipDigestSerializer implements IVersionedSerializer { public void serialize(GossipDigest gDigest, DataOutputPlus out, int version) throws IOException { - CompactEndpointSerializationHelper.instance.serialize(gDigest.endpoint, out, version); + inetAddressAndPortSerializer.serialize(gDigest.endpoint, out, version); out.writeInt(gDigest.generation); out.writeInt(gDigest.maxVersion); } public GossipDigest deserialize(DataInputPlus in, int version) throws IOException { - InetAddressAndPort endpoint = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort endpoint = inetAddressAndPortSerializer.deserialize(in, version); int generation = in.readInt(); int maxVersion = in.readInt(); return new GossipDigest(endpoint, generation, maxVersion); @@ -98,7 +99,7 @@ public GossipDigest deserialize(DataInputPlus in, int version) throws IOExceptio public long serializedSize(GossipDigest gDigest, int version) { - long size = CompactEndpointSerializationHelper.instance.serializedSize(gDigest.endpoint, version); + long size = inetAddressAndPortSerializer.serializedSize(gDigest.endpoint, version); size += TypeSizes.sizeof(gDigest.generation); size += TypeSizes.sizeof(gDigest.maxVersion); return size; diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck.java b/src/java/org/apache/cassandra/gms/GossipDigestAck.java index a7d5b92b303f..26494eaba9d4 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigestAck.java +++ b/src/java/org/apache/cassandra/gms/GossipDigestAck.java @@ -27,7 +27,8 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; /** * This ack gets sent out as a result of the receipt of a GossipDigestSynMessage by an @@ -66,7 +67,7 @@ public void serialize(GossipDigestAck gDigestAckMessage, DataOutputPlus out, int for (Map.Entry entry : gDigestAckMessage.epStateMap.entrySet()) { InetAddressAndPort ep = entry.getKey(); - CompactEndpointSerializationHelper.instance.serialize(ep, out, version); + inetAddressAndPortSerializer.serialize(ep, out, version); EndpointState.serializer.serialize(entry.getValue(), out, version); } } @@ -79,7 +80,7 @@ public GossipDigestAck deserialize(DataInputPlus in, int version) throws IOExcep for (int i = 0; i < size; ++i) { - InetAddressAndPort ep = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort ep = inetAddressAndPortSerializer.deserialize(in, version); EndpointState epState = EndpointState.serializer.deserialize(in, version); epStateMap.put(ep, epState); } @@ -91,7 +92,7 @@ public long serializedSize(GossipDigestAck ack, int version) int size = GossipDigestSerializationHelper.serializedSize(ack.gDigestList, version); size += TypeSizes.sizeof(ack.epStateMap.size()); for (Map.Entry entry : ack.epStateMap.entrySet()) - size += CompactEndpointSerializationHelper.instance.serializedSize(entry.getKey(), version) + size += inetAddressAndPortSerializer.serializedSize(entry.getKey(), version) + EndpointState.serializer.serializedSize(entry.getValue(), version); return size; } diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java index a6d1d2b196d3..0e4062bb0f44 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigestAck2.java +++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2.java @@ -26,7 +26,8 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; /** * This ack gets sent out as a result of the receipt of a GossipDigestAckMessage. This the @@ -57,7 +58,7 @@ public void serialize(GossipDigestAck2 ack2, DataOutputPlus out, int version) th for (Map.Entry entry : ack2.epStateMap.entrySet()) { InetAddressAndPort ep = entry.getKey(); - CompactEndpointSerializationHelper.instance.serialize(ep, out, version); + inetAddressAndPortSerializer.serialize(ep, out, version); EndpointState.serializer.serialize(entry.getValue(), out, version); } } @@ -69,7 +70,7 @@ public GossipDigestAck2 deserialize(DataInputPlus in, int version) throws IOExce for (int i = 0; i < size; ++i) { - InetAddressAndPort ep = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort ep = inetAddressAndPortSerializer.deserialize(in, version); EndpointState epState = EndpointState.serializer.deserialize(in, version); epStateMap.put(ep, epState); } @@ -80,7 +81,7 @@ public long serializedSize(GossipDigestAck2 ack2, int version) { long size = TypeSizes.sizeof(ack2.epStateMap.size()); for (Map.Entry entry : ack2.epStateMap.entrySet()) - size += CompactEndpointSerializationHelper.instance.serializedSize(entry.getKey(), version) + size += inetAddressAndPortSerializer.serializedSize(entry.getKey(), version) + EndpointState.serializer.serializedSize(entry.getValue(), version); return size; } diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java index fd5d4876b45f..58c1589eca91 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java +++ b/src/java/org/apache/cassandra/gms/GossipDigestAck2VerbHandler.java @@ -23,18 +23,19 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; -public class GossipDigestAck2VerbHandler implements IVerbHandler +public class GossipDigestAck2VerbHandler extends GossipVerbHandler { + public static final GossipDigestAck2VerbHandler instance = new GossipDigestAck2VerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(GossipDigestAck2VerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { if (logger.isTraceEnabled()) { - InetAddressAndPort from = message.from; + InetAddressAndPort from = message.from(); logger.trace("Received a GossipDigestAck2Message from {}", from); } if (!Gossiper.instance.isEnabled()) @@ -47,5 +48,7 @@ public void doVerb(MessageIn message, int id) /* Notify the Failure Detector */ Gossiper.instance.notifyFailureDetector(remoteEpStateMap); Gossiper.instance.applyStateLocally(remoteEpStateMap); + + super.doVerb(message); } } diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java index 2a12b7c3995a..1e8604b66606 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java +++ b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java @@ -25,18 +25,20 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -public class GossipDigestAckVerbHandler implements IVerbHandler +import static org.apache.cassandra.net.Verb.GOSSIP_DIGEST_ACK2; + +public class GossipDigestAckVerbHandler extends GossipVerbHandler { + public static final GossipDigestAckVerbHandler instance = new GossipDigestAckVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(GossipDigestAckVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - InetAddressAndPort from = message.from; + InetAddressAndPort from = message.from(); if (logger.isTraceEnabled()) logger.trace("Received a GossipDigestAckMessage from {}", from); if (!Gossiper.instance.isEnabled() && !Gossiper.instance.isInShadowRound()) @@ -88,11 +90,11 @@ public void doVerb(MessageIn message, int id) deltaEpStateMap.put(addr, localEpStatePtr); } - MessageOut gDigestAck2Message = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK2, - new GossipDigestAck2(deltaEpStateMap), - GossipDigestAck2.serializer); + Message gDigestAck2Message = Message.out(GOSSIP_DIGEST_ACK2, new GossipDigestAck2(deltaEpStateMap)); if (logger.isTraceEnabled()) logger.trace("Sending a GossipDigestAck2Message to {}", from); - MessagingService.instance().sendOneWay(gDigestAck2Message, from); + MessagingService.instance().send(gDigestAck2Message, from); + + super.doVerb(message); } } diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java index b06c24dcdf54..520dbec3f606 100644 --- a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java +++ b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java @@ -24,18 +24,20 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -public class GossipDigestSynVerbHandler implements IVerbHandler +import static org.apache.cassandra.net.Verb.*; + +public class GossipDigestSynVerbHandler extends GossipVerbHandler { + public static final GossipDigestSynVerbHandler instance = new GossipDigestSynVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(GossipDigestSynVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - InetAddressAndPort from = message.from; + InetAddressAndPort from = message.from(); if (logger.isTraceEnabled()) logger.trace("Received a GossipDigestSynMessage from {}", from); if (!Gossiper.instance.isEnabled() && !Gossiper.instance.isInShadowRound()) @@ -79,10 +81,8 @@ public void doVerb(MessageIn message, int id) logger.debug("Received a shadow round syn from {}. Gossip is disabled but " + "currently also in shadow round, responding with a minimal ack", from); MessagingService.instance() - .sendOneWay(new MessageOut<>(MessagingService.Verb.GOSSIP_DIGEST_ACK, - new GossipDigestAck(new ArrayList<>(), new HashMap<>()), - GossipDigestAck.serializer), - from); + .send(Message.out(GOSSIP_DIGEST_ACK, new GossipDigestAck(Collections.emptyList(), Collections.emptyMap())), + from); return; } @@ -101,11 +101,11 @@ public void doVerb(MessageIn message, int id) Map deltaEpStateMap = new HashMap(); Gossiper.instance.examineGossiper(gDigestList, deltaGossipDigestList, deltaEpStateMap); logger.trace("sending {} digests and {} deltas", deltaGossipDigestList.size(), deltaEpStateMap.size()); - MessageOut gDigestAckMessage = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK, - new GossipDigestAck(deltaGossipDigestList, deltaEpStateMap), - GossipDigestAck.serializer); + Message gDigestAckMessage = Message.out(GOSSIP_DIGEST_ACK, new GossipDigestAck(deltaGossipDigestList, deltaEpStateMap)); if (logger.isTraceEnabled()) logger.trace("Sending a GossipDigestAckMessage to {}", from); - MessagingService.instance().sendOneWay(gDigestAckMessage, from); + MessagingService.instance().send(gDigestAckMessage, from); + + super.doVerb(message); } } diff --git a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java index 169110733c50..83c8568274f9 100644 --- a/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java +++ b/src/java/org/apache/cassandra/gms/GossipShutdownVerbHandler.java @@ -18,23 +18,25 @@ package org.apache.cassandra.gms; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class GossipShutdownVerbHandler implements IVerbHandler { + public static final GossipShutdownVerbHandler instance = new GossipShutdownVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(GossipShutdownVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { if (!Gossiper.instance.isEnabled()) { - logger.debug("Ignoring shutdown message from {} because gossip is disabled", message.from); + logger.debug("Ignoring shutdown message from {} because gossip is disabled", message.from()); return; } - Gossiper.instance.markAsShutdown(message.from); + Gossiper.instance.markAsShutdown(message.from()); } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/gms/GossipVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipVerbHandler.java new file mode 100644 index 000000000000..02aeaf4467c8 --- /dev/null +++ b/src/java/org/apache/cassandra/gms/GossipVerbHandler.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.gms; + +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.Message; + +public class GossipVerbHandler implements IVerbHandler +{ + public void doVerb(Message message) + { + Gossiper.instance.setLastProcessedMessageAt(message.creationTimeMillis()); + } +} diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 8955bf950673..a6c9be7858f1 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -39,6 +39,8 @@ import com.google.common.util.concurrent.Uninterruptibles; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.utils.CassandraVersion; import io.netty.util.concurrent.FastThreadLocal; import org.apache.cassandra.utils.MBeanWrapper; @@ -53,14 +55,17 @@ import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.net.IAsyncCallback; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; +import static org.apache.cassandra.net.NoPayload.noPayload; +import static org.apache.cassandra.net.Verb.ECHO_REQ; +import static org.apache.cassandra.net.Verb.GOSSIP_DIGEST_SYN; + /** * This module is responsible for Gossiping information for the local endpoint. This abstraction * maintains the list of live and dead endpoints. Periodically i.e. every 1 second this module @@ -236,9 +241,7 @@ public void run() GossipDigestSyn digestSynMessage = new GossipDigestSyn(DatabaseDescriptor.getClusterName(), DatabaseDescriptor.getPartitionerName(), gDigests); - MessageOut message = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_SYN, - digestSynMessage, - GossipDigestSyn.serializer); + Message message = Message.out(GOSSIP_DIGEST_SYN, digestSynMessage); /* Gossip to some random live member */ boolean gossipedToSeed = doGossipToLiveMember(message); @@ -545,11 +548,11 @@ public void removeEndpoint(InetAddressAndPort endpoint) liveEndpoints.remove(endpoint); unreachableEndpoints.remove(endpoint); - MessagingService.instance().resetVersion(endpoint); + MessagingService.instance().versions.reset(endpoint); quarantineEndpoint(endpoint); - MessagingService.instance().destroyConnectionPool(endpoint); - if (logger.isDebugEnabled()) - logger.debug("removing endpoint {}", endpoint); + MessagingService.instance().closeOutbound(endpoint); + MessagingService.instance().removeInbound(endpoint); + logger.debug("removing endpoint {}", endpoint); GossiperDiagnostics.removedEndpoint(this, endpoint); } @@ -777,7 +780,7 @@ public int getCurrentGenerationNumber(InetAddressAndPort endpoint) * @param epSet a set of endpoint from which a random endpoint is chosen. * @return true if the chosen endpoint is also a seed. */ - private boolean sendGossip(MessageOut message, Set epSet) + private boolean sendGossip(Message message, Set epSet) { List liveEndpoints = ImmutableList.copyOf(epSet); @@ -791,7 +794,7 @@ private boolean sendGossip(MessageOut message, Set message, Set message) + private boolean doGossipToLiveMember(Message message) { int size = liveEndpoints.size(); if (size == 0) @@ -808,7 +811,7 @@ private boolean doGossipToLiveMember(MessageOut message) } /* Sends a Gossip message to an unreachable member */ - private void maybeGossipToUnreachableMember(MessageOut message) + private void maybeGossipToUnreachableMember(Message message) { double liveEndpointCount = liveEndpoints.size(); double unreachableEndpointCount = unreachableEndpoints.size(); @@ -823,7 +826,7 @@ private void maybeGossipToUnreachableMember(MessageOut message) } /* Possibly gossip to a seed for facilitating partition healing */ - private void maybeGossipToSeed(MessageOut prod) + private void maybeGossipToSeed(Message prod) { int size = seeds.size(); if (size > 0) @@ -1145,23 +1148,15 @@ private void markAlive(final InetAddressAndPort addr, final EndpointState localS { localState.markDead(); - MessageOut echoMessage = new MessageOut(MessagingService.Verb.ECHO, EchoMessage.instance, EchoMessage.serializer); - logger.trace("Sending a EchoMessage to {}", addr); - IAsyncCallback echoHandler = new IAsyncCallback() + Message echoMessage = Message.out(ECHO_REQ, noPayload); + logger.trace("Sending ECHO_REQ to {}", addr); + RequestCallback echoHandler = msg -> { - public boolean isLatencyForSnitch() - { - return false; - } - - public void response(MessageIn msg) - { - // force processing of the echo response onto the gossip stage, as it comes in on the REQUEST_RESPONSE stage - runInGossipStageBlocking(() -> realMarkAlive(addr, localState)); - } + // force processing of the echo response onto the gossip stage, as it comes in on the REQUEST_RESPONSE stage + runInGossipStageBlocking(() -> realMarkAlive(addr, localState)); }; - MessagingService.instance().sendRR(echoMessage, addr, echoHandler); + MessagingService.instance().sendWithCallback(echoMessage, addr, echoHandler); GossiperDiagnostics.markedAlive(this, addr, localState); } @@ -1447,7 +1442,7 @@ void examineGossiper(List gDigestList, List deltaGos if (gDigestList.size() == 0) { /* we've been sent a *completely* empty syn, which should normally never happen since an endpoint will at least send a syn with itself. - If this is happening then the node is attempting shadow gossip, and we should reply with everything we know. + If this is happening then the node is attempting shadow gossip, and we should respond with everything we know. */ logger.debug("Shadow request received, adding all states"); for (Map.Entry entry : endpointStateMap.entrySet()) @@ -1582,9 +1577,7 @@ public synchronized Map doShadowRound(Set message = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_SYN, - digestSynMessage, - GossipDigestSyn.serializer); + Message message = Message.out(GOSSIP_DIGEST_SYN, digestSynMessage); inShadowRound = true; boolean includePeers = false; @@ -1598,14 +1591,14 @@ public synchronized Map doShadowRound(Set serializer = new Serializer(); - - private final UUID hostId; - private final ByteBuffer hint; - private final int version; - - EncodedHintMessage(UUID hostId, ByteBuffer hint, int version) - { - this.hostId = hostId; - this.hint = hint; - this.version = version; - } - - MessageOut createMessageOut() - { - return new MessageOut<>(MessagingService.Verb.HINT, this, serializer); - } - - public long getHintCreationTime() - { - return Hint.serializer.getHintCreationTime(hint, version); - } - - private static class Serializer implements IVersionedSerializer - { - public long serializedSize(EncodedHintMessage message, int version) - { - if (version != message.version) - throw new IllegalArgumentException("serializedSize() called with non-matching version " + version); - - long size = UUIDSerializer.serializer.serializedSize(message.hostId, version); - size += TypeSizes.sizeofUnsignedVInt(message.hint.remaining()); - size += message.hint.remaining(); - return size; - } - - public void serialize(EncodedHintMessage message, DataOutputPlus out, int version) throws IOException - { - if (version != message.version) - throw new IllegalArgumentException("serialize() called with non-matching version " + version); - - UUIDSerializer.serializer.serialize(message.hostId, out, version); - out.writeUnsignedVInt(message.hint.remaining()); - out.write(message.hint); - } - - public EncodedHintMessage deserialize(DataInputPlus in, int version) throws IOException - { - throw new UnsupportedOperationException(); - } - } -} diff --git a/src/java/org/apache/cassandra/hints/HintMessage.java b/src/java/org/apache/cassandra/hints/HintMessage.java index 683b894cb6a7..333af842dc09 100644 --- a/src/java/org/apache/cassandra/hints/HintMessage.java +++ b/src/java/org/apache/cassandra/hints/HintMessage.java @@ -19,6 +19,7 @@ package org.apache.cassandra.hints; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Objects; import java.util.UUID; @@ -28,11 +29,9 @@ import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.exceptions.UnknownTableException; -import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.IVersionedAsymmetricSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.io.util.TrackedDataInputPlus; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.utils.UUIDSerializer; @@ -49,9 +48,9 @@ * Scenario (2) means that we got a hint from a node that's going through decommissioning and is streaming its hints * elsewhere first. */ -public final class HintMessage +public final class HintMessage implements SerializableHintMessage { - public static final IVersionedSerializer serializer = new Serializer(); + public static final IVersionedAsymmetricSerializer serializer = new Serializer(); final UUID hostId; @@ -75,37 +74,72 @@ public final class HintMessage this.unknownTableID = unknownTableID; } - public MessageOut createMessageOut() + public static class Serializer implements IVersionedAsymmetricSerializer { - return new MessageOut<>(MessagingService.Verb.HINT, this, serializer); - } - - public static class Serializer implements IVersionedSerializer - { - public long serializedSize(HintMessage message, int version) + public long serializedSize(SerializableHintMessage obj, int version) { - long size = UUIDSerializer.serializer.serializedSize(message.hostId, version); + if (obj instanceof HintMessage) + { + HintMessage message = (HintMessage) obj; + long size = UUIDSerializer.serializer.serializedSize(message.hostId, version); + + long hintSize = Hint.serializer.serializedSize(message.hint, version); + size += TypeSizes.sizeofUnsignedVInt(hintSize); + size += hintSize; + + return size; + } + else if (obj instanceof Encoded) + { + Encoded message = (Encoded) obj; - long hintSize = Hint.serializer.serializedSize(message.hint, version); - size += TypeSizes.sizeofUnsignedVInt(hintSize); - size += hintSize; + if (version != message.version) + throw new IllegalArgumentException("serializedSize() called with non-matching version " + version); - return size; + long size = UUIDSerializer.serializer.serializedSize(message.hostId, version); + size += TypeSizes.sizeofUnsignedVInt(message.hint.remaining()); + size += message.hint.remaining(); + return size; + } + else + { + throw new IllegalStateException("Unexpected type: " + obj); + } } - public void serialize(HintMessage message, DataOutputPlus out, int version) throws IOException + public void serialize(SerializableHintMessage obj, DataOutputPlus out, int version) throws IOException { - Objects.requireNonNull(message.hint); // we should never *send* a HintMessage with null hint + if (obj instanceof HintMessage) + { + HintMessage message = (HintMessage) obj; - UUIDSerializer.serializer.serialize(message.hostId, out, version); + Objects.requireNonNull(message.hint); // we should never *send* a HintMessage with null hint - /* - * We are serializing the hint size so that the receiver of the message could gracefully handle - * deserialize failure when a table had been dropped, by simply skipping the unread bytes. - */ - out.writeUnsignedVInt(Hint.serializer.serializedSize(message.hint, version)); + UUIDSerializer.serializer.serialize(message.hostId, out, version); - Hint.serializer.serialize(message.hint, out, version); + /* + * We are serializing the hint size so that the receiver of the message could gracefully handle + * deserialize failure when a table had been dropped, by simply skipping the unread bytes. + */ + out.writeUnsignedVInt(Hint.serializer.serializedSize(message.hint, version)); + + Hint.serializer.serialize(message.hint, out, version); + } + else if (obj instanceof Encoded) + { + Encoded message = (Encoded) obj; + + if (version != message.version) + throw new IllegalArgumentException("serialize() called with non-matching version " + version); + + UUIDSerializer.serializer.serialize(message.hostId, out, version); + out.writeUnsignedVInt(message.hint.remaining()); + out.write(message.hint); + } + else + { + throw new IllegalStateException("Unexpected type: " + obj); + } } /* @@ -130,4 +164,32 @@ public HintMessage deserialize(DataInputPlus in, int version) throws IOException } } } + + /** + * A specialized version of {@link HintMessage} that takes an already encoded in a bytebuffer hint and sends it verbatim. + * + * An optimization for when dispatching a hint file of the current messaging version to a node of the same messaging version, + * which is the most common case. Saves on extra ByteBuffer allocations one redundant hint deserialization-serialization cycle. + * + * Never deserialized as an HintMessage.Encoded - the receiving side will always deserialize the message as vanilla + * {@link HintMessage}. + */ + static final class Encoded implements SerializableHintMessage + { + private final UUID hostId; + private final ByteBuffer hint; + private final int version; + + Encoded(UUID hostId, ByteBuffer hint, int version) + { + this.hostId = hostId; + this.hint = hint; + this.version = version; + } + + public long getHintCreationTime() + { + return Hint.serializer.getHintCreationTime(hint, version); + } + } } diff --git a/src/java/org/apache/cassandra/hints/HintResponse.java b/src/java/org/apache/cassandra/hints/HintResponse.java deleted file mode 100644 index 8aa888f55154..000000000000 --- a/src/java/org/apache/cassandra/hints/HintResponse.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.cassandra.hints; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; - -/** - * An empty successful response to a HintMessage. - */ -public final class HintResponse -{ - public static final IVersionedSerializer serializer = new Serializer(); - - static final HintResponse instance = new HintResponse(); - static final MessageOut message = - new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, instance, serializer); - - private HintResponse() - { - } - - private static final class Serializer implements IVersionedSerializer - { - public long serializedSize(HintResponse response, int version) - { - return 0; - } - - public void serialize(HintResponse response, DataOutputPlus out, int version) - { - } - - public HintResponse deserialize(DataInputPlus in, int version) - { - return instance; - } - } -} diff --git a/src/java/org/apache/cassandra/hints/HintVerbHandler.java b/src/java/org/apache/cassandra/hints/HintVerbHandler.java index cec6f0b2b448..2fbe4754b62a 100644 --- a/src/java/org/apache/cassandra/hints/HintVerbHandler.java +++ b/src/java/org/apache/cassandra/hints/HintVerbHandler.java @@ -26,7 +26,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.service.StorageProxy; @@ -41,9 +41,11 @@ */ public final class HintVerbHandler implements IVerbHandler { + public static final HintVerbHandler instance = new HintVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(HintVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { UUID hostId = message.payload.hostId; Hint hint = message.payload.hint; @@ -59,7 +61,7 @@ public void doVerb(MessageIn message, int id) address, hostId, message.payload.unknownTableID); - reply(id, message.from); + respond(message); return; } @@ -71,7 +73,7 @@ public void doVerb(MessageIn message, int id) catch (MarshalException e) { logger.warn("Failed to validate a hint for {}: {} - skipped", address, hostId); - reply(id, message.from); + respond(message); return; } @@ -80,24 +82,24 @@ public void doVerb(MessageIn message, int id) // the node is not the final destination of the hint (must have gotten it from a decommissioning node), // so just store it locally, to be delivered later. HintsService.instance.write(hostId, hint); - reply(id, message.from); + respond(message); } else if (!StorageProxy.instance.appliesLocally(hint.mutation)) { // the topology has changed, and we are no longer a replica of the mutation - since we don't know which node(s) // it has been handed over to, re-address the hint to all replicas; see CASSANDRA-5902. HintsService.instance.writeForAllReplicas(hint); - reply(id, message.from); + respond(message); } else { // the common path - the node is both the destination and a valid replica for the hint. - hint.applyFuture().thenAccept(o -> reply(id, message.from)).exceptionally(e -> {logger.debug("Failed to apply hint", e); return null;}); + hint.applyFuture().thenAccept(o -> respond(message)).exceptionally(e -> {logger.debug("Failed to apply hint", e); return null;}); } } - private static void reply(int id, InetAddressAndPort to) + private static void respond(Message respondTo) { - MessagingService.instance().sendReply(HintResponse.message, id, to); + MessagingService.instance().send(respondTo.emptyResponse(), respondTo.from()); } } diff --git a/src/java/org/apache/cassandra/hints/HintsDispatcher.java b/src/java/org/apache/cassandra/hints/HintsDispatcher.java index 2cff18608c0d..39e4b25c0b22 100644 --- a/src/java/org/apache/cassandra/hints/HintsDispatcher.java +++ b/src/java/org/apache/cassandra/hints/HintsDispatcher.java @@ -20,7 +20,6 @@ import java.io.File; import java.nio.ByteBuffer; import java.util.*; -import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; import java.util.function.Function; @@ -28,19 +27,21 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.monitoring.ApproximateTime; +import org.apache.cassandra.net.RequestCallback; import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.metrics.HintsServiceMetrics; -import org.apache.cassandra.net.IAsyncCallbackWithFailure; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.utils.concurrent.SimpleCondition; +import static org.apache.cassandra.net.Verb.HINT_REQ; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + /** * Dispatches a single hints file to a specified node in a batched manner. * - * Uses either {@link EncodedHintMessage} - when dispatching hints into a node with the same messaging version as the hints file, + * Uses either {@link HintMessage.Encoded} - when dispatching hints into a node with the same messaging version as the hints file, * or {@link HintMessage}, when conversion is required. */ final class HintsDispatcher implements AutoCloseable @@ -70,7 +71,7 @@ private HintsDispatcher(HintsReader reader, UUID hostId, InetAddressAndPort addr static HintsDispatcher create(File file, RateLimiter rateLimiter, InetAddressAndPort address, UUID hostId, BooleanSupplier abortRequested) { - int messagingVersion = MessagingService.instance().getVersion(address); + int messagingVersion = MessagingService.instance().versions.get(address); HintsDispatcher dispatcher = new HintsDispatcher(HintsReader.open(file, rateLimiter), hostId, address, messagingVersion, abortRequested); HintDiagnostics.dispatcherCreated(dispatcher); return dispatcher; @@ -187,8 +188,8 @@ private Action sendHints(Iterator hints, Collection callbacks, private Callback sendHint(Hint hint) { Callback callback = new Callback(hint.creationTime); - HintMessage message = new HintMessage(hostId, hint); - MessagingService.instance().sendRRWithFailure(message.createMessageOut(), address, callback); + Message message = Message.out(HINT_REQ, new HintMessage(hostId, hint)); + MessagingService.instance().sendWithCallback(message, address, callback); return callback; } @@ -198,34 +199,32 @@ private Callback sendHint(Hint hint) private Callback sendEncodedHint(ByteBuffer hint) { - EncodedHintMessage message = new EncodedHintMessage(hostId, hint, messagingVersion); + HintMessage.Encoded message = new HintMessage.Encoded(hostId, hint, messagingVersion); Callback callback = new Callback(message.getHintCreationTime()); - MessagingService.instance().sendRRWithFailure(message.createMessageOut(), address, callback); + MessagingService.instance().sendWithCallback(Message.out(HINT_REQ, message), address, callback); return callback; } - private static final class Callback implements IAsyncCallbackWithFailure + private static final class Callback implements RequestCallback { enum Outcome { SUCCESS, TIMEOUT, FAILURE, INTERRUPTED } - private final long start = System.nanoTime(); + private final long start = approxTime.now(); private final SimpleCondition condition = new SimpleCondition(); private volatile Outcome outcome; - private final long hintCreationTime; + private final long hintCreationNanoTime; - private Callback(long hintCreationTime) + private Callback(long hintCreationTimeMillisSinceEpoch) { - this.hintCreationTime = hintCreationTime; + this.hintCreationNanoTime = approxTime.translate().fromMillisSinceEpoch(hintCreationTimeMillisSinceEpoch); } Outcome await() { - long timeout = TimeUnit.MILLISECONDS.toNanos(MessagingService.Verb.HINT.getTimeout()) - (System.nanoTime() - start); boolean timedOut; - try { - timedOut = !condition.await(timeout, TimeUnit.NANOSECONDS); + timedOut = !condition.awaitUntil(HINT_REQ.expiresAtNanos(start)); } catch (InterruptedException e) { @@ -236,24 +235,27 @@ Outcome await() return timedOut ? Outcome.TIMEOUT : outcome; } + @Override + public boolean invokeOnFailure() + { + return true; + } + + @Override public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) { outcome = Outcome.FAILURE; condition.signalAll(); } - public void response(MessageIn msg) + @Override + public void onResponse(Message msg) { - HintsServiceMetrics.updateDelayMetrics(msg.from, ApproximateTime.currentTimeMillis() - this.hintCreationTime); + HintsServiceMetrics.updateDelayMetrics(msg.from(), approxTime.now() - this.hintCreationNanoTime); outcome = Outcome.SUCCESS; condition.signalAll(); } - public boolean isLatencyForSnitch() - { - return false; - } - @Override public boolean supportsBackPressure() { diff --git a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java b/src/java/org/apache/cassandra/hints/SerializableHintMessage.java similarity index 84% rename from src/java/org/apache/cassandra/locator/ILatencySubscriber.java rename to src/java/org/apache/cassandra/hints/SerializableHintMessage.java index f6c1c7f20f88..43c289c7518e 100644 --- a/src/java/org/apache/cassandra/locator/ILatencySubscriber.java +++ b/src/java/org/apache/cassandra/hints/SerializableHintMessage.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.locator; -public interface ILatencySubscriber +package org.apache.cassandra.hints; + +public interface SerializableHintMessage { - public void receiveTiming(InetAddressAndPort address, long latency); } diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java index ec54a652a5f2..b37251af76bc 100644 --- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java +++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java @@ -71,11 +71,15 @@ import org.apache.cassandra.service.pager.SinglePartitionPager; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.concurrent.Refs; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; +import static org.apache.cassandra.utils.ExecutorUtils.shutdown; + /** * Handles the core maintenance functionality associated with indexes: adding/removing them to or from * a table, (re)building during bootstrap or other streaming operations, flushing, reloading metadata @@ -1487,12 +1491,9 @@ public void handleNotification(INotification notification, Object sender) } @VisibleForTesting - public static void shutdownExecutors() throws InterruptedException + public static void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException { - ExecutorService[] executors = new ExecutorService[]{ asyncExecutor, blockingExecutor }; - for (ExecutorService executor : executors) - executor.shutdown(); - for (ExecutorService executor : executors) - executor.awaitTermination(60, TimeUnit.SECONDS); + shutdown(asyncExecutor, blockingExecutor); + awaitTermination(timeout, units, asyncExecutor, blockingExecutor); } } diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java index 19c09cc51624..07327ea6d6ca 100644 --- a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java +++ b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java @@ -19,6 +19,7 @@ import java.util.*; import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import com.googlecode.concurrenttrees.common.Iterables; @@ -60,6 +61,8 @@ import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.concurrent.OpOrder; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + public class SASIIndex implements Index, INotificationConsumer { public final static String USAGE_WARNING = "SASI indexes are experimental and are not recommended for production use."; @@ -295,7 +298,7 @@ public Searcher searcherFor(ReadCommand command) throws InvalidRequestException { TableMetadata config = command.metadata(); ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(config.id); - return controller -> new QueryPlan(cfs, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller); + return controller -> new QueryPlan(cfs, command, DatabaseDescriptor.getRangeRpcTimeout(MILLISECONDS)).execute(controller); } public SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationType opType) diff --git a/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java b/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java deleted file mode 100644 index d82ff7d7da07..000000000000 --- a/src/java/org/apache/cassandra/io/DummyByteVersionedSerializer.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.io; - -import java.io.IOException; - -import com.google.common.base.Preconditions; - -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessagingService; - -/** - * Serializes a dummy byte that can't be set. Will always write 0 and return 0 in a correctly formed message. - */ -public class DummyByteVersionedSerializer implements IVersionedSerializer -{ - public static final DummyByteVersionedSerializer instance = new DummyByteVersionedSerializer(); - - private DummyByteVersionedSerializer() {} - - public void serialize(byte[] bytes, DataOutputPlus out, int version) throws IOException - { - Preconditions.checkArgument(bytes == MessagingService.ONE_BYTE); - out.write(0); - } - - public byte[] deserialize(DataInputPlus in, int version) throws IOException - { - assert(0 == in.readByte()); - return MessagingService.ONE_BYTE; - } - - public long serializedSize(byte[] bytes, int version) - { - //Payload - return 1; - } -} diff --git a/src/java/org/apache/cassandra/io/IVersionedAsymmetricSerializer.java b/src/java/org/apache/cassandra/io/IVersionedAsymmetricSerializer.java new file mode 100644 index 000000000000..8ad2c285c326 --- /dev/null +++ b/src/java/org/apache/cassandra/io/IVersionedAsymmetricSerializer.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.io; + +import java.io.IOException; + +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +public interface IVersionedAsymmetricSerializer +{ + /** + * Serialize the specified type into the specified DataOutputStream instance. + * + * @param t type that needs to be serialized + * @param out DataOutput into which serialization needs to happen. + * @param version protocol version + * @throws IOException if serialization fails + */ + public void serialize(In t, DataOutputPlus out, int version) throws IOException; + + /** + * Deserialize into the specified DataInputStream instance. + * @param in DataInput from which deserialization needs to happen. + * @param version protocol version + * @return the type that was deserialized + * @throws IOException if deserialization fails + */ + public Out deserialize(DataInputPlus in, int version) throws IOException; + + /** + * Calculate serialized size of object without actually serializing. + * @param t object to calculate serialized size + * @param version protocol version + * @return serialized size of object t + */ + public long serializedSize(In t, int version); +} diff --git a/src/java/org/apache/cassandra/io/IVersionedSerializer.java b/src/java/org/apache/cassandra/io/IVersionedSerializer.java index e5555735079d..6730ec08249e 100644 --- a/src/java/org/apache/cassandra/io/IVersionedSerializer.java +++ b/src/java/org/apache/cassandra/io/IVersionedSerializer.java @@ -17,37 +17,6 @@ */ package org.apache.cassandra.io; -import java.io.IOException; - -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; - -public interface IVersionedSerializer +public interface IVersionedSerializer extends IVersionedAsymmetricSerializer { - /** - * Serialize the specified type into the specified DataOutputStream instance. - * - * @param t type that needs to be serialized - * @param out DataOutput into which serialization needs to happen. - * @param version protocol version - * @throws java.io.IOException if serialization fails - */ - public void serialize(T t, DataOutputPlus out, int version) throws IOException; - - /** - * Deserialize into the specified DataInputStream instance. - * @param in DataInput from which deserialization needs to happen. - * @param version protocol version - * @return the type that was deserialized - * @throws IOException if deserialization fails - */ - public T deserialize(DataInputPlus in, int version) throws IOException; - - /** - * Calculate serialized size of object without actually serializing. - * @param t object to calculate serialized size - * @param version protocol version - * @return serialized size of object t - */ - public long serializedSize(T t, int version); } diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index e3059c873f6d..b545d5105f3b 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -56,6 +56,7 @@ import org.apache.cassandra.dht.Bounds; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.UnknownColumnException; import org.apache.cassandra.io.FSError; import org.apache.cassandra.io.compress.CompressionMetadata; import org.apache.cassandra.io.sstable.*; @@ -431,13 +432,22 @@ public static SSTableReader openForBatch(Descriptor descriptor, Set c long fileLength = new File(descriptor.filenameFor(Component.DATA)).length(); if (logger.isDebugEnabled()) logger.debug("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength)); - SSTableReader sstable = internalOpen(descriptor, - components, - metadata, - System.currentTimeMillis(), - statsMetadata, - OpenReason.NORMAL, - header.toHeader(metadata.get())); + + final SSTableReader sstable; + try + { + sstable = internalOpen(descriptor, + components, + metadata, + System.currentTimeMillis(), + statsMetadata, + OpenReason.NORMAL, + header.toHeader(metadata.get())); + } + catch (UnknownColumnException e) + { + throw new IllegalStateException(e); + } try(FileHandle.Builder ibuilder = new FileHandle.Builder(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)) .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap) @@ -522,13 +532,22 @@ public static SSTableReader open(Descriptor descriptor, long fileLength = new File(descriptor.filenameFor(Component.DATA)).length(); if (logger.isDebugEnabled()) logger.debug("Opening {} ({})", descriptor, FBUtilities.prettyPrintMemory(fileLength)); - SSTableReader sstable = internalOpen(descriptor, - components, - metadata, - System.currentTimeMillis(), - statsMetadata, - OpenReason.NORMAL, - header.toHeader(metadata.get())); + + final SSTableReader sstable; + try + { + sstable = internalOpen(descriptor, + components, + metadata, + System.currentTimeMillis(), + statsMetadata, + OpenReason.NORMAL, + header.toHeader(metadata.get())); + } + catch (UnknownColumnException e) + { + throw new IllegalStateException(e); + } try { @@ -2475,12 +2494,14 @@ public static SSTableReader moveAndOpenSSTable(ColumnFamilyStore cfs, Descriptor return reader; } - public static void shutdownBlocking() throws InterruptedException + public static void shutdownBlocking(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { if (syncExecutor != null) { syncExecutor.shutdownNow(); - syncExecutor.awaitTermination(0, TimeUnit.SECONDS); + syncExecutor.awaitTermination(timeout, unit); + if (!syncExecutor.isTerminated()) + throw new TimeoutException(); } resetTidying(); } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java index 882638121305..f05ea94cb7ea 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriter.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.io.sstable.format.big; +import java.io.EOFException; import java.io.File; import java.io.IOException; import java.util.Collection; @@ -31,7 +32,6 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.compress.BufferType; @@ -43,7 +43,7 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.SequentialWriter; import org.apache.cassandra.io.util.SequentialWriterOption; -import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadataRef; @@ -202,22 +202,25 @@ public void writeComponent(Component.Type type, DataInputPlus in, long size) { logger.info("Writing component {} to {} length {}", type, componentWriters.get(type).getPath(), prettyPrintMemory(size)); - if (in instanceof RebufferingByteBufDataInputPlus) - write((RebufferingByteBufDataInputPlus) in, size, componentWriters.get(type)); + if (in instanceof AsyncStreamingInputPlus) + write((AsyncStreamingInputPlus) in, size, componentWriters.get(type)); else write(in, size, componentWriters.get(type)); } - private void write(RebufferingByteBufDataInputPlus in, long size, SequentialWriter writer) + private void write(AsyncStreamingInputPlus in, long size, SequentialWriter writer) { logger.info("Block Writing component to {} length {}", writer.getPath(), prettyPrintMemory(size)); try { - long bytesWritten = in.consumeUntil(writer, size); - - if (bytesWritten != size) - throw new IOException(format("Failed to read correct number of bytes from channel %s", writer)); + in.consume(writer::writeDirectlyToChannel, size); + writer.sync(); + } + // FIXME: handle ACIP exceptions properly + catch (EOFException | AsyncStreamingInputPlus.InputTimeoutException e) + { + in.close(); } catch (IOException e) { diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java index 56d88f7c2b3f..7d1e91d641e6 100644 --- a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java @@ -28,8 +28,8 @@ import net.nicoulaj.compilecommand.annotations.DontInline; import org.apache.cassandra.config.Config; +import org.apache.cassandra.utils.FastByteOperations; import org.apache.cassandra.utils.memory.MemoryUtil; -import org.apache.cassandra.utils.vint.VIntCoding; /** * An implementation of the DataOutputStreamPlus interface using a ByteBuffer to stage writes @@ -43,15 +43,6 @@ public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus protected ByteBuffer buffer; - //Allow derived classes to specify writing to the channel - //directly shouldn't happen because they intercept via doFlush for things - //like compression or checksumming - //Another hack for this value is that it also indicates that flushing early - //should not occur, flushes aligned with buffer size are desired - //Unless... it's the last flush. Compression and checksum formats - //expect block (same as buffer size) alignment for everything except the last block - protected boolean strictFlushing = false; - public BufferedDataOutputStreamPlus(RandomAccessFile ras) { this(ras.getChannel()); @@ -132,9 +123,6 @@ public void write(byte[] b, int off, int len) throws IOException } } - // ByteBuffer to use for defensive copies - private final ByteBuffer hollowBuffer = MemoryUtil.getHollowDirectByteBuffer(); - /* * Makes a defensive copy of the incoming ByteBuffer and don't modify the position or limit * even temporarily so it is thread-safe WRT to the incoming buffer @@ -142,48 +130,20 @@ public void write(byte[] b, int off, int len) throws IOException * @see org.apache.cassandra.io.util.DataOutputPlus#write(java.nio.ByteBuffer) */ @Override - public void write(ByteBuffer toWrite) throws IOException - { - if (toWrite.hasArray()) - { - write(toWrite.array(), toWrite.arrayOffset() + toWrite.position(), toWrite.remaining()); - } - else - { - assert toWrite.isDirect(); - MemoryUtil.duplicateDirectByteBuffer(toWrite, hollowBuffer); - int toWriteRemaining = toWrite.remaining(); - - if (toWriteRemaining > buffer.remaining()) - { - if (strictFlushing) - { - writeExcessSlow(); - } - else - { - doFlush(toWriteRemaining - buffer.remaining()); - while (hollowBuffer.remaining() > buffer.capacity()) - channel.write(hollowBuffer); - } - } - - buffer.put(hollowBuffer); - } - } - - // writes anything we can't fit into the buffer - @DontInline - private void writeExcessSlow() throws IOException + public void write(ByteBuffer src) throws IOException { - int originalLimit = hollowBuffer.limit(); - while (originalLimit - hollowBuffer.position() > buffer.remaining()) + int srcPos = src.position(); + int srcCount; + int trgAvailable; + while ((srcCount = src.limit() - srcPos) > (trgAvailable = buffer.remaining())) { - hollowBuffer.limit(hollowBuffer.position() + buffer.remaining()); - buffer.put(hollowBuffer); - doFlush(originalLimit - hollowBuffer.position()); + FastByteOperations.copy(src, srcPos, buffer, buffer.position(), trgAvailable); + buffer.position(buffer.position() + trgAvailable); + srcPos += trgAvailable; + doFlush(src.limit() - srcPos); } - hollowBuffer.limit(originalLimit); + FastByteOperations.copy(src, srcPos, buffer, buffer.position(), srcCount); + buffer.position(buffer.position() + srcCount); } @Override @@ -241,25 +201,6 @@ public void writeLong(long v) throws IOException buffer.putLong(v); } - @Override - public void writeVInt(long value) throws IOException - { - writeUnsignedVInt(VIntCoding.encodeZigZag64(value)); - } - - @Override - public void writeUnsignedVInt(long value) throws IOException - { - int size = VIntCoding.computeUnsignedVIntSize(value); - if (size == 1) - { - write((int) value); - return; - } - - write(VIntCoding.encodeVInt(value, size), 0, size); - } - @Override public void writeFloat(float v) throws IOException { @@ -302,13 +243,6 @@ public void writeUTF(String s) throws IOException UnbufferedDataOutputStreamPlus.writeUTF(s, this); } - @Override - public void write(Memory memory, long offset, long length) throws IOException - { - for (ByteBuffer buffer : memory.asByteBuffers(offset, length)) - write(buffer); - } - /* * Count is the number of bytes remaining to write ignoring already remaining capacity */ @@ -338,16 +272,6 @@ public void close() throws IOException buffer = null; } - @Override - public R applyToChannel(CheckedFunction f) throws IOException - { - if (strictFlushing) - throw new UnsupportedOperationException(); - //Don't allow writes to the underlying channel while data is buffered - flush(); - return f.apply(channel); - } - public BufferedDataOutputStreamPlus order(ByteOrder order) { this.buffer.order(order); diff --git a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java index 16be42f889be..b94d097b6031 100644 --- a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java +++ b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java @@ -20,7 +20,6 @@ import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.WritableByteChannel; import org.apache.cassandra.utils.vint.VIntCoding; @@ -33,13 +32,11 @@ public interface DataOutputPlus extends DataOutput // write the buffer without modifying its position void write(ByteBuffer buffer) throws IOException; - void write(Memory memory, long offset, long length) throws IOException; - - /** - * Safe way to operate against the underlying channel. Impossible to stash a reference to the channel - * and forget to flush - */ - R applyToChannel(CheckedFunction c) throws IOException; + default void write(Memory memory, long offset, long length) throws IOException + { + for (ByteBuffer buffer : memory.asByteBuffers(offset, length)) + write(buffer); + } default void writeVInt(long i) throws IOException { diff --git a/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java index 4adb6d20fcc4..e931899c0763 100644 --- a/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java @@ -119,7 +119,7 @@ public int write(ByteBuffer src) throws IOException { int toWriteThisTime = Math.min(buf.length, toWrite - totalWritten); - ByteBufferUtil.arrayCopy(src, src.position() + totalWritten, buf, 0, toWriteThisTime); + ByteBufferUtil.copyBytes(src, src.position() + totalWritten, buf, 0, toWriteThisTime); DataOutputStreamPlus.this.write(buf, 0, toWriteThisTime); diff --git a/src/java/org/apache/cassandra/io/util/FastByteArrayInputStream.java b/src/java/org/apache/cassandra/io/util/FastByteArrayInputStream.java deleted file mode 100644 index f61546c95e85..000000000000 --- a/src/java/org/apache/cassandra/io/util/FastByteArrayInputStream.java +++ /dev/null @@ -1,249 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.io.util; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; - -/* - * This file has been modified from Apache Harmony's ByteArrayInputStream - * implementation. The synchronized methods of the original have been - * replaced by non-synchronized methods. This makes this certain operations - * FASTer, but also *not thread-safe*. - * - * This file remains formatted the same as the Apache Harmony original to - * make patching easier if any bug fixes are made to the Harmony version. - */ - -/** - * A specialized {@link InputStream } for reading the contents of a byte array. - * - * @see ByteArrayInputStream - */ -public class FastByteArrayInputStream extends InputStream -{ - /** - * The {@code byte} array containing the bytes to stream over. - */ - protected byte[] buf; - - /** - * The current position within the byte array. - */ - protected int pos; - - /** - * The current mark position. Initially set to 0 or the offset - * parameter within the constructor. - */ - protected int mark; - - /** - * The total number of bytes initially available in the byte array - * {@code buf}. - */ - protected int count; - - /** - * Constructs a new {@code ByteArrayInputStream} on the byte array - * {@code buf}. - * - * @param buf - * the byte array to stream over. - */ - public FastByteArrayInputStream(byte buf[]) - { - this.mark = 0; - this.buf = buf; - this.count = buf.length; - } - - /** - * Constructs a new {@code ByteArrayInputStream} on the byte array - * {@code buf} with the initial position set to {@code offset} and the - * number of bytes available set to {@code offset} + {@code length}. - * - * @param buf - * the byte array to stream over. - * @param offset - * the initial position in {@code buf} to start streaming from. - * @param length - * the number of bytes available for streaming. - */ - public FastByteArrayInputStream(byte buf[], int offset, int length) - { - this.buf = buf; - pos = offset; - mark = offset; - count = offset + length > buf.length ? buf.length : offset + length; - } - - /** - * Returns the number of bytes that are available before this stream will - * block. This method returns the number of bytes yet to be read from the - * source byte array. - * - * @return the number of bytes available before blocking. - */ - @Override - public int available() - { - return count - pos; - } - - /** - * Closes this stream and frees resources associated with this stream. - * - * @throws IOException - * if an I/O error occurs while closing this stream. - */ - @Override - public void close() throws IOException - { - // Do nothing on close, this matches JDK behaviour. - } - - /** - * Sets a mark position in this ByteArrayInputStream. The parameter - * {@code readlimit} is ignored. Sending {@code reset()} will reposition the - * stream back to the marked position. - * - * @param readlimit - * ignored. - * @see #markSupported() - * @see #reset() - */ - @Override - public void mark(int readlimit) - { - mark = pos; - } - - /** - * Indicates whether this stream supports the {@code mark()} and - * {@code reset()} methods. Returns {@code true} since this class supports - * these methods. - * - * @return always {@code true}. - * @see #mark(int) - * @see #reset() - */ - @Override - public boolean markSupported() - { - return true; - } - - /** - * Reads a single byte from the source byte array and returns it as an - * integer in the range from 0 to 255. Returns -1 if the end of the source - * array has been reached. - * - * @return the byte read or -1 if the end of this stream has been reached. - */ - @Override - public int read() - { - return pos < count ? buf[pos++] & 0xFF : -1; - } - - /** - * Reads at most {@code len} bytes from this stream and stores - * them in byte array {@code b} starting at {@code offset}. This - * implementation reads bytes from the source byte array. - * - * @param b - * the byte array in which to store the bytes read. - * @param offset - * the initial position in {@code b} to store the bytes read from - * this stream. - * @param length - * the maximum number of bytes to store in {@code b}. - * @return the number of bytes actually read or -1 if no bytes were read and - * the end of the stream was encountered. - * @throws IndexOutOfBoundsException - * if {@code offset < 0} or {@code length < 0}, or if - * {@code offset + length} is greater than the size of - * {@code b}. - * @throws NullPointerException - * if {@code b} is {@code null}. - */ - @Override - public int read(byte b[], int offset, int length) - { - if (b == null) { - throw new NullPointerException(); - } - // avoid int overflow - if (offset < 0 || offset > b.length || length < 0 - || length > b.length - offset) - { - throw new IndexOutOfBoundsException(); - } - // Are there any bytes available? - if (this.pos >= this.count) - { - return -1; - } - if (length == 0) - { - return 0; - } - - int copylen = this.count - pos < length ? this.count - pos : length; - System.arraycopy(buf, pos, b, offset, copylen); - pos += copylen; - return copylen; - } - - /** - * Resets this stream to the last marked location. This implementation - * resets the position to either the marked position, the start position - * supplied in the constructor or 0 if neither has been provided. - * - * @see #mark(int) - */ - @Override - public void reset() - { - pos = mark; - } - - /** - * Skips {@code count} number of bytes in this InputStream. Subsequent - * {@code read()}s will not return these bytes unless {@code reset()} is - * used. This implementation skips {@code count} number of bytes in the - * target stream. It does nothing and returns 0 if {@code n} is negative. - * - * @param n - * the number of bytes to skip. - * @return the number of bytes actually skipped. - */ - @Override - public long skip(long n) - { - if (n <= 0) - { - return 0; - } - int temp = pos; - pos = this.count - pos < n ? this.count : (int) (pos + n); - return pos - temp; - } -} diff --git a/src/java/org/apache/cassandra/io/util/Memory.java b/src/java/org/apache/cassandra/io/util/Memory.java index 0ca6aa214a6c..eaa6e919e6d1 100644 --- a/src/java/org/apache/cassandra/io/util/Memory.java +++ b/src/java/org/apache/cassandra/io/util/Memory.java @@ -417,7 +417,7 @@ public ByteBuffer asByteBuffer(long offset, int length) public void setByteBuffer(ByteBuffer buffer, long offset, int length) { checkBounds(offset, offset + length); - MemoryUtil.setByteBuffer(buffer, peer + offset, length); + MemoryUtil.setDirectByteBuffer(buffer, peer + offset, length); } public String toString() diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java index 3eb1a7d81500..9ad944be3bc0 100644 --- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java +++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; @@ -43,6 +44,15 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr protected final FileChannel fchannel; + //Allow derived classes to specify writing to the channel + //directly shouldn't happen because they intercept via doFlush for things + //like compression or checksumming + //Another hack for this value is that it also indicates that flushing early + //should not occur, flushes aligned with buffer size are desired + //Unless... it's the last flush. Compression and checksum formats + //expect block (same as buffer size) alignment for everything except the last block + private final boolean strictFlushing; + // whether to do trickling fsync() to avoid sudden bursts of dirty buffer flushing by kernel causing read // latency spikes private final SequentialWriterOption option; @@ -388,6 +398,15 @@ public final void close() txnProxy.close(); } + public int writeDirectlyToChannel(ByteBuffer buf) throws IOException + { + if (strictFlushing) + throw new UnsupportedOperationException(); + // Don't allow writes to the underlying channel while data is buffered + flush(); + return channel.write(buf); + } + public final void finish() { txnProxy.finish(); diff --git a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java index d9ef01064058..3d8321296cd1 100644 --- a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java +++ b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java @@ -371,15 +371,4 @@ public void write(ByteBuffer buf) throws IOException } } - public void write(Memory memory, long offset, long length) throws IOException - { - for (ByteBuffer buffer : memory.asByteBuffers(offset, length)) - write(buffer); - } - - @Override - public R applyToChannel(CheckedFunction f) throws IOException - { - return f.apply(channel); - } } diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java index ddc8fba276f5..0b241ce0d519 100644 --- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java +++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java @@ -35,6 +35,7 @@ import org.apache.cassandra.gms.EndpointState; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.net.LatencySubscribers; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; @@ -43,7 +44,7 @@ /** * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector */ -public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements ILatencySubscriber, DynamicEndpointSnitchMBean +public class DynamicEndpointSnitch extends AbstractEndpointSnitch implements LatencySubscribers.Subscriber, DynamicEndpointSnitchMBean { private static final boolean USE_SEVERITY = !Boolean.getBoolean("cassandra.ignore_dynamic_snitch_severity"); @@ -253,7 +254,7 @@ public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) throw new UnsupportedOperationException("You shouldn't wrap the DynamicEndpointSnitch (within itself or otherwise)"); } - public void receiveTiming(InetAddressAndPort host, long latency) // this is cheap + public void receiveTiming(InetAddressAndPort host, long latency, TimeUnit unit) // this is cheap { ExponentiallyDecayingReservoir sample = samples.get(host); if (sample == null) @@ -263,7 +264,7 @@ public void receiveTiming(InetAddressAndPort host, long latency) // this is chea if (sample == null) sample = maybeNewSample; } - sample.update(latency); + sample.update(unit.toMillis(latency)); } private void updateScores() // this is expensive @@ -274,7 +275,7 @@ private void updateScores() // this is expensive { if (MessagingService.instance() != null) { - MessagingService.instance().register(this); + MessagingService.instance().latencySubscribers.subscribe(this); registered = true; } diff --git a/src/java/org/apache/cassandra/locator/Endpoints.java b/src/java/org/apache/cassandra/locator/Endpoints.java index a2bad6ce6fb2..c1a928214487 100644 --- a/src/java/org/apache/cassandra/locator/Endpoints.java +++ b/src/java/org/apache/cassandra/locator/Endpoints.java @@ -21,11 +21,15 @@ import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict; import org.apache.cassandra.utils.FBUtilities; +import java.util.AbstractList; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import com.google.common.collect.Lists; + /** * A collection of Endpoints for a given ring position. This will typically reside in a ReplicaLayout, * representing some subset of the endpoints for the Token or Range @@ -52,6 +56,22 @@ public Set endpoints() return byEndpoint().keySet(); } + public List endpointList() + { + return new AbstractList() + { + public InetAddressAndPort get(int index) + { + return list.get(index).endpoint(); + } + + public int size() + { + return list.size; + } + }; + } + public Map byEndpoint() { ReplicaMap map = byEndpoint; diff --git a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java index a47c72a71696..6821f139867b 100644 --- a/src/java/org/apache/cassandra/locator/InetAddressAndPort.java +++ b/src/java/org/apache/cassandra/locator/InetAddressAndPort.java @@ -15,16 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.locator; +import java.io.IOException; import java.io.Serializable; +import java.net.Inet4Address; +import java.net.Inet6Address; import java.net.InetAddress; import java.net.UnknownHostException; +import java.nio.ByteBuffer; import com.google.common.base.Preconditions; import com.google.common.net.HostAndPort; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.FastByteOperations; @@ -41,6 +49,7 @@ * need to sometimes return a port and sometimes not. * */ +@SuppressWarnings("UnstableApiUsage") public final class InetAddressAndPort implements Comparable, Serializable { private static final long serialVersionUID = 0; @@ -65,6 +74,11 @@ private InetAddressAndPort(InetAddress address, byte[] addressBytes, int port) this.addressBytes = addressBytes; } + public InetAddressAndPort withPort(int port) + { + return new InetAddressAndPort(address, addressBytes, port); + } + private static void validatePortRange(int port) { if (port < 0 | port > 65535) @@ -127,7 +141,7 @@ public String toString(boolean withPort) { if (withPort) { - return HostAndPort.fromParts(address.getHostAddress(), port).toString(); + return toString(address, port); } else { @@ -135,6 +149,11 @@ public String toString(boolean withPort) } } + public static String toString(InetAddress address, int port) + { + return HostAndPort.fromParts(address.getHostAddress(), port).toString(); + } + public static InetAddressAndPort getByName(String name) throws UnknownHostException { return getByNameOverrideDefaults(name, null); @@ -144,8 +163,6 @@ public static InetAddressAndPort getByName(String name) throws UnknownHostExcept * * @param name Hostname + optional ports string * @param port Port to connect on, overridden by values in hostname string, defaults to DatabaseDescriptor default if not specified anywhere. - * @return - * @throws UnknownHostException */ public static InetAddressAndPort getByNameOverrideDefaults(String name, Integer port) throws UnknownHostException { @@ -201,4 +218,114 @@ public static void initializeDefaultPort(int port) { defaultPort = port; } + + static int getDefaultPort() + { + return defaultPort; + } + + /* + * As of version 4.0 the endpoint description includes a port number as an unsigned short + */ + public static final class Serializer implements IVersionedSerializer + { + public static final int MAXIMUM_SIZE = 19; + + // We put the static instance here, to avoid complexity with dtests. + // InetAddressAndPort is one of the only classes we share between instances, which is possible cleanly + // because it has no type-dependencies in its public API, however Serializer requires DataOutputPlus, which requires... + // and the chain becomes quite unwieldy + public static final Serializer inetAddressAndPortSerializer = new Serializer(); + + private Serializer() {} + + public void serialize(InetAddressAndPort endpoint, DataOutputPlus out, int version) throws IOException + { + byte[] buf = endpoint.addressBytes; + + if (version >= MessagingService.VERSION_40) + { + out.writeByte(buf.length + 2); + out.write(buf); + out.writeShort(endpoint.port); + } + else + { + out.writeByte(buf.length); + out.write(buf); + } + } + + public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException + { + int size = in.readByte() & 0xFF; + switch(size) + { + //The original pre-4.0 serialiation of just an address + case 4: + case 16: + { + byte[] bytes = new byte[size]; + in.readFully(bytes, 0, bytes.length); + return getByAddress(bytes); + } + //Address and one port + case 6: + case 18: + { + byte[] bytes = new byte[size - 2]; + in.readFully(bytes); + + int port = in.readShort() & 0xFFFF; + return getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port); + } + default: + throw new AssertionError("Unexpected size " + size); + + } + } + + /** + * Extract {@link InetAddressAndPort} from the provided {@link ByteBuffer} without altering its state. + */ + public InetAddressAndPort extract(ByteBuffer buf, int position) throws IOException + { + int size = buf.get(position++) & 0xFF; + if (size == 4 || size == 16) + { + byte[] bytes = new byte[size]; + ByteBufferUtil.copyBytes(buf, position, bytes, 0, size); + return getByAddress(bytes); + } + else if (size == 6 || size == 18) + { + byte[] bytes = new byte[size - 2]; + ByteBufferUtil.copyBytes(buf, position, bytes, 0, size - 2); + position += (size - 2); + int port = buf.getShort(position) & 0xFFFF; + return getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port); + } + + throw new AssertionError("Unexpected pre-4.0 InetAddressAndPort size " + size); + } + + public long serializedSize(InetAddressAndPort from, int version) + { + //4.0 includes a port number + if (version >= MessagingService.VERSION_40) + { + if (from.address instanceof Inet4Address) + return 1 + 4 + 2; + assert from.address instanceof Inet6Address; + return 1 + 16 + 2; + } + else + { + if (from.address instanceof Inet4Address) + return 1 + 4; + assert from.address instanceof Inet6Address; + return 1 + 16; + } + } + } } diff --git a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java index 547901086d9a..dea8c76f4e4e 100644 --- a/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java +++ b/src/java/org/apache/cassandra/locator/ReconnectableSnitchHelper.java @@ -22,13 +22,16 @@ import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.gms.*; +import org.apache.cassandra.net.ConnectionCategory; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.OutboundConnectionSettings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES; + /** * Sidekick helper for snitches that want to reconnect from one IP addr for a node to another. * Typically, this is for situations like EC2 where a node will have a public address and a private address, @@ -63,16 +66,15 @@ private void reconnect(InetAddressAndPort publicAddress, VersionedValue localAdd @VisibleForTesting static void reconnect(InetAddressAndPort publicAddress, InetAddressAndPort localAddress, IEndpointSnitch snitch, String localDc) { - if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(publicAddress.address, MessagingService.instance().portFor(publicAddress))) + if (!new OutboundConnectionSettings(publicAddress, localAddress).withDefaults(ConnectionCategory.MESSAGING).authenticate()) { logger.debug("InternodeAuthenticator said don't reconnect to {} on {}", publicAddress, localAddress); return; } - if (snitch.getDatacenter(publicAddress).equals(localDc) - && !MessagingService.instance().getCurrentEndpoint(publicAddress).equals(localAddress)) + if (snitch.getDatacenter(publicAddress).equals(localDc)) { - MessagingService.instance().reconnectWithNewIp(publicAddress, localAddress); + MessagingService.instance().maybeReconnectWithNewIp(publicAddress, localAddress); logger.debug("Initiated reconnect to an Internal IP {} for the {}", localAddress, publicAddress); } } diff --git a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java deleted file mode 100644 index 3655a404d080..000000000000 --- a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.metrics; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Meter; -import org.apache.cassandra.net.async.OutboundMessagingPool; - -import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; - -import org.apache.cassandra.locator.InetAddressAndPort; - -/** - * Metrics for internode connections. - */ -public class ConnectionMetrics -{ - public static final String TYPE_NAME = "Connection"; - - /** Total number of timeouts happened on this node */ - public static final Meter totalTimeouts = Metrics.meter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null)); - - public final String address; - /** Pending tasks for large message TCP Connections */ - public final Gauge largeMessagePendingTasks; - /** Completed tasks for large message TCP Connections */ - public final Gauge largeMessageCompletedTasks; - /** Dropped tasks for large message TCP Connections */ - public final Gauge largeMessageDroppedTasks; - /** Pending tasks for small message TCP Connections */ - public final Gauge smallMessagePendingTasks; - /** Completed tasks for small message TCP Connections */ - public final Gauge smallMessageCompletedTasks; - /** Dropped tasks for small message TCP Connections */ - public final Gauge smallMessageDroppedTasks; - /** Pending tasks for gossip message TCP Connections */ - public final Gauge gossipMessagePendingTasks; - /** Completed tasks for gossip message TCP Connections */ - public final Gauge gossipMessageCompletedTasks; - /** Dropped tasks for gossip message TCP Connections */ - public final Gauge gossipMessageDroppedTasks; - - /** Number of timeouts for specific IP */ - public final Meter timeouts; - - private final MetricNameFactory factory; - - /** - * Create metrics for given connection pool. - * - * @param ip IP address to use for metrics label - */ - public ConnectionMetrics(InetAddressAndPort ip, final OutboundMessagingPool messagingPool) - { - // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName - address = ip.toString().replace(':', '.'); - - factory = new DefaultNameFactory("Connection", address); - - largeMessagePendingTasks = Metrics.register(factory.createMetricName("LargeMessagePendingTasks"), new Gauge() - { - public Integer getValue() - { - return messagingPool.largeMessageChannel.getPendingMessages(); - } - }); - largeMessageCompletedTasks = Metrics.register(factory.createMetricName("LargeMessageCompletedTasks"), new Gauge() - { - public Long getValue() - { - return messagingPool.largeMessageChannel.getCompletedMessages(); - } - }); - largeMessageDroppedTasks = Metrics.register(factory.createMetricName("LargeMessageDroppedTasks"), new Gauge() - { - public Long getValue() - { - return messagingPool.largeMessageChannel.getDroppedMessages(); - } - }); - smallMessagePendingTasks = Metrics.register(factory.createMetricName("SmallMessagePendingTasks"), new Gauge() - { - public Integer getValue() - { - return messagingPool.smallMessageChannel.getPendingMessages(); - } - }); - smallMessageCompletedTasks = Metrics.register(factory.createMetricName("SmallMessageCompletedTasks"), new Gauge() - { - public Long getValue() - { - return messagingPool.smallMessageChannel.getCompletedMessages(); - } - }); - smallMessageDroppedTasks = Metrics.register(factory.createMetricName("SmallMessageDroppedTasks"), new Gauge() - { - public Long getValue() - { - return messagingPool.smallMessageChannel.getDroppedMessages(); - } - }); - gossipMessagePendingTasks = Metrics.register(factory.createMetricName("GossipMessagePendingTasks"), new Gauge() - { - public Integer getValue() - { - return messagingPool.gossipChannel.getPendingMessages(); - } - }); - gossipMessageCompletedTasks = Metrics.register(factory.createMetricName("GossipMessageCompletedTasks"), new Gauge() - { - public Long getValue() - { - return messagingPool.gossipChannel.getCompletedMessages(); - } - }); - gossipMessageDroppedTasks = Metrics.register(factory.createMetricName("GossipMessageDroppedTasks"), new Gauge() - { - public Long getValue() - { - return messagingPool.gossipChannel.getDroppedMessages(); - } - }); - timeouts = Metrics.meter(factory.createMetricName("Timeouts")); - } - - public void release() - { - Metrics.remove(factory.createMetricName("LargeMessagePendingTasks")); - Metrics.remove(factory.createMetricName("LargeMessageCompletedTasks")); - Metrics.remove(factory.createMetricName("LargeMessageDroppedTasks")); - Metrics.remove(factory.createMetricName("SmallMessagePendingTasks")); - Metrics.remove(factory.createMetricName("SmallMessageCompletedTasks")); - Metrics.remove(factory.createMetricName("SmallMessageDroppedTasks")); - Metrics.remove(factory.createMetricName("GossipMessagePendingTasks")); - Metrics.remove(factory.createMetricName("GossipMessageCompletedTasks")); - Metrics.remove(factory.createMetricName("GossipMessageDroppedTasks")); - Metrics.remove(factory.createMetricName("Timeouts")); - } -} diff --git a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java index 794fa9cb8e77..8c227783f559 100644 --- a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java +++ b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java @@ -21,6 +21,7 @@ import com.codahale.metrics.Timer; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; @@ -38,7 +39,7 @@ public class DroppedMessageMetrics /** The cross node dropped latency */ public final Timer crossNodeDroppedLatency; - public DroppedMessageMetrics(MessagingService.Verb verb) + public DroppedMessageMetrics(Verb verb) { this(new DefaultNameFactory("DroppedMessage", verb.toString())); } diff --git a/src/java/org/apache/cassandra/metrics/FrequencySampler.java b/src/java/org/apache/cassandra/metrics/FrequencySampler.java index c09434714d7a..8a8918b9fa57 100644 --- a/src/java/org/apache/cassandra/metrics/FrequencySampler.java +++ b/src/java/org/apache/cassandra/metrics/FrequencySampler.java @@ -26,6 +26,8 @@ import com.clearspring.analytics.stream.StreamSummary; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + /** * Find the most frequent sample. A sample adds to the sum of its key ie *

add("x", 10); and add("x", 20); will result in "x" = 30

This uses StreamSummary to only store the @@ -37,7 +39,7 @@ public abstract class FrequencySampler extends Sampler { private static final Logger logger = LoggerFactory.getLogger(FrequencySampler.class); - private long endTimeMillis = -1; + private long endTimeNanos = -1; private StreamSummary summary; @@ -51,10 +53,10 @@ public abstract class FrequencySampler extends Sampler */ public synchronized void beginSampling(int capacity, int durationMillis) { - if (endTimeMillis == -1 || clock.currentTimeMillis() > endTimeMillis) + if (endTimeNanos == -1 || clock.now() > endTimeNanos) { - summary = new StreamSummary(capacity); - endTimeMillis = clock.currentTimeMillis() + durationMillis; + summary = new StreamSummary<>(capacity); + endTimeNanos = clock.now() + MILLISECONDS.toNanos(durationMillis); } else throw new RuntimeException("Sampling already in progress"); @@ -67,9 +69,9 @@ public synchronized void beginSampling(int capacity, int durationMillis) public synchronized List> finishSampling(int count) { List> results = Collections.emptyList(); - if (endTimeMillis != -1) + if (endTimeNanos != -1) { - endTimeMillis = -1; + endTimeNanos = -1; results = summary.topK(count) .stream() .map(c -> new Sample(c.getItem(), c.getCount(), c.getError())) @@ -82,7 +84,7 @@ protected synchronized void insert(final T item, final long value) { // samplerExecutor is single threaded but still need // synchronization against jmx calls to finishSampling - if (value > 0 && clock.currentTimeMillis() <= endTimeMillis) + if (value > 0 && clock.now() <= endTimeNanos) { try { @@ -96,7 +98,7 @@ protected synchronized void insert(final T item, final long value) public boolean isEnabled() { - return endTimeMillis != -1 && clock.currentTimeMillis() <= endTimeMillis; + return endTimeNanos != -1 && clock.now() <= endTimeNanos; } } diff --git a/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java b/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java new file mode 100644 index 000000000000..cc3c1c0d2d73 --- /dev/null +++ b/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.metrics; + +import com.codahale.metrics.Gauge; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.InboundMessageHandlers; +import org.apache.cassandra.metrics.CassandraMetricsRegistry.MetricName; + +/** + * Metrics for internode connections. + */ +public class InternodeInboundMetrics +{ + private final MetricName corruptFramesRecovered; + private final MetricName corruptFramesUnrecovered; + private final MetricName errorBytes; + private final MetricName errorCount; + private final MetricName expiredBytes; + private final MetricName expiredCount; + private final MetricName pendingBytes; + private final MetricName pendingCount; + private final MetricName processedBytes; + private final MetricName processedCount; + private final MetricName receivedBytes; + private final MetricName receivedCount; + private final MetricName throttledCount; + private final MetricName throttledNanos; + + /** + * Create metrics for given inbound message handlers. + * + * @param peer IP address and port to use for metrics label + */ + public InternodeInboundMetrics(InetAddressAndPort peer, InboundMessageHandlers handlers) + { + // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName + MetricNameFactory factory = new DefaultNameFactory("InboundConnection", peer.toString().replace(':', '_')); + + register(corruptFramesRecovered = factory.createMetricName("CorruptFramesRecovered"), handlers::corruptFramesRecovered); + register(corruptFramesUnrecovered = factory.createMetricName("CorruptFramesUnrecovered"), handlers::corruptFramesUnrecovered); + register(errorBytes = factory.createMetricName("ErrorBytes"), handlers::errorBytes); + register(errorCount = factory.createMetricName("ErrorCount"), handlers::errorCount); + register(expiredBytes = factory.createMetricName("ExpiredBytes"), handlers::expiredBytes); + register(expiredCount = factory.createMetricName("ExpiredCount"), handlers::expiredCount); + register(pendingBytes = factory.createMetricName("ScheduledBytes"), handlers::scheduledBytes); + register(pendingCount = factory.createMetricName("ScheduledCount"), handlers::scheduledCount); + register(processedBytes = factory.createMetricName("ProcessedBytes"), handlers::processedBytes); + register(processedCount = factory.createMetricName("ProcessedCount"), handlers::processedCount); + register(receivedBytes = factory.createMetricName("ReceivedBytes"), handlers::receivedBytes); + register(receivedCount = factory.createMetricName("ReceivedCount"), handlers::receivedCount); + register(throttledCount = factory.createMetricName("ThrottledCount"), handlers::throttledCount); + register(throttledNanos = factory.createMetricName("ThrottledNanos"), handlers::throttledNanos); + } + + public void release() + { + remove(corruptFramesRecovered); + remove(corruptFramesUnrecovered); + remove(errorBytes); + remove(errorCount); + remove(expiredBytes); + remove(expiredCount); + remove(pendingBytes); + remove(pendingCount); + remove(processedBytes); + remove(processedCount); + remove(receivedBytes); + remove(receivedCount); + remove(throttledCount); + remove(throttledNanos); + } + + private static void register(MetricName name, Gauge gauge) + { + CassandraMetricsRegistry.Metrics.register(name, gauge); + } + + private static void remove(MetricName name) + { + CassandraMetricsRegistry.Metrics.remove(name); + } +} diff --git a/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java new file mode 100644 index 000000000000..4e64f663b661 --- /dev/null +++ b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.metrics; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Meter; +import org.apache.cassandra.net.OutboundConnections; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +import org.apache.cassandra.locator.InetAddressAndPort; + +/** + * Metrics for internode connections. + */ +public class InternodeOutboundMetrics +{ + public static final String TYPE_NAME = "Connection"; + + /** Total number of callbacks that were not completed successfully for messages that were sent to this node + * TODO this was always broken, as it never counted those messages without callbacks? So perhaps we can redefine it. */ + public static final Meter totalExpiredCallbacks = Metrics.meter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null)); + + /** Number of timeouts for specific IP */ + public final Meter expiredCallbacks; + + public final String address; + /** Pending tasks for large message TCP Connections */ + public final Gauge largeMessagePendingTasks; + /** Pending bytes for large message TCP Connections */ + public final Gauge largeMessagePendingBytes; + /** Completed tasks for large message TCP Connections */ + public final Gauge largeMessageCompletedTasks; + /** Completed bytes for large message TCP Connections */ + public final Gauge largeMessageCompletedBytes; + /** Dropped tasks for large message TCP Connections */ + public final Gauge largeMessageDropped; + /** Dropped tasks because of timeout for large message TCP Connections */ + public final Gauge largeMessageDroppedTasksDueToTimeout; + /** Dropped bytes because of timeout for large message TCP Connections */ + public final Gauge largeMessageDroppedBytesDueToTimeout; + /** Dropped tasks because of overload for large message TCP Connections */ + public final Gauge largeMessageDroppedTasksDueToOverload; + /** Dropped bytes because of overload for large message TCP Connections */ + public final Gauge largeMessageDroppedBytesDueToOverload; + /** Dropped tasks because of error for large message TCP Connections */ + public final Gauge largeMessageDroppedTasksDueToError; + /** Dropped bytes because of error for large message TCP Connections */ + public final Gauge largeMessageDroppedBytesDueToError; + /** Pending tasks for small message TCP Connections */ + public final Gauge smallMessagePendingTasks; + /** Pending bytes for small message TCP Connections */ + public final Gauge smallMessagePendingBytes; + /** Completed tasks for small message TCP Connections */ + public final Gauge smallMessageCompletedTasks; + /** Completed bytes for small message TCP Connections */ + public final Gauge smallMessageCompletedBytes; + /** Dropped tasks for small message TCP Connections */ + public final Gauge smallMessageDroppedTasks; + /** Dropped tasks because of timeout for small message TCP Connections */ + public final Gauge smallMessageDroppedTasksDueToTimeout; + /** Dropped bytes because of timeout for small message TCP Connections */ + public final Gauge smallMessageDroppedBytesDueToTimeout; + /** Dropped tasks because of overload for small message TCP Connections */ + public final Gauge smallMessageDroppedTasksDueToOverload; + /** Dropped bytes because of overload for small message TCP Connections */ + public final Gauge smallMessageDroppedBytesDueToOverload; + /** Dropped tasks because of error for small message TCP Connections */ + public final Gauge smallMessageDroppedTasksDueToError; + /** Dropped bytes because of error for small message TCP Connections */ + public final Gauge smallMessageDroppedBytesDueToError; + /** Pending tasks for small message TCP Connections */ + public final Gauge urgentMessagePendingTasks; + /** Pending bytes for urgent message TCP Connections */ + public final Gauge urgentMessagePendingBytes; + /** Completed tasks for urgent message TCP Connections */ + public final Gauge urgentMessageCompletedTasks; + /** Completed bytes for urgent message TCP Connections */ + public final Gauge urgentMessageCompletedBytes; + /** Dropped tasks for urgent message TCP Connections */ + public final Gauge urgentMessageDroppedTasks; + /** Dropped tasks because of timeout for urgent message TCP Connections */ + public final Gauge urgentMessageDroppedTasksDueToTimeout; + /** Dropped bytes because of timeout for urgent message TCP Connections */ + public final Gauge urgentMessageDroppedBytesDueToTimeout; + /** Dropped tasks because of overload for urgent message TCP Connections */ + public final Gauge urgentMessageDroppedTasksDueToOverload; + /** Dropped bytes because of overload for urgent message TCP Connections */ + public final Gauge urgentMessageDroppedBytesDueToOverload; + /** Dropped tasks because of error for urgent message TCP Connections */ + public final Gauge urgentMessageDroppedTasksDueToError; + /** Dropped bytes because of error for urgent message TCP Connections */ + public final Gauge urgentMessageDroppedBytesDueToError; + + private final MetricNameFactory factory; + + /** + * Create metrics for given connection pool. + * + * @param ip IP address to use for metrics label + */ + public InternodeOutboundMetrics(InetAddressAndPort ip, final OutboundConnections messagingPool) + { + // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName + address = ip.toString().replace(':', '_'); + + factory = new DefaultNameFactory("Connection", address); + + largeMessagePendingTasks = Metrics.register(factory.createMetricName("LargeMessagePendingTasks"), messagingPool.large::pendingCount); + largeMessagePendingBytes = Metrics.register(factory.createMetricName("LargeMessagePendingBytes"), messagingPool.large::pendingBytes); + largeMessageCompletedTasks = Metrics.register(factory.createMetricName("LargeMessageCompletedTasks"),messagingPool.large::sentCount); + largeMessageCompletedBytes = Metrics.register(factory.createMetricName("LargeMessageCompletedBytes"),messagingPool.large::sentBytes); + largeMessageDropped = Metrics.register(factory.createMetricName("LargeMessageDroppedTasks"), messagingPool.large::dropped); + largeMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToOverload"), messagingPool.large::overloadedBytes); + largeMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToOverload"), messagingPool.large::overloadedCount); + largeMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToTimeout"), messagingPool.large::expiredCount); + largeMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToTimeout"), messagingPool.large::expiredBytes); + largeMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToError"), messagingPool.large::errorCount); + largeMessageDroppedBytesDueToError = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToError"), messagingPool.large::errorBytes); + smallMessagePendingTasks = Metrics.register(factory.createMetricName("SmallMessagePendingTasks"), messagingPool.small::pendingCount); + smallMessagePendingBytes = Metrics.register(factory.createMetricName("SmallMessagePendingBytes"), messagingPool.small::pendingBytes); + smallMessageCompletedTasks = Metrics.register(factory.createMetricName("SmallMessageCompletedTasks"), messagingPool.small::sentCount); + smallMessageCompletedBytes = Metrics.register(factory.createMetricName("SmallMessageCompletedBytes"),messagingPool.small::sentBytes); + smallMessageDroppedTasks = Metrics.register(factory.createMetricName("SmallMessageDroppedTasks"), messagingPool.small::dropped); + smallMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("SmallMessageDroppedTasksDueToOverload"), messagingPool.small::overloadedCount); + smallMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("SmallMessageDroppedBytesDueToOverload"), messagingPool.small::overloadedBytes); + smallMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("SmallMessageDroppedTasksDueToTimeout"), messagingPool.small::expiredCount); + smallMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("SmallMessageDroppedBytesDueToTimeout"), messagingPool.small::expiredBytes); + smallMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("SmallMessageDroppedTasksDueToError"), messagingPool.small::errorCount); + smallMessageDroppedBytesDueToError = Metrics.register(factory.createMetricName("SmallMessageDroppedBytesDueToError"), messagingPool.small::errorBytes); + urgentMessagePendingTasks = Metrics.register(factory.createMetricName("UrgentMessagePendingTasks"), messagingPool.urgent::pendingCount); + urgentMessagePendingBytes = Metrics.register(factory.createMetricName("UrgentMessagePendingBytes"), messagingPool.urgent::pendingBytes); + urgentMessageCompletedTasks = Metrics.register(factory.createMetricName("UrgentMessageCompletedTasks"), messagingPool.urgent::sentCount); + urgentMessageCompletedBytes = Metrics.register(factory.createMetricName("UrgentMessageCompletedBytes"),messagingPool.urgent::sentBytes); + urgentMessageDroppedTasks = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasks"), messagingPool.urgent::dropped); + urgentMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToOverload"), messagingPool.urgent::overloadedBytes); + urgentMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToOverload"), messagingPool.urgent::overloadedCount); + urgentMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToTimeout"), messagingPool.urgent::expiredCount); + urgentMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToTimeout"), messagingPool.urgent::expiredBytes); + urgentMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToError"), messagingPool.urgent::errorCount); + urgentMessageDroppedBytesDueToError = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToError"), messagingPool.urgent::errorBytes); + expiredCallbacks = Metrics.meter(factory.createMetricName("Timeouts")); + + // deprecated + Metrics.register(factory.createMetricName("GossipMessagePendingTasks"), (Gauge) messagingPool.urgent::pendingCount); + Metrics.register(factory.createMetricName("GossipMessageCompletedTasks"), (Gauge) messagingPool.urgent::sentCount); + Metrics.register(factory.createMetricName("GossipMessageDroppedTasks"), (Gauge) messagingPool.urgent::dropped); + } + + public void release() + { + Metrics.remove(factory.createMetricName("LargeMessagePendingTasks")); + Metrics.remove(factory.createMetricName("LargeMessagePendingBytes")); + Metrics.remove(factory.createMetricName("LargeMessageCompletedTasks")); + Metrics.remove(factory.createMetricName("LargeMessageCompletedBytes")); + Metrics.remove(factory.createMetricName("LargeMessageDroppedTasks")); + Metrics.remove(factory.createMetricName("LargeMessageDroppedTasksDueToTimeout")); + Metrics.remove(factory.createMetricName("LargeMessageDroppedBytesDueToTimeout")); + Metrics.remove(factory.createMetricName("LargeMessageDroppedTasksDueToOverload")); + Metrics.remove(factory.createMetricName("LargeMessageDroppedBytesDueToOverload")); + Metrics.remove(factory.createMetricName("LargeMessageDroppedTasksDueToError")); + Metrics.remove(factory.createMetricName("LargeMessageDroppedBytesDueToError")); + Metrics.remove(factory.createMetricName("SmallMessagePendingTasks")); + Metrics.remove(factory.createMetricName("SmallMessagePendingBytes")); + Metrics.remove(factory.createMetricName("SmallMessageCompletedTasks")); + Metrics.remove(factory.createMetricName("SmallMessageCompletedBytes")); + Metrics.remove(factory.createMetricName("SmallMessageDroppedTasks")); + Metrics.remove(factory.createMetricName("SmallMessageDroppedTasksDueToTimeout")); + Metrics.remove(factory.createMetricName("SmallMessageDroppedBytesDueToTimeout")); + Metrics.remove(factory.createMetricName("SmallMessageDroppedTasksDueToOverload")); + Metrics.remove(factory.createMetricName("SmallMessageDroppedBytesDueToOverload")); + Metrics.remove(factory.createMetricName("SmallMessageDroppedTasksDueToError")); + Metrics.remove(factory.createMetricName("SmallMessageDroppedBytesDueToError")); + Metrics.remove(factory.createMetricName("GossipMessagePendingTasks")); + Metrics.remove(factory.createMetricName("GossipMessageCompletedTasks")); + Metrics.remove(factory.createMetricName("GossipMessageDroppedTasks")); + Metrics.remove(factory.createMetricName("UrgentMessagePendingTasks")); + Metrics.remove(factory.createMetricName("UrgentMessagePendingBytes")); + Metrics.remove(factory.createMetricName("UrgentMessageCompletedTasks")); + Metrics.remove(factory.createMetricName("UrgentMessageCompletedBytes")); + Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasks")); + Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasksDueToTimeout")); + Metrics.remove(factory.createMetricName("UrgentMessageDroppedBytesDueToTimeout")); + Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasksDueToOverload")); + Metrics.remove(factory.createMetricName("UrgentMessageDroppedBytesDueToOverload")); + Metrics.remove(factory.createMetricName("UrgentMessageDroppedTasksDueToError")); + Metrics.remove(factory.createMetricName("UrgentMessageDroppedBytesDueToError")); + Metrics.remove(factory.createMetricName("Timeouts")); + } +} diff --git a/src/java/org/apache/cassandra/metrics/MaxSampler.java b/src/java/org/apache/cassandra/metrics/MaxSampler.java index f4fb87351207..df24bb96298b 100644 --- a/src/java/org/apache/cassandra/metrics/MaxSampler.java +++ b/src/java/org/apache/cassandra/metrics/MaxSampler.java @@ -24,23 +24,25 @@ import com.google.common.collect.MinMaxPriorityQueue; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + public abstract class MaxSampler extends Sampler { private int capacity; private MinMaxPriorityQueue> queue; - private long endTimeMillis = -1; + private long endTimeNanos = -1; private final Comparator> comp = Collections.reverseOrder(Comparator.comparing(p -> p.count)); public boolean isEnabled() { - return endTimeMillis != -1 && clock.currentTimeMillis() <= endTimeMillis; + return endTimeNanos != -1 && clock.now() <= endTimeNanos; } public synchronized void beginSampling(int capacity, int durationMillis) { - if (endTimeMillis == -1 || clock.currentTimeMillis() > endTimeMillis) + if (endTimeNanos == -1 || clock.now() > endTimeNanos) { - endTimeMillis = clock.currentTimeMillis() + durationMillis; + endTimeNanos = clock.now() + MILLISECONDS.toNanos(durationMillis); queue = MinMaxPriorityQueue .orderedBy(comp) .maximumSize(Math.max(1, capacity)) @@ -54,9 +56,9 @@ public synchronized void beginSampling(int capacity, int durationMillis) public synchronized List> finishSampling(int count) { List> result = new ArrayList<>(count); - if (endTimeMillis != -1) + if (endTimeNanos != -1) { - endTimeMillis = -1; + endTimeNanos = -1; Sample next; while ((next = queue.poll()) != null && result.size() <= count) result.add(next); @@ -67,7 +69,7 @@ public synchronized List> finishSampling(int count) @Override protected synchronized void insert(T item, long value) { - if (value > 0 && clock.currentTimeMillis() <= endTimeMillis + if (value > 0 && clock.now() <= endTimeNanos && (queue.isEmpty() || queue.size() < capacity || queue.peekLast().count < value)) queue.add(new Sample(item, value, 0)); } diff --git a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java index 2f096f6c371c..0ea2e10ccb8d 100644 --- a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java +++ b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java @@ -17,59 +17,215 @@ */ package org.apache.cassandra.metrics; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; -import org.apache.cassandra.config.DatabaseDescriptor; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.config.DatabaseDescriptor; + import com.codahale.metrics.Timer; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.net.InboundMessageHandlers; +import org.apache.cassandra.net.LatencyConsumer; +import org.apache.cassandra.utils.StatusLogger; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; /** * Metrics for messages */ -public class MessagingMetrics +public class MessagingMetrics implements InboundMessageHandlers.GlobalMetricCallbacks { - private static Logger logger = LoggerFactory.getLogger(MessagingMetrics.class); private static final MetricNameFactory factory = new DefaultNameFactory("Messaging"); - public final Timer crossNodeLatency; - public final ConcurrentHashMap dcLatency; - public final ConcurrentHashMap queueWaitLatency; + private static final Logger logger = LoggerFactory.getLogger(MessagingMetrics.class); + private static final int LOG_DROPPED_INTERVAL_IN_MS = 5000; + + public static class DCLatencyRecorder implements LatencyConsumer + { + public final Timer dcLatency; + public final Timer allLatency; + + DCLatencyRecorder(Timer dcLatency, Timer allLatency) + { + this.dcLatency = dcLatency; + this.allLatency = allLatency; + } + + public void accept(long timeTaken, TimeUnit units) + { + if (timeTaken > 0) + { + dcLatency.update(timeTaken, units); + allLatency.update(timeTaken, units); + } + } + } + + private static final class DroppedForVerb + { + final DroppedMessageMetrics metrics; + final AtomicInteger droppedFromSelf; + final AtomicInteger droppedFromPeer; + + DroppedForVerb(Verb verb) + { + this(new DroppedMessageMetrics(verb)); + } + + DroppedForVerb(DroppedMessageMetrics metrics) + { + this.metrics = metrics; + this.droppedFromSelf = new AtomicInteger(0); + this.droppedFromPeer = new AtomicInteger(0); + } + } + + private final Timer allLatency; + public final ConcurrentHashMap dcLatency; + public final EnumMap internalLatency; + + // total dropped message counts for server lifetime + private final Map droppedMessages = new EnumMap<>(Verb.class); public MessagingMetrics() { - crossNodeLatency = Metrics.timer(factory.createMetricName("CrossNodeLatency")); + allLatency = Metrics.timer(factory.createMetricName("CrossNodeLatency")); dcLatency = new ConcurrentHashMap<>(); - queueWaitLatency = new ConcurrentHashMap<>(); + internalLatency = new EnumMap<>(Verb.class); + for (Verb verb : Verb.VERBS) + internalLatency.put(verb, Metrics.timer(factory.createMetricName(verb + "-WaitLatency"))); + for (Verb verb : Verb.values()) + droppedMessages.put(verb, new DroppedForVerb(verb)); + } + + public DCLatencyRecorder internodeLatencyRecorder(InetAddressAndPort from) + { + String dcName = DatabaseDescriptor.getEndpointSnitch().getDatacenter(from); + DCLatencyRecorder dcUpdater = dcLatency.get(dcName); + if (dcUpdater == null) + dcUpdater = dcLatency.computeIfAbsent(dcName, k -> new DCLatencyRecorder(Metrics.timer(factory.createMetricName(dcName + "-Latency")), allLatency)); + return dcUpdater; + } + + public void recordInternalLatency(Verb verb, long timeTaken, TimeUnit units) + { + if (timeTaken > 0) + internalLatency.get(verb).update(timeTaken, units); + } + + public void recordSelfDroppedMessage(Verb verb) + { + recordDroppedMessage(droppedMessages.get(verb), false); } - public void addTimeTaken(InetAddressAndPort from, long timeTaken) + public void recordSelfDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit) { - String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(from); - Timer timer = dcLatency.get(dc); - if (timer == null) + recordDroppedMessage(verb, timeElapsed, timeUnit, false); + } + + public void recordInternodeDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit) + { + recordDroppedMessage(verb, timeElapsed, timeUnit, true); + } + + public void recordDroppedMessage(Message message, long timeElapsed, TimeUnit timeUnit) + { + recordDroppedMessage(message.verb(), timeElapsed, timeUnit, message.isCrossNode()); + } + + public void recordDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit, boolean isCrossNode) + { + recordDroppedMessage(droppedMessages.get(verb), timeElapsed, timeUnit, isCrossNode); + } + + private static void recordDroppedMessage(DroppedForVerb droppedMessages, long timeTaken, TimeUnit units, boolean isCrossNode) + { + if (isCrossNode) + droppedMessages.metrics.crossNodeDroppedLatency.update(timeTaken, units); + else + droppedMessages.metrics.internalDroppedLatency.update(timeTaken, units); + recordDroppedMessage(droppedMessages, isCrossNode); + } + + private static void recordDroppedMessage(DroppedForVerb droppedMessages, boolean isCrossNode) + { + droppedMessages.metrics.dropped.mark(); + if (isCrossNode) + droppedMessages.droppedFromPeer.incrementAndGet(); + else + droppedMessages.droppedFromSelf.incrementAndGet(); + } + + public void scheduleLogging() + { + ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(this::logDroppedMessages, + LOG_DROPPED_INTERVAL_IN_MS, + LOG_DROPPED_INTERVAL_IN_MS, + MILLISECONDS); + } + + public Map getDroppedMessages() + { + Map map = new HashMap<>(droppedMessages.size()); + for (Map.Entry entry : droppedMessages.entrySet()) + map.put(entry.getKey().toString(), (int) entry.getValue().metrics.dropped.getCount()); + return map; + } + + private void logDroppedMessages() + { + if (resetAndConsumeDroppedErrors(logger::info) > 0) + StatusLogger.log(); + } + + @VisibleForTesting + public int resetAndConsumeDroppedErrors(Consumer messageConsumer) + { + int count = 0; + for (Map.Entry entry : droppedMessages.entrySet()) { - timer = dcLatency.computeIfAbsent(dc, k -> Metrics.timer(factory.createMetricName(dc + "-Latency"))); + Verb verb = entry.getKey(); + DroppedForVerb droppedForVerb = entry.getValue(); + + int droppedInternal = droppedForVerb.droppedFromSelf.getAndSet(0); + int droppedCrossNode = droppedForVerb.droppedFromPeer.getAndSet(0); + if (droppedInternal > 0 || droppedCrossNode > 0) + { + messageConsumer.accept(String.format("%s messages were dropped in last %d ms: %d internal and %d cross node." + + " Mean internal dropped latency: %d ms and Mean cross-node dropped latency: %d ms", + verb, + LOG_DROPPED_INTERVAL_IN_MS, + droppedInternal, + droppedCrossNode, + TimeUnit.NANOSECONDS.toMillis((long) droppedForVerb.metrics.internalDroppedLatency.getSnapshot().getMean()), + TimeUnit.NANOSECONDS.toMillis((long) droppedForVerb.metrics.crossNodeDroppedLatency.getSnapshot().getMean()))); + ++count; + } } - timer.update(timeTaken, TimeUnit.MILLISECONDS); - crossNodeLatency.update(timeTaken, TimeUnit.MILLISECONDS); + return count; } - public void addQueueWaitTime(String verb, long timeTaken) + @VisibleForTesting + public void resetDroppedMessages(String scope) { - if (timeTaken < 0) - // the measurement is not accurate, ignore the negative timeTaken - return; - - Timer timer = queueWaitLatency.get(verb); - if (timer == null) + for (Verb verb : droppedMessages.keySet()) { - timer = queueWaitLatency.computeIfAbsent(verb, k -> Metrics.timer(factory.createMetricName(verb + "-WaitLatency"))); + droppedMessages.put(verb, new DroppedForVerb(new DroppedMessageMetrics(metricName -> + new CassandraMetricsRegistry.MetricName("DroppedMessages", metricName, scope) + ))); } - timer.update(timeTaken, TimeUnit.MILLISECONDS); } + } diff --git a/src/java/org/apache/cassandra/metrics/Sampler.java b/src/java/org/apache/cassandra/metrics/Sampler.java index 4bff332eeaab..cfe3f3b24943 100644 --- a/src/java/org/apache/cassandra/metrics/Sampler.java +++ b/src/java/org/apache/cassandra/metrics/Sampler.java @@ -26,8 +26,8 @@ import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.MessagingService.Verb; -import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.utils.MonotonicClock; import com.google.common.annotations.VisibleForTesting; @@ -39,7 +39,7 @@ public enum SamplerType } @VisibleForTesting - Clock clock = Clock.instance; + MonotonicClock clock = MonotonicClock.approxTime; @VisibleForTesting static final ThreadPoolExecutor samplerExecutor = new JMXEnabledThreadPoolExecutor(1, 1, @@ -52,7 +52,7 @@ public enum SamplerType { samplerExecutor.setRejectedExecutionHandler((runnable, executor) -> { - MessagingService.instance().incrementDroppedMessages(Verb._SAMPLE); + MessagingService.instance().metrics.recordSelfDroppedMessage(Verb._SAMPLE); }); } diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index c854c43c7cfc..beb6d2e6a6f9 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -202,7 +202,7 @@ public class TableMetrics /** Time spent waiting for free memtable space, either on- or off-heap */ public final Histogram waitingOnFreeMemtableSpace; - /** Dropped Mutations Count */ + @Deprecated public final Counter droppedMutations; private final MetricNameFactory factory; diff --git a/src/java/org/apache/cassandra/net/IMessageSink.java b/src/java/org/apache/cassandra/net/AcceptVersions.java similarity index 62% rename from src/java/org/apache/cassandra/net/IMessageSink.java rename to src/java/org/apache/cassandra/net/AcceptVersions.java index 090d2c21cd91..61ae0491a18c 100644 --- a/src/java/org/apache/cassandra/net/IMessageSink.java +++ b/src/java/org/apache/cassandra/net/AcceptVersions.java @@ -17,21 +17,26 @@ */ package org.apache.cassandra.net; -import org.apache.cassandra.locator.InetAddressAndPort; - -public interface IMessageSink +/** + * Encapsulates minimum and maximum messaging versions that a node accepts. + */ +class AcceptVersions { - /** - * Allow or drop an outgoing message - * - * @return true if the message is allowed, false if it should be dropped - */ - boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to); + final int min, max; + + AcceptVersions(int min, int max) + { + this.min = min; + this.max = max; + } + + @Override + public boolean equals(Object that) + { + if (!(that instanceof AcceptVersions)) + return false; - /** - * Allow or drop an incoming message - * - * @return true if the message is allowed, false if it should be dropped - */ - boolean allowIncomingMessage(MessageIn message, int id); + return min == ((AcceptVersions) that).min + && max == ((AcceptVersions) that).max; + } } diff --git a/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java new file mode 100644 index 000000000000..163981c901d3 --- /dev/null +++ b/src/java/org/apache/cassandra/net/AsyncChannelOutputPlus.java @@ -0,0 +1,268 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; +import java.util.concurrent.locks.LockSupport; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelPromise; +import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; +import org.apache.cassandra.io.util.DataOutputStreamPlus; + +import static java.lang.Math.max; + +/** + * A {@link DataOutputStreamPlus} that writes ASYNCHRONOUSLY to a Netty Channel. + * + * The close() and flush() methods synchronously wait for pending writes, and will propagate any exceptions + * encountered in writing them to the wire. + * + * The correctness of this class depends on the ChannelPromise we create against a Channel always being completed, + * which appears to be a guarantee provided by Netty so long as the event loop is running. + * + * There are two logical threads accessing the state in this class: the eventLoop of the channel, and the writer + * (the writer thread may change, so long as only one utilises the class at any time). + * Each thread has exclusive write access to certain state in the class, with the other thread only viewing the state, + * simplifying concurrency considerations. + */ +public abstract class AsyncChannelOutputPlus extends BufferedDataOutputStreamPlus +{ + public static class FlushException extends IOException + { + public FlushException(String message) + { + super(message); + } + + public FlushException(String message, Throwable cause) + { + super(message, cause); + } + } + + final Channel channel; + + /** the number of bytes we have begun flushing; updated only by writer */ + private volatile long flushing; + /** the number of bytes we have finished flushing, successfully or otherwise; updated only by eventLoop */ + private volatile long flushed; + /** the number of bytes we have finished flushing to the network; updated only by eventLoop */ + private long flushedToNetwork; + /** any error that has been thrown during a flush; updated only by eventLoop */ + private volatile Throwable flushFailed; + + /** + * state for pausing until flushing has caught up - store the number of bytes we need to be flushed before + * we should be signalled, and store ourselves in {@link #waiting}; once the flushing thread exceeds this many + * total bytes flushed, any Thread stored in waiting will be signalled. + * + * This works exactly like using a WaitQueue, except that we only need to manage a single waiting thread. + */ + private volatile long signalWhenFlushed; // updated only by writer + private volatile Thread waiting; // updated only by writer + + public AsyncChannelOutputPlus(Channel channel) + { + super(null, null); + this.channel = channel; + } + + /** + * Create a ChannelPromise for a flush of the given size. + *

+ * This method will not return until the write is permitted by the provided watermarks and in flight bytes, + * and on its completion will mark the requested bytes flushed. + *

+ * If this method returns normally, the ChannelPromise MUST be writtenAndFlushed, or else completed exceptionally. + */ + protected ChannelPromise beginFlush(int byteCount, int lowWaterMark, int highWaterMark) throws IOException + { + waitForSpace(byteCount, lowWaterMark, highWaterMark); + + return AsyncChannelPromise.withListener(channel, future -> { + if (future.isSuccess() && null == flushFailed) + { + flushedToNetwork += byteCount; + releaseSpace(byteCount); + } + else if (null == flushFailed) + { + Throwable cause = future.cause(); + if (cause == null) + { + cause = new FlushException("Flush failed for unknown reason"); + cause.fillInStackTrace(); + } + flushFailed = cause; + releaseSpace(flushing - flushed); + } + else + { + assert flushing == flushed; + } + }); + } + + /** + * Imposes our lowWaterMark/highWaterMark constraints, and propagates any exceptions thrown by prior flushes. + * + * If we currently have lowWaterMark or fewer bytes flushing, we are good to go. + * If our new write will not take us over our highWaterMark, we are good to go. + * Otherwise we wait until either of these conditions are met. + * + * This may only be invoked by the writer thread, never by the eventLoop. + * + * @throws IOException if a prior asynchronous flush failed + */ + private void waitForSpace(int bytesToWrite, int lowWaterMark, int highWaterMark) throws IOException + { + // decide when we would be willing to carry on writing + // we are always writable if we have lowWaterMark or fewer bytes, no matter how many bytes we are flushing + // our callers should not be supplying more than (highWaterMark - lowWaterMark) bytes, but we must work correctly if they do + int wakeUpWhenFlushing = highWaterMark - bytesToWrite; + waitUntilFlushed(max(lowWaterMark, wakeUpWhenFlushing), lowWaterMark); + flushing += bytesToWrite; + } + + /** + * Implementation of waitForSpace, which calculates what flushed points we need to wait for, + * parks if necessary and propagates flush failures. + * + * This may only be invoked by the writer thread, never by the eventLoop. + */ + void waitUntilFlushed(int wakeUpWhenExcessBytesWritten, int signalWhenExcessBytesWritten) throws IOException + { + // we assume that we are happy to wake up at least as early as we will be signalled; otherwise we will never exit + assert signalWhenExcessBytesWritten <= wakeUpWhenExcessBytesWritten; + // flushing shouldn't change during this method invocation, so our calculations for signal and flushed are consistent + long wakeUpWhenFlushed = flushing - wakeUpWhenExcessBytesWritten; + if (flushed < wakeUpWhenFlushed) + parkUntilFlushed(wakeUpWhenFlushed, flushing - signalWhenExcessBytesWritten); + propagateFailedFlush(); + } + + /** + * Utility method for waitUntilFlushed, which actually parks the current thread until the necessary + * number of bytes have been flushed + * + * This may only be invoked by the writer thread, never by the eventLoop. + */ + protected void parkUntilFlushed(long wakeUpWhenFlushed, long signalWhenFlushed) + { + assert wakeUpWhenFlushed <= signalWhenFlushed; + assert waiting == null; + this.waiting = Thread.currentThread(); + this.signalWhenFlushed = signalWhenFlushed; + + while (flushed < wakeUpWhenFlushed) + LockSupport.park(); + waiting = null; + } + + /** + * Update our flushed count, and signal any waiters. + * + * This may only be invoked by the eventLoop, never by the writer thread. + */ + protected void releaseSpace(long bytesFlushed) + { + long newFlushed = flushed + bytesFlushed; + flushed = newFlushed; + + Thread thread = waiting; + if (thread != null && signalWhenFlushed <= newFlushed) + LockSupport.unpark(thread); + } + + private void propagateFailedFlush() throws IOException + { + Throwable t = flushFailed; + if (t != null) + { + if (SocketFactory.isCausedByConnectionReset(t)) + throw new FlushException("The channel this output stream was writing to has been closed", t); + throw new FlushException("This output stream is in an unsafe state after an asynchronous flush failed", t); + } + } + + @Override + abstract protected void doFlush(int count) throws IOException; + + abstract public long position(); + + public long flushed() + { + // external flushed (that which has had flush() invoked implicitly or otherwise) == internal flushing + return flushing; + } + + public long flushedToNetwork() + { + return flushedToNetwork; + } + + /** + * Perform an asynchronous flush, then waits until all outstanding flushes have completed + * + * @throws IOException if any flush fails + */ + @Override + public void flush() throws IOException + { + doFlush(0); + waitUntilFlushed(0, 0); + } + + /** + * Flush any remaining writes, and release any buffers. + * + * The channel is not closed, as it is assumed to be managed externally. + * + * WARNING: This method requires mutual exclusivity with all other producer methods to run safely. + * It should only be invoked by the owning thread, never the eventLoop; the eventLoop should propagate + * errors to {@link #flushFailed}, which will propagate them to the producer thread no later than its + * final invocation to {@link #close()} or {@link #flush()} (that must not be followed by any further writes). + */ + @Override + public void close() throws IOException + { + try + { + flush(); + } + finally + { + discard(); + } + } + + /** + * Discard any buffered data, and the buffers that contain it. + * May be invoked instead of {@link #close()} if we terminate exceptionally. + */ + public abstract void discard(); + + @Override + protected WritableByteChannel newDefaultChannel() + { + throw new UnsupportedOperationException(); + } + +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/AsyncChannelPromise.java b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java new file mode 100644 index 000000000000..d2c9d0bfb6de --- /dev/null +++ b/src/java/org/apache/cassandra/net/AsyncChannelPromise.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPromise; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; + +/** + * See {@link AsyncPromise} and {@link io.netty.channel.ChannelPromise} + * + * This class is all boiler plate, just ensuring we return ourselves and invoke the correct Promise method. + */ +public class AsyncChannelPromise extends AsyncPromise implements ChannelPromise +{ + private final Channel channel; + + @SuppressWarnings("unused") + public AsyncChannelPromise(Channel channel) + { + super(channel.eventLoop()); + this.channel = channel; + } + + AsyncChannelPromise(Channel channel, GenericFutureListener> listener) + { + super(channel.eventLoop(), listener); + this.channel = channel; + } + + public static AsyncChannelPromise withListener(ChannelHandlerContext context, GenericFutureListener> listener) + { + return withListener(context.channel(), listener); + } + + public static AsyncChannelPromise withListener(Channel channel, GenericFutureListener> listener) + { + return new AsyncChannelPromise(channel, listener); + } + + public static ChannelFuture writeAndFlush(ChannelHandlerContext context, Object message, GenericFutureListener> listener) + { + return context.writeAndFlush(message, withListener(context.channel(), listener)); + } + + public static ChannelFuture writeAndFlush(Channel channel, Object message, GenericFutureListener> listener) + { + return channel.writeAndFlush(message, withListener(channel, listener)); + } + + public static ChannelFuture writeAndFlush(ChannelHandlerContext context, Object message) + { + return context.writeAndFlush(message, new AsyncChannelPromise(context.channel())); + } + + public static ChannelFuture writeAndFlush(Channel channel, Object message) + { + return channel.writeAndFlush(message, new AsyncChannelPromise(channel)); + } + + public Channel channel() + { + return channel; + } + + public boolean isVoid() + { + return false; + } + + public ChannelPromise setSuccess() + { + return setSuccess(null); + } + + public ChannelPromise setSuccess(Void v) + { + super.setSuccess(v); + return this; + } + + public boolean trySuccess() + { + return trySuccess(null); + } + + public ChannelPromise setFailure(Throwable throwable) + { + super.setFailure(throwable); + return this; + } + + public ChannelPromise sync() throws InterruptedException + { + super.sync(); + return this; + } + + public ChannelPromise syncUninterruptibly() + { + super.syncUninterruptibly(); + return this; + } + + public ChannelPromise await() throws InterruptedException + { + super.await(); + return this; + } + + public ChannelPromise awaitUninterruptibly() + { + super.awaitUninterruptibly(); + return this; + } + + public ChannelPromise addListener(GenericFutureListener> listener) + { + super.addListener(listener); + return this; + } + + public ChannelPromise addListeners(GenericFutureListener>... listeners) + { + super.addListeners(listeners); + return this; + } + + public ChannelPromise removeListener(GenericFutureListener> listener) + { + super.removeListener(listener); + return this; + } + + public ChannelPromise removeListeners(GenericFutureListener>... listeners) + { + super.removeListeners(listeners); + return this; + } + + public ChannelPromise unvoid() + { + return this; + } +} diff --git a/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java new file mode 100644 index 000000000000..8ef0a8f0e631 --- /dev/null +++ b/src/java/org/apache/cassandra/net/AsyncMessageOutputPlus.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.channels.ClosedChannelException; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelPromise; +import io.netty.channel.WriteBufferWaterMark; +import org.apache.cassandra.io.util.DataOutputStreamPlus; + +/** + * A {@link DataOutputStreamPlus} that writes ASYNCHRONOUSLY to a Netty Channel. + * + * Intended as single use, to write one (large) message. + * + * The close() and flush() methods synchronously wait for pending writes, and will propagate any exceptions + * encountered in writing them to the wire. + * + * The correctness of this class depends on the ChannelPromise we create against a Channel always being completed, + * which appears to be a guarantee provided by Netty so long as the event loop is running. + */ +public class AsyncMessageOutputPlus extends AsyncChannelOutputPlus +{ + /** + * the maximum {@link #highWaterMark} and minimum {@link #lowWaterMark} number of bytes we have flushing + * during which we should still be writing to the channel. + * + * i.e., if we are at or below the {@link #lowWaterMark} we should definitely start writing again; + * if we are at or above the {@link #highWaterMark} we should definitely stop writing; + * if we are inbetween, it is OK to either write or not write + * + * note that we consider the bytes we are about to write to our high water mark, but not our low. + * i.e., we will not begin a write that would take us over our high water mark, unless not doing so would + * take us below our low water mark. + * + * This is somewhat arbitrary accounting, and a meaningless distinction for flushes of a consistent size. + */ + @SuppressWarnings("JavaDoc") + private final int highWaterMark; + private final int lowWaterMark; + private final int bufferSize; + private final int messageSize; + private boolean closing; + + private final FrameEncoder.PayloadAllocator payloadAllocator; + private volatile FrameEncoder.Payload payload; + + AsyncMessageOutputPlus(Channel channel, int bufferSize, int messageSize, FrameEncoder.PayloadAllocator payloadAllocator) + { + super(channel); + WriteBufferWaterMark waterMark = channel.config().getWriteBufferWaterMark(); + this.lowWaterMark = waterMark.low(); + this.highWaterMark = waterMark.high(); + this.messageSize = messageSize; + this.bufferSize = Math.min(messageSize, bufferSize); + this.payloadAllocator = payloadAllocator; + allocateBuffer(); + } + + private void allocateBuffer() + { + payload = payloadAllocator.allocate(false, bufferSize); + buffer = payload.buffer; + } + + @Override + protected void doFlush(int count) throws IOException + { + if (!channel.isOpen()) + throw new ClosedChannelException(); + + // flush the current backing write buffer only if there's any pending data + FrameEncoder.Payload flush = payload; + int byteCount = flush.length(); + if (byteCount == 0) + return; + + if (byteCount + flushed() > (closing ? messageSize : messageSize - 1)) + throw new InvalidSerializedSizeException(messageSize, byteCount + flushed()); + + flush.finish(); + ChannelPromise promise = beginFlush(byteCount, lowWaterMark, highWaterMark); + channel.writeAndFlush(flush, promise); + allocateBuffer(); + } + + public void close() throws IOException + { + closing = true; + if (flushed() == 0 && payload != null) + payload.setSelfContained(true); + super.close(); + } + + public long position() + { + return flushed() + payload.length(); + } + + /** + * Discard any buffered data, and the buffers that contain it. + * May be invoked instead of {@link #close()} if we terminate exceptionally. + */ + public void discard() + { + if (payload != null) + { + payload.release(); + payload = null; + buffer = null; + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/AsyncOneResponse.java b/src/java/org/apache/cassandra/net/AsyncOneResponse.java index 3fe0a2aebbd6..ba83c84c91cb 100644 --- a/src/java/org/apache/cassandra/net/AsyncOneResponse.java +++ b/src/java/org/apache/cassandra/net/AsyncOneResponse.java @@ -17,54 +17,31 @@ */ package org.apache.cassandra.net; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.AbstractFuture; + +import io.netty.util.concurrent.ImmediateEventExecutor; /** * A callback specialized for returning a value from a single target; that is, this is for messages * that we only send to one recipient. */ -public class AsyncOneResponse extends AbstractFuture implements IAsyncCallback +public class AsyncOneResponse extends AsyncPromise implements RequestCallback { - private final long start = System.nanoTime(); - - public void response(MessageIn response) - { - set(response.payload); - } - - public boolean isLatencyForSnitch() + public AsyncOneResponse() { - return false; + super(ImmediateEventExecutor.INSTANCE); } - @Override - public T get(long timeout, TimeUnit unit) throws TimeoutException + public void onResponse(Message response) { - long adjustedTimeout = unit.toNanos(timeout) - (System.nanoTime() - start); - if (adjustedTimeout <= 0) - { - throw new TimeoutException("Operation timed out."); - } - try - { - return super.get(adjustedTimeout, TimeUnit.NANOSECONDS); - } - catch (InterruptedException | ExecutionException e) - { - throw new AssertionError(e); - } + setSuccess(response.payload); } @VisibleForTesting public static AsyncOneResponse immediate(T value) { AsyncOneResponse response = new AsyncOneResponse<>(); - response.set(value); + response.setSuccess(value); return response; } } diff --git a/src/java/org/apache/cassandra/net/AsyncPromise.java b/src/java/org/apache/cassandra/net/AsyncPromise.java new file mode 100644 index 000000000000..36bc304a6b08 --- /dev/null +++ b/src/java/org/apache/cassandra/net/AsyncPromise.java @@ -0,0 +1,488 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.util.concurrent.EventExecutor; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import io.netty.util.concurrent.Promise; +import io.netty.util.internal.PlatformDependent; +import io.netty.util.internal.ThrowableUtil; +import org.apache.cassandra.utils.concurrent.WaitQueue; + +import static java.util.concurrent.atomic.AtomicReferenceFieldUpdater.*; + +/** + * Netty's DefaultPromise uses a mutex to coordinate notifiers AND waiters between the eventLoop and the other threads. + * Since we register cross-thread listeners, this has the potential to block internode messaging for an unknown + * number of threads for an unknown period of time, if we are unlucky with the scheduler (which will certainly + * happen, just with some unknown but low periodicity) + * + * At the same time, we manage some other efficiencies: + * - We save some space when registering listeners, especially if there is only one listener, as we perform no + * extra allocations in this case. + * - We permit efficient initial state declaration, avoiding unnecessary CAS or lock acquisitions when mutating + * a Promise we are ourselves constructing (and can easily add more; only those we use have been added) + * + * We can also make some guarantees about our behaviour here, although we primarily mirror Netty. + * Specifically, we can guarantee that notifiers are always invoked in the order they are added (which may be true + * for netty, but was unclear and is not declared). This is useful for ensuring the correctness of some of our + * behaviours in OutboundConnection without having to jump through extra hoops. + * + * The implementation loosely follows that of Netty's DefaultPromise, with some slight changes; notably that we have + * no synchronisation on our listeners, instead using a CoW list that is cleared each time we notify listeners. + * + * We handle special values slightly differently. We do not use a special value for null, instead using + * a special value to indicate the result has not been set yet. This means that once isSuccess() holds, + * the result must be a correctly typed object (modulo generics pitfalls). + * All special values are also instances of FailureHolder, which simplifies a number of the logical conditions. + * + * @param + */ +public class AsyncPromise implements Promise +{ + private static final Logger logger = LoggerFactory.getLogger(AsyncPromise.class); + + private final EventExecutor executor; + private volatile Object result; + private volatile GenericFutureListener> listeners; + private volatile WaitQueue waiting; + private static final AtomicReferenceFieldUpdater resultUpdater = newUpdater(AsyncPromise.class, Object.class, "result"); + private static final AtomicReferenceFieldUpdater listenersUpdater = newUpdater(AsyncPromise.class, GenericFutureListener.class, "listeners"); + private static final AtomicReferenceFieldUpdater waitingUpdater = newUpdater(AsyncPromise.class, WaitQueue.class, "waiting"); + + private static final FailureHolder UNSET = new FailureHolder(null); + private static final FailureHolder UNCANCELLABLE = new FailureHolder(null); + private static final FailureHolder CANCELLED = new FailureHolder(ThrowableUtil.unknownStackTrace(new CancellationException(), AsyncPromise.class, "cancel(...)")); + + private static final DeferredGenericFutureListener NOTIFYING = future -> {}; + private static interface DeferredGenericFutureListener> extends GenericFutureListener {} + + private static final class FailureHolder + { + final Throwable cause; + private FailureHolder(Throwable cause) + { + this.cause = cause; + } + } + + public AsyncPromise(EventExecutor executor) + { + this(executor, UNSET); + } + + private AsyncPromise(EventExecutor executor, FailureHolder initialState) + { + this.executor = executor; + this.result = initialState; + } + + public AsyncPromise(EventExecutor executor, GenericFutureListener> listener) + { + this(executor); + this.listeners = listener; + } + + AsyncPromise(EventExecutor executor, FailureHolder initialState, GenericFutureListener> listener) + { + this(executor, initialState); + this.listeners = listener; + } + + public static AsyncPromise uncancellable(EventExecutor executor) + { + return new AsyncPromise<>(executor, UNCANCELLABLE); + } + + public static AsyncPromise uncancellable(EventExecutor executor, GenericFutureListener> listener) + { + return new AsyncPromise<>(executor, UNCANCELLABLE); + } + + public Promise setSuccess(V v) + { + if (!trySuccess(v)) + throw new IllegalStateException("complete already: " + this); + return this; + } + + public Promise setFailure(Throwable throwable) + { + if (!tryFailure(throwable)) + throw new IllegalStateException("complete already: " + this); + return this; + } + + public boolean trySuccess(V v) + { + return trySet(v); + } + + public boolean tryFailure(Throwable throwable) + { + return trySet(new FailureHolder(throwable)); + } + + public boolean setUncancellable() + { + if (trySet(UNCANCELLABLE)) + return true; + return result == UNCANCELLABLE; + } + + public boolean cancel(boolean b) + { + return trySet(CANCELLED); + } + + /** + * Shared implementation of various promise completion methods. + * Updates the result if it is possible to do so, returning success/failure. + * + * If the promise is UNSET the new value will succeed; + * if it is UNCANCELLABLE it will succeed only if the new value is not CANCELLED + * otherwise it will fail, as isDone() is implied + * + * If the update succeeds, and the new state implies isDone(), any listeners and waiters will be notified + */ + private boolean trySet(Object v) + { + while (true) + { + Object current = result; + if (isDone(current) || (current == UNCANCELLABLE && v == CANCELLED)) + return false; + if (resultUpdater.compareAndSet(this, current, v)) + { + if (v != UNCANCELLABLE) + { + notifyListeners(); + notifyWaiters(); + } + return true; + } + } + } + + public boolean isSuccess() + { + return isSuccess(result); + } + + private static boolean isSuccess(Object result) + { + return !(result instanceof FailureHolder); + } + + public boolean isCancelled() + { + return isCancelled(result); + } + + private static boolean isCancelled(Object result) + { + return result == CANCELLED; + } + + public boolean isDone() + { + return isDone(result); + } + + private static boolean isDone(Object result) + { + return result != UNSET && result != UNCANCELLABLE; + } + + public boolean isCancellable() + { + Object result = this.result; + return result == UNSET; + } + + public Throwable cause() + { + Object result = this.result; + if (result instanceof FailureHolder) + return ((FailureHolder) result).cause; + return null; + } + + /** + * if isSuccess(), returns the value, otherwise returns null + */ + @SuppressWarnings("unchecked") + public V getNow() + { + Object result = this.result; + if (isSuccess(result)) + return (V) result; + return null; + } + + public V get() throws InterruptedException, ExecutionException + { + await(); + return getWhenDone(); + } + + public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException + { + if (!await(timeout, unit)) + throw new TimeoutException(); + return getWhenDone(); + } + + /** + * Shared implementation of get() after suitable await(); assumes isDone(), and returns + * either the success result or throws the suitable exception under failure + */ + @SuppressWarnings("unchecked") + private V getWhenDone() throws ExecutionException + { + Object result = this.result; + if (isSuccess(result)) + return (V) result; + if (result == CANCELLED) + throw new CancellationException(); + throw new ExecutionException(((FailureHolder) result).cause); + } + + /** + * waits for completion; in case of failure rethrows the original exception without a new wrapping exception + * so may cause problems for reporting stack traces + */ + public Promise sync() throws InterruptedException + { + await(); + rethrowIfFailed(); + return this; + } + + /** + * waits for completion; in case of failure rethrows the original exception without a new wrapping exception + * so may cause problems for reporting stack traces + */ + public Promise syncUninterruptibly() + { + awaitUninterruptibly(); + rethrowIfFailed(); + return this; + } + + private void rethrowIfFailed() + { + Throwable cause = this.cause(); + if (cause != null) + { + PlatformDependent.throwException(cause); + } + } + + public Promise addListener(GenericFutureListener> listener) + { + listenersUpdater.accumulateAndGet(this, listener, AsyncPromise::appendListener); + if (isDone()) + notifyListeners(); + return this; + } + + public Promise addListeners(GenericFutureListener> ... listeners) + { + // this could be more efficient if we cared, but we do not + return addListener(future -> { + for (GenericFutureListener> listener : listeners) + AsyncPromise.invokeListener((GenericFutureListener>)listener, future); + }); + } + + public Promise removeListener(GenericFutureListener> listener) + { + throw new UnsupportedOperationException(); + } + + public Promise removeListeners(GenericFutureListener> ... listeners) + { + throw new UnsupportedOperationException(); + } + + @SuppressWarnings("unchecked") + private void notifyListeners() + { + if (!executor.inEventLoop()) + { + // submit this method, to guarantee we invoke in the submitted order + executor.execute(this::notifyListeners); + return; + } + + if (listeners == null || listeners instanceof DeferredGenericFutureListener) + return; // either no listeners, or we are already notifying listeners, so we'll get to the new one when ready + + // first run our notifiers + while (true) + { + GenericFutureListener listeners = listenersUpdater.getAndSet(this, NOTIFYING); + if (listeners != null) + invokeListener(listeners, this); + + if (listenersUpdater.compareAndSet(this, NOTIFYING, null)) + return; + } + } + + private static > void invokeListener(GenericFutureListener listener, F future) + { + try + { + listener.operationComplete(future); + } + catch (Throwable t) + { + logger.error("Failed to invoke listener {} to {}", listener, future, t); + } + } + + private static > GenericFutureListener appendListener(GenericFutureListener prevListener, GenericFutureListener newListener) + { + GenericFutureListener result = newListener; + + if (prevListener != null && prevListener != NOTIFYING) + { + result = future -> { + invokeListener(prevListener, future); + // we will wrap the outer invocation with invokeListener, so no need to do it here too + newListener.operationComplete(future); + }; + } + + if (prevListener instanceof DeferredGenericFutureListener) + { + GenericFutureListener wrap = result; + result = (DeferredGenericFutureListener) wrap::operationComplete; + } + + return result; + } + + public Promise await() throws InterruptedException + { + await(0L, (signal, nanos) -> { signal.await(); return true; } ); + return this; + } + + public Promise awaitUninterruptibly() + { + await(0L, (signal, nanos) -> { signal.awaitUninterruptibly(); return true; } ); + return this; + } + + public boolean await(long timeout, TimeUnit unit) throws InterruptedException + { + return await(unit.toNanos(timeout), + (signal, nanos) -> signal.awaitUntil(nanos + System.nanoTime())); + } + + public boolean await(long timeoutMillis) throws InterruptedException + { + return await(timeoutMillis, TimeUnit.MILLISECONDS); + } + + public boolean awaitUninterruptibly(long timeout, TimeUnit unit) + { + return await(unit.toNanos(timeout), + (signal, nanos) -> signal.awaitUntilUninterruptibly(nanos + System.nanoTime())); + } + + public boolean awaitUninterruptibly(long timeoutMillis) + { + return awaitUninterruptibly(timeoutMillis, TimeUnit.MILLISECONDS); + } + + interface Awaiter + { + boolean await(WaitQueue.Signal value, long nanos) throws T; + } + + /** + * A clean way to implement each variant of await using lambdas; we permit a nanos parameter + * so that we can implement this without any unnecessary lambda allocations, although not + * all implementations need the nanos parameter (i.e. those that wait indefinitely) + */ + private boolean await(long nanos, Awaiter awaiter) throws T + { + if (isDone()) + return true; + + WaitQueue.Signal await = registerToWait(); + if (null != await) + return awaiter.await(await, nanos); + + return true; + } + + /** + * Register a signal that will be notified when the promise is completed; + * if the promise becomes completed before this signal is registered, null is returned + */ + private WaitQueue.Signal registerToWait() + { + WaitQueue waiting = this.waiting; + if (waiting == null && !waitingUpdater.compareAndSet(this, null, waiting = new WaitQueue())) + waiting = this.waiting; + assert waiting != null; + + WaitQueue.Signal signal = waiting.register(); + if (!isDone()) + return signal; + signal.cancel(); + return null; + } + + private void notifyWaiters() + { + WaitQueue waiting = this.waiting; + if (waiting != null) + waiting.signalAll(); + } + + public String toString() + { + Object result = this.result; + if (isSuccess(result)) + return "(success: " + result + ')'; + if (result == UNCANCELLABLE) + return "(uncancellable)"; + if (result == CANCELLED) + return "(cancelled)"; + if (isDone(result)) + return "(failure: " + ((FailureHolder) result).cause + ')'; + return "(incomplete)"; + } +} diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java new file mode 100644 index 000000000000..84fb8ac167e2 --- /dev/null +++ b/src/java/org/apache/cassandra/net/AsyncStreamingInputPlus.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.apache.cassandra.io.util.RebufferingInputStream; + +// TODO: rewrite +public class AsyncStreamingInputPlus extends RebufferingInputStream +{ + public static class InputTimeoutException extends IOException + { + } + + private static final long DEFAULT_REBUFFER_BLOCK_IN_MILLIS = TimeUnit.MINUTES.toMillis(3); + + private final Channel channel; + + /** + * The parent, or owning, buffer of the current buffer being read from ({@link super#buffer}). + */ + private ByteBuf currentBuf; + + private final BlockingQueue queue; + + private final long rebufferTimeoutNanos; + + private volatile boolean isClosed; + + public AsyncStreamingInputPlus(Channel channel) + { + this(channel, DEFAULT_REBUFFER_BLOCK_IN_MILLIS, TimeUnit.MILLISECONDS); + } + + AsyncStreamingInputPlus(Channel channel, long rebufferTimeout, TimeUnit rebufferTimeoutUnit) + { + super(Unpooled.EMPTY_BUFFER.nioBuffer()); + currentBuf = Unpooled.EMPTY_BUFFER; + + queue = new LinkedBlockingQueue<>(); + rebufferTimeoutNanos = rebufferTimeoutUnit.toNanos(rebufferTimeout); + + this.channel = channel; + channel.config().setAutoRead(false); + } + + /** + * Append a {@link ByteBuf} to the end of the einternal queue. + * + * Note: it's expected this method is invoked on the netty event loop. + */ + public boolean append(ByteBuf buf) throws IllegalStateException + { + if (isClosed) return false; + + queue.add(buf); + + /* + * it's possible for append() to race with close(), so we need to ensure + * that the bytebuf gets released in that scenario + */ + if (isClosed) + while ((buf = queue.poll()) != null) + buf.release(); + + return true; + } + + /** + * {@inheritDoc} + * + * Release open buffers and poll the {@link #queue} for more data. + *

+ * This is best, and more or less expected, to be invoked on a consuming thread (not the event loop) + * becasue if we block on the queue we can't fill it on the event loop (as that's where the buffers are coming from). + * + * @throws EOFException when no further reading from this instance should occur. Implies this instance is closed. + * @throws InputTimeoutException when no new buffers arrive for reading before + * the {@link #rebufferTimeoutNanos} elapses while blocking. It's then not safe to reuse this instance again. + */ + @Override + protected void reBuffer() throws EOFException, InputTimeoutException + { + if (queue.isEmpty()) + channel.read(); + + currentBuf.release(); + currentBuf = null; + buffer = null; + + ByteBuf next = null; + try + { + next = queue.poll(rebufferTimeoutNanos, TimeUnit.NANOSECONDS); + } + catch (InterruptedException ie) + { + // nop + } + + if (null == next) + throw new InputTimeoutException(); + + if (next == Unpooled.EMPTY_BUFFER) // Unpooled.EMPTY_BUFFER is the indicator that the input is closed + throw new EOFException(); + + currentBuf = next; + buffer = next.nioBuffer(); + } + + public interface Consumer + { + int accept(ByteBuffer buffer) throws IOException; + } + + /** + * Consumes bytes in the stream until the given length + */ + public void consume(Consumer consumer, long length) throws IOException + { + while (length > 0) + { + if (!buffer.hasRemaining()) + reBuffer(); + + final int position = buffer.position(); + final int limit = buffer.limit(); + + buffer.limit(position + (int) Math.min(length, limit - position)); + try + { + int copied = consumer.accept(buffer); + buffer.position(position + copied); + length -= copied; + } + finally + { + buffer.limit(limit); + } + } + } + + /** + * {@inheritDoc} + * + * As long as this method is invoked on the consuming thread the returned value will be accurate. + */ + @VisibleForTesting + public int unsafeAvailable() + { + long count = buffer != null ? buffer.remaining() : 0; + for (ByteBuf buf : queue) + count += buf.readableBytes(); + + return Ints.checkedCast(count); + } + + // TODO:JEB add docs + // TL;DR if there's no Bufs open anywhere here, issue a channle read to try and grab data. + public void maybeIssueRead() + { + if (isEmpty()) + channel.read(); + } + + public boolean isEmpty() + { + return queue.isEmpty() && (buffer == null || !buffer.hasRemaining()); + } + + /** + * {@inheritDoc} + * + * Note: This should invoked on the consuming thread. + */ + @Override + public void close() + { + if (isClosed) + return; + + if (currentBuf != null) + { + currentBuf.release(); + currentBuf = null; + buffer = null; + } + + while (true) + { + try + { + ByteBuf buf = queue.poll(Long.MAX_VALUE, TimeUnit.NANOSECONDS); + if (buf == Unpooled.EMPTY_BUFFER) + break; + else + buf.release(); + } + catch (InterruptedException e) + { + // + } + } + + isClosed = true; + } + + /** + * Mark this stream as closed, but do not release any of the resources. + * + * Note: this is best to be called from the producer thread. + */ + public void requestClosure() + { + queue.add(Unpooled.EMPTY_BUFFER); + } + + // TODO: let's remove this like we did for AsyncChannelOutputPlus + public ByteBufAllocator getAllocator() + { + return channel.alloc(); + } +} diff --git a/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java new file mode 100644 index 000000000000..a52070e365fb --- /dev/null +++ b/src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.FileChannel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelPromise; +import io.netty.channel.WriteBufferWaterMark; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.util.DataOutputStreamPlus; +import org.apache.cassandra.net.SharedDefaultFileRegion.SharedFileChannel; +import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter; +import org.apache.cassandra.utils.memory.BufferPool; + +import static java.lang.Math.min; + +/** + * A {@link DataOutputStreamPlus} that writes ASYNCHRONOUSLY to a Netty Channel. + * + * The close() and flush() methods synchronously wait for pending writes, and will propagate any exceptions + * encountered in writing them to the wire. + * + * The correctness of this class depends on the ChannelPromise we create against a Channel always being completed, + * which appears to be a guarantee provided by Netty so long as the event loop is running. + */ +public class AsyncStreamingOutputPlus extends AsyncChannelOutputPlus +{ + private static final Logger logger = LoggerFactory.getLogger(AsyncStreamingOutputPlus.class); + + final int defaultLowWaterMark; + final int defaultHighWaterMark; + + public AsyncStreamingOutputPlus(Channel channel) + { + super(channel); + WriteBufferWaterMark waterMark = channel.config().getWriteBufferWaterMark(); + this.defaultLowWaterMark = waterMark.low(); + this.defaultHighWaterMark = waterMark.high(); + allocateBuffer(); + } + + private void allocateBuffer() + { + // this buffer is only used for small quantities of data + buffer = BufferPool.getAtLeast(8 << 10, BufferType.OFF_HEAP); + } + + @Override + protected void doFlush(int count) throws IOException + { + if (!channel.isOpen()) + throw new ClosedChannelException(); + + // flush the current backing write buffer only if there's any pending data + ByteBuffer flush = buffer; + if (flush.position() == 0) + return; + + flush.flip(); + int byteCount = flush.limit(); + ChannelPromise promise = beginFlush(byteCount, 0, Integer.MAX_VALUE); + channel.writeAndFlush(GlobalBufferPoolAllocator.wrap(flush), promise); + allocateBuffer(); + } + + public long position() + { + return flushed() + buffer.position(); + } + + public interface BufferSupplier + { + /** + * Request a buffer with at least the given capacity. + * This method may only be invoked once, and the lifetime of buffer it returns will be managed + * by the AsyncChannelOutputPlus it was created for. + */ + ByteBuffer get(int capacity) throws IOException; + } + + public interface Write + { + /** + * Write to a buffer, and flush its contents to the channel. + *

+ * The lifetime of the buffer will be managed by the AsyncChannelOutputPlus you issue this Write to. + * If the method exits successfully, the contents of the buffer will be written to the channel, otherwise + * the buffer will be cleaned and the exception propagated to the caller. + */ + void write(BufferSupplier supplier) throws IOException; + } + + /** + * Provide a lambda that can request a buffer of suitable size, then fill the buffer and have + * that buffer written and flushed to the underlying channel, without having to handle buffer + * allocation, lifetime or cleanup, including in case of exceptions. + *

+ * Any exception thrown by the Write will be propagated to the caller, after any buffer is cleaned up. + */ + public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOException + { + doFlush(0); + class Holder + { + ChannelPromise promise; + ByteBuffer buffer; + } + Holder holder = new Holder(); + + try + { + write.write(size -> { + if (holder.buffer != null) + throw new IllegalStateException("Can only allocate one ByteBuffer"); + limiter.acquire(size); + holder.promise = beginFlush(size, defaultLowWaterMark, defaultHighWaterMark); + holder.buffer = BufferPool.get(size); + return holder.buffer; + }); + } + catch (Throwable t) + { + // we don't currently support cancelling the flush, but at this point we are recoverable if we want + if (holder.buffer != null) + BufferPool.put(holder.buffer); + if (holder.promise != null) + holder.promise.tryFailure(t); + throw t; + } + + ByteBuffer buffer = holder.buffer; + BufferPool.putUnusedPortion(buffer); + + int length = buffer.limit(); + channel.writeAndFlush(GlobalBufferPoolAllocator.wrap(buffer), holder.promise); + return length; + } + + /** + *

+ * Writes all data in file channel to stream, 1MiB at a time, with at most 2MiB in flight at once. + * This method takes ownership of the provided {@code FileChannel}. + *

+ * WARNING: this method blocks only for permission to write to the netty channel; it exits before + * the write is flushed to the network. + */ + public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException + { + // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk, + // the data is never brought in process and is gated by the wire anyway + return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20); + } + + public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter, int batchSize, int lowWaterMark, int highWaterMark) throws IOException + { + final long length = file.size(); + long bytesTransferred = 0; + + final SharedFileChannel sharedFile = SharedDefaultFileRegion.share(file); + try + { + while (bytesTransferred < length) + { + int toWrite = (int) min(batchSize, length - bytesTransferred); + + limiter.acquire(toWrite); + ChannelPromise promise = beginFlush(toWrite, lowWaterMark, highWaterMark); + + SharedDefaultFileRegion fileRegion = new SharedDefaultFileRegion(sharedFile, bytesTransferred, toWrite); + channel.writeAndFlush(fileRegion, promise); + + if (logger.isTraceEnabled()) + logger.trace("Writing {} bytes at position {} of {}", toWrite, bytesTransferred, length); + bytesTransferred += toWrite; + } + + return bytesTransferred; + } + finally + { + sharedFile.release(); + } + } + + /** + * Discard any buffered data, and the buffers that contain it. + * May be invoked instead of {@link #close()} if we terminate exceptionally. + */ + public void discard() + { + if (buffer != null) + { + BufferPool.put(buffer); + buffer = null; + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/BackPressureState.java b/src/java/org/apache/cassandra/net/BackPressureState.java index 886c075468b8..de19bf301f57 100644 --- a/src/java/org/apache/cassandra/net/BackPressureState.java +++ b/src/java/org/apache/cassandra/net/BackPressureState.java @@ -27,7 +27,7 @@ public interface BackPressureState /** * Called when a message is sent to a replica. */ - void onMessageSent(MessageOut message); + void onMessageSent(Message message); /** * Called when a response is received from a replica. diff --git a/src/java/org/apache/cassandra/net/BufferPoolAllocator.java b/src/java/org/apache/cassandra/net/BufferPoolAllocator.java new file mode 100644 index 000000000000..8782c030693b --- /dev/null +++ b/src/java/org/apache/cassandra/net/BufferPoolAllocator.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; + +import io.netty.buffer.AbstractByteBufAllocator; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.buffer.UnpooledUnsafeDirectByteBuf; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.memory.BufferPool; + +/** + * A trivial wrapper around BufferPool for integrating with Netty, but retaining ownership of pooling behaviour + * that is integrated into Cassandra's other pooling. + */ +abstract class BufferPoolAllocator extends AbstractByteBufAllocator +{ + BufferPoolAllocator() + { + super(true); + } + + @Override + public boolean isDirectBufferPooled() + { + return true; + } + + /** shouldn't be invoked */ + @Override + protected ByteBuf newHeapBuffer(int minCapacity, int maxCapacity) + { + return Unpooled.buffer(minCapacity, maxCapacity); + } + + @Override + protected ByteBuf newDirectBuffer(int minCapacity, int maxCapacity) + { + ByteBuf result = new Wrapped(this, getAtLeast(minCapacity)); + result.clear(); + return result; + } + + ByteBuffer get(int size) + { + return BufferPool.get(size, BufferType.OFF_HEAP); + } + + ByteBuffer getAtLeast(int size) + { + return BufferPool.getAtLeast(size, BufferType.OFF_HEAP); + } + + void put(ByteBuffer buffer) + { + BufferPool.put(buffer); + } + + void putUnusedPortion(ByteBuffer buffer) + { + BufferPool.putUnusedPortion(buffer); + } + + void release() + { + } + + /** + * A simple extension to UnpooledUnsafeDirectByteBuf that returns buffers to BufferPool on deallocate, + * and permits extracting the buffer from it to take ownership and use directly. + */ + public static class Wrapped extends UnpooledUnsafeDirectByteBuf + { + private ByteBuffer wrapped; + + Wrapped(BufferPoolAllocator allocator, ByteBuffer wrap) + { + super(allocator, wrap, wrap.capacity()); + wrapped = wrap; + } + + @Override + public void deallocate() + { + if (wrapped != null) + BufferPool.put(wrapped); + } + + public ByteBuffer adopt() + { + if (refCnt() > 1) + throw new IllegalStateException(); + ByteBuffer adopt = wrapped; + adopt.position(readerIndex()).limit(writerIndex()); + wrapped = null; + return adopt; + } + } +} diff --git a/src/java/org/apache/cassandra/net/CallbackInfo.java b/src/java/org/apache/cassandra/net/CallbackInfo.java deleted file mode 100644 index f2ed8a10fa97..000000000000 --- a/src/java/org/apache/cassandra/net/CallbackInfo.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.net; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.locator.InetAddressAndPort; - -/** - * Encapsulates the callback information. - * The ability to set the message is useful in cases for when a hint needs - * to be written due to a timeout in the response from a replica. - */ -public class CallbackInfo -{ - protected final InetAddressAndPort target; - protected final IAsyncCallback callback; - protected final IVersionedSerializer serializer; - private final boolean failureCallback; - - /** - * Create CallbackInfo without sent message - * - * @param target target to send message - * @param callback - * @param serializer serializer to deserialize response message - * @param failureCallback True when we have a callback to handle failures - */ - public CallbackInfo(InetAddressAndPort target, IAsyncCallback callback, IVersionedSerializer serializer, boolean failureCallback) - { - this.target = target; - this.callback = callback; - this.serializer = serializer; - this.failureCallback = failureCallback; - } - - public boolean shouldHint() - { - return false; - } - - public boolean isFailureCallback() - { - return failureCallback; - } - - public String toString() - { - return "CallbackInfo(" + - "target=" + target + - ", callback=" + callback + - ", serializer=" + serializer + - ", failureCallback=" + failureCallback + - ')'; - } -} diff --git a/src/java/org/apache/cassandra/net/ChunkedInputPlus.java b/src/java/org/apache/cassandra/net/ChunkedInputPlus.java new file mode 100644 index 000000000000..3aad8d96150e --- /dev/null +++ b/src/java/org/apache/cassandra/net/ChunkedInputPlus.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.EOFException; + +import com.google.common.collect.Iterators; +import com.google.common.collect.PeekingIterator; + +import org.apache.cassandra.io.util.RebufferingInputStream; + +/** + * A specialised {@link org.apache.cassandra.io.util.DataInputPlus} implementation for deserializing large messages + * that are split over multiple {@link FrameDecoder.Frame}s. + * + * Ensures that every underlying {@link ShareableBytes} frame is released, and promptly so, as frames are consumed. + * + * {@link #close()} MUST be invoked in the end. + */ +class ChunkedInputPlus extends RebufferingInputStream +{ + private final PeekingIterator iter; + + private ChunkedInputPlus(PeekingIterator iter) + { + super(iter.peek().get()); + this.iter = iter; + } + + /** + * Creates a {@link ChunkedInputPlus} from the provided {@link ShareableBytes} buffers. + * + * The provided iterable must contain at least one buffer. + */ + static ChunkedInputPlus of(Iterable buffers) + { + PeekingIterator iter = Iterators.peekingIterator(buffers.iterator()); + if (!iter.hasNext()) + throw new IllegalArgumentException(); + return new ChunkedInputPlus(iter); + } + + @Override + protected void reBuffer() throws EOFException + { + buffer = null; + iter.peek().release(); + iter.next(); + + if (!iter.hasNext()) + throw new EOFException(); + + buffer = iter.peek().get(); + } + + @Override + public void close() + { + buffer = null; + iter.forEachRemaining(ShareableBytes::release); + } + + /** + * Returns the number of unconsumed bytes. Will release any outstanding buffers and consume the underlying iterator. + * + * Should only be used for sanity checking, once the input is no longer needed, as it will implicitly close the input. + */ + int remainder() + { + buffer = null; + + int bytes = 0; + while (iter.hasNext()) + { + ShareableBytes chunk = iter.peek(); + bytes += chunk.remaining(); + chunk.release(); + iter.next(); + } + return bytes; + } +} diff --git a/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java b/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java deleted file mode 100644 index b58ca472540a..000000000000 --- a/src/java/org/apache/cassandra/net/CompactEndpointSerializationHelper.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.net; - -import java.io.*; -import java.net.Inet4Address; -import java.net.Inet6Address; -import java.net.InetAddress; -import java.nio.ByteBuffer; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.streaming.messages.StreamMessage; - -/* - * As of version 4.0 the endpoint description includes a port number as an unsigned short - */ -public class CompactEndpointSerializationHelper implements IVersionedSerializer -{ - public static final IVersionedSerializer instance = new CompactEndpointSerializationHelper(); - - /** - * Streaming uses its own version numbering so we need to ignore it and always use currrent version. - * There is no cross version streaming so it will always use the latest address serialization. - **/ - public static final IVersionedSerializer streamingInstance = new IVersionedSerializer() - { - public void serialize(InetAddressAndPort inetAddressAndPort, DataOutputPlus out, int version) throws IOException - { - instance.serialize(inetAddressAndPort, out, MessagingService.current_version); - } - - public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException - { - return instance.deserialize(in, MessagingService.current_version); - } - - public long serializedSize(InetAddressAndPort inetAddressAndPort, int version) - { - return instance.serializedSize(inetAddressAndPort, MessagingService.current_version); - } - }; - - private CompactEndpointSerializationHelper() {} - - public void serialize(InetAddressAndPort endpoint, DataOutputPlus out, int version) throws IOException - { - if (version >= MessagingService.VERSION_40) - { - byte[] buf = endpoint.addressBytes; - out.writeByte(buf.length + 2); - out.write(buf); - out.writeShort(endpoint.port); - } - else - { - byte[] buf = endpoint.addressBytes; - out.writeByte(buf.length); - out.write(buf); - } - } - - public InetAddressAndPort deserialize(DataInputPlus in, int version) throws IOException - { - int size = in.readByte() & 0xFF; - switch(size) - { - //The original pre-4.0 serialiation of just an address - case 4: - case 16: - { - byte[] bytes = new byte[size]; - in.readFully(bytes, 0, bytes.length); - return InetAddressAndPort.getByAddress(bytes); - } - //Address and one port - case 6: - case 18: - { - byte[] bytes = new byte[size - 2]; - in.readFully(bytes); - - int port = in.readShort() & 0xFFFF; - return InetAddressAndPort.getByAddressOverrideDefaults(InetAddress.getByAddress(bytes), bytes, port); - } - default: - throw new AssertionError("Unexpected size " + size); - - } - } - - public long serializedSize(InetAddressAndPort from, int version) - { - //4.0 includes a port number - if (version >= MessagingService.VERSION_40) - { - if (from.address instanceof Inet4Address) - return 1 + 4 + 2; - assert from.address instanceof Inet6Address; - return 1 + 16 + 2; - } - else - { - if (from.address instanceof Inet4Address) - return 1 + 4; - assert from.address instanceof Inet6Address; - return 1 + 16; - } - } -} diff --git a/src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java b/src/java/org/apache/cassandra/net/ConnectionCategory.java similarity index 69% rename from src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java rename to src/java/org/apache/cassandra/net/ConnectionCategory.java index 2b91f2056c09..d739e9d1ff00 100644 --- a/src/java/org/apache/cassandra/net/IAsyncCallbackWithFailure.java +++ b/src/java/org/apache/cassandra/net/ConnectionCategory.java @@ -15,16 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.net; -import org.apache.cassandra.exceptions.RequestFailureReason; -import org.apache.cassandra.locator.InetAddressAndPort; +package org.apache.cassandra.net; -public interface IAsyncCallbackWithFailure extends IAsyncCallback +public enum ConnectionCategory { + MESSAGING, STREAMING; + + public boolean isStreaming() + { + return this == STREAMING; + } - /** - * Called when there is an exception on the remote node or timeout happens - */ - void onFailure(InetAddressAndPort from, RequestFailureReason failureReason); + public boolean isMessaging() + { + return this == MESSAGING; + } } diff --git a/src/java/org/apache/cassandra/net/ConnectionType.java b/src/java/org/apache/cassandra/net/ConnectionType.java new file mode 100644 index 000000000000..db83d06856a2 --- /dev/null +++ b/src/java/org/apache/cassandra/net/ConnectionType.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.List; + +import com.google.common.collect.ImmutableList; + +public enum ConnectionType +{ + LEGACY_MESSAGES (0), // only used for inbound + URGENT_MESSAGES (1), + SMALL_MESSAGES (2), + LARGE_MESSAGES (3), + STREAMING (4); + + public static final List MESSAGING_TYPES = ImmutableList.of(URGENT_MESSAGES, SMALL_MESSAGES, LARGE_MESSAGES); + + public final int id; + + ConnectionType(int id) + { + this.id = id; + } + + public int twoBitID() + { + if (id < 0 || id > 0b11) + throw new AssertionError(); + return id; + } + + public boolean isStreaming() + { + return this == STREAMING; + } + + public boolean isMessaging() + { + return !isStreaming(); + } + + public ConnectionCategory category() + { + return this == STREAMING ? ConnectionCategory.STREAMING : ConnectionCategory.MESSAGING; + } + + private static final ConnectionType[] values = values(); + + public static ConnectionType fromId(int id) + { + return values[id]; + } +} diff --git a/src/java/org/apache/cassandra/net/Crc.java b/src/java/org/apache/cassandra/net/Crc.java new file mode 100644 index 000000000000..dbd26014d566 --- /dev/null +++ b/src/java/org/apache/cassandra/net/Crc.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.zip.CRC32; + +import io.netty.buffer.ByteBuf; +import io.netty.util.concurrent.FastThreadLocal; + +class Crc +{ + private static final FastThreadLocal crc32 = new FastThreadLocal() + { + @Override + protected CRC32 initialValue() + { + return new CRC32(); + } + }; + + private static final byte[] initialBytes = new byte[] { (byte) 0xFA, (byte) 0x2D, (byte) 0x55, (byte) 0xCA }; + + static final class InvalidCrc extends IOException + { + InvalidCrc(int read, int computed) + { + super(String.format("Read %d, Computed %d", read, computed)); + } + } + + static CRC32 crc32() + { + CRC32 crc = crc32.get(); + crc.reset(); + crc.update(initialBytes); + return crc; + } + + static int computeCrc32(ByteBuf buffer, int startReaderIndex, int endReaderIndex) + { + CRC32 crc = crc32(); + crc.update(buffer.internalNioBuffer(startReaderIndex, endReaderIndex - startReaderIndex)); + return (int) crc.getValue(); + } + + static int computeCrc32(ByteBuffer buffer, int start, int end) + { + CRC32 crc = crc32(); + updateCrc32(crc, buffer, start, end); + return (int) crc.getValue(); + } + + static void updateCrc32(CRC32 crc, ByteBuffer buffer, int start, int end) + { + int savePosition = buffer.position(); + int saveLimit = buffer.limit(); + buffer.limit(end); + buffer.position(start); + crc.update(buffer); + buffer.limit(saveLimit); + buffer.position(savePosition); + } + + private static final int CRC24_INIT = 0x875060; + /** + * Polynomial chosen from https://users.ece.cmu.edu/~koopman/crc/index.html, by Philip Koopman + * + * This webpage claims a copyright to Philip Koopman, which he licenses under the + * Creative Commons Attribution 4.0 International License (https://creativecommons.org/licenses/by/4.0) + * + * It is unclear if this copyright can extend to a 'fact' such as this specific number, particularly + * as we do not use Koopman's notation to represent the polynomial, but we anyway attribute his work and + * link the terms of his license since they are not incompatible with our usage and we greatly appreciate his work. + * + * This polynomial provides hamming distance of 8 for messages up to length 105 bits; + * we only support 8-64 bits at present, with an expected range of 40-48. + */ + private static final int CRC24_POLY = 0x1974F0B; + + /** + * NOTE: the order of bytes must reach the wire in the same order the CRC is computed, with the CRC + * immediately following in a trailer. Since we read in least significant byte order, if you + * write to a buffer using putInt or putLong, the byte order will be reversed and + * you will lose the guarantee of protection from burst corruptions of 24 bits in length. + * + * Make sure either to write byte-by-byte to the wire, or to use Integer/Long.reverseBytes if you + * write to a BIG_ENDIAN buffer. + * + * See http://users.ece.cmu.edu/~koopman/pubs/ray06_crcalgorithms.pdf + * + * Complain to the ethernet spec writers, for having inverse bit to byte significance order. + * + * Note we use the most naive algorithm here. We support at most 8 bytes, and typically supply + * 5 or fewer, so any efficiency of a table approach is swallowed by the time to hit L3, even + * for a tiny (4bit) table. + * + * @param bytes an up to 8-byte register containing bytes to compute the CRC over + * the bytes AND bits will be read least-significant to most significant. + * @param len the number of bytes, greater than 0 and fewer than 9, to be read from bytes + * @return the least-significant bit AND byte order crc24 using the CRC24_POLY polynomial + */ + static int crc24(long bytes, int len) + { + int crc = CRC24_INIT; + while (len-- > 0) + { + crc ^= (bytes & 0xff) << 16; + bytes >>= 8; + + for (int i = 0; i < 8; i++) + { + crc <<= 1; + if ((crc & 0x1000000) != 0) + crc ^= CRC24_POLY; + } + } + return crc; + } +} diff --git a/src/java/org/apache/cassandra/net/EndpointMessagingVersions.java b/src/java/org/apache/cassandra/net/EndpointMessagingVersions.java new file mode 100644 index 000000000000..e8cf8f68daa5 --- /dev/null +++ b/src/java/org/apache/cassandra/net/EndpointMessagingVersions.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.net.UnknownHostException; +import java.util.concurrent.ConcurrentMap; + +import org.cliffc.high_scale_lib.NonBlockingHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.locator.InetAddressAndPort; + +/** + * Map of hosts to their known current messaging versions. + */ +public class EndpointMessagingVersions +{ + private static final Logger logger = LoggerFactory.getLogger(EndpointMessagingVersions.class); + + // protocol versions of the other nodes in the cluster + private final ConcurrentMap versions = new NonBlockingHashMap<>(); + + /** + * @return the last version associated with address, or @param version if this is the first such version + */ + public int set(InetAddressAndPort endpoint, int version) + { + logger.trace("Setting version {} for {}", version, endpoint); + + Integer v = versions.put(endpoint, version); + return v == null ? version : v; + } + + public void reset(InetAddressAndPort endpoint) + { + logger.trace("Resetting version for {}", endpoint); + versions.remove(endpoint); + } + + /** + * Returns the messaging-version as announced by the given node but capped + * to the min of the version as announced by the node and {@link MessagingService#current_version}. + */ + public int get(InetAddressAndPort endpoint) + { + Integer v = versions.get(endpoint); + if (v == null) + { + // we don't know the version. assume current. we'll know soon enough if that was incorrect. + logger.trace("Assuming current protocol version for {}", endpoint); + return MessagingService.current_version; + } + else + return Math.min(v, MessagingService.current_version); + } + + public int get(String endpoint) throws UnknownHostException + { + return get(InetAddressAndPort.getByName(endpoint)); + } + + /** + * Returns the messaging-version exactly as announced by the given endpoint. + */ + public int getRaw(InetAddressAndPort endpoint) + { + Integer v = versions.get(endpoint); + if (v == null) + throw new IllegalStateException("getRawVersion() was called without checking knowsVersion() result first"); + return v; + } + + public boolean knows(InetAddressAndPort endpoint) + { + return versions.containsKey(endpoint); + } +} diff --git a/src/java/org/apache/cassandra/net/ForwardToSerializer.java b/src/java/org/apache/cassandra/net/ForwardToSerializer.java deleted file mode 100644 index c4e8843ec2ea..000000000000 --- a/src/java/org/apache/cassandra/net/ForwardToSerializer.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; - -public class ForwardToSerializer implements IVersionedSerializer -{ - public static ForwardToSerializer instance = new ForwardToSerializer(); - - private ForwardToSerializer() {} - - public void serialize(ForwardToContainer forwardToContainer, DataOutputPlus out, int version) throws IOException - { - out.writeInt(forwardToContainer.targets.size()); - Iterator iter = forwardToContainer.targets.iterator(); - for (int ii = 0; ii < forwardToContainer.messageIds.length; ii++) - { - CompactEndpointSerializationHelper.instance.serialize(iter.next(), out, version); - out.writeInt(forwardToContainer.messageIds[ii]); - } - } - - public ForwardToContainer deserialize(DataInputPlus in, int version) throws IOException - { - int[] ids = new int[in.readInt()]; - List hosts = new ArrayList<>(ids.length); - for (int ii = 0; ii < ids.length; ii++) - { - hosts.add(CompactEndpointSerializationHelper.instance.deserialize(in, version)); - ids[ii] = in.readInt(); - } - return new ForwardToContainer(hosts, ids); - } - - public long serializedSize(ForwardToContainer forwardToContainer, int version) - { - //Number of forward addresses, 4 bytes per for each id - long size = 4 + - (4 * forwardToContainer.targets.size()); - //Depending on ipv6 or ipv4 the address size is different. - for (InetAddressAndPort forwardTo : forwardToContainer.targets) - { - size += CompactEndpointSerializationHelper.instance.serializedSize(forwardTo, version); - } - - return size; - } - - public static ForwardToContainer fromBytes(byte[] bytes, int version) - { - try (DataInputBuffer input = new DataInputBuffer(bytes)) - { - return instance.deserialize(input, version); - } - catch (IOException e) - { - throw new RuntimeException(e); - } - } -} diff --git a/src/java/org/apache/cassandra/net/ForwardingInfo.java b/src/java/org/apache/cassandra/net/ForwardingInfo.java new file mode 100644 index 000000000000..737da48f40d3 --- /dev/null +++ b/src/java/org/apache/cassandra/net/ForwardingInfo.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiConsumer; + +import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; + +import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.utils.vint.VIntCoding.computeUnsignedVIntSize; + +/** + * A container used to store a node -> message_id map for inter-DC write forwarding. + * We pick one node in each external DC to forward the message to its local peers. + * + * TODO: in the next protocol version only serialize peers, message id will become redundant once 3.0 is out of the picture + */ +public final class ForwardingInfo implements Serializable +{ + final List targets; + final long[] messageIds; + + public ForwardingInfo(List targets, long[] messageIds) + { + Preconditions.checkArgument(targets.size() == messageIds.length); + this.targets = targets; + this.messageIds = messageIds; + } + + /** + * @return {@code true} if all host are to use the same message id, {@code false} otherwise. Starting with 4.0 and + * above, we should be reusing the same id, always, but it won't always be true until 3.0/3.11 are phased out. + */ + public boolean useSameMessageID() + { + if (messageIds.length < 2) + return true; + + long id = messageIds[0]; + for (int i = 1; i < messageIds.length; i++) + if (id != messageIds[i]) + return false; + + return true; + } + + /** + * Apply the provided consumer to all (host, message_id) pairs. + */ + public void forEach(BiConsumer biConsumer) + { + for (int i = 0; i < messageIds.length; i++) + biConsumer.accept(messageIds[i], targets.get(i)); + } + + static final IVersionedSerializer serializer = new IVersionedSerializer() + { + public void serialize(ForwardingInfo forwardTo, DataOutputPlus out, int version) throws IOException + { + long[] ids = forwardTo.messageIds; + List targets = forwardTo.targets; + + int count = ids.length; + if (version >= VERSION_40) + out.writeUnsignedVInt(count); + else + out.writeInt(count); + + for (int i = 0; i < count; i++) + { + inetAddressAndPortSerializer.serialize(targets.get(i), out, version); + if (version >= VERSION_40) + out.writeUnsignedVInt(ids[i]); + else + out.writeInt(Ints.checkedCast(ids[i])); + } + } + + public long serializedSize(ForwardingInfo forwardTo, int version) + { + long[] ids = forwardTo.messageIds; + List targets = forwardTo.targets; + + int count = ids.length; + long size = version >= VERSION_40 ? computeUnsignedVIntSize(count) : TypeSizes.sizeof(count); + + for (int i = 0; i < count; i++) + { + size += inetAddressAndPortSerializer.serializedSize(targets.get(i), version); + size += version >= VERSION_40 ? computeUnsignedVIntSize(ids[i]) : 4; + } + + return size; + } + + public ForwardingInfo deserialize(DataInputPlus in, int version) throws IOException + { + int count = version >= VERSION_40 ? Ints.checkedCast(in.readUnsignedVInt()) : in.readInt(); + + long[] ids = new long[count]; + List targets = new ArrayList<>(count); + + for (int i = 0; i < count; i++) + { + targets.add(inetAddressAndPortSerializer.deserialize(in, version)); + ids[i] = version >= VERSION_40 ? Ints.checkedCast(in.readUnsignedVInt()) : in.readInt(); + } + + return new ForwardingInfo(targets, ids); + } + }; +} diff --git a/src/java/org/apache/cassandra/net/FrameDecoder.java b/src/java/org/apache/cassandra/net/FrameDecoder.java new file mode 100644 index 000000000000..ed96adda32b8 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameDecoder.java @@ -0,0 +1,400 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Deque; + +import com.google.common.annotations.VisibleForTesting; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelPipeline; + +import static org.apache.cassandra.utils.ByteBufferUtil.copyBytes; + +/** + * A Netty inbound handler that decodes incoming frames and passes them forward to + * {@link InboundMessageHandler} for processing. + * + * Handles work stashing, and together with {@link InboundMessageHandler} - flow control. + * + * Unlike most Netty inbound handlers, doesn't use the pipeline to talk to its + * upstream handler. Instead, a {@link FrameProcessor} must be registered with + * the frame decoder, to be invoked on new frames. See {@link #deliver(FrameProcessor)}. + * + * See {@link #activate(FrameProcessor)}, {@link #reactivate()}, and {@link FrameProcessor} + * for flow control implementation. + * + * Five frame decoders currently exist, one used for each connection depending on flags and messaging version: + * 1. {@link FrameDecoderCrc}: + no compression; payload is protected by CRC32 + * 2. {@link FrameDecoderLZ4}: + LZ4 compression with custom frame format; payload is protected by CRC32 + * 3. {@link FrameDecoderUnprotected}: + no compression; no integrity protection + * 4. {@link FrameDecoderLegacy}: + no compression; no integrity protection; turns unframed streams of legacy messages (< 4.0) into frames + * 5. {@link FrameDecoderLegacyLZ4} + * LZ4 compression using standard LZ4 frame format; groups legacy messages (< 4.0) into frames + */ +abstract class FrameDecoder extends ChannelInboundHandlerAdapter +{ + private static final FrameProcessor NO_PROCESSOR = + frame -> { throw new IllegalStateException("Frame processor invoked on an unregistered FrameDecoder"); }; + + private static final FrameProcessor CLOSED_PROCESSOR = + frame -> { throw new IllegalStateException("Frame processor invoked on a closed FrameDecoder"); }; + + interface FrameProcessor + { + /** + * Frame processor that the frames should be handed off to. + * + * @return true if more frames can be taken by the processor, false if the decoder should pause until + * it's explicitly resumed. + */ + boolean process(Frame frame) throws IOException; + } + + abstract static class Frame + { + final boolean isSelfContained; + final int frameSize; + + Frame(boolean isSelfContained, int frameSize) + { + this.isSelfContained = isSelfContained; + this.frameSize = frameSize; + } + + abstract void release(); + abstract boolean isConsumed(); + } + + /** + * The payload bytes of a complete frame, i.e. a frame stripped of its headers and trailers, + * with any verification supported by the protocol confirmed. + * + * If {@code isSelfContained} the payload contains one or more {@link Message}, all of which + * may be parsed entirely from the bytes provided. Otherwise, only a part of exactly one + * {@link Message} is contained in the payload; it can be relied upon that this partial {@link Message} + * will only be delivered in its own unique {@link Frame}. + */ + final static class IntactFrame extends Frame + { + final ShareableBytes contents; + + IntactFrame(boolean isSelfContained, ShareableBytes contents) + { + super(isSelfContained, contents.remaining()); + this.contents = contents; + } + + void release() + { + contents.release(); + } + + boolean isConsumed() + { + return !contents.hasRemaining(); + } + + void consume() + { + contents.consume(); + } + } + + /** + * A corrupted frame was encountered; this represents the knowledge we have about this frame, + * and whether or not the stream is recoverable. + * + * Generally we consider a frame with corrupted header as unrecoverable, and frames with intact header, + * but corrupted payload - as recoverable, since we know and can skip payload size. + * + * {@link InboundMessageHandler} further has its own idea of which frames are and aren't recoverable. + * A recoverable {@link CorruptFrame} can be considered unrecoverable by {@link InboundMessageHandler} + * if it's the first frame of a large message (isn't self contained). + */ + final static class CorruptFrame extends Frame + { + final int readCRC, computedCRC; + + CorruptFrame(boolean isSelfContained, int frameSize, int readCRC, int computedCRC) + { + super(isSelfContained, frameSize); + this.readCRC = readCRC; + this.computedCRC = computedCRC; + } + + static CorruptFrame recoverable(boolean isSelfContained, int frameSize, int readCRC, int computedCRC) + { + return new CorruptFrame(isSelfContained, frameSize, readCRC, computedCRC); + } + + static CorruptFrame unrecoverable(int readCRC, int computedCRC) + { + return new CorruptFrame(false, Integer.MIN_VALUE, readCRC, computedCRC); + } + + boolean isRecoverable() + { + return frameSize != Integer.MIN_VALUE; + } + + void release() { } + + boolean isConsumed() + { + return true; + } + } + + protected final BufferPoolAllocator allocator; + + @VisibleForTesting + final Deque frames = new ArrayDeque<>(4); + ByteBuffer stash; + + private boolean isActive; + private boolean isClosed; + private ChannelHandlerContext ctx; + private FrameProcessor processor = NO_PROCESSOR; + + FrameDecoder(BufferPoolAllocator allocator) + { + this.allocator = allocator; + } + + abstract void decode(Collection into, ShareableBytes bytes); + abstract void addLastTo(ChannelPipeline pipeline); + + /** + * For use by InboundMessageHandler (or other upstream handlers) that want to start receiving frames. + */ + void activate(FrameProcessor processor) + { + if (this.processor != NO_PROCESSOR) + throw new IllegalStateException("Attempted to activate an already active FrameDecoder"); + + this.processor = processor; + + isActive = true; + ctx.read(); + } + + /** + * For use by InboundMessageHandler (or other upstream handlers) that want to resume + * receiving frames after previously indicating that processing should be paused. + */ + void reactivate() throws IOException + { + if (isActive) + throw new IllegalStateException("Tried to reactivate an already active FrameDecoder"); + + if (deliver(processor)) + { + isActive = true; + onExhausted(); + } + } + + /** + * For use by InboundMessageHandler (or other upstream handlers) that want to resume + * receiving frames after previously indicating that processing should be paused. + * + * Does not reactivate processing or reading from the wire, but permits processing as many frames (or parts thereof) + * that are already waiting as the processor requires. + */ + void processBacklog(FrameProcessor processor) throws IOException + { + deliver(processor); + } + + /** + * For use by InboundMessageHandler (or other upstream handlers) that want to permanently + * stop receiving frames, e.g. because of an exception caught. + */ + void discard() + { + isActive = false; + processor = CLOSED_PROCESSOR; + if (stash != null) + { + ByteBuffer bytes = stash; + stash = null; + allocator.put(bytes); + } + while (!frames.isEmpty()) + frames.poll().release(); + } + + /** + * Called by Netty pipeline when a new message arrives; we anticipate in normal operation + * this will receive messages of type {@link BufferPoolAllocator.Wrapped} or + * {@link BufferPoolAllocator.Wrapped}. + * + * These buffers are unwrapped and passed to {@link #decode(Collection, ShareableBytes)}, + * which collects decoded frames into {@link #frames}, which we send upstream in {@link #deliver} + */ + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws IOException + { + if (msg instanceof BufferPoolAllocator.Wrapped) + { + ByteBuffer buf = ((BufferPoolAllocator.Wrapped) msg).adopt(); + // netty will probably have mis-predicted the space needed + allocator.putUnusedPortion(buf); + channelRead(ShareableBytes.wrap(buf)); + } + else if (msg instanceof ShareableBytes) // legacy LZ4 decoder + { + channelRead((ShareableBytes) msg); + } + else + { + throw new IllegalArgumentException(); + } + } + + void channelRead(ShareableBytes bytes) throws IOException + { + decode(frames, bytes); + + if (isActive) isActive = deliver(processor); + } + + @Override + public void channelReadComplete(ChannelHandlerContext ctx) + { + if (isActive) + onExhausted(); + } + + /** + * Only to be invoked when frames.isEmpty(). + * + * If we have been closed, we will now propagate up the channelInactive notification, + * and otherwise we will ask the channel for more data. + */ + private void onExhausted() + { + if (isClosed) + close(); + else + ctx.read(); + } + + /** + * Deliver any waiting frames, including those that were incompletely read last time, to the provided processor + * until the processor returns {@code false}, or we finish the backlog. + * + * Propagate the final return value of the processor. + */ + private boolean deliver(FrameProcessor processor) throws IOException + { + boolean deliver = true; + while (deliver && !frames.isEmpty()) + { + Frame frame = frames.peek(); + deliver = processor.process(frame); + + assert !deliver || frame.isConsumed(); + if (deliver || frame.isConsumed()) + { + frames.poll(); + frame.release(); + } + } + return deliver; + } + + void stash(ShareableBytes in, int stashLength, int begin, int length) + { + ByteBuffer out = allocator.getAtLeast(stashLength); + copyBytes(in.get(), begin, out, 0, length); + out.position(length); + stash = out; + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) + { + this.ctx = ctx; + ctx.channel().config().setAutoRead(false); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) + { + isClosed = true; + if (frames.isEmpty()) + close(); + } + + private void close() + { + discard(); + ctx.fireChannelInactive(); + allocator.release(); + } + + /** + * Utility: fill {@code out} from {@code in} up to {@code toOutPosition}, + * updating the position of both buffers with the result + * @return true if there were sufficient bytes to fill to {@code toOutPosition} + */ + static boolean copyToSize(ByteBuffer in, ByteBuffer out, int toOutPosition) + { + int bytesToSize = toOutPosition - out.position(); + if (bytesToSize <= 0) + return true; + + if (bytesToSize > in.remaining()) + { + out.put(in); + return false; + } + + copyBytes(in, in.position(), out, out.position(), bytesToSize); + in.position(in.position() + bytesToSize); + out.position(toOutPosition); + return true; + } + + /** + * @return {@code in} if has sufficient capacity, otherwise + * a replacement from {@code BufferPool} that {@code in} is copied into + */ + ByteBuffer ensureCapacity(ByteBuffer in, int capacity) + { + if (in.capacity() >= capacity) + return in; + + ByteBuffer out = allocator.getAtLeast(capacity); + in.flip(); + out.put(in); + allocator.put(in); + return out; + } +} diff --git a/src/java/org/apache/cassandra/net/FrameDecoderCrc.java b/src/java/org/apache/cassandra/net/FrameDecoderCrc.java new file mode 100644 index 000000000000..7cd52ac7df63 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameDecoderCrc.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Collection; +import java.util.zip.CRC32; + +import io.netty.channel.ChannelPipeline; + +import static org.apache.cassandra.net.Crc.*; +import static org.apache.cassandra.net.Crc.updateCrc32; + +/** + * Framing format that protects integrity of data in movement with CRCs (of both header and payload). + * + * Every on-wire frame contains: + * 1. Payload length (17 bits) + * 2. {@code isSelfContained} flag (1 bit) + * 3. Header padding (6 bits) + * 4. CRC24 of the header (24 bits) + * 5. Payload (up to 2 ^ 17 - 1 bits) + * 6. Payload CRC32 (32 bits) + * + * 0 1 2 3 + * 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | Payload Length |C| | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * CRC24 of Header | | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + + * | | + * + + + * | Payload | + * + + + * | | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | CRC32 of Payload | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + */ +final class FrameDecoderCrc extends FrameDecoderWith8bHeader +{ + private FrameDecoderCrc(BufferPoolAllocator allocator) + { + super(allocator); + } + + public static FrameDecoderCrc create(BufferPoolAllocator allocator) + { + return new FrameDecoderCrc(allocator); + } + + static final int HEADER_LENGTH = 6; + private static final int TRAILER_LENGTH = 4; + private static final int HEADER_AND_TRAILER_LENGTH = 10; + + static boolean isSelfContained(long header6b) + { + return 0 != (header6b & (1L << 17)); + } + + static int payloadLength(long header6b) + { + return ((int) header6b) & 0x1FFFF; + } + + private static int headerCrc(long header6b) + { + return ((int) (header6b >>> 24)) & 0xFFFFFF; + } + + static long readHeader6b(ByteBuffer frame, int begin) + { + long header6b; + if (frame.limit() - begin >= 8) + { + header6b = frame.getLong(begin); + if (frame.order() == ByteOrder.BIG_ENDIAN) + header6b = Long.reverseBytes(header6b); + header6b &= 0xffffffffffffL; + } + else + { + header6b = 0; + for (int i = 0 ; i < HEADER_LENGTH ; ++i) + header6b |= (0xffL & frame.get(begin + i)) << (8 * i); + } + return header6b; + } + + static CorruptFrame verifyHeader6b(long header6b) + { + int computeLengthCrc = crc24(header6b, 3); + int readLengthCrc = headerCrc(header6b); + + return readLengthCrc == computeLengthCrc ? null : CorruptFrame.unrecoverable(readLengthCrc, computeLengthCrc); + } + + final long readHeader(ByteBuffer frame, int begin) + { + return readHeader6b(frame, begin); + } + + final CorruptFrame verifyHeader(long header6b) + { + return verifyHeader6b(header6b); + } + + final int frameLength(long header6b) + { + return payloadLength(header6b) + HEADER_AND_TRAILER_LENGTH; + } + + final Frame unpackFrame(ShareableBytes bytes, int begin, int end, long header6b) + { + ByteBuffer in = bytes.get(); + boolean isSelfContained = isSelfContained(header6b); + + CRC32 crc = crc32(); + int readFullCrc = in.getInt(end - TRAILER_LENGTH); + if (in.order() == ByteOrder.BIG_ENDIAN) + readFullCrc = Integer.reverseBytes(readFullCrc); + + updateCrc32(crc, in, begin + HEADER_LENGTH, end - TRAILER_LENGTH); + int computeFullCrc = (int) crc.getValue(); + + if (readFullCrc != computeFullCrc) + return CorruptFrame.recoverable(isSelfContained, (end - begin) - HEADER_AND_TRAILER_LENGTH, readFullCrc, computeFullCrc); + + return new IntactFrame(isSelfContained, bytes.slice(begin + HEADER_LENGTH, end - TRAILER_LENGTH)); + } + + void decode(Collection into, ShareableBytes bytes) + { + decode(into, bytes, HEADER_LENGTH); + } + + void addLastTo(ChannelPipeline pipeline) + { + pipeline.addLast("frameDecoderCrc", this); + } +} diff --git a/src/java/org/apache/cassandra/net/FrameDecoderLZ4.java b/src/java/org/apache/cassandra/net/FrameDecoderLZ4.java new file mode 100644 index 000000000000..941139a0155f --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameDecoderLZ4.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Collection; +import java.util.zip.CRC32; + +import io.netty.channel.ChannelPipeline; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4FastDecompressor; + +import static org.apache.cassandra.net.Crc.*; + +/** + * Framing format that compresses payloads with LZ4, and protects integrity of data in movement with CRCs + * (of both header and payload). + * + * Every on-wire frame contains: + * 1. Compressed length (17 bits) + * 2. Uncompressed length (17 bits) + * 3. {@code isSelfContained} flag (1 bit) + * 4. Header padding (5 bits) + * 5. CRC24 of Header contents (24 bits) + * 6. Compressed Payload (up to 2 ^ 17 - 1 bits) + * 7. CRC32 of Compressed Payload (32 bits) + * + * 0 1 2 3 + * 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | Compressed Length | Uncompressed Length + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * |C| | CRC24 of Header | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | | + * + + + * | Compressed Payload | + * + + + * | | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | CRC32 of Compressed Payload | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + */ +final class FrameDecoderLZ4 extends FrameDecoderWith8bHeader +{ + public static FrameDecoderLZ4 fast(BufferPoolAllocator allocator) + { + return new FrameDecoderLZ4(allocator, LZ4Factory.fastestInstance().fastDecompressor()); + } + + private static final int HEADER_LENGTH = 8; + private static final int TRAILER_LENGTH = 4; + private static final int HEADER_AND_TRAILER_LENGTH = 12; + + private static int compressedLength(long header8b) + { + return ((int) header8b) & 0x1FFFF; + } + private static int uncompressedLength(long header8b) + { + return ((int) (header8b >>> 17)) & 0x1FFFF; + } + private static boolean isSelfContained(long header8b) + { + return 0 != (header8b & (1L << 34)); + } + private static int headerCrc(long header8b) + { + return ((int) (header8b >>> 40)) & 0xFFFFFF; + } + + private final LZ4FastDecompressor decompressor; + + private FrameDecoderLZ4(BufferPoolAllocator allocator, LZ4FastDecompressor decompressor) + { + super(allocator); + this.decompressor = decompressor; + } + + final long readHeader(ByteBuffer frame, int begin) + { + long header8b = frame.getLong(begin); + if (frame.order() == ByteOrder.BIG_ENDIAN) + header8b = Long.reverseBytes(header8b); + return header8b; + } + + final CorruptFrame verifyHeader(long header8b) + { + int computeLengthCrc = crc24(header8b, 5); + int readLengthCrc = headerCrc(header8b); + + return readLengthCrc == computeLengthCrc ? null : CorruptFrame.unrecoverable(readLengthCrc, computeLengthCrc); + } + + final int frameLength(long header8b) + { + return compressedLength(header8b) + HEADER_AND_TRAILER_LENGTH; + } + + final Frame unpackFrame(ShareableBytes bytes, int begin, int end, long header8b) + { + ByteBuffer input = bytes.get(); + + boolean isSelfContained = isSelfContained(header8b); + int uncompressedLength = uncompressedLength(header8b); + + CRC32 crc = crc32(); + int readFullCrc = input.getInt(end - TRAILER_LENGTH); + if (input.order() == ByteOrder.BIG_ENDIAN) + readFullCrc = Integer.reverseBytes(readFullCrc); + + updateCrc32(crc, input, begin + HEADER_LENGTH, end - TRAILER_LENGTH); + int computeFullCrc = (int) crc.getValue(); + + if (readFullCrc != computeFullCrc) + return CorruptFrame.recoverable(isSelfContained, uncompressedLength, readFullCrc, computeFullCrc); + + if (uncompressedLength == 0) + { + return new IntactFrame(isSelfContained, bytes.slice(begin + HEADER_LENGTH, end - TRAILER_LENGTH)); + } + else + { + ByteBuffer out = allocator.get(uncompressedLength); + try + { + decompressor.decompress(input, begin + HEADER_LENGTH, out, 0, uncompressedLength); + return new IntactFrame(isSelfContained, ShareableBytes.wrap(out)); + } + catch (Throwable t) + { + allocator.put(out); + throw t; + } + } + } + + void decode(Collection into, ShareableBytes bytes) + { + // TODO: confirm in assembly output that we inline the relevant nested method calls + decode(into, bytes, HEADER_LENGTH); + } + + void addLastTo(ChannelPipeline pipeline) + { + pipeline.addLast("frameDecoderLZ4", this); + } +} diff --git a/src/java/org/apache/cassandra/net/FrameDecoderLegacy.java b/src/java/org/apache/cassandra/net/FrameDecoderLegacy.java new file mode 100644 index 000000000000..a3d7bc593ea4 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameDecoderLegacy.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.util.Collection; + +import io.netty.channel.ChannelPipeline; + +import static java.lang.Math.max; +import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD; + +/** + * {@link InboundMessageHandler} operates on frames that adhere to a certain contract + * (see {@link FrameDecoder.IntactFrame} and {@link FrameDecoder.CorruptFrame} javadoc). + * + * Legacy (pre-4.0) messaging protocol does not natively support framing, however. The job + * of {@link FrameDecoderLegacy} is turn a raw stream of messages, serialized back to back, + * into a sequence of frames that adhere to 4.0+ conventions. + */ +class FrameDecoderLegacy extends FrameDecoder +{ + private final int messagingVersion; + + private int remainingBytesInLargeMessage = 0; + + FrameDecoderLegacy(BufferPoolAllocator allocator, int messagingVersion) + { + super(allocator); + this.messagingVersion = messagingVersion; + } + + final void decode(Collection into, ShareableBytes newBytes) + { + ByteBuffer in = newBytes.get(); + try + { + if (stash != null) + { + int length = Message.serializer.inferMessageSize(stash, 0, stash.position(), messagingVersion); + while (length < 0) + { + if (!in.hasRemaining()) + return; + + if (stash.position() == stash.capacity()) + stash = ensureCapacity(stash, stash.capacity() * 2); + copyToSize(in, stash, stash.capacity()); + + length = Message.serializer.inferMessageSize(stash, 0, stash.position(), messagingVersion); + if (length >= 0 && length < stash.position()) + { + int excess = stash.position() - length; + in.position(in.position() - excess); + stash.position(length); + } + } + + final boolean isSelfContained; + if (length <= LARGE_MESSAGE_THRESHOLD) + { + isSelfContained = true; + + if (length > stash.capacity()) + stash = ensureCapacity(stash, length); + + stash.limit(length); + allocator.putUnusedPortion(stash); // we may be over capacity from earlier doubling + if (!copyToSize(in, stash, length)) + return; + } + else + { + isSelfContained = false; + remainingBytesInLargeMessage = length - stash.position(); + + stash.limit(stash.position()); + allocator.putUnusedPortion(stash); + } + + stash.flip(); + assert !isSelfContained || stash.limit() == length; + ShareableBytes stashed = ShareableBytes.wrap(stash); + into.add(new IntactFrame(isSelfContained, stashed)); + stash = null; + } + + if (remainingBytesInLargeMessage > 0) + { + if (remainingBytesInLargeMessage >= newBytes.remaining()) + { + remainingBytesInLargeMessage -= newBytes.remaining(); + into.add(new IntactFrame(false, newBytes.sliceAndConsume(newBytes.remaining()))); + return; + } + else + { + Frame frame = new IntactFrame(false, newBytes.sliceAndConsume(remainingBytesInLargeMessage)); + remainingBytesInLargeMessage = 0; + into.add(frame); + } + } + + // we loop incrementing our end pointer until we have no more complete messages, + // at which point we slice the complete messages, and stash the remainder + int begin = in.position(); + int end = begin; + int limit = in.limit(); + + if (begin == limit) + return; + + while (true) + { + int length = Message.serializer.inferMessageSize(in, end, limit, messagingVersion); + + if (length >= 0) + { + if (end + length <= limit) + { + // we have a complete message, so just bump our end pointer + end += length; + + // if we have more bytes, continue to look for another message + if (end < limit) + continue; + + // otherwise reset length, as we have accounted for it in end + length = 0; + } + } + + // we are done; if we have found any complete messages, slice them all into a single frame + if (begin < end) + into.add(new IntactFrame(true, newBytes.slice(begin, end))); + + // now consider stashing anything leftover + if (length < 0) + { + stash(newBytes, max(64, limit - end), end, limit - end); + } + else if (length > LARGE_MESSAGE_THRESHOLD) + { + remainingBytesInLargeMessage = length - (limit - end); + Frame frame = new IntactFrame(false, newBytes.slice(end, limit)); + into.add(frame); + } + else if (length > 0) + { + stash(newBytes, length, end, limit - end); + } + break; + } + } + catch (Message.InvalidLegacyProtocolMagic e) + { + into.add(CorruptFrame.unrecoverable(e.read, Message.PROTOCOL_MAGIC)); + } + finally + { + newBytes.release(); + } + } + + void addLastTo(ChannelPipeline pipeline) + { + pipeline.addLast("frameDecoderNone", this); + } +} diff --git a/src/java/org/apache/cassandra/net/FrameDecoderLegacyLZ4.java b/src/java/org/apache/cassandra/net/FrameDecoderLegacyLZ4.java new file mode 100644 index 000000000000..f2556a5c880e --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameDecoderLegacyLZ4.java @@ -0,0 +1,377 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Deque; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.compression.Lz4FrameDecoder; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4FastDecompressor; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; +import org.apache.cassandra.utils.memory.BufferPool; + +import static java.lang.Integer.reverseBytes; +import static java.lang.String.format; +import static org.apache.cassandra.net.LegacyLZ4Constants.*; +import static org.apache.cassandra.utils.ByteBufferUtil.copyBytes; + +/** + * A {@link FrameDecoder} consisting of two chained handlers: + * 1. A legacy LZ4 block decoder, described below in the description of {@link LZ4Decoder}, followed by + * 2. An instance of {@link FrameDecoderLegacy} - transforming the raw messages in the uncompressed stream + * into properly formed frames expected by {@link InboundMessageHandler} + */ +class FrameDecoderLegacyLZ4 extends FrameDecoderLegacy +{ + FrameDecoderLegacyLZ4(BufferPoolAllocator allocator, int messagingVersion) + { + super(allocator, messagingVersion); + } + + @Override + void addLastTo(ChannelPipeline pipeline) + { + pipeline.addLast("legacyLZ4Decoder", new LZ4Decoder(allocator)); + pipeline.addLast("frameDecoderNone", this); + } + + /** + * An implementation of LZ4 decoder, used for legacy (3.0, 3.11) connections. + * + * Netty's provided implementation - {@link Lz4FrameDecoder} couldn't be reused for + * two reasons: + * 1. It has very poor performance when coupled with xxHash, which we use for legacy connections - + * allocating a single-byte array and making a JNI call for every byte of the payload + * 2. It was tricky to efficiently integrate with upstream {@link FrameDecoder}, and impossible + * to make it play nicely with flow control - Netty's implementation, based on + * {@link io.netty.handler.codec.ByteToMessageDecoder}, would potentially keep triggering + * reads on its own volition for as long as its last read had no completed frames to supply + * - defying our goal to only ever trigger channel reads when explicitly requested + * + * Since the original LZ4 block format does not contains size of compressed block and size of original data + * this encoder uses format like LZ4 Java library + * written by Adrien Grand and approved by Yann Collet (author of original LZ4 library), as implemented by + * Netty's {@link Lz4FrameDecoder}, but adapted for our interaction model. + * + * 0 1 2 3 + * 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | | + * + Magic + + * | | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * |T| Compressed Length + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | Uncompressed Length + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | xxHash32 of Uncompressed Payload + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | | + * +-+ + + * | | + * + Payload + + * | | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + */ + private static class LZ4Decoder extends ChannelInboundHandlerAdapter + { + private static final XXHash32 xxhash = + XXHashFactory.fastestInstance().hash32(); + + private static final LZ4FastDecompressor decompressor = + LZ4Factory.fastestInstance().fastDecompressor(); + + private final BufferPoolAllocator allocator; + + LZ4Decoder(BufferPoolAllocator allocator) + { + this.allocator = allocator; + } + + private final Deque frames = new ArrayDeque<>(4); + + // total # of frames decoded between two subsequent invocations of channelReadComplete() + private int decodedFrameCount = 0; + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws CorruptLZ4Frame + { + assert msg instanceof BufferPoolAllocator.Wrapped; + ByteBuffer buf = ((BufferPoolAllocator.Wrapped) msg).adopt(); + // netty will probably have mis-predicted the space needed + BufferPool.putUnusedPortion(buf); + + CorruptLZ4Frame error = null; + try + { + decode(frames, ShareableBytes.wrap(buf)); + } + catch (CorruptLZ4Frame e) + { + error = e; + } + finally + { + decodedFrameCount += frames.size(); + while (!frames.isEmpty()) + ctx.fireChannelRead(frames.poll()); + } + + if (null != error) + throw error; + } + + @Override + public void channelReadComplete(ChannelHandlerContext ctx) + { + /* + * If no frames have been decoded from the entire batch of channelRead() calls, + * then we must trigger another channel read explicitly, or else risk stalling + * forever without bytes to complete the current in-flight frame. + */ + if (null != stash && decodedFrameCount == 0 && !ctx.channel().config().isAutoRead()) + ctx.read(); + + decodedFrameCount = 0; + ctx.fireChannelReadComplete(); + } + + private void decode(Collection into, ShareableBytes newBytes) throws CorruptLZ4Frame + { + try + { + doDecode(into, newBytes); + } + finally + { + newBytes.release(); + } + } + + private void doDecode(Collection into, ShareableBytes newBytes) throws CorruptLZ4Frame + { + ByteBuffer in = newBytes.get(); + + if (null != stash) + { + if (!copyToSize(in, stash, HEADER_LENGTH)) + return; + + header.read(stash, 0); + header.validate(); + + int frameLength = header.frameLength(); + stash = ensureCapacity(stash, frameLength); + + if (!copyToSize(in, stash, frameLength)) + return; + + stash.flip(); + ShareableBytes stashed = ShareableBytes.wrap(stash); + stash = null; + + try + { + into.add(decompressFrame(stashed, 0, frameLength, header)); + } + finally + { + stashed.release(); + } + } + + int begin = in.position(); + int limit = in.limit(); + while (begin < limit) + { + int remaining = limit - begin; + if (remaining < HEADER_LENGTH) + { + stash(newBytes, HEADER_LENGTH, begin, remaining); + return; + } + + header.read(in, begin); + header.validate(); + + int frameLength = header.frameLength(); + if (remaining < frameLength) + { + stash(newBytes, frameLength, begin, remaining); + return; + } + + into.add(decompressFrame(newBytes, begin, begin + frameLength, header)); + begin += frameLength; + } + } + + private ShareableBytes decompressFrame(ShareableBytes bytes, int begin, int end, Header header) throws CorruptLZ4Frame + { + ByteBuffer buf = bytes.get(); + + if (header.uncompressedLength == 0) + return bytes.slice(begin + HEADER_LENGTH, end); + + if (!header.isCompressed()) + { + validateChecksum(buf, begin + HEADER_LENGTH, header); + return bytes.slice(begin + HEADER_LENGTH, end); + } + + ByteBuffer out = allocator.get(header.uncompressedLength); + try + { + decompressor.decompress(buf, begin + HEADER_LENGTH, out, 0, header.uncompressedLength); + validateChecksum(out, 0, header); + return ShareableBytes.wrap(out); + } + catch (Throwable t) + { + BufferPool.put(out); + throw t; + } + } + + private void validateChecksum(ByteBuffer buf, int begin, Header header) throws CorruptLZ4Frame + { + int checksum = xxhash.hash(buf, begin, header.uncompressedLength, XXHASH_SEED) & XXHASH_MASK; + if (checksum != header.checksum) + except("Invalid checksum detected: %d (expected: %d)", checksum, header.checksum); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) + { + if (null != stash) + { + BufferPool.put(stash); + stash = null; + } + + while (!frames.isEmpty()) + frames.poll().release(); + + ctx.fireChannelInactive(); + } + + /* reusable container for deserialized header fields */ + private static final class Header + { + long magicNumber; + byte token; + int compressedLength; + int uncompressedLength; + int checksum; + + int frameLength() + { + return HEADER_LENGTH + compressedLength; + } + + boolean isCompressed() + { + return (token & 0xF0) == 0x20; + } + + int maxUncompressedLength() + { + return 1 << ((token & 0x0F) + 10); + } + + void read(ByteBuffer in, int begin) + { + magicNumber = in.getLong(begin + MAGIC_NUMBER_OFFSET ); + token = in.get (begin + TOKEN_OFFSET ); + compressedLength = reverseBytes(in.getInt (begin + COMPRESSED_LENGTH_OFFSET )); + uncompressedLength = reverseBytes(in.getInt (begin + UNCOMPRESSED_LENGTH_OFFSET)); + checksum = reverseBytes(in.getInt (begin + CHECKSUM_OFFSET )); + } + + void validate() throws CorruptLZ4Frame + { + if (magicNumber != MAGIC_NUMBER) + except("Invalid magic number at the beginning of an LZ4 block: %d", magicNumber); + + int blockType = token & 0xF0; + if (!(blockType == BLOCK_TYPE_COMPRESSED || blockType == BLOCK_TYPE_NON_COMPRESSED)) + except("Invalid block type encountered: %d", blockType); + + if (compressedLength < 0 || compressedLength > MAX_BLOCK_LENGTH) + except("Invalid compressedLength: %d (expected: 0-%d)", compressedLength, MAX_BLOCK_LENGTH); + + if (uncompressedLength < 0 || uncompressedLength > maxUncompressedLength()) + except("Invalid uncompressedLength: %d (expected: 0-%d)", uncompressedLength, maxUncompressedLength()); + + if ( uncompressedLength == 0 && compressedLength != 0 + || uncompressedLength != 0 && compressedLength == 0 + || !isCompressed() && uncompressedLength != compressedLength) + { + except("Stream corrupted: compressedLength(%d) and decompressedLength(%d) mismatch", compressedLength, uncompressedLength); + } + } + } + private final Header header = new Header(); + + /** + * @return {@code in} if has sufficient capacity, otherwise a replacement from {@code BufferPool} that {@code in} is copied into + */ + private ByteBuffer ensureCapacity(ByteBuffer in, int capacity) + { + if (in.capacity() >= capacity) + return in; + + ByteBuffer out = allocator.getAtLeast(capacity); + in.flip(); + out.put(in); + BufferPool.put(in); + return out; + } + + private ByteBuffer stash; + + private void stash(ShareableBytes in, int stashLength, int begin, int length) + { + ByteBuffer out = allocator.getAtLeast(stashLength); + copyBytes(in.get(), begin, out, 0, length); + out.position(length); + stash = out; + } + + static final class CorruptLZ4Frame extends IOException + { + CorruptLZ4Frame(String message) + { + super(message); + } + } + + private static void except(String format, Object... args) throws CorruptLZ4Frame + { + throw new CorruptLZ4Frame(format(format, args)); + } + } +} diff --git a/src/java/org/apache/cassandra/net/FrameDecoderUnprotected.java b/src/java/org/apache/cassandra/net/FrameDecoderUnprotected.java new file mode 100644 index 000000000000..44414e3ba3e8 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameDecoderUnprotected.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.util.Collection; + +import io.netty.channel.ChannelPipeline; + +import static org.apache.cassandra.net.FrameDecoderCrc.HEADER_LENGTH; +import static org.apache.cassandra.net.FrameDecoderCrc.isSelfContained; +import static org.apache.cassandra.net.FrameDecoderCrc.payloadLength; +import static org.apache.cassandra.net.FrameDecoderCrc.readHeader6b; +import static org.apache.cassandra.net.FrameDecoderCrc.verifyHeader6b; + +/** + * A frame decoder for unprotected frames, i.e. those without any modification or payload protection. + * This is non-standard, and useful for systems that have a trusted transport layer that want + * to avoid incurring the (very low) cost of computing a CRC. All we do is accumulate the bytes + * of the frame, verify the frame header, and pass through the bytes stripped of the header. + * + * Every on-wire frame contains: + * 1. Payload length (17 bits) + * 2. {@code isSelfContained} flag (1 bit) + * 3. Header padding (6 bits) + * 4. CRC24 of the header (24 bits) + * 5. Payload (up to 2 ^ 17 - 1 bits) + * + * 0 1 2 3 + * 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * | Payload Length |C| | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + * CRC24 of Header | | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + + * | | + * + + + * | Payload | + * + + + * | | + * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + */ +final class FrameDecoderUnprotected extends FrameDecoderWith8bHeader +{ + FrameDecoderUnprotected(BufferPoolAllocator allocator) + { + super(allocator); + } + + public static FrameDecoderUnprotected create(BufferPoolAllocator allocator) + { + return new FrameDecoderUnprotected(allocator); + } + + final long readHeader(ByteBuffer frame, int begin) + { + return readHeader6b(frame, begin); + } + + final CorruptFrame verifyHeader(long header6b) + { + return verifyHeader6b(header6b); + } + + final int frameLength(long header6b) + { + return payloadLength(header6b) + HEADER_LENGTH; + } + + final Frame unpackFrame(ShareableBytes bytes, int begin, int end, long header6b) + { + boolean isSelfContained = isSelfContained(header6b); + return new IntactFrame(isSelfContained, bytes.slice(begin + HEADER_LENGTH, end)); + } + + void decode(Collection into, ShareableBytes bytes) + { + decode(into, bytes, HEADER_LENGTH); + } + + void addLastTo(ChannelPipeline pipeline) + { + pipeline.addLast("frameDecoderUnprotected", this); + } +} diff --git a/src/java/org/apache/cassandra/net/FrameDecoderWith8bHeader.java b/src/java/org/apache/cassandra/net/FrameDecoderWith8bHeader.java new file mode 100644 index 000000000000..ed87d8272f0a --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameDecoderWith8bHeader.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.util.Collection; + +import net.nicoulaj.compilecommand.annotations.Inline; + +/** + * An abstract frame decoder for frames utilising a fixed length header of 8 bytes or smaller. + * Implements a generic frame decode method, that is backed by the four abstract methods + * (three of which simply decode and verify the header as a long). + * + * Implementors are expected to declare their implementation methods final, and an outer decode + * method implemented to invoke this class' {@link #decode}, so that it may be inlined with the + * abstract method implementations then inlined into it. + */ +abstract class FrameDecoderWith8bHeader extends FrameDecoder +{ + FrameDecoderWith8bHeader(BufferPoolAllocator allocator) + { + super(allocator); + } + + /** + * Read a header that is 8 bytes or shorter, without modifying the buffer position. + * If your header is longer than this, you will need to implement your own {@link #decode} + */ + abstract long readHeader(ByteBuffer in, int begin); + /** + * Verify the header, and return an unrecoverable CorruptFrame if it is corrupted + * @return null or CorruptFrame.unrecoverable + */ + abstract CorruptFrame verifyHeader(long header); + + /** + * Calculate the full frame length from info provided by the header, including the length of the header and any triler + */ + abstract int frameLength(long header); + + /** + * Extract a frame known to cover the given range. + * If {@code transferOwnership}, the method is responsible for ensuring bytes.release() is invoked at some future point. + */ + abstract Frame unpackFrame(ShareableBytes bytes, int begin, int end, long header); + + /** + * Decode a number of frames using the above abstract method implementations. + * It is expected for this method to be invoked by the implementing class' {@link #decode(Collection, ShareableBytes)} + * so that this implementation will be inlined, and all of the abstract method implementations will also be inlined. + */ + @Inline + protected void decode(Collection into, ShareableBytes newBytes, int headerLength) + { + ByteBuffer in = newBytes.get(); + + try + { + if (stash != null) + { + if (!copyToSize(in, stash, headerLength)) + return; + + long header = readHeader(stash, 0); + CorruptFrame c = verifyHeader(header); + if (c != null) + { + discard(); + into.add(c); + return; + } + + int frameLength = frameLength(header); + stash = ensureCapacity(stash, frameLength); + + if (!copyToSize(in, stash, frameLength)) + return; + + stash.flip(); + ShareableBytes stashed = ShareableBytes.wrap(stash); + stash = null; + + try + { + into.add(unpackFrame(stashed, 0, frameLength, header)); + } + finally + { + stashed.release(); + } + } + + int begin = in.position(); + int limit = in.limit(); + while (begin < limit) + { + int remaining = limit - begin; + if (remaining < headerLength) + { + stash(newBytes, headerLength, begin, remaining); + return; + } + + long header = readHeader(in, begin); + CorruptFrame c = verifyHeader(header); + if (c != null) + { + into.add(c); + return; + } + + int frameLength = frameLength(header); + if (remaining < frameLength) + { + stash(newBytes, frameLength, begin, remaining); + return; + } + + into.add(unpackFrame(newBytes, begin, begin + frameLength, header)); + begin += frameLength; + } + } + finally + { + newBytes.release(); + } + } +} diff --git a/src/java/org/apache/cassandra/net/FrameEncoder.java b/src/java/org/apache/cassandra/net/FrameEncoder.java new file mode 100644 index 000000000000..d9df1666b785 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameEncoder.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.memory.BufferPool; + +abstract class FrameEncoder extends ChannelOutboundHandlerAdapter +{ + /** + * An abstraction useful for transparently allocating buffers that can be written to upstream + * of the {@code FrameEncoder} without knowledge of the encoder's frame layout, while ensuring + * enough space to write the remainder of the frame's contents is reserved. + */ + static class Payload + { + // isSelfContained is a flag in the Frame API, indicating if the contents consists of only complete messages + private boolean isSelfContained; + // the buffer to write to + final ByteBuffer buffer; + // the number of header bytes to reserve + final int headerLength; + // the number of trailer bytes to reserve + final int trailerLength; + // an API-misuse detector + private boolean isFinished = false; + + Payload(boolean isSelfContained, int payloadCapacity) + { + this(isSelfContained, payloadCapacity, 0, 0); + } + + Payload(boolean isSelfContained, int payloadCapacity, int headerLength, int trailerLength) + { + this.isSelfContained = isSelfContained; + this.headerLength = headerLength; + this.trailerLength = trailerLength; + + buffer = BufferPool.getAtLeast(payloadCapacity + headerLength + trailerLength, BufferType.OFF_HEAP); + assert buffer.capacity() >= payloadCapacity + headerLength + trailerLength; + buffer.position(headerLength); + buffer.limit(buffer.capacity() - trailerLength); + } + + void setSelfContained(boolean isSelfContained) + { + this.isSelfContained = isSelfContained; + } + + // do not invoke after finish() + boolean isEmpty() + { + assert !isFinished; + return buffer.position() == headerLength; + } + + // do not invoke after finish() + int length() + { + assert !isFinished; + return buffer.position() - headerLength; + } + + // do not invoke after finish() + int remaining() + { + assert !isFinished; + return buffer.remaining(); + } + + // do not invoke after finish() + void trim(int length) + { + assert !isFinished; + buffer.position(headerLength + length); + } + + // may not be written to or queried, after this is invoked; must be passed straight to an encoder (or release called) + void finish() + { + assert !isFinished; + isFinished = true; + buffer.limit(buffer.position() + trailerLength); + buffer.position(0); + BufferPool.putUnusedPortion(buffer); + } + + void release() + { + BufferPool.put(buffer); + } + } + + interface PayloadAllocator + { + public static final PayloadAllocator simple = Payload::new; + Payload allocate(boolean isSelfContained, int capacity); + } + + PayloadAllocator allocator() + { + return PayloadAllocator.simple; + } + + /** + * Takes ownership of the lifetime of the provided buffer, which can be assumed to be managed by BufferPool + */ + abstract ByteBuf encode(boolean isSelfContained, ByteBuffer buffer); + + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception + { + if (!(msg instanceof Payload)) + throw new IllegalStateException("Unexpected type: " + msg); + + Payload payload = (Payload) msg; + ByteBuf write = encode(payload.isSelfContained, payload.buffer); + ctx.write(write, promise); + } +} diff --git a/src/java/org/apache/cassandra/net/FrameEncoderCrc.java b/src/java/org/apache/cassandra/net/FrameEncoderCrc.java new file mode 100644 index 000000000000..2d07d6d1cbc0 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameEncoderCrc.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.zip.CRC32; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import org.apache.cassandra.utils.memory.BufferPool; + +import static org.apache.cassandra.net.Crc.*; + +/** + * Please see {@link FrameDecoderCrc} for description of the framing produced by this encoder. + */ +@ChannelHandler.Sharable +class FrameEncoderCrc extends FrameEncoder +{ + static final int HEADER_LENGTH = 6; + private static final int TRAILER_LENGTH = 4; + static final int HEADER_AND_TRAILER_LENGTH = 10; + + static final FrameEncoderCrc instance = new FrameEncoderCrc(); + static final PayloadAllocator allocator = (isSelfContained, capacity) -> + new Payload(isSelfContained, capacity, HEADER_LENGTH, TRAILER_LENGTH); + + PayloadAllocator allocator() + { + return allocator; + } + + static void writeHeader(ByteBuffer frame, boolean isSelfContained, int dataLength) + { + int header3b = dataLength; + if (isSelfContained) + header3b |= 1 << 17; + int crc = crc24(header3b, 3); + + put3b(frame, 0, header3b); + put3b(frame, 3, crc); + } + + private static void put3b(ByteBuffer frame, int index, int put3b) + { + frame.put(index , (byte) put3b ); + frame.put(index + 1, (byte)(put3b >>> 8) ); + frame.put(index + 2, (byte)(put3b >>> 16)); + } + + ByteBuf encode(boolean isSelfContained, ByteBuffer frame) + { + try + { + int frameLength = frame.remaining(); + int dataLength = frameLength - HEADER_AND_TRAILER_LENGTH; + if (dataLength >= 1 << 17) + throw new IllegalArgumentException("Maximum payload size is 128KiB"); + + writeHeader(frame, isSelfContained, dataLength); + + CRC32 crc = crc32(); + frame.position(HEADER_LENGTH); + frame.limit(dataLength + HEADER_LENGTH); + crc.update(frame); + + int frameCrc = (int) crc.getValue(); + if (frame.order() == ByteOrder.BIG_ENDIAN) + frameCrc = Integer.reverseBytes(frameCrc); + + frame.limit(frameLength); + frame.putInt(frameLength - TRAILER_LENGTH, frameCrc); + frame.position(0); + return GlobalBufferPoolAllocator.wrap(frame); + } + catch (Throwable t) + { + BufferPool.put(frame); + throw t; + } + } +} diff --git a/src/java/org/apache/cassandra/net/FrameEncoderLZ4.java b/src/java/org/apache/cassandra/net/FrameEncoderLZ4.java new file mode 100644 index 000000000000..12351ce887ba --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameEncoderLZ4.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.zip.CRC32; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.memory.BufferPool; + +import static org.apache.cassandra.net.Crc.*; + +/** + * Please see {@link FrameDecoderLZ4} for description of the framing produced by this encoder. + */ +@ChannelHandler.Sharable +class FrameEncoderLZ4 extends FrameEncoder +{ + static final FrameEncoderLZ4 fastInstance = new FrameEncoderLZ4(LZ4Factory.fastestInstance().fastCompressor()); + + private final LZ4Compressor compressor; + + private FrameEncoderLZ4(LZ4Compressor compressor) + { + this.compressor = compressor; + } + + private static final int HEADER_LENGTH = 8; + static final int HEADER_AND_TRAILER_LENGTH = 12; + + private static void writeHeader(ByteBuffer frame, boolean isSelfContained, long compressedLength, long uncompressedLength) + { + long header5b = compressedLength | (uncompressedLength << 17); + if (isSelfContained) + header5b |= 1L << 34; + + long crc = crc24(header5b, 5); + + long header8b = header5b | (crc << 40); + if (frame.order() == ByteOrder.BIG_ENDIAN) + header8b = Long.reverseBytes(header8b); + + frame.putLong(0, header8b); + } + + public ByteBuf encode(boolean isSelfContained, ByteBuffer in) + { + ByteBuffer frame = null; + try + { + int uncompressedLength = in.remaining(); + if (uncompressedLength >= 1 << 17) + throw new IllegalArgumentException("Maximum uncompressed payload size is 128KiB"); + + int maxOutputLength = compressor.maxCompressedLength(uncompressedLength); + frame = BufferPool.getAtLeast(HEADER_AND_TRAILER_LENGTH + maxOutputLength, BufferType.OFF_HEAP); + + int compressedLength = compressor.compress(in, in.position(), uncompressedLength, frame, HEADER_LENGTH, maxOutputLength); + + if (compressedLength >= uncompressedLength) + { + ByteBufferUtil.copyBytes(in, in.position(), frame, HEADER_LENGTH, uncompressedLength); + compressedLength = uncompressedLength; + uncompressedLength = 0; + } + + writeHeader(frame, isSelfContained, compressedLength, uncompressedLength); + + CRC32 crc = crc32(); + frame.position(HEADER_LENGTH); + frame.limit(compressedLength + HEADER_LENGTH); + crc.update(frame); + + int frameCrc = (int) crc.getValue(); + if (frame.order() == ByteOrder.BIG_ENDIAN) + frameCrc = Integer.reverseBytes(frameCrc); + int frameLength = compressedLength + HEADER_AND_TRAILER_LENGTH; + + frame.limit(frameLength); + frame.putInt(frameCrc); + frame.position(0); + + BufferPool.putUnusedPortion(frame); + return GlobalBufferPoolAllocator.wrap(frame); + } + catch (Throwable t) + { + if (frame != null) + BufferPool.put(frame); + throw t; + } + finally + { + BufferPool.put(in); + } + } +} diff --git a/src/java/org/apache/cassandra/net/FrameEncoderLegacy.java b/src/java/org/apache/cassandra/net/FrameEncoderLegacy.java new file mode 100644 index 000000000000..8bfd2678ad1b --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameEncoderLegacy.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; + +/** + * A no-op frame encoder: legacy format doesn't support framing. Instead, the byte stream + * contains messages, serialized back to back. + */ +@ChannelHandler.Sharable +class FrameEncoderLegacy extends FrameEncoder +{ + static final FrameEncoderLegacy instance = new FrameEncoderLegacy(); + + ByteBuf encode(boolean isSelfContained, ByteBuffer buffer) + { + return GlobalBufferPoolAllocator.wrap(buffer); + } +} diff --git a/src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java b/src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java new file mode 100644 index 000000000000..3b29ecb7ae56 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameEncoderLegacyLZ4.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.memory.BufferPool; + +import static java.lang.Integer.reverseBytes; +import static java.lang.Math.min; +import static org.apache.cassandra.net.LegacyLZ4Constants.*; + +/** + * LZ4 {@link FrameEncoder} implementation for compressed legacy (3.0, 3.11) connections. + * + * Netty's provided implementation - {@link io.netty.handler.codec.compression.Lz4FrameEncoder} couldn't be reused + * for two reasons: + * 1. It notifies flushes as successful when they may not be, by flushing an empty buffer ahead + * of the compressed buffer + * 2. It has very poor performance when coupled with xxHash, which we use for legacy connections - + * allocating a single-byte array and making a JNI call for every byte of the payload + * + * Please see {@link FrameDecoderLegacyLZ4} for the description of the on-wire format of the LZ4 blocks + * used by this encoder. + */ +@ChannelHandler.Sharable +class FrameEncoderLegacyLZ4 extends FrameEncoder +{ + static final FrameEncoderLegacyLZ4 instance = + new FrameEncoderLegacyLZ4(XXHashFactory.fastestInstance().hash32(), + LZ4Factory.fastestInstance().fastCompressor()); + + private final XXHash32 xxhash; + private final LZ4Compressor compressor; + + private FrameEncoderLegacyLZ4(XXHash32 xxhash, LZ4Compressor compressor) + { + this.xxhash = xxhash; + this.compressor = compressor; + } + + @Override + ByteBuf encode(boolean isSelfContained, ByteBuffer payload) + { + ByteBuffer frame = null; + try + { + frame = BufferPool.getAtLeast(calculateMaxFrameLength(payload), BufferType.OFF_HEAP); + + int frameOffset = 0; + int payloadOffset = 0; + + int payloadLength = payload.remaining(); + while (payloadOffset < payloadLength) + { + int blockLength = min(DEFAULT_BLOCK_LENGTH, payloadLength - payloadOffset); + frameOffset += compressBlock(frame, frameOffset, payload, payloadOffset, blockLength); + payloadOffset += blockLength; + } + + frame.limit(frameOffset); + BufferPool.putUnusedPortion(frame); + + return GlobalBufferPoolAllocator.wrap(frame); + } + catch (Throwable t) + { + if (null != frame) + BufferPool.put(frame); + throw t; + } + finally + { + BufferPool.put(payload); + } + } + + private int compressBlock(ByteBuffer frame, int frameOffset, ByteBuffer payload, int payloadOffset, int blockLength) + { + int frameBytesRemaining = frame.limit() - (frameOffset + HEADER_LENGTH); + int compressedLength = compressor.compress(payload, payloadOffset, blockLength, frame, frameOffset + HEADER_LENGTH, frameBytesRemaining); + if (compressedLength >= blockLength) + { + ByteBufferUtil.copyBytes(payload, payloadOffset, frame, frameOffset + HEADER_LENGTH, blockLength); + compressedLength = blockLength; + } + int checksum = xxhash.hash(payload, payloadOffset, blockLength, XXHASH_SEED) & XXHASH_MASK; + writeHeader(frame, frameOffset, compressedLength, blockLength, checksum); + return HEADER_LENGTH + compressedLength; + } + + private static final byte TOKEN_NON_COMPRESSED = 0x15; + private static final byte TOKEN_COMPRESSED = 0x25; + + private static void writeHeader(ByteBuffer frame, int frameOffset, int compressedLength, int uncompressedLength, int checksum) + { + byte token = compressedLength == uncompressedLength + ? TOKEN_NON_COMPRESSED + : TOKEN_COMPRESSED; + + frame.putLong(frameOffset + MAGIC_NUMBER_OFFSET, MAGIC_NUMBER ); + frame.put (frameOffset + TOKEN_OFFSET, token ); + frame.putInt (frameOffset + COMPRESSED_LENGTH_OFFSET, reverseBytes(compressedLength) ); + frame.putInt (frameOffset + UNCOMPRESSED_LENGTH_OFFSET, reverseBytes(uncompressedLength)); + frame.putInt (frameOffset + CHECKSUM_OFFSET, reverseBytes(checksum) ); + } + + private int calculateMaxFrameLength(ByteBuffer payload) + { + int payloadLength = payload.remaining(); + int blockCount = payloadLength / DEFAULT_BLOCK_LENGTH + (payloadLength % DEFAULT_BLOCK_LENGTH != 0 ? 1 : 0); + return compressor.maxCompressedLength(payloadLength) + HEADER_LENGTH * blockCount; + } +} diff --git a/src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java b/src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java new file mode 100644 index 000000000000..3bca41c25532 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FrameEncoderUnprotected.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import org.apache.cassandra.utils.memory.BufferPool; + +import static org.apache.cassandra.net.FrameEncoderCrc.HEADER_LENGTH; +import static org.apache.cassandra.net.FrameEncoderCrc.writeHeader; + +/** + * A frame encoder that writes frames, just without any modification or payload protection. + * This is non-standard, and useful for systems that have a trusted transport layer that want + * to avoid incurring the (very low) cost of computing a CRC. + * + * Please see {@link FrameDecoderUnprotected} for description of the framing produced by this encoder. + */ +@ChannelHandler.Sharable +class FrameEncoderUnprotected extends FrameEncoder +{ + static final FrameEncoderUnprotected instance = new FrameEncoderUnprotected(); + static final PayloadAllocator allocator = (isSelfContained, capacity) -> + new Payload(isSelfContained, capacity, HEADER_LENGTH, 0); + + PayloadAllocator allocator() + { + return allocator; + } + + ByteBuf encode(boolean isSelfContained, ByteBuffer frame) + { + try + { + int frameLength = frame.remaining(); + int dataLength = frameLength - HEADER_LENGTH; + if (dataLength >= 1 << 17) + throw new IllegalArgumentException("Maximum uncompressed payload size is 128KiB"); + + writeHeader(frame, isSelfContained, dataLength); + return GlobalBufferPoolAllocator.wrap(frame); + } + catch (Throwable t) + { + BufferPool.put(frame); + throw t; + } + } +} diff --git a/src/java/org/apache/cassandra/net/FutureCombiner.java b/src/java/org/apache/cassandra/net/FutureCombiner.java new file mode 100644 index 000000000000..dd094bdcfdb6 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FutureCombiner.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.Collection; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; + +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import io.netty.util.concurrent.GlobalEventExecutor; +import io.netty.util.concurrent.Promise; + +/** + * Netty's PromiseCombiner is not threadsafe, and we combine futures from multiple event executors. + * + * This class groups a number of Future into a single logical Future, by registering a listener to each that + * decrements a shared counter; if any of them fail, the FutureCombiner is completed with the first cause, + * but in all scenario only completes when all underlying future have completed (exceptionally or otherwise) + * + * This Future is always uncancellable. + * + * We extend FutureDelegate, and simply provide it an uncancellable Promise that will be completed by the listeners + * registered to the input futures. + */ +class FutureCombiner extends FutureDelegate +{ + private volatile boolean failed; + + private volatile Throwable firstCause; + private static final AtomicReferenceFieldUpdater firstCauseUpdater = + AtomicReferenceFieldUpdater.newUpdater(FutureCombiner.class, Throwable.class, "firstCause"); + + private volatile int waitingOn; + private static final AtomicIntegerFieldUpdater waitingOnUpdater = + AtomicIntegerFieldUpdater.newUpdater(FutureCombiner.class, "waitingOn"); + + FutureCombiner(Collection> combine) + { + this(AsyncPromise.uncancellable(GlobalEventExecutor.INSTANCE), combine); + } + + private FutureCombiner(Promise combined, Collection> combine) + { + super(combined); + + if (0 == (waitingOn = combine.size())) + combined.trySuccess(null); + + GenericFutureListener> listener = result -> + { + if (!result.isSuccess()) + { + firstCauseUpdater.compareAndSet(this, null, result.cause()); + failed = true; + } + + if (0 == waitingOnUpdater.decrementAndGet(this)) + { + if (failed) + combined.tryFailure(firstCause); + else + combined.trySuccess(null); + } + }; + + for (Future future : combine) + future.addListener(listener); + } +} diff --git a/src/java/org/apache/cassandra/net/FutureDelegate.java b/src/java/org/apache/cassandra/net/FutureDelegate.java new file mode 100644 index 000000000000..f04a43275fd0 --- /dev/null +++ b/src/java/org/apache/cassandra/net/FutureDelegate.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; + +/** + * A delegating future, that we can extend to provide subtly modified behaviour. + * + * See {@link FutureCombiner} and {@link FutureResult} + */ +class FutureDelegate implements Future +{ + final Future delegate; + + FutureDelegate(Future delegate) + { + this.delegate = delegate; + } + + public boolean isSuccess() + { + return delegate.isSuccess(); + } + + public boolean isCancellable() + { + return delegate.isCancellable(); + } + + public Throwable cause() + { + return delegate.cause(); + } + + public Future addListener(GenericFutureListener> genericFutureListener) + { + return delegate.addListener(genericFutureListener); + } + + public Future addListeners(GenericFutureListener>... genericFutureListeners) + { + return delegate.addListeners(genericFutureListeners); + } + + public Future removeListener(GenericFutureListener> genericFutureListener) + { + return delegate.removeListener(genericFutureListener); + } + + public Future removeListeners(GenericFutureListener>... genericFutureListeners) + { + return delegate.removeListeners(genericFutureListeners); + } + + public Future sync() throws InterruptedException + { + return delegate.sync(); + } + + public Future syncUninterruptibly() + { + return delegate.syncUninterruptibly(); + } + + public Future await() throws InterruptedException + { + return delegate.await(); + } + + public Future awaitUninterruptibly() + { + return delegate.awaitUninterruptibly(); + } + + public boolean await(long l, TimeUnit timeUnit) throws InterruptedException + { + return delegate.await(l, timeUnit); + } + + public boolean await(long l) throws InterruptedException + { + return delegate.await(l); + } + + public boolean awaitUninterruptibly(long l, TimeUnit timeUnit) + { + return delegate.awaitUninterruptibly(l, timeUnit); + } + + public boolean awaitUninterruptibly(long l) + { + return delegate.awaitUninterruptibly(l); + } + + public V getNow() + { + return delegate.getNow(); + } + + public boolean cancel(boolean b) + { + return delegate.cancel(b); + } + + public boolean isCancelled() + { + return delegate.isCancelled(); + } + + public boolean isDone() + { + return delegate.isDone(); + } + + public V get() throws InterruptedException, ExecutionException + { + return delegate.get(); + } + + public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException + { + return delegate.get(timeout, unit); + } +} diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/FutureResult.java similarity index 54% rename from src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java rename to src/java/org/apache/cassandra/net/FutureResult.java index e0be7151b1ef..8d43dbe39a78 100644 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataInputPlus.java +++ b/src/java/org/apache/cassandra/net/FutureResult.java @@ -15,37 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.cassandra.net; -package org.apache.cassandra.net.async; +import io.netty.util.concurrent.Future; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufInputStream; -import org.apache.cassandra.io.util.DataInputPlus; - -import java.io.IOException; - -public class ByteBufDataInputPlus extends ByteBufInputStream implements DataInputPlus +/** + * An abstraction for yielding a result performed by an asynchronous task, + * for whom we may wish to offer cancellation, + * but no other access to the underlying task + */ +class FutureResult extends FutureDelegate { + private final Future tryCancel; + /** - * The parent class does not expose the buffer to derived classes, so we need - * to stash a reference here so it can be exposed via {@link #buffer()}. + * @param result the Future that will be completed by {@link #cancel} + * @param cancel the Future that is performing the work, and to whom any cancellation attempts will be proxied */ - private final ByteBuf buf; - - public ByteBufDataInputPlus(ByteBuf buffer) - { - super(buffer); - this.buf = buffer; - } - - public ByteBuf buffer() + FutureResult(Future result, Future cancel) { - return buf; + super(result); + this.tryCancel = cancel; } @Override - public String readUTF() throws IOException + public boolean cancel(boolean b) { - return DataInputStreamPlus.readUTF(this); + tryCancel.cancel(true); + return delegate.cancel(b); } } diff --git a/test/unit/org/apache/cassandra/net/async/TestAuthenticator.java b/src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java similarity index 60% rename from test/unit/org/apache/cassandra/net/async/TestAuthenticator.java rename to src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java index 3107f2aba0bc..66cbc9e1ebb0 100644 --- a/test/unit/org/apache/cassandra/net/async/TestAuthenticator.java +++ b/src/java/org/apache/cassandra/net/GlobalBufferPoolAllocator.java @@ -15,28 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.cassandra.net; -package org.apache.cassandra.net.async; +import java.nio.ByteBuffer; -import java.net.InetAddress; +import io.netty.buffer.ByteBuf; +import org.apache.cassandra.utils.memory.BufferPool; -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.exceptions.ConfigurationException; - -class TestAuthenticator implements IInternodeAuthenticator +/** + * Primary {@link ByteBuf} / {@link ByteBuffer} allocator - using the global {@link BufferPool}. + */ +class GlobalBufferPoolAllocator extends BufferPoolAllocator { - private final boolean authAll; + static final GlobalBufferPoolAllocator instance = new GlobalBufferPoolAllocator(); - TestAuthenticator(boolean authAll) + private GlobalBufferPoolAllocator() { - this.authAll = authAll; + super(); } - public boolean authenticate(InetAddress remoteAddress, int remotePort) + static ByteBuf wrap(ByteBuffer buffer) { - return authAll; + return new Wrapped(instance, buffer); } - - public void validateConfiguration() throws ConfigurationException - { } } diff --git a/src/java/org/apache/cassandra/net/HandshakeProtocol.java b/src/java/org/apache/cassandra/net/HandshakeProtocol.java new file mode 100644 index 000000000000..47d0ec6dffa3 --- /dev/null +++ b/src/java/org/apache/cassandra/net/HandshakeProtocol.java @@ -0,0 +1,414 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +import com.google.common.annotations.VisibleForTesting; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputBufferFixed; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.utils.memory.BufferPool; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; +import static org.apache.cassandra.net.MessagingService.VERSION_30; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.Message.validateLegacyProtocolMagic; +import static org.apache.cassandra.net.Crc.*; +import static org.apache.cassandra.net.Crc.computeCrc32; +import static org.apache.cassandra.net.OutboundConnectionSettings.*; + +/** + * Messages for the handshake phase of the internode protocol. + * + * The modern handshake is composed of 2 messages: Initiate and Accept + *

+ * The legacy handshake is composed of 3 messages, the first being sent by the initiator of the connection. The other + * side then answer with the 2nd message. At that point, if a version mismatch is detected by the connection initiator, + * it will simply disconnect and reconnect with a more appropriate version. But if the version is acceptable, the connection + * initiator sends the third message of the protocol, after which it considers the connection ready. + */ +class HandshakeProtocol +{ + static final long TIMEOUT_MILLIS = 3 * DatabaseDescriptor.getRpcTimeout(MILLISECONDS); + + /** + * The initial message sent when a node creates a new connection to a remote peer. This message contains: + * 1) the {@link Message#PROTOCOL_MAGIC} number (4 bytes). + * 2) the connection flags (4 bytes), which encodes: + * - the version the initiator thinks should be used for the connection (in practice, either the initiator + * version if it's the first time we connect to that remote since startup, or the last version known for that + * peer otherwise). + * - the "mode" of the connection: whether it is for streaming or for messaging. + * - whether compression should be used or not (if it is, compression is enabled _after_ the last message of the + * handshake has been sent). + * 3) the connection initiator's broadcast address + * 4) a CRC protecting the message from corruption + *

+ * More precisely, connection flags: + *

+     * {@code
+     *                      1 1 1 1 1 1 1 1 1 1 2 2 2 2 2 2 2 2 2 2 3 3
+     *  0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * |C C C M C      |    REQUEST    |      MIN      |      MAX      |
+     * |A A M O R      |    VERSION    |   SUPPORTED   |   SUPPORTED   |
+     * |T T P D C      |  (DEPRECATED) |    VERSION    |    VERSION    |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * }
+     * 
+ * CAT - QOS category, 2 bits: SMALL, LARGE, URGENT, or LEGACY (unset) + * CMP - compression enabled bit + * MOD - connection mode; if the bit is on, the connection is for streaming; if the bit is off, it is for inter-node messaging. + * CRC - crc enabled bit + * VERSION - {@link org.apache.cassandra.net.MessagingService#current_version} + */ + static class Initiate + { + /** Contains the PROTOCOL_MAGIC (int) and the flags (int). */ + private static final int MIN_LENGTH = 8; + private static final int MAX_LENGTH = 12 + InetAddressAndPort.Serializer.MAXIMUM_SIZE; + + @Deprecated // this is ignored by post40 nodes, i.e. if maxMessagingVersion is set + final int requestMessagingVersion; + // the messagingVersion bounds the sender will accept to initiate a connection; + // if the remote peer supports any, the newest supported version will be selected; otherwise the nearest supported version + final AcceptVersions acceptVersions; + final ConnectionType type; + final Framing framing; + final InetAddressAndPort from; + + Initiate(int requestMessagingVersion, AcceptVersions acceptVersions, ConnectionType type, Framing framing, InetAddressAndPort from) + { + this.requestMessagingVersion = requestMessagingVersion; + this.acceptVersions = acceptVersions; + this.type = type; + this.framing = framing; + this.from = from; + } + + @VisibleForTesting + int encodeFlags() + { + int flags = 0; + if (type.isMessaging()) + flags |= type.twoBitID(); + if (type.isStreaming()) + flags |= 1 << 3; + + // framing id is split over 2nd and 4th bits, for backwards compatibility + flags |= ((framing.id & 1) << 2) | ((framing.id & 2) << 3); + flags |= (requestMessagingVersion << 8); + + if (requestMessagingVersion < VERSION_40 || acceptVersions.max < VERSION_40) + return flags; // for testing, permit serializing as though we are pre40 + + flags |= (acceptVersions.min << 16); + flags |= (acceptVersions.max << 24); + return flags; + } + + ByteBuf encode() + { + ByteBuffer buffer = BufferPool.get(MAX_LENGTH, BufferType.OFF_HEAP); + try (DataOutputBufferFixed out = new DataOutputBufferFixed(buffer)) + { + out.writeInt(Message.PROTOCOL_MAGIC); + out.writeInt(encodeFlags()); + + if (requestMessagingVersion >= VERSION_40 && acceptVersions.max >= VERSION_40) + { + inetAddressAndPortSerializer.serialize(from, out, requestMessagingVersion); + out.writeInt(computeCrc32(buffer, 0, buffer.position())); + } + buffer.flip(); + return GlobalBufferPoolAllocator.wrap(buffer); + } + catch (IOException e) + { + throw new IllegalStateException(e); + } + } + + static Initiate maybeDecode(ByteBuf buf) throws IOException + { + if (buf.readableBytes() < MIN_LENGTH) + return null; + + ByteBuffer nio = buf.nioBuffer(); + int start = nio.position(); + try (DataInputBuffer in = new DataInputBuffer(nio, false)) + { + validateLegacyProtocolMagic(in.readInt()); + int flags = in.readInt(); + + int requestedMessagingVersion = getBits(flags, 8, 8); + int minMessagingVersion = getBits(flags, 16, 8); + int maxMessagingVersion = getBits(flags, 24, 8); + int framingBits = getBits(flags, 2, 1) | (getBits(flags, 4, 1) << 1); + Framing framing = Framing.forId(framingBits); + + boolean isStream = getBits(flags, 3, 1) == 1; + + ConnectionType type = isStream + ? ConnectionType.STREAMING + : ConnectionType.fromId(getBits(flags, 0, 2)); + + InetAddressAndPort from = null; + + if (requestedMessagingVersion >= VERSION_40 && maxMessagingVersion >= MessagingService.VERSION_40) + { + from = inetAddressAndPortSerializer.deserialize(in, requestedMessagingVersion); + + int computed = computeCrc32(nio, start, nio.position()); + int read = in.readInt(); + if (read != computed) + throw new InvalidCrc(read, computed); + } + + buf.skipBytes(nio.position() - start); + return new Initiate(requestedMessagingVersion, + minMessagingVersion == 0 && maxMessagingVersion == 0 + ? null : new AcceptVersions(minMessagingVersion, maxMessagingVersion), + type, framing, from); + + } + catch (EOFException e) + { + return null; + } + } + + @VisibleForTesting + @Override + public boolean equals(Object other) + { + if (!(other instanceof Initiate)) + return false; + + Initiate that = (Initiate)other; + return this.type == that.type + && this.framing == that.framing + && this.requestMessagingVersion == that.requestMessagingVersion + && Objects.equals(this.acceptVersions, that.acceptVersions); + } + + @Override + public String toString() + { + return String.format("Initiate(request: %d, min: %d, max: %d, type: %s, framing: %b, from: %s)", + requestMessagingVersion, + acceptVersions == null ? requestMessagingVersion : acceptVersions.min, + acceptVersions == null ? requestMessagingVersion : acceptVersions.max, + type, framing, from); + } + } + + + /** + * The second message of the handshake, sent by the node receiving the {@link Initiate} back to the + * connection initiator. + * + * This message contains + * 1) the messaging version of the peer sending this message + * 2) the negotiated messaging version if one could be accepted by both peers, + * or if not the closest version that this peer could support to the ones requested + * 3) a CRC protectingn the integrity of the message + * + * Note that the pre40 equivalent of this message contains ONLY the messaging version of the peer. + */ + static class Accept + { + /** The messaging version sent by the receiving peer (int). */ + private static final int MAX_LENGTH = 12; + + final int useMessagingVersion; + final int maxMessagingVersion; + + Accept(int useMessagingVersion, int maxMessagingVersion) + { + this.useMessagingVersion = useMessagingVersion; + this.maxMessagingVersion = maxMessagingVersion; + } + + ByteBuf encode(ByteBufAllocator allocator) + { + ByteBuf buffer = allocator.directBuffer(MAX_LENGTH); + buffer.clear(); + buffer.writeInt(maxMessagingVersion); + buffer.writeInt(useMessagingVersion); + buffer.writeInt(computeCrc32(buffer, 0, 8)); + return buffer; + } + + /** + * Respond to pre40 nodes only with our current messagingVersion + */ + static ByteBuf respondPre40(int messagingVersion, ByteBufAllocator allocator) + { + ByteBuf buffer = allocator.directBuffer(4); + buffer.clear(); + buffer.writeInt(messagingVersion); + return buffer; + } + + static Accept maybeDecode(ByteBuf in, int handshakeMessagingVersion) throws InvalidCrc + { + int readerIndex = in.readerIndex(); + if (in.readableBytes() < 4) + return null; + int maxMessagingVersion = in.readInt(); + int useMessagingVersion = 0; + + // if the other node is pre-4.0, it will respond only with its maxMessagingVersion + if (maxMessagingVersion < VERSION_40 || handshakeMessagingVersion < VERSION_40) + return new Accept(useMessagingVersion, maxMessagingVersion); + + if (in.readableBytes() < 8) + { + in.readerIndex(readerIndex); + return null; + } + useMessagingVersion = in.readInt(); + + // verify crc + int computed = computeCrc32(in, readerIndex, readerIndex + 8); + int read = in.readInt(); + if (read != computed) + throw new InvalidCrc(read, computed); + + return new Accept(useMessagingVersion, maxMessagingVersion); + } + + @VisibleForTesting + @Override + public boolean equals(Object other) + { + return other instanceof Accept + && this.useMessagingVersion == ((Accept) other).useMessagingVersion + && this.maxMessagingVersion == ((Accept) other).maxMessagingVersion; + } + + @Override + public String toString() + { + return String.format("Accept(use: %d, max: %d)", useMessagingVersion, maxMessagingVersion); + } + } + + /** + * The third message of the handshake, sent by pre40 nodes on reception of {@link Accept}. + * This message contains: + * 1) The connection initiator's {@link org.apache.cassandra.net.MessagingService#current_version} (4 bytes). + * This indicates the max messaging version supported by this node. + * 2) The connection initiator's broadcast address as encoded by {@link InetAddressAndPort.Serializer}. + * This can be either 7 bytes for an IPv4 address, or 19 bytes for an IPv6 one, post40. + * This can be either 5 bytes for an IPv4 address, or 17 bytes for an IPv6 one, pre40. + *

+ * This message concludes the legacy handshake protocol. + */ + static class ConfirmOutboundPre40 + { + private static final int MAX_LENGTH = 4 + InetAddressAndPort.Serializer.MAXIMUM_SIZE; + + final int maxMessagingVersion; + final InetAddressAndPort from; + + ConfirmOutboundPre40(int maxMessagingVersion, InetAddressAndPort from) + { + this.maxMessagingVersion = maxMessagingVersion; + this.from = from; + } + + ByteBuf encode() + { + ByteBuffer buffer = BufferPool.get(MAX_LENGTH, BufferType.OFF_HEAP); + try (DataOutputBufferFixed out = new DataOutputBufferFixed(buffer)) + { + out.writeInt(maxMessagingVersion); + // pre-4.0 nodes should only receive the address, never port, and it's ok to hardcode VERSION_30 + inetAddressAndPortSerializer.serialize(from, out, VERSION_30); + buffer.flip(); + return GlobalBufferPoolAllocator.wrap(buffer); + } + catch (IOException e) + { + throw new IllegalStateException(e); + } + } + + @SuppressWarnings("resource") + static ConfirmOutboundPre40 maybeDecode(ByteBuf in) + { + ByteBuffer nio = in.nioBuffer(); + int start = nio.position(); + DataInputPlus input = new DataInputBuffer(nio, false); + try + { + int version = input.readInt(); + InetAddressAndPort address = inetAddressAndPortSerializer.deserialize(input, version); + in.skipBytes(nio.position() - start); + return new ConfirmOutboundPre40(version, address); + } + catch (EOFException e) + { + // makes the assumption we didn't have enough bytes to deserialize an IPv6 address, + // as we only check the MIN_LENGTH of the buf. + return null; + } + catch (IOException e) + { + throw new IllegalStateException(e); + } + } + + @VisibleForTesting + @Override + public boolean equals(Object other) + { + if (!(other instanceof ConfirmOutboundPre40)) + return false; + + ConfirmOutboundPre40 that = (ConfirmOutboundPre40) other; + return this.maxMessagingVersion == that.maxMessagingVersion + && Objects.equals(this.from, that.from); + } + + @Override + public String toString() + { + return String.format("ConfirmOutboundPre40(maxMessagingVersion: %d; address: %s)", maxMessagingVersion, from); + } + } + + private static int getBits(int packed, int start, int count) + { + return (packed >>> start) & ~(-1 << count); + } + +} diff --git a/src/java/org/apache/cassandra/net/IVerbHandler.java b/src/java/org/apache/cassandra/net/IVerbHandler.java index 0995a68e9aed..ac0efe7359b0 100644 --- a/src/java/org/apache/cassandra/net/IVerbHandler.java +++ b/src/java/org/apache/cassandra/net/IVerbHandler.java @@ -24,7 +24,6 @@ * The concrete implementation of this interface would provide the functionality * for a given verb. */ - public interface IVerbHandler { /** @@ -34,7 +33,6 @@ public interface IVerbHandler * because the implementation may be synchronized. * * @param message - incoming message that needs handling. - * @param id */ - void doVerb(MessageIn message, int id) throws IOException; + void doVerb(Message message) throws IOException; } diff --git a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java new file mode 100644 index 000000000000..d26abfdb2529 --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java @@ -0,0 +1,495 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.List; +import java.util.concurrent.Future; +import java.util.function.Consumer; + +import javax.net.ssl.SSLSession; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.group.ChannelGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslHandler; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.OutboundConnectionSettings.Framing; +import org.apache.cassandra.security.SSLFactory; +import org.apache.cassandra.streaming.async.StreamingInboundHandler; +import org.apache.cassandra.utils.memory.BufferPool; + +import static java.lang.Math.*; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.cassandra.net.MessagingService.*; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.MessagingService.minimum_version; +import static org.apache.cassandra.net.SocketFactory.WIRETRACE; +import static org.apache.cassandra.net.SocketFactory.encryptionLogStatement; +import static org.apache.cassandra.net.SocketFactory.newSslHandler; + +public class InboundConnectionInitiator +{ + private static final Logger logger = LoggerFactory.getLogger(InboundConnectionInitiator.class); + + private static class Initializer extends ChannelInitializer + { + private final InboundConnectionSettings settings; + private final ChannelGroup channelGroup; + private final Consumer pipelineInjector; + + Initializer(InboundConnectionSettings settings, ChannelGroup channelGroup, + Consumer pipelineInjector) + { + this.settings = settings; + this.channelGroup = channelGroup; + this.pipelineInjector = pipelineInjector; + } + + @Override + public void initChannel(SocketChannel channel) throws Exception + { + channelGroup.add(channel); + + channel.config().setOption(ChannelOption.ALLOCATOR, GlobalBufferPoolAllocator.instance); + channel.config().setOption(ChannelOption.SO_KEEPALIVE, true); + channel.config().setOption(ChannelOption.SO_REUSEADDR, true); + channel.config().setOption(ChannelOption.TCP_NODELAY, true); // we only send handshake messages; no point ever delaying + + ChannelPipeline pipeline = channel.pipeline(); + + pipelineInjector.accept(pipeline); + + // order of handlers: ssl -> logger -> handshakeHandler + if (settings.encryption.enabled) + { + if (settings.encryption.optional) + { + pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption)); + } + else + { + SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER); + InetSocketAddress peer = settings.encryption.require_endpoint_verification ? channel.remoteAddress() : null; + SslHandler sslHandler = newSslHandler(channel, sslContext, peer); + logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName()); + pipeline.addFirst("ssl", sslHandler); + } + } + + if (WIRETRACE) + pipeline.addLast("logger", new LoggingHandler(LogLevel.INFO)); + + channel.pipeline().addLast("handshake", new Handler(settings)); + + } + } + + /** + * Create a {@link Channel} that listens on the {@code localAddr}. This method will block while trying to bind to the address, + * but it does not make a remote call. + */ + private static ChannelFuture bind(Initializer initializer) throws ConfigurationException + { + logger.info("Listening on {}", initializer.settings); + + ServerBootstrap bootstrap = initializer.settings.socketFactory + .newServerBootstrap() + .option(ChannelOption.SO_BACKLOG, 1 << 9) + .childHandler(initializer); + + int socketReceiveBufferSizeInBytes = initializer.settings.socketReceiveBufferSizeInBytes; + if (socketReceiveBufferSizeInBytes > 0) + bootstrap.childOption(ChannelOption.SO_RCVBUF, socketReceiveBufferSizeInBytes); + + InetAddressAndPort bind = initializer.settings.bindAddress; + ChannelFuture channelFuture = bootstrap.bind(new InetSocketAddress(bind.address, bind.port)); + + if (!channelFuture.awaitUninterruptibly().isSuccess()) + { + if (channelFuture.channel().isOpen()) + channelFuture.channel().close(); + + Throwable failedChannelCause = channelFuture.cause(); + + String causeString = ""; + if (failedChannelCause != null && failedChannelCause.getMessage() != null) + causeString = failedChannelCause.getMessage(); + + if (causeString.contains("in use")) + { + throw new ConfigurationException(bind + " is in use by another process. Change listen_address:storage_port " + + "in cassandra.yaml to values that do not conflict with other services"); + } + // looking at the jdk source, solaris/windows bind failue messages both use the phrase "cannot assign requested address". + // windows message uses "Cannot" (with a capital 'C'), and solaris (a/k/a *nux) doe not. hence we search for "annot" + else if (causeString.contains("annot assign requested address")) + { + throw new ConfigurationException("Unable to bind to address " + bind + + ". Set listen_address in cassandra.yaml to an interface you can bind to, e.g., your private IP address on EC2"); + } + else + { + throw new ConfigurationException("failed to bind to: " + bind, failedChannelCause); + } + } + + return channelFuture; + } + + public static ChannelFuture bind(InboundConnectionSettings settings, ChannelGroup channelGroup, + Consumer pipelineInjector) + { + return bind(new Initializer(settings, channelGroup, pipelineInjector)); + } + + /** + * 'Server-side' component that negotiates the internode handshake when establishing a new connection. + * This handler will be the first in the netty channel for each incoming connection (secure socket (TLS) notwithstanding), + * and once the handshake is successful, it will configure the proper handlers ({@link InboundMessageHandler} + * or {@link StreamingInboundHandler}) and remove itself from the working pipeline. + */ + static class Handler extends ByteToMessageDecoder + { + private final InboundConnectionSettings settings; + + private HandshakeProtocol.Initiate initiate; + private HandshakeProtocol.ConfirmOutboundPre40 confirmOutboundPre40; + + /** + * A future the essentially places a timeout on how long we'll wait for the peer + * to complete the next step of the handshake. + */ + private Future handshakeTimeout; + + Handler(InboundConnectionSettings settings) + { + this.settings = settings; + } + + /** + * On registration, immediately schedule a timeout to kill this connection if it does not handshake promptly, + * and authenticate the remote address. + */ + public void handlerAdded(ChannelHandlerContext ctx) throws Exception + { + handshakeTimeout = ctx.executor().schedule(() -> { + logger.error("Timeout handshaking with {} (on {})", SocketFactory.addressId(initiate.from, (InetSocketAddress) ctx.channel().remoteAddress()), settings.bindAddress); + failHandshake(ctx); + }, HandshakeProtocol.TIMEOUT_MILLIS, MILLISECONDS); + + logSsl(ctx); + authenticate(ctx.channel().remoteAddress()); + } + + private void authenticate(SocketAddress socketAddress) throws IOException + { + if (socketAddress.getClass().getSimpleName().equals("EmbeddedSocketAddress")) + return; + + if (!(socketAddress instanceof InetSocketAddress)) + throw new IOException(String.format("Unexpected SocketAddress type: %s, %s", socketAddress.getClass(), socketAddress)); + + InetSocketAddress addr = (InetSocketAddress)socketAddress; + if (!settings.authenticate(addr.getAddress(), addr.getPort())) + throw new IOException("Authentication failure for inbound connection from peer " + addr); + } + + private void logSsl(ChannelHandlerContext ctx) + { + SslHandler sslHandler = ctx.pipeline().get(SslHandler.class); + if (sslHandler != null) + { + SSLSession session = sslHandler.engine().getSession(); + logger.info("connection from peer {}, protocol = {}, cipher suite = {}", + ctx.channel().remoteAddress(), session.getProtocol(), session.getCipherSuite()); + } + } + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception + { + if (initiate == null) initiate(ctx, in); + else if (initiate.acceptVersions == null && confirmOutboundPre40 == null) confirmPre40(ctx, in); + else throw new IllegalStateException("Should no longer be on pipeline"); + } + + void initiate(ChannelHandlerContext ctx, ByteBuf in) throws IOException + { + initiate = HandshakeProtocol.Initiate.maybeDecode(in); + if (initiate == null) + return; + + logger.trace("Received handshake initiation message from peer {}, message = {}", ctx.channel().remoteAddress(), initiate); + if (initiate.acceptVersions != null) + { + logger.trace("Connection version {} (min {}) from {}", initiate.acceptVersions.max, initiate.acceptVersions.min, initiate.from); + + final AcceptVersions accept; + + if (initiate.type.isStreaming()) + accept = settings.acceptStreaming; + else + accept = settings.acceptMessaging; + + int useMessagingVersion = max(accept.min, min(accept.max, initiate.acceptVersions.max)); + ByteBuf flush = new HandshakeProtocol.Accept(useMessagingVersion, accept.max).encode(ctx.alloc()); + + AsyncChannelPromise.writeAndFlush(ctx, flush, (ChannelFutureListener) future -> { + if (!future.isSuccess()) + exceptionCaught(future.channel(), future.cause()); + }); + + if (initiate.acceptVersions.min > accept.max) + { + logger.info("peer {} only supports messaging versions higher ({}) than this node supports ({})", ctx.channel().remoteAddress(), initiate.acceptVersions.min, current_version); + failHandshake(ctx); + } + else if (initiate.acceptVersions.max < accept.min) + { + logger.info("peer {} only supports messaging versions lower ({}) than this node supports ({})", ctx.channel().remoteAddress(), initiate.acceptVersions.max, minimum_version); + failHandshake(ctx); + } + else + { + if (initiate.type.isStreaming()) + setupStreamingPipeline(initiate.from, ctx); + else + setupMessagingPipeline(initiate.from, useMessagingVersion, initiate.acceptVersions.max, ctx.pipeline()); + } + } + else + { + int version = initiate.requestMessagingVersion; + assert version < VERSION_40 && version >= settings.acceptMessaging.min; + logger.trace("Connection version {} from {}", version, ctx.channel().remoteAddress()); + + if (initiate.type.isStreaming()) + { + // streaming connections are per-session and have a fixed version. we can't do anything with a wrong-version stream connection, so drop it. + if (version != settings.acceptStreaming.max) + { + logger.warn("Received stream using protocol version {} (my version {}). Terminating connection", version, settings.acceptStreaming.max); + failHandshake(ctx); + } + setupStreamingPipeline(initiate.from, ctx); + } + else + { + // if this version is < the MS version the other node is trying + // to connect with, the other node will disconnect + ByteBuf response = HandshakeProtocol.Accept.respondPre40(settings.acceptMessaging.max, ctx.alloc()); + AsyncChannelPromise.writeAndFlush(ctx, response, + (ChannelFutureListener) future -> { + if (!future.isSuccess()) + exceptionCaught(future.channel(), future.cause()); + }); + + if (version < VERSION_30) + throw new IOException(String.format("Unable to read obsolete message version %s from %s; The earliest version supported is 3.0.0", version, ctx.channel().remoteAddress())); + + // we don't setup the messaging pipeline here, as the legacy messaging handshake requires one more message to finish + } + } + } + + /** + * Handles the third (and last) message in the internode messaging handshake protocol for pre40 nodes. + * Grabs the protocol version and IP addr the peer wants to use. + */ + @VisibleForTesting + void confirmPre40(ChannelHandlerContext ctx, ByteBuf in) + { + confirmOutboundPre40 = HandshakeProtocol.ConfirmOutboundPre40.maybeDecode(in); + if (confirmOutboundPre40 == null) + return; + + logger.trace("Received third handshake message from peer {}, message = {}", ctx.channel().remoteAddress(), confirmOutboundPre40); + setupMessagingPipeline(confirmOutboundPre40.from, initiate.requestMessagingVersion, confirmOutboundPre40.maxMessagingVersion, ctx.pipeline()); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) + { + exceptionCaught(ctx.channel(), cause); + } + + private void exceptionCaught(Channel channel, Throwable cause) + { + logger.error("Failed to properly handshake with peer {}. Closing the channel.", channel.remoteAddress(), cause); + try + { + failHandshake(channel); + } + catch (Throwable t) + { + logger.error("Unexpected exception in {}.exceptionCaught", this.getClass().getSimpleName(), t); + } + } + + private void failHandshake(ChannelHandlerContext ctx) + { + failHandshake(ctx.channel()); + } + + private void failHandshake(Channel channel) + { + channel.close(); + if (handshakeTimeout != null) + handshakeTimeout.cancel(true); + } + + private void setupStreamingPipeline(InetAddressAndPort from, ChannelHandlerContext ctx) + { + handshakeTimeout.cancel(true); + assert initiate.framing == Framing.UNPROTECTED; + + ChannelPipeline pipeline = ctx.pipeline(); + Channel channel = ctx.channel(); + + if (from == null) + { + InetSocketAddress address = (InetSocketAddress) channel.remoteAddress(); + from = InetAddressAndPort.getByAddressOverrideDefaults(address.getAddress(), address.getPort()); + } + + BufferPool.setRecycleWhenFreeForCurrentThread(false); + pipeline.replace(this, "streamInbound", new StreamingInboundHandler(from, current_version, null)); + } + + @VisibleForTesting + void setupMessagingPipeline(InetAddressAndPort from, int useMessagingVersion, int maxMessagingVersion, ChannelPipeline pipeline) + { + handshakeTimeout.cancel(true); + // record the "true" endpoint, i.e. the one the peer is identified with, as opposed to the socket it connected over + instance().versions.set(from, maxMessagingVersion); + + BufferPool.setRecycleWhenFreeForCurrentThread(false); + BufferPoolAllocator allocator = GlobalBufferPoolAllocator.instance; + if (initiate.type == ConnectionType.LARGE_MESSAGES) + { + // for large messages, swap the global pool allocator for a local one, to optimise utilisation of chunks + allocator = new LocalBufferPoolAllocator(pipeline.channel().eventLoop()); + pipeline.channel().config().setAllocator(allocator); + } + + FrameDecoder frameDecoder; + switch (initiate.framing) + { + case LZ4: + { + if (useMessagingVersion >= VERSION_40) + frameDecoder = FrameDecoderLZ4.fast(allocator); + else + frameDecoder = new FrameDecoderLegacyLZ4(allocator, useMessagingVersion); + break; + } + case CRC: + { + if (useMessagingVersion >= VERSION_40) + { + frameDecoder = FrameDecoderCrc.create(allocator); + break; + } + } + case UNPROTECTED: + { + if (useMessagingVersion >= VERSION_40) + frameDecoder = new FrameDecoderUnprotected(allocator); + else + frameDecoder = new FrameDecoderLegacy(allocator, useMessagingVersion); + break; + } + default: + throw new AssertionError(); + } + + frameDecoder.addLastTo(pipeline); + + InboundMessageHandler handler = + settings.handlers.apply(from).createHandler(frameDecoder, initiate.type, pipeline.channel(), useMessagingVersion); + + logger.info("{} connection established, version = {}, framing = {}, encryption = {}", + handler.id(true), + useMessagingVersion, + initiate.framing, + pipeline.get("ssl") != null ? encryptionLogStatement(settings.encryption) : "disabled"); + + pipeline.addLast("deserialize", handler); + + pipeline.remove(this); + } + } + + private static class OptionalSslHandler extends ByteToMessageDecoder + { + private final EncryptionOptions.ServerEncryptionOptions encryptionOptions; + + OptionalSslHandler(EncryptionOptions.ServerEncryptionOptions encryptionOptions) + { + this.encryptionOptions = encryptionOptions; + } + + protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception + { + if (in.readableBytes() < 5) + { + // To detect if SSL must be used we need to have at least 5 bytes, so return here and try again + // once more bytes a ready. + return; + } + + if (SslHandler.isEncrypted(in)) + { + // Connection uses SSL/TLS, replace the detection handler with a SslHandler and so use encryption. + SslContext sslContext = SSLFactory.getOrCreateSslContext(encryptionOptions, true, SSLFactory.SocketType.SERVER); + Channel channel = ctx.channel(); + InetSocketAddress peer = encryptionOptions.require_endpoint_verification ? (InetSocketAddress) channel.remoteAddress() : null; + SslHandler sslHandler = newSslHandler(channel, sslContext, peer); + ctx.pipeline().replace(this, "ssl", sslHandler); + } + else + { + // Connection use no TLS/SSL encryption, just remove the detection handler and continue without + // SslHandler in the pipeline. + ctx.pipeline().remove(this); + } + } + } +} diff --git a/src/java/org/apache/cassandra/net/InboundConnectionSettings.java b/src/java/org/apache/cassandra/net/InboundConnectionSettings.java new file mode 100644 index 000000000000..a07395b8f270 --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundConnectionSettings.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.net.InetAddress; +import java.util.function.Function; + +import com.google.common.base.Preconditions; + +import org.apache.cassandra.auth.IInternodeAuthenticator; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.utils.FBUtilities; + +import static java.lang.String.format; +import static org.apache.cassandra.net.MessagingService.*; + +public class InboundConnectionSettings +{ + public final IInternodeAuthenticator authenticator; + public final InetAddressAndPort bindAddress; + public final ServerEncryptionOptions encryption; + public final Integer socketReceiveBufferSizeInBytes; + public final Integer applicationReceiveQueueCapacityInBytes; + public final AcceptVersions acceptMessaging; + public final AcceptVersions acceptStreaming; + public final SocketFactory socketFactory; + public final Function handlers; + + private InboundConnectionSettings(IInternodeAuthenticator authenticator, + InetAddressAndPort bindAddress, + ServerEncryptionOptions encryption, + Integer socketReceiveBufferSizeInBytes, + Integer applicationReceiveQueueCapacityInBytes, + AcceptVersions acceptMessaging, + AcceptVersions acceptStreaming, + SocketFactory socketFactory, + Function handlers) + { + this.authenticator = authenticator; + this.bindAddress = bindAddress; + this.encryption = encryption; + this.socketReceiveBufferSizeInBytes = socketReceiveBufferSizeInBytes; + this.applicationReceiveQueueCapacityInBytes = applicationReceiveQueueCapacityInBytes; + this.acceptMessaging = acceptMessaging; + this.acceptStreaming = acceptStreaming; + this.socketFactory = socketFactory; + this.handlers = handlers; + } + + public InboundConnectionSettings() + { + this(null, null, null, null, null, null, null, null, null); + } + + public boolean authenticate(InetAddressAndPort endpoint) + { + return authenticator.authenticate(endpoint.address, endpoint.port); + } + + public boolean authenticate(InetAddress address, int port) + { + return authenticator.authenticate(address, port); + } + + public String toString() + { + return format("address: (%s), nic: %s, encryption: %s", + bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionLogStatement(encryption)); + } + + public InboundConnectionSettings withAuthenticator(IInternodeAuthenticator authenticator) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + @SuppressWarnings("unused") + public InboundConnectionSettings withBindAddress(InetAddressAndPort bindAddress) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + public InboundConnectionSettings withEncryption(ServerEncryptionOptions encryption) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + public InboundConnectionSettings withSocketReceiveBufferSizeInBytes(int socketReceiveBufferSizeInBytes) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + @SuppressWarnings("unused") + public InboundConnectionSettings withApplicationReceiveQueueCapacityInBytes(int applicationReceiveQueueCapacityInBytes) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + public InboundConnectionSettings withAcceptMessaging(AcceptVersions acceptMessaging) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + public InboundConnectionSettings withAcceptStreaming(AcceptVersions acceptMessaging) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + public InboundConnectionSettings withSocketFactory(SocketFactory socketFactory) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + public InboundConnectionSettings withHandlers(Function handlers) + { + return new InboundConnectionSettings(authenticator, bindAddress, encryption, + socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, + acceptMessaging, acceptStreaming, socketFactory, handlers); + } + + public InboundConnectionSettings withLegacyDefaults() + { + ServerEncryptionOptions encryption = this.encryption; + if (encryption == null) + encryption = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); + encryption = encryption.withOptional(false); + + return this.withBindAddress(bindAddress.withPort(DatabaseDescriptor.getSSLStoragePort())) + .withEncryption(encryption) + .withDefaults(); + } + + // note that connectTo is updated even if specified, in the case of pre40 messaging and using encryption (to update port) + public InboundConnectionSettings withDefaults() + { + // this is for the socket that can be plain, only ssl, or optional plain/ssl + if (bindAddress.port != DatabaseDescriptor.getStoragePort() && bindAddress.port != DatabaseDescriptor.getSSLStoragePort()) + throw new ConfigurationException(format("Local endpoint port %d doesn't match YAML configured port %d or legacy SSL port %d", + bindAddress.port, DatabaseDescriptor.getStoragePort(), DatabaseDescriptor.getSSLStoragePort())); + + IInternodeAuthenticator authenticator = this.authenticator; + ServerEncryptionOptions encryption = this.encryption; + Integer socketReceiveBufferSizeInBytes = this.socketReceiveBufferSizeInBytes; + Integer applicationReceiveQueueCapacityInBytes = this.applicationReceiveQueueCapacityInBytes; + AcceptVersions acceptMessaging = this.acceptMessaging; + AcceptVersions acceptStreaming = this.acceptStreaming; + SocketFactory socketFactory = this.socketFactory; + Function handlersFactory = this.handlers; + + if (authenticator == null) + authenticator = DatabaseDescriptor.getInternodeAuthenticator(); + + if (encryption == null) + encryption = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); + + if (socketReceiveBufferSizeInBytes == null) + socketReceiveBufferSizeInBytes = DatabaseDescriptor.getInternodeSocketReceiveBufferSizeInBytes(); + + if (applicationReceiveQueueCapacityInBytes == null) + applicationReceiveQueueCapacityInBytes = DatabaseDescriptor.getInternodeApplicationReceiveQueueCapacityInBytes(); + + if (acceptMessaging == null) + acceptMessaging = accept_messaging; + + if (acceptStreaming == null) + acceptStreaming = accept_streaming; + + if (socketFactory == null) + socketFactory = instance().socketFactory; + + if (handlersFactory == null) + handlersFactory = instance()::getInbound; + + Preconditions.checkArgument(socketReceiveBufferSizeInBytes == 0 || socketReceiveBufferSizeInBytes >= 1 << 10, "illegal socket send buffer size: " + socketReceiveBufferSizeInBytes); + Preconditions.checkArgument(applicationReceiveQueueCapacityInBytes >= 1 << 10, "illegal application receive queue capacity: " + applicationReceiveQueueCapacityInBytes); + + return new InboundConnectionSettings(authenticator, bindAddress, encryption, socketReceiveBufferSizeInBytes, applicationReceiveQueueCapacityInBytes, acceptMessaging, acceptStreaming, socketFactory, handlersFactory); + } +} diff --git a/src/java/org/apache/cassandra/net/InboundCounters.java b/src/java/org/apache/cassandra/net/InboundCounters.java new file mode 100644 index 000000000000..da035f226d8e --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundCounters.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.atomic.AtomicLongFieldUpdater; + +/** + * Aggregates counters for (from, connection type) for the duration of host uptime. + * + * If contention/false sharing ever become a problem, consider introducing padding. + */ +class InboundCounters +{ + private volatile long errorCount; + private volatile long errorBytes; + + private static final AtomicLongFieldUpdater errorCountUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "errorCount"); + private static final AtomicLongFieldUpdater errorBytesUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "errorBytes"); + + void addError(int bytes) + { + errorCountUpdater.incrementAndGet(this); + errorBytesUpdater.addAndGet(this, bytes); + } + + long errorCount() + { + return errorCount; + } + + long errorBytes() + { + return errorBytes; + } + + private volatile long expiredCount; + private volatile long expiredBytes; + + private static final AtomicLongFieldUpdater expiredCountUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "expiredCount"); + private static final AtomicLongFieldUpdater expiredBytesUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "expiredBytes"); + + void addExpired(int bytes) + { + expiredCountUpdater.incrementAndGet(this); + expiredBytesUpdater.addAndGet(this, bytes); + } + + long expiredCount() + { + return expiredCount; + } + + long expiredBytes() + { + return expiredBytes; + } + + private volatile long processedCount; + private volatile long processedBytes; + + private static final AtomicLongFieldUpdater processedCountUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "processedCount"); + private static final AtomicLongFieldUpdater processedBytesUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "processedBytes"); + + void addProcessed(int bytes) + { + processedCountUpdater.incrementAndGet(this); + processedBytesUpdater.addAndGet(this, bytes); + } + + long processedCount() + { + return processedCount; + } + + long processedBytes() + { + return processedBytes; + } + + private volatile long scheduledCount; + private volatile long scheduledBytes; + + private static final AtomicLongFieldUpdater scheduledCountUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "scheduledCount"); + private static final AtomicLongFieldUpdater scheduledBytesUpdater = + AtomicLongFieldUpdater.newUpdater(InboundCounters.class, "scheduledBytes"); + + void addPending(int bytes) + { + scheduledCountUpdater.incrementAndGet(this); + scheduledBytesUpdater.addAndGet(this, bytes); + } + + void removePending(int bytes) + { + scheduledCountUpdater.decrementAndGet(this); + scheduledBytesUpdater.addAndGet(this, -bytes); + } + + long scheduledCount() + { + return scheduledCount; + } + + long scheduledBytes() + { + return scheduledBytes; + } +} diff --git a/src/java/org/apache/cassandra/net/InboundMessageCallbacks.java b/src/java/org/apache/cassandra/net/InboundMessageCallbacks.java new file mode 100644 index 000000000000..ffa4243b9d10 --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundMessageCallbacks.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.net.Message.Header; + +/** + * Encapsulates the callbacks that {@link InboundMessageHandler} invokes during the lifecycle of an inbound message + * passing through it: from arrival to dispatch to execution. + * + * The flow will vary slightly between small and large messages. Small messages will be deserialized first and only + * then dispatched to one of the {@link Stage} stages for execution, whereas a large message will be dispatched first, + * and deserialized in-place on the relevant stage before being immediately processed. + * + * This difference will only show in case of deserialization failure. For large messages, it's possible for + * {@link #onFailedDeserialize(int, Header, Throwable)} to be invoked after {@link #onExecuting(int, Header, long, TimeUnit)}, + * whereas for small messages it isn't. + */ +interface InboundMessageCallbacks +{ + /** + * Invoked once the header of a message has arrived, small or large. + */ + void onHeaderArrived(int messageSize, Header header, long timeElapsed, TimeUnit unit); + + /** + * Invoked once an entire message worth of bytes has arrived, small or large. + */ + void onArrived(int messageSize, Header header, long timeElapsed, TimeUnit unit); + + /** + * Invoked if a message arrived too late to be processed, after its expiration. {@code wasCorrupt} might + * be set to {@code true} if 1+ corrupt frames were encountered while assembling an expired large message. + */ + void onArrivedExpired(int messageSize, Header header, boolean wasCorrupt, long timeElapsed, TimeUnit unit); + + /** + * Invoked if a large message arrived in time, but had one or more of its frames corrupted in flight. + */ + void onArrivedCorrupt(int messageSize, Header header, long timeElapsed, TimeUnit unit); + + /** + * Invoked if {@link InboundMessageHandler} was closed before receiving all frames of a large message. + * {@code wasCorrupt} will be set to {@code true} if some corrupt frames had been already encountered, + * {@code wasExpired} will be set to {@code true} if the message had expired in flight. + */ + void onClosedBeforeArrival(int messageSize, Header header, int bytesReceived, boolean wasCorrupt, boolean wasExpired); + + /** + * Invoked if a deserializer threw an exception while attempting to deserialize a message. + */ + void onFailedDeserialize(int messageSize, Header header, Throwable t); + + /** + * Invoked just before a message-processing task is scheduled on the appropriate {@link Stage} + * for the {@link Verb} of the message. + */ + void onDispatched(int messageSize, Header header); + + /** + * Invoked at the very beginning of execution of the message-processing task on the appropriate {@link Stage}. + */ + void onExecuting(int messageSize, Header header, long timeElapsed, TimeUnit unit); + + /** + * Invoked upon 'successful' processing of the message. Alternatively, {@link #onExpired(int, Header, long, TimeUnit)} + * will be invoked if the message had expired while waiting to be processed in the queue of the {@link Stage}. + */ + void onProcessed(int messageSize, Header header); + + /** + * Invoked if the message had expired while waiting to be processed in the queue of the {@link Stage}. Otherwise, + * {@link #onProcessed(int, Header)} will be invoked. + */ + void onExpired(int messageSize, Header header, long timeElapsed, TimeUnit unit); + + /** + * Invoked at the very end of execution of the message-processing task, no matter the outcome of processing. + */ + void onExecuted(int messageSize, Header header, long timeElapsed, TimeUnit unit); +} diff --git a/src/java/org/apache/cassandra/net/InboundMessageHandler.java b/src/java/org/apache/cassandra/net/InboundMessageHandler.java new file mode 100644 index 000000000000..891176682790 --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundMessageHandler.java @@ -0,0 +1,1194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.function.Consumer; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.EventLoop; +import org.apache.cassandra.concurrent.ExecutorLocals; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.concurrent.StageManager; +import org.apache.cassandra.exceptions.IncompatibleSchemaException; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message.Header; +import org.apache.cassandra.net.FrameDecoder.Frame; +import org.apache.cassandra.net.FrameDecoder.FrameProcessor; +import org.apache.cassandra.net.FrameDecoder.IntactFrame; +import org.apache.cassandra.net.FrameDecoder.CorruptFrame; +import org.apache.cassandra.net.ResourceLimits.Limit; +import org.apache.cassandra.tracing.TraceState; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.NoSpamLogger; + +import static java.lang.Math.max; +import static java.lang.Math.min; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.net.Crc.*; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +/** + * Core logic for handling inbound message deserialization and execution (in tandem with {@link FrameDecoder}). + * + * Handles small and large messages, corruption, flow control, dispatch of message processing onto an appropriate + * thread pool. + * + * # Interaction with {@link FrameDecoder} + * + * {@link InboundMessageHandler} sits on top of a {@link FrameDecoder} in the Netty pipeline, and is tightly + * coupled with it. + * + * {@link FrameDecoder} decodes inbound frames and relies on a supplied {@link FrameProcessor} to act on them. + * {@link InboundMessageHandler} provides two implementations of that interface: + * - {@link #process(Frame)} is the default, primary processor, and the primary entry point to this class + * - {@link UpToOneMessageFrameProcessor}, supplied to the decoder when the handler is reactivated after being + * put in waiting mode due to lack of acquirable reserve memory capacity permits + * + * Return value of {@link FrameProcessor#process(Frame)} determines whether the decoder should keep processing + * frames (if {@code true} is returned) or stop until explicitly reactivated (if {@code false} is). To reactivate + * the decoder (once notified of available resource permits), {@link FrameDecoder#reactivate()} is invoked. + * + * # Frames + * + * {@link InboundMessageHandler} operates on frames of messages, and there are several kinds of them: + * 1. {@link IntactFrame} that are contained. As names suggest, these contain one or multiple fully contained + * messages believed to be uncorrupted. Guaranteed to not contain an part of an incomplete message. + * See {@link #processFrameOfContainedMessages(ShareableBytes, Limit, Limit)}. + * 2. {@link IntactFrame} that are NOT contained. These are uncorrupted parts of a large message split over multiple + * parts due to their size. Can represent first or subsequent frame of a large message. + * See {@link #processFirstFrameOfLargeMessage(IntactFrame, Limit, Limit)} and + * {@link #processSubsequentFrameOfLargeMessage(Frame)}. + * 3. {@link CorruptFrame} with corrupt header. These are unrecoverable, and force a connection to be dropped. + * 4. {@link CorruptFrame} with a valid header, but corrupt payload. These can be either contained or uncontained. + * - contained frames with corrupt payload can be gracefully dropped without dropping the connection + * - uncontained frames with corrupt payload can be gracefully dropped unless they represent the first + * frame of a new large message, as in that case we don't know how many bytes to skip + * See {@link #processCorruptFrame(CorruptFrame)}. + * + * Fundamental frame invariants: + * 1. A contained frame can only have fully-encapsulated messages - 1 to n, that don't cross frame boundaries + * 2. An uncontained frame can hold a part of one message only. It can NOT, say, contain end of one large message + * and a beginning of another one. All the bytes in an uncontained frame always belong to a single message. + * + * # Small vs large messages + * + * A single handler is equipped to process both small and large messages, potentially interleaved, but the logic + * differs depending on size. Small messages are deserialized in place, and then handed off to an appropriate + * thread pool for processing. Large messages accumulate frames until completion of a message, then hand off + * the untouched frames to the correct thread pool for the verb to be deserialized there and immediately processed. + * + * See {@link LargeMessage} for details of the large-message accumulating state-machine, and {@link ProcessMessage} + * and its inheritors for the differences in execution. + * + * # Flow control (backpressure) + * + * To prevent nodes from overwhelming and bringing each other to the knees with more inbound messages that + * can be processed in a timely manner, {@link InboundMessageHandler} implements a strict flow control policy. + * + * Before we attempt to process a message fully, we first infer its size from the stream. Then we attempt to + * acquire memory permits for a message of that size. If we succeed, then we move on actually process the message. + * If we fail, the frame decoder deactivates until sufficient permits are released for the message to be processed + * and the handler is activated again. Permits are released back once the message has been fully processed - + * after the verb handler has been invoked - on the {@link Stage} for the {@link Verb} of the message. + * + * Every connection has an exclusive number of permits allocated to it (by default 4MiB). In addition to it, + * there is a per-endpoint reserve capacity and a global reserve capacity {@link Limit}, shared between all + * connections from the same host and all connections, respectively. So long as long as the handler stays within + * its exclusive limit, it doesn't need to tap into reserve capacity. + * + * If tapping into reserve capacity is necessary, but the handler fails to acquire capacity from either + * endpoint of global reserve (and it needs to acquire from both), the handler and its frame decoder become + * inactive and register with a {@link WaitQueue} of the appropriate type, depending on which of the reserves + * couldn't be tapped into. Once enough messages have finished processing and had their permits released back + * to the reserves, {@link WaitQueue} will reactivate the sleeping handlers and they'll resume processing frames. + * + * The reason we 'split' reserve capacity into two limits - endpoing and global - is to guarantee liveness, and + * prevent single endpoint's connections from taking over the whole reserve, starving other connections. + * + * One permit per byte of serialized message gets acquired. When inflated on-heap, each message will occupy more + * than that, necessarily, but despite wide variance, it's a good enough proxy that correlates with on-heap footprint. + */ +public class InboundMessageHandler extends ChannelInboundHandlerAdapter implements FrameProcessor +{ + private static final Logger logger = LoggerFactory.getLogger(InboundMessageHandler.class); + private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.SECONDS); + + private static final Message.Serializer serializer = Message.serializer; + + private final FrameDecoder decoder; + + private final ConnectionType type; + private final Channel channel; + private final InetAddressAndPort self; + private final InetAddressAndPort peer; + private final int version; + + private final int largeThreshold; + private LargeMessage largeMessage; + + private final long queueCapacity; + volatile long queueSize = 0L; + private static final AtomicLongFieldUpdater queueSizeUpdater = + AtomicLongFieldUpdater.newUpdater(InboundMessageHandler.class, "queueSize"); + + private final Limit endpointReserveCapacity; + private final WaitQueue endpointWaitQueue; + + private final Limit globalReserveCapacity; + private final WaitQueue globalWaitQueue; + + private final OnHandlerClosed onClosed; + private final InboundMessageCallbacks callbacks; + private final Consumer> consumer; + + // wait queue handle, non-null if we overrun endpoint or global capacity and request to be resumed once it's released + private WaitQueue.Ticket ticket = null; + + long corruptFramesRecovered, corruptFramesUnrecovered; + long receivedCount, receivedBytes; + long throttledCount, throttledNanos; + + private boolean isClosed; + + InboundMessageHandler(FrameDecoder decoder, + + ConnectionType type, + Channel channel, + InetAddressAndPort self, + InetAddressAndPort peer, + int version, + int largeThreshold, + + long queueCapacity, + Limit endpointReserveCapacity, + Limit globalReserveCapacity, + WaitQueue endpointWaitQueue, + WaitQueue globalWaitQueue, + + OnHandlerClosed onClosed, + InboundMessageCallbacks callbacks, + Consumer> consumer) + { + this.decoder = decoder; + + this.type = type; + this.channel = channel; + this.self = self; + this.peer = peer; + this.version = version; + this.largeThreshold = largeThreshold; + + this.queueCapacity = queueCapacity; + this.endpointReserveCapacity = endpointReserveCapacity; + this.endpointWaitQueue = endpointWaitQueue; + this.globalReserveCapacity = globalReserveCapacity; + this.globalWaitQueue = globalWaitQueue; + + this.onClosed = onClosed; + this.callbacks = callbacks; + this.consumer = consumer; + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) + { + /* + * InboundMessageHandler works in tandem with FrameDecoder to implement flow control + * and work stashing optimally. We rely on FrameDecoder to invoke the provided + * FrameProcessor rather than on the pipeline and invocations of channelRead(). + * process(Frame) is the primary entry point for this class. + */ + throw new IllegalStateException("InboundMessageHandler doesn't expect channelRead() to be invoked"); + } + + @Override + public void handlerAdded(ChannelHandlerContext ctx) + { + decoder.activate(this); // the frame decoder starts inactive until explicitly activated by the added inbound message handler + } + + @Override + public boolean process(Frame frame) throws IOException + { + if (frame instanceof IntactFrame) + return processIntactFrame((IntactFrame) frame, endpointReserveCapacity, globalReserveCapacity); + + processCorruptFrame((CorruptFrame) frame); + return true; + } + + private boolean processIntactFrame(IntactFrame frame, Limit endpointReserve, Limit globalReserve) throws IOException + { + if (frame.isSelfContained) + return processFrameOfContainedMessages(frame.contents, endpointReserve, globalReserve); + else if (null == largeMessage) + return processFirstFrameOfLargeMessage(frame, endpointReserve, globalReserve); + else + return processSubsequentFrameOfLargeMessage(frame); + } + + /* + * Handle contained messages (not crossing boundaries of the frame) - both small and large, for the inbound + * definition of large (breaching the size threshold for what we are willing to process on event-loop vs. + * off event-loop). + */ + private boolean processFrameOfContainedMessages(ShareableBytes bytes, Limit endpointReserve, Limit globalReserve) throws IOException + { + while (bytes.hasRemaining()) + if (!processOneContainedMessage(bytes, endpointReserve, globalReserve)) + return false; + return true; + } + + private boolean processOneContainedMessage(ShareableBytes bytes, Limit endpointReserve, Limit globalReserve) throws IOException + { + ByteBuffer buf = bytes.get(); + + long currentTimeNanos = approxTime.now(); + Header header = serializer.extractHeader(buf, peer, currentTimeNanos, version); + long timeElapsed = currentTimeNanos - header.createdAtNanos; + int size = serializer.inferMessageSize(buf, buf.position(), buf.limit(), version); + + if (approxTime.isAfter(currentTimeNanos, header.expiresAtNanos)) + { + callbacks.onHeaderArrived(size, header, timeElapsed, NANOSECONDS); + callbacks.onArrivedExpired(size, header, false, timeElapsed, NANOSECONDS); + receivedCount++; + receivedBytes += size; + bytes.skipBytes(size); + return true; + } + + if (!acquireCapacity(endpointReserve, globalReserve, size, currentTimeNanos, header.expiresAtNanos)) + return false; + + callbacks.onHeaderArrived(size, header, timeElapsed, NANOSECONDS); + callbacks.onArrived(size, header, timeElapsed, NANOSECONDS); + receivedCount++; + receivedBytes += size; + + if (size <= largeThreshold) + processSmallMessage(bytes, size, header); + else + processLargeMessage(bytes, size, header); + + return true; + } + + private void processSmallMessage(ShareableBytes bytes, int size, Header header) + { + ByteBuffer buf = bytes.get(); + final int begin = buf.position(); + final int end = buf.limit(); + buf.limit(begin + size); // cap to expected message size + + Message message = null; + try (DataInputBuffer in = new DataInputBuffer(buf, false)) + { + Message m = serializer.deserialize(in, header, version); + if (in.available() > 0) // bytes remaining after deser: deserializer is busted + throw new InvalidSerializedSizeException(header.verb, size, size - in.available()); + message = m; + } + catch (IncompatibleSchemaException e) + { + callbacks.onFailedDeserialize(size, header, e); + noSpamLogger.info("{} incompatible schema encountered while deserializing a message", id(), e); + } + catch (Throwable t) + { + JVMStabilityInspector.inspectThrowable(t, false); + callbacks.onFailedDeserialize(size, header, t); + logger.error("{} unexpected exception caught while deserializing a message", id(), t); + } + finally + { + if (null == message) + releaseCapacity(size); + + // no matter what, set position to the beginning of the next message and restore limit, so that + // we can always keep on decoding the frame even on failure to deserialize previous message + buf.position(begin + size); + buf.limit(end); + } + + if (null != message) + dispatch(new ProcessSmallMessage(message, size)); + } + + // for various reasons, it's possible for a large message to be contained in a single frame + private void processLargeMessage(ShareableBytes bytes, int size, Header header) + { + new LargeMessage(size, header, bytes.sliceAndConsume(size).share()).schedule(); + } + + /* + * Handling of multi-frame large messages + */ + + private boolean processFirstFrameOfLargeMessage(IntactFrame frame, Limit endpointReserve, Limit globalReserve) throws IOException + { + ShareableBytes bytes = frame.contents; + ByteBuffer buf = bytes.get(); + + long currentTimeNanos = approxTime.now(); + Header header = serializer.extractHeader(buf, peer, currentTimeNanos, version); + int size = serializer.inferMessageSize(buf, buf.position(), buf.limit(), version); + + boolean expired = approxTime.isAfter(currentTimeNanos, header.expiresAtNanos); + if (!expired && !acquireCapacity(endpointReserve, globalReserve, size, currentTimeNanos, header.expiresAtNanos)) + return false; + + callbacks.onHeaderArrived(size, header, currentTimeNanos - header.createdAtNanos, NANOSECONDS); + receivedBytes += buf.remaining(); + largeMessage = new LargeMessage(size, header, expired); + largeMessage.supply(frame); + return true; + } + + private boolean processSubsequentFrameOfLargeMessage(Frame frame) + { + receivedBytes += frame.frameSize; + if (largeMessage.supply(frame)) + { + receivedCount++; + largeMessage = null; + } + return true; + } + + /* + * We can handle some corrupt frames gracefully without dropping the connection and losing all the + * queued up messages, but not others. + * + * Corrupt frames that *ARE NOT* safe to skip gracefully and require the connection to be dropped: + * - any frame with corrupt header (!frame.isRecoverable()) + * - first corrupt-payload frame of a large message (impossible to infer message size, and without it + * impossible to skip the message safely + * + * Corrupt frames that *ARE* safe to skip gracefully, without reconnecting: + * - any self-contained frame with a corrupt payload (but not header): we lose all the messages in the + * frame, but that has no effect on subsequent ones + * - any non-first payload-corrupt frame of a large message: we know the size of the large message in + * flight, so we just skip frames until we've seen all its bytes; we only lose the large message + */ + private void processCorruptFrame(CorruptFrame frame) throws InvalidCrc + { + if (!frame.isRecoverable()) + { + corruptFramesUnrecovered++; + throw new InvalidCrc(frame.readCRC, frame.computedCRC); + } + else if (frame.isSelfContained) + { + receivedBytes += frame.frameSize; + corruptFramesRecovered++; + noSpamLogger.warn("{} invalid, recoverable CRC mismatch detected while reading messages (corrupted self-contained frame)", id()); + } + else if (null == largeMessage) // first frame of a large message + { + receivedBytes += frame.frameSize; + corruptFramesUnrecovered++; + noSpamLogger.error("{} invalid, unrecoverable CRC mismatch detected while reading messages (corrupted first frame of a large message)", id()); + throw new InvalidCrc(frame.readCRC, frame.computedCRC); + } + else // subsequent frame of a large message + { + processSubsequentFrameOfLargeMessage(frame); + corruptFramesRecovered++; + noSpamLogger.warn("{} invalid, recoverable CRC mismatch detected while reading a large message", id()); + } + } + + private void onEndpointReserveCapacityRegained(Limit endpointReserve, long elapsedNanos) + { + onReserveCapacityRegained(endpointReserve, globalReserveCapacity, elapsedNanos); + } + + private void onGlobalReserveCapacityRegained(Limit globalReserve, long elapsedNanos) + { + onReserveCapacityRegained(endpointReserveCapacity, globalReserve, elapsedNanos); + } + + private void onReserveCapacityRegained(Limit endpointReserve, Limit globalReserve, long elapsedNanos) + { + if (isClosed) + return; + + assert channel.eventLoop().inEventLoop(); + + ticket = null; + throttledNanos += elapsedNanos; + + try + { + /* + * Process up to one message using supplied overriden reserves - one of them pre-allocated, + * and guaranteed to be enough for one message - then, if no obstacles enountered, reactivate + * the frame decoder using normal reserve capacities. + */ + if (processUpToOneMessage(endpointReserve, globalReserve)) + decoder.reactivate(); + } + catch (Throwable t) + { + exceptionCaught(t); + } + } + + // return true if the handler should be reactivated - if no new hurdles were encountered, + // like running out of the other kind of reserve capacity + private boolean processUpToOneMessage(Limit endpointReserve, Limit globalReserve) throws IOException + { + UpToOneMessageFrameProcessor processor = new UpToOneMessageFrameProcessor(endpointReserve, globalReserve); + decoder.processBacklog(processor); + return processor.isActive; + } + + /* + * Process at most one message. Won't always be an entire one (if the message in the head of line + * is a large one, and there aren't sufficient frames to decode it entirely), but will never be more than one. + */ + private class UpToOneMessageFrameProcessor implements FrameProcessor + { + private final Limit endpointReserve; + private final Limit globalReserve; + + boolean isActive = true; + boolean firstFrame = true; + + private UpToOneMessageFrameProcessor(Limit endpointReserve, Limit globalReserve) + { + this.endpointReserve = endpointReserve; + this.globalReserve = globalReserve; + } + + @Override + public boolean process(Frame frame) throws IOException + { + if (firstFrame) + { + if (!(frame instanceof IntactFrame)) + throw new IllegalStateException("First backlog frame must be intact"); + firstFrame = false; + return processFirstFrame((IntactFrame) frame); + } + + return processSubsequentFrame(frame); + } + + private boolean processFirstFrame(IntactFrame frame) throws IOException + { + if (frame.isSelfContained) + { + isActive = processOneContainedMessage(frame.contents, endpointReserve, globalReserve); + return false; // stop after one message + } + else + { + isActive = processFirstFrameOfLargeMessage(frame, endpointReserve, globalReserve); + return isActive; // continue unless fallen behind coprocessor or ran out of reserve capacity again + } + } + + private boolean processSubsequentFrame(Frame frame) throws IOException + { + if (frame instanceof IntactFrame) + processSubsequentFrameOfLargeMessage(frame); + else + processCorruptFrame((CorruptFrame) frame); + + return largeMessage != null; // continue until done with the large message + } + } + + /** + * Try to acquire permits for the inbound message. In case of failure, register with the right wait queue to be + * reactivated once permit capacity is regained. + */ + @SuppressWarnings("BooleanMethodIsAlwaysInverted") + private boolean acquireCapacity(Limit endpointReserve, Limit globalReserve, int bytes, long currentTimeNanos, long expiresAtNanos) + { + ResourceLimits.Outcome outcome = acquireCapacity(endpointReserve, globalReserve, bytes); + + if (outcome == ResourceLimits.Outcome.INSUFFICIENT_ENDPOINT) + ticket = endpointWaitQueue.register(this, bytes, currentTimeNanos, expiresAtNanos); + else if (outcome == ResourceLimits.Outcome.INSUFFICIENT_GLOBAL) + ticket = globalWaitQueue.register(this, bytes, currentTimeNanos, expiresAtNanos); + + if (outcome != ResourceLimits.Outcome.SUCCESS) + throttledCount++; + + return outcome == ResourceLimits.Outcome.SUCCESS; + } + + private ResourceLimits.Outcome acquireCapacity(Limit endpointReserve, Limit globalReserve, int bytes) + { + long currentQueueSize = queueSize; + + /* + * acquireCapacity() is only ever called on the event loop, and as such queueSize is only ever increased + * on the event loop. If there is enough capacity, we can safely addAndGet() and immediately return. + */ + if (currentQueueSize + bytes <= queueCapacity) + { + queueSizeUpdater.addAndGet(this, bytes); + return ResourceLimits.Outcome.SUCCESS; + } + + // we know we don't have enough local queue capacity for the entire message, so we need to borrow some from reserve capacity + long allocatedExcess = min(currentQueueSize + bytes - queueCapacity, bytes); + + if (!globalReserve.tryAllocate(allocatedExcess)) + return ResourceLimits.Outcome.INSUFFICIENT_GLOBAL; + + if (!endpointReserve.tryAllocate(allocatedExcess)) + { + globalReserve.release(allocatedExcess); + globalWaitQueue.signal(); + return ResourceLimits.Outcome.INSUFFICIENT_GLOBAL; + } + + long newQueueSize = queueSizeUpdater.addAndGet(this, bytes); + long actualExcess = max(0, min(newQueueSize - queueCapacity, bytes)); + + /* + * It's possible that some permits were released at some point after we loaded current queueSize, + * and we can satisfy more of the permits using our exclusive per-connection capacity, needing + * less than previously estimated from the reserves. If that's the case, release the now unneeded + * permit excess back to endpoint/global reserves. + */ + if (actualExcess != allocatedExcess) // actualExcess < allocatedExcess + { + long excess = allocatedExcess - actualExcess; + + endpointReserve.release(excess); + globalReserve.release(excess); + + endpointWaitQueue.signal(); + globalWaitQueue.signal(); + } + + return ResourceLimits.Outcome.SUCCESS; + } + + private void releaseCapacity(int bytes) + { + long oldQueueSize = queueSizeUpdater.getAndAdd(this, -bytes); + if (oldQueueSize > queueCapacity) + { + long excess = min(oldQueueSize - queueCapacity, bytes); + + endpointReserveCapacity.release(excess); + globalReserveCapacity.release(excess); + + endpointWaitQueue.signal(); + globalWaitQueue.signal(); + } + } + + /** + * Invoked to release capacity for a message that has been fully, successfully processed. + * + * Normally no different from invoking {@link #releaseCapacity(int)}, but is necessary for the verifier + * to be able to delay capacity release for backpressure testing. + */ + @VisibleForTesting + protected void releaseProcessedCapacity(int size, Header header) + { + releaseCapacity(size); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) + { + try + { + exceptionCaught(cause); + } + catch (Throwable t) + { + logger.error("Unexpected exception in {}.exceptionCaught", this.getClass().getSimpleName(), t); + } + } + + private void exceptionCaught(Throwable cause) + { + decoder.discard(); + + JVMStabilityInspector.inspectThrowable(cause, false); + + if (cause instanceof Message.InvalidLegacyProtocolMagic) + logger.error("{} invalid, unrecoverable CRC mismatch detected while reading messages - closing the connection", id()); + else + logger.error("{} unexpected exception caught while processing inbound messages; terminating connection", id(), cause); + + channel.close(); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) + { + isClosed = true; + + if (null != largeMessage) + largeMessage.abort(); + + if (null != ticket) + ticket.invalidate(); + + onClosed.call(this); + } + + private EventLoop eventLoop() + { + return channel.eventLoop(); + } + + String id(boolean includeReal) + { + if (!includeReal) + return id(); + + return SocketFactory.channelId(peer, (InetSocketAddress) channel.remoteAddress(), + self, (InetSocketAddress) channel.localAddress(), + type, channel.id().asShortText()); + } + + String id() + { + return SocketFactory.channelId(peer, self, type, channel.id().asShortText()); + } + + /* + * A large-message frame-accumulating state machine. + * + * Collects intact frames until it's has all the bytes necessary to deserialize the large message, + * at which point it schedules a task on the appropriate {@link Stage}, + * a task that deserializes the message and immediately invokes the verb handler. + * + * Also handles corrupt frames and potential expiry of the large message during accumulation: + * if it's taking the frames too long to arrive, there is no point in holding on to the + * accumulated frames, or in gathering more - so we release the ones we already have, and + * skip any remaining ones, alongside with returning memory permits early. + */ + private class LargeMessage + { + private final int size; + private final Header header; + + private final List buffers = new ArrayList<>(); + private int received; + + private boolean isExpired; + private boolean isCorrupt; + + private LargeMessage(int size, Header header, boolean isExpired) + { + this.size = size; + this.header = header; + this.isExpired = isExpired; + } + + private LargeMessage(int size, Header header, ShareableBytes bytes) + { + this(size, header, false); + buffers.add(bytes); + } + + private void schedule() + { + dispatch(new ProcessLargeMessage(this)); + } + + /** + * Return true if this was the last frame of the large message. + */ + private boolean supply(Frame frame) + { + if (frame instanceof IntactFrame) + onIntactFrame((IntactFrame) frame); + else + onCorruptFrame(); + + received += frame.frameSize; + if (size == received) + onComplete(); + return size == received; + } + + private void onIntactFrame(IntactFrame frame) + { + boolean expires = approxTime.isAfter(header.expiresAtNanos); + if (!isExpired && !isCorrupt) + { + if (!expires) + { + buffers.add(frame.contents.sliceAndConsume(frame.frameSize).share()); + return; + } + releaseBuffersAndCapacity(); // release resources once we transition from normal state to expired + } + frame.consume(); + isExpired |= expires; + } + + private void onCorruptFrame() + { + if (!isExpired && !isCorrupt) + releaseBuffersAndCapacity(); // release resources once we transition from normal state to corrupt + isCorrupt = true; + isExpired |= approxTime.isAfter(header.expiresAtNanos); + } + + private void onComplete() + { + long timeElapsed = approxTime.now() - header.createdAtNanos; + + if (!isExpired && !isCorrupt) + { + callbacks.onArrived(size, header, timeElapsed, NANOSECONDS); + schedule(); + } + else if (isExpired) + { + callbacks.onArrivedExpired(size, header, isCorrupt, timeElapsed, NANOSECONDS); + } + else + { + callbacks.onArrivedCorrupt(size, header, timeElapsed, NANOSECONDS); + } + } + + private void abort() + { + if (!isExpired && !isCorrupt) + releaseBuffersAndCapacity(); // release resources if in normal state when abort() is invoked + callbacks.onClosedBeforeArrival(size, header, received, isCorrupt, isExpired); + } + + private void releaseBuffers() + { + buffers.forEach(ShareableBytes::release); buffers.clear(); + } + + private void releaseBuffersAndCapacity() + { + releaseBuffers(); releaseCapacity(size); + } + + private Message deserialize() + { + try (ChunkedInputPlus input = ChunkedInputPlus.of(buffers)) + { + Message m = serializer.deserialize(input, header, version); + int remainder = input.remainder(); + if (remainder > 0) + throw new InvalidSerializedSizeException(header.verb, size, size - remainder); + return m; + } + catch (IncompatibleSchemaException e) + { + callbacks.onFailedDeserialize(size, header, e); + noSpamLogger.info("{} incompatible schema encountered while deserializing a message", id(), e); + } + catch (Throwable t) + { + JVMStabilityInspector.inspectThrowable(t, false); + callbacks.onFailedDeserialize(size, header, t); + logger.error("{} unexpected exception caught while deserializing a message", id(), t); + } + finally + { + buffers.clear(); // closing the input will have ensured that the buffers were released no matter what + } + + return null; + } + } + + /** + * Submit a {@link ProcessMessage} task to the appropriate {@link Stage} for the {@link Verb}. + */ + private void dispatch(ProcessMessage task) + { + Header header = task.header(); + + TraceState state = Tracing.instance.initializeFromMessage(header); + if (state != null) state.trace("{} message received from {}", header.verb, header.from); + + callbacks.onDispatched(task.size(), header); + StageManager.getStage(header.verb.stage).execute(task, ExecutorLocals.create(state)); + } + + private abstract class ProcessMessage implements Runnable + { + /** + * Actually handle the message. Runs on the appropriate {@link Stage} for the {@link Verb}. + * + * Small messages will come pre-deserialized. Large messages will be deserialized on the stage, + * just in time, and only then processed. + */ + public void run() + { + Header header = header(); + long currentTimeNanos = approxTime.now(); + boolean expired = approxTime.isAfter(currentTimeNanos, header.expiresAtNanos); + + boolean processed = false; + try + { + callbacks.onExecuting(size(), header, currentTimeNanos - header.createdAtNanos, NANOSECONDS); + + if (expired) + { + callbacks.onExpired(size(), header, currentTimeNanos - header.createdAtNanos, NANOSECONDS); + return; + } + + Message message = provideMessage(); + if (null != message) + { + consumer.accept(message); + processed = true; + callbacks.onProcessed(size(), header); + } + } + finally + { + if (processed) + releaseProcessedCapacity(size(), header); + else + releaseCapacity(size()); + + releaseResources(); + + callbacks.onExecuted(size(), header, approxTime.now() - currentTimeNanos, NANOSECONDS); + } + } + + abstract int size(); + abstract Header header(); + abstract Message provideMessage(); + void releaseResources() {} + } + + private class ProcessSmallMessage extends ProcessMessage + { + private final int size; + private final Message message; + + ProcessSmallMessage(Message message, int size) + { + this.size = size; + this.message = message; + } + + int size() + { + return size; + } + + Header header() + { + return message.header; + } + + Message provideMessage() + { + return message; + } + } + + private class ProcessLargeMessage extends ProcessMessage + { + private final LargeMessage message; + + ProcessLargeMessage(LargeMessage message) + { + this.message = message; + } + + int size() + { + return message.size; + } + + Header header() + { + return message.header; + } + + Message provideMessage() + { + return message.deserialize(); + } + + @Override + void releaseResources() + { + message.releaseBuffers(); // releases buffers if they haven't been yet (by deserialize() call) + } + } + + /** + * A special-purpose wait queue to park inbound message handlers that failed to allocate + * reserve capacity for a message in. Upon such failure a handler registers itself with + * a {@link WaitQueue} of the appropriate kind (either ENDPOINT or GLOBAL - if failed + * to allocate endpoint or global reserve capacity, respectively), stops processing any + * accumulated frames or receiving new ones, and waits - until reactivated. + * + * Every time permits are returned to an endpoint or global {@link Limit}, the respective + * queue gets signalled, and if there are any handlers registered in it, we will attempt + * to reactivate as many waiting handlers as current available reserve capacity allows + * us to - immediately, on the {@link #signal()}-calling thread. At most one such attempt + * will be in progress at any given time. + * + * Handlers that can be reactivated will be grouped by their {@link EventLoop} and a single + * {@link ReactivateHandlers} task will be scheduled per event loop, on the corresponding + * event loops. + * + * When run, the {@link ReactivateHandlers} task will ask each handler in its group to first + * process one message - using preallocated reserve capacity - and if no obstacles were met - + * reactivate the handlers, this time using their regular reserves. + * + * See {@link WaitQueue#schedule()}, {@link ReactivateHandlers#run()}, {@link Ticket#reactivateHandler(Limit)}. + */ + public static final class WaitQueue + { + enum Kind { ENDPOINT, GLOBAL } + + private static final int NOT_RUNNING = 0; + @SuppressWarnings("unused") + private static final int RUNNING = 1; + private static final int RUN_AGAIN = 2; + + private volatile int scheduled; + private static final AtomicIntegerFieldUpdater scheduledUpdater = + AtomicIntegerFieldUpdater.newUpdater(WaitQueue.class, "scheduled"); + + private final Kind kind; + private final Limit reserveCapacity; + + private final ManyToOneConcurrentLinkedQueue queue = new ManyToOneConcurrentLinkedQueue<>(); + + private WaitQueue(Kind kind, Limit reserveCapacity) + { + this.kind = kind; + this.reserveCapacity = reserveCapacity; + } + + public static WaitQueue endpoint(Limit endpointReserveCapacity) + { + return new WaitQueue(Kind.ENDPOINT, endpointReserveCapacity); + } + + public static WaitQueue global(Limit globalReserveCapacity) + { + return new WaitQueue(Kind.GLOBAL, globalReserveCapacity); + } + + private Ticket register(InboundMessageHandler handler, int bytesRequested, long registeredAtNanos, long expiresAtNanos) + { + Ticket ticket = new Ticket(this, handler, bytesRequested, registeredAtNanos, expiresAtNanos); + Ticket previous = queue.relaxedPeekLastAndOffer(ticket); + if (null == previous || !previous.isWaiting()) + signal(); // only signal the queue if this handler is first to register + return ticket; + } + + private void signal() + { + if (queue.relaxedIsEmpty()) + return; // we can return early if no handlers have registered with the wait queue + + if (NOT_RUNNING == scheduledUpdater.getAndUpdate(this, i -> min(RUN_AGAIN, i + 1))) + { + do + { + schedule(); + } + while (RUN_AGAIN == scheduledUpdater.getAndDecrement(this)); + } + } + + private void schedule() + { + Map tasks = null; + + long currentTimeNanos = approxTime.now(); + + Ticket t; + while ((t = queue.peek()) != null) + { + if (!t.call()) // invalidated + { + queue.remove(); + continue; + } + + boolean isLive = t.isLive(currentTimeNanos); + if (isLive && !reserveCapacity.tryAllocate(t.bytesRequested)) + { + if (!t.reset()) // the ticket was invalidated after being called but before now + { + queue.remove(); + continue; + } + break; // TODO: traverse the entire queue to unblock handlers that have expired or invalidated tickets + } + + if (null == tasks) + tasks = new IdentityHashMap<>(); + + queue.remove(); + tasks.computeIfAbsent(t.handler.eventLoop(), e -> new ReactivateHandlers()).add(t, isLive); + } + + if (null != tasks) + tasks.forEach(EventLoop::execute); + } + + private class ReactivateHandlers implements Runnable + { + List tickets = new ArrayList<>(); + long capacity = 0L; + + private void add(Ticket ticket, boolean isLive) + { + tickets.add(ticket); + if (isLive) capacity += ticket.bytesRequested; + } + + public void run() + { + Limit limit = new ResourceLimits.Basic(capacity); + try + { + for (Ticket ticket : tickets) + ticket.reactivateHandler(limit); + } + finally + { + /* + * Free up any unused capacity, if any. Will be non-zero if one or more handlers were closed + * when we attempted to run their callback, or used more of their other reserve; or if the first + * message in the unprocessed stream has expired in the narrow time window. + */ + long remaining = limit.remaining(); + if (remaining > 0) + { + reserveCapacity.release(remaining); + signal(); + } + } + } + } + + private static final class Ticket + { + private static final int WAITING = 0; + private static final int CALLED = 1; + private static final int INVALIDATED = 2; // invalidated by a handler that got closed + + private volatile int state; + private static final AtomicIntegerFieldUpdater stateUpdater = + AtomicIntegerFieldUpdater.newUpdater(Ticket.class, "state"); + + private final WaitQueue waitQueue; + private final InboundMessageHandler handler; + private final int bytesRequested; + private final long reigsteredAtNanos; + private final long expiresAtNanos; + + private Ticket(WaitQueue waitQueue, InboundMessageHandler handler, int bytesRequested, long registeredAtNanos, long expiresAtNanos) + { + this.waitQueue = waitQueue; + this.handler = handler; + this.bytesRequested = bytesRequested; + this.reigsteredAtNanos = registeredAtNanos; + this.expiresAtNanos = expiresAtNanos; + } + + private void reactivateHandler(Limit capacity) + { + long elapsedNanos = approxTime.now() - reigsteredAtNanos; + try + { + if (waitQueue.kind == Kind.ENDPOINT) + handler.onEndpointReserveCapacityRegained(capacity, elapsedNanos); + else + handler.onGlobalReserveCapacityRegained(capacity, elapsedNanos); + } + catch (Throwable t) + { + logger.error("{} exception caught while reactivating a handler", handler.id(), t); + } + } + + private boolean isWaiting() + { + return state == WAITING; + } + + private boolean isLive(long currentTimeNanos) + { + return !approxTime.isAfter(currentTimeNanos, expiresAtNanos); + } + + private void invalidate() + { + state = INVALIDATED; + waitQueue.signal(); + } + + private boolean call() + { + return stateUpdater.compareAndSet(this, WAITING, CALLED); + } + + private boolean reset() + { + return stateUpdater.compareAndSet(this, CALLED, WAITING); + } + } + } + + public interface OnHandlerClosed + { + void call(InboundMessageHandler handler); + } +} diff --git a/src/java/org/apache/cassandra/net/InboundMessageHandlers.java b/src/java/org/apache/cassandra/net/InboundMessageHandlers.java new file mode 100644 index 000000000000..4ebd5ad76afb --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundMessageHandlers.java @@ -0,0 +1,447 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.Collection; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.function.Consumer; +import java.util.function.ToLongFunction; + +import io.netty.channel.Channel; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.metrics.InternodeInboundMetrics; +import org.apache.cassandra.net.Message.Header; +import org.apache.cassandra.utils.ApproximateTime; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +/** + * An aggregation of {@link InboundMessageHandler}s for all connections from a peer. + * + * Manages metrics and shared resource limits. Can have multiple connections of a single + * type open simultaneousely (legacy in particular). + */ +public final class InboundMessageHandlers +{ + private final InetAddressAndPort self; + private final InetAddressAndPort peer; + + private final int queueCapacity; + private final ResourceLimits.Limit endpointReserveCapacity; + private final ResourceLimits.Limit globalReserveCapacity; + + private final InboundMessageHandler.WaitQueue endpointWaitQueue; + private final InboundMessageHandler.WaitQueue globalWaitQueue; + + private final InboundCounters urgentCounters = new InboundCounters(); + private final InboundCounters smallCounters = new InboundCounters(); + private final InboundCounters largeCounters = new InboundCounters(); + private final InboundCounters legacyCounters = new InboundCounters(); + + private final InboundMessageCallbacks urgentCallbacks; + private final InboundMessageCallbacks smallCallbacks; + private final InboundMessageCallbacks largeCallbacks; + private final InboundMessageCallbacks legacyCallbacks; + + private final InternodeInboundMetrics metrics; + private final MessageConsumer messageConsumer; + + private final HandlerProvider handlerProvider; + private final Collection handlers = new CopyOnWriteArrayList<>(); + + static class GlobalResourceLimits + { + final ResourceLimits.Limit reserveCapacity; + final InboundMessageHandler.WaitQueue waitQueue; + + GlobalResourceLimits(ResourceLimits.Limit reserveCapacity) + { + this.reserveCapacity = reserveCapacity; + this.waitQueue = InboundMessageHandler.WaitQueue.global(reserveCapacity); + } + } + + public interface MessageConsumer extends Consumer> + { + void fail(Message.Header header, Throwable failure); + } + + public interface GlobalMetricCallbacks + { + LatencyConsumer internodeLatencyRecorder(InetAddressAndPort to); + void recordInternalLatency(Verb verb, long timeElapsed, TimeUnit timeUnit); + void recordInternodeDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit); + } + + public InboundMessageHandlers(InetAddressAndPort self, + InetAddressAndPort peer, + int queueCapacity, + long endpointReserveCapacity, + GlobalResourceLimits globalResourceLimits, + GlobalMetricCallbacks globalMetricCallbacks, + MessageConsumer messageConsumer) + { + this(self, peer, queueCapacity, endpointReserveCapacity, globalResourceLimits, globalMetricCallbacks, messageConsumer, InboundMessageHandler::new); + } + + public InboundMessageHandlers(InetAddressAndPort self, + InetAddressAndPort peer, + int queueCapacity, + long endpointReserveCapacity, + GlobalResourceLimits globalResourceLimits, + GlobalMetricCallbacks globalMetricCallbacks, + MessageConsumer messageConsumer, + HandlerProvider handlerProvider) + { + this.self = self; + this.peer = peer; + + this.queueCapacity = queueCapacity; + this.endpointReserveCapacity = new ResourceLimits.Concurrent(endpointReserveCapacity); + this.globalReserveCapacity = globalResourceLimits.reserveCapacity; + this.endpointWaitQueue = InboundMessageHandler.WaitQueue.endpoint(this.endpointReserveCapacity); + this.globalWaitQueue = globalResourceLimits.waitQueue; + this.messageConsumer = messageConsumer; + + this.handlerProvider = handlerProvider; + + urgentCallbacks = makeMessageCallbacks(peer, urgentCounters, globalMetricCallbacks, messageConsumer); + smallCallbacks = makeMessageCallbacks(peer, smallCounters, globalMetricCallbacks, messageConsumer); + largeCallbacks = makeMessageCallbacks(peer, largeCounters, globalMetricCallbacks, messageConsumer); + legacyCallbacks = makeMessageCallbacks(peer, legacyCounters, globalMetricCallbacks, messageConsumer); + + metrics = new InternodeInboundMetrics(peer, this); + } + + InboundMessageHandler createHandler(FrameDecoder frameDecoder, ConnectionType type, Channel channel, int version) + { + InboundMessageHandler handler = + handlerProvider.provide(frameDecoder, + + type, + channel, + self, + peer, + version, + OutboundConnections.LARGE_MESSAGE_THRESHOLD, + + queueCapacity, + endpointReserveCapacity, + globalReserveCapacity, + endpointWaitQueue, + globalWaitQueue, + + this::onHandlerClosed, + callbacksFor(type), + messageConsumer); + handlers.add(handler); + return handler; + } + + void releaseMetrics() + { + metrics.release(); + } + + private void onHandlerClosed(InboundMessageHandler handler) + { + handlers.remove(handler); + absorbCounters(handler); + } + + /* + * Message callbacks + */ + + private InboundMessageCallbacks callbacksFor(ConnectionType type) + { + switch (type) + { + case URGENT_MESSAGES: return urgentCallbacks; + case SMALL_MESSAGES: return smallCallbacks; + case LARGE_MESSAGES: return largeCallbacks; + case LEGACY_MESSAGES: return legacyCallbacks; + } + + throw new IllegalArgumentException(); + } + + private static InboundMessageCallbacks makeMessageCallbacks(InetAddressAndPort peer, InboundCounters counters, GlobalMetricCallbacks globalMetrics, MessageConsumer messageConsumer) + { + LatencyConsumer internodeLatency = globalMetrics.internodeLatencyRecorder(peer); + + return new InboundMessageCallbacks() + { + @Override + public void onHeaderArrived(int messageSize, Header header, long timeElapsed, TimeUnit unit) + { + // do not log latency if we are within error bars of zero + if (timeElapsed > unit.convert(approxTime.error(), NANOSECONDS)) + internodeLatency.accept(timeElapsed, unit); + } + + @Override + public void onArrived(int messageSize, Header header, long timeElapsed, TimeUnit unit) + { + } + + @Override + public void onArrivedExpired(int messageSize, Header header, boolean wasCorrupt, long timeElapsed, TimeUnit unit) + { + counters.addExpired(messageSize); + + globalMetrics.recordInternodeDroppedMessage(header.verb, timeElapsed, unit); + } + + @Override + public void onArrivedCorrupt(int messageSize, Header header, long timeElapsed, TimeUnit unit) + { + counters.addError(messageSize); + + messageConsumer.fail(header, new Crc.InvalidCrc(0, 0)); // could use one of the original exceptions? + } + + @Override + public void onClosedBeforeArrival(int messageSize, Header header, int bytesReceived, boolean wasCorrupt, boolean wasExpired) + { + counters.addError(messageSize); + + messageConsumer.fail(header, new InvalidSerializedSizeException(header.verb, messageSize, bytesReceived)); + } + + @Override + public void onExpired(int messageSize, Header header, long timeElapsed, TimeUnit unit) + { + counters.addExpired(messageSize); + + globalMetrics.recordInternodeDroppedMessage(header.verb, timeElapsed, unit); + } + + @Override + public void onFailedDeserialize(int messageSize, Header header, Throwable t) + { + counters.addError(messageSize); + + /* + * If an exception is caught during deser, return a failure response immediately + * instead of waiting for the callback on the other end to expire. + */ + messageConsumer.fail(header, t); + } + + @Override + public void onDispatched(int messageSize, Header header) + { + counters.addPending(messageSize); + } + + @Override + public void onExecuting(int messageSize, Header header, long timeElapsed, TimeUnit unit) + { + globalMetrics.recordInternalLatency(header.verb, timeElapsed, unit); + } + + @Override + public void onExecuted(int messageSize, Header header, long timeElapsed, TimeUnit unit) + { + counters.removePending(messageSize); + } + + @Override + public void onProcessed(int messageSize, Header header) + { + counters.addProcessed(messageSize); + } + }; + } + + /* + * Aggregated counters + */ + + InboundCounters countersFor(ConnectionType type) + { + switch (type) + { + case URGENT_MESSAGES: return urgentCounters; + case SMALL_MESSAGES: return smallCounters; + case LARGE_MESSAGES: return largeCounters; + case LEGACY_MESSAGES: return legacyCounters; + } + + throw new IllegalArgumentException(); + } + + public long receivedCount() + { + return sumHandlers(h -> h.receivedCount) + closedReceivedCount; + } + + public long receivedBytes() + { + return sumHandlers(h -> h.receivedBytes) + closedReceivedBytes; + } + + public long throttledCount() + { + return sumHandlers(h -> h.throttledCount) + closedThrottledCount; + } + + public long throttledNanos() + { + return sumHandlers(h -> h.throttledNanos) + closedThrottledNanos; + } + + public long usingCapacity() + { + return sumHandlers(h -> h.queueSize); + } + + public long usingEndpointReserveCapacity() + { + return endpointReserveCapacity.using(); + } + + public long corruptFramesRecovered() + { + return sumHandlers(h -> h.corruptFramesRecovered) + closedCorruptFramesRecovered; + } + + public long corruptFramesUnrecovered() + { + return sumHandlers(h -> h.corruptFramesUnrecovered) + closedCorruptFramesUnrecovered; + } + + public long errorCount() + { + return sumCounters(InboundCounters::errorCount); + } + + public long errorBytes() + { + return sumCounters(InboundCounters::errorBytes); + } + + public long expiredCount() + { + return sumCounters(InboundCounters::expiredCount); + } + + public long expiredBytes() + { + return sumCounters(InboundCounters::expiredBytes); + } + + public long processedCount() + { + return sumCounters(InboundCounters::processedCount); + } + + public long processedBytes() + { + return sumCounters(InboundCounters::processedBytes); + } + + public long scheduledCount() + { + return sumCounters(InboundCounters::scheduledCount); + } + + public long scheduledBytes() + { + return sumCounters(InboundCounters::scheduledBytes); + } + + /* + * 'Archived' counter values, combined for all connections that have been closed. + */ + + private volatile long closedReceivedCount, closedReceivedBytes; + + private static final AtomicLongFieldUpdater closedReceivedCountUpdater = + AtomicLongFieldUpdater.newUpdater(InboundMessageHandlers.class, "closedReceivedCount"); + private static final AtomicLongFieldUpdater closedReceivedBytesUpdater = + AtomicLongFieldUpdater.newUpdater(InboundMessageHandlers.class, "closedReceivedBytes"); + + private volatile long closedThrottledCount, closedThrottledNanos; + + private static final AtomicLongFieldUpdater closedThrottledCountUpdater = + AtomicLongFieldUpdater.newUpdater(InboundMessageHandlers.class, "closedThrottledCount"); + private static final AtomicLongFieldUpdater closedThrottledNanosUpdater = + AtomicLongFieldUpdater.newUpdater(InboundMessageHandlers.class, "closedThrottledNanos"); + + private volatile long closedCorruptFramesRecovered, closedCorruptFramesUnrecovered; + + private static final AtomicLongFieldUpdater closedCorruptFramesRecoveredUpdater = + AtomicLongFieldUpdater.newUpdater(InboundMessageHandlers.class, "closedCorruptFramesRecovered"); + private static final AtomicLongFieldUpdater closedCorruptFramesUnrecoveredUpdater = + AtomicLongFieldUpdater.newUpdater(InboundMessageHandlers.class, "closedCorruptFramesUnrecovered"); + + private void absorbCounters(InboundMessageHandler handler) + { + closedReceivedCountUpdater.addAndGet(this, handler.receivedCount); + closedReceivedBytesUpdater.addAndGet(this, handler.receivedBytes); + + closedThrottledCountUpdater.addAndGet(this, handler.throttledCount); + closedThrottledNanosUpdater.addAndGet(this, handler.throttledNanos); + + closedCorruptFramesRecoveredUpdater.addAndGet(this, handler.corruptFramesRecovered); + closedCorruptFramesUnrecoveredUpdater.addAndGet(this, handler.corruptFramesUnrecovered); + } + + private long sumHandlers(ToLongFunction counter) + { + long sum = 0L; + for (InboundMessageHandler h : handlers) + sum += counter.applyAsLong(h); + return sum; + } + + private long sumCounters(ToLongFunction mapping) + { + return mapping.applyAsLong(urgentCounters) + + mapping.applyAsLong(smallCounters) + + mapping.applyAsLong(largeCounters) + + mapping.applyAsLong(legacyCounters); + } + + interface HandlerProvider + { + InboundMessageHandler provide(FrameDecoder decoder, + + ConnectionType type, + Channel channel, + InetAddressAndPort self, + InetAddressAndPort peer, + int version, + int largeMessageThreshold, + + int queueCapacity, + ResourceLimits.Limit endpointReserveCapacity, + ResourceLimits.Limit globalReserveCapacity, + InboundMessageHandler.WaitQueue endpointWaitQueue, + InboundMessageHandler.WaitQueue globalWaitQueue, + + InboundMessageHandler.OnHandlerClosed onClosed, + InboundMessageCallbacks callbacks, + Consumer> consumer); + } +} diff --git a/src/java/org/apache/cassandra/net/InboundSink.java b/src/java/org/apache/cassandra/net/InboundSink.java new file mode 100644 index 000000000000..df63be2d8bf8 --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundSink.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Predicate; + +import org.slf4j.LoggerFactory; + +import net.openhft.chronicle.core.util.ThrowingConsumer; +import org.apache.cassandra.db.filter.TombstoneOverwhelmingException; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.index.IndexNotAvailableException; +import org.apache.cassandra.utils.NoSpamLogger; + +/** + * A message sink that all inbound messages go through. + * + * Default sink used by {@link MessagingService} is {@link IVerbHandler#doVerb(Message)}, but it can be overridden + * to filter out certain messages, record the fact of attempted delivery, or delay arrival. + * + * This facility is most useful for test code. + * + * {@link #accept(Message)} is invoked on a thread belonging to the {@link org.apache.cassandra.concurrent.Stage} + * assigned to the {@link Verb} of the message. + */ +public class InboundSink implements InboundMessageHandlers.MessageConsumer +{ + private static final NoSpamLogger noSpamLogger = + NoSpamLogger.getLogger(LoggerFactory.getLogger(InboundSink.class), 1L, TimeUnit.SECONDS); + + private static class Filtered implements ThrowingConsumer, IOException> + { + final Predicate> condition; + final ThrowingConsumer, IOException> next; + + private Filtered(Predicate> condition, ThrowingConsumer, IOException> next) + { + this.condition = condition; + this.next = next; + } + + public void accept(Message message) throws IOException + { + if (condition.test(message)) + next.accept(message); + } + } + + @SuppressWarnings("FieldMayBeFinal") + private volatile ThrowingConsumer, IOException> sink; + private static final AtomicReferenceFieldUpdater sinkUpdater + = AtomicReferenceFieldUpdater.newUpdater(InboundSink.class, ThrowingConsumer.class, "sink"); + + private final MessagingService messaging; + + InboundSink(MessagingService messaging) + { + this.messaging = messaging; + this.sink = message -> message.header.verb.handler().doVerb((Message) message); + } + + public void fail(Message.Header header, Throwable failure) + { + if (header.callBackOnFailure()) + { + Message response = Message.failureResponse(header.id, header.expiresAtNanos, RequestFailureReason.forException(failure)); + messaging.send(response, header.from); + } + } + + public void accept(Message message) + { + try + { + sink.accept(message); + } + catch (Throwable t) + { + fail(message.header, t); + + if (t instanceof TombstoneOverwhelmingException || t instanceof IndexNotAvailableException) + noSpamLogger.error(t.getMessage()); + else if (t instanceof RuntimeException) + throw (RuntimeException) t; + else + throw new RuntimeException(t); + } + } + + public void add(Predicate> allow) + { + sinkUpdater.updateAndGet(this, sink -> new Filtered(allow, sink)); + } + + public void remove(Predicate> allow) + { + sinkUpdater.updateAndGet(this, sink -> without(sink, allow)); + } + + public void clear() + { + sinkUpdater.updateAndGet(this, InboundSink::clear); + } + + @Deprecated // TODO: this is not the correct way to do things + public boolean allow(Message message) + { + return allows(sink, message); + } + + private static ThrowingConsumer, IOException> clear(ThrowingConsumer, IOException> sink) + { + while (sink instanceof Filtered) + sink = ((Filtered) sink).next; + return sink; + } + + private static ThrowingConsumer, IOException> without(ThrowingConsumer, IOException> sink, Predicate> condition) + { + if (!(sink instanceof Filtered)) + return sink; + + Filtered filtered = (Filtered) sink; + ThrowingConsumer, IOException> next = without(filtered.next, condition); + return condition.equals(filtered.condition) ? next + : next == filtered.next + ? sink + : new Filtered(filtered.condition, next); + } + + private static boolean allows(ThrowingConsumer, IOException> sink, Message message) + { + while (sink instanceof Filtered) + { + Filtered filtered = (Filtered) sink; + if (!filtered.condition.test(message)) + return false; + sink = filtered.next; + } + return true; + } + +} diff --git a/src/java/org/apache/cassandra/net/InboundSockets.java b/src/java/org/apache/cassandra/net/InboundSockets.java new file mode 100644 index 000000000000..ad5c96fedd80 --- /dev/null +++ b/src/java/org/apache/cassandra/net/InboundSockets.java @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.group.ChannelGroup; +import io.netty.channel.group.DefaultChannelGroup; +import io.netty.util.concurrent.DefaultEventExecutor; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GlobalEventExecutor; +import io.netty.util.concurrent.PromiseNotifier; +import io.netty.util.concurrent.SucceededFuture; +import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.utils.FBUtilities; + +class InboundSockets +{ + /** + * A simple struct to wrap up the components needed for each listening socket. + */ + @VisibleForTesting + static class InboundSocket + { + public final InboundConnectionSettings settings; + + /** + * The base {@link Channel} that is doing the socket listen/accept. + * Null only until open() is invoked and {@link #binding} has yet to complete. + */ + private volatile Channel listen; + /** + * Once open() is invoked, this holds the future result of opening the socket, + * so that its completion can be waited on. Once complete, it sets itself to null. + */ + private volatile ChannelFuture binding; + + // purely to prevent close racing with open + private boolean closedWithoutOpening; + + /** + * A group of the open, inbound {@link Channel}s connected to this node. This is mostly interesting so that all of + * the inbound connections/channels can be closed when the listening socket itself is being closed. + */ + private final ChannelGroup connections; + private final DefaultEventExecutor executor; + + private InboundSocket(InboundConnectionSettings settings) + { + this.settings = settings; + this.executor = new DefaultEventExecutor(new NamedThreadFactory("Listen-" + settings.bindAddress)); + this.connections = new DefaultChannelGroup(settings.bindAddress.toString(), executor); + } + + private Future open() + { + return open(pipeline -> {}); + } + + private Future open(Consumer pipelineInjector) + { + synchronized (this) + { + if (listen != null) + return new SucceededFuture<>(GlobalEventExecutor.INSTANCE, null); + if (binding != null) + return binding; + if (closedWithoutOpening) + throw new IllegalStateException(); + binding = InboundConnectionInitiator.bind(settings, connections, pipelineInjector); + } + + return binding.addListener(ignore -> { + synchronized (this) + { + if (binding.isSuccess()) + listen = binding.channel(); + binding = null; + } + }); + } + + /** + * Close this socket and any connections created on it. Once closed, this socket may not be re-opened. + * + * This may not execute synchronously, so a Future is returned encapsulating its result. + */ + private Future close() + { + AsyncPromise done = AsyncPromise.uncancellable(GlobalEventExecutor.INSTANCE); + + Runnable close = () -> { + List> closing = new ArrayList<>(); + if (listen != null) + closing.add(listen.close()); + closing.add(connections.close()); + new FutureCombiner(closing) + .addListener(future -> executor.shutdownGracefully()) + .addListener(new PromiseNotifier<>(done)); + }; + + synchronized (this) + { + if (listen == null && binding == null) + { + closedWithoutOpening = true; + return new SucceededFuture<>(GlobalEventExecutor.INSTANCE, null); + } + + if (listen != null) + { + close.run(); + } + else + { + binding.cancel(true); + binding.addListener(future -> close.run()); + } + + return done; + } + } + + public boolean isOpen() + { + return listen != null && listen.isOpen(); + } + } + + private final List sockets; + + InboundSockets(InboundConnectionSettings template) + { + this(withDefaultBindAddresses(template)); + } + + InboundSockets(List templates) + { + this.sockets = bindings(templates); + } + + private static List withDefaultBindAddresses(InboundConnectionSettings template) + { + ImmutableList.Builder templates = ImmutableList.builder(); + templates.add(template.withBindAddress(FBUtilities.getLocalAddressAndPort())); + if (shouldListenOnBroadcastAddress()) + templates.add(template.withBindAddress(FBUtilities.getBroadcastAddressAndPort())); + return templates.build(); + } + + private static List bindings(List templates) + { + ImmutableList.Builder sockets = ImmutableList.builder(); + for (InboundConnectionSettings template : templates) + addBindings(template, sockets); + return sockets.build(); + } + + private static void addBindings(InboundConnectionSettings template, ImmutableList.Builder out) + { + InboundConnectionSettings settings = template.withDefaults(); + out.add(new InboundSocket(settings)); + if (settings.encryption.enable_legacy_ssl_storage_port && settings.encryption.enabled) + out.add(new InboundSocket(template.withLegacyDefaults())); + } + + public Future open(Consumer pipelineInjector) + { + List> opening = new ArrayList<>(); + for (InboundSocket socket : sockets) + opening.add(socket.open(pipelineInjector)); + + return new FutureCombiner(opening); + } + + public Future open() + { + List> opening = new ArrayList<>(); + for (InboundSocket socket : sockets) + opening.add(socket.open()); + return new FutureCombiner(opening); + } + + public boolean isListening() + { + for (InboundSocket socket : sockets) + if (socket.isOpen()) + return true; + return false; + } + + public Future close() + { + List> closing = new ArrayList<>(); + for (InboundSocket address : sockets) + closing.add(address.close()); + return new FutureCombiner(closing); + } + + private static boolean shouldListenOnBroadcastAddress() + { + return DatabaseDescriptor.shouldListenOnBroadcastAddress() + && !FBUtilities.getLocalAddressAndPort().equals(FBUtilities.getBroadcastAddressAndPort()); + } + + @VisibleForTesting + public List sockets() + { + return sockets; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/InvalidSerializedSizeException.java b/src/java/org/apache/cassandra/net/InvalidSerializedSizeException.java new file mode 100644 index 000000000000..5660fd1c3884 --- /dev/null +++ b/src/java/org/apache/cassandra/net/InvalidSerializedSizeException.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; + +import static java.lang.String.format; + +class InvalidSerializedSizeException extends IOException +{ + final Verb verb; + final long expectedSize; + final long actualSizeAtLeast; + + InvalidSerializedSizeException(Verb verb, long expectedSize, long actualSizeAtLeast) + { + super(format("Invalid serialized size; expected %d, actual size at least %d, for verb %s", expectedSize, actualSizeAtLeast, verb)); + this.verb = verb; + this.expectedSize = expectedSize; + this.actualSizeAtLeast = actualSizeAtLeast; + } + + InvalidSerializedSizeException(long expectedSize, long actualSizeAtLeast) + { + super(format("Invalid serialized size; expected %d, actual size at least %d", expectedSize, actualSizeAtLeast)); + this.verb = null; + this.expectedSize = expectedSize; + this.actualSizeAtLeast = actualSizeAtLeast; + } +} diff --git a/src/java/org/apache/cassandra/net/LatencyConsumer.java b/src/java/org/apache/cassandra/net/LatencyConsumer.java new file mode 100644 index 000000000000..3f10d4146a13 --- /dev/null +++ b/src/java/org/apache/cassandra/net/LatencyConsumer.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.TimeUnit; + +public interface LatencyConsumer +{ + void accept(long timeElapsed, TimeUnit unit); +} diff --git a/src/java/org/apache/cassandra/net/LatencySubscribers.java b/src/java/org/apache/cassandra/net/LatencySubscribers.java new file mode 100644 index 000000000000..823e6d0b4917 --- /dev/null +++ b/src/java/org/apache/cassandra/net/LatencySubscribers.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; + +import org.apache.cassandra.locator.InetAddressAndPort; + +/** + * Callback that {@link org.apache.cassandra.locator.DynamicEndpointSnitch} listens to in order + * to update host scores. + * + * FIXME: rename/specialise, since only used by DES? + */ +public class LatencySubscribers +{ + public interface Subscriber + { + void receiveTiming(InetAddressAndPort address, long latency, TimeUnit unit); + } + + private volatile Subscriber subscribers; + private static final AtomicReferenceFieldUpdater subscribersUpdater + = AtomicReferenceFieldUpdater.newUpdater(LatencySubscribers.class, Subscriber.class, "subscribers"); + + private static Subscriber merge(Subscriber a, Subscriber b) + { + if (a == null) return b; + if (b == null) return a; + return (address, latency, unit) -> { + a.receiveTiming(address, latency, unit); + b.receiveTiming(address, latency, unit); + }; + } + + public void subscribe(Subscriber subscriber) + { + subscribersUpdater.accumulateAndGet(this, subscriber, LatencySubscribers::merge); + } + + public void add(InetAddressAndPort address, long latency, TimeUnit unit) + { + Subscriber subscribers = this.subscribers; + if (subscribers != null) + subscribers.receiveTiming(address, latency, unit); + } + + /** + * Track latency information for the dynamic snitch + * + * @param cb the callback associated with this message -- this lets us know if it's a message type we're interested in + * @param address the host that replied to the message + */ + public void maybeAdd(RequestCallback cb, InetAddressAndPort address, long latency, TimeUnit unit) + { + if (cb.trackLatencyForSnitch()) + add(address, latency, unit); + } +} diff --git a/src/java/org/apache/cassandra/db/WriteResponse.java b/src/java/org/apache/cassandra/net/LegacyFlag.java similarity index 52% rename from src/java/org/apache/cassandra/db/WriteResponse.java rename to src/java/org/apache/cassandra/net/LegacyFlag.java index 0dddaaba88dd..b2781a1fa7b2 100644 --- a/src/java/org/apache/cassandra/db/WriteResponse.java +++ b/src/java/org/apache/cassandra/net/LegacyFlag.java @@ -15,48 +15,51 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.cassandra.db; +package org.apache.cassandra.net; import java.io.IOException; +import com.google.common.base.Preconditions; + import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -/* - * This empty response is sent by a replica to inform the coordinator that the write succeeded +/** + * Before 4.0 introduced flags field to {@link Message}, we used to encode flags in params field, + * using a dummy value (single byte set to 0). From now on, {@link MessageFlag} should be extended + * instead. + * + * Once 3.0/3.11 compatibility is phased out, this class should be removed. */ -public final class WriteResponse +@Deprecated +final class LegacyFlag { - public static final Serializer serializer = new Serializer(); - - private static final WriteResponse instance = new WriteResponse(); + static final LegacyFlag instance = new LegacyFlag(); - private WriteResponse() + private LegacyFlag() { } - public static MessageOut createMessage() + static IVersionedSerializer serializer = new IVersionedSerializer() { - return new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, instance, serializer); - } - - public static class Serializer implements IVersionedSerializer - { - public void serialize(WriteResponse wm, DataOutputPlus out, int version) throws IOException + public void serialize(LegacyFlag param, DataOutputPlus out, int version) throws IOException { + Preconditions.checkArgument(param == instance); + out.write(0); } - public WriteResponse deserialize(DataInputPlus in, int version) throws IOException + public LegacyFlag deserialize(DataInputPlus in, int version) throws IOException { + byte b = in.readByte(); + assert b == 0; return instance; } - public long serializedSize(WriteResponse response, int version) + public long serializedSize(LegacyFlag param, int version) { - return 0; + Preconditions.checkArgument(param == instance); + return 1; } - } + }; } diff --git a/src/java/org/apache/cassandra/net/LegacyLZ4Constants.java b/src/java/org/apache/cassandra/net/LegacyLZ4Constants.java new file mode 100644 index 000000000000..f4fca446fabe --- /dev/null +++ b/src/java/org/apache/cassandra/net/LegacyLZ4Constants.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +abstract class LegacyLZ4Constants +{ + static final int XXHASH_SEED = 0x9747B28C; + + static final int HEADER_LENGTH = 8 // magic number + + 1 // token + + 4 // compressed length + + 4 // uncompressed length + + 4; // checksum + + static final long MAGIC_NUMBER = (long) 'L' << 56 + | (long) 'Z' << 48 + | (long) '4' << 40 + | (long) 'B' << 32 + | 'l' << 24 + | 'o' << 16 + | 'c' << 8 + | 'k'; + + // offsets of header fields + static final int MAGIC_NUMBER_OFFSET = 0; + static final int TOKEN_OFFSET = 8; + static final int COMPRESSED_LENGTH_OFFSET = 9; + static final int UNCOMPRESSED_LENGTH_OFFSET = 13; + static final int CHECKSUM_OFFSET = 17; + + static final int DEFAULT_BLOCK_LENGTH = 1 << 15; // 32 KiB + static final int MAX_BLOCK_LENGTH = 1 << 25; // 32 MiB + + static final int BLOCK_TYPE_NON_COMPRESSED = 0x10; + static final int BLOCK_TYPE_COMPRESSED = 0x20; + + // xxhash to Checksum adapter discards most significant nibble of value ¯\_(ツ)_/¯ + static final int XXHASH_MASK = 0xFFFFFFF; +} diff --git a/src/java/org/apache/cassandra/net/LocalBufferPoolAllocator.java b/src/java/org/apache/cassandra/net/LocalBufferPoolAllocator.java new file mode 100644 index 000000000000..384563f7c5ff --- /dev/null +++ b/src/java/org/apache/cassandra/net/LocalBufferPoolAllocator.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; + +import io.netty.channel.EventLoop; +import org.apache.cassandra.utils.memory.BufferPool; + +/** + * Equivalent to {@link GlobalBufferPoolAllocator}, except explicitly using a specified + * {@link org.apache.cassandra.utils.memory.BufferPool.LocalPool} to allocate from. + * + * Exists to facilitate more efficient handling large messages on the inbound path, + * used by {@link ConnectionType#LARGE_MESSAGES} connections. + */ +class LocalBufferPoolAllocator extends BufferPoolAllocator +{ + private final BufferPool.LocalPool pool; + private final EventLoop eventLoop; + + LocalBufferPoolAllocator(EventLoop eventLoop) + { + this.pool = new BufferPool.LocalPool().recycleWhenFree(false); + this.eventLoop = eventLoop; + } + + @Override + ByteBuffer get(int size) + { + if (!eventLoop.inEventLoop()) + throw new IllegalStateException("get() called from outside of owning event loop"); + return pool.get(size, false); + } + + @Override + ByteBuffer getAtLeast(int size) + { + if (!eventLoop.inEventLoop()) + throw new IllegalStateException("getAtLeast() called from outside of owning event loop"); + return pool.get(size, true); + } + + @Override + public void release() + { + pool.release(); + } +} diff --git a/src/java/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueue.java b/src/java/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueue.java new file mode 100644 index 000000000000..4c73bdc9cd2e --- /dev/null +++ b/src/java/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueue.java @@ -0,0 +1,350 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.Collection; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Consumer; + +/** + * A concurrent many-producers-to-single-consumer linked queue. + * + * Based roughly on {@link java.util.concurrent.ConcurrentLinkedQueue}, except with simpler/cheaper consumer-side + * method implementations ({@link #poll()}, {@link #remove()}, {@link #drain(Consumer)}), and padding added + * to prevent false sharing. + * + * {@link #offer(Object)} provides volatile visibility semantics. {@link #offer(Object)} is lock-free, {@link #poll()} + * and all related consumer methods are wait-free. + * + * In addition to that, provides a {@link #relaxedPeekLastAndOffer(Object)} method that we use to avoid a CAS when + * putting message handlers onto the wait queue. + */ +class ManyToOneConcurrentLinkedQueue extends ManyToOneConcurrentLinkedQueueHead implements Queue +{ + @SuppressWarnings("unused") // pad two cache lines after the head to prevent false sharing + protected long p31, p32, p33, p34, p35, p36, p37, p38, p39, p40, p41, p42, p43, p44, p45; + + ManyToOneConcurrentLinkedQueue() + { + head = tail = new Node<>(null); + } + + /** + * See {@link #relaxedIsEmpty()}. + */ + @Override + public boolean isEmpty() + { + return relaxedIsEmpty(); + } + + /** + * When invoked by the consumer thread, the answer will always be accurate. + * When invoked by a non-consumer thread, it won't always be the case: + * - {@code true} result indicates that the queue IS empty, no matter what; + * - {@code false} result indicates that the queue MIGHT BE non-empty - the value of {@code head} might + * not yet have been made externally visible by the consumer thread. + */ + boolean relaxedIsEmpty() + { + return null == head.next; + } + + @Override + public int size() + { + int size = 0; + Node next = head; + while (null != (next = next.next)) + size++; + return size; + } + + @Override + public E peek() + { + Node next = head.next; + if (null == next) + return null; + return next.item; + } + + @Override + public E element() + { + E item = peek(); + if (null == item) + throw new NoSuchElementException("Queue is empty"); + return item; + } + + @Override + public E poll() + { + Node head = this.head; + Node next = head.next; + + if (null == next) + return null; + + this.lazySetHead(next); // update head reference to next before making previous head node unreachable, + head.lazySetNext(head); // to maintain the guarantee of tail being always reachable from head + + E item = next.item; + next.item = null; + return item; + } + + @Override + public E remove() + { + E item = poll(); + if (null == item) + throw new NoSuchElementException("Queue is empty"); + return item; + } + + @Override + public boolean remove(Object o) + { + if (null == o) + throw new NullPointerException(); + + Node prev = this.head; + Node next = prev.next; + + while (null != next) + { + if (o.equals(next.item)) + { + prev.lazySetNext(next.next); // update prev reference to next before making removed node unreachable, + next.lazySetNext(next); // to maintain the guarantee of tail being always reachable from head + + next.item = null; + return true; + } + + prev = next; + next = next.next; + } + + return false; + } + + /** + * Consume the queue in its entirety and feed every item to the provided {@link Consumer}. + * + * Exists primarily for convenience, and essentially just wraps {@link #poll()} in a loop. + * Yields no performance benefit over invoking {@link #poll()} manually - there just isn't + * anything to meaningfully amortise on the consumer side of this queue. + */ + void drain(Consumer consumer) + { + E item; + while ((item = poll()) != null) + consumer.accept(item); + } + + @Override + public boolean add(E e) + { + return offer(e); + } + + @Override + public boolean offer(E e) + { + internalOffer(e); return true; + } + + /** + * Adds the element to the queue and returns the item of the previous tail node. + * It's possible for the returned item to already have been consumed. + * + * @return previously last tail item in the queue, potentially stale + */ + E relaxedPeekLastAndOffer(E e) + { + return internalOffer(e); + } + + /** + * internalOffer() is based on {@link java.util.concurrent.ConcurrentLinkedQueue#offer(Object)}, + * written by Doug Lea and Martin Buchholz with assistance from members of JCP JSR-166 Expert Group + * and released to the public domain, as explained at http://creativecommons.org/publicdomain/zero/1.0/ + */ + private E internalOffer(E e) + { + if (null == e) + throw new NullPointerException(); + + final Node node = new Node<>(e); + + for (Node t = tail, p = t;;) + { + Node q = p.next; + if (q == null) + { + // p is last node + if (p.casNext(null, node)) + { + // successful CAS is the linearization point for e to become an element of this queue and for node to become "live". + if (p != t) // hop two nodes at a time + casTail(t, node); // failure is ok + return p.item; + } + // lost CAS race to another thread; re-read next + } + else if (p == q) + { + /* + * We have fallen off list. If tail is unchanged, it will also be off-list, in which case we need to + * jump to head, from which all live nodes are always reachable. Else the new tail is a better bet. + */ + p = (t != (t = tail)) ? t : head; + } + else + { + // check for tail updates after two hops + p = (p != t && t != (t = tail)) ? t : q; + } + } + } + + @Override + public boolean contains(Object o) + { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator iterator() + { + throw new UnsupportedOperationException(); + } + + @Override + public Object[] toArray() + { + throw new UnsupportedOperationException(); + } + + @Override + public T[] toArray(T[] a) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean containsAll(Collection c) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addAll(Collection c) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean removeAll(Collection c) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean retainAll(Collection c) + { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() + { + throw new UnsupportedOperationException(); + } +} + +class ManyToOneConcurrentLinkedQueueHead extends ManyToOneConcurrentLinkedQueuePadding2 +{ + protected volatile ManyToOneConcurrentLinkedQueue.Node head; + + private static final AtomicReferenceFieldUpdater headUpdater = + AtomicReferenceFieldUpdater.newUpdater(ManyToOneConcurrentLinkedQueueHead.class, Node.class, "head"); + + @SuppressWarnings("WeakerAccess") + protected void lazySetHead(Node val) + { + headUpdater.lazySet(this, val); + } +} + +class ManyToOneConcurrentLinkedQueuePadding2 extends ManyToOneConcurrentLinkedQueueTail +{ + @SuppressWarnings("unused") // pad two cache lines between tail and head to prevent false sharing + protected long p16, p17, p18, p19, p20, p21, p22, p23, p24, p25, p26, p27, p28, p29, p30; +} + +class ManyToOneConcurrentLinkedQueueTail extends ManyToOneConcurrentLinkedQueuePadding1 +{ + protected volatile ManyToOneConcurrentLinkedQueue.Node tail; + + private static final AtomicReferenceFieldUpdater tailUpdater = + AtomicReferenceFieldUpdater.newUpdater(ManyToOneConcurrentLinkedQueueTail.class, Node.class, "tail"); + + @SuppressWarnings({ "WeakerAccess", "UnusedReturnValue" }) + protected boolean casTail(Node expect, Node update) + { + return tailUpdater.compareAndSet(this, expect, update); + } +} + +class ManyToOneConcurrentLinkedQueuePadding1 +{ + @SuppressWarnings("unused") // pad two cache lines before the tail to prevent false sharing + protected long p1, p2, p3, p4, p5, p6, p7, p8, p9, p10, p11, p12, p13, p14, p15; + + static final class Node + { + E item; + volatile Node next; + + private static final AtomicReferenceFieldUpdater nextUpdater = + AtomicReferenceFieldUpdater.newUpdater(Node.class, Node.class, "next"); + + Node(E item) + { + this.item = item; + } + + @SuppressWarnings("SameParameterValue") + boolean casNext(Node expect, Node update) + { + return nextUpdater.compareAndSet(this, expect, update); + } + + void lazySetNext(Node val) + { + nextUpdater.lazySet(this, val); + } + } +} diff --git a/src/java/org/apache/cassandra/net/Message.java b/src/java/org/apache/cassandra/net/Message.java new file mode 100644 index 000000000000..05c1bfaa7b1a --- /dev/null +++ b/src/java/org/apache/cassandra/net/Message.java @@ -0,0 +1,1338 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.EnumMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.io.IVersionedAsymmetricSerializer; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.tracing.Tracing.TraceType; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MonotonicClockTranslation; + +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.db.TypeSizes.sizeof; +import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; +import static org.apache.cassandra.net.MessagingService.VERSION_3014; +import static org.apache.cassandra.net.MessagingService.VERSION_30; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.MessagingService.instance; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; +import static org.apache.cassandra.utils.vint.VIntCoding.computeUnsignedVIntSize; +import static org.apache.cassandra.utils.vint.VIntCoding.getUnsignedVInt; +import static org.apache.cassandra.utils.vint.VIntCoding.skipUnsignedVInt; + +/** + * Immutable main unit of internode communication - what used to be {@code MessageIn} and {@code MessageOut} fused + * in one class. + * + * @param The type of the message payload. + */ +public class Message +{ + public final Header header; + public final T payload; + + private Message(Header header, T payload) + { + this.header = header; + this.payload = payload; + } + + /** Sender of the message. */ + public InetAddressAndPort from() + { + return header.from; + } + + /** Whether the message has crossed the node boundary, that is whether it originated from another node. */ + public boolean isCrossNode() + { + return !from().equals(FBUtilities.getBroadcastAddressAndPort()); + } + + /** + * id of the request/message. In 4.0+ can be shared between multiple messages of the same logical request, + * whilst in versions above a new id would be allocated for each message sent. + */ + public long id() + { + return header.id; + } + + public Verb verb() + { + return header.verb; + } + + boolean isFailureResponse() + { + return verb() == Verb.FAILURE_RSP; + } + + /** + * Creation time of the message. If cross-node timeouts are enabled ({@link DatabaseDescriptor#hasCrossNodeTimeout()}, + * {@code deserialize()} will use the marshalled value, otherwise will use current time on the deserializing machine. + */ + public long createdAtNanos() + { + return header.createdAtNanos; + } + + public long expiresAtNanos() + { + return header.expiresAtNanos; + } + + /** For how long the message has lived. */ + public long elapsedSinceCreated(TimeUnit units) + { + return units.convert(approxTime.now() - createdAtNanos(), NANOSECONDS); + } + + public long creationTimeMillis() + { + return approxTime.translate().toMillisSinceEpoch(createdAtNanos()); + } + + /** Whether a failure response should be returned upon failure */ + boolean callBackOnFailure() + { + return header.callBackOnFailure(); + } + + /** See CASSANDRA-14145 */ + public boolean trackRepairedData() + { + return header.trackRepairedData(); + } + + /** Used for cross-DC write optimisation - pick one node in the DC and have it relay the write to its local peers */ + @Nullable + public ForwardingInfo forwardTo() + { + return header.forwardTo(); + } + + /** The originator of the request - used when forwarding and will differ from {@link #from()} */ + @Nullable + public InetAddressAndPort respondTo() + { + return header.respondTo(); + } + + @Nullable + public UUID traceSession() + { + return header.traceSession(); + } + + @Nullable + public TraceType traceType() + { + return header.traceType(); + } + + /* + * request/response convenience + */ + + /** + * Make a request {@link Message} with supplied verb and payload. Will fill in remaining fields + * automatically. + * + * If you know that you will need to set some params or flags - prefer using variants of {@code out()} + * that allow providing them at point of message constructions, rather than allocating new messages + * with those added flags and params. See {@code outWithFlag()}, {@code outWithFlags()}, and {@code outWithParam()} + * family. + */ + public static Message out(Verb verb, T payload) + { + assert !verb.isResponse(); + + return outWithParam(nextId(), verb, payload, null, null); + } + + public static Message outWithFlag(Verb verb, T payload, MessageFlag flag) + { + assert !verb.isResponse(); + return outWithParam(nextId(), verb, 0, payload, flag.addTo(0), null, null); + } + + public static Message outWithFlags(Verb verb, T payload, MessageFlag flag1, MessageFlag flag2) + { + assert !verb.isResponse(); + return outWithParam(nextId(), verb, 0, payload, flag2.addTo(flag1.addTo(0)), null, null); + } + + static Message outWithParam(long id, Verb verb, T payload, ParamType paramType, Object paramValue) + { + return outWithParam(id, verb, 0, payload, paramType, paramValue); + } + + private static Message outWithParam(long id, Verb verb, long expiresAtNanos, T payload, ParamType paramType, Object paramValue) + { + return outWithParam(id, verb, expiresAtNanos, payload, 0, paramType, paramValue); + } + + private static Message outWithParam(long id, Verb verb, long expiresAtNanos, T payload, int flags, ParamType paramType, Object paramValue) + { + if (payload == null) + throw new IllegalArgumentException(); + + InetAddressAndPort from = FBUtilities.getBroadcastAddressAndPort(); + long createdAtNanos = approxTime.now(); + if (expiresAtNanos == 0) + expiresAtNanos = verb.expiresAtNanos(createdAtNanos); + + return new Message<>(new Header(id, verb, from, createdAtNanos, expiresAtNanos, flags, buildParams(paramType, paramValue)), payload); + } + + public static Message internalResponse(Verb verb, T payload) + { + assert verb.isResponse(); + return outWithParam(0, verb, payload, null, null); + } + + /** Builds a response Message with provided payload, and all the right fields inferred from request Message */ + public Message responseWith(T payload) + { + return outWithParam(id(), verb().responseVerb, expiresAtNanos(), payload, null, null); + } + + /** Builds a response Message with no payload, and all the right fields inferred from request Message */ + public Message emptyResponse() + { + return responseWith(NoPayload.noPayload); + } + + /** Builds a failure response Message with an explicit reason, and fields inferred from request Message */ + public Message failureResponse(RequestFailureReason reason) + { + return failureResponse(id(), expiresAtNanos(), reason); + } + + static Message failureResponse(long id, long expiresAtNanos, RequestFailureReason reason) + { + return outWithParam(id, Verb.FAILURE_RSP, expiresAtNanos, reason, null, null); + } + + Message withCallBackOnFailure() + { + return new Message<>(header.withFlag(MessageFlag.CALL_BACK_ON_FAILURE), payload); + } + + public Message withForwardTo(ForwardingInfo peers) + { + return new Message<>(header.withParam(ParamType.FORWARD_TO, peers), payload); + } + + private static final EnumMap NO_PARAMS = new EnumMap<>(ParamType.class); + + private static Map buildParams(ParamType type, Object value) + { + Map params = NO_PARAMS; + if (Tracing.isTracing()) + params = Tracing.instance.addTraceHeaders(new EnumMap<>(ParamType.class)); + + if (type != null) + { + if (params.isEmpty()) + params = new EnumMap<>(ParamType.class); + params.put(type, value); + } + + return params; + } + + private static Map addParam(Map params, ParamType type, Object value) + { + if (type == null) + return params; + + params = new EnumMap<>(params); + params.put(type, value); + return params; + } + + /* + * id generation + */ + + private static final long NO_ID = 0L; // this is a valid ID for pre40 nodes + + private static final AtomicInteger nextId = new AtomicInteger(0); + + private static long nextId() + { + long id; + do + { + id = nextId.incrementAndGet(); + } + while (id == NO_ID); + + return id; + } + + /** + * WARNING: this is inaccurate for messages from pre40 nodes, which can use 0 as an id (but will do so rarely) + */ + @VisibleForTesting + boolean hasId() + { + return id() != NO_ID; + } + + /** we preface every message with this number so the recipient can validate the sender is sane */ + static final int PROTOCOL_MAGIC = 0xCA552DFA; + + static void validateLegacyProtocolMagic(int magic) throws InvalidLegacyProtocolMagic + { + if (magic != PROTOCOL_MAGIC) + throw new InvalidLegacyProtocolMagic(magic); + } + + public static final class InvalidLegacyProtocolMagic extends IOException + { + public final int read; + private InvalidLegacyProtocolMagic(int read) + { + super(String.format("Read %d, Expected %d", read, PROTOCOL_MAGIC)); + this.read = read; + } + } + + public String toString() + { + return "(from:" + from() + ", type:" + verb().stage + " verb:" + verb() + ')'; + } + + /** + * Split into a separate object to allow partial message deserialization without wasting work and allocation + * afterwards, if the entire message is necessary and available. + */ + public static class Header + { + public final long id; + public final Verb verb; + public final InetAddressAndPort from; + public final long createdAtNanos; + public final long expiresAtNanos; + private final int flags; + private final Map params; + + private Header(long id, Verb verb, InetAddressAndPort from, long createdAtNanos, long expiresAtNanos, int flags, Map params) + { + this.id = id; + this.verb = verb; + this.from = from; + this.createdAtNanos = createdAtNanos; + this.expiresAtNanos = expiresAtNanos; + this.flags = flags; + this.params = params; + } + + Header withFlag(MessageFlag flag) + { + return new Header(id, verb, from, createdAtNanos, expiresAtNanos, flag.addTo(flags), params); + } + + Header withParam(ParamType type, Object value) + { + return new Header(id, verb, from, createdAtNanos, expiresAtNanos, flags, addParam(params, type, value)); + } + + boolean callBackOnFailure() + { + return MessageFlag.CALL_BACK_ON_FAILURE.isIn(flags); + } + + boolean trackRepairedData() + { + return MessageFlag.TRACK_REPAIRED_DATA.isIn(flags); + } + + @Nullable + ForwardingInfo forwardTo() + { + return (ForwardingInfo) params.get(ParamType.FORWARD_TO); + } + + @Nullable + InetAddressAndPort respondTo() + { + return (InetAddressAndPort) params.get(ParamType.RESPOND_TO); + } + + @Nullable + public UUID traceSession() + { + return (UUID) params.get(ParamType.TRACE_SESSION); + } + + @Nullable + public TraceType traceType() + { + return (TraceType) params.getOrDefault(ParamType.TRACE_TYPE, TraceType.QUERY); + } + } + + @SuppressWarnings("WeakerAccess") + public static class Builder + { + private Verb verb; + private InetAddressAndPort from; + private T payload; + private int flags = 0; + private final Map params = new EnumMap<>(ParamType.class); + private long createdAtNanos; + private long expiresAtNanos; + private long id; + + private boolean hasId; + + private Builder() + { + } + + public Builder from(InetAddressAndPort from) + { + this.from = from; + return this; + } + + public Builder withPayload(T payload) + { + this.payload = payload; + return this; + } + + public Builder withFlag(MessageFlag flag) + { + flags = flag.addTo(flags); + return this; + } + + public Builder withFlags(int flags) + { + this.flags = flags; + return this; + } + + public Builder withParam(ParamType type, Object value) + { + params.put(type, value); + return this; + } + + public Builder withoutParam(ParamType type) + { + params.remove(type); + return this; + } + + public Builder withParams(Map params) + { + this.params.putAll(params); + return this; + } + + public Builder ofVerb(Verb verb) + { + this.verb = verb; + if (expiresAtNanos == 0 && verb != null && createdAtNanos != 0) + expiresAtNanos = verb.expiresAtNanos(createdAtNanos); + if (!this.verb.isResponse() && from == null) // default to sending from self if we're a request verb + from = FBUtilities.getBroadcastAddressAndPort(); + return this; + } + + public Builder withCreatedAt(long createdAtNanos) + { + this.createdAtNanos = createdAtNanos; + if (expiresAtNanos == 0 && verb != null) + expiresAtNanos = verb.expiresAtNanos(createdAtNanos); + return this; + } + + public Builder withExpiresAt(long expiresAtNanos) + { + this.expiresAtNanos = expiresAtNanos; + return this; + } + + public Builder withId(long id) + { + this.id = id; + hasId = true; + return this; + } + + public Message build() + { + if (verb == null) + throw new IllegalArgumentException(); + if (from == null) + throw new IllegalArgumentException(); + if (payload == null) + throw new IllegalArgumentException(); + + return new Message<>(new Header(hasId ? id : nextId(), verb, from, createdAtNanos, expiresAtNanos, flags, params), payload); + } + } + + public static Builder builder(Message message) + { + return new Builder().from(message.from()) + .withId(message.id()) + .ofVerb(message.verb()) + .withCreatedAt(message.createdAtNanos()) + .withExpiresAt(message.expiresAtNanos()) + .withFlags(message.header.flags) + .withParams(message.header.params) + .withPayload(message.payload); + } + + public static Builder builder(Verb verb, T payload) + { + return new Builder().ofVerb(verb) + .withCreatedAt(approxTime.now()) + .withPayload(payload); + } + + public static final Serializer serializer = new Serializer(); + + /** + * Each message contains a header with several fixed fields, an optional key-value params section, and then + * the message payload itself. Below is a visualization of the layout. + * + * The params are prefixed by the count of key-value pairs; this value is encoded as unsigned vint. + * An individual param has an unsvint id (more specifically, a {@link ParamType}), and a byte array value. + * The param value is prefixed with it's length, encoded as an unsigned vint, followed by by the value's bytes. + * + * Legacy Notes (see {@link Serializer#serialize(Message, DataOutputPlus, int)} for complete details): + * - pre 4.0, the IP address was sent along in the header, before the verb. The IP address may be either IPv4 (4 bytes) or IPv6 (16 bytes) + * - pre-4.0, the verb was encoded as a 4-byte integer; in 4.0 and up it is an unsigned vint + * - pre-4.0, the payloadSize was encoded as a 4-byte integer; in 4.0 and up it is an unsigned vint + * - pre-4.0, the count of param key-value pairs was encoded as a 4-byte integer; in 4.0 and up it is an unsigned vint + * - pre-4.0, param names were encoded as strings; in 4.0 they are encoded as enum id vints + * - pre-4.0, expiry time wasn't encoded at all; in 4.0 it's an unsigned vint + * - pre-4.0, message id was an int; in 4.0 and up it's an unsigned vint + * - pre-4.0, messages included PROTOCOL MAGIC BYTES; post-4.0, we rely on frame CRCs instead + * - pre-4.0, messages would serialize boolean params as dummy ONE_BYTEs; post-4.0 we have a dedicated 'flags' vint + * + *
+     * {@code
+     *            1 1 1 1 1 2 2 2 2 2 3
+     *  0 2 4 6 8 0 2 4 6 8 0 2 4 6 8 0
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * | Message ID (vint)             |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * | Creation timestamp (int)      |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * | Expiry (vint)                 |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * | Verb (vint)                   |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * | Flags (vint)                  |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * | Param count (vint)            |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * |                               /
+     * /           Params              /
+     * /                               |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * | Payload size (vint)           |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * |                               /
+     * /           Payload             /
+     * /                               |
+     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
+     * }
+     * 
+ */ + public static final class Serializer + { + private static final int CREATION_TIME_SIZE = 4; + + private Serializer() + { + } + + public void serialize(Message message, DataOutputPlus out, int version) throws IOException + { + if (version >= VERSION_40) + serializePost40(message, out, version); + else + serializePre40(message, out, version); + } + + public Message deserialize(DataInputPlus in, InetAddressAndPort peer, int version) throws IOException + { + return version >= VERSION_40 ? deserializePost40(in, peer, version) : deserializePre40(in, version); + } + + /** + * A partial variant of deserialize, taking in a previously deserialized {@link Header} as an argument. + * + * Skip deserializing the {@link Header} from the input stream in favour of using the provided header. + */ + public Message deserialize(DataInputPlus in, Header header, int version) throws IOException + { + return version >= VERSION_40 ? deserializePost40(in, header, version) : deserializePre40(in, header, version); + } + + private int serializedSize(Message message, int version) + { + return version >= VERSION_40 ? serializedSizePost40(message, version) : serializedSizePre40(message, version); + } + + /** + * Size of the next message in the stream. Returns -1 if there aren't sufficient bytes read yet to determine size. + */ + int inferMessageSize(ByteBuffer buf, int index, int limit, int version) throws InvalidLegacyProtocolMagic + { + int size = version >= VERSION_40 ? inferMessageSizePost40(buf, index, limit) : inferMessageSizePre40(buf, index, limit); + if (size > DatabaseDescriptor.getInternodeMaxMessageSizeInBytes()) + throw new OversizedMessageException(size); + return size; + } + + /** + * Partially deserialize the message - by only extracting the header and leaving the payload alone. + * + * To get the rest of the message without repeating the work done here, use {@link #deserialize(DataInputPlus, Header, int)} + * method. + * + * It's assumed that the provided buffer contains all the bytes necessary to deserialize the header fully. + */ + Header extractHeader(ByteBuffer buf, InetAddressAndPort from, long currentTimeNanos, int version) throws IOException + { + return version >= VERSION_40 + ? extractHeaderPost40(buf, from, currentTimeNanos, version) + : extractHeaderPre40(buf, currentTimeNanos, version); + } + + private static long getExpiresAtNanos(long createdAtNanos, long currentTimeNanos, long expirationPeriodNanos) + { + if (!DatabaseDescriptor.hasCrossNodeTimeout() || createdAtNanos > currentTimeNanos) + createdAtNanos = currentTimeNanos; + return createdAtNanos + expirationPeriodNanos; + } + + /* + * 4.0 ser/deser + */ + + private void serializeHeaderPost40(Header header, DataOutputPlus out, int version) throws IOException + { + out.writeUnsignedVInt(header.id); + // int cast cuts off the high-order half of the timestamp, which we can assume remains + // the same between now and when the recipient reconstructs it. + out.writeInt((int) approxTime.translate().toMillisSinceEpoch(header.createdAtNanos)); + out.writeUnsignedVInt(1 + NANOSECONDS.toMillis(header.expiresAtNanos - header.createdAtNanos)); + out.writeUnsignedVInt(header.verb.id); + out.writeUnsignedVInt(header.flags); + serializeParams(header.params, out, version); + } + + private Header deserializeHeaderPost40(DataInputPlus in, InetAddressAndPort peer, int version) throws IOException + { + long id = in.readUnsignedVInt(); + long currentTimeNanos = approxTime.now(); + MonotonicClockTranslation timeSnapshot = approxTime.translate(); + long creationTimeNanos = calculateCreationTimeNanos(in.readInt(), timeSnapshot, currentTimeNanos); + long expiresAtNanos = getExpiresAtNanos(creationTimeNanos, currentTimeNanos, TimeUnit.MILLISECONDS.toNanos(in.readUnsignedVInt())); + Verb verb = Verb.fromId(Ints.checkedCast(in.readUnsignedVInt())); + int flags = Ints.checkedCast(in.readUnsignedVInt()); + Map params = deserializeParams(in, version); + return new Header(id, verb, peer, creationTimeNanos, expiresAtNanos, flags, params); + } + + private void skipHeaderPost40(DataInputPlus in) throws IOException + { + skipUnsignedVInt(in); // id + in.skipBytesFully(4); // createdAt + skipUnsignedVInt(in); // expiresIn + skipUnsignedVInt(in); // verb + skipUnsignedVInt(in); // flags + skipParamsPost40(in); // params + } + + private int serializedHeaderSizePost40(Header header, int version) + { + long size = 0; + size += sizeofUnsignedVInt(header.id); + size += CREATION_TIME_SIZE; + size += sizeofUnsignedVInt(1 + NANOSECONDS.toMillis(header.expiresAtNanos - header.createdAtNanos)); + size += sizeofUnsignedVInt(header.verb.id); + size += sizeofUnsignedVInt(header.flags); + size += serializedParamsSize(header.params, version); + return Ints.checkedCast(size); + } + + private Header extractHeaderPost40(ByteBuffer buf, InetAddressAndPort from, long currentTimeNanos, int version) throws IOException + { + MonotonicClockTranslation timeSnapshot = approxTime.translate(); + + int index = buf.position(); + + long id = getUnsignedVInt(buf, index); + index += computeUnsignedVIntSize(id); + + int createdAtMillis = buf.getInt(index); + index += sizeof(createdAtMillis); + + long expiresInMillis = getUnsignedVInt(buf, index); + index += computeUnsignedVIntSize(expiresInMillis); + + Verb verb = Verb.fromId(Ints.checkedCast(getUnsignedVInt(buf, index))); + index += computeUnsignedVIntSize(verb.id); + + int flags = Ints.checkedCast(getUnsignedVInt(buf, index)); + index += computeUnsignedVIntSize(flags); + + Map params = extractParams(buf, index, version); + + long createdAtNanos = calculateCreationTimeNanos(createdAtMillis, timeSnapshot, currentTimeNanos); + long expiresAtNanos = getExpiresAtNanos(createdAtNanos, currentTimeNanos, TimeUnit.MILLISECONDS.toNanos(expiresInMillis)); + + return new Header(id, verb, from, createdAtNanos, expiresAtNanos, flags, params); + } + + private void serializePost40(Message message, DataOutputPlus out, int version) throws IOException + { + serializeHeaderPost40(message.header, out, version); + out.writeUnsignedVInt(message.payloadSize(version)); + message.verb().serializer().serialize(message.payload, out, version); + } + + private Message deserializePost40(DataInputPlus in, InetAddressAndPort peer, int version) throws IOException + { + Header header = deserializeHeaderPost40(in, peer, version); + skipUnsignedVInt(in); // payload size, not needed by payload deserializer + T payload = (T) header.verb.serializer().deserialize(in, version); + return new Message<>(header, payload); + } + + private Message deserializePost40(DataInputPlus in, Header header, int version) throws IOException + { + skipHeaderPost40(in); + skipUnsignedVInt(in); // payload size, not needed by payload deserializer + T payload = (T) header.verb.serializer().deserialize(in, version); + return new Message<>(header, payload); + } + + private int serializedSizePost40(Message message, int version) + { + long size = 0; + size += serializedHeaderSizePost40(message.header, version); + int payloadSize = message.payloadSize(version); + size += sizeofUnsignedVInt(payloadSize) + payloadSize; + return Ints.checkedCast(size); + } + + private int inferMessageSizePost40(ByteBuffer buf, int readerIndex, int readerLimit) + { + int index = readerIndex; + + int idSize = computeUnsignedVIntSize(buf, index, readerLimit); + if (idSize < 0) + return -1; // not enough bytes to read id + index += idSize; + + index += CREATION_TIME_SIZE; + if (index > readerLimit) + return -1; + + int expirationSize = computeUnsignedVIntSize(buf, index, readerLimit); + if (expirationSize < 0) + return -1; + index += expirationSize; + + int verbIdSize = computeUnsignedVIntSize(buf, index, readerLimit); + if (verbIdSize < 0) + return -1; + index += verbIdSize; + + int flagsSize = computeUnsignedVIntSize(buf, index, readerLimit); + if (flagsSize < 0) + return -1; + index += flagsSize; + + int paramsSize = extractParamsSizePost40(buf, index, readerLimit); + if (paramsSize < 0) + return -1; + index += paramsSize; + + long payloadSize = getUnsignedVInt(buf, index, readerLimit); + if (payloadSize < 0) + return -1; + index += computeUnsignedVIntSize(payloadSize) + payloadSize; + + return index - readerIndex; + } + + /* + * legacy ser/deser + */ + + private void serializeHeaderPre40(Header header, DataOutputPlus out, int version) throws IOException + { + out.writeInt(PROTOCOL_MAGIC); + out.writeInt(Ints.checkedCast(header.id)); + // int cast cuts off the high-order half of the timestamp, which we can assume remains + // the same between now and when the recipient reconstructs it. + out.writeInt((int) approxTime.translate().toMillisSinceEpoch(header.createdAtNanos)); + inetAddressAndPortSerializer.serialize(header.from, out, version); + out.writeInt(header.verb.toPre40Verb().id); + serializeParams(addFlagsToLegacyParams(header.params, header.flags), out, version); + } + + private Header deserializeHeaderPre40(DataInputPlus in, int version) throws IOException + { + validateLegacyProtocolMagic(in.readInt()); + int id = in.readInt(); + long currentTimeNanos = approxTime.now(); + MonotonicClockTranslation timeSnapshot = approxTime.translate(); + long creationTimeNanos = calculateCreationTimeNanos(in.readInt(), timeSnapshot, currentTimeNanos); + InetAddressAndPort from = inetAddressAndPortSerializer.deserialize(in, version); + Verb verb = Verb.fromId(in.readInt()); + Map params = deserializeParams(in, version); + int flags = removeFlagsFromLegacyParams(params); + return new Header(id, verb, from, creationTimeNanos, verb.expiresAtNanos(creationTimeNanos), flags, params); + } + + private static final int PRE_40_MESSAGE_PREFIX_SIZE = 12; // protocol magic + id + createdAt + + private void skipHeaderPre40(DataInputPlus in) throws IOException + { + in.skipBytesFully(PRE_40_MESSAGE_PREFIX_SIZE); // magic, id, createdAt + in.skipBytesFully(in.readByte()); // from + in.skipBytesFully(4); // verb + skipParamsPre40(in); // params + } + + private int serializedHeaderSizePre40(Header header, int version) + { + long size = 0; + size += PRE_40_MESSAGE_PREFIX_SIZE; + size += inetAddressAndPortSerializer.serializedSize(header.from, version); + size += sizeof(header.verb.id); + size += serializedParamsSize(addFlagsToLegacyParams(header.params, header.flags), version); + return Ints.checkedCast(size); + } + + private Header extractHeaderPre40(ByteBuffer buf, long currentTimeNanos, int version) throws IOException + { + MonotonicClockTranslation timeSnapshot = approxTime.translate(); + + int index = buf.position(); + + index += 4; // protocol magic + + long id = buf.getInt(index); + index += 4; + + int createdAtMillis = buf.getInt(index); + index += 4; + + InetAddressAndPort from = inetAddressAndPortSerializer.extract(buf, index); + index += 1 + buf.get(index); + + Verb verb = Verb.fromId(buf.getInt(index)); + index += 4; + + Map params = extractParams(buf, index, version); + int flags = removeFlagsFromLegacyParams(params); + + long createdAtNanos = calculateCreationTimeNanos(createdAtMillis, timeSnapshot, currentTimeNanos); + long expiresAtNanos = verb.expiresAtNanos(createdAtNanos); + + return new Header(id, verb, from, createdAtNanos, expiresAtNanos, flags, params); + } + + private void serializePre40(Message message, DataOutputPlus out, int version) throws IOException + { + if (message.isFailureResponse()) + message = toPre40FailureResponse(message); + + serializeHeaderPre40(message.header, out, version); + + if (message.payload != null && message.payload != NoPayload.noPayload) + { + int payloadSize = message.payloadSize(version); + out.writeInt(payloadSize); + message.verb().serializer().serialize(message.payload, out, version); + } + else + { + out.writeInt(0); + } + } + + private Message deserializePre40(DataInputPlus in, int version) throws IOException + { + Header header = deserializeHeaderPre40(in, version); + return deserializePre40(in, header, false, version); + } + + private Message deserializePre40(DataInputPlus in, Header header, int version) throws IOException + { + return deserializePre40(in, header, true, version); + } + + private Message deserializePre40(DataInputPlus in, Header header, boolean skipHeader, int version) throws IOException + { + if (skipHeader) + skipHeaderPre40(in); + + IVersionedAsymmetricSerializer payloadSerializer = header.verb.serializer(); + if (null == payloadSerializer) + payloadSerializer = instance().callbacks.responseSerializer(header.id, header.from); + int payloadSize = in.readInt(); + T payload = deserializePayloadPre40(in, version, payloadSerializer, payloadSize); + + Message message = new Message<>(header, payload); + + return header.params.containsKey(ParamType.FAILURE_RESPONSE) + ? (Message) toPost40FailureResponse(message) + : message; + } + + private T deserializePayloadPre40(DataInputPlus in, int version, IVersionedAsymmetricSerializer serializer, int payloadSize) throws IOException + { + if (payloadSize == 0 || serializer == null) + { + // if there's no deserializer for the verb, skip the payload bytes to leave + // the stream in a clean state (for the next message) + in.skipBytesFully(payloadSize); + return null; + } + + return serializer.deserialize(in, version); + } + + private int serializedSizePre40(Message message, int version) + { + if (message.isFailureResponse()) + message = toPre40FailureResponse(message); + + long size = 0; + size += serializedHeaderSizePre40(message.header, version); + int payloadSize = message.payloadSize(version); + size += sizeof(payloadSize); + size += payloadSize; + return Ints.checkedCast(size); + } + + private int inferMessageSizePre40(ByteBuffer buf, int readerIndex, int readerLimit) throws InvalidLegacyProtocolMagic + { + int index = readerIndex; + // protocol magic + index += 4; + if (index > readerLimit) + return -1; + validateLegacyProtocolMagic(buf.getInt(index - 4)); + + // rest of prefix + index += PRE_40_MESSAGE_PREFIX_SIZE - 4; + // ip address + index += 1; + if (index > readerLimit) + return -1; + index += buf.get(index - 1); + // verb + index += 4; + if (index > readerLimit) + return -1; + + int paramsSize = extractParamsSizePre40(buf, index, readerLimit); + if (paramsSize < 0) + return -1; + index += paramsSize; + + // payload + index += 4; + + if (index > readerLimit) + return -1; + index += buf.getInt(index - 4); + + return index - readerIndex; + } + + private Message toPre40FailureResponse(Message post40) + { + Map params = new EnumMap<>(ParamType.class); + params.putAll(post40.header.params); + + params.put(ParamType.FAILURE_RESPONSE, LegacyFlag.instance); + params.put(ParamType.FAILURE_REASON, post40.payload); + + Header header = new Header(post40.id(), post40.verb().toPre40Verb(), post40.from(), post40.createdAtNanos(), post40.expiresAtNanos(), 0, params); + return new Message<>(header, NoPayload.noPayload); + } + + private Message toPost40FailureResponse(Message pre40) + { + Map params = new EnumMap<>(ParamType.class); + params.putAll(pre40.header.params); + + params.remove(ParamType.FAILURE_RESPONSE); + + RequestFailureReason reason = (RequestFailureReason) params.remove(ParamType.FAILURE_REASON); + if (null == reason) + reason = RequestFailureReason.UNKNOWN; + + Header header = new Header(pre40.id(), Verb.FAILURE_RSP, pre40.from(), pre40.createdAtNanos(), pre40.expiresAtNanos(), pre40.header.flags, params); + return new Message<>(header, reason); + } + + /* + * created at + cross-node + */ + + private static final long TIMESTAMP_WRAPAROUND_GRACE_PERIOD_START = 0xFFFFFFFFL - MINUTES.toMillis(15L); + private static final long TIMESTAMP_WRAPAROUND_GRACE_PERIOD_END = MINUTES.toMillis(15L); + + private static long calculateCreationTimeNanos(int messageTimestampMillis, MonotonicClockTranslation timeSnapshot, long currentTimeNanos) + { + long currentTimeMillis = timeSnapshot.toMillisSinceEpoch(currentTimeNanos); + // Reconstruct the message construction time sent by the remote host (we sent only the lower 4 bytes, assuming the + // higher 4 bytes wouldn't change between the sender and receiver) + long highBits = currentTimeMillis & 0xFFFFFFFF00000000L; + + long sentLowBits = messageTimestampMillis & 0x00000000FFFFFFFFL; + long currentLowBits = currentTimeMillis & 0x00000000FFFFFFFFL; + + // if our sent bits occur within a grace period of a wrap around event, + // and our current bits are no more than the same grace period after a wrap around event, + // assume a wrap around has occurred, and deduct one highBit + if ( sentLowBits > TIMESTAMP_WRAPAROUND_GRACE_PERIOD_START + && currentLowBits < TIMESTAMP_WRAPAROUND_GRACE_PERIOD_END) + { + highBits -= 0x0000000100000000L; + } + + long sentTimeMillis = (highBits | sentLowBits); + return timeSnapshot.fromMillisSinceEpoch(sentTimeMillis); + } + + /* + * param ser/deser + */ + + private Map addFlagsToLegacyParams(Map params, int flags) + { + if (flags == 0) + return params; + + Map extended = new EnumMap<>(ParamType.class); + extended.putAll(params); + + if (MessageFlag.CALL_BACK_ON_FAILURE.isIn(flags)) + extended.put(ParamType.FAILURE_CALLBACK, LegacyFlag.instance); + + if (MessageFlag.TRACK_REPAIRED_DATA.isIn(flags)) + extended.put(ParamType.TRACK_REPAIRED_DATA, LegacyFlag.instance); + + return extended; + } + + private int removeFlagsFromLegacyParams(Map params) + { + int flags = 0; + + if (null != params.remove(ParamType.FAILURE_CALLBACK)) + flags = MessageFlag.CALL_BACK_ON_FAILURE.addTo(flags); + + if (null != params.remove(ParamType.TRACK_REPAIRED_DATA)) + flags = MessageFlag.TRACK_REPAIRED_DATA.addTo(flags); + + return flags; + } + + private void serializeParams(Map params, DataOutputPlus out, int version) throws IOException + { + if (version >= VERSION_40) + out.writeUnsignedVInt(params.size()); + else + out.writeInt(params.size()); + + for (Map.Entry kv : params.entrySet()) + { + ParamType type = kv.getKey(); + if (version >= VERSION_40) + out.writeUnsignedVInt(type.id); + else + out.writeUTF(type.legacyAlias); + + IVersionedSerializer serializer = type.serializer; + Object value = kv.getValue(); + + int length = Ints.checkedCast(serializer.serializedSize(value, version)); + if (version >= VERSION_40) + out.writeUnsignedVInt(length); + else + out.writeInt(length); + + serializer.serialize(value, out, version); + } + } + + private Map deserializeParams(DataInputPlus in, int version) throws IOException + { + int count = version >= VERSION_40 ? Ints.checkedCast(in.readUnsignedVInt()) : in.readInt(); + + if (count == 0) + return NO_PARAMS; + + Map params = new EnumMap<>(ParamType.class); + + for (int i = 0; i < count; i++) + { + ParamType type = version >= VERSION_40 + ? ParamType.lookUpById(Ints.checkedCast(in.readUnsignedVInt())) + : ParamType.lookUpByAlias(in.readUTF()); + + int length = version >= VERSION_40 + ? Ints.checkedCast(in.readUnsignedVInt()) + : in.readInt(); + + if (null != type) + params.put(type, type.serializer.deserialize(in, version)); + else + in.skipBytesFully(length); // forward compatibiliy with minor version changes + } + + return params; + } + + /* + * Extract post-4.0 params map from a ByteBuffer without modifying it. + */ + private Map extractParams(ByteBuffer buf, int readerIndex, int version) throws IOException + { + long count = version >= VERSION_40 ? getUnsignedVInt(buf, readerIndex) : buf.getInt(readerIndex); + + if (count == 0) + return NO_PARAMS; + + final int position = buf.position(); + buf.position(readerIndex); + + try (DataInputBuffer in = new DataInputBuffer(buf, false)) + { + return deserializeParams(in, version); + } + finally + { + buf.position(position); + } + } + + private void skipParamsPost40(DataInputPlus in) throws IOException + { + int count = Ints.checkedCast(in.readUnsignedVInt()); + + for (int i = 0; i < count; i++) + { + skipUnsignedVInt(in); + in.skipBytesFully(Ints.checkedCast(in.readUnsignedVInt())); + } + } + + private void skipParamsPre40(DataInputPlus in) throws IOException + { + int count = in.readInt(); + + for (int i = 0; i < count; i++) + { + in.skipBytesFully(in.readShort()); + in.skipBytesFully(in.readInt()); + } + } + + private long serializedParamsSize(Map params, int version) + { + long size = version >= VERSION_40 + ? computeUnsignedVIntSize(params.size()) + : sizeof(params.size()); + + for (Map.Entry kv : params.entrySet()) + { + ParamType type = kv.getKey(); + Object value = kv.getValue(); + + long valueLength = type.serializer.serializedSize(value, version); + + if (version >= VERSION_40) + size += sizeofUnsignedVInt(type.id) + sizeofUnsignedVInt(valueLength); + else + size += sizeof(type.legacyAlias) + 4; + + size += valueLength; + } + + return size; + } + + private int extractParamsSizePost40(ByteBuffer buf, int readerIndex, int readerLimit) + { + int index = readerIndex; + + long paramsCount = getUnsignedVInt(buf, index, readerLimit); + if (paramsCount < 0) + return -1; + index += computeUnsignedVIntSize(paramsCount); + + for (int i = 0; i < paramsCount; i++) + { + long type = getUnsignedVInt(buf, index, readerLimit); + if (type < 0) + return -1; + index += computeUnsignedVIntSize(type); + + long length = getUnsignedVInt(buf, index, readerLimit); + if (length < 0) + return -1; + index += computeUnsignedVIntSize(length) + length; + } + + return index - readerIndex; + } + + private int extractParamsSizePre40(ByteBuffer buf, int readerIndex, int readerLimit) + { + int index = readerIndex; + + index += 4; + if (index > readerLimit) + return -1; + int paramsCount = buf.getInt(index - 4); + + for (int i = 0; i < paramsCount; i++) + { + // try to read length and skip to the end of the param name + index += 2; + + if (index > readerLimit) + return -1; + index += buf.getShort(index - 2); + // try to read length and skip to the end of the param value + index += 4; + if (index > readerLimit) + return -1; + index += buf.getInt(index - 4); + } + + return index - readerIndex; + } + + private int payloadSize(Message message, int version) + { + long payloadSize = message.payload != null && message.payload != NoPayload.noPayload + ? message.verb().serializer().serializedSize(message.payload, version) + : 0; + return Ints.checkedCast(payloadSize); + } + } + + private int serializedSize30; + private int serializedSize3014; + private int serializedSize40; + + /** + * Serialized size of the entire message, for the provided messaging version. Caches the calculated value. + */ + public int serializedSize(int version) + { + switch (version) + { + case VERSION_30: + if (serializedSize30 == 0) + serializedSize30 = serializer.serializedSize(this, VERSION_30); + return serializedSize30; + case VERSION_3014: + if (serializedSize3014 == 0) + serializedSize3014 = serializer.serializedSize(this, VERSION_3014); + return serializedSize3014; + case VERSION_40: + if (serializedSize40 == 0) + serializedSize40 = serializer.serializedSize(this, VERSION_40); + return serializedSize40; + default: + throw new IllegalStateException(); + } + } + + private int payloadSize30 = -1; + private int payloadSize3014 = -1; + private int payloadSize40 = -1; + + private int payloadSize(int version) + { + switch (version) + { + case VERSION_30: + if (payloadSize30 < 0) + payloadSize30 = serializer.payloadSize(this, VERSION_30); + return payloadSize30; + case VERSION_3014: + if (payloadSize3014 < 0) + payloadSize3014 = serializer.payloadSize(this, VERSION_3014); + return payloadSize3014; + case VERSION_40: + if (payloadSize40 < 0) + payloadSize40 = serializer.payloadSize(this, VERSION_40); + return payloadSize40; + default: + throw new IllegalStateException(); + } + } + + static class OversizedMessageException extends RuntimeException + { + OversizedMessageException(int size) + { + super("Message of size " + size + " bytes exceeds allowed maximum of " + DatabaseDescriptor.getInternodeMaxMessageSizeInBytes() + " bytes"); + } + } +} diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java deleted file mode 100644 index 1b9090c3081b..000000000000 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.net; - -import java.io.IOException; -import java.util.EnumSet; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.primitives.Shorts; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.db.filter.TombstoneOverwhelmingException; -import org.apache.cassandra.db.monitoring.ApproximateTime; -import org.apache.cassandra.exceptions.RequestFailureReason; -import org.apache.cassandra.gms.Gossiper; -import org.apache.cassandra.index.IndexNotAvailableException; -import org.apache.cassandra.io.DummyByteVersionedSerializer; -import org.apache.cassandra.io.util.DataOutputBuffer; - -public class MessageDeliveryTask implements Runnable -{ - private static final Logger logger = LoggerFactory.getLogger(MessageDeliveryTask.class); - - private final MessageIn message; - private final int id; - private final long enqueueTime; - - public MessageDeliveryTask(MessageIn message, int id) - { - assert message != null; - this.message = message; - this.id = id; - this.enqueueTime = ApproximateTime.currentTimeMillis(); - } - - public void run() - { - process(); - } - - /** - * A helper function for making unit testing reasonable. - * - * @return true if the message was processed; else false. - */ - @VisibleForTesting - boolean process() - { - MessagingService.Verb verb = message.verb; - if (verb == null) - { - logger.trace("Unknown verb {}", verb); - return false; - } - - MessagingService.instance().metrics.addQueueWaitTime(verb.toString(), - ApproximateTime.currentTimeMillis() - enqueueTime); - - long timeTaken = message.getLifetimeInMS(); - if (MessagingService.DROPPABLE_VERBS.contains(verb) - && timeTaken > message.getTimeout()) - { - MessagingService.instance().incrementDroppedMessages(message, timeTaken); - return false; - } - - IVerbHandler verbHandler = MessagingService.instance().getVerbHandler(verb); - if (verbHandler == null) - { - logger.trace("No handler for verb {}", verb); - return false; - } - - try - { - verbHandler.doVerb(message, id); - } - catch (IOException ioe) - { - handleFailure(ioe); - throw new RuntimeException(ioe); - } - catch (TombstoneOverwhelmingException | IndexNotAvailableException e) - { - handleFailure(e); - logger.error(e.getMessage()); - } - catch (Throwable t) - { - handleFailure(t); - throw t; - } - - if (GOSSIP_VERBS.contains(message.verb)) - Gossiper.instance.setLastProcessedMessageAt(message.constructionTime); - return true; - } - - private void handleFailure(Throwable t) - { - if (message.doCallbackOnFailure()) - { - MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE) - .withParameter(ParameterType.FAILURE_RESPONSE, MessagingService.ONE_BYTE); - - if (t instanceof TombstoneOverwhelmingException) - { - response = response.withParameter(ParameterType.FAILURE_REASON, Shorts.checkedCast(RequestFailureReason.READ_TOO_MANY_TOMBSTONES.code)); - } - - MessagingService.instance().sendReply(response, id, message.from); - } - } - - private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, - MessagingService.Verb.GOSSIP_DIGEST_ACK2, - MessagingService.Verb.GOSSIP_DIGEST_SYN); -} diff --git a/src/java/org/apache/cassandra/net/MessageFlag.java b/src/java/org/apache/cassandra/net/MessageFlag.java new file mode 100644 index 000000000000..c74784d4caba --- /dev/null +++ b/src/java/org/apache/cassandra/net/MessageFlag.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import static java.lang.Math.max; + +/** + * Binary message flags to be passed as {@code flags} field of {@link Message}. + */ +public enum MessageFlag +{ + /** a failure response should be sent back in case of failure */ + CALL_BACK_ON_FAILURE (0), + /** track repaired data - see CASSANDRA-14145 */ + TRACK_REPAIRED_DATA (1); + + private final int id; + + MessageFlag(int id) + { + this.id = id; + } + + /** + * @return {@code true} if the flag is present in provided flags, {@code false} otherwise + */ + boolean isIn(int flags) + { + return (flags & (1 << id)) != 0; + } + + /** + * @return new flags value with this flag added + */ + int addTo(int flags) + { + return flags | (1 << id); + } + + private static final MessageFlag[] idToFlagMap; + static + { + MessageFlag[] flags = values(); + + int max = -1; + for (MessageFlag flag : flags) + max = max(flag.id, max); + + MessageFlag[] idMap = new MessageFlag[max + 1]; + for (MessageFlag flag : flags) + { + if (idMap[flag.id] != null) + throw new RuntimeException("Two MessageFlag-s that map to the same id: " + flag.id); + idMap[flag.id] = flag; + } + idToFlagMap = idMap; + } + + @SuppressWarnings("unused") + MessageFlag lookUpById(int id) + { + if (id < 0) + throw new IllegalArgumentException("MessageFlag id must be non-negative (got " + id + ')'); + + return id < idToFlagMap.length ? idToFlagMap[id] : null; + } +} + diff --git a/src/java/org/apache/cassandra/net/MessageIn.java b/src/java/org/apache/cassandra/net/MessageIn.java deleted file mode 100644 index c8f4bfcd2608..000000000000 --- a/src/java/org/apache/cassandra/net/MessageIn.java +++ /dev/null @@ -1,234 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.net; - -import java.io.IOException; -import java.util.Collections; -import java.util.Map; - -import com.google.common.collect.ImmutableMap; - -import org.apache.cassandra.concurrent.Stage; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.monitoring.ApproximateTime; -import org.apache.cassandra.exceptions.RequestFailureReason; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService.Verb; -import org.apache.cassandra.utils.FBUtilities; - -/** - * The receiving node's view of a {@link MessageOut}. See documentation on {@link MessageOut} for details on the - * serialization format. - * - * @param The type of the payload - */ -public class MessageIn -{ - public final InetAddressAndPort from; - public final T payload; - public final Map parameters; - public final MessagingService.Verb verb; - public final int version; - public final long constructionTime; - - public MessageIn(InetAddressAndPort from, - T payload, - Map parameters, - Verb verb, - int version, - long constructionTime) - { - this.from = from; - this.payload = payload; - this.parameters = parameters; - this.verb = verb; - this.version = version; - this.constructionTime = constructionTime; - } - - public static MessageIn create(InetAddressAndPort from, - T payload, - Map parameters, - Verb verb, - int version, - long constructionTime) - { - return new MessageIn<>(from, payload, parameters, verb, version, constructionTime); - } - - public static MessageIn create(InetAddressAndPort from, - T payload, - Map parameters, - MessagingService.Verb verb, - int version) - { - return new MessageIn<>(from, payload, parameters, verb, version, ApproximateTime.currentTimeMillis()); - } - - public static MessageIn read(DataInputPlus in, int version, int id) throws IOException - { - return read(in, version, id, ApproximateTime.currentTimeMillis()); - } - - public static MessageIn read(DataInputPlus in, int version, int id, long constructionTime) throws IOException - { - InetAddressAndPort from = CompactEndpointSerializationHelper.instance.deserialize(in, version); - - MessagingService.Verb verb = MessagingService.Verb.fromId(in.readInt()); - Map parameters = readParameters(in, version); - int payloadSize = in.readInt(); - return read(in, version, id, constructionTime, from, payloadSize, verb, parameters); - } - - public static Map readParameters(DataInputPlus in, int version) throws IOException - { - int parameterCount = in.readInt(); - Map parameters; - if (parameterCount == 0) - { - return Collections.emptyMap(); - } - else - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (int i = 0; i < parameterCount; i++) - { - String key = in.readUTF(); - ParameterType type = ParameterType.byName.get(key); - if (type != null) - { - byte[] value = new byte[in.readInt()]; - in.readFully(value); - try (DataInputBuffer buffer = new DataInputBuffer(value)) - { - builder.put(type, type.serializer.deserialize(buffer, version)); - } - } - else - { - in.skipBytes(in.readInt()); - } - } - return builder.build(); - } - } - - public static MessageIn read(DataInputPlus in, int version, int id, long constructionTime, - InetAddressAndPort from, int payloadSize, Verb verb, Map parameters) throws IOException - { - IVersionedSerializer serializer = (IVersionedSerializer) MessagingService.verbSerializers.get(verb); - if (serializer instanceof MessagingService.CallbackDeterminedSerializer) - { - CallbackInfo callback = MessagingService.instance().getRegisteredCallback(id); - if (callback == null) - { - // reply for expired callback. we'll have to skip it. - in.skipBytesFully(payloadSize); - return null; - } - serializer = (IVersionedSerializer) callback.serializer; - } - - if (payloadSize == 0 || serializer == null) - { - // if there's no deserializer for the verb, skip the payload bytes to leave - // the stream in a clean state (for the next message) - in.skipBytesFully(payloadSize); - return create(from, null, parameters, verb, version, constructionTime); - } - - T2 payload = serializer.deserialize(in, version); - return MessageIn.create(from, payload, parameters, verb, version, constructionTime); - } - - public static long deriveConstructionTime(InetAddressAndPort from, int messageTimestamp, long currentTime) - { - // Reconstruct the message construction time sent by the remote host (we sent only the lower 4 bytes, assuming the - // higher 4 bytes wouldn't change between the sender and receiver) - long sentConstructionTime = (currentTime & 0xFFFFFFFF00000000L) | (((messageTimestamp & 0xFFFFFFFFL) << 2) >> 2); - - // Because nodes may not have their clock perfectly in sync, it's actually possible the sentConstructionTime is - // later than the currentTime (the received time). If that's the case, as we definitively know there is a lack - // of proper synchronziation of the clock, we ignore sentConstructionTime. We also ignore that - // sentConstructionTime if we're told to. - long elapsed = currentTime - sentConstructionTime; - if (elapsed > 0) - MessagingService.instance().metrics.addTimeTaken(from, elapsed); - - boolean useSentTime = DatabaseDescriptor.hasCrossNodeTimeout() && elapsed > 0; - return useSentTime ? sentConstructionTime : currentTime; - } - - /** - * Since how long (in milliseconds) the message has lived. - */ - public long getLifetimeInMS() - { - return ApproximateTime.currentTimeMillis() - constructionTime; - } - - /** - * Whether the message has crossed the node boundary, that is whether it originated from another node. - * - */ - public boolean isCrossNode() - { - return !from.equals(FBUtilities.getBroadcastAddressAndPort()); - } - - public Stage getMessageType() - { - return MessagingService.verbStages.get(verb); - } - - public boolean doCallbackOnFailure() - { - return parameters.containsKey(ParameterType.FAILURE_CALLBACK); - } - - public boolean isFailureResponse() - { - return parameters.containsKey(ParameterType.FAILURE_RESPONSE); - } - - public RequestFailureReason getFailureReason() - { - Short code = (Short)parameters.get(ParameterType.FAILURE_REASON); - return code != null ? RequestFailureReason.fromCode(code) : RequestFailureReason.UNKNOWN; - } - - public long getTimeout() - { - return verb.getTimeout(); - } - - public long getSlowQueryTimeout() - { - return DatabaseDescriptor.getSlowQueryTimeout(); - } - - public String toString() - { - StringBuilder sbuf = new StringBuilder(); - sbuf.append("FROM:").append(from).append(" TYPE:").append(getMessageType()).append(" VERB:").append(verb); - return sbuf.toString(); - } -} diff --git a/src/java/org/apache/cassandra/net/MessageOut.java b/src/java/org/apache/cassandra/net/MessageOut.java deleted file mode 100644 index 834435e3bd73..000000000000 --- a/src/java/org/apache/cassandra/net/MessageOut.java +++ /dev/null @@ -1,406 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net; - -import java.io.IOError; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.common.primitives.Ints; - -import org.apache.cassandra.concurrent.Stage; -import org.apache.cassandra.db.TypeSizes; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType; -import org.apache.cassandra.tracing.Tracing; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.vint.VIntCoding; - -import static org.apache.cassandra.tracing.Tracing.isTracing; - -/** - * Each message contains a header with several fixed fields, an optional key-value parameters section, and then - * the message payload itself. Note: the legacy IP address (pre-4.0) in the header may be either IPv4 (4 bytes) - * or IPv6 (16 bytes). The diagram below shows the IPv4 address for brevity. In pre-4.0, the payloadSize was - * encoded as a 4-byte integer; in 4.0 and up it is an unsigned byte (255 parameters should be enough for anyone). - * - *
- * {@code
- *            1 1 1 1 1 2 2 2 2 2 3
- *  0 2 4 6 8 0 2 4 6 8 0 2 4 6 8 0
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * |       PROTOCOL MAGIC          |
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * |        Message ID             |
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * |        Timestamp              |
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * |          Verb                 |
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * |ParmLen| Parameter data (var)  |
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * |   Payload size (vint)         |
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * |                               /
- * /           Payload             /
- * /                               |
- * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
- * }
- * 
- * - * An individual parameter has a String key and a byte array value. The key is serialized with it's length, - * encoded as two bytes, followed by the UTF-8 byte encoding of the string (see {@link java.io.DataOutput#writeUTF(java.lang.String)}). - * The body is serialized with it's length, encoded as four bytes, followed by the bytes of the value. - * - * * @param The type of the message payload. - */ -public class MessageOut -{ - private static final int SERIALIZED_SIZE_VERSION_UNDEFINED = -1; - //Parameters are stored in an object array as tuples of size two - public static final int PARAMETER_TUPLE_SIZE = 2; - //Offset in a parameter tuple containing the type of the parameter - public static final int PARAMETER_TUPLE_TYPE_OFFSET = 0; - //Offset in a parameter tuple containing the actual parameter represented as a POJO - public static final int PARAMETER_TUPLE_PARAMETER_OFFSET = 1; - - public final InetAddressAndPort from; - public final MessagingService.Verb verb; - public final T payload; - public final IVersionedSerializer serializer; - //A list of tuples, first object is the ParameterType enum value, - //the second object is the POJO to serialize - public final List parameters; - - /** - * Allows sender to explicitly state which connection type the message should be sent on. - */ - public final ConnectionType connectionType; - - /** - * Memoization of the serialized size of the just the payload. - */ - private int payloadSerializedSize = -1; - - /** - * Memoization of the serialized size of the entire message. - */ - private int serializedSize = -1; - - /** - * The internode protocol messaging version that was used to calculate the memoized serailized sizes. - */ - private int serializedSizeVersion = SERIALIZED_SIZE_VERSION_UNDEFINED; - - // we do support messages that just consist of a verb - public MessageOut(MessagingService.Verb verb) - { - this(verb, null, null); - } - - public MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer serializer) - { - this(verb, - payload, - serializer, - isTracing() ? Tracing.instance.getTraceHeaders() : ImmutableList.of(), - null); - } - - public MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer serializer, ConnectionType connectionType) - { - this(verb, - payload, - serializer, - isTracing() ? Tracing.instance.getTraceHeaders() : ImmutableList.of(), - connectionType); - } - - private MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer serializer, List parameters, ConnectionType connectionType) - { - this(FBUtilities.getBroadcastAddressAndPort(), verb, payload, serializer, parameters, connectionType); - } - - @VisibleForTesting - public MessageOut(InetAddressAndPort from, MessagingService.Verb verb, T payload, IVersionedSerializer serializer, List parameters, ConnectionType connectionType) - { - this.from = from; - this.verb = verb; - this.payload = payload; - this.serializer = serializer; - this.parameters = parameters; - this.connectionType = connectionType; - } - - public MessageOut withParameter(ParameterType type, VT value) - { - List newParameters = new ArrayList<>(parameters.size() + 2); - newParameters.addAll(parameters); - newParameters.add(type); - newParameters.add(value); - return new MessageOut(from, verb, payload, serializer, newParameters, connectionType); - } - - public Stage getStage() - { - return MessagingService.verbStages.get(verb); - } - - public long getTimeout() - { - return verb.getTimeout(); - } - - public String toString() - { - StringBuilder sbuf = new StringBuilder(); - sbuf.append("TYPE:").append(getStage()).append(" VERB:").append(verb); - return sbuf.toString(); - } - - public void serialize(DataOutputPlus out, int version) throws IOException - { - if (version >= MessagingService.VERSION_40) - serialize40(out, version); - else - serializePre40(out, version); - } - - private void serialize40(DataOutputPlus out, int version) throws IOException - { - out.writeInt(verb.getId()); - - // serialize the headers, if any - assert parameters.size() % PARAMETER_TUPLE_SIZE == 0; - if (parameters.isEmpty()) - { - out.writeVInt(0); - } - else - { - try (DataOutputBuffer buf = new DataOutputBuffer()) - { - serializeParams(buf, version); - out.writeUnsignedVInt(buf.getLength()); - out.write(buf.buffer()); - } - } - - if (payload != null) - { - int payloadSize = payloadSerializedSize >= 0 - ? payloadSerializedSize - : (int) serializer.serializedSize(payload, version); - - out.writeUnsignedVInt(payloadSize); - serializer.serialize(payload, out, version); - } - else - { - out.writeUnsignedVInt(0); - } - } - - private void serializePre40(DataOutputPlus out, int version) throws IOException - { - CompactEndpointSerializationHelper.instance.serialize(from, out, version); - out.writeInt(verb.getId()); - - assert parameters.size() % PARAMETER_TUPLE_SIZE == 0; - out.writeInt(parameters.size() / PARAMETER_TUPLE_SIZE); - serializeParams(out, version); - - if (payload != null) - { - int payloadSize = payloadSerializedSize >= 0 - ? payloadSerializedSize - : (int) serializer.serializedSize(payload, version); - - out.writeInt(payloadSize); - serializer.serialize(payload, out, version); - } - else - { - out.writeInt(0); - } - } - - private void serializeParams(DataOutputPlus out, int version) throws IOException - { - for (int ii = 0; ii < parameters.size(); ii += PARAMETER_TUPLE_SIZE) - { - ParameterType type = (ParameterType)parameters.get(ii + PARAMETER_TUPLE_TYPE_OFFSET); - out.writeUTF(type.key); - IVersionedSerializer serializer = type.serializer; - Object parameter = parameters.get(ii + PARAMETER_TUPLE_PARAMETER_OFFSET); - - int valueLength = Ints.checkedCast(serializer.serializedSize(parameter, version)); - if (version >= MessagingService.VERSION_40) - out.writeUnsignedVInt(valueLength); - else - out.writeInt(valueLength); - - serializer.serialize(parameter, out, version); - } - } - - private MessageOutSizes calculateSerializedSize(int version) - { - return version >= MessagingService.VERSION_40 - ? calculateSerializedSize40(version) - : calculateSerializedSizePre40(version); - } - - private MessageOutSizes calculateSerializedSize40(int version) - { - long size = 0; - size += TypeSizes.sizeof(verb.getId()); - - if (parameters.isEmpty()) - { - size += VIntCoding.computeVIntSize(0); - } - else - { - // calculate the params size independently, as we write that before the actual params block - int paramsSize = 0; - for (int ii = 0; ii < parameters.size(); ii += PARAMETER_TUPLE_SIZE) - { - ParameterType type = (ParameterType)parameters.get(ii + PARAMETER_TUPLE_TYPE_OFFSET); - paramsSize += TypeSizes.sizeof(type.key()); - IVersionedSerializer serializer = type.serializer; - Object parameter = parameters.get(ii + PARAMETER_TUPLE_PARAMETER_OFFSET); - int valueLength = Ints.checkedCast(serializer.serializedSize(parameter, version)); - paramsSize += VIntCoding.computeUnsignedVIntSize(valueLength);//length prefix - paramsSize += valueLength; - } - size += VIntCoding.computeUnsignedVIntSize(paramsSize); - size += paramsSize; - } - - long payloadSize = payload == null ? 0 : serializer.serializedSize(payload, version); - assert payloadSize <= Integer.MAX_VALUE; // larger values are supported in sstables but not messages - size += VIntCoding.computeUnsignedVIntSize(payloadSize); - size += payloadSize; - return new MessageOutSizes(size, payloadSize); - } - - private MessageOutSizes calculateSerializedSizePre40(int version) - { - long size = 0; - size += CompactEndpointSerializationHelper.instance.serializedSize(from, version); - - size += TypeSizes.sizeof(verb.getId()); - size += TypeSizes.sizeof(parameters.size() / PARAMETER_TUPLE_SIZE); - for (int ii = 0; ii < parameters.size(); ii += PARAMETER_TUPLE_SIZE) - { - ParameterType type = (ParameterType)parameters.get(ii + PARAMETER_TUPLE_TYPE_OFFSET); - size += TypeSizes.sizeof(type.key()); - size += 4;//length prefix - IVersionedSerializer serializer = type.serializer; - Object parameter = parameters.get(ii + PARAMETER_TUPLE_PARAMETER_OFFSET); - size += serializer.serializedSize(parameter, version); - } - - long payloadSize = payload == null ? 0 : serializer.serializedSize(payload, version); - assert payloadSize <= Integer.MAX_VALUE; // larger values are supported in sstables but not messages - size += TypeSizes.sizeof((int) payloadSize); - size += payloadSize; - return new MessageOutSizes(size, payloadSize); - } - - /** - * Calculate the size of this message for the specified protocol version and memoize the result for the specified - * protocol version. Memoization only covers the protocol version of the first invocation. - * - * It is not safe to call this function concurrently from multiple threads unless it has already been invoked - * once from a single thread and there is a happens before relationship between that invocation and other - * threads concurrently invoking this function. - * - * For instance it would be safe to invokePayload size to make a decision in the thread that created the message - * and then hand it off to other threads via a thread-safe queue, volatile write, or synchronized/ReentrantLock. - * - * @param version Protocol version to use when calculating size - * @return Size of this message in bytes, which will be less than or equal to {@link Integer#MAX_VALUE} - */ - public int serializedSize(int version) - { - if (serializedSize > 0 && serializedSizeVersion == version) - return serializedSize; - - MessageOutSizes sizes = calculateSerializedSize(version); - if (sizes.messageSize > Integer.MAX_VALUE) - throw new IllegalStateException("message size exceeds maximum allowed size: size = " + sizes.messageSize); - - if (serializedSizeVersion == SERIALIZED_SIZE_VERSION_UNDEFINED) - { - serializedSize = Ints.checkedCast(sizes.messageSize); - payloadSerializedSize = Ints.checkedCast(sizes.payloadSize); - serializedSizeVersion = version; - } - - return Ints.checkedCast(sizes.messageSize); - } - - public Object getParameter(ParameterType type) - { - for (int ii = 0; ii < parameters.size(); ii += PARAMETER_TUPLE_SIZE) - { - if (((ParameterType)parameters.get(ii + PARAMETER_TUPLE_TYPE_OFFSET)).equals(type)) - { - return parameters.get(ii + PARAMETER_TUPLE_PARAMETER_OFFSET); - } - } - return null; - } - - private static class MessageOutSizes - { - public final long messageSize; - public final long payloadSize; - - private MessageOutSizes(long messageSize, long payloadSize) - { - this.messageSize = messageSize; - this.payloadSize = payloadSize; - } - - @Override - public final int hashCode() - { - int hashCode = (int) messageSize ^ (int) (messageSize >>> 32); - return 31 * (hashCode ^ (int) ((int) payloadSize ^ (payloadSize >>> 32))); - } - - @Override - public final boolean equals(Object o) - { - if (!(o instanceof MessageOutSizes)) - return false; - MessageOutSizes that = (MessageOutSizes) o; - return messageSize == that.messageSize && payloadSize == that.payloadSize; - } - } -} diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java index f72cd619d855..f5aadc6dbc4b 100644 --- a/src/java/org/apache/cassandra/net/MessagingService.java +++ b/src/java/org/apache/cassandra/net/MessagingService.java @@ -1,4 +1,4 @@ - /* +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -17,109 +17,187 @@ */ package org.apache.cassandra.net; -import java.io.IOError; -import java.io.IOException; -import java.net.UnknownHostException; +import java.nio.channels.ClosedChannelException; import java.util.ArrayList; -import java.util.Collection; -import java.util.EnumMap; -import java.util.EnumSet; -import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Optional; import java.util.Set; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.TimeoutException; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.Lists; -import org.cliffc.high_scale_lib.NonBlockingHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.carrotsearch.hppc.IntObjectMap; -import com.carrotsearch.hppc.IntObjectOpenHashMap; -import io.netty.channel.Channel; -import io.netty.channel.group.ChannelGroup; -import io.netty.channel.group.DefaultChannelGroup; -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.batchlog.Batch; -import org.apache.cassandra.concurrent.ExecutorLocals; -import org.apache.cassandra.concurrent.LocalAwareExecutorService; +import io.netty.util.concurrent.Future; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.concurrent.Stage; import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.ConsistencyLevel; -import org.apache.cassandra.db.CounterMutation; -import org.apache.cassandra.db.IMutation; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.db.ReadCommand; -import org.apache.cassandra.db.ReadResponse; -import org.apache.cassandra.db.SnapshotCommand; import org.apache.cassandra.db.SystemKeyspace; -import org.apache.cassandra.db.TruncateResponse; -import org.apache.cassandra.db.Truncation; -import org.apache.cassandra.db.WriteResponse; -import org.apache.cassandra.dht.AbstractBounds; -import org.apache.cassandra.dht.BootStrapper; -import org.apache.cassandra.dht.IPartitioner; -import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.RequestFailureReason; -import org.apache.cassandra.gms.EchoMessage; -import org.apache.cassandra.gms.GossipDigestAck; -import org.apache.cassandra.gms.GossipDigestAck2; -import org.apache.cassandra.gms.GossipDigestSyn; -import org.apache.cassandra.hints.HintMessage; -import org.apache.cassandra.hints.HintResponse; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.IEndpointSnitch; -import org.apache.cassandra.locator.ILatencySubscriber; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.metrics.CassandraMetricsRegistry; -import org.apache.cassandra.metrics.ConnectionMetrics; -import org.apache.cassandra.metrics.DroppedMessageMetrics; -import org.apache.cassandra.metrics.MessagingMetrics; -import org.apache.cassandra.net.async.OutboundMessagingPool; -import org.apache.cassandra.net.async.NettyFactory; -import org.apache.cassandra.net.async.NettyFactory.InboundInitializer; -import org.apache.cassandra.repair.messages.RepairMessage; -import org.apache.cassandra.schema.MigrationManager; -import org.apache.cassandra.schema.TableId; -import org.apache.cassandra.security.SSLFactory; import org.apache.cassandra.service.AbstractWriteResponseHandler; -import org.apache.cassandra.service.StorageProxy; -import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.service.paxos.Commit; -import org.apache.cassandra.service.paxos.PrepareResponse; -import org.apache.cassandra.tracing.TraceState; -import org.apache.cassandra.tracing.Tracing; -import org.apache.cassandra.utils.BooleanSerializer; -import org.apache.cassandra.utils.ExpiringMap; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.MBeanWrapper; -import org.apache.cassandra.utils.NativeLibrary; -import org.apache.cassandra.utils.Pair; -import org.apache.cassandra.utils.StatusLogger; -import org.apache.cassandra.utils.UUIDSerializer; -import org.apache.cassandra.utils.concurrent.SimpleCondition; -public final class MessagingService implements MessagingServiceMBean +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.concurrent.Stage.MUTATION; +import static org.apache.cassandra.utils.Throwables.maybeFail; + +/** + * MessagingService implements all internode communication - with the exception of SSTable streaming (for now). + * + * Specifically, it's responsible for dispatch of outbound messages to other nodes and routing of inbound messages + * to their appropriate {@link IVerbHandler}. + * + *

Using MessagingService: sending requests and responses

+ * + * The are two ways to send a {@link Message}, and you should pick one depending on the desired behaviour: + * 1. To send a request that expects a response back, use + * {@link #sendWithCallback(Message, InetAddressAndPort, RequestCallback)} method. Once a response + * message is received, {@link RequestCallback#onResponse(Message)} method will be invoked on the + * provided callback - in case of a success response. In case of a failure response (see {@link Verb#FAILURE_RSP}), + * or if a response doesn't arrive within verb's configured expiry time, + * {@link RequestCallback#onFailure(InetAddressAndPort, RequestFailureReason)} will be invoked instead. + * 2. To send a response back, or a message that expects no response, use {@link #send(Message, InetAddressAndPort)} + * method. + * + * See also: {@link Message#out(Verb, Object)}, {@link Message#responseWith(Object)}, + * and {@link Message#failureResponse(RequestFailureReason)}. + * + *

Using MessagingService: handling a request

+ * + * As described in the previous section, to handle responses you only need to implement {@link RequestCallback} + * interface - so long as your response verb handler is the default {@link ResponseVerbHandler}. + * + * There are two steps you need to perform to implement request handling: + * 1. Create a {@link IVerbHandler} to process incoming requests and responses for the new type (if applicable). + * 2. Add a new {@link Verb} to the enum for the new request type, and, if applicable, one for the response message. + * + * MessagingService will now automatically invoke your handler whenever a {@link Message} with this verb arrives. + * + *

Architecture of MessagingService

+ * + *

QOS

+ * + * Since our messaging protocol is TCP-based, and also doesn't yet support interleaving messages with each other, + * we need a way to prevent head-of-line blocking adversely affecting all messages - in particular, large messages + * being in the way of smaller ones. To achive that (somewhat), we maintain three messaging connections to and + * from each peer: + * - one for large messages - defined as being larger than {@link OutboundConnections#LARGE_MESSAGE_THRESHOLD} + * (65KiB by default) + * - one for small messages - defined as smaller than that threshold + * - and finally, a connection for urgent messages - usually small and/or that are important to arrive + * promptly, e.g. gossip-related ones + * + *

Wire format and framing

+ * + * Small messages are grouped together into frames, and large messages are split over multiple frames. + * Framing provides application-level integrity protection to otherwise raw streams of data - we use + * CRC24 for frame headers and CRC32 for the entire payload. LZ4 is optionally used for compression. + * + * You can find the on-wire format description of individual messages in the comments for + * {@link Message.Serializer}, alongside with format evolution notes. + * For the list and descriptions of available frame decoders see {@link FrameDecoder} comments. You can + * find wire format documented in the javadoc of {@link FrameDecoder} implementations: + * see {@link FrameDecoderCrc} and {@link FrameDecoderLZ4} in particular. + * + *

Architecture of outbound messaging

+ * + * {@link OutboundConnection} is the core class implementing outbound connection logic, with + * {@link OutboundConnection#enqueue(Message)} being its main entry point. The connections are initiated + * by {@link OutboundConnectionInitiator}. + * + * Netty pipeline for outbound messaging connections generally consists of the following handlers: + * + * [(optional) SslHandler] <- [FrameEncoder] + * + * {@link OutboundConnection} handles the entire lifetime of a connection: from the very first handshake + * to any necessary reconnects if necessary. + * + * Message-delivery flow varies depending on the connection type. + * + * For {@link ConnectionType#SMALL_MESSAGES} and {@link ConnectionType#URGENT_MESSAGES}, + * {@link Message} serialization and delivery occurs directly on the event loop. + * See {@link OutboundConnection.EventLoopDelivery} for details. + * + * For {@link ConnectionType#LARGE_MESSAGES}, to ensure that servicing large messages doesn't block + * timely service of other requests, message serialization is offloaded to a companion thread pool + * ({@link SocketFactory#synchronousWorkExecutor}). Most of the work will be performed by + * {@link AsyncChannelOutputPlus}. Please see {@link OutboundConnection.LargeMessageDelivery} + * for details. + * + * To prevent fast clients, or slow nodes on the other end of the connection from overwhelming + * a host with enqueued, unsent messages on heap, we impose strict limits on how much memory enqueued, + * undelivered messages can claim. + * + * Every individual connection gets an exclusive permit quota to use - 4MiB by default; every endpoint + * (group of large, small, and urgent connection) is capped at, by default, at 128MiB of undelivered messages, + * and a global limit of 512MiB is imposed on all endpoints combined. + * + * On an attempt to {@link OutboundConnection#enqueue(Message)}, the connection will attempt to allocate + * permits for message-size number of bytes from its exclusive quota; if successful, it will add the + * message to the queue; if unsuccessful, it will need to allocate remainder from both endpoint and lobal + * reserves, and if it fails to do so, the message will be rejected, and its callbacks, if any, + * immediately expired. + * + * For a more detailed description please see the docs and comments of {@link OutboundConnection}. + * + *

Architecture of inbound messaging

+ * + * {@link InboundMessageHandler} is the core class implementing inbound connection logic, paired + * with {@link FrameDecoder}. Inbound connections are initiated by {@link InboundConnectionInitiator}. + * The primary entry points to these classes are {@link FrameDecoder#channelRead(ShareableBytes)} + * and {@link InboundMessageHandler#process(FrameDecoder.Frame)}. + * + * Netty pipeline for inbound messaging connections generally consists of the following handlers: + * + * [(optional) SslHandler] -> [FrameDecoder] -> [InboundMessageHandler] + * + * {@link FrameDecoder} is responsible for decoding incoming frames and work stashing; {@link InboundMessageHandler} + * then takes decoded frames from the decoder and processes the messages contained in them. + * + * The flow differs between small and large messages. Small ones are deserialized immediately, and only + * then scheduled on the right thread pool for the {@link Verb} for execution. Large messages, OTOH, + * aren't deserialized until they are just about to be executed on the appropriate {@link Stage}. + * + * Similarly to outbound handling, inbound messaging imposes strict memory utilisation limits on individual + * endpoints and on global aggregate consumption, and implements simple flow control, to prevent a single + * fast endpoint from overwhelming a host. + * + * Every individual connection gets an exclusive permit quota to use - 4MiB by default; every endpoint + * (group of large, small, and urgent connection) is capped at, by default, at 128MiB of unprocessed messages, + * and a global limit of 512MiB is imposed on all endpoints combined. + * + * On arrival of a message header, the handler will attempt to allocate permits for message-size number + * of bytes from its exclusive quota; if successful, it will proceed to deserializing and processing the message. + * If unsuccessful, the handler will attempt to allocate the remainder from its endpoint and global reserve; + * if either allocation is unsuccessful, the handler will cease any further frame processing, and tell + * {@link FrameDecoder} to stop reading from the network; subsequently, it will put itself on a special + * {@link org.apache.cassandra.net.InboundMessageHandler.WaitQueue}, to be reactivated once more permits + * become available. + * + * For a more detailed description please see the docs and comments of {@link InboundMessageHandler} and + * {@link FrameDecoder}. + * + *

Observability

+ * + * MessagingService exposes diagnostic counters for both outbound and inbound directions - received and sent + * bytes and message counts, overload bytes and message count, error bytes and error counts, and many more. + * + * See {@link org.apache.cassandra.metrics.InternodeInboundMetrics} and + * {@link org.apache.cassandra.metrics.InternodeOutboundMetrics} for JMX-exposed counters. + * + * We also provide {@code system_views.internode_inbound} and {@code system_views.internode_outbound} virtual tables - + * implemented in {@link org.apache.cassandra.db.virtual.InternodeInboundTable} and + * {@link org.apache.cassandra.db.virtual.InternodeOutboundTable} respectively. + */ +public final class MessagingService extends MessagingServiceMBeanImpl { - public static final String MBEAN_NAME = "org.apache.cassandra.net:type=MessagingService"; + private static final Logger logger = LoggerFactory.getLogger(MessagingService.class); // 8 bits version, so don't waste versions public static final int VERSION_30 = 10; @@ -127,529 +205,142 @@ public final class MessagingService implements MessagingServiceMBean public static final int VERSION_40 = 12; public static final int minimum_version = VERSION_30; public static final int current_version = VERSION_40; + static AcceptVersions accept_messaging = new AcceptVersions(minimum_version, current_version); + static AcceptVersions accept_streaming = new AcceptVersions(current_version, current_version); - public static final byte[] ONE_BYTE = new byte[1]; - - /** - * we preface every message with this number so the recipient can validate the sender is sane - */ - public static final int PROTOCOL_MAGIC = 0xCA552DFA; - - public final MessagingMetrics metrics = new MessagingMetrics(); - - /* All verb handler identifiers */ - public enum Verb + private static class MSHandle { - MUTATION - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, - HINT - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, - READ_REPAIR - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, - READ - { - public long getTimeout() - { - return DatabaseDescriptor.getReadRpcTimeout(); - } - }, - REQUEST_RESPONSE, // client-initiated reads and writes - BATCH_STORE - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, // was @Deprecated STREAM_INITIATE, - BATCH_REMOVE - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, // was @Deprecated STREAM_INITIATE_DONE, - @Deprecated STREAM_REPLY, - @Deprecated STREAM_REQUEST, - RANGE_SLICE - { - public long getTimeout() - { - return DatabaseDescriptor.getRangeRpcTimeout(); - } - }, - @Deprecated BOOTSTRAP_TOKEN, - @Deprecated TREE_REQUEST, - @Deprecated TREE_RESPONSE, - @Deprecated JOIN, - GOSSIP_DIGEST_SYN, - GOSSIP_DIGEST_ACK, - GOSSIP_DIGEST_ACK2, - @Deprecated DEFINITIONS_ANNOUNCE, - DEFINITIONS_UPDATE, - TRUNCATE - { - public long getTimeout() - { - return DatabaseDescriptor.getTruncateRpcTimeout(); - } - }, - SCHEMA_CHECK, - @Deprecated INDEX_SCAN, - REPLICATION_FINISHED, - INTERNAL_RESPONSE, // responses to internal calls - COUNTER_MUTATION - { - public long getTimeout() - { - return DatabaseDescriptor.getCounterWriteRpcTimeout(); - } - }, - @Deprecated STREAMING_REPAIR_REQUEST, - @Deprecated STREAMING_REPAIR_RESPONSE, - SNAPSHOT, // Similar to nt snapshot - MIGRATION_REQUEST, - GOSSIP_SHUTDOWN, - _TRACE, // dummy verb so we can use MS.droppedMessagesMap - ECHO, - REPAIR_MESSAGE, - PAXOS_PREPARE - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, - PAXOS_PROPOSE - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, - PAXOS_COMMIT - { - public long getTimeout() - { - return DatabaseDescriptor.getWriteRpcTimeout(); - } - }, - @Deprecated PAGED_RANGE - { - public long getTimeout() - { - return DatabaseDescriptor.getRangeRpcTimeout(); - } - }, - PING - { - public long getTimeout() - { - return DatabaseDescriptor.getPingTimeout(); - } - }, - - // UNUSED verbs were used as padding for backward/forward compatability before 4.0, - // but it wasn't quite as bullet/future proof as needed. We still need to keep these entries - // around, at least for a major rev or two (post-4.0). see CASSANDRA-13993 for a discussion. - // For now, though, the UNUSED are legacy values (placeholders, basically) that should only be used - // for correctly adding VERBs that need to be emergency additions to 3.0/3.11. - // We can reclaim them (their id's, to be correct) in future versions, if desireed, though. - UNUSED_2, - UNUSED_3, - UNUSED_4, - UNUSED_5, - _SAMPLE // dummy verb so we can use MS.droppedMessagesMap - ; - // add new verbs after the existing verbs, since we serialize by ordinal. - - private final int id; - Verb() - { - id = ordinal(); - } - - /** - * Unused, but it is an extension point for adding custom verbs - * @param id - */ - Verb(int id) - { - this.id = id; - } - - public long getTimeout() - { - return DatabaseDescriptor.getRpcTimeout(); - } - - public int getId() - { - return id; - } - private static final IntObjectMap idToVerbMap = new IntObjectOpenHashMap<>(values().length); - static - { - for (Verb v : values()) - { - Verb existing = idToVerbMap.put(v.getId(), v); - if (existing != null) - throw new IllegalArgumentException("cannot have two verbs that map to the same id: " + v + " and " + existing); - } - } - - public static Verb fromId(int id) - { - return idToVerbMap.get(id); - } + public static final MessagingService instance = new MessagingService(false); } - public static final EnumMap verbStages = new EnumMap(MessagingService.Verb.class) - {{ - put(Verb.MUTATION, Stage.MUTATION); - put(Verb.COUNTER_MUTATION, Stage.COUNTER_MUTATION); - put(Verb.READ_REPAIR, Stage.MUTATION); - put(Verb.HINT, Stage.MUTATION); - put(Verb.TRUNCATE, Stage.MUTATION); - put(Verb.PAXOS_PREPARE, Stage.MUTATION); - put(Verb.PAXOS_PROPOSE, Stage.MUTATION); - put(Verb.PAXOS_COMMIT, Stage.MUTATION); - put(Verb.BATCH_STORE, Stage.MUTATION); - put(Verb.BATCH_REMOVE, Stage.MUTATION); - - put(Verb.READ, Stage.READ); - put(Verb.RANGE_SLICE, Stage.READ); - put(Verb.INDEX_SCAN, Stage.READ); - put(Verb.PAGED_RANGE, Stage.READ); - - put(Verb.REQUEST_RESPONSE, Stage.REQUEST_RESPONSE); - put(Verb.INTERNAL_RESPONSE, Stage.INTERNAL_RESPONSE); - - put(Verb.STREAM_REPLY, Stage.MISC); // actually handled by FileStreamTask and streamExecutors - put(Verb.STREAM_REQUEST, Stage.MISC); - put(Verb.REPLICATION_FINISHED, Stage.MISC); - put(Verb.SNAPSHOT, Stage.MISC); - - put(Verb.TREE_REQUEST, Stage.ANTI_ENTROPY); - put(Verb.TREE_RESPONSE, Stage.ANTI_ENTROPY); - put(Verb.STREAMING_REPAIR_REQUEST, Stage.ANTI_ENTROPY); - put(Verb.STREAMING_REPAIR_RESPONSE, Stage.ANTI_ENTROPY); - put(Verb.REPAIR_MESSAGE, Stage.ANTI_ENTROPY); - put(Verb.GOSSIP_DIGEST_ACK, Stage.GOSSIP); - put(Verb.GOSSIP_DIGEST_ACK2, Stage.GOSSIP); - put(Verb.GOSSIP_DIGEST_SYN, Stage.GOSSIP); - put(Verb.GOSSIP_SHUTDOWN, Stage.GOSSIP); - - put(Verb.DEFINITIONS_UPDATE, Stage.MIGRATION); - put(Verb.SCHEMA_CHECK, Stage.MIGRATION); - put(Verb.MIGRATION_REQUEST, Stage.MIGRATION); - put(Verb.INDEX_SCAN, Stage.READ); - put(Verb.REPLICATION_FINISHED, Stage.MISC); - put(Verb.SNAPSHOT, Stage.MISC); - put(Verb.ECHO, Stage.GOSSIP); + public static MessagingService instance() + { + return MSHandle.instance; + } - put(Verb.UNUSED_2, Stage.INTERNAL_RESPONSE); - put(Verb.UNUSED_3, Stage.INTERNAL_RESPONSE); - put(Verb.UNUSED_4, Stage.INTERNAL_RESPONSE); - put(Verb.UNUSED_5, Stage.INTERNAL_RESPONSE); + public final SocketFactory socketFactory = new SocketFactory(); + public final LatencySubscribers latencySubscribers = new LatencySubscribers(); + public final RequestCallbacks callbacks = new RequestCallbacks(this); - put(Verb.PING, Stage.READ); - }}; + // a public hook for filtering messages intended for delivery to this node + public final InboundSink inboundSink = new InboundSink(this); - /** - * Messages we receive from peers have a Verb that tells us what kind of message it is. - * Most of the time, this is enough to determine how to deserialize the message payload. - * The exception is the REQUEST_RESPONSE verb, which just means "a reply to something you told me to do." - * Traditionally, this was fine since each VerbHandler knew what type of payload it expected, and - * handled the deserialization itself. Now that we do that in ITC, to avoid the extra copy to an - * intermediary byte[] (See CASSANDRA-3716), we need to wire that up to the CallbackInfo object - * (see below). - */ - public static final EnumMap> verbSerializers = new EnumMap>(Verb.class) - {{ - put(Verb.REQUEST_RESPONSE, CallbackDeterminedSerializer.instance); - put(Verb.INTERNAL_RESPONSE, CallbackDeterminedSerializer.instance); + // the inbound global reserve limits and associated wait queue + private final InboundMessageHandlers.GlobalResourceLimits inboundGlobalReserveLimits = new InboundMessageHandlers.GlobalResourceLimits( + new ResourceLimits.Concurrent(DatabaseDescriptor.getInternodeApplicationReceiveQueueReserveGlobalCapacityInBytes())); - put(Verb.MUTATION, Mutation.serializer); - put(Verb.READ_REPAIR, Mutation.serializer); - put(Verb.READ, ReadCommand.serializer); - put(Verb.RANGE_SLICE, ReadCommand.serializer); - put(Verb.PAGED_RANGE, ReadCommand.serializer); - put(Verb.BOOTSTRAP_TOKEN, BootStrapper.StringSerializer.instance); - put(Verb.REPAIR_MESSAGE, RepairMessage.serializer); - put(Verb.GOSSIP_DIGEST_ACK, GossipDigestAck.serializer); - put(Verb.GOSSIP_DIGEST_ACK2, GossipDigestAck2.serializer); - put(Verb.GOSSIP_DIGEST_SYN, GossipDigestSyn.serializer); - put(Verb.DEFINITIONS_UPDATE, MigrationManager.MigrationsSerializer.instance); - put(Verb.TRUNCATE, Truncation.serializer); - put(Verb.REPLICATION_FINISHED, null); - put(Verb.COUNTER_MUTATION, CounterMutation.serializer); - put(Verb.SNAPSHOT, SnapshotCommand.serializer); - put(Verb.ECHO, EchoMessage.serializer); - put(Verb.PAXOS_PREPARE, Commit.serializer); - put(Verb.PAXOS_PROPOSE, Commit.serializer); - put(Verb.PAXOS_COMMIT, Commit.serializer); - put(Verb.HINT, HintMessage.serializer); - put(Verb.BATCH_STORE, Batch.serializer); - put(Verb.BATCH_REMOVE, UUIDSerializer.serializer); - put(Verb.PING, PingMessage.serializer); - }}; + // the socket bindings we accept incoming connections on + private final InboundSockets inboundSockets = new InboundSockets(new InboundConnectionSettings() + .withHandlers(this::getInbound) + .withSocketFactory(socketFactory)); - /** - * A Map of what kind of serializer to wire up to a REQUEST_RESPONSE callback, based on outbound Verb. - */ - public static final EnumMap> callbackDeserializers = new EnumMap>(Verb.class) - {{ - put(Verb.MUTATION, WriteResponse.serializer); - put(Verb.HINT, HintResponse.serializer); - put(Verb.READ_REPAIR, WriteResponse.serializer); - put(Verb.COUNTER_MUTATION, WriteResponse.serializer); - put(Verb.RANGE_SLICE, ReadResponse.serializer); - put(Verb.PAGED_RANGE, ReadResponse.serializer); - put(Verb.READ, ReadResponse.serializer); - put(Verb.TRUNCATE, TruncateResponse.serializer); - put(Verb.SNAPSHOT, null); + // a public hook for filtering messages intended for delivery to another node + public final OutboundSink outboundSink = new OutboundSink(this::doSend); - put(Verb.MIGRATION_REQUEST, MigrationManager.MigrationsSerializer.instance); - put(Verb.SCHEMA_CHECK, UUIDSerializer.serializer); - put(Verb.BOOTSTRAP_TOKEN, BootStrapper.StringSerializer.instance); - put(Verb.REPLICATION_FINISHED, null); + final ResourceLimits.Limit outboundGlobalReserveLimit = + new ResourceLimits.Concurrent(DatabaseDescriptor.getInternodeApplicationSendQueueReserveGlobalCapacityInBytes()); - put(Verb.PAXOS_PREPARE, PrepareResponse.serializer); - put(Verb.PAXOS_PROPOSE, BooleanSerializer.serializer); + // back-pressure implementation + private final BackPressureStrategy backPressure = DatabaseDescriptor.getBackPressureStrategy(); - put(Verb.BATCH_STORE, WriteResponse.serializer); - put(Verb.BATCH_REMOVE, WriteResponse.serializer); - }}; + private volatile boolean isShuttingDown; - /* This records all the results mapped by message Id */ - private final ExpiringMap callbacks; + @VisibleForTesting + MessagingService(boolean testOnly) + { + super(testOnly); + OutboundConnections.scheduleUnusedConnectionMonitoring(this, ScheduledExecutors.scheduledTasks, 1L, TimeUnit.HOURS); + } /** - * a placeholder class that means "deserialize using the callback." We can't implement this without - * special-case code in InboundTcpConnection because there is no way to pass the message id to IVersionedSerializer. + * Send a non-mutation message to a given endpoint. This method specifies a callback + * which is invoked with the actual response. + * + * @param message message to be sent. + * @param to endpoint to which the message needs to be sent + * @param cb callback interface which is used to pass the responses or + * suggest that a timeout occurred to the invoker of the send(). */ - static class CallbackDeterminedSerializer implements IVersionedSerializer + public void sendWithCallback(Message message, InetAddressAndPort to, RequestCallback cb) { - public static final CallbackDeterminedSerializer instance = new CallbackDeterminedSerializer(); - - public Object deserialize(DataInputPlus in, int version) throws IOException - { - throw new UnsupportedOperationException(); - } - - public void serialize(Object o, DataOutputPlus out, int version) throws IOException - { - throw new UnsupportedOperationException(); - } - - public long serializedSize(Object o, int version) - { - throw new UnsupportedOperationException(); - } + sendWithCallback(message, to, cb, null); } - public static IVersionedSerializer getVerbSerializer(Verb verb, int id) + public void sendWithCallback(Message message, InetAddressAndPort to, RequestCallback cb, ConnectionType specifyConnection) { - IVersionedSerializer serializer = verbSerializers.get(verb); - if (serializer instanceof MessagingService.CallbackDeterminedSerializer) - { - CallbackInfo callback = MessagingService.instance().getRegisteredCallback(id); - if (callback == null) - return null; - - serializer = callback.serializer; - } - return serializer; + callbacks.addWithExpiration(cb, message, to); + updateBackPressureOnSend(to, cb, message); + if (cb.invokeOnFailure() && !message.callBackOnFailure()) + message = message.withCallBackOnFailure(); + send(message, to, specifyConnection); } - /* Lookup table for registering message handlers based on the verb. */ - private final Map verbHandlers; - - @VisibleForTesting - public final ConcurrentMap channelManagers = new NonBlockingHashMap<>(); - final List serverChannels = Lists.newArrayList(); - - private static final Logger logger = LoggerFactory.getLogger(MessagingService.class); - private static final int LOG_DROPPED_INTERVAL_IN_MS = 5000; - - private final SimpleCondition listenGate; - /** - * Verbs it's okay to drop if the request has been queued longer than the request timeout. These - * all correspond to client requests or something triggered by them; we don't want to - * drop internal messages like bootstrap or repair notifications. + * Send a mutation message or a Paxos Commit to a given endpoint. This method specifies a callback + * which is invoked with the actual response. + * Also holds the message (only mutation messages) to determine if it + * needs to trigger a hint (uses StorageProxy for that). + * + * @param message message to be sent. + * @param to endpoint to which the message needs to be sent + * @param handler callback interface which is used to pass the responses or + * suggest that a timeout occurred to the invoker of the send(). */ - public static final EnumSet DROPPABLE_VERBS = EnumSet.of(Verb._TRACE, - Verb._SAMPLE, - Verb.MUTATION, - Verb.COUNTER_MUTATION, - Verb.HINT, - Verb.READ_REPAIR, - Verb.READ, - Verb.RANGE_SLICE, - Verb.PAGED_RANGE, - Verb.REQUEST_RESPONSE, - Verb.BATCH_STORE, - Verb.BATCH_REMOVE); - - private static final class DroppedMessages - { - final DroppedMessageMetrics metrics; - final AtomicInteger droppedInternal; - final AtomicInteger droppedCrossNode; - - DroppedMessages(Verb verb) - { - this(new DroppedMessageMetrics(verb)); - } - - DroppedMessages(DroppedMessageMetrics metrics) - { - this.metrics = metrics; - this.droppedInternal = new AtomicInteger(0); - this.droppedCrossNode = new AtomicInteger(0); - } - } - - @VisibleForTesting - public void resetDroppedMessagesMap(String scope) + public void sendWriteWithCallback(Message message, Replica to, AbstractWriteResponseHandler handler, boolean allowHints) { - for (Verb verb : droppedMessagesMap.keySet()) - droppedMessagesMap.put(verb, new DroppedMessages(new DroppedMessageMetrics(metricName -> { - return new CassandraMetricsRegistry.MetricName("DroppedMessages", metricName, scope); - }))); + assert message.callBackOnFailure(); + callbacks.addWithExpiration(handler, message, to, handler.consistencyLevel(), allowHints); + updateBackPressureOnSend(to.endpoint(), handler, message); + send(message, to.endpoint(), null); } - // total dropped message counts for server lifetime - private final Map droppedMessagesMap = new EnumMap<>(Verb.class); - - private final List subscribers = new ArrayList(); - - // protocol versions of the other nodes in the cluster - private final ConcurrentMap versions = new NonBlockingHashMap<>(); - - // message sinks are a testing hook - private final Set messageSinks = new CopyOnWriteArraySet<>(); - - // back-pressure implementation - private final BackPressureStrategy backPressure = DatabaseDescriptor.getBackPressureStrategy(); - - private static class MSHandle + /** + * Send a message to a given endpoint. This method adheres to the fire and forget + * style messaging. + * + * @param message messages to be sent. + * @param to endpoint to which the message needs to be sent + */ + public void send(Message message, InetAddressAndPort to) { - public static final MessagingService instance = new MessagingService(false); + send(message, to, null); } - public static MessagingService instance() + public void send(Message message, InetAddressAndPort to, ConnectionType specifyConnection) { - return MSHandle.instance; - } + if (logger.isTraceEnabled()) + { + logger.trace("{} sending {} to {}@{}", FBUtilities.getBroadcastAddressAndPort(), message.verb(), message.id(), to); - private static class MSTestHandle - { - public static final MessagingService instance = new MessagingService(true); - } + if (to.equals(FBUtilities.getBroadcastAddressAndPort())) + logger.trace("Message-to-self {} going over MessagingService", message); + } - static MessagingService test() - { - return MSTestHandle.instance; + outboundSink.accept(message, to, specifyConnection); } - private MessagingService(boolean testOnly) + private void doSend(Message message, InetAddressAndPort to, ConnectionType specifyConnection) { - for (Verb verb : DROPPABLE_VERBS) - droppedMessagesMap.put(verb, new DroppedMessages(verb)); - - listenGate = new SimpleCondition(); - verbHandlers = new EnumMap<>(Verb.class); - if (!testOnly) + // expire the callback if the message failed to enqueue (failed to establish a connection or exceeded queue capacity) + while (true) { - Runnable logDropped = new Runnable() + OutboundConnections connections = getOutbound(to); + try { - public void run() - { - logDroppedMessages(); - } - }; - ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(logDropped, LOG_DROPPED_INTERVAL_IN_MS, LOG_DROPPED_INTERVAL_IN_MS, TimeUnit.MILLISECONDS); - } - - Function>, ?> timeoutReporter = new Function>, Object>() - { - public Object apply(Pair> pair) + connections.enqueue(message, specifyConnection); + return; + } + catch (ClosedChannelException e) { - final CallbackInfo expiredCallbackInfo = pair.right.value; + if (isShuttingDown) + return; // just drop the message, and let others clean up - maybeAddLatency(expiredCallbackInfo.callback, expiredCallbackInfo.target, pair.right.timeout); - - ConnectionMetrics.totalTimeouts.mark(); - markTimeout(expiredCallbackInfo.target); - - if (expiredCallbackInfo.callback.supportsBackPressure()) - { - updateBackPressureOnReceive(expiredCallbackInfo.target, expiredCallbackInfo.callback, true); - } - - if (expiredCallbackInfo.isFailureCallback()) - { - StageManager.getStage(Stage.INTERNAL_RESPONSE).submit(new Runnable() - { - @Override - public void run() - { - ((IAsyncCallbackWithFailure)expiredCallbackInfo.callback).onFailure(expiredCallbackInfo.target, RequestFailureReason.UNKNOWN); - } - }); - } - - if (expiredCallbackInfo.shouldHint()) - { - WriteCallbackInfo writeCallbackInfo = ((WriteCallbackInfo) expiredCallbackInfo); - Mutation mutation = writeCallbackInfo.mutation(); - return StorageProxy.submitHint(mutation, writeCallbackInfo.getReplica(), null); - } - - return null; + // remove the connection and try again + channelManagers.remove(to, connections); } - }; - - callbacks = new ExpiringMap<>(DatabaseDescriptor.getMinRpcTimeout(), timeoutReporter); - - if (!testOnly) - { - MBeanWrapper.instance.registerMBean(this, MBEAN_NAME); } } - public void addMessageSink(IMessageSink sink) - { - messageSinks.add(sink); - } - - public void removeMessageSink(IMessageSink sink) - { - messageSinks.remove(sink); - } - - public void clearMessageSinks() - { - messageSinks.clear(); - } - /** * Updates the back-pressure state on sending to the given host if enabled and the given message callback supports it. * @@ -657,7 +348,7 @@ public void clearMessageSinks() * @param callback The message callback. * @param message The actual message. */ - public void updateBackPressureOnSend(InetAddressAndPort host, IAsyncCallback callback, MessageOut message) + void updateBackPressureOnSend(InetAddressAndPort host, RequestCallback callback, Message message) { if (DatabaseDescriptor.backPressureEnabled() && callback.supportsBackPressure()) { @@ -674,7 +365,7 @@ public void updateBackPressureOnSend(InetAddressAndPort host, IAsyncCallback cal * @param callback The message callback. * @param timeout True if updated following a timeout, false otherwise. */ - public void updateBackPressureOnReceive(InetAddressAndPort host, IAsyncCallback callback, boolean timeout) + void updateBackPressureOnReceive(InetAddressAndPort host, RequestCallback callback, boolean timeout) { if (DatabaseDescriptor.backPressureEnabled() && callback.supportsBackPressure()) { @@ -701,1027 +392,193 @@ public void applyBackPressure(Iterable hosts, long timeoutIn { if (DatabaseDescriptor.backPressureEnabled()) { - Set states = new HashSet(); + Set states = new HashSet<>(); for (InetAddressAndPort host : hosts) { if (host.equals(FBUtilities.getBroadcastAddressAndPort())) continue; - OutboundMessagingPool pool = getMessagingConnection(host); - if (pool != null) - states.add(pool.getBackPressureState()); + states.add(getOutbound(host).getBackPressureState()); } - backPressure.apply(states, timeoutInNanos, TimeUnit.NANOSECONDS); + //noinspection unchecked + backPressure.apply(states, timeoutInNanos, NANOSECONDS); } } BackPressureState getBackPressureState(InetAddressAndPort host) { - OutboundMessagingPool messagingConnection = getMessagingConnection(host); - return messagingConnection != null ? messagingConnection.getBackPressureState() : null; + return getOutbound(host).getBackPressureState(); } - void markTimeout(InetAddressAndPort addr) + void markExpiredCallback(InetAddressAndPort addr) { - OutboundMessagingPool conn = channelManagers.get(addr); + OutboundConnections conn = channelManagers.get(addr); if (conn != null) - conn.incrementTimeout(); + conn.incrementExpiredCallbackCount(); } /** - * Track latency information for the dynamic snitch + * Only to be invoked once we believe the endpoint will never be contacted again. * - * @param cb the callback associated with this message -- this lets us know if it's a message type we're interested in - * @param address the host that replied to the message - * @param latency + * We close the connection after a five minute delay, to give asynchronous operations a chance to terminate */ - public void maybeAddLatency(IAsyncCallback cb, InetAddressAndPort address, long latency) + public void closeOutbound(InetAddressAndPort to) { - if (cb.isLatencyForSnitch()) - addLatency(address, latency); + OutboundConnections pool = channelManagers.get(to); + if (pool != null) + pool.scheduleClose(5L, MINUTES, true) + .addListener(future -> channelManagers.remove(to, pool)); } - public void addLatency(InetAddressAndPort address, long latency) + /** + * Only to be invoked once we believe the connections will never be used again. + */ + void closeOutboundNow(OutboundConnections connections) { - for (ILatencySubscriber subscriber : subscribers) - subscriber.receiveTiming(address, latency); + connections.close(true).addListener( + future -> channelManagers.remove(connections.template().to, connections) + ); } /** - * called from gossiper when it notices a node is not responding. + * Only to be invoked once we believe the connections will never be used again. */ - public void convict(InetAddressAndPort ep) + public void removeInbound(InetAddressAndPort from) { - logger.trace("Resetting pool for {}", ep); - reset(ep); + InboundMessageHandlers handlers = messageHandlers.remove(from); + if (null != handlers) + handlers.releaseMetrics(); } - public void listen() + /** + * Closes any current open channel/connection to the endpoint, but does not cause any message loss, and we will + * try to re-establish connections immediately + */ + public void interruptOutbound(InetAddressAndPort to) { - listen(DatabaseDescriptor.getInternodeMessagingEncyptionOptions()); + OutboundConnections pool = channelManagers.get(to); + if (pool != null) + pool.interrupt(); } - public void listen(ServerEncryptionOptions serverEncryptionOptions) + /** + * Reconnect to the peer using the given {@code addr}. Outstanding messages in each channel will be sent on the + * current channel. Typically this function is used for something like EC2 public IP addresses which need to be used + * for communication between EC2 regions. + * + * @param address IP Address to identify the peer + * @param preferredAddress IP Address to use (and prefer) going forward for connecting to the peer + */ + @SuppressWarnings("UnusedReturnValue") + public Future maybeReconnectWithNewIp(InetAddressAndPort address, InetAddressAndPort preferredAddress) { - callbacks.reset(); // hack to allow tests to stop/restart MS - listen(FBUtilities.getLocalAddressAndPort(), serverEncryptionOptions); - if (shouldListenOnBroadcastAddress()) - listen(FBUtilities.getBroadcastAddressAndPort(), serverEncryptionOptions); - listenGate.signalAll(); - } + if (!SystemKeyspace.updatePreferredIP(address, preferredAddress)) + return null; - public static boolean shouldListenOnBroadcastAddress() - { - return DatabaseDescriptor.shouldListenOnBroadcastAddress() - && !FBUtilities.getLocalAddressAndPort().equals(FBUtilities.getBroadcastAddressAndPort()); + OutboundConnections messagingPool = channelManagers.get(address); + if (messagingPool != null) + return messagingPool.reconnectWithNewIp(preferredAddress); + + return null; } /** - * Listen on the specified port. - * - * @param localEp InetAddressAndPort whose port to listen on. + * Wait for callbacks and don't allow any more to be created (since they could require writing hints) */ - private void listen(InetAddressAndPort localEp, ServerEncryptionOptions serverEncryptionOptions) throws ConfigurationException + public void shutdown() { - IInternodeAuthenticator authenticator = DatabaseDescriptor.getInternodeAuthenticator(); - int receiveBufferSize = DatabaseDescriptor.getInternodeRecvBufferSize(); + shutdown(1L, MINUTES, true, true); + } - // this is the legacy socket, for letting peer nodes that haven't upgrade yet connect to this node. - // should only occur during cluster upgrade. we can remove this block at 5.0! - if (serverEncryptionOptions.enabled && serverEncryptionOptions.enable_legacy_ssl_storage_port) + public void shutdown(long timeout, TimeUnit units, boolean shutdownGracefully, boolean shutdownExecutors) + { + isShuttingDown = true; + logger.info("Waiting for messaging service to quiesce"); + // We may need to schedule hints on the mutation stage, so it's erroneous to shut down the mutation stage first + assert !StageManager.getStage(MUTATION).isShutdown(); + + if (shutdownGracefully) + { + callbacks.shutdownGracefully(); + List> closing = new ArrayList<>(); + for (OutboundConnections pool : channelManagers.values()) + closing.add(pool.close(true)); + + long deadline = System.nanoTime() + units.toNanos(timeout); + maybeFail(() -> new FutureCombiner(closing).get(timeout, units), + () -> inboundSockets.close().get(), + () -> { + if (shutdownExecutors) + shutdownExecutors(deadline); + }, + () -> callbacks.awaitTerminationUntil(deadline), + inboundSink::clear, + outboundSink::clear); + } + else { - // clone the encryption options, and explicitly set the optional field to false - // (do not allow non-TLS connections on the legacy ssl port) - ServerEncryptionOptions legacyEncOptions = new ServerEncryptionOptions(serverEncryptionOptions); - legacyEncOptions.optional = false; + callbacks.shutdownNow(false); + List> closing = new ArrayList<>(); + closing.add(inboundSockets.close()); + for (OutboundConnections pool : channelManagers.values()) + closing.add(pool.close(false)); - InetAddressAndPort localAddr = InetAddressAndPort.getByAddressOverrideDefaults(localEp.address, DatabaseDescriptor.getSSLStoragePort()); - ChannelGroup channelGroup = new DefaultChannelGroup("LegacyEncryptedInternodeMessagingGroup", NettyFactory.executorForChannelGroups()); - InboundInitializer initializer = new InboundInitializer(authenticator, legacyEncOptions, channelGroup); - Channel encryptedChannel = NettyFactory.instance.createInboundChannel(localAddr, initializer, receiveBufferSize); - serverChannels.add(new ServerChannel(encryptedChannel, channelGroup, localAddr, ServerChannel.SecurityLevel.REQUIRED)); + long deadline = System.nanoTime() + units.toNanos(timeout); + maybeFail(() -> new FutureCombiner(closing).get(timeout, units), + () -> { + if (shutdownExecutors) + shutdownExecutors(deadline); + }, + () -> callbacks.awaitTerminationUntil(deadline), + inboundSink::clear, + outboundSink::clear); } + } - // this is for the socket that can be plain, only ssl, or optional plain/ssl - assert localEp.port == DatabaseDescriptor.getStoragePort() : String.format("Local endpoint port %d doesn't match YAML configured port %d%n", localEp.port, DatabaseDescriptor.getStoragePort()); - InetAddressAndPort localAddr = InetAddressAndPort.getByAddressOverrideDefaults(localEp.address, DatabaseDescriptor.getStoragePort()); - ChannelGroup channelGroup = new DefaultChannelGroup("InternodeMessagingGroup", NettyFactory.executorForChannelGroups()); - InboundInitializer initializer = new InboundInitializer(authenticator, serverEncryptionOptions, channelGroup); - Channel channel = NettyFactory.instance.createInboundChannel(localAddr, initializer, receiveBufferSize); - ServerChannel.SecurityLevel securityLevel = !serverEncryptionOptions.enabled ? ServerChannel.SecurityLevel.NONE : - serverEncryptionOptions.optional ? ServerChannel.SecurityLevel.OPTIONAL : - ServerChannel.SecurityLevel.REQUIRED; - serverChannels.add(new ServerChannel(channel, channelGroup, localAddr, securityLevel)); + private void shutdownExecutors(long deadlineNanos) throws TimeoutException, InterruptedException + { + socketFactory.shutdownNow(); + socketFactory.awaitTerminationUntil(deadlineNanos); } - /** - * A simple struct to wrap up the the components needed for each listening socket. - *

- * The {@link #securityLevel} is captured independently of the {@link #channel} as there's no real way to inspect a s - * erver-side 'channel' to check if it using TLS or not (the channel's configured pipeline will only apply to - * connections that get created, so it's not inspectible). {@link #securityLevel} is really only used for testing, anyway. - */ - @VisibleForTesting - static class ServerChannel - { - /** - * Declares the type of TLS used with the channel. - */ - enum SecurityLevel { NONE, OPTIONAL, REQUIRED } - - /** - * The base {@link Channel} that is doing the spcket listen/accept. - */ - private final Channel channel; - - /** - * A group of the open, inbound {@link Channel}s connected to this node. This is mostly interesting so that all of - * the inbound connections/channels can be closed when the listening socket itself is being closed. - */ - private final ChannelGroup connectedChannels; - private final InetAddressAndPort address; - private final SecurityLevel securityLevel; - - private ServerChannel(Channel channel, ChannelGroup channelGroup, InetAddressAndPort address, SecurityLevel securityLevel) - { - this.channel = channel; - this.connectedChannels = channelGroup; - this.address = address; - this.securityLevel = securityLevel; - } - - void close() - { - if (channel.isOpen()) - channel.close().awaitUninterruptibly(); - connectedChannels.close().awaitUninterruptibly(); - } - - int size() - { - return connectedChannels.size(); - } - - /** - * For testing only! - */ - Channel getChannel() - { - return channel; - } - - InetAddressAndPort getAddress() - { - return address; - } - - SecurityLevel getSecurityLevel() - { - return securityLevel; - } - } - - public void waitUntilListening() - { - try - { - listenGate.await(); - } - catch (InterruptedException ie) - { - logger.trace("await interrupted"); - } - } - - public boolean isListening() - { - return listenGate.isSignaled(); - } - - - public void destroyConnectionPool(InetAddressAndPort to) - { - OutboundMessagingPool pool = channelManagers.remove(to); - if (pool != null) - pool.close(true); - } - - /** - * Reconnect to the peer using the given {@code addr}. Outstanding messages in each channel will be sent on the - * current channel. Typically this function is used for something like EC2 public IP addresses which need to be used - * for communication between EC2 regions. - * - * @param address IP Address to identify the peer - * @param preferredAddress IP Address to use (and prefer) going forward for connecting to the peer - */ - public void reconnectWithNewIp(InetAddressAndPort address, InetAddressAndPort preferredAddress) - { - SystemKeyspace.updatePreferredIP(address, preferredAddress); - - OutboundMessagingPool messagingPool = channelManagers.get(address); - if (messagingPool != null) - messagingPool.reconnectWithNewIp(InetAddressAndPort.getByAddressOverrideDefaults(preferredAddress.address, portFor(address))); - } - - private void reset(InetAddressAndPort address) - { - OutboundMessagingPool messagingPool = channelManagers.remove(address); - if (messagingPool != null) - messagingPool.close(false); - } - - public InetAddressAndPort getCurrentEndpoint(InetAddressAndPort publicAddress) - { - OutboundMessagingPool messagingPool = getMessagingConnection(publicAddress); - return messagingPool != null ? messagingPool.getPreferredRemoteAddr() : null; - } - - /** - * Register a verb and the corresponding verb handler with the - * Messaging Service. - * - * @param verb - * @param verbHandler handler for the specified verb - */ - public void registerVerbHandlers(Verb verb, IVerbHandler verbHandler) - { - assert !verbHandlers.containsKey(verb); - verbHandlers.put(verb, verbHandler); - } - - /** - * SHOULD ONLY BE USED FOR TESTING!! - */ - public void removeVerbHandler(Verb verb) - { - verbHandlers.remove(verb); - } - - /** - * This method returns the verb handler associated with the registered - * verb. If no handler has been registered then null is returned. - * - * @param type for which the verb handler is sought - * @return a reference to IVerbHandler which is the handler for the specified verb - */ - public IVerbHandler getVerbHandler(Verb type) - { - return verbHandlers.get(type); - } - - public int addWriteCallback(IAsyncCallback cb, MessageOut message, InetAddressAndPort to, long timeout, boolean failureCallback) - { - assert message.verb != Verb.MUTATION; // mutations need to call the overload with a ConsistencyLevel - int messageId = nextId(); - CallbackInfo previous = callbacks.put(messageId, new CallbackInfo(to, cb, callbackDeserializers.get(message.verb), failureCallback), timeout); - assert previous == null : String.format("Callback already exists for id %d! (%s)", messageId, previous); - return messageId; - } - - public int addWriteCallback(IAsyncCallback cb, - MessageOut message, - Replica to, - long timeout, - ConsistencyLevel consistencyLevel, - boolean allowHints) - { - assert message.verb == Verb.MUTATION - || message.verb == Verb.COUNTER_MUTATION - || message.verb == Verb.PAXOS_COMMIT; - int messageId = nextId(); - - CallbackInfo previous = callbacks.put(messageId, - new WriteCallbackInfo(to, - cb, - message, - callbackDeserializers.get(message.verb), - consistencyLevel, - allowHints), - timeout); - assert previous == null : String.format("Callback already exists for id %d! (%s)", messageId, previous); - return messageId; - } - - private static final AtomicInteger idGen = new AtomicInteger(0); - - private static int nextId() - { - return idGen.incrementAndGet(); - } - - public int sendRR(MessageOut message, InetAddressAndPort to, IAsyncCallback cb) - { - return sendRR(message, to, cb, message.getTimeout(), false); - } - - public int sendRRWithFailure(MessageOut message, InetAddressAndPort to, IAsyncCallbackWithFailure cb) - { - return sendRR(message, to, cb, message.getTimeout(), true); - } - - /** - * Send a non-mutation message to a given endpoint. This method specifies a callback - * which is invoked with the actual response. - * - * @param message message to be sent. - * @param to endpoint to which the message needs to be sent - * @param cb callback interface which is used to pass the responses or - * suggest that a timeout occurred to the invoker of the send(). - * @param timeout the timeout used for expiration - * @return an reference to message id used to match with the result - */ - public int sendRR(MessageOut message, InetAddressAndPort to, IAsyncCallback cb, long timeout, boolean failureCallback) - { - int id = addWriteCallback(cb, message, to, timeout, failureCallback); - updateBackPressureOnSend(to, cb, message); - sendOneWay(failureCallback ? message.withParameter(ParameterType.FAILURE_CALLBACK, ONE_BYTE) : message, id, to); - return id; - } - - /** - * Send a mutation message or a Paxos Commit to a given endpoint. This method specifies a callback - * which is invoked with the actual response. - * Also holds the message (only mutation messages) to determine if it - * needs to trigger a hint (uses StorageProxy for that). - * - * @param message message to be sent. - * @param to endpoint to which the message needs to be sent - * @param handler callback interface which is used to pass the responses or - * suggest that a timeout occurred to the invoker of the send(). - * @return an reference to message id used to match with the result - */ - public int sendWriteRR(MessageOut message, - Replica to, - AbstractWriteResponseHandler handler, - boolean allowHints) - { - int id = addWriteCallback(handler, message, to, message.getTimeout(), handler.consistencyLevel(), allowHints); - updateBackPressureOnSend(to.endpoint(), handler, message); - sendOneWay(message.withParameter(ParameterType.FAILURE_CALLBACK, ONE_BYTE), id, to.endpoint()); - return id; - } - - public void sendOneWay(MessageOut message, InetAddressAndPort to) - { - sendOneWay(message, nextId(), to); - } - - public void sendReply(MessageOut message, int id, InetAddressAndPort to) - { - sendOneWay(message, id, to); - } - - /** - * Send a message to a given endpoint. This method adheres to the fire and forget - * style messaging. - * - * @param message messages to be sent. - * @param to endpoint to which the message needs to be sent - */ - public void sendOneWay(MessageOut message, int id, InetAddressAndPort to) - { - if (logger.isTraceEnabled()) - logger.trace("{} sending {} to {}@{}", FBUtilities.getBroadcastAddressAndPort(), message.verb, id, to); - - if (to.equals(FBUtilities.getBroadcastAddressAndPort())) - logger.trace("Message-to-self {} going over MessagingService", message); - - // message sinks are a testing hook - for (IMessageSink ms : messageSinks) - if (!ms.allowOutgoingMessage(message, id, to)) - return; - - OutboundMessagingPool outboundMessagingPool = getMessagingConnection(to); - if (outboundMessagingPool != null) - outboundMessagingPool.sendMessage(message, id); - } - - public AsyncOneResponse sendRR(MessageOut message, InetAddressAndPort to) - { - AsyncOneResponse iar = new AsyncOneResponse(); - sendRR(message, to, iar); - return iar; - } - - public void register(ILatencySubscriber subcriber) - { - subscribers.add(subcriber); - } - - public void clearCallbacksUnsafe() - { - callbacks.reset(); - } - - /** - * Wait for callbacks and don't allow any more to be created (since they could require writing hints) - */ - public void shutdown() - { - shutdown(false); - } - - public void shutdown(boolean isTest) - { - logger.info("Waiting for messaging service to quiesce"); - // We may need to schedule hints on the mutation stage, so it's erroneous to shut down the mutation stage first - assert !StageManager.getStage(Stage.MUTATION).isShutdown(); - - // the important part - if (!callbacks.shutdownBlocking()) - logger.warn("Failed to wait for messaging service callbacks shutdown"); - - // attempt to humor tests that try to stop and restart MS - try - { - // first close the recieve channels - for (ServerChannel serverChannel : serverChannels) - serverChannel.close(); - - // now close the send channels - for (OutboundMessagingPool pool : channelManagers.values()) - pool.close(false); - - if (!isTest) - NettyFactory.instance.close(); - - clearMessageSinks(); - } - catch (Exception e) - { - throw new IOError(e); - } - } - - /** - * For testing only! - */ - void clearServerChannels() - { - serverChannels.clear(); - } - - public void receive(MessageIn message, int id) - { - TraceState state = Tracing.instance.initializeFromMessage(message); - if (state != null) - state.trace("{} message received from {}", message.verb, message.from); - - // message sinks are a testing hook - for (IMessageSink ms : messageSinks) - if (!ms.allowIncomingMessage(message, id)) - return; - - Runnable runnable = new MessageDeliveryTask(message, id); - LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType()); - assert stage != null : "No stage for message type " + message.verb; - - stage.execute(runnable, ExecutorLocals.create(state)); - } - - public void setCallbackForTests(int messageId, CallbackInfo callback) - { - callbacks.put(messageId, callback); - } - - public CallbackInfo getRegisteredCallback(int messageId) - { - return callbacks.get(messageId); - } - - public CallbackInfo removeRegisteredCallback(int messageId) - { - return callbacks.remove(messageId); - } - - /** - * @return System.nanoTime() when callback was created. - */ - public long getRegisteredCallbackAge(int messageId) - { - return callbacks.getAge(messageId); - } - - public static void validateMagic(int magic) throws IOException - { - if (magic != PROTOCOL_MAGIC) - throw new IOException("invalid protocol header"); - } - - public static int getBits(int packed, int start, int count) + private OutboundConnections getOutbound(InetAddressAndPort to) { - return packed >>> (start + 1) - count & ~(-1 << count); + OutboundConnections connections = channelManagers.get(to); + if (connections == null) + connections = OutboundConnections.tryRegister(channelManagers, to, new OutboundConnectionSettings(to).withDefaults(ConnectionCategory.MESSAGING), backPressure.newState(to)); + return connections; } - /** - * @return the last version associated with address, or @param version if this is the first such version - */ - public int setVersion(InetAddressAndPort endpoint, int version) - { - logger.trace("Setting version {} for {}", version, endpoint); - - Integer v = versions.put(endpoint, version); - return v == null ? version : v; - } - - public void resetVersion(InetAddressAndPort endpoint) - { - logger.trace("Resetting version for {}", endpoint); - versions.remove(endpoint); - } - - /** - * Returns the messaging-version as announced by the given node but capped - * to the min of the version as announced by the node and {@link #current_version}. - */ - public int getVersion(InetAddressAndPort endpoint) - { - Integer v = versions.get(endpoint); - if (v == null) - { - // we don't know the version. assume current. we'll know soon enough if that was incorrect. - logger.trace("Assuming current protocol version for {}", endpoint); - return MessagingService.current_version; - } - else - return Math.min(v, MessagingService.current_version); - } - - public int getVersion(String endpoint) throws UnknownHostException - { - return getVersion(InetAddressAndPort.getByName(endpoint)); - } - - /** - * Returns the messaging-version exactly as announced by the given endpoint. - */ - public int getRawVersion(InetAddressAndPort endpoint) - { - Integer v = versions.get(endpoint); - if (v == null) - throw new IllegalStateException("getRawVersion() was called without checking knowsVersion() result first"); - return v; - } - - public boolean knowsVersion(InetAddressAndPort endpoint) - { - return versions.containsKey(endpoint); - } - - public void incrementDroppedMutations(Optional mutationOpt, long timeTaken) - { - if (mutationOpt.isPresent()) - { - updateDroppedMutationCount(mutationOpt.get()); - } - incrementDroppedMessages(Verb.MUTATION, timeTaken); - } - - public void incrementDroppedMessages(Verb verb) - { - incrementDroppedMessages(verb, false); - } - - public void incrementDroppedMessages(Verb verb, long timeTaken) - { - incrementDroppedMessages(verb, timeTaken, false); - } - - public void incrementDroppedMessages(MessageIn message, long timeTaken) - { - if (message.payload instanceof IMutation) - { - updateDroppedMutationCount((IMutation) message.payload); - } - incrementDroppedMessages(message.verb, timeTaken, message.isCrossNode()); - } - - public void incrementDroppedMessages(Verb verb, long timeTaken, boolean isCrossNode) - { - assert DROPPABLE_VERBS.contains(verb) : "Verb " + verb + " should not legally be dropped"; - incrementDroppedMessages(droppedMessagesMap.get(verb), timeTaken, isCrossNode); - } - - public void incrementDroppedMessages(Verb verb, boolean isCrossNode) - { - assert DROPPABLE_VERBS.contains(verb) : "Verb " + verb + " should not legally be dropped"; - incrementDroppedMessages(droppedMessagesMap.get(verb), isCrossNode); - } - - private void updateDroppedMutationCount(IMutation mutation) - { - assert mutation != null : "Mutation should not be null when updating dropped mutations count"; - - for (TableId tableId : mutation.getTableIds()) - { - ColumnFamilyStore cfs = Keyspace.open(mutation.getKeyspaceName()).getColumnFamilyStore(tableId); - if (cfs != null) - { - cfs.metric.droppedMutations.inc(); - } - } - } - - private void incrementDroppedMessages(DroppedMessages droppedMessages, long timeTaken, boolean isCrossNode) - { - if (isCrossNode) - droppedMessages.metrics.crossNodeDroppedLatency.update(timeTaken, TimeUnit.MILLISECONDS); - else - droppedMessages.metrics.internalDroppedLatency.update(timeTaken, TimeUnit.MILLISECONDS); - incrementDroppedMessages(droppedMessages, isCrossNode); - } - - private void incrementDroppedMessages(DroppedMessages droppedMessages, boolean isCrossNode) - { - droppedMessages.metrics.dropped.mark(); - if (isCrossNode) - droppedMessages.droppedCrossNode.incrementAndGet(); - else - droppedMessages.droppedInternal.incrementAndGet(); - } - - private void logDroppedMessages() - { - List logs = getDroppedMessagesLogs(); - for (String log : logs) - logger.info(log); - - if (logs.size() > 0) - StatusLogger.log(); - } - - @VisibleForTesting - List getDroppedMessagesLogs() - { - List ret = new ArrayList<>(); - for (Map.Entry entry : droppedMessagesMap.entrySet()) - { - Verb verb = entry.getKey(); - DroppedMessages droppedMessages = entry.getValue(); - - int droppedInternal = droppedMessages.droppedInternal.getAndSet(0); - int droppedCrossNode = droppedMessages.droppedCrossNode.getAndSet(0); - if (droppedInternal > 0 || droppedCrossNode > 0) - { - ret.add(String.format("%s messages were dropped in last %d ms: %d internal and %d cross node." - + " Mean internal dropped latency: %d ms and Mean cross-node dropped latency: %d ms", - verb, - LOG_DROPPED_INTERVAL_IN_MS, - droppedInternal, - droppedCrossNode, - TimeUnit.NANOSECONDS.toMillis((long)droppedMessages.metrics.internalDroppedLatency.getSnapshot().getMean()), - TimeUnit.NANOSECONDS.toMillis((long)droppedMessages.metrics.crossNodeDroppedLatency.getSnapshot().getMean()))); - } - } - return ret; - } - - - private static void handleIOExceptionOnClose(IOException e) throws IOException - { - // dirty hack for clean shutdown on OSX w/ Java >= 1.8.0_20 - // see https://bugs.openjdk.java.net/browse/JDK-8050499; - // also CASSANDRA-12513 - if (NativeLibrary.osType == NativeLibrary.OSType.MAC) - { - switch (e.getMessage()) - { - case "Unknown error: 316": - case "No such file or directory": - return; - } - } - - throw e; - } - - public Map getLargeMessagePendingTasks() - { - Map pendingTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - pendingTasks.put(entry.getKey().toString(false), entry.getValue().largeMessageChannel.getPendingMessages()); - return pendingTasks; - } - - public Map getLargeMessageCompletedTasks() - { - Map completedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - completedTasks.put(entry.getKey().toString(false), entry.getValue().largeMessageChannel.getCompletedMessages()); - return completedTasks; - } - - public Map getLargeMessageDroppedTasks() - { - Map droppedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - droppedTasks.put(entry.getKey().toString(false), entry.getValue().largeMessageChannel.getDroppedMessages()); - return droppedTasks; - } - - public Map getSmallMessagePendingTasks() - { - Map pendingTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - pendingTasks.put(entry.getKey().toString(false), entry.getValue().smallMessageChannel.getPendingMessages()); - return pendingTasks; - } - - public Map getSmallMessageCompletedTasks() - { - Map completedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - completedTasks.put(entry.getKey().toString(false), entry.getValue().smallMessageChannel.getCompletedMessages()); - return completedTasks; - } - - public Map getSmallMessageDroppedTasks() - { - Map droppedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - droppedTasks.put(entry.getKey().toString(false), entry.getValue().smallMessageChannel.getDroppedMessages()); - return droppedTasks; - } - - public Map getGossipMessagePendingTasks() - { - Map pendingTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - pendingTasks.put(entry.getKey().toString(false), entry.getValue().gossipChannel.getPendingMessages()); - return pendingTasks; - } - - public Map getGossipMessageCompletedTasks() - { - Map completedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - completedTasks.put(entry.getKey().toString(false), entry.getValue().gossipChannel.getCompletedMessages()); - return completedTasks; - } - - public Map getGossipMessageDroppedTasks() - { - Map droppedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - droppedTasks.put(entry.getKey().toString(false), entry.getValue().gossipChannel.getDroppedMessages()); - return droppedTasks; - } - - public Map getLargeMessagePendingTasksWithPort() + InboundMessageHandlers getInbound(InetAddressAndPort from) { - Map pendingTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - pendingTasks.put(entry.getKey().toString(), entry.getValue().largeMessageChannel.getPendingMessages()); - return pendingTasks; - } + InboundMessageHandlers handlers = messageHandlers.get(from); + if (null != handlers) + return handlers; - public Map getLargeMessageCompletedTasksWithPort() - { - Map completedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - completedTasks.put(entry.getKey().toString(), entry.getValue().largeMessageChannel.getCompletedMessages()); - return completedTasks; - } - - public Map getLargeMessageDroppedTasksWithPort() - { - Map droppedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - droppedTasks.put(entry.getKey().toString(), entry.getValue().largeMessageChannel.getDroppedMessages()); - return droppedTasks; - } - - public Map getSmallMessagePendingTasksWithPort() - { - Map pendingTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - pendingTasks.put(entry.getKey().toString(), entry.getValue().smallMessageChannel.getPendingMessages()); - return pendingTasks; - } - - public Map getSmallMessageCompletedTasksWithPort() - { - Map completedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - completedTasks.put(entry.getKey().toString(), entry.getValue().smallMessageChannel.getCompletedMessages()); - return completedTasks; - } - - public Map getSmallMessageDroppedTasksWithPort() - { - Map droppedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - droppedTasks.put(entry.getKey().toString(), entry.getValue().smallMessageChannel.getDroppedMessages()); - return droppedTasks; - } - - public Map getGossipMessagePendingTasksWithPort() - { - Map pendingTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - pendingTasks.put(entry.getKey().toString(), entry.getValue().gossipChannel.getPendingMessages()); - return pendingTasks; - } - - public Map getGossipMessageCompletedTasksWithPort() - { - Map completedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - completedTasks.put(entry.getKey().toString(), entry.getValue().gossipChannel.getCompletedMessages()); - return completedTasks; - } - - public Map getGossipMessageDroppedTasksWithPort() - { - Map droppedTasks = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - droppedTasks.put(entry.getKey().toString(), entry.getValue().gossipChannel.getDroppedMessages()); - return droppedTasks; - } - - public Map getDroppedMessages() - { - Map map = new HashMap<>(droppedMessagesMap.size()); - for (Map.Entry entry : droppedMessagesMap.entrySet()) - map.put(entry.getKey().toString(), (int) entry.getValue().metrics.dropped.getCount()); - return map; - } - - public long getTotalTimeouts() - { - return ConnectionMetrics.totalTimeouts.getCount(); - } - - public Map getTimeoutsPerHost() - { - Map result = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - { - String ip = entry.getKey().toString(false); - long recent = entry.getValue().getTimeouts(); - result.put(ip, recent); - } - return result; - } - - public Map getTimeoutsPerHostWithPort() - { - Map result = new HashMap(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - { - String ip = entry.getKey().toString(); - long recent = entry.getValue().getTimeouts(); - result.put(ip, recent); - } - return result; - } - - public Map getBackPressurePerHost() - { - Map map = new HashMap<>(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - map.put(entry.getKey().toString(false), entry.getValue().getBackPressureState().getBackPressureRateLimit()); - - return map; - } - - public Map getBackPressurePerHostWithPort() - { - Map map = new HashMap<>(channelManagers.size()); - for (Map.Entry entry : channelManagers.entrySet()) - map.put(entry.getKey().toString(false), entry.getValue().getBackPressureState().getBackPressureRateLimit()); - - return map; - } - - @Override - public void setBackPressureEnabled(boolean enabled) - { - DatabaseDescriptor.setBackPressureEnabled(enabled); - } - - @Override - public boolean isBackPressureEnabled() - { - return DatabaseDescriptor.backPressureEnabled(); - } - - public static IPartitioner globalPartitioner() - { - return StorageService.instance.getTokenMetadata().partitioner; - } - - public static void validatePartitioner(Collection> allBounds) - { - for (AbstractBounds bounds : allBounds) - validatePartitioner(bounds); - } - - public static void validatePartitioner(AbstractBounds bounds) - { - if (globalPartitioner() != bounds.left.getPartitioner()) - throw new AssertionError(String.format("Partitioner in bounds serialization. Expected %s, was %s.", - globalPartitioner().getClass().getName(), - bounds.left.getPartitioner().getClass().getName())); - } - - /** - * This method is used to determine the preferred IP & Port of a peer using the - * {@link OutboundMessagingPool} and SystemKeyspace. - */ - public InetAddressAndPort getPreferredRemoteAddr(InetAddressAndPort to) - { - OutboundMessagingPool pool = channelManagers.get(to); - return pool != null ? pool.getPreferredRemoteAddr() : SystemKeyspace.getPreferredIP(to); - } - - private OutboundMessagingPool getMessagingConnection(InetAddressAndPort to) - { - OutboundMessagingPool pool = channelManagers.get(to); - if (pool == null) - { - final boolean secure = isEncryptedConnection(to); - final int port = portFor(to, secure); - if (!DatabaseDescriptor.getInternodeAuthenticator().authenticate(to.address, port)) - return null; - - InetAddressAndPort preferredRemote = SystemKeyspace.getPreferredIP(to); - InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort(); - ServerEncryptionOptions encryptionOptions = secure ? DatabaseDescriptor.getInternodeMessagingEncyptionOptions() : null; - IInternodeAuthenticator authenticator = DatabaseDescriptor.getInternodeAuthenticator(); - - pool = new OutboundMessagingPool(preferredRemote, local, encryptionOptions, backPressure.newState(to), authenticator); - OutboundMessagingPool existing = channelManagers.putIfAbsent(to, pool); - if (existing != null) - { - pool.close(false); - pool = existing; - } - } - return pool; - } - - public int portFor(InetAddressAndPort addr) - { - final boolean secure = isEncryptedConnection(addr); - return portFor(addr, secure); - } - - private int portFor(InetAddressAndPort address, boolean secure) - { - if (!secure) - return address.port; - - Integer v = versions.get(address); - // if we don't know the version of the peer, assume it is 4.0 (or higher) as the only time is would be lower - // (as in a 3.x version) is during a cluster upgrade (from 3.x to 4.0). In that case the outbound connection will - // unfortunately fail - however the peer should connect to this node (at some point), and once we learn it's version, it'll be - // in versions map. thus, when we attempt to reconnect to that node, we'll have the version and we can get the correct port. - // we will be able to remove this logic at 5.0. - // Also as of 4.0 we will propagate the "regular" port (which will support both SSL and non-SSL) via gossip so - // for SSL and version 4.0 always connect to the gossiped port because if SSL is enabled it should ALWAYS - // listen for SSL on the "regular" port. - int version = v != null ? v.intValue() : VERSION_40; - return version < VERSION_40 ? DatabaseDescriptor.getSSLStoragePort() : address.port; + return messageHandlers.computeIfAbsent(from, addr -> + new InboundMessageHandlers(FBUtilities.getLocalAddressAndPort(), + addr, + DatabaseDescriptor.getInternodeApplicationReceiveQueueCapacityInBytes(), + DatabaseDescriptor.getInternodeApplicationReceiveQueueReserveEndpointCapacityInBytes(), + inboundGlobalReserveLimits, metrics, inboundSink) + ); } @VisibleForTesting - boolean isConnected(InetAddressAndPort address, MessageOut messageOut) + boolean isConnected(InetAddressAndPort address, Message messageOut) { - OutboundMessagingPool pool = channelManagers.get(address); + OutboundConnections pool = channelManagers.get(address); if (pool == null) return false; - return pool.getConnection(messageOut).isConnected(); + return pool.connectionFor(messageOut).isConnected(); } - public static boolean isEncryptedConnection(InetAddressAndPort address) + public void listen() { - IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch(); - switch (DatabaseDescriptor.getInternodeMessagingEncyptionOptions().internode_encryption) - { - case none: - return false; // if nothing needs to be encrypted then return immediately. - case all: - break; - case dc: - if (snitch.getDatacenter(address).equals(snitch.getLocalDatacenter())) - return false; - break; - case rack: - // for rack then check if the DC's are the same. - if (snitch.getRack(address).equals(snitch.getLocalRack()) - && snitch.getDatacenter(address).equals(snitch.getLocalDatacenter())) - return false; - break; - } - return true; + inboundSockets.open(); } - @Override - public void reloadSslCertificates() throws IOException + public void waitUntilListening() throws InterruptedException { - final ServerEncryptionOptions serverOpts = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); - final EncryptionOptions clientOpts = DatabaseDescriptor.getNativeProtocolEncryptionOptions(); - SSLFactory.validateSslCerts(serverOpts, clientOpts); - SSLFactory.checkCertFilesForHotReloading(serverOpts, clientOpts); + inboundSockets.open().await(); } } diff --git a/src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java b/src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java new file mode 100644 index 000000000000..b48ae1c625e1 --- /dev/null +++ b/src/java/org/apache/cassandra/net/MessagingServiceMBeanImpl.java @@ -0,0 +1,304 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.metrics.InternodeOutboundMetrics; +import org.apache.cassandra.metrics.MessagingMetrics; +import org.apache.cassandra.security.SSLFactory; +import org.apache.cassandra.utils.MBeanWrapper; + +public class MessagingServiceMBeanImpl implements MessagingServiceMBean +{ + public static final String MBEAN_NAME = "org.apache.cassandra.net:type=MessagingService"; + + // we use CHM deliberately instead of NBHM, as both are non-blocking for readers (which this map mostly is used for) + // and CHM permits prompter GC + public final ConcurrentMap channelManagers = new ConcurrentHashMap<>(); + public final ConcurrentMap messageHandlers = new ConcurrentHashMap<>(); + + public final EndpointMessagingVersions versions = new EndpointMessagingVersions(); + public final MessagingMetrics metrics = new MessagingMetrics(); + + MessagingServiceMBeanImpl(boolean testOnly) + { + if (!testOnly) + { + MBeanWrapper.instance.registerMBean(this, MBEAN_NAME); + metrics.scheduleLogging(); + } + } + + @Override + public Map getLargeMessagePendingTasks() + { + Map pendingTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + pendingTasks.put(entry.getKey().toString(false), entry.getValue().large.pendingCount()); + return pendingTasks; + } + + @Override + public Map getLargeMessageCompletedTasks() + { + Map completedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + completedTasks.put(entry.getKey().toString(false), entry.getValue().large.sentCount()); + return completedTasks; + } + + @Override + public Map getLargeMessageDroppedTasks() + { + Map droppedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + droppedTasks.put(entry.getKey().toString(false), entry.getValue().large.dropped()); + return droppedTasks; + } + + @Override + public Map getSmallMessagePendingTasks() + { + Map pendingTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + pendingTasks.put(entry.getKey().toString(false), entry.getValue().small.pendingCount()); + return pendingTasks; + } + + @Override + public Map getSmallMessageCompletedTasks() + { + Map completedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + completedTasks.put(entry.getKey().toString(false), entry.getValue().small.sentCount()); + return completedTasks; + } + + @Override + public Map getSmallMessageDroppedTasks() + { + Map droppedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + droppedTasks.put(entry.getKey().toString(false), entry.getValue().small.dropped()); + return droppedTasks; + } + + @Override + public Map getGossipMessagePendingTasks() + { + Map pendingTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + pendingTasks.put(entry.getKey().toString(false), entry.getValue().urgent.pendingCount()); + return pendingTasks; + } + + @Override + public Map getGossipMessageCompletedTasks() + { + Map completedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + completedTasks.put(entry.getKey().toString(false), entry.getValue().urgent.sentCount()); + return completedTasks; + } + + @Override + public Map getGossipMessageDroppedTasks() + { + Map droppedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + droppedTasks.put(entry.getKey().toString(false), entry.getValue().urgent.dropped()); + return droppedTasks; + } + + @Override + public Map getLargeMessagePendingTasksWithPort() + { + Map pendingTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + pendingTasks.put(entry.getKey().toString(), entry.getValue().large.pendingCount()); + return pendingTasks; + } + + @Override + public Map getLargeMessageCompletedTasksWithPort() + { + Map completedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + completedTasks.put(entry.getKey().toString(), entry.getValue().large.sentCount()); + return completedTasks; + } + + @Override + public Map getLargeMessageDroppedTasksWithPort() + { + Map droppedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + droppedTasks.put(entry.getKey().toString(), entry.getValue().large.dropped()); + return droppedTasks; + } + + @Override + public Map getSmallMessagePendingTasksWithPort() + { + Map pendingTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + pendingTasks.put(entry.getKey().toString(), entry.getValue().small.pendingCount()); + return pendingTasks; + } + + @Override + public Map getSmallMessageCompletedTasksWithPort() + { + Map completedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + completedTasks.put(entry.getKey().toString(), entry.getValue().small.sentCount()); + return completedTasks; + } + + @Override + public Map getSmallMessageDroppedTasksWithPort() + { + Map droppedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + droppedTasks.put(entry.getKey().toString(), entry.getValue().small.dropped()); + return droppedTasks; + } + + @Override + public Map getGossipMessagePendingTasksWithPort() + { + Map pendingTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + pendingTasks.put(entry.getKey().toString(), entry.getValue().urgent.pendingCount()); + return pendingTasks; + } + + @Override + public Map getGossipMessageCompletedTasksWithPort() + { + Map completedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + completedTasks.put(entry.getKey().toString(), entry.getValue().urgent.sentCount()); + return completedTasks; + } + + @Override + public Map getGossipMessageDroppedTasksWithPort() + { + Map droppedTasks = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + droppedTasks.put(entry.getKey().toString(), entry.getValue().urgent.dropped()); + return droppedTasks; + } + + @Override + public Map getDroppedMessages() + { + return metrics.getDroppedMessages(); + } + + @Override + public long getTotalTimeouts() + { + return InternodeOutboundMetrics.totalExpiredCallbacks.getCount(); + } + + // these are not messages that time out on sending, but callbacks that timedout without receiving a response + @Override + public Map getTimeoutsPerHost() + { + Map result = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + { + String ip = entry.getKey().toString(false); + long recent = entry.getValue().expiredCallbacks(); + result.put(ip, recent); + } + return result; + } + + // these are not messages that time out on sending, but callbacks that timedout without receiving a response + @Override + public Map getTimeoutsPerHostWithPort() + { + Map result = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + { + String ip = entry.getKey().toString(); + long recent = entry.getValue().expiredCallbacks(); + result.put(ip, recent); + } + return result; + } + + @Override + public Map getBackPressurePerHost() + { + Map map = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + map.put(entry.getKey().toString(false), entry.getValue().getBackPressureState().getBackPressureRateLimit()); + + return map; + } + + @Override + public Map getBackPressurePerHostWithPort() + { + Map map = new HashMap<>(channelManagers.size()); + for (Map.Entry entry : channelManagers.entrySet()) + map.put(entry.getKey().toString(false), entry.getValue().getBackPressureState().getBackPressureRateLimit()); + + return map; + } + + @Override + public void setBackPressureEnabled(boolean enabled) + { + DatabaseDescriptor.setBackPressureEnabled(enabled); + } + + @Override + public boolean isBackPressureEnabled() + { + return DatabaseDescriptor.backPressureEnabled(); + } + + @Override + public void reloadSslCertificates() throws IOException + { + final EncryptionOptions.ServerEncryptionOptions serverOpts = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); + final EncryptionOptions clientOpts = DatabaseDescriptor.getNativeProtocolEncryptionOptions(); + SSLFactory.validateSslCerts(serverOpts, clientOpts); + SSLFactory.checkCertFilesForHotReloading(serverOpts, clientOpts); + } + + @Override + public int getVersion(String address) throws UnknownHostException + { + return versions.get(address); + } +} diff --git a/src/java/org/apache/cassandra/net/PongMessage.java b/src/java/org/apache/cassandra/net/NoPayload.java similarity index 59% rename from src/java/org/apache/cassandra/net/PongMessage.java rename to src/java/org/apache/cassandra/net/NoPayload.java index bb89cdf51368..3b2b1772a8cc 100644 --- a/src/java/org/apache/cassandra/net/PongMessage.java +++ b/src/java/org/apache/cassandra/net/NoPayload.java @@ -15,36 +15,39 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.net; -import java.io.IOException; - import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -public class PongMessage +/** + * Empty message payload - primarily used for responses. + * + * Prefer this singleton to writing one-off specialised classes. + */ +public class NoPayload { - public static final PongMessage instance = new PongMessage(); - public static IVersionedSerializer serializer = new PongMessage.PongMessageSerializer(); + public static final NoPayload noPayload = new NoPayload(); - private PongMessage() - { } + private NoPayload() {} - public static class PongMessageSerializer implements IVersionedSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { - public void serialize(PongMessage t, DataOutputPlus out, int version) throws IOException - { } + public void serialize(NoPayload noPayload, DataOutputPlus out, int version) + { + if (noPayload != NoPayload.noPayload) + throw new IllegalArgumentException(); + } - public PongMessage deserialize(DataInputPlus in, int version) throws IOException + public NoPayload deserialize(DataInputPlus in, int version) { - return instance; + return noPayload; } - public long serializedSize(PongMessage t, int version) + public long serializedSize(NoPayload noPayload, int version) { return 0; } - } + }; } diff --git a/src/java/org/apache/cassandra/net/NoSizeEstimator.java b/src/java/org/apache/cassandra/net/NoSizeEstimator.java new file mode 100644 index 000000000000..848d4f566269 --- /dev/null +++ b/src/java/org/apache/cassandra/net/NoSizeEstimator.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import io.netty.channel.MessageSizeEstimator; + +/** + * We want to manage the bytes we have in-flight, so this class asks Netty not to by returning zero for every object. + */ +class NoSizeEstimator implements MessageSizeEstimator, MessageSizeEstimator.Handle +{ + public static final NoSizeEstimator instance = new NoSizeEstimator(); + private NoSizeEstimator() {} + public Handle newHandle() { return this; } + public int size(Object o) { return 0; } +} diff --git a/src/java/org/apache/cassandra/net/OutboundConnection.java b/src/java/org/apache/cassandra/net/OutboundConnection.java new file mode 100644 index 000000000000..63b909c58a13 --- /dev/null +++ b/src/java/org/apache/cassandra/net/OutboundConnection.java @@ -0,0 +1,1729 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.nio.channels.ClosedChannelException; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.Uninterruptibles; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.EventLoop; +import io.netty.channel.unix.Errors; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.Promise; +import io.netty.util.concurrent.PromiseNotifier; +import io.netty.util.concurrent.SucceededFuture; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.util.DataOutputBufferFixed; +import org.apache.cassandra.net.OutboundConnectionInitiator.Result.MessagingSuccess; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.NoSpamLogger; + +import static java.lang.Math.max; +import static java.lang.Math.min; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.OutboundConnectionInitiator.*; +import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD; +import static org.apache.cassandra.net.ResourceLimits.*; +import static org.apache.cassandra.net.ResourceLimits.Outcome.*; +import static org.apache.cassandra.net.SocketFactory.*; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; +import static org.apache.cassandra.utils.Throwables.isCausedBy; + +/** + * Represents a connection type to a peer, and handles the state transistions on the connection and the netty {@link Channel}. + * The underlying socket is not opened until explicitly requested (by sending a message). + * + * TODO: complete this description + * + * Aside from a few administrative methods, the main entry point to sending a message is {@link #enqueue(Message)}. + * Any thread may send a message (enqueueing it to {@link #queue}), but only one thread may consume messages from this + * queue. There is a single delivery thread - either the event loop, or a companion thread - that has logical ownership + * of the queue, but other threads may temporarily take ownership in order to perform book keeping, pruning, etc., + * to ensure system stability. + * + * {@link Delivery#run()} is the main entry point for consuming messages from the queue, and executes either on the event + * loop or on a non-dedicated companion thread. This processing is activated via {@link Delivery#execute()}. + * + * Almost all internal state maintenance on this class occurs on the eventLoop, a single threaded executor which is + * assigned in the constructor. Further details are outlined below in the class. Some behaviours require coordination + * between the eventLoop and the companion thread (if any). Some minimal set of behaviours are permitted to occur on + * producers to ensure the connection remains healthy and does not overcommit resources. + * + * All methods are safe to invoke from any thread unless otherwise stated. + */ +@SuppressWarnings({ "WeakerAccess", "FieldMayBeFinal", "NonAtomicOperationOnVolatileField", "SameParameterValue" }) +public class OutboundConnection +{ + static final Logger logger = LoggerFactory.getLogger(OutboundConnection.class); + private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 30L, TimeUnit.SECONDS); + + private static final AtomicLongFieldUpdater submittedUpdater = AtomicLongFieldUpdater.newUpdater(OutboundConnection.class, "submittedCount"); + private static final AtomicLongFieldUpdater pendingCountAndBytesUpdater = AtomicLongFieldUpdater.newUpdater(OutboundConnection.class, "pendingCountAndBytes"); + private static final AtomicLongFieldUpdater overloadedCountUpdater = AtomicLongFieldUpdater.newUpdater(OutboundConnection.class, "overloadedCount"); + private static final AtomicLongFieldUpdater overloadedBytesUpdater = AtomicLongFieldUpdater.newUpdater(OutboundConnection.class, "overloadedBytes"); + private static final AtomicReferenceFieldUpdater closingUpdater = AtomicReferenceFieldUpdater.newUpdater(OutboundConnection.class, Future.class, "closing"); + private static final AtomicReferenceFieldUpdater scheduledCloseUpdater = AtomicReferenceFieldUpdater.newUpdater(OutboundConnection.class, Future.class, "scheduledClose"); + + private final EventLoop eventLoop; + private final Delivery delivery; + + private final OutboundMessageCallbacks callbacks; + private final OutboundDebugCallbacks debug; + private final OutboundMessageQueue queue; + /** the number of bytes we permit to queue to the network without acquiring any shared resource permits */ + private final long pendingCapacityInBytes; + /** the number of messages and bytes queued for flush to the network, + * including those that are being flushed but have not been completed, + * packed into a long (top 20 bits for count, bottom 42 for bytes)*/ + private volatile long pendingCountAndBytes = 0; + /** global shared limits that we use only if our local limits are exhausted; + * we allocate from here whenever queueSize > queueCapacity */ + private final EndpointAndGlobal reserveCapacityInBytes; + + private volatile long submittedCount = 0; // updated with cas + private volatile long overloadedCount = 0; // updated with cas + private volatile long overloadedBytes = 0; // updated with cas + private long expiredCount = 0; // updated with queue lock held + private long expiredBytes = 0; // updated with queue lock held + private long errorCount = 0; // updated only by delivery thread + private long errorBytes = 0; // updated by delivery thread only + private long sentCount; // updated by delivery thread only + private long sentBytes; // updated by delivery thread only + private long successfulConnections; // updated by event loop only + private long connectionAttempts; // updated by event loop only + + private static final int pendingByteBits = 42; + private static boolean isMaxPendingCount(long pendingCountAndBytes) + { + return (pendingCountAndBytes & (-1L << pendingByteBits)) == (-1L << pendingByteBits); + } + + private static int pendingCount(long pendingCountAndBytes) + { + return (int) (pendingCountAndBytes >>> pendingByteBits); + } + + private static long pendingBytes(long pendingCountAndBytes) + { + return pendingCountAndBytes & (-1L >>> (64 - pendingByteBits)); + } + + private static long pendingCountAndBytes(long pendingCount, long pendingBytes) + { + return (pendingCount << pendingByteBits) | pendingBytes; + } + + private final ConnectionType type; + + /** + * Contains the base settings for this connection, _including_ any defaults filled in. + * + */ + private OutboundConnectionSettings template; + + private static class State + { + static final State CLOSED = new State(Kind.CLOSED); + + enum Kind { ESTABLISHED, CONNECTING, DORMANT, CLOSED } + + final Kind kind; + + State(Kind kind) + { + this.kind = kind; + } + + boolean isEstablished() { return kind == Kind.ESTABLISHED; } + boolean isConnecting() { return kind == Kind.CONNECTING; } + boolean isDisconnected() { return kind == Kind.CONNECTING || kind == Kind.DORMANT; } + boolean isClosed() { return kind == Kind.CLOSED; } + + Established established() { return (Established) this; } + Connecting connecting() { return (Connecting) this; } + Disconnected disconnected() { return (Disconnected) this; } + } + + /** + * We have successfully negotiated a channel, and believe it to still be valid. + * + * Before using this, we should check isConnected() to check the Channel hasn't + * become invalid. + */ + private static class Established extends State + { + final int messagingVersion; + final Channel channel; + final FrameEncoder.PayloadAllocator payloadAllocator; + final OutboundConnectionSettings settings; + + Established(int messagingVersion, Channel channel, FrameEncoder.PayloadAllocator payloadAllocator, OutboundConnectionSettings settings) + { + super(Kind.ESTABLISHED); + this.messagingVersion = messagingVersion; + this.channel = channel; + this.payloadAllocator = payloadAllocator; + this.settings = settings; + } + + boolean isConnected() { return channel.isOpen(); } + } + + private static class Disconnected extends State + { + /** Periodic message expiry scheduled while we are disconnected; this will be cancelled and cleared each time we connect */ + final Future maintenance; + Disconnected(Kind kind, Future maintenance) + { + super(kind); + this.maintenance = maintenance; + } + + public static Disconnected dormant(Future maintenance) + { + return new Disconnected(Kind.DORMANT, maintenance); + } + } + + private static class Connecting extends Disconnected + { + /** + * Currently (or scheduled to) (re)connect; this may be cancelled (if closing) or waited on (for delivery) + * + * - The work managed by this future is partially performed asynchronously, not necessarily on the eventLoop. + * - It is only completed on the eventLoop + * - It may not be executing, but might be scheduled to be submitted if {@link #scheduled} is not null + */ + final Future> attempt; + + /** + * If we are retrying to connect with some delay, this represents the scheduled inititation of another attempt + */ + @Nullable + final Future scheduled; + + /** + * true iff we are retrying to connect after some failure (immediately or following a delay) + */ + final boolean isFailingToConnect; + + Connecting(Disconnected previous, Future> attempt) + { + this(previous, attempt, null); + } + + Connecting(Disconnected previous, Future> attempt, Future scheduled) + { + super(Kind.CONNECTING, previous.maintenance); + this.attempt = attempt; + this.scheduled = scheduled; + this.isFailingToConnect = scheduled != null || (previous.isConnecting() && previous.connecting().isFailingToConnect); + } + + /** + * Cancel the connection attempt + * + * No cleanup is needed here, as {@link #attempt} is only completed on the eventLoop, + * so we have either already invoked the callbacks and are no longer in {@link #state}, + * or the {@link OutboundConnectionInitiator} will handle our successful cancellation + * when it comes to complete, by closing the channel (if we could not cancel it before then) + */ + void cancel() + { + if (scheduled != null) + scheduled.cancel(true); + + // we guarantee that attempt is only ever completed by the eventLoop + boolean cancelled = attempt.cancel(true); + assert cancelled; + } + } + + private volatile State state; + + /** The connection is being permanently closed */ + private volatile Future closing; + /** The connection is being permanently closed in the near future */ + private volatile Future scheduledClose; + + OutboundConnection(ConnectionType type, OutboundConnectionSettings settings, EndpointAndGlobal reserveCapacityInBytes) + { + this.template = settings.withDefaults(ConnectionCategory.MESSAGING); + this.type = type; + this.eventLoop = template.socketFactory.defaultGroup().next(); + this.pendingCapacityInBytes = template.applicationSendQueueCapacityInBytes; + this.reserveCapacityInBytes = reserveCapacityInBytes; + this.callbacks = template.callbacks; + this.debug = template.debug; + this.queue = new OutboundMessageQueue(this::onExpired); + this.delivery = type == ConnectionType.LARGE_MESSAGES + ? new LargeMessageDelivery(template.socketFactory.synchronousWorkExecutor) + : new EventLoopDelivery(); + setDisconnected(); + } + + /** + * This is the main entry point for enqueuing a message to be sent to the remote peer. + */ + public void enqueue(Message message) throws ClosedChannelException + { + if (isClosing()) + throw new ClosedChannelException(); + + final int canonicalSize = canonicalSize(message); + if (canonicalSize > DatabaseDescriptor.getInternodeMaxMessageSizeInBytes()) + throw new Message.OversizedMessageException(canonicalSize); + + submittedUpdater.incrementAndGet(this); + switch (acquireCapacity(canonicalSize)) + { + case INSUFFICIENT_ENDPOINT: + // if we're overloaded to one endpoint, we may be accumulating expirable messages, so + // attempt an expiry to see if this makes room for our newer message. + // this is an optimisation only; messages will be expired on ~100ms cycle, and by Delivery when it runs + if (queue.maybePruneExpired() && SUCCESS == acquireCapacity(canonicalSize)) + break; + case INSUFFICIENT_GLOBAL: + onOverloaded(message); + return; + } + + queue.add(message); + delivery.execute(); + + // we might race with the channel closing; if this happens, to ensure this message eventually arrives + // we need to remove ourselves from the queue and throw a ClosedChannelException, so that another channel + // can be opened in our place to try and send on. + if (isClosing() && queue.remove(message)) + { + releaseCapacity(1, canonicalSize); + throw new ClosedChannelException(); + } + } + + /** + * Try to acquire the necessary resource permits for a number of pending bytes for this connection. + * + * Since the owner limit is shared amongst multiple connections, our semantics cannot be super trivial. + * Were they per-connection, we could simply perform an atomic increment of the queue size, then + * allocate any excess we need in the reserve, and on release free everything we see from both. + * Since we are coordinating two independent atomic variables we have to track every byte we allocate in reserve + * and ensure it is matched by a corresponding released byte. We also need to be sure we do not permit another + * releasing thread to release reserve bytes we have not yet - and may never - actually reserve. + * + * As such, we have to first check if we would need reserve bytes, then allocate them *before* we increment our + * queue size. We only increment the queue size if the reserve bytes are definitely not needed, or we could first + * obtain them. If in the process of obtaining any reserve bytes the queue size changes, we have some bytes that are + * reserved for us, but may be a different number to that we need. So we must continue to track these. + * + * In the happy path, this is still efficient as we simply CAS + */ + private Outcome acquireCapacity(long bytes) + { + return acquireCapacity(1, bytes); + } + + private Outcome acquireCapacity(long count, long bytes) + { + long increment = pendingCountAndBytes(count, bytes); + long unusedClaimedReserve = 0; + Outcome outcome = null; + loop: while (true) + { + long current = pendingCountAndBytes; + if (isMaxPendingCount(current)) + { + outcome = INSUFFICIENT_ENDPOINT; + break; + } + + long next = current + increment; + if (pendingBytes(next) <= pendingCapacityInBytes) + { + if (pendingCountAndBytesUpdater.compareAndSet(this, current, next)) + { + outcome = SUCCESS; + break; + } + continue; + } + + State state = this.state; + if (state.isConnecting() && state.connecting().isFailingToConnect) + { + outcome = INSUFFICIENT_ENDPOINT; + break; + } + + long requiredReserve = min(bytes, pendingBytes(next) - pendingCapacityInBytes); + if (unusedClaimedReserve < requiredReserve) + { + long extraGlobalReserve = requiredReserve - unusedClaimedReserve; + switch (outcome = reserveCapacityInBytes.tryAllocate(extraGlobalReserve)) + { + case INSUFFICIENT_ENDPOINT: + case INSUFFICIENT_GLOBAL: + break loop; + case SUCCESS: + unusedClaimedReserve += extraGlobalReserve; + } + } + + if (pendingCountAndBytesUpdater.compareAndSet(this, current, next)) + { + unusedClaimedReserve -= requiredReserve; + break; + } + } + + if (unusedClaimedReserve > 0) + reserveCapacityInBytes.release(unusedClaimedReserve); + + return outcome; + } + + /** + * Mark a number of pending bytes as flushed to the network, releasing their capacity for new outbound messages. + */ + private void releaseCapacity(long count, long bytes) + { + long decrement = pendingCountAndBytes(count, bytes); + long prev = pendingCountAndBytesUpdater.getAndAdd(this, -decrement); + if (pendingBytes(prev) > pendingCapacityInBytes) + { + long excess = min(pendingBytes(prev) - pendingCapacityInBytes, bytes); + reserveCapacityInBytes.release(excess); + } + } + + private void onOverloaded(Message message) + { + overloadedCountUpdater.incrementAndGet(this); + overloadedBytesUpdater.addAndGet(this, canonicalSize(message)); + noSpamLogger.warn("{} overloaded; dropping {} message (queue: {} local, {} endpoint, {} global)", + id(), + FBUtilities.prettyPrintMemory(canonicalSize(message)), + FBUtilities.prettyPrintMemory(pendingBytes()), + FBUtilities.prettyPrintMemory(reserveCapacityInBytes.endpoint.using()), + FBUtilities.prettyPrintMemory(reserveCapacityInBytes.global.using())); + callbacks.onOverloaded(message, template.to); + } + + /** + * Take any necessary cleanup action after a message has been selected to be discarded from the queue. + * + * Only to be invoked while holding OutboundMessageQueue.WithLock + */ + private boolean onExpired(Message message) + { + releaseCapacity(1, canonicalSize(message)); + expiredCount += 1; + expiredBytes += canonicalSize(message); + noSpamLogger.warn("{} dropping message of type {} whose timeout expired before reaching the network", id(), message.verb()); + callbacks.onExpired(message, template.to); + return true; + } + + /** + * Take any necessary cleanup action after a message has been selected to be discarded from the queue. + * + * Only to be invoked by the delivery thread + */ + private void onFailedSerialize(Message message, int messagingVersion, int bytesWrittenToNetwork, Throwable t) + { + JVMStabilityInspector.inspectThrowable(t, false); + releaseCapacity(1, canonicalSize(message)); + errorCount += 1; + errorBytes += message.serializedSize(messagingVersion); + logger.warn("{} dropping message of type {} due to error", id(), message.verb(), t); + callbacks.onFailedSerialize(message, template.to, messagingVersion, bytesWrittenToNetwork, t); + } + + /** + * Take any necessary cleanup action after a message has been selected to be discarded from the queue on close. + * Note that this is only for messages that were queued prior to closing without graceful flush, OR + * for those that are unceremoniously dropped when we decide close has been trying to complete for too long. + */ + private void onClosed(Message message) + { + releaseCapacity(1, canonicalSize(message)); + callbacks.onDiscardOnClose(message, template.to); + } + + /** + * Delivery bundles the following: + * + * - the work that is necessary to actually deliver messages safely, and handle any exceptional states + * - the ability to schedule delivery for some time in the future + * - the ability to schedule some non-delivery work to happen some time in the future, that is guaranteed + * NOT to coincide with delivery for its duration, including any data that is being flushed (e.g. for closing channels) + * - this feature is *not* efficient, and should only be used for infrequent operations + */ + private abstract class Delivery extends AtomicInteger implements Runnable + { + final ExecutorService executor; + + // the AtomicInteger we extend always contains some combination of these bit flags, representing our current run state + + /** Not running, and will not be scheduled again until transitioned to a new state */ + private static final int STOPPED = 0; + /** Currently executing (may only be scheduled to execute, or may be about to terminate); + * will stop at end of this run, without rescheduling */ + private static final int EXECUTING = 1; + /** Another execution has been requested; a new execution will begin some time after this state is taken */ + private static final int EXECUTE_AGAIN = 2; + /** We are currently executing and will submit another execution before we terminate */ + private static final int EXECUTING_AGAIN = EXECUTING | EXECUTE_AGAIN; + /** Will begin a new execution some time after this state is taken, but only once some condition is met. + * This state will initially be taken in tandem with EXECUTING, but if delivery completes without clearing + * the state, the condition will be held on its own until {@link #executeAgain} is invoked */ + private static final int WAITING_TO_EXECUTE = 4; + + /** + * Force all task execution to stop, once any currently in progress work is completed + */ + private volatile boolean terminated; + + /** + * Is there asynchronous delivery work in progress. + * + * This temporarily prevents any {@link #stopAndRun} work from being performed. + * Once both inProgress and stopAndRun are set we perform no more delivery work until one is unset, + * to ensure we eventually run stopAndRun. + * + * This should be updated and read only on the Delivery thread. + */ + private boolean inProgress = false; + + /** + * Request a task's execution while there is no delivery work in progress. + * + * This is to permit cleanly tearing down a connection without interrupting any messages that might be in flight. + * If stopAndRun is set, we should not enter doRun() until a corresponding setInProgress(false) occurs. + */ + final AtomicReference stopAndRun = new AtomicReference<>(); + + Delivery(ExecutorService executor) + { + this.executor = executor; + } + + /** + * Ensure that any messages or stopAndRun that were queued prior to this invocation will be seen by at least + * one future invocation of the delivery task, unless delivery has already been terminated. + */ + public void execute() + { + if (get() < EXECUTE_AGAIN && STOPPED == getAndUpdate(i -> i == STOPPED ? EXECUTING: i | EXECUTE_AGAIN)) + executor.execute(this); + } + + private boolean isExecuting(int state) + { + return 0 != (state & EXECUTING); + } + + /** + * This method is typically invoked after WAITING_TO_EXECUTE is set. + * + * However WAITING_TO_EXECUTE does not need to be set; all this method needs to ensure is that + * delivery unconditionally performs one new execution promptly. + */ + void executeAgain() + { + // if we are already executing, set EXECUTING_AGAIN and leave scheduling to the currently running one. + // otherwise, set ourselves unconditionally to EXECUTING and schedule ourselves immediately + if (!isExecuting(getAndUpdate(i -> !isExecuting(i) ? EXECUTING : EXECUTING_AGAIN))) + executor.execute(this); + } + + /** + * Invoke this when we cannot make further progress now, but we guarantee that we will execute later when we can. + * This simply communicates to {@link #run} that we should not schedule ourselves again, just unset the EXECUTING bit. + */ + void promiseToExecuteLater() + { + set(EXECUTING | WAITING_TO_EXECUTE); + } + + /** + * Called when exiting {@link #run} to schedule another run if necessary. + * + * If we are currently executing, we only reschedule if the present state is EXECUTING_AGAIN. + * If this is the case, we clear the EXECUTE_AGAIN bit (setting ourselves to EXECUTING), and reschedule. + * Otherwise, we clear the EXECUTING bit and terminate, which will set us to either STOPPED or WAITING_TO_EXECUTE + * (or possibly WAITING_TO_EXECUTE | EXECUTE_AGAIN, which is logically the same as WAITING_TO_EXECUTE) + */ + private void maybeExecuteAgain() + { + if (EXECUTING_AGAIN == getAndUpdate(i -> i == EXECUTING_AGAIN ? EXECUTING : (i & ~EXECUTING))) + executor.execute(this); + } + + /** + * No more tasks or delivery will be executed, once any in progress complete. + */ + public void terminate() + { + terminated = true; + } + + /** + * Only to be invoked by the Delivery task. + * + * If true, indicates that we have begun asynchronous delivery work, so that + * we cannot safely stopAndRun until it completes. + * + * Once it completes, we ensure any stopAndRun task has a chance to execute + * by ensuring delivery is scheduled. + * + * If stopAndRun is also set, we should not enter doRun() until a corresponding + * setInProgress(false) occurs. + */ + void setInProgress(boolean inProgress) + { + boolean wasInProgress = this.inProgress; + this.inProgress = inProgress; + if (!inProgress && wasInProgress) + executeAgain(); + } + + /** + * Perform some delivery work. + * + * Must never be invoked directly, only via {@link #execute()} + */ + public void run() + { + /* do/while handling setup for {@link #doRun()}, and repeat invocations thereof */ + while (true) + { + if (terminated) + return; + + if (null != stopAndRun.get()) + { + // if we have an external request to perform, attempt it - if no async delivery is in progress + + if (inProgress) + { + // if we are in progress, we cannot do anything; + // so, exit and rely on setInProgress(false) executing us + // (which must happen later, since it must happen on this thread) + promiseToExecuteLater(); + break; + } + + stopAndRun.getAndSet(null).run(); + } + + State state = OutboundConnection.this.state; + if (!state.isEstablished() || !state.established().isConnected()) + { + // if we have messages yet to deliver, or a task to run, we need to reconnect and try again + // we try to reconnect before running another stopAndRun so that we do not infinite loop in close + if (hasPending() || null != stopAndRun.get()) + { + promiseToExecuteLater(); + requestConnect().addListener(f -> executeAgain()); + } + break; + } + + if (!doRun(state.established())) + break; + } + + maybeExecuteAgain(); + } + + /** + * @return true if we should run again immediately; + * always false for eventLoop executor, as want to service other channels + */ + abstract boolean doRun(Established established); + + /** + * Schedule a task to run later on the delivery thread while delivery is not in progress, + * i.e. there are no bytes in flight to the network buffer. + * + * Does not guarantee to run promptly if there is no current connection to the remote host. + * May wait until a new connection is established, or a connection timeout elapses, before executing. + * + * Update the shared atomic property containing work we want to interrupt message processing to perform, + * the invoke schedule() to be certain it gets run. + */ + void stopAndRun(Runnable run) + { + stopAndRun.accumulateAndGet(run, OutboundConnection::andThen); + execute(); + } + + /** + * Schedule a task to run on the eventLoop, guaranteeing that delivery will not occur while the task is performed. + */ + abstract void stopAndRunOnEventLoop(Runnable run); + + } + + /** + * Delivery that runs entirely on the eventLoop + * + * Since this has single threaded access to most of its environment, it can be simple and efficient, however + * it must also have bounded run time, and limit its resource consumption to ensure other channels serviced by the + * eventLoop can also make progress. + * + * This operates on modest buffers, no larger than the {@link OutboundConnections#LARGE_MESSAGE_THRESHOLD} and + * filling at most one at a time before writing (potentially asynchronously) to the socket. + * + * We track the number of bytes we have in flight, ensuring no more than a user-defined maximum at any one time. + */ + class EventLoopDelivery extends Delivery + { + private int flushingBytes; + private boolean isWritable = true; + + EventLoopDelivery() + { + super(eventLoop); + } + + /** + * {@link Delivery#doRun} + * + * Since we are on the eventLoop, in order to ensure other channels are serviced + * we never return true to request another run immediately. + * + * If there is more work to be done, we submit ourselves for execution once the eventLoop has time. + */ + @SuppressWarnings("resource") + boolean doRun(Established established) + { + if (!isWritable) + return false; + + // pendingBytes is updated before queue.size() (which triggers notEmpty, and begins delivery), + // so it is safe to use it here to exit delivery + // this number is inaccurate for old versions, but we don't mind terribly - we'll send at least one message, + // and get round to it eventually (though we could add a fudge factor for some room for older versions) + int maxSendBytes = (int) min(pendingBytes() - flushingBytes, LARGE_MESSAGE_THRESHOLD); + if (maxSendBytes == 0) + return false; + + OutboundConnectionSettings settings = established.settings; + int messagingVersion = established.messagingVersion; + + FrameEncoder.Payload sending = null; + int canonicalSize = 0; // number of bytes we must use for our resource accounting + int sendingBytes = 0; + int sendingCount = 0; + try (OutboundMessageQueue.WithLock withLock = queue.lockOrCallback(approxTime.now(), this::execute)) + { + if (withLock == null) + return false; // we failed to acquire the queue lock, so return; we will be scheduled again when the lock is available + + sending = established.payloadAllocator.allocate(true, maxSendBytes); + DataOutputBufferFixed out = new DataOutputBufferFixed(sending.buffer); + + Message next; + while ( null != (next = withLock.peek()) ) + { + try + { + int messageSize = next.serializedSize(messagingVersion); + + // actual message size for this version is larger than permitted maximum + if (messageSize > DatabaseDescriptor.getInternodeMaxMessageSizeInBytes()) + throw new Message.OversizedMessageException(messageSize); + + if (messageSize > sending.remaining()) + { + // if we don't have enough room to serialize the next message, we have either + // 1) run out of room after writing some messages successfully; this might mean that we are + // overflowing our highWaterMark, or that we have just filled our buffer + // 2) we have a message that is too large for this connection; this can happen if a message's + // size was calculated for the wrong messaging version when enqueued. + // In this case we want to write it anyway, so simply allocate a large enough buffer. + + if (sendingBytes > 0) + break; + + sending.release(); + sending = null; // set to null to prevent double-release if we fail to allocate our new buffer + sending = established.payloadAllocator.allocate(true, messageSize); + //noinspection IOResourceOpenedButNotSafelyClosed + out = new DataOutputBufferFixed(sending.buffer); + } + + Tracing.instance.traceOutgoingMessage(next, settings.connectTo); + Message.serializer.serialize(next, out, messagingVersion); + + if (sending.length() != sendingBytes + messageSize) + throw new InvalidSerializedSizeException(next.verb(), messageSize, sending.length() - sendingBytes); + + canonicalSize += canonicalSize(next); + sendingCount += 1; + sendingBytes += messageSize; + } + catch (Throwable t) + { + onFailedSerialize(next, messagingVersion, 0, t); + + assert sending != null; + // reset the buffer to ignore the message we failed to serialize + sending.trim(sendingBytes); + } + withLock.removeHead(next); + } + if (0 == sendingBytes) + return false; + + sending.finish(); + debug.onSendSmallFrame(sendingCount, sendingBytes); + ChannelFuture flushResult = AsyncChannelPromise.writeAndFlush(established.channel, sending); + sending = null; + + if (flushResult.isSuccess()) + { + sentCount += sendingCount; + sentBytes += sendingBytes; + debug.onSentSmallFrame(sendingCount, sendingBytes); + } + else + { + flushingBytes += canonicalSize; + setInProgress(true); + + boolean hasOverflowed = flushingBytes >= settings.flushHighWaterMark; + if (hasOverflowed) + { + isWritable = false; + promiseToExecuteLater(); + } + + int releaseBytesFinal = canonicalSize; + int sendingBytesFinal = sendingBytes; + int sendingCountFinal = sendingCount; + flushResult.addListener(future -> { + + releaseCapacity(sendingCountFinal, releaseBytesFinal); + flushingBytes -= releaseBytesFinal; + if (flushingBytes == 0) + setInProgress(false); + + if (!isWritable && flushingBytes <= settings.flushLowWaterMark) + { + isWritable = true; + executeAgain(); + } + + if (future.isSuccess()) + { + sentCount += sendingCountFinal; + sentBytes += sendingBytesFinal; + debug.onSentSmallFrame(sendingCountFinal, sendingBytesFinal); + } + else + { + errorCount += sendingCountFinal; + errorBytes += sendingBytesFinal; + invalidateChannel(established, future.cause()); + debug.onFailedSmallFrame(sendingCountFinal, sendingBytesFinal); + } + }); + canonicalSize = 0; + } + } + catch (Throwable t) + { + errorCount += sendingCount; + errorBytes += sendingBytes; + invalidateChannel(established, t); + } + finally + { + if (canonicalSize > 0) + releaseCapacity(sendingCount, canonicalSize); + + if (sending != null) + sending.release(); + + if (pendingBytes() > flushingBytes && isWritable) + execute(); + } + + return false; + } + + void stopAndRunOnEventLoop(Runnable run) + { + stopAndRun(run); + } + } + + /** + * Delivery that coordinates between the eventLoop and another (non-dedicated) thread + * + * This is to service messages that are too large to fully serialize on the eventLoop, as they could block + * prompt service of other requests. Since our serializers assume blocking IO, the easiest approach is to + * ensure a companion thread performs blocking IO that, under the hood, is serviced by async IO on the eventLoop. + * + * Most of the work here is handed off to {@link AsyncChannelOutputPlus}, with our main job being coordinating + * when and what we should run. + * + * To avoid allocating a huge number of threads across a cluster, we utilise the shared methods of {@link Delivery} + * to ensure that only one run() is actually scheduled to run at a time - this permits us to use any {@link ExecutorService} + * as a backing, with the number of threads defined only by the maximum concurrency needed to deliver all large messages. + * We use a shared caching {@link java.util.concurrent.ThreadPoolExecutor}, and rename the Threads that service + * our connection on entry and exit. + */ + class LargeMessageDelivery extends Delivery + { + static final int DEFAULT_BUFFER_SIZE = 32 * 1024; + + LargeMessageDelivery(ExecutorService executor) + { + super(executor); + } + + /** + * A simple wrapper of {@link Delivery#run} to set the current Thread name for the duration of its execution. + */ + public void run() + { + String threadName, priorThreadName = null; + try + { + priorThreadName = Thread.currentThread().getName(); + threadName = "Messaging-OUT-" + template.from() + "->" + template.to + '-' + type; + Thread.currentThread().setName(threadName); + + super.run(); + } + finally + { + if (priorThreadName != null) + Thread.currentThread().setName(priorThreadName); + } + } + + @SuppressWarnings({ "resource", "RedundantSuppression" }) // make eclipse warnings go away + boolean doRun(Established established) + { + Message send = queue.tryPoll(approxTime.now(), this::execute); + if (send == null) + return false; + + AsyncMessageOutputPlus out = null; + try + { + int messageSize = send.serializedSize(established.messagingVersion); + out = new AsyncMessageOutputPlus(established.channel, DEFAULT_BUFFER_SIZE, messageSize, established.payloadAllocator); + // actual message size for this version is larger than permitted maximum + if (messageSize > DatabaseDescriptor.getInternodeMaxMessageSizeInBytes()) + throw new Message.OversizedMessageException(messageSize); + + Tracing.instance.traceOutgoingMessage(send, established.settings.connectTo); + Message.serializer.serialize(send, out, established.messagingVersion); + + if (out.position() != messageSize) + throw new InvalidSerializedSizeException(send.verb(), messageSize, out.position()); + + out.close(); + sentCount += 1; + sentBytes += messageSize; + releaseCapacity(1, canonicalSize(send)); + return hasPending(); + } + catch (Throwable t) + { + boolean tryAgain = true; + + if (out != null) + { + out.discard(); + if (out.flushed() > 0 || + isCausedBy(t, cause -> isConnectionReset(cause) + || cause instanceof Errors.NativeIoException + || cause instanceof AsyncChannelOutputPlus.FlushException)) + { + // close the channel, and wait for eventLoop to execute + disconnectNow(established).awaitUninterruptibly(); + tryAgain = false; + try + { + // after closing, wait until we are signalled about the in flight writes; + // this ensures flushedToNetwork() is correct below + out.waitUntilFlushed(0, 0); + } + catch (Throwable ignore) + { + // irrelevant + } + } + } + + onFailedSerialize(send, established.messagingVersion, out == null ? 0 : (int) out.flushedToNetwork(), t); + return tryAgain; + } + } + + void stopAndRunOnEventLoop(Runnable run) + { + stopAndRun(() -> { + try + { + runOnEventLoop(run).await(); + } + catch (InterruptedException e) + { + throw new RuntimeException(e); + } + }); + } + } + + /* + * Size used for capacity enforcement purposes. Using current messaging version no matter what the peer's version is. + */ + private int canonicalSize(Message message) + { + return message.serializedSize(current_version); + } + + private void invalidateChannel(Established established, Throwable cause) + { + JVMStabilityInspector.inspectThrowable(cause, false); + + if (state != established) + return; // do nothing; channel already invalidated + + if (isCausedByConnectionReset(cause)) + logger.info("{} channel closed by provider", id(), cause); + else + logger.error("{} channel in potentially inconsistent state after error; closing", id(), cause); + + disconnectNow(established); + } + + /** + * Attempt to open a new channel to the remote endpoint. + * + * Most of the actual work is performed by OutboundConnectionInitiator, this method just manages + * our book keeping on either success or failure. + * + * This method is only to be invoked by the eventLoop, and the inner class' methods should only be evaluated by the eventtLoop + */ + Future initiate() + { + class Initiate + { + /** + * If we fail to connect, we want to try and connect again before any messages timeout. + * However, we update this each time to ensure we do not retry unreasonably often, and settle on a periodicity + * that might lead to timeouts in some aggressive systems. + */ + long retryRateMillis = DatabaseDescriptor.getMinRpcTimeout(MILLISECONDS) / 2; + + // our connection settings, possibly updated on retry + int messagingVersion = template.endpointToVersion().get(template.to); + OutboundConnectionSettings settings; + + /** + * If we failed for any reason, try again + */ + void onFailure(Throwable cause) + { + if (cause instanceof ConnectException) + noSpamLogger.info("{} failed to connect", id(), cause); + else + noSpamLogger.error("{} failed to connect", id(), cause); + + JVMStabilityInspector.inspectThrowable(cause, false); + + if (hasPending()) + { + Promise> result = new AsyncPromise<>(eventLoop); + state = new Connecting(state.disconnected(), result, eventLoop.schedule(() -> attempt(result), max(100, retryRateMillis), MILLISECONDS)); + retryRateMillis = min(1000, retryRateMillis * 2); + } + else + { + // this Initiate will be discarded + state = Disconnected.dormant(state.disconnected().maintenance); + } + } + + void onCompletedHandshake(Result result) + { + switch (result.outcome) + { + case SUCCESS: + // it is expected that close, if successful, has already cancelled us; so we do not need to worry about leaking connections + assert !state.isClosed(); + + MessagingSuccess success = result.success(); + debug.onConnect(success.messagingVersion, settings); + state.disconnected().maintenance.cancel(false); + + FrameEncoder.PayloadAllocator payloadAllocator = success.allocator; + Channel channel = success.channel; + Established established = new Established(messagingVersion, channel, payloadAllocator, settings); + state = established; + channel.pipeline().addLast("handleExceptionalStates", new ChannelInboundHandlerAdapter() { + @Override + public void channelInactive(ChannelHandlerContext ctx) + { + disconnectNow(established); + ctx.fireChannelInactive(); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) + { + try + { + invalidateChannel(established, cause); + } + catch (Throwable t) + { + logger.error("Unexpected exception in {}.exceptionCaught", this.getClass().getSimpleName(), t); + } + } + }); + ++successfulConnections; + + logger.info("{} successfully connected, version = {}, framing = {}, encryption = {}", + id(true), + success.messagingVersion, + settings.framing, + encryptionLogStatement(settings.encryption)); + break; + + case RETRY: + if (logger.isTraceEnabled()) + logger.trace("{} incorrect legacy peer version predicted; reconnecting", id()); + + // the messaging version we connected with was incorrect; try again with the one supplied by the remote host + messagingVersion = result.retry().withMessagingVersion; + settings.endpointToVersion.set(settings.to, messagingVersion); + + initiate(); + break; + + case INCOMPATIBLE: + // we cannot communicate with this peer given its messaging version; mark this as any other failure, and continue trying + Throwable t = new IOException(String.format("Incompatible peer: %s, messaging version: %s", + settings.to, result.incompatible().maxMessagingVersion)); + t.fillInStackTrace(); + onFailure(t); + break; + + default: + throw new AssertionError(); + } + } + + /** + * Initiate all the actions required to establish a working, valid connection. This includes + * opening the socket, negotiating the internode messaging handshake, and setting up the working + * Netty {@link Channel}. However, this method will not block for all those actions: it will only + * kick off the connection attempt, setting the @{link #connecting} future to track its completion. + * + * Note: this should only be invoked on the event loop. + */ + private void attempt(Promise> result) + { + ++connectionAttempts; + + settings = template; + if (messagingVersion > settings.acceptVersions.max) + messagingVersion = settings.acceptVersions.max; + + // ensure we connect to the correct SSL port + settings = settings.withLegacyPortIfNecessary(messagingVersion); + + initiateMessaging(eventLoop, type, settings, messagingVersion, result) + .addListener(future -> { + if (future.isCancelled()) + return; + if (future.isSuccess()) //noinspection unchecked + onCompletedHandshake((Result) future.getNow()); + else + onFailure(future.cause()); + }); + } + + Future> initiate() + { + Promise> result = new AsyncPromise<>(eventLoop); + state = new Connecting(state.disconnected(), result); + attempt(result); + return result; + } + } + + return new Initiate().initiate(); + } + + /** + * Returns a future that completes when we are _maybe_ reconnected. + * + * The connection attempt is guaranteed to have completed (successfully or not) by the time any listeners are invoked, + * so if a reconnection attempt is needed, it is already scheduled. + */ + private Future requestConnect() + { + // we may race with updates to this variable, but this is fine, since we only guarantee that we see a value + // that did at some point represent an active connection attempt - if it is stale, it will have been completed + // and the caller can retry (or utilise the successfully established connection) + { + State state = this.state; + if (state.isConnecting()) + return state.connecting().attempt; + } + + Promise promise = AsyncPromise.uncancellable(eventLoop); + runOnEventLoop(() -> { + if (isClosed()) // never going to connect + { + promise.tryFailure(new ClosedChannelException()); + } + else if (state.isEstablished() && state.established().isConnected()) // already connected + { + promise.trySuccess(null); + } + else + { + if (state.isEstablished()) + setDisconnected(); + + if (!state.isConnecting()) + { + assert eventLoop.inEventLoop(); + assert !isConnected(); + initiate().addListener(new PromiseNotifier<>(promise)); + } + else + { + state.connecting().attempt.addListener(new PromiseNotifier<>(promise)); + } + } + }); + return promise; + } + + /** + * Change the IP address on which we connect to the peer. We will attempt to connect to the new address if there + * was a previous connection, and new incoming messages as well as existing {@link #queue} messages will be sent there. + * Any outstanding messages in the existing channel will still be sent to the previous address (we won't/can't move them from + * one channel to another). + * + * Returns null if the connection is closed. + */ + Future reconnectWith(OutboundConnectionSettings reconnectWith) + { + OutboundConnectionSettings newTemplate = reconnectWith.withDefaults(ConnectionCategory.MESSAGING); + if (newTemplate.socketFactory != template.socketFactory) throw new IllegalArgumentException(); + if (newTemplate.callbacks != template.callbacks) throw new IllegalArgumentException(); + if (!Objects.equals(newTemplate.applicationSendQueueCapacityInBytes, template.applicationSendQueueCapacityInBytes)) throw new IllegalArgumentException(); + if (!Objects.equals(newTemplate.applicationSendQueueReserveEndpointCapacityInBytes, template.applicationSendQueueReserveEndpointCapacityInBytes)) throw new IllegalArgumentException(); + if (newTemplate.applicationSendQueueReserveGlobalCapacityInBytes != template.applicationSendQueueReserveGlobalCapacityInBytes) throw new IllegalArgumentException(); + + logger.info("{} updating connection settings", id()); + + Promise done = AsyncPromise.uncancellable(eventLoop); + delivery.stopAndRunOnEventLoop(() -> { + template = newTemplate; + // delivery will immediately continue after this, triggering a reconnect if necessary; + // this might mean a slight delay for large message delivery, as the connect will be scheduled + // asynchronously, so we must wait for a second turn on the eventLoop + if (state.isEstablished()) + { + disconnectNow(state.established()); + } + else if (state.isConnecting()) + { + // cancel any in-flight connection attempt and restart with new template + state.connecting().cancel(); + initiate(); + } + done.setSuccess(null); + }); + return done; + } + + /** + * Close any currently open connection, forcing a reconnect if there are messages outstanding + * (or leaving it closed for now otherwise) + */ + public boolean interrupt() + { + State state = this.state; + if (!state.isEstablished()) + return false; + + disconnectGracefully(state.established()); + return true; + } + + /** + * Schedule a safe close of the provided channel, if it has not already been closed. + * + * This means ensuring that delivery has stopped so that we do not corrupt or interrupt any + * in progress transmissions. + * + * The actual closing of the channel is performed asynchronously, to simplify our internal state management + * and promptly get the connection going again; the close is considered to have succeeded as soon as we + * have set our internal state. + */ + private void disconnectGracefully(Established closeIfIs) + { + // delivery will immediately continue after this, triggering a reconnect if necessary; + // this might mean a slight delay for large message delivery, as the connect will be scheduled + // asynchronously, so we must wait for a second turn on the eventLoop + delivery.stopAndRunOnEventLoop(() -> disconnectNow(closeIfIs)); + } + + /** + * The channel is already known to be invalid, so there's no point waiting for a clean break in delivery. + * + * Delivery will be executed again as soon as we have logically closed the channel; we do not wait + * for the channel to actually be closed. + * + * The Future returned _does_ wait for the channel to be completely closed, so that callers can wait to be sure + * all writes have been completed either successfully or not. + */ + private Future disconnectNow(Established closeIfIs) + { + return runOnEventLoop(() -> { + if (state == closeIfIs) + { + // no need to wait until the channel is closed to set ourselves as disconnected (and potentially open a new channel) + setDisconnected(); + if (hasPending()) + delivery.execute(); + closeIfIs.channel.close() + .addListener(future -> { + if (!future.isSuccess()) + logger.info("Problem closing channel {}", closeIfIs, future.cause()); + }); + } + }); + } + + /** + * Schedules regular cleaning of the connection's state while it is disconnected from its remote endpoint. + * + * To be run only by the eventLoop or in the constructor + */ + private void setDisconnected() + { + assert state == null || state.isEstablished(); + state = Disconnected.dormant(eventLoop.scheduleAtFixedRate(queue::maybePruneExpired, 100L, 100L, TimeUnit.MILLISECONDS)); + } + + /** + * Schedule this connection to be permanently closed; only one close may be scheduled, + * any future scheduled closes are referred to the original triggering one (which may have a different schedule) + */ + Future scheduleClose(long time, TimeUnit unit, boolean flushQueue) + { + Promise scheduledClose = AsyncPromise.uncancellable(eventLoop); + if (!scheduledCloseUpdater.compareAndSet(this, null, scheduledClose)) + return this.scheduledClose; + + eventLoop.schedule(() -> close(flushQueue).addListener(new PromiseNotifier<>(scheduledClose)), time, unit); + return scheduledClose; + } + + /** + * Permanently close this connection. + * + * Immediately prevent any new messages from being enqueued - these will throw ClosedChannelException. + * The close itself happens asynchronously on the eventLoop, so a Future is returned to help callers + * wait for its completion. + * + * The flushQueue parameter indicates if any outstanding messages should be delivered before closing the connection. + * + * - If false, any already flushed or in-progress messages are completed, and the remaining messages are cleared + * before the connection is promptly torn down. + * + * - If true, we attempt delivery of all queued messages. If necessary, we will continue to open new connections + * to the remote host until they have been delivered. Only if we continue to fail to open a connection for + * an extended period of time will we drop any outstanding messages and close the connection. + */ + public Future close(boolean flushQueue) + { + // ensure only one close attempt can be in flight + Promise closing = AsyncPromise.uncancellable(eventLoop); + if (!closingUpdater.compareAndSet(this, null, closing)) + return this.closing; + + /* + * Now define a cleanup closure, that will be deferred until it is safe to do so. + * Once run it: + * - immediately _logically_ closes the channel by updating this object's fields, but defers actually closing + * - cancels any in-flight connection attempts + * - cancels any maintenance work that might be scheduled + * - clears any waiting messages on the queue + * - terminates the delivery thread + * - finally, schedules any open channel's closure, and propagates its completion to the close promise + */ + Runnable eventLoopCleanup = () -> { + Runnable onceNotConnecting = () -> { + // start by setting ourselves to definitionally closed + State state = this.state; + this.state = State.CLOSED; + + try + { + // note that we never clear the queue, to ensure that an enqueue has the opportunity to remove itself + // if it raced with close, to potentially requeue the message on a replacement connection + + // we terminate delivery here, to ensure that any listener to {@link connecting} do not schedule more work + delivery.terminate(); + + // stop periodic cleanup + if (state.isDisconnected()) + { + state.disconnected().maintenance.cancel(true); + closing.setSuccess(null); + } + else + { + assert state.isEstablished(); + state.established().channel.close() + .addListener(new PromiseNotifier<>(closing)); + } + } + catch (Throwable t) + { + // in case of unexpected exception, signal completion and try to close the channel + closing.trySuccess(null); + try + { + if (state.isEstablished()) + state.established().channel.close(); + } + catch (Throwable t2) + { + t.addSuppressed(t2); + logger.error("Failed to close connection cleanly:", t); + } + throw t; + } + }; + + if (state.isConnecting()) + { + // stop any in-flight connection attempts; these should be running on the eventLoop, so we should + // be able to cleanly cancel them, but executing on a listener guarantees correct semantics either way + Connecting connecting = state.connecting(); + connecting.cancel(); + connecting.attempt.addListener(future -> onceNotConnecting.run()); + } + else + { + onceNotConnecting.run(); + } + }; + + /* + * If we want to shutdown gracefully, flushing any outstanding messages, we have to do it very carefully. + * Things to note: + * + * - It is possible flushing messages will require establishing a new connection + * (However, if a connection cannot be established, we do not want to keep trying) + * - We have to negotiate with a separate thread, so we must be sure it is not in-progress before we stop (like channel close) + * - Cleanup must still happen on the eventLoop + * + * To achieve all of this, we schedule a recurring operation on the delivery thread, executing while delivery + * is between messages, that checks if the queue is empty; if it is, it schedules cleanup on the eventLoop. + */ + + Runnable clearQueue = () -> + { + CountDownLatch done = new CountDownLatch(1); + queue.runEventually(withLock -> { + withLock.consume(this::onClosed); + done.countDown(); + }); + //noinspection UnstableApiUsage + Uninterruptibles.awaitUninterruptibly(done); + }; + + if (flushQueue) + { + // just keep scheduling on delivery executor a check to see if we're done; there should always be one + // delivery attempt between each invocation, unless there is a wider problem with delivery scheduling + class FinishDelivery implements Runnable + { + public void run() + { + if (!hasPending()) + delivery.stopAndRunOnEventLoop(eventLoopCleanup); + else + delivery.stopAndRun(() -> { + if (state.isConnecting() && state.connecting().isFailingToConnect) + clearQueue.run(); + run(); + }); + } + } + + delivery.stopAndRun(new FinishDelivery()); + } + else + { + delivery.stopAndRunOnEventLoop(() -> { + clearQueue.run(); + eventLoopCleanup.run(); + }); + } + + return closing; + } + + /** + * Run the task immediately if we are the eventLoop, otherwise queue it for execution on the eventLoop. + */ + private Future runOnEventLoop(Runnable runnable) + { + if (!eventLoop.inEventLoop()) + return eventLoop.submit(runnable); + + runnable.run(); + return new SucceededFuture<>(eventLoop, null); + } + + public boolean isConnected() + { + State state = this.state; + return state.isEstablished() && state.established().isConnected(); + } + + boolean isClosing() + { + return closing != null; + } + + boolean isClosed() + { + return state.isClosed(); + } + + private String id(boolean includeReal) + { + State state = this.state; + if (!includeReal || !state.isEstablished()) + return id(); + Established established = state.established(); + Channel channel = established.channel; + OutboundConnectionSettings settings = established.settings; + return SocketFactory.channelId(settings.from, (InetSocketAddress) channel.remoteAddress(), + settings.to, (InetSocketAddress) channel.localAddress(), + type, channel.id().asShortText()); + } + + private String id() + { + State state = this.state; + Channel channel = null; + OutboundConnectionSettings settings = template; + if (state.isEstablished()) + { + channel = state.established().channel; + settings = state.established().settings; + } + String channelId = channel != null ? channel.id().asShortText() : "[no-channel]"; + return SocketFactory.channelId(settings.from(), settings.to, type, channelId); + } + + @Override + public String toString() + { + return id(); + } + + public boolean hasPending() + { + return 0 != pendingCountAndBytes; + } + + public int pendingCount() + { + return pendingCount(pendingCountAndBytes); + } + + public long pendingBytes() + { + return pendingBytes(pendingCountAndBytes); + } + + public long sentCount() + { + // not volatile, but shouldn't matter + return sentCount; + } + + public long sentBytes() + { + // not volatile, but shouldn't matter + return sentBytes; + } + + public long submittedCount() + { + // not volatile, but shouldn't matter + return submittedCount; + } + + public long dropped() + { + return overloadedCount + expiredCount; + } + + public long overloadedBytes() + { + return overloadedBytes; + } + + public long overloadedCount() + { + return overloadedCount; + } + + public long expiredCount() + { + return expiredCount; + } + + public long expiredBytes() + { + return expiredBytes; + } + + public long errorCount() + { + return errorCount; + } + + public long errorBytes() + { + return errorBytes; + } + + public long successfulConnections() + { + return successfulConnections; + } + + public long connectionAttempts() + { + return connectionAttempts; + } + + private static Runnable andThen(Runnable a, Runnable b) + { + if (a == null || b == null) + return a == null ? b : a; + return () -> { a.run(); b.run(); }; + } + + @VisibleForTesting + public ConnectionType type() + { + return type; + } + + @VisibleForTesting + OutboundConnectionSettings settings() + { + State state = this.state; + return state.isEstablished() ? state.established().settings : template; + } + + @VisibleForTesting + int messagingVersion() + { + State state = this.state; + return state.isEstablished() ? state.established().messagingVersion + : template.endpointToVersion().get(template.to); + } + + @VisibleForTesting + void unsafeRunOnDelivery(Runnable run) + { + delivery.stopAndRun(run); + } + + @VisibleForTesting + Channel unsafeGetChannel() + { + State state = this.state; + return state.isEstablished() ? state.established().channel : null; + } + + @VisibleForTesting + boolean unsafeAcquireCapacity(long amount) + { + return SUCCESS == acquireCapacity(amount); + } + + @VisibleForTesting + boolean unsafeAcquireCapacity(long count, long amount) + { + return SUCCESS == acquireCapacity(count, amount); + } + + @VisibleForTesting + void unsafeReleaseCapacity(long amount) + { + releaseCapacity(1, amount); + } + + Limit unsafeGetEndpointReserveLimits() + { + return reserveCapacityInBytes.endpoint; + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java new file mode 100644 index 000000000000..a63ccf92858b --- /dev/null +++ b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java @@ -0,0 +1,469 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.channels.ClosedChannelException; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandler; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.EventLoop; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.ByteToMessageDecoder; + +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslHandler; +import io.netty.util.concurrent.FailedFuture; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.Promise; +import io.netty.util.concurrent.ScheduledFuture; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.HandshakeProtocol.Initiate; +import org.apache.cassandra.net.OutboundConnectionInitiator.Result.MessagingSuccess; +import org.apache.cassandra.net.OutboundConnectionInitiator.Result.StreamingSuccess; +import org.apache.cassandra.security.SSLFactory; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.memory.BufferPool; + +import static java.util.concurrent.TimeUnit.*; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.HandshakeProtocol.*; +import static org.apache.cassandra.net.ConnectionType.STREAMING; +import static org.apache.cassandra.net.OutboundConnectionInitiator.Result.incompatible; +import static org.apache.cassandra.net.OutboundConnectionInitiator.Result.messagingSuccess; +import static org.apache.cassandra.net.OutboundConnectionInitiator.Result.retry; +import static org.apache.cassandra.net.OutboundConnectionInitiator.Result.streamingSuccess; +import static org.apache.cassandra.net.SocketFactory.*; + +/** + * A {@link ChannelHandler} to execute the send-side of the internode handshake protocol. + * As soon as the handler is added to the channel via {@link ChannelInboundHandler#channelActive(ChannelHandlerContext)} + * (which is only invoked if the underlying TCP connection was properly established), the {@link Initiate} + * handshake is sent. See {@link HandshakeProtocol} for full details. + *

+ * Upon completion of the handshake (on success or fail), the {@link #resultPromise} is completed. + * See {@link Result} for details about the different result states. + *

+ * This class extends {@link ByteToMessageDecoder}, which is a {@link ChannelInboundHandler}, because this handler + * waits for the peer's handshake response (the {@link Accept} of the internode messaging handshake protocol). + */ +public class OutboundConnectionInitiator +{ + private static final Logger logger = LoggerFactory.getLogger(OutboundConnectionInitiator.class); + + private final ConnectionType type; + private final OutboundConnectionSettings settings; + private final int requestMessagingVersion; // for pre40 nodes + private final Promise> resultPromise; + + private OutboundConnectionInitiator(ConnectionType type, OutboundConnectionSettings settings, + int requestMessagingVersion, Promise> resultPromise) + { + this.type = type; + this.requestMessagingVersion = requestMessagingVersion; + this.settings = settings; + this.resultPromise = resultPromise; + } + + /** + * Initiate a connection with the requested messaging version. + * if the other node supports a newer version, or doesn't support this version, we will fail to connect + * and try again with the version they reported + * + * The returned {@code Future} is guaranteed to be completed on the supplied eventLoop. + */ + public static Future> initiateStreaming(EventLoop eventLoop, OutboundConnectionSettings settings, int requestMessagingVersion) + { + return new OutboundConnectionInitiator(STREAMING, settings, requestMessagingVersion, new AsyncPromise<>(eventLoop)) + .initiate(eventLoop); + } + + /** + * Initiate a connection with the requested messaging version. + * if the other node supports a newer version, or doesn't support this version, we will fail to connect + * and try again with the version they reported + * + * The returned {@code Future} is guaranteed to be completed on the supplied eventLoop. + */ + static Future> initiateMessaging(EventLoop eventLoop, ConnectionType type, OutboundConnectionSettings settings, int requestMessagingVersion, Promise> result) + { + return new OutboundConnectionInitiator<>(type, settings, requestMessagingVersion, result) + .initiate(eventLoop); + } + + private Future> initiate(EventLoop eventLoop) + { + if (logger.isTraceEnabled()) + logger.trace("creating outbound bootstrap to {}, requestVersion: {}", settings, requestMessagingVersion); + + if (!settings.authenticate()) + { + // interrupt other connections, so they must attempt to re-authenticate + MessagingService.instance().interruptOutbound(settings.to); + return new FailedFuture<>(eventLoop, new IOException("authentication failed to " + settings.to)); + } + + // this is a bit ugly, but is the easiest way to ensure that if we timeout we can propagate a suitable error message + // and still guarantee that, if on timing out we raced with success, the successfully created channel is handled + AtomicBoolean timedout = new AtomicBoolean(); + Future bootstrap = createBootstrap(eventLoop) + .connect() + .addListener(future -> { + eventLoop.execute(() -> { + if (!future.isSuccess()) + { + if (future.isCancelled() && !timedout.get()) + resultPromise.cancel(true); + else if (future.isCancelled()) + resultPromise.tryFailure(new IOException("Timeout handshaking with " + settings.connectTo)); + else + resultPromise.tryFailure(future.cause()); + } + }); + }); + + ScheduledFuture timeout = eventLoop.schedule(() -> { + timedout.set(true); + bootstrap.cancel(false); + }, TIMEOUT_MILLIS, MILLISECONDS); + bootstrap.addListener(future -> timeout.cancel(true)); + + // Note that the bootstrap future's listeners may be invoked outside of the eventLoop, + // as Epoll failures on connection and disconnect may be run on the GlobalEventExecutor + // Since this FutureResult's listeners are all given to our resultPromise, they are guaranteed to be invoked by the eventLoop. + return new FutureResult<>(resultPromise, bootstrap); + } + + /** + * Create the {@link Bootstrap} for connecting to a remote peer. This method does not attempt to connect to the peer, + * and thus does not block. + */ + private Bootstrap createBootstrap(EventLoop eventLoop) + { + Bootstrap bootstrap = newBootstrap(eventLoop, settings.tcpUserTimeoutInMS) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, settings.tcpConnectTimeoutInMS) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.SO_REUSEADDR, true) + .option(ChannelOption.TCP_NODELAY, settings.tcpNoDelay) + .option(ChannelOption.MESSAGE_SIZE_ESTIMATOR, NoSizeEstimator.instance) + .handler(new Initializer()); + + if (settings.socketSendBufferSizeInBytes > 0) + bootstrap.option(ChannelOption.SO_SNDBUF, settings.socketSendBufferSizeInBytes); + + InetAddressAndPort remoteAddress = settings.connectTo; + bootstrap.remoteAddress(new InetSocketAddress(remoteAddress.address, remoteAddress.port)); + return bootstrap; + } + + private class Initializer extends ChannelInitializer + { + public void initChannel(SocketChannel channel) throws Exception + { + ChannelPipeline pipeline = channel.pipeline(); + + // order of handlers: ssl -> logger -> handshakeHandler + if (settings.withEncryption()) + { + // check if we should actually encrypt this connection + SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.CLIENT); + // for some reason channel.remoteAddress() will return null + InetAddressAndPort address = settings.to; + InetSocketAddress peer = settings.encryption.require_endpoint_verification ? new InetSocketAddress(address.address, address.port) : null; + SslHandler sslHandler = newSslHandler(channel, sslContext, peer); + logger.trace("creating outbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName()); + pipeline.addFirst("ssl", sslHandler); + } + + if (WIRETRACE) + pipeline.addLast("logger", new LoggingHandler(LogLevel.INFO)); + + pipeline.addLast("handshake", new Handler()); + } + + } + + private class Handler extends ByteToMessageDecoder + { + /** + * {@inheritDoc} + * + * Invoked when the channel is made active, and sends out the {@link Initiate}. + * In the case of streaming, we do not require a full bi-directional handshake; the initial message, + * containing the streaming protocol version, is all that is required. + */ + @Override + public void channelActive(final ChannelHandlerContext ctx) + { + Initiate msg = new Initiate(requestMessagingVersion, settings.acceptVersions, type, settings.framing, settings.from); + logger.trace("starting handshake with peer {}, msg = {}", settings.connectTo, msg); + AsyncChannelPromise.writeAndFlush(ctx, msg.encode(), + future -> { if (!future.isSuccess()) exceptionCaught(ctx, future.cause()); }); + + if (type.isStreaming() && requestMessagingVersion < VERSION_40) + ctx.pipeline().remove(this); + + ctx.fireChannelActive(); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception + { + super.channelInactive(ctx); + resultPromise.tryFailure(new ClosedChannelException()); + } + + /** + * {@inheritDoc} + * + * Invoked when we get the response back from the peer, which should contain the second message of the internode messaging handshake. + *

+ * If the peer's protocol version does not equal what we were expecting, immediately close the channel (and socket); + * do *not* send out the third message of the internode messaging handshake. + * We will reconnect on the appropriate protocol version. + */ + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) + { + try + { + Accept msg = Accept.maybeDecode(in, requestMessagingVersion); + if (msg == null) + return; + + int useMessagingVersion = msg.useMessagingVersion; + int peerMessagingVersion = msg.maxMessagingVersion; + logger.trace("received second handshake message from peer {}, msg = {}", settings.connectTo, msg); + + FrameEncoder frameEncoder = null; + Result result; + if (useMessagingVersion > 0) + { + if (useMessagingVersion < settings.acceptVersions.min || useMessagingVersion > settings.acceptVersions.max) + { + result = incompatible(useMessagingVersion, peerMessagingVersion); + } + else + { + // This is a bit ugly + if (type.isMessaging()) + { + switch (settings.framing) + { + case LZ4: + frameEncoder = FrameEncoderLZ4.fastInstance; + break; + case CRC: + frameEncoder = FrameEncoderCrc.instance; + break; + case UNPROTECTED: + frameEncoder = FrameEncoderUnprotected.instance; + break; + } + + result = (Result) messagingSuccess(ctx.channel(), useMessagingVersion, frameEncoder.allocator()); + } + else + { + result = (Result) streamingSuccess(ctx.channel(), useMessagingVersion); + } + } + } + else + { + assert type.isMessaging(); + + // pre40 handshake responses only (can be a post40 node) + if (peerMessagingVersion == requestMessagingVersion + || peerMessagingVersion > settings.acceptVersions.max) // this clause is for impersonating 3.0 node in testing only + { + switch (settings.framing) + { + case CRC: + case UNPROTECTED: + frameEncoder = FrameEncoderLegacy.instance; + break; + case LZ4: + frameEncoder = FrameEncoderLegacyLZ4.instance; + break; + } + + result = (Result) messagingSuccess(ctx.channel(), requestMessagingVersion, frameEncoder.allocator()); + } + else if (peerMessagingVersion < settings.acceptVersions.min) + result = incompatible(-1, peerMessagingVersion); + else + result = retry(peerMessagingVersion); + + if (result.isSuccess()) + { + ConfirmOutboundPre40 message = new ConfirmOutboundPre40(settings.acceptVersions.max, settings.from); + AsyncChannelPromise.writeAndFlush(ctx, message.encode()); + } + } + + ChannelPipeline pipeline = ctx.pipeline(); + if (result.isSuccess()) + { + BufferPool.setRecycleWhenFreeForCurrentThread(false); + if (type.isMessaging()) + { + assert frameEncoder != null; + pipeline.addLast("frameEncoder", frameEncoder); + } + pipeline.remove(this); + } + else + { + pipeline.close(); + } + + if (!resultPromise.trySuccess(result) && result.isSuccess()) + result.success().channel.close(); + } + catch (Throwable t) + { + exceptionCaught(ctx, t); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) + { + try + { + JVMStabilityInspector.inspectThrowable(cause, false); + resultPromise.tryFailure(cause); + if (isCausedByConnectionReset(cause)) + logger.info("Failed to connect to peer {}", settings.to, cause); + else + logger.error("Failed to handshake with peer {}", settings.to, cause); + ctx.close(); + } + catch (Throwable t) + { + logger.error("Unexpected exception in {}.exceptionCaught", this.getClass().getSimpleName(), t); + } + } + } + + /** + * The result of the handshake. Handshake has 3 possible outcomes: + * 1) it can be successful, in which case the channel and version to used is returned in this result. + * 2) we may decide to disconnect to reconnect with another protocol version (namely, the version is passed in this result). + * 3) we can have a negotiation failure for an unknown reason. (#sadtrombone) + */ + public static class Result + { + /** + * Describes the result of receiving the response back from the peer (Message 2 of the handshake) + * and implies an action that should be taken. + */ + enum Outcome + { + SUCCESS, RETRY, INCOMPATIBLE + } + + public static class Success extends Result + { + public final Channel channel; + public final int messagingVersion; + Success(Channel channel, int messagingVersion) + { + super(Outcome.SUCCESS); + this.channel = channel; + this.messagingVersion = messagingVersion; + } + } + + public static class StreamingSuccess extends Success + { + StreamingSuccess(Channel channel, int messagingVersion) + { + super(channel, messagingVersion); + } + } + + public static class MessagingSuccess extends Success + { + public final FrameEncoder.PayloadAllocator allocator; + MessagingSuccess(Channel channel, int messagingVersion, FrameEncoder.PayloadAllocator allocator) + { + super(channel, messagingVersion); + this.allocator = allocator; + } + } + + static class Retry extends Result + { + final int withMessagingVersion; + Retry(int withMessagingVersion) + { + super(Outcome.RETRY); + this.withMessagingVersion = withMessagingVersion; + } + } + + static class Incompatible extends Result + { + final int closestSupportedVersion; + final int maxMessagingVersion; + Incompatible(int closestSupportedVersion, int maxMessagingVersion) + { + super(Outcome.INCOMPATIBLE); + this.closestSupportedVersion = closestSupportedVersion; + this.maxMessagingVersion = maxMessagingVersion; + } + } + + final Outcome outcome; + + private Result(Outcome outcome) + { + this.outcome = outcome; + } + + boolean isSuccess() { return outcome == Outcome.SUCCESS; } + public SuccessType success() { return (SuccessType) this; } + static MessagingSuccess messagingSuccess(Channel channel, int messagingVersion, FrameEncoder.PayloadAllocator allocator) { return new MessagingSuccess(channel, messagingVersion, allocator); } + static StreamingSuccess streamingSuccess(Channel channel, int messagingVersion) { return new StreamingSuccess(channel, messagingVersion); } + + public Retry retry() { return (Retry) this; } + static Result retry(int withMessagingVersion) { return new Retry<>(withMessagingVersion); } + + public Incompatible incompatible() { return (Incompatible) this; } + static Result incompatible(int closestSupportedVersion, int maxMessagingVersion) { return new Incompatible(closestSupportedVersion, maxMessagingVersion); } + } + +} diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java b/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java new file mode 100644 index 000000000000..c78df61326c3 --- /dev/null +++ b/src/java/org/apache/cassandra/net/OutboundConnectionSettings.java @@ -0,0 +1,517 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; + +import io.netty.channel.WriteBufferWaterMark; +import org.apache.cassandra.auth.IInternodeAuthenticator; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.locator.IEndpointSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.config.DatabaseDescriptor.getEndpointSnitch; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.MessagingService.instance; +import static org.apache.cassandra.net.SocketFactory.encryptionLogStatement; +import static org.apache.cassandra.utils.FBUtilities.getBroadcastAddressAndPort; + +/** + * A collection of settings to be passed around for outbound connections. + */ +@SuppressWarnings({ "WeakerAccess", "unused" }) +public class OutboundConnectionSettings +{ + private static final String INTRADC_TCP_NODELAY_PROPERTY = Config.PROPERTY_PREFIX + "otc_intradc_tcp_nodelay"; + /** + * Enabled/disable TCP_NODELAY for intradc connections. Defaults to enabled. + */ + private static final boolean INTRADC_TCP_NODELAY = Boolean.parseBoolean(System.getProperty(INTRADC_TCP_NODELAY_PROPERTY, "true")); + + public enum Framing + { + // for < VERSION_40, implies no framing + // for >= VERSION_40, uses simple unprotected frames with header crc but no payload protection + UNPROTECTED(0), + // for < VERSION_40, uses the jpountz framing format + // for >= VERSION_40, uses our framing format with header crc24 + LZ4(1), + // for < VERSION_40, implies UNPROTECTED + // for >= VERSION_40, uses simple frames with separate header and payload crc + CRC(2); + + public static Framing forId(int id) + { + switch (id) + { + case 0: return UNPROTECTED; + case 1: return LZ4; + case 2: return CRC; + } + throw new IllegalStateException(); + } + + final int id; + Framing(int id) + { + this.id = id; + } + } + + public final IInternodeAuthenticator authenticator; + public final InetAddressAndPort to; + public final InetAddressAndPort connectTo; // may be represented by a different IP address on this node's local network + public final EncryptionOptions encryption; + public final Framing framing; + public final Integer socketSendBufferSizeInBytes; + public final Integer applicationSendQueueCapacityInBytes; + public final Integer applicationSendQueueReserveEndpointCapacityInBytes; + public final ResourceLimits.Limit applicationSendQueueReserveGlobalCapacityInBytes; + public final Boolean tcpNoDelay; + public final int flushLowWaterMark, flushHighWaterMark; + public final Integer tcpConnectTimeoutInMS; + public final Integer tcpUserTimeoutInMS; + public final AcceptVersions acceptVersions; + public final InetAddressAndPort from; + public final SocketFactory socketFactory; + public final OutboundMessageCallbacks callbacks; + public final OutboundDebugCallbacks debug; + public final EndpointMessagingVersions endpointToVersion; + + public OutboundConnectionSettings(InetAddressAndPort to) + { + this(to, null); + } + + public OutboundConnectionSettings(InetAddressAndPort to, InetAddressAndPort preferred) + { + this(null, to, preferred, null, null, null, null, null, null, null, 1 << 15, 1 << 16, null, null, null, null, null, null, null, null); + } + + private OutboundConnectionSettings(IInternodeAuthenticator authenticator, + InetAddressAndPort to, + InetAddressAndPort connectTo, + EncryptionOptions encryption, + Framing framing, + Integer socketSendBufferSizeInBytes, + Integer applicationSendQueueCapacityInBytes, + Integer applicationSendQueueReserveEndpointCapacityInBytes, + ResourceLimits.Limit applicationSendQueueReserveGlobalCapacityInBytes, + Boolean tcpNoDelay, + int flushLowWaterMark, + int flushHighWaterMark, + Integer tcpConnectTimeoutInMS, + Integer tcpUserTimeoutInMS, + AcceptVersions acceptVersions, + InetAddressAndPort from, + SocketFactory socketFactory, + OutboundMessageCallbacks callbacks, + OutboundDebugCallbacks debug, + EndpointMessagingVersions endpointToVersion) + { + Preconditions.checkArgument(socketSendBufferSizeInBytes == null || socketSendBufferSizeInBytes == 0 || socketSendBufferSizeInBytes >= 1 << 10, "illegal socket send buffer size: " + socketSendBufferSizeInBytes); + Preconditions.checkArgument(applicationSendQueueCapacityInBytes == null || applicationSendQueueCapacityInBytes >= 1 << 10, "illegal application send queue capacity: " + applicationSendQueueCapacityInBytes); + Preconditions.checkArgument(tcpUserTimeoutInMS == null || tcpUserTimeoutInMS >= 0, "tcp user timeout must be non negative: " + tcpUserTimeoutInMS); + Preconditions.checkArgument(tcpConnectTimeoutInMS == null || tcpConnectTimeoutInMS > 0, "tcp connect timeout must be positive: " + tcpConnectTimeoutInMS); + + this.authenticator = authenticator; + this.to = to; + this.connectTo = connectTo; + this.encryption = encryption; + this.framing = framing; + this.socketSendBufferSizeInBytes = socketSendBufferSizeInBytes; + this.applicationSendQueueCapacityInBytes = applicationSendQueueCapacityInBytes; + this.applicationSendQueueReserveEndpointCapacityInBytes = applicationSendQueueReserveEndpointCapacityInBytes; + this.applicationSendQueueReserveGlobalCapacityInBytes = applicationSendQueueReserveGlobalCapacityInBytes; + this.tcpNoDelay = tcpNoDelay; + this.flushLowWaterMark = flushLowWaterMark; + this.flushHighWaterMark = flushHighWaterMark; + this.tcpConnectTimeoutInMS = tcpConnectTimeoutInMS; + this.tcpUserTimeoutInMS = tcpUserTimeoutInMS; + this.acceptVersions = acceptVersions; + this.from = from; + this.socketFactory = socketFactory; + this.callbacks = callbacks; + this.debug = debug; + this.endpointToVersion = endpointToVersion; + } + + public boolean authenticate() + { + return authenticator.authenticate(to.address, to.port); + } + + public boolean withEncryption() + { + return encryption != null; + } + + public String toString() + { + return String.format("peer: (%s, %s), framing: %s, encryption: %s", + to, connectTo, framing, encryptionLogStatement(encryption)); + } + + public OutboundConnectionSettings withAuthenticator(IInternodeAuthenticator authenticator) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + @SuppressWarnings("unused") + public OutboundConnectionSettings toEndpoint(InetAddressAndPort endpoint) + { + return new OutboundConnectionSettings(authenticator, endpoint, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withConnectTo(InetAddressAndPort connectTo) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withEncryption(ServerEncryptionOptions encryption) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + @SuppressWarnings("unused") + public OutboundConnectionSettings withFraming(Framing framing) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withSocketSendBufferSizeInBytes(int socketSendBufferSizeInBytes) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + @SuppressWarnings("unused") + public OutboundConnectionSettings withApplicationSendQueueCapacityInBytes(int applicationSendQueueCapacityInBytes) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withApplicationReserveSendQueueCapacityInBytes(Integer applicationReserveSendQueueEndpointCapacityInBytes, ResourceLimits.Limit applicationReserveSendQueueGlobalCapacityInBytes) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationReserveSendQueueEndpointCapacityInBytes, applicationReserveSendQueueGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + @SuppressWarnings("unused") + public OutboundConnectionSettings withTcpNoDelay(boolean tcpNoDelay) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + @SuppressWarnings("unused") + public OutboundConnectionSettings withNettyBufferBounds(WriteBufferWaterMark nettyBufferBounds) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withTcpConnectTimeoutInMS(int tcpConnectTimeoutInMS) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withTcpUserTimeoutInMS(int tcpUserTimeoutInMS) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withAcceptVersions(AcceptVersions acceptVersions) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withFrom(InetAddressAndPort from) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withSocketFactory(SocketFactory socketFactory) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withCallbacks(OutboundMessageCallbacks callbacks) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withDebugCallbacks(OutboundDebugCallbacks debug) + { + return new OutboundConnectionSettings(authenticator, to, connectTo, encryption, framing, + socketSendBufferSizeInBytes, applicationSendQueueCapacityInBytes, + applicationSendQueueReserveEndpointCapacityInBytes, applicationSendQueueReserveGlobalCapacityInBytes, + tcpNoDelay, flushLowWaterMark, flushHighWaterMark, tcpConnectTimeoutInMS, + tcpUserTimeoutInMS, acceptVersions, from, socketFactory, callbacks, debug, endpointToVersion); + } + + public OutboundConnectionSettings withDefaultReserveLimits() + { + Integer applicationReserveSendQueueEndpointCapacityInBytes = this.applicationSendQueueReserveEndpointCapacityInBytes; + ResourceLimits.Limit applicationReserveSendQueueGlobalCapacityInBytes = this.applicationSendQueueReserveGlobalCapacityInBytes; + + if (applicationReserveSendQueueEndpointCapacityInBytes == null) + applicationReserveSendQueueEndpointCapacityInBytes = DatabaseDescriptor.getInternodeApplicationSendQueueReserveEndpointCapacityInBytes(); + if (applicationReserveSendQueueGlobalCapacityInBytes == null) + applicationReserveSendQueueGlobalCapacityInBytes = MessagingService.instance().outboundGlobalReserveLimit; + + return withApplicationReserveSendQueueCapacityInBytes(applicationReserveSendQueueEndpointCapacityInBytes, applicationReserveSendQueueGlobalCapacityInBytes); + } + + public IInternodeAuthenticator authenticator() + { + return authenticator != null ? authenticator : DatabaseDescriptor.getInternodeAuthenticator(); + } + + public EndpointMessagingVersions endpointToVersion() + { + if (endpointToVersion == null) + return instance().versions; + return endpointToVersion; + } + + public InetAddressAndPort from() + { + return from != null ? from : FBUtilities.getBroadcastAddressAndPort(); + } + + public OutboundDebugCallbacks debug() + { + return debug != null ? debug : OutboundDebugCallbacks.NONE; + } + + public EncryptionOptions encryption() + { + return encryption != null ? encryption : defaultEncryptionOptions(to); + } + + public SocketFactory socketFactory() + { + return socketFactory != null ? socketFactory : instance().socketFactory; + } + + public OutboundMessageCallbacks callbacks() + { + return callbacks != null ? callbacks : instance().callbacks; + } + + public int socketSendBufferSizeInBytes() + { + return socketSendBufferSizeInBytes != null ? socketSendBufferSizeInBytes + : DatabaseDescriptor.getInternodeSocketSendBufferSizeInBytes(); + } + + public int applicationSendQueueCapacityInBytes() + { + return applicationSendQueueCapacityInBytes != null ? applicationSendQueueCapacityInBytes + : DatabaseDescriptor.getInternodeApplicationSendQueueCapacityInBytes(); + } + + public ResourceLimits.Limit applicationSendQueueReserveGlobalCapacityInBytes() + { + return applicationSendQueueReserveGlobalCapacityInBytes != null ? applicationSendQueueReserveGlobalCapacityInBytes + : instance().outboundGlobalReserveLimit; + } + + public int applicationSendQueueReserveEndpointCapacityInBytes() + { + return applicationSendQueueReserveEndpointCapacityInBytes != null ? applicationSendQueueReserveEndpointCapacityInBytes + : DatabaseDescriptor.getInternodeApplicationReceiveQueueReserveEndpointCapacityInBytes(); + } + + public int tcpConnectTimeoutInMS() + { + return tcpConnectTimeoutInMS != null ? tcpConnectTimeoutInMS + : DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS(); + } + + public int tcpUserTimeoutInMS() + { + return tcpUserTimeoutInMS != null ? tcpUserTimeoutInMS + : DatabaseDescriptor.getInternodeTcpUserTimeoutInMS(); + } + + public boolean tcpNoDelay() + { + if (tcpNoDelay != null) + return tcpNoDelay; + + if (isInLocalDC(getEndpointSnitch(), getBroadcastAddressAndPort(), to)) + return INTRADC_TCP_NODELAY; + + return DatabaseDescriptor.getInterDCTcpNoDelay(); + } + + public AcceptVersions acceptVersions(ConnectionCategory category) + { + return acceptVersions != null ? acceptVersions + : category.isStreaming() + ? MessagingService.accept_streaming + : MessagingService.accept_messaging; + } + + public OutboundConnectionSettings withLegacyPortIfNecessary(int messagingVersion) + { + return withConnectTo(maybeWithSecurePort(connectTo(), messagingVersion, withEncryption())); + } + + public InetAddressAndPort connectTo() + { + InetAddressAndPort connectTo = this.connectTo; + if (connectTo == null) + connectTo = SystemKeyspace.getPreferredIP(to); + return connectTo; + } + + public Framing framing(ConnectionCategory category) + { + if (framing != null) + return framing; + + if (category.isStreaming()) + return Framing.UNPROTECTED; + + return shouldCompressConnection(getEndpointSnitch(), getBroadcastAddressAndPort(), to) + ? Framing.LZ4 : Framing.CRC; + } + + // note that connectTo is updated even if specified, in the case of pre40 messaging and using encryption (to update port) + public OutboundConnectionSettings withDefaults(ConnectionCategory category) + { + if (to == null) + throw new IllegalArgumentException(); + + return new OutboundConnectionSettings(authenticator(), to, connectTo(), + encryption(), framing(category), + socketSendBufferSizeInBytes(), applicationSendQueueCapacityInBytes(), + applicationSendQueueReserveEndpointCapacityInBytes(), + applicationSendQueueReserveGlobalCapacityInBytes(), + tcpNoDelay(), flushLowWaterMark, flushHighWaterMark, + tcpConnectTimeoutInMS(), tcpUserTimeoutInMS(), acceptVersions(category), + from(), socketFactory(), callbacks(), debug(), endpointToVersion()); + } + + private static boolean isInLocalDC(IEndpointSnitch snitch, InetAddressAndPort localHost, InetAddressAndPort remoteHost) + { + String remoteDC = snitch.getDatacenter(remoteHost); + String localDC = snitch.getDatacenter(localHost); + return remoteDC != null && remoteDC.equals(localDC); + } + + @VisibleForTesting + static EncryptionOptions defaultEncryptionOptions(InetAddressAndPort endpoint) + { + ServerEncryptionOptions options = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); + return options.shouldEncrypt(endpoint) ? options : null; + } + + @VisibleForTesting + static boolean shouldCompressConnection(IEndpointSnitch snitch, InetAddressAndPort localHost, InetAddressAndPort remoteHost) + { + return (DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.all) + || ((DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.dc) && !isInLocalDC(snitch, localHost, remoteHost)); + } + + private static InetAddressAndPort maybeWithSecurePort(InetAddressAndPort address, int messagingVersion, boolean isEncrypted) + { + if (!isEncrypted || messagingVersion >= VERSION_40) + return address; + + // if we don't know the version of the peer, assume it is 4.0 (or higher) as the only time is would be lower + // (as in a 3.x version) is during a cluster upgrade (from 3.x to 4.0). In that case the outbound connection will + // unfortunately fail - however the peer should connect to this node (at some point), and once we learn it's version, it'll be + // in versions map. thus, when we attempt to reconnect to that node, we'll have the version and we can get the correct port. + // we will be able to remove this logic at 5.0. + // Also as of 4.0 we will propagate the "regular" port (which will support both SSL and non-SSL) via gossip so + // for SSL and version 4.0 always connect to the gossiped port because if SSL is enabled it should ALWAYS + // listen for SSL on the "regular" port. + return address.withPort(DatabaseDescriptor.getSSLStoragePort()); + } + +} diff --git a/src/java/org/apache/cassandra/net/OutboundConnections.java b/src/java/org/apache/cassandra/net/OutboundConnections.java new file mode 100644 index 000000000000..5f9190b5b671 --- /dev/null +++ b/src/java/org/apache/cassandra/net/OutboundConnections.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.channels.ClosedChannelException; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import com.carrotsearch.hppc.ObjectObjectOpenHashMap; +import io.netty.util.concurrent.Future; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.metrics.InternodeOutboundMetrics; +import org.apache.cassandra.utils.concurrent.SimpleCondition; + +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.ConnectionType.URGENT_MESSAGES; +import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES; +import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES; + +/** + * Groups a set of outbound connections to a given peer, and routes outgoing messages to the appropriate connection + * (based upon message's type or size). Contains a {@link OutboundConnection} for each of the + * {@link ConnectionType} types. + */ +public class OutboundConnections +{ + @VisibleForTesting + public static final int LARGE_MESSAGE_THRESHOLD = Integer.getInteger(Config.PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64) + - Math.max(Math.max(LegacyLZ4Constants.HEADER_LENGTH, FrameEncoderCrc.HEADER_AND_TRAILER_LENGTH), FrameEncoderLZ4.HEADER_AND_TRAILER_LENGTH); + + private final SimpleCondition metricsReady = new SimpleCondition(); + private volatile InternodeOutboundMetrics metrics; + private final BackPressureState backPressureState; + private final ResourceLimits.Limit reserveCapacity; + + private OutboundConnectionSettings template; + public final OutboundConnection small; + public final OutboundConnection large; + public final OutboundConnection urgent; + + private OutboundConnections(OutboundConnectionSettings template, BackPressureState backPressureState) + { + this.backPressureState = backPressureState; + this.template = template = template.withDefaultReserveLimits(); + reserveCapacity = new ResourceLimits.Concurrent(template.applicationSendQueueReserveEndpointCapacityInBytes); + ResourceLimits.EndpointAndGlobal reserveCapacityInBytes = new ResourceLimits.EndpointAndGlobal(reserveCapacity, template.applicationSendQueueReserveGlobalCapacityInBytes); + this.small = new OutboundConnection(SMALL_MESSAGES, template, reserveCapacityInBytes); + this.large = new OutboundConnection(LARGE_MESSAGES, template, reserveCapacityInBytes); + this.urgent = new OutboundConnection(URGENT_MESSAGES, template, reserveCapacityInBytes); + } + + /** + * Select the appropriate connection for the provided message and use it to send the message. + */ + public void enqueue(Message msg, ConnectionType type) throws ClosedChannelException + { + connectionFor(msg, type).enqueue(msg); + } + + static OutboundConnections tryRegister(ConcurrentMap in, K key, OutboundConnectionSettings settings, BackPressureState backPressureState) + { + OutboundConnections connections = in.get(key); + if (connections == null) + { + connections = new OutboundConnections(settings, backPressureState); + OutboundConnections existing = in.putIfAbsent(key, connections); + + if (existing == null) + { + connections.metrics = new InternodeOutboundMetrics(settings.to, connections); + connections.metricsReady.signalAll(); + } + else + { + connections.metricsReady.signalAll(); + connections.close(false); + connections = existing; + } + } + return connections; + } + + BackPressureState getBackPressureState() + { + return backPressureState; + } + + /** + * Reconnect to the peer using the given {@code addr}. Outstanding messages in each channel will be sent on the + * current channel. Typically this function is used for something like EC2 public IP addresses which need to be used + * for communication between EC2 regions. + * + * @param addr IP Address to use (and prefer) going forward for connecting to the peer + */ + synchronized Future reconnectWithNewIp(InetAddressAndPort addr) + { + template = template.withConnectTo(addr); + return new FutureCombiner( + apply(c -> c.reconnectWith(template)) + ); + } + + /** + * Close the connections permanently + * + * @param flushQueues {@code true} if existing messages in the queue should be sent before closing. + */ + synchronized Future scheduleClose(long time, TimeUnit unit, boolean flushQueues) + { + // immediately release our metrics, so that if we need to re-open immediately we can safely register a new one + releaseMetrics(); + return new FutureCombiner( + apply(c -> c.scheduleClose(time, unit, flushQueues)) + ); + } + + /** + * Close the connections permanently + * + * @param flushQueues {@code true} if existing messages in the queue should be sent before closing. + */ + synchronized Future close(boolean flushQueues) + { + // immediately release our metrics, so that if we need to re-open immediately we can safely register a new one + releaseMetrics(); + return new FutureCombiner( + apply(c -> c.close(flushQueues)) + ); + } + + private void releaseMetrics() + { + try + { + metricsReady.await(); + } + catch (InterruptedException e) + { + throw new RuntimeException(e); + } + + if (metrics != null) + metrics.release(); + } + + /** + * Close each netty channel and its socket + */ + void interrupt() + { + // must return a non-null value for ImmutableList.of() + apply(OutboundConnection::interrupt); + } + + /** + * Apply the given function to each of the connections we are pooling, returning the results as a list + */ + private List apply(Function f) + { + return ImmutableList.of( + f.apply(urgent), f.apply(small), f.apply(large) + ); + } + + @VisibleForTesting + OutboundConnection connectionFor(Message message) + { + return connectionFor(message, null); + } + + private OutboundConnection connectionFor(Message msg, ConnectionType forceConnection) + { + return connectionFor(connectionTypeFor(msg, forceConnection)); + } + + private static ConnectionType connectionTypeFor(Message msg, ConnectionType specifyConnection) + { + if (specifyConnection != null) + return specifyConnection; + + if (msg.verb().priority == Verb.Priority.P0) + return URGENT_MESSAGES; + + return msg.serializedSize(current_version) <= LARGE_MESSAGE_THRESHOLD + ? SMALL_MESSAGES + : LARGE_MESSAGES; + } + + @VisibleForTesting + final OutboundConnection connectionFor(ConnectionType type) + { + switch (type) + { + case SMALL_MESSAGES: + return small; + case LARGE_MESSAGES: + return large; + case URGENT_MESSAGES: + return urgent; + default: + throw new IllegalArgumentException("unsupported connection type: " + type); + } + } + + public long usingReserveBytes() + { + return reserveCapacity.using(); + } + + long expiredCallbacks() + { + return metrics.expiredCallbacks.getCount(); + } + + void incrementExpiredCallbackCount() + { + metrics.expiredCallbacks.mark(); + } + + OutboundConnectionSettings template() + { + return template; + } + + private static class UnusedConnectionMonitor + { + UnusedConnectionMonitor(MessagingService messagingService) + { + this.messagingService = messagingService; + } + + static class Counts + { + final long small, large, urgent; + Counts(long small, long large, long urgent) + { + this.small = small; + this.large = large; + this.urgent = urgent; + } + } + + final MessagingService messagingService; + ObjectObjectOpenHashMap prevEndpointToCounts = new ObjectObjectOpenHashMap<>(); + + private void closeUnusedSinceLastRun() + { + ObjectObjectOpenHashMap curEndpointToCounts = new ObjectObjectOpenHashMap<>(); + for (OutboundConnections connections : messagingService.channelManagers.values()) + { + Counts cur = new Counts( + connections.small.submittedCount(), + connections.large.submittedCount(), + connections.urgent.submittedCount() + ); + curEndpointToCounts.put(connections.template.to, cur); + + Counts prev = prevEndpointToCounts.get(connections.template.to); + if (prev == null) + continue; + + if (cur.small != prev.small && cur.large != prev.large && cur.urgent != prev.urgent) + continue; + + if (cur.small == prev.small && cur.large == prev.large && cur.urgent == prev.urgent + && !Gossiper.instance.isKnownEndpoint(connections.template.to)) + { + // close entirely if no traffic and the endpoint is unknown + messagingService.closeOutboundNow(connections); + continue; + } + + if (cur.small == prev.small) + connections.small.interrupt(); + + if (cur.large == prev.large) + connections.large.interrupt(); + + if (cur.urgent == prev.urgent) + connections.urgent.interrupt(); + } + + prevEndpointToCounts = curEndpointToCounts; + } + } + + static void scheduleUnusedConnectionMonitoring(MessagingService messagingService, ScheduledExecutorService executor, long delay, TimeUnit units) + { + executor.scheduleWithFixedDelay(new UnusedConnectionMonitor(messagingService)::closeUnusedSinceLastRun, 0L, delay, units); + } + + @VisibleForTesting + static OutboundConnections unsafeCreate(OutboundConnectionSettings template, BackPressureState backPressureState) + { + OutboundConnections connections = new OutboundConnections(template, backPressureState); + connections.metricsReady.signalAll(); + return connections; + } + +} diff --git a/src/java/org/apache/cassandra/net/OutboundDebugCallbacks.java b/src/java/org/apache/cassandra/net/OutboundDebugCallbacks.java new file mode 100644 index 000000000000..3b83519fc3fc --- /dev/null +++ b/src/java/org/apache/cassandra/net/OutboundDebugCallbacks.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +interface OutboundDebugCallbacks +{ + static final OutboundDebugCallbacks NONE = new OutboundDebugCallbacks() + { + public void onSendSmallFrame(int messageCount, int payloadSizeInBytes) {} + public void onSentSmallFrame(int messageCount, int payloadSizeInBytes) {} + public void onFailedSmallFrame(int messageCount, int payloadSizeInBytes) {} + public void onConnect(int messagingVersion, OutboundConnectionSettings settings) {} + }; + + /** A complete Frame has been handed to Netty to write to the wire. */ + void onSendSmallFrame(int messageCount, int payloadSizeInBytes); + + /** A complete Frame has been serialized to the wire */ + void onSentSmallFrame(int messageCount, int payloadSizeInBytes); + + /** Failed to send an entire frame due to network problems; presumed to be invoked in same order as onSendSmallFrame */ + void onFailedSmallFrame(int messageCount, int payloadSizeInBytes); + + void onConnect(int messagingVersion, OutboundConnectionSettings settings); +} diff --git a/src/java/org/apache/cassandra/net/ForwardToContainer.java b/src/java/org/apache/cassandra/net/OutboundMessageCallbacks.java similarity index 55% rename from src/java/org/apache/cassandra/net/ForwardToContainer.java rename to src/java/org/apache/cassandra/net/OutboundMessageCallbacks.java index b22eed6b9f91..abf3f4117d0e 100644 --- a/src/java/org/apache/cassandra/net/ForwardToContainer.java +++ b/src/java/org/apache/cassandra/net/OutboundMessageCallbacks.java @@ -15,30 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.net; -import java.io.Serializable; -import java.util.Collection; - -import com.google.common.base.Preconditions; - import org.apache.cassandra.locator.InetAddressAndPort; -/** - * Contains forward to information until it can be serialized as part of a message using a version - * specific serialization - */ -public class ForwardToContainer implements Serializable +interface OutboundMessageCallbacks { - public final Collection targets; - public final int[] messageIds; + /** A message was not enqueued to the link because too many messages are already waiting to send */ + void onOverloaded(Message message, InetAddressAndPort peer); + + /** A message was not serialized to a frame because it had expired */ + void onExpired(Message message, InetAddressAndPort peer); + + /** A message was not fully or successfully serialized to a frame because an exception was thrown */ + void onFailedSerialize(Message message, InetAddressAndPort peer, int messagingVersion, int bytesWrittenToNetwork, Throwable failure); - public ForwardToContainer(Collection targets, - int[] messageIds) - { - Preconditions.checkArgument(targets.size() == messageIds.length); - this.targets = targets; - this.messageIds = messageIds; - } + /** A message was not sent because the connection was forcibly closed */ + void onDiscardOnClose(Message message, InetAddressAndPort peer); } diff --git a/src/java/org/apache/cassandra/net/OutboundMessageQueue.java b/src/java/org/apache/cassandra/net/OutboundMessageQueue.java new file mode 100644 index 000000000000..48c766629d4d --- /dev/null +++ b/src/java/org/apache/cassandra/net/OutboundMessageQueue.java @@ -0,0 +1,484 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Consumer; + +import com.google.common.util.concurrent.Uninterruptibles; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.lang.Math.min; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +/** + * A composite queue holding messages to be delivered by an {@link OutboundConnection}. + * + * Contains two queues: + * 1. An external MPSC {@link ManyToOneConcurrentLinkedQueue} for producers to enqueue messages onto + * 2. An internal intermediate {@link PrunableArrayQueue} into which the external queue is + * drained with exclusive access and from which actual deliveries happen + * The second, intermediate queue exists to enable efficient in-place pruning of expired messages. + * + * Said pruning will be attempted in several scenarios: + * 1. By callers invoking {@link #add(Message)} - if metadata indicates presence of expired messages + * in the queue, and if exclusive access can be immediately obtained (non-blockingly) + * 2. By {@link OutboundConnection}, periodically, while disconnected + * 3. As an optimisation, in an attempt to free up endpoint capacity on {@link OutboundConnection#enqueue(Message)} + * if current endpoint reserve was insufficient + */ +class OutboundMessageQueue +{ + private static final Logger logger = LoggerFactory.getLogger(OutboundMessageQueue.class); + + interface MessageConsumer + { + boolean accept(Message message) throws Produces; + } + + private final MessageConsumer onExpired; + + private final ManyToOneConcurrentLinkedQueue> externalQueue = new ManyToOneConcurrentLinkedQueue<>(); + private final PrunableArrayQueue> internalQueue = new PrunableArrayQueue<>(256); + + private volatile long earliestExpiresAt = Long.MAX_VALUE; + private static final AtomicLongFieldUpdater earliestExpiresAtUpdater = + AtomicLongFieldUpdater.newUpdater(OutboundMessageQueue.class, "earliestExpiresAt"); + + OutboundMessageQueue(MessageConsumer onExpired) + { + this.onExpired = onExpired; + } + + /** + * Add the provided message to the queue. Always succeeds. + */ + void add(Message m) + { + maybePruneExpired(); + externalQueue.offer(m); + maybeUpdateMinimumExpiryTime(m.expiresAtNanos()); + } + + /** + * Try to obtain the lock; if this fails, a callback will be registered to be invoked when + * the lock is relinquished. + * + * This callback will run WITHOUT ownership of the lock, so must re-obtain the lock. + * + * @return null if failed to obtain the lock + */ + WithLock lockOrCallback(long nowNanos, Runnable callbackIfDeferred) + { + if (!lockOrCallback(callbackIfDeferred)) + return null; + + return new WithLock(nowNanos); + } + + /** + * Try to obtain the lock. If successful, invoke the provided consumer immediately, otherwise + * register it to be invoked when the lock is relinquished. + */ + void runEventually(Consumer runEventually) + { + try (WithLock withLock = lockOrCallback(approxTime.now(), () -> runEventually(runEventually))) + { + if (withLock != null) + runEventually.accept(withLock); + } + } + + /** + * If succeeds to obtain the lock, polls the queue, otherwise registers the provided callback + * to be invoked when the lock is relinquished. + * + * May return null when the queue is non-empty - if the lock could not be acquired. + */ + Message tryPoll(long nowNanos, Runnable elseIfDeferred) + { + try (WithLock withLock = lockOrCallback(nowNanos, elseIfDeferred)) + { + if (withLock == null) + return null; + + return withLock.poll(); + } + } + + class WithLock implements AutoCloseable + { + private final long nowNanos; + + private WithLock(long nowNanos) + { + this.nowNanos = nowNanos; + earliestExpiresAt = Long.MAX_VALUE; + externalQueue.drain(internalQueue::offer); + } + + Message poll() + { + Message m; + while (null != (m = internalQueue.poll())) + { + if (shouldSend(m, nowNanos)) + break; + + onExpired.accept(m); + } + + return m; + } + + void removeHead(Message expectHead) + { + assert expectHead == internalQueue.peek(); + internalQueue.poll(); + } + + Message peek() + { + Message m; + while (null != (m = internalQueue.peek())) + { + if (shouldSend(m, nowNanos)) + break; + + internalQueue.poll(); + onExpired.accept(m); + } + + return m; + } + + void consume(Consumer> consumer) + { + Message m; + while (null != (m = poll())) + consumer.accept(m); + } + + @Override + public void close() + { + pruneInternalQueueWithLock(nowNanos); + unlock(); + } + } + + /** + * Call periodically if cannot expect to promptly invoke consume() + */ + boolean maybePruneExpired() + { + return maybePruneExpired(approxTime.now()); + } + + private boolean maybePruneExpired(long nowNanos) + { + if (approxTime.isAfter(nowNanos, earliestExpiresAt)) + return tryRun(() -> pruneWithLock(nowNanos)); + return false; + } + + private void maybeUpdateMinimumExpiryTime(long newTime) + { + if (newTime < earliestExpiresAt) + earliestExpiresAtUpdater.accumulateAndGet(this, newTime, Math::min); + } + + /* + * Drain external queue into the internal one and prune the latter in-place. + */ + private void pruneWithLock(long nowNanos) + { + earliestExpiresAt = Long.MAX_VALUE; + externalQueue.drain(internalQueue::offer); + pruneInternalQueueWithLock(nowNanos); + } + + /* + * Prune the internal queue in-place. + */ + private void pruneInternalQueueWithLock(long nowNanos) + { + class Pruner implements PrunableArrayQueue.Pruner> + { + private long earliestExpiresAt = Long.MAX_VALUE; + + public boolean shouldPrune(Message message) + { + return !shouldSend(message, nowNanos); + } + + public void onPruned(Message message) + { + onExpired.accept(message); + } + + public void onKept(Message message) + { + earliestExpiresAt = min(message.expiresAtNanos(), earliestExpiresAt); + } + } + + Pruner pruner = new Pruner(); + internalQueue.prune(pruner); + + maybeUpdateMinimumExpiryTime(pruner.earliestExpiresAt); + } + + private static class Locked implements Runnable + { + final Runnable run; + final Locked next; + private Locked(Runnable run, Locked next) + { + this.run = run; + this.next = next; + } + + Locked andThen(Runnable next) + { + return new Locked(next, this); + } + + public void run() + { + Locked cur = this; + while (cur != null) + { + try + { + cur.run.run(); + } + catch (Throwable t) + { + logger.error("Unexpected error when executing deferred lock-intending functions", t); + } + cur = cur.next; + } + } + } + + private static final Locked LOCKED = new Locked(() -> {}, null); + + private volatile Locked locked = null; + private static final AtomicReferenceFieldUpdater lockedUpdater = + AtomicReferenceFieldUpdater.newUpdater(OutboundMessageQueue.class, Locked.class, "locked"); + + /** + * Run runOnceLocked either immediately in the calling thread if we can obtain the lock, or ask the lock's current + * owner attempt to run it when the lock is released. This may be passed between a sequence of owners, as the present + * owner releases the lock before trying to acquire it again and execute the task. + */ + private void runEventually(Runnable runEventually) + { + if (!lockOrCallback(() -> runEventually(runEventually))) + return; + + try + { + runEventually.run(); + } + finally + { + unlock(); + } + } + + /** + * If we can immediately obtain the lock, execute runIfLocked and return true; + * otherwise do nothing and return false. + */ + private boolean tryRun(Runnable runIfAvailable) + { + if (!tryLock()) + return false; + + try + { + runIfAvailable.run(); + return true; + } + finally + { + unlock(); + } + } + + /** + * @return true iff the caller now owns the lock + */ + private boolean tryLock() + { + return locked == null && lockedUpdater.compareAndSet(this, null, LOCKED); + } + + /** + * Try to obtain the lock; if this fails, a callback will be registered to be invoked when the lock is relinquished. + * This callback will run WITHOUT ownership of the lock, so must re-obtain the lock. + * + * @return true iff the caller now owns the lock + */ + private boolean lockOrCallback(Runnable callbackWhenAvailable) + { + if (callbackWhenAvailable == null) + return tryLock(); + + while (true) + { + Locked current = locked; + if (current == null && lockedUpdater.compareAndSet(this, null, LOCKED)) + return true; + else if (current != null && lockedUpdater.compareAndSet(this, current, current.andThen(callbackWhenAvailable))) + return false; + } + } + + private void unlock() + { + Locked locked = lockedUpdater.getAndSet(this, null); + locked.run(); + } + + + /** + * While removal happens extremely infrequently, it seems possible for many to still interleave with a connection + * being closed, as experimentally we have encountered enough pending removes to overflow the Locked call stack + * (prior to making its evaluation iterative). + * + * While the stack can no longer be exhausted, this suggests a high potential cost for evaluating all removals, + * so to ensure system stability we aggregate all pending removes into a single shared object that evaluate + * together with only a single lock acquisition. + */ + private volatile RemoveRunner removeRunner = null; + private static final AtomicReferenceFieldUpdater removeRunnerUpdater = + AtomicReferenceFieldUpdater.newUpdater(OutboundMessageQueue.class, RemoveRunner.class, "removeRunner"); + + static class Remove + { + final Message message; + final Remove next; + + Remove(Message message, Remove next) + { + this.message = message; + this.next = next; + } + } + + private class RemoveRunner extends AtomicReference implements Runnable + { + final CountDownLatch done = new CountDownLatch(1); + final Set> removed = Collections.newSetFromMap(new IdentityHashMap<>()); + + RemoveRunner() { super(new Remove(null, null)); } + + boolean undo(Message message) + { + return null != updateAndGet(prev -> prev == null ? null : new Remove(message, prev)); + } + + public void run() + { + Set> remove = Collections.newSetFromMap(new IdentityHashMap<>()); + removeRunner = null; + Remove undo = getAndSet(null); + while (undo.message != null) + { + remove.add(undo.message); + undo = undo.next; + } + + class Remover implements PrunableArrayQueue.Pruner> + { + private long earliestExpiresAt = Long.MAX_VALUE; + + @Override + public boolean shouldPrune(Message message) + { + return remove.contains(message); + } + + @Override + public void onPruned(Message message) + { + removed.add(message); + } + + @Override + public void onKept(Message message) + { + earliestExpiresAt = min(message.expiresAtNanos(), earliestExpiresAt); + } + } + + Remover remover = new Remover(); + earliestExpiresAt = Long.MAX_VALUE; + externalQueue.drain(internalQueue::offer); + internalQueue.prune(remover); + maybeUpdateMinimumExpiryTime(remover.earliestExpiresAt); + done.countDown(); + } + } + + /** + * Remove the provided Message from the queue, if present. + * + * WARNING: This is a blocking call. + */ + boolean remove(Message remove) + { + if (remove == null) + throw new NullPointerException(); + + RemoveRunner runner; + while (true) + { + runner = removeRunner; + if (runner != null && runner.undo(remove)) + break; + + if (runner == null && removeRunnerUpdater.compareAndSet(this, null, runner = new RemoveRunner())) + { + runner.undo(remove); + runEventually(runner); + break; + } + } + + //noinspection UnstableApiUsage + Uninterruptibles.awaitUninterruptibly(runner.done); + return runner.removed.contains(remove); + } + + private static boolean shouldSend(Message m, long nowNanos) + { + return !approxTime.isAfter(nowNanos, m.expiresAtNanos()); + } +} diff --git a/src/java/org/apache/cassandra/net/OutboundSink.java b/src/java/org/apache/cassandra/net/OutboundSink.java new file mode 100644 index 000000000000..d19b3e2107a2 --- /dev/null +++ b/src/java/org/apache/cassandra/net/OutboundSink.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.BiPredicate; + +import org.apache.cassandra.locator.InetAddressAndPort; + +/** + * A message sink that all outbound messages go through. + * + * Default sink {@link Sink} used by {@link MessagingService} is MessagingService#doSend(), which proceeds to + * send messages over the network, but it can be overridden to filter out certain messages, record the fact + * of attempted delivery, or delay they delivery. + * + * This facility is most useful for test code. + */ +public class OutboundSink +{ + public interface Sink + { + void accept(Message message, InetAddressAndPort to, ConnectionType connectionType); + } + + private static class Filtered implements Sink + { + final BiPredicate, InetAddressAndPort> condition; + final Sink next; + + private Filtered(BiPredicate, InetAddressAndPort> condition, Sink next) + { + this.condition = condition; + this.next = next; + } + + public void accept(Message message, InetAddressAndPort to, ConnectionType connectionType) + { + if (condition.test(message, to)) + next.accept(message, to, connectionType); + } + } + + private volatile Sink sink; + private static final AtomicReferenceFieldUpdater sinkUpdater + = AtomicReferenceFieldUpdater.newUpdater(OutboundSink.class, Sink.class, "sink"); + + OutboundSink(Sink sink) + { + this.sink = sink; + } + + public void accept(Message message, InetAddressAndPort to, ConnectionType connectionType) + { + sink.accept(message, to, connectionType); + } + + public void add(BiPredicate, InetAddressAndPort> allow) + { + sinkUpdater.updateAndGet(this, sink -> new Filtered(allow, sink)); + } + + public void remove(BiPredicate, InetAddressAndPort> allow) + { + sinkUpdater.updateAndGet(this, sink -> without(sink, allow)); + } + + public void clear() + { + sinkUpdater.updateAndGet(this, OutboundSink::clear); + } + + private static Sink clear(Sink sink) + { + while (sink instanceof OutboundSink.Filtered) + sink = ((OutboundSink.Filtered) sink).next; + return sink; + } + + private static Sink without(Sink sink, BiPredicate, InetAddressAndPort> condition) + { + if (!(sink instanceof Filtered)) + return sink; + + Filtered filtered = (Filtered) sink; + Sink next = without(filtered.next, condition); + return condition.equals(filtered.condition) ? next + : next == filtered.next + ? sink + : new Filtered(filtered.condition, next); + } + +} diff --git a/src/java/org/apache/cassandra/net/ParamType.java b/src/java/org/apache/cassandra/net/ParamType.java new file mode 100644 index 000000000000..65723487e5da --- /dev/null +++ b/src/java/org/apache/cassandra/net/ParamType.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.HashMap; +import java.util.Map; + +import javax.annotation.Nullable; + +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.utils.UUIDSerializer; + +import static java.lang.Math.max; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + +/** + * Type names and serializers for various parameters that can be put in {@link Message} params map. + * + * It should be safe to add new params without bumping messaging version - {@link Message} serializer + * will skip over any params it doesn't recognise. + * + * Please don't add boolean params here. Extend and use {@link MessageFlag} instead. + */ +public enum ParamType +{ + FORWARD_TO (0, "FORWARD_TO", ForwardingInfo.serializer), + RESPOND_TO (1, "FORWARD_FROM", inetAddressAndPortSerializer), + + @Deprecated + FAILURE_RESPONSE (2, "FAIL", LegacyFlag.serializer), + @Deprecated + FAILURE_REASON (3, "FAIL_REASON", RequestFailureReason.serializer), + @Deprecated + FAILURE_CALLBACK (4, "CAL_BAC", LegacyFlag.serializer), + + TRACE_SESSION (5, "TraceSession", UUIDSerializer.serializer), + TRACE_TYPE (6, "TraceType", Tracing.traceTypeSerializer), + + @Deprecated + TRACK_REPAIRED_DATA (7, "TrackRepaired", LegacyFlag.serializer); + + final int id; + @Deprecated final String legacyAlias; // pre-4.0 we used to serialize entire param name string + final IVersionedSerializer serializer; + + ParamType(int id, String legacyAlias, IVersionedSerializer serializer) + { + if (id < 0) + throw new IllegalArgumentException("ParamType id must be non-negative"); + + this.id = id; + this.legacyAlias = legacyAlias; + this.serializer = serializer; + } + + private static final ParamType[] idToTypeMap; + private static final Map aliasToTypeMap; + + static + { + ParamType[] types = values(); + + int max = -1; + for (ParamType t : types) + max = max(t.id, max); + + ParamType[] idMap = new ParamType[max + 1]; + Map aliasMap = new HashMap<>(); + + for (ParamType type : types) + { + if (idMap[type.id] != null) + throw new RuntimeException("Two ParamType-s that map to the same id: " + type.id); + idMap[type.id] = type; + + if (aliasMap.put(type.legacyAlias, type) != null) + throw new RuntimeException("Two ParamType-s that map to the same legacy alias: " + type.legacyAlias); + } + + idToTypeMap = idMap; + aliasToTypeMap = aliasMap; + } + + @Nullable + static ParamType lookUpById(int id) + { + if (id < 0) + throw new IllegalArgumentException("ParamType id must be non-negative (got " + id + ')'); + + return id < idToTypeMap.length ? idToTypeMap[id] : null; + } + + @Nullable + static ParamType lookUpByAlias(String alias) + { + return aliasToTypeMap.get(alias); + } +} diff --git a/src/java/org/apache/cassandra/net/ParameterType.java b/src/java/org/apache/cassandra/net/ParameterType.java deleted file mode 100644 index b7a88a8dd93c..000000000000 --- a/src/java/org/apache/cassandra/net/ParameterType.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net; - -import java.util.Map; - -import com.google.common.collect.ImmutableMap; - -import org.apache.cassandra.io.DummyByteVersionedSerializer; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.ShortVersionedSerializer; -import org.apache.cassandra.tracing.Tracing; -import org.apache.cassandra.utils.UUIDSerializer; - -/** - * Type names and serializers for various parameters that - */ -public enum ParameterType -{ - FORWARD_TO("FORWARD_TO", ForwardToSerializer.instance), - FORWARD_FROM("FORWARD_FROM", CompactEndpointSerializationHelper.instance), - FAILURE_RESPONSE("FAIL", DummyByteVersionedSerializer.instance), - FAILURE_REASON("FAIL_REASON", ShortVersionedSerializer.instance), - FAILURE_CALLBACK("CAL_BAC", DummyByteVersionedSerializer.instance), - TRACE_SESSION("TraceSession", UUIDSerializer.serializer), - TRACE_TYPE("TraceType", Tracing.traceTypeSerializer), - TRACK_REPAIRED_DATA("TrackRepaired", DummyByteVersionedSerializer.instance); - - public static final Map byName; - public final String key; - public final IVersionedSerializer serializer; - - static - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (ParameterType type : values()) - { - builder.put(type.key, type); - } - byName = builder.build(); - } - - ParameterType(String key, IVersionedSerializer serializer) - { - this.key = key; - this.serializer = serializer; - } - - public String key() - { - return key; - } - -} diff --git a/src/java/org/apache/cassandra/net/PingMessage.java b/src/java/org/apache/cassandra/net/PingMessage.java deleted file mode 100644 index 4a19f22b112d..000000000000 --- a/src/java/org/apache/cassandra/net/PingMessage.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net; - -import java.io.IOException; - -import org.apache.cassandra.hints.HintResponse; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType; - -/** - * Conceptually the same as {@link org.apache.cassandra.gms.EchoMessage}, but indicates to the recipient which - * {@link ConnectionType} should be used for the response. - */ -public class PingMessage -{ - public static IVersionedSerializer serializer = new PingMessageSerializer(); - - public static final PingMessage smallChannelMessage = new PingMessage(ConnectionType.SMALL_MESSAGE); - public static final PingMessage largeChannelMessage = new PingMessage(ConnectionType.LARGE_MESSAGE); - public static final PingMessage gossipChannelMessage = new PingMessage(ConnectionType.GOSSIP); - - public final ConnectionType connectionType; - - public PingMessage(ConnectionType connectionType) - { - this.connectionType = connectionType; - } - - public static class PingMessageSerializer implements IVersionedSerializer - { - public void serialize(PingMessage t, DataOutputPlus out, int version) throws IOException - { - out.writeByte(t.connectionType.getId()); - } - - public PingMessage deserialize(DataInputPlus in, int version) throws IOException - { - ConnectionType connectionType = ConnectionType.fromId(in.readByte()); - - // if we ever create a new connection type, then during a rolling upgrade, the old nodes won't know about - // the new connection type (as it won't recognize the id), so just default to the small message type. - if (connectionType == null) - connectionType = ConnectionType.SMALL_MESSAGE; - - switch (connectionType) - { - case LARGE_MESSAGE: - return largeChannelMessage; - case GOSSIP: - return gossipChannelMessage; - case SMALL_MESSAGE: - default: - return smallChannelMessage; - } - } - - public long serializedSize(PingMessage t, int version) - { - return 1; - } - } -} diff --git a/src/java/org/apache/cassandra/net/PingRequest.java b/src/java/org/apache/cassandra/net/PingRequest.java new file mode 100644 index 000000000000..c02bd8099d4c --- /dev/null +++ b/src/java/org/apache/cassandra/net/PingRequest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; + +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; + +import static org.apache.cassandra.net.ConnectionType.URGENT_MESSAGES; +import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES; +import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES; + +/** + * Indicates to the recipient which {@link ConnectionType} should be used for the response. + */ +class PingRequest +{ + static final PingRequest forUrgent = new PingRequest(URGENT_MESSAGES); + static final PingRequest forSmall = new PingRequest(SMALL_MESSAGES); + static final PingRequest forLarge = new PingRequest(LARGE_MESSAGES); + + final ConnectionType connectionType; + + private PingRequest(ConnectionType connectionType) + { + this.connectionType = connectionType; + } + + static IVersionedSerializer serializer = new IVersionedSerializer() + { + public void serialize(PingRequest t, DataOutputPlus out, int version) throws IOException + { + out.writeByte(t.connectionType.id); + } + + public PingRequest deserialize(DataInputPlus in, int version) throws IOException + { + ConnectionType type = ConnectionType.fromId(in.readByte()); + + switch (type) + { + case URGENT_MESSAGES: return forUrgent; + case SMALL_MESSAGES: return forSmall; + case LARGE_MESSAGES: return forLarge; + } + + throw new IllegalStateException(); + } + + public long serializedSize(PingRequest t, int version) + { + return 1; + } + }; +} diff --git a/src/java/org/apache/cassandra/net/PingVerbHandler.java b/src/java/org/apache/cassandra/net/PingVerbHandler.java index d959b919bd62..a70cddc293b5 100644 --- a/src/java/org/apache/cassandra/net/PingVerbHandler.java +++ b/src/java/org/apache/cassandra/net/PingVerbHandler.java @@ -15,17 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.net; -public class PingVerbHandler implements IVerbHandler +class PingVerbHandler implements IVerbHandler { + static final PingVerbHandler instance = new PingVerbHandler(); + @Override - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - MessageOut msg = new MessageOut<>(MessagingService.Verb.REQUEST_RESPONSE, PongMessage.instance, - PongMessage.serializer, - message.payload.connectionType); - MessagingService.instance().sendReply(msg, id, message.from); + MessagingService.instance().send(message.emptyResponse(), message.from(), message.payload.connectionType); } } diff --git a/src/java/org/apache/cassandra/net/PrunableArrayQueue.java b/src/java/org/apache/cassandra/net/PrunableArrayQueue.java new file mode 100644 index 000000000000..1fca43ca2b33 --- /dev/null +++ b/src/java/org/apache/cassandra/net/PrunableArrayQueue.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.function.Predicate; + +/** + * A growing array-based queue that allows efficient bulk in-place removal. + * + * Can think of this queue as if it were an {@link java.util.ArrayDeque} with {@link #prune(Pruner)} method - an efficient + * way to prune the queue in-place that is more expressive, and faster than {@link java.util.ArrayDeque#removeIf(Predicate)}. + * + * The latter has to perform O(n*n) shifts, whereas {@link #prune(Pruner)} only needs O(n) shifts at worst. + */ +final class PrunableArrayQueue +{ + public interface Pruner + { + /** + * @return whether the element should be pruned + * if {@code true}, the element will be removed from the queue, and {@link #onPruned(Object)} will be invoked, + * if {@code false}, the element will be kept, and {@link #onKept(Object)} will be invoked. + */ + boolean shouldPrune(E e); + + void onPruned(E e); + void onKept(E e); + } + + private int capacity; + private E[] buffer; + + /* + * mask = capacity - 1; + * since capacity is a power of 2, value % capacity == value & (capacity - 1) == value & mask + */ + private int mask; + + private int head = 0; + private int tail = 0; + + @SuppressWarnings("unchecked") + PrunableArrayQueue(int requestedCapacity) + { + capacity = Math.max(8, findNextPositivePowerOfTwo(requestedCapacity)); + mask = capacity - 1; + buffer = (E[]) new Object[capacity]; + } + + @SuppressWarnings("UnusedReturnValue") + boolean offer(E e) + { + buffer[tail] = e; + if ((tail = (tail + 1) & mask) == head) + doubleCapacity(); + return true; + } + + E peek() + { + return buffer[head]; + } + + E poll() + { + E result = buffer[head]; + if (null == result) + return null; + + buffer[head] = null; + head = (head + 1) & mask; + + return result; + } + + int size() + { + return (tail - head) & mask; + } + + boolean isEmpty() + { + return head == tail; + } + + /** + * Prunes the queue using the specified {@link Pruner} + * + * @return count of removed elements. + */ + int prune(Pruner pruner) + { + E e; + int removed = 0; + + try + { + int size = size(); + for (int i = 0; i < size; i++) + { + /* + * We start at the tail and work backwards to minimise the number of copies + * as we expect to primarily prune from the front. + */ + int k = (tail - 1 - i) & mask; + e = buffer[k]; + + if (pruner.shouldPrune(e)) + { + buffer[k] = null; + removed++; + pruner.onPruned(e); + } + else + { + if (removed > 0) + { + buffer[(k + removed) & mask] = e; + buffer[k] = null; + } + pruner.onKept(e); + } + } + } + finally + { + head = (head + removed) & mask; + } + + return removed; + } + + @SuppressWarnings("unchecked") + private void doubleCapacity() + { + assert head == tail; + + int newCapacity = capacity << 1; + E[] newBuffer = (E[]) new Object[newCapacity]; + + int headPortionLen = capacity - head; + System.arraycopy(buffer, head, newBuffer, 0, headPortionLen); + System.arraycopy(buffer, 0, newBuffer, headPortionLen, tail); + + head = 0; + tail = capacity; + + capacity = newCapacity; + mask = newCapacity - 1; + buffer = newBuffer; + } + + private static int findNextPositivePowerOfTwo(int value) + { + return 1 << (32 - Integer.numberOfLeadingZeros(value - 1)); + } +} diff --git a/src/java/org/apache/cassandra/net/RateBasedBackPressure.java b/src/java/org/apache/cassandra/net/RateBasedBackPressure.java index 22cec655638d..02d8cce35772 100644 --- a/src/java/org/apache/cassandra/net/RateBasedBackPressure.java +++ b/src/java/org/apache/cassandra/net/RateBasedBackPressure.java @@ -39,6 +39,8 @@ import org.apache.cassandra.utils.TimeSource; import org.apache.cassandra.utils.concurrent.IntervalLock; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + /** * Back-pressure algorithm based on rate limiting according to the ratio between incoming and outgoing rates, computed * over a sliding time window with size equal to write RPC timeout. @@ -84,7 +86,7 @@ public static ParameterizedClass withDefaultParams() public RateBasedBackPressure(Map args) { - this(args, new SystemTimeSource(), DatabaseDescriptor.getWriteRpcTimeout()); + this(args, new SystemTimeSource(), DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS)); } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java b/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java index 9df056e9454a..a15087493c0b 100644 --- a/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java +++ b/src/java/org/apache/cassandra/net/RateBasedBackPressureState.java @@ -61,7 +61,7 @@ class RateBasedBackPressureState extends IntervalLock implements BackPressureSta } @Override - public void onMessageSent(MessageOut message) {} + public void onMessageSent(Message message) {} @Override public void onResponseReceived() diff --git a/src/java/org/apache/cassandra/net/IAsyncCallback.java b/src/java/org/apache/cassandra/net/RequestCallback.java similarity index 61% rename from src/java/org/apache/cassandra/net/IAsyncCallback.java rename to src/java/org/apache/cassandra/net/RequestCallback.java index ceaf0721963f..9ed3a4b296be 100644 --- a/src/java/org/apache/cassandra/net/IAsyncCallback.java +++ b/src/java/org/apache/cassandra/net/RequestCallback.java @@ -17,24 +17,45 @@ */ package org.apache.cassandra.net; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.locator.InetAddressAndPort; + /** - * implementors of IAsyncCallback need to make sure that any public methods - * are threadsafe with respect to response() being called from the message + * implementors of {@link RequestCallback} need to make sure that any public methods + * are threadsafe with respect to {@link #onResponse} being called from the message * service. In particular, if any shared state is referenced, making * response alone synchronized will not suffice. */ -public interface IAsyncCallback +public interface RequestCallback { /** * @param msg response received. */ - void response(MessageIn msg); + void onResponse(Message msg); + + /** + * Called when there is an exception on the remote node or timeout happens + */ + default void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) + { + } + + /** + * @return true if the callback should be invoked on failure + */ + default boolean invokeOnFailure() + { + return false; + } /** * @return true if this callback is on the read path and its latency should be * given as input to the dynamic snitch. */ - boolean isLatencyForSnitch(); + default boolean trackLatencyForSnitch() + { + return false; + } default boolean supportsBackPressure() { diff --git a/src/java/org/apache/cassandra/net/RequestCallbacks.java b/src/java/org/apache/cassandra/net/RequestCallbacks.java new file mode 100644 index 000000000000..fd3a09600114 --- /dev/null +++ b/src/java/org/apache/cassandra/net/RequestCallbacks.java @@ -0,0 +1,382 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeoutException; + +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor; +import org.apache.cassandra.concurrent.StageManager; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.io.IVersionedAsymmetricSerializer; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Replica; +import org.apache.cassandra.metrics.InternodeOutboundMetrics; +import org.apache.cassandra.service.AbstractWriteResponseHandler; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.paxos.Commit; +import org.apache.cassandra.utils.FBUtilities; + +import static java.lang.String.format; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.concurrent.Stage.INTERNAL_RESPONSE; +import static org.apache.cassandra.utils.MonotonicClock.preciseTime; + +/** + * An expiring map of request callbacks. + * + * Used to match response (id, peer) pairs to corresponding {@link RequestCallback}s, or, if said responses + * don't arrive in a timely manner (within verb's timeout), to expire the callbacks. + * + * Since we reuse the same request id for multiple messages now, the map is keyed by (id, peer) tuples + * rather than just id as it used to before 4.0. + */ +public class RequestCallbacks implements OutboundMessageCallbacks +{ + private static final Logger logger = LoggerFactory.getLogger(RequestCallbacks.class); + + private final MessagingService messagingService; + private final ScheduledExecutorService executor = new DebuggableScheduledThreadPoolExecutor("Callback-Map-Reaper"); + private final ConcurrentMap callbacks = new ConcurrentHashMap<>(); + + RequestCallbacks(MessagingService messagingService) + { + this.messagingService = messagingService; + + long expirationInterval = DatabaseDescriptor.getMinRpcTimeout(NANOSECONDS) / 2; + executor.scheduleWithFixedDelay(this::expire, expirationInterval, expirationInterval, NANOSECONDS); + } + + /** + * @return the registered {@link CallbackInfo} for this id and peer, or {@code null} if unset or expired. + */ + @Nullable + CallbackInfo get(long id, InetAddressAndPort peer) + { + return callbacks.get(key(id, peer)); + } + + /** + * Remove and return the {@link CallbackInfo} associated with given id and peer, if known. + */ + @Nullable + CallbackInfo remove(long id, InetAddressAndPort peer) + { + return callbacks.remove(key(id, peer)); + } + + /** + * Register the provided {@link RequestCallback}, inferring expiry and id from the provided {@link Message}. + */ + void addWithExpiration(RequestCallback cb, Message message, InetAddressAndPort to) + { + // mutations need to call the overload with a ConsistencyLevel + assert message.verb() != Verb.MUTATION_REQ && message.verb() != Verb.COUNTER_MUTATION_REQ && message.verb() != Verb.PAXOS_COMMIT_REQ; + CallbackInfo previous = callbacks.put(key(message.id(), to), new CallbackInfo(message, to, cb)); + assert previous == null : format("Callback already exists for id %d/%s! (%s)", message.id(), to, previous); + } + + // FIXME: shouldn't need a special overload for writes; hinting should be part of AbstractWriteResponseHandler + public void addWithExpiration(AbstractWriteResponseHandler cb, + Message message, + Replica to, + ConsistencyLevel consistencyLevel, + boolean allowHints) + { + assert message.verb() == Verb.MUTATION_REQ || message.verb() == Verb.COUNTER_MUTATION_REQ || message.verb() == Verb.PAXOS_COMMIT_REQ; + CallbackInfo previous = callbacks.put(key(message.id(), to.endpoint()), new WriteCallbackInfo(message, to, cb, consistencyLevel, allowHints)); + assert previous == null : format("Callback already exists for id %d/%s! (%s)", message.id(), to.endpoint(), previous); + } + + IVersionedAsymmetricSerializer responseSerializer(long id, InetAddressAndPort peer) + { + CallbackInfo info = get(id, peer); + return info == null ? null : info.responseVerb.serializer(); + } + + @VisibleForTesting + public void removeAndRespond(long id, InetAddressAndPort peer, Message message) + { + CallbackInfo ci = remove(id, peer); + if (null != ci) ci.callback.onResponse(message); + } + + private void removeAndExpire(long id, InetAddressAndPort peer) + { + CallbackInfo ci = remove(id, peer); + if (null != ci) onExpired(ci); + } + + private void expire() + { + long start = preciseTime.now(); + int n = 0; + for (Map.Entry entry : callbacks.entrySet()) + { + if (entry.getValue().isReadyToDieAt(start)) + { + if (callbacks.remove(entry.getKey(), entry.getValue())) + { + n++; + onExpired(entry.getValue()); + } + } + } + logger.trace("Expired {} entries", n); + } + + private void forceExpire() + { + for (Map.Entry entry : callbacks.entrySet()) + if (callbacks.remove(entry.getKey(), entry.getValue())) + onExpired(entry.getValue()); + } + + private void onExpired(CallbackInfo info) + { + messagingService.latencySubscribers.maybeAdd(info.callback, info.peer, info.timeout(), NANOSECONDS); + + InternodeOutboundMetrics.totalExpiredCallbacks.mark(); + messagingService.markExpiredCallback(info.peer); + + if (info.callback.supportsBackPressure()) + messagingService.updateBackPressureOnReceive(info.peer, info.callback, true); + + if (info.invokeOnFailure()) + StageManager.getStage(INTERNAL_RESPONSE).submit(() -> info.callback.onFailure(info.peer, RequestFailureReason.TIMEOUT)); + + // FIXME: this has never belonged here, should be part of onFailure() in AbstractWriteResponseHandler + if (info.shouldHint()) + { + WriteCallbackInfo writeCallbackInfo = ((WriteCallbackInfo) info); + Mutation mutation = writeCallbackInfo.mutation(); + StorageProxy.submitHint(mutation, writeCallbackInfo.getReplica(), null); + } + } + + void shutdownNow(boolean expireCallbacks) + { + executor.shutdownNow(); + if (expireCallbacks) + forceExpire(); + } + + void shutdownGracefully() + { + expire(); + if (!callbacks.isEmpty()) + executor.schedule(this::shutdownGracefully, 100L, MILLISECONDS); + else + executor.shutdownNow(); + } + + void awaitTerminationUntil(long deadlineNanos) throws TimeoutException, InterruptedException + { + if (!executor.isTerminated()) + { + long wait = deadlineNanos - System.nanoTime(); + if (wait <= 0 || !executor.awaitTermination(wait, NANOSECONDS)) + throw new TimeoutException(); + } + } + + @VisibleForTesting + public void unsafeClear() + { + callbacks.clear(); + } + + private static CallbackKey key(long id, InetAddressAndPort peer) + { + return new CallbackKey(id, peer); + } + + private static class CallbackKey + { + final long id; + final InetAddressAndPort peer; + + CallbackKey(long id, InetAddressAndPort peer) + { + this.id = id; + this.peer = peer; + } + + @Override + public boolean equals(Object o) + { + if (!(o instanceof CallbackKey)) + return false; + CallbackKey that = (CallbackKey) o; + return this.id == that.id && this.peer.equals(that.peer); + } + + @Override + public int hashCode() + { + return Long.hashCode(id) + 31 * peer.hashCode(); + } + + @Override + public String toString() + { + return "{id:" + id + ", peer:" + peer + '}'; + } + } + + static class CallbackInfo + { + final long createdAtNanos; + final long expiresAtNanos; + + final InetAddressAndPort peer; + final RequestCallback callback; + + @Deprecated // for 3.0 compatibility purposes only + public final Verb responseVerb; + + private CallbackInfo(Message message, InetAddressAndPort peer, RequestCallback callback) + { + this.createdAtNanos = message.createdAtNanos(); + this.expiresAtNanos = message.expiresAtNanos(); + this.peer = peer; + this.callback = callback; + this.responseVerb = message.verb().responseVerb; + } + + public long timeout() + { + return expiresAtNanos - createdAtNanos; + } + + boolean isReadyToDieAt(long atNano) + { + return atNano > expiresAtNanos; + } + + boolean shouldHint() + { + return false; + } + + boolean invokeOnFailure() + { + return callback.invokeOnFailure(); + } + + public String toString() + { + return "{peer:" + peer + ", callback:" + callback + ", invokeOnFailure:" + invokeOnFailure() + '}'; + } + } + + // FIXME: shouldn't need a specialized container for write callbacks; hinting should be part of + // AbstractWriteResponseHandler implementation. + static class WriteCallbackInfo extends CallbackInfo + { + // either a Mutation, or a Paxos Commit (MessageOut) + private final Object mutation; + private final Replica replica; + + @VisibleForTesting + WriteCallbackInfo(Message message, Replica replica, RequestCallback callback, ConsistencyLevel consistencyLevel, boolean allowHints) + { + super(message, replica.endpoint(), callback); + this.mutation = shouldHint(allowHints, message, consistencyLevel) ? message.payload : null; + //Local writes shouldn't go through messaging service (https://issues.apache.org/jira/browse/CASSANDRA-10477) + //noinspection AssertWithSideEffects + assert !peer.equals(FBUtilities.getBroadcastAddressAndPort()); + this.replica = replica; + } + + public boolean shouldHint() + { + return mutation != null && StorageProxy.shouldHint(replica); + } + + public Replica getReplica() + { + return replica; + } + + public Mutation mutation() + { + return getMutation(mutation); + } + + private static Mutation getMutation(Object object) + { + assert object instanceof Commit || object instanceof Mutation : object; + return object instanceof Commit ? ((Commit) object).makeMutation() + : (Mutation) object; + } + + private static boolean shouldHint(boolean allowHints, Message sentMessage, ConsistencyLevel consistencyLevel) + { + return allowHints && sentMessage.verb() != Verb.COUNTER_MUTATION_REQ && consistencyLevel != ConsistencyLevel.ANY; + } + } + + @Override + public void onOverloaded(Message message, InetAddressAndPort peer) + { + removeAndExpire(message, peer); + } + + @Override + public void onExpired(Message message, InetAddressAndPort peer) + { + removeAndExpire(message, peer); + } + + @Override + public void onFailedSerialize(Message message, InetAddressAndPort peer, int messagingVersion, int bytesWrittenToNetwork, Throwable failure) + { + removeAndExpire(message, peer); + } + + @Override + public void onDiscardOnClose(Message message, InetAddressAndPort peer) + { + removeAndExpire(message, peer); + } + + private void removeAndExpire(Message message, InetAddressAndPort peer) + { + removeAndExpire(message.id(), peer); + + /* in case of a write sent to a different DC, also expire all forwarding targets */ + ForwardingInfo forwardTo = message.forwardTo(); + if (null != forwardTo) + forwardTo.forEach(this::removeAndExpire); + } +} diff --git a/src/java/org/apache/cassandra/net/ResourceLimits.java b/src/java/org/apache/cassandra/net/ResourceLimits.java new file mode 100644 index 000000000000..83e9b432d272 --- /dev/null +++ b/src/java/org/apache/cassandra/net/ResourceLimits.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.atomic.AtomicLongFieldUpdater; + +abstract class ResourceLimits +{ + /** + * Represents permits to utilise a resource and ways to allocate and release them. + * + * Two implementations are currently provided: + * 1. {@link Concurrent}, for shared limits, which is thread-safe; + * 2. {@link Basic}, for limits that are not shared between threads, is not thread-safe. + */ + interface Limit + { + /** + * @return total amount of permits represented by this {@link Limit} - the capacity + */ + long limit(); + + /** + * @return remaining, unallocated permit amount + */ + long remaining(); + + /** + * @return amount of permits currently in use + */ + long using(); + + /** + * Attempts to allocate an amount of permits from this limit. If allocated, MUST eventually + * be released back with {@link #release(long)}. + * + * @return {@code true} if the allocation was successful, {@code false} otherwise + */ + boolean tryAllocate(long amount); + + /** + * @param amount return the amount of permits back to this limit + */ + void release(long amount); + } + + /** + * A thread-safe permit container. + */ + public static class Concurrent implements Limit + { + private final long limit; + + private volatile long using; + private static final AtomicLongFieldUpdater usingUpdater = + AtomicLongFieldUpdater.newUpdater(Concurrent.class, "using"); + + Concurrent(long limit) + { + this.limit = limit; + } + + public long limit() + { + return limit; + } + + public long remaining() + { + return limit - using; + } + + public long using() + { + return using; + } + + public boolean tryAllocate(long amount) + { + long current, next; + do + { + current = using; + next = current + amount; + + if (next > limit) + return false; + } + while (!usingUpdater.compareAndSet(this, current, next)); + + return true; + } + + public void release(long amount) + { + assert amount >= 0; + long using = usingUpdater.addAndGet(this, -amount); + assert using >= 0; + } + } + + /** + * A cheaper, thread-unsafe permit container to be used for unshared limits. + */ + static class Basic implements Limit + { + private final long limit; + private long using; + + Basic(long limit) + { + this.limit = limit; + } + + public long limit() + { + return limit; + } + + public long remaining() + { + return limit - using; + } + + public long using() + { + return using; + } + + public boolean tryAllocate(long amount) + { + if (using + amount > limit) + return false; + + using += amount; + return true; + } + + public void release(long amount) + { + assert amount >= 0 && amount <= using; + using -= amount; + } + } + + /** + * A convenience class that groups a per-endpoint limit with the global one + * to allow allocating/releasing permits from/to both limits as one logical operation. + */ + static class EndpointAndGlobal + { + final Limit endpoint; + final Limit global; + + EndpointAndGlobal(Limit endpoint, Limit global) + { + this.endpoint = endpoint; + this.global = global; + } + + /** + * @return {@code INSUFFICIENT_GLOBAL} if there weren't enough permits in the global limit, or + * {@code INSUFFICIENT_ENDPOINT} if there weren't enough permits in the per-endpoint limit, or + * {@code SUCCESS} if there were enough permits to take from both. + */ + Outcome tryAllocate(long amount) + { + if (!global.tryAllocate(amount)) + return Outcome.INSUFFICIENT_GLOBAL; + + if (endpoint.tryAllocate(amount)) + return Outcome.SUCCESS; + + global.release(amount); + return Outcome.INSUFFICIENT_ENDPOINT; + } + + void release(long amount) + { + endpoint.release(amount); + global.release(amount); + } + } + + enum Outcome { SUCCESS, INSUFFICIENT_ENDPOINT, INSUFFICIENT_GLOBAL } +} diff --git a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java index fe22e42e356b..e5779ab91938 100644 --- a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java +++ b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java @@ -17,45 +17,50 @@ */ package org.apache.cassandra.net; -import java.util.concurrent.TimeUnit; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.tracing.Tracing; -public class ResponseVerbHandler implements IVerbHandler +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +class ResponseVerbHandler implements IVerbHandler { - private static final Logger logger = LoggerFactory.getLogger( ResponseVerbHandler.class ); + public static final ResponseVerbHandler instance = new ResponseVerbHandler(); - public void doVerb(MessageIn message, int id) + private static final Logger logger = LoggerFactory.getLogger(ResponseVerbHandler.class); + + @Override + public void doVerb(Message message) { - long latency = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - MessagingService.instance().getRegisteredCallbackAge(id)); - CallbackInfo callbackInfo = MessagingService.instance().removeRegisteredCallback(id); + RequestCallbacks.CallbackInfo callbackInfo = MessagingService.instance().callbacks.remove(message.id(), message.from()); if (callbackInfo == null) { String msg = "Callback already removed for {} (from {})"; - logger.trace(msg, id, message.from); - Tracing.trace(msg, id, message.from); + logger.trace(msg, message.id(), message.from()); + Tracing.trace(msg, message.id(), message.from()); return; } - Tracing.trace("Processing response from {}", message.from); - IAsyncCallback cb = callbackInfo.callback; + long latencyNanos = approxTime.now() - callbackInfo.createdAtNanos; + Tracing.trace("Processing response from {}", message.from()); + + RequestCallback cb = callbackInfo.callback; if (message.isFailureResponse()) { - ((IAsyncCallbackWithFailure) cb).onFailure(message.from, message.getFailureReason()); + cb.onFailure(message.from(), (RequestFailureReason) message.payload); } else { - //TODO: Should we add latency only in success cases? - MessagingService.instance().maybeAddLatency(cb, message.from, latency); - cb.response(message); + MessagingService.instance().latencySubscribers.maybeAdd(cb, message.from(), latencyNanos, NANOSECONDS); + cb.onResponse(message); } if (callbackInfo.callback.supportsBackPressure()) { - MessagingService.instance().updateBackPressureOnReceive(message.from, cb, false); + MessagingService.instance().updateBackPressureOnReceive(message.from(), cb, false); } } } diff --git a/src/java/org/apache/cassandra/net/ShareableBytes.java b/src/java/org/apache/cassandra/net/ShareableBytes.java new file mode 100644 index 000000000000..feea25e34394 --- /dev/null +++ b/src/java/org/apache/cassandra/net/ShareableBytes.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; + +import org.apache.cassandra.utils.memory.BufferPool; + +/** + * A wrapper for possibly sharing portions of a single, {@link BufferPool} managed, {@link ByteBuffer}; + * optimised for the case where no sharing is necessary. + * + * When sharing is necessary, {@link #share()} method must be invoked by the owning thread + * before a {@link ShareableBytes} instance can be shared with another thread. + */ +class ShareableBytes +{ + private final ByteBuffer bytes; + private final ShareableBytes owner; + private volatile int count; + + private static final int UNSHARED = -1; + private static final int RELEASED = 0; + private static final AtomicIntegerFieldUpdater countUpdater = + AtomicIntegerFieldUpdater.newUpdater(ShareableBytes.class, "count"); + + ShareableBytes(ByteBuffer bytes) + { + this.count = UNSHARED; + this.owner = this; + this.bytes = bytes; + } + + ShareableBytes(ShareableBytes owner, ByteBuffer bytes) + { + this.owner = owner; + this.bytes = bytes; + } + + ByteBuffer get() + { + assert owner.count != 0; + return bytes; + } + + boolean hasRemaining() + { + return bytes.hasRemaining(); + } + + int remaining() + { + return bytes.remaining(); + } + + void skipBytes(int skipBytes) + { + bytes.position(bytes.position() + skipBytes); + } + + void consume() + { + bytes.position(bytes.limit()); + } + + /** + * Ensure this ShareableBytes will use atomic operations for updating its count from now on. + * The first invocation must occur while the calling thread has exclusive access (though there may be more + * than one 'owner', these must all either be owned by the calling thread or otherwise not being used) + */ + ShareableBytes share() + { + int count = owner.count; + if (count < 0) + owner.count = -count; + return this; + } + + private ShareableBytes retain() + { + owner.doRetain(); + return this; + } + + private void doRetain() + { + int count = this.count; + if (count < 0) + { + countUpdater.lazySet(this, count - 1); + return; + } + + while (true) + { + if (count == RELEASED) + throw new IllegalStateException("Attempted to reference an already released SharedByteBuffer"); + + if (countUpdater.compareAndSet(this, count, count + 1)) + return; + + count = this.count; + } + } + + void release() + { + owner.doRelease(); + } + + private void doRelease() + { + int count = this.count; + + if (count < 0) + countUpdater.lazySet(this, count += 1); + else if (count > 0) + count = countUpdater.decrementAndGet(this); + else + throw new IllegalStateException("Already released"); + + if (count == RELEASED) + BufferPool.put(bytes); + } + + boolean isReleased() + { + return owner.count == RELEASED; + } + + /** + * Create a slice over the next {@code length} bytes, consuming them from our buffer, and incrementing the owner count + */ + ShareableBytes sliceAndConsume(int length) + { + int begin = bytes.position(); + int end = begin + length; + ShareableBytes result = slice(begin, end); + bytes.position(end); + return result; + } + + /** + * Create a new slice, incrementing the number of owners (making it shared if it was previously unshared) + */ + ShareableBytes slice(int begin, int end) + { + ByteBuffer slice = bytes.duplicate(); + slice.position(begin).limit(end); + return new ShareableBytes(owner.retain(), slice); + } + + static ShareableBytes wrap(ByteBuffer buffer) + { + return new ShareableBytes(buffer); + } +} + diff --git a/src/java/org/apache/cassandra/net/SharedDefaultFileRegion.java b/src/java/org/apache/cassandra/net/SharedDefaultFileRegion.java new file mode 100644 index 000000000000..6b47c22d240e --- /dev/null +++ b/src/java/org/apache/cassandra/net/SharedDefaultFileRegion.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.nio.channels.FileChannel; +import java.util.concurrent.atomic.AtomicInteger; + +import io.netty.channel.DefaultFileRegion; +import org.apache.cassandra.utils.concurrent.Ref; +import org.apache.cassandra.utils.concurrent.RefCounted; + +/** + * Netty's DefaultFileRegion closes the underlying FileChannel as soon as + * the refCnt() for the region drops to zero, this is an implementation of + * the DefaultFileRegion that doesn't close the FileChannel. + * + * See {@link AsyncChannelOutputPlus} for its usage. + */ +public class SharedDefaultFileRegion extends DefaultFileRegion +{ + public static class SharedFileChannel + { + // we don't call .ref() on this, because it would generate a lot of PhantomReferences and GC overhead, + // but we use it to ensure we can spot memory leaks + final Ref ref; + final AtomicInteger refCount = new AtomicInteger(1); + + SharedFileChannel(FileChannel fileChannel) + { + this.ref = new Ref<>(fileChannel, new RefCounted.Tidy() + { + public void tidy() throws Exception + { + // don't mind invoking this on eventLoop, as only used with sendFile which is also blocking + // so must use streaming eventLoop + fileChannel.close(); + } + + public String name() + { + return "SharedFileChannel[" + fileChannel.toString() + ']'; + } + }); + } + + public void release() + { + if (0 == refCount.decrementAndGet()) + ref.release(); + } + } + + private final SharedFileChannel shared; + private boolean deallocated = false; + + SharedDefaultFileRegion(SharedFileChannel shared, long position, long count) + { + super(shared.ref.get(), position, count); + this.shared = shared; + if (1 >= this.shared.refCount.incrementAndGet()) + throw new IllegalStateException(); + } + + @Override + protected void deallocate() + { + if (deallocated) + return; + deallocated = true; + shared.release(); + } + + public static SharedFileChannel share(FileChannel fileChannel) + { + return new SharedFileChannel(fileChannel); + } +} diff --git a/src/java/org/apache/cassandra/net/SocketFactory.java b/src/java/org/apache/cassandra/net/SocketFactory.java new file mode 100644 index 000000000000..18bb0d5c70f0 --- /dev/null +++ b/src/java/org/apache/cassandra/net/SocketFactory.java @@ -0,0 +1,327 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.nio.channels.ClosedChannelException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; +import javax.annotation.Nullable; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLParameters; + +import com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.bootstrap.Bootstrap; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.Channel; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoop; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.MultithreadEventLoopGroup; +import io.netty.channel.SingleThreadEventLoop; +import io.netty.channel.epoll.EpollChannelOption; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.epoll.EpollServerSocketChannel; +import io.netty.channel.epoll.EpollSocketChannel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.channel.unix.Errors; +import io.netty.handler.ssl.OpenSsl; +import io.netty.handler.ssl.SslContext; +import io.netty.handler.ssl.SslHandler; +import io.netty.util.concurrent.DefaultThreadFactory; +import io.netty.util.concurrent.EventExecutor; +import io.netty.util.concurrent.MultithreadEventExecutorGroup; +import io.netty.util.concurrent.SingleThreadEventExecutor; +import io.netty.util.internal.logging.InternalLoggerFactory; +import io.netty.util.internal.logging.Slf4JLoggerFactory; +import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.service.NativeTransportService; +import org.apache.cassandra.utils.ExecutorUtils; +import org.apache.cassandra.utils.FBUtilities; + +import static io.netty.channel.unix.Errors.ERRNO_ECONNRESET_NEGATIVE; +import static io.netty.channel.unix.Errors.ERROR_ECONNREFUSED_NEGATIVE; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.utils.Throwables.isCausedBy; + +/** + * A factory for building Netty {@link Channel}s. Channels here are setup with a pipeline to participate + * in the internode protocol handshake, either the inbound or outbound side as per the method invoked. + */ +public final class SocketFactory +{ + private static final Logger logger = LoggerFactory.getLogger(SocketFactory.class); + + private static final int EVENT_THREADS = Integer.getInteger(Config.PROPERTY_PREFIX + "internode-event-threads", FBUtilities.getAvailableProcessors()); + + public enum Provider { EPOLL, NIO } + private static final Provider DEFAULT_PROVIDER = NativeTransportService.useEpoll() ? Provider.EPOLL : Provider.NIO; + + /** a useful addition for debugging; simply set to true to get more data in your logs */ + static final boolean WIRETRACE = false; + static + { + if (WIRETRACE) + InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE); + } + + private final EventLoopGroup acceptGroup; + private final EventLoopGroup defaultGroup; + // we need a separate EventLoopGroup for outbound streaming because sendFile is blocking + private final EventLoopGroup outboundStreamingGroup; + final ExecutorService synchronousWorkExecutor = Executors.newCachedThreadPool(new NamedThreadFactory("Messaging-SynchronousWork")); + + SocketFactory() { this(DEFAULT_PROVIDER); } + SocketFactory(Provider provider) + { + this.acceptGroup = getEventLoopGroup(provider, 1, "Messaging-AcceptLoop"); + this.defaultGroup = getEventLoopGroup(provider, EVENT_THREADS, NamedThreadFactory.globalPrefix() + "Messaging-EventLoop"); + this.outboundStreamingGroup = getEventLoopGroup(provider, EVENT_THREADS, "Streaming-EventLoop"); + assert provider == providerOf(acceptGroup) + && provider == providerOf(defaultGroup) + && provider == providerOf(outboundStreamingGroup); + } + + private static EventLoopGroup getEventLoopGroup(Provider provider, int threadCount, String threadNamePrefix) + { + switch (provider) + { + case EPOLL: + logger.debug("using netty epoll event loop for pool prefix {}", threadNamePrefix); + return overwriteMPSCQueues(new EpollEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix, true))); + case NIO: + logger.debug("using netty nio event loop for pool prefix {}", threadNamePrefix); + return overwriteMPSCQueues(new NioEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix, true))); + default: + throw new IllegalStateException(); + } + } + + private static Provider providerOf(EventLoopGroup eventLoopGroup) + { + while (eventLoopGroup instanceof SingleThreadEventLoop) + eventLoopGroup = ((SingleThreadEventLoop) eventLoopGroup).parent(); + + if (eventLoopGroup instanceof EpollEventLoopGroup) + return Provider.EPOLL; + if (eventLoopGroup instanceof NioEventLoopGroup) + return Provider.NIO; + throw new IllegalStateException(); + } + + static Bootstrap newBootstrap(EventLoop eventLoop, int tcpUserTimeoutInMS) + { + if (eventLoop == null) + throw new IllegalArgumentException("must provide eventLoop"); + + Bootstrap bootstrap = new Bootstrap() + .group(eventLoop) + .option(ChannelOption.ALLOCATOR, GlobalBufferPoolAllocator.instance) + .option(ChannelOption.SO_KEEPALIVE, true); + + switch (providerOf(eventLoop)) + { + case EPOLL: + bootstrap.channel(EpollSocketChannel.class); + bootstrap.option(EpollChannelOption.TCP_USER_TIMEOUT, tcpUserTimeoutInMS); + break; + case NIO: + bootstrap.channel(NioSocketChannel.class); + } + return bootstrap; + } + + ServerBootstrap newServerBootstrap() + { + return newServerBootstrap(acceptGroup, defaultGroup); + } + + private static ServerBootstrap newServerBootstrap(EventLoopGroup acceptGroup, EventLoopGroup defaultGroup) + { + ServerBootstrap bootstrap = new ServerBootstrap() + .group(acceptGroup, defaultGroup) + .option(ChannelOption.ALLOCATOR, GlobalBufferPoolAllocator.instance) + .option(ChannelOption.SO_REUSEADDR, true); + + switch (providerOf(defaultGroup)) + { + case EPOLL: + bootstrap.channel(EpollServerSocketChannel.class); + break; + case NIO: + bootstrap.channel(NioServerSocketChannel.class); + } + + return bootstrap; + } + + /** + * Creates a new {@link SslHandler} from provided SslContext. + * @param peer enables endpoint verification for remote address when not null + */ + static SslHandler newSslHandler(Channel channel, SslContext sslContext, @Nullable InetSocketAddress peer) + { + if (peer == null) + return sslContext.newHandler(channel.alloc()); + + logger.debug("Creating SSL handler for {}:{}", peer.getHostString(), peer.getPort()); + SslHandler sslHandler = sslContext.newHandler(channel.alloc(), peer.getHostString(), peer.getPort()); + SSLEngine engine = sslHandler.engine(); + SSLParameters sslParameters = engine.getSSLParameters(); + sslParameters.setEndpointIdentificationAlgorithm("HTTPS"); + engine.setSSLParameters(sslParameters); + return sslHandler; + } + + static String encryptionLogStatement(EncryptionOptions options) + { + if (options == null) + return "disabled"; + + String encryptionType = OpenSsl.isAvailable() ? "openssl" : "jdk"; + return "enabled (" + encryptionType + ')'; + } + + EventLoopGroup defaultGroup() + { + return defaultGroup; + } + + public EventLoopGroup outboundStreamingGroup() + { + return outboundStreamingGroup; + } + + public void shutdownNow() + { + acceptGroup.shutdownGracefully(0, 2, SECONDS); + defaultGroup.shutdownGracefully(0, 2, SECONDS); + outboundStreamingGroup.shutdownGracefully(0, 2, SECONDS); + synchronousWorkExecutor.shutdownNow(); + } + + void awaitTerminationUntil(long deadlineNanos) throws InterruptedException, TimeoutException + { + List groups = ImmutableList.of(acceptGroup, defaultGroup, outboundStreamingGroup, synchronousWorkExecutor); + ExecutorUtils.awaitTerminationUntil(deadlineNanos, groups); + } + + static boolean isConnectionReset(Throwable t) + { + if (t instanceof ClosedChannelException) + return true; + if (t instanceof ConnectException) + return true; + if (t instanceof Errors.NativeIoException) + { + int errorCode = ((Errors.NativeIoException) t).expectedErr(); + return errorCode == ERRNO_ECONNRESET_NEGATIVE || errorCode != ERROR_ECONNREFUSED_NEGATIVE; + } + return IOException.class == t.getClass() && ("Broken pipe".equals(t.getMessage()) || "Connection reset by peer".equals(t.getMessage())); + } + + static boolean isCausedByConnectionReset(Throwable t) + { + return isCausedBy(t, SocketFactory::isConnectionReset); + } + + static String channelId(InetAddressAndPort from, InetSocketAddress realFrom, InetAddressAndPort to, InetSocketAddress realTo, ConnectionType type, String id) + { + return addressId(from, realFrom) + "->" + addressId(to, realTo) + '-' + type + '-' + id; + } + + static String addressId(InetAddressAndPort address, InetSocketAddress realAddress) + { + String str = address.toString(); + if (!address.address.equals(realAddress.getAddress()) || address.port != realAddress.getPort()) + str += '(' + InetAddressAndPort.toString(realAddress.getAddress(), realAddress.getPort()) + ')'; + return str; + } + + static String channelId(InetAddressAndPort from, InetAddressAndPort to, ConnectionType type, String id) + { + return from + "->" + to + '-' + type + '-' + id; + } + + /** + * The default task queue used by {@code NioEventLoop} and {@code EpollEventLoop} is {@code MpscUnboundedArrayQueue}, + * provided by JCTools. While efficient, it has an undesirable quality for a queue backing an event loop: it is + * not non-blocking, and can cause the event loop to busy-spin while waiting for a partially completed task + * offer, if the producer thread has been suspended mid-offer. Sadly, there is currently no way to work around + * this behaviour in application-logic. + * + * As it happens, however, we have an MPSC queue implementation that is perfectly fit for this purpose - + * {@link ManyToOneConcurrentLinkedQueue}, that is non-blocking, and already used throughout the codebase. + * + * Unfortunately, there is no Netty API or to override the default queue, so we have to resort to reflection, + * for now. + * + * We filed a Netty issue asking for this capability to be provided cleanly: + * https://github.com/netty/netty/issues/9105, and hopefully Netty will implement it some day. When and if + * that happens, this reflection-based workaround should be removed. + */ + private static EventLoopGroup overwriteMPSCQueues(MultithreadEventLoopGroup eventLoopGroup) + { + try + { + for (EventExecutor eventExecutor : (EventExecutor[]) childrenField.get(eventLoopGroup)) + { + SingleThreadEventLoop eventLoop = (SingleThreadEventLoop) eventExecutor; + taskQueueField.set(eventLoop, new ManyToOneConcurrentLinkedQueue<>()); + tailTasksField.set(eventLoop, new ManyToOneConcurrentLinkedQueue<>()); + } + return eventLoopGroup; + } + catch (IllegalAccessException e) + { + throw new IllegalStateException(e); + } + } + + private static final Field childrenField, taskQueueField, tailTasksField; + static + { + try + { + childrenField = MultithreadEventExecutorGroup.class.getDeclaredField("children"); + taskQueueField = SingleThreadEventExecutor.class.getDeclaredField("taskQueue"); + tailTasksField = SingleThreadEventLoop.class.getDeclaredField("tailTasks"); + + childrenField.setAccessible(true); + taskQueueField.setAccessible(true); + tailTasksField.setAccessible(true); + } + catch (NoSuchFieldException e) + { + throw new IllegalStateException(e); + } + } +} diff --git a/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java b/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java index 8e3747068c99..b901338006be 100644 --- a/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java +++ b/src/java/org/apache/cassandra/net/StartupClusterConnectivityChecker.java @@ -42,12 +42,11 @@ import org.apache.cassandra.gms.IEndpointStateChangeSubscriber; import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.net.MessagingService.Verb.PING; -import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.LARGE_MESSAGE; -import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE; +import static org.apache.cassandra.net.Verb.PING_REQ; +import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES; +import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES; public class StartupClusterConnectivityChecker { @@ -149,11 +148,11 @@ public boolean execute(Set peers, Function peers, Function peers, Map dcToRemainingPeers, AckMap acks, Function getDatacenter) { - IAsyncCallback responseHandler = new IAsyncCallback() - { - public boolean isLatencyForSnitch() - { - return false; - } - - public void response(MessageIn msg) + RequestCallback responseHandler = msg -> { + if (acks.incrementAndCheck(msg.from())) { - if (acks.incrementAndCheck(msg.from)) - { - String datacenter = getDatacenter.apply(msg.from); - // We have to check because we might only have the local DC in the map - if (dcToRemainingPeers.containsKey(datacenter)) - dcToRemainingPeers.get(datacenter).countDown(); - } + String datacenter = getDatacenter.apply(msg.from()); + // We have to check because we might only have the local DC in the map + if (dcToRemainingPeers.containsKey(datacenter)) + dcToRemainingPeers.get(datacenter).countDown(); } }; - MessageOut smallChannelMessageOut = new MessageOut<>(PING, PingMessage.smallChannelMessage, - PingMessage.serializer, SMALL_MESSAGE); - MessageOut largeChannelMessageOut = new MessageOut<>(PING, PingMessage.largeChannelMessage, - PingMessage.serializer, LARGE_MESSAGE); + Message small = Message.out(PING_REQ, PingRequest.forSmall); + Message large = Message.out(PING_REQ, PingRequest.forLarge); for (InetAddressAndPort peer : peers) { - MessagingService.instance().sendRR(smallChannelMessageOut, peer, responseHandler); - MessagingService.instance().sendRR(largeChannelMessageOut, peer, responseHandler); + MessagingService.instance().sendWithCallback(small, peer, responseHandler, SMALL_MESSAGES); + MessagingService.instance().sendWithCallback(large, peer, responseHandler, LARGE_MESSAGES); } } diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java new file mode 100644 index 000000000000..6463a5a1ed1b --- /dev/null +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -0,0 +1,331 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import java.util.function.ToLongFunction; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.batchlog.Batch; +import org.apache.cassandra.batchlog.BatchRemoveVerbHandler; +import org.apache.cassandra.batchlog.BatchStoreVerbHandler; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.CounterMutation; +import org.apache.cassandra.db.CounterMutationVerbHandler; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.MutationVerbHandler; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.ReadCommandVerbHandler; +import org.apache.cassandra.db.ReadRepairVerbHandler; +import org.apache.cassandra.db.ReadResponse; +import org.apache.cassandra.db.SnapshotCommand; +import org.apache.cassandra.db.TruncateResponse; +import org.apache.cassandra.db.TruncateVerbHandler; +import org.apache.cassandra.db.TruncateRequest; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.gms.GossipDigestAck; +import org.apache.cassandra.gms.GossipDigestAck2; +import org.apache.cassandra.gms.GossipDigestAck2VerbHandler; +import org.apache.cassandra.gms.GossipDigestAckVerbHandler; +import org.apache.cassandra.gms.GossipDigestSyn; +import org.apache.cassandra.gms.GossipDigestSynVerbHandler; +import org.apache.cassandra.gms.GossipShutdownVerbHandler; +import org.apache.cassandra.hints.HintMessage; +import org.apache.cassandra.hints.HintVerbHandler; +import org.apache.cassandra.io.IVersionedAsymmetricSerializer; +import org.apache.cassandra.repair.RepairMessageVerbHandler; +import org.apache.cassandra.repair.messages.RepairMessage; +import org.apache.cassandra.schema.SchemaPullVerbHandler; +import org.apache.cassandra.schema.SchemaPushVerbHandler; +import org.apache.cassandra.schema.SchemaVersionVerbHandler; +import org.apache.cassandra.utils.BooleanSerializer; +import org.apache.cassandra.service.EchoVerbHandler; +import org.apache.cassandra.service.SnapshotVerbHandler; +import org.apache.cassandra.service.paxos.Commit; +import org.apache.cassandra.service.paxos.CommitVerbHandler; +import org.apache.cassandra.service.paxos.PrepareResponse; +import org.apache.cassandra.service.paxos.PrepareVerbHandler; +import org.apache.cassandra.service.paxos.ProposeVerbHandler; +import org.apache.cassandra.streaming.ReplicationDoneVerbHandler; +import org.apache.cassandra.utils.UUIDSerializer; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.concurrent.Stage.*; +import static org.apache.cassandra.concurrent.Stage.INTERNAL_RESPONSE; +import static org.apache.cassandra.concurrent.Stage.MISC; +import static org.apache.cassandra.net.VerbTimeouts.*; +import static org.apache.cassandra.net.Verb.Priority.*; +import static org.apache.cassandra.schema.MigrationManager.MigrationsSerializer; + +/** + * Note that priorities except P0 are presently unused. P0 corresponds to urgent, i.e. what used to be the "Gossip" connection. + */ +public enum Verb +{ + MUTATION_RSP (60, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + MUTATION_REQ (0, P3, writeTimeout, MUTATION, () -> Mutation.serializer, () -> MutationVerbHandler.instance, MUTATION_RSP ), + HINT_RSP (61, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + HINT_REQ (1, P4, writeTimeout, MUTATION, () -> HintMessage.serializer, () -> HintVerbHandler.instance, HINT_RSP ), + READ_REPAIR_RSP (62, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + READ_REPAIR_REQ (2, P1, writeTimeout, MUTATION, () -> Mutation.serializer, () -> ReadRepairVerbHandler.instance, READ_REPAIR_RSP ), + BATCH_STORE_RSP (65, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + BATCH_STORE_REQ (5, P3, writeTimeout, MUTATION, () -> Batch.serializer, () -> BatchStoreVerbHandler.instance, BATCH_STORE_RSP ), + BATCH_REMOVE_RSP (66, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + BATCH_REMOVE_REQ (6, P3, writeTimeout, MUTATION, () -> UUIDSerializer.serializer, () -> BatchRemoveVerbHandler.instance, BATCH_REMOVE_RSP ), + + PAXOS_PREPARE_RSP (93, P2, writeTimeout, REQUEST_RESPONSE, () -> PrepareResponse.serializer, () -> ResponseVerbHandler.instance ), + PAXOS_PREPARE_REQ (33, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> PrepareVerbHandler.instance, PAXOS_PREPARE_RSP ), + PAXOS_PROPOSE_RSP (94, P2, writeTimeout, REQUEST_RESPONSE, () -> BooleanSerializer.serializer, () -> ResponseVerbHandler.instance ), + PAXOS_PROPOSE_REQ (34, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> ProposeVerbHandler.instance, PAXOS_PROPOSE_RSP ), + PAXOS_COMMIT_RSP (95, P2, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + PAXOS_COMMIT_REQ (35, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> CommitVerbHandler.instance, PAXOS_COMMIT_RSP ), + + TRUNCATE_RSP (79, P0, truncateTimeout, REQUEST_RESPONSE, () -> TruncateResponse.serializer, () -> ResponseVerbHandler.instance ), + TRUNCATE_REQ (19, P0, truncateTimeout, MUTATION, () -> TruncateRequest.serializer, () -> TruncateVerbHandler.instance, TRUNCATE_RSP ), + + COUNTER_MUTATION_RSP (84, P1, counterTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + COUNTER_MUTATION_REQ (24, P2, counterTimeout, COUNTER_MUTATION, () -> CounterMutation.serializer, () -> CounterMutationVerbHandler.instance, COUNTER_MUTATION_RSP), + + READ_RSP (63, P2, readTimeout, REQUEST_RESPONSE, () -> ReadResponse.serializer, () -> ResponseVerbHandler.instance ), + READ_REQ (3, P3, readTimeout, READ, () -> ReadCommand.serializer, () -> ReadCommandVerbHandler.instance, READ_RSP ), + RANGE_RSP (69, P2, rangeTimeout, REQUEST_RESPONSE, () -> ReadResponse.serializer, () -> ResponseVerbHandler.instance ), + RANGE_REQ (9, P3, rangeTimeout, READ, () -> ReadCommand.serializer, () -> ReadCommandVerbHandler.instance, RANGE_RSP ), + + GOSSIP_DIGEST_SYN (14, P0, longTimeout, GOSSIP, () -> GossipDigestSyn.serializer, () -> GossipDigestSynVerbHandler.instance ), + GOSSIP_DIGEST_ACK (15, P0, longTimeout, GOSSIP, () -> GossipDigestAck.serializer, () -> GossipDigestAckVerbHandler.instance ), + GOSSIP_DIGEST_ACK2 (16, P0, longTimeout, GOSSIP, () -> GossipDigestAck2.serializer, () -> GossipDigestAck2VerbHandler.instance ), + GOSSIP_SHUTDOWN (29, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> GossipShutdownVerbHandler.instance ), + + ECHO_RSP (91, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + ECHO_REQ (31, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> EchoVerbHandler.instance, ECHO_RSP ), + PING_RSP (97, P1, pingTimeout, GOSSIP, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + PING_REQ (37, P1, pingTimeout, GOSSIP, () -> PingRequest.serializer, () -> PingVerbHandler.instance, PING_RSP ), + + // P1 because messages can be arbitrarily large or aren't crucial + SCHEMA_PUSH_RSP (98, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + SCHEMA_PUSH_REQ (18, P1, rpcTimeout, MIGRATION, () -> MigrationsSerializer.instance, () -> SchemaPushVerbHandler.instance, SCHEMA_PUSH_RSP ), + SCHEMA_PULL_RSP (88, P1, rpcTimeout, MIGRATION, () -> MigrationsSerializer.instance, () -> ResponseVerbHandler.instance ), + SCHEMA_PULL_REQ (28, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> SchemaPullVerbHandler.instance, SCHEMA_PULL_RSP ), + SCHEMA_VERSION_RSP (80, P1, rpcTimeout, MIGRATION, () -> UUIDSerializer.serializer, () -> ResponseVerbHandler.instance ), + SCHEMA_VERSION_REQ (20, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> SchemaVersionVerbHandler.instance, SCHEMA_VERSION_RSP ), + REPAIR_RSP (92, P1, rpcTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + REPAIR_REQ (32, P1, rpcTimeout, ANTI_ENTROPY, () -> RepairMessage.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + + REPLICATION_DONE_RSP (82, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + REPLICATION_DONE_REQ (22, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ReplicationDoneVerbHandler.instance, REPLICATION_DONE_RSP), + SNAPSHOT_RSP (87, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + SNAPSHOT_REQ (27, P0, rpcTimeout, MISC, () -> SnapshotCommand.serializer, () -> SnapshotVerbHandler.instance, SNAPSHOT_RSP ), + + // generic failure response + FAILURE_RSP (99, P0, noTimeout, REQUEST_RESPONSE, () -> RequestFailureReason.serializer, () -> ResponseVerbHandler.instance ), + + // dummy verbs + _TRACE (30, P1, rpcTimeout, TRACING, () -> NoPayload.serializer, () -> null ), + _SAMPLE (42, P1, rpcTimeout, INTERNAL_RESPONSE, () -> NoPayload.serializer, () -> null ), + _TEST_1 (10, P0, writeTimeout, IMMEDIATE, () -> NoPayload.serializer, () -> null ), + _TEST_2 (11, P1, rpcTimeout, IMMEDIATE, () -> NoPayload.serializer, () -> null ), + + @Deprecated + REQUEST_RSP (4, P1, rpcTimeout, REQUEST_RESPONSE, () -> null, () -> ResponseVerbHandler.instance ), + @Deprecated + INTERNAL_RSP (23, P1, rpcTimeout, INTERNAL_RESPONSE, () -> null, () -> ResponseVerbHandler.instance ), + + // largest used ID: 99 + ; + + public static final List VERBS = ImmutableList.copyOf(Verb.values()); + + public enum Priority + { + P0, // sends on the urgent connection (i.e. for Gossip, Echo) + P1, // small or empty responses + P2, // larger messages that can be dropped but who have a larger impact on system stability (e.g. READ_REPAIR, READ_RSP) + P3, + P4 + } + + public final int id; + public final Priority priority; + public final Stage stage; + + /** + * Messages we receive from peers have a Verb that tells us what kind of message it is. + * Most of the time, this is enough to determine how to deserialize the message payload. + * The exception is the REQUEST_RSP verb, which just means "a response to something you told me to do." + * Traditionally, this was fine since each VerbHandler knew what type of payload it expected, and + * handled the deserialization itself. Now that we do that in ITC, to avoid the extra copy to an + * intermediary byte[] (See CASSANDRA-3716), we need to wire that up to the CallbackInfo object + * (see below). + * + * NOTE: we use a Supplier to avoid loading the dependent classes until necessary. + */ + private final Supplier> serializer; + private final Supplier> handler; + + final Verb responseVerb; + + private final ToLongFunction expiration; + + + /** + * Verbs it's okay to drop if the request has been queued longer than the request timeout. These + * all correspond to client requests or something triggered by them; we don't want to + * drop internal messages like bootstrap or repair notifications. + */ + Verb(int id, Priority priority, ToLongFunction expiration, Stage stage, Supplier> serializer, Supplier> handler) + { + this(id, priority, expiration, stage, serializer, handler, null); + } + + Verb(int id, Priority priority, ToLongFunction expiration, Stage stage, Supplier> serializer, Supplier> handler, Verb responseVerb) + { + this.stage = stage; + if (id < 0) + throw new IllegalArgumentException("Verb id must be non-negative, got " + id + " for verb " + name()); + + this.id = id; + this.priority = priority; + this.serializer = serializer; + this.handler = handler; + this.responseVerb = responseVerb; + this.expiration = expiration; + } + + public IVersionedAsymmetricSerializer serializer() + { + return (IVersionedAsymmetricSerializer) serializer.get(); + } + + public IVerbHandler handler() + { + return (IVerbHandler) handler.get(); + } + + public long expiresAtNanos(long nowNanos) + { + return nowNanos + expiresAfterNanos(); + } + + public long expiresAfterNanos() + { + return expiration.applyAsLong(NANOSECONDS); + } + + // this is a little hacky, but reduces the number of parameters up top + public boolean isResponse() + { + return handler.get() == ResponseVerbHandler.instance; + } + + Verb toPre40Verb() + { + if (!isResponse()) + return this; + if (priority == P0) + return INTERNAL_RSP; + return REQUEST_RSP; + } + + @VisibleForTesting + Supplier> unsafeSetHandler(Supplier> handler) throws NoSuchFieldException, IllegalAccessException + { + Supplier> original = this.handler; + Field field = Verb.class.getDeclaredField("handler"); + field.setAccessible(true); + Field modifiers = Field.class.getDeclaredField("modifiers"); + modifiers.setAccessible(true); + modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL); + field.set(this, handler); + return original; + } + + @VisibleForTesting + Supplier> unsafeSetSerializer(Supplier> serializer) throws NoSuchFieldException, IllegalAccessException + { + Supplier> original = this.serializer; + Field field = Verb.class.getDeclaredField("serializer"); + field.setAccessible(true); + Field modifiers = Field.class.getDeclaredField("modifiers"); + modifiers.setAccessible(true); + modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL); + field.set(this, serializer); + return original; + } + + @VisibleForTesting + ToLongFunction unsafeSetExpiration(ToLongFunction expiration) throws NoSuchFieldException, IllegalAccessException + { + ToLongFunction original = this.expiration; + Field field = Verb.class.getDeclaredField("expiration"); + field.setAccessible(true); + Field modifiers = Field.class.getDeclaredField("modifiers"); + modifiers.setAccessible(true); + modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL); + field.set(this, expiration); + return original; + } + + private static final Verb[] idToVerbMap; + + static + { + Verb[] verbs = values(); + int max = -1; + for (Verb v : verbs) + max = Math.max(v.id, max); + + Verb[] idMap = new Verb[max + 1]; + for (Verb v : verbs) + { + if (idMap[v.id] != null) + throw new IllegalArgumentException("cannot have two verbs that map to the same id: " + v + " and " + idMap[v.id]); + idMap[v.id] = v; + } + + idToVerbMap = idMap; + } + + static Verb fromId(int id) + { + Verb verb = id >= 0 && id < idToVerbMap.length ? idToVerbMap[id] : null; + if (verb == null) + throw new IllegalArgumentException("Unknown verb id " + id); + return verb; + } +} + +@SuppressWarnings("unused") +class VerbTimeouts +{ + static final ToLongFunction rpcTimeout = DatabaseDescriptor::getRpcTimeout; + static final ToLongFunction writeTimeout = DatabaseDescriptor::getWriteRpcTimeout; + static final ToLongFunction readTimeout = DatabaseDescriptor::getReadRpcTimeout; + static final ToLongFunction rangeTimeout = DatabaseDescriptor::getRangeRpcTimeout; + static final ToLongFunction counterTimeout = DatabaseDescriptor::getCounterWriteRpcTimeout; + static final ToLongFunction truncateTimeout = DatabaseDescriptor::getTruncateRpcTimeout; + static final ToLongFunction pingTimeout = DatabaseDescriptor::getPingTimeout; + static final ToLongFunction longTimeout = units -> Math.max(DatabaseDescriptor.getRpcTimeout(units), units.convert(5L, TimeUnit.MINUTES)); + static final ToLongFunction noTimeout = units -> { throw new IllegalStateException(); }; +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java b/src/java/org/apache/cassandra/net/WriteCallbackInfo.java deleted file mode 100644 index c54e7dcba96f..000000000000 --- a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.cassandra.net; - -import org.apache.cassandra.db.ConsistencyLevel; -import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.service.StorageProxy; -import org.apache.cassandra.service.paxos.Commit; -import org.apache.cassandra.utils.FBUtilities; - -public class WriteCallbackInfo extends CallbackInfo -{ - // either a Mutation, or a Paxos Commit (MessageOut) - private final Object mutation; - private final Replica replica; - - public WriteCallbackInfo(Replica replica, - IAsyncCallback callback, - MessageOut message, - IVersionedSerializer serializer, - ConsistencyLevel consistencyLevel, - boolean allowHints) - { - super(replica.endpoint(), callback, serializer, true); - assert message != null; - this.mutation = shouldHint(allowHints, message, consistencyLevel); - //Local writes shouldn't go through messaging service (https://issues.apache.org/jira/browse/CASSANDRA-10477) - assert (!target.equals(FBUtilities.getBroadcastAddressAndPort())); - this.replica = replica; - } - - public boolean shouldHint() - { - return mutation != null && StorageProxy.shouldHint(replica); - } - - public Replica getReplica() - { - return replica; - } - - public Mutation mutation() - { - return getMutation(mutation); - } - - private static Mutation getMutation(Object object) - { - assert object instanceof Commit || object instanceof Mutation : object; - return object instanceof Commit ? ((Commit) object).makeMutation() - : (Mutation) object; - } - - private static Object shouldHint(boolean allowHints, MessageOut sentMessage, ConsistencyLevel consistencyLevel) - { - return allowHints - && sentMessage.verb != MessagingService.Verb.COUNTER_MUTATION - && consistencyLevel != ConsistencyLevel.ANY - ? sentMessage.payload : null; - } - -} diff --git a/src/java/org/apache/cassandra/net/async/BaseMessageInHandler.java b/src/java/org/apache/cassandra/net/async/BaseMessageInHandler.java deleted file mode 100644 index 2f2a9739acc5..000000000000 --- a/src/java/org/apache/cassandra/net/async/BaseMessageInHandler.java +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.EOFException; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.function.BiConsumer; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.handler.codec.ByteToMessageDecoder; -import org.apache.cassandra.db.monitoring.ApproximateTime; -import org.apache.cassandra.exceptions.UnknownTableException; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; - -/** - * Parses out individual messages from the incoming buffers. Each message, both header and payload, is incrementally built up - * from the available input data, then passed to the {@link #messageConsumer}. - * - * Note: this class derives from {@link ByteToMessageDecoder} to take advantage of the {@link ByteToMessageDecoder.Cumulator} - * behavior across {@link #decode(ChannelHandlerContext, ByteBuf, List)} invocations. That way we don't have to maintain - * the not-fully consumed {@link ByteBuf}s. - */ -public abstract class BaseMessageInHandler extends ByteToMessageDecoder -{ - public static final Logger logger = LoggerFactory.getLogger(BaseMessageInHandler.class); - - enum State - { - READ_FIRST_CHUNK, - READ_IP_ADDRESS, - READ_VERB, - READ_PARAMETERS_SIZE, - READ_PARAMETERS_DATA, - READ_PAYLOAD_SIZE, - READ_PAYLOAD, - CLOSED - } - - /** - * The byte count for magic, msg id, timestamp values. - */ - @VisibleForTesting - static final int FIRST_SECTION_BYTE_COUNT = 12; - - static final int VERB_LENGTH = Integer.BYTES; - - /** - * The default target for consuming deserialized {@link MessageIn}. - */ - static final BiConsumer MESSAGING_SERVICE_CONSUMER = (messageIn, id) -> MessagingService.instance().receive(messageIn, id); - - /** - * Abstracts out depending directly on {@link MessagingService#receive(MessageIn, int)}; this makes tests more sane - * as they don't require nor trigger the entire message processing circus. - */ - final BiConsumer messageConsumer; - - final InetAddressAndPort peer; - final int messagingVersion; - - protected State state; - - public BaseMessageInHandler(InetAddressAndPort peer, int messagingVersion, BiConsumer messageConsumer) - { - this.peer = peer; - this.messagingVersion = messagingVersion; - this.messageConsumer = messageConsumer; - } - - // redeclared here to make the method public (for testing) - @VisibleForTesting - public void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception - { - if (state == State.CLOSED) - { - in.skipBytes(in.readableBytes()); - return; - } - - try - { - handleDecode(ctx, in, out); - } - catch (Exception e) - { - // prevent any future attempts at reading messages from any inbound buffers, as we're already in a bad state - state = State.CLOSED; - - // force the buffer to appear to be consumed, thereby exiting the ByteToMessageDecoder.callDecode() loop, - // and other paths in that class, more efficiently - in.skipBytes(in.readableBytes()); - - // throwing the exception up causes the ByteToMessageDecoder.callDecode() loop to exit. if we don't do that, - // we'll keep trying to process data out of the last received buffer (and it'll be really, really wrong) - throw e; - } - } - - public abstract void handleDecode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception; - - MessageHeader readFirstChunk(ByteBuf in) throws IOException - { - if (in.readableBytes() < FIRST_SECTION_BYTE_COUNT) - return null; - MessagingService.validateMagic(in.readInt()); - MessageHeader messageHeader = new MessageInHandler.MessageHeader(); - messageHeader.messageId = in.readInt(); - int messageTimestamp = in.readInt(); // make sure to read the sent timestamp, even if DatabaseDescriptor.hasCrossNodeTimeout() is not enabled - messageHeader.constructionTime = MessageIn.deriveConstructionTime(peer, messageTimestamp, ApproximateTime.currentTimeMillis()); - - return messageHeader; - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) - { - if (cause instanceof EOFException) - logger.trace("eof reading from socket; closing", cause); - else if (cause instanceof UnknownTableException) - logger.warn(" Got message from unknown table while reading from socket {}[{}]; closing", - ctx.channel().remoteAddress(), ctx.channel().id(), cause); - else if (cause instanceof IOException) - logger.trace("IOException reading from socket; closing", cause); - else - logger.warn("Unexpected exception caught in inbound channel pipeline from {}[{}]", - ctx.channel().remoteAddress(), ctx.channel().id(), cause); - - ctx.close(); - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) - { - logger.trace("received channel closed message for peer {} on local addr {}", ctx.channel().remoteAddress(), ctx.channel().localAddress()); - state = State.CLOSED; - ctx.fireChannelInactive(); - } - - // should ony be used for testing!!! - @VisibleForTesting - abstract MessageHeader getMessageHeader(); - - /** - * A simple struct to hold the message header data as it is being built up. - */ - static class MessageHeader - { - int messageId; - long constructionTime; - InetAddressAndPort from; - MessagingService.Verb verb; - int payloadSize; - - Map parameters = Collections.emptyMap(); - - /** - * Length of the parameter data. If the message's version is {@link MessagingService#VERSION_40} or higher, - * this value is the total number of header bytes; else, for legacy messaging, this is the number of - * key/value entries in the header. - */ - int parameterLength; - } - - // for testing purposes only!!! - @VisibleForTesting - public State getState() - { - return state; - } -} diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java deleted file mode 100644 index a77cb0713e84..000000000000 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputPlus.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.WritableByteChannel; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufOutputStream; -import org.apache.cassandra.io.util.CheckedFunction; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.io.util.Memory; -import org.apache.cassandra.io.util.UnbufferedDataOutputStreamPlus; -import org.apache.cassandra.utils.memory.MemoryUtil; -import org.apache.cassandra.utils.vint.VIntCoding; - -/** - * A {@link DataOutputPlus} that uses a {@link ByteBuf} as a backing buffer. This class is completely thread unsafe and - * it is expected that the backing buffer is sized correctly for all the writes you want to do (or the buffer needs - * to be growable). - */ -public class ByteBufDataOutputPlus extends ByteBufOutputStream implements DataOutputPlus -{ - private final ByteBuf buffer; - - /** - * ByteBuffer to use for defensive copies of direct {@link ByteBuffer}s - see {@link #write(ByteBuffer)}. - */ - private final ByteBuffer hollowBuffer = MemoryUtil.getHollowDirectByteBuffer(); - - public ByteBufDataOutputPlus(ByteBuf buffer) - { - super(buffer); - this.buffer = buffer; - } - - /** - * {@inheritDoc} - "write the buffer without modifying its position" - * - * Unfortunately, netty's {@link ByteBuf#writeBytes(ByteBuffer)} modifies the byteBuffer's position, - * and that is unsafe in our world wrt multithreading. Hence we need to be careful: reference the backing array - * on heap ByteBuffers, and use a reusable "hollow" ByteBuffer ({@link #hollowBuffer}) for direct ByteBuffers. - */ - @Override - public void write(ByteBuffer byteBuffer) throws IOException - { - if (byteBuffer.hasArray()) - { - write(byteBuffer.array(), byteBuffer.arrayOffset() + byteBuffer.position(), byteBuffer.remaining()); - } - else - { - assert byteBuffer.isDirect(); - MemoryUtil.duplicateDirectByteBuffer(byteBuffer, hollowBuffer); - buffer.writeBytes(hollowBuffer); - } - } - - @Override - public void write(Memory memory, long offset, long length) throws IOException - { - for (ByteBuffer buffer : memory.asByteBuffers(offset, length)) - write(buffer); - } - - @Override - public R applyToChannel(CheckedFunction c) throws IOException - { - throw new UnsupportedOperationException(); - } - - @Override - public void writeVInt(long v) throws IOException - { - writeUnsignedVInt(VIntCoding.encodeZigZag64(v)); - } - - @Override - public void writeUnsignedVInt(long v) throws IOException - { - int size = VIntCoding.computeUnsignedVIntSize(v); - if (size == 1) - { - buffer.writeByte((byte) (v & 0xFF)); - return; - } - - buffer.writeBytes(VIntCoding.encodeVInt(v, size), 0, size); - } - - @Override - public void write(int b) throws IOException - { - buffer.writeByte((byte) (b & 0xFF)); - } - - @Override - public void writeByte(int v) throws IOException - { - buffer.writeByte((byte) (v & 0xFF)); - } - - @Override - public void writeBytes(String s) throws IOException - { - for (int index = 0; index < s.length(); index++) - buffer.writeByte(s.charAt(index) & 0xFF); - } - - @Override - public void writeChars(String s) throws IOException - { - for (int index = 0; index < s.length(); index++) - buffer.writeChar(s.charAt(index)); - } - - @Override - public void writeUTF(String s) throws IOException - { - UnbufferedDataOutputStreamPlus.writeUTF(s, this); - } -} diff --git a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java b/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java deleted file mode 100644 index 777bc3e73522..000000000000 --- a/src/java/org/apache/cassandra/net/async/ByteBufDataOutputStreamPlus.java +++ /dev/null @@ -1,243 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - -import com.google.common.util.concurrent.Uninterruptibles; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.Unpooled; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.util.concurrent.Future; -import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; -import org.apache.cassandra.io.util.DataOutputStreamPlus; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter; -import org.apache.cassandra.streaming.StreamSession; - -/** - * A {@link DataOutputStreamPlus} that writes to a {@link ByteBuf}. The novelty here is that all writes - * actually get written in to a {@link ByteBuffer} that shares a backing buffer with a {@link ByteBuf}. - * The trick to do that is allocate the ByteBuf, get a ByteBuffer from it by calling {@link ByteBuf#nioBuffer()}, - * and passing that to the super class as {@link #buffer}. When the {@link #buffer} is full or {@link #doFlush(int)} - * is invoked, the {@link #currentBuf} is published to the netty channel. - */ -public class ByteBufDataOutputStreamPlus extends BufferedDataOutputStreamPlus -{ - private final StreamSession session; - private final Channel channel; - private final int bufferSize; - private final Logger logger = LoggerFactory.getLogger(ByteBufDataOutputStreamPlus.class); - - /** - * Tracks how many bytes we've written to the netty channel. This more or less follows the channel's - * high/low water marks and ultimately the 'writablility' status of the channel. Unfortunately there's - * no notification mechanism that can poke a producer to let it know when the channel becomes writable - * (after it was unwritable); hence, the use of a {@link Semaphore}. - */ - private final Semaphore channelRateLimiter; - - /** - * This *must* be the owning {@link ByteBuf} for the {@link BufferedDataOutputStreamPlus#buffer} - */ - private ByteBuf currentBuf; - - private ByteBufDataOutputStreamPlus(StreamSession session, Channel channel, ByteBuf buffer, int bufferSize) - { - super(buffer.nioBuffer(0, bufferSize)); - this.session = session; - this.channel = channel; - this.currentBuf = buffer; - this.bufferSize = bufferSize; - channelRateLimiter = new Semaphore(channel.config().getWriteBufferHighWaterMark(), true); - } - - @Override - protected WritableByteChannel newDefaultChannel() - { - return new WritableByteChannel() - { - @Override - public int write(ByteBuffer src) throws IOException - { - assert src == buffer; - int size = src.position(); - doFlush(size); - return size; - } - - @Override - public boolean isOpen() - { - return channel.isOpen(); - } - - @Override - public void close() - { } - }; - } - - public static ByteBufDataOutputStreamPlus create(StreamSession session, Channel channel, int bufferSize) - { - ByteBuf buf = channel.alloc().directBuffer(bufferSize, bufferSize); - return new ByteBufDataOutputStreamPlus(session, channel, buf, bufferSize); - } - - /** - * Writes the incoming buffer directly to the backing {@link #channel}, without copying to the intermediate {@link #buffer}. - */ - public ChannelFuture writeToChannel(ByteBuf buf) throws IOException - { - doFlush(buffer.position()); - - int byteCount = buf.readableBytes(); - - if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, byteCount, 5, TimeUnit.MINUTES)) - throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", byteCount)); - - // the (possibly naive) assumption that we should always flush after each incoming buf - ChannelFuture channelFuture = channel.writeAndFlush(buf); - channelFuture.addListener(future -> handleBuffer(future, byteCount)); - return channelFuture; - } - - /** - * Writes the incoming buffer directly to the backing {@link #channel}, without copying to the intermediate {@link #buffer}. - * The incoming buffer will be automatically released when the netty channel invokes the listeners of success/failure to - * send the buffer. - */ - public ChannelFuture writeToChannel(ByteBuffer buffer) throws IOException - { - ChannelFuture channelFuture = writeToChannel(Unpooled.wrappedBuffer(buffer)); - channelFuture.addListener(future -> FileUtils.clean(buffer)); - return channelFuture; - } - - /** - * Writes all data in file channel to stream BUFFER_SIZE at a time. - * Closes file channel when done - * - * @param f - * @return number of bytes transferred - * @throws IOException - */ - public long writeToChannel(FileChannel f, StreamRateLimiter limiter) throws IOException - { - final long length = f.size(); - long bytesTransferred = 0; - - try - { - while (bytesTransferred < length) - { - int toRead = (int) Math.min(bufferSize, length - bytesTransferred); - NonClosingDefaultFileRegion fileRegion = new NonClosingDefaultFileRegion(f, bytesTransferred, toRead); - - if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, toRead, 5, TimeUnit.MINUTES)) - throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", toRead)); - - limiter.acquire(toRead); - - bytesTransferred += toRead; - final boolean shouldClose = (bytesTransferred == length); // this is the last buffer, can safely close channel - - channel.writeAndFlush(fileRegion).addListener(future -> { - handleBuffer(future, toRead); - - if ((shouldClose || !future.isSuccess()) && f.isOpen()) - f.close(); - }); - logger.trace("{} of {} (toRead {} cs {})", bytesTransferred, length, toRead, f.isOpen()); - } - - return bytesTransferred; - } catch (Exception e) - { - if (f.isOpen()) - f.close(); - - throw e; - } - } - - @Override - protected void doFlush(int count) throws IOException - { - // flush the current backing write buffer only if there's any pending data - if (buffer.position() > 0 && channel.isOpen()) - { - int byteCount = buffer.position(); - currentBuf.writerIndex(byteCount); - - if (!Uninterruptibles.tryAcquireUninterruptibly(channelRateLimiter, byteCount, 2, TimeUnit.MINUTES)) - throw new IOException(String.format("outbound channel was not writable. Failed to acquire sufficient permits %d", byteCount)); - - channel.writeAndFlush(currentBuf).addListener(future -> handleBuffer(future, byteCount)); - currentBuf = channel.alloc().directBuffer(bufferSize, bufferSize); - buffer = currentBuf.nioBuffer(0, bufferSize); - } - } - - /** - * Handles the result of publishing a buffer to the channel. - * - * Note: this will be executed on the event loop. - */ - private void handleBuffer(Future future, int bytesWritten) - { - channelRateLimiter.release(bytesWritten); - logger.trace("bytesWritten {} {} because {}", bytesWritten, (future.isSuccess() == true) ? "Succeeded" : "Failed", future.cause()); - if (!future.isSuccess() && channel.isOpen()) - session.onError(future.cause()); - } - - public ByteBufAllocator getAllocator() - { - return channel.alloc(); - } - - /** - * {@inheritDoc} - * - * Flush any last buffered (if the channel is open), and release any buffers. *Not* responsible for closing - * the netty channel as we might use it again for transferring more files. - * - * Note: should be called on the producer thread, not the netty event loop. - */ - @Override - public void close() throws IOException - { - doFlush(0); - if (currentBuf.refCnt() > 0) - currentBuf.release(); - currentBuf = null; - buffer = null; - } -} diff --git a/src/java/org/apache/cassandra/net/async/ChannelWriter.java b/src/java/org/apache/cassandra/net/async/ChannelWriter.java deleted file mode 100644 index e9847366c2e5..000000000000 --- a/src/java/org/apache/cassandra/net/async/ChannelWriter.java +++ /dev/null @@ -1,418 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.util.Optional; -import java.util.Queue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; - -import com.google.common.annotations.VisibleForTesting; - -import io.netty.buffer.Unpooled; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelOutboundBuffer; -import io.netty.channel.ChannelPromise; -import io.netty.channel.MessageSizeEstimator; -import io.netty.handler.timeout.IdleStateEvent; -import io.netty.util.Attribute; -import io.netty.util.AttributeKey; -import io.netty.util.concurrent.Future; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.utils.CoalescingStrategies; -import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy; - -/** - * Represents a ready and post-handshake channel that can send outbound messages. This class groups a netty channel - * with any other channel-related information we track and, most importantly, handles the details on when the channel is flushed. - * - *

Flushing

- * - * We don't flush to the socket on every message as it's a bit of a performance drag (making the system call, copying - * the buffer, sending out a small packet). Thus, by waiting until we have a decent chunk of data (for some definition - * of 'decent'), we can achieve better efficiency and improved performance (yay!). - *

- * When to flush mainly depends on whether we use message coalescing or not (see {@link CoalescingStrategies}). - *

- * Note that the callback functions are invoked on the netty event loop, which is (in almost all cases) different - * from the thread that will be invoking {@link #write(QueuedMessage, boolean)}. - * - *

Flushing without coalescing

- * - * When no coalescing is in effect, we want to send new message "right away". However, as said above, flushing after - * every message would be particularly inefficient when there is lots of message in our sending queue, and so in - * practice we want to flush in 2 cases: - * 1) After any message if there is no pending message in the send queue. - * 2) When we've filled up or exceeded the netty outbound buffer (see {@link ChannelOutboundBuffer}) - *

- * The second part is relatively simple and handled generically in {@link MessageOutHandler#write(ChannelHandlerContext, Object, ChannelPromise)} [1]. - * The first part however is made a little more complicated by how netty's event loop executes. It is woken up by - * external callers to the channel invoking a flush, via either {@link Channel#flush} or one of the {@link Channel#writeAndFlush} - * methods [2]. So a plain {@link Channel#write} will only queue the message in the channel, and not wake up the event loop. - *

- * This means we don't want to simply call {@link Channel#write} as we want the message processed immediately. But we - * also don't want to flush on every message if there is more in the sending queue, so simply calling - * {@link Channel#writeAndFlush} isn't completely appropriate either. In practice, we handle this by calling - * {@link Channel#writeAndFlush} (so the netty event loop does wake up), but we override the flush behavior so - * it actually only flushes if there are no pending messages (see how {@link MessageOutHandler#flush} delegates the flushing - * decision back to this class through {@link #onTriggeredFlush}, and how {@link SimpleChannelWriter} makes this a no-op; - * instead {@link SimpleChannelWriter} flushes after any message if there are no more pending ones in - * {@link #onMessageProcessed}). - * - *

Flushing with coalescing

- * - * The goal of coalescing is to (artificially) delay the flushing of data in order to aggregate even more data before - * sending a group of packets out. So we don't want to flush after messages even if there is no pending messages in the - * sending queue, but we rather want to delegate the decision on when to flush to the {@link CoalescingStrategy}. In - * pratice, when coalescing is enabled we will flush in 2 cases: - * 1) When the coalescing strategies decides that we should. - * 2) When we've filled up or exceeded the netty outbound buffer ({@link ChannelOutboundBuffer}), exactly like in the - * no coalescing case. - *

- * The second part is handled exactly like in the no coalescing case, see above. - * The first part is handled by {@link CoalescingChannelWriter#write(QueuedMessage, boolean)}. Whenever a message is sent, we check - * if a flush has been already scheduled by the coalescing strategy. If one has, we're done, otherwise we ask the - * strategy when the next flush should happen and schedule one. - * - *

Message timeouts and retries

- * - * The main outward-facing method is {@link #write(QueuedMessage, boolean)}, where callers pass a - * {@link QueuedMessage}. If a message times out, as defined in {@link QueuedMessage#isTimedOut()}, - * the message listener {@link #handleMessageFuture(Future, QueuedMessage, boolean)} is invoked - * with the cause being a {@link ExpiredException}. The message is not retried and it is dropped on the floor. - *

- * If there is some {@link IOException} on the socket after the message has been written to the netty channel, - * the message listener {@link #handleMessageFuture(Future, QueuedMessage, boolean)} is invoked - * and 1) we check to see if the connection should be re-established, and 2) possibly createRetry the message. - * - *

Failures

- * - *

Failure to make progress sending bytes

- * If we are unable to make progress sending messages, we'll receive a netty notification - * ({@link IdleStateEvent}) at {@link MessageOutHandler#userEventTriggered(ChannelHandlerContext, Object)}. - * We then want to close the socket/channel, and purge any messages in {@link OutboundMessagingConnection#backlog} - * to try to free up memory as quickly as possible. Any messages in the netty pipeline will be marked as fail - * (as we close the channel), but {@link MessageOutHandler#userEventTriggered(ChannelHandlerContext, Object)} also - * sets a channel attribute, {@link #PURGE_MESSAGES_CHANNEL_ATTR} to true. This is essentially as special flag - * that we can look at in the promise handler code ({@link #handleMessageFuture(Future, QueuedMessage, boolean)}) - * to indicate that any backlog should be thrown away. - * - *

Notes

- * [1] For those desperately interested, and only after you've read the entire class-level doc: You can register a custom - * {@link MessageSizeEstimator} with a netty channel. When a message is written to the channel, it will check the - * message size, and if the max ({@link ChannelOutboundBuffer}) size will be exceeded, a task to signal the "channel - * writability changed" will be executed in the channel. That task, however, will wake up the event loop. - * Thus if coalescing is enabled, the event loop will wake up prematurely and process (and possibly flush!) the messages - * currently in the queue, thus defeating an aspect of coalescing. Hence, we're not using that feature of netty. - * [2]: The netty event loop is also woken up by it's internal timeout on the epoll_wait() system call. - */ -abstract class ChannelWriter -{ - /** - * A netty channel {@link Attribute} to indicate, when a channel is closed, any backlogged messages should be purged, - * as well. See the class-level documentation for more information. - */ - static final AttributeKey PURGE_MESSAGES_CHANNEL_ATTR = AttributeKey.newInstance("purgeMessages"); - - protected final Channel channel; - private volatile boolean closed; - - /** Number of currently pending messages on this channel. */ - final AtomicLong pendingMessageCount = new AtomicLong(0); - - /** - * A consuming function that handles the result of each message sent. - */ - private final Consumer messageResultConsumer; - - /** - * A reusable instance to avoid creating garbage on preciessing the result of every message sent. - * As we have the guarantee that the netty evet loop is single threaded, there should be no contention over this - * instance, as long as it (not it's state) is shared across threads. - */ - private final MessageResult messageResult = new MessageResult(); - - protected ChannelWriter(Channel channel, Consumer messageResultConsumer) - { - this.channel = channel; - this.messageResultConsumer = messageResultConsumer; - channel.attr(PURGE_MESSAGES_CHANNEL_ATTR).set(false); - } - - /** - * Creates a new {@link ChannelWriter} using the (assumed properly connected) provided channel, and using coalescing - * based on the provided strategy. - */ - static ChannelWriter create(Channel channel, Consumer messageResultConsumer, Optional coalescingStrategy) - { - return coalescingStrategy.isPresent() - ? new CoalescingChannelWriter(channel, messageResultConsumer, coalescingStrategy.get()) - : new SimpleChannelWriter(channel, messageResultConsumer); - } - - /** - * Writes a message to this {@link ChannelWriter} if the channel is writable. - *

- * We always want to write to the channel *unless* it's not writable yet still open. - * If the channel is closed, the promise will be notifed as a fail (due to channel closed), - * and let the handler ({@link #handleMessageFuture(Future, QueuedMessage, boolean)}) - * do the reconnect magic/dance. Thus we simplify when to reconnect by not burdening the (concurrent) callers - * of this method, and instead keep it all in the future handler/event loop (which is single threaded). - * - * @param message the message to write/send. - * @param checkWritability a flag to indicate if the status of the channel should be checked before passing - * the message on to the {@link #channel}. - * @return true if the message was written to the channel; else, false. - */ - boolean write(QueuedMessage message, boolean checkWritability) - { - if ( (checkWritability && (channel.isWritable()) || !channel.isOpen()) || !checkWritability) - { - write0(message).addListener(f -> handleMessageFuture(f, message, true)); - return true; - } - return false; - } - - /** - * Handles the future of sending a particular message on this {@link ChannelWriter}. - *

- * Note: this is called from the netty event loop, so there is no race across multiple execution of this method. - */ - @VisibleForTesting - void handleMessageFuture(Future future, QueuedMessage msg, boolean allowReconnect) - { - messageResult.setAll(this, msg, future, allowReconnect); - messageResultConsumer.accept(messageResult); - messageResult.clearAll(); - } - - boolean shouldPurgeBacklog() - { - if (!channel.attr(PURGE_MESSAGES_CHANNEL_ATTR).get()) - return false; - - channel.attr(PURGE_MESSAGES_CHANNEL_ATTR).set(false); - return true; - } - - /** - * Writes a backlog of message to this {@link ChannelWriter}. This is mostly equivalent to calling - * {@link #write(QueuedMessage, boolean)} for every message of the provided backlog queue, but - * it ignores any coalescing, triggering a flush only once after all messages have been sent. - * - * @param backlog the backlog of message to send. - * @return the count of items written to the channel from the queue. - */ - int writeBacklog(Queue backlog, boolean allowReconnect) - { - int count = 0; - while (true) - { - if (!channel.isWritable()) - break; - - QueuedMessage msg = backlog.poll(); - if (msg == null) - break; - - pendingMessageCount.incrementAndGet(); - ChannelFuture future = channel.write(msg); - future.addListener(f -> handleMessageFuture(f, msg, allowReconnect)); - count++; - } - - // as this is an infrequent operation, don't bother coordinating with the instance-level flush task - if (count > 0) - channel.flush(); - - return count; - } - - void close() - { - if (closed) - return; - - closed = true; - channel.close(); - } - - long pendingMessageCount() - { - return pendingMessageCount.get(); - } - - /** - * Close the underlying channel but only after having make sure every pending message has been properly sent. - */ - void softClose() - { - if (closed) - return; - - closed = true; - channel.writeAndFlush(Unpooled.EMPTY_BUFFER).addListener(ChannelFutureListener.CLOSE); - } - - @VisibleForTesting - boolean isClosed() - { - return closed; - } - - /** - * Write the message to the {@link #channel}. - *

- * Note: this method, in almost all cases, is invoked from an app-level writing thread, not the netty event loop. - */ - protected abstract ChannelFuture write0(QueuedMessage message); - - /** - * Invoked after a message has been processed in the pipeline. Should only be used for essential bookkeeping operations. - *

- * Note: this method is invoked on the netty event loop. - */ - abstract void onMessageProcessed(ChannelHandlerContext ctx); - - /** - * Invoked when pipeline receives a flush request. - *

- * Note: this method is invoked on the netty event loop. - */ - abstract void onTriggeredFlush(ChannelHandlerContext ctx); - - /** - * Handles the non-coalescing flush case. - */ - @VisibleForTesting - static class SimpleChannelWriter extends ChannelWriter - { - private SimpleChannelWriter(Channel channel, Consumer messageResultConsumer) - { - super(channel, messageResultConsumer); - } - - protected ChannelFuture write0(QueuedMessage message) - { - pendingMessageCount.incrementAndGet(); - // We don't truly want to flush on every message but we do want to wake-up the netty event loop for the - // channel so the message is processed right away, which is why we use writeAndFlush. This won't actually - // flush, though, because onTriggeredFlush, which MessageOutHandler delegates to, does nothing. We will - // flush after the message is processed though if there is no pending one due to onMessageProcessed. - // See the class javadoc for context and much more details. - return channel.writeAndFlush(message); - } - - void onMessageProcessed(ChannelHandlerContext ctx) - { - if (pendingMessageCount.decrementAndGet() == 0) - ctx.flush(); - } - - void onTriggeredFlush(ChannelHandlerContext ctx) - { - // Don't actually flush on "normal" flush calls to the channel. - } - } - - /** - * Handles the coalescing flush case. - */ - @VisibleForTesting - static class CoalescingChannelWriter extends ChannelWriter - { - private static final int MIN_MESSAGES_FOR_COALESCE = DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages(); - - private final CoalescingStrategy strategy; - private final int minMessagesForCoalesce; - - @VisibleForTesting - final AtomicBoolean scheduledFlush = new AtomicBoolean(false); - - CoalescingChannelWriter(Channel channel, Consumer messageResultConsumer, CoalescingStrategy strategy) - { - this (channel, messageResultConsumer, strategy, MIN_MESSAGES_FOR_COALESCE); - } - - @VisibleForTesting - CoalescingChannelWriter(Channel channel, Consumer messageResultConsumer, CoalescingStrategy strategy, int minMessagesForCoalesce) - { - super(channel, messageResultConsumer); - this.strategy = strategy; - this.minMessagesForCoalesce = minMessagesForCoalesce; - } - - protected ChannelFuture write0(QueuedMessage message) - { - long pendingCount = pendingMessageCount.incrementAndGet(); - ChannelFuture future = channel.write(message); - strategy.newArrival(message); - - // if we lost the race to set the state, simply write to the channel (no flush) - if (!scheduledFlush.compareAndSet(false, true)) - return future; - - long flushDelayNanos; - // if we've hit the minimum number of messages for coalescing or we've run out of coalesce time, flush. - // note: we check the exact count, instead of greater than or equal to, of message here to prevent a flush task - // for each message (if there's messages coming in on multiple threads). There will be, of course, races - // with the consumer decrementing the pending counter, but that's still less excessive flushes. - if (pendingCount == minMessagesForCoalesce || (flushDelayNanos = strategy.currentCoalescingTimeNanos()) <= 0) - { - scheduledFlush.set(false); - channel.flush(); - } - else - { - // calling schedule() on the eventLoop will force it to wake up (if not already executing) and schedule the task - channel.eventLoop().schedule(() -> { - // NOTE: this executes on the event loop - scheduledFlush.set(false); - // we execute() the flush() as an additional task rather than immediately in-line as there is a - // race condition when this task runs (executing on the event loop) and a thread that writes the channel (top of this method). - // If this task is picked up but before the scheduledFlush falg is flipped, the other thread writes - // and then checks the scheduledFlush (which is still true) and exits. - // This task changes the flag and if it calls flush() in-line, and netty flushs everything immediately (that is, what's been serialized) - // to the transport as we're on the event loop. The other thread's write became a task that executes *after* this task in the netty queue, - // and if there's not a subsequent followup flush scheduled, that write can be orphaned until another write comes in. - channel.eventLoop().execute(channel::flush); - }, flushDelayNanos, TimeUnit.NANOSECONDS); - } - return future; - } - - void onMessageProcessed(ChannelHandlerContext ctx) - { - pendingMessageCount.decrementAndGet(); - } - - void onTriggeredFlush(ChannelHandlerContext ctx) - { - // When coalescing, obey the flush calls normally - ctx.flush(); - } - } -} diff --git a/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java b/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java deleted file mode 100644 index ebf26bddde4d..000000000000 --- a/src/java/org/apache/cassandra/net/async/HandshakeProtocol.java +++ /dev/null @@ -1,314 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.net.async; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.net.InetAddress; -import java.util.Objects; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.primitives.Ints; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.ByteBufInputStream; -import io.netty.buffer.ByteBufOutputStream; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; -import org.apache.cassandra.net.MessagingService; - -/** - * Messages for the handshake phase of the internode protocol. - *

- * The handshake's main purpose is to establish a protocol version that both side can talk, as well as exchanging a few connection - * options/parameters. The handshake is composed of 3 messages, the first being sent by the initiator of the connection. The other - * side then answer with the 2nd message. At that point, if a version mismatch is detected by the connection initiator, - * it will simply disconnect and reconnect with a more appropriate version. But if the version is acceptable, the connection - * initiator sends the third message of the protocol, after which it considers the connection ready. - *

- * See below for a more precise description of each of those 3 messages. - *

- * Note that this handshake protocol doesn't fully apply to streaming. For streaming, only the first message is sent, - * after which the streaming protocol takes over (not documented here) - */ -public class HandshakeProtocol -{ - /** - * The initial message sent when a node creates a new connection to a remote peer. This message contains: - * 1) the {@link MessagingService#PROTOCOL_MAGIC} number (4 bytes). - * 2) the connection flags (4 bytes), which encodes: - * - the version the initiator thinks should be used for the connection (in practice, either the initiator - * version if it's the first time we connect to that remote since startup, or the last version known for that - * peer otherwise). - * - the "mode" of the connection: whether it is for streaming or for messaging. - * - whether compression should be used or not (if it is, compression is enabled _after_ the last message of the - * handshake has been sent). - *

- * More precisely, connection flags: - *

-     * {@code
-     *                      1 1 1 1 1 1 1 1 1 1 2 2 2 2 2 2 2 2 2 2 3 3
-     *  0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1
-     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
-     * |U U C M       |                |                               |
-     * |N N M O       |     VERSION    |             unused            |
-     * |U U P D       |                |                               |
-     * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
-     * }
-     * 
- * UNU - unused bits lowest two bits; from a historical note: used to be "serializer type," which was always Binary - * CMP - compression enabled bit - * MOD - connection mode. If the bit is on, the connection is for streaming; if the bit is off, it is for inter-node messaging. - * VERSION - if a streaming connection, indicates the streaming protocol version {@link org.apache.cassandra.streaming.messages.StreamMessage#CURRENT_VERSION}; - * if a messaging connection, indicates the messaging protocol version the initiator *thinks* should be used. - */ - public static class FirstHandshakeMessage - { - /** Contains the PROTOCOL_MAGIC (int) and the flags (int). */ - private static final int LENGTH = 8; - - final int messagingVersion; - final NettyFactory.Mode mode; - final boolean compressionEnabled; - - public FirstHandshakeMessage(int messagingVersion, NettyFactory.Mode mode, boolean compressionEnabled) - { - assert messagingVersion > 0; - this.messagingVersion = messagingVersion; - this.mode = mode; - this.compressionEnabled = compressionEnabled; - } - - @VisibleForTesting - int encodeFlags() - { - int flags = 0; - if (compressionEnabled) - flags |= 1 << 2; - if (mode == NettyFactory.Mode.STREAMING) - flags |= 1 << 3; - - flags |= (messagingVersion << 8); - return flags; - } - - public ByteBuf encode(ByteBufAllocator allocator) - { - ByteBuf buffer = allocator.directBuffer(LENGTH, LENGTH); - buffer.writerIndex(0); - buffer.writeInt(MessagingService.PROTOCOL_MAGIC); - buffer.writeInt(encodeFlags()); - return buffer; - } - - static FirstHandshakeMessage maybeDecode(ByteBuf in) throws IOException - { - if (in.readableBytes() < LENGTH) - return null; - - MessagingService.validateMagic(in.readInt()); - int flags = in.readInt(); - int version = MessagingService.getBits(flags, 15, 8); - NettyFactory.Mode mode = MessagingService.getBits(flags, 3, 1) == 1 - ? NettyFactory.Mode.STREAMING - : NettyFactory.Mode.MESSAGING; - boolean compressed = MessagingService.getBits(flags, 2, 1) == 1; - return new FirstHandshakeMessage(version, mode, compressed); - } - - @Override - public boolean equals(Object other) - { - if (!(other instanceof FirstHandshakeMessage)) - return false; - - FirstHandshakeMessage that = (FirstHandshakeMessage)other; - return this.messagingVersion == that.messagingVersion - && this.mode == that.mode - && this.compressionEnabled == that.compressionEnabled; - } - - @Override - public int hashCode() - { - return Objects.hash(messagingVersion, mode, compressionEnabled); - } - - @Override - public String toString() - { - return String.format("FirstHandshakeMessage - messaging version: %d, mode: %s, compress: %b", messagingVersion, mode, compressionEnabled); - } - } - - /** - * The second message of the handshake, sent by the node receiving the {@link FirstHandshakeMessage} back to the - * connection initiator. This message contains the messaging version of the peer sending this message, - * so {@link org.apache.cassandra.net.MessagingService#current_version}. - */ - static class SecondHandshakeMessage - { - /** The messaging version sent by the receiving peer (int). */ - private static final int LENGTH = 4; - - final int messagingVersion; - - SecondHandshakeMessage(int messagingVersion) - { - this.messagingVersion = messagingVersion; - } - - public ByteBuf encode(ByteBufAllocator allocator) - { - ByteBuf buffer = allocator.directBuffer(LENGTH, LENGTH); - buffer.writerIndex(0); - buffer.writeInt(messagingVersion); - return buffer; - } - - static SecondHandshakeMessage maybeDecode(ByteBuf in) - { - return in.readableBytes() >= LENGTH ? new SecondHandshakeMessage(in.readInt()) : null; - } - - @Override - public boolean equals(Object other) - { - return other instanceof SecondHandshakeMessage - && this.messagingVersion == ((SecondHandshakeMessage) other).messagingVersion; - } - - @Override - public int hashCode() - { - return Integer.hashCode(messagingVersion); - } - - @Override - public String toString() - { - return String.format("SecondHandshakeMessage - messaging version: %d", messagingVersion); - } - } - - /** - * The third message of the handshake, sent by the connection initiator on reception of {@link SecondHandshakeMessage}. - * This message contains: - * 1) the connection initiator's messaging version (4 bytes) - {@link org.apache.cassandra.net.MessagingService#current_version}. - * This indicates the max messaging version supported by this node. - * 2) the connection initiator's broadcast address as encoded by {@link org.apache.cassandra.net.CompactEndpointSerializationHelper}. - * This can be either 5 bytes for an IPv4 address, or 17 bytes for an IPv6 one. - *

- * This message concludes the handshake protocol. After that, the connection will used either for streaming, or to - * send messages. If the connection is to be compressed, compression is enabled only after this message is sent/received. - */ - static class ThirdHandshakeMessage - { - /** - * The third message contains the version and IP address of the sending node. Because the IP can be either IPv4 or - * IPv6, this can be either 9 (4 for version + 5 for IP) or 21 (4 for version + 17 for IP) bytes. Since we can't know - * a priori if the IP address will be v4 or v6, go with the minimum required bytes and hope that if the address is - * v6, we'll have the extra 12 bytes in the packet. - */ - private static final int MIN_LENGTH = 9; - - /** - * The internode messaging version of the peer; used for serializing to a version the peer understands. - */ - final int messagingVersion; - final InetAddressAndPort address; - - ThirdHandshakeMessage(int messagingVersion, InetAddressAndPort address) - { - this.messagingVersion = messagingVersion; - this.address = address; - } - - @SuppressWarnings("resource") - public ByteBuf encode(ByteBufAllocator allocator) - { - int bufLength = Ints.checkedCast(Integer.BYTES + CompactEndpointSerializationHelper.instance.serializedSize(address, messagingVersion)); - ByteBuf buffer = allocator.directBuffer(bufLength, bufLength); - buffer.writerIndex(0); - - // the max messaging version supported by the local node (not #messagingVersion) - buffer.writeInt(MessagingService.current_version); - try - { - DataOutputPlus dop = new ByteBufDataOutputPlus(buffer); - CompactEndpointSerializationHelper.instance.serialize(address, dop, messagingVersion); - return buffer; - } - catch (IOException e) - { - // Shouldn't happen, we're serializing in memory. - throw new AssertionError(e); - } - } - - @SuppressWarnings("resource") - static ThirdHandshakeMessage maybeDecode(ByteBuf in) - { - if (in.readableBytes() < MIN_LENGTH) - return null; - - in.markReaderIndex(); - int version = in.readInt(); - DataInputPlus input = new ByteBufDataInputPlus(in); - try - { - InetAddressAndPort address = CompactEndpointSerializationHelper.instance.deserialize(input, version); - return new ThirdHandshakeMessage(version, address); - } - catch (IOException e) - { - // makes the assumption we didn't have enough bytes to deserialize an IPv6 address, - // as we only check the MIN_LENGTH of the buf. - in.resetReaderIndex(); - return null; - } - } - - @Override - public boolean equals(Object other) - { - if (!(other instanceof ThirdHandshakeMessage)) - return false; - - ThirdHandshakeMessage that = (ThirdHandshakeMessage)other; - return this.messagingVersion == that.messagingVersion - && Objects.equals(this.address, that.address); - } - - @Override - public int hashCode() - { - return Objects.hash(messagingVersion, address); - } - - @Override - public String toString() - { - return String.format("ThirdHandshakeMessage - messaging version: %d, address = %s", messagingVersion, address); - } - } -} diff --git a/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java b/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java deleted file mode 100644 index e66a589c8a5c..000000000000 --- a/src/java/org/apache/cassandra/net/async/InboundHandshakeHandler.java +++ /dev/null @@ -1,321 +0,0 @@ -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.util.List; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import javax.net.ssl.SSLSession; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.AdaptiveRecvByteBufAllocator; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.ByteToMessageDecoder; -import io.netty.handler.ssl.SslHandler; -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage; -import org.apache.cassandra.streaming.async.StreamingInboundHandler; -import org.apache.cassandra.streaming.messages.StreamMessage; - -/** - * 'Server'-side component that negotiates the internode handshake when establishing a new connection. - * This handler will be the first in the netty channel for each incoming connection (secure socket (TLS) notwithstanding), - * and once the handshake is successful, it will configure the proper handlers (mostly {@link MessageInHandler}) - * and remove itself from the working pipeline. - */ -class InboundHandshakeHandler extends ByteToMessageDecoder -{ - private static final Logger logger = LoggerFactory.getLogger(InboundHandshakeHandler.class); - - enum State { START, AWAITING_HANDSHAKE_BEGIN, AWAIT_MESSAGING_START_RESPONSE, HANDSHAKE_COMPLETE, HANDSHAKE_FAIL } - - private State state; - - private final IInternodeAuthenticator authenticator; - - private boolean hasAuthenticated; - /** - * The peer's declared messaging version. - */ - private int version; - - /** - * Does the peer support (or want to use) compressed data? - */ - private boolean compressed; - - /** - * A future the essentially places a timeout on how long we'll wait for the peer - * to complete the next step of the handshake. - */ - private Future handshakeTimeout; - - InboundHandshakeHandler(IInternodeAuthenticator authenticator) - { - this.authenticator = authenticator; - state = State.START; - } - - @Override - protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) - { - try - { - if (!hasAuthenticated) - { - logSecureSocketDetails(ctx); - if (!handleAuthenticate(ctx.channel().remoteAddress(), ctx)) - return; - } - - switch (state) - { - case START: - state = handleStart(ctx, in); - break; - case AWAIT_MESSAGING_START_RESPONSE: - state = handleMessagingStartResponse(ctx, in); - break; - case HANDSHAKE_FAIL: - throw new IllegalStateException("channel should be closed after determining the handshake failed with peer: " + ctx.channel().remoteAddress()); - default: - logger.error("unhandled state: {}", state); - state = State.HANDSHAKE_FAIL; - ctx.close(); - } - } - catch (Exception e) - { - logger.error("unexpected error while negotiating internode messaging handshake", e); - state = State.HANDSHAKE_FAIL; - ctx.close(); - } - } - - /** - * Ensure the peer is allowed to connect to this node. - */ - @VisibleForTesting - boolean handleAuthenticate(SocketAddress socketAddress, ChannelHandlerContext ctx) - { - // the only reason addr would not be instanceof InetSocketAddress is in unit testing, when netty's EmbeddedChannel - // uses EmbeddedSocketAddress. Normally, we'd do an instanceof for that class name, but it's marked with default visibility, - // so we can't reference it outside of it's package (and so it doesn't compile). - if (socketAddress instanceof InetSocketAddress) - { - InetSocketAddress addr = (InetSocketAddress)socketAddress; - if (!authenticator.authenticate(addr.getAddress(), addr.getPort())) - { - if (logger.isTraceEnabled()) - logger.trace("Failed to authenticate peer {}", addr); - ctx.close(); - return false; - } - } - else if (!socketAddress.getClass().getSimpleName().equals("EmbeddedSocketAddress")) - { - ctx.close(); - return false; - } - hasAuthenticated = true; - return true; - } - - /** - * If the connection is using SSL/TLS, log some details about it. - */ - private void logSecureSocketDetails(ChannelHandlerContext ctx) - { - SslHandler sslHandler = ctx.pipeline().get(SslHandler.class); - if (sslHandler != null) - { - SSLSession session = sslHandler.engine().getSession(); - logger.info("connection from peer {}, protocol = {}, cipher suite = {}", - ctx.channel().remoteAddress(), session.getProtocol(), session.getCipherSuite()); - } - } - - /** - * Handles receiving the first message in the internode messaging handshake protocol. If the sender's protocol version - * is accepted, we respond with the second message of the handshake protocol. - */ - @VisibleForTesting - State handleStart(ChannelHandlerContext ctx, ByteBuf in) throws IOException - { - FirstHandshakeMessage msg = FirstHandshakeMessage.maybeDecode(in); - if (msg == null) - return State.START; - - logger.trace("received first handshake message from peer {}, message = {}", ctx.channel().remoteAddress(), msg); - version = msg.messagingVersion; - - if (msg.mode == NettyFactory.Mode.STREAMING) - { - // streaming connections are per-session and have a fixed version. we can't do anything with a wrong-version stream connection, so drop it. - if (version != StreamMessage.CURRENT_VERSION) - { - logger.warn("Received stream using protocol version {} (my version {}). Terminating connection", version, MessagingService.current_version); - ctx.close(); - return State.HANDSHAKE_FAIL; - } - - setupStreamingPipeline(ctx, version); - return State.HANDSHAKE_COMPLETE; - } - else - { - if (version < MessagingService.VERSION_30) - { - logger.error("Unable to read obsolete message version {} from {}; The earliest version supported is 3.0.0", version, ctx.channel().remoteAddress()); - ctx.close(); - return State.HANDSHAKE_FAIL; - } - - logger.trace("Connection version {} from {}", version, ctx.channel().remoteAddress()); - compressed = msg.compressionEnabled; - - // if this version is < the MS version the other node is trying - // to connect with, the other node will disconnect - ctx.writeAndFlush(new SecondHandshakeMessage(MessagingService.current_version).encode(ctx.alloc())) - .addListener(ChannelFutureListener.CLOSE_ON_FAILURE); - - // outbound side will reconnect to change the version - if (version > MessagingService.current_version) - { - logger.info("peer wants to use a messaging version higher ({}) than what this node supports ({})", version, MessagingService.current_version); - ctx.close(); - return State.HANDSHAKE_FAIL; - } - - long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getRpcTimeout()); - handshakeTimeout = ctx.executor().schedule(() -> failHandshake(ctx), timeout, TimeUnit.MILLISECONDS); - return State.AWAIT_MESSAGING_START_RESPONSE; - } - } - - private void setupStreamingPipeline(ChannelHandlerContext ctx, int protocolVersion) - { - ChannelPipeline pipeline = ctx.pipeline(); - InetSocketAddress address = (InetSocketAddress) ctx.channel().remoteAddress(); - pipeline.addLast(NettyFactory.instance.streamingGroup, "streamInbound", new StreamingInboundHandler(InetAddressAndPort.getByAddressOverrideDefaults(address.getAddress(), address.getPort()), protocolVersion, null)); - pipeline.remove(this); - - // pass a custom recv ByteBuf allocator to the channel. the default recv ByteBuf size is 1k, but in streaming we're - // dealing with large bulk blocks of data, let's default to larger sizes - ctx.channel().config().setRecvByteBufAllocator(new AdaptiveRecvByteBufAllocator(1 << 8, 1 << 13, 1 << 16)); - } - - /** - * Handles the third (and last) message in the internode messaging handshake protocol. Grabs the protocol version and - * IP addr the peer wants to use. - */ - @VisibleForTesting - State handleMessagingStartResponse(ChannelHandlerContext ctx, ByteBuf in) throws IOException - { - ThirdHandshakeMessage msg = ThirdHandshakeMessage.maybeDecode(in); - if (msg == null) - return State.AWAIT_MESSAGING_START_RESPONSE; - - logger.trace("received third handshake message from peer {}, message = {}", ctx.channel().remoteAddress(), msg); - if (handshakeTimeout != null) - { - handshakeTimeout.cancel(false); - handshakeTimeout = null; - } - - int maxVersion = msg.messagingVersion; - if (maxVersion > MessagingService.current_version) - { - logger.error("peer wants to use a messaging version higher ({}) than what this node supports ({})", maxVersion, MessagingService.current_version); - ctx.close(); - return State.HANDSHAKE_FAIL; - } - - // record the (true) version of the endpoint - InetAddressAndPort from = msg.address; - MessagingService.instance().setVersion(from, maxVersion); - if (logger.isTraceEnabled()) - logger.trace("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from)); - - setupMessagingPipeline(ctx.pipeline(), from, compressed, version); - return State.HANDSHAKE_COMPLETE; - } - - @VisibleForTesting - void setupMessagingPipeline(ChannelPipeline pipeline, InetAddressAndPort peer, boolean compressed, int messagingVersion) - { - if (compressed) - pipeline.addLast(NettyFactory.INBOUND_COMPRESSOR_HANDLER_NAME, NettyFactory.createLz4Decoder(messagingVersion)); - - BaseMessageInHandler messageInHandler = messagingVersion >= MessagingService.VERSION_40 - ? new MessageInHandler(peer, messagingVersion) - : new MessageInHandlerPre40(peer, messagingVersion); - - pipeline.addLast("messageInHandler", messageInHandler); - pipeline.remove(this); - } - - @VisibleForTesting - void failHandshake(ChannelHandlerContext ctx) - { - // we're not really racing on the handshakeTimeout as we're in the event loop, - // but, hey, defensive programming is beautiful thing! - if (state == State.HANDSHAKE_COMPLETE || (handshakeTimeout != null && handshakeTimeout.isCancelled())) - return; - - state = State.HANDSHAKE_FAIL; - ctx.close(); - - if (handshakeTimeout != null) - { - handshakeTimeout.cancel(false); - handshakeTimeout = null; - } - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) - { - logger.trace("Failed to properly handshake with peer {}. Closing the channel.", ctx.channel().remoteAddress()); - failHandshake(ctx); - ctx.fireChannelInactive(); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) - { - logger.error("Failed to properly handshake with peer {}. Closing the channel.", ctx.channel().remoteAddress(), cause); - failHandshake(ctx); - } - - @VisibleForTesting - public State getState() - { - return state; - } - - @VisibleForTesting - public void setState(State nextState) - { - state = nextState; - } - - @VisibleForTesting - void setHandshakeTimeout(Future timeout) - { - handshakeTimeout = timeout; - } -} diff --git a/src/java/org/apache/cassandra/net/async/MessageInHandler.java b/src/java/org/apache/cassandra/net/async/MessageInHandler.java deleted file mode 100644 index dafa9933331b..000000000000 --- a/src/java/org/apache/cassandra/net/async/MessageInHandler.java +++ /dev/null @@ -1,211 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.util.Collections; -import java.util.EnumMap; -import java.util.List; -import java.util.Map; -import java.util.function.BiConsumer; -import java.util.function.BooleanSupplier; - -import com.google.common.primitives.Ints; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; -import org.apache.cassandra.utils.vint.VIntCoding; - -/** - * Parses incoming messages as per the 4.0 internode messaging protocol. - */ -public class MessageInHandler extends BaseMessageInHandler -{ - public static final Logger logger = LoggerFactory.getLogger(MessageInHandler.class); - - private MessageHeader messageHeader; - - MessageInHandler(InetAddressAndPort peer, int messagingVersion) - { - this (peer, messagingVersion, MESSAGING_SERVICE_CONSUMER); - } - - public MessageInHandler(InetAddressAndPort peer, int messagingVersion, BiConsumer messageConsumer) - { - super(peer, messagingVersion, messageConsumer); - - assert messagingVersion >= MessagingService.VERSION_40 : String.format("wrong messaging version for this handler: got %d, but expect %d or higher", - messagingVersion, MessagingService.VERSION_40); - state = State.READ_FIRST_CHUNK; - } - - /** - * For each new message coming in, builds up a {@link MessageHeader} instance incrementally. This method - * attempts to deserialize as much header information as it can out of the incoming {@link ByteBuf}, and - * maintains a trivial state machine to remember progress across invocations. - */ - @SuppressWarnings("resource") - public void handleDecode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception - { - ByteBufDataInputPlus inputPlus = new ByteBufDataInputPlus(in); - while (true) - { - switch (state) - { - case READ_FIRST_CHUNK: - MessageHeader header = readFirstChunk(in); - if (header == null) - return; - header.from = peer; - messageHeader = header; - state = State.READ_VERB; - // fall-through - case READ_VERB: - if (in.readableBytes() < VERB_LENGTH) - return; - messageHeader.verb = MessagingService.Verb.fromId(in.readInt()); - state = State.READ_PARAMETERS_SIZE; - // fall-through - case READ_PARAMETERS_SIZE: - long length = VIntCoding.readUnsignedVInt(in); - if (length < 0) - return; - messageHeader.parameterLength = (int) length; - messageHeader.parameters = messageHeader.parameterLength == 0 ? Collections.emptyMap() : new EnumMap<>(ParameterType.class); - state = State.READ_PARAMETERS_DATA; - // fall-through - case READ_PARAMETERS_DATA: - if (messageHeader.parameterLength > 0) - { - if (in.readableBytes() < messageHeader.parameterLength) - return; - readParameters(in, inputPlus, messagingVersion, messageHeader.parameterLength, messageHeader.parameters); - } - state = State.READ_PAYLOAD_SIZE; - // fall-through - case READ_PAYLOAD_SIZE: - length = VIntCoding.readUnsignedVInt(in); - if (length < 0) - return; - messageHeader.payloadSize = (int) length; - state = State.READ_PAYLOAD; - // fall-through - case READ_PAYLOAD: - if (in.readableBytes() < messageHeader.payloadSize) - return; - - // TODO consider deserializing the message not on the event loop - MessageIn messageIn = MessageIn.read(inputPlus, messagingVersion, - messageHeader.messageId, messageHeader.constructionTime, messageHeader.from, - messageHeader.payloadSize, messageHeader.verb, messageHeader.parameters); - - if (messageIn != null) - messageConsumer.accept(messageIn, messageHeader.messageId); - - state = State.READ_FIRST_CHUNK; - messageHeader = null; - break; - default: - throw new IllegalStateException("unknown/unhandled state: " + state); - } - } - } - - private static void readParameters(ByteBuf buf, DataInputPlus in, int messagingVersion, int parameterLength, Map parameters) throws IOException - { - // makes the assumption we have all the bytes required to read the headers - final int endIndex = buf.readerIndex() + parameterLength; - while (buf.readerIndex() < endIndex) - { - String key = in.readUTF(); - ParameterType parameterType = ParameterType.byName.get(key); - long valueLength = in.readUnsignedVInt(); - byte[] value = new byte[Ints.checkedCast(valueLength)]; - in.readFully(value); - try (DataInputBuffer buffer = new DataInputBuffer(value)) - { - parameters.put(parameterType, parameterType.serializer.deserialize(buffer, messagingVersion)); - } - } - } - - private static void readParameters(BooleanSupplier isDone, DataInputPlus in, int messagingVersion, Map parameters) throws IOException - { - // makes the assumption we have all the bytes required to read the headers - while (!isDone.getAsBoolean()) - { - String key = in.readUTF(); - ParameterType parameterType = ParameterType.byName.get(key); - in.readUnsignedVInt(); - parameters.put(parameterType, parameterType.serializer.deserialize(in, messagingVersion)); - } - } - - public static MessageIn deserialize(DataInputPlus in, int id, int version, InetAddressAndPort from) throws IOException - { - if (version >= MessagingService.VERSION_40) - return deserialize40(in, id, version, from); - else - return MessageInHandlerPre40.deserializePre40(in, id, version, from); - } - - private static MessageIn deserialize40(DataInputPlus in, int id, int version, InetAddressAndPort from) throws IOException - { - MessagingService.Verb verb = MessagingService.Verb.fromId(in.readInt()); - - Map parameters = Collections.emptyMap(); - int parameterLength = (int) in.readUnsignedVInt(); - if (parameterLength != 0) - { - parameters = new EnumMap<>(ParameterType.class); - byte[] bytes = new byte[parameterLength]; - in.readFully(bytes); - try (DataInputBuffer buffer = new DataInputBuffer(bytes)) - { - readParameters(() -> buffer.available() == 0, buffer, version, parameters); - } - } - - Object payload = null; - int payloadSize = (int) in.readUnsignedVInt(); - if (payloadSize > 0) - { - IVersionedSerializer serializer = MessagingService.getVerbSerializer(verb, id); - if (serializer == null) in.skipBytesFully(payloadSize); - else payload = serializer.deserialize(in, version); - } - - return new MessageIn<>(from, payload, parameters, verb, version, System.nanoTime()); - } - - @Override - MessageHeader getMessageHeader() - { - return messageHeader; - } -} diff --git a/src/java/org/apache/cassandra/net/async/MessageInHandlerPre40.java b/src/java/org/apache/cassandra/net/async/MessageInHandlerPre40.java deleted file mode 100644 index 6eeeea7dd2e2..000000000000 --- a/src/java/org/apache/cassandra/net/async/MessageInHandlerPre40.java +++ /dev/null @@ -1,269 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.DataInputStream; -import java.io.IOException; -import java.util.Collections; -import java.util.EnumMap; -import java.util.List; -import java.util.Map; -import java.util.function.BiConsumer; -import java.util.function.BooleanSupplier; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; - -/** - * Parses incoming messages as per the pre-4.0 internode messaging protocol. - */ -public class MessageInHandlerPre40 extends BaseMessageInHandler -{ - public static final Logger logger = LoggerFactory.getLogger(MessageInHandlerPre40.class); - - static final int PARAMETERS_SIZE_LENGTH = Integer.BYTES; - static final int PARAMETERS_VALUE_SIZE_LENGTH = Integer.BYTES; - static final int PAYLOAD_SIZE_LENGTH = Integer.BYTES; - - private MessageHeader messageHeader; - - MessageInHandlerPre40(InetAddressAndPort peer, int messagingVersion) - { - this (peer, messagingVersion, MESSAGING_SERVICE_CONSUMER); - } - - public MessageInHandlerPre40(InetAddressAndPort peer, int messagingVersion, BiConsumer messageConsumer) - { - super(peer, messagingVersion, messageConsumer); - - assert messagingVersion < MessagingService.VERSION_40 : String.format("wrong messaging version for this handler: got %d, but expect lower than %d", - messagingVersion, MessagingService.VERSION_40); - state = State.READ_FIRST_CHUNK; - } - - /** - * For each new message coming in, builds up a {@link MessageHeader} instance incrementally. This method - * attempts to deserialize as much header information as it can out of the incoming {@link ByteBuf}, and - * maintains a trivial state machine to remember progress across invocations. - */ - @SuppressWarnings("resource") - public void handleDecode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception - { - ByteBufDataInputPlus inputPlus = new ByteBufDataInputPlus(in); - while (true) - { - switch (state) - { - case READ_FIRST_CHUNK: - MessageHeader header = readFirstChunk(in); - if (header == null) - return; - messageHeader = header; - state = State.READ_IP_ADDRESS; - // fall-through - case READ_IP_ADDRESS: - // unfortunately, this assumes knowledge of how CompactEndpointSerializationHelper serializes data (the first byte is the size). - // first, check that we can actually read the size byte, then check if we can read that number of bytes. - // the "+ 1" is to make sure we have the size byte in addition to the serialized IP addr count of bytes in the buffer. - int readableBytes = in.readableBytes(); - if (readableBytes < 1 || readableBytes < in.getByte(in.readerIndex()) + 1) - return; - messageHeader.from = CompactEndpointSerializationHelper.instance.deserialize(inputPlus, messagingVersion); - state = State.READ_VERB; - // fall-through - case READ_VERB: - if (in.readableBytes() < VERB_LENGTH) - return; - messageHeader.verb = MessagingService.Verb.fromId(in.readInt()); - state = State.READ_PARAMETERS_SIZE; - // fall-through - case READ_PARAMETERS_SIZE: - if (in.readableBytes() < PARAMETERS_SIZE_LENGTH) - return; - messageHeader.parameterLength = in.readInt(); - messageHeader.parameters = messageHeader.parameterLength == 0 ? Collections.emptyMap() : new EnumMap<>(ParameterType.class); - state = State.READ_PARAMETERS_DATA; - // fall-through - case READ_PARAMETERS_DATA: - if (messageHeader.parameterLength > 0) - { - if (!readParameters(in, inputPlus, messageHeader.parameterLength, messageHeader.parameters)) - return; - } - state = State.READ_PAYLOAD_SIZE; - // fall-through - case READ_PAYLOAD_SIZE: - if (in.readableBytes() < PAYLOAD_SIZE_LENGTH) - return; - messageHeader.payloadSize = in.readInt(); - state = State.READ_PAYLOAD; - // fall-through - case READ_PAYLOAD: - if (in.readableBytes() < messageHeader.payloadSize) - return; - - // TODO consider deserailizing the messge not on the event loop - MessageIn messageIn = MessageIn.read(inputPlus, messagingVersion, - messageHeader.messageId, messageHeader.constructionTime, messageHeader.from, - messageHeader.payloadSize, messageHeader.verb, messageHeader.parameters); - - if (messageIn != null) - messageConsumer.accept(messageIn, messageHeader.messageId); - - state = State.READ_FIRST_CHUNK; - messageHeader = null; - break; - default: - throw new IllegalStateException("unknown/unhandled state: " + state); - } - } - } - - /** - * @return true if all the parameters have been read from the {@link ByteBuf}; else, false. - */ - private boolean readParameters(ByteBuf in, ByteBufDataInputPlus inputPlus, int parameterCount, Map parameters) throws IOException - { - // makes the assumption that map.size() is a constant time function (HashMap.size() is) - while (parameters.size() < parameterCount) - { - if (!canReadNextParam(in)) - return false; - - String key = inputPlus.readUTF(); - ParameterType parameterType = ParameterType.byName.get(key); - byte[] value = new byte[inputPlus.readInt()]; - inputPlus.readFully(value); - try (DataInputBuffer buffer = new DataInputBuffer(value)) - { - parameters.put(parameterType, parameterType.serializer.deserialize(buffer, messagingVersion)); - } - } - - return true; - } - - private static boolean readParameters(DataInputPlus in, int messagingVersion, int parameterCount, Map parameters) throws IOException - { - // makes the assumption that map.size() is a constant time function (HashMap.size() is) - while (parameters.size() < parameterCount) - { - String key = in.readUTF(); - ParameterType parameterType = ParameterType.byName.get(key); - in.readInt(); - parameters.put(parameterType, parameterType.serializer.deserialize(in, messagingVersion)); - } - - return true; - } - - static MessageIn deserializePre40(DataInputPlus in, int id, int version, InetAddressAndPort from) throws IOException - { - assert from.equals(CompactEndpointSerializationHelper.instance.deserialize(in, version)); - MessagingService.Verb verb = MessagingService.Verb.fromId(in.readInt()); - - Map parameters = Collections.emptyMap(); - int parameterCount = in.readInt(); - if (parameterCount != 0) - { - parameters = new EnumMap<>(ParameterType.class); - readParameters(in, version, parameterCount, parameters); - } - - Object payload = null; - int payloadSize = in.readInt(); - if (payloadSize > 0) - { - IVersionedSerializer serializer = MessagingService.getVerbSerializer(verb, id); - if (serializer == null) in.skipBytesFully(payloadSize); - else payload = serializer.deserialize(in, version); - } - - return new MessageIn<>(from, payload, parameters, verb, version, System.nanoTime()); - } - - - - /** - * Determine if we can read the next parameter from the {@link ByteBuf}. This method will *always* set the {@code in} - * readIndex back to where it was when this method was invoked. - * - * NOTE: this function would be sooo much simpler if we included a parameters length int in the messaging format, - * instead of checking the remaining readable bytes for each field as we're parsing it. c'est la vie ... - */ - @VisibleForTesting - static boolean canReadNextParam(ByteBuf in) - { - in.markReaderIndex(); - // capture the readableBytes value here to avoid all the virtual function calls. - // subtract 6 as we know we'll be reading a short and an int (for the utf and value lengths). - final int minimumBytesRequired = 6; - int readableBytes = in.readableBytes() - minimumBytesRequired; - if (readableBytes < 0) - return false; - - // this is a tad invasive, but since we know the UTF string is prefaced with a 2-byte length, - // read that to make sure we have enough bytes to read the string itself. - short strLen = in.readShort(); - // check if we can read that many bytes for the UTF - if (strLen > readableBytes) - { - in.resetReaderIndex(); - return false; - } - in.skipBytes(strLen); - readableBytes -= strLen; - - // check if we can read the value length - if (readableBytes < PARAMETERS_VALUE_SIZE_LENGTH) - { - in.resetReaderIndex(); - return false; - } - int valueLength = in.readInt(); - // check if we read that many bytes for the value - if (valueLength > readableBytes) - { - in.resetReaderIndex(); - return false; - } - - in.resetReaderIndex(); - return true; - } - - - @Override - MessageHeader getMessageHeader() - { - return messageHeader; - } -} diff --git a/src/java/org/apache/cassandra/net/async/MessageOutHandler.java b/src/java/org/apache/cassandra/net/async/MessageOutHandler.java deleted file mode 100644 index f1647ab03809..000000000000 --- a/src/java/org/apache/cassandra/net/async/MessageOutHandler.java +++ /dev/null @@ -1,324 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelDuplexHandler; -import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelOutboundBuffer; -import io.netty.channel.ChannelPromise; -import io.netty.handler.codec.UnsupportedMessageTypeException; -import io.netty.handler.timeout.IdleState; -import io.netty.handler.timeout.IdleStateEvent; - -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; -import org.apache.cassandra.tracing.TraceState; -import org.apache.cassandra.tracing.Tracing; -import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis; -import org.apache.cassandra.utils.NoSpamLogger; -import org.apache.cassandra.utils.UUIDGen; - -import static org.apache.cassandra.config.Config.PROPERTY_PREFIX; - -/** - * A Netty {@link ChannelHandler} for serializing outbound messages. - *

- * On top of transforming a {@link QueuedMessage} into bytes, this handler also feeds back progress to the linked - * {@link ChannelWriter} so that the latter can take decision on when data should be flushed (with and without coalescing). - * See the javadoc on {@link ChannelWriter} for more details about the callbacks as well as message timeouts. - *

- * Note: this class derives from {@link ChannelDuplexHandler} so we can intercept calls to - * {@link #userEventTriggered(ChannelHandlerContext, Object)} and {@link #channelWritabilityChanged(ChannelHandlerContext)}. - */ -class MessageOutHandler extends ChannelDuplexHandler -{ - private static final Logger logger = LoggerFactory.getLogger(MessageOutHandler.class); - private static final NoSpamLogger errorLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.SECONDS); - - /** - * The default size threshold for deciding when to auto-flush the channel. - */ - private static final int DEFAULT_AUTO_FLUSH_THRESHOLD = 1 << 16; - - // reatining the pre 4.0 property name for backward compatibility. - private static final String AUTO_FLUSH_PROPERTY = PROPERTY_PREFIX + "otc_buffer_size"; - static final int AUTO_FLUSH_THRESHOLD = Integer.getInteger(AUTO_FLUSH_PROPERTY, DEFAULT_AUTO_FLUSH_THRESHOLD); - - /** - * The amount of prefix data, in bytes, before the serialized message. - */ - private static final int MESSAGE_PREFIX_SIZE = 12; - - private final OutboundConnectionIdentifier connectionId; - - /** - * The version of the messaging protocol we're communicating at. - */ - private final int targetMessagingVersion; - - /** - * The minumum size at which we'll automatically flush the channel. - */ - private final int flushSizeThreshold; - - private final ChannelWriter channelWriter; - - private final Supplier backlogSupplier; - - MessageOutHandler(OutboundConnectionIdentifier connectionId, int targetMessagingVersion, ChannelWriter channelWriter, Supplier backlogSupplier) - { - this (connectionId, targetMessagingVersion, channelWriter, backlogSupplier, AUTO_FLUSH_THRESHOLD); - } - - MessageOutHandler(OutboundConnectionIdentifier connectionId, int targetMessagingVersion, ChannelWriter channelWriter, Supplier backlogSupplier, int flushThreshold) - { - this.connectionId = connectionId; - this.targetMessagingVersion = targetMessagingVersion; - this.channelWriter = channelWriter; - this.flushSizeThreshold = flushThreshold; - this.backlogSupplier = backlogSupplier; - } - - @Override - public void write(ChannelHandlerContext ctx, Object o, ChannelPromise promise) - { - // this is a temporary fix until https://github.com/netty/netty/pull/6867 is released (probably netty 4.1.13). - // TL;DR a closed channel can still process messages in the pipeline that were queued before the close. - // the channel handlers are removed from the channel potentially saync from the close operation. - if (!ctx.channel().isOpen()) - { - logger.debug("attempting to process a message in the pipeline, but channel {} is closed", ctx.channel().id()); - return; - } - - ByteBuf out = null; - try - { - if (!isMessageValid(o, promise)) - return; - - QueuedMessage msg = (QueuedMessage) o; - - // frame size includes the magic and and other values *before* the actual serialized message. - // note: don't even bother to check the compressed size (if compression is enabled for the channel), - // cuz if it's this large already, we're probably screwed anyway - long currentFrameSize = MESSAGE_PREFIX_SIZE + msg.message.serializedSize(targetMessagingVersion); - if (currentFrameSize > Integer.MAX_VALUE || currentFrameSize < 0) - { - promise.tryFailure(new IllegalStateException(String.format("%s illegal frame size: %d, ignoring message", connectionId, currentFrameSize))); - return; - } - - out = ctx.alloc().ioBuffer((int)currentFrameSize); - - captureTracingInfo(msg); - serializeMessage(msg, out); - ctx.write(out, promise); - - // check to see if we should flush based on buffered size - ChannelOutboundBuffer outboundBuffer = ctx.channel().unsafe().outboundBuffer(); - if (outboundBuffer != null && outboundBuffer.totalPendingWriteBytes() >= flushSizeThreshold) - ctx.flush(); - } - catch(Exception e) - { - if (out != null && out.refCnt() > 0) - out.release(out.refCnt()); - exceptionCaught(ctx, e); - promise.tryFailure(e); - } - finally - { - // Make sure we signal the outChanel even in case of errors. - channelWriter.onMessageProcessed(ctx); - } - } - - /** - * Test to see if the message passed in is a {@link QueuedMessage} and if it has timed out or not. If the checks fail, - * this method has the side effect of modifying the {@link ChannelPromise}. - */ - boolean isMessageValid(Object o, ChannelPromise promise) - { - // optimize for the common case - if (o instanceof QueuedMessage) - { - if (!((QueuedMessage)o).isTimedOut()) - { - return true; - } - else - { - promise.tryFailure(ExpiredException.INSTANCE); - } - } - else - { - promise.tryFailure(new UnsupportedMessageTypeException(connectionId + - " msg must be an instance of " + QueuedMessage.class.getSimpleName())); - } - return false; - } - - /** - * Record any tracing data, if enabled on this message. - */ - @VisibleForTesting - void captureTracingInfo(QueuedMessage msg) - { - try - { - UUID sessionId = (UUID)msg.message.getParameter(ParameterType.TRACE_SESSION); - if (sessionId != null) - { - TraceState state = Tracing.instance.get(sessionId); - String message = String.format("Sending %s message to %s, size = %d bytes", - msg.message.verb, connectionId.connectionAddress(), - msg.message.serializedSize(targetMessagingVersion) + MESSAGE_PREFIX_SIZE); - // session may have already finished; see CASSANDRA-5668 - if (state == null) - { - Tracing.TraceType traceType = (Tracing.TraceType)msg.message.getParameter(ParameterType.TRACE_TYPE); - traceType = traceType == null ? Tracing.TraceType.QUERY : traceType; - Tracing.instance.trace(ByteBuffer.wrap(UUIDGen.decompose(sessionId)), message, traceType.getTTL()); - } - else - { - state.trace(message); - if (msg.message.verb == MessagingService.Verb.REQUEST_RESPONSE) - Tracing.instance.doneWithNonLocalSession(state); - } - } - } - catch (Exception e) - { - logger.warn("{} failed to capture the tracing info for an outbound message, ignoring", connectionId, e); - } - } - - private void serializeMessage(QueuedMessage msg, ByteBuf out) throws IOException - { - out.writeInt(MessagingService.PROTOCOL_MAGIC); - out.writeInt(msg.id); - - // int cast cuts off the high-order half of the timestamp, which we can assume remains - // the same between now and when the recipient reconstructs it. - out.writeInt((int) NanoTimeToCurrentTimeMillis.convert(msg.timestampNanos)); - @SuppressWarnings("resource") - DataOutputPlus outStream = new ByteBufDataOutputPlus(out); - msg.message.serialize(outStream, targetMessagingVersion); - - // next few lines are for debugging ... massively helpful!! - // if we allocated too much buffer for this message, we'll log here. - // if we allocated to little buffer space, we would have hit an exception when trying to write more bytes to it - if (out.isWritable()) - errorLogger.error("{} reported message size {}, actual message size {}, msg {}", - connectionId, out.capacity(), out.writerIndex(), msg.message); - } - - @Override - public void flush(ChannelHandlerContext ctx) - { - channelWriter.onTriggeredFlush(ctx); - } - - - /** - * {@inheritDoc} - * - * When the channel becomes writable (assuming it was previously unwritable), try to eat through any backlogged messages - * {@link #backlogSupplier}. As we're on the event loop when this is invoked, no one else can fill up the netty - * {@link ChannelOutboundBuffer}, so we should be able to make decent progress chewing through the backlog - * (assuming not large messages). Any messages messages written from {@link OutboundMessagingConnection} threads won't - * be processed immediately; they'll be queued up as tasks, and once this function return, those messages can begin - * to be consumed. - *

- * Note: this is invoked on the netty event loop. - */ - @Override - public void channelWritabilityChanged(ChannelHandlerContext ctx) - { - if (!ctx.channel().isWritable()) - return; - - // guarantee at least a minimal amount of progress (one messge from the backlog) by using a do-while loop. - do - { - QueuedMessage msg = backlogSupplier.get(); - if (msg == null || !channelWriter.write(msg, false)) - break; - } while (ctx.channel().isWritable()); - } - - /** - * {@inheritDoc} - * - * If we get an {@link IdleStateEvent} for the write path, we want to close the channel as we can't make progress. - * That assumes, of course, that there's any outstanding bytes in the channel to write. We don't necesarrily care - * about idleness (for example, gossip channels will be idle most of the time), but instead our concern is - * the ability to make progress when there's work to be done. - *

- * Note: this is invoked on the netty event loop. - */ - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) - { - if (evt instanceof IdleStateEvent && ((IdleStateEvent)evt).state() == IdleState.WRITER_IDLE) - { - ChannelOutboundBuffer cob = ctx.channel().unsafe().outboundBuffer(); - if (cob != null && cob.totalPendingWriteBytes() > 0) - { - ctx.channel().attr(ChannelWriter.PURGE_MESSAGES_CHANNEL_ATTR) - .compareAndSet(Boolean.FALSE, Boolean.TRUE); - ctx.close(); - } - } - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) - { - if (cause instanceof IOException) - logger.trace("{} io error", connectionId, cause); - else - logger.warn("{} error", connectionId, cause); - - ctx.close(); - } - - @Override - public void close(ChannelHandlerContext ctx, ChannelPromise promise) - { - ctx.flush(); - ctx.close(promise); - } -} diff --git a/src/java/org/apache/cassandra/net/async/MessageResult.java b/src/java/org/apache/cassandra/net/async/MessageResult.java deleted file mode 100644 index b0dc4dce1c66..000000000000 --- a/src/java/org/apache/cassandra/net/async/MessageResult.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import io.netty.util.concurrent.Future; - -/** - * A simple, reusable struct that holds the unprocessed result of sending a message via netty. This object is intended - * to be reusable to avoid creating a bunch of garbage (just for processing the results of sending a message). - * - * The intended use is to be a member field in a class, like {@link ChannelWriter}, repopulated on each message result, - * and then immediately cleared (via {@link #clearAll()}) when done. - */ -public class MessageResult -{ - ChannelWriter writer; - QueuedMessage msg; - Future future; - boolean allowReconnect; - - void setAll(ChannelWriter writer, QueuedMessage msg, Future future, boolean allowReconnect) - { - this.writer = writer; - this.msg = msg; - this.future = future; - this.allowReconnect = allowReconnect; - } - - void clearAll() - { - this.writer = null; - this.msg = null; - this.future = null; - } -} diff --git a/src/java/org/apache/cassandra/net/async/NettyFactory.java b/src/java/org/apache/cassandra/net/async/NettyFactory.java deleted file mode 100644 index 346a0672fe6c..000000000000 --- a/src/java/org/apache/cassandra/net/async/NettyFactory.java +++ /dev/null @@ -1,418 +0,0 @@ -package org.apache.cassandra.net.async; - -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.concurrent.TimeUnit; -import java.util.zip.Checksum; - -import javax.annotation.Nullable; -import javax.net.ssl.SSLEngine; -import javax.net.ssl.SSLParameters; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.bootstrap.Bootstrap; -import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelInitializer; -import io.netty.channel.ChannelOption; -import io.netty.channel.ChannelPipeline; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.ServerChannel; -import io.netty.channel.epoll.EpollChannelOption; -import io.netty.channel.epoll.EpollEventLoopGroup; -import io.netty.channel.epoll.EpollServerSocketChannel; -import io.netty.channel.epoll.EpollSocketChannel; -import io.netty.channel.group.ChannelGroup; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.SocketChannel; -import io.netty.channel.socket.nio.NioServerSocketChannel; -import io.netty.channel.socket.nio.NioSocketChannel; -import io.netty.handler.codec.compression.Lz4FrameDecoder; -import io.netty.handler.codec.compression.Lz4FrameEncoder; -import io.netty.handler.logging.LogLevel; -import io.netty.handler.logging.LoggingHandler; -import io.netty.handler.ssl.OpenSsl; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslHandler; -import io.netty.util.concurrent.DefaultEventExecutor; -import io.netty.util.concurrent.DefaultThreadFactory; -import io.netty.util.concurrent.EventExecutor; -import io.netty.util.internal.logging.InternalLoggerFactory; -import io.netty.util.internal.logging.Slf4JLoggerFactory; - -import net.jpountz.lz4.LZ4Factory; -import net.jpountz.xxhash.XXHashFactory; -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.security.SSLFactory; -import org.apache.cassandra.service.NativeTransportService; -import org.apache.cassandra.utils.ChecksumType; -import org.apache.cassandra.utils.CoalescingStrategies; -import org.apache.cassandra.utils.FBUtilities; - -/** - * A factory for building Netty {@link Channel}s. Channels here are setup with a pipeline to participate - * in the internode protocol handshake, either the inbound or outbound side as per the method invoked. - */ -public final class NettyFactory -{ - private static final Logger logger = LoggerFactory.getLogger(NettyFactory.class); - - /** - * The block size for use with netty's lz4 code. - */ - private static final int COMPRESSION_BLOCK_SIZE = 1 << 16; - - private static final int LZ4_HASH_SEED = 0x9747b28c; - - public enum Mode { MESSAGING, STREAMING } - - static final String SSL_CHANNEL_HANDLER_NAME = "ssl"; - private static final String OPTIONAL_SSL_CHANNEL_HANDLER_NAME = "optionalSsl"; - static final String INBOUND_COMPRESSOR_HANDLER_NAME = "inboundCompressor"; - static final String OUTBOUND_COMPRESSOR_HANDLER_NAME = "outboundCompressor"; - private static final String HANDSHAKE_HANDLER_NAME = "handshakeHandler"; - public static final String INBOUND_STREAM_HANDLER_NAME = "inboundStreamHandler"; - - /** a useful addition for debugging; simply set to true to get more data in your logs */ - private static final boolean WIRETRACE = false; - static - { - if (WIRETRACE) - InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE); - } - - private static final boolean DEFAULT_USE_EPOLL = NativeTransportService.useEpoll(); - - /** - * The size of the receive queue for the outbound channels. As outbound channels do not receive data - * (outside of the internode messaging protocol's handshake), this value can be relatively small. - */ - private static final int OUTBOUND_CHANNEL_RECEIVE_BUFFER_SIZE = 1 << 10; - - /** - * The size of the send queue for the inbound channels. As inbound channels do not send data - * (outside of the internode messaging protocol's handshake), this value can be relatively small. - */ - private static final int INBOUND_CHANNEL_SEND_BUFFER_SIZE = 1 << 10; - - /** - * A factory instance that all normal, runtime code should use. Separate instances should only be used for testing. - */ - public static final NettyFactory instance = new NettyFactory(DEFAULT_USE_EPOLL); - - private final boolean useEpoll; - private final EventLoopGroup acceptGroup; - - private final EventLoopGroup inboundGroup; - private final EventLoopGroup outboundGroup; - public final EventLoopGroup streamingGroup; - - /** - * Constructor that allows modifying the {@link NettyFactory#useEpoll} for testing purposes. Otherwise, use the - * default {@link #instance}. - */ - @VisibleForTesting - NettyFactory(boolean useEpoll) - { - this.useEpoll = useEpoll; - acceptGroup = getEventLoopGroup(useEpoll, determineAcceptGroupSize(DatabaseDescriptor.getInternodeMessagingEncyptionOptions()), - "MessagingService-NettyAcceptor-Thread", false); - inboundGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "MessagingService-NettyInbound-Thread", false); - outboundGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "MessagingService-NettyOutbound-Thread", true); - streamingGroup = getEventLoopGroup(useEpoll, FBUtilities.getAvailableProcessors(), "Streaming-Netty-Thread", false); - } - - /** - * Determine the number of accept threads we need, which is based upon the number of listening sockets we will have. - * The idea is one accept thread per listening socket. - */ - public static int determineAcceptGroupSize(ServerEncryptionOptions serverEncryptionOptions) - { - int listenSocketCount = 1; - - boolean listenOnBroadcastAddr = MessagingService.shouldListenOnBroadcastAddress(); - if (listenOnBroadcastAddr) - listenSocketCount++; - - if (serverEncryptionOptions.enable_legacy_ssl_storage_port) - { - listenSocketCount++; - - if (listenOnBroadcastAddr) - listenSocketCount++; - } - - return listenSocketCount; - } - - /** - * Create an {@link EventLoopGroup}, for epoll or nio. The {@code boostIoRatio} flag passes a hint to the netty - * event loop threads to optimize comsuming all the tasks from the netty channel before checking for IO activity. - * By default, netty will process some maximum number of tasks off it's queue before it will check for activity on - * any of the open FDs, which basically amounts to checking for any incoming data. If you have a class of event loops - * that that do almost *no* inbound activity (like cassandra's outbound channels), then it behooves us to have the - * outbound netty channel consume as many tasks as it can before making the system calls to check up on the FDs, - * as we're not expecting any incoming data on those sockets, anyways. Thus, we pass the magic value {@code 100} - * to achieve the maximum consuption from the netty queue. (for implementation details, as of netty 4.1.8, - * see {@link io.netty.channel.epoll.EpollEventLoop#run()}. - */ - static EventLoopGroup getEventLoopGroup(boolean useEpoll, int threadCount, String threadNamePrefix, boolean boostIoRatio) - { - if (useEpoll) - { - logger.debug("using netty epoll event loop for pool prefix {}", threadNamePrefix); - EpollEventLoopGroup eventLoopGroup = new EpollEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix, true)); - if (boostIoRatio) - eventLoopGroup.setIoRatio(100); - return eventLoopGroup; - } - - logger.debug("using netty nio event loop for pool prefix {}", threadNamePrefix); - NioEventLoopGroup eventLoopGroup = new NioEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix, true)); - if (boostIoRatio) - eventLoopGroup.setIoRatio(100); - return eventLoopGroup; - } - - /** - * Create a {@link Channel} that listens on the {@code localAddr}. This method will block while trying to bind to the address, - * but it does not make a remote call. - */ - public Channel createInboundChannel(InetAddressAndPort localAddr, InboundInitializer initializer, int receiveBufferSize) throws ConfigurationException - { - String nic = FBUtilities.getNetworkInterface(localAddr.address); - logger.info("Starting Messaging Service on {} {}, encryption: {}", - localAddr, nic == null ? "" : String.format(" (%s)", nic), encryptionLogStatement(initializer.encryptionOptions)); - Class transport = useEpoll ? EpollServerSocketChannel.class : NioServerSocketChannel.class; - ServerBootstrap bootstrap = new ServerBootstrap().group(acceptGroup, inboundGroup) - .channel(transport) - .option(ChannelOption.SO_BACKLOG, 500) - .childOption(ChannelOption.SO_KEEPALIVE, true) - .childOption(ChannelOption.TCP_NODELAY, true) - .childOption(ChannelOption.SO_REUSEADDR, true) - .childOption(ChannelOption.SO_SNDBUF, INBOUND_CHANNEL_SEND_BUFFER_SIZE) - .childHandler(initializer); - - if (useEpoll) - bootstrap.childOption(EpollChannelOption.TCP_USER_TIMEOUT, DatabaseDescriptor.getInternodeTcpUserTimeoutInMS()); - - if (receiveBufferSize > 0) - bootstrap.childOption(ChannelOption.SO_RCVBUF, receiveBufferSize); - - ChannelFuture channelFuture = bootstrap.bind(new InetSocketAddress(localAddr.address, localAddr.port)); - - if (!channelFuture.awaitUninterruptibly().isSuccess()) - { - if (channelFuture.channel().isOpen()) - channelFuture.channel().close(); - - Throwable failedChannelCause = channelFuture.cause(); - - String causeString = ""; - if (failedChannelCause != null && failedChannelCause.getMessage() != null) - causeString = failedChannelCause.getMessage(); - - if (causeString.contains("in use")) - { - throw new ConfigurationException(localAddr + " is in use by another process. Change listen_address:storage_port " + - "in cassandra.yaml to values that do not conflict with other services"); - } - // looking at the jdk source, solaris/windows bind failue messages both use the phrase "cannot assign requested address". - // windows message uses "Cannot" (with a capital 'C'), and solaris (a/k/a *nux) doe not. hence we search for "annot" - else if (causeString.contains("annot assign requested address")) - { - throw new ConfigurationException("Unable to bind to address " + localAddr - + ". Set listen_address in cassandra.yaml to an interface you can bind to, e.g., your private IP address on EC2"); - } - else - { - throw new ConfigurationException("failed to bind to: " + localAddr, failedChannelCause); - } - } - - return channelFuture.channel(); - } - - /** - * Creates a new {@link SslHandler} from provided SslContext. - * @param peer enables endpoint verification for remote address when not null - */ - static SslHandler newSslHandler(Channel channel, SslContext sslContext, @Nullable InetSocketAddress peer) - { - if (peer == null) - { - return sslContext.newHandler(channel.alloc()); - } - else - { - logger.debug("Creating SSL handler for {}:{}", peer.getHostString(), peer.getPort()); - SslHandler sslHandler = sslContext.newHandler(channel.alloc(), peer.getHostString(), peer.getPort()); - SSLEngine engine = sslHandler.engine(); - SSLParameters sslParameters = engine.getSSLParameters(); - sslParameters.setEndpointIdentificationAlgorithm("HTTPS"); - engine.setSSLParameters(sslParameters); - return sslHandler; - } - } - - public static class InboundInitializer extends ChannelInitializer - { - private final IInternodeAuthenticator authenticator; - private final ServerEncryptionOptions encryptionOptions; - private final ChannelGroup channelGroup; - - public InboundInitializer(IInternodeAuthenticator authenticator, ServerEncryptionOptions encryptionOptions, ChannelGroup channelGroup) - { - this.authenticator = authenticator; - this.encryptionOptions = encryptionOptions; - this.channelGroup = channelGroup; - } - - @Override - public void initChannel(SocketChannel channel) throws Exception - { - channelGroup.add(channel); - ChannelPipeline pipeline = channel.pipeline(); - - // order of handlers: ssl -> logger -> handshakeHandler - if (encryptionOptions.enabled) - { - if (encryptionOptions.optional) - { - pipeline.addFirst(OPTIONAL_SSL_CHANNEL_HANDLER_NAME, new OptionalSslHandler(encryptionOptions)); - } - else - { - SslContext sslContext = SSLFactory.getOrCreateSslContext(encryptionOptions, true, SSLFactory.SocketType.SERVER); - InetSocketAddress peer = encryptionOptions.require_endpoint_verification ? channel.remoteAddress() : null; - SslHandler sslHandler = newSslHandler(channel, sslContext, peer); - logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName()); - pipeline.addFirst(SSL_CHANNEL_HANDLER_NAME, sslHandler); - } - } - - if (WIRETRACE) - pipeline.addLast("logger", new LoggingHandler(LogLevel.INFO)); - - channel.pipeline().addLast(HANDSHAKE_HANDLER_NAME, new InboundHandshakeHandler(authenticator)); - } - } - - private static String encryptionLogStatement(ServerEncryptionOptions options) - { - if (options == null) - return "disabled"; - - String encryptionType = OpenSsl.isAvailable() ? "openssl" : "jdk"; - return "enabled (" + encryptionType + ')'; - } - - /** - * Create the {@link Bootstrap} for connecting to a remote peer. This method does not attempt to connect to the peer, - * and thus does not block. - */ - @VisibleForTesting - public Bootstrap createOutboundBootstrap(OutboundConnectionParams params) - { - logger.debug("creating outbound bootstrap to peer {}, compression: {}, encryption: {}, coalesce: {}, protocolVersion: {}", - params.connectionId.connectionAddress(), - params.compress, encryptionLogStatement(params.encryptionOptions), - params.coalescingStrategy.isPresent() ? params.coalescingStrategy.get() : CoalescingStrategies.Strategy.DISABLED, - params.protocolVersion); - Class transport = useEpoll ? EpollSocketChannel.class : NioSocketChannel.class; - Bootstrap bootstrap = new Bootstrap().group(params.mode == Mode.MESSAGING ? outboundGroup : streamingGroup) - .channel(transport) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, params.tcpConnectTimeoutInMS) - .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.SO_REUSEADDR, true) - .option(ChannelOption.SO_SNDBUF, params.sendBufferSize) - .option(ChannelOption.SO_RCVBUF, OUTBOUND_CHANNEL_RECEIVE_BUFFER_SIZE) - .option(ChannelOption.TCP_NODELAY, params.tcpNoDelay) - .option(ChannelOption.WRITE_BUFFER_WATER_MARK, params.waterMark) - .handler(new OutboundInitializer(params)); - if (useEpoll) - bootstrap.option(EpollChannelOption.TCP_USER_TIMEOUT, params.tcpUserTimeoutInMS); - - InetAddressAndPort remoteAddress = params.connectionId.connectionAddress(); - bootstrap.remoteAddress(new InetSocketAddress(remoteAddress.address, remoteAddress.port)); - return bootstrap; - } - - public static class OutboundInitializer extends ChannelInitializer - { - private final OutboundConnectionParams params; - - OutboundInitializer(OutboundConnectionParams params) - { - this.params = params; - } - - /** - * {@inheritDoc} - * - * To determine if we should enable TLS, we only need to check if {@link #params#encryptionOptions} is set. - * The logic for figuring that out is is located in {@link MessagingService#getMessagingConnection(InetAddress)}; - */ - public void initChannel(SocketChannel channel) throws Exception - { - ChannelPipeline pipeline = channel.pipeline(); - - // order of handlers: ssl -> logger -> handshakeHandler - if (params.encryptionOptions != null) - { - SslContext sslContext = SSLFactory.getOrCreateSslContext(params.encryptionOptions, true, SSLFactory.SocketType.CLIENT); - // for some reason channel.remoteAddress() will return null - InetAddressAndPort address = params.connectionId.remote(); - InetSocketAddress peer = params.encryptionOptions.require_endpoint_verification ? new InetSocketAddress(address.address, address.port) : null; - SslHandler sslHandler = newSslHandler(channel, sslContext, peer); - logger.trace("creating outbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName()); - pipeline.addFirst(SSL_CHANNEL_HANDLER_NAME, sslHandler); - } - - if (NettyFactory.WIRETRACE) - pipeline.addLast("logger", new LoggingHandler(LogLevel.INFO)); - - pipeline.addLast(HANDSHAKE_HANDLER_NAME, new OutboundHandshakeHandler(params)); - } - } - - public void close() throws InterruptedException - { - EventLoopGroup[] groups = new EventLoopGroup[] { acceptGroup, outboundGroup, inboundGroup, streamingGroup }; - for (EventLoopGroup group : groups) - group.shutdownGracefully(0, 2, TimeUnit.SECONDS); - for (EventLoopGroup group : groups) - group.awaitTermination(60, TimeUnit.SECONDS); - } - - static Lz4FrameEncoder createLz4Encoder(int protocolVersion) - { - return new Lz4FrameEncoder(LZ4Factory.fastestInstance(), false, COMPRESSION_BLOCK_SIZE, checksumForFrameEncoders(protocolVersion)); - } - - private static Checksum checksumForFrameEncoders(int protocolVersion) - { - if (protocolVersion >= MessagingService.current_version) - return ChecksumType.CRC32.newInstance(); - return XXHashFactory.fastestInstance().newStreamingHash32(LZ4_HASH_SEED).asChecksum(); - } - - static Lz4FrameDecoder createLz4Decoder(int protocolVersion) - { - return new Lz4FrameDecoder(LZ4Factory.fastestInstance(), checksumForFrameEncoders(protocolVersion)); - } - - public static EventExecutor executorForChannelGroups() - { - return new DefaultEventExecutor(); - } -} diff --git a/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java b/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java deleted file mode 100644 index 46f0ce162a34..000000000000 --- a/src/java/org/apache/cassandra/net/async/NonClosingDefaultFileRegion.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.File; -import java.nio.channels.FileChannel; - -import io.netty.channel.DefaultFileRegion; - -/** - * Netty's DefaultFileRegion closes the underlying FileChannel as soon as - * the refCnt() for the region drops to zero, this is an implementation of - * the DefaultFileRegion that doesn't close the FileChannel. - * - * See {@link ByteBufDataOutputStreamPlus} for its usage. - */ -public class NonClosingDefaultFileRegion extends DefaultFileRegion -{ - - public NonClosingDefaultFileRegion(FileChannel file, long position, long count) - { - super(file, position, count); - } - - public NonClosingDefaultFileRegion(File f, long position, long count) - { - super(f, position, count); - } - - @Override - protected void deallocate() - { - // Overridden to avoid closing the file - } -} diff --git a/src/java/org/apache/cassandra/net/async/OptionalSslHandler.java b/src/java/org/apache/cassandra/net/async/OptionalSslHandler.java deleted file mode 100644 index 3fb856244c51..000000000000 --- a/src/java/org/apache/cassandra/net/async/OptionalSslHandler.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.net.InetSocketAddress; -import java.util.List; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.Channel; -import io.netty.channel.ChannelHandlerContext; -import io.netty.handler.codec.ByteToMessageDecoder; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslHandler; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.security.SSLFactory; - -public class OptionalSslHandler extends ByteToMessageDecoder -{ - private final ServerEncryptionOptions encryptionOptions; - - OptionalSslHandler(ServerEncryptionOptions encryptionOptions) - { - this.encryptionOptions = encryptionOptions; - } - - protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception - { - if (in.readableBytes() < 5) - { - // To detect if SSL must be used we need to have at least 5 bytes, so return here and try again - // once more bytes a ready. - return; - } - - if (SslHandler.isEncrypted(in)) - { - // Connection uses SSL/TLS, replace the detection handler with a SslHandler and so use encryption. - SslContext sslContext = SSLFactory.getOrCreateSslContext(encryptionOptions, true, SSLFactory.SocketType.SERVER); - Channel channel = ctx.channel(); - InetSocketAddress peer = encryptionOptions.require_endpoint_verification ? (InetSocketAddress) channel.remoteAddress() : null; - SslHandler sslHandler = NettyFactory.newSslHandler(channel, sslContext, peer); - ctx.pipeline().replace(this, NettyFactory.SSL_CHANNEL_HANDLER_NAME, sslHandler); - } - else - { - // Connection use no TLS/SSL encryption, just remove the detection handler and continue without - // SslHandler in the pipeline. - ctx.pipeline().remove(this); - } - } -} diff --git a/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java b/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java deleted file mode 100644 index e3090657c947..000000000000 --- a/src/java/org/apache/cassandra/net/async/OutboundConnectionIdentifier.java +++ /dev/null @@ -1,194 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import com.carrotsearch.hppc.IntObjectMap; -import com.carrotsearch.hppc.IntObjectOpenHashMap; -import org.apache.cassandra.locator.InetAddressAndPort; - -/** - * Identifies an outbound messaging connection. - * - * This mainly hold the remote address and the type (small/large messages or gossip) of connection used, but with the - * additional detail that in some case (typically public EC2 address across regions) the address to which we connect - * to the remote is different from the address by which the node is known by the rest of the C*. - */ -public class OutboundConnectionIdentifier -{ - public enum ConnectionType - { - GOSSIP (0), LARGE_MESSAGE (1), SMALL_MESSAGE (2), STREAM (3); - - private final int id; - - ConnectionType(int id) - { - this.id = id; - } - - public int getId() - { - return id; - } - - private static final IntObjectMap idMap = new IntObjectOpenHashMap<>(values().length); - static - { - for (ConnectionType type : values()) - idMap.put(type.id, type); - } - - public static ConnectionType fromId(int id) - { - return idMap.get(id); - } - - } - - /** - * Memoization of the local node's broadcast address. - */ - private final InetAddressAndPort localAddr; - - /** - * The address by which the remote is identified. This may be different from {@link #remoteConnectionAddr} for - * something like EC2 public IP address which need to be used for communication between EC2 regions. - */ - private final InetAddressAndPort remoteAddr; - - /** - * The address to which we're connecting to the node (often the same as {@link #remoteAddr} but not always). - */ - private final InetAddressAndPort remoteConnectionAddr; - - private final ConnectionType connectionType; - - private OutboundConnectionIdentifier(InetAddressAndPort localAddr, - InetAddressAndPort remoteAddr, - InetAddressAndPort remoteConnectionAddr, - ConnectionType connectionType) - { - this.localAddr = localAddr; - this.remoteAddr = remoteAddr; - this.remoteConnectionAddr = remoteConnectionAddr; - this.connectionType = connectionType; - } - - private OutboundConnectionIdentifier(InetAddressAndPort localAddr, - InetAddressAndPort remoteAddr, - ConnectionType connectionType) - { - this(localAddr, remoteAddr, remoteAddr, connectionType); - } - - /** - * Creates an identifier for a small message connection and using the remote "identifying" address as its connection - * address. - */ - public static OutboundConnectionIdentifier small(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr) - { - return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.SMALL_MESSAGE); - } - - /** - * Creates an identifier for a large message connection and using the remote "identifying" address as its connection - * address. - */ - public static OutboundConnectionIdentifier large(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr) - { - return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.LARGE_MESSAGE); - } - - /** - * Creates an identifier for a gossip connection and using the remote "identifying" address as its connection - * address. - */ - public static OutboundConnectionIdentifier gossip(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr) - { - return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.GOSSIP); - } - - /** - * Creates an identifier for a gossip connection and using the remote "identifying" address as its connection - * address. - */ - public static OutboundConnectionIdentifier stream(InetAddressAndPort localAddr, InetAddressAndPort remoteAddr) - { - return new OutboundConnectionIdentifier(localAddr, remoteAddr, ConnectionType.STREAM); - } - - /** - * Returns a newly created connection identifier to the same remote that this identifier, but using the provided - * address as connection address. - * - * @param remoteConnectionAddr the address to use for connection to the remote in the new identifier. - * @return a newly created connection identifier that differs from this one only by using {@code remoteConnectionAddr} - * as connection address to the remote. - */ - public OutboundConnectionIdentifier withNewConnectionAddress(InetAddressAndPort remoteConnectionAddr) - { - return new OutboundConnectionIdentifier(localAddr, remoteAddr, remoteConnectionAddr, connectionType); - } - - public OutboundConnectionIdentifier withNewConnectionPort(int port) - { - return new OutboundConnectionIdentifier(localAddr, InetAddressAndPort.getByAddressOverrideDefaults(remoteAddr.address, port), - InetAddressAndPort.getByAddressOverrideDefaults(remoteConnectionAddr.address, port), connectionType); - } - - /** - * The local node address. - */ - public InetAddressAndPort local() - { - return localAddr; - } - - /** - * The remote node identifying address (the one to use for anything else than connecting to the node). - */ - public InetAddressAndPort remote() - { - return remoteAddr; - } - - /** - * The remote node connection address (the one to use to actually connect to the remote, and only that). - */ - public InetAddressAndPort connectionAddress() - { - return remoteConnectionAddr; - } - - /** - * The type of this connection. - */ - ConnectionType type() - { - return connectionType; - } - - @Override - public String toString() - { - return remoteAddr.equals(remoteConnectionAddr) - ? String.format("%s (%s)", remoteAddr, connectionType) - : String.format("%s on %s (%s)", remoteAddr, remoteConnectionAddr, connectionType); - } -} diff --git a/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java b/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java deleted file mode 100644 index 64968c6cf403..000000000000 --- a/src/java/org/apache/cassandra/net/async/OutboundConnectionParams.java +++ /dev/null @@ -1,230 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.util.Optional; -import java.util.function.Consumer; -import java.util.function.Supplier; - -import com.google.common.base.Preconditions; - -import io.netty.channel.WriteBufferWaterMark; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult; -import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy; - -/** - * A collection of data points to be passed around for outbound connections. - */ -public class OutboundConnectionParams -{ - public static final int DEFAULT_SEND_BUFFER_SIZE = 1 << 16; - - final OutboundConnectionIdentifier connectionId; - final Consumer callback; - final ServerEncryptionOptions encryptionOptions; - final NettyFactory.Mode mode; - final boolean compress; - final Optional coalescingStrategy; - final int sendBufferSize; - final boolean tcpNoDelay; - final Supplier backlogSupplier; - final Consumer messageResultConsumer; - final WriteBufferWaterMark waterMark; - final int protocolVersion; - final int tcpConnectTimeoutInMS; - final int tcpUserTimeoutInMS; - - private OutboundConnectionParams(OutboundConnectionIdentifier connectionId, - Consumer callback, - ServerEncryptionOptions encryptionOptions, - NettyFactory.Mode mode, - boolean compress, - Optional coalescingStrategy, - int sendBufferSize, - boolean tcpNoDelay, - Supplier backlogSupplier, - Consumer messageResultConsumer, - WriteBufferWaterMark waterMark, - int protocolVersion, - int tcpConnectTimeoutInMS, - int tcpUserTimeoutInMS) - { - this.connectionId = connectionId; - this.callback = callback; - this.encryptionOptions = encryptionOptions; - this.mode = mode; - this.compress = compress; - this.coalescingStrategy = coalescingStrategy; - this.sendBufferSize = sendBufferSize; - this.tcpNoDelay = tcpNoDelay; - this.backlogSupplier = backlogSupplier; - this.messageResultConsumer = messageResultConsumer; - this.waterMark = waterMark; - this.protocolVersion = protocolVersion; - this.tcpConnectTimeoutInMS = tcpConnectTimeoutInMS; - this.tcpUserTimeoutInMS = tcpUserTimeoutInMS; - } - - public static Builder builder() - { - return new Builder(); - } - - public static Builder builder(OutboundConnectionParams params) - { - return new Builder(params); - } - - public static class Builder - { - private OutboundConnectionIdentifier connectionId; - private Consumer callback; - private ServerEncryptionOptions encryptionOptions; - private NettyFactory.Mode mode; - private boolean compress; - private Optional coalescingStrategy = Optional.empty(); - private int sendBufferSize = DEFAULT_SEND_BUFFER_SIZE; - private boolean tcpNoDelay; - private Supplier backlogSupplier; - private Consumer messageResultConsumer; - private WriteBufferWaterMark waterMark = WriteBufferWaterMark.DEFAULT; - private int protocolVersion; - private int tcpConnectTimeoutInMS; - private int tcpUserTimeoutInMS; - - private Builder() - { - this.tcpConnectTimeoutInMS = DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS(); - this.tcpUserTimeoutInMS = DatabaseDescriptor.getInternodeTcpUserTimeoutInMS(); - } - - private Builder(OutboundConnectionParams params) - { - this.connectionId = params.connectionId; - this.callback = params.callback; - this.encryptionOptions = params.encryptionOptions; - this.mode = params.mode; - this.compress = params.compress; - this.coalescingStrategy = params.coalescingStrategy; - this.sendBufferSize = params.sendBufferSize; - this.tcpNoDelay = params.tcpNoDelay; - this.backlogSupplier = params.backlogSupplier; - this.messageResultConsumer = params.messageResultConsumer; - this.tcpConnectTimeoutInMS = params.tcpConnectTimeoutInMS; - this.tcpUserTimeoutInMS = params.tcpUserTimeoutInMS; - } - - public Builder connectionId(OutboundConnectionIdentifier connectionId) - { - this.connectionId = connectionId; - return this; - } - - public Builder callback(Consumer callback) - { - this.callback = callback; - return this; - } - - public Builder encryptionOptions(ServerEncryptionOptions encryptionOptions) - { - this.encryptionOptions = encryptionOptions; - return this; - } - - public Builder mode(NettyFactory.Mode mode) - { - this.mode = mode; - return this; - } - - public Builder compress(boolean compress) - { - this.compress = compress; - return this; - } - - public Builder coalescingStrategy(Optional coalescingStrategy) - { - this.coalescingStrategy = coalescingStrategy; - return this; - } - - public Builder sendBufferSize(int sendBufferSize) - { - this.sendBufferSize = sendBufferSize; - return this; - } - - public Builder tcpNoDelay(boolean tcpNoDelay) - { - this.tcpNoDelay = tcpNoDelay; - return this; - } - - public Builder backlogSupplier(Supplier backlogSupplier) - { - this.backlogSupplier = backlogSupplier; - return this; - } - - public Builder messageResultConsumer(Consumer messageResultConsumer) - { - this.messageResultConsumer = messageResultConsumer; - return this; - } - - public Builder waterMark(WriteBufferWaterMark waterMark) - { - this.waterMark = waterMark; - return this; - } - - public Builder protocolVersion(int protocolVersion) - { - this.protocolVersion = protocolVersion; - return this; - } - - public Builder tcpConnectTimeoutInMS(int tcpConnectTimeoutInMS) - { - this.tcpConnectTimeoutInMS = tcpConnectTimeoutInMS; - return this; - } - - public Builder tcpUserTimeoutInMS(int tcpUserTimeoutInMS) - { - this.tcpUserTimeoutInMS = tcpUserTimeoutInMS; - return this; - } - - public OutboundConnectionParams build() - { - Preconditions.checkArgument(protocolVersion > 0, "illegal protocol version: " + protocolVersion); - Preconditions.checkArgument(sendBufferSize > 0 && sendBufferSize < 1 << 20, "illegal send buffer size: " + sendBufferSize); - Preconditions.checkArgument(tcpUserTimeoutInMS >= 0, "tcp user timeout must be non negative: " + tcpUserTimeoutInMS); - Preconditions.checkArgument(tcpConnectTimeoutInMS > 0, "tcp connect timeout must be positive: " + tcpConnectTimeoutInMS); - - return new OutboundConnectionParams(connectionId, callback, encryptionOptions, mode, compress, coalescingStrategy, sendBufferSize, - tcpNoDelay, backlogSupplier, messageResultConsumer, waterMark, protocolVersion, tcpConnectTimeoutInMS, tcpUserTimeoutInMS); - } - } -} diff --git a/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java b/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java deleted file mode 100644 index 3ccbf49ccf3c..000000000000 --- a/src/java/org/apache/cassandra/net/async/OutboundHandshakeHandler.java +++ /dev/null @@ -1,262 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.function.Consumer; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandler; -import io.netty.channel.ChannelPipeline; -import io.netty.handler.codec.ByteToMessageDecoder; -import io.netty.handler.timeout.IdleStateHandler; -import io.netty.util.concurrent.Future; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage; - -import static org.apache.cassandra.config.Config.PROPERTY_PREFIX; - -/** - * A {@link ChannelHandler} to execute the send-side of the internode communication handshake protocol. - * As soon as the handler is added to the channel via {@link #channelActive(ChannelHandlerContext)} - * (which is only invoked if the underlying TCP connection was properly established), the {@link FirstHandshakeMessage} - * of the internode messaging protocol is automatically sent out. See {@link HandshakeProtocol} for full details - * about the internode messaging hndshake protocol. - *

- * Upon completion of the handshake (on success or fail), the {@link #callback} is invoked to let the listener - * know the result of the handshake. See {@link HandshakeResult} for details about the different result states. - *

- * This class extends {@link ByteToMessageDecoder}, which is a {@link ChannelInboundHandler}, because this handler - * waits for the peer's handshake response (the {@link SecondHandshakeMessage} of the internode messaging handshake protocol). - */ -public class OutboundHandshakeHandler extends ByteToMessageDecoder -{ - private static final Logger logger = LoggerFactory.getLogger(OutboundHandshakeHandler.class); - - /** - * The number of milliseconds to wait before closing a channel if there has been no progress (when there is - * data to be sent).See {@link IdleStateHandler} and {@link MessageOutHandler#userEventTriggered(ChannelHandlerContext, Object)}. - */ - private static final long DEFAULT_WRITE_IDLE_MS = TimeUnit.SECONDS.toMillis(10); - private static final String WRITE_IDLE_PROPERTY = PROPERTY_PREFIX + "outbound_write_idle_ms"; - private static final long WRITE_IDLE_MS = Long.getLong(WRITE_IDLE_PROPERTY, DEFAULT_WRITE_IDLE_MS); - - private final OutboundConnectionIdentifier connectionId; - - /** - * The expected messaging service version to use. - */ - private final int messagingVersion; - - /** - * A function to invoke upon completion of the attempt, success or failure, to connect to the peer. - */ - private final Consumer callback; - private final NettyFactory.Mode mode; - private final OutboundConnectionParams params; - - OutboundHandshakeHandler(OutboundConnectionParams params) - { - this.params = params; - this.connectionId = params.connectionId; - this.messagingVersion = params.protocolVersion; - this.callback = params.callback; - this.mode = params.mode; - } - - /** - * {@inheritDoc} - * - * Invoked when the channel is made active, and sends out the {@link FirstHandshakeMessage}. - * In the case of streaming, we do not require a full bi-directional handshake; the initial message, - * containing the streaming protocol version, is all that is required. - */ - @Override - public void channelActive(final ChannelHandlerContext ctx) throws Exception - { - FirstHandshakeMessage msg = new FirstHandshakeMessage(messagingVersion, mode, params.compress); - logger.trace("starting handshake with peer {}, msg = {}", connectionId.connectionAddress(), msg); - ctx.writeAndFlush(msg.encode(ctx.alloc())).addListener(future -> firstHandshakeMessageListener(future, ctx)); - - if (mode == NettyFactory.Mode.STREAMING) - ctx.pipeline().remove(this); - - ctx.fireChannelActive(); - } - - /** - * A simple listener to make sure we could send the {@link FirstHandshakeMessage} to the socket, - * and fail the handshake attempt if we could not (for example, maybe we could create the TCP socket, but then - * the connection gets closed for some reason). - */ - void firstHandshakeMessageListener(Future future, ChannelHandlerContext ctx) - { - if (future.isSuccess()) - return; - - ChannelFuture channelFuture = (ChannelFuture)future; - exceptionCaught(ctx, channelFuture.cause()); - } - - /** - * {@inheritDoc} - * - * Invoked when we get the response back from the peer, which should contain the second message of the internode messaging handshake. - *

- * If the peer's protocol version does not equal what we were expecting, immediately close the channel (and socket); - * do *not* send out the third message of the internode messaging handshake. - * We will reconnect on the appropriate protocol version. - */ - @Override - protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception - { - SecondHandshakeMessage msg = SecondHandshakeMessage.maybeDecode(in); - if (msg == null) - return; - - logger.trace("received second handshake message from peer {}, msg = {}", connectionId.connectionAddress(), msg); - final int peerMessagingVersion = msg.messagingVersion; - - // we expected a higher protocol version, but it was actually lower - if (messagingVersion > peerMessagingVersion) - { - logger.trace("peer's max version is {}; will reconnect with that version", peerMessagingVersion); - try - { - if (DatabaseDescriptor.getSeeds().contains(connectionId.remote())) - logger.warn("Seed gossip version is {}; will not connect with that version", peerMessagingVersion); - } - catch (Throwable e) - { - // If invalid yaml has been added to the config since startup, getSeeds() will throw an AssertionError - // Additionally, third party seed providers may throw exceptions if network is flakey. - // Regardless of what's thrown, we must catch it, disconnect, and try again - logger.warn("failed to reread yaml (on trying to connect to a seed): {}", e.getLocalizedMessage()); - } - ctx.close(); - callback.accept(HandshakeResult.disconnect(peerMessagingVersion)); - return; - } - // we anticipate a version that is lower than what peer is actually running - else if (messagingVersion < peerMessagingVersion && messagingVersion < MessagingService.current_version) - { - logger.trace("peer has a higher max version than expected {} (previous value {})", peerMessagingVersion, messagingVersion); - ctx.close(); - callback.accept(HandshakeResult.disconnect(peerMessagingVersion)); - return; - } - - try - { - ctx.writeAndFlush(new ThirdHandshakeMessage(peerMessagingVersion, connectionId.local()).encode(ctx.alloc())); - ChannelWriter channelWriter = setupPipeline(ctx.channel(), peerMessagingVersion); - callback.accept(HandshakeResult.success(channelWriter, peerMessagingVersion)); - } - catch (Exception e) - { - logger.info("failed to finalize internode messaging handshake", e); - ctx.close(); - callback.accept(HandshakeResult.failed()); - } - } - - @VisibleForTesting - ChannelWriter setupPipeline(Channel channel, int messagingVersion) - { - ChannelPipeline pipeline = channel.pipeline(); - pipeline.addLast("idleWriteHandler", new IdleStateHandler(true, 0, WRITE_IDLE_MS, 0, TimeUnit.MILLISECONDS)); - if (params.compress) - pipeline.addLast(NettyFactory.OUTBOUND_COMPRESSOR_HANDLER_NAME, NettyFactory.createLz4Encoder(messagingVersion)); - - ChannelWriter channelWriter = ChannelWriter.create(channel, params.messageResultConsumer, params.coalescingStrategy); - pipeline.addLast("messageOutHandler", new MessageOutHandler(connectionId, messagingVersion, channelWriter, params.backlogSupplier)); - pipeline.remove(this); - return channelWriter; - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) - { - logger.error("Failed to properly handshake with peer {}. Closing the channel.", connectionId, cause); - ctx.close(); - callback.accept(HandshakeResult.failed()); - } - - /** - * The result of the handshake. Handshake has 3 possible outcomes: - * 1) it can be successful, in which case the channel and version to used is returned in this result. - * 2) we may decide to disconnect to reconnect with another protocol version (namely, the version is passed in this result). - * 3) we can have a negotiation failure for an unknown reason. (#sadtrombone) - */ - public static class HandshakeResult - { - static final int UNKNOWN_PROTOCOL_VERSION = -1; - - /** - * Describes the result of receiving the response back from the peer (Message 2 of the handshake) - * and implies an action that should be taken. - */ - enum Outcome - { - SUCCESS, DISCONNECT, NEGOTIATION_FAILURE - } - - /** The channel for the connection, only set for successful handshake. */ - final ChannelWriter channelWriter; - /** The version negotiated with the peer. Set unless this is a {@link Outcome#NEGOTIATION_FAILURE}. */ - final int negotiatedMessagingVersion; - /** The handshake {@link Outcome}. */ - final Outcome outcome; - - private HandshakeResult(ChannelWriter channelWriter, int negotiatedMessagingVersion, Outcome outcome) - { - this.channelWriter = channelWriter; - this.negotiatedMessagingVersion = negotiatedMessagingVersion; - this.outcome = outcome; - } - - static HandshakeResult success(ChannelWriter channel, int negotiatedMessagingVersion) - { - return new HandshakeResult(channel, negotiatedMessagingVersion, Outcome.SUCCESS); - } - - static HandshakeResult disconnect(int negotiatedMessagingVersion) - { - return new HandshakeResult(null, negotiatedMessagingVersion, Outcome.DISCONNECT); - } - - static HandshakeResult failed() - { - return new HandshakeResult(null, UNKNOWN_PROTOCOL_VERSION, Outcome.NEGOTIATION_FAILURE); - } - } -} diff --git a/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java b/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java deleted file mode 100644 index 265ece9b26ee..000000000000 --- a/src/java/org/apache/cassandra/net/async/OutboundMessagingConnection.java +++ /dev/null @@ -1,747 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.util.Optional; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.util.concurrent.Future; -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.concurrent.ScheduledExecutors; -import org.apache.cassandra.config.Config; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.NettyFactory.Mode; -import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult; -import org.apache.cassandra.utils.CoalescingStrategies; -import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy; -import org.apache.cassandra.utils.JVMStabilityInspector; -import org.apache.cassandra.utils.NoSpamLogger; - -/** - * Represents one connection to a peer, and handles the state transistions on the connection and the netty {@link Channel} - * The underlying socket is not opened until explicitly requested (by sending a message). - * - * The basic setup for the channel is like this: a message is requested to be sent via {@link #sendMessage(MessageOut, int)}. - * If the channel is not established, then we need to create it (obviously). To prevent multiple threads from creating - * independent connections, they attempt to update the {@link #state}; one thread will win the race and create the connection. - * Upon sucessfully setting up the connection/channel, the {@link #state} will be updated again (to {@link State#READY}, - * which indicates to other threads that the channel is ready for business and can be written to. - * - */ -public class OutboundMessagingConnection -{ - static final Logger logger = LoggerFactory.getLogger(OutboundMessagingConnection.class); - private static final NoSpamLogger errorLogger = NoSpamLogger.getLogger(logger, 10, TimeUnit.SECONDS); - - private static final String INTRADC_TCP_NODELAY_PROPERTY = Config.PROPERTY_PREFIX + "otc_intradc_tcp_nodelay"; - - /** - * Enabled/disable TCP_NODELAY for intradc connections. Defaults to enabled. - */ - private static final boolean INTRADC_TCP_NODELAY = Boolean.parseBoolean(System.getProperty(INTRADC_TCP_NODELAY_PROPERTY, "true")); - - /** - * Number of milliseconds between connection createRetry attempts. - */ - private static final int OPEN_RETRY_DELAY_MS = 100; - - /** - * A minimum number of milliseconds to wait for a connection (TCP socket connect + handshake) - */ - private static final int MINIMUM_CONNECT_TIMEOUT_MS = 2000; - private final IInternodeAuthenticator authenticator; - - /** - * Describes this instance's ability to send messages into it's Netty {@link Channel}. - */ - enum State - { - /** waiting to create the connection */ - NOT_READY, - /** we've started to create the connection/channel */ - CREATING_CHANNEL, - /** channel is established and we can send messages */ - READY, - /** a dead state which should not be transitioned away from */ - CLOSED - } - - /** - * Backlog to hold messages passed by upstream threads while the Netty {@link Channel} is being set up or recreated. - */ - private final Queue backlog; - - /** - * Reference to a {@link ScheduledExecutorService} rther than directly depending on something like {@link ScheduledExecutors}. - */ - private final ScheduledExecutorService scheduledExecutor; - - final AtomicLong droppedMessageCount; - final AtomicLong completedMessageCount; - - private volatile OutboundConnectionIdentifier connectionId; - - private final ServerEncryptionOptions encryptionOptions; - - /** - * A future for retrying connections. Bear in mind that this future does not execute in the - * netty event event loop, so there's some races to be careful of. - */ - private volatile ScheduledFuture connectionRetryFuture; - - /** - * A future for notifying when the timeout for creating the connection and negotiating the handshake has elapsed. - * It will be cancelled when the channel is established correctly. Bear in mind that this future does not execute in the - * netty event event loop, so there's some races to be careful of. - */ - private volatile ScheduledFuture connectionTimeoutFuture; - - private final AtomicReference state; - - private final Optional coalescingStrategy; - - /** - * A running count of the number of times we've tried to create a connection. - */ - private volatile int connectAttemptCount; - - /** - * The netty channel, once a socket connection is established; it won't be in it's normal working state until the handshake is complete. - */ - private volatile ChannelWriter channelWriter; - - /** - * the target protocol version to communicate to the peer with, discovered/negotiated via handshaking - */ - private int targetVersion; - - OutboundMessagingConnection(OutboundConnectionIdentifier connectionId, - ServerEncryptionOptions encryptionOptions, - Optional coalescingStrategy, - IInternodeAuthenticator authenticator) - { - this(connectionId, encryptionOptions, coalescingStrategy, authenticator, ScheduledExecutors.scheduledFastTasks); - } - - @VisibleForTesting - OutboundMessagingConnection(OutboundConnectionIdentifier connectionId, - ServerEncryptionOptions encryptionOptions, - Optional coalescingStrategy, - IInternodeAuthenticator authenticator, - ScheduledExecutorService sceduledExecutor) - { - this.connectionId = connectionId; - this.encryptionOptions = encryptionOptions; - this.authenticator = authenticator; - backlog = new ConcurrentLinkedQueue<>(); - droppedMessageCount = new AtomicLong(0); - completedMessageCount = new AtomicLong(0); - state = new AtomicReference<>(State.NOT_READY); - this.scheduledExecutor = sceduledExecutor; - this.coalescingStrategy = coalescingStrategy; - - // We want to use the most precise protocol version we know because while there is version detection on connect(), - // the target version might be accessed by the pool (in getConnection()) before we actually connect (as we - // only connect when the first message is submitted). Note however that the only case where we'll connect - // without knowing the true version of a node is if that node is a seed (otherwise, we can't know a node - // unless it has been gossiped to us or it has connected to us, and in both cases that will set the version). - // In that case we won't rely on that targetVersion before we're actually connected and so the version - // detection in connect() will do its job. - targetVersion = MessagingService.instance().getVersion(connectionId.remote()); - } - - /** - * If the connection is set up and ready to use (the normal case), simply send the message to it and return. - * Otherwise, one lucky thread is selected to create the Channel, while other threads just add the {@code msg} to - * the backlog queue. - * - * @return true if the message was accepted by the {@link #channelWriter}; else false if it was not accepted - * and added to the backlog or the channel is {@link State#CLOSED}. See documentation in {@link ChannelWriter} and - * {@link MessageOutHandler} how the backlogged messages get consumed. - */ - boolean sendMessage(MessageOut msg, int id) - { - return sendMessage(new QueuedMessage(msg, id)); - } - - boolean sendMessage(QueuedMessage queuedMessage) - { - State state = this.state.get(); - if (state == State.READY) - { - if (channelWriter.write(queuedMessage, false)) - return true; - - backlog.add(queuedMessage); - return false; - } - else if (state == State.CLOSED) - { - errorLogger.warn("trying to write message to a closed connection"); - return false; - } - else - { - backlog.add(queuedMessage); - connect(); - return true; - } - } - - /** - * Initiate all the actions required to establish a working, valid connection. This includes - * opening the socket, negotiating the internode messaging handshake, and setting up the working - * Netty {@link Channel}. However, this method will not block for all those actions: it will only - * kick off the connection attempt as everything is asynchronous. - *

- * Threads compete to update the {@link #state} field to {@link State#CREATING_CHANNEL} to ensure only one - * connection is attempted at a time. - * - * @return true if kicking off the connection attempt was started by this thread; else, false. - */ - public boolean connect() - { - // try to be the winning thread to create the channel - if (!state.compareAndSet(State.NOT_READY, State.CREATING_CHANNEL)) - return false; - - // clean up any lingering connection attempts - if (connectionTimeoutFuture != null) - { - connectionTimeoutFuture.cancel(false); - connectionTimeoutFuture = null; - } - - return tryConnect(); - } - - private boolean tryConnect() - { - if (state.get() != State.CREATING_CHANNEL) - return false; - - logger.debug("connection attempt {} to {}", connectAttemptCount, connectionId); - - - InetAddressAndPort remote = connectionId.remote(); - if (!authenticator.authenticate(remote.address, remote.port)) - { - logger.warn("Internode auth failed connecting to {}", connectionId); - //Remove the connection pool and other thread so messages aren't queued - MessagingService.instance().destroyConnectionPool(remote); - - // don't update the state field as destroyConnectionPool() *should* call OMC.close() - // on all the connections in the OMP for the remoteAddress - return false; - } - - boolean compress = shouldCompressConnection(connectionId.local(), connectionId.remote()); - maybeUpdateConnectionId(); - Bootstrap bootstrap = buildBootstrap(compress); - - ChannelFuture connectFuture = bootstrap.connect(); - connectFuture.addListener(this::connectCallback); - - long timeout = Math.max(MINIMUM_CONNECT_TIMEOUT_MS, DatabaseDescriptor.getRpcTimeout()); - if (connectionTimeoutFuture == null || connectionTimeoutFuture.isDone()) - connectionTimeoutFuture = scheduledExecutor.schedule(() -> connectionTimeout(connectFuture), timeout, TimeUnit.MILLISECONDS); - return true; - } - - @VisibleForTesting - static boolean shouldCompressConnection(InetAddressAndPort localHost, InetAddressAndPort remoteHost) - { - return (DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.all) - || ((DatabaseDescriptor.internodeCompression() == Config.InternodeCompression.dc) && !isLocalDC(localHost, remoteHost)); - } - - /** - * After a bounce we won't necessarily know the peer's version, so we assume the peer is at least 4.0 - * and thus using a single port for secure and non-secure communication. However, during a rolling upgrade from - * 3.0.x/3.x to 4.0, the not-yet upgraded peer is still listening on separate ports, but we don't know the peer's - * version until we can successfully connect. Fortunately, the peer can connect to this node, at which point - * we'll grab it's version. We then use that knowledge to use the {@link Config#ssl_storage_port} to connect on, - * and to do that we need to update some member fields in this instance. - * - * Note: can be removed at 5.0 - */ - void maybeUpdateConnectionId() - { - if (encryptionOptions != null) - { - int version = MessagingService.instance().getVersion(connectionId.remote()); - if (version < targetVersion) - { - targetVersion = version; - int port = MessagingService.instance().portFor(connectionId.remote()); - connectionId = connectionId.withNewConnectionPort(port); - logger.debug("changing connectionId to {}, with a different port for secure communication, because peer version is {}", connectionId, version); - } - } - } - - private Bootstrap buildBootstrap(boolean compress) - { - boolean tcpNoDelay = isLocalDC(connectionId.local(), connectionId.remote()) ? INTRADC_TCP_NODELAY : DatabaseDescriptor.getInterDCTcpNoDelay(); - int sendBufferSize = DatabaseDescriptor.getInternodeSendBufferSize() > 0 - ? DatabaseDescriptor.getInternodeSendBufferSize() - : OutboundConnectionParams.DEFAULT_SEND_BUFFER_SIZE; - - int tcpConnectTimeout = DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS(); - int tcpUserTimeout = DatabaseDescriptor.getInternodeTcpUserTimeoutInMS(); - - OutboundConnectionParams params = OutboundConnectionParams.builder() - .connectionId(connectionId) - .callback(this::finishHandshake) - .encryptionOptions(encryptionOptions) - .mode(Mode.MESSAGING) - .compress(compress) - .coalescingStrategy(coalescingStrategy) - .sendBufferSize(sendBufferSize) - .tcpNoDelay(tcpNoDelay) - .tcpConnectTimeoutInMS(tcpConnectTimeout) - .tcpUserTimeoutInMS(tcpUserTimeout) - .backlogSupplier(() -> nextBackloggedMessage()) - .messageResultConsumer(this::handleMessageResult) - .protocolVersion(targetVersion) - .build(); - - return NettyFactory.instance.createOutboundBootstrap(params); - } - - private QueuedMessage nextBackloggedMessage() - { - QueuedMessage msg = backlog.poll(); - if (msg == null) - return null; - - if (!msg.isTimedOut()) - return msg; - - if (msg.shouldRetry()) - return msg.createRetry(); - - droppedMessageCount.incrementAndGet(); - return null; - } - - static boolean isLocalDC(InetAddressAndPort localHost, InetAddressAndPort remoteHost) - { - String remoteDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(remoteHost); - String localDC = DatabaseDescriptor.getEndpointSnitch().getDatacenter(localHost); - return remoteDC != null && remoteDC.equals(localDC); - } - - /** - * Handles the callback of the TCP connection attempt (not including the handshake negotiation!), and really all - * we're handling here is the TCP connection failures. On failure, we close the channel (which should disconnect - * the socket, if connected). If there was an {@link IOException} while trying to connect, the connection will be - * retried after a short delay. - *

- * This method does not alter the {@link #state} as it's only evaluating the TCP connect, not TCP connect and handshake. - * Thus, {@link #finishHandshake(HandshakeResult)} will handle any necessary state updates. - *

- * Note: this method is called from the event loop, so be careful wrt thread visibility - * - * @return true iff the TCP connection was established and the {@link #state} is not {@link State#CLOSED}; else false. - */ - @VisibleForTesting - boolean connectCallback(Future future) - { - ChannelFuture channelFuture = (ChannelFuture)future; - - // make sure this instance is not (terminally) closed - if (state.get() == State.CLOSED) - { - channelFuture.channel().close(); - return false; - } - - // this is the success state - final Throwable cause = future.cause(); - if (cause == null) - { - connectAttemptCount = 0; - return true; - } - - setStateIfNotClosed(state, State.NOT_READY); - if (cause instanceof IOException) - { - logger.trace("unable to connect on attempt {} to {}", connectAttemptCount, connectionId, cause); - connectAttemptCount++; - connectionRetryFuture = scheduledExecutor.schedule(this::connect, OPEN_RETRY_DELAY_MS * connectAttemptCount, TimeUnit.MILLISECONDS); - } - else - { - JVMStabilityInspector.inspectThrowable(cause); - logger.error("non-IO error attempting to connect to {}", connectionId, cause); - } - return false; - } - - /** - * A callback for handling timeouts when creating a connection/negotiating the handshake. - *

- * Note: this method is *not* invoked from the netty event loop, - * so there's an inherent race with {@link #finishHandshake(HandshakeResult)}, - * as well as any possible connect() reattempts (a seemingly remote race condition, however). - * Therefore, this function tries to lose any races, as much as possible. - * - * @return true if there was a timeout on the connect/handshake; else false. - */ - boolean connectionTimeout(ChannelFuture channelFuture) - { - if (connectionRetryFuture != null) - { - connectionRetryFuture.cancel(false); - connectionRetryFuture = null; - } - connectAttemptCount = 0; - State initialState = state.get(); - if (initialState == State.CLOSED) - return true; - - if (initialState != State.READY) - { - logger.debug("timed out while trying to connect to {}", connectionId); - - channelFuture.channel().close(); - // a last-ditch attempt to let finishHandshake() win the race - if (state.compareAndSet(initialState, State.NOT_READY)) - { - backlog.clear(); - return true; - } - } - return false; - } - - /** - * Process the results of the handshake negotiation. - *

- * Note: this method will be invoked from the netty event loop, - * so there's an inherent race with {@link #connectionTimeout(ChannelFuture)}. - */ - void finishHandshake(HandshakeResult result) - { - // clean up the connector instances before changing the state - if (connectionTimeoutFuture != null) - { - connectionTimeoutFuture.cancel(false); - connectionTimeoutFuture = null; - } - if (connectionRetryFuture != null) - { - connectionRetryFuture.cancel(false); - connectionRetryFuture = null; - } - connectAttemptCount = 0; - - if (result.negotiatedMessagingVersion != HandshakeResult.UNKNOWN_PROTOCOL_VERSION) - { - targetVersion = result.negotiatedMessagingVersion; - MessagingService.instance().setVersion(connectionId.remote(), targetVersion); - } - - switch (result.outcome) - { - case SUCCESS: - assert result.channelWriter != null; - logger.debug("successfully connected to {}, compress = {}, coalescing = {}", connectionId, - shouldCompressConnection(connectionId.local(), connectionId.remote()), - coalescingStrategy.isPresent() ? coalescingStrategy.get() : CoalescingStrategies.Strategy.DISABLED); - if (state.get() == State.CLOSED) - { - result.channelWriter.close(); - backlog.clear(); - break; - } - channelWriter = result.channelWriter; - // drain the backlog to the channel - channelWriter.writeBacklog(backlog, true); - // change the state so newly incoming messages can be sent to the channel (without adding to the backlog) - setStateIfNotClosed(state, State.READY); - // ship out any stragglers that got added to the backlog - channelWriter.writeBacklog(backlog, true); - break; - case DISCONNECT: - reconnect(); - break; - case NEGOTIATION_FAILURE: - setStateIfNotClosed(state, State.NOT_READY); - backlog.clear(); - break; - default: - throw new IllegalArgumentException("unhandled result type: " + result.outcome); - } - } - - @VisibleForTesting - static boolean setStateIfNotClosed(AtomicReference state, State newState) - { - State s = state.get(); - if (s == State.CLOSED) - return false; - state.set(newState); - return true; - } - - int getTargetVersion() - { - return targetVersion; - } - - /** - * Handles the result of each message sent. - * - * Note: this function is expected to be invoked on the netty event loop. Also, do not retain any state from - * the input {@code messageResult}. - */ - void handleMessageResult(MessageResult messageResult) - { - completedMessageCount.incrementAndGet(); - - // checking the cause() is an optimized way to tell if the operation was successful (as the cause will be null) - // Note that ExpiredException is just a marker for timeout-ed message we're dropping, but as we already - // incremented the dropped message count in MessageOutHandler, we have nothing to do. - Throwable cause = messageResult.future.cause(); - if (cause == null) - return; - - if (cause instanceof ExpiredException) - { - droppedMessageCount.incrementAndGet(); - return; - } - - JVMStabilityInspector.inspectThrowable(cause); - - if (cause instanceof IOException || cause.getCause() instanceof IOException) - { - ChannelWriter writer = messageResult.writer; - if (writer.shouldPurgeBacklog()) - purgeBacklog(); - - // This writer needs to be closed and we need to trigger a reconnection. We really only want to do that - // once for this channel however (and again, no race because we're on the netty event loop). - if (!writer.isClosed() && messageResult.allowReconnect) - { - reconnect(); - writer.close(); - } - - QueuedMessage msg = messageResult.msg; - if (msg != null && msg.shouldRetry()) - { - sendMessage(msg.createRetry()); - } - } - else if (messageResult.future.isCancelled()) - { - // Someone cancelled the future, which we assume meant it doesn't want the message to be sent if it hasn't - // yet. Just ignore. - } - else - { - // Non IO exceptions are likely a programming error so let's not silence them - logger.error("Unexpected error writing on " + connectionId, cause); - } - } - - /** - * Change the IP address on which we connect to the peer. We will attempt to connect to the new address if there - * was a previous connection, and new incoming messages as well as existing {@link #backlog} messages will be sent there. - * Any outstanding messages in the existing channel will still be sent to the previous address (we won't/can't move them from - * one channel to another). - */ - void reconnectWithNewIp(InetAddressAndPort newAddr) - { - State currentState = state.get(); - - // if we're closed, ignore the request - if (currentState == State.CLOSED) - return; - - // capture a reference to the current channel, in case it gets swapped out before we can call close() on it - ChannelWriter currentChannel = channelWriter; - connectionId = connectionId.withNewConnectionAddress(newAddr); - - if (currentState != State.NOT_READY) - reconnect(); - - // lastly, push through anything remaining in the existing channel. - if (currentChannel != null) - currentChannel.softClose(); - } - - /** - * Sets the state properly so {@link #connect()} can attempt to reconnect. - */ - void reconnect() - { - if (setStateIfNotClosed(state, State.NOT_READY)) - connect(); - } - - void purgeBacklog() - { - backlog.clear(); - } - - public void close(boolean softClose) - { - state.set(State.CLOSED); - - if (connectionTimeoutFuture != null) - { - connectionTimeoutFuture.cancel(false); - connectionTimeoutFuture = null; - } - - // drain the backlog - if (channelWriter != null) - { - if (softClose) - { - channelWriter.writeBacklog(backlog, false); - channelWriter.softClose(); - } - else - { - backlog.clear(); - channelWriter.close(); - } - - channelWriter = null; - } - } - - @Override - public String toString() - { - return connectionId.toString(); - } - - public Integer getPendingMessages() - { - int pending = backlog.size(); - ChannelWriter chan = channelWriter; - if (chan != null) - pending += (int)chan.pendingMessageCount(); - return pending; - } - - public Long getCompletedMessages() - { - return completedMessageCount.get(); - } - - public Long getDroppedMessages() - { - return droppedMessageCount.get(); - } - - /* - methods specific to testing follow - */ - - @VisibleForTesting - int backlogSize() - { - return backlog.size(); - } - - @VisibleForTesting - void addToBacklog(QueuedMessage msg) - { - backlog.add(msg); - } - - @VisibleForTesting - void setChannelWriter(ChannelWriter channelWriter) - { - this.channelWriter = channelWriter; - } - - @VisibleForTesting - ChannelWriter getChannelWriter() - { - return channelWriter; - } - - @VisibleForTesting - void setState(State state) - { - this.state.set(state); - } - - @VisibleForTesting - State getState() - { - return state.get(); - } - - @VisibleForTesting - void setTargetVersion(int targetVersion) - { - this.targetVersion = targetVersion; - } - - @VisibleForTesting - OutboundConnectionIdentifier getConnectionId() - { - return connectionId; - } - - @VisibleForTesting - void setConnectionTimeoutFuture(ScheduledFuture connectionTimeoutFuture) - { - this.connectionTimeoutFuture = connectionTimeoutFuture; - } - - @VisibleForTesting - ScheduledFuture getConnectionTimeoutFuture() - { - return connectionTimeoutFuture; - } - - public boolean isConnected() - { - return state.get() == State.READY; - } -} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java b/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java deleted file mode 100644 index 14650a74d3ff..000000000000 --- a/src/java/org/apache/cassandra/net/async/OutboundMessagingPool.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.util.Optional; - -import com.google.common.annotations.VisibleForTesting; - -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.concurrent.Stage; -import org.apache.cassandra.config.Config; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.metrics.ConnectionMetrics; -import org.apache.cassandra.net.BackPressureState; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType; -import org.apache.cassandra.utils.CoalescingStrategies; -import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy; - -/** - * Groups a set of outbound connections to a given peer, and routes outgoing messages to the appropriate connection - * (based upon message's type or size). Contains a {@link OutboundMessagingConnection} for each of the - * {@link ConnectionType} type. - */ -public class OutboundMessagingPool -{ - @VisibleForTesting - static final long LARGE_MESSAGE_THRESHOLD = Long.getLong(Config.PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64); - - private final ConnectionMetrics metrics; - private final BackPressureState backPressureState; - - public OutboundMessagingConnection gossipChannel; - public OutboundMessagingConnection largeMessageChannel; - public OutboundMessagingConnection smallMessageChannel; - - /** - * An override address on which to communicate with the peer. Typically used for something like EC2 public IP addresses - * which need to be used for communication between EC2 regions. - */ - private InetAddressAndPort preferredRemoteAddr; - - public OutboundMessagingPool(InetAddressAndPort remoteAddr, InetAddressAndPort localAddr, ServerEncryptionOptions encryptionOptions, - BackPressureState backPressureState, IInternodeAuthenticator authenticator) - { - preferredRemoteAddr = remoteAddr; - this.backPressureState = backPressureState; - metrics = new ConnectionMetrics(localAddr, this); - - - smallMessageChannel = new OutboundMessagingConnection(OutboundConnectionIdentifier.small(localAddr, preferredRemoteAddr), - encryptionOptions, coalescingStrategy(remoteAddr), authenticator); - largeMessageChannel = new OutboundMessagingConnection(OutboundConnectionIdentifier.large(localAddr, preferredRemoteAddr), - encryptionOptions, coalescingStrategy(remoteAddr), authenticator); - - // don't attempt coalesce the gossip messages, just ship them out asap (let's not anger the FD on any peer node by any artificial delays) - gossipChannel = new OutboundMessagingConnection(OutboundConnectionIdentifier.gossip(localAddr, preferredRemoteAddr), - encryptionOptions, Optional.empty(), authenticator); - } - - private static Optional coalescingStrategy(InetAddressAndPort remoteAddr) - { - String strategyName = DatabaseDescriptor.getOtcCoalescingStrategy(); - String displayName = remoteAddr.toString(); - return CoalescingStrategies.newCoalescingStrategy(strategyName, - DatabaseDescriptor.getOtcCoalescingWindow(), - OutboundMessagingConnection.logger, - displayName); - - } - - public BackPressureState getBackPressureState() - { - return backPressureState; - } - - public void sendMessage(MessageOut msg, int id) - { - getConnection(msg).sendMessage(msg, id); - } - - @VisibleForTesting - public OutboundMessagingConnection getConnection(MessageOut msg) - { - if (msg.connectionType == null) - { - // optimize for the common path (the small message channel) - if (Stage.GOSSIP != msg.getStage()) - { - return msg.serializedSize(smallMessageChannel.getTargetVersion()) < LARGE_MESSAGE_THRESHOLD - ? smallMessageChannel - : largeMessageChannel; - } - return gossipChannel; - } - else - { - return getConnection(msg.connectionType); - } - } - - /** - * Reconnect to the peer using the given {@code addr}. Outstanding messages in each channel will be sent on the - * current channel. Typically this function is used for something like EC2 public IP addresses which need to be used - * for communication between EC2 regions. - * - * @param addr IP Address to use (and prefer) going forward for connecting to the peer - */ - public void reconnectWithNewIp(InetAddressAndPort addr) - { - preferredRemoteAddr = addr; - gossipChannel.reconnectWithNewIp(addr); - largeMessageChannel.reconnectWithNewIp(addr); - smallMessageChannel.reconnectWithNewIp(addr); - } - - /** - * Close each netty channel and it's socket. - * - * @param softClose {@code true} if existing messages in the queue should be sent before closing. - */ - public void close(boolean softClose) - { - gossipChannel.close(softClose); - largeMessageChannel.close(softClose); - smallMessageChannel.close(softClose); - } - - @VisibleForTesting - final OutboundMessagingConnection getConnection(ConnectionType connectionType) - { - switch (connectionType) - { - case SMALL_MESSAGE: - return smallMessageChannel; - case LARGE_MESSAGE: - return largeMessageChannel; - case GOSSIP: - return gossipChannel; - default: - throw new IllegalArgumentException("unsupported connection type: " + connectionType); - } - } - - public void incrementTimeout() - { - metrics.timeouts.mark(); - } - - public long getTimeouts() - { - return metrics.timeouts.getCount(); - } - - public InetAddressAndPort getPreferredRemoteAddr() - { - return preferredRemoteAddr; - } -} diff --git a/src/java/org/apache/cassandra/net/async/QueuedMessage.java b/src/java/org/apache/cassandra/net/async/QueuedMessage.java deleted file mode 100644 index 28e4ba47f5aa..000000000000 --- a/src/java/org/apache/cassandra/net/async/QueuedMessage.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.util.concurrent.TimeUnit; - -import com.google.common.annotations.VisibleForTesting; - -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.utils.CoalescingStrategies; - -/** - * A wrapper for outbound messages. All messages will be retried once. - */ -public class QueuedMessage implements CoalescingStrategies.Coalescable -{ - public final MessageOut message; - public final int id; - public final long timestampNanos; - public final boolean droppable; - private final boolean retryable; - - public QueuedMessage(MessageOut message, int id) - { - this(message, id, System.nanoTime(), MessagingService.DROPPABLE_VERBS.contains(message.verb), true); - } - - @VisibleForTesting - public QueuedMessage(MessageOut message, int id, long timestampNanos, boolean droppable, boolean retryable) - { - this.message = message; - this.id = id; - this.timestampNanos = timestampNanos; - this.droppable = droppable; - this.retryable = retryable; - } - - /** don't drop a non-droppable message just because it's timestamp is expired */ - public boolean isTimedOut() - { - return droppable && timestampNanos < System.nanoTime() - TimeUnit.MILLISECONDS.toNanos(message.getTimeout()); - } - - public boolean shouldRetry() - { - return retryable; - } - - public QueuedMessage createRetry() - { - return new QueuedMessage(message, id, System.nanoTime(), droppable, false); - } - - public long timestampNanos() - { - return timestampNanos; - } -} diff --git a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java b/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java deleted file mode 100644 index 4e667da83788..000000000000 --- a/src/java/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlus.java +++ /dev/null @@ -1,291 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.EOFException; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicInteger; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.Unpooled; -import io.netty.channel.ChannelConfig; -import io.netty.util.ReferenceCountUtil; -import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; -import org.apache.cassandra.io.util.RebufferingInputStream; - -public class RebufferingByteBufDataInputPlus extends RebufferingInputStream implements ReadableByteChannel -{ - /** - * The parent, or owning, buffer of the current buffer being read from ({@link super#buffer}). - */ - private ByteBuf currentBuf; - - private final BlockingQueue queue; - - /** - * The count of live bytes in all {@link ByteBuf}s held by this instance. - */ - private final AtomicInteger queuedByteCount; - - private final int lowWaterMark; - private final int highWaterMark; - private final ChannelConfig channelConfig; - - private volatile boolean closed; - - public RebufferingByteBufDataInputPlus(int lowWaterMark, int highWaterMark, ChannelConfig channelConfig) - { - super(Unpooled.EMPTY_BUFFER.nioBuffer()); - - if (lowWaterMark > highWaterMark) - throw new IllegalArgumentException(String.format("low water mark is greater than high water mark: %d vs %d", lowWaterMark, highWaterMark)); - - currentBuf = Unpooled.EMPTY_BUFFER; - this.lowWaterMark = lowWaterMark; - this.highWaterMark = highWaterMark; - this.channelConfig = channelConfig; - queue = new LinkedBlockingQueue<>(); - queuedByteCount = new AtomicInteger(); - } - - /** - * Append a {@link ByteBuf} to the end of the einternal queue. - * - * Note: it's expected this method is invoked on the netty event loop. - */ - public void append(ByteBuf buf) throws IllegalStateException - { - assert buf != null : "buffer cannot be null"; - - if (closed) - { - ReferenceCountUtil.release(buf); - throw new IllegalStateException("stream is already closed, so cannot add another buffer"); - } - - // this slightly undercounts the live count as it doesn't include the currentBuf's size. - // that's ok as the worst we'll do is allow another buffer in and add it to the queue, - // and that point we'll disable auto-read. this is a tradeoff versus making some other member field - // atomic or volatile. - int queuedCount = queuedByteCount.addAndGet(buf.readableBytes()); - if (channelConfig.isAutoRead() && queuedCount > highWaterMark) - channelConfig.setAutoRead(false); - - queue.add(buf); - } - - /** - * {@inheritDoc} - * - * Release open buffers and poll the {@link #queue} for more data. - *

- * This is best, and more or less expected, to be invoked on a consuming thread (not the event loop) - * becasue if we block on the queue we can't fill it on the event loop (as that's where the buffers are coming from). - */ - @Override - protected void reBuffer() throws IOException - { - currentBuf.release(); - buffer = null; - currentBuf = null; - - // possibly re-enable auto-read, *before* blocking on the queue, because if we block on the queue - // without enabling auto-read we'll block forever :( - if (!channelConfig.isAutoRead() && queuedByteCount.get() < lowWaterMark) - channelConfig.setAutoRead(true); - - try - { - currentBuf = queue.take(); - int bytes; - // if we get an explicitly empty buffer, we treat that as an indicator that the input is closed - if (currentBuf == null || (bytes = currentBuf.readableBytes()) == 0) - { - releaseResources(); - throw new EOFException(); - } - - buffer = currentBuf.nioBuffer(currentBuf.readerIndex(), bytes); - assert buffer.remaining() == bytes; - queuedByteCount.addAndGet(-bytes); - return; - } - catch (InterruptedException ie) - { - // nop - ignore - } - } - - @Override - public int read(ByteBuffer dst) throws IOException - { - int readLength = dst.remaining(); - int remaining = readLength; - - while (remaining > 0) - { - if (closed) - throw new EOFException(); - - if (!buffer.hasRemaining()) - reBuffer(); - int copyLength = Math.min(remaining, buffer.remaining()); - - int originalLimit = buffer.limit(); - buffer.limit(buffer.position() + copyLength); - dst.put(buffer); - buffer.limit(originalLimit); - remaining -= copyLength; - } - - return readLength; - } - - /** - * {@inheritDoc} - * - * As long as this method is invoked on the consuming thread the returned value will be accurate. - * - * @throws EOFException thrown when no bytes are buffered and {@link #closed} is true. - */ - @Override - public int available() throws EOFException - { - final int availableBytes = queuedByteCount.get() + (buffer != null ? buffer.remaining() : 0); - - if (availableBytes == 0 && closed) - throw new EOFException(); - - if (!channelConfig.isAutoRead() && availableBytes < lowWaterMark) - channelConfig.setAutoRead(true); - - return availableBytes; - } - - @Override - public boolean isOpen() - { - return !closed; - } - - /** - * {@inheritDoc} - * - * Note: This should invoked on the consuming thread. - */ - @Override - public void close() - { - closed = true; - releaseResources(); - } - - private void releaseResources() - { - if (currentBuf != null) - { - if (currentBuf.refCnt() > 0) - currentBuf.release(currentBuf.refCnt()); - currentBuf = null; - buffer = null; - } - - ByteBuf buf; - while ((buf = queue.poll()) != null && buf.refCnt() > 0) - buf.release(buf.refCnt()); - } - - /** - * Mark this stream as closed, but do not release any of the resources. - * - * Note: this is best to be called from the producer thread. - */ - public void markClose() - { - if (!closed) - { - closed = true; - queue.add(Unpooled.EMPTY_BUFFER); - } - } - - /** - * {@inheritDoc} - * - * Note: this is best to be called from the consumer thread. - */ - @Override - public String toString() - { - return new StringBuilder(128).append("RebufferingByteBufDataInputPlus: currentBuf = ").append(currentBuf) - .append(" (super.buffer = ").append(buffer).append(')') - .append(", queuedByteCount = ").append(queuedByteCount) - .append(", queue buffers = ").append(queue) - .append(", closed = ").append(closed) - .toString(); - } - - public ByteBufAllocator getAllocator() - { - return channelConfig.getAllocator(); - } - - /** - * Consumes bytes in the stream until the given length - * - * @param writer - * @param len - * @return - * @throws IOException - */ - public long consumeUntil(BufferedDataOutputStreamPlus writer, long len) throws IOException - { - long copied = 0; // number of bytes copied - while (copied < len) - { - if (buffer.remaining() == 0) - { - try - { - reBuffer(); - } - catch (EOFException e) - { - throw new EOFException("EOF after " + copied + " bytes out of " + len); - } - if (buffer.remaining() == 0 && copied < len) - throw new AssertionError("reBuffer() failed to return data"); - } - - int originalLimit = buffer.limit(); - int toCopy = (int) Math.min(len - copied, buffer.remaining()); - buffer.limit(buffer.position() + toCopy); - int written = writer.applyToChannel(c -> c.write(buffer)); - buffer.limit(originalLimit); - copied += written; - } - - return copied; - } -} diff --git a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java index d2a6aebd5777..68a58242b940 100644 --- a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java +++ b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java @@ -24,14 +24,15 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.RepairException; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.AsymmetricSyncRequest; -import org.apache.cassandra.repair.messages.SyncRequest; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.SessionSummary; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.MerkleTrees; + +import static org.apache.cassandra.net.Verb.REPAIR_REQ; /** * AsymmetricRemoteSyncTask sends {@link AsymmetricSyncRequest} to target node to repair(stream) @@ -52,7 +53,7 @@ public void startSync() AsymmetricSyncRequest request = new AsymmetricSyncRequest(desc, local, nodePair.coordinator, nodePair.peer, rangesToSync, previewKind); String message = String.format("Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", request.ranges.size(), request.fetchingNode, request.fetchFrom); Tracing.traceRepair(message); - MessagingService.instance().sendOneWay(request.createMessage(), request.fetchingNode); + MessagingService.instance().send(Message.out(REPAIR_REQ, request), request.fetchingNode); } public void syncComplete(boolean success, List summaries) diff --git a/src/java/org/apache/cassandra/repair/RepairJobDesc.java b/src/java/org/apache/cassandra/repair/RepairJobDesc.java index 7e7de0738fbe..4aaf655b8258 100644 --- a/src/java/org/apache/cassandra/repair/RepairJobDesc.java +++ b/src/java/org/apache/cassandra/repair/RepairJobDesc.java @@ -26,6 +26,7 @@ import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.IVersionedSerializer; @@ -106,7 +107,7 @@ public void serialize(RepairJobDesc desc, DataOutputPlus out, int version) throw UUIDSerializer.serializer.serialize(desc.sessionId, out, version); out.writeUTF(desc.keyspace); out.writeUTF(desc.columnFamily); - MessagingService.validatePartitioner(desc.ranges); + IPartitioner.validate(desc.ranges); out.writeInt(desc.ranges.size()); for (Range rt : desc.ranges) AbstractBounds.tokenSerializer.serialize(rt, out, version); @@ -128,7 +129,7 @@ public RepairJobDesc deserialize(DataInputPlus in, int version) throws IOExcepti for (int i = 0; i < nRanges; i++) { range = (Range) AbstractBounds.tokenSerializer.deserialize(in, - MessagingService.globalPartitioner(), version); + IPartitioner.global(), version); ranges.add(range); } diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java index 1e92a81165ab..2a87fa2ec612 100644 --- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java +++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java @@ -23,17 +23,17 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; import org.apache.cassandra.repair.messages.*; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.streaming.PreviewKind; +import static org.apache.cassandra.net.Verb.REPAIR_REQ; + /** * Handles all repair related message. * @@ -41,6 +41,8 @@ */ public class RepairMessageVerbHandler implements IVerbHandler { + public static RepairMessageVerbHandler instance = new RepairMessageVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(RepairMessageVerbHandler.class); private boolean isIncremental(UUID sessionID) @@ -54,7 +56,7 @@ private PreviewKind previewKind(UUID sessionID) return prs != null ? prs.previewKind : PreviewKind.NONE; } - public void doVerb(final MessageIn message, final int id) + public void doVerb(final Message message) { // TODO add cancel/interrupt message RepairJobDesc desc = message.payload.desc; @@ -72,20 +74,20 @@ public void doVerb(final MessageIn message, final int id) if (columnFamilyStore == null) { logErrorAndSendFailureResponse(String.format("Table with id %s was dropped during prepare phase of repair", - tableId), message.from, id); + tableId), message); return; } columnFamilyStores.add(columnFamilyStore); } ActiveRepairService.instance.registerParentRepairSession(prepareMessage.parentRepairSession, - message.from, + message.from(), columnFamilyStores, prepareMessage.ranges, prepareMessage.isIncremental, prepareMessage.timestamp, prepareMessage.isGlobal, prepareMessage.previewKind); - MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), id, message.from); + MessagingService.instance().send(message.emptyResponse(), message.from()); break; case SNAPSHOT: @@ -94,7 +96,7 @@ public void doVerb(final MessageIn message, final int id) if (cfs == null) { logErrorAndSendFailureResponse(String.format("Table %s.%s was dropped during snapshot phase of repair", - desc.keyspace, desc.columnFamily), message.from, id); + desc.keyspace, desc.columnFamily), message); return; } @@ -108,8 +110,8 @@ public void doVerb(final MessageIn message, final int id) { repairManager.snapshot(desc.parentSessionId.toString(), desc.ranges, true); } - logger.debug("Enqueuing response to snapshot request {} to {}", desc.sessionId, message.from); - MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), id, message.from); + logger.debug("Enqueuing response to snapshot request {} to {}", desc.sessionId, message.from()); + MessagingService.instance().send(message.emptyResponse(), message.from()); break; case VALIDATION_REQUEST: @@ -120,12 +122,12 @@ public void doVerb(final MessageIn message, final int id) if (store == null) { logger.error("Table {}.{} was dropped during snapshot phase of repair", desc.keyspace, desc.columnFamily); - MessagingService.instance().sendOneWay(new ValidationComplete(desc).createMessage(), message.from); + MessagingService.instance().send(Message.out(REPAIR_REQ, new ValidationComplete(desc)), message.from()); return; } ActiveRepairService.instance.consistent.local.maybeSetRepairing(desc.parentSessionId); - Validator validator = new Validator(desc, message.from, validationRequest.nowInSec, + Validator validator = new Validator(desc, message.from(), validationRequest.nowInSec, isIncremental(desc.parentSessionId), previewKind(desc.parentSessionId)); ValidationManager.instance.submitValidation(store, validator); break; @@ -164,11 +166,11 @@ public void doVerb(final MessageIn message, final int id) logger.debug("cleaning up repair"); CleanupMessage cleanup = (CleanupMessage) message.payload; ActiveRepairService.instance.removeParentRepairSession(cleanup.parentRepairSession); - MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), id, message.from); + MessagingService.instance().send(message.emptyResponse(), message.from()); break; case CONSISTENT_REQUEST: - ActiveRepairService.instance.consistent.local.handlePrepareMessage(message.from, (PrepareConsistentRequest) message.payload); + ActiveRepairService.instance.consistent.local.handlePrepareMessage(message.from(), (PrepareConsistentRequest) message.payload); break; case CONSISTENT_RESPONSE: @@ -176,7 +178,7 @@ public void doVerb(final MessageIn message, final int id) break; case FINALIZE_PROPOSE: - ActiveRepairService.instance.consistent.local.handleFinalizeProposeMessage(message.from, (FinalizePropose) message.payload); + ActiveRepairService.instance.consistent.local.handleFinalizeProposeMessage(message.from(), (FinalizePropose) message.payload); break; case FINALIZE_PROMISE: @@ -184,25 +186,25 @@ public void doVerb(final MessageIn message, final int id) break; case FINALIZE_COMMIT: - ActiveRepairService.instance.consistent.local.handleFinalizeCommitMessage(message.from, (FinalizeCommit) message.payload); + ActiveRepairService.instance.consistent.local.handleFinalizeCommitMessage(message.from(), (FinalizeCommit) message.payload); break; case FAILED_SESSION: FailSession failure = (FailSession) message.payload; ActiveRepairService.instance.consistent.coordinated.handleFailSessionMessage(failure); - ActiveRepairService.instance.consistent.local.handleFailSessionMessage(message.from, failure); + ActiveRepairService.instance.consistent.local.handleFailSessionMessage(message.from(), failure); break; case STATUS_REQUEST: - ActiveRepairService.instance.consistent.local.handleStatusRequest(message.from, (StatusRequest) message.payload); + ActiveRepairService.instance.consistent.local.handleStatusRequest(message.from(), (StatusRequest) message.payload); break; case STATUS_RESPONSE: - ActiveRepairService.instance.consistent.local.handleStatusResponse(message.from, (StatusResponse) message.payload); + ActiveRepairService.instance.consistent.local.handleStatusResponse(message.from(), (StatusResponse) message.payload); break; default: - ActiveRepairService.instance.handleMessage(message.from, message.payload); + ActiveRepairService.instance.handleMessage(message.from(), message.payload); break; } } @@ -215,11 +217,10 @@ public void doVerb(final MessageIn message, final int id) } } - private void logErrorAndSendFailureResponse(String errorMessage, InetAddressAndPort to, int id) + private void logErrorAndSendFailureResponse(String errorMessage, Message respondTo) { logger.error(errorMessage); - MessageOut reply = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE) - .withParameter(ParameterType.FAILURE_RESPONSE, MessagingService.ONE_BYTE); - MessagingService.instance().sendReply(reply, id, to); + Message reply = respondTo.failureResponse(RequestFailureReason.UNKNOWN); + MessagingService.instance().send(reply, respondTo.from()); } } diff --git a/src/java/org/apache/cassandra/repair/SnapshotTask.java b/src/java/org/apache/cassandra/repair/SnapshotTask.java index acc5186a5fea..fab4b28e5cbb 100644 --- a/src/java/org/apache/cassandra/repair/SnapshotTask.java +++ b/src/java/org/apache/cassandra/repair/SnapshotTask.java @@ -18,17 +18,18 @@ package org.apache.cassandra.repair; import java.util.concurrent.RunnableFuture; -import java.util.concurrent.TimeUnit; import com.google.common.util.concurrent.AbstractFuture; import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IAsyncCallbackWithFailure; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.SnapshotMessage; +import static org.apache.cassandra.net.Verb.REPAIR_REQ; + /** * SnapshotTask is a task that sends snapshot request. */ @@ -37,7 +38,7 @@ public class SnapshotTask extends AbstractFuture implements private final RepairJobDesc desc; private final InetAddressAndPort endpoint; - public SnapshotTask(RepairJobDesc desc, InetAddressAndPort endpoint) + SnapshotTask(RepairJobDesc desc, InetAddressAndPort endpoint) { this.desc = desc; this.endpoint = endpoint; @@ -45,15 +46,15 @@ public SnapshotTask(RepairJobDesc desc, InetAddressAndPort endpoint) public void run() { - MessagingService.instance().sendRR(new SnapshotMessage(desc).createMessage(), - endpoint, - new SnapshotCallback(this), TimeUnit.HOURS.toMillis(1), true); + MessagingService.instance().sendWithCallback(Message.out(REPAIR_REQ, new SnapshotMessage(desc)), + endpoint, + new SnapshotCallback(this)); } /** * Callback for snapshot request. Run on INTERNAL_RESPONSE stage. */ - static class SnapshotCallback implements IAsyncCallbackWithFailure + static class SnapshotCallback implements RequestCallback { final SnapshotTask task; @@ -67,13 +68,19 @@ static class SnapshotCallback implements IAsyncCallbackWithFailure * * @param msg response received. */ - public void response(MessageIn msg) + @Override + public void onResponse(Message msg) { task.set(task.endpoint); } - public boolean isLatencyForSnitch() { return false; } + @Override + public boolean invokeOnFailure() + { + return true; + } + @Override public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) { //listener.failedSnapshot(); diff --git a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java index e9cba8932327..53407c9ba1fc 100644 --- a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java +++ b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java @@ -29,6 +29,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.SyncComplete; import org.apache.cassandra.streaming.PreviewKind; @@ -38,6 +39,8 @@ import org.apache.cassandra.streaming.StreamState; import org.apache.cassandra.streaming.StreamOperation; +import static org.apache.cassandra.net.Verb.REPAIR_REQ; + /** * StreamingRepairTask performs data streaming between two remote replicas, neither of which is repair coordinator. * Task will send {@link SyncComplete} message back to coordinator upon streaming completion. @@ -95,19 +98,19 @@ public void handleStreamEvent(StreamEvent event) } /** - * If we succeeded on both stream in and out, reply back to coordinator + * If we succeeded on both stream in and out, respond back to coordinator */ public void onSuccess(StreamState state) { logger.info("[repair #{}] streaming task succeed, returning response to {}", desc.sessionId, initiator); - MessagingService.instance().sendOneWay(new SyncComplete(desc, src, dst, true, state.createSummaries()).createMessage(), initiator); + MessagingService.instance().send(Message.out(REPAIR_REQ, new SyncComplete(desc, src, dst, true, state.createSummaries())), initiator); } /** - * If we failed on either stream in or out, reply fail to coordinator + * If we failed on either stream in or out, respond fail to coordinator */ public void onFailure(Throwable t) { - MessagingService.instance().sendOneWay(new SyncComplete(desc, src, dst, false, Collections.emptyList()).createMessage(), initiator); + MessagingService.instance().send(Message.out(REPAIR_REQ, new SyncComplete(desc, src, dst, false, Collections.emptyList())), initiator); } } diff --git a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java index c731bc185ec5..b608d679091e 100644 --- a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java +++ b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java @@ -19,7 +19,6 @@ import java.util.List; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,6 +27,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.RepairException; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.RepairMessage; import org.apache.cassandra.repair.messages.SyncRequest; @@ -35,7 +35,8 @@ import org.apache.cassandra.streaming.SessionSummary; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.MerkleTrees; + +import static org.apache.cassandra.net.Verb.REPAIR_REQ; /** * SymmetricRemoteSyncTask sends {@link SyncRequest} to remote(non-coordinator) node @@ -54,7 +55,7 @@ public SymmetricRemoteSyncTask(RepairJobDesc desc, InetAddressAndPort r1, InetAd void sendRequest(RepairMessage request, InetAddressAndPort to) { - MessagingService.instance().sendOneWay(request.createMessage(), to); + MessagingService.instance().send(Message.out(REPAIR_REQ, request), to); } @Override diff --git a/src/java/org/apache/cassandra/repair/SyncNodePair.java b/src/java/org/apache/cassandra/repair/SyncNodePair.java index b353eb39a729..e10ad5a374f9 100644 --- a/src/java/org/apache/cassandra/repair/SyncNodePair.java +++ b/src/java/org/apache/cassandra/repair/SyncNodePair.java @@ -25,7 +25,8 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; /** * SyncNodePair is used for repair message body to indicate the pair of nodes. @@ -71,21 +72,21 @@ public static class NodePairSerializer implements IVersionedSerializer= version 2.0 - MessagingService.instance().sendOneWay(new ValidationComplete(desc).createMessage(), initiator); + MessagingService.instance().send(Message.out(REPAIR_REQ, new ValidationComplete(desc)), initiator); } /** @@ -417,6 +419,6 @@ public void run() logger.info("{} Sending completed merkle tree to {} for {}.{}", previewKind.logPrefix(desc.sessionId), initiator, desc.keyspace, desc.columnFamily); Tracing.traceRepair("Sending completed merkle tree to {} for {}.{}", initiator, desc.keyspace, desc.columnFamily); } - MessagingService.instance().sendOneWay(new ValidationComplete(desc, trees).createMessage(), initiator); + MessagingService.instance().send(Message.out(REPAIR_REQ, new ValidationComplete(desc, trees)), initiator); } } diff --git a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java index b921342ec78b..28f5d0809281 100644 --- a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java +++ b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java @@ -40,8 +40,9 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.RepairSessionResult; import org.apache.cassandra.repair.messages.FailSession; import org.apache.cassandra.repair.messages.FinalizeCommit; @@ -141,8 +142,8 @@ synchronized boolean hasFailed() protected void sendMessage(InetAddressAndPort destination, RepairMessage message) { logger.trace("Sending {} to {}", message, destination); - MessageOut messageOut = new MessageOut(MessagingService.Verb.REPAIR_MESSAGE, message, RepairMessage.serializer); - MessagingService.instance().sendOneWay(messageOut, destination); + Message messageOut = Message.out(Verb.REPAIR_REQ, message); + MessagingService.instance().send(messageOut, destination); } public ListenableFuture prepare() diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java index c39c4e6f7626..e93ccb08449c 100644 --- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java +++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java @@ -72,7 +72,7 @@ import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.FailSession; import org.apache.cassandra.repair.messages.FinalizeCommit; @@ -88,6 +88,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.net.Verb.REPAIR_REQ; import static org.apache.cassandra.repair.consistent.ConsistentSession.State.*; /** @@ -449,7 +450,7 @@ synchronized void putSessionUnsafe(LocalSession session) private synchronized void putSession(LocalSession session) { Preconditions.checkArgument(!sessions.containsKey(session.sessionID), - "LocalSession {} already exists", session.sessionID); + "LocalSession %s already exists", session.sessionID); Preconditions.checkArgument(started, "sessions cannot be added before LocalSessions is started"); sessions = ImmutableMap.builder() .putAll(sessions) @@ -493,8 +494,8 @@ protected ActiveRepairService.ParentRepairSession getParentRepairSession(UUID se protected void sendMessage(InetAddressAndPort destination, RepairMessage message) { logger.trace("sending {} to {}", message, destination); - MessageOut messageOut = new MessageOut(MessagingService.Verb.REPAIR_MESSAGE, message, RepairMessage.serializer); - MessagingService.instance().sendOneWay(messageOut, destination); + Message messageOut = Message.out(REPAIR_REQ, message); + MessagingService.instance().send(messageOut, destination); } private void setStateAndSave(LocalSession session, ConsistentSession.State state) diff --git a/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java b/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java index 6d7626972c30..0a6d257bbf0b 100644 --- a/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.repair.messages; import java.io.IOException; -import java.net.InetAddress; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -26,16 +25,17 @@ import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; -import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairJobDesc; import org.apache.cassandra.streaming.PreviewKind; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + public class AsymmetricSyncRequest extends RepairMessage { public static MessageSerializer serializer = new SyncRequestSerializer(); @@ -81,13 +81,13 @@ public static class SyncRequestSerializer implements MessageSerializer range : message.ranges) { - MessagingService.validatePartitioner(range); + IPartitioner.validate(range); AbstractBounds.tokenSerializer.serialize(range, out, version); } out.writeInt(message.previewKind.getSerializationVal()); @@ -96,13 +96,13 @@ public void serialize(AsymmetricSyncRequest message, DataOutputPlus out, int ver public AsymmetricSyncRequest deserialize(DataInputPlus in, int version) throws IOException { RepairJobDesc desc = RepairJobDesc.serializer.deserialize(in, version); - InetAddressAndPort owner = CompactEndpointSerializationHelper.instance.deserialize(in, version); - InetAddressAndPort src = CompactEndpointSerializationHelper.instance.deserialize(in, version); - InetAddressAndPort dst = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort owner = inetAddressAndPortSerializer.deserialize(in, version); + InetAddressAndPort src = inetAddressAndPortSerializer.deserialize(in, version); + InetAddressAndPort dst = inetAddressAndPortSerializer.deserialize(in, version); int rangesCount = in.readInt(); List> ranges = new ArrayList<>(rangesCount); for (int i = 0; i < rangesCount; ++i) - ranges.add((Range) AbstractBounds.tokenSerializer.deserialize(in, MessagingService.globalPartitioner(), version)); + ranges.add((Range) AbstractBounds.tokenSerializer.deserialize(in, IPartitioner.global(), version)); PreviewKind previewKind = PreviewKind.deserialize(in.readInt()); return new AsymmetricSyncRequest(desc, owner, src, dst, ranges, previewKind); } @@ -110,9 +110,9 @@ public AsymmetricSyncRequest deserialize(DataInputPlus in, int version) throws I public long serializedSize(AsymmetricSyncRequest message, int version) { long size = RepairJobDesc.serializer.serializedSize(message.desc, version); - size += CompactEndpointSerializationHelper.instance.serializedSize(message.initiator, version); - size += CompactEndpointSerializationHelper.instance.serializedSize(message.fetchingNode, version); - size += CompactEndpointSerializationHelper.instance.serializedSize(message.fetchFrom, version); + size += inetAddressAndPortSerializer.serializedSize(message.initiator, version); + size += inetAddressAndPortSerializer.serializedSize(message.fetchingNode, version); + size += inetAddressAndPortSerializer.serializedSize(message.fetchFrom, version); size += TypeSizes.sizeof(message.ranges.size()); for (Range range : message.ranges) size += AbstractBounds.tokenSerializer.serializedSize(range, version); diff --git a/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java b/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java index 449748ab45a4..07e7e0d2f956 100644 --- a/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java +++ b/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java @@ -25,9 +25,10 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; import org.apache.cassandra.utils.UUIDSerializer; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + public class FinalizePromise extends RepairMessage { public final UUID sessionID; @@ -69,21 +70,21 @@ public int hashCode() public void serialize(FinalizePromise msg, DataOutputPlus out, int version) throws IOException { UUIDSerializer.serializer.serialize(msg.sessionID, out, version); - CompactEndpointSerializationHelper.instance.serialize(msg.participant, out, version); + inetAddressAndPortSerializer.serialize(msg.participant, out, version); out.writeBoolean(msg.promised); } public FinalizePromise deserialize(DataInputPlus in, int version) throws IOException { return new FinalizePromise(UUIDSerializer.serializer.deserialize(in, version), - CompactEndpointSerializationHelper.instance.deserialize(in, version), + inetAddressAndPortSerializer.deserialize(in, version), in.readBoolean()); } public long serializedSize(FinalizePromise msg, int version) { long size = UUIDSerializer.serializer.serializedSize(msg.sessionID, version); - size += CompactEndpointSerializationHelper.instance.serializedSize(msg.participant, version); + size += inetAddressAndPortSerializer.serializedSize(msg.participant, version); size += TypeSizes.sizeof(msg.promised); return size; } diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java index 9aae256e5a80..b1e9b047d3c7 100644 --- a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java @@ -29,9 +29,10 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; import org.apache.cassandra.utils.UUIDSerializer; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + public class PrepareConsistentRequest extends RepairMessage { public final UUID parentSession; @@ -84,23 +85,23 @@ public String toString() public void serialize(PrepareConsistentRequest request, DataOutputPlus out, int version) throws IOException { UUIDSerializer.serializer.serialize(request.parentSession, out, version); - CompactEndpointSerializationHelper.instance.serialize(request.coordinator, out, version); + inetAddressAndPortSerializer.serialize(request.coordinator, out, version); out.writeInt(request.participants.size()); for (InetAddressAndPort peer : request.participants) { - CompactEndpointSerializationHelper.instance.serialize(peer, out, version); + inetAddressAndPortSerializer.serialize(peer, out, version); } } public PrepareConsistentRequest deserialize(DataInputPlus in, int version) throws IOException { UUID sessionId = UUIDSerializer.serializer.deserialize(in, version); - InetAddressAndPort coordinator = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort coordinator = inetAddressAndPortSerializer.deserialize(in, version); int numPeers = in.readInt(); Set peers = new HashSet<>(numPeers); for (int i = 0; i < numPeers; i++) { - InetAddressAndPort peer = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort peer = inetAddressAndPortSerializer.deserialize(in, version); peers.add(peer); } return new PrepareConsistentRequest(sessionId, coordinator, peers); @@ -109,11 +110,11 @@ public PrepareConsistentRequest deserialize(DataInputPlus in, int version) throw public long serializedSize(PrepareConsistentRequest request, int version) { long size = UUIDSerializer.serializer.serializedSize(request.parentSession, version); - size += CompactEndpointSerializationHelper.instance.serializedSize(request.coordinator, version); + size += inetAddressAndPortSerializer.serializedSize(request.coordinator, version); size += TypeSizes.sizeof(request.participants.size()); for (InetAddressAndPort peer : request.participants) { - size += CompactEndpointSerializationHelper.instance.serializedSize(peer, version); + size += inetAddressAndPortSerializer.serializedSize(peer, version); } return size; } diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java index 630f18efda42..3362a40826f7 100644 --- a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java +++ b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java @@ -25,9 +25,10 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; import org.apache.cassandra.utils.UUIDSerializer; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + public class PrepareConsistentResponse extends RepairMessage { public final UUID parentSession; @@ -69,21 +70,21 @@ public int hashCode() public void serialize(PrepareConsistentResponse response, DataOutputPlus out, int version) throws IOException { UUIDSerializer.serializer.serialize(response.parentSession, out, version); - CompactEndpointSerializationHelper.instance.serialize(response.participant, out, version); + inetAddressAndPortSerializer.serialize(response.participant, out, version); out.writeBoolean(response.success); } public PrepareConsistentResponse deserialize(DataInputPlus in, int version) throws IOException { return new PrepareConsistentResponse(UUIDSerializer.serializer.deserialize(in, version), - CompactEndpointSerializationHelper.instance.deserialize(in, version), + inetAddressAndPortSerializer.deserialize(in, version), in.readBoolean()); } public long serializedSize(PrepareConsistentResponse response, int version) { long size = UUIDSerializer.serializer.serializedSize(response.parentSession, version); - size += CompactEndpointSerializationHelper.instance.serializedSize(response.participant, version); + size += inetAddressAndPortSerializer.serializedSize(response.participant, version); size += TypeSizes.sizeof(response.success); return size; } diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java index 4d59942b8d78..5a0701c6d390 100644 --- a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java +++ b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java @@ -25,6 +25,7 @@ import java.util.UUID; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.util.DataInputPlus; @@ -92,7 +93,7 @@ public void serialize(PrepareMessage message, DataOutputPlus out, int version) t out.writeInt(message.ranges.size()); for (Range r : message.ranges) { - MessagingService.validatePartitioner(r); + IPartitioner.validate(r); Range.tokenSerializer.serialize(r, out, version); } out.writeBoolean(message.isIncremental); @@ -111,7 +112,7 @@ public PrepareMessage deserialize(DataInputPlus in, int version) throws IOExcept int rangeCount = in.readInt(); List> ranges = new ArrayList<>(rangeCount); for (int i = 0; i < rangeCount; i++) - ranges.add((Range) Range.tokenSerializer.deserialize(in, MessagingService.globalPartitioner(), version)); + ranges.add((Range) Range.tokenSerializer.deserialize(in, IPartitioner.global(), version)); boolean isIncremental = in.readBoolean(); long timestamp = in.readLong(); boolean isGlobal = in.readBoolean(); diff --git a/src/java/org/apache/cassandra/repair/messages/RepairMessage.java b/src/java/org/apache/cassandra/repair/messages/RepairMessage.java index 09c60604552f..db1a134bb1d2 100644 --- a/src/java/org/apache/cassandra/repair/messages/RepairMessage.java +++ b/src/java/org/apache/cassandra/repair/messages/RepairMessage.java @@ -24,7 +24,6 @@ import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessageOut; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairJobDesc; @@ -66,7 +65,7 @@ public enum Type private final byte type; private final MessageSerializer serializer; - private Type(int type, MessageSerializer serializer) + Type(int type, MessageSerializer serializer) { this.type = (byte) type; this.serializer = serializer; @@ -92,12 +91,6 @@ protected RepairMessage(Type messageType, RepairJobDesc desc) this.desc = desc; } - public MessageOut createMessage() - { - return new MessageOut<>(MessagingService.Verb.REPAIR_MESSAGE, this, RepairMessage.serializer); - } - - public static class RepairMessageSerializer implements MessageSerializer { public void serialize(RepairMessage message, DataOutputPlus out, int version) throws IOException diff --git a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java index a0bf4e2f90de..71fcdb0832c7 100644 --- a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java @@ -25,16 +25,17 @@ import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; -import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairJobDesc; import org.apache.cassandra.streaming.PreviewKind; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + /** * Body part of SYNC_REQUEST repair message. * Request {@code src} node to sync data with {@code dst} node for range {@code ranges}. @@ -87,13 +88,13 @@ public static class SyncRequestSerializer implements MessageSerializer range : message.ranges) { - MessagingService.validatePartitioner(range); + IPartitioner.validate(range); AbstractBounds.tokenSerializer.serialize(range, out, version); } out.writeInt(message.previewKind.getSerializationVal()); @@ -102,13 +103,13 @@ public void serialize(SyncRequest message, DataOutputPlus out, int version) thro public SyncRequest deserialize(DataInputPlus in, int version) throws IOException { RepairJobDesc desc = RepairJobDesc.serializer.deserialize(in, version); - InetAddressAndPort owner = CompactEndpointSerializationHelper.instance.deserialize(in, version); - InetAddressAndPort src = CompactEndpointSerializationHelper.instance.deserialize(in, version); - InetAddressAndPort dst = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort owner = inetAddressAndPortSerializer.deserialize(in, version); + InetAddressAndPort src = inetAddressAndPortSerializer.deserialize(in, version); + InetAddressAndPort dst = inetAddressAndPortSerializer.deserialize(in, version); int rangesCount = in.readInt(); List> ranges = new ArrayList<>(rangesCount); for (int i = 0; i < rangesCount; ++i) - ranges.add((Range) AbstractBounds.tokenSerializer.deserialize(in, MessagingService.globalPartitioner(), version)); + ranges.add((Range) AbstractBounds.tokenSerializer.deserialize(in, IPartitioner.global(), version)); PreviewKind previewKind = PreviewKind.deserialize(in.readInt()); return new SyncRequest(desc, owner, src, dst, ranges, previewKind); } @@ -116,7 +117,7 @@ public SyncRequest deserialize(DataInputPlus in, int version) throws IOException public long serializedSize(SyncRequest message, int version) { long size = RepairJobDesc.serializer.serializedSize(message.desc, version); - size += 3 * CompactEndpointSerializationHelper.instance.serializedSize(message.initiator, version); + size += 3 * inetAddressAndPortSerializer.serializedSize(message.initiator, version); size += TypeSizes.sizeof(message.ranges.size()); for (Range range : message.ranges) size += AbstractBounds.tokenSerializer.serializedSize(range, version); diff --git a/src/java/org/apache/cassandra/schema/CompressionParams.java b/src/java/org/apache/cassandra/schema/CompressionParams.java index 40a4be351b8b..102edd817e50 100644 --- a/src/java/org/apache/cassandra/schema/CompressionParams.java +++ b/src/java/org/apache/cassandra/schema/CompressionParams.java @@ -41,6 +41,7 @@ import org.apache.cassandra.io.compress.*; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.streaming.messages.StreamMessage; import static java.lang.String.format; @@ -596,7 +597,7 @@ public void serialize(CompressionParams parameters, DataOutputPlus out, int vers out.writeUTF(entry.getValue()); } out.writeInt(parameters.chunkLength()); - if (version >= StreamMessage.VERSION_40) + if (version >= MessagingService.VERSION_40) out.writeInt(parameters.maxCompressedLength); else if (parameters.maxCompressedLength != Integer.MAX_VALUE) @@ -616,7 +617,7 @@ public CompressionParams deserialize(DataInputPlus in, int version) throws IOExc } int chunkLength = in.readInt(); int minCompressRatio = Integer.MAX_VALUE; // Earlier Cassandra cannot use uncompressed chunks. - if (version >= StreamMessage.VERSION_40) + if (version >= MessagingService.VERSION_40) minCompressRatio = in.readInt(); CompressionParams parameters; @@ -641,7 +642,7 @@ public long serializedSize(CompressionParams parameters, int version) size += TypeSizes.sizeof(entry.getValue()); } size += TypeSizes.sizeof(parameters.chunkLength()); - if (version >= StreamMessage.VERSION_40) + if (version >= MessagingService.VERSION_40) size += TypeSizes.sizeof(parameters.maxCompressedLength()); return size; } diff --git a/src/java/org/apache/cassandra/schema/MigrationManager.java b/src/java/org/apache/cassandra/schema/MigrationManager.java index 32a6cf1f7dc2..69a72bf631af 100644 --- a/src/java/org/apache/cassandra/schema/MigrationManager.java +++ b/src/java/org/apache/cassandra/schema/MigrationManager.java @@ -38,11 +38,13 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.Keyspaces.KeyspacesDiff; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.net.Verb.SCHEMA_PUSH_REQ; + public class MigrationManager { private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class); @@ -152,8 +154,8 @@ static boolean shouldPullSchemaFrom(InetAddressAndPort endpoint) * Don't request schema from nodes with a differnt or unknonw major version (may have incompatible schema) * Don't request schema from fat clients */ - return MessagingService.instance().knowsVersion(endpoint) - && MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version + return MessagingService.instance().versions.knows(endpoint) + && MessagingService.instance().versions.getRaw(endpoint) == MessagingService.current_version && !Gossiper.instance.isGossipOnlyMember(endpoint); } @@ -161,8 +163,8 @@ private static boolean shouldPushSchemaTo(InetAddressAndPort endpoint) { // only push schema to nodes with known and equal versions return !endpoint.equals(FBUtilities.getBroadcastAddressAndPort()) - && MessagingService.instance().knowsVersion(endpoint) - && MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version; + && MessagingService.instance().versions.knows(endpoint) + && MessagingService.instance().versions.getRaw(endpoint) == MessagingService.current_version; } public static boolean isReadyForBootstrap() @@ -315,14 +317,6 @@ private static void announce(Mutation.SimpleBuilder schema, boolean announceLoca announce(mutations); } - private static void pushSchemaMutation(InetAddressAndPort endpoint, Collection schema) - { - MessageOut> msg = new MessageOut<>(MessagingService.Verb.DEFINITIONS_UPDATE, - schema, - MigrationsSerializer.instance); - MessagingService.instance().sendOneWay(msg, endpoint); - } - // Returns a future on the local application of the schema private static void announce(Collection schema) { @@ -330,11 +324,12 @@ private static void announce(Collection schema) Set schemaDestinationEndpoints = new HashSet<>(); Set schemaEndpointsIgnored = new HashSet<>(); + Message> message = Message.out(SCHEMA_PUSH_REQ, schema); for (InetAddressAndPort endpoint : Gossiper.instance.getLiveMembers()) { if (shouldPushSchemaTo(endpoint)) { - pushSchemaMutation(endpoint, schema); + MessagingService.instance().send(message, endpoint); schemaDestinationEndpoints.add(endpoint); } else @@ -363,11 +358,12 @@ public static KeyspacesDiff announce(SchemaTransformation transformation, boolea Set schemaDestinationEndpoints = new HashSet<>(); Set schemaEndpointsIgnored = new HashSet<>(); + Message> message = Message.out(SCHEMA_PUSH_REQ, result.mutations); for (InetAddressAndPort endpoint : Gossiper.instance.getLiveMembers()) { if (shouldPushSchemaTo(endpoint)) { - pushSchemaMutation(endpoint, result.mutations); + MessagingService.instance().send(message, endpoint); schemaDestinationEndpoints.add(endpoint); } else diff --git a/src/java/org/apache/cassandra/schema/MigrationTask.java b/src/java/org/apache/cassandra/schema/MigrationTask.java index bf96fb27b959..3308893ffcc6 100644 --- a/src/java/org/apache/cassandra/schema/MigrationTask.java +++ b/src/java/org/apache/cassandra/schema/MigrationTask.java @@ -32,12 +32,14 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IAsyncCallback; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.utils.WrappedRunnable; +import static org.apache.cassandra.net.NoPayload.noPayload; +import static org.apache.cassandra.net.Verb.SCHEMA_PULL_REQ; + final class MigrationTask extends WrappedRunnable { private static final Logger logger = LoggerFactory.getLogger(MigrationTask.class); @@ -78,32 +80,23 @@ public void runMayThrow() throws Exception return; } - MessageOut message = new MessageOut<>(MessagingService.Verb.MIGRATION_REQUEST, null, MigrationManager.MigrationsSerializer.instance); + Message message = Message.out(SCHEMA_PULL_REQ, noPayload); final CountDownLatch completionLatch = new CountDownLatch(1); - IAsyncCallback> cb = new IAsyncCallback>() + RequestCallback> cb = msg -> { - @Override - public void response(MessageIn> message) + try { - try - { - Schema.instance.mergeAndAnnounceVersion(message.payload); - } - catch (ConfigurationException e) - { - logger.error("Configuration exception merging remote schema", e); - } - finally - { - completionLatch.countDown(); - } + Schema.instance.mergeAndAnnounceVersion(msg.payload); } - - public boolean isLatencyForSnitch() + catch (ConfigurationException e) + { + logger.error("Configuration exception merging remote schema", e); + } + finally { - return false; + completionLatch.countDown(); } }; @@ -111,7 +104,7 @@ public boolean isLatencyForSnitch() if (monitoringBootstrapStates.contains(SystemKeyspace.getBootstrapState())) inflightTasks.offer(completionLatch); - MessagingService.instance().sendRR(message, endpoint, cb); + MessagingService.instance().sendWithCallback(message, endpoint, cb); SchemaMigrationDiagnostics.taskRequestSend(endpoint); } diff --git a/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java b/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java index 2c1723589808..45844b3403b9 100644 --- a/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java +++ b/src/java/org/apache/cassandra/schema/SchemaMigrationEvent.java @@ -85,8 +85,8 @@ enum MigrationManagerEventType if (endpoint == null) return; - if (MessagingService.instance().knowsVersion(endpoint)) - endpointMessagingVersion = MessagingService.instance().getRawVersion(endpoint); + if (MessagingService.instance().versions.knows(endpoint)) + endpointMessagingVersion = MessagingService.instance().versions.getRaw(endpoint); endpointGossipOnlyMember = Gossiper.instance.isGossipOnlyMember(endpoint); this.isAlive = FailureDetector.instance.isAlive(endpoint); diff --git a/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java b/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java index 45cf365d8505..ed30792fd490 100644 --- a/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java +++ b/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java @@ -24,27 +24,24 @@ import org.apache.cassandra.db.Mutation; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.NoPayload; /** - * Sends it's current schema state in form of mutations in reply to the remote node's request. + * Sends it's current schema state in form of mutations in response to the remote node's request. * Such a request is made when one of the nodes, by means of Gossip, detects schema disagreement in the ring. */ -public final class SchemaPullVerbHandler implements IVerbHandler +public final class SchemaPullVerbHandler implements IVerbHandler { + public static final SchemaPullVerbHandler instance = new SchemaPullVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(SchemaPullVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - logger.trace("Received schema pull request from {}", message.from); - - MessageOut> response = - new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE, - SchemaKeyspace.convertSchemaToMutations(), - MigrationManager.MigrationsSerializer.instance); - - MessagingService.instance().sendReply(response, id, message.from); + logger.trace("Received schema pull request from {}", message.from()); + Message> response = message.responseWith(SchemaKeyspace.convertSchemaToMutations()); + MessagingService.instance().send(response, message.from()); } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java b/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java index 358739a1b57e..8d1bb0ff0760 100644 --- a/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java +++ b/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java @@ -26,7 +26,7 @@ import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; /** * Called when node receives updated schema state from the schema migration coordinator node. @@ -36,13 +36,15 @@ */ public final class SchemaPushVerbHandler implements IVerbHandler> { + public static final SchemaPushVerbHandler instance = new SchemaPushVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(SchemaPushVerbHandler.class); - public void doVerb(final MessageIn> message, int id) + public void doVerb(final Message> message) { - logger.trace("Received schema push request from {}", message.from); + logger.trace("Received schema push request from {}", message.from()); - SchemaAnnouncementDiagnostics.schemataMutationsReceived(message.from); + SchemaAnnouncementDiagnostics.schemataMutationsReceived(message.from()); StageManager.getStage(Stage.MIGRATION).submit(() -> Schema.instance.mergeAndAnnounceVersion(message.payload)); } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java b/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java index 0a506e3cdda6..80090de5576a 100644 --- a/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java +++ b/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java @@ -23,24 +23,20 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.utils.UUIDSerializer; +import org.apache.cassandra.net.NoPayload; -public final class SchemaVersionVerbHandler implements IVerbHandler +public final class SchemaVersionVerbHandler implements IVerbHandler { + public static final SchemaVersionVerbHandler instance = new SchemaVersionVerbHandler(); + private final Logger logger = LoggerFactory.getLogger(SchemaVersionVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - logger.trace("Received schema version request from {}", message.from); - - MessageOut response = - new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE, - Schema.instance.getVersion(), - UUIDSerializer.serializer); - - MessagingService.instance().sendReply(response, id, message.from); + logger.trace("Received schema version request from {}", message.from()); + Message response = message.responseWith(Schema.instance.getVersion()); + MessagingService.instance().send(response, message.from()); } } diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java index 75198759f1f3..667496388c1b 100644 --- a/src/java/org/apache/cassandra/security/SSLFactory.java +++ b/src/java/org/apache/cassandra/security/SSLFactory.java @@ -290,8 +290,8 @@ static SslContext createNettySslContext(EncryptionOptions options, boolean build // only set the cipher suites if the opertor has explicity configured values for it; else, use the default // for each ssl implemention (jdk or openssl) - if (options.cipher_suites != null && options.cipher_suites.length > 0) - builder.ciphers(Arrays.asList(options.cipher_suites), SupportedCipherSuiteFilter.INSTANCE); + if (options.cipher_suites != null && !options.cipher_suites.isEmpty()) + builder.ciphers(options.cipher_suites, SupportedCipherSuiteFilter.INSTANCE); if (buildTruststore) builder.trustManager(buildTrustManagerFactory(options)); diff --git a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java index d988cc0f131c..3efdef987c13 100644 --- a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java +++ b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java @@ -166,7 +166,7 @@ protected ByteBuffer copyAsNewCollection(ByteBuffer input, int count, int startP ByteBuffer output = ByteBuffer.allocate(sizeLen + bodyLen); writeCollectionSize(output, count, version); output.position(0); - ByteBufferUtil.arrayCopy(input, startPos, output, sizeLen, bodyLen); + ByteBufferUtil.copyBytes(input, startPos, output, sizeLen, bodyLen); return output; } } diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java index 1470cadc84bf..1889c79c28a4 100644 --- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.stream.Collectors; @@ -40,13 +39,15 @@ import org.apache.cassandra.exceptions.WriteFailureException; import org.apache.cassandra.exceptions.WriteTimeoutException; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IAsyncCallbackWithFailure; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.utils.concurrent.SimpleCondition; +import static java.util.concurrent.TimeUnit.NANOSECONDS; -public abstract class AbstractWriteResponseHandler implements IAsyncCallbackWithFailure + +public abstract class AbstractWriteResponseHandler implements RequestCallback { protected static final Logger logger = LoggerFactory.getLogger(AbstractWriteResponseHandler.class); @@ -90,12 +91,12 @@ protected AbstractWriteResponseHandler(ReplicaPlan.ForTokenWrite replicaPlan, public void get() throws WriteTimeoutException, WriteFailureException { - long timeout = currentTimeout(); + long timeoutNanos = currentTimeoutNanos(); boolean success; try { - success = condition.await(timeout, TimeUnit.NANOSECONDS); + success = condition.await(timeoutNanos, NANOSECONDS); } catch (InterruptedException ex) { @@ -120,12 +121,12 @@ public void get() throws WriteTimeoutException, WriteFailureException } } - public final long currentTimeout() + public final long currentTimeoutNanos() { long requestTimeout = writeType == WriteType.COUNTER - ? DatabaseDescriptor.getCounterWriteRpcTimeout() - : DatabaseDescriptor.getWriteRpcTimeout(); - return TimeUnit.MILLISECONDS.toNanos(requestTimeout) - (System.nanoTime() - queryStartNanoTime); + ? DatabaseDescriptor.getCounterWriteRpcTimeout(NANOSECONDS) + : DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS); + return requestTimeout - (System.nanoTime() - queryStartNanoTime); } /** @@ -143,7 +144,7 @@ public void setIdealCLResponseHandler(AbstractWriteResponseHandler handler) * on whether the CL was achieved. Only call this after the subclass has completed all it's processing * since the subclass instance may be queried to find out if the CL was achieved. */ - protected final void logResponseToIdealCLDelegate(MessageIn m) + protected final void logResponseToIdealCLDelegate(Message m) { //Tracking ideal CL was not configured if (idealCLDelegate == null) @@ -162,7 +163,7 @@ protected final void logResponseToIdealCLDelegate(MessageIn m) //Let the delegate do full processing, this will loop back into the branch above //with idealCLDelegate == this, because the ideal write handler idealCLDelegate will always //be set to this in the delegate. - idealCLDelegate.response(m); + idealCLDelegate.onResponse(m); } } @@ -187,7 +188,7 @@ public final void expired() } /** - * @return the minimum number of endpoints that must reply. + * @return the minimum number of endpoints that must respond. */ protected int blockFor() { @@ -227,7 +228,7 @@ protected boolean waitingFor(InetAddressAndPort from) /** * null message means "response from local write" */ - public abstract void response(MessageIn msg); + public abstract void onResponse(Message msg); protected void signal() { @@ -251,6 +252,12 @@ public void onFailure(InetAddressAndPort from, RequestFailureReason failureReaso signal(); } + @Override + public boolean invokeOnFailure() + { + return true; + } + @Override public boolean supportsBackPressure() { @@ -301,12 +308,12 @@ public void maybeTryAdditionalReplicas(IMutation mutation, StorageProxy.WritePer timeout = Math.min(timeout, cf.additionalWriteLatencyNanos); // no latency information, or we're overloaded - if (timeout > TimeUnit.MILLISECONDS.toNanos(mutation.getTimeout())) + if (timeout > mutation.getTimeout(NANOSECONDS)) return; try { - if (!condition.await(timeout, TimeUnit.NANOSECONDS)) + if (!condition.await(timeout, NANOSECONDS)) { for (ColumnFamilyStore cf : cfs) cf.metric.additionalWrites.inc(); diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index 525bebaf9627..abfd6d9c7927 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -58,9 +58,8 @@ import org.apache.cassandra.gms.VersionedValue; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.TokenMetadata; -import org.apache.cassandra.net.IAsyncCallbackWithFailure; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.CommonRange; import org.apache.cassandra.streaming.PreviewKind; @@ -72,7 +71,7 @@ import org.apache.cassandra.repair.messages.*; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.utils.CassandraVersion; -import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.Pair; @@ -80,6 +79,7 @@ import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.transform; +import static org.apache.cassandra.net.Verb.REPAIR_REQ; /** * ActiveRepairService is the starting point for manual "active" repairs. @@ -381,7 +381,7 @@ static long getRepairedAt(RepairOption options, boolean force) // end up skipping replicas if (options.isIncremental() && options.isGlobal() && ! force) { - return Clock.instance.currentTimeMillis(); + return System.currentTimeMillis(); } else { @@ -396,24 +396,27 @@ public UUID prepareForRepair(UUID parentRepairSession, InetAddressAndPort coordi final CountDownLatch prepareLatch = new CountDownLatch(endpoints.size()); final AtomicBoolean status = new AtomicBoolean(true); final Set failedNodes = Collections.synchronizedSet(new HashSet()); - IAsyncCallbackWithFailure callback = new IAsyncCallbackWithFailure() + RequestCallback callback = new RequestCallback() { - public void response(MessageIn msg) + @Override + public void onResponse(Message msg) { prepareLatch.countDown(); } - public boolean isLatencyForSnitch() - { - return false; - } - + @Override public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) { status.set(false); failedNodes.add(from.toString()); prepareLatch.countDown(); } + + @Override + public boolean invokeOnFailure() + { + return true; + } }; List tableIds = new ArrayList<>(columnFamilyStores.size()); @@ -425,8 +428,8 @@ public void onFailure(InetAddressAndPort from, RequestFailureReason failureReaso if (FailureDetector.instance.isAlive(neighbour)) { PrepareMessage message = new PrepareMessage(parentRepairSession, tableIds, options.getRanges(), options.isIncremental(), repairedAt, options.isGlobal(), options.getPreviewKind()); - MessageOut msg = message.createMessage(); - MessagingService.instance().sendRR(msg, neighbour, callback, DatabaseDescriptor.getRpcTimeout(), true); + Message msg = Message.out(REPAIR_REQ, message); + MessagingService.instance().sendWithCallback(msg, neighbour, callback); } else { diff --git a/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java b/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java index 63fbc729f345..b28f468af7e8 100644 --- a/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java +++ b/src/java/org/apache/cassandra/service/BatchlogResponseHandler.java @@ -24,7 +24,7 @@ import org.apache.cassandra.exceptions.WriteFailureException; import org.apache.cassandra.exceptions.WriteTimeoutException; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; public class BatchlogResponseHandler extends AbstractWriteResponseHandler { @@ -47,21 +47,21 @@ protected int ackCount() return wrapped.ackCount(); } - public void response(MessageIn msg) + public void onResponse(Message msg) { - wrapped.response(msg); + wrapped.onResponse(msg); if (requiredBeforeFinishUpdater.decrementAndGet(this) == 0) cleanup.ackMutation(); } - public boolean isLatencyForSnitch() + public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) { - return wrapped.isLatencyForSnitch(); + wrapped.onFailure(from, failureReason); } - public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) + public boolean invokeOnFailure() { - wrapped.onFailure(from, failureReason); + return wrapped.invokeOnFailure(); } public void get() throws WriteTimeoutException, WriteFailureException diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index b8f06f6ad209..9d05371c89f3 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -71,6 +71,8 @@ import org.apache.cassandra.utils.*; import org.apache.cassandra.security.ThreadAwareSecurityManager; +import static java.util.concurrent.TimeUnit.NANOSECONDS; + /** * The CassandraDaemon is an abstraction for a Cassandra daemon * service, which defines not only a way to activate and deactivate it, but also @@ -431,9 +433,9 @@ public void uncaughtException(Thread t, Throwable e) // schedule periodic recomputation of speculative retry thresholds ScheduledExecutors.optionalTasks.scheduleWithFixedDelay( () -> Keyspace.all().forEach(k -> k.getColumnFamilyStores().forEach(ColumnFamilyStore::updateSpeculationThreshold)), - DatabaseDescriptor.getReadRpcTimeout(), - DatabaseDescriptor.getReadRpcTimeout(), - TimeUnit.MILLISECONDS + DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS), + DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS), + NANOSECONDS ); // Native transport diff --git a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java index 4c892ffad993..1f536c7db3b8 100644 --- a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java @@ -26,7 +26,7 @@ import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaPlan; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.WriteType; @@ -65,13 +65,13 @@ public DatacenterSyncWriteResponseHandler(ReplicaPlan.ForTokenWrite replicaPlan, } } - public void response(MessageIn message) + public void onResponse(Message message) { try { String dataCenter = message == null ? DatabaseDescriptor.getLocalDataCenter() - : snitch.getDatacenter(message.from); + : snitch.getDatacenter(message.from()); responses.get(dataCenter).getAndDecrement(); acks.incrementAndGet(); @@ -96,10 +96,4 @@ protected int ackCount() { return acks.get(); } - - public boolean isLatencyForSnitch() - { - return false; - } - } diff --git a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java index f30b4525fffc..a9583a3c3512 100644 --- a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java @@ -21,7 +21,7 @@ import org.apache.cassandra.locator.InOurDcTester; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.ReplicaPlan; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import java.util.function.Predicate; @@ -42,11 +42,11 @@ public DatacenterWriteResponseHandler(ReplicaPlan.ForTokenWrite replicaPlan, } @Override - public void response(MessageIn message) + public void onResponse(Message message) { - if (message == null || waitingFor(message.from)) + if (message == null || waitingFor(message.from())) { - super.response(message); + super.onResponse(message); } else { diff --git a/src/java/org/apache/cassandra/service/EchoVerbHandler.java b/src/java/org/apache/cassandra/service/EchoVerbHandler.java index 1cc52e9cc6b5..76f23d46d816 100644 --- a/src/java/org/apache/cassandra/service/EchoVerbHandler.java +++ b/src/java/org/apache/cassandra/service/EchoVerbHandler.java @@ -19,28 +19,23 @@ * under the License. * */ - - -import org.apache.cassandra.gms.EchoMessage; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.NoPayload; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType; - -public class EchoVerbHandler implements IVerbHandler +public class EchoVerbHandler implements IVerbHandler { + public static final EchoVerbHandler instance = new EchoVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(EchoVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - MessageOut echoMessage = new MessageOut(MessagingService.Verb.REQUEST_RESPONSE, EchoMessage.instance, - EchoMessage.serializer, ConnectionType.GOSSIP); - logger.trace("Sending a EchoMessage reply {}", message.from); - MessagingService.instance().sendReply(echoMessage, id, message.from); + logger.trace("Sending ECHO_RSP to {}", message.from()); + MessagingService.instance().send(message.emptyResponse(), message.from()); } } diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java index a3f6b5232372..6901968c6cc7 100644 --- a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java +++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java @@ -23,6 +23,8 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.utils.ExecutorUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,9 +123,9 @@ public static void calculatePendingRanges(AbstractReplicationStrategy strategy, } @VisibleForTesting - public void shutdownExecutor() throws InterruptedException + public void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException { executor.shutdown(); - executor.awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.awaitTermination(timeout, units, executor); } } diff --git a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java index a9975338b780..cf2872b4b51e 100644 --- a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java +++ b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java @@ -23,24 +23,24 @@ import org.apache.cassandra.db.SnapshotCommand; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; public class SnapshotVerbHandler implements IVerbHandler { + public static final SnapshotVerbHandler instance = new SnapshotVerbHandler(); + private static final Logger logger = LoggerFactory.getLogger(SnapshotVerbHandler.class); - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { SnapshotCommand command = message.payload; if (command.clear_snapshot) - { Keyspace.clearSnapshot(command.snapshot_name, command.keyspace); - } else Keyspace.open(command.keyspace).getColumnFamilyStore(command.column_family).snapshot(command.snapshot_name); - logger.debug("Enqueuing response to snapshot request {} to {}", command.snapshot_name, message.from); - MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), id, message.from); + + logger.debug("Enqueuing response to snapshot request {} to {}", command.snapshot_name, message.from()); + MessagingService.instance().send(message.emptyResponse(), message.from()); } } diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index ce7867429b7a..d2dd956607b2 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.service; -import java.io.File; import java.nio.ByteBuffer; import java.nio.file.Paths; import java.util.*; @@ -26,7 +25,6 @@ import java.util.concurrent.atomic.AtomicLong; import com.google.common.base.Preconditions; -import com.google.common.base.Strings; import com.google.common.cache.CacheLoader; import com.google.common.collect.*; import com.google.common.primitives.Ints; @@ -69,17 +67,25 @@ import org.apache.cassandra.service.paxos.Commit; import org.apache.cassandra.service.paxos.PaxosState; import org.apache.cassandra.service.paxos.PrepareCallback; -import org.apache.cassandra.service.paxos.PrepareResponse; -import org.apache.cassandra.service.paxos.PrepareVerbHandler; import org.apache.cassandra.service.paxos.ProposeCallback; -import org.apache.cassandra.service.paxos.ProposeVerbHandler; -import org.apache.cassandra.net.MessagingService.Verb; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.triggers.TriggerExecutor; import org.apache.cassandra.utils.*; import org.apache.cassandra.utils.AbstractIterator; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.net.NoPayload.noPayload; +import static org.apache.cassandra.net.Verb.BATCH_STORE_REQ; +import static org.apache.cassandra.net.Verb.MUTATION_REQ; +import static org.apache.cassandra.net.Verb.PAXOS_COMMIT_REQ; +import static org.apache.cassandra.net.Verb.PAXOS_PREPARE_REQ; +import static org.apache.cassandra.net.Verb.PAXOS_PROPOSE_REQ; +import static org.apache.cassandra.net.Verb.TRUNCATE_REQ; import static org.apache.cassandra.service.BatchlogResponseHandler.BatchlogCleanup; +import static org.apache.cassandra.service.paxos.PrepareVerbHandler.doPrepare; +import static org.apache.cassandra.service.paxos.ProposeVerbHandler.doPropose; public class StorageProxy implements StorageProxyMBean { @@ -173,7 +179,7 @@ private StorageProxy() * 1. Prepare: the coordinator generates a ballot (timeUUID in our case) and asks replicas to (a) promise * not to accept updates from older ballots and (b) tell us about the most recent update it has already * accepted. - * 2. Accept: if a majority of replicas reply, the coordinator asks replicas to accept the value of the + * 2. Accept: if a majority of replicas respond, the coordinator asks replicas to accept the value of the * highest proposal ballot it heard about, or a new value if no in-progress proposals were reported. * 3. Commit (Learn): if a majority of replicas acknowledge the accept request, we can commit the new * value. @@ -219,8 +225,8 @@ public static RowIterator cas(String keyspaceName, consistencyForPaxos.validateForCas(); consistencyForCommit.validateForCasCommit(keyspaceName); - long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout()); - while (System.nanoTime() - queryStartNanoTime < timeout) + long timeoutNanos = DatabaseDescriptor.getCasContentionTimeout(NANOSECONDS); + while (System.nanoTime() - queryStartNanoTime < timeoutNanos) { // for simplicity, we'll do a single liveness check at the start of each attempt ReplicaPlan.ForPaxosWrite replicaPlan = ReplicaPlans.forPaxos(Keyspace.open(keyspaceName), key, consistencyForPaxos); @@ -276,7 +282,7 @@ public static RowIterator cas(String keyspaceName, Tracing.trace("Paxos proposal not accepted (pre-empted by a higher ballot)"); contentions++; - Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), MILLISECONDS); // continue to retry } @@ -332,11 +338,11 @@ private static PaxosBallotAndContention beginAndRepairPaxos(long queryStartNanoT ClientState state) throws WriteTimeoutException, WriteFailureException { - long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout()); + long timeoutNanos = DatabaseDescriptor.getCasContentionTimeout(NANOSECONDS); PrepareCallback summary = null; int contentions = 0; - while (System.nanoTime() - queryStartNanoTime < timeout) + while (System.nanoTime() - queryStartNanoTime < timeoutNanos) { // We want a timestamp that is guaranteed to be unique for that node (so that the ballot is globally unique), but if we've got a prepare rejected // already we also want to make sure we pick a timestamp that has a chance to be promised, i.e. one that is greater that the most recently known @@ -357,7 +363,7 @@ private static PaxosBallotAndContention beginAndRepairPaxos(long queryStartNanoT Tracing.trace("Some replicas have already promised a higher ballot than ours; aborting"); contentions++; // sleep a random amount to give the other proposer a chance to finish - Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), MILLISECONDS); continue; } @@ -392,7 +398,7 @@ private static PaxosBallotAndContention beginAndRepairPaxos(long queryStartNanoT Tracing.trace("Some replicas have already promised a higher ballot than ours; aborting"); // sleep a random amount to give the other proposer a chance to finish contentions++; - Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), MILLISECONDS); } continue; } @@ -426,43 +432,34 @@ private static PaxosBallotAndContention beginAndRepairPaxos(long queryStartNanoT */ private static void sendCommit(Commit commit, Iterable replicas) { - MessageOut message = new MessageOut(MessagingService.Verb.PAXOS_COMMIT, commit, Commit.serializer); + Message message = Message.out(PAXOS_COMMIT_REQ, commit); for (InetAddressAndPort target : replicas) - MessagingService.instance().sendOneWay(message, target); + MessagingService.instance().send(message, target); } private static PrepareCallback preparePaxos(Commit toPrepare, ReplicaPlan.ForPaxosWrite replicaPlan, long queryStartNanoTime) throws WriteTimeoutException { PrepareCallback callback = new PrepareCallback(toPrepare.update.partitionKey(), toPrepare.update.metadata(), replicaPlan.requiredParticipants(), replicaPlan.consistencyLevel(), queryStartNanoTime); - MessageOut message = new MessageOut(MessagingService.Verb.PAXOS_PREPARE, toPrepare, Commit.serializer); + Message message = Message.out(PAXOS_PREPARE_REQ, toPrepare); for (Replica replica: replicaPlan.contacts()) { if (replica.isSelf()) { - StageManager.getStage(MessagingService.verbStages.get(MessagingService.Verb.PAXOS_PREPARE)).execute(new Runnable() - { - public void run() + StageManager.getStage(PAXOS_PREPARE_REQ.stage).execute(() -> { + try { - try - { - MessageIn message = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(), - PrepareVerbHandler.doPrepare(toPrepare), - Collections.emptyMap(), - MessagingService.Verb.INTERNAL_RESPONSE, - MessagingService.current_version); - callback.response(message); - } - catch (Exception ex) - { - logger.error("Failed paxos prepare locally", ex); - } + callback.onResponse(message.responseWith(doPrepare(toPrepare))); + } + catch (Exception ex) + { + logger.error("Failed paxos prepare locally", ex); } }); } else { - MessagingService.instance().sendRR(message, replica.endpoint(), callback); + MessagingService.instance().sendWithCallback(message, replica.endpoint(), callback); } } callback.await(); @@ -473,34 +470,26 @@ private static boolean proposePaxos(Commit proposal, ReplicaPlan.ForPaxosWrite r throws WriteTimeoutException { ProposeCallback callback = new ProposeCallback(replicaPlan.contacts().size(), replicaPlan.requiredParticipants(), !timeoutIfPartial, replicaPlan.consistencyLevel(), queryStartNanoTime); - MessageOut message = new MessageOut(MessagingService.Verb.PAXOS_PROPOSE, proposal, Commit.serializer); + Message message = Message.out(PAXOS_PROPOSE_REQ, proposal); for (Replica replica : replicaPlan.contacts()) { if (replica.isSelf()) { - StageManager.getStage(MessagingService.verbStages.get(MessagingService.Verb.PAXOS_PROPOSE)).execute(new Runnable() - { - public void run() + StageManager.getStage(PAXOS_PROPOSE_REQ.stage).execute(() -> { + try { - try - { - MessageIn message = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(), - ProposeVerbHandler.doPropose(proposal), - Collections.emptyMap(), - MessagingService.Verb.INTERNAL_RESPONSE, - MessagingService.current_version); - callback.response(message); - } - catch (Exception ex) - { - logger.error("Failed paxos propose locally", ex); - } + Message response = message.responseWith(doPropose(proposal)); + callback.onResponse(response); + } + catch (Exception ex) + { + logger.error("Failed paxos propose locally", ex); } }); } else { - MessagingService.instance().sendRR(message, replica.endpoint(), callback); + MessagingService.instance().sendWithCallback(message, replica.endpoint(), callback); } } callback.await(); @@ -531,7 +520,7 @@ private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLev responseHandler.setSupportsBackPressure(false); } - MessageOut message = new MessageOut<>(MessagingService.Verb.PAXOS_COMMIT, proposal, Commit.serializer); + Message message = Message.outWithFlag(PAXOS_COMMIT_REQ, proposal, MessageFlag.CALL_BACK_ON_FAILURE); for (Replica replica : replicaPlan.liveAndDown()) { InetAddressAndPort destination = replica.endpoint(); @@ -544,11 +533,11 @@ private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLev if (replica.isSelf()) commitPaxosLocal(replica, message, responseHandler); else - MessagingService.instance().sendWriteRR(message, replica, responseHandler, allowHints && shouldHint(replica)); + MessagingService.instance().sendWriteWithCallback(message, replica, responseHandler, allowHints && shouldHint(replica)); } else { - MessagingService.instance().sendOneWay(message, destination); + MessagingService.instance().send(message, destination); } } else @@ -573,9 +562,9 @@ private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLev * submit a fake one that executes immediately on the mutation stage, but generates the necessary backpressure * signal for hints */ - private static void commitPaxosLocal(Replica localReplica, final MessageOut message, final AbstractWriteResponseHandler responseHandler) + private static void commitPaxosLocal(Replica localReplica, final Message message, final AbstractWriteResponseHandler responseHandler) { - StageManager.getStage(MessagingService.verbStages.get(MessagingService.Verb.PAXOS_COMMIT)).maybeExecuteImmediately(new LocalMutationRunnable(localReplica) + StageManager.getStage(PAXOS_COMMIT_REQ.stage).maybeExecuteImmediately(new LocalMutationRunnable(localReplica) { public void runMayThrow() { @@ -583,20 +572,20 @@ public void runMayThrow() { PaxosState.commit(message.payload); if (responseHandler != null) - responseHandler.response(null); + responseHandler.onResponse(null); } catch (Exception ex) { if (!(ex instanceof WriteTimeoutException)) logger.error("Failed to apply paxos commit locally : ", ex); - responseHandler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN); + responseHandler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.forException(ex)); } } @Override protected Verb verb() { - return MessagingService.Verb.PAXOS_COMMIT; + return PAXOS_COMMIT_REQ; } }); } @@ -999,22 +988,22 @@ private static void syncWriteToBatchlog(Collection mutations, ReplicaP queryStartNanoTime); Batch batch = Batch.createLocal(uuid, FBUtilities.timestampMicros(), mutations); - MessageOut message = new MessageOut<>(MessagingService.Verb.BATCH_STORE, batch, Batch.serializer); + Message message = Message.out(BATCH_STORE_REQ, batch); for (Replica replica : replicaPlan.liveAndDown()) { logger.trace("Sending batchlog store request {} to {} for {} mutations", batch.id, replica, batch.size()); if (replica.isSelf()) - performLocally(Stage.MUTATION, replica, Optional.empty(), () -> BatchlogManager.store(batch), handler); + performLocally(Stage.MUTATION, replica, () -> BatchlogManager.store(batch), handler); else - MessagingService.instance().sendRR(message, replica.endpoint(), handler); + MessagingService.instance().sendWithCallback(message, replica.endpoint(), handler); } handler.get(); } private static void asyncRemoveFromBatchlog(ReplicaPlan.ForTokenWrite replicaPlan, UUID uuid) { - MessageOut message = new MessageOut<>(MessagingService.Verb.BATCH_REMOVE, uuid, UUIDSerializer.serializer); + Message message = Message.out(Verb.BATCH_REMOVE_REQ, uuid); for (Replica target : replicaPlan.contacts()) { if (logger.isTraceEnabled()) @@ -1023,7 +1012,7 @@ private static void asyncRemoveFromBatchlog(ReplicaPlan.ForTokenWrite replicaPla if (target.isSelf()) performLocally(Stage.MUTATION, target, () -> BatchlogManager.remove(uuid)); else - MessagingService.instance().sendOneWay(message, target.endpoint()); + MessagingService.instance().send(message, target.endpoint()); } } @@ -1040,7 +1029,7 @@ private static void asyncWriteBatchedMutations(List } catch (OverloadedException | WriteTimeoutException e) { - wrapper.handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN); + wrapper.handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.forException(e)); } } } @@ -1136,7 +1125,7 @@ private static WriteResponseHandlerWrapper wrapViewBatchResponseHandler(Mutation AbstractWriteResponseHandler writeHandler = rs.getWriteResponseHandler(replicaPlan, () -> { long delay = Math.max(0, System.currentTimeMillis() - baseComplete.get()); - viewWriteMetrics.viewWriteLatency.update(delay, TimeUnit.MILLISECONDS); + viewWriteMetrics.viewWriteLatency.update(delay, MILLISECONDS); }, writeType, queryStartNanoTime); BatchlogResponseHandler batchHandler = new ViewWriteMetricsWrapped(writeHandler, batchConsistencyLevel.blockFor(keyspace), cleanup, queryStartNanoTime); return new WriteResponseHandlerWrapper(batchHandler, mutation); @@ -1184,7 +1173,7 @@ public static void sendToHintedReplicas(final Mutation mutation, // extra-datacenter replicas, grouped by dc Map> dcGroups = null; // only need to create a Message for non-local writes - MessageOut message = null; + Message message = null; boolean insertLocal = false; Replica localReplica = null; @@ -1207,7 +1196,7 @@ public static void sendToHintedReplicas(final Mutation mutation, { // belongs on a different server if (message == null) - message = mutation.createMessage(); + message = Message.outWithFlag(MUTATION_REQ, mutation, MessageFlag.CALL_BACK_ON_FAILURE); String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(destination); @@ -1253,7 +1242,7 @@ public static void sendToHintedReplicas(final Mutation mutation, } if (backPressureHosts != null) - MessagingService.instance().applyBackPressure(backPressureHosts, responseHandler.currentTimeout()); + MessagingService.instance().applyBackPressure(backPressureHosts, responseHandler.currentTimeoutNanos()); if (endpointsToHint != null) submitHint(mutation, EndpointsForToken.copyOf(mutation.key().getToken(), endpointsToHint), responseHandler); @@ -1261,13 +1250,13 @@ public static void sendToHintedReplicas(final Mutation mutation, if (insertLocal) { Preconditions.checkNotNull(localReplica); - performLocally(stage, localReplica, Optional.of(mutation), mutation::apply, responseHandler); + performLocally(stage, localReplica, mutation::apply, responseHandler); } if (localDc != null) { for (Replica destination : localDc) - MessagingService.instance().sendWriteRR(message, destination, responseHandler, true); + MessagingService.instance().sendWriteWithCallback(message, destination, responseHandler, true); } if (dcGroups != null) { @@ -1293,33 +1282,34 @@ private static void checkHintOverload(Replica destination) } } - private static void sendMessagesToNonlocalDC(MessageOut message, + /* + * Send the message to the first replica of targets, and have it forward the message to others in its DC + * + * TODO: are targets shuffled? do we want them to be to spread out forwarding burden? + */ + private static void sendMessagesToNonlocalDC(Message message, EndpointsForToken targets, AbstractWriteResponseHandler handler) { - Iterator iter = targets.iterator(); - int[] messageIds = new int[targets.size()]; - Replica target = iter.next(); - - int idIdx = 0; - // Add the other destinations of the same message as a FORWARD_HEADER entry - while (iter.hasNext()) + if (targets.size() > 1) { - Replica destination = iter.next(); - int id = MessagingService.instance().addWriteCallback(handler, - message, - destination, - message.getTimeout(), - handler.replicaPlan.consistencyLevel(), - true); - messageIds[idIdx++] = id; - logger.trace("Adding FWD message to {}@{}", id, destination); + EndpointsForToken forwardToReplicas = targets.subList(1, targets.size()); + + for (Replica replica : forwardToReplicas) + { + MessagingService.instance().callbacks.addWithExpiration(handler, message, replica, handler.replicaPlan.consistencyLevel(), true); + logger.trace("Adding FWD message to {}@{}", message.id(), replica); + } + + // starting with 4.0, use the same message id for all replicas + long[] messageIds = new long[forwardToReplicas.size()]; + Arrays.fill(messageIds, message.id()); + + message = message.withForwardTo(new ForwardingInfo(forwardToReplicas.endpointList(), messageIds)); } - message = message.withParameter(ParameterType.FORWARD_TO, new ForwardToContainer(targets.endpoints(), messageIds)); - // send the combined message + forward headers - int id = MessagingService.instance().sendWriteRR(message, target, handler, true); - logger.trace("Sending message to {}@{}", id, target); + MessagingService.instance().sendWriteWithCallback(message, targets.get(0), handler, true); + logger.trace("Sending message to {}@{}", message.id(), targets.get(0)); } private static void performLocally(Stage stage, Replica localReplica, final Runnable runnable) @@ -1341,34 +1331,34 @@ public void runMayThrow() @Override protected Verb verb() { - return MessagingService.Verb.MUTATION; + return Verb.MUTATION_REQ; } }); } - private static void performLocally(Stage stage, Replica localReplica, Optional mutation, final Runnable runnable, final IAsyncCallbackWithFailure handler) + private static void performLocally(Stage stage, Replica localReplica, final Runnable runnable, final RequestCallback handler) { - StageManager.getStage(stage).maybeExecuteImmediately(new LocalMutationRunnable(localReplica, mutation) + StageManager.getStage(stage).maybeExecuteImmediately(new LocalMutationRunnable(localReplica) { public void runMayThrow() { try { runnable.run(); - handler.response(null); + handler.onResponse(null); } catch (Exception ex) { if (!(ex instanceof WriteTimeoutException)) logger.error("Failed to apply mutation locally : ", ex); - handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN); + handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.forException(ex)); } } @Override protected Verb verb() { - return MessagingService.Verb.MUTATION; + return Verb.MUTATION_REQ; } }); } @@ -1410,7 +1400,8 @@ public static AbstractWriteResponseHandler mutateCounter(CounterMutat WriteType.COUNTER, queryStartNanoTime); Tracing.trace("Enqueuing counter update to {}", replica); - MessagingService.instance().sendWriteRR(cm.makeMutationMessage(), replica, responseHandler, false); + Message message = Message.outWithFlag(Verb.COUNTER_MUTATION_REQ, cm, MessageFlag.CALL_BACK_ON_FAILURE); + MessagingService.instance().sendWriteWithCallback(message, replica, responseHandler, false); return responseHandler; } } @@ -1479,7 +1470,7 @@ private static Runnable counterWriteTask(final IMutation mutation, final AbstractWriteResponseHandler responseHandler, final String localDataCenter) { - return new DroppableRunnable(MessagingService.Verb.COUNTER_MUTATION) + return new DroppableRunnable(Verb.COUNTER_MUTATION_REQ) { @Override public void runMayThrow() throws OverloadedException, WriteTimeoutException @@ -1487,7 +1478,7 @@ public void runMayThrow() throws OverloadedException, WriteTimeoutException assert mutation instanceof CounterMutation; Mutation result = ((CounterMutation) mutation).applyCounterMutation(); - responseHandler.response(null); + responseHandler.onResponse(null); sendToHintedReplicas(result, replicaPlan, responseHandler, localDataCenter, Stage.COUNTER_MUTATION); } }; @@ -1764,11 +1755,10 @@ public static class LocalReadRunnable extends DroppableRunnable { private final ReadCommand command; private final ReadCallback handler; - private final long start = System.nanoTime(); public LocalReadRunnable(ReadCommand command, ReadCallback handler) { - super(MessagingService.Verb.READ); + super(Verb.READ_REQ); this.command = command; this.handler = handler; } @@ -1777,7 +1767,7 @@ protected void runMayThrow() { try { - command.setMonitoringTime(constructionTime, false, verb.getTimeout(), DatabaseDescriptor.getSlowQueryTimeout()); + command.setMonitoringTime(approxCreationTimeNanos, false, verb.expiresAfterNanos(), DatabaseDescriptor.getSlowQueryTimeout(NANOSECONDS)); ReadResponse response; try (ReadExecutionController executionController = command.executionController(); @@ -1792,11 +1782,11 @@ protected void runMayThrow() } else { - MessagingService.instance().incrementDroppedMessages(verb, System.currentTimeMillis() - constructionTime); + MessagingService.instance().metrics.recordSelfDroppedMessage(verb, MonotonicClock.approxTime.now() - approxCreationTimeNanos, NANOSECONDS); handler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.UNKNOWN); } - MessagingService.instance().addLatency(FBUtilities.getBroadcastAddressAndPort(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); + MessagingService.instance().latencySubscribers.add(FBUtilities.getBroadcastAddressAndPort(), MonotonicClock.approxTime.now() - approxCreationTimeNanos, NANOSECONDS); } catch (Throwable t) { @@ -2075,10 +2065,8 @@ private SingleRangeResponse query(ReplicaPlan.ForRangeRead replicaPlan, boolean { Tracing.trace("Enqueuing request to {}", replica); ReadCommand command = replica.isFull() ? rangeCommand : rangeCommand.copyAsTransientQuery(replica); - MessageOut message = command.createMessage(); - if (command.isTrackingRepairedStatus() && replica.isFull()) - message = message.withParameter(ParameterType.TRACK_REPAIRED_DATA, MessagingService.ONE_BYTE); - MessagingService.instance().sendRRWithFailure(message, replica.endpoint(), handler); + Message message = command.createMessage(command.isTrackingRepairedStatus() && replica.isFull()); + MessagingService.instance().sendWithCallback(message, replica.endpoint(), handler); } } @@ -2181,29 +2169,21 @@ public static Map> describeSchemaVersions(boolean withPort) final Set liveHosts = Gossiper.instance.getLiveMembers(); final CountDownLatch latch = new CountDownLatch(liveHosts.size()); - IAsyncCallback cb = new IAsyncCallback() + RequestCallback cb = message -> { - public void response(MessageIn message) - { - // record the response from the remote node. - versions.put(message.from, message.payload); - latch.countDown(); - } - - public boolean isLatencyForSnitch() - { - return false; - } + // record the response from the remote node. + versions.put(message.from(), message.payload); + latch.countDown(); }; // an empty message acts as a request to the SchemaVersionVerbHandler. - MessageOut message = new MessageOut(MessagingService.Verb.SCHEMA_CHECK); + Message message = Message.out(Verb.SCHEMA_VERSION_REQ, noPayload); for (InetAddressAndPort endpoint : liveHosts) - MessagingService.instance().sendRR(message, endpoint, cb); + MessagingService.instance().sendWithCallback(message, endpoint, cb); try { // wait for as long as possible. timeout-1s if possible. - latch.await(DatabaseDescriptor.getRpcTimeout(), TimeUnit.MILLISECONDS); + latch.await(DatabaseDescriptor.getRpcTimeout(NANOSECONDS), NANOSECONDS); } catch (InterruptedException ex) { @@ -2385,10 +2365,9 @@ public static void truncateBlocking(String keyspace, String cfname) throws Unava // Send out the truncate calls and track the responses with the callbacks. Tracing.trace("Enqueuing truncate messages to hosts {}", allEndpoints); - final Truncation truncation = new Truncation(keyspace, cfname); - MessageOut message = truncation.createMessage(); + Message message = Message.out(TRUNCATE_REQ, new TruncateRequest(keyspace, cfname)); for (InetAddressAndPort endpoint : allEndpoints) - MessagingService.instance().sendRR(message, endpoint, responseHandler); + MessagingService.instance().sendWithCallback(message, endpoint, responseHandler); // Wait for all try @@ -2430,9 +2409,9 @@ public ViewWriteMetricsWrapped(AbstractWriteResponseHandler writeHand viewWriteMetrics.viewReplicasAttempted.inc(candidateReplicaCount()); } - public void response(MessageIn msg) + public void onResponse(Message msg) { - super.response(msg); + super.onResponse(msg); viewWriteMetrics.viewReplicasSuccess.inc(); } } @@ -2442,21 +2421,23 @@ public void response(MessageIn msg) */ private static abstract class DroppableRunnable implements Runnable { - final long constructionTime; - final MessagingService.Verb verb; + final long approxCreationTimeNanos; + final Verb verb; - public DroppableRunnable(MessagingService.Verb verb) + public DroppableRunnable(Verb verb) { - this.constructionTime = System.currentTimeMillis(); + this.approxCreationTimeNanos = MonotonicClock.approxTime.now(); this.verb = verb; } public final void run() { - long timeTaken = System.currentTimeMillis() - constructionTime; - if (timeTaken > verb.getTimeout()) + long approxCurrentTimeNanos = MonotonicClock.approxTime.now(); + long expirationTimeNanos = verb.expiresAtNanos(approxCreationTimeNanos); + if (approxCurrentTimeNanos > expirationTimeNanos) { - MessagingService.instance().incrementDroppedMessages(verb, timeTaken); + long timeTakenNanos = approxCurrentTimeNanos - approxCreationTimeNanos; + MessagingService.instance().metrics.recordSelfDroppedMessage(verb, timeTakenNanos, NANOSECONDS); return; } try @@ -2478,32 +2459,24 @@ public final void run() */ private static abstract class LocalMutationRunnable implements Runnable { - private final long constructionTime = System.currentTimeMillis(); + private final long approxCreationTimeNanos = MonotonicClock.approxTime.now(); private final Replica localReplica; - private final Optional mutationOpt; - public LocalMutationRunnable(Replica localReplica, Optional mutationOpt) + LocalMutationRunnable(Replica localReplica) { this.localReplica = localReplica; - this.mutationOpt = mutationOpt; - } - - public LocalMutationRunnable(Replica localReplica) - { - this.localReplica = localReplica; - this.mutationOpt = Optional.empty(); } public final void run() { - final MessagingService.Verb verb = verb(); - long mutationTimeout = verb.getTimeout(); - long timeTaken = System.currentTimeMillis() - constructionTime; - if (timeTaken > mutationTimeout) + final Verb verb = verb(); + long nowNanos = MonotonicClock.approxTime.now(); + long expirationTimeNanos = verb.expiresAtNanos(approxCreationTimeNanos); + if (nowNanos > expirationTimeNanos) { - if (MessagingService.DROPPABLE_VERBS.contains(verb)) - MessagingService.instance().incrementDroppedMutations(mutationOpt, timeTaken); + long timeTakenNanos = nowNanos - approxCreationTimeNanos; + MessagingService.instance().metrics.recordSelfDroppedMessage(Verb.MUTATION_REQ, timeTakenNanos, NANOSECONDS); HintRunnable runnable = new HintRunnable(EndpointsForToken.of(localReplica.range().right, localReplica)) { @@ -2526,7 +2499,7 @@ protected void runMayThrow() throws Exception } } - abstract protected MessagingService.Verb verb(); + abstract protected Verb verb(); abstract protected void runMayThrow() throws Exception; } @@ -2634,7 +2607,7 @@ public void runMayThrow() validTargets.forEach(HintsService.instance.metrics::incrCreatedHints); // Notify the handler only for CL == ANY if (responseHandler != null && responseHandler.replicaPlan.consistencyLevel() == ConsistencyLevel.ANY) - responseHandler.response(null); + responseHandler.onResponse(null); } }; @@ -2649,25 +2622,25 @@ private static Future submitHint(HintRunnable runnable) return (Future) StageManager.getStage(Stage.MUTATION).submit(runnable); } - public Long getRpcTimeout() { return DatabaseDescriptor.getRpcTimeout(); } + public Long getRpcTimeout() { return DatabaseDescriptor.getRpcTimeout(MILLISECONDS); } public void setRpcTimeout(Long timeoutInMillis) { DatabaseDescriptor.setRpcTimeout(timeoutInMillis); } - public Long getReadRpcTimeout() { return DatabaseDescriptor.getReadRpcTimeout(); } + public Long getReadRpcTimeout() { return DatabaseDescriptor.getReadRpcTimeout(MILLISECONDS); } public void setReadRpcTimeout(Long timeoutInMillis) { DatabaseDescriptor.setReadRpcTimeout(timeoutInMillis); } - public Long getWriteRpcTimeout() { return DatabaseDescriptor.getWriteRpcTimeout(); } + public Long getWriteRpcTimeout() { return DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS); } public void setWriteRpcTimeout(Long timeoutInMillis) { DatabaseDescriptor.setWriteRpcTimeout(timeoutInMillis); } - public Long getCounterWriteRpcTimeout() { return DatabaseDescriptor.getCounterWriteRpcTimeout(); } + public Long getCounterWriteRpcTimeout() { return DatabaseDescriptor.getCounterWriteRpcTimeout(MILLISECONDS); } public void setCounterWriteRpcTimeout(Long timeoutInMillis) { DatabaseDescriptor.setCounterWriteRpcTimeout(timeoutInMillis); } - public Long getCasContentionTimeout() { return DatabaseDescriptor.getCasContentionTimeout(); } + public Long getCasContentionTimeout() { return DatabaseDescriptor.getCasContentionTimeout(MILLISECONDS); } public void setCasContentionTimeout(Long timeoutInMillis) { DatabaseDescriptor.setCasContentionTimeout(timeoutInMillis); } - public Long getRangeRpcTimeout() { return DatabaseDescriptor.getRangeRpcTimeout(); } + public Long getRangeRpcTimeout() { return DatabaseDescriptor.getRangeRpcTimeout(MILLISECONDS); } public void setRangeRpcTimeout(Long timeoutInMillis) { DatabaseDescriptor.setRangeRpcTimeout(timeoutInMillis); } - public Long getTruncateRpcTimeout() { return DatabaseDescriptor.getTruncateRpcTimeout(); } + public Long getTruncateRpcTimeout() { return DatabaseDescriptor.getTruncateRpcTimeout(MILLISECONDS); } public void setTruncateRpcTimeout(Long timeoutInMillis) { DatabaseDescriptor.setTruncateRpcTimeout(timeoutInMillis); } public Long getNativeTransportMaxConcurrentConnections() { return DatabaseDescriptor.getNativeTransportMaxConcurrentConnections(); } @@ -2739,13 +2712,13 @@ public void stopFullQueryLogger() AuditLogManager.getInstance().disableFQL(); } + @Deprecated public int getOtcBacklogExpirationInterval() { - return DatabaseDescriptor.getOtcBacklogExpirationInterval(); + return 0; } - public void setOtcBacklogExpirationInterval(int intervalInMillis) { - DatabaseDescriptor.setOtcBacklogExpirationInterval(intervalInMillis); - } + @Deprecated + public void setOtcBacklogExpirationInterval(int intervalInMillis) { } @Override public void enableRepairedDataTrackingForRangeReads() diff --git a/src/java/org/apache/cassandra/service/StorageProxyMBean.java b/src/java/org/apache/cassandra/service/StorageProxyMBean.java index 95f5f26afff8..08b5cbd3b479 100644 --- a/src/java/org/apache/cassandra/service/StorageProxyMBean.java +++ b/src/java/org/apache/cassandra/service/StorageProxyMBean.java @@ -63,7 +63,9 @@ public interface StorageProxyMBean public long getReadRepairRepairedBlocking(); public long getReadRepairRepairedBackground(); + @Deprecated public int getOtcBacklogExpirationInterval(); + @Deprecated public void setOtcBacklogExpirationInterval(int intervalInMillis); /** Returns each live node's schema version */ diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 2b6bf1d77190..e5ae30b08587 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -56,8 +56,6 @@ import org.apache.cassandra.audit.AuditLogOptions; import org.apache.cassandra.auth.AuthKeyspace; import org.apache.cassandra.auth.AuthSchemaChangeListener; -import org.apache.cassandra.batchlog.BatchRemoveVerbHandler; -import org.apache.cassandra.batchlog.BatchStoreVerbHandler; import org.apache.cassandra.batchlog.BatchlogManager; import org.apache.cassandra.concurrent.ExecutorLocals; import org.apache.cassandra.concurrent.NamedThreadFactory; @@ -78,7 +76,6 @@ import org.apache.cassandra.dht.Token.TokenFactory; import org.apache.cassandra.exceptions.*; import org.apache.cassandra.gms.*; -import org.apache.cassandra.hints.HintVerbHandler; import org.apache.cassandra.hints.HintsService; import org.apache.cassandra.io.sstable.SSTableLoader; import org.apache.cassandra.io.util.FileUtils; @@ -93,16 +90,9 @@ import org.apache.cassandra.schema.ReplicationParams; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.schema.SchemaPullVerbHandler; -import org.apache.cassandra.schema.SchemaPushVerbHandler; -import org.apache.cassandra.schema.SchemaVersionVerbHandler; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.TableMetadataRef; import org.apache.cassandra.schema.ViewMetadata; -import org.apache.cassandra.repair.RepairMessageVerbHandler; -import org.apache.cassandra.service.paxos.CommitVerbHandler; -import org.apache.cassandra.service.paxos.PrepareVerbHandler; -import org.apache.cassandra.service.paxos.ProposeVerbHandler; import org.apache.cassandra.streaming.*; import org.apache.cassandra.tracing.TraceKeyspace; import org.apache.cassandra.transport.ProtocolVersion; @@ -110,15 +100,20 @@ import org.apache.cassandra.utils.logging.LoggingSupportFactory; import org.apache.cassandra.utils.progress.ProgressEvent; import org.apache.cassandra.utils.progress.ProgressEventType; +import org.apache.cassandra.utils.progress.ProgressListener; import org.apache.cassandra.utils.progress.jmx.JMXBroadcastExecutor; import org.apache.cassandra.utils.progress.jmx.JMXProgressSupport; import static com.google.common.collect.Iterables.transform; import static com.google.common.collect.Iterables.tryFind; import static java.util.Arrays.asList; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.stream.Collectors.toList; import static org.apache.cassandra.index.SecondaryIndexManager.getIndexName; import static org.apache.cassandra.index.SecondaryIndexManager.isIndexColumnFamily; +import static org.apache.cassandra.net.NoPayload.noPayload; +import static org.apache.cassandra.net.Verb.REPLICATION_DONE_REQ; /** * This abstraction contains the token/identifier of this node @@ -283,44 +278,6 @@ public StorageService() jmxObjectName = "org.apache.cassandra.db:type=StorageService"; MBeanWrapper.instance.registerMBean(this, jmxObjectName); MBeanWrapper.instance.registerMBean(StreamManager.instance, StreamManager.OBJECT_NAME); - - ReadCommandVerbHandler readHandler = new ReadCommandVerbHandler(); - - /* register the verb handlers */ - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.MUTATION, new MutationVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.READ_REPAIR, new ReadRepairVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.READ, readHandler); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.RANGE_SLICE, readHandler); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAGED_RANGE, readHandler); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.COUNTER_MUTATION, new CounterMutationVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.TRUNCATE, new TruncateVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_PREPARE, new PrepareVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_PROPOSE, new ProposeVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_COMMIT, new CommitVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.HINT, new HintVerbHandler()); - - // see BootStrapper for a summary of how the bootstrap verbs interact - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REPLICATION_FINISHED, new ReplicationFinishedVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REQUEST_RESPONSE, new ResponseVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.INTERNAL_RESPONSE, new ResponseVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REPAIR_MESSAGE, new RepairMessageVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_SHUTDOWN, new GossipShutdownVerbHandler()); - - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_SYN, new GossipDigestSynVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_ACK, new GossipDigestAckVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_ACK2, new GossipDigestAck2VerbHandler()); - - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.DEFINITIONS_UPDATE, new SchemaPushVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SCHEMA_CHECK, new SchemaVersionVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.MIGRATION_REQUEST, new SchemaPullVerbHandler()); - - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SNAPSHOT, new SnapshotVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.ECHO, new EchoVerbHandler()); - - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.BATCH_STORE, new BatchStoreVerbHandler()); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.BATCH_REMOVE, new BatchRemoveVerbHandler()); - - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PING, new PingVerbHandler()); } public void registerDaemon(CassandraDaemon daemon) @@ -625,8 +582,7 @@ public void unsafeInitialize() throws ConfigurationException Gossiper.instance.register(this); Gossiper.instance.start((int) (System.currentTimeMillis() / 1000)); // needed for node-ring gathering. Gossiper.instance.addLocalApplicationState(ApplicationState.NET_VERSION, valueFactory.networkVersion()); - if (!MessagingService.instance().isListening()) - MessagingService.instance().listen(); + MessagingService.instance().listen(); } public void populateTokenMetadata() @@ -808,8 +764,7 @@ private void prepareToJoin() throws ConfigurationException if (DatabaseDescriptor.getReplaceTokens().size() > 0 || DatabaseDescriptor.getReplaceNode() != null) throw new RuntimeException("Replace method removed; use cassandra.replace_address instead"); - if (!MessagingService.instance().isListening()) - MessagingService.instance().listen(); + MessagingService.instance().listen(); UUID localHostId = SystemKeyspace.getLocalHostId(); @@ -1356,7 +1311,7 @@ public void setRpcTimeout(long value) public long getRpcTimeout() { - return DatabaseDescriptor.getRpcTimeout(); + return DatabaseDescriptor.getRpcTimeout(MILLISECONDS); } public void setReadRpcTimeout(long value) @@ -1367,7 +1322,7 @@ public void setReadRpcTimeout(long value) public long getReadRpcTimeout() { - return DatabaseDescriptor.getReadRpcTimeout(); + return DatabaseDescriptor.getReadRpcTimeout(MILLISECONDS); } public void setRangeRpcTimeout(long value) @@ -1378,7 +1333,7 @@ public void setRangeRpcTimeout(long value) public long getRangeRpcTimeout() { - return DatabaseDescriptor.getRangeRpcTimeout(); + return DatabaseDescriptor.getRangeRpcTimeout(MILLISECONDS); } public void setWriteRpcTimeout(long value) @@ -1389,7 +1344,7 @@ public void setWriteRpcTimeout(long value) public long getWriteRpcTimeout() { - return DatabaseDescriptor.getWriteRpcTimeout(); + return DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS); } public void setInternodeTcpConnectTimeoutInMS(int value) @@ -1422,7 +1377,7 @@ public void setCounterWriteRpcTimeout(long value) public long getCounterWriteRpcTimeout() { - return DatabaseDescriptor.getCounterWriteRpcTimeout(); + return DatabaseDescriptor.getCounterWriteRpcTimeout(MILLISECONDS); } public void setCasContentionTimeout(long value) @@ -1433,7 +1388,7 @@ public void setCasContentionTimeout(long value) public long getCasContentionTimeout() { - return DatabaseDescriptor.getCasContentionTimeout(); + return DatabaseDescriptor.getCasContentionTimeout(MILLISECONDS); } public void setTruncateRpcTimeout(long value) @@ -1444,7 +1399,7 @@ public void setTruncateRpcTimeout(long value) public long getTruncateRpcTimeout() { - return DatabaseDescriptor.getTruncateRpcTimeout(); + return DatabaseDescriptor.getTruncateRpcTimeout(MILLISECONDS); } public void setStreamThroughputMbPerSec(int value) @@ -1581,7 +1536,7 @@ private boolean bootstrap(final Collection tokens) valueFactory.bootstrapping(tokens))); Gossiper.instance.addLocalApplicationStates(states); setMode(Mode.JOINING, "sleeping " + RING_DELAY + " ms for pending range setup", true); - Uninterruptibles.sleepUninterruptibly(RING_DELAY, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(RING_DELAY, MILLISECONDS); } else { @@ -2212,7 +2167,7 @@ private void updateNetVersion(InetAddressAndPort endpoint, VersionedValue value) { try { - MessagingService.instance().setVersion(endpoint, Integer.parseInt(value.value)); + MessagingService.instance().versions.set(endpoint, Integer.parseInt(value.value)); } catch (NumberFormatException e) { @@ -2754,8 +2709,8 @@ private void excise(Collection tokens, InetAddressAndPort endpoint) { // enough time for writes to expire and MessagingService timeout reporter callback to fire, which is where // hints are mostly written from - using getMinRpcTimeout() / 2 for the interval. - long delay = DatabaseDescriptor.getMinRpcTimeout() + DatabaseDescriptor.getWriteRpcTimeout(); - ScheduledExecutors.optionalTasks.schedule(() -> HintsService.instance.excise(hostId), delay, TimeUnit.MILLISECONDS); + long delay = DatabaseDescriptor.getMinRpcTimeout(MILLISECONDS) + DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS); + ScheduledExecutors.optionalTasks.schedule(() -> HintsService.instance.excise(hostId), delay, MILLISECONDS); } removeEndpoint(endpoint); @@ -2859,22 +2814,22 @@ private Multimap getNewSourceReplicas(String k private void sendReplicationNotification(InetAddressAndPort remote) { // notify the remote token - MessageOut msg = new MessageOut(MessagingService.Verb.REPLICATION_FINISHED); + Message msg = Message.out(REPLICATION_DONE_REQ, noPayload); IFailureDetector failureDetector = FailureDetector.instance; if (logger.isDebugEnabled()) logger.debug("Notifying {} of replication completion\n", remote); while (failureDetector.isAlive(remote)) { - AsyncOneResponse iar = MessagingService.instance().sendRR(msg, remote); - try - { - iar.get(DatabaseDescriptor.getRpcTimeout(), TimeUnit.MILLISECONDS); - return; // done - } - catch(TimeoutException e) - { - // try again - } + AsyncOneResponse ior = new AsyncOneResponse(); + MessagingService.instance().sendWithCallback(msg, remote, ior); + + if (!ior.awaitUninterruptibly(DatabaseDescriptor.getRpcTimeout(NANOSECONDS), NANOSECONDS)) + continue; // try again if we timeout + + if (!ior.isSuccess()) + throw new AssertionError(ior.cause()); + + return; } } @@ -3098,7 +3053,9 @@ public void onRemove(InetAddressAndPort endpoint) public void onDead(InetAddressAndPort endpoint, EndpointState state) { - MessagingService.instance().convict(endpoint); + // interrupt any outbound connection; if the node is failing and we cannot reconnect, + // this will rapidly lower the number of bytes we are willing to queue to the node + MessagingService.instance().interruptOutbound(endpoint); notifyDown(endpoint); } @@ -3781,6 +3738,11 @@ public void forceKeyspaceFlush(String keyspaceName, String... tableNames) throws } public int repairAsync(String keyspace, Map repairSpec) + { + return repair(keyspace, repairSpec, Collections.emptyList()).left; + } + + public Pair> repair(String keyspace, Map repairSpec, List listeners) { RepairOption option = RepairOption.parse(repairSpec, tokenMetadata.partitioner); // if ranges are not specified @@ -3803,11 +3765,10 @@ else if (option.isInLocalDCOnly()) } } if (option.getRanges().isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor().allReplicas < 2) - return 0; + return Pair.create(0, Futures.immediateFuture(null)); int cmd = nextRepairCommand.incrementAndGet(); - ActiveRepairService.repairCommandExecutor.execute(createRepairTask(cmd, keyspace, option)); - return cmd; + return Pair.create(cmd, ActiveRepairService.repairCommandExecutor.submit(createRepairTask(cmd, keyspace, option, listeners))); } /** @@ -3853,7 +3814,7 @@ public TokenFactory getTokenFactory() return tokenMetadata.partitioner.getTokenFactory(); } - private FutureTask createRepairTask(final int cmd, final String keyspace, final RepairOption options) + private FutureTask createRepairTask(final int cmd, final String keyspace, final RepairOption options, List listeners) { if (!options.getDataCenters().isEmpty() && !options.getDataCenters().contains(DatabaseDescriptor.getLocalDataCenter())) { @@ -3862,6 +3823,9 @@ private FutureTask createRepairTask(final int cmd, final String keyspace RepairRunnable task = new RepairRunnable(this, cmd, options, keyspace); task.addProgressListener(progressSupport); + for (ProgressListener listener : listeners) + task.addProgressListener(listener); + if (options.isTraced()) { Runnable r = () -> @@ -4249,7 +4213,7 @@ private void leaveRing() Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.left(getLocalTokens(),Gossiper.computeExpireTime())); int delay = Math.max(RING_DELAY, Gossiper.intervalInMillis * 2); logger.info("Announcing that I have left the ring for {}ms", delay); - Uninterruptibles.sleepUninterruptibly(delay, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(delay, MILLISECONDS); } private void unbootstrap(Runnable onFinish) throws ExecutionException, InterruptedException @@ -4378,7 +4342,7 @@ private void move(Token newToken) throws IOException setMode(Mode.MOVING, String.format("Moving %s from %s to %s.", localAddress, getLocalTokens().iterator().next(), newToken), true); setMode(Mode.MOVING, String.format("Sleeping %s ms before start streaming/fetching ranges", RING_DELAY), true); - Uninterruptibles.sleepUninterruptibly(RING_DELAY, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(RING_DELAY, MILLISECONDS); RangeRelocator relocator = new RangeRelocator(Collections.singleton(newToken), keyspacesToProcess, tokenMetadata); relocator.calculateToFromStreams(); @@ -4534,10 +4498,10 @@ public void removeNode(String hostIdString) // kick off streaming commands restoreReplicaCount(endpoint, myAddress); - // wait for ReplicationFinishedVerbHandler to signal we're done + // wait for ReplicationDoneVerbHandler to signal we're done while (!replicatingNodes.isEmpty()) { - Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(100, MILLISECONDS); } excise(tokens, endpoint); @@ -5266,7 +5230,7 @@ public Map> samplePartitions(int durationMillis, int table.beginLocalSampling(sampler, capacity, durationMillis); } } - Uninterruptibles.sleepUninterruptibly(durationMillis, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(durationMillis, MILLISECONDS); for (String sampler : samplers) { diff --git a/src/java/org/apache/cassandra/service/TruncateResponseHandler.java b/src/java/org/apache/cassandra/service/TruncateResponseHandler.java index cce8ecc111eb..bcd7426fd04e 100644 --- a/src/java/org/apache/cassandra/service/TruncateResponseHandler.java +++ b/src/java/org/apache/cassandra/service/TruncateResponseHandler.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.service; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -25,11 +24,13 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.net.IAsyncCallback; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; import org.apache.cassandra.utils.concurrent.SimpleCondition; -public class TruncateResponseHandler implements IAsyncCallback +import static java.util.concurrent.TimeUnit.NANOSECONDS; + +public class TruncateResponseHandler implements RequestCallback { protected static final Logger logger = LoggerFactory.getLogger(TruncateResponseHandler.class); protected final SimpleCondition condition = new SimpleCondition(); @@ -49,11 +50,11 @@ public TruncateResponseHandler(int responseCount) public void get() throws TimeoutException { - long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getTruncateRpcTimeout()) - (System.nanoTime() - start); + long timeoutNanos = DatabaseDescriptor.getTruncateRpcTimeout(NANOSECONDS) - (System.nanoTime() - start); boolean success; try { - success = condition.await(timeout, TimeUnit.NANOSECONDS); // TODO truncate needs a much longer timeout + success = condition.await(timeoutNanos, NANOSECONDS); // TODO truncate needs a much longer timeout } catch (InterruptedException ex) { @@ -66,15 +67,10 @@ public void get() throws TimeoutException } } - public void response(MessageIn message) + public void onResponse(Message message) { responses.incrementAndGet(); if (responses.get() >= responseCount) condition.signalAll(); } - - public boolean isLatencyForSnitch() - { - return false; - } } diff --git a/src/java/org/apache/cassandra/service/WriteResponseHandler.java b/src/java/org/apache/cassandra/service/WriteResponseHandler.java index f9bfedf007fc..94f5a80729a7 100644 --- a/src/java/org/apache/cassandra/service/WriteResponseHandler.java +++ b/src/java/org/apache/cassandra/service/WriteResponseHandler.java @@ -23,7 +23,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.db.WriteType; /** @@ -51,7 +51,7 @@ public WriteResponseHandler(ReplicaPlan.ForTokenWrite replicaPlan, WriteType wri this(replicaPlan, null, writeType, queryStartNanoTime); } - public void response(MessageIn m) + public void onResponse(Message m) { if (responsesUpdater.decrementAndGet(this) == 0) signal(); @@ -65,9 +65,4 @@ protected int ackCount() { return blockFor() - responses; } - - public boolean isLatencyForSnitch() - { - return false; - } } diff --git a/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java b/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java index 90bfc5d9d759..ab24f50efb8c 100644 --- a/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java +++ b/src/java/org/apache/cassandra/service/paxos/AbstractPaxosCallback.java @@ -1,4 +1,3 @@ -package org.apache.cassandra.service.paxos; /* * * Licensed to the Apache Software Foundation (ASF) under one @@ -19,18 +18,19 @@ * under the License. * */ - +package org.apache.cassandra.service.paxos; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.WriteType; import org.apache.cassandra.exceptions.WriteTimeoutException; -import org.apache.cassandra.net.IAsyncCallback; +import org.apache.cassandra.net.RequestCallback; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; -public abstract class AbstractPaxosCallback implements IAsyncCallback +public abstract class AbstractPaxosCallback implements RequestCallback { protected final CountDownLatch latch; protected final int targets; @@ -45,11 +45,6 @@ public AbstractPaxosCallback(int targets, ConsistencyLevel consistency, long que this.queryStartNanoTime = queryStartNanoTime; } - public boolean isLatencyForSnitch() - { - return false; - } - public int getResponseCount() { return (int) (targets - latch.getCount()); @@ -59,8 +54,8 @@ public void await() throws WriteTimeoutException { try { - long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getWriteRpcTimeout()) - (System.nanoTime() - queryStartNanoTime); - if (!latch.await(timeout, TimeUnit.NANOSECONDS)) + long timeout = DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS) - (System.nanoTime() - queryStartNanoTime); + if (!latch.await(timeout, NANOSECONDS)) throw new WriteTimeoutException(WriteType.CAS, consistency, getResponseCount(), targets); } catch (InterruptedException ex) diff --git a/src/java/org/apache/cassandra/service/paxos/CommitVerbHandler.java b/src/java/org/apache/cassandra/service/paxos/CommitVerbHandler.java index a702a4dfd750..12466ddc691e 100644 --- a/src/java/org/apache/cassandra/service/paxos/CommitVerbHandler.java +++ b/src/java/org/apache/cassandra/service/paxos/CommitVerbHandler.java @@ -20,19 +20,20 @@ */ package org.apache.cassandra.service.paxos; -import org.apache.cassandra.db.WriteResponse; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.tracing.Tracing; public class CommitVerbHandler implements IVerbHandler { - public void doVerb(MessageIn message, int id) + public static final CommitVerbHandler instance = new CommitVerbHandler(); + + public void doVerb(Message message) { PaxosState.commit(message.payload); - Tracing.trace("Enqueuing acknowledge to {}", message.from); - MessagingService.instance().sendReply(WriteResponse.createMessage(), id, message.from); + Tracing.trace("Enqueuing acknowledge to {}", message.from()); + MessagingService.instance().send(message.emptyResponse(), message.from()); } } diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java index ed70e964c03e..26890a9b102a 100644 --- a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java +++ b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java @@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.db.SystemKeyspace; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.utils.UUIDGen; public class PrepareCallback extends AbstractPaxosCallback @@ -53,16 +53,16 @@ public class PrepareCallback extends AbstractPaxosCallback public PrepareCallback(DecoratedKey key, TableMetadata metadata, int targets, ConsistencyLevel consistency, long queryStartNanoTime) { super(targets, consistency, queryStartNanoTime); - // need to inject the right key in the empty commit so comparing with empty commits in the reply works as expected + // need to inject the right key in the empty commit so comparing with empty commits in the response works as expected mostRecentCommit = Commit.emptyCommit(key, metadata); mostRecentInProgressCommit = Commit.emptyCommit(key, metadata); mostRecentInProgressCommitWithUpdate = Commit.emptyCommit(key, metadata); } - public synchronized void response(MessageIn message) + public synchronized void onResponse(Message message) { PrepareResponse response = message.payload; - logger.trace("Prepare response {} from {}", response, message.from); + logger.trace("Prepare response {} from {}", response, message.from()); // In case of clock skew, another node could be proposing with ballot that are quite a bit // older than our own. In that case, we record the more recent commit we've received to make @@ -78,7 +78,7 @@ public synchronized void response(MessageIn message) return; } - commitsByReplica.put(message.from, response.mostRecentCommit); + commitsByReplica.put(message.from(), response.mostRecentCommit); if (response.mostRecentCommit.isAfter(mostRecentCommit)) mostRecentCommit = response.mostRecentCommit; diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareVerbHandler.java b/src/java/org/apache/cassandra/service/paxos/PrepareVerbHandler.java index 2750b7611207..157630f277af 100644 --- a/src/java/org/apache/cassandra/service/paxos/PrepareVerbHandler.java +++ b/src/java/org/apache/cassandra/service/paxos/PrepareVerbHandler.java @@ -19,23 +19,22 @@ * under the License. * */ - - import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; public class PrepareVerbHandler implements IVerbHandler { + public static PrepareVerbHandler instance = new PrepareVerbHandler(); + public static PrepareResponse doPrepare(Commit toPrepare) { return PaxosState.prepare(toPrepare); } - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - MessageOut reply = new MessageOut(MessagingService.Verb.REQUEST_RESPONSE, doPrepare(message.payload), PrepareResponse.serializer); - MessagingService.instance().sendReply(reply, id, message.from); + Message reply = message.responseWith(doPrepare(message.payload)); + MessagingService.instance().send(reply, message.from()); } } diff --git a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java index c9cb1f0604b3..7e755a03d8a6 100644 --- a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java +++ b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java @@ -27,7 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; /** * ProposeCallback has two modes of operation, controlled by the failFast parameter. @@ -35,7 +35,7 @@ * In failFast mode, we will return a failure as soon as a majority of nodes reject * the proposal. This is used when replaying a proposal from an earlier leader. * - * Otherwise, we wait for either all replicas to reply or until we achieve + * Otherwise, we wait for either all replicas to respond or until we achieve * the desired quorum. We continue to wait for all replicas even after we know we cannot succeed * because we need to know if no node at all have accepted or if at least one has. * In the former case, a proposer is guaranteed no-one will @@ -57,9 +57,9 @@ public ProposeCallback(int totalTargets, int requiredTargets, boolean failFast, this.failFast = failFast; } - public void response(MessageIn msg) + public void onResponse(Message msg) { - logger.trace("Propose response {} from {}", msg.payload, msg.from); + logger.trace("Propose response {} from {}", msg.payload, msg.from()); if (msg.payload) accepts.incrementAndGet(); diff --git a/src/java/org/apache/cassandra/service/paxos/ProposeVerbHandler.java b/src/java/org/apache/cassandra/service/paxos/ProposeVerbHandler.java index 81c90174e054..5a20b674580f 100644 --- a/src/java/org/apache/cassandra/service/paxos/ProposeVerbHandler.java +++ b/src/java/org/apache/cassandra/service/paxos/ProposeVerbHandler.java @@ -19,24 +19,22 @@ * under the License. * */ - - import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.utils.BooleanSerializer; public class ProposeVerbHandler implements IVerbHandler { + public static final ProposeVerbHandler instance = new ProposeVerbHandler(); + public static Boolean doPropose(Commit proposal) { return PaxosState.propose(proposal); } - public void doVerb(MessageIn message, int id) + public void doVerb(Message message) { - MessageOut reply = new MessageOut(MessagingService.Verb.REQUEST_RESPONSE, doPropose(message.payload), BooleanSerializer.serializer); - MessagingService.instance().sendReply(reply, id, message.from); + Message reply = message.responseWith(doPropose(message.payload)); + MessagingService.instance().send(reply, message.from()); } } diff --git a/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java index 174ed7bff6f7..27e928072577 100644 --- a/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java +++ b/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java @@ -17,8 +17,6 @@ */ package org.apache.cassandra.service.reads; -import java.util.concurrent.TimeUnit; - import com.google.common.base.Preconditions; import com.google.common.base.Predicates; @@ -43,7 +41,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaCollection; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.reads.repair.ReadRepair; import org.apache.cassandra.service.StorageProxy.LocalReadRunnable; @@ -51,6 +49,7 @@ import org.apache.cassandra.tracing.Tracing; import static com.google.common.collect.Iterables.all; +import static java.util.concurrent.TimeUnit.NANOSECONDS; /** * Sends a read request to the replicas needed to satisfy a given ConsistencyLevel. @@ -95,7 +94,7 @@ public abstract class AbstractReadExecutor // we stop being compatible with pre-3.0 nodes. int digestVersion = MessagingService.current_version; for (Replica replica : replicaPlan.contacts()) - digestVersion = Math.min(digestVersion, MessagingService.instance().getVersion(replica.endpoint())); + digestVersion = Math.min(digestVersion, MessagingService.instance().versions.get(replica.endpoint())); command.setDigestVersion(digestVersion); } @@ -132,6 +131,7 @@ protected void makeDigestRequests(Iterable replicas) private void makeRequests(ReadCommand readCommand, Iterable replicas) { boolean hasLocalEndpoint = false; + Message message = null; for (Replica replica: replicas) { @@ -146,8 +146,11 @@ private void makeRequests(ReadCommand readCommand, Iterable replicas) if (traceState != null) traceState.trace("reading {} from {}", readCommand.isDigestQuery() ? "digest" : "data", endpoint); - MessageOut message = readCommand.createMessage(); - MessagingService.instance().sendRRWithFailure(message, endpoint, handler); + + if (null == message) + message = readCommand.createMessage(false); + + MessagingService.instance().sendWithCallback(message, endpoint, handler); } // We delay the local (potentially blocking) read till the end to avoid stalling remote requests. @@ -213,10 +216,10 @@ public static AbstractReadExecutor getReadExecutor(SinglePartitionReadCommand co boolean shouldSpeculateAndMaybeWait() { // no latency information, or we're overloaded - if (cfs.sampleReadLatencyNanos > TimeUnit.MILLISECONDS.toNanos(command.getTimeout())) + if (cfs.sampleReadLatencyNanos > command.getTimeout(NANOSECONDS)) return false; - return !handler.await(cfs.sampleReadLatencyNanos, TimeUnit.NANOSECONDS); + return !handler.await(cfs.sampleReadLatencyNanos, NANOSECONDS); } ReplicaPlan.ForTokenRead replicaPlan() @@ -261,7 +264,7 @@ public SpeculatingReadExecutor(ColumnFamilyStore cfs, { // We're hitting additional targets for read repair (??). Since our "extra" replica is the least- // preferred by the snitch, we do an extra data read to start with against a replica more - // likely to reply; better to let RR fail than the entire query. + // likely to respond; better to let RR fail than the entire query. super(cfs, command, replicaPlan, replicaPlan.blockFor() < replicaPlan.contacts().size() ? 2 : 1, queryStartNanoTime); } @@ -308,7 +311,7 @@ public void maybeTryAdditionalReplicas() if (traceState != null) traceState.trace("speculating read retry on {}", extraReplica); logger.trace("speculating read retry on {}", extraReplica); - MessagingService.instance().sendRRWithFailure(retryCommand.createMessage(), extraReplica.endpoint(), handler); + MessagingService.instance().sendWithCallback(retryCommand.createMessage(false), extraReplica.endpoint(), handler); } } diff --git a/src/java/org/apache/cassandra/service/reads/DataResolver.java b/src/java/org/apache/cassandra/service/reads/DataResolver.java index 03f718f25a41..45bf9186004f 100644 --- a/src/java/org/apache/cassandra/service/reads/DataResolver.java +++ b/src/java/org/apache/cassandra/service/reads/DataResolver.java @@ -43,14 +43,13 @@ import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.locator.Endpoints; import org.apache.cassandra.locator.ReplicaPlan; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.reads.repair.ReadRepair; import org.apache.cassandra.service.reads.repair.RepairedDataTracker; import org.apache.cassandra.service.reads.repair.RepairedDataVerifier; import static com.google.common.collect.Iterables.*; -import static org.apache.cassandra.db.partitions.UnfilteredPartitionIterators.MergeListener; public class DataResolver, P extends ReplicaPlan.ForRead> extends ResponseResolver { @@ -80,10 +79,10 @@ public PartitionIterator resolve() { // We could get more responses while this method runs, which is ok (we're happy to ignore any response not here // at the beginning of this method), so grab the response count once and use that through the method. - Collection> messages = responses.snapshot(); + Collection> messages = responses.snapshot(); assert !any(messages, msg -> msg.payload.isDigestResponse()); - E replicas = replicaPlan().candidates().select(transform(messages, msg -> msg.from), false); + E replicas = replicaPlan().candidates().select(transform(messages, msg -> msg.from()), false); List iters = new ArrayList<>( Collections2.transform(messages, msg -> msg.payload.makeIterator(command))); assert replicas.size() == iters.size(); @@ -95,9 +94,9 @@ public PartitionIterator resolve() if (repairedDataTracker != null) { messages.forEach(msg -> { - if (msg.payload.mayIncludeRepairedDigest() && replicas.byEndpoint().get(msg.from).isFull()) + if (msg.payload.mayIncludeRepairedDigest() && replicas.byEndpoint().get(msg.from()).isFull()) { - repairedDataTracker.recordDigest(msg.from, + repairedDataTracker.recordDigest(msg.from(), msg.payload.repairedDataDigest(), msg.payload.isRepairedDigestConclusive()); } @@ -157,7 +156,7 @@ private UnfilteredPartitionIterator mergeWithShortReadProtection(List m.from + " => " + m.payload.toDebugString(command, partitionKey))); + return Joiner.on(",\n").join(transform(getMessages().snapshot(), m -> m.from() + " => " + m.payload.toDebugString(command, partitionKey))); } private UnfilteredPartitionIterators.MergeListener wrapMergeListener(UnfilteredPartitionIterators.MergeListener partitionListener, diff --git a/src/java/org/apache/cassandra/service/reads/DigestResolver.java b/src/java/org/apache/cassandra/service/reads/DigestResolver.java index 899baf9830f0..cf7ec315ca97 100644 --- a/src/java/org/apache/cassandra/service/reads/DigestResolver.java +++ b/src/java/org/apache/cassandra/service/reads/DigestResolver.java @@ -33,16 +33,15 @@ import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.ReplicaPlan; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.service.reads.repair.NoopReadRepair; -import org.apache.cassandra.service.reads.repair.ReadRepair; import org.apache.cassandra.utils.ByteBufferUtil; import static com.google.common.collect.Iterables.any; public class DigestResolver, P extends ReplicaPlan.ForRead> extends ResponseResolver { - private volatile MessageIn dataResponse; + private volatile Message dataResponse; public DigestResolver(ReadCommand command, ReplicaPlan.Shared replicaPlan, long queryStartNanoTime) { @@ -52,10 +51,10 @@ public DigestResolver(ReadCommand command, ReplicaPlan.Shared replicaPlan, } @Override - public void preprocess(MessageIn message) + public void preprocess(Message message) { super.preprocess(message); - Replica replica = replicaPlan().getReplicaFor(message.from); + Replica replica = replicaPlan().getReplicaFor(message.from()); if (dataResponse == null && !message.payload.isDigestResponse() && replica.isFull()) dataResponse = message; } @@ -66,18 +65,18 @@ public boolean hasTransientResponse() return hasTransientResponse(responses.snapshot()); } - private boolean hasTransientResponse(Collection> responses) + private boolean hasTransientResponse(Collection> responses) { return any(responses, msg -> !msg.payload.isDigestResponse() - && replicaPlan().getReplicaFor(msg.from).isTransient()); + && replicaPlan().getReplicaFor(msg.from()).isTransient()); } public PartitionIterator getData() { assert isDataPresent(); - Collection> responses = this.responses.snapshot(); + Collection> responses = this.responses.snapshot(); if (!hasTransientResponse(responses)) { @@ -92,9 +91,9 @@ public PartitionIterator getData() dataResolver.preprocess(dataResponse); // Reconcile with transient replicas - for (MessageIn response : responses) + for (Message response : responses) { - Replica replica = replicaPlan().getReplicaFor(response.from); + Replica replica = replicaPlan().getReplicaFor(response.from()); if (replica.isTransient()) dataResolver.preprocess(response); } @@ -109,9 +108,14 @@ public boolean responsesMatch() // validate digests against each other; return false immediately on mismatch. ByteBuffer digest = null; - for (MessageIn message : responses.snapshot()) + Collection> snapshot = responses.snapshot(); + if (snapshot.size() <= 1) + return true; + + // TODO: should also not calculate if only one full node + for (Message message : snapshot) { - if (replicaPlan().getReplicaFor(message.from).isTransient()) + if (replicaPlan().getReplicaFor(message.from()).isTransient()) continue; ByteBuffer newDigest = message.payload.digest(command); @@ -138,10 +142,10 @@ public DigestResolverDebugResult[] getDigestsByEndpoint() DigestResolverDebugResult[] ret = new DigestResolverDebugResult[responses.size()]; for (int i = 0; i < responses.size(); i++) { - MessageIn message = responses.get(i); + Message message = responses.get(i); ReadResponse response = message.payload; String digestHex = ByteBufferUtil.bytesToHex(response.digest(command)); - ret[i] = new DigestResolverDebugResult(message.from, digestHex, message.payload.isDigestResponse()); + ret[i] = new DigestResolverDebugResult(message.from(), digestHex, message.payload.isDigestResponse()); } return ret; } diff --git a/src/java/org/apache/cassandra/service/reads/ReadCallback.java b/src/java/org/apache/cassandra/service/reads/ReadCallback.java index 7a2385c6ca4b..2968dbce09b5 100644 --- a/src/java/org/apache/cassandra/service/reads/ReadCallback.java +++ b/src/java/org/apache/cassandra/service/reads/ReadCallback.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.service.reads; -import java.util.Collections; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; @@ -36,14 +35,15 @@ import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.locator.Endpoints; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IAsyncCallbackWithFailure; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.tracing.Tracing; -import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.SimpleCondition; -public class ReadCallback, P extends ReplicaPlan.ForRead> implements IAsyncCallbackWithFailure +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class ReadCallback, P extends ReplicaPlan.ForRead> implements RequestCallback { protected static final Logger logger = LoggerFactory.getLogger( ReadCallback.class ); @@ -98,7 +98,7 @@ public boolean await(long timePastStart, TimeUnit unit) public void awaitResults() throws ReadFailureException, ReadTimeoutException { - boolean signaled = await(command.getTimeout(), TimeUnit.MILLISECONDS); + boolean signaled = await(command.getTimeout(MILLISECONDS), TimeUnit.MILLISECONDS); boolean failed = failures > 0 && blockFor + failures > replicaPlan().contacts().size(); if (signaled && !failed) return; @@ -125,10 +125,10 @@ public int blockFor() return blockFor; } - public void response(MessageIn message) + public void onResponse(Message message) { resolver.preprocess(message); - int n = waitingFor(message.from) + int n = waitingFor(message.from()) ? recievedUpdater.incrementAndGet(this) : received; @@ -146,15 +146,14 @@ private boolean waitingFor(InetAddressAndPort from) public void response(ReadResponse result) { - MessageIn message = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(), - result, - Collections.emptyMap(), - MessagingService.Verb.INTERNAL_RESPONSE, - MessagingService.current_version); - response(message); + Verb kind = command.isRangeRequest() ? Verb.RANGE_RSP : Verb.READ_RSP; + Message message = Message.internalResponse(kind, result); + onResponse(message); } - public boolean isLatencyForSnitch() + + @Override + public boolean trackLatencyForSnitch() { return true; } @@ -171,4 +170,10 @@ public void onFailure(InetAddressAndPort from, RequestFailureReason failureReaso if (blockFor + n > replicaPlan().contacts().size()) condition.signalAll(); } + + @Override + public boolean invokeOnFailure() + { + return true; + } } diff --git a/src/java/org/apache/cassandra/service/reads/ResponseResolver.java b/src/java/org/apache/cassandra/service/reads/ResponseResolver.java index aaead8439fa6..8e15c1a9a018 100644 --- a/src/java/org/apache/cassandra/service/reads/ResponseResolver.java +++ b/src/java/org/apache/cassandra/service/reads/ResponseResolver.java @@ -24,7 +24,7 @@ import org.apache.cassandra.db.ReadResponse; import org.apache.cassandra.locator.Endpoints; import org.apache.cassandra.locator.ReplicaPlan; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.utils.concurrent.Accumulator; public abstract class ResponseResolver, P extends ReplicaPlan.ForRead> @@ -35,7 +35,7 @@ public abstract class ResponseResolver, P extends Replica protected final ReplicaPlan.Shared replicaPlan; // Accumulator gives us non-blocking thread-safety with optimal algorithmic constraints - protected final Accumulator> responses; + protected final Accumulator> responses; protected final long queryStartNanoTime; public ResponseResolver(ReadCommand command, ReplicaPlan.Shared replicaPlan, long queryStartNanoTime) @@ -53,9 +53,9 @@ protected P replicaPlan() public abstract boolean isDataPresent(); - public void preprocess(MessageIn message) + public void preprocess(Message message) { - if (replicaPlan().getReplicaFor(message.from).isTransient() && + if (replicaPlan().getReplicaFor(message.from()).isTransient() && message.payload.isDigestResponse()) throw new IllegalArgumentException("Digest response received from transient replica"); @@ -71,7 +71,7 @@ public void preprocess(MessageIn message) } } - public Accumulator> getMessages() + public Accumulator> getMessages() { return responses; } diff --git a/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java b/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java index 7b7c4d339fc1..8735ceddaca8 100644 --- a/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java +++ b/src/java/org/apache/cassandra/service/reads/ShortReadPartitionsProtection.java @@ -189,7 +189,7 @@ UnfilteredPartitionIterator executeReadCommand(ReadCommand cmd, ReplicaPlan.Shar { if (source.isTransient()) cmd = cmd.copyAsTransientQuery(source); - MessagingService.instance().sendRRWithFailure(cmd.createMessage(), source.endpoint(), handler); + MessagingService.instance().sendWithCallback(cmd.createMessage(false), source.endpoint(), handler); } // We don't call handler.get() because we want to preserve tombstones since we're still in the middle of merging node results. diff --git a/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java index 761ffb0233a6..79c124b5ef4d 100644 --- a/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java +++ b/src/java/org/apache/cassandra/service/reads/repair/AbstractReadRepair.java @@ -18,7 +18,6 @@ package org.apache.cassandra.service.reads.repair; -import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import com.google.common.base.Preconditions; @@ -40,15 +39,17 @@ import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaPlan; import org.apache.cassandra.metrics.ReadRepairMetrics; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessageFlag; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; import org.apache.cassandra.service.StorageProxy; import org.apache.cassandra.service.reads.DataResolver; import org.apache.cassandra.service.reads.DigestResolver; import org.apache.cassandra.service.reads.ReadCallback; import org.apache.cassandra.tracing.Tracing; +import static java.util.concurrent.TimeUnit.NANOSECONDS; + public abstract class AbstractReadRepair, P extends ReplicaPlan.ForRead> implements ReadRepair { @@ -113,12 +114,9 @@ void sendReadCommand(Replica to, ReadCallback readCallback, boolean speculative) else type = to.isFull() ? "full" : "transient"; Tracing.trace("Enqueuing {} data read to {}", type, to); } - MessageOut message = command.createMessage(); // if enabled, request additional info about repaired data from any full replicas - if (command.isTrackingRepairedStatus() && to.isFull()) - message = message.withParameter(ParameterType.TRACK_REPAIRED_DATA, MessagingService.ONE_BYTE); - - MessagingService.instance().sendRRWithFailure(message, to.endpoint(), readCallback); + Message message = command.createMessage(command.isTrackingRepairedStatus() && to.isFull()); + MessagingService.instance().sendWithCallback(message, to.endpoint(), readCallback); } abstract Meter getRepairMeter(); @@ -160,7 +158,7 @@ private boolean shouldSpeculate() ConsistencyLevel speculativeCL = consistency.isDatacenterLocal() ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM; return consistency != ConsistencyLevel.EACH_QUORUM && consistency.satisfies(speculativeCL, cfs.keyspace) - && cfs.sampleReadLatencyNanos <= TimeUnit.MILLISECONDS.toNanos(command.getTimeout()); + && cfs.sampleReadLatencyNanos <= command.getTimeout(NANOSECONDS); } public void maybeSendAdditionalReads() @@ -171,7 +169,7 @@ public void maybeSendAdditionalReads() if (repair == null) return; - if (shouldSpeculate() && !repair.readCallback.await(cfs.sampleReadLatencyNanos, TimeUnit.NANOSECONDS)) + if (shouldSpeculate() && !repair.readCallback.await(cfs.sampleReadLatencyNanos, NANOSECONDS)) { Replica uncontacted = replicaPlan().firstUncontactedCandidate(Predicates.alwaysTrue()); if (uncontacted == null) diff --git a/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java b/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java index 624c78f310c7..220ada5a5ab7 100644 --- a/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java +++ b/src/java/org/apache/cassandra/service/reads/repair/BlockingPartitionRepair.java @@ -45,15 +45,16 @@ import org.apache.cassandra.locator.Replicas; import org.apache.cassandra.locator.InOurDcTester; import org.apache.cassandra.metrics.ReadRepairMetrics; -import org.apache.cassandra.net.IAsyncCallback; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.tracing.Tracing; +import static org.apache.cassandra.net.Verb.*; + public class BlockingPartitionRepair, P extends ReplicaPlan.ForRead> - extends AbstractFuture implements IAsyncCallback + extends AbstractFuture implements RequestCallback { private final DecoratedKey key; private final P replicaPlan; @@ -111,15 +112,9 @@ void ack(InetAddressAndPort from) } @Override - public void response(MessageIn msg) - { - ack(msg.from); - } - - @Override - public boolean isLatencyForSnitch() + public void onResponse(Message msg) { - return false; + ack(msg.from()); } private static PartitionUpdate extractUpdate(Mutation mutation) @@ -138,9 +133,9 @@ private PartitionUpdate mergeUnackedUpdates() } @VisibleForTesting - protected void sendRR(MessageOut message, InetAddressAndPort endpoint) + protected void sendRR(Message message, InetAddressAndPort endpoint) { - MessagingService.instance().sendRR(message, endpoint, this); + MessagingService.instance().sendWithCallback(message, endpoint, this); } public void sendInitialRepairs() @@ -157,7 +152,7 @@ public void sendInitialRepairs() Tracing.trace("Sending read-repair-mutation to {}", destination); // use a separate verb here to avoid writing hints on timeouts - sendRR(mutation.createMessage(MessagingService.Verb.READ_REPAIR), destination.endpoint()); + sendRR(Message.out(READ_REPAIR_REQ, mutation), destination.endpoint()); ColumnFamilyStore.metricsFor(tableId).readRepairRequests.mark(); if (!shouldBlockOn.test(destination.endpoint())) @@ -214,7 +209,7 @@ public void maybeSendAdditionalWrites(long timeout, TimeUnit timeoutUnit) for (Replica replica : newCandidates) { - int versionIdx = msgVersionIdx(MessagingService.instance().getVersion(replica.endpoint())); + int versionIdx = msgVersionIdx(MessagingService.instance().versions.get(replica.endpoint())); Mutation mutation = versionedMutations[versionIdx]; @@ -232,7 +227,7 @@ public void maybeSendAdditionalWrites(long timeout, TimeUnit timeoutUnit) } Tracing.trace("Sending speculative read-repair-mutation to {}", replica); - sendRR(mutation.createMessage(MessagingService.Verb.READ_REPAIR), replica.endpoint()); + sendRR(Message.out(READ_REPAIR_REQ, mutation), replica.endpoint()); ReadRepairDiagnostics.speculatedWrite(this, replica.endpoint(), mutation); } } diff --git a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java index 8016b8944a41..ef624d69e144 100644 --- a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java +++ b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepair.java @@ -39,6 +39,9 @@ import org.apache.cassandra.metrics.ReadRepairMetrics; import org.apache.cassandra.tracing.Tracing; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; + /** * 'Classic' read repair. Doesn't allow the client read to return until * updates have been written to nodes needing correction. Breaks write @@ -84,7 +87,7 @@ public void awaitWrites() boolean timedOut = false; for (BlockingPartitionRepair repair: repairs) { - if (!repair.awaitRepairs(DatabaseDescriptor.getWriteRpcTimeout(), TimeUnit.MILLISECONDS)) + if (!repair.awaitRepairs(DatabaseDescriptor.getWriteRpcTimeout(NANOSECONDS), NANOSECONDS)) { timedOut = true; } diff --git a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java index ceb176569044..3a4978e7cce0 100644 --- a/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java +++ b/src/java/org/apache/cassandra/service/reads/repair/BlockingReadRepairs.java @@ -54,7 +54,7 @@ public static Mutation createRepairMutation(PartitionUpdate update, ConsistencyL Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName()); TableMetadata metadata = update.metadata(); - int messagingVersion = MessagingService.instance().getVersion(destination); + int messagingVersion = MessagingService.instance().versions.get(destination); int mutationSize = (int) Mutation.serializer.serializedSize(mutation, messagingVersion); int maxMutationSize = DatabaseDescriptor.getMaxMutationSize(); diff --git a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java index b19280377823..5f2163f5410b 100644 --- a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java +++ b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java @@ -19,97 +19,40 @@ package org.apache.cassandra.streaming; import java.io.IOException; -import java.util.concurrent.TimeUnit; - -import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.Uninterruptibles; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.WriteBufferWaterMark; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.net.async.NettyFactory; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; -import org.apache.cassandra.net.async.OutboundConnectionParams; +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.Future; +import org.apache.cassandra.net.ConnectionCategory; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.OutboundConnectionInitiator.Result; +import org.apache.cassandra.net.OutboundConnectionInitiator.Result.StreamingSuccess; +import org.apache.cassandra.net.OutboundConnectionSettings; + +import static org.apache.cassandra.net.OutboundConnectionInitiator.initiateStreaming; public class DefaultConnectionFactory implements StreamConnectionFactory { - private static final Logger logger = LoggerFactory.getLogger(DefaultConnectionFactory.class); - - private static final int DEFAULT_CHANNEL_BUFFER_SIZE = 1 << 22; - - @VisibleForTesting - public static long MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(30); @VisibleForTesting public static int MAX_CONNECT_ATTEMPTS = 3; @Override - public Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion) throws IOException - { - ServerEncryptionOptions encryptionOptions = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); - - if (encryptionOptions.internode_encryption == ServerEncryptionOptions.InternodeEncryption.none) - encryptionOptions = null; - - return createConnection(connectionId, protocolVersion, encryptionOptions); - } - - protected Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion, @Nullable ServerEncryptionOptions encryptionOptions) throws IOException + public Channel createConnection(OutboundConnectionSettings template, int messagingVersion) throws IOException { - // this is the amount of data to allow in memory before netty sets the channel writablility flag to false - int channelBufferSize = DEFAULT_CHANNEL_BUFFER_SIZE; - WriteBufferWaterMark waterMark = new WriteBufferWaterMark(channelBufferSize >> 2, channelBufferSize); - - int sendBufferSize = DatabaseDescriptor.getInternodeSendBufferSize() > 0 - ? DatabaseDescriptor.getInternodeSendBufferSize() - : OutboundConnectionParams.DEFAULT_SEND_BUFFER_SIZE; - - int tcpConnectTimeout = DatabaseDescriptor.getInternodeTcpConnectTimeoutInMS(); - int tcpUserTimeout = DatabaseDescriptor.getInternodeTcpUserTimeoutInMS(); + EventLoop eventLoop = MessagingService.instance().socketFactory.outboundStreamingGroup().next(); - OutboundConnectionParams params = OutboundConnectionParams.builder() - .connectionId(connectionId) - .encryptionOptions(encryptionOptions) - .mode(NettyFactory.Mode.STREAMING) - .protocolVersion(protocolVersion) - .sendBufferSize(sendBufferSize) - .tcpConnectTimeoutInMS(tcpConnectTimeout) - .tcpUserTimeoutInMS(tcpUserTimeout) - .waterMark(waterMark) - .build(); - - Bootstrap bootstrap = NettyFactory.instance.createOutboundBootstrap(params); - - int connectionAttemptCount = 0; - long now = System.nanoTime(); - final long end = now + MAX_WAIT_TIME_NANOS; - final Channel channel; + int attempts = 0; while (true) { - ChannelFuture channelFuture = bootstrap.connect(); - channelFuture.awaitUninterruptibly(end - now, TimeUnit.MILLISECONDS); - if (channelFuture.isSuccess()) - { - channel = channelFuture.channel(); - break; - } + Future> result = initiateStreaming(eventLoop, template.withDefaults(ConnectionCategory.STREAMING), messagingVersion); + result.awaitUninterruptibly(); // initiate has its own timeout, so this is "guaranteed" to return relatively promptly + if (result.isSuccess()) + return result.getNow().success().channel; - connectionAttemptCount++; - now = System.nanoTime(); - if (connectionAttemptCount == MAX_CONNECT_ATTEMPTS || end - now <= 0) - throw new IOException("failed to connect to " + connectionId + " for streaming data", channelFuture.cause()); - - long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, connectionAttemptCount); - logger.warn("Failed attempt {} to connect to {}. Retrying in {} ms.", connectionAttemptCount, connectionId, waitms); - Uninterruptibles.sleepUninterruptibly(waitms, TimeUnit.MILLISECONDS); + if (++attempts == MAX_CONNECT_ATTEMPTS) + throw new IOException("failed to connect to " + template.to + " for streaming data", result.cause()); } - - return channel; } } diff --git a/src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java b/src/java/org/apache/cassandra/streaming/ReplicationDoneVerbHandler.java similarity index 67% rename from src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java rename to src/java/org/apache/cassandra/streaming/ReplicationDoneVerbHandler.java index ce8a921ad10f..7d73b114181d 100644 --- a/src/java/org/apache/cassandra/streaming/ReplicationFinishedVerbHandler.java +++ b/src/java/org/apache/cassandra/streaming/ReplicationDoneVerbHandler.java @@ -21,21 +21,20 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.StorageService; -public class ReplicationFinishedVerbHandler implements IVerbHandler +public class ReplicationDoneVerbHandler implements IVerbHandler { - private static final Logger logger = LoggerFactory.getLogger(ReplicationFinishedVerbHandler.class); + public static ReplicationDoneVerbHandler instance = new ReplicationDoneVerbHandler(); - public void doVerb(MessageIn msg, int id) + private static final Logger logger = LoggerFactory.getLogger(ReplicationDoneVerbHandler.class); + + public void doVerb(Message msg) { - StorageService.instance.confirmReplication(msg.from); - MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE); - if (logger.isDebugEnabled()) - logger.debug("Replying to {}@{}", id, msg.from); - MessagingService.instance().sendReply(response, id, msg.from); + StorageService.instance.confirmReplication(msg.from()); + logger.debug("Replying to {}@{}", msg.id(), msg.from()); + MessagingService.instance().send(msg.emptyResponse(), msg.from()); } } diff --git a/src/java/org/apache/cassandra/streaming/SessionSummary.java b/src/java/org/apache/cassandra/streaming/SessionSummary.java index cf63a572cdb9..5b168a0841ed 100644 --- a/src/java/org/apache/cassandra/streaming/SessionSummary.java +++ b/src/java/org/apache/cassandra/streaming/SessionSummary.java @@ -28,7 +28,9 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; +import org.apache.cassandra.locator.InetAddressAndPort.Serializer; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; public class SessionSummary { @@ -80,8 +82,8 @@ public int hashCode() { public void serialize(SessionSummary summary, DataOutputPlus out, int version) throws IOException { - CompactEndpointSerializationHelper.instance.serialize(summary.coordinator, out, version); - CompactEndpointSerializationHelper.instance.serialize(summary.peer, out, version); + inetAddressAndPortSerializer.serialize(summary.coordinator, out, version); + inetAddressAndPortSerializer.serialize(summary.peer, out, version); out.writeInt(summary.receivingSummaries.size()); for (StreamSummary streamSummary: summary.receivingSummaries) @@ -98,8 +100,8 @@ public void serialize(SessionSummary summary, DataOutputPlus out, int version) t public SessionSummary deserialize(DataInputPlus in, int version) throws IOException { - InetAddressAndPort coordinator = CompactEndpointSerializationHelper.instance.deserialize(in, version); - InetAddressAndPort peer = CompactEndpointSerializationHelper.instance.deserialize(in, version); + InetAddressAndPort coordinator = inetAddressAndPortSerializer.deserialize(in, version); + InetAddressAndPort peer = inetAddressAndPortSerializer.deserialize(in, version); int numRcvd = in.readInt(); List receivingSummaries = new ArrayList<>(numRcvd); @@ -121,8 +123,8 @@ public SessionSummary deserialize(DataInputPlus in, int version) throws IOExcept public long serializedSize(SessionSummary summary, int version) { long size = 0; - size += CompactEndpointSerializationHelper.instance.serializedSize(summary.coordinator, version); - size += CompactEndpointSerializationHelper.instance.serializedSize(summary.peer, version); + size += inetAddressAndPortSerializer.serializedSize(summary.coordinator, version); + size += inetAddressAndPortSerializer.serializedSize(summary.peer, version); size += TypeSizes.sizeof(summary.receivingSummaries.size()); for (StreamSummary streamSummary: summary.receivingSummaries) diff --git a/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java b/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java index 4cfe41e01251..95208e400bd6 100644 --- a/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java +++ b/src/java/org/apache/cassandra/streaming/StreamConnectionFactory.java @@ -21,9 +21,9 @@ import java.io.IOException; import io.netty.channel.Channel; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; +import org.apache.cassandra.net.OutboundConnectionSettings; public interface StreamConnectionFactory { - Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion) throws IOException; + Channel createConnection(OutboundConnectionSettings template, int messagingVersion) throws IOException; } diff --git a/src/java/org/apache/cassandra/streaming/StreamRequest.java b/src/java/org/apache/cassandra/streaming/StreamRequest.java index f37268f200c4..0c8542fcfaa8 100644 --- a/src/java/org/apache/cassandra/streaming/StreamRequest.java +++ b/src/java/org/apache/cassandra/streaming/StreamRequest.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.IVersionedSerializer; @@ -32,8 +33,8 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.RangesAtEndpoint; import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; -import org.apache.cassandra.net.MessagingService; + +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; public class StreamRequest { @@ -67,7 +68,7 @@ public void serialize(StreamRequest request, DataOutputPlus out, int version) th out.writeUTF(request.keyspace); out.writeInt(request.columnFamilies.size()); - CompactEndpointSerializationHelper.streamingInstance.serialize(request.full.endpoint(), out, version); + inetAddressAndPortSerializer.serialize(request.full.endpoint(), out, version); serializeReplicas(request.full, out, version); serializeReplicas(request.transientReplicas, out, version); for (String cf : request.columnFamilies) @@ -80,7 +81,7 @@ private void serializeReplicas(RangesAtEndpoint replicas, DataOutputPlus out, in for (Replica replica : replicas) { - MessagingService.validatePartitioner(replica.range()); + IPartitioner.validate(replica.range()); Token.serializer.serialize(replica.range().left, out, version); Token.serializer.serialize(replica.range().right, out, version); } @@ -90,7 +91,7 @@ public StreamRequest deserialize(DataInputPlus in, int version) throws IOExcepti { String keyspace = in.readUTF(); int cfCount = in.readInt(); - InetAddressAndPort endpoint = CompactEndpointSerializationHelper.streamingInstance.deserialize(in, version); + InetAddressAndPort endpoint = inetAddressAndPortSerializer.deserialize(in, version); RangesAtEndpoint full = deserializeReplicas(in, version, endpoint, true); RangesAtEndpoint transientReplicas = deserializeReplicas(in, version, endpoint, false); @@ -110,8 +111,8 @@ RangesAtEndpoint deserializeReplicas(DataInputPlus in, int version, InetAddressA //TODO, super need to review the usage of streaming vs not streaming endpoint serialization helper //to make sure I'm not using the wrong one some of the time, like do repair messages use the //streaming version? - Token left = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version); - Token right = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version); + Token left = Token.serializer.deserialize(in, IPartitioner.global(), version); + Token right = Token.serializer.deserialize(in, IPartitioner.global(), version); replicas.add(new Replica(endpoint, new Range<>(left, right), isFull)); } return replicas.build(); @@ -121,7 +122,7 @@ public long serializedSize(StreamRequest request, int version) { int size = TypeSizes.sizeof(request.keyspace); size += TypeSizes.sizeof(request.columnFamilies.size()); - size += CompactEndpointSerializationHelper.streamingInstance.serializedSize(request.full.endpoint(), version); + size += inetAddressAndPortSerializer.serializedSize(request.full.endpoint(), version); size += replicasSerializedSize(request.transientReplicas, version); size += replicasSerializedSize(request.full, version); for (String cf : request.columnFamilies) diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java index 08a1b078027a..95d3755d6c43 100644 --- a/src/java/org/apache/cassandra/streaming/StreamSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamSession.java @@ -21,7 +21,6 @@ import java.util.*; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.*; @@ -43,8 +42,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.metrics.StreamingMetrics; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; +import org.apache.cassandra.net.OutboundConnectionSettings; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.streaming.async.NettyStreamingMessageSender; import org.apache.cassandra.streaming.messages.*; @@ -52,6 +50,7 @@ import org.apache.cassandra.utils.JVMStabilityInspector; import static com.google.common.collect.Iterables.all; +import static org.apache.cassandra.net.MessagingService.current_version; /** * Handles the streaming a one or more streams to and from a specific remote node. @@ -136,11 +135,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber * Each {@code StreamSession} is identified by this InetAddressAndPort which is broadcast address of the node streaming. */ public final InetAddressAndPort peer; - - /** - * Preferred IP Address/Port of the peer; this is the address that will be connect to. Can be the same as {@linkplain #peer}. - */ - private final InetAddressAndPort preferredPeerInetAddressAndPort; + private final OutboundConnectionSettings template; private final int index; @@ -184,30 +179,25 @@ public enum State public StreamSession(StreamOperation streamOperation, InetAddressAndPort peer, StreamConnectionFactory factory, int index, UUID pendingRepair, PreviewKind previewKind) { - this(streamOperation, peer, factory, index, pendingRepair, previewKind, MessagingService.instance()::getPreferredRemoteAddr); + this(streamOperation, new OutboundConnectionSettings(peer), factory, index, pendingRepair, previewKind); } - - @VisibleForTesting - public StreamSession(StreamOperation streamOperation, InetAddressAndPort peer, StreamConnectionFactory factory, - int index, UUID pendingRepair, PreviewKind previewKind, - Function preferredIpMapper) + /** + * Create new streaming session with the peer. + */ + public StreamSession(StreamOperation streamOperation, OutboundConnectionSettings template, StreamConnectionFactory factory, + int index, UUID pendingRepair, PreviewKind previewKind) { this.streamOperation = streamOperation; - this.peer = peer; + this.peer = template.to; + this.template = template; this.index = index; - InetAddressAndPort preferredPeerEndpoint = preferredIpMapper.apply(peer); - this.preferredPeerInetAddressAndPort = (preferredPeerEndpoint == null) ? peer : preferredPeerEndpoint; - - OutboundConnectionIdentifier id = OutboundConnectionIdentifier.stream(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getJustLocalAddress(), 0), - preferredPeerInetAddressAndPort); - this.messageSender = new NettyStreamingMessageSender(this, id, factory, StreamMessage.CURRENT_VERSION, previewKind.isPreview()); - this.metrics = StreamingMetrics.get(preferredPeerInetAddressAndPort); + this.messageSender = new NettyStreamingMessageSender(this, template, factory, current_version, previewKind.isPreview()); + this.metrics = StreamingMetrics.get(peer); this.pendingRepair = pendingRepair; this.previewKind = previewKind; - logger.debug("Creating stream session peer={} preferredPeerInetAddressAndPort={}", peer, - preferredPeerInetAddressAndPort); + logger.debug("Creating stream session to {}", template); } public UUID planId() @@ -286,7 +276,7 @@ public void start() { logger.info("[Stream #{}] Starting streaming to {}{}", planId(), peer, - peer.equals(preferredPeerInetAddressAndPort) ? "" : " through " + preferredPeerInetAddressAndPort); + template.connectTo == null ? "" : " through " + template.connectTo); messageSender.initialize(); onInitializationComplete(); } @@ -545,7 +535,7 @@ private void logError(Throwable e) logger.error("[Stream #{}] Did not receive response from peer {}{} for {} secs. Is peer down? " + "If not, maybe try increasing streaming_keep_alive_period_in_secs.", planId(), peer.getHostAddress(true), - peer.equals(preferredPeerInetAddressAndPort) ? "" : " through " + preferredPeerInetAddressAndPort.getHostAddress(true), + template.connectTo == null ? "" : " through " + template.connectTo.getHostAddress(true), 2 * DatabaseDescriptor.getStreamingKeepAlivePeriod(), e); } @@ -553,7 +543,7 @@ private void logError(Throwable e) { logger.error("[Stream #{}] Streaming error occurred on session with peer {}{}", planId(), peer.getHostAddress(true), - peer.equals(preferredPeerInetAddressAndPort) ? "" : " through " + preferredPeerInetAddressAndPort.getHostAddress(true), + template.connectTo == null ? "" : " through " + template.connectTo.getHostAddress(true), e); } } @@ -707,7 +697,8 @@ public SessionInfo getSessionInfo() List transferSummaries = Lists.newArrayList(); for (StreamTask transfer : transfers.values()) transferSummaries.add(transfer.getSummary()); - return new SessionInfo(peer, index, preferredPeerInetAddressAndPort, receivingSummaries, transferSummaries, state); + // TODO: the connectTo treatment here is peculiar, and needs thinking about - since the connection factory can change it + return new SessionInfo(peer, index, template.connectTo == null ? peer : template.connectTo, receivingSummaries, transferSummaries, state); } public synchronized void taskCompleted(StreamReceiveTask completedTask) diff --git a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java index accf5548c4a0..96e76267ec49 100644 --- a/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java +++ b/src/java/org/apache/cassandra/streaming/StreamingMessageSender.java @@ -20,7 +20,9 @@ import java.io.IOException; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.streaming.messages.StreamMessage; public interface StreamingMessageSender @@ -29,8 +31,6 @@ public interface StreamingMessageSender void sendMessage(StreamMessage message) throws IOException; - OutboundConnectionIdentifier getConnectionId(); - boolean connected(); void close(); diff --git a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java index 8511a87975f5..1314e1d9d8a1 100644 --- a/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java +++ b/src/java/org/apache/cassandra/streaming/async/NettyStreamingMessageSender.java @@ -49,9 +49,9 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.util.DataOutputBufferFixed; import org.apache.cassandra.io.util.DataOutputStreamPlus; -import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; -import org.apache.cassandra.net.async.NettyFactory; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; +import org.apache.cassandra.net.AsyncChannelPromise; +import org.apache.cassandra.net.OutboundConnectionSettings; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; import org.apache.cassandra.streaming.StreamConnectionFactory; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.StreamingMessageSender; @@ -85,13 +85,15 @@ public class NettyStreamingMessageSender implements StreamingMessageSender private static final int DEFAULT_MAX_PARALLEL_TRANSFERS = FBUtilities.getAvailableProcessors(); private static final int MAX_PARALLEL_TRANSFERS = Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "streaming.session.parallelTransfers", Integer.toString(DEFAULT_MAX_PARALLEL_TRANSFERS))); + private static final long DEFAULT_CLOSE_WAIT_IN_MILLIS = TimeUnit.MINUTES.toMillis(5); + // a simple mechansim for allowing a degree of fairnes across multiple sessions private static final Semaphore fileTransferSemaphore = new Semaphore(DEFAULT_MAX_PARALLEL_TRANSFERS, true); private final StreamSession session; private final boolean isPreview; - private final int protocolVersion; - private final OutboundConnectionIdentifier connectionId; + private final int streamingVersion; + private final OutboundConnectionSettings template; private final StreamConnectionFactory factory; private volatile boolean closed; @@ -120,11 +122,11 @@ public class NettyStreamingMessageSender implements StreamingMessageSender @VisibleForTesting static final AttributeKey TRANSFERRING_FILE_ATTR = AttributeKey.valueOf("transferringFile"); - public NettyStreamingMessageSender(StreamSession session, OutboundConnectionIdentifier connectionId, StreamConnectionFactory factory, int protocolVersion, boolean isPreview) + public NettyStreamingMessageSender(StreamSession session, OutboundConnectionSettings template, StreamConnectionFactory factory, int streamingVersion, boolean isPreview) { this.session = session; - this.protocolVersion = protocolVersion; - this.connectionId = connectionId; + this.streamingVersion = streamingVersion; + this.template = template; this.factory = factory; this.isPreview = isPreview; @@ -181,9 +183,9 @@ private void scheduleKeepAliveTask(Channel channel) private Channel createChannel() throws IOException { - Channel channel = factory.createConnection(connectionId, protocolVersion); + Channel channel = factory.createConnection(template, streamingVersion); ChannelPipeline pipeline = channel.pipeline(); - pipeline.addLast(NettyFactory.instance.streamingGroup, NettyFactory.INBOUND_STREAM_HANDLER_NAME, new StreamingInboundHandler(connectionId.remote(), protocolVersion, session)); + pipeline.addLast("stream", new StreamingInboundHandler(template.to, streamingVersion, session)); channel.attr(TRANSFERRING_FILE_ATTR).set(Boolean.FALSE); logger.debug("Creating channel id {} local {} remote {}", channel.id(), channel.localAddress(), channel.remoteAddress()); return channel; @@ -238,7 +240,7 @@ private void sendControlMessage(Channel channel, StreamMessage message, GenericF logger.debug("{} Sending {}", createLogTag(session, channel), message); // we anticipate that the control messages are rather small, so allocating a ByteBuf shouldn't blow out of memory. - long messageSize = StreamMessage.serializedSize(message, protocolVersion); + long messageSize = StreamMessage.serializedSize(message, streamingVersion); if (messageSize > 1 << 30) { throw new IllegalStateException(String.format("%s something is seriously wrong with the calculated stream control message's size: %d bytes, type is %s", @@ -250,12 +252,11 @@ private void sendControlMessage(Channel channel, StreamMessage message, GenericF ByteBuffer nioBuf = buf.nioBuffer(0, (int) messageSize); @SuppressWarnings("resource") DataOutputBufferFixed out = new DataOutputBufferFixed(nioBuf); - StreamMessage.serialize(message, out, protocolVersion, session); + StreamMessage.serialize(message, out, streamingVersion, session); assert nioBuf.position() == nioBuf.limit(); buf.writerIndex(nioBuf.position()); - ChannelFuture channelFuture = channel.writeAndFlush(buf); - channelFuture.addListener(future -> listener.operationComplete(future)); + AsyncChannelPromise.writeAndFlush(channel, buf, listener); } /** @@ -275,7 +276,7 @@ java.util.concurrent.Future onControlMessageComplete(Future future, StreamMes Channel channel = channelFuture.channel(); logger.error("{} failed to send a stream message/data to peer {}: msg = {}", - createLogTag(session, channel), connectionId, msg, future.cause()); + createLogTag(session, channel), template.to, msg, future.cause()); // StreamSession will invoke close(), but we have to mark this sender as closed so the session doesn't try // to send any failure messages @@ -322,10 +323,9 @@ public void run() throw new IllegalStateException("channel's transferring state is currently set to true. refusing to start new stream"); // close the DataOutputStreamPlus as we're done with it - but don't close the channel - try (DataOutputStreamPlus outPlus = ByteBufDataOutputStreamPlus.create(session, channel, 1 << 20)) + try (DataOutputStreamPlus outPlus = new AsyncStreamingOutputPlus(channel)) { - StreamMessage.serialize(msg, outPlus, protocolVersion, session); - channel.flush(); + StreamMessage.serialize(msg, outPlus, streamingVersion, session); } finally { @@ -393,6 +393,18 @@ private Channel getOrCreateChannel() } } + private void onError(Throwable t) + { + try + { + session.onError(t).get(DEFAULT_CLOSE_WAIT_IN_MILLIS, TimeUnit.MILLISECONDS); + } + catch (Exception e) + { + // nop - let the Throwable param be the main failure point here, and let session handle it + } + } + /** * For testing purposes */ @@ -477,7 +489,7 @@ private void keepAliveListener(Future future) /** * For testing purposes only. */ - void setClosed() + public void setClosed() { closed = true; } @@ -495,7 +507,7 @@ int semaphoreAvailablePermits() @Override public boolean connected() { - return !closed; + return !closed && (controlMessageChannel == null || controlMessageChannel.isOpen()); } @Override @@ -503,7 +515,7 @@ public void close() { closed = true; if (logger.isDebugEnabled()) - logger.debug("{} Closing stream connection channels on {}", createLogTag(session, null), connectionId); + logger.debug("{} Closing stream connection channels on {}", createLogTag(session, null), template.to); for (ScheduledFuture future : channelKeepAlives) future.cancel(false); channelKeepAlives.clear(); @@ -518,10 +530,4 @@ public void close() if (controlMessageChannel != null) controlMessageChannel.close(); } - - @Override - public OutboundConnectionIdentifier getConnectionId() - { - return connectionId; - } } diff --git a/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java b/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java index ca15b78dc1bc..1d834097d9a9 100644 --- a/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java +++ b/src/java/org/apache/cassandra/streaming/async/StreamCompressionSerializer.java @@ -27,6 +27,9 @@ import net.jpountz.lz4.LZ4Compressor; import net.jpountz.lz4.LZ4FastDecompressor; import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; + +import static org.apache.cassandra.net.MessagingService.current_version; /** * A serialiazer for stream compressed files (see package-level documentation). Much like a typical compressed @@ -51,29 +54,20 @@ public StreamCompressionSerializer(ByteBufAllocator allocator) */ private static final int HEADER_LENGTH = 8; - /** - * @return A buffer with decompressed data. - */ - public ByteBuf serialize(LZ4Compressor compressor, ByteBuffer in, int version) + public static AsyncStreamingOutputPlus.Write serialize(LZ4Compressor compressor, ByteBuffer in, int version) { - final int uncompressedLength = in.remaining(); - int maxLength = compressor.maxCompressedLength(uncompressedLength); - ByteBuf out = allocator.directBuffer(maxLength); - try - { - ByteBuffer compressedNioBuffer = out.nioBuffer(HEADER_LENGTH, maxLength - HEADER_LENGTH); - compressor.compress(in, compressedNioBuffer); - final int compressedLength = compressedNioBuffer.position(); - out.setInt(0, compressedLength); - out.setInt(4, uncompressedLength); - out.writerIndex(HEADER_LENGTH + compressedLength); - } - catch (Exception e) - { - if (out != null) - out.release(); - } - return out; + assert version == current_version; + return bufferSupplier -> { + int uncompressedLength = in.remaining(); + int maxLength = compressor.maxCompressedLength(uncompressedLength); + ByteBuffer out = bufferSupplier.get(maxLength); + out.position(HEADER_LENGTH); + compressor.compress(in, out); + int compressedLength = out.position() - HEADER_LENGTH; + out.putInt(0, compressedLength); + out.putInt(4, uncompressedLength); + out.flip(); + }; } /** diff --git a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java index 7c10ef96efdf..79f30f3336ad 100644 --- a/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java +++ b/src/java/org/apache/cassandra/streaming/async/StreamingInboundHandler.java @@ -37,16 +37,17 @@ import io.netty.util.ReferenceCountUtil; import io.netty.util.concurrent.FastThreadLocalThread; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus.InputTimeoutException; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamReceiveException; import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamSession; -import org.apache.cassandra.streaming.messages.StreamMessageHeader; import org.apache.cassandra.streaming.messages.IncomingStreamMessage; import org.apache.cassandra.streaming.messages.KeepAliveMessage; import org.apache.cassandra.streaming.messages.StreamInitMessage; import org.apache.cassandra.streaming.messages.StreamMessage; +import org.apache.cassandra.streaming.messages.StreamMessageHeader; import org.apache.cassandra.utils.JVMStabilityInspector; import static org.apache.cassandra.streaming.async.NettyStreamingMessageSender.createLogTag; @@ -59,10 +60,7 @@ public class StreamingInboundHandler extends ChannelInboundHandlerAdapter { private static final Logger logger = LoggerFactory.getLogger(StreamingInboundHandler.class); - static final Function DEFAULT_SESSION_PROVIDER = sid -> StreamManager.instance.findSession(sid.from, sid.planId, sid.sessionIndex); - - private static final int AUTO_READ_LOW_WATER_MARK = 1 << 15; - private static final int AUTO_READ_HIGH_WATER_MARK = 1 << 20; + private static final Function DEFAULT_SESSION_PROVIDER = sid -> StreamManager.instance.findSession(sid.from, sid.planId, sid.sessionIndex); private final InetAddressAndPort remoteAddress; private final int protocolVersion; @@ -74,10 +72,10 @@ public class StreamingInboundHandler extends ChannelInboundHandlerAdapter * structure, and then consumed. *

* For thread safety, this structure's resources are released on the consuming thread - * (via {@link RebufferingByteBufDataInputPlus#close()}, - * but the producing side calls {@link RebufferingByteBufDataInputPlus#markClose()} to notify the input that is should close. + * (via {@link AsyncStreamingInputPlus#close()}, + * but the producing side calls {@link AsyncStreamingInputPlus#requestClosure()} to notify the input that is should close. */ - private RebufferingByteBufDataInputPlus buffers; + private AsyncStreamingInputPlus buffers; private volatile boolean closed; @@ -92,7 +90,7 @@ public StreamingInboundHandler(InetAddressAndPort remoteAddress, int protocolVer @SuppressWarnings("resource") public void handlerAdded(ChannelHandlerContext ctx) { - buffers = new RebufferingByteBufDataInputPlus(AUTO_READ_LOW_WATER_MARK, AUTO_READ_HIGH_WATER_MARK, ctx.channel().config()); + buffers = new AsyncStreamingInputPlus(ctx.channel()); Thread blockingIOThread = new FastThreadLocalThread(new StreamDeserializingTask(DEFAULT_SESSION_PROVIDER, session, ctx.channel()), String.format("Stream-Deserializer-%s-%s", remoteAddress.toString(), ctx.channel().id())); blockingIOThread.setDaemon(true); @@ -102,9 +100,7 @@ public void handlerAdded(ChannelHandlerContext ctx) @Override public void channelRead(ChannelHandlerContext ctx, Object message) { - if (!closed && message instanceof ByteBuf) - buffers.append((ByteBuf) message); - else + if (closed || !(message instanceof ByteBuf) || !buffers.append((ByteBuf) message)) ReferenceCountUtil.release(message); } @@ -118,7 +114,7 @@ public void channelInactive(ChannelHandlerContext ctx) void close() { closed = true; - buffers.markClose(); + buffers.requestClosure(); } @Override @@ -134,7 +130,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) /** * For testing only!! */ - void setPendingBuffers(RebufferingByteBufDataInputPlus bufChannel) + void setPendingBuffers(AsyncStreamingInputPlus bufChannel) { this.buffers = bufChannel; } @@ -164,9 +160,11 @@ public void run() { while (true) { + buffers.maybeIssueRead(); + // do a check of available bytes and possibly sleep some amount of time (then continue). // this way we can break out of run() sanely or we end up blocking indefintely in StreamMessage.deserialize() - while (buffers.available() == 0) + while (buffers.isEmpty()) { if (closed) return; @@ -190,10 +188,11 @@ public void run() if (logger.isDebugEnabled()) logger.debug("{} Received {}", createLogTag(session, channel), message); + session.messageReceived(message); } } - catch (EOFException eof) + catch (InputTimeoutException | EOFException e) { // ignore } @@ -223,7 +222,7 @@ else if (t instanceof StreamReceiveException) } } - StreamSession deriveSession(StreamMessage message) throws IOException + StreamSession deriveSession(StreamMessage message) { StreamSession streamSession = null; // StreamInitMessage starts a new channel, and IncomingStreamMessage potentially, as well. diff --git a/src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.java b/src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.java deleted file mode 100644 index 3f1b22b53ce0..000000000000 --- a/src/java/org/apache/cassandra/streaming/compress/ByteBufCompressionDataOutputStreamPlus.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.streaming.compress; - -import java.io.IOException; -import java.nio.ByteBuffer; - -import io.netty.buffer.ByteBuf; -import net.jpountz.lz4.LZ4Compressor; -import net.jpountz.lz4.LZ4Factory; -import org.apache.cassandra.io.util.DataOutputStreamPlus; -import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus; -import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; -import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter; -import org.apache.cassandra.streaming.async.StreamCompressionSerializer; -import org.apache.cassandra.streaming.messages.StreamMessage; - -/** - * The intent of this class is to only be used in a very narrow use-case: on the stream compression path of streaming. - * This class should really only get calls to {@link #write(ByteBuffer)}, where the incoming buffer is compressed and sent - * downstream. - */ -public class ByteBufCompressionDataOutputStreamPlus extends WrappedDataOutputStreamPlus -{ - private final StreamRateLimiter limiter; - private final LZ4Compressor compressor; - private final StreamCompressionSerializer serializer; - - public ByteBufCompressionDataOutputStreamPlus(DataOutputStreamPlus out, StreamRateLimiter limiter) - { - super(out); - assert out instanceof ByteBufDataOutputStreamPlus; - compressor = LZ4Factory.fastestInstance().fastCompressor(); - serializer = new StreamCompressionSerializer(((ByteBufDataOutputStreamPlus)out).getAllocator()); - this.limiter = limiter; - } - - /** - * {@inheritDoc} - * - * Compress the incoming buffer and send the result downstream. The buffer parameter will not be used nor passed - * to downstream components, and thus callers can safely free the buffer upon return. - */ - @Override - public void write(ByteBuffer buffer) throws IOException - { - ByteBuf compressed = serializer.serialize(compressor, buffer, StreamMessage.CURRENT_VERSION); - - // this is a blocking call - you have been warned - limiter.acquire(compressed.readableBytes()); - - ((ByteBufDataOutputStreamPlus)out).writeToChannel(compressed); - } - - @Override - public void close() - { - // explicitly overriding close() to avoid closing the wrapped stream; it will be closed via other means - } -} diff --git a/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java b/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java index daf6d28e9424..50d746aeb66f 100644 --- a/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java +++ b/src/java/org/apache/cassandra/streaming/compress/StreamCompressionInputStream.java @@ -28,7 +28,7 @@ import net.jpountz.lz4.LZ4FastDecompressor; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.RebufferingInputStream; -import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus; import org.apache.cassandra.streaming.async.StreamCompressionSerializer; public class StreamCompressionInputStream extends RebufferingInputStream implements AutoCloseable @@ -56,8 +56,8 @@ public StreamCompressionInputStream(DataInputPlus dataInputPlus, int protocolVer this.protocolVersion = protocolVersion; this.decompressor = LZ4Factory.fastestInstance().fastDecompressor(); - ByteBufAllocator allocator = dataInputPlus instanceof RebufferingByteBufDataInputPlus - ? ((RebufferingByteBufDataInputPlus)dataInputPlus).getAllocator() + ByteBufAllocator allocator = dataInputPlus instanceof AsyncStreamingInputPlus + ? ((AsyncStreamingInputPlus)dataInputPlus).getAllocator() : PooledByteBufAllocator.DEFAULT; deserializer = new StreamCompressionSerializer(allocator); } diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java index a591a43f5ce1..e14879039017 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java @@ -24,13 +24,14 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputStreamPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.utils.UUIDSerializer; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + /** * StreamInitMessage is first sent from the node where {@link org.apache.cassandra.streaming.StreamSession} is started, * to initiate corresponding {@link org.apache.cassandra.streaming.StreamSession} on the other side. @@ -72,7 +73,7 @@ private static class StreamInitMessageSerializer implements Serializer /** StreamMessage types */ public enum Type { - PREPARE_SYN(1, 5, PrepareSynMessage.serializer), - STREAM(2, 0, IncomingStreamMessage.serializer, OutgoingStreamMessage.serializer), - RECEIVED(3, 4, ReceivedMessage.serializer), - COMPLETE(5, 1, CompleteMessage.serializer), - SESSION_FAILED(6, 5, SessionFailedMessage.serializer), - KEEP_ALIVE(7, 5, KeepAliveMessage.serializer), - PREPARE_SYNACK(8, 5, PrepareSynAckMessage.serializer), - PREPARE_ACK(9, 5, PrepareAckMessage.serializer), - STREAM_INIT(10, 5, StreamInitMessage.serializer); - - public static Type get(byte type) + PREPARE_SYN (1, 5, PrepareSynMessage.serializer ), + STREAM (2, 0, IncomingStreamMessage.serializer, OutgoingStreamMessage.serializer), + RECEIVED (3, 4, ReceivedMessage.serializer ), + COMPLETE (5, 1, CompleteMessage.serializer ), + SESSION_FAILED (6, 5, SessionFailedMessage.serializer), + KEEP_ALIVE (7, 5, KeepAliveMessage.serializer ), + PREPARE_SYNACK (8, 5, PrepareSynAckMessage.serializer), + PREPARE_ACK (9, 5, PrepareAckMessage.serializer ), + STREAM_INIT (10, 5, StreamInitMessage.serializer ); + + private static final Type[] idToTypeMap; + + static { - for (Type t : Type.values()) + Type[] values = values(); + + int max = Integer.MIN_VALUE; + for (Type t : values) + max = max(t.id, max); + + Type[] idMap = new Type[max + 1]; + for (Type t : values) { - if (t.type == type) - return t; + if (idMap[t.id] != null) + throw new RuntimeException("Two StreamMessage Types map to the same id: " + t.id); + idMap[t.id] = t; } - throw new IllegalArgumentException("Unknown type " + type); + + idToTypeMap = idMap; } - private final byte type; + public static Type lookupById(int id) + { + if (id < 0 || id >= idToTypeMap.length) + throw new IllegalArgumentException("Invalid type id: " + id); + + return idToTypeMap[id]; + } + + public final int id; public final int priority; + public final Serializer inSerializer; public final Serializer outSerializer; - @SuppressWarnings("unchecked") - private Type(int type, int priority, Serializer serializer) + Type(int id, int priority, Serializer serializer) { - this(type, priority, serializer, serializer); + this(id, priority, serializer, serializer); } @SuppressWarnings("unchecked") - private Type(int type, int priority, Serializer inSerializer, Serializer outSerializer) + Type(int id, int priority, Serializer inSerializer, Serializer outSerializer) { - this.type = (byte) type; + if (id < 0 || id > Byte.MAX_VALUE) + throw new IllegalArgumentException("StreamMessage Type id must be non-negative and less than " + Byte.MAX_VALUE); + + this.id = id; this.priority = priority; this.inSerializer = inSerializer; this.outSerializer = outSerializer; diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessageHeader.java index 84cf3a392aa0..e76777a00e51 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamMessageHeader.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessageHeader.java @@ -24,12 +24,13 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.utils.UUIDSerializer; +import static org.apache.cassandra.locator.InetAddressAndPort.Serializer.inetAddressAndPortSerializer; + /** * StreamingFileHeader is appended before sending actual data to describe what it's sending. */ @@ -102,7 +103,7 @@ static class FileMessageHeaderSerializer public void serialize(StreamMessageHeader header, DataOutputPlus out, int version) throws IOException { header.tableId.serialize(out); - CompactEndpointSerializationHelper.streamingInstance.serialize(header.sender, out, version); + inetAddressAndPortSerializer.serialize(header.sender, out, version); UUIDSerializer.serializer.serialize(header.planId, out, version); out.writeInt(header.sessionIndex); out.writeInt(header.sequenceNumber); @@ -117,7 +118,7 @@ public void serialize(StreamMessageHeader header, DataOutputPlus out, int versio public StreamMessageHeader deserialize(DataInputPlus in, int version) throws IOException { TableId tableId = TableId.deserialize(in); - InetAddressAndPort sender = CompactEndpointSerializationHelper.streamingInstance.deserialize(in, version); + InetAddressAndPort sender = inetAddressAndPortSerializer.deserialize(in, version); UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version); int sessionIndex = in.readInt(); int sequenceNumber = in.readInt(); @@ -130,7 +131,7 @@ public StreamMessageHeader deserialize(DataInputPlus in, int version) throws IOE public long serializedSize(StreamMessageHeader header, int version) { long size = header.tableId.serializedSize(); - size += CompactEndpointSerializationHelper.streamingInstance.serializedSize(header.sender, version); + size += inetAddressAndPortSerializer.serializedSize(header.sender, version); size += UUIDSerializer.serializer.serializedSize(header.planId, version); size += TypeSizes.sizeof(header.sessionIndex); size += TypeSizes.sizeof(header.sequenceNumber); diff --git a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java index cce686fb0103..177f811898e7 100644 --- a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java +++ b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java @@ -19,18 +19,16 @@ package org.apache.cassandra.tools; import java.io.IOException; -import java.net.InetSocketAddress; import io.netty.channel.Channel; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; +import org.apache.cassandra.net.OutboundConnectionSettings; import org.apache.cassandra.streaming.DefaultConnectionFactory; -import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.streaming.StreamConnectionFactory; public class BulkLoadConnectionFactory extends DefaultConnectionFactory implements StreamConnectionFactory { + // TODO: what is this unused variable for? private final boolean outboundBindAny; private final int secureStoragePort; private final EncryptionOptions.ServerEncryptionOptions encryptionOptions; @@ -38,21 +36,19 @@ public class BulkLoadConnectionFactory extends DefaultConnectionFactory implemen public BulkLoadConnectionFactory(int secureStoragePort, EncryptionOptions.ServerEncryptionOptions encryptionOptions, boolean outboundBindAny) { this.secureStoragePort = secureStoragePort; - this.encryptionOptions = encryptionOptions != null && encryptionOptions.internode_encryption == EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none - ? null - : encryptionOptions; + this.encryptionOptions = encryptionOptions; this.outboundBindAny = outboundBindAny; } - public Channel createConnection(OutboundConnectionIdentifier connectionId, int protocolVersion) throws IOException + public Channel createConnection(OutboundConnectionSettings template, int messagingVersion) throws IOException { // Connect to secure port for all peers if ServerEncryptionOptions is configured other than 'none' // When 'all', 'dc' and 'rack', server nodes always have SSL port open, and since thin client like sstableloader // does not know which node is in which dc/rack, connecting to SSL port is always the option. - int port = encryptionOptions != null && encryptionOptions.internode_encryption != EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none ? - secureStoragePort : connectionId.remote().port; - connectionId = connectionId.withNewConnectionAddress(InetAddressAndPort.getByAddressOverrideDefaults(connectionId.remote().address, port)); - return createConnection(connectionId, protocolVersion, encryptionOptions); + if (encryptionOptions != null && encryptionOptions.internode_encryption != EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none) + template = template.withConnectTo(template.to.withPort(secureStoragePort)); + + return super.createConnection(template, messagingVersion); } } diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java index d85c6054ba6d..2ca2a3d5b273 100644 --- a/src/java/org/apache/cassandra/tools/BulkLoader.java +++ b/src/java/org/apache/cassandra/tools/BulkLoader.java @@ -265,7 +265,7 @@ private static SSLOptions buildSSLOptions(EncryptionOptions clientEncryptionOpti return JdkSSLOptions.builder() .withSSLContext(sslContext) - .withCipherSuites(clientEncryptionOptions.cipher_suites) + .withCipherSuites(clientEncryptionOptions.cipher_suites.toArray(new String[0])) .build(); } diff --git a/src/java/org/apache/cassandra/tools/LoaderOptions.java b/src/java/org/apache/cassandra/tools/LoaderOptions.java index d6cb670655e3..7ad3299e298a 100644 --- a/src/java/org/apache/cassandra/tools/LoaderOptions.java +++ b/src/java/org/apache/cassandra/tools/LoaderOptions.java @@ -468,50 +468,49 @@ public Builder parseArgs(String cmdArgs[]) if (cmd.hasOption(SSL_TRUSTSTORE) || cmd.hasOption(SSL_TRUSTSTORE_PW) || cmd.hasOption(SSL_KEYSTORE) || cmd.hasOption(SSL_KEYSTORE_PW)) { - clientEncOptions.enabled = true; + clientEncOptions = clientEncOptions.withEnabled(true); } if (cmd.hasOption(SSL_TRUSTSTORE)) { - clientEncOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE); + clientEncOptions = clientEncOptions.withTrustStore(cmd.getOptionValue(SSL_TRUSTSTORE)); } if (cmd.hasOption(SSL_TRUSTSTORE_PW)) { - clientEncOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW); + clientEncOptions = clientEncOptions.withTrustStorePassword(cmd.getOptionValue(SSL_TRUSTSTORE_PW)); } if (cmd.hasOption(SSL_KEYSTORE)) { - clientEncOptions.keystore = cmd.getOptionValue(SSL_KEYSTORE); // if a keystore was provided, lets assume we'll need to use - // it - clientEncOptions.require_client_auth = true; + clientEncOptions = clientEncOptions.withKeyStore(cmd.getOptionValue(SSL_KEYSTORE)) + .withRequireClientAuth(true); } if (cmd.hasOption(SSL_KEYSTORE_PW)) { - clientEncOptions.keystore_password = cmd.getOptionValue(SSL_KEYSTORE_PW); + clientEncOptions = clientEncOptions.withKeyStorePassword(cmd.getOptionValue(SSL_KEYSTORE_PW)); } if (cmd.hasOption(SSL_PROTOCOL)) { - clientEncOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL); + clientEncOptions = clientEncOptions.withProtocol(cmd.getOptionValue(SSL_PROTOCOL)); } if (cmd.hasOption(SSL_ALGORITHM)) { - clientEncOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM); + clientEncOptions = clientEncOptions.withAlgorithm(cmd.getOptionValue(SSL_ALGORITHM)); } if (cmd.hasOption(SSL_STORE_TYPE)) { - clientEncOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE); + clientEncOptions = clientEncOptions.withStoreType(cmd.getOptionValue(SSL_STORE_TYPE)); } if (cmd.hasOption(SSL_CIPHER_SUITES)) { - clientEncOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(","); + clientEncOptions = clientEncOptions.withCipherSuites(cmd.getOptionValue(SSL_CIPHER_SUITES).split(",")); } if (cmd.hasOption(TARGET_KEYSPACE)) diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java index 55e36a4a7a7f..5891981f400d 100644 --- a/src/java/org/apache/cassandra/tracing/Tracing.java +++ b/src/java/org/apache/cassandra/tracing/Tracing.java @@ -23,13 +23,11 @@ import java.net.InetAddress; import java.nio.ByteBuffer; import java.util.Collections; -import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import com.google.common.collect.ImmutableList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,9 +39,8 @@ import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.ParamType; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.UUIDGen; @@ -246,12 +243,11 @@ public TraceState begin(final String request, final Map paramete /** * Determines the tracing context from a message. Does NOT set the threadlocal state. * - * @param message The internode message + * @param header The internode message header */ - public TraceState initializeFromMessage(final MessageIn message) + public TraceState initializeFromMessage(final Message.Header header) { - final UUID sessionId = (UUID)message.parameters.get(ParameterType.TRACE_SESSION); - + final UUID sessionId = header.traceSession(); if (sessionId == null) return null; @@ -259,31 +255,60 @@ public TraceState initializeFromMessage(final MessageIn message) if (ts != null && ts.acquireReference()) return ts; - TraceType tmpType; - TraceType traceType = TraceType.QUERY; - if ((tmpType = (TraceType)message.parameters.get(ParameterType.TRACE_TYPE)) != null) - traceType = tmpType; + TraceType traceType = header.traceType(); - if (message.verb == MessagingService.Verb.REQUEST_RESPONSE) + if (header.verb.isResponse()) { // received a message for a session we've already closed out. see CASSANDRA-5668 - return new ExpiredTraceState(newTraceState(message.from, sessionId, traceType)); + return new ExpiredTraceState(newTraceState(header.from, sessionId, traceType)); } else { - ts = newTraceState(message.from, sessionId, traceType); + ts = newTraceState(header.from, sessionId, traceType); sessions.put(sessionId, ts); return ts; } } - public List getTraceHeaders() + /** + * Record any tracing data, if enabled on this message. + */ + public void traceOutgoingMessage(Message message, InetAddressAndPort sendTo) + { + try + { + final UUID sessionId = message.traceSession(); + if (sessionId == null) + return; + + String logMessage = String.format("Sending %s message to %s", message.verb(), sendTo); + + TraceState state = get(sessionId); + if (state == null) // session may have already finished; see CASSANDRA-5668 + { + TraceType traceType = message.traceType(); + trace(ByteBuffer.wrap(UUIDGen.decompose(sessionId)), logMessage, traceType.getTTL()); + } + else + { + state.trace(logMessage); + if (message.verb().isResponse()) + doneWithNonLocalSession(state); + } + } + catch (Exception e) + { + logger.warn("failed to capture the tracing info for an outbound message to {}, ignoring", sendTo, e); + } + } + + public Map addTraceHeaders(Map addToMutable) { assert isTracing(); - return ImmutableList.of( - ParameterType.TRACE_SESSION, Tracing.instance.getSessionId(), - ParameterType.TRACE_TYPE, Tracing.instance.getTraceType()); + addToMutable.put(ParamType.TRACE_SESSION, Tracing.instance.getSessionId()); + addToMutable.put(ParamType.TRACE_TYPE, Tracing.instance.getTraceType()); + return addToMutable; } protected abstract TraceState newTraceState(InetAddressAndPort coordinator, UUID sessionId, Tracing.TraceType traceType); diff --git a/src/java/org/apache/cassandra/transport/ProtocolVersion.java b/src/java/org/apache/cassandra/transport/ProtocolVersion.java index 546983ffafd5..5c8c299952a8 100644 --- a/src/java/org/apache/cassandra/transport/ProtocolVersion.java +++ b/src/java/org/apache/cassandra/transport/ProtocolVersion.java @@ -104,13 +104,13 @@ public static ProtocolVersion decode(int versionNum, boolean allowOlderProtocols // if this is not a supported version check the old versions for (ProtocolVersion version : UNSUPPORTED) { - // if it is an old version that is no longer supported this ensures that we reply + // if it is an old version that is no longer supported this ensures that we respond // with that same version if (version.num == versionNum) throw new ProtocolException(ProtocolVersion.invalidVersionMessage(versionNum), version); } - // If the version is invalid reply with the highest version that we support + // If the version is invalid response with the highest version that we support throw new ProtocolException(invalidVersionMessage(versionNum), MAX_SUPPORTED_VERSION); } diff --git a/src/java/org/apache/cassandra/transport/ProtocolVersionTracker.java b/src/java/org/apache/cassandra/transport/ProtocolVersionTracker.java index 72bb9012d9cc..f2893778c70d 100644 --- a/src/java/org/apache/cassandra/transport/ProtocolVersionTracker.java +++ b/src/java/org/apache/cassandra/transport/ProtocolVersionTracker.java @@ -25,7 +25,6 @@ import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; -import org.apache.cassandra.utils.Clock; /** * This class tracks the last 100 connections per protocol version @@ -48,13 +47,13 @@ private ProtocolVersionTracker(int capacity) for (ProtocolVersion version : ProtocolVersion.values()) { clientsByProtocolVersion.put(version, Caffeine.newBuilder().maximumSize(capacity) - .build(key -> Clock.instance.currentTimeMillis())); + .build(key -> System.currentTimeMillis())); } } void addConnection(InetAddress addr, ProtocolVersion version) { - clientsByProtocolVersion.get(version).put(addr, Clock.instance.currentTimeMillis()); + clientsByProtocolVersion.get(version).put(addr, System.currentTimeMillis()); } List getAll() diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java index 8e0b19ff64bc..a358015b3510 100644 --- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java +++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java @@ -377,7 +377,7 @@ else if (e instanceof WrappedException) if (e instanceof ProtocolException) { // if the driver attempted to connect with a protocol version not supported then - // reply with the appropiate version, see ProtocolVersion.decode() + // respond with the appropiate version, see ProtocolVersion.decode() ProtocolVersion forcedProtocolVersion = ((ProtocolException) e).getForcedProtocolVersion(); if (forcedProtocolVersion != null) message.forcedProtocolVersion = forcedProtocolVersion; diff --git a/src/java/org/apache/cassandra/utils/ApproximateTime.java b/src/java/org/apache/cassandra/utils/ApproximateTime.java new file mode 100644 index 000000000000..32b6e44317ca --- /dev/null +++ b/src/java/org/apache/cassandra/utils/ApproximateTime.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.config.Config; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.cassandra.utils.ApproximateTime.Measurement.ALMOST_NOW; +import static org.apache.cassandra.utils.ApproximateTime.Measurement.ALMOST_SAME_TIME; + +/** + * This class provides approximate time utilities: + * - An imprecise nanoTime (monotonic) and currentTimeMillis (non-monotonic), that are faster than their regular counterparts + * They have a configured approximate precision (default of 10ms), which is the cadence they will be updated if the system is healthy + * - A mechanism for converting between nanoTime and currentTimeMillis measurements. + * These conversions may have drifted, and they offer no absolute guarantees on precision + */ +public class ApproximateTime +{ + private static final Logger logger = LoggerFactory.getLogger(ApproximateTime.class); + private static final int ALMOST_NOW_UPDATE_INTERVAL_MS = Math.max(1, Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "approximate_time_precision_ms", "2"))); + private static final String CONVERSION_UPDATE_INTERVAL_PROPERTY = Config.PROPERTY_PREFIX + "NANOTIMETOMILLIS_TIMESTAMP_UPDATE_INTERVAL"; + private static final long ALMOST_SAME_TIME_UPDATE_INTERVAL_MS = Long.getLong(CONVERSION_UPDATE_INTERVAL_PROPERTY, 10000); + + public static class AlmostSameTime + { + final long millis; + final long nanos; + final long error; // maximum error of millis measurement (in nanos) + + private AlmostSameTime(long millis, long nanos, long error) + { + this.millis = millis; + this.nanos = nanos; + this.error = error; + } + + public long toCurrentTimeMillis(long nanoTime) + { + return millis + TimeUnit.NANOSECONDS.toMillis(nanoTime - nanos); + } + + public long toNanoTime(long currentTimeMillis) + { + return nanos + MILLISECONDS.toNanos(currentTimeMillis - millis); + } + } + + public enum Measurement { ALMOST_NOW, ALMOST_SAME_TIME } + + private static volatile Future almostNowUpdater; + private static volatile Future almostSameTimeUpdater; + + private static volatile long almostNowMillis; + private static volatile long almostNowNanos; + + private static volatile AlmostSameTime almostSameTime = new AlmostSameTime(0L, 0L, Long.MAX_VALUE); + private static double failedAlmostSameTimeUpdateModifier = 1.0; + + private static final Runnable refreshAlmostNow = () -> { + almostNowMillis = System.currentTimeMillis(); + almostNowNanos = System.nanoTime(); + }; + + private static final Runnable refreshAlmostSameTime = () -> { + final int tries = 3; + long[] samples = new long[2 * tries + 1]; + samples[0] = System.nanoTime(); + for (int i = 1 ; i < samples.length ; i += 2) + { + samples[i] = System.currentTimeMillis(); + samples[i + 1] = System.nanoTime(); + } + + int best = 1; + // take sample with minimum delta between calls + for (int i = 3 ; i < samples.length - 1 ; i += 2) + { + if ((samples[i+1] - samples[i-1]) < (samples[best+1]-samples[best-1])) + best = i; + } + + long millis = samples[best]; + long nanos = (samples[best+1] / 2) + (samples[best-1] / 2); + long error = (samples[best+1] / 2) - (samples[best-1] / 2); + + AlmostSameTime prev = almostSameTime; + AlmostSameTime next = new AlmostSameTime(millis, nanos, error); + + if (next.error > prev.error && next.error > prev.error * failedAlmostSameTimeUpdateModifier) + { + failedAlmostSameTimeUpdateModifier *= 1.1; + return; + } + + failedAlmostSameTimeUpdateModifier = 1.0; + almostSameTime = next; + }; + + static + { + start(ALMOST_NOW); + start(ALMOST_SAME_TIME); + } + + public static synchronized void stop(Measurement measurement) + { + switch (measurement) + { + case ALMOST_NOW: + almostNowUpdater.cancel(true); + try { almostNowUpdater.get(); } catch (Throwable t) { } + almostNowUpdater = null; + break; + case ALMOST_SAME_TIME: + almostSameTimeUpdater.cancel(true); + try { almostSameTimeUpdater.get(); } catch (Throwable t) { } + almostSameTimeUpdater = null; + break; + } + } + + public static synchronized void start(Measurement measurement) + { + switch (measurement) + { + case ALMOST_NOW: + if (almostNowUpdater != null) + throw new IllegalStateException("Already running"); + refreshAlmostNow.run(); + logger.info("Scheduling approximate time-check task with a precision of {} milliseconds", ALMOST_NOW_UPDATE_INTERVAL_MS); + almostNowUpdater = ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(refreshAlmostNow, ALMOST_NOW_UPDATE_INTERVAL_MS, ALMOST_NOW_UPDATE_INTERVAL_MS, MILLISECONDS); + break; + case ALMOST_SAME_TIME: + if (almostSameTimeUpdater != null) + throw new IllegalStateException("Already running"); + refreshAlmostSameTime.run(); + logger.info("Scheduling approximate time conversion task with an interval of {} milliseconds", ALMOST_SAME_TIME_UPDATE_INTERVAL_MS); + almostSameTimeUpdater = ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(refreshAlmostSameTime, ALMOST_SAME_TIME_UPDATE_INTERVAL_MS, ALMOST_SAME_TIME_UPDATE_INTERVAL_MS, MILLISECONDS); + break; + } + } + + + /** + * Request an immediate refresh; this shouldn't generally be invoked, except perhaps by tests + */ + @VisibleForTesting + public static synchronized void refresh(Measurement measurement) + { + stop(measurement); + start(measurement); + } + + /** no guarantees about relationship to nanoTime; non-monotonic (tracks currentTimeMillis as closely as possible) */ + public static long currentTimeMillis() + { + return almostNowMillis; + } + + /** no guarantees about relationship to currentTimeMillis; monotonic */ + public static long nanoTime() + { + return almostNowNanos; + } +} diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java index d6c9e52b7793..788300c7461e 100644 --- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java +++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java @@ -255,14 +255,14 @@ public static ByteBuffer clone(ByteBuffer buffer) return clone; } - public static void arrayCopy(ByteBuffer src, int srcPos, byte[] dst, int dstPos, int length) + public static void copyBytes(ByteBuffer src, int srcPos, byte[] dst, int dstPos, int length) { FastByteOperations.copy(src, srcPos, dst, dstPos, length); } /** * Transfer bytes from one ByteBuffer to another. - * This function acts as System.arrayCopy() but for ByteBuffers. + * This function acts as System.arrayCopy() but for ByteBuffers, and operates safely on direct memory. * * @param src the source ByteBuffer * @param srcPos starting position in the source ByteBuffer @@ -270,7 +270,7 @@ public static void arrayCopy(ByteBuffer src, int srcPos, byte[] dst, int dstPos, * @param dstPos starting position in the destination ByteBuffer * @param length the number of bytes to copy */ - public static void arrayCopy(ByteBuffer src, int srcPos, ByteBuffer dst, int dstPos, int length) + public static void copyBytes(ByteBuffer src, int srcPos, ByteBuffer dst, int dstPos, int length) { FastByteOperations.copy(src, srcPos, dst, dstPos, length); } @@ -278,7 +278,7 @@ public static void arrayCopy(ByteBuffer src, int srcPos, ByteBuffer dst, int dst public static int put(ByteBuffer src, ByteBuffer trg) { int length = Math.min(src.remaining(), trg.remaining()); - arrayCopy(src, src.position(), trg, trg.position(), length); + copyBytes(src, src.position(), trg, trg.position(), length); trg.position(trg.position() + length); src.position(src.position() + length); return length; diff --git a/src/java/org/apache/cassandra/utils/Clock.java b/src/java/org/apache/cassandra/utils/Clock.java deleted file mode 100644 index eb9822c1cff9..000000000000 --- a/src/java/org/apache/cassandra/utils/Clock.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.utils; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Wrapper around time related functions that are either implemented by using the default JVM calls - * or by using a custom implementation for testing purposes. - * - * See {@link #instance} for how to use a custom implementation. - * - * Please note that {@link java.time.Clock} wasn't used, as it would not be possible to provide an - * implementation for {@link #nanoTime()} with the exact same properties of {@link System#nanoTime()}. - */ -public class Clock -{ - private static final Logger logger = LoggerFactory.getLogger(Clock.class); - - /** - * Static singleton object that will be instanciated by default with a system clock - * implementation. Set cassandra.clock system property to a FQCN to use a - * different implementation instead. - */ - public static Clock instance; - - static - { - String sclock = System.getProperty("cassandra.clock"); - if (sclock == null) - { - instance = new Clock(); - } - else - { - try - { - logger.debug("Using custom clock implementation: {}", sclock); - instance = (Clock) Class.forName(sclock).newInstance(); - } - catch (Exception e) - { - logger.error(e.getMessage(), e); - } - } - } - - /** - * @see System#nanoTime() - */ - public long nanoTime() - { - return System.nanoTime(); - } - - /** - * @see System#currentTimeMillis() - */ - public long currentTimeMillis() - { - return System.currentTimeMillis(); - } - -} diff --git a/src/java/org/apache/cassandra/utils/CoalescingStrategies.java b/src/java/org/apache/cassandra/utils/CoalescingStrategies.java deleted file mode 100644 index 2f9e5bbf084d..000000000000 --- a/src/java/org/apache/cassandra/utils/CoalescingStrategies.java +++ /dev/null @@ -1,444 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.utils; - -import org.apache.cassandra.concurrent.ScheduledExecutors; -import org.apache.cassandra.config.Config; -import org.apache.cassandra.io.util.FileUtils; -import org.slf4j.Logger; - -import java.io.File; -import java.io.RandomAccessFile; -import java.lang.reflect.Constructor; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel.MapMode; -import java.util.Arrays; -import java.util.Collection; -import java.util.Optional; -import java.util.concurrent.TimeUnit; -import java.util.Locale; - -import com.google.common.annotations.VisibleForTesting; - -/** - * Groups strategies to coalesce messages. - */ -public class CoalescingStrategies -{ - /* - * Log debug information at info level about what the average is and when coalescing is enabled/disabled - */ - private static final String DEBUG_COALESCING_PROPERTY = Config.PROPERTY_PREFIX + "coalescing_debug"; - private static final boolean DEBUG_COALESCING = Boolean.getBoolean(DEBUG_COALESCING_PROPERTY); - - private static final String DEBUG_COALESCING_PATH_PROPERTY = Config.PROPERTY_PREFIX + "coalescing_debug_path"; - private static final String DEBUG_COALESCING_PATH = System.getProperty(DEBUG_COALESCING_PATH_PROPERTY, "/tmp/coleascing_debug"); - - public enum Strategy { MOVINGAVERAGE, FIXED, TIMEHORIZON, DISABLED } - - static - { - if (DEBUG_COALESCING) - { - File directory = new File(DEBUG_COALESCING_PATH); - - if (directory.exists()) - FileUtils.deleteRecursive(directory); - - if (!directory.mkdirs()) - throw new ExceptionInInitializerError("Couldn't create log dir"); - } - } - - public static interface Coalescable - { - long timestampNanos(); - } - - @VisibleForTesting - static long determineCoalescingTime(long averageGap, long maxCoalesceWindow) - { - // Don't bother waiting at all if we're unlikely to get any new message within our max window - if (averageGap > maxCoalesceWindow) - return -1; - - // avoid the degenerate case of zero (very unlikely, but let's be safe) - if (averageGap <= 0) - return maxCoalesceWindow; - - // assume we receive as many messages as we expect; apply the same logic to the future batch: - // expect twice as many messages to consider sleeping for "another" interval; this basically translates - // to doubling our sleep period until we exceed our max sleep window. - long sleep = averageGap; - while (sleep * 2 < maxCoalesceWindow) - sleep *= 2; - return sleep; - } - - /** - * A coalescing strategy, that decides when to coalesce messages. - *

- * The general principle is that, when asked, the strategy returns the time delay we want to wait for more messages - * to arrive before sending so message can be coalesced. For that, the strategy must be fed new messages through - * the {@link #newArrival(Coalescable)} method (the only assumption we make on messages is that they have an associated - * timestamp). The strategy can then be queried for the time to wait for coalescing through - * {@link #currentCoalescingTimeNanos()}. - *

- * Note that it is expected that a call {@link #currentCoalescingTimeNanos()} will come just after a call to - * {@link #newArrival(Coalescable))}, as the intent of the value returned by the former method is "Given a new message, how much - * time should I wait for more messages to arrive and be coalesced with that message". But both calls are separated - * as one may not want to call {@link #currentCoalescingTimeNanos()} after every call to {@link #newArrival(Coalescable)} - * and we thus save processing. How arrivals influence the coalescing time is however entirely up to the strategy and some - * strategy may ignore arrivals completely and return a constant coalescing time. - */ - public interface CoalescingStrategy - { - /** - * Inform the strategy of a new message to consider. - * - * @param message the message to consider. - */ - void newArrival(Coalescable message); - - /** - * The current time to wait for the purpose of coalescing messages. - * - * @return the coalescing time. A negative value can be returned if no coalescing should be done (which can be a - * transient thing). - */ - long currentCoalescingTimeNanos(); - } - - public static abstract class AbstractCoalescingStrategy implements CoalescingStrategy - { - protected final Logger logger; - protected volatile boolean shouldLogAverage = false; - protected final ByteBuffer logBuffer; - private RandomAccessFile ras; - private final String displayName; - - protected AbstractCoalescingStrategy(Logger logger, String displayName) - { - this.logger = logger; - this.displayName = displayName; - - RandomAccessFile rasTemp = null; - ByteBuffer logBufferTemp = null; - if (DEBUG_COALESCING) - { - ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(() -> shouldLogAverage = true, 5, 5, TimeUnit.SECONDS); - try - { - File outFile = FileUtils.createTempFile("coalescing_" + this.displayName + "_", ".log", new File(DEBUG_COALESCING_PATH)); - rasTemp = new RandomAccessFile(outFile, "rw"); - logBufferTemp = ras.getChannel().map(MapMode.READ_WRITE, 0, Integer.MAX_VALUE); - logBufferTemp.putLong(0); - } - catch (Exception e) - { - logger.error("Unable to create output file for debugging coalescing", e); - } - } - ras = rasTemp; - logBuffer = logBufferTemp; - } - - /* - * If debugging is enabled log to the logger the current average gap calculation result. - */ - final protected void debugGap(long averageGap) - { - if (DEBUG_COALESCING && shouldLogAverage) - { - shouldLogAverage = false; - logger.info("{} gap {}μs", this, TimeUnit.NANOSECONDS.toMicros(averageGap)); - } - } - - /* - * If debugging is enabled log the provided nanotime timestamp to a file. - */ - final protected void debugTimestamp(long timestamp) - { - if(DEBUG_COALESCING && logBuffer != null) - { - logBuffer.putLong(0, logBuffer.getLong(0) + 1); - logBuffer.putLong(timestamp); - } - } - - /* - * If debugging is enabled log the timestamps of all the items in the provided collection - * to a file. - */ - final protected void debugTimestamps(Collection coalescables) - { - if (DEBUG_COALESCING) - { - for (C coalescable : coalescables) - { - debugTimestamp(coalescable.timestampNanos()); - } - } - } - } - - @VisibleForTesting - static class TimeHorizonMovingAverageCoalescingStrategy extends AbstractCoalescingStrategy - { - // for now we'll just use 64ms per bucket; this can be made configurable, but results in ~1s for 16 samples - private static final int INDEX_SHIFT = 26; - private static final long BUCKET_INTERVAL = 1L << 26; - private static final int BUCKET_COUNT = 16; - private static final long INTERVAL = BUCKET_INTERVAL * BUCKET_COUNT; - private static final long MEASURED_INTERVAL = BUCKET_INTERVAL * (BUCKET_COUNT - 1); - - // the minimum timestamp we will now accept updates for; only moves forwards, never backwards - private long epoch; - // the buckets, each following on from epoch; the measurements run from ix(epoch) to ix(epoch - 1) - // ix(epoch-1) is a partial result, that is never actually part of the calculation, and most updates - // are expected to hit this bucket - private final int samples[] = new int[BUCKET_COUNT]; - private long sum = 0; - private final long maxCoalesceWindow; - - public TimeHorizonMovingAverageCoalescingStrategy(int maxCoalesceWindow, Logger logger, String displayName, long initialEpoch) - { - super(logger, displayName); - this.maxCoalesceWindow = TimeUnit.MICROSECONDS.toNanos(maxCoalesceWindow); - sum = 0; - epoch = initialEpoch; - } - - private long averageGap() - { - if (sum == 0) - return Integer.MAX_VALUE; - return MEASURED_INTERVAL / sum; - } - - // this sample extends past the end of the range we cover, so rollover - private long rollEpoch(long delta, long epoch, long nanos) - { - if (delta > 2 * INTERVAL) - { - // this sample is more than twice our interval ahead, so just clear our counters completely - epoch = epoch(nanos); - sum = 0; - Arrays.fill(samples, 0); - } - else - { - // ix(epoch - 1) => last index; this is our partial result bucket, so we add this to the sum - sum += samples[ix(epoch - 1)]; - // then we roll forwards, clearing buckets, until our interval covers the new sample time - while (epoch + INTERVAL < nanos) - { - int index = ix(epoch); - sum -= samples[index]; - samples[index] = 0; - epoch += BUCKET_INTERVAL; - } - } - // store the new epoch - this.epoch = epoch; - return epoch; - } - - private long epoch(long latestNanos) - { - return (latestNanos - MEASURED_INTERVAL) & ~(BUCKET_INTERVAL - 1); - } - - private int ix(long nanos) - { - return (int) ((nanos >>> INDEX_SHIFT) & 15); - } - - public void newArrival(Coalescable message) - { - final long timestamp = message.timestampNanos(); - debugTimestamp(timestamp); - long epoch = this.epoch; - long delta = timestamp - epoch; - if (delta < 0) - // have to simply ignore, but would be a bit unlucky to get such reordering - return; - - if (delta > INTERVAL) - epoch = rollEpoch(delta, epoch, timestamp); - - int ix = ix(timestamp); - samples[ix]++; - - // if we've updated an old bucket, we need to update the sum to match - if (ix != ix(epoch - 1)) - sum++; - } - - public long currentCoalescingTimeNanos() - { - long averageGap = averageGap(); - debugGap(averageGap); - return determineCoalescingTime(averageGap, maxCoalesceWindow); - } - - @Override - public String toString() - { - return "Time horizon moving average"; - } - } - - /** - * Start coalescing by sleeping if the moving average is < the requested window. - * The actual time spent waiting to coalesce will be the min( window, moving average * 2) - * The actual amount of time spent waiting can be greater then the window. For instance - * observed time spent coalescing was 400 microseconds with the window set to 200 in one benchmark. - */ - @VisibleForTesting - static class MovingAverageCoalescingStrategy extends AbstractCoalescingStrategy - { - static final int SAMPLE_SIZE = 16; - private final int samples[] = new int[SAMPLE_SIZE]; - private final long maxCoalesceWindow; - - private long lastSample = 0; - private int index = 0; - private long sum = 0; - private long currentGap; - - public MovingAverageCoalescingStrategy(int maxCoalesceWindow, Logger logger, String displayName) - { - super(logger, displayName); - this.maxCoalesceWindow = TimeUnit.MICROSECONDS.toNanos(maxCoalesceWindow); - for (int ii = 0; ii < samples.length; ii++) - samples[ii] = Integer.MAX_VALUE; - sum = Integer.MAX_VALUE * (long)samples.length; - } - - private long logSample(int value) - { - sum -= samples[index]; - sum += value; - samples[index] = value; - index++; - index = index & ((1 << 4) - 1); - return sum / SAMPLE_SIZE; - } - - public void newArrival(Coalescable message) - { - final long timestamp = message.timestampNanos(); - debugTimestamp(timestamp); - if (timestamp > lastSample) - { - final int delta = (int)(Math.min(Integer.MAX_VALUE, timestamp - lastSample)); - lastSample = timestamp; - currentGap = logSample(delta); - } - else - { - currentGap = logSample(1); - } - } - - public long currentCoalescingTimeNanos() - { - debugGap(currentGap); - return determineCoalescingTime(currentGap, maxCoalesceWindow); - } - - @Override - public String toString() - { - return "Moving average"; - } - } - - /** - * A fixed strategy as a backup in case MovingAverage or TimeHorizongMovingAverage fails in some scenario - */ - @VisibleForTesting - static class FixedCoalescingStrategy extends AbstractCoalescingStrategy - { - private final long coalesceWindow; - - public FixedCoalescingStrategy(int coalesceWindowMicros, Logger logger, String displayName) - { - super(logger, displayName); - coalesceWindow = TimeUnit.MICROSECONDS.toNanos(coalesceWindowMicros); - } - - public void newArrival(Coalescable message) - { - debugTimestamp(message.timestampNanos()); - } - - public long currentCoalescingTimeNanos() - { - return coalesceWindow; - } - - @Override - public String toString() - { - return "Fixed"; - } - } - - public static Optional newCoalescingStrategy(String strategy, int coalesceWindow, Logger logger, String displayName) - { - String strategyCleaned = strategy.trim().toUpperCase(Locale.ENGLISH); - - try - { - switch (Enum.valueOf(Strategy.class, strategyCleaned)) - { - case MOVINGAVERAGE: - return Optional.of(new MovingAverageCoalescingStrategy(coalesceWindow, logger, displayName)); - case FIXED: - return Optional.of(new FixedCoalescingStrategy(coalesceWindow, logger, displayName)); - case TIMEHORIZON: - long initialEpoch = System.nanoTime(); - return Optional.of(new TimeHorizonMovingAverageCoalescingStrategy(coalesceWindow, logger, displayName, initialEpoch)); - case DISABLED: - return Optional.empty(); - default: - throw new IllegalArgumentException("supported coalese strategy"); - } - } - catch (IllegalArgumentException iae) - { - try - { - Class clazz = Class.forName(strategy); - - if (!CoalescingStrategy.class.isAssignableFrom(clazz)) - throw new RuntimeException(strategy + " is not an instance of CoalescingStrategy"); - - Constructor constructor = clazz.getConstructor(int.class, Logger.class, String.class); - return Optional.of((CoalescingStrategy)constructor.newInstance(coalesceWindow, logger, displayName)); - } - catch (Exception e) - { - throw new RuntimeException(e); - } - } - } -} diff --git a/src/java/org/apache/cassandra/utils/ExecutorUtils.java b/src/java/org/apache/cassandra/utils/ExecutorUtils.java new file mode 100644 index 000000000000..114fafa42642 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/ExecutorUtils.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.Collection; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.concurrent.InfiniteLoopExecutor; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; + +public class ExecutorUtils +{ + + public static Runnable runWithThreadName(Runnable runnable, String threadName) + { + return () -> { + String oldThreadName = Thread.currentThread().getName(); + try + { + Thread.currentThread().setName(threadName); + runnable.run(); + } + finally + { + Thread.currentThread().setName(oldThreadName); + } + }; + } + + public static void shutdownNow(Collection executors) + { + for (Object executor : executors) + { + if (executor instanceof ExecutorService) + ((ExecutorService) executor).shutdownNow(); + else if (executor instanceof InfiniteLoopExecutor) + ((InfiniteLoopExecutor) executor).shutdownNow(); + else if (executor != null) + throw new IllegalArgumentException(executor.toString()); + } + } + + public static void shutdown(Collection executors) + { + for (ExecutorService executor : executors) + executor.shutdown(); + } + + public static void shutdown(ExecutorService ... executors) + { + for (ExecutorService executor : executors) + executor.shutdown(); + } + + public static void awaitTermination(long timeout, TimeUnit unit, Collection executors) throws InterruptedException, TimeoutException + { + long deadline = System.nanoTime() + unit.toNanos(timeout); + awaitTerminationUntil(deadline, executors); + } + + public static void awaitTerminationUntil(long deadline, Collection executors) throws InterruptedException, TimeoutException + { + for (Object executor : executors) + { + long wait = deadline - System.nanoTime(); + if (executor instanceof ExecutorService) + { + if (wait <= 0 || !((ExecutorService)executor).awaitTermination(wait, NANOSECONDS)) + throw new TimeoutException(executor + " did not terminate on time"); + } + else if (executor instanceof InfiniteLoopExecutor) + { + if (wait <= 0 || !((InfiniteLoopExecutor)executor).awaitTermination(wait, NANOSECONDS)) + throw new TimeoutException(executor + " did not terminate on time"); + } + else if (executor != null) + { + throw new IllegalArgumentException(executor.toString()); + } + } + } + + public static void awaitTermination(long timeout, TimeUnit unit, ExecutorService ... executors) throws InterruptedException, TimeoutException + { + awaitTermination(timeout, unit, ImmutableList.copyOf(executors)); + } + +} diff --git a/src/java/org/apache/cassandra/utils/ExpiringMap.java b/src/java/org/apache/cassandra/utils/ExpiringMap.java deleted file mode 100644 index ef013f57fdd6..000000000000 --- a/src/java/org/apache/cassandra/utils/ExpiringMap.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.utils; - -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -import com.google.common.base.Function; -import com.google.common.util.concurrent.Uninterruptibles; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor; - -public class ExpiringMap -{ - private static final Logger logger = LoggerFactory.getLogger(ExpiringMap.class); - private volatile boolean shutdown; - - public static class CacheableObject - { - public final T value; - public final long timeout; - private final long createdAt; - - private CacheableObject(T value, long timeout) - { - assert value != null; - this.value = value; - this.timeout = timeout; - this.createdAt = Clock.instance.nanoTime(); - } - - private boolean isReadyToDieAt(long atNano) - { - return atNano - createdAt > TimeUnit.MILLISECONDS.toNanos(timeout); - } - } - - // if we use more ExpiringMaps we may want to add multiple threads to this executor - private static final ScheduledExecutorService service = new DebuggableScheduledThreadPoolExecutor("EXPIRING-MAP-REAPER"); - - private final ConcurrentMap> cache = new ConcurrentHashMap>(); - private final long defaultExpiration; - - public ExpiringMap(long defaultExpiration) - { - this(defaultExpiration, null); - } - - /** - * - * @param defaultExpiration the TTL for objects in the cache in milliseconds - */ - public ExpiringMap(long defaultExpiration, final Function>, ?> postExpireHook) - { - this.defaultExpiration = defaultExpiration; - - if (defaultExpiration <= 0) - { - throw new IllegalArgumentException("Argument specified must be a positive number"); - } - - Runnable runnable = new Runnable() - { - public void run() - { - long start = Clock.instance.nanoTime(); - int n = 0; - for (Map.Entry> entry : cache.entrySet()) - { - if (entry.getValue().isReadyToDieAt(start)) - { - if (cache.remove(entry.getKey()) != null) - { - n++; - if (postExpireHook != null) - postExpireHook.apply(Pair.create(entry.getKey(), entry.getValue())); - } - } - } - logger.trace("Expired {} entries", n); - } - }; - service.scheduleWithFixedDelay(runnable, defaultExpiration / 2, defaultExpiration / 2, TimeUnit.MILLISECONDS); - } - - public boolean shutdownBlocking() - { - service.shutdown(); - try - { - return service.awaitTermination(defaultExpiration * 2, TimeUnit.MILLISECONDS); - } - catch (InterruptedException e) - { - throw new AssertionError(e); - } - } - - public void reset() - { - shutdown = false; - cache.clear(); - } - - public V put(K key, V value) - { - return put(key, value, this.defaultExpiration); - } - - public V put(K key, V value, long timeout) - { - if (shutdown) - { - // StorageProxy isn't equipped to deal with "I'm nominally alive, but I can't send any messages out." - // So we'll just sit on this thread until the rest of the server shutdown completes. - // - // See comments in CustomTThreadPoolServer.serve, CASSANDRA-3335, and CASSANDRA-3727. - Uninterruptibles.sleepUninterruptibly(Long.MAX_VALUE, TimeUnit.NANOSECONDS); - } - CacheableObject previous = cache.put(key, new CacheableObject(value, timeout)); - return (previous == null) ? null : previous.value; - } - - public V get(K key) - { - CacheableObject co = cache.get(key); - return co == null ? null : co.value; - } - - public V remove(K key) - { - CacheableObject co = cache.remove(key); - return co == null ? null : co.value; - } - - /** - * @return System.nanoTime() when key was put into the map. - */ - public long getAge(K key) - { - CacheableObject co = cache.get(key); - return co == null ? 0 : co.createdAt; - } - - public int size() - { - return cache.size(); - } - - public boolean containsKey(K key) - { - return cache.containsKey(key); - } - - public boolean isEmpty() - { - return cache.isEmpty(); - } - - public Set keySet() - { - return cache.keySet(); - } -} diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index 129c0f56885b..c37dccacdc97 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -449,12 +449,6 @@ public static T waitOnFuture(Future future) } } - public static void waitOnFutures(List results, long ms) throws TimeoutException - { - for (AsyncOneResponse result : results) - result.get(ms, TimeUnit.MILLISECONDS); - } - public static Future waitOnFirstFuture(Iterable> futures) { return waitOnFirstFuture(futures, 100); diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java index e058ae2bcce6..cdcbf4bdd545 100644 --- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java +++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java @@ -53,7 +53,12 @@ private JVMStabilityInspector() {} * @param t * The Throwable to check for server-stop conditions */ - public static void inspectThrowable(Throwable t) + public static void inspectThrowable(Throwable t) throws OutOfMemoryError + { + inspectThrowable(t, true); + } + + public static void inspectThrowable(Throwable t, boolean propagateOutOfMemory) throws OutOfMemoryError { boolean isUnstable = false; if (t instanceof OutOfMemoryError) @@ -76,6 +81,9 @@ public static void inspectThrowable(Throwable t) StorageService.instance.removeShutdownHook(); // We let the JVM handle the error. The startup checks should have warned the user if it did not configure // the JVM behavior in case of OOM (CASSANDRA-13006). + if (!propagateOutOfMemory) + return; + throw (OutOfMemoryError) t; } diff --git a/src/java/org/apache/cassandra/utils/MonotonicClock.java b/src/java/org/apache/cassandra/utils/MonotonicClock.java new file mode 100644 index 000000000000..5a1aa3c0361e --- /dev/null +++ b/src/java/org/apache/cassandra/utils/MonotonicClock.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.utils; + +import java.lang.reflect.Constructor; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.function.LongSupplier; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.config.Config; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +/** + * Wrapper around time related functions that are either implemented by using the default JVM calls + * or by using a custom implementation for testing purposes. + * + * See {@link #preciseTime} for how to use a custom implementation. + * + * Please note that {@link java.time.Clock} wasn't used, as it would not be possible to provide an + * implementation for {@link #now()} with the exact same properties of {@link System#nanoTime()}. + */ +public interface MonotonicClock +{ + /** + * Static singleton object that will be instantiated by default with a system clock + * implementation. Set cassandra.clock system property to a FQCN to use a + * different implementation instead. + */ + public static final MonotonicClock preciseTime = Defaults.precise(); + public static final MonotonicClock approxTime = Defaults.approx(preciseTime); + + /** + * @see System#nanoTime() + * + * Provides a monotonic time that can be compared with any other such value produced by the same clock + * since the application started only; these times cannot be persisted or serialized to other nodes. + * + * Nanosecond precision. + */ + public long now(); + + /** + * @return nanoseconds of potential error + */ + public long error(); + + public MonotonicClockTranslation translate(); + + public boolean isAfter(long instant); + public boolean isAfter(long now, long instant); + + static class Defaults + { + private static final Logger logger = LoggerFactory.getLogger(MonotonicClock.class); + + private static MonotonicClock precise() + { + String sclock = System.getProperty("cassandra.clock"); + if (sclock == null) + sclock = System.getProperty("cassandra.monotonic_clock.precise"); + + if (sclock != null) + { + try + { + logger.debug("Using custom clock implementation: {}", sclock); + return (MonotonicClock) Class.forName(sclock).newInstance(); + } + catch (Exception e) + { + logger.error(e.getMessage(), e); + } + } + + return new SystemClock(); + } + + private static MonotonicClock approx(MonotonicClock precise) + { + String sclock = System.getProperty("cassandra.monotonic_clock.approx"); + if (sclock != null) + { + try + { + logger.debug("Using custom clock implementation: {}", sclock); + Class clazz = (Class) Class.forName(sclock); + + if (SystemClock.class.equals(clazz) && SystemClock.class.equals(precise.getClass())) + return precise; + + try + { + Constructor withPrecise = clazz.getConstructor(MonotonicClock.class); + return withPrecise.newInstance(precise); + } + catch (NoSuchMethodException nme) + { + } + + return clazz.newInstance(); + } + catch (Exception e) + { + logger.error(e.getMessage(), e); + } + } + + return new SampledClock(precise); + } + } + + static abstract class AbstractEpochSamplingClock implements MonotonicClock + { + private static final Logger logger = LoggerFactory.getLogger(AbstractEpochSamplingClock.class); + private static final String UPDATE_INTERVAL_PROPERTY = Config.PROPERTY_PREFIX + "NANOTIMETOMILLIS_TIMESTAMP_UPDATE_INTERVAL"; + private static final long UPDATE_INTERVAL_MS = Long.getLong(UPDATE_INTERVAL_PROPERTY, 10000); + + private static class AlmostSameTime implements MonotonicClockTranslation + { + final long millisSinceEpoch; + final long monotonicNanos; + final long error; // maximum error of millis measurement (in nanos) + + private AlmostSameTime(long millisSinceEpoch, long monotonicNanos, long errorNanos) + { + this.millisSinceEpoch = millisSinceEpoch; + this.monotonicNanos = monotonicNanos; + this.error = errorNanos; + } + + public long fromMillisSinceEpoch(long currentTimeMillis) + { + return monotonicNanos + MILLISECONDS.toNanos(currentTimeMillis - millisSinceEpoch); + } + + public long toMillisSinceEpoch(long nanoTime) + { + return millisSinceEpoch + TimeUnit.NANOSECONDS.toMillis(nanoTime - monotonicNanos); + } + + public long error() + { + return error; + } + } + + final LongSupplier millisSinceEpoch; + + private volatile AlmostSameTime almostSameTime = new AlmostSameTime(0L, 0L, Long.MAX_VALUE); + private Future almostSameTimeUpdater; + private static double failedAlmostSameTimeUpdateModifier = 1.0; + + AbstractEpochSamplingClock(LongSupplier millisSinceEpoch) + { + this.millisSinceEpoch = millisSinceEpoch; + resumeEpochSampling(); + } + + public MonotonicClockTranslation translate() + { + return almostSameTime; + } + + public synchronized void pauseEpochSampling() + { + if (almostSameTimeUpdater == null) + return; + + almostSameTimeUpdater.cancel(true); + try { almostSameTimeUpdater.get(); } catch (Throwable t) { } + almostSameTimeUpdater = null; + } + + public synchronized void resumeEpochSampling() + { + if (almostSameTimeUpdater != null) + throw new IllegalStateException("Already running"); + updateAlmostSameTime(); + logger.info("Scheduling approximate time conversion task with an interval of {} milliseconds", UPDATE_INTERVAL_MS); + almostSameTimeUpdater = ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(this::updateAlmostSameTime, UPDATE_INTERVAL_MS, UPDATE_INTERVAL_MS, MILLISECONDS); + } + + private void updateAlmostSameTime() + { + final int tries = 3; + long[] samples = new long[2 * tries + 1]; + samples[0] = System.nanoTime(); + for (int i = 1 ; i < samples.length ; i += 2) + { + samples[i] = millisSinceEpoch.getAsLong(); + samples[i + 1] = now(); + } + + int best = 1; + // take sample with minimum delta between calls + for (int i = 3 ; i < samples.length - 1 ; i += 2) + { + if ((samples[i+1] - samples[i-1]) < (samples[best+1]-samples[best-1])) + best = i; + } + + long millis = samples[best]; + long nanos = (samples[best+1] / 2) + (samples[best-1] / 2); + long error = (samples[best+1] / 2) - (samples[best-1] / 2); + + AlmostSameTime prev = almostSameTime; + AlmostSameTime next = new AlmostSameTime(millis, nanos, error); + + if (next.error > prev.error && next.error > prev.error * failedAlmostSameTimeUpdateModifier) + { + failedAlmostSameTimeUpdateModifier *= 1.1; + return; + } + + failedAlmostSameTimeUpdateModifier = 1.0; + almostSameTime = next; + } + } + + public static class SystemClock extends AbstractEpochSamplingClock + { + private SystemClock() + { + super(System::currentTimeMillis); + } + + @Override + public long now() + { + return System.nanoTime(); + } + + @Override + public long error() + { + return 1; + } + + @Override + public boolean isAfter(long instant) + { + return now() > instant; + } + + @Override + public boolean isAfter(long now, long instant) + { + return now > instant; + } + } + + public static class SampledClock implements MonotonicClock + { + private static final Logger logger = LoggerFactory.getLogger(SampledClock.class); + private static final int UPDATE_INTERVAL_MS = Math.max(1, Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "approximate_time_precision_ms", "2"))); + private static final long ERROR_NANOS = MILLISECONDS.toNanos(UPDATE_INTERVAL_MS); + + private final MonotonicClock precise; + + private volatile long almostNow; + private Future almostNowUpdater; + + public SampledClock(MonotonicClock precise) + { + this.precise = precise; + resumeNowSampling(); + } + + @Override + public long now() + { + return almostNow; + } + + @Override + public long error() + { + return ERROR_NANOS; + } + + @Override + public MonotonicClockTranslation translate() + { + return precise.translate(); + } + + @Override + public boolean isAfter(long instant) + { + return isAfter(almostNow, instant); + } + + @Override + public boolean isAfter(long now, long instant) + { + return now - ERROR_NANOS > instant; + } + + public synchronized void pauseNowSampling() + { + if (almostNowUpdater == null) + return; + + almostNowUpdater.cancel(true); + try { almostNowUpdater.get(); } catch (Throwable t) { } + almostNowUpdater = null; + } + + public synchronized void resumeNowSampling() + { + if (almostNowUpdater != null) + throw new IllegalStateException("Already running"); + + almostNow = precise.now(); + logger.info("Scheduling approximate time-check task with a precision of {} milliseconds", UPDATE_INTERVAL_MS); + almostNowUpdater = ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(() -> almostNow = precise.now(), UPDATE_INTERVAL_MS, UPDATE_INTERVAL_MS, MILLISECONDS); + } + + public synchronized void refreshNow() + { + pauseNowSampling(); + resumeNowSampling(); + } + } + +} diff --git a/src/java/org/apache/cassandra/utils/MonotonicClockTranslation.java b/src/java/org/apache/cassandra/utils/MonotonicClockTranslation.java new file mode 100644 index 000000000000..f7f83e471109 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/MonotonicClockTranslation.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +public interface MonotonicClockTranslation +{ + /** accepts millis since epoch, returns nanoTime in the related clock */ + public long fromMillisSinceEpoch(long currentTimeMillis); + /** accepts nanoTime in the related MonotinicClock, returns millis since epoch */ + public long toMillisSinceEpoch(long nanoTime); + /** Nanoseconds of probable error in the translation */ + public long error(); +} diff --git a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java b/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java deleted file mode 100644 index 5aafbe55dc2f..000000000000 --- a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.utils; - -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.concurrent.ScheduledExecutors; -import org.apache.cassandra.config.Config; - -/* - * Convert from nanotime to non-monotonic current time millis. Beware of weaker ordering guarantees. - */ -public class NanoTimeToCurrentTimeMillis -{ - /* - * How often to pull a new timestamp from the system. - */ - private static final String TIMESTAMP_UPDATE_INTERVAL_PROPERTY = Config.PROPERTY_PREFIX + "NANOTIMETOMILLIS_TIMESTAMP_UPDATE_INTERVAL"; - private static final long TIMESTAMP_UPDATE_INTERVAL = Long.getLong(TIMESTAMP_UPDATE_INTERVAL_PROPERTY, 10000); - - private static volatile long TIMESTAMP_BASE[] = new long[] { System.currentTimeMillis(), System.nanoTime() }; - - /* - * System.currentTimeMillis() is 25 nanoseconds. This is 2 nanoseconds (maybe) according to JMH. - * Faster than calling both currentTimeMillis() and nanoTime(). - * - * There is also the issue of how scalable nanoTime() and currentTimeMillis() are which is a moving target. - * - * These timestamps don't order with System.currentTimeMillis() because currentTimeMillis() can tick over - * before this one does. I have seen it behind by as much as 2ms on Linux and 25ms on Windows. - */ - public static long convert(long nanoTime) - { - final long timestampBase[] = TIMESTAMP_BASE; - return timestampBase[0] + TimeUnit.NANOSECONDS.toMillis(nanoTime - timestampBase[1]); - } - - public static void updateNow() - { - ScheduledExecutors.scheduledFastTasks.submit(NanoTimeToCurrentTimeMillis::updateTimestampBase); - } - - static - { - ScheduledExecutors.scheduledFastTasks.scheduleWithFixedDelay(NanoTimeToCurrentTimeMillis::updateTimestampBase, - TIMESTAMP_UPDATE_INTERVAL, - TIMESTAMP_UPDATE_INTERVAL, - TimeUnit.MILLISECONDS); - } - - private static void updateTimestampBase() - { - TIMESTAMP_BASE = new long[] { - Math.max(TIMESTAMP_BASE[0], System.currentTimeMillis()), - Math.max(TIMESTAMP_BASE[1], System.nanoTime()) }; - } -} diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java index ec4e5c849000..5d6d96fac60c 100644 --- a/src/java/org/apache/cassandra/utils/Throwables.java +++ b/src/java/org/apache/cassandra/utils/Throwables.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Iterator; import java.util.Optional; +import java.util.function.Predicate; import java.util.stream.Stream; import org.apache.cassandra.io.FSReadError; @@ -37,6 +38,11 @@ public interface DiscreteAction void perform() throws E; } + public static boolean isCausedBy(Throwable t, Predicate cause) + { + return cause.test(t) || (t.getCause() != null && cause.test(t.getCause())); + } + public static T merge(T existingFail, T newFail) { if (existingFail == null) @@ -74,6 +80,12 @@ public static boolean failIfCanCast(Throwable fail, Class< return true; } + @SafeVarargs + public static void maybeFail(DiscreteAction ... actions) + { + maybeFail(Throwables.perform(null, Stream.of(actions))); + } + @SafeVarargs public static void perform(DiscreteAction ... actions) throws E { @@ -88,7 +100,7 @@ public static void perform(Stream void perform(Stream> actions) throws E { - Throwable fail = perform((Throwable) null, actions); + Throwable fail = perform(null, actions); if (failIfCanCast(fail, null)) throw (E) fail; } diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java index 3c1b7cc1cdf7..7986a77ef7c8 100644 --- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java +++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java @@ -49,8 +49,11 @@ import org.apache.cassandra.utils.Pair; import org.cliffc.high_scale_lib.NonBlockingHashMap; +import static com.google.common.collect.ImmutableList.*; import static java.util.Collections.emptyList; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; +import static org.apache.cassandra.utils.ExecutorUtils.shutdownNow; import static org.apache.cassandra.utils.Throwables.maybeFail; import static org.apache.cassandra.utils.Throwables.merge; @@ -705,14 +708,9 @@ private void removeExpected(Set candidates) } @VisibleForTesting - public static void shutdownReferenceReaper() throws InterruptedException + public static void shutdownReferenceReaper(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - EXEC.shutdown(); - EXEC.awaitTermination(60, TimeUnit.SECONDS); - if (STRONG_LEAK_DETECTOR != null) - { - STRONG_LEAK_DETECTOR.shutdownNow(); - STRONG_LEAK_DETECTOR.awaitTermination(60, TimeUnit.SECONDS); - } + shutdownNow(Arrays.asList(EXEC, STRONG_LEAK_DETECTOR)); + awaitTermination(timeout, unit, Arrays.asList(EXEC, STRONG_LEAK_DETECTOR)); } } diff --git a/src/java/org/apache/cassandra/utils/concurrent/SimpleCondition.java b/src/java/org/apache/cassandra/utils/concurrent/SimpleCondition.java index 57614e005caa..0ff90185a5c2 100644 --- a/src/java/org/apache/cassandra/utils/concurrent/SimpleCondition.java +++ b/src/java/org/apache/cassandra/utils/concurrent/SimpleCondition.java @@ -48,10 +48,15 @@ public void await() throws InterruptedException public boolean await(long time, TimeUnit unit) throws InterruptedException { - if (isSignaled()) - return true; long start = System.nanoTime(); long until = start + unit.toNanos(time); + return awaitUntil(until); + } + + public boolean awaitUntil(long deadlineNanos) throws InterruptedException + { + if (isSignaled()) + return true; if (waiting == null) waitingUpdater.compareAndSet(this, null, new WaitQueue()); WaitQueue.Signal s = waiting.register(); @@ -60,7 +65,7 @@ public boolean await(long time, TimeUnit unit) throws InterruptedException s.cancel(); return true; } - return s.awaitUntil(until) || isSignaled(); + return s.awaitUntil(deadlineNanos) || isSignaled(); } public void signal() diff --git a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java index 5b453b06475a..3647623095de 100644 --- a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java +++ b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java @@ -263,6 +263,15 @@ public static interface Signal * @throws InterruptedException */ public boolean awaitUntil(long nanos) throws InterruptedException; + + /** + * Wait until signalled, or the provided time is reached, or the thread is interrupted. If signalled, + * isSignalled() will be true on exit, and the method will return true; if timedout, the method will return + * false and isCancelled() will be true + * @param nanos System.nanoTime() to wait until + * @return true if signalled, false if timed out + */ + public boolean awaitUntilUninterruptibly(long nanos); } /** @@ -306,6 +315,17 @@ public boolean awaitUntil(long until) throws InterruptedException return checkAndClear(); } + public boolean awaitUntilUninterruptibly(long until) + { + long now; + while (until > (now = System.nanoTime()) && !isSignalled()) + { + long delta = until - now; + LockSupport.parkNanos(delta); + } + return checkAndClear(); + } + private void checkInterrupted() throws InterruptedException { if (Thread.interrupted()) diff --git a/src/java/org/apache/cassandra/utils/memory/BufferPool.java b/src/java/org/apache/cassandra/utils/memory/BufferPool.java index a67f5208e6d9..c2e81089c8e3 100644 --- a/src/java/org/apache/cassandra/utils/memory/BufferPool.java +++ b/src/java/org/apache/cassandra/utils/memory/BufferPool.java @@ -21,12 +21,21 @@ import java.lang.ref.PhantomReference; import java.lang.ref.ReferenceQueue; import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayDeque; +import java.util.Collections; import java.util.Queue; +import java.util.Set; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.function.BiPredicate; +import java.util.function.Consumer; +import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; + +import net.nicoulaj.compilecommand.annotations.Inline; import org.apache.cassandra.concurrent.InfiniteLoopExecutor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,29 +45,41 @@ import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.metrics.BufferPoolMetrics; -import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.NoSpamLogger; import org.apache.cassandra.utils.concurrent.Ref; +import static com.google.common.collect.ImmutableList.of; +import static org.apache.cassandra.utils.ExecutorUtils.*; +import static org.apache.cassandra.utils.FBUtilities.prettyPrintMemory; + /** * A pool of ByteBuffers that can be recycled. + * + * TODO: document the semantics of this class carefully + * Notably: we do not automatically release from the local pool any chunk that has been incompletely allocated from */ public class BufferPool { - /** The size of a page aligned buffer, 64KiB */ - public static final int CHUNK_SIZE = 64 << 10; + /** The size of a page aligned buffer, 128KiB */ + public static final int NORMAL_CHUNK_SIZE = 128 << 10; + public static final int NORMAL_ALLOCATION_UNIT = NORMAL_CHUNK_SIZE / 64; + public static final int TINY_CHUNK_SIZE = NORMAL_ALLOCATION_UNIT; + public static final int TINY_ALLOCATION_UNIT = TINY_CHUNK_SIZE / 64; + public static final int TINY_ALLOCATION_LIMIT = TINY_CHUNK_SIZE / 2; + private final static BufferPoolMetrics metrics = new BufferPoolMetrics(); + + // TODO: this should not be using FileCacheSizeInMB @VisibleForTesting public static long MEMORY_USAGE_THRESHOLD = DatabaseDescriptor.getFileCacheSizeInMB() * 1024L * 1024L; @VisibleForTesting public static boolean ALLOCATE_ON_HEAP_WHEN_EXAHUSTED = DatabaseDescriptor.getBufferPoolUseHeapIfExhausted(); - @VisibleForTesting - public static boolean DISABLED = Boolean.parseBoolean(System.getProperty("cassandra.test.disable_buffer_pool", "false")); + private static Debug debug; @VisibleForTesting - public static boolean DEBUG = false; + public static boolean DISABLED = Boolean.parseBoolean(System.getProperty("cassandra.test.disable_buffer_pool", "false")); private static final Logger logger = LoggerFactory.getLogger(BufferPool.class); private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 15L, TimeUnit.MINUTES); @@ -75,6 +96,11 @@ protected LocalPool initialValue() { return new LocalPool(); } + + protected void onRemoval(LocalPool value) + { + value.release(); + } }; public static ByteBuffer get(int size) @@ -82,65 +108,43 @@ public static ByteBuffer get(int size) if (DISABLED) return allocate(size, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); else - return takeFromPool(size, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); + return localPool.get().get(size, false, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); } public static ByteBuffer get(int size, BufferType bufferType) { - boolean direct = bufferType == BufferType.OFF_HEAP; - if (DISABLED || !direct) - return allocate(size, !direct); + boolean onHeap = bufferType == BufferType.ON_HEAP; + if (DISABLED || onHeap) + return allocate(size, onHeap); else - return takeFromPool(size, !direct); + return localPool.get().get(size, false, onHeap); } - /** Unlike the get methods, this will return null if the pool is exhausted */ - public static ByteBuffer tryGet(int size) + public static ByteBuffer getAtLeast(int size, BufferType bufferType) { - if (DISABLED) - return allocate(size, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); + boolean onHeap = bufferType == BufferType.ON_HEAP; + if (DISABLED || onHeap) + return allocate(size, onHeap); else - return maybeTakeFromPool(size, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); + return localPool.get().get(size, true, onHeap); } - private static ByteBuffer allocate(int size, boolean onHeap) + /** Unlike the get methods, this will return null if the pool is exhausted */ + public static ByteBuffer tryGet(int size) { - return onHeap - ? ByteBuffer.allocate(size) - : ByteBuffer.allocateDirect(size); + return localPool.get().tryGet(size, true); } - private static ByteBuffer takeFromPool(int size, boolean allocateOnHeapWhenExhausted) + public static ByteBuffer tryGetAtLeast(int size) { - ByteBuffer ret = maybeTakeFromPool(size, allocateOnHeapWhenExhausted); - if (ret != null) - return ret; - - if (logger.isTraceEnabled()) - logger.trace("Requested buffer size {} has been allocated directly due to lack of capacity", FBUtilities.prettyPrintMemory(size)); - - return localPool.get().allocate(size, allocateOnHeapWhenExhausted); + return localPool.get().tryGet(size, true); } - private static ByteBuffer maybeTakeFromPool(int size, boolean allocateOnHeapWhenExhausted) + private static ByteBuffer allocate(int size, boolean onHeap) { - if (size < 0) - throw new IllegalArgumentException("Size must be positive (" + size + ")"); - - if (size == 0) - return EMPTY_BUFFER; - - if (size > CHUNK_SIZE) - { - if (logger.isTraceEnabled()) - logger.trace("Requested buffer size {} is bigger than {}, allocating directly", - FBUtilities.prettyPrintMemory(size), - FBUtilities.prettyPrintMemory(CHUNK_SIZE)); - - return localPool.get().allocate(size, allocateOnHeapWhenExhausted); - } - - return localPool.get().get(size); + return onHeap + ? ByteBuffer.allocate(size) + : ByteBuffer.allocateDirect(size); } public static void put(ByteBuffer buffer) @@ -149,61 +153,43 @@ public static void put(ByteBuffer buffer) localPool.get().put(buffer); } - /** This is not thread safe and should only be used for unit testing. */ - @VisibleForTesting - static void reset() + public static void putUnusedPortion(ByteBuffer buffer) { - localPool.get().reset(); - globalPool.reset(); + + if (!(DISABLED || buffer.hasArray())) + { + LocalPool pool = localPool.get(); + if (buffer.limit() > 0) + pool.putUnusedPortion(buffer); + else + pool.put(buffer); + } } - @VisibleForTesting - static Chunk currentChunk() + public static void setRecycleWhenFreeForCurrentThread(boolean recycleWhenFree) { - return localPool.get().chunks[0]; + localPool.get().recycleWhenFree(recycleWhenFree); } - @VisibleForTesting - static int numChunks() + public static long sizeInBytes() { - int ret = 0; - for (Chunk chunk : localPool.get().chunks) - { - if (chunk != null) - ret++; - } - return ret; + return globalPool.sizeInBytes(); } - @VisibleForTesting - static void assertAllRecycled() + interface Debug { - globalPool.debug.check(); + void registerNormal(Chunk chunk); + void recycleNormal(Chunk oldVersion, Chunk newVersion); } - public static long sizeInBytes() + public static void debug(Debug setDebug) { - return globalPool.sizeInBytes(); + debug = setDebug; } - static final class Debug + interface Recycler { - long recycleRound = 1; - final Queue allChunks = new ConcurrentLinkedQueue<>(); - void register(Chunk chunk) - { - allChunks.add(chunk); - } - void recycle(Chunk chunk) - { - chunk.lastRecycled = recycleRound; - } - void check() - { - for (Chunk chunk : allChunks) - assert chunk.lastRecycled == recycleRound; - recycleRound++; - } + void recycle(Chunk chunk); } /** @@ -213,26 +199,25 @@ void check() * * This class is shared by multiple thread local pools and must be thread-safe. */ - static final class GlobalPool + static final class GlobalPool implements Supplier, Recycler { - /** The size of a bigger chunk, 1-mbit, must be a multiple of CHUNK_SIZE */ - static final int MACRO_CHUNK_SIZE = 1 << 20; + /** The size of a bigger chunk, 1 MiB, must be a multiple of NORMAL_CHUNK_SIZE */ + static final int MACRO_CHUNK_SIZE = 64 * NORMAL_CHUNK_SIZE; static { - assert Integer.bitCount(CHUNK_SIZE) == 1; // must be a power of 2 + assert Integer.bitCount(NORMAL_CHUNK_SIZE) == 1; // must be a power of 2 assert Integer.bitCount(MACRO_CHUNK_SIZE) == 1; // must be a power of 2 - assert MACRO_CHUNK_SIZE % CHUNK_SIZE == 0; // must be a multiple + assert MACRO_CHUNK_SIZE % NORMAL_CHUNK_SIZE == 0; // must be a multiple if (DISABLED) logger.info("Global buffer pool is disabled, allocating {}", ALLOCATE_ON_HEAP_WHEN_EXAHUSTED ? "on heap" : "off heap"); else logger.info("Global buffer pool is enabled, when pool is exhausted (max is {}) it will allocate {}", - FBUtilities.prettyPrintMemory(MEMORY_USAGE_THRESHOLD), + prettyPrintMemory(MEMORY_USAGE_THRESHOLD), ALLOCATE_ON_HEAP_WHEN_EXAHUSTED ? "on heap" : "off heap"); } - private final Debug debug = new Debug(); private final Queue macroChunks = new ConcurrentLinkedQueue<>(); // TODO (future): it would be preferable to use a CLStack to improve cache occupancy; it would also be preferable to use "CoreLocal" storage private final Queue chunks = new ConcurrentLinkedQueue<>(); @@ -265,7 +250,8 @@ private Chunk allocateMoreChunks() if (cur + MACRO_CHUNK_SIZE > MEMORY_USAGE_THRESHOLD) { noSpamLogger.info("Maximum memory usage reached ({}), cannot allocate chunk of {}", - MEMORY_USAGE_THRESHOLD, MACRO_CHUNK_SIZE); + prettyPrintMemory(MEMORY_USAGE_THRESHOLD), + prettyPrintMemory(MACRO_CHUNK_SIZE)); return null; } if (memoryUsage.compareAndSet(cur, cur + MACRO_CHUNK_SIZE)) @@ -276,36 +262,41 @@ private Chunk allocateMoreChunks() Chunk chunk; try { - chunk = new Chunk(allocateDirectAligned(MACRO_CHUNK_SIZE)); + chunk = new Chunk(null, allocateDirectAligned(MACRO_CHUNK_SIZE)); } catch (OutOfMemoryError oom) { noSpamLogger.error("Buffer pool failed to allocate chunk of {}, current size {} ({}). " + "Attempting to continue; buffers will be allocated in on-heap memory which can degrade performance. " + "Make sure direct memory size (-XX:MaxDirectMemorySize) is large enough to accommodate off-heap memtables and caches.", - MACRO_CHUNK_SIZE, sizeInBytes(), oom.toString()); + prettyPrintMemory(MACRO_CHUNK_SIZE), + prettyPrintMemory(sizeInBytes()), + oom.toString()); return null; } chunk.acquire(null); macroChunks.add(chunk); - final Chunk callerChunk = new Chunk(chunk.get(CHUNK_SIZE)); - if (DEBUG) - debug.register(callerChunk); - for (int i = CHUNK_SIZE ; i < MACRO_CHUNK_SIZE; i += CHUNK_SIZE) + final Chunk callerChunk = new Chunk(this, chunk.get(NORMAL_CHUNK_SIZE)); + if (debug != null) + debug.registerNormal(callerChunk); + for (int i = NORMAL_CHUNK_SIZE; i < MACRO_CHUNK_SIZE; i += NORMAL_CHUNK_SIZE) { - Chunk add = new Chunk(chunk.get(CHUNK_SIZE)); + Chunk add = new Chunk(this, chunk.get(NORMAL_CHUNK_SIZE)); chunks.add(add); - if (DEBUG) - debug.register(add); + if (debug != null) + debug.registerNormal(add); } return callerChunk; } public void recycle(Chunk chunk) { - chunks.add(chunk); + Chunk recycleAs = new Chunk(chunk); + if (debug != null) + debug.recycleNormal(chunk, recycleAs); + chunks.add(recycleAs); } public long sizeInBytes() @@ -315,25 +306,21 @@ public long sizeInBytes() /** This is not thread safe and should only be used for unit testing. */ @VisibleForTesting - void reset() + void unsafeFree() { while (!chunks.isEmpty()) - chunks.poll().reset(); + chunks.poll().unsafeFree(); while (!macroChunks.isEmpty()) - macroChunks.poll().reset(); + macroChunks.poll().unsafeFree(); memoryUsage.set(0); } } - /** - * A thread local class that grabs chunks from the global pool for this thread allocations. - * Only one thread can do the allocations but multiple threads can release the allocations. - */ - static final class LocalPool + private static class MicroQueueOfChunks { - private final static BufferPoolMetrics metrics = new BufferPoolMetrics(); + // a microqueue of Chunks: // * if any are null, they are at the end; // * new Chunks are added to the last null index @@ -341,92 +328,284 @@ static final class LocalPool // * this results in a queue that will typically be visited in ascending order of available space, so that // small allocations preferentially slice from the Chunks with the smallest space available to furnish them // WARNING: if we ever change the size of this, we must update removeFromLocalQueue, and addChunk - private final Chunk[] chunks = new Chunk[3]; - private byte chunkCount = 0; + private Chunk chunk0, chunk1, chunk2; + private int count; - public LocalPool() + // add a new chunk, if necessary evicting the chunk with the least available memory (returning the evicted chunk) + private Chunk add(Chunk chunk) { - localPoolReferences.add(new LocalPoolRef(this, localPoolRefQueue)); + switch (count) + { + case 0: + chunk0 = chunk; + count = 1; + break; + case 1: + chunk1 = chunk; + count = 2; + break; + case 2: + chunk2 = chunk; + count = 3; + break; + case 3: + { + Chunk release; + int chunk0Free = chunk0.freeSlotCount(); + int chunk1Free = chunk1.freeSlotCount(); + int chunk2Free = chunk2.freeSlotCount(); + if (chunk0Free < chunk1Free) + { + if (chunk0Free < chunk2Free) + { + release = chunk0; + chunk0 = chunk; + } + else + { + release = chunk2; + chunk2 = chunk; + } + } + else + { + if (chunk1Free < chunk2Free) + { + release = chunk1; + chunk1 = chunk; + } + else + { + release = chunk2; + chunk2 = chunk; + } + } + return release; + } + default: + throw new IllegalStateException(); + } + return null; } - private Chunk addChunkFromGlobalPool() + private void remove(Chunk chunk) { - Chunk chunk = globalPool.get(); - if (chunk == null) - return null; + // since we only have three elements in the queue, it is clearer, easier and faster to just hard code the options + if (chunk0 == chunk) + { // remove first by shifting back second two + chunk0 = chunk1; + chunk1 = chunk2; + } + else if (chunk1 == chunk) + { // remove second by shifting back last + chunk1 = chunk2; + } + else if (chunk2 != chunk) + { + return; + } + // whatever we do, the last element must be null + chunk2 = null; + --count; + } - addChunk(chunk); - return chunk; + ByteBuffer get(int size, boolean sizeIsLowerBound, ByteBuffer reuse) + { + ByteBuffer buffer; + if (null != chunk0) + { + if (null != (buffer = chunk0.get(size, sizeIsLowerBound, reuse))) + return buffer; + if (null != chunk1) + { + if (null != (buffer = chunk1.get(size, sizeIsLowerBound, reuse))) + return buffer; + if (null != chunk2 && null != (buffer = chunk2.get(size, sizeIsLowerBound, reuse))) + return buffer; + } + } + return null; } - private void addChunk(Chunk chunk) + private void forEach(Consumer consumer) { - chunk.acquire(this); + forEach(consumer, count, chunk0, chunk1, chunk2); + } - if (chunkCount < 3) + private void clearForEach(Consumer consumer) + { + Chunk chunk0 = this.chunk0, chunk1 = this.chunk1, chunk2 = this.chunk2; + this.chunk0 = this.chunk1 = this.chunk2 = null; + forEach(consumer, count, chunk0, chunk1, chunk2); + count = 0; + } + + private static void forEach(Consumer consumer, int count, Chunk chunk0, Chunk chunk1, Chunk chunk2) + { + switch (count) { - chunks[chunkCount++] = chunk; - return; + case 3: + consumer.accept(chunk2); + case 2: + consumer.accept(chunk1); + case 1: + consumer.accept(chunk0); } + } - int smallestChunkIdx = 0; - if (chunks[1].free() < chunks[0].free()) - smallestChunkIdx = 1; - if (chunks[2].free() < chunks[smallestChunkIdx].free()) - smallestChunkIdx = 2; + private void removeIf(BiPredicate predicate, T value) + { + switch (count) + { + case 3: + if (predicate.test(chunk2, value)) + { + --count; + Chunk chunk = chunk2; + chunk2 = null; + chunk.release(); + } + case 2: + if (predicate.test(chunk1, value)) + { + --count; + Chunk chunk = chunk1; + chunk1 = null; + chunk.release(); + } + case 1: + if (predicate.test(chunk0, value)) + { + --count; + Chunk chunk = chunk0; + chunk0 = null; + chunk.release(); + } + break; + case 0: + return; + } + switch (count) + { + case 2: + // Find the only null item, and shift non-null so that null is at chunk2 + if (chunk0 == null) + { + chunk0 = chunk1; + chunk1 = chunk2; + chunk2 = null; + } + else if (chunk1 == null) + { + chunk1 = chunk2; + chunk2 = null; + } + break; + case 1: + // Find the only non-null item, and shift it to chunk0 + if (chunk1 != null) + { + chunk0 = chunk1; + chunk1 = null; + } + else if (chunk2 != null) + { + chunk0 = chunk2; + chunk2 = null; + } + break; + } + } - chunks[smallestChunkIdx].release(); - if (smallestChunkIdx != 2) - chunks[smallestChunkIdx] = chunks[2]; - chunks[2] = chunk; + private void release() + { + clearForEach(Chunk::release); } - public ByteBuffer get(int size) + private void unsafeRecycle() { - for (Chunk chunk : chunks) - { // first see if our own chunks can serve this buffer - if (chunk == null) - break; + clearForEach(Chunk::unsafeRecycle); + } + } - ByteBuffer buffer = chunk.get(size); - if (buffer != null) - return buffer; - } + /** + * A thread local class that grabs chunks from the global pool for this thread allocations. + * Only one thread can do the allocations but multiple threads can release the allocations. + */ + public static final class LocalPool implements Recycler + { + private final Queue reuseObjects; + private final Supplier parent; + private final LocalPoolRef leakRef; - // else ask the global pool - Chunk chunk = addChunkFromGlobalPool(); - if (chunk != null) - return chunk.get(size); + private final MicroQueueOfChunks chunks = new MicroQueueOfChunks(); + /** + * If we are on outer LocalPool, whose chunks are == NORMAL_CHUNK_SIZE, we may service allocation requests + * for buffers much smaller than + */ + private LocalPool tinyPool; + private final int tinyLimit; + private boolean recycleWhenFree = true; - return null; + public LocalPool() + { + this.parent = globalPool; + this.tinyLimit = TINY_ALLOCATION_LIMIT; + this.reuseObjects = new ArrayDeque<>(); + localPoolReferences.add(leakRef = new LocalPoolRef(this, localPoolRefQueue)); } - private ByteBuffer allocate(int size, boolean onHeap) + /** + * Invoked by an existing LocalPool, to create a child pool + */ + private LocalPool(LocalPool parent) { - metrics.misses.mark(); - return BufferPool.allocate(size, onHeap); + this.parent = () -> { + ByteBuffer buffer = parent.tryGetInternal(TINY_CHUNK_SIZE, false); + if (buffer == null) + return null; + return new Chunk(parent, buffer); + }; + this.tinyLimit = 0; // we only currently permit one layer of nesting (which brings us down to 32 byte allocations, so is plenty) + this.reuseObjects = parent.reuseObjects; // we share the same ByteBuffer object reuse pool, as we both have the same exclusive access to it + localPoolReferences.add(leakRef = new LocalPoolRef(this, localPoolRefQueue)); + } + + private LocalPool tinyPool() + { + if (tinyPool == null) + tinyPool = new LocalPool(this).recycleWhenFree(recycleWhenFree); + return tinyPool; } public void put(ByteBuffer buffer) { Chunk chunk = Chunk.getParentChunk(buffer); if (chunk == null) - { FileUtils.clean(buffer); + else + put(buffer, chunk); + } + + public void put(ByteBuffer buffer, Chunk chunk) + { + LocalPool owner = chunk.owner; + if (owner != null && owner == tinyPool) + { + tinyPool.put(buffer, chunk); return; } - LocalPool owner = chunk.owner; // ask the free method to take exclusive ownership of the act of recycling // if we are either: already not owned by anyone, or owned by ourselves - long free = chunk.free(buffer, owner == null | owner == this); + long free = chunk.free(buffer, owner == null || (owner == this && recycleWhenFree)); if (free == 0L) { // 0L => we own recycling responsibility, so must recycle; - chunk.recycle(); - // if we are also the owner, we must remove the Chunk from our local queue + // if we are the owner, we must remove the Chunk from our local queue if (owner == this) - removeFromLocalQueue(chunk); + remove(chunk); + chunk.recycle(); } else if (((free == -1L) && owner != this) && chunk.owner == null) { @@ -434,46 +613,186 @@ else if (((free == -1L) && owner != this) && chunk.owner == null) // we must also check after completely freeing if the owner has since been unset, and try to recycle chunk.tryRecycle(); } + + if (owner == this) + { + MemoryUtil.setAttachment(buffer, null); + MemoryUtil.setDirectByteBuffer(buffer, 0, 0); + reuseObjects.add(buffer); + } } - private void removeFromLocalQueue(Chunk chunk) + public void putUnusedPortion(ByteBuffer buffer) { - // since we only have three elements in the queue, it is clearer, easier and faster to just hard code the options - if (chunks[0] == chunk) - { // remove first by shifting back second two - chunks[0] = chunks[1]; - chunks[1] = chunks[2]; + Chunk chunk = Chunk.getParentChunk(buffer); + if (chunk == null) + return; + + chunk.freeUnusedPortion(buffer); + } + + public ByteBuffer get(int size) + { + return get(size, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); + } + + public ByteBuffer get(int size, boolean allocateOnHeapWhenExhausted) + { + return get(size, false, allocateOnHeapWhenExhausted); + } + + public ByteBuffer getAtLeast(int size) + { + return getAtLeast(size, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); + } + + public ByteBuffer getAtLeast(int size, boolean allocateOnHeapWhenExhausted) + { + return get(size, true, allocateOnHeapWhenExhausted); + } + + private ByteBuffer get(int size, boolean sizeIsLowerBound, boolean allocateOnHeapWhenExhausted) + { + ByteBuffer ret = tryGet(size, sizeIsLowerBound); + if (ret != null) + return ret; + + if (size > NORMAL_CHUNK_SIZE) + { + if (logger.isTraceEnabled()) + logger.trace("Requested buffer size {} is bigger than {}; allocating directly", + prettyPrintMemory(size), + prettyPrintMemory(NORMAL_CHUNK_SIZE)); } - else if (chunks[1] == chunk) - { // remove second by shifting back last - chunks[1] = chunks[2]; + else + { + if (logger.isTraceEnabled()) + logger.trace("Requested buffer size {} has been allocated directly due to lack of capacity", prettyPrintMemory(size)); } - else assert chunks[2] == chunk; - // whatever we do, the last element myst be null - chunks[2] = null; - chunkCount--; + + metrics.misses.mark(); + return allocate(size, allocateOnHeapWhenExhausted); } - @VisibleForTesting - void reset() + public ByteBuffer tryGet(int size) + { + return tryGet(size, ALLOCATE_ON_HEAP_WHEN_EXAHUSTED); + } + + public ByteBuffer tryGetAtLeast(int size) { - chunkCount = 0; - for (int i = 0; i < chunks.length; i++) + return tryGet(size, true); + } + + private ByteBuffer tryGet(int size, boolean sizeIsLowerBound) + { + LocalPool pool = this; + if (size <= tinyLimit) { - if (chunks[i] != null) + if (size <= 0) { - chunks[i].owner = null; - chunks[i].freeSlots = 0L; - chunks[i].recycle(); - chunks[i] = null; + if (size == 0) + return EMPTY_BUFFER; + throw new IllegalArgumentException("Size must be non-negative (" + size + ')'); } + + pool = tinyPool(); } + else if (size > NORMAL_CHUNK_SIZE) + { + return null; + } + + return pool.tryGetInternal(size, sizeIsLowerBound); + } + + @Inline + private ByteBuffer tryGetInternal(int size, boolean sizeIsLowerBound) + { + ByteBuffer reuse = this.reuseObjects.poll(); + ByteBuffer buffer = chunks.get(size, sizeIsLowerBound, reuse); + if (buffer != null) + return buffer; + + // else ask the global pool + Chunk chunk = addChunkFromParent(); + if (chunk != null) + { + ByteBuffer result = chunk.get(size, sizeIsLowerBound, reuse); + if (result != null) + return result; + } + + if (reuse != null) + this.reuseObjects.add(reuse); + return null; + } + + // recycle + public void recycle(Chunk chunk) + { + ByteBuffer buffer = chunk.slab; + Chunk parentChunk = Chunk.getParentChunk(buffer); + put(buffer, parentChunk); + } + + private void remove(Chunk chunk) + { + chunks.remove(chunk); + if (tinyPool != null) + tinyPool.chunks.removeIf((child, parent) -> Chunk.getParentChunk(child.slab) == parent, chunk); + } + + private Chunk addChunkFromParent() + { + Chunk chunk = parent.get(); + if (chunk == null) + return null; + + addChunk(chunk); + return chunk; + } + + private void addChunk(Chunk chunk) + { + chunk.acquire(this); + Chunk evict = chunks.add(chunk); + if (evict != null) + { + if (tinyPool != null) + tinyPool.chunks.removeIf((child, parent) -> Chunk.getParentChunk(child.slab) == parent, evict); + evict.release(); + } + } + + public void release() + { + chunks.release(); + reuseObjects.clear(); + localPoolReferences.remove(leakRef); + leakRef.clear(); + if (tinyPool != null) + tinyPool.release(); + } + + @VisibleForTesting + void unsafeRecycle() + { + chunks.unsafeRecycle(); + } + + public LocalPool recycleWhenFree(boolean recycleWhenFree) + { + this.recycleWhenFree = recycleWhenFree; + if (tinyPool != null) + tinyPool.recycleWhenFree = recycleWhenFree; + return this; } } - private static final class LocalPoolRef extends PhantomReference + private static final class LocalPoolRef extends PhantomReference { - private final Chunk[] chunks; + private final MicroQueueOfChunks chunks; public LocalPoolRef(LocalPool localPool, ReferenceQueue q) { super(localPool, q); @@ -482,18 +801,11 @@ public LocalPoolRef(LocalPool localPool, ReferenceQueue q) public void release() { - for (int i = 0 ; i < chunks.length ; i++) - { - if (chunks[i] != null) - { - chunks[i].release(); - chunks[i] = null; - } - } + chunks.release(); } } - private static final ConcurrentLinkedQueue localPoolReferences = new ConcurrentLinkedQueue<>(); + private static final Set localPoolReferences = Collections.newSetFromMap(new ConcurrentHashMap<>()); private static final ReferenceQueue localPoolRefQueue = new ReferenceQueue<>(); private static final InfiniteLoopExecutor EXEC = new InfiniteLoopExecutor("LocalPool-Cleaner", BufferPool::cleanupOneReference).start(); @@ -561,8 +873,10 @@ final static class Chunk // if this is set, it means the chunk may not be recycled because we may still allocate from it; // if it has been unset the local pool has finished with it, and it may be recycled private volatile LocalPool owner; - private long lastRecycled; - private final Chunk original; + private final Recycler recycler; + + @VisibleForTesting + Object debugAttachment; Chunk(Chunk recycle) { @@ -571,14 +885,13 @@ final static class Chunk this.baseAddress = recycle.baseAddress; this.shift = recycle.shift; this.freeSlots = -1L; - this.original = recycle.original; - if (DEBUG) - globalPool.debug.recycle(original); + this.recycler = recycle.recycler; } - Chunk(ByteBuffer slab) + Chunk(Recycler recycler, ByteBuffer slab) { assert !slab.hasArray(); + this.recycler = recycler; this.slab = slab; this.baseAddress = MemoryUtil.getAddress(slab); @@ -587,7 +900,6 @@ final static class Chunk this.shift = 31 & (Integer.numberOfTrailingZeros(slab.capacity() / 64)); // -1 means all free whilst 0 means all in use this.freeSlots = slab.capacity() == 0 ? 0L : -1L; - this.original = DEBUG ? this : null; } /** @@ -621,7 +933,7 @@ void tryRecycle() void recycle() { assert freeSlots == 0L; - globalPool.recycle(new Chunk(this)); + recycler.recycle(this); } /** @@ -642,14 +954,12 @@ static Chunk getParentChunk(ByteBuffer buffer) return null; } - ByteBuffer setAttachment(ByteBuffer buffer) + void setAttachment(ByteBuffer buffer) { if (Ref.DEBUG_ENABLED) MemoryUtil.setAttachment(buffer, new Ref<>(this, null)); else MemoryUtil.setAttachment(buffer, this); - - return buffer; } boolean releaseAttachment(ByteBuffer buffer) @@ -658,22 +968,12 @@ boolean releaseAttachment(ByteBuffer buffer) if (attachment == null) return false; - if (attachment instanceof Ref) + if (Ref.DEBUG_ENABLED) ((Ref) attachment).release(); return true; } - @VisibleForTesting - void reset() - { - Chunk parent = getParentChunk(slab); - if (parent != null) - parent.free(slab, false); - else - FileUtils.clean(slab); - } - @VisibleForTesting long setFreeSlots(long val) { @@ -703,15 +1003,27 @@ int free() return Long.bitCount(freeSlots) * unit(); } + int freeSlotCount() + { + return Long.bitCount(freeSlots); + } + + ByteBuffer get(int size) + { + return get(size, false, null); + } + /** * Return the next available slice of this size. If * we have exceeded the capacity we return null. */ - ByteBuffer get(int size) + ByteBuffer get(int size, boolean sizeIsLowerBound, ByteBuffer into) { // how many multiples of our units is the size? // we add (unit - 1), so that when we divide by unit (>>> shift), we effectively round up int slotCount = (size - 1 + unit()) >>> shift; + if (sizeIsLowerBound) + size = slotCount << shift; // if we require more than 64 slots, we cannot possibly accommodate the allocation if (slotCount > 64) @@ -775,17 +1087,18 @@ ByteBuffer get(int size) // make sure no other thread has cleared the candidate bits assert ((candidate & cur) == candidate); } - return get(index << shift, size); + return set(index << shift, size, into); } } } - private ByteBuffer get(int offset, int size) + private ByteBuffer set(int offset, int size, ByteBuffer into) { - slab.limit(offset + size); - slab.position(offset); - - return setAttachment(slab.slice()); + if (into == null) + into = MemoryUtil.getHollowDirectByteBuffer(ByteOrder.BIG_ENDIAN); + MemoryUtil.sliceDirectByteBuffer(slab, into, offset, size); + setAttachment(into); + return into; } /** @@ -807,25 +1120,16 @@ long free(ByteBuffer buffer, boolean tryRelease) if (!releaseAttachment(buffer)) return 1L; + int size = roundUp(buffer.capacity()); long address = MemoryUtil.getAddress(buffer); - assert (address >= baseAddress) & (address <= baseAddress + capacity()); + assert (address >= baseAddress) & (address + size <= baseAddress + capacity()); - int position = (int)(address - baseAddress); - int size = roundUp(buffer.capacity()); + int position = ((int)(address - baseAddress)) >> shift; - position >>= shift; int slotCount = size >> shift; - - long slotBits = (1L << slotCount) - 1; + long slotBits = 0xffffffffffffffffL >>> (64 - slotCount); long shiftedSlotBits = (slotBits << position); - if (slotCount == 64) - { - assert size == capacity(); - assert position == 0; - shiftedSlotBits = -1L; - } - long next; while (true) { @@ -839,29 +1143,121 @@ long free(ByteBuffer buffer, boolean tryRelease) } } + void freeUnusedPortion(ByteBuffer buffer) + { + int size = roundUp(buffer.limit()); + int capacity = roundUp(buffer.capacity()); + if (size == capacity) + return; + + long address = MemoryUtil.getAddress(buffer); + assert (address >= baseAddress) & (address + size <= baseAddress + capacity()); + + // free any spare slots above the size we are using + int position = ((int)(address + size - baseAddress)) >> shift; + int slotCount = (capacity - size) >> shift; + + long slotBits = 0xffffffffffffffffL >>> (64 - slotCount); + long shiftedSlotBits = (slotBits << position); + + long next; + while (true) + { + long cur = freeSlots; + next = cur | shiftedSlotBits; + assert next == (cur ^ shiftedSlotBits); // ensure no double free + if (freeSlotsUpdater.compareAndSet(this, cur, next)) + break; + } + MemoryUtil.setByteBufferCapacity(buffer, size); + } + @Override public String toString() { return String.format("[slab %s, slots bitmap %s, capacity %d, free %d]", slab, Long.toBinaryString(freeSlots), capacity(), free()); } + + @VisibleForTesting + void unsafeFree() + { + Chunk parent = getParentChunk(slab); + if (parent != null) + parent.free(slab, false); + else + FileUtils.clean(slab); + } + + static void unsafeRecycle(Chunk chunk) + { + if (chunk != null) + { + chunk.owner = null; + chunk.freeSlots = 0L; + chunk.recycle(); + } + } } @VisibleForTesting - public static int roundUpNormal(int size) + public static int roundUp(int size) { - return roundUp(size, CHUNK_SIZE / 64); + if (size <= TINY_ALLOCATION_LIMIT) + return roundUp(size, TINY_ALLOCATION_UNIT); + return roundUp(size, NORMAL_ALLOCATION_UNIT); } - private static int roundUp(int size, int unit) + @VisibleForTesting + public static int roundUp(int size, int unit) { int mask = unit - 1; return (size + mask) & ~mask; } @VisibleForTesting - public static void shutdownLocalCleaner() throws InterruptedException + public static void shutdownLocalCleaner(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - EXEC.shutdown(); - EXEC.awaitTermination(60, TimeUnit.SECONDS); + shutdownNow(of(EXEC)); + awaitTermination(timeout, unit, of(EXEC)); } + + public static long unsafeGetBytesInUse() + { + long totalMemory = globalPool.memoryUsage.get(); + class L { long v; } + final L availableMemory = new L(); + for (Chunk chunk : globalPool.chunks) + { + availableMemory.v += chunk.capacity(); + } + for (LocalPoolRef ref : localPoolReferences) + { + ref.chunks.forEach(chunk -> availableMemory.v += chunk.free()); + } + return totalMemory - availableMemory.v; + } + + /** This is not thread safe and should only be used for unit testing. */ + @VisibleForTesting + static void unsafeReset() + { + localPool.get().unsafeRecycle(); + globalPool.unsafeFree(); + } + + @VisibleForTesting + static Chunk unsafeCurrentChunk() + { + return localPool.get().chunks.chunk0; + } + + @VisibleForTesting + static int unsafeNumChunks() + { + LocalPool pool = localPool.get(); + return (pool.chunks.chunk0 != null ? 1 : 0) + + (pool.chunks.chunk1 != null ? 1 : 0) + + (pool.chunks.chunk2 != null ? 1 : 0); + } + } diff --git a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java index 998cbbf1a0d7..6b30f44aa6cb 100644 --- a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java +++ b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java @@ -157,7 +157,7 @@ public static ByteBuffer getByteBuffer(long address, int length) public static ByteBuffer getByteBuffer(long address, int length, ByteOrder order) { ByteBuffer instance = getHollowDirectByteBuffer(order); - setByteBuffer(instance, address, length); + setDirectByteBuffer(instance, address, length); return instance; } @@ -196,13 +196,6 @@ public static ByteBuffer getHollowByteBuffer() return instance; } - public static void setByteBuffer(ByteBuffer instance, long address, int length) - { - unsafe.putLong(instance, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, address); - unsafe.putInt(instance, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, length); - unsafe.putInt(instance, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, length); - } - public static Object getAttachment(ByteBuffer instance) { assert instance.getClass() == DIRECT_BYTE_BUFFER_CLASS; @@ -225,6 +218,26 @@ public static ByteBuffer duplicateDirectByteBuffer(ByteBuffer source, ByteBuffer return hollowBuffer; } + public static ByteBuffer sliceDirectByteBuffer(ByteBuffer source, ByteBuffer hollowBuffer, int offset, int length) + { + assert source.getClass() == DIRECT_BYTE_BUFFER_CLASS || source.getClass() == RO_DIRECT_BYTE_BUFFER_CLASS; + setDirectByteBuffer(hollowBuffer, offset + unsafe.getLong(source, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET), length); + return hollowBuffer; + } + + public static void setDirectByteBuffer(ByteBuffer instance, long address, int length) + { + unsafe.putLong(instance, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, address); + unsafe.putInt(instance, DIRECT_BYTE_BUFFER_POSITION_OFFSET, 0); + unsafe.putInt(instance, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, length); + unsafe.putInt(instance, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, length); + } + + public static void setByteBufferCapacity(ByteBuffer instance, int capacity) + { + unsafe.putInt(instance, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, capacity); + } + public static long getLongByByte(long address) { if (BIG_ENDIAN) diff --git a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java index 684db93bb36d..fdcd948fa10d 100644 --- a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java +++ b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java @@ -19,6 +19,7 @@ package org.apache.cassandra.utils.memory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import com.google.common.annotations.VisibleForTesting; @@ -66,10 +67,11 @@ MemtableCleanerThread getCleaner(Runnable cleaner) return cleaner == null ? null : new MemtableCleanerThread<>(this, cleaner); } - public void shutdown() throws InterruptedException + public void shutdown(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - cleaner.shutdown(); - cleaner.awaitTermination(60, TimeUnit.SECONDS); + cleaner.shutdownNow(); + if (!cleaner.awaitTermination(timeout, unit)) + throw new TimeoutException(); } public abstract MemtableAllocator newAllocator(); diff --git a/src/java/org/apache/cassandra/utils/vint/VIntCoding.java b/src/java/org/apache/cassandra/utils/vint/VIntCoding.java index d4fdd630ea5d..6961d9f19148 100644 --- a/src/java/org/apache/cassandra/utils/vint/VIntCoding.java +++ b/src/java/org/apache/cassandra/utils/vint/VIntCoding.java @@ -49,10 +49,11 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.nio.ByteBuffer; -import io.netty.buffer.ByteBuf; import io.netty.util.concurrent.FastThreadLocal; import net.nicoulaj.compilecommand.annotations.Inline; +import org.apache.cassandra.io.util.DataInputPlus; /** * Borrows idea from @@ -82,37 +83,45 @@ public static long readUnsignedVInt(DataInput input) throws IOException return retval; } + public static void skipUnsignedVInt(DataInputPlus input) throws IOException + { + int firstByte = input.readByte(); + if (firstByte < 0) + input.skipBytesFully(numberOfExtraBytesToRead(firstByte)); + } + /** * Note this method is the same as {@link #readUnsignedVInt(DataInput)}, * except that we do *not* block if there are not enough bytes in the buffer * to reconstruct the value. * + * WARNING: this method is only safe for vints we know to be representable by a positive long value. + * * @return -1 if there are not enough bytes in the input to read the value; else, the vint unsigned value. */ - public static long readUnsignedVInt(ByteBuf input) + public static long getUnsignedVInt(ByteBuffer input, int readerIndex) { - if (!input.isReadable()) + return getUnsignedVInt(input, readerIndex, input.limit()); + } + public static long getUnsignedVInt(ByteBuffer input, int readerIndex, int readerLimit) + { + if (readerIndex >= readerLimit) return -1; - input.markReaderIndex(); - int firstByte = input.readByte(); + int firstByte = input.get(readerIndex++); //Bail out early if this is one byte, necessary or it fails later if (firstByte >= 0) return firstByte; int size = numberOfExtraBytesToRead(firstByte); - - if (input.readableBytes() < size) - { - input.resetReaderIndex(); + if (readerIndex + size > readerLimit) return -1; - } long retval = firstByte & firstByteValueMask(size); for (int ii = 0; ii < size; ii++) { - byte b = input.readByte(); + byte b = input.get(readerIndex++); retval <<= 8; retval |= b & 0xff; } @@ -120,6 +129,24 @@ public static long readUnsignedVInt(ByteBuf input) return retval; } + /** + * Computes size of an unsigned vint that starts at readerIndex of the provided ByteBuf. + * + * @return -1 if there are not enough bytes in the input to calculate the size; else, the vint unsigned value size in bytes. + */ + public static int computeUnsignedVIntSize(ByteBuffer input, int readerIndex) + { + return computeUnsignedVIntSize(input, readerIndex, input.limit()); + } + public static int computeUnsignedVIntSize(ByteBuffer input, int readerIndex, int readerLimit) + { + if (readerIndex >= readerLimit) + return -1; + + int firstByte = input.get(readerIndex); + return 1 + ((firstByte >= 0) ? 0 : numberOfExtraBytesToRead(firstByte)); + } + public static long readVInt(DataInput input) throws IOException { return decodeZigZag64(readUnsignedVInt(input)); @@ -155,6 +182,7 @@ public byte[] initialValue() } }; + @Inline public static void writeUnsignedVInt(long value, DataOutput output) throws IOException { int size = VIntCoding.computeUnsignedVIntSize(value); @@ -164,24 +192,47 @@ public static void writeUnsignedVInt(long value, DataOutput output) throws IOExc return; } - output.write(VIntCoding.encodeVInt(value, size), 0, size); + output.write(VIntCoding.encodeUnsignedVInt(value, size), 0, size); } @Inline - public static byte[] encodeVInt(long value, int size) + public static void writeUnsignedVInt(long value, ByteBuffer output) { - byte encodingSpace[] = encodingBuffer.get(); - int extraBytes = size - 1; + int size = VIntCoding.computeUnsignedVIntSize(value); + if (size == 1) + { + output.put((byte) value); + return; + } + + output.put(VIntCoding.encodeUnsignedVInt(value, size), 0, size); + } + /** + * @return a TEMPORARY THREAD LOCAL BUFFER containing the encoded bytes of the value + * This byte[] must be discarded by the caller immediately, and synchronously + */ + @Inline + private static byte[] encodeUnsignedVInt(long value, int size) + { + byte[] encodingSpace = encodingBuffer.get(); + encodeUnsignedVInt(value, size, encodingSpace); + return encodingSpace; + } + + @Inline + private static void encodeUnsignedVInt(long value, int size, byte[] encodeInto) + { + int extraBytes = size - 1; for (int i = extraBytes ; i >= 0; --i) { - encodingSpace[i] = (byte) value; + encodeInto[i] = (byte) value; value >>= 8; } - encodingSpace[0] |= VIntCoding.encodeExtraBytesToRead(extraBytes); - return encodingSpace; + encodeInto[0] |= VIntCoding.encodeExtraBytesToRead(extraBytes); } + @Inline public static void writeVInt(long value, DataOutput output) throws IOException { writeUnsignedVInt(encodeZigZag64(value), output); diff --git a/test/burn/org/apache/cassandra/net/BytesInFlightController.java b/test/burn/org/apache/cassandra/net/BytesInFlightController.java new file mode 100644 index 000000000000..edd9a7e1b91e --- /dev/null +++ b/test/burn/org/apache/cassandra/net/BytesInFlightController.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.locks.LockSupport; +import java.util.function.IntConsumer; + +import org.apache.cassandra.utils.Pair; + +public class BytesInFlightController +{ + private static final AtomicLongFieldUpdater sentBytesUpdater = AtomicLongFieldUpdater.newUpdater(BytesInFlightController.class, "sentBytes"); + private static final AtomicLongFieldUpdater receivedBytesUpdater = AtomicLongFieldUpdater.newUpdater(BytesInFlightController.class, "receivedBytes"); + + private volatile long minimumInFlightBytes, maximumInFlightBytes; + private volatile long sentBytes; + private volatile long receivedBytes; + private final ConcurrentLinkedQueue> deferredBytes = new ConcurrentLinkedQueue<>(); + private final ConcurrentSkipListMap waitingToSend = new ConcurrentSkipListMap<>(); + + BytesInFlightController(long maximumInFlightBytes) + { + this.maximumInFlightBytes = maximumInFlightBytes; + } + + public void send(long bytes) throws InterruptedException + { + long sentBytes = sentBytesUpdater.getAndAdd(this, bytes); + maybeProcessDeferred(); + if ((sentBytes - receivedBytes) >= maximumInFlightBytes) + { + long waitUntilReceived = sentBytes - maximumInFlightBytes; + // overlap shouldn't occur, but cannot guarantee it when we modify maximumInFlightBytes + Thread prev = waitingToSend.putIfAbsent(waitUntilReceived, Thread.currentThread()); + while (prev != null) + prev = waitingToSend.putIfAbsent(++waitUntilReceived, Thread.currentThread()); + + boolean isInterrupted; + while (!(isInterrupted = Thread.currentThread().isInterrupted()) + && waitUntilReceived - receivedBytes >= 0 + && waitingToSend.get(waitUntilReceived) != null) + { + LockSupport.park(); + } + waitingToSend.remove(waitUntilReceived); + + if (isInterrupted) + throw new InterruptedException(); + } + } + + public long minimumInFlightBytes() { return minimumInFlightBytes; } + public long maximumInFlightBytes() { return maximumInFlightBytes; } + + void adjust(int predictedSentBytes, int actualSentBytes) + { + receivedBytesUpdater.addAndGet(this, predictedSentBytes - actualSentBytes); + if (predictedSentBytes > actualSentBytes) wakeupSenders(); + else maybeProcessDeferred(); + } + + public long inFlight() + { + return sentBytes - receivedBytes; + } + + public void fail(int bytes) + { + receivedBytesUpdater.addAndGet(this, bytes); + wakeupSenders(); + } + + public void process(int bytes, IntConsumer releaseBytes) + { + while (true) + { + long sent = sentBytes; + long received = receivedBytes; + long newReceived = received + bytes; + if (sent - newReceived <= minimumInFlightBytes) + { + deferredBytes.add(Pair.create(bytes, releaseBytes)); + break; + } + if (receivedBytesUpdater.compareAndSet(this, received, newReceived)) + { + releaseBytes.accept(bytes); + break; + } + } + maybeProcessDeferred(); + wakeupSenders(); + } + + void setInFlightByteBounds(long minimumInFlightBytes, long maximumInFlightBytes) + { + this.minimumInFlightBytes = minimumInFlightBytes; + this.maximumInFlightBytes = maximumInFlightBytes; + maybeProcessDeferred(); + } + + // unlike the rest of the class, this method does not handle wrap-around of sent/received; + // since this shouldn't happen it's no big deal, but maybe for absurdly long runs it might. + // if so, fix it. + private void wakeupSenders() + { + Map.Entry next; + while (null != (next = waitingToSend.firstEntry())) + { + if (next.getKey() - receivedBytes >= 0) + break; + + if (waitingToSend.remove(next.getKey(), next.getValue())) + LockSupport.unpark(next.getValue()); + } + } + + private void maybeProcessDeferred() + { + while (true) + { + long sent = sentBytes; + long received = receivedBytes; + if (sent - received <= minimumInFlightBytes) + break; + + Pair next = deferredBytes.poll(); + if (next == null) + break; + + int receive = next.left; + IntConsumer callbacks = next.right; + while (true) + { + long newReceived = received + receive; + if (receivedBytesUpdater.compareAndSet(this, received, newReceived)) + { + callbacks.accept(receive); + wakeupSenders(); + break; + } + + sent = sentBytes; + received = receivedBytes; + if (sent - received <= minimumInFlightBytes) + { + deferredBytes.add(next); + break; // continues with outer loop to maybe process it if minimumInFlightBytes has changed meanwhile + } + } + } + } + +} diff --git a/test/burn/org/apache/cassandra/net/Connection.java b/test/burn/org/apache/cassandra/net/Connection.java new file mode 100644 index 000000000000..c74c0ae0a166 --- /dev/null +++ b/test/burn/org/apache/cassandra/net/Connection.java @@ -0,0 +1,397 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.channels.ClosedChannelException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Verifier.Destiny; + +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.utils.ExecutorUtils.runWithThreadName; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +public class Connection implements InboundMessageCallbacks, OutboundMessageCallbacks, OutboundDebugCallbacks +{ + static class IntentionalIOException extends IOException {} + static class IntentionalRuntimeException extends RuntimeException {} + + final InetAddressAndPort sender; + final InetAddressAndPort recipient; + final BytesInFlightController controller; + final InboundMessageHandlers inbound; + final OutboundConnection outbound; + final OutboundConnectionSettings outboundTemplate; + final Verifier verifier; + final MessageGenerator sendGenerator; + final String linkId; + final long minId; + final long maxId; + final AtomicInteger isSending = new AtomicInteger(); + private volatile Runnable onSync; + final Lock managementLock = new ReentrantLock(); + + private final AtomicLong nextSendId = new AtomicLong(); + + Connection(InetAddressAndPort sender, InetAddressAndPort recipient, ConnectionType type, + InboundMessageHandlers inbound, + OutboundConnectionSettings outboundTemplate, ResourceLimits.EndpointAndGlobal reserveCapacityInBytes, + MessageGenerator generator, + long minId, long maxId) + { + this.sender = sender; + this.recipient = recipient; + this.controller = new BytesInFlightController(1 << 20); + this.sendGenerator = generator.copy(); + this.minId = minId; + this.maxId = maxId; + this.nextSendId.set(minId); + this.linkId = sender.toString(false) + "->" + recipient.toString(false) + "-" + type; + this.outboundTemplate = outboundTemplate.toEndpoint(recipient) + .withFrom(sender) + .withCallbacks(this) + .withDebugCallbacks(this); + this.inbound = inbound; + this.outbound = new OutboundConnection(type, this.outboundTemplate, reserveCapacityInBytes); + this.verifier = new Verifier(controller, outbound, inbound); + } + + void startVerifier(Runnable onFailure, Executor executor, long deadlineNanos) + { + executor.execute(runWithThreadName(() -> verifier.run(onFailure, deadlineNanos), "Verify-" + linkId)); + } + + boolean isSending() + { + return isSending.get() > 0; + } + + boolean registerSender() + { + return isSending.updateAndGet(i -> i < 0 ? i : i + 1) > 0; + } + + void unregisterSender() + { + if (isSending.updateAndGet(i -> i < 0 ? i + 1 : i - 1) == -1) + { + Runnable onSync = this.onSync; + this.onSync = null; + verifier.onSync(() -> { + onSync.run(); + isSending.set(0); + }); + } + } + + boolean setInFlightByteBounds(long minBytes, long maxBytes) + { + if (managementLock.tryLock()) + { + try + { + if (isSending.get() >= 0) + { + controller.setInFlightByteBounds(minBytes, maxBytes); + return true; + } + } + finally + { + managementLock.unlock(); + } + } + return false; + } + + void sync(Runnable onCompletion) + { + managementLock.lock(); + try + { + assert onSync == null; + assert isSending.get() >= 0; + isSending.updateAndGet(i -> -2 -i); + long previousMin = controller.minimumInFlightBytes(); + long previousMax = controller.maximumInFlightBytes(); + controller.setInFlightByteBounds(0, Long.MAX_VALUE); + onSync = () -> { + long inFlight = controller.inFlight(); + if (inFlight != 0) + verifier.logFailure("%s has %d bytes in flight, but connection is idle", linkId, inFlight); + controller.setInFlightByteBounds(previousMin, previousMax); + onCompletion.run(); + }; + unregisterSender(); + } + finally + { + managementLock.unlock(); + } + } + + void sendOne() throws InterruptedException + { + long id = nextSendId.getAndUpdate(i -> i == maxId ? minId : i + 1); + try + { + Destiny destiny = Destiny.SUCCEED; + byte realDestiny = 0; + Message msg; + synchronized (sendGenerator) + { + if (0 == sendGenerator.uniformInt(1 << 10)) + { + // abnormal destiny + realDestiny = (byte) (1 + sendGenerator.uniformInt(6)); + destiny = realDestiny <= 3 ? Destiny.FAIL_TO_SERIALIZE : Destiny.FAIL_TO_DESERIALIZE; + } + msg = sendGenerator.generate(id, realDestiny); + } + + controller.send(msg.serializedSize(current_version)); + Verifier.EnqueueMessageEvent e = verifier.onEnqueue(msg, destiny); + outbound.enqueue(msg); + e.complete(verifier); + } + catch (ClosedChannelException e) + { + // TODO: make this a tested, not illegal, state + throw new IllegalStateException(e); + } + } + + void reconnectWith(OutboundConnectionSettings template) + { + outbound.reconnectWith(template); + } + + void serialize(long id, byte[] payload, DataOutputPlus out, int messagingVersion) throws IOException + { + verifier.onSerialize(id, messagingVersion); + int firstWrite = payload.length, remainder = 0; + boolean willFail = false; + if (outbound.type() != ConnectionType.LARGE_MESSAGES || messagingVersion >= VERSION_40) + { + // We cannot (with Netty) know how many bytes make it to the network as any partially written block + // will be failed despite having partially succeeded. So to support this behaviour here, we would + // need to accept either outcome, in which case what is the point? + // TODO: it would be nice to fix this, still + willFail = outbound.type() != ConnectionType.LARGE_MESSAGES; + byte info = MessageGenerator.getInfo(payload); + switch (info) + { + case 1: + switch ((int) (id & 1)) + { + case 0: throw new IntentionalIOException(); + case 1: throw new IntentionalRuntimeException(); + } + break; + case 2: + willFail = true; + firstWrite -= (int)id % payload.length; + break; + case 3: + willFail = true; + remainder = (int)id & 65535; + break; + } + } + + MessageGenerator.writeLength(payload, out, messagingVersion); + out.write(payload, 0, firstWrite); + while (remainder > 0) + { + out.write(payload, 0, Math.min(remainder, payload.length)); + remainder -= payload.length; + } + if (!willFail) + verifier.onFinishSerializeLarge(id); + } + + byte[] deserialize(MessageGenerator.Header header, DataInputPlus in, int messagingVersion) throws IOException + { + verifier.onDeserialize(header.id, messagingVersion); + int length = header.length; + switch (header.info) + { + case 4: + switch ((int) (header.id & 1)) + { + case 0: throw new IntentionalIOException(); + case 1: throw new IntentionalRuntimeException(); + } + break; + case 5: { + length -= (int)header.id % header.length; + break; + } + case 6: { + length += (int)header.id & 65535; + break; + } + } + byte[] result = header.read(in, Math.min(header.length, length), messagingVersion); + if (length > header.length) + { + length -= header.length; + while (length >= 8) + { + in.readLong(); + length -= 8; + } + while (length-- > 0) + in.readByte(); + } + return result; + } + + public void process(Message message) + { + verifier.process(message); + } + + public void onHeaderArrived(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + } + + public void onArrived(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + verifier.onArrived(header.id, messageSize); + } + + public void onArrivedExpired(int messageSize, Message.Header header, boolean wasCorrupt, long timeElapsed, TimeUnit timeUnit) + { + controller.fail(messageSize); + verifier.onArrivedExpired(header.id, messageSize, wasCorrupt, timeElapsed, timeUnit); + } + + public void onArrivedCorrupt(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + controller.fail(messageSize); + verifier.onFailedDeserialize(header.id, messageSize); + } + + public void onClosedBeforeArrival(int messageSize, Message.Header header, int bytesReceived, boolean wasCorrupt, boolean wasExpired) + { + controller.fail(messageSize); + verifier.onClosedBeforeArrival(header.id, messageSize); + } + + public void onFailedDeserialize(int messageSize, Message.Header header, Throwable t) + { + controller.fail(messageSize); + verifier.onFailedDeserialize(header.id, messageSize); + } + + public void onDispatched(int messageSize, Message.Header header) + { + } + + public void onExecuting(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + } + + public void onProcessed(int messageSize, Message.Header header) + { + } + + public void onExpired(int messageSize, Message.Header header, long timeElapsed, TimeUnit timeUnit) + { + controller.fail(messageSize); + verifier.onProcessExpired(header.id, messageSize, timeElapsed, timeUnit); + } + + public void onExecuted(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + } + + InboundCounters inboundCounters() + { + return inbound.countersFor(outbound.type()); + } + + public void onSendSmallFrame(int messageCount, int payloadSizeInBytes) + { + verifier.onSendFrame(messageCount, payloadSizeInBytes); + } + + public void onSentSmallFrame(int messageCount, int payloadSizeInBytes) + { + verifier.onSentFrame(messageCount, payloadSizeInBytes); + } + + public void onFailedSmallFrame(int messageCount, int payloadSizeInBytes) + { + controller.fail(payloadSizeInBytes); + verifier.onFailedFrame(messageCount, payloadSizeInBytes); + } + + public void onConnect(int messagingVersion, OutboundConnectionSettings settings) + { + verifier.onConnectOutbound(messagingVersion, settings); + } + + public void onConnectInbound(int messagingVersion, InboundMessageHandler handler) + { + verifier.onConnectInbound(messagingVersion, handler); + } + + public void onOverloaded(Message message, InetAddressAndPort peer) + { + controller.fail(message.serializedSize(current_version)); + verifier.onOverloaded(message.id()); + } + + public void onExpired(Message message, InetAddressAndPort peer) + { + controller.fail(message.serializedSize(current_version)); + verifier.onExpiredBeforeSend(message.id(), message.serializedSize(current_version), approxTime.now() - message.createdAtNanos(), TimeUnit.NANOSECONDS); + } + + public void onFailedSerialize(Message message, InetAddressAndPort peer, int messagingVersion, int bytesWrittenToNetwork, Throwable failure) + { + if (bytesWrittenToNetwork == 0) + controller.fail(message.serializedSize(messagingVersion)); + verifier.onFailedSerialize(message.id(), bytesWrittenToNetwork, failure); + } + + public void onDiscardOnClose(Message message, InetAddressAndPort peer) + { + controller.fail(message.serializedSize(current_version)); + verifier.onFailedClosing(message.id()); + } + + public String toString() + { + return linkId; + } +} + diff --git a/test/burn/org/apache/cassandra/net/ConnectionBurnTest.java b/test/burn/org/apache/cassandra/net/ConnectionBurnTest.java new file mode 100644 index 000000000000..81b6402c5479 --- /dev/null +++ b/test/burn/org/apache/cassandra/net/ConnectionBurnTest.java @@ -0,0 +1,656 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.IntConsumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.Uninterruptibles; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.channel.Channel; +import net.openhft.chronicle.core.util.ThrowingBiConsumer; +import net.openhft.chronicle.core.util.ThrowingRunnable; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.MessageGenerator.UniformPayloadGenerator; +import org.apache.cassandra.utils.ExecutorUtils; +import org.apache.cassandra.utils.MonotonicClock; +import org.apache.cassandra.utils.memory.BufferPool; + +import static java.lang.Math.min; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; +import static org.apache.cassandra.utils.MonotonicClock.preciseTime; + +public class ConnectionBurnTest +{ + private static final Logger logger = LoggerFactory.getLogger(ConnectionBurnTest.class); + + static + { + // stop updating ALMOST_SAME_TIME so that we get consistent message expiration times + ((MonotonicClock.AbstractEpochSamplingClock) preciseTime).pauseEpochSampling(); + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setCrossNodeTimeout(true); + } + + static class NoGlobalInboundMetrics implements InboundMessageHandlers.GlobalMetricCallbacks + { + static final NoGlobalInboundMetrics instance = new NoGlobalInboundMetrics(); + public LatencyConsumer internodeLatencyRecorder(InetAddressAndPort to) + { + return (timeElapsed, timeUnit) -> {}; + } + public void recordInternalLatency(Verb verb, long timeElapsed, TimeUnit timeUnit) {} + public void recordInternodeDroppedMessage(Verb verb, long timeElapsed, TimeUnit timeUnit) {} + } + + static class Inbound + { + final Map> handlersByRecipientThenSender; + final InboundSockets sockets; + + Inbound(List endpoints, GlobalInboundSettings settings, Test test) + { + final InboundMessageHandlers.GlobalResourceLimits globalInboundLimits = new InboundMessageHandlers.GlobalResourceLimits(new ResourceLimits.Concurrent(settings.globalReserveLimit)); + Map> handlersByRecipientThenSender = new HashMap<>(); + List bind = new ArrayList<>(); + for (InetAddressAndPort recipient : endpoints) + { + Map handlersBySender = new HashMap<>(); + for (InetAddressAndPort sender : endpoints) + handlersBySender.put(sender, new InboundMessageHandlers(recipient, sender, settings.queueCapacity, settings.endpointReserveLimit, globalInboundLimits, NoGlobalInboundMetrics.instance, test, test)); + + handlersByRecipientThenSender.put(recipient, handlersBySender); + bind.add(settings.template.withHandlers(handlersBySender::get).withBindAddress(recipient)); + } + this.sockets = new InboundSockets(bind); + this.handlersByRecipientThenSender = handlersByRecipientThenSender; + } + } + + private static class ConnectionKey + { + final InetAddressAndPort from; + final InetAddressAndPort to; + final ConnectionType type; + + private ConnectionKey(InetAddressAndPort from, InetAddressAndPort to, ConnectionType type) + { + this.from = from; + this.to = to; + this.type = type; + } + + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConnectionKey that = (ConnectionKey) o; + return Objects.equals(from, that.from) && + Objects.equals(to, that.to) && + type == that.type; + } + + public int hashCode() + { + return Objects.hash(from, to, type); + } + } + + private static class Test implements InboundMessageHandlers.HandlerProvider, InboundMessageHandlers.MessageConsumer + { + private final IVersionedSerializer serializer = new IVersionedSerializer() + { + public void serialize(byte[] payload, DataOutputPlus out, int version) throws IOException + { + long id = MessageGenerator.getId(payload); + forId(id).serialize(id, payload, out, version); + } + + public byte[] deserialize(DataInputPlus in, int version) throws IOException + { + MessageGenerator.Header header = MessageGenerator.readHeader(in, version); + return forId(header.id).deserialize(header, in, version); + } + + public long serializedSize(byte[] payload, int version) + { + return MessageGenerator.serializedSize(payload, version); + } + }; + + static class Builder + { + long time; + TimeUnit timeUnit; + int endpoints; + MessageGenerators generators; + OutboundConnectionSettings outbound; + GlobalInboundSettings inbound; + public Builder time(long time, TimeUnit timeUnit) { this.time = time; this.timeUnit = timeUnit; return this; } + public Builder endpoints(int endpoints) { this.endpoints = endpoints; return this; } + public Builder inbound(GlobalInboundSettings inbound) { this.inbound = inbound; return this; } + public Builder outbound(OutboundConnectionSettings outbound) { this.outbound = outbound; return this; } + public Builder generators(MessageGenerators generators) { this.generators = generators; return this; } + Test build() { return new Test(endpoints, generators, inbound, outbound, timeUnit.toNanos(time)); } + } + + static Builder builder() { return new Builder(); } + + private static final int messageIdsPerConnection = 1 << 20; + + final long runForNanos; + final int version; + final List endpoints; + final Inbound inbound; + final Connection[] connections; + final long[] connectionMessageIds; + final ExecutorService executor = Executors.newCachedThreadPool(); + final Map connectionLookup = new HashMap<>(); + + private Test(int simulateEndpoints, MessageGenerators messageGenerators, GlobalInboundSettings inboundSettings, OutboundConnectionSettings outboundTemplate, long runForNanos) + { + this.endpoints = endpoints(simulateEndpoints); + this.inbound = new Inbound(endpoints, inboundSettings, this); + this.connections = new Connection[endpoints.size() * endpoints.size() * 3]; + this.connectionMessageIds = new long[connections.length]; + this.version = outboundTemplate.acceptVersions == null ? current_version : outboundTemplate.acceptVersions.max; + this.runForNanos = runForNanos; + + int i = 0; + long minId = 0, maxId = messageIdsPerConnection - 1; + for (InetAddressAndPort recipient : endpoints) + { + for (InetAddressAndPort sender : endpoints) + { + InboundMessageHandlers inboundHandlers = inbound.handlersByRecipientThenSender.get(recipient).get(sender); + OutboundConnectionSettings template = outboundTemplate.withDefaultReserveLimits(); + ResourceLimits.Limit reserveEndpointCapacityInBytes = new ResourceLimits.Concurrent(template.applicationSendQueueReserveEndpointCapacityInBytes); + ResourceLimits.EndpointAndGlobal reserveCapacityInBytes = new ResourceLimits.EndpointAndGlobal(reserveEndpointCapacityInBytes, template.applicationSendQueueReserveGlobalCapacityInBytes); + for (ConnectionType type : ConnectionType.MESSAGING_TYPES) + { + Connection connection = new Connection(sender, recipient, type, inboundHandlers, template, reserveCapacityInBytes, messageGenerators.get(type), minId, maxId); + this.connections[i] = connection; + this.connectionMessageIds[i] = minId; + connectionLookup.put(new ConnectionKey(sender, recipient, type), connection); + minId = maxId + 1; + maxId += messageIdsPerConnection; + ++i; + } + } + } + } + + Connection forId(long messageId) + { + int i = Arrays.binarySearch(connectionMessageIds, messageId); + if (i < 0) i = -2 -i; + Connection connection = connections[i]; + assert connection.minId <= messageId && connection.maxId >= messageId; + return connection; + } + + List getConnections(InetAddressAndPort endpoint, boolean inbound) + { + List result = new ArrayList<>(); + for (ConnectionType type : ConnectionType.MESSAGING_TYPES) + { + for (InetAddressAndPort other : endpoints) + { + result.add(connectionLookup.get(inbound ? new ConnectionKey(other, endpoint, type) + : new ConnectionKey(endpoint, other, type))); + } + } + result.forEach(c -> {assert endpoint.equals(inbound ? c.recipient : c.sender); }); + return result; + } + + public void run() throws ExecutionException, InterruptedException, NoSuchFieldException, IllegalAccessException, TimeoutException + { + Reporters reporters = new Reporters(endpoints, connections); + try + { + long deadline = System.nanoTime() + runForNanos; + Verb._TEST_2.unsafeSetHandler(() -> message -> {}); + Verb._TEST_2.unsafeSetSerializer(() -> serializer); + inbound.sockets.open().get(); + + CountDownLatch failed = new CountDownLatch(1); + for (Connection connection : connections) + connection.startVerifier(failed::countDown, executor, deadline); + + for (int i = 0 ; i < 2 * connections.length ; ++i) + { + executor.execute(() -> { + String threadName = Thread.currentThread().getName(); + try + { + ThreadLocalRandom random = ThreadLocalRandom.current(); + while (approxTime.now() < deadline && !Thread.currentThread().isInterrupted()) + { + Connection connection = connections[random.nextInt(connections.length)]; + if (!connection.registerSender()) + continue; + + try + { + Thread.currentThread().setName("Generate-" + connection.linkId); + int count = 0; + switch (random.nextInt() & 3) + { + case 0: count = random.nextInt(100, 200); break; + case 1: count = random.nextInt(200, 1000); break; + case 2: count = random.nextInt(1000, 2000); break; + case 3: count = random.nextInt(2000, 10000); break; + } + + if (connection.outbound.type() == LARGE_MESSAGES) + count /= 2; + + while (connection.isSending() + && count-- > 0 + && approxTime.now() < deadline + && !Thread.currentThread().isInterrupted()) + connection.sendOne(); + } + finally + { + Thread.currentThread().setName(threadName); + connection.unregisterSender(); + } + } + } + catch (Throwable t) + { + if (t instanceof InterruptedException) + return; + logger.error("Unexpected exception", t); + failed.countDown(); + } + }); + } + + executor.execute(() -> { + Thread.currentThread().setName("Test-SetInFlight"); + ThreadLocalRandom random = ThreadLocalRandom.current(); + List connections = new ArrayList<>(Arrays.asList(this.connections)); + while (!Thread.currentThread().isInterrupted()) + { + Collections.shuffle(connections); + int total = random.nextInt(1 << 20, 128 << 20); + for (int i = connections.size() - 1; i >= 1 ; --i) + { + int average = total / (i + 1); + int max = random.nextInt(1, min(2 * average, total - 2)); + int min = random.nextInt(0, max); + connections.get(i).setInFlightByteBounds(min, max); + total -= max; + } + // note that setInFlightByteBounds might not + connections.get(0).setInFlightByteBounds(random.nextInt(0, total), total); + Uninterruptibles.sleepUninterruptibly(1L, TimeUnit.SECONDS); + } + }); + + // TODO: slowly modify the pattern of interrupts, from often to infrequent + executor.execute(() -> { + Thread.currentThread().setName("Test-Reconnect"); + ThreadLocalRandom random = ThreadLocalRandom.current(); + while (deadline > System.nanoTime()) + { + try + { + Thread.sleep(random.nextInt(60000)); + } + catch (InterruptedException e) + { + break; + } + Connection connection = connections[random.nextInt(connections.length)]; + OutboundConnectionSettings template = connection.outboundTemplate; + template = ConnectionTest.SETTINGS.get(random.nextInt(ConnectionTest.SETTINGS.size())) + .outbound.apply(template); + connection.reconnectWith(template); + } + }); + + executor.execute(() -> { + Thread.currentThread().setName("Test-Sync"); + ThreadLocalRandom random = ThreadLocalRandom.current(); + BiConsumer> checkStoppedTo = (to, from) -> { + InboundMessageHandlers handlers = from.get(0).inbound; + long using = handlers.usingCapacity(); + long usingReserve = handlers.usingEndpointReserveCapacity(); + if (using != 0 || usingReserve != 0) + { + String message = to + " inbound using %d capacity and %d reserve; should be zero"; + from.get(0).verifier.logFailure(message, using, usingReserve); + } + }; + BiConsumer> checkStoppedFrom = (from, to) -> { + long using = to.stream().map(c -> c.outbound).mapToLong(OutboundConnection::pendingBytes).sum(); + long usingReserve = to.get(0).outbound.unsafeGetEndpointReserveLimits().using(); + if (using != 0 || usingReserve != 0) + { + String message = from + " outbound using %d capacity and %d reserve; should be zero"; + to.get(0).verifier.logFailure(message, using, usingReserve); + } + }; + ThrowingBiConsumer, ThrowingRunnable, InterruptedException> sync = + (connections, exec) -> { + logger.info("Syncing connections: {}", connections); + final CountDownLatch ready = new CountDownLatch(connections.size()); + final CountDownLatch done = new CountDownLatch(1); + for (Connection connection : connections) + { + connection.sync(() -> { + ready.countDown(); + try { done.await(); } + catch (InterruptedException e) { Thread.interrupted(); } + }); + } + ready.await(); + try + { + exec.run(); + } + finally + { + done.countDown(); + } + logger.info("Sync'd connections: {}", connections); + }; + + int count = 0; + while (deadline > System.nanoTime()) + { + + try + { + Thread.sleep(random.nextInt(10000)); + + if (++count % 10 == 0) +// { +// boolean checkInbound = random.nextBoolean(); +// BiConsumer> verifier = checkInbound ? checkStoppedTo : checkStoppedFrom; +// InetAddressAndPort endpoint = endpoints.get(random.nextInt(endpoints.size())); +// List connections = getConnections(endpoint, checkInbound); +// sync.accept(connections, () -> verifier.accept(endpoint, connections)); +// } +// else if (count % 100 == 0) + { + sync.accept(ImmutableList.copyOf(connections), () -> { + + for (InetAddressAndPort endpoint : endpoints) + { + checkStoppedTo .accept(endpoint, getConnections(endpoint, true )); + checkStoppedFrom.accept(endpoint, getConnections(endpoint, false)); + } + long inUse = BufferPool.unsafeGetBytesInUse(); + if (inUse > 0) + { +// try +// { +// ManagementFactory.getPlatformMXBean(HotSpotDiagnosticMXBean.class).dumpHeap("/Users/belliottsmith/code/cassandra/cassandra/leak.hprof", true); +// } +// catch (IOException e) +// { +// throw new RuntimeException(e); +// } + connections[0].verifier.logFailure("Using %d bytes of BufferPool, but all connections are idle", inUse); + } + }); + } + else + { + CountDownLatch latch = new CountDownLatch(1); + Connection connection = connections[random.nextInt(connections.length)]; + connection.sync(latch::countDown); + latch.await(); + } + } + catch (InterruptedException e) + { + break; + } + } + }); + + while (deadline > System.nanoTime() && failed.getCount() > 0) + { + reporters.update(); + reporters.print(); + Uninterruptibles.awaitUninterruptibly(failed, 30L, TimeUnit.SECONDS); + } + + executor.shutdownNow(); + ExecutorUtils.awaitTermination(1L, TimeUnit.MINUTES, executor); + } + finally + { + reporters.update(); + reporters.print(); + + inbound.sockets.close().get(); + new FutureCombiner(Arrays.stream(connections) + .map(c -> c.outbound.close(false)) + .collect(Collectors.toList())) + .get(); + } + } + + class WrappedInboundCallbacks implements InboundMessageCallbacks + { + private final InboundMessageCallbacks wrapped; + + WrappedInboundCallbacks(InboundMessageCallbacks wrapped) + { + this.wrapped = wrapped; + } + + public void onHeaderArrived(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + forId(header.id).onHeaderArrived(messageSize, header, timeElapsed, unit); + wrapped.onHeaderArrived(messageSize, header, timeElapsed, unit); + } + + public void onArrived(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + forId(header.id).onArrived(messageSize, header, timeElapsed, unit); + wrapped.onArrived(messageSize, header, timeElapsed, unit); + } + + public void onArrivedExpired(int messageSize, Message.Header header, boolean wasCorrupt, long timeElapsed, TimeUnit unit) + { + forId(header.id).onArrivedExpired(messageSize, header, wasCorrupt, timeElapsed, unit); + wrapped.onArrivedExpired(messageSize, header, wasCorrupt, timeElapsed, unit); + } + + public void onArrivedCorrupt(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + forId(header.id).onArrivedCorrupt(messageSize, header, timeElapsed, unit); + wrapped.onArrivedCorrupt(messageSize, header, timeElapsed, unit); + } + + public void onClosedBeforeArrival(int messageSize, Message.Header header, int bytesReceived, boolean wasCorrupt, boolean wasExpired) + { + forId(header.id).onClosedBeforeArrival(messageSize, header, bytesReceived, wasCorrupt, wasExpired); + wrapped.onClosedBeforeArrival(messageSize, header, bytesReceived, wasCorrupt, wasExpired); + } + + public void onFailedDeserialize(int messageSize, Message.Header header, Throwable t) + { + forId(header.id).onFailedDeserialize(messageSize, header, t); + wrapped.onFailedDeserialize(messageSize, header, t); + } + + public void onDispatched(int messageSize, Message.Header header) + { + forId(header.id).onDispatched(messageSize, header); + wrapped.onDispatched(messageSize, header); + } + + public void onExecuting(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + forId(header.id).onExecuting(messageSize, header, timeElapsed, unit); + wrapped.onExecuting(messageSize, header, timeElapsed, unit); + } + + public void onProcessed(int messageSize, Message.Header header) + { + forId(header.id).onProcessed(messageSize, header); + wrapped.onProcessed(messageSize, header); + } + + public void onExpired(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + forId(header.id).onExpired(messageSize, header, timeElapsed, unit); + wrapped.onExpired(messageSize, header, timeElapsed, unit); + } + + public void onExecuted(int messageSize, Message.Header header, long timeElapsed, TimeUnit unit) + { + forId(header.id).onExecuted(messageSize, header, timeElapsed, unit); + wrapped.onExecuted(messageSize, header, timeElapsed, unit); + } + } + + public void fail(Message.Header header, Throwable failure) + { +// forId(header.id).verifier.logFailure("Unexpected failure", failure); + } + + public void accept(Message message) + { + forId(message.id()).process(message); + } + + public InboundMessageCallbacks wrap(InboundMessageCallbacks wrap) + { + return new WrappedInboundCallbacks(wrap); + } + + public InboundMessageHandler provide( + FrameDecoder decoder, + ConnectionType type, + Channel channel, + InetAddressAndPort self, + InetAddressAndPort peer, + int version, + int largeMessageThreshold, + int queueCapacity, + ResourceLimits.Limit endpointReserveCapacity, + ResourceLimits.Limit globalReserveCapacity, + InboundMessageHandler.WaitQueue endpointWaitQueue, + InboundMessageHandler.WaitQueue globalWaitQueue, + InboundMessageHandler.OnHandlerClosed onClosed, + InboundMessageCallbacks callbacks, + Consumer> messageSink + ) + { + return new InboundMessageHandler(decoder, type, channel, self, peer, version, largeMessageThreshold, queueCapacity, endpointReserveCapacity, globalReserveCapacity, endpointWaitQueue, globalWaitQueue, onClosed, wrap(callbacks), messageSink) + { + final IntConsumer releaseCapacity = size -> super.releaseProcessedCapacity(size, null); + protected void releaseProcessedCapacity(int bytes, Message.Header header) + { + forId(header.id).controller.process(bytes, releaseCapacity); + } + }; + } + } + + static List endpoints(int count) + { + return IntStream.rangeClosed(1, count) + .mapToObj(ConnectionBurnTest::endpoint) + .collect(Collectors.toList()); + } + + private static InetAddressAndPort endpoint(int i) + { + try + { + return InetAddressAndPort.getByName("127.0.0." + i); + } + catch (UnknownHostException e) + { + throw new RuntimeException(e); + } + } + + public static void test(GlobalInboundSettings inbound, OutboundConnectionSettings outbound) throws ExecutionException, InterruptedException, NoSuchFieldException, IllegalAccessException, TimeoutException + { + MessageGenerator small = new UniformPayloadGenerator(0, 1, (1 << 15)); + MessageGenerator large = new UniformPayloadGenerator(0, 1, (1 << 16) + (1 << 15)); + MessageGenerators generators = new MessageGenerators(small, large); + outbound = outbound.withApplicationSendQueueCapacityInBytes(1 << 18) + .withApplicationReserveSendQueueCapacityInBytes(1 << 30, new ResourceLimits.Concurrent(Integer.MAX_VALUE)); + + Test.builder() + .generators(generators) + .endpoints(4) + .inbound(inbound) + .outbound(outbound) + .time(2L, TimeUnit.DAYS) + .build().run(); + } + + public static void main(String[] args) throws ExecutionException, InterruptedException, NoSuchFieldException, IllegalAccessException, TimeoutException + { + GlobalInboundSettings inboundSettings = new GlobalInboundSettings() + .withQueueCapacity(1 << 18) + .withEndpointReserveLimit(1 << 20) + .withGlobalReserveLimit(1 << 21) + .withTemplate(new InboundConnectionSettings() + .withEncryption(ConnectionTest.encryptionOptions)); + + test(inboundSettings, new OutboundConnectionSettings(null) + .withTcpUserTimeoutInMS(0)); + MessagingService.instance().socketFactory.shutdownNow(); + } + +} diff --git a/test/burn/org/apache/cassandra/net/GlobalInboundSettings.java b/test/burn/org/apache/cassandra/net/GlobalInboundSettings.java new file mode 100644 index 000000000000..9b23041ae41c --- /dev/null +++ b/test/burn/org/apache/cassandra/net/GlobalInboundSettings.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +class GlobalInboundSettings +{ + final int queueCapacity; + final long endpointReserveLimit; + final long globalReserveLimit; + final InboundConnectionSettings template; + + GlobalInboundSettings() + { + this(0, 0, 0, null); + } + + GlobalInboundSettings(int queueCapacity, long endpointReserveLimit, long globalReserveLimit, InboundConnectionSettings template) + { + this.queueCapacity = queueCapacity; + this.endpointReserveLimit = endpointReserveLimit; + this.globalReserveLimit = globalReserveLimit; + this.template = template; + } + + GlobalInboundSettings withQueueCapacity(int queueCapacity) + { + return new GlobalInboundSettings(queueCapacity, endpointReserveLimit, globalReserveLimit, template); + } + GlobalInboundSettings withEndpointReserveLimit(int endpointReserveLimit) + { + return new GlobalInboundSettings(queueCapacity, endpointReserveLimit, globalReserveLimit, template); + } + GlobalInboundSettings withGlobalReserveLimit(int globalReserveLimit) + { + return new GlobalInboundSettings(queueCapacity, endpointReserveLimit, globalReserveLimit, template); + } + GlobalInboundSettings withTemplate(InboundConnectionSettings template) + { + return new GlobalInboundSettings(queueCapacity, endpointReserveLimit, globalReserveLimit, template); + } +} \ No newline at end of file diff --git a/test/burn/org/apache/cassandra/net/LogbackFilter.java b/test/burn/org/apache/cassandra/net/LogbackFilter.java new file mode 100644 index 000000000000..94aa2f9a6a71 --- /dev/null +++ b/test/burn/org/apache/cassandra/net/LogbackFilter.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.EOFException; +import java.nio.BufferOverflowException; +import java.util.Set; +import java.util.regex.Pattern; + +import com.google.common.collect.ImmutableSet; + +import ch.qos.logback.classic.spi.IThrowableProxy; +import ch.qos.logback.classic.spi.LoggingEvent; +import ch.qos.logback.core.filter.Filter; +import ch.qos.logback.core.spi.FilterReply; + +public class LogbackFilter extends Filter +{ + private static final Pattern ignore = Pattern.compile("(successfully connected|connection established), version ="); + + public FilterReply decide(Object o) + { + if (!(o instanceof LoggingEvent)) + return FilterReply.NEUTRAL; + + LoggingEvent e = (LoggingEvent) o; +// if (ignore.matcher(e.getMessage()).find()) +// return FilterReply.DENY; + + IThrowableProxy t = e.getThrowableProxy(); + if (t == null) + return FilterReply.NEUTRAL; + + if (!isIntentional(t)) + return FilterReply.NEUTRAL; + +// logger.info("Filtered exception {}: {}", t.getClassName(), t.getMessage()); + return FilterReply.DENY; + } + + private static final Set intentional = ImmutableSet.of( + Connection.IntentionalIOException.class.getName(), + Connection.IntentionalRuntimeException.class.getName(), + InvalidSerializedSizeException.class.getName(), + BufferOverflowException.class.getName(), + EOFException.class.getName() + ); + + public static boolean isIntentional(IThrowableProxy t) + { + while (true) + { + if (intentional.contains(t.getClassName())) + return true; + + if (null == t.getCause()) + return false; + + t = t.getCause(); + } + } + + +} diff --git a/test/burn/org/apache/cassandra/net/MessageGenerator.java b/test/burn/org/apache/cassandra/net/MessageGenerator.java new file mode 100644 index 000000000000..3c03689a566e --- /dev/null +++ b/test/burn/org/apache/cassandra/net/MessageGenerator.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.utils.vint.VIntCoding; +import sun.misc.Unsafe; + +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +abstract class MessageGenerator +{ + final long seed; + final Random random; + + private MessageGenerator(long seed) + { + this.seed = seed; + this.random = new Random(); + } + + Message.Builder builder(long id) + { + random.setSeed(id ^ seed); + long now = approxTime.now(); + + int expiresInMillis; + int expiryMask = random.nextInt(); + if (0 == (expiryMask & 0xffff)) expiresInMillis = 2; + else if (0 == (expiryMask & 0xfff)) expiresInMillis = 10; + else if (0 == (expiryMask & 0xff)) expiresInMillis = 100; + else if (0 == (expiryMask & 0xf)) expiresInMillis = 1000; + else expiresInMillis = 60 * 1000; + + long expiresInNanos = TimeUnit.MILLISECONDS.toNanos((expiresInMillis / 2) + random.nextInt(expiresInMillis / 2)); + + return Message.builder(Verb._TEST_2, null) + .withId(id) + .withCreatedAt(now) + .withExpiresAt(now + expiresInNanos); // don't expire for now + } + + public int uniformInt(int limit) + { + return random.nextInt(limit); + } + + // generate a Message with the provided id and with both id and info encoded in its payload + abstract Message generate(long id, byte info); + abstract MessageGenerator copy(); + + static final class UniformPayloadGenerator extends MessageGenerator + { + final int minSize; + final int maxSize; + final byte[] fillWithBytes; + UniformPayloadGenerator(long seed, int minSize, int maxSize) + { + super(seed); + this.minSize = Math.max(9, minSize); + this.maxSize = Math.max(9, maxSize); + this.fillWithBytes = new byte[32]; + random.setSeed(seed); + random.nextBytes(fillWithBytes); + } + + Message generate(long id, byte info) + { + Message.Builder builder = builder(id); + byte[] payload = new byte[minSize + random.nextInt(maxSize - minSize)]; + ByteBuffer wrapped = ByteBuffer.wrap(payload); + setId(payload, id); + payload[8] = info; + wrapped.position(9); + while (wrapped.hasRemaining()) + wrapped.put(fillWithBytes, 0, Math.min(fillWithBytes.length, wrapped.remaining())); + builder.withPayload(payload); + return builder.build(); + } + + MessageGenerator copy() + { + return new UniformPayloadGenerator(seed, minSize, maxSize); + } + } + + static long getId(byte[] payload) + { + return unsafe.getLong(payload, BYTE_ARRAY_BASE_OFFSET); + } + static byte getInfo(byte[] payload) + { + return payload[8]; + } + private static void setId(byte[] payload, long id) + { + unsafe.putLong(payload, BYTE_ARRAY_BASE_OFFSET, id); + } + + static class Header + { + public final int length; + public final long id; + public final byte info; + + Header(int length, long id, byte info) + { + this.length = length; + this.id = id; + this.info = info; + } + + public byte[] read(DataInputPlus in, int length, int messagingVersion) throws IOException + { + byte[] result = new byte[Math.max(9, length)]; + setId(result, id); + result[8] = info; + in.readFully(result, 9, Math.max(0, length - 9)); + return result; + } + } + + static Header readHeader(DataInputPlus in, int messagingVersion) throws IOException + { + int length = messagingVersion < VERSION_40 + ? in.readInt() + : (int) in.readUnsignedVInt(); + long id = in.readLong(); + if (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN) + id = Long.reverseBytes(id); + byte info = in.readByte(); + return new Header(length, id, info); + } + + static void writeLength(byte[] payload, DataOutputPlus out, int messagingVersion) throws IOException + { + if (messagingVersion < VERSION_40) + out.writeInt(payload.length); + else + out.writeUnsignedVInt(payload.length); + } + + static long serializedSize(byte[] payload, int messagingVersion) + { + return payload.length + (messagingVersion < VERSION_40 ? 4 : VIntCoding.computeUnsignedVIntSize(payload.length)); + } + + private static final Unsafe unsafe; + static + { + try + { + Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe"); + field.setAccessible(true); + unsafe = (sun.misc.Unsafe) field.get(null); + } + catch (Exception e) + { + throw new AssertionError(e); + } + } + private static final long BYTE_ARRAY_BASE_OFFSET = unsafe.arrayBaseOffset(byte[].class); + +} + diff --git a/test/burn/org/apache/cassandra/net/MessageGenerators.java b/test/burn/org/apache/cassandra/net/MessageGenerators.java new file mode 100644 index 000000000000..92aab3a4a388 --- /dev/null +++ b/test/burn/org/apache/cassandra/net/MessageGenerators.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +final class MessageGenerators +{ + final MessageGenerator small; + final MessageGenerator large; + + MessageGenerators(MessageGenerator small, MessageGenerator large) + { + this.small = small; + this.large = large; + } + + MessageGenerator get(ConnectionType type) + { + switch (type) + { + case SMALL_MESSAGES: + case URGENT_MESSAGES: + return small; + case LARGE_MESSAGES: + return large; + default: + throw new IllegalStateException(); + } + } +} diff --git a/test/burn/org/apache/cassandra/net/Reporters.java b/test/burn/org/apache/cassandra/net/Reporters.java new file mode 100644 index 000000000000..9ab46438a317 --- /dev/null +++ b/test/burn/org/apache/cassandra/net/Reporters.java @@ -0,0 +1,322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.LongFunction; +import java.util.function.ToLongFunction; + +import com.google.common.collect.ImmutableList; + +import org.apache.cassandra.locator.InetAddressAndPort; + +class Reporters +{ + final Collection endpoints; + final Connection[] connections; + final List reporters; + final long start = System.nanoTime(); + + Reporters(Collection endpoints, Connection[] connections) + { + this.endpoints = endpoints; + this.connections = connections; + this.reporters = ImmutableList.of( + outboundReporter (true, "Outbound Throughput", OutboundConnection::sentBytes, Reporters::prettyPrintMemory), + inboundReporter (true, "Inbound Throughput", InboundCounters::processedBytes, Reporters::prettyPrintMemory), + + outboundReporter (false, "Outbound Pending Bytes", OutboundConnection::pendingBytes, Reporters::prettyPrintMemory), + reporter (false, "Inbound Pending Bytes", c -> c.inbound.usingCapacity(), Reporters::prettyPrintMemory), + + outboundReporter (true, "Outbound Expirations", OutboundConnection::expiredCount, Long::toString), + inboundReporter (true, "Inbound Expirations", InboundCounters::expiredCount, Long::toString), + + outboundReporter (true, "Outbound Errors", OutboundConnection::errorCount, Long::toString), + inboundReporter (true, "Inbound Errors", InboundCounters::errorCount, Long::toString), + + outboundReporter (true, "Outbound Connection Attempts", OutboundConnection::connectionAttempts, Long::toString) + ); + } + + void update() + { + for (Reporter reporter : reporters) + reporter.update(); + } + + void print() + { + System.out.println("==" + prettyPrintElapsed(System.nanoTime() - start) + "==\n"); + + for (Reporter reporter : reporters) + { + reporter.print(); + } + } + + private Reporter outboundReporter(boolean accumulates, String name, ToLongFunction get, LongFunction printer) + { + return new Reporter(accumulates, name, (conn) -> get.applyAsLong(conn.outbound), printer); + } + + private Reporter inboundReporter(boolean accumulates, String name, ToLongFunction get, LongFunction printer) + { + return new Reporter(accumulates, name, (conn) -> get.applyAsLong(conn.inboundCounters()), printer); + } + + private Reporter reporter(boolean accumulates, String name, ToLongFunction get, LongFunction printer) + { + return new Reporter(accumulates, name, get, printer); + } + + class Reporter + { + boolean accumulates; + final String name; + final ToLongFunction get; + final LongFunction print; + final long[][] previousValue; + final long[] columnTotals = new long[1 + endpoints.size() * 3]; + final Table table; + + Reporter(boolean accumulates, String name, ToLongFunction get, LongFunction print) + { + this.accumulates = accumulates; + this.name = name; + this.get = get; + this.print = print; + + previousValue = accumulates ? new long[endpoints.size()][endpoints.size() * 3] : null; + + String[] rowNames = new String[endpoints.size() + 1]; + for (int row = 0 ; row < endpoints.size() ; ++row) + { + rowNames[row] = Integer.toString(1 + row); + } + rowNames[rowNames.length - 1] = "Total"; + + String[] columnNames = new String[endpoints.size() * 3 + 1]; + for (int column = 0 ; column < endpoints.size() * 3 ; column += 3) + { + String endpoint = Integer.toString(1 + column / 3); + columnNames[ column] = endpoint + ".Urgent"; + columnNames[1 + column] = endpoint + ".Small"; + columnNames[2 + column] = endpoint + ".Large"; + } + columnNames[columnNames.length - 1] = "Total"; + + table = new Table(rowNames, columnNames, "Recipient"); + } + + public void update() + { + Arrays.fill(columnTotals, 0); + int row = 0, connection = 0; + for (InetAddressAndPort recipient : endpoints) + { + int column = 0; + long rowTotal = 0; + for (InetAddressAndPort sender : endpoints) + { + for (ConnectionType type : ConnectionType.MESSAGING_TYPES) + { + assert recipient.equals(connections[connection].recipient); + assert sender.equals(connections[connection].sender); + assert type == connections[connection].outbound.type(); + + long cur = get.applyAsLong(connections[connection]); + long value; + if (accumulates) + { + long prev = previousValue[row][column]; + previousValue[row][column] = cur; + value = cur - prev; + } + else + { + value = cur; + } + table.set(row, column, print.apply(value)); + columnTotals[column] += value; + rowTotal += value; + ++column; + ++connection; + } + } + columnTotals[column] += rowTotal; + table.set(row, column, print.apply(rowTotal)); + table.displayRow(row, rowTotal > 0); + ++row; + } + + boolean displayTotalRow = false; + for (int column = 0 ; column < columnTotals.length ; ++column) + { + table.set(endpoints.size(), column, print.apply(columnTotals[column])); + table.displayColumn(column, columnTotals[column] > 0); + displayTotalRow |= columnTotals[column] > 0; + } + table.displayRow(endpoints.size(), displayTotalRow); + } + + public void print() + { + table.print("===" + name + "==="); + } + } + + private static class Table + { + final String[][] print; + final int[] width; + final BitSet rowMask = new BitSet(); + final BitSet columnMask = new BitSet(); + + public Table(String[] rowNames, String[] columnNames, String rowNameHeader) + { + print = new String[rowNames.length + 1][columnNames.length + 1]; + width = new int[columnNames.length + 1]; + print[0][0] = rowNameHeader; + for (int i = 0 ; i < columnNames.length ; ++i) + print[0][1 + i] = columnNames[i]; + for (int i = 0 ; i < rowNames.length ; ++i) + print[1 + i][0] = rowNames[i]; + } + + void set(int row, int column, String value) + { + print[row + 1][column + 1] = value; + } + + void displayRow(int row, boolean display) + { + rowMask.set(row, display); + } + + void displayColumn(int column, boolean display) + { + columnMask.set(column, display); + } + + void print(String heading) + { + if (rowMask.isEmpty() && columnMask.isEmpty()) + return; + + System.out.println(heading + '\n'); + + Arrays.fill(width, 0); + for (int row = 0 ; row < print.length ; ++row) + { + for (int column = 0 ; column < width.length ; ++column) + { + width[column] = Math.max(width[column], print[row][column].length()); + } + } + + for (int row = 0 ; row < print.length ; ++row) + { +// if (row > 0 && !rowMask.get(row - 1)) +// continue; + + StringBuilder builder = new StringBuilder(); + for (int column = 0 ; column < width.length ; ++column) + { +// if (column > 0 && !columnMask.get(column - 1)) +// continue; + + String s = print[row][column]; + int pad = width[column] - s.length(); + for (int i = 0 ; i < pad ; ++i) + builder.append(' '); + builder.append(s); + builder.append(" "); + } + System.out.println(builder.toString()); + } + System.out.println(); + } + } + + private static final class OneTimeUnit + { + final TimeUnit unit; + final String symbol; + final long nanos; + + private OneTimeUnit(TimeUnit unit, String symbol) + { + this.unit = unit; + this.symbol = symbol; + this.nanos = unit.toNanos(1L); + } + } + + private static final List prettyPrintElapsed = ImmutableList.of( + new OneTimeUnit(TimeUnit.DAYS, "d"), + new OneTimeUnit(TimeUnit.HOURS, "h"), + new OneTimeUnit(TimeUnit.MINUTES, "m"), + new OneTimeUnit(TimeUnit.SECONDS, "s"), + new OneTimeUnit(TimeUnit.MILLISECONDS, "ms"), + new OneTimeUnit(TimeUnit.MICROSECONDS, "us"), + new OneTimeUnit(TimeUnit.NANOSECONDS, "ns") + ); + + private static String prettyPrintElapsed(long nanos) + { + if (nanos == 0) + return "0ns"; + + int count = 0; + StringBuilder builder = new StringBuilder(); + for (OneTimeUnit unit : prettyPrintElapsed) + { + if (count == 2) + break; + + if (nanos >= unit.nanos) + { + if (count > 0) + builder.append(' '); + long inUnit = unit.unit.convert(nanos, TimeUnit.NANOSECONDS); + nanos -= unit.unit.toNanos(inUnit); + builder.append(inUnit); + builder.append(unit.symbol); + ++count; + } else if (count > 0) + ++count; + } + + return builder.toString(); + } + + static String prettyPrintMemory(long size) + { + if (size >= 1000 * 1000 * 1000) + return String.format("%.0fG", size / (double) (1 << 30)); + if (size >= 1000 * 1000) + return String.format("%.0fM", size / (double) (1 << 20)); + return String.format("%.0fK", size / (double) (1 << 10)); + } +} + diff --git a/test/burn/org/apache/cassandra/net/Verifier.java b/test/burn/org/apache/cassandra/net/Verifier.java new file mode 100644 index 000000000000..8b48c9a094f8 --- /dev/null +++ b/test/burn/org/apache/cassandra/net/Verifier.java @@ -0,0 +1,1637 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.nio.BufferOverflowException; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.concurrent.locks.LockSupport; +import java.util.function.Consumer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.carrotsearch.hppc.LongObjectOpenHashMap; +import org.apache.cassandra.net.Verifier.ExpiredMessageEvent.ExpirationType; +import org.apache.cassandra.utils.ApproximateTime; +import org.apache.cassandra.utils.concurrent.WaitQueue; + +import static java.util.concurrent.TimeUnit.*; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES; +import static org.apache.cassandra.net.OutboundConnection.LargeMessageDelivery.DEFAULT_BUFFER_SIZE; +import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD; +import static org.apache.cassandra.net.Verifier.EventCategory.OTHER; +import static org.apache.cassandra.net.Verifier.EventCategory.RECEIVE; +import static org.apache.cassandra.net.Verifier.EventCategory.SEND; +import static org.apache.cassandra.net.Verifier.EventType.ARRIVE; +import static org.apache.cassandra.net.Verifier.EventType.CLOSED_BEFORE_ARRIVAL; +import static org.apache.cassandra.net.Verifier.EventType.DESERIALIZE; +import static org.apache.cassandra.net.Verifier.EventType.ENQUEUE; +import static org.apache.cassandra.net.Verifier.EventType.FAILED_CLOSING; +import static org.apache.cassandra.net.Verifier.EventType.FAILED_DESERIALIZE; +import static org.apache.cassandra.net.Verifier.EventType.FAILED_EXPIRED_ON_SEND; +import static org.apache.cassandra.net.Verifier.EventType.FAILED_EXPIRED_ON_RECEIVE; +import static org.apache.cassandra.net.Verifier.EventType.FAILED_FRAME; +import static org.apache.cassandra.net.Verifier.EventType.FAILED_OVERLOADED; +import static org.apache.cassandra.net.Verifier.EventType.FAILED_SERIALIZE; +import static org.apache.cassandra.net.Verifier.EventType.FINISH_SERIALIZE_LARGE; +import static org.apache.cassandra.net.Verifier.EventType.PROCESS; +import static org.apache.cassandra.net.Verifier.EventType.SEND_FRAME; +import static org.apache.cassandra.net.Verifier.EventType.SENT_FRAME; +import static org.apache.cassandra.net.Verifier.EventType.SERIALIZE; +import static org.apache.cassandra.net.Verifier.ExpiredMessageEvent.ExpirationType.ON_SENT; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +/** + * This class is a single-threaded verifier monitoring a single link, with events supplied by inbound and outbound threads + * + * By making verification single threaded, it is easier to reason about (and complex enough as is), but also permits + * a dedicated thread to monitor timeliness of events, e.g. elapsed time between a given SEND and its corresponding RECEIVE + * + * TODO: timeliness of events + * TODO: periodically stop all activity to/from a given endpoint, until it stops (and verify queues all empty, counters all accurate) + * TODO: integrate with proxy that corrupts frames + * TODO: test _OutboundConnection_ close + */ +@SuppressWarnings("WeakerAccess") +public class Verifier +{ + private static final Logger logger = LoggerFactory.getLogger(Verifier.class); + + public enum Destiny + { + SUCCEED, + FAIL_TO_SERIALIZE, + FAIL_TO_DESERIALIZE, + } + + enum EventCategory + { + SEND, RECEIVE, OTHER + } + + enum EventType + { + FAILED_OVERLOADED(SEND), + ENQUEUE(SEND), + FAILED_EXPIRED_ON_SEND(SEND), + SERIALIZE(SEND), + FAILED_SERIALIZE(SEND), + FINISH_SERIALIZE_LARGE(SEND), + SEND_FRAME(SEND), + FAILED_FRAME(SEND), + SENT_FRAME(SEND), + ARRIVE(RECEIVE), + FAILED_EXPIRED_ON_RECEIVE(RECEIVE), + DESERIALIZE(RECEIVE), + CLOSED_BEFORE_ARRIVAL(RECEIVE), + FAILED_DESERIALIZE(RECEIVE), + PROCESS(RECEIVE), + + FAILED_CLOSING(SEND), + + CONNECT_OUTBOUND(OTHER), + SYNC(OTHER), // the connection will stop sending messages, and promptly process any waiting inbound messages + CONTROLLER_UPDATE(OTHER); + + final EventCategory category; + + EventType(EventCategory category) + { + this.category = category; + } + } + + public static class Event + { + final EventType type; + Event(EventType type) + { + this.type = type; + } + } + + static class SimpleEvent extends Event + { + final long at; + SimpleEvent(EventType type, long at) + { + super(type); + this.at = at; + } + public String toString() { return type.toString(); } + } + + static class BoundedEvent extends Event + { + final long start; + volatile long end; + BoundedEvent(EventType type, long start) + { + super(type); + this.start = start; + } + public void complete(Verifier verifier) + { + end = verifier.sequenceId.getAndIncrement(); + verifier.events.put(end, this); + } + } + + static class SimpleMessageEvent extends SimpleEvent + { + final long messageId; + SimpleMessageEvent(EventType type, long at, long messageId) + { + super(type, at); + this.messageId = messageId; + } + } + + static class BoundedMessageEvent extends BoundedEvent + { + final long messageId; + BoundedMessageEvent(EventType type, long start, long messageId) + { + super(type, start); + this.messageId = messageId; + } + } + + static class EnqueueMessageEvent extends BoundedMessageEvent + { + final Message message; + final Destiny destiny; + EnqueueMessageEvent(EventType type, long start, Message message, Destiny destiny) + { + super(type, start, message.id()); + this.message = message; + this.destiny = destiny; + } + public String toString() { return String.format("%s{%s}", type, destiny); } + } + + static class SerializeMessageEvent extends SimpleMessageEvent + { + final int messagingVersion; + SerializeMessageEvent(EventType type, long at, long messageId, int messagingVersion) + { + super(type, at, messageId); + this.messagingVersion = messagingVersion; + } + public String toString() { return String.format("%s{ver=%d}", type, messagingVersion); } + } + + static class SimpleMessageEventWithSize extends SimpleMessageEvent + { + final int messageSize; + SimpleMessageEventWithSize(EventType type, long at, long messageId, int messageSize) + { + super(type, at, messageId); + this.messageSize = messageSize; + } + public String toString() { return String.format("%s{size=%d}", type, messageSize); } + } + + static class FailedSerializeEvent extends SimpleMessageEvent + { + final int bytesWrittenToNetwork; + final Throwable failure; + FailedSerializeEvent(long at, long messageId, int bytesWrittenToNetwork, Throwable failure) + { + super(FAILED_SERIALIZE, at, messageId); + this.bytesWrittenToNetwork = bytesWrittenToNetwork; + this.failure = failure; + } + public String toString() { return String.format("FAILED_SERIALIZE{written=%d, failure=%s}", bytesWrittenToNetwork, failure); } + } + + static class ExpiredMessageEvent extends SimpleMessageEvent + { + enum ExpirationType {ON_SENT, ON_ARRIVED, ON_PROCESSED } + final int messageSize; + final long timeElapsed; + final TimeUnit timeUnit; + final ExpirationType expirationType; + ExpiredMessageEvent(long at, long messageId, int messageSize, long timeElapsed, TimeUnit timeUnit, ExpirationType expirationType) + { + super(expirationType == ON_SENT ? FAILED_EXPIRED_ON_SEND : FAILED_EXPIRED_ON_RECEIVE, at, messageId); + this.messageSize = messageSize; + this.timeElapsed = timeElapsed; + this.timeUnit = timeUnit; + this.expirationType = expirationType; + } + public String toString() { return String.format("EXPIRED_%s{size=%d,elapsed=%d,unit=%s}", expirationType, messageSize, timeElapsed, timeUnit); } + } + + static class FrameEvent extends SimpleEvent + { + final int messageCount; + final int payloadSizeInBytes; + FrameEvent(EventType type, long at, int messageCount, int payloadSizeInBytes) + { + super(type, at); + this.messageCount = messageCount; + this.payloadSizeInBytes = payloadSizeInBytes; + } + } + + static class ProcessMessageEvent extends SimpleMessageEvent + { + final Message message; + ProcessMessageEvent(long at, Message message) + { + super(PROCESS, at, message.id()); + this.message = message; + } + } + + EnqueueMessageEvent onEnqueue(Message message, Destiny destiny) + { + EnqueueMessageEvent enqueue = new EnqueueMessageEvent(ENQUEUE, nextId(), message, destiny); + events.put(enqueue.start, enqueue); + return enqueue; + } + void onOverloaded(long messageId) + { + long at = nextId(); + events.put(at, new SimpleMessageEvent(FAILED_OVERLOADED, at, messageId)); + } + void onFailedClosing(long messageId) + { + long at = nextId(); + events.put(at, new SimpleMessageEvent(FAILED_CLOSING, at, messageId)); + } + void onSerialize(long messageId, int messagingVersion) + { + long at = nextId(); + events.put(at, new SerializeMessageEvent(SERIALIZE, at, messageId, messagingVersion)); + } + void onFinishSerializeLarge(long messageId) + { + long at = nextId(); + events.put(at, new SimpleMessageEvent(FINISH_SERIALIZE_LARGE, at, messageId)); + } + void onFailedSerialize(long messageId, int bytesWrittenToNetwork, Throwable failure) + { + long at = nextId(); + events.put(at, new FailedSerializeEvent(at, messageId, bytesWrittenToNetwork, failure)); + } + void onExpiredBeforeSend(long messageId, int messageSize, long timeElapsed, TimeUnit timeUnit) + { + onExpired(messageId, messageSize, timeElapsed, timeUnit, ON_SENT); + } + void onSendFrame(int messageCount, int payloadSizeInBytes) + { + long at = nextId(); + events.put(at, new FrameEvent(SEND_FRAME, at, messageCount, payloadSizeInBytes)); + } + void onSentFrame(int messageCount, int payloadSizeInBytes) + { + long at = nextId(); + events.put(at, new FrameEvent(SENT_FRAME, at, messageCount, payloadSizeInBytes)); + } + void onFailedFrame(int messageCount, int payloadSizeInBytes) + { + long at = nextId(); + events.put(at, new FrameEvent(FAILED_FRAME, at, messageCount, payloadSizeInBytes)); + } + void onArrived(long messageId, int messageSize) + { + long at = nextId(); + events.put(at, new SimpleMessageEventWithSize(ARRIVE, at, messageId, messageSize)); + } + void onArrivedExpired(long messageId, int messageSize, boolean wasCorrupt, long timeElapsed, TimeUnit timeUnit) + { + onExpired(messageId, messageSize, timeElapsed, timeUnit, ExpirationType.ON_ARRIVED); + } + void onDeserialize(long messageId, int messagingVersion) + { + long at = nextId(); + events.put(at, new SerializeMessageEvent(DESERIALIZE, at, messageId, messagingVersion)); + } + void onClosedBeforeArrival(long messageId, int messageSize) + { + long at = nextId(); + events.put(at, new SimpleMessageEventWithSize(CLOSED_BEFORE_ARRIVAL, at, messageId, messageSize)); + } + void onFailedDeserialize(long messageId, int messageSize) + { + long at = nextId(); + events.put(at, new SimpleMessageEventWithSize(FAILED_DESERIALIZE, at, messageId, messageSize)); + } + void process(Message message) + { + long at = nextId(); + events.put(at, new ProcessMessageEvent(at, message)); + } + void onProcessExpired(long messageId, int messageSize, long timeElapsed, TimeUnit timeUnit) + { + onExpired(messageId, messageSize, timeElapsed, timeUnit, ExpirationType.ON_PROCESSED); + } + private void onExpired(long messageId, int messageSize, long timeElapsed, TimeUnit timeUnit, ExpirationType expirationType) + { + long at = nextId(); + events.put(at, new ExpiredMessageEvent(at, messageId, messageSize, timeElapsed, timeUnit, expirationType)); + } + + + + static class ConnectOutboundEvent extends SimpleEvent + { + final int messagingVersion; + final OutboundConnectionSettings settings; + ConnectOutboundEvent(long at, int messagingVersion, OutboundConnectionSettings settings) + { + super(EventType.CONNECT_OUTBOUND, at); + this.messagingVersion = messagingVersion; + this.settings = settings; + } + } + + // TODO: do we need this? + static class ConnectInboundEvent extends SimpleEvent + { + final int messagingVersion; + final InboundMessageHandler handler; + ConnectInboundEvent(long at, int messagingVersion, InboundMessageHandler handler) + { + super(EventType.CONNECT_OUTBOUND, at); + this.messagingVersion = messagingVersion; + this.handler = handler; + } + } + + static class SyncEvent extends SimpleEvent + { + final Runnable onCompletion; + SyncEvent(long at, Runnable onCompletion) + { + super(EventType.SYNC, at); + this.onCompletion = onCompletion; + } + } + + static class ControllerEvent extends BoundedEvent + { + final long minimumBytesInFlight; + final long maximumBytesInFlight; + ControllerEvent(long start, long minimumBytesInFlight, long maximumBytesInFlight) + { + super(EventType.CONTROLLER_UPDATE, start); + this.minimumBytesInFlight = minimumBytesInFlight; + this.maximumBytesInFlight = maximumBytesInFlight; + } + } + + void onSync(Runnable onCompletion) + { + SyncEvent connect = new SyncEvent(nextId(), onCompletion); + events.put(connect.at, connect); + } + + void onConnectOutbound(int messagingVersion, OutboundConnectionSettings settings) + { + ConnectOutboundEvent connect = new ConnectOutboundEvent(nextId(), messagingVersion, settings); + events.put(connect.at, connect); + } + + void onConnectInbound(int messagingVersion, InboundMessageHandler handler) + { + ConnectInboundEvent connect = new ConnectInboundEvent(nextId(), messagingVersion, handler); + events.put(connect.at, connect); + } + + private final BytesInFlightController controller; + private final AtomicLong sequenceId = new AtomicLong(); + private final EventSequence events = new EventSequence(); + private final InboundMessageHandlers inbound; + private final OutboundConnection outbound; + + Verifier(BytesInFlightController controller, OutboundConnection outbound, InboundMessageHandlers inbound) + { + this.controller = controller; + this.inbound = inbound; + this.outbound = outbound; + } + + private long nextId() + { + return sequenceId.getAndIncrement(); + } + + public void logFailure(String message, Object ... params) + { + fail(message, params); + } + + private void fail(String message, Object ... params) + { + logger.error("{}", String.format(message, params)); + logger.error("Connection: {}", currentConnection); + } + + private void fail(String message, Throwable t, Object ... params) + { + logger.error("{}", String.format(message, params), t); + logger.error("Connection: {}", currentConnection); + } + + private void failinfo(String message, Object ... params) + { + logger.error("{}", String.format(message, params)); + } + + private static class MessageState + { + final Message message; + final Destiny destiny; + int messagingVersion; + // set initially to message.expiresAtNanos, but if at serialization time we use + // an older messaging version we may not be able to serialize expiration + long expiresAtNanos; + long enqueueStart, enqueueEnd, serialize, arrive, deserialize; + boolean processOnEventLoop, processOutOfOrder; + Event sendState, receiveState; + long lastUpdateAt; + long lastUpdateNanos; + ConnectionState sentOn; + boolean doneSend, doneReceive; + + int messageSize() + { + return message.serializedSize(messagingVersion); + } + + MessageState(Message message, Destiny destiny, long enqueueStart) + { + this.message = message; + this.destiny = destiny; + this.enqueueStart = enqueueStart; + this.expiresAtNanos = message.expiresAtNanos(); + } + + void update(SimpleEvent event, long now) + { + update(event, event.at, now); + } + void update(Event event, long at, long now) + { + lastUpdateAt = at; + lastUpdateNanos = now; + switch (event.type.category) + { + case SEND: + sendState = event; + break; + case RECEIVE: + receiveState = event; + break; + default: throw new IllegalStateException(); + } + } + + boolean is(EventType type) + { + switch (type.category) + { + case SEND: return sendState != null && sendState.type == type; + case RECEIVE: return receiveState != null && receiveState.type == type; + default: return false; + } + } + + boolean is(EventType type1, EventType type2) + { + return is(type1) || is(type2); + } + + boolean is(EventType type1, EventType type2, EventType type3) + { + return is(type1) || is(type2) || is(type3); + } + + void require(EventType event, Verifier verifier, EventType type) + { + if (!is(type)) + verifier.fail("Invalid state at %s for %s: expected %s", event, this, type); + } + + void require(EventType event, Verifier verifier, EventType type1, EventType type2) + { + if (!is(type1) && !is(type2)) + verifier.fail("Invalid state at %s for %s: expected %s or %s", event, this, type1, type2); + } + + void require(EventType event, Verifier verifier, EventType type1, EventType type2, EventType type3) + { + if (!is(type1) && !is(type2) && !is(type3)) + verifier.fail("Invalid state %s for %s: expected %s, %s or %s", event, this, type1, type2, type3); + } + + public String toString() + { + return String.format("{id:%d, state:[%s,%s], upd:%d, ver:%d, enqueue:[%d,%d], ser:%d, arr:%d, deser:%d, expires:%d, sentOn: %d}", + message.id(), sendState, receiveState, lastUpdateAt, messagingVersion, enqueueStart, enqueueEnd, serialize, arrive, deserialize, approxTime.translate().toMillisSinceEpoch(expiresAtNanos), sentOn == null ? -1 : sentOn.connectionId); + } + } + + private final LongObjectOpenHashMap messages = new LongObjectOpenHashMap<>(); + + // messages start here, but may enter in a haphazard (non-sequential) fashion; + // ENQUEUE_START, ENQUEUE_END both take place here, with the latter imposing bounds on the out-of-order appearance of messages. + // note that ENQUEUE_END - being concurrent - may not appear before the message's lifespan has completely ended. + private final Queue enqueueing = new Queue<>(); + + static final class ConnectionState + { + final long connectionId; + final int messagingVersion; + // Strict message order will then be determined at serialization time, since this happens on a single thread. + // The order in which messages arrive here determines the order they will arrive on the other node. + // must follow either ENQUEUE_START or ENQUEUE_END + final Queue serializing = new Queue<>(); + + // Messages sent on the small connection will all be sent in frames; this is a concurrent operation, + // so only the sendingFrame MUST be encountered before any future events - + // large connections skip this step and goes straight to arriving + // we consult the queues in reverse order in arriving, as it is acceptable to find our frame in any of these queues + final FramesInFlight framesInFlight = new FramesInFlight(); // unknown if the messages will arrive, accept either + + // for large messages OR < VERSION_40, arriving can occur BEFORE serializing completes successfully + // OR a frame is fully serialized + final Queue arriving = new Queue<>(); + + final Queue deserializingOnEventLoop = new Queue<>(), + deserializingOffEventLoop = new Queue<>(); + + InboundMessageHandler inbound; + + // TODO + long sentCount, sentBytes; + long receivedCount, receivedBytes; + + ConnectionState(long connectionId, int messagingVersion) + { + this.connectionId = connectionId; + this.messagingVersion = messagingVersion; + } + + public String toString() + { + return String.format("{id: %d, ver: %d, ser: %d, inFlight: %s, arriving: %d, deserOn: %d, deserOff: %d}", + connectionId, messagingVersion, serializing.size(), framesInFlight, arriving.size(), deserializingOnEventLoop.size(), deserializingOffEventLoop.size()); + } + } + + private final Queue processingOutOfOrder = new Queue<>(); + + private SyncEvent sync; + private long nextMessageId = 0; + private long now; + private long connectionCounter; + private ConnectionState currentConnection = new ConnectionState(connectionCounter++, current_version); + + private long outboundSentCount, outboundSentBytes; + private long outboundSubmittedCount; + private long outboundOverloadedCount, outboundOverloadedBytes; + private long outboundExpiredCount, outboundExpiredBytes; + private long outboundErrorCount, outboundErrorBytes; + + public void run(Runnable onFailure, long deadlineNanos) + { + try + { + long lastEventAt = approxTime.now(); + while ((now = approxTime.now()) < deadlineNanos) + { + Event next = events.await(nextMessageId, 100L, MILLISECONDS); + if (next == null) + { + // decide if we have any messages waiting too long to proceed + while (!processingOutOfOrder.isEmpty()) + { + MessageState m = processingOutOfOrder.get(0); + if (now - m.lastUpdateNanos > SECONDS.toNanos(10L)) + { + fail("Unreasonably long period spent waiting for out-of-order deser/delivery of received message %d", m.message.id()); + MessageState v = maybeRemove(m.message.id(), PROCESS); + controller.fail(v.message.serializedSize(v.messagingVersion == 0 ? current_version : v.messagingVersion)); + processingOutOfOrder.remove(0); + } + else break; + } + + if (sync != null) + { + // if we have waited 100ms since beginning a sync, with no events, and ANY of our queues are + // non-empty, something is probably wrong; however, let's give ourselves a little bit longer + + boolean done = + currentConnection.serializing.isEmpty() + && currentConnection.arriving.isEmpty() + && currentConnection.deserializingOnEventLoop.isEmpty() + && currentConnection.deserializingOffEventLoop.isEmpty() + && currentConnection.framesInFlight.isEmpty() + && enqueueing.isEmpty() + && processingOutOfOrder.isEmpty() + && messages.isEmpty() + && controller.inFlight() == 0; + + //outbound.pendingCount() > 0 ? 5L : 2L + if (!done && now - lastEventAt > SECONDS.toNanos(5L)) + { + // TODO: even 2s or 5s are unreasonable periods of time without _any_ movement on a message waiting to arrive + // this seems to happen regularly on MacOS, but we should confirm this does not happen on Linux + fail("Unreasonably long period spent waiting for sync (%dms)", NANOSECONDS.toMillis(now - lastEventAt)); + messages.forEach((k, v) -> { + failinfo("%s", v); + controller.fail(v.message.serializedSize(v.messagingVersion == 0 ? current_version : v.messagingVersion)); + }); + currentConnection.serializing.clear(); + currentConnection.arriving.clear(); + currentConnection.deserializingOnEventLoop.clear(); + currentConnection.deserializingOffEventLoop.clear(); + enqueueing.clear(); + processingOutOfOrder.clear(); + messages.clear(); + while (!currentConnection.framesInFlight.isEmpty()) + currentConnection.framesInFlight.poll(); + done = true; + } + + if (done) + { + ConnectionUtils.check(outbound) + .pending(0, 0) + .error(outboundErrorCount, outboundErrorBytes) + .submitted(outboundSubmittedCount) + .expired(outboundExpiredCount, outboundExpiredBytes) + .overload(outboundOverloadedCount, outboundOverloadedBytes) + .sent(outboundSentCount, outboundSentBytes) + .check((message, expect, actual) -> fail("%s: expect %d, actual %d", message, expect, actual)); + + sync.onCompletion.run(); + sync = null; + } + } + continue; + } + events.clear(nextMessageId); // TODO: simplify collection if we end up using it exclusively as a queue, as we are now + lastEventAt = now; + + switch (next.type) + { + case ENQUEUE: + { + MessageState m; + EnqueueMessageEvent e = (EnqueueMessageEvent) next; + assert nextMessageId == e.start || nextMessageId == e.end; + assert e.message != null; + if (nextMessageId == e.start) + { + if (sync != null) + fail("Sync in progress - there should be no messages beginning to enqueue"); + + m = new MessageState(e.message, e.destiny, e.start); + messages.put(e.messageId, m); + enqueueing.add(m); + m.update(e, e.start, now); + } + else + { + // warning: enqueueEnd can occur at any time in the future, since it's a different thread; + // it could be arbitrarily paused, long enough even for the messsage to be fully processed + m = messages.get(e.messageId); + if (m != null) + m.enqueueEnd = e.end; + outboundSubmittedCount += 1; + } + break; + } + case FAILED_OVERLOADED: + { + // TODO: verify that we could have exceeded our memory limits + SimpleMessageEvent e = (SimpleMessageEvent) next; + assert nextMessageId == e.at; + MessageState m = remove(e.messageId, enqueueing, messages); + m.require(FAILED_OVERLOADED, this, ENQUEUE); + outboundOverloadedBytes += m.message.serializedSize(current_version); + outboundOverloadedCount += 1; + break; + } + case FAILED_CLOSING: + { + // TODO: verify if this is acceptable due to e.g. inbound refusing to process for long enough + SimpleMessageEvent e = (SimpleMessageEvent) next; + assert nextMessageId == e.at; + MessageState m = messages.remove(e.messageId); // definitely cannot have been sent (in theory) + enqueueing.remove(m); + m.require(FAILED_CLOSING, this, ENQUEUE); + fail("Invalid discard of %d: connection was closing for too long", m.message.id()); + break; + } + case SERIALIZE: + { + // serialize happens serially, so we can compress the asynchronicity of the above enqueue + // into a linear sequence of events we expect to occur on arrival + SerializeMessageEvent e = (SerializeMessageEvent) next; + assert nextMessageId == e.at; + MessageState m = get(e); + assert m.is(ENQUEUE); + m.serialize = e.at; + m.messagingVersion = e.messagingVersion; + if (current_version != e.messagingVersion) + controller.adjust(m.message.serializedSize(current_version), m.message.serializedSize(e.messagingVersion)); + + m.processOnEventLoop = willProcessOnEventLoop(outbound.type(), m.message, e.messagingVersion); + m.expiresAtNanos = expiresAtNanos(m.message, e.messagingVersion); + int mi = enqueueing.indexOf(m); + for (int i = 0 ; i < mi ; ++i) + { + MessageState pm = enqueueing.get(i); + if (pm.enqueueEnd != 0 && pm.enqueueEnd < m.enqueueStart) + { + fail("Invalid order of events: %s enqueued strictly before %s, but serialized after", + pm, m); + } + } + enqueueing.remove(mi); + m.sentOn = currentConnection; + currentConnection.serializing.add(m); + m.update(e, now); + break; + } + case FINISH_SERIALIZE_LARGE: + { + // serialize happens serially, so we can compress the asynchronicity of the above enqueue + // into a linear sequence of events we expect to occur on arrival + SimpleMessageEvent e = (SimpleMessageEvent) next; + assert nextMessageId == e.at; + MessageState m = maybeRemove(e); + outboundSentBytes += m.messageSize(); + outboundSentCount += 1; + m.sentOn.serializing.remove(m); + m.update(e, now); + break; + } + case FAILED_SERIALIZE: + { + FailedSerializeEvent e = (FailedSerializeEvent) next; + assert nextMessageId == e.at; + MessageState m = maybeRemove(e); + + if (outbound.type() == LARGE_MESSAGES) + assert e.failure instanceof InvalidSerializedSizeException || e.failure instanceof Connection.IntentionalIOException || e.failure instanceof Connection.IntentionalRuntimeException; + else + assert e.failure instanceof InvalidSerializedSizeException || e.failure instanceof Connection.IntentionalIOException || e.failure instanceof Connection.IntentionalRuntimeException || e.failure instanceof BufferOverflowException; + + if (e.bytesWrittenToNetwork == 0) // TODO: use header size + messages.remove(m.message.id()); + + InvalidSerializedSizeException ex; + if (outbound.type() != LARGE_MESSAGES + || !(e.failure instanceof InvalidSerializedSizeException) + || ((ex = (InvalidSerializedSizeException) e.failure).expectedSize <= DEFAULT_BUFFER_SIZE && ex.actualSizeAtLeast <= DEFAULT_BUFFER_SIZE) + || (ex.expectedSize > DEFAULT_BUFFER_SIZE && ex.actualSizeAtLeast < DEFAULT_BUFFER_SIZE)) + { + assert e.bytesWrittenToNetwork == 0; + } + + m.require(FAILED_SERIALIZE, this, SERIALIZE); + m.sentOn.serializing.remove(m); + if (m.destiny != Destiny.FAIL_TO_SERIALIZE) + fail("%s failed to serialize, but its destiny was to %s", m, m.destiny); + outboundErrorBytes += m.messageSize(); + outboundErrorCount += 1; + m.update(e, now); + break; + } + case SEND_FRAME: + { + FrameEvent e = (FrameEvent) next; + assert nextMessageId == e.at; + int size = 0; + Frame frame = new Frame(); + MessageState first = currentConnection.serializing.get(0); + int messagingVersion = first.messagingVersion; + for (int i = 0 ; i < e.messageCount ; ++i) + { + MessageState m = currentConnection.serializing.get(i); + size += m.message.serializedSize(m.messagingVersion); + if (m.messagingVersion != messagingVersion) + { + fail("Invalid sequence of events: %s encoded to same frame as %s", + m, first); + } + + frame.add(m); + m.update(e, now); + assert !m.doneSend; + m.doneSend = true; + if (m.doneReceive) + messages.remove(m.message.id()); + } + frame.payloadSizeInBytes = e.payloadSizeInBytes; + frame.messageCount = e.messageCount; + frame.messagingVersion = messagingVersion; + currentConnection.framesInFlight.add(frame); + currentConnection.serializing.removeFirst(e.messageCount); + if (e.payloadSizeInBytes != size) + fail("Invalid frame payload size with %s: expected %d, actual %d", first, size, e.payloadSizeInBytes); + break; + } + case SENT_FRAME: + { + Frame frame = currentConnection.framesInFlight.supplySendStatus(Frame.Status.SUCCESS); + frame.forEach(m -> m.update((SimpleEvent) next, now)); + + outboundSentBytes += frame.payloadSizeInBytes; + outboundSentCount += frame.messageCount; + break; + } + case FAILED_FRAME: + { + // TODO: is it possible for this to be signalled AFTER our reconnect event? probably, in which case this will be wrong + // TODO: verify that this was expected + Frame frame = currentConnection.framesInFlight.supplySendStatus(Frame.Status.FAILED); + frame.forEach(m -> m.update((SimpleEvent) next, now)); + if (frame.messagingVersion >= VERSION_40) + { + // the contents cannot be delivered without the whole frame arriving, so clear the contents now + clear(frame, messages); + currentConnection.framesInFlight.remove(frame); + } + outboundErrorBytes += frame.payloadSizeInBytes; + outboundErrorCount += frame.messageCount; + break; + } + case ARRIVE: + { + SimpleMessageEventWithSize e = (SimpleMessageEventWithSize) next; + assert nextMessageId == e.at; + MessageState m = get(e); + + m.arrive = e.at; + if (e.messageSize != m.messageSize()) + fail("onArrived with invalid size for %s: %d vs %d", m, e.messageSize, m.messageSize()); + + if (outbound.type() == LARGE_MESSAGES) + { + m.require(ARRIVE, this, SERIALIZE, FAILED_SERIALIZE, FINISH_SERIALIZE_LARGE); + } + else + { + if (!m.is(SEND_FRAME, SENT_FRAME)) + { + fail("Invalid order of events: %s arrived before being sent in a frame", m); + break; + } + + int fi = -1, mi = -1; + while (fi + 1 < m.sentOn.framesInFlight.size() && mi < 0) + mi = m.sentOn.framesInFlight.get(++fi).indexOf(m); + + if (fi == m.sentOn.framesInFlight.size()) + { + fail("Invalid state: %s, but no frame in flight was found to contain it", m); + break; + } + + if (fi > 0) + { + // we have skipped over some frames, meaning these have either failed (and we know it) + // or we have not yet heard about them and they have presumably failed, or something + // has gone wrong + fail("BEGIN: Successfully sent frames were not delivered"); + for (int i = 0 ; i < fi ; ++i) + { + Frame skip = m.sentOn.framesInFlight.get(i); + skip.receiveStatus = Frame.Status.FAILED; + if (skip.sendStatus == Frame.Status.SUCCESS) + { + failinfo("Frame %s", skip); + for (int j = 0 ; j < skip.size() ; ++j) + failinfo("Containing: %s", skip.get(j)); + } + clear(skip, messages); + } + m.sentOn.framesInFlight.removeFirst(fi); + failinfo("END: Successfully sent frames were not delivered"); + } + + Frame frame = m.sentOn.framesInFlight.get(0); + for (int i = 0; i < mi; ++i) + fail("Invalid order of events: %s serialized strictly before %s, but arrived after", frame.get(i), m); + + frame.remove(mi); + if (frame.isEmpty()) + m.sentOn.framesInFlight.poll(); + } + m.sentOn.arriving.add(m); + m.update(e, now); + break; + } + case DESERIALIZE: + { + // deserialize may happen in parallel for large messages, but in sequence for small messages + // we currently require that this event be issued before any possible error is thrown + SimpleMessageEvent e = (SimpleMessageEvent) next; + assert nextMessageId == e.at; + MessageState m = get(e); + m.require(DESERIALIZE, this, ARRIVE); + m.deserialize = e.at; + // deserialize may be off-loaded, so we can only impose meaningful ordering constraints + // on those messages we know to have been processed on the event loop + int mi = m.sentOn.arriving.indexOf(m); + if (m.processOnEventLoop) + { + for (int i = 0 ; i < mi ; ++i) + { + MessageState pm = m.sentOn.arriving.get(i); + if (pm.processOnEventLoop) + { + fail("Invalid order of events: %d (%d, %d) arrived strictly before %d (%d, %d), but deserialized after", + pm.message.id(), pm.arrive, pm.deserialize, m.message.id(), m.arrive, m.deserialize); + } + } + m.sentOn.deserializingOnEventLoop.add(m); + } + else + { + m.sentOn.deserializingOffEventLoop.add(m); + } + m.sentOn.arriving.remove(mi); + m.update(e, now); + break; + } + case CLOSED_BEFORE_ARRIVAL: + { + SimpleMessageEventWithSize e = (SimpleMessageEventWithSize) next; + assert nextMessageId == e.at; + MessageState m = maybeRemove(e); + + if (e.messageSize != m.messageSize()) + fail("onClosedBeforeArrival has invalid size for %s: %d vs %d", m, e.messageSize, m.messageSize()); + + m.sentOn.deserializingOffEventLoop.remove(m); + if (m.destiny == Destiny.FAIL_TO_SERIALIZE && outbound.type() == LARGE_MESSAGES) + break; + fail("%s closed before arrival, but its destiny was to %s", m, m.destiny); + break; + } + case FAILED_DESERIALIZE: + { + SimpleMessageEventWithSize e = (SimpleMessageEventWithSize) next; + assert nextMessageId == e.at; + MessageState m = maybeRemove(e); + + if (e.messageSize != m.messageSize()) + fail("onFailedDeserialize has invalid size for %s: %d vs %d", m, e.messageSize, m.messageSize()); + m.require(FAILED_DESERIALIZE, this, ARRIVE, DESERIALIZE); + (m.processOnEventLoop ? m.sentOn.deserializingOnEventLoop : m.sentOn.deserializingOffEventLoop).remove(m); + switch (m.destiny) + { + case FAIL_TO_DESERIALIZE: + break; + case FAIL_TO_SERIALIZE: + if (outbound.type() == LARGE_MESSAGES) + break; + default: + fail("%s failed to deserialize, but its destiny was to %s", m, m.destiny); + } + break; + } + case PROCESS: + { + ProcessMessageEvent e = (ProcessMessageEvent) next; + assert nextMessageId == e.at; + MessageState m = maybeRemove(e); + + m.require(PROCESS, this, DESERIALIZE); + if (!Arrays.equals((byte[]) e.message.payload, (byte[]) m.message.payload)) + { + fail("Invalid message payload for %d: %s supplied by processor, but %s implied by original message and messaging version", + e.messageId, Arrays.toString((byte[]) e.message.payload), Arrays.toString((byte[]) m.message.payload)); + } + + if (m.processOutOfOrder) + { + assert !m.processOnEventLoop; // will have already been reported small (processOnEventLoop) messages + processingOutOfOrder.remove(m); + } + else if (m.processOnEventLoop) + { + // we can expect that processing happens sequentially in this case, more specifically + // we can actually expect that this event will occur _immediately_ after the deserialize event + // so that we have exactly one mess + // c + int mi = m.sentOn.deserializingOnEventLoop.indexOf(m); + for (int i = 0 ; i < mi ; ++i) + { + MessageState pm = m.sentOn.deserializingOnEventLoop.get(i); + fail("Invalid order of events: %s deserialized strictly before %s, but processed after", + pm, m); + } + clearFirst(mi, m.sentOn.deserializingOnEventLoop, messages); + m.sentOn.deserializingOnEventLoop.poll(); + } + else + { + int mi = m.sentOn.deserializingOffEventLoop.indexOf(m); + // process may be off-loaded, so we can only impose meaningful ordering constraints + // on those messages we know to have been processed on the event loop + for (int i = 0 ; i < mi ; ++i) + { + MessageState pm = m.sentOn.deserializingOffEventLoop.get(i); + pm.processOutOfOrder = true; + processingOutOfOrder.add(pm); + } + m.sentOn.deserializingOffEventLoop.removeFirst(mi + 1); + } + // this message has been fully validated + break; + } + case FAILED_EXPIRED_ON_SEND: + case FAILED_EXPIRED_ON_RECEIVE: + { + ExpiredMessageEvent e = (ExpiredMessageEvent) next; + assert nextMessageId == e.at; + MessageState m; + switch (e.expirationType) + { + case ON_SENT: + { + m = messages.remove(e.messageId); + m.require(e.type, this, ENQUEUE); + outboundExpiredBytes += m.message.serializedSize(current_version); + outboundExpiredCount += 1; + messages.remove(m.message.id()); + break; + } + case ON_ARRIVED: + m = maybeRemove(e); + if (!m.is(ARRIVE)) + { + if (outbound.type() != LARGE_MESSAGES) m.require(e.type, this, SEND_FRAME, SENT_FRAME, FAILED_FRAME); + else m.require(e.type, this, SERIALIZE, FAILED_SERIALIZE, FINISH_SERIALIZE_LARGE); + } + break; + case ON_PROCESSED: + m = maybeRemove(e); + m.require(e.type, this, DESERIALIZE); + break; + default: + throw new IllegalStateException(); + } + + now = System.nanoTime(); + if (m.expiresAtNanos > now) + { + // we fix the conversion AlmostSameTime for an entire run, which should suffice to guarantee these comparisons + fail("Invalid expiry of %d: expiry should occur in %dms; event believes %dms have elapsed, and %dms have actually elapsed", m.message.id(), + NANOSECONDS.toMillis(m.expiresAtNanos - m.message.createdAtNanos()), + e.timeUnit.toMillis(e.timeElapsed), + NANOSECONDS.toMillis(now - m.message.createdAtNanos())); + } + + switch (e.expirationType) + { + case ON_SENT: + enqueueing.remove(m); + break; + case ON_ARRIVED: + if (m.is(ARRIVE)) + m.sentOn.arriving.remove(m); + switch (m.sendState.type) + { + case SEND_FRAME: + case SENT_FRAME: + case FAILED_FRAME: + // TODO: this should be robust to re-ordering; should perhaps extract a common method + m.sentOn.framesInFlight.get(0).remove(m); + if (m.sentOn.framesInFlight.get(0).isEmpty()) + m.sentOn.framesInFlight.poll(); + break; + } + break; + case ON_PROCESSED: + (m.processOnEventLoop ? m.sentOn.deserializingOnEventLoop : m.sentOn.deserializingOffEventLoop).remove(m); + break; + } + + if (m.messagingVersion != 0 && e.messageSize != m.messageSize()) + fail("onExpired %s with invalid size for %s: %d vs %d", e.expirationType, m, e.messageSize, m.messageSize()); + + break; + } + case CONTROLLER_UPDATE: + { + break; + } + case CONNECT_OUTBOUND: + { + ConnectOutboundEvent e = (ConnectOutboundEvent) next; + currentConnection = new ConnectionState(connectionCounter++, e.messagingVersion); + break; + } + case SYNC: + { + sync = (SyncEvent) next; + break; + } + default: + throw new IllegalStateException(); + } + ++nextMessageId; + } + } + catch (InterruptedException e) + { + } + catch (Throwable t) + { + logger.error("Unexpected error:", t); + onFailure.run(); + } + } + + private MessageState get(SimpleMessageEvent onEvent) + { + MessageState m = messages.get(onEvent.messageId); + if (m == null) + throw new IllegalStateException("Missing " + onEvent + ": " + onEvent.messageId); + return m; + } + private MessageState maybeRemove(SimpleMessageEvent onEvent) + { + return maybeRemove(onEvent.messageId, onEvent.type, onEvent); + } + private MessageState maybeRemove(long messageId, EventType onEvent) + { + return maybeRemove(messageId, onEvent, onEvent); + } + private MessageState maybeRemove(long messageId, EventType onEvent, Object id) + { + MessageState m = messages.get(messageId); + if (m == null) + throw new IllegalStateException("Missing " + id + ": " + messageId); + switch (onEvent.category) + { + case SEND: + if (m.doneSend) + fail("%s already doneSend %s", onEvent, m); + m.doneSend = true; + if (m.doneReceive) messages.remove(messageId); + break; + case RECEIVE: + if (m.doneReceive) + fail("%s already doneReceive %s", onEvent, m); + m.doneReceive = true; + if (m.doneSend) messages.remove(messageId); + } + return m; + } + + + private static class Frame extends Queue + { + enum Status { SUCCESS, FAILED, UNKNOWN } + Status sendStatus = Status.UNKNOWN, receiveStatus = Status.UNKNOWN; + int messagingVersion; + int messageCount; + int payloadSizeInBytes; + + public String toString() + { + return String.format("{count:%d, size:%d, version:%d, send:%s, receive:%s}", + messageCount, payloadSizeInBytes, messagingVersion, sendStatus, receiveStatus); + } + } + + private static MessageState remove(long messageId, Queue queue, LongObjectOpenHashMap lookup) + { + MessageState m = lookup.remove(messageId); + queue.remove(m); + return m; + } + + private static void clearFirst(int count, Queue queue, LongObjectOpenHashMap lookup) + { + if (count > 0) + { + for (int i = 0 ; i < count ; ++i) + lookup.remove(queue.get(i).message.id()); + queue.removeFirst(count); + } + } + + private static void clear(Queue queue, LongObjectOpenHashMap lookup) + { + if (!queue.isEmpty()) + clearFirst(queue.size(), queue, lookup); + } + + private static class EventSequence + { + static final int CHUNK_SIZE = 1 << 10; + static class Chunk extends AtomicReferenceArray + { + final long sequenceId; + int removed = 0; + Chunk(long sequenceId) + { + super(CHUNK_SIZE); + this.sequenceId = sequenceId; + } + Event get(long sequenceId) + { + return get((int)(sequenceId - this.sequenceId)); + } + void set(long sequenceId, Event event) + { + lazySet((int)(sequenceId - this.sequenceId), event); + } + } + + // we use a concurrent skip list to permit efficient searching, even if we always append + final ConcurrentSkipListMap chunkList = new ConcurrentSkipListMap<>(); + final WaitQueue writerWaiting = new WaitQueue(); + + volatile Chunk writerChunk = new Chunk(0); + Chunk readerChunk = writerChunk; + + long readerWaitingFor; + volatile Thread readerWaiting; + + EventSequence() + { + chunkList.put(0L, writerChunk); + } + + public void put(long sequenceId, Event event) + { + long chunkSequenceId = sequenceId & -CHUNK_SIZE; + Chunk chunk = writerChunk; + if (chunk.sequenceId != chunkSequenceId) + { + try + { + chunk = ensureChunk(chunkSequenceId); + } + catch (InterruptedException e) + { + throw new RuntimeException(e); + } + } + + chunk.set(sequenceId, event); + + Thread wake = readerWaiting; + long wakeIf = readerWaitingFor; // we are guarded by the above volatile read + if (wake != null && wakeIf == sequenceId) + LockSupport.unpark(wake); + } + + Chunk ensureChunk(long chunkSequenceId) throws InterruptedException + { + Chunk chunk = chunkList.get(chunkSequenceId); + if (chunk == null) + { + Map.Entry e; + while ( null != (e = chunkList.firstEntry()) && chunkSequenceId - e.getKey() > 1 << 12) + { + WaitQueue.Signal signal = writerWaiting.register(); + if (null != (e = chunkList.firstEntry()) && chunkSequenceId - e.getKey() > 1 << 12) + signal.await(); + else + signal.cancel(); + } + chunk = chunkList.get(chunkSequenceId); + if (chunk == null) + { + synchronized (this) + { + chunk = chunkList.get(chunkSequenceId); + if (chunk == null) + chunkList.put(chunkSequenceId, chunk = new Chunk(chunkSequenceId)); + } + } + } + return chunk; + } + + Chunk readerChunk(long readerId) throws InterruptedException + { + long chunkSequenceId = readerId & -CHUNK_SIZE; + if (readerChunk.sequenceId != chunkSequenceId) + readerChunk = ensureChunk(chunkSequenceId); + return readerChunk; + } + + public Event await(long id, long timeout, TimeUnit unit) throws InterruptedException + { + return await(id, System.nanoTime() + unit.toNanos(timeout)); + } + + public Event await(long id, long deadlineNanos) throws InterruptedException + { + Chunk chunk = readerChunk(id); + Event result = chunk.get(id); + if (result != null) + return result; + + readerWaitingFor = id; + readerWaiting = Thread.currentThread(); + while (null == (result = chunk.get(id))) + { + long waitNanos = deadlineNanos - System.nanoTime(); + if (waitNanos <= 0) + return null; + LockSupport.parkNanos(waitNanos); + if (Thread.interrupted()) + throw new InterruptedException(); + } + readerWaitingFor = -1; + readerWaiting = null; + return result; + } + + public Event find(long sequenceId) + { + long chunkSequenceId = sequenceId & -CHUNK_SIZE; + Chunk chunk = readerChunk; + if (chunk.sequenceId != chunkSequenceId) + { + chunk = writerChunk; + if (chunk.sequenceId != chunkSequenceId) + chunk = chunkList.get(chunkSequenceId); + } + return chunk.get(sequenceId); + } + + public void clear(long sequenceId) + { + long chunkSequenceId = sequenceId & -CHUNK_SIZE; + Chunk chunk = chunkList.get(chunkSequenceId); + chunk.set(sequenceId, null); + if (++chunk.removed == CHUNK_SIZE) + { + chunkList.remove(chunkSequenceId); + writerWaiting.signalAll(); + } + } + } + + static class Queue + { + private Object[] items = new Object[10]; + private int begin, end; + + int size() + { + return end - begin; + } + + T get(int i) + { + //noinspection unchecked + return (T) items[i + begin]; + } + + int indexOf(T item) + { + for (int i = begin ; i < end ; ++i) + { + if (item == items[i]) + return i - begin; + } + return -1; + } + + void remove(T item) + { + int i = indexOf(item); + if (i >= 0) + remove(i); + } + + void remove(int i) + { + i += begin; + assert i < end; + + if (i == begin || i + 1 == end) + { + items[i] = null; + if (begin + 1 == end) begin = end = 0; + else if (i == begin) ++begin; + else --end; + } + else if (i - begin < end - i) + { + System.arraycopy(items, begin, items, begin + 1, i - begin); + items[begin++] = null; + } + else + { + System.arraycopy(items, i + 1, items, i, (end - 1) - i); + items[--end] = null; + } + } + + void add(T item) + { + if (end == items.length) + { + Object[] src = items; + Object[] trg; + if (end - begin < src.length / 2) trg = src; + else trg = new Object[src.length * 2]; + System.arraycopy(src, begin, trg, 0, end - begin); + end -= begin; + begin = 0; + items = trg; + } + items[end++] = item; + } + + void clear() + { + Arrays.fill(items, begin, end, null); + begin = end = 0; + } + + void removeFirst(int count) + { + Arrays.fill(items, begin, begin + count, null); + begin += count; + if (begin == end) + begin = end = 0; + } + + T poll() + { + if (begin == end) + return null; + //noinspection unchecked + T result = (T) items[begin]; + items[begin++] = null; + if (begin == end) + begin = end = 0; + return result; + } + + void forEach(Consumer consumer) + { + for (int i = 0 ; i < size() ; ++i) + consumer.accept(get(i)); + } + + boolean isEmpty() + { + return begin == end; + } + + public String toString() + { + StringBuilder result = new StringBuilder(); + result.append('['); + toString(result); + result.append(']'); + return result.toString(); + } + + void toString(StringBuilder out) + { + for (int i = 0 ; i < size() ; ++i) + { + if (i > 0) out.append(", "); + out.append(get(i)); + } + } + } + + + + static class FramesInFlight + { + // this may be negative, indicating we have processed a frame whose status we did not know at the time + // TODO: we should verify the status of these frames by logging the inferred status and verifying it matches + final Queue inFlight = new Queue<>(); + final Queue retiredWithoutStatus = new Queue<>(); + private int withStatus; + + Frame supplySendStatus(Frame.Status status) + { + Frame frame; + if (withStatus >= 0) frame = inFlight.get(withStatus); + else frame = retiredWithoutStatus.poll(); + assert frame.sendStatus == Frame.Status.UNKNOWN; + frame.sendStatus = status; + ++withStatus; + return frame; + } + + boolean isEmpty() + { + return inFlight.isEmpty(); + } + + int size() + { + return inFlight.size(); + } + + Frame get(int i) + { + return inFlight.get(i); + } + + void add(Frame frame) + { + assert frame.sendStatus == Frame.Status.UNKNOWN; + inFlight.add(frame); + } + + void remove(Frame frame) + { + int i = inFlight.indexOf(frame); + if (i > 0) throw new IllegalStateException(); + if (i == 0) poll(); + } + + void removeFirst(int count) + { + while (count-- > 0) + poll(); + } + + Frame poll() + { + Frame frame = inFlight.poll(); + if (--withStatus < 0) + { + assert frame.sendStatus == Frame.Status.UNKNOWN; + retiredWithoutStatus.add(frame); + } + else + assert frame.sendStatus != Frame.Status.UNKNOWN; + return frame; + } + + public String toString() + { + StringBuilder result = new StringBuilder(); + result.append("[withStatus="); + result.append(withStatus); + result.append("; "); + inFlight.toString(result); + result.append("; "); + retiredWithoutStatus.toString(result); + result.append(']'); + return result.toString(); + } + } + + private static boolean willProcessOnEventLoop(ConnectionType type, Message message, int messagingVersion) + { + int size = message.serializedSize(messagingVersion); + if (type == ConnectionType.SMALL_MESSAGES && messagingVersion >= VERSION_40) + return size <= LARGE_MESSAGE_THRESHOLD; + else if (messagingVersion >= VERSION_40) + return size <= DEFAULT_BUFFER_SIZE; + else + return size <= LARGE_MESSAGE_THRESHOLD; + } + + private static long expiresAtNanos(Message message, int messagingVersion) + { + return messagingVersion < VERSION_40 ? message.verb().expiresAtNanos(message.createdAtNanos()) + : message.expiresAtNanos(); + } + +} diff --git a/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java b/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java index 57aa940abd94..838038a6756b 100644 --- a/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java +++ b/test/burn/org/apache/cassandra/utils/memory/LongBufferPoolTest.java @@ -33,6 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.utils.DynamicList; @@ -68,6 +69,42 @@ public class LongBufferPoolTest private static final int STDEV_BUFFER_SIZE = 10 << 10; // picked to ensure exceeding buffer size is rare, but occurs private static final DateFormat DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss"); + static final class Debug implements BufferPool.Debug + { + static class DebugChunk + { + volatile long lastRecycled; + static DebugChunk get(BufferPool.Chunk chunk) + { + if (chunk.debugAttachment == null) + chunk.debugAttachment = new DebugChunk(); + return (DebugChunk) chunk.debugAttachment; + } + } + long recycleRound = 1; + final List normalChunks = new ArrayList<>(); + final List tinyChunks = new ArrayList<>(); + public synchronized void registerNormal(BufferPool.Chunk chunk) + { + chunk.debugAttachment = new DebugChunk(); + normalChunks.add(chunk); + } + public void recycleNormal(BufferPool.Chunk oldVersion, BufferPool.Chunk newVersion) + { + newVersion.debugAttachment = oldVersion.debugAttachment; + DebugChunk.get(oldVersion).lastRecycled = recycleRound; + } + public synchronized void check() + { +// for (BufferPool.Chunk chunk : tinyChunks) +// assert DebugChunk.get(chunk).lastRecycled == recycleRound; + for (BufferPool.Chunk chunk : normalChunks) + assert DebugChunk.get(chunk).lastRecycled == recycleRound; + tinyChunks.clear(); // they don't survive a recycleRound + recycleRound++; + } + } + @BeforeClass public static void setup() throws Exception { @@ -133,7 +170,7 @@ private static final class TestEnvironment makingProgress = new AtomicBoolean[threadCount]; burnFreed = new AtomicBoolean(false); freedAllMemory = new AtomicBoolean[threadCount]; - executorService = Executors.newFixedThreadPool(threadCount + 2); + executorService = Executors.newFixedThreadPool(threadCount + 2, new NamedThreadFactory("test")); threadResultFuture = new ArrayList<>(threadCount); for (int i = 0; i < sharedRecycle.length; i++) @@ -149,7 +186,7 @@ private static final class TestEnvironment // using their own algorithm the targetSize should be poolSize / targetSizeQuanta. // // This should divide double the poolSize across the working threads, - // plus CHUNK_SIZE for thread0 and 1/10 poolSize for the burn producer/consumer pair. + // plus NORMAL_CHUNK_SIZE for thread0 and 1/10 poolSize for the burn producer/consumer pair. targetSizeQuanta = 2 * poolSize / sum1toN(threadCount - 1); } @@ -209,7 +246,8 @@ public void testAllocate(int threadCount, long duration, int poolSize) throws In long prevPoolSize = BufferPool.MEMORY_USAGE_THRESHOLD; logger.info("Overriding configured BufferPool.MEMORY_USAGE_THRESHOLD={} and enabling BufferPool.DEBUG", poolSize); BufferPool.MEMORY_USAGE_THRESHOLD = poolSize; - BufferPool.DEBUG = true; + Debug debug = new Debug(); + BufferPool.debug(debug); TestEnvironment testEnv = new TestEnvironment(threadCount, duration, poolSize); @@ -230,7 +268,7 @@ public void testAllocate(int threadCount, long duration, int poolSize) throws In for (AtomicBoolean freedMemory : testEnv.freedAllMemory) allFreed = allFreed && freedMemory.getAndSet(false); if (allFreed) - BufferPool.assertAllRecycled(); + debug.check(); else logger.info("All threads did not free all memory in this time slot - skipping buffer recycle check"); } @@ -250,7 +288,7 @@ public void testAllocate(int threadCount, long duration, int poolSize) throws In logger.info("Reverting BufferPool.MEMORY_USAGE_THRESHOLD={}", prevPoolSize); BufferPool.MEMORY_USAGE_THRESHOLD = prevPoolSize; - BufferPool.DEBUG = false; + BufferPool.debug(null); testEnv.assertCheckedThreadsSucceeded(); @@ -262,7 +300,7 @@ private Future startWorkerThread(TestEnvironment testEnv, final int thr { return testEnv.executorService.submit(new TestUntil(testEnv.until) { - final int targetSize = threadIdx == 0 ? BufferPool.CHUNK_SIZE : testEnv.targetSizeQuanta * threadIdx; + final int targetSize = threadIdx == 0 ? BufferPool.NORMAL_CHUNK_SIZE : testEnv.targetSizeQuanta * threadIdx; final SPSCQueue shareFrom = testEnv.sharedRecycle[threadIdx]; final DynamicList checks = new DynamicList<>((int) Math.max(1, targetSize / (1 << 10))); final SPSCQueue shareTo = testEnv.sharedRecycle[(threadIdx + 1) % testEnv.threadCount]; @@ -279,7 +317,6 @@ void checkpoint() void testOne() throws Exception { - long currentTargetSize = (rand.nextInt(testEnv.poolSize / 1024) == 0 || !testEnv.freedAllMemory[threadIdx].get()) ? 0 : targetSize; int spinCount = 0; while (totalSize > currentTargetSize - freeingSize) @@ -309,8 +346,8 @@ else if (!recycleFromNeighbour()) checks.remove(check.listnode); check.validate(); - size = BufferPool.roundUpNormal(check.buffer.capacity()); - if (size > BufferPool.CHUNK_SIZE) + size = BufferPool.roundUp(check.buffer.capacity()); + if (size > BufferPool.NORMAL_CHUNK_SIZE) size = 0; // either share to free, or free immediately @@ -334,9 +371,9 @@ else if (!recycleFromNeighbour()) // allocate a new buffer size = (int) Math.max(1, AVG_BUFFER_SIZE + (STDEV_BUFFER_SIZE * rand.nextGaussian())); - if (size <= BufferPool.CHUNK_SIZE) + if (size <= BufferPool.NORMAL_CHUNK_SIZE) { - totalSize += BufferPool.roundUpNormal(size); + totalSize += BufferPool.roundUp(size); allocate(size); } else if (rand.nextBoolean()) @@ -349,10 +386,10 @@ else if (rand.nextBoolean()) while (totalSize < testEnv.poolSize) { size = (int) Math.max(1, AVG_BUFFER_SIZE + (STDEV_BUFFER_SIZE * rand.nextGaussian())); - if (size <= BufferPool.CHUNK_SIZE) + if (size <= BufferPool.NORMAL_CHUNK_SIZE) { allocate(size); - totalSize += BufferPool.roundUpNormal(size); + totalSize += BufferPool.roundUp(size); } } } @@ -443,7 +480,7 @@ private void startBurnerThreads(TestEnvironment testEnv) final ThreadLocalRandom rand = ThreadLocalRandom.current(); void testOne() throws Exception { - if (count * BufferPool.CHUNK_SIZE >= testEnv.poolSize / 10) + if (count * BufferPool.NORMAL_CHUNK_SIZE >= testEnv.poolSize / 10) { if (burn.exhausted) { @@ -456,7 +493,8 @@ void testOne() throws Exception return; } - ByteBuffer buffer = BufferPool.tryGet(BufferPool.CHUNK_SIZE); + ByteBuffer buffer = rand.nextInt(4) < 1 ? BufferPool.tryGet(BufferPool.NORMAL_CHUNK_SIZE) + : BufferPool.tryGet(BufferPool.TINY_ALLOCATION_LIMIT); if (buffer == null) { Thread.yield(); @@ -523,7 +561,7 @@ public Boolean call() throws Exception { logger.error("Got exception {}, current chunk {}", ex.getMessage(), - BufferPool.currentChunk()); + BufferPool.unsafeCurrentChunk()); ex.printStackTrace(); return false; } @@ -531,7 +569,7 @@ public Boolean call() throws Exception { logger.error("Got throwable {}, current chunk {}", tr.getMessage(), - BufferPool.currentChunk()); + BufferPool.unsafeCurrentChunk()); tr.printStackTrace(); return false; } @@ -555,6 +593,7 @@ public static void main(String[] args) catch (Throwable tr) { System.out.println(String.format("Test failed - %s", tr.getMessage())); + tr.printStackTrace(); System.exit(1); // Force exit so that non-daemon threads like REQUEST-SCHEDULER do not hang the process on failure } } diff --git a/test/conf/cassandra-murmur.yaml b/test/conf/cassandra-murmur.yaml index e933837ccdee..3c263a5af17c 100644 --- a/test/conf/cassandra-murmur.yaml +++ b/test/conf/cassandra-murmur.yaml @@ -13,7 +13,7 @@ cdc_enabled: false hints_directory: build/test/cassandra/hints partitioner: org.apache.cassandra.dht.Murmur3Partitioner listen_address: 127.0.0.1 -storage_port: 7010 +storage_port: 7012 start_native_transport: true native_transport_port: 9042 column_index_size_in_kb: 4 @@ -24,7 +24,7 @@ disk_access_mode: mmap seed_provider: - class_name: org.apache.cassandra.locator.SimpleSeedProvider parameters: - - seeds: "127.0.0.1" + - seeds: "127.0.0.1:7012" endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch dynamic_snitch: true server_encryption_options: diff --git a/test/conf/cassandra-seeds.yaml b/test/conf/cassandra-seeds.yaml index 02d25d232cfb..f3279aeb1253 100644 --- a/test/conf/cassandra-seeds.yaml +++ b/test/conf/cassandra-seeds.yaml @@ -14,7 +14,7 @@ cdc_enabled: false hints_directory: build/test/cassandra/hints partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner listen_address: 127.0.0.1 -storage_port: 7010 +storage_port: 7012 start_native_transport: true native_transport_port: 9042 column_index_size_in_kb: 4 diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml index d94c478b9a41..89b7ff180769 100644 --- a/test/conf/cassandra.yaml +++ b/test/conf/cassandra.yaml @@ -16,7 +16,7 @@ cdc_enabled: false hints_directory: build/test/cassandra/hints partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner listen_address: 127.0.0.1 -storage_port: 7010 +storage_port: 7012 ssl_storage_port: 7011 start_native_transport: true native_transport_port: 9042 @@ -28,7 +28,7 @@ disk_access_mode: mmap seed_provider: - class_name: org.apache.cassandra.locator.SimpleSeedProvider parameters: - - seeds: "127.0.0.1:7010" + - seeds: "127.0.0.1:7012" endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch dynamic_snitch: true server_encryption_options: diff --git a/test/conf/logback-burntest.xml b/test/conf/logback-burntest.xml new file mode 100644 index 000000000000..e1e48a9d3fae --- /dev/null +++ b/test/conf/logback-burntest.xml @@ -0,0 +1,66 @@ + + + + + + + + + + + ./build/test/logs/${cassandra.testtag}/TEST-${suitename}.log + + ./build/test/logs/${cassandra.testtag}/TEST-${suitename}.log.%i.gz + 1 + 20 + + + + 20MB + + + + %-5level [%thread] ${instance_id} %date{ISO8601} %msg%n + + false + + + + 0 + 0 + 1024 + + + + + + + %-5level [%thread] ${instance_id} %date{ISO8601} %F:%L - %msg%n + + + + + + + + + + + + diff --git a/test/conf/logback-dtest.xml b/test/conf/logback-dtest.xml index b62539fb3c58..13a729defc77 100644 --- a/test/conf/logback-dtest.xml +++ b/test/conf/logback-dtest.xml @@ -58,15 +58,6 @@ - - - %-5level %date{HH:mm:ss,SSS} %msg%n - - - WARN - - - %-5level [%thread] ${instance_id} %date{ISO8601} %F:%L - %msg%n diff --git a/test/data/serialization/4.0/service.SyncComplete.bin b/test/data/serialization/4.0/service.SyncComplete.bin index 15cccb85be8e0596775ec955700849c308f16edf..849faf219b9a1365fdfb53f738feed76f3e6296a 100644 GIT binary patch delta 58 xcmZ3*(!?|&$DpbHL4!u8;L8c}hBv12$E{#sU|g?07+FEY ze=xyT&%nSWodlworIQ#zVyHqO5nrvC0()n{=iBb3_UD{sTLlyY836(*Kn#*EM3ra! p0+wZ9;9{nQJHdv7+{q}N1Y|2fEn7ho%V-cNm`;zPiPOR10sx-wjqv~g diff --git a/test/data/serialization/4.0/service.SyncRequest.bin b/test/data/serialization/4.0/service.SyncRequest.bin index f4eb53285db6f4e796e9fe5efc7d8c27826d695d..f5f2e3d81fe0d675c2c388c0aa0df1c651e0a6cf 100644 GIT binary patch delta 5 Mcmey!m_H#O00)`^VE_OC delta 27 ecmd1*$T%Tin5~|Hfl)e%fq_8*h(REKLOuXfqXh*3 diff --git a/test/data/serialization/4.0/service.ValidationComplete.bin b/test/data/serialization/4.0/service.ValidationComplete.bin index edc90b359706677a9db6ada65f67359883ec7fd2..3f5b31897499937ceecdab6a955edb6bb59de694 100644 GIT binary patch delta 48 zcmaFKd4pv`CgbD=MxDuZjGU9bSOg|tU}j}BnQXu!HTfx{DwyX1g?07+FEY ze=xxaBI|$z5Hx^EAi>C>m0y&umk89Gk*b%RSX>O$T$HGnl2M`;1Z3vt1|$}hWR_&+ z=cN{b)I-hs&q!@Y)&KtwG8+WA$#WDFP^*9g$N(e&7F0kI05QRW0%(Ga|G~l%7{X8q z12hS+Y7K0HDrka?|3MI}-U5phSRw&U0xTGSCI~TzNU;qI4mjoUF diff --git a/test/data/serialization/4.0/service.ValidationRequest.bin b/test/data/serialization/4.0/service.ValidationRequest.bin index e45eb703e08825a5bb80acf4df185c8c4384550d..72d4a1fa000cc77eff8e4de2ffbc864f9a14b5fc 100644 GIT binary patch delta 5 McmZ3<=sm$300p7}(EtDd delta 27 ecmeZX$vDAVn5~|Hfl)e%fq_8*h(W-6f;Rv}y96x& diff --git a/test/distributed/org/apache/cassandra/distributed/Cluster.java b/test/distributed/org/apache/cassandra/distributed/Cluster.java index c7f7675cb847..32750ce09c1f 100644 --- a/test/distributed/org/apache/cassandra/distributed/Cluster.java +++ b/test/distributed/org/apache/cassandra/distributed/Cluster.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.nio.file.Files; import java.util.List; +import java.util.function.Consumer; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.impl.AbstractCluster; @@ -40,18 +41,24 @@ private Cluster(File root, Versions.Version version, List config super(root, version, configs, sharedClassLoader); } - protected IInvokableInstance newInstanceWrapper(Versions.Version version, InstanceConfig config) + protected IInvokableInstance newInstanceWrapper(int generation, Versions.Version version, InstanceConfig config) { - return new Wrapper(version, config); + return new Wrapper(generation, version, config); } - public static Cluster create(int nodeCount) throws Throwable + public static Builder build(int nodeCount) + { + return new Builder<>(nodeCount, Cluster::new); + } + + public static Cluster create(int nodeCount, Consumer configUpdater) throws IOException { - return create(nodeCount, Cluster::new); + return build(nodeCount).withConfig(configUpdater).start(); } - public static Cluster create(int nodeCount, File root) + + public static Cluster create(int nodeCount) throws Throwable { - return create(nodeCount, Versions.CURRENT, root, Cluster::new); + return build(nodeCount).start(); } } diff --git a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java index 0c8e63ae9101..a064ce1975aa 100644 --- a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java @@ -25,6 +25,7 @@ import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.impl.AbstractCluster; +import org.apache.cassandra.distributed.impl.IInvokableInstance; import org.apache.cassandra.distributed.impl.IUpgradeableInstance; import org.apache.cassandra.distributed.impl.InstanceConfig; import org.apache.cassandra.distributed.impl.Versions; @@ -43,28 +44,24 @@ private UpgradeableCluster(File root, Versions.Version version, List build(int nodeCount) { - return create(nodeCount, Versions.CURRENT, root, UpgradeableCluster::new); + return new Builder<>(nodeCount, UpgradeableCluster::new); } - public static UpgradeableCluster create(int nodeCount, Versions.Version version) throws IOException + public static UpgradeableCluster create(int nodeCount) throws Throwable { - return create(nodeCount, version, Files.createTempDirectory("dtests").toFile(), UpgradeableCluster::new); + return build(nodeCount).start(); } - public static UpgradeableCluster create(int nodeCount, Versions.Version version, File root) + + public static UpgradeableCluster create(int nodeCount, Versions.Version version) throws Throwable { - return create(nodeCount, version, root, UpgradeableCluster::new); + return build(nodeCount).withVersion(version).start(); } - } diff --git a/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java index 6741b3fdd59b..c57870342339 100644 --- a/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java +++ b/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java @@ -38,4 +38,5 @@ public interface IInstanceConfig Object get(String fieldName); String getString(String fieldName); int getInt(String fieldName); + boolean has(long featureFlag); } diff --git a/test/distributed/org/apache/cassandra/distributed/api/IMessage.java b/test/distributed/org/apache/cassandra/distributed/api/IMessage.java index 1e537ed1ee82..7bc7931a83c1 100644 --- a/test/distributed/org/apache/cassandra/distributed/api/IMessage.java +++ b/test/distributed/org/apache/cassandra/distributed/api/IMessage.java @@ -27,6 +27,7 @@ public interface IMessage { int verb(); byte[] bytes(); + // TODO: need to make this a long int id(); int version(); InetAddressAndPort from(); diff --git a/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java b/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java index b5fde840e748..70491e6ffe31 100644 --- a/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java +++ b/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java @@ -20,6 +20,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; import java.util.function.BiConsumer; @@ -39,10 +40,11 @@ public interface Builder Filter drop(); } - Builder verbs(MessagingService.Verb... verbs); + Builder verbs(Verb ... verbs); Builder allVerbs(); void reset(); // internal - BiConsumer filter(BiConsumer applyIfNotFiltered); + boolean permit(IInstance from, IInstance to, int verb); + } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java index 93a749815f63..86ea14ba83ab 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java @@ -30,11 +30,9 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -57,7 +55,7 @@ import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.SimpleCondition; @@ -91,6 +89,7 @@ public abstract class AbstractCluster implements ICluster, // to ensure we have instantiated the main classloader's LoggerFactory (and any LogbackStatusListener) // before we instantiate any for a new instance private static final Logger logger = LoggerFactory.getLogger(AbstractCluster.class); + private static final AtomicInteger generation = new AtomicInteger(); private final File root; private final ClassLoader sharedClassLoader; @@ -104,6 +103,7 @@ public abstract class AbstractCluster implements ICluster, protected class Wrapper extends DelegatingInvokableInstance implements IUpgradeableInstance { + private final int generation; private final InstanceConfig config; private volatile IInvokableInstance delegate; private volatile Versions.Version version; @@ -112,21 +112,22 @@ protected class Wrapper extends DelegatingInvokableInstance implements IUpgradea protected IInvokableInstance delegate() { if (delegate == null) - delegate = newInstance(); + delegate = newInstance(generation); return delegate; } - public Wrapper(Versions.Version version, InstanceConfig config) + public Wrapper(int generation, Versions.Version version, InstanceConfig config) { + this.generation = generation; this.config = config; this.version = version; // we ensure there is always a non-null delegate, so that the executor may be used while the node is offline - this.delegate = newInstance(); + this.delegate = newInstance(generation); } - private IInvokableInstance newInstance() + private IInvokableInstance newInstance(int generation) { - ClassLoader classLoader = new InstanceClassLoader(config.num(), version.classpath, sharedClassLoader); + ClassLoader classLoader = new InstanceClassLoader(generation, version.classpath, sharedClassLoader); return Instance.transferAdhoc((SerializableBiFunction)Instance::new, classLoader) .apply(config.forVersion(version.major), classLoader); } @@ -187,19 +188,20 @@ protected AbstractCluster(File root, Versions.Version version, List(); this.instanceMap = new HashMap<>(); + int generation = AbstractCluster.generation.incrementAndGet(); for (InstanceConfig config : configs) { - I instance = newInstanceWrapper(version, config); + I instance = newInstanceWrapper(generation, version, config); instances.add(instance); // we use the config().broadcastAddressAndPort() here because we have not initialised the Instance I prev = instanceMap.put(instance.broadcastAddressAndPort(), instance); if (null != prev) throw new IllegalStateException("Cluster cannot have multiple nodes with same InetAddressAndPort: " + instance.broadcastAddressAndPort() + " vs " + prev.broadcastAddressAndPort()); } - this.filters = new MessageFilters(this); + this.filters = new MessageFilters(); } - protected abstract I newInstanceWrapper(Versions.Version version, InstanceConfig config); + protected abstract I newInstanceWrapper(int generation, Versions.Version version, InstanceConfig config); /** * WARNING: we index from 1 here, for consistency with inet address! @@ -231,7 +233,7 @@ public void parallelForEach(IIsolatedExecutor.SerializableConsumer co public IMessageFilters filters() { return filters; } - public MessageFilters.Builder verbs(MessagingService.Verb ... verbs) { return filters.verbs(verbs); } + public MessageFilters.Builder verbs(Verb... verbs) { return filters.verbs(verbs); } public void disableAutoCompaction(String keyspace) { @@ -335,44 +337,74 @@ protected interface Factory> C newCluster(File root, Versions.Version version, List configs, ClassLoader sharedClassLoader); } - protected static > C - create(int nodeCount, Factory factory) throws Throwable + public static class Builder> { - return create(nodeCount, Files.createTempDirectory("dtests").toFile(), factory); - } - - protected static > C - create(int nodeCount, File root, Factory factory) - { - return create(nodeCount, Versions.CURRENT, root, factory); - } + private final int nodeCount; + private final Factory factory; + private int subnet; + private File root; + private Versions.Version version; + private Consumer configUpdater; + public Builder(int nodeCount, Factory factory) + { + this.nodeCount = nodeCount; + this.factory = factory; + } - protected static > C - create(int nodeCount, Versions.Version version, Factory factory) throws IOException - { - return create(nodeCount, version, Files.createTempDirectory("dtests").toFile(), factory); - } + public Builder withSubnet(int subnet) + { + this.subnet = subnet; + return this; + } - protected static > C - create(int nodeCount, Versions.Version version, File root, Factory factory) - { - root.mkdirs(); - setupLogging(root); + public Builder withRoot(File root) + { + this.root = root; + return this; + } - ClassLoader sharedClassLoader = Thread.currentThread().getContextClassLoader(); + public Builder withVersion(Versions.Version version) + { + this.version = version; + return this; + } - List configs = new ArrayList<>(); - long token = Long.MIN_VALUE + 1, increment = 2 * (Long.MAX_VALUE / nodeCount); - for (int i = 0 ; i < nodeCount ; ++i) + public Builder withConfig(Consumer updater) { - InstanceConfig config = InstanceConfig.generate(i + 1, root, String.valueOf(token)); - configs.add(config); - token += increment; + this.configUpdater = updater; + return this; } - C cluster = factory.newCluster(root, version, configs, sharedClassLoader); - cluster.startup(); - return cluster; + public C start() throws IOException + { + File root = this.root; + Versions.Version version = this.version; + + if (root == null) + root = Files.createTempDirectory("dtests").toFile(); + if (version == null) + version = Versions.CURRENT; + + root.mkdirs(); + setupLogging(root); + + ClassLoader sharedClassLoader = Thread.currentThread().getContextClassLoader(); + + List configs = new ArrayList<>(); + long token = Long.MIN_VALUE + 1, increment = 2 * (Long.MAX_VALUE / nodeCount); + for (int i = 0 ; i < nodeCount ; ++i) + { + InstanceConfig config = InstanceConfig.generate(i + 1, subnet, root, String.valueOf(token)); + if (configUpdater != null) + configUpdater.accept(config); + configs.add(config); + token += increment; + } + + C cluster = factory.newCluster(root, version, configs, sharedClassLoader); + cluster.startup(); + return cluster; + } } private static void setupLogging(File root) diff --git a/src/java/org/apache/cassandra/io/ShortVersionedSerializer.java b/test/distributed/org/apache/cassandra/distributed/impl/ExecUtil.java similarity index 53% rename from src/java/org/apache/cassandra/io/ShortVersionedSerializer.java rename to test/distributed/org/apache/cassandra/distributed/impl/ExecUtil.java index 8731f4c94f7c..b907626ae85a 100644 --- a/src/java/org/apache/cassandra/io/ShortVersionedSerializer.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/ExecUtil.java @@ -16,32 +16,36 @@ * limitations under the License. */ -package org.apache.cassandra.io; +package org.apache.cassandra.distributed.impl; -import java.io.IOException; +import java.io.Serializable; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.distributed.api.IIsolatedExecutor; -public class ShortVersionedSerializer implements IVersionedSerializer +public class ExecUtil { - public static final ShortVersionedSerializer instance = new ShortVersionedSerializer(); - - private ShortVersionedSerializer() {} - - public void serialize(Short aShort, DataOutputPlus out, int version) throws IOException + public interface ThrowingSerializableRunnable extends Serializable { - out.writeShort(aShort); + public void run() throws T; } - public Short deserialize(DataInputPlus in, int version) throws IOException + public static IIsolatedExecutor.SerializableRunnable rethrow(ThrowingSerializableRunnable run) { - return in.readShort(); + return () -> { + try + { + run.run(); + } + catch (RuntimeException | Error t) + { + throw t; + } + catch (Throwable t) + { + throw new RuntimeException(t); + } + }; } - public long serializedSize(Short aShort, int version) - { - return 2; - } } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 53e109afb89d..d1d4dd48cee2 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -27,8 +27,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; +import java.util.function.BiPredicate; import org.slf4j.LoggerFactory; @@ -66,13 +67,8 @@ import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IMessageSink; -import org.apache.cassandra.net.MessageDeliveryTask; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.MessageInHandler; -import org.apache.cassandra.net.async.NettyFactory; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.ActiveRepairService; @@ -81,11 +77,16 @@ import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.transport.messages.ResultMessage; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.Ref; import org.apache.cassandra.utils.memory.BufferPool; +import static java.util.concurrent.TimeUnit.MINUTES; +import static org.apache.cassandra.distributed.impl.InstanceConfig.GOSSIP; +import static org.apache.cassandra.distributed.impl.InstanceConfig.NETWORK; + public class Instance extends IsolatedExecutor implements IInvokableInstance { public final IInstanceConfig config; @@ -173,12 +174,21 @@ public void schemaChangeInternal(String query) private void registerMockMessaging(ICluster cluster) { BiConsumer deliverToInstance = (to, message) -> cluster.get(to).receiveMessage(message); - BiConsumer deliverToInstanceIfNotFiltered = cluster.filters().filter(deliverToInstance); + BiConsumer deliverToInstanceIfNotFiltered = (to, message) -> { + if (cluster.filters().permit(this, cluster.get(to), message.verb())) + deliverToInstance.accept(to, message); + }; + + MessagingService.instance().outboundSink.add(new MessageDeliverySink(deliverToInstanceIfNotFiltered)); + } - MessagingService.instance().addMessageSink(new MessageDeliverySink(deliverToInstanceIfNotFiltered)); + // unnecessary if registerMockMessaging used + private void registerFilter(ICluster cluster) + { + MessagingService.instance().outboundSink.add((message, to) -> cluster.filters().permit(this, cluster.get(to), message.verb().id)); } - private class MessageDeliverySink implements IMessageSink + private class MessageDeliverySink implements BiPredicate, InetAddressAndPort> { private final BiConsumer deliver; MessageDeliverySink(BiConsumer deliver) @@ -187,14 +197,13 @@ private class MessageDeliverySink implements IMessageSink } @Override - public boolean allowOutgoingMessage(MessageOut messageOut, int id, InetAddressAndPort to) + public boolean test(Message messageOut, InetAddressAndPort to) { try (DataOutputBuffer out = new DataOutputBuffer(1024)) { InetAddressAndPort from = broadcastAddressAndPort(); - int version = MessagingService.instance().getVersion(to); - messageOut.serialize(out, version); - deliver.accept(to, new Message(messageOut.verb.getId(), out.toByteArray(), id, version, from)); + Message.serializer.serialize(messageOut, out, MessagingService.current_version); + deliver.accept(to, new MessageImpl(messageOut.verb().id, out.toByteArray(), messageOut.id(), MessagingService.current_version, from)); } catch (IOException e) { @@ -202,13 +211,6 @@ public boolean allowOutgoingMessage(MessageOut messageOut, int id, InetAddressAn } return false; } - - @Override - public boolean allowIncomingMessage(MessageIn message, int id) - { - // we can filter to our heart's content on the outgoing message; no need to worry about incoming - return true; - } } @Override @@ -217,9 +219,8 @@ public void receiveMessage(IMessage message) sync(() -> { try (DataInputBuffer in = new DataInputBuffer(message.bytes())) { - MessageIn messageIn = MessageInHandler.deserialize(in, message.id(), message.version(), message.from()); - Runnable deliver = new MessageDeliveryTask(messageIn, message.id()); - deliver.run(); + Message messageIn = Message.serializer.deserialize(in, message.from(), message.version()); + messageIn.verb().handler().doVerb((Message) messageIn); } catch (Throwable t) { @@ -235,7 +236,7 @@ public int getMessagingVersion() public void setMessagingVersion(InetAddressAndPort endpoint, int version) { - runOnInstance(() -> MessagingService.instance().setVersion(endpoint, version)); + MessagingService.instance().versions.set(endpoint, version); } @Override @@ -244,6 +245,14 @@ public void startup(ICluster cluster) sync(() -> { try { + if (config.has(GOSSIP)) + { + // TODO: hacky + System.setProperty("cassandra.ring_delay_ms", "5000"); + System.setProperty("cassandra.consistent.rangemovement", "false"); + System.setProperty("cassandra.consistent.simultaneousmoves.allow", "true"); + } + mkdirs(); Config.setOverrideLoadConfig(() -> loadConfig(config)); DatabaseDescriptor.daemonInitialization(); @@ -277,13 +286,28 @@ public void startup(ICluster cluster) throw new RuntimeException(e); } - // Even though we don't use MessagingService, access the static NettyFactory - // instance here so that we start the static event loop state - // (e.g. acceptGroup, inboundGroup, outboundGroup, etc ...). We can remove this - // once we actually use the MessagingService to communicate between nodes - NettyFactory.instance.getClass(); - initializeRing(cluster); - registerMockMessaging(cluster); + if (config.has(NETWORK)) + { + registerFilter(cluster); + MessagingService.instance().listen(); + } + else + { + // Even though we don't use MessagingService, access the static SocketFactory + // instance here so that we start the static event loop state +// -- not sure what that means? SocketFactory.instance.getClass(); + registerMockMessaging(cluster); + } + + // TODO: this is more than just gossip + if (config.has(GOSSIP)) + { + StorageService.instance.initServer(); + } + else + { + initializeRing(cluster); + } SystemKeyspace.finishStartup(); @@ -356,8 +380,7 @@ private void initializeRing(ICluster cluster) new VersionedValue.VersionedValueFactory(partitioner).normal(Collections.singleton(token))); Gossiper.instance.realMarkAlive(ep, Gossiper.instance.getEndpointStateForEndpoint(ep)); }); - int version = Math.min(MessagingService.current_version, cluster.get(ep).getMessagingVersion()); - MessagingService.instance().setVersion(ep, version); + MessagingService.instance().versions.set(ep, MessagingService.current_version); } // check that all nodes are in token metadata @@ -380,26 +403,23 @@ public Future shutdown() CompactionManager.instance::forceShutdown, BatchlogManager.instance::shutdown, HintsService.instance::shutdownBlocking, - CommitLog.instance::shutdownBlocking, - SecondaryIndexManager::shutdownExecutors, - ColumnFamilyStore::shutdownFlushExecutor, - ColumnFamilyStore::shutdownPostFlushExecutor, - ColumnFamilyStore::shutdownReclaimExecutor, - ColumnFamilyStore::shutdownPerDiskFlushExecutors, - PendingRangeCalculatorService.instance::shutdownExecutor, - BufferPool::shutdownLocalCleaner, - Ref::shutdownReferenceReaper, - Memtable.MEMORY_POOL::shutdown, - ScheduledExecutors::shutdownAndWait, - SSTableReader::shutdownBlocking, - () -> shutdownAndWait(ActiveRepairService.repairCommandExecutor) + () -> SecondaryIndexManager.shutdownAndWait(1L, MINUTES), + () -> ColumnFamilyStore.shutdownExecutorsAndWait(1L, MINUTES), + () -> PendingRangeCalculatorService.instance.shutdownAndWait(1L, MINUTES), + () -> BufferPool.shutdownLocalCleaner(1L, MINUTES), + () -> Ref.shutdownReferenceReaper(1L, MINUTES), + () -> Memtable.MEMORY_POOL.shutdown(1L, MINUTES), + () -> ScheduledExecutors.shutdownAndWait(1L, MINUTES), + () -> SSTableReader.shutdownBlocking(1L, MINUTES), + () -> shutdownAndWait(Collections.singletonList(ActiveRepairService.repairCommandExecutor)) ); error = parallelRun(error, executor, - MessagingService.instance()::shutdown + CommitLog.instance::shutdownBlocking, + () -> MessagingService.instance().shutdown(1L, MINUTES, false, true) ); error = parallelRun(error, executor, - StageManager::shutdownAndWait, - SharedExecutorPool.SHARED::shutdown + () -> StageManager.shutdownAndWait(1L, MINUTES), + () -> SharedExecutorPool.SHARED.shutdown(1L, MINUTES) ); LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); @@ -411,18 +431,10 @@ public Future shutdown() .thenRun(super::shutdown); } - private static void shutdownAndWait(ExecutorService executor) + private static void shutdownAndWait(List executors) throws TimeoutException, InterruptedException { - try - { - executor.shutdownNow(); - executor.awaitTermination(20, TimeUnit.SECONDS); - } - catch (InterruptedException e) - { - throw new RuntimeException(e); - } - assert executor.isTerminated() && executor.isShutdown() : executor; + ExecutorUtils.shutdownNow(executors); + ExecutorUtils.awaitTermination(1L, MINUTES, executors); } private static Throwable parallelRun(Throwable accumulate, ExecutorService runOn, ThrowingRunnable ... runnables) diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java index 56c80740d9d3..5bf373475032 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java @@ -20,6 +20,8 @@ import com.google.common.base.Predicate; import org.apache.cassandra.config.ParameterizedClass; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.io.util.Memory; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.utils.Pair; @@ -61,16 +63,16 @@ public static interface Factory InstanceClassLoader create(int id, URL[] urls, ClassLoader sharedClassLoader); } - private final int id; private final URL[] urls; + private final int generation; // used to help debug class loader leaks, by helping determine which classloaders should have been collected private final ClassLoader sharedClassLoader; - InstanceClassLoader(int id, URL[] urls, ClassLoader sharedClassLoader) + InstanceClassLoader(int generation, URL[] urls, ClassLoader sharedClassLoader) { super(urls, null); - this.id = id; this.urls = urls; this.sharedClassLoader = sharedClassLoader; + this.generation = generation; } @Override @@ -107,7 +109,7 @@ public static boolean wasLoadedByAnInstanceClassLoader(Class clazz) public String toString() { return "InstanceClassLoader{" + - "id=" + id + + "generation=" + generation + ", urls=" + Arrays.toString(urls) + '}'; } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java index 6361995d75ae..ad5d4d980399 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java @@ -36,6 +36,9 @@ public class InstanceConfig implements IInstanceConfig { + public static long NETWORK = 1; + public static long GOSSIP = 1 << 1; + private static final Object NULL = new Object(); public final int num; @@ -45,6 +48,8 @@ public class InstanceConfig implements IInstanceConfig public UUID hostId() { return hostId; } private final Map params = new TreeMap<>(); + private long featureFlags; + private volatile InetAddressAndPort broadcastAddressAndPort; @Override @@ -97,10 +102,10 @@ private InstanceConfig(int num, .set("concurrent_compactors", 1) .set("memtable_heap_space_in_mb", 10) .set("commitlog_sync", "batch") - .set("storage_port", 7010) + .set("storage_port", 7012) .set("endpoint_snitch", SimpleSnitch.class.getName()) .set("seed_provider", new ParameterizedClass(SimpleSeedProvider.class.getName(), - Collections.singletonMap("seeds", "127.0.0.1:7010"))) + Collections.singletonMap("seeds", "127.0.0.1:7012"))) // required settings for dtest functionality .set("diagnostic_events_enabled", true) // legacy parameters @@ -115,6 +120,17 @@ private InstanceConfig(InstanceConfig copy) this.hostId = copy.hostId; } + public InstanceConfig with(long featureFlag) + { + featureFlags |= featureFlag; + return this; + } + + public boolean has(long featureFlag) + { + return 0 != (featureFlags & featureFlag); + } + public InstanceConfig set(String fieldName, Object value) { if (value == null) @@ -203,13 +219,14 @@ public String getString(String name) return (String)params.get(name); } - public static InstanceConfig generate(int nodeNum, File root, String token) + public static InstanceConfig generate(int nodeNum, int subnet, File root, String token) { + String ipPrefix = "127.0." + subnet + "."; return new InstanceConfig(nodeNum, - "127.0.0." + nodeNum, - "127.0.0." + nodeNum, - "127.0.0." + nodeNum, - "127.0.0." + nodeNum, + ipPrefix + nodeNum, + ipPrefix + nodeNum, + ipPrefix + nodeNum, + ipPrefix + nodeNum, String.format("%s/node%d/saved_caches", root, nodeNum), new String[] { String.format("%s/node%d/data", root, nodeNum) }, String.format("%s/node%d/commitlog", root, nodeNum), diff --git a/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java b/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java index 7f0f0fcf8d53..8ae12430a14a 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java @@ -21,44 +21,27 @@ import java.util.Arrays; import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; -import java.util.function.BiConsumer; import org.apache.cassandra.distributed.api.IInstance; -import org.apache.cassandra.distributed.api.IMessage; import org.apache.cassandra.distributed.api.IMessageFilters; -import org.apache.cassandra.distributed.api.ICluster; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; public class MessageFilters implements IMessageFilters { - private final ICluster cluster; private final Set filters = new CopyOnWriteArraySet<>(); - public MessageFilters(AbstractCluster cluster) + public boolean permit(IInstance from, IInstance to, int verb) { - this.cluster = cluster; - } + if (from == null || to == null) + return false; // cannot deliver + int fromNum = from.config().num(); + int toNum = to.config().num(); - public BiConsumer filter(BiConsumer applyIfNotFiltered) - { - return (toAddress, message) -> - { - IInstance from = cluster.get(message.from()); - IInstance to = cluster.get(toAddress); - if (from == null || to == null) - return; // cannot deliver - int fromNum = from.config().num(); - int toNum = to.config().num(); - int verb = message.verb(); - for (Filter filter : filters) - { - if (filter.matches(fromNum, toNum, verb)) - return; - } + for (Filter filter : filters) + if (filter.matches(fromNum, toNum, verb)) + return false; - applyIfNotFiltered.accept(toAddress, message); - }; + return true; } public class Filter implements IMessageFilters.Filter @@ -162,12 +145,11 @@ public Filter drop() } } - @Override - public Builder verbs(MessagingService.Verb... verbs) + public Builder verbs(Verb... verbs) { int[] ids = new int[verbs.length]; for (int i = 0 ; i < verbs.length ; ++i) - ids[i] = verbs[i].getId(); + ids[i] = verbs[i].id; return new Builder(ids); } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Message.java b/test/distributed/org/apache/cassandra/distributed/impl/MessageImpl.java similarity index 79% rename from test/distributed/org/apache/cassandra/distributed/impl/Message.java rename to test/distributed/org/apache/cassandra/distributed/impl/MessageImpl.java index 6f8085c6e7df..e5c72c5438e7 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Message.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/MessageImpl.java @@ -22,15 +22,15 @@ import org.apache.cassandra.locator.InetAddressAndPort; // a container for simplifying the method signature for per-instance message handling/delivery -public class Message implements IMessage +public class MessageImpl implements IMessage { - private final int verb; - private final byte[] bytes; - private final int id; - private final int version; - private final InetAddressAndPort from; + public final int verb; + public final byte[] bytes; + public final long id; + public final int version; + public final InetAddressAndPort from; - public Message(int verb, byte[] bytes, int id, int version, InetAddressAndPort from) + public MessageImpl(int verb, byte[] bytes, long id, int version, InetAddressAndPort from) { this.verb = verb; this.bytes = bytes; @@ -39,31 +39,26 @@ public Message(int verb, byte[] bytes, int id, int version, InetAddressAndPort f this.from = from; } - @Override public int verb() { return verb; } - @Override public byte[] bytes() { return bytes; } - @Override public int id() { - return id; + return (int) id; } - @Override public int version() { return version; } - @Override public InetAddressAndPort from() { return from; diff --git a/test/distributed/org/apache/cassandra/distributed/test/DistributedReadWritePathTest.java b/test/distributed/org/apache/cassandra/distributed/test/DistributedReadWritePathTest.java index b95f166bbedc..780e65e8ed65 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/DistributedReadWritePathTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/DistributedReadWritePathTest.java @@ -28,7 +28,9 @@ import static org.junit.Assert.assertEquals; -import static org.apache.cassandra.net.MessagingService.Verb.READ_REPAIR; +import static org.apache.cassandra.distributed.impl.InstanceConfig.NETWORK; +import static org.apache.cassandra.net.Verb.READ_REPAIR_REQ; +import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD; public class DistributedReadWritePathTest extends DistributedTestBase { @@ -53,6 +55,26 @@ public void coordinatorReadTest() throws Throwable } } + @Test + public void largeMessageTest() throws Throwable + { + try (Cluster cluster = init(Cluster.create(2))) + { + cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v text, PRIMARY KEY (pk, ck))"); + StringBuilder builder = new StringBuilder(); + for (int i = 0; i < LARGE_MESSAGE_THRESHOLD ; i++) + builder.append('a'); + String s = builder.toString(); + cluster.coordinator(1).execute("INSERT INTO " + KEYSPACE + ".tbl (pk, ck, v) VALUES (1, 1, ?)", + ConsistencyLevel.ALL, + s); + assertRows(cluster.coordinator(1).execute("SELECT * FROM " + KEYSPACE + ".tbl WHERE pk = ?", + ConsistencyLevel.ALL, + 1), + row(1, 1, s)); + } + } + @Test public void coordinatorWriteTest() throws Throwable { @@ -109,7 +131,7 @@ public void failingReadRepairTest() throws Throwable assertRows(cluster.get(3).executeInternal("SELECT * FROM " + KEYSPACE + ".tbl WHERE pk = 1")); - cluster.verbs(READ_REPAIR).to(3).drop(); + cluster.verbs(READ_REPAIR_REQ).to(3).drop(); assertRows(cluster.coordinator(1).execute("SELECT * FROM " + KEYSPACE + ".tbl WHERE pk = 1", ConsistencyLevel.QUORUM), row(1, 1, 1)); @@ -122,7 +144,7 @@ public void failingReadRepairTest() throws Throwable @Test public void writeWithSchemaDisagreement() throws Throwable { - try (Cluster cluster = init(Cluster.create(3))) + try (Cluster cluster = init(Cluster.build(3).withConfig(config -> config.with(NETWORK)).start())) { cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, PRIMARY KEY (pk, ck))"); @@ -144,15 +166,15 @@ public void writeWithSchemaDisagreement() throws Throwable thrown = e; } - Assert.assertTrue(thrown.getMessage().contains("Exception occurred on node")); - Assert.assertTrue(thrown.getCause().getCause().getCause().getMessage().contains("Unknown column v2 during deserialization")); + Assert.assertTrue(thrown.getMessage().contains("INCOMPATIBLE_SCHEMA from 127.0.0.2")); + Assert.assertTrue(thrown.getMessage().contains("INCOMPATIBLE_SCHEMA from 127.0.0.3")); } } @Test public void readWithSchemaDisagreement() throws Throwable { - try (Cluster cluster = init(Cluster.create(3))) + try (Cluster cluster = init(Cluster.create(3, config -> config.with(NETWORK)))) { cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v1 int, PRIMARY KEY (pk, ck))"); @@ -174,8 +196,9 @@ public void readWithSchemaDisagreement() throws Throwable { thrown = e; } - Assert.assertTrue(thrown.getMessage().contains("Exception occurred on node")); - Assert.assertTrue(thrown.getCause().getCause().getCause().getMessage().contains("Unknown column v2 during deserialization")); + + Assert.assertTrue(thrown.getMessage().contains("INCOMPATIBLE_SCHEMA from 127.0.0.2")); + Assert.assertTrue(thrown.getMessage().contains("INCOMPATIBLE_SCHEMA from 127.0.0.3")); } } @@ -243,7 +266,7 @@ public void pagingWithRepairTest() throws Throwable public void pagingTests() throws Throwable { try (Cluster cluster = init(Cluster.create(3)); - Cluster singleNode = init(Cluster.create(1))) + Cluster singleNode = init(Cluster.build(1).withSubnet(1).start())) { cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); singleNode.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); diff --git a/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java index 9e4579dcba47..7acb944e69ed 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java @@ -47,6 +47,11 @@ public static void setup() System.setProperty("org.apache.cassandra.disable_mbean_registration", "true"); } + static String withKeyspace(String replaceIn) + { + return String.format(replaceIn, KEYSPACE); + } + protected static > C init(C cluster) { cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + cluster.size() + "};"); @@ -62,7 +67,7 @@ public static void assertRows(Object[][] actual, Object[]... expected) { Object[] expectedRow = expected[i]; Object[] actualRow = actual[i]; - Assert.assertTrue(rowsNotEqualErrorMessage(actual, expected), + Assert.assertTrue(rowsNotEqualErrorMessage(expected, actual), Arrays.equals(expectedRow, actualRow)); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/GossipSettlesTest.java b/test/distributed/org/apache/cassandra/distributed/test/GossipSettlesTest.java new file mode 100644 index 000000000000..90aa7ac7e855 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/GossipSettlesTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; + +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; + +import static org.apache.cassandra.distributed.impl.InstanceConfig.GOSSIP; +import static org.apache.cassandra.distributed.impl.InstanceConfig.NETWORK; + +public class GossipSettlesTest extends DistributedTestBase +{ + + @Test + public void test() + { + try (Cluster cluster = Cluster.create(3, config -> config.with(GOSSIP).with(NETWORK))) + { + } + catch (IOException e) + { + e.printStackTrace(); + } + } + +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java new file mode 100644 index 000000000000..0c28d38ffde3 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/LargeColumnTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.distributed.Cluster; + +import static java.util.concurrent.TimeUnit.SECONDS; + +public class LargeColumnTest extends DistributedTestBase +{ + private static final Logger logger = LoggerFactory.getLogger(LargeColumnTest.class); + private static String str(int length, Random random, long seed) + { + random.setSeed(seed); + char[] chars = new char[length]; + int i = 0; + int s = 0; + long v = 0; + while (i < length) + { + if (s == 0) + { + v = random.nextLong(); + s = 8; + } + chars[i] = (char) (((v & 127) + 32) & 127); + v >>= 8; + --s; + ++i; + } + return new String(chars); + } + + private void testLargeColumns(int nodes, int columnSize, int rowCount) throws Throwable + { + Random random = new Random(); + long seed = ThreadLocalRandom.current().nextLong(); + logger.info("Using seed {}", seed); + + try (Cluster cluster = init(Cluster.build(nodes) + .withConfig(config -> + config.set("commitlog_segment_size_in_mb", (columnSize * 3) >> 20) + .set("internode_application_send_queue_reserve_endpoint_capacity_in_bytes", columnSize * 2) + .set("internode_application_send_queue_reserve_global_capacity_in_bytes", columnSize * 3) + .set("write_request_timeout_in_ms", SECONDS.toMillis(30L)) + .set("read_request_timeout_in_ms", SECONDS.toMillis(30L)) + .set("memtable_heap_space_in_mb", 1024) + ) + .start())) + { + cluster.schemaChange(String.format("CREATE TABLE %s.cf (k int, c text, PRIMARY KEY (k))", KEYSPACE)); + + for (int i = 0 ; i < rowCount ; ++i) + cluster.coordinator(1).execute(String.format("INSERT INTO %s.cf (k, c) VALUES (?, ?);", KEYSPACE), ConsistencyLevel.ALL, i, str(columnSize, random, seed | i)); + + for (int i = 0 ; i < rowCount ; ++i) + { + Object[][] results = cluster.coordinator(1).execute(String.format("SELECT k, c FROM %s.cf WHERE k = ?;", KEYSPACE), ConsistencyLevel.ALL, i); + Assert.assertTrue(str(columnSize, random, seed | i).equals(results[0][1])); + } + } + } + + @Test + public void test() throws Throwable + { + testLargeColumns(2, 16 << 20, 5); + } + +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java new file mode 100644 index 000000000000..24d7a98dcc06 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java @@ -0,0 +1,134 @@ +///* +// * Licensed to the Apache Software Foundation (ASF) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. The ASF licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// +//package org.apache.cassandra.distributed.test; +// +//import java.io.IOException; +//import java.util.Arrays; +//import java.util.Map; +//import java.util.function.Consumer; +// +//import com.google.common.collect.ImmutableList; +//import com.google.common.collect.ImmutableMap; +//import org.junit.Test; +// +//import org.apache.cassandra.distributed.Cluster; +//import org.apache.cassandra.distributed.impl.InstanceConfig; +//import org.apache.cassandra.service.StorageService; +//import org.apache.cassandra.utils.concurrent.SimpleCondition; +//import org.apache.cassandra.utils.progress.ProgressEventType; +// +//import static java.util.concurrent.TimeUnit.MINUTES; +//import static org.apache.cassandra.distributed.impl.ExecUtil.rethrow; +//import static org.apache.cassandra.distributed.impl.InstanceConfig.GOSSIP; +//import static org.apache.cassandra.distributed.impl.InstanceConfig.NETWORK; +// +//public class RepairTest extends DistributedTestBase +//{ +// private static final String insert = withKeyspace("INSERT INTO %s.test (k, c1, c2) VALUES (?, 'value1', 'value2');"); +// private static final String query = withKeyspace("SELECT k, c1, c2 FROM %s.test WHERE k = ?;"); +// private static void insert(Cluster cluster, int start, int end, int ... nodes) +// { +// for (int i = start ; i < end ; ++i) +// for (int node : nodes) +// cluster.get(node).executeInternal(insert, Integer.toString(i)); +// } +// +// private static void verify(Cluster cluster, int start, int end, int ... nodes) +// { +// for (int i = start ; i < end ; ++i) +// { +// for (int node = 1 ; node <= cluster.size() ; ++node) +// { +// Object[][] rows = cluster.get(node).executeInternal(query, Integer.toString(i)); +// if (Arrays.binarySearch(nodes, node) >= 0) +// assertRows(rows, new Object[] { Integer.toString(i), "value1", "value2" }); +// else +// assertRows(rows); +// } +// } +// } +// +// private static void flush(Cluster cluster, int ... nodes) +// { +// for (int node : nodes) +// cluster.get(node).runOnInstance(rethrow(() -> StorageService.instance.forceKeyspaceFlush(KEYSPACE))); +// } +// +// private Cluster create(Consumer configModifier) throws IOException +// { +// configModifier = configModifier.andThen( +// config -> config.set("hinted_handoff_enabled", false) +// .set("commitlog_sync_batch_window_in_ms", 5) +// .with(NETWORK) +// .with(GOSSIP) +// ); +// +// Cluster cluster = init(Cluster.build(3).withConfig(configModifier).start()); +// try +// { +// cluster.schemaChange(withKeyspace("CREATE TABLE %s.test (k text, c1 text, c2 text, PRIMARY KEY (k));")); +// +// insert(cluster, 0, 1000, 1, 2, 3); +// flush(cluster, 1); +// insert(cluster, 1000, 1001, 1, 2); +// insert(cluster, 1001, 2001, 1, 2, 3); +// flush(cluster, 1, 2, 3); +// +// verify(cluster, 0, 1000, 1, 2, 3); +// verify(cluster, 1000, 1001, 1, 2); +// verify(cluster, 1001, 2001, 1, 2, 3); +// return cluster; +// } +// catch (Throwable t) +// { +// cluster.close(); +// throw t; +// } +// } +// +// private void repair(Cluster cluster, Map options) +// { +// cluster.get(1).runOnInstance(rethrow(() -> { +// SimpleCondition await = new SimpleCondition(); +// StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> { +// if (event.getType() == ProgressEventType.COMPLETE) +// await.signalAll(); +// })).right.get(); +// await.await(1L, MINUTES); +// })); +// } +// +// void simpleRepair(boolean orderPreservingPartitioner, boolean sequential) throws IOException +// { +// Cluster cluster = create(config -> { +// if (orderPreservingPartitioner) +// config.set("partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner"); +// }); +// repair(cluster, ImmutableMap.of("parallelism", sequential ? "sequential" : "parallel")); +// verify(cluster, 0, 2001, 1, 2, 3); +// } +// +// @Test +// public void testSimpleSequentialRepair() throws IOException +// { +// simpleRepair(false, true); +// } +// +// +//} diff --git a/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java b/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java new file mode 100644 index 000000000000..39c89e6ab73f --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/StreamingTest.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.util.Arrays; +import java.util.Comparator; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.service.StorageService; + +import static org.apache.cassandra.distributed.impl.InstanceConfig.NETWORK; + +public class StreamingTest extends DistributedTestBase +{ + + private void testStreaming(int nodes, int replicationFactor, int rowCount, String compactionStrategy) throws Throwable + { + try (Cluster cluster = Cluster.create(nodes, config -> config.with(NETWORK))) + { + cluster.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + replicationFactor + "};"); + cluster.schemaChange(String.format("CREATE TABLE %s.cf (k text, c1 text, c2 text, PRIMARY KEY (k)) WITH compaction = {'class': '%s', 'enabled': 'true'}", KEYSPACE, compactionStrategy)); + + for (int i = 0 ; i < rowCount ; ++i) + { + for (int n = 1 ; n < nodes ; ++n) + cluster.get(n).executeInternal(String.format("INSERT INTO %s.cf (k, c1, c2) VALUES (?, 'value1', 'value2');", KEYSPACE), Integer.toString(i)); + } + + cluster.get(nodes).executeInternal("TRUNCATE system.available_ranges;"); + { + Object[][] results = cluster.get(nodes).executeInternal(String.format("SELECT k, c1, c2 FROM %s.cf;", KEYSPACE)); + Assert.assertEquals(0, results.length); + } + + cluster.get(nodes).runOnInstance(() -> StorageService.instance.rebuild(null, KEYSPACE, null, null)); + { + Object[][] results = cluster.get(nodes).executeInternal(String.format("SELECT k, c1, c2 FROM %s.cf;", KEYSPACE)); + Assert.assertEquals(1000, results.length); + Arrays.sort(results, Comparator.comparingInt(a -> Integer.parseInt((String) a[0]))); + for (int i = 0 ; i < results.length ; ++i) + { + Assert.assertEquals(Integer.toString(i), results[i][0]); + Assert.assertEquals("value1", results[i][1]); + Assert.assertEquals("value2", results[i][2]); + } + } + } + } + + @Test + public void test() throws Throwable + { + testStreaming(2, 2, 1000, "LeveledCompactionStrategy"); + } + +} diff --git a/test/distributed/org/apache/cassandra/distributed/util/PyDtest.java b/test/distributed/org/apache/cassandra/distributed/util/PyDtest.java new file mode 100644 index 000000000000..3b2425f74d8c --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/util/PyDtest.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.util; + +import java.util.Arrays; +import java.util.List; + +public class PyDtest +{ + + public static class CreateCf + { + final String keyspace; + final String name; + String primaryKey, clustering, keyType, speculativeRetry, compression, validation, compactionStrategy; + Float readRepair; + Integer gcGrace; + List columns; + Boolean compactStorage; + + public CreateCf(String keyspace, String name) + { + this.keyspace = keyspace; + this.name = name; + } + + public CreateCf withPrimaryKey(String primaryKey) + { + this.primaryKey = primaryKey; + return this; + } + + public CreateCf withClustering(String clustering) + { + this.clustering = clustering; + return this; + } + + public CreateCf withKeyType(String keyType) + { + this.keyType = keyType; + return this; + } + + public CreateCf withSpeculativeRetry(String speculativeRetry) + { + this.speculativeRetry = speculativeRetry; + return this; + } + + public CreateCf withCompression(String compression) + { + this.compression = compression; + return this; + } + + public CreateCf withValidation(String validation) + { + this.validation = validation; + return this; + } + + public CreateCf withCompactionStrategy(String compactionStrategy) + { + this.compactionStrategy = compactionStrategy; + return this; + } + + public CreateCf withReadRepair(Float readRepair) + { + this.readRepair = readRepair; + return this; + } + + public CreateCf withGcGrace(Integer gcGrace) + { + this.gcGrace = gcGrace; + return this; + } + + public CreateCf withColumns(List columns) + { + this.columns = columns; + return this; + } + + public CreateCf withColumns(String ... columns) + { + this.columns = Arrays.asList(columns); + return this; + } + + public CreateCf withCompactStorage(Boolean compactStorage) + { + this.compactStorage = compactStorage; + return this; + } + + public String build() + { + if (keyspace == null) + throw new IllegalArgumentException(); + if (name == null) + throw new IllegalArgumentException(); + if (keyType == null) + keyType = "varchar"; + if (validation == null) + validation = "UTF8Type"; + if (compactionStrategy == null) + compactionStrategy = "SizeTieredCompactionStrategy"; + if (compactStorage == null) + compactStorage = false; + + + String compaction_fragment = String.format("compaction = {'class': '%s', 'enabled': 'true'}", compactionStrategy); + + String query; + String additional_columns = ""; + if (columns == null) + { + query = String.format("CREATE COLUMNFAMILY %s.%s (key %s, c varchar, v varchar, PRIMARY KEY(key, c)) WITH comment=\'test cf\'", keyspace, name, keyType); + } + else + { + for (String pair : columns) + { + String[] split = pair.split(":"); + String key = split[0]; + String type = split[1]; + additional_columns += ", " + key + " " + type; + } + + if (primaryKey != null) + query = String.format("CREATE COLUMNFAMILY %s.%s (key %s%s, PRIMARY KEY(%s)) WITH comment=\'test cf\'", keyspace, name, keyType, additional_columns, primaryKey); + else + query = String.format("CREATE COLUMNFAMILY %s.%s (key %s PRIMARY KEY%s) WITH comment=\'test cf\'", keyspace, name, keyType, additional_columns); + } + + + if (compaction_fragment != null) + query += " AND " + compaction_fragment; + + if (clustering != null) + query += String.format(" AND CLUSTERING ORDER BY (%s)", clustering); + + if (compression != null) + query += String.format(" AND compression = { \'sstable_compression\': \'%sCompressor\' }", compression); + else + query += " AND compression = {}"; + + if (readRepair != null) + query += String.format(" AND read_repair_chance=%f AND dclocal_read_repair_chance=%f", readRepair, readRepair); + if (gcGrace != null) + query += String.format(" AND gc_grace_seconds=%d", gcGrace); + if (speculativeRetry != null) + query += String.format(" AND speculative_retry=\'%s\'", speculativeRetry); + + if (compactStorage != null && compactStorage) + query += " AND COMPACT STORAGE"; + + return query; + } + } + + public static CreateCf createCf(String keyspace, String name) + { + return new CreateCf(keyspace, name); + } + +} diff --git a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java index 94a3bd3569f1..2e27738efc18 100644 --- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java +++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java @@ -30,6 +30,8 @@ import org.apache.cassandra.utils.FBUtilities; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + public class DynamicEndpointSnitchLongTest { static @@ -101,7 +103,7 @@ public void run() { Replica host = hosts.get(random.nextInt(hosts.size())); int score = random.nextInt(SCORE_RANGE); - dsnitch.receiveTiming(host.endpoint(), score); + dsnitch.receiveTiming(host.endpoint(), score, MILLISECONDS); } } } diff --git a/test/microbench/org/apache/cassandra/test/microbench/MessageOutBench.java b/test/microbench/org/apache/cassandra/test/microbench/MessageOutBench.java index 2aec66883a8b..4ab607f49c30 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/MessageOutBench.java +++ b/test/microbench/org/apache/cassandra/test/microbench/MessageOutBench.java @@ -19,31 +19,23 @@ package org.apache.cassandra.test.microbench; import java.io.IOException; -import java.util.Collections; import java.util.EnumMap; import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; -import java.util.function.BiConsumer; -import com.google.common.collect.ImmutableList; import com.google.common.net.InetAddresses; -import com.google.common.primitives.Shorts; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; -import org.apache.cassandra.net.async.BaseMessageInHandler; -import org.apache.cassandra.net.async.ByteBufDataOutputPlus; -import org.apache.cassandra.net.async.MessageInHandler; -import org.apache.cassandra.net.async.MessageInHandlerPre40; -import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis; +import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.net.ParamType; import org.apache.cassandra.utils.UUIDGen; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -57,7 +49,7 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE; +import static org.apache.cassandra.net.Verb.ECHO_REQ; @State(Scope.Thread) @Warmup(iterations = 4, time = 1, timeUnit = TimeUnit.SECONDS) @@ -70,60 +62,51 @@ public class MessageOutBench @Param({ "true", "false" }) private boolean withParams; - private MessageOut msgOut; + private Message msgOut; private ByteBuf buf; - BaseMessageInHandler handler40; - BaseMessageInHandler handlerPre40; + private InetAddressAndPort addr; @Setup public void setup() { DatabaseDescriptor.daemonInitialization(); - InetAddressAndPort addr = InetAddressAndPort.getByAddress(InetAddresses.forString("127.0.73.101")); UUID uuid = UUIDGen.getTimeUUID(); - Map parameters = new EnumMap<>(ParameterType.class); + Map parameters = new EnumMap<>(ParamType.class); if (withParams) { - parameters.put(ParameterType.FAILURE_RESPONSE, MessagingService.ONE_BYTE); - parameters.put(ParameterType.FAILURE_REASON, Shorts.checkedCast(RequestFailureReason.READ_TOO_MANY_TOMBSTONES.code)); - parameters.put(ParameterType.TRACE_SESSION, uuid); + parameters.put(ParamType.TRACE_SESSION, uuid); } - msgOut = new MessageOut<>(addr, MessagingService.Verb.ECHO, null, null, ImmutableList.of(), SMALL_MESSAGE); + addr = InetAddressAndPort.getByAddress(InetAddresses.forString("127.0.73.101")); + msgOut = Message.builder(ECHO_REQ, NoPayload.noPayload) + .from(addr) + .build(); buf = Unpooled.buffer(1024, 1024); // 1k should be enough for everybody! - - handler40 = new MessageInHandler(addr, MessagingService.VERSION_40, messageConsumer); - handlerPre40 = new MessageInHandlerPre40(addr, MessagingService.VERSION_30, messageConsumer); } @Benchmark public int serialize40() throws Exception { - return serialize(MessagingService.VERSION_40, handler40); + return serialize(MessagingService.VERSION_40); } - private int serialize(int messagingVersion, BaseMessageInHandler handler) throws Exception + private int serialize(int messagingVersion) throws IOException { - buf.resetReaderIndex(); - buf.resetWriterIndex(); - buf.writeInt(MessagingService.PROTOCOL_MAGIC); - buf.writeInt(42); // this is the id - buf.writeInt((int) NanoTimeToCurrentTimeMillis.convert(System.nanoTime())); - - msgOut.serialize(new ByteBufDataOutputPlus(buf), messagingVersion); - handler.decode(null, buf, Collections.emptyList()); - return msgOut.serializedSize(messagingVersion); + try (DataOutputBuffer out = new DataOutputBuffer()) + { + Message.serializer.serialize(Message.builder(msgOut).withCreatedAt(System.nanoTime()).withId(42).build(), + out, messagingVersion); + DataInputBuffer in = new DataInputBuffer(out.buffer(), false); + Message.serializer.deserialize(in, addr, messagingVersion); + return msgOut.serializedSize(messagingVersion); + } } @Benchmark public int serializePre40() throws Exception { - return serialize(MessagingService.VERSION_30, handlerPre40); + return serialize(MessagingService.VERSION_30); } - - private final BiConsumer messageConsumer = (messageIn, integer) -> - { - }; } diff --git a/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java index 4a0e64643c80..074e183f2a9b 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java +++ b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java @@ -34,8 +34,6 @@ import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.DataOutputBufferFixed; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; @@ -73,7 +71,6 @@ public class MutationBench static String keyspace = "keyspace1"; private Mutation mutation; - private MessageOut messageOut; private ByteBuffer buffer; private DataOutputBuffer outputBuffer; @@ -83,7 +80,7 @@ public class MutationBench @State(Scope.Thread) public static class ThreadState { - MessageIn in; + Mutation in; int counter = 0; } @@ -103,19 +100,18 @@ public void setup() throws IOException Schema.instance.load(ksm.withSwapped(ksm.tables.with(metadata))); mutation = (Mutation)UpdateBuilder.create(metadata, 1L).newRow(1L).add("commentid", 32L).makeMutation(); - messageOut = mutation.createMessage(); - buffer = ByteBuffer.allocate(messageOut.serializedSize(MessagingService.current_version)); + buffer = ByteBuffer.allocate((int) Mutation.serializer.serializedSize(mutation, MessagingService.current_version)); outputBuffer = new DataOutputBufferFixed(buffer); inputBuffer = new DataInputBuffer(buffer, false); - messageOut.serialize(outputBuffer, MessagingService.current_version); + Mutation.serializer.serialize(mutation, outputBuffer, MessagingService.current_version); } @Benchmark public void serialize(ThreadState state) throws IOException { buffer.rewind(); - messageOut.serialize(outputBuffer, MessagingService.current_version); + Mutation.serializer.serialize(mutation, outputBuffer, MessagingService.current_version); state.counter++; } @@ -123,7 +119,7 @@ public void serialize(ThreadState state) throws IOException public void deserialize(ThreadState state) throws IOException { buffer.rewind(); - state.in = MessageIn.read(inputBuffer, MessagingService.current_version, 0); + state.in = Mutation.serializer.deserialize(inputBuffer, MessagingService.current_version); state.counter++; } diff --git a/test/microbench/org/apache/cassandra/test/microbench/PreaggregatedByteBufsBench.java b/test/microbench/org/apache/cassandra/test/microbench/PreaggregatedByteBufsBench.java new file mode 100644 index 000000000000..9971cc5b4a48 --- /dev/null +++ b/test/microbench/org/apache/cassandra/test/microbench/PreaggregatedByteBufsBench.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.test.microbench; + +import java.util.concurrent.TimeUnit; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +@State(Scope.Thread) +@Warmup(iterations = 4, time = 1, timeUnit = TimeUnit.SECONDS) +@Measurement(iterations = 8, time = 4, timeUnit = TimeUnit.SECONDS) +@Fork(value = 1,jvmArgsAppend = "-Xmx512M") +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@BenchmarkMode(Mode.SampleTime) +public class PreaggregatedByteBufsBench +{ + @Param({ "1000", "2500", "5000", "10000", "20000", "40000"}) + private int len; + + private static final int subBufferCount = 64; + + private EmbeddedChannel channel; + + @Setup + public void setUp() + { + channel = new EmbeddedChannel(); + } + + @Benchmark + public boolean oneBigBuf() + { + boolean success = true; + try + { + ByteBuf buf = channel.alloc().directBuffer(len); + buf.writerIndex(len); + channel.writeAndFlush(buf); + } + catch (Exception e) + { + success = false; + } + finally + { + channel.releaseOutbound(); + } + + return success; + } + + @Benchmark + public boolean chunkedBuf() + { + boolean success = true; + try + { + int chunkLen = len / subBufferCount; + + for (int i = 0; i < subBufferCount; i++) + { + ByteBuf buf = channel.alloc().directBuffer(chunkLen); + buf.writerIndex(chunkLen); + channel.write(buf); + } + channel.flush(); + } + catch (Exception e) + { + success = false; + } + finally + { + channel.releaseOutbound(); + } + + return success; + } +} diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index 3192bccddefd..2b642a83dd81 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -51,8 +51,8 @@ import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; -import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.streaming.DefaultConnectionFactory; @@ -122,7 +122,7 @@ public void setupBenchmark() throws IOException blockStreamWriter = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); CapturingNettyChannel blockStreamCaptureChannel = new CapturingNettyChannel(STREAM_SIZE); - ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, blockStreamCaptureChannel, 1024 * 1024); + AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(blockStreamCaptureChannel); blockStreamWriter.write(out); serializedBlockStream = blockStreamCaptureChannel.getSerializedStream(); out.close(); @@ -152,7 +152,7 @@ public void setupBenchmark() throws IOException partialStreamWriter = new CassandraStreamWriter(sstable, sstable.getPositionsForRanges(requestedRanges), session); CapturingNettyChannel partialStreamChannel = new CapturingNettyChannel(STREAM_SIZE); - partialStreamWriter.write(ByteBufDataOutputStreamPlus.create(session, partialStreamChannel, 1024 * 1024)); + partialStreamWriter.write(new AsyncStreamingOutputPlus(partialStreamChannel)); serializedPartialStream = partialStreamChannel.getSerializedStream(); CassandraStreamHeader partialSSTableStreamHeader = @@ -230,7 +230,7 @@ private StreamSession setupStreamingSessionForTest() public void blockStreamWriter(BenchmarkState state) throws Exception { EmbeddedChannel channel = createMockNettyChannel(); - ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(state.session, channel, 1024 * 1024); + AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel); state.blockStreamWriter.write(out); out.close(); channel.finishAndReleaseAll(); @@ -241,7 +241,7 @@ public void blockStreamWriter(BenchmarkState state) throws Exception public void blockStreamReader(BenchmarkState state) throws Exception { EmbeddedChannel channel = createMockNettyChannel(); - RebufferingByteBufDataInputPlus in = new RebufferingByteBufDataInputPlus(STREAM_SIZE, STREAM_SIZE, channel.config()); + AsyncStreamingInputPlus in = new AsyncStreamingInputPlus(channel); in.append(state.serializedBlockStream.retainedDuplicate()); SSTableMultiWriter sstableWriter = state.blockStreamReader.read(in); Collection newSstables = sstableWriter.finished(); @@ -254,7 +254,7 @@ public void blockStreamReader(BenchmarkState state) throws Exception public void partialStreamWriter(BenchmarkState state) throws Exception { EmbeddedChannel channel = createMockNettyChannel(); - ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(state.session, channel, 1024 * 1024); + AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel); state.partialStreamWriter.write(out); out.close(); channel.finishAndReleaseAll(); @@ -265,7 +265,7 @@ public void partialStreamWriter(BenchmarkState state) throws Exception public void partialStreamReader(BenchmarkState state) throws Exception { EmbeddedChannel channel = createMockNettyChannel(); - RebufferingByteBufDataInputPlus in = new RebufferingByteBufDataInputPlus(STREAM_SIZE, STREAM_SIZE, channel.config()); + AsyncStreamingInputPlus in = new AsyncStreamingInputPlus(channel); in.append(state.serializedPartialStream.retainedDuplicate()); SSTableMultiWriter sstableWriter = state.partialStreamReader.read(in); Collection newSstables = sstableWriter.finished(); diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java index 33fb209389d9..361759f11b9a 100644 --- a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java +++ b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java @@ -54,6 +54,7 @@ import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.UUIDGen; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; import static org.junit.Assert.*; @@ -277,7 +278,7 @@ public void testAddBatch() throws IOException long initialAllBatches = BatchlogManager.instance.countAllBatches(); TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD5).metadata(); - long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000; + long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS) * 2) * 1000; UUID uuid = UUIDGen.getTimeUUID(); // Add a batch with 10 mutations @@ -309,7 +310,7 @@ public void testRemoveBatch() long initialAllBatches = BatchlogManager.instance.countAllBatches(); TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD5).metadata(); - long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000; + long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS) * 2) * 1000; UUID uuid = UUIDGen.getTimeUUID(); // Add a batch with 10 mutations @@ -351,7 +352,7 @@ public void testReplayWithNoPeers() throws Exception TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata(); - long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000; + long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout(MILLISECONDS) * 2) * 1000; UUID uuid = UUIDGen.getTimeUUID(); // Add a batch with 10 mutations diff --git a/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java b/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java index 011a8bac1746..ff9f3af97403 100644 --- a/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java +++ b/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java @@ -28,6 +28,8 @@ import org.apache.cassandra.utils.FBUtilities; +import static java.util.concurrent.TimeUnit.MINUTES; + public class SEPExecutorTest { @Test @@ -56,7 +58,7 @@ public void write(int b) { } } // shutdown does not guarantee that threads are actually dead once it exits, only that they will stop promptly afterwards - sharedPool.shutdown(); + sharedPool.shutdown(1L, MINUTES); for (Thread thread : Thread.getAllStackTraces().keySet()) { if (thread.getName().contains(MAGIC)) diff --git a/test/unit/org/apache/cassandra/db/ReadCommandTest.java b/test/unit/org/apache/cassandra/db/ReadCommandTest.java index 3bb30d961cb4..8b735026f8c7 100644 --- a/test/unit/org/apache/cassandra/db/ReadCommandTest.java +++ b/test/unit/org/apache/cassandra/db/ReadCommandTest.java @@ -55,8 +55,9 @@ import org.apache.cassandra.locator.EndpointsForToken; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.ReplicaUtils; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.consistent.LocalSessionAccessor; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; @@ -396,9 +397,9 @@ public void testSerializer() throws IOException int messagingVersion = MessagingService.current_version; FakeOutputStream out = new FakeOutputStream(); Tracing.instance.newSession(Tracing.TraceType.QUERY); - MessageOut messageOut = new MessageOut(MessagingService.Verb.READ, readCommand, ReadCommand.serializer); + Message messageOut = Message.out(Verb.READ_REQ, readCommand); long size = messageOut.serializedSize(messagingVersion); - messageOut.serialize(new WrappedDataOutputStreamPlus(out), messagingVersion); + Message.serializer.serialize(messageOut, new WrappedDataOutputStreamPlus(out), messagingVersion); Assert.assertEquals(size, out.count); } diff --git a/test/unit/org/apache/cassandra/db/ReadCommandVerbHandlerTest.java b/test/unit/org/apache/cassandra/db/ReadCommandVerbHandlerTest.java index b7e053b1db22..868227390427 100644 --- a/test/unit/org/apache/cassandra/db/ReadCommandVerbHandlerTest.java +++ b/test/unit/org/apache/cassandra/db/ReadCommandVerbHandlerTest.java @@ -19,17 +19,14 @@ package org.apache.cassandra.db; import java.net.UnknownHostException; -import java.util.Map; import java.util.Random; import java.util.UUID; -import com.google.common.collect.ImmutableMap; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.filter.DataLimits; @@ -37,18 +34,17 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.TokenMetadata; -import org.apache.cassandra.net.IMessageSink; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessageFlag; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; +import org.apache.cassandra.net.ParamType; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.Util.token; +import static org.apache.cassandra.net.Verb.*; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -84,19 +80,10 @@ public static void init() throws Throwable @Before public void setup() { - MessagingService.instance().clearMessageSinks(); - MessagingService.instance().addMessageSink(new IMessageSink() - { - public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to) - { - return false; - } - - public boolean allowIncomingMessage(MessageIn message, int id) - { - return false; - } - }); + MessagingService.instance().inboundSink.clear(); + MessagingService.instance().outboundSink.clear(); + MessagingService.instance().outboundSink.add((message, to) -> false); + MessagingService.instance().inboundSink.add((message) -> false); handler = new ReadCommandVerbHandler(); } @@ -104,59 +91,50 @@ public boolean allowIncomingMessage(MessageIn message, int id) @Test public void setRepairedDataTrackingFlagIfHeaderPresent() { - SinglePartitionReadCommand command = command(metadata); + ReadCommand command = command(metadata); assertFalse(command.isTrackingRepairedStatus()); - Map params = ImmutableMap.of(ParameterType.TRACK_REPAIRED_DATA, - MessagingService.ONE_BYTE); - handler.doVerb(MessageIn.create(peer(), - command, - params, - MessagingService.Verb.READ, - MessagingService.current_version), - messageId()); + + handler.doVerb(Message.builder(READ_REQ, command) + .from(peer()) + .withFlag(MessageFlag.TRACK_REPAIRED_DATA) + .withId(messageId()) + .build()); assertTrue(command.isTrackingRepairedStatus()); } @Test public void dontSetRepairedDataTrackingFlagUnlessHeaderPresent() { - SinglePartitionReadCommand command = command(metadata); + ReadCommand command = command(metadata); assertFalse(command.isTrackingRepairedStatus()); - Map params = ImmutableMap.of(ParameterType.TRACE_SESSION, - UUID.randomUUID()); - handler.doVerb(MessageIn.create(peer(), - command, - params, - MessagingService.Verb.READ, - MessagingService.current_version), - messageId()); + handler.doVerb(Message.builder(READ_REQ, command) + .from(peer()) + .withId(messageId()) + .withParam(ParamType.TRACE_SESSION, UUID.randomUUID()) + .build()); assertFalse(command.isTrackingRepairedStatus()); } @Test public void dontSetRepairedDataTrackingFlagIfHeadersEmpty() { - SinglePartitionReadCommand command = command(metadata); + ReadCommand command = command(metadata); assertFalse(command.isTrackingRepairedStatus()); - handler.doVerb(MessageIn.create(peer(), - command, - ImmutableMap.of(), - MessagingService.Verb.READ, - MessagingService.current_version), - messageId()); + handler.doVerb(Message.builder(READ_REQ, command) + .withId(messageId()) + .from(peer()) + .build()); assertFalse(command.isTrackingRepairedStatus()); } @Test (expected = InvalidRequestException.class) public void rejectsRequestWithNonMatchingTransientness() { - SinglePartitionReadCommand command = command(metadata_with_transient); - handler.doVerb(MessageIn.create(peer(), - command, - ImmutableMap.of(), - MessagingService.Verb.READ, - MessagingService.current_version), - messageId()); + ReadCommand command = command(metadata_with_transient); + handler.doVerb(Message.builder(READ_REQ, command) + .from(peer()) + .withId(messageId()) + .build()); } private static int messageId() diff --git a/test/unit/org/apache/cassandra/db/commitlog/AbstractCommitLogServiceTest.java b/test/unit/org/apache/cassandra/db/commitlog/AbstractCommitLogServiceTest.java index bc5cb29e25df..741b1454b5c9 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/AbstractCommitLogServiceTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/AbstractCommitLogServiceTest.java @@ -28,7 +28,6 @@ import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.commitlog.AbstractCommitLogService.SyncRunnable; -import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.FreeRunningClock; import static org.apache.cassandra.db.commitlog.AbstractCommitLogService.DEFAULT_MARKER_INTERVAL_MILLIS; diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java index 431fb323a3ce..de7ddfcb1f38 100644 --- a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java @@ -23,7 +23,6 @@ import java.util.Set; import java.util.UUID; -import com.google.common.collect.Iterables; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Ignore; @@ -33,10 +32,6 @@ import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IMessageSink; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.AbstractRepairTest; import org.apache.cassandra.repair.consistent.LocalSessionAccessor; @@ -65,18 +60,8 @@ public static void setupClass() LocalSessionAccessor.startup(); // cutoff messaging service - MessagingService.instance().addMessageSink(new IMessageSink() - { - public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to) - { - return false; - } - - public boolean allowIncomingMessage(MessageIn message, int id) - { - return false; - } - }); + MessagingService.instance().outboundSink.add((message, to) -> false); + MessagingService.instance().inboundSink.add((message) -> false); } @Before diff --git a/test/unit/org/apache/cassandra/db/monitoring/MonitoringTaskTest.java b/test/unit/org/apache/cassandra/db/monitoring/MonitoringTaskTest.java index acc988f7228f..454d0b4915c7 100644 --- a/test/unit/org/apache/cassandra/db/monitoring/MonitoringTaskTest.java +++ b/test/unit/org/apache/cassandra/db/monitoring/MonitoringTaskTest.java @@ -32,6 +32,11 @@ import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.utils.ApproximateTime; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -39,8 +44,8 @@ public class MonitoringTaskTest { - private static final long timeout = 100; - private static final long slowTimeout = 10; + private static final long timeout = MILLISECONDS.toNanos(100); + private static final long slowTimeout = MILLISECONDS.toNanos(10); private static final long MAX_SPIN_TIME_NANOS = TimeUnit.SECONDS.toNanos(5); @@ -90,8 +95,8 @@ private static void waitForOperationsToComplete(Monitorable... operations) throw private static void waitForOperationsToComplete(List operations) throws InterruptedException { - long timeout = operations.stream().map(Monitorable::timeout).reduce(0L, Long::max); - Thread.sleep(timeout * 2 + ApproximateTime.precision()); + long timeout = operations.stream().map(Monitorable::timeoutNanos).reduce(0L, Long::max); + Thread.sleep(NANOSECONDS.toMillis(timeout * 2 + approxTime.error())); long start = System.nanoTime(); while(System.nanoTime() - start <= MAX_SPIN_TIME_NANOS) @@ -109,8 +114,8 @@ private static void waitForOperationsToBeReportedAsSlow(Monitorable... operation private static void waitForOperationsToBeReportedAsSlow(List operations) throws InterruptedException { - long timeout = operations.stream().map(Monitorable::slowTimeout).reduce(0L, Long::max); - Thread.sleep(timeout * 2 + ApproximateTime.precision()); + long timeout = operations.stream().map(Monitorable::slowTimeoutNanos).reduce(0L, Long::max); + Thread.sleep(NANOSECONDS.toMillis(timeout * 2 + approxTime.error())); long start = System.nanoTime(); while(System.nanoTime() - start <= MAX_SPIN_TIME_NANOS) @@ -124,7 +129,7 @@ private static void waitForOperationsToBeReportedAsSlow(List operat @Test public void testAbort() throws InterruptedException { - Monitorable operation = new TestMonitor("Test abort", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation = new TestMonitor("Test abort", System.nanoTime(), false, timeout, slowTimeout); waitForOperationsToComplete(operation); assertTrue(operation.isAborted()); @@ -135,7 +140,7 @@ public void testAbort() throws InterruptedException @Test public void testAbortIdemPotent() throws InterruptedException { - Monitorable operation = new TestMonitor("Test abort", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation = new TestMonitor("Test abort", System.nanoTime(), false, timeout, slowTimeout); waitForOperationsToComplete(operation); assertTrue(operation.abort()); @@ -148,7 +153,7 @@ public void testAbortIdemPotent() throws InterruptedException @Test public void testAbortCrossNode() throws InterruptedException { - Monitorable operation = new TestMonitor("Test for cross node", System.currentTimeMillis(), true, timeout, slowTimeout); + Monitorable operation = new TestMonitor("Test for cross node", System.nanoTime(), true, timeout, slowTimeout); waitForOperationsToComplete(operation); assertTrue(operation.isAborted()); @@ -159,7 +164,7 @@ public void testAbortCrossNode() throws InterruptedException @Test public void testComplete() throws InterruptedException { - Monitorable operation = new TestMonitor("Test complete", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation = new TestMonitor("Test complete", System.nanoTime(), false, timeout, slowTimeout); operation.complete(); waitForOperationsToComplete(operation); @@ -171,7 +176,7 @@ public void testComplete() throws InterruptedException @Test public void testCompleteIdemPotent() throws InterruptedException { - Monitorable operation = new TestMonitor("Test complete", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation = new TestMonitor("Test complete", System.nanoTime(), false, timeout, slowTimeout); operation.complete(); waitForOperationsToComplete(operation); @@ -185,7 +190,7 @@ public void testCompleteIdemPotent() throws InterruptedException @Test public void testReportSlow() throws InterruptedException { - Monitorable operation = new TestMonitor("Test report slow", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation = new TestMonitor("Test report slow", System.nanoTime(), false, timeout, slowTimeout); waitForOperationsToBeReportedAsSlow(operation); assertTrue(operation.isSlow()); @@ -199,7 +204,7 @@ public void testReportSlow() throws InterruptedException public void testNoReportSlowIfZeroSlowTimeout() throws InterruptedException { // when the slow timeout is set to zero then operation won't be reported as slow - Monitorable operation = new TestMonitor("Test report slow disabled", System.currentTimeMillis(), false, timeout, 0); + Monitorable operation = new TestMonitor("Test report slow disabled", System.nanoTime(), false, timeout, 0); waitForOperationsToBeReportedAsSlow(operation); assertTrue(operation.isSlow()); @@ -212,7 +217,7 @@ public void testNoReportSlowIfZeroSlowTimeout() throws InterruptedException @Test public void testReport() throws InterruptedException { - Monitorable operation = new TestMonitor("Test report", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation = new TestMonitor("Test report", System.nanoTime(), false, timeout, slowTimeout); waitForOperationsToComplete(operation); assertTrue(operation.isSlow()); @@ -220,10 +225,10 @@ public void testReport() throws InterruptedException assertFalse(operation.isCompleted()); // aborted operations are not logged as slow - assertFalse(MonitoringTask.instance.logSlowOperations(ApproximateTime.currentTimeMillis())); + assertFalse(MonitoringTask.instance.logSlowOperations(approxTime.now())); assertEquals(0, MonitoringTask.instance.getSlowOperations().size()); - assertTrue(MonitoringTask.instance.logFailedOperations(ApproximateTime.currentTimeMillis())); + assertTrue(MonitoringTask.instance.logFailedOperations(approxTime.now())); assertEquals(0, MonitoringTask.instance.getFailedOperations().size()); } @@ -233,20 +238,20 @@ public void testRealScheduling() throws InterruptedException MonitoringTask.instance = MonitoringTask.make(10, -1); try { - Monitorable operation1 = new TestMonitor("Test report 1", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation1 = new TestMonitor("Test report 1", System.nanoTime(), false, timeout, slowTimeout); waitForOperationsToComplete(operation1); assertTrue(operation1.isAborted()); assertFalse(operation1.isCompleted()); - Monitorable operation2 = new TestMonitor("Test report 2", System.currentTimeMillis(), false, timeout, slowTimeout); + Monitorable operation2 = new TestMonitor("Test report 2", System.nanoTime(), false, timeout, slowTimeout); waitForOperationsToBeReportedAsSlow(operation2); operation2.complete(); assertFalse(operation2.isAborted()); assertTrue(operation2.isCompleted()); - Thread.sleep(ApproximateTime.precision() + 500); + Thread.sleep(2 * NANOSECONDS.toMillis(approxTime.error()) + 500); assertEquals(0, MonitoringTask.instance.getFailedOperations().size()); assertEquals(0, MonitoringTask.instance.getSlowOperations().size()); } @@ -266,7 +271,7 @@ public void testMultipleThreads() throws InterruptedException for (int i = 0; i < opCount; i++) { executorService.submit(() -> - operations.add(new TestMonitor(UUID.randomUUID().toString(), System.currentTimeMillis(), false, timeout, slowTimeout)) + operations.add(new TestMonitor(UUID.randomUUID().toString(), System.nanoTime(), false, timeout, slowTimeout)) ); } @@ -311,14 +316,14 @@ private static void doTestMaxTimedoutOperations(int maxTimedoutOperations, for (int j = 0; j < numTimes; j++) { Monitorable operation1 = new TestMonitor(operationName, - System.currentTimeMillis(), + System.nanoTime(), false, timeout, slowTimeout); waitForOperationsToComplete(operation1); Monitorable operation2 = new TestMonitor(operationName, - System.currentTimeMillis(), + System.nanoTime(), false, timeout, slowTimeout); @@ -366,7 +371,7 @@ public void testMultipleThreadsSameNameFailed() throws InterruptedException try { Monitorable operation = new TestMonitor("Test testMultipleThreadsSameName failed", - System.currentTimeMillis(), + System.nanoTime(), false, timeout, slowTimeout); @@ -400,7 +405,7 @@ public void testMultipleThreadsSameNameSlow() throws InterruptedException try { Monitorable operation = new TestMonitor("Test testMultipleThreadsSameName slow", - System.currentTimeMillis(), + System.nanoTime(), false, timeout, slowTimeout); @@ -436,7 +441,7 @@ public void testMultipleThreadsNoFailedOps() throws InterruptedException try { Monitorable operation = new TestMonitor("Test thread " + Thread.currentThread().getName(), - System.currentTimeMillis(), + System.nanoTime(), false, timeout, slowTimeout); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java index 947f9687f4a0..c722738a272e 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java @@ -42,10 +42,10 @@ import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.io.sstable.SSTableMultiWriter; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.ByteBufDataInputPlus; -import org.apache.cassandra.net.async.ByteBufDataOutputStreamPlus; -import org.apache.cassandra.net.async.NonClosingDefaultFileRegion; +import org.apache.cassandra.net.SharedDefaultFileRegion; +import org.apache.cassandra.net.AsyncStreamingOutputPlus; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.streaming.DefaultConnectionFactory; @@ -114,7 +114,7 @@ public void testBlockWriterOverWire() throws IOException CassandraEntireSSTableStreamWriter writer = new CassandraEntireSSTableStreamWriter(sstable, session, CassandraOutgoingFile.getComponentManifest(sstable)); EmbeddedChannel channel = new EmbeddedChannel(); - ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); + AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel); writer.write(out); Queue msgs = channel.outboundMessages(); @@ -133,7 +133,7 @@ public void testBlockReadingAndWritingOverWire() throws Exception // This is needed as Netty releases the ByteBuffers as soon as the channel is flushed ByteBuf serializedFile = Unpooled.buffer(8192); EmbeddedChannel channel = createMockNettyChannel(serializedFile); - ByteBufDataOutputStreamPlus out = ByteBufDataOutputStreamPlus.create(session, channel, 1024 * 1024); + AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel); writer.write(out); @@ -155,7 +155,7 @@ public void testBlockReadingAndWritingOverWire() throws Exception CassandraEntireSSTableStreamReader reader = new CassandraEntireSSTableStreamReader(new StreamMessageHeader(sstable.metadata().id, peer, session.planId(), 0, 0, 0, null), header, session); - SSTableMultiWriter sstableWriter = reader.read(new ByteBufDataInputPlus(serializedFile)); + SSTableMultiWriter sstableWriter = reader.read(new DataInputBuffer(serializedFile.nioBuffer(), false)); Collection newSstables = sstableWriter.finished(); assertEquals(1, newSstables.size()); @@ -188,7 +188,7 @@ public void close() throws IOException @Override public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - ((NonClosingDefaultFileRegion) msg).transferTo(wbc, 0); + ((SharedDefaultFileRegion) msg).transferTo(wbc, 0); super.write(ctx, msg, promise); } }); diff --git a/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java index f478a008d908..4909263440af 100644 --- a/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/ComponentManifestTest.java @@ -20,18 +20,15 @@ import java.io.EOFException; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.LinkedHashMap; import org.junit.Test; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; import org.apache.cassandra.io.sstable.Component; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataOutputBufferFixed; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.ByteBufDataInputPlus; -import org.apache.cassandra.net.async.ByteBufDataOutputPlus; import org.apache.cassandra.serializers.SerializationUtils; import static org.junit.Assert.assertNotEquals; @@ -48,17 +45,17 @@ public void testSerialization() @Test(expected = EOFException.class) public void testSerialization_FailsOnBadBytes() throws IOException { - ByteBuf buf = Unpooled.buffer(512); + ByteBuffer buf = ByteBuffer.allocate(512); ComponentManifest expected = new ComponentManifest(new LinkedHashMap() {{ put(Component.DATA, 100L); }}); - DataOutputPlus output = new ByteBufDataOutputPlus(buf); - ComponentManifest.serializer.serialize(expected, output, MessagingService.VERSION_40); + DataOutputBufferFixed out = new DataOutputBufferFixed(buf); - buf.setInt(0, -100); + ComponentManifest.serializer.serialize(expected, out, MessagingService.VERSION_40); - DataInputPlus input = new ByteBufDataInputPlus(buf); - ComponentManifest actual = ComponentManifest.serializer.deserialize(input, MessagingService.VERSION_40); + buf.putInt(0, -100); + DataInputBuffer in = new DataInputBuffer(out.buffer(), false); + ComponentManifest actual = ComponentManifest.serializer.deserialize(in, MessagingService.VERSION_40); assertNotEquals(expected, actual); } } diff --git a/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java b/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java index 3e566617b9b2..d34878d68532 100644 --- a/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/SettingsTableTest.java @@ -136,40 +136,40 @@ public void testEncryptionOverride() throws Throwable String all = "SELECT * FROM vts.settings WHERE " + "name > 'server_encryption' AND name < 'server_encryptionz' ALLOW FILTERING"; - config.server_encryption_options.enabled = true; + config.server_encryption_options = config.server_encryption_options.withEnabled(true); Assert.assertEquals(9, executeNet(all).all().size()); check(pre + "enabled", "true"); check(pre + "algorithm", null); - config.server_encryption_options.algorithm = "SUPERSSL"; + config.server_encryption_options = config.server_encryption_options.withAlgorithm("SUPERSSL"); check(pre + "algorithm", "SUPERSSL"); check(pre + "cipher_suites", "[]"); - config.server_encryption_options.cipher_suites = new String[]{"c1", "c2"}; + config.server_encryption_options = config.server_encryption_options.withCipherSuites("c1", "c2"); check(pre + "cipher_suites", "[c1, c2]"); check(pre + "protocol", config.server_encryption_options.protocol); - config.server_encryption_options.protocol = "TLSv5"; + config.server_encryption_options = config.server_encryption_options.withProtocol("TLSv5"); check(pre + "protocol", "TLSv5"); check(pre + "optional", "false"); - config.server_encryption_options.optional = true; + config.server_encryption_options = config.server_encryption_options.withOptional(true); check(pre + "optional", "true"); check(pre + "client_auth", "false"); - config.server_encryption_options.require_client_auth = true; + config.server_encryption_options = config.server_encryption_options.withRequireClientAuth(true); check(pre + "client_auth", "true"); check(pre + "endpoint_verification", "false"); - config.server_encryption_options.require_endpoint_verification = true; + config.server_encryption_options = config.server_encryption_options.withRequireEndpointVerification(true); check(pre + "endpoint_verification", "true"); check(pre + "internode_encryption", "none"); - config.server_encryption_options.internode_encryption = InternodeEncryption.all; + config.server_encryption_options = config.server_encryption_options.withInternodeEncryption(InternodeEncryption.all); check(pre + "internode_encryption", "all"); check(pre + "legacy_ssl_storage_port", "false"); - config.server_encryption_options.enable_legacy_ssl_storage_port = true; + config.server_encryption_options = config.server_encryption_options.withLegacySslStoragePort(true); check(pre + "legacy_ssl_storage_port", "true"); } diff --git a/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java b/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java index f8cc49cd645e..57cd4a9f37b8 100644 --- a/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java +++ b/test/unit/org/apache/cassandra/gms/ShadowRoundTest.java @@ -33,10 +33,10 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.IEndpointSnitch; import org.apache.cassandra.locator.PropertyFileSnitch; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MockMessagingService; import org.apache.cassandra.net.MockMessagingSpy; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.service.StorageService; import static org.apache.cassandra.net.MockMessagingService.verb; @@ -71,11 +71,11 @@ public void testDelayedResponse() int noOfSeeds = Gossiper.instance.seeds.size(); final AtomicBoolean ackSend = new AtomicBoolean(false); - MockMessagingSpy spySyn = MockMessagingService.when(verb(MessagingService.Verb.GOSSIP_DIGEST_SYN)) + MockMessagingSpy spySyn = MockMessagingService.when(verb(Verb.GOSSIP_DIGEST_SYN)) .respondN((msgOut, to) -> { // ACK once to finish shadow round, then busy-spin until gossiper has been enabled - // and then reply with remaining ACKs from other seeds + // and then respond with remaining ACKs from other seeds if (!ackSend.compareAndSet(false, true)) { while (!Gossiper.instance.isEnabled()) ; @@ -87,15 +87,17 @@ public void testDelayedResponse() Collections.singletonList(new GossipDigest(to, hb.getGeneration(), hb.getHeartBeatVersion())), Collections.singletonMap(to, state)); - logger.debug("Simulating digest ACK reply"); - return MessageIn.create(to, payload, Collections.emptyMap(), MessagingService.Verb.GOSSIP_DIGEST_ACK, MessagingService.current_version); + logger.debug("Simulating digest ACK response"); + return Message.builder(Verb.GOSSIP_DIGEST_ACK, payload) + .from(to) + .build(); }, noOfSeeds); // GossipDigestAckVerbHandler will send ack2 for each ack received (after the shadow round) - MockMessagingSpy spyAck2 = MockMessagingService.when(verb(MessagingService.Verb.GOSSIP_DIGEST_ACK2)).dontReply(); + MockMessagingSpy spyAck2 = MockMessagingService.when(verb(Verb.GOSSIP_DIGEST_ACK2)).dontReply(); // Migration request messages should not be emitted during shadow round - MockMessagingSpy spyMigrationReq = MockMessagingService.when(verb(MessagingService.Verb.MIGRATION_REQUEST)).dontReply(); + MockMessagingSpy spyMigrationReq = MockMessagingService.when(verb(Verb.SCHEMA_PULL_REQ)).dontReply(); try { @@ -109,7 +111,7 @@ public void testDelayedResponse() // we expect one SYN for each seed during shadow round + additional SYNs after gossiper has been enabled assertTrue(spySyn.messagesIntercepted > noOfSeeds); - // we don't expect to emit any GOSSIP_DIGEST_ACK2 or MIGRATION_REQUEST messages + // we don't expect to emit any GOSSIP_DIGEST_ACK2 or SCHEMA_PULL messages assertEquals(0, spyAck2.messagesIntercepted); assertEquals(0, spyMigrationReq.messagesIntercepted); } diff --git a/test/unit/org/apache/cassandra/hints/HintTest.java b/test/unit/org/apache/cassandra/hints/HintTest.java index aac975b0c220..e3e26d0d7c24 100644 --- a/test/unit/org/apache/cassandra/hints/HintTest.java +++ b/test/unit/org/apache/cassandra/hints/HintTest.java @@ -41,13 +41,12 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.TokenMetadata; import org.apache.cassandra.metrics.StorageMetrics; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.schema.TableParams; import org.apache.cassandra.schema.MigrationManager; import org.apache.cassandra.service.StorageProxy; import org.apache.cassandra.service.StorageService; @@ -58,6 +57,7 @@ import static org.apache.cassandra.Util.dk; import static org.apache.cassandra.hints.HintsTestUtil.assertHintsEqual; import static org.apache.cassandra.hints.HintsTestUtil.assertPartitionsEqual; +import static org.apache.cassandra.net.Verb.HINT_REQ; public class HintTest { @@ -246,9 +246,7 @@ public void testChangedTopology() throws Exception long totalHintCount = StorageProxy.instance.getTotalHints(); // Process hint message. HintMessage message = new HintMessage(localId, hint); - MessagingService.instance().getVerbHandler(MessagingService.Verb.HINT).doVerb( - MessageIn.create(local, message, Collections.emptyMap(), MessagingService.Verb.HINT, MessagingService.current_version), - -1); + HINT_REQ.handler().doVerb(Message.out(HINT_REQ, message)); // hint should not be applied as we no longer are a replica assertNoPartitions(key, TABLE0); @@ -291,9 +289,8 @@ public void testChangedTopologyNotHintable() throws Exception long totalHintCount = StorageMetrics.totalHints.getCount(); // Process hint message. HintMessage message = new HintMessage(localId, hint); - MessagingService.instance().getVerbHandler(MessagingService.Verb.HINT).doVerb( - MessageIn.create(local, message, Collections.emptyMap(), MessagingService.Verb.HINT, MessagingService.current_version), - -1); + HINT_REQ.handler().doVerb( + Message.builder(HINT_REQ, message).from(local).build()); // hint should not be applied as we no longer are a replica assertNoPartitions(key, TABLE0); diff --git a/test/unit/org/apache/cassandra/hints/HintsServiceTest.java b/test/unit/org/apache/cassandra/hints/HintsServiceTest.java index b71140f09380..77783311c794 100644 --- a/test/unit/org/apache/cassandra/hints/HintsServiceTest.java +++ b/test/unit/org/apache/cassandra/hints/HintsServiceTest.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.hints; -import java.util.Collections; import java.util.UUID; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -34,6 +33,7 @@ import com.datastax.driver.core.utils.MoreFutures; import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.db.DecoratedKey; @@ -41,15 +41,16 @@ import org.apache.cassandra.gms.IFailureDetectionEventListener; import org.apache.cassandra.gms.IFailureDetector; import org.apache.cassandra.metrics.StorageMetrics; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.MockMessagingService; import org.apache.cassandra.net.MockMessagingSpy; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.Util.dk; +import static org.apache.cassandra.net.Verb.HINT_REQ; +import static org.apache.cassandra.net.Verb.HINT_RSP; import static org.apache.cassandra.net.MockMessagingService.verb; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -80,7 +81,8 @@ public void cleanup() @Before public void reinstanciateService() throws Throwable { - MessagingService.instance().clearMessageSinks(); + MessagingService.instance().inboundSink.clear(); + MessagingService.instance().outboundSink.clear(); if (!HintsService.instance.isShutDown()) { @@ -182,20 +184,16 @@ public void testPageSeek() throws InterruptedException, ExecutionException private MockMessagingSpy sendHintsAndResponses(int noOfHints, int noOfResponses) { // create spy for hint messages, but only create responses for noOfResponses hints - MessageIn messageIn = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(), - HintResponse.instance, - Collections.emptyMap(), - MessagingService.Verb.REQUEST_RESPONSE, - MessagingService.current_version); + Message message = Message.internalResponse(HINT_RSP, NoPayload.noPayload); MockMessagingSpy spy; if (noOfResponses != -1) { - spy = MockMessagingService.when(verb(MessagingService.Verb.HINT)).respondN(messageIn, noOfResponses); + spy = MockMessagingService.when(verb(HINT_REQ)).respondN(message, noOfResponses); } else { - spy = MockMessagingService.when(verb(MessagingService.Verb.HINT)).respond(messageIn); + spy = MockMessagingService.when(verb(HINT_REQ)).respond(message); } // create and write noOfHints using service diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java index 90a59ddca6b1..e3ba45b44c1e 100644 --- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java +++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java @@ -88,6 +88,8 @@ import org.junit.*; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + public class SASIIndexTest { private static final IPartitioner PARTITIONER; @@ -1416,7 +1418,7 @@ public void testSearchTimeouts() throws Exception try (ReadExecutionController controller = command.executionController()) { - Set rows = getKeys(new QueryPlan(store, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller)); + Set rows = getKeys(new QueryPlan(store, command, DatabaseDescriptor.getRangeRpcTimeout(MILLISECONDS)).execute(controller)); Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2", "key3", "key4" }, rows.toArray(new String[rows.size()]))); } } diff --git a/test/unit/org/apache/cassandra/io/compress/CompressorTest.java b/test/unit/org/apache/cassandra/io/compress/CompressorTest.java index b649f526d9e4..d45d94165785 100644 --- a/test/unit/org/apache/cassandra/io/compress/CompressorTest.java +++ b/test/unit/org/apache/cassandra/io/compress/CompressorTest.java @@ -82,7 +82,7 @@ public void testArrayUncompress(byte[] data, int off, int len) throws IOExceptio // need byte[] representation which direct buffers don't have byte[] compressedBytes = new byte[compressed.capacity()]; - ByteBufferUtil.arrayCopy(compressed, outOffset, compressedBytes, outOffset, compressed.limit() - outOffset); + ByteBufferUtil.copyBytes(compressed, outOffset, compressedBytes, outOffset, compressed.limit() - outOffset); final int decompressedLength = compressor.uncompress(compressedBytes, outOffset, compressed.remaining(), restored, restoreOffset); diff --git a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java index c3931e0ce8c5..3cf96f2698f8 100644 --- a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java @@ -50,7 +50,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReadsListener; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.FileHandle; -import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.Schema; @@ -130,12 +130,16 @@ public void writeDataFile_DataInputPlus() @Test public void writeDataFile_RebufferingByteBufDataInputPlus() { - writeDataTestCycle(buffer -> { - EmbeddedChannel channel = new EmbeddedChannel(); - RebufferingByteBufDataInputPlus inputPlus = new RebufferingByteBufDataInputPlus(1 << 10, 1 << 20, channel.config()); - inputPlus.append(Unpooled.wrappedBuffer(buffer)); - return inputPlus; - }); + try (AsyncStreamingInputPlus input = new AsyncStreamingInputPlus(new EmbeddedChannel())) + { + writeDataTestCycle(buffer -> + { + input.append(Unpooled.wrappedBuffer(buffer)); + return input; + }); + + input.requestClosure(); + } } diff --git a/test/unit/org/apache/cassandra/io/util/BufferedDataOutputStreamTest.java b/test/unit/org/apache/cassandra/io/util/BufferedDataOutputStreamTest.java index 7ca2273a05f8..7c1a0da18198 100644 --- a/test/unit/org/apache/cassandra/io/util/BufferedDataOutputStreamTest.java +++ b/test/unit/org/apache/cassandra/io/util/BufferedDataOutputStreamTest.java @@ -610,48 +610,4 @@ public void testWriteSlowByteOrder() throws Exception } } - @Test - public void testWriteExcessSlow() throws Exception - { - try (DataOutputBuffer dob = new DataOutputBuffer(4)) - { - dob.strictFlushing = true; - ByteBuffer buf = ByteBuffer.allocateDirect(8); - buf.putLong(0, 42); - dob.write(buf); - assertEquals(42, ByteBuffer.wrap(dob.toByteArray()).getLong()); - } - } - - @Test - public void testApplyToChannel() throws Exception - { - setUp(); - Object obj = new Object(); - Object retval = ndosp.applyToChannel( channel -> { - ByteBuffer buf = ByteBuffer.allocate(8); - buf.putLong(0, 42); - try - { - channel.write(buf); - } - catch (Exception e) - { - throw new RuntimeException(e); - } - return obj; - }); - assertEquals(obj, retval); - assertEquals(42, ByteBuffer.wrap(generated.toByteArray()).getLong()); - } - - @Test(expected = UnsupportedOperationException.class) - public void testApplyToChannelThrowsForMisaligned() throws Exception - { - setUp(); - ndosp.strictFlushing = true; - ndosp.applyToChannel( channel -> { - return null; - }); - } } diff --git a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java index fbf6e89ff454..069c2227e8e9 100644 --- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.*; +import java.util.concurrent.TimeUnit; import org.junit.BeforeClass; import org.junit.Test; @@ -30,6 +31,8 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + public class DynamicEndpointSnitchTest { @@ -44,7 +47,7 @@ private static void setScores(DynamicEndpointSnitch dsnitch, int rounds, List= MessagingService.VERSION_40) @@ -65,8 +65,8 @@ private void testAddress(InetAddressAndPort address, int version) throws Excepti } else { - assertEquals(roundtripped.address, address.address); - assertEquals(7000, roundtripped.port); + assertEquals(address.address, roundtripped.address); + assertEquals(InetAddressAndPort.getDefaultPort(), roundtripped.port); } } } diff --git a/test/unit/org/apache/cassandra/net/AsyncChannelPromiseTest.java b/test/unit/org/apache/cassandra/net/AsyncChannelPromiseTest.java new file mode 100644 index 000000000000..c4e62950d936 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/AsyncChannelPromiseTest.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import org.junit.After; +import org.junit.Test; + +import io.netty.channel.ChannelPromise; +import io.netty.channel.embedded.EmbeddedChannel; + +public class AsyncChannelPromiseTest extends TestAbstractAsyncPromise +{ + @After + public void shutdown() + { + exec.shutdownNow(); + } + + private ChannelPromise newPromise() + { + return new AsyncChannelPromise(new EmbeddedChannel()); + } + + @Test + public void testSuccess() + { + for (boolean setUncancellable : new boolean[] { false, true }) + for (boolean tryOrSet : new boolean[]{ false, true }) + testOneSuccess(newPromise(), setUncancellable, tryOrSet, null, null); + } + + @Test + public void testFailure() + { + for (boolean setUncancellable : new boolean[] { false, true }) + for (boolean tryOrSet : new boolean[] { false, true }) + for (Throwable v : new Throwable[] { null, new NullPointerException() }) + testOneFailure(newPromise(), setUncancellable, tryOrSet, v, null); + } + + + @Test + public void testCancellation() + { + for (boolean interruptIfRunning : new boolean[] { true, false }) + testOneCancellation(newPromise(), interruptIfRunning, null); + } + + + @Test + public void testTimeout() + { + for (boolean setUncancellable : new boolean[] { true, false }) + testOneTimeout(newPromise(), setUncancellable); + } + +} diff --git a/test/unit/org/apache/cassandra/net/AsyncMessageOutputPlusTest.java b/test/unit/org/apache/cassandra/net/AsyncMessageOutputPlusTest.java new file mode 100644 index 000000000000..633207c5f82d --- /dev/null +++ b/test/unit/org/apache/cassandra/net/AsyncMessageOutputPlusTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; + +import org.junit.Test; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.net.FrameEncoder.PayloadAllocator; + +import static org.junit.Assert.assertEquals; + +public class AsyncMessageOutputPlusTest +{ + + static + { + DatabaseDescriptor.daemonInitialization(); + } + + @Test + public void testSuccess() throws IOException + { + EmbeddedChannel channel = new TestChannel(4); + ByteBuf read; + try (AsyncMessageOutputPlus out = new AsyncMessageOutputPlus(channel, 32, Integer.MAX_VALUE, PayloadAllocator.simple)) + { + out.writeInt(1); + assertEquals(0, out.flushed()); + assertEquals(0, out.flushedToNetwork()); + assertEquals(4, out.position()); + + out.doFlush(0); + assertEquals(4, out.flushed()); + assertEquals(4, out.flushedToNetwork()); + + out.writeInt(2); + assertEquals(8, out.position()); + assertEquals(4, out.flushed()); + assertEquals(4, out.flushedToNetwork()); + + out.doFlush(0); + assertEquals(8, out.position()); + assertEquals(8, out.flushed()); + assertEquals(4, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(4, read.readableBytes()); + assertEquals(1, read.getInt(0)); + assertEquals(8, out.flushed()); + assertEquals(8, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(4, read.readableBytes()); + assertEquals(2, read.getInt(0)); + + out.write(new byte[64]); + assertEquals(72, out.position()); + assertEquals(40, out.flushed()); + assertEquals(40, out.flushedToNetwork()); + + out.doFlush(0); + assertEquals(72, out.position()); + assertEquals(72, out.flushed()); + assertEquals(40, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(32, read.readableBytes()); + assertEquals(0, read.getLong(0)); + assertEquals(72, out.position()); + assertEquals(72, out.flushed()); + assertEquals(72, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(32, read.readableBytes()); + assertEquals(0, read.getLong(0)); + } + + } + +} diff --git a/test/unit/org/apache/cassandra/net/AsyncOneResponseTest.java b/test/unit/org/apache/cassandra/net/AsyncOneResponseTest.java index 15b3327be203..3d0508cc07c1 100644 --- a/test/unit/org/apache/cassandra/net/AsyncOneResponseTest.java +++ b/test/unit/org/apache/cassandra/net/AsyncOneResponseTest.java @@ -19,45 +19,35 @@ package org.apache.cassandra.net; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import org.junit.Assert; import org.junit.Test; import static org.junit.Assert.assertTrue; public class AsyncOneResponseTest { - @Test(expected = TimeoutException.class) - public void getThrowsExceptionAfterTimeout() throws InterruptedException, TimeoutException + @Test + public void getThrowsExceptionAfterTimeout() throws InterruptedException { AsyncOneResponse response = new AsyncOneResponse<>(); Thread.sleep(2000); - response.get(1, TimeUnit.SECONDS); + Assert.assertFalse(response.await(1, TimeUnit.SECONDS)); } @Test - public void getThrowsExceptionAfterCorrectTimeout() + public void getThrowsExceptionAfterCorrectTimeout() throws InterruptedException { AsyncOneResponse response = new AsyncOneResponse<>(); final long expectedTimeoutMillis = 1000; // Should time out after roughly this time final long schedulingError = 10; // Scheduling is imperfect - boolean hitException = false; // Ensure we actually hit the TimeoutException - - long startTime = System.currentTimeMillis(); - - try - { - response.get(expectedTimeoutMillis, TimeUnit.MILLISECONDS); - } - catch(TimeoutException e) - { - hitException = true; - } - long endTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); + boolean timeout = !response.await(expectedTimeoutMillis, TimeUnit.MILLISECONDS); + long endTime = System.nanoTime(); - assertTrue(hitException); - assertTrue(endTime - startTime > (expectedTimeoutMillis - schedulingError)); + assertTrue(timeout); + assertTrue(TimeUnit.NANOSECONDS.toMillis(endTime - startTime) > (expectedTimeoutMillis - schedulingError)); } } diff --git a/test/unit/org/apache/cassandra/net/AsyncPromiseTest.java b/test/unit/org/apache/cassandra/net/AsyncPromiseTest.java new file mode 100644 index 000000000000..0d2a2e96c25e --- /dev/null +++ b/test/unit/org/apache/cassandra/net/AsyncPromiseTest.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import org.junit.After; +import org.junit.Test; + +import io.netty.util.concurrent.ImmediateEventExecutor; +import io.netty.util.concurrent.Promise; + +public class AsyncPromiseTest extends TestAbstractAsyncPromise +{ + @After + public void shutdown() + { + exec.shutdownNow(); + } + + private Promise newPromise() + { + return new AsyncPromise<>(ImmediateEventExecutor.INSTANCE); + } + + @Test + public void testSuccess() + { + for (boolean setUncancellable : new boolean[] { false, true }) + for (boolean tryOrSet : new boolean[]{ false, true }) + for (Integer v : new Integer[]{ null, 1 }) + testOneSuccess(newPromise(), setUncancellable, tryOrSet, v, 2); + } + + @Test + public void testFailure() + { + for (boolean setUncancellable : new boolean[] { false, true }) + for (boolean tryOrSet : new boolean[] { false, true }) + for (Throwable v : new Throwable[] { null, new NullPointerException() }) + testOneFailure(newPromise(), setUncancellable, tryOrSet, v, 2); + } + + + @Test + public void testCancellation() + { + for (boolean interruptIfRunning : new boolean[] { true, false }) + testOneCancellation(newPromise(), interruptIfRunning, 2); + } + + + @Test + public void testTimeout() + { + for (boolean setUncancellable : new boolean[] { true, false }) + testOneTimeout(newPromise(), setUncancellable); + } + +} diff --git a/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java b/test/unit/org/apache/cassandra/net/AsyncStreamingInputPlusTest.java similarity index 60% rename from test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java rename to test/unit/org/apache/cassandra/net/AsyncStreamingInputPlusTest.java index 69df0403a223..b57574741731 100644 --- a/test/unit/org/apache/cassandra/net/async/RebufferingByteBufDataInputPlusTest.java +++ b/test/unit/org/apache/cassandra/net/AsyncStreamingInputPlusTest.java @@ -16,12 +16,13 @@ * limitations under the License. */ -package org.apache.cassandra.net.async; +package org.apache.cassandra.net; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; +import java.util.concurrent.TimeUnit; import org.junit.After; import org.junit.Assert; @@ -32,65 +33,65 @@ import io.netty.buffer.Unpooled; import io.netty.channel.embedded.EmbeddedChannel; import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus; +import org.apache.cassandra.net.AsyncStreamingInputPlus.InputTimeoutException; -public class RebufferingByteBufDataInputPlusTest +import static org.junit.Assert.assertFalse; + +public class AsyncStreamingInputPlusTest { private EmbeddedChannel channel; - private RebufferingByteBufDataInputPlus inputPlus; + private AsyncStreamingInputPlus inputPlus; private ByteBuf buf; @Before public void setUp() { channel = new EmbeddedChannel(); - inputPlus = new RebufferingByteBufDataInputPlus(1 << 10, 1 << 11, channel.config()); } @After public void tearDown() { - inputPlus.close(); channel.close(); if (buf != null && buf.refCnt() > 0) buf.release(buf.refCnt()); } - @Test (expected = IllegalArgumentException.class) - public void ctor_badWaterMarks() - { - inputPlus = new RebufferingByteBufDataInputPlus(2, 1, null); - } +// @Test +// public void isOpen() +// { +// Assert.assertTrue(inputPlus.isOpen()); +// inputPlus.requestClosure(); +// Assert.assertFalse(inputPlus.isOpen()); +// } @Test - public void isOpen() - { - Assert.assertTrue(inputPlus.isOpen()); - inputPlus.markClose(); - Assert.assertFalse(inputPlus.isOpen()); - } - - @Test (expected = IllegalStateException.class) public void append_closed() { - inputPlus.markClose(); + inputPlus = new AsyncStreamingInputPlus(channel); + inputPlus.requestClosure(); + inputPlus.close(); buf = channel.alloc().buffer(4); - inputPlus.append(buf); + assertFalse(inputPlus.append(buf)); } @Test - public void append_normal() throws EOFException + public void append_normal() { + inputPlus = new AsyncStreamingInputPlus(channel); int size = 4; buf = channel.alloc().buffer(size); buf.writerIndex(size); inputPlus.append(buf); - Assert.assertEquals(buf.readableBytes(), inputPlus.available()); + Assert.assertEquals(buf.readableBytes(), inputPlus.unsafeAvailable()); } @Test public void read() throws IOException { + inputPlus = new AsyncStreamingInputPlus(channel); // put two buffers of 8 bytes each into the queue. // then read an int, then a long. the latter tests offset into the inputPlus, as well as spanning across queued buffers. // the values of those int/long will both be '42', but spread across both queue buffers. @@ -102,57 +103,60 @@ public void read() throws IOException buf.writeInt(42); buf.writerIndex(8); inputPlus.append(buf); - Assert.assertEquals(16, inputPlus.available()); - - ByteBuffer out = ByteBuffer.allocate(4); - int readCount = inputPlus.read(out); - Assert.assertEquals(4, readCount); - out.flip(); - Assert.assertEquals(42, out.getInt()); - Assert.assertEquals(12, inputPlus.available()); - - out = ByteBuffer.allocate(8); - readCount = inputPlus.read(out); - Assert.assertEquals(8, readCount); - out.flip(); - Assert.assertEquals(42, out.getLong()); - Assert.assertEquals(4, inputPlus.available()); + Assert.assertEquals(16, inputPlus.unsafeAvailable()); + +// ByteBuffer out = ByteBuffer.allocate(4); +// int readCount = inputPlus.read(out); +// Assert.assertEquals(4, readCount); +// out.flip(); +// Assert.assertEquals(42, out.getInt()); +// Assert.assertEquals(12, inputPlus.unsafeAvailable()); + +// out = ByteBuffer.allocate(8); +// readCount = inputPlus.read(out); +// Assert.assertEquals(8, readCount); +// out.flip(); +// Assert.assertEquals(42, out.getLong()); +// Assert.assertEquals(4, inputPlus.unsafeAvailable()); } - @Test (expected = EOFException.class) - public void read_closed() throws IOException - { - inputPlus.markClose(); - ByteBuffer buf = ByteBuffer.allocate(1); - inputPlus.read(buf); - } +// @Test (expected = EOFException.class) +// public void read_closed() throws IOException +// { +// inputPlus.requestClosure(); +// ByteBuffer buf = ByteBuffer.allocate(1); +// inputPlus.read(buf); +// } - @Test (expected = EOFException.class) - public void available_closed() throws EOFException + @Test + public void available_closed() { - inputPlus.markClose(); - inputPlus.available(); + inputPlus = new AsyncStreamingInputPlus(channel); + inputPlus.requestClosure(); + inputPlus.unsafeAvailable(); } @Test - public void available_HappyPath() throws EOFException + public void available_HappyPath() { + inputPlus = new AsyncStreamingInputPlus(channel); int size = 4; buf = channel.alloc().heapBuffer(size); buf.writerIndex(size); inputPlus.append(buf); - Assert.assertEquals(size, inputPlus.available()); + Assert.assertEquals(size, inputPlus.unsafeAvailable()); } @Test - public void available_ClosedButWithBytes() throws EOFException + public void available_ClosedButWithBytes() { + inputPlus = new AsyncStreamingInputPlus(channel); int size = 4; buf = channel.alloc().heapBuffer(size); buf.writerIndex(size); inputPlus.append(buf); - inputPlus.markClose(); - Assert.assertEquals(size, inputPlus.available()); + inputPlus.requestClosure(); + Assert.assertEquals(size, inputPlus.unsafeAvailable()); } @Test @@ -193,6 +197,8 @@ public void consumeUntil_MultipleBuffer_Fails() throws IOException private void consumeUntilTestCycle(int nBuffs, int buffSize, int startOffset, int len) throws IOException { + inputPlus = new AsyncStreamingInputPlus(channel); + byte[] expectedBytes = new byte[len]; int count = 0; for (int j=0; j < nBuffs; j++) @@ -208,16 +214,17 @@ private void consumeUntilTestCycle(int nBuffs, int buffSize, int startOffset, in inputPlus.append(buf); } - inputPlus.append(channel.alloc().buffer(0)); + inputPlus.requestClosure(); TestableWritableByteChannel wbc = new TestableWritableByteChannel(len); inputPlus.skipBytesFully(startOffset); BufferedDataOutputStreamPlus writer = new BufferedDataOutputStreamPlus(wbc); - inputPlus.consumeUntil(writer, len); + inputPlus.consume(buffer -> { writer.write(buffer); return buffer.remaining(); }, len); + writer.close(); - Assert.assertEquals(String.format("Test with {} buffers starting at {} consuming {} bytes", nBuffs, startOffset, - len), len, wbc.writtenBytes.readableBytes()); + Assert.assertEquals(String.format("Test with %d buffers starting at %d consuming %d bytes", nBuffs, startOffset, len), + len, wbc.writtenBytes.readableBytes()); Assert.assertArrayEquals(expectedBytes, wbc.writtenBytes.array()); } @@ -232,7 +239,7 @@ public TestableWritableByteChannel(int initialCapacity) writtenBytes = Unpooled.buffer(initialCapacity); } - public int write(ByteBuffer src) throws IOException + public int write(ByteBuffer src) { int size = src.remaining(); writtenBytes.writeBytes(src); @@ -244,9 +251,30 @@ public boolean isOpen() return isOpen; } - public void close() throws IOException + public void close() { isOpen = false; } - }; + } + + @Test + public void rebufferTimeout() throws IOException + { + long timeoutMillis = 1000; + inputPlus = new AsyncStreamingInputPlus(channel, timeoutMillis, TimeUnit.MILLISECONDS); + + long startNanos = System.nanoTime(); + try + { + inputPlus.readInt(); + Assert.fail("should not have been able to read from the queue"); + } + catch (InputTimeoutException e) + { + // this is the success case, and is expected. any other exception is a failure. + } + + long durationNanos = System.nanoTime() - startNanos; + Assert.assertTrue(TimeUnit.MILLISECONDS.toNanos(timeoutMillis) <= durationNanos); + } } diff --git a/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java b/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java new file mode 100644 index 000000000000..fa5009a3f50e --- /dev/null +++ b/test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import org.junit.Test; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.streaming.StreamManager; +import org.apache.cassandra.utils.FBUtilities; + +import static org.junit.Assert.assertEquals; + +public class AsyncStreamingOutputPlusTest +{ + + static + { + DatabaseDescriptor.daemonInitialization(); + } + + @Test + public void testSuccess() throws IOException + { + EmbeddedChannel channel = new TestChannel(4); + ByteBuf read; + try (AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel)) + { + out.writeInt(1); + assertEquals(0, out.flushed()); + assertEquals(0, out.flushedToNetwork()); + assertEquals(4, out.position()); + + out.doFlush(0); + assertEquals(4, out.flushed()); + assertEquals(4, out.flushedToNetwork()); + + out.writeInt(2); + assertEquals(8, out.position()); + assertEquals(4, out.flushed()); + assertEquals(4, out.flushedToNetwork()); + + out.doFlush(0); + assertEquals(8, out.position()); + assertEquals(8, out.flushed()); + assertEquals(4, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(4, read.readableBytes()); + assertEquals(1, read.getInt(0)); + assertEquals(8, out.flushed()); + assertEquals(8, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(4, read.readableBytes()); + assertEquals(2, read.getInt(0)); + + out.write(new byte[16]); + assertEquals(24, out.position()); + assertEquals(8, out.flushed()); + assertEquals(8, out.flushedToNetwork()); + + out.doFlush(0); + assertEquals(24, out.position()); + assertEquals(24, out.flushed()); + assertEquals(24, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(16, read.readableBytes()); + assertEquals(0, read.getLong(0)); + assertEquals(0, read.getLong(8)); + assertEquals(24, out.position()); + assertEquals(24, out.flushed()); + assertEquals(24, out.flushedToNetwork()); + + out.writeToChannel(alloc -> { + ByteBuffer buffer = alloc.get(16); + buffer.putLong(1); + buffer.putLong(2); + buffer.flip(); + }, new StreamManager.StreamRateLimiter(FBUtilities.getBroadcastAddressAndPort())); + + assertEquals(40, out.position()); + assertEquals(40, out.flushed()); + assertEquals(40, out.flushedToNetwork()); + + read = channel.readOutbound(); + assertEquals(16, read.readableBytes()); + assertEquals(1, read.getLong(0)); + assertEquals(2, read.getLong(8)); + } + + } + +} diff --git a/test/unit/org/apache/cassandra/net/ChunkedInputPlusTest.java b/test/unit/org/apache/cassandra/net/ChunkedInputPlusTest.java new file mode 100644 index 000000000000..f90fcd17ca6d --- /dev/null +++ b/test/unit/org/apache/cassandra/net/ChunkedInputPlusTest.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import com.google.common.collect.Lists; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.net.ChunkedInputPlus; +import org.apache.cassandra.net.ShareableBytes; + +import static org.junit.Assert.*; + +public class ChunkedInputPlusTest +{ + @BeforeClass + public static void setUp() + { + DatabaseDescriptor.clientInitialization(); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyIterable() + { + ChunkedInputPlus.of(Collections.emptyList()); + } + + @Test + public void testUnderRead() throws IOException + { + List chunks = Lists.newArrayList( + chunk(1, 1), chunk(2, 2), chunk(3, 3) + ); + + try (ChunkedInputPlus input = ChunkedInputPlus.of(chunks)) + { + byte[] readBytes = new byte[5]; + input.readFully(readBytes); + assertArrayEquals(new byte[] { 1, 2, 2, 3, 3 }, readBytes); + + assertFalse(chunks.get(0).hasRemaining()); + assertFalse(chunks.get(1).hasRemaining()); + assertTrue (chunks.get(2).hasRemaining()); + + assertTrue (chunks.get(0).isReleased()); + assertTrue (chunks.get(1).isReleased()); + assertFalse(chunks.get(2).isReleased()); + } + + // close should release the last chunk + assertTrue(chunks.get(2).isReleased()); + } + + @Test + public void testExactRead() throws IOException + { + List chunks = Lists.newArrayList( + chunk(1, 1), chunk(2, 2), chunk(3, 3) + ); + + try (ChunkedInputPlus input = ChunkedInputPlus.of(chunks)) + { + byte[] readBytes = new byte[6]; + input.readFully(readBytes); + assertArrayEquals(new byte[] { 1, 2, 2, 3, 3, 3 }, readBytes); + + assertFalse(chunks.get(0).hasRemaining()); + assertFalse(chunks.get(1).hasRemaining()); + assertFalse(chunks.get(2).hasRemaining()); + + assertTrue (chunks.get(0).isReleased()); + assertTrue (chunks.get(1).isReleased()); + assertFalse(chunks.get(2).isReleased()); + } + + // close should release the last chunk + assertTrue(chunks.get(2).isReleased()); + } + + @Test + public void testOverRead() throws IOException + { + List chunks = Lists.newArrayList( + chunk(1, 1), chunk(2, 2), chunk(3, 3) + ); + + boolean eofCaught = false; + try (ChunkedInputPlus input = ChunkedInputPlus.of(chunks)) + { + byte[] readBytes = new byte[7]; + input.readFully(readBytes); + assertArrayEquals(new byte[] { 1, 2, 2, 3, 3, 3, 4 }, readBytes); + } + catch (EOFException e) + { + eofCaught = true; + + assertFalse(chunks.get(0).hasRemaining()); + assertFalse(chunks.get(1).hasRemaining()); + assertFalse(chunks.get(2).hasRemaining()); + + assertTrue (chunks.get(2).isReleased()); + assertTrue (chunks.get(1).isReleased()); + assertTrue (chunks.get(2).isReleased()); + } + assertTrue(eofCaught); + } + + @Test + public void testRemainder() throws IOException + { + List chunks = Lists.newArrayList( + chunk(1, 1), chunk(2, 2), chunk(3, 3) + ); + + try (ChunkedInputPlus input = ChunkedInputPlus.of(chunks)) + { + byte[] readBytes = new byte[5]; + input.readFully(readBytes); + assertArrayEquals(new byte[] { 1, 2, 2, 3, 3 }, readBytes); + + assertEquals(1, input.remainder()); + + assertTrue(chunks.get(0).isReleased()); + assertTrue(chunks.get(1).isReleased()); + assertTrue(chunks.get(2).isReleased()); // should be released by remainder() + } + } + + private ShareableBytes chunk(int size, int fill) + { + ByteBuffer buffer = ByteBuffer.allocate(size); + Arrays.fill(buffer.array(), (byte) fill); + return ShareableBytes.wrap(buffer); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/net/ConnectionTest.java b/test/unit/org/apache/cassandra/net/ConnectionTest.java new file mode 100644 index 000000000000..17cae7145804 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/ConnectionTest.java @@ -0,0 +1,811 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.function.ToLongFunction; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.exceptions.UnknownColumnException; +import org.apache.cassandra.io.IVersionedAsymmetricSerializer; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.utils.FBUtilities; + +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.net.MessagingService.VERSION_30; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.NoPayload.noPayload; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.ConnectionUtils.*; +import static org.apache.cassandra.net.ConnectionType.LARGE_MESSAGES; +import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES; +import static org.apache.cassandra.net.OutboundConnectionSettings.Framing.LZ4; +import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +public class ConnectionTest +{ + private static final Logger logger = LoggerFactory.getLogger(ConnectionTest.class); + private static final SocketFactory factory = new SocketFactory(); + + private final Map>> serializers = new HashMap<>(); + private final Map>> handlers = new HashMap<>(); + private final Map> timeouts = new HashMap<>(); + + private void unsafeSetSerializer(Verb verb, Supplier> supplier) throws Throwable + { + serializers.putIfAbsent(verb, verb.unsafeSetSerializer(supplier)); + } + + private void unsafeSetHandler(Verb verb, Supplier> supplier) throws Throwable + { + handlers.putIfAbsent(verb, verb.unsafeSetHandler(supplier)); + } + + private void unsafeSetExpiration(Verb verb, ToLongFunction expiration) throws Throwable + { + timeouts.putIfAbsent(verb, verb.unsafeSetExpiration(expiration)); + } + + @After + public void resetVerbs() throws Throwable + { + for (Map.Entry>> e : serializers.entrySet()) + e.getKey().unsafeSetSerializer(e.getValue()); + serializers.clear(); + for (Map.Entry>> e : handlers.entrySet()) + e.getKey().unsafeSetHandler(e.getValue()); + handlers.clear(); + for (Map.Entry> e : timeouts.entrySet()) + e.getKey().unsafeSetExpiration(e.getValue()); + timeouts.clear(); + } + + @BeforeClass + public static void startup() + { + DatabaseDescriptor.daemonInitialization(); + } + + @AfterClass + public static void cleanup() throws InterruptedException + { + factory.shutdownNow(); + } + + interface SendTest + { + void accept(InboundMessageHandlers inbound, OutboundConnection outbound, InetAddressAndPort endpoint) throws Throwable; + } + + interface ManualSendTest + { + void accept(Settings settings, InboundSockets inbound, OutboundConnection outbound, InetAddressAndPort endpoint) throws Throwable; + } + + static class Settings + { + static final Settings SMALL = new Settings(SMALL_MESSAGES); + static final Settings LARGE = new Settings(LARGE_MESSAGES); + final ConnectionType type; + final Function outbound; + final Function inbound; + Settings(ConnectionType type) + { + this(type, Function.identity(), Function.identity()); + } + Settings(ConnectionType type, Function outbound, + Function inbound) + { + this.type = type; + this.outbound = outbound; + this.inbound = inbound; + } + Settings outbound(Function outbound) + { + return new Settings(type, this.outbound.andThen(outbound), inbound); + } + Settings inbound(Function inbound) + { + return new Settings(type, outbound, this.inbound.andThen(inbound)); + } + Settings override(Settings settings) + { + return new Settings(settings.type != null ? settings.type : type, + outbound.andThen(settings.outbound), + inbound.andThen(settings.inbound)); + } + } + + static final EncryptionOptions.ServerEncryptionOptions encryptionOptions = + new EncryptionOptions.ServerEncryptionOptions() + .withEnabled(true) + .withLegacySslStoragePort(true) + .withOptional(true) + .withInternodeEncryption(EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.all) + .withKeyStore("test/conf/cassandra_ssl_test.keystore") + .withKeyStorePassword("cassandra") + .withTrustStore("test/conf/cassandra_ssl_test.truststore") + .withTrustStorePassword("cassandra") + .withRequireClientAuth(false) + .withCipherSuites("TLS_RSA_WITH_AES_128_CBC_SHA"); + + static final AcceptVersions legacy = new AcceptVersions(VERSION_30, VERSION_30); + + static final List> MODIFIERS = ImmutableList.of( + settings -> settings.outbound(outbound -> outbound.withAcceptVersions(legacy)) + .inbound(inbound -> inbound.withAcceptMessaging(legacy)), + settings -> settings.outbound(outbound -> outbound.withEncryption(encryptionOptions)) + .inbound(inbound -> inbound.withEncryption(encryptionOptions)), + settings -> settings.outbound(outbound -> outbound.withFraming(LZ4)) + ); + + static final List SETTINGS = applyPowerSet( + ImmutableList.of(Settings.SMALL, Settings.LARGE), + MODIFIERS + ); + + private static List applyPowerSet(List settings, List> modifiers) + { + List result = new ArrayList<>(); + for (Set> set : Sets.powerSet(new HashSet<>(modifiers))) + { + for (T s : settings) + { + for (Function f : set) + s = f.apply(s); + result.add(s); + } + } + return result; + } + + private void test(Settings extraSettings, SendTest test) throws Throwable + { + for (Settings s : SETTINGS) + doTest(s.override(extraSettings), test); + } + private void test(SendTest test) throws Throwable + { + for (Settings s : SETTINGS) + doTest(s, test); + } + + private void testManual(ManualSendTest test) throws Throwable + { + for (Settings s : SETTINGS) + doTestManual(s, test); + } + + private void doTest(Settings settings, SendTest test) throws Throwable + { + doTestManual(settings, (ignore, inbound, outbound, endpoint) -> { + inbound.open().sync(); + test.accept(MessagingService.instance().getInbound(endpoint), outbound, endpoint); + }); + } + + private void doTestManual(Settings settings, ManualSendTest test) throws Throwable + { + InetAddressAndPort endpoint = FBUtilities.getBroadcastAddressAndPort(); + InboundConnectionSettings inboundSettings = settings.inbound.apply(new InboundConnectionSettings()) + .withBindAddress(endpoint) + .withSocketFactory(factory); + InboundSockets inbound = new InboundSockets(Collections.singletonList(inboundSettings)); + OutboundConnectionSettings outboundTemplate = settings.outbound.apply(new OutboundConnectionSettings(endpoint)) + .withDefaultReserveLimits() + .withSocketFactory(factory) + .withDefaults(ConnectionCategory.MESSAGING); + ResourceLimits.EndpointAndGlobal reserveCapacityInBytes = new ResourceLimits.EndpointAndGlobal(new ResourceLimits.Concurrent(outboundTemplate.applicationSendQueueReserveEndpointCapacityInBytes), outboundTemplate.applicationSendQueueReserveGlobalCapacityInBytes); + OutboundConnection outbound = new OutboundConnection(settings.type, outboundTemplate, reserveCapacityInBytes); + try + { + logger.info("Running {} {} -> {}", outbound.messagingVersion(), outbound.settings(), inboundSettings); + test.accept(settings, inbound, outbound, endpoint); + } + finally + { + outbound.close(false); + inbound.close().get(30L, SECONDS); + outbound.close(false).get(30L, SECONDS); + resetVerbs(); + MessagingService.instance().messageHandlers.clear(); + } + } + + @Test + public void testSendSmall() throws Throwable + { + test((inbound, outbound, endpoint) -> { + int version = outbound.settings().acceptVersions.max; + int count = 10; + + CountDownLatch deliveryDone = new CountDownLatch(1); + CountDownLatch receiveDone = new CountDownLatch(count); + + unsafeSetHandler(Verb._TEST_1, () -> msg -> receiveDone.countDown()); + Message message = Message.out(Verb._TEST_1, noPayload); + for (int i = 0 ; i < count ; ++i) + outbound.enqueue(message); + + Assert.assertTrue(receiveDone.await(10, SECONDS)); + outbound.unsafeRunOnDelivery(deliveryDone::countDown); + Assert.assertTrue(deliveryDone.await(10, SECONDS)); + + check(outbound).submitted(10) + .sent (10, 10 * message.serializedSize(version)) + .pending ( 0, 0) + .overload ( 0, 0) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + check(inbound) .received (10, 10 * message.serializedSize(version)) + .processed(10, 10 * message.serializedSize(version)) + .pending ( 0, 0) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + }); + } + + @Test + public void testSendLarge() throws Throwable + { + test((inbound, outbound, endpoint) -> { + int version = outbound.settings().acceptVersions.max; + int count = 10; + + CountDownLatch deliveryDone = new CountDownLatch(1); + CountDownLatch receiveDone = new CountDownLatch(count); + + unsafeSetSerializer(Verb._TEST_1, () -> new IVersionedSerializer() + { + public void serialize(Object noPayload, DataOutputPlus out, int version) throws IOException + { + for (int i = 0 ; i < LARGE_MESSAGE_THRESHOLD + 1 ; ++i) + out.writeByte(i); + } + public Object deserialize(DataInputPlus in, int version) throws IOException + { + in.skipBytesFully(LARGE_MESSAGE_THRESHOLD + 1); + return noPayload; + } + public long serializedSize(Object noPayload, int version) + { + return LARGE_MESSAGE_THRESHOLD + 1; + } + }); + unsafeSetHandler(Verb._TEST_1, () -> msg -> receiveDone.countDown()); + Message message = Message.builder(Verb._TEST_1, new Object()) + .withExpiresAt(System.nanoTime() + SECONDS.toNanos(30L)) + .build(); + for (int i = 0 ; i < count ; ++i) + outbound.enqueue(message); + Assert.assertTrue(receiveDone.await(10, SECONDS)); + + outbound.unsafeRunOnDelivery(deliveryDone::countDown); + Assert.assertTrue(deliveryDone.await(10, SECONDS)); + + check(outbound).submitted(10) + .sent (10, 10 * message.serializedSize(version)) + .pending ( 0, 0) + .overload ( 0, 0) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + check(inbound) .received (10, 10 * message.serializedSize(version)) + .processed(10, 10 * message.serializedSize(version)) + .pending ( 0, 0) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + }); + } + + @Test + public void testInsufficientSpace() throws Throwable + { + test(new Settings(null).outbound(settings -> settings + .withApplicationReserveSendQueueCapacityInBytes(1 << 15, new ResourceLimits.Concurrent(1 << 16)) + .withApplicationSendQueueCapacityInBytes(1 << 16)), + (inbound, outbound, endpoint) -> { + + CountDownLatch done = new CountDownLatch(1); + Message message = Message.out(Verb._TEST_1, new Object()); + MessagingService.instance().callbacks.addWithExpiration(new RequestCallback() + { + @Override + public void onFailure(InetAddressAndPort from, RequestFailureReason failureReason) + { + done.countDown(); + } + + @Override + public boolean invokeOnFailure() + { + return true; + } + + @Override + public void onResponse(Message msg) + { + throw new IllegalStateException(); + } + + }, message, endpoint); + AtomicInteger delivered = new AtomicInteger(); + unsafeSetSerializer(Verb._TEST_1, () -> new IVersionedSerializer() + { + public void serialize(Object o, DataOutputPlus out, int version) throws IOException + { + for (int i = 0 ; i <= 4 << 16 ; i += 8L) + out.writeLong(1L); + } + + public Object deserialize(DataInputPlus in, int version) throws IOException + { + in.skipBytesFully(4 << 16); + return null; + } + + public long serializedSize(Object o, int version) + { + return 4 << 16; + } + }); + unsafeSetHandler(Verb._TEST_1, () -> msg -> delivered.incrementAndGet()); + outbound.enqueue(message); + Assert.assertTrue(done.await(10, SECONDS)); + Assert.assertEquals(0, delivered.get()); + check(outbound).submitted( 1) + .sent ( 0, 0) + .pending ( 0, 0) + .overload ( 1, message.serializedSize(current_version)) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + check(inbound) .received ( 0, 0) + .processed( 0, 0) + .pending ( 0, 0) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + }); + } + + @Test + public void testSerializeError() throws Throwable + { + test((inbound, outbound, endpoint) -> { + int version = outbound.settings().acceptVersions.max; + int count = 100; + + CountDownLatch deliveryDone = new CountDownLatch(1); + CountDownLatch receiveDone = new CountDownLatch(90); + + AtomicInteger serialized = new AtomicInteger(); + Message message = Message.builder(Verb._TEST_1, new Object()) + .withExpiresAt(System.nanoTime() + SECONDS.toNanos(30L)) + .build(); + unsafeSetSerializer(Verb._TEST_1, () -> new IVersionedSerializer() + { + public void serialize(Object o, DataOutputPlus out, int version) throws IOException + { + int i = serialized.incrementAndGet(); + if (0 == (i & 15)) + { + if (0 == (i & 16)) + out.writeByte(i); + throw new IOException(); + } + + if (1 != (i & 31)) + out.writeByte(i); + } + + public Object deserialize(DataInputPlus in, int version) throws IOException + { + in.readByte(); + return null; + } + + public long serializedSize(Object o, int version) + { + return 1; + } + }); + + unsafeSetHandler(Verb._TEST_1, () -> msg -> receiveDone.countDown()); + for (int i = 0 ; i < count ; ++i) + outbound.enqueue(message); + + Assert.assertTrue(receiveDone.await(1, MINUTES)); + outbound.unsafeRunOnDelivery(deliveryDone::countDown); + Assert.assertTrue(deliveryDone.await(10, SECONDS)); + + check(outbound).submitted(100) + .sent ( 90, 90 * message.serializedSize(version)) + .pending ( 0, 0) + .overload ( 0, 0) + .expired ( 0, 0) + .error ( 10, 10 * message.serializedSize(version)) + .check(); + check(inbound) .received ( 90, 90 * message.serializedSize(version)) + .processed( 90, 90 * message.serializedSize(version)) + .pending ( 0, 0) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + }); + } + + @Test + public void testTimeout() throws Throwable + { + test((inbound, outbound, endpoint) -> { + int version = outbound.settings().acceptVersions.max; + int count = 10; + CountDownLatch enqueueDone = new CountDownLatch(1); + CountDownLatch deliveryDone = new CountDownLatch(1); + AtomicInteger delivered = new AtomicInteger(); + Verb._TEST_1.unsafeSetHandler(() -> msg -> delivered.incrementAndGet()); + Message message = Message.builder(Verb._TEST_1, noPayload) + .withExpiresAt(approxTime.now() + TimeUnit.DAYS.toNanos(1L)) + .build(); + long sentSize = message.serializedSize(version); + outbound.enqueue(message); + long timeoutMillis = 10L; + while (delivered.get() < 1); + outbound.unsafeRunOnDelivery(() -> Uninterruptibles.awaitUninterruptibly(enqueueDone, 1L, TimeUnit.DAYS)); + message = Message.builder(Verb._TEST_1, noPayload) + .withExpiresAt(approxTime.now() + TimeUnit.MILLISECONDS.toNanos(timeoutMillis)) + .build(); + for (int i = 0 ; i < count ; ++i) + outbound.enqueue(message); + Uninterruptibles.sleepUninterruptibly(timeoutMillis * 2, TimeUnit.MILLISECONDS); + enqueueDone.countDown(); + outbound.unsafeRunOnDelivery(deliveryDone::countDown); + Assert.assertTrue(deliveryDone.await(1, MINUTES)); + Assert.assertEquals(1, delivered.get()); + check(outbound).submitted( 11) + .sent ( 1, sentSize) + .pending ( 0, 0) + .overload ( 0, 0) + .expired ( 10, 10 * message.serializedSize(current_version)) + .error ( 0, 0) + .check(); + check(inbound) .received ( 1, sentSize) + .processed( 1, sentSize) + .pending ( 0, 0) + .expired ( 0, 0) + .error ( 0, 0) + .check(); + }); + } + + @Test + public void testPre40() throws Throwable + { + MessagingService.instance().versions.set(FBUtilities.getBroadcastAddressAndPort(), + MessagingService.VERSION_30); + + try + { + test((inbound, outbound, endpoint) -> { + CountDownLatch done = new CountDownLatch(1); + unsafeSetHandler(Verb._TEST_1, + () -> (msg) -> done.countDown()); + + Message message = Message.out(Verb._TEST_1, noPayload); + outbound.enqueue(message); + Assert.assertTrue(done.await(1, MINUTES)); + Assert.assertTrue(outbound.isConnected()); + }); + } + finally + { + MessagingService.instance().versions.set(FBUtilities.getBroadcastAddressAndPort(), + current_version); + } + } + + @Test + public void testCloseIfEndpointDown() throws Throwable + { + testManual((settings, inbound, outbound, endpoint) -> { + Message message = Message.builder(Verb._TEST_1, noPayload) + .withExpiresAt(System.nanoTime() + SECONDS.toNanos(30L)) + .build(); + + for (int i = 0 ; i < 1000 ; ++i) + outbound.enqueue(message); + + outbound.close(true).get(10L, MINUTES); + }); + } + + @Test + public void testMessagePurging() throws Throwable + { + testManual((settings, inbound, outbound, endpoint) -> { + Runnable testWhileDisconnected = () -> { + try + { + for (int i = 0; i < 5; i++) + { + Message message = Message.builder(Verb._TEST_1, noPayload) + .withExpiresAt(System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(50L)) + .build(); + outbound.enqueue(message); + Assert.assertFalse(outbound.isConnected()); + Assert.assertEquals(1, outbound.pendingCount()); + CompletableFuture.runAsync(() -> { + while (outbound.pendingCount() > 0 && !Thread.interrupted()) {} + }).get(10, SECONDS); + // Message should have been purged + Assert.assertEquals(0, outbound.pendingCount()); + } + } + catch (Throwable t) + { + throw new RuntimeException(t); + } + }; + + testWhileDisconnected.run(); + + try + { + inbound.open().sync(); + CountDownLatch receiveDone = new CountDownLatch(1); + CountDownLatch deliveryDone = new CountDownLatch(1); + + unsafeSetHandler(Verb._TEST_1, () -> msg -> receiveDone.countDown()); + outbound.enqueue(Message.out(Verb._TEST_1, noPayload)); + Assert.assertEquals(1, outbound.pendingCount()); + outbound.unsafeRunOnDelivery(deliveryDone::countDown); + + Assert.assertTrue(receiveDone.await(10, SECONDS)); + Assert.assertTrue(deliveryDone.await(10, SECONDS)); + Assert.assertEquals(0, receiveDone.getCount()); + Assert.assertEquals(0, outbound.pendingCount()); + } + finally + { + inbound.close().get(10, SECONDS); + // Wait until disconnected + CompletableFuture.runAsync(() -> { + while (outbound.isConnected() && !Thread.interrupted()) {} + }).get(10, SECONDS); + } + + testWhileDisconnected.run(); + }); + } + + @Test + public void testMessageDeliveryOnReconnect() throws Throwable + { + testManual((settings, inbound, outbound, endpoint) -> { + try + { + inbound.open().sync(); + CountDownLatch done = new CountDownLatch(1); + unsafeSetHandler(Verb._TEST_1, () -> msg -> done.countDown()); + outbound.enqueue(Message.out(Verb._TEST_1, noPayload)); + Assert.assertTrue(done.await(10, SECONDS)); + Assert.assertEquals(done.getCount(), 0); + + // Simulate disconnect + inbound.close().get(10, SECONDS); + MessagingService.instance().removeInbound(endpoint); + inbound = new InboundSockets(settings.inbound.apply(new InboundConnectionSettings())); + inbound.open().sync(); + + CountDownLatch latch2 = new CountDownLatch(1); + unsafeSetHandler(Verb._TEST_1, () -> msg -> latch2.countDown()); + outbound.enqueue(Message.out(Verb._TEST_1, noPayload)); + + latch2.await(10, SECONDS); + Assert.assertEquals(latch2.getCount(), 0); + } + finally + { + inbound.close().get(10, SECONDS); + outbound.close(false).get(10, SECONDS); + } + }); + } + + @Test + public void testRecoverableCorruptedMessageDelivery() throws Throwable + { + test((inbound, outbound, endpoint) -> { + int version = outbound.settings().acceptVersions.max; + if (version < VERSION_40) + return; + + AtomicInteger counter = new AtomicInteger(); + unsafeSetSerializer(Verb._TEST_1, () -> new IVersionedSerializer() + { + public void serialize(Object o, DataOutputPlus out, int version) throws IOException + { + out.writeInt((Integer) o); + } + + public Object deserialize(DataInputPlus in, int version) throws IOException + { + if (counter.getAndIncrement() == 3) + throw new UnknownColumnException(""); + + return in.readInt(); + } + + public long serializedSize(Object o, int version) + { + return Integer.BYTES; + } + }); + + // Connect + connect(outbound); + + CountDownLatch latch = new CountDownLatch(4); + unsafeSetHandler(Verb._TEST_1, () -> message -> latch.countDown()); + for (int i = 0; i < 5; i++) + outbound.enqueue(Message.out(Verb._TEST_1, 0xffffffff)); + + latch.await(10, SECONDS); + Assert.assertEquals(0, latch.getCount()); + Assert.assertEquals(6, counter.get()); + }); + } + + @Test + public void testCRCCorruption() throws Throwable + { + test((inbound, outbound, endpoint) -> { + int version = outbound.settings().acceptVersions.max; + if (version < VERSION_40) + return; + + unsafeSetSerializer(Verb._TEST_1, () -> new IVersionedSerializer() + { + public void serialize(Object o, DataOutputPlus out, int version) throws IOException + { + out.writeInt((Integer) o); + } + + public Object deserialize(DataInputPlus in, int version) throws IOException + { + return in.readInt(); + } + + public long serializedSize(Object o, int version) + { + return Integer.BYTES; + } + }); + + connect(outbound); + + outbound.unsafeGetChannel().pipeline().addFirst(new ChannelOutboundHandlerAdapter() { + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { + ByteBuf bb = (ByteBuf) msg; + bb.setByte(0, 0xAB); + ctx.write(msg, promise); + } + }); + outbound.enqueue(Message.out(Verb._TEST_1, 0xffffffff)); + CompletableFuture.runAsync(() -> { + while (outbound.isConnected() && !Thread.interrupted()) {} + }).get(10, SECONDS); + Assert.assertFalse(outbound.isConnected()); + // TODO: count corruptions + + connect(outbound); + }); + } + + @Test + public void testAcquireReleaseOutbound() throws Throwable + { + test((inbound, outbound, endpoint) -> { + ExecutorService executor = Executors.newFixedThreadPool(100); + int acquireStep = 123; + Assert.assertTrue(outbound.unsafeAcquireCapacity(100 * 10000, 100 * 10000 * acquireStep)); + AtomicLong acquisitionFailures = new AtomicLong(); + for (int i = 0; i < 100; i++) + { + executor.submit(() -> { + for (int j = 0; j < 10000; j++) + { + if (!outbound.unsafeAcquireCapacity(acquireStep)) + acquisitionFailures.incrementAndGet(); + } + + }); + } + + for (int i = 0; i < 100; i++) + { + executor.submit(() -> { + for (int j = 0; j < 10000; j++) + outbound.unsafeReleaseCapacity(acquireStep); + }); + } + + executor.shutdown(); + executor.awaitTermination(10, TimeUnit.SECONDS); + + // We can release more than we acquire, which certainly should not happen in + // real life, but since it's a test just for acquisition and release, it is fine + Assert.assertEquals(100 * 10000 * acquireStep - (acquisitionFailures.get() * acquireStep), outbound.pendingBytes()); + }); + } + + private void connect(OutboundConnection outbound) throws Throwable + { + CountDownLatch latch = new CountDownLatch(1); + unsafeSetHandler(Verb._TEST_1, () -> message -> latch.countDown()); + outbound.enqueue(Message.out(Verb._TEST_1, 0xffffffff)); + latch.await(10, SECONDS); + Assert.assertEquals(0, latch.getCount()); + Assert.assertTrue(outbound.isConnected()); + } + +} diff --git a/test/unit/org/apache/cassandra/net/ConnectionUtils.java b/test/unit/org/apache/cassandra/net/ConnectionUtils.java new file mode 100644 index 000000000000..e3917851d855 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/ConnectionUtils.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.Assert; + +import org.apache.cassandra.net.InboundMessageHandlers; +import org.apache.cassandra.net.OutboundConnection; + +public class ConnectionUtils +{ + public interface FailCheck + { + public void accept(String message, long expected, long actual); + } + + public static class OutboundCountChecker + { + private final OutboundConnection connection; + private long submitted; + private long pending, pendingBytes; + private long sent, sentBytes; + private long overload, overloadBytes; + private long expired, expiredBytes; + private long error, errorBytes; + private boolean checkSubmitted, checkPending, checkSent, checkOverload, checkExpired, checkError; + + private OutboundCountChecker(OutboundConnection connection) + { + this.connection = connection; + } + + public OutboundCountChecker submitted(long count) + { + submitted = count; + checkSubmitted = true; + return this; + } + + public OutboundCountChecker pending(long count, long bytes) + { + pending = count; + pendingBytes = bytes; + checkPending = true; + return this; + } + + public OutboundCountChecker sent(long count, long bytes) + { + sent = count; + sentBytes = bytes; + checkSent = true; + return this; + } + + public OutboundCountChecker overload(long count, long bytes) + { + overload = count; + overloadBytes = bytes; + checkOverload = true; + return this; + } + + public OutboundCountChecker expired(long count, long bytes) + { + expired = count; + expiredBytes = bytes; + checkExpired = true; + return this; + } + + public OutboundCountChecker error(long count, long bytes) + { + error = count; + errorBytes = bytes; + checkError = true; + return this; + } + + public void check() + { + doCheck(Assert::assertEquals); + } + + public void check(FailCheck failCheck) + { + doCheck((message, expect, actual) -> { if (expect != actual) failCheck.accept(message, expect, actual); }); + } + + private void doCheck(FailCheck testAndFailCheck) + { + if (checkSubmitted) + { + testAndFailCheck.accept("submitted count values don't match", submitted, connection.submittedCount()); + } + if (checkPending) + { + testAndFailCheck.accept("pending count values don't match", pending, connection.pendingCount()); + testAndFailCheck.accept("pending bytes values don't match", pendingBytes, connection.pendingBytes()); + } + if (checkSent) + { + testAndFailCheck.accept("sent count values don't match", sent, connection.sentCount()); + testAndFailCheck.accept("sent bytes values don't match", sentBytes, connection.sentBytes()); + } + if (checkOverload) + { + testAndFailCheck.accept("overload count values don't match", overload, connection.overloadedCount()); + testAndFailCheck.accept("overload bytes values don't match", overloadBytes, connection.overloadedBytes()); + } + if (checkExpired) + { + testAndFailCheck.accept("expired count values don't match", expired, connection.expiredCount()); + testAndFailCheck.accept("expired bytes values don't match", expiredBytes, connection.expiredBytes()); + } + if (checkError) + { + testAndFailCheck.accept("error count values don't match", error, connection.errorCount()); + testAndFailCheck.accept("error bytes values don't match", errorBytes, connection.errorBytes()); + } + } + } + + public static class InboundCountChecker + { + private final InboundMessageHandlers connection; + private long scheduled, scheduledBytes; + private long received, receivedBytes; + private long processed, processedBytes; + private long expired, expiredBytes; + private long error, errorBytes; + private boolean checkScheduled, checkReceived, checkProcessed, checkExpired, checkError; + + private InboundCountChecker(InboundMessageHandlers connection) + { + this.connection = connection; + } + + public InboundCountChecker pending(long count, long bytes) + { + scheduled = count; + scheduledBytes = bytes; + checkScheduled = true; + return this; + } + + public InboundCountChecker received(long count, long bytes) + { + received = count; + receivedBytes = bytes; + checkReceived = true; + return this; + } + + public InboundCountChecker processed(long count, long bytes) + { + processed = count; + processedBytes = bytes; + checkProcessed = true; + return this; + } + + public InboundCountChecker expired(long count, long bytes) + { + expired = count; + expiredBytes = bytes; + checkExpired = true; + return this; + } + + public InboundCountChecker error(long count, long bytes) + { + error = count; + errorBytes = bytes; + checkError = true; + return this; + } + + public void check() + { + doCheck(Assert::assertEquals); + } + + public void check(FailCheck failCheck) + { + doCheck((message, expect, actual) -> { if (expect != actual) failCheck.accept(message, expect, actual); }); + } + + private void doCheck(FailCheck testAndFailCheck) + { + if (checkReceived) + { + testAndFailCheck.accept("received count values don't match", received, connection.receivedCount()); + testAndFailCheck.accept("received bytes values don't match", receivedBytes, connection.receivedBytes()); + } + if (checkProcessed) + { + testAndFailCheck.accept("processed count values don't match", processed, connection.processedCount()); + testAndFailCheck.accept("processed bytes values don't match", processedBytes, connection.processedBytes()); + } + if (checkExpired) + { + testAndFailCheck.accept("expired count values don't match", expired, connection.expiredCount()); + testAndFailCheck.accept("expired bytes values don't match", expiredBytes, connection.expiredBytes()); + } + if (checkError) + { + testAndFailCheck.accept("error count values don't match", error, connection.errorCount()); + testAndFailCheck.accept("error bytes values don't match", errorBytes, connection.errorBytes()); + } + if (checkScheduled) + { + // scheduled cannot relied upon to not race with completion of the task, + // so if it is currently above the value we expect, sleep for a bit + if (scheduled < connection.scheduledCount()) + for (int i = 0; i < 10 && scheduled < connection.scheduledCount() ; ++i) + Uninterruptibles.sleepUninterruptibly(1L, TimeUnit.MILLISECONDS); + testAndFailCheck.accept("scheduled count values don't match", scheduled, connection.scheduledCount()); + testAndFailCheck.accept("scheduled bytes values don't match", scheduledBytes, connection.scheduledBytes()); + } + } + } + + public static OutboundCountChecker check(OutboundConnection outbound) + { + return new OutboundCountChecker(outbound); + } + + public static InboundCountChecker check(InboundMessageHandlers inbound) + { + return new InboundCountChecker(inbound); + } + +} diff --git a/test/unit/org/apache/cassandra/net/ForwardToContainerTest.java b/test/unit/org/apache/cassandra/net/ForwardingInfoTest.java similarity index 89% rename from test/unit/org/apache/cassandra/net/ForwardToContainerTest.java rename to test/unit/org/apache/cassandra/net/ForwardingInfoTest.java index 195d734c1bf9..16dec9f34639 100644 --- a/test/unit/org/apache/cassandra/net/ForwardToContainerTest.java +++ b/test/unit/org/apache/cassandra/net/ForwardingInfoTest.java @@ -33,7 +33,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class ForwardToContainerTest +public class ForwardingInfoTest { @Test public void testCurrent() throws Exception @@ -57,20 +57,20 @@ private void testVersion(int version) throws Exception InetAddressAndPort.getByName("2001:0db8:0000:0000:0000:ff00:0042:8329"), InetAddressAndPort.getByName("[2001:0db8:0000:0000:0000:ff00:0042:8329]:7000")); - ForwardToContainer ftc = new ForwardToContainer(addresses, new int[] { 44, 45, 46, 47, 48, 49 }); + ForwardingInfo ftc = new ForwardingInfo(addresses, new long[] { 44, 45, 46, 47, 48, 49 }); ByteBuffer buffer; try (DataOutputBuffer dob = new DataOutputBuffer()) { - ForwardToSerializer.instance.serialize(ftc, dob, version); + ForwardingInfo.serializer.serialize(ftc, dob, version); buffer = dob.buffer(); } - assertEquals(buffer.remaining(), ForwardToSerializer.instance.serializedSize(ftc, version)); + assertEquals(buffer.remaining(), ForwardingInfo.serializer.serializedSize(ftc, version)); - ForwardToContainer deserialized; + ForwardingInfo deserialized; try (DataInputBuffer dib = new DataInputBuffer(buffer, false)) { - deserialized = ForwardToSerializer.instance.deserialize(dib, version); + deserialized = ForwardingInfo.serializer.deserialize(dib, version); } assertTrue(Arrays.equals(ftc.messageIds, deserialized.messageIds)); diff --git a/test/unit/org/apache/cassandra/net/FramingTest.java b/test/unit/org/apache/cassandra/net/FramingTest.java new file mode 100644 index 000000000000..ec6cd84e994d --- /dev/null +++ b/test/unit/org/apache/cassandra/net/FramingTest.java @@ -0,0 +1,432 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.netty.buffer.ByteBuf; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.compress.BufferType; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.utils.memory.BufferPool; +import org.apache.cassandra.utils.vint.VIntCoding; + +import static java.lang.Math.*; +import static org.apache.cassandra.net.MessagingService.VERSION_30; +import static org.apache.cassandra.net.MessagingService.VERSION_3014; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.MessagingService.minimum_version; +import static org.apache.cassandra.net.OutboundConnections.LARGE_MESSAGE_THRESHOLD; +import static org.apache.cassandra.net.ShareableBytes.wrap; + +// TODO: test corruption +// TODO: use a different random seed each time +// TODO: use quick theories +public class FramingTest +{ + private static final Logger logger = LoggerFactory.getLogger(FramingTest.class); + + @BeforeClass + public static void begin() throws NoSuchFieldException, IllegalAccessException + { + DatabaseDescriptor.daemonInitialization(); + Verb._TEST_1.unsafeSetSerializer(() -> new IVersionedSerializer() + { + + public void serialize(byte[] t, DataOutputPlus out, int version) throws IOException + { + out.writeUnsignedVInt(t.length); + out.write(t); + } + + public byte[] deserialize(DataInputPlus in, int version) throws IOException + { + byte[] r = new byte[(int) in.readUnsignedVInt()]; + in.readFully(r); + return r; + } + + public long serializedSize(byte[] t, int version) + { + return VIntCoding.computeUnsignedVIntSize(t.length) + t.length; + } + }); + } + + @AfterClass + public static void after() throws NoSuchFieldException, IllegalAccessException + { + Verb._TEST_1.unsafeSetSerializer(() -> null); + } + + private static class SequenceOfFrames + { + final List original; + final int[] boundaries; + final ShareableBytes frames; + + private SequenceOfFrames(List original, int[] boundaries, ByteBuffer frames) + { + this.original = original; + this.boundaries = boundaries; + this.frames = wrap(frames); + } + } + + @Test + public void testRandomLZ4() + { + testSomeFrames(FrameEncoderLZ4.fastInstance, FrameDecoderLZ4.fast(GlobalBufferPoolAllocator.instance)); + } + + @Test + public void testRandomCrc() + { + testSomeFrames(FrameEncoderCrc.instance, FrameDecoderCrc.create(GlobalBufferPoolAllocator.instance)); + } + + public void testSomeFrames(FrameEncoder encoder, FrameDecoder decoder) + { + long seed = new SecureRandom().nextLong(); + logger.info("seed: {}, decoder: {}", seed, decoder.getClass().getSimpleName()); + Random random = new Random(seed); + for (int i = 0 ; i < 1000 ; ++i) + testRandomSequenceOfFrames(random, encoder, decoder); + } + + private void testRandomSequenceOfFrames(Random random, FrameEncoder encoder, FrameDecoder decoder) + { + SequenceOfFrames sequenceOfFrames = sequenceOfFrames(random, encoder); + + List uncompressed = sequenceOfFrames.original; + ShareableBytes frames = sequenceOfFrames.frames; + int[] boundaries = sequenceOfFrames.boundaries; + + int end = frames.get().limit(); + List out = new ArrayList<>(); + int prevBoundary = -1; + for (int i = 0 ; i < end ; ) + { + int limit = i + random.nextInt(1 + end - i); + decoder.decode(out, frames.slice(i, limit)); + int boundary = Arrays.binarySearch(boundaries, limit); + if (boundary < 0) boundary = -2 -boundary; + + while (prevBoundary < boundary) + { + ++prevBoundary; + Assert.assertTrue(out.size() >= 1 + prevBoundary); + verify(uncompressed.get(prevBoundary), ((FrameDecoder.IntactFrame) out.get(prevBoundary)).contents); + } + i = limit; + } + for (FrameDecoder.Frame frame : out) + frame.release(); + frames.release(); + Assert.assertEquals(null, decoder.stash); + Assert.assertTrue(decoder.frames.isEmpty()); + } + + private static void verify(byte[] expect, ShareableBytes actual) + { + verify(expect, 0, expect.length, actual); + } + + private static void verify(byte[] expect, int start, int end, ShareableBytes actual) + { + byte[] fetch = new byte[end - start]; + Assert.assertEquals(end - start, actual.remaining()); + actual.get().get(fetch); + boolean equals = true; + for (int i = start ; equals && i < end ; ++i) + equals = expect[i] == fetch[i - start]; + if (!equals) + Assert.assertArrayEquals(Arrays.copyOfRange(expect, start, end), fetch); + } + + private static SequenceOfFrames sequenceOfFrames(Random random, FrameEncoder encoder) + { + int frameCount = 1 + random.nextInt(8); + List uncompressed = new ArrayList<>(); + List compressed = new ArrayList<>(); + int[] cumulativeCompressedLength = new int[frameCount]; + for (int i = 0 ; i < frameCount ; ++i) + { + byte[] bytes = randomishBytes(random, 1, 1 << 15); + uncompressed.add(bytes); + + FrameEncoder.Payload payload = encoder.allocator().allocate(true, bytes.length); + payload.buffer.put(bytes); + payload.finish(); + + ByteBuf buffer = encoder.encode(true, payload.buffer); + compressed.add(buffer); + cumulativeCompressedLength[i] = (i == 0 ? 0 : cumulativeCompressedLength[i - 1]) + buffer.readableBytes(); + } + + ByteBuffer frames = BufferPool.getAtLeast(cumulativeCompressedLength[frameCount - 1], BufferType.OFF_HEAP); + for (ByteBuf buffer : compressed) + { + frames.put(buffer.internalNioBuffer(buffer.readerIndex(), buffer.readableBytes())); + buffer.release(); + } + frames.flip(); + return new SequenceOfFrames(uncompressed, cumulativeCompressedLength, frames); + } + + @Test + public void burnRandomLegacy() + { + burnRandomLegacy(1000); + } + + public void burnRandomLegacy(int count) + { + SecureRandom seed = new SecureRandom(); + Random random = new Random(); + for (int i = 0 ; i < count ; ++i) + { + long innerSeed = seed.nextLong(); + float ratio = seed.nextFloat(); + int version = minimum_version + random.nextInt(1 + current_version - minimum_version); + logger.debug("seed: {}, ratio: {}, version: {}", innerSeed, ratio, version); + random.setSeed(innerSeed); + testRandomSequenceOfMessages(random, ratio, version, new FrameDecoderLegacy(GlobalBufferPoolAllocator.instance, version)); + } + } + + @Test + public void testRandomLegacy() + { + testRandomLegacy(250); + } + + public void testRandomLegacy(int count) + { + SecureRandom seeds = new SecureRandom(); + for (int messagingVersion : new int[] { VERSION_30, VERSION_3014, current_version}) + { + FrameDecoder decoder = new FrameDecoderLegacy(GlobalBufferPoolAllocator.instance, messagingVersion); + testSomeMessages(seeds.nextLong(), count, 0.0f, messagingVersion, decoder); + testSomeMessages(seeds.nextLong(), count, 0.1f, messagingVersion, decoder); + testSomeMessages(seeds.nextLong(), count, 0.95f, messagingVersion, decoder); + testSomeMessages(seeds.nextLong(), count, 1.0f, messagingVersion, decoder); + } + } + + public void testSomeMessages(long seed, int count, float largeRatio, int messagingVersion, FrameDecoder decoder) + { + logger.info("seed: {}, iterations: {}, largeRatio: {}, messagingVersion: {}, decoder: {}", seed, count, largeRatio, messagingVersion, decoder.getClass().getSimpleName()); + Random random = new Random(seed); + for (int i = 0 ; i < count ; ++i) + { + long innerSeed = random.nextLong(); + logger.debug("inner seed: {}, iteration: {}", innerSeed, i); + random.setSeed(innerSeed); + testRandomSequenceOfMessages(random, largeRatio, messagingVersion, decoder); + } + } + + private void testRandomSequenceOfMessages(Random random, float largeRatio, int messagingVersion, FrameDecoder decoder) + { + SequenceOfFrames sequenceOfMessages = sequenceOfMessages(random, largeRatio, messagingVersion); + + List messages = sequenceOfMessages.original; + ShareableBytes stream = sequenceOfMessages.frames; + + int end = stream.get().limit(); + List out = new ArrayList<>(); + + int messageStart = 0; + int messageIndex = 0; + for (int i = 0 ; i < end ; ) + { + int limit = i + random.nextInt(1 + end - i); + decoder.decode(out, stream.slice(i, limit)); + + int outIndex = 0; + byte[] message = messages.get(messageIndex); + if (i > messageStart) + { + int start; + if (message.length <= LARGE_MESSAGE_THRESHOLD) + { + start = 0; + } + else if (!lengthIsReadable(message, i - messageStart, messagingVersion)) + { + // we should have an initial frame containing only some prefix of the message (probably 64 bytes) + // that was stashed only to decide how big the message was + FrameDecoder.IntactFrame frame = (FrameDecoder.IntactFrame) out.get(outIndex++); + Assert.assertEquals(false, frame.isSelfContained); + start = frame.contents.remaining(); + verify(message, 0, frame.contents.remaining(), frame.contents); + } + else + { + start = i - messageStart; + } + + if (limit >= message.length + messageStart) + { + FrameDecoder.IntactFrame frame = (FrameDecoder.IntactFrame) out.get(outIndex++); + Assert.assertEquals(start == 0, frame.isSelfContained); + // verify remainder of a large message, or a single fully stashed small message + verify(message, start, message.length, frame.contents); + + messageStart += message.length; + if (++messageIndex < messages.size()) + message = messages.get(messageIndex); + } + else if (message.length > LARGE_MESSAGE_THRESHOLD) + { + FrameDecoder.IntactFrame frame = (FrameDecoder.IntactFrame) out.get(outIndex++); + Assert.assertFalse(frame.isSelfContained); + // verify next portion of a large message + verify(message, start, limit - messageStart, frame.contents); + + Assert.assertEquals(outIndex, out.size()); + for (FrameDecoder.Frame f : out) + f.release(); + out.clear(); + i = limit; + continue; + } + } + + // message is fresh + int beginFrame = messageStart; + int beginFrameIndex = messageIndex; + while (messageStart + message.length <= limit) + { + messageStart += message.length; + if (++messageIndex < messages.size()) + message = messages.get(messageIndex); + } + + if (beginFrameIndex < messageIndex) + { + FrameDecoder.IntactFrame frame = (FrameDecoder.IntactFrame) out.get(outIndex++); + Assert.assertTrue(frame.isSelfContained); + while (beginFrameIndex < messageIndex) + { + byte[] m = messages.get(beginFrameIndex); + verify(m, frame.contents.sliceAndConsume(m.length)); + ++beginFrameIndex; + } + Assert.assertFalse(frame.contents.hasRemaining()); + } + + if (limit > messageStart + && message.length > LARGE_MESSAGE_THRESHOLD + && lengthIsReadable(message, limit - messageStart, messagingVersion)) + { + FrameDecoder.IntactFrame frame = (FrameDecoder.IntactFrame) out.get(outIndex++); + Assert.assertFalse(frame.isSelfContained); + verify(message, 0, limit - messageStart, frame.contents); + } + + Assert.assertEquals(outIndex, out.size()); + for (FrameDecoder.Frame frame : out) + frame.release(); + out.clear(); + + i = limit; + } + stream.release(); + Assert.assertEquals(null, decoder.stash); + Assert.assertTrue(decoder.frames.isEmpty()); + } + + private static boolean lengthIsReadable(byte[] message, int limit, int messagingVersion) + { + try + { + return Message.serializer.inferMessageSize(ByteBuffer.wrap(message), 0, limit, messagingVersion) >= 0; + } + catch (Message.InvalidLegacyProtocolMagic e) + { + throw new IllegalStateException(e); + } + } + + private static SequenceOfFrames sequenceOfMessages(Random random, float largeRatio, int messagingVersion) + { + int messageCount = 1 + random.nextInt(63); + List messages = new ArrayList<>(); + int[] cumulativeLength = new int[messageCount]; + for (int i = 0 ; i < messageCount ; ++i) + { + byte[] payload; + if (random.nextFloat() < largeRatio) payload = randomishBytes(random, 1 << 16, 1 << 17); + else payload = randomishBytes(random, 1, 1 << 16); + Message messageObj = Message.out(Verb._TEST_1, payload); + + byte[] message; + try (DataOutputBuffer out = new DataOutputBuffer(messageObj.serializedSize(messagingVersion))) + { + Message.serializer.serialize(messageObj, out, messagingVersion); + message = out.toByteArray(); + } + catch (IOException e) + { + throw new IllegalStateException(e); + } + messages.add(message); + + cumulativeLength[i] = (i == 0 ? 0 : cumulativeLength[i - 1]) + message.length; + } + + ByteBuffer frames = BufferPool.getAtLeast(cumulativeLength[messageCount - 1], BufferType.OFF_HEAP); + for (byte[] buffer : messages) + frames.put(buffer); + frames.flip(); + return new SequenceOfFrames(messages, cumulativeLength, frames); + } + + private static byte[] randomishBytes(Random random, int minLength, int maxLength) + { + byte[] bytes = new byte[minLength + random.nextInt(maxLength - minLength)]; + int runLength = 1 + random.nextInt(255); + for (int i = 0 ; i < bytes.length ; i += runLength) + { + byte b = (byte) random.nextInt(256); + Arrays.fill(bytes, i, min(bytes.length, i + runLength), b); + } + return bytes; + } + +} diff --git a/test/unit/org/apache/cassandra/net/HandshakeTest.java b/test/unit/org/apache/cassandra/net/HandshakeTest.java new file mode 100644 index 000000000000..c9d4e8715b33 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/HandshakeTest.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.nio.channels.ClosedChannelException; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import io.netty.channel.EventLoop; +import io.netty.util.concurrent.Future; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.OutboundConnectionInitiator.Result; +import org.apache.cassandra.net.OutboundConnectionInitiator.Result.MessagingSuccess; + +import static org.apache.cassandra.net.MessagingService.VERSION_30; +import static org.apache.cassandra.net.MessagingService.VERSION_3014; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.MessagingService.minimum_version; +import static org.apache.cassandra.net.ConnectionType.SMALL_MESSAGES; +import static org.apache.cassandra.net.OutboundConnectionInitiator.*; + +// TODO: test failure due to exception, timeout, etc +public class HandshakeTest +{ + private static final SocketFactory factory = new SocketFactory(); + + @BeforeClass + public static void startup() + { + DatabaseDescriptor.daemonInitialization(); + } + + @AfterClass + public static void cleanup() throws InterruptedException + { + factory.shutdownNow(); + } + + private Result handshake(int req, int outMin, int outMax) throws ExecutionException, InterruptedException + { + return handshake(req, new AcceptVersions(outMin, outMax), null); + } + private Result handshake(int req, int outMin, int outMax, int inMin, int inMax) throws ExecutionException, InterruptedException + { + return handshake(req, new AcceptVersions(outMin, outMax), new AcceptVersions(inMin, inMax)); + } + private Result handshake(int req, AcceptVersions acceptOutbound, AcceptVersions acceptInbound) throws ExecutionException, InterruptedException + { + InboundSockets inbound = new InboundSockets(new InboundConnectionSettings().withAcceptMessaging(acceptInbound)); + try + { + inbound.open(); + InetAddressAndPort endpoint = inbound.sockets().stream().map(s -> s.settings.bindAddress).findFirst().get(); + EventLoop eventLoop = factory.defaultGroup().next(); + Future> future = + initiateMessaging(eventLoop, + SMALL_MESSAGES, + new OutboundConnectionSettings(endpoint) + .withAcceptVersions(acceptOutbound) + .withDefaults(ConnectionCategory.MESSAGING), + req, new AsyncPromise<>(eventLoop)); + return future.get(); + } + finally + { + inbound.close().await(1L, TimeUnit.SECONDS); + } + } + + @Test + public void testBothCurrentVersion() throws InterruptedException, ExecutionException + { + Result result = handshake(current_version, minimum_version, current_version); + Assert.assertEquals(Result.Outcome.SUCCESS, result.outcome); + result.success().channel.close(); + } + + @Test + public void testSendCompatibleOldVersion() throws InterruptedException, ExecutionException + { + Result result = handshake(current_version, current_version, current_version + 1, current_version +1, current_version + 2); + Assert.assertEquals(Result.Outcome.SUCCESS, result.outcome); + Assert.assertEquals(current_version + 1, result.success().messagingVersion); + result.success().channel.close(); + } + + @Test + public void testSendCompatibleFutureVersion() throws InterruptedException, ExecutionException + { + Result result = handshake(current_version + 1, current_version - 1, current_version + 1); + Assert.assertEquals(Result.Outcome.SUCCESS, result.outcome); + Assert.assertEquals(current_version, result.success().messagingVersion); + result.success().channel.close(); + } + + @Test + public void testSendIncompatibleFutureVersion() throws InterruptedException, ExecutionException + { + Result result = handshake(current_version + 1, current_version + 1, current_version + 1); + Assert.assertEquals(Result.Outcome.INCOMPATIBLE, result.outcome); + Assert.assertEquals(current_version, result.incompatible().closestSupportedVersion); + Assert.assertEquals(current_version, result.incompatible().maxMessagingVersion); + } + + @Test + public void testSendIncompatibleOldVersion() throws InterruptedException, ExecutionException + { + Result result = handshake(current_version + 1, current_version + 1, current_version + 1, current_version + 2, current_version + 3); + Assert.assertEquals(Result.Outcome.INCOMPATIBLE, result.outcome); + Assert.assertEquals(current_version + 2, result.incompatible().closestSupportedVersion); + Assert.assertEquals(current_version + 3, result.incompatible().maxMessagingVersion); + } + + @Test + public void testSendCompatibleMaxVersionPre40() throws InterruptedException, ExecutionException + { + Result result = handshake(VERSION_3014, VERSION_30, VERSION_3014, VERSION_30, VERSION_3014); + Assert.assertEquals(Result.Outcome.SUCCESS, result.outcome); + Assert.assertEquals(VERSION_3014, result.success().messagingVersion); + result.success().channel.close(); + } + + @Test + public void testSendCompatibleFutureVersionPre40() throws InterruptedException, ExecutionException + { + Result result = handshake(VERSION_3014, VERSION_30, VERSION_3014, VERSION_30, VERSION_30); + Assert.assertEquals(Result.Outcome.RETRY, result.outcome); + Assert.assertEquals(VERSION_30, result.retry().withMessagingVersion); + } + + @Test + public void testSendIncompatibleFutureVersionPre40() throws InterruptedException, ExecutionException + { + Result result = handshake(VERSION_3014, VERSION_3014, VERSION_3014, VERSION_30, VERSION_30); + Assert.assertEquals(Result.Outcome.INCOMPATIBLE, result.outcome); + Assert.assertEquals(-1, result.incompatible().closestSupportedVersion); + Assert.assertEquals(VERSION_30, result.incompatible().maxMessagingVersion); + } + + @Test + public void testSendCompatibleOldVersionPre40() throws InterruptedException + { + try + { + handshake(VERSION_30, VERSION_30, VERSION_3014, VERSION_3014, VERSION_3014); + Assert.fail("Should have thrown"); + } + catch (ExecutionException e) + { + Assert.assertTrue(e.getCause() instanceof ClosedChannelException); + } + } + + @Test + public void testSendIncompatibleOldVersionPre40() throws InterruptedException + { + try + { + handshake(VERSION_30, VERSION_30, VERSION_30, VERSION_3014, VERSION_3014); + Assert.fail("Should have thrown"); + } + catch (ExecutionException e) + { + Assert.assertTrue(e.getCause() instanceof ClosedChannelException); + } + } + + @Test + public void testSendCompatibleOldVersion40() throws InterruptedException, ExecutionException + { + Result result = handshake(VERSION_30, VERSION_30, VERSION_30, VERSION_30, current_version); + Assert.assertEquals(Result.Outcome.SUCCESS, result.outcome); + Assert.assertEquals(VERSION_30, result.success().messagingVersion); + } + + @Test + public void testSendIncompatibleOldVersion40() throws InterruptedException + { + try + { + Assert.fail(Objects.toString(handshake(VERSION_30, VERSION_30, VERSION_30, current_version, current_version))); + } + catch (ExecutionException e) + { + Assert.assertTrue(e.getCause() instanceof ClosedChannelException); + } + } + + @Test // fairly contrived case, but since we introduced logic for testing we need to be careful it doesn't make us worse + public void testSendToFuturePost40BelievedToBePre40() throws InterruptedException, ExecutionException + { + Result result = handshake(VERSION_30, VERSION_30, current_version, VERSION_30, current_version + 1); + Assert.assertEquals(Result.Outcome.SUCCESS, result.outcome); + Assert.assertEquals(VERSION_30, result.success().messagingVersion); + } +} diff --git a/test/unit/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueueTest.java b/test/unit/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueueTest.java new file mode 100644 index 000000000000..2c92a392e09f --- /dev/null +++ b/test/unit/org/apache/cassandra/net/ManyToOneConcurrentLinkedQueueTest.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.BitSet; +import java.util.NoSuchElementException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; + +import org.junit.Test; + +import static org.junit.Assert.*; + +@SuppressWarnings("ConstantConditions") +public class ManyToOneConcurrentLinkedQueueTest +{ + private final ManyToOneConcurrentLinkedQueue queue = new ManyToOneConcurrentLinkedQueue<>(); + + @Test + public void testRelaxedIsEmptyWhenEmpty() + { + assertTrue(queue.relaxedIsEmpty()); + } + + @Test + public void testRelaxedIsEmptyWhenNotEmpty() + { + queue.offer(0); + assertFalse(queue.relaxedIsEmpty()); + } + + @Test + public void testSizeWhenEmpty() + { + assertEquals(0, queue.size()); + } + + @Test + public void testSizeWhenNotEmpty() + { + queue.offer(0); + assertEquals(1, queue.size()); + + for (int i = 1; i < 100; i++) + queue.offer(i); + assertEquals(100, queue.size()); + } + + @Test + public void testEmptyPeek() + { + assertNull(queue.peek()); + } + + @Test + public void testNonEmptyPeek() + { + queue.offer(0); + assertEquals(0, (int) queue.peek()); + } + + @Test + public void testEmptyPoll() + { + assertNull(queue.poll()); + } + + @Test + public void testNonEmptyPoll() + { + queue.offer(0); + assertEquals(0, (int) queue.poll()); + } + + @Test(expected = NoSuchElementException.class) + public void testEmptyRemove() + { + queue.remove(); + } + + @Test + public void testNonEmptyRemove() + { + queue.offer(0); + assertEquals(0, (int) queue.remove()); + } + + @Test + public void testOtherRemoveWhenEmpty() + { + assertFalse(queue.remove(0)); + } + + @Test + public void testOtherRemoveSingleNode() + { + queue.offer(0); + assertTrue(queue.remove(0)); + assertTrue(queue.isEmpty()); + } + + @Test + public void testOtherRemoveWhenFirst() + { + queue.offer(0); + queue.offer(1); + queue.offer(2); + + assertTrue(queue.remove(0)); + + assertEquals(1, (int) queue.poll()); + assertEquals(2, (int) queue.poll()); + assertNull(queue.poll()); + } + + @Test + public void testOtherRemoveFromMiddle() + { + queue.offer(0); + queue.offer(1); + queue.offer(2); + + assertTrue(queue.remove(1)); + + assertEquals(0, (int) queue.poll()); + assertEquals(2, (int) queue.poll()); + assertNull(queue.poll()); + } + + @Test + public void testOtherRemoveFromEnd() + { + queue.offer(0); + queue.offer(1); + queue.offer(2); + + assertTrue(queue.remove(2)); + + assertEquals(0, (int) queue.poll()); + assertEquals(1, (int) queue.poll()); + assertNull(queue.poll()); + } + + @Test + public void testOtherRemoveWhenDoesnNotExist() + { + queue.offer(0); + queue.offer(1); + queue.offer(2); + + assertFalse(queue.remove(3)); + + assertEquals(0, (int) queue.poll()); + assertEquals(1, (int) queue.poll()); + assertEquals(2, (int) queue.poll()); + } + + @Test + public void testTransfersInCorrectOrder() + { + for (int i = 0; i < 1024; i++) + queue.offer(i); + + for (int i = 0; i < 1024; i++) + assertEquals(i, (int) queue.poll()); + + assertTrue(queue.relaxedIsEmpty()); + } + + @Test + public void testTransfersInCorrectOrderWhenInterleaved() + { + for (int i = 0; i < 1024; i++) + { + queue.offer(i); + assertEquals(i, (int) queue.poll()); + } + + assertTrue(queue.relaxedIsEmpty()); + } + + @Test + public void testDrain() + { + for (int i = 0; i < 1024; i++) + queue.offer(i); + + class Consumer + { + private int previous = -1; + + public void accept(int i) + { + assertEquals(++previous, i); + } + } + + Consumer consumer = new Consumer(); + queue.drain(consumer::accept); + + assertEquals(1023, consumer.previous); + assertTrue(queue.relaxedIsEmpty()); + } + + @Test + public void testPeekLastAndOffer() + { + assertNull(queue.relaxedPeekLastAndOffer(0)); + for (int i = 1; i < 1024; i++) + assertEquals(i - 1, (int) queue.relaxedPeekLastAndOffer(i)); + + for (int i = 0; i < 1024; i++) + assertEquals(i, (int) queue.poll()); + + assertTrue(queue.relaxedIsEmpty()); + } + + enum Strategy + { + PEEK_AND_REMOVE, POLL + } + + @Test + public void testConcurrentlyWithPoll() + { + testConcurrently(Strategy.POLL); + } + + @Test + public void testConcurrentlyWithPeekAndRemove() + { + testConcurrently(Strategy.PEEK_AND_REMOVE); + } + + private void testConcurrently(Strategy strategy) + { + int numThreads = 4; + int numItems = 1_000_000 * numThreads; + + class Producer implements Runnable + { + private final int start, step, limit; + + private Producer(int start, int step, int limit) + { + this.start = start; + this.step = step; + this.limit = limit; + } + + public void run() + { + for (int i = start; i < limit; i += step) + queue.offer(i); + } + } + + Executor executor = Executors.newFixedThreadPool(numThreads); + for (int i = 0; i < numThreads; i++) + executor.execute(new Producer(i, numThreads, numItems)); + + BitSet itemsPolled = new BitSet(numItems); + for (int i = 0; i < numItems; i++) + { + Integer item; + switch (strategy) + { + case PEEK_AND_REMOVE: + //noinspection StatementWithEmptyBody + while ((item = queue.peek()) == null) ; + assertFalse(queue.relaxedIsEmpty()); + assertEquals(item, queue.remove()); + itemsPolled.set(item); + break; + case POLL: + //noinspection StatementWithEmptyBody + while ((item = queue.poll()) == null) ; + itemsPolled.set(item); + break; + } + } + + assertEquals(numItems, itemsPolled.cardinality()); + assertTrue(queue.relaxedIsEmpty()); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/net/Matcher.java b/test/unit/org/apache/cassandra/net/Matcher.java index 27b685f6bb26..6f8e1e7aee69 100644 --- a/test/unit/org/apache/cassandra/net/Matcher.java +++ b/test/unit/org/apache/cassandra/net/Matcher.java @@ -28,5 +28,5 @@ public interface Matcher * @param obj intercepted outgoing message * @param to destination address */ - public boolean matches(MessageOut obj, InetAddressAndPort to); + public boolean matches(Message obj, InetAddressAndPort to); } diff --git a/test/unit/org/apache/cassandra/net/MatcherResponse.java b/test/unit/org/apache/cassandra/net/MatcherResponse.java index 7a1772aec550..b2bba8b0a06c 100644 --- a/test/unit/org/apache/cassandra/net/MatcherResponse.java +++ b/test/unit/org/apache/cassandra/net/MatcherResponse.java @@ -17,15 +17,20 @@ */ package org.apache.cassandra.net; -import java.util.Collections; -import java.util.HashSet; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Queue; -import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.BiPredicate; import java.util.function.Function; +import com.google.common.collect.Multimap; +import com.google.common.collect.Multimaps; + +import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.locator.InetAddressAndPort; /** @@ -36,10 +41,11 @@ public class MatcherResponse { private final Matcher matcher; - private final Set sendResponses = new HashSet<>(); + private final Multimap sendResponses = + Multimaps.newListMultimap(new HashMap<>(), ArrayList::new); private final MockMessagingSpy spy = new MockMessagingSpy(); private final AtomicInteger limitCounter = new AtomicInteger(Integer.MAX_VALUE); - private IMessageSink sink; + private BiPredicate, InetAddressAndPort> sink; MatcherResponse(Matcher matcher) { @@ -51,33 +57,33 @@ public class MatcherResponse */ public MockMessagingSpy dontReply() { - return respond((MessageIn)null); + return respond((Message)null); } /** - * Respond with provided message in reply to each intercepted outbound message. - * @param message the message to use as mock reply from the cluster + * Respond with provided message in response to each intercepted outbound message. + * @param message the message to use as mock response from the cluster */ - public MockMessagingSpy respond(MessageIn message) + public MockMessagingSpy respond(Message message) { return respondN(message, Integer.MAX_VALUE); } /** - * Respond a limited number of times with the provided message in reply to each intercepted outbound message. - * @param response the message to use as mock reply from the cluster + * Respond a limited number of times with the provided message in response to each intercepted outbound message. + * @param response the message to use as mock response from the cluster * @param limit number of times to respond with message */ - public MockMessagingSpy respondN(final MessageIn response, int limit) + public MockMessagingSpy respondN(final Message response, int limit) { return respondN((in, to) -> response, limit); } /** * Respond with the message created by the provided function that will be called with each intercepted outbound message. - * @param fnResponse function to call for creating reply based on intercepted message and target address + * @param fnResponse function to call for creating response based on intercepted message and target address */ - public MockMessagingSpy respond(BiFunction, InetAddressAndPort, MessageIn> fnResponse) + public MockMessagingSpy respond(BiFunction, InetAddressAndPort, Message> fnResponse) { return respondN(fnResponse, Integer.MAX_VALUE); } @@ -86,9 +92,9 @@ public MockMessagingSpy respond(BiFunction, InetAddressAndP * Respond with message wrapping the payload object created by provided function called for each intercepted outbound message. * The target address from the intercepted message will automatically be used as the created message's sender address. * @param fnResponse function to call for creating payload object based on intercepted message and target address - * @param verb verb to use for reply message + * @param verb verb to use for response message */ - public MockMessagingSpy respondWithPayloadForEachReceiver(Function, S> fnResponse, MessagingService.Verb verb) + public MockMessagingSpy respondWithPayloadForEachReceiver(Function, S> fnResponse, Verb verb) { return respondNWithPayloadForEachReceiver(fnResponse, verb, Integer.MAX_VALUE); } @@ -98,40 +104,40 @@ public MockMessagingSpy respondWithPayloadForEachReceiver(Function MockMessagingSpy respondNWithPayloadForEachReceiver(Function, S> fnResponse, MessagingService.Verb verb, int limit) + public MockMessagingSpy respondNWithPayloadForEachReceiver(Function, S> fnResponse, Verb verb, int limit) { - return respondN((MessageOut msg, InetAddressAndPort to) -> { + return respondN((Message msg, InetAddressAndPort to) -> { S payload = fnResponse.apply(msg); if (payload == null) return null; else - return MessageIn.create(to, payload, Collections.emptyMap(), verb, MessagingService.current_version); + return Message.builder(verb, payload).from(to).build(); }, limit); } /** * Responds to each intercepted outbound message by creating a response message wrapping the next element consumed - * from the provided queue. No reply will be send when the queue has been exhausted. + * from the provided queue. No response will be send when the queue has been exhausted. * @param cannedResponses prepared payload messages to use for responses - * @param verb verb to use for reply message + * @param verb verb to use for response message */ - public MockMessagingSpy respondWithPayloadForEachReceiver(Queue cannedResponses, MessagingService.Verb verb) + public MockMessagingSpy respondWithPayloadForEachReceiver(Queue cannedResponses, Verb verb) { - return respondWithPayloadForEachReceiver((MessageOut msg) -> cannedResponses.poll(), verb); + return respondWithPayloadForEachReceiver((Message msg) -> cannedResponses.poll(), verb); } /** * Responds to each intercepted outbound message by creating a response message wrapping the next element consumed * from the provided queue. This method will block until queue elements are available. * @param cannedResponses prepared payload messages to use for responses - * @param verb verb to use for reply message + * @param verb verb to use for response message */ - public MockMessagingSpy respondWithPayloadForEachReceiver(BlockingQueue cannedResponses, MessagingService.Verb verb) + public MockMessagingSpy respondWithPayloadForEachReceiver(BlockingQueue cannedResponses, Verb verb) { - return respondWithPayloadForEachReceiver((MessageOut msg) -> { + return respondWithPayloadForEachReceiver((Message msg) -> { try { return cannedResponses.take(); @@ -146,17 +152,17 @@ public MockMessagingSpy respondWithPayloadForEachReceiver(BlockingQueue MockMessagingSpy respondN(BiFunction, InetAddressAndPort, MessageIn> fnResponse, int limit) + public MockMessagingSpy respondN(BiFunction, InetAddressAndPort, Message> fnResponse, int limit) { limitCounter.set(limit); assert sink == null: "destroy() must be called first to register new response"; - sink = new IMessageSink() + sink = new BiPredicate, InetAddressAndPort>() { - public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to) + public boolean test(Message message, InetAddressAndPort to) { // prevent outgoing message from being send in case matcher indicates a match // and instead send the mocked response @@ -169,23 +175,25 @@ public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPo synchronized (sendResponses) { - // I'm not sure about retry semantics regarding message/ID relationships, but I assume - // sending a message multiple times using the same ID shouldn't happen.. - assert !sendResponses.contains(id) : "ID re-use for outgoing message"; - sendResponses.add(id); + if (message.hasId()) + { + assert !sendResponses.get(message.id()).contains(to) : "ID re-use for outgoing message"; + sendResponses.put(message.id(), to); + } } // create response asynchronously to match request/response communication execution behavior new Thread(() -> { - MessageIn response = fnResponse.apply(message, to); + Message response = fnResponse.apply(message, to); if (response != null) { - CallbackInfo cb = MessagingService.instance().getRegisteredCallback(id); + RequestCallbacks.CallbackInfo cb = MessagingService.instance().callbacks.get(message.id(), to); if (cb != null) - cb.callback.response(response); + cb.callback.onResponse(response); else - MessagingService.instance().receive(response, id); + processResponse(response); + spy.matchingResponse(response); } }).start(); @@ -194,22 +202,34 @@ public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPo } return true; } - - public boolean allowIncomingMessage(MessageIn message, int id) - { - return true; - } }; - MessagingService.instance().addMessageSink(sink); + MessagingService.instance().outboundSink.add(sink); return spy; } + private void processResponse(Message message) + { + if (!MessagingService.instance().inboundSink.allow(message)) + return; + + StageManager.getStage(message.verb().stage).execute(() -> { + try + { + message.verb().handler().doVerb((Message)message); + } + catch (IOException e) + { + // + } + }); + } + /** * Stops currently registered response from being send. */ public void destroy() { - MessagingService.instance().removeMessageSink(sink); + MessagingService.instance().outboundSink.remove(sink); } } diff --git a/test/unit/org/apache/cassandra/net/MessageDeliveryTaskTest.java b/test/unit/org/apache/cassandra/net/MessageDeliveryTaskTest.java deleted file mode 100644 index db38efb7234d..000000000000 --- a/test/unit/org/apache/cassandra/net/MessageDeliveryTaskTest.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net; - -import java.net.UnknownHostException; -import java.util.Collections; - -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.InetAddressAndPort; - -public class MessageDeliveryTaskTest -{ - private static final MockVerbHandler VERB_HANDLER = new MockVerbHandler(); - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - MessagingService.instance().registerVerbHandlers(MessagingService.Verb.UNUSED_2, VERB_HANDLER); - } - - @AfterClass - public static void after() - { - MessagingService.instance().removeVerbHandler(MessagingService.Verb.UNUSED_2); - } - - @Before - public void setUp() - { - VERB_HANDLER.reset(); - } - - @Test - public void process_HappyPath() throws UnknownHostException - { - InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.0.1"); - MessageIn msg = MessageIn.create(addr, null, Collections.emptyMap(), MessagingService.Verb.UNUSED_2, 1); - MessageDeliveryTask task = new MessageDeliveryTask(msg, 42); - Assert.assertTrue(task.process()); - Assert.assertEquals(1, VERB_HANDLER.invocationCount); - } - - @Test - public void process_NullVerb() throws UnknownHostException - { - InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.0.1"); - MessageIn msg = MessageIn.create(addr, null, Collections.emptyMap(), null, 1); - MessageDeliveryTask task = new MessageDeliveryTask(msg, 42); - Assert.assertFalse(task.process()); - } - - @Test - public void process_NoHandler() throws UnknownHostException - { - InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.0.1"); - MessageIn msg = MessageIn.create(addr, null, Collections.emptyMap(), MessagingService.Verb.UNUSED_5, 1); - MessageDeliveryTask task = new MessageDeliveryTask(msg, 42); - Assert.assertFalse(task.process()); - } - - @Test - public void process_ExpiredDroppableMessage() throws UnknownHostException - { - InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.0.1"); - - // we need any droppable verb, so just grab it from the enum itself rather than hard code a value - MessageIn msg = MessageIn.create(addr, null, Collections.emptyMap(), MessagingService.DROPPABLE_VERBS.iterator().next(), 1, 0); - MessageDeliveryTask task = new MessageDeliveryTask(msg, 42); - Assert.assertFalse(task.process()); - } - - // non-droppable message should still be processed even if they are expired - @Test - public void process_ExpiredMessage() throws UnknownHostException - { - InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.0.1"); - MessageIn msg = MessageIn.create(addr, null, Collections.emptyMap(), MessagingService.Verb.UNUSED_2, 1, 0); - MessageDeliveryTask task = new MessageDeliveryTask(msg, 42); - Assert.assertTrue(task.process()); - Assert.assertEquals(1, VERB_HANDLER.invocationCount); - } - - private static class MockVerbHandler implements IVerbHandler - { - private int invocationCount; - - @Override - public void doVerb(MessageIn message, int id) - { - invocationCount++; - } - - void reset() - { - invocationCount = 0; - } - } -} diff --git a/test/unit/org/apache/cassandra/net/MessageInTest.java b/test/unit/org/apache/cassandra/net/MessageInTest.java deleted file mode 100644 index b9ea7da7f95b..000000000000 --- a/test/unit/org/apache/cassandra/net/MessageInTest.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Collections; - -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; - -public class MessageInTest -{ - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - } - - // make sure deserializing message doesn't crash with an unknown verb - @Test - public void read_NullVerb() throws IOException - { - read(null); - } - - @Test - public void read_NoSerializer() throws IOException - { - read(MessagingService.Verb.UNUSED_5); - } - - private void read(MessagingService.Verb verb) throws IOException - { - InetAddressAndPort addr = InetAddressAndPort.getByName("127.0.0.1"); - ByteBuffer buf = ByteBuffer.allocate(64); - buf.limit(buf.capacity()); - DataInputPlus dataInputBuffer = new DataInputBuffer(buf, false); - int payloadSize = 27; - Assert.assertEquals(0, buf.position()); - Assert.assertNotNull(MessageIn.read(dataInputBuffer, 1, 42, 0, addr, payloadSize, verb, Collections.emptyMap())); - Assert.assertEquals(payloadSize, buf.position()); - } -} diff --git a/test/unit/org/apache/cassandra/net/MessageTest.java b/test/unit/org/apache/cassandra/net/MessageTest.java new file mode 100644 index 000000000000..78eb4c0f3e6d --- /dev/null +++ b/test/unit/org/apache/cassandra/net/MessageTest.java @@ -0,0 +1,254 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.RequestFailureReason; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.tracing.Tracing.TraceType; +import org.apache.cassandra.utils.FBUtilities; + +import static org.apache.cassandra.net.Message.serializer; +import static org.apache.cassandra.net.MessagingService.VERSION_3014; +import static org.apache.cassandra.net.MessagingService.VERSION_30; +import static org.apache.cassandra.net.MessagingService.VERSION_40; +import static org.apache.cassandra.net.NoPayload.noPayload; +import static org.apache.cassandra.net.ParamType.RESPOND_TO; +import static org.apache.cassandra.net.ParamType.TRACE_SESSION; +import static org.apache.cassandra.net.ParamType.TRACE_TYPE; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; +import static org.junit.Assert.*; + +public class MessageTest +{ + @BeforeClass + public static void setUpClass() throws Exception + { + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setCrossNodeTimeout(true); + + Verb._TEST_2.unsafeSetSerializer(() -> new IVersionedSerializer() + { + public void serialize(Integer value, DataOutputPlus out, int version) throws IOException + { + out.writeInt(value); + } + + public Integer deserialize(DataInputPlus in, int version) throws IOException + { + return in.readInt(); + } + + public long serializedSize(Integer value, int version) + { + return 4; + } + }); + } + + @AfterClass + public static void tearDownClass() throws Exception + { + Verb._TEST_2.unsafeSetSerializer(() -> NoPayload.serializer); + } + + @Test + public void testInferMessageSize() throws Exception + { + Message msg = + Message.builder(Verb._TEST_2, 37) + .withId(1) + .from(FBUtilities.getLocalAddressAndPort()) + .withCreatedAt(approxTime.now()) + .withExpiresAt(approxTime.now()) + .withFlag(MessageFlag.CALL_BACK_ON_FAILURE) + .withFlag(MessageFlag.TRACK_REPAIRED_DATA) + .withParam(TRACE_TYPE, TraceType.QUERY) + .withParam(TRACE_SESSION, UUID.randomUUID()) + .build(); + + testInferMessageSize(msg, VERSION_30); + testInferMessageSize(msg, VERSION_3014); + testInferMessageSize(msg, VERSION_40); + } + + private void testInferMessageSize(Message msg, int version) throws Exception + { + try (DataOutputBuffer out = new DataOutputBuffer()) + { + serializer.serialize(msg, out, version); + assertEquals(msg.serializedSize(version), out.getLength()); + + ByteBuffer buffer = out.buffer(); + + int payloadSize = (int) msg.verb().serializer().serializedSize(msg.payload, version); + int serializedSize = msg.serializedSize(version); + + // should return -1 - fail to infer size - for all lengths of buffer until payload length can be read + for (int limit = 0; limit < serializedSize - payloadSize; limit++) + assertEquals(-1, serializer.inferMessageSize(buffer, 0, limit, version)); + + // once payload size can be read, should correctly infer message size + for (int limit = serializedSize - payloadSize; limit < serializedSize; limit++) + assertEquals(serializedSize, serializer.inferMessageSize(buffer, 0, limit, version)); + } + } + + @Test + public void testBuilder() + { + long id = 1; + InetAddressAndPort from = FBUtilities.getLocalAddressAndPort(); + long createAtNanos = approxTime.now(); + long expiresAtNanos = createAtNanos + TimeUnit.SECONDS.toNanos(1); + TraceType traceType = TraceType.QUERY; + UUID traceSession = UUID.randomUUID(); + + Message msg = + Message.builder(Verb._TEST_1, noPayload) + .withId(1) + .from(from) + .withCreatedAt(createAtNanos) + .withExpiresAt(expiresAtNanos) + .withFlag(MessageFlag.CALL_BACK_ON_FAILURE) + .withParam(TRACE_TYPE, TraceType.QUERY) + .withParam(TRACE_SESSION, traceSession) + .build(); + + assertEquals(id, msg.id()); + assertEquals(from, msg.from()); + assertEquals(createAtNanos, msg.createdAtNanos()); + assertEquals(expiresAtNanos, msg.expiresAtNanos()); + assertTrue(msg.callBackOnFailure()); + assertFalse(msg.trackRepairedData()); + assertEquals(traceType, msg.traceType()); + assertEquals(traceSession, msg.traceSession()); + assertNull(msg.forwardTo()); + assertNull(msg.respondTo()); + } + + @Test + public void testCycleNoPayload() throws IOException + { + Message msg = + Message.builder(Verb._TEST_1, noPayload) + .withId(1) + .from(FBUtilities.getLocalAddressAndPort()) + .withCreatedAt(approxTime.now()) + .withExpiresAt(approxTime.now() + TimeUnit.SECONDS.toNanos(1)) + .withFlag(MessageFlag.CALL_BACK_ON_FAILURE) + .withParam(TRACE_SESSION, UUID.randomUUID()) + .build(); + testCycle(msg); + } + + @Test + public void testCycleWithPayload() throws Exception + { + testCycle(Message.out(Verb._TEST_2, 42)); + testCycle(Message.outWithFlag(Verb._TEST_2, 42, MessageFlag.CALL_BACK_ON_FAILURE)); + testCycle(Message.outWithFlags(Verb._TEST_2, 42, MessageFlag.CALL_BACK_ON_FAILURE, MessageFlag.TRACK_REPAIRED_DATA)); + testCycle(Message.outWithParam(1, Verb._TEST_2, 42, RESPOND_TO, FBUtilities.getBroadcastAddressAndPort())); + } + + @Test + public void testFailureResponse() throws IOException + { + long expiresAt = approxTime.now(); + Message msg = Message.failureResponse(1, expiresAt, RequestFailureReason.INCOMPATIBLE_SCHEMA); + + assertEquals(1, msg.id()); + assertEquals(Verb.FAILURE_RSP, msg.verb()); + assertEquals(expiresAt, msg.expiresAtNanos()); + assertEquals(RequestFailureReason.INCOMPATIBLE_SCHEMA, msg.payload); + assertTrue(msg.isFailureResponse()); + + testCycle(msg); + } + + private void testCycle(Message msg) throws IOException + { + testCycle(msg, VERSION_30); + testCycle(msg, VERSION_3014); + testCycle(msg, VERSION_40); + } + + // serialize (using both variants, all in one or header then rest), verify serialized size, deserialize, compare to the original + private void testCycle(Message msg, int version) throws IOException + { + try (DataOutputBuffer out = new DataOutputBuffer()) + { + serializer.serialize(msg, out, version); + assertEquals(msg.serializedSize(version), out.getLength()); + + // deserialize the message in one go, compare outcomes + try (DataInputBuffer in = new DataInputBuffer(out.buffer(), true)) + { + Message msgOut = serializer.deserialize(in, msg.from(), version); + assertEquals(0, in.available()); + assertMessagesEqual(msg, msgOut); + } + + // extract header first, then deserialize the rest of the message and compare outcomes + ByteBuffer buffer = out.buffer(); + try (DataInputBuffer in = new DataInputBuffer(out.buffer(), false)) + { + Message.Header headerOut = serializer.extractHeader(buffer, msg.from(), approxTime.now(), version); + Message msgOut = serializer.deserialize(in, headerOut, version); + assertEquals(0, in.available()); + assertMessagesEqual(msg, msgOut); + } + } + } + + private static void assertMessagesEqual(Message msg1, Message msg2) + { + assertEquals(msg1.id(), msg2.id()); + assertEquals(msg1.verb(), msg2.verb()); + assertEquals(msg1.callBackOnFailure(), msg2.callBackOnFailure()); + assertEquals(msg1.trackRepairedData(), msg2.trackRepairedData()); + assertEquals(msg1.traceType(), msg2.traceType()); + assertEquals(msg1.traceSession(), msg2.traceSession()); + assertEquals(msg1.respondTo(), msg2.respondTo()); + assertEquals(msg1.forwardTo(), msg2.forwardTo()); + + Object payload1 = msg1.payload; + Object payload2 = msg2.payload; + + if (null == payload1) + assertTrue(payload2 == noPayload || payload2 == null); + else if (null == payload2) + assertSame(payload1, noPayload); + else + assertEquals(payload1, payload2); + } +} diff --git a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java index b56cd62950dd..76922f614f38 100644 --- a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java +++ b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java @@ -23,13 +23,14 @@ import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.regex.*; import java.util.regex.Matcher; @@ -39,20 +40,13 @@ import com.codahale.metrics.Timer; -import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; import org.apache.cassandra.auth.IInternodeAuthenticator; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.db.SystemKeyspace; -import org.apache.cassandra.db.monitoring.ApproximateTime; +import org.apache.cassandra.metrics.MessagingMetrics; +import org.apache.cassandra.utils.ApproximateTime; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService.ServerChannel; -import org.apache.cassandra.net.async.NettyFactory; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier; -import org.apache.cassandra.net.async.OutboundConnectionParams; -import org.apache.cassandra.net.async.OutboundMessagingPool; import org.apache.cassandra.utils.FBUtilities; import org.caffinitas.ohc.histo.EstimatedHistogram; import org.junit.After; @@ -61,6 +55,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.junit.Assert.*; public class MessagingServiceTest @@ -83,7 +78,7 @@ public void validateConfiguration() throws ConfigurationException private static ServerEncryptionOptions originalServerEncryptionOptions; private static InetAddressAndPort originalListenAddress; - private final MessagingService messagingService = MessagingService.test(); + private final MessagingService messagingService = new MessagingService(true); @BeforeClass public static void beforeClass() throws UnknownHostException @@ -101,10 +96,10 @@ public static void beforeClass() throws UnknownHostException @Before public void before() throws UnknownHostException { - messagingService.resetDroppedMessagesMap(Integer.toString(metricScopeId++)); + messagingService.metrics.resetDroppedMessages(Integer.toString(metricScopeId++)); MockBackPressureStrategy.applied = false; - messagingService.destroyConnectionPool(InetAddressAndPort.getByName("127.0.0.2")); - messagingService.destroyConnectionPool(InetAddressAndPort.getByName("127.0.0.3")); + messagingService.closeOutbound(InetAddressAndPort.getByName("127.0.0.2")); + messagingService.closeOutbound(InetAddressAndPort.getByName("127.0.0.3")); } @After @@ -120,29 +115,32 @@ public void tearDown() @Test public void testDroppedMessages() { - MessagingService.Verb verb = MessagingService.Verb.READ; + Verb verb = Verb.READ_REQ; for (int i = 1; i <= 5000; i++) - messagingService.incrementDroppedMessages(verb, i, i % 2 == 0); + messagingService.metrics.recordDroppedMessage(verb, i, MILLISECONDS, i % 2 == 0); - List logs = messagingService.getDroppedMessagesLogs(); + List logs = new ArrayList<>(); + messagingService.metrics.resetAndConsumeDroppedErrors(logs::add); assertEquals(1, logs.size()); - Pattern regexp = Pattern.compile("READ messages were dropped in last 5000 ms: (\\d+) internal and (\\d+) cross node. Mean internal dropped latency: (\\d+) ms and Mean cross-node dropped latency: (\\d+) ms"); + Pattern regexp = Pattern.compile("READ_REQ messages were dropped in last 5000 ms: (\\d+) internal and (\\d+) cross node. Mean internal dropped latency: (\\d+) ms and Mean cross-node dropped latency: (\\d+) ms"); Matcher matcher = regexp.matcher(logs.get(0)); assertTrue(matcher.find()); assertEquals(2500, Integer.parseInt(matcher.group(1))); assertEquals(2500, Integer.parseInt(matcher.group(2))); assertTrue(Integer.parseInt(matcher.group(3)) > 0); assertTrue(Integer.parseInt(matcher.group(4)) > 0); - assertEquals(5000, (int) messagingService.getDroppedMessages().get(verb.toString())); + assertEquals(5000, (int) messagingService.metrics.getDroppedMessages().get(verb.toString())); - logs = messagingService.getDroppedMessagesLogs(); + logs.clear(); + messagingService.metrics.resetAndConsumeDroppedErrors(logs::add); assertEquals(0, logs.size()); for (int i = 0; i < 2500; i++) - messagingService.incrementDroppedMessages(verb, i, i % 2 == 0); + messagingService.metrics.recordDroppedMessage(verb, i, MILLISECONDS, i % 2 == 0); - logs = messagingService.getDroppedMessagesLogs(); + logs.clear(); + messagingService.metrics.resetAndConsumeDroppedErrors(logs::add); assertEquals(1, logs.size()); matcher = regexp.matcher(logs.get(0)); assertTrue(matcher.find()); @@ -150,57 +148,53 @@ public void testDroppedMessages() assertEquals(1250, Integer.parseInt(matcher.group(2))); assertTrue(Integer.parseInt(matcher.group(3)) > 0); assertTrue(Integer.parseInt(matcher.group(4)) > 0); - assertEquals(7500, (int) messagingService.getDroppedMessages().get(verb.toString())); + assertEquals(7500, (int) messagingService.metrics.getDroppedMessages().get(verb.toString())); } @Test public void testDCLatency() throws Exception { int latency = 100; - ConcurrentHashMap dcLatency = MessagingService.instance().metrics.dcLatency; + ConcurrentHashMap dcLatency = MessagingService.instance().metrics.dcLatency; dcLatency.clear(); - long now = ApproximateTime.currentTimeMillis(); + long now = System.currentTimeMillis(); long sentAt = now - latency; assertNull(dcLatency.get("datacenter1")); addDCLatency(sentAt, now); assertNotNull(dcLatency.get("datacenter1")); - assertEquals(1, dcLatency.get("datacenter1").getCount()); - long expectedBucket = bucketOffsets[Math.abs(Arrays.binarySearch(bucketOffsets, TimeUnit.MILLISECONDS.toNanos(latency))) - 1]; - assertEquals(expectedBucket, dcLatency.get("datacenter1").getSnapshot().getMax()); + assertEquals(1, dcLatency.get("datacenter1").dcLatency.getCount()); + long expectedBucket = bucketOffsets[Math.abs(Arrays.binarySearch(bucketOffsets, MILLISECONDS.toNanos(latency))) - 1]; + assertEquals(expectedBucket, dcLatency.get("datacenter1").dcLatency.getSnapshot().getMax()); } @Test - public void testNegativeDCLatency() throws Exception + public void testNegativeDCLatency() { + MessagingMetrics.DCLatencyRecorder updater = MessagingService.instance().metrics.internodeLatencyRecorder(InetAddressAndPort.getLocalHost()); + // if clocks are off should just not track anything int latency = -100; - ConcurrentHashMap dcLatency = MessagingService.instance().metrics.dcLatency; - dcLatency.clear(); - - long now = ApproximateTime.currentTimeMillis(); + long now = System.currentTimeMillis(); long sentAt = now - latency; - assertNull(dcLatency.get("datacenter1")); - addDCLatency(sentAt, now); - assertNull(dcLatency.get("datacenter1")); + long count = updater.dcLatency.getCount(); + updater.accept(now - sentAt, MILLISECONDS); + // negative value shoudln't be recorded + assertEquals(count, updater.dcLatency.getCount()); } @Test - public void testQueueWaitLatency() throws Exception + public void testQueueWaitLatency() { int latency = 100; - String verb = MessagingService.Verb.MUTATION.toString(); - - ConcurrentHashMap queueWaitLatency = MessagingService.instance().metrics.queueWaitLatency; - queueWaitLatency.clear(); + Verb verb = Verb.MUTATION_REQ; - assertNull(queueWaitLatency.get(verb)); - MessagingService.instance().metrics.addQueueWaitTime(verb, latency); - assertNotNull(queueWaitLatency.get(verb)); + Map queueWaitLatency = MessagingService.instance().metrics.internalLatency; + MessagingService.instance().metrics.recordInternalLatency(verb, latency, MILLISECONDS); assertEquals(1, queueWaitLatency.get(verb).getCount()); - long expectedBucket = bucketOffsets[Math.abs(Arrays.binarySearch(bucketOffsets, TimeUnit.MILLISECONDS.toNanos(latency))) - 1]; + long expectedBucket = bucketOffsets[Math.abs(Arrays.binarySearch(bucketOffsets, MILLISECONDS.toNanos(latency))) - 1]; assertEquals(expectedBucket, queueWaitLatency.get(verb).getSnapshot().getMax()); } @@ -208,13 +202,13 @@ public void testQueueWaitLatency() throws Exception public void testNegativeQueueWaitLatency() throws Exception { int latency = -100; - String verb = MessagingService.Verb.MUTATION.toString(); + Verb verb = Verb.MUTATION_REQ; - ConcurrentHashMap queueWaitLatency = MessagingService.instance().metrics.queueWaitLatency; + Map queueWaitLatency = MessagingService.instance().metrics.internalLatency; queueWaitLatency.clear(); assertNull(queueWaitLatency.get(verb)); - MessagingService.instance().metrics.addQueueWaitTime(verb, latency); + MessagingService.instance().metrics.recordInternalLatency(verb, latency, MILLISECONDS); assertNull(queueWaitLatency.get(verb)); } @@ -222,9 +216,9 @@ public void testNegativeQueueWaitLatency() throws Exception public void testUpdatesBackPressureOnSendWhenEnabledAndWithSupportedCallback() throws UnknownHostException { MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddressAndPort.getByName("127.0.0.2")); - IAsyncCallback bpCallback = new BackPressureCallback(); - IAsyncCallback noCallback = new NoBackPressureCallback(); - MessageOut ignored = null; + RequestCallback bpCallback = new BackPressureCallback(); + RequestCallback noCallback = new NoBackPressureCallback(); + Message ignored = null; DatabaseDescriptor.setBackPressureEnabled(true); messagingService.updateBackPressureOnSend(InetAddressAndPort.getByName("127.0.0.2"), noCallback, ignored); @@ -243,8 +237,8 @@ public void testUpdatesBackPressureOnSendWhenEnabledAndWithSupportedCallback() t public void testUpdatesBackPressureOnReceiveWhenEnabledAndWithSupportedCallback() throws UnknownHostException { MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddressAndPort.getByName("127.0.0.2")); - IAsyncCallback bpCallback = new BackPressureCallback(); - IAsyncCallback noCallback = new NoBackPressureCallback(); + RequestCallback bpCallback = new BackPressureCallback(); + RequestCallback noCallback = new NoBackPressureCallback(); boolean timeout = false; DatabaseDescriptor.setBackPressureEnabled(true); @@ -267,8 +261,8 @@ public void testUpdatesBackPressureOnReceiveWhenEnabledAndWithSupportedCallback( public void testUpdatesBackPressureOnTimeoutWhenEnabledAndWithSupportedCallback() throws UnknownHostException { MockBackPressureStrategy.MockBackPressureState backPressureState = (MockBackPressureStrategy.MockBackPressureState) messagingService.getBackPressureState(InetAddressAndPort.getByName("127.0.0.2")); - IAsyncCallback bpCallback = new BackPressureCallback(); - IAsyncCallback noCallback = new NoBackPressureCallback(); + RequestCallback bpCallback = new BackPressureCallback(); + RequestCallback noCallback = new NoBackPressureCallback(); boolean timeout = true; DatabaseDescriptor.setBackPressureEnabled(true); @@ -309,7 +303,7 @@ public void testDoesntApplyBackPressureToBroadcastAddress() throws UnknownHostEx private static void addDCLatency(long sentAt, long nowTime) throws IOException { - MessageIn.deriveConstructionTime(InetAddressAndPort.getLocalHost(), (int) sentAt, nowTime); + MessagingService.instance().metrics.internodeLatencyRecorder(InetAddressAndPort.getLocalHost()).accept(nowTime - sentAt, MILLISECONDS); } public static class MockBackPressureStrategy implements BackPressureStrategy @@ -346,7 +340,7 @@ private MockBackPressureState(InetAddressAndPort host) } @Override - public void onMessageSent(MessageOut message) + public void onMessageSent(Message message) { onSend = true; } @@ -377,7 +371,7 @@ public InetAddressAndPort getHost() } } - private static class BackPressureCallback implements IAsyncCallback + private static class BackPressureCallback implements RequestCallback { @Override public boolean supportsBackPressure() @@ -386,19 +380,13 @@ public boolean supportsBackPressure() } @Override - public boolean isLatencyForSnitch() - { - return false; - } - - @Override - public void response(MessageIn msg) + public void onResponse(Message msg) { throw new UnsupportedOperationException("Not supported."); } } - private static class NoBackPressureCallback implements IAsyncCallback + private static class NoBackPressureCallback implements RequestCallback { @Override public boolean supportsBackPressure() @@ -407,13 +395,7 @@ public boolean supportsBackPressure() } @Override - public boolean isLatencyForSnitch() - { - return false; - } - - @Override - public void response(MessageIn msg) + public void onResponse(Message msg) { throw new UnsupportedOperationException("Not supported."); } @@ -433,167 +415,111 @@ public void testFailedInternodeAuth() throws Exception InetAddressAndPort address = InetAddressAndPort.getByName("127.0.0.250"); //Should return null - MessageOut messageOut = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK); + Message messageOut = Message.out(Verb.ECHO_REQ, NoPayload.noPayload); assertFalse(ms.isConnected(address, messageOut)); //Should tolerate null - ms.convict(address); - ms.sendOneWay(messageOut, address); - } - - @Test - public void testOutboundMessagingConnectionCleansUp() throws Exception - { - MessagingService ms = MessagingService.instance(); - InetAddressAndPort local = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", 9876); - InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", 9876); - - OutboundMessagingPool pool = new OutboundMessagingPool(remote, local, null, new MockBackPressureStrategy(null).newState(remote), ALLOW_NOTHING_AUTHENTICATOR); - ms.channelManagers.put(remote, pool); - pool.sendMessage(new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK), 0); - assertFalse(ms.channelManagers.containsKey(remote)); - } - - @Test - public void reconnectWithNewIp() throws Exception - { - InetAddressAndPort publicIp = InetAddressAndPort.getByName("127.0.0.2"); - InetAddressAndPort privateIp = InetAddressAndPort.getByName("127.0.0.3"); - - // reset the preferred IP value, for good test hygene - SystemKeyspace.updatePreferredIP(publicIp, publicIp); - - // create pool/conn with public addr - Assert.assertEquals(publicIp, messagingService.getCurrentEndpoint(publicIp)); - messagingService.reconnectWithNewIp(publicIp, privateIp); - Assert.assertEquals(privateIp, messagingService.getCurrentEndpoint(publicIp)); - - messagingService.destroyConnectionPool(publicIp); - - // recreate the pool/conn, and make sure the preferred ip addr is used - Assert.assertEquals(privateIp, messagingService.getCurrentEndpoint(publicIp)); - } - - @Test - public void testCloseInboundConnections() throws UnknownHostException, InterruptedException - { - try - { - messagingService.listen(); - Assert.assertTrue(messagingService.isListening()); - Assert.assertTrue(messagingService.serverChannels.size() > 0); - for (ServerChannel serverChannel : messagingService.serverChannels) - Assert.assertEquals(0, serverChannel.size()); - - // now, create a connection and make sure it's in a channel group - InetAddressAndPort server = FBUtilities.getBroadcastAddressAndPort(); - OutboundConnectionIdentifier id = OutboundConnectionIdentifier.small(InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", 0), server); - - CountDownLatch latch = new CountDownLatch(1); - OutboundConnectionParams params = OutboundConnectionParams.builder() - .mode(NettyFactory.Mode.MESSAGING) - .sendBufferSize(1 << 10) - .connectionId(id) - .callback(handshakeResult -> latch.countDown()) - .protocolVersion(MessagingService.current_version) - .build(); - Bootstrap bootstrap = NettyFactory.instance.createOutboundBootstrap(params); - Channel channel = bootstrap.connect().awaitUninterruptibly().channel(); - Assert.assertNotNull(channel); - latch.await(1, TimeUnit.SECONDS); // allow the netty pipeline/c* handshake to get set up - - int connectCount = 0; - for (ServerChannel serverChannel : messagingService.serverChannels) - connectCount += serverChannel.size(); - Assert.assertTrue(connectCount > 0); - } - finally - { - // last, shutdown the MS and make sure connections are removed - messagingService.shutdown(true); - for (ServerChannel serverChannel : messagingService.serverChannels) - Assert.assertEquals(0, serverChannel.size()); - messagingService.clearServerChannels(); - } - } + ms.closeOutbound(address); + ms.send(messageOut, address); + } + +// @Test +// public void reconnectWithNewIp() throws Exception +// { +// InetAddressAndPort publicIp = InetAddressAndPort.getByName("127.0.0.2"); +// InetAddressAndPort privateIp = InetAddressAndPort.getByName("127.0.0.3"); +// +// // reset the preferred IP value, for good test hygene +// SystemKeyspace.updatePreferredIP(publicIp, publicIp); +// +// // create pool/conn with public addr +// Assert.assertEquals(publicIp, messagingService.getCurrentEndpoint(publicIp)); +// messagingService.maybeReconnectWithNewIp(publicIp, privateIp).await(1L, TimeUnit.SECONDS); +// Assert.assertEquals(privateIp, messagingService.getCurrentEndpoint(publicIp)); +// +// messagingService.closeOutbound(publicIp); +// +// // recreate the pool/conn, and make sure the preferred ip addr is used +// Assert.assertEquals(privateIp, messagingService.getCurrentEndpoint(publicIp)); +// } @Test - public void listenPlainConnection() + public void listenPlainConnection() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = false; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(false); listen(serverEncryptionOptions, false); } @Test - public void listenPlainConnectionWithBroadcastAddr() + public void listenPlainConnectionWithBroadcastAddr() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = false; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(false); listen(serverEncryptionOptions, true); } @Test - public void listenRequiredSecureConnection() + public void listenRequiredSecureConnection() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = true; - serverEncryptionOptions.optional = false; - serverEncryptionOptions.enable_legacy_ssl_storage_port = false; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(true) + .withOptional(false) + .withLegacySslStoragePort(false); listen(serverEncryptionOptions, false); } @Test - public void listenRequiredSecureConnectionWithBroadcastAddr() + public void listenRequiredSecureConnectionWithBroadcastAddr() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = true; - serverEncryptionOptions.optional = false; - serverEncryptionOptions.enable_legacy_ssl_storage_port = false; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(true) + .withOptional(false) + .withLegacySslStoragePort(false); listen(serverEncryptionOptions, true); } @Test - public void listenRequiredSecureConnectionWithLegacyPort() + public void listenRequiredSecureConnectionWithLegacyPort() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = true; - serverEncryptionOptions.optional = false; - serverEncryptionOptions.enable_legacy_ssl_storage_port = true; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(true) + .withOptional(false) + .withLegacySslStoragePort(true); listen(serverEncryptionOptions, false); } @Test - public void listenRequiredSecureConnectionWithBroadcastAddrAndLegacyPort() + public void listenRequiredSecureConnectionWithBroadcastAddrAndLegacyPort() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = true; - serverEncryptionOptions.optional = false; - serverEncryptionOptions.enable_legacy_ssl_storage_port = true; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(true) + .withOptional(false) + .withLegacySslStoragePort(true); listen(serverEncryptionOptions, true); } @Test - public void listenOptionalSecureConnection() + public void listenOptionalSecureConnection() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = true; - serverEncryptionOptions.optional = true; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(true) + .withOptional(true); listen(serverEncryptionOptions, false); } @Test - public void listenOptionalSecureConnectionWithBroadcastAddr() + public void listenOptionalSecureConnectionWithBroadcastAddr() throws InterruptedException { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = true; - serverEncryptionOptions.optional = true; + ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions() + .withEnabled(true) + .withOptional(true); listen(serverEncryptionOptions, true); } - private void listen(ServerEncryptionOptions serverEncryptionOptions, boolean listenOnBroadcastAddr) + private void listen(ServerEncryptionOptions serverEncryptionOptions, boolean listenOnBroadcastAddr) throws InterruptedException { - InetAddress listenAddress = null; + InetAddress listenAddress = FBUtilities.getJustLocalAddress(); if (listenOnBroadcastAddr) { DatabaseDescriptor.setShouldListenOnBroadcastAddress(true); @@ -602,111 +528,93 @@ private void listen(ServerEncryptionOptions serverEncryptionOptions, boolean lis FBUtilities.reset(); } + InboundConnectionSettings settings = new InboundConnectionSettings() + .withEncryption(serverEncryptionOptions); + InboundSockets connections = new InboundSockets(settings); try { - messagingService.listen(serverEncryptionOptions); - Assert.assertTrue(messagingService.isListening()); - int expectedListeningCount = NettyFactory.determineAcceptGroupSize(serverEncryptionOptions); - Assert.assertEquals(expectedListeningCount, messagingService.serverChannels.size()); + connections.open().await(); + Assert.assertTrue(connections.isListening()); - if (!serverEncryptionOptions.enabled) - { - // make sure no channel is using TLS - for (ServerChannel serverChannel : messagingService.serverChannels) - Assert.assertEquals(ServerChannel.SecurityLevel.NONE, serverChannel.getSecurityLevel()); - } - else + Set expect = new HashSet<>(); + expect.add(InetAddressAndPort.getByAddressOverrideDefaults(listenAddress, DatabaseDescriptor.getStoragePort())); + if (settings.encryption.enable_legacy_ssl_storage_port) + expect.add(InetAddressAndPort.getByAddressOverrideDefaults(listenAddress, DatabaseDescriptor.getSSLStoragePort())); + if (listenOnBroadcastAddr) { - final int legacySslPort = DatabaseDescriptor.getSSLStoragePort(); - boolean foundLegacyListenSslAddress = false; - for (ServerChannel serverChannel : messagingService.serverChannels) - { - if (serverEncryptionOptions.optional) - Assert.assertEquals(ServerChannel.SecurityLevel.OPTIONAL, serverChannel.getSecurityLevel()); - else - Assert.assertEquals(ServerChannel.SecurityLevel.REQUIRED, serverChannel.getSecurityLevel()); - - if (serverEncryptionOptions.enable_legacy_ssl_storage_port) - { - if (legacySslPort == serverChannel.getAddress().port) - { - foundLegacyListenSslAddress = true; - Assert.assertEquals(ServerChannel.SecurityLevel.REQUIRED, serverChannel.getSecurityLevel()); - } - } - } - - if (serverEncryptionOptions.enable_legacy_ssl_storage_port && !foundLegacyListenSslAddress) - Assert.fail("failed to find legacy ssl listen address"); + expect.add(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getBroadcastAddressAndPort().address, DatabaseDescriptor.getStoragePort())); + if (settings.encryption.enable_legacy_ssl_storage_port) + expect.add(InetAddressAndPort.getByAddressOverrideDefaults(FBUtilities.getBroadcastAddressAndPort().address, DatabaseDescriptor.getSSLStoragePort())); } - // check the optional listen address - if (listenOnBroadcastAddr) + Assert.assertEquals(expect.size(), connections.sockets().size()); + + final int legacySslPort = DatabaseDescriptor.getSSLStoragePort(); + for (InboundSockets.InboundSocket socket : connections.sockets()) { - int expectedCount = (serverEncryptionOptions.enabled && serverEncryptionOptions.enable_legacy_ssl_storage_port) ? 2 : 1; - int found = 0; - for (ServerChannel serverChannel : messagingService.serverChannels) - { - if (serverChannel.getAddress().address.equals(listenAddress)) - found++; - } - - Assert.assertEquals(expectedCount, found); + Assert.assertEquals(serverEncryptionOptions.enabled, socket.settings.encryption.enabled); + Assert.assertEquals(serverEncryptionOptions.optional, socket.settings.encryption.optional); + if (!serverEncryptionOptions.enabled) + Assert.assertFalse(legacySslPort == socket.settings.bindAddress.port); + if (legacySslPort == socket.settings.bindAddress.port) + Assert.assertFalse(socket.settings.encryption.optional); + Assert.assertTrue(socket.settings.bindAddress.toString(), expect.remove(socket.settings.bindAddress)); } } finally { - messagingService.shutdown(true); - messagingService.clearServerChannels(); - Assert.assertEquals(0, messagingService.serverChannels.size()); + connections.close().await(); + Assert.assertFalse(connections.isListening()); } } - @Test - public void getPreferredRemoteAddrUsesPrivateIp() throws UnknownHostException - { - MessagingService ms = MessagingService.instance(); - InetAddressAndPort local = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.4", 7000); - InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.151", 7000); - InetAddressAndPort privateIp = InetAddressAndPort.getByName("127.0.0.6"); - - OutboundMessagingPool pool = new OutboundMessagingPool(privateIp, local, null, - new MockBackPressureStrategy(null).newState(remote), - ALLOW_NOTHING_AUTHENTICATOR); - ms.channelManagers.put(remote, pool); - - Assert.assertEquals(privateIp, ms.getPreferredRemoteAddr(remote)); - } - - @Test - public void getPreferredRemoteAddrUsesPreferredIp() throws UnknownHostException - { - MessagingService ms = MessagingService.instance(); - InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.115", 7000); - - InetAddressAndPort preferredIp = InetAddressAndPort.getByName("127.0.0.16"); - SystemKeyspace.updatePreferredIP(remote, preferredIp); - - Assert.assertEquals(preferredIp, ms.getPreferredRemoteAddr(remote)); - } - - @Test - public void getPreferredRemoteAddrUsesPrivateIpOverridesPreferredIp() throws UnknownHostException - { - MessagingService ms = MessagingService.instance(); - InetAddressAndPort local = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.4", 7000); - InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.105", 7000); - InetAddressAndPort privateIp = InetAddressAndPort.getByName("127.0.0.6"); - - OutboundMessagingPool pool = new OutboundMessagingPool(privateIp, local, null, - new MockBackPressureStrategy(null).newState(remote), - ALLOW_NOTHING_AUTHENTICATOR); - ms.channelManagers.put(remote, pool); - - InetAddressAndPort preferredIp = InetAddressAndPort.getByName("127.0.0.16"); - SystemKeyspace.updatePreferredIP(remote, preferredIp); - - Assert.assertEquals(privateIp, ms.getPreferredRemoteAddr(remote)); - } +// @Test +// public void getPreferredRemoteAddrUsesPrivateIp() throws UnknownHostException +// { +// MessagingService ms = MessagingService.instance(); +// InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.151", 7000); +// InetAddressAndPort privateIp = InetAddressAndPort.getByName("127.0.0.6"); +// +// OutboundConnectionSettings template = new OutboundConnectionSettings(remote) +// .withConnectTo(privateIp) +// .withAuthenticator(ALLOW_NOTHING_AUTHENTICATOR); +// OutboundConnections pool = new OutboundConnections(template, new MockBackPressureStrategy(null).newState(remote)); +// ms.channelManagers.put(remote, pool); +// +// Assert.assertEquals(privateIp, ms.getPreferredRemoteAddr(remote)); +// } +// +// @Test +// public void getPreferredRemoteAddrUsesPreferredIp() throws UnknownHostException +// { +// MessagingService ms = MessagingService.instance(); +// InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.115", 7000); +// +// InetAddressAndPort preferredIp = InetAddressAndPort.getByName("127.0.0.16"); +// SystemKeyspace.updatePreferredIP(remote, preferredIp); +// +// Assert.assertEquals(preferredIp, ms.getPreferredRemoteAddr(remote)); +// } +// +// @Test +// public void getPreferredRemoteAddrUsesPrivateIpOverridesPreferredIp() throws UnknownHostException +// { +// MessagingService ms = MessagingService.instance(); +// InetAddressAndPort local = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.4", 7000); +// InetAddressAndPort remote = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.105", 7000); +// InetAddressAndPort privateIp = InetAddressAndPort.getByName("127.0.0.6"); +// +// OutboundConnectionSettings template = new OutboundConnectionSettings(remote) +// .withConnectTo(privateIp) +// .withAuthenticator(ALLOW_NOTHING_AUTHENTICATOR); +// +// OutboundConnections pool = new OutboundConnections(template, new MockBackPressureStrategy(null).newState(remote)); +// ms.channelManagers.put(remote, pool); +// +// InetAddressAndPort preferredIp = InetAddressAndPort.getByName("127.0.0.16"); +// SystemKeyspace.updatePreferredIP(remote, preferredIp); +// +// Assert.assertEquals(privateIp, ms.getPreferredRemoteAddr(remote)); +// } } diff --git a/test/unit/org/apache/cassandra/net/MockMessagingService.java b/test/unit/org/apache/cassandra/net/MockMessagingService.java index 79edae8398ad..3749bafba70e 100644 --- a/test/unit/org/apache/cassandra/net/MockMessagingService.java +++ b/test/unit/org/apache/cassandra/net/MockMessagingService.java @@ -26,7 +26,7 @@ * Starting point for mocking {@link MessagingService} interactions. Outgoing messages can be * intercepted by first creating a {@link MatcherResponse} by calling {@link MockMessagingService#when(Matcher)}. * Alternatively {@link Matcher}s can be created by using helper methods such as {@link #to(InetAddressAndPort)}, - * {@link #verb(MessagingService.Verb)} or {@link #payload(Predicate)} and may also be + * {@link #verb(Verb)} or {@link #payload(Predicate)} and may also be * nested using {@link MockMessagingService#all(Matcher[])} or {@link MockMessagingService#any(Matcher[])}. * After each test, {@link MockMessagingService#cleanup()} must be called for free listeners registered * in {@link MessagingService}. @@ -47,12 +47,13 @@ public static MatcherResponse when(Matcher matcher) } /** - * Unsubscribes any handlers added by calling {@link MessagingService#addMessageSink(IMessageSink)}. + * Unsubscribes any handlers. * This should be called after each test. */ public static void cleanup() { - MessagingService.instance().clearMessageSinks(); + MessagingService.instance().outboundSink.clear(); + MessagingService.instance().inboundSink.clear(); } /** @@ -92,15 +93,15 @@ public static Matcher to(Predicate predi * Creates a matcher that will indicate if the verb of the outgoing message equals the * provided value. */ - public static Matcher verb(MessagingService.Verb verb) + public static Matcher verb(Verb verb) { - return (in, to) -> in.verb == verb; + return (in, to) -> in.verb() == verb; } /** * Creates a matcher based on the result of the provided predicate called with the outgoing message. */ - public static Matcher message(Predicate> fn) + public static Matcher message(Predicate> fn) { return (msg, to) -> fn.test(msg); } @@ -126,7 +127,7 @@ public static Matcher not(Matcher matcher) */ public static Matcher all(Matcher... matchers) { - return (MessageOut out, InetAddressAndPort to) -> { + return (Message out, InetAddressAndPort to) -> { for (Matcher matcher : matchers) { if (!matcher.matches(out, to)) @@ -141,7 +142,7 @@ public static Matcher all(Matcher... matchers) */ public static Matcher any(Matcher... matchers) { - return (MessageOut out, InetAddressAndPort to) -> { + return (Message out, InetAddressAndPort to) -> { for (Matcher matcher : matchers) { if (matcher.matches(out, to)) diff --git a/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java index 8d0f91bf8d55..e4787f74ed38 100644 --- a/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java +++ b/test/unit/org/apache/cassandra/net/MockMessagingServiceTest.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.net; -import java.util.Collections; import java.util.concurrent.ExecutionException; import org.junit.Before; @@ -26,14 +25,15 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.gms.EchoMessage; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.net.Verb.ECHO_REQ; import static org.apache.cassandra.net.MockMessagingService.all; import static org.apache.cassandra.net.MockMessagingService.to; import static org.apache.cassandra.net.MockMessagingService.verb; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; public class MockMessagingServiceTest @@ -54,40 +54,28 @@ public void cleanup() @Test public void testRequestResponse() throws InterruptedException, ExecutionException { - // echo message that we like to mock as incoming reply for outgoing echo message - MessageIn echoMessageIn = MessageIn.create(FBUtilities.getBroadcastAddressAndPort(), - EchoMessage.instance, - Collections.emptyMap(), - MessagingService.Verb.ECHO, - MessagingService.current_version); + // echo message that we like to mock as incoming response for outgoing echo message + Message echoMessage = Message.out(ECHO_REQ, NoPayload.noPayload); MockMessagingSpy spy = MockMessagingService .when( all( to(FBUtilities.getBroadcastAddressAndPort()), - verb(MessagingService.Verb.ECHO) + verb(ECHO_REQ) ) ) - .respond(echoMessageIn); + .respond(echoMessage); - MessageOut echoMessageOut = new MessageOut<>(MessagingService.Verb.ECHO, EchoMessage.instance, EchoMessage.serializer); - MessagingService.instance().sendRR(echoMessageOut, FBUtilities.getBroadcastAddressAndPort(), new IAsyncCallback() + Message echoMessageOut = Message.out(ECHO_REQ, NoPayload.noPayload); + MessagingService.instance().sendWithCallback(echoMessageOut, FBUtilities.getBroadcastAddressAndPort(), msg -> { - public void response(MessageIn msg) - { - assertEquals(MessagingService.Verb.ECHO, msg.verb); - assertEquals(echoMessageIn.payload, msg.payload); - } - - public boolean isLatencyForSnitch() - { - return false; - } + assertEquals(ECHO_REQ, msg.verb()); + assertEquals(echoMessage.payload, msg.payload); }); // we must have intercepted the outgoing message at this point - MessageOut msg = spy.captureMessageOut().get(); + Message msg = spy.captureMessageOut().get(); assertEquals(1, spy.messagesIntercepted); - assertTrue(msg == echoMessageOut); + assertSame(echoMessage.payload, msg.payload); // and return a mocked response assertEquals(1, spy.mockedMessageResponses); diff --git a/test/unit/org/apache/cassandra/net/MockMessagingSpy.java b/test/unit/org/apache/cassandra/net/MockMessagingSpy.java index 80bdb39a0f1a..bf4c2267c8e2 100644 --- a/test/unit/org/apache/cassandra/net/MockMessagingSpy.java +++ b/test/unit/org/apache/cassandra/net/MockMessagingSpy.java @@ -43,25 +43,25 @@ public class MockMessagingSpy public int messagesIntercepted = 0; public int mockedMessageResponses = 0; - private final BlockingQueue> interceptedMessages = new LinkedBlockingQueue<>(); - private final BlockingQueue> deliveredResponses = new LinkedBlockingQueue<>(); + private final BlockingQueue> interceptedMessages = new LinkedBlockingQueue<>(); + private final BlockingQueue> deliveredResponses = new LinkedBlockingQueue<>(); private static final Executor executor = Executors.newSingleThreadExecutor(); /** * Returns a future with the first mocked incoming message that has been created and delivered. */ - public ListenableFuture> captureMockedMessageIn() + public ListenableFuture> captureMockedMessage() { - return Futures.transform(captureMockedMessageInN(1), (List> result) -> result.isEmpty() ? null : result.get(0)); + return Futures.transform(captureMockedMessageN(1), (List> result) -> result.isEmpty() ? null : result.get(0)); } /** * Returns a future with the specified number mocked incoming messages that have been created and delivered. */ - public ListenableFuture>> captureMockedMessageInN(int noOfMessages) + public ListenableFuture>> captureMockedMessageN(int noOfMessages) { - CapturedResultsFuture> ret = new CapturedResultsFuture<>(noOfMessages, deliveredResponses); + CapturedResultsFuture> ret = new CapturedResultsFuture<>(noOfMessages, deliveredResponses); executor.execute(ret); return ret; } @@ -69,17 +69,17 @@ public ListenableFuture>> captureMockedMessageInN(int noOfMess /** * Returns a future that will indicate if a mocked incoming message has been created and delivered. */ - public ListenableFuture expectMockedMessageIn() + public ListenableFuture expectMockedMessage() { - return expectMockedMessageIn(1); + return expectMockedMessage(1); } /** * Returns a future that will indicate if the specified number of mocked incoming message have been created and delivered. */ - public ListenableFuture expectMockedMessageIn(int noOfMessages) + public ListenableFuture expectMockedMessage(int noOfMessages) { - ResultsCompletionFuture> ret = new ResultsCompletionFuture<>(noOfMessages, deliveredResponses); + ResultsCompletionFuture> ret = new ResultsCompletionFuture<>(noOfMessages, deliveredResponses); executor.execute(ret); return ret; } @@ -87,17 +87,17 @@ public ListenableFuture expectMockedMessageIn(int noOfMessages) /** * Returns a future with the first intercepted outbound message that would have been send. */ - public ListenableFuture> captureMessageOut() + public ListenableFuture> captureMessageOut() { - return Futures.transform(captureMessageOut(1), (List> result) -> result.isEmpty() ? null : result.get(0)); + return Futures.transform(captureMessageOut(1), (List> result) -> result.isEmpty() ? null : result.get(0)); } /** * Returns a future with the specified number of intercepted outbound messages that would have been send. */ - public ListenableFuture>> captureMessageOut(int noOfMessages) + public ListenableFuture>> captureMessageOut(int noOfMessages) { - CapturedResultsFuture> ret = new CapturedResultsFuture<>(noOfMessages, interceptedMessages); + CapturedResultsFuture> ret = new CapturedResultsFuture<>(noOfMessages, interceptedMessages); executor.execute(ret); return ret; } @@ -115,7 +115,7 @@ public ListenableFuture interceptMessageOut() */ public ListenableFuture interceptMessageOut(int noOfMessages) { - ResultsCompletionFuture> ret = new ResultsCompletionFuture<>(noOfMessages, interceptedMessages); + ResultsCompletionFuture> ret = new ResultsCompletionFuture<>(noOfMessages, interceptedMessages); executor.execute(ret); return ret; } @@ -125,19 +125,19 @@ public ListenableFuture interceptMessageOut(int noOfMessages) */ public ListenableFuture interceptNoMsg(long time, TimeUnit unit) { - ResultAbsenceFuture> ret = new ResultAbsenceFuture<>(interceptedMessages, time, unit); + ResultAbsenceFuture> ret = new ResultAbsenceFuture<>(interceptedMessages, time, unit); executor.execute(ret); return ret; } - void matchingMessage(MessageOut message) + void matchingMessage(Message message) { messagesIntercepted++; logger.trace("Received matching message: {}", message); interceptedMessages.add(message); } - void matchingResponse(MessageIn response) + void matchingResponse(Message response) { mockedMessageResponses++; logger.trace("Responding to intercepted message: {}", response); diff --git a/test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java b/test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java new file mode 100644 index 000000000000..7cf78a77a91d --- /dev/null +++ b/test/unit/org/apache/cassandra/net/OutboundConnectionSettingsTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Function; + +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.Config; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.locator.AbstractEndpointSnitch; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Replica; + +import static org.apache.cassandra.config.DatabaseDescriptor.getEndpointSnitch; +import static org.apache.cassandra.net.MessagingService.current_version; +import static org.apache.cassandra.net.ConnectionType.*; +import static org.apache.cassandra.net.OutboundConnectionsTest.LOCAL_ADDR; +import static org.apache.cassandra.net.OutboundConnectionsTest.REMOTE_ADDR; + +public class OutboundConnectionSettingsTest +{ + @BeforeClass + public static void before() + { + DatabaseDescriptor.daemonInitialization(); + } + + @Test (expected = IllegalArgumentException.class) + public void build_SmallSendSize() + { + test(settings -> settings.withSocketSendBufferSizeInBytes(999)); + } + + @Test (expected = IllegalArgumentException.class) + public void build_SendSizeLessThanZero() + { + test(settings -> settings.withSocketSendBufferSizeInBytes(-1)); + } + + @Test (expected = IllegalArgumentException.class) + public void build_TcpConnectTimeoutLessThanZero() + { + test(settings -> settings.withTcpConnectTimeoutInMS(-1)); + } + + @Test(expected = IllegalArgumentException.class) + public void build_TcpUserTimeoutLessThanZero() + { + test(settings -> settings.withTcpUserTimeoutInMS(-1)); + } + + @Test + public void build_TcpUserTimeoutEqualsZero() + { + test(settings -> settings.withTcpUserTimeoutInMS(0)); + } + + private static void test(Function f) + { + f.apply(new OutboundConnectionSettings(LOCAL_ADDR)).withDefaults(ConnectionCategory.MESSAGING); + } + + private static class TestSnitch extends AbstractEndpointSnitch + { + private final Map nodeToDc = new HashMap<>(); + + void add(InetAddressAndPort node, String dc) + { + nodeToDc.put(node, dc); + } + + public String getRack(InetAddressAndPort endpoint) + { + return null; + } + + public String getDatacenter(InetAddressAndPort endpoint) + { + return nodeToDc.get(endpoint); + } + + public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) + { + return 0; + } + } + + @Test + public void shouldCompressConnection_None() + { + DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.none); + Assert.assertFalse(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + } + + @Test + public void shouldCompressConnection_DifferentDc() + { + TestSnitch snitch = new TestSnitch(); + snitch.add(LOCAL_ADDR, "dc1"); + snitch.add(REMOTE_ADDR, "dc2"); + DatabaseDescriptor.setEndpointSnitch(snitch); + DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc); + Assert.assertTrue(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + } + + @Test + public void shouldCompressConnection_All() + { + DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.all); + Assert.assertTrue(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + } + + @Test + public void shouldCompressConnection_SameDc() + { + TestSnitch snitch = new TestSnitch(); + snitch.add(LOCAL_ADDR, "dc1"); + snitch.add(REMOTE_ADDR, "dc1"); + DatabaseDescriptor.setEndpointSnitch(snitch); + DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc); + Assert.assertFalse(OutboundConnectionSettings.shouldCompressConnection(getEndpointSnitch(), LOCAL_ADDR, REMOTE_ADDR)); + } + +} diff --git a/test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java b/test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java new file mode 100644 index 000000000000..20180fb9b4e6 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/OutboundConnectionsTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import com.google.common.collect.ImmutableList; +import com.google.common.net.InetAddresses; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.gms.GossipDigestSyn; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.BackPressureState; +import org.apache.cassandra.net.ConnectionType; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.OutboundConnectionSettings; +import org.apache.cassandra.net.OutboundConnections; +import org.apache.cassandra.net.PingRequest; +import org.apache.cassandra.net.Verb; + +public class OutboundConnectionsTest +{ + static final InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9476); + static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9476); + private static final InetAddressAndPort RECONNECT_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.3"), 9476); + private static final List INTERNODE_MESSAGING_CONN_TYPES = ImmutableList.of(ConnectionType.URGENT_MESSAGES, ConnectionType.LARGE_MESSAGES, ConnectionType.SMALL_MESSAGES); + + private OutboundConnections connections; + + @BeforeClass + public static void before() + { + DatabaseDescriptor.daemonInitialization(); + } + + @Before + public void setup() + { + BackPressureState backPressureState = DatabaseDescriptor.getBackPressureStrategy().newState(REMOTE_ADDR); + connections = OutboundConnections.unsafeCreate(new OutboundConnectionSettings(REMOTE_ADDR), backPressureState); + } + + @After + public void tearDown() throws ExecutionException, InterruptedException, TimeoutException + { + if (connections != null) + connections.close(false).get(10L, TimeUnit.SECONDS); + } + + @Test + public void getConnection_Gossip() + { + GossipDigestSyn syn = new GossipDigestSyn("cluster", "partitioner", new ArrayList<>(0)); + Message message = Message.out(Verb.GOSSIP_DIGEST_SYN, syn); + Assert.assertEquals(ConnectionType.URGENT_MESSAGES, connections.connectionFor(message).type()); + } + + @Test + public void getConnection_SmallMessage() + { + Message message = Message.out(Verb.PING_REQ, PingRequest.forSmall); + Assert.assertEquals(ConnectionType.SMALL_MESSAGES, connections.connectionFor(message).type()); + } + + @Test + public void getConnection_LargeMessage() throws NoSuchFieldException, IllegalAccessException + { + // just need a serializer to report a size, as fake as it may be + IVersionedSerializer serializer = new IVersionedSerializer() + { + public void serialize(Object o, DataOutputPlus out, int version) + { + + } + + public Object deserialize(DataInputPlus in, int version) + { + return null; + } + + public long serializedSize(Object o, int version) + { + return OutboundConnections.LARGE_MESSAGE_THRESHOLD + 1; + } + }; + Verb._TEST_2.unsafeSetSerializer(() -> serializer); + Message message = Message.out(Verb._TEST_2, "payload"); + Assert.assertEquals(ConnectionType.LARGE_MESSAGES, connections.connectionFor(message).type()); + } + + @Test + public void close_SoftClose() throws ExecutionException, InterruptedException, TimeoutException + { + for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) + Assert.assertFalse(connections.connectionFor(type).isClosed()); + connections.close(true).get(10L, TimeUnit.SECONDS); + for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) + Assert.assertTrue(connections.connectionFor(type).isClosed()); + } + + @Test + public void close_NotSoftClose() throws ExecutionException, InterruptedException, TimeoutException + { + for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) + Assert.assertFalse(connections.connectionFor(type).isClosed()); + connections.close(false).get(10L, TimeUnit.SECONDS); + for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) + Assert.assertTrue(connections.connectionFor(type).isClosed()); + } + + @Test + public void reconnectWithNewIp() throws InterruptedException + { + for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) + { + Assert.assertEquals(REMOTE_ADDR, connections.connectionFor(type).settings().connectTo); + } + + connections.reconnectWithNewIp(RECONNECT_ADDR).await(); + + for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) + { + Assert.assertEquals(RECONNECT_ADDR, connections.connectionFor(type).settings().connectTo); + } + } + +// @Test +// public void timeoutCounter() +// { +// long originalValue = connections.getTimeouts(); +// connections.incrementTimeout(); +// Assert.assertEquals(originalValue + 1, connections.getTimeouts()); +// } +} diff --git a/test/unit/org/apache/cassandra/net/OutboundMessageQueueTest.java b/test/unit/org/apache/cassandra/net/OutboundMessageQueueTest.java new file mode 100644 index 000000000000..db571ac2e7bf --- /dev/null +++ b/test/unit/org/apache/cassandra/net/OutboundMessageQueueTest.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.concurrent.CountDownLatch; + +import com.google.common.util.concurrent.Uninterruptibles; + +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.OutboundMessageQueue; +import org.apache.cassandra.net.Verb; + +import static org.apache.cassandra.net.NoPayload.noPayload; + +// TODO: incomplete +public class OutboundMessageQueueTest +{ + + @BeforeClass + public static void init() + { + DatabaseDescriptor.daemonInitialization(); + } + + @Test + public void testRemove() throws InterruptedException + { + final Message m1 = Message.out(Verb._TEST_1, noPayload); + final Message m2 = Message.out(Verb._TEST_1, noPayload); + final Message m3 = Message.out(Verb._TEST_1, noPayload); + + final OutboundMessageQueue queue = new OutboundMessageQueue(message -> true); + queue.add(m1); + queue.add(m2); + queue.add(m3); + + Assert.assertTrue(queue.remove(m1)); + Assert.assertFalse(queue.remove(m1)); + + CountDownLatch locked = new CountDownLatch(1); + CountDownLatch lockUntil = new CountDownLatch(1); + new Thread(() -> { + try (OutboundMessageQueue.WithLock lock = queue.lockOrCallback(0, () -> {})) + { + locked.countDown(); + Uninterruptibles.awaitUninterruptibly(lockUntil); + } + }).start(); + Uninterruptibles.awaitUninterruptibly(locked); + + CountDownLatch start = new CountDownLatch(2); + CountDownLatch finish = new CountDownLatch(2); + new Thread(() -> { + start.countDown(); + Assert.assertTrue(queue.remove(m2)); + finish.countDown(); + }).start(); + new Thread(() -> { + start.countDown(); + Assert.assertTrue(queue.remove(m3)); + finish.countDown(); + }).start(); + Uninterruptibles.awaitUninterruptibly(start); + lockUntil.countDown(); + Uninterruptibles.awaitUninterruptibly(finish); + + try (OutboundMessageQueue.WithLock lock = queue.lockOrCallback(0, () -> {})) + { + Assert.assertNull(lock.peek()); + } + } + +} diff --git a/test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java b/test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java new file mode 100644 index 000000000000..c4fd55a8aa27 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/PrunableArrayQueueTest.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import org.junit.Test; + +import org.apache.cassandra.net.PrunableArrayQueue; + +import static org.junit.Assert.*; + +public class PrunableArrayQueueTest +{ + private final PrunableArrayQueue queue = new PrunableArrayQueue<>(8); + + @Test + public void testIsEmptyWhenEmpty() + { + assertTrue(queue.isEmpty()); + } + + @Test + public void testIsEmptyWhenNotEmpty() + { + queue.offer(0); + assertFalse(queue.isEmpty()); + } + + @Test + public void testEmptyPeek() + { + assertNull(queue.peek()); + } + + @Test + public void testNonEmptyPeek() + { + queue.offer(0); + assertEquals((Integer) 0, queue.peek()); + } + + @Test + public void testEmptyPoll() + { + assertNull(queue.poll()); + } + + @Test + public void testNonEmptyPoll() + { + queue.offer(0); + assertEquals((Integer) 0, queue.poll()); + } + + @Test + public void testTransfersInCorrectOrder() + { + for (int i = 0; i < 1024; i++) + queue.offer(i); + + for (int i = 0; i < 1024; i++) + assertEquals((Integer) i, queue.poll()); + + assertTrue(queue.isEmpty()); + } + + @Test + public void testTransfersInCorrectOrderWhenInterleaved() + { + for (int i = 0; i < 1024; i++) + { + queue.offer(i); + assertEquals((Integer) i, queue.poll()); + } + + assertTrue(queue.isEmpty()); + } + + @Test + public void testPrune() + { + for (int i = 0; i < 1024; i++) + queue.offer(i); + + class Pruner implements PrunableArrayQueue.Pruner + { + private int pruned, kept; + + public boolean shouldPrune(Integer val) + { + return val % 2 == 0; + } + + public void onPruned(Integer val) + { + pruned++; + } + + public void onKept(Integer val) + { + kept++; + } + } + + Pruner pruner = new Pruner(); + assertEquals(512, queue.prune(pruner)); + + assertEquals(512, pruner.kept); + assertEquals(512, pruner.pruned); + assertEquals(512, queue.size()); + + for (int i = 1; i < 1024; i += 2) + assertEquals((Integer) i, queue.poll()); + assertTrue(queue.isEmpty()); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/net/ResourceLimitsTest.java b/test/unit/org/apache/cassandra/net/ResourceLimitsTest.java new file mode 100644 index 000000000000..734d69afe776 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/ResourceLimitsTest.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.function.LongFunction; + +import org.junit.Test; + +import org.apache.cassandra.net.ResourceLimits.*; + +import static org.junit.Assert.*; + +public class ResourceLimitsTest +{ + @Test + public void testAllocatesWithinLimits() + { + testAllocatesWithinLimits(Basic::new); + testAllocatesWithinLimits(Concurrent::new); + } + + private void testAllocatesWithinLimits(LongFunction supplier) + { + Limit limit = supplier.apply(100); + + assertEquals(100, limit.limit()); + assertEquals(0, limit.using()); + assertEquals(100, limit.remaining()); + + assertTrue(limit.tryAllocate(10)); + assertEquals(10, limit.using()); + assertEquals(90, limit.remaining()); + + assertTrue(limit.tryAllocate(30)); + assertEquals(40, limit.using()); + assertEquals(60, limit.remaining()); + + assertTrue(limit.tryAllocate(60)); + assertEquals(100, limit.using()); + assertEquals(0, limit.remaining()); + } + + @Test + public void testFailsToAllocateOverCapacity() + { + testFailsToAllocateOverCapacity(Basic::new); + testFailsToAllocateOverCapacity(Concurrent::new); + } + + private void testFailsToAllocateOverCapacity(LongFunction supplier) + { + Limit limit = supplier.apply(100); + + assertEquals(100, limit.limit()); + assertEquals(0, limit.using()); + assertEquals(100, limit.remaining()); + + assertTrue(limit.tryAllocate(10)); + assertEquals(10, limit.using()); + assertEquals(90, limit.remaining()); + + assertFalse(limit.tryAllocate(91)); + assertEquals(10, limit.using()); + assertEquals(90, limit.remaining()); + } + + @Test + public void testRelease() + { + testRelease(Basic::new); + testRelease(Concurrent::new); + } + + private void testRelease(LongFunction supplier) + { + Limit limit = supplier.apply(100); + + assertEquals(100, limit.limit()); + assertEquals(0, limit.using()); + assertEquals(100, limit.remaining()); + + assertTrue(limit.tryAllocate(10)); + assertTrue(limit.tryAllocate(30)); + assertTrue(limit.tryAllocate(60)); + assertEquals(100, limit.using()); + assertEquals(0, limit.remaining()); + + limit.release(10); + assertEquals(90, limit.using()); + assertEquals(10, limit.remaining()); + + limit.release(30); + assertEquals(60, limit.using()); + assertEquals(40, limit.remaining()); + + limit.release(60); + assertEquals(0, limit.using()); + assertEquals(100, limit.remaining()); + } + + @Test + public void testConcurrentLimit() throws Exception + { + int numThreads = 4; + int numPermitsPerThread = 1_000_000; + int numPermits = numThreads * numPermitsPerThread; + + CountDownLatch latch = new CountDownLatch(numThreads); + Limit limit = new Concurrent(numPermits); + + class Worker implements Runnable + { + public void run() + { + for (int i = 0; i < numPermitsPerThread; i += 10) + assertTrue(limit.tryAllocate(10)); + + for (int i = 0; i < numPermitsPerThread; i += 10) + limit.release(10); + + latch.countDown(); + } + } + + Executor executor = Executors.newFixedThreadPool(numThreads); + for (int i = 0; i < numThreads; i++) + executor.execute(new Worker()); + latch.await(10, TimeUnit.SECONDS); + + assertEquals(0, limit.using()); + assertEquals(numPermits, limit.remaining()); + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/net/SocketUtils.java b/test/unit/org/apache/cassandra/net/SocketUtils.java new file mode 100644 index 000000000000..a0a149029b24 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/SocketUtils.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.net.ServerSocket; + +import com.google.common.base.Throwables; + +public class SocketUtils +{ + public static synchronized int findAvailablePort() throws RuntimeException + { + ServerSocket ss = null; + try + { + // let the system pick an ephemeral port + ss = new ServerSocket(0); + ss.setReuseAddress(true); + return ss.getLocalPort(); + } + catch (IOException e) + { + throw Throwables.propagate(e); + } + finally + { + if (ss != null) + { + try + { + ss.close(); + } + catch (IOException e) + { + Throwables.propagate(e); + } + } + } + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/net/StartupClusterConnectivityCheckerTest.java b/test/unit/org/apache/cassandra/net/StartupClusterConnectivityCheckerTest.java index af72456ec444..0785f277daba 100644 --- a/test/unit/org/apache/cassandra/net/StartupClusterConnectivityCheckerTest.java +++ b/test/unit/org/apache/cassandra/net/StartupClusterConnectivityCheckerTest.java @@ -19,11 +19,11 @@ package org.apache.cassandra.net; import java.net.UnknownHostException; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.function.BiPredicate; import org.junit.After; import org.junit.Assert; @@ -38,8 +38,6 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE; - public class StartupClusterConnectivityCheckerTest { private StartupClusterConnectivityChecker localQuorumConnectivityChecker; @@ -108,32 +106,30 @@ public void setUp() throws UnknownHostException @After public void tearDown() { - MessagingService.instance().clearMessageSinks(); + MessagingService.instance().outboundSink.clear(); } @Test public void execute_HappyPath() { Sink sink = new Sink(true, true, peers); - MessagingService.instance().addMessageSink(sink); + MessagingService.instance().outboundSink.add(sink); Assert.assertTrue(localQuorumConnectivityChecker.execute(peers, this::getDatacenter)); - Assert.assertTrue(checkAllConnectionTypesSeen(sink)); } @Test public void execute_NotAlive() { Sink sink = new Sink(false, true, peers); - MessagingService.instance().addMessageSink(sink); + MessagingService.instance().outboundSink.add(sink); Assert.assertFalse(localQuorumConnectivityChecker.execute(peers, this::getDatacenter)); - Assert.assertTrue(checkAllConnectionTypesSeen(sink)); } @Test public void execute_NoConnectionsAcks() { Sink sink = new Sink(true, false, peers); - MessagingService.instance().addMessageSink(sink); + MessagingService.instance().outboundSink.add(sink); Assert.assertFalse(localQuorumConnectivityChecker.execute(peers, this::getDatacenter)); } @@ -143,12 +139,12 @@ public void execute_LocalQuorum() // local peer plus 3 peers from same dc shouldn't pass (4/6) Set available = new HashSet<>(); copyCount(peersAMinusLocal, available, NUM_PER_DC - 3); - checkAvailable(localQuorumConnectivityChecker, available, false, true); + checkAvailable(localQuorumConnectivityChecker, available, false); // local peer plus 4 peers from same dc should pass (5/6) available.clear(); copyCount(peersAMinusLocal, available, NUM_PER_DC - 2); - checkAvailable(localQuorumConnectivityChecker, available, true, true); + checkAvailable(localQuorumConnectivityChecker, available, true); } @Test @@ -159,56 +155,45 @@ public void execute_GlobalQuorum() copyCount(peersAMinusLocal, available, NUM_PER_DC - 2); copyCount(peersB, available, NUM_PER_DC - 2); copyCount(peersC, available, NUM_PER_DC - 1); - checkAvailable(globalQuorumConnectivityChecker, available, false, true); + checkAvailable(globalQuorumConnectivityChecker, available, false); // All three datacenters should be able to have a single node down available.clear(); copyCount(peersAMinusLocal, available, NUM_PER_DC - 2); copyCount(peersB, available, NUM_PER_DC - 1); copyCount(peersC, available, NUM_PER_DC - 1); - checkAvailable(globalQuorumConnectivityChecker, available, true, true); + checkAvailable(globalQuorumConnectivityChecker, available, true); // Everything being up should work of course available.clear(); copyCount(peersAMinusLocal, available, NUM_PER_DC - 1); copyCount(peersB, available, NUM_PER_DC); copyCount(peersC, available, NUM_PER_DC); - checkAvailable(globalQuorumConnectivityChecker, available, true, true); + checkAvailable(globalQuorumConnectivityChecker, available, true); } @Test public void execute_Noop() { - checkAvailable(noopChecker, new HashSet<>(), true, false); + checkAvailable(noopChecker, new HashSet<>(), true); } @Test public void execute_ZeroWaitHasConnections() throws InterruptedException { Sink sink = new Sink(true, true, new HashSet<>()); - MessagingService.instance().addMessageSink(sink); + MessagingService.instance().outboundSink.add(sink); Assert.assertFalse(zeroWaitChecker.execute(peers, this::getDatacenter)); - boolean hasConnections = false; - for (int i = 0; i < TIMEOUT_NANOS; i+= 10) - { - hasConnections = checkAllConnectionTypesSeen(sink); - if (hasConnections) - break; - Thread.sleep(0, 10); - } - MessagingService.instance().clearMessageSinks(); - Assert.assertTrue(hasConnections); + MessagingService.instance().outboundSink.clear(); } private void checkAvailable(StartupClusterConnectivityChecker checker, Set available, - boolean shouldPass, boolean checkConnections) + boolean shouldPass) { Sink sink = new Sink(true, true, available); - MessagingService.instance().addMessageSink(sink); + MessagingService.instance().outboundSink.add(sink); Assert.assertEquals(shouldPass, checker.execute(peers, this::getDatacenter)); - if (checkConnections) - Assert.assertTrue(checkAllConnectionTypesSeen(sink)); - MessagingService.instance().clearMessageSinks(); + MessagingService.instance().outboundSink.clear(); } private void copyCount(Set source, Set dest, int count) @@ -223,25 +208,7 @@ private void copyCount(Set source, Set d } } - private boolean checkAllConnectionTypesSeen(Sink sink) - { - boolean result = true; - for (InetAddressAndPort peer : peers) - { - if (peer.equals(FBUtilities.getBroadcastAddressAndPort())) - continue; - ConnectionTypeRecorder recorder = sink.seenConnectionRequests.get(peer); - result = recorder != null; - if (!result) - break; - - result = recorder.seenSmallMessageRequest; - result &= recorder.seenLargeMessageRequest; - } - return result; - } - - private static class Sink implements IMessageSink + private static class Sink implements BiPredicate, InetAddressAndPort> { private final boolean markAliveInGossip; private final boolean processConnectAck; @@ -257,39 +224,25 @@ private static class Sink implements IMessageSink } @Override - public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to) + public boolean test(Message message, InetAddressAndPort to) { ConnectionTypeRecorder recorder = seenConnectionRequests.computeIfAbsent(to, inetAddress -> new ConnectionTypeRecorder()); - if (message.connectionType == SMALL_MESSAGE) - { - Assert.assertFalse(recorder.seenSmallMessageRequest); - recorder.seenSmallMessageRequest = true; - } - else - { - Assert.assertFalse(recorder.seenLargeMessageRequest); - recorder.seenLargeMessageRequest = true; - } if (!aliveHosts.contains(to)) return false; if (processConnectAck) { - MessageIn msgIn = MessageIn.create(to, message.payload, Collections.emptyMap(), MessagingService.Verb.REQUEST_RESPONSE, 1); - MessagingService.instance().getRegisteredCallback(id).callback.response(msgIn); + Message msgIn = Message.builder(Verb.REQUEST_RSP, message.payload) + .from(to) + .build(); + MessagingService.instance().callbacks.get(message.id(), to).callback.onResponse(msgIn); } if (markAliveInGossip) Gossiper.runInGossipStageBlocking(() -> Gossiper.instance.realMarkAlive(to, new EndpointState(new HeartBeatState(1, 1)))); return false; } - - @Override - public boolean allowIncomingMessage(MessageIn message, int id) - { - return false; - } } private static class ConnectionTypeRecorder diff --git a/test/unit/org/apache/cassandra/net/TestAbstractAsyncPromise.java b/test/unit/org/apache/cassandra/net/TestAbstractAsyncPromise.java new file mode 100644 index 000000000000..fd61b093cbfa --- /dev/null +++ b/test/unit/org/apache/cassandra/net/TestAbstractAsyncPromise.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.Assert; + +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import io.netty.util.concurrent.Promise; + +abstract class TestAbstractAsyncPromise extends TestAbstractPromise +{ + void testOneSuccess(Promise promise, boolean setUncancellable, boolean tryOrSet, V value, V otherValue) + { + List results = new ArrayList<>(); + List order = new ArrayList<>(); + class ListenerFactory + { + int count = 0; + + public GenericFutureListener> get() + { + int id = count++; + return p -> { results.add(p.getNow()); order.add(id); }; + } + public GenericFutureListener> getRecursive() + { + int id = count++; + return p -> { promise.addListener(get()); results.add(p.getNow()); order.add(id); }; + } + } + ListenerFactory listeners = new ListenerFactory(); + Async async = new Async(); + promise.addListener(listeners.get()); + promise.addListeners(listeners.getRecursive(), listeners.get()); + promise.addListener(listeners.getRecursive()); + success(promise, Promise::getNow, null); + success(promise, Promise::isSuccess, false); + success(promise, Promise::isDone, false); + success(promise, Promise::isCancelled, false); + success(promise, Promise::isCancellable, true); + if (setUncancellable) + { + success(promise, Promise::setUncancellable, true); + success(promise, Promise::setUncancellable, true); + success(promise, p -> p.cancel(true), false); + success(promise, p -> p.cancel(false), false); + } + success(promise, Promise::isCancellable, !setUncancellable); + async.success(promise, Promise::get, value); + async.success(promise, p -> p.get(1L, TimeUnit.SECONDS), value); + async.success(promise, Promise::await, promise); + async.success(promise, Promise::awaitUninterruptibly, promise); + async.success(promise, p -> p.await(1L, TimeUnit.SECONDS), true); + async.success(promise, p -> p.await(1000L), true); + async.success(promise, p -> p.awaitUninterruptibly(1L, TimeUnit.SECONDS), true); + async.success(promise, p -> p.awaitUninterruptibly(1000L), true); + async.success(promise, Promise::sync, promise); + async.success(promise, Promise::syncUninterruptibly, promise); + if (tryOrSet) promise.trySuccess(value); + else promise.setSuccess(value); + success(promise, p -> p.cancel(true), false); + success(promise, p -> p.cancel(false), false); + failure(promise, p -> p.setSuccess(null), IllegalStateException.class); + failure(promise, p -> p.setFailure(new NullPointerException()), IllegalStateException.class); + success(promise, Promise::getNow, value); + success(promise, p -> p.trySuccess(otherValue), false); + success(promise, p -> p.tryFailure(new NullPointerException()), false); + success(promise, Promise::getNow, value); + success(promise, Promise::cause, null); + promise.addListener(listeners.get()); + promise.addListeners(listeners.getRecursive(), listeners.get()); + promise.addListener(listeners.getRecursive()); + success(promise, Promise::isSuccess, true); + success(promise, Promise::isDone, true); + success(promise, Promise::isCancelled, false); + success(promise, Promise::isCancellable, false); + async.verify(); + Assert.assertEquals(listeners.count, results.size()); + Assert.assertEquals(listeners.count, order.size()); + for (V result : results) + Assert.assertEquals(value, result); + for (int i = 0 ; i < order.size() ; ++i) + Assert.assertEquals(i, order.get(i).intValue()); + } + + void testOneFailure(Promise promise, boolean setUncancellable, boolean tryOrSet, Throwable cause, V otherValue) + { + List results = new ArrayList<>(); + List order = new ArrayList<>(); + Async async = new Async(); + class ListenerFactory + { + int count = 0; + + public GenericFutureListener> get() + { + int id = count++; + return p -> { results.add(p.cause()); order.add(id); }; + } + public GenericFutureListener> getRecursive() + { + int id = count++; + return p -> { promise.addListener(get()); results.add(p.cause()); order.add(id); }; + } + } + ListenerFactory listeners = new ListenerFactory(); + promise.addListener(listeners.get()); + promise.addListeners(listeners.getRecursive(), listeners.get()); + promise.addListener(listeners.getRecursive()); + success(promise, Promise::isSuccess, false); + success(promise, Promise::isDone, false); + success(promise, Promise::isCancelled, false); + success(promise, Promise::isCancellable, true); + if (setUncancellable) + { + success(promise, Promise::setUncancellable, true); + success(promise, Promise::setUncancellable, true); + success(promise, p -> p.cancel(true), false); + success(promise, p -> p.cancel(false), false); + } + success(promise, Promise::isCancellable, !setUncancellable); + success(promise, Promise::getNow, null); + success(promise, Promise::cause, null); + async.failure(promise, Promise::get, ExecutionException.class); + async.failure(promise, p -> p.get(1L, TimeUnit.SECONDS), ExecutionException.class); + async.success(promise, Promise::await, promise); + async.success(promise, Promise::awaitUninterruptibly, promise); + async.success(promise, p -> p.await(1L, TimeUnit.SECONDS), true); + async.success(promise, p -> p.await(1000L), true); + async.success(promise, p -> p.awaitUninterruptibly(1L, TimeUnit.SECONDS), true); + async.success(promise, p -> p.awaitUninterruptibly(1000L), true); + async.failure(promise, Promise::sync, cause); + async.failure(promise, Promise::syncUninterruptibly, cause); + if (tryOrSet) promise.tryFailure(cause); + else promise.setFailure(cause); + success(promise, p -> p.cancel(true), false); + success(promise, p -> p.cancel(false), false); + failure(promise, p -> p.setSuccess(null), IllegalStateException.class); + failure(promise, p -> p.setFailure(new NullPointerException()), IllegalStateException.class); + success(promise, Promise::cause, cause); + success(promise, Promise::getNow, null); + success(promise, p -> p.trySuccess(otherValue), false); + success(promise, p -> p.tryFailure(new NullPointerException()), false); + success(promise, Promise::getNow, null); + success(promise, Promise::cause, cause); + promise.addListener(listeners.get()); + promise.addListeners(listeners.getRecursive(), listeners.get()); + promise.addListener(listeners.getRecursive()); + success(promise, Promise::isSuccess, false); + success(promise, Promise::isDone, true); + success(promise, Promise::isCancelled, false); + success(promise, Promise::isCancellable, false); + async.verify(); + Assert.assertEquals(listeners.count, results.size()); + Assert.assertEquals(listeners.count, order.size()); + for (Throwable result : results) + Assert.assertEquals(cause, result); + for (int i = 0 ; i < order.size() ; ++i) + Assert.assertEquals(i, order.get(i).intValue()); + } + + public void testOneCancellation(Promise promise, boolean interruptIfRunning, V otherValue) + { + Async async = new Async(); + success(promise, Promise::isCancellable, true); + success(promise, Promise::getNow, null); + success(promise, Promise::cause, null); + async.failure(promise, Promise::get, CancellationException.class); + async.failure(promise, p -> p.get(1L, TimeUnit.SECONDS), CancellationException.class); + async.success(promise, Promise::await, promise); + async.success(promise, Promise::awaitUninterruptibly, promise); + async.success(promise, p -> p.await(1L, TimeUnit.SECONDS), true); + async.success(promise, p -> p.await(1000L), true); + async.success(promise, p -> p.awaitUninterruptibly(1L, TimeUnit.SECONDS), true); + async.success(promise, p -> p.awaitUninterruptibly(1000L), true); + async.failure(promise, Promise::sync, CancellationException.class); + async.failure(promise, Promise::syncUninterruptibly, CancellationException.class); + promise.cancel(interruptIfRunning); + failure(promise, p -> p.setFailure(null), IllegalStateException.class); + failure(promise, p -> p.setFailure(null), IllegalStateException.class); + Assert.assertTrue(promise.cause() instanceof CancellationException); + success(promise, Promise::getNow, null); + success(promise, p -> p.trySuccess(otherValue), false); + success(promise, Promise::getNow, null); + Assert.assertTrue(promise.cause() instanceof CancellationException); + success(promise, Promise::isSuccess, false); + success(promise, Promise::isDone, true); + success(promise, Promise::isCancelled, true); + success(promise, Promise::isCancellable, false); + async.verify(); + } + + + public void testOneTimeout(Promise promise, boolean setUncancellable) + { + Async async = new Async(); + if (setUncancellable) + success(promise, Promise::setUncancellable, true); + success(promise, Promise::isCancellable, !setUncancellable); + async.failure(promise, p -> p.get(1L, TimeUnit.MILLISECONDS), TimeoutException.class); + async.success(promise, p -> p.await(1L, TimeUnit.MILLISECONDS), false); + async.success(promise, p -> p.await(1L), false); + async.success(promise, p -> p.awaitUninterruptibly(1L, TimeUnit.MILLISECONDS), false); + async.success(promise, p -> p.awaitUninterruptibly(1L), false); + Uninterruptibles.sleepUninterruptibly(10L, TimeUnit.MILLISECONDS); + async.verify(); + } + +} diff --git a/test/unit/org/apache/cassandra/net/TestAbstractPromise.java b/test/unit/org/apache/cassandra/net/TestAbstractPromise.java new file mode 100644 index 000000000000..963c61fe35ee --- /dev/null +++ b/test/unit/org/apache/cassandra/net/TestAbstractPromise.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; + +import org.junit.Assert; + +import io.netty.util.concurrent.Promise; +import net.openhft.chronicle.core.util.ThrowingBiConsumer; +import net.openhft.chronicle.core.util.ThrowingConsumer; +import net.openhft.chronicle.core.util.ThrowingFunction; + +abstract class TestAbstractPromise +{ + final ExecutorService exec = Executors.newCachedThreadPool(); + + class Async + { + final List> waitingOn = new ArrayList<>(); + void verify() + { + for (int i = 0 ; i < waitingOn.size() ; ++i) + { + try + { + waitingOn.get(i).accept(100L, TimeUnit.MILLISECONDS); + } + catch (Throwable t) + { + throw new AssertionError("" + i, t); + } + } + } + void failure(Promise promise, ThrowingConsumer, ?> action, Throwable failsWith) + { + waitingOn.add(exec.submit(() -> TestAbstractPromise.failure(promise, action, failsWith))::get); + } + void failure(Promise promise, ThrowingConsumer, ?> action, Class failsWith) + { + waitingOn.add(exec.submit(() -> TestAbstractPromise.failure(promise, action, failsWith))::get); + } + void failure(Promise promise, ThrowingConsumer, ?> action, Predicate failsWith) + { + waitingOn.add(exec.submit(() -> TestAbstractPromise.failure(promise, action, failsWith))::get); + } +

, R> void success(P promise, ThrowingFunction action, R result) + { + waitingOn.add(exec.submit(() -> TestAbstractPromise.success(promise, action, result))::get); + } + } + + private static void failure(Promise promise, ThrowingConsumer, ?> action, Throwable failsWith) + { + failure(promise, action, t -> Objects.equals(failsWith, t)); + } + + static void failure(Promise promise, ThrowingConsumer, ?> action, Class failsWith) + { + failure(promise, action, failsWith::isInstance); + } + + private static void failure(Promise promise, ThrowingConsumer, ?> action, Predicate failsWith) + { + Throwable fail = null; + try + { + action.accept(promise); + } + catch (Throwable t) + { + fail = t; + } + if (!failsWith.test(fail)) + throw new AssertionError(fail); + } + + static

, R> void success(P promise, ThrowingFunction action, R result) + { + try + { + Assert.assertEquals(result, action.apply(promise)); + } + catch (Throwable t) + { + throw new AssertionError(t); + } + } + +} diff --git a/test/unit/org/apache/cassandra/net/TestChannel.java b/test/unit/org/apache/cassandra/net/TestChannel.java new file mode 100644 index 000000000000..feddab0c93e0 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/TestChannel.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelOutboundBuffer; +import io.netty.channel.FileRegion; +import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.cassandra.net.FrameEncoder; +import org.apache.cassandra.net.GlobalBufferPoolAllocator; + +public class TestChannel extends EmbeddedChannel +{ + final int inFlightLimit; + int inFlight; + + ChannelOutboundBuffer flush; + long flushBytes; + + public TestChannel(int inFlightLimit) + { + this.inFlightLimit = inFlightLimit; + } + + // we override ByteBuf to prevent retain() from working, to avoid release() since it is not needed in our usage + // since the lifetime must live longer, we simply copy any outbound ByteBuf here for our tests + protected void doWrite(ChannelOutboundBuffer in) + { + assert flush == null || flush == in; + doWrite(in, in.totalPendingWriteBytes()); + } + + private void doWrite(ChannelOutboundBuffer flush, long flushBytes) + { + while (true) { + Object msg = flush.current(); + if (msg == null) { + this.flush = null; + this.flushBytes = 0; + return; + } + + if (inFlight >= inFlightLimit) + { + this.flush = flush; + this.flushBytes = flushBytes; + return; + } + + ByteBuf buf; + if (msg instanceof FileRegion) + { + buf = GlobalBufferPoolAllocator.instance.directBuffer((int) ((FileRegion) msg).count()); + try + { + ((FileRegion) msg).transferTo(new WritableByteChannel() + { + public int write(ByteBuffer src) + { + buf.setBytes(0, src); + return buf.writerIndex(); + } + + public boolean isOpen() { return false; } + + public void close() { } + }, 0); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + } + else if (msg instanceof ByteBuf) + { + buf = ((ByteBuf)msg).copy(); + } + else if (msg instanceof FrameEncoder.Payload) + { + buf = Unpooled.wrappedBuffer(((FrameEncoder.Payload)msg).buffer).copy(); + } + else + { + System.err.println("Unexpected message type " + msg); + throw new IllegalArgumentException(); + } + + inFlight += buf.readableBytes(); + handleOutboundMessage(buf); + flush.remove(); + } + } + + public T readOutbound() + { + T msg = super.readOutbound(); + if (msg instanceof ByteBuf) + { + inFlight -= ((ByteBuf) msg).readableBytes(); + if (flush != null && inFlight < inFlightLimit) + doWrite(flush, flushBytes); + } + return msg; + } +} + diff --git a/test/unit/org/apache/cassandra/net/async/TestScheduledFuture.java b/test/unit/org/apache/cassandra/net/TestScheduledFuture.java similarity index 97% rename from test/unit/org/apache/cassandra/net/async/TestScheduledFuture.java rename to test/unit/org/apache/cassandra/net/TestScheduledFuture.java index f5475ce2b0c3..456f8c4a852b 100644 --- a/test/unit/org/apache/cassandra/net/async/TestScheduledFuture.java +++ b/test/unit/org/apache/cassandra/net/TestScheduledFuture.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.net.async; +package org.apache.cassandra.net; import java.util.concurrent.Delayed; import java.util.concurrent.ExecutionException; diff --git a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java index e226d32e75b0..b4bf8b7d62ef 100644 --- a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java +++ b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java @@ -30,8 +30,6 @@ import org.apache.cassandra.db.RegularAndStaticColumns; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.ReplicaUtils; -import org.apache.cassandra.net.MessagingService.Verb; import org.apache.cassandra.schema.MockSchema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.paxos.Commit; @@ -50,8 +48,8 @@ public static void initDD() @Test public void testShouldHint() throws Exception { - testShouldHint(Verb.COUNTER_MUTATION, ConsistencyLevel.ALL, true, false); - for (Verb verb : new Verb[] { Verb.PAXOS_COMMIT, Verb.MUTATION }) + testShouldHint(Verb.COUNTER_MUTATION_REQ, ConsistencyLevel.ALL, true, false); + for (Verb verb : new Verb[] { Verb.PAXOS_COMMIT_REQ, Verb.MUTATION_REQ }) { testShouldHint(verb, ConsistencyLevel.ALL, true, true); testShouldHint(verb, ConsistencyLevel.ANY, true, false); @@ -62,11 +60,11 @@ public void testShouldHint() throws Exception private void testShouldHint(Verb verb, ConsistencyLevel cl, boolean allowHints, boolean expectHint) throws Exception { TableMetadata metadata = MockSchema.newTableMetadata("", ""); - Object payload = verb == Verb.PAXOS_COMMIT + Object payload = verb == Verb.PAXOS_COMMIT_REQ ? new Commit(UUID.randomUUID(), new PartitionUpdate.Builder(metadata, ByteBufferUtil.EMPTY_BYTE_BUFFER, RegularAndStaticColumns.NONE, 1).build()) : new Mutation(PartitionUpdate.simpleBuilder(metadata, "").build()); - WriteCallbackInfo wcbi = new WriteCallbackInfo(full(InetAddressAndPort.getByName("192.168.1.1")), null, new MessageOut(verb, payload, null), null, cl, allowHints); + RequestCallbacks.WriteCallbackInfo wcbi = new RequestCallbacks.WriteCallbackInfo(Message.out(verb, payload), full(InetAddressAndPort.getByName("192.168.1.1")), null, cl, allowHints); Assert.assertEquals(expectHint, wcbi.shouldHint()); if (expectHint) { diff --git a/test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java b/test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java deleted file mode 100644 index 959c37aa2528..000000000000 --- a/test/unit/org/apache/cassandra/net/async/ByteBufDataOutputPlusTest.java +++ /dev/null @@ -1,178 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.nio.ByteBuffer; - -import org.junit.After; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.PooledByteBufAllocator; -import io.netty.buffer.Unpooled; -import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.db.RowUpdateBuilder; -import org.apache.cassandra.db.compaction.CompactionManager; -import org.apache.cassandra.db.marshal.AsciiType; -import org.apache.cassandra.db.marshal.BytesType; -import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.io.util.Memory; -import org.apache.cassandra.io.util.SafeMemory; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.schema.KeyspaceParams; - -public class ByteBufDataOutputPlusTest -{ - private static final String KEYSPACE1 = "NettyPipilineTest"; - private static final String STANDARD1 = "Standard1"; - private static final int columnCount = 128; - - private ByteBuf buf; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - SchemaLoader.prepareServer(); - SchemaLoader.createKeyspace(KEYSPACE1, - KeyspaceParams.simple(1), - SchemaLoader.standardCFMD(KEYSPACE1, STANDARD1, columnCount, AsciiType.instance, BytesType.instance)); - CompactionManager.instance.disableAutoCompaction(); - } - - @After - public void tearDown() - { - if (buf != null) - buf.release(); - } - - @Test - public void compareBufferSizes() throws IOException - { - final int currentFrameSize = getMessage().message.serializedSize(MessagingService.current_version); - - ByteBuffer buffer = ByteBuffer.allocateDirect(currentFrameSize); //bufferedOut.nioBuffer(0, bufferedOut.writableBytes()); - getMessage().message.serialize(new DataOutputBuffer(buffer), MessagingService.current_version); - Assert.assertFalse(buffer.hasRemaining()); - Assert.assertEquals(buffer.capacity(), buffer.position()); - - ByteBuf bbosOut = PooledByteBufAllocator.DEFAULT.ioBuffer(currentFrameSize, currentFrameSize); - try - { - getMessage().message.serialize(new ByteBufDataOutputPlus(bbosOut), MessagingService.current_version); - - Assert.assertFalse(bbosOut.isWritable()); - Assert.assertEquals(bbosOut.capacity(), bbosOut.writerIndex()); - - Assert.assertEquals(buffer.position(), bbosOut.writerIndex()); - for (int i = 0; i < currentFrameSize; i++) - { - Assert.assertEquals(buffer.get(i), bbosOut.getByte(i)); - } - } - finally - { - bbosOut.release(); - } - } - - private QueuedMessage getMessage() - { - ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1); - ByteBuffer buf = ByteBuffer.allocate(1 << 10); - RowUpdateBuilder rowUpdateBuilder = new RowUpdateBuilder(cfs1.metadata.get(), 0, "k") - .clustering("bytes"); - for (int i = 0; i < columnCount; i++) - rowUpdateBuilder.add("val" + i, buf); - - Mutation mutation = rowUpdateBuilder.build(); - return new QueuedMessage(mutation.createMessage(), 42); - } - - @Test - public void compareDOS() throws IOException - { - buf = PooledByteBufAllocator.DEFAULT.ioBuffer(1024, 1024); - ByteBuffer buffer = ByteBuffer.allocateDirect(1024); - - ByteBufDataOutputPlus byteBufDataOutputPlus = new ByteBufDataOutputPlus(buf); - DataOutputBuffer dataOutputBuffer = new DataOutputBuffer(buffer); - - write(byteBufDataOutputPlus); - write(dataOutputBuffer); - - Assert.assertEquals(buffer.position(), buf.writerIndex()); - for (int i = 0; i < buffer.position(); i++) - { - Assert.assertEquals(buffer.get(i), buf.getByte(i)); - } - } - - private void write(DataOutputPlus out) throws IOException - { - ByteBuffer b = ByteBuffer.allocate(8); - b.putLong(29811134237462734L); - out.write(b); - b = ByteBuffer.allocateDirect(8); - b.putDouble(92367.4253647890626); - out.write(b); - - out.writeInt(29319236); - - byte[] array = new byte[17]; - for (int i = 0; i < array.length; i++) - array[i] = (byte)i; - out.write(array, 0 , array.length); - - out.write(42); - out.writeUTF("This is a great string!!"); - out.writeByte(-100); - out.writeUnsignedVInt(3247634L); - out.writeVInt(12313695L); - out.writeBoolean(true); - out.writeShort(4371); - out.writeChar('j'); - out.writeLong(472348263487234L); - out.writeFloat(34534.12623F); - out.writeDouble(0.2384253D); - out.writeBytes("Write my bytes"); - out.writeChars("These are some swell chars"); - - Memory memory = new SafeMemory(8); - memory.setLong(0, -21365123651231L); - out.write(memory, 0, memory.size()); - memory.close(); - } - - @Test (expected = UnsupportedOperationException.class) - public void applyToChannel() throws IOException - { - ByteBufDataOutputPlus out = new ByteBufDataOutputPlus(Unpooled.wrappedBuffer(new byte[0])); - out.applyToChannel(null); - } -} diff --git a/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java b/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java deleted file mode 100644 index 02115124d4de..000000000000 --- a/test/unit/org/apache/cassandra/net/async/ChannelWriterTest.java +++ /dev/null @@ -1,314 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.Optional; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; - -import com.google.common.net.InetAddresses; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelOption; -import io.netty.channel.ChannelOutboundHandlerAdapter; -import io.netty.channel.ChannelPromise; -import io.netty.channel.WriteBufferWaterMark; -import io.netty.channel.embedded.EmbeddedChannel; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.ChannelWriter.CoalescingChannelWriter; -import org.apache.cassandra.utils.CoalescingStrategies; -import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy; - -import static org.apache.cassandra.net.MessagingService.Verb.ECHO; - -/** - * with the write_Coalescing_* methods, if there's data in the channel.unsafe().outboundBuffer() - * it means that there's something in the channel that hasn't yet been flushed to the transport (socket). - * once a flush occurs, there will be an entry in EmbeddedChannel's outboundQueue. those two facts are leveraged in these tests. - */ -public class ChannelWriterTest -{ - private static final int COALESCE_WINDOW_MS = 10; - - private EmbeddedChannel channel; - private ChannelWriter channelWriter; - private NonSendingOutboundMessagingConnection omc; - private Optional coalescingStrategy; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - } - - @Before - public void setup() - { - OutboundConnectionIdentifier id = OutboundConnectionIdentifier.small(InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 0), - InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0)); - channel = new EmbeddedChannel(); - omc = new NonSendingOutboundMessagingConnection(id, null, Optional.empty()); - channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()); - channel.pipeline().addFirst(new MessageOutHandler(id, MessagingService.current_version, channelWriter, () -> null)); - coalescingStrategy = CoalescingStrategies.newCoalescingStrategy(CoalescingStrategies.Strategy.FIXED.name(), COALESCE_WINDOW_MS, null, "test"); - } - - @Test - public void create_nonCoalescing() - { - Assert.assertSame(ChannelWriter.SimpleChannelWriter.class, ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()).getClass()); - } - - @Test - public void create_Coalescing() - { - Assert.assertSame(CoalescingChannelWriter.class, ChannelWriter.create(channel, omc::handleMessageResult, coalescingStrategy).getClass()); - } - - @Test - public void write_IsWritable() - { - Assert.assertTrue(channel.isWritable()); - Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true)); - Assert.assertTrue(channel.isWritable()); - Assert.assertTrue(channel.releaseOutbound()); - } - - @Test - public void write_NotWritable() - { - channel.config().setOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(1, 2)); - - // send one message through, which will trigger the writability check (and turn it off) - Assert.assertTrue(channel.isWritable()); - ByteBuf buf = channel.alloc().buffer(8, 8); - channel.unsafe().outboundBuffer().addMessage(buf, buf.capacity(), channel.newPromise()); - Assert.assertFalse(channel.isWritable()); - Assert.assertFalse(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true)); - Assert.assertFalse(channel.isWritable()); - Assert.assertFalse(channel.releaseOutbound()); - buf.release(); - } - - @Test - public void write_NotWritableButWriteAnyway() - { - channel.config().setOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(1, 2)); - - // send one message through, which will trigger the writability check (and turn it off) - Assert.assertTrue(channel.isWritable()); - ByteBuf buf = channel.alloc().buffer(8, 8); - channel.unsafe().outboundBuffer().addMessage(buf, buf.capacity(), channel.newPromise()); - Assert.assertFalse(channel.isWritable()); - Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), false)); - Assert.assertTrue(channel.isWritable()); - Assert.assertTrue(channel.releaseOutbound()); - } - - @Test - public void write_Coalescing_LostRaceForFlushTask() - { - CoalescingChannelWriter channelWriter = resetEnvForCoalescing(DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages()); - channelWriter.scheduledFlush.set(true); - Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0); - Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true)); - Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() > 0); - Assert.assertFalse(channel.releaseOutbound()); - Assert.assertTrue(channelWriter.scheduledFlush.get()); - } - - @Test - public void write_Coalescing_HitMinMessageCountForImmediateCoalesce() - { - CoalescingChannelWriter channelWriter = resetEnvForCoalescing(1); - - Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0); - Assert.assertFalse(channelWriter.scheduledFlush.get()); - Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true)); - - Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0); - Assert.assertTrue(channel.releaseOutbound()); - Assert.assertFalse(channelWriter.scheduledFlush.get()); - } - - @Test - public void write_Coalescing_ScheduleFlushTask() - { - CoalescingChannelWriter channelWriter = resetEnvForCoalescing(DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages()); - - Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0); - Assert.assertFalse(channelWriter.scheduledFlush.get()); - Assert.assertTrue(channelWriter.write(new QueuedMessage(new MessageOut<>(ECHO), 42), true)); - - Assert.assertTrue(channelWriter.scheduledFlush.get()); - Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() > 0); - Assert.assertTrue(channelWriter.scheduledFlush.get()); - - // this unfortunately know a little too much about how the sausage is made in CoalescingChannelWriter :-/ - channel.runScheduledPendingTasks(); - channel.runPendingTasks(); - Assert.assertTrue(channel.unsafe().outboundBuffer().totalPendingWriteBytes() == 0); - Assert.assertFalse(channelWriter.scheduledFlush.get()); - Assert.assertTrue(channel.releaseOutbound()); - } - - private CoalescingChannelWriter resetEnvForCoalescing(int minMessagesForCoalesce) - { - channel = new EmbeddedChannel(); - CoalescingChannelWriter cw = new CoalescingChannelWriter(channel, omc::handleMessageResult, coalescingStrategy.get(), minMessagesForCoalesce); - channel.pipeline().addFirst(new ChannelOutboundHandlerAdapter() - { - public void flush(ChannelHandlerContext ctx) throws Exception - { - cw.onTriggeredFlush(ctx); - } - }); - omc.setChannelWriter(cw); - return cw; - } - - @Test - public void writeBacklog_Empty() - { - BlockingQueue queue = new LinkedBlockingQueue<>(); - Assert.assertEquals(0, channelWriter.writeBacklog(queue, false)); - Assert.assertFalse(channel.releaseOutbound()); - } - - @Test - public void writeBacklog_ChannelNotWritable() - { - Assert.assertTrue(channel.isWritable()); - // force the channel to be non writable - channel.config().setOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(1, 2)); - ByteBuf buf = channel.alloc().buffer(8, 8); - channel.unsafe().outboundBuffer().addMessage(buf, buf.capacity(), channel.newPromise()); - Assert.assertFalse(channel.isWritable()); - - Assert.assertEquals(0, channelWriter.writeBacklog(new LinkedBlockingQueue<>(), false)); - Assert.assertFalse(channel.releaseOutbound()); - Assert.assertFalse(channel.isWritable()); - buf.release(); - } - - @Test - public void writeBacklog_NotEmpty() - { - BlockingQueue queue = new LinkedBlockingQueue<>(); - int count = 12; - for (int i = 0; i < count; i++) - queue.offer(new QueuedMessage(new MessageOut<>(ECHO), i)); - Assert.assertEquals(count, channelWriter.writeBacklog(queue, false)); - Assert.assertTrue(channel.releaseOutbound()); - } - - @Test - public void close() - { - Assert.assertFalse(channelWriter.isClosed()); - Assert.assertTrue(channel.isOpen()); - channelWriter.close(); - Assert.assertFalse(channel.isOpen()); - Assert.assertTrue(channelWriter.isClosed()); - } - - @Test - public void softClose() - { - Assert.assertFalse(channelWriter.isClosed()); - Assert.assertTrue(channel.isOpen()); - channelWriter.softClose(); - Assert.assertFalse(channel.isOpen()); - Assert.assertTrue(channelWriter.isClosed()); - } - - @Test - public void handleMessagePromise_FutureIsCancelled() - { - ChannelPromise promise = channel.newPromise(); - promise.cancel(false); - channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1), true); - Assert.assertTrue(channel.isActive()); - Assert.assertEquals(1, omc.getCompletedMessages().longValue()); - Assert.assertEquals(0, omc.getDroppedMessages().longValue()); - } - - @Test - public void handleMessagePromise_ExpiredException_DoNotRetryMsg() - { - ChannelPromise promise = channel.newPromise(); - promise.setFailure(new ExpiredException()); - - channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1), true); - Assert.assertTrue(channel.isActive()); - Assert.assertEquals(1, omc.getCompletedMessages().longValue()); - Assert.assertEquals(1, omc.getDroppedMessages().longValue()); - Assert.assertFalse(omc.sendMessageInvoked); - } - - @Test - public void handleMessagePromise_NonIOException() - { - ChannelPromise promise = channel.newPromise(); - promise.setFailure(new NullPointerException("this is a test")); - channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1), true); - Assert.assertTrue(channel.isActive()); - Assert.assertEquals(1, omc.getCompletedMessages().longValue()); - Assert.assertEquals(0, omc.getDroppedMessages().longValue()); - Assert.assertFalse(omc.sendMessageInvoked); - } - - @Test - public void handleMessagePromise_IOException_ChannelNotClosed_RetryMsg() - { - ChannelPromise promise = channel.newPromise(); - promise.setFailure(new IOException("this is a test")); - Assert.assertTrue(channel.isActive()); - channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1, 0, true, true), true); - - Assert.assertFalse(channel.isActive()); - Assert.assertEquals(1, omc.getCompletedMessages().longValue()); - Assert.assertEquals(0, omc.getDroppedMessages().longValue()); - Assert.assertTrue(omc.sendMessageInvoked); - } - - @Test - public void handleMessagePromise_Cancelled() - { - ChannelPromise promise = channel.newPromise(); - promise.cancel(false); - Assert.assertTrue(channel.isActive()); - channelWriter.handleMessageFuture(promise, new QueuedMessage(new MessageOut<>(ECHO), 1, 0, true, true), true); - - Assert.assertTrue(channel.isActive()); - Assert.assertEquals(1, omc.getCompletedMessages().longValue()); - Assert.assertEquals(0, omc.getDroppedMessages().longValue()); - Assert.assertFalse(omc.sendMessageInvoked); - } -} diff --git a/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java b/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java deleted file mode 100644 index 087f49ed2201..000000000000 --- a/test/unit/org/apache/cassandra/net/async/HandshakeHandlersTest.java +++ /dev/null @@ -1,227 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Optional; - -import com.google.common.net.InetAddresses; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import io.netty.channel.embedded.EmbeddedChannel; -import org.apache.cassandra.SchemaLoader; -import org.apache.cassandra.auth.AllowAllInternodeAuthenticator; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.db.RowUpdateBuilder; -import org.apache.cassandra.db.compaction.CompactionManager; -import org.apache.cassandra.db.marshal.AsciiType; -import org.apache.cassandra.db.marshal.BytesType; -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.schema.KeyspaceParams; - -import static org.apache.cassandra.net.async.InboundHandshakeHandler.State.HANDSHAKE_COMPLETE; -import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.READY; - -@RunWith(Parameterized.class) -public class HandshakeHandlersTest -{ - private static final String KEYSPACE1 = "NettyPipilineTest"; - private static final String STANDARD1 = "Standard1"; - - private static final InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9999); - private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9999); - private static final OutboundConnectionIdentifier connectionId = OutboundConnectionIdentifier.small(LOCAL_ADDR, REMOTE_ADDR); - private final int messagingVersion; - - @BeforeClass - public static void beforeClass() throws ConfigurationException - { - SchemaLoader.prepareServer(); - SchemaLoader.createKeyspace(KEYSPACE1, - KeyspaceParams.simple(1), - SchemaLoader.standardCFMD(KEYSPACE1, STANDARD1, 0, AsciiType.instance, BytesType.instance)); - CompactionManager.instance.disableAutoCompaction(); - } - - public HandshakeHandlersTest(int messagingVersion) - { - this.messagingVersion = messagingVersion; - } - - @Parameters() - public static Iterable generateData() - { - return Arrays.asList(MessagingService.VERSION_30, MessagingService.VERSION_40); - } - - @Test - public void handshake_HappyPath() - { - // beacuse both CHH & SHH are ChannelInboundHandlers, we can't use the same EmbeddedChannel to handle them - InboundHandshakeHandler inboundHandshakeHandler = new InboundHandshakeHandler(new TestAuthenticator(true)); - EmbeddedChannel inboundChannel = new EmbeddedChannel(inboundHandshakeHandler); - - OutboundMessagingConnection imc = new OutboundMessagingConnection(connectionId, null, Optional.empty(), new AllowAllInternodeAuthenticator()); - OutboundConnectionParams params = OutboundConnectionParams.builder() - .connectionId(connectionId) - .callback(imc::finishHandshake) - .mode(NettyFactory.Mode.MESSAGING) - .protocolVersion(MessagingService.current_version) - .coalescingStrategy(Optional.empty()) - .build(); - OutboundHandshakeHandler outboundHandshakeHandler = new OutboundHandshakeHandler(params); - EmbeddedChannel outboundChannel = new EmbeddedChannel(outboundHandshakeHandler); - Assert.assertEquals(1, outboundChannel.outboundMessages().size()); - - // move internode protocol Msg1 to the server's channel - Object o; - while ((o = outboundChannel.readOutbound()) != null) - inboundChannel.writeInbound(o); - Assert.assertEquals(1, inboundChannel.outboundMessages().size()); - - // move internode protocol Msg2 to the client's channel - while ((o = inboundChannel.readOutbound()) != null) - outboundChannel.writeInbound(o); - Assert.assertEquals(1, outboundChannel.outboundMessages().size()); - - // move internode protocol Msg3 to the server's channel - while ((o = outboundChannel.readOutbound()) != null) - inboundChannel.writeInbound(o); - - Assert.assertEquals(READY, imc.getState()); - Assert.assertEquals(HANDSHAKE_COMPLETE, inboundHandshakeHandler.getState()); - } - - @Test - public void lotsOfMutations_NoCompression() throws IOException - { - lotsOfMutations(false); - } - - @Test - public void lotsOfMutations_WithCompression() throws IOException - { - lotsOfMutations(true); - } - - private void lotsOfMutations(boolean compress) - { - TestChannels channels = buildChannels(compress); - EmbeddedChannel outboundChannel = channels.outboundChannel; - EmbeddedChannel inboundChannel = channels.inboundChannel; - - // now the actual test! - ByteBuffer buf = ByteBuffer.allocate(1 << 10); - byte[] bytes = "ThisIsA16CharStr".getBytes(); - while (buf.remaining() > 0) - buf.put(bytes); - - // write a bunch of messages to the channel - ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1); - int count = 1024; - for (int i = 0; i < count; i++) - { - if (i % 2 == 0) - { - Mutation mutation = new RowUpdateBuilder(cfs1.metadata.get(), 0, "k") - .clustering("bytes") - .add("val", buf) - .build(); - - QueuedMessage msg = new QueuedMessage(mutation.createMessage(), i); - outboundChannel.writeAndFlush(msg); - } - else - { - outboundChannel.writeAndFlush(new QueuedMessage(new MessageOut<>(MessagingService.Verb.ECHO), i)); - } - } - outboundChannel.flush(); - - // move the messages to the other channel - Object o; - while ((o = outboundChannel.readOutbound()) != null) - inboundChannel.writeInbound(o); - - Assert.assertTrue(outboundChannel.outboundMessages().isEmpty()); - Assert.assertFalse(inboundChannel.finishAndReleaseAll()); - } - - private TestChannels buildChannels(boolean compress) - { - OutboundConnectionParams params = OutboundConnectionParams.builder() - .connectionId(connectionId) - .callback(this::nop) - .mode(NettyFactory.Mode.MESSAGING) - .compress(compress) - .coalescingStrategy(Optional.empty()) - .protocolVersion(MessagingService.current_version) - .backlogSupplier(this::nopBacklog) - .build(); - OutboundHandshakeHandler outboundHandshakeHandler = new OutboundHandshakeHandler(params); - EmbeddedChannel outboundChannel = new EmbeddedChannel(outboundHandshakeHandler); - OutboundMessagingConnection omc = new OutboundMessagingConnection(connectionId, null, Optional.empty(), new AllowAllInternodeAuthenticator()); - omc.setTargetVersion(messagingVersion); - outboundHandshakeHandler.setupPipeline(outboundChannel, messagingVersion); - - // remove the outbound handshake message from the outbound messages - outboundChannel.outboundMessages().clear(); - - InboundHandshakeHandler handler = new InboundHandshakeHandler(new TestAuthenticator(true)); - EmbeddedChannel inboundChannel = new EmbeddedChannel(handler); - handler.setupMessagingPipeline(inboundChannel.pipeline(), REMOTE_ADDR, compress, messagingVersion); - - return new TestChannels(outboundChannel, inboundChannel); - } - - private static class TestChannels - { - final EmbeddedChannel outboundChannel; - final EmbeddedChannel inboundChannel; - - TestChannels(EmbeddedChannel outboundChannel, EmbeddedChannel inboundChannel) - { - this.outboundChannel = outboundChannel; - this.inboundChannel = inboundChannel; - } - } - - private Void nop(OutboundHandshakeHandler.HandshakeResult handshakeResult) - { - // do nothing, really - return null; - } - - private QueuedMessage nopBacklog() - { - return null; - } -} diff --git a/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java b/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java deleted file mode 100644 index af486368c607..000000000000 --- a/test/unit/org/apache/cassandra/net/async/HandshakeProtocolTest.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.net.async; - -import org.junit.After; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.PooledByteBufAllocator; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage; -import org.apache.cassandra.utils.FBUtilities; - -import static org.junit.Assert.assertEquals; - -public class HandshakeProtocolTest -{ - private ByteBuf buf; - - @BeforeClass - public static void before() - { - // Kind of stupid, but the test trigger the initialization of the MessagingService class and that require - // DatabaseDescriptor to be configured ... - DatabaseDescriptor.daemonInitialization(); - } - - @After - public void tearDown() - { - if (buf != null && buf.refCnt() > 0) - buf.release(); - } - - @Test - public void firstMessageTest() throws Exception - { - firstMessageTest(NettyFactory.Mode.MESSAGING, false); - firstMessageTest(NettyFactory.Mode.MESSAGING, true); - firstMessageTest(NettyFactory.Mode.STREAMING, false); - firstMessageTest(NettyFactory.Mode.STREAMING, true); - } - - private void firstMessageTest(NettyFactory.Mode mode, boolean compression) throws Exception - { - FirstHandshakeMessage before = new FirstHandshakeMessage(MessagingService.current_version, mode, compression); - buf = before.encode(PooledByteBufAllocator.DEFAULT); - FirstHandshakeMessage after = FirstHandshakeMessage.maybeDecode(buf); - assertEquals(before, after); - assertEquals(before.hashCode(), after.hashCode()); - Assert.assertFalse(before.equals(null)); - } - - @Test - public void secondMessageTest() throws Exception - { - SecondHandshakeMessage before = new SecondHandshakeMessage(MessagingService.current_version); - buf = before.encode(PooledByteBufAllocator.DEFAULT); - SecondHandshakeMessage after = SecondHandshakeMessage.maybeDecode(buf); - assertEquals(before, after); - assertEquals(before.hashCode(), after.hashCode()); - Assert.assertFalse(before.equals(null)); - } - - @Test - public void thirdMessageTest() throws Exception - { - ThirdHandshakeMessage before = new ThirdHandshakeMessage(MessagingService.current_version, FBUtilities.getBroadcastAddressAndPort()); - buf = before.encode(PooledByteBufAllocator.DEFAULT); - ThirdHandshakeMessage after = ThirdHandshakeMessage.maybeDecode(buf); - assertEquals(before, after); - assertEquals(before.hashCode(), after.hashCode()); - Assert.assertFalse(before.equals(null)); - } -} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java b/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java deleted file mode 100644 index 93a1c204554d..000000000000 --- a/test/unit/org/apache/cassandra/net/async/InboundHandshakeHandlerTest.java +++ /dev/null @@ -1,291 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.util.ArrayList; - -import com.google.common.net.InetAddresses; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufOutputStream; -import io.netty.buffer.PooledByteBufAllocator; -import io.netty.buffer.Unpooled; -import io.netty.channel.ChannelPipeline; -import io.netty.channel.ChannelPromise; -import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.handler.codec.compression.Lz4FrameDecoder; -import io.netty.handler.codec.compression.Lz4FrameEncoder; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.CompactEndpointSerializationHelper; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.HandshakeProtocol.FirstHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage; -import org.apache.cassandra.net.async.InboundHandshakeHandler.State; - -import static org.apache.cassandra.net.async.NettyFactory.Mode.MESSAGING; - -public class InboundHandshakeHandlerTest -{ - private static final InetAddressAndPort addr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 0); - private static final int MESSAGING_VERSION = MessagingService.current_version; - private static final int VERSION_30 = MessagingService.VERSION_30; - - private InboundHandshakeHandler handler; - private EmbeddedChannel channel; - private ByteBuf buf; - - @BeforeClass - public static void beforeClass() - { - DatabaseDescriptor.daemonInitialization(); - } - - @Before - public void setUp() - { - TestAuthenticator authenticator = new TestAuthenticator(false); - handler = new InboundHandshakeHandler(authenticator); - channel = new EmbeddedChannel(handler); - } - - @After - public void tearDown() - { - if (buf != null) - buf.release(); - channel.finishAndReleaseAll(); - } - - @Test - public void handleAuthenticate_Good() - { - handler = new InboundHandshakeHandler(new TestAuthenticator(true)); - channel = new EmbeddedChannel(handler); - boolean result = handler.handleAuthenticate(new InetSocketAddress(addr.address, addr.port), channel.pipeline().firstContext()); - Assert.assertTrue(result); - Assert.assertTrue(channel.isOpen()); - } - - @Test - public void handleAuthenticate_Bad() - { - boolean result = handler.handleAuthenticate(new InetSocketAddress(addr.address, addr.port), channel.pipeline().firstContext()); - Assert.assertFalse(result); - Assert.assertFalse(channel.isOpen()); - Assert.assertFalse(channel.isActive()); - } - - @Test - public void handleAuthenticate_BadSocketAddr() - { - boolean result = handler.handleAuthenticate(new FakeSocketAddress(), channel.pipeline().firstContext()); - Assert.assertFalse(result); - Assert.assertFalse(channel.isOpen()); - Assert.assertFalse(channel.isActive()); - } - - private static class FakeSocketAddress extends SocketAddress - { } - - @Test - public void decode_AlreadyFailed() - { - handler.setState(State.HANDSHAKE_FAIL); - buf = new FirstHandshakeMessage(MESSAGING_VERSION, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT); - handler.decode(channel.pipeline().firstContext(), buf, new ArrayList<>()); - Assert.assertFalse(channel.isOpen()); - Assert.assertFalse(channel.isActive()); - Assert.assertSame(State.HANDSHAKE_FAIL, handler.getState()); - } - - @Test - public void handleStart_NotEnoughInputBytes() throws IOException - { - ByteBuf buf = Unpooled.EMPTY_BUFFER; - State state = handler.handleStart(channel.pipeline().firstContext(), buf); - Assert.assertEquals(State.START, state); - Assert.assertTrue(channel.isOpen()); - Assert.assertTrue(channel.isActive()); - } - - @Test (expected = IOException.class) - public void handleStart_BadMagic() throws IOException - { - InboundHandshakeHandler handler = new InboundHandshakeHandler(new TestAuthenticator(false)); - EmbeddedChannel channel = new EmbeddedChannel(handler); - buf = Unpooled.buffer(32, 32); - - FirstHandshakeMessage first = new FirstHandshakeMessage(MESSAGING_VERSION, - MESSAGING, - true); - - buf.writeInt(MessagingService.PROTOCOL_MAGIC << 2); - buf.writeInt(first.encodeFlags()); - handler.handleStart(channel.pipeline().firstContext(), buf); - } - - @Test - public void handleStart_VersionTooHigh() throws IOException - { - channel.eventLoop(); - buf = new FirstHandshakeMessage(MESSAGING_VERSION + 1, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT); - State state = handler.handleStart(channel.pipeline().firstContext(), buf); - Assert.assertEquals(State.HANDSHAKE_FAIL, state); - Assert.assertFalse(channel.isOpen()); - Assert.assertFalse(channel.isActive()); - } - - @Test - public void handleStart_VersionLessThan3_0() throws IOException - { - buf = new FirstHandshakeMessage(VERSION_30 - 1, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT); - State state = handler.handleStart(channel.pipeline().firstContext(), buf); - Assert.assertEquals(State.HANDSHAKE_FAIL, state); - - Assert.assertFalse(channel.isOpen()); - Assert.assertFalse(channel.isActive()); - } - - @Test - public void handleStart_HappyPath_Messaging() throws IOException - { - buf = new FirstHandshakeMessage(MESSAGING_VERSION, MESSAGING, true).encode(PooledByteBufAllocator.DEFAULT); - State state = handler.handleStart(channel.pipeline().firstContext(), buf); - Assert.assertEquals(State.AWAIT_MESSAGING_START_RESPONSE, state); - if (buf.refCnt() > 0) - buf.release(); - - buf = new ThirdHandshakeMessage(MESSAGING_VERSION, addr).encode(PooledByteBufAllocator.DEFAULT); - state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf); - - Assert.assertEquals(State.HANDSHAKE_COMPLETE, state); - Assert.assertTrue(channel.isOpen()); - Assert.assertTrue(channel.isActive()); - Assert.assertFalse(channel.outboundMessages().isEmpty()); - channel.releaseOutbound(); - } - - @Test - public void handleMessagingStartResponse_NotEnoughInputBytes() throws IOException - { - ByteBuf buf = Unpooled.EMPTY_BUFFER; - State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf); - Assert.assertEquals(State.AWAIT_MESSAGING_START_RESPONSE, state); - Assert.assertTrue(channel.isOpen()); - Assert.assertTrue(channel.isActive()); - } - - @Test - public void handleMessagingStartResponse_BadMaxVersion() throws IOException - { - buf = Unpooled.buffer(32, 32); - buf.writeInt(MESSAGING_VERSION + 1); - CompactEndpointSerializationHelper.instance.serialize(addr, new ByteBufDataOutputPlus(buf), MESSAGING_VERSION + 1); - State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf); - Assert.assertEquals(State.HANDSHAKE_FAIL, state); - Assert.assertFalse(channel.isOpen()); - Assert.assertFalse(channel.isActive()); - } - - @Test - public void handleMessagingStartResponse_HappyPath() throws IOException - { - buf = Unpooled.buffer(32, 32); - buf.writeInt(MESSAGING_VERSION); - CompactEndpointSerializationHelper.instance.serialize(addr, new ByteBufDataOutputPlus(buf), MESSAGING_VERSION); - State state = handler.handleMessagingStartResponse(channel.pipeline().firstContext(), buf); - Assert.assertEquals(State.HANDSHAKE_COMPLETE, state); - Assert.assertTrue(channel.isOpen()); - Assert.assertTrue(channel.isActive()); - } - - @Test - public void setupPipeline_NoCompression() - { - ChannelPipeline pipeline = channel.pipeline(); - Assert.assertNotNull(pipeline.get(InboundHandshakeHandler.class)); - - handler.setupMessagingPipeline(pipeline, addr, false, MESSAGING_VERSION); - Assert.assertNotNull(pipeline.get(MessageInHandler.class)); - Assert.assertNull(pipeline.get(Lz4FrameDecoder.class)); - Assert.assertNull(pipeline.get(Lz4FrameEncoder.class)); - Assert.assertNull(pipeline.get(InboundHandshakeHandler.class)); - } - - @Test - public void setupPipeline_WithCompression() - { - ChannelPipeline pipeline = channel.pipeline(); - Assert.assertNotNull(pipeline.get(InboundHandshakeHandler.class)); - - handler.setupMessagingPipeline(pipeline, addr, true, MESSAGING_VERSION); - Assert.assertNotNull(pipeline.get(MessageInHandler.class)); - Assert.assertNotNull(pipeline.get(Lz4FrameDecoder.class)); - Assert.assertNull(pipeline.get(Lz4FrameEncoder.class)); - Assert.assertNull(pipeline.get(InboundHandshakeHandler.class)); - } - - @Test - public void failHandshake() - { - ChannelPromise future = channel.newPromise(); - handler.setHandshakeTimeout(future); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(channel.isOpen()); - handler.failHandshake(channel.pipeline().firstContext()); - Assert.assertSame(State.HANDSHAKE_FAIL, handler.getState()); - Assert.assertTrue(future.isCancelled()); - Assert.assertFalse(channel.isOpen()); - } - - @Test - public void failHandshake_AlreadyConnected() - { - ChannelPromise future = channel.newPromise(); - handler.setHandshakeTimeout(future); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(channel.isOpen()); - handler.setState(State.HANDSHAKE_COMPLETE); - handler.failHandshake(channel.pipeline().firstContext()); - Assert.assertSame(State.HANDSHAKE_COMPLETE, handler.getState()); - Assert.assertTrue(channel.isOpen()); - } - - @Test - public void failHandshake_TaskIsCancelled() - { - ChannelPromise future = channel.newPromise(); - future.cancel(false); - handler.setHandshakeTimeout(future); - handler.setState(State.AWAIT_MESSAGING_START_RESPONSE); - Assert.assertTrue(channel.isOpen()); - handler.failHandshake(channel.pipeline().firstContext()); - Assert.assertSame(State.AWAIT_MESSAGING_START_RESPONSE, handler.getState()); - Assert.assertTrue(channel.isOpen()); - } -} diff --git a/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java b/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java deleted file mode 100644 index 5997861ef4e7..000000000000 --- a/test/unit/org/apache/cassandra/net/async/MessageInHandlerTest.java +++ /dev/null @@ -1,339 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.EOFException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.EnumMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.function.BiConsumer; - -import com.google.common.collect.ImmutableList; -import com.google.common.net.InetAddresses; -import com.google.common.primitives.Shorts; -import org.junit.After; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import io.netty.channel.embedded.EmbeddedChannel; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.exceptions.RequestFailureReason; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; -import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis; -import org.apache.cassandra.utils.UUIDGen; - -import static org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType.SMALL_MESSAGE; - -@RunWith(Parameterized.class) -public class MessageInHandlerTest -{ - private static final int MSG_ID = 42; - private static InetAddressAndPort addr; - - private final int messagingVersion; - - private ByteBuf buf; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - addr = InetAddressAndPort.getByAddress(InetAddresses.forString("127.0.73.101")); - } - - public MessageInHandlerTest(int messagingVersion) - { - this.messagingVersion = messagingVersion; - } - - @Parameters() - public static Iterable generateData() - { - return Arrays.asList(MessagingService.VERSION_30, MessagingService.VERSION_40); - } - - @After - public void tearDown() - { - if (buf != null && buf.refCnt() > 0) - buf.release(); - } - - private BaseMessageInHandler getHandler(InetAddressAndPort addr, int messagingVersion, BiConsumer messageConsumer) - { - return messagingVersion >= MessagingService.VERSION_40 ? - new MessageInHandler(addr, messagingVersion, messageConsumer) : - new MessageInHandlerPre40(addr, messagingVersion, messageConsumer); - } - - @Test(expected = AssertionError.class) - public void testBadVersionForHandler() - { - if (messagingVersion < MessagingService.VERSION_40) - new MessageInHandler(addr, messagingVersion, null); - else - new MessageInHandlerPre40(addr, messagingVersion, null); - } - - @Test - public void decode_BadMagic() - { - int len = MessageInHandler.FIRST_SECTION_BYTE_COUNT; - buf = Unpooled.buffer(len, len); - buf.writeInt(-1); - buf.writerIndex(len); - - BaseMessageInHandler handler = getHandler(addr, messagingVersion, null); - EmbeddedChannel channel = new EmbeddedChannel(handler); - Assert.assertTrue(channel.isOpen()); - channel.writeInbound(buf); - Assert.assertFalse(channel.isOpen()); - } - - @Test - public void decode_HappyPath_NoParameters() throws Exception - { - MessageInWrapper result = decode_HappyPath(Collections.emptyMap()); - Assert.assertTrue(result.messageIn.parameters.isEmpty()); - } - - @Test - public void decode_HappyPath_WithParameters() throws Exception - { - UUID uuid = UUIDGen.getTimeUUID(); - Map parameters = new EnumMap<>(ParameterType.class); - parameters.put(ParameterType.FAILURE_RESPONSE, MessagingService.ONE_BYTE); - parameters.put(ParameterType.FAILURE_REASON, Shorts.checkedCast(RequestFailureReason.READ_TOO_MANY_TOMBSTONES.code)); - parameters.put(ParameterType.TRACE_SESSION, uuid); - MessageInWrapper result = decode_HappyPath(parameters); - Assert.assertEquals(3, result.messageIn.parameters.size()); - Assert.assertTrue(result.messageIn.isFailureResponse()); - Assert.assertEquals(RequestFailureReason.READ_TOO_MANY_TOMBSTONES, result.messageIn.getFailureReason()); - Assert.assertEquals(uuid, result.messageIn.parameters.get(ParameterType.TRACE_SESSION)); - } - - private MessageInWrapper decode_HappyPath(Map parameters) throws Exception - { - MessageOut msgOut = new MessageOut<>(addr, MessagingService.Verb.ECHO, null, null, ImmutableList.of(), SMALL_MESSAGE); - for (Map.Entry param : parameters.entrySet()) - msgOut = msgOut.withParameter(param.getKey(), param.getValue()); - serialize(msgOut, MSG_ID); - - MessageInWrapper wrapper = new MessageInWrapper(); - BaseMessageInHandler handler = getHandler(addr, messagingVersion, wrapper.messageConsumer); - List out = new ArrayList<>(); - handler.decode(null, buf, out); - - Assert.assertNotNull(wrapper.messageIn); - Assert.assertEquals(MSG_ID, wrapper.id); - Assert.assertEquals(msgOut.from, wrapper.messageIn.from); - Assert.assertEquals(msgOut.verb, wrapper.messageIn.verb); - Assert.assertTrue(out.isEmpty()); - - return wrapper; - } - - private void serialize(MessageOut msgOut, int id) throws IOException - { - if (buf == null) - buf = Unpooled.buffer(1024, 1024); // 1k should be enough for everybody! - buf.writeInt(MessagingService.PROTOCOL_MAGIC); - buf.writeInt(id); // this is the id - buf.writeInt((int) NanoTimeToCurrentTimeMillis.convert(System.nanoTime())); - - msgOut.serialize(new ByteBufDataOutputPlus(buf), messagingVersion); - } - - @Test - public void decode_WithHalfReceivedParameters() throws Exception - { - MessageOut msgOut = new MessageOut<>(addr, MessagingService.Verb.ECHO, null, null, ImmutableList.of(), SMALL_MESSAGE); - UUID uuid = UUIDGen.getTimeUUID(); - msgOut = msgOut.withParameter(ParameterType.TRACE_SESSION, uuid); - - serialize(msgOut, MSG_ID); - - // move the write index pointer back a few bytes to simulate like the full bytes are not present. - // yeah, it's lame, but it tests the basics of what is happening during the deserialiization - int originalWriterIndex = buf.writerIndex(); - buf.writerIndex(originalWriterIndex - 6); - - MessageInWrapper wrapper = new MessageInWrapper(); - BaseMessageInHandler handler = getHandler(addr, messagingVersion, wrapper.messageConsumer); - List out = new ArrayList<>(); - handler.decode(null, buf, out); - - Assert.assertNull(wrapper.messageIn); - - BaseMessageInHandler.MessageHeader header = handler.getMessageHeader(); - Assert.assertEquals(MSG_ID, header.messageId); - Assert.assertEquals(msgOut.verb, header.verb); - Assert.assertEquals(msgOut.from, header.from); - Assert.assertTrue(out.isEmpty()); - - // now, set the writer index back to the original value to pretend that we actually got more bytes in - buf.writerIndex(originalWriterIndex); - handler.decode(null, buf, out); - Assert.assertNotNull(wrapper.messageIn); - Assert.assertTrue(out.isEmpty()); - } - - @Test - public void canReadNextParam_HappyPath() throws IOException - { - buildParamBufPre40(13); - Assert.assertTrue(MessageInHandlerPre40.canReadNextParam(buf)); - } - - @Test - public void canReadNextParam_OnlyFirstByte() throws IOException - { - buildParamBufPre40(13); - buf.writerIndex(1); - Assert.assertFalse(MessageInHandlerPre40.canReadNextParam(buf)); - } - - @Test - public void canReadNextParam_PartialUTF() throws IOException - { - buildParamBufPre40(13); - buf.writerIndex(5); - Assert.assertFalse(MessageInHandlerPre40.canReadNextParam(buf)); - } - - @Test - public void canReadNextParam_TruncatedValueLength() throws IOException - { - buildParamBufPre40(13); - buf.writerIndex(buf.writerIndex() - 13 - 2); - Assert.assertFalse(MessageInHandlerPre40.canReadNextParam(buf)); - } - - @Test - public void canReadNextParam_MissingLastBytes() throws IOException - { - buildParamBufPre40(13); - buf.writerIndex(buf.writerIndex() - 2); - Assert.assertFalse(MessageInHandlerPre40.canReadNextParam(buf)); - } - - private void buildParamBufPre40(int valueLength) throws IOException - { - buf = Unpooled.buffer(1024, 1024); // 1k should be enough for everybody! - - try (ByteBufDataOutputPlus output = new ByteBufDataOutputPlus(buf)) - { - output.writeUTF("name"); - byte[] array = new byte[valueLength]; - output.writeInt(array.length); - output.write(array); - } - } - - @Test - public void exceptionHandled() - { - BaseMessageInHandler handler = getHandler(addr, messagingVersion, null); - EmbeddedChannel channel = new EmbeddedChannel(handler); - Assert.assertTrue(channel.isOpen()); - handler.exceptionCaught(channel.pipeline().firstContext(), new EOFException()); - Assert.assertFalse(channel.isOpen()); - } - - /** - * this is for handling the bug uncovered by CASSANDRA-14574. - * - * TL;DR if we run into a problem processing a message out an incoming buffer (and we close the channel, etc), - * do not attempt to process anymore messages from the buffer (force the channel closed and - * reject any more read attempts from the buffer). - * - * The idea here is to put several messages into a ByteBuf, pass that to the channel/handler, and make sure that - * only the initial, correct messages in the buffer are processed. After one messages fails the rest of the buffer - * should be ignored. - */ - @Test - public void exceptionHandled_14574() throws IOException - { - Map parameters = new EnumMap<>(ParameterType.class); - parameters.put(ParameterType.FAILURE_RESPONSE, MessagingService.ONE_BYTE); - parameters.put(ParameterType.FAILURE_REASON, Shorts.checkedCast(RequestFailureReason.READ_TOO_MANY_TOMBSTONES.code)); - MessageOut msgOut = new MessageOut<>(addr, MessagingService.Verb.ECHO, null, null, ImmutableList.of(), SMALL_MESSAGE); - for (Map.Entry param : parameters.entrySet()) - msgOut = msgOut.withParameter(param.getKey(), param.getValue()); - - // put one complete, correct message into the buffer - serialize(msgOut, 1); - - // add a second message, but intentionally corrupt it by manipulating a byte in it's range - int startPosition = buf.writerIndex(); - serialize(msgOut, 2); - int positionToHack = startPosition + 2; - buf.setByte(positionToHack, buf.getByte(positionToHack) - 1); - - // add one more complete, correct message into the buffer - serialize(msgOut, 3); - - MessageIdsWrapper wrapper = new MessageIdsWrapper(); - BaseMessageInHandler handler = getHandler(addr, messagingVersion, wrapper.messageConsumer); - EmbeddedChannel channel = new EmbeddedChannel(handler); - Assert.assertTrue(channel.isOpen()); - channel.writeOneInbound(buf); - - Assert.assertFalse(buf.isReadable()); - Assert.assertEquals(BaseMessageInHandler.State.CLOSED, handler.getState()); - Assert.assertFalse(channel.isOpen()); - Assert.assertEquals(1, wrapper.ids.size()); - Assert.assertEquals(Integer.valueOf(1), wrapper.ids.get(0)); - } - - private static class MessageInWrapper - { - MessageIn messageIn; - int id; - - final BiConsumer messageConsumer = (messageIn, integer) -> - { - this.messageIn = messageIn; - this.id = integer; - }; - } - - private static class MessageIdsWrapper - { - private final ArrayList ids = new ArrayList<>(); - - final BiConsumer messageConsumer = (messageIn, integer) -> ids.add(integer); - } -} diff --git a/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java b/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java deleted file mode 100644 index 9aa26251cfd0..000000000000 --- a/test/unit/org/apache/cassandra/net/async/MessageOutHandlerTest.java +++ /dev/null @@ -1,288 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Optional; -import java.util.UUID; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; - -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelOutboundHandlerAdapter; -import io.netty.channel.ChannelPromise; -import io.netty.channel.DefaultChannelPromise; -import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.handler.codec.UnsupportedMessageTypeException; -import io.netty.handler.timeout.IdleStateEvent; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.ParameterType; -import org.apache.cassandra.tracing.Tracing; - -public class MessageOutHandlerTest -{ - private static final int MESSAGING_VERSION = MessagingService.current_version; - - private ChannelWriter channelWriter; - private EmbeddedChannel channel; - private MessageOutHandler handler; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - DatabaseDescriptor.createAllDirectories(); - } - - @Before - public void setup() throws Exception - { - setup(MessageOutHandler.AUTO_FLUSH_THRESHOLD); - } - - private void setup(int flushThreshold) throws Exception - { - OutboundConnectionIdentifier connectionId = OutboundConnectionIdentifier.small(InetAddressAndPort.getByNameOverrideDefaults("127.0.0.1", 0), - InetAddressAndPort.getByNameOverrideDefaults("127.0.0.2", 0)); - OutboundMessagingConnection omc = new NonSendingOutboundMessagingConnection(connectionId, null, Optional.empty()); - channel = new EmbeddedChannel(); - channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()); - handler = new MessageOutHandler(connectionId, MESSAGING_VERSION, channelWriter, () -> null, flushThreshold); - channel.pipeline().addLast(handler); - } - - @Test - public void write_NoFlush() throws ExecutionException, InterruptedException, TimeoutException - { - MessageOut message = new MessageOut(MessagingService.Verb.ECHO); - ChannelFuture future = channel.write(new QueuedMessage(message, 42)); - Assert.assertTrue(!future.isDone()); - Assert.assertFalse(channel.releaseOutbound()); - } - - @Test - public void write_WithFlush() throws Exception - { - setup(1); - MessageOut message = new MessageOut(MessagingService.Verb.ECHO); - ChannelFuture future = channel.write(new QueuedMessage(message, 42)); - Assert.assertTrue(future.isSuccess()); - Assert.assertTrue(channel.releaseOutbound()); - } - - @Test - public void serializeMessage() throws IOException - { - channelWriter.pendingMessageCount.set(1); - QueuedMessage msg = new QueuedMessage(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), 1); - ChannelFuture future = channel.writeAndFlush(msg); - - Assert.assertTrue(future.isSuccess()); - Assert.assertTrue(1 <= channel.outboundMessages().size()); - Assert.assertTrue(channel.releaseOutbound()); - } - - @Test - public void wrongMessageType() - { - ChannelPromise promise = new DefaultChannelPromise(channel); - Assert.assertFalse(handler.isMessageValid("this is the wrong message type", promise)); - - Assert.assertFalse(promise.isSuccess()); - Assert.assertNotNull(promise.cause()); - Assert.assertSame(UnsupportedMessageTypeException.class, promise.cause().getClass()); - } - - @Test - public void unexpiredMessage() - { - QueuedMessage msg = new QueuedMessage(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), 1); - ChannelPromise promise = new DefaultChannelPromise(channel); - Assert.assertTrue(handler.isMessageValid(msg, promise)); - - // we won't know if it was successful yet, but we'll know if it's a failure because cause will be set - Assert.assertNull(promise.cause()); - } - - @Test - public void expiredMessage() - { - QueuedMessage msg = new QueuedMessage(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), 1, 0, true, true); - ChannelPromise promise = new DefaultChannelPromise(channel); - Assert.assertFalse(handler.isMessageValid(msg, promise)); - - Assert.assertFalse(promise.isSuccess()); - Assert.assertNotNull(promise.cause()); - Assert.assertSame(ExpiredException.class, promise.cause().getClass()); - Assert.assertTrue(channel.outboundMessages().isEmpty()); - } - - @Test - public void write_MessageTooLarge() - { - write_BadMessageSize(Integer.MAX_VALUE + 1); - } - - @Test - public void write_MessageSizeIsBananas() - { - write_BadMessageSize(Integer.MIN_VALUE + 10000); - } - - private void write_BadMessageSize(long size) - { - IVersionedSerializer serializer = new IVersionedSerializer() - { - public void serialize(Object o, DataOutputPlus out, int version) - { } - - public Object deserialize(DataInputPlus in, int version) - { - return null; - } - - public long serializedSize(Object o, int version) - { - return size; - } - }; - MessageOut message = new MessageOut(MessagingService.Verb.UNUSED_5, "payload", serializer); - ChannelFuture future = channel.write(new QueuedMessage(message, 42)); - Throwable t = future.cause(); - Assert.assertNotNull(t); - Assert.assertSame(IllegalStateException.class, t.getClass()); - Assert.assertTrue(channel.isOpen()); - Assert.assertFalse(channel.releaseOutbound()); - } - - @Test - public void writeForceExceptionPath() - { - IVersionedSerializer serializer = new IVersionedSerializer() - { - public void serialize(Object o, DataOutputPlus out, int version) - { - throw new RuntimeException("this exception is part of the test - DON'T PANIC"); - } - - public Object deserialize(DataInputPlus in, int version) - { - return null; - } - - public long serializedSize(Object o, int version) - { - return 42; - } - }; - MessageOut message = new MessageOut(MessagingService.Verb.UNUSED_5, "payload", serializer); - ChannelFuture future = channel.write(new QueuedMessage(message, 42)); - Throwable t = future.cause(); - Assert.assertNotNull(t); - Assert.assertFalse(channel.isOpen()); - Assert.assertFalse(channel.releaseOutbound()); - } - - @Test - public void captureTracingInfo_ForceException() - { - MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE) - .withParameter(ParameterType.TRACE_SESSION, new byte[9]); - handler.captureTracingInfo(new QueuedMessage(message, 42)); - } - - @Test - public void captureTracingInfo_UnknownSession() - { - UUID uuid = UUID.randomUUID(); - MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE) - .withParameter(ParameterType.TRACE_SESSION, uuid); - handler.captureTracingInfo(new QueuedMessage(message, 42)); - } - - @Test - public void captureTracingInfo_KnownSession() - { - Tracing.instance.newSession(new HashMap<>()); - MessageOut message = new MessageOut(MessagingService.Verb.REQUEST_RESPONSE); - handler.captureTracingInfo(new QueuedMessage(message, 42)); - } - - @Test - public void userEventTriggered_RandomObject() - { - Assert.assertTrue(channel.isOpen()); - ChannelUserEventSender sender = new ChannelUserEventSender(); - channel.pipeline().addFirst(sender); - sender.sendEvent("ThisIsAFakeEvent"); - Assert.assertTrue(channel.isOpen()); - } - - @Test - public void userEventTriggered_Idle_NoPendingBytes() - { - Assert.assertTrue(channel.isOpen()); - ChannelUserEventSender sender = new ChannelUserEventSender(); - channel.pipeline().addFirst(sender); - sender.sendEvent(IdleStateEvent.WRITER_IDLE_STATE_EVENT); - Assert.assertTrue(channel.isOpen()); - } - - @Test - public void userEventTriggered_Idle_WithPendingBytes() - { - Assert.assertTrue(channel.isOpen()); - ChannelUserEventSender sender = new ChannelUserEventSender(); - channel.pipeline().addFirst(sender); - - MessageOut message = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE); - channel.writeOutbound(new QueuedMessage(message, 42)); - sender.sendEvent(IdleStateEvent.WRITER_IDLE_STATE_EVENT); - Assert.assertFalse(channel.isOpen()); - } - - private static class ChannelUserEventSender extends ChannelOutboundHandlerAdapter - { - private ChannelHandlerContext ctx; - - @Override - public void handlerAdded(final ChannelHandlerContext ctx) throws Exception - { - this.ctx = ctx; - } - - private void sendEvent(Object event) - { - ctx.fireUserEventTriggered(event); - } - } -} diff --git a/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java b/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java deleted file mode 100644 index 18d17e88664b..000000000000 --- a/test/unit/org/apache/cassandra/net/async/NettyFactoryTest.java +++ /dev/null @@ -1,336 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.net.InetSocketAddress; -import java.util.Optional; - -import com.google.common.net.InetAddresses; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.epoll.EpollEventLoopGroup; -import io.netty.channel.epoll.EpollServerSocketChannel; -import io.netty.channel.group.ChannelGroup; -import io.netty.channel.group.DefaultChannelGroup; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.nio.NioServerSocketChannel; -import io.netty.channel.socket.nio.NioSocketChannel; -import io.netty.handler.ssl.SslHandler; -import io.netty.util.concurrent.GlobalEventExecutor; -import org.apache.cassandra.auth.AllowAllInternodeAuthenticator; -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.NettyFactory.InboundInitializer; -import org.apache.cassandra.net.async.NettyFactory.OutboundInitializer; -import org.apache.cassandra.service.NativeTransportService; -import org.apache.cassandra.utils.FBUtilities; -import org.apache.cassandra.utils.NativeLibrary; - -public class NettyFactoryTest -{ - private static final InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9876); - private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9876); - private static final int receiveBufferSize = 1 << 16; - private static final IInternodeAuthenticator AUTHENTICATOR = new AllowAllInternodeAuthenticator(); - private static final boolean EPOLL_AVAILABLE = NativeTransportService.useEpoll(); - - private ChannelGroup channelGroup; - private NettyFactory factory; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - } - - @Before - public void setUp() - { - channelGroup = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE); - } - - @After - public void tearDown() throws Exception - { - if (factory != null) - factory.close(); - } - - @Test - public void createServerChannel_Epoll() - { - if (!EPOLL_AVAILABLE) - return; - Channel inboundChannel = createServerChannel(true); - if (inboundChannel == null) - return; - Assert.assertEquals(EpollServerSocketChannel.class, inboundChannel.getClass()); - inboundChannel.close(); - } - - private Channel createServerChannel(boolean useEpoll) - { - InboundInitializer inboundInitializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup); - factory = new NettyFactory(useEpoll); - - try - { - return factory.createInboundChannel(LOCAL_ADDR, inboundInitializer, receiveBufferSize); - } - catch (Exception e) - { - if (NativeLibrary.osType == NativeLibrary.OSType.LINUX) - throw e; - - return null; - } - } - - @Test - public void createServerChannel_Nio() - { - Channel inboundChannel = createServerChannel(false); - Assert.assertNotNull("we should always be able to get a NIO channel", inboundChannel); - Assert.assertEquals(NioServerSocketChannel.class, inboundChannel.getClass()); - inboundChannel.close(); - } - - @Test(expected = ConfigurationException.class) - public void createServerChannel_SecondAttemptToBind() - { - Channel inboundChannel = null; - try - { - InboundInitializer inboundInitializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup); - inboundChannel = NettyFactory.instance.createInboundChannel(LOCAL_ADDR, inboundInitializer, receiveBufferSize); - NettyFactory.instance.createInboundChannel(LOCAL_ADDR, inboundInitializer, receiveBufferSize); - } - finally - { - if (inboundChannel != null) - inboundChannel.close(); - } - } - - @Test(expected = ConfigurationException.class) - public void createServerChannel_UnbindableAddress() - { - InetAddressAndPort addr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("1.1.1.1"), 9876); - InboundInitializer inboundInitializer = new InboundInitializer(AUTHENTICATOR, null, channelGroup); - NettyFactory.instance.createInboundChannel(addr, inboundInitializer, receiveBufferSize); - } - - @Test - public void deterineAcceptGroupSize() - { - ServerEncryptionOptions serverEncryptionOptions = new ServerEncryptionOptions(); - serverEncryptionOptions.enabled = false; - Assert.assertEquals(1, NettyFactory.determineAcceptGroupSize(serverEncryptionOptions)); - serverEncryptionOptions.enabled = true; - Assert.assertEquals(1, NettyFactory.determineAcceptGroupSize(serverEncryptionOptions)); - - serverEncryptionOptions.enable_legacy_ssl_storage_port = true; - Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(serverEncryptionOptions)); - serverEncryptionOptions.enable_legacy_ssl_storage_port = false; - - InetAddressAndPort originalBroadcastAddr = FBUtilities.getBroadcastAddressAndPort(); - try - { - FBUtilities.setBroadcastInetAddress(InetAddresses.increment(FBUtilities.getLocalAddressAndPort().address)); - DatabaseDescriptor.setListenOnBroadcastAddress(true); - - serverEncryptionOptions.enabled = false; - Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(serverEncryptionOptions)); - serverEncryptionOptions.enabled = true; - Assert.assertEquals(2, NettyFactory.determineAcceptGroupSize(serverEncryptionOptions)); - - serverEncryptionOptions.enable_legacy_ssl_storage_port = true; - Assert.assertEquals(4, NettyFactory.determineAcceptGroupSize(serverEncryptionOptions)); - } - finally - { - FBUtilities.setBroadcastInetAddress(originalBroadcastAddr.address); - DatabaseDescriptor.setListenOnBroadcastAddress(false); - } - } - - @Test - public void getEventLoopGroup_EpollWithIoRatioBoost() - { - if (!EPOLL_AVAILABLE) - return; - getEventLoopGroup_Epoll(true); - } - - private EpollEventLoopGroup getEventLoopGroup_Epoll(boolean ioBoost) - { - EventLoopGroup eventLoopGroup; - try - { - eventLoopGroup = NettyFactory.getEventLoopGroup(true, 1, "testEventLoopGroup", ioBoost); - } - catch (Exception e) - { - if (NativeLibrary.osType == NativeLibrary.OSType.LINUX) - throw e; - - // ignore as epoll is only available on linux platforms, so don't fail the test on other OSes - return null; - } - - Assert.assertTrue(eventLoopGroup instanceof EpollEventLoopGroup); - return (EpollEventLoopGroup) eventLoopGroup; - } - - @Test - public void getEventLoopGroup_EpollWithoutIoRatioBoost() - { - if (!EPOLL_AVAILABLE) - return; - getEventLoopGroup_Epoll(false); - } - - @Test - public void getEventLoopGroup_NioWithoutIoRatioBoost() - { - getEventLoopGroup_Nio(true); - } - - private NioEventLoopGroup getEventLoopGroup_Nio(boolean ioBoost) - { - EventLoopGroup eventLoopGroup = NettyFactory.getEventLoopGroup(false, 1, "testEventLoopGroup", ioBoost); - Assert.assertTrue(eventLoopGroup instanceof NioEventLoopGroup); - return (NioEventLoopGroup) eventLoopGroup; - } - - @Test - public void getEventLoopGroup_NioWithIoRatioBoost() - { - getEventLoopGroup_Nio(true); - } - - @Test - public void createOutboundBootstrap_Epoll() - { - if (!EPOLL_AVAILABLE) - return; - Bootstrap bootstrap = createOutboundBootstrap(true); - Assert.assertEquals(EpollEventLoopGroup.class, bootstrap.config().group().getClass()); - } - - private Bootstrap createOutboundBootstrap(boolean useEpoll) - { - factory = new NettyFactory(useEpoll); - OutboundConnectionIdentifier id = OutboundConnectionIdentifier.gossip(LOCAL_ADDR, REMOTE_ADDR); - OutboundConnectionParams params = OutboundConnectionParams.builder() - .connectionId(id) - .coalescingStrategy(Optional.empty()) - .protocolVersion(MessagingService.current_version) - .build(); - return factory.createOutboundBootstrap(params); - } - - @Test - public void createOutboundBootstrap_Nio() - { - Bootstrap bootstrap = createOutboundBootstrap(false); - Assert.assertEquals(NioEventLoopGroup.class, bootstrap.config().group().getClass()); - } - - @Test - public void createInboundInitializer_WithoutSsl() throws Exception - { - ServerEncryptionOptions encryptionOptions = new ServerEncryptionOptions(); - encryptionOptions.enabled = false; - InboundInitializer initializer = new InboundInitializer(AUTHENTICATOR, encryptionOptions, channelGroup); - NioSocketChannel channel = new NioSocketChannel(); - initializer.initChannel(channel); - Assert.assertNull(channel.pipeline().get(SslHandler.class)); - Assert.assertNull(channel.pipeline().get(OptionalSslHandler.class)); - } - - private ServerEncryptionOptions encOptions() - { - ServerEncryptionOptions encryptionOptions; - encryptionOptions = new ServerEncryptionOptions(); - encryptionOptions.keystore = "test/conf/cassandra_ssl_test.keystore"; - encryptionOptions.keystore_password = "cassandra"; - encryptionOptions.truststore = "test/conf/cassandra_ssl_test.truststore"; - encryptionOptions.truststore_password = "cassandra"; - encryptionOptions.require_client_auth = false; - encryptionOptions.cipher_suites = new String[] {"TLS_RSA_WITH_AES_128_CBC_SHA"}; - return encryptionOptions; - } - - @Test - public void createInboundInitializer_WithSsl() throws Exception - { - ServerEncryptionOptions encryptionOptions = encOptions(); - encryptionOptions.enabled = true; - encryptionOptions.optional = false; - InboundInitializer initializer = new InboundInitializer(AUTHENTICATOR, encryptionOptions, channelGroup); - NioSocketChannel channel = new NioSocketChannel(); - Assert.assertNull(channel.pipeline().get(SslHandler.class)); - initializer.initChannel(channel); - Assert.assertNotNull(channel.pipeline().get(SslHandler.class)); - Assert.assertNull(channel.pipeline().get(OptionalSslHandler.class)); - } - - @Test - public void createInboundInitializer_WithOptionalSsl() throws Exception - { - ServerEncryptionOptions encryptionOptions = encOptions(); - encryptionOptions.enabled = true; - encryptionOptions.optional = true; - InboundInitializer initializer = new InboundInitializer(AUTHENTICATOR, encryptionOptions, channelGroup); - NioSocketChannel channel = new NioSocketChannel(); - Assert.assertNull(channel.pipeline().get(SslHandler.class)); - initializer.initChannel(channel); - Assert.assertNotNull(channel.pipeline().get(OptionalSslHandler.class)); - Assert.assertNull(channel.pipeline().get(SslHandler.class)); - } - - @Test - public void createOutboundInitializer_WithSsl() throws Exception - { - OutboundConnectionIdentifier id = OutboundConnectionIdentifier.gossip(LOCAL_ADDR, REMOTE_ADDR); - OutboundConnectionParams params = OutboundConnectionParams.builder() - .connectionId(id) - .encryptionOptions(encOptions()) - .protocolVersion(MessagingService.current_version) - .build(); - OutboundInitializer outboundInitializer = new OutboundInitializer(params); - NioSocketChannel channel = new NioSocketChannel(); - Assert.assertNull(channel.pipeline().get(SslHandler.class)); - outboundInitializer.initChannel(channel); - Assert.assertNotNull(channel.pipeline().get(SslHandler.class)); - } -} diff --git a/test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.java b/test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.java deleted file mode 100644 index b0b15b81ab9b..000000000000 --- a/test/unit/org/apache/cassandra/net/async/NonSendingOutboundMessagingConnection.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.util.Optional; - -import org.apache.cassandra.auth.AllowAllInternodeAuthenticator; -import org.apache.cassandra.config.EncryptionOptions; -import org.apache.cassandra.utils.CoalescingStrategies; - -class NonSendingOutboundMessagingConnection extends OutboundMessagingConnection -{ - boolean sendMessageInvoked; - - NonSendingOutboundMessagingConnection(OutboundConnectionIdentifier connectionId, EncryptionOptions.ServerEncryptionOptions encryptionOptions, Optional coalescingStrategy) - { - super(connectionId, encryptionOptions, coalescingStrategy, new AllowAllInternodeAuthenticator()); - } - - @Override - boolean sendMessage(QueuedMessage queuedMessage) - { - sendMessageInvoked = true; - return true; - } -} diff --git a/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java b/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java deleted file mode 100644 index 23a4a6873678..000000000000 --- a/test/unit/org/apache/cassandra/net/async/OutboundConnectionParamsTest.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.net.MessagingService; - -public class OutboundConnectionParamsTest -{ - static int version; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - version = MessagingService.current_version; - } - - @Test (expected = IllegalArgumentException.class) - public void build_SendSizeLessThanZero() - { - OutboundConnectionParams.builder().protocolVersion(version).sendBufferSize(-1).build(); - } - - @Test (expected = IllegalArgumentException.class) - public void build_SendSizeHuge() - { - OutboundConnectionParams.builder().protocolVersion(version).sendBufferSize(1 << 30).build(); - } - - @Test (expected = IllegalArgumentException.class) - public void build_TcpConnectTimeoutLessThanZero() - { - OutboundConnectionParams.builder().protocolVersion(version).tcpConnectTimeoutInMS(-1).build(); - } - - @Test (expected = IllegalArgumentException.class) - public void build_TcpUserTimeoutLessThanZero() - { - OutboundConnectionParams.builder().protocolVersion(version).tcpUserTimeoutInMS(-1).build(); - } - - @Test - public void build_TcpUserTimeoutEqualsZero() - { - OutboundConnectionParams.builder().protocolVersion(version).tcpUserTimeoutInMS(0).build(); - } -} diff --git a/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java b/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java deleted file mode 100644 index 172667cb30f5..000000000000 --- a/test/unit/org/apache/cassandra/net/async/OutboundHandshakeHandlerTest.java +++ /dev/null @@ -1,258 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.util.LinkedList; -import java.util.List; -import java.util.Optional; - -import com.google.common.net.InetAddresses; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.PooledByteBufAllocator; -import io.netty.buffer.Unpooled; -import io.netty.channel.ChannelOutboundHandlerAdapter; -import io.netty.channel.ChannelPipeline; -import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.handler.codec.compression.Lz4FrameDecoder; -import io.netty.handler.codec.compression.Lz4FrameEncoder; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.HandshakeProtocol.SecondHandshakeMessage; -import org.apache.cassandra.net.async.HandshakeProtocol.ThirdHandshakeMessage; -import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult; - -import static org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult.UNKNOWN_PROTOCOL_VERSION; - -public class OutboundHandshakeHandlerTest -{ - private static final int MESSAGING_VERSION = MessagingService.current_version; - private static final InetAddressAndPort localAddr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 0); - private static final InetAddressAndPort remoteAddr = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0); - private static final String HANDLER_NAME = "clientHandshakeHandler"; - - private EmbeddedChannel channel; - private OutboundHandshakeHandler handler; - private OutboundConnectionIdentifier connectionId; - private OutboundConnectionParams params; - private CallbackHandler callbackHandler; - private ByteBuf buf; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - } - - @Before - public void setup() - { - channel = new EmbeddedChannel(new ChannelOutboundHandlerAdapter()); - connectionId = OutboundConnectionIdentifier.small(localAddr, remoteAddr); - callbackHandler = new CallbackHandler(); - params = OutboundConnectionParams.builder() - .connectionId(connectionId) - .callback(handshakeResult -> callbackHandler.receive(handshakeResult)) - .mode(NettyFactory.Mode.MESSAGING) - .protocolVersion(MessagingService.current_version) - .coalescingStrategy(Optional.empty()) - .build(); - handler = new OutboundHandshakeHandler(params); - channel.pipeline().addFirst(HANDLER_NAME, handler); - } - - @After - public void tearDown() - { - if (buf != null && buf.refCnt() > 0) - buf.release(); - Assert.assertFalse(channel.finishAndReleaseAll()); - } - - @Test - public void decode_SmallInput() throws Exception - { - buf = Unpooled.buffer(2, 2); - List out = new LinkedList<>(); - handler.decode(channel.pipeline().firstContext(), buf, out); - Assert.assertEquals(0, buf.readerIndex()); - Assert.assertTrue(out.isEmpty()); - } - - @Test - public void decode_HappyPath() - { - buf = new SecondHandshakeMessage(MESSAGING_VERSION).encode(PooledByteBufAllocator.DEFAULT); - channel.writeInbound(buf); - Assert.assertEquals(1, channel.outboundMessages().size()); - Assert.assertTrue(channel.isOpen()); - - Assert.assertEquals(MESSAGING_VERSION, callbackHandler.result.negotiatedMessagingVersion); - Assert.assertEquals(HandshakeResult.Outcome.SUCCESS, callbackHandler.result.outcome); - Assert.assertFalse(channel.outboundMessages().isEmpty()); - - ByteBuf thridMsgBuf = (ByteBuf) channel.outboundMessages().poll(); - try - { - ThirdHandshakeMessage thirdHandshakeMessage = ThirdHandshakeMessage.maybeDecode(thridMsgBuf); - Assert.assertEquals(MESSAGING_VERSION, thirdHandshakeMessage.messagingVersion); - } - finally - { - thridMsgBuf.release(); - } - } - - @Test - public void decode_HappyPathThrowsException() - { - callbackHandler.failOnCallback = true; - buf = new SecondHandshakeMessage(MESSAGING_VERSION).encode(PooledByteBufAllocator.DEFAULT); - channel.writeInbound(buf); - Assert.assertFalse(channel.isOpen()); - Assert.assertEquals(1, channel.outboundMessages().size()); - Assert.assertTrue(channel.releaseOutbound()); // throw away any responses from decode() - - Assert.assertEquals(UNKNOWN_PROTOCOL_VERSION, callbackHandler.result.negotiatedMessagingVersion); - Assert.assertEquals(HandshakeResult.Outcome.NEGOTIATION_FAILURE, callbackHandler.result.outcome); - } - - @Test - public void decode_ReceivedUnexpectedLowerMsgVersion() - { - int msgVersion = MESSAGING_VERSION - 1; - buf = new SecondHandshakeMessage(msgVersion).encode(PooledByteBufAllocator.DEFAULT); - channel.writeInbound(buf); - Assert.assertTrue(channel.inboundMessages().isEmpty()); - - Assert.assertEquals(msgVersion, callbackHandler.result.negotiatedMessagingVersion); - Assert.assertEquals(HandshakeResult.Outcome.DISCONNECT, callbackHandler.result.outcome); - Assert.assertFalse(channel.isOpen()); - Assert.assertTrue(channel.outboundMessages().isEmpty()); - } - - @Test - public void decode_ReceivedExpectedLowerMsgVersion() - { - int msgVersion = MESSAGING_VERSION - 1; - channel.pipeline().remove(HANDLER_NAME); - params = OutboundConnectionParams.builder() - .connectionId(connectionId) - .callback(handshakeResult -> callbackHandler.receive(handshakeResult)) - .mode(NettyFactory.Mode.MESSAGING) - .protocolVersion(msgVersion) - .coalescingStrategy(Optional.empty()) - .build(); - handler = new OutboundHandshakeHandler(params); - channel.pipeline().addFirst(HANDLER_NAME, handler); - - buf = new SecondHandshakeMessage(msgVersion).encode(PooledByteBufAllocator.DEFAULT); - channel.writeInbound(buf); - Assert.assertTrue(channel.inboundMessages().isEmpty()); - - Assert.assertEquals(msgVersion, callbackHandler.result.negotiatedMessagingVersion); - Assert.assertEquals(HandshakeResult.Outcome.SUCCESS, callbackHandler.result.outcome); - Assert.assertTrue(channel.isOpen()); - Assert.assertFalse(channel.outboundMessages().isEmpty()); - - ByteBuf thridMsgBuf = (ByteBuf) channel.outboundMessages().poll(); - try - { - ThirdHandshakeMessage thirdHandshakeMessage = ThirdHandshakeMessage.maybeDecode(thridMsgBuf); - Assert.assertEquals(MESSAGING_VERSION, thirdHandshakeMessage.messagingVersion); - } - finally - { - thridMsgBuf.release(); - } - } - - @Test - public void decode_ReceivedHigherMsgVersion() - { - int msgVersion = MESSAGING_VERSION - 1; - channel.pipeline().remove(HANDLER_NAME); - params = OutboundConnectionParams.builder() - .connectionId(connectionId) - .callback(handshakeResult -> callbackHandler.receive(handshakeResult)) - .mode(NettyFactory.Mode.MESSAGING) - .protocolVersion(msgVersion) - .coalescingStrategy(Optional.empty()) - .build(); - handler = new OutboundHandshakeHandler(params); - channel.pipeline().addFirst(HANDLER_NAME, handler); - buf = new SecondHandshakeMessage(MESSAGING_VERSION).encode(PooledByteBufAllocator.DEFAULT); - channel.writeInbound(buf); - - Assert.assertEquals(MESSAGING_VERSION, callbackHandler.result.negotiatedMessagingVersion); - Assert.assertEquals(HandshakeResult.Outcome.DISCONNECT, callbackHandler.result.outcome); - } - - @Test - public void setupPipeline_WithCompression() - { - EmbeddedChannel chan = new EmbeddedChannel(new ChannelOutboundHandlerAdapter()); - ChannelPipeline pipeline = chan.pipeline(); - params = OutboundConnectionParams.builder(params).compress(true).protocolVersion(MessagingService.current_version).build(); - handler = new OutboundHandshakeHandler(params); - pipeline.addFirst(handler); - handler.setupPipeline(chan, MESSAGING_VERSION); - Assert.assertNotNull(pipeline.get(Lz4FrameEncoder.class)); - Assert.assertNull(pipeline.get(Lz4FrameDecoder.class)); - Assert.assertNotNull(pipeline.get(MessageOutHandler.class)); - } - - @Test - public void setupPipeline_NoCompression() - { - EmbeddedChannel chan = new EmbeddedChannel(new ChannelOutboundHandlerAdapter()); - ChannelPipeline pipeline = chan.pipeline(); - params = OutboundConnectionParams.builder(params).compress(false).protocolVersion(MessagingService.current_version).build(); - handler = new OutboundHandshakeHandler(params); - pipeline.addFirst(handler); - handler.setupPipeline(chan, MESSAGING_VERSION); - Assert.assertNull(pipeline.get(Lz4FrameEncoder.class)); - Assert.assertNull(pipeline.get(Lz4FrameDecoder.class)); - Assert.assertNotNull(pipeline.get(MessageOutHandler.class)); - } - - private static class CallbackHandler - { - boolean failOnCallback; - HandshakeResult result; - - Void receive(HandshakeResult handshakeResult) - { - if (failOnCallback) - { - // only fail the first callback - failOnCallback = false; - throw new RuntimeException("this exception is expected in the test - DON'T PANIC"); - } - result = handshakeResult; - return null; - } - } -} diff --git a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java b/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java deleted file mode 100644 index 379031c6cf63..000000000000 --- a/test/unit/org/apache/cassandra/net/async/OutboundMessagingConnectionTest.java +++ /dev/null @@ -1,521 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.io.IOException; -import java.net.InetAddress; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.atomic.AtomicReference; -import javax.net.ssl.SSLHandshakeException; - -import com.google.common.net.InetAddresses; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelPromise; -import io.netty.channel.embedded.EmbeddedChannel; -import org.apache.cassandra.auth.AllowAllInternodeAuthenticator; -import org.apache.cassandra.auth.IInternodeAuthenticator; -import org.apache.cassandra.config.Config; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions; -import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.locator.AbstractEndpointSnitch; -import org.apache.cassandra.locator.IEndpointSnitch; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.MessagingServiceTest; -import org.apache.cassandra.net.async.OutboundHandshakeHandler.HandshakeResult; -import org.apache.cassandra.net.async.OutboundMessagingConnection.State; - -import static org.apache.cassandra.net.MessagingService.Verb.ECHO; -import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.CLOSED; -import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.CREATING_CHANNEL; -import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.NOT_READY; -import static org.apache.cassandra.net.async.OutboundMessagingConnection.State.READY; - -public class OutboundMessagingConnectionTest -{ - private static final InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9998); - private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9999); - private static final InetAddressAndPort RECONNECT_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.3"), 9999); - private static final int MESSAGING_VERSION = MessagingService.current_version; - - private OutboundConnectionIdentifier connectionId; - private OutboundMessagingConnection omc; - private EmbeddedChannel channel; - - private IEndpointSnitch snitch; - private ServerEncryptionOptions encryptionOptions; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - } - - @Before - public void setup() - { - connectionId = OutboundConnectionIdentifier.small(LOCAL_ADDR, REMOTE_ADDR); - omc = new OutboundMessagingConnection(connectionId, null, Optional.empty(), new AllowAllInternodeAuthenticator()); - channel = new EmbeddedChannel(); - omc.setChannelWriter(ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty())); - - snitch = DatabaseDescriptor.getEndpointSnitch(); - encryptionOptions = DatabaseDescriptor.getInternodeMessagingEncyptionOptions(); - } - - @After - public void tearDown() - { - DatabaseDescriptor.setEndpointSnitch(snitch); - DatabaseDescriptor.setInternodeMessagingEncyptionOptions(encryptionOptions); - channel.finishAndReleaseAll(); - } - - @Test - public void sendMessage_CreatingChannel() - { - Assert.assertEquals(0, omc.backlogSize()); - omc.setState(CREATING_CHANNEL); - Assert.assertTrue(omc.sendMessage(new MessageOut<>(ECHO), 1)); - Assert.assertEquals(1, omc.backlogSize()); - Assert.assertEquals(1, omc.getPendingMessages().intValue()); - } - - @Test - public void sendMessage_HappyPath() - { - Assert.assertEquals(0, omc.backlogSize()); - omc.setState(READY); - Assert.assertTrue(omc.sendMessage(new MessageOut<>(ECHO), 1)); - Assert.assertEquals(0, omc.backlogSize()); - Assert.assertTrue(channel.releaseOutbound()); - } - - @Test - public void sendMessage_Closed() - { - Assert.assertEquals(0, omc.backlogSize()); - omc.setState(CLOSED); - Assert.assertFalse(omc.sendMessage(new MessageOut<>(ECHO), 1)); - Assert.assertEquals(0, omc.backlogSize()); - Assert.assertFalse(channel.releaseOutbound()); - } - - @Test - public void shouldCompressConnection_None() - { - DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.none); - Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); - } - - @Test - public void shouldCompressConnection_All() - { - DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.all); - Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); - } - - @Test - public void shouldCompressConnection_SameDc() - { - TestSnitch snitch = new TestSnitch(); - snitch.add(LOCAL_ADDR, "dc1"); - snitch.add(REMOTE_ADDR, "dc1"); - DatabaseDescriptor.setEndpointSnitch(snitch); - DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc); - Assert.assertFalse(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); - } - - private static class TestSnitch extends AbstractEndpointSnitch - { - private Map nodeToDc = new HashMap<>(); - - void add(InetAddressAndPort node, String dc) - { - nodeToDc.put(node, dc); - } - - public String getRack(InetAddressAndPort endpoint) - { - return null; - } - - public String getDatacenter(InetAddressAndPort endpoint) - { - return nodeToDc.get(endpoint); - } - - public int compareEndpoints(InetAddressAndPort target, Replica a1, Replica a2) - { - return 0; - } - } - - @Test - public void shouldCompressConnection_DifferentDc() - { - TestSnitch snitch = new TestSnitch(); - snitch.add(LOCAL_ADDR, "dc1"); - snitch.add(REMOTE_ADDR, "dc2"); - DatabaseDescriptor.setEndpointSnitch(snitch); - DatabaseDescriptor.setInternodeCompression(Config.InternodeCompression.dc); - Assert.assertTrue(OutboundMessagingConnection.shouldCompressConnection(LOCAL_ADDR, REMOTE_ADDR)); - } - - @Test - public void close_softClose() - { - close(true); - } - - @Test - public void close_hardClose() - { - close(false); - } - - private void close(boolean softClose) - { - int count = 32; - for (int i = 0; i < count; i++) - omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i)); - Assert.assertEquals(count, omc.backlogSize()); - Assert.assertEquals(count, omc.getPendingMessages().intValue()); - - ScheduledFuture connectionTimeoutFuture = new TestScheduledFuture(); - Assert.assertFalse(connectionTimeoutFuture.isCancelled()); - omc.setConnectionTimeoutFuture(connectionTimeoutFuture); - ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()); - omc.setChannelWriter(channelWriter); - - omc.close(softClose); - Assert.assertFalse(channel.isActive()); - Assert.assertEquals(State.CLOSED, omc.getState()); - Assert.assertEquals(0, omc.backlogSize()); - Assert.assertEquals(0, omc.getPendingMessages().intValue()); - int sentMessages = channel.outboundMessages().size(); - - if (softClose) - Assert.assertTrue(count <= sentMessages); - else - Assert.assertEquals(0, sentMessages); - Assert.assertTrue(connectionTimeoutFuture.isCancelled()); - Assert.assertTrue(channelWriter.isClosed()); - } - - @Test - public void connect_IInternodeAuthFail() - { - IInternodeAuthenticator auth = new IInternodeAuthenticator() - { - public boolean authenticate(InetAddress remoteAddress, int remotePort) - { - return false; - } - - public void validateConfiguration() throws ConfigurationException - { - - } - }; - - MessageOut messageOut = new MessageOut(MessagingService.Verb.GOSSIP_DIGEST_ACK); - OutboundMessagingPool pool = new OutboundMessagingPool(REMOTE_ADDR, LOCAL_ADDR, null, - new MessagingServiceTest.MockBackPressureStrategy(null).newState(REMOTE_ADDR), auth); - omc = pool.getConnection(messageOut); - Assert.assertSame(State.NOT_READY, omc.getState()); - Assert.assertFalse(omc.connect()); - } - - @Test - public void connect_ConnectionAlreadyStarted() - { - omc.setState(State.CREATING_CHANNEL); - Assert.assertFalse(omc.connect()); - Assert.assertSame(State.CREATING_CHANNEL, omc.getState()); - } - - @Test - public void connect_ConnectionClosed() - { - omc.setState(State.CLOSED); - Assert.assertFalse(omc.connect()); - Assert.assertSame(State.CLOSED, omc.getState()); - } - - @Test - public void connectionTimeout_StateIsReady() - { - omc.setState(READY); - ChannelFuture channelFuture = channel.newPromise(); - Assert.assertFalse(omc.connectionTimeout(channelFuture)); - Assert.assertEquals(READY, omc.getState()); - } - - @Test - public void connectionTimeout_StateIsClosed() - { - omc.setState(CLOSED); - ChannelFuture channelFuture = channel.newPromise(); - Assert.assertTrue(omc.connectionTimeout(channelFuture)); - Assert.assertEquals(CLOSED, omc.getState()); - } - - @Test - public void connectionTimeout_AssumeConnectionTimedOut() - { - int count = 32; - for (int i = 0; i < count; i++) - omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i)); - Assert.assertEquals(count, omc.backlogSize()); - Assert.assertEquals(count, omc.getPendingMessages().intValue()); - - omc.setState(CREATING_CHANNEL); - ChannelFuture channelFuture = channel.newPromise(); - Assert.assertTrue(omc.connectionTimeout(channelFuture)); - Assert.assertEquals(NOT_READY, omc.getState()); - Assert.assertEquals(0, omc.backlogSize()); - Assert.assertEquals(0, omc.getPendingMessages().intValue()); - } - - @Test - public void connectCallback_FutureIsSuccess() - { - ChannelPromise promise = channel.newPromise(); - promise.setSuccess(); - Assert.assertTrue(omc.connectCallback(promise)); - } - - @Test - public void connectCallback_Closed() - { - ChannelPromise promise = channel.newPromise(); - omc.setState(State.CLOSED); - Assert.assertFalse(omc.connectCallback(promise)); - } - - @Test - public void connectCallback_FailCauseIsSslHandshake() - { - ChannelPromise promise = channel.newPromise(); - promise.setFailure(new SSLHandshakeException("test is only a test")); - Assert.assertFalse(omc.connectCallback(promise)); - Assert.assertSame(State.NOT_READY, omc.getState()); - } - - @Test - public void connectCallback_FailCauseIsNPE() - { - ChannelPromise promise = channel.newPromise(); - promise.setFailure(new NullPointerException("test is only a test")); - Assert.assertFalse(omc.connectCallback(promise)); - Assert.assertSame(State.NOT_READY, omc.getState()); - } - - @Test - public void connectCallback_FailCauseIsIOException() - { - ChannelPromise promise = channel.newPromise(); - promise.setFailure(new IOException("test is only a test")); - Assert.assertFalse(omc.connectCallback(promise)); - Assert.assertSame(State.NOT_READY, omc.getState()); - } - - @Test - public void connectCallback_FailedAndItsClosed() - { - ChannelPromise promise = channel.newPromise(); - promise.setFailure(new IOException("test is only a test")); - omc.setState(CLOSED); - Assert.assertFalse(omc.connectCallback(promise)); - Assert.assertSame(State.CLOSED, omc.getState()); - } - - @Test - public void finishHandshake_GOOD() - { - ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()); - HandshakeResult result = HandshakeResult.success(channelWriter, MESSAGING_VERSION); - ScheduledFuture connectionTimeoutFuture = new TestScheduledFuture(); - Assert.assertFalse(connectionTimeoutFuture.isCancelled()); - - omc.setChannelWriter(null); - omc.setConnectionTimeoutFuture(connectionTimeoutFuture); - omc.finishHandshake(result); - Assert.assertFalse(channelWriter.isClosed()); - Assert.assertEquals(channelWriter, omc.getChannelWriter()); - Assert.assertEquals(READY, omc.getState()); - Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR)); - Assert.assertNull(omc.getConnectionTimeoutFuture()); - Assert.assertTrue(connectionTimeoutFuture.isCancelled()); - } - - @Test - public void finishHandshake_GOOD_ButClosed() - { - ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()); - HandshakeResult result = HandshakeResult.success(channelWriter, MESSAGING_VERSION); - ScheduledFuture connectionTimeoutFuture = new TestScheduledFuture(); - Assert.assertFalse(connectionTimeoutFuture.isCancelled()); - - omc.setChannelWriter(null); - omc.setState(CLOSED); - omc.setConnectionTimeoutFuture(connectionTimeoutFuture); - omc.finishHandshake(result); - Assert.assertTrue(channelWriter.isClosed()); - Assert.assertNull(omc.getChannelWriter()); - Assert.assertEquals(CLOSED, omc.getState()); - Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR)); - Assert.assertNull(omc.getConnectionTimeoutFuture()); - Assert.assertTrue(connectionTimeoutFuture.isCancelled()); - } - - @Test - public void finishHandshake_DISCONNECT() - { - int count = 32; - for (int i = 0; i < count; i++) - omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i)); - Assert.assertEquals(count, omc.backlogSize()); - - HandshakeResult result = HandshakeResult.disconnect(MESSAGING_VERSION); - omc.finishHandshake(result); - Assert.assertNotNull(omc.getChannelWriter()); - Assert.assertEquals(CREATING_CHANNEL, omc.getState()); - Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR)); - Assert.assertEquals(count, omc.backlogSize()); - } - - @Test - public void finishHandshake_CONNECT_FAILURE() - { - int count = 32; - for (int i = 0; i < count; i++) - omc.addToBacklog(new QueuedMessage(new MessageOut<>(ECHO), i)); - Assert.assertEquals(count, omc.backlogSize()); - - HandshakeResult result = HandshakeResult.failed(); - omc.finishHandshake(result); - Assert.assertEquals(NOT_READY, omc.getState()); - Assert.assertEquals(MESSAGING_VERSION, MessagingService.instance().getVersion(REMOTE_ADDR)); - Assert.assertEquals(0, omc.backlogSize()); - } - - @Test - public void setStateIfNotClosed_AlreadyClosed() - { - AtomicReference state = new AtomicReference<>(CLOSED); - OutboundMessagingConnection.setStateIfNotClosed(state, NOT_READY); - Assert.assertEquals(CLOSED, state.get()); - } - - @Test - public void setStateIfNotClosed_NotClosed() - { - AtomicReference state = new AtomicReference<>(READY); - OutboundMessagingConnection.setStateIfNotClosed(state, NOT_READY); - Assert.assertEquals(NOT_READY, state.get()); - } - - @Test - public void reconnectWithNewIp_HappyPath() - { - ChannelWriter channelWriter = ChannelWriter.create(channel, omc::handleMessageResult, Optional.empty()); - omc.setChannelWriter(channelWriter); - omc.setState(READY); - OutboundConnectionIdentifier originalId = omc.getConnectionId(); - omc.reconnectWithNewIp(RECONNECT_ADDR); - Assert.assertFalse(omc.getConnectionId().equals(originalId)); - Assert.assertTrue(channelWriter.isClosed()); - Assert.assertNotSame(CLOSED, omc.getState()); - } - - @Test - public void reconnectWithNewIp_Closed() - { - omc.setState(CLOSED); - OutboundConnectionIdentifier originalId = omc.getConnectionId(); - omc.reconnectWithNewIp(RECONNECT_ADDR); - Assert.assertSame(omc.getConnectionId(), originalId); - Assert.assertSame(CLOSED, omc.getState()); - } - - @Test - public void reconnectWithNewIp_UnsedConnection() - { - omc.setState(NOT_READY); - OutboundConnectionIdentifier originalId = omc.getConnectionId(); - omc.reconnectWithNewIp(RECONNECT_ADDR); - Assert.assertNotSame(omc.getConnectionId(), originalId); - Assert.assertSame(NOT_READY, omc.getState()); - } - - @Test - public void maybeUpdateConnectionId_NoEncryption() - { - OutboundConnectionIdentifier connectionId = omc.getConnectionId(); - int version = omc.getTargetVersion(); - omc.maybeUpdateConnectionId(); - Assert.assertEquals(connectionId, omc.getConnectionId()); - Assert.assertEquals(version, omc.getTargetVersion()); - } - - @Test - public void maybeUpdateConnectionId_SameVersion() - { - ServerEncryptionOptions encryptionOptions = new ServerEncryptionOptions(); - omc = new OutboundMessagingConnection(connectionId, encryptionOptions, Optional.empty(), new AllowAllInternodeAuthenticator()); - OutboundConnectionIdentifier connectionId = omc.getConnectionId(); - int version = omc.getTargetVersion(); - omc.maybeUpdateConnectionId(); - Assert.assertEquals(connectionId, omc.getConnectionId()); - Assert.assertEquals(version, omc.getTargetVersion()); - } - - @Test - public void maybeUpdateConnectionId_3_X_Version() - { - ServerEncryptionOptions encryptionOptions = new ServerEncryptionOptions(); - encryptionOptions.enabled = true; - encryptionOptions.internode_encryption = ServerEncryptionOptions.InternodeEncryption.all; - DatabaseDescriptor.setInternodeMessagingEncyptionOptions(encryptionOptions); - omc = new OutboundMessagingConnection(connectionId, encryptionOptions, Optional.empty(), new AllowAllInternodeAuthenticator()); - int peerVersion = MessagingService.VERSION_30; - MessagingService.instance().setVersion(connectionId.remote(), MessagingService.VERSION_30); - - OutboundConnectionIdentifier connectionId = omc.getConnectionId(); - omc.maybeUpdateConnectionId(); - Assert.assertNotEquals(connectionId, omc.getConnectionId()); - Assert.assertEquals(InetAddressAndPort.getByAddressOverrideDefaults(REMOTE_ADDR.address, DatabaseDescriptor.getSSLStoragePort()), omc.getConnectionId().remote()); - Assert.assertEquals(InetAddressAndPort.getByAddressOverrideDefaults(REMOTE_ADDR.address, DatabaseDescriptor.getSSLStoragePort()), omc.getConnectionId().connectionAddress()); - Assert.assertEquals(peerVersion, omc.getTargetVersion()); - } -} diff --git a/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java b/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java deleted file mode 100644 index ecd8697ff339..000000000000 --- a/test/unit/org/apache/cassandra/net/async/OutboundMessagingPoolTest.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.net.async; - -import java.util.ArrayList; -import java.util.List; - -import com.google.common.net.InetAddresses; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.cassandra.auth.AllowAllInternodeAuthenticator; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.WriteResponse; -import org.apache.cassandra.gms.GossipDigestSyn; -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.BackPressureState; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.net.async.OutboundConnectionIdentifier.ConnectionType; - -public class OutboundMessagingPoolTest -{ - private static final InetAddressAndPort LOCAL_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.1"), 9476); - private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 9476); - private static final InetAddressAndPort RECONNECT_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.3"), 9476); - private static final List INTERNODE_MESSAGING_CONN_TYPES = new ArrayList() - {{ add(ConnectionType.GOSSIP); add(ConnectionType.LARGE_MESSAGE); add(ConnectionType.SMALL_MESSAGE); }}; - - private OutboundMessagingPool pool; - - @BeforeClass - public static void before() - { - DatabaseDescriptor.daemonInitialization(); - } - - @Before - public void setup() - { - BackPressureState backPressureState = DatabaseDescriptor.getBackPressureStrategy().newState(REMOTE_ADDR); - pool = new OutboundMessagingPool(REMOTE_ADDR, LOCAL_ADDR, null, backPressureState, new AllowAllInternodeAuthenticator()); - } - - @After - public void tearDown() - { - if (pool != null) - pool.close(false); - } - - @Test - public void getConnection_Gossip() - { - GossipDigestSyn syn = new GossipDigestSyn("cluster", "partitioner", new ArrayList<>(0)); - MessageOut message = new MessageOut<>(MessagingService.Verb.GOSSIP_DIGEST_SYN, - syn, GossipDigestSyn.serializer); - Assert.assertEquals(ConnectionType.GOSSIP, pool.getConnection(message).getConnectionId().type()); - } - - @Test - public void getConnection_SmallMessage() - { - MessageOut message = WriteResponse.createMessage(); - Assert.assertEquals(ConnectionType.SMALL_MESSAGE, pool.getConnection(message).getConnectionId().type()); - } - - @Test - public void getConnection_LargeMessage() - { - // just need a serializer to report a size, as fake as it may be - IVersionedSerializer serializer = new IVersionedSerializer() - { - public void serialize(Object o, DataOutputPlus out, int version) - { - - } - - public Object deserialize(DataInputPlus in, int version) - { - return null; - } - - public long serializedSize(Object o, int version) - { - return OutboundMessagingPool.LARGE_MESSAGE_THRESHOLD + 1; - } - }; - MessageOut message = new MessageOut<>(MessagingService.Verb.UNUSED_5, "payload", serializer); - Assert.assertEquals(ConnectionType.LARGE_MESSAGE, pool.getConnection(message).getConnectionId().type()); - } - - @Test - public void close() - { - for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) - Assert.assertNotSame(OutboundMessagingConnection.State.CLOSED, pool.getConnection(type).getState()); - pool.close(false); - for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) - Assert.assertEquals(OutboundMessagingConnection.State.CLOSED, pool.getConnection(type).getState()); - } - - @Test - public void reconnectWithNewIp() - { - for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) - { - Assert.assertEquals(REMOTE_ADDR, pool.getPreferredRemoteAddr()); - Assert.assertEquals(REMOTE_ADDR, pool.getConnection(type).getConnectionId().connectionAddress()); - } - - pool.reconnectWithNewIp(RECONNECT_ADDR); - - for (ConnectionType type : INTERNODE_MESSAGING_CONN_TYPES) - { - Assert.assertEquals(RECONNECT_ADDR, pool.getPreferredRemoteAddr()); - Assert.assertEquals(RECONNECT_ADDR, pool.getConnection(type).getConnectionId().connectionAddress()); - } - } - - @Test - public void timeoutCounter() - { - long originalValue = pool.getTimeouts(); - pool.incrementTimeout(); - Assert.assertEquals(originalValue + 1, pool.getTimeouts()); - } -} diff --git a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java index 903a273178dd..e787595bab6d 100644 --- a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java +++ b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java @@ -137,14 +137,12 @@ public void testDifference() throws Throwable LocalSyncTask task = new LocalSyncTask(desc, r1.endpoint, r2.endpoint, MerkleTrees.difference(r1.trees, r2.trees), NO_PENDING_REPAIR, true, true, PreviewKind.NONE); DefaultConnectionFactory.MAX_CONNECT_ATTEMPTS = 1; - DefaultConnectionFactory.MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(2); try { task.run(); } finally { - DefaultConnectionFactory.MAX_WAIT_TIME_NANOS = TimeUnit.SECONDS.toNanos(30); DefaultConnectionFactory.MAX_CONNECT_ATTEMPTS = 3; } diff --git a/test/unit/org/apache/cassandra/repair/RepairJobTest.java b/test/unit/org/apache/cassandra/repair/RepairJobTest.java index 6db29dc720ca..78fa588b19db 100644 --- a/test/unit/org/apache/cassandra/repair/RepairJobTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairJobTest.java @@ -52,9 +52,7 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.IMessageSink; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.RepairMessage; import org.apache.cassandra.repair.messages.SyncRequest; @@ -156,7 +154,8 @@ public void setup() public void reset() { ActiveRepairService.instance.terminateSessions(); - MessagingService.instance().clearMessageSinks(); + MessagingService.instance().outboundSink.clear(); + MessagingService.instance().inboundSink.clear(); FBUtilities.reset(); } @@ -167,11 +166,11 @@ public void reset() public void testEndToEndNoDifferences() throws InterruptedException, ExecutionException, TimeoutException { Map mockTrees = new HashMap<>(); - mockTrees.put(FBUtilities.getBroadcastAddressAndPort(), createInitialTree(false)); + mockTrees.put(addr1, createInitialTree(false)); mockTrees.put(addr2, createInitialTree(false)); mockTrees.put(addr3, createInitialTree(false)); - List observedMessages = new ArrayList<>(); + List> observedMessages = new ArrayList<>(); interceptRepairMessages(mockTrees, observedMessages); job.run(); @@ -208,7 +207,7 @@ public void testNoTreesRetainedAfterDifference() throws Throwable List mockTreeResponses = mockTrees.entrySet().stream() .map(e -> new TreeResponse(e.getKey(), e.getValue())) .collect(Collectors.toList()); - List messages = new ArrayList<>(); + List> messages = new ArrayList<>(); interceptRepairMessages(mockTrees, messages); long singleTreeSize = ObjectSizes.measureDeep(mockTrees.get(addr1)); @@ -792,49 +791,37 @@ private MerkleTrees createInitialTree(boolean invalidate) } private void interceptRepairMessages(Map mockTrees, - List messageCapture) + List> messageCapture) { - MessagingService.instance().addMessageSink(new IMessageSink() - { - public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to) - { - if (message == null || !(message.payload instanceof RepairMessage)) - return false; - - // So different Thread's messages don't overwrite each other. - synchronized (messageLock) - { - messageCapture.add(message); - } - - RepairMessage rm = (RepairMessage) message.payload; - switch (rm.messageType) - { - case SNAPSHOT: - MessageIn messageIn = MessageIn.create(to, null, - Collections.emptyMap(), - MessagingService.Verb.REQUEST_RESPONSE, - MessagingService.current_version); - MessagingService.instance().receive(messageIn, id); - break; - case VALIDATION_REQUEST: - session.validationComplete(sessionJobDesc, to, mockTrees.get(to)); - break; - case SYNC_REQUEST: - SyncRequest syncRequest = (SyncRequest) rm; - session.syncComplete(sessionJobDesc, new SyncNodePair(syncRequest.src, syncRequest.dst), - true, Collections.emptyList()); - break; - default: - break; - } + MessagingService.instance().inboundSink.add(message -> message.verb().isResponse()); + MessagingService.instance().outboundSink.add((message, to) -> { + if (message == null || !(message.payload instanceof RepairMessage)) return false; + + // So different Thread's messages don't overwrite each other. + synchronized (messageLock) + { + messageCapture.add(message); } - public boolean allowIncomingMessage(MessageIn message, int id) + RepairMessage rm = (RepairMessage) message.payload; + switch (rm.messageType) { - return message.verb == MessagingService.Verb.REQUEST_RESPONSE; + case SNAPSHOT: + MessagingService.instance().callbacks.removeAndRespond(message.id(), to, message.emptyResponse()); + break; + case VALIDATION_REQUEST: + session.validationComplete(sessionJobDesc, to, mockTrees.get(to)); + break; + case SYNC_REQUEST: + SyncRequest syncRequest = (SyncRequest) rm; + session.syncComplete(sessionJobDesc, new SyncNodePair(syncRequest.src, syncRequest.dst), + true, Collections.emptyList()); + break; + default: + break; } + return false; }); } } diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java index ff6b11c94d49..aec2612b712a 100644 --- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java +++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java @@ -40,6 +40,7 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.db.BufferDecoratedKey; import org.apache.cassandra.db.ColumnFamilyStore; @@ -48,9 +49,7 @@ import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.net.IMessageSink; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.RepairMessage; import org.apache.cassandra.repair.messages.ValidationComplete; @@ -92,7 +91,7 @@ public static void defineSchema() throws Exception @After public void tearDown() { - MessagingService.instance().clearMessageSinks(); + MessagingService.instance().outboundSink.clear(); DatabaseDescriptor.setRepairSessionSpaceInMegabytes(testSizeMegabytes); } @@ -108,7 +107,7 @@ public void testValidatorComplete() throws Throwable Range range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken()); final RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), keyspace, columnFamily, Arrays.asList(range)); - final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); + final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); InetAddressAndPort remote = InetAddressAndPort.getByName("127.0.0.2"); @@ -131,8 +130,8 @@ public void testValidatorComplete() throws Throwable Token min = tree.partitioner().getMinimumToken(); assertNotNull(tree.hash(new Range<>(min, min))); - MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - assertEquals(MessagingService.Verb.REPAIR_MESSAGE, message.verb); + Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); + assertEquals(Verb.REPAIR_REQ, message.verb()); RepairMessage m = (RepairMessage) message.payload; assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType); assertEquals(desc, m.desc); @@ -147,15 +146,15 @@ public void testValidatorFailed() throws Throwable Range range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken()); final RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), keyspace, columnFamily, Arrays.asList(range)); - final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); + final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); InetAddressAndPort remote = InetAddressAndPort.getByName("127.0.0.2"); Validator validator = new Validator(desc, remote, 0, PreviewKind.NONE); validator.fail(); - MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - assertEquals(MessagingService.Verb.REPAIR_MESSAGE, message.verb); + Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); + assertEquals(Verb.REPAIR_REQ, message.verb()); RepairMessage m = (RepairMessage) message.payload; assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType); assertEquals(desc, m.desc); @@ -208,12 +207,12 @@ public void simpleValidationTest(int n) throws Exception Collections.singletonList(cfs), desc.ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, false, PreviewKind.NONE); - final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); + final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), 0, true, false, PreviewKind.NONE); ValidationManager.instance.submitValidation(cfs, validator); - MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - assertEquals(MessagingService.Verb.REPAIR_MESSAGE, message.verb); + Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); + assertEquals(Verb.REPAIR_REQ, message.verb()); RepairMessage m = (RepairMessage) message.payload; assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType); assertEquals(desc, m.desc); @@ -265,11 +264,11 @@ public void testSizeLimiting() throws Exception Collections.singletonList(cfs), desc.ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, false, PreviewKind.NONE); - final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); + final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), 0, true, false, PreviewKind.NONE); ValidationManager.instance.submitValidation(cfs, validator); - MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); + Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); MerkleTrees trees = ((ValidationComplete) message.payload).trees; Iterator, MerkleTree>> iterator = trees.iterator(); @@ -325,11 +324,11 @@ public void testRangeSplittingTreeSizeLimit() throws Exception Collections.singletonList(cfs), desc.ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, false, PreviewKind.NONE); - final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); + final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), 0, true, false, PreviewKind.NONE); ValidationManager.instance.submitValidation(cfs, validator); - MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); + Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); MerkleTrees trees = ((ValidationComplete) message.payload).trees; // Should have 4 trees each with a depth of on average 10 (since each range should have gotten 0.25 megabytes) @@ -410,22 +409,10 @@ public void testCountingHasher() assertEquals(len, ((Validator.CountingHasher)hashers[0]).getCount()); } - private CompletableFuture registerOutgoingMessageSink() + private CompletableFuture registerOutgoingMessageSink() { - final CompletableFuture future = new CompletableFuture<>(); - MessagingService.instance().addMessageSink(new IMessageSink() - { - public boolean allowOutgoingMessage(MessageOut message, int id, InetAddressAndPort to) - { - future.complete(message); - return false; - } - - public boolean allowIncomingMessage(MessageIn message, int id) - { - return false; - } - }); + final CompletableFuture future = new CompletableFuture<>(); + MessagingService.instance().outboundSink.add((message, to) -> future.complete(message)); return future; } } diff --git a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java index b532abdcd573..237e9a86df71 100644 --- a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java +++ b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java @@ -42,10 +42,11 @@ import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.MockMessagingService; import org.apache.cassandra.net.MockMessagingSpy; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.AbstractRepairTest; import org.apache.cassandra.repair.RepairSessionResult; import org.apache.cassandra.repair.messages.FailSession; @@ -256,7 +257,7 @@ public void testMockedMessagingPrepareTimeout() throws InterruptedException, Exe // expected } // we won't send out any fail session message in case of timeouts - spyPrepare.expectMockedMessageIn(2).get(100, TimeUnit.MILLISECONDS); + spyPrepare.expectMockedMessage(2).get(100, TimeUnit.MILLISECONDS); sendFailSessionUnexpectedSpy.interceptNoMsg(100, TimeUnit.MILLISECONDS); Assert.assertFalse(repairSubmitted.get()); Assert.assertFalse(hasFailures.get()); @@ -275,16 +276,12 @@ private MockMessagingSpy createPrepareSpy(Collection failed, Function sessionIdFunc) { return MockMessagingService.when( - all(verb(MessagingService.Verb.REPAIR_MESSAGE), + all(verb(Verb.REPAIR_REQ), payload((p) -> p instanceof PrepareConsistentRequest)) ).respond((msgOut, to) -> { if(timeout.contains(to)) return null; - else return MessageIn.create(to, - new PrepareConsistentResponse(sessionIdFunc.apply((PrepareConsistentRequest) msgOut.payload), to, !failed.contains(to)), - Collections.emptyMap(), - MessagingService.Verb.REPAIR_MESSAGE, - MessagingService.current_version); + else return Message.out(Verb.REPAIR_REQ, new PrepareConsistentResponse(sessionIdFunc.apply((PrepareConsistentRequest) msgOut.payload), to, !failed.contains(to))); }); } @@ -292,23 +289,20 @@ private MockMessagingSpy createFinalizeSpy(Collection failed Collection timeout) { return MockMessagingService.when( - all(verb(MessagingService.Verb.REPAIR_MESSAGE), + all(verb(Verb.REPAIR_REQ), payload((p) -> p instanceof FinalizePropose)) ).respond((msgOut, to) -> { if(timeout.contains(to)) return null; - else return MessageIn.create(to, - new FinalizePromise(((FinalizePropose) msgOut.payload).sessionID, to, !failed.contains(to)), - Collections.emptyMap(), - MessagingService.Verb.REPAIR_MESSAGE, - MessagingService.current_version); + else return Message.out(Verb.REPAIR_REQ, + new FinalizePromise(((FinalizePropose) msgOut.payload).sessionID, to, !failed.contains(to))); }); } private MockMessagingSpy createCommitSpy() { return MockMessagingService.when( - all(verb(MessagingService.Verb.REPAIR_MESSAGE), + all(verb(Verb.REPAIR_REQ), payload((p) -> p instanceof FinalizeCommit)) ).dontReply(); } @@ -316,7 +310,7 @@ private MockMessagingSpy createCommitSpy() private MockMessagingSpy createFailSessionSpy(Collection participants) { return MockMessagingService.when( - all(verb(MessagingService.Verb.REPAIR_MESSAGE), + all(verb(Verb.REPAIR_REQ), payload((p) -> p instanceof FailSession), to(participants::contains)) ).dontReply(); diff --git a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java index 5fdbe7b40f4d..d2e85b222a4e 100644 --- a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java +++ b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java @@ -21,7 +21,6 @@ import java.io.File; import java.io.IOException; import java.security.cert.CertificateException; -import java.util.Arrays; import javax.net.ssl.TrustManagerFactory; import org.apache.commons.io.FileUtils; @@ -65,11 +64,11 @@ public class SSLFactoryTest @Before public void setup() { - encryptionOptions = new ServerEncryptionOptions(); - encryptionOptions.truststore = "test/conf/cassandra_ssl_test.truststore"; - encryptionOptions.truststore_password = "cassandra"; - encryptionOptions.require_client_auth = false; - encryptionOptions.cipher_suites = new String[] {"TLS_RSA_WITH_AES_128_CBC_SHA"}; + encryptionOptions = new ServerEncryptionOptions() + .withTrustStore("test/conf/cassandra_ssl_test.truststore") + .withTrustStorePassword("cassandra") + .withRequireClientAuth(false) + .withCipherSuites("TLS_RSA_WITH_AES_128_CBC_SHA"); SSLFactory.checkedExpiry = false; } @@ -108,28 +107,25 @@ public void getSslContext_JdkSsl() throws IOException SslContext sslContext = SSLFactory.getOrCreateSslContext(options, true, SSLFactory.SocketType.CLIENT, false); Assert.assertNotNull(sslContext); Assert.assertTrue(sslContext instanceof JdkSslContext); - Assert.assertEquals(Arrays.asList(encryptionOptions.cipher_suites), sslContext.cipherSuites()); + Assert.assertEquals(encryptionOptions.cipher_suites, sslContext.cipherSuites()); } - private EncryptionOptions addKeystoreOptions(EncryptionOptions options) + private ServerEncryptionOptions addKeystoreOptions(ServerEncryptionOptions options) { - options.keystore = "test/conf/cassandra_ssl_test.keystore"; - options.keystore_password = "cassandra"; - return options; + return options.withKeyStore("test/conf/cassandra_ssl_test.keystore") + .withKeyStorePassword("cassandra"); } @Test(expected = IOException.class) public void buildTrustManagerFactory_NoFile() throws IOException { - encryptionOptions.truststore = "/this/is/probably/not/a/file/on/your/test/machine"; - SSLFactory.buildTrustManagerFactory(encryptionOptions); + SSLFactory.buildTrustManagerFactory(encryptionOptions.withTrustStore("/this/is/probably/not/a/file/on/your/test/machine")); } @Test(expected = IOException.class) public void buildTrustManagerFactory_BadPassword() throws IOException { - encryptionOptions.truststore_password = "HomeOfBadPasswords"; - SSLFactory.buildTrustManagerFactory(encryptionOptions); + SSLFactory.buildTrustManagerFactory(encryptionOptions.withTrustStorePassword("HomeOfBadPasswords")); } @Test @@ -142,16 +138,16 @@ public void buildTrustManagerFactory_HappyPath() throws IOException @Test(expected = IOException.class) public void buildKeyManagerFactory_NoFile() throws IOException { - EncryptionOptions options = addKeystoreOptions(encryptionOptions); - options.keystore = "/this/is/probably/not/a/file/on/your/test/machine"; + EncryptionOptions options = addKeystoreOptions(encryptionOptions) + .withKeyStore("/this/is/probably/not/a/file/on/your/test/machine"); SSLFactory.buildKeyManagerFactory(options); } @Test(expected = IOException.class) public void buildKeyManagerFactory_BadPassword() throws IOException { - EncryptionOptions options = addKeystoreOptions(encryptionOptions); - encryptionOptions.keystore_password = "HomeOfBadPasswords"; + EncryptionOptions options = addKeystoreOptions(encryptionOptions) + .withKeyStorePassword("HomeOfBadPasswords"); SSLFactory.buildKeyManagerFactory(options); } @@ -169,8 +165,8 @@ public void testSslContextReload_HappyPath() throws IOException, InterruptedExce { try { - EncryptionOptions options = addKeystoreOptions(encryptionOptions); - options.enabled = true; + EncryptionOptions options = addKeystoreOptions(encryptionOptions) + .withEnabled(true); SSLFactory.initHotReloading((ServerEncryptionOptions) options, options, true); @@ -201,9 +197,9 @@ public void testSslContextReload_HappyPath() throws IOException, InterruptedExce @Test(expected = IOException.class) public void testSslFactorySslInit_BadPassword_ThrowsException() throws IOException { - EncryptionOptions options = addKeystoreOptions(encryptionOptions); - options.keystore_password = "bad password"; - options.enabled = true; + EncryptionOptions options = addKeystoreOptions(encryptionOptions) + .withKeyStorePassword("bad password") + .withEnabled(true); SSLFactory.initHotReloading((ServerEncryptionOptions) options, options, true); } @@ -213,10 +209,8 @@ public void testSslFactoryHotReload_BadPassword_DoesNotClearExistingSslContext() { try { - addKeystoreOptions(encryptionOptions); - - ServerEncryptionOptions options = new ServerEncryptionOptions(encryptionOptions); - options.enabled = true; + ServerEncryptionOptions options = addKeystoreOptions(encryptionOptions) + .withEnabled(true); SSLFactory.initHotReloading(options, options, true); SslContext oldCtx = SSLFactory.getOrCreateSslContext(options, true, SSLFactory.SocketType.CLIENT, OpenSsl @@ -226,8 +220,8 @@ public void testSslFactoryHotReload_BadPassword_DoesNotClearExistingSslContext() SSLFactory.checkCertFilesForHotReloading(options, options); keystoreFile.setLastModified(System.currentTimeMillis() + 5000); - ServerEncryptionOptions modOptions = new ServerEncryptionOptions(options); - modOptions.keystore_password = "bad password"; + ServerEncryptionOptions modOptions = new ServerEncryptionOptions(options) + .withKeyStorePassword("bad password"); SSLFactory.checkCertFilesForHotReloading(modOptions, modOptions); SslContext newCtx = SSLFactory.getOrCreateSslContext(options, true, SSLFactory.SocketType.CLIENT, OpenSsl .isAvailable()); @@ -241,29 +235,28 @@ public void testSslFactoryHotReload_BadPassword_DoesNotClearExistingSslContext() } @Test - public void testSslFactoryHotReload_CorruptOrNonExistentFile_DoesNotClearExistingSslContext() throws IOException, - InterruptedException + public void testSslFactoryHotReload_CorruptOrNonExistentFile_DoesNotClearExistingSslContext() throws IOException { try { - addKeystoreOptions(encryptionOptions); + ServerEncryptionOptions options = addKeystoreOptions(encryptionOptions); - File testKeystoreFile = new File(encryptionOptions.keystore + ".test"); - FileUtils.copyFile(new File(encryptionOptions.keystore),testKeystoreFile); - encryptionOptions.keystore = testKeystoreFile.getPath(); + File testKeystoreFile = new File(options.keystore + ".test"); + FileUtils.copyFile(new File(options.keystore),testKeystoreFile); + options = options + .withKeyStore(testKeystoreFile.getPath()) + .withEnabled(true); - EncryptionOptions options = new ServerEncryptionOptions(encryptionOptions); - options.enabled = true; - SSLFactory.initHotReloading((ServerEncryptionOptions) options, options, true); + SSLFactory.initHotReloading(options, options, true); SslContext oldCtx = SSLFactory.getOrCreateSslContext(options, true, SSLFactory.SocketType.CLIENT, OpenSsl .isAvailable()); - SSLFactory.checkCertFilesForHotReloading((ServerEncryptionOptions) options, options); + SSLFactory.checkCertFilesForHotReloading(options, options); testKeystoreFile.setLastModified(System.currentTimeMillis() + 15000); FileUtils.forceDelete(testKeystoreFile); - SSLFactory.checkCertFilesForHotReloading((ServerEncryptionOptions) options, options);; + SSLFactory.checkCertFilesForHotReloading(options, options);; SslContext newCtx = SSLFactory.getOrCreateSslContext(options, true, SSLFactory.SocketType.CLIENT, OpenSsl .isAvailable()); @@ -283,22 +276,22 @@ public void testSslFactoryHotReload_CorruptOrNonExistentFile_DoesNotClearExistin @Test public void getSslContext_ParamChanges() throws IOException { - EncryptionOptions options = addKeystoreOptions(encryptionOptions); - options.enabled = true; - options.cipher_suites = new String[]{ "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256" }; + EncryptionOptions options = addKeystoreOptions(encryptionOptions) + .withEnabled(true) + .withCipherSuites("TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256"); SslContext ctx1 = SSLFactory.getOrCreateSslContext(options, true, SSLFactory.SocketType.SERVER, OpenSsl.isAvailable()); Assert.assertTrue(ctx1.isServer()); - Assert.assertArrayEquals(ctx1.cipherSuites().toArray(), options.cipher_suites); + Assert.assertEquals(ctx1.cipherSuites(), options.cipher_suites); - options.cipher_suites = new String[]{ "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256" }; + options = options.withCipherSuites("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); SslContext ctx2 = SSLFactory.getOrCreateSslContext(options, true, SSLFactory.SocketType.CLIENT, OpenSsl.isAvailable()); Assert.assertTrue(ctx2.isClient()); - Assert.assertArrayEquals(ctx2.cipherSuites().toArray(), options.cipher_suites); + Assert.assertEquals(ctx2.cipherSuites(), options.cipher_suites); } } diff --git a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java index c918fd6f1f38..80b3596c80a8 100644 --- a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java +++ b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java @@ -47,7 +47,7 @@ public static void setupDD() @After public void resetConfig() { - DatabaseDescriptor.getNativeProtocolEncryptionOptions().enabled = false; + DatabaseDescriptor.updateNativeProtocolEncryptionOptions(options -> options.withEnabled(false)); DatabaseDescriptor.setNativeTransportPortSSL(null); } @@ -128,8 +128,8 @@ public void testPlainDefaultPort() public void testSSLOnly() { // default ssl settings: client encryption enabled and default native transport port used for ssl only - DatabaseDescriptor.getNativeProtocolEncryptionOptions().enabled = true; - DatabaseDescriptor.getNativeProtocolEncryptionOptions().optional = false; + DatabaseDescriptor.updateNativeProtocolEncryptionOptions(options -> options.withEnabled(true) + .withOptional(false)); withService((NativeTransportService service) -> { @@ -145,8 +145,8 @@ public void testSSLOnly() public void testSSLOptional() { // default ssl settings: client encryption enabled and default native transport port used for optional ssl - DatabaseDescriptor.getNativeProtocolEncryptionOptions().enabled = true; - DatabaseDescriptor.getNativeProtocolEncryptionOptions().optional = true; + DatabaseDescriptor.updateNativeProtocolEncryptionOptions(options -> options.withEnabled(true) + .withOptional(true)); withService((NativeTransportService service) -> { @@ -162,7 +162,7 @@ public void testSSLOptional() public void testSSLWithNonSSL() { // ssl+non-ssl settings: client encryption enabled and additional ssl port specified - DatabaseDescriptor.getNativeProtocolEncryptionOptions().enabled = true; + DatabaseDescriptor.updateNativeProtocolEncryptionOptions(options -> options.withEnabled(true)); DatabaseDescriptor.setNativeTransportPortSSL(8432); withService((NativeTransportService service) -> diff --git a/test/unit/org/apache/cassandra/service/RemoveTest.java b/test/unit/org/apache/cassandra/service/RemoveTest.java index 0d39322ef678..e6fbe7beedeb 100644 --- a/test/unit/org/apache/cassandra/service/RemoveTest.java +++ b/test/unit/org/apache/cassandra/service/RemoveTest.java @@ -41,10 +41,12 @@ import org.apache.cassandra.gms.VersionedValue.VersionedValueFactory; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.TokenMetadata; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.net.NoPayload.noPayload; +import static org.apache.cassandra.net.Verb.REPLICATION_DONE_REQ; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -96,8 +98,9 @@ public void setup() throws IOException, ConfigurationException @After public void tearDown() { - MessagingService.instance().clearMessageSinks(); - MessagingService.instance().clearCallbacksUnsafe(); + MessagingService.instance().inboundSink.clear(); + MessagingService.instance().outboundSink.clear(); + MessagingService.instance().callbacks.unsafeClear(); } @Test(expected = UnsupportedOperationException.class) @@ -161,8 +164,10 @@ public void testRemoveHostId() throws InterruptedException for (InetAddressAndPort host : hosts) { - MessageOut msg = new MessageOut(host, MessagingService.Verb.REPLICATION_FINISHED, null, null, Collections.emptyList(), null); - MessagingService.instance().sendRR(msg, FBUtilities.getBroadcastAddressAndPort()); + Message msg = Message.builder(REPLICATION_DONE_REQ, noPayload) + .from(host) + .build(); + MessagingService.instance().send(msg, FBUtilities.getBroadcastAddressAndPort()); } remover.join(); diff --git a/test/unit/org/apache/cassandra/service/SerializationsTest.java b/test/unit/org/apache/cassandra/service/SerializationsTest.java index 12236831abc0..273d23302e8b 100644 --- a/test/unit/org/apache/cassandra/service/SerializationsTest.java +++ b/test/unit/org/apache/cassandra/service/SerializationsTest.java @@ -41,7 +41,7 @@ import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus; import org.apache.cassandra.io.util.DataOutputStreamPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.repair.SyncNodePair; import org.apache.cassandra.repair.RepairJobDesc; import org.apache.cassandra.repair.Validator; @@ -88,9 +88,6 @@ private void testRepairMessageWrite(String fileName, RepairMessage... messages) testSerializedSize(message, RepairMessage.serializer); RepairMessage.serializer.serialize(message, out, getVersion()); } - // also serialize MessageOut - for (RepairMessage message : messages) - message.createMessage().serialize(out, getVersion()); } } @@ -112,8 +109,6 @@ public void testValidationRequestRead() throws IOException assert message.messageType == RepairMessage.Type.VALIDATION_REQUEST; assert DESC.equals(message.desc); assert ((ValidationRequest) message).nowInSec == 1234; - - assert MessageIn.read(in, getVersion(), -1) != null; } } @@ -173,10 +168,6 @@ public void testValidationCompleteRead() throws IOException assert !((ValidationComplete) message).success(); assert ((ValidationComplete) message).trees == null; - - // MessageOuts - for (int i = 0; i < 3; i++) - assert MessageIn.read(in, getVersion(), -1) != null; } } @@ -209,8 +200,6 @@ public void testSyncRequestRead() throws IOException assert src.equals(((SyncRequest) message).src); assert dest.equals(((SyncRequest) message).dst); assert ((SyncRequest) message).ranges.size() == 1 && ((SyncRequest) message).ranges.contains(FULL_RANGE); - - assert MessageIn.read(in, getVersion(), -1) != null; } } @@ -260,10 +249,6 @@ public void testSyncCompleteRead() throws IOException assert nodes.equals(((SyncComplete) message).nodes); assert !((SyncComplete) message).success; - - // MessageOuts - for (int i = 0; i < 2; i++) - assert MessageIn.read(in, getVersion(), -1) != null; } } } diff --git a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java index 2c186ba98d90..f06b706c68f5 100644 --- a/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java +++ b/test/unit/org/apache/cassandra/service/WriteResponseHandlerTest.java @@ -44,10 +44,12 @@ import org.apache.cassandra.locator.ReplicaCollection; import org.apache.cassandra.locator.ReplicaUtils; import org.apache.cassandra.locator.TokenMetadata; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.utils.ByteBufferUtil; +import static org.apache.cassandra.net.NoPayload.noPayload; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -146,11 +148,11 @@ public void idealCLLatencyTracked() throws Throwable AbstractWriteResponseHandler awr = createWriteResponseHandler(ConsistencyLevel.LOCAL_QUORUM, ConsistencyLevel.EACH_QUORUM, System.nanoTime() - TimeUnit.DAYS.toNanos(1)); //dc1 - awr.response(createDummyMessage(0)); - awr.response(createDummyMessage(1)); + awr.onResponse(createDummyMessage(0)); + awr.onResponse(createDummyMessage(1)); //dc2 - awr.response(createDummyMessage(4)); - awr.response(createDummyMessage(5)); + awr.onResponse(createDummyMessage(4)); + awr.onResponse(createDummyMessage(5)); //Don't need the others awr.expired(); @@ -172,13 +174,13 @@ public void idealCLWriteResponeHandlerWorks() throws Throwable AbstractWriteResponseHandler awr = createWriteResponseHandler(ConsistencyLevel.LOCAL_QUORUM, ConsistencyLevel.ALL); //dc1 - awr.response(createDummyMessage(0)); - awr.response(createDummyMessage(1)); - awr.response(createDummyMessage(2)); + awr.onResponse(createDummyMessage(0)); + awr.onResponse(createDummyMessage(1)); + awr.onResponse(createDummyMessage(2)); //dc2 - awr.response(createDummyMessage(3)); - awr.response(createDummyMessage(4)); - awr.response(createDummyMessage(5)); + awr.onResponse(createDummyMessage(3)); + awr.onResponse(createDummyMessage(4)); + awr.onResponse(createDummyMessage(5)); assertEquals(0, ks.metric.writeFailedIdealCL.getCount()); assertEquals(startingCount + 1, ks.metric.idealCLWriteLatency.latency.getCount()); @@ -195,13 +197,13 @@ public void idealCLDatacenterWriteResponeHandlerWorks() throws Throwable AbstractWriteResponseHandler awr = createWriteResponseHandler(ConsistencyLevel.ONE, ConsistencyLevel.LOCAL_QUORUM); //dc1 - awr.response(createDummyMessage(0)); - awr.response(createDummyMessage(1)); - awr.response(createDummyMessage(2)); + awr.onResponse(createDummyMessage(0)); + awr.onResponse(createDummyMessage(1)); + awr.onResponse(createDummyMessage(2)); //dc2 - awr.response(createDummyMessage(3)); - awr.response(createDummyMessage(4)); - awr.response(createDummyMessage(5)); + awr.onResponse(createDummyMessage(3)); + awr.onResponse(createDummyMessage(4)); + awr.onResponse(createDummyMessage(5)); assertEquals(0, ks.metric.writeFailedIdealCL.getCount()); assertEquals(startingCount + 1, ks.metric.idealCLWriteLatency.latency.getCount()); @@ -218,9 +220,9 @@ public void failedIdealCLIncrementsStat() throws Throwable AbstractWriteResponseHandler awr = createWriteResponseHandler(ConsistencyLevel.LOCAL_QUORUM, ConsistencyLevel.EACH_QUORUM); //Succeed in local DC - awr.response(createDummyMessage(0)); - awr.response(createDummyMessage(1)); - awr.response(createDummyMessage(2)); + awr.onResponse(createDummyMessage(0)); + awr.onResponse(createDummyMessage(1)); + awr.onResponse(createDummyMessage(2)); //Fail in remote DC awr.expired(); @@ -241,8 +243,10 @@ private static AbstractWriteResponseHandler createWriteResponseHandler(Consisten null, WriteType.SIMPLE, queryStartTime, ideal); } - private static MessageIn createDummyMessage(int target) + private static Message createDummyMessage(int target) { - return MessageIn.create(targets.get(target).endpoint(), null, null, null, 0, 0L); + return Message.builder(Verb.ECHO_REQ, noPayload) + .from(targets.get(target).endpoint()) + .build(); } } diff --git a/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java b/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java index 582aff8bf67a..545731b87d3d 100644 --- a/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java +++ b/test/unit/org/apache/cassandra/service/reads/AbstractReadResponseTest.java @@ -65,7 +65,7 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessageIn; +import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.KeyspaceParams; @@ -73,6 +73,8 @@ import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; +import static org.apache.cassandra.net.Verb.READ_REQ; + /** * Base class for testing various components which deal with read responses */ @@ -216,7 +218,7 @@ static DecoratedKey dk(int k) } - static MessageIn response(ReadCommand command, + static Message response(ReadCommand command, InetAddressAndPort from, UnfilteredPartitionIterator data, boolean isDigestResponse, @@ -227,10 +229,12 @@ static MessageIn response(ReadCommand command, ReadResponse response = isDigestResponse ? ReadResponse.createDigestResponse(data, command) : ReadResponse.createRemoteDataResponse(data, repairedDataDigest, hasPendingRepair, command, fromVersion); - return MessageIn.create(from, response, Collections.emptyMap(), MessagingService.Verb.READ, fromVersion); + return Message.builder(READ_REQ, response) + .from(from) + .build(); } - static MessageIn response(InetAddressAndPort from, + static Message response(InetAddressAndPort from, UnfilteredPartitionIterator partitionIterator, ByteBuffer repairedDigest, boolean hasPendingRepair, @@ -239,12 +243,12 @@ static MessageIn response(InetAddressAndPort from, return response(cmd, from, partitionIterator, false, MessagingService.current_version, repairedDigest, hasPendingRepair); } - static MessageIn response(ReadCommand command, InetAddressAndPort from, UnfilteredPartitionIterator data, boolean isDigestResponse) + static Message response(ReadCommand command, InetAddressAndPort from, UnfilteredPartitionIterator data, boolean isDigestResponse) { return response(command, from, data, false, MessagingService.current_version, ByteBufferUtil.EMPTY_BYTE_BUFFER, isDigestResponse); } - static MessageIn response(ReadCommand command, InetAddressAndPort from, UnfilteredPartitionIterator data) + static Message response(ReadCommand command, InetAddressAndPort from, UnfilteredPartitionIterator data) { return response(command, from, data, false, MessagingService.current_version, ByteBufferUtil.EMPTY_BYTE_BUFFER, false); } diff --git a/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java b/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java index 34be5ee5ed68..e0a59276ae5b 100644 --- a/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java +++ b/test/unit/org/apache/cassandra/service/reads/ReadExecutorTest.java @@ -20,10 +20,8 @@ import java.util.concurrent.TimeUnit; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.ReplicaPlan; import org.junit.Before; import org.junit.BeforeClass; @@ -40,12 +38,12 @@ import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.locator.EndpointsForToken; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.locator.ReplicaLayout; -import org.apache.cassandra.locator.ReplicaUtils; -import org.apache.cassandra.net.MessageOut; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.NoPayload; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.schema.KeyspaceParams; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.cassandra.locator.ReplicaUtils.full; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -205,24 +203,16 @@ public static class MockSinglePartitionReadCommand extends SinglePartitionReadCo } @Override - public long getTimeout() + public long getTimeout(TimeUnit unit) { - return timeout; + return unit.convert(timeout, MILLISECONDS); } @Override - public MessageOut createMessage() + public Message createMessage(boolean trackRepairedData) { - return new MessageOut(MessagingService.Verb.BATCH_REMOVE) - { - @Override - public int serializedSize(int version) - { - return 0; - } - }; + return Message.out(Verb.ECHO_REQ, NoPayload.noPayload); } - } private ReplicaPlan.ForTokenRead plan(EndpointsForToken targets, ConsistencyLevel consistencyLevel) diff --git a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java index 68afedcf7696..3d3973273cab 100644 --- a/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java +++ b/test/unit/org/apache/cassandra/service/reads/repair/AbstractReadRepairTest.java @@ -1,6 +1,5 @@ package org.apache.cassandra.service.reads.repair; -import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -42,10 +41,8 @@ import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.locator.ReplicaLayout; import org.apache.cassandra.locator.ReplicaUtils; -import org.apache.cassandra.net.MessageIn; -import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Message; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.MigrationManager; @@ -55,6 +52,7 @@ import static org.apache.cassandra.locator.Replica.fullReplica; import static org.apache.cassandra.locator.ReplicaUtils.FULL_RANGE; +import static org.apache.cassandra.net.Verb.INTERNAL_RSP; @Ignore public abstract class AbstractReadRepairTest @@ -163,14 +161,12 @@ static Mutation mutation(Cell... cells) } @SuppressWarnings("resource") - static MessageIn msg(InetAddressAndPort from, Cell... cells) + static Message msg(InetAddressAndPort from, Cell... cells) { UnfilteredPartitionIterator iter = new SingletonUnfilteredPartitionIterator(update(cells).unfilteredIterator()); - return MessageIn.create(from, - ReadResponse.createDataResponse(iter, command), - Collections.emptyMap(), - MessagingService.Verb.INTERNAL_RESPONSE, - MessagingService.current_version); + return Message.builder(INTERNAL_RSP, ReadResponse.createDataResponse(iter, command)) + .from(from) + .build(); } static class ResultConsumer implements Consumer @@ -306,8 +302,8 @@ public void noSpeculationRequired() repair.startRepair(null, consumer); Assert.assertEquals(epSet(target1, target2), repair.getReadRecipients()); - repair.getReadCallback().response(msg(target1, cell1)); - repair.getReadCallback().response(msg(target2, cell1)); + repair.getReadCallback().onResponse(msg(target1, cell1)); + repair.getReadCallback().onResponse(msg(target2, cell1)); repair.maybeSendAdditionalReads(); Assert.assertEquals(epSet(target1, target2), repair.getReadRecipients()); diff --git a/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java index 6ea593deb520..7538832add9b 100644 --- a/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java +++ b/test/unit/org/apache/cassandra/service/reads/repair/BlockingReadRepairTest.java @@ -39,7 +39,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaUtils; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.service.reads.ReadCallback; public class BlockingReadRepairTest extends AbstractReadRepairTest @@ -55,7 +55,7 @@ public InstrumentedReadRepairHandler(Map repairs, int maxBloc Map mutationsSent = new HashMap<>(); - protected void sendRR(MessageOut message, InetAddressAndPort endpoint) + protected void sendRR(Message message, InetAddressAndPort endpoint) { mutationsSent.put(endpoint, message.payload); } diff --git a/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java index befa07af1f1a..3bcd757f2d1b 100644 --- a/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java +++ b/test/unit/org/apache/cassandra/service/reads/repair/DiagEventsBlockingReadRepairTest.java @@ -44,7 +44,7 @@ import org.apache.cassandra.locator.EndpointsForRange; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.service.reads.ReadCallback; import org.apache.cassandra.service.reads.repair.ReadRepairEvent.ReadRepairEventType; @@ -187,7 +187,7 @@ private void onRepairEvent(PartitionRepairEvent e) Assert.assertNotNull(e.toMap()); } - protected void sendRR(MessageOut message, InetAddressAndPort endpoint) + protected void sendRR(Message message, InetAddressAndPort endpoint) { } } diff --git a/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java b/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java index c3f05c0777bd..232644db58db 100644 --- a/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java +++ b/test/unit/org/apache/cassandra/service/reads/repair/ReadRepairTest.java @@ -51,7 +51,7 @@ import org.apache.cassandra.locator.Replica; import org.apache.cassandra.locator.ReplicaLayout; import org.apache.cassandra.locator.ReplicaUtils; -import org.apache.cassandra.net.MessageOut; +import org.apache.cassandra.net.Message; import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.MigrationManager; @@ -82,7 +82,7 @@ public InstrumentedReadRepairHandler(Map repairs, int maxBloc Map mutationsSent = new HashMap<>(); - protected void sendRR(MessageOut message, InetAddressAndPort endpoint) + protected void sendRR(Message message, InetAddressAndPort endpoint) { mutationsSent.put(endpoint, message.payload); } diff --git a/test/unit/org/apache/cassandra/streaming/StreamSessionTest.java b/test/unit/org/apache/cassandra/streaming/StreamSessionTest.java deleted file mode 100644 index 7ea09ea5b908..000000000000 --- a/test/unit/org/apache/cassandra/streaming/StreamSessionTest.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.streaming; - -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.Collections; -import java.util.UUID; - -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.MessagingServiceTest; - -import static org.junit.Assert.assertEquals; - -public class StreamSessionTest -{ - @BeforeClass - public static void beforeClass() throws UnknownHostException - { - DatabaseDescriptor.daemonInitialization(); - DatabaseDescriptor.setBackPressureStrategy(new MessagingServiceTest.MockBackPressureStrategy(Collections.emptyMap())); - DatabaseDescriptor.setBroadcastAddress(InetAddress.getByName("127.0.0.3")); - } - - @Test - public void testStreamSessionUsesCorrectRemoteIp_Succeeds() throws UnknownHostException - { - InetAddressAndPort localAddr = InetAddressAndPort.getByName("127.0.0.1:7000"); - InetAddressAndPort preferredAddr = InetAddressAndPort.getByName("127.0.0.2:7000"); - StreamSession streamSession = new StreamSession(StreamOperation.BOOTSTRAP, localAddr, - new DefaultConnectionFactory(), 0, UUID.randomUUID(), PreviewKind.ALL, - inetAddressAndPort -> preferredAddr); - - assertEquals(preferredAddr, streamSession.getMessageSender().getConnectionId().connectionAddress()); - } - - @Test - public void testStreamSessionUsesCorrectRemoteIpNullMapper_Succeeds() throws UnknownHostException - { - InetAddressAndPort localAddr = InetAddressAndPort.getByName("127.0.0.1:7000"); - - StreamSession streamSession = new StreamSession(StreamOperation.BOOTSTRAP, localAddr, - new DefaultConnectionFactory(), 0, UUID.randomUUID(), PreviewKind.ALL, (peer) -> null); - - assertEquals(localAddr, streamSession.getMessageSender().getConnectionId().connectionAddress()); - } -} diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java index 8ebe333622b8..80607c56ea9a 100644 --- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java +++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java @@ -76,7 +76,7 @@ public void tearDown() public void testScheduleTimeout() throws Exception { InetAddressAndPort peer = FBUtilities.getBroadcastAddressAndPort(); - StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, peer, (connectionId, protocolVersion) -> new EmbeddedChannel(), 0, UUID.randomUUID(), PreviewKind.ALL); + StreamSession session = new StreamSession(StreamOperation.BOOTSTRAP, peer, (template, messagingVersion) -> new EmbeddedChannel(), 0, UUID.randomUUID(), PreviewKind.ALL); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD); // create two sstables diff --git a/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java b/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java index 52f097af6211..957869b38269 100644 --- a/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java +++ b/test/unit/org/apache/cassandra/streaming/async/NettyStreamingMessageSenderTest.java @@ -32,10 +32,10 @@ import org.junit.Test; import io.netty.channel.ChannelPromise; -import io.netty.channel.embedded.EmbeddedChannel; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.TestScheduledFuture; +import org.apache.cassandra.net.TestChannel; +import org.apache.cassandra.net.TestScheduledFuture; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamOperation; import org.apache.cassandra.streaming.StreamResultFuture; @@ -46,7 +46,7 @@ public class NettyStreamingMessageSenderTest { private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0); - private EmbeddedChannel channel; + private TestChannel channel; private StreamSession session; private NettyStreamingMessageSender sender; private NettyStreamingMessageSender.FileStreamTask fileStreamTask; @@ -60,10 +60,10 @@ public static void before() @Before public void setUp() { - channel = new EmbeddedChannel(); + channel = new TestChannel(Integer.MAX_VALUE); channel.attr(NettyStreamingMessageSender.TRANSFERRING_FILE_ATTR).set(Boolean.FALSE); UUID pendingRepair = UUID.randomUUID(); - session = new StreamSession(StreamOperation.BOOTSTRAP, REMOTE_ADDR, (connectionId, protocolVersion) -> null, 0, pendingRepair, PreviewKind.ALL); + session = new StreamSession(StreamOperation.BOOTSTRAP, REMOTE_ADDR, (template, messagingVersion) -> null, 0, pendingRepair, PreviewKind.ALL); StreamResultFuture future = StreamResultFuture.initReceivingSide(0, UUID.randomUUID(), StreamOperation.REPAIR, REMOTE_ADDR, channel, pendingRepair, session.getPreviewKind()); session.init(future); sender = session.getMessageSender(); diff --git a/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java b/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java index e274f27e95aa..a88092ed40e1 100644 --- a/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java +++ b/test/unit/org/apache/cassandra/streaming/async/StreamCompressionSerializerTest.java @@ -31,17 +31,18 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.Unpooled; import net.jpountz.lz4.LZ4Compressor; import net.jpountz.lz4.LZ4Factory; import net.jpountz.lz4.LZ4FastDecompressor; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.streaming.messages.StreamMessage; +import org.apache.cassandra.net.MessagingService; public class StreamCompressionSerializerTest { - private static final int VERSION = StreamMessage.CURRENT_VERSION; + private static final int VERSION = MessagingService.current_version; private static final Random random = new Random(2347623847623L); private final ByteBufAllocator allocator = PooledByteBufAllocator.DEFAULT; @@ -50,7 +51,7 @@ public class StreamCompressionSerializerTest private final LZ4FastDecompressor decompressor = LZ4Factory.fastestInstance().fastDecompressor(); private ByteBuffer input; - private ByteBuf compressed; + private ByteBuffer compressed; private ByteBuf output; @BeforeClass @@ -64,8 +65,8 @@ public void tearDown() { if (input != null) FileUtils.clean(input); - if (compressed != null && compressed.refCnt() > 0) - compressed.release(compressed.refCnt()); + if (compressed != null) + FileUtils.clean(compressed); if (output != null && output.refCnt() > 0) output.release(output.refCnt()); } @@ -74,10 +75,9 @@ public void tearDown() public void roundTrip_HappyPath_NotReadabaleByteBuffer() throws IOException { populateInput(); - compressed = serializer.serialize(compressor, input, VERSION); + StreamCompressionSerializer.serialize(compressor, input, VERSION).write(size -> compressed = ByteBuffer.allocateDirect(size)); input.flip(); - ByteBuffer compressedNioBuffer = compressed.nioBuffer(0, compressed.writerIndex()); - output = serializer.deserialize(decompressor, new DataInputBuffer(compressedNioBuffer, false), VERSION); + output = serializer.deserialize(decompressor, new DataInputBuffer(compressed, false), VERSION); validateResults(); } @@ -101,9 +101,14 @@ private void validateResults() public void roundTrip_HappyPath_ReadabaleByteBuffer() throws IOException { populateInput(); - compressed = serializer.serialize(compressor, input, VERSION); + StreamCompressionSerializer.serialize(compressor, input, VERSION) + .write(size -> { + if (compressed != null) + FileUtils.clean(compressed); + return compressed = ByteBuffer.allocateDirect(size); + }); input.flip(); - output = serializer.deserialize(decompressor, new ByteBufRCH(compressed), VERSION); + output = serializer.deserialize(decompressor, new ByteBufRCH(Unpooled.wrappedBuffer(compressed)), VERSION); validateResults(); } diff --git a/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java index 0a135960f970..6a2afe8e4024 100644 --- a/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java +++ b/test/unit/org/apache/cassandra/streaming/async/StreamingInboundHandlerTest.java @@ -18,9 +18,6 @@ package org.apache.cassandra.streaming.async; -import java.io.EOFException; -import java.io.IOException; -import java.util.ArrayList; import java.util.UUID; import com.google.common.net.InetAddresses; @@ -33,10 +30,9 @@ import io.netty.buffer.ByteBuf; import io.netty.channel.embedded.EmbeddedChannel; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.io.sstable.format.SSTableFormat; -import org.apache.cassandra.io.sstable.format.big.BigFormat; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.async.RebufferingByteBufDataInputPlus; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.AsyncStreamingInputPlus; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamManager; @@ -45,19 +41,18 @@ import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.streaming.async.StreamingInboundHandler.SessionIdentifier; import org.apache.cassandra.streaming.messages.CompleteMessage; -import org.apache.cassandra.streaming.messages.StreamMessageHeader; import org.apache.cassandra.streaming.messages.IncomingStreamMessage; import org.apache.cassandra.streaming.messages.StreamInitMessage; -import org.apache.cassandra.streaming.messages.StreamMessage; +import org.apache.cassandra.streaming.messages.StreamMessageHeader; public class StreamingInboundHandlerTest { - private static final int VERSION = StreamMessage.CURRENT_VERSION; + private static final int VERSION = MessagingService.current_version; private static final InetAddressAndPort REMOTE_ADDR = InetAddressAndPort.getByAddressOverrideDefaults(InetAddresses.forString("127.0.0.2"), 0); private StreamingInboundHandler handler; private EmbeddedChannel channel; - private RebufferingByteBufDataInputPlus buffers; + private AsyncStreamingInputPlus buffers; private ByteBuf buf; @BeforeClass @@ -71,7 +66,7 @@ public void setup() { handler = new StreamingInboundHandler(REMOTE_ADDR, VERSION, null); channel = new EmbeddedChannel(handler); - buffers = new RebufferingByteBufDataInputPlus(1 << 9, 1 << 10, channel.config()); + buffers = new AsyncStreamingInputPlus(channel); handler.setPendingBuffers(buffers); } @@ -88,19 +83,19 @@ public void tearDown() } @Test - public void channelRead_Normal() throws EOFException + public void channelRead_Normal() { - Assert.assertEquals(0, buffers.available()); + Assert.assertEquals(0, buffers.unsafeAvailable()); int size = 8; buf = channel.alloc().buffer(size); buf.writerIndex(size); channel.writeInbound(buf); - Assert.assertEquals(size, buffers.available()); + Assert.assertEquals(size, buffers.unsafeAvailable()); Assert.assertFalse(channel.releaseInbound()); } - @Test (expected = EOFException.class) - public void channelRead_Closed() throws EOFException + @Test + public void channelRead_Closed() { int size = 8; buf = channel.alloc().buffer(size); @@ -108,21 +103,21 @@ public void channelRead_Closed() throws EOFException buf.writerIndex(size); handler.close(); channel.writeInbound(buf); - Assert.assertEquals(0, buffers.available()); + Assert.assertEquals(0, buffers.unsafeAvailable()); Assert.assertEquals(0, buf.refCnt()); Assert.assertFalse(channel.releaseInbound()); } @Test - public void channelRead_WrongObject() throws EOFException + public void channelRead_WrongObject() { channel.writeInbound("homer"); - Assert.assertEquals(0, buffers.available()); + Assert.assertEquals(0, buffers.unsafeAvailable()); Assert.assertFalse(channel.releaseInbound()); } @Test - public void StreamDeserializingTask_deriveSession_StreamInitMessage() throws InterruptedException, IOException + public void StreamDeserializingTask_deriveSession_StreamInitMessage() { StreamInitMessage msg = new StreamInitMessage(REMOTE_ADDR, 0, UUID.randomUUID(), StreamOperation.REPAIR, UUID.randomUUID(), PreviewKind.ALL); StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(sid -> createSession(sid), null, channel); @@ -132,11 +127,11 @@ public void StreamDeserializingTask_deriveSession_StreamInitMessage() throws Int private StreamSession createSession(SessionIdentifier sid) { - return new StreamSession(StreamOperation.BOOTSTRAP, sid.from, (connectionId, protocolVersion) -> null, sid.sessionIndex, UUID.randomUUID(), PreviewKind.ALL); + return new StreamSession(StreamOperation.BOOTSTRAP, sid.from, (template, messagingVersion) -> null, sid.sessionIndex, UUID.randomUUID(), PreviewKind.ALL); } @Test (expected = IllegalStateException.class) - public void StreamDeserializingTask_deriveSession_NoSession() throws InterruptedException, IOException + public void StreamDeserializingTask_deriveSession_NoSession() { CompleteMessage msg = new CompleteMessage(); StreamingInboundHandler.StreamDeserializingTask task = handler.new StreamDeserializingTask(sid -> createSession(sid), null, channel); @@ -144,7 +139,7 @@ public void StreamDeserializingTask_deriveSession_NoSession() throws Interrupted } @Test (expected = IllegalStateException.class) - public void StreamDeserializingTask_deriveSession_IFM_NoSession() throws InterruptedException, IOException + public void StreamDeserializingTask_deriveSession_IFM_NoSession() { StreamMessageHeader header = new StreamMessageHeader(TableId.generate(), REMOTE_ADDR, UUID.randomUUID(), 0, 0, 0, UUID.randomUUID()); @@ -154,7 +149,7 @@ public void StreamDeserializingTask_deriveSession_IFM_NoSession() throws Interru } @Test - public void StreamDeserializingTask_deriveSession_IFM_HasSession() throws InterruptedException, IOException + public void StreamDeserializingTask_deriveSession_IFM_HasSession() { UUID planId = UUID.randomUUID(); StreamResultFuture future = StreamResultFuture.initReceivingSide(0, planId, StreamOperation.REPAIR, REMOTE_ADDR, channel, UUID.randomUUID(), PreviewKind.ALL); diff --git a/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java b/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java index c5de60b64c49..4ae8626caeb2 100644 --- a/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java +++ b/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java @@ -153,11 +153,11 @@ private void checkArrayCopy(ByteBuffer bb) { byte[] bytes = new byte[s.length()]; - ByteBufferUtil.arrayCopy(bb, bb.position(), bytes, 0, s.length()); + ByteBufferUtil.copyBytes(bb, bb.position(), bytes, 0, s.length()); assertArrayEquals(s.getBytes(), bytes); bytes = new byte[5]; - ByteBufferUtil.arrayCopy(bb, bb.position() + 3, bytes, 1, 4); + ByteBufferUtil.copyBytes(bb, bb.position() + 3, bytes, 1, 4); assertArrayEquals(Arrays.copyOfRange(s.getBytes(), 3, 7), Arrays.copyOfRange(bytes, 1, 5)); } diff --git a/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java b/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java deleted file mode 100644 index 8877fe9c1ace..000000000000 --- a/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.utils; - -import java.util.concurrent.TimeUnit; - -import org.junit.Assert; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.utils.CoalescingStrategies.Coalescable; -import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy; -import org.apache.cassandra.utils.CoalescingStrategies.FixedCoalescingStrategy; -import org.apache.cassandra.utils.CoalescingStrategies.MovingAverageCoalescingStrategy; -import org.apache.cassandra.utils.CoalescingStrategies.TimeHorizonMovingAverageCoalescingStrategy; - -public class CoalescingStrategiesTest -{ - private static final Logger logger = LoggerFactory.getLogger(CoalescingStrategiesTest.class); - private static final int WINDOW_IN_MICROS = 200; - private static final long WINDOW_IN_NANOS = TimeUnit.MICROSECONDS.toNanos(WINDOW_IN_MICROS); - private static final String DISPLAY_NAME = "Stupendopotamus"; - - static class SimpleCoalescable implements Coalescable - { - final long timestampNanos; - - SimpleCoalescable(long timestampNanos) - { - this.timestampNanos = timestampNanos; - } - - public long timestampNanos() - { - return timestampNanos; - } - } - - static long toNanos(long micros) - { - return TimeUnit.MICROSECONDS.toNanos(micros); - } - - @Test - public void testFixedCoalescingStrategy() - { - CoalescingStrategy cs = new FixedCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME); - Assert.assertEquals(WINDOW_IN_NANOS, cs.currentCoalescingTimeNanos()); - } - - @Test - public void testMovingAverageCoalescingStrategy_DoCoalesce() - { - CoalescingStrategy cs = new MovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME); - - for (int i = 0; i < MovingAverageCoalescingStrategy.SAMPLE_SIZE; i++) - cs.newArrival(new SimpleCoalescable(toNanos(i))); - Assert.assertTrue(0 < cs.currentCoalescingTimeNanos()); - } - - @Test - public void testMovingAverageCoalescingStrategy_DoNotCoalesce() - { - CoalescingStrategy cs = new MovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME); - - for (int i = 0; i < MovingAverageCoalescingStrategy.SAMPLE_SIZE; i++) - cs.newArrival(new SimpleCoalescable(toNanos(WINDOW_IN_MICROS + i) * i)); - Assert.assertTrue(0 >= cs.currentCoalescingTimeNanos()); - } - - @Test - public void testTimeHorizonStrategy_DoCoalesce() - { - long initialEpoch = 0; - CoalescingStrategy cs = new TimeHorizonMovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME, initialEpoch); - - for (int i = 0; i < 10_000; i++) - cs.newArrival(new SimpleCoalescable(toNanos(i))); - Assert.assertTrue(0 < cs.currentCoalescingTimeNanos()); - } - - @Test - public void testTimeHorizonStrategy_DoNotCoalesce() - { - long initialEpoch = 0; - CoalescingStrategy cs = new TimeHorizonMovingAverageCoalescingStrategy(WINDOW_IN_MICROS, logger, DISPLAY_NAME, initialEpoch); - - for (int i = 0; i < 1_000_000; i++) - cs.newArrival(new SimpleCoalescable(toNanos(WINDOW_IN_MICROS + i) * i)); - Assert.assertTrue(0 >= cs.currentCoalescingTimeNanos()); - } - - @Test - public void determineCoalescingTime_LargeAverageGap() - { - Assert.assertTrue(0 >= CoalescingStrategies.determineCoalescingTime(WINDOW_IN_NANOS * 2, WINDOW_IN_NANOS)); - Assert.assertTrue(0 >= CoalescingStrategies.determineCoalescingTime(Integer.MAX_VALUE, WINDOW_IN_NANOS)); - } - - @Test - public void determineCoalescingTime_SmallAvgGap() - { - Assert.assertTrue(WINDOW_IN_NANOS >= CoalescingStrategies.determineCoalescingTime(WINDOW_IN_NANOS / 2, WINDOW_IN_NANOS)); - Assert.assertTrue(WINDOW_IN_NANOS >= CoalescingStrategies.determineCoalescingTime(WINDOW_IN_NANOS - 1, WINDOW_IN_NANOS)); - Assert.assertTrue(WINDOW_IN_NANOS >= CoalescingStrategies.determineCoalescingTime(1, WINDOW_IN_NANOS)); - Assert.assertEquals(WINDOW_IN_NANOS, CoalescingStrategies.determineCoalescingTime(0, WINDOW_IN_NANOS)); - } -} diff --git a/test/unit/org/apache/cassandra/utils/FreeRunningClock.java b/test/unit/org/apache/cassandra/utils/FreeRunningClock.java index 83c8db703aaa..d85383389dbe 100644 --- a/test/unit/org/apache/cassandra/utils/FreeRunningClock.java +++ b/test/unit/org/apache/cassandra/utils/FreeRunningClock.java @@ -20,23 +20,41 @@ import java.util.concurrent.TimeUnit; /** - * A freely adjustable clock that can be used for unit testing. See {@link Clock#instance} how to + * A freely adjustable clock that can be used for unit testing. See {@link MonotonicClock#instance} how to * enable this class. */ -public class FreeRunningClock extends Clock +public class FreeRunningClock implements MonotonicClock { private long nanoTime = 0; @Override - public long nanoTime() + public long now() { return nanoTime; } @Override - public long currentTimeMillis() + public long error() { - return TimeUnit.NANOSECONDS.toMillis(nanoTime()); + return 0; + } + + @Override + public MonotonicClockTranslation translate() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isAfter(long instant) + { + return instant > nanoTime; + } + + @Override + public boolean isAfter(long now, long instant) + { + return now > instant; } public void advance(long time, TimeUnit unit) diff --git a/test/unit/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillisTest.java b/test/unit/org/apache/cassandra/utils/MonotonicClockTest.java similarity index 87% rename from test/unit/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillisTest.java rename to test/unit/org/apache/cassandra/utils/MonotonicClockTest.java index 25aeada117fe..b2891a9950e2 100644 --- a/test/unit/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillisTest.java +++ b/test/unit/org/apache/cassandra/utils/MonotonicClockTest.java @@ -17,11 +17,12 @@ */ package org.apache.cassandra.utils; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; import static org.junit.Assert.*; import org.junit.Test; -public class NanoTimeToCurrentTimeMillisTest +public class MonotonicClockTest { @Test public void testTimestampOrdering() throws Exception @@ -34,12 +35,12 @@ public void testTimestampOrdering() throws Exception now = Math.max(now, System.currentTimeMillis()); if (ii % 10000 == 0) { - NanoTimeToCurrentTimeMillis.updateNow(); + ((MonotonicClock.SampledClock) approxTime).refreshNow(); Thread.sleep(1); } nowNanos = Math.max(nowNanos, System.nanoTime()); - long convertedNow = NanoTimeToCurrentTimeMillis.convert(nowNanos); + long convertedNow = approxTime.translate().toMillisSinceEpoch(nowNanos); int maxDiff = FBUtilities.isWindows ? 15 : 1; assertTrue("convertedNow = " + convertedNow + " lastConverted = " + lastConverted + " in iteration " + ii, diff --git a/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java b/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java index 74889a1fa2b8..62cb33b91ba5 100644 --- a/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java +++ b/test/unit/org/apache/cassandra/utils/memory/BufferPoolTest.java @@ -53,7 +53,7 @@ public void setUp() @After public void cleanUp() { - BufferPool.reset(); + BufferPool.unsafeReset(); } @Test @@ -66,12 +66,12 @@ public void testGetPut() throws InterruptedException assertEquals(size, buffer.capacity()); assertEquals(true, buffer.isDirect()); - BufferPool.Chunk chunk = BufferPool.currentChunk(); + BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk); assertEquals(BufferPool.GlobalPool.MACRO_CHUNK_SIZE, BufferPool.sizeInBytes()); BufferPool.put(buffer); - assertEquals(null, BufferPool.currentChunk()); + assertEquals(null, BufferPool.unsafeCurrentChunk()); assertEquals(BufferPool.GlobalPool.MACRO_CHUNK_SIZE, BufferPool.sizeInBytes()); } @@ -81,7 +81,7 @@ public void testPageAligned() { final int size = 1024; for (int i = size; - i <= BufferPool.CHUNK_SIZE; + i <= BufferPool.NORMAL_CHUNK_SIZE; i += size) { checkPageAligned(i); @@ -115,14 +115,14 @@ public void testDifferentSizes() throws InterruptedException assertNotNull(buffer2); assertEquals(size2, buffer2.capacity()); - BufferPool.Chunk chunk = BufferPool.currentChunk(); + BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk); assertEquals(BufferPool.GlobalPool.MACRO_CHUNK_SIZE, BufferPool.sizeInBytes()); BufferPool.put(buffer1); BufferPool.put(buffer2); - assertEquals(null, BufferPool.currentChunk()); + assertEquals(null, BufferPool.unsafeCurrentChunk()); assertEquals(BufferPool.GlobalPool.MACRO_CHUNK_SIZE, BufferPool.sizeInBytes()); } @@ -165,7 +165,7 @@ public void testMaxMemoryExceeded_SmallerThanChunkSize() @Test public void testRecycle() { - requestUpToSize(RandomAccessReader.DEFAULT_BUFFER_SIZE, 3 * BufferPool.CHUNK_SIZE); + requestUpToSize(RandomAccessReader.DEFAULT_BUFFER_SIZE, 3 * BufferPool.NORMAL_CHUNK_SIZE); } private void requestDoubleMaxMemory() @@ -192,13 +192,12 @@ private void requestUpToSize(int bufferSize, int totalSize) for (ByteBuffer buffer : buffers) BufferPool.put(buffer); - } @Test public void testBigRequest() { - final int size = BufferPool.CHUNK_SIZE + 1; + final int size = BufferPool.NORMAL_CHUNK_SIZE + 1; ByteBuffer buffer = BufferPool.get(size); assertNotNull(buffer); @@ -210,30 +209,30 @@ public void testBigRequest() public void testFillUpChunks() { final int size = RandomAccessReader.DEFAULT_BUFFER_SIZE; - final int numBuffers = BufferPool.CHUNK_SIZE / size; + final int numBuffers = BufferPool.NORMAL_CHUNK_SIZE / size; List buffers1 = new ArrayList<>(numBuffers); List buffers2 = new ArrayList<>(numBuffers); for (int i = 0; i < numBuffers; i++) buffers1.add(BufferPool.get(size)); - BufferPool.Chunk chunk1 = BufferPool.currentChunk(); + BufferPool.Chunk chunk1 = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk1); for (int i = 0; i < numBuffers; i++) buffers2.add(BufferPool.get(size)); - assertEquals(2, BufferPool.numChunks()); + assertEquals(2, BufferPool.unsafeNumChunks()); for (ByteBuffer buffer : buffers1) BufferPool.put(buffer); - assertEquals(1, BufferPool.numChunks()); + assertEquals(1, BufferPool.unsafeNumChunks()); for (ByteBuffer buffer : buffers2) BufferPool.put(buffer); - assertEquals(0, BufferPool.numChunks()); + assertEquals(0, BufferPool.unsafeNumChunks()); buffers2.clear(); } @@ -242,7 +241,7 @@ public void testFillUpChunks() public void testOutOfOrderFrees() { final int size = 4096; - final int maxFreeSlots = BufferPool.CHUNK_SIZE / size; + final int maxFreeSlots = BufferPool.NORMAL_CHUNK_SIZE / size; final int[] idxs = new int[maxFreeSlots]; for (int i = 0; i < maxFreeSlots; i++) @@ -255,7 +254,7 @@ public void testOutOfOrderFrees() public void testInOrderFrees() { final int size = 4096; - final int maxFreeSlots = BufferPool.CHUNK_SIZE / size; + final int maxFreeSlots = BufferPool.NORMAL_CHUNK_SIZE / size; final int[] idxs = new int[maxFreeSlots]; for (int i = 0; i < maxFreeSlots; i++) @@ -269,23 +268,23 @@ public void testRandomFrees() { doTestRandomFrees(12345567878L); - BufferPool.reset(); + BufferPool.unsafeReset(); doTestRandomFrees(20452249587L); - BufferPool.reset(); + BufferPool.unsafeReset(); doTestRandomFrees(82457252948L); - BufferPool.reset(); + BufferPool.unsafeReset(); doTestRandomFrees(98759284579L); - BufferPool.reset(); + BufferPool.unsafeReset(); doTestRandomFrees(19475257244L); } private void doTestRandomFrees(long seed) { final int size = 4096; - final int maxFreeSlots = BufferPool.CHUNK_SIZE / size; + final int maxFreeSlots = BufferPool.NORMAL_CHUNK_SIZE / size; final int[] idxs = new int[maxFreeSlots]; for (int i = 0; i < maxFreeSlots; i++) @@ -312,10 +311,10 @@ private void doTestFrees(final int size, final int maxFreeSlots, final int[] toR buffers.add(BufferPool.get(size)); } - BufferPool.Chunk chunk = BufferPool.currentChunk(); + BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertFalse(chunk.isFree()); - int freeSize = BufferPool.CHUNK_SIZE - maxFreeSlots * size; + int freeSize = BufferPool.NORMAL_CHUNK_SIZE - maxFreeSlots * size; assertEquals(freeSize, chunk.free()); for (int i : toReleaseIdxs) @@ -352,33 +351,34 @@ public void testDifferentSizeBuffersOnOneChunk() assertTrue(buffer.capacity() >= sizes[i]); buffers.add(buffer); - sum += BufferPool.currentChunk().roundUp(buffer.capacity()); + sum += BufferPool.unsafeCurrentChunk().roundUp(buffer.capacity()); } // else the test will fail, adjust sizes as required assertTrue(sum <= BufferPool.GlobalPool.MACRO_CHUNK_SIZE); - BufferPool.Chunk chunk = BufferPool.currentChunk(); + BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk); Random rnd = new Random(); rnd.setSeed(298347529L); - while (!buffers.isEmpty()) + while (buffers.size() > 1) { int index = rnd.nextInt(buffers.size()); ByteBuffer buffer = buffers.remove(index); BufferPool.put(buffer); } + BufferPool.put(buffers.remove(0)); - assertEquals(null, BufferPool.currentChunk()); + assertEquals(null, BufferPool.unsafeCurrentChunk()); assertEquals(0, chunk.free()); } @Test public void testChunkExhausted() { - final int size = BufferPool.CHUNK_SIZE / 64; // 1kbit + final int size = BufferPool.NORMAL_CHUNK_SIZE / 64; // 1kbit int[] sizes = new int[128]; Arrays.fill(sizes, size); @@ -397,7 +397,7 @@ public void testChunkExhausted() // else the test will fail, adjust sizes as required assertTrue(sum <= BufferPool.GlobalPool.MACRO_CHUNK_SIZE); - BufferPool.Chunk chunk = BufferPool.currentChunk(); + BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk); for (int i = 0; i < sizes.length; i++) @@ -405,7 +405,7 @@ public void testChunkExhausted() BufferPool.put(buffers.get(i)); } - assertEquals(null, BufferPool.currentChunk()); + assertEquals(null, BufferPool.unsafeCurrentChunk()); assertEquals(0, chunk.free()); } @@ -505,9 +505,9 @@ private void checkBuffer(int size) ByteBuffer buffer = BufferPool.get(size); assertEquals(size, buffer.capacity()); - if (size > 0 && size < BufferPool.CHUNK_SIZE) + if (size > 0 && size < BufferPool.NORMAL_CHUNK_SIZE) { - BufferPool.Chunk chunk = BufferPool.currentChunk(); + BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk); assertEquals(chunk.capacity(), chunk.free() + chunk.roundUp(size)); } @@ -552,7 +552,7 @@ private void checkBufferWithGivenSlots(int size, long freeSlots) ByteBuffer buffer = BufferPool.get(size); // now get the current chunk and override the free slots mask - BufferPool.Chunk chunk = BufferPool.currentChunk(); + BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk); long oldFreeSlots = chunk.setFreeSlots(freeSlots); @@ -561,7 +561,7 @@ private void checkBufferWithGivenSlots(int size, long freeSlots) assertEquals(size, buffer.capacity()); BufferPool.put(buffer2); - // reset the free slots + // unsafeReset the free slots chunk.setFreeSlots(oldFreeSlots); BufferPool.put(buffer); } @@ -587,22 +587,22 @@ public void testBufferPoolDisabled() BufferPool.DISABLED = true; BufferPool.ALLOCATE_ON_HEAP_WHEN_EXAHUSTED = true; ByteBuffer buffer = BufferPool.get(1024); - assertEquals(0, BufferPool.numChunks()); + assertEquals(0, BufferPool.unsafeNumChunks()); assertNotNull(buffer); assertEquals(1024, buffer.capacity()); assertFalse(buffer.isDirect()); assertNotNull(buffer.array()); BufferPool.put(buffer); - assertEquals(0, BufferPool.numChunks()); + assertEquals(0, BufferPool.unsafeNumChunks()); BufferPool.ALLOCATE_ON_HEAP_WHEN_EXAHUSTED = false; buffer = BufferPool.get(1024); - assertEquals(0, BufferPool.numChunks()); + assertEquals(0, BufferPool.unsafeNumChunks()); assertNotNull(buffer); assertEquals(1024, buffer.capacity()); assertTrue(buffer.isDirect()); BufferPool.put(buffer); - assertEquals(0, BufferPool.numChunks()); + assertEquals(0, BufferPool.unsafeNumChunks()); // clean-up BufferPool.DISABLED = false; @@ -794,10 +794,10 @@ private void doMultipleThreadsReleaseBuffers(final int threadCount, final int .. buffers[i] = BufferPool.get(sizes[i]); assertNotNull(buffers[i]); assertEquals(sizes[i], buffers[i].capacity()); - sum += BufferPool.currentChunk().roundUp(buffers[i].capacity()); + sum += BufferPool.unsafeCurrentChunk().roundUp(buffers[i].capacity()); } - final BufferPool.Chunk chunk = BufferPool.currentChunk(); + final BufferPool.Chunk chunk = BufferPool.unsafeCurrentChunk(); assertNotNull(chunk); assertFalse(chunk.isFree()); @@ -819,7 +819,7 @@ public void run() { try { - assertNotSame(chunk, BufferPool.currentChunk()); + assertNotSame(chunk, BufferPool.unsafeCurrentChunk()); BufferPool.put(buffer); } catch (AssertionError ex) @@ -849,7 +849,7 @@ public void run() System.gc(); System.gc(); - assertTrue(BufferPool.currentChunk().isFree()); + assertTrue(BufferPool.unsafeCurrentChunk().isFree()); //make sure the main thread can still allocate buffers ByteBuffer buffer = BufferPool.get(sizes[0]); diff --git a/test/unit/org/apache/cassandra/utils/vint/VIntCodingTest.java b/test/unit/org/apache/cassandra/utils/vint/VIntCodingTest.java index 2189be358f6c..c7c3324f7f50 100644 --- a/test/unit/org/apache/cassandra/utils/vint/VIntCodingTest.java +++ b/test/unit/org/apache/cassandra/utils/vint/VIntCodingTest.java @@ -22,10 +22,8 @@ import java.io.DataOutputStream; import java.io.IOException; -import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.net.async.ByteBufDataOutputPlus; import org.junit.Test; @@ -92,9 +90,11 @@ public void testOneByteCapacity() throws Exception { public void testByteBufWithNegativeNumber() throws IOException { int i = -1231238694; - ByteBuf buf = Unpooled.buffer(8); - VIntCoding.writeUnsignedVInt(i, new ByteBufDataOutputPlus(buf)); - long result = VIntCoding.readUnsignedVInt(buf); - Assert.assertEquals(i, result); + try (DataOutputBuffer out = new DataOutputBuffer()) + { + VIntCoding.writeUnsignedVInt(i, out); + long result = VIntCoding.getUnsignedVInt(out.buffer(), 0); + Assert.assertEquals(i, result); + } } } diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/SeedManager.java b/tools/stress/src/org/apache/cassandra/stress/generate/SeedManager.java index 4eeb47d59100..6874fba170c0 100644 --- a/tools/stress/src/org/apache/cassandra/stress/generate/SeedManager.java +++ b/tools/stress/src/org/apache/cassandra/stress/generate/SeedManager.java @@ -228,7 +228,6 @@ void finishWrite(Seed seed) private class LookbackReadGenerator extends Generator { - final Distribution lookback; public LookbackReadGenerator(Distribution lookback) diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java index 9b8eaa0efed5..4ea4cd2f4221 100644 --- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java +++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java @@ -43,23 +43,26 @@ public EncryptionOptions getEncryptionOptions() EncryptionOptions encOptions = new EncryptionOptions(); if (options.trustStore.present()) { - encOptions.enabled = true; - encOptions.truststore = options.trustStore.value(); - encOptions.truststore_password = options.trustStorePw.value(); + encOptions = encOptions + .withEnabled(true) + .withTrustStore(options.trustStore.value()) + .withTrustStorePassword(options.trustStorePw.value()) + .withAlgorithm(options.alg.value()) + .withProtocol(options.protocol.value()) + .withCipherSuites(options.ciphers.value().split(",")); if (options.keyStore.present()) { - encOptions.keystore = options.keyStore.value(); - encOptions.keystore_password = options.keyStorePw.value(); + encOptions = encOptions + .withKeyStore(options.keyStore.value()) + .withKeyStorePassword(options.keyStorePw.value()); } else { // mandatory for SSLFactory.createSSLContext(), see CASSANDRA-9325 - encOptions.keystore = encOptions.truststore; - encOptions.keystore_password = encOptions.truststore_password; + encOptions = encOptions + .withKeyStore(encOptions.truststore) + .withKeyStorePassword(encOptions.truststore_password); } - encOptions.algorithm = options.alg.value(); - encOptions.protocol = options.protocol.value(); - encOptions.cipher_suites = options.ciphers.value().split(","); } return encOptions; } diff --git a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java index fbcab4b95741..36361f7de2f7 100644 --- a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java +++ b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java @@ -148,7 +148,7 @@ public void connect(ProtocolOptions.Compression compression) throws Exception sslContext = SSLFactory.createSSLContext(encryptionOptions, true); SSLOptions sslOptions = JdkSSLOptions.builder() .withSSLContext(sslContext) - .withCipherSuites(encryptionOptions.cipher_suites).build(); + .withCipherSuites(encryptionOptions.cipher_suites.toArray(new String[0])).build(); clusterBuilder.withSSL(sslOptions); } From 0d48503c8c99d715164cdc59de646366408889ff Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Wed, 12 Jun 2019 15:02:37 +0100 Subject: [PATCH 08/78] Introduce a proxy test handler, extra unit tests for connection closure and message expirations patch by Alex Petrov; reviewed by Aleksey Yeschenko and Benedict Elliott Smith for CASSANDRA-15066 --- .../net/ProxyHandlerConnectionsTest.java | 405 ++++++++++++++++++ .../net/proxy/InboundProxyHandler.java | 234 ++++++++++ .../cassandra/net/proxy/ProxyHandlerTest.java | 222 ++++++++++ 3 files changed, 861 insertions(+) create mode 100644 test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java create mode 100644 test/unit/org/apache/cassandra/net/proxy/InboundProxyHandler.java create mode 100644 test/unit/org/apache/cassandra/net/proxy/ProxyHandlerTest.java diff --git a/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java b/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java new file mode 100644 index 000000000000..270a910720af --- /dev/null +++ b/test/unit/org/apache/cassandra/net/ProxyHandlerConnectionsTest.java @@ -0,0 +1,405 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import java.util.function.ToLongFunction; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.After; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import io.netty.buffer.ByteBuf; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.IVersionedAsymmetricSerializer; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.proxy.InboundProxyHandler; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.Pair; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.net.ConnectionTest.SETTINGS; +import static org.apache.cassandra.net.OutboundConnectionSettings.Framing.CRC; +import static org.apache.cassandra.utils.MonotonicClock.approxTime; + +public class ProxyHandlerConnectionsTest +{ + private static final SocketFactory factory = new SocketFactory(); + + private final Map>> serializers = new HashMap<>(); + private final Map>> handlers = new HashMap<>(); + private final Map> timeouts = new HashMap<>(); + + private void unsafeSetSerializer(Verb verb, Supplier> supplier) throws Throwable + { + serializers.putIfAbsent(verb, verb.unsafeSetSerializer(supplier)); + } + + protected void unsafeSetHandler(Verb verb, Supplier> supplier) throws Throwable + { + handlers.putIfAbsent(verb, verb.unsafeSetHandler(supplier)); + } + + private void unsafeSetExpiration(Verb verb, ToLongFunction expiration) throws Throwable + { + timeouts.putIfAbsent(verb, verb.unsafeSetExpiration(expiration)); + } + + @BeforeClass + public static void startup() + { + DatabaseDescriptor.daemonInitialization(); + } + + @After + public void cleanup() throws Throwable + { + for (Map.Entry>> e : serializers.entrySet()) + e.getKey().unsafeSetSerializer(e.getValue()); + serializers.clear(); + for (Map.Entry>> e : handlers.entrySet()) + e.getKey().unsafeSetHandler(e.getValue()); + handlers.clear(); + for (Map.Entry> e : timeouts.entrySet()) + e.getKey().unsafeSetExpiration(e.getValue()); + timeouts.clear(); + } + + @Test + public void testExpireInbound() throws Throwable + { + DatabaseDescriptor.setCrossNodeTimeout(true); + testOneManual((settings, inbound, outbound, endpoint, handler) -> { + unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new); + + CountDownLatch connectionLatch = new CountDownLatch(1); + unsafeSetHandler(Verb._TEST_1, () -> v -> { + connectionLatch.countDown(); + }); + outbound.enqueue(Message.out(Verb._TEST_1, 1L)); + connectionLatch.await(10, SECONDS); + Assert.assertEquals(0, connectionLatch.getCount()); + + // Slow things down + unsafeSetExpiration(Verb._TEST_1, unit -> unit.convert(50, MILLISECONDS)); + handler.withLatency(100, MILLISECONDS); + + unsafeSetHandler(Verb._TEST_1, () -> v -> { + throw new RuntimeException("Should have not been triggered " + v); + }); + int expireMessages = 10; + for (int i = 0; i < expireMessages; i++) + outbound.enqueue(Message.out(Verb._TEST_1, 1L)); + + InboundMessageHandlers handlers = MessagingService.instance().getInbound(endpoint); + waitForCondition(() -> handlers.expiredCount() == expireMessages); + Assert.assertEquals(expireMessages, handlers.expiredCount()); + }); + } + + @Test + public void testExpireSome() throws Throwable + { + DatabaseDescriptor.setCrossNodeTimeout(true); + testOneManual((settings, inbound, outbound, endpoint, handler) -> { + unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new); + connect(outbound); + + AtomicInteger counter = new AtomicInteger(); + unsafeSetHandler(Verb._TEST_1, () -> v -> { + counter.incrementAndGet(); + }); + + int expireMessages = 10; + for (int i = 0; i < expireMessages; i++) + outbound.enqueue(Message.out(Verb._TEST_1, 1L)); + waitForCondition(() -> counter.get() == 10); + + unsafeSetExpiration(Verb._TEST_1, unit -> unit.convert(50, MILLISECONDS)); + handler.withLatency(100, MILLISECONDS); + + InboundMessageHandlers handlers = MessagingService.instance().getInbound(endpoint); + for (int i = 0; i < expireMessages; i++) + outbound.enqueue(Message.out(Verb._TEST_1, 1L)); + waitForCondition(() -> handlers.expiredCount() == 10); + + handler.withLatency(2, MILLISECONDS); + + for (int i = 0; i < expireMessages; i++) + outbound.enqueue(Message.out(Verb._TEST_1, 1L)); + waitForCondition(() -> counter.get() == 20); + }); + } + + @Test + public void testExpireSomeFromBatch() throws Throwable + { + DatabaseDescriptor.setCrossNodeTimeout(true); + testManual((settings, inbound, outbound, endpoint, handler) -> { + unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new); + connect(outbound); + + Message msg = Message.out(Verb._TEST_1, 1L); + int messageSize = msg.serializedSize(MessagingService.current_version); + DatabaseDescriptor.setInternodeMaxMessageSizeInBytes(messageSize * 40); + + AtomicInteger counter = new AtomicInteger(); + unsafeSetHandler(Verb._TEST_1, () -> v -> { + counter.incrementAndGet(); + }); + + unsafeSetExpiration(Verb._TEST_1, unit -> unit.convert(200, MILLISECONDS)); + handler.withLatency(100, MILLISECONDS); + + int expireMessages = 20; + long nanoTime = approxTime.now(); + CountDownLatch enqueueDone = new CountDownLatch(1); + outbound.unsafeRunOnDelivery(() -> Uninterruptibles.awaitUninterruptibly(enqueueDone, 10, SECONDS)); + for (int i = 0; i < expireMessages; i++) + { + boolean expire = i % 2 == 0; + Message.Builder builder = Message.builder(Verb._TEST_1, 1L); + + if (settings.right.acceptVersions == ConnectionTest.legacy) + { + // backdate messages; leave 50 milliseconds to leave outbound path + builder.withCreatedAt(nanoTime - (expire ? 0 : MILLISECONDS.toNanos(150))); + } + else + { + // Give messages 50 milliseconds to leave outbound path + builder.withCreatedAt(nanoTime) + .withExpiresAt(nanoTime + (expire ? MILLISECONDS.toNanos(50) : MILLISECONDS.toNanos(1000))); + } + outbound.enqueue(builder.build()); + } + enqueueDone.countDown(); + + InboundMessageHandlers handlers = MessagingService.instance().getInbound(endpoint); + waitForCondition(() -> handlers.expiredCount() == 10 && counter.get() == 10, + () -> String.format("Expired: %d, Arrived: %d", handlers.expiredCount(), counter.get())); + }); + } + + @Test + public void suddenDisconnect() throws Throwable + { + testManual((settings, inbound, outbound, endpoint, handler) -> { + handler.onDisconnect(() -> handler.reset()); + + unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new); + connect(outbound); + + CountDownLatch closeLatch = new CountDownLatch(1); + handler.withCloseAfterRead(closeLatch::countDown); + AtomicInteger counter = new AtomicInteger(); + unsafeSetHandler(Verb._TEST_1, () -> v -> counter.incrementAndGet()); + + outbound.enqueue(Message.out(Verb._TEST_1, 1L)); + waitForCondition(() -> !outbound.isConnected()); + + connect(outbound); + Assert.assertTrue(outbound.isConnected()); + Assert.assertEquals(0, counter.get()); + }); + } + + @Test + public void testCorruptionOnHandshake() throws Throwable + { + testManual((settings, inbound, outbound, endpoint, handler) -> { + unsafeSetSerializer(Verb._TEST_1, FakePayloadSerializer::new); + // Invalid CRC + handler.withPayloadTransform(msg -> { + ByteBuf bb = (ByteBuf) msg; + bb.setByte(bb.readableBytes() / 2, 0xffff); + return msg; + }); + tryConnect(outbound, 1, SECONDS, false); + Assert.assertTrue(!outbound.isConnected()); + + // Invalid protocol magic + handler.withPayloadTransform(msg -> { + ByteBuf bb = (ByteBuf) msg; + bb.setByte(0, 0xffff); + return msg; + }); + tryConnect(outbound, 1, SECONDS, false); + Assert.assertTrue(!outbound.isConnected()); + if (settings.right.framing == CRC) + { + Assert.assertEquals(2, outbound.connectionAttempts()); + Assert.assertEquals(0, outbound.successfulConnections()); + } + }); + } + + private static void waitForCondition(Supplier cond) throws Throwable + { + CompletableFuture.runAsync(() -> { + while (!cond.get()) {} + }).get(10, SECONDS); + } + + private static void waitForCondition(Supplier cond, Supplier s) throws Throwable + { + try + { + CompletableFuture.runAsync(() -> { + while (!cond.get()) {} + }).get(10, SECONDS); + } + catch (TimeoutException e) + { + throw new AssertionError(s.get()); + } + } + + private static class FakePayloadSerializer implements IVersionedSerializer + { + private final int size; + private FakePayloadSerializer() + { + this(1); + } + + // Takes long and repeats it size times + private FakePayloadSerializer(int size) + { + this.size = size; + } + + public void serialize(Long i, DataOutputPlus out, int version) throws IOException + { + for (int j = 0; j < size; j++) + { + out.writeLong(i); + } + } + + public Long deserialize(DataInputPlus in, int version) throws IOException + { + long l = in.readLong(); + for (int i = 0; i < size - 1; i++) + { + if (in.readLong() != l) + throw new AssertionError(); + } + + return l; + } + + public long serializedSize(Long t, int version) + { + return Long.BYTES * size; + } + } + interface ManualSendTest + { + void accept(Pair settings, InboundSockets inbound, OutboundConnection outbound, InetAddressAndPort endpoint, InboundProxyHandler.Controller handler) throws Throwable; + } + + private void testManual(ManualSendTest test) throws Throwable + { + for (ConnectionTest.Settings s: SETTINGS) + { + doTestManual(s, test); + cleanup(); + } + } + + private void testOneManual(ManualSendTest test) throws Throwable + { + testOneManual(test, 1); + } + + private void testOneManual(ManualSendTest test, int i) throws Throwable + { + ConnectionTest.Settings s = SETTINGS.get(i); + doTestManual(s, test); + cleanup(); + } + + private void doTestManual(ConnectionTest.Settings settings, ManualSendTest test) throws Throwable + { + InetAddressAndPort endpoint = FBUtilities.getBroadcastAddressAndPort(); + + InboundConnectionSettings inboundSettings = settings.inbound.apply(new InboundConnectionSettings()) + .withBindAddress(endpoint) + .withSocketFactory(factory); + + InboundSockets inbound = new InboundSockets(Collections.singletonList(inboundSettings)); + + OutboundConnectionSettings outboundSettings = settings.outbound.apply(new OutboundConnectionSettings(endpoint)) + .withConnectTo(endpoint) + .withDefaultReserveLimits() + .withSocketFactory(factory); + + ResourceLimits.EndpointAndGlobal reserveCapacityInBytes = new ResourceLimits.EndpointAndGlobal(new ResourceLimits.Concurrent(outboundSettings.applicationSendQueueReserveEndpointCapacityInBytes), outboundSettings.applicationSendQueueReserveGlobalCapacityInBytes); + OutboundConnection outbound = new OutboundConnection(settings.type, outboundSettings, reserveCapacityInBytes); + try + { + InboundProxyHandler.Controller controller = new InboundProxyHandler.Controller(); + inbound.open(pipeline -> { + InboundProxyHandler handler = new InboundProxyHandler(controller); + pipeline.addLast(handler); + }).sync(); + test.accept(Pair.create(inboundSettings, outboundSettings), inbound, outbound, endpoint, controller); + } + finally + { + outbound.close(false); + inbound.close().get(30L, SECONDS); + outbound.close(false).get(30L, SECONDS); + MessagingService.instance().messageHandlers.clear(); + } + } + + private void connect(OutboundConnection outbound) throws Throwable + { + tryConnect(outbound, 10, SECONDS, true); + } + + private void tryConnect(OutboundConnection outbound, long timeout, TimeUnit timeUnit, boolean throwOnFailure) throws Throwable + { + CountDownLatch connectionLatch = new CountDownLatch(1); + unsafeSetHandler(Verb._TEST_1, () -> v -> { + connectionLatch.countDown(); + }); + outbound.enqueue(Message.out(Verb._TEST_1, 1L)); + connectionLatch.await(timeout, timeUnit); + if (throwOnFailure) + Assert.assertEquals(0, connectionLatch.getCount()); + } +} diff --git a/test/unit/org/apache/cassandra/net/proxy/InboundProxyHandler.java b/test/unit/org/apache/cassandra/net/proxy/InboundProxyHandler.java new file mode 100644 index 000000000000..7e3b004041e1 --- /dev/null +++ b/test/unit/org/apache/cassandra/net/proxy/InboundProxyHandler.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net.proxy; + +import java.util.ArrayDeque; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.util.concurrent.EventExecutor; +import io.netty.util.concurrent.ScheduledFuture; + +public class InboundProxyHandler extends ChannelInboundHandlerAdapter +{ + private final ArrayDeque forwardQueue; + private ScheduledFuture scheduled = null; + private final Controller controller; + public InboundProxyHandler(Controller controller) + { + this.controller = controller; + this.forwardQueue = new ArrayDeque<>(1024); + } + + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception + { + super.channelActive(ctx); + ctx.read(); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + controller.onDisconnect.run(); + + if (scheduled != null) + { + scheduled.cancel(true); + scheduled = null; + } + + if (!forwardQueue.isEmpty()) + forwardQueue.clear(); + + super.channelInactive(ctx); + } + + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) + { + Forward forward = controller.forwardStrategy.forward(ctx, msg); + forwardQueue.offer(forward); + maybeScheduleNext(ctx.channel().eventLoop()); + controller.onRead.run(); + ctx.channel().read(); + } + + private void maybeScheduleNext(EventExecutor executor) + { + if (forwardQueue.isEmpty()) + { + // Ran out of items to process + scheduled = null; + } + else if (scheduled == null) + { + // Schedule next available or let the last in line schedule it + Forward forward = forwardQueue.poll(); + scheduled = forward.schedule(executor); + scheduled.addListener((e) -> { + scheduled = null; + maybeScheduleNext(executor); + }); + } + } + + private static class Forward + { + final long arrivedAt; + final long latency; + final Runnable handler; + + private Forward(long arrivedAt, long latency, Runnable handler) + { + this.arrivedAt = arrivedAt; + this.latency = latency; + this.handler = handler; + } + + ScheduledFuture schedule(EventExecutor executor) + { + long now = System.currentTimeMillis(); + long elapsed = now - arrivedAt; + long runIn = latency - elapsed; + + if (runIn > 0) + return executor.schedule(handler, runIn, TimeUnit.MILLISECONDS); + else + return executor.schedule(handler, 0, TimeUnit.MILLISECONDS); + } + } + + private static class ForwardNormally implements ForwardStrategy + { + static ForwardNormally instance = new ForwardNormally(); + + public Forward forward(ChannelHandlerContext ctx, Object msg) + { + return new Forward(System.currentTimeMillis(), + 0, + () -> ctx.fireChannelRead(msg)); + } + } + + public interface ForwardStrategy + { + public Forward forward(ChannelHandlerContext ctx, Object msg); + } + + private static class ForwardWithLatency implements ForwardStrategy + { + private final long latency; + private final TimeUnit timeUnit; + + ForwardWithLatency(long latency, TimeUnit timeUnit) + { + this.latency = latency; + this.timeUnit = timeUnit; + } + + public Forward forward(ChannelHandlerContext ctx, Object msg) + { + return new Forward(System.currentTimeMillis(), + timeUnit.toMillis(latency), + () -> ctx.fireChannelRead(msg)); + } + } + + private static class CloseAfterRead implements ForwardStrategy + { + private final Runnable afterClose; + + CloseAfterRead(Runnable afterClose) + { + this.afterClose = afterClose; + } + + public Forward forward(ChannelHandlerContext ctx, Object msg) + { + return new Forward(System.currentTimeMillis(), + 0, + () -> { + ctx.channel().close().syncUninterruptibly(); + afterClose.run(); + }); + } + } + + private static class TransformPayload implements ForwardStrategy + { + private final Function fn; + + TransformPayload(Function fn) + { + this.fn = fn; + } + + public Forward forward(ChannelHandlerContext ctx, Object msg) + { + return new Forward(System.currentTimeMillis(), + 0, + () -> ctx.fireChannelRead(fn.apply((T) msg))); + } + } + + public static class Controller + { + private volatile InboundProxyHandler.ForwardStrategy forwardStrategy; + private volatile Runnable onRead = () -> {}; + private volatile Runnable onDisconnect = () -> {}; + + public Controller() + { + this.forwardStrategy = ForwardNormally.instance; + } + public void onRead(Runnable onRead) + { + this.onRead = onRead; + } + + public void onDisconnect(Runnable onDisconnect) + { + this.onDisconnect = onDisconnect; + } + + public void reset() + { + this.forwardStrategy = ForwardNormally.instance; + } + + public void withLatency(long latency, TimeUnit timeUnit) + { + this.forwardStrategy = new ForwardWithLatency(latency, timeUnit); + } + + public void withCloseAfterRead(Runnable afterClose) + { + this.forwardStrategy = new CloseAfterRead(afterClose); + } + + public void withPayloadTransform(Function fn) + { + this.forwardStrategy = new TransformPayload<>(fn); + } + } + +} diff --git a/test/unit/org/apache/cassandra/net/proxy/ProxyHandlerTest.java b/test/unit/org/apache/cassandra/net/proxy/ProxyHandlerTest.java new file mode 100644 index 000000000000..d070f5630fff --- /dev/null +++ b/test/unit/org/apache/cassandra/net/proxy/ProxyHandlerTest.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net.proxy; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; + +import org.junit.Assert; +import org.junit.Test; + +import io.netty.bootstrap.Bootstrap; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.local.LocalAddress; +import io.netty.channel.local.LocalChannel; +import io.netty.channel.local.LocalServerChannel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; + +public class ProxyHandlerTest +{ + private final Object PAYLOAD = new Object(); + + @Test + public void testLatency() throws Throwable + { + test((proxyHandler, testHandler, channel) -> { + int count = 1; + long latency = 100; + CountDownLatch latch = new CountDownLatch(count); + long start = System.currentTimeMillis(); + testHandler.onRead = new Consumer() + { + int last = -1; + public void accept(Object o) + { + // Make sure that order is preserved + Assert.assertEquals(last + 1, o); + last = (int) o; + + long elapsed = System.currentTimeMillis() - start; + Assert.assertTrue("Latency was:" + elapsed, elapsed > latency); + latch.countDown(); + } + }; + + proxyHandler.withLatency(latency, TimeUnit.MILLISECONDS); + + for (int i = 0; i < count; i++) + { + ByteBuf bb = Unpooled.buffer(Integer.BYTES); + bb.writeInt(i); + channel.writeAndFlush(i); + } + + Assert.assertTrue(latch.await(10, TimeUnit.SECONDS)); + }); + } + + @Test + public void testNormalDelivery() throws Throwable + { + test((proxyHandler, testHandler, channelPipeline) -> { + int count = 10; + CountDownLatch latch = new CountDownLatch(count); + AtomicLong end = new AtomicLong(); + testHandler.onRead = (o) -> { + end.set(System.currentTimeMillis()); + latch.countDown(); + }; + + for (int i = 0; i < count; i++) + channelPipeline.writeAndFlush(PAYLOAD); + Assert.assertTrue(latch.await(10, TimeUnit.SECONDS)); + + }); + } + + @Test + public void testLatencyForMany() throws Throwable + { + class Event { + private final long latency; + private final long start; + private final int idx; + + Event(long latency, int idx) + { + this.latency = latency; + this.start = System.currentTimeMillis(); + this.idx = idx; + } + } + + test((proxyHandler, testHandler, channel) -> { + int count = 150; + CountDownLatch latch = new CountDownLatch(count); + AtomicInteger counter = new AtomicInteger(); + testHandler.onRead = new Consumer() + { + int lastSeen = -1; + public void accept(Object o) + { + Event e = (Event) o; + Assert.assertEquals(lastSeen + 1, e.idx); + lastSeen = e.idx; + long elapsed = System.currentTimeMillis() - e.start; + Assert.assertTrue(elapsed >= e.latency); + counter.incrementAndGet(); + latch.countDown(); + } + }; + + int idx = 0; + for (int i = 0; i < count / 3; i++) + { + for (long latency : new long[]{ 100, 200, 0 }) + { + proxyHandler.withLatency(latency, TimeUnit.MILLISECONDS); + CountDownLatch read = new CountDownLatch(1); + proxyHandler.onRead(read::countDown); + channel.writeAndFlush(new Event(latency, idx++)); + Assert.assertTrue(read.await(10, TimeUnit.SECONDS)); + } + } + + Assert.assertTrue(latch.await(10, TimeUnit.SECONDS)); + Assert.assertEquals(counter.get(), count); + }); + } + + private interface DoTest + { + public void doTest(InboundProxyHandler.Controller proxy, TestHandler testHandler, Channel channel) throws Throwable; + } + + + public void test(DoTest test) throws Throwable + { + EventLoopGroup serverGroup = new NioEventLoopGroup(1); + EventLoopGroup clientGroup = new NioEventLoopGroup(1); + + InboundProxyHandler.Controller controller = new InboundProxyHandler.Controller(); + InboundProxyHandler proxyHandler = new InboundProxyHandler(controller); + TestHandler testHandler = new TestHandler(); + + ServerBootstrap sb = new ServerBootstrap(); + sb.group(serverGroup) + .channel(LocalServerChannel.class) + .childHandler(new ChannelInitializer() { + @Override + public void initChannel(LocalChannel ch) + { + ch.pipeline() + .addLast(proxyHandler) + .addLast(testHandler); + } + }) + .childOption(ChannelOption.AUTO_READ, false); + + Bootstrap cb = new Bootstrap(); + cb.group(clientGroup) + .channel(LocalChannel.class) + .handler(new ChannelInitializer() { + @Override + public void initChannel(LocalChannel ch) throws Exception { + ch.pipeline() + .addLast(new LoggingHandler(LogLevel.TRACE)); + } + }); + + final LocalAddress addr = new LocalAddress("test"); + + Channel serverChannel = sb.bind(addr).sync().channel(); + + Channel clientChannel = cb.connect(addr).sync().channel(); + test.doTest(controller, testHandler, clientChannel); + + clientChannel.close(); + serverChannel.close(); + serverGroup.shutdownGracefully(); + clientGroup.shutdownGracefully(); + } + + + public static class TestHandler extends ChannelInboundHandlerAdapter + { + private Consumer onRead = (o) -> {}; + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) + { + onRead.accept(msg); + } + } +} From 4b45ee591361446a50f54a7c97f0cc09e64fbf0d Mon Sep 17 00:00:00 2001 From: Jeremy Hanna Date: Wed, 12 Jun 2019 17:44:02 -0500 Subject: [PATCH 09/78] Update docs to point to Slack rather than IRC Patch by Jeremy Hanna, Reviewed by Michael Shuler for CASSANDRA-15150 --- doc/source/bugs.rst | 2 +- doc/source/contactus.rst | 2 +- doc/source/development/patches.rst | 2 +- doc/source/development/release_process.rst | 7 +++---- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/doc/source/bugs.rst b/doc/source/bugs.rst index bd58a8f171d3..196ea3a41d87 100644 --- a/doc/source/bugs.rst +++ b/doc/source/bugs.rst @@ -18,7 +18,7 @@ Reporting Bugs ============== If you encounter a problem with Cassandra, the first places to ask for help are the :ref:`user mailing list -` and the ``#cassandra`` :ref:`IRC channel `. +` and the ``Cassandra`` :ref:`Slack room `. If, after having asked for help, you suspect that you have found a bug in Cassandra, you should report it by opening a ticket through the `Apache Cassandra JIRA `__. Please provide as much diff --git a/doc/source/contactus.rst b/doc/source/contactus.rst index 8cecd901b0cf..308d872c53fd 100644 --- a/doc/source/contactus.rst +++ b/doc/source/contactus.rst @@ -17,7 +17,7 @@ Contact us ========== -You can get in touch with the Cassandra community either via the mailing lists or the freenode IRC channels. +You can get in touch with the Cassandra community either via the mailing lists or slack_ rooms. .. _mailing-lists: diff --git a/doc/source/development/patches.rst b/doc/source/development/patches.rst index e8e50f6a42ef..cc61c6cbca86 100644 --- a/doc/source/development/patches.rst +++ b/doc/source/development/patches.rst @@ -38,7 +38,7 @@ As a general rule of thumb: Before You Start Coding ======================= -Although contributions are highly appreciated, we do not guarantee that each contribution will become a part of Cassandra. Therefor it's generally a good idea to first get some feedback on the things you plan to work on, especially about any new features or major changes to the code base. You can reach out to other developers on the mailing list or IRC channel listed on our `community page `_. +Although contributions are highly appreciated, we do not guarantee that each contribution will become a part of Cassandra. Therefore it's generally a good idea to first get some feedback on the things you plan to work on, especially about any new features or major changes to the code base. You can reach out to other developers on the mailing list or :ref:`Slack `. You should also * Avoid redundant work by searching for already reported issues in `JIRA `_ diff --git a/doc/source/development/release_process.rst b/doc/source/development/release_process.rst index 23bd7abafd0a..464ecd646914 100644 --- a/doc/source/development/release_process.rst +++ b/doc/source/development/release_process.rst @@ -241,12 +241,11 @@ Fill out the following email template and send to both user and dev mailing list [2]: (NEWS.txt) https://git1-us-west.apache.org/repos/asf?p=cassandra.git;a=blob_plain;f=NEWS.txt;hb= [3]: https://issues.apache.org/jira/browse/CASSANDRA -Update IRC #cassandra topic +Update Slack Cassandra topic --------------------------- -Update #cassandra topic on irc:: - /msg chanserv op #cassandra - /topic #cassandra "cassandra.apache.org | Latest: 3.11.2 (https://goo.gl/M34ZbG) | Stable: 3.0.16 (https://goo.gl/B4Zumg) | Oldstable: 2.2.12 (https://goo.gl/Uf3GVw) | ask, don't ask to ask" +Update Cassandra topic in Slack:: + /topic cassandra.apache.org | Latest releases: 3.11.4, 3.0.18, 2.2.14, 2.1.21 | ask, don't ask to ask Tweet from @Cassandra --------------------- From c24102906d41eb2f06b737c3c3d89be9a151e9f7 Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Thu, 13 Jun 2019 11:47:22 -0500 Subject: [PATCH 10/78] ninja: Fix slack channel names in docs --- doc/source/bugs.rst | 2 +- doc/source/contactus.rst | 10 ++++++---- doc/source/development/release_process.rst | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/doc/source/bugs.rst b/doc/source/bugs.rst index 196ea3a41d87..32d676f9d3f4 100644 --- a/doc/source/bugs.rst +++ b/doc/source/bugs.rst @@ -18,7 +18,7 @@ Reporting Bugs ============== If you encounter a problem with Cassandra, the first places to ask for help are the :ref:`user mailing list -` and the ``Cassandra`` :ref:`Slack room `. +` and the ``cassandra`` :ref:`Slack room `. If, after having asked for help, you suspect that you have found a bug in Cassandra, you should report it by opening a ticket through the `Apache Cassandra JIRA `__. Please provide as much diff --git a/doc/source/contactus.rst b/doc/source/contactus.rst index 308d872c53fd..3ed9004ddcfc 100644 --- a/doc/source/contactus.rst +++ b/doc/source/contactus.rst @@ -17,7 +17,7 @@ Contact us ========== -You can get in touch with the Cassandra community either via the mailing lists or slack_ rooms. +You can get in touch with the Cassandra community either via the mailing lists or :ref:`Slack rooms `. .. _mailing-lists: @@ -40,9 +40,11 @@ email to confirm your subscription. Make sure to keep the welcome email as it co unsubscribe. .. _slack: + Slack -_____ +----- To chat with developers or users in real-time, join our rooms on `ASF Slack `__: -- ``#cassandra`` - for user questions and general discussions. -- ``#cassandra-dev`` - strictly for questions or discussions related to Cassandra development. + +- ``cassandra`` - for user questions and general discussions. +- ``cassandra-dev`` - strictly for questions or discussions related to Cassandra development. diff --git a/doc/source/development/release_process.rst b/doc/source/development/release_process.rst index 464ecd646914..b3c403215fd7 100644 --- a/doc/source/development/release_process.rst +++ b/doc/source/development/release_process.rst @@ -244,7 +244,7 @@ Fill out the following email template and send to both user and dev mailing list Update Slack Cassandra topic --------------------------- -Update Cassandra topic in Slack:: +Update topic in ``cassandra`` :ref:`Slack room ` /topic cassandra.apache.org | Latest releases: 3.11.4, 3.0.18, 2.2.14, 2.1.21 | ask, don't ask to ask Tweet from @Cassandra From 7c0cb6db32ecbde5534087ce53364bd6cf1ddcec Mon Sep 17 00:00:00 2001 From: Jeremy Hanna Date: Thu, 13 Jun 2019 13:54:24 -0500 Subject: [PATCH 11/78] Change low hanging fruit JIRA link/text in docs Patch by Jeremy Hanna, Reviewed by Michael Shuler for CASSANDRA-15149 --- doc/source/development/patches.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/development/patches.rst b/doc/source/development/patches.rst index cc61c6cbca86..43bdb8c68091 100644 --- a/doc/source/development/patches.rst +++ b/doc/source/development/patches.rst @@ -33,7 +33,7 @@ As a general rule of thumb: .. hint:: - Not sure what to work? Just pick an issue tagged with the `low hanging fruit label `_ in JIRA, which we use to flag issues that could turn out to be good starter tasks for beginners. + Not sure what to work? Just pick an issue marked as `Low Hanging Fruit `_ Complexity in JIRA, which we use to flag issues that could turn out to be good starter tasks for beginners. Before You Start Coding ======================= From abb0e17785b50baee6e53ee02bd367a5ce9455f8 Mon Sep 17 00:00:00 2001 From: nvharikrishna Date: Mon, 17 Jun 2019 15:18:03 +0530 Subject: [PATCH 12/78] Fix leak in FramingTest unit test case patch by Hari Nukala; reviewed by Aleksey Yeschenko for CASSANDRA-15165 --- .../apache/cassandra/net/ShareableBytes.java | 4 ++-- .../org/apache/cassandra/net/FramingTest.java | 20 ++++++++++--------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/java/org/apache/cassandra/net/ShareableBytes.java b/src/java/org/apache/cassandra/net/ShareableBytes.java index feea25e34394..e4f24608e4d2 100644 --- a/src/java/org/apache/cassandra/net/ShareableBytes.java +++ b/src/java/org/apache/cassandra/net/ShareableBytes.java @@ -40,14 +40,14 @@ class ShareableBytes private static final AtomicIntegerFieldUpdater countUpdater = AtomicIntegerFieldUpdater.newUpdater(ShareableBytes.class, "count"); - ShareableBytes(ByteBuffer bytes) + private ShareableBytes(ByteBuffer bytes) { this.count = UNSHARED; this.owner = this; this.bytes = bytes; } - ShareableBytes(ShareableBytes owner, ByteBuffer bytes) + private ShareableBytes(ShareableBytes owner, ByteBuffer bytes) { this.owner = owner; this.bytes = bytes; diff --git a/test/unit/org/apache/cassandra/net/FramingTest.java b/test/unit/org/apache/cassandra/net/FramingTest.java index ec6cd84e994d..8a7f4283bdfa 100644 --- a/test/unit/org/apache/cassandra/net/FramingTest.java +++ b/test/unit/org/apache/cassandra/net/FramingTest.java @@ -118,7 +118,7 @@ public void testRandomCrc() testSomeFrames(FrameEncoderCrc.instance, FrameDecoderCrc.create(GlobalBufferPoolAllocator.instance)); } - public void testSomeFrames(FrameEncoder encoder, FrameDecoder decoder) + private void testSomeFrames(FrameEncoder encoder, FrameDecoder decoder) { long seed = new SecureRandom().nextLong(); logger.info("seed: {}, decoder: {}", seed, decoder.getClass().getSimpleName()); @@ -156,7 +156,7 @@ private void testRandomSequenceOfFrames(Random random, FrameEncoder encoder, Fra for (FrameDecoder.Frame frame : out) frame.release(); frames.release(); - Assert.assertEquals(null, decoder.stash); + Assert.assertNull(decoder.stash); Assert.assertTrue(decoder.frames.isEmpty()); } @@ -213,7 +213,7 @@ public void burnRandomLegacy() burnRandomLegacy(1000); } - public void burnRandomLegacy(int count) + private void burnRandomLegacy(int count) { SecureRandom seed = new SecureRandom(); Random random = new Random(); @@ -234,7 +234,7 @@ public void testRandomLegacy() testRandomLegacy(250); } - public void testRandomLegacy(int count) + private void testRandomLegacy(int count) { SecureRandom seeds = new SecureRandom(); for (int messagingVersion : new int[] { VERSION_30, VERSION_3014, current_version}) @@ -247,7 +247,7 @@ public void testRandomLegacy(int count) } } - public void testSomeMessages(long seed, int count, float largeRatio, int messagingVersion, FrameDecoder decoder) + private void testSomeMessages(long seed, int count, float largeRatio, int messagingVersion, FrameDecoder decoder) { logger.info("seed: {}, iterations: {}, largeRatio: {}, messagingVersion: {}, decoder: {}", seed, count, largeRatio, messagingVersion, decoder.getClass().getSimpleName()); Random random = new Random(seed); @@ -291,7 +291,7 @@ else if (!lengthIsReadable(message, i - messageStart, messagingVersion)) // we should have an initial frame containing only some prefix of the message (probably 64 bytes) // that was stashed only to decide how big the message was FrameDecoder.IntactFrame frame = (FrameDecoder.IntactFrame) out.get(outIndex++); - Assert.assertEquals(false, frame.isSelfContained); + Assert.assertFalse(frame.isSelfContained); start = frame.contents.remaining(); verify(message, 0, frame.contents.remaining(), frame.contents); } @@ -328,7 +328,6 @@ else if (message.length > LARGE_MESSAGE_THRESHOLD) } // message is fresh - int beginFrame = messageStart; int beginFrameIndex = messageIndex; while (messageStart + message.length <= limit) { @@ -344,7 +343,9 @@ else if (message.length > LARGE_MESSAGE_THRESHOLD) while (beginFrameIndex < messageIndex) { byte[] m = messages.get(beginFrameIndex); - verify(m, frame.contents.sliceAndConsume(m.length)); + ShareableBytes bytesToVerify = frame.contents.sliceAndConsume(m.length); + verify(m, bytesToVerify); + bytesToVerify.release(); ++beginFrameIndex; } Assert.assertFalse(frame.contents.hasRemaining()); @@ -367,7 +368,8 @@ && lengthIsReadable(message, limit - messageStart, messagingVersion)) i = limit; } stream.release(); - Assert.assertEquals(null, decoder.stash); + Assert.assertTrue(stream.isReleased()); + Assert.assertNull(decoder.stash); Assert.assertTrue(decoder.frames.isEmpty()); } From 58456fc15d047810f0010e64ca496fd3223376ab Mon Sep 17 00:00:00 2001 From: Jeff Jirsa Date: Mon, 17 Jun 2019 11:00:25 -0700 Subject: [PATCH 13/78] CQL Native Transport: Apply noSpamLogger to ConnectionLimitHandler Patch by Jeff Jirsa; Reviewed by Chris Lohfink for CASSANDRA-15167 --- CHANGES.txt | 1 + .../cassandra/transport/ConnectionLimitHandler.java | 9 +++++++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 7680b8d841cc..6435d033b081 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Native Transport - Apply noSpamLogger to ConnectionLimitHandler (CASSANDRA-15167) * Reduce heap pressure during compactions (CASSANDRA-14654) * Support building Cassandra with JDK 11 (CASSANDRA-15108) * Use quilt to patch cassandra.in.sh in Debian packaging (CASSANDRA-14710) diff --git a/src/java/org/apache/cassandra/transport/ConnectionLimitHandler.java b/src/java/org/apache/cassandra/transport/ConnectionLimitHandler.java index 7bcf280cd04c..3b2765f1e02e 100644 --- a/src/java/org/apache/cassandra/transport/ConnectionLimitHandler.java +++ b/src/java/org/apache/cassandra/transport/ConnectionLimitHandler.java @@ -22,6 +22,8 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.utils.NoSpamLogger; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,6 +31,7 @@ import java.net.InetSocketAddress; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -40,6 +43,8 @@ final class ConnectionLimitHandler extends ChannelInboundHandlerAdapter { private static final Logger logger = LoggerFactory.getLogger(ConnectionLimitHandler.class); + private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.MINUTES); + private final ConcurrentMap connectionsPerClient = new ConcurrentHashMap<>(); private final AtomicLong counter = new AtomicLong(0); @@ -56,7 +61,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception if (count > limit) { // The decrement will be done in channelClosed(...) - logger.warn("Exceeded maximum native connection limit of {} by using {} connections", limit, count); + noSpamLogger.warn("Exceeded maximum native connection limit of {} by using {} connections", limit, count); ctx.close(); } else @@ -80,7 +85,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception if (perIpCount.incrementAndGet() > perIpLimit) { // The decrement will be done in channelClosed(...) - logger.warn("Exceeded maximum native connection limit per ip of {} by using {} connections", perIpLimit, perIpCount); + noSpamLogger.warn("Exceeded maximum native connection limit per ip of {} by using {} connections", perIpLimit, perIpCount); ctx.close(); return; } From 802989dce75847f87a5a47e2fe5d590d02f086fa Mon Sep 17 00:00:00 2001 From: Jackels Date: Wed, 19 Jun 2019 21:53:09 -0500 Subject: [PATCH 14/78] Repair: fix log for inconsistent-left-subrange log statement Closes #327 Patch by Nathan Jackles; Reviewed by Jeff Jirsa for CASSANDRA-15173 --- src/java/org/apache/cassandra/utils/MerkleTree.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/utils/MerkleTree.java b/src/java/org/apache/cassandra/utils/MerkleTree.java index 1d51f03b11a8..d131ff58f539 100644 --- a/src/java/org/apache/cassandra/utils/MerkleTree.java +++ b/src/java/org/apache/cassandra/utils/MerkleTree.java @@ -313,7 +313,7 @@ static int differenceHelper(MerkleTree ltree, MerkleTree rtree, List } else if (!lreso) { - logger.debug("({}) Left sub-range fully inconsistent {}", active.depth, right); + logger.debug("({}) Left sub-range fully inconsistent {}", active.depth, left); ldiff = FULLY_INCONSISTENT; } From 225fa868884bdda1c20e0fcef61628eb6d941fbe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Per=20Otterstr=C3=B6m?= Date: Fri, 21 Jun 2019 17:41:01 -0700 Subject: [PATCH 15/78] Fix flaky unit test AuditLoggerTest MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Patch by Per Otterström, Sumanth Pasupuleti; Reviewed by Vinay Chella for CASSANDRA-15105 Co-Authored-By: Per Otterström Co-Authored-By: Sumanth Pasupuleti --- .../cassandra/audit/AuditLoggerTest.java | 23 +++++++++++-------- .../service/StorageServiceServerTest.java | 15 ++++++++++-- 2 files changed, 27 insertions(+), 11 deletions(-) diff --git a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java index b0299dc58be2..a44554729e65 100644 --- a/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java +++ b/test/unit/org/apache/cassandra/audit/AuditLoggerTest.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.audit; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -63,6 +64,12 @@ public void beforeTestMethod() enableAuditLogOptions(options); } + @After + public void afterTestMethod() + { + disableAuditLogOptions(); + } + private void enableAuditLogOptions(AuditLogOptions options) { String loggerName = "InMemoryAuditLogger"; @@ -89,7 +96,7 @@ public void testAuditLogFilters() throws Throwable execute("INSERT INTO %s (id, v1, v2) VALUES (?, ?, ?)", 2, "trace", "test"); AuditLogOptions options = new AuditLogOptions(); - options.excluded_keyspaces = KEYSPACE; + options.excluded_keyspaces += ',' + KEYSPACE; enableAuditLogOptions(options); String cql = "SELECT id, v1, v2 FROM " + KEYSPACE + '.' + currentTable() + " WHERE id = ?"; @@ -106,7 +113,7 @@ public void testAuditLogFilters() throws Throwable options = new AuditLogOptions(); options.included_keyspaces = KEYSPACE; - options.excluded_keyspaces = KEYSPACE; + options.excluded_keyspaces += ',' + KEYSPACE; enableAuditLogOptions(options); cql = "SELECT id, v1, v2 FROM " + KEYSPACE + '.' + currentTable() + " WHERE id = ?"; @@ -129,7 +136,7 @@ public void testAuditLogFiltersTransitions() throws Throwable execute("INSERT INTO %s (id, v1, v2) VALUES (?, ?, ?)", 2, "trace", "test"); AuditLogOptions options = new AuditLogOptions(); - options.excluded_keyspaces = KEYSPACE; + options.excluded_keyspaces += ',' + KEYSPACE; enableAuditLogOptions(options); String cql = "SELECT id, v1, v2 FROM " + KEYSPACE + '.' + currentTable() + " WHERE id = ?"; @@ -144,7 +151,7 @@ public void testAuditLogFiltersTransitions() throws Throwable options = new AuditLogOptions(); options.included_keyspaces = KEYSPACE; - options.excluded_keyspaces = KEYSPACE; + options.excluded_keyspaces += ',' + KEYSPACE; enableAuditLogOptions(options); cql = "SELECT id, v1, v2 FROM " + KEYSPACE + '.' + currentTable() + " WHERE id = ?"; @@ -162,11 +169,9 @@ public void testAuditLogFiltersTransitions() throws Throwable public void testAuditLogExceptions() { AuditLogOptions options = new AuditLogOptions(); - options.excluded_keyspaces = KEYSPACE; + options.excluded_keyspaces += ',' + KEYSPACE; enableAuditLogOptions(options); Assert.assertTrue(AuditLogManager.getInstance().isAuditingEnabled()); - - disableAuditLogOptions(); } @Test @@ -602,7 +607,7 @@ public void testIncludeSystemKeyspaces() throws Throwable { AuditLogOptions options = new AuditLogOptions(); options.included_categories = "QUERY,DML,PREPARE"; - options.excluded_keyspaces = "system_schema"; + options.excluded_keyspaces = "system_schema,system_virtual_schema"; enableAuditLogOptions(options); Session session = sessionNet(); @@ -620,7 +625,7 @@ public void testExcludeSystemKeyspaces() throws Throwable { AuditLogOptions options = new AuditLogOptions(); options.included_categories = "QUERY,DML,PREPARE"; - options.excluded_keyspaces = "system"; + options.excluded_keyspaces = "system,system_schema,system_virtual_schema"; enableAuditLogOptions(options); Session session = sessionNet(); diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java index 2db221bff8c0..565d91a7f9e3 100644 --- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java +++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java @@ -28,6 +28,7 @@ import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -621,12 +622,20 @@ public void testGetNativeAddress() throws Exception assertEquals("127.0.0.3:666", StorageService.instance.getNativeaddress(internalAddress, true)); } - @Test(expected = IllegalStateException.class) + @Test public void testAuditLogEnableLoggerNotFound() throws Exception { StorageService.instance.enableAuditLog(null, null, null, null, null, null, null); assertTrue(AuditLogManager.getInstance().isAuditingEnabled()); - StorageService.instance.enableAuditLog("foobar", null, null, null, null, null, null); + try + { + StorageService.instance.enableAuditLog("foobar", null, null, null, null, null, null); + Assert.fail(); + } + catch (IllegalStateException ex) + { + StorageService.instance.disableAuditLog(); + } } @Test @@ -646,5 +655,7 @@ public void testAuditLogEnableLoggerTransitions() throws Exception StorageService.instance.enableAuditLog(null, null, null, null, null, null, null); assertTrue(AuditLogManager.getInstance().isAuditingEnabled()); + + StorageService.instance.disableAuditLog(); } } From dad82fbd30c8b87c4c9fa02abc6796ba8c2bf99a Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Wed, 19 Jun 2019 16:31:35 +0100 Subject: [PATCH 16/78] Handle paging states serialized with a different version than the session's patch by Aleksey Yeschenko; reviewed for CASSANDRA-15176 by Sam Tunnicliffe an Blake Eggleston --- CHANGES.txt | 1 + .../cassandra/io/util/DataOutputBuffer.java | 13 + .../cassandra/service/pager/PagingState.java | 285 +++++++++++++----- .../cassandra/utils/vint/VIntCoding.java | 41 +++ .../cassandra/service/QueryPagerTest.java | 35 ++- .../service/pager/PagingStateTest.java | 72 ++++- 6 files changed, 347 insertions(+), 100 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 652833ee76a1..eae281518fb9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Handle paging states serialized with a different version than the session's (CASSANDRA-15176) * Throw IOE instead of asserting on unsupporter peer versions (CASSANDRA-15066) * Update token metadata when handling MOVING/REMOVING_TOKEN events (CASSANDRA-15120) * Add ability to customize cassandra log directory using $CASSANDRA_LOG_DIR (CASSANDRA-15090) diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java index 7586543f3abc..195fdb4e91bb 100644 --- a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java +++ b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java @@ -163,6 +163,19 @@ public void close() public ByteBuffer buffer() { + return buffer(true); + } + + public ByteBuffer buffer(boolean duplicate) + { + if (!duplicate) + { + ByteBuffer buf = buffer; + buf.flip(); + buffer = null; + return buf; + } + ByteBuffer result = buffer.duplicate(); result.flip(); return result; diff --git a/src/java/org/apache/cassandra/service/pager/PagingState.java b/src/java/org/apache/cassandra/service/pager/PagingState.java index 30e14c3c5040..7de7e6f69c79 100644 --- a/src/java/org/apache/cassandra/service/pager/PagingState.java +++ b/src/java/org/apache/cassandra/service/pager/PagingState.java @@ -21,10 +21,12 @@ import java.nio.ByteBuffer; import java.util.*; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; + import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.LegacyLayout; -import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.BytesType; import org.apache.cassandra.db.rows.Cell; @@ -33,10 +35,17 @@ import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.DataOutputBufferFixed; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.transport.Server; import org.apache.cassandra.transport.ProtocolException; -import org.apache.cassandra.utils.ByteBufferUtil; +import static org.apache.cassandra.db.TypeSizes.sizeof; +import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt; +import static org.apache.cassandra.transport.Server.VERSION_3; +import static org.apache.cassandra.transport.Server.VERSION_4; +import static org.apache.cassandra.utils.ByteBufferUtil.*; +import static org.apache.cassandra.utils.vint.VIntCoding.computeUnsignedVIntSize; +import static org.apache.cassandra.utils.vint.VIntCoding.getUnsignedVInt; + +@SuppressWarnings("WeakerAccess") public class PagingState { public final ByteBuffer partitionKey; // Can be null for single partition queries. @@ -52,92 +61,228 @@ public PagingState(ByteBuffer partitionKey, RowMark rowMark, int remaining, int this.remainingInPartition = remainingInPartition; } - public static PagingState deserialize(ByteBuffer bytes, int protocolVersion) + public ByteBuffer serialize(int protocolVersion) { - if (bytes == null) - return null; - - try (DataInputBuffer in = new DataInputBuffer(bytes, true)) + assert rowMark == null || protocolVersion == rowMark.protocolVersion; + try { - ByteBuffer pk; - RowMark mark; - int remaining, remainingInPartition; - if (protocolVersion <= Server.VERSION_3) - { - pk = ByteBufferUtil.readWithShortLength(in); - mark = new RowMark(ByteBufferUtil.readWithShortLength(in), protocolVersion); - remaining = in.readInt(); - // Note that while 'in.available()' is theoretically an estimate of how many bytes are available - // without blocking, we know that since we're reading a ByteBuffer it will be exactly how many - // bytes remain to be read. And the reason we want to condition this is for backward compatility - // as we used to not set this. - remainingInPartition = in.available() > 0 ? in.readInt() : Integer.MAX_VALUE; - } - else - { - pk = ByteBufferUtil.readWithVIntLength(in); - mark = new RowMark(ByteBufferUtil.readWithVIntLength(in), protocolVersion); - remaining = (int)in.readUnsignedVInt(); - remainingInPartition = (int)in.readUnsignedVInt(); - } - return new PagingState(pk.hasRemaining() ? pk : null, - mark.mark.hasRemaining() ? mark : null, - remaining, - remainingInPartition); + return protocolVersion > VERSION_3 ? modernSerialize() : legacySerialize(true); } catch (IOException e) { - throw new ProtocolException("Invalid value for the paging state"); + throw new RuntimeException(e); } } - public ByteBuffer serialize(int protocolVersion) + public int serializedSize(int protocolVersion) { assert rowMark == null || protocolVersion == rowMark.protocolVersion; - try (DataOutputBuffer out = new DataOutputBufferFixed(serializedSize(protocolVersion))) + + return protocolVersion > VERSION_3 ? modernSerializedSize() : legacySerializedSize(true); + } + + /** + * It's possible to receive a V3 paging state on a V4 client session, and vice versa - so we cannot + * blindly rely on the protocol version provided. We must verify first that the buffer indeed contains + * a paging state that adheres to the protocol version provided, or, if not - see if it is in a different + * version, in which case we try the other format. + */ + public static PagingState deserialize(ByteBuffer bytes, int protocolVersion) + { + if (bytes == null) + return null; + + try { - ByteBuffer pk = partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : partitionKey; - ByteBuffer mark = rowMark == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : rowMark.mark; - if (protocolVersion <= Server.VERSION_3) + /* + * We can't just attempt to deser twice, as we risk to misinterpet short/vint + * lengths and allocate huge byte arrays for readWithVIntLength() or, + * to a lesser extent, readWithShortLength() + */ + + if (protocolVersion > VERSION_3) { - ByteBufferUtil.writeWithShortLength(pk, out); - ByteBufferUtil.writeWithShortLength(mark, out); - out.writeInt(remaining); - out.writeInt(remainingInPartition); + if (isModernSerialized(bytes)) return modernDeserialize(bytes, protocolVersion); + if (isLegacySerialized(bytes)) return legacyDeserialize(bytes, VERSION_3); } - else + + if (protocolVersion < VERSION_4) { - ByteBufferUtil.writeWithVIntLength(pk, out); - ByteBufferUtil.writeWithVIntLength(mark, out); - out.writeUnsignedVInt(remaining); - out.writeUnsignedVInt(remainingInPartition); + if (isLegacySerialized(bytes)) return legacyDeserialize(bytes, protocolVersion); + if (isModernSerialized(bytes)) return modernDeserialize(bytes, VERSION_4); } - return out.buffer(); } catch (IOException e) { - throw new RuntimeException(e); + throw new ProtocolException("Invalid value for the paging state"); } + + throw new ProtocolException("Invalid value for the paging state"); } - public int serializedSize(int protocolVersion) + /* + * Modern serde (> VERSION_3) + */ + + @SuppressWarnings({ "resource", "RedundantSuppression" }) + private ByteBuffer modernSerialize() throws IOException { - assert rowMark == null || protocolVersion == rowMark.protocolVersion; - ByteBuffer pk = partitionKey == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : partitionKey; - ByteBuffer mark = rowMark == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : rowMark.mark; - if (protocolVersion <= Server.VERSION_3) - { - return ByteBufferUtil.serializedSizeWithShortLength(pk) - + ByteBufferUtil.serializedSizeWithShortLength(mark) - + 8; // remaining & remainingInPartition - } - else + DataOutputBuffer out = new DataOutputBufferFixed(modernSerializedSize()); + writeWithVIntLength(null == partitionKey ? EMPTY_BYTE_BUFFER : partitionKey, out); + writeWithVIntLength(null == rowMark ? EMPTY_BYTE_BUFFER : rowMark.mark, out); + out.writeUnsignedVInt(remaining); + out.writeUnsignedVInt(remainingInPartition); + return out.buffer(false); + } + + private static boolean isModernSerialized(ByteBuffer bytes) + { + int index = bytes.position(); + int limit = bytes.limit(); + + long partitionKeyLen = getUnsignedVInt(bytes, index, limit); + if (partitionKeyLen < 0) + return false; + index += computeUnsignedVIntSize(partitionKeyLen) + partitionKeyLen; + if (index >= limit) + return false; + + long rowMarkerLen = getUnsignedVInt(bytes, index, limit); + if (rowMarkerLen < 0) + return false; + index += computeUnsignedVIntSize(rowMarkerLen) + rowMarkerLen; + if (index >= limit) + return false; + + long remaining = getUnsignedVInt(bytes, index, limit); + if (remaining < 0) + return false; + index += computeUnsignedVIntSize(remaining); + if (index >= limit) + return false; + + long remainingInPartition = getUnsignedVInt(bytes, index, limit); + if (remainingInPartition < 0) + return false; + index += computeUnsignedVIntSize(remainingInPartition); + return index == limit; + } + + @SuppressWarnings({ "resource", "RedundantSuppression" }) + private static PagingState modernDeserialize(ByteBuffer bytes, int protocolVersion) throws IOException + { + if (protocolVersion < VERSION_4) + throw new IllegalArgumentException(); + + DataInputBuffer in = new DataInputBuffer(bytes, false); + + ByteBuffer partitionKey = readWithVIntLength(in); + ByteBuffer rawMark = readWithVIntLength(in); + int remaining = Ints.checkedCast(in.readUnsignedVInt()); + int remainingInPartition = Ints.checkedCast(in.readUnsignedVInt()); + + return new PagingState(partitionKey.hasRemaining() ? partitionKey : null, + rawMark.hasRemaining() ? new RowMark(rawMark, protocolVersion) : null, + remaining, + remainingInPartition); + } + + private int modernSerializedSize() + { + return serializedSizeWithVIntLength(null == partitionKey ? EMPTY_BYTE_BUFFER : partitionKey) + + serializedSizeWithVIntLength(null == rowMark ? EMPTY_BYTE_BUFFER : rowMark.mark) + + sizeofUnsignedVInt(remaining) + + sizeofUnsignedVInt(remainingInPartition); + } + + /* + * Legacy serde (< VERSION_4) + * + * There are two versions of legacy PagingState format - one used by 2.1/2.2 and one used by 3.0+. + * The latter includes remainingInPartition count, while the former doesn't. + */ + + @VisibleForTesting + @SuppressWarnings({ "resource", "RedundantSuppression" }) + ByteBuffer legacySerialize(boolean withRemainingInPartition) throws IOException + { + DataOutputBuffer out = new DataOutputBufferFixed(legacySerializedSize(withRemainingInPartition)); + writeWithShortLength(null == partitionKey ? EMPTY_BYTE_BUFFER : partitionKey, out); + writeWithShortLength(null == rowMark ? EMPTY_BYTE_BUFFER : rowMark.mark, out); + out.writeInt(remaining); + if (withRemainingInPartition) + out.writeInt(remainingInPartition); + return out.buffer(false); + } + + private static boolean isLegacySerialized(ByteBuffer bytes) + { + int index = bytes.position(); + int limit = bytes.limit(); + + if (limit - index < 2) + return false; + short partitionKeyLen = bytes.getShort(index); + if (partitionKeyLen < 0) + return false; + index += 2 + partitionKeyLen; + + if (limit - index < 2) + return false; + short rowMarkerLen = bytes.getShort(index); + if (rowMarkerLen < 0) + return false; + index += 2 + rowMarkerLen; + + if (limit - index < 4) + return false; + int remaining = bytes.getInt(index); + if (remaining < 0) + return false; + index += 4; + + // V3 encoded by 2.1/2.2 - sans remainingInPartition + if (index == limit) + return true; + + if (limit - index == 4) { - return ByteBufferUtil.serializedSizeWithVIntLength(pk) - + ByteBufferUtil.serializedSizeWithVIntLength(mark) - + TypeSizes.sizeofUnsignedVInt(remaining) - + TypeSizes.sizeofUnsignedVInt(remainingInPartition); + int remainingInPartition = bytes.getInt(index); + return remainingInPartition >= 0; // the value must make sense } + return false; + } + + @SuppressWarnings({ "resource", "RedundantSuppression" }) + private static PagingState legacyDeserialize(ByteBuffer bytes, int protocolVersion) throws IOException + { + if (protocolVersion > VERSION_3) + throw new IllegalArgumentException(); + + DataInputBuffer in = new DataInputBuffer(bytes, false); + + ByteBuffer partitionKey = readWithShortLength(in); + ByteBuffer rawMark = readWithShortLength(in); + int remaining = in.readInt(); + /* + * 2.1/2.2 implementations of V3 protocol did not write remainingInPartition, but C* 3.0+ does, so we need + * to handle both variants of V3 serialization for compatibility. + */ + int remainingInPartition = in.available() > 0 ? in.readInt() : Integer.MAX_VALUE; + + return new PagingState(partitionKey.hasRemaining() ? partitionKey : null, + rawMark.hasRemaining() ? new RowMark(rawMark, protocolVersion) : null, + remaining, + remainingInPartition); + } + + @VisibleForTesting + int legacySerializedSize(boolean withRemainingInPartition) + { + return serializedSizeWithShortLength(null == partitionKey ? EMPTY_BYTE_BUFFER : partitionKey) + + serializedSizeWithShortLength(null == rowMark ? EMPTY_BYTE_BUFFER : rowMark.mark) + + sizeof(remaining) + + (withRemainingInPartition ? sizeof(remainingInPartition) : 0); } @Override @@ -162,7 +307,7 @@ public final boolean equals(Object o) public String toString() { return String.format("PagingState(key=%s, cellname=%s, remaining=%d, remainingInPartition=%d", - partitionKey != null ? ByteBufferUtil.bytesToHex(partitionKey) : null, + partitionKey != null ? bytesToHex(partitionKey) : null, rowMark, remaining, remainingInPartition); @@ -205,7 +350,7 @@ private static List> makeClusteringTypes(CFMetaData metadata) public static RowMark create(CFMetaData metadata, Row row, int protocolVersion) { ByteBuffer mark; - if (protocolVersion <= Server.VERSION_3) + if (protocolVersion <= VERSION_3) { // We need to be backward compatible with 2.1/2.2 nodes paging states. Which means we have to send // the full cellname of the "last" cell in the row we get (since that's how 2.1/2.2 nodes will start after @@ -216,7 +361,7 @@ public static RowMark create(CFMetaData metadata, Row row, int protocolVersion) // If the last returned row has no cell, this means in 2.1/2.2 terms that we stopped on the row // marker. Note that this shouldn't happen if the table is COMPACT. assert !metadata.isCompactTable(); - mark = LegacyLayout.encodeCellName(metadata, row.clustering(), ByteBufferUtil.EMPTY_BYTE_BUFFER, null); + mark = LegacyLayout.encodeCellName(metadata, row.clustering(), EMPTY_BYTE_BUFFER, null); } else { @@ -238,7 +383,7 @@ public Clustering clustering(CFMetaData metadata) if (mark == null) return null; - return protocolVersion <= Server.VERSION_3 + return protocolVersion <= VERSION_3 ? LegacyLayout.decodeClustering(metadata, mark) : Clustering.serializer.deserialize(mark, MessagingService.VERSION_30, makeClusteringTypes(metadata)); } @@ -261,7 +406,7 @@ public final boolean equals(Object o) @Override public String toString() { - return mark == null ? "null" : ByteBufferUtil.bytesToHex(mark); + return mark == null ? "null" : bytesToHex(mark); } } } diff --git a/src/java/org/apache/cassandra/utils/vint/VIntCoding.java b/src/java/org/apache/cassandra/utils/vint/VIntCoding.java index daf5006b9419..27448e2f7cbb 100644 --- a/src/java/org/apache/cassandra/utils/vint/VIntCoding.java +++ b/src/java/org/apache/cassandra/utils/vint/VIntCoding.java @@ -49,6 +49,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.nio.ByteBuffer; import net.nicoulaj.compilecommand.annotations.Inline; @@ -78,6 +79,46 @@ public static long readUnsignedVInt(DataInput input) throws IOException { return retval; } + /** + * Note this method is the same as {@link #readUnsignedVInt(DataInput)}, + * except that we do *not* block if there are not enough bytes in the buffer + * to reconstruct the value. + * + * WARNING: this method is only safe for vints we know to be representable by a positive long value. + * + * @return -1 if there are not enough bytes in the input to read the value; else, the vint unsigned value. + */ + public static long getUnsignedVInt(ByteBuffer input, int readerIndex) + { + return getUnsignedVInt(input, readerIndex, input.limit()); + } + + public static long getUnsignedVInt(ByteBuffer input, int readerIndex, int readerLimit) + { + if (readerIndex >= readerLimit) + return -1; + + int firstByte = input.get(readerIndex++); + + //Bail out early if this is one byte, necessary or it fails later + if (firstByte >= 0) + return firstByte; + + int size = numberOfExtraBytesToRead(firstByte); + if (readerIndex + size > readerLimit) + return -1; + + long retval = firstByte & firstByteValueMask(size); + for (int ii = 0; ii < size; ii++) + { + byte b = input.get(readerIndex++); + retval <<= 8; + retval |= b & 0xff; + } + + return retval; + } + public static long readVInt(DataInput input) throws IOException { return decodeZigZag64(readUnsignedVInt(input)); } diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java index 34f1bcf9fc10..27c630d8dbd0 100644 --- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java +++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java @@ -242,9 +242,10 @@ public void namesQueryTest() throws Exception public void sliceQueryTest() throws Exception { sliceQueryTest(false, Server.VERSION_3); - sliceQueryTest(true, Server.VERSION_4); - sliceQueryTest(false, Server.VERSION_3); - sliceQueryTest(true, Server.VERSION_4); + sliceQueryTest(true, Server.VERSION_3); + + sliceQueryTest(false, Server.VERSION_4); + sliceQueryTest(true, Server.VERSION_4); } public void sliceQueryTest(boolean testPagingState, int protocolVersion) throws Exception @@ -275,9 +276,10 @@ public void sliceQueryTest(boolean testPagingState, int protocolVersion) throws public void reversedSliceQueryTest() throws Exception { reversedSliceQueryTest(false, Server.VERSION_3); - reversedSliceQueryTest(true, Server.VERSION_4); - reversedSliceQueryTest(false, Server.VERSION_3); - reversedSliceQueryTest(true, Server.VERSION_4); + reversedSliceQueryTest(true, Server.VERSION_3); + + reversedSliceQueryTest(false, Server.VERSION_4); + reversedSliceQueryTest(true, Server.VERSION_4); } public void reversedSliceQueryTest(boolean testPagingState, int protocolVersion) throws Exception @@ -308,9 +310,10 @@ public void reversedSliceQueryTest(boolean testPagingState, int protocolVersion) public void multiQueryTest() throws Exception { multiQueryTest(false, Server.VERSION_3); - multiQueryTest(true, Server.VERSION_4); - multiQueryTest(false, Server.VERSION_3); - multiQueryTest(true, Server.VERSION_4); + multiQueryTest(true, Server.VERSION_3); + + multiQueryTest(false, Server.VERSION_4); + multiQueryTest(true, Server.VERSION_4); } public void multiQueryTest(boolean testPagingState, int protocolVersion) throws Exception @@ -346,9 +349,10 @@ public void multiQueryTest(boolean testPagingState, int protocolVersion) throws public void rangeNamesQueryTest() throws Exception { rangeNamesQueryTest(false, Server.VERSION_3); - rangeNamesQueryTest(true, Server.VERSION_4); - rangeNamesQueryTest(false, Server.VERSION_3); - rangeNamesQueryTest(true, Server.VERSION_4); + rangeNamesQueryTest(true, Server.VERSION_3); + + rangeNamesQueryTest(false, Server.VERSION_4); + rangeNamesQueryTest(true, Server.VERSION_4); } public void rangeNamesQueryTest(boolean testPagingState, int protocolVersion) throws Exception @@ -375,9 +379,10 @@ public void rangeNamesQueryTest(boolean testPagingState, int protocolVersion) th public void rangeSliceQueryTest() throws Exception { rangeSliceQueryTest(false, Server.VERSION_3); - rangeSliceQueryTest(true, Server.VERSION_4); - rangeSliceQueryTest(false, Server.VERSION_3); - rangeSliceQueryTest(true, Server.VERSION_4); + rangeSliceQueryTest(true, Server.VERSION_3); + + rangeSliceQueryTest(false, Server.VERSION_4); + rangeSliceQueryTest(true, Server.VERSION_4); } public void rangeSliceQueryTest(boolean testPagingState, int protocolVersion) throws Exception diff --git a/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java b/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java index ba82e85d67b0..8e48771f52b4 100644 --- a/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java +++ b/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -19,6 +18,7 @@ */ package org.apache.cassandra.service.pager; +import java.io.IOException; import java.nio.ByteBuffer; import org.junit.Test; @@ -29,15 +29,21 @@ import org.apache.cassandra.db.*; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.db.marshal.*; -import org.apache.cassandra.transport.Server; import org.apache.cassandra.utils.ByteBufferUtil; +import static org.apache.cassandra.transport.Server.VERSION_3; +import static org.apache.cassandra.transport.Server.VERSION_4; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; public class PagingStateTest { private PagingState makeSomePagingState(int protocolVersion) + { + return makeSomePagingState(protocolVersion, 0); + } + + private PagingState makeSomePagingState(int protocolVersion, int remainingInPartition) { CFMetaData metadata = CFMetaData.Builder.create("ks", "tbl") .addPartitionKey("k", AsciiType.instance) @@ -52,7 +58,7 @@ private PagingState makeSomePagingState(int protocolVersion) Clustering c = new Clustering(ByteBufferUtil.bytes("c1"), ByteBufferUtil.bytes(42)); Row row = BTreeRow.singleCellRow(c, BufferCell.live(metadata, def, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER)); PagingState.RowMark mark = PagingState.RowMark.create(metadata, row, protocolVersion); - return new PagingState(pk, mark, 10, 0); + return new PagingState(pk, mark, 10, remainingInPartition); } @Test @@ -70,9 +76,9 @@ public void testSerializationBackwardCompatibility() * PagingState state = new PagingState(pk, cn.toByteBuffer(), 10); * System.out.println("PagingState = " + ByteBufferUtil.bytesToHex(state.serialize())); */ - PagingState state = makeSomePagingState(Server.VERSION_3); + PagingState state = makeSomePagingState(VERSION_3); - String serializedState = ByteBufferUtil.bytesToHex(state.serialize(Server.VERSION_3)); + String serializedState = ByteBufferUtil.bytesToHex(state.serialize(VERSION_3)); // Note that we don't assert exact equality because we know 3.0 nodes include the "remainingInPartition" number // that is not present on 2.1/2.2 nodes. We know this is ok however because we know that 2.1/2.2 nodes will ignore // anything remaining once they have properly deserialized a paging state. @@ -80,20 +86,56 @@ public void testSerializationBackwardCompatibility() } @Test - public void testSerializeDeserializeV3() + public void testSerializeV3DeserializeV3() + { + PagingState state = makeSomePagingState(VERSION_3); + ByteBuffer serialized = state.serialize(VERSION_3); + assertEquals(serialized.remaining(), state.serializedSize(VERSION_3)); + assertEquals(state, PagingState.deserialize(serialized, VERSION_3)); + } + + @Test + public void testSerializeV4DeserializeV4() + { + PagingState state = makeSomePagingState(VERSION_4); + ByteBuffer serialized = state.serialize(VERSION_4); + assertEquals(serialized.remaining(), state.serializedSize(VERSION_4)); + assertEquals(state, PagingState.deserialize(serialized, VERSION_4)); + } + + @Test + public void testSerializeV3DeserializeV4() + { + PagingState state = makeSomePagingState(VERSION_3); + ByteBuffer serialized = state.serialize(VERSION_3); + assertEquals(serialized.remaining(), state.serializedSize(VERSION_3)); + assertEquals(state, PagingState.deserialize(serialized, VERSION_4)); + } + + @Test + public void testSerializeV4DeserializeV3() + { + PagingState state = makeSomePagingState(VERSION_4); + ByteBuffer serialized = state.serialize(VERSION_4); + assertEquals(serialized.remaining(), state.serializedSize(VERSION_4)); + assertEquals(state, PagingState.deserialize(serialized, VERSION_3)); + } + + @Test + public void testSerializeV3WithoutRemainingInPartitionDeserializeV3() throws IOException { - PagingState state = makeSomePagingState(Server.VERSION_3); - ByteBuffer serialized = state.serialize(Server.VERSION_3); - assertEquals(serialized.remaining(), state.serializedSize(Server.VERSION_3)); - assertEquals(state, PagingState.deserialize(serialized, Server.VERSION_3)); + PagingState state = makeSomePagingState(VERSION_3, Integer.MAX_VALUE); + ByteBuffer serialized = state.legacySerialize(false); + assertEquals(serialized.remaining(), state.legacySerializedSize(false)); + assertEquals(state, PagingState.deserialize(serialized, VERSION_3)); } @Test - public void testSerializeDeserializeV4() + public void testSerializeV3WithoutRemainingInPartitionDeserializeV4() throws IOException { - PagingState state = makeSomePagingState(Server.VERSION_4); - ByteBuffer serialized = state.serialize(Server.VERSION_4); - assertEquals(serialized.remaining(), state.serializedSize(Server.VERSION_4)); - assertEquals(state, PagingState.deserialize(serialized, Server.VERSION_4)); + PagingState state = makeSomePagingState(VERSION_3, Integer.MAX_VALUE); + ByteBuffer serialized = state.legacySerialize(false); + assertEquals(serialized.remaining(), state.legacySerializedSize(false)); + assertEquals(state, PagingState.deserialize(serialized, VERSION_4)); } } From 07ff9e57344f1d837e4aef3cbca26b953745bcd7 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Fri, 7 Jun 2019 16:27:01 -0500 Subject: [PATCH 17/78] Add Table Metrics Virtual Table Patch by Chris Lohfink; Reviewed by Jon Haddad for CASSANDRA-14670 --- .../db/virtual/SystemViewsKeyspace.java | 17 +- .../db/virtual/TableMetricTables.java | 169 ++++++++++++++++++ 2 files changed, 179 insertions(+), 7 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/virtual/TableMetricTables.java diff --git a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java index fa1ddae36632..4fcc41aced0c 100644 --- a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java @@ -27,12 +27,15 @@ public final class SystemViewsKeyspace extends VirtualKeyspace private SystemViewsKeyspace() { - super(NAME, ImmutableList.of(new CachesTable(NAME), - new ClientsTable(NAME), - new SettingsTable(NAME), - new SSTableTasksTable(NAME), - new ThreadPoolsTable(NAME), - new InternodeOutboundTable(NAME), - new InternodeInboundTable(NAME))); + super(NAME, new ImmutableList.Builder() + .add(new CachesTable(NAME)) + .add(new ClientsTable(NAME)) + .add(new SettingsTable(NAME)) + .add(new SSTableTasksTable(NAME)) + .add(new ThreadPoolsTable(NAME)) + .add(new InternodeOutboundTable(NAME)) + .add(new InternodeOutboundTable(NAME)) + .addAll(TableMetricTables.getAll(NAME)) + .build()); } } diff --git a/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java b/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java new file mode 100644 index 000000000000..acae2d0e0ccb --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual; + +import java.util.Collection; +import java.util.function.Function; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Counting; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Metered; +import com.codahale.metrics.Metric; +import com.codahale.metrics.Sampling; +import com.codahale.metrics.Snapshot; +import com.codahale.metrics.Timer; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.metrics.TableMetrics; +import org.apache.cassandra.schema.TableMetadata; + +/** + * Contains multiple the Table Metric virtual tables. This is not a direct wrapper over the Metrics like with JMX but a + * view to the metrics so that the underlying mechanism can change but still give same appearance (like nodetool). + */ +public class TableMetricTables +{ + private final static String KEYSPACE_NAME = "keyspace_name"; + private final static String TABLE_NAME = "table_name"; + private final static String MEDIAN = "median"; + private final static String P99 = "99th"; + private final static String MAX = "max"; + private final static String RATE = "per_second"; + + private final static AbstractType TYPE = CompositeType.getInstance(ReversedType.getInstance(LongType.instance), + UTF8Type.instance, + UTF8Type.instance); + private final static IPartitioner PARTITIONER = new LocalPartitioner(TYPE); + + /** + * Generates all table metric tables in a collection + */ + public static Collection getAll(String name) + { + return ImmutableList.of( + getMetricTable(name, "local_reads", t -> t.readLatency.latency), + getMetricTable(name, "local_scans", t -> t.rangeLatency.latency), + getMetricTable(name, "coordinator_reads", t -> t.coordinatorReadLatency), + getMetricTable(name, "coordinator_scans", t -> t.coordinatorScanLatency), + getMetricTable(name, "local_writes", t -> t.writeLatency.latency), + getMetricTable(name, "coordinator_writes", t -> t.coordinatorWriteLatency), + getMetricTable(name, "tombstones_scanned", t -> t.tombstoneScannedHistogram.cf), + getMetricTable(name, "live_scanned", t -> t.liveScannedHistogram.cf), + getMetricTable(name, "disk_usage", t -> t.totalDiskSpaceUsed, "disk_space"), + getMetricTable(name, "max_partition_size", t -> t.maxPartitionSize, "max_partition_size")); + } + + public static VirtualTable getMetricTable(String keyspace, String table, Function func) + { + return getMetricTable(keyspace, table, func, "count"); + } + + /** + * Abstraction over the Metrics Gauge, Counter, and Timer that will turn it into a ([pk], keyspace_name, table_name) + * table. The primary key (default 'count') is in descending orde in order to visually sort the rows when selecting + * the entire table in CQLSH. + */ + public static VirtualTable getMetricTable(String keyspace, String table, Function func, String pk) + { + TableMetadata.Builder metadata = TableMetadata.builder(keyspace, table) + .kind(TableMetadata.Kind.VIRTUAL) + .addPartitionKeyColumn(pk, ReversedType.getInstance(LongType.instance)) + .addPartitionKeyColumn(KEYSPACE_NAME, UTF8Type.instance) + .addPartitionKeyColumn(TABLE_NAME, UTF8Type.instance) + .partitioner(PARTITIONER); + + Keyspace system = Keyspace.system().iterator().next(); + + // Identify the type of Metric it is (gauge, counter etc) and verify the types work + Metric test = func.apply(system.getColumnFamilyStores().iterator().next().metric); + if(test instanceof Counting) + { + if (test instanceof Sampling) + { + metadata.addRegularColumn(MEDIAN, LongType.instance) + .addRegularColumn(P99, LongType.instance) + .addRegularColumn(MAX, LongType.instance); + } + if (test instanceof Metered) + { + metadata.addRegularColumn(RATE, DoubleType.instance); + } + } + else if (test instanceof Gauge) + { + Preconditions.checkArgument(((Gauge) test).getValue().getClass().isAssignableFrom(Long.class)); + } + + // Create the VirtualTable that will walk through all tables and get the Metric for each to build the tables + // SimpleDataSet + return new AbstractVirtualTable(metadata.build()) + { + public DataSet data() + { + SimpleDataSet result = new SimpleDataSet(metadata()); + for (ColumnFamilyStore cfs : ColumnFamilyStore.all()) + { + Metric metric = func.apply(cfs.metric); + + if(metric instanceof Counting) + { + Counting counting = (Counting) metric; + result.row(counting.getCount(), cfs.keyspace.getName(), cfs.name); + if (metric instanceof Sampling) + { + Sampling histo = (Sampling) metric; + Snapshot snapshot = histo.getSnapshot(); + result.column(MEDIAN, (long) snapshot.getMedian()) + .column(P99, (long) snapshot.get99thPercentile()) + .column(MAX, (long) snapshot.getMax()); + } + if (metric instanceof Metered) + { + Metered timer = (Metered) metric; + result.column(RATE, timer.getFiveMinuteRate()); + } + } + else if (metric instanceof Gauge) + { + result.row(((Gauge) metric).getValue(), cfs.keyspace.getName(), cfs.name); + } + else if (metric instanceof Counter) + { + result.row(((Counter) metric).getCount(), cfs.keyspace.getName(), cfs.name); + } + } + return result; + } + }; + } +} From ebe45c8f9121aa614512a7101d3ee87ca1e2db16 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Wed, 26 Jun 2019 11:19:47 -0500 Subject: [PATCH 18/78] ninja fix merge error --- .../org/apache/cassandra/db/virtual/SystemViewsKeyspace.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java index 4fcc41aced0c..abcdf87c26a6 100644 --- a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java @@ -34,7 +34,7 @@ private SystemViewsKeyspace() .add(new SSTableTasksTable(NAME)) .add(new ThreadPoolsTable(NAME)) .add(new InternodeOutboundTable(NAME)) - .add(new InternodeOutboundTable(NAME)) + .add(new InternodeInboundTable(NAME)) .addAll(TableMetricTables.getAll(NAME)) .build()); } From 61b77ac2b01098776a1bea15a775701230028514 Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Wed, 26 Jun 2019 14:30:39 -0500 Subject: [PATCH 19/78] Update log directory patch for 4.0 deb package Patch by Michael Shuler; Reviewed by Jon Haddad for CASSANDRA-15185 --- debian/patches/cassandra_logdir_fix.diff | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/debian/patches/cassandra_logdir_fix.diff b/debian/patches/cassandra_logdir_fix.diff index d75553c11a37..85973313324c 100644 --- a/debian/patches/cassandra_logdir_fix.diff +++ b/debian/patches/cassandra_logdir_fix.diff @@ -1,14 +1,14 @@ --- a/bin/cassandra +++ b/bin/cassandra -@@ -171,7 +171,7 @@ - props="$3" - class="$4" - cassandra_parms="-Dlogback.configurationFile=logback.xml" -- cassandra_parms="$cassandra_parms -Dcassandra.logdir=$CASSANDRA_HOME/logs" -+ cassandra_parms="$cassandra_parms -Dcassandra.logdir=/var/log/cassandra" - cassandra_parms="$cassandra_parms -Dcassandra.storagedir=$cassandra_storagedir" +@@ -109,7 +109,7 @@ + fi + + if [ -z "$CASSANDRA_LOG_DIR" ]; then +- CASSANDRA_LOG_DIR=$CASSANDRA_HOME/logs ++ CASSANDRA_LOG_DIR=/var/log/cassandra + fi - if [ "x$pidpath" != "x" ]; then + # Special-case path variables. --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -93,16 +93,16 @@ From 2ed2b87b634c1b9d9ec9b3ba3f580f1be753972a Mon Sep 17 00:00:00 2001 From: Mick Semb Wever Date: Sun, 7 Apr 2019 21:29:19 +1000 Subject: [PATCH 20/78] Fix accessing java.nio.Bits.totalCapacity in Java11 patch by Mick Semb Wever; reviewed by Robert Stupp for CASSANDRA-14757 --- .../org/apache/cassandra/service/GCInspector.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java index 657d3adeacb8..e0a935dcade9 100644 --- a/src/java/org/apache/cassandra/service/GCInspector.java +++ b/src/java/org/apache/cassandra/service/GCInspector.java @@ -69,7 +69,16 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean try { Class bitsClass = Class.forName("java.nio.Bits"); - Field f = bitsClass.getDeclaredField("totalCapacity"); + Field f; + try + { + f = bitsClass.getDeclaredField("totalCapacity"); + } + catch (NoSuchFieldException ex) + { + // in Java11 it changed name to "TOTAL_CAPACITY" + f = bitsClass.getDeclaredField("TOTAL_CAPACITY"); + } f.setAccessible(true); temp = f; } From dc23631ab17c2c0cc21b05732f905bb23cb67682 Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Thu, 27 Jun 2019 09:49:43 -0500 Subject: [PATCH 21/78] Update log directory patch for deb package Patch by Michael Shuler; Reviewed by Jon Haddad for CASSANDRA-15185 --- debian/patches/002cassandra_logdir_fix.dpatch | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) mode change 100644 => 100755 debian/patches/002cassandra_logdir_fix.dpatch diff --git a/debian/patches/002cassandra_logdir_fix.dpatch b/debian/patches/002cassandra_logdir_fix.dpatch old mode 100644 new mode 100755 index 87387b9c6704..5e76b91fe8c4 --- a/debian/patches/002cassandra_logdir_fix.dpatch +++ b/debian/patches/002cassandra_logdir_fix.dpatch @@ -6,21 +6,21 @@ @DPATCH@ diff -urNad '--exclude=CVS' '--exclude=.svn' '--exclude=.git' '--exclude=.arch' '--exclude=.hg' '--exclude=_darcs' '--exclude=.bzr' cassandra~/bin/cassandra cassandra/bin/cassandra ---- cassandra~/bin/cassandra 2015-10-27 14:35:22.000000000 -0500 -+++ cassandra/bin/cassandra 2015-10-27 14:41:38.000000000 -0500 -@@ -139,7 +139,7 @@ - props="$3" - class="$4" - cassandra_parms="-Dlogback.configurationFile=logback.xml" -- cassandra_parms="$cassandra_parms -Dcassandra.logdir=$CASSANDRA_HOME/logs" -+ cassandra_parms="$cassandra_parms -Dcassandra.logdir=/var/log/cassandra" - cassandra_parms="$cassandra_parms -Dcassandra.storagedir=$cassandra_storagedir" +--- cassandra~/bin/cassandra 2019-06-27 09:35:32.000000000 -0500 ++++ cassandra/bin/cassandra 2019-06-27 09:43:28.756343141 -0500 +@@ -127,7 +127,7 @@ + fi - if [ "x$pidpath" != "x" ]; then + if [ -z "$CASSANDRA_LOG_DIR" ]; then +- CASSANDRA_LOG_DIR=$CASSANDRA_HOME/logs ++ CASSANDRA_LOG_DIR=/var/log/cassandra + fi + + # Special-case path variables. diff -urNad '--exclude=CVS' '--exclude=.svn' '--exclude=.git' '--exclude=.arch' '--exclude=.hg' '--exclude=_darcs' '--exclude=.bzr' cassandra~/conf/cassandra-env.sh cassandra/conf/cassandra-env.sh ---- cassandra~/conf/cassandra-env.sh 2015-10-27 14:40:39.000000000 -0500 -+++ cassandra/conf/cassandra-env.sh 2015-10-27 14:42:40.647449856 -0500 -@@ -204,7 +204,7 @@ +--- cassandra~/conf/cassandra-env.sh 2019-06-27 09:35:32.000000000 -0500 ++++ cassandra/conf/cassandra-env.sh 2019-06-27 09:42:25.747715490 -0500 +@@ -122,7 +122,7 @@ esac #GC log path has to be defined here because it needs to access CASSANDRA_HOME From f4b6e1d51f683e0c77c6ff7f199373052b082b9e Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe Date: Thu, 13 Jun 2019 14:44:26 +0100 Subject: [PATCH 22/78] Filter illegal legacy cells when collating rows Alternative solution for CASSANDRA-15086, which allows the illegal cells to be read from disk and deserialized as normal so as not to interfere with tracking bytes read during reverse iteration of indexed partitions. Patch by Sam Tunnicliffe; reviewed by Marcus Eriksson for CASSANDRA-15178 --- CHANGES.txt | 1 + .../db/IllegalLegacyColumnException.java | 41 ------- .../org/apache/cassandra/db/LegacyLayout.java | 76 ++++-------- .../org/apache/cassandra/db/ReadCommand.java | 2 +- .../cassandra/db/UnfilteredDeserializer.java | 7 -- .../cassandra/db/UnknownColumnException.java | 16 ++- .../cassandra/thrift/CassandraServer.java | 34 +++--- .../cassandra/thrift/ThriftValidation.java | 10 +- ...ith_illegal_cell_names_indexed-ka-1-CRC.db | Bin 0 -> 8 bytes ...th_illegal_cell_names_indexed-ka-1-Data.db | Bin 0 -> 6487 bytes ...llegal_cell_names_indexed-ka-1-Digest.sha1 | 1 + ..._illegal_cell_names_indexed-ka-1-Filter.db | Bin 0 -> 16 bytes ...h_illegal_cell_names_indexed-ka-1-Index.db | Bin 0 -> 453 bytes ...egal_cell_names_indexed-ka-1-Statistics.db | Bin 0 -> 4472 bytes ...th_illegal_cell_names_indexed-ka-1-TOC.txt | 8 ++ .../validation/ThriftIllegalColumnsTest.java | 100 ++++++++++++++++ .../cassandra/db/LegacyCellNameTest.java | 18 ++- .../io/sstable/LegacySSTableTest.java | 109 ++++++++++++++---- 18 files changed, 273 insertions(+), 150 deletions(-) delete mode 100644 src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java create mode 100644 test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-CRC.db create mode 100644 test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Data.db create mode 100644 test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Digest.sha1 create mode 100644 test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Filter.db create mode 100644 test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Index.db create mode 100644 test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Statistics.db create mode 100644 test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-TOC.txt create mode 100644 test/unit/org/apache/cassandra/cql3/validation/ThriftIllegalColumnsTest.java diff --git a/CHANGES.txt b/CHANGES.txt index eae281518fb9..d34406b70af6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Skipping illegal legacy cells can break reverse iteration of indexed partitions (CASSANDRA-15178) * Handle paging states serialized with a different version than the session's (CASSANDRA-15176) * Throw IOE instead of asserting on unsupporter peer versions (CASSANDRA-15066) * Update token metadata when handling MOVING/REMOVING_TOKEN events (CASSANDRA-15120) diff --git a/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java b/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java deleted file mode 100644 index b70d24899d60..000000000000 --- a/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.db; - -import java.nio.ByteBuffer; - -import org.apache.cassandra.config.CFMetaData; - -import static org.apache.cassandra.db.LegacyLayout.stringify; - -/** - * Exception thrown when we attempt to decode a legacy cellname - * and the column name component refers to a primary key column. - */ -public class IllegalLegacyColumnException extends Exception -{ - public final ByteBuffer columnName; - - public IllegalLegacyColumnException(CFMetaData metaData, ByteBuffer columnName) - { - super(String.format("Illegal cell name for CQL3 table %s.%s. %s is defined as a primary key column", - metaData.ksName, metaData.cfName, stringify(columnName))); - this.columnName = columnName; - } -} diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java index cfaa71f7703b..b03f56eb0820 100644 --- a/src/java/org/apache/cassandra/db/LegacyLayout.java +++ b/src/java/org/apache/cassandra/db/LegacyLayout.java @@ -124,7 +124,7 @@ public static AbstractType makeLegacyComparator(CFMetaData metadata) } public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer cellname) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { assert cellname != null; if (metadata.isSuper()) @@ -151,12 +151,12 @@ private static LegacyCellName decodeForSuperColumn(CFMetaData metadata, Clusteri return new LegacyCellName(clustering, def, subcol); } - public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException, IllegalLegacyColumnException + public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException { return decodeCellName(metadata, cellname, false); } - public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException, IllegalLegacyColumnException + public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException { Clustering clustering = decodeClustering(metadata, cellname); @@ -184,22 +184,16 @@ public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cell return new LegacyCellName(clustering, null, null); ColumnDefinition def = metadata.getColumnDefinition(column); - if ((def == null) || def.isPrimaryKeyColumn()) + + if (metadata.isCompactTable()) { - // If it's a compact table, it means the column is in fact a "dynamic" one - if (metadata.isCompactTable()) + if (def == null || def.isPrimaryKeyColumn()) + // If it's a compact table, it means the column is in fact a "dynamic" one return new LegacyCellName(new Clustering(column), metadata.compactValueColumn(), null); - - if (def == null) - { - throw new UnknownColumnException(metadata, column); - } - else - { - noSpamLogger.warn("Illegal cell name for CQL3 table {}.{}. {} is defined as a primary key column", - metadata.ksName, metadata.cfName, stringify(column)); - throw new IllegalLegacyColumnException(metadata, column); - } + } + else if (def == null) + { + throw new UnknownColumnException(metadata, column); } ByteBuffer collectionElement = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size() + 1) : null; @@ -750,13 +744,6 @@ private static LegacyAtom readLegacyAtomSkippingUnknownColumn(CFMetaData metadat { // Simply skip, as the method name implies. } - catch (IllegalLegacyColumnException e) - { - // We can arrive here if the table is non-compact and an sstable contains cells whose column name components - // refer to a primary key column. This is not possible through CQL, but is through thrift or side loading - // sstables. In this case, we treat the column as an unknown and skip, which is equivalent to the pre - // 3.0 read path behaviour - } } } @@ -1116,7 +1103,7 @@ private static Comparator legacyAtomComparator(CFMetaData metadata) } public static LegacyAtom readLegacyAtom(CFMetaData metadata, DataInputPlus in, boolean readAllAsDynamic) - throws IOException, UnknownColumnException, IllegalLegacyColumnException + throws IOException, UnknownColumnException { ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in); if (!cellname.hasRemaining()) @@ -1148,7 +1135,7 @@ public static LegacyAtom readLegacyAtom(CFMetaData metadata, DataInputPlus in, b } } - public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException, IllegalLegacyColumnException + public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException { ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in); int b = in.readUnsignedByte(); @@ -1156,7 +1143,7 @@ public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, Seria } public static LegacyCell readLegacyCellBody(CFMetaData metadata, DataInput in, ByteBuffer cellname, int mask, SerializationHelper.Flag flag, boolean readAllAsDynamic) - throws IOException, UnknownColumnException, IllegalLegacyColumnException + throws IOException, UnknownColumnException { // Note that we want to call decodeCellName only after we've deserialized other parts, since it can throw // and we want to throw only after having deserialized the full cell. @@ -1228,14 +1215,6 @@ protected LegacyCell computeNext() else throw new IOError(e); } - catch (IllegalLegacyColumnException e) - { - // We can arrive here if the table is non-compact and an sstable contains cells whose column name components - // refer to a primary key column. This is not possible through CQL, but is through thrift or side loading - // sstables. In this case, we treat the atom as though it were unknown and skip on to the next, which is - // equivalent to the pre 3.0 behaviour - return computeNext(); - } catch (IOException e) { throw new IOError(e); @@ -1334,6 +1313,13 @@ else if (metadata.isIndex()) if (collectionDeletion != null && collectionDeletion.start.collectionName.name.equals(column.name) && collectionDeletion.deletionTime.deletes(cell.timestamp)) return true; + if (column.isPrimaryKeyColumn() && metadata.isCQLTable()) + { + noSpamLogger.warn("Illegal cell name for CQL3 table {}.{}. {} is defined as a primary key column", + metadata.ksName, metadata.cfName, column.name); + return true; + } + if (helper.includes(column)) { CellPath path = null; @@ -1633,13 +1619,13 @@ private LegacyCell(Kind kind, LegacyCellName name, ByteBuffer value, long timest } public static LegacyCell regular(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { return new LegacyCell(Kind.REGULAR, decodeCellName(metadata, superColumnName, name), value, timestamp, Cell.NO_DELETION_TIME, Cell.NO_TTL); } public static LegacyCell expiring(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp, int ttl, int nowInSec) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { /* * CASSANDRA-14092: Max expiration date capping is maybe performed here, expiration overflow policy application @@ -1649,13 +1635,13 @@ public static LegacyCell expiring(CFMetaData metadata, ByteBuffer superColumnNam } public static LegacyCell tombstone(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { return new LegacyCell(Kind.DELETED, decodeCellName(metadata, superColumnName, name), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, nowInSec, LivenessInfo.NO_TTL); } public static LegacyCell counterUpdate(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long value) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { // See UpdateParameters.addCounter() for more details on this ByteBuffer counterValue = CounterContext.instance().createUpdate(value); @@ -2606,16 +2592,4 @@ private long serializedSizeSimple() return size; } } - - public static String stringify(ByteBuffer name) - { - try - { - return UTF8Type.instance.getString(name); - } - catch (Exception e) - { - return ByteBufferUtil.bytesToHex(name); - } - } } diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index eb5cddf24081..b499dafb9ba5 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -1340,7 +1340,7 @@ static Pair deserializeNamesSelectionA { cellName = LegacyLayout.decodeCellName(metadata, buffer); } - catch (UnknownColumnException | IllegalLegacyColumnException exc) + catch (UnknownColumnException exc) { // TODO this probably needs a new exception class that shares a parent with UnknownColumnFamilyException throw new UnknownColumnFamilyException( diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java index 50c9a1ad2a79..62ad76a1b3f8 100644 --- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java +++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java @@ -302,13 +302,6 @@ private LegacyLayout.LegacyAtom readAtom() // we should *not* count the byte of that discarded cell as part of the bytes for the atom // we will eventually return, as doing so could throw the logic bytesReadForNextAtom participates in. } - catch (IllegalLegacyColumnException e) - { - // This can occur if the table is non-compact and an sstable contains cells whose column name components - // refer to a primary key column. This is not possible through CQL, but is through thrift or side loading - // sstables. We treat this as we wold an unknown column and skip without counting the read bytes as - // part of the following atom - } catch (IOException e) { throw new IOError(e); diff --git a/src/java/org/apache/cassandra/db/UnknownColumnException.java b/src/java/org/apache/cassandra/db/UnknownColumnException.java index a480022f2886..55dc453865d4 100644 --- a/src/java/org/apache/cassandra/db/UnknownColumnException.java +++ b/src/java/org/apache/cassandra/db/UnknownColumnException.java @@ -20,8 +20,8 @@ import java.nio.ByteBuffer; import org.apache.cassandra.config.CFMetaData; - -import static org.apache.cassandra.db.LegacyLayout.stringify; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.utils.ByteBufferUtil; /** * Exception thrown when we read a column internally that is unknown. Note that @@ -36,4 +36,16 @@ public UnknownColumnException(CFMetaData metadata, ByteBuffer columnName) super(String.format("Unknown column %s in table %s.%s", stringify(columnName), metadata.ksName, metadata.cfName)); this.columnName = columnName; } + + private static String stringify(ByteBuffer name) + { + try + { + return UTF8Type.instance.getString(name); + } + catch (Exception e) + { + return ByteBufferUtil.bytesToHex(name); + } + } } diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java index 3a41df028582..163eb2dcb9cd 100644 --- a/src/java/org/apache/cassandra/thrift/CassandraServer.java +++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java @@ -416,7 +416,7 @@ private ClusteringIndexFilter toInternalFilter(CFMetaData metadata, ColumnParent return toInternalFilter(metadata, parent, predicate.slice_range); } } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage()); } @@ -521,7 +521,7 @@ private ColumnFilter makeColumnFilter(CFMetaData metadata, ColumnParent parent, return makeColumnFilter(metadata, parent, predicate.slice_range); } } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage()); } @@ -660,7 +660,7 @@ public ColumnOrSuperColumn get(ByteBuffer key, ColumnPath column_path, Consisten return tcolumns.get(0); } } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new InvalidRequestException(e.getMessage()); } @@ -859,7 +859,7 @@ private void internal_insert(ByteBuffer key, ColumnParent column_parent, Column mutation = new org.apache.cassandra.db.Mutation(update); } - catch (MarshalException | UnknownColumnException | IllegalLegacyColumnException e) + catch (MarshalException|UnknownColumnException e) { throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage()); } @@ -964,7 +964,7 @@ public CASResult cas(ByteBuffer key, : new CASResult(false).setCurrent_values(thriftifyColumnsAsColumns(metadata, LegacyLayout.fromRowIterator(result).right)); } } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new InvalidRequestException(e.getMessage()); } @@ -986,13 +986,13 @@ public CASResult cas(ByteBuffer key, } } - private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, Column column, int nowInSec) throws UnknownColumnException, IllegalLegacyColumnException + private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, Column column, int nowInSec) throws UnknownColumnException { return toLegacyCell(metadata, null, column, nowInSec); } private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, ByteBuffer superColumnName, Column column, int nowInSec) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { return column.ttl > 0 ? LegacyLayout.LegacyCell.expiring(metadata, superColumnName, column.name, column.value, column.timestamp, column.ttl, nowInSec) @@ -1000,25 +1000,25 @@ private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, ByteBuffer sup } private LegacyLayout.LegacyCell toLegacyDeletion(CFMetaData metadata, ByteBuffer name, long timestamp, int nowInSec) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { return toLegacyDeletion(metadata, null, name, timestamp, nowInSec); } private LegacyLayout.LegacyCell toLegacyDeletion(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { return LegacyLayout.LegacyCell.tombstone(metadata, superColumnName, name, timestamp, nowInSec); } private LegacyLayout.LegacyCell toCounterLegacyCell(CFMetaData metadata, CounterColumn column) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { return toCounterLegacyCell(metadata, null, column); } private LegacyLayout.LegacyCell toCounterLegacyCell(CFMetaData metadata, ByteBuffer superColumnName, CounterColumn column) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { return LegacyLayout.LegacyCell.counterUpdate(metadata, superColumnName, column.name, column.value); } @@ -1085,7 +1085,7 @@ private void sortAndMerge(CFMetaData metadata, List cel } private List toLegacyCells(CFMetaData metadata, List columns, int nowInSec) - throws UnknownColumnException, IllegalLegacyColumnException + throws UnknownColumnException { List cells = new ArrayList<>(columns.size()); for (Column column : columns) @@ -1205,7 +1205,7 @@ else if (cosc.counter_super_column != null) cells.add(toCounterLegacyCell(cfm, cosc.counter_column)); } } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new InvalidRequestException(e.getMessage()); } @@ -1232,7 +1232,7 @@ else if (del.super_column != null) else cells.add(toLegacyDeletion(cfm, c, del.timestamp, nowInSec)); } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new InvalidRequestException(e.getMessage()); } @@ -1370,7 +1370,7 @@ else if (column_path.super_column != null && column_path.column == null) Cell cell = BufferCell.tombstone(name.column, timestamp, nowInSec, path); update = PartitionUpdate.singleRowUpdate(metadata, dk, BTreeRow.singleCellRow(name.clustering, cell)); } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage()); } @@ -1633,7 +1633,7 @@ public List get_paged_slice(String column_family, KeyRange range, Byte return thriftifyKeySlices(results, new ColumnParent(column_family), limits.perPartitionCount()); } } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new InvalidRequestException(e.getMessage()); } @@ -2178,7 +2178,7 @@ public void add(ByteBuffer key, ColumnParent column_parent, CounterColumn column org.apache.cassandra.db.Mutation mutation = new org.apache.cassandra.db.Mutation(update); doInsert(consistency_level, Arrays.asList(new CounterMutation(mutation, ThriftConversion.fromThrift(consistency_level)))); } - catch (MarshalException | UnknownColumnException | IllegalLegacyColumnException e) + catch (MarshalException|UnknownColumnException e) { throw new InvalidRequestException(e.getMessage()); } diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java index e7a344aa2b01..4b208baf84af 100644 --- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java +++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java @@ -31,7 +31,6 @@ import org.apache.cassandra.cql3.Attributes; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.*; -import org.apache.cassandra.db.IllegalLegacyColumnException; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; @@ -266,11 +265,13 @@ private static void validateColumnNames(CFMetaData metadata, ByteBuffer superCol i, metadata.comparator.size() + 1, metadata.cfName)); } + + // On top of that, if we have a collection component, the (CQL3) column must be a collection if (cname.column != null && cname.collectionElement != null && !cname.column.type.isCollection()) throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Invalid collection component, %s is not a collection", cname.column.name)); } - catch (IllegalArgumentException | UnknownColumnException | IllegalLegacyColumnException e ) + catch (IllegalArgumentException | UnknownColumnException e) { throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Error validating cell name for CQL3 table %s: %s", metadata.cfName, e.getMessage())); } @@ -464,10 +465,13 @@ public static void validateColumnData(CFMetaData metadata, ByteBuffer scName, Co try { LegacyLayout.LegacyCellName cn = LegacyLayout.decodeCellName(metadata, scName, column.name); + if (cn.column.isPrimaryKeyColumn()) + throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Cannot add primary key column %s to partition update", cn.column.name)); + cn.column.validateCellValue(column.value); } - catch (UnknownColumnException | IllegalLegacyColumnException e) + catch (UnknownColumnException e) { throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage()); } diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-CRC.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-CRC.db new file mode 100644 index 0000000000000000000000000000000000000000..82ca06af159e47986dc9640e257f2f5275502f9f GIT binary patch literal 8 PcmZQzWMC-Qlz#~T1C0Ty literal 0 HcmV?d00001 diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Data.db new file mode 100644 index 0000000000000000000000000000000000000000..269a739d9452e57721860495aa4479773e9c0bea GIT binary patch literal 6487 zcmd7WD-HrN6h`3^!VXvn!<*#{1ckwZft{B-lM6Xb&~PgHi<%=o)iS-DZ;x8*v*ekV z*HMtZ-Da|*cn6^SPejc0Zua}GQdO)p`t*VF&U;y5W-+##iptX zuGbi<7f3TELG`kTq3cCf1+)R`8dIoVAPsU27;s3V>qS-tv;nHu45}AMGXh-$20RK7 z8sTPhs45_h_aTqR zV|dGD9e1L*Bc3~G?nvi(Wyw>;Y&M&cCKIQyIXP^mRL+W}X$o=ZNrfz0E|tk;(im2* zNLQq1Wy<8iMS1TcnygQ8H*1QSTCH9*cc7^Atu6MP%a^V$e~7labkck%ESe98MfZTZ zYkXZrdlnEjAZ$q3h_DG^Gr~f`mV~VdYx)X{69iXxCG1Jqhwx6qk%adXP9>aAxQuWM z;Zbag7!PVYkt+yyW7Bqo3$YpE?=zJ+=W8Y;C<0#qu4)j7ZAkRjXaVOY{n%zI>^0}G zMc1)y4{EKQw~iw?U+;yzu?Ab5i0xm69omH*V}qUKfSt1!yO_nUCFXCx1m`MeWXky~ zcH%zlDm*_a)k~DqadN_cPXx-X~>ZF$3FE!4Vgb3tZ}upZ0n~(;jW;`<1>bd;xRDF&E7brsIe) z%=x*-LC}K3qo}_IS?RejH*8nVLK{oMzeAfkccS%Vo+uiExdju|2W{o=j^?wSaQ+(1 z*EU?5g5Iz#@(#4etAw4P@DQ|c>1xc+IGa{NBHpPpz8$^&K; zl=3n#-)&G71wA*H564IFqqGeEei&}HnhXK$jmmH5-32y@zR@QKHalAK1@>R~+Hwl^ z-*PyzMFiU0u*KQ%_rRepxKj*r=f{&GI9_hwBBF{xz9nAe4)+6ozMuMpAP*k>(0KxQ zPi1o>e4az906#dNGh-#SrJ#LLa(n*-aP`}_(Ds(CruI{ps+a}P#?do^&D6^1A9))d`|V!3Sj;@0Jh+= 0 ORDER BY cc ASC;")); - Assert.assertEquals(5, forward.size()); - Assert.assertEquals(5, reverse.size()); + assertEquals(5, forward.size()); + assertEquals(5, reverse.size()); } @Test @@ -301,7 +304,7 @@ public void testMultiBlockRangeTombstones() throws Exception UntypedResultSet reverse = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM legacy_tables.%s WHERE k=100 ORDER BY c1 DESC, c2 DESC", table)); Assert.assertFalse(forward.isEmpty()); - Assert.assertEquals(table, forward.size(), reverse.size()); + assertEquals(table, forward.size(), reverse.size()); } } @@ -320,7 +323,7 @@ public void testInaccurateSSTableMinMax() throws Exception String query = "SELECT * FROM legacy_tables.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=1 AND c2=1"; List unfiltereds = SinglePartitionSliceCommandTest.getUnfilteredsFromSinglePartition(query); - Assert.assertEquals(2, unfiltereds.size()); + assertEquals(2, unfiltereds.size()); Assert.assertTrue(unfiltereds.get(0).isRangeTombstoneMarker()); Assert.assertTrue(((RangeTombstoneMarker) unfiltereds.get(0)).isOpen(false)); Assert.assertTrue(unfiltereds.get(1).isRangeTombstoneMarker()); @@ -442,6 +445,66 @@ public void testReadingLegacyTablesWithIllegalCellNames() throws Exception { assertRows(results, row(1, "a", "aa", "aaa"), row(2, "b", "bb", "bbb")); } + @Test + public void testReadingIndexedLegacyTablesWithIllegalCellNames() throws Exception { + /** + * The sstable can be generated externally with SSTableSimpleUnsortedWriter: + * column_index_size_in_kb: 1 + * [ + * {"key": "key", + * "cells": [ + * ["00000:000000:a","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0], + * ["00000:000000:b","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ["00000:000000:c","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ["00000:000000:z","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ["00001:000001:a","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0], + * ["00001:000001:b","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ["00001:000001:c","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ["00001:000001:z","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * . + * . + * . + * ["00010:000010:a","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0], + * ["00010:000010:b","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ["00010:000010:c","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ["00010:000010:z","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0] + * ] + * } + * ] + * Each row in the partition contains only 1 valid cell. The ones with the column name components 'a', 'b' & 'z' are illegal as they refer to PRIMARY KEY + * columns, but SSTables such as this can be generated with offline tools and loaded via SSTableLoader or nodetool refresh (see CASSANDRA-15086) (see + * CASSANDRA-15086) Only 'c' is a valid REGULAR column in the table schema. + * In the initial fix for CASSANDRA-15086, the bytes read by OldFormatDeserializer for these invalid cells are not correctly accounted for, causing + * ReverseIndexedReader to assert that the end of a block has been reached earlier than it actually has, which in turn causes rows to be incorrectly + * ommitted from the results. + * + * This sstable has been crafted to hit a further potential error condition. Rows 00001:00001 and 00008:00008 interact with the index block boundaries + * in a very specific way; for both of these rows, the (illegal) cells 'a' & 'b', along with the valid 'c' cell are at the end of an index block, but + * the 'z' cell is over the boundary, in the following block. We need to ensure that the bytes consumed for the 'z' cell are properly accounted for and + * not counted toward those for the next row on disk. + */ + QueryProcessor.executeInternal("CREATE TABLE legacy_tables.legacy_ka_with_illegal_cell_names_indexed (" + + " a text," + + " b text," + + " z text," + + " c text," + + " PRIMARY KEY(a, b, z))"); + loadLegacyTable("legacy_%s_with_illegal_cell_names_indexed%s", "ka", ""); + String queryForward = "SELECT * FROM legacy_tables.legacy_ka_with_illegal_cell_names_indexed WHERE a = 'key'"; + String queryReverse = queryForward + " ORDER BY b DESC, z DESC"; + + List forward = new ArrayList<>(); + QueryProcessor.executeOnceInternal(queryForward).forEach(r -> forward.add(r.getString("b") + ":" + r.getString("z"))); + + List reverse = new ArrayList<>(); + QueryProcessor.executeOnceInternal(queryReverse).forEach(r -> reverse.add(r.getString("b") + ":" + r.getString("z"))); + + assertEquals(11, reverse.size()); + assertEquals(11, forward.size()); + for (int i=0; i < 11; i++) + assertEquals(forward.get(i), reverse.get(10 - i)); + } + private void assertExpectedRowsWithDroppedCollection(boolean droppedCheckSuccessful) { for (int i=0; i<=1; i++) @@ -449,7 +512,7 @@ private void assertExpectedRowsWithDroppedCollection(boolean droppedCheckSuccess UntypedResultSet rows = QueryProcessor.executeOnceInternal( String.format("SELECT * FROM legacy_tables.legacy_ka_14912 WHERE k = %s;", i)); - Assert.assertEquals(1, rows.size()); + assertEquals(1, rows.size()); UntypedResultSet.Row row = rows.one(); // If the best-effort attempt to filter dropped columns was successful, then the row @@ -462,9 +525,9 @@ private void assertExpectedRowsWithDroppedCollection(boolean droppedCheckSuccess if (droppedCheckSuccessful || i == 0) Assert.assertFalse(row.has("v1")); else - Assert.assertEquals("", row.getString("v1")); + assertEquals("", row.getString("v1")); - Assert.assertEquals("abc", row.getString("v2")); + assertEquals("abc", row.getString("v2")); } } @@ -517,12 +580,12 @@ private static void verifyCache(String legacyVersion, long startCount) throws In Assert.assertTrue(endCount > startCount); CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get(); CacheService.instance.invalidateKeyCache(); - Assert.assertEquals(startCount, CacheService.instance.keyCache.size()); + assertEquals(startCount, CacheService.instance.keyCache.size()); CacheService.instance.keyCache.loadSaved(); if (BigFormat.instance.getVersion(legacyVersion).storeRows()) - Assert.assertEquals(endCount, CacheService.instance.keyCache.size()); + assertEquals(endCount, CacheService.instance.keyCache.size()); else - Assert.assertEquals(startCount, CacheService.instance.keyCache.size()); + assertEquals(startCount, CacheService.instance.keyCache.size()); } private static void verifyReads(String legacyVersion) @@ -557,8 +620,8 @@ private static void readClusteringCounterTable(String legacyVersion, String comp UntypedResultSet rs; rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue); Assert.assertNotNull(rs); - Assert.assertEquals(1, rs.size()); - Assert.assertEquals(1L, rs.one().getLong("val")); + assertEquals(1, rs.size()); + assertEquals(1L, rs.one().getLong("val")); } private static void readClusteringTable(String legacyVersion, String compactSuffix, int ck, String ckValue, String pkValue) @@ -580,8 +643,8 @@ private static void readSimpleCounterTable(String legacyVersion, String compactS UntypedResultSet rs; rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter%s WHERE pk=?", legacyVersion, compactSuffix), pkValue); Assert.assertNotNull(rs); - Assert.assertEquals(1, rs.size()); - Assert.assertEquals(1L, rs.one().getLong("val")); + assertEquals(1, rs.size()); + assertEquals(1L, rs.one().getLong("val")); } private static void readSimpleTable(String legacyVersion, String compactSuffix, String pkValue) @@ -590,8 +653,8 @@ private static void readSimpleTable(String legacyVersion, String compactSuffix, UntypedResultSet rs; rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple%s WHERE pk=?", legacyVersion, compactSuffix), pkValue); Assert.assertNotNull(rs); - Assert.assertEquals(1, rs.size()); - Assert.assertEquals("foo bar baz", rs.one().getString("val")); + assertEquals(1, rs.size()); + assertEquals("foo bar baz", rs.one().getString("val")); } private static void createKeyspace() @@ -633,12 +696,12 @@ private static void truncateTables(String legacyVersion) private static void assertLegacyClustRows(int count, UntypedResultSet rs) { Assert.assertNotNull(rs); - Assert.assertEquals(count, rs.size()); + assertEquals(count, rs.size()); for (int i = 0; i < count; i++) { for (UntypedResultSet.Row r : rs) { - Assert.assertEquals(128, r.getString("val").length()); + assertEquals(128, r.getString("val").length()); } } } From 97eae441dab742f0eaffcedc360991350232cfd6 Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Fri, 7 Dec 2018 14:32:54 +0000 Subject: [PATCH 23/78] Multiget Thrift query returns null records after digest mismatch patch by Benedict; reviewed by Mick Semb Wever for CASSANDRA-14812 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/filter/DataLimits.java | 6 ++++++ .../apache/cassandra/db/partitions/PartitionIterators.java | 6 ++++-- .../org/apache/cassandra/db/transform/BasePartitions.java | 4 ++-- 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d34406b70af6..c891ce284164 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Multiget thrift query returns null records after digest mismatch (CASSANDRA-14812) * Skipping illegal legacy cells can break reverse iteration of indexed partitions (CASSANDRA-15178) * Handle paging states serialized with a different version than the session's (CASSANDRA-15176) * Throw IOE instead of asserting on unsupporter peer versions (CASSANDRA-15066) diff --git a/src/java/org/apache/cassandra/db/filter/DataLimits.java b/src/java/org/apache/cassandra/db/filter/DataLimits.java index 46a1c6d892f1..fa9d47abd08d 100644 --- a/src/java/org/apache/cassandra/db/filter/DataLimits.java +++ b/src/java/org/apache/cassandra/db/filter/DataLimits.java @@ -67,6 +67,12 @@ public UnfilteredRowIterator filter(UnfilteredRowIterator iter, { return iter; } + + @Override + public PartitionIterator filter(PartitionIterator iter, int nowInSec, boolean countPartitionsWithOnlyStaticData, boolean enforceStrictLiveness) + { + return iter; + } }; // We currently deal with distinct queries by querying full partitions but limiting the result at 1 row per diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java index 9b7d7eb39008..a3cf74652a8d 100644 --- a/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java +++ b/src/java/org/apache/cassandra/db/partitions/PartitionIterators.java @@ -21,6 +21,7 @@ import java.security.MessageDigest; import org.apache.cassandra.db.EmptyIterators; +import org.apache.cassandra.db.transform.FilteredPartitions; import org.apache.cassandra.db.transform.MorePartitions; import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.utils.AbstractIterator; @@ -67,7 +68,7 @@ public static PartitionIterator concat(final List iterators) class Extend implements MorePartitions { - int i = 1; + int i = 0; public PartitionIterator moreContents() { if (i >= iterators.size()) @@ -75,7 +76,8 @@ public PartitionIterator moreContents() return iterators.get(i++); } } - return MorePartitions.extend(iterators.get(0), new Extend()); + + return MorePartitions.extend(EmptyIterators.partition(), new Extend()); } public static PartitionIterator singletonIterator(RowIterator iterator) diff --git a/src/java/org/apache/cassandra/db/transform/BasePartitions.java b/src/java/org/apache/cassandra/db/transform/BasePartitions.java index 2f76452aa2af..f6c486dfba7d 100644 --- a/src/java/org/apache/cassandra/db/transform/BasePartitions.java +++ b/src/java/org/apache/cassandra/db/transform/BasePartitions.java @@ -89,7 +89,7 @@ public final boolean hasNext() Transformation[] fs = stack; int len = length; - while (!stop.isSignalled && input.hasNext()) + while (!stop.isSignalled && !stopChild.isSignalled && input.hasNext()) { next = input.next(); for (int i = 0 ; next != null & i < len ; i++) @@ -102,7 +102,7 @@ public final boolean hasNext() } } - if (stop.isSignalled || stopChild.isSignalled || !hasMoreContents()) + if (stop.isSignalled || !hasMoreContents()) return false; } return true; From a339aa9e9811723e52896ec3c96395461cad0fd0 Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Tue, 2 Jul 2019 10:10:19 +0100 Subject: [PATCH 24/78] Update Netty dependencies to latest, clean up SocketFactory patch by Aleksey Yeschenko; reviewed by Benedict Elliott Smith for CASSANDRA-15195 --- CHANGES.txt | 1 + build.xml | 3 +- .../{netty-4.1.28.txt => netty-4.1.37.txt} | 0 lib/licenses/netty-tcnative-2.0.25.txt | 201 +++++++++++++++ ...8.Final.jar => netty-all-4.1.37.Final.jar} | Bin 3839841 -> 4024948 bytes ...tcnative-boringssl-static-2.0.25.Final.jar | Bin 0 -> 3108312 bytes .../net/InboundConnectionInitiator.java | 2 + .../net/OutboundConnectionInitiator.java | 16 +- .../apache/cassandra/net/SocketFactory.java | 230 ++++++++---------- .../apache/cassandra/security/SSLFactory.java | 14 +- .../service/NativeTransportService.java | 2 +- 11 files changed, 327 insertions(+), 142 deletions(-) rename lib/licenses/{netty-4.1.28.txt => netty-4.1.37.txt} (100%) create mode 100644 lib/licenses/netty-tcnative-2.0.25.txt rename lib/{netty-all-4.1.28.Final.jar => netty-all-4.1.37.Final.jar} (56%) create mode 100644 lib/netty-tcnative-boringssl-static-2.0.25.Final.jar diff --git a/CHANGES.txt b/CHANGES.txt index 2faca24e20ae..4d3a9a9051e9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Update Netty dependencies to latest, clean up SocketFactory (CASSANDRA-15195) * Native Transport - Apply noSpamLogger to ConnectionLimitHandler (CASSANDRA-15167) * Reduce heap pressure during compactions (CASSANDRA-14654) * Support building Cassandra with JDK 11 (CASSANDRA-15108) diff --git a/build.xml b/build.xml index bdf5ae2ab08c..acfc6133c77e 100644 --- a/build.xml +++ b/build.xml @@ -548,7 +548,8 @@ - + + diff --git a/lib/licenses/netty-4.1.28.txt b/lib/licenses/netty-4.1.37.txt similarity index 100% rename from lib/licenses/netty-4.1.28.txt rename to lib/licenses/netty-4.1.37.txt diff --git a/lib/licenses/netty-tcnative-2.0.25.txt b/lib/licenses/netty-tcnative-2.0.25.txt new file mode 100644 index 000000000000..261eeb9e9f8b --- /dev/null +++ b/lib/licenses/netty-tcnative-2.0.25.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/lib/netty-all-4.1.28.Final.jar b/lib/netty-all-4.1.37.Final.jar similarity index 56% rename from lib/netty-all-4.1.28.Final.jar rename to lib/netty-all-4.1.37.Final.jar index 058662ecc9f4467eb2d0849ab09351f0bae68616..93cff04768e6bb8cd51fb194c2ce487947dbd97c 100644 GIT binary patch delta 2000135 zcmZU319)UxvvzFTwrxyob7I?ePi%H?3;y?5=sTYCn2zwiKxrYr{z0S5vD0|Roev!Q5{Ow5poMh>tAV?_zQ4txuL3mj{2 zE=gLa!$_`8HSuLMW;fx;U~tyF+2(GaA+mEExaoR2Qm`Gnz8SoUd^teJ)i{sDm{@_1 zk+y4NFh1waSFttgR#$-8f#nKOVj^t^SLUb$i2Fn3Lrr@V?wu(0_?J z*RJ3w7#ZP5v;~pflWu!bDp#7^08dUS5j5&S23}q7BszQCsx+! zn3iDGb++)9VeA_FQ)%UJhd&-P4P+&NiPc7T&Ounkk(aQ}l(zU8N?8sX=C9}$KtMoL zK|w(N#0>%i@^{?7h5vcMf`99)~sO6|1tPI4F5j|Y4!``|DW|P z@*m$uAzU!PUtlnQtba?>VrA%%fGpr}5P$6=@GrX>D5QT)_P2N%iW=B^2no#lZTB}< zV@PTs3IsH8{eJ|f;QW6yr{K&e9L)b4+TY_@VX4zTv2a8Ex8L9L)HWU{;CG0B=HG-6 z{zHTWQt%HjhUEDJav(YXfWMt(gZ|su-<(jP`2J8(f)e>-`I~kc2@4n`uovnd|I<)h z|EoeN27|=^zfAwVNmOZ|d9bwqP5Bg@I_+kO2kyV+B#_i;PMf?4|1DSg%|TfO90e!{ z^A86(82LXOvS76T5dMoN%M}Xbe~)E_<^E&&mva^{I0RrG?0*8GPWy=Af=2kO2>(jP zzn6d&mL?5F8Sejc^n1I~SoFD}fy!|Ij5mP$XCZQatCaB|ApU#qSvbBwb0fp^{$VK( z&-MrWPombzLBIg3;QvX9f23H(W@awtuCD(|f&Y`;)uT8t|C;M>od`N^n!g=8;(yC2 z5E%b(&_JO10|F7)|A4m6p&Q^fSlhxP+V~T_5p|LVgIin{?21$c-%BYMl{H@FYes`uCc$B zfe0i?$mkFw<>@Bs4G=`JYAGMJ7#4=z{!rE|*LG zU5BiH*4hw_6v&N&{^x)d?DucMJjcIt?9bYwu>P4u1f4V`p9~7Pi30n-Bc@F85P-<2 z|GQ0Te@W63P`UrT>3{ME3>RpRiuE67X)Hc?kieSXWB-ouH<`aAu>abpf6F1$5~wgB zfP`rOO!b>h+E*W%e`n+0vlyar{9*TJO>qAlOhe%UNzjqg>e9KO{%=Nq}BVA;{&5G{s}A{LjX}lbVJc1 z*A-Zb3<4qx+{NGmz~qtm`u%o$&4Ze9K~}v%vIJPkY5n zE56q5wwG6Kg%Zny47AXpBxLSRACT#dUr0aj`%a_48v(q3ZUuAeY%sItj@&bw+&lD- zZq>k#LBMU&SH2rg?z5X@bi?C!{EfF3^li62-Of8}B0yFeAUu5g@W^+=FE5K2XQ}W4 zTHuwG`(pqw#lG4zM2z)-d&;d3*( zHTTj!x80sAlEC#KDmT}UX9A|1J#4Qd30L1%?|WSv2t+!J<=fHoQE7dDVHa!}`I4;#`aqf;cL^eIGvmPIh42FkjJaQ*MlHy`sUV z34YJ?7-QNxm0sd$3ZRcb7c;Ix*@M^iEr;WRfS)vfAueBkWNTZgnjE!7;qJ7-DkAkB zus9#Emc3D9i>lVXM?-76U&!A61&2&A$-*`J6AwEu()#oSwaZHv5K#CjCq6Fk>iZN9 z-`k{_wZ9!*#;CmFl*QbUDi)&|Uxu}`hGof}ucn2>WIc5Ug=H;u2iZtP>HeUs{P%{a ziwXp#(_QP+dvYE5UaV#9Ez4H!(%IOU(ibAYSYosFzKc{@)8{?A<-J6l?bG(R97Wtt zMOSmnYw@U2C!a~#Bz3+AQy(6&#QsEM@1i0zBMS6+a`2;Zl?D#HC7BggQ}K*X3W04M z?6<94sQZw)>Y83Gz-fw8Jq6~*$oe>O{i4~iRtuPBz0{e~8LXr#4*N4iR|;i2LXz)* z@8YKQC*_I_iG5}!7UI#=5aXG;G*@=NB4M_7oH%OZ%`B+y4{-3Ajg(57&2*U7C!P{v zZ0oyp*v1==&~$z1GEIIkodwKa{)$8}ypco_UYa-bJ5DLGW)$dB%bGe7CjFF9SZD)f$st%x|63U@_4k2%5sq&P;+?Y6)m=t1OScTJ{+;a`fk4U=N=i|6z* zPW9NvbkT+FXt{X_;X^`M^Wx+r2dXH<(cDu0q!O;oFuJ&08jylAjV$^V>BW~}lt39N z=9SclmLxInF|AnKJ)O*DtlKm!Tv6uP1F>Il6g^m}VSs>9PBRW0;C);2QcYC_=-{DM zciua|G>!Df<1g>@wz85A^Pux#AFHT8k>~K`MbNoSFPQ04$fK09vKADz+?*2OD!Ps> zEyM`vy7eMc$fDkJtNhg0?TxDl(wS1PrBlfeO&ymB3w%oO6x&d@2xF%Y4^T;ENb_!t zXL3)Cu=kPYSCq}5rBpAyo4rf~EUUVyXGH#NXpeSjV9#Ok6F(?*nuG}R71O8$wE>UY3JT97tD-F;!`WN{R^p#4PiPSJADB z^JhIk1xl3k&89^Tlldfl>mziFOqQPOvlB&{dw@cp#}BGJLCM*uE^S42fUSj1)D4EF zm4nhKPE@&t5z3EZs+W{V&p{2Z6*aX9tFU%Ja&}GkmMKNs5YOiy)TnILsn%ef$PV5C zbMn4XW*s{#Wg(sU)nQHq5PH&uNIl+UONtz-l^gwn$Ah(G$=msYc49^CbsF_HM_8XO zR59cO1ZLo}vGnM<{kW8R5qP?N*a+oU-%yM_*|qfsA=bQA)YP0dwtDu=iDI7waKmpf zE~$xUCZZHmU|eu#qY&gQK1e5kHUvJ$?KJ!6RB#p2`Krxxkj_H@zL#fn+znJ#`)ST8 zCNXGb<;+AK+BUqICFZiBp|{?#iLH@xm75f;h_6ie+Aj~13d*x zmxO-xjJ3nZEN@`~pszl>T}+EMa|_}KmP&cp>LoeYeeuqlBWzNOMLjG=mjcp%o{t?De5~WcUAQx9Y?&@iCO3M~r$rQx9(HX>-J?)3+TJ7`lI?`ZZ~WQyqr{_y^Lj_2Hb6oMuUH^MT6MOP#PK>}{+w&t z>5N_=@GWLz+*crl?lUcyLfUueaOW?gH|iS=zIQ!fA*HSu=%vq)Qgyf zFyl`frrw1ofOCDK|HcKng|O`zzulQkrVA}^?GYN4FI`bXSv#ZfM8%nEM+!V6#Jl=< zckZzILnKoFkVc-xyI!1~>aaInA=LP!Od;f!v_0>`Cw%WOvR~SKkQ9?u0Qx7!D~i1y zQ9eJdlfMs$B)byT);!euZiRAkxa3LMj&ecko-oi21H?=~&j{$m1jGO-wXfmI?odR~ zv^7FQ!WN_YVoqwlct~{girFRe!lJnV46QxZhC_m-ebZK`cvEblxSsqJP(J+CVD4S! z3d10W4H*NpXL>x|d>&?_8A+*Dde0DkYjPoNZsJ`bAyM3Q8suHd+-m{4R`{s_CI1I+ zb!SU{KrkNU&$}?LMEpQ%Du0v2rLK^LOgvdOyhoin%d_d*9h zP&q%j9$!0o-b3K<}2}DcPc{_K;hV2sdk2Wf+{FfK>v__^qBpiXQ0>dtBCzc**xSV&et1*QT)U?o zA&J*21b{rUyYPcR%NwFM805VXw+Z1yCirn4e#`clik{Rb9g*ET&BF0we0^o6UQ1sh zz`FIB!&~g>xXpZulo&ill%+E!@97yY$I)1dw~Kx5#We6yV3kgW?-$q-D#)5p;CQ39 z@YQ< z8?eWH2_Hl_c{oT`m9T;SIPyY0A2JG5fDgxUWD(l2sA@JShD?cz(5={8Dq6@?pf7<} z7Pw=JjP!PF4`)|ns-K!-=lTcAmUYV3s=(EH!SgK1i;eIPV{2Sb4)0mRkt9V&2-+vRH!rlp zVWRxY%!rPHnaYYmQNjEV@@1PyN!IH1-124I3N@;kMs4lez?)>yA%FESTMl)Mnh9Ab z`6Iqso-nVCd`?Fbu^$kmt_x}y0r0Dcof?tkl@jM)499pQ90~RWvEdBzwGR2nvg132 z=J!j%R*2aR;mt-n)YpI{3-T8EIQ4Lh$9i;_xu!ncFMjKl03?6pRJ;lD z9{QKM)aukw@@+CO_ZPFP=6}zbKjwOXeUIUMs43v=44$t}r*>U48^WB2q#UreLA$pY z9rw(<^GI@}#Z<(CS$FLZX9yK5=^tSX-B0bWk_~li#>lP5aG=BN9#G}6J8ewC67p>< z`OYEtA_@P}@_BrO;qVgc4sczUoY$0WXiNGbB=Hmf~+$x#hA0LOZS#g@?$&t z=`ch>vGt)3ZI?Z@AK=*Ls(U(p%n-Vr=UWw77Oh}wWub7GVzPj;aAgphky=h<7WOHcI3|g>Jsh4s)++$gW zoxxX+O33FKc70K8?9=5f1OaH;)djHu0<%|vUk$`=ktQR|)- z!oX6JXo+L>1J{5_#xZ)-;FzGu`1_o|;-y9!BcV^=(ZvPsf$eGBO-z>RTL%(LVM~Vy zHS}yv(=1I)wpRM}1*1zd>*ai+Zv-B<@LogAyV&1NGM+i{h5(M`<-*}iw*pI%@Hi;U zMj-}P+$tsM>JUj$LCCaR0<|m72F^QjQRs#$8(umUy`IT*0!3ao)vAmHQCjNsLiOsm z(Souy>6&0<1q@g%H12zv<`u1KpQ*sF41D-E#hEjqX^U-m^%@GKl5u^+Gk$-e`pD;y zjf-#*4~vPn0Y>j={X)v&8USF(=o&I*t${>4e3$i#7m3Ku%sE#4N0E2)?C)LL2|tCR zTjr&*6_@u|>??qd;IB4!bGz4}ve)#w@5IVlzX%qvS>=M;Z5r+-OM1EVz_?0&X@IWd z+SXlm@~q%B8rdcpouAy^w5VX28-RPif9b)n3lQA-1rY2#Vh+mTToI5k`f<>=MA~=6 zYT7CmRFf2Ys~bG^v+~Q7uS%VFRPWJPaLsRV?~F+ut*()l0r=Mj%zzT)iR1tdy7ie# zgnT*NL99bO1e$f$LIl*6TFN>^^c4@Z6_4MTyzY{!E*5R&5oP5OZDknN4jRU88GXfp zq;3-+ZPXFj`=|q!C20hYzCs4^Eg0h4m@v(XP$u~LXV`Cy_c*!V121d7@~se!r3^Wu zhTyO~{aEz5eTR&B_BaGc-{@P;FfLv#-#Q`y20J&QnAJz|Ilb&dJ6VHAAji$FjUNqy zH?{k>EbDZCT>Khl#(gni?9ole!J=okQrlZGuB2cooix$mjag5oxGHM%BkkGlB%pq$ zeCoGKVE;0~9e@Wd6x2(L@$tSWt}95#?T5fU9v&8iuU)t%oHt3~P8v0W}(% z9_Shv?}}_J6d6f6YdV$EeTCLEF*NTgQOQb7E?cv8{U*g`^#OFZ)WT7CprUJC+*UM^9;Am8d^1@q*H5Y2~WkL@sQ zFpNQmWlty{77{mpok~TV9&Gi{B zsimtyGFS+VV}?EH@80Wi8m+(w)1>`4`7=MyN0dt z(rvmlg|&nI2?gv^F4#Y*v8b)Y}1?bKuJ+MvoK43v(&_vZ<8TY!~@ zFRiSO=ZvbE z+oNQADb8A6k1CuRVNeqw0CppSjbm15c%RV>d1>}5gS{D6wB4K#G4VB^IT=>9J?eta zV-EAsRmI@4=f=-anogA9bJ z`W37FJp+X({b4#kf@-Ect=;#hz3aC)9EVKenwqZ9+wc1GKgacJ0u-Q(=Y;%5{EFA; za(^VrSTtPx>MuO4%JAPA=jgoJrYBrvooD^L${20!l|!Q{a*>*Auqt!W2>kdu^sAos z*WjtPu}%EC>dDU#jc`yx#3vv;!JHB);jj2xqy>wLnh6hu&1m$Hh+IsH$(y+MTxt&T zvxjpChj5El?1j_gYCyZt`q^7m6EW3uJx)k3zO-{K&*!&HgpA$2j6FvT$GdITEr%qf zACuUCK$_fuFY3;(pmR^?Lq(Ap5`0oVkhsZC>5aUKW8%vQj=O{;ZG@IMi5v3y7}w}L zv+zHln~`ty7#&EvAkV3_p8!LF6JdhyDA3-e-;5mh6MDJWzyQtc>!WaQ5zYoKU_AqE zMug4ZSd10`@5juh#?{1obuwHz6hDd7Ahaa0;76U!4^)gxG$vi_lk3G#+3@rT`5`A& zKPaexXj;A{2Nm8?xjeQ9Pzlq5!>kjMyN;Ua{pvyTs3T~0s_H! z-hT=c{qM^B)2#oie3kzyU#(zhGf0TPE4e2Eng;3*MB%{H1cUh7z<7W*5De)rc&L=w z{apzJ1f@BxzMLLAjb#)GKaGi*4*@Xb4{M|@G3WGfyOp(nXn__+fubme=74~L1WPJS zL6Po+BW^4W6%j%~wW|t>dZXvQj4uF!d5cw;umeh0+fvnW-dtPTeqO&-bv_pKrtxd5 zGlK&C)8kJqH-qiP#YK(Z?_UPFA1}LIAS~(~eJHG95$=ukWN}2d_tN=Ng#c(`nfnt_ z8RDU&(?rp++WVXNB78yB(?rRVQ;c)Pc%u8!o4SX>vMhZ%AKbzgXjO6I@vC#JUxTn1 z6AMV|9|*H~Yz-%gm#DlZU!uD%P{D^t#1Kj+Nr||$5I6@};KF?eBgvLg$PrkR=86Oo z81DoUC?z{sdBgBx3bg42hXJZC1Vf5=LAu)TqfoTHXvgY!z#& z*b|H>g#zt*=+<;u4OxLgb@`%|ihCN@{i1A>c|_LW%4v#w3)K91fkM&A$Ef)s0-tP= z=IP=UJ+VpG$S=@}m_PIjlkTeY$fS3{;mm=Hs*4t40w-w5D$E^v zCN0no=#0tRvb~SWo8g;&*?+DGPe-l2Ez(qqE2``?rOQcS+i?z{VR?xH!WJ{npn zJI?NZh1jrpPkZhlecuJ863V|14|^U3{~Gxc_Phe_h$6U+^1(4k z&17&8*kZ*cXH3@0 zkNOp@j|sS87pKPhsP5IX@KW@?zUIWMX+)Q) z$sH;G1PvoNNeWhuJ2VQ4x2dcmI7=ERcj9pe6v`c;b(7Pu&QVIKq#q|0(GrNo*E8|P z^#k=>1a<+bNX|+%hzHrWa;u+43yWdUqINg+7@Pg%r{HE_8>f z`BVudM7#<~0ZHb`T31Q)k-f!v2lC^&67};|pD;P{SDC!q<))p-*b%%-!>2?c#D#g& zh<=p&fM~>JUyDrXJC4eVX{xxx&*+Cn5X-2DDIHd-*`OvS)qlpevmgcM*qy3NWGZDSjZ2+0{ZJZwY` z4%i;t3F@88ES-BbviCMs@UE+`Z?9cS&|;0qAtbDzpTTPrmZv(L9c(I~Ig=P4hZrqM){H2uoi*Si~n0XyfA=WpSG%e<8L=Hy#Ucj^^u^5nbmoFOSDbr{me^|1zon!^2x&{Gj zfl*?x@}eGViLAD7cE7I3pky9SZhRv8I8`xASC!9G=kw|PT+rNJn*=2~yi!&)BzriU zkpzMX+?fr27Fe?aJ2d6V`ab^47${A8gJK%2R zjLwb|igxxqm3^m-oUvVgO4$CYY45Et=`^%VAx~FN+vV4su5dLJ z0gVbPRf(Cth9;ZGbGRw;WgdP4Cx%9dzK+32!Q5Qm{%{T|yP#Agu(X|WF(vp;5{=9X z1#5$hh_ki#t~ds0%IT*_WNISHBZI^K-HUWbGaMdO`5#k6Hk zhFoR!sB}gf>U_u%)?p$05`y}8M;J}0sx-)|acoBW%1T{>X#MY!#?^3KTma668U#&N zX}E>uWpyp6Ol2*3+j85=s%kL^^qPzc*nB7&N1IO)UuN{0`-~RvKkb`9B_)FdZ%HK_ zoFWE}57}hW#~GZrmZ>|!FwvCTnR+=54{ZjvH``jFg?z`=E>rCG>D z>JDQ(i@XSmrY{z&Ui@S1~~WXkbBxLzY3 ze(=MI%QzY9t2J*n&RPL#su5m^P~*BlOru*Ri(M7!ItyHa@Q~3~kFs?jjF24ylQ28Z z28TS%C2^;50zYChbZxh5n;#&OZ=so+*S zs*b*rSp>WmhP59;VEW3Ri9L#eYuIFKZ%VhpXcDXG>uz$yiRc54n9&ptR!euHN@u0= z`r~4>Iea}nGRk$~Ft`UX_p4JF^x1@0SaK4`P)S&`ajnWrn{rNdlw+49JkQHkYqkT}<;S^!T3*wmy5ik+V3R7O#)^W;-OLEHI4dpAAopVwrv`egm zx;2W^;gTjNv1%gLi=EDl!ptttn|Y^*PsQz0WIoDBmNIIhuGq8+!C#{A{oS{EYRL9a3;~O}G zfuTs~Y{J)HK5?>gfE@7?}9Ys4THtJc;8h3jWy z6q>?cQ5%TC#Oj(@XsYKAQ-v~^zE&b<)jZIw%(w-xd38FJVhm3g53;M8?ArBN@$_*g zN33K8e9mtVGwZJV=5X5Y^uhP|b%ov|Aj@XdV?rY|Y4T)3UPZ;Nfp8JeVo`i0Y<$=x z(?QLhAO5%{GD#nt6ao(HAy8ItpHN=#VJTnsEVZbY*@olbLHZll(b=5jS_~0tpwJ~= z%r|X7Yh6JgSD^z2OE#3mUhB)E`;wkK3{!cLVA+<{Q=o^K#%T~_d4^Nw0?6p3PJ0NJ zAbo6fz~Y>&>#rbV%bW} zF9!Wv(-PXK8b9Xf^j*9Vy8anC2Uf+<+*)P|P4P?PK>4I@f+P~RMQCSF)dX)hJ17Kl~=S1YT_o?opJU>w4LmLK6(Skr?k z6+%(TY~5*^A6b?tPnngWm(m0+GJtL$9!A1=V%|sDdWKmL7|rEhSIMAlyZ&TjQs%)X zGwT{3H~B>ot77^ai2e*099K%qHk-HwK*|nTwoptsARrTsaK{;``#CM&)6MDDNo7|C8zZ&c(nRXg(i%SgYVml7I6dcqBnIjb&SN(wuQH|FWi0(S!@@^WRttF@4&24qP`NW1*p{5L83#+Q7nz6_U}5r_FVEU~LQC}0nPHp=fQ=WMuQt^< zxH~5;sP%gnjXd7x3UPL48E9ZoU#Fps`!AP!^mV)4iwc%AF{l8r7n!o=obr&;cD1@R z7#T5qx~RlC;mmaErB$Y|(Iu!WJcjAnbNnFq%EUZXJ~y^0CS#h<)nC5RaY0WbaK=dm zaW*Q06^)}c6Q*%2+Hk*j+igfmE0{MZL-!57%$R~hkYUx9)eOH zDIxu1#@X=vo;v#Df;1|*6Ic=E~0AdF2FgFTf46JD@I%0jKa)=AqG_ zsG5TrE;`{DrE^Yt&xRv+$-(ke#Sc@fGJJAnr%gDju~w}IKeK)@7l*7W^ip;voQ@e^ zXI+vM?x}>k%o?64yrr~2AbvabT z{c7^v@un%%E5E3GW4jz0upaZrwhBl5(?^7NoT+{a`mUJ)@;f!ZjhJ1JWhr>Wua6l$ zJKs#BK=dMK0WMG<%Speo9$I``HjA(CvI>^`j41yI8Zo1jaIpN+{0MX-Bi+EUoEWb2PN1WYzq!$I_9Vq0#`m`#_q~J6+gqrMY2xW zchnNSaV^9bJT@}4VEhdUG$Ozs! zw8E($9Ea3s^nO&r+=tkAa%usg8s1#Xh%d})0aMWFclbdMj@9GuvPu5*f?K7l&pH9N zcLnONgi|w56sy69YUMh7%G^E1yQ;s>3P|c?rguw`FQ5+`d*`GXIKTo-Q(#w>5= zH-HVl<|l`A9nYHZ3Uez#T(|SL>fY4AqUU#onzW@XktZi=N@>KO7~HBW zqv%9fUrmpFZLs^E@Jf`pR|N+LJom-U#ol;=Y1XyQM@Z<;Y=SfEvh^-g+yRIhiOUkw zB~Ugp+(?-QdT|$$mR1+cVa+F@*v( zTe?L@JbUtwRyxyeKF7(rtIb5GImiTbyqdkX2!r4MPu{o>?yWZaCMmNEbxlibtZZNk zZWhi-6^BV6F(hEI#UnJbN2nFA=oO-6=3LQImi8vQv!z^;`0HU~j1mAlt9|)P3dBQh z!z?2!$WpFELn58_nhsY)#K4rMd5&gLRXE_YFhbkdg5pvb&YGDT{hT6eGp*H!)leyy z20iHyR7l5sLE{3*Hr9jksq`MU;Hv`~jcnteHBgy46$Aq_C1T7B%O%sfIX# zN#VRo){SZkHS+j!^$Y;E{E6r^#2kXbRqke&!VNdF%+dt57tMgGim7~ptgm6|Ym*TN z%4%5>$wCU8s&(=;{W2uj(@Qh5oBiJkP2g68M;{^yCHWKjW(=oQN^X7Zc-NDDA@glLfje`}DJj0UMf@xn@og+r!#C z(M*qUDXaFg=8<2mLspFAE2-Ysk`ec{hj02=Lt&8=)(diq2vAa#*2(rs?wcvkS6=G$ zHV1DXYrjy|B;5jvkZcKt{3r8|Rk}i@XB4@z5un}*T_K_T;|^sZZL;HzGZ0MH+hK5v ziTc8SKb%qtz~)RGoHe7A>T(w`plu&E5EAUMgTe#lK>qHW?EPV3Uh*1((a-^L5v@H02|FtM9~r zX5;(I$5J$yBNI|B1#*Ue(N?*gjxns|Qli7)oTB2UMg7-d3nE*6_a7X}<`TY^kHV_d zTy~#}=->xq0uAu>YE!WpePt$YgF z35naMm5K(i>J<)Pqv|ew2!A~=x5vxDX}-+G19pWRn$)t<<8GJODsRR;zIyPKslYxU zE>5suSMhuM9MP81tPQp}hWT4adV=M7?Cq%76ZWaZZ&}ctEMZzx(-px7di5filtv(fGy;NSYAU{Mz2q+ z0w|qp?_GOJ-`ZAA>rGo=gh$iLb`h||aC=rx9vGOI&>j0_NN;wW{Rfgwv>k54X#9q^3PkSh7Ytn(|w&2jvjg>2=ysQ(Ll^cu_Wr`18u(zq(L z6!?19m+MhutyHJxhQFy_-fR`Gdyk-Ht(2Z@^v{=KVq;<`FxN(mGGyXS-%ECO;Rdk^ z_V~e_ky&!B*REjMuF3XE+3FjA{tU2Q-B=AZFL9lKEBj6}P-Cmg1JDc@bDR7w1$8;c zv}}OV(L7OFV&vd#eVU9Lq59n;6f(3mU6GD%G|zwso43*m4XIG6gKvI3%0l%exj(Ay zcb9tNt_(YcOf}m;Vo{7aKZyeq8m}f%+fk*!Dl81k#EZCMmS$uXJYZIC_Y8n`Fu3zp z0-qC2In3r$(e4JiN^f8M-GD~2%2sd%%)W(YkKj^sP5z-<(OoU@AxTS-ntO=PdxCK* zbsm=zT`b6E^F8uaQo3dO${P53?T_%S2C@rXuFh9&V7wpmVWkxacfW+PIvfpF9-p0t z*Xc$%))rRj)%d%YU1vqb<_>r{z@4Y2YmpT_*yL!2(qSCHcD~_ID%^Ce=*WQTl ze?Q`FnIk-{r#edk-mA%`m8WnA&W)RVURvI0>rCdV4eV-buAkl0<88TGW zS=dR^)0%G341cVWqoU$rkQmtm-0Eq@qPXbnt+p0La}p#hX}6AXl5GEFyC zwv2t6<6NXQWRQ{J$cXR}20EnZXn`!6Z>be-^Ik1RMpR`LRI07Ejb`98I$9-n8r$d7 zK6!>`8{acY=reYOt(H4{FCyIADfI9)ljo8o!SQ~qYpADf4FNcC>i!HrU6cz_tFpB@ zQP6kP&_C$19I3b4ywtePW>8NPzx-Ld#JTLzT@c41I2X#;<)V^L+A}uS3`t{c9$Gd{ zLpSiSrok7Bwct0g*D_gKU1VQFmJ`;s5ISG^O5S~#hn|vQtxY~o&YW23%Rh<&!=7oo z>OEi4xL!Ext_t|7=u3>-PH*Py>78{Ak06xV-4FJ=YghM~ruY?ZVy0T8lxc9NwXQgS z1`ki4JuabWE{qOsFIVbHF_NU~RL?+_WAVjLuhbSW1~C80A^ zX?H@%!SCA{WE*tHcM_uoLT}igJAhyZ`8w90H6+f^MN=Yvgh=Zb`&{gl^1ZgCyb^D+ zAGuvudb|RNP!3a`6?M@JwXe-2Z`)&eUTOGVdeyWJv149wNe#fnu8+=}4IL2wNEY9WRLs)QEq`Q;6VYKQMR>HHZ>zeYlA?josQv6qp=Ujdjuv|CYXJk6mox9@wOBWHC09|C8y& zI2^ZThMk_eUW@)dfZ+8asQdG3N5S^KQ$G`6hBmVeK1rtl&{>b!PCio&=Vzv+ETagI zU=C&aUYny`p*XRFOD8}me&V@SG1deGqz z58gNU_6rEa$Ns!s!t6fJ9evDzi8E~gxO^D%>WdxW^e&$VYbeQz@XZSg!r-xR-xpmH zEM)_hGV1I*d^I?1_yjSKczC4A_!?r^3^%@3;3ARnIJE~Shwhz zUl4iY>oazcA1oU3fDH@6V{;#%n{}lJ@sZJVMQpH6=|&hp~qg%TvDmHa@GI zfn{pvh8Pzp*u6*C(kbEYrFc_jQQ_gC1xJ(e!}n8|Qpj1GONbgAy*1bxZ6fTPHOr>5 zM^lvw!ioLMM^pO{zy?>&v>o!!7>xP`5}iYCJ`G0*^&JKNf@rnarWyBuUdYiF;t!~; z;8XP==`%Dy0RNRGf1T_kQaC=NH30OrPM8F`8!gb?1LCQ`;#4fhV^^qSUc{peul|?+ zr4xW2EXHDAC}hj64*zS*SzoyYj2Ubj{*hw{0R%_-m&-s|0B9qFb>;ZDOJ1N{lTOp& zg+9s#B)pFC?zG3;y-+$#EB?{_U~S{76b*k~A=@(pSfUE*K5DNbgretH#+ViR2vi%a zRw!@NSsp+2V_Co4W>iLeIYCCnZ4_zS&`iClvQ03~)l(t+EeCBjTmH!bJpcK}PmJLL zg8YJrjpr6G0QbPlb4&A|6qK+LhK`|)10;K7pIRH|Y4?^r#ReD}P!Z_QxF27dd=nS; zjWP8ip7mj{t7h>0U@1tiJ|yt~(HF1aYeB-Ij_My77a#~ykb=2{VPn~S91H%9ts#)* z>sUqo+HrHuQNG56<0GLs27$ZRUe#nv1Wsa(W{VD^082cEF9`fxM2d$M0*I#}>MqZ> zc)8OZ^mhgWzD9y^oL>)canM^W4Eud6Q=UCZ7-*uN$v^RjoR9{fr}z0>hSm zj}0-DY#VdvsklxnzIM9e8P zGK^dP1aSCf4nI>WpmRP=5PD}CRz}Xu=$Z066|Xu$Wk)UnIl=cykZ$h+hk+?=iGU2zK(DHk@ESS*leAkGUcS0=4(Dh%Q1~N{*(g@*x_JToBZkjV9nx^)X{QD{UXskkmh+=4E3?6;%mPjux-&!fM zT$H|R@w)p;O>I7|ORVx17^w@qCJ2bj1#=-8%+%u~qLcG^J6fle&9c>+j&R6eZC^-H zZCtcCE4w^NJz+$JhG>+Y=p8gn9S&hgw8SH^gd?%Uqk(*CVkG~UNS5^Y5IrF{I4>fX zHvmrh#E2**ak!fhEN_57^$}!Uu2M)eODEbu)$(`tRea-~mSFbu~QxA6M@bU0K*|i&ljd+jhl9#kRR( z+jdrL+pIXL*v^V=+o{-2ZvA_o)Be-$+t=pHd>G&CePE8>_YIquO4ZJ?&A8n4yuX=? zeskdY=JfmxPw-c4stpYh7?-`sU#P$a5?c}1d<=En4Z;;LNIH^+fICX8!B{#a82z`*YfE! z?)cwuzeKXvS^r^q(q51v9bK=vA(}ENHAmMxrz-S3hmLbmm!%!j3mD|DajU&}zK}eN zBt9{@ZyI$%h;ha}-8Uv=hqow8Qy_OJu=7A5*Ju0(JkB_lD)}GlfQPqtmgEh=(Z=(x zmOxH=U%khzb&@BP*4jOk*0MrK2Ki1CPMyqVJO-I}1o1g)r(fdrPs-=8(HEh#+B+?8 zk|JMrWFABTm9?vjjq}U71D@?>WS5?9pr)+&Q2O|Mx94SB=q{W;Pu6el;W{}5{Rz^* zvxgs+pM%MD79#$Ec)ztoM!-%g=ThBRRdvCevbI+c@XApS8~aOi>Y;-0Uc%|rbd%^E z_9`)3JIgZ<^U$x(dSZ4+q)zbJk)&I*yrSWF+rU#Tw5*#i*?|yvs~6YOT)4w@scZ{;R1)-`7b8A$8Ub zk&%rzT9SFNC)+?*Sh&2;%7R_zRaT9ZN-~D0ugq3rYSm%s`w+W9!j;g8b$A}Pfb|T4 z=yM;f**}`QZw9%?z_^;QlIsFqcXG&&@asQ3@h$I{`e@8~esCK(DPk+nLpFb^O35_3j7F#PWY45!z4s1L9f32jBC`| zg4>9EK6*fSBTmNUN)~(Yv}tc3S}RlLnXY~PMVNiVee?8s#vaGllL?b!!7VpyV(5Mv zl1VATBv}7{3l({X;dG|OQ%U{ORsTT-TJPdHNXpl{!6XRvAZ>E zw?4e@rIT$u>u&Gs2^6(m!1Op6Qu{Et1=fLFu;N)@XfGVB6zc1!cIN=RiByR?ua!gI zpU1$}R>XMSTtdG5a1E{1{R!-0Y{qy*bGoQ*d>}vx=Yrf-dvc+ z=|@>WLmVZtN4VPto_!-~G!6MzE&Uslp_CEpj35e?im>64*umUeMOA6#m>* z)c>>*r+l(?j3Yy`_tXfRcb6;u?@$ig!=#}c{_Ig8re?1P=3Cmraw!8ot09mskJ?Dp3Uh zWPmH}y4W)#zPNqw1m(B3Jb}TtNHt!Gsw0sBTz;XdL*TJd&UoFFE1)?+V6x@L{8_$g zKGKyC#0^BY4k1~B=>af$-I$B&e+G{1F-GhHSYmrfVP7f>gL@dvuAjsNt5Bl`tm;Io z`i`|`121wxZovNfcY+Kh8WigeYg0@M^=+c(<1OH^YCeQd2T)mnBHAq;pL?mp!(K8QV zh9^nEGI^96ud=(d3Rrp{yeLIASaq*rq(Wc79LWWHl+jmPz%fHYNlT>5hi9GiR}rAP zhhgc2D5^<36kraIaTYU6x`|EYh4WGv=NNfnEsORMjk5<~96Q1WR@kFN0fikkh-_Rv z!tO^aj=db2Fr;ggQYTWy;TFL@H2>qQ7H}+$e5SiOAgR%geKCfzuOl)b^(Vn7kX9dv z-dYkK@kg0?lwBeFyaJ_CC&sABIs~xaE?VWv7h$VE*nDNM>RmYUfaDl?Cc`}V(3Sp| zn_PNBR@Qkq3g^;=d7e*k!s*!Xom^=*msGuNe8B37u5PGHvdYyS<7@*d^5IgV&L%H? z-w$Q{EBes>-+aTUx-<=NjKi?xJroK2m3y-XKW&0?J%L=w>pWnkAXs(7W{~4uJ7o+w zwxsixpLlv#TJYQgaT6KmOxI13)GcMvE_qT#J#49n<+!((JuQV(UP;%Dd3OUYHi?%! zYCznK#yL-QQ>B2%5WwY!-u(k1`f(;h-B>hQm?Vq#JwsEBSLmTz~p903q<$ zkZ$Rjeq%twm~JWsDeDD0vP~iI?ghY4xR=n+kgMQ11mC9C#~Sm?2`$gxlbH)Rt*yAD za;4V!&6pwKkYOLKu3DR@C=PfWl77s5+cfKg`T)n7-tat%J-;jN%$+wee}X;wPI7R; z0^AWhm%DW{JVzg252>U6s}p2&aFufKv`3rIz`5 zEmGgjQAbh726Me`X^sOR>By5GB@ zG0%{+_FB@&5w&P9f9~L%{|5-!$?ONF0DK5CaDYShMYVLzm{&{J4&fddzSkUk|8v(6 z5k?{`q>vhgeZ_`p^xU{zu3s z@j2JH`D~I-72bEwg(unKD7mHP+mLLRc8z9FiC8C?w)z8x@M2)$x#IW6wmFhNP$D|u zb$i^B=y)WW8$WypC*#RDK*)7!yF43z7yI1tBcEJCuDC*eY1B!jh8smQr8Qe?kq>)0 zBD2OzFBPZ9aCzYRX{oC>uSF&Rzc9_G#Z0`Jq6@{Ec}Pc6kMa|a;^`+zv@^vx7GH>W zT$Ts8$1`yE_@jVIpbWzsYu0Mk_`jI_E-2AM+kCV6(Z9Dgov z7U;N#Ic<{t&5V3Pa;3f13lD;OPCk2Xb?^DXKt|ajdwbIZh*fBCT7%e zzFud3D9;p7cpJx#Okk7Hu82=3a#G3JL=Zs$T!vx-N^=I^=Y`T4%p zd@MW)b+}PW!qgAAGF(U6u6d8TMA>4yEleeGd^?JRNx^#E0mNy_Ge%yh>^@piOh+Yl4b{F4_pqZ|;v1Nmw&q z+={K=Pxu@d|V?n(yCQ63>#?PgoeFC1{IZSPGR)#Ij{iqX)1 z3TDeQr=7g94L|I8EbkPx@^a=33nm`1 zPe}f>C|sy-|aI%BbYBjhSxZ$%Sz1 zK=?gmVMf1ZD|6=>uE!&H-u3(o1DE|k$YJNSGvdp2bMY{rjK9M}SSZo)KO zz7gw2qp&s>wC1Wt*8EJ?%P8#ct``c?82O;_(eRThIGRmV<5I7}IRf>Y^L$bC0JsR9-LzdP?Wp=EhKXHD0U{v=mlgM>O zSRbx4>2`vfN4oFY+z{Ge5%q5FiL6lb2TBYM*XqBclRnPe}!rvLi8{nRl z(b+O_wCePOF;V2j(uu1kxvf(ucJFwM`Ca)}tbH=o4bP#+r~ib6G{*Ju!RR}IC+wHF z{NeT}>Sqz%ewayHyi*C#VO*0`eF~HM)@4?13ggr%_k=OXs7!}=Xrw_jK!>hmgu*Da zQ(_|ZDN=g%C+7H}QJW5l`gn>_5U<3$crGo+sG3oO4*g~Pan#$P)=j())#KR19z~l_ zCnY1gzTumF-N*Mf6_!;xT_?V&EeaN~EwcH8t`SF#vA)|Dr_N5G7Uqcc0lH@nUDEP+ zhEtJEFZaK6n8v1qa9uCdQ)2}0kl+@xjc7b~uwKAm)f(`~anwcIGRDaKwVnQzv=QN* zVm2AC%^tT=GB|RfRo^Kglk#Nc%XA*Ms|YV)=#ki|pG8bG16wD@+?+v4v@piZ550fK zq(|*;eLRL~8{-26bZGJmb6{loiy!!nXk@uEo(JNLee4)^8H78L46qQ%s(Lba;w{`s zM^iX_PrIdw9*lBf@{BJHP`C&0!!ppT$A`D4_O8nMX)yhBgxZLCeFDo!PopkdcM{8R zVjAg2cu^uT-Y45h#RbK<)eeL=womUd;;oyKMVi`0MtBSi8XfG*+pp2WzP?hdkXHwM zjI>b0@g>N%#;26o!nIK|soQEsVDpUEA22s#%M?d0mYrFnYZq=1x{rVTE0_e<5_t|= zIuY+S=;ZZ=&oDwz4;MS3Q@v86iWp@hV{mY@JocuM$t>sNy!W;o1%%$G0cvaV-(KwF$j{pjtxvBEK5mA}=sw zu1&ck*!$fC{FpgIu>u)hoK(lS+nXB;IM?55r9bn2yU`mU&Y=kPe~H6L2frYGrjw-p zLWnkjgGva;nh%|EhZl^0992oF>l-yHjvd(@8}lxr>bMx(_yTntA7yH|qG4;iAUTrcJ`2bDXiH3k0$AdV`G zOWoyIFy7_h-Z8#k4)7Ov9DP5={h#AH37k-ov?!lQSlWQ%>+`GClmNXnP8w7svWLfz zD+)puYicF0gA|`6L3+B7P`0 z>cOY?Mr-bczkDawLes71=ZcBzZ*wm4ePd?2BW@F+N~aAZ`e;D?G(~*s+a%dnae=_a zu`+EEH&+!fOU^(%Go42oy32`&%PE*+^X0%D^Ho!;)~VRkot9_srLGW@WiP3t z!0)!b>)>5H0P=|48)l|fCq-d)e;EKJa#f$D;q)1eWC2v=OsM)u9_(m)OtQU9Vo?zW zZFY#=oA^E*o9L+@q8 zHl7D~nBk|SrarrgkE{7KP43Z3`qDhAP9e#4IihcmMem)P|9tr2tcJXFS-UT5=~T^# zttH@Koe{1KG#RH8rKJ+C7pBYb51qyJ>;B-#!)tvb9>DgYH*yRpl4fj#ybv9iA$3n| z*E49Vim7d3^vr6HjO}OwIJp7InUUI9`ngqql{O_IH{9&Z!4iwp1~aUgpr=#M=+lPo zYFSb-o?8;@IF=`{74{UO9725sy0t7c-Im!aR6DzjR6kZ4?^m69EzJZ@dAw>xNw|AI z{i1Bj@YLA1XQj!(Y(?U1w0hrVHY>RNe!+%s3>oW)7<(hD3gujo$AJV4wfH`=NMo3c z8Cry|D_nanP;Ir8pEb~m`{+Nd`x zAliPn&ples{f<=J3_f&sufBxv{f$|6Q__!|b-3or&=TLQI?d*7mkQbdDVNu{*RQ9J zaTr&josV!YUo{(u?9ZlJkNz0UPcsp87qI204~7`lxz_Xo0EH!SkfOLB=rIMa>n z_FgW&coQDHX>w#)J07RGU-Ux#=d4{2Ytwpx_&Sv9nj0YCrLw7GNBP1|tb^sZ0;(Mt z5X2}!Qxae{EKMAx-{(f=XI>cDiE7LpJxG5n`LLzPFmVj*F;$?2YVRxY7O<#YyWL)O zHd^!=Qvc&ja)ca-t+nXlVUYv(GBPBdrB}fgOlLP%E1%?J@%RN_2h|fPkAfte+1EWe zye+h$T4xUgTmC*nS%KB-lKouy!u~iQ5Qx07a*Aeg>1H3Y)nH?PrOHD$YG=yl1l3p{ zt-i!+&Zz^;pn{%JQBlb_Uv_KIlXlhfuV~=(9BiPub;$~WUC=gAZmj2TsQna=Dt*uV z9eoj>XFdbB;HDr|mFK${`F`eC6p~Y0GjHT|xK>kXPu0qF-=Sy>m;T*c1nn%!xgO4) zpz+Q_HU#BL+KNXJ#4ghLNL&%DJ0r0kvoQk})WioYLnRC)y-+#`9ySPlB%hei4TM~q zcW8bDW3!y@W4Z8bRriL;Y)ad8yudh~>@uAR(jTu;!CXa~>VJXg(Yul0JwGUahguu# z*YzPAf}qJbd!cmoVa4_vwkN+-ViiD+1}U24JnSXkWt{*Y=|2@D_NaCy_NHEsqS@X= zzHtE)T14j|G9>08YQz^IvIyj^Q)~g9a`gl?Ew!J6Be`I-BmR?54yV;@Q?V7~Z);FwMp#?NB!^#7dUGR$flkx>aQXhX7EvGj2?4aIi=W2u&hio3Z-sUr#Z@F@RCEECnMPU1 zj1uM#bqjTw&M9oPMsMK}E!P9%M>}L2HeO~DOmmtT%3~uJE0@+HoHS6+O@n8Uwp)>J zX*`b-HqS;!TUy`emJw_Ey5jPoj$LHD`hy0~Dh&r<7A?nV7XlAHuKXdsC!_{Ujz8)r zU0!inuEvhmuGwEpi}g}j>u<^ohFitceDO&pdPnHrMt(=&=}Cz}<4~Kk3t`|#I6a() z&dsZ&X3En()<2ax|4qMpSK}gub?i!No6M0{)o*~I2_ZC$7ccoKOnebl|4%-cqZ58W85PKg z4>I+{k4yUsYW$uJP+1BJ@p5kkO#6>N{r_ac1Hj|%_>y#k&!6%3T8 z&Fy@_qL!)Z171gM!uc}}PsOT`LDaA4914tPO|PoWs37*&@^hoEXx);uhK>iEDz&`R zbnst4W8H~?cs5+W%OLg5h1kHVR*=(+$piWM8-`yTCJA4Q67DHr+^Bj~BK%N0g0>HJ zFsY$>q_7yOu4bN3Dj9(MXW|P09jHBB7`w9`51kR;5z2U5d^vP{NQGg# zA7=E7HL*fXP~Gb)lcs&Jau*EEy{SVI151X$QiFw$C8no#Atf}aRs~?5{LJfnb`43))YN{u5u+m;=5oy-**&b`6&dqkS*3xs4PAa1H#jMMoOzm zgzLHAxPgmBD+=f0g?oPpVy&@NI^2J1s!HyB=E-dpbT`z8VL0}tuR_i(h^45zjWHY^ zi#FJer7vU^@EU2zAAhydmhOK6y!*8@Z(`Y8UGm~feDiq7kxVtaKI~$TsqOptK)2cL_fjul+wc8vkB#IrHKD19o*j zD>QGKg&V%9OMaNnt@KvaroJZ^lr8X`NXsGy!lSsAcb`zcQ{9HF-CUl9Ds6MT17}YQ z37}gC(&Y(onBqo6a!DXI{pQb9w6;XxR5Vq~O(S2Vv%RC%49BZCX@Sh?W)BA)awdi7 z&y5bJ&r^NkfF9AlXDq8~`}o#vaY?)B=DZK7`yDPKS9Epumxc(cYKA1zcG-*H4SBT< z;43(nVh?#FgZ?M7@-;duA|-Y=5-<(ivT4kOo?Sq9ge4*Y$i;m9)TXJDL ziKB5azw1U4iP4SIYzcaUixo06x+n8X`;)=5I%OEZu-H(X_-v5oOa|5iY0@ zzz*{c3EdaM5g4Ia?m`r%R<_up4Hpe(TqbWCoe*yj9ofq;+sBa5b7Bxy0z!XdTl{Pd zdw>);zo&rS@1GGTV3r@`zY6b8$2I1$TgaLAF7sU=;wBhAmmg;SKCE=T4UFDDoEl>8 zXmHH-IdFvEt+^IhBaX_+>TA6H-Bdrno)e2R-)oz_HhQrmu$m>0^wr?Y^Q`und9iq6 zxBbY^DdP)NylfQUNze3!KKb?L&e8B|wPO`$tb_n5RZiym)S#5;GZaOfbSvnmZP;h> z%(*Bo#qXSOvT=;>lh%gcn}9Q=;(l|A#2AT&<)jWluxb6U7+Uw4-)^`DPA%T11q0F{ zOB``sn9YiYvV%xHDP@_CgbPsom5uq6anzWGfAs4ZlhAQg6l`(9O)+Ci$X>J|T239F zt?GGO`R=$8h;|M5jWEg0YnuQcm;!n(9aJ(V<(SLyc(61Zxud-XYJnt$!y_WwSY%>L zqR;kR(MiTpbXXUMu9^^)4(3?5lwVPsw>;}+*+JIR??!A;LnX6MrZV*nghnYHn*Sz6 z>dnOTTTN=ZERv!fv|0V?%`;jI?WXIom1fNrt`;d4ri%;rRqD-LMmg%&n z+6azAxY?0+1z~lVuRvTZ!Pq*)rb8%fTb&>KF;Na$3B<_2ybTM@L|lnFLB*;> zn&opNA?_z_@dD}q(2wPQ&4cP|{gf}T1Ujo)~?qWhFTIqk&VdLUPQ zSygxQvG7{@VS)Q@t-ULVX}Ip7lp#Kc>YHyi>~6iALqXU6H^`0H9lJ_JgE#Z3L0LVO zybn}+nF`w$m<|*s6*Zi2t(aanPw|#gGQ3Ajx5kgzQY*%j8T(NU2E4|MOI^Lb$a5IXG)q63Yd{nBolxhpcKRLenw`8v~ zG)$k&U*^ebTGq$jOt}&D(%#Qh@AGcrcKY5tO?Xpl=v(ql^(hw6yO)RAZ`Y2OzEd8H z?D+eT*FYRb;ffp7sflqEf`>!K+e<^T<>LtVo9X9crR$41ME!KE2*$6#>%k%Y+nU%u z^3&fpJaqmyHBlUKIS35>IShkhvtpU3J~kobVwI?>f8o5?{r z;R@Wef)Ka^DVfbQ+yUZyF%n$!jXCCKeaY$8<{YUENm}=XV2x*C{}fG7zQoklIV^~q z81Gxev{})GV46Zb7-tF<@FdN+y`pbXWR31RnRK*D+209te6o{P&-7icE)8yTZ) zfKfag3MaVGGD@_+dR3%s&h6avEMPp4n~lV2wI7H(H^$T}``MBDlRZ9^6hbV_Kr*x@ ztmE|6Gntz5mLXdcP$}xzlQxo;A(a^jw7&6&>903oFos=|;y8xE`YuojU6Bp${%_WJlQi`9QjVP(^YkU~A5FvmiV=SM_J6!o{;#HCgO*w? z`2Q#tmKO@8zCwQcmVl6UJxGn0=3l}IndbAGjEN{b@TV1e8m#m;cGa^`WmCrpQh?Fy zf;vHB785F^qgmScZ!%6`^GK@_`9k76(&AIRH8|+A$Oc5}}p2D9ungy{pUsfiUF; z8?+CvpjrT5^13Hm3Tg2R%5+Q12^i2R^!7b8Xrqc z#k_k++zNb3RS7X8=knS!ye%kEnD+!a*`(tQFYaPjFFXv2SlLLjv)$zCx>zdl+0Fc> zm`+0Gq|--2&=Lj0B{Y3t3G*6SI_jo}XCs{n8=~CVBr8z#q}yj)wzF#l;2kRZ!Z@;rwCg9J|H^ox$5H zKCFk>ODmd5a^P%{=RR7rn{6uWdTNJ`S_VIVmm}|~9-n+NoAa1$t^K~-1_zE_$O(7($#LuXfv6pg$u`t!0iE#4w52J2S zmU73L;`&!~oPYM>pQEmiGB4uZxLPaq*xmf(48BwtaERGpNdg^vggaf)*KmNN6-4Fa zNn3P}6QcQAW=ieKIcuW%;iZsh5Na4}r7NN?rIi9{qtN7iIfzhLB(Q~^`pOP`m0%u_xqO0v{vu|KBJXHI} zGv>Bb*|ekM-GolRaDk~?U(D4_5T3vuH%P&5V}Krp=gZ!G7KdGX3w#x}e6q&lw)&(_ zG6$&gN=H6?oa=|0n#TQ?oG!O1Lq$nCSz8mGz;^5X&TmTdR2dDO#?(e7A*-s`fO>s! zPqKM$-jLx26AVU4U$%&AqtoxEjO|z5lFlvIB+2qt^TOG58+JWs7_vt_&F$+-YV}Pj z&njf$EYH=UOc{;Pzxi892zeEq$wI2h4_n}MDmttf?IcQy)w5_x=+haA@!Zr}Myt4# zMdq1p3A!!pyWyWG`Gz+8$1Mc;bZrf~&usbZ5`)~VIr%K97~(z+#?HUGvkTUg~V)|f~G*c5_4Z1{a_+4p-D#wzjJr}nUF)ISL(_A4so zK?E)(wFa8<7I7#nV27H=%Nx+$i5X&t4c-5;RbZtlcbI~(q(jmkC0QfVzoh?$lG2=BgC*@DEi zeppi?w*|$mNaD{rTgYTFa{UZ1q<-b_X#!CEO^(z88MFfkg<#la;iIh!`oSy5+6#@P zq$TpZ86}56^n_R{%yOs>P3gQ)*V|%SZ*4V#wv|Yu4rzHNMEhjOp37gr0T~F~WBMFI zc@N^>=u?s!`gQ?zQB6sp_PaO;)y33IJfh!j4$gZ?N{hm~In705rGa5i&vM$!;>cg7 zLY(n&UVxch8B&u^(ywKg5kHV9+UJBEDK`FyUwIXFn7~QjjB9LWnyJ@e;?vjPFrMUF z#BM9gPbgkV4`@vy(vJH@u?F!A4%)=)A5%-{mtCeBS@{#DjEHLmZLHrQZsNniN^KTE zU*G>j1^-3)|5u#)KZ>=Dp1kyiX|;`r*}zZO2XSD?jci(tXdlM@psd1ri-N*7YmR~d zzYl~1o*|-c>k=GNk#B1(9{-Tf2I@6uaVk3tSka({lZ;(hrcHW@W2>9JpWz~#O2>vW z9juFhqSk5{m;zN4&1kY4Rh5ix%xEh+Ey7$eo8BCIv85&l=GQe4pBy^U@$m`E72DT|eFDe-2~0+L4~z9!f-Z!&Ss8C=jC6~66gS6lM~HhA2EqhDRNeEO zCvFKkd(2)63eH(Hy+i(z8aaKF4-m^isVW%w3_LIj=?N9Nr#O65Emj zy)46F%U5~Ghy!zw+Dv8zNI6*;CV!HVnistXn1U@*hE{|7&v&`>0A)CnWHh#EDd%-y#t4@ObQOb{m7exG@^4zb5Q60!gucR;V;8 zg=1KrnU&Ev0B?!9n?umv7#)rw5RG%c>~!B-_CY2@;1UGy_xWiOPWXJsepD^KFQ@Lv zFBvT-9v7m%@B9}f{H!~0Q9YzPTNX00DqP^LCv-(2QS}Hc@5ED%S-Z=C3rD zJ$cp~+kCzrFK>U-=?nU~=13n3LCNNW-Z#uFyXOV3lCBPlP6SSf&s6S)hAJ3Y3}uHZ zVEjZAiBis7uFW@80woHrhuS?H$!WKg;IF4;bqi`76?7Nam)M?`@=Z&dH)KTG$mUya zG|B;m^Lf;WmRd-WGo752^VyBpoC-R-DK|}hC->w@_p#fX3Oh=)G&n^3X73^PIBm7t zRZ=fK6UP&xPk}%EfV8>`7ugh>y>muVYns#ToV*^ctMuNVopxuCr(3QM%2MrJA}$TH z^y^>*<=wrbjm7IB_j_w1mSxy$AGp<;Dq6n+Ob}*LJflWjh)MbPY8a31a!u?8@N=rv zG;Z*XNNoVG`@?K%WbplkRf{WfD$OS9(Gp#WnoTn!=D;;Wc`qv$tZbXrpHSBsaw4fd z^L0{-y5OVIMdz8H>eZdj_J3NW7k%8%pR7B9fzPQ`HF$AH56a)*)(+HXs9RQrZSdPO z+2NWm({vLj2%~-6hbvnX^_(UV<`W?^ve=Sk_MzF6R{=}Z9iiFmDb~la3|FRuz2s9X zp@$wNy1?`_zu8gCk)hcD&b^a?`rcEoE-B-$(mcmOM`&8y%OOXkN=Kj2H^+ax4l#9Q zs<<^_^Ex**x^bKPI!#n6akY}0n-(Uy65(n})659D#oJE5_bmMaJPMye#^zt@+jFh! z@l$jEo2s*;nv*n0dTA{jP+S$CqK9?Y|uOrj4kai)J=(+k1DF2#f|IZ&33&Up|Zg@S^Ctb z4&m~oifdYPVvWJb7Ys?j7H&w;8wPuPefQGf7n`0pxtVf)U;}KCZ z`gx4Z$5;>(npm*hn(&*P1y?>4O-WchD^U(2#$r^lv_x>$4isk#H7^3*AaygdB)v$U`B|`+>URHHnrK~Z@&Jvjru__yPb3c4e&rUp_m?F zOe}!5XN40v@D8Jm;ia05az{D*i#xRs))f;^A%-6%I>j%Idjg`rN5dj@28SA<0FG4W zGC;mG)4p`;l=m`dXW9g9-~=~=s%t}^*mPhlvF88^@H)#(bf4mPhL^D&8 ziknD|?CBEBL%&3ULfd_MmIlLR3c+7ELyl;=Uts^irT;?C|BFlip}Xd&UYXs0_PPcC zOc>IPgP<|~BhgN0ylyuyh|IT6?q&s#P_O};ECVO)5l+jnSD;o*qZshRHl*MxmX92d zz^V;t49U=#SXE?1R+$g*B*c-GUGMJw<#S|K!i6IVpZ+(aAb}F_?`ZeX_FcfnncP~Uf= z^^96to{2wG#h;;jbT7Xb#B^}I>Gy8|`hAPcUNMJ|TZGK zCs(E!`;Tq%|FMnWx3otBB(yXG7H-CtV*=)~|JC15=JW-I{^OcA*rWh%th5~ud^n(@ zlck%Xi>ZvM=l`-zin_JJx;nO>yq$cu1GRBg@}Jt+`hgklbpu^!>ws_87XHnKiUyqf z_~F{dWWxoAHL`2RzM92qYX(}ymv4RxjZa*u%p^0aN>dNM|A@w3;`6DeXZzbTyug?T z5+qJny3DK}sHhrr_}N8G?Q0N#|jO8Qy7 zKQq`h^fSfRF2(vtWSDD4=hL5vVfFw!b2Oe>3)3C8M7{C`7gYQqXjY=^=n*ruLMmc3 zRtx{cAVSxXu3c1apur`gFqNEto_!lbo)lr=v^-^D%WME#bKW?o{c%rzCYUkey^SxE_HGQ<}N%B*6}OIExF`q z8FJ-KM^&|C;VbzhVEradyJ#=EgA_^D-^S_F4pf0kV@~YDI2fqNOVnt|)qHfO4We2x zz~-5(x5)I-U{y%U0T$f(F4xk-WZ~Et5`#k~8KsX2VpLvT^MPEv-@~STIUpv9Aoj|7 z3DH1hD~Xk47lvU#NIsxhb2&Hrrfn=g=x|;Pbtu1ilRErwX<$igJEowrjKlDp?BXK* zBs*fO7DHOuo;&^&=vyL4?$6=?n%F}FwO0P-LGvn&^~k*S23+XVDG^F}F=tQMrv{TC zm4;=L|Id$an!m(Fao*8pH5#LKL~wb!vV;bhg3 zr>I6DE1>)ZE(g1D2Ctz!OF~`-$vyK+Cs!@BJMER7o0?)eq6;G{|1g>eOQ)0~$x?-i zCCM9XRS=>LWyyhO$@$&6b^D`?9;SFYkRGidB{jHA-#47GbB>&^_?LYp32uVFI7Hr@ zc@)!|C-MQ~2mDy1VH6ooh|hU#!Ai^m;y&Ljh&hlPk9%5QM2`2)os_q-tjpAd_u zXb&M@z+x#5Ri^HwKRAik4HPl;Jl;9&@y9(?asZi&{_mV0mMtv4<$Uur7;X)xvt;*; z#@tc}!j6ng1UGU-L0z3Q_l?`TtXo`BNS!|OE!cdaz9;{joBmm6c<%P8bBxk z!J>$W5jnP5PNH|EA=YAszg`hpCG&0BcR5$tS1#Q?zF#Q45O#|o!Jb%g_>&pt6@L== z%GK~pUFFg=_UON5Apg+lSmtQ!^cjbc0(M&{;PDx?1Svbq+XBmV2P%Q@&GcXoX+$k0ZgpAz6G$j~Q@G&xTjT~4L)!}V+ z&hxI+6jD){LMdL=hh71Rg$(9xN+b^yH3!=7e~NrwKQsHvBBfKZuwaK#&fs(~`PKzd z>7kTQnBxb3hf;&jw_5FFuC>MDfXi8!0Z({I_};qsj(oT8a40`}vT>cD?EG0q)QccR zzNQy*=^^Gg>0c^|)J7WV2o;xOmA3z=FS}#RK^ZRIZEWb2lx1yDL3)2#ilV-5-1-;48i4o5joGsf@BEmKP1zR(bb^&;4fJ4d5 zNntOJix|Kxc#*FY`Vo;mFCp5DOd@+%I_QfjFmSw2N<-E3iJAa|L3FlpR50m?oa-=xP<%XK>vT+*Db%M5fQ%s zPyaeBeL?T@Uow~TBP}2W2J1fw0cx58E)5>=|J0N)bn%d|$~0?*foaIS#o-=O5Z~*R z!B;{_*Zz4{h}5^Ao4Ii&zbAhN%J-bg@*rNwmgfD97AO)Z+F#%|FG7YE7p6*b-nz?u zy6tS)^nJOq2Y$l_JyoCdV}NUD&-z=_pA0-e%#C2T7T8lh7}5k#ZY+#9@=uWinWD5| zCU%?k_~*w7e3|WcoWb28aqn34Rl&>~;to?A#|kb0p3{?QzWQwi>>V-;Bg$I&6y{hu zIH|rBBiyPiYB&<@khEB@&DxUYw$n~(*y>e6n-2X_X}R-U|XNA?AhP&R=hoDHeS;eenC-E8Eqx0pgig6RZTm?Kc!Swlovu_nntB z+-d%Iu$-5|W5|l3l-{BP!uF=yA*jAdRbh6Ot#mj1A#^Y3qr%JIa09%+F#p)XS<>xx z%o728!Yq02W|NXEPWOGQ@er1;U;?zoK>A6ODcfT;>P9|Jrg#jdB{k$%y%6N2_SQ*` zluEr0Yc0o5gfu^|t!k&@`{p?n=n?bSnRF=3aK_`5kIr^m>!@QS+qkH;;)pnh?71-6 zDFPhy3^;n*46aZ+@F+@Ps)NFgaL!M;d*j{#tmipV#Bs{*{>X?o0Q+_Cz)wOJC^Y@E zEc@%~h^xj@2>@pLGw_1ZU=hYvVSh!h^b#`*RNKR4Qw1X8u=zLK$k;TQgrKYDQ-6s^ ztBs?G|Cs`Mi8*cZf6aaaU--*r@;s)*qNrree_%YWQP7Aprg#hd8MM$ykyF2CfUkhK zP=GXoph{?oZiev$(u1EiW0GmK$zc21aTX*un5D0nG+hPQU9n5W( z!U}t&ldlzk1<@_Y{X?D_(1RFv_AxoVq*i&>oUYW+@NOy-tueq|dwEIeM(%T|Ul1SOL1P2{MAPEP z3s{j~Lz9i-X>>iZgXWmNzaH<6|AjW5TLkX+#urJo3OuEuSOhGER<0KaFiE$Abd@5U zVFI!E8^t(2`#~A+Q5eAUm{!Hh~{;l0>`EQB|Yiu|wde*MC># zKcLL(7tbQoI7R@?FPpGVQ~a5M#1Y zVF?aULflRl@6#lr6#xK`F_)3=F&CFG$POEqPNg;ne?(!Q*&%nWpi_B zZ*DGQY+-YA#aat|l+~60pJbBn&g4r-0zncC0Zbue5)vX0A&4X)Kp+W`iAcZ-Lo$$& zVJ6N@5Tw?&*0d^KXCk5P$}H)s3t0-yVDzZ@94y|0BbH$mdN1|Mc+8aSw9w zmdw2E#ybQL5oH;;iDxLAH;_v{IUdR-BM16l_U~j$<-BA6^YB);Z18v}k4)+18&sh1 zkOyTnLSe5UD>SIcg9<9nqmeYqpb`Naorj5Zf0p#d$aJZE%H%Uvh&fw&!3@1ykN^nh)PqnOi-Bd%h(wk~fouYaV=FkY)JxwMXcXm7>)itQS5m5nWv?_&?~OODW4kYh>mcId-iQfXjXuYBh zvcF$GLHS%JpN$5EJhX|z21N`SFz9kZe?cq|iv=U$mPj}r*b*nyv^=4+yco;D)e2J{aY#Gls9uJ&~A{nPuxP3-rbt=P{0vmHt>vSc-&W z3xYBKx=?@#rB!@zygwMLtS3y*e=Pgu0j4t$?hUj=23hWXwk}%38-igGP43)aI2fNt z=&d>v1gE3$DJuxBmPlVfq{P6TgPYa`qTRxb=q1wY53TS=gEF=!UBVm-enDWpe=rmu zj+S-Q77nu=ghV#HZOIuhP?-@m%cV3NxWeaaw*he1&3sPPe~EgVVw*0X zuTRYk%ZOcJzZg``hCsZHb?B;^BIcIwi~D;wvPo#pwS%!a7V#!unvb1Rb@HQwFRg)=NtIP01+M<%GqN{!FuKU7+8;IT(o=G)SoMY4|{2doUIc zu;(rG2SXgzn(a-5uFM{o;Y7C7^}!95?}1T=dK-cnC`^f-0>R6bX;T>@>byp#-k{Bd z>#EWbyx;V`yFEGhki(`8_JsGM0f1*($MZyy8n>41Q&neuuqa@0RBM%4mgc5T+ zIO|0GEtY*(CC+&7M4-?{(#1sc9qwnF9c#zOB&@S2l9$ZE3apyI6u=x!9cMKXI(0ta z8w^SW995N3KzVYu<0^SJC5t1mcyiPg5y$0_*=7rrgM1*Kf0{gu;n}9ldBwPRbZhID z0ap4+Hn^&^!;c!!Sla{P4e|a3{u&e>B7@Oh4mxtFPJGu3ajI}? z;xo9@#9jEZNmt-j6Q7XJNAOX0xKp|2Oxi+Q8J?~*aU*UrXq!nN;ubfVbQOKr#Lc+H zq^s#8CViAXe`e4%CViZ)HR(F~n2AU6m`T^u4F=t4(oHhBnXYANBvsHk7z%Yof=qkV z37$gjArrUZldRpOS1tC(`uSFs)z|qFG3gfi1V`-mU?Sl4RwV$fYCeU|QKy=S;Ye@xoWIO!g`*QC$UeJ0&c516-xYoqS`9r&Kjfs!pnuK^y>eTG{r{Iw$p2xRL+AVDDp|23U@0K2( z0DKoce@Uo+H;^SlLpmG{(O!f0nY5n{n0ST0YS2NGzD7^69O+Q72)xT>ukJ9&lAh)f7|#yN4yiXDZ%5S@a902nGE!$=(b1# z*m%xB#wQjHQREBs9c~R#=TGoccJd7BJV92XB0XM{33QD3D!)~n$C;b z+t*y&*1g#3$%o4?2U7c+cFM3nWb-fJ1eo{(esAKBqJ@{`^F6#qXgK|J-5?hAE7(m~ zfBb&s`onzJ%HG)6Pob<~&rSk%_eUdFSVc?zuoBOp9}qg<-z%Iwe0bi#cR)we$~9f> zO>Ldstt-3Nw6}IH>R!x&->;MU%31YXA+A273UA7(lz4m{)>WOfEICOwJn1a%%WLiE>h4+7+|%9alN2}Hh*6uC>NF za1Lwn-Z1NXb3lZ{nL6M7RsrU!S|SjMgU^$LJuz!FGAOx!)e^lU^`6^E|!Bh)1*;NlLjo0}UDGxE9vT zI>nutEWxE*XFce8eT0)FE{sIAGD~1@&Isxy0JIkk*kl)fSmwPI7AeeAUQ zf5C`Zg-vG=XBQ``Twh$L4h88Ge`;Y{XPa+v>jFZ{nLOGI$Jdy`?X>y^!q?u%r>xT= zVOvX+j-=(*SY&%lhP6Phc1R|Mm|Qt(s;Bb6FMrH!GIF;t$M&}1Q- zwVG>Hl^i#oKG%sMb294;K{Nyi^T>5(KIY$Hu<--hI^x12bjFBl;?;P7IhAqe^PJhcYDU!7l;L; zfj+0Sk}x>P3sX?HMjndgu>wGr6t}wNa#y!pS?l%&yS))N>Go#a!fms> zbm;MI_$1@{lxgA~Y9RgMFLss=3WDmhr zT{DE7-5SSs{uaXpe;B`l+xRI8jbEe0_zmt6I9`|Uy|`ch{dBaZK8t@Iz=QS~3m8^* z6&^%xPxX_a{V=@ypmsUfb9pC>-=$zIbTAg;A?BC(=kxf2jqxU%>PdpFr~Mgj+{fVF z1CRIUE*b1a-VvBQ=iOtR^DSkGh~=PlULZeS;`ei4t@bk%7(fgGkGC!M`FQqbkt=t!Q7Y~bxY zD@-*enaX$Hu^knb$O^Wjq}u9^*pI?dC<+cC-+Oc?lx9-4KfwnYn2KqPW*VmJHkHHU z%;8%oGu}pxe*$JIN+GE+g~JiZK8_mZZ~=4pCaaI|xI^a}9t)kN7BM5|SB%0arqffs z7o&H=Rq=##sz-FGa-oz;(QT=Hl0p%lVA~~qrV^>1<(e;teFns z)~FEGmQGaNevENcR4OV84An&F*sk9VUZe}LVa5b^~K0Jj@eB8e9B5Pu!jlP4A5F08@w;#J%;>)DC;R6yANj{ zu1So_e`QqNGsh*YjhmBGnsW@r)dx}0ld})wR}EQn-kJlLkd4k#7ju#~CzspO+`~0b zW8&Q?syTv*)$HZx96)6js%yFX0a4BbEXGnkdMUbe>(cDaK>@Qj9xgQj1?n6Wt4f@$ zCS!uCVkJ#sbM>N8)uL5Rg-_LEm1;o0nvSrVf1yP%k2NwG<8*D=Wun6rEX!Mg5wc5 z>UwoM6%D$z)LzXIf-5R!U=cI0lo@Epe;C!l40K|=>cSLtA!aBatG-(^Go2MPhNFh4 z*~KEs(drWh=Q<3|wHchj!m(Bp%C#VGXUKWUD*%k^sCgD9dqkD2^Hns3Zyz336SAKI%cv%cVQYD^(`)J0g|Rw7f1l0Q ze`>(CL{lc>S1VaGJ#ed47_C;bM|=>|)TNGI-5jz+6e56`wg3{$T5GrS+Qa&c?6)x5 zVm0R-w$Cihm&GUVc^c2yd?oe+2}KlX|r*JwYUnuXnt?G?x_wT=T)A4=4EP1AS_ zG9?)hKO@}}rC79NgGzYvD989Le|_X%>=++U52qI<@4LDoxb?`Hso#%jLug2DId=%t zIf0qMzca1LS<>vmdHXT@;+>z3;i|nZf2JmBI4e-Xa=jLL>N+-@>)EJpz&Ywh)T)~~Be?|^s83+A z`V5EpJ2}kX1;6?%HmdE2s(Wy)x)=AT`>;dZkKO74999qF1@#cVr*=37=;I9ZS)C!W zByBC^;W-?`H|?-DQf|ju1J9Km$3z2W-J6)j+n>j`Y#(@xeNX;Bf8fmQdF9?i5260- z^0hNv!eog{xb&7cB>0qxrPf5d@8j9D^PQKHtXe+&2&X@fB40g*aq4l@@py*%k`vDI zIYs$4C+?CWpNki`m!0Cx(z6Jc-gkJ_@*%e*OIh?LTWOa3oz3zjx|0z~P=P}`2O+}_U9eM#XbA5^6+jaATH!s4aG^W8bX<3aVus> zJ;gS12$!nE7*JowHuVi$t)9iF)VFZ2`ZgX=FW@o7@=-70e`)m!o>Q+n=>msTyu?8% z8{3@Ty__*xJqhrZ3iA0@{4QG(u{w9*75pIOMzLmIwUsgkoLuuw`fmpQJ!|~<<0$2u z<4rD=hWY!8_#xvLGy5CQiJ7gf*@K=P$f@1M*96z|GhKVIdOHeBT%Qv`T*89*;9+m= z0bD9t=54%xf6WdK%_S~Z-vO+Zi^aQ3qT50EdGosB-o5Cn<=?#sNVs!kIEf7h>cdi86}QorF8>$i?^7GW8Fq>YoQbWC!smICRXNG417EV(5l zW9@p?_Bc0O?&Bz4bN?s!seKjPXx~1yf61$$Nk=FS-s^jcH|#^d6go|N z5j=v+occ~;_+-BDB!~CvHZ^PhGxPfw=Jj=qRewX3`g=;8u1&A&#wN0%1flEs_h(F? z6&m`u)fKO0!5iB-Kgyw9-n|GhLA#ujb-k?q!LI)%O!ZIBZr;jJFDt%Af6mo?q5>kF zja|m$e_!C2Cl^Ue29dPv#ik<&pPESAyz0(Ik(+Z6cP^^j#%YV>SNOF};38e_bIHW@ zU!>ie-Eag0dvSTP{MA*U+eD6gM2ZYX*vI7X`fs!hQqlhhqqig;veB125`PxTduY2B zyM0(RP4MnfQ0@}uZ!|`_$D|xu?g+jFzjXw!f1_y;3tp=C-{JT6$=BGg3>8VqHynd1 zj$4lm4a1%nia8j_)l#b4GCrv+P3Gk+^KmF}S1?TzQ0AVPBJ+xLnU7?dFSL^K|8xl2 zW#^od*Gw17tJ9QqBr~R>WH!{k6S>8k+iN7%(dC3!zzbQu^^BsPFPnOYz=x!OEJdXk>TM)dVl4xOo@DY3={k} z=gnlgrUED`-nJJX+HSu%dK#&(IT10U=G{5wJ0Z>Z^2p8|uB%&1n0FW^N08mQ<1PTBM5eNtX zP(s{JoP-`7LqPxlznha#gdCTkr8WhhpW_B{W*-e7>&&SNXdGp@)ec$(e@AqbX=IdWR1OSudyB;Xuze>Z)<8*m{f*zlw z(o%p3Ypt&_#v^mWzJ$kcrrcERV<*In*>)7yZ-OD3RwD1E|BmbTLSOc&W``4=WNw zbdm2CLp^W`Vu8=YbTK>~0>VdaN6_U2x{M@Hqte+lF`BxLp}MiUC~%8$9@xs?*M(mf zg}Ny6Kv)!eM2Q$rgHE7s6AA7l>UJWPCR5!Mx|Hf-sz*!{)6+zmm_ZF@65LsInN3aS z&}FU%8bvu>DyY*uy3F@~h)S^_jZYT~>4i@hi`=5h0~ae&ip5m9ggRHJv2|i8wXX57 zFU2x?yWB0lgQOQLXq;NQtfaA5(Yw{ubd4@f^1xoPHeIX}>(j8`$y9v`l};tl8|ZQx zF1QHjVk14&(WO3J1jQyQZKlf>x`gPml~!XLl^W>MNJxb}i2X8udJ##7uSFBRI9(Ub z9@xj@mfnDMpf^e>f ztrq8b#QEX^s=Lr5_K1tmAcYELi@pGEbJ#O&}8sJ{)b01yy z(dB-+>?fcHs8t6Mp)*|^6o=@=FHs)FVIt@Q^!7owct{t2M+oJI-Qp3qc$D5h<`%yq z-g%rB^a))&NeDfaE}j;olEkld@vMh=#B=H5d6B7$7YMcAxW$VE_P1$>-|wgcDx-_V zId1VXQSkR}@rql#N>q5wE&f2S{z!}Qn>02|{E139x7R)Rpyu`lUEZWg{8<-o>Edk< z8!6r)iTMkEVeqal{_0@`q91kq8`b<>7ylrBevd|0JpKV)J|y9Li=_Ua^!R^n_OLEK z@~|o5U%L3WEhAyA#;y)fXOZ+R1e$5=N?urz96i=q{~s_ z{L#Aj%EK0juXS#0SJo_lT)Bt=t3tHrMZ(dRK)7XfprJJ=0OkS% zQP;AQK|vSdbtZ|%Tcon7gP8AZ!J%D2HS-vLJE`0FXtZH`ZEdwl6l~-PI>_Uo$`%uv z+1@7os5p9J_q_UGLvVATC0I|SuiP1|Yi)@jeTO?pQWpu=wKg{g!!6@u4X|hbt@Yb~ z=0zHt0xhA9p@vY)uBDNBM1ieCeu&}Jfw+Y1G`ZvH^J} zD~jM3gqou*OM}s9U^7A*u&JRnx+Pd&5!@7M4$=goTLRngMQ(F2P(Q!583%FHW^IX; zYYA$w9+$i+7_P@wtHQx>UGuIctX$K79BLsRE$<7Zdg9j^&Jk@cKOuoF9i+3`V8685 z(B^QUrL`G5%V>SZIM6~BMM50|dOzw4R-Q|-QDI5&eM;8#yQ5GYuEgK`P z;d%ucNts?pg_=l9XwxoyYHE(u;W`n|;Bq$w>b3=2s*w|0uy59;P;;=}g1-WPmkO8C zrqs$%V-R644K*}`a2eQQ216a7)$j>z@*VF9T*7N(M{}TQexM~#8#;rSV-#tj=9Umj z;QUZDu(5&Acg+liLoKrrSV2lz8n>E(oTr#Lvl=UES{pY8n^zJ9(rJ;pK*Q=lbBLa; zO4+i7Fc{Gr!I8Q{Q;)lj2A+|B`h72wNWEwF0KE_0Gf-mcwk-`bsd=Hhtjqqo%tOVq zIoPr&#nOp_akzN7(e5o6j4W83d{}C+3Rgl6o0Z5FNYPe&nNs`My%Dn{sf6C1`N5{z+4DwI zwLtPD7h43IvQrG~kb-eNSiIaoQ&U5zPAvnfua-z%qyf#5)yG_q+7_}7q#kV^ThMB1 z+>zKYrOq`QVZS-Ox9UZIF>7l6XcC-|5*pgW%}ebT#58mtpy*988!)0*a47W(!DW`B z*X00I`*Bqh7>w!zUMNuvhs<-3zj9|?u!+`Z2F9xhHfEz@YmjN_$rZuI$o3!&RM=;~ zh{p5S1CY3H424lt>-!AB@4}-{ zS50d}gS~>LM0W8`RRcMs4QP^Df?a%s^0b5cGC1`(iK&1t+qe}m>f033f?cRh>u%cX z&_k#jzbYJUZEA{tke_KetmIHT@fD5H&Ez+9LjtF=$@H?cxSAS zz_tkp_R0+F6!H2IN;Q#id6V)aj*s{l47o7KX=#n>(u;a;kt0UIN{ZNeB>AX>4l@*( z?-%q=(bVo_@AOD`ky7AxpjM4*i~jq{=0IJr(tJ&S=BF_hNV}~*NabPcd+EiPX9#oXZmL&M-JAxSC>O{QG_cPidM#AZ53^$WGWJkjaG&r7gZ{%)x}*I1sZPt zx0i8RsiNfJ8G4@>p{>Ayo9mt0Fk>SzBqeZK8)*&$Ox#cdB%qgPLNj%iz23;l#NIBm z7lW05{{uzU6%19Q)iz_N#7gwl5~^l)H$CK+Zuy@hlyo3MGPkZP2|$!<2$BKspQIMm zhCvFUYEuoG0U~>bvXNcSS*_tjMeJ&|%*BFlKc%XwaaAZ&7{zT5MOvfFwg;OxHAHr- zh*we`vlm5^y39umVrqO$FKXP<-p+U=*42%FPJ4Z89nH~QXLFs)%&A}@vT0LP6&kGk zO20^R2tfxLVhNWtqC#~*xpNs|6;zQrRGxNTB>7CzO#VbyUroeliFD~|29r|DEfOk) ze%C^#W*3MkDS~k2cGM}=kw_B-LA|9R){&mF(~td|Tf=C}<~x&<#IaqHZiWHhC01+oZgs-D1%Xh5TE89w!@gLmb*2wvsJN~wtin2ErLdL9%MCQ_ z2<$SGe&trO5B>2Zn4p;mN6Fu{G0SRy=~k89jI<3zgXFJb9WEf6v@VA1N>QdVm8)YJ zi&zLsNSU)`u=IZysVJg2(=j$?hGil!SW+2em7%tYvD{31sue|dCLG+1y#u7wDjmqB z%KY{v@6ZDnB~G@$=;BERXZ1d6k6DUuo8uVZpusYyypQe+If&`e;d4^gT!h?zNJD*M ztu8)hQ2uS_OW~cZfl((=Drr~u{AHdB^1CCJ(PFlUywnVNIgk@_o~yflWb6P8+_xKv)R&DK1V|7CAO`UvDsVIZT(sQBMTw zO~pd40KssH7*V89CY<0MClO{}xHykO3Qr?{1ORagIdO1g)JRu>uor;*wPE-2Y z$%&uhlcoPV0>;l{Y8(ZBr~S{za?T`l&=BYENyXl^lT(B^5Hez?6`tF!&Yx z2L?NrooBGi*`)@*knb^AfNeB*JAczC%NREKFZ}WSk&MX@hv}#E~il6ad=@{**1f3q|&e0;|5>Es|?n{S`F5~8Vz;@`-v`n zh8!VJpogpR!cRuWEs1`{X!7{!g7D->gPq0BHrORNJ$4zpl)>;`02%Ci3_bHCzu18r!5(FQ$kB3)!NM$JuqJl8!L}nIShF097c9vFS!l>2S**(vLyngd40bEK z4dd8u`xNEPn_C-|V{Rt`2PP)*Dq2Gg^+bt5w(l9Y<5`B3oG2&J+n=sChli4AI{eoR@uzT5!2HQut4Da#W1j1w;)NELPPsb;yRRil}Fd->v%e(Ki zmMd7un=<@Eos~6-ecH4YeYOoF5F= z1?OQDiwkMUc_?@6IXT~8FUm@Ty~K_fd^6u-$OZIb7jfS!av`p2FFRrADWy8qeld9Q9>--n?>$r8cT_0V3$B--dCSBGVawYq~U^|IKAIepRT#XcyYvf6W zTq{wTNx9yTC*y@*#xEx=cTAUq40(z?)sP!FYS_W4cfU|1ZMW2?1O*wXfijW14E7)P znITVqqrn1*y4)!13|TLOhTKGh)$)}HdR2H^II<(`vjy8{ZugiXz1d)YkXz^yGT8Z~ zYA4~;QKr%KrHXxh%##)=m zAtiUwV2f|#E?-kH*z6nc+t?i077Y8ELQTQ3zF>26q}kW9CD7uF)G61Z-nU~5+Ds*X zw**x!>jpCo+fUo5b*&&SHLy;1-NF}-)Y{(zUD-C&- z{IS96sPj+cPYrps+^fqrLtZ1VjVXhFI6|BI{}!PQd7Zo-S*d$7%8+kv4m8>gZy=t_ zuZuL6L|elp*tR5E*V0r{ix<1ijfB8Wgy=L>E|v}I(x(<(8zO;vlddT3K3qY20}CH8 z-cQ^NAY$IkU}!3SUxf-28Q5>~t*@~)+JbzrJ?O(FrF3G8Z;CIrS?E)nGlsl>Mc!(# zi^$mf`@Xq@>JOLK7YO@M=EtwBu0^oapvbhVl^xFT_4dmuh1v?8C{??g!0gI4rzgOB2)4PJm3`E&9oPb44nV$$#T z$c2Xdg}hgn_Ze~@Ih+^sOOST*enajjEpmXcow{;IL=iw`)^WXKTNP1%MWV2J*@9E_ z?Fd9|N|pGm^@&sHwUm`5QS2b@9YV+E9KK71y=znEBmb%~uxlfUf5aDUM4I4yI^{uB z=B+&_^Fw@wA%DrfLdT$Q8CpXgmJjH1s39LD>QwWk3`+WN=-6=tp4Rp*br5s>ac)i+ z@*(+SoGMD^qVk&gwTsGsmsHl2p?n(h2x+5-B*rXRq9&HU9 z@=^I1gUO^BiG@Pu{sFQJ9mpo(35K@QizFHpIhl03&1}vM2#3!Wj`P-jB_C%nc9XfC zfZ7I?d`yXxv=TcsM;d&aP_G4*pK&+M!-=#MMr4Pw>*;pEFiQ_ODDCjL{?;?q>x2A;DbB27Ln5|a?rWo=Cc8ei@BVX)8>r#UKTO=Oo zo8OW5+-sMq)a9jrJ~W(04PQ3+NOFRHFJCeErKGuDm9LQ#^an%!kxDz{pA7jrU#H7A z4Ed(~GlOxbmiP^42p z-GPm0N_LxW&p7K|;wS%S9KwkOoZUZPp%LC;P?*BCC&zGqWyJF8=d1nT<{qRWwkKy% zs1G!p&Y-vtEbaWjXJCq)G5QZZZXi3=a*s)^B%X5O*(qzRl)L#jtK&Mf73UL~83D9W zJ)FZ~Fsd)d2^i#b-v@VwqDl8wEa1KEML9jI#HLk^Xoy29DveR!T~$-Nvb<(qW!!n2 zUOQrRw(xj=a)_xC2`Y)0SJ3_Vl-lt_a~0#F#XWrgaoitI_4F~xIy%{9?xIARb@>;3 zoELjU-zU|%(qu9@%I-*8Ms#|<3s6}S>h%c{5hl4v#lEt@MQw>Vzo}Tvz!hks>;pvYJ@ zJSu|fs36%<6SXaaezkq@LAS=Gd8Wo@hEpB(C6P_%eFM#5wK>*}+DT3~cdH>p^-#33 zv57YD%2V%J#&mmCcsm-G=G41Ux^={Z!TN%4eki^7PZ^@Aq5ppz)fI})kC2-1_pprJ zw$a9a7Ide)^PHu5(vzr{;4H;w2`LU{kWL%?D`G^>O9EPPA8G-c`=+HhP16Ul5P^CW z=2?B&P3~b&G+U)e%ft0%!cm?3DM-QT%TfKHK5m>JBt@c(UB4E>*_o3|#Aa2vDH2g1 z@)?y&m#m71(hpkFDK>py7O>(S}1CNpc9Jy!l8)i@hhIesiC@zxdR2Mzne-thv=GDy{xu!K5`O~#!RlzL5li% z3)^Xb5KmG?~uGD1f&vYbq*O{0VJsfRXNKHa@CZt%Gz6_aGQ&~BG z!^-m7B@8lamaW_nO9oWcEToS)qUK?HlHfqN#d?Zu+hxaOwkF1M47>%Y&wqMs)}$0# zLCVt_Mt$5c`2{^5k?|{>REaP8EE3x;SEY%o)xTb!O1J)a*1PNj$tX2G4>_XRS=VX$4+rnr((n`E7)uzK2CyUZXz6$WSQL zry1t6N1YXot_jhZ?4+>>ee!_#2Poa0)Uwkd=-41g>$vaYV&9AOi#4c@1XRH8FX--T zB-WU1>D%O_FOW|ut8a}vx)Q8O0{2rW;1~j7W;;X9wyWFuM0HPQB)nYP7Ih|r}W`Wn&QSR)0f?_s%C!W ziUrln)}Xj3v%RWj&5H7W<=6nni~DE+Stm|2*l{U3iOsEHGd8uqE>QX+)P(Ak711ejBK%e@bS~w>a!KL;VSw~p|Ex&n#U1alSG^APo;okuLw+{t&zzMcX>ojEd!d96Ecd?qL>D_a=6_x;L|^ zy0@@ab>FU@Ih{6B_np>#mvukGx}V8*tNU5(Y#i^qs(cRiKbM`S%I90{F0k$wTK7HH z{UXGZ?iZ`)3OcH)?w48j%hi1@9ZLns8bqf{@VODkx`O=xuRp}6IUs=JUH2D?P8fWj z0{A1mdZ?OzT@NmH1G^HR2bj&+RqV&=*G+Xr4yJ(p!~#|g62W0#e_{SU7}5#(iSX`# zGg8Wkp@?s0J8__IE0(U>+bHD-DZNqijaNTObOt@E@a9I&( z2pM3HzY~V~<%2Nn01WaUhT&`dtP^|(V1)C5`}@Ox2?t;l-bNpQvHtA2Z~(>?J_#9z z!M`@AuoH@o6vv*5>B(Ps0KERfgHUn+#`}v8zyyE(0hs9b?}zb)9WdztoT$qCVR8pd zp;-wvtHqFm*gXsqS;|26@FIJR!TA-lt6(zHdj@+1=CDVhiv0?fvd3W!dlF7&PeDC< z8ba)U83?mq!x`*ZxR5=ksNy$gX;Wp7O_e zK$=~L9AU1=c%mfUHAH{Z&+X-KQ6>A08q_cwSc5WTq#Lcy+YlW>Xpl{NR`1!STmc>wL5f} zR65LJ#%}11gfu=Xg$}#VB7?^y147<^*B9QaOu>6&6L^pNaR~}x3@?E~K0XESU6#mu z_a~){NTkfvq?Fm8nlcL#DKi%-Q=Wn{2f9(Fqc4JTh{W^iu; zuIrJ(PX>dZ0{!@D4qRjEOoLogKPz0_z>Y#cRKxiEOLiDBOGJ1?BEnmfh`#AxJb*wR zWDnW;gW39HNg|NuKI)GL*%1p|nXTH;wpAy>J0pciKa934nMgDGyu+gKG>dWjk+tl9 z2yGrD8Q}s4#{KMddKB?9p?u$e;3+1{1KMGrS$i0kuceyr z9D)@b3cD6?0D=IHfcO=V%YOhv_>W*b-|JxdiT2bcnp49bMaqZdYB|$@AB7=0dn^|# zeucK%#I4=(aHjd?0jMP|?0}VOwur{6b~s^aA)>a1LmLd-2U^ZaCLz{;GQjF}67Iu+ z49J5)u-ZcP1mqmx*CDFcBd^^EdHgmQ#BTyGzZpjITVN``-A3CB!`Kt(L(tkxvnMdk zS{p)J2Y!;yo*IUaPbXzUp12zK_Q-_Kpod}txfBVc#zi3OJK*Ft=uf4>!*I&loKrht zLkFDZX9r+C5#0TReFK_ zMq%JlbQN)n}MORzPyN+VVa^6tYv2X6kngV)pSVbknk)6@_=ZiwI8LpU7yBH0O> z5=ZzQ==`M=j(ltccajkvOge%{GQw*(!XLiD2(Q=-yU|i}UQ)e>U^7FRbd%U_DLjPU zx054Heg6i^-sx71T zm5S>Uy1(MnAJ`u)OrJ%uCH7cW^awPz!I;AkUR#7wPGqgfX{_yprZQP{5KdQY){N`j zjzJBI-jmQ@RUd?CtUiy3-Ugy+1U>LmLivQLJm&%4AP** zFbSr>E^BswWzY|2_a$2Cqp0k^LY?w8_ymJdg2QNmYF)UXSa@KP=m%3oI+O|nW{M1$ zD>7{cE5!wTi@mMpRBq3>+_BziAV=#9b@qE2jf=Q6RC)w$l$D{I1$~R$i+B?ciCUfiHKhzlN!jccd;5)CK36w7?f$E z)WynloWZzKH`bkvlVEGinU%>r_x0v%y;`2zD?8x3?O>>9ET2=Rd-W&5)y~FZAerkg z^y(dda4ucW>xA=b{9fhFd36lMFDOg%rggxD`(V%QP)SWLT7#n=hKtwcT+#`bmU+C= z>rn<4>!o5GuLmuUSAPQ1%=Z356qNitS0`L{AJtsG*R)h#osjt+c59C{MbP`;`!wPe z9q@yaI~A>f(o4!3;@>DB^9gVw*&$L}{`^jV_+cBYYln5nsTufZSR>*}UWUL~O*_n# zrLGhhkxGbp%|AkC*&6&x~#SXp}VSKf5w7zw#>7rO#ce3Z?iJ6p-N ztNRt~hpOJ>r1wxX04Iov;1`pSqfUfMF&P$%DX>g` zl)?%z9af7n*d%5_tC$1lh`F#wl*1*W0Tbp8mYiV8QgK*%PbFM9yMw*YK0wLV*zKlSM&)=t z`;h$;86=Bc!~PH3xiA#Cntg=rbR}cw!hmDwt);2N%wtSv|I*pNb@nlp5c=qUMR-rk z{R)=p>=T`R`URv9(k6L?%+5&v0)~A7+(QC-48_n$1_CI8e+S?gWPm$)gE-m#5BtnA zO>g5w(EEZ#MNfhURnyXT$Spz{X#=B(*ovx(pMXr|LR}@urlgab@naOYJh|6YTR+Lu zsD#Sur(HIt6mbD@CP=Xx(#2VSkR#59f#SO`M4Si1#rdeNE`$@s#V}1=0<*;BP$8~> zh2n>5k!G7X+3ISMZD|%M;&W9}mPVCmX{0I<@ii;{a;0Zx>S^e>)6W@Lx{Tu@q3vVU ztj1DgGN!zIl-2SVX1hU}YKYW{7YU&Bkml7o;OgsPRAI52{bAUvLXEb6LvRh!B0ok7 zRznT1D07h_aoJkrOotAcqFe%iQ}J&-{>>%DAxd?mv%8pxo~Pe!sg={(;SvO$llT?~ z_OwGqJIqlEC%?@kzemQm>!w_^IOU?I)CT>$i0rkup+bgyT)hQQB+b$Vio3hJyDaYR z?(XjH%;GFA3j>S0Ebh+Y?(VwC;_mR+@4N5a`~HZTh>Y&6?uwbrlU3DS=ajcqA=YV( z1vtN*sh`+>JbOT#Xc(`NwvRg??|KlU0=)Jr_?;;;zPYu6o~u1T?{IhkfC{;M86%cI zt&+Kj$O!3Hm5=p9s7|5OIl0K3*5Sze&#Zd`!$%L0)Y)EgHP?};zk$~%5`=!`h5$zQ zU4mbM=+nM~3SnOKG-`K=@I##KqPD2R9Po3#B)z1>lHZP0g0Rf+K+wsr0yU!oqdeCUe>%N_1i9Dc?Rq&{EPz!iNd++D0> z*`~jwNq_VHr9vYi@GGuYJS!jo1Hgt$3vyC*6H}0&SY*PJW9Hl2v2kq*zhAlWhp>K>V)>hwZ-SoBdc8 z7$XEaH@}U1?x~cB;-l16&$K9JCq}nsF&UHWKp4k8;cGLFmVnrefuDtcpaAvwx~>gF zXYNlr(CaFxn)@Ef93A!JYVw?KZAdM43Kgh=Mgh9e>#=Zu4sV@Jpx4zzkD)E{((izK zDWwB}E->emUMF1~{oC1f?s!Huy48}4YHIeGfle>AZeuYNPBG+k9lrQo!e4WGR6Mv= zy!j9YcDqF)l}Hr);nH{E-2q?Xd%Wi*)C#{osfHA>5iUxDbB-b`lfCpPrh@~A^;=Z$ zBTmeT*2$rJ>u2Es-$Ql~(~+IVP%n{r(B5gKdZ-^?(+QxAhEe5*6UP~)Y6D$kOuONo zYm0@cpckXc9a(KSHQFG0GhO0j&t*%s#=mude(ZudWWclS1TfVW_5rXS6Bj~K`3GxO zqA1hpROq5z*H-k^5ac}i+i*&11J}bS)n*ohpx^A+%J+r3u$BSK5iqQ_V#Wm4@(G1sxccVuSj1BAAfkmmNh2{kGrT?%6wvBQ<^9_}^GrVTj6aRy zu)wMfWu;-1^+eWhF{Npt@{3uXwmO$>?s<&QQ8TW*S1|J+q|6W(F{U6Gc5{!PH(wq0 zW*)y9G=B%SVPDO8Vv&TB@T8zOhQv)R(&K{R^!|qShqr6Ri`7gIslsA3njax4SSTI zr<+vaij1y4bE`bb)(s^PShNMo=6lO47)}9rB=fljOR3nZ(9aI^dfX<-k==P8W8SJF;9m>q1C6{iX^-5n?}Nw z8_11lYYw7066;)t&`Z?{E9y!w{+n}|DPtSVhxas*Z->qYYMo&!G-qVPxr85@cfa+r zlpi^ek~x_AH{@kQ2Z~{;-!@^#2cgNknqe{4jk*5gPh+sD-IY?%?C>31Sb-Y2s(l`Q zY@H|b7C`31kSgB1F#gmYxH3IL81|EJ#oq9G#3RYM2E$e21RC*G4@}q?nVg(y^bcP z-H9<}EwaVG><36l930h0SOx@;%kv;OEh8;u964hWh6)XN3UAWQmEN_Q53+ogq8WZ0 zDZqWtnnME5Ew8i?!i8~nHv#SsYNxO-AJgEM&jEq%fre5=k_8B7pBk4%O(LLov9T{)1kTpbuL6g}+QVXYPhK3aeo|3KJ52NFP?DozhPSS6wDrz{uULjn z$<%r#EJ>$W`yXp03Y$nP(z66U*w+^uECBg;f|McVT;a`V{|@AX;g^}NvxtRjl1wrH zY{>|8u8v?N=Q&I!C2@HDZ*=D_f4D%BoN%KttDWX+$o+-GZOkYBRbIDdA4u0-)tJ=m zZL@^^5aZ-JCA2^gO#ceH&S0vq-avv^E(b;bIZZ_;x)<@dH{$sfIQFM~=mrU%BY+t0 zsa#sd8)U5|2QQyZ6I1bUbVdByqJC!|8(HCp+amho4A+|S8O!G5=GNjh|}+(ZP0U7A3S$-RGg`pvhZ3oY^}CUJX$ zsTxJ0Tv81eh%?F<;n)vBDf$F8Zvc*7a;^g zLA34CDJsL7?2hZmbd>^|_^+l7P43nGcC7MR53<^&=W#en@3?IHdI`)QaOC^Y+dyK` z+E;wG;VlzhL9J~!%1z*2r{+6)R^CUwj0)KCdVCOf;x0u72K7oI_lSuX0wCdOjFrIp+2?1Sa-{cC`Zvb!v2m^eGaL#2` z;cx+ANq?#>;m72&updqi9;8xdA?%R8-!G`nYpn&Pmn6A`z1CJ;!VPYtg{kZ-V_YR* z{@n7H5fI1R`p{02AVXf?)HP_k?m|X&ze-r+4{to>j~4`g7Unhs70XiZ(l26RGDUCU z2X}l9p4sCkT1xl3hk#9YK5nr2yWC6ixQXUbCG>GOIrf|NA|0P}?zTF59L|6d66X4@ zFMP~5zL#pgdvl^rtEAlJb(J3!^1rDUz*kw!U4$6tZ1*rN=o5<>En&18# zphK70aw;nf)DT}ZTVnJU?#`)~ux>KIa_A3Uvdzr+;IV#t=CO58*s8|8?_ zN;UPV4l%f<=h{c@NEa1em|3&lbcD+mZNoM6#W$0*mAg`Ko{FvPKz zgFG`U@IFu7e*m0Q284MR9Uk|OB9LP{t)eT7xoec2v{(q6D+D)-spuea@RMrveW}mm zI+V^%r9K(4Ap8|^3AY~vCzx|y+);z39sGqsxQGSGVi1|A5{0dS3@2CN>&~h;PPIa? z>X~?2kwW;>Du#BxLdfq`ckP0SuN$ij+(sFQ*fwt4Yz_b>{huj)3c8T%RTCjb*6_wT zO(?vLw%)RONK`4sjUZ?@6B%cE)B5+a5PHh<0X@MWjZi zT_=wslr_bwfRcKU)LtO_dsHK-BT>s(20Y8@_l-K)Y9H6lllClW$kL5$$}Z8br5i{X zN`>QgMI2F=ijpyA)em)uK!o6gl{d(uvM15300-YYxthf{%JCQG{kGN|D~m#0-!5SO z*Fs^yld;n3v;^HHF;eS@ZrX=BL9WUH2`x^9#A*q~L;x4>q8Kh+pAbbixFf01 zS2!>CEI3xgKDTKjKO!jGsevrzLV4&n{C@18H-756!IW#vV^#C5_|R1ca+N`L6=s-* zALQc_?E%Q_Ko3PqRu+}^u2m;O&~Lz{=~i>V*yp|#R-&tErz@&VjnO+Vmd>_InOj#l zWLB<_doyAYeOWO-NWuuMnIGNZ!7oz7v+w45l!zOZzhC)^hitQ>92*yPAG}&PZAQPYpM7qF<(1m@g+N#;NcioG;Hn?bm)g+0zbJNT6PhPc6c0tw0op6g(n-RBFZqB;e-$~;!6Lti-~wEfu1mugf3LC| zlEvxH8hK0|MLzY}Z21`HxsEENRJrsbYU`goz`F}$9|xA;C~$R`uViP!b7gmJrt=M8 zRh-Uq#lRAjkUt^z2~jN5Zbw=N0M2N`2h`P%DY9&J6LtE`*roa%0uL_&Y_B)MFH1rR z5}Q(3Pbg4rU0-{u0HR$dAI23Y=rk??a5wrNL3B%2&UGg=p$0}GKPR{4v-(!Nw?ZD| z+)ajNDUay8d)yN3ZcLc`z(VL{&Gc>6u2eIkfUD31Ouz9H*23trEFFpr0Hfgdnm@2_ zJ#c&iASt~Uo~EZrsn;Z)@qsac=WDv_Mtan_bT)p1K%Z#d6=QZ;Pt={LJ+cK;KGnuK zV7e`H+$c`urj;a!ZE^tK_p)~KqB6f<({rR!T*fWgL6H_c7y|gkJDmgtWI1F_<;_A@GG0%cvryz^3t#nRRE4EdH5I$3%!j{TFp&$bJw!Uz$31mX<| zE&CR+JnME2UP9G8pHUKg1rg>WdbKy8lW$R_%TPN{A{04OJG?V5Of+j2<6k+dLY`0`ciGOodu}g2cS+C|(}%b&hp?}O zn6?iUZ^9kL&(nh_0Ei=Ev|vK+$Z(3W`?v%6<>#r%D5=HL4Wq`n5V4NuOv(&FC!WE$ zjf|qmcoD09wf(F)M_CjI@t8X1h;#cowWbc(JV-p8=(8`s^`p$TaK=O~bNgDX+%|M< z-E??#y45&bCxr@quegZqa=69)M$?8lkZs{|Wt|*dU)d);3>acOOHeV-!Mh=6$D}$< zKH3tQGz9f|mc?jZV39T2+rFIPyf1J#mC}>CVmKd>i(b25wvAY9Pl%rPaeCe_?*3q* z*FE7{p`rhoj7cmk;&$$F^yiByX6wN$1?PTJR#ABGtak4|x#)dti zPO z8|G!fX}Rtz8`7)dJCy}8f^On~jpQ+^pHqJGWp>bt2LPRuAEv+8nnXT!`|RPe!i34kF5nupr;6A+(MvJ_UoQ<}Jba!sTv*bl|SiN!*` zX=OBIB>*YBgw~7={{sn8^u!&qKi}B=FgT z`zL;`bT_J%C%XKilMBokhM%OGxB7&12y6$Woc=_gzL2k1+}k<9A?|H0uMwn4+V5d7t=tUoXTpTMt=Qyl=vIIb!VEI$3^I8Wa;{4tj^aFVkm{RATDcHJ z{begy4*gXZ#D(0QlmnOoBsMQ&v7V|~;wU!o{U&&EiaXrY>KsF$*!7X4VJ@_P*#>>j z0#Ha;wn+CwaKc-?mHDjvq=zP^Ana)cB%|ojtI3Ht!!2@R;YfYjM6+u#9xF;CEjp)V zIeyPnC@koBuGA;Hhes&F%@5trQdc!$FkVcHy8bk3lCJ{hFz#Z=*m9GcKR zd*5`t9;PdPEBp4xju$a?&4Zgb14Ykat|cXBY)Z|1)t1mi5Alt#LfF)}R|2XgS7pQ= z(=4hs7x5LYcv>(hpkM0na4u0O5ESqWyO53>Q{WcxRLgD2lW!k*weK7Qmttm)K7bn2 zYL|dBmTAO$75(UA%NtS=Z_fQD+Yb5WZ(raUH=gXlv6z_GnP8Y%jRd+8?bVb<9jN`= z5V6e<(Y##=tlBS2%wKYO9^XXG38MRZ9m_Cd$bt^$jKxZad&jrAkLf1(rTQ0Qu^whqNKOK-b?KM*u z1bS|jbt|ZJ3-DxC?1kO12Sz|zg};r#iY*d;9dGvo!4MiK1#_qF{)sR89psEplq$3R zX{!B7nA)5II(Zlz@Nj8FacobKBWHT(_Y{zt=T+euNL>g>UWnt(pNK|%6x3(l4Cv$^ zYBY2~xj1}D&d*5oZyJg?2j%PDlo$@#K6k*x!zAVNwqo0ye-T`&$D}8aCk?D}VhW6S zWM)iw`hz&>O>7NIoYmv7d|h*f&rRimlC_sjZQ=uJV0#S;csC{%)Kuz8LcSuG!7$wg zmeKMG+2fm1hY&z4;)l4e#35%kfv>-!@$%YBT{y19*%cv?f8|%tVoK-A8eD?duT^;?E=Jl>n}UjidAObVU0iWJEV~8X zy&(L8v-IC*xx&!txVl{I;|s4C_2n9CD z0S|xl*vZV0u=e>Y4Cu?5A$iLkNm8H_4I<)Rr?L_n7Ac>}XyL-y$nDIJ{wQ^XOU{<5%pOs*gTc3Z6=Rk*J7(!bYMm7izVQXz6@mA(Bd_k)AGr5 z0EC+w5GDo#Y52ks*dg@P0Wx%n(rPlZqZ>B7_axmS>VCghg5<-eqRIc+XWc}@wv*5XuP<44=ONDe;*}y zE5em0i3dAEGq{J?TLiAn*_ov6!dT?cPR;VF8$Lp68uCjVEPs=vwTo+6BnEpc>ib}k zTya;5v+@RQF+gR8we-3N$GX_lAfB>m?DT(vNH4&5?!jaxU^#+){)J%JuHEaV8}P@1 zRi++T^$+#OfB)zI8v#3QOYf$iJoJY^7ECN0pfWovdh^mPbO;#4-}q=|x+bsw*Q_sH zuYvye;9ChM)4%Fx&kq1SvCq~3NMLO#JvI=fj++`dQb2$SXe&}oZulx3@GUa(d$70) zy+#N3a4TS)PInj!YRbat!3pKnhmFUTMq>-4!_ zAt^8sH#{{QG@L9lCX&qHQiTstQU-qE5l_HQ6F$pge z_pB&q#-5_8R#t#;ul3gd#)v*LTW^ZxNQUife6q0KMOv4j3wSn!-G|vC5Zgj?5Mi zzcARFj`b3*7rn&`LWrrrJNs)9@hY`Rt*bS(m>k(KSnw)&sJn*i7X{>MnYCpPk@J!* z{7K7c6`)wb$=Kg+NW$w|#GyEwQ?=C{y>Tf-kmD~;T=)8M?3N&+5!J2c-ka6C8lWhe!_@#T+Z6V0;|dJ$4!xI z8se>lRcwY#XKnu^(xy5Pw7%PPS1rjcl|_l?>n{cq+KFBseGoLcde-L@7OrhyIi9wB z695#S$N~AHD*X$n-0Em1Npnv4^_DRU1(J`|y{^ZS{A^<=L08r@P;%>KHJch$vfq@zVe8+omDk7n!-EjWxG7(x$T-AE8)hykmNGE;AKHq;T$x&34{I7T@fY4K`pqa1AY?yKt1wXS=kr~zXZpYy9=(%!{= z`QRO}2dqjMG1}K~b9z6I>|w`pzH(KySVYPT?Y7!PZgbk!6u88~dxC6NdRUbPqoPhz zaUsqj?GEBLD@DQxwAym5mO&!>H*uK8<7Wb-shY&k3#ZQ9ny~u@`T!8AavAys^xtx#V|Hdq z7ZTyDwmSSuePrh$Di88X5*)XHJ`Lz*FoZR&afWKH>pDg0Ci?X)vH@aiNvTEWjJgM& z#hbhKbaF}I@eezOX6i8JG|D3hvRqLeCmw>R;?2o=>`Y^ zfm5&Xra)ldP*5mLmb)?Nh1pj@CIqb`xj7MlFXr?ghdKbp58MfZrH7tupB+@Zc7a2h zs#zK?Q16IE5fB-%p!)Ucf2j!9`D<3OOTIv zf+z+LT1h0X@7Y9tuwD&KTQtwBbfnR-7(>H1juR&v@(Nf}i);>w4#I4Y*k!)sjvQ76 ztH&dRyz*P*!8Aax&^o&$b@1yvOKAJ?MzAQ5%AaF*bh8MMkPp6z`~L5k>c3F&zex2L zM9g8VJv=@^gbxV*0q5Fs(5LjYPgaw9am_7TKG{EI=yhIB;#K_+-(x=zr4FlaJA zFVW+(q~<{W)oLIFq8QOpG!x@d|shl{EM&9loSyN5c+?iG3GQFBlL)32>|fKD6s8vx=`R+eR8*fNeRl2v^50(} z_Z}0mdDxD15bU4RL8qfRm!GF0p;Kx>34tVZFbK^9j;sve|2DpD$?E)+y@`WKF+u(3 zEJibnE9)WnKiw#?MaBY0{Cg8VlHcDvKZ}AFuvQ6~2)N141q&?x!iC!0#|@JJ1yS&~ zqY0>lvY@DcP0F3oxUkQwu6(v&NzQ}90}A6{K?9Wqd7**r?cDHy6m=D+HFb>ls=fZQ zlg6SmUhP~v+BH1ttRbi{Mj4Z1PEP5aJQh%;b*kd41MVJ|X_R2%I*%xNS5}`SEQI@al}|8eS-Hr!;SLEe!t! zVG9BceGNqg9S(4S9o1m%k0HH3Pffa2WoN!bC!=NfElkFvG;p1jBK!A{#du!7fwQ>c z*%x0r0@YD*)^3}?aS=tea@pz(9J~#voePfWOBh2^FTxe;*IY9u-{LO(*RQJ2imEDD z)vEqL8a7ZlTkwqRjTx`4m`m3f@F1ynkmpv{fBj|N2fzLRfX-s%dR2CO5--NzD89WI zo>mKl`=hdEruX|}+o#Wq&k@Bzl1-iQBA!a!uwXad0l%zja+BlRWOBgj{q~W)oE$x&P3Ag}XDrVN-HpLl3(YL4jI5R#ZO_Rn!GAWJHa?zDQ8;i(E({GV6H~V zdC}s`J6KF^KQzs{;9-GQs$Xzw<%*eSS@qPmNpHuk7J3Co_7&+Ubc(JU(}wkvC@7OCJxLe!QkaGPWQEOK z7O|94%4;iO2f~;oMt_r2sXQ|a402QV{1B{_a~UxMx`(IbbG;D&4dgG3jCzYF{kjyt z{rLi2-C?};QuD;%LMhdgdp1!X_ zazpHf-nv{SCsOBrk4*9}Enk_O=YTV{oag!O{0oPvbOS6>c&bL8Ub1Y_wdB#Qk0M=(Is&}j{le^`<61b zYkc;`M1K&T&2T>v=)G@|{fvoggjfLSiDT5a^t8s+@>Cq7)obm=x0Q5div!y+n6;$y z+!S*WIumqpJ&e!qN*VL>F+1jB>@g^`GQ`qkvcW?t!9-gNY z;@MD0EHYO^xZtc<{*;g9d&2ecyfy%MWv3!7)Je6GIU-t^XvpK4 zEV`|*-M{{kGWi3*!6N?J~CBgo(rfC)W)-0dk>_YyC z&+cF{c2O*hXt9v5u-K%YNxF0Q(Y_e#leam|B+vbQK?n9?y$J7tPbN?dc#e2R>uFRe@KaYePL@9lhAmJ!A}G{hC_20Vy8(6YQ^LC6R z-3C2fbX+z-2|d*k*H%zKB{QR*MlTL!sB3SvUSeYw?gx!pqq?^PQ?m{nlim*QXn{b~ zls!~9%c$~{IoHu!%HJJ&RVk+xsRB|Mer@Gt zL(ofHewAnhV?W$^Ee2Y5iJ+#O%*Ar(aPe{k<$h@ZptZN525|oZI{>*_GKPDXX`B8)qK!f@Ts<-#MNBKgMHDrZmacEL{hP4R(m5L)YbQgAH)F4vOrqctTlZF*}_L;#BdG zfWygWcZF7Ll5llcJEJfUOnRd*tzTyNys2q}`{5{yw-z7a#$h3*^nJUaOnDq7EAEYB zPv52UjiZPj6>Ktd!7 zOyo(-2ZBA3&r_;R+~uRc!z5x-PV^H$)t3^fGMX4;4b>Iw!5Psz~GBR3fNtd zm-kB8LiXXTySM0!s@A%?t?O@sga{#x@1YglO+l@z)WTU(c3cm97EsQo*rosWC`;Cr z1Vc1rB|zDiorGORoWC_`UhV~2yWrUb>11|fGM`k%Eg=l;Fhn}#E_%)5{!G9dfRsf9 zow~eFBC!G!=WUPBi?Q}e=_dK)0O(JhcK(cjh;C+m%88AhW(rSrs`Om5XJp?H*c&~4^zrHscAr>CqGz2H+Fpa@+eNpWUU9bPz zs4L~sQ|$~9u9zMJOsh6)AF~flnU(Q7)vK`VV>l&$AGg2x{1<|v zWmNq-BH&Ua{E!*6n+?gCLyyF1n$)@ld7B>1)^}GyuF}(vXwLEED=vU6SbN{$+|JE0 z3R5*I8aq79w{%EgI!9dlxa%~(Wwq}(hGo-;MV$f4$>lc(BVj9^T1vY6?jgG}0rR>1&7*91^N z^uz^lMlBUJKU7()F;ytz2x|u;|EBAktwu(RHpMZQNVh6$&SUl=K!Cf=bUL3pw&)@l@00g+%1x@}`N1%@3w4yK^ZjYU z(x0rAg*!cX(*F2OA%yi=lxsRDfMjMt95mur(07`>^hVBEXx6fo@mBxC@#e#%r}w=R zE)c}W`eBpGul(n_E>lr2Ohhf-(!&{Dc$%EBe1w<_jb9EbHnBfnL2e-u7r>o=86*I3 z{bBx{C@orVm!*giS%WEAG9jJ84aegZo!*q9X&Hf{SyJ%)a(1h`)US%rw0iOqYM)#X z`V=?o%8W}e`WVZNfJw zTPi=jc7-ta1NOc9K}x%ZU4x8YxeXhT<+&hL^?fz8NE=EuYZ$)!tkIOUMnH3}2gA2F zGH(7tJ||0y7|h3By)fdFU)=GRCR+mYWQ@a-p^seJ%!qL)Mw<#6bY0QuJRR`%<*}0+ zl}LZwJ58dekId0qd(Z*_<_q44(1xw0xQ@%0ZZ^+C6 z{U6^2GA_&oV|8v~g0kOw4q{d~2`WCh8(NZgf6~-4)u)I--q`=v5emLR{~JdC6D%PA z|M6^dL;yu5DB|CtZMEz$@DT(Aq#7uq`?-3MumFPwRL)|B0n+yKpf_`N(xHJu{EfWk z$xzT!Fo?fRnwcVzKNsZx9zENt@S*u!e4v5_l4`+_18WXpAc4lbTrj{bNh}J0770`g zA^vG(N~kxa{jX(MC~`6q5`;k?b;O^nx_T9#45rSCUB8j{cL`kY(&>JRx_-()>`M(C zr>6w*(-3K_9%pB7o>v>yKi^av0l-;8o${@Fy#?@C^HOCR*VV?p>wfOJ2g#v3XK$33 zx<7u3P97RW%STki_wpfqK@#s!Qu+Pq`+65LZD;`>B6rWYmwD9cwoGU>9+HSJV zu3g`lOPuql9+*3doOS|rit5%qqWMx5fRr}7@Zr|m(r3rqXFhfO1}G_jD2n-ll3XU( z!vV1}@!y-|>=?p!i0OK_mN4QntOz6Q6g$dU72;F|Q+FjeP-e$&`%dpRTX-R?3Wo@> zxJ${=d}|q@3iepivq`kP`88CdRjW&(;W=Kk+r@j~byW|+CfEEtk4tNG;Z_)8VQj=v zwCcb*!DHq~)d{Ct7Q<`6CYdbd@o&v&sYWZDIE(CPgG4$QCwh)7?B5lyo>IPXncOl} zugW%7Na*xfb=r;2;9QLE`4jzS@OIkG&X~mM-litt^JP~6x7(m^%#8kF)wqMq zhA`!)DoClEIdTzfYXDEbAVhK!a%^0<@cWPI-t@XW-xZ(Jc1X75kD&n;wG5X(4cfd% zulvW3VEXk{45irg=2pimVY;zQMP2MAA**uSaTV&m&`?k~T_t6LBU@qTSzXbFS|y>* z73tP-5^=7ae*78$5ix~VEmyMK2Q5sfqnjBBJdXJp^%?ycuTqKF)Z#A&KW4w8{~g(Q7w+Y&L0Oyxri0q1QtT& zmRYyNK8J23&TVK6c$(9+wEYHgsoMTS0AW<{Le=^;QCbJ~*(+Atlf(KxE8+;;_yP&# zG3&22{OM|*HtS;<6<-fFZD>gA>2D2OoB4v-+_{Ps`G#j?`wfO7Pdn6UgZMakS{jTo zuaVYY$Cjz_0D-ZNdOdM-v(6wtqr$l&nN~;W{(J|Erw;7-jk5SKa*TmhD^C(E91g?e z9en*mJf?(Y6qSd06h?yN3bZQ0rnP7gt+jze0;PIPP&e# zLXD)1x9|J!!h3WdDIVDL$Nqkrw_o2ZF4QIC&7&+TaE0-y2FY#@2sfa;Tnv_&LLHyE z+|ER(Jr2lk6AROiR4JbSnRoveME-BNYkrRbMFaf@X|7AkBac5TUj!5|F@S;sC|pJe z0c42)-s@xqBLCo@$K?|(33V=~e2hY*`Ls7|%T0;dPCg~}BlvPt_6G>pcXBwOWa za^if?cfg;FDQc|}fkj6?nD}9fpWR%g`K8?+H?CYQzen^r+!0cZb3Fxh3ficex$)BV-qu6xeAb;X61j--(iCfe`h=5oa)Uc zX4_S4w^#=_Nb@J=?lT-H930d;r%$5C;&JewDah1Y{edhmHAIqep0uN`pjZETP^zUf z=h5d;A0`@P!nQ<4FNJ1^@xxh+;9O_SxpHgN5S;v@q-0qf@3-`9gCiu6j(PD`ztu#`$DGaKRCUC{mT~k2mqoCayp(f|3Ik zjCV|}ehbLtBQt4E6(^u3;zogsL)+_8@3aP|{v7Klq9Gnj^?AOql4>3hUotAXSR?A> zzMFzj+Vf`9hIuAF4um2^Qx3%%vhap4a$juC_^Ge~k#uYF5^}~Nw26jpyTu5T)GvY? z3$*|+3UQtUz_q!iYo|F9fx2{4f1!Uv!Y-7Oj66x_zdiU-#WM-iUSTA!DF`7%1 ziaZ5ZtmyQ%$Mbct_dB@g7HRkv3BjtpJIWm(g#JZ+5S%lPAZXe%g_1IJFdak*@sf;! zyJLmB(;#zKFcYSZAR=lisaPnwEWJ5`PB4BQbf9-9?FE^^GJ>X9WRz>rdOjZHjP{9q zl`~k(AKB%EId^;h`_Ee^iKXE_@Gpg5rU<>pgM>OxeE|bRTo0tNd(I2=i~f-)6)@16Cy>V*>s@uXIGKqtc-uQ5AQn z6-UTB!zyp2Y*>{fYs_=XYuG)pXr*jsYER$DSbc}@=G_}R;-0%Qvy$a|?74CSeBa~A zKwwcM6=vh{JLfs%dFXI7e0^Ek0x2eCKZ@wv|B_tB0L79eMm)KHDJW=s&frFT;EZ&b zKF&gW$Px`-0q7iFjc4%Bq(30k#vf(BP@CpLc}e(Z$FNvNhyw5Pq~0^ zIfUh1$GR0bs9_yhNt=sOF3}?%zl@@%twJSPFQ}24Xr-KdiLp`SyOBTPBi*NVz~iOM z#7g6#d2~g=ffMFj8LPN&Y^AvrySj92D+$4 z)zT}*Ea!f_?M%bQIkRM(*5jIea}?p^D>FYNYAYXUmc%-MfE(DL@7No)7qAYiD!097 zZIiMs0i2IPC&BR@wi}|@031ZpC2MT+Z1>d0*Kh{$IiarFVo3>VS%c|01^YDMIq>0I zEcbY;R$D%ovmIATZ%j~|9T;}1!YuHOShnT-E%xmE$(`LV;Fh`LSbyyGdC8JY(S9x` zE@ht3Kcv_}3_f%<7LylPt3w#B4LV{PZm*m40Z@j? z03gEwW6-%PusduXP0lB(We;?$MTOeB8BJY-tdcz&-~)6$s{RaE@b+%rW|0L5g6H~3 zu;m=yK^)28;|bYV9ZxaIbfb32XgZ;C#Y3px5-=z@3V&C$CsE;T4KhFGQB{)8{hozg{9~4(GGLpDs31tlt1!P=b%ti!K-N6d8(x51tgkb`J06J^4eJI-)Avm=T+fOZ z*HSk5BliOLnwdG3@Fsl{QLl<-QPp1P8+z?2g`&!h@PwQ){92BVe;nWOIN&dc{5PBb zZ{dw04XDoj%yTPP5D-luXc8z15FUXN9eCyhg9se(#KHwsYF@bEN#F!>q96@($kdiK z>V|MMLd$@f**T!=!BQg1CgIImzcul2?*DNaT4!|F%h@%`n@o$b9E>x1OST`nGRRE` z>MM@oKfN1yk;Y{x862O|94wIGa>~Z&MXa1sYsoDt-&%hIm zCJ!H=aV!J8H`%7D*Aqx#Hp*G* zUsO4<>YogZeAmC**=ioktA>CLWdR4Ms<&Fexka1k2%ZQq`s5XzuQ?2t zQPymjl*C~|+9K#%;~-rMySVvLsx1_m&s@n1tQtySQw@m|Q6AVuHlxKYp@2m>*;19i z^{(zxvDfSMuX80Ckac5rT66z=FBx7mn~rmH4>6K4A$3Py!u96GWAo^{V#hA0fLevF zEgK);b(=l@3B78MF>{VJ1QQXGP@*EKL3em{z087`H?V%jS%9`Y198X2rd9DOofgkF zcm3Q>drn0(edoChESp|1xwkCpkM2)dT&fpDpx-s+8WxA+VW)0A6Tyl`G5B$iwOXp0 zo{g@jhVrhXdvcmf{~TvoqU>Tum%OhirvNozB&)FfMLUh{_?+^$$Y(M6qdgRE%jv=Y zYsuD5k5{l)0}(SE`r~24w1q99($6%W%V8y=9YfiOIw!?R3ejP~GbbtXR$tcDV`1;^ z-45t4K%u)@YJbQDClP)j8Ti$ZT8x8;ZQc;t4a=X}Y$If;M#1Uj=jbS8r(}@C>b){x z3v$z1*@I1=ee{h|O3POolj&}UOK;&m=dsj%jVj!2kR$^Tam)5%27g;2=tVAeq1iy|(WTjG54!I|6oTmbzSCH!G2EkoOVg zCh3}wE;%>TF4{C+XS`0G%I}>JB%5%6Z*gIu$WaNCp7@HiLG97m9TL0%@ng+iyS#M~ zih$(f4C-Lc!&7`tB-v^#Srm`HP1)?(1q&fcx9IN!fLQ2`lC_ZOJ+BlRt5mqTndCFtKjU|C>!F^0A8MlFd z3(gPpLIsd+J-N%YNh$jOkoAv2lC91DKRna6ZFAbTZQGu<-PNA9ZQHhO+qOBafA4*t zc<%E%Peesj#adObD)Y)*-~41|lQV+DL$Z1cDWyvaIZ_qQhg`)#U-JkF6#( zvY(eIrOL{!{bUzZV_Xm=ypgyj9J*$?MLGlACm2H!50&U5!X5XTKm6x<3$L?9h+-p3 zIe`^IYgtQSk2;I~1^x9$ z+$<-)*k!TGf)|CjpGFM;NEe`JjE{vo>>S~p9GkZ6u#g&K53NfKOQgzu-Ny5m|AI!d zAshy<@v95{#9JrIW@PvK^~Ozl4kU(!GhRcpaAq_kmFyVSc#wCUdiyi{LWN92y$@=q zgGt>a*hHnMf~>y&dnAvyWyFO2iWu$R1AXlH2F!`{z6w{gWA=3|!*V67Kh<$YTE-WV z40Xe;*{x^c`%7f*Og8pIG_tKrTK(=<*}b5{8pCzOv05t@%H98hm;e1k|M!Xhk7&aa zMHg@a`r`*V?7x{?QUNYZ^VDA|OfblQQbGE;)sLs-iEaolz~J8HH9Ie3eje`&@Vi2k z=bR+8k{Q9;RK~^C_jlHL(&MwcyS{$Vhg$v6!FZm}1KAOy@`aRkF@l;$GHqd`HjwS( z7AjU^rm98hm!7IYQF6^7E-XpoBHdJFwiH`V$lFgKPn790rI}w~8LpX_Y?Dg+fd)Ho zmxOU7#!WZ?xQ8)J`W~Xo(Z7(b3J=xRS&1fT40-%ray(R7-|^_Da2SKR3$DaFvE-aK zi2s`sb7>_}sqC~i=|+fb8FbKi*+SIsB4v%Wo9ZDkx|ALHy=?O#E#BrYYFT@G0*m9p zN$QymXLB`PndBmaq~ufyyUk`q&f7UQ>Eeb`Q+U%OAPili)jLhesKrhftd|}W>@zJJ zp|Y$Wfm1G!(4lEu?2s=rD)U~)YWHtNja;V`or2eNQvPqc>=F~U2(k#m8r}H^Mq6u< zKA-@=_zIIs@o(hcx7}>@G}UgpK(X~5k%h&#sdjQIC511Hz4XV3L4TV~RIFvT>0cON z8eMUY0V3luv&6){HnfrA>It_HQb!uyX6-50tch%Jp{dMRe5t#harqLCcnXrrzH@#BzO12 zB^uyqkv>;>u;+dt7OchDCl+>^VM==e*`*#hI9XwH3VHFJ@gK34*!Fz^ovRH;5KgfZ zLFen+U$F#PTh^bVv3eW=aO5KQR_2V%+`{7UC4fa2Y&cEjUEgoEh|6jK_somS2`+h>Q z{XPYmzTYj9Q&->+l5-r;aFT7E(eD1$9_~lR>|DQpUI&_NFvmcWO#jV~BNu#*Zi+`W0w+^+dnB7 zAMj+V*!U6C-$FdDITXWF0D|e%R;%MwDwpf?b2dKTPvskeP>`dE7%9Tgc1mLg+h9+$ zJ4z@zbqaL~O?6|8^?lu{eE%i@45&~OE$}e9X1C)8+OzjA$KvOEM|Ry1!ZHo!(HKmj z0OKJuPc9CdIn=U+#bzlx3zH4S7Mcn&3m3km-kBH--Ce#l&-`K^tI)&&qx3={U6#u_ zY_9HYuvqM<$Bu(-I<-Ysv_FF*6qskcx$@$^0mVrzcU-l05L_Z8r=UH62Dlrmfh%i9 zm^Ybk_EuEQRb|`mT?U?MA}et+MswR$WVj>{?TYCj7^O|q!+74cT@q4{O9-hnSwOAT zP$-R9?#rmVVf>jL)oRnVlOW-9Dru%N$0FCbUto1eBw`&Sz}g5%iq(0kE)unZaU$bt zc?eaz@Drml3C6-`7-K&>-JJ(1qY!bDRgHv z_0k>AGNq7C0!cVhoYQ&@Bq!A)m{(BY@Atf(a!f%L;lHthw?8TXHEzD*yIgsC6eztU zk@k`xZ@!d$$&{w7a>#=4n3AG=zo|5t0=1aaE2MX+R|pNO!$FAtw0xsXY+)C~NEgU+ z7xxIxM=6{EmMyXaY!JidU8z3OM9opcNzuY_L3?xq-MxRi_15*NgtH$~!Vb(lUYkqh z$&vhy-XAr&cLb6!8RqL7T;pqI1p{=d zLw(~HdhKCnG{n5l_Vk!p`2)Zu<;|WV%v{^K7+(G4n`lO|n_?{zqycsSm? zTVGm#kddsxqrA(PdFb{=qjco#)cMlcST@n-3`r|BgUxBu79WEEad5on@B)f=hJhBz zE2UMNFF4>J9wIPOOG?+Dw3!Q2M;mNJkW&k?LW-=o)=_zA*Zp92k(y!|w8o4*ORoiW zr9C|76xSHc(f|6gWc)@P!!s1Y{dv zJsJ2--4y-0w&o$;!QBD!OGE5m5WAm8ZgZCGiywuMD3~@+BHV@!(B(3bx4d4mE{x0- zj`i>P{9Zv zI@9Jjij^cxg{S=z;3-d+*p!~gtaBow-c|p_OUq&-RS@AchZ;r-+GNHe#dbO83lL=^ zQ(|?bm|9peLq7GIY7vKFn8F}UW;uuQ=^Heo>0hGp1MjP?jBy?G)PlT}YSF+=Su%yz zxX7l=sg75-G-PVv*aslJg5!eP7hjLJ;$2s}xz6QZkQB+9YnUbvyd^er^1 zm|dc71^o=tOEQ&i9}6Cv*6>*Z`?5+$#4<`p)gJ7-jYf#Lj6f* z_mu>@H+U9ESeY6>`-g1G(G%Lq`jZ)ryA@?*Rw(@Y5Rq=R^oF!4(?_vuf(wdldOJXVGi`{PTDDqnTNrdF`mh)_~Ey) zvkGHe0jy(PbK)UD#>V9E>*qq@0Bk|SMogj1xvc1TpY8W1z*Nj;k3EFtm;PG&V$&)O zd$XF7U$Fq5+*L}+ni8YMK`LcQu=-RdkC7S&ZYh^QwMadFwoE!&joh{}u}6u?M^WEg zN^!P8wb4IJirKy1hmL2%Vc|-9rLz2@Oi$*#lP@@$B9~M@H79G%M(;9fwoE#gWYtkW zw`fSoIA8q$@O;btH+i~Sy+5)$(m0kzyTEnc_bUPC5AIs`8k4PWb|IG^UY?LX%D1oj z#y>wtXef-vfPn6AgwHxs#OUXm0%P>2hh-akLZ3!pUu*<$!yD-I*O8jkcCFTF79tRH zTp*qCbdwY{;Mdy`^w%XEzA2IoS;bW;MgzrD;vfUv;!bV*Vkti(eDI7d1MZy^XSVV{ zpF9AhBp!`?)a_>fansS?zakqhtaH*?q=oCaHHZ}KKT-3e{6*DrO7zqBk$tdZd*pw* z_%k1tZO$Fw0zuz2(d7BjMRv#})hO+d@hZ;EL*TFfQ3znivV&@_?}jcOf4HPQTaT36 zY00M(&DZg^|C9hx@PBZG!f+*M+Z-hd%uxfhWx5OOx8Z`qfWuZec);(Hz#_bml)4oF z8QkdXZm@_>)CZDfRkhRRXW6Rs9#`?->i%gybM695cP&J>e-o%~uhFQnQSkJV=_+&I zm_2Cs)1m7D;EJWu8AW?-dV_Ku&XH{+J-N*s+&gSKJF&Q>XP$~|jBW>f9)SVssz(9K zJGR}Sorfz%JYCP9j?SQnjFk=@^nsjB(L{m67!!_8zkw=eQXAqQM4*QCXHu0KnEFvQ zrnGm=@#5$*nYLtCx;bXa7%V=iN6KtV3TMVnP5wY={K-kE(G$s;I&cwdM{V%LL0{JH z+eHWygkOBEy_*F@2`Grf>}qR}>bsWY>Qwd%^t%^9$|4FaV`CLIggW)NqZ6|=S)+>O ziJs`rZW$M(tdk$v%jq}7I-;t&vFbt1m}a*|Lp=T}@tL|d>N#(X-pqC>njU@^fVq3& z7&`9v8hZ5#+OPf_@CEu$di&oX|9``Ombi@B*td?y{H@n#GLR=H^RmGw*CSF(0-|iY z*t=ej(rtN5n4)AJd9SzedGJ-01x6jbG!!`G*|JyO zqfehKd4x@;h?P0IGbH2P81(;CPOjF7u;I^Uu^J zE7H&?M`|@3{ygE$q{EiE*#PJhuHLg8EHYVYmR>$G_*fD6)1d82Myq1b>tbcAAlMwXwIr9PXW8bzrPQl-_U_ghNGLuBF+p3WL9m4b$ilWIHAx z#VORx8lMmyt8ysErXstk3-lwdR55}rGK`a2ES}?;FE35mt2?3O19;lh8#JffuuZ}a z$zdw5;yIR0)LucSV>DxOnA~I5yIN+2VaAS+4v9l(JQfKR;uPWW%~xJ-7$u1v%6?6^ zNwm0og$(4gY+sgTOau@`x?t<@O3!}b@L+k37o@3DP9JX%GYPdjCF<_jd)y@!Qd}eP zJIxg<@n$|*OcXO^00j4Wz=LUT35jt8`QloDAghtmh{n3b!RKsGJR^Fh14#w?G8+RVSKMgH{l1qC^jPFqMCM!*pZG+N9})9hwIqN18OunPK!sB4ts@0^>Q+% zu8NITgS=d!w))mTkA&jxuE+ThD$GBA(l~^m;|ZePXvg=kW~Pz^HyRQWhmT6L z$7HP3n;GpUlK8ZY^9}&@k~(~cTUUN4ZcHJkEWX|$o&Dj#5!-uMc+&lkWB=EE|F2`u z@(HO-eLs?({#y>GNS?vNMM{<%;{XR#D(n2igz?NTRq3En2IT=6Qlm;i|H3Dt8X90G zAQK5CxE^=O318ndb%lD_+rGsl#_*QU{$S$8c${=I(~`n}krYas>U2Dw;y9k#$h-#d z0e;d)zG4`Tj1toQVL+uC;+pY;MAex=l`cae<0{nG?1d4!jxsRQZdb0+$p)a)=-K`8 zgjvlyJcC2ui?m6C9o}uyEm~|HR8}3CSx+6rZ_{j>VIyX zzPSqtx|3pkcABTLSE7Sf$XrT3P`9%Pr~I|7QC4g$n@_-_pwRR_LDFxnf2EyUDHN5xzX4}hKRHQ;}trqunL}1F-WOSj{Hra)%z=;t;X!GjN}P* z!)T--W+~7_>1UM2J@*wvXON1Hvt`yOI*rCh&ux&56*d`Ks6X_CRPoZ=*!Rym&jgLD?pMr~wL*W{dw`1bPJ*cgy)WHav& z^6F|go58ztSmddwDq?_0brF;(yahX6G8=qM;W+%SM&QmagTLgnVj60rT7nRWb)Bv6 zKhd_r7- z)cz8kmC^M#)L{;vk5bQp{Q8F1z#;x?@c-8pNK%gY5xxF5}uEY#+pi2pqM zVm9*(UVro0^(R?49sA3sX~$Z z%hh>$0l#nT15688|AN!1P+;|@^vN?Ny0b}pg%T1lV62%sD{)iU9tCyw+=E%HnbAaI zOKmJ2t|@^T!(#r0FVKNJAJkW`BECP_?Wsd1tzMZQR-y=2-JH-esow!LKH+L`gUR%; z$Xf4n!h_ZwNxN@SX%Gn=+m9Yp!^rE9zwtu*1+a*#Wu>GPn}Dv*Ngot5Q9!F=Xe7*V zY-wT`Ztf8L)2w|EVZv#+ZXiCe_lM!mDpZ}qEYVs<`W&hBttSqy`uI8C+ob!CL=YtS_WsLmdn+2 zaT0R21#~vvgKLYbRD^U~AD(#y?sPI=Uca{0sl8oJdMzkhPR5H>u`yLjfxm z$Lmb^_pDZ2SXUzs=poP^=pl>DB6LX+86W}U^+O5HPa&X>Q#Y!b_QBF~sa5&7vBDlP zA9BC+<78}~V}IfC&;1F%!YFJIpx_PT^=DnPE!zGg+wcYQkGuVE-}!%QM$M8^)B~XZ z0YiMw7(o7q30+CVCH)^#^k*_I{r`|7_8mVbfd9jfevH6-g8VC|1~yfj;(RBiNnpvJ zu|J8EW8kR?ll2v-sR2+yT|Eu_?hVQbD8ljucphJ8Q9>KgTkmAwY24LV(dp zXb0o40hasFIVgPi30u{nJML~Lrr)grLbXio;+L4;147^20Yz=49eij6gB=Mdy)K9s+h^CPS8 zt4+uoK1;8*C%Tm*YM$akq^RNyuL=%|{p7}s6L2u2PRO1kqh-RWsTu7|b|&OR6r)DN zsO}X);sM?wE?hTy?==DVr8tV}K;ehK4{A}oI$sl|av`Z%RI;$&hb#9I6W^QTS!!jU zJ3wg2s>;=WW6aQCIHmjb_=X}cksuVTD*(HVWmzl*bkfCOSzH^vLcBhK(<{m8V?cXP z&(4hZ<1(*BfR8j_%IVkk?tu2|>}0}93>;zwT>*$>o@0vN)CJ&Y1)}3m+Pg$cM3(Cx zS7MBrLn687F(2}Q(=p0{D_0_9i&}%0lL3 zf;YDg{{6mZEao!^rw{VaS4}WXCvUnPG9Aj#q>fVZRKM(Efz@C)7P-F25%O<{M~o zlggX2|8xcfSL$e~TKohElr$Nx0%>W^Ku3mJ5Ckn`gYq1nFK!)R*dALRNVosM)L!A+!Pe+GI~)( z1=3r^^^-TYjK`GBlJ*Pt$M4BBZ=&paZaqjau0m*EUEj^jUwaz>1kwF^*ND0eCXW?J zP99e&UQw1x$X0WR1*-}hb2e{-i11&>*IQRrHsuHG$@vDjg=}J2S-kg0P;oCw8F%SX zrrb}cri;xB*I|VS6CI~SJjhRit5zPIj+oHRY8`suITCG!+I$e)9ljdnEA^*^SNo`_ z(8ywH>EXuvU^f~7F(Qk7SWB2kRdDNlJR8LZ*rjzan7GONj;+@n)_5~4{e6pY*t!KE z&WQ$R6Hg6xFpG#p1{m-nBnz^DU0(Rk(th~cr^1-)zMM;uh4EfwLlo!!;nwpo3aW~v z@rGl)&tw+d4!DuoW(H6!4UE(XJT_kn8Et6`u+#hBGkoCyiMceQswf7M5ZU!YWO~n2 z5TmC%NSHS+Bs~eViCjS_2;5A`&`S+cx zWI`1vyyTpIcIf0Cb!v!a=~CP!p#MY;uSs|ZgnuK4o((8Z{&%IW3@F)e<~wNYgpj~B z%V|+h{D)4gZtj0rK>G1Rjx(9C^&4k^?_h^Y#;wLB1weZ#A0~cf8^>{_Y7;?as=N{*WkPxW92TUHHZfwkmeB9gryZliFM6VGI#7iFEsrig34J3lv=3G#% ze6y{Q&h`9v`|nM$&C9DMEj`bv%yf4XASMf+&#~x@>-J}#ZH{BWx6imvxgIA@X8{m< zkp4Ca0!*Q83UHu{VoAZ-i#~*MO=3!0`M@@<_<+Mg5U!f-JZoK`tGpvU$ zp)!*mTW{rZIiOHA<@q^GId#^U4j7lE%_KdOK8nf-&80b;4nLs6`*NW+?0#a(V)+CvpB-=+Kyys|H_j#GFrVB}x z6Q!g|$E1%Tqwi7M__y>*p;4?>>>BY4fEH>vVHj z0nk-BE&sf*tI!#7|2Hn6{~kqEH?*%lrCsG(-qO zSrbm!np>4^&dha{w$?V~+J6)^4K1;X0owEXXSZ5Qb89IlwMw$A zKf0KG_Yx^fk3UaW<(A8{?-4ZUavKjGTexsJWPNHd9rMZT8Bdq5>iMO{ zVfp6S#uQyUgsE$e%)Kq*=PS)Jdow#VkF2epN#^wNY=-WUKCYdORyC-xw4By=mjF1> ztq|^3&QdY7k^QUsVpY!iF{yHaFQAB*G0Tja$TS+lL03}=yKg?w|0oj$w13KQo<=sh zeHe2A2A<#`IP&299Jwu#GF?2o2L<}gSyF=M-b!do)V6y1dTxocxmLO=X1m%w08u5| z1jgaP-P;Ef8BNA3XVN=UG{y48z_Fy+wa20)e~aV9iGhi=F2HUrXPa#Y4q(o6-e@VL zNwAXpy4h*-vw<6wKba(ZV#kz2NIhHfPb^GxOIx(NZ7cKGz+gA@CQO)DE-n*Yb%6f8MFMy&dE7EZPQ+fqgRoaGW9mf-K~4qxh@Zs& zu%Y#>V@iKyx=lSJ>laL_d?M1$06`NAs_boi5yaou_?-%16a1r*wlcRz6iCKOniD5$ z-E$mf9ENntXofwCOk8@yhqApY%Dt4ygdukJz#5}52(G?B^uhxedw?fDJ*wxxL<&)C z`J74le*5>~Dk*#s_JphfqYCmt=dgr}HzJWGSmt@$6Y-5{^kT%^p{E=9 zGX)}UlQ8BcX6r7_i=zaBqL_UL4Z)SfFa%13bfV0PR4RKXmCU0oit;u}R9<8gQKi7g zJw6FA+?}xmFM~Z8TY#uw@NjE9mwY<&`S=-H?>xK>nAKC01DT?B-zjo&3}-D_QaNu? z(M*wxX9_<891ie4E)iYPfyJ%+n{n~|xzQX2^wK%9@tJBjSIW{t z&B3}U4pz3v6w>X(p>iGE5#TAFQo9Kc_B(L+>OgIl%mHK`RFcS@^;B*>-vtIQU$AVs zEzr6G>y4(~x$!EePfH)4gTR<=>FX3Z@+F-xh42IO;_*~e*Ol8+ZDPkO10Agf*O~-2 zgUYQB!>unWS(}^1&QZ-~t-RFg=bB1{3)fFv6xN5St_F0Vyk!b;+4J^vy>tf6UbF|j zUxLHlZUHihS2>wfS3^8dmx`yd9XWG4PR-n1f8>**_Dfd+z?CnCbf}*8&`{q3A3OUV zQ@V1itge@>exm8smEfh@`%Fj`$3@s28ILg8ou#& zwq3NKE?M)Gum|-)A5pRA*-ab_vP{0Rl~5h3E@KSQ+ndmvGEhJ|x%w;2%iTC|>X zNyHfkp<10(Ab2t3TS^ME#atbmT`|q%Q>1u>-Wewu9#?Jh^s5ISfoBrxgLc%pH#rrN z#NW#u1BclUNq(_E`nj^haZ6J)xO)p|5iql#+&f~-2=%N`u8d}qN|(}-gogvmM{q|= z!zm1m!0*GN@}XsWCcu@rA$RXfl1k8y^x9}wSV40JXiDe&x^ljAnKOd^Yvn;^?YI_a zOVM|sm7#Q}${#Js|5hb@WA-8#eXEMMBmpqFC{Au2BxOyWr~Z4s)N+i-yO}nQ31!aO zhxy*Njk!af=6X>%Puw{11SPnc-IYr9&>|@pvJ1b5R+WL|s!WTwXIZ+CzqukPjXvFmHusXeE)WTUsp`?_&K&ZoZDJ7 zq2jX~^#dK}XS3dUsPR;sbl}Ti#L`eyN_IbU+R-l91_S|qv6M(pCrl~nW70Tp=V6$l zEeFT4R#nDBk|lBH;huzTs}o>Ot19ckrI{hB9KUnZa%S6%d)u}_yW)%{xy-|p|K(sZ zmecn-TtM;o?$N#rnlRq8NAL}T@j@(y10?=~zAiY%db%D~U>Exy!QrBaP(5qcFV)Sz zx|leph)NnIz7F!^qa2n2+!{?QW0d4fpqj-(+P3aFmqGVOg3(o z!A$bjycJW}t}zSyM`G2i&_t|K56KTx5}+0uksrW9Ip1W$HY(br|0DJr0|2F-Q%qro!+&*Y zKA54{93yCHzLhD0NtFIMrBi%k@`w4HyH+dcTWxA*OA1eOhBqselLW{Kk~hmBkdC1^ zbH#p+A@D5(^1<&#{tc6qU)hM+?l|)Wm_r)i28=wIv4jy{TnsOO`{CW`=-s5Ch*WgP zn_dNO+mb!B{v^~TqVmYp2`>NoR<7)J=)BFKjmB(ykYkwL2{*3k;CJ5}4PB#YdjP~e zh8SdiNLe0HUKrF+Gqo%#sy!>d$!Lo*8?#+&uAj!l*G3K46t3v4fbroYN!gAk162x( zENDHFvb+>$em5&Xxu#!UKn04bCJE;!cuJS@56{~;KbL5~rlH^UNg=YJd6SwXaA;SezUz5C zQkAUUxyiLb`+k3bFV|t-voTM&VBU2Sc9+$IpACDKg?p9(C+%XtUeefa0R3|gpy?p* z{=zriGKlXV7x??5G}0u`e^5p{Bf4SF<0`oI_C_X`UKK#6qRX6p7P61D##^-zuT&je z6+)OV38dr2O>~NeNnH9sU06r@UV4-x zPrW{HT7NwPto9^h80AG%r3V8!$NDrCXJk#aWzn>SS|R3~tco#T7$+A{^`V8rFfL5J z)5$x@vAsnj6m6okG#(iIgHfaF>B-~3>P2=7J1E@g^ZOnGR`M}d3oJ88US)y8SsHQWrQ{!>GOIRXn)7hv6D4}>jA##0NcXSVeg zf7(P=wjLf=duiH4yJS5L()O`S>HMZ|jO~mW1AVa+?%cE#9-X6#^gt2;o>GhMeh_Y; zjbNaiqXtnM0X<|L1HXSgeBMS|an7l;{HTZuAldCqbhkCqvCZaRR!o$xILANj*cPvQ za70S}l%b^k07jhGF-f7Fwy~aZ9$}GTgy6#H!^PQk9pWDe=O5|kBUz!b&Diw8(YUKc z_$5ci-$Ew>!9|R)1`C0P*u@~BzzGvYvW0tb9!-0m+x2VRWK+E92qENNBpYfWYeP+}Li4v-7ZcfPJ|D;fLWt?Mq zUuN`gq%qcT^|` zn(>qhUoQ?QAB*xm)8`w958_aq!iY&IAT|4Y%hEahQYG#nsrHUGm0}egmtTg4ITu8s z6$Z9jAfta2qLtO*l?XY#BTDip`_Q^G0;rMeU7V{wqPDJeoc9%bn5ex-an@qQ&e)QI z_M(P4)V6gQYvkV9)STcvM8={9cjw#th5rTk%*bj)xEr6Qc8W8KgRU0}7Gcvn;Owxh zoWK*qd+F2sg9Xaf%wP$ks&oo#(DZxP&6F_DI~zTaZKENb@7up2`RBivl$}_|7o!6L7ac-G1Z;PT3i3w}7ny ztq0@|Iljfq2HuM0kZ~X5mC*=zzzy1G?4#b)>Ji^=@H4q{mgl;8po4 zmn!R)7@V2vb&ajC^}9kuBpd&0h7P$>Z@ZN#Xty95w#rxa9pJBa-1N!#MN4v#oP5E~ zax&FRvRn;+uV=PlWgo$>;C8PbKzHvSQM~fH5XG*F+T^4lo}pzA1T)oiz?c(p?Q9h) z@hey5PX>#d6!lDN6e|3or2G}hjPpYYy(#o4Lu~5Zm?-k({ly#uL@rO7lVqGmpFysv zb)5HB{Tx>}k&bXf5BTsC?}{AwogDb@k*+mS4w(Dhd)qH*!v-(gxA-2jon(0*EU_C- zk+JD$MqLa-4OFdGDMPl8fC`t$-0d3*ce1ps;0aYwd~bh2&@T%n<_A48NVx9(Ex@^~ z$P51duP*{U#mXTCtIRK({~pMGQLhX^03$*_GD0Ao!(vZLony?S4& z)TxWPiJ+PJVws>h%`aKpT^_qNG|BA1K4L z(*Q$n3wn^m{kYzoGoqS19Ci96Eewyg(xkY>=~u5ZNACm-b$lI#mMv3iCc+^WL3kK$w>M&L4RZ(<*#k8y{IUd0R^2{E$|j>@~d3k6W&-V~G43e}yY z1~VDGG5(kVt9`qNR>@tkwHF(oPvPDnHkT7QdXa(drc(8nzHgA^`IK9&cyZ+;%=^# z8F6`p;+9Tw1h>av>t0tVY(IP9%H@d-vp$%!SMEC}FiW4N3;{5MR}Bc=+$Gwipy@6h z6&`isI-FMRs*8_Js!qqVva4caw@hFQ(nr~))108W6vt;)A%dqgQGMi zEZB?pWipoSe|lo7cA8x2xpY{bT83}PAjgg~fmRs_c^_Yvq;pp&&Ju1YX9(TZTxi_V|@Ju8*8^(MC8m2SIQuLs?lNiwo#^MlV+c%si0&G_jbxvJbz5@ zUyuPw^4*f8_Qu-k`ZU5IFwO9{a0WcMSR5<=nQ#sH#Y(hINtqV6jK&hdVqGH}grddY=IQvL?oO|G$Ire;=9u)GPWepln!^bGJy;0KtJ3kn4h{ z;UXehSSP#w%apNLVo@P1nGTj;PHU+h;_gA;ve2l zk&xW>a3BPbk=H*@ZM|(Ir+S**9qVdpT)4p!qL+Yz>BL+= zd>-f0+R$TZ#z{L>!?G;0mFnWfOc0MBqQ_L0S}3F8 z@0Wr@Tx4>K#L^}#2#hK}9d?b{N>}#mmS|8tAGMfdG>f**B(pq(ybzRZM6Tbc)L%RQ zgfk`<8ZOj~k=2lV3IIv4p#2D_G$Y|(*96SMnWyvCe;S(N6kB$5z!-H1Tg+(S{b2s>LjRwuAavr!qf*= zv*=Ka@B6U@f5Ie5oevP`rr2{wWe8r}gX`$x(<@hlJCl&Lx`B0Tt^Nct( z1doslFlsd|I);aV`}Fn0Du&0~(MDg|e>{#&0SpMUc13R)>B2~ZM^C_k0KRusnL<%! zsL90-q12sO^eImJhS_LJToAw{FD>V*hlT8G&6OG=dKRb%V);9jC!F)lRUkG0(eWy- zV-T?qW{}iyOTL@?L!o21MSj8Tj?iB-rcxNITrSC@Ml5^|A$!Oi!MlLaEsbSFwa;cy zl9|}Uz9qj3i}d@_F~BsR@b#I*FIt>cDArhL_di{_{~jXWdDlA3|2Zg}FOP@Vz6#th`Vjpx5Co<>F34x}Nc_a20WARxJR2-_(>*PIkb;&H zZAlbWlFh~(Pm&H<{LzqrNyuSZN`rD0gw<|o7CXGV+M=zBR=j_ah3Vs)Xf}lv5+vgY=Y|Q26qAta3l-OE z{cG+;PKO+1b&m`|3!6lZ!aX*`&iR7zvUg#yihivtG6m+!v#3`A&RcGc-4f-qcVuKa zOjaFAJ5m#`>PFC3=aU>9q+|8-R<<0M%KZmlUCA>hX^M?0-Sg&x(j+-x|4`paWs(qT zJM(ADXza)NC;+*97EQtAUaZwrcoP$6-@^UnjaYIdkXbgK(`fY?2uWH5^_jt=|_7Z-e(_0{#v_2&jqie$8ZjJJOH z{^3|g{r1vXEqLjMy$(MOWHx#;!yklw?_V)^gqpxG0CI8*qJiliWP`k-6G4r`Otq&x z6KWe~vB@39zwV~?ChokQO~NS^m#=Vyx8t29UA6CTEV^O~wa z4&E;m;*CRKeUsilx*!I3ZK(iA-i92R@W4yf>8rhLpjwwU_X~q*riFU&lL%oxf0vzb zFOkjoF5Cq`@K*r(+_+IVuQ0HA0RksQMEFI!CZsSvE%4SE)Z2)bPQP;$$&$&}mdL%; zn;R=}_D(e5DzEoXBd}K{yE%dY)?H0ImDzKx%*FtXC=rdr)_jOj)?_V~Jv!L#2d{3x zRxTp^ctyrqf7uMe(FS!nMBWznh(>_X`OK8E3&D<8%g>IMyd%Q__f6`v{0~NObAp@tKObGcr(MAOi?B86 zQ~Uq?cN$WE+Q(bPHF-R;JEgzCe=GGEmF6Y*9>5>Ooa$kk zUb6vTf3bVGecE7<9UW4V`~7jy4r>wyen3fRdS~`Ac1i5vsUFzz1=-&A^Z;DJ@E9ZT z)OAg}QJ2`#1^QuMlgu5;73?w8gL-3Zt?fr*D%%9Mjs-iW)}6ObtOw_hV4Iif&)w14NQ}AXF#i5WA0pb?6~yM4FSAgDa#c}EPb7^Lw+(AzIt68W)~97&YMa zyQaz_>O=8TDIXZqpqn((X{i5?t9Oj9EPB?xJGO1xwr#Ux+t!Y4+Z~%b?ASIt$&S-; z$DO2a{^y)=&wbx-bF8sveb{5nRkdc-^ZX=~L2kw=R@l)_c1dwg?B<(v{DopuEwjVP z4$t^IIEVzB1r2e#+V#@Jv5inH9d?zkg{Qs0+b<$RH8vPhJ0=6I%e1vaP$4>oT>#Zy zAm>YQXUwwkK=vnc&d4qlL7wPkWImd$-4hlfOeq?+cxiC z13fJD5OGWG7u!at>L2y)YAA#dR2WIkb|89b#eh&T3<;o_nQp_98!+-lDgAC_Km$Xo zj=;yt3?h#@XuuufJGIf5odM`G5s*=`vp#mlX%W~RH2tMaku9n%2m(y0uunB@sPK9m zRx=jNI$@}>;34}t;ghsmA*pVDE?H}w-Gt6n*QPdjD;J+ZJ=K~`KWe%q&L7uxa7ygl+t5gyw0)uA0hCOPXtF#1w2G|pDyts7308+4SgeE8044(LyV4&( zy4hZq9Qq+zdb5mw1Epp_^q0zu?@JahU8naA59(v%2h~YR03=J~qUGnH;7{C&x!Jvv>?yy zrz8kT&lb0gP0W81kq0vsK<#Z#$OG1%YIN2bqH?EqMA2vw%k~V@w)YQ$qs(L`Q4Df8 zdX1?iH4i}1!#f+XgBCBq14sUZorOD=iQv6MHgM?Pzuky_Krqjd8XDfua-!IUZWCdDA+6Zt=P9slP89pOpP;ENv%iP*&|;f3(X(N!HgLM}67q zRcl=ZTKPaiQns$;j|Y7|ZT`LVCw})l#rZ~NFga{~_E1b{*vmgLOoq41j_kiLo<$%e z_Ad=WEflk;HEq98(sG-l9cDX#mj4U6@V>YrfLbkf%TSURns|q)# ztk7;#8?He&yOXicLFV>nxU2yU`y5f8J4f-Ror-A&OPxsjiz>d9oCh ze0-IRD_d3LW3T~i8P}=^Gglx(PFb2G{|`{M1N%YdIcmsJ0BadqN@&U_=gCg+mnGr; zP=?7fM}@eC0=|TPk1eIUhczR>wA|LqIf=@R>hzqG-Ad7Zoo{v(;fEcCOr;3~H$ALb zQ7FD(gNe`uWx5i~FF{|WEyd+xQ5}`m3Hse1nem0uf6#oW-8<{$2N`&Whig4zwcIfZ zGJKQY>zFUVW-o)qCMF))pS!w)U2by z$k2>7v%dKidTIvKMW{<#Z9T;nA>Y>8V)IW4@m+KSIK$7#F+{t+&3IXVm(-K3R!^@I z+%>!`3Imu%$1Rw+m95P&PPx15cB}?f-DM%abNON3lFg7WQiSk3104gi?qzzX)-8a&g$o@Fo8M5sRWK6>ff0le8dgq))X@OpbOrHW)W| zp?N9Gol#q6&-pXwy6e3XV-6 zTAq!LyrBA^l^_)%01KX0v5Yu}ixF$te^D--gJ&OCtjP@WvA?#e55aog5a{P5_S8L| zqQi!q#8ZUb?@D{9>xuyN91x<0Io8Q5PpET%@G=IwLu*?04>DGs(0UcFPLQzb9lhlc zpYVyzhoPag?VbTsGqE`MC{~6PW~@23tSOFm4J~LNy`IG!K+s?UBP_$vFUr#|O}J-* zySK!1CV;kk!gyVMHkT~TBsE$q%`cy)>qwJZz{}(lsg2Bu=E-DGBBDldmDzmgyd_|p zc?LtkBL70_ZH9xlpY~Hl{*o5cCBDQ*Xt3k2Q&>1SrrI5srlhmUBeZkQhjqG&$!wn^ z@*u7EPLFaTpdkGpyFBE4{s%j_wgAJrE8X;v%mem{gX>3iDq@E)Z)G>h= zoXpwuNh7%nK?b*?^_arj0A$9oBP~!uH3+>ay)J?fkWEd-;(9Dox$|(c(_~vyuP$7K z=S19G;@?r^e{`zT@s)Q@5!Bj70i-w z#MASYISKMu+PCh1W9VBw^zoq78R9v}<~~CwKZV4mahOuBAt6M)M0P+c*{DL$Je1St zI>gi`H}L&$kSc z{~khxeA8CS{;EYCzo33LU+RV%NN~KY4-p=u&g?tbmj5Y0*I`iI{zn(l{Q>6s-%s0F zz>^^V3rIETO9}n_Rpp9c|F>@^!t@s$wO;2{gK!+{Gt!WA`kIk+foWVCIR$lnD-)y} zIuR|w4M(bHvZ;=qZrY&-eKl8inj8VzP`;$}a@dSHBqWb=*XJXR31A%39CZtl)A}q= zXUKTAiqzBff`VMt?z{H=cnjR~-~H%*-uw4)M)C!jQ72Rr#E`Hi#CE__M%p?6wA{DE zTVo`cSP5LJdro9f8`00 zd@^SHSq{1I8pmBx8t$G6*m@_0g9boRjK9q8D6pMdU)bV`nqKJQikg9QX57jaLCl9Z zuJ7nSQ%MmdIF^S}ZwHc{Di99g@QEsBqOnN7yv8i^!3);ktKpE zX?p90rN?kJn2V>~h6DkHKeCxyOO+XMLWEam71d;g@)SEe!Nw~X|L8R^is9Bwn_e>7 ziyfRlou%8oM({2HhDM!U{)r)V1a`q={9bDre3blF4qOuo=$(n#bm05RK$A1A7_j2O zq?EtJ2lIEzh`-=ca!~Ea;$3>s`A(qw8_&3o7>zo-^1d3}bEAskAZ$?MK8jBkuNJb{ zAX0Dbg2wOYcoF&+jEeJp9Cv@W*FjbjJ!&-{!Ys@$JB}0=Ko$7kU`HP^xeY@Po(tXX zlylEV#+-?44c4Te^e+UT%m@7y`St=zntDqO)VYcj0uth!$M1|D^uU;B-iRB9mv1Dg zd&z_00(^3ie%6GGzm?^Afz(1BIPQ~S!39{DjQg44AC?OcfN;TBHmFlqu|b_XE~rCB z=9U+J8!Z}LfDPjkBP2D}S@wwolDy5{%%S91X=suU8h)JK_mt_kB}Wx3-iI3^f`Y_M z%c@P2cey3C^M#6VFWV6h=^e&EVfB+7cFk^i!>DN?VXwAyE$7yVgFeS+rlhPMZac=3 zbKl*bLwlVfk8Uatf6^>(t19`eH?5z!h}y{WH0Mz|0PEjf0oo6i=YayHCpOoEZYa&| zWazG3MK}TDUTr&k5Val5O7VOZjr6l}XMWavN$+M;uu(EF(^hT{QjZ3GjW@b6*PJII zTZn#DmbY`haJDO0o<{oTuLam0PB9+3=x2BrKyKuPy?ok_@3o~rRW(aQ|2g+I!u)=A z(JKiO0-Ps;{*?P?_gA2m5HBorf2(eNYp^t zIG!h;bshNqYGYVC);-(|BSCFkPMk6O?O)MK%dQ3xQ~+v8fBGI^FvBqfksR8*^#w?Zj7qRWU-*^(^D&R7KVVDMw3uL! zqE5gFv%}8;piD7(c`%aGnPSIKtrQxI?v4|h=aE>*fIV{d7a(s7*cgcqzfy6gE+Thy zMF0w*F6}myN`r+EHvYQGSfKr_aS`n1d4qd0mhqRdfP2p0Bj`)`2vT>VxUP)hM|hbb zzfRb%Cde?@)A0!8P&A=Ir~Dbei#sOP-A6oNtKY`q18(gaB7X~6IAZS;OYVp7UVacx z{0M=(%$vlL0yFvh5n8Tr9Mw18M_8J6QVqZ|sq~9F8>JI*y9oJeeu0)oIOOcwya<#X zPm`-{vAd^ZU9G|+(bVXHf9RfkyR3-kMDKJ4JoU}riX*KAYU11$4DkaKvMbjhrQ1T53{PDa!~hmiE<+7^%myfW zK;>BAv3m$WTa4La&&U0>fCv6epX9W2p%cWaGqgfS$r%w^sllqcW3M{QP!stcYlm+4 z*-&g$7r$i?dx2WxucLZCh*q4aKHYCGxB@P$1J5t<bqSV2a zZh>&4LQ1^$BK?k7HaoC-cd`?1m4BI0r~a`}K8CfNM?L*TBp+ggNfBHP^#x8!HLi|4 z3i)0QF1fmg9*$6PWN`+fFICnM4$j4xo-@!{mw#1p%k8jWaZc?t>4eacUXH0Yiic+% zKmW_P_kZhy|F=%z|0+&8W6{7rARzhvv!o$MwE;)|&(ed7BqizVZfoQV(^2pR`D~&^ zCGBLw1Mh`^{I5+U#NgQfvq|&@yAAxm_ecN1=J}r=IK2{iLm>Z$Z%iX7(X;Ut>Kx`j z2yXI>0bzd3pA2Wrz=_yg$uv$SPNsd7_mKAbweF!B$^mukc23*# z*zaLs3NWn8y`n|MDho8S+#E`k1y=XoBsI(K%(Z2<6&z?*bJjmkgPsfh=Pk3otVa&H zf~5!}QAfg9iOE^arKH*7>9Pss;jg1?n?@2FF)6bxq2wxl08DcA)L4giv{P-suw3-IK5&-$1lu)V@b12)BPWa56y~!%M=nN#G@6*{4p?Y;!3+-x|3(lTb z4HROtT@A}~oL9=@=d!Y=1oi2^J$mNVRUDq1&f7E|*Ge>0Jc9%SGFxaJ+OIHZe!FxC zQBmtg<}Ww9Z;wGD(Eyvl7(%{Wgs{=$$uZ{JhH4C$T!~ zTkHJT{I&Ze*R3N>Q#QFD13!8t8$;4$nHG`br`;dk+h==od}+y2 zXZ7&<@`e5&!mq`8ju{6)ike>zcE!* zkn_Dpg>BTOvEB1TWH(|?lddYgYHWLC=xVo}fHJi)wOVn<-#vS}tUCz|MjA0aCMV7R zRf`K4*aH);yTk2obS0UaWxTzMIIAGYYF&@Ec9*$lF{A$kw`e=R@i-d-*IiUp)*4#g zJoS|~0Coi(zvr~W)`gxrToY@nUcyO^v9R`rGhInd?>^$HbtMcV5|yx?(R5PwlaE+u zO(TaHqxg{NkpB>3^D?^Wf|5^~;(_5M(?rEI-M<{V+Ts>{K;qsrVdj;;eiY30&X=C! z1l;+I{5E|w6v>^A)7{Ok?3}~wgD)dE>{SWQ2HfU6@Ljo&oz~6s710&_5ylB%elxhw zdFq;3p?=A(aziBxh@B@ra94kheuwi<4wBd-fGkW)ZIX8tjXys!9>(bxNf1s!8V{Y^ zTl@zt_VjFc->9MK&m0hg#HI|RkYcSX8r&b8pWpuo55e+z>V=1A#NRSxE;@)U)r?v# z0U$WYQ2vZD;i85y67!8oMm;D-4HK{W5SI+c#YZKl52r?8W0X`xGn0%~t3M^cxX5+x zWnMw5KPAMm6Gwpa)TsKduUoY(9$1V7?}SOcZ)?z)S`*UGjrF zAAOB!ppbXPK(o7qY=1Pjv(C+}F(;6^+-Gy+9fNq61T2V9BpIs?G;-b(QX8c)?30Xg z`{wZFhN$Ei$sfJjn1hV}bG!DxNB{qQ>--0QE-(xZGd@A|?Hij!7TY%C7ebd696sxo zofs+WN(%xeD;Nw12{0{BogB_If!-n>+{8kLfEW&$CMt{SI)TW_o)g1?fnBY@pkJ?L zzq&LMXn(O(LkfrKTCG*P*4?K4eOtxe@v3I6gC6|8z5ok2(o=S$oBhbgko@4FA~1Pz3BGnO4? z{srhksyIac1+byH>4~c}ssdeGWX~GRj5WHrqTvX10kdfgh6D_1KssrrZ1ei@#s`Is zch_02?v=6rsU^@nx0lmez2tbAhwELy)VNyU3v`lUJ%}f*gt*a^%y4?tlRhYJ-lZ>g zPdU=+SAR~9&54q;Y^3K-E3&hNK9B1i(JyNjj(>(u052MIlwbMro%{8ZJ_Nt!Mv1sZ zd($d+et>r0`=W4mr$?WGo70#-i{hUTKDYzEk!aq*aNi!F_zp(_eX^~ENGceymNg<+B-LEeQ1yFZSE_-lMpzJb=JQmA_03o983p`m#Gq@OdV$Pr!{i`2}G0;+4*-d)?%p#=?wmP(p+$S)F~UOzjn ztb2Mmj_&@RP?U3W@vLb!Z0TyPFKB8cu$FdLbhs%Q{u#rT3G|eh9;24|js(tcW0kQ7 zz(M2-ENNGE&LSARzVj6)uI!w=a&PwuVE&g#Js+_mGIi$c(b*%kz1u8tNUjhnMlLcE zZ(+5DE1k`MnG#>^k?eu4Y?HHhV+RklGlv|7VCwL~SWCa|SZ7z&sK30StFhOsCgO%M zG!fi0OyagYxYYv=*2z-dNCju@|8b=ZxIu+DQC>n1jJufgcJj{jj1V*e(FxYi;N5%s zY)#W2j~^g(bpjzq4D^$svtAjzdA(Pm^NBoE-6UAaixCE}cv82OEB3jOxff93OVMrD zm3}zLrjg}l77H{P;;TfBpQ6-x!g!SEV>e-Alci%KG8maJzIRmHqbX`}%34>!-iKZ2Bhm467g(qws4xYE1-4`-`&--eI}sUGIxFZ6 zSNz$%DtT3;-6qw%dWy_%wzv!ckoK7?RZio%5p=M=7ZA5aH;Pws46N5};kn@xU?KIW zS=jbe<<o zf)@M?rfG}2UQCB+GODJp%Q_d+}f^ShV*m%^%_B_@lknP}k3wTrt$}ZO0W`)nvu;*MQ`DZOBkb%aa ze+)aJ3KTu5Zb=BJoL)ok@aQj~?F=>8ER1U01!DbEOisGR4xtN`3y*|-LY)(9k}@+@ z9sQY{fNMc0Z0#Q%07fJPFeQG8Oo}cc@%l&qnZfN91*gT=)8T+b#`wxUSMF;+@H31k zzud~n+Y4$l5Rorzd~*Tq0-eUQ#y6aZUr?-|ZmXIH^**XVMe(xuyPW&{hY0U{_(sd` z%+&>~t)rX!F71>XS0}O|yPkd)+q~ZL*14-~^sU%re8O)AmyzCy02ogC&xD2w8r)0x z@uOVP0Tt8sD+Soc2eM>x-6{s!c{8{vKQm-fFScFb0v+Gs`E0H6+oPi~wmM`$`(fsS z*2X0qu(s4S!-&npRd7)HyxG$k)2)ZM#wD1#m{O1#nHzoH#oH(=-aD}b>}&(LNEX_C z;&9qi)hhL*wy(PdSR3g31Sz4#=?nRUeGMG(v`Bif!Td zO4Z)&y^vsuEZ%B31w_hdy&DXh^TM0Xop;oDIeU5eN=Wo$!xGy15USX`KqW5UmVZNf ziem`+WEb?G1z*A^e99|5~J(TLFr?vqxvZ-N}cTSqYmslxbHYq7$MdZ?g zevFnBeSP3_{$JnhGc>;vCmxm&D}wV49v}#W*_*v~sGE zS^ddLT>A7e%bsT4$WzLn+nyh_Lri-IACL zK&diDZB;PMzMRrZ>+csZL0KGKP8+j!-tS?119{dB5bR~r`tg%T!t*|Nib|X@R$6t7 zBl=G!a-~ujzc>1_XlP@A3LkAYtCD%TNy!37!&Sv?Z5}oBbBiO;t_?Fq&4}ZjQSKDY zw3>5NZT|GE>btAwo;RGxsiLHb0w~l5Zma@H=06~ ze?jFYL4LHd^Bdqqq}vD=&_)C?6-ymJi90$RaW!N=avAPSLAm&ScfSM;R>;z{HfPUH z=kH4Eq3*~?A>7^rNy$nB09HDg7qQ?)9B7r!48U&V(c)Mb5Ej=n9}Cv61MmkhQJ1WXPxB2Jr0D(i4#2LkWa`^ou*VSnANfe=HKY z`2{%Xq^&P2kK-%-Q7JoOYn~qlE4x2Fhg;(dvtY={Pr51{kKR7^im3M!7XxQkxrZe3 z6HFF2qIXBM4?Awoz4^UO+7{xQcu&Bc;udnj|etdbv=Soz5_^VbF5u+1 z;dwvLW^wqq1d|tVpN3V42k@u=#c;p8ADxY;!kMP;ga4ht!=p=Tb|C*86k6h55_8jI z1)G3aWJvg#xG@2;Dc)$ z?6%-wg=~#idLkozAH-7Ipe%M)B=o2PzS6U&r{-9Xh=v$}jB?)0IzQpP%sFR_P8r&F za%{d-dKq%3+={n%2mtp?f`sm3+!i*p#~qG^lUa$)1f)#`@d}7X6KSxuTctkn^_>T+ z64$)b{u_CCjRuYU{6JbvWCpuIP)xiTx&)$~-xgXXlI^YWiHat5A-I^#!;i4Dc;-Fs zB0!YNCy}W8z~n}?sf`TbGTpxltIzWrD-W`xFuZ=0Z+OYmZ~(}fb+iql}KmbMv|>IbGx?byQd1xcapB&jID z+Sxg7-_$L0qyfX{^&iQu-`L{{!_tRYOSw2|NiJ0|6NSHRMo0y+-hYet3xCqt*P~~M z1_U3Qgo-zbp_NOpSxC?+M!3<2;jzYj8iONOMPjL=q20IERDltcL%_r`Qa?BqjJ;*=HOpTUYXv|CYTUECBa1sJm=YJbB;%ES-^f8TxA527J!>_0+{ zm#$)ktBDs=KiR0qvO+Jb*q9AuZ5T4^+!0@5+=4m$E9N`mRMJe@`s_q|61xkkQGMK` zpLOCxeEl-!yo!SRQzyl@CgGA9j7{8`_hw>-l8%uf2g~Gz;i=qDu!InISrN$kg9P4> zEbgu1Ni_DCOfItzWp7*6*B6$0V4H5@mePmn62$9)?GGV{3Z)pQQWBZ=+Ji|;Z=`I& z1e)#D?TD+e^)Ka)L&ES88m>uN$=UgiPk zkroqWxuYaaBIKt#rv|mO{YTTYoWzpGmjr|n)GK zjslF%45NodP2m#zl?q}?`9v?9+Lrs$Srk99Z@N z^CGi;D>5~*s3C_)pz3%n&zfXa5_|ezR%-?BIPmN#UJp4@@2RFL7cfAotyy?d*^?Rm zD7e7XTh3a5N>*U}XsW@Pm2B@PutHz;n*?Y=v`XMF#L9a}-4C+=HSt8rj=$Js>bpg+ zx4(=$5?=A38gv}YW+VVE!aVQU5zw56b z>vybk>xsd^C1|)=dreDlXVouj3&oI*lBa}cglQ;UZ&4ekK2pd4RH^JcB1Q8g3eue*+?Wfe?lQ3}x*HzH`0f?M43FH{Nf2(Rumy4~hiD_5zzU{4)aXlY;X_ z(qga@#Q36^KA@%sKR!yx85d!L#+6MuC!MzuJXuP_wHQJ-6om&8pTYEZ2?qcPTSwco zDW1$4fH33HBxWHSNgAWex|Bvk=y&*;F`X{CF9?}&01)CGn|3td(7_e)ls4EM3xQdi z-6bY9P9Y;kw_b4&{|Bah1|Co`CypVV-k^+e9NQA9L4(_*){WUowFA@@3p205JfJk? z!tmaZI&+%0skT>Cf0VB_Vg43E+z>JaH#fJ7O*#ew%e69At}OxOtO=Z{=G z`CIaquqfC9LYlUFkzS7T283VK`Nhf50uNZBhtD?$hF>Gl)B_|FiHzl~1}#Mt#jz1h zP@k{>aH*96Y4g#b*qNK*qUswie5zfID<)o?PimQ@LGl*?_u_PM_V+M8)l3ZfmtaOU zwT3d_2ikj@oqvh?9zeyHME@uT^H?G^^w%>TBgdRr2`^z&^%L8BoGbY1C(=JQmmw@w zpVWo~deu64UX)*rzoB+buMNacYIdKJf3wXl`3|7O4GMi~$_~pm3Q=4gF zc=OQ>4At@>Hugg_UJ%CxFY+rrVP_z6#|j3kR0M|XpGyz|EkNx6VdPHk7EkwpvK3Y_ zyQ*vf3~ERIjcw5z?Qe?kh7atVM5w=VxYiK`cXJrN0+#rq3V3SgC4|mONKruEMYv;G zP`D`KB;-wLasbcFfPXRgO+>^?A8bReA;$(AgY6xHQfeApk59B7!p&B}FA6w6T3+%-q#Y*c>)^xYxAd{@QrB;qDd;j;w5LH~~6QE|PwH z7!|J~zFlXjFL%rfcqp;N;Mx%3caiW3PduQQdeeyd0)f<@RC(lKuiT;)6anNx<=puP zN;5@j-m-Mr2@iFcxKnoBl^>ik@-c_prs*qSr2tb>n}b-%U;|TG8>W>IQVPw;P_{U~ zJ8dF(W^>vq4PA5vGA3y+%kARtqc2*!ou6XSX;31CX{8AZCH=0hwJwTA$91J4ma$><`!fhx%^ zmhUiwL2<{JqQGj!DVD_s6pbIoXCo$%6mG!AXx363QZi$A!qB)CR}1zZ(d;$AJoGHZ zbVj%I$)g?rFoPyWTphD+{3>+pl(;KQE|uYmKqvR8Gx`a z`ilBd;$}f;&02)$r*V}LD!r8Npi>a|T{moOr=)BzjaPKii1J<6232BA9A+y}R}Rfs zOFC9az8`3iL)xGjJEDdDQZ7LZ_$pe^k0?_ZW=aYHV3Tm{7ECdmsZ3&tF%PNagCRM= z!#$9ecv~xj1le?A%lbZWGP1#(5cpIRYzN@*Pdy%MAaUy)RUjYA1B&KJI->M2J2KtIwQ@r9mH8+C$jK?JxdLPkz~g^PKH zeK`|M^F|i%ufdxuw2N%ji8asfLvT|bV3cMv7RCBqst_GA%7 z3e`)6i!q<(1x6f%0}dscHxRl&J!~bsEaOEU{D$%&&Y(9-$Ga4Pz>eKqfaa;OB{6_Qww{rTs%4@3U_8K|6}DH#-CeM zt9cb-QF9>!V0D8x>)UgI<&l>QWO&wN#$q_Ip-}eQ!0t3cww1Fgyvr2 zZR|XCgA#TrHiXKxWgM@-sZumBi?I znrVk^@W7z^=SF{oM+-@e6e?wJNakCh{>^Wj^)pa`%!SQx4XVpCB0<4EQDX$zY%mEd z`yGd9l#!m|_Z%zprEK|@jZcI}t(xA)FX<*NX=r80)9l~F~)H1E*M<8(~QeXuH) zIBYszpk}ZdCBl3#az`6WG6FK#PI%CEbwU>|B)`q%MjzDP5v4q zC=?r$r_Q+Cg~ky>Aih{-9LHuFAjj>GF`NWA9grkFRC}2d};1_75H*M4ssaXErZk+n9;hfEFlJk}xGe zfOzID-BI-XL?bap4oMU`yb#s{Y6B*XDOF6L9@;_;?LtNNK(+3Gipq5(E%!zl+%)OEP-WI9^Sa~p5h<(+=tz} zO}}BOp`X;Am{_~8*~TUn`vA6#8N=SH>^c}QNK{i}icBj9Y;#Hxw|XY>Zi)~zCM_z*p74ZzxCrx1gg5@6 z{BmX3h2~#4XAZEZ_LuC5a9t$>aU>8iq6Jg)N325=zFRu}gifXuaGGp_3*~hByFThc zE9sDRNC;N^_uL|5&`0+Y`4 z<~NMX%vbmA#qnt+o#SsQ@y z4_W{G5e&2@QpPwr#+cz2-a6R3_VCnjEx0{a#Juj`^&4KTCwvBbEL?cHF=PjR-Ro|6 zZ7rtu*w4Kbjw+qG)ZT58yGKZ~LN>&G$f4>PH)Xi z_VC{nS^_?Uj z9@ZoYw4#0Dl7c>0<@w(G7J3i(NdEELh-}Mj>-dhg?ijDaN|)H1aTifRmxjR6&pe4I zNWax^m;>2&brY00%PQ`DxfN9v^ce$1?jz;o_cPPi#dXz@Fmvm`$Ek^0%!8BTEbS%ea?*nnzoB zWruG`S0p&t@R8^;x81oCP!Ks~Y=n5~k0Iep6ab6VcJoI1q{=ob9*UL$L?Zj#uO3r* z2~$1S6uU3~rQH0#XZwr9@xQvP|D5_x5ft_W2+03h`m3Qfp+LTieLdafBK`vd_n)P- z3;9xVR?--QB484I;~QKOm##r(gsMUqI_jVB1yoMBB+|ybzO17mEAKx4tV;QMwKy$? z4gC=M6mgAqT20xe8QDsWHuC~ukBVhw0rN_)I*yIcV-d@Rhre^XlnSJZ0*+Qs`8U1& zH_t-zUB1t8Mg36wUcCp>-)_sk3GGn6XUCO?rlUOnJpe>xsHE9CIXJPyd%ZOy^mJ_w zbNJ_Mjjuo`3QZ^yt2Y@Bu76~Qm!qJ#|3h*jzAIUelF12}Ns4KD89599<<0DW(@??O z|L&8DQwZv+{CMFmJctN`M}EDnBiS4D{>Z)}3{oC^hFs}*V)Cs0{(XIC`IjmSX)YxE z@6?#KdMdzWyS|_~FzQT2v?Nf8N$JXIi4dHMv(CBdyW!qSOMt@k#hMn^Y%e)v#Zkjb zTgRG*gRQ-@W0l_MZ)Pw6yCtO^6O&ar)J{eJ_hR0LA6MEE%Sg3!B?O#@RIeRkXETyp zD+%1&>==gOP7sdQi(ZY_J5D2)Ruqet-Uy0{+nIop#}(boj!H782VA&3-BTtujEX>< zN!WQkhW%=Vi;*VarXns=PNDL!-8%mvr^Zr#yzO0kuH`+iz+ftmp$(($JH zat+;Yr-|Ntgj%~xs6bJ(p?pkC-TAgnO9^IJNnPIu!uyv?+9kra15<~2MrP_I9Wrbo z^E?2yElMV0L{@$%T4#;#173!@Yna<>d$sH@KgV_lTIQBehg#mI_HM7?cD0t31BY_( zvRAw004h{PqJMOqEj(s2gnp_c-8d1s^iO?uy%}nn_wfNp1L{dP8jcQ86Gr?c&n!!x z<}!uG_Q%4NraosPcbgcFMEjTq`SFoeZ!G|WPvKCdsrQWRCUrzr=rH)Jqm_<$D+jjg z`RC8P`$*n$nHYr8rYl8whtEAJ91*K!bTeAv>zTJ2#6$8-g80>nUDvRO2Db zv&=TTgk(4h_rx#Zk{Xj7x~|!qxJ*B{PXhpVA5R%( zTQ4-QEML~oN@*9!vB_U3#}~HiB~%@B`J-FLzbW%*bUHP7y4*6d+?P@G%e129OF_aj zTLI4S+(EZtne#M3tD>NG`5RMLlT=p2jlFfO9B9``pL!DtgP0`&yeRlOL4E@`nQ~gVQLP*74{-SHU>BR zcf6o3M9^_pm0_gdmbOCrg@Xjq#c^1)ol)i}JMh?uO8uHR+@CRy-%O1~FP(q8g)v*7 zoXgBALC?zDI_DKu27HwUzLp0ecOL6G^^GJHnR1Ya-KP5|mfeE)5>fev zsWMAX&8MW>o+i(;?*P=L@l2t%_QeD?ZIOfo)=lBp%=!7tXOf2^)LZy8z1Q_i{hzQd)0&-c)fDkVT0^54t%xH6+787_ew~Cil3Z zXxF|mrw{d~lxmvCCFgKjzlyV@`!RI+hax$dx`Jbfa9FF&Q2~RgOJIs8LJ-;N>=|dp zS=H>N(4x?VUTkM@eo1kKHVYiV=UTq;)5}p~vaCrJ(32k4Y=5G4iF2mOW5bB&wXYw+ z!m$}NGCE7<@@jfxz8mP`9FKx&$|8Q4p|{>Y9UL2IwyHlclni!G#A!3w&3s5Xm z3Vcb<(V<%`76;hO9j*=2Ci>|%8c98%7*fOeWuR;NtSi8tI85JYMZQG|<3&~<4Lm44 z%4PSnu@c3^O2#>IZh0$)b~mC}hWnx#PuVpsg)AiHniOJIAgeG&1Lp<-V68Hjfdtzd z91d4!zTr_yLZ@bpG7kXPXdzEXC^Vke7LwPL;;4ct8^Ao7i}V=6IOv9sCqxNzJ&!Hq zdH~&nJRaB56Guv&$8yH2PCCD|73K>U3a0A=Z7Yb9cm064o9XX@@9zrt_m-J>0LBLw zP!a!vKs+krgP3*4D?ALw2Bi`UTZoQO)i@;93>h`j=c8RW>`tz1toXdl;K7m@D{~v_ z1I`j#30R~-1_61c*I(rFF)J5Xpl@S+5E^5>UF3VC)<^d4x?d18fuTHS2xoN+BoGZ| z`QA!ndF)B*n*W$f2xoPnq-KfGks+*W0lctFHE(>dq4?D@Wu?dpZ_ za_!5)S>o;^jdgs1F^fy zq1s#Y`^lXQLn$bwe+U?c5kEp0qW$7}fhupZ$*Vk*7i~Gx57Lus`A8$%(~q?L1&5?= ze&wOZlk#^QwjmL7h4~j5t5LMGi&0g|L)V#_iWi3E5<3}Hz}K64BZ!` zoSROIhP}a&znK01hpcxFj${qrhBvltCmY+gZQHgp8*i*lCeFqi+qP}n+}QbY&Ux$m zy{MW$x_aiBuC9Li=5=2ek<#)M%Ee}(g ztcEvgN0kawF0((IwnK#ZaCgXfkr4*>Zy<6Rmuh-h$gfF3IT>H-h*xP6oxgE3YBU$G znl^%Y=dXAhh(xB80wXQ{B4pDd1&E@$B&L1pgPp4aE@K=Pjw?Uda5w`TEMhkuC{z~4 z$obDyTTecOOW#)n!;)ZH6k^pufco!{NrXZW>Ayg*Qy0_-{C}M_pg;ye{3kqA{py$9 zef^a*9MFg#lLY9bNdX5`808>oIyD7F1pjYkjWt#U=)ZFcql;Ta0L*XS2vv&q4MMQ|Ws&C$OrX$U-h9Gwo3EuTmN3~zWp zKfN`-CKo1K*5dpDOZ;N>W$A~z-V9&EJ&~Sh%lbxfet_<5P+u52+-?ETSJ-o^ z>aHJ}MhQYOLKQz0e=Gbp_$}Pc==7sI5Hto<9uy?-w@_WTFepi2wUF3vaif}U)WBc} z9f%ZId{U=B-NvATXjW)kdc-uSI=3`THn&~C79F#Qm*$XX23^wlE7B=NuY@6fLk zGEwO-cB27KS6T)mWn#MYUH2H*%6|2Jgk(8_SWlsTglD^ZdVcLu0Bs?8zn~r}*iV^n z6?o4DegbI#eIWw@@oi4fV%SVd^#I-OuDv;kNv0<`YHr!>bI?|V#^XIuf$%B!zuu7k z213dLhOHe!Mln5DkodqM^D=SM49&EyvRNVIynHp8L(}F2;sZsh?glhpFvTIoJkCqv2fn9r_T|@ z-!=z(P(4*;P5ZYn!%8r@l>Z23;Q8CLBM1ww8?$5tMfo?_PYAf72xSav6t%Y+$$kS& z?uYw#HX0T5UT;v(h{9#|C|niHl1~NPFCC*J)(G_cEui0Zyn)aqT?wqRy^AVqx$pUA zCuemBaY}|Dsv~bFLDwTxVW?Yn`+`^KHj1e`t6I=Nj|$(&?jw57q+EYcoDwgnOZbbM zse}UA!CGRI>C#?aDxryfb~XBCjrU#|{|55ZpJ$XQ6KUzWE>OEOI3q#viU|oo$?@E` z!uJ}s-~5MSTaM5A)N6>Xh*`_^cRfqp#F~x&MCZ2~y}8mgaM72!E zOI)-GF8!kmW=^9he0k~i4`CF*GNMTx6Tj4yqsdr+UV>|dT;3^w2!F#GAwkL&4?cWA zH*0|&teefdhI9WB(o7&U*dD~tADNb#l(t|vF0i!hcInC7C5$ZfWVgZqxigq7k)wV~ zU^|KqCJYhZzx7~ru%|zu+(fzK9U+pX%3m=6l}D0aiOw8> zd~uq^=g`FWHkgxBt22d_88?=h_2kH+nYslpF*o!2R^IgEa>DAIrgBEpzD6E16du{- zvRcYiKl2ZwCC1N_HDiYuL|+k8K4T~!O}aYOgyUL=3k4R;3j=)lfrB+_Iw?!CX?s1| z{EZHe>R@Yww9l?%AGuXPggJhs$yi1ne=C-rRrNffx^$IhKW#_bzgN#tpQX5wl=)q?%)0f;wN8zh)+mSY6?- zkgBRH97S7-mpGb)1^&i-C{yQAPRuq@!fBOUwLI4SFriyiad&+HmFG~?cw8y`+Ielx z6<9fL^8+oJLi2+)Jwfp&{}5|K{f8V=yv|e?R)79fo-5N+H}N_B6&xN&GOP;;rvcF~ z-nw&W-jtO#d_q^3r&ia1*)t!AqBH{O(esZI@08&fITu-)L8AvNKoL?+eR)$pSQw_U zKjZ1eblRja&fGje`79bW+^vym@gkb$rc>(@^6w| z9Bl8%rrLOA)Yt>l^HB1kl%m|h=ZG6Zx>;TXJf!WUqHEtosNTMX_*Rrf>j|RHRZ&17 z0w^BDC3>^0QBpS%_$6ku%47lBo@hA%6I;#Cx*rODnUvMq zZ=ZTL???}j%_G-eLZ>}}r_`b~q#z#D2O8WoKx;soupns1#>T{m7pZA6B)+&N%~Zu; zcdH9^aEb9T&SeAzTACGK!UH*4Bh|{zh}%kbteK~$3kK=UT|uJ$U-Q54K|go2A_k4P zJGx!1PN%_1bK_2OOa!*%bC6NrxczrjARO2=S~hGlaj%LPPqrLhqCY}QQ8%z`f@CpT zlS=?}!1QAc@zUiA%90TWc5!+4TZQ(z_0%0@z8TxL6IyK0UQo5mS&Q2W-` z{f(q^`TDUiO?8fIriD^ZZMvid9ybeo$FRQnrXQ22nM za!S05ReqKIWM$y!-aT;A>;X}|^!-+f!2u(7v%lUpp8LzO7!gPUKXpu5Sxl4G&AkwmXJ zdaY<9UEu2*&zV*vFQq(>)-ytb1PHP7M-Q*&vm-Fao-F3wos5}jYK3}^M!YpLQRT~; zsN*Le5~q*IKF>@liuJs?lpaixrDa`PSG9dyhS}#{EM6m5T}D;X3ucG-*Du_;73Ez# z#Ot9WjSU#JP#bfIqxtLNOCyCHSXUd9AC!D4ZC{-H;~2&;acwSy~^rZ&A;t$ zkYQbij2FAjqE+)IX-U6`EA0;nxx4k7I;Umsz+1F>m0^WAx=YM1GF%bz*3qQt2)ww_ zuD`f^nsMihY5FSwc>)_i_JQ zCLu4PGNk$?`!pj>ohT{rg1-+)kIxraz~Ii_0#`X4n+LBP;Q*Yzw)=53F>6+Vn?ojZ z58|f!DwF#wFYe6kt>n~-sC!#9^8HNR&$!UIXVS@mY;<&_g2|K-e=nZ4n8^{?hg zTMX_6$1FJj6!(~eK5khQGiH-{f>^u>W$vOLNj`ZA3V_nQwS$1@==P)#e&yno)CX{S zatOb2eliHZa(xO2|2)11L;M`sfg%VP-U%lN=--hf2pHcPA_(Z+p&|$v-6tp2pq9m!}dcz7gbbn$G|2(}uL+skQ&Oq!sxZc7KSljW$4_Mmi z;(g2R16=XGA%zP%J&A@3&f1Nz>#;KI7*Sw3yvMvYTdeo?45$%v+C;x_KCQ;xC?#2) zQ>s+V(YSIdP13jiZR%$JBX*vD^;>aNBY&b5ZKj{jqNTkU-!xn`wvyd?Rbte>-0!De zvMs?F-gm|GL#s9jl_;=ZLp>yoRNNr@3DSbZ|3Z|C=L2FF2|x|E=G8wM#t znkJU0*0rb<2X{e|Nere^_xGlPoaWgfVgUU1u6i`hT}u`>B}^S!V)T%~5|OBXM29w$ z>=Eygpb>{6qnQX(ibN6*#7oDI#8bzY#9POs4!cQFoAC}mAS0PUnn4ZA4adf35%Y># z2#*pU#KTHa6u`3Ju|TOHq`|E~)Y6&}nhBV34qwLe6Z?uh1kwFCN6>~|Il?5=g8&ds zgNhLxQZ>`=UcHr`nm4 zH~(5692%$(m#LvVmQefEr%wuM5(;Qs3=fsD2nehHt(w;p(4II_ozM0w)fw(mf4w~^ zO_@?d@N2Z$cU5Q2sUmHcAQ;E`8kBxUb>}iHT78CH7|<+n=#;p>)=AzuTd?U=Cg|fV z*?Y2ZC*3Lym}n}S^XlLU=KSOOq~H$XsDgF7aq7~ksXy^p_BFlI5KA;)ww|S5v7%nN z^dL~b`hhY6#Ao-5eyCFY~b^6r{(Nbellj{@aQN>srgcFneg2B;P+r zjyuB|HGEqjg{>xP19Z=gZZAvg(#hnn`ntqUKqObsooW=KMINGsGY24%B46C)x?0LY zc}0jEb9~GTM@AXPBdrrD(DZIu(x=` zp2moT%|&tH^%xI0BIyrNxUtt0Ca<($*ZajlS9L|cOc?W0tKnzO$rOQMb2)*`7Oq$@iW$bDTu0^q=FYRP(aqTugmUEMNYySbr7ww3J#sow| zm3yX`HL)2D+ZD*jjf^^92WJKi)z@S-L-oux#fMAp0dJSp$t8iGP%a20xKw;@nLP1f zrRULWLRpX5sbjW)m|lsAnE1BH7(@9W$1Z1@50z{=cO*y3lky9GMc zcc!61`}({b)HCW9Olza-3b3&fL?EO};jgmB{F+fww0ZdSQjdbHc?CASA)8>DV(hQR zuL_zCgZVszTl;FVLhsBA9(#_1M%x^Ljbc_eq})F3;uR*yf;gK`OJSb>2ui4NkI{opsd!MvB0B7a%$jFIqNz2%qG zV(wu5i*s^s3{Rz{)}2S@hO2mfvMJ}ECuj^fxqoje7+gm=P4=d|U!Y^!njF@-zK)yc zt=D(fKZ|}lE1DkkscDgb7-`pYR2FBC_G~b=3H1FcCI$Se(y>BypbVxpjx^pmg8Ofw3pMFn-;ovpFJVR$wE;|P*3d>5X22BDRj1UmG5PFSr(7^XEDIJDaJok zWGNuj=mMCGntoW0W??s4Ai$@{x2$9-Vn)MbW0ssC7pC8$iUDK&BNztyE7bn!f ztZvlVGCOJFE%N+##U@Rf*AC)f(iJwl7YD=Die$fXD|bn3uSc098G$=*40U})e#0Tn zd3Eeey74U8;#5Pr<81ySW4(^itm$Hn4uS3L-r^V2WyW2Id%MO^+FilDY(*^R3H2Fx zg#v)-#QIFE0yACD{9UH3Xr!(psoo}yZ-_^`3{AeWF`#0BH0emE*s5W9U%!0I$~|%9 zwp_J=XJ!9+(OM_V5QoGo;p?|-wg0-G!W8bx$wL!<`re#rE zXJc~zw!E&PwPWN+_sD9&^af(FMJLPn1{ZL~S8-`{6T0}UlVyD)z6hbSF|!|4!LGA0 zvM*Z>*U(xw0@T5>c25}rYGc_pjPH+C$kyvP++Zvg>Qpd2+B32?S&p%%5zIU^i{yd!ZsK<+ahu~6q?EBPtd)yfo{Gs#x zr=@#HiYo18vtZv#)Qt7K_|Y6}Ya&3!4XvWL;|AO3TNj@=)eNm0O^2xEG?g2}OAE5+ z+*osb<=~DJzwa@mohavCpbr#d0dGt89RlIf)@@W~pJPqn&uzA&Cs{pEN$% zi5(jDILopQFHJs<2PI*HKPk1E4Sz|^>#&NA8tC}5X%#K5D?j1>flvKUCHsG`uK$ae zU~~hG27&pfl3vuuz=#XHa7VTPz{iCAB;JK=4oT}5?vJT9XAVQg6oua1OgoXorJAcL zh8pnSQg*_^T6Y3N*P|qkB)kj4;`jQyE6(_V^5M5ISLfkg6gJ7&)n0O0dF89SP^MY@ z@p?)A!<-98gw`W;I>=YF8R_wM2F&k{7z>HoE49lzBucWyD>`aO)C*J*a5JMy^4%*v zE_)~6l6%cnk8f(gfx#eE!O&2x+7j~6gb|C;`R{iu#I-TK_TJ*o((MN1$)Pin_CU^O zE27T_Oe{y&zlc~@E;Tt(kIh~=QAo{C@lkQ;e&yR}5IM|jz1iAzZ}-u z^2l^(vga$T#-q<_j(^(60mOZ!?#?{T@9H-+GP!=?5Egk8wApo#v94<*=SnW^wkA91 zvJ5LNNpE3W+~ZLDn_9R@H9^+rl6P32vYP6+DAmD#oOhPjNk+5vy*-Ha&Z4DP47IV2 zm_?|=pSG&WH1QN;jZQmo%7{e>+j2M4*u5i=9m4_yA}VFnpS&{o_qKz z5o2Pco1Va?0>df5!gG^bFx@|1qYpoG7$Ix`EE-}pl_rz;U>R?8)6JLbZ;QMdKBy_c z%vV;dtGaKX$&!)5Nm2ZHIvd^~H~SET=c2(vXK$cc*Cat>agwjVvdk0a^VK+)z+TOG z^|0b?uu_qw$ZMfcrma(TCxa}uzhOz(taosnzCdA}tgd*W9FwnD2?|rz46TRkjW^e& z&nE@f1^Rg*;{dqAl|XkXU5x6?S1rcn5N9ozR!8df{7qs`|H|fNO3!rMG2%1ySOCTx7Mt?J3&CPNg9gnNOHlC(Ekd|7E zz;RzF`IKlZoV#)hkk(X1)#;W#>j-q5xxLdBx0sj^jRDep$H}v*LUfk({3r8R0DhDe z@i)Ct?S3X}tc?lL4`(6ss>~g8Rj@4>X;#QsGjkPagVk7&WLv2v>L*_e8qJ*#dQcLTXbEkJTgHz zYEL^`>1B6Nxduz|LB?Zk= z6TrU*jrA9a^)QTu1Q`k-fgx^j89FVxTLd~o#L8Y zo)+2D)@Ex&PiFgzTpKstk7&gFijDFRk$>HbZ%tBPp&(QemrL%p!!zW>x=fN{FAL;?63LsLG% zP{}+glYbq8VlSbf4<9)Q}kcn#ugf?8e80}We@A>(H1mz*ylQw4+P#@ zX7&V78{5-WbEwq01oz2MwF#cBkS?nBzmy@JeQV??pHFNq*KF&_IH@HZ@(UYvf7X?L zvcT+>q>o`7;WkJV3?a{eO96=KC;I3#!_o8rQhuDlZ3hNH6mGlH_5fgd=D8wj0oj~+ z0M&=jzat;?%?rx@x;?*=6+rSfJV1{eb0+>cwiEloV_DQx&sG9t&uvDXIT}?*n)SEm z^JK0as#L)ZXu%m%z^X8V7!$!BVSBK~g07r;WH~2VbOL;&^fQq&2 zTQ%P;q6Zc>6+zQ=u*V3MY@(1actWnGlP)B}6@rZjm0*^nC^SIRW!a!;@hs#As${_? z+OR5>ikB9u@?(BqAT?i@Ex`;dRHX-%Eq8Cuy7b$sfAxY3BXJ=Z$@`H$24*Pnec;1ZGiKTfY%jxZ7W z-yW~1RuoO=7ZYzB7%*3noB*iw0g4BF)F2cBC~p6y{mH6;{L2kQ|7mmYmFsA;?D=hP zkLp|7vbHD+r_A^5BK9UeJKtvd?M_wJz?g?zzC>ZVd#2-bw%I-`&;c?W%Tu!(`%k6N zun`qbTpR1!Ul0yeL&GwxvkBHP>s5?(^s0PZ$luQ?z#f2W2^v{;>zdYz4|{qnFy?;5 zfVtrh#vlmR$Rff^*z&IVm1eRb8$$)~H%^^0H^!8?lJwg0q$B}bN$KCo%ymZO_rx03 z$Sf$3WTuoow(0)n5+lDKpfo2MU6N0wpI7#mpeyPUIlS{Q(>N7d5L6$q?=7yLZ_;#% z2>=xtX4ZcU#65~`99M=K7sKPy<1oAg02<1uUOt0q5HtW=-)5@n*H1q`GWWK<$*zW7 zm>jEwt3}}f1`)NIwjX6jGw!5LQ?C6-5iCBy&h@8nGhSZs#ea{nFMY%Gepz!$LHp&t z%cMKny(q{S`EG{6{Q>2!bs?41nC`*YrFn0+aaC>@Madjs(i~xy)NM|AJ!}4DOT+Va zxlOtUkvF#d0`t$~{GTWFzdz1@Apjd%glC}tHTx$VVf6kBXO=ni@hpk+^rvGTnnjUd9T^jTTX*Hfiioc-hg1N)4>hB;$h z(>jy-dEnvq_@%9JRs2@?s(!Y*@QpC#Fqruu6zqG8D{3_P1<71HAMpIPhh16;O3B?L(2BnXJ{L)o)$ALa%dM3-&$I?w)@Y zIZva6zU!3L8{A_%cG308wR*Mr>x2`cwP*ogea)6h?qjxCjL#^zic+V7r{T1nsTOBTE^Vbnltf$qOK*hLI}tno99EgXf1 zfV>0xmh+!|yQGNywGqGAye%RApN;$G3!?x6KJ^cXc}9@Ml;kB7}8)Q{(@sjhFC{Zr0&CIVsaO3V9SBByl3TM^<` zb}7qzCi!*Rv3);AinwnqRZ-g#NrFjkTq(nU_wYFtx$N20~F3`6BL1- z@^kt~j5cy>7v_1JI;EAP;tL}$0?&zVxF%vvBvj?8q3YQ*7;f^p^fc9FdU))djoh5C zc`5c4Pls^v$x$4xW}}i%!w&>^q>UH9v0QSka~qzBEk}wTr#(PzHx%vZD?J$HQh6?L zMef-*88LE1*+p=q9L@u{2$#IuH;Lf2M&DZCeWH-1x#k_}F^n8Muk%1kx3!nms9?|vsGl@^AIN2aFQ z-eeg^z!qL7DZZK2h3fB;5_4Xcc3VbpPe6{gJ_0YBjl3=YLTW-&m8P!ImejY6gHj#m zB(N-5A7xdOw_R|rLM+nwUA&2BIyV>80>>!4#-J>?j>d%AHqEg;Z>5<;Pr38#n^~JR zfQuAvkz?xC6g1Wp*N~fTVwn|g`3h>`n)A(7-+5Z!a7+xsY&+q}0W49zGN92}ZE9JniX_?C5U}QR)FORp6jJzwvKGxkN)=_%qb4|klOSH~ z_Si|o(`%%%YXS_utV4zmJ9U0&MsMG8rF@r8_T01XV#?f2ck*^N+y$Q!Vfo5kdr3%r z&VY~-_w#GibTKU}Auoih<`9IN^^*fyYez5KlBL{eaNHsCBISbuu{6@g>MND3HpjxX@z5 zzEB(RsPo5jkkhSOL-92TjB^YLzgatAyL+N_OQmk}>-(#QdAhFYFF-yN;yRt6)3(KP zAQoRd|5DX*l*C# z(BJ;?22pX;MR{L9M|TkD|3H(!mncx0_yp-j{$pSs?>tS1gZ=i60|yA=#6k+Zz6C{U znhv9&h5ZjFsYc&O`z0>e0sD_U1u2lJ1Pd1Et4VRJy|)=Mh_fjC73GrjGOeF^kncY5d3_3NBV}f{Rg{yO06`W~@&KtB5CYd}DTxJZ56s1c87mkh!cRDiLc9RjY`O`dKJRzSlk zlw@W*prd6nKD#a*BK-BkBiDiJjKdSo!Zj-+wDqb9mbHFdb^@mN$$bounq?nx;COvw z*FhT^;yUj@>w_X}cV`okR1%1@Yce*?YRTQc=uqzL2^i6;lb+>VVngs`^ynHdJG7Ji4~56-LaJYZcXj_^z~bZp&m z4SU!X4dSf~z~LielOf%7Xd#xSHN zKE{=UIK$O^mcLL++Td`Ww0Emiu@x|fm-~Yj|_#*(66^@8Gbs1;lkyWp zDS$nn#KQh+)}*9JQ5{ROtl>t)ypcm*^){lZ(Qzd-(NZVnvMr@f_+d)CK*ZeLJHJJ| zLRUcR(PhKQ{S^8#mF;J?r4iI>d{LlrI$Nj2U?iI|C9mHIgq!)b=4v}Vr1umnsZUKq zI>*M#$KA1DTYC1Snk)wT2?QXqlcM!l65t9cRoMSTA{|7|7SnyrlC}8@_ec7OfRLsI zQjIU_edN4ZeJgg6dXg_jl}X6Qo(W4{2@xn)hq6#}Tw&Si>!4r8% zi;->|^anWf0?lN|Fb`)^kwBH-s2Xvhm^20SbmU?`&$niTJ-$@h_ z&sp#!WHc&-sd#{CPn5vI-wW$1V!5DN z1@(};eagv1gt1DFmKMO5i;lA|t$!q;rz%5XVB#c?Rh!vUS88QmL#UIg0*8{7aVSAv zb*vqOmjEx!lF)9ANv3Hn<-}?@>>-fcph}Ko@kh3lTc3Fh(j32ndq#?e9Y5brAVXDz znx3^vok(fC|7sWW^r&6UB zX(DCdmvq*%Fen5yEGK~wZ-W4Vb>ta#%5R=xd1ybV6kiLl^Pg2?8QZTO){5brWHNYV zUx^sd=XPB;JBF2~r!JEbQ@zt8x5h2a#=G<$Wlu8L#Zgt$E~eWhjq9$k%zbX=bHfbg zton81VF(@zrVRKDl~56toQfup>;uyMih=LSi!8JDBCR(xGSBRTLOnEJ?*{ENieXQj z3G=`aZYSRpg&)NR&GNg1$&i-c;}*wbbveMO%s@NCF<@H=QZVN#cWf`W(RP?-Ejrci zB>d%!MijAQ8rEo3^xN7df;8CLq0+e$>O#`>yVM$a5*pUla<A9$$VT^AeRXBTUoNJC&;L_O|Emz$eyPL6H0kJ~4* zAx)-}Ae!9^9|R7eV3o)5Z3F&Tv;JN%2hVohWZeJ|MK`Fw^z##rJ)B(;XaOxjFF%Td zl0G17ADGZ3;Q(X^8mQ@bjvCh@5FhfHK34fU>m!VM=aZM3f~fph2DvrB&f;#&ihx0C zK4R;&yM}voRp>r~{C# z0FpUlX`9--*2j=#-}ikqk<*FB{~_V{ zZ>8wcjLsP1ze-WsJ7E#Xe}@;IbUaWn|FpvvlJnLKfdqnJfneeK)U>zVJzCa^k zFd>ky3ZXCYasy~kfa4!0zl5rU}|UV>RO?3ri!PI{<%S3EnjOtM_Qxey*!8udZ9w2 z4qq604GXE%E7~o|*@op}yfSg_Q2j>zehTr&^Jqs^Hd ze}|T=Yq`3zbKYfKB61ilQjYIssj|XQhr}A(}|WDVj9@|;2`V8{P^C7 zbaKegIa%+$R@XrbpMY=8tj7qQF{SA#yRJVgi|Ooui7(ou;ift4nuU`f%_7Hk!N)Ep z-m#q<2vauuq)u*oZ*mqqqb1h%14{8kXfHMXnss*2RQR>)wmB8?^%{ua39In z(9CPJTZHgsaP5xwV|{!dRsP_|;>H0lpvuQjF7QmKi?z}WCI9Pm9>a@M`%nJLVJ zDx>M_O%HO%0N`n<{$=k}zOIP8y4DMgMQvz)Ml2?j(F_Asg%~OdqQ@|Y##meS=0$`o z1d|AC|An$Gs3W zJCo2dSLP-EC&Wo-9WA^)9kKV?k?wjH>N*RMw&)THneVO({X`fSoS}7W$gXa<1plfC z&T^E6xq%bkJWUSd*!|sZ7pF3nu9E_R6 z_CIRR%Z4z#tw4Y1tqLsn(c_#*(robpp;44YF>Zg3uLD|$OwTI_##n75OOumBf{DR| zfyqeWPdj)^q<;8bgGg`XZt!L4MtGRLAa=971E8SIFNugVr6mxNN7^OE;@t3h^>uWm zp>oMp>J?Gr>aR&IL3$K)inpjxUyl`5KhWp{n_@ae>si|SRSqpf5Oas$#~GVNhMc%= zU}u2XmHGqN0iu2}Z>9Y|-$Kg5n-_yxRxjMWGqlF;x&~uJ*Wr!TA+cbBWD{9KcxDJs z0U|W>F@Kd97e&E@tNGCfxg2t_(umJv_NqCj!vkPn1TNI`*h`?u3sH(8&K=85iJf|{ znV2P35Ekov#nf=4EM7i@)@a*Z19A)0QA>UKIK)x-VfO_GmAZYS6^EPX%u>^}UuEa5 z+a*3A{t3bU$Nca?sN!cS-}%*6na2P!<jyfF; z?4a%%+(~0gkHgL9%lE^{thSdaz*YcMx2abX4uV&ppaDAb9Q4tZ7nc;6X3$R&z@ci( zHwDOC;weKdXies^BiQz0NM{6B(t6_FR3zc>p5!P}fstZ)0sFt<1R^kS6I%VKIa-Ht zKoKvTs*vee#fT@jZ#gi#!~$awfz&X@ zXNvJ&AazA@EW3z|*O{|c*Sx3$Mb&@2QhXE#`QW04v63(0!K0lxN=-|4#f_;igco0A z7%dvz+7Ry2-`-U46SBMM<3Ja^wR5s2845ewp8D=7< z<4nGrfv#LR8gEhtw+|Qw_~)3hRQOvg*%nuhz`4jJ-LEWrUE-?JSRvdl(oy-075w-y zQckKZoJwVb%BnY8Pwq~s?>%cjvRumoOo^H-n(@Z?OAkWeQuBwNigSPwXJ$bDWcta0 zs;S>BNwF6>-^;Gw?YSlS$?eA>3)liBY+Bi0a zs&OC9R?wzCXy!IE^$fp@Oz+;)pC2_ZJj6RNnLBayA=oeCev-YOfB9jkW~g>wQ4RW# z$TR!qPy>DvMLgAn0CKi@%(Gw8ozKaec8W;@D0hy$^hIE;Nly$AGeZowfv5qYc&Gs} zTS`wdPxKBypE^PWcNI}T#*06Lx+0%4Lb~=4U}&33ww3PZi5%NQ_vGp55v(Vn##o;hY_~OXa=hrJrA?gwditEyanst11KF9a+-&@dhJolP&Y-i z``1txucu!2GIzN?ciV1be{^011&B@IQ1IvIDZabR9D06a(+uHxZ7`+Nrw*D>a1*TV z7^z;i@%omyb<625t{3o&n@)!I{Az*eQ=!Eg+^n2IIN>+VC>7vcmT!1LB|L(F;x8I@n4rRM0>DiSz zRjf8qt34FXBr{=%YmN2Bj@$;Pl}|>dMcyMxW#1h zXFJ&7t$1xK0L$ils{?~Xf!;i0EG^L$F0z@-=rd5o?43f z@$o0fn4y90^MLEgXLzyqKe~;Y6tTv$FRW-=+>Ej*sYS`<*Ji-x>kO?X=M|Fjj!ZuW zjvOLnd!h0!7B$$U(G*#+#EkGgy18{IQs(` z)O3`gk$s+c%9^oLUnS3m&w!a5v)Ax|57 zta=B}gSG*UJpJwdK?9um=iHlt&iYM2{yR@W_9hDt zv~A&|etn#?t%K>6JG|P(X@h{pR=Yb8mvILjwrW4W3caTR&$Vyt*tgJ z7`Td9LF!`JeHDB0xJ7b~7`+3#V<0wZjfbBD`BQNlRLrCk!`j3lh@8q98fP_wE_5D! zMNqyKSn`N(EvUE(%-w|;di>KDarj)>(s^qT#I@Tz*RPs*En!s;fw1!!kIc+N3P80> zCMEhDb_R2Zf!@u?+&3sbMzIOFa5pK9?gRCAde*d(`o8y5C{Z%&oo6uAVRwdw2QT4L zyYH{;%%-|q^2|huyHax-)U=pl+TxTk5jvk(&ow#baJME0xb3WaaYCbgw1^yW8Wgj_ z{L{5FF0eg88W+boHs*@|94&(F1u%xHiXmw-@4|rFLgwfdIKL0yc}gsU5eRF-B?PW` zCBf|iym76(p;*nT80}^FW&MFRdCTJShhM~xpz4h$D&GOmV@>=G5#VL<`JasO|9JWT zJIm2>H5`Td>NM6OfBPo??b|nNM@9#rlm!?WuzL^-1{iP3jSOViPrc5qtCMFLIUip|uS>JjP)qdk{2!WjY$ zmf?@ojv9Q@E;mL00QM&Olr;uHDhdc=gb+~dhpfqtIfN#JS>M+fdXeRxhI8u?cBfP}d zCg@YK*mQ~-f}EAF^$@7-yA~#8v#e*(l|uhhXsFJ3d-oT9YzCIwZpy<+WQMA5BZ$h$ ztcW3SP=jQiSXCyEa8`!Xu)hR9_laLPkt1t{LqQU ze6VI>W<=3O>P5*Lc4(F3*P1xR0KRYOhdk_@=Lf@@;eR-^deAwFOz+vxaM>x}Ah=?_Z| znt)A$-eic)rhUkwA`RDfeTX(9A9@QEaYk>!^P(AIkJ%=$G|!}-PLe`sPohrxG>IbQ zLQ}@LqG!2dAKSNiqQErnLKH=aE{zM*@?&{Js>E>PEhxRGkmw2W288R~ zLog7h0~Vw66H2?kVQhCwd-Wfuy8l}9J>*qT6JGn&*>Xe3MJ0A736Kkz4n^WY{tkO# zTH7c=FP%m*$9-AcCx^sMvr^WfPwI-grZ&y9AzYTNT}BQR+Jw;ycZeQxSS|*%To~R# z7)D7uwJ{A5a@eCC_RHF62`Z(adIqX$WVJN-MV+7*6eZVCw>)z+-fs>r?$;Hp4>p?e zJ>nN)#1@?A*84_Ny+zl6W=}Uuc|W5GKf$SAb8q|fYjX0>r>p=kh)`1niOTY9wDgHC%_V{DM7D+Sd&JjQKgCxmK*1-U{+4&MhcIr| z`j);lFkw+h%9^;aA!b%f!jitA9z#hBmbwx=0p^06Jx%#4zKu0=Q)~uuKMU*iKit&+ zM$iAUNB?kBt@q^~ub+4d3i9s(?0T7lx@kBE3GaVMs&5jY$58(%&#{0GH@SY2NNBLh z_O72YCQKdBq{-hEI3bgxaVfx)z2-R)lfSQVfF(1|aY7^`>2W~-9ctRzxT07-!C)LV z$;wh+zuJ2ImYoe6@W}ga4GIPg|5cJpmEMj6RI_%ae*)qp@uS-su}SY|Ab(N`U$n>}vjpAy0s=Zm~&!ablyNTDEBgQU}PeYnP(TJveNK~P;1m*}&luK!Mc`@9} z=j-qHiZe3LO!R-T0qn|4|;AEzBMkB$9{RYnLl1a%iII5&IsNqEYjZ_v40kNT2HsBe9DkUy zHZwveDDqC056^!%6E|XO%MZnsJ{iUJWG~OSvMoi13?YndZw9m>qn!^xXaTt zci`SWnme=^F{C&c!}n0VlGEYn$P}-6H9luxFw3OSIM19|&p46`sO%{AP+sOQA6XMPzAcCFhyRCh7d%iQn8!>*-HyU^yrQ1Au$=9JE@Yq2I zck)!=oDT7V0JNKbu05lW#?QV`6Kh-&ZKGf?T|ta!atCnj<)U@2nkphl;oc}ez7i1c zca&y5KtNbpr0Bb$C&tylI6`1_+19gej98PTN_IYNDBFTyrOK{a5-|#&?y_)k(J8jX zqhU(eEcO_Gu|~SlF?^O9S)}w7Z;aJUqH#QJlwnzrXp5<(*#q7_qEFz>X_ zxa31KnpdBhlJf=UKqGgb&vA&xGlHub&xS{m=avSlthQ@cXEr5;kF*!-U2SSX712@1pD@8vB+qD zS%PExL*7L#9eU!}&D! z-oU*v0$t+YH?<*~oqu&!&+J_ZBx$ekkAMp7p$An@10g+;*U`slXY()RSgYnQvI+6W zd0Pixk}NVvXe=X)8+zj7hY~1n5S~+8L^>?6+hxv-?eH#9I+ z$nJ6xf`JlTuNqvh7?w?%nr?x$M&0>n8?Dbehoo`bVezv(jxn$7X#ufp+Poy_Q?cyq zLnWg3n6f@k=7|ii_puBiZ)WRJRR9u*TTx`ZceCeY$5B?tQQK!x@$%~3`;YOL$L%IP zpoELsXg!?7fh)SMD8bXPmIBB(*UlvP+Avd2NE;6p;m81Fh%C!MAOeZ z{7tW9PDCjv&l@BHjGJ#&0wg;Tn04;+;k?&a$({OPLpLJSS^dg|PhYbRCs;|c60dcr z8+U#pGdiqclDGH&N$z+e){Ave>&*yp0`%B++CyMT^w1TYQc_XJYZDX~2jncmCVJ;M z=I9ru!H54Lp^A$%FzUb(9>wx*awY1^?WqXD;ex@TkiVK{e$q~1c8D}uY3QgTD4Z8u zN%o^{W8m}n2#=3J#x+r(JMQYuZN zrn#_=5+w?uU!6}$pKe_M>r#Isxgq>vfcGU2bl-J=p=&Cci9(4YP=@&7w`37C6jhp; z%s>kF$T+d8d|XAA8QSLWUsko>dSv3sdD#qN324<;DAa?)`jzm z@uzCp$cM4zPom%uqmFW_Btg2SWB>hm9Oc1GaB2A1o#AUH{c}+4oA56|KpSKDij+~i zx+M3E62EdMGLrFF`G|Bt9XzKgvTj&qJ%2VfAR6^b;% zMLQpCCrb9T#7-;^<%`z>D4fnba_TE9xvV``M=oMrIu{?f`5K$To?=6mN~~b72P5iJ z4f++ejy_@-+#H8U^pbCkbPX9c=$E4 z@MRiMuZh=YG^6b)CkdT9Cb$(*qP8)nQP+seS#(pEYq#jy4cT6ev4!| zyy4LAmk*^0T<9>RonZh*H)J1YBVK-M4=zm;8w(`!w9E%0AYTo;V+x>719#<75lpSz z=}zKy${gk)uq>Ne< zb7h8#@t$@`%WOx0=E0*UnB3$W>DMqbQx=DdifBjGeo7Ypf$EdIm_L=cHzE{_dmN5E zDf^!4@h&x&+o_DexB7l~ z?FX^D1Y|#k+K~&!hY&TDsnKOhVHe4+b;TLrd(<=e3NCax2NK57pHB16rM2XJR=e^@e1xQzUcYbzX{H!Z)ZYgM9krhs_HV6m4 zV}5O2tq*Z=(kR2I-d6kS`NJR|Pww_5=X!hzR2T03BU0?739eXw>&H)Agu5DZd;?t2wPRSp0S^}h}-r-P1G5Jzysn5Mo z-o&g)Im~kDV#Byq%3dfpVj?Wk_zIUY3t_?Egn8g75;RIu(1AiyT5MhvUvn2->sDd{f2XgUWAN;8?L}bft(;v$ zde{#2=CDR}%wPj?dt6y=oB>eqVic@bN6+q)PLrOs!0X!_6_SY?ur)!1o3dHPPR>IvB>s~&LC)XNiX^jVB zUov$`Xbt4wquWzBvxAL!sZtbX5^GY-W!h;Y-DcQn^sKzCBnJx-CZ`a65;_ZgX+~*N zecX~``{<-}XV{_$M>2Itn~Jldx|>Sl8xCb!jJ?xbRaRx(J=>RO`i2%jNE}9AU~?Ju z-J+`&bDR>7Jf*FpZqhAnT<)#+^Y5PVQqiE{4@bLTrO0l|__(6|WOV92ho-A#jb{=( zm&4$wixRnr^R(EfqI(&bBXV2P_Lxl-6V_hPYm*l5ptV0nTB@_qo|d4be0D4ArX{!# z)Hax$T!O{d3TY@sWcCw)BzQU#4FSZ}cvS_OjOtPt2tQ@nK?aF^VO-5jfMp4e=BK1; zW@D|UAOG9BN>OD6zG1%8;IyW4L7Y<^jb8-Yj?k=v6LI?gJ{hyI0_A*Uq?$ zyeR7BJ4C`i7!$(yxnYSn_b_>7xl!#x2Vn`ZJLk6=Y6RwUD3lU@l^2zefQ=C~`OWZa z+LW2j2$cJ4S%|C+^lgaMLVBlTymE zU?j6p;8n4{em)5ZbN?wDpaP&DP?1dZ!e494t3M)G?8lWJX%5{jf9M9`{#oF9+|IZ= zTGYQYjR}v^a@*IXXFx-%)GCT#DnxhwR;?g_YJP|jZ!KS>T(dmR!XAJx%$`V1k2iH#iNb2$PzM0kF&DJv(n@_vEUCtbW1mE% z@`SRWWvwYzLdP{IL*6K*&a1gsv)o5+=6&^99$|;Nr|4AFAq3N=fJM7l8`DIDnf&&b zb4cxTp2)}tLr(r2MQ5;Se?&{O*Vs5T>N=uz69jkWYlte8PW&hIs-OJleY(A$>Yf(M z{umBjLBYUcq-QhQ6LPP;dw;QayK=9Rfdio|NaVjsSjG0i z&=tlgFnR7lFHbmiPY6y=S^O+q@Lyq6`_+%hdF=oK=TxN5(_(G2YCn^WKtZjh?mNsu zumCMyo_7({pU%x)l8^H@qp#+}N#TbK1dCS|(614P4WRYyL7y^2FU<~ka_JKfD1mh{ zXJq6Yroz^4GAdtOc2HrWSD4lfZ0ida56IxRrjdsT^9nk&h2*c1D&c}HjH!AEy#m97 zoA(uZX`c{hE_=%+P7dOU7Pl!IJxYBMcWdBx(~KKHNK_2T1Jx3LU%1d%wt(Ml8 z1^#IT_%D?G-yql2Gz_f@hWroSDJJi_%Y4@QVLzQ$WRnHWKxsaCn@@Jh;qNB`(hVaa z021?)6C-j!-MN9R|4PS?Bvwcp3--{aMrz-lTdJOtO_07!0%{L!2ohH}suRgpV z5Wht4TzzFC3ZNPky(vUGz43x#+#btI0gwj5^A!54Q<0(s;gx7<84#xfE1g8ET}RnPC1 zKc!i8knNlawnx~5-N<^;2IRXJVpJPxIkaQ+Hr=!`y+s9ZlIfVonE#|OEt^}%24oSK z*bsCStEicEnTw35hh`&6l}(J>Qrk^=imXmKgmsh3&62dg6)~5}GH2jUkEitDoAW1q zV$Wn{%9C#70UjkGsn=Rxc6el^!k;8iB*6*GiRp|wxSi{c*T;v5rb6s84=BG}pYN>S)^%6823u}CwBTO@Ky};}Ern?e zlu$r~A&@HW@gpn|g(GNmq=eys>hwM*?vX{JHyti|b$vJrr!jphJ*ls*v&{J`-6D9(wEIcm3!MIAvf)_9jEx?2X9D9g3*HLSq%`;s zu>wC8DjEG|F4NYe(HFKFX}`Zr=UPu-nzig?e)QIdo0275j;Re}gI-n$R#7;Cip^-rv{%0pCwCJsh8076*@aC8clfV z&A{`yVE0rmyZr26gluPu<=qb)n$u*xUtjdxUb30q7+D%6$#lKPoaJ)Y9uBkd1k1>R zivCqPl~%{af>py4kk8jC&ivb6exg|;EA1mKH%(LP;XKeda4Q+W*+ma9@J3_ATDlB0 zfrsoExqH2c=)zGpB3AH_ronA!|4Cc!?A^8D+U;wODjH(z+eykj9=MKDA`Fz7aCGq) zb`GL~#+FOU{sZ2|KKZxNGxAub{Cc?{V8k(SUMZ;Fo=Ke(@;hDW zCM&e&>3O~|%Y2U%^~9Yev6-!Zn(z8&lk-|x(KF{fy!lL5-Z!QJONNEbjQnxnA27AfKCU|?EQyq+ZQ z(fgmMxn6!|p#8@sKbqc03PWZ)E=2d^@Z@`J>`v%p=%m!84LK)X^}S57b^2Z$B92%v zBUzj09_q-GQ+}vPdvb36Zn)sr0E_IcCziH6$GUB!%3xZ2VZ}Pp-KeVcW@`~I)bRUb zv7$ilVqcMIS5bKa^MjfnHpYnXKi1TWo=U?voeURe0ld}@vlr*3$;z~u(anZ~bkeA4 zZR$Vi3Sf!O=64M(Uq4<&i z6^mdz->b6 zV;xEdoxMd-EMhMlg=bx%Jy-?gy>sA5oj!O>0uGk_?4M0Gu7*(xM`L+2yX96lstr$N zTJ0&}78&C5il52_%E#@SK9FtL8_PJDH-SCJT`}RC{aK=H^hQ^Z%Wjx24B~dN?b-#G z7&uA40be+T1Z+GV(`h<5tx-5Fb!16a44?y}83SiMgElL=~(1KN_laVg)Oqm_ku7a;0{fbxq4X_ufq!!ajTzH+TEGS1JaODkJiu% zHyk1vIaGu??&52L1y0BOQ^KsNFGm2bdw^KtIy$XV*dmlzs57Q!A+uP%+jcm?sHxZ1 z9cYKMqI{IVHSo_K{lB;0|Fc-cyEzB1{Ml*@0tWi8l>N3e4{VdiE_BpaFk9izapI{{ z=o?V5xPQV7W(s=9f5WgBq!F-x?UB4i2oItE4a0Il6XD?BKDG9F)O6VnKj+4Rl5;nq z36hyWVBi254>k=A2}36fR+fKzl|ScimDkje-uKfZ6n5bxa~PTuiuyU|^MdqpEtv)e zM0_cJ2@4U>>>JaOm_##R4@+nByuqIk9xD^BLJYI9PIv+;Ct0Ke7OTAlh?{k7=2J~u zmzww=&r4Up7plEjzw(rm9&|6VJ!9~Z=_e6juo#9V_a@bEig~Po8(e>z0#pBoqjV1+ zjh7p-ti^C!tVlckTXWhg26*UUFCbc+S`U6F8rY||oDDVt zvRV8cwm7aE4)k{7lL=e3e9+3R3yUFvKu!Z(u_3EEEA${)%aUX2k*2NqT=rzYhN+)i z-!-(*Y7!A?;% z<(h~!nxL5E-126BboBBku>sxn_?jXkd|bXFhP}=(Sw4juByALmk1bgVs;(Zux5OWo z-i-gj?;EuLQ3ReWcvETSu9~$dPP;GbWq==*>!wR2s3Wu&Q{=Q1dYBi6M_|jUO=oBE zYPzBx*8->4Sdrju1ci3ZD!SmW@00jsx;pW~$g%McmcMQTKh}}h^?2(jRr~JO7#0$& zak8fPx(?ktcT(82mS6|!1h$R=ybmocMQib?@M1(*!rAa2iCK|ok+`xbmfl`+ma^Ah z(avIin@+F!BFE*MsE%aks8Zt#)4&su+=Pd8h~1KO800`w2Ws9MAY8>P5Q-VYUEQjO zv1JAFo9kQ|O^T+hBunsfiJhuqG7BSjS7IqOh9?Yo>K5EUoj~%$=DD;3_jo7f$KIiE zUq`E@x=47L*q#sn!wy1}t;ufj-D<$g3<7!hN6Q*>J<8nx9x{dr;rS~NdtV7$6fR;GY#O|C0-0Rm!Nr;Qz zHso!42%&MNH^RXSZS(!cl&;tgmJM>gcz+2oR>{szh+u^*PEj5ltB60OsnkeGw*v&4 z;1r2c8*_;aSYn~1Pb3qC>v2EqJ^s$E{qPSoY7WUcvN4dKf9)0UKXK^)hlS6hz<+V* zKQ3$La)ii`2>)QmWgNLd-=}*=DM_**1U+?f6(K!jvi>d?WYZo3%pUT;P=kCiye6r3VH)67d7AN7S>8$O)PL3!V8W@5| zU%^jdtx)D0RHVo;jPvSez8L2^Q|UKrUZw?3Fx0j6QrOZ=H<2Ar{)i*s{!I=`C>i@E^P+d?kdmy&adn@+3>jpY@?Ix;rk9vg0alxdhL zBaQ26v3qF(+-+tpRTY`PSE%Yg_?jqUs9&_Da;i_=tx8u)N)1#Ob*v!DI>|TKvAJ&E zBvM09<*1nc>cS4MC>Op-B&I#AS%8QQ+^HzInqtwk0t~A7TlXqx7wIl1wkp4L6Q0(#KEOCV+Xwoz>a~(@5zo%hR_h0#*~l_u^!#{7GGu+z3KcwhjAFg#j>v z?@lP%mOt;glx8PSa4aA1e)QwpPi-06UzwIu4(0uJ)Jo*7lwx@sewC{SaeE5+bJMUV zjXXtPh_>_vqLf?w1pCS|>)UgwI?{7`7l}&@7rdiDd(=q^(s##Gq?2ssgrn4*Q>-;S0g)r={o5|0fkM-FktGigE1OqN z=W0Bmj~#N4BNm`r0|?w)g6zjVi2v3{brP@!#&*f^cvNpQx0vGsS?6%`#i#cVgYi^c zkGV-^sAjvdZojFHqz0xVdc34!ShE;)m- zOwqh>d$v{$XjU~0%|-y0ov~#OCf&M=1LEV!DK9m=@S@2sMAo{4{(NI|^@$KdQ<5>Y zHYUsC$2FX85zoF3Lp?xJ+>AW+7)q^AGb0Tw%}9SsJ+Oq$$CH}aIwQ5I3%ZzFc^F`> zt)rUM*tsZObvWLVxPNi~+Cr-sAQ%ouHY13zKH6I~liw2>B0}^v~zy z_&Ga53Gx4Yeq36}6koxJL_YJ-bqi#12r#dIQqr~t!nywm`Gb*nK>z)9VgU3QujOl?+1Ow*|2#}a3+oBw-%vsqi~WDXO%yP4@PBWE z!~!D#0VDn7vy!>V!5$I*i5w<V3u3X7G#OBUl|aG#b7er*eX0Q@nNT13 zv$LuV8KKFx9A*Oq`JYQVa)0-8Kk0w7Pw51vWI+lr!sK@=ZpdV6%g=V#L^W;o&k??N zIWr-N6bE@^c#@<>NeW{6`h+BD=m2Re94SzhawW=lB}JrrK`A{SR+6vQ%(K0p4z99&Aef^3Adi?-JJXApqr4!$?^x~n9C8a4;FM}?Sm8Q3DE!S&6x9TNN9Jk% z*kSNj@@*@#-en{EkyFnLZkt?E2bQV6x>)<-R_|}6+De0vP(zj%lPE~AVzg2)zd34G zcwe6^V2%8~L{(`H{H&U@KA=&pz+z~?iYkj`NUE&pv~e(dqeO7Bk$=%qW(sk^yrh;e z_80snJVuSxjSB;UrHIrM0-Fxg$dtRyQq}CB-}Ca+9S8}--!BfsKiCM1i0%b4nkuL! z7_}yrOS{~|&^J&Nhz6NQgecrY@PI8;>Bb<;S&U0>acJiZ(ASBR8y%OITiFVKL)ktc zC>>x}6C<`tX|zk#$~|q9KK#1$F8Ilu>p zQhkL+6#-^6LQ7h%TlAaPh2H&wt9c)ThqccG9R2k-d-o$SutWD;O=>S zI(9h;h)0`C&W&8NufG!n4b}bAf|_f7%p=q!{8XycC&hcwfFGrg@-zg+?hv@>=*}$I zTul;U&Gt|=#}g>J{vCv~Cde`^h82Fj5u6l?S|Z~3qK8LFAyhl1-qrvhky=Wh+O$&4 zMltpY*gIvOC-Yqd9n!yMd4SaXByfAO!)PAt zC6~myklE>Cw?yc_zkHDD19*Jm5)y ztCdvXaWsX^8;D!|CI#~ny0;ZUilLeyC+a>Pe$$`lZNU35Lsej{Bf*_rPfNc17ks zJn=5v_K^uD-?Gv~@PPl4#UbFn9DN>>{w9J6JbagbV86hZ^TSjo3KwR%-;dzPKww{W znjQ=tZw}|MTP17xSqlkKR~-xTxJ@AM=(GVE$d}0AmM^OXptPrfl=5XFuvjq0b7>oVZ*?V%2I~L*0-uA6~k|etjLCo%_Z2O7bkR=4EeB#^^oIPGV z(*mSzCHq?1IT#P6k+&dT^~*+X;HOR%~@) zN~Q8QKpJG3O2eKbMTOPYXC;DK`DjTdlPBT`x%;)q|Tu}PHv$Qi%m%d^!jK9h3jo7bRcl1h;#xXjmHWs%{^wxq#d(pr8 zpHhn>(z&hI!N?9Y0*bNm+(dpvyy(}+Y3w0lW3iJB4qLL$&(|aLpo&Br=BH3rDzs320eSwY@TZK%=v09x@0D-r#c@x@CY5>O# zmWQ?f<$cR*XBg*N4>2V^<`9X<0lr4Tbd5n4VzNVbt`nrL(0l-Z2 z4Ip!Pf5-#u(E-viQ3sOZYk3CNkizIuV55tq8Ka|x6os*&u`v&Dchb!uUzwjK^`th4YdvAkQNuexPKuYs;(mcm$;=Q2e% zC>NSJbjKA$L6fcKWJ@~vF6E0n2iM9|#$8(ae%~E;c!1V_^1BAv7_b=_xRyZ#8c}SM z6^aulc5F4H zcNc$pGGk>ngv3}{)lxb^Xe0Ec_JS$x*Ap53McR^LVPs+= z@+*KE6#&1&J)f}Cl};5(s_UWytYG}UVsDu~XXdUDUm*d)y`x!rie(;9#7oVp(?loe zZ67wQM4E6`Wbq19HfYj}oXZquRRbhBJ6cM6I>ieOl&L*cgCx~Am(QbnN_3oQlQ%~D z27gH1vBb(P|4AqLD;xXPuUNEVZ{{GGL$karu_9A@60@AcN1Psl+8(L^xb1K$-koi8 zDK={0JTOr!3r&P(t0{yO2JBV1q1zMtWf%7g}eyHL5F#d-)Y z%vqYRir&T!uZ0ywQZ%Z!O$e#}QHv}B#>|PrN)m$KFe*~28Z^x;9Dv_J9XgwCG%YRa zu{bR_i!-wv<)N83KL&S7P*7Ba%7s%st6afu=fY6@wFVEe3)A|B zzvrmxN&ODyyxJ<8BXVkcBiFCtHZ7B=vx+`BdbUb3gXl?+6I#JU-(dNHBrnhUi@=*k zr1bJ1U^T@(d11P8HuP2|!eM`Vf{M(vV#gbF_`NliafLWt08w27OYPq+(B{+nZ9h}T zb1tAwSF&6ltAvKJ9yP1u1+M`o@Xpz#^)6Ysa73ny+*TStP?yIGWuaHSebOhFbKC1xBf zD=G?LtS(usZSNf`h2P0#rA5=CQ}tgN$Yvi~!x$v_`HDvB+sjngVyl>AnxxaX6E0@t70Nk;y`xot z?05x@#_==hVLpy2-i6gVh`cCnmE!?SioRFTmza7EvygU4_xY5C@s_a ze8aZpQ)UUQ@Y1FMwg&{UixTW$Rs$^C+1aG-TfpFs#rZ=#>bn8^=+Jm`hBKxEMU=bl z-E1QMs@D8lw_<1`eAms<&C&w6miC>rSOn6TP-qQp1ZpwUYZVqU34dJ3lyu5d?8X>p zD!%x*S11dr&!Bojz~AV+cX^7{kB>U-YjiH~O|JKZS&PRkQztt!%kh_Mjv`j#o00C# zYrrZkxnUlB)IUMTH1Lc$LgMUv>RdJR+fdfhj$N~OaPSUtrhX*uGKBo>A$DeUkZgSu zBIhd(=SSuwFjYN}!JJ!C-Us0x)8i=az$+UZY=G+DiQnp*u3vn2_6*O{6}z>42uwfA zp8td(&m@q>zYKjVv8|(RWE7x!rj> z`!IgK+0`d|a_R2H<^9W6&K=~lSf)Tg@HO|?v>~lkFK4XLHw}5tt%|!~O$4)PAl47H zp52$@yt`KN3tJ%ls~TsQi1|;lJ~f}V$@hM}`IR%edP66P`E zH)pl}zlM3Hr2q~tDE#}NBt86C(l%Qjj15@zCSe`vmBd4dIL!2LfrRoKtsX2LnA6(pzKcfl{xb&=!GfEZBVL0xK8{ix)0Ot z-8%E?K;B>m&l6eDT(&~F@jzb|$^~(aEc3r~J{>%~8#L(H!IPIf!fS zJ~{f*j%Wit4#lW>-+(fc}v^U=S%$jCgv^ zz`m-v{$(_f7M^kA4*0b=`3~FGgjq zcvG=yN0ZdPC=xwWi<*;VX7Yk}!KiMuY?+QWW=#Md!N>wbK6BH(kHZprLkM`e=Fkr<1EurW%^bvI~KIm;u@=Y>d>>^yw!uK)}RkZ5E#g+sUD_jMp{8BeAbhL7^{!QP^=U{dB(>Dnl);wzo zZ;#(Suttp-^Lt-8EqP|MHj)kTyiF*P&K{~9mrYyoXyNj3p{A+e3b>Pvs5CP!Ox-9$ z%DN$~@d86z&iK0{0JkEws^=;g>1{}y9@T(s z-DVJG+XlBX**d#>&xqEUh+q3rYnH)!O7&}&#PydQEUa5mnQc-{lp9Q?%-L9^GqHj# zpCGC?Y0PE#lXkhNC9WBEQRNg2EDo1TN+)#`aXMRs;X7CQlE;I+D8u0Z~X~Oel(@PE)f3egOvtp54`YAWLl(I<^8D`_H)9 zbAUwP+pV1vpzsX}UuohV6PK6o z$f2Kbam0-Ex!mNVybD$qQXR{%iUWXjbB``)Wh~nI1UbHxMhqYtw7`phu*970V*MjfPhRC)KD)GP% zC%5J5U^@s3@4CB4%Ak@PvT#jx`G*Oom%$zG`#CnqIYL{bxb9j>iU5PgWg?w+w=Z5y z=cgwZKCm9kCqMMg?q?3<7w(?K@>xdDANYBeED$VM5DNT z9kNN-8^`*=N8Zg&9!5T-48^C6lKO*>e&1be)DfFY5k)v^_x}DO29J73lWK^MIC#lo z#TSo)B`t<(oQg*?$F z?mHVBM}ySV?%8oM<7L_2#h>8SpL5|juiu95@zYIkYV{(f4&5VI z1wSMR0!bw3=sN^m5T6AYAzL4rMwpm81A<8SFM|Z)7{44fL4|Unl0>BGO(%;#2(VZU zFhGPBxL74fJ_lUH(G88-4Fr}ARF+$;<`c_HPGg-SX|O%z{tvF1^l$`rUzKy-+W8)- zOztRiL-#0S*!u}x(3U)M{DX^6&Z)IC7~!vS9X(Ud{EUw~5aom#=L$vD&PB0bVyE%h z*NY!G;zyk1XTs4A#h81X%7p$Naq3jyfYKh} zi8G50TW#>&k*GnjUgJ}QygRkBn;7RMHfYAD!wUGa))lnm^qn`NSjWzN1KytLFv$N2 zmWfQ$=&`8#l5sX>ahs9(`0$Udzwg4pOSfiyDObUdqGk_pnjhpe0Q&LoRio>T1i)&e zz^eWcK~bg(HX@jmIgmzeQ;mDeP)FKZOu>AYDBC03wxe<3L_#@qkC5*2%lIcc6tHWYd}=LWxz zeYpqM60Mi8w+FLjJe8nr1b&%g{zYI)CB z30&w+ED3iuhYnM651b4;dS|^<>*>>U(RNp4b-dwzUWw7J8P_~o(2{CDgYm=aFv2OnwHX8iTYsD%p#H^lSR!LJyyse6Za65E4&HgtL&@&9 z+1H6CTXdj;jZ49U%av4hSnxk7;r6)H5w?(+QKmGfht-r!XtT#KvdsRx+mOq$ZQHgz&6zfDzwbG5&-dLQ zH)8FGRjaacS48Er)~c+`CrMpnxzPZm%Bk;fHzD&`_+Rk{e=TQ1o=N#nETkV_Pj5n) zTV4TqKwn28&%Ox&n8m9H#x(%6sr|zj5AbIt^xaL^`yx-@mzVD6*SXC5xk1v=E}90@ z{3g>3#WxJ=oY0RsoVULqRE(M#e@VeJrA zGvt}X(w}&%zm&2L2k1yP8z924tV3v1B-|JoiUvRb%$=_Zg@elIUxfpnv`^BAwE}=& zi*WgwVf~8`7l{ZZOOcdwBC)1LVEJ+FL*4Q%q{`h43)~Ee+(-G-BGIEEw3$T7&=Jl+ zDWgjUpkFc!lvMUnu&|qFKtkvLX_1KqBaI1J^RE1lCGB#MSC(i=E>S3%%JtW4uC$XQ z^sy&o@8mLIn$|-?fJU~$7zeh5iAw}tQ4r=1MXK>>apN-`PqNkQ&(^*+_s2yx7?h$N z{wN<$X?=z9OjK&HM-%P^7()r+ahDO_*8XSRzHU&3ttMECrGtIxP2&T%@y@CUkLyr^ zo+5iDYf8Bjrp}0&R>IFAOc`|X<;bS(8qSh>y|eLG5y3P_fT*@0*7&O^HLg{|`JKvI z9upS%MeM<2aF0n>4F|sgjZxD@u}2c=4|sMgs~RCXK{nP1s)`G-^T4a(ff;F>XNrvdL2Xwv zaW`rb{i6cM{0&{^4yM!#ylWPsux%YGuF#7CO&6^kfVW%VEHuAXY;~#0_Gekhbr5=e zk8lnGPA#mDJYqsAQii8UWWKafR1GSD-&Ah{%d>@d(;f4E#@V7%GK~T|gN&G>daLH+ zR`AjYJ02uIqFzo&OCf09jwC0fS9RSCe9{UUwzgM|MvDfWF114CN1%CAkhvBZT`@Tw zMJ#_5U?f1iSZN>$_MHhXluY}TiIF%2w@L^_Pv?Aa)hq4LPi1 z)*5U|6S&L?~7f_LP_{i2w$<1wMqpWb&3Vr zK=*goih3|)l?*oe8DZ*{3naVZe8K1-{Gj&iyk1M0cuvIb%)o)PNN~GhS86Adym;qB1#nXhlli8Lp_O za!J^aP#8~L*P}!o00Pq0Ma;(lFxeY-GD2uy(H%DAfvKcl$?HvVw0BgW)g4tRmgpqu zl_~FxDW20Vemm|79k2?O(>Xo(RRDtW;Yyp%2ybw%_0)uspJCT_&E|qVuiNqEIc+TIk+t9!NGa{$(chm=gGGIihVJ<1>>!l7Ha6&<4)|K3kOxE_%Z$4 za55Y#KkC|~O2$yThqXu|;mdHJlf%D0m@BEueqM#|rd|1XPuoI0*m>U-djP2V?T*zT z)Wuyy@RZ0XhPs_}nlVpD7P0qQ!W?MT_qLvAEJCNjq(uAchL7HS=GJ2)rV0Bd9r}mf zG%Czmjc0j{5p+S}?daKl`4)uM23KqIZuMUWvtI_Jj$9~>Dv087uOPUJ^Ze{z&bZHu zV?~4i&nORItu(vc^h?qyf=5ltjY_s*q6~eu3AxB9tQuB#Cb=9K9@}^SiR@wNZ zlDd9cIn`sSW?@*B!k`WkCw| zBv!yd&Gi$K$&jW1Q5A?g&7gPUV0YwTcjjPs=wQx*L0;cnf2(ughT$|GNEU~8e2-}$ z&Q)NEF8*(+2&D6qp)TMD(#S*#sYF?9MXQ`54XZOH>dow~EhmvFJ(!VJS-V4n> zCU5N*5?T97-ZK4sl?C$`f!@X1YB;VbqVQq~ z!uqu|qshg91Nq#ba)nqxZfOCsf*Rh5*m~N4MVw1f0~HjvvcE~2xKBLemamIw1$kx# zG^furLfZ{e8{v0DZp%k%%Qpt$l|vEYKE(g z8YCx?cL7omlzoKpU|CD~ zz0=b=mR-;w7JXQ-T=qeb*B};oSg=mk77oyb!x;5dz)*XKF;sn5^o;3d9jp28)rMQ8 z>>B=EIJ8k;V+^$oHtMm>C^5XB+x|Pg#LJ!V7PE2c26qj6N29^v`9RiwIi+0}KaT+C z1nLIvnksJb>ITdhN(Yy~hx6IWWD3j*)Rru01uH6#X4vM4D}47gVsHDic0=gVfBZ{lTKyCQ2(9IYiOAjI01 zT;xBlEb>MIyMYO7&d?yrYgObQhriJ)-p{BAbVZ1^k?sL<75%M50vMp#usbB~%gk;S z6Z~j{_Uak&c!PXp8;`P)u08izt90VR+mxPR#kpSSj|vmu3S#wDGuKmuYGqh9*MkDy z6{U}ld1YZ!&xWw!Sup2GvSEvK)hr2pwT#hJhdl8r76u zSny<>@Ph4RXV^Pc4n}td+Kmkqj0?OG@5|?s3EZ^+e)lMx!UuSqbT6>o{8bKrk43h+ z5$(^9cHAk_V4r%6&t5n(u!C{&$f0CU#t{!()MQkK(zHx|m? zU*>UXQ6lBU8|M)oBg{0e*R71M0o&hF4`Thk_@w}rQ{exKtzd>;hg&^KHbLSQ|2t}_ zE@>Xefdfq%0)GKAI~8SWe>A!T^Bte+1@71?GFi zOyqnI&p7hZxHhd*J!eW*PI-^@OpV{zWfo717M_xBSdwl?+Sc+m{yW~zr;u69Uf|8h z^L=w=UX4q=_GrSw(Yzud3KHQl$t})l&8ddv~enY>u`pFcLA!!en+gE zPhfi9v|P>b27q0IHkB?&Pj#y}WJGvt%%TWP;YN?j1r{JO*qa?oXC- z!z6y6k#q6&NNjDW)p3DMY2x{D$K~GgB9tmH#>Uqzi34VFKJtM1wL%1gV!R3@@_J#z z(B@6qdG~eVVvpjy_7wuH_>>j{o-HJyMDMgJJorm&IEAu2P~BD?!U#6(m5R0z4x4xj zYQissCwj=>O~{Jdz>p5%#m_mDsygz(hHTWgy9Q>nLa^n(vSWG%bfpZ49i-_%j{F2 zzzRvzZFo)M^B6fon?ATsIdSJ=W2(2#z|IBmUW&`}bs2X?w%2TTQrfW=>>}noAeXX7 zwc4;4TOq8e?_@f3WoX|z+`dqpGwGMqFHBgP(tqiLM*w4ieV>>uAU@1}zvq5) zbsH|Vpr}Q#GOc8Vl@DQTawGmSU={Yx2I+mbc(6%d$f#gO#l`|AV(|uA#qZxa9Jq(Q zwb_@K8l+z&4J50 zgv(01K}0{{da8^^iaaUuga?O)HUBLcX|YcSzJZ%ifC)thy!o^T zcIBT@;SZF$5zZv~Mw?Umfe)Bj5eRc_nk^82!_%-e>~r5pcZz;Q-zxTp|K04khkUi0 z)>_tD$-G`RVwR1QZ0{nMWK{efs7$d%Drl4f>IpSk-^1Q5AG^rGtspV`P<0Btyy$1! zf!T&AoP~?<8RDGPGvRV8nNdTIIGITL7KOF0S<3wh8G{Kj*(Gqe3=MbyuE2wwDd8VZ zj5lJR(QFo#Nu0$JF~pilb6mx8e&m%S=n03rf;_AIfC6QT{9`RR=zMl-iVsJ@-8P&u ziDqYR*ZZu7cZH9{NKR>r$IxA^{*PQjLx^NR8U(13O8^ z_HE=kJRH#TqPzUJ+QsC)*;>Y@o@<>Pxy?g7y`wwnxw7U$?%sg6f>`_dD|@R07N6*D z#IYX+g9zT@CnphwbC*nKcIa+qJcKoXiKK zg*C_}Fl~y!x!#dFD(%M+w%Il~NkS-idJOm(>FX`?Ixv(J$tb@B_Gfmh;K&{2fejuV59yT>oEhlQX_7yTBumE{_vF^Zs8MeT(KBb1fh~v z^jAKLtJ@BSN&zqJl$}$w91V0zID3E;D?z6&r3c?BY@$@hQ0b%bj#;d7L}dQAA^#H< z?etX45N35h{R@RP4&~Ncb`3he5LOHd=dA(goE9gHYK)cJ*ieWS`mNEhP>LS*yFL-v zYz%@w3j=APlFtfz@0+PQZ6T9O6{7s8v1q?MKs2woa5oi8%B}Hs1C$5jn@a%DsBI5D z_Eb*q)6I~x#7g)eN_}?W4@8lb_*GxK8Xspm6-|CW7t^QGyr*BBY45t4`p3SrL+i5% zWLHnE_Gl$%>>bgtZH8QvGsJRfG;ti=>^+KZB^|gggby2jLoRU~i}tcgoa;NFa;@-Z zz>vthGUMD!nv`_4j0jo0=hFa+2fc(p;bs37q><-S*TbS-sFf^5Q$@&9MbXv|vtbOW(?zjd6a~R6%VTer%#EO@oTc}JG~`l5t8*f% zu{oogFDMaj3YBrmV%sVAqNho%H}^E+HvRg{(f`0be0$!zCDFD+9VU1OMoY|>KD68~ z>EYHJrEa7c_e@ey6?32@ArD#oCldC50R2Cp4fGFm0lYwxr|JCiV0M7J?Ye_sSy-lU zgnfwSe61V_&mi(Q+^3$`FTS%cAE&3-f$n^bPN=m!Hf`#Z*206E6xUWxn#sNt81h?| zUYBsYlWbT|FT9d8lo-~p{n&rGV#E$g{)ko3x$}t(rj!R2x}7mE8Gt3mn7<|!Zk=DV zVrA2mOj;X`m{9U!N16fR<4G2#UTOB`1r$q0LS3Rvz#XYg<0|POY%3bckTz9zXY)^e zAL2ny_rgBb>cJe)}h2-|sWGQbb* zsf+5<$!Hqxl(K6Jr|T(Fv!5<8_&ddCufN#2e#5n3j*_g$JevVXDBz}}@j8o5kxSsV zX-7@9dbZLEeI|S>>)l zsnHWkHewGq9P2iY97YePtK!Z!011{VffD@{`WzCXG=4D&%^%=}Tbm01 z=TOwq!NJ_rP0HB8%+B29KiSf`-`UcOLTH~Y^0;=vTSqF1vcL^HjJ;~K^zUx(FQX9cXEK!8L>o;-OH7Afk4md?}5fVW)R7Qq0(2f`l3iK-MS zAYBB8)CEgDQL#9%fo-I)Izvi$*eolsRc%XydANYz1MixCBkaaCpP1X0J1xUB;#6V< zJ4L*806S|UZ7^TGm9wqY!fvf;LB~UW(14Rq_F}$>goFMdgF*M_wG;88cS^S;VqH!I5?%DL_J^f4=nK*Af&+xX54++boP5ADN)8StPKUJq9|*4*0$=RihY=|qx;Yf zTPu~UY;|YFLA+~LIsJ$*<1USze9hr7U3VH}mqL?EDDQ&9qa)_~3dE zKZ=P}88bE=LQ$v$Jy-#{JzJ|%c&lVM07V;gx^#YE&2lIz(kKoynQ&eYpvEdA!7oG- zmj3o4N0K~61t~@I4Do=DZOhe1-j1HYiP{|GW#mItFtw;f5{ApU&3pIZlR&) z5M^lsqwJe}L~Rw;Gr0V)9g#n@Oe=8W7~K?VEmTdlih|0SP70A9@C`#qsQ-cL|Ab}JcYdO?A>l{!H|LNO z9}o~tGBX@T`rH;4GT=Xm>ehz!Q#(!-v^4N!X@@3G5F&|30zHyWfRaI(<0D2fAtT1x zO&*uW%A5)rheN7u4vepQ*nV|rUcuOWh0;JF0io$>-njUAZPU8#?%DafWz*lI)m_NnCGvy5_jcQ{$G7M2GvJi>)Nj}SZ}e##a36riZx?*iISrO!5D(h2voDGp zn=NT_8qeXxC3#d{fb%blvj7HH|LK4TpZoy0g5Ft>o!*cihOE|6NGF-zL69EZN#SDd zo~H@4XSz(=6nJP`4j`qY9&Yl*G88q>s4R6$joWx2k~T+cT8y@xdoZ&4S$l9gQvpim zo5Ox}PaCk6Gd;Z_S2N55{ae2tWmURuM?5Td&~avZ2*U0tQXYkaRa=&#E`BwN3pz%`>~F!a>alENTR*GmSIb9>7!wSE-5wv8T@amau&M#c--*1rLk zv0oOuYqHIgo3%a2u#L3=kw0@S9)=SX1;Iju(*ck^H#tk2=>+NeXnuf%qVz)!ZzNZN zj8w7>H;L&oh*pYmE3~t{H9T9{;pf4Ljl~^0bY0?YHB@MBRPXL@!mT1WA4w76Z<3B% zPY6)a9b|A}hre&BM#;1MEJZOyYmmV|ZkYy>JRA!)>Ow?{m-DV1Nbb~IFIC}Z zkKlX`hN1v~5;n2?X{+J+&hg3kvOe%E0(nU{u?#iS6bH&YT`CpcpVunRPVMuRweKqX`1^$d%wa={R7pKR)-`xi3Fnh;1*jNJf)V|Jf+xdlM8#|jnIFQhdBe4bh0>XeJgoX7oJIK z=-J!i*6Pk_X>9d|*h1E!bC6Dk>IA3Xs;>Dzf+@%)G zEBU%vbYx|ACXNvl0_#eUUi$@*IOehDp zX?@`fLDZha&v?Pk_Ma8j=diP!mPp&Aku{-a;aLet3^F}MHa*Pah}O26Z0G<-7~0OQ zo_ZA6TnY%|bG7#RPTjWB)I7CnBDQ7E)vTjmm3C?`O2y!k3MEkdrPM5^=|6@$L<={} zglVpsaXFdoVg1Rw()6rkqLZgqYTXxLyxyqq=e~K7fCc$;g_Sd$9~n6;BA)39PqHF` zfB$`btK2CKqP5_qC^qCD4wD6NINIa9k*{xUxS1$7j>mlpk!DJ*J94R%Q6SdnkjpOO zEc}T`I$CDJEN{Z7BB*>!{%LKjg7PQ5PP#39g5~Ej&ijY%y|_e_zIPb;`3?cx<;g%HJ0WiL3|NLJIXa_f!dG;$)!vpt6FWct3r#Js!4uLeN;6-uUE|&eOC6e zY;pQgc0hP|ANm=xD_Bpdq+BGqpghyIjJ;p!w}ltTpg}2sb+t~s)VVyNF+?>|l)T)C zeTemBIG4JEbJ_hV0;?(CMe2hh8EeUuAbsj3g&ezFs{}uHEZHKR;-Z~7!0zOfGMq~= zG3Nx0uiQC=CZ?X57nuRTZa5mUQA2{y(=EXI&=az_afwWh77UGEj9I?6>s&DUWB)-F z?yl2+xuOzS3i|Vm8`blOzl~AiqGeBCiZ6#Dg`B)g27F zPPC2Ip38#90)pzXRO!M*Rn3x!Y&QpWz+w@TRRa4n!s*Km{uKZKEUT!Qfu#;JB9k&a|<{k!OqaV)1i2`97FTT=q3zQp=YQP@W-4oDIZAxj=A?3x7 zj_VK~K43LIk&0GJ?U8Jg%(k7!-L25fo)ntK;GhJ|G;w zaB=)r@ebxp>y-xh-u;KO=)oHncHy}H@u&lYFZ9myH|B2XharWSuT94yhMC=p3WX`R z$%GLpg*F?G;ND6=_c9#Hd$A52RPfjf>9xnmgE31eL-<)XAqx|#O1&VlQA;q zYwh+1QFbT}ue>1Rx2O;Gy;@^-ha8E`Dm9)#i7n*S+bHU&km#aNYv;#q`L-wqJDqa) zCcKPs$-U4G68FKqLN%uN3{{Wm2JyY(W1bHDS$5(Pm5AvNgP)P``+CeGXX9=K&@-$X zxPyUTT4DgZ!a--q*uAnm{l0vFIOzw5!{X3Dhf{?W3cQ4LxjuFhi+lGT+~lC<5ez%zio}%zi>6!A^Q3mbaAW zIPFoyFG>TJ=6oTCN1Li@Y%B`5L3nA-C;`(tyHtQf*IQcb0BLxF{!?;3nnUJqH@f}^ zrdBEDJi!k-j{dP{lwO?_9=QoGkSi8H1jB^w=f;q_v1gi|Fs1h8-a*2(E5?Owl{rlM z-q);SaZqJKbWoHYhC|$E3X$R{b7d+bj+phM?VY(t5@$`5nHUrezZjDfM{yB3laSO~ zdu#w8>SpYj;WH*X-}P@^OaumYoQoQjv}L!6pJIGy(|q#k6g5yBp8F{!X#}!mgrhcd zbz}nW$F{`zxIFU@xTwV@QltwQL`29S;?mu|NoG6*_Gc+IhXLV8m(3)yb0u+q+Stf9 z`{7@~Yn~yQv+JeQtPyEw@(cK>lN6rhHNybC9oNWZG1iwGs}%hE`(fzC6>)ORmcOW<%NinlyxgdJ&9iKOd|df4ADsdQ z*V3M|#T5H)pdo{!1l78KNOuva>zK*#tXj*;(Z$!XV0rmZ@(z3=KLOPSnpUq!H=~i$ zDCsFrG10y-Gm|6iuzPj@S=;nQPCt)`>r0YRFR64+wKO(R5qO=zr z*-~|CQHH>glY}KnO?0;Kn8L_-MH5kjh#|wlW!OW%ebAVJ)gIwnW8)y))rx-y--7!U zv0yw=D1`1t$FR~~62jd3LJnD4Gasn6m7N`kus>c$g(Jcy+diV^^%jV?T2 z5)sT1l{eoH_OaANNJ@Ds7mEYrq}{FScF8E@`xD7$h=0N5*7 zCikfodSGR;Rl0x5$o_!pwuc7Tg--(Y!y<5{D<%WYjX}!b;Fk4(dn`EeXs8IL&*r{9 zGrV?h-ufcW&ts>&9^dC9)Zd@rTzL1%@CPnVQ1+<=fYo%P8u5ZXTCF-K80hX-i-g2( zxG((-)mDNHbwbdSC2gDl>4ES)Z)dbH6AVQ1!lxDnxdS;cFHf950+|HR=7FChNxi!f zfKUpuA_zkW>WBA!DZA%{;a%uLdOo!N)X_bT1lha` zE(rj26TzgO2j@TV<~Hs~&f|q<4%i53Z%&Z65ANTRpbDc9smIn3hK+4n*o2=0#@!dO zB4s$8$FV1=*#xJ1F=GK>SM?8c|E>%NDLCu=cA3=SgvVHC6hdkAnBm~q zqm{vsP>@ZxD>Km2FVaY7BirC0+i)wvM(5FBF$q7Q@6=Pa!g&Vxs&&dPTk#<=H<5kG zxK>ACcB$-7ri0Eio6o4`b1ZM*-BXA?MXH>hI8KB;e$+UcA@(&4Y)4JtM0inVUcU#! z0-}T)u=A0qomi}$2=3gw&Oo17)Qs5mSKU?IEy^RjCO*;%)h)7`U7Yifn9Cy+-n0&T zWGd`Vc7sLPFcty$YN9kP^Y=2IYzKU7D@?nBT|Y$`t53h}1;Fn^o{iH8Rury?Dq4tj z+T+0-Xqx5=!-xM23qBD#iphp& z(eIE;hjNsVbRB*kAQf5gWS71WE4uG6yx9 zfbmm+`5U6TAv-r!QP6Y2!OE9xQI?=pszJC0070^`YUPnv8pp$W3oae$O8_yK+Zk&O z2SIC@k^D6dp0d-o;23{T^vcgAV&#=MiMPIl_ELdeULG11KFmdLk6y2)H4H{U=;+3` zxy%hAfwcovtC-rNkFJ0!*^yP`gpjX%zgbXSjxGEwi>Jt!A3FtX%cM zvn-r@fn9Xq--L5nE=8XW)g)M`Wee4srph$?9t#0D_%(Em6hj%LWd}50XmTV4Q>1Wf z3F4e2Gnb>h=?00oCLW6hhq~i1}jGy5=&m%R=Lh%=Gt{ujSR~e z&T$pMY^pGy1(WTP6xA_9O=!@C8{2FC}N5{JaN0!ASO?l6u&85Z)dC%1`VUt z$m%1#{x)6>d&Hq6*W(oCR_C%q8eN?LqBM<>!FPWplC0{Kc&Tpc-oY)&3;7>&)!-iP zN*eL!O3g6Gpp>1)iGohsHuarARmq-tqMBxj=URl=LlSNFb-2W!H7Y%4!dhD2esg(% z8Nq8gvG=z^SRA%9Va}3Z&4z*0grd}hv)3HS5cCl)gYN&Q=f35!cr?^xZnN7#3EZE zK}22^%hMVtUD0UXzz+c0r;IEBEZqv1I{i^8drkE8uAix)Xb8w`zuuVe_(C9!7GYq- zKLuA6?1P;?6;Xi1fv!O~`nQJy|K0pPjVM z9&)ORv2p$-6W@*&okH z>JYO7G&ythfbmtj{X>~p2Wj~8XGvpe`5f|>EOLa{3)bl*x_UogY?ODx14639>Xw|6 zws&s7!u3BG@(07(lE6PPe;{&WYE!dWs2>~X37e*`Rix3Dfhu2@E-1Ivtm@1!+g4v* zG^cw$@W9yVCp)#P^R}Wd7@yQ+FnOZ6++v{}Kx;)X0{Tq+kT34cjgK?Q+?eNGXVmIg zNHu;gDn|cq3PyNZGceunS1V_u%{8L8D*w#slWtr@PnN!B&WYVBHwohk0&Vmyap_$8 z7bTtl6PXKOzbzvyjHIlV@)>2jXCwy=U5SN@$-m@u%b`Ew;glQOsfzkZ$zYJ+#gz9^ z9P?eaq@wir_N3eYZRhkO(5LJuQXy#6+g2kxTPTG#}Ad2O}<3IPoNUyy%nX{Es1x<#Yv+e2@s3x$sNHmJqz~qOoQxbr9e< zy4Ucaw@Ll^i6_aYV_H8!IeU{6`&D^z6H#3O6$v|J8Q+x&eJ zyVzXH#z%C>2yqAfijp}}ObnF$R^DydW*dCQwF$i$xxBYzxkhwktMpI=H6J6q1j%~} zK~IKnmY)1PR&Q51Dt!Js-t~%mtB>d6Cufz zUUphAIONv6jhdDUw8>e0u2V{JQRR{d)dgZU|&IiV5fLLt$l==FXAPe6LndUzt&ywJd1~ z9b-H$Z7v6020%0A^0GQBDbqpG8_K0l&NfoPluWh{j|yT+kg66=5Qncql63G- z%)DhuY9g{ZRjD(#c(a$O-7oSWIQH`(CObyB>{C=&{lP_*6){u@|)x z4B8h_uNdP>o<$&0qcnj_H-v6!RaKPvn5vSF6aX@GJZoFni!O|^EuB=li1jc5qE3Cf zai$KEMYf2)0`SR({Ms+~JfV!=mzG*;PUMBT#9J{+1pAFYGecuA^%Zk6w*>gG0zF^Kf=8|7G#M$w&h>ewEvha;LQ8AXxW032v z=M_P8WDQvq!=xn6B7aBqng z&%bbv?gYQ;+*}xdcE(d{I|_B#c=UR{%L!m(G!mqk#uRSJTr@@2WHHB<*(>-F+Ni6> z2^bcD6zV(V6n~c9?beo9-`iz>hQRg~4|8T?-l5#2rkksY9PrLmKUSgd>89h>jXQ0= zN2|V5-R&kQoB!Qyd}SR5!VeTHmb8;Q87#_W1*g>ObVu7*<-|9N@23zFBWd+1QAiN3!K;Sh zMsA1^sgb;8LI*qudB18?^ltdV)dgscWbu>gO%1{&9yhP}Au9zDIogi?DP6NyWd0lx zV4f}Q(3afgf+>IzCg{vH8L4p>UwX)OOx8GXqt6}*5Bdqsw}5YzAynWDA`kkb3$mB~ z)N8LCsrFqihoDpyn@fwgJ{jcrvs>g7N}7=TllhOgp4ZN?mjiJ-i-8D0bPzz6^*7GO z{xO9Vx-+!cPet0eVubmg7uyB-%1I?-s0X@99QXansUEJksHgglbLqR-Q0yTSIYmG= z1dij|x!BjZw`Oqf9f6NN5rd@gP{U5k#n2s6)_gedAMNl!sb6sUJfW>Ouen&QDsF+& zIvCq#(g86X3yW-jX9}QCsqO%Ot;)k6GS5IBDYMAL$X=y1_pyhf`~2)2G8i#JGf&np znP@>yP}S0p28>sn8-Y%2$^3JlIib?C2mV$TF_mM2{7oNF>Oz|$_0Cwvn<@i(4tbVW z2qu53I3aX? zo$+VN?_USWptK(AVk-3$Pa)vC<`<0Aygmqeau;Uw-bK4_e8Ly5^Vvnc&w2_gXiZc5 zjFER<X$3?6r+XJVMb!Ph#^`@DMN6d|EoIxl)~b_qXo#=5MciipnrfV1O899?gOR| z1nM8H#1eFD=>nBtN37trl+YXe%I^u_FugLfG=PpTziO7M1AsR0aw&3$e(ph zF=&We#l~cSmx4FtD6w$m`1s$O6j-8nDT-~lPVNWSPl}~;6mVg&liR^!)H{eJk?3I) z=$Ylz$%PGe+NjlVw0?XDGgt^XJIf-xJ6}2^Y9x^)l}($uO7D~{i+XNrVWE7Y)&s#E zIpxWt=Aqbs&Yh9tB=q31CZb`OkMiCoRvEDj7;3u!QWf5IrueHX3H@cUAW`R+Ji+r) ztmJC+X3oLH)!_kCbB}2^Oqpy&V$524P}rjn3gv@yIvWl)dkigfHH8+32vy!kPWi@v z|FBlrJn!GUc}Wp?4*n=PJq!(k9Oe77IHin(1(R2IWWaJUY1Q&8prYMNQNhyoH?BxEPFlBX8=KfIQS`GEDU%9M?b1+%NPdGC+0IJ>+HKEIva|6(Ls^7a6~2$QCss_Roi_oo$UurIeCR&y%U{)2VjWO+vxm07#~2@GxlWbD6g5 zNCW56O4(x=U1wE84sgwZfRSoH0p{u-%r&QoR)Df>z0`-(VNm0?b({mJViVb-wMKLG ztRlKcmoQrT<~gN?*Ab*qf(-pBmLUceYbchO&uj%zT2alz8H+BEushp~IV1}#x^aeC71jo2c5nT}}XEg?%R zdcc_p6d8XMmV*S62%_lRb~?hK*{4n|GOp{Dxe-p(sW z>*T;k@=xOg;sAqsha_Kz6F5;W1<@6zmt%p9F*6ppK|tkK+B(v8Pjh;#30f;@1kp@F z$G~#h_;!*xVQqwI;bVvkE6zxTA#Zl*7GQ%W*LJ6p>O+brtfv^y0A(nvPYmm@P1e#i zwIz%!QL53G3hN{}{pRfVq|yZonmCRsa*0#?H-ViSYO5wk>WmTHS8tw*ieh7>iy#Dv zo4Uoh4V|tp*-kDczH9ns5~h5^RW}yTUzNz|31#Iq_mbj0mqVvu!qwCLAUfJ>4S*ev zB#g!|$57V2tY)gQPMq4Y&LURP@2H<^jpO)|Tw@77P;mE|qb57krGRiCer7S~+~RpW z^CqR3l5&AU-Vf~V8|#&z@cTy>EAZAi$vb5JS^Vg_7*=$Sgv z2l93(t6Z;FxKqGIO2FmP>_zDWF+dQ(IOOr*e25xh?gGy30Pj%}+mL#vfM13o5JjB; zMKY7-F;Z0J6Sy|hORo9~s%wdR1JgvCt?LppWh`^Z2KMb-Q6E7@_3+^B?AHA=O zaPv0BB%x#eN6jw!$s6!Sz`t)P|3kq41ryMJ)|pc@1}NZvak8-2*KqkA(Z~P)KO-~$ zCt|2pRaX&J8R@f(USCVYw*kX3BE8C7R`0kUjJ*d*jtrJU%1dxNZ%(wuGP|4Y&o4PL zgYU4>PpZL7WAz9zn}fm3%-qbI>`CsokI_{}AdPv_Hkv<8a*Lvjvg&8lxf5E^bDQ`h zDgZc+ok=3uVlUYkUPXGO!T=(Q0oc%;r=3ScvjiO~RE*eFCa^?=My@oThkzpbDWnYm zlfaB{$aNFGRtEW0#2ptO8C>V5p>5CY>_VPGX8n6=s~>PYDDGb{OOk;&x}1Ov3l}f$?iSqL9f}usTHM{Gq)^;FxI=Mw zcXxM+ySsn9a__n4eE%}ClZ=eL=U&fTHrK`&RQ4VnLqmysil zlrrabz#(TG2gLn|c~*U7iP%9%MfVcS3#c5`ihXz(&FkjJD7s6Ftf46w>#|F?WS6m1 z!tC90g6z`|{d;7I!`!$go_u;bV17^YfT6yaTe6Mjiy(gD z^8ZT$R>7%A{(Z^sNJ8-sJy9cBCO`O~Kh0<$2W%=qkggXGaYG6m6*x5FUpymnfHpMq z!$#YHE(CCOKw9R!C=L6WY)$|E4w4A&)4yb5q#mvC-*-hk;XVF=t^+`siT4A#FyJ6C zK`O$ACORsUe^%_eQS<&PGNpt5{~7;(fgGV~@d0EEfeEa@`5er?AJ}za%q9+9WP>T+ zCE1H)!!wCy7Ow2w#Y{<`DrOrg08bDvd&I063Z$y=meU?1$xi12=c`<=Z`UJSpNuro z8jF9dd<;J|8i={GwjsmBIb6EVh?Rsw-bqX&z^fyHOiXh*vsO$g0KS^L=rt+TMfeX# zK#XBTLA2!y%&FdLaZR@;A4MjW!|l9_Gvq;yzYBl@^nJ0;p!>7@o3{eY?Y?3(Q5_B5)s%F=n55&-WGLu9{ z&O1}vuRStpXfa~;0ww=wW-p$`cLwNL_f_K>$lrLNBI4Wng%EGDk80EED z*L!Q!<0u+Wswzo!TKI;wfQ=9#Ca#wr7c-xHPsgM_bMhO=2*>lOtjgj{D567Gqfh&7 zqo`q7BO7)Sjo#nCKACML#F1@XXo-aFK=TW}k$zH&<2nQ35I8lpF`o7_4{6hKWY`M) zHRZQ^em&u_n8=?CVsCiLy`@}PEZeWJRt(e&alXY35$KKI4hprxz-?0IKzgja0jKWE z!ZAdpT(Ba?c}atM3a0HnTFVRmFZbc^z2N_j6)R zO>Gmi&NGA5NR8w3FWmNXu?nKt5yy|DEwwY9tJRw}UzYPkRUa_#gnt4RAH9!jb~={9 z1Yfzuol{C+&vd@jtG)$$AF@TZ%W|_3IPF{@MjAw4{{RZM!7Bf8C!nfldm&jKW_}~m z5ZrtcW_iWj#V+#QT&4==HOYbW)X$S4j;3;c_INC-xfTocm58^eO%8GZB-3#ZaIuB7 z5@8Yd63SKUCI_7549|V3PT8_#*IJYHk8u0a7KkulaApHnUfEB*JrEqu=IJ*mPu9RJ zJ5QQ7&mfwXQez)D!IQ@VN{a6GX1zD?bxKkH6c$2B#=|;!5C7MF|Ig+5-}{cp+{!Ek zYOiOr1kO^621^ZsV?_!uNRHzyZNp)G@&44&?I$0tSs4^1x>~gcsf_}rlfft0zQ5`< zF}TqVd;|ZGhG$uI{-I(|vyl|zEHHx^3)6iy3X?(E@S4>z^vz+@o)nuaZ^P}hp2;=E zCvO^oIdB&=U!GuZ^KDTHUigybjyby$C z1`rPj+ztAt<~!~mwF{aR?fQ^i~>*H zaG8^Jko`==EXl6EkZX`HpaMJrXqO7`OR+5 z03eCX#&}l?&d(CcyX98M+gTZ$M8v6U0#k64xJppND$0U5;V zD?^!f5hd9&-KFyY!5h&5%F+V;ljPz}nKZjege(vC=IdFLWSJ7Ht-m?@-_ZFm^&9`c zkZFT(3!C#l-XWHFWiRZbK4b7dJ5}m zr~Qn!{nZ+`z57h!j!)JGsbAIGPo>L*Rv^lBbv(48Gwe^vai4y5#u%g;03pN&Gee@f zT>}EwOoZ}9nlu8$X#WCN16N11Z9SQ80gdLf~7r3B4xl^h! zFu-yGI1Wa6g}38SqlC^a18e;Pa53cRv&6qNMBOjAu{O$TCK-3cyr8o(+h;P*M4)$% zbGX{FyVq@t@v~ljiqbkwxQh3Yr`Ef7qQc5B91_Qx_}nW%H?|U<6h(#|-j9VG$1LQM z;Xg`){Qfnq#@3*jhg6FYuQ2&bw!)Xgd)jdR)x9$op*sADqREc|R=|v4;gCqWsN8_4 zmLh%+ZtnBJ@BTa5+T6rQ;=uUSGEMqs688{Do!|_}vkQ;QzJ6%4Z#eRfr7z-ldOuFV zudy}hSDz!PK$>r!%psDWYJQT*l*agvJyS-84H`k2sGAa&UWK{e6R4GM?7Uj9=8m?i zm6uoISHf80m&OI}v3+qgati0oI|gY!W2HuCL0{RcDu zYpqs+Sva&GXxfALZxR3r=*jj{qe4dd$<}2cG z=ABcc2L4vSkeP^~#n5(`uyFUI%O<`C>-e_H+KH1=hRzG*A<_`om);j|H28r7>yK|C zjiFda@I~>a_p%&c=9^8F3Bg_6d#D?~(nb;6;(if=G0yy(q8Z5mGoSv_8n>^(E4Vz) zAX}+(9SU+DpsTHL1@kEivd7Uvvz)BHZRRTi>g`obfNUksDA;{yI>lfmGwMk6N}^04 z9~Pq#APNI$iaVX3)ZL#JLcfR0YVZvjA+-!XB7=K|R~bPz|Ekx4byWT{zGil4({U&X z^}HnqyoNzCvqJ>w@f9^nhu1@p-5XQ>l!}gKKV}?nvjidgi_tJstvAR^N_T|&l9>~M zjiIPl=yxe=%5oBO#BOJGWeRT zy9?Y^Y3lepq4*zA{V&|Mt9VF3A8R zzwR}@`g(bN?DlEKy)znAUgW<|n9QNbTcDaT8Opbp($iL5a*y5X`;{`gwdfKT;v+B7 zIDp)AXs-rj$vioMU7?$eDXz5AR?gyJK+Y$p&AA!8%PTQWwHw96K{Hc=FbLcXGr~1D zI58^4PtLYcr#&2FxiL-|TN>kLBqfr7 z=fx4leP4|n$I*^4{_NFKJm1LCkE#@HmoV9jh>70jf=x)!;2WR^$9-EdNt`1{oJe1Y|`k zI}dGOv5Y|r&9DWTc08f-XsJ{dB+rf!PLDjaq~edd2W&#=Bh^cA+##Y2H(8N|X#R1) zLhL!W;acXGgdA1Fy7~@sX9$Z1e%lWHO_&I$*;vfJwv>V&qmEX~ShB2*&4tzjWoUgH zC)*=3u!Qy)YH~_7b$-4+qvGA?f=?wn=C#{E*6apzx1e!Bw2@sQ@|b78a%A5L_A!Sm zQuyX6v11!N8^P|3>D91yNq;y0CsawN%xG>&UXQw4{Qi2;Ar>pT7gVid9Q5-rk9YVN z=puusUmWt}7{++n91`==Nf<5^i}FxQQi>+$l#Jh3a11A644I-$nTksEQDHiU6V0Hd zo`~@=*uq=Df>6$&tRUHH1BYN5i65%7&`7TSwxIt3?*Br1Qdt>inrGuW|8^w(P?LD~ADHYymL- zT1EwVnc~8O)LY?^fNEAM=+YPh>5u>f5ejZIJ2&5UZ4ruB5Q@LK8t|<7$7TGLki&s0?T=bJ?*vw+TW64cH{^U9A}sc{cHCFA*4@n=Hjq>mW}IW*NfI}BHj$#xaIh%I3ITW$tv~R@=Xz*Yca+K z1TzR?A1ZYWUa{80w0zEK3oHTf2m&hwm2fNmeUnz#Ye2tK&jP(ogvW_pA3dSycL8IK z%I`^!^oAu%#+rDzCX4jH+UdrglmuxkS6=Eu(Hg_)csOPeaOz^py%x8?sLgyHOsAdu`b!f_D0Nvdn$Y&YIXL=84m}y zxB^QEQGiZ5d_Oh+NWdGgB>i5L2p$8H$;Z!M#MdKNX&0o=uDGt!9@`}O+R$KbKP?V&+Vge zuYkqBanZ)Sp?dOQzQp*k3z#v7Xn8h=OJN8E48kOol8Sc6#XE$!LVG42=w$bsAz$=> z`FOi{5C3HF8O0hPc&&r7UTcSN!pJ>@k0FYa!Au`)?dyg|ATV?wzZSx}yBv{q={$iZ ze0(a|o`GakAPGv%4g)*NsYtFxTAQ~2MXhtC8ufFlfC&~yR`N~609pHl(r{`4hhN

Wf3G5?59sIn13arFhD3Q zxClTKM-y8k6EQm%TW2+UV*}^^+sUhbtf9Ijk4Gt#7 z^^1bTyGayevV3Q~5S~8bwoAW4uPxbr zT*zaKB~NFBgkehmyy!gt#}7Fr@VBSb{TZRYkM*N1+sX`=_dK_q^Dd%Q&hbTZjcNbc z)TE}s{;%?E7YU7(K7GX7HXb^&dy5MSzQ~}>UqYXm!10{DST&|AogPyEoC667PY=nX zPd5%QklE#u?pR8U!&8F>24oA$46LgCHig%z6Fg}Fk|&1aDF=r*)0I9JuFln(jo;IT zQU%#9zX*QniD@;6!VXn|8MT-hPQg;MoJL7vmq7wA>MLSSL4`LI;6%~GL?hOw<|HL% zX7k83H-&LB6)Z8(afAxSQ~^pj@lO;w51k4a)NW;yaFUjjYDZtj!ka*C4Z#qd;=tt+ zrb4y)-?u-JpUJ(&2I|dq-@2Bk$Rmi92jH@tSFGi~;Rebt6=+W5BcB1efae-6-3&gG*iyc^C z=a!&AGM3Y=#I~RJ!o##VcA(S#ST5#D>rm~ghFZv>_j+1%3vI3YnPGogy%W?(5Y?Qt z7N#Op6~3ZMbZIh8ib_ExejVKWC3qYSh%1Wr^M-Uez(tBTT!EiRoG^pb6`-U|uo_*; za6nphuXTcMaE6=<0i%w;aw*A9QgD=5uO2Gmlx?DH zCXstAn7uq-$C=@UY?^LyzFXewEWKIZiLmS*5@%5xYNIWsOrW)4u-oBLvV{}fz)>;= zb1J71j%wuy3*mMq^;wLAS~Rz->E_rq z89KgmrD2Xygx~jab+xorc~JLM=;>6V*vzM=|^2Lyp!GK;il6z8Rs}@+EHs* zy@d7S71qtKG4}^q4`ad<6iT>Fa|mR6KB6x55Lw~B?^vQ~{GA_D&qr#yjYPF=G>Su} z!c_fD#+jzYxaCR~^mI>Uq|Fv8yD{Uff;_f@L(SI&dT;heP+PkO2^}ue_f$@hLD}?M z2i}h1)asoTKIOk>(By9+DKrLqBM=pumvHW>z%wyHNk zj`xEd8P1H!q;ujgGwG)IS&zv+-UHf&iuje$Vw9L!x(npy2N=2wzAAyP8Y+*Y+Sw{F zeW?L2cg^b)x{*iJA;KO9i8D8L#z;r`u0JsVay0Ju zwj2}$Eh$K*%J=#y#8zEkjEKL+*>ngSqMb;oH!lmvsf zzf>Q00Wh-BsH-3Z)O(jS&YCF2YuBRHFygSMjPFzwL1SsS^A&!R+O$~4WE4orqb^dN zzL52m(_faLW7ujR@nj&i)*F{;+a@_3?BG6nRx}1HzsGF$uyqCXZ7?gu??AzY@y`~U z!?~HctP3llehVq1&<`-7w7A{&3=Lrjsx5~rFDEde^0U-6#wJ^d@(i~*bT!JkQ?$c! zl4%O^cO*q!10KEIB04nhuRnoyGyECYbh2;!hr9>x@>&6vZD>Ef>9ihFjWTSllnod) z=S`~@pt%*X$A%`EhaHZ%bBi3gc-%CL93{9Z5?-DrhVzT3bt{(m%^o`*m;KpWeK@~5 z@#1$uRNVftuCzeWfpSw3J^ke(fVFGmit8AST}{!+(wh`e zL67K>|A)^t{xLk5{ljf*7+8m&hWNXbxdfQaO$JgFr*Z_o{}v2=rcgqYCdK7$Xj{;J zb{v0sc|C>s37ChXXR5^PyNpTi&9D<9zaPA%DQ&0lc%ZOWd0@}>FJ@3A#6tnUb-QXi zFGX{)V#>@GBy9qUd(UEg;$iX~Uo)MoJR z&Cbx~s?H+|G$Y%eYL2NHS6d9e&SmWy;<(swgIQ!>qswKLRNEHJZ_*S{L6w?6Lq;Ux zneE5|N(eZ7jvXS{AZ|QWdcMsTNaqMT0F~GJzZSLtdE*82sLKt(%v{bIB;!-SBZ}hF znscu=vF*-0VPyKH3z>j?=GWDeaJq-*Lz8jOf&vS1!XX~Dan5%%1e$OyB52JwgEK)n zxflT}?crphlYbD{UKI0r^lUjHZLetp*!UYi9h?)4UzaL*TZL=xuKl=!*O#5g)~#*d z8g|104ro5jl5cD_ExK)MN456nb`t&Z5AYe=cOc|~R_3h0ORW<$I@|_=YpuReMIUP^ zBg*e9hE+rc9qzV!2rux3d65PN;27LrXq}Nh3>Dy-IL2Ti z*%~G?(5#PTU;Q0>|NHPiB2)E$#on=HAt2#Ceg6FU(_a}_Z0ofux{qO`+A<(8M_h~s zl65v^z~41sqp}+?pyM|*b>L;ers!l+7QnKRNfv_AD)q!Q7$i(tx-O5XTXo4&nJ7heMvRM=Nm)3pkIXekF#87_^|v7a1Vk!Ei5DphO95{@mK~HT6n;}yr{pn*5D&@lK*ig z|6LS#z6{CD^|8YEvBLXtgDvcsY(WSEBr+&cp^Zwb@mTT2-(~L)*=uP)rUN9TK=@RD zauag$wvH&}3TF%*^xxysA7WLk=nOC_RDF$XX8BH3+p{_iYjDE;{c@ucc>&4d3%SYR zKYo&BKjyFJ^#{i!f(`qZeH$P&CiI`*h=43aqy0Th4V0ulJ3$hgt<)5QCC%RXO-o8} z{}y-Qzuz=_5noRz8#z}~H7T~K3QRPQ2FqSok<`y`AU_n0*WM^#Ow>ytc9Sc$>5kHs z5vns1b+-+7wqb}B2XKq~raOYag@I%1?61L9keM1Xo(mWrk_y?f2A-28Dw*l;7VpS_ zsxFsWClZQo+G2dz^|ONOUetfO?4e=da5*x?dx00*{k4JrXPXHA+Nb~C!TfLy66p^= zbQ&{J0D%}}Ps@%Bx&Y&X2G(iX*sE&b`i%SR08-(HY!?|)uTcHs$85Wmmz61K7&6hM z!P!X{f9AsU<{*XG|E_2WiFj(#x@P#wS$;xi%~_{RHY|8n>s z_I?zu={?>=<>pqym)?tudzaC&-+u0uaqrP_w#P;M@<8}W=Yqjk6UYl1)vgyvPph0s z-uc4F#~L%Cqs7<6q*e)|uwOQ7u4qeOLL7PACfGt$eY)N)d&;XF-m*8NxFoO&oUhyWexnT*6ABaL($cg6?|QD zB}-U}=fQct{%T1T09-1wlR-L?Pv`g!Hyf$tdSB!-g4M`1KVL%ukq$`ACH-k)&Tf!Z zk3{4TiH9L*<^1)T1SQg?LCw+75UN|5Pc$>~=JuQr<+B>PPKY^SUkX%W(Qi0>QhZIy z?~6_8kKUFkUBQGUv$iGGW`>CK^F|!nkZ1#*SooI;5M|&TNWf!o9Ya~;5}V0hsY+7e z#IqQ;+v)oxVHH#o4o8((HV6mkxp6tx!Z7GX%#AMbFs5}<-|rGRum?BkW=&SoA4158 zyU@HPyETzZ1${H;?o@+=jxFJuzB_2-AG*GI6FSA#&GP)gHmn%LCae+KP<)xGoKuJX zzrz+)a&sY?x_~knkxU$5Sj0G5qf=xdYq=OQD>Kh30W-z+#0EsJ`85_^K_Pl*hYBaA z$(S6j`>ePHp1dFioW+suA#D|ezIJb0U6y$?x{w%|aVQqHOO_Ml@5GDtqEL6i0UU%O zE*eGkr06h{owSun4Kf}d&6O?G` zyoPMDQ!Y`%hXqBvZi_|E1wf380oD3d_p8A!>XM6{7GNqyIC3U{!C72>#@oz79COAL zvE=w4v_KH?hEaN5HERb#IA4B3s-rx7DSbr7N`@-a$fortE$`u8Bz}5DazWCTABvdK zvGUc!Rw9nt95hKc5#F98sj0(vyC>vYl+9*CChC3FA=iy0EI!)$i_WicvD7?r@Rko% z=m@V&0_h`ceqcC ziXcesFmpJm(chx!nKDss;fv&)P}%u-gtnujjNElSv~x_NokC_haB?zkT>A(5HdR^c zy4JkiZ^?`-#V0R4peKE5HPuxs@mq%r@6xQv>T4Be0^vOq&nKBOK)=MY$M>QqZKgi} zT3{UaWM=AMl;-otIH}{ohD@d)BK8ptg7pAnv(08dt53Hm4{#)rDmjrYHxavcR>7@ zLMGiBlNMBtEw0MTB|H|VeLE-fRFT&^ZD9G-vYXkQ5(LhcgBW+YI&u5(@vDC`F^Bm8 zz)vseIl8wkUcnGz!&YZg0;PoplhT|Re-UC)rmW&}rrRviG^nI$X|6R&kx9i88_#1+ zW(`KVi!9pK~?Vx>wqU)=(%F4~>ep zmQvBZ?OT$Hpx|fbp7{#o!kTQn5c@VPs5dVrsRN;~7^fJD>CTpv=HHIjhz%8}|3Z4+Mui zni$fy?a&x#ce~t>&EVC*DAKMH+WT+0BM{{y8~e0+wq?#sT*tgzntQsdQ$Tw?OBX%M zc}9s5-B9XgojURNPQv8xlnN?a{3;iIj($cP4&aR5q~GG4y^$TH0MWP| zA-R`42(O<9%torE^o69|6>Kov6b`(#%QbyM83p7^CMux=FB<{-l;u~4#XM;z(!Cqq zmtIo#lg#|bK9Qx4^zc5+{+hm7- zJUtSNrxi~_>XP=dU~lD<^=4TsZptL(5Ew?M@!A*rmLW}rOw+}CCB%3}aF&1Fb~yv) zYI=`VKZN20u@tlD&*Eq>#&##*Vd|>7s8Q=``At~BZRD>atSsL@yaHh)bhF9aDz4eK zwx04C@3E#v%Hs}AcNfCOM}VPQLjNuC%; zW1;e4PUfKUO%ychtGkd=O(H+qB$~DMsg%#$rQUo9=GiH`Dm;qfyFE-=b>?3C>T#Sp z3BYd=SPfjBnB>GZ<^ablEsfvq`d5Z-_UGze#X>)if$KYm9K8~vIHAK#WoT*xQVA2eC~bF0Sum_^q=onaRA3v9$b9JTNX2B7e#G3 zbw$dCg8~{w8Ox)?=u~~_cDCRW{x~*I5GPFoDcFF@Nv$X%*vip1hq!AecL$~#vKU?X z?)Bbs!`9yx4=iOZ^CVASB(z!!88UJtU>QfpQK#n~UodgnaUWVW^n?hJyUQ~-Lrqc3 zxeqFav4!~MUx2>&@j$XEvoof9!|iU@*G+xy+~;$I9ijL+#O0%B&&WPBM|%qm{gVVR zuc4vxe(qlnyN`I8`TA`0)lV{3o$nE_1!C#9x!>_x{f<~1a*|A1s_wRNPu}pd^dElQgxTgIsBO79$1nh zvo#_k)Nsu_8!11o8iMM5G*6W(aiFLcyKnVXYr!@Ps;Mm?#(m{6VcNSM+D;^aW#Di) zX>)ctMGp#l#$y0lZcBY{l`@WZsJ|%Le|XpbGP0~qx=A6ZhH@D8XsEw^QO!-E+d~in zDS|a_Ym)gEu}PYMm1=BGe^B~T>f>3e-QHficS9^Gf+ZL@@et^Z3;`d~JX25w^r6*& zjraGLKN#}lhoAf3%&+0QCKSfMZ740nB(MK=wX}|s-2YoSNo|DKL?-| zt>-R2;VwP4sKM4q=o9J@It!)}{U|lCnMf}F2JB;EF~~Cxj~u!R?kk}5o!6G@n1ub^ zmO?my*{L+*j>>;)PhChH(@8BJ1x?1M6usMEUkEnJJwp~z(PAlkOOudJVijA#EW{HR z#gfowR?U;|>zP=QtIbg!@4}yY_yPtX?R3f$z(u=xMMWNFIsPxD1 zxK(FItygYz_f%?BXlDSi|3a@9+;UV_m#xWhSo7U_eREj#Y5j{3-o)2lM<2RQI49Ve z@RBC%VF9oNuD?$I@BRGOZ~5<2;S?|>oBN1Il%xL5$A!STF@a_3Rx0>vI6g>_Xx%wD zzS^ghlf|S((9!d^>*u#V?|)2MBO}=&w{1e}cyoQG zaoSKYP0*a`z!u)kB$2@q4&o_>`xHxIwYox(z0%8`Nmx=qj{z} z+N7Wv1}YD;aiBi))l*>`vz4YN<7d)IA6$$6oE#vQrqR6vj_?tYd$1ku5^B8zCAb0r{ zNdSjj9YCJp3vs$hn94QJb3Ct=ObzMGARq50{7u>ECQzNm!2B6rf%Nz9ABdKUc3(}* zZ9Ek1sRd8Ekq*O})$OgjL(9r)&M7jVy`?QVkZ}_?5#ZVTJ%2+r4|8wim^n4~7n)In zbZ52}yZtg)sOyKFQ9vlui@iIugS`iWzHfD;YFyFE_B4!EU8dHkUuMS&-Lyrh>I0^J zo_^?nuLB3!NUZ?fGLi*(t{OOCCn$}UgcS*%85<)#pSR$kP8$mYJ{ywi(oeQ62OIEs zjw{P|M4#>HqQHVlq%PE=*@$&Sf$lhi!<^7@6CG2OUfGRI! zvJ|B7t{%6-_)@lveGs57;S zhqzEQxG&HMRR?QzEF(}U^{1tN26mSKDAL+5rR3l%Ru4#qs#yvC(|tMVx-j;)#iwOM zU%LcP?D`lQ6DVCq)3^Hvjnrb-%Br5ble%8vy7iF=FhBD(76N3kpVN;6NPtja$qC`$yXH8Y?nSh}d zRW*VkhNq@?h@vnBI_h3tT3DcBu8YCacrTi+T-0!6pX6%n(*Mx4YM0C*SV9>-`VDj) z`tu2S6A`Dh9oNd;Li2=)N*^nr>ndvP%4q971@@Wd@X=&%E8wh7Hq5 zexZkX60cysY7>z^Q1@*SLWz7E2o%k zrkO5_+^-KLkq1DJaRRW7=^%f;3%H$)5=Js#d4l{o8*iDCd-Rg{H7ud9^h;nSDt|YK z4m5?YD840>bIyf8Z&bNcG7@+9V~8t6tx=zx@myNa$e2B#tTC0YC1llzSx&ELjX?3` z=P{iPC$dg;q-$KqezaAV z-!tj{#HKMsd$IkT%3kqO`s1|Alz(6(dumH-#*;$-544yob0+m3Vjkr81%!336Z(y^ zKOKbv{*?SXsS1wqq4KPP8|uz+0DfIC2bct6gK4-9e2+#9Eue|zpMz1>*$P@~YBd}F zz=4ua%*PNHkaWE&3XizB;Zz4La%-|RSrv>M8HONC%)ksrRvIY>qJSgnjH2S!2Ekpk zOI^&}p4&^IvDh+I(}cOr22j7(rpU>>NPv&z}V$Xi$S;1b;?}hI18(s$pW1s#lw_XYdR-S)WH`ZnQ&aiUnme z{*Zz#3sT6--*0Sp%>B@o;8T={lFINQe2`+rm{Y(C&$b_gS-osL;;Xnr3Mi@h!b(4y zV@H<%kU>@bg$+Z|s8$ZIXBdj6Cdo)swLP(oG~mhivltH$MGx09TO5To&q_z}7dBP`%lVM8Q@3Yoaf227tM; zjHdaH;~t0gNjPwhbI3<gp7(b9aQ}cH(Fm?&ZqE)*t%{H8&9-*K4V3g+lB8 zGw5T~xNacUBi7PViWs>OS;#$m#dnY4Cbp0qo-GnuZr6?9Cn!)W=tFj{O(I`VVCO2TckM^CnRLr(=qVnjPbx z`a9vT3O1e}j4bcpwmxP81bEQ;4{mr+DH<O`vyxnF8&58B#RSzpKBg;tt-LE3nqd=1PYOr{4_~!_ zCzq#Hr?WPOH}B^a;!lB9fauC=8Y$0p{o;eArrDg@yV5l07WMq5wm+u%&z01`u z_8MM);cfEks^sD1aZ^i!?U-7ALNMRv8LwYQnpw}HrM$&Z+Uv-3O5|VP8Q;1s+3IV` za@B=Z*pOt}h3V>+c$~P=}tb>olu%@G{UGAuhg04l7k{I^e(Bh_iC-BuFM# z8Vs{=%M8rT*(QJXqZCgnM64(3H8x5nkXV*J&byALy$YzTIjbM+GH|`8PnYEObS5a* zMj`5oic7T6qHr?EV!`Qg1qOO6SwO5cVo81$01u=qOS3qqXIY_`_neUdIuoNA=qot( z#3=GZK!lEUl5o`*_6wsk(mfr6+n8ws6rH$Fn>}2Iej9{m$!DE+hs<0y2@j5%Ql*~= z940_2U0^q&_EYym`US))$K3U?{W0=T;HOJ`iWa>L^wnuBJ5gX2NR74qt?DW10J0kx zK0m1@q%o5og1swv#b;{-{zkb^Se$7$3Ft4=ZPLLTuVzg!vO65S;4v}jGPTpVs@+M} zull1Akaw;5*$4^wz?e3Z@*725!{}=0;c_Uj2_w&!fo?yH4}Y5Bd^VzQ_3U?n#jlOC^SU{mQE!J zEvkL05VM0}ie9G=`3|;j*&Cv4p+|(GCJHW<*W}kY;GM${{mp)MVw^esZ5dqLxzB{~ zk{nQx&;zxvaUIb5Ktg3G9JWI~HEk)BJa9b@j@+QhE;;ewlzhuN?41g`hK}FWHTrjX zY4pXV!P>kyu>(v7Z+;gXfN6Y^gJD5%GUXl^%Sdl2 zS`9$^lEcI6>nrpvZ@7{8`N09d2!mg<3=Q@wbmY_}6J_}WP#uz&=PD)<-Y5|s{T-@D zaSG)bwywj-_!LUet53DhP^z)XQ?u$Dty-&cF?T=9r9#oN1#Mtb@beZp$_cDs>_^ZS zFuW-$_$a%|^Y9JdpLR9dX?3m`B?p4zs4bz$CktP)_m5XRv4JNU(KBvWrwAEQlu1-0Z_xGK1U?@l-vW$MXlqr-yL6V3cTMH9^i&oDiS4~$;_oh+_MpJpT~H!1-?{Q~ITCF8(7RK=c`e$__;cvV-G= zX&?(`EdRHA)Ikox7!IlCBj5Sh&KbJ>VHVLLLCkez*q}esOrJq-klf6`vRE6dSz(lc z_qd%{eZ2E!?FlolI)+ukdaw=cCxG?^cX1Ymce3n#tIbT_j>v@=EL@@=)5|9xs*)P7 z%lLXh3<_g6SJyqRcig*?!`~C1hz!Rh5F3sw?WGvp2-TD*iD|LqFe!po8gwmd)Op_5jzBjzSoP>LmKL7q zkVP8gATlppB%?ceglXzm z9p%BO%z%RSU<}66QJ?BrF%PfK;EA&byZY}6Py0v!Nw>`JXJj#;s=<|y+m*Mn5^#vYU(~eKXu0BuaWOl zdsJEwLI9i>hwT?rY}NBtotKI3?G-xfHoqc)v31zO24{ra1|NiLh!zBP=alBR;bR(z z_G}S?(F{&1S}P==2AE9|YaWBzps^R~LNTl0%&cD7J!*do&8oUWHTz^Lq^fYjYH@4> zPC@9B_J*Pb<9QaZ<&LGOYncoU)91)S-M{?fv_CMG4+~PdWW)wN{h%TSs;{W8NMfx0 z3Ni8T>GPL`mBtY5#O|b#UcpdtbP$!bO5^1Gnz9bXUjJut!lW7!6Hyu^@~dXxPb)Ls zfy%3^bu#%k7dj7XMiXpoJ~M4w-fLqo(9RcUXFTIRW#_BlMP**Wi_H$3UU6p18YyqLMB!t1OPD>*Y zoWEGV!J}oD0J(po;CLHoEetf$l^gUUj$lUi$ty;?RI@s<<)C z;aj*E9*n?qEv+Kk1!dN`mO6RmWrAe7hgn7gCKSALywTE^=3kGjpGPFCj6?Af*_nnp zR|`t&y>xSJCTndVlppBKV)O2yInva$on)y#vr#!;0dK77E=i|?moG10HN6HHHEcx8O{)Y@P`=ezh2YB}h!VZ{U*oKqe z3MbJj4MeS&Mu=}pWxZGKSCX4KL)0y3y=NAqn~bce&k#wvK!FoSfwHW@F{Rlf7?Z;D zx{he$K3&UsL}`A@t<-~C*Ep#0qiGb_RcaGsB>p27bua-b9;g-k1}$`nUrKg^9jKp= z#IpLGOEi4P2QybK9JOH)Y~4%N;iurucC1uc1rV@AueHZ4oK+wY4IzF_zFPLkuqFD2n=ULdpZ)A0YWUskWh5g5#$o?gbC)wAw-*bE*Dh5{;{7;bj zgFXEF4<+x+;WG+@evgPTFR{vV_CsYiQN0nUUEdZOYZgzAFP0nFy8ghs!?)2`zVVGi;zOAU|3*UNeD+=W&hNi^`YN{gEzpAL)?<8 ztM{p^TTT{qewyd{X7d|jS%Hw~X|^MW>6MB%y?+0MQT7WsB%i`>{p<*4t|(AX39T&d50+;nw=^d zLL`4|*aGu~9ex*#obB{Rhu@cov3RO1TZqps{={2&Sn@dJkgD_V%upEnd_2uw5ty4> zTs{KSbboPF2ADj|;%4v` z9tGMuCWW1-#lvPfUC{+}&a$JH@riU-pvQbLs+HLc00$nj)@h;Wcxm_l+J{kc+qxpW6)-F%mHsy60mb*E2hQl+Mi-G=r zw7L5*0+%Sqw@zSGng?sRBKo*EqrNph;L{KS@Dbf39e4GJGwiNPIQuk4H|k1pVf6H+ zv|7gSbtE&E9nXDfP$U7=(+PptmusjO#NXvO5bSdf6IK9zi^9HUMgrCz>WaDac^i%J z0^9-XkxCF`ZSdz8fqn~6UMD9NLyc&l-X+72dLzeN(4uS{zpPSHieT-PYK5C&+RtS|H2NhoI@LKw#`&-0HpBb_p>D^Ok$*epdJ`k9nER>==vXE_7%3i zOpFO{>+^oDhu^Qa(%YR>0h_A`L2$WGO^GZ{2h)W~=2>r-DaK>0BC`P15zbk?k8nMfs-r1HRtW z;IWfI*wW7#Q;tVgj|y?Y#uS_)B}Z`YjrSx(&x8uU2n|uC+sZ)&74XcVA?`vDv@^KF zpitFb3)fv0f)QpvXG3V3m9_ebA%2xE zF8M4Q**2<2#JA*%Y!SK4tCY%bxd}}d9$8_b_%SAraG8(jZyyj21VxlK=>VXU)#m=A z+O&?q>ZBA^%xNHS-gZZPIhnJIXEksamEsUEiG2n;n1TA@)n z8B}(??$X&bdd5i%HPV#hbo8cq{mj0ZH^U5S3D?153o1Lx8U5}4uq?Vk!->tRZ3ks~ zxAT?H*7!X-3oF)?46=LGB7i+alxyziVc~o_01TlNrJxFCOfQG{!k@9*Vn>g$!;pfe zAA#P6;uVN_RiSahPV}tGwVzuuvuXh9~s_cP{_DI)w z4%SE$bfCA_Vf8Ugd+sZxEjIiGqQ_DH0Z#5AdCnQPe%`5pK-m7Ad82?_-Xk)(yXXPY ztiETW^42&*i8-x9PrNBlN!Z~uEUzSCL~%g-4k_$?8KLKvGH*v^=26yT;3&U{qLZa!p`*nieid za^4slJ>b?mK}cI7I3KAT%-HQqLlnHXM87Wu&))yszd`w#0liqzhyjjOxTCR`&|q-z ztTr7?u+3;4o>&o@-cy8-rv-^5NA7&^VnbKur!S4@N_YLPkQusJ9Uof#tAn$=CKmT( zQ-ko!mRrH|m_IY&6$&bwXQI?`f*S%`ij+KkB8X?NU)fT<7n62e{?t$gJhmUk6)fEo zT-aTAslD{2C{?i3qXBB0O6HxBWq54DojqB1Cf<2I*!CL6~^XgMDumOBGv~N1#GqO z``Gn1PRs+~1S=XQz(VpQHmi_# z(~{7i9Wy%C+FRtM46$9(A{n%s&?tFND>Fj^2Ka@ZS`zmQLmnT<+5(TF6+Vee=opAJ zSq;iWWGug5$rvei_}2=U32NWQ092iRe6QTFa^}9kqwr`mCuiMx7rkEmbsX4~BIkMe zaAlg5iO!@?1(* z#V{=iFOF7yL#6?fkmNOPj^ZlNDcWEnkbrzFExJlGX22uo_%tbC$Mjmdzht7ovl)eA zS_NqpM0XD7<>M(UM1yJCN(&`g&qpAO?`7Yka5PpjhKHw z%m4epQ5DC8D?gty7Z?y_7#<7Qg~*8n+(P1p1a^jVLN_TAAPj;06OM#jr+&caJB|YZ z+;aK*j&DJHhT{+d9ViG+0tL)K=Ya(RUO&AWKdt8EbzF}d3ot|z!*3LHvR6sM^9SI* z2`A)+VP7b$&em4*f7&zB2>4e`xor)Il6xB6ErhU4XxT8|pj{guj%B3Yj&ri|ZUVgD zK@DNbAahe#3=MgtS0baez);4Hhx)RK%Sek%ga&gF6Nnd*p=LbD-O4!w{F89%C6em_ ze+J>>WVzd923u7qHENBeuItxJVILu?R-9Qie^{?=>{k|?U6}Q4*6~4_M2xo5k5q`H z6YD1{H=lAm@(Fd$hwrB-KfT9kt?usCjw4KO*{yD3W(}qb|M1v(R*!g%=Jg3zC2?om z;?5{Jy9KV*x7emlmcHg50BigB*KX1P;Y&D_BF2M<4L|LCv61JMOf@GGtHcX5l@VY% z7^sB41eFOC4V-b=HBoTT&zE1zW_Ck5`w;i-y}CE0G~lqYl8ftIRPPyqm6sFDJ+4B4v8nO>>}k&e|g8` zJ$$8cTiy5`Oi>E#xc?;O-#}yv-yw28Tcm(CAb5vDhTsmN)|oP>yH!^2|nTeZ>;_& z-2Wz(CL0oj8xVwl)~psXVf^_-z8x@@0)ZB&3BiaCNJ|!$exm8KOTaV^%*H|n#g{0R zw9OB6JhIx=98c~Sk5=j1-0n$L7lzJx|0H+D3$^H;aLI1w8)p;knVElgVRQRQxNCOv zR05U6J7gCSAike87j)T?Ur^QL>{doY_L?|dNs+zhPYR}yu=IYF68QAd4l(*Fhx4qP z#!n77A7htvM`hYWe8iES^rjY$k>^H86~hvLJ8 zwU{6u+5b(sgFcIaL-W%T^HoTr_$>(#wmSGHP{o?^&WsIXTS-uoVp=n##ALRMBMp+F zn`x>i+Fx(YqPPrM=H_b1F|T{3WVerKL}1l%UrD5A2;G6pI*a=L8* z%dwfTK`bTQ(fb$P4>fL)3)RmpY*TEPwDpVBbf|P=Ir7z9A>;v`u)l>!JK5(Hvz6=( zBknk&-qB{f+`|x$`AYjVRLgBcF>0Oa)vfeIjx(caj||Y!Ym&2Cr76F`v&TQqq~O;~ zYtWkqP`wJ3EBKudDPmpwg{!`f?in%+R*_)b~ir9X4GBX8c z#c^W-l9c2WK$($!dKY1Hx1a)^eq4h}>fV=L3o|hbF{1J<7D&6Crt)66 zv1dz3_vI#gg_sOiNCVZ8zF>cO>q0r$E`gK|&AB-%8ttAb3orr`x<2O!Z~^%XnY3CiIz zUt(CP#b^|ZmPwJov5Ht?F=eu`avCy@mO89Z5IpO{-^1U-`AWw$U2Kx6z`|oasUBq& zUprJtJ`r7OV*HMTNB;r(_gwoM9?XBm=Re={3(?Ov-JKs-83N_+$6LhpleEt(h-aiv z*KME?9RdzeAekE;kfd?!hBuG#5rq{+PbWTnk|Vh zVmJyzPtinpN~Tw)aGj%1UcTVhJ~#@`H3$k#J5W!=Dw9l4D0uG4-7%>?(ItmCe?$iq z-j;@wFS;j9+qVV@%`eQZNPH)@$RQb7pCHQGd*;ey-{MOavD+uM5RyBF=X7K{#^-q9 zH_Sd4Nn+s`0OGH0a4v_Rq5`7>t*s74$EIB38WQ2Ol&aZ8fXJZ99EOirjRb+`{FtylGQC^rn560R($r2P3rH6Bp`ty^bGsz{# z7SYO{3mH}!Rs5;Zisu*ZoArT(DRF419P*Z}l8FZjFsyl5BMJFFJwv6>R`YWyZb6+M zM&mf#0R4*HcBPcUKm;q9aRH7JOkDy1Is6OwMv>>SG5fdp0#PM*Ftrm%o*Wz$JUvxh&NGgWSfn-WO6V;qz3Ofc)4&~0K#MJNgKGZVO6`Pr2;62nc`YjmY~fI8 zayIL^bds}7I@YYVQeHyDNvl!GLyOXfyK?yn{+mIR81YNARjWBcQ6==UkNLQjY0+b_ z%Xa9TtvY(dYjALIak_|)OI!QCCdZ{VkZnc^kDQfOoZY^OV{q``kLSJ z04*9lF7iA4)XGE(zgaG{Zy6YFJmEdzWRZzVwW0&qoZ-n9m)PHVHEjgbID^u39M(fK zB=!Z=myOC|Atkby%9Zl+5Bc~P)UI`3h1@C`M;)X_4+C+!pq}U(0G1Bb-xsZt;18qE zV7FKV_A&(GncQunhP8=CpUC_auC>1W2J~KPvEY1z9|XfTy5hzJYyMSI<)ps7BCKf7 zI#=m(awW${N9iq%QGOSN1)(zJapEtO0IejZvUMF}pC4N%0Wu6i|lJ zQZ%_4Y{o_6Nhs2Q_Z(Gh5hpM`;rZ)EN`F~Js@&A-IkZrxXKvnGG?h^kPH9#Z>#0G5 zqKJjH?pQW)V?ve&H*c}j?fg*YQg}eMPIx%NBS*(uZkAm=NoI+lwFXjRmVyC2Lty7H zxDuu$Ra~l_53?W7^ z6rv{^K_QE|D&2A~L_=t=nnZYbF^{C;IylrZ0g+{_qv^!dR-kA(FAbM^XcZ!~^O@La zeck?T)FQXrno#e02V1ZGH!K4|QI~5do84DEe3$F$2onavA2mBWN?0*1?tr!aA9!xJ z4KmF^zGx|;V5ogQLFT!B5?+L&G~_4^Oi$Cd-DJ(ESzV**g!xQMtyJ820# z7*Ozf0aj-g`!f7NB!NshYEUqwRP$MU0fTu=tZCD}5rQRj_n+}WNxwq1?;V{(tTN=( z@7He+X3{SbRQkTdw2^DxW(kGwDqI52cK|sySFQ(HNEjq7z~ZX&9H4A7ng+&gW@Emt zya(;n+GE!dv$lX&B?VZNkB?s#89sXxEK#3y^NHwCc=rAFtS?R}kpJ6^DB%xUX}?8t zy8hT}^$~Z2Sy6;K%nDL)}?t-H1O z4rsnNX$;Z%P_41VOSmzru^RliMH&e&I2>L2t|z900|kAb{DT*eLtqvNeV>Fv;anZ* zE4*$7kDN<^O@>Zw5;{d-hEFo5JdhcPJJKAXczXcp>3rD4JX9<~!gd*cQ59z)tGz zPqU@lv=7~*n>+lJ;8babkn1i7)%!+JLw9_K#lDzS`7Jk}d*%k*^QJJfF8^?S@)#n@ zJG><F6+6buO;YSa8Ki9oKifHiN(AG_qf1) zWb=7%SBGsB_s{WrlEih4I=dUqVav@}|Kb(+hfNPyxw^6LBW&nOSy9JrR<#|j^Q_OA zPN?m*_CTPAn7XF)%At3}1*|X@M5Jo2Fh+#HfHEpw3gP(l+5#vN*$bXO5Jq?A8S0Cn zz`;A%T702Y)vOM)OsYSi#~|nU)kD0Y)_jFQSK1umMRT#sit(#g=^7%PaXG{Ivg+zv z_aaSaw+P1O-z(^{ClDkEmQ@AOts&Zp-9mPW5H@IPC)wXC+@Ry>cJbfrLtup^bT$%N zFPG=+?2LZBWC66sK06|>h^PAYOpe)@*h|@y3sN(f*t6aYj=a_y(MyjntaPz6@T8Lx z{X0&q>Wy75s=>?sbG= zul^N(%!5Z)C!Ta4TgmV5Eh}~}SC3|f-^5`S6wdWEK%W}iVad-wEfaLiRK_C;$4)!Cw~?@KAOWF!&_3aMpbnGxQ^ zG@x@|s*G`qv=!|Eji5KBi%Jjdm?N|-Wxz#OhrTVoGqZ11Rg(23VBI3Xf`n!RhDve^ zVg`1@$G;bLqwF0nikaqP&iYw{DtMSD1x4X&@zxXSgQ@xyey{4&g79~*W4j8InZPe= z*`Gsw7fqIW^g~hzak=*p8xr5^rJKjmiEKYS)+t76hcvOZG=gTIzJ{L_@U+m7CuA0a zFFt;g(XycCLc8}1K>A4-iWYUY*(8MYd5iA_!xwf%iNen~E4M16`tgtRnsCgBbUPC1&?&4cOMSJ&2ZVlpOATqL&I}W?}@=V+h(>4 z=*F0HnZKe;RF#%vPNa;#ZnkD}7tx+6D=TanP3LF`)R>i3pA{M}fpN)!b7!p;nn52O*DBw^Id34^*pCJvBYGwt?m3 zbbhm7!iTwo;Ba!q>AfLw1-RAtW#-gMb{)omLxE?Iqm-sQGT(=Q`H1-Ye{(aXn!!3d8|$x+F*1t_tMKU#t>VXq6^Ddapx!Gc z)BL)(@^c-ZinQyqH*!YI4+v!9HHv^}n){%hIAw+aW~tL;^>z1 zkR!xcTmwq*2)}L9&$;n6!umf6x7(iczma_7NGQV{^m-u;=-%4WOzCDV)5?|NPvd}B z;>gipU2-iq777DO zj*5llGOu1(H+9hYle01HY-*^kAEMx@ql{}~XiJ!bOM{oqb#P0(&fiTV1N87`oEv!g z1obRZ0?L^v(*+HC5}tJ=S^d4j8B`E>)>Ef>g9@|qJ`%~I!)Gd zuNSBn=SVv~UP-F~hs+%;1Xx*?)U~-3R=sl%jrjonH|lbWV;qn=BD-H)YPd_|mf1h5 zp7d0m2_sOWN=xQW8f=ci7ueZ4PhY>;M^V&*aPE-5CisPx8{y>hO9)3_ zT(<#g?c(Zn!amlzGDH$!u-o%Xg|G)4+3?X+b?={d@e^A!Q4!o(Qgxo`+nvn7efjwmga_T)x+I!5oXz=F+P4Nt}Ut2TVV{6z*>N zJd9e5k`Hzzi-*h}ub)xsylJO__7g0?r>Y62M4!GZQTk9A?C?Hpf8^0GPBwJ;M}K@r zFozVavg{7MmWD?ZC@T@I0fIBkgcjeTLm>zpM1JksRi&#t6n$ELO#~)FgRCOoZ;4^m zSA22+5D2y~^V1xGpIO5MV)O9M2FG3;E~nzpHc=Dux_J&KYR9nfYp8wGly5G8Q|IW* zGy!I7!Vx_tkL2AT-Cc!f2%R(5^g3p1$dNvr58kv7CXcLhpYS6ZzhO=js!v-GA(>}- zQ0LDhUiv#dRbY(H5pnKi<5Yv_EQX}B>J)S0Y?*&J$_oMI9=7e)cd`w%>db&QKKvYt zNvlNQHeY__0Z?f_HJDHCJG2}i>tNPlrMlr69g}dMQ60Nr62lZ(6_PS3CvC<+{@6gq z%8N=Z>NmaFmQS+u-Vhdef*AYy{%={0h0Pxk1^WPtqGX9btV7ZO^QxBeWyHuD^gEyiI1y zjZ0Sx)L_p+4aVEQcRm`Q&(dvi&*S@#jTzJ=CwzT%c*bHqM~~p$(L*M&vD6MP?R8of z>bJH9y;0Uu+NMz7uY5-i2ARO$WRZHL$W6L{Q~h3u6AR#&4?zfC`2_CmetUb-J@j7Xffd_4VSzlg3D%Ne z$tjDT2fW-}OBC*&{K>Dj>Oe>n3cY=56<-oe}hw~i*RS)q!wMR0popU zt8N-TOnE{Y6bny|Vf%#+bbmhJ6QK4MuvU5|VyjL|OIt}RaM^fiZEY=}wXt3$KWjWZ z-2Vy)$fw?Bt?DHP1lOLxr$O<{jjKFOu~iBpT~e3t|FlCRr#p;x=wsXHl2)GJwNg6a{)|h<`!a z63P}XRN+Zct3<Hm)RENeu*yQ*kWh&AW$0dW@$LX z=FI0e$NBoGgWoaeM&`x`DzKuXe9~S!4LmV)Uy7>OPkMn# zr5X{m5IaXv6QnMa(_W8CZnr2uKHW`4W4`Kg88GD%^++pV+5P2P(I_8u~43}PwK4I&E(X%os+ zQU2iT2RgO@NJ8eZ7pe(+Av@!HS&wUR2A#YK`Z<-_Lw9(CmryySTZc5(gBMKqgy+Vjc3ePoM=(9F)(^wRZB5o;9#Szh55GpjX@-$xuC^iMQl1kqemcZHLdu^D6YU*hF z@QpIfM=CXODJ`%@48lCD*Pmo%jB$=Qwc3}ivDgJ#jZc;11S`a63d!0s1!9iH8y@K_ zz+X2j(TA_#GM<NQM2>lNCw%Cr9Y@(SD$g=W^*S+jYSeU&_)Tj(xlMu61HW)hPq1KZ);*&QNV(cRH ziWK_X7wL*Dj6wnJ(lEWQi*a~D+G*3sGZL&c95sjV$BjBto(AQFk2@qy?sY_7>hzTt z=uee8f(7DQn6jmQu{|m@By*RITtb}bd;@BLq@@^(y1CqlO!k!DTgqo$zKt*$qvm;A z@Pf`u?U~FY%P#O|hu9ac;orP8+40!b_-678hdFZX?Qi)j4bh3Hum@rTP`KM;XlKOz zoGz&@-Q__sFmwmJU}u_9f=aW&a|{*DTLW{bMqp`{!i80;qwru|_n5+Xm5yLbQ;bsp zJ`%XjdLvQh&i1M-3gN554Y^F%_P@ILF^;x7$s<2)i2UF?DuPQUXdXV;nC{1qv>xH~ zY|^;D|2AosJ$w_iuh=M5g|N4qwko&ISK~^Z9PM>fuH%57^-*>)5X4|VBr~|ty7}${ zic{I-Hi+C5-cDZCpeA8@JnKyek9Fh)AdE$j{Ppg4CiKdECLnQU@!MAGYg}xR|ANn5 za5RjVLBTwUa6KfX^07#uautgFoBt4nTyDsbr`2xcG*iO?t|~&=KE%;wWU%OTIARMl$!RZ&7$6*nm;j5gREKZ|;5}(` z>Q)ABB{3#;2jvBtQ!i2tIAs|AMOfm9pNz|5#Vyp`DW(H@#yuuf7Omw~ApR%eu#l;w zj}(i9U}FY&0tYC$&vK9L1XMR4bg+&pHVpHb5S@zkagt;dK1u9Ker9xHs0D3H2((F< zbjl+!CEp&nz1Pqntoa#xab*=Uz%r?FD}mumic>0{r@Kk_xvfB7PeQUDwEXEytIroV zT|XC3o^vtY7nra+ybO~kl&knKp4YGRjO;i z;0A1#%TJ;VT<_e@B8Z>zAMI{0V6Hl1%Eyb85YD+01W20?7RQ7}54PRn(26=ELlR^d z(43-f6Oaru*#5Vi7H$?;>l&Louab{~f182-ne+a;8Hn0Wm+1F-VVnIaRiXX@R5QmV z17-&CKmnqZR;7DC_r|P>o0!zsY+Qz`)d!RaJmo$q(?BZ;6%DE0iC9d9tC=K=&7O0$HPpf%Yf}ya@Q8RvHxZJiXdq z8>&Rp1YV4QwB#fPD2@~^w?f5iQ!np+MkK1{B1lE_;1 z5FG(XR7yFwPkvdO(qo>>if788#KlHi;SvUuNR_tT^Q6{VCe*_*O|qI>4MMy~J3tc@ z9)&vou zy3mj{v#f<*-=HG&I)mF^>P2VH26w+byfK5^t@j_>k^hX(e@6~LVA%;2;A6A^-$cF> zY%1Yn3gaQ^uW=YRh?@X(g(1^nyZv+$Q3Nh=8S-;ZZ7mDIp3LB?P}xjWsK z4YC`BzcTe#2|CTE37<^XN9LQ5%gYTgO^H!q7#S5h1}!R+qr6a=`P=oeh(Mx2%+b@L2K~~+6Kr93OFRtIg&;n z=Wpgk5#}8A`$(mnL&mL*A-e7ZwsMvbmRT30d{7?xzA8QpX{pMgVn*xl_D}0yg&J;J z*lY!nNRe5rkLy|3;S(tg;%`IG#`LRfztv-YY+yB!*0OEH=UxUW!<+t6Eavp2s6^HO z6C!j-Br3t!@4ld(o7dNhx(Hn%B?-OYnt|Z$!Ki4HE|^wb7;I$%n~Z=4z$loPW3%P5 zM795zU4XXPy~cA?Fg4T$Uz1-l89Qa-Kspe{pe&k{$nd)G*O z-V~ZIO=Gixz_z~Hq8wkqTon_rD|*Hap4ImG*(1}4utaCU1^W_+>NCLR)6M)qPfa9Y z;ebE2f`5yvojw?F!0Vp?AbrR4M*W3xh!NRDMd+vlD`H?Y!#Qq)bQ|2R<`7jRfzLap zIfE~!MnvG4h!HGc{pa3jqNC*tJIDShwbD*;!f=)Yz;`oD zAIW1Lau@9`MBXM za-o@K;XL@+#1)G+uv)oBAjpIjvzIw8I?PEP{*ZK{oj>P}e%y^_FQfpW5C)^-r&>gN zB<-HwZS;?6^J$%$#hrg9>Hi7o--!N~KId<+D@KQu4*o0F5T9}`bU2_w2oFZlXTU-K zl_C^VahG7h|52EZn?PRt>$a#OQ;PuP6GM>@Kqd0ej-_H6B@}S-uYmayj|&>;!ikOs zA`J}VL~j7pNV7DxoY;)mpMx_*=U=)8aXxsX@K{N-3D6S6;q=11NJDao>?;rrTmrnN zGX+|i>&oTMd$RDU2>Xdqyhq$`J+Kd$ZTMfo5uczgUUU)G?x3wso&muJxBV+{J5y*1 z1+JYpi(Ml0e)iXB`_J8>jryUM*mrg*RH^P=TNA(zE_6$f6CedQx+>Yf|Lp&S{$GNS zz5a##?>}_;KVr@*Pbxyd|LjXkNDnh9K916X3o0c4LALe)LRd_aQydrDBStgQwp$)w zv9>;frTShfxz-0p)P+RsADH5j`xR99dgL^7ncKoV`tYAO0zX?#45!IX)nRY|veEWa zVbE-0v?`pbC^g7Za*2Y=U*bbPa~DMv4OSq!_sfdlSr6dB5S$ryNX|>hdZ0*?+6Yq- zBs4CNGdC?Fq?s|yP2XLV`6Zb(Mo>Tigrgqm7bMp>13dNz9&hWst_Xhp zQ~sgX&>o#e*ygb|p@7b9*k6m3N}4@tUj=bEcKUg4GPgYwSHI1BLA?Z$=#3TIf)!m> zA4$j`%7PX-m;Fgzsyh%cYQ3k<6fB_iB2sL9vQR@yyWvB)T)cTYZH@zc-%09KQgPqfcZHU*S5G&CMl zMNx@kpGdjA9?t3*Hw9kPZyy1_KR^u8y<9AN`LEp!{)|ITMMDXPLm~HkI#!LzJn{Ky z^8t|4LWJnH;9v|ENO1fj`@be%QH7d$lj{p&#!z23lWd0t&VO0BY6x_J!9(pdYQ|N! zP1lJw$9>2!r%mTkV68?7-m1(EnWyh(YMz*in;T2%CO9;stBUA%I#u0Zvq|Mm8^`g4 zE3C{rW}atLU94^1?n`#!s4o~;o;kZ>`kWSN-z~Q~sqwJ;Rdh?7EJgevYZH7h#$-yf z;rwG^KjF;4>}qhFEom-x4!(2m!q)L~h_7!dgH27$K#fk0?=Z<%3H@jLHbhS-8Z z%b?-l!IA+A)=_$~dQ$rfQ>^@raSYTahh+=n}MrY8~bg< znLcUK1~g)78FK@f_o$idv#)M9J_Mk1v>25geIisabIy|(7(7b6hG{juGgcOdsvLg$ zG{2L>p|N~efm5>Bj56=3m6d@UWQD=O(V4<-g>{8%onQ@@$L9Tmt`=-#F?+h9a%LZg zqY!bmikdY#RYTexYZQ>E)gEiqoy-g!BL1u!sYSbcfvhfORH@lh)WfEBPxB>1!0MIc;_^P1E_+64UkOZg|TX>SCB) z6}hep!E73h{cu5Y-&)$XReWLUy|{T^>_)TX_w1WHOPg1*AsAQG!{}(${10Jo1r%7Z zaC63eO*W|wcQ;_~uiB;-)+&Hy>mX@DT)eJ3OneZdsz$0fs)?2B$bB`9pX)7{kj}W7 z_lX6GV^%edbW@ZA{rD>aM8l&TiKBv(PM!Npq5-&5GY5()o*pN*K)~-BJw>#sB`WMu zov`}$B+JN38nwIA{vuNXieqB$tJxJ)0xr-9M8a+?0v=(tJM4GP{71kx>q`}}O|`Es zWlNH?X^g-fSQgoznRzfJ_*l1_5uV|qP^T-jQlx2=H;6ONxr}edT%uR>){8Mpj~4O- z?m>j{-jvWR<#DR_m^YqIZ_Je$+1()xoM4oertU!xYrzyoz4BACm-?i_->YU0jjtH% z@kta-^I!6O%j>2kXeZ4o}M zNP@LmRd@C@*6CaLeX&qI-PF)f&HmUzOtQ6fT{mo`WKr?n5c8QnnIE}gJxhX?UCls* zUbzwRZ{86MK~2o5--nx95)+g)(o zI;e=pz6vY*HP(ChUjM@Y|DCh`Z#H|a_aiy}ED6rR{$&X8nav=8e)f>40QUb$X)1c( zXPMCX^`(1jzx84l*n$l}myi3Ye$D#kRg&_J#@5)fYG}NCKH*eVo+a#8&>U-Ch_(GE)Q2)Bk2 z%bqq|fAt2?s$47!00^ViSxZ0Gz;L#mi5mm&eZbe=Q|^~4P3XbSxqWg7CI)&6Bx)_0 z9qE!07L|m@>{*tJV(iA!EgUz@v22=PH)1`s@?8RombuM~?`P0xN~eL^-c;gXMgrIV zB%{`Xa?Lyk;wxTe<;%6$f85Z!@6ab%gcjGu>JSW+lcK5F0K<(@7X$@CQhlBEd*g6V z78ZXpR2;uM=g!EU%s!G36;d(~zAg!^4zlY$*1%5=!g>=S)O$N4E3RTqgc4UP8Ta2? zc{6dLNy+z5$`Qd;SoB}g4GnMM=72Z#-aIohJ`*81bDRDif-|8U-<2Y%rSqOXH;yP= zhv-$~5;A-R1;iFEB&MZ_ZJLC+f`R1NPaV^*+vG3iu2XB?nT8h}9iz+NAv9ExdnTD? zP8rg#yn4V6htfKIcGVLJvSG!cKLvh(z15WTSm5xq3&wfk*F2XY$mbcJ4yVMD)hW z?4Jd7i;Dgs7Lke)Gj+k%uCx7qhT3w859NP(*Y728QV_=vYdJ2CLxY%_{dgs}5Rh=3UG&n~AoFivz&K?KZtXGq>W$Ws7_)MQ9aenY`;^<|`; zsa>C(;k1vU?*uKB3|1Z#E?Pl+YuncNk?{3$OPZWNaM;l(7*dN(ru!f{q-4~o)irt> z;khVyj+CG4NN_&4Anmr@OnI>L?PTS?j0snnP|v=Uo8{#Tll{CU<4GdywJ!7Vt$V2n z7vbknTfB8g*IEbIhnOFl9dHlzbfhLl@sz$cJA)Lc^Wq|i=5rC?1ivjjlVqKuZx?g{ zV&O|`WCt7JOUWJYTM6O(=*FQ5&hhzO#%5O`J@{7+hZoN7S(&_c;kjSYZiwRhXzP=y zP|heTn0 zE81-4R07+Cw*|qW#r_UZ_IVrK+ja zD}ulZuRZFNK7$31rgoeb4IQ8dV1>`O)2=iL%A(BZ(zSa{#D=tX?A;bRQu?+(=66#Q z(lAUQv2W|DlT{bIvhyt!9$!I^n!=1~ zBr+Sl-_zt!-G}Qgykiv$KC=p@Uu+mRYazuA(ba%aWHvh<%I>+UJj|{GXs|U6#A-U@ z%^fMK@fefaR>Dope-t=TI`!BQ^6muyHXgUsla@0% zbj%Dhw0fdqNO)_X<1?h%0#OGCN1wC=$h&;) z_jj1nKvp(`Ek#ERQw|$!=pd`iaM}O4YeM`Z3`*b&5iIkmII^Q;l>3$S#4mGmi?`dw zN1`vT(;P|zjX|)L+ReeIC(TA%h5kD5Nw(O9t)DVndc6!|J{w~q4CCaB zsZAX^=PqM!Av|Z*5Rr4*EJ|f}^Qu3&Q08qrS~)4wxVenVmiS&FJQ=N4W*ts;e3x3? zRV_V(W*KA#XqbJ2Uub#7Sdma>UvkdUU|1K?dNeu)nTR__eWuFr#_|&`MNt=l!BQd; zd6d*pGC%x(WW8f_WL>ug8r!yQ+qToOZ9D0TZQHhuj%_>XbZo1W+`Qj=#{JH@_eYJf zs%q7)J;t8TT650z%x9u%KI1<0Ny&0Q5dU~^1hM$+&(NgA?IEzuXU2R6FR|JRcXS4} zM@@q+GO8Nu2((TR*HHlEurK`t{l_T&^XR|*rPS{T@Y^WR|EJZ^KUT4aH_#&eZ7*|R zX#p?TSgFaJSZOiIoUm<>Iq)X`R_XTkT5i~UPho^e+fSwZw=6Ujg$FuKo`?swZAYIm zx_OC`hcIl*OIDRfxBW|y!-4k*(s}xQ~g1_*e(2Ck;=VaQ3f!_6{ zrMoNFK9k*EguOjpkb5XcgIt8*gCS5TkvPa?MN&p`(5F;9%(X)iRK`r`oE7a+M!EiWJ->lmKWk>6@!iJWr^ak^V%U;(X#^lg4)2gR0|TNZJy3k+J0Dgqg2 zvT&cR7o4}gW1CX03r7dJ#-C7E_dkILulHTx4S++{R=ADA~#g=(x9= zH=yB1?q@g27QbH(s=1R0U~NpBk#iw;1r?|EH;yP#L70=?AiSK-Sa}g7D~|o|)X>fL zqJ4%?_9+rvh?wz#uQT>R!x{n*R-MgFICe4O-WGDx1fwlWW;6Lp3B~dXO6`YCZ9SAv zzkRQ6F_E}=IH6qR-HLlQh?+o^AIyQ=F916?($|N%>AX@e9b1@FRO>0;3l`JA>!KkM zO6j(KM7&FcYkP||Ujs!Ww6}4)a7oRiJXU>7aQU#?VsS}mLfQIrnh%MkaqXCvMUd$- zJ>Wi#aYEfSk%pN&p1k-b>cQb^$UBl`;wqBp%HnbA1^hqtf-cHfV~KJMm;%4W^YlGh z(j61oyz%kM93je`g1BT0&^=TCID;5Ls>m35&eIO15V^q`xDJ?|P1hO( zM%xL>1PkAMVLVvG;mO>FvCi+rhw$C7i^fo|>QZ&p8lbAN*<&#>My}G;*f=Zje$g}A z=3w#p_x;!a2ybu|9&5l63z8XSgg9EplN1o`t!qUjtsub{%{MZDHL9Mfn>0V32)!dG z)6TFjfu+Mtb0Dwpz><;ZBpW($;ff_sHFjcBl))G>(z zk;F|=3MMNeJ)i?3Gs4EsYgc~Q_yYrK`qX@f`CCeg^DM9n8e?gonqn?txhA=Sdj?=U zu`-d$T(@3<-IOhJ?^Guws=%08soEgjX2ZJOdx7t!FOOZRC3lFY!x`0MF($B7meuJZ zL4*_)2fI{OgMHV`UX``sXlAe7bbr;?U==Q-h_@ zr=^f}mi>h0%#=EQqZh+1`7KzeA`7VHltBuhvX1>#uqIJ`8$$x@wL-&tk$Jn!h1emT zjcoa(KpSgc+~~bt(3IK&OrM$y;pUiQZ(va~C0!5xW&^ZMh3%V~J5Vjt!l?p!pLuFZ zU$rzD0fo_x)vDR5za+nWvtbmEVjCYT^lnbDwlk*|*i`gmhXQ1>`*! zcp5A8hg!_NdjpFkGdvwyb7GNB>93i`dEN4pk!Xp8iJhqjG4$>@#PqEqrX*ztZ|_1^ zVYX8GQtFb=>y-F5CW(^9ZR7NlZ@$Dv(Y;- zkWCYI!pzJhac*VZ;K5W@U1qAks#VL~-AkT`zj0H*5cmX(m+MlC(;$yzW(wc+6-#($8^HHVOdgLjzt7ygbXg(nETGz z89dn}z0ki2S)ipn<)JvM2_D{z{A779NawM=yufQTPz8yXI)e2?$9D_J2#e(7Wt8~j zAP9f~4$wL4f=VPjH)2Ob9ax5zK+(8DG z)3B#dp6VPL&t0!$)a5BOvZ8k!c1O(?Ot+6(mbq>|cnsRc($3zjZy&1Kv;4N))L>i3 zNshGVN4NM#`~LB({`VsLf7j<4e5#zdZ})`<3J6H#-z z69Q2v%Mu4I0<2sQ z<1z+Ytl;mG`FiS`$edZT7{MN4jBHYis&4c<>BqS> zD+w$NfJW+|g|dS!gjLpB(&0wD?TW_8M#5?=KD1c*CWnQM3a;gHKQ$Xjcm%b7HiY~Ja zBQ?0hS;l3OBx}AY1SF_wQu(4ptmA_ty;V+mw)u~$em;(i>{Ug`0(;x3*6+v~gpM!L z*n^2}A;vd2xO|brA5rHJZ%*m^O0r6`2edSWfB?L^iI>m-8gsWTHb+<&2nnVcrW-4s zC0$7Bt&?Dwx1vZF-BG*H-@pe)c1q$wr|KLM-51^+q=s ze0}LyL2}O3?$bxB4@84ch32EO!gl_U1hi2wReyJEZw)6IkJ?cA#iocS5q?P3OHpSp z2PTL9-a-H5@)GJ^F~Y`~{}F|Zul0m@$?Dc$=vx~2*WWcA>h{B~N!e-r@JjasNT?O4n8g2PNx^*LOhr-+WrEDSJ%uPvZYL&1PH0zS&L835~KM#90ck>Vn#S|64 zjvUW8;yOf+w#0Jm8~5=P#BhGObc!AyA$YV8;uG+0?KjBg*k65hzuq0$B&OPbN&vcd zefUPdxOa}d?uYuWeZs?!A~6u@i@PSugL*Ie0gfKn_{Bb+UQ$Iq{Aw~`Ud%(AR=MN* zP>gY@^UR%bY0vjJK!}KUMC*jV_l+=K(Fq9l_mv5HCk9vf9^Bp_lK4g@N4;JsVEqCu zc7N}0^6{Mp#9k-Qdz|_Zyb16R`v~65A)ZR|y*HfOof9VUPwNO$mx({9=6*zn<@s0= z0Qe6I4CWZUUTFz_P7gNu-egP6-v179euMu=wdgaE>U*AxdVtjbY{>o;1FW2o?|5ky zY4APJe25qI!Q4?y&DAO`$z@v&4JfM?^}*6iPl2EIYfj0zipQ8P$iZZGlb|?6WmGKH zw)U%$ZM2RU=j0X`5S3T+F5rH54u@u91YqZM@_THtw!2kjSn~1`#AWV_cc~l|_San6 ztTLSZ$|KS!cVKGOQ}R;|We@;cPEJr{Ej0X*XO^lNE8e$GsCa>*)?=LI)G&|1Zro#6 z<pT3ncSprAC?+Jd_ASnRO-WEGXtFPm zbYl;)^CU%=c!tDQbI4@`JEKiGwvLWC7(aoYkltZk-#02(ZDw>qNEKDfih``KlQpAe zVCpG?7DarMGN`3D;*T6zym!8_4`_6UH$9Xmlx2a^_RdbW#eP1_5D#jkFdw-Nu)W(T2=B* z(n%w+G=?4ETqvNl)gGQm1{8=~uvMiQ;zM6g1*0KX_trx_MbvvNN;OUnaEGNlnQ||g zBR8EDzemd6S8gt%6GrcsdM`<^fSRrxA!}~;m`}*GqiP3o7b0m(PeoH~reaqJup&o1 zvZ!mAdjvFhk_kYm)0-RuSCaFJ9{E@Q!TA7H*z>WX&f_=dR(Xyt1Zbv76h_*16leM3 z(Qp|O&WT~P_aEheU#Oe&56X)(VqEZd4p~p~8*AM4sgU820?M>q} z(R2p^HSwdh;QVQPYPpAH6qoEbySR`pMUkLt85x?04w1`Hc(Ok}jYXsoq2L9#60A0v zG6Ltxk%Z3Sv5HV|0z_IqWGYxJa`V^~P6tx_VRALkdP@`|$u(@Pa^pz~R;F;0N63b`j=raW!A#=l5w$Iu}OOZcwZg+p-^~Ovr9nfM1B-2%eOuE-vviZ z!pU}}H^AK!aniCmVPeYvqzfh;k57QAV?)_=+VAOr)3>^LZ!gZPD=i**WY`TLu%`&j zh}AF~Nqi}o!-mNiHx=szue^bRwlgqu*uD~eM58_q1eC9BYjL%^L+jAu>~kk}m0I!& z>Fcen5nC!du7}9}phs}86#=E+dAGnwtJU`Xp?Cz35>2iu5smBTKwSGRq@576_8G+w zFQEh*ycn4@0V#=~ck&fzk&a^MWG|o2^mF^`hJ#?yX4&9^d$4ZWvN(_Mkez-o7ks@} zKcg*V4ludYgffvb#W#~Vy5ibT#EMJaT~f|`hfaoG4eY&1+v>CwJ29L2d&Ve$b#(0- z3G7L0#qyN;HHiRmt3D$-f49;-sLjioiF()N1zJm|JC?gl36h?y(7EzRU*g3mG$klC zAt@_an{gE963T`0nZBhD$EKOTqfg>zL{`e*LO|F!qjq+&xCv=x<}ZAdbLu^=xS7#U zaLva9odGTocu6TR?XWQ?#8h%uVu?DwOi>1FQp6l79kMPm${_FJL}yrmSrMSUe5Rw8 z7xQvC@fP)f>)|y?WY_^awNODKxaJxAyRN?_KWk?Ef@Pe!_*6FdDvOiIL)rhBdxsQE zI0LX}Tl+1;vKuyK{zT&+CbGJ#S7bLdv2mMtbl8tkD@ zo7IGkug?)h`U{{d@J%1J%Yb*P1k@(iU&L0aA<={hhwlUU&bgT36HG_3!^T?DeC5qn zcjXSBRZmAqi}~rEX>VfL3I{s+e=(IrB&d*$}8&se0Haa&a==MT{3p^w=>d2+66b>el{Gth@QV(^hrGP~0ti28fVyvmRX8!`&>}u7vjaxA6=XpkvsoYy7re#= zGDlJKg6t{lwyG*&u8YR8X`r&zX8jl0GWx7C8wp^FZTXe14oZ z`UH65(a4aVjlt=LqarPKgx~RPP1=hAKnu6yJV&h{^+b5>m^W*TTX8K00Vb`RcSHUU z+21muD*WAzN$%(odC!KqNxPh-FRv&35BjoOBC^@{Nb=KZ?j7=(O+uNf(p_NQtP4DJ zRtO1<=Q3Gzk^yvKi7h(gziS6AcKoAJM^C8aC)0dtNZ1G{;c?0NF@W~qhnN<%3__)trncqQ1cbAo2HEW@+k@FDof^LN$;524Uk#rAINeN(MSE(7G{<^YTo4=4eSe zmVDdRtaS<*8l+vV+ub}mDxK2Z;s@++{)^*%nCb@YD83r=%NW(ffMjV7TjqMT6g>cH z&P-BB08Zxkp`PPw+ZDDzv|nR2QCJ0weA7#JHl)WM6u5IqpVm_*qv2NE+K%wgHxl_` z-uc8nvHMe?9AS?F@U5EayWhdEJe&3HM7$WA)y@f?sMQFc`!*Slsp=aOz!S(^--^2v z*VSP+J*%k!hu}OD0QIpO-i;ru89G6R1|O+^Wind9#Z|ou(EntcN7Vk#+fO~=C5qsc zpw$>(zl`C$dmJ|%{COg~2z{U9HuDxrZR8zC>~0 zLZwRcvW#-B9~@^Dn6LVSlWE){MDy~IMGW3216f?Rfe+mhKVU`v_IF|NsaP~s&-FxK zTpZ9^Q7i9(x_>uJeB0j4ADLf=MJKe>h1bj{K7a8~-h-_ZE9rRdO+Nl$P5$Nx@k4#V zj*&FZ%PLK}JD*VHm?8*SFJkh6Ve(1NJwHuK&4neB*S_?PKeD4s*ig8dS$%xqy_tnC z_s{UKW&GV=p#fm#RoaCrik+uMnDAI;RZ%ArEmu>gQb&KV>QsiP6``STpsuM;zDns} zPy+QU4`|aos}uRzv{V7S_Hy;3m#J(bI-ZLM1h>I{izS#ijM_yb7yQ_zy>fOCoN?_& zUj3z1MonafrM4WrH0%CGo<8sz;w?Jln&cuy`|7r9FAq@4mXms&i)p9fM)tgA-Equu z9+TyieBK`WRt^3KVvZ&L$bYwj@6e;MsuXB}n&FhDFUs%Gm@ic$ z{Wom;-^Zs7--7?*#q>b4-|BLd*546dT7w@cHNex(_7dO5Ws~F12LN6f$l|i_EmY6bb@}z5;vKp-&H}m4r>jj6Z1*V_q4xxU4Q7CJCwM3KhjlzoB9qVi z25^Jy#cOPuX2nnHP|RiCAVCDWc0x=Yr&XFx;yr|peV2?(KPHD(JcdG1kdbhiJ#v$$ zf$`AW|0kR@ZUFKR1IVMU$Mw#$`AhnY6a^bKTGhV!V)|j`P4tj)c3HMLGwTb5Zrqe9 zg1ppEC#j)=QXoYmF@(NQH!6`w^I<$GTqCS|A z3;T&)DoaVYDIpAlmI#7H@R5+F9Ady$vO;)#Fo86L9}yOjDM?4DTf1U1)){n4>Sr)5 z{i{&{usuUZqMTtL0+4eN_Ng6Vd`=L->-$sxOuygV;>;%CqQ}Rk{-e34udnCb9dJAh zyuL{}`{bT?-MziO8Dbz9f4OZiCg|aww@V3U3W3^S#gG6=lw2gIg^~yws?t`RA_f;i zE^|nCBuW74v92r}hsBLjpN-8V?o2VNyFWog#0>n1lwDcO`2vD<7AHQi0MJ4w#De>UhR_Yh#?cg{(Xkaw$waiOF6e)Nbaxf3bkiJL{7#7OA2BsGpTG>>QOL{4Xsudc=CGN68( z?^3ArF($ZFs7}cTkai)61KbCsg5Rh*G#FPm9IlB`s@GPO%kBY%eR-3*a@Nt4&o zeRDhpjI!V*XkKO%u~4)s85FD2ggDpjwZT^Ru^KX;0Ic= z={v9>cW)rI7vv5_&VJot`PXMM6yg~0WWM^=9@_RptKYb|Lr~ul%JqG`eXJ_JDc2Ms z)DcBD0ZRbpESY5Pn-GYcUC-Fue&~EupZY7RLi}+F8E^izu2xvR< zUMf%(b>KkT;g8nbZsY9o9D3YGJLE?1T}{)uq(r9w6*bMj;EP%$VV3?9830ZVplM4s z33e$Y9_+u|2sWocE~r4!8>{|1UCZDZ^yHxIRzef{Cj<|CB_=O(tA_%+07Hzs*d%vm zqW?-mKGwxi2@nd=%kTxt&ET|-5K!FfAAQ}){cTBctOPT-GXg!5LehP7$@)qZqF~`& zTahpagZIhSme_-_I66)`j7~O+jxlWg+w{eO?av%AT360e)NQlP-n8!Gd5n!|z2L^D zejrWeVKmI|Nd=~GU!N|oxP}LnJ0oyCxxMCo0rxBGfWN0A!-UqD``uIOcCBOWl)_Yk zx@u#Xf|^?!4>j_;8xL=`8)nGo4W}@#|9M*@Uw0f7#HfThrafKg#Ql8M1*{1VR`~?G z2Ug*xh&PNxP?keiDbZS~$I5B`(rN!z4FaRluD~ey<41&rN)-NM z?I8i}(OofJI!MUmC^LBh>5UqwPb}oQJ%FP>7VM%f^Rm_gDB3?asrargO9T(nm;v+Yoy(*diib7eUK!YJtRctdsnkA~WM_?aIx0D#4yz zg6z^Yo-SO1>HCtiO3FjU0S+kg9Wm>ipV<{2qKZ=Mg(gfQS;Q*@6^n$fNKh5`y8);a z&dDMe0h6w%=`QoXRy;qh48i{toQmSh8%|VMP=ZaULNk5s(qTdXGeLPx=WpT0RHiA>hZuu=ad~mZ1 zoQo-Oa*Rl-xdr@rD%yX5YO-e0x(65}MM|pdZYXfUbVstTN4}F|g^pPKh>4d!<&w;~ z6ZQrPpvG1nCJshCBw2GlzoT+w^9tA>AIWkdooDd{-KAd9sF%h^eqj?biFBBd!box4 zmx?1!stWaLUuX$${xA=_Xd+HB1gYl#bs7ugq2>ChEZi(gu5R$0?%&6Equw+WEj;44_4Iiyoy)s8>xjj!A%t z3+8gR^w*;|P7ndY2}Da4r<9FIx9VhSQyS5Hc=QtMz~N0n$ivGj2fs0Ya={AV9x+UK z^t6a7o5&?Xm((0`K@Rqx6$UV>Sw$96j?NF>%;SlphDE>@Kg-AkqM6B}R8j{!w&AkX z7D!IubkD(f7NO6wVv$uw6G_(M{Pws-H;9O;Hrya$62N}G=o}9I2pWQI+E;) zP7n!Kj29qj_se<3J{sbo=9G;ir+7I!Rgso_)S(^UNYQVQcLui zw~DvaoQb!F?Ud`5lH?cdGspBh>MSIA|84aTfIqN-+dbryV`$W|n7Bggmnb*wzuso< z%wRyjn2p+o0KvTVw*p)*-P;2gN5B;6TGSh-<8eZfXf}+kLgxJzEQ!kzSk~6U*}ywa z#%d+Qs~{%#J2%b4JIhWa@#Qpf1gtzRL3o|(V5@Lbb!0xIA8N9#A7MNq@N6S4T^bvz z7=?81-n)EF9|zI78STyK{(2ew;@9bz^hLe#gZ}mF!X+Z@RSdB4>ec+dXz9|`=1gFc zkdN8^Ls01O>iLn+zgk&Kn3qWYSAt8Q zy%&2&D&pmjIbAu#+4ro)hBmZ4o`wj6uUgKyE1QiNLzZ@6>RMBo;<8mgVOfT{ud8aB z{!U_B2VaK} zYJXB?xrxYh5%p6E8bwJwP4^t9DxJ3zFZBcfd(}k{#LxS9O zAocEPCBFv%eZzOU1idcE84qL|YnFnEjQxmmhJZv(IX-jrm;Sq+=(l*nnowGYnP#}A z=prwfZC}ari)_-Xd{7bvKwiw^vr0|XgVzAGR7e;S z77YCX^IJDr46a~dewh|5M+%m_c|p_^D4KUvlIv7r?K=i&*tU#NBczt z&TD}WXCug;*Smf8pU-t!d+FB#+gogcSNuI5Ws(}?1hd_!T3X660s!rn zRhSt)%>MU5ot_D04swm3!AkqO_&teE;YIhQ*OH;@sR~L^QUdVTmi=RZHo<5HdERN| zTpj+c3^y4kQm3QHw?A5ahTi`o7dHup%nVGs$3;1(Ouh0;2L zcNq`rsudRPRfC0g+>4~HS0t?5-hh$iaPp@}R=U44_n2hJa%Kir7jhSyMHGX;R0m6p z2a}hn%WpaPl@_)hWdK9Y&i3M5yqE4r-2`p6IoHi&@0O{>wSsetca4e3FCuy8Q@pT9d}NGvY4BDF)_2Z6d31CpWlH>* za$STh?SaX7W&s_jolo?jEr8h+(B2v7I_nL3Or1L8MYR#qwGbcMPBlu0Ptcjej+7Gw zimQXW#7)>O?)tmlQ!l05sS3Z9$o!RgpeDUuFM@6k4{(rf0v-OJgqfTtA58eo%}KBw zNLf7Qm%H$k$LKe2Ie75`rPRv5`W^R`_m{<`dByiiH5Id6&EKJQGXaX>_gR0_?hWpp zXaz5YrviL*_nOAQ1iLnVvJh%>`@92a`XyPazS0H{#WZr1YG{F$Vt=g=KumD6b0vJF z%ybnlvU$|^v|8%mX9HA}1$FkTX;GR%7O|W=oB>J}@2Iz3j-z#*yB4+g`?09qBmcGp z{BIuMo9g=C17T&6Z&Kdw7j8NT*gwOjDY6Qr|BTu^gGr4+5R|?@OesI-`}NJqvqMQ^ z-lW4!Lu%#5Pctm#f@s@7!rK6c|7VUiJdIh#_o4Eiw4G8&*0jDn4urH+a!x8hi<*`L zDl^jOR`K?czsTgebMgZ1X#+o@3^Gb&WN@-d1vNOZh%<-1ai<$e9qK|}$PhWf3-Dj; zJ@6f*CIv(=J1r7oo7IEcDb53jk?o(fdXm?{vMk z0rhdrjknt6U>dN`NE%u4A6|CcH16g}#JN?eV4{k-zJyC2ks=N^g^HOwzRL%n6ss({ z*VG54B%q-MMSC0}sXATUSMTCicC5wMwB)Wc$HGKo$FK9_by4&MA$MYcT5S!98$4)d zS?{gzAN;y`RgF4K+LeuFy;UiUgCe{Er`_zraLQOD9%L#6YJ&QlDfO)Y)1i1>EAK*c zQ+03z3QN86l@`jY@%kG;29;YTg7Tu2;X=O}&Z6U@%fMfxlC==M3Lo%@h6>#qk4%*ziY2s*t; zuhL$L5uY#75>3UacQgZ=Gp{OCCLUtcQx((iw>p6#B0uXcMO#w=8}`JZuN;}6tPk-X z^XCd>aPJw#2yZA-i4@`sqC=KI{Ob`OhU?>NKCa!u7b)y=i(k##pK?qFQlF8k%{aCl zb8mEpCHfD@hEyTK14ze?xkpSxq(fFf6&E%`6lD}UBtN0@nn!jmKkW;e=odjx&=~d# zLlpK=7sC9|C6?`FRK4H^bDpX@vA#m}vr?kZ^5138ar;2><+& z#aO-~?;8$G5NSq1keF$6K^)L&dF5QVX$E;55NRp#`kENx31jrN;xH3N|b&;Z3u zV;V!i{Rcht-@uXmrh+4VhJtZG(<%xF!6++p>^A@tLiJ-Jqv;k*ghO&`4ziF{P13b= z2~$r3_GkAB4?+1oANCwnnmthU3o%pNW9(eZtnG8P8BHw2zS5<-Gba0tPE#`TC*Sv}%--oe^ z>{d+$^lEN_pk2{m^OAXAd3?>on(1`m*-oIfy&Uyd!+MMYI_WXI#z&LU;6lj1kIa!-EcJ&;$#uYSH2G(Nhtj-T=d zNGi7ean<(V>|#8Pn}k0RogsQQTG+HoNScLAkHASIrZ8~2h#X}=+w?i7}ulae4uOC-%R zTFOYg?-IQcv|KB<^#YfWue1cbNRp9+S0{;+J!}$aHbJRjh_hCSQbrZT4U2v$Vh>RK zvHJq$3P}3Hxt1eBNnFkq#851C%_ePPxI0%7Ukza~IMwMy7oCO7$@6;l0{st`{|l%8 z-zJAsi4X(xpZ!b$b(e+xoBU*o{-6CUNl6g}p!Tf>?4sgkZ*S)OFSD^m<5Wj&75gg( zc*ZQ0Hac)G6bH}AFd2IbJ%$d!7;g#4+Mn!*Mc2?YyWqP>gt=6DafvJ@rZi?@WX+sd zK1)m&$pq(HXgrhSGE6{G;7QT4@9A{$!S5R1T$oY_*YhabW!5ikw=360@MHF!@LLTy zs&8#i7!mO&Eqo8hlE|@o9z6TWv5}Cl|1l{a9E(-Fh9oSKh-QEM_d#SRI74Oa-I?OL3}?M@OO-|%3YmuDusUzo=1%~@AB z&o^=D$8S%C{(Fwb-6uBOkW40&&P9 zyIUoxIH>Y2FMAm4`EmmEF~nem;8@9KV&HjlrHj1}DLNzsaSw<>ZAGPP^!SDVp{0G~ z_EAizE>Ukf_W}O=5bq%GA^aGK?c$YC)FHv0vj77?V7z z7793eKA$f)sTmbLZL!~o&cIQ34>DUzq9{)VZ<*3oT5+w1TUAN1y@y2Ava>-PCH>jM z{cBt^8AWptM6Eerd04->axI#kruEJml%w(m`DgWSblqaPjwl-Ry5EJ7vM&*l@AZXnWz%|r$2i?FAXzUF|zSqs`gVadG8(DP1LWN;F-{RjUe`SU#Us@s| zTHiRU$44Q)F#1$)ef|m$msu8R=jhSlwb)9gMGF3AMTeZ@@_`1}Xbp%apEt?L1eh&a zs@zbb1I!UurVx_+`48{>_@n)KbgjQj3eE32er0NLLd@D`l z$PGV@xfMuMiC0=IwHa#v0Nh4p{0UV}G-B&?>CNS@myGM9Y66`B$hYQqd8-WA3}D0GqAG^g9{0#9}<=0sWu3^nm6<964R=&wDd9EIac z%_YD!pGYQ{iqFPp!MwC!;9#_)_h%{lpZ36 zt(LmOK@kT)X&NtiijnOTvZ^B}Sy{~Dd!`)J`Aj*L*u^1ulRkLtLP2bIwN0A;FQq8uq|tdH;{#`W`PW0!F*oO zr|u!exDetutkz5k`8-WcK=O@w(RB2CWdEc0OC=NF{K5%*n(5CHS-Eny%t<8Cz;~a* ze(40#%A8mhKN2!V=@ng$@UCZDv?RD_DFX)~vZUez-h^e2rKZzIZai0{Lz( zI0xeBjqJ$V(@HTvV}-0kwZHZ+x!7W2^B$WsJS@#3kb0MWz9HTj)lmyY>yOs4iW9r7OalcckjVrk6d6RREA*Sz?*P=l>ET-u4ys zF`$<(Bb6WIiJ3XB6*r6XOu%5xiJ0WOh{;cblr^V;;u2CQoggfzeGS672NarMkHyia zF~z873+2>sX3e<_^7n-7%b2eGWjRy;P+2$Owj=qr!cuQYX?9%Hh`*yR2DH4Oj{ksi zG4eVIyB{R_VqH%V_!9ZtCM?}X@CEiw2>+FbxDAQk%e&+1O^%D?zDus3939g2Qc0c| z70>Whp-WYaD7nK{rp`lEziTc}orP)h-I2<8G#%!aD#Fwp+8Rf9YXTS&ZV-t9+-M4d zDd`Jn4C`JvTWKxxNC}Ip6UoWRQ{KIZzraHTZru-&6(t)3cM9x{GpdPXa-pl)SnQo> zGHUbhrl)MPriOGhm!$(7cf?Y_lZeYXbQC}XvRrRftg+)62Pg~*X}qxE@6<1n>weKP zs-)=3{D|O`O^L^|Xan9JFmQtgoaCXhuUbFj4?6*qR}?y8Z~MAX*lSQ;*C_xQZ1I$Y z7Sk^CYoNqSRF9wB)fI^K#yY=3xXrYcR~d*F`69~SvH0H6bpG*d%;HcM9FFkPW@{mjqj7ldEd)6R25NpWnM7!mxw?Hz~sP=z@L5` z*v{LwZd)>LS53e{ba>#%!o7lvQw}>*g&pPtCUZsQ>{46!qL45Rffl8$=)uot+`ST> z*JVb)o{dvol(Y+e@1SnHqnOay_mf!RHi-Z0KjkL>^|$`NPFZcNRoKOM^z#J_msYU; z9r+MZaiXQaj{tlx&z|$JaX)(V&HN-UIJ5nq_imIOCcbzPM6aBVX zSBQ4sEoeN9opLKt1R|2Yj5^6rGIp{z4m#3mx-L_XR5U;WA`W?fKFaO<5TRLjlPyqC z@uS(%PKL{D&+}G>L0=y}Q2Ib~gnqJ*RfpZcPsd!B{@n&Jxd0mXF zC9a#~yOgKoSaYySeoIq@i%Rl2s?UCmLR0SC(AyXI3&475PT=iebKN9w%YP$asRZ)Ih8Ny_ z@W>xJVa9#Q8L5hq-jrC%qIfm)lAS#i$-nJaYpN9{rG%8417*T{E&j1;&T6@xowN2C zyNE9BqjEtf_1W|vafKAFIhlS=Gc}aAEpvT>rdX(#%9EWS}8(pB>3fzr& zJm5l7jzs~Ij;Ra%Pe*b|iLS4WO5tEcCuf!ztgk`C98Dq(iggMc!OF9x$ z%N7w+M(^`!ttLwS{ZF~;f35U?+xI_Nu{I1!I)Ba0$?)+c*lq|1=*D zN!qpl7W^3zEGt=8NlWg3LEO|7Dj`ydZwDTdZ-y=&pa}hL=df}%?o!H?(Y%@tb{)_R zJp_*UOO!Y^Nd_I0D&qHn%iXKTOvA-4;O8gE4M<$76ST*cBc~ALGsxC}AEAxFik~f$ zt733$?EQt5)_6Vh4=r0$fO7W8sT>xmTlSKBTWYuY2^yEBVUX>F>7!qP*j!YL;9-B!1GxQLuc`QDlBYzP>aH z8Y?e`2i%D#t9i7idFUBVaH(VB9@+~!s+}I>Y2i$@7>J`AERZs?1YgTukL6v<(xHlx zRL!OuYC6|F`=xH1CZ5qHiWJV4vLeAKNm{+Qf9BsGwtC%V4&1oJjV}}@5KHjUBi9uC z7h~P=Kh6{v3$pO>h3WUrl)T%1i|P#9lI|W0Gyw)v#T)S``+%8vnnoTL1fi|F;$VlME7d z+Acxqt{jO`q5Rtxv1)oVgQ;P#VL!~NRal}XKC63`kAS|b=&DbhBYYpssEfC13Z zg$yX1T&WaO&C|R{RwGCfz6QtxE^o%rfGrcIq=q^iJy`kB5>a%xVDBRR5+f=Th)=ai zeWK*V$o|v!1pEoISIanYE*Mi=TJ*I)Zt;w`08q;p$)6{bao7l%_dH*$`zy$z4VW?E z$|cufcj9mER0jb9CgqQ$ieQo?fTu_AMy(|<>@Ds=PTh%}@WuF}(pYVQ=!LZ;MPGRB zj5jye%;qLSZ{gyZ&q+{E*ZFhwg+qT&qCmq}B*mlS1IDh)v z``>$?b9u(7i#2M!@2a|(^I4P6+;5W`nhh@s<%E(LZhY^_faRfkLTlAzQ6R4DN46V;A8oTH?jCw%B=$$o)%3?k(y zv*$1WaAW*eZ2ni={(??v&~@maPf^o`O(V%+`6r}E22gUb_Aqucmo@j6H+QpgH2Zgu zL*rlVn~a`Gq2>@VxY6Ze37KLf>ek*3&7eBH;fv)db1f8Y?JG)8B+WS$7k9E{ zO?h@z`K$~?IVX;%jRJot!G9fx&ySvOZn#c+Pu*tw{+@ilpWYJ!$qsNN23}*H_i6*Y z`IaL8I2#Oy#7cu1M|153;j{5F;->eNSm>o*5{;~3fRUIXz;>)g+O{(03I1~K4+ylj zev%rF4x}D(_bc2}CCFoGgl;kF*ez5y;ZFSC*+>^;M5Euh{*&z0t?j&+DJ3L}fzNV@ zK$6gdg_wkkk-^p?a@s5f5H*IQ2*FN`9p+l{MAH;F4F&TO&u6g64iI5@zw#%s0v)5ol`%${H`qbEA=>ZQ1_6H`cTr84*sQlQ-$XYB+@t|*suk}UK z5%bX%9>lWD4mw&Bb`Auk47b>140q|7VP~u4b&k(W7M_mE?p7o1$W5=_3A1ZaBZ+FG z_VaZ(iKN!V_d%etxi#*#VEJJH7@Ur9ZPuImomMBAw{KermzQ-sAc*$j{1r;uBEf8# z&eC!8v?9~IA)}od7>iJ$ts~UJtn)qR=M|2J5-k;xXD6|sWye!123G=@`%FX)Qz)YG z=M=lsk;m+#VVdn9mPuK4U!y4P){{XG<;mlJ?WA+ZLZDsl3IKBuQu{;T0Mi3BGmLRn zPd`S~t>jbZR_@g+`OY4Lj9)*j+)(R<59iM16N*YstH|i2YY&Qp@s;bbLaQHcvH3|n z-KOaXF!uG4e~4D2PAWM$lHQ(Y#*pW+Eu<#2KNtzfQ`F}mM5=e&niZ***fAyJ7bt|% z*aq^}TDoGIJI061d@_~419tZ(12s8V&Sluamp7yk)sL1n6r@fAX(#6G4Fn(;TRhYV z=Z;{i^%@TcO9kpJ*+Q$tw_a3oF(bv7qKs|JsSJy1t|;FU2EXc}$%l_H$O2MO+H9@i zXBR@bCkczotZ%m2o69tMR3wS=Y24329_!3(`baQJL0&o8-!XI^0ZUeUTlrb%*EZOQ zd(ip0X1JxfW`&f`vl>Mma-UpSsE9z7KeNB?!)KTmUceAHhG-U$$IE?Ig2%VIX=3O3L*_!KHN$NemK zb9O}9S-tN=0kP~002r|-1^M>a`C+k?bJi6%TwfJ^x@AFp(kXt-RigRN!6(45@O3=I zVUh1wtR91Xw&;~rE|-3h+i^2bis9FJv!j>0$RFY;{PKnbId|^PsBd>-G^c95Hfjg+ zx?Z|4C%s=FF*3j+ze&wrezIH7g}T98%MI6~i|k41Pb#!40CsXRGdsX3RNXZftYl&q zU!Xg@)n%zm2Srq5%PT2laB>%2;u>Fpi0^zSZE@Uy{KmU^z3NhNPIXUx+0PFb4Iw#w zj*lqqOma}U+6;(1trdMwJe8|_4|oh}6~jGvO;xx`gIje8CvKtqVx&-Mu1@Iahtm3l z4fvFjM|rj&{zNrZIO=}6%{9wU;UeK?fyZ>%u75H1{}(U+ZzNT-L;jlf>7ERvC0N2@ z|Mflq+~ohZ+)~s(G(N|dyi>y1LC;f))kIpQC8i?7lI_GTG_r@&0($vIU}M9BTwG z&M{#YFKG>4XTllWqC91U2+UNUOV>OFoKkQ09>CXbZzz8J_W6{8z$u=pE3!mA>z zaTG%b`G0mK+Dm!?FZm?KJ%8V&hT3v~wi?UwwT?3D^M}6OOi+qe{%(d4Z8nw#e8=@@ z328BQ;3*#E(i?ZObIamk(*mjrB!b}uMRgFws-Xl(ZxTy;vF9M3N^*Y;y%@yamq49@nolWrr$%0 zh19^tBobI$^UzIm$oK6-eaM{w6s>QiDc`v3p2iV+zGZe@pvJi1%OQD;OqHzb240Nu z#HzD%R6F}cx+5^eBKVaWC+#r`9zBBo;qDsH#5df-i+y=am2r<;XWswCf-sy+P}FuG zP^dfKSTD+vVQUaTXr%TKde}L#j!GU$mJNWsK|tC@hVfJ8Ojv z#ZgnPJbVJ7gfA*JhuLCMtJ9#Z44xo**h{y~U2F2+b@w!~T(p4nSn3XpFt{J=qy}m3 z-j+e(IO*&H)6;-TXL?YE0b66S!+}=JjNNI?L2Qcpj!uW$7@Jrup2qJr@=1$jBI2Hw zO$52`cH)*ALU_q0cjxL0n2JK4Cj6t)pOQmZr^McZV?fI|&Hmmdk9dH^OWm)eYZn^g z+qoRJ0L%ZQh9md#;$x=#_qf#cn=(b|`_b27al**o%%2hJb3ryi9LBCMY7Xkv8Mn!`F}(0EL&>_KnX0IO2slagL?#(@AK6De<_nmh1w7!@e& zXNZ2GokfsFzxFbG9x-6xVFXw-VNe7pf4ylQkJN|Ez!h#O4X9X;io zIl6S}JO=yxXVuE3_+Xs2!fYP)TLfhrR-vi>07W8=k5bvS4tRR8_aztRIFFCDki@&= z-(1jtv={1s_4=Q@uI?COC&)jE+9nyE!}m{PX&Eui=l~fjZQBbU4xnb|>cPUM#lg%Y z?&W6gU}kQn=Hl+^CU5LuY-#R7F6M4;@2#pTC+=Wr?eK3>q8bfZReW{y_uu?h*pP_< z$Rg2k;TFU&Nhr*eV(@V!I5>oJU@-Y6(@t!S);^`rfl9UOtu8xbp?F-i^y92uDw&at zdRpcwaGH#IZ2+f(K7f(;v%);QydorherA{dv)`8IRNK+|VcK7U>Ym$emPXRG5KKAQ z^azq@Z5US;F@n9S^fY4G5e^d)3C1XS>|O>KS=3Y<9jfz~7<=uh()+?nSD=@~QX5Mx zP3fpS5ExYIvSDJhzwRWzpl%yNlBo4tzj)2dI)-={@FOJqJJ-$n2#t3Pz5R-c@GK^V zsC%xy>41)AgfnebY9en-^~HXBVV(85RAy;vHiN4^XH&L?Ln)*Kl74&Q=8x8{p(Pif zz0Q1OEqdE*1J|lC@PV|&0X3$rcKv{rOTMX?jX(Ezva7_7RyQ_2pDOw#^=m>4LdgVk zk`{^qKstvMcdhv`-Tw%Y3&@u7)%A-xMYOlWYNq(lN$=?_)C+ivGL=dM%dGrXN_dG}0Tc*yTNedrvvhX4l88F7mEKf^gn+lpLpSO*@|5*z6n%IyeoY zavakx`e7u=HY;x@Rk}Ld5gc0`d)lB%@-YrLgW9`I;jggFC%JfS8F9C);lj1I%_E6f zhqrc;q}1aMXSK>wiusL+#EZ)q){(_!sYZf=%D5M7~MNWHt$bFfyQ znzXcV7=f2(JGmRMaZt-Oks|c*s(t@}w`w7(0LtD8JCrzb6MRuvnybvJ_28FC1t99k zlZthJt-ci0+B$JWZkJRfahu!Hc%BLtM=?gQGC@^HtZCSdVax^{7&9)h_WW!D#3#lt z@|_0@Nekz?36)&g{%JHBL~G#eQgNIW$+mxhOg}Fm^JfSuoN+_4KThzK4^&9&`dY+# z0L@FqiF22Rp8?c00_obO~J7ft0LdNb(p|KXJqHG zYm}QXdW>3nH}9poI@P}V5kCp-_gt)Lw~bqntvXOvEG>p_C~x1cM=S+nkw6ENK(D2) zk-!FUnhgC%<3K4??;I^5RxU}&3`K61PWNc-2Ko{4p9r;TT_tz7`Dy>Cx>V4lvu__l3IKWt;}hSzWH)@?U7?z$yL zwmQ2ZN`;gSKmPDOpS%^JJ!nULpm6gh-4A|LlYQI!vI~HS+^S}J_awNU02%mzK}Ooa zxb}n{-tjx?ySwO!F*Q+t!s)g3dU~$-_^d30g8nb2{x9(U6AweCV1WYx|7R&PgBbNM z4Blk|=b`(Ifb4>&F&ENfB?Tn$r8!n{LAQ8#;ERKz{6%bt{Gf9B&-$F3Pim5|Bxg|U zzma07wA^Yg;k3>=Zd$;ghOU#UIQBcIbc4N%%w-A00BA}nmq;5718GTPKqG`T9J(04 zeDg(MlI!lMv^BV=^G>;(MkGE_iv;ol~+S66uE7FmIQiAN$idcz0%hHk2)z%GPXFdWuDbHLtb?351w z9&Q6Ct9p;fF&4k{O^QGz*jaZzxN&A6e!c_&yON*&)t%HvxtYRN)k|5uTuFw5*#_+i{6jcIlST&GNU69BU?8RJwDM7dBhqf-}l4aBxr} zO*ttzoC`AyT@&}e5-1zH4c?tut^JC(*CEqajuBq5!0rRs%r0oj#dnF1c<3C)V=7dz z&`awqTTATmCJ=haQH~i8tZEzDvvg(KFMU(W<%&4uw$oieF^$~ z!I%AGu-@q*pT$h_WK&X(ZlZ^P6XIGso5pP*SuPZl4Rs-ufvh2}z|CevqA3W0_G-P1DpQnZo~OZ8Sjy9!^D`mSO( zpZKOed?i(SR45#tJmc-x>lEk?wyX|+D>ovtb@Y3sb<}b*H$im={K!X>@iPY=$INAAh z-XiBfE6+^6E@agNdWm6s$mA68XAEE@cumwEqhSE8e?>|CB1|yK)3}0HU{%{Nibv;< zIzk8V$6;|B>~oiFQ$*8x;atEMefWN719!Y4Orqj8Fg|WA>A^vKEfZs=Pp`SEz_wU? z+AmD1nkrN^wT4*~u*#4`t$dodjW9lt=fW%NX-qDg`|AE82d>uZrxfEJX}m@wbDBrk zpMkD;T9(G{K1AclDey=#U#~tHj)Lif0mcTHmdXk|$tfNV&n#u^o9Bb=U0GjUT%X^Z zRAMXoc%7%|JBS^Exm$&QXCoeSW2qg~F_%~95W~&*4I<2#YSC^&8EhacWrA%3#x)<< zsUTWnq0LEQ#6WqUrgTGA#68a|Dn1s~&?Jj&*#{kg$>y*u&?wf@7{XYNqr8Wh@YN5% zweD77#N$soRfZ%#X1!u%ttZ8XvFloe%(`A zUv`BQkKzBWI+Y}g=IFENCBt8)tZ@rqJj@mrHf_#!=AoPJB5xW_pwd4y`1;M!xEC9i zDTnO5=8nSmOWR!xUO!8pKYY>DaWCrgr5MT^npU%Uy+kf`WdjtLE{37 zdH((cT&pmiIhj>=o;liT%7H=dP>$9lnbW$d$&Y?4&-@D5_ z+`l0y>rwokS*7!x_dav+!Bt5`-k^f}a>`^?jf@Db@f`ML?HDp+nYK%xNl9U_#0auX5N%022Ie)b=g> zpCt-0W(C2>qtu+_39@$IA{gRKoDf{C8|HVMp-%B}*WDJVZKNxytZ`ygU`;l+{@z#r zd%66dmC+bqS_6|my#qLew4Uw%C-30@4;R#=`YYl|A`j&?Ny88Q6cx>yN&MPN2_kBp zNLef$Z5q`36CDq>m_c9)ZgbzyawCV~*wUt?Sp}m4a0hlLs-7cYi0`_IUL`Ve1Xx>n z6JX=xYdE|FyuZTrU_E|^(;pE-8I-{?Fi{u$ar2#$=nQ3__d80j1-LGX9h;t~Y+pup zoKaZ8AMqwV!RaGcoLjZ@Y?4$Cd51EDN$1x7`dl=Px&2z-TzYGZ*=>(G=jeTn&@As^ zYHS62W9N_e*P@Ax@kvr9%=5}U2Y^+n$JoUu3!3cyQv`G9OR_QBx6t+o0bp#Sdhzj# zi`jB@YFwNCOf13=_(OVqwg-0Ik?)S@@i<1yt?k5=SXRDE3T;^oiGKXoHW|hNK=0q& zeXR4TCpTwG%0?sJ-w^Ta?6>S0p7zGMj2zm{v4*Z2 z_by}j8C|J^?8f@pS2!-B*G~?m=nkl~rE7=gRYU%EVxh|P0+-2LO+Tu~QDnZ;vS$?7 zD?m)xYi2{|xa@jq|Iy^rs{^bCA?lSujFzeoNU&M*){SKIq3q)T&6~~d_0T(JPf7LX zLPn7RM~C#=l9;pNhHqUXoP#w{krvGBcd~l|Yw3EGSV>rNXtQ+VVrnw$Q<|atuRC5M zDaj5e6U}O3)dK>x0)J-Q%Z1KZ>{&iKn7%K6|G!-FYzk1>4R zl|KB|9u$zp{FA*(UlZF+DnW$NP&Zw-81Ed`H+zaft2a3f>E0Dw5`KlwRq=jdS2}mS zR(p?A9g~UkwB=h!6<~jc#@ex!2&up3qt(A$;WxWUjS6N`P%YWouV}MUJ?ClF=9(N; ziw-utwb0>tgdu}_8;qs_M7x`ZN4-Tlas=16D5f6$i1@G4j&S#fu#M+Vv zl^01?ZIGA>kF4Lst4`3R(F`EP0(KZ0}l zKjcmWwc_z@NJSBDFo$r>$_1D1Wdrcl6Z4vdV`)aGsq@@#YPm`knYGd6iERT~oT*wT zuckpZJ=gh=JO5N}v$)DDY;|+RV}u^OawG8Ab#(>)|%}iaz*pFr`Z< zf-qB}S%a9Y_ZU$0kbaz{iG32<`KKUZy*C$2~DqjS%>+^yPn{?thK;zqYsA9eWPxr+NPR351egzNDE8V4|k=0GWx?>gagj z0IF_|F8>aKlq6j^pbDb{x3sCF=rD`5nhKj`Qjji05yzc7t?L52?uu}gU-Ww2RJ@gL=|u?YeEUqFoTYQgZ9?DRa1j0GYP zprqHGoX{ZfTI|*>q%MhI7GU3~04B?kpbXe5aVeq6T) znb*H?;t(m!ZP?AXX;Dy~;;U+z8u5T#d>B?Y@grq=*SRyk>ts#+eJuCx0AMs1+C2BZ z3X9zoMRxBuCSVt|G_36!HS5m|*Ks)Lx&`^1>NxfJUvdqT^$p13z19%^7ZC#1m2J){ zkEtsPu$BphK6&lyp_#%d$BER0qk+stKVoEHJ)M?@F#CU3xH4D2`-3+`_`<1;ukStR zmDx-kDSv;u+^-?i7UI0u13-bTVECN=X*+GJk&<1&E4rbIC?xV%o1-EX zt+vuweD}>NpX*TU^TT{|TQ%aAEaFT>E)vvyM?6#SVik!t-y@^<7vCM{_Q_Z^N9!LO zsaMo8&P$-&dNw0q@3~HBPPH#n#RZ zc@`#sVNGNa{I&NdwezApl}h>{ap9(T6|MzI4FAmC(fmpeM|I z6_NF7hMg-VVUUtCLG9TV)$U8Ax8rnf=w&UMdb|@63Sf(6^4}YWWiwjCIit&Qeue)A zDHO2h^;@3SxCaOO@b&lGmhUvrpY{Hm7a_%Okj$}26k_J<*8#>W!PvX&;pf;A?zbCoV{#3 zR!VLToTTh5)>7^U)>57p@?%0hN#OA6VW2DO4Jte&O|BHvkSPtYpj;VVr+x1n>Nc~< z{M3V4cYUs*_vc=65S&@6JC)&jrg8icSp~q!<=POA_Gh%JCRB}teAJ@!O1X`ljySsT8QxTOm8!Wmyhd2 z?5Ma&uFVHw3^NClD^dPVPH12+GdY8oEz`8DQdMnm=}k7p({MN}X*1#d z=@6T`QNkA~`b)Ht(^d<}HoT2{QNC0$Oauj&yK4jhz6x%>7;b%ZIe zI&Ez+GNlgNe(>s<8gv$#*swKxmcOmOo_G}qisy#6HE2_71lhnJ<_in_+wtKb(F)_j zHa3aPwYM(k#5OvK&NbiK>-1EdgxTnlOIy>k^H~rbXM7bTUb32_$KBOd1h720sLfuI zuPSTSP=60^RyI1uzR6ocU{G)_QJcG{*C^FYjCKiGl4tMXZ5pZQhEQ?kh4`}oRcFDz zCq$>r#yxnX-715@>wT7;gmojH@-RP)aCdOBJ1tzq_)Z!`#sO)IU*lXobd5`8lU?mx zIb_}Iut|A0=pQqL;6=dL3h;)Dzk#ILn*SN-#dn3LlNV(E3uxM!3lTfx6;euJa@*o3 zJXU_gvObNt4Z|-X3-&D#mEavizCR9BsNf!@&B-{nI&K{dX*S%yafi(7nx%FZ4SrWu z=88?`z*S~fcD+=8)5dCDIGj5IbG;-Pr;q)%=hMvKQiHi|j9P6<1zdqApk#p#TfHc+ zTPxTd)pixu8^SHgxTF0x4Wu@`ZUcKDppX%=yZVxbI9qE_dWTd{$pr0Dt_j|G8Wogd zlV;TNoM;s1qh7D+uU)U`FJ5ohNy&YMzQi_sMbPKk16UwaBEDw<>S>|Lgemc!CL0vo zEz(?wSl6kxhtBDj0o-6q8WeDl$G+ywF(tqA4$`4J=FCwgckz};QVXburlm)qIu<6K zbFiz9H!HLGRP_2(B>3Igkf|M7KMM(lv19n;NZ@IepyAD&oHfQU#rJcK>_K6B1rfXk z>c2`8^>iy4A=GO@MNWjFiu|eUj#uC54O(eC7#-kp7nN z5H#3ay@7$her~kXz>zzFgZurRe88_qs{rz^B?&S)9n`;;N?7Cpp#Ln>qO55T5&pgf zm=*LC{=9Id(t2f}3DdqZbKwJORJRo8l|J(eoEpcpa>&Ri7-Abk&6Tra4WR=FF%hDK z>FLL+OB?q3GP0|>4R6{%AJ2Lp&xy?WTdL^cZHT`k{mJ(AnfjCSvf&DNdw&P(G38e& zFw!4{{B2Pm;S2k_N@_1KC_9K>wB3d$b=gAd@CFLLqJ8ZU-W6ed$1xbtkNGE4V!U+3 zub%_qrRR#iEiQ;=w!sgw)3~j@KNR~^px=6JeO^9@<$h7g$*^wrx$ic52QkgKPNe!` z#@y;~sEHDTZ+-EM?8OIM-IJgfs*rWc79s~xS!!8lrh@%>h)NVG#9WoAt9-)!?OHl@RaI-QF!1MLeoCm< z8`e}ZN?&Pdk0k+(Blh{w&vng;PrNY5K85e|*1V#0p!R6H#M9q&BcsZUv^j?nNqE@k zVSA#Jh3(+xQ8pi~Gsb0uZDvPTR#95a-y`XFfv*PgnBmBz;pzi`pZ))X^~@S)`-M0f zrtLnGxU6!XZSf#Jy0&dv`Zq+j@C{`pWjq|K_KDP<52-gq^$LoHfv<^1Doi@+t0!hS zD;0fM^`M1>IQ&27eC}ekh_aCjK>X{NNi!CRe`|-=0|pBy_{%?v@`%9<`Y)Y%l_2*7 z2mh;AEuOOE`v3lRS)RNDI0!ZAnuh^JSX@(qJsDK(xT_@c^oOg{DQ)$WOb1jnQM2R-Kz6en3W(lBu&uha87|agy@{gIh+(g-P^Pemblkh5 zbSemsk*}L{VK2B&-LK!8K#)c*m;AH>+`tIcu9UzCAB<_{zJ10R4|w8eK7)&BpfN?O zT6q;Qr(iye_U7PX-2s+K32fDjAOOuqzy;$spIaLJeCwW(la^$OFn3oeyeXFwbK<;T zKRVdw<8Mz3$H%b617ktRh^9|c-MMpTT`H;_hLY5hqT^yL_F7^rE>v5NT_(@bq2YCO zZniyYjBH8x`=nO%0t=3#Yspg%?(BZH+;mnHwsYk*+_6--N;tH#v-XksA_8=M6s?P0 zw*x$+qSENe@Q9crHOkoO)MoC3{VC?hjx9VTmQ>!_k_9EGT{aBjJ5CY_pQUT>k8#K} z_3fA~AAteBVwq$!oJ^n~mUD#<whCjfAToI79;+c+l$kKkZoN#ZPj;FcM z1hwT5v)g`Mid?MW$%);}TmVSk>&0wY?fbO2A-N-+N{^4I0vgm*7d|QSkrmO_j^FG! z$S^IeX{J|lSm^!h^C|8tlXbq=r}BI|LD{J0W#QHkYNM>b+O^}+D>t7Q;~BEAoM_Nt zBFAbSb{(_OxO2D&LsmNQ?SD6OCI;R~qsqr~l?mtZHO zi!#f~ev-6En@veW5dEz#%?zqxND+ZL>;3)yPhF=N(3_S{PO=C!ltRKeEtNxso^gKn z$-;0mTN);!(~<#A0-h}gUIL)T_{Z#aH249LwiO(`ts$esjCXQ#4{suEqIlDCWVv~Y z=Kg*zgSPMy*mHw3`~a|Q@X;Cf$T~?3Q9`Qksf^-NGiy|9B<=SRi$cE%TzYKlOA+g?nSug;@uL-_g!;P@LaQCbcgfY zIJSa2_VP#ANwOCO)J`z6n z2N*E=e2SPq3iJVbHFEnI)DtcNY&I>g$35%VD-g%s!)C5k_l#pVnD8UtDDjf{Be61D zyk@A?P6lUX7Yh;>YsNN+9cOZ*m`7>NDrC>pi>$a73hgQ}v{RE+WlIi;Ynlm0@2wyn zY+uTJG_9NAp#a!FJghmbiF+cgJl|j#{csGCDI4UZb-e(jo#Kkd^m^f3$DeNIS;2BhzvIxK zKyxJEozsfymp|T}CBx({kW(_gI)n0w#2n3X$5Lwc$(K@Q=&LnUM`PdTO3t@-j$2g! z&qW?+BnqzA2e1@-{VpHVjHdQl zbX_88aCy?fa7k&STp#S8VLSGBc=DlKcNg=jh)lb;TAryP2Z=R4%^z`d8j-A*fD zd1}oSnQDOG=c-%sAc%p9=Uy~-P{S^^0czQmKT8Zzc-~IfF4kj$7u3-}$4K~=vC1om z>aM^~Qo)|8T{1a7e6LBAp0ilYpF39lS+NI?ZTeA;KbEJAAZm>vIA=4Vt|9cIBywAj zV5D~>uGwqJFnS=bK{=HaN>Yit!ejOUKk#kcvYmBpX@F%1Da3AYOc82APUmI=U42i|QuGJ26 zs(`Fd6fK-q=wokd|AIl+BU#AUmx`y)ml*u1*jsDU1BJ}S7WQXA( z@fspjiE}bNGL3K0?;I{qredL5W1N)p()B49@g64eLS4X0{cCP{=bH}Kia*EF17+Wh zrrCA=$dUZ|;e@I=lFJ?;&R}1hrcl^PA!mXbRaU7d6e@Mm=iI6AC$+v#3)KMs^^kI8QHE~lDxSJU;{^YX3lcM?oV25*yhWN(N z?fio2TJ^2_7P71IfUL>K;JZ(jct48)5uwLe$@h9+KvNz=c&Gq}&u?pnJMt?+(HpGB zy}HKzPVJrW)lsCZRi*Cbb*JiP5G(gDj+!85c3*lSNn5Lc+p(-3zmV^K91QhE$~)4S zn$oWu&2N}j$At!C6p%FQStumqmZc;CGb`I5t1bLN0H%Z8am?=u@gGJ|k5qJbhRd-? z*!2K1`S)n$jCSP=Z3!BuS1dAiS!N$ru*8pls^9eTl8t0@?+BuQ-AtmuYa8< zF#ujkW6AnHWj^#pb3JqrRxJt(3yAyxG7B!jm=snTLuySRHXsPl<9QGYC0|UQiJj+l zMf7MG?PKB5hDqdi{Mjxsb-=g9vq99|?FYsjJ_@}nrAhKdKQj8Kh3IiqJ`oTdj6^S% zINA+6SOrPinIhV_LQ@7&VZOdg5!m{@X(?y|14Ev~vLP_#9gg$F-Wt<7w=!>uck zyEdLmjmsmG#KyTqzqgagVaiw9;BLr|C2o8K4pOaHGTldj&Lq2D7>zItvZbqo0pPJ@ ze+v)9Q1CbxpdruZtGAK@%C1C_WXB+(MHkZQW zVDoGpS0^j_R(WbxMXXB|x_B5HZ6R z=(FXcAR#3Zt!bGIwmZSkkj79;NZYdDj%oPP9p7G|PRO?0r+@$eRYP350Z>S^`wz4g zU&bZ$gtXR6c^a*3H*3%(@|Fa7)%iY!YzHFlO5+}4rpwi(5{f*?JPJuhDiKjgJcTY1 z2p;h+q;&z1InbED7dm&&0YIcYid3<`@148dvWLk#a!7`RoS|NT{Sw-Y+#Xa-L(Nlg z|FG&OYalr1??A=TeoESi(9M~8Y&F^OSfYJ9F+HRcM?8!k0;3ZxZN^+;#L*5TpykdU z>!4TYf;!>-Xj`^SM0-KT4IVmmH6=({aE~8*`F$1QnHC`K<&300Dkp`rz}R&v=7=+E z7Gpbu2p_>kJ{F#ndq44A8;NkLUDLbd9rSMw^gn9zf4GIvp9ewGS4IOgYz#pNQkq+NgY|={H0u35VKBoymz}8^upjUaM^Nb@B zSU6iWqv!dZ5X}@b$>Z+n`p>Qo%AjQbtOnvCR~`R$uh7VKNq{yUGo0te%z(3UfaR!5MribTr|W8gs8 z4TL0?K&hA<0KTOOZCtC6!f3vyZGUq zxBw2?X6Y#w3|@E$5phglr1QR7db*M`Vd}aRjzD3;|0OoY+1%v!i5dn@~b%v>#7@lC))M<{=S z_ri}=zGP3I&8f^i2uV18<;^Cqt+vc_IDj%9rg6Lh0TQrBiLF8(o*PnbDX9rS4=~fo zx~A*?=qhCB$ojU*#sEBBCoMO@izt0I&sd^Y?t{m}Z}*n*!o8O=9bt9X)~x9IXm?Txd|82jLeO=W?i)SF&Iz=j;+%nUKxvS5lHKxWWPC63IPdC7hYs{-ZCu zC#AYZCu`eWZ@Rix^dy8=TuPE>r+@&sxO~DZP1G2a6a#czibdzl6G)r+hVbGo$d2G+ zd=q$AYU5V066j7s6rD|x(tY~fmiphkm2;D$Jju>di894mfAr)HP*FWG4*+SkdT>hn zxsOYR;v$T19b`+htN5&x(@*J8CQaW1R8Z={vj(KvIVr^x#!$Ad2qK|I?W^z4DL_xwtK^n3!G7d; z$g0$JjC9$A`kGSjz~KYQKfjsF z8TAv=J{ewNOA+R}c+@1AiGw%p7W1<`x{k)z&uy`mbs0GJgp}g}5{ZYrnJw3bnBacC zn5dI+eNen;-ss&smTz~Pl6`>b zKIXxhEiuFsfUHS@K0NU}b7LTv%aN){TrA6s;TPt?6;YH>&%BEye<1wD=>Nbd{QrjT z{sQba!4J8xPbh|^N+WTjAW8cQPe+o5B+3H`7}SB$!(YJp=r%V_-PjqSB;SE10Z*C~ zX?1~R3Hf5H1d5VKT*2w!0kzzEzI$#b)fo0UnlGlQxtU|ro};GJegZ_eQh9Ai!}fBq zsKx%`)1)QjeA~=g2}TIim)3oJ+x+Q$yRr1L8C>@Mc2)GnW~B{Pg4KA?Suk41+ADn; zF!XpX>-PXnM0n5-0~42l2OS0QU5`ZMPk_wkH5ZPPLC?H_k!F?zG9$*y|#aU;={B{yV8eSN9`J>@!JAIJO&~N4K$$UHtgc_ zlk9lYVeUa7evk>id`G~$LKA#VMiSZs^iMM^4^uorq-~2&9i^duGw-dd(E_G%4>&$v z>C$}*+miLt?jx!$??=OYO%VDR2ot*9VbaKsO5;xHb_t<+y5gcU=&ka5z$G#`+TV)5 zq6EAPpndcR@w|3dd%=CAI*mofkaGkNVE0cr81~@dud*Jy+m2}%j+8;DL$+}Mkd=N> zcJraDh`?ts0-VZa$)KNqRo}Juf1C80w1&6sRVZ0H`h!OoI>}Hw#9*J*;@Q&7<>}_Q z^`oNA*>(J3YG|^C0DfXHegPdmgx&M&+UY|WO#dkiDqq|o93(dFnqO~aK%up7OUSj4&bt+b-+Zc7CCauT@|HBZY~%bF0|$ zVpTg-vcp0djagzPxm248bV%OetOpyni5H;=JoS4e^D&i*>Dh+9rzG>2Ys6Ff)kFTEx+y5P2mFVr@xejp_R z{Pm*BT&{D)amZFxy7$+frnA-eFVwG#3bG6;7TS@C?aj{GaO<-(vHtUKUvURa7QbcG@ybFpP9i%X=5K^CX^X#(m7WiV09X->iIKEM#Qa9_ z<1kS0&zmEWWro*!3Mq?695RtBJE)))&vZD8DVg}70u*~_!Jz8b)Lr^x+(*9R%c?sv zfm-k`lC3RiU)l=K`aADyNNp$0up*~k*@Be!5!*Wr4D;h;exhD^+&qm2KI;_yaN-h7&t^Zt zF3q&q2vkRwBj_HT)J!ZgN*dsSk!rNSXhOR^mrZ)M2yN_AEKwLJ>I`#Mx`<&r_?mfA ze1K9?I(-8@AXR?yuF-1IY1G9mb=+bOn5ZJfLNCKOY<|~I)u2tUXwqQL88&I2KHgt3 zc8PeW&LJ^H!B9EYF)jbu?z&HVI9?rqF;)#NY5o(**BknG$_l?8MS}^arp2vxM0AI9 z+ytf(Q~z$dpKkdZE}x{ISsCSR zm?~QJg346=Ld=u;`)j`5;&}2lCSr}%pkCqGu$eZqH+BY6J;9_ghp4srEh(J~71GU4 z0Q(b}%6=(nNo#7;HxBM!tG0If;@L(Y3p2 zq(tIh2~Ii1OB!D7B+tOx6f6;w;+$^q(frsMeVSwF;wO9STLY5#djo}A#vJ|eCnt9i z2L-sL!YrIKvl}a;*G#4ENyBV^Ok*;X1DIGQT(S|RkX@&o@-0V5i@cvovYZc$J;JlT z=ock$TWoS0mnQ4sbAWb^XIoo%&OmY@w)+k1z-SEx4{rRK@#UcotHRr%XcNKa+4E!|UaSiE12?=hI_-n%o zFL4ky_&_vVA9x#`n+rYK$sL^46p9(o#mX=-%1-v)h(A1AZ0~P+ydgQuCu$28e9^<7 zYs{YT-U5E+oC`mT;w6}mzT6BkdjeifzUA;dXJ$AOiVNemj}I#?i)KekuuOw9dS}M} zOq>9>BVW)+!n{Zq9ikVZ!YqJWSdPZDD_GHBRYb~(-ehRhtsN&=P+)7^V{`aa=?b$x z_CEcuUjAZ+MGO4t3I!k*Fvc()av?4WAtWLt-^S~2oV#xYV@Fx-P9jR6ZSdwgfGHklw@MR~1>US~o z{1UtwCo3Ma2nyi@GdbxVyW%ySux)JBhnP;_gtm zySux)Q@FcRK;cl-@OMviPfWy#`*I`m;l{ana<9Giw>%jf%GTeZGtV&fBNJe0a(Vw% zq}qF7y+vJ;ou-tdC%YoJ+!OYsTiXP)>V;MVOeqU&Ls@+hz%*b^0^Jx-1}h9$M-HjL zxpbhIJ6L$LdiI1Jh^-x%vYY+Daf+#}q{*G7a&+ufg6v;^2wG=6$&b(~QhEGF2aeQH zFkK3ErQ~f(KHUpuxl0nkq}|BTNTp>7J5-vV$9I~yL4SNqQa~vOK{dc56~%|fbQTHH z%J-7M=U>!S6b$s_p^A7yXe65ga+n+jBH#;+D6th85_1+Lo~w>3ZW%Wd9$P|As2aOI zxow_{!)=5apk{D~n+QK1P-?hEfNzeJCz@Fe9(fBIz)qi=o)7>On(PqC(jOAo(&36h zl%*%`Rn)6L_son#WF6oMsx*8e4}yd%|4?Du@FbQ}u2tl~i_)N#sa@rq{!)}+$vyN0 zJ5zRC#ELXrJ`!{D0Fbrc1{aCj)R>8TWnV0a#xJ{x({GDeyWc&~6ArfXfGod~2(E5y z!<`K;m{$>tO~9`dM?Uiy*nOfslFBCu=SL{SC@9R^sXc~@pFwq>#NTdNQY#eegz5R? zCC`S;q+^`hVz|B1`0w#`UcIN5unt@5;%y8Vj*T11&rG*jMQqCbNd}{t6f-snX_+(D~RB3h*%RY@7h}(Ue{cqP58;Qy<-qwUlO=QRb{OgErP&DkB5Q07=52 z)(qjyF6sGSZ=yZCG)7a!vzr#63T_!-4QHi$Ve|x1tgU<|N0z9VKo+>n_3#tiW2 zJZ(nab$I=mYp3SSyGH)db^Jmrs>vM@mC;!$lTS=SmS)M5uaq&&0K601tc;uT9!@kR ztZ!GBn+?cnl)Zo$w@7%WwolBuTn8E7oDF3_rcLMpug}q=)gK0?%{*lZmID?$?$rJC z;Es+QduhNSRug0^@G0E3d77$N)h~ROUzO~3OklB2EPd`4AA*1b@hJF%Bok=>Ukl4U ztJFDp1A;=}-?GV+!MF=vEx5J?kEPZg|TVzA(;f78BtHcABe&YxYot_ZH4fX8}GO+y*3@yd3S*EL%MHlZ# zs=AKlelUzQlDeo2fN(0-q)PNooU&1v*|EcrA~R}q+zd-f`WNJjG=GFoZm{oM4C8rj z+tunebGp;B|KFz%xG`A!bVuL{s{Sw8V}q84+rwmtbOU|o#<7K$;r6LWnS7#G+9x!? z3?FbP1xTH?i3K1J6Jj|v9tN`J;mn44TdG*M9Dmq%rBAFA$q1X{yG*(@9hUI7Q!Tw9A|pzsml%dc2CXG^#DzXw zXX*9ZX1;AF+et(Y;Uzh#q807Nl^mN48P*kJ9niZ2WKRsn0*Pz!&gq*<)TIDpns*Hm z!k!Z|nU?JQYms+_P}iDyTMLi~gN6@>WjfCT=^Mb$DJo@ zf_Stc#u)%KEB^x?cA}udH%}%0%`5*-ne46`y%4xT1k@i%0a+X*fjOL z1HO-1cL-dDnposY_DUDe({gN?U!fR=N`0Uf(mWi4%rY5E?M>w;epvT^T5-?a3FMsN z&?D_Z(#p7Coo2I9^thCK7nA&U(@90wR@e&Xwd&4m1!8r6Bc5q+uJ*x;Fh!mnBI z`B@{&g2h=X-G2aECsgN%0fqCd}+PEAh6i^?=Pkm)Y=xp|ZG9_IA4H||y3VDe9JEw&t7!(AClAO{c1`a+Y z-F(KJa!&Nei)O8E`f95(ZKRy;T5xip+=YX!eYF94wPv-weW}ME+a2XT*IVB|rwY=> z1JV7RFTLG4zH|TRU$#G%1V1j&_CNwNRr6voST)NA6|_f3*@28zx$5%i&f#e0Ide>G zxrrgo9gdt>yhzV(kdene!gc@|7MHU#8?gQhBkyy?tp~#KqY1#d7%Ejp;q1v5ACfhb}QEjCEz>wV009b(PPT19gXvlzN~#Z z>Se$8(pe790`J|H{_j&$r23BxFe1-= zLnuUEC*VZThxsMx zaO|Y1gr`-ULZ0EvIDI!L@CF*N-9e6zh_cc$2W)89vftV+SKT88mm-(#?R_G6)J-tH zg7@mS-u#7*X*4AUn2i}P^dkMhAjCP!+q#SgC(bc6V;V!u3IMOM zmMLB9mWj1_awHMCF8>JmItK-pd;6sUy*et6O;SBbq^>pM!&Py>`8)ha>oI4j-|hgd zBXE8Gw@)$0<_%Spt>QO~cvN?HF2>US5%QB@x57pkX!90jn@Dp)b$9jXIXWb2je~@| zB%=TwGo`ku_JEp_Y~AqgHz__Z9aW(tHB4jaKBlpZ5But$jT4;;pe%#2yTU<%tIWo> zkMr>th}#u3aGs;c_Ol_X$3e-m`tt-K3PicYrMpD=E6*LG4MSqNl$fv%TNQLUs@p4X zmV8!~UDO)ojAxpgJMvjHtKzOw)6A*Si2IpTf%}Uiw#7pOmJI;}Ci*ZOt{1O&1;nwT zx{E>*^{MLAA$uOrY|^02#D3>MS;$P{FQ!o{vAm8U={eH#Xc2T+akSzfS%F*_Gw>)h zs9;Aebm8~Zs{l@ay?xPfvHwA$Qk*#ZNzzm?xrXywg+6m$az1)?xsu&#QIa5=9Fn43 zFu~IxqB|O274EaAL){H^E3(S7IA*1j^_T?U^aFq_vA?& znAr(dsGTLZj#k^3a)%k@+louD>drCT2I8i>Y$F~QkF0;mH#ucZ$guhgUQf={ch?0@KV(xMS4@*usY%(-zkI^`jMUr|se02gnk)w(lPbNa~GU zo~VJA$Zk!vRpdT0VdbHsU(&^rBqhnuzq{V&z_QuL18-fU)O1sRf=M9lMNeJJhR+{c zumMFFt=9VoJ6j%4C;>8ZWd=z#n;}SWFmyz2UVP$3m1o|F=kVQ=37}*WdNEJ0S3>^A z&{Gb%QpubH(}RehfJta_r=<$$UAeV|+Uw=>!{ku`kCD7S>AoCRFh)r2j_0!zbnl=w|Mc*xA&!*+0`7nY0vObFGhm?XLg&D z#!X}g&CSaTaLlJuVKC&iDY$=F3w<4NPdFp_5JY28>z-o?Rv@ryv9AUWL3fM zqZox_LFQ^TM_oB@aPA%m>o;Kg7wTX2zz6;x#rSo3%ICgdqWuWifKVbjFKi)^yOKO( z+E*HWiX)Vz6!7TKZ)c^?=ZIDk{i_x%YO+R>-7A_zlib53EK5viB7Sr#B@YHoE}R5# zN;m_6y>t1F;jU1DNXl9IaJJ&|_VUbK(@F>>oUi6M8{4#X*7oC0%^6>xtn=zDZ!>EBh800Fjp~ zUBAiz003-cZo2el%G@}{(PN?08BgUgrEba;opm!eoqj=QrPcAo&JA0fROSGGv{EDp zuH|eo^Iyqb#G3X|NXIUAj<`|ESxlQ|m*-ze;sveE8eK?lbZ z^|LGYo<(n}k z6?^g(k<`MF^nacQeV$ZxmR8mRoYlm(sTd56kT|HqZ7Pm`GO^B|Nj?J^lgHeCk~z#% zAO3-e>WGWkW9JqV={s1g`<#<~o2m?WjLGp3%(HsBV75_C@o*5FWb^T#qHk?0cZ(ty zT!MjHa)s2bmFsS|p>_O6WTd=v*D2>ON@^7z?EJCHZkXFO24fVYOKrWqY{OoqTHcsL za~v9czn!M2gd6w$7X=y!Yq{rmgBVsU6Hw^wZ}{fYY~(}+d`8)GQlv5B@39SSRD39u zWJPww>YQbu!XQR#e07b=>XgHyzA@ZxDXOJFp<+dQwb`!|jAFtb?|-qBaPzx8Wf1hV zOx#$?3U5^}3*eYr&fRpguS12kN#EVPT0&#qH4AW=Mu3=oIidny<34o(DfcWRR4E&% zpQN4Sd00PYe9h5i_bDep;I`q7g6c(aj~_UQe@OcUQC-$wJ$bx_SNskpDB6YbjPXhT z3P9SXyZjjWU>EL#DtX9$_+b<$>hZEa#7q*HQ7$DOwKwgEuUZ9|YK}<@yC*QD#OJR704_q|f zhPf$?8a8;l%!T$AkXvI#3eEB)U}q@($VJL@0_iZ$1>}O7<${6`)&H^gnn5NE(^i~6 zGTuVM*~Tij#}W`sxl_8GdDLbs;?3fsZbaBkfLOR z|D_zD#0yji|7*?!tpp9rjgnUaMaO{-=a9E_Qn7l{fPyzAHZ0z`P98mrd;%DLx5P<_ zAO?2K>WcuY4R}C7qUw{~L2J9-PM0s!h{Q0pLiQi_4I9YC=P6`=XeFI%rNrrjc{|lc zIrJ%NVSOXfkp?_D8t!$nrX*LzQsLS}B`r>fV{t+ot3B1k-Zdwl)e%JspY3*TDl2Mm zgPlLS;tTrwxJB)GdSsnpX|w{`5#I>ORQ zl#v~I2(3k@&zu>=o1Lc~b)A1fRjJxtXK(avAKFt~Kpe>59E0s(sd=2}V#BtK`L`AS z+?MF@O%nbt>wH<5`LMn%ZrS|nM0wzcIJh9Gu8CrLhN_B%ta&y0rtz=^H)(`DZXy0A zKXecDz4H&Y#o#`_fO>4nV}BESO%m=OXRaZ9W#K1F{2i1(pcGc;yRY+#<2=o)jR4^B z&U|9!_Y7^ZNI5uT{S*y1n!4j}1yLPF;fy#SK;_s+SxQ^k9zHT6HtmGId5W_iwNDq8wiVNU)%vS2>ea}_s$3$Ta8UQ;lGD-|! zc}SQ7(U#R%`H>^C(gMOAS9J-JA+quaq6>hwgdd<$pT!R}t+GT&B!eHk2AL?i`~?C` ztgMpp8huihJuw>@F>cu1z2Ak8BW&kOceXCfBTldWq=&N=W~`u0v7dn|1$E^P{w z)PpRogWb6{g%}Mufch}?2(w(Sv43!}UrA)$qXOP7aNqcFpJ<=@Bn9e+MZCZagSaj( z7-;{=|NFaw9=OYSeQf{Apc8=^=t2Tq=D9NC-gEeDCa?S>F>FFa0%jZ`FcCjhwOPt@ zh#z2RWVX_l(m@(nB3*IFZh$a5Wc7pB?X`X3x7=fQSisigxPj)w!!jf>Ixvs#hv4kX z8Nz_>O{K?=td{a+C}SDF<4=$bx%rTWK*KrgK-tAUq&fURRQtyvkV8GZ9_t^#)Yp+w zbp+9!%h0bOa-MBN`wDA(`+gZ-q*GkD(t)M1^DoY z@LIqA5h-Mm;Rk&2hm#J_QZi?Yz%6eywY)Q7 z<~8a&!&Ae1_&)s(g+NGy9hQ8gqyN(?`TX$?Q;-_XD48&qb|W~xHtYsB4CTIgMv#RQ z?TTyTb_?E&+%Cc?CCh8m;Qn8q)n-$0$V{GE-M%k32gXf^6W(g2p$%JLL;*>=b7Gn5 z8ZkS9Xmi-QlBEwQ$PaB1c`ejsjS)AF4rkufBY5kP4ax|eLbmI&nOo1Xc!iJOCb$_R zf9W7!{_^(xgnSu^Hj@HfaE*tk7~8@>ykS4IC%o5$E8o9h4!eZaHwJBH#hHmF_b}vG zzgS!jb7A9qK*Z)ErAv_q)M8wSZ}KPKyeAFvlz-Z@kMI95T!edm45F zlrjV$wH&wE&=zqsp>MTmz%hC{J8Is_syPDMdVe#T9x$P9h%BA+4(h~;nvn@oiRW%43lYDhR~BF6s| z{$L{hts`YuNA-Q(Z5}j_b^c9L&~x}?ZkCg67weeeXAlxlSNE1g%rxt9e%kP^onbRm zHDX6qVulqfA(Xlw#!TRA%(+rt$kUl;JCxGneLX@B`Olcyt9WjI2;D9Od#9Ae2fZpK z-lb5`^pRAQLv#w5ko>?kq<$U2=G+`L88#{3m6GYVY}}+|)kGU_dYk3nf{~j+W*`c0 zAbVCYJs74XOc5|RBCXLxra^gspEfw|>to$1`RW2nKg~8HA?MIND6t{P-L4TaTSaku zb`wa$f9^-Nd?-n_(S$p>8reMnG2V0pkthLbr~qmx7HI^Oi!_2N^%6)`xf#TpWOoQ| zzO&R8d-C>z(~oQ)D)qv<$!H&1@rrOk*P#_;LNv0y858T4*bDK|1u*%fr~M0s*XV#+1lN;rltb${A$-%84jb` zedsFvIX}ZT?Zo>-Rl5jUWNFZc_}Il1QLmKs>1h&@)8#LDM)2351I{)$P%IMUC{Z|2 ze$_gGO>XjxtL#ri)J-otY*i7J*yL;<_J3|>xnTjz-B)uD>8&m3E~6Z)*;}`}zE*K1 zpyVwFUi(NJ(gJIoO1sh1mnpC|BMQ2)l!xVX%&*+T}x72ac&kywVa_gL@`; zI)(&pYHb}M#=nCp?v!4!bc|9Tc7bYokaR=J-}#Fd5%8r%tf>LgT3xwzj{=2mVTXVJ zE#fRGmS2SoA-~FVqI`)dAs*0K7z>EXmEr))*`~zu%lCjZOr-@cF!G|GKX6mzPyCOv z01zuPV!Df(&WV#sKQbrj4}>NQza(S^e9LaAXOXE81d-R|eI8wuUn1_=!0xTF-u}^= zn5}|dG2=D6!fj<-Cns2$#nT=HFm9YUZM=w$AP%>ezsMXBEv!D0*eejJT)$R2<3EY3}T7RG4!gxi-d!`g1hQoTcnPJ6iYtA(XmEP2)~G?fLxH{e62 zYmD>4oF~9D2bakA(M+ukER4`J+(uL&pt6V?GcGObFPpWEN*u204oo=n$`#3<@ab1yi zU{=?3G>}wXMGI^Hao_@H66%&`q!{kkSod6-C>B#1q~;>pS2ExQqyd{Mkux*Sqkmzu3mJ+eoro+({K zQY}Z*20J}hI`Lej7610?h>Q34d!;P~%ZF&qt`P_ic*cI;Fgv$OxFEG;EI9$fzEKy) z!T&)|IfET9)g+Yxw*F<7FJPrQ1Dkk*wj&Z<6B%aT8XOLo(&|2vO6I zwi4|oyE-k7-7mvfHRk)#&z}->@ZH0ym?L4c_|#XUL_tiud4WVJHt;K87z3IiHjL^<;3 zcn$A7k}?t*j~;g7ip375611L6JtwNDEkDVxVz#hm+nMudKYbfA7){Zx7FbH@>cZ4Z zRsh3H@}|OpeOu6@9oWT2cwQ4n*Ukcno$lVmT3=*rH=n>AC*MCadFoiz_L$( zg|Llhg5WYyf#{?}7i4_!*{DJMAyNDU2m&a|);~dymZv0sjr5>Lk+DkTyp3+PvhBLx zv+X)1Y(oQ7g+?6$#;$jLFR1TJJ#LNm3g1%I;uVSy})yPS#m?y21OMBZ1d;g2DvrO%V%Iqh}r z6a-VNV+m%(IcPG18^OxPNye}%7?`%vo`RoNr@VN0kG3!O5}ODg*r!L+GWiWNSFkI( z=1$u9%686K;6#o;1PuM|+12%OT3i}Igu4+3#M@5nH!B~*ib*-V0g6kFt1 zBj!3(SQ)rAp4D5Hp9LN^v(%e7 z4fuZ!_vRcNsS_p;b|AcR%7p95p3}S1!FchlcxqGJ11EkAX|w`CdUxYE8piV=jl!}! zzkvVo{r+z!@Bi=dq5D>o{wF&|1|mJ^=XZBFu$2=!-9Vfh2binA=eW%L9lPyul@``& z_h~s02x_m%z)TX#9y&r>MdHrSyiUHmJz@}|+U>G!^S3z>Tbe%*-0@Q-(sk3Jhj^26 zDtEr`_n+MP^zVGE0I-IzIEk5qa11VjnWMB2_wBhOMM6&;%@2OYFjD10=$9~-Bz{@! zsyOMl*mJGaetX=7!&|pzd~Z9>sREy@ED}zheCV=&;huW}JCej~oo={Q zQtXT?q)J|#X;vT|6g&@{88tg8jo}NgLgC(nr92xQ0><5I;MK%t@nnud;-Ei8=|XD; zRTS}`3;Dt{LY2O0+4x~^vTtDetJW?^qYTPNYvC?|3vex!{cv4A7r@Cp4v)6XMo(V; zGO(c~KoIF;j%0E}1Y;kt@2@EJrV8083*A8_uNUT>s=qz#j}_yjf84ll3WM}O-CZ$I zJk7dw!Sdbl)n4l%GYtJ!L@d4drxn#pOpJ0`!?FscXAP?*@NVH_*DqSz=$73z+wA1d z72*pYfDl!D6}BrTUP3x-aZJ;5o8v10pY{Tv~2Yy|1)Ps+2|1~i8zdPms-9!JG z(@7h@BU|`(7+j_!U1JcY7eF8*r~A!(pJ)Ap-&hAPU4lyx@c($s+iM&#=RpzwYqUp~ z(p&V~Zk|b(PND_P^PL6H0+rsNPXGzr(ScFdU%~leg#Qzsnf@bXw17NWIu$;YBwGeN zLj9usHA4NGeJ(<4*nlfBoPe`oxYyobA+JVk;PGfa`sYT3 z%jal>?B_~|0PNT4patw#-98bJeo)9?I}i)wb2{e4D-Y}6^MOy#$nZ3I`41CVk^X3Y zQ<8_MDyRFVfe5^?Y^!J`ukT>mo}RO#%&7r3Y%fZtn8+sT=V;UeE1A3_Bda%MGRr^A zQYfyyhv+OyD5cTd8Yty36}3r`gtoA@_Z@^-8NCc;p0*^t6lG5r_LAGcUl&?IsGYlY zwgyMOw@xkP5mbYmWVo^CWjxsO8Is)A;_?ERi$fxi%%lQ?n!d#fq`zQhaKI*wo9%K` z>oG52uNsZ1H%lXed&@TZ`8zVl+0lzm5F}(m5&`05pd5(M>=Xwu-Q0F=SA~Ph+Iv+J z?3Ht}Q0$wm4Ux%3QrNLT=$a{+NK3dl>esIE*c_9UN_!I$V4yp*-`ab@>Kpg! z9M_cOLXX197P?)LATh-DFQkn-=+%Tx@Z(?U65(p4v|Pqw_(EUf9VW6bxTRZ$uJxW~ ztx-7mPkmmg^Vb5`(0HVv6-9WK!anBi%_)k`wD!!MgL9Eh7PCdCjAr**GF;Xsy(Q(S z_1N1-o8F};%8L_V){6Flbm$DL(Jw~=c{2)e;(iwxovk&BX31c4?sap2 zF1;LWGXDYj`@g&5o?J2u+}4+IQMX-f)2(1x?ujhEG%f)X_{dK3dc^Yu!{0I{p*Twx z^Tp$&IVCe!Zx#G=f<{_2{JM`V!RmRS0Py$8@k5upA3i4#DfBVG1E`N;0UAaisU*&Z z#zMOI5v@m6jcnJ5W1?xs&tikm;}4&Q)a-@Y&dVjwWdPk3M<_8v#gcW*kFs3zG1%Fu z5gE zO^rqU-B<=X8(6}07U$%k$cA#Hz)l5HAao}*a!KGJ+Ken|bXrVec3nJKfGTSg|1APv z(e}3Sbf!<#OXHRKhsaiD>+de7JIUnCg9Xh!P8K6YmKJ)Qq#J}e&a&NQu|#c|QsH*O z!N6Iot^h8hT`_LDnBl&Q<#FMQCV4Km_zfFt(jH-n#>>@=^j!q= z9w2STA++Z}v8l*~9Y8X<8tigDqBr5&^YEXj+?Y5&e$EJbENxg_=#7zUkD`wJi}@mQ z^cw?;b7xDOfBb$W-r$I1F`AvJ=f0RJw+FDKAR@chzu zpQW1JrpxGbHaT|}i+9GJyrf11yJb+>PL>rJ_J|o$ZyZyWS{q(3O6yUmHi~$yUT9Ju zjohWKA1tS#AFEZl9ulLjKSunrA41DG;-CSfH|HiWA$<4h5ZnK}*s37iLPJc#hykC= z3E2W*otk!6htSQwi|FfVq%2T9>@pPz6hE(%{|+Pgj=79zV=kE_9c z?^0~7DG{??5{`cTlyY-t~`p^n6f!FQ$|0|-3D`t8i1B7;#pmF#HLV*?@qOlMH5^% zaiv&;Qe4bMZ%2R^xle{q?B%2u9We1uttEmrHQF5OK0V3yhZOn)?MuMX-LTdRhZ<-+u*s`_z=VE zHET}oELPbkNay!_$&9ka&?Ye`olH3@{p&67@Q|=Qr)@8)g*O) zGT*hag>C$Nz-`7Z7Dtq@b3W&SFpIx%JFmyr9=UzeOrbcvr03kRM}rGcE?)j~l8f=J zEh{x6zte~9MFUjvQr_8Wgzs@3WDYqWv#eq}E%N0ya;Ykxau!AM+O+Wbm^w9NIN7L7 zRMS6a!%QeTt~Z>if`8Xn>rxCCMH;wV1o~4gWvwV|#yWORi0H(`mfXQ6DRIu3n*`1E zH=(U-oH@NX-lm8F8mu#rmdohJLkC)MeSS85XW6b?T_sA2(dE?fmjx&OlkI~X1D^jG zngGEL`}0B-8idp08dgP>;XAV*nKDuGL@&F~1swZAI|}41C?+-IxS;TyHaLmZ48PJ` zhJT+qYFnnY6B=H&Oyx93FixgC=W@FSs}3ND*K~+&b&g`=N#`0U9!uSE(M}eGG6Rn< zv3MJZEpm@Mk`tUQ1I+@y=3h}6fJ)K6|5$wYTQFLuTZ(y0$O_x_FU2so1@<3F<#+Ld zcWhZ`UgZ;^Czc!!cnh&R$h7yf>Cs>kbnvRb<A*sL23*mA73w);Y>BV| zE9Ny_g3-er4Ki+^E4tMQ3B*1&ZM9Tms8$faTa)`ksrzbS(>Ma?B&EN?dPtQWMgpXS51lOCRl_fnIj;Uy%sNiF9|B@zkb3_j$5MY1*tZ%mO? zyA7p35Kg`FA$o%W5G)YR5M{yOC^{uQ(#|y0ut@8-4kWxlc~{EJTG^~(A86UK@fR~l zQ`NpsOgGiOOx7ZZzXp*ZoU;Y`A@E<+JrOA-+b(|s$%}P=aAu-9IM^Ng;BCxL03_#2 z9xu^l1G19`Fj|CET1!=1vq-#HHGBmspTlL;k4n&!Pi>njPAAdNtLUqO`125S%t?(> zATk{u(8)1@50t@aS??jSg0Y*vg~~P^4@eYX3q@`QnB9+ZRoKbeZD)g=<{ACPez3HSM;ok{-Ilt;MVqYXDtFRr1vMJnHbPjEXfO4wvyWVs+ zJ#nWw|JL?zRR7unz7Tts>mXvFgRdr6q#AQn{Uq>!avae=LG@@3D*if_&u{$7@2UbF z))t87_`YuGScFaS;M1yi)#FO~Tdgdq#Q0=`vqeXw%7q6u}xvAVUI7^Re8H`8fVfH1zjTFM%G_K?iDjoyh`D zPNsVm)M-vbr<&mw-M~urEx$lqFnsyWJo;ujS$`_oSvMRh#o*!i8o4rw$qK2VQ|%D# zm71`nbi+|j%@l^Y|JNvK&RL%Et&&=S&`;Gdx#a8rET@dS@6VW^k{itt4i>XE0%e&~ z8KoN|{Fo0*8tSC!h!a@MLgIvDJD_@??uc!`F`b5jSd zMfnxyG9zoa#RMe@b0e=k!argBSm9iP*p8Kw#HdKSb4Y|Fu%eDP9GzyzlC3HN86O7N;@c!h8WiHPJ_#IT|eBx$6+j2am@ zdvCbE_6a6eWD~yBV&^}_kL8j-)15&&I=UL;SfukvyF?w!AFxXtq5SL1^=5px!#Y&7 z_pp>H)4JgPn|sp!Kb6=2TgLsTJnP~9D=yt0>VjGI-*kpLxPxr+@3QU|Bt2;1dm8w^ zqyb|X3Min6nXROYiT(d&Hvd#nkOLEe_XpT|Zq@0w1{eyg8{$(Nj*zC>s8C;-#OynJ z@U^M6tLc;n3(4_A!+w2t%`#9168e{W7IaHXKRi&J9< zz_j`lbEUngmMpU9S*`%g)(;w#n*B|*a!#us?xM}?(lB#*9| z2~K~<8tvC^z}8cwAEYdr$w7nxeywBfkHyoW<_%#WK=u;)NtPW-k;X^>KVZ)LLtW^< zVxIr^iT=OO{y+6_>lK{=%zrU4py1-Vw!Z}#ZK(8xcxbZp0&!kq;QvkVEmhb1PiDCw zTkm49@E{zQ1jI=hz27w}FUlPIB!x%@( z%gl||x&=`*!ZJ43Q@=gGJs!Tl4_CufKPCoMAyJw0LFvHM=iz9)5`*!Bj++uiYmp-r z=Iu}->wOTZfD2ma$@Ob=-P^!X4}(~Ebb;(ny%{VYBFLjEdK+*bSXCF`ibX5#_)SXD z9sT4_-li#eUd9&&JRSS$8g&jUfg|6V+p04kGwDBdYuZmG4+2=ORIUjy=D4D~hQ;K& zF2yy%#MSPwrw&qz$s)M0nT*y?;D5$KTl@_VR53t>fiiD78g0Wfp~D8F(C}P-gp{VjozVrLYf;@qbMDNCPNr9e|MZRZWHpY z$h&^?35$t8xVdY=XaUII%;Wpc*-?^z_4tarRv2zsB9NNLE=E6LVmTQkUBKwP09@&F zV}|>>K%*8j+G2#CJzb7}dKPN4`O6XhKwTqG|7xeSU#o`c{1;r^bXH>A2&K9pVJu6f z&6x7BN|{xsDPZJmRXi0(u|ZE5^fSgE5&Zq1L2IM$fV~7UkVLVUf#w<}$jw~9ld{qz zW?i>05?niwyHENv^xoTWD2QX!09Olh4*YE78px9P(ov8&xL1OhT)z-^l}8M6W|-3` zw?Gv9JIol@*Aw)OA-F`rybbL5C2ax(8$M|oyOi70AN>6r%aO0N55)dlA*LF+(!gZQ z=BsDV#6&<<-1@>PBvyb8P6|B4g+I{Q{9I8XYst;nLtz7s!XHJNnS+m0$A-zvbrjda z0!5EdI9e*hhun)Pr#y~6AC}yvAL55+k-g)|Hn)b7#`Z321l$@Fd>w{E7x|-Z3JF*c zT1~oKs~A1Ab8d7M$LH<5Q|(~O_*!@uCHK*1_PyaP6~6E}<(l`9@=t$^9hz9879*u( z3jIf`{NESge_j%FXy_mR`GsNx#1aYO{P>|S`h6=MBpHM_9S)uXEnUEmju<$~>urp0 z@$}n2#XNaOCZ)_XJI!8Z)6KQLG&PSM3x*sZvl)&K&Y3OK<<^Y%Y^{rkB65N%GMB0f z667RSeO!=Jq^=Btfh?h{uB?KLlv+ISyVuduit2)Lhwy7#$g=MnQt&b>__9Hk05$;X zTos){&;V9`XGSYjc5mhbDFC$O-nX3jE^NS-oG}ydPsFk2G@awY*k?}W7Ws3|evJ50 z&5e31(hs~Tb`b3FYo3pg(t8Xk(jQ3CB>XHj<8d?^$PI%xLurq9J~ZV|)069P$5(4i z4PrWKJn3-=i>4z;OsMW~ZdM8qmeKVg=rb^G|}5YR|hO zY}*{Ck=r0R8cippc-Kru&34QMT{(k!>b*L{&QH15(mJ>e(DMJ}D`$Z*+J`7FKf+Pu zM-$ivxE5h;Y5ZDl41wZfb3%LfI;DqlkOV>sbamhv6N}t>xW3rkhQFemI*5-_d$u{1 zN0fH{)Z2P>MiiPm*#Q3OaPNyYavQ+$0ukyPA;W4S@0%%LI?TmtX73+LsCRj>r8PVR zCo-JMoql!ZaqJQut*`bf1q-gZb7X$4&T-;2oK4wtJLJ$VK5V5mJPRb8Q5e5^^9QT% zNYd*?TsNh5>qc`N!a&>(;^K3r3WynZP`THH(hqIF-)`Ti@BuF0dqC|5d6t>wM|<+@ zv-T*AcI#BU+!gs|27ajbk9T1gyXIbMI2+lCY>2yF2z&Jkv@dcVZE*|bSK=+8 z-f&s|oHX0-pocZ|=MIX+rtbQCPaNq!n7{sV{IB`^IKj?26x>J|@SdD6A-(kUNDjcP!vNX4V_=B`MQ?{I@LZgUuk=y4a zjzofQ)F52l$Js{#iMO6ha&_+n&NpEcqZ{H`}%|0-^7+1W5)%KX59vB4ft6J(R zBZYXCl$BT>!&_>GS2~M*0% zKt}9}-OH8k&cXLMuc2%dO3L7B)`eJ3Q-%d`=eS4!lIDo74VKFJMG1#u!p?IuwC4y` ziHtutajU=t7t5x*ubFI&-JbW$rIGFrA7Q$ z)$^KrXLcbQw=CIpN91?`@ig{VbivhqvVb1&dF$_CfP%qZC!%&kVlt(oh4RwZ63BK? z0_D7=JKtCKp`H%Dl!Bt2ok3)aevTcV2{!r0nx&kKoqI7fA!AYPhptr4uAj@V_2WA*+cLIyz2R>4eznQNiE1eOl2n0BLEYC z3wI1fkO5Fk)=o%fet7?YC5L>tUl9MQ4!d#tzUN}JxhWrLdxz1U@H(<k z|LRkv$tH=;GC2WAt`lT^NA=oFX&F__{>l(DKbo-d+C+KEJkq{zuYk|+fSg8Y&msBN zZnzpknfxIkxDk_49pzml93t4Hz>*K}WWx$OrYw?1w=EJsCPOF8c&1evF$=^jrBC}# z)0ui}4E)=6Jb-K?Gw(xfNBI}@=uT{FMj4%vgBEUi)9w;TFIZt$Pf}#Oie+Pp(pM90 zhI~45q?Nl@@3|zy@k<87oY+*1&B8oP6Xoe`i)S*uW~-#z9OpFosjQk#c9xUXaE&sA z#;j*#S4Y+k&Xa#f`O{R*egMwzx%c=DAx#D9*A92B;$&UgUB#J_1w zTZiEgJrpe=ha9;{DvvHQb`dtBXT7XNyZGD~Q3@K^JjSSl9M5-blMt8?FvlDpTKr&v zvARM2=Lw{qYJcT#z)^q{O-6r`4e6X)D*Jb|nk1hzeg~yeam~U(@le1VcB@yZezP6C zvqqM3ncHQOIB2<;WfPZjD5X$km>V<9xd4ko*_17gU^IRtg15)jvlD4aG623UD!HZqeP|8s7gbL+le_FnZ=tq*J0s(R+!V~*b_#SbM* zazxZmXWV%EwXmZ%iw40}%D!0_TU9N;M=nr{EM%k&NEE>^zQR)1uWC$h&o-*nFe%{B z6mGi**fRwNVP zUcXWV=pio=#s&-{5>iS$Q4P-EF#n3YK=>yWgbI5?*@5Z~|I$d&9b^P%Iyhm#hQ#br z8|u<~vv0#q-CtY;XNn77V(&k_o0NEVycl{80KEOaP5HCfD7q?AT8VT=u z*QX>-KxSk07+d{xbBLLFoCfLlij1^(Wjpenqn3#iL(DlipkU1 zf8YB1B%Bh396v(k(k4=isJ83s5%Xi!$VVA05rPJR5XO=Q%jzW|AF0+g#=Oc_%UMJP z;OfkJ97`%%DCd9Vq414Z6KQ*dN8E1Crk2;$)8VP>ni3=UTiEX!V2BDWSfuSj&$Bhr zVuw{XNNaeBwPB*~QC;=;yUZ0BCNXCO^Vy?VjOou5(45gp0vT=tx^tiRulOqipN+^i2u#tOx7Ixp7jlc} zM9l&JpJoK16QpR=xq(dpIy74tX5=xL3pm#hO*&0%THiJV5Gq&eKSLpsY>1#Glm>RN zIHYWT6dWFle@Y%xMfq8&cMnbn_?F}R4C0HBa?C)vR7hkTu^aYYiXbpV%!jaBSe`Z>gC0u*W%3qIJHRd9P!+#bnl2@ zmJ?s`nJ6M&Ah~SBpW$mtXWjK<3owIt!R6gd_2ExRthPWUSvUvf$-?RLB9+d_T5x|^ z!wgcFxhpUmstqwWmbF0S3(7*4ykN>*NoCPWMRK{|ABM3NN%$P1??gf@!o2cDE64F zw%nX?o2KYvH`y8p`&cPXQrP-a(@z+#!q!Q3$>GXLNct5x)PL_Sk}0ZR?O@UQBx#Z! zFm=z>=i^hk|!DfVniu~(Sdqpd<53@Y^AQ-a;xMCvoZ4swavZp^VO(o>= zL4h-;R1*>A*=HnX0jn$h1ejDY^`Sk9-V-JN1@W}8ytJdjjc53EiTQVMK-|54{x=)g zXk7+_+sk16$PD>xd>VE&F&9F9y$55_Q$SK5z3ut?M;u>RP+iq9TlFzLhVfL~cxo0< z%^1FaieqsIfGDk~Jtes%CfSW48uK(j#s_)p0E7w?a{&4Pr3^hyIH(PTBb;{M#Sy(J z!g?RY5r2!WCJb^2zE0aFOkx$mc;k`QYyv8ERbe_sxv9m&Po`0$V2O}5)&$v@16~U3dtVs# zyFnm~;2YMGi`Fw=mqF$X4XZ$mJ0j+ppZZ@Slc+G88vY%pYfO4x9WmAVkn$UmVYD24 zFk&RcO*o}yI3@OR%#Sb>)U{)i+$;Ey5YodTH-}8E4KYh2wMyQ!sq0r@P5jkmIuK~1 zaF-`2!1)EU)5oY`s7K(zX=72XhV6{H1+tO_N`G~pHj6CqK9<6D8XW^aMOu64wgRyh zO?r=GkYa|4Shp4UL2|S)-2r>zPHjXWr_I%=R=Ts?REhKKpYxxx6LmXns*Yt*Q4yY) zE1=O@9rUoW(7>7G(71z@I7_)az@6nce7HL?K&4BeZI{&g6W8(EJ?ilu4F>>RD=lrN zl{^)!KTs}7fs>NtRHSh&O45esC><*V^6HHYE%ulAJR)|{1XeM#W zrXdIJ0**A*XJVFfK1cV9coRn~H$8!lC59}<7_cAC^yf&pBGGe}dgx&S@8JFA4pK@ZB<51JPr-nleJ8M04rRCC_$n=n{oywO ztI+W5Fj}wE1xz~lXu*e{u+oqR!1ayw7CptxY8UIMhmlkpq>ONw3?5&M2da`{OsxMK z1U99(PWd-5`UFjxT5Q;pG44x(ab-SM#Of3J z%ckK2Yq}nBIkO05J16-dN;%XZ2?!>vM-Rsb#YfZuBj;~ac-l0`>oF!1;IXIR3eMPn zm#tvS{Qi()6R0@%)zV*T0%uHZWkBcHgi1YgtR>5^z9IK@1BiWsT3uHltzetCi1wsp z<)8%P3j~m|wUo(WE51C;_R7#@URTgY)uj~K$7?jKB~H8w*;$q76iWPDqc=u50#ma;`{tneBJnsg)SkcxZu_$+t&AIFYJmZwA3zY`X zSSU}v^!l&!b;KuW0H#MWP*%mjyr zC^zPg(o7J5`n0e@Cpz;%ZAz7)-{x33yGSQ=F=et|vmJ!?Ot(@e!gXSKt-=R&Guhgq z(Fy;VYV|N=0^TF00o7jBZZIoZ+ab(}cBjTid&3!W{hOBw5N0&HOSQr$;rQzN%IkQY zKlq^Eb~7l2ruGf|U8IgUm3paMmqNp!SGtEF83yQwY@q2Htbni<=vR7LUDi4Me76E$A;%?`9#W#;rv#*uxN%Za`_vT zN}V0EJqmIUfJ-{dEk4$bs@fn{>V(<;+|3lTC7|!w=hl^L&3&u$8oVaF&H|;rPgS8W z$P~AVE%XLJNk|h@$2@Ow~e@a1syOD2b z;~hV!nHa&(fq}ny?#EN_AMMk6&3>X>zhaB?bf((US(JG?O>&SE)ak^n=|pXq^8Z!i z{wW%AKMDHVkH*`u;6c>kCq!vtmK#IYnG9b2?Hr^Z;M5EwaM-pZ+0YGK4pqa9V^+}( zis3v2u!u(6Wh)+DPF5yc-At@JgHOl=bvZ#?63}#74Z7%AlD~jkKh& zEk3b@88sOVhebQa$E{jgqlQ(J<>bW7$O4budh4+*WpXnE-D0$d^)OHQ-B5J@#^<_v zw_Px$j=y#TH7CU^xnhat$trA8ddM}!Ld+1e%Lf;4fGWafj}IjeS1?kFAD9$#D95!7 zI3W&`kIil$A(3kaDhC_8qeORJ%Y!-!RuYO71(Q>PXo>Zrb{zu zV@698+(E4SW7Py__+-C25d;68U z>S^|UpV#{VFU0tjJi_7A8Jmf@uoin;bWhTgHPXlgXwNtuIiQ%;CbJh9UR>nU-J9m% zG{2~a3X_m&=E@C)jiWSg_l(F2j5Wg1-d`;YcJ6AJ(meHa2>@oi@c2DS%hN+S$D&f_ z>%8M;^F7dZy3wP7#3Zh(f#DSUn{J|@&usHp*Gajw;)aY z1Ll!v)92g9-c2)?MJ5&yOgVa-cse;4+nW8T z)4RvrTe#GCPC>x7uD<&;^Zw;WL{@ z*5F~b;@0Ap3r5NDk@+|0DUQjw zgwC`y(+G%Hi(M7Pa^3Bbt(g(!&g1Eorjo5OZKH%0wKM2ogl3HV{NMd@W|xakQlM-H zfHB2HihvyE+;*=zr8oY&jI2v5MBAB|4TzTn2HStQdLvC}p zPa%hf7ss>?0!J(A@MN9oC@bxD+GT|Ag_17?%vGmArA8-qv#EJ1-p;9bIG|9(QhpQ> zoIx$KlP0-c0$ZMUcUms4&tTZYB!#*bpjvlin6=#E+)e5|lYS}-nzLrQC-ce@_s>O# z^(xPhVkfY17uS34u67^N{`+`9 zcr<@uEbUEGnIPu$YVU*y9|mt5Aj#i2uQ9Dle`8Fqt_KvGiu{gMugZK7G`q_QCsp5# z2#8ESy`5R>=Upw2CP)TeimL6o`lB{aFD!sAob0nXzQM44oGG8)D8GR7mfTEKJuFo# zxxB#5t(2Ywu3$hLp@Lv?AkVm~c5-2wcq~;bRHHmH5k*R0#`Q!{FIW}?I0gyeGwfKf zIuY2#s4(03WcoEmSK2i|yr_lMv__}0k2wO1`^}Nj5_RzqV?p$fDx#@(qRZR!VT6=p zSd?6(;&4pk-&b!}DCPN~H!oPO5D3mHGtiUj4v?Ij4%~cKfO;86fi8BYZ;Tnv)4A*_ zy+y{R??kc z_2S5vr~`dyWzTRZOmB?qj{&3uQ!p3GMEV8EzOuFHW)zOCzHEysAUGwmP8kc%6qjU4 zsAV2W&0O`9AO)Vd_}Hk<3#?M2j>idxgW-)yd;oT$$-VZE8~qp7Jb`}NGqPA?^losW zhk)uNlU1^$xY;ws#RfxGt77YOYEs#%nQ;qp)1g(}`z`0$1jIvnLbwUx zbbd}i4Sj*SHp%x(ofs`i;oI4w^x5i+p-^YI;k9YBqM-aIAoy9DYzgClw;XMV86$jD z#8yP^+MH>qx8C~KDCb|b7DBR3us}bNg~mVaEvYBG78#bXXZACKqd_ga7CF%BEg$IG z=B(GP_aO@R>7g0V))ol4J_&3JZXQ1&po?x!To$#7vbFiYz47C$)8?wXBS1Gcu@-h> zxK$WfBoRha0BF-_#{rE5u(Ge%p+hvI1&nob@Fz<+b*w}SErHLIY*k6F8tV0(wfbFz zlW@Gok)-x(j*4sn4ZAK!eTM!f9d^4}ge|`E4pWK*@PUU}>1uDz@;M(Exgg)KnFVwo zBsq&?vHiP^Y`bw7cmvj4*%gE14xi9;2k+$_{~>o$KtlcBugCvQqWaH^^U2H%ZUz$w z2%jyje+G#-ZB85m0l;8vW9aPMto1+OMKcOUVrW?+XyPCUC@D!Gb^{@zNg)WzC@Iar zW=Bbi@v$HD;($81c5!iAYZLk}vFZXUbP_V*IZaNjbEC_Risg;v;_Fk5&!*Jk)2$b? z2NV>PtJ}%jmz&N@?$g{K{9^Vy1!Jz)0X0y13?LQY7zlrrJ|h+i&oCehd)2)EYpOwc zmW-jH-NAwR_sFA7y#J{)sl*61hvY!{Ryu`o4ZN}g&*J!@R!=mzx(B-XS{+hUTlzSv zxg&FyxvoJ7L|1%N6w00#-O0GKhk+f4UNpH*)~N4eyBjnnZ2E*q4gEpE5=>=XwDmEI z`q&wuNf=g0r5RFSzQO-Yt&DhzM;`xIv%L;usKLt*>2Hd5QW~oW>qLc zXM0$%4YQg*E3VxsJ;KhR?Ej+=X)^WwgL?ol@S1jQi2p!9uv!d)Uyz&oH5{lBYf>C@ z;sLDVv-p=8R}c&wk$wb;_tqSDh7@u7gTu{yJrPmU99;;#cK!C6_g-BAoOQ#2yG{G8 z4wccnEqffWI~M&}bRLc1_Vs(!7+uqWt=f0I7#}s!o5*MyM0@Zl@BI69oYC<**bxAi znVP>|T3z`L-HdhOBe%p@%NsAqHW!M6q4JCaOEBi1av_bD8Jp{D*3VlMcIwM3+?SWS z?8|a9DaSJ#5EaBz4)1|tKQzXM{IYFSi+gWTvFUycduJtHXId1#oXf249AteaCd7OP zr2WuL3U|4u#6~7A=YYx%!`CR6H}D<*3e?T(Q~55Ml#9&Rkqp(5fo3Y4nG}KN(30tWEPXW zj%MB?hBh2P5;bb%vm@FcZrfbmt167S9Ld_6xgdv6ks~`cT9wYV{WV5l$f3YGoQ}X8 zGdc~m7>a{=A&^ul8^*QZl4~w3N$5tPP)AbK` zvS19JF=1hC9Y(3Re|%0HybdTNVsn5`kUqbw(%7E6hLyPR#g!t_k)|utpH;&jzk?VW zsK|$)M655$CbN88@vM#gC$fV<(j6&z$|MlH#y^PO!=zA*{OF_KhxyyzU_$pIvnSi3 zAe)Og+VLG^=fYyt3Is;Yu>8iLakO{qXFNmul1+TW7mWL@P|3{a5fzY8JbOs^C&dX- z?m>(rzK#JUstEa@Xav1@v!Pv#PT1kN;~#e|h0{=*jp&dzhHVZfOxFUF(+Lya3hAxRY-m%*%Hd112g%OFnwN390k+}S8G1S^)=oqXeQejS zg`9U&*Md_LszyB&-v9u{t(Se&1aiIw+cU!n!?WinJsExt#iQ2boH){c&qU*j$`R!| z9SJS`a#Bp9Yom9}1nJWPvKle&b>7tOnpn^t|VDL-=eSs=g=*`HDnY=fooQRZ0T`wmU!(DAlp_k3FI{shVhd z6SF%tt#`Ix8zIgZ_wd~-tAVjpol8zw_K5oaHJoRbWhFga_`J~2>b~v`?0dMVB@q%C zdD%)r39tVbJK1Yn3-NDUr&3gg_03R>&8yWV*+rd!%y!NQvrAcjwv5Y6zZo;uR2N5z zE%MgvdD93_^Z`IC6jCU2o%4O2MJJH3EOGV1t0Lq+=IEJ6 zVHUe9vry8R4ebGf>j>m*FJbW7a|W_0Mgx0Fo+vlbMVwrT|H8(Em^{?W7WsHCn5 zY)t@E!7B(}xY6|Oac<=yqW6&g*BRs+Gp{zxZHEBL12|w~t0$cE=$=%EjjyFcJx7cg zB2(7!*h%_%VGKJbz@VwRdjR!*C;-=R|E}pNS=X<(b^4GIzFCa-?COcTwO~YS;tzE| z3ET_aKtpS$=#Ik@A@LAgM4>~uL8SF;hYR�QJ_5b#aQZ8cMJyu}H$g7>zxZ@aF+in5)ABB?dpRxcQLdtp{O>_gE0b z0fI=K(@fw7u1<6hC<1Y+w}*th@vX`}C9aY^UkiXlx`34y%no9$iiPrB*sFga;sF7; zT}U;|*4QxhYZeIO;4B)KJh(tGB5fO{p2ps;Lm){8$n78`%tsEIf1D!cM45x3(w!2H z5Nam+uVl2XFhS)#wso}V^Hu|MDm1cQ!rk3Muiw9yPBFvd(d`h$Kj{%3&>%C+oI4A* zaUTFYzsMLJsa~;eP7nM8bziw?_y0tPpA1&ToOPDI;6VnG4XvsABi zKH>xAN(P}Vjp`$sqfOL5Ls%;Y$!DHgWCdW1Y;i<6` zp`jZE>#;Myf5FImY!ttQaP0Go1q~lxseMZJqUePDQtZ1)*QDv7HZ3^2pa=ZrC+GoP zpb;UkrhGH0PMQH(>t%Ka)gN$xnmxun(2m1-O*T&;@HfP0hk=FMTKx>BWhsABEQw1aj)6f^ zh+$8~pBVSF0%M^L8{@dfeQOvSV>Q5)d0)nyU@@B;D{4*t9@cZg>ODMo&-7w1xb}Yo zGa<*gX{==?8C|Cw>Rj;Ang$KE&Mcl@Kjx7_t~k+sZ~>)zysr%AG9BUZ+L~OtBfGS( z1Yd@#LKTZ4{3FYRa82BS*4_E)00(^2n0h~bqJqno%Qp@%De$T6g|JtEi`peZf-i5= zWPqG*u{=I$s4A+Wcd{l<0Y~X`qqxdXvj znt*tQ|7dMK6Rad=*@+v#{xNgIU=8!Wx|rRvU1mFobeL~OIJ*4+wx8~k(;xYEPi9zm zcS~AIBE5!w#wT7?^f`^x-?XeQKDik($tV?|4TeR)s*j9ND!SZ9=u?j(L0Q_r7X`Y;vJhv z20vPkZ;UC<>h%WfF)2wBO+1pK{fv*XG?xVse)2OIWFE_yPMUgRA>%x> zfLG6EF4>pLP%4v!2iN=x;)BWVp*2*R;`~uJ(8GOd!#UV^5X}w|W!OWer^oS0r!O_l z#<1rGL@_*V1SZQfxvJ5Cgr30-mJHvf9j@PhHl)>M7`t{q6d6Y>4eC7GkL7%^S(uQz z*C9j>PBQ>F44598Sm=l>`8Qz-kIqh8LCusKm4iq$Z3XYKTAmtsH~0ky7b7!WLeaS42kM5atz&RhvOf){w?Cq)q;Yno0LBZN{#u3Hsk zqlw6B0ICCr8{*<=L`d#7^?VoaH0}uL*-TG5&2rKLrSoCZ7SI@QHOPF8$lH%%7CH zGxiXdWP#2tUSvW^`Q?F-ODWgg*SoRsK{d?>Gpc$n`))yo+((3sffX4OYLt|}W0net}mA%_KG|n3K!=*Mv1n#XJ4G+&ku`b^@NnhP8a-yLsk-%Hp_oyNy;|k z-h#^bG0I*}PPXD+J0TvWYJx>2*~bovWDkS$=*q6laFS+V=WR*#2w`yYZ*PElcI78& zH%7{htJl~ni_L50h@d6Oqa^`oA_vG8Qv6_|;)MJTk*O@M0**azrnmgGr>pyX!Mpp5cLW4s zf_t=g5beL`gtA3E~E(glxK*@@uUVkrla zObE(ikO{uiKV;LQK}|r@%HR_VFinygf7G<%pcmN7B(s>MI*>;tTMZgqcuS-=9jja* zOQhV5LR|>ulL?0C&JDBaT=zYd5#cH`rkEBNwJmY<2 z&uEdXeT4KmCb^*aQ9Epr_yRL3O@l!%n7i1uqC!wg<@MD`zZ&CJXq5r&ZcAa5;*PaIJ~g?ouNJ3i$l#|R*uTXx|W zJN6EiP*^kE3;Q=hNe-+?owuvIdeoyUYf( zw*$B3Guq`QpNpH>Y1{B=a1-cb|Kv1_C()-IAx=ElI0n5{Sx5E2$uGz(^3tw)A>U{4ze8@mPBEq$Qk-lcN_z_bB(w0;EE?HwV+(g&Egvknct|^A3$e`>g z(hu^Ani_jDnvy@EaEBFYGR zPeh0CCRN&f0lT5pZSWU-CF!`E}Mf=IaC@kcoz$7po|?sE_-(A z!fzKgU4prbv`zI%X#8Xo&&KQr1cLKM)3I!fa*lBE?`YT+G%w=QimJlzt4c7+WzEc7$07r9&bsG+fr8dLilNz@c7j zYnq;zk@TJ9Dtr7MY(3E#dg63qbM^Ez@+StB|5tqVzNQPZO$<|qadzeN=04qxQVucblf|hZ7@@P_c{m8K{Q;3c^ZfUNe z0Jh6Fsa1dnszWb7{*5MM$(jjax4^a+KIsmBJH)MFB-+XX-OMyF!U9!WgX&unsXv*M z4m7uDSbV$Ww}@C!leiJUJU^kc^N%ZKkqVyerXQvzb;V6!FzFsCxz4ggbC#?XbLL!=#U#)Y!&&`u05Lr9$Z&9-5ZA z$>uTrkaOd9SAV0zuBJ;6v^~?LxBTr*;1AFk{jnSKTRO`(kM9})8o5F^0P+Eaak$$T zl!x-_ivawASyh<*3c&~5y7(&JvDBVn~q=It}&~HZ1pf(`Ix_ zG3+Ee7Se3OVj~qYC-wWwa5}M}pGUrhfn>wDPZql{VNr}oC_6S(Dle#8e}5c9Hg*`P zfd!rFIQk<{FCGH$8|}JQP$7|7EXV>|g*`mgvC7}#(=t}H(`q2OIsBU|Ouu-elNgJ) zaWp^WRyIf1!Id#s`MP{xFPsm4ew4WH(|Y#mcJIse{_CmUkMYGm{KY={$sF$LTdV53 zA>ea$=%01__lC!Ji6(&i#aEPQ!3C?2?}jib@5*o-A=^m zqzA?RR}Z?_fjn6lx0^Q~WF_gDbJ3fYj##xKm~hUUlP)A|jEbPpol$jhf3=U_NR`S<>Y486l4*ZW#^8eT*neb(a&VKZEOoTLG9Tws=BtcHx*6~v)5J>p{ zs0v>V$+ILs4m^QpjOX?uNK7-@5t_)vfx72W^Xi}@R?bvtxTfqRk5lZH@ zLxb6nWFrL(!fO}Lh0<)yv$lU13Qi8yz8d&EOJLvHWVIlINwThI@VejRI^S%y1HM21 z!T$+}A%W3!G$_0>fR3}}a~x~5Dl|GQk6{lDyn&GV0brZbhWn5=e_gq^Uj(x}ivgAr z25zmk?O(&p?8K#|@`EJt5k%ae_FVsfbWhgl=D-+!1{3cVafPm{|2f)hT1~z8-a!~= z-HrSSm-WC)|D4OzXcGMJ%R>U5+pT+o#6< zH{I_Qf5DcTaWi0P#+qCkY3A@O2n|^0q>G3#hf9GDRfQxjS0jlrE^iVP-p7-$q&pHe zVA1d5ii&amN?3flrbQN)`ZE(-bR&+BtA=@5)=o9xs3fvI?A&Fb*`rex1-?}R$-L~# z;vpVG=;V6RYn`{qJh*WjTun5eXfUitmY4(>P|RK{q8C&!;Q zb}zC`R3Oybl`BfTNEa|YGL zP&C1Rr_KG(6$1Nz?_TTd4V2S=Gu|3{G+C&B?in>i8gcuNdbJl$29{RqgGiFL3qei@ zMyG0ic2Ab9R5VZ> zA7?rn)dXkKCSrlu-M3ho77a<>3U~7mk*Ydz4bYbgZBA2wA{UWbSl7JrI;7Y9JiAX{`=tGq8v3fKj4I8&@^UgXyVpB z3)&Ol|1$TFH#1BN{baBI{2D)rYE)@DzqnA+Sq`Ojy|VNa>^X8j z1l!~$4=Rk8)_UEu^`yqs9^&fxpw5~c{^mO!e)JG7003k-cApP;UuyzyJ?mj0(R_BJ z*k6Sa{K5&xIR$6>eX}(Ynrt8SiNs{bF{hUaJFlsb7QqH_%|t z=Xpe_4#H#aEaSAE*uip@aTA&n`D8#e>DiKn9_Tmu%EYgadW z;bSJ3o-F1$J|o)*V9qW%oQ&)~S8T6&-Pu=~Z0uV4F!lLl+2;zC@I_}5eG4%z*yx2qf z0q02DIGj@kLvt_djbt+Ran-;8SXDWE(Y@@0dj`h|@l7Wy15~P&)L3CUK>evK0St=k z+maA?H0LDG`F$3?X&+>vSLq=^j5^(1{RvuHv?RK#;b7gqR6WgjWpcZT8(D=!Q`t?` zV#O%QzG8H&kKDb%)-@}rmQErq zR=%>g0$Y~`1Yxry_>?b*B`cv`e?FxmGr28TaxSqQxn-qzWt=>IcS8POrQ`!Vhc5Y> zjV2Bi*sGliu_{ZLuc1b{{yg*#v3`Be=LZKzG;2gyo!9Sm>GM*Nyde{Lyro-Mz!~zW zf>=C2PBw*j1TzW<9DtfW44uuE+=-hrNRlk@L?)l>q&<>{CH?D?eL$;JX#Z{~)@BUS>`OH?s*^n(oY)XCCS9nzA* z7283zjxOh8I^a559DRDN!U)o3eXF7X{Iq;6^ii$JsIspjSu^X2Ax2_zZ9`lPcSDV> zAR?iXBoa{eN@-^RSNV$hHQlCuPuNv-VA?f6W53p%nZDV{UP!TlzgrjvmxWL1`FpUtyzni12D7p<8p{nqAaV6(53x zJ*9>E-V*t;$AiAvzn#e~tfxNjPY(inQ8@3gw2GZpBhjBy9i~(I6TV)-B{!w? zE1%{h(lau>Hga(oM2t=~M1f9u$ViP2S&YVRgr3H3NG@7kjc!)jm^PA1MXbFHNi#Ln8&}I+7JV{AAD#Bsv58*>AUKqyLbYDGN_ogi ztqR#3hAx}z#IqlJ=k1Y)!TdNsycvW&DJP;3!@Vz3+$sMh<|~8uUoaXMX}3@EhWkja zU3oSC0~cX&2fj1hAQ^TnS!Ag))l^!4LYrL6a%(GRvyG*5O$wh%hil^q(^=;5_-wBu zrUE%uVTdyq5iVT!yWO(P33;5M4`8C77okMQ$;h|$Q$S2;V=p=yiYNb%YEU(Oqz^s5DIeS zb?huyk(8W`m5y{kfKN0z8Cl1Ad|vQHygA%-V+mQni(nr{0y5BN)JYA4)ug4n+Ys~d zacjRF`28~;fCqd+zHcL1rlYe`B@c%?oK(q0JbwhIyc{v^x23x>o?_Q<}*b*~%(?Rxlh z677X{b}Z(Ws>#Fb{9?vmy30B6zZ#shS50(1>0H8|DvsL9v1)MeKa#Qo;kEq#H4=^O z&u^ebX5gSk30;?ElM2Si>99`69e|_yCKVVh8liEzu;MjLwiO&b4UQdwePQ#8<+%n5i3%vIDHc@f_l3 z9pm|%gx|p0AkeO`11_cZyuw#3_r0pNC;NDhe@yYdw8h(3n5oaCjZIfn4bqcwBGlDf z6UDK_-sXZ`5(%tbTRJB7F%Apps^pP6Lrka?7({mT`c{MFsru=P*7?RvZ)6c|TpJ+( zyI4t*$!a-}t+0Fo*{W32m?y2Viq9S47TxABV__XTT2c(sr0;<3YLAL+nZ4nE4ZF6Xh;6FtvM727Sa$4i_{9%1Vx?@NxyqMz_@mRl*E5E}4=b(t9 z*h9*kZex^R^k;+EFUIOJzlVt;S?hud$Xz}}eU3dr*q?p8)h!|DvN7In+B%(MI{SPn zG1f@XMPf19DsBE+Z&Th1z8w(D3>H(3uw0PawuNGYw`I(@!H(V@Woat~ZYoA>)~Din zV%!X@bM;+0bCMnrTp0t{Z{*eh&0wBuaqdXh$|Hm9TkxEwKRMl}zvt@(4fj8xVQ<~( z=&HFuZwSCwPr5@%z;7`a7pt%z7Gn=98Bu=OJwVW?Q78%kiF*$NrM9Z#g#v(RE4E#7 zfK}C5i(pvamDF<7=EG8WVHy$CX`hpv0u5LdLsP)CB9_%}arT7~@2*7wO?|>&0wV2& zn*Dy{?^b`w*>a*py3%%zOP1ig)}B^E?Ih1yiSu_M{_Vi8WFEf)9qXi)I1|aEuX_;&TpiiP)Fc6VBWJ~Au`S8hSP$~`iVZR@H&?Knw`8*;MaQ;A{ zP+`<5(5jl2S~=rA3%#3A^xn`XrM%uP$zY3@HhR0R8U(Uk*NG}v>zZ{0Xh-W}H>%y? z4TA92+6rZ@lmxQV&9vS8Ld$P|C;G;M<%Rv&h{yUkn6nDDi1n-hWfgMsL2(O(wk)`G z4?>1tCBWIfU;<_Zz)N@m7hY)15_^g58k$ij+UNbV=d|&nYuIHZYql8Wn;8~GzaWA} zzgn?-M+qAnc7JB^A))@a+4z6kmP1H8xo03Cpu3+YMfvC3(w@=I)Ws$3rj-Vx^%e|7 z3>b|2=Z}I`E(j0~V1)lHGpS>Cw11ktEu<7fWc)M(Vi4%G+(YdX@a>RA!Ef=8MK< z8p-8^8P-!z+SvfRQYz7gl%be#tIx10Y2KYF1C>rUp3L9U@B4rH`q)CteHN`?w^72p z22b{*qM};ilroXsveZ>A?v|Ql@1u#c_4rNF$2jA-2VKUSUTb`{j;letH~wU-D=*IB zhIvgU8@oG+EXf_%q)nArITFt-b=cu^ddRRt8JCrQEX@I8cr+?Yid$!v==Bz_L+7%U z)_vP&H|@)$b4}YaZjvh8s>O+}WgVt#Oidm4^WLxr2gcap;>1Bm5(_^x<`*_+80Jc3 zTiVOovWkjnEB&5g&@gNr^RqBS8SwE+ilh38lS)jRUa4Ebo`e&KNitiM%*%>oYM=C7 zn)9_OFZ2LU?3}+iJX#_;s8OfA*5Mor-$IRoXB1`iZ**m|6FPM%A%0R}VQ{oqXs5%w<}JP*rS?C{ZK8an52P*%U~GTs zOBJe<)5vw2-r1_`{C`}%b8sbl)HNJVCbn(cwr$(CabjC1nAo;6$s`low(U%u`Eu`l z>V2Q@tvY{nb@%CCS6BDO+Iy{SG;WG}noGwXb*^Qy(iuG^R|p7oL2=Dc+(TF@_Ra_; z;fF*_R%)RA*)>-^(q?U~@oqV-zV4QUZDKXeR1L*5;-+zGLU&GCO3bSTyV3VK{idbw zw$&g8h@3ad5dIGPjMQ{sSk$|QBv43b&-HT8=84Za!Z34!EY9RJ#1iZN>(dDXAjcuB zIMXWG5G;O|ya_OGk5W9LM$YSugck|L3uS>TK-udF$+%{4MV@XH;69YireR^tkP*ySyUE;Pp&IDW>Y;Cc6Uh_s0cpLFT_%gd) zpNh3d%D3fuSu})X0B@7s`~{Vix;2&Xsz$gVsYGF;eT^g3W@cxCPwKw}_jocMSOSWJ z18|gQfbGXl>SsU(`{9xjGWvoMMv3*$0M3Q-TGX@fR}jy7+#ZsI$%BK#J(ugQ4J!%m zVWdvH9ogZ@a_;WBAXCl=>wq zo_*OH3ou}iEExf?euxtvi1>s7-^O)?0Rj52u{cu=WaWhd0ofr0CS5aY06siX%`pN> z<+C~D`Y>n)8nx=j)0>H@2*uB!)L;y+7|pE>+dv5VEtw}w ziNgixhL}M3zx3AHaYP{`0F?3!x2P#$=%S#2BjqH_bgdP-F}m8@`7a7kj^udv6(qsW zk)q!EXWE$F+YMPdgQIoLLa(&Cx`y_?J&KrR?e4xkc?^K%Z35LN_8>jv><{y`{$}(6 z*3LARFnh+bqp|z@!VDS4brSttLiL%?NQbjOrZ5PI9D0j4eLeRu0Fr$T1FDteS$SrC zgDlj+rnMi!to+!tW!T3pg;P&kW&;KbvM{;jS$Yc--NGEMW)wgj6j^TiHYyd@&I)v( zDyoSAE$$z)Ri?Qqxg}jEiBy<$Z4|MB$4V^ zV$Qn4+dDlL#`G|{fO&aHN*f^i@mlJ^!Op@E*4e_Oo-!SGSb-e1Ra1?NgL2=6(NVOA zyexKBM>#@Lv?Fe1hibc9UkFj*eFT_TPVH0+>2%t=#FnPC1&B3`AEs z*jz03Pt3(Dis|cT_s`8W*_m4%T-0$Xmj-LLrusnK&RP&4jfqrkVLI*3CC*~=c`%zz zhiX`UM%)#*S~C4Agt`h|uv%Q`>>tW5&z+PCoEc?L_y(P)_vn@(D(m3l&pc{H{>G`h z?=XzSMBy?i1Yl083XhK3Fl)b5Z)r-uRA1DY|5I6M-ajB5{?i)n>W|YL%Uz+AP8aSI z=Vunb=dEr1i53wDe5xs{Mn?u}3J+(3?_WpG_Pi*G+w^F_u048)SK^=mWLG6}2F2w( zMhjV?*Pf38n%!oh0KI@JgOVfh-G{;YMjh}~UZ*pU9`LR^Xz(sP$m^v!=t*We{GNR=SroIEH=Lon=&PPpGoqrz2f5<3Wr;j1mTmq0VehhxbcgbJhJ zpk`XVhw3-RL}1L^{~X_lW$Z|L;TZ}VN0p(KE!)BCgByNjhGyregb3qN9pExWrA0H~1fEhu{g)cqN$n5)X-8Ww+LgNC%B^9=l|d z1_FLzmz+fH#Ujj<4>Et`TfVyL;1}3_aXs?cU%veA91BsLc+3!pe-jIX%e? z*brS7Kj~a`g5+;9tYvwFtKgG;W?0&}xO0wwQQN++%z(AAwK(y0Lz=M1?mVAFnBQdD zG_QcO-ZQ|dHG0Dg$hjmr3Y3`55T$fE&^DqyYQTqGf&N)~gPMEHv8L!yV_%}~0I1D?ML-lt2hfXX5ebUl>5*)65O-t1?6xP|L^k+ny1f8cNWWMv%Q5P%P9T z^=d*N!uR{9B=#!2Z&j~ef(J;$1Gso~iF4lYOX-VeFPZGqSnPK->muezYB&Utukx?^ zXPlC!x=oYB_*D9)yv*Fqw@C`yPH+6igPcdyAB-PfBNXZYCl5Xm@iJx<*7^sz%vXqf z$53DWkGc6^&zol;!?s`>S3b_oWoHWIFdpox4){S+2^*m&#>sC>wk ztBnUjDYVFOJ5`#G1*v}6IhLy#!lj(a7EG~Zo(G1p1RaZ%=p6CXCIIDo0ooXLVu+VSkV7O{=`5hO>$;{tSYV~(x*fnx(TSZ zL@Ow-BKZ<{L&geO!*^#SiV+NtM5baXnzTimL=8Gxk~pdnZUD)$o{NXNN+_PnD7*TL zC^y$Bb`<2e$7Pd zRd`R5XE@p4-$?}lvF00#^@Y2x;fdwzmjiU7tyY}yKv-w91^KqKn{;FLstuMikqg|a z=2Y+Vz_vRVR>^qSR@)fyaKvPAZw#2Q1qx9i8#q((^+QA+U-YdtCR&8cfoS=0H;&14 z`o}E`p+>WI7mVtG-t(JQts|7m!F)_TAQ4{E=8;(k%l9(CKuo_%*1qqNaS%$2yrc@f zOB=-*T=g#|5y`3zvaZGt{-g!z>1Ez42hE?{PWr9p#zP2l!?jt#o}#J@cGG?J++41zwBR69+18p&*Ot&I9D9xl;g&%%c z&84cBNBCA%!zE?77gm3pM^XHzV5cqWr9dUe`kOtLCq8m@53ULZpTaiX)^Rkj!^b=( z*~QDmEgzQ+c8`;j)e};)bJ7Pj?MAJ;>xeCiQ{t5Yv8R!RL_lAcj=F+!({HvsTYX(T z1UtmPAA#;Vj#6xX1*Z7q_~v$2Ud5)Ux^UxlrgvBt21#MZPPf%eTX9PFWWJ1e^xizi zyTzOYMT%d=FRIMK(Lc*vqqk+2O??ABF88ICjZB z#Qajzvu;P4^$&}-x4)SE60CjxvZ_xP0^h7KvUU*q#J>AXwm`X z`s5Wc?llpM`BcOmuHop21i!ZvTID?qk(cWL=xg$WjAsSn{LSSN82k}`zHME2cQ5|t zCsK>cZMbj0onZ$17}-uPlWwV?IuIY^I3gbAJIIK@yH7MFaDj82H;DAGMjMUhVyXx` ztxn|^PBa!soRYIlKt8MJ^D+`ZLmUhdG-kj>%n`cZZRIHV2)jwhkoybv2REFeSrAzO zARM|2@ms*|1Ia}2@6)uYdl^f_6SLSk!tO%3kGo#DJRDBO5QF+`pa0zQtXfCfh)vcr z#SOut1@1Z#9{yKuXkGg+WmN+@*7?KQ*|y5YZi^8d9H|JySCV_yzk zgk7P(?29QF5Y&lIDfP z;Qw2zV3Ah$f0pfa2NMPV*Y<&C0U;~C?kxc*EDwPN+;InkZ4*kvTmk*}l1rx?3dArV zAbe;*W&qh&`CEek5;ze6h6u2B!(BxG97<+yWKoEX%qj6GgaAu7(`62VaFS}s5t1|E z6^oyzv&R}Y^+nM}nWG{TTJ36^fn1|NlY&Nqp`H{uK%kLR?OB`0(k#;57I_)WKQ zHmF(rx021*%(CwL*6HtMCt&PV`VqnECgtR^xlbM2TS-3^wk}_!lRfE>kg4 z`;wdlO1SxDeC$b7UwcygKyru#9gB%}#fS~ZhXju)#UMbqMm;E9hO)+t4*Pmm@>B$O zv5j&*z8I`THhn^DGT#61-S~v2+BY#|#0}Pt6CxM5JyFd~y|Q7_{!@d6apFt7DHnS*V$@VGb8&w_U-zul z)2Iu%X-!OCTEXl`lX;$I*=?nk3tLH9oWK}6d(Sl=w3d0XraE{!Fqx-M*kj1^D}jzaR%x@3K^@-J*Ue z<;vnL8P$Y4SR4ErpX)9xfTCr8Fv$rYZ!oxlPagHQDhZSO{e@rc9zTag8`?ah8!X<~ zc8>w9F!Bc^q=Ka+$}l+z4oGVlm_xso6J2pT)H;y%ytI zcT!s#%BO7(W=|RWLfL`-@TK{P@@fcWDXXbJYO3rq`DzXbZs*`(wHSW`nYnUE$n;fm zcoYv?dVaLs+AD5I!ReCXeNS6E9hMJMfGkslOP2rY{IDpcT^onml%d=wS*H%msKNzE z%JpPW^ILr9h1Fa^DA5;2@!23A`Bpp27lOUsz>vvI-vjmYbW(zcno&#hMRY3DN(IRr zZaCFb)J@m7xhclP^?nUnX84W-Qu##>i=#UJy-T3Z*`_bQ`=zOr>S$t4-8#29l@n$v zJ6Wa7Nwi&HJw2qNz*$!E1}!0yrY;kp->N9w&9ZH+1@dB~?!;VbT#LpZ05vU6so`Re zK8&8x_d-tY3h!F%7Q8o zV!(rA&KcpBnnEfZHykKgq9`QhXF*$G7uXf6>2FnlGe_e1MMfysN3jy>&D(B=84g;+ zx1kjn2;+zO|1>0RJmXhNFnT5a8@pG0l*kfaijZda7HJJ$dD;|Da5PuwwS4D+cfFS) zT%N8jpy%xO4Mw$aa(VBD1`Zs%MNK159O&39x+1vyGAnWr3|GsW zW?bf_{MNggoZJQXAj`Hc_{72ynM}jO8}2sEAhh{|HWZXtgyn+WHH^0IB1G%t-RdkG*jz^CV0A&h8<{ggK?Z^CH@AXMX-G*v>F@yrpo7B_)-DD7v zu;U|;#OZV_aWO3Kg~WOPG*0hPE%HL8Br!r!M35cNf?}dRGsnI%bkTkxH5Gi$&*RGM zOY^)Kyn0PiZt@GkVM4lO%Fs1A*Sf&TU$kbOJ-q2lGUcl6o^hi+=L4T3*!8|p9Ae(C zpZ1Rwm7#G$=mUcy>?bO!i}xS34!cyij&vUzem(fD7^n91DB_pj zzRX2T4m47nF%R?vhrfIom)~^bNgEsbdbK*0wC;2nqwJ|ht=bc(UPTyihfuZ8EhucG z%q$|Y1@^z|xN=2!^MhO$zdKMA1@ZHp8XP?-gugo+JrNvv65e+K@;EOAFyh~^( z=TQy)6=VLS{P8?~;u23|uR#~6skQ5@)hC#Zg)WnSX0O&0QsaPgFr4r^;k&Q167_?i znpya;fnVbRf+kNn0PV$qrikzF4%wmQIE^Pz9YIbbprP7DQG3NwF8o7A`w%(9JaziV z#Y;=$*!sBFl<}z9bRAHAOPTN9T){X;{?OHxtIp!{yQFO0$kTW&78L8I!YwY{rQ=;; zud<1!$1FqdBYxMDlI+1Wq9FLTfm$a*cwTs&t{ulMfbbduu<6w9MrbqWnR00u-SGyp zQ5z!OINeltEx7WT0{LM$aEmee=}6ZPka}pd!wc%JBaxq$3_esWu!|&jVsEf(91p2F z(`QSHz?2<$Ccg~)ISIN>Ce+W7c+Xh59WwSB5}Zsc3pIQ^&Sl8Kg}t*5H^f`r<=1^f zaE1*8iRw`TI0HfKXu#}9guL(BuFNic$w1WAl=L(E3I<(rpxSaOT_(W}h6;~&5HVwZ zL*gO|rJNSxF>G)d5WKrUqBE+Mz_^dGO=woi5^M<}iv^1Rr8(X;pRK-MYG$SC!t0f# zqQ6ro9@RszPx0pN4Fyw=S7tN>R^fwhdgwj(zX1uAhs7nCReSK>X*U=!dkLm^qH!#z z1bqcQ>L&~KptZPY>JuqM&9p-IZS6po_li6Kn}7d>*#B=p`TqvVZP0PhCt&|Ys<|o9 zi=f~CiD|LHS5SezY0$C&RorFt&m|Nc;<`A}kV`Dnko|U%mGpg06{pMZ#LMkqs=pMZ zBS#xMQEclp#?Hos&M52&>{EJlpOQRZrPhk2hdp53vNB)WJW2sd06U+n#m4UqCQ?S> zZ*wnG`(4KyO@Cju1wWy8AU;w0khvzQLO6?{(9t=w1zi6Q^i@X#I+zHeh+@4P01Uy7 zdm32V(L3JVL<&9lL!KmF$czX)<5WbbY??kbzCZ7d~ z&q>jQl1?Ck$k}-?jsXfHKCSDE)(1uwb#h%Zn#}-(fEU9lY z4{q_p-C;LX?=-+wG+`g#M`3SH=}bst6%R>GI2p*G(O_`OcK${kJ!SE93o$I-2hBP6 z4>qjDc!4OL=}2ChfODPNnPuU|Ah& z$oH4`GsGnc5?j|)WZ?-8tx7H8Z%t$mGc?n<(m6WYMH2M|@(?UiJU@Oo)MrJ$a`$4>VI854_68{fx}vDac<1rUsf zW@LXb2B-~o;cWdb!kOApjs}=EAlCm-$ToM!lyaZVT%L(hh51vb#H80yom5DjaR(!t z@lg}LFIE1W6TZEVn7W!nv$cE!`dPk6K%HHabdx>M7c$h?_&aW(j^=XELlXoCo1F@SytO#{BN7m{m;kCBVdJ=#bPp~mkA8=a> zdC?*3{wFNmR6xMBz=;k3Uqb>1S`}f%^&NN zb;Jxy!?=+5YaYlzbc@$QK!QHZcftovOL;4a%M1oav-Rz-e|G{kbmx{rlA^MT{tJIb znM|{5Ek%$E)M%*;v{mmDhwI#N1(;D174JhuHgys3knm#@$?4dvkUtSmUnh1W_>enC zx!EUmmQr&RREcuU`Mt?s@DORVZ6jY?Wkw3LhyHxhEZ<-VgNo{xU5Sv3%S4!#gF>dB zc#usc84!zB>Y4kC6wVKWmJr;^>y8%ttm2e8wH&-ISPO1=3kr4X9^1L23Sf9C@NxpZ ze0-NjIvT}+C!T)CG9S{p5AxZrdwEsxAAl5;ul_DtW6*kyy2_x!Rp0NXMv)_Inyxi% zOnJv{*XCo_Hiq`FvZXLi={H|pBrxgl{Q3fKcys0BG>XY5JfYsF>X1l(D}K?N@`46M z;gde&Z3q?&vD!pa9tXQlnhZMMFR-O^h4g#<*M1!ec-_ZCKUO zSMM2OOd|p=qlCPfGlI#=Qx4uC6-`@EjbV?tZ z=Y==y3tZwO`-cxe(ngdA-3|92F174T!1F&~R97Jw&HtcPT#aDj@c*3fT8XDD0x)*huHzbk|}*At?`g&7Eg<- zi65?4QvrW(PhkUb%dT?mp39(#1xD-Pq?k0QCDP@VCFA4)WjOU}L|aOOKBA@L7r_nn^x6K%JszF#V2jrG-d?6zQe;_O#KOQ(Cbv_#{UafKLw47};?U!KDD74w$;08b)3A9=w15 zqEs#fuhQB~`jz(C#*(*=5F5Wrr<9R?e< zDHf%ea_zD*M8gsA?=Z^$^FICmI6VKnX=>FrFIkWvAdd(@<2h#HHliYeNl55_LQ3T# z)F({fOcjAJK=sN0?}y#4r7WVMkti2D7E>}JSh0M2Tw+5M9qQ}RNf%Y5G4xXU5>>X&fove|Pe9N?(H_CY%AiyA7ZFOgR; z-C2?TsrTnoBz$`G#vA)nCA>Fz7nT4JU`O=N=^_3tNb?}S`j!s2{b$DWBoE@XR%DyDY;Pvuch`oX3(d4I3Q(r z+x!_3c`u^mA&=k{`2%=!v#;_R*_FT#%Y4b^H-CWT%$LpmyFo4eR%RXLT zvxDy8yXAHd<;ZUjcJk9rY@p9Z(?cz=j&)eV2^?op!_#HN_UT=O6yaLkR$zti`w=KS zZBMKWHQV=vBqJ+$yfh!&n#l z1V$Fvql{qzb0mP$Aurj?nVtJKU#jicYWiJQGE6^4 zPYD>4hS618+!*}P7%7kuiIa(jwaIGld)&>n0dIHc!^4s)haJF=K7NuvGE8)7#P1&Lb@O&`SL!G2Fo%K>=D;eJ~aKje$$*SmJV!2!* zSTWdO(q|qH^}1BUIhJU2d!L`4)>NhNQG~NZ)#0laf6MeZq#E9`y45wzR^H%fZW@Zm z%RA@s24b+v$)}O2^6;fsa9R8?r@_KyfGdM4>uij}sKsM0Y*5XaMOJkR)BIHc0JAh* zi!As(jBWruuLVXZV-4=?@ARGHnn5pFHQ@0Wjr@4)jEPxOkjUiViI!x4!^96 z0UqgHW%G&>%a0gzwLnA$|0xx4KyiOJyEB6)Z)((V(>?z^QKlpzfBK#7{_O!aK<0cI zC7d8m^kva!twD+_dj$uU*uocLKsCIv^an5Lnnr|RwFCi#Y)}*FX!8-5sCfFg!X%c= zpW|9C%K@~U*`}Hhr#?tamCm5+oWBwEIk!gzYA#xBKNYYWDCBI9_z*Fm0Lc|8K7tQ+ z7;wyRF5@46{~~ku&QSA~Sd-Q`f3OWuVUK7XmO4n3u1%oUtK;_vSkgHxC+vOeJB%k| zt;kp^m8GbikUp3!GzwcT?@nK))~mbo*~(Y>!C9G_9}7pQ(GkR1!6wGTVU=NR8U$9w zCKN)WZ&ueYkcQgz&=tS@Dh$K;iYb@rdzS3>@;~S;7?_D9ym-}g2yp!-UWO&=(Q<377P2$gyxRw)0`Oea`)vq@9NRnypqou+t?9X4I*7k@1rS?egA zYiDIRIEe4SE<)di<-=5uUi(Ap%sR4_(uJ@ ziFA&)c60!6Nn2PDlo5)NaiAH@A4RT@78RV-f76z2bWA#&HxR5j&8X$Q&Mcnn%Fgy_<-qB0DfdAe72@<>jeDkurU?fyR z4O7ripou$SGO3gDW5jO-Ec0B8XaiDGI;IFO7x}*0B4@tsycFwM`c1BYm%F)@nDe!%L*3w2 zbk=PI4n?#7oz7t-dmQ83y394fOhV%3W{D`;1e5pSuPT)o+r2cWIjFRuqT-)7i(XnT z86(~%MX0zRJ)3}4$E@XP3M=4SOA2;shVC6-mc`^DH=~}WqU|#kU;E-Uq8N$)-ob9I zk)bWq%HsKZw+hg6nGoSZQpm0O8)*l%&9{4PMG&gSb0tqnO)8c{cK&nxlQy%Jo=bY7 zH?B4f1yf$U_Dd(Cxy@zl6Nl?HM8Ua0AL({7X?ET5gnOg|4MlpW?blAt5%0-chAk zL<`ZWRGN&`RtawU(@I;Ao-HRP&P20y7*cDg%kMB`Vu&B}ciMf7tsPrzUa&3hVNdQC z5rQn6x8nd>NEGHr*Ug5xGD*d4o)-by9ptTwlKENTDzlMmDDF1c=O_~2^=ft4Nl!Q1 z!e07ZX+&oGY)#Z1D)*t*-#o+}nz9*DBZOjZ6Mx=T7I1%E3I}C~Avt6$u@$rqu~4Uy z6&(iR&6>_Lq5b-7^)8ujwSo%+E4UVpSTg=x!V3YMJ;_5qc(L)9sG&dHbJP^zo^l~m zH_L751g6hIN2AP3{}OM;z1N!ZgZ;d@V>jkYysb$eX8&`|vaGO+cy0HqMN3u*W&gVc zR0S3<*X5)7Jr{bMYJ+2}o~Cu5>(S3H=FUk}-->7*YI74NN2M${5>%8iyS+&x#hcur zHmxauPn}9soep-{oI%|+Oq#c*9&y_a1BdhS99Rl0=B5k*j0(BC3VfIxk2}V<2!$H@ zBq0)6lD{?DHMsVlGon)8!xh;+1#c}`}6 z!cTMSa?*-3{%(R~z997x(OdaBF#KNlS2wg;PP-#@Cih=9-hRhXC5rls7p&%s7RvxY zb_RAD?c$1%TsJD~5N{u{WX#B5YmC-+S_hvUyk{}%_$Fa+&yP1d#Gd)$4qN+wwdGp> z;E)6IOT_G;kMCl--wZ$ANHNkafW6s=6g*E{olbl$oUDjLBEsGa z&GvtWXEI?y3^hWp4J<(0`(#R3^MVl0k1t@^;1rcA>&!gR1zI@GsEDne9>o-EI0`%2 zvuiSZ?6C(dDX)Et6lJ=Q$!I0mvkzo%>rdwFiwAXb4cr19CGhAsAzygRR;v9noin?< zQoZJ-p-^&}GFwD#BG*14>9h}5chTnq%krCg`A=Fbw=7a*SW}8Ip;JS)ZaQq9amkCi zIP&S3+F2WXGUh$bYezmqS)yx;7;f`a>R$eur}QfU=j19k9o)W^9kUwZ9f$(1F+ZI6 zT{BxXudSL9WOLP8I#e(rg8c2B89;U8Tjb%g zV2@5_1Kwtj%t*lb_CYlO!r||V znC)Q~4Ql3vz!^~&G^M%Xt;Y4#?^Ve}Xo0K{5oaa7bi~1QJ}L(A!FzAzuZOZqg|ISK zITGDG3{=c#rYWo9_NlB_hIUIh9TdOos*HPxjcZ=LZGrEh^I0kI;q-ej;W%QP-#S~O zlONBioLun-g}xWp5eE>KQTR|tcr=+7S*%MSW^i^uPY57fH)eQ^D9~sBEQN(&oEH!Z z2&a3KwChDB>K4emAq&_y z?4|21SI5d2ma8>631^;@{F4=@FRF7*AX$C>g!qf`VCDz5Eo-rje23k#yERwdp8aC z`>#{gNG>B207}%Yc>34%w{Z0=CTp02(vGl%p>y)D_BIeq(D?#SXo()xe;F3VJ-BmS zaw?(qgk9~fQ|PV>Ii=B8wm&gM^yw>HZkT(*f_*T2asl)r|Io;&y1I!xm3leGnY~l6C4;M1YJ!? ziUEMpex@$Xxl5W0oz~Y+SqKxwKq{XS=6BEyZOXigk{JLu9d4$r=xIm#OCI3-SB_lb zys_68IpgQ?ro$40@6tK|KdSyZ(!ge|8-Bc zLGGIuet||RDGeL{HV~n-RaTRaK>dgPC~Xr2L;l-G$;NrfME|AeVh#%oN`oc=PV3P@ z0rSrZ;DDS;Jn(HKhXj(}{soEXgs0>NU_n5Dz}Xf=l9U^$@Bf@e0}z5hM|U4FL1LQ= zM3xc~5miFf9?l(sYoRI*n-Ep4}@Z(YOyO0{TIkw;PON$Bb(EkSZFXy{&= zV9H`9l^kzzvSt#a%3WyoGCpk{_Wu3(^aFtF_vaU>E~p%eIOxP6Jx>q15CR7T6K>n9 zvogr`kf)m;CnAqS3Sjawz)z7OKrk?_U36-L+ShaR$btJ}&N;!8e&es3-Q7Nv%cTMS zllP|d^c(0q(i0RDjNsjV5D39Qg897?2;x2^4~H!KX28RUV;y0_$Tfx_L7ERiN=Qur z9|P3--3>(U#BWtE9>n=uIfB;xvc{Lp_&g{k!{LsFL^a1xit z1NIpQXS%A!0WCwtQgWO$ecvs;Ei37$9-Y=PnnPFquqJ+f%E2wnK8;hWAx)!N2i`*b zz{jpKu*_kI1Ynb8A!%18BLu6@WDF{}sE0vS-bAY5FS)KlyJW=x?^;^Vux1vHnZc|Z zkD8J-Z!=s$b~SK8At|KyE2t!XXH&#|d@{|#dnNX(u|bJsj-}=AMChmL0xt9VLZ^CzkpsuX(-(a= zt@`Etcb*=~%ZCd85q998)UL2ROS4|3va#w03YT7J`VgGu*?uvx=trfH zZR_%1K;R!mTNui_R7$dWwS3|=WZOoIb(WLZspK7nBpFRcJ&uk`G< z3_rMc+`mbal01LLu7~q(*0}8|i)h%noB`l(2cj*ma%GFZ0#=_E#n$C`O?E&85?i7d z2bky`-DikMe%wu4+woZjkZo9hCw9S_hTlcT0Z$|D?zjA||BQVMpCsEXCUgKDdxDt_ zZ~R?;C$dL#Dspf_!CK6(aD;P!N%@!Ppp$+qlUM;)zYyt^Vxh~i2HrS&UpZ-=nx#{4Obv#t&DaM&JV-=+fjjoQ+ifICiJIL1Txmwn46?- z&+ubQtZ1V}_#gRT3JUeH~^Ne=B|@w$}dM#I_(Xw7JmxDIO7GlJ}iqhk5`Y@tUNUTCrQ3J=kGuayZDKAIPjIL5IOS$FqiOS~{7!FlW?~4J|kZw)_h- zjWc%LR!>E{<87f1Cz**SH5Zk&lB_r9ZZ!#8#riRLYOiif=W=9Q=`7`xjC%Wao)fcO zu((;Nrm+PhW=|NQaTKbly#m1YHLs=X4!c7`w`=}%D5YO1Jy}W~a#f z#gi)!izwN1)Wdo+emZ&iSH=4wAEppOv{|!xozyqIfR7)hNgzS4T$OuN5r&4h84i(9 zi}9|j-xe{>jiO7p=F@%9v4?v_|ax9oB|@ih{##3u?g*j6JE}^XZ~CV|?Ikt!!=GPm@IH+1VD~aUNwoF& z;p?ZtZA?mrdOEGx-uz57>9B=qW>%`sPj$+>qio6j`#X%D8i+nsvdkdifFoxo3h)H(bL+Or? z)h?+~-%vw7Xa9biIxYM;K9`Fo>+5wkcu*XOzDF`R=XIgx=k1kZ=|zZ&;|PHlx8cvF zAUO~Bc{$8%=g#&fFjQz%pC5QM7#*@s-oe_Tos3I+B_8#wD661md`x2O09 z?vPEjbqnSvRGzm`P-|M?By9jz8m^PrsK%wjfn(po;AX}Wq`VXvKg6tVqdVyo4YUuX z`m0P#jNgMLCZMnJl1rdG4gDpR^xJ8g<)6YaQPgF2nw#pHnI&)NyVm?xQ17|JsXaGq z!o8GdQP6hymv($ot1MF|DCAyL%N>S^Uz>2e`eL61S*sLTU)@L3Zz>~s@_Ic|q7MZq zi!k0%Pp_$&wPnJFGVeo3{G-AM?2P@uk)Vjz>0#hY8G!o0T+6j&kD8h?$P*ll^p7!~ zaBi*XCFrx89>sugXLjwr3%+W5VX2!QXh)zOEVsyo_>_Scyp47mkxfptgP}B1OA7GT zH9$4F{PbrAv3*ZMwZViMMlO+EDsNIwGs+=Xn!sqBOOAG9nV)3c!f0Xc)LxXZqjnOm zLI}=BAwV!`t2Sg)_~Pc-ha1z*MsOKcc&8lT+K5bA)c~I}RAK~EYax&d% zc>rS*$m$@(Jq+nmRp<%(M(i>+taZxl z^5U*iM03$0(;s8HsO)6x)$;pO9-)R!K*PX4nIDSjz(%F-EGa`Dd%k3S&9E1M-xS=EOP#rTl}9W zALoDP7XPvvHjP;@5dKRao*Y>ap#I}LWNwmggCYM@+Y56ABolvWF^GI+9r8e73ve3X z%?u|2fQS>tp)&UE!od^td(*4mwAjcmXZP zowYM}U8b8<*F3?FQF<>GpCZe4o2xTNw2B?XE_ZIu32*Gnw^2T=#G@ug>#SsSOHZgN z<;1O6@p6H193Sx5ZGst0c)BQ^w{{wJi~AW=cG%q%bipc4IY+V%JJ{V;(&mo)Oy(tk z&W2#qm0Cw`+$KFWebqjA^!*GaX8_zV_M9Z|<$d6r$e{d&36{MX%jELoFO1${X<>yy zG6&lBqCJKy=?7}aC-np7n*992NGKW$B@?fkyvESoGht5^7aE71jK><)QUlZt)0361 zNf_^>G+q^(s1WthX2P@ zXyZ>`+j}JbJveE>5b5HR{0sI95IHI41L`A`ER3IBVQGYaqF2eyv;VDQ0<$Ke=FXmn zM`3Bg5i;K>{o_x5u4L3u_ynei9!d6V)CfWWTqOar_XED{^=v1+DScNENAeMZ7O{qJ zqzTvWAh2+hpFj3={e@~!6*N$6WkeCVW8kbqhU5&gn!E-1$+QTO1=~05J*&(L8Q8X zTol#XqOaw$J|A*e&$&hPlj;i&HR>n92k;l={)MjZtI)^H)}fjCky);}xw#wQg>UTz1^f`Ho%m%|QLsVrAfMDI4+zM_9mP&L1$^$nt zZC7$0QI{&o>)Net15xC_Yq$GmqehAMx%y-v-<@=UyM>OcTErHtJMN4Vd|20yl-S!i z5exriPF)*rI!h5~$ql;!K4VD1=1r;dg1}wmA=Dk@E-5pAI7k=&bq$~6H-4nRG%7KyTDU+VR>V>={DmRc$4fr~cB~_)DfJS=g_SDxh=@wr?dBl3U2N-xt=E zR-1AsD}>mLc4L4Jl?FqnN=u`^J!HMr*r>lxccTy{nwTUsK`$D|!E>kk3>QXzR_x2Q z>qT|+R4jq#7(cQk^fT6w3-pMVi`8H4!R!47sc%^hcI35wL&mb**5mLrxtlYt zrk5KE8|T_)@l?^6D?|`Tnq(L5xk;lLb&>E+P) zts-54g^D7h4abm@kae*?=-Mi24ksx+S!Gtg>5t!NYP}{P#&J1(Z7whWBKQ7h&Hdjy zp`sI0ofEue8J-{&{9nv8X%qUP0Vp6K63l;uS=mzkL9mch`3*QB0bLrA+S=yWfAOF= zB{_kdIWKN5pSq~B zHwnWRTU5$LPI>;Bc;7tz0?Or!o#O4@9)+g_S9*Kl@$A0W-MaS%Y1W60Dybs zI_(0nfl*1}G8lpl0ZU+-&Pl8!HmoKClqRSZ$VZH31B@b33V9g@LM)iWjCOrs>hSyugL}d9=9_Sk$G&y+2@K%x3yp2qrCRciVvS|40LWwAkR}2mhUaa3*1))P zc5fw7Z6B3+atscOsU2so;l~7B`NL4ollRo89z3PmIZt$Ufx0@nSt+G$XJ>(__D~^( zP?WKsBJ=GS0VpQ>14VLp2-LVgw1W&gXv@!vd^M6liKvX8QKPo`sw#}194!T)-^EE? z`?g5nJ(_CNBXCoR=}4>KB1x2r8dBH@qbO24xU?MLWo2*B zCJw4V*!JCuMj*&e%LWc*0aW^!Y5aq`DXd0LE0ogvRQGODV_2F|xNa@3ng=BolnX-H za;F+9N(+ZALtxsZ@r~5*WR=0u_k?~pXjG)Hs#H6&-08w@B3whgDahgo9}ZdIqHh(@ z*6k(n92zG5IUsMS9a!b^q}hsHhuotLVj ze@BQMWyXo|bB}JZ@uzQ3h&tl7y%N4;Igp=|za zIk4=dW&~CdYNU+bz?d-sd+!`MiuIV^^2B0-0XS~Jf$4n|1yIj@q0Q4}m}4=)Wp>AH zZqN}rQjAwA;Ui|_+E=uBx~U06E%A@RCU$s&GRT}^Eryppd~|!jbq;Rf@wd@#=sHPv z)?W}H%}XF8ZRdD``tLJv<4bEY2!Uwm$b!#B7{u)g&YT&MPWx7n`CHv~{D$4V@P`vXY}~cv+_^s7r24 zfwzZgc>D-=!d{REOZ*A;DR}&-^Kkf4Z=64bdVbzg$Ld4Bu=@2I+Iy82rJ!3#n^ud? zN@#>8(yo-8)3xMMQaBIz4v3+H_(g6*173Lb_HH480VXxA{`}A{%;%pWtN_!=?Hl)s zv+#jbkQ7e>s7fUxLvr;3W8+;YI;L+9z>l!c^tGA>H@`b5i}2wC{u(^$czU?Q+$2|P zSMsnsnK6v8Z=oI<1a{gOUWE6hhs*pxfn;*zUX@H`-d>V%QZy9w5Olq1R+SuZKtwY| zTsc@iWdy zxv67Ky9<|EU8M~vjD*d}OU7(kWa4-r*QDDM*gS0|3v7A zDK;*vdYl|+i{c=~-m6|qCBJVt_sP|vLede2Y7kL}@GWFeIgQR!oQT#~wK}(M3a#rk zjm}<5-$%TyX|*0MPNE4@o^7|rdygWQ@6Ui#dd}uzgVcETvfTqrlR{TXK*3RnwmG-* zNsYL#SI7AJ^-VjEi|slJK%LwYphHSE))*I(l}OEet#-Z$F1-@a!8LD%R9eu2?`Cy< z4k;qKrYoy2&-QyE<{|%P2I<{t1+$HnvQkL|L)eX6putVvs6f>_Af3Gslj8t9SmX&dm?*C;_395NFq@<(!1J}zs&89b1SkY*dvSK-)WW6Kp+aJ~}c* zeeGH@0X`lcUwb{gnF?s0jUc?+{O=So%@q*d$LGA5HLnb0%Piq(z{Xs|-JX!{6H6;5 zYcGVA>;o8dri9PW$BO-em9S6$r4G>`vo7uO)Z#~62V(8>wi&4GlBsM3_XMPe2U=p+ z<2Y4ZVYhsG2wg)txQBW9Ao2WEp=SLwHtecBV0NBgB-~=02U?% zP2;~A9_5JKVft8>nv(T}a%>?yleihHuBkG6;c<1f?~N(TmeXCGHP|EJyTEZu6+Ij= zGQCRcS7kyw>J6mfL>V_!a*e0bK~|m5IXjE>>;X%;LB%<_ea+aN$^FJ7rFOmR6I{&v zL6=+usOO}nzxHft z$Cj7s;6`{1k9L3J4a$DY@j;%&6LZrVy$1Meq;XF0%5FKUc=JF}V9Rc4j2o=E;w^1l zmL;~hs?vJe)O6gG4dvJd2Ef~r)x;TIN$z2oGBW>pcvUk9WPS{cg$ksJHKWDPS3AfW zMkMiqJMkCvpIrSvDbN4Sj$FhtY)8KVaSil$*NoW$m>@L|0E!L}F*UX~F?ABQw==Ue z{~u_qrmMWJhQzzPl5$Z2?dU+nN*Y>Htq2{1I@h>M0GQ0-sS#$;pMvY#Q*o}6QPIh!h|tul~#-) zmB>2BFklzJ*Y6I8$UdXr|T+aDTA z!1_UIprWh(CLObAHJakz>|wzzdkTB-T>VsGe>o0+p1iL)nc?Cd#0#v5J~H|gU;}2Y zb{=9=K`4dJ4U$Sbd>ga!$E@Y4I!_Q9GugfvPof22>P;^i!-88&w!8_y4ci%dY_pnU z4DLb^m-AbfTLWoEdJ)e{)()C9wX=jYC*yK#IPBf^l)wHA!PAMg){s*>#s2<%BpvR_ zro!=$nYXDrL~rF?&LJD#X)2D$Jmocbui&f1lD6J{2;ACXX+wAUgy|^oSHkg^+4U3* zToO5;66VzTG@u3hrMZKKK_#P#w?$9OblAy#JU_L7Iba!2H(BVd!jHFrc&mygb_%I7 z|3OkO{ji*TjUAgBt_91Ed4gdB*<;2X-pY5L`vJ#k@n~4-7h=|t;*S@&Wo$P1D5HrU zRUKvp6D;$z9@c=}t$g=VCOmoF4_39_dx#vsjyc2+A-@U_y)o)ND&EubJxhfP(E_kUwQ79T#-l6C}#OVz|M_9MVw25V-pnI5}B_mkZU**k~^?{ zl`~K@5_7c*5G{_t%&k$}t%0@Sf9b!)vqIuf$z(vr_O^l>T06e~e}GLL!z4`|W2qu= zL$F;nN)nlskMyx#ey@vZabA}@iD3bdl2KLmD$@>3?C*qZoEX_S7c)DtGI{Vbv#wZwBow^XJYl>&K{K^oF!Jr~;`NXC;;wxQ42V;l1!>#@z3d#iNhB}hL z-!XzB``l?IbC7wkLCram!90(Sx3f(2ooMi-%V!I8o>FmEN%6t$x395C4Ck}KD>;|R z2Js1eKa%jQ<`v%LsLtuTiKf81Rnf0@m7DaT*MPm?&mIu1u)uX*5&tzTX9!vg`iQ=}T+f)J<18F2ncMH)mxOhuSwhX8bGditoSqkq|%C8WyeNiY!w zD1bmKq|q`c$-z-2v;fH zOqz`R@dRihK>0*w+B-W}M*ncQ=H9s2KkC@*$>j>cK1(h$^rTQSo`!i6RHZTv9qj+=9 z!V|e&wA=EM4LZI1jKCAi&tWkNi$FURdHO<StXvCsarSr`N}p$U^Zsa1EDsy<5Elr5 zIsmYr_ilR4WUSj*gz9x^KjVOKHXUYGWLS^0tFbqwwluOd*;j2*Gjwg?IJ~>|ubd;` z=;d~7Fa&PJ)aBbRJlCK|SUw5>nHn5{+|{?KU(-6>I5T~2CQY5_EA>#S?Dv8dSF*;?9LU?91IU?&3uMrJ;S zFeWj{)v!};#3oOW9hRFeQYxlYh*^|0Zmq1X%1r(>FvcF)xWn=O&?sWOc(_;mN=obd@ihjkusgA9h#r3gSz zZT2`X>T2oBDy$c?d;cubMaBV>`X=$y*P~?9E{>&Gv0CJ3!*9*dPu|2~rjk<&^pWvs z?^JSkmu333#>AFpq|krxLmbt4g+fh>iVVQhX@#V!3}v$L}*t> z;mkTBVp|diI(AK(*In{GD7amlhRstL*iw1UCIcL13YR@z2PW|U^Z+z^K)@>~inJ6R z>ABi&o#qFldu0*Xx!a%}NRBTRZxOasZ*n4@Zv!G=Z|u>zTRB7j-URd6eN`5qYpXOm z-Xq(n-sDTI>Xfa7nyHdPXZC28CW)%p40A@*RT66!LD_J*3QL4B(g_9Vqen=nie@-a z66>N@IK&wd3!^8BlmL1nz;3moYYv&F6v{6z7oz-G_9j7SP|*}lVS^vyiKGXkiKPc0 zTLsZq`^qD588wmHDn?M1RTxzdXpAzbZoF!$d z6v&qQFa!Wq&Z0r7k_-bd{Ob)|QybW|pepTxP+74+w)d;aK zn7!gg><4Q;X6@!Vc52iEM7ZdHxaFIxrPnnb%;Pa_adl`;N9CRs5kql+dk|7iITMN-LhG=9-3jZFq zQ8Hd=y?KfqvD6Y*H?=da!3Ogv zer3XJTwAPsDyc>|@*MwdR0IgB{MO}rv|0)XV&KoC=$2HSRh!u$qI6FsYi+F`>zGZ- zL=4^DqSeIy;KP|*DqN0UaP#qUwUo71_7!{`O@Q_`QG;E_e6bpRR$@o3R1v8xG^!sD z3KHifsZ}eeQfIRq-txbM-EW^@P6f5>YqvCsbR)0SYA(?S_dRr0lIt|(RLU6@IdNt2 ztP@AkbhmaKB)MHO_0J;!(()hbC7Syp#N$mD8HRDIDTD2ozW_hbVp%0XN}?FiG6|}x z%mA>CXQR>XbE9(eLnVUYChUKM44g43{wk)^=qcfeQ=kdvVfY7H>a5IblXQKXT6ozD zF?Y3$Jd<*~-bwgUY!e(N*2LXOwurwLbrA60TTZrd#ICt{(S&6$fmo6iC&se?z57vJ z`P=};JP3nTux83qG1Gq`cpUs)o+HhR+XHaPI=Yfh$xSaLC>i1iI8`5Z3$K%EX1fHm6AW8*_8WzP!^KUkU}u$cko2f0BH*oNqP9vtY} zT>@m@5Qtwo&>v#pWeMvn%AL4pGU*dNR4#!TS~_Ns>`dAwJa9i|^4%258yQ%hd64PG z9%j0KUQsb;TA(~;z_z9Nl7GP4jZ23+58093_@Z>GWt<-LwhgmM#Ipc|$l(!?XxY7g ziP=Ro_E8eI5W%*HBab%!GSL=kpd((zh_{K9Xci^ah6sT|?k9Bu-z|zJHJVn2h(ne> z07B-Z3}=x9l>8(JFCbl;cLK#E4Q4 z6DJ$TkJU>Y8n8bS7d-*eiYUcU$Q4A2(&Y8d3}wceo=KS5+XytlKS!>@oPjv|(aa^K z#4UGu4`iPcR^A!FpOtHTfWRNRkPxNLjz{1r8_F;>7b0oTL@^$TBfa89yG9E1Oq7oC zwb@juXU(Y{xvP!kt`c-$EE1)q5Q$ofE|Bh|U=R$YB=s7c&kO+v1bvMOK7*cXhBEvx zKhhYh=HRH_kORGv0=JD1P-00i<>RR3jev6^?vz`KW7d_1ZOivlf^8dygD)E}q0u+? z_xvyaU}9!0r!ykO>H7GT$YPNVq{t5$jgqmhRD9iCq;}Xl*(mEO(1z;g*QN zBaKEd8Ui|k+j9YMuZ?9nPY#NbU`!`R9^2vdbdHEDZIH9(l25*~)O&^cTxqK8a#H)} zlumjuQp>*4Re!n+{3O1DA=`}I-7k6^-0?|qfm4N{cZ?rP-DA=k80!+Sukz4c<#pH$ zW)&&b*kO=TTf4tR+-q#`OKLt`C&?X;>069#X!aeUJDC$l&K+3@n|)-lem*bKJhcR;3wwz&t_u>npX06P_GWPJLH!}O-8@ax7=d54AMt_ zk=Ce|oCkiuJ~=I4;Gj05!6+~-+2r)aa6`WapFesgF$7_$K9;?!J&U{wKqbS=hg z2L@-Led679oYCJtnNDe`?l4mi0_$5~D~AbcRI>yM{jlkAUmbg@?ky!B-fsV&8SfIN zXAP2yQpI=juO+25DP+jWtz?i6ON2|Lib+~DC!Mu`j(#q2DyN|^6mBCf3cG>Sp2s8E zBsY=-D^n_X6U_vP-aFC(5kW)Yk~(qKLCq4{t6Q}>vbebyb=mR|Cj=rqL2yqmvfn8E zRly0WqBXCq&ekaztSsu<;)<+)2o~Rs7c?-6hZKs@Z8K_>sbd9FGiFYwT&FVnH2@S- zYxemn@(C%|C8oSZ>g=+gp}u6^auZ-#|keNudOu+JduZjbo^aFqGSv7 zJ3*NqVv_y?-{n|&g{Hp;hW$2NK8++V+1_JKPn60yWPR!k8WGjw0g~9ihxb&kL;d+u z=tNeY^>+e+#+W69>3BX(t^met#XO;YXf;BZH}B?O#PomSLHO^0{NGUFi$SYOei!Ok zF#a7XhAwE5Q~?EcG(eZSw+os&_TNpD--$BhJP9aJq;-dfN5HVfGi>22q@*Yr!6{%y zc`%8ql1vQgn^RMvB6UCMd^OjtXyCQbE0wBaK}TOBeD*tgaV%bE zZrZSA+Gxh_kB)Y^Uv+I?djp==VXt|<1xl=MQlh+8z{`Y52>|d+xFTe5x6gX`p=41K zFLlAbC{N>*!P&8~H$ELvudTvE*{zY^R*OXYg(!UFyE!1ZJzY#!{hv%yH~vws9tuM! zK01BEJb=I!f)waaWV2h!Yu^T>+&<$F58T|N`5=^>ol8aRup4ZAzVUtwz8sU*X~x@Q z&zHa`AKVm0e?UJEKd!ts8&kPo!J*@9oI4M+3fp1G0$P^Mgpc9eW1S`_VuF5h4RMvg8GL*7F>G++hiJ<6_&7jI3gV ze8-a5DInc8wAfuFylJ!eDWg$&17l9bInxT~BYWk615`{TZ=b1XMkvToJN8E0AZW|H zA||Jf4Pd#lTie-Cl2abq-;QomzbKfn$pCKkx#}d**S_GIj7Fp;xFpe78a!Jfcd7Wll6KvLU}-$aasTy$zE$t4vw0wc_r%=Bn-omp(28Zuwp6i z+)sy1(+1R$#Vo^sszu^JL9;aJEO4{pNSZkNbAS~CzVjiiBc&0?TVvnPoP-nE{bvfM zZWkwp*x+GMFUue2Z!az7TAu-vT^+8I<=XNOqS-MOmzL`N^y$IID!XMr6elh5mmcc+ zcPV!{l2hagx0SNXWLF@+ruA~v+@3z7z|Jsy8CZ%S%Rfx$uoI_cbS#@xCK*E|m?fEA z7XcBWqXB;1@zN=n7Snp$RDp7sb+3;Z>6AWB=9E^}T*d5;Kht&i)R@Wx&C+1D!f?oV zN_W7FKr(A|L=|WJ#lU4(2pxn|>U4v-R}Dd2HRsrVq*%okDgPW7m?YDYh9EY53-;v3 zTd#Y=DGWXB=8h~#;lm{lIu|6!TvCwLCjbP!g0?SV>!C5h7lDW(8%*iE5HWigwUg7w&9>YX+ zeSfde5|t8i+x<2znHHfQRmr{TdrgF@Trj}AE3 z%Ea=@KC6zHvX&nm#K|7TngGrhR7501Md2ZI(Z-vu-xdgXEuSVMeB;saWcEaLekw5Y z&Hrkgp|Msny&!PGrhBlrpPUGPFB_GGA&M`u#7@~`vnTlGXag(#dfdi!MS)05oieUO zM+lLzn}~R7kH^7fcL)K9?{|o*J1#_IM(QRVTWVxs@qIoDtZCS;il>qcEU4bJeJwP7%|zSs@ggcEFq}tRkG{`ay;oFW` zV7!O&&@cxX3duv$87B%<;eHP*d?+niCOvpe?NsD!7^gX-6lWW!d4r(nW?>4iav}E5 zy?*3uyetJyVU2ZEo)j9P-IL438XCeyZuFli-$0)Y)sztAQ~?Un`uBcoD4o;Y=L9`N z`Izbj@_~gM7jKGxeTr%3ge{L-e6Hb6Q(`0?4%U50Cf$7G*ew;TOz)wrk10+Ne%nt; z9Mn)Rxuf07ll%!o)T}#e=ww`2Kq=L=fp~5-xI2a6vn`rWzB(NnEd}PwHtA5EU$qlW ze@0U~&U-IAM+H3XUTTOK<9HU~Iggm1kLcr_h_JUa{Hp$Cgo7`Dh+qVn(^u{BgJ6Jp zC47HH6hgV&b@2u!kR#U53KH^$3$7Cy^)KleV3(~ERC_{DB&i?S8VS7#3#G+CWMzfo z$?y*jw;>wJlhIx~zMi6tkB*mMwsUGo<4UDEp|j65pnDebWeA z(uc&-Gwz@*xoC#T?4f;$pN#Z2?kqu_tTwO&) z2aGlHNBLL@>dfphxe$MKxoZ)+xfJXqXI9O{X~h|%4s(1Y^F~%jV%c?pPUpI67mFWv zdYF??G4!{xq4%83iS^tpBGO`4u%CX)3Fn42MZ-rp*qhaqzq%r-e6Y2+BC>vUags%) z8)`uvR}Q4TDzp?Y2Jtew5obLFevyS2)bCq{v0JF*4Dr#1*xuM-McFNlkl7-K!-EjP zJFexJMQFxVRMDG>?q91WUoBL+=C*lvhGaH%DWrho3^ff^U1yAQwSjN^g1m=b&Qwf{ zPOdu6iclmD zZDpYw%~X{KS5Y!wM>A;qZ-MszBwqhFCEH9tTr>b_*&T$Q1NoOKx+;uHuElpcy_vcz z`|Sv^LqR~AinPiG{XZ%7|MXIo#-;iN(0GlFQRgkr*Wa|-EZ7lT%ZVk*^Fpb`v|LCe zxh!nBs74ot%_T7d0||D-!`W}^QFUA}oTqu6k2fHnKlcyedoY$V@J3ViaO8Si}+l55iypa!W-!;@E%lss-7HG zdjpP@t_!Vw3(XAC$zaK03emS342mH-c93ii z9qcS&OO8_goe9I@2_Y!4M>_DK=RP@VJ}3<2csz<8TsWakpaaufPfE)7Mof@PM#4*x zt!M;?iuI}8W=sX5_Ap7I$l^r4ul}3PU>YuPshBl%R^MZ1opT#oIW4^I3tZX13Nqfs zQGA@Z4~RR^Ln1F4G*ygy0L4Cc#%^4oG?T2QM&E|4)NQ4BbAr9N~!2zp1I^Cznb%^)LnedUfOPKzcNR~o}Je?Z+Cs7 zxw2$~M6lzkt3NS-c7=qeU0XrH`k*KVq4}fQjR1w{$pAKy^3_D~HyGtPb%ux>4S?%W zvnp+X7Vr8D8{EMhvb&Yu=W;D)=)2b%VA_nMgTAsq;ajt0egI_c5sn_y&dj6cnSaAl zOUW~7gO+o|#%Z-LGCI{kAIT$;2diDHH z8Hmk}k>SsJi#pO?_T|?6O#W}YL=Fk{Pv9=Tqigj4D0%%8-BSxlA^vaWxhnjBCOwq_ zgXmurDUAu8lDNZ81*Ax#2MQ+K5zfPM!X}#a=sra$v1|FV%XdQ<@nj^`P7aTO5JL7A zRQZibTXabSNSs867ShTvNe++xLY%OTGw*hw50e&6e0wkccfK#b&9}DcOkZXH2b36 zWct-T?p2}nfA5`VeT2eq542pouyyyZ+U+UZA5FU7JHjGpIsW2*C)=>(K{{=wI*Fx;vkVcXCc0f2AouJDpJgI@@Bc z|KvXN_GIR2-R8kl+$iwj4>4}>7?m!Ilku1hPIDaPuGeGy*`fBc9Nzcni_pJKrKt9Z zfPHDg1LmOaA~@2AOl^L~fMPEibYr<+@2En~x^Gj5F0n2}I2Xq93tVIM`2~fCh~bEPZL^ zGN+k3e6lTc<-~4@0_o@%FP7@*7VS$JGGcC3+t^y!OE}mDRavLQdnDRSa)s}-sLh>@xRteLsXk0m$0a) zOAq9aRFY+t;ujL7uhQF$Fz}nf`Z7>fHESQjMP<2Tg4Id0e(tJ5|4F@ptWa*eDd2|E zXfqFN|4r7Az&)F>0NBP#k+ox^Dp9H`hJL+s3}V+vd4;ocaCX(W*)|yC4u@jso0&}Is{Z71sGNk!?F0e`Jfi>z{%UW zI-1Ep4ppcr25Yb7d5`&_+P7`;f%iA=R{kS$Y-z+xDLH1Sy&<6{;nw~mEdQ*!Bg1ev zV(~oiYw~N+wD<9_a!jqd`YE+qAMYj=TaBiO(oIIx@+}0NN_quF zVtzdpQ%0St7=ZT&X$l21D)ao44W-K|cUh%RF%5B2hnLfuu8J464yK;Y*_bM{y4~TO zUAku4K%EgSPURw=6jw

7|olz#4y(%zM$vnVqcbd&A)XH}Zk(ytJPUdyYbFH&7!>%dIKsp^VwXULBadAm zZx!!cP>$9>qv7Fu1)R(30}RDDuvYBKw*2JkvTGEYq78)&?=zghlqu3jG3gYH8E*Lm3z1NqU~e{`;TpHt!(VzSmqX|8GZ6tSI{unCIE0P zAsL`e`Jq38IC*0JitFW^6tM;P@Ep#W5LqWKaw zK3A!8UI*4gpTDK^>@j;F!gIj466Tn~MlIbmWIRC)!h#zStS*K0;Y&z@Z2=;SX0ML) zTQq!am^;2Ja&sJyew2Y7K-s}W@(Q>ik_s-j{+bHZZgJqa;w>@z)P1IfrN&5M!cH2I z0tOQ#y%z$=USMg?zZ2mY5J02)f%X}CIR5o24B1b0MJcJUp{fR4Cl2YsYZB-I(_@4K z+!zJs!3n(a6X+^0rv(u}SX#g#&k;bs!nM$Gs#{W;DSRjpX&dtaETI>6%21^j=7Pmm z+wX<-B)Bmsv^oXfCbyRyS(~LuPhyyw^ghcR2jp=IZtMB`sxY}U{1cL$TGGZp_Z2b&@{aiK|w?I-YA$Kb3 zIOxRd{qu#0>7N~CTw#cVz8zte_%WW06(J(u>x&5R1!?I}5$#8jlb|Y+D7Tiy;!;&L zvX}NL-L#kiH+VfP^hTTjP1>Sl%o?PkL&Eel3ZYrSUZ5+o;Z6A(5C4?$AdO&DjeszP zaPd0q*aTDpic;PH(wqT3o{-obY@KLxN4%W=Xb*h$UGR+vOh>ezyBvB${T?A4`gjdy zIZOSP4QC*8y#6&e-iwfP%7fKjxFzKaNCiiA>=25eF&L>pQ8{G@90Y@} z8P01!0cR%wia(b4(T2xAM3{4$T#ip_uA#j6xBcGUq};uujC+PBhUpGJ5N58HhG{ow zz!Og?=Z4*$@GeQ&(L4fnesvEElQWDAE#m7$)apc_>jVV<@4}eF!Z6u>dN#awP2d_s zW=Bw(KBA+ZAe&~W&7msqsALZif|8kedeL%#6BT{o%;7#hastoGEwm5XyTt{>OBwNH z;mjrad&nzmRiZ=4uK@YJFH?%$qG2ARE);_K$dT0o0p$SBPy=uLbmL-^EgAWDAS;9) zuEGrUo*>y(%T2UgsC;rz4Sh>U_*~ma#Oli=)cU2>6!kR%asi7o#dOQXnw|H|9Ohsc9-#*HCr(u_ zaFY{Rd_IH@f@>QnKI1jrx6~lFV-m zqV_#sEo=r#`48V1C-qvH3k^`MrulC#NGP_1rL=yeeNwieU?^x>g0^VeI;41GN3c<( z_3%+E*1Ql?^@%B?5A{39y9<>%*i!1`W1a)VV(R3aWTX`xCnRbwFRym*w~VXL^X+8- zP%ALDAY!+|jx>0;(vCH_o#_H6`V4 zTXX32W1Me`T$2t!0`0XHHP#hg;{MLnL+#R@23PO0oX{}1pm)26V$($9m{ZtZ zy!179QhIY(qbstepse9W*%!9L_i*=~6Z*+)_DjAC$kVKBvy))@38R}GEDx}iFcH{9}t zh3u7asbk;_2<>4mTs(T}A8amAP1r!_dHeCW(W*OF`!c}HNJnT|%EjU+zgSHl3*oX$Q8iRqmyKYOTX;{iw zY%SS-b>qX&mm9ubA zg+hA)7DCrNOXLxPwGo%Xv621?NK0cdm|4^|w0;tYEcoj($};8>-Tf72rZE{0lvGCP%>D5e$8!wh&RH}8uwlIW04?m5HkBC_tUIdwOn$Po2 zZa`#1@Za8wjw6?15Loenv@C_dXyC&QiaC)1RV>0M+RY(fkPab55YRM)!E9RZsH3)Z z10!rg;j^}lq;GxeP!w-kEwdROAk=p)5k!GqA|#qVfrf-V3wTj}3vH(| zn>Xx|l=KvBpwYYLaAbbMzu2xBF9=ckh%dF>o)y>r21*V#1YuR;f5G@dnpg+awfF4l9)IoZz=-LF}NY&PqowS63` z{5(u_rFeH_wc~DaKoCVD>XBx#%tbyje$M*Sey+lVB>W)v#4p@A3{nR1M&dHCoto)U ztOtOa3(+_|GQjceOj`uG_{YsB=>4FNiS*ld!}s3>&i_8C{@)52x7_P5?eDtT#J3{4 z+P@VOdWJBeQwgh}zhx7c|EHd*qWjJ1M&q4VTS{5PK>EO#r7k9njC%puEeZUrhAFQX(WR>>Ja$y*9*{t<-)bb5JAzCeO<|4}Ff zJ5)w|=ny#Z5yL|nUVZ~FI=talf+Q2i+9$l$c!!ytO?b!&==1}L1|-G z+^8_swqCs`!7RB~1pxZi&-fxc-<93D2MP5Rwa!a%P~M7GuYq=6nlrPY;9T=7{5oNq>ar zL632ha@CjjoR$CZ2N!OheIdy3g6Y+$7ONRzJi#z4%y6db=K4twHPmMC>coVg-DTUi z#s_&uwc=v>VxN;&P1wL8jw-oo4Cs-Rlh*?KO?M-IwZml5BCEJtBbyhraw?lh`7;!7 zk;+mhQ5^y-r%>QVEK6?$W-MT(DbbGI8s->S?=Kr6nOf#2O8$=52KPj(#U@Hq%tqRG9OKHa+K;FxJed>=7{QLMO&lv`z3& z5%<5>;s3uvVWEKjxn{M+ma@fCwGWXzxIfK8RD>fWEWopB5_a?1RGzWqtnJHJ z9g&B6>(;G-1@p>7q>K3SF{g*(_0NVJ}O zGP*xQGqkkPS~ko@XDq3PU6K$#0|mufYdwa^C&lI&+cuAC(EmTC-Z4nBCEOaGY1`Jc zZQHhO+g7!0+t##g+qUiQX?M@p=bZb!_eNAztjNfnyMAP@y&tV7#>j+H8hBu=UT{>c zzZi1nP^dr+YD$S0upcsIGE?^4wt;j5Od#VRul*QkU@~~+M|y&mc`56p=PDa~mMbFN zQk8S)yBp$B?p5e|x=S80XY%qwXJxUL=HS{&K8z(&-jvnLGNeScnYxLyugjjUYELY) zaYw!8Mf=gcuv8Zr$~oU!W(4a+34{CYJ{XsMBRzs@T%M8&Xg+{RUxQ)n}|0gNa(gFX^Bp`nP2WE#YMe-!AA_pZ@;DP z;wgNL{;LgUyTDT(V1P=U2pJhcyGYz+rA}rOQiO%H_@i3+3rO`XWY7$Y#6H|J*xC<_ zwQMG)x;A_Qz)u{^EE;`IXUY?!A<6LvcTA2cE0H3GagTgeFs8suFsqWG6w7J8_@1t) z+t37!1x_ON&^b!dfUCYe27B(A)K)|@Iw3EyQ3iTNz8^FJB`Q~CUR$ZL=btU~?>_ma zGRto%$^U-~LHw}&ZUBx)62|87{i0^pf||`Z0|N5TSj%ZXBk50&gn!@^^RFu4|Dbsh zqvKDw4}neDo5qg>(Qj7#0h7wGL5-Wb&BTU~8gtD7+k(}~s1FMA&zzPvo?lGB|4v8j z!%L?5XF7ChT^ZGPk6jK7*i`Lf4!Bf%F(@!VwyKsIvKq=4DWo)xD1*SY;gThG010ZN zE&?iqxg|tWrMMPue^3+*)0r{Q(6-gzAiAq_CA~Uy{p6*Zr(*a|1)a}nVg!VK&HK1? zpQkmm8@rp&?CjU)o}IrSv%xngi@P~d?@-?2I6@FNRPKI7^6=nuR7503!U70|jJ)yy zdl)0AVFm&t#xN_W5_+10{3uL^$M`4>(9+xdsMx_bBg|M-=DlzrSf~AXv{;W|ul6`b zsQy;66lf?ckeI5Vc83MsGd-6-UxKFim8Iwp!6cBD{q@dfy^HrAJaqXU(lcqE>ntgM zd`kZKe79y$cUw6an{(e8+&$Rq3x6j9I6%^jm9E#LF|k+?9HYdJgt)L-%{#$+!)cx=SarZ8Wo6?a@)R_$V$N(=9-mE@jlg_yLqkh~CGDIoy?KXuCT zNes*@1s8K{rOzHqTlPzERi`Uu7=^h=VA?l;C3Yyr#4lBOD7hJ0v`aj+B{^XL#%ilq zsVtZt#|T&FMMcThsp1W%@fOWQX`79h#y(;+Jk1b4NL!B7R6u=zEr4 z$|MU)u=8G9o(&n$oE%A9*XB~iq)s?bYp?8Q=6WP^-C7Peny%WMtN5 z7q{jtRC{bWk}QUuDzy`Ndmx7u!P1^yvq}+m*mac>_XshEU05hNJ-+VEclBVenXq4T zhBVDGb=+L$GZB}ilZs^lS_5R_g~y!fge;bn#{$!Mmg~Dl57=A-DIDG5(-JCtdL;YX zpoXZhq!mXL4yS69u1cUtQ&-?2T6(a+MLRSdB2;RU%pI(b&PKL|p&G(jrY&E2N9I(! zO!*SN(3b$-I!2cI|%L)#qinQ5q5C9 zGFYyJq7pQCy&bKWSNEdHSw69rGtFMeDrA9t`-;xEO9mXb-~miuw{>?e-EIB*iQMi)h*a=lO8tLE8Qt|H%^;OS?Q&w(!=_H?5%AiC#Azax;sK2u^K- z5F{2}?IAC`PTQQ$>1Wt4;}>d7HXr9DA#jl@z9b|$k%6X6b`IQ6?0j?x(Z7P`(T*>m z=K#?yzOt?U@^tbc*u}_bFuTIK&N*_{=2>E>l}^!Ek(UzyCTtkf3%(K-;plr%ral%t z1f_(Hr5cM;$F6Fh-pZn|3!QChFE82Zj~#TbLclAsazQS+xZfPX@X36`#^Xe422?Q#c)t(aL|+ zEg5O}so8>{lC#!}oyq+2mN zPJgF>Ww@J?;PmmY0sgTaRfYH#V`Wmsrx3n{0RQ=C`z>foQALze}P6~egf`a|${~1ox^zVK< z2W(RJSZJgGH&eHQ;`%`TLELBz9>@?t^~5Q-#`MJEXv>rXlTzP_o8b-kVb$vqB36|& z>Zooab+loN#N^ehnoXM3%9Ts&%gfcOmCa3=XiX=Z9vbYYb3Z+|11t>-2`Q z@ecRUV0Vt5XwlrPTEzysu;6Z2eDGj~Y;4!>@gv_gz6;!W(BNxN_2?+@RI5bVlCXWO8NTs$*^(`J>>2pa_gU%BhA!5 zqJ|m()G>809C98pLl4xxefywjboDPJk*Ug^ReNe!x_A1#wg)Ob$6q^TFV!JxEFZE1 zH$3X&rky4GE~s`8t!y@Onl5*7sOT694IH&B@|dtAl<2O{s?abt(AHX5@QoTQs*NOT zl=vS&uZXX8%+?};E>lyGRBIi-y{^Nex=9oOX6Loe&jGW?%7kmCBIQ=@>{zlPz=$ET z%!B8NclI5A%Qz5XDPTy<0#(T5Fu}-u{M;xp&CG*_EOq9QjnaAe>33%s?O9bjN!xY~ zp80+k%wQQ(lIc4)baqJcXhVXHBDyl{mP;k;tQvPAy(JKKn#j;#gfjl_X+2l6Bx_gz zS%@r6A!d$MW(ylB)#u5LBuiV; zN=zGriZ;&ZT*(lkWR^YQ`V57Jz%d&D(BD7XKpJ~x{BwC2jYiZ%D?8}Y{4&2t2CF9e z#PQb^HCikhP!IR|x7F&py52cSpVrPBzba2uE8V4f8;rGPE#*Rt=7J*l5I(YJWxy)%+lElH-H8FZi>tF6-f1eltYteW2 zAJWp6afl==s(ACffn^wsiUFE}DX6uW%`T}iNPbKuSeHy)2J#$0o}b;v2b%(K9B?!Y zNMbpEQ3+rjD?=bQmuW7p6*celK}-nE1K{gv@O$BexwK)r42^0H19p^pIW*5A5c%*K z9_8XFsjA+BRK#Vnc@p((II?b`AZ&o&on5bxE)734wdYRQwkkPqFxGdG z;bkV4{SvXjaXHh#%QuqLCyQ`Y;|!09C!1JSyJubP3@7J9jS8?A zjIlJtNNk!zCF7KM;*4nDrUi1MPR9YW$I3kJ1QlM95xu+o7){9A`5*T5Rch z#};Q`nuG!uH0G6qm18f(Uh|zOla`(d{jc1)I<(a}gVD6J9N`hp@=XY`Q?B^ak%&ZJ z_e>Y{-$pdc@-;UNhJNV4bn20~oUVIBxOyW;oZUfm7p-waA}D2RIGJJSmt2SYfL_q9vw9+WJ(fT1`x-5l!_<&_&Y$ep9FAT%9X} z?Jt~Ec!enF=5{Rw!iR_c)~72bWU-btf4`7nt;W>bBe$mkCZA;YUY*ZEfvpfhf0)pRCE-77oIWBFeiC#1akHay(?HzaPuv=gZO^JdnrIVlL)PlQdIIuD>Z{~u zFnLGLYIN)+y9sBVOgv&0m|>+F?9!J1IQnxa^j*$djS_f<_?4Z&P~Q&~tYqWdAH}P2 z>p_bvQ|uc9pc|8BL^z%z+{+vKosGQ26;+t~UF~vmhJa!pL0awC&}gl;X4AFBx@uFn z#kxjwWMkRDJ~x(9pJUB7$_gKFzN$eyQ}UbiExuCzf?)(D)!58UV3Y2|_K?@oKXrF_ z0wT_|#LN&mz;AJtk$|O{M#|D!r+E-!4Z}tr5_HrA0P}l)N})Y&3H=!O^hLgZQbx{I z-w`)W8=Q0{#2`BMw|F$<==a+{Y!@(=Kk(}@f$_$vt{Qq0?{}j0fUtj%UW)E+?+MrK zE1Y*XaPCd-$;NHf9c*h6!IXuNu-~ycbMwZfV9X8VA)ddLjsH66mAQleJ3V)$6jTUK zNcZ{$fO7PxxoEXA=*i$15k|)g#KWqCz6!qw8FVNcLzg}v2}!lH$Q^YVO5$R0ls1af z@{Lp87|l(;QDfepp+xPBzwT`6j6Ylg#`>~Xec|Gqzx+&vOP?NfEg_=Vm{XVpg#k5# z9Te0MEMia^!;p1c7_}tA66E{25FuiY9EpeykgAoT9!ecLYZay*IO><1@jFTwRyjN1 zWWG+hrY>rpFL7ugX4bO&!y4T|lFy-NB&hTKRcValh0@54kF}!IR0#_H0Ok=^H3aEW zX`Z$9zEM+{@QPygLCwZcD@LJ+70T5v;tsphP@fW0J&cFOP+pvOxQL@tEs(EDIGzEB z)45Il3Ws>Di_=75u=o;hJiq~(CokC-pOuw3U%Jw`5}(7!tP;Qx%a5pLSKEiO9&kMa zoQ#og8E{<`GY^*?9RsJ;_TIV}?oSV}TdYVAjks}JH0`OlYC=mF0wYU9EhTzoxl?mh zn+wO73wEnVD;UBKM>M73FE32lj6wi*98biL?(vgi7!*oEsM|jk*!8vpLvuNsH3TyH zs7BIBtOpU3IN&s7W!2Upl{j;5%yZK?v1#iR1hMZEMkM&fuYiD16s_WDclXSaVKM4{ zPJQ{Ivg-F4a{+;g+7cWt!x%3Ju;Lmoh)(afWJP=X*{|mc-;Qb<1~2@V0ePCO6a zqI9fheD5hE$|M`&4ncm+9NK=jBqGL@N+h=t6y8H)fI&U!r9vV{)nZV{f>yE~aCX2m zjl{)dV;X5o=Xjc`b@lJWWGmUPETWjtS2o5KD`(7-^{N}t%$&-Yq26L4w^5Q4-w2i; z62WYI#aZ`;v*E=|UIP*7fiVZ5lC_9ItP2^9j5<*L4mMSF-b3ln65lbhBPrz;|Fv*x zG&$U6DgsF{^r*KOe%64Jxe*~4O!jLSru;Um4%-dPJ!>hu>o|)td5-IO6Jo1qn|2Zf z96DKtNeAZKMM6A7;oRlwLRZxjqk*#3#CA0+y<%l(CqEsF1oHR!5+?(oW1OZbIyQm|heLX6DC40|t0c~#%s#?33NAZ7=Nw7G%?*7VI>`j%s`RtJH`KqG+ z{^6}qUE(r3x*U;Il}kKFQtiQCqEtPpXH9PMsFL9fc#S!r#Rr%G-1eYaYCHOAI5pFp zemk+KU`Q@=i>f_bX+UYqkwZMIw8YscPkmfS#J+3peQv-7jD(ugj@)tA0A2HVIUrFep{3bM3Y)T?WP|H#L%YC$XUqi;y(^?$ zj~CVukj|8B=SUknT1Ti+Aa{wTpSr-H+sXU5Cwf2ARKVH|kZtF`c%VBCy~|$051GGO z9~MqJ!EwgIHH1ta0>k(_;vgr*<{LY}7VGsje_mY9SE?id?3XkhXC)Xv2=^6NmGoH5 ztfm|sVRz~-Dnlx5;8?~;AB20XZL+!=fn!-x+9}|b@%j`;tYqc9rp%u*Ot#8rcCL6Q zV&PbyqjD(Vyo4}=svJxpr6!)xs@`u9Hi;3I?z5ql3T7fI+1Hx!zxcW(^4L9~gpU zaESYDzu6=H=gqU1$3RGJ*C2%hsB1Z)h@gCNw#br7u?pi;q660?K#zeT$oI3bAXaK9c02B#!96-nNz_%wR|2~}6m$=RZA$Sh)v zoFmL!VDfqMKJj*He|Dbvx_^%4`?(q_XE__g3;OA<+4lpCBWKS&#YkKL02;y}>yV`? zAB`Au!ai*uMOik4F~U9)6hNPI;w~TnOv*9sn59$;N#~e#;vRlr0vhceETmGaf;EU@ z9ZH(X9=e;RSc@EPDLqk1n~AxmEk6q)B!E=X*o1N^5I{8K5I`$>jlG-jb;Hh zgyJ+nn5~K&V36-2-cnEku+MruZL}PQH4tN=qHQ$4FE|hmEk9eK3xDyGKj^>4%1|&{ zD$CCZKRsKGn$1FGrm^yz6uB`hJIzN7xs9`pP_ktty<}i9jVm!2omVyzY;swfp%@qw zVy;Kv=Nj*g`a<9kinKbJH?8SD{vq6eQPhyq0qyAkZr`QU-uw zjnyO~#2hU7l_fR;vLp+{`~*E2*MW|FX>QA~rL3CqkYC+Hfu@)BJJ|{P0$&yHv3&S| zcRw&8&2OOZIfa~QW?R#rLgQ!h$Tb8j)=k(`0u@WQuUM z7{gC=3Kd3{r-WN7vD^Om1QC#toG*G1f**Tmn9o#EH*dAtC>qPGFaF@gCnYj~y z__a^8_R7v*0!;-cenT%SL0NqB6+jkyFY$W_rA|-=+>g^wfOiqJR{2G(^q2PUZ?NC%_aM_x|Qff zKNveeEdPB6kES?rdOk~BJUK%WGBIZL>g&MCl_)O}a7O7Rq!#br*bK#6d>&$@l+CTVpULk}MCj=8sW%)DY@6Pz+c;-c z_)*CZTW|_Ogu4nvsUW+~wd3DJTVRcNgsZ3Y*9}QjQEXm}QJ^56op?o@m*_R(&?_O= z=l&uAuqixbY_(ydeBBmZ0rL`_S@wj&f=Rwu{)dwQYb#Cwm!jW8#0LypVlGa_`1Xej z>ya3nS#2;%bk4qjB6m{XK+$Tuad$N5MtNdi%%u)k?5xbfq0D90q`ulfVPu$b?w>jl z?wr1BE#cOz?XM%c9|V>y1|VE{gh0q04xm>r0F-wJT)BO0oB^80eKwd~c$fpZ`c1j8 z5^JP85V-HoO@V>zha|Uf`EdC#a&tZyO#p);Ugew573GERNP?Ox>1f_Ka1e>vu=)kO z`E=e9Sot`&?b@CLoi4dm1QENW+S#%C^c4MCGF_X6jggnCtJAU=S5_(R^ulyIZ6SQN z@Umk~p>wiPa?;$gT_jG<^IgTDgtONHkM2e!TstN6GHv0a+J7>G|I4BO*Ae$G(RS;T zii7;exsc9KltBMMC=*il|AC@rq@(`@N{F90=P|z9$mbBI8u&2arDA(SVFPq`QpeEK zU|{^?g7$Dp{rUaKLE^Xx5CTX9U}|6oCdIy;V-jYh5M?e+$}P+1H6mKQ$v-r`^((o7 z!XmC*b>7#D=~}cpHnv6Uy6ly-DE=X6)jJsW@cWM8kGf(Lxb zmj;R2LA=F!{NTHv=bMXV@l-PPC1m=0S)BYxrwg@kf%U0e3ivbMGuPu)p0g80Oo_{t^w)Cr-ql{kmo&Am%CCvW z%yi8X$Y_^r9jnX^R%Vv!>vQ#urEbTo_L!RHb`}|FBilOH!GZAWiYIBxl+)pugpl^- zlIG!d7agnRW|yi?ZzWwREiW-0Eg@uCn5N+b6-cJUHH&O42~q&zXL(Zks4`3FLV8S2 z5@t*@DbXJL%W>04JMx}1VBA>KZ3;UT=JzEeS%Cvn@Q?zN;u6Mm+SHh`9eW~!QqzRJ zAM!-Hq*G%pp@}P*Qk{JP0S+7ToIO+}&e_!LI4b(@o7kbk66(>3LC?N6K17rKXzA;m zZ_%vwV>HrgwiLkWOxST#BV>MIuc+7n*NUx1qO`H+rOx&qCXA;O`L(VUqg^Sc?3xp8 zUpmxVe|cjYW8zt2ra;RP2P5lakkZvL+1r%xU{!XqlaK~5UaR%5Sh}oiOdko;n2l+{7Gp*Ym`s5T z88x`2wzg^U(PIc!73RIC)hh4dTd?cw{zuf2E4|wEcWk6n81TE@6zk?k4ovG&+-x*Q z{C?OZDm{D-lW;e2$RX>4BluEV;44SPsWReGm@)wRM_;93r4TKYOU2I^N{EpILK96; z!tLJ#Sa6t`2oFO>9h2?CC@Wza-4M|Hp|A1AcE3;zK`VN^gW_hXWBVyGjw*YHHCIKM zlC4=<_yy+5Agk%9?OqeJmbN( z)Ai-tJ1~P>1Ec~dGyO1AnVK|}O;Ysj7)`*K;@H(2@G<5QiWp5lN54Y1HwAhK8-adyjiLX1?wKrIvJt>xLupW$>xc%y6Al>mDj-S3OG(qd%H)IlUI zZo`i_)*wDRx{61!vxygRaCyG{%v1s}Xb{oGJE^EIGiPR+bSc{F=t=Hz1y{UOoySgi z_v~9g{j7-|-EKZ~?G;6C^B!&*S|FLP+3H+U_8uo010VnG4Md_#rl(0hgruJYF>Aw$<6fhaxM?*q4x^@tI&gLC{w<2;SA1-xHV-R+BZlbr z!mu$=^XSO%p2Dm6vu&5-uQaseSmxh3!Z@L^Tu5r+eDLr_l{!K0#Bz%!RV?lfWZIqs z?ast#*!P(q()BDJ*y9)W>%Km9edW3uZXS`Gf(^lHH^;NxQps`Edh&$pRwLRPx2uW+ z_YJ>m{1G2~B|Ij?60vJ!OFT{i_Wh^#-sVYhPJXCT_!V|7pCU*^CDpySjnN+o`hM>u zU1c1`7=J_(6n3IMuUdLGY~F01+(UV`En?l)4mOtDxsUdmT9L_okqWUKfruCdr)M!8 z=@la$JeuBU`C|*JOZie!mpR)rulv;8y~mFoP_iqacZNui3nwI><1=#sbUsl~(+QOe zH@qS_PQ~3va;ru%!=)1NuMe_`&nn}&>W&dG9CPKeS%RFqq_@z(8&Tc#f`xf4ce9$9 zk1B9F;miaV!dh5;t_QAFwQ|5te!4=>Ku+^MX-i`s{+-(NSP6eYeTGxK>wLU&6Uz7{ zn9ot7yS0JKabRCgec!48^S!{29GNp%S;xI@g0#6%J#qxWrK13j0WOpQhh@^te0@Vl zi;z3p^{R_Il@Yu`hCt>g6{spW*!k93`Voaz5dU2CaSTuwnDFx<4fuII!^7m%zGJm$ zoqjn!QIVlQ=C4zegsoI3mF#a#@5L{%`ggU}_s^m3Ht_Pi)s|BLy+sYtp(EZD*4+N% z8jh@QW@1{u4BU<*BD`JBRO7hmIoQJK5fV?*z=+L*gZP<|9hw>9dekgM1_GzlQ1(}N zDWt86XTmr=?d4-~PJsj&1$l%Q!GsS^Dp6W_RlF=aQwtUE8)ky+idG-I87jhO8!T$?0<@#-l z(#tMnbulxi_^KGOm8m3fBCX))R+0Ncr`#n%6dw(zmX>1N7l67h+>&LHM*6o4sHhh zrlskS1Ap44a+i5(YAzs(8lguDCp$189K9RvSs0?&BtEhR#Wy6^&_ zEn0yTqFLao*VuFt!dAp7FoQMXyJHw0!F-3>D zX#iGNkdv^&11f5>J#5Dk4+Pdew=&gh(hL~tt$9W-uXca}vZb9Cyxv+fAf3tG;Y zkNI!6dum^Vsay2cgl9*UWsyBbAz6q?S3&(|WT1xZxL-}#_CPyM-R$;C? z?=nk*NlhW6SI&&V6{0Sh0#03N*v=%)Ijr9&*yhM|r8e4P8*MpwM_`tZslRu`}B#n*yz?HXHg=29uzxvg8=x#DFJl){(4<<20u= z94FdLBi+d{2Hi0R*vl7Sc8;Rz7MbNh7sZbpF#-mUUsJHbl#=qh%2FHw>0RWWva^p} zD7|JSMMucYRyXp|q(yr?55?LFkrP$bY8y;W?lV9X){7NcJPTs0Ojdwj$5lMg#UP52 zQoMrARj{e~K!n}#%wHHeRW9~VuA05rWhr(grq@L*orsuU2M`mq>qD8Rwr~qZ*7F7+ zy{Y-X>e83+vu4gey;W~c8a(;zEv50$t~aiS$yhOBMf2J@~<0NVp4?hpE+> zx=E`Hn05b24ou#<$x^dryQrt9LKSIj5^M>&bUtCD6wMjJE+Pvk!VP@4z}XQs=?pBn zqs^FTQDWJVSb5NCjJhfDtkM{Ju6X|)RCQ3Ms3M5>QLdI8cS@YGB*9#nXs$}8phYIj z90Xophlqa#d{3{J2j$u$=TW9kN*GApY+7*Lu1ObJUX8sr12lz50TdU?yQ6_I4v4So#@SE9O{V? z(3yg_2HKY}pFY{3WPsF!C0M)KQ~@Ztu3&S$5IMZ~8@>22kcy7jwc^NugLExybRUVMvTT#5=_A@-t3qk!ghgetsZb`PGuP_ZH$ zDR_BbXdpI#P2VDM9*NW>o2N1{b-dNFqBrOM_m7^v%<>(RTc@H2 zHFHMA0dHS_%@>TV{t<%*WN|IBrRczAoqD!yqaAhn*I++jPR>xYf%f+)jL48-l=sfi zQLVj9Do5qKR6C4m-g7q~Z!m&;LQB6?M-(owY$;6uF4ORr&47<3I{FKQ%%7D@B{YiY z!aY&bgFV@$^U;&TTPL$2*TUa#oTr>`lpR{fz^-;**BY8@9ub-`N^=a9_eZevlzO$z zt7^e7^}dL86{}5Wd+S8#J9kTlzON7uE&9F--@is*iA0lDbPMOQr9Dku3rBhj+hO|s533C>mcP+g+3kOguCjS!0Xpvv?R{qIRvRWZp;bC`R(1(Z*x#*mEp5p1j&~r+ z-tB%)KK#YZTLDn%cmF-uU35-qx1W(2v6zI`Fk)6BkN`dwxqn>nW*&mx#eTRGUizUc z7I{}4I;;k)6?!A8DTt^Uv1|lcJX^sV<9eiJgMutHqxl0+r5E^J2_>U(xlHYv8=?0J z1Ufqx-Q}0$I|NSZEra=dW8M%UQbKDY{TcgDeBpn6N&nX)Mf9CC&fPJ_bM_q>(1!Xi zydaMS0s^dTWS>f4!H)V}nsw&5u8Q(i(>Ln3?=I2M=zv>$U`Vf!K&lgPffl$myGtFi zJGezS*rX6aB8pVae3+BFSh<`twlYE2|^>A^TJO@7MFm&JWuH z#39^6!?b?KZ~I=lUlPWETI}rj0i&B=H%`St4WXj|-^!utV1gZRT|hjlhI9=XRF991 zbS>rHi|-)BguQ8~PI*ycXg|UspHVSq?jVIomoZOoU2MJ@%el#^{?O_K6K}FLn~fSG zZ&^2tKs(DbuvY)A9G~6A*-~NICBmj;qNJq7vNS)m)&s*Ovt=p|R$gU!rPib?2V3Wu zXjQVrj)stuUFQk-jZt4Jh%eR!!Enmc&Vt0^E=A#fp#xv0j|I2m<|sh8HlTcSd5JJ{ z?mwW$#Z@y$Ftug|Q56UBbg2_k>CcSZvgUw6`-jD!pt&$vh2j#0!*rqErWtvt1$IQ|%mEvNoj`6@ZCm@`x zaJHX#r^<~5H&_)5b)#>VJs39j8>(&|ODK-MGPK}OrxD_YLzn7UdMhG9w4LknD?vlb z;F-7sbIgnhMwQ!F>K-q?7knIA?9I3Q6mRVK&T zV=)D2eawgW12P-e4{Vj~$1J6h~WAZ~zxN;cBKxGKx zkeC+6=ImLP_zcFf@oXVyiBMR^D5W1$VmIS~GmKY80P7l<@>#iN#!- zl$fyHH!%-L14qgh@>j>9(R)9LVvg(A5qQI;C`%XsnOw;w77BqgMUucvbRZYvAqsAm z43njA{YPDGrV$DN6AXBb^2`xr`sG8*_ZNDuLE_E4-`C7OlV_3Smg%mD>q@^6y2MLk z=>@s;sXwt-5k-BAKOtxlqvxQA2@K$F=4GE=Zt-Y4+!y6}hrxXQg5DUvLF7N`LGVrC z7(BuQgk^h&5eYL0hef32r=WWua^m30U2lw1@COPQPG$JrPEkr&;-~Ngv3y*^>Z%`a z!l>sMGGGy-qgmk)cQ_STj-a%~vb_01wLn}qR8pV(+Ru>=H?l62fF4@Uu@ZZMc=45# z8sQf<2o)^lcV=?K#q^2IeEXD-NsL3r=ymr2lMuR(I3s7`+WB}RH*wms9|C{W6D~FX z=QD^@PwX`%Vz_`T;wXGq=(rIp$(^n725v?Hg-rG(rd<@t!x8zO(XNZAoKlK6skVOj&9Z1+uQpLo!-wVBZeS^eF5g+xL$c8EC!qcgCfX6dLfJf8f_*{ost3zkfzZ3ou`;efhkYsoTU|kJyFevZi0P7_gF0w7hE&N z`W~oL9V1`)p{6tL*aV}8w{2l#n*;{*djee&YvLJ=)eNH)KoMoKk$k+UT`Zes0tS$k zNq~Q5@f+QJaTMLcVaesF+4{M)vk#ji!Q#d1nX^BD zX|c93v0U@3#3+K`vDNxPheErw>;O*4Nc0d#DusJIVS=i80^T_g&$$!3Dt z14+K}dF7lCppbHwAH&UMlZieW6Ig~G#(BRarpRAo@+^(b{uew9_aM-~>5dDSdLo}1$pqDnahGs*KU6gO0~ zG@hvS=>fGW2B&>x8fq}JR44^x5x!6=X!Yl#aE0l^XqsKZe!O`!cZgLjJhzZ87vbCR zv)l1ZeyUg-`LA|aA&`;`65SMy@nnjSMc+M?T2Ouz_PZ5C>MOsoF|RD($iTqX`lS`HU<~)$uCSMQ}fyiv%+wy_W)L1rZ=Wo#4+V^9w4kC*BSbR_UBSx zT_Y3%?*)r;dg`kjMdzKuLH6DIohn|XJDzf9lm>CoF5=?0z|H02f3^^bWg?}a(2!T*S!2JXB!X$kl5x>*Ass9P^ zqo>wNfk340T(Kbl{;N!!QipU`UUB(~WiDi6@&yvYjW{$}I6}FO^y^ zs$R&*aoSnPWGlpEknHi*`{UDov*|U%@qGEb`MO)k2G{|s4ZwRgg8xzt>4CnzeFpjr zgyDO(pl_zjdu2&$L zom9*aSponpCCyv63$m|_U*kj{gUw32Fs7HzCI6%@<%~WCuT4O=__+kV)UT*13|s7s zN|SQ7{DUSxFE_)y9A;&7mRcwhX2Mv#W)3P^6>K&J-p82+V37<{>C-6MZDpd4 z-b<>DWV4*QZ(|8N3)U_xyG^dBOSgaxVu@^?$Pc(;v_{KxHg6Ou#AaESoaHiNAezI2 zB}o)bkY7TJc7+p-{u(VqluQ52KNJxBlOsIHjFhnXVwx24cCZwl9;9)&!yaohZRRQH)`$B zPym>c?di4BnR(-wkwT+zDV(TH7(0A|(!*i$Jq7F7FIXPzL!h24MaZkGyOh&Mm&0pY zFZfU8%ScQ_3uyM-sk0=Bqte#q(eW6-loX6+6C#i9o<%~82+|Q!P=p%^TI0{0S$imk zlZYT2LI@$PY$4|z!YDKvzXiMWp&^PLV*#1WS9F<;+(~ta#20tRztA)+IzW%Wxo#EJO^Mpon;^LW2d99;oTmzT_j{eqAU zXJr(7Dijx51?7jM)56~DeMLbDu<`1Ho1^?jW@aY|TOhGR=AvrErjJFU;8DT=a{&H0 zm5R=tJ$XAj6OOnW9qpy zHrIs;Eih-K8jI~lGfhh-s2zIc66ITx*|#B>UWy>8bE#2?qJ(s;7s3K=deGfZOm6gN z0K7xUXxxCKsj=BQI6Nx-#ViVY(^etkDb#Yc!^&~kIz^mO=B$zCsDTcFs(kSOW{%`7zNPj1Jaldb8K(C#9-GCvZ)qL{8ocb1h8_1YlAn9 zh>aA49OkKsK?&86E@=2u|5L}D<0)}AYg&rc-%&X9F|OZl0BP8OQOZ% zP=JK9W|yCffy1$7b2)_5eMW3sQjy`P^7n}d^WqE`OtC?jS{DJD%WD^c+n$SV{Y0#D5kOIrC#hM4rON`Icn$O1B*fwpKNhM_58%#7^CCE7eDJ?>-(Y%M~?tBQtTbF{)U@m;?dMEXa(?h+!}1%vcq{nM~f zXXTPmV@_gu#XK$(456@hylmTk3EYvpJeb%~`;d;Lfar>ukPoiI-J)Cn64L%p|;BwbkO~FDLPr%oB-N+nydoOEyr0LU_DE{2!Jq3IU01>1OES(>T#k z7fwO3SJl}wT0m#b0t05ugnbLluf#pPPFeM|Cp|M}OPJ6CoAA9wagZ1#-&hm=1F|!%R9AO#=5=*Uj!S?fyW!nw{L33Ak7yPpd`*2F0&1> z%5gWk$9p5s-48^U5n%V$)j$=hnF>(VIR>Xb_BvX`a5RF0D*+%~xPZ#Ler=MI$J;O4 z8G(-0zW*%%jw7$OAG%gZF-rob95%4}Y~5pu0buMUVagf5vTD&KFXFP=R&w2xd|OF# zFAH4@_xHJ*X)Gx7y$-d(LUpx^Fmi?GQI^ExZX|6y2;uL@@p}eTxiuE2TsRB^L|gfF zA>y3^;dA=$MSV{afd|O8b+%`RnHTrl2N8&Jw*&YnD&dpDPR-6wPk(OgF3#6?+S)3O z002FJ7GUHnW($|+^Fw~Ri+iTv?8WD}ah>6w}a)WS>rWvp~_b9Hf9e+ECV17`zH3+9*~VpJxf zseN+;j;_nCUu+&$+z@#A zu9h5_O8VT?pNuTF#Czb3Y;pTWnZ54@hV~rwwUM36PI5V*9i1>7uml6(V}E`D0=RA% zuiN;hx#0EKc6t-u8sBnL2BIr8rTV8I zq@g}W6$*ptgv@L1u*th3E_qb~ET@g0xcb@-S`GdGmYXSHJ;oB<5}JZp`9M$SKT#id z{@jLLbkDN6iJbhstajjVy93CNn)-E7KvLLn%uP!K``>TD#o~C)~%P!J)AZX-Z?iNGrHtIg&9?JBQ{g+9bQ?etS zyeFDV-wGmosE3|&r$#|@cd79_4YL;Me#3q3?XmvdXBM^WiIGB5;1IlHy*rc~Q%Wty z@$Xd=g9T7+iB`YYATxO;0D~4jlV0r^_xXY^mD=62(Bht$HaxE~*`m5O&Ia9J=`EA@ zJcWqJpBD`Co7;-Gs=FmA&hxwvKe)VFOP63+2rG#NuF> z$oUFo3Gz7bqr)NTd3(@TZSZAI@MY5qCZ>bK8BVet^IYxotfe?rfMj~KX|EJ2gnaQu zmf}g)JWd49MeUAZ1AJIh(7VdoX60T!){8}Rq*w1A-e}(^2e+M5PwdKIodBfyn1PPr z_}@jRvINCFcrc##Rzv$&b4|~mk&(4zHX~RR!p!I@%Asq_@PC+pNtyd682S$_#S@xK z5zIoraPWR6C6ty`e5Wne=ej-e(sq2<U3c0AjqIyz zpv?=f{zK^8G!W@kPgI*uvl39;h*N1^d{LlrHQKaZw{E#%3HblGdIuoOnxuCqFIXlDh z8KKIXkR?RF?)a^y)tKdK&JCqUfCgiSN0 zW1?$z3+I-V6=g;hox$}@jZJqNtamaGnid)e8m$y4{L-z|!COp8-xq-axI`q)I3MN^ z*BawiBaP-R?IH&tOT^}GOuf{hwy$Fu{(yL z%m=jB@2WmLrq00DduR`|KhAdCeI64wn&K9<(fUIGarxr!gh-xaIM`{h$mxvG8I9Q7 zX2V$xr^w9%uLpej*`*syid~d@IE{vUJ~+ov51f}5 z(ntD1=IrjG)kC*+_=rw2RC%69{tMGFD-Bh)9u+{C+|}NsdLKvY(hC|Kf#tK>rJtf_ z*j+wgd2?rXZ%_Vh&AqZwnpx%rb~0s*u)PP@aX}BR+k7n7DPj@<-TWSBy;Ombx~azB zr8Q>SfG#yI57U{BKZ3@f&DS@T%S-KpXx^kFV{4W&r=!#BC+# z`GNFLD=DIxV&V6gQAi*l>Xe%yXmSA4zpzY{nw`qG+KHb%va?*Bk|>yv*t}#PIBe;{ zA|YL|(9ExSMcN{VlnFYm_Dgejj>uPmR}k*G^1bFsR_crS7{6nHb;%Julx$W%E!O17 zrtc)*H}KL*0r+|w!U4iCBAt$jNqo^vvl&HAOq1JXtS(^;QHXExS4aS)4#7nmL+9ib zCS!2glgABKp`trw?7>T<3F$Q0ouJe0c+Q$|Hqli^&N%8>y$z&WsX4UC7fd=9+j4vA zyngO=D50PlAIPi)Qt`<1{X)eI-i+!bJ1-~Oh@KhH+wyk}@6vga8+znS#(I@*y5cv- zT&EO3TqEAPmZRw=Jxv1`c256(>bPoGRapO%rf;53RIgw(=j1cxA;xDAciH6X(2!%5 z^IU0_@rm1~u)_F*uYI=udJ@b{KTFdj6fLXVlv9=6u66xq3)O~Nt<}t^;0jY3d>%*Ffx!Y7@=JFPZ*QrbyQzM%>a2H!CE7T5!Fd6Ac~gj%O8J zn(F|@V2rA_BGiyxJhMpM{cxbfr5Hw zu+1=-3ft8&4_;gDWT3V|9o*QBthbHSD_hCvO>3|>=8_|F2`g@o65J4DsSzXzwmc0QjZq2q zNccX+BwPh~!=%e2!3Tu3611I)Px$MUK(8E-aLPK~$9>nhj|#}!Ctbm8bYswLavqd< zZ=@7p02TrGRrNNtU9P2Q*;PB>Bv~Mu40Z_RUwOFy?a2S5ob->2gG{eSzd!&1si6V^ z3I5~ad?|R3NC*JhiN?YagQ=x0Ms&-*t&(iFHN&{nnO{Jc)I9lSbZqVHofsAR!XPxzzxI z%k_iquiKBYkAsxYyIwIMzUL$ao!nPP67mk#v2Y`0*mMA9v?K1rZxRahAx>8TA4bkw zzYefs1aw;hXwFLsR0_`G{+kBPj)fSAYjy$reyQl@w;F6Nfz96`oSRgZ@-I%<2+g~C zejPYhH+x9qp%)S#KN&VDf}bJ|AUd82`S*uHZjgj-F^BeW6&Nd}$`73uI_4N{$JmqJ z6}hVp?fL+wu@Y{2Y!3n}x{>XzhmOuW8?8G z5TVK<=q9fi?u}HZjB=W?Om5YhsE_h^ES1Uv*bBCXH<&gNp$6sdVw*>xzJk2q;C^(s zq5SQoq2|xa#Q@mLmN=qe4N_LU4HeF0Sp# z88E;POk2cQP%wtG$(m{=R8*Qfw$w?DhUcL#f*u>VE)BZnz(4P1GsKIS_70K4D~T<6 zYp02NAvxQFkR#FHE5qlfX)boQv#j!oAgyF}3QV+GoO+z*W@5a0MPNx_6f9WoonBn6 zv{p1H1^xX{=q0@T12;_NbGRF-)@8V|nC1ZOs3X`2#A7RTnFk=m=_pXy2;e3`2=-nVG=64V-##6bjxApOT~Upe)&aM4c9#BHz;kIM9XBf^Of3w9<~G6 zjfN~%Xw!#Ka^zB9A$xla?h(#VZ?g_~g)vBzj7WM>Q(KAV$PA%x1tGqpfDQjE3Nyg{ zu6l4|)Oh?2l~4N4t61#*b7Z8&wN~`h$TM%(UnlXwZ=vK!h8?c*ZyJaAV9E^|mcf7v zu0g_$Tjs$}{Xq`L%cQiWJ|>#I49=;V@@Q|f@g1;e80M~^60BYlXip-Pny6o8=4@l? z6Tk8yP(yGoQmQUtc&`bEBADrS*H8g{%-~K^gWu{!>-2l=&)Jbzdl1Zip#)Cvnu7}0 z$U_Qd9g#UB&s=^z(;gS(OrqGHMyd)}V4bswux6;r={~n(;TLxasYYY7$N@$Ny*}p4 z%e)|mJ$e%$<(x!uihFq)x3X;XF#aAk`ePMbUMyJtc~x*$Pf2B^^kC=(c^&|z5mV%F zG5452c!pPbI;qp*0i+gX<~FgB<7&2{nUvf0%?hb1(Q>Jes;$QNm{lkTDXoR#`;uOX!3EFYb!arGU>y-T{mf-y&GL}` z+$2z3RcA2g-E?qO)Fe}Wz6wCz>(tGHnuFh-urT$4f4CaTJoeClq|p6RExDCiey+TT zXnt%FjMW>)t}ugYn_+sPzH15L%xqCa5{=A^5cP2~Gtp){$ic`Vc^E1!(m;9#d=GSv znbl20%rwcLBImirBwMNSv{-o=Q`TR$Qk%F1{!#qUzu1Gky0(Vp6)= zi_wm1gUCL5crJ~1hDb>A-@yv?-q3e_1$(yf(4T!tffYA5sqV6Tf&%)b!ek+P-cRdE zq#UH_@q^Y|I~Al#v^pnJY$}Ko!ykEG0;!k!fnUA0m3dOhS{CKCP1eFJ3p+u4YlfwM z3{Cfw?kgRyfj~{$;y?iWhA{YWbG!U7=dO8)-)(Z~?T#o_4%J#)!ZxRonHjm_C+w?d z1uM8!WUs-!M?26S6ZvGgX+b>eC)q(1>nCYJs9iw(_?zzdRrS}|WUJ`?mg|u6dk))! z4m$%^HMrdpJfFYZo=L2pDemXKH4ixKORzFm2rmh<`eiMzoXr8kU+5O(w2nM1dF>D) ztOk0;tMcq>llFP_@p>M}&77C{ag|1ImfcB@7H#xB%@oRkIm#De!+Uf)E{+rtnuooe zNa@~(H9qh(mgEnuJR2C^d~}`4A@e();Cfk?@MR@265O6hG+6XIZN#g(h+}H7Vh!z% z+Qt_2U+MPEv5|mp;WWdCpPbJ4HsP8w$diA!j-p-Lj{Z)@n^Vj061m1d0yke=&plhr z-ZN;%bR(++NMJsr-;OJGb2vvZl?(={{7;Y$%Xcv;PK_cFPfIKtQQC1?&C8c34N*OK zmM6FINg;6%@CWUC+Xs4-++&e>9}#0Z8|tAh$3V%BmWKd0UeO=8QqfHDF#sa3)IFdO z1=n*jS0-`fAcn=@2u+IEpKt}wTANM$9aW)l_L*+H#J0Nh?<2gR-GOY8JnV_@EsHad zld{W#-5@vMktqc?TC-Z3Y?81X&`?V(kR0Hc*}{yo%KdPl4hN1X53NVr58vBYF;G6& zX|;Zvux)^JY&FV{;rH0INcbjToMq|zI(OI_c!@GcDP6Sw9_r8lPSMj=$fk@^B0rKF zl-Nf951dY1hu#dU1M<-~^`_lVYEfTTkyNUnX0FLUK*U_U2Gu`={guImg-3t%1~rJ} z^r%a)AQZKU?a+TlXuR$nIG`#RhwKknannd|egRYdvzhqcvBCdD4Tj%QL&r2Bgw=Pl zH3Ccu5)mwF%C!vzY6{#XI~*WMO-2RvTL$qmOQMfcR8&z=QFQ({cyI`Iz+frm50+oI zs6I2S7?xITwIvcg9hHd{w0bAKa_qLsRBYVjwWG6g=voI?q-Lx~?#+?BqEx|eyiYzy zep{_2UtS+WEIx)@iKj6GjxDAGpl_drPSDCl2hh5=3!uNq9uU1P&7qBTLyN*(oFG| zjKEIKlT-54(#54WKhR3Dve=AG<+lm(xe9LbC%?&ZC$Hu$9j**Z07^hZ@h|(L`%HX! zw};H=Jr}8IB}HjeStw-jISX)HISg2}uvM5b#x46S_JPV8-Cn+E6ei!TpFp$n{e?x%)uATN8|pZZk?97$H1$7|Eqms5ivl%ukQnc%)=kaGi$E=rN8Eq zraKbDimlb0g^{UC6Opv?b@G^^%RkNI!?(SKu!z(5EMd5mGKxy!`mcw&lR2T^Ik+-h z=BPJp&nJ|$_U`1kFdzye?r@xI^Xf7@R5?1Ssa0}L^@Vt54y}`U3e&K>ICk8(; z&s6W&%HIL^N?5UQeTRugVcFy7Esp+4zzQ+;cJ!%3uYAYwBJ#U>$z-?;wW8Q^ff&UI zdV$s^{KkeY@+|OLG+|QM7$aUs4?6x_3*4u7$K9A@uDjq-enO$2zf3n&5*HV`MPK-A zs{y}nw`voNkfeRly|WDgS+AgtZl@-tJGB&w1harR#KW?x+n9n)kT?>kp$;VwY^p*q zY=;8D9tfnvzU6_bg5a3kxR_n?i?VG8#qe0*%a8KEsI9189wS#zUbPef%M5W)C0*kY zSygysPx z;_m=e30F|Jbi+@iO}A8a1u;oi8ELL?LAF-{B6q{dyu1UyRX0q|zfjf|@9+K$ncDAOI#I9g6Z6X5U#bol$kXEf}BN9b*Mtikj&6vCp zik$KB@6_o%X=cP!x1w(cH~bG1g`xq%wJhsL`V}qH7D2joB@bN!2_|=e&DY9LU>*CjN z4|Uj%%2Wsk|7LFge=lTY1epH~{QUE-Zl>=iKm7svA9E8NxeN4P6Fii~5*YrUR-*Bu znTl_ehXOLCXNnL#r6w2@D)|W-EhQEU8Lk=r2D1+Y?w@}A9BdlX-}1ZaDNMwKynqdP z6eYw@`MixL+9twz1JPPbokAHJLT$mKU#wJ?z(h)alAUMiwcJPC*M@mKiJ&Cs`ysBW z?t5_@?x7-G+%b@(jWyQJ)3{vByI$PQAAMe5K1qJ@O2ZbUmtn3qc_al_ok$PNyXn`m z%_&L;f4G`Nu*+C?HdS7d-i>5}4gwgJH`d;35*z=MYXT;e_!L zu7mwUGwhUd+jQ&EgyE=tG6c5J{GmtC^fR)kOp(Jaku5j&kkbD;^B7&7a6zozkG4V;Y7=-58x>`KnZ%1f~@nq zYvu{EEcrN0iZ&z_+6#&o!2@9`GwY-?I_xy%UDev@ZFzTWwbK)hvn}l|u81@?;09K< z4vu5>pek1BV!_AdD>c+UN-pc@k>U1>Yz1xO7wN4dgekBAI%0ZNO8`&2rLtKe#R17d zu@fnPD2D^k_bD8cT%pG-i)1!zj8sKD;McyO`96?Hnh2}~Kj9;9eN3Kj;R?|uE@8U_ zJYH%6+YYx#59YQdaAKZho9ykgZFY9)-Uk+vPacbPMv7~8;WeB}`ngVI_h;0VaUfr$ zafDcIFz%R!HU*;;7688iVrwJ$@ZW`pjZYpcVHZaVt)3`w6`>Uk%dq1PU( zOGU#^@j>q$o|c&ft5n2LYzGtL4z88Fm8PGzt<=v1152#-vfGJV4dnv(6o1)DNa zNjLJZm!?9>GsIs8V6yF|2q!-hTISM-`RMvVS4{-o~RSng8bWBwn}FU`LAh3jTHp`pPw?I z#_u9Z@?HK!=HFm%+KwA1Mc$qR98l#0FzkFkdii<(^6X^xJOmgMW1?6uwr?>n~hzgx=mw~nhnNQf%{x} zk>%%>H=sLhT-N;O?N*AvFa$dSNL|Iot-2d1Rw02goHIKXgp>vtIj z!|HC1U6=Hfw9x+}4YgWmZ?ERaSPPbcahE)mEi8XQg`WKAwH)(QV0x zi%o1cXRF$ajB<9^Zga5?EZW z`fP1eYoocRh=kt6>LNI~2Ckou2j$N^50$?Vs2XxADl&R{amx%bSsD@X!th59EA(J@U zQtSx0TE_>pxLmGo)_ByfA3me8Rng?$d7alcR@Wje!#%9d#rQQNw`1v5WR!f|S94iV zOinD3UzK%$m4{n5n7MgNtU)gW#AH-D)AkkMkrME{?W}nL^{qdshiPtu`??hS5pWjX z`jpr7mJ0>eYdRUWukIz%1KJ#AwVg)1DCA18+>u?ed52R#m5(kE-l$_6!)0bQoiYV9 zt8rU8mj2c%$-U~Xw2|Q6JXFGiy%Fw2H)_@^9+R>Brbm zEDeqfk-rMwV>zt8X4kNp*+) zd0s*RYuvP{8EXFAXS1XCN!Qs$PeGT5eDah+vfJKigf}R4z^XB+Kf(&;R5(=~Wxqyn zE#Y3u>8@Ocf1*JG^C%Iv=<5F3TcfWoD(*H8K!}}2lpf&5eCnImqB0?+GD70!M$y5(e+LfSBk#~(QZRrB z5Z5ckurA#|x;a5jVNWj13?A2WinM8K6PA8LQBk4NIw|x%;fLsNA^1+<8l*N*U~B+! zr_^s%2IdpoiucCpU-b6H7`%d;#rPvj5|z*;w-wj_1;bQ30_0JIw7z(9akpyAav*p zWCK@}quk&4RuPw{&QF`Qjwy%?Mb=uQNk}8O^h4^<1E$F)!?8}|El_3y2kxTV{Dlwi zbaG8K?_mNHKe%ZZ&q=u2K?JIY+Z&U~nlW04Iz?`)Q{E>%Wx`>1QK44L*rtvF_?re7 z@Sf^tT!L{t5Lh6S&3J10lEzzuy;snysSL3=2JaQ-2TxK+lT&#QgSm<@G9_WG6;BMx zQOKF4Jg{wZF9Xpj{f8wCu2J0!#E%1d0TCLE==_hFw)k+TShrBuJ=tPhSrQyaT0;J_ zOoh$gjNY2^sIHJE-J}@EOicS9kmaCo`PA!7fvF_1>57aVZt_!w32z5=z=qA z^7bdOO1KJVxPPox0$wH&RW1=5&8B34)sEYS6xvx>@!|*XW)5ZdFrObE z!#-Yw;v1qyyfmED|lwW9pD-wII&N9Fh(R>I5i~v)qk+> zZl92Qmc^))u^&0Q3BZPT8Hon?0@ynI!}7&`LWrr`a1AfTv3wM zPRfz9zev{Fvya^(r1uSqXU} zAzq?`s8lEVVdTXr0n;MLhws?kR43Pg*hGakU^1che5-@Y2${3>6@C84&P7zii`j!f zTh?HyA$f~_x-~bx#^SsY&6Z&x0q!w&*GLb??GhnNbkOH7*}BgfKOk(OdVC^+4e(Tl zaGAHX(xSFAe}GHF8agw|Z z*|wffGiZl|e8VY9UFpHwfoK8$&9BU7baGNE$eE3@N89Xhf-+n=_*{UMgBC3tSf^gy zzO18VXC$>$_KZ;qIeY3Uaqnm;uCrP6liOp6;zsrDBfx86LRZLGAf=^aekA6)0NyVZ zfi#gnJR^x9LCu4kD)SwY!1WptVXWAx^13eTfPj-S0D*+4^q{84w_2Y;h$#+{8Myq1 zc;4ALbT#}-^NW(wNJMrJ_yp$cv9Vl9dAc&*+B+$M(=|$@^-_|uJP+cNhn~9M4?l<(VgBca^zT!x2QA1)6u+4sfx{f`7tgY0 zf;$=bx-oS?2Wv*1F)j;5H*sF}jsfQ|bslr1b{QJ{$-cp;D$id^UAzpDuE0-Vts^); zmFiFI%aGkFwbq&qg6fkCXs{1U`f#gTR;;qc&u`Y!x`0;n1REdN1R&I@Za=@ zplXbofhWvHOTGg1#(8g$wfCd7=P9Wrsu>oDwC)_GAaG2zYp83vJetQ!sRtjiqYorj;z;39*gw~93uWI#G^K3Sg=ew~LLi!~HT6;9EIyjpKXszl;PZ1-6Q}z-} z2Ci;WUHNi#a3mCD?Y{J5yxPs`FrScA(mgRV7OP*W#7e{W_h8;gWk#NECEig?k8q(K(wvPfxtACyZw?=UcNBUO;aHrj?LI~T$QK%F7n^>Z69k?CS->+A zLYUUD7loTl`#Q~91D?84Bxv4hoMn!B#wjOPs|TyR616#xXc(o-?1^hIE1)$&Kxva^ z`yHS%$#6)qZcE-ks$R=@i7eV}<`gb=9~!`qss=xrQl69KI;Vwy@>tYz*ARAEDB@fh zF=TMbWmUz=7O>iH1^J9p6Kdkv2H+l!-|{I9kj-_^AyjOik|k%;J|UkjmoUtPWHquV zy_FecmiJ96F4Wo~Oc69<(K`VDm3TIGFh~?fII5*6sNDj}rA}nqQI}z#NT+L4!m%ym z+r+^w7NHr%BAz+nn(jxWn;m#^6B0MWJ21S#*#4(tmE}+2`W$ZL&)2@148YLeIw)4F zxy48E9^SIDm45BJ(3^zMY$d9a4IxipH{&=zv6C4GP`<2!Cv2!wp(Ca<&{!yFZUu6C z3sso{4|w^Vg@xJ?p}j&C9*z?S!^IK#1u7qA)pRiGQ>}vU65hQBwQFabaijY}gr*mm zup)3-8|<}7Go+zFx8t4zp8zt3&H&T{$P>`G)f0Oqi6nZtyOhd>c*dDaxZS88z9)Z>HmzF8h`9ko}ny z1}(mC9=DIvmFQp@)ZV6Ft6x2 zvNKpZ!B8Y$p(e&*eew34_~6U%$z!)d2HG{e;sqo%Pd#U^fl>jVbr5yG+sn40P-#VCeGrgRJiI9tX0EJe>5KT`uK_3$AI|cKsNCJ|L}BJT;@boa6QPuMd^B!;HaSCoU1M!e5CLq zC3V-{M|+8 zNU-OHlABIDi397$g z@PObCK8{z*RYoUKCbzootGmN^+O0=@)P!ZCFVc$zG=U_)K#K#*>Hq2~ckh7JR;sX8 zh@eFAZVRsTp=*8i8vRyC0pF5zc~&pqU$dmTDl+ScNxr+7@9|LN*pkAy=97ldGUbN< zJHAIM=*Q3MsU@4B0xOq}Uqn*~&LtnP1-gH!%Dt5!e}pi2wl1zQ;EF$Vm>!a_d=9bS zXdsyj2>IcOr{^F@AMi?Q3lB;?(dm?G&m-&|a{MTfsHi+5@0(>GTU~xUi~-&sqr{Hz zdl9dU-9w{o0#AB$s62`<&fbK9)c#zG+n8_* z0f@?jES8|?XgnsNOwT*fZ*`U4%&W*P)Xyz&vwtfj1Ab2Dq8?$^AqN+aN)^F*WbTwJ zBb&)*D_}y7g6AW86a0&r%}ZiLOX_OSeJ@zFU!cJ19a-)u#ftqabMzUFjAs0}ads{W zpdR1;!R|YR`1}Ks_HI3%sD;hBhxowKtKZ%u6pW;cx1*Ti{U!>dslxMGz`D^9m#+Ojyc>BK)$uvBpgp}tArZ${|2{ft?A@<* z)>xh4B&%VytXN&z`jh=|L3{FHxUktH$6565FnCZ=bDiUp>6$x6wo1+JchP$NseO1EKcRX{JE(#;PP08+pq25C;= zSrDN?nQJF;bO%)&FRR}o23bz&+W=uV=zhk}B%!Vm#v+lTZ7z}Zc-fVniYVOk?RqnK z^#Zokxn0T}X?G`V%4X*bp%-A>VouC5!_@9XllB~D7(V+sMhDD7*#ma_Im8^AG%KfZ z=x5$OC+HRt=s)5(Ow3f~fL3KqB^ZqI0^J!wmHxETNVd4)Mv=-#%(wQxo52=AYo^@~ z^ywGuQ6=R@mz8zL>vY?f?5%WN)GB#vyL@;oo9R7qOX6hlX0M#AvRD4?>yl!!P2s7w zXijxa1vnU?t@#%OXF6RzQwdCaxxhvD@Fb@@v1<)_T1fKK!uBxYGgrPL6~ zv2}2FheDnG3*87#Po+O$%%TGQ`y4baz#Z5sSpud~7L^Kqp(;cb<|cEJRRp9a;4M8!8qSJ@Ys1XL<*Cv1SQVZv^OkYv zE`cqZL%gHpDuF%9%3;lZVyss)tPHUjzfcD-Fv@1+DMwIE4t?w+>Y$y(3cbn%$M7U^ z7q^a!;9?c&m9y_e4#cpa3&;zlSN61UYRd*Te*#uvTIo}uq_lJMx7-*u`M{N2vr`QG?n z<|)%|$l?210eB6FBcOa&o}CpM2%OR&ER7QbDWr{E`3^-f^5V;Koj1WIMhv1W5C0hM z=Mu{8{wYUd?N7ndx+dvbAND7>K}$ukZoqL#$F}Pq20iort6^+fZ2L-brS$tUzxjpw zpp-)i!+?&8=B#pPMI2a5E|5|u5Qz(^FvpJ?dCju61rR%lzE*~|ch@g}ikvcy?nDJP zK)*`TC9DxKc0%q1=BsyJ+-Ho?8GdB5Ywe|@D?$)nk+g@dG6>0jTr-<$`f;)*EdtYtZg5)eh%5N^U!y1)i-grzkFm12j zz?x-P2bdQy0|KIllV#2KygG$ZQ~O!A`#C1iUBY5dWs%X6NrI#)I@`4V~M zxGFr6#B>gZbpTU{i7AB>+a;{vzaPOe0Ly#+Enr=i_+B2zXkbn${LZ}0zG(d`v;%i6 zNn8eMjuF+#8CXnY4#8?1q%IDn>~HA`WR3)hb^mc!n#S!`eV;7u0sU37t{)}K5FssD zkKsJWzgke_ojCa5IDY4kgxgnHh<@Q&E~2w*0tK%!^oeBV3n<0Zk66JkhW ziU9?y3sN7JR9G!L5wxlnS|rM|JLuLhU^f&cNDMLARQ)24a74Udmj9g*2kJJqK5oE4 z>r!9m|4Yx~PCtqG&N2llKf$scFx%;eKR^1VE%__pCnC$%Z@fk5>B5B4dx+(XxDRVw^BW;Qn?hyevF79TB zZo@;Q*}&Z;NSXq@GheU=5f(+4?14=-v5Wp&Usk4EZ-Ch+F2SI*TS_CS3Aq03UFUTS!B3j5N?Y ztupeG)j7dGvTR&c!51jos^1_;LAYgr$_Of}6Lh7{Pqf5Lot4g7M_z!1M)Xm91v|M% zD~g()g7v*&`~ zLvmT^&^Ua`ypKvj0AG)Dbmbv-g;A^sTlE74-rKET?}z{lMP$ziClPk5)b20d73q-Y z>An!Eg3f!wFUQmz{@$tK2wD}~0jYP|4^Y?3h@71xSp2ql{=j&Ce;(h;bs${kdk=zL zKSn?{0(Sfo&XP?6n#&O5mu;v4xVQJ4(B})~6X(O;A2E|rzbDp#@m$l5)ZrXe+I=C6|&jeC!+9AC_S`Hu|4VyH{Ad24s3^HVLujA#>}9Gdr~$ zqF3abh*m*{kKo-_+YHhjMqV^8<$RyPQfhUpu65Qdj9);$H10boj!uFI(XH1J?VQhY zm7WU&X<$x%fQ((eAKh6UH_EI`SN(*2xs}xgch66K?~aA}Dn#@_Fr;(5x3m1>JY2N~fiN1Oovg zpcG5;Gc0}ZzeL=BKE6fz@XFG}U+y)|q2)P|Y73f+0`j25rU$=d8=qq0pY$Py8<`No z0^T{XZe<}~q+pp%MOPb)KA(VATxew=iURNr%2n@MJMNaY;>gNJ zm6)6e4wBMFU1*<_2921?K5|pvNxQBf{vI!yo+ZoC(EVv>{d+qC))`S{vl9p&8Ju_|vQG0L9pLFl?57eJ#aPRBP0OT4MMF1ve_D zx^|)j1y|NTLM!Gqy%c5T;F;^ML#@6Q06+wU+bN4Bpq4XoX;r&c-CB($Gm)$M;6uY& zjWx%)1ZhGB)&PR>fT2m*U)TTTCz zR)wDi6$~FG4_ZJaLzgJ?OpG`EVSINd2WWMG4`XY8&^P^6O!b$&*)27i`cKLqr@Q{P zfCi{nCZVvU{S5f&Y@fM1RZjg(mtXwqIXN~E$I$$O9y?no0q;M^D+tZ-CuwShMlGkB z1;hwor?p81A>tjA*tYxv`KJw3sobkB$P;dgPjMU7?UZ^mZO#ijR(lqbh0@Jq7>_&q3#uakAQLbNX1} zA1~wcd2aC(lznm@I@Wm#QnsxA>qTOiQD^^ zbuLu5k-)OAcuDD5w0Y3_rY|Z7x}~YEmSk^CKe|95?8+aQPM|O zb#V)x-LgbaS1hExq`gc&KP7NA1!5#FXn9Q^S@6XLD08CUzob=wQ)%RDSir{X5l{|- z1hX^5!RB{WPD0m?F$R2IRGjKyMP@nh3r>}_jxE@jdS@^5Nhb5wN?08t0qSl+-l@Ck z!{_fj?i;*5!?Jhy{DWwn^^E(B%=2*)r<}dS|JZZ-!)k@3A5cG_dxBA?Rw!ww zS<;Le`MP32@7gfd&H5z{E#Y8~tH zVRruZdE#-cf%lVft9QP{18_+=m|S))ufJJe;*H})O>9W>9ea-{wy{Jl9R*_OtxRz8 zT^M6b2x4@|JPfrShgrm`bReWT^i49e@l5Kh@S0lH!KAD+H$t;{L3D0J>`Of9f=Bkb zHm~Xgr%teeZ}GiLsBviKg+0OA)9L=wZ->>YQ`51vwCh@=@9J3@1{A_Hy7d3zc0u1! zTrJtW(l4mZ8zEc8zCSb@ZmJ8{MW35ujdZOVp7E}onK>R_ZpJ;Cm78l<^UHb_DuWE! zr%ex?D?*%)u!I-CQ`BE7;IzpNYQPm+gw__OQKHNSv*9puM>9{qv0!W5Re+d0p)0j) zQVx6-ymM}gtopOzvZsAAJf!)MqM{&980wYWf$u>$!>yo!cRN=xS`*GQh4U~ zm5!M?V(pj5KGfyzpF45NmQrw{dUqn?8O?yeZ_LfPw|BL$n-u& zgP&t~7WPVFC|-JIenqBGC?I4H*aoxH<@;CVhuh^xPFfbq?=~KJkSnXQtliTmk4fyKPfPY7aF*yB3_9z0ojleue47Xgna*tu(vgRA28gmDm$ZxT|-WV=T(@DK`r(PTs z`t|6L{)|Nz8Wl6+zl+5A+m|*w8VU?8%c`j4<_hZ!5^ylO=t3KU@U8!;8W9eD@qt?! z>v{#bOF|bmH6Ijh1*(583D-9$@1d;j_3PN6vV-dP#O)HPJS?r5j~CeEPguDFCTGZy z_eOe|@QYvBJO{wm!|ZB?5s72zU9%agQom$MaFBki6eyjZ>KJA7~TJxcC9Q^|D9WD7@6l^J{0dT zfys7?6D2)AK+gu136;;I+XHXyBxLbbsat&`D_&`xq%?o~$&^)2k4d9?QeE z9grt8-$OOgiC~RFW{PcUULko-Fm_Xgqyb=57Km%6yFXKwAhmt1E91%ajw`mQG(iek z%sT$@mWRQo|4YIuA>3?27aarTqGBp+CxUgr4i#Y`dF7qjV~avIjxw6%b7fK+L`{!G zeJNN32n?A4@S<3fWEx`gSJ33_84ZE*GYS8i+cN%~U=qN+tLTBUSBS5$K>f3= z|C@Tx4+-bz-`ZYrhI|#w;(Rjt51fDKcK`Xmzn98C1l0eRZU-m8IXZIp{b=m_k$L%{iNL39e-O#?Of62Nf-h5ZmI_pEudq^JpS5%>fj$SQ4?0omR&kj~Q0d zKLzv*DP)aO#29}?2PrCa`NjRs2iW^(kknw2b3$8FC<_Qn8!JK;(l1KN>gyI6-cAT{ zxZ>B8-S*{X#PTVwjZxU6a#U4zue})QRJLr5g7m ze7fvuV_BS1CV~<^AC){=BvG0(u^OZ36Hqf_`*4%gnEt}k`Uq_9up@S|APfJ z`ELO}I6zrb7E2WQbI?+)9=(@(I*y5$Tp9+Fs+ow|#Y_vNN3W{MJU4OLjhkLFNxLn% z;5~}x0PQ-h0WRkEL!$yf0Y#a+zJd+`KX5UQ+i~)oeap3r-Sd5kxBCl74~CZkCN;_i zdTVP5WUIkx?lI8PQBt?IDDy{%j;qACpXn%@`l?c<&#HSHH$z$ zBQ3M^u8m>*gOq;MWdQORl{ZGPm_$t2BEkZ_2VL;xt3+OuopdN5D3}58+fYqjkj2`u z+RF326(Ze{F|_)+5ey|BlW8Tlx%ZS?ACv|%&lFbBO=q&!d#RRcr_4=YLd{u43vgFx z9-3CdAXNZ`2etrLA7snn!ZCY4K9(V~i^a7-7f9tlD46g$6N?Ze-{vcMUz4FRQGon1 zERV0*l0Xq`t`5@ay+^cYN3ed~=u?3J*K;!MTLI-J((S?cV*dq63)iByr2ge=12H8Damn3*uC z9W1@2#JSOCQ($_1%MoO>jxRc_D0{2N!F#TD1{V|Kh$4|?hKh(~-!{SOCSaB%2=RkN zHp5I5ywdp9!U@Bg%xn9nq2HiTBpfClQyMxlS;-Fno_{}hBwUx7*Yq>~Yt9Y7p~3qW zXMKJB#!6iSfb0Bzll>^`o5g;=%-npw+B*|!h|Mx?P?ipiuhY+fGKEowE*BSEjE_TI zR0zu>NwgzENrqugJm(-kxy@l|YNZ7ZuTVuYY&bv-Z#OU(Ox_i#HXf#-Hf_jL+lED& zg$$J!p8C855|KvWfwe@;gT>_VvoW1dNpe=64Km^eME(HxbY(Q~PCNp%+wZJPAHd{O zNDCP>ki^X<+F|yP_MVV4ccUx*+9~5$D6rrfH?yt*{dy!2scd#CFA@=12(y9_gG`mj zU)E6zNlqKq6%8l*Yi3_Fy~(prj7NPV_mlb|F8S25G~%*%5q-}hZKi3MOEWlbY^Tpm zyk&d(?K^lEpzjz@$E0nWNGLTaKY@ zaSN#~$mv<@3CW(BgnX_OteGX8M)JBJ4=~X3>f&KbGw(QVTE)|G5o&l020`CzC-PbW-!)AD4zEb?eiP@OT$xrw4VWZS&H70H_roh-uVWU(&6(kf| z11s*D+FxOw>k$@#nH*&hK7%rJfGandKGjN|T zOFO~k(bH>$KNt%P5HdjMpK%5N1kwxbuyIl9kr$^FAX0H1$*$#NR%69wEI}H21t^Wo zaq7#=C(X5U>3q_!*G@H5>Zw<)!dd408>p)*BF*MWmhE0N6w+^!-#l98nOA#`y*cZa z8q{Ll6DV#>D6`=5l}!TpkMEYdrcqX3C>DZ7IZaB=^8OMoDP(76N4w+#xJ)HE>9>4Y zt_m0pP|DGOHDzh+K;Q|NY^{PSR$(P=EZ=2Stp6rp z)f+KaB40gbJ@C@9`A}ZXE;e;pwLN>W=H|tL_wZs0k#7}jh2!yzl?SM_hE58wbC~7^ zOVZp!;7m|D@lt`ynK5+(ruEa!6svj??(AAII9N2xdloHfIKJImfoc`c=%_^La3?KZ zw%HV57)z1IMXn=OQiQ3;-J0yPtn-H=>wB#n*`o|-3@g|iBUUm7Dn72$p?Ip~K$)Hf zr@*WcQDMEs4_7B1>T-ywrh$ue{HG}eH2$K(Ec*eaH}Comni3_1Gg7ruLK5gsuB$Etwup1G1nkC&PKiGz@_(r+OSvKfHM;2fJzW2h^}A z0~;OPD5@$Vp%;Oh$5!3Pb^ksKlUjyRTJGfm`3BZLB2v-j168wO<+RkX(5OML(=Y1y zPo0UK!2ZucQ2>RbsnztFjzER2N!o?TM*AKOJ)nwCkg@jTD^Hold4me+q~29HV_#P-SZMWSq7PYdh=dykU0l>zNrr&y3>?RT~ z0`%GsqG~^gy=?Fcs~xNy%~=-_{Th>g);uy(ZRK=t0XWnpQ4OujkQcl999I*WwQEp2 zv=_UE-oYaPPIW97qiHlAcv|R|ygj#!9$(@0PjgkBOTt(Ft`$$+y+yt7@~mPX?Q98 z!D@IyOmqk&LU{zgTNGfK5Xfvu`8mdW7;f`6Ls?mcmgzbtgz|aex{z=yr(EcII%N@P zw9g+KZBN;1T}Muw%sd%VnoL5p*dvSGfS>51XDzBgjLCFjSJu+Lcm>YkN|Pg{eyFRbF1fwUiY(vAYy7YOO+7F+La;~yiq zWuWA*szIFgtN{H4qU2ATeJ!~1^a)IbPQ5nw9{}>dC2067k;6jCuWd^F?5qO(Aa~Sq zlmt8niz;0lp_zjIq(a@g%kBjw+C^-Xy0qHW1SS z>KfB+`aD%wEPp>CYJX;8C0W?_4WTDD+O*VhvU&vpUL1ieHtd+Iuxtpo(AbKMKj=I0 zo>IDi-z-ZtuNo{pT~~1mx+u*XY=*yYg1g}w)_^?|t3qSl4$n~MEU0m#bw_i23sUYZ z$UuwLE82?}LK9#xxhZ=tTHy0z?%<)waWCzXwBcIr`!;q&v{<=--_ZI;Z(aJj|7lw` zN9cc4yuLRCNpSy9-zsUwZ~b>^X}7}Y_wN>T+6e>x-~FWs%g-2y|5<$+iJmo${r>R- zK_z+H_$OZSAGdF9==D7`B*06_bm578N{S(VL>j`#w>5GVz@0 zpFu`{$m3J#pe3y<8yi=JEh!fTT$2qnHOLc)kP~Sv8c)v@ntd8+s;kF}t5v+c7gS=d z09l?#lSZgHrJr4o;Eq?0*+*7W>|c(Pj`Tm6k%PaxCe<{LegRau zj%9Vv_0p0YZ_>n)dE;t{qRp*j?2B~I)-DV+9wt#z=a07z!@&&?^BZIBqInUapYv%P z7-6SwV$2`$*|c}_dI3F^85enZJ(V5H!h&GU4oTrbzibYq&=MJ>&8##wr)Orn+@1SS z(wA5oSy-B@me-wx=jx&!na^07SsSz$2m2?aUV5${FFqj@wa55OeC5XRAYc5P8GzPL z(<*&YJM{<@?KR&H7nv{8*H*C>8(En-wx{hRW!L^73@H63qy%uTt_cSXGGl2NWAmt) zMUn<(NC~4`*lIMlwKn!>zztEZPur<04rx#`E2@XPX=+b{$1V;C5^OEz;PoOmRH?_O z08^-SwHjIz`YaMCj1!-*+z<3__)fNgeGsnlR~`TXYRdDA$I#{|T;Obsy~jAWS^~p+ zY?oJ^Ef-lTEdjPR7LsASU%&xsM_WN9u+Fb@$ZTxOjs6M*Pign)i{BPKUNf1=Lm&3M zbJNKD5Aa#C;&|m#O-};kyhSH{f5;i7vfMNf$M|B>T=6O5*KEy)nP}Sit&#p}JMRna za?|sI?n*;I#sef<97*SdS0JgNhXiW}ue0RBX%|~Ti~wPIoV&F&FcmPtt^&IR7d7`_TQY!#eDH!cZ_lMyLG5Kx4DcHSgs;X=83?OW1c+djR{?GQ zp1^QNO$+TP;ca%Nf4t-yCdE!3GQ5EpB-u~GIqBdP-@`bfRbL@qUL5P5$TdNe9N}?U2o?`RHYGr(f zFCf+Ql?1qGf{}q)87()WtOlzg((CuIl4Dm!-d;&^Ol)lp$%k4GgJm&i&{p7fh6l!+ z2MGJXInS~bVO)*t6xncc?HBx*>xcehjRv%xxiE1!Y6JI^+-O=Wjg;lb^)K03Gspm! zu*^Kp9~(5l(;Q_J#`JK)T7v(m$rXci`8y{w+c9&H0T2Mq%A9Sb=g zX@shH0+$kp4GUf9FkYN6f%kz@3LEQOjVn2vCw;hW{yYZLzvrazjLRnFM>XILYIQHp z>}R-h_Rmfrds{di@b`Wuu$o(B21oI~o$TShqHkDE4=`y4bVZLqygW6KLgz8G$OA;A zji&kAbp0-X{4LDWD!hiJ(^ChJrrjn~Lc@PWv>wY}^NB9E)-z*UMuPiG-IXa?fH|J_ zCd4+Wfexq3kvhlmXXnX!9>?Ef6s2CVk!d>(=L|TNhJ3NjeFCT>jM1&-Y7UDuw$(Ao z(YOP(bTeCNrBKL4EbgC#-ZM)a3jog*lN41LpOXG$kopGU&y2Vn5(G7klwx!Gux0re z1{Hf?+6F-)cx9{i5=+9Qa*At~upH!Sx7gmeBa^#pHi;vbS$1?}lVCmN#U_;4uo}h= z9{j2e9(WqPJGhaG=`bFlT|_?5#`94l3$UTTOUMxs#N1Z|bqm?nX?kqfw*lmpp!BdQ zfoi zWmIgWKf1hn3SppqKm-~sT1&0{{Q}vDYv2%=ag&v01JdI6wyJz9#&UE+q`$c;4%Oeo0;9s+0zkh!9LR6#+dP)Mf|Xvz79*mU$D{U)@Issiw#!NmWG;>S>(0Ay)f`jj*uSn#Q;AzNa6XgHKm&b{+~-uYu%$pZUATyvvQ5e&W|=U ztQKy@t(n7=i9U8jizZlOjhKyJAX+r0Q_1;hI5ldRYw%5KRF-rjpB$+J1PB)9zt}rG zlpo=Z!8!HiCk1g~ZP!R}4Be9|fQR+a+7Sx0{@_QNSi$~b=sUWgYC^TvCxTxyCFZ9w z5_Ci5pMqB3*(JC(a02vz!3Dx-_4a`RBe)bf$h0!GX!%2^Zm;~(oj;Ax_BB&aFmXdiG}ks?L$g1QwG(ZO1TBI5TWZ94%s zN0l}*F3ohS3nslm{@c~`!M>soUR0-guHv7ULOQ&_6ySlMb_^(lt(x2EVJxutE)XiI zkmvTkcIuLGR$G{^{`GP8kOwiYdA;6tY{i20R5RVApBDsUUOM zlwT;sHH(kkF#u+hgm>MNwdwSCc0B@|u4oKKbodSH(*p{UNg$x<(&V(@*tZ(7*vBey z!co0UFoa1_9HB014dFF5+t}D59Ni=PVP4QQjLxhi$Nt; zUI5L5)v{Kc4w0q+`nRYO6-w6q87Py1Y8?qSD#r#>j9GY~%z_oqSV6JpSl!A68`ag3 zws%Mhhd#!3{v>}(3eQZCB&+;SV-qM2Z+5~K(Jg0}grRyXWe7-(|0MY4BQ=+AiH1&My~r#dXLb zQ64CvwMu_0eR@!pWz-O-<`twtW41byxZQzcK@z8-{Mxe0&oVwUSM*Ag1r{R1NB|-5 zxiWu)gUx4cur8V^+1Y&{#|MZdkJ^5*$>>KazjP7K=iIjYA?jCh&&J_GHDYOlq|B4Q z4;k7S~}jN5mlg*NDK5TB43_qQhTZluZ?swVmoZ z73I|8R-4cKjHBUi|BP{453=DeL%?;kl>FDy_Kk_dYdSL2>xA!$-koMqnJpIAT1Bi( zU*}T_tyEI5v%c{)?}VtSb27DxHN=JFuDQtVh*Ou4XUfO50iS0Hwv7@j6BP;FT1O@O ziZ2!go-rQz0Ru8+V4ikv!DM)9;Amb&@BGrf+u1%e*w4hXwKbse@c1M26Tm24f zcN>6}DBDG0ao}TI$fWOI48Ro|Y?^pI{O;-RVTYbEEsHRIw3B!gW)xuIJoEqECSwwfvz?+%?QCQ*w+iM!9LZCI=)>?~d68huAYCV$}B~Kg)2PDWxYq$#$AhG-V=@;boBKl$clN_o9VckrED-!%13!%MuF_>t!nR z%9&SMCC6-VN zCgufPJl3z`j3(oH7MR$35mVsvn;7DwjHdV+ES+ncbO`z@JiT(BrX(v9h{9mzJ<_EL zQ`0d#Zk{j+^!!f-xtxqo?n;xgd5EAU-OwLt@kTTV40q*@9RS(3KUc|!0f!yX?kdD1 z<0uTOU6x;*OQy7tDt7@3VXhMt_S32OoVvq0^kQ{r*W{t?U5sYl_w+<3dSiZFWN`=1 zd=oSPCiolI6HUgu!IWu)!Stn&uBJ6KB zuGq3cZJHZ|trvziz$t-KBi}2GsZZwYnr>B!Kv{e&O#rPbj5A+1hdg*}dvai7kKvg& zKf4^Q8MKw?P0n4Jqk792ZK6TT9GccuI4DtlzPF7xxI#7hF{SxR*tiqZ=jS%K~v-ypX&cq2%COP1a2LlSc;Ku_Gd-faA;c z*3~7$RDiAgpuAgDq54*1L1WsHR=4=yyA8bb)aojd>@T{ZjN{sHa<$X2*TDA+yb?vm%A(l;fB zR{&Id#Dx2duo29E{gUOOO}GNuDaqlUgakX7NlTp#t?{&ZilSGuV+T(4-!GW)6u~Q( zOkD-dK(u|SW6C2NAgT5xgj`BQ?Ji~r+e=d(zq9f*BJ!3Syxx1mX1n`hb2Vd@TL>R{QemK_a+&+QxNkFF6OQ#_-x^*c9)wh&PTr_FY5f@T)Oy4@ z+U%9^a3tfZ=w$)?lC$)&oCVgWEj*@YTi74<*&4q`?0Ez_fQ(mE}G@Vz0 z*-erUsBjvm(J zfZcF#^OWY~E{7N>*;&;cDqYT>>bRH1wz##e=|NcW25gL7vO&NX1VuX}{tqOuj^lwK5WQ6Kzt--CYS6-s2@14Pee%S`KLFu3yLvN-Vb z;FXShVl1zT0@i+T08H=HjW<=vx*4v9bL+tgK7lo)El-$1pJ?2MER)<-1TKFL|2Dt{ z|D!#ldC$USL1=KsI!y9QXvFJ-2@YX5A25w!{rE;DTk0RYFsr`2BR{r$STWbako_@t zn`hY*cr1N>48V&`{|c~rLmPjlJp6)9y}gRrweH{K{lQs$kWI?B2$C$H4EV-L zQn!}JD8T)4dJL|omgoT&A%>?6uD<4gX>F>*FP41tCVO_c*g1Q)$fX1r9mgKatiwdv zs9OTbloZiB^zaR{23s1pB+@_Dv2(a zb+`0reZ+4)aP{CokE-o)i3r6Nr`uP?2VlVNEC84?G|ic+7kW9xVd+bRc&h8h5RnQ; zcZe9_-9hmkzP6ChM(7ZfK+*8KGO;%vv%5%Axl@nk={yJj+we)xg zb@q6!>kP`N7$DV{W&^PeduURc5W%2eiVT>`GbERQQc%y&TS#$FM<24xcy(BSrc%`@ig{Lg`!m$ipnGcT}x(g<9lkIZ%iiiGq|}k6sn7s}0OR51jCLDZscJ}4! zGaQteq|bsug_&hZy%v}kDkk(S;p-)pFEG;fH_kLZx_Jr4cZ*cU?CU|iC3{=HU+6+! ze&8`V;<6o_2x+iY>3xAX&4~Nx?TP!C8RGG3jYdD-n*MRfral0W+AWO9KHVJ2MB)Xz zJA)i@`xoy$bQnH{?Tw(!MyLskWzz_FytIP+iPrh-MX>K=e<>xLWTy!|V!yn72T_XQ zE#8J-NNUd}imWZTus&3}^C-yLA$Of3w;Knd%f3x&AA|OI83QNk$$jx0*GvVE?efUb z?OAa1et?xa-LnNikC~qMi;ikxkP2TS(2r{ixLv?5 zQ@Jy1OOs7k;%-7qAH8M z;F_t9ZwI4*7B|r~?REwyvE&A~9NryxmHH@NrSic=4kIafMAz9mhC8La6@IWuQ``K+(cq<$j2PdsDYF|W&`aNU2G zX%DKR0p5QG)3Q>JxFXD}hnZRfT)kD3*Ch9E_%S(TLY{AySRZ+b823LFS^>si{$vft zJ{Vs`*#Njq4V++k9mWj_S70<9tQd_a%`6uH3Y4xlSq{nug)R5IyxRgjgwD`us9_4L6hR)dZ=67GX_n}jzS#4UyJ0^Bjl1Lj7V zPBG_#>@m%Q;cLk&#zwwRX{4HnNYC`_?`G03rbxblLqDfFaR6VWk;q48lC-Imq^L81 z{5RFur?}>$`YN-H2JM${T0(cxlPlvn;`dtz$wgJaBopk&1SH)D&4Qh+p7t+3lv8QU z>Q5==(W==X809vCJ`FMQufP2cM952+N*uFmy2CAEX>Z-pJYw;+onLfME(wOU5m~X? zIu+3?gUb^y*5LBc3-=U=+vt_P{xe>H$)eW#pu!j@BONQdIJ|Kb1UAt{M;S%OJ_L6ko}P24eJBq@Bmbv5}<%0h3DcREOQbj-k;Po2Cm! zCpxT>oW$VN4{lZyJih>NQmCsD#p551D#+r8NZF(U1MPq~AB4^K2e`c~uZnEUYuxA8AR@)?rnK|)tUla`4@0>!JVWqXQH7+agoMQ&Z+_DmMjtRz8 z3=7EE6+9(s>VbdGH2}#d`yw|9XhOj#)V2F?A1(9Si2fPHUQ!4I5Q-7vY%r}N6j6x6 zp$GfMaw}bhOdE)&;m2K zDxA&(Ew#ay+5%w!-;c4w_S}h_Hh)N1dk1XtFo!0lj~4?Z8_iaRPP<_1#?Y3M&NZFA zNTb&f&~-8F)_t~#FtBpx7D(q&g-3o&a@nxzK9XjcL}OussxY_c*1FJuEc&nv$derd=8d!;wEuz&}?CAxbRk zsV+O|P+ZD#8r>RjLCSCs)}mn|IJxUJ1M6y9FMEw(%Cn?citB1u3V*X;CF=EcyPf(i z3n^2DN3_$7BilDHq=w^l)eLyfZ&HObz732Iw0TQ}d{hMbcIlU?c79uJe)C0Y3JPVL zs%18(s(9ys;`XSK0H22)L$5b+inZlp9*2l7KwQwyX_et9Crs!|{)fdEJr8m6LgDG& z$7z?L#iqP-cX4vJN5oPp8@a(azg{>xG1;TYG%*uY_n!Qtbb&ilV$$>g**@xwg@#1s zF~QQb#(VeXj;ysv!tDO`x+rt0*T>c43Yqq%+KgsM|SG8%$@T5kamm zfs%A!E_?1WOEdNa(>de>DndAnm2d8*`r^3|ub^&I61<>M>I}>Th6%dZqRA>ZZ9c+~ z-*s+iPd4%!M4BK(P9On3qCtyV{1gbFL(#-%j2Xf)@!pzrs%c?{w1n zf2<_lY+Nfk6!;NNAE2e9byVr;y`VsK&kuh}I0W3Q}(2kX~ zJaLCm!lp9SZ=yu5Kg>sJ1>6bTeni!L8)AX6LU4lnf~tFyRLxBFY9<%D8pOpkcX^T< z%m*A*2endvr}AjU{?5(6K4$sBYrD3NVYGdQj;M#3n@_eCi~4}s$vnHK5*#Z{AQ{64 zD`Lz^I|tHTV(9**>~)LDz3YAe5NjF5FnKyO?pJt9x8WN?q|(vsFz*(e=H`8J%vf3C z*~_|W43Z7@qW8EpT7wC#Q7w9?MhzCR9hMapiFO)~GdWkGgW4l1=Jn7pI(^wPLe9ZN zmbGdqQ1MtGJFe@ONYYS2^xOW%>O~nJh7d$Wd&HD3wHS-IgiRD5Lo_@BctAMr{$g~P z@i4MPC~Y!Kk}!A&#jZ}*h-&kNL0S>SHnliyld^C#OIc^aaS=3y#ZayuuM%T^{LXk$ z=;dTW30q&J(pe8qc0?X(H0}tNg1FlC9`#y0HM< zOhIZrRaTk&sDw3A-^P#+NT7I=n{@i#$(kDw%7(ii77ke7BfSAlF9*`D(u@KLX2A}n@97^nP~L1Tgay;pk!`_Xd7)9ddIK>;o5fR^m-{}gKxhdtar=s@{n-f$7``E&}!;@ZP7Nb z)8G;=<|T+N_is2HnLP%5L!Lh(TFg{_)OJwp5G-)D9`IekjwlGvkfvMuJ-Gg~nfi4} zt8Q+QSab(%yEp?sU17TpGCSP07KqcdI!~9AfxA+Ow901ES2K_(a-~o|f z!{fKw^${c)0TLfMSr3*8CV3kb1|!)Azy=P;RMfVeqeJGFX*U^Tc;#QZWp6^rxd2OL z>6b(hC=26r!k;l0>u3#aGy&C=$N=Vr4h@Y4_xZsKWuK{n3GrAX5_Z*i)p>-Ax5f+b z{+1@9xnf+3jwTD7Mf()xFT*9k)Da;ly!DCU4(xurm*PH>Hq3tE`$?7euhsXUQw#yT zdOuRe2)_=F)j(H8d~;peUMb?^q7C#7ef2M-T0D#V^q<~^tf&|g+|hpxrT$v*d{HWG zhM8F0)HQ2)~XEhrwHKfgv`4|YUqe2OQ+FdL1Y)qDuV7VeFa zNUYX!Z158;qIp!#EL99e8eCY*fDH3nB?otSe0G8GAP^@G77XqyChmuWn9I9FvO1@| z|5Grxouvjd@U&^Gr3KO!BfUlV?W|O8ni{L*&rmQ| zXaoniNlrRlNP$~yCswFY(=Lp166mgYLhu_yh_Xb)Qy?y-#Su@u0#Rp5aj6)fRwfLU zzZkq+fgTaWNZiST0`)_Ap3Eb-19t-2Xe_f5X`XY9)Xb@C`?i>?R@jh`mTV8I3b@r& z4}En%dAPUEtRv_U=9+#SyTq9%{}biR2y@GBt@lh($~BF?k4Y@*#DF@-A~8gUbgH3N zxJitoeRz_(HAo~6P6~}*uy9XaKnVJXY7vMiyxlC1eFQo;PPZ5>sCBY~QH&6PTzh@#jqkka8D9 zobc1=#>-p;yU)ECO;XSvF}#DYgveG3Y-a=^xmQp)0DUBc7OkAp-#(Ocn(S%Ft)0rx3-;O9j#; z!+G2zX4xZ2l`%FCR#BpYV++%94G}bbT6sH4N}f9+u(>3r%&dL^7}`ONTo{((BGQJ8 zk`zN4Oc_Bxb>@E;=+hjc;KM;PV@FG*QvaqZ$qI5kVdLEiMAL^oD2b7tOki>uQVbAY zh(C}=a2NrI=2oen2_wiXZV%blEvW}MBz?a_I1+2N!K_8`sK$-d1PRz~u;aoSgOc+* zIM?HiuPdS=9K%M!a6at?pGT@`L@sgmgf z%Wc4t_cOul7(pfMCDJdQc3NInsF(5gjTAVdfkoRE0K%Vi&L3iXP80Xfo~B}dv?uPa zsqz6(ER$F?iMiW|%!aEU>(^`j#iU7I4*TmpA#d$z78>jg48zz0p*gfto{Z`!A7w7_ zG?n(y>Rzn>Lp6jldm;}n>Dquj=sTl})Y#;;HmKaL@rNVq9a*j}Jv@B{6OmNonO!HF z3rMm|v$uhC+XiU%>`!+In@`7O&50J{1|ma%2(MzLrfg*d>4Q+b%}H5n6;^VHa9qgr zJz~#FH_94u94*}vwW4eh%cw8Q7rHS`&=7+|(K2@tu;%OU%b42^#&VCKk}OZEg^ZP3 zVlQ==^$FFAmJEK6(cwoMacs_7B>&%zw7Pk9-Z|3EZ5u7w1RL104o@XE@^$SsYvJI4 zYm~0Q{bPSDxTvw=6GWX!;c>9;uug#%Z=})G_6fq12T{^9twzUu$s>YallZbzS9N6=i;a9$vbjKEqk7<+KL*u_Z6*tFId` zv1bWtTn|mr3%$?x*B68&p*U?{FA^Ed=7gF{roOkOB$mSQ_HGS01v=AZ6~^2sIci#z zu0Y4(!}|szTqe&Glx!1O1Y$iEmW!n{mx$@jQS(1fqhb&dHB$$_br3r(v*Pfb_z$)* zeOk@{)k)PMyFbgd+ldZ$*$=Hf2mDV^X`MNk?o(Fkzh%_ecd^zb(xUiixa~f^{y}3^ zcI<37jrAxC>r4*SaESWc#z6=U&41#1T=+g294OgNJIq2+GFV$cX*OgVKKb6Ky&gSk z>k#8p%XN@bTW)^~(c7RgLJZ04H=OCD(ZcP#oXS=uNABbk{s7mrDCWX^9SdsdU$AC%0GC3e56sK-T51j!js9_5YmbVAnH#q-YI zL=$|f(B=sM-12-?R3_%&)|>T6(f85ZVS8_kbtfk{1Q&OZHb4t7oP*Ej7Zl;-bNIRe zK)~t27SZ%kJb!4N(YUctT-exBz|Q)T4GkHhBpbyXg~J3)rQ*4vcOx+4dFEx{8JR#v zbB#7QM#4#H+%n8cV#nhA$sZ$PaX@+KHW#~ia%L`mzPBV^nYeJvR5`ZzVPrmwKa6at!@Abg!CkcI0GfW z=ec2>H$uibm8f!ou%S|Nme^d(p6z(JOQ-NyUPX$S#&kL9T~J##$wjt|Pu6ur{M zNqnGc3WO&b7%;-bZJ|J1%$S4q5WAQuQOm%ZDPGg6PjRYLa0wu*PT{l{HCSK(!phBs z*=)b>!@YAAhQ}0mQ^J7tDA}wGrb%_jI;M`Wf&f=R%zNoRB-2j0`l6HucE3lCdUl$= zrbJ->f`eX-TU?Gt@{^0lFC!GQZ>!0VQBrXIR;S;lM31(;%+v(;+mT7p+Szg)=?~9J zni9#XCQl?SZBPPH%+AWginPQbfXVK2X&Fg;Q$m0>eAD*rnGdsidEH2_4aKA!P1|y} z=!r9XDXij9fNOz2J5#n1XDnQnw61KKoJHPO6`JAd5iML5Tgqf)6JuHn5$>u-IgJPz=fNAq# z$gO8RduVw+BtAxY+ z1LGUZcXa3N6LJrqp^w`S=nbN}uUK9bzt()=$172v|IL7g-xCmb2>@NS1wp_1VDf*jpswz?kVobdNy9+th(R=2A5&y^X2mB$r11W7hU{|Mo)t z`nkRE$*#?~>o>1lI=(DIkie#~vIo#S33$2y@H{?@7!KzVv5T@rr_wo~?-xw<;>Mq-BusMV1 zljb}1;U>!Q>LOAkb;LCb%wF^&e41^==`sgL;qhb8TAc5DHPi1aS@w!LO z!ljoAH{ z`Drpn#Sb7O5l6mrD856KSp)dLV^GR4o}G7z-zj4s`Lgx~06*(y+?8$^vM5tJd;Lo) zj&sQgUf~3qE6REW590YHJYmMthh6Va^41{`3_I{7NS|hjU&-+oaE}iP z*r|E|1Fil8a=Ju_AIc>1eq_FJz;s>$ifk%0mMJk_j9bm$r$xLIJC5u$?~s4#_`ekP zzv*=SXD@5j>GP1;Cq=IQ3N(oMMgjCk;)DUV)1eal-IghY_Fha!{}UD}&_K-(tsz+$ z`GcN0wH#T6xfXqQ--V}128dn9bAO0@hdm`~-#(w2Xt4wKgv1|tExRhXH>j63K9Ti2 z!L^xnJAU^5`gRWKPreqtodq9y%N}dEqRGnl%iti&OcKjuAvDGWt_Q@&- z;FJpB=J-Sx;vqhtvBadbW$tvXI_e_sQ>*H3FM!A*3gkOJUWFTtc!dBA&nh3?Wq1{w zXrs-8p`UnH>(&-U(A8rqD%!~%*|gw=GmVW^v1)RuVZecTDnvH^F_Zc#rgsC4JwgjS z*;YlbT+J)0&>Lb(NKv+|5_`VF$Ieot7K;bmdSbyyYYuylpyh4;Sf`-`=1!yv<2z^^ zYM&k>Bx8r_)eZ{G*ILdnAZ1ENxi!NXOB@THbY1xX)G)reTZ@%PYU9g|HSPE^&>kDH zDON{ApxI!m@;UVTPExV7SI2=Ti7b1Uf#ZAB)hT}}jG^%0jCzD{q7WDiBZ!@>XEXso zd>!T`qAKoRp{LAi2ET5*3Z|u8<0JBE4XXllk^dlD_RtoWbZOhqEBH!0FQmqshCcTR zJ@!Pq4W|2E$I0OSp8trxoGu4vj>c<3pI^1M1H>+h&~T)1-sQz|cMi*!8zCROeDy7L zy`!F@GRZpiyq({S#{%_6cf)2HUumocqM*)MFzCT?k;T9-(c94#Kl9(Rv;E?|p~K+!Gj4U}9NrV|4?Mu}IQ>t%!Wm(N$K$Ajz{Q2-#F-!nL+a?VWdd5rnbKz&J-mz! zZ;mRtrGv7YHszpqp)=*6)JqNzWCQ3JE;$yRLFQP3KZe*(I6}M?Bdv|{6o0lb61P}z zAA?V3H~X&es@CX=ToHK~YN{on;gjkoxm(eo!r{;Sh7YR4jU1VGu#5GEOAigL2UsS) zF<4$|^ycmGqk6I?`#qxNq(v2Pr|r}#m3v*V)}Kl0p9{@D%#y;9W?nBwR05u*KIBw! za+^c&bxbtNom(ju6QW~gKteQQuYY?o42%!Z@ceoTJ*fj zMg*Cno21EnBSDdGg@ug8TVovKW*R%N(iUbdorwcWMhqp<)~D*9lIho7--9xe=#FZV zemm)GV#JkrD!S5gSCLd33jh#~mV!FCTWb^~IJ{i$j06YSWUbO>9Pz<$?2dj#eCM_+ z6cp7yNFYeao5eA68QVeVz2Humcf@=w)z%g_mwXGzq!-v$7fgS;J<+2QrAHNyTAB+y z?UygAMMAk~8Cb7V(u)}>UO=YALg1hWAMpXO;_v7>Mtx*m8YIms)dHS)sX|lxeQ8EW z(|hQ3a2-2`#1tLY!9zQ}Fp^GRe()|h4Jzl4Ufq2QHJy5LwhIBt?xIy$%36gIWXWW) zrM#0ow<-1`$e0lmSYM@{GKFMIN$dcL|M3t`<|;{6=6JNAada$3Xi4~E8Lo0w^1&ob zy}PcLRQ`1)z2OphI_fApWq=Hn^-Z0?jzVqloG05EAkwte(4faxqV4UT9Q3v z=uIZ`?cgYh+?0+aQ6w?EU0B#NMt&bZH2HCe+TMW5o>~61twD<+!9^0r)H0|>$W0=G zCh5mQDPbXHMA}q*ekT2_90NpQxGA%!a?#}x6TYxGmH4gFnY!AD`Z=&P;-8$%KR4-f zGF|-t?1}Bw{#>{`eoOQL0^jg?-K+&VyNAAf`SKkI@)ZpWxFy911*FBHy7=dz~QgtPyl3YPy`mE zSWZ?utp^{h9s^~l#4yhw&my6t0@E4d0-Flk0qZP7nMCu!11mDV$Eyc0UqxwE`A85{ zTc@qN^3$GaY@z6q3R#R0uX&id*I6%;X%D`ZK_GITDUqJwStx{fJ{kn#z;YvmIe7xe zzy7JB6Du>h3noseMsFITdui*#u96|wYy8ebvb4pMkyZJ^@Yv)*bl-X?D=LSb<8>10 zF}SLCM|h68Vx|8aCjA1y){G&p?AzWXIk*k+8)d@Ot=1Sa&|5UDd^zt%S1SupF)yV) zzXr40g>ixyVb?te3mtq*X7zPSz^OObMPXnayYHQfC}a;!FdVWFiXBwz)mmLfwd-SY zuCklryDMt*$rxg#F&V-SYbHJBQWhgoTsu8pBO~k$Uq!okw-x{-C-5R!_k2fwf-8L1 ze)~~%PZeLTL_O8PI3*4!rwyS`KDO1$`+x$&nzb#INT2tmXl6|6<&a zuq5iX#RIX478k#iV}MWAMSP#m4(UJ-L1y72y~_&9m~hF}BUFmz?{O3n^L}3Dg|Qd!H-hS((#z^ ztTx2$qYsD@c89U#EU#-8sV_xwnag)xemaG-_owH39{(_ly@Y-%0&7i@6d^x;Z4!&)dFI&NeTaf1lhdh@+VyrA1*I1-S7`(BTQ zeFL~u#rkEWpJABHc=ysRD0wd&4Z7__{|AoNUPOO1|7gTmigoSUx2Pp`q=YMZSB38a z;55NEPI90$vg3!$Ze^xoMwLyA=q>SkzF3cAlwHKN>D0i10csavGn}X;TWatG5GKz@ z_t-B%;MIEjC|KcU+myhc^9SF@Lg9fx*AU>`wEA9xi+cyyGsyXd1<$Nt9pDxhUjR2W zSIN9&KB7!DlTiX?V~hY#r>GvS_A5{I;+^6FPyo~v$)DSB7ljT?H!QWqv z>MArzqI+~@9IB7N=_5nDiVS^4yrNsAwm|xU$PRp|?+_R$dBdrWd?zuT#WK_BQZ9nP zotLsB++R{wu(Sy;7&HkxZr_MPdAk2ui2V;>KFw6Y|3K+4nW!n35kvc&g9ZTybclz* z1PTtLQUDauG_c+`re^uTDz{ij?DC9Ekd+Ei!P$P7B7+MlhRYab#`J?tmBNY9WD_X5C|;=sT)k% zS&6nUvdW~Nti}v?)AViHHrG+f^I5wYIO8>4@@=eKml*q@-<3{tB0k#mIt~g;n?YZ) z!2VfW5w)$`tg&TP%N+7MrR4!n4xSaM4;fFjy_kS=$-K}jN4D!0^El;pS{l?~}ebD%O5$#$c|ik;n( z$m5?3LMj~xN_0FH{Ei~BH6nT2lOn19#?AWN%&F!&*uJhCekO98$Ieq-7b07hD~DQy zw>y4(SBmxgd#qe|9d#2dk9!Zz-3A-ya6D((drNyGdZ=s z*4<|fyirTIHmlb$UxK=`GGAkk=T0>vogkPYs(M^wP*L)A%BPhgS?~vIr3T+P^NsQ4 zPdq>sEIq^r>pkQLZ#{$tMV9h+(c}TAoI&-44|cXh@Qf!*fk2n`XpJ-_{OTPx{Aw9W zd&wIL`BW^pQeyjaJheV3`I+o_8MOeuLSXE? zZ%faN6xwLL@I4Mc2359jWJxydY>LdhMXLJx@cW)O*I7o+v5bXEKmBC0@Th#O-NG4l ziRw3SSl%I@(TTJ~T$4md5{0ni=8IYXaohAoO5cG}@0Wq1TD?(x+lrq+kN$HfceNk~ z7E_$|Di^K&VAgo~bbeGd-U&eO(gEB!}wSHBwg_drkX712vEBRIS&F zj(PV@@li;YB1RW!(I_RuWN8hZNasGA$?IAQ+BVto_n;db7qE#yDW93;y0#PifELtc zMU2~sn(#^VhZ5eCC*3hED4BRjF@o4VI?|GmTcj@5^73jyRo|egcUr*E3(825;4A;k zF&c=3VL7@X*GB-C7P)F33Y557X|WVItzr@zKDS5*74lfsAJA>*22HWH?ebB@OsNKp z#385x&^GIGiFiU`Jso7%ZJJ10pc0aX@d;OHFI7z0A+p-e2$R`j-yHH0@m#b)Mt93; z7XqnvNouLh9dQrkyJ<&+5@?w2g z_X2eptRW3NQ4%{)X0R?P!2nT5sVDEg(7c(C$iyqC14Zg$xcua+sIVF#@q`$Ys=V<; zq)?@t7!QWt42fi_sO=wi^tM-FHsN$efHYzyE+INc7KeltvD1S#>rI+J~9KdVqr57=k-O*Gz% z)(XYkfV_XpQ7!xc`%6v!^&0<8P5%0mX2VHTeh{d?EF}VjDHYhwNM-VsF@8&;nTm-j z5QOaSo7JBsa*F?%75(?)OinAoO`kj)38c>y+BDoiVhK#P_3#R`F-TO^_uh2gHO zTv>nd&9MWd8%rA`7(^6qB^X%ANw|oB&vNvD8DvW1)RV220Xz;CN-O1qK4sCzelL~X z48UkO1mH@Lx#3WiruS)DDm)x;CF;qPOX3}Z|3PZ!<>1`Ghl!A!{`0SEG@azOyA{J9~4{?Qy5 zz+)7m4Xe@6iNKkm(D=KEli|E`My9=x}~+2OXrL1 z^v}WSK44RQm2@;XSXH7sRtSsVRBMs3GjYD@Op+p!!m>u;h%h5{?#U}W1>CKbWJ|!E zG4~3uI)nA=&hN@o%@AH}9ES<`X~Tgt6F`5D=mAt+s#t{)JL|V9^BHI|x7t53H{)Fp zXhV(wFAOV;AFz~d?TN4^T#9HPdJ8+Ra<4i?a+DYAp)*mz>PjY(aYp=G&z;HRz^@&k z?cHsn5=cVfyux_>R=0(Ss4m?!yi#eCy{wl-nqBD&=Mr2_%@(VZ2hk#z#Q_-UkT zyk9Jd%nvR}VyQ#3bK0eP6|yGzc)B72U!hwb9p6}n`+uE)(|AO{ zxBrIGN|B=ciNbDl4muKyYKiO?tkc&|T_oVi4cI0FtjK>S_^-Gvsi7cd)u&ay_!%-Yz~8etD>E4n|>b{8b_*5%r?Y zOJxj=IBHR7Z8#ceY`7|ToCS{GBR~W8mQUCkNP9ngPXUmHBKQSaC@FPN*0b{-7~$Bx zPlOhU6bDad-LN;ScpY@Dc|*lg!_8}aD2v)6k^h)eaR=M7*x-%iMfbF;J6(*(lJy6f zqvqNj#`35c7L$a!Jc7{mkBi3+tUezb)2Z1wZ~X^3xvtfRKK6G?h%q6~;*1Ir{j0@m zm%e>wTyTI%8KP^HeA?YTA6rJN4cLQBK|+wf7;?#}-+(lCTxpOeHf*A9w&*iebkQK9 zFTU|@?otjz>Wwc$J5jWct@h&PYlU)^`e_^e!gM%uqvE#RH^KlG>&{Z|d$7*AT_*L~xhNgO90{H5 z4!>OPTdo%r{Ws;AUBv>8zy;6{ZMjJlBDq~z7q7<8e6AA9x2YAJ^G-FB*odr+iLhMo)&=ESQ9LH<2ODmfFIPqC7a6x{}H96uQ z5GsnKH9KSB-r{qtBsuw4@C)G9TCDrp7JCaQX5Il(>9@bKJvMrDXDIrY1!;w*lhQ;7 z?Ef0PfARF+@cYMAL_@Kifcyixa871hCh#v`3}AtVncpyh-3T0rz%9g2>lUGag9#ur zL=!3!U4ap%hm-!os0P)9^m{zs;uqD1gCqhLJZgrl>MN$M@2U6aBIhOxMr+sm7J>2# z@7#0eGhc?6n{)e1*6qnIzc>7DBt20>B1|#d1JVXd5cEY_|~XFT`u2 zz1v$ScR-@hF%S!DYH@@Jz&B_xtmyy?TD(@Gkday2wE>h$4MrP+QbcJ^Ya^-tT)Uj* ziFC{J<>fHJ&SiT}EZHN_4MkD=zY2BSY4x0=iag}^n{UX~u#v;ZFc{Y>VFR4XWSR*>c3MXf_fy7v-Oqs~n$Qg1U|EKcRHqbX5zSvZVhq>4|= z&*ViH3K73)O42Awq?vlgClLTBxkO&D1+enH)NHni7LkvF!z?8;aMP%)YO1Axd#8L~ zOk_AbpGnlEzS$6f`xO@{+t+p3#QxDTd7gALb#4+K6YIsktE$VOB6$DWEW%pEx*BC#@a^G449NxUcaWvUzyj3_jlgs^5xP$)l9Ll@Ci_jSWqcSpnG>mt| zMTP|yE9li_NAT?+kD_t=`Ad`#$mv$>x9kX?VZ!PopBDAuieVBBZz~O;-$io7=u=>S zas(_EizAM?Y{RZGm?>)J(~6+j)|w#}vm-`sW~ozy5C&LyCTG|!1;b7v$f78`0z5D4 zHJ-`(Q>CuJ}mCF>P+WZ>jbXQgJH@@j*QAFb{8bcZCm4ZmzXJ zmeRt@?O8ONPZN|yRzA*@rR+}H&C!szEiGLWTR$sHc3)jNJ!I`h+KoCTQ$9ux%B4Ln zXbrt1fDGVISEi_3wX)6QZvjnZ1e5KHO5)w@kZT*z^koABYNkFpxh||LX^#)?hi3O{@3N;vb zUXASeE5?FusWxUnlL=!f6dU$Eub1hE${9^{ff7QYR&1Uw#T2!~G-L+-x-Hps!G4Y4;uPv&@ydW8|) z+KCR@$brK^$*1SYrIR!eCi~ zXj!5RJ?rHiS?|4mOXOY+%O4K~${qE*#CPqJ&J<>SJRzW97-AGdDt(@bYkiT1X7?X( zN}Q|miP|uLC$dmWaA|u8g(oiM9*fZp0)LA8H2Ed+=A)w?e|Nl&kTCyTTE{}%!k`fj zE59JbNEW+t>#pS~y!^v?!yQ$A*qBNys9d_2`EQ&zMIkG0fv=8m-#QoH@=#4ga-g() z?TOvC!mvWmrX0C>qqu({&3Gw$vW3BU1}9YBE)~_QD{pL0wZ38yzJ(Kop#x%^i*kmgIf< z!k^69hC`ql2;>5C9lB3su~yqpGJavNS%QV%#s0yBY?;Je>^jC`Vr5uQ-Hz%>i=`Pt zjNed{OUv|Attiu76lrbm)b@f?cwgp5`&+EoTe$~nxY5#@%1Grcp!>dw+FpX2WyzF^ z?t{Fx&S@a;ybYd4KrCo}GZuRJ z?m6wnOW;M*`;_oSl*puHqwqE=aZ9^!?@-pSd<{%;?D^~$R$l_cQ``^8-h5k zvEZTOfW~~%QN{OEz~`7RUezArF8&`Fo-$9mT>6iMq7b)j)+M8feuJB_7-k>ux((3) zb>RSht4Ke4Q1x$0-QNna2MMO~UIe2kE2Z5Me^^%Y2QF7ANG`F)Np1`#ytyFsJ>DkQ z`wN*2Lkt*2N;QG^X{(Y7jTuI}40$F-n~eAg-dvypW3y$Lg$ACWjto$+VkVj45*j3D znZzujNX(c@YTf_-ssH!a{ZH`p*OvxI?k=N1efeUI{2vGUPnF{RD<|szr^u|-@U+3x zz<)P$cN^4;J2$blZ?#3`A)`KRG^Vs%L0grQ@@bIAKXllE)#lcU+m%{qn>hQ4+8l!6 zrL^KOkC)$&EF+W^A!#J5m@i4CmcaQA5*c3_A1??>^ZtV+`LY8$ZV|VwhbHpoVcfgx z_Qmh=F-QMx5>X!%Py??Ymq+ku%|rsV^{!0jpbGc7HVv6M4M$wY!Tc*Zr+VNza&!+0 z${<@7l?eEZDVBt|2z%gs8dVQ}zKb$k)6y_Bb_txPY~*(G5+{8qb?u8k76BRwV#G3J z{}nOuV+|GoG;^QC6QR*tujvyM0eY`74#DD8aG+)wm(|`ZKwg787uTAQRfUqGPJKWr z&)uA*cs+;RiHo%)??yX7rD;FyeA#D!*%o|654=Jg6IRLIb<)vz#$tj;z4h5(V=S{0 zM*8KaF?R3pYH?st|BEf3dJ~s1_k^mm)tCX?@qyc;hPXrbMC^C2dX7`g!c50N zjf{_XR5_~|b3!qCe?L_q^fs+UbTk!bvZnf~ip_bIei@kUZ(Ze@_tu5<^Wb#sT9!d> zxWS`(g1pq3N}?G<`6aUv2U&4!0a1oZ$NJ3?Em>c807a53+~9~=n%~$ywXI1k8{<*K zyF%HyuA^1JTyUW7({{Gn8w(LqvlV8t zUEZ(g*jipWSoy1onzj9<)*=^f`~6b+*%P85kYR&oXVeK&rz5>`*q%ykY%&?f3d&6F z4HX}8fRCEHE?2+0n8p^YI(is8O^QW7Vw)VAJ=xwQ9DeDhmpV@#Q#Y-^_10$Sd@0QZ zN8A-naM*S8W?Nhp{Da~4vO?_dE^CqJ!Y6L{Jkv!6>%xK^&n|!bsc2^}c(Yk3`>>G< z6fB?8VoR>0W5+F&ryk^CGzYPLbb2X+UleF80Qw1)MQyF&*DBnhOh1hU5RRU{<;*)$ z7DwdAPHxq6^M_4Fb%+=pEjnW8PhDrla|h|+>-XJq>mlXpqu??cX%7Og(RZqMD~$H7 zJ=BGErG@t^YrmYm7Y>f2AzGiiYQrcDfjX}L=y9lPP*u@n^Nk&Mqm(i0u$QNi++h)4 z0XVw{Tf>x_)N{sZlQ-@MM#G#m0v-!>rrAe>ZXE0J6WWZ~eI`OHLncL4G16Po1>2;eQj$g4Hk< zuUh1##~QkV{Qk^Rei%eKw1}lZY_8;()3Y(*>5te{%DIvSbdEX9+(q!iPrwnBbAph* zfdLg^h0+VmXg@{~#{6-DKa)06o3qQUN)lo9ZHrGQqGiNFsKUd}ks^y~MlAEt2*6{C z_8ywlgB^@%BmU!pJ-C$A=C~@^T*eS=sedatgy9eTO*7fD!|g6GKl{-(xJW)~uG-o3 zRrBeo&wHOgH+M-S#;s*z``Fb>ueF_1|FC>uvnRBIsw*%t-{KgKM4&1v_3g_agBy~O z(`bCF-qBLdsC#$;Au$=uY4bohaW6fkrn%UDDS{>xD7}>QOq(%|0U8fcqk`xWR-4|r9}_)ZD&a* zdKV0}n0n2^JVOkQA#Tlm0AR5H@q0DeTsJ*c3gAcZle-jJ9LmBV(je;39ZFA|Tq9k6 zIrZFYTAmS9%3p{>Bk7O;#6Oa8B%W1*RNumAoY1{dy#zs>hQNM=c!79A;4O3c zhQq%!_2!%x)gC|Act&WK_Z17(sQJrX9*8a3d~OQ+v-Gq%YIqB81h|zuc%v78ur<2X z-MQj3at>JX$z9S(R-!!iCCD^DW(Q;OW5(U1tD%#ABaqI1g&!HQwrG#ej5PeCB$PSr z!;x_*e!Hdvt0~yduhR``Sga-t*UO;JLF;3eOd;cW#SlJQx2mg0OKhXGd!m*+yzb12 zdN{z%e_O0CsG>_f2!Pbx*4_YRBq8nJIcZO(=9%TXYZG>bf#|~!vWKqipKobb?Gvfj ztcH?KH2TB9Rq4t>yGz`a=E~(h+^QhjDiTvRzr^&0=O#T_l_~d)5To)6BVh@G*}FM2 z3gb`!TU++t$osSrmj5`(=6haVEO6P{Pg)re%JwT#&Lf1Z3oyd^=B$U&wN-Q_qx?Xd z_+b4Iut^lQxkLOKD{;(5kiB{9`UuGzCvcYy?szRNh58q@ z6)a($==%`A8Vl1<^npSk3)ok$J`A-;T!7!3r~BgwJb}F`4J>zRiY2bJtxb$wjIq{F zPHe>GZgzrc_Ilw!T$FF;L|~)M{EB>`i1aY{Js<)W2Sk3KlpC8ZX@a6ijs8Rr9sUkLgNU5Vp%pBqDE~NJI$fAs)J8H)j3AHqiRlr*hBX$MJk zd2+E^zsPLRQF|j6NP1$#@85!YHPGOwlzsEbY%%YI6V5}AV)2QbVy62i#LMYEzwE0)GInb3a)BLkAIgpCz-{wGxEA-!xK#rai%lk zyCH~MBG{9%)>q=kA(>}fto4VlZk?>=ey^_&_}#F0<2{2-YzmdjwZ(XO{_9^dJr0Z~ zCHe>XMf`|%oaRiIY#;grRU4(tl{75qjR7irz}je*=~z?W_Pa>K4GOC4VbzSYvjL_W z2$7yDv)?`@OBN`?qjWEKxO;i@DLPZ`BzyH$Ec-aU6Z)hIkjm!cWG%JZ)hX)B%p%n+ zVy~(+I3F83@8m%m3RPrfwlwtL-%guk8R-J@=VVEc^&?f6pG~Ig6R|vkG%fis1^|-A zGG2#%ql{994CmH$=r5aFyvHR8J{5Fs+mlnI713@k<%OKg((W6hKQncnp|}tnhLrk1 zCH%ed@E*Ac3~~yC*cRUKF`8>=b_Rvvi9ioL8}El8g;6p%;p6V{wbp<4IIF|5bfcO4NB;1C z$oQYQ>2EMR;oEc@_o@CGMFKL~;7|Z3GPq!yKM>G5ApenfwR6GAw|=715jg#UgO!@W z_1V16{IA)YRYdcHL|u7I!U{2!~KA_N$vc>1h=j|qpNa8T1Y0!QYyCNW=_9;%V|593B&VEC}Y3;g8s@jmnI%CT*cIo;=8_- zwrD*VaJJ<@ltf>7`rWG=)6rX-G=FGwKXU>Dv6QOt!kAE|d9k5?-t{*MjT-x+&3bj) zS07ITB<+(9KUE}kN_9$YMUeYo)E5jt|2nG3JNXeW>(QL*3)WmuCJ(5ZDjCa!x zr4_y&;ugTOi$(>Ry2jTczA63mNfy3AqJ}BJ%cdSUg(xtQOwJ*$1WWkrY6zYos;q4L z_-S&qhEf(U87Bz8Ezh*s?_e-a6lE=xecVM+%K83}#`yox{h4|HUx8V3+BY@@5QM)s z8kE|#=>N065 z7O<|O(MUeN31rooaNQg8lyM%7CB9PKJ+l~(@61D%CCP=rbfJt_dTW(cPo5oX#1vBi^WjfxumV=j9ZU&U0 zkDcZSK;pRm#Gv&`tfa!qp709aMR8!Fzq0r!_GX>QUwVqPL(`pYl7`VM?s`)trF?*> zV8ppKi|E7k4Con8vCWgWf;zp36de8jmD{tl1qG-A1dDw`&WAcfF2XU=KXT zleJ@OX;ir|tXyt!gq#L&6hNDn0}JRZn0;<{@zh{yx1d{*;n~{)53rb~!vUv=>Yn}_ z2>`WV;t`D{Fvu##pmPyv*=m<8-O=0I3tGb7teYzZCRak-@Oyn)sRN1Z_IdhjXy_n< zxhW)_tIcMAQk`=qvI^K*o2BdKV*ESV@zid>MVJJI)1%g=<)DDsI8}DWcLh*q(|Sy~ zsS&}{oX53Vv#x=+tt%pcIheMLzJPXAkH=UIEh#4`X^5U9~+b6UHxZ zC+sO9l5ljA+DEfr;Yx-2CB3^7(xfV~#8JC{(ov1Gj!)o=GO?sFs*QUROv-SWx%A^{Kt8#dsV;PtA6wG*K>B-wuRX5oZ;Y|jYR&+TN@ ziK>%!-f<>w@u<5WaZRmw7x8)!9J+Iw%y)IxJh(T`#;*AKJMO%iGWeLh^t;Dm|rk~TNT&(o=J9zDXC8>XIZ^%BGoC8!m64a;1pxJ)-^R>f+ zCy@H65+33T1WW9x_2~iI&$`XSDdmU<`0e@qW|Q}x9x!bl$uEhM+vBc^yu;h$n!1GB zB^u0*`n(x<+^^W zpPb7fY>2$jy8sZYM36NlR1MTexa~;AwW50;dF4<>K^m^T36yx_-qOSo&5m!#5f?%9 z_5EcY_yd_&s)^6;Kc~g!bSf%%51mqdp=v*0u;)|VW`cyo=ONCfb1aX?#^2L-}k&o~-=uKjO zc^6c_xWx&iF;v;dbei4Cc;_whiO#WSTn{*F-t20}3)iYw7+v3#Q+pvZY@rR@`P_8B#*r4I?n+P}mo6Yh_ux975BqqYrm;e9u4+N70e9pI z8$2m&|4_gsbi9jL6ztOH6r$fa=I7{8k>r#y%jUyE$V-!Z9wO|T#7^Q^7wIV{&hG&- zA6R^!X}4Q?o}m(JdR%3-p)!i0vH_K~4ts^?9c~sdfN#HD}38!t)L5` z&N;dfRHITw1O(S+{;^fYmxYg$;WF(7@;ylnM;i5q4n5YngZ%~%s5URWlft~a?~H?6 z@{g{s|A@o?^{M~uyNNy{DHy+Ec2%q|U;Y5|{BSq{9;);A0*9t7_Nkaq0iWV3P*J=@ z!7pqpAi@}+qP-;F;TBBO+PfuLk|ty|(V}2A`#TzGs+Rhfr0JEU!W#l*=2yyf=d7$N z=P#O?R#wWW&RRn%K87<12d zRDS6I_EpsQ$9r-e6JmT>`1J{3_iCraIGjRIJH%L;XZqE2=&RXoH0k2BQ-DLwMKp}g z(-u`pJ+o^3y<)~KdU~m&hv+lkkGIt9_>(j$+6Q{9L!P-)7G+k6;_6lHl09*QcWkuI zY*GX=-{qJL9{C4iQ>Kny3ZkU%_u)03Tm&qv%se^~>?NN*ggGD#nEa`w2 z#f#KI0#6;{pIt;9wEK=J;f)#xy5~FAPuj=2lpUx0jzR4kR2__w{`x4z9MCU}6+ZQV zk>+0(V4EjfzB3yKJFCwK(_`5;Uce}vXS@6rTA3{RjyIak!W#|InQR8DgKBB}%&4vb zecj?xB@V%i;Sk`5#CmX>}k)Mw&E^Oa%Xk2%V zz9DRVtv7CHH+*E%ahm*9_xYi8jGG348e8_77WHuDZo22_$x9qZsrH@|mAg}-<~Ja= zc=d??`qSnDvaXu2{55N2k~KyP8?G9-(MD|vVW zWurkVjwxjxve%TD4`WqnfdIBC8C-;!59Ovgc{td>LPaZ^P0=1{%-o5tTK7Ak@t2h? z`fLOLj(>|zdgGZB-+}$j;A>j0iuXeKN=9R2)a3WoU*)OP)$gDm>Hw`M!m-6Z5>2qA zZ-9g5n(wO zzjB1U);@$YtYRTd2kyX3we;G04Xxm+bZoA@&QsJ5_QFO^vM!_jnP zF)zXx$NBFhP}6?)mLObG~sx0t9Udr99V0r+hf1s~Mw`*lrr7BV2s6!DrgK&m58#uENF28w~{X0fF>#c+o> zrgN-hj$)YU0GV8w2+K`of_920bC5k!oS{9$!~Y}eor5!rx^>^!>Dad4*tTukw);)T zwrx8d+crD4laAHF&9~3K=j?ND)v8*vR;_MLr<p%H zY;_oiU|m@07+Q{UMvafs>I6}w9`8*@J}&*k(G{6G=|I=U?jWB_!J-?17)3lb5NU;q zfi#zwwnqjP*QId0Snhyza_`}}Au}T_Cm0tOSd}yyomowzM}W|ejMkkogdyq15n?-) z8GKe8%?f=GAL0aKhXARo{Bwbqe9Bs4CC;HGRdN}@(*ch0hXlCuFJyfiodMKQSFnYr z6nWRAe*0zrpcE6*a*UC|1dLCOdN(n&%}u<53J7Y2`=?eRlO`O~fkcmrzGGP>1=;Z> zVEQjhiPS36y%i%hqM1VaOdHmdB4pe|ir;?-&sz@yg(pDd@;Js$#8y0CW8O-wuJMUo zG%{Z@Dm>5@sW*fVwUDhE&V^PpoqWmj8DV8a@V4@dTBuP~F)!><2w>F$V=Uc$sWI)A zSD}l9hu$iYD{XpHrbZ}9O5!YfMg+?gfJ;WWUxC+yq}Gw@)U1mwNyl;q`(f$wAq`Fi z=*uCWB7dw4#F=!9g5EPF2;7K#9_smijIS}WcjR2py|{m-OgIX`eKt#YF8++UP8ij+ z64XMvR>kcdcVuS3udR$)AUGCF-LZ0`+JbB{pXsw!Koxn_jp>mwTzv6ds+dnZ16q{Q z_@6WRai|l~ei)JlB;eL$OhvLLvwQ;TLS-&|+epIqsy6g(~I`vTKw%XT{iA zrxvt|bWb&6a!V~2=EHfaI3!F_fW%PzBi1i*?^H1cCRB-yBaWho@0LF$k9w`&0lic{ z{xV0sw(m$k4G(~Ky-F_D@1VeP;H~7{3n#Gp*7;NHz`zHpIlVM~AhU2cE3ss4frP1< zdO&RN;aA`Xq1t8nC7ZZ9{>MWWwZHms1r1<0Wh7cz!Uv3H6pV$Eu7*499N=cK%ZXW3eq*E)knPBsl z!}=7TfX!qPUz4Txf~Ikk;yh+XY8wip3ZMfI86j9 zc4;9t*!k?%&NIf*7wjDD2xNU(U)9bz3tGw&KPJ!qWhhhwWFDg5?^fuT$J1#yD ze3_miW*0kOexW>!=L$;3YxxA^q*IjNXEDY*6nI(%M)Ms_ITU9|>6c6IIBfPWYd_EJ zQ?MYmPs1J#<*Xmn$cDP~>T+V7FUGL?RdHS=|4M@z>}*$XO_=)vL!tvG1mo0om93X| zKuijWhCdwwYOlrdD2s?GLr05jLV2VbzZo|{X#)>m7yEsi>)Pm;MS6bG7w3|u~-86jEq9g)Z z>`44O^?bb-w?Ed70LE3y$-gD1Z0%?iO(YymEG{#FX>()q{^__0eQJmqglRd}R+G;MFm}kI3z0n-Hi}V0OJ74^ zCHVx9B>hie$0kCG<;|IJMU3vw^yeuYxP}K%H>=-PBNNQ%IDn+eDKAei{P7vlb(<%e zIVKuPcO(AKk)_1_j72%`^blh}tk&9P8L1#MnP5)eO)ay5=)|$v2 zM8s6Be-2^UZyLMJ0{9NBlO>?&&9{G5?94xDsSzom!;1#zL>J3wwm&B~_+d}pJox7T89=?p>Z%RV0{zEQo%Oh zudxm#wqqp7z8;>8?me|}8|>r1j~u~RFm56|2G(8GeY&h#=v>BH=GgKj-EL&e{t4M^ z!nGOUbK4@VxC{!9R`-WI(^h|rJENZ*)*XL8Yh7tpKUF=j-@Ng>!EMPdj!~q!V!uL3 ztRx}w7j6y~%cf_ zl(ZV)x@-bCqqL-rn%3g7s>ilfXws5WTpEIE5~XDQB#=<|<5fa1sNG)>J)EfoQcs_g z)_{^8;}@b>urAoS$VL&BqO<@eY~#13*taLA-KAPzU}7BT0e~Y1T-WfsTb>pa9(>R6 zkquhPEeToK&ap+|6oHK(HWJ6e(G^ih?Rw#iSqo~v#!*V#XLHkQFV}18ysd0o&SRg( zI1iEeEGzhse3%%p-%*q}SIuj{(KL{l4W^!`TF!0xAn|gLS?W`hITc!K86BWqjlMF#k5~17+PmC)$Z&XossJxBRQvk3n!UPIW^kA=nNv* z>qaUyg*$yg%WMliE)d}9gB&*MGXn;o;Lh9&Jj2*6u%@XlT?0DK>W!a(;GrexQ|pTS z4=^4i0e1=ASSD*~JcSFMYHt8J*nk?9v-u-hf~#RS-vv$toE=vIHX+Yw!X^;5*}i~) z`m9p?qJbb-h(9s6jHQsf2i9k0r3Jkd4E~(_386pNnKN@`fRZ|IRA;`JA9hiM*t9Yw z-Bb}PsrlVWxs6tn^y_v&BM!lomu>R>ONiUWhbfSLB1%uJ94F$C&ri0%jnjV*^awCM zN9=22gKN&tg@89>hHOAL**HKIO?3ZBZ7F zb9VT8jr@PSXf6u45dx;v$1Gb@st7wrx5e)domH@ME?EyBxW4D#3xH?%T(dU#EE(M>&uxJ$LByCF+X6?jtKF~pDO%wx;-7KZ5 z69ZGgRbGJL@R6G@c2JEb2=canUchs~gz$Mj5L`(E-0#vHqVXbM$NK2Qo?2qN#6RPw z^>4>T!{`+Y>Q^3MX6%WumlM?zOm|-zU!OXP8WWm<3RrM1T($ih%<@S!m zKf-mSQ}s8_k4+{IJdsa};7g)8B;&iQ-*CyZWa`~uAl)Fqo*&Gf1A}pH@#W)@<1c`e_BB!2eCIZXQTreMH&HO}PO94j91lzxiIFfvd`o3M4=g=+PfU!u}fV{5uBqhM@t{_<6X+HRfPt6r*D&s{H4RcNpH*6@3j+sVTfA6{^>z z_-EPIuI_WiekaB}gC}>`s^|Q*0G(~(SyjAU4LU90sU~5mx@7)SrFs_U(1}z%o`=YCzM0yo-kjg?f8Bz1Z(E;$F5z_QW)_?XTTLH$S1#& zl#eeX=;w2N5DjVZcX+scg^Apkm37b7`@dGx8+UehJ8}UEU z7NA^`+S@4?2Hd9rCDQH|ym|!`+n2puiZ_Hgjr8tsVe3(Or3H;H=@_=EL*~1yqRYe} z$?}F<7<;SB(x!WX4e^s?{8HC>KB<$vKjJ5zNncil)nnk?w^}OR_}t!L^1BhzA*4(N!BYqX-a%9L@@LFJ5hR0e`F|2;{cnSAzuF^cgD=H_E0m?u zV<(XHvm$ndfyZ~GI%6qn1<-jQ07r4d4btYi2aIr!;EdN3>ET zjROnCHIcZ{uQwL90)9Ho9Q+xL0w}F7eGp%gZ1PUfIrz*boj4iZ?}XyEbvl*<952AaZ@WGO*tX*Yy)<9MlDUoEwcl5CjlSrUyM_! zqeToUYDE}L?P3&CybOpmRKgA-0dq-UiF7FOdTt1Jt}bea%-N1E$9W=u6#UwoA7~bW zeJh-1E9)_B?CUER5nZlXRiCfA|4u}aU)zoUOsu(bF=4%QE_^@;icUlTZiyH8C(|t| zEv`?Xcw7>6Giv`O=q4zQ8FVqg51U(B5pEZNHc<1v<#PH|wIZJ}@+CPiATQ~Tfba#x z1ifP!aq5DMBF7wZ;Lhe1Mc{+d7qLZ|s?A*ej%{7w?uyCu;wAna1lWx#Pw`ZVW-PVu zit@NEapN|W@SWcdc%s|}%D3L+4rMqH9NlX{_+vo7!4V>LT9Re3H;k6=;rJS!Mu%2)gpB%7l1&cDDH^+_rp?j4`#8d1eYSOw0Dae*K z9cRz1+3{+upi(J=EKEWjB`N2?5Y#k2-)566G}C~D)i}}8#9O=v3Q$?54x*1o$uLvp zqz8Z#EH>u$gp5RL-n)aGKPHcI_9<~Luqg-xC=k_`%YMp)V02zH-I31I^qKoYPiub? z%byn1fOU|%9c1|7C31RrDQtVNMy26u!se{dD9e*;vLuYpnccWTBl zTO5*plCN@b2_SR_!YMKX@{2XYn}Tr$)KpSQ15-!?<;7DuC~E2*?Dd8z;>{$n^wVA5 z4nJ`8-8;0TKQY6HFO-#FrfB=m3JKR~HgYVyPlgfKHcymsJm-F6l)waclHbg0W%vXG ztmy$XSEaJPrz{~d6bLqI0aiU@Dt!pfexE#GdeFo_i7;+}A8aALpeL`C-t&xmj-b^@ zc!`*zp(5p{5D>f#!sa5!GO`vAUQl^fklbQV_aF+Mgirg^)oGKej|;5923>;-lh?wJ zjiXDG7(o|D%BM%z4|m@e`NJL8!yUwb4bUI%Hor>KHfdk8Y{3Dv9jEXWSQshb-KayO z3#f)bZoYj7H6QSZI$Xlr9KGz&Hao7NAAr3DwxP`o5xo?-sV0OJ9xso5o92Ec4iTO0 z?LoIk(f#*THVgN~W+Y{0z8`v2%zd8ewUSx9O;HX)^j_gJNhNJ{g=NE9ZI~X~GDRrY z_SDi8A2Y(zv}h+8qyZ5~3cOHYE4jdmV$>d```{{oyh;A(-Y8FU!S6XlTQs-dTU5Ew zr|&>(&eC2_;fSGQGWt9++_qw%gT+U_jgG8#`FTUZ4fV0@c)QFpwB({12Y5L`W#}fK zye1M;p!e!200^mnRzrO?(aqdQ{p{!)`DWxK*GQZ~)28n2=@#&w_YHIeQLQ=c=I{26 z6bI3G{ouGwHr}&o{2|jOdd1}N11B;x6zk?@Zo`@g){wQ@fS{Phl%d=L!BuoeI1+nh zBzlHv_5&#*ddRjPe2aUl6llN_nNv<7z`|B5b)SD#TLUpDMo|Ox?2oee+isyN_-pa4 zXu~vnK0l(>>H+6&3_Yr}6y!?H=q)Tg-xF@Jm|>wJR*p8q!`9O1`VEKW8j?~Vi8=^J z$JGqRub;k#A~AvH*a=Tc5h`&$Zau&as$SS>q;mHsWYv7)Yv?8NjGaQt$)T4sY$H2P#V3A5ZXn`Qgr!gLT5{&HUctIF zG!n3~d4m6OwEowW`ZjP${{Jj4_h1C7!{2@|IG8k}G(?89b0v0kpevd>`q$QUqCE$^ zyvT5*no2}^X-k$+$+CvrS=e_MhV=yz4D-%*GWaux%{T>Y0z)_adCfeYo@Z?_IG!Q_ zF1{P0mpZO!pKD4s2A&xY7FXAvYwq4}hwrVS;0M?)Is!(RGo|ooeIy_n<{S~jx+`Od zdIY_zM5I4FvdK#q7(OW-eUKd(0l_8*XCV__7Lw(Oo~_9oqz(C(ww$`H&xy;@i_p`5 zvY}ggl6A$VYobpBXa5s%auNgQ4r6lO{`z05`+cbKMpx-7L0PGyE}}t>mCUg;lZ@vG z{l?ua%Y$UkQtK44BA5EioOXkq09JLKv!^tO@t7L^VmSjIu$JN_RSB~XJ5hFp#}f=V ze&fJw#dFD8kb#7%7|+>H#a%4Zg6{!+2=Lh{Gm9<2PtuxNWwS8yrtZ#FIS<1lVN`SlReT5Pn1L0-@M z@BSbNl>$KW?LySwE|=Vi=DgBx!EP79q_-&<5jjypaaxo>>&M7e+Z$NJ{fF1wN3k&Y5mBfR10dT^U`_7O9%xU$&_>NDF*T{njeO{~q-kSsP#7N^gFZZg z4UO_q8tB3p6%X{qG7s{pbH5gOA@SkZ)t2Pm1X!|^4jIN8h={*7ybJya*2po^Y@wa6 z(!l>>lweRKqmx~#WK$pw>YtU^jn(*hizP}5=}NeO_EL%!nq=p-~F#i3%`saLT4jh7sqx}=}zp8>PwQgdQE zAIc-#w(aCfszgthdc7Ik-(KO5&nNIcd;<>=Q3b*z+72d)iZg1rDKX|KZpdeK>hwqx zLL_uPwOg1t3_j(X~HmWjY&3Qthq#-ABLbrZs zGY)`36#OThn;1sChx%Vb-e|@m2#CV}JUwrv7XR-T%nZ>>LI2kaY>DiZU_a(Xzxm2a zYfsGIfma_YEx-I5sF}YC1)1jLj1HUjl);GsOi}xz^6l8?C$L~4M?fYcr6#>jN@t4X zC7#$vjdqR*EJykw4Igw6oO!0TpZ5aUW0MjrQbnKl0V6mBed__;JYezaQucTKWYf2- z_v8HmvJWJ8Wd0W`^W}&(_{lz396Uo?xU4gqu~H-nO=!uL3%rDrcw`lnM2UAb&cU58 zP&g8nrgmIr6>*cvDT(Abc@h*!l#qq|4t|Div=(l)>E3}lbWGpv#7|NUd}gkj) zA8?5uVN&uC7&Y(HUVtJK8%mn)!WU}Bzu@}L-AN6$nwPX~yy`g&GstY)2ABCr`3-kvouPiwA;lTeCf@)i~F2RWsj2KPb(Odv1bqQYEVIxdGDZOesqupwi}ja zuEljl@SmEkbt0OZ81Hkyi46}CpqtI;qdhwrMixdudr3ja%Nef^kxemOZsdkec~2pa zKV*23UaWDM7vlZHzWl<+4QiGj!7PY~xx`+_%~I5TUX4fNa}uM-52-iXC?b458R}AT z{Y_u+FSMgdoBPNO6RX|OV$6XVxIfMM)+xik@}h8U@r)EXv(w}tB*{F|g@*ZJTM(~o z%5F##)9GyK@hk0cb_c|1s{<`iYfRjLdG*`2eU`sUCzALC`I|}RLPRB2F-N#+UUUZ> zgWWTLztptlgIF`p0hjl)^@%^Lra+%@HGYgzg@so>RDV+HXTB}_L*k`m6j%vL zHMaJTJz^GeD(Cc-M;Hqy1Pc67FnS271S~GuJ}TqFxP3S20FyY7bGPJh7WYJkaHS#t zN~i4YT{JWHua}N7%Ze;A6Whl8U&+6I|4M>lfokkSqE-KEC`d6cD3sVCK~)-gdKM?u zqY0WQsfN=8XZV5XPk_PiL2I*%^`B(-|0wH!2E6}hE;c~*Oz~TDliwxcBHz8K$f(mU z$hgsfFB&Oe^JU*o7BhD66(bw3A-*)2k@)KSH$do?Vv+{o%x+_M?KnXr?`%| zgQcRKgt&Yoj_tJC9dEg|p^;PY*AhAE(VSRu9Vf0+q@SoJamNnNO18)9BFG49UDDoM$Mi*GPsQ z{Oolp#=~#^$d}OlqSOV_0W{K?MMq^_Ldk&D__bckC-*fN4`ppSis-(lzu1Ly;2P9q z(jbMBZGs8+us{CQhx9m5)Jk3aDCB=FST%YLszaIV6(5~ca>kq1FAjgPdj*VcA*{(zt!^eTLs>uCE>ZzM;A^hifxZi)F^8Tmv{!gG?_!cDw1GMtqz;~q0 zOG_AnBK%JbL<0VIby10$?tl2vpQWuOlbEz=fxD2+LXM4kV#%n^!d!-1(~YSNR?FG%O_80c>$p6_UeI(&Zdi;( zTAu{f0%AsaVQ~}+3{PbhrXrO)Mi%gqPc7Dlcq0la42 zdTtMi!EMKpY3+wy2~ zACBs^w)y7^ZU~}>8h-MUPKIunO?bJC^3}H+dRm7ue)TgL&81{k6_<4!Ea{8#F_)xc zAps>4i~hA)qcku58!Jsh@vC?HCf-=-yffL;>0W!2h)C(X0M=m%mn z!cKseG6``kP7Dx-{_0DzdMJrE3Dv^LG}owH6MI0Hxg;~0hk`6T*@ZjVRk3uU(Hq9* zCqjae@F9{lxCN+3s;s&fcIc}m`bQh64&l^Vm`FJm}B!8z(r;U>dz zxGFpIDr8x>x5F9u|blRD5(+C#rGgxbTnJj8-Uk6zZ9 zFXI=)IJpJfM`#d1v+OX=xS@`zkm=uNrr zEDc#;*&(Oh!0eb{@2m`!yub1KwMR3iQAN-Vr9=Y{s<0n;0Qxl%wt7wxZ7U45XsfpS zP`QIKe(e!u-c6Ja&ixoBQMz^*M|%ErX$}VqS6z3!Z97MNr=IN}Iq&-bfc@goR_04C zUQ4)dn@guGTZFH7IRoGIvp3W;v*_k5vIi5@S3pIX5q~^6Av!P46rktQ0tD%HBW%kg z&I))uX&`@j;0urTf^r=9v3ag6F@lsK-pVx$TILlU1)Z}msc_pynd{UqATf5oFAexe z?<8=E7~H2x^U~oPEXlLl*LJwcru=yxI690e|2_q&7;1ELZYWCOR<_J5 zqW^0GQZdT#zOUCgY6q?$VyAs*a{pQ3HWCQ`NowF8DcCk-yYDXFwTBLLixlJ@&db@C zFY}s&WQ_RBHEfu@Uzh2ni}siJfx#iV8aDJD#c>E2TQ{Klx9Tor<~9WR;ZCRzi?-rE zbg+kMAV2|k<}r*c^965UH__-r2H2;miwlES7inuN+T+wli+bmMf#S0m#i1^-+6yEz zQ-AF$$Yd|+_o)-CXR7Bw*279CK#-hRS$)AJ{WKCn`w}`;{GjvSGuqr z{q)IO2GS0cCr0jjo72%AYu2T+1A)fk)@4p3YEy>Hb!{DH9^S(=bhMK)$C@779KObo ztz0FAemLFb20l94W%5l+H|pjoQ4_RPr^A=~A?nNaSzNeXB!Xt9m^0Gc;|j7h8dh{t)K;~FqrSi2mE^HNI+lBl~?YH5nL$dvNRO$dR{{9wm0h1aCgAgBsmK#Mjq9 z)xX78USO?Fd5FqtOh}`jt>MEkO$W|JKsLZ4769$E`!(9Uvm5y!@hz)Eoo3wZT!sCX zhh_! z#6Z_~;Z;x&!qD|!;#Zp_wSTJjWFoS=_q@^#+N4)ZcAO+&yQ$T|LasI#mRbGjWGF4N zdK#(%AOV-&!b5T_vsg>4Sm)Uj09?G+hbWW}Ufp6r>h3Xn>=Q6-EQ6LK`b3Z<_G!i; zhqtDLF1D))gQrc0R#uXO+9sR2j|sIa=I;IK_8r4<`{f(Uuu?`~1qRn?7#AdZug#JC z?6(2~j$ct?)vO_ZDh|ET59krnS2-n1-oJe82vJDN zsIdYHBal!= zKY`Hghq5};y^1&KkgCxY-)qbViHUfyJ*=G=%gBKYDlU7 z`A;h)Tp;)c|87N$N0>dqRyEDpa+OY=3plxoSn+68ZSn$7Gp~0CmGsOK_=`pLq7}Eb zu6K?h(qeItqCXVTo-^av8*fcaY)b}1M{*3f&yXIUC<2ZpZDEj{HRRBwZ9RT?lWAm1 zuvap3V*Z;+3QhEfsna$MeDJC>l)mv|PvkX(gUiO5T+|>mfejZ#!jNGmS7Ai!7jQ=w zyG&b$0N}cMIWgqCkV@1;O86U6E$}SkAX_8Q@AmJo&)tLsP~YA;X9us8hSW}OElbu) zknY#?FKikG4hW4i9!w@h+vk*+Dvfy(})I1XL+O=9Y4-z+2+>qi7OqW zlASB|5w);ZL2-1+N9%oTk;PRTdHsw%+gK{EdQ=wv#bDUWJg1?>vX&&%P)cPmSCA`;O=f`og;fNJ&5ds^p|04>2DjD3huFN=NT zIL|co>hGL1a3ulW>#&fpj^dcj2z@~;GZrz_%6W&3U+e2j?^>feZ!AsPFJmfg zK;u%*;?hol@Y2tnMQO~{=fqa50XV49q$OB_0ji1(>iG3cvI}3m1LUqLo#C6LZE7d= z)p1uI9h!XbYRA@8uV+>}$T~W?&-UlZx1{%+zh&ktKOohsKCpf1h>h`zjrDjD3!B|b zk9DmV3ya$|CD8Xu=G(+J{GE!q8)uh(g;-Z$B$Jdcn=CR)zJ&bNJX6NyBh*Td4j1a? zg|xl8_uVbN%tP z`35N-_Y87&`3|?U)Q(sxQvs13g?Bf7%1QK|;&T@H(hC;Rr$S=&v~F|CfpoFOC^Q(| z#yHCDMRUVHilZ69VrK>16Q7BtX(6KSe2|u^5Y>~vxO7JiO@Ji;=<$BJa7{PvVC5K7 z_Y8`#bC-1d+6cuq^O9Hva%wW{C8?>^TVZY)w7(ZFY=TS1-GHVeZ29AxXjk_PD&I2} z!&nMIg|q`ENwlw}BQNd(Xj_`yX7n2oN*k9%0M)ngE;@_Gk6So*4crI~M=JqPHeubt zCylBaclEnU8hGAm)@nqB|&9{rtgMOhEfX05?0I-}u4OWhT6y(XJl4ZaB1(>|? zBVg=zw021aC?<$?$GL41PqN<@08Vy&2t4;32!H`dr)?OH%X>sOu>i#sS9k8XjtMw! z!?4!&YqqJ?Loj`f30i%P$q=4YG}>FrF733aHirg1U}1!!dwX*Jgt?fGH&0qmf637u zf_`nh<@o}g=6R#U3aOkdIzfrEfW7hIi(9))?aC3Zc8Jm)h1on#d7hR+ZpmbG*Cvnt z%47f_s8Kl0()5RjPZce*I?Oi&d_Np8nAnSF#aPVqV-$rawjZAWQYP4-hB#7`{*+N7^27Ow71)#Cc?)Z$R2xGU<3v&bWN!d1%hlHHqdD zZ+39-5)5EW(^L|wnWRI6rLrJckZ)7OE(#e3Rvj*PbcOoH49s_QX&6Xy=^Rmf7-nI> zT4Wqf;l|{fduxjO70|D^uac3QXAl*bDeG}@M3EU8GPqYh*S2o6Dcw^;h=`|I!%MtE z=x-A{E*;6=S$r>WX|3O%w^=w@^U?)yOu;iX7c6kn8-=9*cl8pAZxZ}oD|M=SWM2)8 z8p3w6NZ2UcTwFOmzw^Lc5wO6Xyd;(l%!0qHC{u z1mxL};KuP#zswjS9SvJM)DfdaLpq&_S#(BWG`I#-pr$)3*y#>S%@d{xHDnIBRTkES z2D!BLecs0EYlOPKG6aJ2hbDdvNdURAXkb@WJaNfftQ!098n52wpjX9CW7>RR$hoj`ul7WWfFqa|{}<}^@3#H-h_T^RR=lS!g!fSnL*9k}CIqGNDlFO0V67$#xCtx}RsQ^(zmI z?u->AeC(|R4Iefx&7uDDJcPJs;&ne={llxmbL&+;$oxWXSQYDBo-}=c@-uf=`A@AY z8|=^P-_;x1l<_i`4y>_A709L%aUYD7%S3x|A_@vlC07;}VCFx8z%wrw7kY~#W_b_Ot;i0 zeOqP2NO(qJ*#y<-Ke>i~+$K+H!YXL%Uqc!9GpBAkqy<~#`-PGlymag{Ny-YJKWD%t~nQ?ysGN|0lA0#@{Ks-kULVgGD}dU z&7PWgcWS~dNTtjx1u*(eSji0&PMH~--wyszPCkTfDADlTjn$6Uu#jTh2}ROXVy1n! z)85Lg^|KiIrc%e>4TX=D8APHi48$NJ z_A8O=*{LO|e&-*4@IWO3f6=e%v~I+B{5o{qvTKx{jCy<^O2L&v)`M-1UOlr4{=X>R zlV`s`!%V2~5yEQAH8PTe1cH-Rq_?UaBxZrddSWqkXY(#_k}IJHuFUkg!F&I6n1;m7 zQ$Fo|+5Pu=(b*%`3NPcw%h!~nW!Vjyaw|Vs(>UoHpe94|$Ya>#IUXrqp*Fv%$Ga;> zG3f9B?M1N96#S=ip_G$Dd|V!#JU4BpX;vF^N)VX)uq6(`9Tppw=#Pu z9=QGGd{UHTX0N@Vc*IHQDc^Fi^9<>%MD)pTvX3y-mxnPZ%-_8SoP~BH-fE(?^El9A zxWy}Pkp#+BZ)0)U%`M{0X-N)|(xAWv9!0gFfFIn3JSCP?+NSscziwHeYP<`{1#FZ| z{0Q?_BQpLTR(aueb5tJa%K?wkN2lUZ2sjNo@^Hw*V(1 z23|Cva{OB`gNv)9N?@e)w|>tw$kQ=Qrx&krpDK!!Gb8Jk+OJQaHb)#t%>8JQ81O|` zHAOqN5AuO82o1`1$nDkWRnZ{0TS4BLikmHU-Kc5E5_7SY)WzgOssj^(dN`^1N&9%h zZ{hyc>HQVv4b|w4w@KL=4i79DF+mlC)=+ z4o5;A3NsZZp@ndwH0ho$(rtm~I;D9f>|;8!7Ji#YHVJ1bRi{AdS3lgpW$J-}i+4R}P{O@ntD-*l_UF zn=7_HS90N=8sDDZa`7H}zw*=-z<*-g6Ud4=s{MGdd=J5X0G2JqJ{401?5m*m3t~PH zzQ6lnM?$EA6x1q2o04s9`rMBUxEH-Pn9DR`XLek95Z3J*c9&-|6GWi!AQ(}cX5zEA zV~rk5f>g5$O%Ug>DIce-{{wdgb*jw3l(|$nc(8`_N7j|)s@*&2X&@bQ45dEZYxBS= zbW`KU3@Da9P{h?AGjz}u$n@hURFc$Z+8}&A)KB5}H&VSHz>teQf+$<6ez3RLiOyN<~~J zy6F8A(CPR*jwglFX6;>}Btp+$TP>!ngtk0Sjhn_KWpSuY!1xZIG_g1Sv9dd{7p3nu z#@B(&TtS%fwG1TP=`!@clX&ZQ4zU~Dg5D9_hN=~DxQHrQUmzVQryBi`%q^rQOJsRI z%;yXl-!3)~OzG}if%oB(9@^D;15rcdr;4;p@b2t&_~79oF9k9g`IU4Os4yz)xT*8H z)(L%9j>NgT@=ZGCN_61;O?|^FRhMnswr$(C{dU>5 zZQHihW!pCU)HgH#nmHG_GS|+$%pI{K;=yy@3u-EUE+ODAJ$rg@8ehxlik_pmEfgiX z3b=Js5UYmyOrzOavw$nyK0VQbZtOM?(wGL>hRhn2ZNOfFn@zLPLY-b(lr`w<*{9*X z4N@U0$7KD;?FwzW1|U@^9XmJPJ?ola{*)j><>&`eD_&u&>s0SoY#4AJ-RZzrsz@P* zXe47MDFXoLNS~IvH%?w+Mv5M5Y1Rqh9$)y`>D2ec zpzsUacLIC&3$Bc4oaPgHX!9=iFikYXCs};m3{hwHu*L-#^4v68qNQ|M_H>cdD*ViB z0d@d4l3W0$rVK&_zJJ6hFP#E&s%%9${o`9~#H6#JoHcbb4@-e5zAep#|zGQX=fMJXU$6?x| zi;%olP*cSU;`1_)zFJ&suX*liZ<jSGo9+MVKNHYEHPA4$$$@Lb5K&sV(~hP#e9OLQE80xJ$PKMXY803+H6ivOi7>iyQBg4}Z|KC0bJ0579-Dm| zVnTpBSQwv(yM~ zOJt=WP0TOE;jD)JMb3Ys-tk@KRaHqgKn(wXQ>gK>l>0p^kNdgtUHZ98)#H z{_2bwy!KTL;KIkkBeuK_q-c#`zaUTqzzh9=N9xY;mvR2f$_uNu$7GN81#;DAW`NfX z_kJ6LT2vEQcK`u4>WCOjoz7QY8dN^`2d%vEk8=v1%Ab}pM5XO&NriMi%Xx03^w4u& z(sVgLsan623=n92n(Qg&n7ju-0csH*3x!SNv(}~1pY9aJ#^yhWhq<+DA28Jm^wQ%C zg`R<5m|-O-i$j~twv+~v&nudtpS$?fRP)@L(3!TOSGF(1uB{|w+qvty>fNC6+|54r z!?cX%ROgH>D;yOouV^IqHN91lZfC@oAvTDf0#A_#w0w|(p|aa zX~x_hTo&k7jhF>9rdd^mBi&e*TD@ewrH5lx?ciYXUv*h&ayXeZwVFRp(Kev-A&2cl zSVX}!WAO~Aj4`8}#0A6~@_?2=#zZK}4UlOlF z$rl^?TvGbz%e^wRE8Ic>=U<|-huZ{dk@(YIXrw#u_JqC=LodH73)>f8g^qIb( zX~{^ukrp?@bNXzQTV(Jra*uTcdu?}riv@zoL6LEbJ?lZBedxjjvkf7 zeQHMvBNNR^xV4}cvA6iWxG+5oxD@up?+UN{#ws05|FmoUHKVctcoyS^pCO~g#|^z> zNs`tPB~eOI82e+Q-2Q{kIJ+=@Y*|^+d-H6H!W@IN5&DL(p=(cjsK}`I5kq>`J|P$l zFGvu!=aqw4$ywYr$2*cJR&x4uRB3dJEG@Dl?Tfb0E-jKPq;7ANd_sPHePpy+ zSJr#qKd!I6Aq8B})Xupe6${!jN7o)fTB{_Ai)rqxT-+)H1VWEf6EFS1b#!QV|Jk_n8w zrtzAxM?;x1L?Q}@7$0FDfhYrCMk(5?3^5Awt{_gtvv&lF=l4~Op`3eYGo%GOSErC$ z=$%&rSPzxe%90^8f>g>QgWVbTfiZCxcG8Y7=KF{q!vp~y}I z4l1jsr(vRDa>0VRD>N=fK`!!gmV0v*uSX$e6PQ0@7W{LbHYOR!%TQ>@W^EZ9QO5Q2 zqxdoPOyaSSdElr@z7daBW4w4*OT;1@87TD#AYtcc2dz0&^pJQaQDUm4tbZiI=Y0X& zelwuzh|GQw&H1Zw)bs_;Ph6>HCMoDNO#(Z`<7Ck!h`KlhuejHdPrHtJstv7r131Mg zi>wN?ny{uQ>_Uf0yGtc#r2Gj}l@r5eo!vU$jCAp;a1T120X2%oMZaCQ*8(2^aRTjoxhY#{>Q z@bnjq_Q4qjjSoboL8%qJPJf}H3U<|xpK^#b21V{TU3)R=R9l0g?$FC&J$oK|Hl6Al zAt~W*x16`QThv=4>x1C#DK8KigCp)WFCciRVmH4WkaG}osM_4)UbsNU#&e4j0R3YF z>o>0A-{vHJ9Rb9|SsO1%iEEBRDA7tsvLPa=A=@M+To|@NY)QmL-y{{&t3pLesMb*J zHydK+G30OUm&Py~%;~y0Bq;MPm1cabhwGfXIkLc3rlY1?g~RH@wBs3&zn)@wk#v+&Gh1>oRj0 zIKe%-wCUJ&>Bkk3T9BzP%&?>L=9rgagmH+1hQaSI;}90G)H$CJ-ZQ}ST+-Vr@1gOk zKJ4|e_G;}G#U47$nWJW5=sU+om$Xc0z?biD_*B0!@_u3qnNsTlf{XfDKwQb7DHY7v z{R#~U|JFHycr%J(tb(x}=aXcy5p>$Fb_`FB$zd_+tVE`*>fRqpHc;$0aqvR%{BCDK zz-jQDVhDE{%)T*{bRvd4xz~Vl8f1fso;p5HI3AUSX~Ed0S1fy&l5kRTT}5N0PSM`a zJV^y3HH$k*s8gpSEv{Q_K;2#hI+~=w1brSuQqU@s<%M&t&+wm%0HXOU(ks_^l=Ywv z89EDWhotz7%u>djXc19Q%UM?UP%Ea(U^us0R6`tD2;!J8+@%OLcWgd?twi6g0|)Ft z8k*Y0Q0S+=s*5+O%eHYNE*Ol;)P>A|r#>66-6ujUAuE*~+Gs>&z}Th3Wx%aS`^Z#_ zF(bs-l>LY*disRO)uRpw;YAjlx>bXu;->|x(lB5`sxx*Ja8Wg zxIjHNjL!iAI&DL!r;tHBHki_ZIJ-u)f0`qV@+h`CxsyTKWWb>#c4gxJt~JLm+W3SW zMh?*GB(h$bJK+^lK*Wv#2MQcwr_i0T$YljY%D}E`ksfxeSa|AV1Q-fJe+YT5bH5T!h~?+?&6kp*L<>8vCQ zio12kZuaGl1(I`FZ*+fyuIYwpgzDwq(R!5olmj{xg!Y2!sEN=@VI{dkQTufe(AhxP zybi;_F}nU0zy{?E3g3B@V^%5~D&Wu?ov=bbu`>A9ksuR4r5C0xG0+3DZt&DTm>XDC zobKfi(>&3vPgFy!QXYPJ46Id0bO(18{hOG-iFkjhx@6Xo^tftk;?u09k#i}zb6Gy< z&_hH@(*Dm|$=1v8NLhOrj0`AH+5}R*{GO3pNF_jR0|cel@--GS2S+d|Qq7wJUhScc-=PBCY)c$=u>XK1+CHRo#{#&yeor(N6)A( zR~A#L1=CZ%EuyW~u+VpZpdE*~9Yd5F*Z3V*8hdc6ML^H;hQ15q4~!RMdKv#T!tx~x zrw5>vlO}@eGQ>xUr`Z&}4NWur!Xt#@rR9Ufi-~(4gf@ofn}rz3XQ7H;f9mpmY$7xRTKKPVDjpV*Q1K*=PjA$`+@-i}-DQ@QhO zmDu{>Hj1@t>w&aEX6MJ_2+d8{(*M(giyJ_7{rmV(=O(=qYb*Y_*UPb^=z%QOU|$kT=~hDI`kAKNeS5=l4{<@OSHqs-OC8dlC^IJHhW^nf`HeS7Ze)-XOuGm483Woj z7_`MjYTN@E%;F`1h~DvsH`pQZq`5DC_tup~iNz5D*-EE%qAQK$K8ZR|QgXG<>KjjV8< zC(C~k=Jm%zu^Id(_^eOqM$zo>WVXQ8H1vy8Q0ndr^n98=DcKMZtR8iVeGcj8gt;mn zE&cQCTV2C@fyW?zr^%|?Ws$sb(lWpjSeshX5uV++p@~KGliL4*$>{)Yv^Rxa1uX&N z`ukcIzy@_R;bwA=(YzVCEXMTqADt#RA*ELAQ`B`P3zaEh3@qS&8i?)0!_|{qJB5TR zzf zvKSW%W&X*gaT{00P0LZAA3!F#qhZY1jad8lb1b}N$!kvG=+5t^*^%Ga`!Ck@FFpyU zC*|2MTuN-&yaRmeQqnuVk>jyIN?PK4dC;dtW(&z!8Gy*lIp1k{c}FqX z?!ueMs<=sFDU(76WJg8FB87ZdQzb$@lD1c-UzhO2Om@B0uVjo8c8CUBio!2o4%(D? z*>UIiry0hiQ5N;6XDM146%$E(gvzHXO?)FHTeh`j@do~;Ld8GnhCygr5W2aCLE@;D zDgVgWifrPYT(fv@CIV(G&!#?=B6HtL2Iu&YX0NmZy+`5!mKIZ7L;7jx9MTM?aYut} zH!(*L`;$y)-u1n0N-}wmy+e_yG}>HWVLz3Q`SvqRj~NAj5(!Myu~u=pvZ>25wO7n* zk@B*9Kws-3j`r-WI~?tcTk~G;Z0|WYUyGOCZ{OmX9z)FOC;?hoo97Zcj3RnS2V^Ab zmJ$W6-ew1H^`zNIK(|>o6Op}V#7`7YeVhsiLHTw9X{UCCK1MSI;v{T{9uWYyU`aSj zqj264WzzP%!+>;qT1HUXDRqhBHX`D^nxc7gD*@?-rFhdSk||9o%&H;#ws~`!VNtF0 zI#iXnz{3#JdjJLQDOAmi$*xOV&@vKrXZ*XE>8qampUggjMNItETW_Y$S`_ARd+R#_ z@#&XW-WQSKdkng(06pBm6kLB6j2=cyU!1Boj(s-KDZ2s|wyThoWYZ5@x=!r%2G;#& z(GInImUXQZ^+Nwa4CKg3Vo((z+;;c?4SFD1Kkaroe*lL(L`lDdbB@T;5teWwQ!bP@ za&&!uZNRKquFJP1>4)}aOy7C9FYOEPcSBtOz@BquH~Sgm{1*wf$?pnKe9U(Gg}V7y z@VO_xN*7|d2kPE8++&8-_AmH9e5L=zfB*l;a3ju*-PCW7AW$6GZ-#CPK`t^Cfa%}9 zE1PQPvRG;eU(pP#tWt*fc??sqXUtEI3^HrTh>{8Hc`ioMp^y|0$&YV8pr2?rbrNz# zKUdvOkPh66)Z0`Pl*<5b2tNMZ0!8IaR!CPX(>an&jo&nhv@|BB?M#kyzAukks9w;@ z;Mu5iX+>#U`nECS_%SnSUVvG=>75Ar$af#MVzki+YSmSGpEV&3{|Y1yn#jMSlN)WY z3%?u2P&d~}hicgRRDH;WL&J5V8Tn)aU3EUOx+9LdLzJORt5~<$#-e?!d=$lxc8RGe zep13qXKfh@RBOlBR77i8oh6MlzTrXT^e)~Rd!D0CjqG?$kqQ#V4q&6u zEiqH4xe8pfdzA5~8(`JmkO(L&z^7fwM^2-d3a`DA*)}zpi$Iurt5$V0oE_%%s8)sz zN@;=q&&*r5h3q8ci_(FA?_8w8xOH1%$`f_X9LK17yK3fwb-=b&f^Trp8J$h~yetaK zNZbSGta-B$8e~y+RxsNVYm}R&)%+4WFhxb9Y9j5_(4NcBJpehU)jO&shFJ~aVk%En znfsS%eS+{X4MG+ax>~Q&BZe7 z&6SLm@CM=*l|E-*k+a8%oGQuw_Ojf5jtc8@$N8O5&nM+R`U=IdP3Fh^L-eEhMCX($ zK?G>1?*3({Ex?EeD37~XAEYqK!=4T(nS*4XMW%v1st?rLZMHx9G)3q>7U=d~2mTe+ zY%IK>n*49&+E5ZdBCb!=RO?37GKhO;a;WzlQ@@1}=vuQ0D40rlB&!#1`PHjo@g45~ ze47+*Zz8`^H^>bZHh9@&+srAiotc-vou^HMX~8TyWB@TEG|`jnwvCshxD{7a$N6(A zF4EGPm3rf<`$yDmuLDGAAhR=k$*5j?N0S-88J&4ZY6&(g9t3t&xr-*s8zrQaErXk@ zv-GLHE}ohc_8LIfl*-38>QcX!8tP-Q?VB>m#;eice%`5Zs! zPZrEd{8G_%N^_-xJSf34r9c3|CMpa9Y^+Eq8G4~ZOv?m$a4bmYuWK55L8G9Mmz4lwv3;d*5~sx(w_T=F8AjFY{wjhZmwFVu=Gbb!!4B4 z9?K`*3oKvgc9*(Je2fAN+AYBlgQ&Wb1%YeOQ)%?@ZP*CeP1*6OiuM3~RJ-}N6Omq^soC7)zYM4TFMj21TAGJ6#UyP zr<(%qKTf{i@Oh_oVPqaFE(Pj}!bS^mDGLkwK2_d>HQWjef`@uR_*Oe&;HAh72Y>?Sq>mca{hqmL^CL;F;x%}=zkS1B3$;p$fSrsd$ z;1bsvj24J&8AqT+X-9uxDG*`hOz>)H<`xkGze$s$O3Ms1^T@U#f8x=es>J2*4p@c# zWp4kN(6B$AJR>C5*n`%cpW;H{dvfFH#gTpXyZZRH*-)JVr^DlbtnK<>@V2t7UaKwIoJSa=CKjM}@1KDxx@xlQnGLK5+4v11q6^QJ3x zlwSwT(I!t;8HBO1ep=oW6l70kC7|pvBHyq`(V#RLET#KHame@y!mtaze_pjva9PP} z(~-m;`jdDe$b>aO_QavSSg>tn!bJ9+QCs(1oW%7tI20tTnb;(EL%GI_*vn}IHbJmq zMzQ*(zu9uvMZ4A>avx@hsD|B2;4FBaWSieJi~c|{4gvM-WxiJp6|>b&AF$XD>}|7y z1#M-ncp?BwyY)u_4^zFR)=vyZO;&BRU!_xogo~VY;8JakB)Bf&EQys^_erPi5?iIr z&#r?ai-7s^#uaTxxXjzq>fsMgQ^k5d@P%TT!{lz?k`d?f-di?ir5JkT16LE>5bspP zo?PZrRNl_3z+ej3SQ*DEIDo9K?8)_3gT}3%@GY}zi4Tr}C=QcQQTt3s`y2`r`V>c{ zwjp08FonI_z6>sQ?2##P7`8n^^pvq+L5~e33e6(958d%W7Dsf3&D}gyGC9u=-hbKBLehet^jlMX>7ScAqIc zLuMZx`uLR`UcWy3AjAOE0o&y&aB2v-@eO&49PcXo5<9^U63^tY1{rHJdNH*6Xe+W6 z3++!=*kvPHlunsf(^0E*_iiC|5lx(*@8CvYTSLl|3ARi3ZlN|5CR}o^8d*6!Jojjo zCG0JL#C!WluL3GPCZ#B^NESitOJ$J|Q5%-Rgs(>ceS!{sx*?%fJq}Z9)Ch;a_=Vlj zp8x#JY!B$XvYNdU{LH^nIE$4CIz(BwLSq)>3xX=G) z%e8z*5B-$Baw7VaUR+8PfVG8*t+SAcqqBvng^_`?iI9P{wV{EL)xQr$P2Yh5X&J*e zy4#h6?kBV{bqfMoD9r@=n7(vuSOAT*H9Y}7)Xx%`<@vhKT8*?#l)ClS9OhHEj6#bm zrvVY$%rw$i)BcC(w6B1z=X$=}OKHKJhWId?h(FWPEDfAy+W@Dnnq!~oj$^x=zlXfP z2i)2Gc-$$X?e@6Nt~+epF}9`bO)XPxgfLKG#aKo`_EM#CfmLAdA=O)}_eHU+JL7)` zq{ma6x2EwURx3UyR@dZJ?82jKI_(czY)n3zohGzL+#1y#r~&lnU*5L2 z=DayxaR9p3WTj?Q>5SsQAv4L`D3Q*_M*Ob$2Nd3M{^~0&*Jz-t1+`^RSJug6>xHr- zN!}=xHUNPp>p8BFWR*rI^YxOhio7F_^pBG>m^#aJOuf6Lw%N&6{*qIUfRaM4-9(Mt z@dAi8+-wCbLs&~F7#qknFs+krBOjmBZjJrVb0wZSh8Bf#JF3^f>`LfH<`4OS80G{2 z2B!R73|s-f!9ha}+85QSTjmdS{T4Mn_0qP;Vt~SAgC2jX!4#{0C5xYewYA@kWf&$h zDo9bmhe#iP_8?KLR82TIVPKzCyPMX;- z(87dbHaYdA0zS-Ee>B+606s|UHO&^nYr5oN^)aOcI`D*9;W#^_p<%5=B$C#rzvoQn zJG^{eI?10+8tRREoW)u96UHenQb(F=?10Y41J>yppG(PDqSV$x?5nQdwEbS|Gq` z^uD?Y9LVd}-eN>nWJ`OYr71R%_tX9lwVb^RO1D4`+nKGP(I-Ht{L@Yd1b|-wg2xXN z1!NIqy%k~?@|P&x;U+AL@hm(D6W~x3gz1oc@KpfDz7Xq{)ll5thU48VF#7{Njvi{x z&npzw@KpUpX^=M1pa@F4Oc!5ucEh(^|sR9HxbDdBRUKwcJg)v{+!Us^*#nkQt2OH=511SFz2{;zO zpb7>M?1_hiJV63?8DfDAx?-bn*()~g1C3A_qEY4}sl@q{aLS`~+@_0z^Hb--YWO=q zaGbcO(;qO5dO0AXc_i@m4?Y`U3$a6OrL~pR&GgH{?4zqejuRoIlmWPZIADNAf*{cC zgJc_Yg4r@|nf)4)kc7Rm1h}IYL^(*_h5APSgcX4;VLr(r$TJZ90QbUU94if8_ys#- zk$M1^boYlg6LJOLKDlVOrAmSFBSDuXx}*UPzx|^NKys5;J531`3IHZngm;c3*z!5~ zrD3=kGhq1J{g_^Im4X3%C%tIv$J#CTG}8dC^B$cLf#@v$UhSNw&hgs+iEB?uJM?=C1+hDd^2=IAgWDE}ao z0^(H(yHO7Ke2B3xGd9=$3y zUH9uCwl5^zu^N<|>x|XS3xYBhv~BNqpX*zXt>>({cr!kE_^n?TW9J*%|XTN*_<~ z0ayN~nuw=<1!38_mA0E!`qmyJC9S$d~P#=E_RUx8GdP;}BUMR=gA~FvI8@ zn3nWA<5F@-sPZc2Lj7e&7%M`VQiL7>s2T6J5AzyYZBXq-$0O;ZL8#=rBFOA~#XG)d zSq_m74Ym3!Y5Av^(-L0OwJ5^dLDd+PTvim<>UticxoDJ}!F1J=5Y(vxoE&8xtDvjU zAzxe~s@4z8DT)Xdo({yclGZPO?WviAu4$YzyAGHL?2}&sMa5^jZIxuznq@Y;Us_+_Y zd?XUTr3_nXA9NRQc)WA&g$LjPH}%1IdtxAlS#2V=%fb=^F}LZ#IXflRTb4#7Y4e@g zT&6QDOO9`sB6y>{SFKyvin7)3?$QR-1$@^!jOZNHg zHo{qkd9f`zMz-{vTFwP%cOq6-waT7419T$fpD#(lKB=>{k$O(m%kM}4#Gbhut&Kmi zsINKwSeUZ=Zj`8^+m(TfO8aIi(52Ar z39Qu+(8=D;DI{tnavV{BI6SBNVh!Aq&nYnX2wt`kccUm&VO$%}JbiU!^l%nRwzYdJ zsm_CPwSA6FgEPv3TS5%WBWoNj0=8?M#UTwiqj9{h3XQdm7sCSe3il%EuozUzBN#S2 z59Nm{9EGU}deRYJ<0uL8tM-gd_pykX)bU*B^sa16Ufnmd6%{YQOB2=C+Ajno2q>@@ zs=ktbGb?Yu}dHxE*6+E(~8Z{W1=W zEm`{wf@f_6-kg_xB06{9MG3~23h4v45@!6Jz*f|K4p1B-Qon#VKdBGO_uS~`i4bax zJz6jQjMvM<6`b0YsJpAM|I!m{yE6CM9bEgXJi{AnurfBFD`G`w+*{t?JCZ+ot^foP zgBRF8lb zacU{{NC7**Pa%yzF%0%TCx2v2I08b1^vC#Bkv0bk-IPR{ESz8K0z$KbXU z%b0RbAzn@W%xwE_qJCweTeg~%g+Ij_XI(9|m!_cmvKXW2aad+2NEPM0{>TBpow0;- zyMk!v*;i+|Nv!p%t+I_}?%kb(UWVUdYq-yxwotbLnu3jMo(6pPs+{GBTdD+|uyS>} z?Bg$cT>A^2<%U+dfjEyt_sBLTOSLRP4J<=MvePQzX6d@7_FK<)bebrazW$-OWw zDba3#g-2iSz)9{Y&M$OmQL;*nv~hvpdc2HVgDvnfhu@%&%)pmLO)-k9^qzvn;B83U zs!|gP^Hhq*-q9MG*STs#)X-LLJKHGu^&3-TS-1a6$p4#m|Nn$6jMPII1OMYk8R3r~ zEdNT#De4>a%m8(0Z>>X@zj#_tUG3dJ!cLE;=pE&d8#8jpy^#IvF5_T*e4X)u&6atm>!jE-}@YC+$^MYPtam z^MBjanoTrZ1wR{lMqaz`eZ7z0GVFN2h=0Q}9*EK5E(3NWhj!&;xbyB4P^b4OK>w!M z_E;M5b(`+Z(0Jn0x9jX7zXYy73*fe!ZscCfbUq{e08RE7-DZ2+)ZTb}v%4W+Upnx5vk7<@ zFYqKEMgZWy7J0HFZ**CIwL)@EZumSuzH1n!n;z6^#Wv5|d&bVt6za}MHR{grzf8C3 zcs2WfX*&J%Qr(q~?~8wf2uoAs|5mIFn2AM%wt2}8vQe}QBOnF;2o)Z(+lA^h4mXiZ zqi808MlR3}fl@+Mgwrq3Z%qqqRcTF?iwLCw!y+x(Y3VCjc zw*pw++-fNLd*oA<_2~n2ikv8trA*3-91;IJcY{dcmebII#5}6d(Cxl-f`jtrRHsk# zT5&Bby4diKK%R^jYjsbxO**nd#I`jO~XRrRXKo6VR2l)@a2 z6gy8r2poIa*Qg&mf!~!eGvYRi#E^3i=o^4>pssOe+`3^BlgY%x)Ggs0+x<2J8* zr7w_)U5UBkQ~VFc*kC*)qPkL+-N;AR_>mkjQ`NLQqsTPrGK~<1%V9jVBe!77qlw}} zw$~<`^Rm7mXpRfziu#XZL*W&AvaLjycjw)QdAKp5sjP72bVoM-qEfJ#B?WBYVqePT*fSVYE(FSVwB$ptb@^RJ^@w;fj0L^WH zgl!1C3`gp+lqg}E!I`Ru6@eYH3F~YW2ifO9*?|Ep!(n0?r%|Gd?F>mqL;|#4YJx&m zSe!yU#I!gSn)sKjFlP+T z#y44)0kU0QwmQ?r!IT${(?RUK_b$+>RsRkm6Ix6hZYtteMRBekGs0UIQDgQYaXvLn zKvjW|<#?GhA3Z1YptT)Ox`VqgveKBD16&Cq5s?NqNm240p+kqK@o6Gt+Z4bhL~5YJ z`XqiNxbmpNWE7LvHMo-G*4cP;HIrpA|153GJ|alhJc4Y2TQ9{;H?__%ZCPx8)zkcu zL`+FxB*92TK#rLb2TK8F&E8pjkA9-fiP+Jc_=;IHAq4q>R*1Ic5K6<$2m&0%u?CKa zx~^$ImqSV@Hm^KahSD7-6P{)GNxQj*&N~ye_UvxEe=XY)e zk>C)RP@3^cZpxc1IijS}L~vt;skt1H5piOIc-|cI0AV7o46|(bl6#B_N}FbD)gUTpWHnb=H2VltD8{U0sF*#5 zqZ*oTAlxXbmTj`EQ1jY6$tHCO?AyXxcTodbV?m3PniRk*R3{#~31-z)D#qGuuR@;- zBt`v7!fWPewZ*1QKwC+N7nGdi_i5X@1;YcnDUOYl!NTLw7aO%lxKg zMH|Dh0&K~~n#7D1h_*~LMcJrbp9jl1n6Thhdd4`%WOUu8P3H-temgl7dsaFW3X?8f zx>+RSC_^PLjuOCJuFrw|qFRQ92+lOn3}YD(XVh{Der2D}d>Gz_MbHck+L0s?g658u zM(xCH;y)CG3)!dQD{@H66cV=FV7By_-iXckf<=C6?bU>X!L!u-SGZRoW>g?9q*{;u2639Ab({o8A zYk0ns#T39P=HR3V*2jZ(^_#RPO|AjjbsBU%Ke*-Lbo6N1M+c^kpZ zYm6hxgh8_2kG9qOD}D1<7-GoktoN@zW^l$!MUK-PjH$Dxg()@LWgmk2w>gBzwHDC2 zhbwW)xom*^K1{n_x!0}%8K3`@$8%RZbU+Y|WIXja#wph`H^RlL7%Z)?R-97*sADgT!rN1iTH?Ds=7(J4hF)NOF zY0L$QhFa+Iw{$f&S##LN3t)007p}1}Z#C})LG7o1r+(=+bd9)r<$sD(?=K}srsG$k zc;c=umXL&C)E#K&!x@!hrdxh=7g5`lF-jNdSXD#W+=_E0W^A8wU9X+cydJWN)mFO@D~)Vb3nDWnGU>U-)dM!LlAhh58ERLAW`% z;e@w=k6X$K%I2G650SOf2!BnDfzg{6Ut`NC1NmfME8O#7S zy=rQF(ed7n?`1-RBxrZEremjP5G=pkToNntxlx~uHLOXcDOQ_Y5(H=V5DQY>TD6ls z-AYM&39jLSUA*+5^l;MQ%>C5C& zTZ2TYJQ~#o45On4O2y<6T4xFj@z0`<^f$Ql&A;W+sWbYC80@2`4;)D-gQ0>?EvPF- zo-P!5wo_g#FPI4QczJ?NfCKgA02G~?`1}d)S_suFzee(?3o=^mk z8D{pY{Fg@REYXVw4Xm}MKvn_Y`u_kZwaotZTo1VG1)oQT&UF(v%qD;Ey5I?>QY>^a zpIif&C`m&Y86)+Yd2NkIqZ7#YnVs^VC?{tP22v6>oJ%CbVU_E>8duq16z4 z)%l&pl~5V!dbkGzv}3Xx6>l6CcL*eR3@m3TiiiH||=B)p}5c-90&+Fqo@t45HU*2;d2 zwx4E)q}+#KTS}#c)`u@HA7C(&YLVN6@93Q>FmvxWzWuC-Pg*ZO)O?gGcwoA>hr!e4 z4YUKccgG7sTn1kd3~Jh2d!YF^JZl+YC&zY@l=v$&6y)sDV1f)`W=NOH!AW|cB8WRq+gNKW}!EiL=TEc$E z@;KD6m`JT0>XJ4w1n8_NnsLyu>1G<^T+k=AEOH^$m2gLma9@>hUlusja^P0BIPEIn zfLXq>06mvle`-MW0;hU497y&+a4w&DS)*^#l2Re1cClE584l>gbTYS}4e*d;cZpqZ zN%7e!m#iC&p@vN{QHDbRmlkydPo`JxK-TrZz-%k$G-2uey?apF*oAT1!Yf9dx-0%q zjf-iWdc9~zUdroP4z2W_sd})F)IR{hXucuKk4SGlV+w$8n6{oU)-jyol_A6VZfC+h zZs+jVj!SYi8~IQM_0jPyzR|S-ML&^Z2MOXqr(&`clN#F?NuqbsMI5C^hqfdREs~h3l=jl7PJQy4 zHtF^57cbz7Z}ch@aRPNNI%fcn0fjRa8}|Yw8+U&kIfCA; z$YL+qq?PKDa!AJ_M*~EPH)tk7lHqBTM#HoH?gS?p*1Oqo^6uT31+S5=KQ_r5l^b`e z@R20@tq8j1p5mRbs#@e-BQd9>PHF@5)dX0N!5)y=V@u!*jM{gccGA6J^n8J)85Gko z($SN1`=7bA{so2E ze^$f0;J!ut|M?asOBpEiTTFZdBPDMY4(p$;JVf#~5m<{_8&N65zdqTf;(E0BRyS@0 zO__~`_y)4LV?zU!t({yMnABP5IYiu@O>B)#jFlZ-oSbD0Yz@pz9ElnJPma7OaZdJI z|2X)s1r<%LI1PSyF1g6MphmB|0kN% zpd9ip;!r9wix72e^Q(2-El5!pon9G-t`RuzQLJMqAegSOD$&mFi5f6SpR1KkNo!U& zs&5187mLNKCP^OrxTP2-Jf)VE^j?EU&Bwh|Jtlr?gL_jPJA_iubp)ENYxbgiCU++N z)(S|0X3^a;8MO|C?V~?)_ge{rdkU<)<;h_LC98A-q)5$hlZ{0(gzq}&WeqkP^NDZS zMt}``v|$>hT0?y*cE$wG=kG$T{&Eb#OJ$-=9G~Xe+I21D*>>eFD)WvW*MlI~EZu}F z66iGTbs-)?>cNndYk`BvB2U_O*2n>uSqCi6L#GSFV`&C+R)+8nyxfBQMT2cKAU@wK zGpj7itav9O5^}WzbKz4aZlH0dvQ{rO9PwocRebHNL&@dxkWPc~xo-u+#MQjSUz}1#jqasfBr2qg*pB-$RF71^&XvrK zb_j0}A9(<&2D%+LuXkq&vl6Cvat?c<+?=v&8&@mrCb)l1JX%*@I9no4pUm{M`jSJrN!&8NNqd98npvQyHL*G7T3Cot#f+}|Ifbo@9+A* zAdP=!QPtA-Tyx)G)MV(Cxt{;~k>wpNT)!b0QYIe6a`q;+N>0}QU6J{I;b>uC{cl`G z(YLZbwhHps6}Ne@1&{#9uOV)H49s|3!_YAPz)1aI7(aRl0)@KMHvWX-kTZ+Y9Z4FU z);Y(5L%~gRjUp42x&wvbNOao>163F|;*F%6^uT3|GuW%vBY%q%S~+#ue7lY4Y!7j=vY;eQaG*rV@ zXHT-wYv(|OFkSjZVLq-NY(5y7LrbkGH3JOsv!Qx>S83D?ozuYOIroEN(T0&!@nyJk za-O^i&XwogsU$9FJy#k>xJQ2@Y)Fm0K!6GQ0rxUZ1%CW4$#3Oc_KiKyTRfH=s09rt z8GuyA0rbXhbKgP`%lh|GkVmI~55FQJq~sc(jYCb{35M4YYLO+$V!yLj`375-oIQ*e zw8~wyKhvLWu6ReIF^(VkK`>>|6F1nO8=|a(*?zeGCEqq_7_ZyzMrE0L1q!vcG#6Y7 z$dVsMZ73e-IUz`v zfy7RGqZW!4r8F)RcbgpAB2O>ALcZ1({ZHX#nX=@AY>|QR|KsW_fZ}SJwSxtBcZcBa z?(XjH?#>2xm&FMV!QI^z5o5bx^=c{r%v~oGqu~(HZ%Q{KN7ZP%Xmur z7kJ8E02mt}a)i8f`xo_GvTa&*Q*){y!dx!JR&6RZof}+ms9x%=K>FvxXaC?u0EfQj z>HO%@AIVr8KJq{*898B{U>{|c-$ZD%ftB4566LE6DP2W*1&bg=di{Exx{R@-w^o|r zB^avIFt@zN)fof+o4CgV`h{;5K#1Q^PD1g;3Gg-hI|ph&ItOYp$#2>C$OJk^ganBJ zPKrDkVGh5ay_O&0x_v)PY@&XX3KG6o>J1s~y0?1h_3pU`RH!?j~PFFm`3XX(#b zSd${R#6zRnS!_SoBzenS=XXSWB?|k2Ip5)wZBE=V*sA#?25YHOSzf zPS_f}pnM>CyEhh`<|J5k3<7f4}fc9Ph>y-F&X|(ZE^ZUpd3_kAo=ZeZLdy* zsGYb*n~;_y=im$NNl=2Q`*m=}kI)swhj`~(mVh^91a$$?N8GDk)?BSDwAmJec88Fx z-2j$%y{Abkfdw_|Q}!I*U+vU;?=91EDIpi1OEWkbrc1g)mfVDQL&RjA)J)#xRo`}L zPyT%Ut2h6n9sjT1Y)|D;!w2&9AP56=G*C6rKIH9D6C*rDaOlP8Rz>H(G-*ONT7FN2 zk%Tuc@8nF!>pNmZrC4z(6 zXhm`PJggk@Y<^7g^W4t$Uv@z%{3e>dlR8w+7D51;jJ*{BlhcgWQm#;vwrI z3aF*(6L4a3uhs8ybVr1f%x5}tw9VW`oVp$i-ZC;-iDpYwUVfC%^$h7)qsLk{vFb)G zN!EjO?Wl^$yky);V+Xf#C9^kJi-U!T8fn+J?S17r$1q=G8?auqL#Y97#&8naiDpn( z7$S94ETJv^Y_p^eDNo#185U3(S&QIyaYtvhaBE9E7CsnUJ2tV~?0u*Z8gmX)*63b4 zb==i7EqPm{5QHoP6iaUUPVj3#sh@)LGOa~1f8N@VdEjf5TefZ6GvYbW`Sa&e1X_pa zkpDDG2rgw9*BM=*L0<#xsFK1#;KbwbT3XTtMA@C+cJ_fe$+cPV$(>9MX-VwE-K zq_OFyTsRA^v!Wh>0ya%|Xki^I*YHUks)V+XpYwprN0INPH@MDKq1@-xb7}^IY8*QaV_P12_pzXro zXTXea_Ny!PQi~iwFwpduVQS}nr+%IQwzazIXxk1>uN2;3Bb+r1g_sTN-}Q-bqXYTc@*gbb;W35fXgB{wi zC3zM7hWt?qE#%@hR`14D9|b=w*zR`^BHQYQ*;MxK&S9;guXsftRkt4h25JT-;d^OA zMeDUt!(4Op6YSg{e%J4s@hK9b-aC|^m|#i74|K~McQ}FoN<4c5XZ?5B+?15uV7hHiUy^#QZoZv6k>B6F4YK zi8Iv^Ga%6bqsV318OThmAm(P7f_@C=r%dB! z$Mm7{CrBAjs3dXghNnQLGWifytb*vc9k!Q)c43|;4612GTh#a9sX{lrYE%ZJqWDy5 z`yFsCVk`J*+cZxw||9Mr4Q_L4lq6wcR?5-MlFmpnj(wyKbL?~g#@p*=8GH~~T3 z-gG)}f*+Z1($L@vewjna(vt#z${SjjS(OBWa(?4H_}+U$)P(=K4klb4p`00^3}_D5 z6VyIDd5$L0*)+})--f>9jlcyI)`7yW3xaV7X1ugG?E2!{1%Fb4b41r@W)Pn1<9>9i zSU8FZb?Ft}q6vqOJH~&+C3X@LZV48~_|6^WU%tE7I`T#;kb{2wgBJ6T&in*kLxTIi z+O!$g5}gL*pUA58`|AVS=MjNsNMJw;G$zoHii4(ExeZbQ?4J_^!JF~$SfBnXJPfdR z5(yJ1NWt_Of&mad_fAo9KsE1mLNfo)0fb@{Y7+2&otK3(8)i*L z?HBUr*t?E>V#4@1>3w}>hzN*V$H{b`_Sd^sm8;XoxZE!&BaepoxO`Y9n!^<3a_Tx2 zO`3M~tL7=DDW;A60PC;?`f@#WpNa>~^UuOrDaN0d`R8Tyd6}ialVA=6^n@E_p~bvy zYtnFpoC^9**>H6Bahg)$&t7f&^A0EF)mn-`^NrcxpAx^o z+(5Mw5-^uCd~>YY#*uQvyawTf*5@M|%s7fp=lV-Jb;D4L|kL4 z_OyDDdwov zX!_Df&}h0pzON>13C4l9Lk;Vi@h8&5)ddeeRgH9u__dJ9Q9g3;QKATI;EeYh6%3ag zrciF)>BMXs7a5g#1_^-@FG%)-o8kFC!P>7ln`rh`bDjm87Tu4)mHQB>?EC%n71>V88^R2uA+! zgb+J|&(~cbfog<`45=+GV33yt)}bWICc1BxY%yFdU_;Dc4{89CcUPu`l9_c0+S(8I zvMzlO>()MDsQn=szn2(giBura84V;T##czNyrA822=?wWo)#&Aecm!Zux2hpwSYTzKHkmtZZceMa(l|J4-14gUOafcS>4Hkm^Y?GxI{GWK zEX+Zx(VpX=WuA873TEoR4Cpfl&${P8u}&qB=;-R~T;z9|CQ>k4;I=(0dxRv9kHvSB z@P>Ac&Di$4Gp(?AC z>EKIVU*{k5m%!F8zVkiw`P4e2f zF7i9wd?7Sc#ZLj7X^d053n_!9PPaKWZ&hND90vTrp;_0QSoLqRO!VaNZpnjW<=RA7 zCq^%yzt2gEK@iC%#k2p)AnU>EzBfRZTnW{7%Zc0ei{7AK#&rabr@st!umV%R$`Wh+ zHqd#N+&)UZ6g;KXPpjokECiNMJugjS_~SA(r0pn%U!s~@kC|)5^{06F!3nR`s^nOr zt{A|FtP37PLG)+W4{yx8snTc3RqBShRZ(_9hG#Ko(ok5k#!Ihl67KI?WHVad$Ru)V zs^jTJU)5gp`rYbMXZXQq0wVSl=IxD%YQaK?yMo%NJESl2+1HvwUgTT{P_GqASA?JV zaJuuh-9$IUcjH8zA>~Y;nIuJ&+rN@OjVr4#N>ah2rX!OvN!vu`e6Rg(oh9s!lpya4 z5g~h)l+y#hP8N7GAEP5$M=5Ad`seBro*Mc6AItT>6Y6uF5dM%NMLqyJwW-1d5e9LrUCiNy)?q`j2h$8FS!=`=o<_ zCMgEv0&7^gkebUjNGbj`hCbw>gZ`fr9s>plqRWh#^gRgzC_uvj*|bSo3;vI}RLzAu zt@s=jKF+`P(2$uM4u}wdjs`e&!PP+jvmtrYxEEuPurNj^CzGL`fKQW55zh_ph90)+ zgva5Wy`|V|?q=suzk=RQGmH>-L%*-|&Ecii;Hd|5)+z~(WjerUtby=}^a=54jIpQH zqiA8SDa%TvBxh!$-ec<4W0KEe>GJ&$5deCH<&Qdyc{K!$$VQXOf1h?x_oKzoj z>2R><+HMNXapbjKtxB!?a{e*jiGg-^{rBht<-ZnUJ7fL+F@3rluTkxuqe+DD+C>BBN z*Jb>ApjK<@O_82fXE(fr&LCoa0pX`NrPbD%Q9d3&lqY)?+`(z5ZLIvYZV9O>*I-Ug z#~xXJf)*|gTo`xOY5yW$=lZR_t+E>CvBCZ+w!Mt?{C16y5f5-AoUwh4G1bQE<@aa| zzx?Z|FXcJ?EGxCld4(As;w13&DRx^kISbvz&DmWY=P*=DAC(>?=d3W9T8v>0PQtxB z+o(ppysef3=DS5)2#YlCgo~xMcr*sWI}WLuEJ< zT>AnC%u>ly-1cL0&Nt55!E1#Me;g&$6i)kK3%xeo;UYh)3G`=NDa6KY6_>q~d{~oW z8J&R@eB5>(KC&V0gVC?PjcV}++XH38c~tR1HYx4NhIPPTfkfDh-lZ5Ci~G8a;FJAJwBfJA6vJ+@3@*MQ zyXANWyJ>)J&jhCH8S0#N?>Bc9MbvbWcAss2LC}dWXfQ5zx)FgYPVmTgtF9#+$IxZL zyQer^0S?xZ+bz1xI9(wQHctp$|zubuVww}5| z3Ae&NOBUc&TRt)Q=Wna+vA&@R>^`-I8f*b^60ra?S2S(0C8t>~nyxR1=8&m7x-eLxt4Av=erQmn z0^)1nytjxjzzBl08gw2N3TuN+)ATZt<@xEX4t4 z7^Ha+4F&c#gYHiAUc-_X(D?Nm@32La33BjFnvty3{(wWvQQwz+ul%X640**^wmLf9 z%PwMD5tKKTIL)Gi_(bXiZ8oe+*pIJ19Yi;TSTa%JX) zcp&*+W7IpVid5}1Y}A`OBeKYIAwUuk1jbI@U4iEr+DDc*9gy((`3aUdW-QCa9q5Ed z;X%eGbR0)@0;QBCU9Jw3_=jo7!RCUkgBm{S{mn+=9oXvvf=*p(Gu=DyIINAwCM z<$9bt@b)N$*$8};G0+rCh&h@l!sIDAlBmo$f=YU5w%{AKjDra6sGZ5Ps}6v6xV-i) zNcr=(kaa&}&)*HRN|;#}r^T0fEc9yQz_%z9m5y2$pQ5|+_NCgNgBQMopM9YibPme* zlo|BS?jXrFs=(%^6skyPT!hQc3{$JX&LX1zSxV`od^|DjhA(?eFkxUFNzyZ{J}h(5Hs31u|uCKWZkr7{nCQt>P6ME*JF$&HMS-4Y_Y zw>-W5rKro6{gs0C925P)FEVlY zQf;1g(oj&zrO+#$cWjsT{PUPtvF1c{^s|)chs*%cFjp)@ti^(NPC-xY$Ti+D$}mK1 z)XcX6xtHjJr}gd`iBpa4j2HR%MYH_lwGef!-&}3dK}=)QJ(o&04t&f1r*eWC)@Es=lzaX9{C*8kh#c>9DPuh-q<0s)`?>t&>aAeuwK&wMtS zSeObnKO@`_V6QY3K~m2?H1L##6Sdi_6a5(!-1ifYwb}0vYjLzyIehF&GFB zRp2*B?= z+z`#OR!k!Q@;uZ&1pz-jtp(`+@HDrxf=q$`JNE4t^x&`mHu1u!^TGbh&SilDgHgg~Vv%9(Ao57=~d7%E`Vc+$forV4KWeE>xy7;9^h7UOPz|}zhqi0prSk$-+JmYTqR6;RiJWTVI0pFa0@G>K*ol3I{ zp~>&H+Y{Ln8h%YI}epwIAg>+=;LhYezsRkspTrzX@@A>~uLl7Q@ zbFtEA0cA`uN;FH{g^y~jmz`u9tzV~gfjF>5AN<1YXaGjARXb4lYhb%S<;$dr{Gcxw z2F?{xwE&tR0|CMgGVf~m@D*pdxA5SjDlgSEXZcjgFX6>e=b6-MuZg^23|ewRx?X_os`8pWJTb9G}N~Q(>rkZ zf+i!FGf6FoTT3(dAzHa(7B6ipnH4KOfG~p=uLmYNrA0!MAJsSkZCY$SW7-_7_-|VH zVktTA_(rn3I;&{loz`v5Xn7H*Rt%X!uDOf9NENiKm^#3%P)e%z7h>$4Yq~UrNb!?&gfvACcX{p-Bu#7zWxE;WS6V>3qWZO8;7DN#-I08esJ6?WM^F0`%;Be}KlapZS_8h=U*V0&F8lZ`89Ov0 zdkp4;u$j&~DiTZ7ctz6o`jD`en>0`!A)5gaY8sSf2QXUL!A&va$@S4{j6*O(qM@)t zKRD{#z)K5*V$YABy%Thjx24*Zr89Riz|0Pd7a9O@=hn+%JZZe;+v;aCNJ6S2(W8tP z%GX(+E{_-du#>QeKt9?xc!n)8c!q7W1O1sRyY+%?wF7;V8}3Vlnkqu(W=tROw#7{* zrP`aXhdma#95P1m4qFnk{uSGDx`x?F)b!RZ!Oh=Dbyy>)*5p@0Dmqm0XXfioU-}#O z!bkz29OC^HcA4T_XM4YDa;DZ?ijj-EEGN{>`;pn-*de!>g@pQfJT(S=aQp*vtvv|^ z)>Z@LcB!YkLpV!p$M}s?$GzRv+RXR@Ayzeji|eP0=iAZwoe=6!*T|u|w~hBw<84A4 zlxQoOvP#~>f#fQb3P}+CSwLziC+kI5O&dVvJ0(`$=#g870nKU4u&Zli8dG}ntZQAZ z?<95Glr-w43&8oXw-u#Zyh^THpEd)+*~!5)iDb2AFOcTKyYb#V$Yd!MoSOs}Vt0wm zLQI*j`jF;8G!(2U!rjX4&Rbn1Q!j9KHm6p)oReUIW(9BmV6t&mr8SXn*eFus6Bv&9 zexBTHjH_>0+C9V267_BVdT*xxv1AD1MR^ICjITFZ4`*(!Rr|MxMB?ZvK{$mIs$bQ* zaIUO>gOdkwRSa%4o{96Fqe{Z=_w`4|?jK4Jet3rQM?fogD46pv@U;Yk?QqvTRL|)7 zAgx`wd%Jf=UtJgX7=zb9Mr-wm8#e&k*5gUh!X^^7*g2Y0C>^z3*g=sHg4vc##cr+|P1GSI>^5ZvIGUbRT?y{CRb$(Uu}0 zvbI*~3%U@LWUHGnv{o2E<@|>;!!a^bfk6eb*BV}yXgQ{gQXg(!-O>}6& zvV>(v_~wINUpjxAXb;u{{GA%It(7-`Ci@3NH-d_yL;#W*15=JyMt2N#Di2L^1}JM=mFs|6m3Y$Mmg&OEt6mE z!S1UX^oAN0e-Mv28>1C4ck+sWP@**$H&d{RvPOup6<(A&m7GGsBp`NYn6DQRv zVk(|UK=+!3{SAmwOC%xM+5BQC&w@1zYrM>M^hpQ6tb{v+B3Nc3Y1{p?Gq&5;Gco0CywlvNDWb>>PUnR`Zj16xGa)DQ=0E&0iwh<`oPm6 zvM#aZ`-Cv)b-AouUo0TN21cuOVxPZAyGv&b8A*LhA`4bvM;XQ6A35K9Wmni%AfxK$ zmH`2I-g^+aLf9XOsYA)gDXh45$vc*_hcjI;X+(fgOJgRP7>TQ~xon5NHVUABIcy+f3PQ?984{7eVH2&j30;G+1qOd1#t|YU67_4hg{MhHXr45T{u6#V21}7~ zB+1Ief4#M78h!YC5gN=SHRrd_MQeAQOB-Q#moJzjd^6hs({B(6jR9+6F?A6HFJw4$ z2zcrtlW&oag-|LaY1GBoXj7|?Sl;R$fG1dc?)44!rOu3V`zdD8L))c?1MC-Pi@4iN zSFbYdCagAfyCv`C<&8RPT;K6paNWj%Y&red%MwbbUl zD{i)LZa)K$BZ)Ql-M1B!at0eEDj@i&C!R za5aHQ?_VZ3L0MpOkzXX+k(AL;p;YjQf@O|+h`g;ldaK0=D~U8fQ@+Yyetelr_AzGW zC%(&rz)Tr6f-ZQ40D(iDgQD*)u@-7KMEgygkB1SPz?cDrdE^x(`BZa-4;af3RCzeB^fNmDHvp(YyW@&whCnuCmtM7wM;JoNiba%C>kIthqYU zybFkcL@T#j5ag)3UmB$fvBF5F$Ck4VWDTjm*k{sxZg{6%gINFxvY>3-~N zKy5|H$h&$r$Q+ei^WesMKrIy#tGmI7qto7ve%?}F?pC~Z4f~NOL8Vbp>d>fK@L=!K zgXGF<$CfJ1n=Stk>_7CVJT|{$D+$UqTKGzfy ztU#5=L&YJkX!OW=$`S){Av%4xBVe;g9wyoxyHh4+Ej|GS3`&p6LQ9|e3uy4B<2r0(Y(i})+B}SiqqZv{u`H*;}l29 z^*&?~#Q74y=HlFo(Hnw1k)Xaneab_{p$jV0r<$__2PRy8hCDr;9QeSFT6Ub~WFHV2P^iB=#B~mBK)fGh zd;sQuS6)+8>}*hlko?(O_YyrDnbhS@bs9_Ld`G23MbVUYLNIPg^veie@@(m4>fL-W z=`()zF79rY{UH&L#xp&+(-OjV{)opN!*v`T2NaV(IdtL^l)Z#){ZnlsTz^6#AGZI8j<6H6Bx7mn)_EvA!kYmL*)qoy63;%V2?KSaNIh}Ih?jILnXFo*7qhG+-SZQAB; ziBy9m$8%8>+301{wa{dy&)J!YU5W~_;>-ITLs0|Us*7_YRSm@aei+h;LMV}d1!5|+^SdZ z;1)aNocq%7&@%Vi-t8sekLfF*b@SZe=@X{=uy)t<8)?;S?+>Zkm#H@bv&X2<>TuoB z`(`+MB=GDs8geERv9t&Wl7y6Z7+R|!9^K#ZeQErJ4+@^^a*ed!J!*1_85I~kpzH#l z(fo-XNidGI_7n$U;1vipWlBLWovn4e?LSkUrx*_yL_|x?@d#GkH z>H=xvFaV%`)y`i*`5U~K|CgLWefg_)WG)f{tUo0%7Zlh#0!<9O z667Rn2BpLZ20{2MVOw^jnk%2=ZLojIh#j0Tz;^{MSm3z>6cjLb5e^EVqGvNNh{jjn zZog{lI%oY}rT4kfRxNrSk_uf2L(J&CM;ewND(S3+$BbKxqCVt~q=rQd)$&KMRGlwTovG=$Cy!Su#^Tkl zF4vhHwm?EP)7ZU;`B{FmC$y;v=Ju6JzfkeGWvo>{>5g=~qCVfPvix7p0#hxkho6;G zq}kWEC4m|PCf<>lAwL{FeiIZ(xGuJ(Th`T{4sDK8QlK&v(lk8){=^_6JTsM4<(|s! zhs^gc^~K%&(Jimp2!eLG%L2K4B!&_kH zMYdqrWzkUcoi?U4AB>bvp(ycTz%0~Q@MM@UTQXZK$Si!Pa%EQ{jQ@=Si+8?nx(p(o z^7!bmbOv>cx3g0aP);!Dc5{BwsZw+Ct}@@YsrW}3bj=hW{tB0PWqkjDJ=GEUn?`Kn zj2|=&`nEhaSmlMrd{;rQBP02(OknpFG;*q6@^oa4JOthiOjV-87EYB(>-yj)R`vZq zE9w6cf&b{Je^r!xCT-sh;%C0+;u|nvnuZt{?~Z{4kTkM4wKa48S0OcO!|LJKGR7B5a-$I~otm&r1BZ+hJA9 zppeVdj7VxqH4TV3k(5_ycRM*);LU1~%PfvaqNqwLuH7GV?@rgdh;n@>603kY`n5 zZ{@hBR}NZF{2zL~h-hyT7}8fF5Dccq08;;oElQY=Ie0|>jF8-Iv6)u}2nOg0e|pY$ zVaJzCS^syL>kGg@y8)HwN(|1MB5Sqj4!8cqfZmeFnG z)^={3o}S+kSe~9-*q+&+T3+5e0hZP-&dHLWqaEEOIVNhYyT!EZTr4BfYr`*C@{1?y zaG{>XOBwKBA!xap6#GKGxRzDGt!zDs@*(c93vIm6EMX99k&FFB5h0?dduGxI=Adc( zN*a+Xxy^BUt)C-XP2I7332yK_240V-a17_Ze@77rr z*v`;l5Nwe7Ox^4F?ea(yt^`Hn>I#m@gqSo3HAnW8aC^5oBrtC>>~vH(O-~9aT;wFN&U`vlc6&p!C6fR4(<5xbwnb3;2E| zZry_dG}PP6C^MnUeJ5zbD=9>VK@lJ|#afq3oIN-7Z-x`Z4TU`l02HY8XTIAjBc@0I zwd(!2zIBIxXg(O|cs(z%%OBXR3uybxQUgNv7{_lvr|E))KdY#P8(I5_PcsgK_k(0- z=VC^kAYVdNZ(M{kG5(Mk7-#)M)LRtQTvi!FCi7V~vbFSFFpS^WnNP`*#iA(QzlqeG zqDZK(sg8~+jCMN=58yX8L_x>SIO2VwgAm;YNxEGRdWa&G?o=F{EyL;)Sx3wUB|jWG zUhRYjReq={hf&8I(&QMTTACGvfgnWz*%xEc$b|busnQH^x=-YknJQ$F78e~QPHkX= z_&WMAf+M-ef__OBb*TJVwJ({e7=|FvloRE_C9hV-;6|by4!}JY{9r%9Q0!%G-GpjS zqa$Cp+1E&8*Fe!*-c)S9(jiZ=JP*wdnL2D9iy|=SzV=6?oh-}_)AmBY*qsXa6ut5B zLvL?GZ(DPqx*02<6LVs;Ra&#s7>XDj)9KR#_DBCJU(OlyBa%^E)Fr~H?Z;|Z;|ztJ z=B*2RLvLd46~MxTEI)?r^l`}bKnz{?2E!wO_RDaZcDyQegIW>wPxEB@9Vm3at8%EG zDK6zAoS$mf2tRkn%4gXLrrq1ritrm$^1M^rw^1m;QgAGrrG3Kv)DuUr?RtOk|A1%> z%xvRG7#}lJeuzE!NW+dzMi|UDx60M0-hAoX-kx;?lmRM}rr$pc5SPujw7^i$oF9)??Yc4Ai#>=qtRctlyH!3 zh^=qbsXPuAMT$UhR7Zcz1)IoBq2&xBnt&!-sKVS>mk`5Hp{ZM?U8qr!ERJbK|7?NG z7{ldKr=T!zgmH+S_V0_`PjRYhqRuW-N*Oxhp8<486>+HG8@*R7tdq-_;PI~kw*$G2 zw%@7D(py5Q`2;tpJ3|OmkBNaVfcM?WecTNZ@aH`$rvvU^}+QOa;i%UX7$v~zTL!=^8QSk?qr1!=Vlbt zQ2^fi;}4c|qk2A!z3s`m70Q4@4%)OwF%_PWJ*kcrwtPJ%t%H5`h(|JxlnH}|bV0WK z><2D2JQUpzL(MZS4tha!0oCGRVWrAkixR?mo=gm)Tg@aKjMD5JD*hGam{i9BhdYR2 zPnNtjv)FO{I;dl1dHWAd(REc-1)7LHe87t;o9X(mBApIxb}|yqiP4=>o==BVrfsNg zzSwmv-Je=A4~cv%Z@D4e*O?G6v7{;5$VBr{FNL07{^0Qr*#|?0M$Z-@@5g5N(eME$ z<3$=+C>8Vs243cXSg0GOV)~dqixcf=?MMU$fO18C3=>m^Sy9+cGI=;`Tst-LTp-N z$Qh=ZXfyN177aTvpI!ee{WkSo)w&(I@uIRm82Vw82P)g_7IPEKIC4Apnlcg1v~01` zIOz(XhmNH%H+!rI8?0@`RIi|tYK{3&EJLzCi?x%b8sT*OpbJHIaI~2r#a;bEuKB}V z80$&S=yPo6gP9gwbM<&}L;-D)T#N7vMh( zD>>Vp7!-7aMl5>#PmUmcLSC(~_HqN}>HuGb{4X#y1mAQAUHY&N9`p$|5Fn=vyY|sp z1MXlGiTWyax?)N2DVhZX*kUA@1RsJu6Mu=@nmg>QAo$zUtRp>Z%O zr$zuCd&kg*=BE7(?L-}@KoWOllSDpRJ>XTbr4M3+Vgn_N9;R9p zP6y%-nf{9!^HPu)@H;}zljG-AR||)8ErCgk0=3daQ+Imq<1GWM6-SL!FIaUQYPRig zP?NeyQ#Bv0GaaEmrP4QWOl1CuGaAjMB#$}p$m&!L-LK@Tqb3ABknwS0T3duQIE2zL z;glE1anTG?=y*TwWgirE@BG(@FK&jn(+0f97xyIpAg{e70LX?eI8B--(yr7yV!D|G zs11?(L;{=-%{Xatg-5y04keDcI2ly+MWkLArEH0G`-k)*JN}p6hZzz}P z;v8KkT?Y6E0rZ~9q<7fsk==qU^xH)AyGS{LlvP(Ztz(R>6m;w3ZJ7B84#N&i7KmWy z&0Lf6#dHUn-70;INX-IdyuQBq z2>2LyM*+<;?yRm=M`$3vZtP?BhH6}oTT8RgMLuEH0zjt>*>VQuo|KghBX=Xw72=*w ztY!rm(aReeZ56%_+d}ZP9N;0hHy39^ERAhBmRyiD@daYi8~fzB;5EmZNGufmI`;ZV z*!w05;&Zl%IZ&~dMJV#(rEAPv>HD=6vzDd$_cfV2UqEO%^KXTyDMXo2%-UCERlVMv zoEu?E3qX2jyDlh5vkVqzuBd#RNX>D^06%oN+JmZpI*kQoWwP)R8O~e{O(&xt%4-O2 zJsblwfCg7VOeGeImYs2XH(9(AzWx_>&$ss@&tff*10 zPyWQMeGQK{*gPF`$>`lRPIdp9#SZAndqSox17Kpo3Yf?JLCYPAs#ufpL=ZS!>F#;F zl6eE~?wh)T(I0YK?Fqu3=a+dt;)ihd&GaI2Wh&{9{- zx%tLpb_KEdL|}FcQcJkNd|j`<9_?F8uqi?A_ClwSbd ze`%t}h9?XRu7e$B?|eeeCuYa86bIS+|6;1V$hC99@SF@oK<69l$PG zHKpvjf}qKeB~Lz*r;Mh;CS8`Oa;;&iY1voc0Niy|G2^Ottnezah9iy{a=ms3oh5v!fjk$}WEb(| zfcNd^E;W6=IJjq}mc}pWmg)MwmK?ebaZC#%nW5(LlKQsA7^;di?M?9au1IpOP;|Tx z*H2h+K3;e~`WZ_}M{nku_Pts}^2f(^xEwbGLHfymSyD;69GGu8RT-tC0^GyN7_!%I zIdvy}O>*s)Ev6OUD=)1z;w`p5Ame-}JZg`Hh6O$p~dQLl1zV-;l!zNV72E7Io zB$LT$zGL~ILENX#+_`LfWrmtuhskM_m zdV!&h%l)I>z+G3g#cqNX3G*xMKgbS9EG?YAN%k_Be~zm|V=r~0mP&!%59lBNl)L{o zaqu^9@ZTwgzv(Cn%wS3F&t1N2pF2nZWZ#;X)1Uwl;D0I2qYU_#px}R#Qq8RiWZ%G{ z|28eh#CYj|v@J}E09a2v4Y!Y}6Q=Tw1XBkxi6*lG&REiF(`vf$Si5Irou(Z%vO$NH zV&V#q;YG4?y;>d{sQ`9CoF4&K6oOF1$-2stUTJ)MJYw#fL!y7^T z(%$Xq>n0F9^hAg zZ`q;UkGqgKAH?Bb-wHwjiI0e2mIDHi*9=b@S|1J|w*B2UPjO$=w*}1l0M7~YSDVaF zq+V~Dk{=340Gq3Bp*Oc}{6!NKKKtWs;75QBS_~|dLW2^%`ldZAiF_=6WG#7&dUEvq3RHvLf2|%=km#pU% zdou?3fb@EaR5C>#*eufHMGN05(ffYIfi)!RBmbai1IQPs;pi!7E}EFrk(!yluoR>7 z*ip#YG;m}ySGtD&kWk^BO^-IK7f(f^W-s|klTN^IJV+a)m@b>kXX9v5I4mon9BLLL z?wpz)oklf2QfNFHF3YH5L8??zAVGIzc=p?J6aPJ|K;rT2d<7tJ7+~VpjPgyo^g_0z zPMy{a9}q|GB1?*-z@p+bE?WY1dh7btxJ-0*M0Dpk7Ib5T)e?>jHg#d@c92CK?!m#LQwlVzRa~DvZbFJ~7do3E1fv=CT+C%({mP5SRDkoOC*msh(YH%8v zz}Xo)SMwi!L+P6IR#X&-xItFqNbQX6>J-^~qkwWf7N!^Dq!^2##CLP*-p+%K=xTU* zt-C8;D!mJG**aSL7^IBbX{RB1>`}>*cYZ?^6fBd09f+k0jOy|5uXPof%g2QYhH;#O z`b{Fsn_|n$yd1aHeDdEe_n<2_30TbT_P;9NrWb%tAS@@v4kr$qV5@XxPdeN5m=169 z2>@bhScmYmI&rFzuisR*Wk+bgjnZWLMCQv09M5jDI0*Jp>VIvd3tzv#Hnx|wq*~d9 zD9Qk1_1AE)9dYXJ`ABGCDijthz=hgAXe@$M^m0)(DvmaC$O5ue?^obpJ-AlVh>bVh zaGN?6WuKZ`ylg^v{e}V>LlINl#gY}+K;v;*F)E~ zpw^1Ra&OOiW0cblS`Y=XAv_3fYw&&|2<_Q%b0=?YhJ@&f{7QZQy2 z95PO>rX4bi>)oGMkLWg}c~M_I29hR~C>_bN#|L8QRjlK1%}x9|ZF6;fQ~Lxw@J!rY zv-)VY}{6O zyTTc%5>%6(Wg(Xvpj(UQ)0hLy5%#0Al#t9oZ*aVLf6T;iv(0G>pXx?SX&NuJBz6}Q z9<3b^+;NALg_Ng4z{qE7Tj9${oU%n8H3YDqwBQm>UsG6TPQcVFuYBvtbhR#_3uY@a zQ^$Ac2jk93lz_3yw$HD`v758vAs}PjvuWj1p?rBykJcO==={~_x%dv4&znQw?QyeY zb)!#ZMU``Bqdz+%V?DR3gP%pS_&TeuBE0Y{cdVLZWfrrC-6uD9@><-hyJ*dA$9itP zTGM6aU6xC@HX6kJJJ#cTdQ6*D8yCDt{ zUX0o@`#}XL-|T2Q2NDT5nU`y}o-a^gxX7g2GvAdf;Lo-%2+|ceppp^aRV|()=B~Ic zJ1LDFURhn|F5QJ*FPUM=j@=cCd2!+u3JzQoieQ*!|MW z_!GC(BbZ**)aO0{i-|}ZnE`v_nmxDB^{w+O9bp#mHqzRC<%jOLY;p=fogbx2-iKu{ z3eY78@3#yKu`=XIlCnhva`uMJQ%cBl)lf0Wr1qCZ+vAZW4@I8zq@&z*fA9nzd0cop zGH0S$()C!1yuAUl$Hc-@eO9Vti=i}nqSsU%lC+UM8TG-_oc-_kGRuxdOnsdUcHzJD zF}_LZ8`Dd7*et}%+Y}z1)-ePf_zc}+d4g2eQo~U>L2&?mL3HmXWgwz;6<6xMPI=!d zZSvkIUDE{wokV-3)>wa)I9GM?d>Bi&KuTq5{`viPCXWkXXymRi>{V+#{kP@;ebw#F zosY@lmpofW_C9Z$*q{H8t9Jm7wClQtCpIUx?M!Ujwmq@gu`yxCwmESowr$&XGJo#- zdB5*@|F7!m>QmQsy1J@+?|t^!XYI8bbU&JmR3ELzGAc3l@@9X)|p&}wNGdy3pwJq zIiLlbKpUoIu6*R{s;>4h0&)-tUL3l8a$KU$(zBivS< zwpkmr6yqcg{=So(D6f7EK}2PLOX60fim%iR(NP|Q1A3e{mCa;Y~1 zU24P#xDrKp^xU@5d)CJ6N|{oraosjJ>i+A$GxoxVmo2ErsVV@;w~ngqM6jeU+J`mT zSCvvVRw2()cSczUSPkpIIz!n%0x>sdjY1n}s+?IkKKSHcT(dpL4|7D7Se_lysx?r` z54By?QwsaK@a3dsf*JNU4Gz`#$?vmbFEnaPIjBw+qw7ec_r%RB&x6L#IMYL#SO!*6 zhyRLru$JGb(+##y`@0vQTs^c5!U99kM^B)iN*CtCmz)6wnLL}hoE`-QkKwljxq(fn z9dR0N1G~ zkd!zRU;SL10}{eP9Nym7k8p`!;k1(Af6DWxP)sL&2+6L0LBKw4K!y0#Rey2R(OeAh z;0rC+Q78iVVkL>YDfueZS}WCd=%3ldp8S4?BQy;$TQxax;M$20uYV_KvBs=k4cWWv zD%hou)$|L%9UAO)@J=sy!H#oLnK@TO&H00W*wsH91(&4=sneGKUI1OGDf^6|8Wk^W z`+fPo0|$=Y-e>;gr#sIXPyn>*#;63)aL=lvHiiq(mDJ@B?4k+JV`#THEE z6aJ~(_6nSTWr4}IKZb|?wO;DS~d_7D=n{Xi!AwC0l0<5@X;4m zJ~jICRC@AMKCwo}2;8yj(E&3B<=kNb-bpIY!W`fGy0$_#4!ajuKYbFP@6?6!sQ+vW zRk{N7&g^3GaVaR5-}SNWUQn$W4cA7PEDkQ>g#P%)ChijP3?h%yPO_g7I=pvnJ0>Jo z*Q=NRFb?Zeenzx<-X{Ou7Po~1mGSY3tl*KlH~N_8jjjD-Mc*rxX2=$AXtG!HgJtMc zqZY>Cq43!a*y=VERP+#DZ#q-@0qwUjGbsijNSa^K!?CH#)5|1((?ir5s(WLee{33! zbWL1V8Dx;u9MUo23%O{v?>!DBXB%8)P+X=p8NWU>jXKPO--SE1_uEHtjsNBQVz)wT zx!25IVg_D<4CgQYs`{~*@o@{?9Ke3~3$`5*`kg|XtgU0=d|G|ZUqtSC-aw@y38oE% z=aTBAIwW+{pRmHDzQxg=^1Yy5}s~;l$v=uFeoS`Uw0;5-P>gqG*mq%AQ|2B^NyQ@U}_N|@%fA3HK z*>6HsH~lUA0#Q)I{NEFA44}dw7Ze~%UFS=65sU9^S?OoRGPr{bv|_iv38fn9Y|6YX zF(-!#B|WDru`b8#;%2@if#>(toL-(d-fmf)L0%@`Lj@)rs{KMl~O}Dz>wvKe%euaRoyO^S;TrxhJSo?c4Jg1v$~%eXmUD7T{-Xi4K(Qeq3nDNRm2BuG1nR!quWqq z53*7W-XukUE+Wj!bwtrZ;=qrgBofqi(%CuP`Xs@HCdO$o*9FLh-VN>db*p_}av13M z_Sh1f>&MWBv`c;dMwq@lC_%wLND0Z$c7x4N3Db{>ETTVW(YkJ6|T#*-V31zbe z!CeVj0aDoUlzYruPh(XV#f3pxI?k_E{%;OrXs;Z0_!%Ec<#{5*Z_pLHC_%;RBk7oe zp*kF5kIZ65cY-7^)R&HG*Kd4f*E36eLVrQ!XHTh57c6;&r*vDLkSck7ks@xNuwEF} zpXr(k8d09HzZ1&>f}A}Nr@)L9G71CZFt2x~x0@R&rKY@!;>h}nM!^a%jqgFCCq{1d z$X>Bbo^jAm8{QpLE$&*?v4SB)9j#wp4gaHVc@Ul29mbv}Y7vM56L{3Gi ze*58}E?_x%&F^BIj3P#iz7HdnuU*h%+4y24#--zJ0S9o@ggUNkHJ&7mnYk2r6%1x3 zymP%r=$juLTqmj2(hV@uK zT)JSD8nJFuDRSKw?amZz3Th@y|I85e(s9dtO_7u(#Hsa?k@)KR&!{-45}kuVV#KY;nS}} zR~H9NU$QlsElW7W9j081QE+qt5#cKQfrFkJnD0%l$6b-z)gOIt&>(@3ksK=u^A4j8YoRlihKL-BeUsG- z6Es$2BQYQFb}ID?7n#?9!l3)f;(5jiOK<_w5A1(1jsM$E{||QdKbXb{Chq-)uhvUg z5FiR47CtZ+0vQF!D8`KhL>Ob9__uDXc{MlnMSfHT%@5@JNd{$=kvLyuM}<6Er8q}{ zW*x42qZlmMgRb;l!dlqlcy3UiO=q0jDN`8WZ?J<+LmaFq;2$SEu8m&hkw@g|emFjv znRYYz`TTN$B?u;ut)WPt7hXX=i^0}mzTF1Naql+QO%K+L&3>`T19Yf{##22=eY4jm zXKB=-LB4tmGwJ!YV#|z{^PH#3;yL=OY5@wqm12N-yr~Fv^Zr7y-5FFiW5(fg>-oAP2)s@;8Z(NMscNk=-O1^aeH zR2Hjp4sT

-u}>YwHVqB`fcH`&*8{9~W_;W(_nmVAEWai?E?dR7EGB+FV`DplET} z*f?b$LYVLuIfBPK`uVf>3v}Al>zJPq!_r6ix*IGU_+7~~k*4GhB`l*+z@{L98r|IR zex0;}d5|fFv#9R${W&zo3AXm#2~7vPqR`Qk_#~!E7=$iKTA$e}&{Az!oeZWzQYyvq zN$UZV>K_aE_fPynwWj|6P_6%@X-y3Cvb$ffRTXgHuXAQn;C44B2Jp0nog7f5GGjN# zgvMvbU2ld>YXZLv2|wINg?bX%FEgW0QXBv(tJC~6-=wZk7J~=X!$BPIOOS}*`5Qm1 zW0oimQa6}&KIhWV5dq=al7OGz8)9ypC5Wvq5>wd2x4Y%*x|6-oxD0sAb5RVXsUeI$ z^Mo_u@;TE4YVzC8*%a}@`dWZ6Odm2o`8=7YI$FLVr)*Jda_yt7?7Jg_?D&J`wTJOV zg{;xPrAdr~OV;`E0f7k7c~YP0ikt?KASMqE2QR6}+S@!u*6 z54i|jYf`~T^^UrrKevd{*`vbn*+t10v!j_l)cQE2NiktgG_hshDwMUxIWfghF7Cbg zLc{*n5&WF#%m7Pf(<}k($%j{)M0Fv{1nU=C0$&kf+MWc0T>s=g)a}x#$Y}XLA>v0a zSz|C5L6nk|3?6qzFNmKhEkGT%=X7`V@@T;7{(d6pddgFVv(Ya&+>sv8?NfVghWR6o zom+=7&|K^9DqL}d`Hyk@*989e0uRph&-u|A2IvzAIQKup2r6VWu>TBek1&`(|1-h= zFLVs#f9~edB13>7{xjDYSu|zf%hYrbfbo?WUx@qx4ocuO6*2_y7cnR#pek9-*}gZg}fW2vB1hV0Nw1GS`Ck$<~=@ z3bXZht^W3WzQ4c67g^uO;Mlr_tXeS(w>u7HAYcxjvw4G88ITWR9anb=5@?P)U0+u@ zv-M&zII*<$Hehr0yyDM9rQa~7QoD~eb|8*TPO<*6x~}@8*K!mEATX<1HIUq9S9(}i zKao+sfdO$;ecDxdlVfWaf!B9%|4p?yLv9x>7J_kkBD0eIom9YF6Q@0NXVarw)G6sN ziR^+?x*%Vxp`+57J9W;c-W{z&W_{^MNI5RN&PjKrPa9q;a}{p-NL&lCCG03X%1m7o zh6uf$XF-=*@^5QA01e}*eG7$CK%!ieOPa-IRI$}Ow}4W(B6fE1lv#A9l$GYL?tp5m zSy>n_<~A52vc+G2F!2-F)s9S8LZ=QT_YW#8boM$+onzagK`Un`k-dRfTjyP*+D^Jl zvb^oqdpqd{nyum@EwkmvG+u+|WB9rL;RvH;==un&oY{CyH6&NLSbO5@=M>+ohlqO1 z29uCaSc@K}FyxeLki(I`zdlVxp2Pj-?QbzpR>X8k+C|-;r~+r$Qj99$9@`t9P3mFV z0^0}%Xi`RG0FNm@X-!$SX?FA;``6{l@lbQNK1VT7;2dbR3Pm!QtlE zl1DbrNs`jM-2=;lKv)ktIu*vDQiUdW^iRa@Td9GxofZ2b!v@`K96`IL;3R1R$S+Mu zvD`=;VU6Oi+5$rlU5mdeTwfGO-XF6UZDDC*X4I(B?F<4%~&jCb|BCqYOM3ZsUYnf zh5_ChaE{PZL>x0iFchKxBs&e0kQhk%VaUDBTLx5M+GBi(i7!+6RJO>P@^V`+fKd8y zC0Zy@epCODBU*S0D)mE_d?=mwQ%%fqR=0isw>Rg%?*6|6$Uj68o#&B>h+pfD_!snD z?SG_a&6=&akUc=b|9Rfr_KE=x4nFnI5Jw&c0Ezey(?kM%?Cu51SNNd*4~K!`CnyYX zLKOxc0OO;2&?I1Co{*8Y9!wUDobU}rOIR3MQIrTec*GEr0_GPoc!guCZUSv$8Yinz zt*cFxO_QB1!f$I^dOblA4+NpDir4blMQt0KiVGWqt@0{+gDUBwzs^@#X&DF=pB4gJ*L`u>FAOME+ld6Hc^tg=&aWZ5fX$>a9j6yF#1H99z+Rrgoc;cZ{lV>; z?kmr$H^&FqM?uDC#OI2L{TxsIBM;I?_GWSu;6{<-qr&HzqxHT0=O>cqs{+yo=0~&7 z2VuykM&92$MeNRak-v;jPy4q}tsPf+TK56MgytZ`8$OroT*O>wc>=Q%uvyNm%ty1r z05^}+V~*`fc?Sz1uNie_mP}Kn2WQwu{cgMr16czsdnRSgi~q=6aA~&)KFRfx?s4&! zGfL9-Nqv##@9|&sdQnDN4YV{Uz^%N@I43VuHz_iT<^45z zhcpAw%Y&^ZeoWi5Ds06gi>Qk(Yzgiq6mOI22{MF(`rK^viJk4-@~+AK_4n0J=tkyb zsC!F2Q5iDCg$K%e2R#wq!ZX{5RDzRg3#aCn^pDL&K^r)gM{d4+%0*VL3JiB9Kx@Yg zLyVQSV4NlWN2hm_rniGV7Ohx-B^B1?7vRLR8AC6EfN!#qiN1;w>jc)Ot_^uP_|ywE zqN|NU*9{aLyuz!4t=Ro6equ(GQN>7Ydv!Y?_t)aZ`c92O2Nct*2{#(fM*wvVJqrAV zJ>sKQ!rg<12yfoN{j*=9Mm50}pxnwA!RX@~KVw-NQK30K0g{_dsAAEQkCApJGk#-) z4NS)P>bTh@ic0%ai4`^xH-qd}Aw+wlX2)kBE>EpYoVcvedSZmW)FzXeLb?b=l9Z7w z#D;b+ZVXwYBuTDV+jl;dyMBz~O0XV>KPYzM--3hUFWXetQ*>)CY@-~yly??$n8{5wyI9c+6dTUe^ zJ~O6S*yAC+`RwYlgugoU0x%hYYge0hXicV84^7mV>l&*;dBW5x40A}x@J-I{UDO$p zbty!>kW7Qt?OmM2odfO7a&zuY%IFU}_l={FzR%g*fXa1UL zp9;~!XhrrHyJ$FB`b3&13lhaI>8~FYAPp|>>^cluXk9_XRg6X{dnb$RJn$hIOnF4k zbJ@1x8A=2DS@mI*0BH<`f9Cn#4}tNo^`QTX&{O9Df`@36@6*1Xu*xp1iXSPPE3pL zB0<;@4#pbhQ{yOz26p0vBG)0#v8>7N4Go;bBk1XsF5D<_0#s}#(EBxtBkxdp%lVL# zT~sIA6qnQ2cx`7sH6_<7Wln}=5OriifYL^%YRfgZ7@Orzf(#Q*Ry$Qxa|<-hx9TqP z$;~%z$?n4?>{?bDo|NuX9Wn>B$J!z86&(_n{K+-@aTPx322~Cvqe~8o2c0fe(j2o@ zG+`|=W-(7?fb|%*X#BR`maC&$>8Cj`S?9UcjJ^Z&zVNbCvvjUS{*2cLrc(jX-$Rz* z$>BMpzB8YaORLi8SrjC|(ox@U3P$2k^s)`t1%@s*%*l2I-e|F9cor>gN}>4bJATPX z*AFy=`yw=3MBKiC0%f-`F7zsD#d;bC8PDb+_DV16fX}4S91QlH_%Wj{xuT&H$Yguq z5|8XB(^ZxaaMXRyC|O|asnn;&=Q&gNa_>zkP0i0gX!z1a%szY|Fsu+N9EL#kh4_{6x!`iNUSo>*6xOORJZj7QI>u#Zc< z^7A#)V}VgZoGJHpe^)^Gh{(2-Rls8r_z5=}4UEx*2?g;E?R-xUWs+CmT5jt!h{9*F z*+0~*ZO8QG28;SZ8K)<$Fw4%SrovS|Z~D6l0v)G=Z*wsPBd2+T5o(J(E#6HtQ7?r9 zFufXj!lh9BE0-Wbhje(qZ`}0EyWf27)p$k}ZGzHua#Zx@Ug|8cSnh_t9zPm)!f255 z94y$PVyd~Sfq47M-*|<&ii5^N>B_V1=V7)ZPCy&$_cJmPE7zOr zuhq+;r{IZ_E^j(+qedHaU4DD;+X#bAQh`FF4l3Snzb6l}YRh=gF2ty8E@qUJFf`_~pVrGuxOBSSWm#x8m(J?E-;6653GN0WEAy{j`#GkkN0@a*?O zlXuAu*1#dlXMa4R1IqCE=Gnsb#Kx1mnMTohFDdnj)AEQX`DHko@{S|g!N>*2kkP%6E40EizezpJTmGn0)* zv2zcKx2;vK2a300RcgB(GMISxjFNu6ht1NA7RZ!ks?V`#i#&KJ^l5S6OlpHo61VjP zI)k-F+g4O4s4(oE7a%HS+oaycO$*b>ncx%$lBycN=~CrD4GK|%Qs#@(P!Ouknt!uy z?9r6V$yLy1svl|X1z2sj#M{+7JogL)c;`_Mqv`Pri&XN$LhXKM1aT|3bj>_|!x)*XTVKFR@Ka0e-PbJ)K$gy}*A+J>MXnGi0C zg#o4x%wJXMYv>8WJP3C@ToqRR#t=0z%&9Ezi7l6^w?i2B2_(^{8Zony~WnSU^ zU1P0hI0ikJoLYit6jW)YdHAnEHZWEH67oS6H=)V8f<{!)CNNdF{8e*BI`qgk&P zbFE18RxuTa@p$=;CV#Pen&@W}-cIqk6G|%2ngWb>G~yRl)!>{&pr4T3J;g;{b7_=c z1nevAJYaIGSe6GL4QWu9-iGIW1j54w~-ZP+pDe1qEF86Zt-XG^5(kAkzK!d*?{0!#laUQ#`t zsgm-%+&2pCg~(FUqyyd*RC6;Y=3vCRqEe}fI9Xi@t|+ug3Xb-&U_N%01?_qnOA0=7 zveMjP@{%_?ymnz`Uv}Lu*Td`7Lky)SYcPgQVC1s8?_4Imw1t zY#L24fUU@+^I)~NFzJ<8o01>@JL%7>*^%En#Wmzj+ZiA-(#s~(yT&}MGrbr-v)p#F zzJ$eNWM)gOs6DQ`TC26*Bh;(Q%HHWXser{c^H%veiWhc+><-x}mAzFurmSQ(*)EXV z;al_Kf}_pyrWV=e8w@S00qd|6r#?Rhk*W#BAPb&s42-`fesrtp)aKf!@|AKh8rtG>rR;{7v*N8zA4@ z2TVa8b{g?Vi9L46Gc)bL{>fUXeXJH=e<5#nz7LJqOQTx4j8vNa{9%uPGDtOSmgYXy z2iWW)0=|N_zMheo9io@R9P|s+5T|qVL>d>h1!A6U>b(yAYv3s`h98kY2htH2@B4*b zZMOLBns2%R2+mZZdLvJSTF_1H0N|OsVslE`%Z#?Z&2waIUtgJ&#d;QX{8?U+;7Ns; zP%xUX@nKb~>^WPk#IoPgyqtLDyosyd!m2nBl4>_cDhBGOhauuUjYJ4c_%SHBA#0Gt zn4!!>?mKIXRJ>&q5`z!c>Sk!Y6A#k{46nNq^_HBO52M=kN12@>Ecn!{0U&TTX7XAC zlEiARSZ#l#-@7wl!xPasv%75C9fF?rK64MOAY4F!(SPI^#P^GY?&08j0i??u4Zv3N zHM{9ZtgoFhAj>MmS3QQliHDer3f9~k z%vj3!Q@2v_MMe3P;$}mmFTi~!R%QMX&$wcPOPFUVnTMc+j(~3Hjp?H8upZX1Jy5C5 zx}s`mu5nF@zDcA}0$RP*yd}8Mt#*;C&&;O*449Yw?Cr1|l{96sU3t|fC6LSYPO9Bs zLox%9#eHVGy;CSnHYmAxhtwOf#~G~xpGKfswTK2;Mk4hgbdDN327D>6S>$`Xcc+?R z2+bnhm_p)@Cj&6}c1g8C1hNmGj6gX$6Vi5S@a&(Pjh@{NpSOChujEIrYYO23H4A#` zv8?X|W7l!<+81;ja+c|@O3LLk>pwk43xD5*Y4X5%lbH9^_x3b)?=@)u< zmuQ@fs?L$V>=*1f2IzN-GmBFTKZ}Dh461;1sKhF+h|4{NS2=>#`(6uqA4nTU9iA=F zzD>hxDBcW&1aPX9IB|?_rEGrL#RMzX&4gV zivuzVKplC*CR%SzTG&!XIP13{&Msy+=m{yfc5PlUnv%<$O zHyl^_&ua9ZG#K0H%@QzkGrcJ09ZHg{2mjeg2^>MZ`7VL#hEAC zZs}ll-HwVE*B-2i!GYM@!K>oq6lpL#gORjve;v!TCf)H!8SnPEs zoP>OuRmXT4^i58aUkuc3rH*}^m^KOON}eJ~53s<2bfft%SWn6sYLH}54x3s~Pa|Pz z_sJ?%hF`Epp=D%UFA)$HkFs-&D09d=)lO>WdGGK9#r~&^1kGi1ii?)Adr_$H->QjP zfRajW5)_}v{=;0G-ykjd$#|}VrygsIxKv8e5k{q5IZ?)GNrRDAO%5w^%{SL0g5))d zpfvvYI_ZXLntD@1FnjhEVxtP$kg-n7pd~da@rvW)DWwK&vBM_kpc=LcnU7+g%dvh# z=cuagNEzAaX zCf1FWS(4lAto%%SfC~>^`IDqQZfcmB12*e8zM;CR@#@_zj{J9XdWmqT@`;>^X|jyP z`a8g1&3F#UlD%bP_@FCeX{Ch-YG;bZ47WASojP9YNA=e zCgjEDY`ZJM>WiO0D}i-Qe1qT-mry<%hK4?&Ov{&h$?H=Rpb0hQG~+uZLs(4j9xy#r zQn5Ka#He^RbI&3lHj~J8T^JMF@o{SXSV$M)&namgTqu2sIL+vst7lS1;MEdMV^mrNX8MnH!G9 zI%Yd}#tD(V%)U#EPgo@?TJmTHM-R?|UKvr96TM)yhzvxHBWa>J2o@6ZmNc__qwKn>*?IPma9IOK?O)BFJ;D~eB7bn{Tvoro z>oZ;RDL5?6P2+Za+kz)^bwn=XfJ#`UbuldagD+`aGaXR$2nv<53Wuk8dIg!#_9J{= z2TbY$ojZKqQh>cP`#**I{|3PS8#|$(p}+lGnIHHCn(#jnQzrow8}vU#dXFT~Kv2Yg z>hQXMDvF|6$#lj`)%~Vv^^EZsz#~V*vJ_Sj!wpZV3LL2>kaKc{lWb?tXR? zbRhj3fqjdFu*<*B2h1RV*8|j;K%#hZ1fYWs7g6(FJ9$6ozjI`|WdrQKD2tsDfy7zN z#K7rxax@^p1{W02UV$5}*+-cr9O^$`!HENw0ru}_M*q^4=6o?FU!ed4xXDR?Q@V6e zKp9;wWT5gec{~6ao4B4u|A|;*g3q&V2CYFDL4GF~*&aHmWZ*oS(0-Sm*pKRWX0oIF zpr;YZLEoD2ROi@rEFnGUm~{tjzqhdNuNzWqfs|U)7JH)4-a1W^I*v`iPrz2!r%A-J z;@|1q;Lw2Yr_^lPOb;$DC3c9`=FP!HbB}jB+w%cxbBlo2&`|x@D_+hY&Ki5vY*i)3 zY2#9QYc=wg5rt)Go-k3`YcyQ<{VddS!W;f}mTz$13$*Pd@#Snl^(o4V!xl0bvhj-w z?*tg~xNFQ7GOf*>8kx%B-aZo|k*Z18Re3DSM-t!WD_v8i66=(NRj|8^*z~ACbd`Qj zTcmUOHW~utx0EtU7O~c}rst?*vV#(di_}c$xHY?mhtmfx&lTkt=H`*)^JVrQzeG7; z&a{tsdDdtqS|#9y(sHV{TS(c5FZ9Qjwv`u@?h`2j{VLuUoRmG8_Q~${!KB$y(tt6|F)S4}s=nO_%H` zk+dWC3~4q{&RlDcs%DaKfkN-wdG~39apKD_L{_=?K5%bTI7eCr(5LjmB&ll zv|0iZFnO`@IVR{z0$WSvwuMjyy}PnaSL^l($I+amY=I2bUK{h(51^soW)$`y9gzhf z?`VY3H+j)|R2=<2GG(s|LS_lMj7p0*Un6VT}nwZ%( zfVi}>mZ2*E?lLnuJ`+hzUWpkQOBiz2(6Uo;e$!Z+FFrcf>V^%gGQ|XH4K1tI_r+h^ zJL2C^8lh^n!s#t7K={=tC{CIb$;1^brJU18AqNx6bzSL#I!e71Y;;wFn%Vp`qg4P{ zm5nEmAh_1tIPMf8dt7pIT3Ok?>EeUI@kh%VIj2NprIBm`=CL{lmVb!GK_vEASrd;j z)sB~UYttvJ#mkLVsvY1)y?ktbejf1M(Q8DIJ#tkJobNF42zD`X57;={@bXXTJxw15 zgKJGaDVR>Ms=qhRXj1<&;vJiUq<;q>?;RdNs*iPGzsWUwvvF~UaV2yuV2;^8gL zf4nr2Bt_a;tqArC9DUKzM25lG=^~P9{`0rF{@VT%rFk|T>xRlU0;Og)5vxr3H@Uc} znS^nFi3^!{5=~Ktvifi3*CY+-PYX?6Oh(Hz)&n0SBsy*>jSy%qcPly1LhLI<5QJv|@c{La6h>-N*|#>MDv^p_-G z`-I_OXU#S%b`4p1Z~3wR0q9q%ft7@w3Y5w$u}k$>#jojS=-HH@;zBVcC=p69ke|DN^*%7>+aqy(E>IzXdsXnzMCysu==KR$6u>Q>3ukE zn?7HAG}znJgNhFJdbiy{U?_kojC9%eW2Q_01H|ENN&|_eXN8L)7gZa}o1U zhg5Qy3FoZ7=tN%ltd8imBAKcvJz-lF6ggR__xb@%0k>3|F#Qj@e#g_n)MLjn+u=Y| zb86e>VYmXqb~-8}r^XC*@IVikfU!*he4(%pCc?nPBrB3sBG4c}8$62XUnKB$JaBJ3 zaHuS}70!s<`Z>s`Qr@g|7wv8mKaPw_IsyhR8<)%%P2WVJO{c)~(`QIE1lPCXq4MA{ zk>7;o0kv~@J9;3K$fFsu4_csH5=2KYX4kgje-YcC+v#qeJhB**m}k^Lm(+~&$BcTZ z;nF;yJi=@{a9fA~>@OrwdA!0OS*UPsFn;H#lx+w0x@SiAT|$qFm-=I{&m?%n}OgjK2VSxD>?rLyoS%w z*lSR>F%X4?Vm56`t6f0SU4-iuCpOP#U9Q?dYCcx#v9Edr0Fe6-yDAz<sc@VR>DB}Qt8oE_G<*0PkC>gE z@Taiwbxu?R&NT0^#>5Fohl|(7HhI|Wr60kBV}>a z!m@+;@|231zyGq&{x^a6znRhhNsK&~$npMXSNT?rc?t4w!a+W;5HI!BjjxUhG~~k| z2Igvj!2_q(IEk7@2~8mWlb+}Wuz12F{*zn7`Xvl2z($7wV*l6g%X^Xp@LA$bq)3$B1lIcuRziu(#Il3BS zkv_kDspf6`DkEg&w;bu)XyEwjxPP>x*y;kjz#3vd{jhAQHo{@WF?SRR<-)cbb&v^N zOE9)>`KBT!C6-3~)_s#{ndPrlqah9Oh9527LW3t|8~Q#Kw}ybyv5x>`F=5*}yd*%v zYgI^)o_7)s9by}5=0y=JQIGPL`|6F5WBI)hnNNBq`a^h&A5CeazT+4VhG%tWC(m;5N)os*0^_dt*RT6CRev<*`}~7v14RcQs8sA6 zHi%|Q2?&K9b5^(+`~A14>35U9cZ-6EKDjOLBfLtfot?uP>QD(VYZlA)Pe;QjTZ!2D z=doyv%{wIBuvka*JMPPF<+{>6mK2wLGb%$Z38j4WO6gtjzE{JiTSCyNCjh1g=d^<* zdp73^r`x)w>;-A(uBWJzC)OwcsmM45vOT0#G|kTBOg9*--2CHRR&8SRCBh5`m4z4E z#o8t{;5}!U5}^$pUZq2_U1L}OT3_6};^EjCI4R#b@7 zy_^9tcEndR2{X;M^{g?k)lK{(wsy>|MIlAIB(5Hnik|7MH0BD!JV7YH&n>ISubW@3 zgSe^LG)d(Tq-(8|BVnTDpq={Ek2kyMF5H5vLCJMV)xu2Fw{%ILs_zo@^#?BmPJrxo zxmW%5GKaxa_t6BXpdZSqkv7#v#MQ?8zBukm@BbS0f6eZH8}&bk8-3NznA9&3)SVa> zxc^amDHjKe13~%c;dOqQ_V8Dkg#JrMRTmhLi_8Y>R{JvVZbc3h;9CkEB0zmh1y>F2 zlL2=bSy;$2C`El%+qOQqTD!K0D1@R%8MFx8DudgVeFW$CPdC=~3{Xb?=k33O7WV^C z`W|@&OsCNT`o8y9j(Ag@8KKAyp?Sw5^`Kwg0% zkT&+REmY0ekSp}JV;@TgOsF6ora`g7EW;_h9(LdCQN=;WC7qRL;txLT_S~=LdXsBM zIqYk7c;JrG7al?8sRO}PtRb@avJ@hax|B9sLuu*0!v<5`)n0F53Bb4Egs|qc!2Nwq z8PmlgRq5!GceGqGvp2tdu0Goxb--*)UUUAAIg#AOB!#0&5;Ld^cA%qS1 zi6?USgO53L0T!ZjcS6&%-5TAdhb1%0EZb^|ZDGDL!Asz;ETe~s<0^XDd5Xsc*|EA3RmzV(AAC;zO0haAczH1C07JSB)x^)IY9wvi^t$g}cdP~v@G28n)s)s^ zG#+R0LyxkMZpr1p+p(j{FAW)ORrMd{l8c-6!e|WxIm8yJv;gTEuEYChf4R^lly^_k zc=+Q*pe2fUlA0>yg-S5;sTe=kP#j})Mm6C{qhyTkjfkShJi?6jMQ1`$29&XKgc0`e zNZ`TVbS-367_BHCVoR3VLyTCYm{;}Z_1I`;A}>Y9_@^vqC~+R57NtaSV>?QX#Asvn zc10%Bck%fscD~qjc7$f(O@6GJPI@9-4hO-Z)&rO*iZ6Tuz7B&3XCO~0#lQd(|r7dQ8||%XmlESj56r! zE|g|`VcD>sH%HfJf{_u<8xAF9NwTZ8%3965c~eSP_kh_%Fhk3o4Uq)dKl~k1>CKYq zfugK}?X!PTeO9yF7)hkr^-45E!p6;Xl%AUrh@)pv14i94PYV`6{2cxE(@!)pD6P2>aa_ptJ0GGJ$2n zx&awzk5p27$N5Ve5VFX?HLQG}GRYY#Ir@0(a59ZJvM^c?;Ig|B#?=mu7&56XM_}F` z2QBeTS4f^BDRDnRM&VGQR|VwyobHmwY>RSxMZm8Q3?pKnCWth+w;xl`%?+Z=A`uz3 z%XwYg$^7|F$^Kjc>W%kgC!XOpXaQF?s;UrjC{9mqz7?ze?7WtOKdVT8FCv4zGZ8`J z9rEBncwzl__gSdA zDgZ(N0}|D58pqgvde1gc^jk1N3b2<-j5w;W<@n`Vj3g zQtF;0=koz542zNCX<5HX$ER5OkNQP*daGLNkMW)j}zL~fPip##1V}O9V zQiv#9*%szRFcKdPfuVAb2y?bF(?wr2QUlE)Q%yd0oM=O?F-ly>{RkItPYEo)iX%u4 z?KC_G!$$*Wga|bJ{lU zy0M{SXiDYS(TnmZuP(Qc<-K!!9w3=;d_Z<=Z?Z6PlL^?`$CSu4hmzsF=|)zzHiqF& zBJP!j8((;2Mv~z(B8N+TX)6{9Z5KQ)JGOE>QW^xW)-1Pnf61f@j_G|^ic72*+7@he z7jgcI>TZ5d9%cR>(U_;>+-4H#XjI=TlisNE$=HJLD8CP16N{3aF|U&F6JW-~AjP2g zmtvi`)~1+mLvmw@)hUYQ5eeJcwu@^Z$Gr6{<}`cq7xZINaDX7LYOO4yJi*TgNohW~ zl*v&5i4PqeS~QLyT9#3Vi!Ap1iOV80=RjCJ-F@8c-f4Q5(2wZZqTL_v>@CiL)Cir& zbKNMYv#v9hOda&lqI4VjF@S#hsG8HoUk&Mcz@G@&H>QF*NXf-~Lya0?4aMY0Eq^^) zv5x0!BMU8M%PF#ix{`N7hk#JrH6iv7mYO2cslR{d4)Wk^iYfNALE>74mdz3(A@D&n zM<>&atHL(-X4IpG0`KgQYB=lB7CYG0f6ZK@&0bVPB2jZQ;nXir*8*T9dQw7deR{4R z{(k*>Q`4y6f9R}Wv!#PjcbO6i=&F>Ar}35A!2BYMd?s0_3TBq4e;SA8@lutWCufg1 zWRz=irgB!xg&ZXnmIQ60MCUxL9Tg$TvP1l>mc*H$#3mEh6EiU7n~O|kH>rf9288{})~VIn1J-+FxlS_#PlCP=jnzS0H23=}r!3yvsI&gAcLZhbHN zDHpMvzD&DUp2(1tC`g|)x(qwY0Noz=qDdJx-sfJ1u&4oylX)%EUI}Z651KLP{?1kx z=Ij>}%(r-;_^Lx$7uBISLVpSrF#-<4J^>~6=`$cIoyJbVE`DH=pCc_ACm&rVOflV) z#8UnwTTt9`O0cRLWmPw%Cf(h1>SEy2dx-3#M%^%!&8OQZw%R70eA8!#QmEqZjXMgN zohbv#pX>Mu7We*vTyx^=RVmV-$z~ObPqHl3pPjpnf^`Y*pBOOY8(jjzCqIi?UQ3IL zM^D_DK(<<%e+hy_*hc)`-?_a#-x>D62hk%I)Z0v{(SVo8_^@gY(aVSE>+y8w#LF*Q&!|WbQ ztQZB9_YV0_N$#HwlH~vFkpK7I>4np7LGzh47KZspJ@7xiq@ytxCdg-z;|EA^ogD(i zu<*HqIxcb$0f$rnBOpHVv91nAq(}=xxg|!27PE)ePXkG%Zs(dwRl=R$Yq$P2vDq zKz|u_haIQ$i|mfp^hTaw<_WzT?`^S8LWL=8c%Yz#?`4_U_X9ak`(M;a6pT7ex|{ig znsdkSrM5|>nfB?jzWh+sFFMrEHSLo9Jl*owe?Y1k(A6Z~DqOLlL=*L3v0g+vD|pA- zv**4*qIhicXu%FkF4p=kRbq!F%GF){_7bCBMqZisxae}jty{>ByZfK)VAy{=bSy?I z6E&6*XJKMRw(9Jj`0MY&1fYCnr0;#3ex@DI;6xq#Io}`l&b)u4Ze7)vx9t>hIGYj2 z#e$`9WnRabK$qz2$K&>l)S>`ukf%A5RSDSyQaV6{3Nq0{moOP*13-L9Pv_TPNB^p` z*NpF1=EMGAGo7v3g^IQQEm$hZA3OSlfPXG4mw-`2s4bVw3K_@B`x6E0*>M9~Y2cHD z2#f^G1YjoBUlv-5nY0lY2ncxmc%BAmZPFSg6+<$E2i$;Hxntxquz+e$o+b)>>=vhi zB2)B?NDCnqH&4=X7O)}90OCXSrerIfMU~YRve-~AJp@`E0yn$NB)ch3C(HsyeK)%)oARa4@h(ZB6nVR~-c+)Fw5bPOX#MUnZ z@K+@XhyNyQaEsPBz(?plVkfrK*hm6>jlID(H{607;T~vjFd`z_~je8 z2ND)hb|uB*p(K=f#+Dj6F{mB^4Bg1%BOUT0mi59l>fYD@$mv3g3@-d)fRrMWik-Fw_oiTj!b(9H_``c3B^(1!PPROY`3UqhImA7ubK7 z-ala+@|REC>t`Ls5&?7*K!O3vBPJ&TIyjrDxEQ&(I{zy|P^|vyjN^{<;X}91J<))d zy-(Vr@|cLe-zuF^Bz4}FU^Wb8Iwl#3(hPY>t9KYR!>z{sEHs$e(v=G-JmG9<_-!D1 z9hm|#)Gf`(@RQ*7OWIfSG^>*z#k|ZalyRG#jg_3CM&uc*o9kD;?vIVHo4~)Pe($ve zZ%}<-eSU>;2;B+&4AMvKN_))5&w$82`l=4$nyH-)1UJQ_T?QvXM!e3yf%U+ zNVrylCP=ylK72+gbCEesOJZ?*9gD;W7vijNGmQQzq$x##3fRR$KBt`mDa-0;x(jJe!=o!= zswA^KR|)ahs-zRRAWr3+BWY%I(R{vXhWflIRtAjNz7`B}KHn5duwi75LKrhxPh_@f z+BgwmL6hRrVx*R=4Ui_)AD5?;zFbIz4n2^nvI+xMHO6bgZ<(_M!_`LDP{ZK<$nI}r z)7;G(`a+a;R|jWl%ED<@e!UJ$DR!f^mc^!CRqN58NI_UNzE;NS)N`{_3TtMs^hQ0; zmlx>0>7MrsN4CSs;Y97F`I5ef9vyC{SmIH+!Ush)HYha-1EAc{&;}QsMa;vtyGg)? z84pSd4Cz#5if1WAq8T?MB$$_(I@`TZMaktKUnRFAJ>{5+RFC*wiC2eui&imND=8DM zK|an;2TEB*ngMmJOjZzvV|}qI_~PZZK6(?m4G?%Nlm5#|o~Qh7&fUJ%E*Sh3sb5Rr za_}$MtAPYE8xXEK4?Eu}zoQJTdQ@7TMyo>$b)~zRiM*mCVR;;^FjFWQS3)UkS_88! zk)t^hS)mB+lE_3&YpL3k%#)ed=V>QjWCT@!KaPR1b4TJEOY~`!UzXr#6vdkdvO$5> z7WPUUbnxGBlHC!|4H}puEZdz4veBtjNVoD^Ctx;C1UQMn(iygjWG<18XTm;i8@&oJ z-=Dl37V2mjXg=BQAIrMmx6AA7XQ{8aan+av$#z>({URnS#+mkSnNF6Qm`o2XNSQEC zziQq?Zf$5NQ=_&PQmOitsNSfntO2hwMdSHP`MHAyp0Bln_M{}AJM0&Gw~=MWc$WUo zS*&!SHqhKtEnh~QM*Z9uZ?2#?C8{D{wdpodee26?DzVy(m9%o9?KrxDyD1iRnU_*T zIXZ#&9RyKY!Lga^pHLO2m&_ge+&MBJFr3D^$mLtx^_G9>~4_7V{h-}$~0cg zZqL+ZnRW-#;%2m7TB)U6UvB2I6D(jMQUhK&4rt4=_knYEghqze*1ZU$XLrG|QSz`y z`8Q=kG_VI@ZFX!^+yEwf!?A_;eB4{tb0|eZ&vP=Rum;!p5TFs;d8@B8&MVnc6hNdk z(kGKHllWY|C&|8R=P-MdEFARJS*SK-w!qOX5d<4vRb2>KE<6|3jD1b zjKpjo2mg-R4Q|f^xF}0M$X-;t-w+DKra|e9e&}zWIybc@f)cKw(fR9KyJZ+sOMgdEJ{_d zFx@HFkxMC)OhS!FhnOykR&Z!hXLEsdPQE6xD77fIDAxq~&0c_CupWV0;A^u?Xwktg z?jfMoZetRB<$P@GaS>MtkB!!*jwUw(b{lEBTf;*^T($? z+IOh`zU}|tT<=n-AOhc~3oh}AkNBz%5;6y;2KD{KfCdp(LE{5S|3@FKTJ7w!FoXKu zf=W!LDJpamNT?g4THWOo*1-Ql!?2Ve;$rd_l zX)jdZ$jz;Tqu`jF683e4^m@C^9{y#fhd&m0Bl&z93!?*Z6eLRA^@=D$l&Z&+JfjFm zKJ3uZeM+0Y23BV-^-4&gvPNg-kFv8I%l)DoBJI=WDN8fNK|-@^R^&b!SA z8|i#bNWREe&aLcI;X>(*b(wagdPO$QUnsWeBp( z#o4dybLF#7>Q2*N}{OSK3jdlS^n}ZYmyBrm#pt*pbI<0>6XG?a;t;|aLK9H{9 z#esR`B%{pMEu5HA-$S1QTn@)!irMtSUTRHAjajkzUkjs;-C!Azo=IKCxV$etr`u<^ zF1maDfUnSo*bK<=@lwXPD9qnXdF2amk_IdR_oP&1&_9znUqW&tNyy%sqD$d?LpE*; zlt#OkIH-ZcWZhlbMwpX4?CwE z{8nrQJR8fj4oN!NbWmG11Rl+oT0Fc@W!<}KByZ~#-Fw~v7M}9?DXb4Oc#Uw~6v$Db z^%V@vWfU^cxAW3cwZGwRSF~%IK6wm&wQ3O3v{(SS;hBP_T#l$8l5SNK@=HxSecJ1F z4*iC!xSNQ>gvYNiu}|h(+AXYOPwcJ4ch)b}*_|Qh7ZWYgr!!90g_U$t4>CG+hu#xK z8#5dAc+xB|IVAv0jpD09Qzfh9aNZMaL7@sM#8yb*Gg|UI?S9l6?l_&!psiD0D+O%* z!B*fCjSef97oc2Hna(op>;OK6PDQn?}K4BAG&9weV4+C6Mi;P|Ysn<<>q z#EtGLY07jKwncl#6y1&WpAX^`qlfz_gxa`s&+St!*VT|u@NUORY7+W4F^VtG?=1?@ z+dQNtG??u8>4qhHCLmZ8*LV z^4lkvJJU1;9k$hxCwd`#ED}Z~eD+8D2!%KT(t0nU9=RteULlbM9AJZD9eJP}>E@tE zL|fxG%JK&JwY(B95>~eGD`MOuciq4{V%p3Z3b+#&&Yb3q`Cq_ zBFHFK4)6Z)*H-r-zH|M9B=_`i@BgVL{_hF=->Lmif;?S-M&151IX+AbVl1H526k&f zd+4g6z0a5?a!kl1$y-_ePMx1GNh(@M5ze5uVz!zmxB6{L1WB8ezI-q@HtDjJNiGHn zITS)2MTm|DkwRQBFP#}YUO-70-WH(zsJ8(Sqx;?NpiiD;XD(d$+~fJ!^E%CUyzxQ% zxRGh04}4PkQdvIy3F7Sw__jgs1tdU?8{7kg!P_DDKoOub5Y*@Eb!|}X(%-x*QC$Ay zD|q~j58WMKp_!a&@gy8`Q6Qg+g1jGPo|5r?>@2wXe9Lt;WXDG>l7Q@Ig3d)z!AB+; z!Apxm*<&L7`Ay7WG$0J1JaCMs87-0Jff@%rKZ5rJ)?+z{Qt&qs#o%XBAaGL!fj6c$ zeB`e@Z;%Dv6LpUBo{h&=e?zbR2*HNCGcC5PwfPGic&x4!zx%euBn>d;|=vR?HiM>b>q3gQ> z1>=xcju-wi>76MK)2_);p5BB7X~zL`=PMql6WX?lLWQZ5j=!e}k^)Cx#0R{8*$*6v zv+Y6CbokAZlmsWIN%;Km@N(fhl{#IxI#KNK5pQKr`FfUwGW!Z$OyUDJNm&)l%NEhX z-=BKoT5NC(9U`@&ervJqR8R zD8yFJ*zeYpuKKl&rVI==BmN33+aypK?&PxkGsS?{fQ-FGs4JCbOipujCQ1Ey&WO+C=Ohr|CW+bMmo;*9Z@pOv zzBm*tFr551II?N}Y%gDt&)tpcbw85a{LmTd`vco^VJHhA^w5-QR{>c+;o2g#$a)Hi zuiT1Qpo%s-G5#bZATy51`klU&}v4K!ll0(#auI#Y1S3GO!g*W?m&@Fn@p0F;sk9o52#Rkc-bfN4K^Y+B^^(aBF+?bOUFGh z#g$CGK?ioJmH1=`t+2~M>y&^|Cj;neP>9XP4+^xHv&9kLW8RE48Rxiw0Ln?y_QEU89vWGBj0 z3l5pv%Vi6k(y1*&MDf$r=c&`x7qBCi#+zcNEr1H=`##N%8!)VN2RBxOu`~-_9sVS( z3JatiiY3x(gi0y0I95rbnZwI;!Hpy4iPZo$(=tst9Z#p?Ftjoxm>yykx~cdv8P|ol zE$e_S_J~t6>%0r(i}><{j1q%sY@$-FG)sznPp|a52VN~|D=2rdy)H2X4BhuK`rj5Y z{=i(?QYf8iR%B&8lf7~G zgPaOYqO58fKNvF$<}Xmrt$iXB;Dza>Gq`|=^pto8u*KU5NstTk=kweY_jU82U^BqU z!E66$dA?ySwu-ef>{bOdYWBc%1FH(%EU@S|A5v;k>ncL;s znxf8`-4QbTnKV69=Am8&)`HPt$+#}6ys|7bm0#gc2DGQu0&}jPW{TV(9ICYkbD*nW zcIdtp|A>e+{mE3c>@6)LUKq*PkUt{>JhN<@U>5hRPGl?SAlop+2l5kM^`aJ9gBvF(4}8;X;~H=EC#2AD94ih zqTkTqpTRfIW<2T?GE>RZ6V+fQBSuvDoj~#)Z?2Y9ttpKZlBvWO)9JF2ZR9cK9m;qJ zOCfvMjhZ~gczh=^GR%kLqCy9%&3KW>)d*OJik|E^9VO45XeMaItM-Cj2|$~phc@o0 z3*bhHB|IiaN|za15mlM^cU+>@5_g_3BU+r!;N^)jr|CAK4#td<8o~~(co^m6Adr`zj|Cb!AObz z)mJ|+Iqjq3kB?SH5Q4_bC-FAkM2EL6y$(1qP^c{39?-WO|^G9y~2*_j0o}7md|HYQ` zJI^eXFiNl1EF7qfY`;fFHyNw?N45C+PnQBai7+8=ChizF4tgfeub@5VqkHnpO?Vno zdzuaKy0(c}k`X;WT=mDd{ZUs8Ej`bm_}IZC`f2CY@4Ftj)BEvp*yj(|&H+sAiRrff zcxq_@F=-)kyggWlE5N{s=QHvH=&#p>6Hm1|A^DtGeEA*2zo8GEqjMCut?!4xZqu0k z300t7aoW-DLxk0wK=4Glj<_u1tQkg* zBN0?UB$Q=M>A=6n>n0Ldg~U}O&RyeE9s`!SgLmSYCV3u^oj7&bN*e(_;=b&idPwW1 zye%tUL#rkHIjhs0?u_ObEP>fj^+6XF7@d*XXNYlIK~R@ss1ly{Mnl(7tqRv07LNX8 z!#BjO4EZy4pGO1V5S`Vv8it(F&Q1K(kG!$Gkj(jnX@OJ&8zBtY(%QGlR#J0O%s^O` zp1uZF{jll6y99S^G334CT9c$;aO4-#``*a4Z_wYfsSpg2G&;;>aT&91@iNID8IPE& z%;0F&uOY^-{7v}+J-H+pDO|gVne3-L2_uYh@|RPn@Z0FiAGt*&m|c<$vol$2V3#fT zdX^V=&4D?DfrcULj0`;6FbFRVLLLZ%!wxhdAJrKVrB}j!!%S83PlIzsjv%I2YWA&P zn$fKQMm-bCJ$Xkq=fK~?X_3)ezho_Tv7)Zs*hRnud1y=PH3!(-)UUL4Eh--U16UOg zwHMCas__Q6VRJ-t&fd(wnepvn(sKsD(o|C@tpd6H;m-%vYGU3fTW$-pfE0Bn{Y4F8 z-)h~j*Z<)5UJb~0eTM}Eeon_lkeo;&dgEw(Gd#QSE{deY5fdf-r-9~4xGL~ zm!h|9+miCbs&roKcg3C9o5G~`l8M_hgAV-<*-#rXbd)Sjs9Mut?;L3ik)T8HWse4V zSr+u|nNftd9iUow)b~AcMz}QRaHrEEdKs)6^|CDAVgE^s{WmoEf74_5L)F6l7T*#nqa?J0|g-Gi# z2>)0#`KzRKeR6O*f`f>t=%_$rd+fNa!gb(FU;lGEJqbjD{aLuPh5NT~H%p9((W;Mw zDfb=X@8`E$#|dEq!6E+Xs8xvtrWfVkrUyCZBh0_e6Hd&_um5=#j@Mu9bDtwUf^N!~ z@Ii=Nn7ANIGCD-yruCvACSQ^%PiESs!e&cLXLIL{n4T86x6YpeG$SK5N16O?C-(Ff zl?Br=h5nv^TEeGq-AJ^782tV}$oyEuk*PHGZU!B5wSvdC*?O5LZeKkF1620InQuoDN9@9gNC= zo<8ZELf*fXq8*)EF`?zjEYrBR#n7U6t3gk!G&;V`M0Io;sh!G5I|GB%<4KR6g5{8! zszsSgwdwK>B=+^7rw{povCf~`=eyX?^#a%VVHBTVY!#$QTG~JH+ztob;C{~)`;m?A z+fj?HS0KjV8SHmTZZj0}Jyb^(Psu^W9zh@ozLTRav>RuIU@A`m5E9;pp%BW&RG;zLFW4blhltTa?$KQVn6w?li{IN3ce<3_$4}9zI;Noeu>z%Xv zY(dpbzfhd4x4?gr!2kc>{h4R_Z;1ZSO!3Q)n34TV0{{G+SA1W-fJzidF+o?oT;E&c zb&x{-XEsfmBf)_|`1~`o`6TGl!4dvxPG*fUfB)<+5d7bTaDB*$-fFBuM+Nqu+be&% z)4zS*E(ZrOiO}JL{LMJ=KzIxsP_5ft1cYGl|8#K`IQ7%y(*_H4gZG036t74Q2g*C- zglzrjCy4)_0oNXwo4)?1M_}GTX7i_YEF>r(1sV_ZIpDuNej@D=zqPjJB8`FlXPwNC za}KwEPHvXZq0v5BN8@)nAX^K}VD`R3$o}&il+?-T!T)W-Yhdz#!~gSX2>3Lv*=O(A zAR`SVd=R4x3^b@$ha00c0+Iy?_OEr%Gyx$vyv(Qd51^IL%;yl=pnhqn|FIT7;Y4XA zntg-Hlqk;8-XI1^n%r^ps4AA$JOpa!|X!e_mEk*U7OyS;I z%%MzDYGQL!3x|^W>E(D0e95+I| z9%X41vX1gibfj5;*ZSze#W|wUQXQn1_>|88v#vW=eV)#0mJtOhMbfCYyz!_Lc?IYZ z7UG-igo^?+ZrlaX%3`F+pIK5h53tx2UB7pzHJGz#;atQTcf{gluiYxg#ejn5-tPi| z+*tvD@tj~{m*s8KT5~ee+l%Oqz)q8W$W5fT=x+B%!RanzI8_!U&Ysa_-Y{ji!HHey zCFw43Y~kdP@GS-D?YjVKdJDx>%6i^U3U=Gfx3DKM=rn+WMG(XthqPF3=}v+S9k8D3 zE+82uhNMC-cAN1SFMAZe9n4hF>mW}$9inl18sx(D637xYi>GPS?0va-K5qm zeI`SlcZw>mTDCE{$RC%RF8|!Q(mN}S8C@?`1`n$HNDr~$QZ*m(VGGEC!5P!&!23BR zK!|zilqXQeJ&K*J)Hu2C<3k*QIJKTt>E^`3yC!InzTzkBt>yi)WqhmKk0k+v*rAW* z$e;5}+&VSpC7x|Zdn-&~RI3`nV@^athsyIx)b7@!n*pJA%f&I#tX>=<9+H>k<#b+$ z`iWGI4l(1NeayE|l^*JZ{PE}OG&g#SQiBsj0_^7CTrjJx6pWc!*V?1nG+8u z^xUV!NKQ6iWnr+A+~&rA*CbWQw}kUFZVdIBYRUnJ)Ti44UNu}EDEaO@ogIZRV+koG zKWP|xW9oE?6GnArU%=?TP6yG9qaCu9jk{95t1Z4V?#~Top1_~o&zgJfbL#;&WgVy& zex;EA8K-^#oL;H%i3H)pc`}~&OIpa}QAj6cMs!8|<(r$8!@a(?RAB9jB{+`|52xw2 zmdR6ys?b2Pg09+&&!fIL_WhGd_2-JYsdhg#oKX=;i9sS%pWJv(=8y>j(>$i1uWz;` zKl8Kd_GMS10fK}f14nG& zFyVVB(Z&E2%VQmsak#dGD9a5=7pB>OM%m4`k~uLxA!#o-z9He{N2tA_fn@Gr2qr7{ zGcO}Vp4hn2KteYHrrfYi1=zn(;UY~9nOj!I1a=qZfSjS+Yo;~aykF=*e)$Qt+Ojn> zBrhiuwD!NBS2B;_EuH0uXeOdflc*X%d6??zLx@O~o3?9;$h)P%c07$}T1KR@mXvS!kNGKy2GR4GU-<;NU+U>{r=7U$v_p zV#ZN7#OV%p&!6Y0`Sg8XrPvbL;eK#;oftgiQj(x}uiF*PYz2CUM&e7L9y>xPT!-_C zIh$*4Vep1cJIW`XC|{;!Eavz_yR&YN5cj2S#x57a>D4K zfG6%KYN#LGwhQDczl8(P=@en)Fv8pN5@DLmV2h&czFH||ku!|#xRgxG+Mb1WmbLAL z!rBu&MZ;+F(HkT?C-*{kU75)eHkIz;3LNxY^mM*m-jV|iUY|L>xbJr|E*a^@M=&n! zrv*w>&Z3Kdi4&{)Hd1+V01#?o%wVLF1WLy@5$eXvVrm^gMA6%4h3`OPBFXln%FqN_ z@lu7;A>cAOqaB3&EJdAEHB#@_#dP1Zf}ElBQ15q!+x^>JI1=V9A^sT6SO*9_wOV(g zbGOf6sZuDZD%V_UX!sU4_IJ#OoV^cEV&b;w@|IHsFW0sG+Rvtkoiftd+wf!((p06U>Nr-XhCg+M@O0O5#&J{3TQx=}81I26bi(Q^ zr`F{Z-D8-=CH^Tp5{QL{q^+LB$tT`g%Q^X>|`7qVq z#gH38pP^?$)U_AUp#-Xbz z;Hu{YFQ={xMZ(*gZ{iM)XR-t*ZcHS$G|2FUH0P(UuFM`l=Q!>2?7_$@^8u+mg!+S2 zxxCF)eAdX?@E{qKccV|WRGjbFBPRY{S<+j^3wq=^skKUKQW~rAQ4Ll~099>Ok6A4= z8}aN_*7FK9tKz%vaT~iLG$i<dr|h5`;fMH;I&{3C3RLlM zLsl-c(rMf|0@LB}r>xwFB7xR{8!fN!Qvl9iKF7+VO@H0lYKSA#2z4N_j4i1G*wQ$n zjbtbgFN{YamuaZ=hi$0W>Xe4bn<_o#f8IMjIkTUE2rCt3m0{XEoL!`{@|^+Ln6 zTN>8^kt|Q~PEx}XPnyFNoRvE)Xu12aXa+F#7LM|?y~di&P=VAfyFkCtW*Jsk>IhnQ zSfUM^Rg;Eh28L&f8|53e9wSRiySPJJn-YJUb9Qfv!PIpbZjBf#p3jVSZPZ45))m-oit84kPXwb7k$1u!m4f2}lL2n5pI_ zMdH1)(U(GTEwMk#I>6rrO+pMVglj|{R-!FSM771@LwfcUldPpJ9p(wqT%wBcq792G z_4FNTAa2L?9{kM`Jv+ht zPeJ;4!V6Ulk)l;WkR?U68**u!cyVKt-nULc6@rfN(8CALdWF19a~fFNm1sKuEa-Y! zm?+NyWYS3kMBtJyR*#cWTteNx@=6ETI^DpKEy^XaJ&{`_#X}QCOY6rMuAvA7)TY)z!^F^bh1o`|OHr$CsFQXxRc7s@t4>1XHNYb9nS7bUI5q?OAHT z?OZQx09L2`Q0)#r; zSm)7G;L`MX~P`J6_D$%bG`HFgOF|@h5))RoN{sbI%{PNwbhTO(o#pSyt`Rye?H|#8C zzRNqfTqZ+)d-6O{d4mSGrXs;j78P{$0AXksY0eEy^%y~|feJ9}Lu11enoa`j<_?N9 z6DW8AX?%~g2d{Z{=omFw?c3z-Kg%O&5#*9F1OsQnhEHGdpaeje(8kDN2)g2~Pl%Hx zB9~$=-dTqdKjaJ*$$6Hr@)0#}{}D!|FTt*+lDxdLp!lcvpWN2}dVl?|XrKEt&ZNHp z?FD|S^i`37cJaXRK{a+^^L^4>$ z`V~^-ct%26+xB_JJAXhZ7BkBzrQSlv)I6K~=8)7nr8Tmv5;h$=A&+ZDNy~zZOlFJy z0=e9JlWPmR-h5My?IX_>A3iv?DRjF)+ZFwB&L_40j9b`D@5_DTmyE@)0x-a9oq1lG zsnWaEw-(migxxF2tH$|PJ%P2GEV$g|>p&bB8`Enptyeyd>oxgLnC&*`=!fa<53}94 zqQ?>DM@Xx8(F_4Jaf9R#0h+$teFm6o;&{hI`-#Blqk;8|EIDv1NfNU>)`K&feICZf+XUJKE1_gVbr+roHVo*Vd(PwWX6mq< zAR-cmruyTU`x7lro;3;=w!iSrHKq-bosNiggiqqU9Q%*;`v{ zLjqfCRs=-6t|}2#Xgpma$rV|{Sl_ZgteZOGbtKf zda0A*k2{WebdE6s5aZFri~(ed4u|&$kt0wodWcsHcq6A?fNwo+~s-?HDEGVozl+ zTN5NthaRc6(zyviXts>?5r39YP-NOtrsW%@iCPcLtT;ME6D_L>Mj{|22=Q+0TZe_m zTY9W3R(Y}cnKssR&yUQHXCEGhvJWpIL6FbX8lJO}00$J>W=643lZ~5b-s+3Nm}5g> zDjAZR&$v~*E`#^^y(CmqWPH|jB1^k9izB16!WJdaoaZqbu>|UY&acP`IHp+M7<%}-WR?0A zLzC?T5G4Nw4MoPC%cRrc3a;l`5n0AWM69F@95rQ_-eXi z6Ek!XI4qEsB`+3znI?KI$7OVme90}pZ^u0?dFSV&IP$m~92+)AGWg@Shh%7BxD?79 z0A>-`zx4~MnuJzX0?DFcRxM}V<n(N$UK?hk>n|Ld#VjvPgB6U=2y5hLn;6%arM#GvlOzbJJw2CXlym@c`g<{1@6mN z**sEGf-5RqT|sO8eQGLZil!tPHRH0FYP%l1&t*u}#>^3N@(S9fJjLbcWXr#)i>$>$ zxvfr%SBuNP>9JL8a>wqc#eQvHC3;XzTB|PS`&8e%T>8b|Y@qR2)`@1Vfv7WZsOSVN zZ-U8o7!nEik%X%id8$oh%CcNKDnw1xnP%oIO+YD`?&FQDG#0SdmXKgpscm6~volmL znxViz!p)f=YgUD}`eaF+3Shr%YnBvT&WV)60yI9Y=<78ae>rI_Pm8b6j|)wwG?OJ) zXk7Ty@keNaE?etpHti4xTGyQ?Eii`C4Al$XcBXg#qEz3E6+TFxGvMwq-PJIeOo6+~Z zOaX-&c|uRQw__iq)Ak1A;-Lt6IZ+EROa?LvNu4=Y075(#i4_)b;Tl0a_AS7!8g2?~ z2htqbyU5n9A)SuWLw6+PFek$446F=EGJ+ID!=vqa-93Px$)89aw@rP#@w}vX!3~fs zR`#x}W`4OBT1q{-%=Nr+S3;+T+Hyl%5<&JuZ3A@)jZR8$mmZI^*2hKE0k-`BI%*!C z;FnBCOujr=z`=kH=j9ak87}bAEGEv;ht;Ml)C8U2+#ylb%ug#Y&A^;)9zN6Mhog_r z*b3RO|Eh(jsUHFRV~XS-(^!SIZIuU~mA7wWL1w7yloHKrt>(qf8W5a z0x_g6%!;q9=?5r#io{5&#oLg^#C#sqidZF1CfsHF1+hmtc$>$Rp#3%KUkTh^yKX~TBr z42eGnuZX@o&pOA(N8KPNjY*=h{eN}u2D_TB}g*h*2q^i&GkMZCi@$d#- z1!x$^CGk1Sqbt=!>hI^VB(DB=4 z0&#z=MQP1)EFc8bqD*pFzHE~wI~ZZ~#E8c}?(!g897drIjzGoz<`BlH^$)w9Up!5ivQoRGVnOusVYYMcN9%zGR-6SJY$%>(n-I=F1% z_(B>ekjlD4ZhU&rXj61J;%>j!W`NI-rT@tiONgj1SarsSG$5RMN7?l;k}>#G!jy}Rx~YfE)q=6n`s|z0Ul+NX%KrRT#BtP3A*!HetO*P+cj}T; z%YfmUoY)px-I|iQXLNkI#`h}a)Erk*{=8rW#DLtO)mN)GxW^0gs=9~wDrfJ82j`Z* z=ps4>PIwuikC&8|%hw)$sm9elH~jB65!MEGs6RkFYr=FF9$#%lW(xL7IqO9tsKY^kH`-zVYCidvD(y$UtfM9Xq z0N@j@wL|FM7bVs^cI`$*Rc=aZJk%5!rDS@c8$X{uH<14N@4xxl-^8#Jy|vtw9*J_r z+da~@xhAi0k4BP8T49cVtYziySnZKawXnT>Mi`TZi_SV_4_J$FlSUd+gS6OGW+r3y z>}ZXaFpHDMGqexX3xuG^tq9~_A>b7}G{9Ll9?J|o)+y0DB4a8cw*3p?yt@Ilm3}aB zKiN7WKM?H@@~d2Je(9amo8aahWi=3gc*gKmQtvPBLqyWWo^4-N`k-HWSy?NEBp^=q zH*?zH*PB5Fo<0#(c-0?Wr&F_P3@mB{Efpjy`LV`vrz3u@0^Gj{Uly5E%D%oxkO3e4 z()jm`{B?ihxp;}5g>PJiZ^Sj0jLv)i;1@pSw+KU{=hFcXIJ=$`!hD~p*RQL(26Q&9 z&^%M-lKa)OS4;s&0;4170g~4y2sa);H|X@-1fa zyIQ)0ou>H}$|+Rqj|Z*DOIG*4>*mAWjVda8q!82 z=Wnh@rCCnG6C4IT8wpc#{MF1InEGGI`a2iJ&2#;K^p2Za_1o4p#(~^w=zt1>DO%8@ zQ3fEHgQIQ3F;lzC24AH@hH0WJ<>`x56ze6o#>Y<}0qc^^kVI;(>9AS7tyUFox^P!J z5}j(yDy=7tsY`^J4!t-{cxu!fbkbZ(7Ip`b*zwY=N?V4>9n1jjHg8lr&aY0^>CFx= z0n-Zit?~r)o5{gHULN!0IY1jOj!K;s*&m*3o}IdEB44j#{okp!!+8`rS@)oy!o<7g zCz5iG@Ud^kC=C(7wRikldit1Jo&)e(-un>O!D{q{B4$h%7;L`LQr>X27lla79%3D>qnX$(pzW^`&{%ky106uP4wm^OxeVvVsr`a8qxCZ?S$wS_&4LS{b zWsGsiiq*$MHR!Ga_(luL1mmRq%?~0~31zhrWQfJWq&14(IH^C){K)CFJ#m!HY`mQ^ zGvm}@%za3YlS9|z9&yU1)g|0PUDLUVUVfOTO#TyxRbXYH*qGhGR+nC0L};58&;^5> zaBGtH6lm&2VbH3;8sj<8rs!VIG6GM814f*e`~40TB$L~w+qawF_CEl1@R z^PD26gzK$BiaQfTn-#x8XL+p=>=P+%%7+wgnMw6Iq^TKg;gNj`oolk|ha|;XMT#Z8 zvLwW&I?{V$t|?7&wrR9f`Q{4^!f(#ohd$b+lbR%?T2Co8A1Saq1y`1yx{edpq;ZfU z%#|WEWD;iF;I;tVGQ z$JR0Ip!?D7u=_FX6vw@TnOQ^H;8dV-{7k3~X@?+e!F@&=ql|Kgh2p}YdY2`}9bQ*5 zbk@^V)}IL}`T531yT3ssE~l(M<%~v7uI=M9aNd~*z~E$9S0T1H%v@(~Yj3|unv5{f zxg!D_3!<8VNn~cyc0OKr?9bX=mNpj8dqSqPwEVFP8@uV`F0R3-YQM^bSinH%AH)Sw zq%)>@P%+f+XWl-}qV=xnjJK#DS!G*+$m(hM9KZ@4^6oRh(UqR#|D$4Y=hbH7?L=l<&)O6+7c11WA+5H=dYb;X6fx6tB71CGRTi)PRc)9iP@OQQiEmD{v*eh#&jWRo zduF=@hXCyeNlacw@)qVYUv=R|JiexTAcG{FInlvdSlkSHPJa+96XMuTi6l^;;F=~K z;N^s^zeUo@=rH5NLG)AfPud}wt3YE=HuVu6JnA$2GY?jdk`DBX)d=k6< z%dGi()Vzb+@}+!IMAB1Hb=YuHYh@u|QiYBAumI8VQ!?bD7(Zt0ZzZEX0PJ{u48asg z3%Rlq!KK?M7Kv#;I_gz3=9_&=tD~MFx+f}!lehb!XPfwQBs!O@#p1Ll53<`^6#f5T zyuH3ZLKvY}NSIv>CD(W}9M*}Y#S|%%ok^irND&jCqaQ&V=c694!rAkaoqaNmbe}Hv z?*TKcq%8%F^&P==HdW%I9QoB0a-7L2!Y`U7^NmB+qls%47%UnrSubvBb%L1aTp4sX zO^pxBm#~|NMb}mB3Cdf~=Yd2&A#PIlMK=7(e0I6^ce7dQC$1HE?$hB0$T#cBTJ7*r zS$xZMYz@M;`jLMK{n-#i*Y7AoplQzUh6Q{U{i0$CIX+%6B`XcK(wmE&_Y9!Yi97aM zFpBrE&Z5qIb*Iprz$=<*`AAR735OegyaWk0H;Xr&ay`uJb#09kpuM(^Ox9|+ho1E} zNtZ$(T45vCU>z%&?+vDv10Va7j!`>8bjY|Eu0gczZ&7)@E7Pw<@E8&WX2F_TnZrFI zCi9Xqe0>54&C+AjCLY@r?< z^rwpCFj@jK({N0|NNw)GAM>IbJUJoktS?>I`4`f+Iz2umkNrL%w}lLbX2batqGVmT zW0=e^~O@~{^YCZ+PFU6^BNEYN@d5$O|$jKvVJP_k`Lu1>>rNawP|p-+pkaGl|w zkE(ar_&M(a3Q%p)HVTs=E|_FlUDVO>HeZmRq4NTWDRW8Fx};pR|1`%DzNP1jIrI^$ zvEtWTY}I&bQ9|g$6#;csS4Zc0j9m#`eWUA$;AbNu;=@f;d92b0HGTx*a|y zPq(9;rASZnt64b0`ThYm29|oJ)z{&OOCELOD^69hBBs?Df)h?4+So?OervIR?NC>} zs>(G0!DmOJjJ&$d2T z>H-4^5$2WUqta~Ou;Fg#mTMKz9c#o9q_tZtjvoS>^#y*1NLC4}c2%u=%s#vEe?x$6 z1DyC9;f(7Ez11z}B6Pi9)WG%Gi*Z%Ww>y;s3W?u{`3=c4zwfE}q2mEQB(|RI@O*NJ z4aqOhBP{uvD~uBJ$Q^xm!}1(^w*OR|q^C4X=wIE#BifIQ#J60~ zqL*ZuKLKH<@hMZA)nCW8O@B*)m22!q!{{_SvW`O)KjdZndX#YvMkpJcDD4dby6Xl7 z@K-2FxY$SW2X9r-2x?p>;T01|1`2>qC;&;4fI^}?z7eCEThCB)G}_=fPKj1Iv2pz< z4xsNcyKFw8v;p6Iz6D>hrV4(gxj`rN-60OUh{q_7B@~BUP5$<|Si%w}hxm&qzjA)W zyzB@qblnUg*Fg2&Ul~U|r`xP+`jHe|he5q-Qe!PM+7A0R%u;(U7u<$Li#q3Sl?j=g&t z8UK?IC=9!uDZ03=n0!Cuc0J?%r~x7w^a%f2)cvXs1Bgy_f5azz=L+r_*usAdCBeJ0 z2j7A<;q9(4>@G2UXAJ}N3x{4YKWY*J-sM99$UB(k0gzjo2_G;!!qh1irrXcT%@;_Yek^xS{8I3JO_DxeY0QkrdqdTzvI~vHOSyy-@t`^ZYNqMUux+_xRMksN9xm8ttOhwv{U6Jo_jHhjda_- z`zQ`7j3m_;2@?uio9pL$Bnu0h0%a^2J*8_Kaah&Qr3%s37{7Z*+akgF=^qdHt}Al}Xk@9_eqc*t9^TN1F^yooWFCK|5}7I-F7q@w4Owt#buSL- zD3-H{vY0PHGmZN7?5|;yma^^`O|eM5_Jyy>@x;`U^wJ&am_7Txv$dQjn5rj=bC-r% zOIgQ%;7w#I0<8tVem$-QrkcLC&HeiyPAtnVjYw>XjCEsNrZQJ_KyQo|*;sP*YU`)u zuR$Xc!A#+d)w5ENc1e|5GdIxU-6jq3m*9PJA$RgLs{)$mV26PgZ^7$z?$8K3Zhx~E z-OFA~0=?{oVQpI@mMd2JgUa({pS@H8sVzcyEix;G%qe0?7Y%Xi!L9oG1_R5IgT+lu zxFUVd_A_t4!c%Y|;Qd04%XQkdyUQ>oM*V@!p49C*dZF2v9S6K$S#U>#T}j}UEHwYkh&CE=~y=6M9sV*O$@;#L}yPj}I@M2FN^MWeK~ zjx%p2)j(>CXpIWdF*D3Kf)#Rg1+6fW5Wb_Ou^?|ccC!}?pkAPza)b0c1U9nqLwtnI zV+ydd=R#nL-_4w$J$u{h3&KWn8N~Y~sZ?3Ht_>=Hw>Q^DG}wGAu`x|Ypy+b2(R{k# zG!yj4iySGNDR)0TnpI3WdU^-!KSwm2x@px6KoN4C%w}vX_-v}?l4rYZ?E6lzX^!AG z1@f`3<`B&Q2CE2R?KLX|tD9_~&|`O5(JzOBs6H!} z`v#Rer;?)k-g{kB^(Jxt8V_p#2rjq1cI%w^Aril6meG4}MNl0s-ay@I4DSb&*HyVA*+-u9F zphEqp<9fw9O$O>`9OjvB>0VOJw>m~LEoVkuZ003rY->EOt>PXHe(7FXKJse?K9cgs ziCx8K0}W@f@FO;~JNRmkK)Ni=rFO9B5ZX63B{=hfd;+6r&iz~hwtXht!jXuf^=d5} z9&2-e94~`pT7f(-9YRRQJ`wuMoa1;G8oRmFh4Bp+CDTBNQlu$cKZnD(v%89zr=Di6 zRl19q@DesNR&iERA#KT3B`K}yJX<-a$B+H%)`DfV?b@;o+Zn3t;S$bf=JeJu_>j)c zq^R0PGkM(9h0gF)bpFZnee*RxWDGglPFHLI`Dn{XrPu(Qno5wLA=X9b!Bz9hTP4j> zRV_I1Kt=R?vT;V&BBqeq_PVi%eF2P{nY41e%ES8pLe}zSL{iclw<+XybQz-(zX3kJ z*`nz>YN*ncWIw}3tG4^fzTQjYa~ITQQkXN)fs~%om%Q}8u#VSFf>vLvU<*48DY^g( z{P)JpE)KuF_G^RqaiPWrr4ar~$*L@7on;xh*8!|ZN7=XN7XCfo8hR9>?qfHSqueRY zf!F=p#k@u?o<9_m^(VKsf82gl<11|in+yze*_0&d*Z@qgGshJur3Zng>_aLF1GSxT zn>Wnfd8qW6{B^zuvwa6ALbgLA-T;kJ-PU;a}2xb`PXad)|_J#Aq^% z+n;Ngo+n$*yHy!t9#{!58*ih+PazCXY3EU_c-t#WD!{M?D=>)G;?#T}?Ew5~-}k!+ z-fJe4siEd6?lHZv5>QF#X21lySjzN&{a&^`)#1-<)T+>dz^+O}`xKc3?G*jl(^aFg z@$$u|sunBDyzINe=+VGSTuM{?xX$cKIbc~q_ea$4T| zAO~%e9AfeW|M216^UE}t>bxiSJA{nBg{r=89=SH?zV92st>s_~^ajM!Um?2e0-%!P zON<19dL=UXhn-1j1FHzQ^RVo-`kzoVPVZ@I&XAiG`2GSws?h(bllQV#`!;J|b&XH*%L3<-B zr@0pJQ%?2>j1gaXIal+f0=NCjMGr z^+^bpbROp%(aXiZgg- z{fd+5l}DlD%2KN8m0OZ2?^!80`?3gdU?@7Z-M1b!WuMI&;k!ad*H$)J<}x@zH`jfu`VLRLEqxC$H{jFdwB|qE%1W*UW3YZF=j% z`+R;<;1BH20a=mjr#jVqmTh38$iGQ7eMgtF=gM#6E?wi1Tff3fVl%Zz4TTbHOtB=9 z7p_AE5EmR+;xt^&uq&;iV?B`B5L`M*Ahc7JWisWLfjox7yx&56IQ>#M5xC;R*0cQM zTCDt`pg1{k7nXevy32fQn?0x^ni)G&yRFP{SZHvC19+XnCNnl{q&>?VWb1>VW6~2H zFk0mfJl%Mm(yaAkL3^Mdf^bM1gtO?b6~<<)z3f2DKi9Z3_)>Gkm= zAbf<_d)3zcU|cK-s=H%)4I?xK|D^!c2}Ee+zcytKz!3gQds~No*%!+mPjU8J=^9T1 z?-3Rp8L$|4(d9L}R@3=Kw~Q%PlDaNI_kO`Lu=WGZ!L9;9lS_WL81ABDJ-d4T__g?; z_L7a_tiy=;mtG(jxt*Lz6P{J#^e?e25uPhZ22!g6r_*jHBQ#s9;M-Ewh_LMzlqpbL z%~=kd@DTqg^xW{3^(0Pv4x?4!aCaNQEy#^n{7>Pok z^$wyM_I4Rc`ihg!7t5vq%vC^@s@QXRw(Sxb$ms6=juNAEbz>hnWy zYvjJw2A_PFq*YzV@=r%!3qen>y;_`4co#r_|4qVgkk<7ELOvh#0mpO)%K$CYFK4JK z=r(K7N%w<-Ypz4UuH_FWa`RJmESK<)<_Rn07XzNyxUaqN#viV)zvd3$U!XQ#XHMfc z7}5r@LU2vp&~!O(mQT*A@HBz;U)6%_GP~X6O?)BmnsmKA)V5!jum=M0vz*_>c&-3H zB`)kZWA*8q?1s)CT(PGP`#79pV0TO#e{*@r_6(<^ zQ)G-2L{UqfD*^wM25btRRh4#*_eBCCWAPt@0~Obuuj!&ztPy?tdP@u>Y+D*4wrV|T zALZZwP%yQ|yQUqT?3tfMc@Vh(xzv$7*Y;Ubm(H-{aN_GBIO~&W8-!{gbq^ePj2C3n zBC!Q|@%4GtHD?R-R|(n66Dmi=%U$9zwrR1qb9ISqNY~>+ISRN&@zem)5KCxN z6^KpQ^fvd@8e|%CBr{G`-w~JeTYryR)ybk|>NX9?L9Xo69(jV=dh1ZZu|Qk0wP2YE zMNK%=q|FedjdhlRG%Re(FMFuG%}ma3qxo2BydgG-j{3z*n{eG`-=yU*jOyuE|F*O| z?-Cd@pa3*;eVf_g)eJ3otsVoUGbU@WRINhR=@8Nt$CRaEk z<{mH}gb?@g{b5`L9qH2@C~NoLnCoAUZ)NQ$IRG!TIv=pLg}fgOo;i$&46c_Jk#l1N z4|C(?#Q*5Q{?APPzd5|o-(peSVD?H-h`+;B_z?Cxq`#x^q2;jRPc^EmPYOKgC%{!3 z0`Xa!M}>xvI#tPtmMS00-h}vfo=+y<9`YwNke*s6kBiw77r`$7+0XZ{)wmFd9x&9u zLxf+gEA5|i6h155BtMH^(fCr8Zn#le%EKWR|1IBzeG1>2L<}yJsc@>|H9-3;^_J@B&QJJK8DA?(HM2vgWqbQmGsJu51x=11*j?Wn@Bq)(S#YJ)iIa`P1jZF`i) zM6jX8EXwRRon!jks-~mfF^v)FReAD=h_g7m<_<__>aMzCWeE2H--eJO#cf+rf(~B1 zKP4m+Bw6huNdXzjDR;%3dXlOLE3z16#Y#9uT~Dx_w!r>k6> z?co$C6CsSqd%AOtS0#b{;vrhfUiAGN4W2j_XglZjp@0Yd$Uf8#sz51CIx~(!R=g3C z;iHJ^2$k!wygAHYmBuiiz19+Wl+i%96t6g9M!be|(JU$JnkQzZnFNDFeB@2&6iBRs zl&zD`!Rgr3VVJ%6?b?H(5NBA={{)EcBUQj3x8czNQSVP@nRjCR#yh*v^$P5rTod^d z(8m#pL;&{lMU)z{p^!UjjM2QY_|WvPrO$*DWBEf}3F1BimZ=2d4C5s3eQa_Ylm-E+ z#f-WsPGBa%k;?7se6u$w)(#Jf_$1Pygi?IA&xY+^75O1kWeJ9_~P!p#8pHIqGzQeRn-9j zxpeRUNaFoBKK>`F_&3@m@v~U|)(&rjrv|p6eU`%-kszdQK5)UbI1fUQfc+C)n+3j9 zzY#$|67f<55inA>P2gcuTU@U;$ogOI;s8GpdH)RKan}@s67@ys1>Xj<}@LCRQ+lB%{Lb|Y1&pqj+ylh|kB^Hpl(v2-Qgp=wI$9*-66;#T0=lMWr#g_mIp6#b@oAyV8;Uv=|<3U5>q`VgG*@@wia7O za-oHES_>D9mk+o7*|EbVCd{P60)|^fNVL_TJj6_|iRO0qg$RB$51Uq`l^5?t*66lh zbFKC)caA7m84#(Z9Cm%`0d9G^17b-sUmY=8&!RWSf-(y^E3J~IRrCQ1oT|{jFD>jQ zB$%7o%Zod*p&g6Z-5ifDq-83ML&+yLAT-oHy90l_qI5{0`SR~k%b;!CTYc~EQiWTA z)*leSklGOg9os8 zNzYkYdN>G_&LM}0J)IR$&i>`N$u!s@840g1TkmW*H^d}Ksu|a;TPgx{KQp+VTk2kh8Zsy+nigp_Y_xJZ!q?aNawo+m zdayxI8N7LV6lZCyeM(lGccb^rSHvFe&(>y@7ArQkD+#I5*_OZChl-cUD>LvmpC}v> zYpca*sm*gT&36ZoyILLBBu*T!4+*(XtLsDfgLjAQ>27H`g@l+hSn|6_F>zC>b8|Hr zXVpx`J~$a3b-Up)i(~juTddkDR_lH|`qZ+@)b@Kc`2>EZ&e!it&E|+usi8sQ*LciC z`A5x7lxWdkckza^25*aw!zG`Bpbd@7j#^B_`LsKIutL278w^CQtVtoE&Xch?>~ueq7pU1=3kxjD$y`N@AA*D+t?4mGf(s z=60At{i-PdNissXp3O%CM6W=Rg`XG}Jxg?(S5d^mgIlV1X=GJ9v;wp(*~M(y-oEN# z>iV_oTZ?{7;~Oeal+2iWma_*q|(>toy(Ykk^0B<$!E+)7#S zjH@($ZRQ~{PlT;lE#k};;G2Mllj<;of7HA^Mt^w&$O_OS-su{|;`!+1ubEStY3?S6 znK#xEvj>TXw-Wv|hX}3W0a|{ZA;c7+z7!d$YU(|SgwY|1NZkO+*p7P|_Cno!V+HKj z1Fg5WYd4a5q7Q=*39R=-!XyaW7PKfaT616dj))--L_pt%RN4n+gok&7KJe*YX*6sk zm6Hol&pwW4-C4V^I64%ersG$D#p+(6{mX}NVrC83cO@i$d`&x_`S`*DDg_h&9!BVk zi+T@ci#z+5>kT&7-r-WB*cJmi!;DcZrk+tWhxDUty?z+esJ0gP~Fg9SvvcclG?VX6eCsoO!oMf^V9O|pv%%g7$(@QDr648 zol{%>OVf$GM0AKWrpT@fD5GjsQXV&((FvDlT0ghWc#GZAn{siq#%f#*LwZm1sUIuL ziNmt`(NtnlJ3njVL>Q&#&EqR9BHI+@%DVwQDm-o`xl5jVnp)0{GIY zvWQy%(><1$zQ4WMS&u(vDoZ|B)$n}U;X0|6dd{@+A;rsIQYaj9Z+tlJu5$1Oa&--WV>|x@$lnu_QHLJK zUgphgiaT6Tb|a%KK!@)Q6uJn11x(|O-dpNQ_dzISpVFHZs>11pQX4{6;}B8^apTrN|ADSLWy z9s$2mHYA6gw;+V#U8wP#(U9$6uy*=JZLnwPWc1J@(#e`G7CkX==HD4{fEF@R8! zK;nY-wepf8AfTN~DVP$&{umg$gB@FZuC#m9yYpr0SH-*mpkbawwT*^MaVSl)M2A5p z*@AXmfT(UxcHU1`B{@(B?G5tyO9aYdV&iWOQB&{7UP0u~f`?M5c{Ol^;KKwp^mLMAcYd$ZlzcaFSXTV+4EA4M0F8TF<~OQfoD)7|I`QPo`67)4 zN_PEZb-2m9e3d9GdvtSw_xb3V64-o$SRTlmGDjxAF|vY;k|2U@$yKKWRA_`4YuxWn{$FB9imfa%tW;rLDO+ z8HeaA%AdT7y$V%TIa#zY`Gmx~>p7~}#yvI0G`~JgDHKYx{#UB+#aI0-m)q5 zOSsO805T-8qst?`3j?jx(o6{*a|SBLHX;*w0fA3mr%}PVB!;F;Z7^(psD-a3;&j&G zGArF83XfghO#`qGZf9y@70BC3d||eD--2PVfZwz*eP{??;YB!$mG*MEgL|kYLhPZ3#`B3GGRxB3+`ejS?5*+HE zEHyAAT%F_-OWFZVRpNsD#DJ>U;ZnO_xd;J2lm!(BSTF=qQ$!}X^>I)z#UbHSaJ16O zm4cN>XLxLkLczmZ5V{wp_6CjkW51nu{GOti$I?%mtJYD*`s^Q0z#P{Vb}B*7qiN|5O0d`y&#(+ti9_ zOm?U4rPq-Xe;Y?`=lM47y$O&()s&}5P|Y)4;Hf*%7@GGvqLg!k(a^nx*>`~UIwF_* z;QgVrs;1M%C|2F+hiU{87%Kn;>#eCtYh1M6MUD@_Y@;)Q-LK`{mXV$~);-K-e2gC% z<Ld%NCiMiP0m@!7%v=au~@0eQ{8T)G+-7G-g?>%PwZt<6I1QzTmp>0ufGv3(+gIczX zxv_0;SMcxDe|@*}it~T`^8Yxma+E##KjC&hfu{7YALqge3je~gmjBa_@2}jSepHrM z1Y5bzc;(c`=hz9z9T`A!@j;u|PxBeUYxLG{X@NJaDt`cAV~Yqu*Z zFK{XNA^t1&T3?=Wi zG+W&&$sfj%FZSm>s7r8_uv$h4AlV$-C`0Luw2gb2Z1J66>>HTzftK#MB23%5LPs#! zLBwdmm(iEN7}$=g4}P1loNiO01%<3Esy8A2exIu3j7Bmyz8irZe17}3&cvn4cO5cu zjshZdx~kF&IbGf=zL3ii5iL9GiIuV10VdN3PvzZqzr2M#*9}hP%JTtCz>DZ2c@k5V zfWbm3&jHR$8d?Eckl$t!gELbH=9pMGg>(CR#8c{>mwGJ+Y27!8aCL2VT2sggt?`Ul zHr{84N`nH|F6F$+Qg{;&pw&>N)7cNd#i3t zKmG;dJvbO#y_lc@n($r(pWOy7h1xTZIpi0xY40G5 z>=KHlZ9%Q!sO0Ap(*BARNP91iU2qV#uW5T&W@4dYei21cQbP5Hl~ERvl_j!IrZbw5 z#Xv!UG<^K}*;yRx1Yk*(DPJ#*XV-l(`O|l%?eO(b_*2keu;YaotX6FjTgxnM-+8m% zBc%kuNUlM|x)<-pRyI;~v&7nmCg0^2?vp)V*5&;{$PUB+p{jrx`$*r?kTDed=FOVMEYq9j_x4i05VD~ofT!`G`Lh&K#i^-NVI?tzfe zXgkK==}j%n@T(d$mCc@58FY6l!>!$9`m35o?#3Hv1fKFgJMqf<%(E1*VHIR#?JSl9 z8j}@HTm;qCqGd@o5;PXV)-~kno2lFx+ao$T6Zs|YIw^fIhcM_P1udiCPZ-Ellu693 zrDbE`k`8`pH%uy!s~Fe|Yh+l+t?VXcZm%*brR^u1Qd*Y82>`$9H6XBGfRL~N%;O*3Hto+{4I1!o>_~@^GnM8zx5g;-cNo0j4d=|g^ zg6ijNlw*vbl$dHMID``ThnbY)W==wRd85w_v*3(y(dqJsOcLtBD#Y8;vN)TY-^2t> z)Ugg@OrtAGRfep2^!v)ZdLn3PlBSudkqe^qCc{pR86Aot{PIF}p)l#4ioe4FP)=?o zuifv)b$LTX5fPD?)Llw!Fh}yF@YD-PkY&{;hY6nAWO5(VVDWm_;7-iF4o$L?-gA4$ z$W>+8+3FCDqI+qRO&Ht|?P z;Q^`CMETkfkytCT-WfQSd?MKhFwf-)5gQ^`zof!yC^SBIgDZ@P=8?=R=CW9>lY($! za+Qf>t11a~VX;k&xOMLPk`+2t04!>H#$qIFfLf{q@iM|fP{$PsCb*of%1BEgJ1q{0 zXlQPNKK>EI2$z2pXFg+R5#yuw=e)`=@Rh2rOt%iqxRiRAQbf?csUSmi>=>5O zz{{Y-8r5x@)j%f{5i5-c0O&o7LPHcIvi&M5iI}(!4{ycLWN{AZ!0IiL$}J^Y@M!0Z zDS?*#ss7l=AfO39IgMB|SLV3wjNWSrK^0OlV4Hbrvm9cZ33(8Mf};5yOF#7V)`?L_ zb^_gkfvT2aktIY)VaB01dcI+1f_r0x8IGVwqJ>OXlX7bMz*zm0g_Qa`@2GEz%L46> zb=v?V%wgy7oGH^w(&9D^6RzQ@De*=hk8^53d8_L8ivj%vvO#kg35%PnG4ra3njMGZ zB_>!<+qjI`J!Lk92wu>tk{;SU69;GFi{y$6%h-X4R)xcKAMpmt)VW5{Z8HGkL>Sks5Lw8ff>#ooxH1#mJ^1UrCtrf=PG)P znQ=EKQePB{Y$jWi&D7(#_j?kVloR#GJszwG6PPCS03GaN;E7A99Xk7!xgUc)7_Zr5RyZdc zf^;D?znEP7uv%-ayeJ8|O7uJHKiLbWBH9VBWz%%zRQU2m1Ok&)^mqK9i}g{`TK{=X_f>02vhU4n0c-d90u#H8PJ|n z1q!h|rh#BhB5ou-fmY|HU`>vi{FzQ%r)qiP6wpD4E%<|Sa0=QWGS4XGH)${RrH3Du zG|v-59JW3T>6T#Hn4>`h+g8KlltujIkDG3DVQ<&n1{ost2X4pDLUC%<3dvs`ylg=7 znyYfE$Ii||>>)~MrS;`ViA}A^l?bl)-!0C&{`tt3XQJlBLcLpV%o-jADT&@L_0{+l zPXGhOz7PkF8%_qu%*U>3Y{pG1#^b)?1Zd?BWa1km^27BFp2j9*R!60~euiW5c6@zg zDexhL_wpF&RZ>tC_NM?ms%uZmJ{`HV@nb7UAYU@bP4+}j!|_F1=}vSW7S)LpY)+~% zrn$$i$%wi$e-SuA_*}|RbmMkCXC#to2mmJaVt^X5JCc5&W83h8exs`tdZjWe$8*n~ zh7Q;RowE_j{P_FT<%0=>DTl;Eb(d-<3gZd6Adbb)dv=vnj^^g_m-`*ocl}Iga5wm3$X8hsW&`>uQoi2+DL z1m{k)E(ARmPSe_-&?v7fJ1@1e-46usPj0SW1JYAtJ_uu@BRS(wk=-oudAr@|!?|^M zD|LRxUviloCd5UAgh{J1f>4qstKSv6DZijLX^>ySyn^caByBu_U0rd0ivXz(hS^5D z<{DIPXJy%I80nJCb(&gi!NH5Q^9Se~7>Ks``(-bN)@US8p7#mv)QhLlifc9_8>R}> zLwkO97{3{|HRX#H!*F)Kp3C76CuWZe+az^vR!)aY>>K)s7=A{@lP_XbPs*~PrS1hJeZ7(4{Gmf@h)iL| ztI9>)hXaIP$M)?nZnMPKNvbE0DHLs_pGNc#0|gq=m&n^2cdN>bb_8#7g+U$cB1x;o zPwaSalVM`IiNsqCV+>nF!}owI%X61RRp1~)KwX)83ymk}pM}TeKt?sqO{Gdl_hf>Z z8c8XNnkq9|{iFub779Xwd}(4Nq( z#NThP{%3;H7dw}GUj50HyQz+0g^ASJa&ZD4Rs{BJ9KRUsyI%bh*0TU{KNa-@gy<)p zabJ@rw%;cFC4_vhj+^_5?|0LKB)Zv(kKhF#t>bH;Ykmqu`6Fr4v0JEFOqRM$Za>m; zk^47`x}M#{l_05g*#6*#>^HUwk~vJbDH!A1-U>zLP|p0>ww zllClP z@!{v|9rmOxroynR;c6cMH7Kwl&3to~Sp98-|KkQY@^6RD|K=e7JADEBBtg9|k!bvz z+_q~;Z@Pch3*y24B~)RgUfEJ(q{h1OK&L9Y@Ie8R)oooDzG4X4@?V1&QGkL4O9D}J z_-U18tkG*Tmnk3!f{!VxbuHG9?FLOCB9h{lS)LYq)Qq5T(vI(30csl^^h2y%(e^W1 z9qqoCZHL_&pJnJ5iXfWKv3W<2#ngpAxpn8Arx;P6t9r3gwiQ=?U^L$aAy#LP?3mERDD(nMjA4G81d$Sk0{d`7(m$ki)Lw|CQ-(Q$ zjD@it7BZM^8CT4;Xtg%&RZimUCcdE=&^^%M^2EaPW(`P$EM$AVvsNQa0IzJBkO$^T zjtdedA68^cQ;bbITrQz2bUNO2n>2cwmP)k{Ds7CAQX5)>XWGT0c+{ZDj7eg}Vj=*V zg-9?ah#_STtu9>#I+Ww+f&nj3w+@v0ZIG-5t8CxJzh?s){W3cciFo`*Fx~HS>HA#-lx@#1!bno&{7ajA zJaaQPJu-+=tG+!5C!ccrWCEH({T9K4FdlMsT|z0%y;zbm`}mG}=hmuHsDb)(PGX_E z6<*oQZEk+COwBo=xBO~;8_!k)%Cwy2+0Lhc?&=ZUHn4IPR0W&G?bHD4sc&T) zso9u5?AGRuF_S%gkMQK%27|e@3qDFye#6?KQJd`D>Od75>@E^F)+0?p&S&)> zYc+gnmEq$O_I_X#dV>$wC` zJu$rH@d8uh7x~*ZVs|Y2G~W%scu1<1rzSr^V3Fccg`^{{+z0eu_=DoV;rYLV_@{!P z_%DPfR1gqw+P{3ie*%{b@J1o4n3ChV>Wst>3My?U+l@vNJP1qEyhjrcG~$JF=XaHP&pIuio^jJa7O1xOxZRK%%W% zILX8|C&|RNIk9cqwkAo(wrx-BWTJ^}+qRv5?)P5(_tjU`r@9)|UDaLt>|N`uy_Vg^ z;szb#?A6y-j;Uq>zPa^FQ0dpppQ-Nd&%dcts54T&@h7vbGimd(&aIBaQ53kqpZi9p zq=-{P-L<)jOs{!XC0Hh9S?8yW>#9|~DyDq0H{6!Xc8p)vOGgY$6C=9HWnq?zMAHbn zl*$(`;To&O;?XKHM8PAr(dlXv+j_sdwDF|UR(`>HrjNqe_)Xvlz@Pnk2ofrmq31Hm zXsWV4YsTMc&S}1msoGI1si$oNM8;KkSD@Nd2BINh{t(53VsRv(hJ=o$b)XrfVPf$| zr}OIlRasqaW@_bI=kx6&%%QsZQKGYZis;tV;(B##WxcYxfNQA54WXng{cuu3_ShIo zoqiZgG6nIMh{6UAHkW}(BmjS{H-GXGKY#MFxjN^^ab*seq!u2wjDSi$P*h&}Xk{=9 z)qpdU2qlrn1-^2G%Gd%TrQWNQ#AA5>a6(lw_+4`VVzMa30v$>~*mzhEdJGOZO0x5I`r<|8QD8Rdlo%9n224pA%j%&+X zng1d{!Cs4FEn&4fC?-u=#o5>vsgmc+;x~WVY$0xcc;BT2^x;woFpI0DA=4mH*1(1i zV+Y;RRM9x8r4uU(($pvJGcnHB(kxzVO?6BLWow{x?^ypVRV0KRmg8-%B@upwmggIU|S7y zj>`%3SFTpyYO75lwhxpGXD3`DQ|$_nU=CY^*UW(LW*lfUZ0_#T)|%Xb4V>%@sRTRk zVS1h>?U&rdL5dl z+Lv30WLQKN^gm;t>}O{`!a!it^>9(KsLAOYvBQHfj^O5 z&I9qWzg@WKPzr|(0LWoOtBR4>y}6a5Fkv&P$V*EhyY`+xb7YjY_oRGe;|Gb1NXAQ5 z&%NgqSiI>%(`1qc?WbT=$Cyi%MStj)G0xX}lAlRbQp-$p~95uvQd%pP^#bG?EEbR`o*@KJY? zdiMx2kmRFpo4%rF8xN&Of8?Y}9ZNd|QqCGy&?H_)LhevdVpv<1g27hHO-mXs)%bmC z@bXyzMXt$WWx9UfgM}QjEFZpf`xaD4DzD&C=O8d7VLKz7;JGN1nU&m#l}^~K81ftL z+ogu+h(1B*t#DH|%&F|0us=y&D-4bhEO=@>6pmww$}X!fY$=W5<}s;#vNx$qYG&3f z>?f~EU_YpoN4Dc|R`J)97lWB9bM!`V&?dD3rIV=>2C3tzW4vhuQYJBu&kKSTZS7;DY|Ow+HI`*E3KoXW(8tOaN< zJ|lj&kvNHy)v76fxR{&mDl&ZC=2M8XPmQT9ecLs2U0o_IcC+;H@|lNS^s4-eHRGxe zbXy`;TQ<_>gOb-VbWzYp%`!`55d`4(pCV8Nx(Ii3#a)Az*$aTlMbuV|z<%`zBKZ(H z3SwdQx;)%>$kKT@ya~VZ7^?j=s(u6RELJMyBQF+uKcNQ~6MXpm!fKLTr>~3OpqB7CH7){& z!T$y%-|QlHyo<4I=jZl(1XS_;oy1g({hW`e_yP%)Dv(n;Aq?J|C;$Sa)5MmSOG~v_ zQS?7GD}M@o5hrZjSI4PJeF!BfeT5Wh3(RoH7B77vdws~aHj}k0Fx$drl~qJnrSS?b9(1JJ&!{Ep+Xg-)**A!**`2izqnp2hm5U>LskwF z%&4%bTg{Y%M$fGd3YDg(tyd3#wpD^v(hvHKiYvU@;Hs8`A zw=39jRP8(RCa6a7qQYYB+Bu+}+0qeICvI}$Q{D!9RSL$7UnII&cj3x$^?<0kcS6%g z94s8%zE(pAX$pqtP|-F}JfQK!h)S(NeF$S>gua6#H%9TV@iI$C9_`Fnf<_fN;QDb9 zRPJPjCLHLmH-IC%9wPu4XJZ{EVUpqal3k8D5vsy+9ElnAs&|wvQIKn)8++P~7t39~FIkx|YM9(xYQ4 zg1gJrcQ$6nPhX95cbu2jH%=?1DY?Qu<*^TEi_h}lcp~C*{5e;0hjtKI>Qgs9rVwcV zOKWQM!LbZSRwfV#oe{>km{XEWMyoH`@y`NtIjyLv%|jsKbSc;4bkb_c1s1VX=En(x zI0r1zA>q4Sq-ogTXyjcKJ5rxua36m+j^Ea-nHo=1@AGY9U*p#^8L-|E5MfA%x(g1c zpkL;wyu$s#q=Vd|nGUJvYH2C5x_Y|xdEe>iwlb;Iz}yGiGLB+@tT(yZoLz39oLs`X zHjAr#QvQm35%jf)_)gC9JS39#3tXk}sh)6cK5ao_rT%ev$qX7CdfEyaVE+|l$c!Pr z>mxbwT`F9!W+3dOz!fvvCN?Dk&SGPiBqwc8veVhZAT_66U5bRf-RPd?ah>Ppl=8DR z0F048@2MY{b3BJkr1UlYn~KShoVbmI6tkxhho#?-mPDD1iLqXn6OW0xnHcNLwy|;a zE2Zq`7S5j+LO@pAz3Xm{y#ZNSWRiUlV#tazT||&3Y;n#7rkxB6WujOrBaq>ag3(!l z>+r7r@Qz9RwWl=!)3i$6uq!Ra&Ydo%#9A=#N zT4XwGcHYr$y+Q7{E2hTr810Kn#=Rj9Y5e8a|LFAoMu!kH128BVN`70galJ@Z2Ek~m zUu%1&MaV1MDLUomSr3-|^AO8^NAId|_0uYk4*N+Z>9YNX41Uk}Gdf-f@d^^cf18oK zU;G+y+?!o^rYnh_M?42xvhs{BGm2`CnpWHX`y{Fq5u+4;XDXNzbeZx@kQ%4EPuY!0 zK%CQJOR2k`1U33$uXKXbUfoh@o#dr=ms^DAWJ%v5LY1GQh|(2$!xx}-s5zdShz(+c z`HzMehXB-dNU694*m06Cj}V_ic$3Uai{1kJblc(%D(6U1Gck&*T>d~;lOV1rS*;PJ zC{&p_`NhTP`G5^;iiv!DsF;nRRMSmEVi0xMaWlp;i3us^Bh=w+zGvgFB)f@X;fZM? zLUdbhTAXrBe9dKjG8C&jRjS&{I!%rr|B_k6&svY91T^RmrEw|9d?O}$@m>OaClg@$ z$D};8@W{UnZBYBi_Jw4x2YqZMxEh4!FM*_IG=uW_;awfe=gJ9q$6}maDZ1Ty3F?VB zl??G&4x%P^8zX))nB$hF8w73ov-iTJ$K3qwS-)?k$BJLO%J;25S}CP7&H!fcnud7_@x57{R5ICx%}j-ILQ*szB!?KIr#r?s zhTTo+Fs?0emlx3O59pcgp?R5~@s}XSN3|N@scOe4kgs4VjH&#B^2;gH#ytLOgxJNU zu?HGCJpVx0uBUNQBnNX++Q%3fr$&7lrRP%T+Mm)>E3jV|2Ef$d8%~yY0+3Vh_%Y31 z4GktNj$g2JG(0eLYC`%3-!-NDfZcfv1QQVEGZ5qlGx~4w36ph#(r!e4U41V$9(vaJ zdk{59hZb!D9jg%}3}~A&jzAz96^850(pd`Yv&%%Abv15ssWoQ+z;HzL56LYg(|4n`E_TpId3aL ze_?y23@l1d#*`9NF-bHBHIq%VlH)iWxrVdE8`=U!JwWNUgMYZklgb7&ZkD#eCU?*{ zMR_>3gn?c8D0;CmXqk&Gx8Lapc2_7{2Q(+!&}X$u&;$^j#Wp z;A+n4`JiTa&ri}j6zH<1k?fNhWFIa(hw^WqyWb#ND-sk_}ue_x`PQ45OD0nZZb0X(5O;rv}ZUD{p@F9lx- zUyE%W7(mMdnp&o&|8)8@l_Nd{_2-iH9Os*c@r@5Xzt<(c^#Sxk_D;nnOGlA=5(BmQ7BoW@-p;90^#F9wjr|EU zlvvwk6WTgsFQDErNtXTaV6N-m@WC_W!B%2TyYAl*#vd7RI;GaC^T9K)AYrD~$O2M8 zjC$7sTI#Df1I?Rge7B!%7?n7_b7F~DG~qi03YP-I+*t>x z;-H}W9H!ksMj}CB(|U-aqS)Xqddo6dwhP!$auK)M9KbxZFn_KgykxL`s3D#m^Uydo zBjnDbb;2Z$$G5jEq3_T*XBnmM9CS=KWOf4Hw9g+3a;&yo>=52t3_B$jR(Q}&x)gzA zQKv(@<393ps1FY=$+?57B@2y0cs?F*8XJR<3u)^a3E@rK_YaO4Dqk~XCk_TMiXS95 zRSvQDfXl@DWT#7^&r#=toILUf9_pY|yh|BHyZar= zYHP-#GJf~d;qH^|qvFFO!`C|hx>4hy&{98(>1k49 z{r};{_T4z_5I@t7k(gU5x^c+BNdM1mARw_2|L?p1^8mjd+iT5#;PV<(szL-jK`Mhk z$G=uoKaTG}mFk%ink4$C7c`E9n17kyZna{)HT^Ki8Q&=^ljFj6u!!43?1GG!sY%-3 zbb@*D6{uyXTB*KvwF`ORSxze_N5+$+QXyeJ>ypcdev9kl;ePGu>l4ZV61X>pE_Tjw|z^t1!;9`?G7-+D`gTuJ0s~SpEys9kb2;_mf7t0={9{1z5#8JjUu? zDE&x>Q2o;F@NzZYGP$wL8uC~E*^Z8nn6p1wz9M+ql}9K2?b8K- z7w-}Th2*n?6=vzim}!{@A`K!^vuk?L1KU^@(Pre%hy5vs=^`BtN=hgq~TQzfUqQGnfJ8#t=vwgnw-}p~lRqjv5?b%5qzil8ZbSuvSYJ1ISuZ@2S zbMe?xeXhcZ)!Q6$i8?P=s$yBOr|f~!cRg3T5$>hpmds`X&30dAgJmdN2Zd}KKQZn@ z7g5Xms2#?tPk{#5&ZlWk1=>z<8o7=jwRM51yWwl`1WD$glp&N1B=N~_=%19x6Dks7 zU&h$6-#(*7&}-GaK_9`wfoW|ZdcSjN`pPzsVQxz-BBhai))pIYnML0#unw5*r#WA^ z#tKQw^jgKXFjm+(4tIbrOm#?#TO?PQCU+ELQCuh?(OHoj#8$^O-M>cLp+YXmb<9#Y zBH{ZLS`iZvL(G*_fJ)&IFbJ;OlE^910DmHO+)NP)WSa&+@I7LWlHSSykH2-)E~V#2&uV?@X)E7@79%>$m_pH{9wwATI5F`|lA)LpP@$X3ie3krm%xw zv+_9$`ZI-em|I=>+oPoj(x!u3H$6MTCFmCeYdW9M+dbrxW{lMuVY z8JG8a8VT{l%KoOfi=TH7k!-AW40DcVb?z*H13qsfnqnH!G3~ zN&>L*bZp>6i#^hb$FDJG7a)ZK+Bjc-{%gr#3=HGvhtwqeondH>tn7SZO%& zHlO0k)bH;0ftn7L0;6unX#9~F6p#b1G)#RwP(^CD7bB1%RG)=?IeM%t(XBtohx260 zYhVF9o|UAQB!|LZ1Vc=@yY}UJ;avN5*Yka_nJeTjv%5>DTzak>W7qsgP;~1n563k- zh1T|@YI@1dj)4tx?hnpH;-29g`-fu4$haf+dAey>Y8LK|ZB-gOfJw{X;4Y?LD>19) z2)~iBpsPGiVyGiBslX3;b>a(qoYc^(y8&?E|JkRl(O9P-TyhVmV^%?Y|ChMzeDRG! zXrL@v8dZEYG%^;|3n-JqpCI*7^!RYT!JpU?eA{Yd?W>FKac#S>UelpgrEoydV-Jp= z)FtU`DC?d(VWqI?bK;%zWtsSBS5{^UBf_ZStSZ3w#OT9TWQt(IiB&;YNK#Z{iUD9S zOl2xOq9~|yj3V-_zqVC$dPJ#f_9UQ$rD~UGH+&2E^fmTFrW~AsFO0c6E(~trn=dNH ztWDzDbYjT#S;J5pGv;K4!0(o-LmIoj`D)H>M+Z+yR|8+cXSA7mBz1LB*oh_Tv$lcS zm^H!mFWuQW#?vusH!3bIv5o!}(f|j%QC3$fiMzA-`IWc)9>{0aW`*KWlONdXdGNjA zJ*5K)FYM{_cn4=#_qkKszfV_ojyT!tkDHY19-|pqvYWy=o6dWv0iW^#W|?e;Iyt*6!yR|I_kAZO1i+juM1zj6 z9Xu}fEoL)3GdB6X{D#34B0p@t)U$}ZB`_!7XfJD|KSOIsk6Mc`OoX|`EmuiWhkJ|u zojV+&2$Zx)&_zP{I3;9;Av*6zwC$ei`o3yEnG4GAv7o)U_Nh4>=za$_)gRSM#MK`t zOCrrrP)V+&?nv4g*x|$PCBjt1?tJs|dt#K*JLId-P4Tp%-D}E@bsXNG|3U}~`oDqm zzeDN;>d%KQ1_%f`U1~f&E>`L`6c^_IRUs|bf-z88Sm38%8Of9(Ko&xUmNXCuVuT_6 zQGkm14F(w+wBGMWMBlhHGh=#qL#>9EEyja=RdZ8QuvWfZotn1b2(hbWlbz~W*Sr;0 zRnu=e)io{FFZat|Ea{|Z;Hu7BolhITF1fZeukdo8pEuvaE!=Q&#XZRW`e7IA^HTpI#x8+oZ}*1 z=RoNUNx7WyQVU*D%p7WSxdo4mK{C0F9uj93~KDQ|jt(vEk$x&OKL@xG{cvrRQBBBr!`c;fFi zF)!&kP-xZ5Utp*V${&$$udi7hS=I*+uq*XS5)5%Aav7?#^X9t)R+fvy>DC8DkWd^} z>-$9z>*xEf9sH3~I|#iKV$oB{O_^DZDN~7?FG3~_@C<+?tK*P=pp7# zm#T(kBFX#g8ubLbC0KoxZbsf9QTj>Tp^#sTA?ubrUkYKSs9%%RwGGD>~ODu8P^j3W#WOtSc1ZnVc+nqBjRj1|#g9OqYIAtG|69d>M| z@9~t0Zz7!40Bk7VA8;b!n;Q0*C~VL^Jlm)cB9Alr5=mhyHbf0DFhs!O5AonWZpem~ zHH2C$du$ofg+-LB3;4ScNpPSIxKvm-`YR@t#zLbYTxY%?@?B^^z4=2*6?WyVi@_Wh zMYEH~hMH}-fQbFPh)wv5lBLGl&Wag$CeQR;C@yavFv4mUvFxL9EMD<}GL_;7dSN>L z1obf!EQ6E^&qra%KJ8Ok$lPSRc+T3vG~;kVL2VS)14E1uB4Pb3V`SjiMsW-WvbM*# zupe%toWKA9nSWh9dUBR_E-l3WJX6LEOqazr*)#RZ5>qM5=?i#^vMN#oE~!_b6538f%P zT#SDeg0jm0Tk42146@T&IO zF90qe$XGDz0*TGv`Y$~=60GEx>Ho6Bv&(s$GstsbnF5Lgj)9A zWW)VFCd=-UULwKI{$Z_b26NHL?GtM!vB_q`R=193DcqCNusLh_6My=Iyj^V1hZ{(B zyzZBm*yg?{_Q3_@kMKOY<+h3d`;&Ww_LTB{kw~uW>O%Xod;HhtK`56`{!agh6YznB z7qQo9Il@`#?MD5XrnrU$G(QH>Xi{VBwQjKUK0(Z(chFps(R$m8ESDyaJpNo7TqmOU zKSpKII){HOvsb2< zto(~}?%l;Vh;ZSV$&9qzL)D6Auk#-?;R8H9q%s0LUvUW!OBjb zbNTF~JTfMFE9pT51GZacrWH*ViJG-A?4p~#{UvptXgt05q{nXg-M_q&KicA14ZGx> z`Y_1`_*<)q4!C50QSx?@#4wVJ_KVhPX4KkZxa-ca_nCs7THQ>bZ+{itlj6XO{*Q?(qGVR6J)71jPTSOkB20gq`6-Z$Uxlf?r2ie?6ktA&{>$p4m+(#x; z=<4{InMcb)Xu$&$738GMv*RPo90A zfh06diY@fY8dA#W@hGK(J2sMS8X1LSPi}6745I1O@A`S5Coo4?WKyk3HTW}eTx6;{ z)4>|?si%NIS!rIonWd_;?%biO>XD6}va-IecDIeQvg+Qbn5wD03HL)#mxO_F3AFA`_Jb|>v0ihJ{u)?`X%Up4ralBEv}2ZYO)MX>5wXLk zF|jlG>Qc?2q>pTN0~9VS2YrL@kSkT2jskqJ)IM@6$)Q^D1V!6%xa|x zITw0&xXf3s>C8>n;_7PuHYm2m8q?bVvw^f47iwoF7?BZJ71}9FGu{LFm9#RtFLWf* zRSg96;paFg@Vb>@}#hPbhIoh zuYjasB#RBlaY?3dTmlhFu2*A#_!Us?BDS}= z7Y$*M^O4Iyk@Ed$tiaPVL~NHmD|a*`nICX$>%SOfR@;&5n)TwtC#d<^KDJNp7l{Qw zOK52mDO%Xmz}+aUgxXOtKxmh$nVUg{BW7&f<0VbGu4gM@qD(-%<)P*((xH}yubgn2 zVHNw#z?>vph#AjwsxX1*?A*7RQtQu$Gx~$XX+)aeP?A_^#Qj3$4w(PT}|s!pZUt=}i{d&Td8Yr~}YEuF2w)p=P2JvIa^OA<5N<4QR$ zqrXu3D&RCThGBi<5@SRaHd>CMUI7{g2o(sJ?KqnhAfijh!YxR1TEFr*^yqaN<9|=f z7uFbgv*`xjEX_ZkC6O!{-8=8cROYlkRvfukQqa$yhQoDx?ZVdNv?fs;`{rVfMT36~ zILT9t5>PzJL=Ej;VYXw?@Pt3(&JO#h|5`6LZ4QVckYdTax2tc9jq?%aC zo|q3&Oq$G>w)nnaRZZ!Lb@`3#4zRw4i#?&N#Kx1l zzFY==%lpolQ-V0*{8MUD*u@G-wsS1noIzH)@5J#$oSMGb?;C?@d_L*)M05n4lKEO# z1MQv%@`e0~c?nF9w@rk=0}=I#IG3N#EB`3BkX@eME69C8s+%H?i}I>8i_J2xbFx;t z{v-5(L6^7v*QsyvK0qn(ZeU0=KO-kiS%&SLZcdh0RT6q^=&&0?^8US(b-;$a_yE5U z2bVA4Z{dB~z$t+N&9NLo%tBqU=z!-VeFY~cbT-^y3G8QabTnrfY8Y~IZ5fFxM=zlh zcjgZ;zRXiT7=+b5%EO%-^l28?dm5SWqMRgJ&0#uZ^hhtu6u{tTgYO^-tB=?T9iwA< zPQHO|%_CMSHSb;GqjA{|(;xkUa=%au9@168fyPLw6Jm+R%*m2S!?o0&n5St7m*9kX zTjtcbd1FZ}`0d;f70YElbx?cdThn;DD-td6Ki>V4-TguMk-2~GG`%L3dU<&vsPWg$ zU1ltATcuZ$8G$2}Z66Fsz%_yB6!V*#SB_rou}#0k;vvRmAycSD%o^lCHlihp`61%s zR#Q^6M;aUhM#0PBV33OEJ+YdDj&PgD2XCiIwz8$?OxP!~^WbY>AoeC4W{td&N02Cw zEdQlOMHi@|;JKOLjvjb3J-0K1kru}dewe$t>r4l=IzX@5d^Foj!2LGb=H{NWk!-#d z$klBq>!!Sb5A*jHqb?KVWdgZ6anR6>TH{!KEg;Ab?pT#efL+b&}27s@SLK+@f+G4|M#}gIF}7-z;jW40?!3 zuKl#tR#5RE!N)+1XfN~1UnmE?HnTrjzuy&ZwE21B`6%4@4)@D4;uKbVtGkTcB2OUNssIm7+}1%ModT<8jss4|`n& zBvs??M^$#Izi%JtDD`xESzV&sQ0)mwG}UzF#Ds2Rbt1We*9Rz?giMZe7`vZhsLd*+ zO>QQ(ZpvV`PvPr_5&l8>aEuPv-HGUPM{3D5ib%j43Iu@=fd_W}6O+fpUKw7Fi0HNS zIq(b8b5HZ!i$L)KLSrNnq=BV4I?9dSdbf_=^R`7%7 z9x5viwEg#qEoxKI%&&WjVv#b6{9Rx!H#uTJMta0EYWCSUV_ZF;@;>$b z#>ashugEa1FLZSA6ZdJS5qeM#`5But8W_|sY1L=2t)F@CU1RF5t*N;pd4I^)59$W< zbbIXFR&y+loNOTcF|jG>*4p!OCuqxy?cRxYXQ|)O@I5 z+4Gm1xz~(DyJAW+$%?<6l%IOUczJw|d~9@|!8}jnlxNxET7QWFy@F+}tI`R2{O4%o zisfmDJNwOLtB*!B9zKx3w^3jiZuz{eY0y{mlUl_T{w(~a;!a3AXc!#7~(6>mOE zqa#*x3-6U}KBLW>xw$iUteYb&{zB`OJC%p@Ec1%|5!=daXrumj{S#7H2CBtu3KRy* ze{AW8AmaE-9&>@Kgd+zP+Aoa98~Uge!^Rte^FC%Cre;w}s8~g+(kg$&aKNxA+%f&4 z%$PMK)$(4R?gIPd5#iWJBKWH~VBmPgxp+a~(U;o_qPsyr@0`0+A)?lwDG7^=toS}? zr*y6xeadX&f#Ce<1H8?{l*7w=a`9Wzitrk@B;J;=zo06<`W$rizfRA$zIzMNWpq@% zQid5n!FTkKzD`5aE+-o0QXn!O3HDtlRfK7hbq?6|$7Bmm&!I7ROb=e^$~;LTna?R3 z3@jUjQ;a+cnpTAH9Qt-Cqyqt&?6)o=XGhLaQ|iP|0aF$Mu7Dv`Y!=j$J{f-e#X;2n zT^hQy|4^?z&a##?GlA5R95xyW4MLzrbvwtDyxM^xx*{=+KvWQ~0fr8Y~ zN64ZQR?RHQb0fxrd7DEeo)mc`7ZaVLL!J}Yv{B%HD^F{gMc)*gn`RIP=wHp9eh~R$ zetr;ECMH`xtX0ZAV&M8DK_q^dvoHE&A@6cUF<)1vMtsX3)HB~D`bzEdVQoltlNw!B zRd262{|s(!d*uCf14O$>Y3BGmcH9TJ?2|6s&x%Ir+_bT_kH@&&z6wfzAWE1?v1-=S zxGQjr5C6qdBEQND?}(J!i?qPW`R(Ce5c3lOB0VWlX=q+sNhWCFAzUqqSJXrw(X$K* z!o0ox;k*|>-T4X(+fO3wp?x7$f*E}&NZ+t4PS<($M$)CN2ViqG8Q*o1S|k_H^x^v% z3TM0_4sHoX_h=Gji1bB+gECnyZ3=Cm*0?sDdVQbbi0u?~SxHv-IE@u__XYgFFO7Rf z*e4xu?fp59e5>Sj=1AV}$?taI=*P)*wnWl()e2?q`Iqa`dpwZBb3%Fjg){N4;~H$tb2TuM ztgMHk>3mDj6lj-W)|0B~iwErgK+p`}kSyxyi8@%I zAs?lnRlGu@ZluK_BFq_$Hd`loLB^ql?2jT5VyjlO>zFiLW$V+o!jgRO@lM@!+eQlO z-(FCcihs(GIBPxlp0hKcAjq^)?Y$>wF06=62>tlZ$!GYE)oOp{fsLX!=P$<3^GGT2SSFq58}3`CfW zAuZc@Tz!AF^Fpl-u3xSxS~~|NOVpMWe%0bKTDvpa2vr9~gMG;blvhbX8?8#*8Iv9#hf-b!fnI$fQ?X?x( zP+>!p9mb`o{R405m>$|Jq=E*LtY+-kFjdc>AmQ1h(i8MY`cNfv?2XYhq{Z)M{H&~{ zd9s~y;j_k@+inF^)u`z45*`k++&(v2Ce+x7r*g^poSx!gsfN|_pSQjZeTRXP;#)y&=McYEpQg$ zi6==2T(GJnlqp3XHmfu@npg^23Y7Z#70wmn6Fu-54v;P@O7;7aF*6n*q!GcuNq7}@ zCZwJ)HORt+wgK!DaL}PgXu_FJjnOXZ0ai(@S8B!gqfu>_>PO8gn#7(Croqec* zSmTPx#+Vu@+PcZnDUuiIa5N#Wbta|ul`Es#VcAH`i~_W5oqx&NgK*9}(#qFRXw*0}-a7;bc5+UfVOjuURA$$@U8dZ_M{{N1>yt?d?OyCJyN2OgYI z&q{oJX5ZMK)4^7|1q-N53wc(<&m;^m#H^Ohf=hiBLII!s3b3!H@WiBrSU${ddsC>3 zGG{kbdkoLNp@2k2_3MBQ z18h?79e6}W?eYE!7`ilbLfWLs*zrcXMRtkY-vIyG|rVTSA znI>E3yx7wcdSIpQkmyw@7f-zdk9{z*SD)~?KM?m3*Z!!A#=p2wdoyr1w(sl8c5ys( zyWGdxpf0BDt9$5_$wQoBcN=x(kmOGnD zj&WyDlIz*}om!^U9Owm16Pwk17^J+}8kB0sqr?FgwfM57DuXO-MBD~Yd1^_m<&nqD zI_X7*^25{O&*l(sp(IXz50*Pqw<9Q|-&9Bng7ANzTTCJZ&G#6T68s|}6}VwT0V={R zyji4u_PAQ48rlm12g@;;M21iw91i}VwaK65m(vm?O>i5@JA}#jmg?GDYgyLM$^bJ)KT7Ue+I)j`s;pDNh)hsPGbJ}&VH%3t%FwPiO3Y4}ttsM6y|BQ= z*Wj4uW|#7fK&%-~e7TtU!k9=J@}h)g)2=%9lt0`=?wDp2`;$al{`d4`R1K1HIEH$U z^64a*bmbBr6=0z7oHyij+_rSb*PRI$W2Mb}5?jb8C9-{P+|4x>z?&<*4Iy!x`*fM$ zqgZL9ZXs%d)!RIeqT4*+HKHSNewz~8kCRobtlWmpqP$u z?1ydYAMzEF9&ysn0T`OxHmnVw&sV_32)J4&Gp0y5mNZdshgV!pqWiJn;!}RlHhF)O zY-0#aDKT~)m+E3|#G9Sj6tj3T7r<+t7OUJ;9BNmjQ5))CRV&%+PHdGk?E&fv={D+S zr!4|iiVHhqr#Ev!FXToc#R{!6p(j#f9-;I3GEgp(F;^WOfk&0nLOwAIW1MNd1CuEj zM;*1&!U32I8_32#%!3DZOrIGp?|1#KMBhGtv+T~XY3{Ezojgw=FE!v?UkzB(*-uYs z2dO2-JbusD9rb(kT1A;X_v#wwu{itQNUZPr{Y=UI=s|L!ba;@+&I>85)!t=DRvz}q zC3m}|;?E|8J5V1Aa}tDY*>@j*{5R1Ig~-;$9Hlc||2c|B@O!-oK`jc`dQmZ5yexk6uKzl-Rt;Ic^Z#k9Q;Yg zpJTuIum8tc{=dcM|1LvWMPI{z|19Kyx9c_8j7%rYb~pzynojFZ$mfaE5-O#T;Nu z2!WwkDof1wB$ydT;RX-Vii1s{qGDo(Gr4GlOaz&RoKFfF;jJ}3SX5O#PFo@U8YGw* zocCMsx^HYc(w|q)HFsOrb?e$gm6#J3kPh!XnA8X)Oys#gme$>zR~(Bgwj0|O?ide1L*&hZ=WbY-;;9RqwIYr zV{7U?(gzqc-V+9DG~P1?*f48qJ(36PHFV5xa3k?R-Zp zhjwK#?dzWTB3m>&`gV0Ouj-zKK>+w|dC+%Fp@RUXe|m(=cr#IP)ftDy3h!A3XI9Kf zGfk2CpvHnT_hl%#>Jv<=>G=68e5lQoBSm$*MWjcsiDKM|!~rf^LasF>)`x~LP~6!a zFK{?)g(1BxmMq2n6MLo4%|G92OKHTa@3pIKVl3Ldh#MFSO9qqENuyasqgoT9sHmc^ zF8)g0WdSKE4!OVcEB$*xY>Yob_)EzsBmAm}h#KYG$>zyaoLy$-;j^=rS648&YG|qQ zqsqyINk)lf%c8AdbeXFD@jriFxvngx{ zQbC+}!(P&-gO%3NSIk$Ck)_v3Nyp{Ej>#;?mXa>ZK(%pW>cc9x^baXVQRve5>9+{5 z5beo=N0Q7cW<+JLIEIUY1`i5A|2Z1n6n9CNtfr&D+O5kW6 zD&&J<=F;6h>4V6bG&ww+SFsSzURTAne*==35H@dB5ov)5uI5PM3cdow63dGXXWEPt zh9RrXydVFv5USlrt=DU19LeAS6_gWwE<8$LUX967Y|d--q8tvD&>!mpiof*4A!SmeTggN(N{N?s0@7OiGUJG z=}`Qkv%oyB#Gi^KiAHNPRu5qBJ&zOuyR=`VT&pvETsss1&=0$0+laq9hfs=BN9#b4SrS`56 z=?e5+b?}Xoe9xXXX8Veneg_37ci_mx+YcDV?#YV+cJ)yMHsQ1endxUe*?{W{mZ+kN zW9H9^{jspoFawc6?CRmP9xRhAL&uZ>^J`I@hUbJR@SM2(9Yq|t_Z|aB05mr>{x1eB zeeBP8NpgAI3^`LS52k9Zzm8&`Bzt!2eNlIn=|&1sf=p&2D=EFrD(v{i@XwhCP&N6WcNs*qy*eE0_HSM?)5u##(Lgi(|#ouC)V&r2H zLZC{-#h@ThT##c##3)Kbf!_GSxsUjn<9=*}5T3}cJHp#%2)*lAf4_$lE^cu`q4l1a zggh!XR?*C2zT$DDb!_}a^ZG`ON1m3p1_P~m8B4E#k{!A8(w{I;wJ<*aA6Nez+({F+ zi^Cf`8{4*R+qP|cGaFkQZftC9Y?6(=v9+;ne3SfgKd0(_&Uvf4X1WG{^=GE9>AA2@ z_2o@1Y?}E)6=7!X^~f&6%AxRIxXB}3@fMOpM^yBZjS2b%^-3iC5AS^V-z4T=0l%Xf z%F6FX0q}J-$h%;Dvg^@zvGs4|rT+PxI0llI3v1<>zuJOMjx_d>APecs+jH5Ia+Vzn z_N#uxT{Irt&EasCA$6O*uFWX6RhGlmrqt zv8*$k2K6zK`OMRQQET#4yk#lC6TE0|PKBvOyyauXn%Xv&yh2RU-o2S{450N>So{7t z4W1z`L9S`le`CuquHdr=@w8*n$`~n5#^J>##~8pibtT3i>oq2GNi~%tIHG+;vmfi= zchnvezoWmNq`YYv90d}aN)4XA>IX!5OQ1SNBzvXV%rQ0DW=Zi^he$#B#w2seb1Z~$ zg|tZ)G3ra$d(KXxJ8q_O0oF5UJDS}KvicOOP}r-y{;9{7X=SDOD$B7R^*rG-O~NLaapv@*rd$qS!K5LyVq&8m%$?}3~(ykfc1&ryS+p9 z33`~oMo{25RN&BOPCvWQ0jo%2IySIegV$}bZG`KNkKRH0Tqi~PGj<>wK!i?n*{xL^ zpjjTko8Aa>mJJo`$X&md)SxYMyb$aI-tGgwD6h7Pb}*NLwC3FZD`bPdt(qtP0bKi6 zBf)HGf0oRvm-DIzEYS3i-uJu(bA4O%cO6ZXMWn3%qHpaG zHnTvNX%`7a>H>oC;{t-%e>Wl-cU{GP-nc5woQ-PlU$q5yeNx?;EF)xZ8*uya-U^40^9IUO;BdhOK;}#ZI^(tk*t?3bKCJXs+td=h5 zRbK)lK9dtHzt138df&Duzh&R%T;}>u3VI8Jru3a5mg9hQQ*0(kmWbys_e^fLD_I01>IA2W=a_I*t5WdA^9gdT3udVwBWnH@58agrE_F2A#-DV*)NUP;1p=fy>EL$I9*zpRs0{DJkEFayPqclHRwY%ni$^p#5R|3Lzq>!+(dzBg3H-f{!WaDk6?o;QpvS z7N*Wi@3&P~3{@^pZ9*%!C-sy|{@Wxv{kUJ_y^0Q}n+0FX5=lV92$cP*H~<5iSq#+t zY7a!)PDYr|v8l?3`cKhkuGF56_J`*xNi~R=cVNlmLo-H1lp#sRKaR4hb03iJIu?Nk zMBMSzm!D{JT9OF*Yq5r(GcG-xvhuRh;nOAL$ma1@&*s7}6RS1TqBSD68Dw~Mv$X%9PIZ@?(Aa6w)Le=m(Gw`ZL8a2?q@O~f)61Uhi|S^SLiGjncj}G&-oEC2 zT77YGbDlLhJ_w4^-<876%dAJ?iOxCm9zv>Qw-svtp?K;19v*E#gpc{V`7d{Ciz7hN z{w5dHjU9|ZodbPr*UYKD@1WHk#^WBepHrv9$ZjvR>h2skTjK<&rLl?Q)(eauF^2G zG>eVcE;@u09T%5SY~9H?RBnlaFb?oHQpMBx1%Bbch@qhsx4nm=qHUA2Zb{Sl@@&@D zpJs>Z&~ZS|yqq4G=d@ht$p%r6xDjj^mCAPJ*z9y`%bWU>#FMuDq@!4t?q4%KYWV=+ zpIJ2?ey3ZsAHKB~`*J-n+*Q4*|8V7b_61pr%i=O-%Wir(F&U}|&ye80ZkYpQft1^t zAU!ci>2}`9zs?jOEN#jFf>$4(xU40KPNr?7PFYW~L%zgmBqo zLyLEkZ$oo<0NYY9 zzB}Sh9jtvqxaf823;Yas$b*7Kon{8fD=H~1Y$Z5k?{wSFProApnJ4OZ%@F{~7Jk`~ zqv;02Z-ZLFwZsftVCG@X85Zno73Q>kCvL98TOQlt$BV=dj(ssL#~Q#st_9O4U$`DO z*7;-b7IZMXTs1ryzhI}OXXi#10Aiqznjvd#QjUQuw|guBG7a?dnFP5nHoX}p!+rjk~vx46Ii$HXv6 zV^zNzkPZADu5QUXyDB$xPL7;y>T5deQ3niwLC*z%eF zi7!NF(u4sTCVaLB=p5|G!f`=kCgpE6OC$`vjj)%T@(*8{!Ct@}%SBM~ijAhzhD1HU zv}*AqU}O9RkD80=XCnHq^t|G$EQizW4~s0y%nkWf1L&qERfi`f8+OZ_A=WA$Inx*l zDR8PD8GAmc27uTuxmk#Zd;)(}$Fu{&27q9f5f4 zLsh-(;lLeKQlgyG8=aL`KLKdjXx~l=9kOp(8PY|yGk+o*Ow45+P2l#qNq7x9bqzWq z<+<@j$=rF(?0*|%8Zap`rBkKLqbsRYJy&hYo}}ssS^@jRiG$O-GUwhU56LNnS^c5% z%wLB?2!s0WveI_JMd4wES$|CYZxTT29Dyj=s`P|-hY;fYAi#o{$Iva&(F*DW;ipPVs`TDz1e3N=sZ!g~vG$sj7qt zVl6XX<2J9FkD#ooHvdQ^t65BKa6eTjmzye8OnbMfDhVv|hm!x{oU881RUbo4S= zfHk4;;E{d^kLgOcPq>9IyT5p|{t9+4l*4bt4*=}~z8fs*$?f=H4Ld!&clYUiz0!w(z81h+(9pSCaa3B1)x zl7c~-^yOw%mh05{I7`?KU+@WgBC0hn2{%I!r9LPLu3!kGH6b_c$XjOu-MXI5uq;xu z;uG3f&XJ`PsI_nR859$$bvFh95W(L%ph#B?3}fQ&KSJFx${MyeLWX>T)$eR!#mBjm zZ(4mQW}cpo{*J?Fee!J|p0G<*U6wbUKd^--xIBrUes~<(50y_i;zGb(ov0i&Z%1Se1d3)e_Tg8Tv{DViP3bo*T zVh0^4cunHS(a|>}V5=gatv;g;4;gCM^A-#jqrQ+2chk1@<$6bzaiJ>bX}DTj*!v@$>*zW^w3j;f2xswg3bvwqoxJru^Ib#YU?$ zM(V#R>V9ZI`t2OOn|1ef3*D zkEGA-8>7xGx7*SRVFE+_$fj9pRr)#aq6+EM6~;n)pY+Y~&$UjlEV4gaw+g;;zLx?- zwr%cl0=bslOV`-}K~laD`>~R9L3i#SVEk#tnjk)@(HFEsh<>!56-z!zi2c2=z!yx~&2o zD}wgCnT{{-EmnJ;dOFjy`=EwSi_n6fehcn0yrcMc_;~BmaT%aFwn?^y?8w$OP*{H^ zC%`!azk7dKVyS0Fzk}Q9X^|=@ByjV_j5d6JwBh5fQVgAcRhSGRUhgyco%JlQ;n;op zu;sDPQJ~qG9{X}sZi-EuNi%NHWReZ91e8l zhKECBrq(W&V>IB(v{tuuI`o_On>fn2kKAF`7KUMec0-%WDQkCNH$9#E>;ZJ$*#hs} z6u&OiZib2rAvZ=-zNRRKBRtb?bg-8PO!1x{5|S63J1W)V{Ml zYgjb=8rjoI8m%Mq);id8c)^mnyAC#Cm^Hf^Nb(~ZY(GZHABWKzaS*Ts`y}z*T_}#S(8V=Cg5u>0~{y1TZ zIu~0N7bFf8J>q1Ic+3wd_I|3D?=tpoL+)!r&zGX{PUI?v2raohN({_Tb|bb_e4+iN z3Q4A0Q5x~iR?N*dPepcwi7%6%;Pzd}_00w=g_DDWNL-yn#F6PkZ2u1bzh~yZCj<6V z{VM&L2L8Xg*#F*wpCYW{iqE&^<(L2S{iu@pX}1{}%P|J~-`RJU0;BXmGjt80B~GR2 zPE52H8xjr`p(1!O@Sj1r#c4Z?e^$P*yp1Yo>+6+tRHJQvA<5pBm6O+@uT}0!SKgoQ zk1usGzdf`2?@gQr%6F_{E88%)y+Zj>xnP@rPOhd2`-*v++zd24<|TU+R_`B8c( zQmr#D8?wN3Wy3LM#q#=R<1-}{(zgf4%>A(X-ER-c0T`z0rU*!3cqi}xs}=^vBF3uC zh%Cbzal!p#lLeWzymQ%Nu9Af{uU%h9eIp3s5A*VUzbG*eJ5l3qQmkkhEw<#VX=_Z) zdL11ss$8`e3REPvp&Nqy4IPKYV`qS1S{p<6N;?34BETbBaf`Xnd^=FOECR7AZjW4) z7>&?Ilj{O!kB~|IgYZE~<4{K@UiBq>3|E>UdiD!_(+5m~-c^dLZ>}%Y=??vkyg~m7 zfg~BZK=Z%<2K>K)>-e19ME@Tl^nXLDvZj@zlrCt4t_aZCFG0ycX@JBG>KMmbM6Qd% zC4Ny+NyaX|=Q)*mjPlt(K;aK8|AxAqDI zfgT8Z@Q8{N`#cHxu+(IBrQp$Z=3UWC-O}gls?^WY!mGlw!v*?%mc78;75m})_ll%Y zUe=L~$ARyykRjI!a@*PlM37|DE?K5sCq8KyrcaX%YuTF}T6r2#-)?=-+=~<>iTGyW zT5-4<%eD?r5J#hzC`8pYrK1mLvcj+IJGa#+)bAZG7TYgr{$MNol4MPhVz$DPy~rSz z#cTXk_dXUApfBL5vH4T@sPYx`J^2^)tXEf0l0U#t=HRagk}{K4X4CB#s8?U59vKaCM36ZmWtg=2auN)GR)9i8p)i_lj{X zhzZ4k-cmj)P9FW`l?;9jgeW_Xts<(qwoq**X57Wbyy0 ztIzy!@BhaL<-dw+J+(t81^*w+1yg!{`#|;Oi=a!oOCt3DU@pRRk|IPHzyKjYV{k#3 z-J9FHu^n#oM<^tk0gFgOmWYqI2uUbpA>7m#aJ>zREDOq&xZn6hdd3TO&34uEbn`EZ zO+s|}opRrrv8ore&R6(>@#zBP2QSNg7T}cYmoy&y$?iHnEsa+2&lFR;B&UG<9Y<=G2kDO2&Xh0^16!ki%{HUFk@@gIJq$K{S~E*N^SlMst9<+{dOgteC~|2g0?j29(lGI-kQ@lHPltb`<1kwF{qmwKTLYiyc$nGWg8{Yu8P0p4x7H`C^^C7x}&w}Lnx z19|V$$W1@G$zn4IO#I_PDkz*U&HORtd5X+Jl7qZ=4+b5#yuSRGD!$ZQT^JbO<%52` zDfy&&^KL(OuE9B95v9Upg9N&NZr=0x2&~y$M0^sX!WZ2by89@%~=kt~UP@C*7aL-r6$LZjnJGjewa^=RN+T3wgRfqj1c1 zOOfZy$*LO)gN%1=$tWw5gW8cv`nEl_d{2&ae>QqA0hb&8XG5hx?mxr58in4qW0&S8 z&bN|W(l|DMd-H|Yr%z)fKs0wGA;I_CBJ;+?XAUQXk#{QWsX&})OZg)4pZMyNHS(Fd zT5>xL_K_1LPw&@$#L^~6G4-40+IbR`v)ow*dOe8b={^!!o+5E^(>SoEn*=A|E(;v};jYES1Q^eD2GBw*PLN<5OBZ&Vc;wW2_avmn z1)*`+E!Gu!TFRU2lP|bJlWTFe)aWeXXjhX+{2k9)J0kivGDCdy_y`?LqoK*>Ee|s6 z4%)q3WOu3 z0R}qsiM~*aT`UHCOZj46GPdcN?MLqy{w!z@%(mqAREHBr*me;Yd7DOKy@x8q-ddEm z#Ms56rP`=MWZc*)l7Fzt5=oRnk+ogv5`Qy^VWA5Va2%=?@P|AGm7!NO$PGM=7Wn36 z#mmv3@n?dY(!bs{zxF#05ZKbrZ=V_-0thfix>~D+fov=@)UisG5RS+L~a1LQ;_3*=c)_s;;nfpQg5t*;z$ih*^wV;{&~6AmRl;YzU6L z_sPk>vwvUQ+{TYYjLYKaYlsDF0S;iVnPhtzx*J~mO`~TADfabZefm09`@k{_;TE3{ zy6IP^ZoMVgf6OZ1?Hr=WLOst)w&Jy1@5&IHqwmu?K;>Plm z@>(b?9hnt0KNE+rHZ98x0T}?=&4FB2>~YEYpB^aqC6n!opX^z+?7s&rq%mz#jI1e= zkU9hhk20#pu&@e5=rj@#9;%z*%~;5bP;6a(FBYy*2L-2Cc1^BVCF4Pn&L~*esV+0= z;o!|@-BJI&3ypJY&D2gB0{s^1_G*EW$DiAYDiEZz{~G0axokp%ceL7VMzTtT2kSLfGgOCopf^=#3tMvIymeMNr(7B{r>bU_ zXvZ~dspD%v7J2NAAHC;%cK^72CI^-*A%QsTgx69q#-#S#Z*wmCv6x!#<9*`tvj+7# z`i^O1?H1%>Q3jZ`KAZW9IbLUJ!owZ*b9z1EERL>ZO z4@rtTo5#d~qWqS&=wAnu#mUJ@9}n|+1uqRl*v+05f|63(pBd^Zo#RL+blh@LZmZy< zNc{ zWfw+YC3TKPbi({0Kf9D_D8sEVaRIRb0(Cd%FLqpbSm6#f=e-W04RPel)Rl8ydfN`G z5gEal)rZ#&$h2cvaYVNEu(df#^7!T1(Q~xdy4NN4*ihXKyQu2niR@l%LOmQC9aGhN z*1+U2JF<4X7GVBRN!o5|)!ym1V(EIaV69Ig4Wk4z2+&K?d^XC)D=e@KGSM(jFw3){ z@#0^H*LYEEWTDdAuxfdiImx}|V~h&+#&I&=J1O%4R}Z5&{#94pb!{QglTem(xGF zxLj7w1H?olITQ^o*5Bjlj~v)n#YfvZgbz`vx7yJOXk^Pta%FwjRj~SEuMqaA@g6xu zy#MNAr?_4+w}lf>&N|dKRI~fa34KKrl9>YbEv$Q=2z!b12{yhG6c`t0Wv&q@vJ3M} znhL1zMy%+>0Wrd*^;3=ZveEE(Cm5;1`PaiM05WtlZs~|RTo`JrR%;jW`6c(Y2cFK@~?wInfqDTnuY#1WIi8x^uV zThL5QKe0;`gwrUP&#{-WBwyv!^y**17+~6fbk)?FbI&@8NWCrcyO>ctUxS?hP-cDu zFvi%jLS+x~2k}lYulj*1R@;gMyUYD6nRhPaC}LAR={~xL#6iAse?*F4P8CDinL8t2 zBxM~%y1BQ^v(aFWTe%pabM1PhcN7l{buog=*xC_ooAD3sH87I%F!K>juH z4L0XH8t}8Xrfr7W&}W9?k?M~O>$*IEhh1xUav_=Im!!75!8yHl8b2*3#o#*W^^wj? zS!}$xjG-kTWNvgYmMb}yTXs!+O>FOgm8B>OOAPb@nY`62cptb)ym2EOiFs?fSRuI7 zx3mho!iRsRpO{Beu#;(-DbqhN7v3^YGRrU2ynYU))pO$6&&05EwgfxQF4F;YT8)Zr z0|Ym>`{h}wq}RO5IAuLSPjN#R<{^kwIW(d%gCs4nA6n)R z4)O;OlO9r6aF5BI(uAEv&4yFXzU+Pv@TY}TE2?-ITcP2W84Q= z>gXh;Wv={~gLRI^ooyAqwH&ZG`ET)wxMCEkc>mVs=Cr^|aNeB#RO3019+<*bN6xv^iv>e4xCSDQ04g_$Gi9Y*EmOO?9Ui+Y>4pN6Nx3o_+&rG z`KMP<9-$#-{NDT3@3emd;u~2Ki&vHuB^6B! z40u;gQzmoT+NRT|_Ak`Q+dDk?op0cz4P1jIPHh@M>7w|bo zT?M{xHZ0I9J6C%%y>Co-ZyQGTF;g4XDUJ=H;CuYvadFkmUezM4 zJhUH5L>iRbh3xR75xSYj5H9dRI2HwD9pD@N0GtDg;{j5LLP&9;=WEdr90K%;8I=uO z*h_uw!k4)E#}}B?p@M^7!Yz+nzh>=j(G=PdsdvQofDE26_dcv#>-wh(^zT2gal}4ForDX@*aR$jhgX$eoRE}E*OWag- zwV{}Rf*D13$C>zui>F$~v`312Bis7T0qSq%dFUd?1ZFd#92YeHL*OIs&4J$e9cM4r(Y782RFQysz=p5n z!1jB$ENtPODYm5-84!onE`LeM5C4R7c|88s_pVXh^-yp6S%b&+FM2{xe;Vp@ZABF2 z+Gil0KR(7Q_E;)ISk5_}ONN#IxuxuTkdRH2E_!t=X3t!C>7AWO-$p#2* zX_6FQ88}$GuhH|9*@(yc0vIGt%d$m^MUOOuNiRL2?Af3UKFAqnO>M^YlwSYJ7yzfI}hf{P&PSI9T`VVQoj9V!Mz!V8?e#`g! z_1KG}Tyaj4avtKQe2?m)B@AWm9c+gWiT}WB`Bkls6>0o>tEU0oVJpPpN}ScB7v~6? z`%VNi8>Xz+DlLZ5>ZqteAP934?l_~IoVTr}w4HPB&4yF3hb zBd`iKv<#!8SJSN@ld7nK#L?o|68^z=OgRi+L7)X6D>Q8;o2bwaX7ecsQ&o8k@o%*e z@3V}0NkkE`?k9yZaY;!XeOI>t`0tg^2sk}khWQ%ZxU{@->G>0`&LUhROawy2UFbq2 z*wz#+q;M>7_$7bnQjP)`( zM+$_PPdJk997$}}G+?+A;4|}2iSEzxmkkGgp&|&4$|Z$YDtNM=iL7B)b(Nz-eovh^UEwwdoGqN$`MkgM|PzzLKSJ7oPD?CpibJ zZ)>q?)hj7?)DtS9hL1pkuk<8<=_)g^`O#&O{A|PWkk5$}*Srj6+bcPc&4?{h+Wzj) zPYHV< z!6o~JZKjw2hUOH|)H<#sRo28JsRZ9UmL(C66eMndTuFu=VIthag}hOaPkc0K#}*BQ z6y1y<0qEHi1#Q0z(8KMm(253v+voMYlwJRAMV+`L8Br!5kw)3GW(Qxf|1O{%oV`2h z4V!XEQ2-K8p`|gCj|l@}DL^pI2OoP%<(!DG5!%?T zdLc-^>Ko>%@i2ESY33j46l3X>e-bGliKt_oS~Q0UDa!I<&r>uD{Ao1GN)Y=2g!@pT zH)8TB|N3v7BX9np?mwdLoBu2j;6?h>cNZAsq3!SHvq6XP{7vWslg|L(_qiN84g`0% z2))73qUQzd6SMby9AFzBNf?wQa4#ElYh-T-%pWFA)dxk8`Si)*x`QKCh#FQ;?Ro;t z1ba927)_^0d8HBi&QAL4EFc6q;J08*`)<0XJz&$+D%CnY87p}q|H4n79~-&K*QJPa zPKvSWP(j2;x~D|*28D}Z)r*B%U}zMfDw^)A#M}a>0@(yvF3|debfJhudo@@?aFI4> z28{QS-fDPAYcU2AJ!uGItTfVbHDXsoru!UR;?Xa3KgiG=JG8Q7D~a~*5uu=dIb(u1 zCyGc?NZQ|v>cl&1FJGM>k=ja6kEbp?Sz&gKKKMNL+u1@eadc+XH6~(;Kx=A+jsA1M zSPTIgo3SM;!ez^qnT^UCD>BFwL*}l((PtspFAI=1;XgBk<2q11P-C{>(mBk9VHIM1 z|HcSUJ5VhEKe!&xY(+|IxGrT&*PaXBroo-YDLd4F5@(h)s^b=y)=t{nis)=(7zQnoPl?7yF30)oX!X z(T}Ohi6Z)qEVtr^6Z5#cEH%HX3!POmYY*2b2hk{(vz_sgEBh`l$NN`h46Op=Z}KIu z=-EIjOMSRQamjux!G2`#5Hvq3CQj`w50fkFTeZ}TdLLZqY)84lP|NBO(r`!y%Nem< zg>>iAQsM;4DD^TOIg#qRGs)ol8g~`Y|J4;moSU7Jo}9&o(F_BGP?Cm0+6@g7&&Y4^ zt0)gUra4g`jpn2Nv}!9aXDly2FE0nG%gY@fI-<>gRm(RvP?IgDs?EA6s= z9f%O;)(mBh(B^!vnX$Z$$))TDDVq6X0o4mrB0+~NO>>=#^#z{9Pn|e^eCT&Y;{lX% z?wEV^Qw8fwJvI-%DbIR$d8ejWzk3sXIa2B=dHyC&G{*TlXWDMk=hMjEZUU@5sam_L zZghMX{(a9=c^n^}C7B<~$Qsj%HVUmy%6t0EHvtl_iHD zKQ}E;BFlVOGCyuKKVDRx4nUNre=ARaD^LIIh?l4L;~rb$LObU)LeUVp9^Co3p8b6- z;5O|V&1I3~DBF%y^5WwUNvo)$fq>iNKoQPk-3ARdL8G|TB=k-J5r{&B9{jTD%H_=gp+!v7~vTD zpB1UE5SozWGPQm3g4X*{Hn*Y)z8|6I+UZ>LO{5?Cefal$KzX^q(5<&+52?ehvlwk`|Md66QX zx~wh^cETWiw_-}TU^DkjZ~yXl8^u5dZhd5*^Jp=Th{C;6>UXM>dMvwi=UQii(=D^Y zc6=emhZOP07i~0YQvrTx%`!XGNz`z!wT`i4eCBrPUc$7~d zgN$m8{*cPw6Nj7!(ce?1De~awkApQHtaG*+&XjLSDqe9KUPyFnQe#wt$sOk323a?)&(=O?gv#O_G-bA;%F4=$p{kMKEkULMV`=#-KU(Mj7r~U z4^9$s0qC9I*?Ol=ews9EmrKDC$fd&FSe9qZCB16VY+L8|Hs@nhv&7ea=8!DFv5WQs zYE*4wDt+W?aJRcxlgdHfE%A@3e4E;o+tu!!SPjp9Bis0<6#qgE^1{}0vw1RvQ;{d~?zO}h`0gPyZJOx^?7CJxI& zpZ!9gp+kLP*g%q@&r<(&0zZFE-OBa_)Fg=j`jZhc0#GrqYgo0uNS6{UmRz%q$Q;<(8j?XA=(n*^njV zUkIwj^mqTxiD0-&$@+PM1K2iPaDk1cMP|rBk+p2JSEWQWca5n!0wF$?foelA{VfVil0k|at=T+pIc&}jtKS3tZ99qV@=+#QHr-(n%^N(pV z95z&P>JGLO3ce_3poKVu7_?AGd;6L793N)?aH-{3MI#jYE8f12Z9Y1Uz4(>U!@zA0X8t{oxP}v6SiU z`+Uoz{f1%yg7P>a&aW}zV6ZQyeWc^W`JcvG3~n69M}Bz3n!U?AI>(k^A5r4G3{zl* z^Y7`0kn+FKu_B+8Jqlr|yFhTf5895$DVC5YsE&K)nSsEtpELB+1AuiT8}gDFE440_ zq-uc?UsSj&T`wRzwv!Y3W9$1ih2xF5NkNUJ-a(+QcT`p~2Z-SPI*PirOk;D^7m;aSvt0UaLTlEU#QXEy>5x_Z1(uLgp;w|u)=$bXa==&BXj}Q zzrC1YAjHK#Ct7?Yg8yWGwo0wJbUuEu3xS~IB(RO0D=xK9n*#Cl3JwEGV-WdS4hw@8 znMTX4K*N=cfOkuagGUn=j6DThr?}0ef(=cJHmpvIHrPE?wN4iog0uuts5&-`9+Ue_ zg{p^9uH<_?0wy1HI*%xpm`TfS_%x3VmK`9BIPs+cPsrg zO0rJW!F1J>whAjSl`@4DCVMbO6Km25s|+1yIW%E`Jyr591ddp=+9)jda6%Y*g)W9j za*lFMffwmaAp<()#Iy{I`fXlqwnJu??#{1Mxz~{2Rgw=@*T3MHWL?lAyl*enmPPI7 zr%K3xrkmh<8@?P`XHP8eNHPH;a?6%lRt2^b(CKV2W2w(9TDp-3ChxGSrr%XeU)F=VRh8--tg&T?jD{AQ zzB@>mT-sj~1>tTko(yBtG&)GO-265%13Z%)@v1HINGq9KkAAdF!Cx=C{48SNcC;SsM7m@uNH@TZ-Mo>JKp_ z-erWHV(GuO`sDRiZL^$lsjfE`SP~03$n_7V6gH8k5W-qgpLC2DTelMQBbyJ{mg{Ve z#rNu@e#6SEwa~^jZoI2UK<+m(2P8OZq|w5b9q{sdSc<`)>l5duN-S*z@r$6K_O0U- zlca6cAsM}iS6neS&&u#Q()wLY`b~M-w!aX#eRCHk7#i5pf8DN!xZ>dig27xpm^D=R zmy#cbwHG;YgxR(w0DA8Y8O+25 zprq0C2$QOFaqTP{w)QjNgMv}jjw1vp!HZHHaXed#FpXh9X;uBlW0ZA9?HGL59lGrV zg=3g?lI`$J>OwF}^xFM|W6dn(1~3V9OZ}+(&~uOvRY6CMZY2`lODYbj#E0{pNk#VA zn&@>qw`Y14UAHQP^BeoNKy@HSZb@Ol#2|mdALASwdSmdK0NmaFzdWAnW_Zh%h-YR? zQRvDz8Fx{_206QrM0D&)vsAS1)GbTi%`p|DL0??WiYC_m*}La518UJj56Y1Ro$?%1JH>1c*NpgVRuk-HJ@rr4r#;BVN-hf5icns&(OQOtoe*9%WuifV2+_pI1zV4^Dq<vUy564^(hEE4M>IA9zSW(R%Y-vC>FC74}wc3D>=%c(V|P6XO$>SXyfd&T>#e?ByVVKmSXY)Z(+F?ZbsxZ{EUJtN z!9sa@1-M0)squ%o)K?~yFPWjKMY2ZjO#8TUo-fe>(ra%pljWD<@i)vg8bTu4g3%AR zUG}8FV2HkOu`!YG;nw|#N%6ps^q!&DP{tacODefqYl zPGgJUj8m=IpBrQ5TNsB*NAmp;3|&m;0?pVc#~152m0;OeJKGbV~B zM)-+1Y@AX!?9ChhP2I0B|CyC|#nwqwbG4eR?emLqB`9^ZyLY`g)kXJ*MVlJGh`9aS zPRas8%rQCG`d$}uwYRos_zmvW9ODeGd!}uhx>A?Ai(OL+wT~DNb()t8$M-e$7{7%C z>$}o&4$GF>#~Nj75&$Lbbb;5iP;$s31WVyka zQTe<{1JT1o&YX3HjwQGvJUW&=U0{iWmJBvFO!FB{H+#ZUmwpv(l?- z(Ht`(+qCdNUfQK(liz3#GZ;o`e}srMm;LVa$csilPy#I38oHA=SVbbw?nTuUNGbpc zXS7S&@@uXZyJ+EK-~)v*2i*VI&i9FXib3m8`8Y!s?;yiZITQJd4f0(7 zb$eB<|8<02BRLoUtgWDSiO^c_aNNWZO;HDKoo1)WpuOX%k6jK>Y4JUxGmegSXoje< zi8xlLb7fFonm?hN?};< z|Hdc2R8A&tROtq;ea5GMFL*&4{`0oA@;PZ|UYKsz)uGh`QC@pdv}}J5kZz@dT5vn> zGcr1fERB{;H<;R0)u3UXJzQ)Py5094f_yV%%dbuvJM=UMYnWdfP-{JgNiDQS5f8;H zU4cd{9*tiv5ohtgKnWKFB2(Uwj;HpxS;WkV3rqug95ers5^s;jgO4iQvUMCQO!NQM zD>3Atvj&1x3rX(@x#N!=8(vDTQPZv~BG{HnCbq$3t$k!}IaZ{fHOYui6VpfS0*090 z#Qte#l%38)LY^dgRmiOP1n2cw>}2(ETu>F_Xfn6Iga221|Gx_Re;AUJ)ZTN<2hVOlG!T}fy+weB(Fa6PY7B-2oGSil?|WiV7qBsvmLw3=H`@?yO(hJ)~mh`3S_w)vy4QJ%3I`^ zVw1gCTysz7E`Kpt<+RW?AT%XEE)^r9Nw?bm7`V58&r$Dsl{%tOKRA(D^_qoeAvOhz zK?O&Yq{+uHg#$?M?+Hs-F$~jm+}$zrIBHA&KeE0uDz0uxJGeFO5Fog_dvJGa+=9CW z3D61d4vlMYcemiK!5xCTJ0E%Pd~@&2{OGk#@2)=GYgO&4I<>2wr)oADJgGgOY#`jx zp`8)H)I1$YSRNqIRyIZcB|PVO`0WP2&}v#G<+>Bwne>Y!lv=q-Y>+h?2b57??l-Qn zr4J{NqG2dAcG2Fr2=q3>-E?<&O;^bw?;|&_-SCSS#E2v+(9dJVtU9JNH}wtD?Wh0^ z-|vE-?sBobZZVx+z3PjCk4pcX50MLRBV~9FbN*17$hEp$JO}($?!Q&|za2{>0tgcT zhx}L1VHk3M&ESr)s5(F2{n zZ#*#PSSN_~xtc-IUMXd|IQI8VcHKlI?mo6`0*Jw9W=W&4&pZiKvt%Pe0YecsjOsL? zs7O^~+nX+VXtC3x0cEdDnqsqCKhkyUXuP&4mx`NwoNpuUrm@7A)w%IE9WqS>7wS6R zzD;bS9+ydnyz+O(MSw9jh2Dy~*VNK^-d53peQxJRk8vd)>=3uyQPIPL&tKTtFQ(3& zgu8GWpb}Kx=g09kAc)y)7ZRkM1S4-jqmC(~U-OmYc@z`~vID8ym=t!FBFC2_^;)Oa z|MXA;M4}y+RJ^Ovkb(|jUS2(z;v#Zk~EwM!L6L z9QF7cG!l~ZzSsM}LzfDCv8nXjBbh`d0%1?M9KT^-vM^q#HOrT|8*n!1;Vlh8q#RP2 z4B0nC$JZ8r=$#FZhe7^(Cm_GO>29)aKmV*)^30>6obP-MRyr^yK+2%p?S~M~Y2G+G znGa`)(WQ7m38$lR_sB~KRBq{Y@?zxz)HKz{f2cSATjTGM<==xPe{EqLFT9NTeYFqz zeHIT`&G8)) zTMX47Y)sihhcZ0y-l@K6Q3-+Lg|Ap0_K6>*8qFNEZwl<0oTy4N&d^97wx6$cBh{5QWcE37_3#4rYMvssEi){z(`t$g(z6(u(8@KPay0Z36vF%Z7_tVd z(l;rB@&O}wOZbZBz`hOha~4;&LBJ{i%%V5!h~M)Epv9s$RF$$x7qrfzH|B`kQxBMD z)dNjzm^G0SGBi3KNd^%IrQvBh<8ixuZe&>*b|maM0ledcnl;65G6l8bpBgXpJ2G4B z2n9XiH?RwIv2>m51`z>J*#KJ^cz2dE0)-c78;uECpR(H@)~SYSG@odUp)#NLqB&%A#nMa_=TI zBFa{xxc-vP>^TAxK5DuTihx^t;1qJmH=LeyRz;XcQrQGowviE1?bl@7y4L@kWRee- z++cA~!i4!bXBanHXtcL#ofK3(qy}J_T=`xEh)Y#ctQ*D}mj4+WhU{3A}Gd4SP!@}+|yd7y& z0l9QPB-sO#2*?U>YyU>QWt0c6#}s&W;i6ybM#gGVfx-+&wlk?bSPKw~a@UM{#42s> zHQBQqC1|WvqUFrs=u=1``{iav)CF2w#7_{*;gu`p8RiHq^5Lk4OtwVdZPdOUT;p`s zFw_EnboEk#>8Y!^+;!4P-%f;h)K|Xyj}clYJ&D%fTz%NbtZKGq9;u!vVxhHHfkfn6Z#N91JPs~#V5#$l6R9|3K@?y+M zHpoa?hPd|IIUAJ^;SZXGr}<*sC3+dGc4?6OF;uvMa}_mf_hYx0rz zu1(3Uu3d!Bx97(!Zt}!blZP&#VB=($h0R;{+i(0mg2`7gr|A<6gt6OxMs39ceLYpQ z$S9adq8It5NPTmvG%ZPNVzK7-(rK>cpKE%i{Jhi{acy{1d2%)b<4=&Fy^h*5jXt;{ zit%Z1)Y@)JU&i7_aR$tU$Hi_c`C$=LEs3)JwniRGo*s|k`U+f?bdy{KiN;P50$d$m zzRVA{C!@)Kqed)mo*@X{bxgtfg*o3(BUJCH!=Algt zx;>}GKztANrbT6NKPFm)wItoKj-pueh~L6^9Yb9MrxqARadl9-<7QM_HhL|~mkXz0 z*3mR^q-}PWUBjjn%Hfg-O0?eBtX^{O;13{@xYa6+roouQE~QnK>~~|h4mx&%%dV~> zsALS(BrF_LYiEBqSO-6og_@kXg8QV+UBjh z8K9wv#FM4{K2S$#O?{}Gc{E(&W^6!1mLbUjrF}|D0*BYCtmY2`a!@#C6p?e~l55W- z-|{SySw57tI75P))w5_JSCDD#u4Qo95j#QnW+AT6hU?M#ZRs2y?r61bCGtqg!{+(? z2k>XEebOy;8@o4fmZ+WL@aCfmM);$f&*6^deEzU5y~UcQwHOc`Q>nvXA{M(?U`IN* ztb)X?13T|#SXWjKblFIF%2AW(T>oAqH$B=9fNCy`M#TU!G=23{;prwijrCuMO8iMr z&D5rgegEfmLBHK;mb5p{{e^ADy#E-!gy)^pEKg12`GY@H0sa;_u*YPKE6VF8svgK4 z7Rqiqa%|hT#HZ{Yw)O2`EF9m!oSJ-2b8OK3(>o+AjViV(C@EU@MMHd|YVkqd`rdI> zy>PV8b}gR3K%hp3B}A!mBMv1ccB7r;Alud=n|VKW@7sfq$-4McH-`|hT%ERnF}9NI z7Oj_F?%jcz%*RhC3H#$8- z+J?`+14_U)$m%_EP82;vbZbsNb$5*#`Z`7h`f?w_Z=d&#vc=$8M!ESQ{s)B+ zP8P7Q(4%?sVTIiCO&kX89g*%DUlb>coO5d;Bu%%qxjuN)2S?cGKBOrtWQtU({puHXzT3CaZGNrSuG(z2}BZDH9d zyDvkczFy(0l)gsX^xo|)XH@hTEWxr^S{9XmEf&$L z=ULX`;N&x!CVnb6EgqUUR56Sz9|rIAgP24p+rYi$D$leT+*3`72;BXb?&X^{sC5sc zM3^&ezcjQAmlq({QXIEVT(2H{#n-!4>{Do}G;i`6XLHyldl^WctxqlxBCjUk4lzIN zXN&ipoNsEMhoVPz}D4Rq2Ow1@0}d0v*+61gI*gZ$EMm>U;H%P(_GHo}>p5i2BZWrxjA`vS2d zlUMp66bYdTS*V6GmJOLI)5j5Bp>j%#v#-+?KQbiDj~E$c&Iw2I`nCC~@$jFSVXgaca=_V0zYf~FsWL%tq;_`|cz4yQbH z@}T@@3o-*4m*&cGh#=cruLiE@12`Rmu2+H}ySixyMH!u`0fMfd(&4+T;}A)l4keJ#kBSd=+IHzXJsfSZeElZ}jmI@uqOVq*1ad8@vML-fq$?m1UTl93 zUtEO+6jn8CYHO1uy3^Khs_NwOuqP{%C_2SCaT)Bp`TpIKEr?W0wLYl&anBNu7^>aU zt?t%+G#n(23uU`wpa#QCFQf*;d@rI#dTwZ6iAS<|vp9C4wMj9Z;!Lko32?}?xe8@( zrGUP`2{)TJ{^DZlHt`%gmjtF1RESD?bCu?uU-D3eIkWT|UWdOV~6q(#boW6YY|x zzw62;BGeNb>qLpgdNI&bjE=s_5}l=kh8&BmF7k6B2i&%2`T1;&O6+%DJf8Q~Lt0n=WCh)T40|jE*Lcew-AT3< z4cAG#Us8?XH?F}8pLP4pTnn30Pcd{4h@D7Q3)P)r^79;Ee?ECKVlI<&w(FN#+1nMu zDd*^|#?vYs31@?Dw2t&nZgj7gas@~vFvEjiXn%DA%6Hwfh1X*0BJ<(Cdg$V5Rr`Nb zQhh$n}-)U zOr=M+!fyTVg`?J{(;tO*s^^d5&Xsok2EzR}m>!QDFqKMFcV4;G^Oa67NCGdM@9!z^ zmh6vjzUjR|D80#dAI%JvT7~9l(b#zij$tNYwF!w$P+hfL8;Dh6wC(;XWrV4mpp14) zaVpC{=R*M*+{ttwT@AG}ao>7bOuX2@2dFrI^;OB)%G}G{|UFIL*oo{ zI1ce}Q1Nl0*e0*!-ac?&Xzw$?QJ@Rg+SiEe6V(G5$1`AyIbaQcivrfUJ7`{iJJH_1 z1I7nVdl4~)PkTX?b6BqT@{(%1+G9>JO1h4PymPbY1x>w(7_$qsG$K2%82k09Rsmsg zM!L5s#Bk2`cRbuFa4oDEsE|!JH{IuWs7#$y*GI%XBeB_%T~%`S3&^I#`hF1_t9|S< zR}BD-Q5+kn@H!RS-r|kQ7lx)^`wh_!%AoNg;P37TW~6wo8<~H>WKe_&zcF$m0H)gy zZbkF4U;TgxXToF{hY61~av}!WvVFP8K?#ornEzac4uAh6h*eD(3|xdY_wnNq-VvAs zXyMnJ47Y?oGs8}hM@_6TaJc>Y6pWnMfh!=40W!v*zQSwmkKq%@&YPUE;jLX;_Ek-A z3=R0ZTN8ZE_rJdP$zn1PLx+o0Eg;~u0<^u0d$#DIa7yK?JP;W&VZ$xDwieN=?+F;D zv2beZqwVpr9SyKs<=O~tR2a!k*rBLp8y>VI9!bjY)L?K?`pen$CGQoj1 zA994J2kS+OT0)qDIXc_e6X^J{af8QR<`2(np2#sNW;F>)=M4H2pw~CR`1tL2=u6>F zb#SC<781$;*{vsS@2g9%4Q0iO1EB5Jj;&9`sjISbs_Vl>d2Eso+%D$$jK<=- zbdm-&%l^R2Oe_^&%y}7&rA>l#Don<3Qg)1EE20~BgX=#-+ITK8>>cUyGp_02%{^iO z=_kvu4<{}UF6QQXmaGuUA{BT>Bpz=;*bX455b?2ttPv=E zL29VqgH>c=Ddl30DX3l!1DHm0t5K$a@pPd5TmLFvP}ipJIT`>Rex zqt^aaN`0(FKwi@&G?_Kb>mzd%USDktvw2jBv#V5EMY?ug2`HUhN`qG_vcx&+v5!s) z96U5k`tm%%$(d?^E@hKvv0>Z2{_2>X)^zqsKhDXCVrW!eP1&aKILFfr0#v6GaG#c{jDbdy9)TrtWa;ZSLmnF5SrtwrFNP3(W zZRWAf>Z&n+%~HbU~yu*h&oZ;wZM1lT1fDuSD?1}`>NjWvuF`yGmjHIx>6CfT^j9rV?f&VCyO`*)&RHyY+Fr(B~}z5B?~ z$~ivY1nfjw=0tG=eo$qIQoBolZ!#fl%B2yU`Yz@}m!EPS@4&)_Z-cU9^PZXF?<~pJ z_9NR=1KvCh1lGA!1I0#Ud+j!~{Uj8E*65r)HaErkq?Q2Q!~}c1Uarhj@}fYwdY6LM zkD$jl7=DYg3(-(fUWT`OiDQ~C=7E#v8R2=q%bGvQfsS+) zf#Ju+IFSQN4cEmm-@mquA9XnrG2mdn$bbUr9#ftC>li7JMRIYKx(x+W%`K`R)6P+D&qKoV}+qV%`rUi0rBAL75T?A}jaV8N}y+$5lK z%@I{XDJ;QAV*lRn2xs6>goQc(Px6pF$=@MN(Qu-n3DeNvq9vgK=mcgID#lq^ET!sP zYgJWh5QSS|;kH-g55ekv;|H;1deTkDSV%XnckOrAKIzNl!iF!m7+oJ4UM#!oP#anE zjOK#P#j!^pHe=Jqnzn=Kg~K0IqCG({NWSXEDojaO-lW)_2|YC*Pi0TOA!pq z4;pK9Yaz~I31C?QCu?>qoa9EV(;6CTe;%w8XhU$O%HrRutB!n$i1;4&^_DkHKRiW~ z!t6`!H*_DlJWV!}pXFcJR8cH{HPp9fI>!-0CKl8m(ZK7`!HngPo{>-a!PX0cS`y`^ z*wL}iqf%+v`KdM#p|zk_)ZXC-o6y+R`bF)j?HAnZJ>vM?Smoy9=u_>6-Hg?e>jv#p z0OlL*ra9!lN3<*U>g)Um)TfadW6|K@WH;H8I3m--1f>ZbT4D~}8%*$B=@U7>;OHnt zY&AzB_e9B`nnez+z}D-@koU2GW+2)3H0iVK_RW`E$4^7-Y+6%$BgJ_c_{S9~FdAZq zlz-dU*?x&&NRfezkAvj2TArp&-&0!tf6Suyi6tp5ieMcDE=}nC6;%9eOaO4%6v^-z;EtX|{0(DAX*j$pmTY{3+=nCtf zdQCUxmL?(L3nRgnL8N>TDH_8-xApQQM$Lhlm(}cTR_f^C0oOi!`eJU};cx!_D6=!{ zYfo~R+-_br<(d2e!Izg8NL{mvptru8=KWGG5_%rg=p2Di8}@47<@ zqKERBh#jrX=ys8E;_=(f0_Y}UFBJ4geC*C1H+c7b7=hA>O~C-r7J(h=3*_BM)+9ye z3G+toPv-|X(#*EkBYZpkrqSvd5ft=FLe3^xl3xNjxbyc!k|Y(%OTD#A2lEZ5LPPU2ip``al$Ec6{bdTB)KJ<1!nxi z3er>)aaA|b@V~Fb(%usBaPW)hIidY-6GgA~_oUvolv!W0euyu@^g$@Z1yR81B1FviO z_GllDt?8W31!u>_<15VHJmKGd?Z5Mc|4I|yFdi@n-j5dvp~0IIfOpcDStd-dax5<@ z`09-s4@3y5AxbKpUH3^RaWAr#LU+`D2MNr)#fxD_i|YM>Brw^H;qfy?2*bp9gvEGz z%EjyR%?sRjxCH~hKth-u73K{Bdz~h;@ns1P$AMq^NOuez_|BU^*ht2p$d1`44CDJn zrvW|MO6|K;;N zocBhoMR}$EDUq`w0l$+uKmzuO_s^4lG~AC*Gul6`P1EoSU!-Gg31GFsQV_=AC#h#B zcYzAY@$Q9r5(Sws{Wa9)Cws7x2?~cm(65dtZ!LlNAT>9GLDIw)+~n#Uv@8s8BXdKvQ@!e#eA%4ZOT`i zF0-nj7C9o7Ou<>DKxRoc_`r#ntijN8%Wy!X`O#S<1$<>34T0^8AuC5x<1}u&Qblrr zYAI8+q^b3`vCC=6YaY#^EB2)k5RyJ|6oj!K^mD(yT7C_C`s*N9rf91RU`|NFnqZ(T zZd3)pA`$w<3?~o?2xl^u&FaOJn{EVypjrNceGGlTe62MxYr6+%##)m%6%Ti2mmc7x z8;ld6TIasZMBAk;;dxT4l^(>_3%jzGxfx-VJPZsxMC_h5`K9HUI1X67J_3d#Fiqc@ zl-wi#HKBhS%KtW@v~2XLt9KJ>MFgvu6X5>SuF%MafJ^`d{WqpI##Ca8L!$q^>?Cdb z3jZE-InY0RkOOZHQgeaWkQTgDms!s_0xKKM1RKJ*NzJG>? z$V2$l9S~f$d~iU4oN-#+N)4fc?&h;u64K&Vdhg5FL5Ws8{bHW@3ik@}1Xq;BU|qX+ z?TSGY#3G~pl+!xVzUOnwyO;U8*}3kGWW+1E4fJbAVCZWHoXBnAJO}20p70)RFbKRn zi>YHIxe%ay5Fat6O};lOP%TU}K_LL~z6|by<3HIdcE4N>E-c`kHOU{c?oh`Jpo#S$ zqEJ<#p;9>EpA|BpFPA+&Nbrwl%l_XKLeHHSg$N za|2n7A~j!TKgQ`vWG}fu=-fzek}RBcP+EpXw2@2|~4oM%vY`wBo!W_JfNAF;`Ye^6~~L1|z*N#3Rd@b5f^$rwBia z**Nt-kp;rN8eZ*Xw%Eh$6|w`o#IaE{d^FADr=LP`8ImZ z#mhy)t8xCNDAoE-{!E#UdJKM7`^STJ54%1&zY}O^W_E86p}Ohk%1@C^Yv1gYSE(e? zY#a<)V&U(LNFKAuX77W69OUX9eSx&M3vzTeW60J5U>M7;hyFHNMxs#E=sA(5vDz>O=gjN zDL}_h*eNyUIl4A{Waj+fIeLK51^=UX{IMStgg&aTbp@Mgf?w=esG`!DmYsU6?FU&+ z7p_nA@S)i6_TeV4sRB?+UWY{vWbrFWoPsQQgv2@BwKJF-4bSYk(TG)~(N#lGG%K zf<&a&8?tmK9!Q_n5BmT(jT@L`%<2((5a96q`1B|I{7)$lgVL7M1A(G6fyAHXd&Lfa zl0U8YGQI(CgG%S&&sd$q6D(B1sETI>y$9cSXRccU>qzqVqCSZ!X43~KVish?amn(! zLbz`+@tyg2zz~TjzPzIrM zg(90o)art{Z_6!AmpytOOjsoCE zBrBhgIk|%;bqc?pduWn@t+#*r!u~U2{<|;iFJVn1;V+I7=zrm|&z>?lK>h&za0NlAkVH)Q(rl-ks4B-!b7k-fy6n1FvGYoqRJOn31@77*&FE#Q{bS z-SIHJPPNyoY^#Xq%U9}H{IJzB{IYsfVh1bc>3|+(zHQRK4aYg>TQpmuycT*uLgSB~ zAn(+E$0k{FvxUJ01) zto^j|wVq6nAIboNUiDF1#hH^5tGwZH*{)ol6^?>{v7?vk5R2Dgm70E#uReg!?zuCV zI$o0>#+EDZygjo(op#^eOO8)N!S*K@#9yJgc7Sq|c*8fZ7_2%mKT4Uo zWzrayE$t8n!Lnn7h1jzA;bhj+tMhuw0gv33PgWIMG8eyOl=z3nZj_OF)YM(5I&8n? zSR84Dm>%T=!NFj{?a^m4D~dd3WVnp_9}LyQS}4Ze5<#0Mw4>8f{j3bzBlt!g)=;C{ zIQEzuC`0&5XE>S&f|5!!zvPtnaTqIlSj~;s2u-`7au+fZySD+SW{8wgx z8Q3-V{yVT>J0naIFitZjDoAz2e&+K#wE)l8t(Lj~BN~L5B&|3YoCyULZ7hwjgwGN= z(u)emq$wu0Lz(L5O13dYVZd_;Z>+8H1=;)(j5L1tsEO_Ih_~mL8^SKr6_-t;9tk4u zL!i@?WTD8Qg1$X((@FM772n>&J3gaTT!=Dj6+(|Bh0vu$S^x705VXUa73yvH#Q`RI z|0$;1mlU)3D_+^!PmuJ8Jpn>ceXM-%QQuzi;M#?iytGR~kbuxm%Ip(H-K^d&K$koP zHdOi025p=-f1umGIx27rUCTQyXD$ukCR#tAl`MO3D$+72T1{QAeU!eti7vcLtT{Dp zHq)IH%-?jB12Nx&_}*f9`sR#3S1*Tl^#!Lbg*fF~j%})`OR+u;J{+40nVg4S%^Id8 zH^I2Xy_N7-e%t^@?mA)A_%n(+Rs7;tP>9quz|E@&G1ZIjFxBg`4r2@wlZt-cs+@W# z6pw#{8{+M%^jvWZYzHbKs$&XYXLjruc9-i?5`L|w6%Kk~38>j$O`UJ-Pzgu^dxAL& zn}NAtu_k1_OHy+l5acDhz^>Ql5rxc}Do|3Tc=F2%^Geur3xgMK%MPw~d+Sm5t| zdn{ zEh`e{r_88+Ae0frWhQS#?3YD({{h$cI|=1l3Qmiony9@q%3b=m_@}F<8{`eZH(oqo zZA^DdAO@GbZR!GB^%x5-ikY&IaZ$t1uL=RL3zWYCD!0>%fk+%!N#8$cT;Lg_t9;8=xVlB;BWe;BX~;E9hB|s*B)r_ zemC0mNlL{tbtJT5iA^mx+2rfk+-zKiPs}{T7Z!@E`tP~uzZ<@%*ATz&~&|vWtYfy0&8;W>lqob zXRwOo!l50NJ64Vjj7OZoi?(P37s2;PCBBozs&1Eda;P!9h z&3giSe==Rp&On0C2wmC?Zh5HWv|k)0Rq|+)Y4TZuYOT_H(yVzaw?(-_*JZasWK5J! z+5=6~O}Y1_!$zXj^R8{>rg5a`70ox@7-DPAK5Ahs&%Jf?O$oo)E5oBbdXPpKzmBhB zy_2@_;Ojf(6DqQl?eH~t={VJBqC(l9Su466c#zRNv0*K5diaKt8r%|fJI<1Q*Nu!! zg~#z2BbVvFv^!)YWgw9 zZTH?MRZ5euO=HS@w$5MhI4H__6APFc0$1T_VI~paI9yC@2ZElI78fLosh8|JZJ_-*5x$7vVMi@Fh}%UF+i5#82*4fGRv^^7kAp0|1kM zJB@VNt$YD8xbS{7*Kht)8GgLf5kL5+wk;ERUi_Bi!A$stfv|3>)}BXnG^kUeu*pUq zF%QtL0we~P?w+JhbFl>9fdHd<)`B#fl7WrrH0Jgg}f(#6?w=1q7K7Pj_ zmx<XrAi6sW@S%?8H9o9=3iRqDK8mpb@b%1f1c8&@O(b|XVu=KxO9-_ zA0E;K2JBT0@=LCcSVeaUce0K-cDm!&2*pvMpy~j<2XY7v zfpZ+|7pEFzFWxPPJW!QbGJlBRXKz%_e6+gC*tw2Sw8*4W%w9Dm$Jftfh~p}F>teNG z3KV!kQtLt+t;uvasgtdaxLOvS67HC^`7Wu0@_ZNY{mx}T{isG3uac1^>H1J|*1F^9 zhMo4i!MzVbF;a2fB|Oa8!CTKLGInr2OFjTgRFLh}b)FH!@Zy6qk4u8YLKRf*0eLwz>vzxyOF-su26C%Qg4~#e;J}`d(Yh1h+ zG7T4B@|H_uF?qH}Opijl$bLgpgOjZcwzn2FVq%-yJUS>@EuVF*o3 z_+gKGn=3Tws~Z+HH0iq{m#OS#el~L|EbUIqc{x_(KN8c)G!?ORK~CU!0=MRmBJO-L1qW=n@vh+no=8!Jw>CX-JV`@Iwy6c4 zXe-+XdnUF#XrsW}_syT>ewSFnkVk1*3LTBX%*u?grQ6Dssz-!%{h*y02L>G(OzgW5 z*d?+pF1chEIHkL;f_Q7$9K|;r+t2+aMM{(Oie0)H2q|YS6C@QzT+eWmiEMRBt-5=_ z&?egaO0TZ+Q@@%Iy~iER>-0i3TMBIT>Gh{S#;^Y|iSR*_R>ad0veB)m-a(AukmEs9 zCSTVXan}E&==Tt}3IFjU03o_Jr3fLmgznG4r6)Nv3l>ItJ&2FK?ATe|9SohSi7qYE z5Z>R`-sN4LZZ$=|&e1=sVJBibG+I|~$gWqj%u1ISqh1~`TD>zEX-k9{i=bpCx4$Z4Ju)fFiM4k zo(WffH0RgPc96t?tJPVt171{|IX3&Ai`LNEZbZnqR+#s%!@8j$(PEvHJSZ!M1Ia~B zJNiZ&E76YIzN;QAU{ zz=^80y!#l;DfOk#^z@GG^uVri)`DcnV*Kd`ow-bAFbJTiN4@7%!=jB}+VVA$hHr37 zf2V=H!w7g9VTu#ZNs`+S8R!-*-Xh;T?`%HgT3Jgfkt8j@tRcm%n0CBHw5f1n-=HKx zAm-Fovc=iP%3apX1GMn^a+Vp z_a3cPI=A|mReVn!uDGRu{kGLUaq**T)lD(-4_$@W1F}MTS4Je8cgNxGaY>Tko0SCfm$0*GSpS>{)DmeE31|c+pVrQtrQv;OE-tKx#TVq zMy`&!f<6V+k6(@82<{@Yfx1y%i5f>A24EW(@(DHnJc^z%B}cJ#bQlV=;H?Fxy`46# z)`4)2ZTiJGbS-Ke$64AOGWF$C%g^Qy#)Pvq@$Hemm)V`u7jENaMP2MhcL&uknP`rK zk!`SFgd|B`VfPb~h#=CmD$-87mJSnVs;R{Fkn%|DtNI+v+dXym+i>MuW}mp>L38n> z=^c@jG(zSZh)2eAi!n%L+(wEoH-Y5W{-8n<^pCwCBK5t~K4fHgYfPP05kC_4>u2{X z)>`Dma5%CEUQ>#D0lg!owXW%Vr+XT+8S*4H-9sa96=-< z%Pb8bG%{HzAuOaG=r~TT*=-%1Dl0x{WW<>ilb) zS110~Q&mU_5mh!bVF2&Od=xM5#R-(26oDEKIHMHoH=`hTb2XKwDTJg|dUTb^Vmc6U z8J^$>&^*df!n{r?$e|_>ZJZa&qTDZjzkbo-l~6*_^~(XrVz>?BP8JAY(4+@Mr=KJ3 zk*3?J^AOzxvdaa!Sk3%Oxm-cP8E=-bTZbpvY$i-c=TA!?)oy>_=HTG_~|G`oGUB~7mtYC$X*CQ)AgapUl+`H^D2 ztdQ2r%4>)^BjzVv?5X*5C8)}xot{aBLD>%%>ihOxAz-o`WPb0ta}vPD4?A+HstU;{d+d|6xLU{e|z z6d+3Nf+9um4$i$i4MK?6z*my$$<``G|M=_UsqH2x^RNjn<6lpL|Bj~rnf(7v^Yv4? z>h0cx^&ZT>$hKlW@}Yp4y*Q8??OBk6AyHW0!}V$|TUqCOxUPo+8-bB=!JkxUP{5s7 zy!ef+L}ZH)(0^qbH8_!l{w1#0cYtnz^5Fv-8Cc{5{~vKm@H#CsTqBDH!FQB@iQhwE ziT*3n@0SQ&ue_%reTe@N>8n!SBYj4awhN&I+Q1umc^SElZqf;wX$6HW*+@Sd#jZ1k zEtlf4IPZC^`XAcT-=7?;+mDurXvZ7LG>mA7?>~6MMZpmlZ9x!kK$K+fYe!N_z?C`| zbMVj1yc`{5o+frY-K9A|w1uB=Y&FGov#rT(y#ORHM1m+tWQU}jfw4Urydb^!9yeZZ zhTz{6b8+_p#^T3|BEF%<6}I0f!AKWvUrtggMlz<6UsJT*1E%zZs!*FuTfGA#DwZl3 zE7*^5f*J^PoK8vo%{?^6Sq!MU+aG-cpZLxDNM$OHwF|B$ukD&OMHTwH9IXfHhe{3#=b>My4Zt&rNu2yezcW6 zji3i1gl~)I3-$71D;U85u{0KWxkoU>*l_@_8_dCLx{V1!WC@(4*gP2eFMVB$v=hWJs<|&oif+X%LH9I z=QXs;hHqR4*Uyt%aKCQ#ddlW+rR{Dg#=qWrpQV&#O&b1UqXh&VbP%ZPmbY@yw%q$F z+feHc103GEHvQmJ>?%)np9>i2T^c}KVW)xx$(IR<=IShhn09utb}QGxEop}pI76Dbq2r-y8` zm9_x2@2D^mM?D$>7xm}~=knh*HMp^v@#i2{R($gpP@cb&WdDLbN+|17DqXIFE z4vJrFct}mzd=Bg8Z7g@mn+g-FZfS0Y8aaf_jaUb<4~sv%PC%L??)!NDtxW6TOD8iH zGl8!!?8`PCzAIb+3q&YF2V2izqdu5hPLt+)t;sMcX#p=0epdOZWPe!8b@H5-?`5~-1@&1h3YU$k zbJr;lzf!J${Q@nPa)eKL<5bc*AYCVXtKQmUybMXO6b|NSZ!Wb2RBj`iHGdqIxy>Cp z2Tz{6y41P>=&<-wUB7Y>T-ZQw=^<6iDitTv`9gdRh~=Us|Fh&EZQbcUsmiERBEc^l zl5||~B2p(zZuuZmr_O!Coclr~(u$T*1|o}dW5tO53=P`CKtyy3cKza$ZzhiaeP-9faD~^(B-=!&&{K-Ay08{#}KqJ-G!NT2pf&=A%M~LQ}KB` z`{;LTbG$TF!Rwuu={2{;l3stxWSBvst5oHur+)xK{_Oz%(>c|Nz`i4de7&j#qY034 zgN77)SkQb50_#X8VQ8K@;KRhf(}y6@<*L%cq1|x+%VVocd(9rGnWPyo;Do+Mi^}Uk zhak~h4qjcFuhkAXoH18^XsAw>AlPQ|V%;TcBlok!+ zB!azU2oaA1`5pA4_-o|{O?3dalGR$C8|S8Rn&mi~bDCN-%^c~uNgW7tG>(muE-_a_ zMSVYEr2Lwgjtt+R`xK0gElY&-TVSgoe}ZV#(=p~Ermrm+2J0a1b<>|Ti{siX89)tr8t|ujM?{!H2pul3^d>>0~&ac=JItjSFpC|6c3*_--vo`0@X7^^U=rMP0XOr(<_)bZna)+qP{xPi)(^ZQHhO z+qrqaw{CssoWFZj?W$dS)>>oEHGtkDr&;ljS~}E7l&W7J12P9L{}3Ho7n>H4v|sp( zgpNFEUO)8h`T(~!9ZDC|kn@LF+=1-Va1+{~TvBJq>tKfjYo007>to6_|5n@#dc)B5gEn$i(JqVX_ zNI?OAawf)PXcv+%S~RXkSrE@(XPJQ}&>dgKo#t=_ivXBtG-N4GB)!Q1gPS;Tcx%Q@ z{UCV-?zmPmg+#{uD5lC`dD6t4=*aBtEFmfBRLNGkm!18WJ|a4WzVO^!M}z>$FtkGw zw_R(N9gO&Agh2(CVaV{0m3u5W6fxMSF}|@0c4jnIUKaB!^BK~|0+d0L^;(r4adAs> z-98km64fp)6wVtlC6B zOMrW3yKxvf@avgmAH#T{hU-c-4I`DFnO2X}B0qyaEZzVnk$?QRi)JeCotfVoXJPAo zQaN!8ms*~8gApj;3gfa!Rt~=H#JEJ%q$9H}kUx-{SIw2^ZdPvKakT>I&OU33Tq29` zYRu9Gqb>!wo!v3{uZdx4`3R^yMgG~w;Nm8CB$wgnEI-~Z#|$i(j#FcbiVuEdMF3MH z;vC&PX-9zfq^nL2R7qk zOZd7bdZrYPwnC?=pxubvd| zQ%%OKqwungA7C>ZeoYs0O&^M9i^>~Ulx+*dJFoQuZT5~*{sPP5O&STW9iIgyJDR2OxeT!tJ>E z@jzXGHXxxE)T)R{of^>X3)L>XT<4ATMJIr!e zY%OTPE@Nd%jm=wueO8bJw%i_xsmE^0wy}=&$vK7(A~R2LSKa z?`*~!H(O|*{BYwVP1kcXy1SS4;NAw-Q^EOb zs;Bv*q<4G#h<9X)vhGa;@2m9pcLad`rVRd@LSq}XY@7Z@3&>pxce}i&d-omX#pNTI z>ubITfbn9p`hmy0v=3hQmg|f63b^I)rt<3~iDsp^LFyoXSr$xF@eCt)A~!0DlRx3j zQgO@Sy1R+1p?EY1r63GqfH}O+7fMtiu_j%F8w$T&ERb_9x^0LPUAUSs(4%9?)3v`Oo<(S!He@Q3g_8ycT_o^3 zuq9baucS__9?eQpZDC6oUMTj3u|of&qF>lJv<_hIlLpm!?&`nvSch+>tjOe#V!KGw zYT9%j3RA}jzsYhuXz6RS1rV`sWZ+6Mk%q(k6upifUrGvF$wLeyI!~Y6_+)Ze z!Va`5;7AKsydY4b#M`(i-0D;kLWep(UU4GTiZIj~`Xx}jGbkSz0-Blg^io+AyO@(g zjMhx+c8UlO64`O{%bSpE1>@EOxun;y{O@OlU0db`sognrI0(5hLs!h|REE)Qg4PBq z47O$_`l~EX71XV8fodB6tT<(;rPc+#|P~zaQ4SEFh(73I1r@R)dJwU zsFJR5YLV6whDB+<0gliL#Mv8?@)Sii3+zkevx6Wt86&0XSBLe4YBDx7zqZvQgs{k| zYxiz#vt;8EBbhF5IRXOS%6wm~Y3%r#7$~Gj$T=2bD zJjrdbj;2mE<2do4m?auE+4YL=W1!Cf{R;6dm$2Y-3Rhp*uDo<>cwmzZ9z@KXHEo3F$# z;s+q435nr~69*fZdGJU^B7iIJ4xPv4UyC5kH&B}-5JMv7R@=~EW;lRe=9v{n4~|8U zJP*yqQYj7H&MHy~ykhzZfnMND!Vrld%|^}k@;O`O0yGc}*fUtc#s_q%rd-Fa+E)oW z?VGk)C%5!V;ERI`=uz3!IYo{~9)EQgYRt4%YBZT@(A2=1XHNfFT*eG-k6heN#+DMg z_@pmgeGHso32y+|u!t_W2uY0fQv)2E5+GqUMGwr;CyhM`5~S8{l*uO-Zy`@fD_9A~ zY0WE60(85UkyC)sbp|D4!0Yv9@@i)Pkzj-rSkA$z0?l&9EXjjwN+Llv4hBT>RmH?j zS5}ZJ)ICNF?FYxXw~gm(D+DaCAU*FFOFGrJ4azplLpE*-g3MBl<|Pg)+Q@Mtx1k=D zI}MVm#2&FAp@K;+`W1w>+rd}1XIhLyTwE z0-FY99&Qy^3eOG`+PWjwC_*U&NtkqYfK(V;wardj25e{+Y4(RIL3c+(qSU-5Y#oKR zk%K)YhIGgqM%N$H^mZchtht17h0Ld^D zPH4D^IUQw`>*A@$vS0M8I(wQ9?6!_K#`f~}@L| zuJe_dd&{X2?Nx}S7rU_&YMK9}?kA=%WJJ8rR`rCW=dh9W|82KBqv2S!AH7P5r_Kim zR&olqGhlceTAPe9%ZO(=)YCYO@t!NyZkraFUS$yC91krv3$bJitge0+0oX|vN>95t zc#KFHb`9+ZM%&fU-|LR!8V6`$-V>-P)*Wv|?$^`s( zIIjnf@FlN9DaX-8g}nxn0PT$s9_*i3CfP6)hOytC4s{3Z>cZBaM>XZ!*e$QxklhoH z1DJcu2(GQa%c^n?`g+A&-^(Qg_Q~4IT)4R&Z?5gC&~9Nbj;3(!Ekq)8NOHyHe395j ztJ%I2`7W91CcNO#XreF4W*0njDd~oG+ee&2i%yTRPJ(>1u(|v603f`R%`>9xObf() z33k`JF_om@5WCrm3tMD(tPdY6OjJETl^>W4QMBjIdJ2t!esTGVYUcvJ2XjG&q zs|Ka&b2NB*!^ERA7-}kyJHTID=SnKdP-IkOeG@wI>=4p*fSzW43^ABA1{COc+BK0KhjRv0bmni4wrE4%2Nct=4v3vI57EWja{1xs9ZglTDmcwiFtT^9NhVgbJ5N?8gGc}*%mBKjIDF=!;H@8 z@|L6-q?4G1qvw9tfEI#%nGvbHQy)qLSH$yo@>Q@vfSu)q^$f!^Q!7KY`PkUTo>j&3 zToDpP(#}b%#kP6JpDw?cd_hi*no%WvY7GAp9xgFo-muCSrHj5{)c_vp5*>pwuENCD zKz+-?mFwaBfd_h3yrQ#=RPac}Yw7)s_~vzb^@%v-Yp* zsaTUoUKy0-+A5Gytl8z)$`sNt$y)v8l<1`jqUT0&wVnZz3g}#JY04XKMB)-t<<>Ik zBhARH(Ys5RTR(`_ZUVuzJ*p{5o_wBTBB>!^^o<(o(a2!s7G#NoV`E7wdVyIouaI{E zz!sH>WNL)Wb0GJ7Qj|Df>3CYp;|rUgTD!KV%DtKeXf}y8Dwld0(Yc*kZ_y1(Q>=1| z&R&5k@6MZd|5Bh;l!tH8=eTr=2&)IWVOUB317W=nZ=L|q*Yb|+yfbfEOH0qg+yDE^ z#PXW9{s544_~p>BdMx$|UGNDj|4m>4I8R3J)*(r*j+tH_JuR2KsHP|FDy{T}Z@tjf z1Vp6D3!iCmxxr7WO~)7CFcrE-#6F)-gvigSM{nxG*IR@uk5O zFl1!cugqP}+*tWX`>WdidA(#1;Ntqko#mZ8_`(d*YTff(Itt%V@OSKwE;XXeNKGRt z={>HBv6_~CQlg>6-6xZh5VH&oTJty#A zoU~59ID)JZg#BjD*lbbRSGMzQ-S5cuMJwKIzsqz%nQu1tYU$>%viXkR0658?;NARz zeCzO}H8qAxS*=m#^E`u~w%neS-9f>sGwboFVW9&kWOSrEBiF3Qf!u{Lmr?A5rQ=s_ zoqpaPys49)hZEV1=fF-k$>^bL~+B6edzUDKo-{$4-RXaDtROtg+pSag-QJQ_? ze~a#{)i_1pSn{rk<|NW)O;%tIrpIaer|-OA=G?h0&%91d#^L*49eUh8Y5y~;{NI-Q z|7_0x(~N6PnU`<#?xE;@c#&qw=AM`+$uT6zNX=5onBV_x*%{^VheUqnoH}61 zdd`^m&G6P_?!W(sEH^#SdN!F{-3Z2*9PUB@3D}fI;zRx_s?<^g(~!LZM_KTbg;}=^ z9U7>>lsc2~u|1r;aLf@1Qmf}&-!Vx2$E5x8iw7u_C;WDzDF|AOs(tmI(WC1+gVEt= z_}XS0$WqtscQW1e;N>5u4M;mA7hoz7RT$gozvvM2aaSR^^NVJNi4NHWt7uCUNZb^6gB)#x#Hqi1Bf zAS9u&X;d<3CPUBECV11tCaKs&h$ukGxTy$LPswokIjcX}+}ERnq=AYRdmOgWXh&&j zC2O)+QA4ES`DDvU%}}asF2e)hG5!1p0{~Ha=@XAXTD$6&tIB@*y>aa7gDMgt7B!x- zvy*bD+J|iQxPn5@+C0hi#o@Uc3ltan-S8f;)cw`+7phZ7T<;o$ns9v?of zLRO&2E~Q%RFNC?vVX7-<#`0P8tH3PSbb|81PI7je(s1xVMT~XLf>ifXgtVlg8<6wX z3S$}bz@%Gj8po~Thm|sL@qh7M|3j9djLs0V_*p^vS)u(; z?by&ZYtbXu0>S-fHLfUOLE?wUR0R89Q7AqpI{_e5<=sI+4d*-K;AoZ!I+v-2Y4mQx zDotyR=BnvZ zLrVd_p7M8w|NWYu@oKr~PyEHc!EnGu2(AD(_zP3^ZhwDHRt*k?U6n@>7{WjqvyDP( z>XidUlc@Xz4cw9uMUow$d~=f$@n17+>8Xrbheh zTxAM%Azcf=o1nR+?SnoGW5Q^Z5vZW>PHGn*PpwvQ9#bWfVjsx;Opu&CO$^pvgRe7i zp*1bAGBx+>fj#UIwz&Q@SFW_W!s@yBya}`uYgDp;Hmb-SeExS8G}q>_D)I^XLO4#5 zvD!UOt65?c7EWo0dfkJXmFoE9)0Bp~jT40t`e24R%D7g;Qgz}OCOP#_W^;?V`db8m zrAp;NOh6*lcvhY|vlj9Ky`y5P3@6AOm1D`tw3jzO9fPrYa)5C@e_$Ozl<_f| zv3nqOA}MmHcMHPL{6~>`6Mun1IeB1^urQ65JHgA_-?9$wAIAI~lzMA<`*`rQgeY41 zVTLn*vb#riLa?(LVMwwaSqKdgC>)OxrvD3_^yqE@ybi=(_)A<*7e_OU0A-1AnP$n9 zX;EU@-^u5R#p)-z&&lVB zw!I(dSpmoAsbkL@paP=(&bxcq4-ca4p7#E?=1?x6=E>sisYv>HtjHYTn32^np}{o7 zt$nxAkkoEpq&jO<1kvMaY-`}uX6SNTRv2eoXM`PLLvXW_q#(!7F?RI;(zH#n)a2T| z9}O)}(6Q=k$yPm-eYeDGsaSuf#?Ea7zTPy-tsks#{|XDf9JjQ{qW8>8GB7DkWrRz= zD@sqb>DJ}y^j*xLPpbo%_>7Txu6G5h^jzO7&@?5?EXmg)`jp41%mD@eNhFxVadIRz z+p?{45tFn(RvPK<|8OIJ#5^W(OTBn4VZ^IH26OKp`d7N|N&WdSGrno(a<}~FIq~Dx z^nipbcF^>o_(l+oa8_ch(Yj9mG8l!OJb4Y?L%lD{<~F^we2okQ!ow#3_zpT;syYe99lu+^N0(rsUWzZi4rbbl^M2Erxo{9K7lF6Q3u?&_ zW0vv`SkPIPv>mvCu-=2P7RIJv)CK;lo}qiXGH3S#>i2Q->lxqRNrMib zEX2L>cc(R=9~+_?-;*cy1vc$veM;w^!FfZyTv6&I+DQqjS}Ane&!8fBZjK#5P#2MD zg4E2S_$JeEZ?ZlVmA0WC;;oJm2vK+-v9EgWSmA~gOU%_ioBZk2tnrFnEiA=u?(kwZvA134U$G2$G}!ywBNULDEUm0k6OM0z0TCD% zYelmm=-~G&yd%5>+N3&srtfUKB{Sj!WS2u&Fja!4;*0y1M*nv=?1rx`DPbL8BOrqS za*X@953j}p(nDGYO<(V^n$q^Zpwp#BhF_*ui*rENaKhWTG8Bh;$c@+Gj^KS&Z)zAa zE_`0fWlPRlX!gLp^R^=bbKNpiEs__UNeArr|LCm#_r(1_QQm*f zVpA&|$&erX_InrfztWUycQVjqA8RngWGU<)eabvLGA4k}Akb9aH)OYXv-5bVbrP2g>LqZWqI8lOMN0;}w%o*> z?&weejP~?OY>LxGVh@=8WYX%zefJWFc)P`DDB_^fdm?jj(5obXDzsqwG>MSoY#r^V zYv8U!?RnX_$Hj*HFi%kvMn&BkaUd9F7xMXIKMq3IF(;5v5arl@FAKaEw+%S;CaeEv z5Lv*Xk4S9JsbljcGOAe6UnQi!(O$&k;@!~z$Y$A#FiZRC7ns)WQJm$kKbjW-^_uKE zA{($R8!kGC#Jz`eP_^RK-gkmh^&%`#BPZfmi;d0D}lZ-Yj&Ts)}E*r^V zc!Pcqoe5`2Wx-jh_fGw-4g}Mu6l=3er=S#fhhG{9DvQ=o(Uw#@wUIDEmId^>{X-}K zEERfF1N3l~EZu;vld9*qzxOsaF!5R}!*urQo_U|EZ{M^snb&2@e&BYEnkZYtu&>fF z{BI4s-8-&M0P}|;;xKnfhiN1^MOCsyI$_>Io*5G$%J|tmAVV=;12KLBF}1hC#~H>5 zgt4i(X6bwFIyaLx$fF6S+5HKYSX*I0B?;9rrF(ZungtU{DcA+5{lPJeH10Az6uL}4 zW7~a8Xh84RuW>`8OfWuHh*rKHSk>*Ag%U{rv?cZm$@KR#82cv>_Hd1X>|K`ECSzR! zmyvdfE+J=_{h)o2N%bvm37#S95~6)ac?sLPpd|OkD0i2TmYNJGTEQ++1ua0YVhzDb6`mc zR-ytDw3EhdK`PlW1|ft|GQ$4x@tbP{LiJ&)#}1i6N68d3l<9DXQX$T4S;Cg>q-n@c z_d1wb@RK3U=*SCM30guVptd4*7^=uzC3@GPw*D@xN5#@pO>&*D)CW;Qw8sVEGzT1S zhP-F#4LPmPc?}~Zp-mfrK`@gO8kl8W*YwJY`S%j=XD?Anq}=Xq@X!rR^`&_OHd(P_ zNoI)o6^f7A`fg#??PV~lSd7MK7q=RAou3f&u`-RU)+3@Zp$RfwJd#i@THI%9U~EEc z+br|vMqOn7@b^2XF;>f9CZRu@6bCqH&k-?^SZOhmG7ZOkA1D45%bX3>OcU#W3?Rn# z;~28b)KfSb_aQWjyU;uiVYKoFa9u@Sm|qB9d}l^YILBwAM3GDztYl55^%-<$%G`3E zjSoRcmRG;ElAMx3l44T^lFy1XlFQ6jXD-$KSXUE5w~c^#75pR2ARBEpB&RU2hlM{g z?y4t=XI#rbWo4#B;$>tebi5{~PJpg+wZHTxSFRDx&7O+ zH~0u0I-;l`HlHosu3eIYi3DLOXNasgu^Mg*2|E4bv~6A=Rvb-2$8MnYTF7~CrWNoB zJM28NYS@=LZ@d1K`GM^glP-M@aLT11YUo>N7H|#MF!U~3(lQ?d*z{Cf^V9UeK%%Q2 z9lY0Sj}I4I)o;V6vmF&js(72|i|;xP&H*PF=e%-E+!Rvfj+4o!R$ditg9$GW_cvZ% zh=5%T4*HvB9?;2s6~u<)okxkA!{TUZA<2R47d&iNzg zH>tFCJL8_;vbKVAvWyfTF5E)KJSxfr;j5kQ7`?(U-(Z{`*xl8ZCxk8ixI=(_i3C(~ zqKy*C514Q&Mq^ip+dkQPve<~k>gK+lsv)TLHKu`QY)aGte9U?Xvm&xIG=v`}(pnDS zw_7NJ5H2$#{0TQ`3n&YB<^3UJKVP$=+8q&uVp-O6(5Ylu|B=#PofF%=Aw*lAc9p|& z?0T;yXg`<4)u6@?Ov)E)c=cIg)D^zMr;``n#gFF|x&Ht=HQOvY>(U~*-Cu-9C&Igj z`WX;?YT^mt>#iuBz2uLy-p7Zt!wF-KOY=^GtJefJ5)o0*5OCs-0@;1FJ_>(E9B-%V zDd(yluZSJ0Y_;TnwimP1;vYAZIr*CwM!Fn9VGO{7;{KPCG$S}}s%L5A%V&|bx!V)e z2Vqe2=T!F>5DRk?jovyz!i-h`#kIqTPf~;|_1^~$Fz&Em?mc>itas~i0G6Mr^d53po}(I1zm9}Iz-92rIc4k%Q# zl9`u9_P#<-=*$U$8C(uS(o=v6LClo1jrG9@0vQ--p3_>U)Vl8IRQgaO!So{`@xBSa zWyO$b&+QMGF-?BT%Hp1Oke=o~_Wpi7h4xXKFqw%SS2NT#mhC|SHu!8w6T%$CuF~7+ zaRNqyc}3YiE<0Q0>j4hpxeUu;15DD%A+dgVCjk#=eU+I;4-pC=;2pM~3vQjt?c*&|Fm)-CmHtO38Y_ z(o9`D>^c?wE;J1YTb?ZHq)gMBUr_8PJF4*2aGTg=aT1xUfvJbg&CE2nk-nPIkPj7j z=gf*Q+=1)A0jkGwnhiAv^w(@J60FOg{iAF*;^wHK5Mlb4{8o3l%v}}5N8_+QZ@qXp zH8s0J*xkf=JQkmrSXo&jmQ9>Bcz}d*ib+kJPLQ*O^DY3>ctdrDO{y67zNT(rP_m9k zl2w-~PJQMzUjG@?ScUTFjXlFf0~M**^B70Xb*3KFj-** z;Iv?Hc&zT|G8GyYp5XRC&Tz#y`ed!d(*$7Fd0{O;OA<`!9FgV|;J zqY;anzfCvgxS%|Xl@E0YfASjf5I!e+jr;bbS8($YZOH=WMYtDDbJ}Nf`g=-0fyl#_ z<|tzrRDJ&Eyc_`K7e_9_yQJJygMPS%AU8D4lq|OOUrLA0@BW_w*8kS+|5@DsVJ4x3 zgP0j&{Q8B$_g_>z;{t||{3n4G96)1WspsI(_>+R9xVXrBY-Fs>;3hT#CHAk+JO!kW z#5_k-)DNs)7cpdK>F>SI?5Jc$YKSRXqk3bb`8kw$rA8?#bvQ^&ThroF*&>azPD5jp z^5t37-RHE2%~pq`aomm9=gViyb=UjVXSTz%^K@&x36J-S96?%jf~y&HHh^fd4x0&f zPJ>3u0FKqJ{y|yPp^mB8^%T)s&aE)n{#7w`yUad4`L0zZ+4!dP zF3-aa*hbMUKH1n#_9!D=0FZp|HWzTvD05hoeD}1>CG3^ zO-8sZdMz=hNjd2>r%5}RT=*(Zv)#iACBizIWRp~@QP8SHWcAP_4uG%D-nC+QWl42u zA6q|sRWiH!nqldObT7!;3m^Ao&^^THa6p4W@NSn!2H7 z2f$XiSLejUQlB<6H9xbwI@3>^aj*LR4bW1JUUm|4pPt``_5G(9J%|ciRNq>e-_Fj? z+TCiU^t|REZDwj?Zgn-#v%skCkD!LNEx&BRpkzYt62^-S0XUWGapjMV#CjxC(6Yb# z$Aw-79klPxbOFCO8ldZHpcT$MV@}mGSJ!lO6;u>-738qSIb~*kYJOpAV`F^*wj1HN z{?Nj^6a8M>wxWHhc?Km!@F`gzOn8#CaE7>N(e)30Sc4u%cR%0Vh6vNNt48HI8k}%< zNtdcB*hpg$1K_4c{|7JREHw4k=v94-zoI+gE^Ke4q!LClD<)%21wOhq9tH&XS`ON` z@T_faDYHOCG3*I2hw-5!C~MFsf2^7C*)#FczY%zPmkM5$>quJdFy?&%;9KMqE>*dW zEfR%N1Ec36Q^R_Im*_~VK@s01mOp+atQ4!IEEA&^6@ZxnL|>{m1s^QM6=$H12pw)H znZZbDS{>MkS@?I3+zb?>4c;&H1`QHMlty9q&s^p&Q`xObJ{3=eKb$2PoFY3$_Qs!$ zAgItt?46-1MWKXJK^x^F&k(*7TZUae>#2+9Lro+$@j3yZ?T`XqlEcc?So>oaC0%k6 ze}>lR_5jvDN<2@&2>QDPHii*&xv8;;x|eR>>(gfu1mF}3hIu`8-aBn!hnw13hJ5$U zidw-=E;hAmH>)r2esWu%kn*!zpaHJwARtCA!zgy+k>4=S(a1(fBGdYb-M}B(-~V6= zs+3>bCAB$Sd)mVL-j4K!m86`~GMP@XK7eywDF8r4r>Dm7+s_LKMXRw3ND)pC=X!)4 z_-lsX2C$vCJD@wI=;Oh|3kJc_#vuhiMi3ekY@MFE{a|)1Ht}202Eg?2vH2Yi_}K7I zrxt#-_{j5r;6(nwN|7hB)5Tl(s1udoC`U;590@xCdI$y;<8)_75#rZOgdHIh&3KYw zBLUeOZ%Fh^LOUD5WSeN~ zA>3%iseag^g~Amq5~SAnf#@fd!vKkFdC^qjD}|B7+vKIuFE zBL7x_@TViPS)fm{>0GI!t^tcP6fX;QYwl^8PNuPM(aWB1Yo#J*SXD0!1|Css{{+yxP1Yd*a7ce;xb=#vpR)62`&Deb|sNk1Y?4boev9TxjyMl_1-O?rLaQtwpr68NRX z3PYXd^@kcDi?lU@&idZS?$+=yg*RBiM_7x1`hjY7ra$!js_V(6L?~2Ykiw$wXZe{5 zEt(-Iy(YjvY~5p%MBho&aloV%`vO$phkXpXOnX%g3aasHS0O@9@A?h(@KIw)NF9~o z-2Or~DyxA4SBrvN>@p{hZL?Oi(jqR$LwER!7H9Uj{f<54t34SzlO(G%niNB>_6=cl zQUs0_p8w|n1b$&l+Q)9_H_su*`1LD;sUmtX|7)v-eW(FHlbw`=IPN1i`~Wb6Q-co( z@&0YT29ZU-sSPFUK@=c56=U}{b3eP%g$NU^m{M5bWFrkU>rR*ZT^AnrJ(&Jejmg1K67(Fvn!XdQFmn> zSyfCMZKl;{e*{^IXZC>7#+Q$En!~hAa1$A4od%g#;l-DqeqtQiB?VvhkvGJ9lud`j zXs=&7>*QD|ti#{Wdzk&#I3jKaZ!zfzT7&lQ?9$8e2Xv(KJuUMa<|d%@wi*JL`ETiL zkf7WqcBF6~CiiH!@+&LmcantcYhN{^V|mz>?VJ6pLpDGVC-+E zQ#RGZbqCB%;b!ssj3MnQBaYHpJU@ss0<1D@H_OM0vdLcAlCg*KvKNxhF>&2;mYtRI zxUsG>UcUr^!(?Xfz;8eZ4y=$k+x1HBktr9D@DTe$?GRv3h53uIyJ+_OA!rB?yOn*y z963A>j+tCncB}Jhf_XfPbdiQ<)i&=`v3eBzSbz60t;h(BBl^nT)t@hEtz>HgfM@=$uR$&^IFv`+9_tyRFkM8$RWInP@j8Zma2T?DqeZmTdV`f zk>X>BWWGCq=*MPs234R2%Wqb7Lls0%C6EBCoB%y)=%~a~P(f6$fFLc;CnK(~g=vsl z-b;9*X^pD`A3!4Uua;w9R2*F^&c}BMn@X$whTS#2K%hW9PPMz)?#K6IO0*p9aVgshZ2t?!q8Vdir zk0Kap0cC7Fq1G)~-Q5Euz`5Isgivdd&Dv2JXEJEmnX~8(*jKq5eKD+7vo|VmoS0C| z?V)T62@t_7BuKU&tU1G0JyXtjVx}~LxBVydiTRXF^!L!Gfw$%oyLd|Dck_k4;QJj@ zld~f;YwrBa%4S{BDh2_JG^bO+B!~E>Fm_84U+aFX@0L`)dYqR_e3j#XU7kin3g~EF ze%q5B8SS&5D4lL_Y;0KM0ca%JVsxjWClswbA|S{SGPyNn2{CH0a6tsW%sSPH>6zfq zf^mJ`sNbk@3EdJ#p$+s}q`T%(LqVcS6?*3~2waqOE8o3NE0sm;bo(pFUNE z4gejGAta6&{T!37>~GPk$m2UfyhS}Pl**kFe6%CRIZ?|aW(sNranZ2Ki~rw!{?7oI zWO##Gj-3@ww*Bwn^5SVm|AV8F?rt5Le8z1R5{v6pgYqmO3pI=YB%wBf!sm$w@ARPn zZu`~0&)1-%(L>iOZ12GCFg)lr<(OhBV1OIFUAa;>PetDG#3L+2Ytdf*SQhS^I(6(7uZ4<8Wf}@{DdXdMGOKfNI zTqvb2A!g;X+d<#Wg&l;otZ!rsAwyCz>Im1Q25{o$4bMAP2wjX>t6kg)joF&*fJl_* zyvIGU1u+Vz+2z^!3AYCSF5hC^4HG>k*(7?_VHE?P76rbko;pU@7zT$tP=e0)({!I! zMQgaA-pju-QkX5jb{=4C;r|tpZ1|&UQnvmIY5c9C)}Fnf*a5Nhh~=tuB#AX0?sq0_ zu&oWOMjzv%Ho|=X#v)hc%VqcE4H%OS{6fykX36n7QpDm&cGj3_1w$A-fsCd4vEC0K z5fA^g)R9u=II0fttFoudwo({rGNkMzy7!>oSVX;-O{1hfXVqVof?EovlyZk$a=@%f8>KH}tUG`@?ii@2Q}RX<)9`*|@GQt&0}-iXYp2FQppYq>rHh zS{0Bejgih1V-jWj@hvL`b|k=IV$(L4uV=3s&iDiyegwM?n_aiE)3mv{VsP(kU*Gxk z8GPq{*<*1%PD60bHm&7p2bg{UrQNV4LSNIwUeIl2JKubHrSb9px}NrV#^WfsJmCI& z>8?@uD4a7|$L`Fh!m58wGCb)z_VE!`nNS~0;T&TEuXNQXn$#OO{PNa8=y-rNA_%|V zEgt65Djs~o(5hRHpB*p=sX=KGUNZ~H-Nl!-z_b1cfm6?N{zppt9N@2B-Uc^^9OzEb zJO{Ha@Mjuds?y~$*KEC{VZ$k=h>>6C>Tep*?WCt=^JElx0$%R;?2KlLKQG zL02oJum&uTgt`3c0HCK<(yCD=R^7v=EM8Luo#ct5^k^ne!WZQ9r`ckRLT_<)-uAU*Wv%WC#+)r%gyrx;iN@gkX`g% zUT_q}Oz_oD4{!ikf>zne%M#$Jq5gCXuLtOgD5&EFy^sNYpC2Cjl9{XP#!viQ%QyTk zeqap$1mYsDNDQlhpXpmlD<08MaNGb=Ur?wYw}t$xKvXpiO`aZ27>OoFpS>f>EKVJ_ znjk_uJI%L)hRS&~LOWXz%wH?sUAOYaJOqeON_-Vl21Ln>k1^ED{ZdZyPb~U|qD-E% zIdw@&?FQi%HMI1vktuOQ<{L$x4&@|%Yg%lW_I8Y=w1{nA&^l`iCz+seny!9cNb-P_ z=gzx1wBE8iVAerBqS=y>a7VPTQb-B?C%C~!qb>Rk|y>3l;JbyyY384|%){gq&8Z zduW;yCur-yqJIwHQ%g0L`3O`^U0hEGX@Z_JLtDh7%#VAqIm)a-(fXvAOPYbadR*%n?=SV{X88ktoeGeGtr7>zHU4%DF5)yjaN2NIGUMQZW> zKyEsje`#kT9kVx_!0Jm)znS^{pr$!4_rJ)}CYHs9MtgXMR2)V5bINBJ`FVWibmeCa zK!z-(Ir!;8p<1|RVU?K6tlygaGJfVATjN6e_~sBsI|^d_g!C#O=%IaG1g9wFptaUN zS2)iM^V*sb(2}Jr`Q<71#goMHXg$1ryMJP^jW{R^&Kfi(1{p8v=#NfxXXkngtTV1% zp~m-XWXe*4Mqu4J126(vtRt^VhEsgC%j z-Bz4wj~5_;q2eA&6NTDCm`eGErHzAMJweVCmmm8$!)9Ye+ zt!{wxxKABcmXb8>*kDwY&yKUlKI4fC^a@IGh{lL#jVZg`cz!{=lzYVdVHv~W9tYXp znwZ_6Qiw zGaM?f@t=36`i>3hhJ~y;71OBXIz<|p37@RaCB8~tFTC{{8D5r9j+3F<3xRm{6S{7t4?(o@ietOJN!( zFBYFD))aBa$FYm9a00Oj2s@^=3^FuKHyUy~kcU^mDYw9$yANoC!J|&l>7r(UrfQ$OO$9QR z68Dkwgi9r7vaJ{2= zTshSj(WGc_Q8yJN44P6Rte^xdhU8z?2%rcYX=BM&RPc8kVi~VRB*M{?RTIW7^c_n) zgtJd%X%!F80N0x#A|=QY8bg3V*UPL&I2**#W+xaK{W~wyahHscWK~sf*}n3O?C2dc zl_1WdhS`dZsI?mdFukSYq;y`v2rKL}5_`WJ(+N?1kZwLf@yRmg_-%Syg{)4A(*PAT zOJO@BFE*-ST-*;aPX!p$8~vi8WXv{Kzg6QTD^^?dR+7P|kWg=_bVx7)IK&c}6Kp4i%jA+`vab}R zK4HPU(FhqEJ$Tj(-)0E!2G7{=z5@al6EbD<2%)w1!Lu&FHbUYTnt2+;9;+`+W-k%w zDHbIz8`*YS)wY>SY=`Mq{g*Y}F+fMexWLw*69A*izP%es)FjgMQJEb#n6_h zqNp!$o-N!5P_2(%y3YCFUeG}P#m5~kzb}bxkLPo2C4ADp!Z&*gXV0uJ@Awy;&?|}e zyjE1(#4WpJ)4eq=^}yf{Xf3RM&Jd`S)Ft0q$qWsQwZ1qcL2&6!8=>poSQLV@U7fhceLN>>OvGTek75 zkF{tNANlp3xIKuzP#q{{hlQea!yNhkI4^}4{%w5iR2KL7-1)Kk9GZ(BN zHT{>Z33Qk%vTC3m$+B&7P28jX$t^HaM4tHr06Z{YvV$>gyu!ki!5h#Q(U4v(RsV`s z^NJ?2`XPx`IkcjYHrajPv%~afxGp+KEA^@R|0C-iqa$nEta9XqLX zY$p|WY}>YNJ008UFVFLy^S%4*^QT7DSXK3-)-^`Wd0!I`xkKE7?4^`9{Sd_{ z_u)g`Yq8ZI7*^Sf7!mr))a}=%$w?yvK8b{Cs+kv!& zXDn?PrQOJrB3pGJLf#~Dp*D-eNenrI)Q3QFWflaQw7%bQW*tcktQ^IpKR1Mddo>rs zcg|uS`Z32CgIngb^cL}ROD!&k$kC= z+r9ynD8BL&nUXmvo8NN~c)$G{1wFR**#Pym#Y_M?7%@r!^nRiWp?88|B<0bsVOo8Z z5sf|$Mhm2|4S`sb|7tLbmh)l4Nf~z=b1O>A5<644>m}NWW!WMSr@{1bkV>g{zgl&E z%tn_> zP}=`d0`dS3a5b!MERJeQAVaxG)Mca6xPRn?(G7#kv()f;-$AT@{{#-k5>$ zg6*MvHU&Nf!M!`OBTb~ir*AexwWKj~@B*IoxsV+2-LCd^x}R~6&eL*63ZLp`$X-BK zEatnHYV*4yaXHq?wlae9ApMuAEB_+zYkAROiKMJcM}G4f{_TshX{_{zm!sgRyjD!+ zyW>9qbxNIWP8hr4Tit)YK?5}T5At?Is?){?){9kB7QFs@HBR8esvi-&{0Y{;&MHe| zAqxd0Jgf`)sbFPOpCLpQ2y#k}?M+#Ewb0;RS$Vm`<{pAZH43I7L7hT$464;uLzYfV z?Y9if<{bN*CKX$*4Yv-0Q1_jxL2rlO_Yut3{#|f2-mCjq|JWq{?`-^kq)7j43PUm9 zx9nj-K#*|$69hJ6#SRCoA>aZB!Wys>02T@U%DdIo+ftIiSb#XH1;za!-4y^&1Yu9_ zM@0;Dk~OC?GGtAhnhIDE(Xm1Dwyaze);Uue)V9T-fGdheth2sNO%7< zuS>Ia18aFP-RjtX^X-1(-R`*Uw)hP-6ZDt%Sr#AlQvF&B;%UI)CqBz%qzShG6@c(} ztO}=Z7&O7P^igsge6h|v;_wI@bx5D zc0U+1U-eE@0M?#m&t|}b&fZO)A>gga!$*Rzt9+-BOMmDI-A6Dc*+Z-s3H@DBn9c0z z2`?sS$Zg)f0e+c(Bh_(IHQ%Z=#Kfvpytb#|6>mM@mm5-zx^RWb{L)eMq)rBfE)VR| zlU0gY0j|zRQ@=xF!I8Btk=*+0xBg_NeBW+2MYJ=#c7)@cT^DaO-MpeFGr&fH?}+tr zSdh$na5i6?-6_s6GaV516lyNoINb11^!F;G(WD2?I|81ayi1y!NA-_3btIXT(G*8& zKE~W4HzOtHLv>y&9oMJ!jcjBuI)+g}YX2NuA}Q3>T9b zcDqQ~!_BH9qLUfO&PEpjLV#tKRobN&O#E8#dh1ssy{wGpBci;-r&gmFB1uWIi}-Rh z+iGFfJyHnBM})X!xCk3nVhkrIXGIu?fKP~_{&VjSLlc<0mPbB5BfVU+2AvhE1DDaEx}NyD<7XH}kY){IC*hp@RZHA;6=C~|A`MafA_?SQ5g(%ibs=1xS!1#R@wz}!r|*lcgaaZIqPlHwMa0_ z#O2<63+Q9k^yptL! zH?RJ*vL7OG6Y9E#0QBztSm;3~N$!NAX7rBV%flX|Xy09H!NsUv2zLtJzpigIgslMMoW z+VLE^yM)U)>ItPOLI&+BJ7byGhCcbsC)jOAxc9T^^f=?}P=Kfl^R)v04Ek|5_UXHz zNeKn!h+V0&`=ng%+$`#`LbM=ThP@O8FaRA1kq!_DqDx*a5cA8XSReh&)dZI{8*C;YBRgI+2WT(~KDZ?V#NcGJefi53YBYC48dW-rL)QWVifL>P@Dz)qaz$-$FbYDpL?Y(%u!g#?(_*^5`C4f<_!8kwSY<3)7wDc;|`s(_gUc8SS%TkMX%YNbIs%p%MRf3L+oWu6{19 zS7?0~5>R0igSABRx0QfM=>};RxqXJLVU6tGJ3puhuDTTVvvkkknKOT1*Hwr5S`Za? z>aa!V5@AO69V?D&XoaF6Gotn|M5CI|SNcnT{c68#vRb2j$RW zgTrc%2dP|h_^k3K<83743FPE@c-mGj_Y$Sd2w*Uifyh`v*y7vnl<#uJPWCWdygR}! zW<;7j3PgfucUX;pJ>IaUR9WZSd@6BrO~FZ4-Sed7B5&tqt;0Q@JsjicLY{m7ex>l` z!=;sT1g6O}osyvRpq@>pDSD|9C=qtE?zLEhDa$mwxIKD@tNPKh`qnDr2oVRXkL@v2 zIY0nu!@N*$a_&@>e)mILCA~#i_qtfy>O#H@9C$6`z%EF+oP%C3%DvlsMHla~svhBU z)mUL{TP`+buG5dCuP&OmHMk}3haqEIj41MFH1A;4CY0n!6}V)E!oNRB9Y)gEUH+oq zi}}75WrRWLc+ByrT9qEeq8n(lYI6)g2h1(P@^DRo5}ntBQV{A?T%sq@87tg$4(vzO zLr^GF&qEU}@A7nHRPLuh!#Nu|XyDX}Fn#uDnwa6+V5Y<&^ z!sDuoeCKgrK}fXGjoE+&YtBU?0_>l3t{hvr@i#W(&-2TOg)zWw$xd8vNeA15bw+aT z3RaMDNv1W;N%u9;hh-JUQNT9QprcqYBcOEm$Kq)FdchGb)2=wCq|gzh8Et1|y)+#| z(K-GyV5@!jAdTeyYO17i_Ct9XQi1Rl&RiSj1owCWKRPnmM(Z2nru%N=53qx;DRiAS zbQ5?sgw{<5yBdS(q6PsZd&Ii%4|2dwbI zu1FLyQnpI(RPPRvc9lbP0raU~*%%U;f;?YvcH~ue8Dj0)Vn;lU**^w%5Pb=oMPpM= zXZ3m!zxScCINW(&<=$=T{1{`(7}bpO;5G+KCf>^-W12!qO2L2HX1ppP8hX!e6VsyO zie5uKkBg10P}VyxP7o6*5ez9~ufx_+C4+hqyUh7CgM2znosP551N4-ymYo~)UuRF- ziGBAbtu~5)n3uzsX#F~-BkbYh8&N}Irr?vkCmRqH50OZ_j79OzJQ>F6Q1Fn@72jP( z_4NNmh8EugvQ2uR1^ZR-92b*m%*!-F9KUmrL?%AxWAoIK_eU!QPOw7{0tclp>dbni zHJ7-?y(8KQ^0T)_8-U~)(h}a;-@?tb#M2}kvciO$+JQHH&^jL;2R7j3G=!~kHSX;H zv=4SXnnvH}2}fp2cMNrt>+j6EJ_4RO?U@M!kPCAL9+&LNwb+hgKdw6osqwjYXcD`R zJUdLSW{8hjqO|G@gy!F=^t-!Yf*IJ%wykRE#uKqs&&BG)0Z!@Dc}>uX%$VaeH3zQt zd#Btzxc{KFI(4%Fwd*w#6h0v#Je_^KDs78i#Z}qTGeT#Qstd&+dY7@l8Tkbrz6)r9 zbe#M2{`Lwr#66woE~GEkANZ;LaI2zu!&U@uVx}NEWxP0_`aOv{n88RR&I@K&X0?5O z{iN!K<3&(Wh6D=wzX0`r^PwA)NyBmArU|1Ez)k^82-&ZxlApV&DF@;P>0wdZFsuYUud`ukT6)7XGE&l9_RsIX1i;HV@3 zfGf1cw6LkTS~6baMPMt}#;9j&X`+&LF^7xxJkl)v{VB%C#Q6+gjuaf>OE{579oSf5YF|vII zsls8~EmcExpT^Tbd=B{b85^tm=_b+~0AZ522Q8*Jj7wP7d1C+_N+M$)2Y2CM6xG=B zMP*O+WYeR*n@U9jp~b}xLpv6?h6RRMq#qHElgZ#dM1U@g^<^AGzcUFyx&)_j;FlsC zUo^LNqShQKmOjga4XUb5zYXcn_7HT`B1|L^==O7#mZV*}ZPn8gbV-X_sPOKKCj6wR zk(!$a5u7FE)_~+>4qX4ydYcrr#cLT0V}Ld!aXs~~Hy5l?rm4pCSxl*0MqHpT2B+L5 zKw@g2;S~2n|A|RQoSF9jwdVeNYrm*&^#9+Qs{&Y(|6;kKUnU8j|5$VF&49L{)SQ6z zui7JIf@X?e9EJ3qK}|_-gWrBp49F}N>;4dxUZV{!S-@(j9lqq4vTxWbNK)AUg|w58 zY~n4JkkElG^Ef*-kOBb4^^uoEH?-b^9blHgvz#9w<*4wR9| zSz39IG94&s z$2(z51FcHwxSEHZ+N311#u{TX-Z-h-_l$Yy^E09_eH#BBW@4oAKS}#c<)&Cg!#f_sq^H0>IJBHmN|xqF>!H^O<@h)f7fx|w##zw22Pa49>OAVvVhTbM15GhRp_ zLe&qFUbR4XgesiBJhtQ>NE~PzERS}9ypWL&g1vexG7j6KYSg^ zT<0So0gtTM=>fv}YoZ0eC}3#BGbCj}$`k>H1ey=wca^J}k{UF#x7TpJsqX>lGcqk0yw*{^r(~&lHgc z!PjTVegL$-pA_gnmt7{;NOq6f+$Jp-)W*|oqoS$Xswi<2#m~d6*&j=eTC5*ei0tGw zoZpPhbdBz`Iwv1|Q6tu(NYsQq+l$KIl=B12R?FcjB~@1K2a-g88(Y6$&_-6E(9`4& za<&N(v@aS_B-ik;i}Vyu&V)94)Ca|#(HQNUMFYfyQziDPDOBI0q9@$SO^R?;=v*cH z#x9fWt@5pZrz}{K)3@1cMik=kM4PvQ#Vg)>>_;JyoZn5e#H#T1=#FmHA@ku%={E8V z>o#GyKF9|p6;j~xCNZ;6U(D{c85hc?=XE;2HG2oP%^)5H^Qz`Gz4s+*|aA%;v2 zPyqe}=uQM!RXqjFM6m1rZMq8TX>2a0?1Kf4x}S@(wgK zy^9|BRZb;EBRN&D)8?tNgP;yj?8|ech_CxD9hW1_D=Q-)E9HQlR`Q75*)S4~jxiMN zI{OMi*DUKRJ^hg#V=v<=Uz6r;uJuwoW!9LLGT-+y{1Y)dOQWXaW05FimMe~ zzIjY)v$80`dVD6XlrmBF!BAh-%o#INbX3zED!g>9Wi1_-kW%3S8=<7{{*QXXS8@g- z-cd>3@B)KMAw#k@cw;>S?j&Qhj4`C|FqdN{p|og#8x}u@l#PwzY$#;|iA_)wGbv#9 zNV84rdxnz2KWmIjo&aaz3aT1kO`Ndf{V~l~;z44hc#| zYW~i{e1fDRf_4av5wXn;!rJKqeGU{NWX0B`sD)Rd30lpj8JlL~`@6*EmE z+I{#ID5*J7DytQe_<>3>Pxsrti2|THcn}yaT{u0-@7hQaRWMeXW>uY?&1*4vxwuF6 z(GW0!ILJq8G#K=AxokD&Vn|fo;vwkgv~6<*p;C%i%8QxjY7kQk@k`&z6vzhb4(N7b z!Q>bqEsX9?clnR3=rR%D~MQ6d)><%UVYM5N`!oOBhtLpt%xDY51!~opFI3WvgiHPDZ&P@o zV|FHJt&_pL|2K@qJ&w zM|#L{h`)DF$VYXE%p++q^!bkD65BBG8)Av$Nqg!jh%>IC8k%r4xlWw3 zhtHr*_1X`*>PcAlm^UO7&vOakG`;PWSfgT%7-N2zV%D>SbFKF4zk-op|IZPh6FQ{&e`Rx*JHk^(275K*b+whK;X0Rv=pAzzWw z(RE3bZA%b^6(FhDQS7f(&;`CmCOuDvQ3HPD`6x{h3{Cz{8&W7dodB2H59Rg3n6pk5U(A}DgaKnHz%VF8P*<#L|yu~%a2w* z$kAjy+B6B=MT_+`QLw}snjq>OHVBJHNJ;0gfX%eDPGwHEb&a;%C*E;iMhNl`2lI{g zjbLR|E&gDPkNC7;tfq(l1B_{HEQ++;q`2%*(wkCZVVzSk8RWGRq%|Y_+F4X%`|=J; z%3KQH8GyK+y^C9`r7zo?*g2k7=g}BV#p50UMYj|lq1(+oyYZ&&Oy=WY7ji%fAsXo-9`99g&ZGng3x_Ju7k!0s4@PDa+6n<7+LbgRE zJ%zU}d9`MfR87A+bZ%GX?(cE?m1AAt=Vm<#z4`MFQw><2by`{pXWR;T_lV_}7hA@h z0Xi%hQP#wW8bWV`@S^SONgKQv*KC3w+mO~mtz4xw8m=K$#n#XBFeGs2M*5uO&mXiB zwa$0qI5%s>wh($dPSY=MiUKV~n6CDXucE(R_mbRhp-*IN(d9XELf#cw#Ey6?F-UIZyLnZRx&D&jUeh2f8~;`Xykmp#`-4wS6W60TOq$o00f@lpnF< z_f&k(rr%(%X;5M#xegCEtWXp@lmZlF&RF&7h7!McWZ|^*nKPk}K4xb8loryE*(6rM zT~e5HokL%wROZu8Bq86$ptNq>Cs34+WG^pF=4EAf(2S1^)&TPc17$kGLw-O*%tvnE zjB0LP%7guVR}RbynYQPZ9`7&hW{rzvOn11|JI zQEp-`4RXd(VX^2}^|^Z7Jof242Aw5ypO=nLiAJoORVPUd0KHKZRJ%kc5_irW(s|+= zS&^ZR_L-blHzraV1350)t{)jS12-+~l0hOG?2&tVlz2IZZEbD93mKB%gdwEsyr3;^ zJkuzBSxLi3bcT3INO`H&j1_o)I!^3dC_n4*nX(lZhus9vwi{Id2|a$r%@g4)nE8d< zFS!*phx+jpfNWrCr7Xm5ia7pZ~q`;gD+k@wHXHlRtIOmnCOwM|-2T6Lt^M z(rx$|PM&ua48^$RdbO=d-($xMaFtij?HBC*5wuU%awgeK;u8Y4Br zU$!*)pk!0tSV)%)mT`z!u+Ge?_QE~I!^Q#|(#^#a02_WVd_-UX;Y>72C&|{fg4|dA zKNTt-GNF9~|SviYjyk z+fjHsWGmy-RABAhxi#&-ACI``g`B`1NnP)m0|vHgeduTDx^caDW1p~le-U0vH>gD- z(rWUhyIG&Q45o}&&Y7-SuBDw?OR?J5I_Yo^YhuKbg zO!U_B`4`Zw_71v9>qoPvLHhpO2n!~m`VQHxb^~cS_nH-v8s+-62)Vj}Qa_?0oHRvV z$fy0JmVrb`=U7J$4|0hP`6O55c~wh=29$CIt$QS^-dI@DQJvaD!KxQK8>UYKP)@od z9=c1R_pb=wMFs_L7%?0>35IYnG+i&8+o+O{X zr9qD-#L-=1H8#Wc=z(IO&cyZ1Z&o^YX0f6vs}f}4oN?$n|58+RMc{dFXz$WO2hb@1 zpc)9-D8jYQ{I-iS-d?^dfBah~d@4sk0RM|jDZpcOvrYRods(h5pA zMv(NN*%Ow6>IDH)QMmhXuT_$?W{x_$67+FGnVbzShr`{IB1(a^3I_{EFa*UtB7DZ= zx4K&5W}x-}9$~j7*7e6ik>%GMOwvQ<5N1Xfdz%NS3}sDHez`3pAYm!Rc|j&qiVkla zN9NfX43_!>-i-H%^-IxI!w$bH&e9QF@f275JE+L5m48>GeIM9hKYS+HMz!nPzMsqg z>i=2KpiUk4`o zeU>!Muv~q$Br%9i-$kf*o6oE*Rfk)+lv1FTaRRU%n$`jU6*} z6S3ijf2flcp`@wb@W^L>a+6%INEzSIyf5_nq@Tx^!P3NP=F)r#2KUO<-cqg9(juHA8}q^rbB#YdfoXF7 zk1CrVj-vp=TIUjt#=+xhNRt4@1?en9p^$ewTJzfmVAft%%sa($x=^TnTWFtPmENAc z)RGbTLK?TheNPV)w|djwlCvG_XNi%qJm9pTN z9?In)0<&MkW1Tjmvp4nPzooba3>PGSTqTXvhKWMyd>-?*pjI)pdv>KmxLn|1A^Hfs%Qzm z9W080*TrpSrsy<&OKz=5Zf@o1(r8}$EFL6;a;-sb{a*TtweyM<@N_!=_%w;wUpFzf zx8FZ8_0)NF)wzB3;d9eH%?FhOSskPp?JevfC_ua)$a0CWCqe>vzS;HE?N+eeVkATw zd6tCMxygx2+p&K3+R=kcQ_R&1&C*4=Nxk)AV;0vOCZV1 z=7dH4x@j3pDrH_&`%$fZCnRAcnt>-yTxPkxJ`Voe3mn4p#R>aH&Z*=)4zjYLr}83( zt}L-|fQ$-YCCL85vWveAL8~pESlMADp25fuSSTLbST4eWhn8wzGLP2QC?c${_2L)N z2^W2lYpFR*tpt`F>0XA80z+PFjEyL-o7G;w^vUFbjml`~vMcG6 z)OHQ&YBn2ct{@!@AEa$8P`?~QDE-PDw9wT*b?)vE)qT;heqn(ueg!+jT+|L*Bt|Ck zzG_1X=KnY9=gNR>qvn9^eI!&`3s!CdZOt2rCQJxG z-*2iwF8)xAg5d7>jl#0dDRV(24h4hcB-6m$@M>0yE4_) zT2s^%ca=ofa?DPfKQX3i6KO$jx+=(WXayZ3~vW@!VJ50~jtW+smz`WY&C zb{N#Z3Ro$sGdaQvtcK-z{M04^cV!+OYAvuObJpuJ-}pyijpBe_q|T&si3odf`?(2+ zF$sAa!|d`>Vvp|}m7Y>14C8rE80PJDbaRhGYUR^RP#J7x1oNmVtag{n69rk7nUdUf zs?(lThog(tS_mCR*K>pAqSgFb9Hs>B+RjAQ%evo8yScVi$SjB#U&+q_V=vb6{m(5X zF|JJe{?jTZE(6Ndt0DcPN8Fpbr^{$%qhC5*0&BU=ybjwDCxa~xDlPglLn0$TrmD*D zf}8x(Q<=vXe(9uaoupPhxDs{=9K&o-Y;P7R8Dk$OXn>F zU1e%dTeia?wlsrrClb_qu|lrPw5P6$YTFAb$5E~m`f_$(-i2RnlU3oizG=!cwH^K2 zBBX6m*81}!U_E_N5o<^Pa%_Kh&tTww*%Hs5|Z zF(2tebj=_2$f;W|Tdrz5J!hZ#0=5o~P$ZKFzO5;;GiZN4`uZ=_AWIY^j>um$qxc&y z97>HAknmW5iC=m~LX!!-AHsgi{ySz@@beD%nAecv*UpbDH@jhxL29;zHbUMh0}UR< zbi2VY(?LXM|9;x`#KIir?85%iqPh((Ta7cJ9&x|^Drj_~id zQT7vgfHC-K6Aq2)dvr_2G&y6jPZUXLX+lHUP;)MH#TWks@rTgE;U&f{>@6uvQ5w; zwd+22Kc4D5{ygqJ&UU^`<9qD=eET+mmdIby+sv;S&Rbstx*b31&Ts4Xy=6R5)<=P3 z8%Xw1x4Pe}e?C66*&@LK1`Ej#NDU(4cCQZjfJO9B40|J=5`uUdKPG_)upt{X51+Tib zb}V|k<)$rwbE8GS_h;dChW-0g^!Lr@&HzE`=lmaj^M9fzZQg$s9X{CoivHzqkTV1L z)udJqW3_vkz9ffrLU1Dn>PdExt`3bJg?-c9RE2q6VGD0-t4CmIZHJPY9)z~oE(~5~ zwX~l>mNjtKKp|x=(|efF2yoCuf=S9a>+6-23+2Wvx0>10c2_;C4T6r-!h(m|kAztA z`bdI5yLpZY@{yyPs}+mIL#FQ?IT<7Z?KZCTxA81cIiC9Ht=XDeu9u31Y>*?-!=;|f z4PLUI&GBi5)VrDNP35pGLw;qNS42?%nWE536*cYD$$t=!rB{tGGCuLlKp)T~+040+VZ_8u#{iq|@iN_eZ zAPo6dN98%gTM`TLC_-m}coV4+DsE!l6VlwMO6`(Z%E>w)T;^WfHC=uxsk~-`_^HP@ zxIDf@0Ru9VF-?|2f;F7m*B;&=3HwfMVCNF+?gGp$TNHYxG61K#RT5XR1`xPpSiQ z1UBZxV~Wo4XLHM#C0hF29fb;UO=Oy9)<`rhSkc!!0KscS&cxZgieCI8s`Q}2wFU-J zuMVwtt@dD=rLrRpLq0-?+NkP6rG(m-%wDx4y1V=to)UfwivM z5&9&xK=V-c%qgIVB9aNzE7N6AO!@O}izD@15>ekH)4*kRsa7iuOMYRZMc#Lht`0#y z+Z?P2a71w{(WIIKFN>{~R21W3P}-GXou96WVBgg=`1uuLwalyIUCHYLekF54lCKAZ zDuxA)

v}3t-S&s*4uUqT3>u;D#QDNh&!OI@I^7(NDchgTSCbSTCI5bWMEs~tpwilLVo2i#Y66hw>XqlhOp z2bPiE%pt=iwdmt+R{&!e3T4^@yh;+)(b25#9S<_pQ};kI_9e%Zc}aAvFG;aW=@`;FmYq?M(TM}V70FWME0{1mf_RqEo67{njIeCKSuW=>&0(Zgl^ zss%Gk#8=TJbt4cE@es-AsSr+-DFyF3MV^R{E)`@}_R)zFi*IhtP}|nN6V`}|>`b<* zmsCGANx*8HYH-EHidJF{X^cz$wd4?03TV@_t_~CT8dCR$i_yC%ESsd_mY=cXu&F(g zRQ(Bm%J0P`54vjP>%HM+M*6^`KmOtZ$TQPH~SzxkB;x^Z*h;~ab#?;L9Nj{Ag^JI zw+tN5eR3t89*D?AO7q-dVqFxG#GcTHWJwUt1S~C09uw{y9jPB zB0Lz1Yb8s}1390u=%hEuof{2-poB3JQ%;RdxTU~ml2C#auF&{(#sX5{R|DM(D3AA;Y=+xs$40=#%}$<7?dNoKS$ACX`~0xAV6q zO?1YF7>kJ5g?-bLRPmRfTF0!a=jS8Mgm^^ z_p&fQwR2HhPEHpM*zQ*@Bq`MPkq}*J9Gb(&WO(C0+$J-R32R;CY|YU}Lsal=SPVlH zm}|mddfK1%N%%tz-(W zpw+**KW-bhe_)&6@=X4PWL?ORA{5v(Hnz$zwrXeY&UW-jJXCkC?6i25hE(L|(VLVG z7;o6wkp1fJDs*>T=+}+c(_{^5Ef4CxHU<8KzI$)q0a#;B2H!>EMz9Rm>ylPq1{y zNWn241Jr2d({NZ2=?+%;rfO!^B8d&Ay)or$u7+xPTM~2Ldj3qFCo{(mH~N;sR1dsc z5(Hw|a{o*r;0tgf9GUD80Wvi`v1%plJ9Y#tKePFse3Z<-~+|xClqWUwMD%nAr zDSaL+Di{r8hJsXvNVN+Od0||oDSTR(q&=ih23S9|+}XCq6gsZEULCQV)r5YiIa%#x zIVN2q!*YJKCM@mv{j&&eYpDMd^(--~Lh*8^jJMTV@aJA29Ri}`!+wXAey&BQ_?2JQ z)16L7^tMr%2NM5%a?QvOQI8p$mTA}fSxBCES#tX-xqF5t@6ZZ;%v#h=o6=p(jj>hP zT>$BCM(JfVD^u6+S=G=#?-)9hr+6Z5y7~wTo&Q$vm7lj-1_N){A6YCTv2ac%8+LU& zdKwlc2A0jt&{X>6J@p|kMTv6YCqCcmgq_uy7{i9>}FHia345m5Dg}ne2 z@gGHl@hRbI!%>2QE@c|+9S4 zu(Ky#xYT>N($VQ<%AP`*#9gg4m&ifufrczjdZSbqzFur)J^OwEvHwV!CF~FmG9O(P zLPTLTaj(Ts1!b4J`q}jKI~2ncqsNI=rxn}18#V{(;y6aQ3r~fNHqgcHjIJcKYUPRT z{=6ExT;n|jZIb`=RpxgWT;{mI zi9%N_>zN~KsnFKNm|p5zi2P+6X`b|;@UPsI`yFlB{I2Szl$*fQbo19{wIY{RCb|0f zZ-Fo=lYbKP;-zMngYD}c;~GTTnlxXMHEQ1Rl&2kKQ`ECy_P=!+lOwU`&4l_9r?BDX zyMqMERZMC^@!T;QUP*TMKSS2D`QJc~dqjM@5Vu5Ql7nzLAcCLY&JkJ?h~pHyp0#q{pk{$cEBA@_|XGr_O z|9vA|Ao~L-*gqff_hmEy1OLZ9-dv4{hw=?2_a8+Jjm8%J*Zk;x$?M61D6idW;224yg+2vhszIQ}-%3?g4IQxm^335uqoK-zf(ZoveI=d<3yTr(|G6i_bs0=e zz#ecR`}6A24e;jJo>9Fm==Y5&=mmEn*MI@c2`4F%P=bH1al}w^@PaGEu^7|cQDIOL z-?2b#0}w0HUlPHLp@p#zA3fjHOxjYb&QHeMO@-TgG+ z^rY$f!`F9TRFg9^kLf<|;&guF_>5tjRtVpuBxS7H>61`ox|HL~ZO2dw718wMBhm84 zJ^VPBtsvd_E8*ZU5}s}v`HoYTG2RF}8d7UV2Vl&6K=BzB>0qKY&+!2h(mZ-=^vMbJ zRpPbX-j{O#9g*8*xMpl*VI$Wv?lVa$Z_>-$(%7e`k}Xn1n%Ead&zmADPN{Q5E>XME zXWXKmfU_S|8VWR->xVT7%%#^3$u%F_v{1K%A4W)dtT z1L}+5TTh#FAUR(cii0LsLJmEOIzL69@#K~I!715wg)%~9EazsTFcE;WWw``MseVwmq?vi7~NlPHdYywv&l%+s4GU zZQGONWS-}|=f2N(uIu~1eyzQ$x>k2rb+&-i$L@Vn?W+8#31Ml>yc3 z4I^gj#5ELvWIr_<8O9C+qR1OBnl49f*JK=H@AMdm-n4jMWRcc;<&Rt@HPXNe<8M6D z*g{KVjua$R%QmxACl_X3{Hj$WUt!mWn3;qrUWCO|uq0oFcgLWd`K0hT{O<6HI|>z4v!;!v2Erd=vi zg(+*G?}W0@s)ASW&ysgRrTnk(@+syj@HzQWa!!axT1BiPoO19}KVB0-iXzW8K>RX< zib8|Zj$mpw`Qh?myaq$2=S_;sei6uZI7Xzxv{USR3wYi`H2^!NRfZ#Bx93aPA`M|j zxawo_%=UQ>e%X*^qItWi@Pn@Vnlyo~*sxkRVtRU_-C5b2-6*s98n+-Clnv4?_xUcr?g7dkCHW16w z$Zf7=(vV7FX8L*JAuldJKtFz9NoF^jH0r#0jxu5RXgSr8g~V7yM{QK&VFvvO6k=qi zHbKxsW{1>v>x%gU68IJO!sC`y+E6@kx7)W5!R_{LL%XxB-J|NZ3HW|Z4-&QWNTr4g zs}?~LD1f1q?AbB`OtPRVXZH&K6Fgp$&MxtUy4hmqdbLYoKpACJ{<|hj)xqq?w>hSr-snb)ipAF5F?iSY~H1b%)my$i1 zIiq~s6Ua(Uzh=4dP`lemP7#F0ot^aNx0~8|ktP{IXPhhvG*sxW`35SI#vD7w6QoG< znURmuYEdCWC&iIzqEjQ5$y7!7U4NwzsL-=HJHDN<7;dSN33pg|BI&*`bKZXzahdwC zbG46c@}ub%oe^6m6ASIZ%$VKUySe2L$dxItECxQbk$Vk<)6XUCC;d*K%n!qeY|5of zW-I4Tt@hqEpt`ne9@`;iD{E2Pw+~tb67QZnRPGhrNFmyF=7S|(&FkUwnz#$5Rn;r8 zG;if8bzAPuLS}g5j3hM2)93c%w5STrfaBQj>oSe@DXau2m$BphepgGDEq+QX%>@Z5 zmI!~&%tPNiX3Dxao?GRdXZn*1Joea)6n^+sBqVP$1AU_=n9{0+e=J6{n4Wqs7f^ic ze)RZ~d~qcnmzg0h0A!Ksk^%<5@$!LWZ|ys#i8bl?0f_5Ter2fTil#cGBV)}Wl6M%Y z_HwOV_@C;dmB2hS%!t#eG0C6pKr)PUni)A19;({T!=wHlb?zR3Ql5Ny zU3W=j0*-u9pEwC@3K@4?t-lA+gj%ppZ;~nGn5rM>yRPrK>W)!eHe-9R^3}8Q)>gX zw6s(5fpt~AaZ-*TMU7D{lnpi~*^=JF@YNU26o_^<#KkP-64~3VI4Aw8pVER_hyph? z-yJ#?FD$aW&qL3(&nY}>AmGp>&B~82=Wa8WPuN5>E!IXk7%$U>bCx#MUBR7TEssDQ zzhodCsv&Fp4*aGU_ynt3sbGWh{ak{Ly~yRR7x?2BynzCPJA%8s=Ky8fS1zj?s@ksQ zpFo#@TWZ&0JT4_q$-bs*JUE2iHf;E<>>x-xvA)+U#K@m%X%sCneb`?a^S2?iI zgR!~YU&O}me`WpL6`Q}}YR}uI^pxtE5CLZH++nav?UV;0c_s%wk%p}RF?}55f%a<<5Z4<4f8I5LCy*cX^UTmcs7>HN?>rF)>GQ+EhgV8SSolbSbI+-Yq2+Yt4sv`y%n7Gq7Hgr$%y$xd*7==h; z{7fsw3a;Gb$0pv}fVx}D(}ij5$pGMa8#kry8WmL!yQXc15pTTp)Wk}kV>1&Bn-oI4 zTau-$?$(E>qiBKUBi_RSo;tFbF?yJp`FmdnCReJo8;-PO+V@;w=1OuHs*lyt{(7Qw zNBedd{k_)9hUiCew4XWE`3C8W%nL7j3|55U!W82iMoXv_nAbE;2;i>Ynh?lYE!p9_ zkoe{g%?cNCb4nG0?-5_#||ZtHLH8(F!m3BZH!t3KEw^pTUF8(vM?k!MpF^S3*E zl;#O=EoH4?s!BsNRb+y0XkQ?8D8i1NFEwSgNPm<;0s{B@M}k<_D~oc&R#eJV+k(rBCq^&Q>I5qn!_z zK9EjNLZ7o_j99v6I^M*iP6n;}-!RJgJgoI@EKUngzI=OW-Vi&w)+t-m_s=_`r{l6U!s2 z0_J{ksLyPv&TKhE4ct*5_VknK7bJNt$$di`X63%p$%s)B6U(JCM9xYOh#DRvCx`j+ zdQ!(klU+vj6TB7GRx39Il47B{Ya^;UpxIF%_--G~9_i!Y1LChh__qxBKXyW7kiUvM ztTeyd=fyLFB zM(>${3GE}!r}C-Sr?d)l-&Pp3K2kFqC+Z$g7Ntf1Y+6{2e2pLa(m-y~L6&+tI9TlJ{Gpok*P`r#bIwIaDtPo&0`lZxq`6C06P> zr*$%45=bY1sk}q<0rMiUSk)7)@EL`?6vcu9wai*nBZFr;5BuV{!&{Krxp!0oZ|ATA zM=)Dc7G@qKFVgdO4=gDc8%w4JC_BQ9zF0!m^-XcG{6f2FfWb<9N8)+^Ez3UtC=oux zUclG+%iAyvlUnw7!1)H=mGtDer*<<1&hrgzegFg_- z`#;bsZ%RqS(0?^3yc%w%*4Q(_flY4|lY>2)(U7FdEu048Lkdw3kgXg-;mulwKRj0i zZ+17Ts3he%uo!m675Tz1JC6hXM-8)Nl8qVQRaGC{*1xJkWApd(9?mxfd=hv6W|7C9es}j%7z=RFJK|q{71DS;XszmCfl3h5A zq%I=%ufPfwNkv>`)OYuTi94xTWK5`SBYrYYmJyHvQE(JnF=7%#BLSADQsY8cPOR3F ziy-3t-biX7D7A~V{;y&|nu?hW*u@L4q|0Ain}=)GUlq8e(4^+qJYF^&x^6Qr(^r|Z zK5nPUL4vL%u%(A?87BJN@aarV2ZS3l5A^(k{eVFx3?=#!{z2i`g?8fob)kzNaD3P; z+(r5#f=EfqW2Mcx{0Zybs@YN$QzI8~W?ZueaRr3TZsS_44rsILLug==&gemCq*i>F zben{a9PYM%rdaX8(KM(ipXi;0nuD4F=^R3M~Ri1J!;L8{Jk{`pb+E(Dt^L? zbwDuYYE@bhYK@K3eab0v+`*e0wAn{mKb*9B?XR%bdr&0$8WGDu_|M`rE`#cP*zQq~Bs|(?B=Kh{|6FaBTJkp0 zC1qQUwaHMWGC6a6ayl0Z-D*US+vA9o9q2r>88`G`qQ$nPK$Cf1kh`V~7@N7Lbl?cf z&{C&M`+~K;>`$febFUIidr6R4lC0Ka0sz33=ACn@VeKsq_ko1Oe4Z!a=WYIQ{*9w> zwX(g7SG?AvrElTS1qD|NQYZSY`~0y9-RP(oMl6Y!RiRYKCB!|ZE&r&}mF(1S90<2m z3tOHQdVKVIol_=Tc~IQ(m^}&|uZW>E&wp`csxlAqoGNqi8uSIN-ujTa(J6Y;BlEPR zxzqrjo~n4$ImIyp0uPMPjG(!XRJlEW*9hL}BBqwYbjjG>>^)UE*G2pFU7GOnrW^%= zdNXk-0gEQCTCy>ZFqRfo?b={<6?g+DPL)Kv`A}{zjpYspPR$EFCB%oq4}+zWTn@01 zaR8H~IfQ@4-Srzv8Ts<=xXZg@y-zizi037{?H!Y8*b&MecV@6A(0f!YfVr+rA)=WZ zmOuTiGz9CKY7Q(zkduRPF44}#V^;IBqFzaFOZ%&Q2dkdI!O(B`QEdc;Z07%Jj)>pWzHa+H zD>w;7FS-e?|3X2pa4F(rGX*66bm-tKoF0)bJ(;ji%zm3WEq=vZnb$0SojWQcww`L8 zc_VU^aLq$!?gQCDPR(Qu&G+Di&T* z;IHo5{#obZ{KGyW6dauAEg#w0a^gMMhdOuFT%Q@DfH^RPPowWLup^#cffqFoPks64 zL$7t>kO3Qym)6B+Kx+3J)2V!0NA%nPT{hkuS|0YzgJQEPdgMEP4$@=x8{gkj^=~fv zTdGiD{$r^6TZrnA&l%@FljLII|DWd2|Fj=fPgPMwQ9s};)x^;Ex64pj&F4sczF3vO zte6WGz=il|h|hX|-4+G$xVA!T+-WwI2|36&ZGD5_Ud&Oy)>sY!6t;g;m#LfVI08MJ7!yLQzz=mKmX0I7+?-|5e?xw*O%vkjH=n zj&WSVp0-KeX+EK%5Oyqk{EKWJ$e9dNwRI4TiS+4d_i*|D+Y5IbD+0 z>XE!{?0U~Yoxi0gyN<^(O4u(Rq-Q0-=4SJqr}zAa)9>t*)&O8Z$$jLJlF3@hGU-+<>ZR|a6u(EEg=}eEXohdHzC2CHG1K?Z~88I=7jz=1?I?;*n#YrLi z!pl0R?5+`Na7qF>Yr5S+B~O~ENDbEK0BS2n7YEav@Uwh^_daFjNgi`j8{Klv&4Cc8 zuWjF0GLa_i4At(ux<^*9`6^)}oDf`HqpSULi$>=FqtoL`2rh%=`F?;(3R8hPzl^1V zpKQG_#359sz7^fRW!ZS@Si3Nl`T_&_ei(L*Z6@qCnOwm-=;Q_u6_Lwu4DahlH;yx{A9P7EZdxU zTnA9+AhoI4OR)!_S_C5?cfESHEfzm_sFiw5p0s+TJ#)4sJqxxhJo~B(98sxU0nv&& z8XfxpR(zmQka?kgkqDh#Bw#~X!$QDC%R0=k1K1FTb%HTB5sXE)+1~42sc^%i#Er6N zcYw0tAAAMc%uTaGwT?ApjveU3=8&1gh)em4`8%2>2@S7I5~-d#*3GyAg|@?Kz^t6- zFS_{Qs)zCYLOCL((6c?X+=aM;>KIqMd#lVwtxRASm?4T;Y8T1lAxuyPbt#=vLJ2n~ zN3bv~tuB*n#>8miZ~FSwW_cc$K=}F@b&~C&tJ)H|x-(MBC%3?YhP>uiNu$|cnT~-E zbeHx>om>*&rvj!=?HI7~xV&BE_564mZzKUQE_N^A&bBQM(>$v2!+|VJYPt9sh{UbV zzysj1+O$&&Y@Rcch*fxHvdpw)S}789xl$MmItMpfZ1&~GzL{Gw9Kgcy5R+iIu(jP# z7P{P-h!$F1R*YMw?#~~CV|Z7Jz-n8fcvu#1p`mZzm4=*Z$Z!Q1@J{6D)J zhD*upw`Z1X^b_#&CoqlT$!5-S5237b2;hL4$AS=W3eu4jhamS#(;b;%t&55kMY{O= zcV<2KaZ%I77)Dp}n}}AaU&Fu4=FzM$Mcy2_n_(;5efQj)HOJB@smSJu@&5G7vQ6Qh z=Wa5-g4mEw9jnv9=)$L+3y_g&q?xA*L*%t`NP(r~6uF`Q0@M}jA9cWf$lakY(Sie0 z&2;pdgUwSZ(t1+4)K=|rSQV^SY0R;VS;BpOkU=8t#mOu=-qs;n|#~!nXSBy8$@&`?M#wLH?WljQ^E&qaobWDGi^e$3$ zhd}q@Dd&a8;CaV;4Po1-dX9uh7J&`St-_NOY9h5$^MRD!p;w`ONzKUzZxb_gH#Z>n|2Eye!q8M60#q3Xm zTXnQ}ENM5lxU_h%`4TrY%s19QfmDP|@}gP+j~M*Z@1ePLb z@0Y@=7WMgmG6YyPx8Xl-;h0~iIfZO(aMA?Giv|dqn+L`C0ra>&ClVB; zBi_R$x0Z!!#Y9J}`T#3QHfyZs_A%HqBemr;r`*Q7b{=`v13U|%*6D%HB(G7?N$>uC)}x_Yi+t!WWN;VP-_c(c`wnAR%e z;o({>Bh@D`^)K*qX;}UYH7^CR`pv+KvG5%c7W_Nm#%2Q#`=-VDwmwY8Pisa|-f6J2 z*zBzrqVTt%Wknd`f(Gkp^dP3d6n?=uQrs~+`hkjXbTws!>paj*@nlNn<63L0vbT*rqqhe}a`U!dUFEij;P|=`AiVSS_`Js$ zTrPP#y2v-;mu6=1X4(4Lj#hyW7vaYqkL$BCaDu!2-7tTI`o3^WxthCzN1M9>AX#yH zcd;|OWm&mHi&(fL`#>j6UC}3@xvRDz+J~x@j^hf)qJ|QzkJi1!y|_p3rpuG2;WX)& zxU5X(7Esnlpy2rWCRzW0GQy+>K|B?7OJeH`AkD`-mr<2f%$J~^9~uEm2k~PQ(v|6?Izf=YIZkJkiYQm$<@HF}nRVP($TTB?_vwJEq z>Z?JtcykFlkWQ~~`3m5bRN^{wtCxdP#frtsC=WxR3H+X?EJy~*+=``?=NFCSS*-3Q zDPD1AdxmO16fCpS6*#PT6luZKjIAjrTATt_uaMLgmOGU5FJTBRi4=k+%;=^2m}e>( zJ-7E;_GZ|JCfn;3M<(3%^^$GXVvimqul?;wX-lFOgbGnoO)H)h4)cO6n9Gw%5xy1n z?1jpg`K+UHfBOccT!#*Bp3K)(qkgk*b1bkLVb6%zFxJTRX_<)&0;-CBo0j^OtA1jp zI`Z^}#!K8**k`@c0YjiU(&J9>Yin4T-X;V8y&Lc{odM4By>t<@LTMU^gYJLhBU0uG zLpay?#9G~ml5973c#_>1WZE5V03DeIMkO>2{X{|*spAY(lgPsEqq}w?vT%zvQVDHtc_Zztzc)4W zUSF^szeEJ$v(s}{FQ|H9*A1HJKA7lvgF1+b;Q-+ryu?MP5K-PKXx?q-L~lVR6g;2^ zp5D|w&mNU|gSZ&d82<2p^96o==S+Pz8#}c52?0b|pCfS=F?Ajd40;V z_4$B{k?8`lUpvnntULa_HsE~00o{0@ zWy=`mck>_HbV|^_AzC5ayTKRckTx^&Ifja7N!4?h#God5BE}EKJpbN zvw@V{fG75?LhGpLiTKdBBAMRUU*xA>ke@t0Z9oY3v`Xg5HW*Nf*bNdR|rY?DRS}U1SnJ$wtWXKA+I;kX=JQ?Oa~>jGmd5O!$QD3@Qvas5I zznfIhU_q;{g{IYy2v|>367e}^kFnQu2|QFw>z#LKETZR%lbo$-aaw<g*qN-VYse)7IbG7vtoL_b})+qvxXf#j> z_FGxWJ4vsNkUzP0&n*C@=85R8RNP1zeBL5A$Cz4s(Xxi@qj;PY8;8v`XY5-#Q~8Y; zC?l}Tp7az|TaYH@eYL0S-geF8#~15QO=lK5(5#9KL1Q)X z?fZ(krJ3(_w0-(V$^}P4aGV)U^L`;KO}<8d6K+vCtc{WnnxPnsy;*MAoTzsxd8FCg`^KH&UqJp3CX|3c3vcuLER#JWDgWB=JE z=lm~t#3sLf18yp=DWSgC@(o!-(X4|uK$!h9PlA9ZT)#00Cl0C9UOC~+xE`-=;g=9& z7Bk^g!LrcIu=9KXmhQU5H`7RB6nPDG9AeTzPKZ?ksFN$IjAz;0v{A&^*ujmHK;E zs$Xa6`O0>x-)(_)P>BZ&6Hcb^lfb%j|M|jp)K%Aa7hLx+rz$hfN2$%k)88r3k5=Xt zb5^nn!N6$1joSp+HclcAX5`z#MI}kOG@B_@)`pNA$z7nEn0y}j4+6Y5NbXJA3*b%s zOt37OQv8J$Vyd@V9X^$stMnX&4%`k6YQCg=pw=~m5`%myC$(D7)AA}<?s^|IxNphBe0G9G@im)%4IU7gPU;?rJcNZ;Y!0> z(Q>(#D&E8*`Bwrb-BxLo1I0;l!UB|tvS;D0!<>>QQAU%}oRj5lqn~64Y*Ai>8AP~^ z0vzQ|*jeyiLKqjPv7PE5fOpHEt8zRe<284fE7L8bm4c!nS`^N>E$2#mmhpnqxcT%s z7?`#r)|_Y?>Pfd|5`ot6p;~C+7L50ZB0Q*r?yU6b7 z`LLL|#ja55)F_zFbWFt~LEUK~-(tntQQM`?xvt&tQue#5GZ%po@)={Fl`t^(40Yx3 z*+W_{ncy|hIcgK4z5iZVW7A&74xfh4AE=P&9flfthvGid{JSOuFeo0S8aw!k+07a; z&5SLohid~Ci&iH+wZJi$x&8ga$onik;NHg})nPl?=?^8-dc7j-bm&_K$x zAg2&LZ>gJN#BYkf)C}me3$HNjSNSMLlJDC2b$VnCu?>Rw1nG-7Mu{i6t&6z6{Mse1 zZI)?4gX!qWPa5tLuxTKRYQ6`6?ZU?lm-C07`+mMEhJ!8EPxNKWGZ2=nWjMXPGO+wv--sI=V z1U$chty45{%Ei&)XcVQkfcJ~t3?~Bne!5)sM{QYZ_If&+Z|nX={=f145Ay$Z92YUV zSm{1fUk|~PH_7FfTW^_1!>rWVIqsE&=2*f?RH9cw*#^fBCa#Wg{(bLcHT>+{N@>m|J z@%wXT)fNLpR%RJ#(p4^GJ1|ug3-pJu%?(*r?TW$2ozo6(l~452%4&g=U_hER!%Bj8 zS=_$3{4?P7Pgni)6aobW)dRz(P_z@3&`RdAc_jj~Ujtz^J1`JiJWO+MP`P6cOhP?Tyt-bS3gUhN3>_a#yH5k6v%z( zDR_0GAOps*WRIGme4|qPGj!ncRy=dhGi93ai+#R2u^Z)eR|NI&2UR6mqfs0N@edXoBD2Kciqe!avN|$iZZ*(dXekCqw3gLT zSRmEqbJWfVWf&xr8nUc_x=c!_p-Y{$jesMf(NVr}LVzFZtNqvEt{`vfhAfmr>V>R7aQrpea&+0<*x zi%R>=>5pmSjESJ0^!PY2Eqri4?_hJdTA)i@oCTj$dq;~Xz9vf8wl`P0Q6OC=@ShQ)97*L?tr8FT#=sgv(0$ihnDg8ymfd#LhwsYw;*F@rSlQXQNoob! zS6=2{PD?JW*nN|!s9)?i9vko8avu6v-H))lkIa}gG#uYRQcIHWFc)SQ1B^MXa^_(G zzfG<2aaX?J;BNRbJ|WdS)}!^QWA7S5J)l+8Taz0w^>fsJ*_KdLFqmGM`&^7Q0btlN z+FyqlM=~$9k(6(as6~3IfZ;VehRiRS6nu?PYYHL3|N8!AV9M4-^o{OhhaHN$#BAAx zrG_9UZJ6QqW=qog?gXY%q{J!KIS2n}z)1i%30vETBE&8AUR?pQCfr?R`-2>NxoQd+ zN`_TauX<*>(HY_rTg{oNl&rW$xAS!afBHf;#ku4yUjc39Jfq*1M5&g*EcKMZ=SMoi z`m6z({`G$rAO8)Xf0+9(urM92Zihcd+TNcdZSMbSwxoOxLg4=fmx`t;&gV>9Ml4yY z(1$Y6o|>XDD27Gs3#D9wq-NfiIVB1!No z=&Ru7#H359AP^#-2fc^ox;Nl|^W-)gCfs-p^=-jPEK^<8&IW{J8==3!2Eu ztr=o*`L`)f>zNeGICzI#pU7PoPxy*BE*m9nqX@I?{rcUFAjQ+m8j6`G;t>SFo#6}i?cL5XSKVa?Wa z*ZLD0X|-1g>3}^8G+Zg`NT!?@+rtOGNjSgcCSjXULrX4~Kvju(OsvpN`82(VtSnOQ zv4B>yU-~k$_x-LDs?CXcq)Z7l*T%50#tRJtI(2wZoHN_?Du-5@lO0l9;wZ#hc2)NB zS=>lj6`!?aCYMAua4Xz(Ol$SW%F2z=Iwl$FS`|Y|zhhn7U)S)DhStRklO)a4USO8H z-?h8j<)T*qAb2|jnXFS|eeYBpbugFrI|h1Aj~uxGmV;F%%XXZOmHL&-OOAw+!mQg% zergNDiiuQ2A(;@BMZUJc<~OSx##YlznVrBbOmoyEePCn z7_}%*m&|`>b7H1OteQzBPdl?kT!|JeQF@KE7m|kT^QKX_kXLEwEq4fJJc9hy zB*4*ZFr;;?UI_-VF2m827oC*_e+7&RfYe*A+0#9Qoc3pXs&*Tu_FU5O1ctA1E@D^S zPxFMBUc$QOp3}?Sk_fYg!~|`VS&^Uf`vjeuHU`I4&M*p1!?J4)XYA=a?)^w_Kat>s z-U1nMvWFHxgZHt~26>y#l4BOxF?Xz!iqM&g1q;==v0 zIKi+R)7omwuMv~zEmA!*G7cnBInko_b^|6z(1DKmj4dum2B7D)$$3fEcxi1Zv6j=F z=U9K2(erBd&9}m+qW#^cR|~u55UVR>6YfF~3A(u*wnqo8UV^0C-rp2nXj`}Te3r$x zbFHoQVB|85jL;;3wYf?vR|R&5tqI*6IEqQ0@3M@1c|B+e`p6g>oD2+#T5Z?AL`i$D2sPxwYx} z^|$VR>4~SBxf9hDcChxhge?XD` zGGtN%j&|9n^f>#J9{>HS1kA@4-QAHmEp~od@}L0fRG*fTyQChCF$Kl@HK2PoFK~ccw+h}{zUFL+#%U*gnEQ= zuzV{fMPrT3XZef`ZD>f`KEQB&*NH~%VO2X>ni~WFzL$RGY(l|7ba0*XsZ;n zQ^}4t@`skJ0qP=z0(zMQwyvIf|Du_IijtDDCrM%%Rj0C@V1H^@8bD**sio^A`A0Wc zkdT#E+wW?nCZ4{1y=H?a?yGdG3miB$h6S(pm{(pKB^Yx77QYdo(o!c9IxJu=h?KVLM0B!$7IFUZ@gjRD6Fvl8B}qW zqi->qpN>m%RX#XFzsIY-;6;B&GpWw3*;uDIfydRaT=hdE__T5K(orLAt;@ukGazD~ zXVmc`t&{dWx+f=?3Ae9z5XNxGGNADOsd3r6Pq{`L$fZ?yV;qgjBmn z4EU*SLsnl;sh{SGd-u!?UoTIi56l5z8@X~bi*)5lm~`#5hZhGV)`9zZA%v1DG~_bc z!!SnFm$zpDjRv^*s{3H)b=yv!;Z@7IOb!IR_ijD8+vw(0owHXwC)g7|wfe|0P1s$E z3l7+U?Pi?HUCU;6Z0*-`JLgjmT5z1`ES_O?*w(vIVQtt8*I*fjd|uf0L(BR=R$Xpx6!f&e#P-Id|-+Jc%OX=0Y3p8bOQj7xg9{ZTq8Z<*B@~?8h#c zIr=U+j4|LEMDF|jpP=*9s59(o5o!F9MiX}6ukm}g@E{Mu(r`Q_fBtqkNB;%Ss`-zb zw+IM#4bdObAgEFy2icGA0(X$(20L5B-+!Ebk($xW_>Ia-vgaVc8Niq->GAW~3o!Np z|6DY>9@ALj%2Nlq|BR13sjGk0^Cjq=Osc0oA~xGDwcR+jA2*CW8cU!)ByGxpWpbJg z2qmnFnwuAtEid*IYWyQ>f`ES<<6YoJIO&OWoZt5-Tx>g+6ffF&7d9nRO&D6|=qu`w zA=^GpMRCDy0GI&yp`{){39(m~zTEQoM`%9CL#+2#r!6*80V`? zDePj%U-2A;?1dGkI+KRDF}d7)w)wJ$?nAW5b%yHF68jPNQ%P$nm+^?Fh>eI3M%igy zJ4lNa4L11!)oU!2H`M9 z0XT4~iJm!gc9e)YcTqxe=FfU1-!iW=iyA#jKmFQv`}m=$1mA`Ek(4hSg#zWiYi}xR z0$gqngDKCL^Gj>VFb}XR=^xyKA&H+d30(JpZdF8h7w5O#^OrTu1@R`;C!Ml{_y~eO zs?rW+^sc|n>j&d4-)hJTFYZPyJ#YtEJP$Z{|7JBUk!dP))PZVPH%IbTqx{ot`{Rj< z-ME>PF2)g=@1a?gLuG(iMxVLhB;;q2Y>FAxMf=kJx>`bt+*gT@Lvu#!-$MA0s|DXU zfN#!#O~p!8^X8nefUJulx#vF&aW9c*fFRdE(`Y)Ap?9D|R9n?|Qzueq2Q4GbfCTGS ztE@5|{_wJR#Vns9SWRpZH_t0A_R^a|(Xv)sk^BeMjTu?_QK|AF%MqQekk`ru12cPs zN^|a1AH2rACHu4%qJ(K&)boV!gp$FK&jbDFM7q4c7ojbnGS8YLjio zOa?7)4}0{=9TM22E28oDXUL%K5lkn{g-O`)n6cBhQ3Q}&Z5J;x&@7Jva(vZELt8Zz z?d_&wu991)hF4-J?PJl=Qx-g{9DD_f$JbyzUS0OGj&pf}mX|sWn^z`8W&E&MAbjZJ z34BVZ_(Se9W6N$#clbV&mr7RWD5PESYS~ToC|IQ-|1urml_whe7ggc3)bX?l#h$3> zon>|}ue~F>;Sr`@V?|qdIc2M*7PuGW(29Vcw~xBkJ6!n_7joB>Cg`PTY{Z3Ra*;

jI_#-AhoQY$aX<=~aVr_0>2hgZ-X zEYGrxZjDdWOOz2^jBUtiMac`j`9TzmVgXLMOPIN1@Dz2;QL-c>H0XBzz!*M~RczNp z9r79<&wP=d%2?!nwNTa^49Zvp znAqTO(T=CemD`;bnbYS1w}Q&Y%&5I$u|JT1nc!b|2lS|hs+d<0m zviir{=U8gc9ajugz3(@r>ppR#cf zydy(J{ee+Bz+kkZkfe%tmUpn08smuqy$ppILAol5tWR;=H4G$>VIEjPoM9l{LGUM>9xv`t@GCqW0K%3`38E7`>>wgNQGq&sN2TZiMexYBUOrtYb zGiY|j9duOG_6%^sAlOBL#G&IA&M2iPF~YFf8kfJ1XG_xK73{^Ve#Ko~^rGnN%9#2U<4scGxz_d+Rq|B!c06wP?GvYT-i5|UL~IZ~jdFl&zG z`GHQGV|ol^S)*?gxKF#m*?xSJ^Q@^tDI?mP9KbAGhWYO2WB6j-MdD~>ljd!ZVAH>1 zsyUVxE3a?_;4$sy61Kr2d)kf)^I#$@dmSfVm6UV@HDC4ccu1B}I&&=~;Udl-_$^l% zaSVFwe3q~A&~CR$&bLoZJ?B^zxl#{Fw*NeL@#hP$rT!_{EYAQFQXBz4w_GCuCq+JJ zeiczLyJePwnVlOdRSPGmV&u(0ko#n*i2RF#<711X?m=OZ)h)=8bVdLv%H7zG5kq>B zI$n#AePin#NO|iYkd_lcsXzMC*D#Ta`A`oeIt9;eaj{gN<|=-sx)?{IrNmfJ@5o`H zvi3V5rd#Xqd-vYOZxX2n=iBJ5dS~*oF5*S^2xbn?``?n&E7e&Hliv8(O1w8p0}$;H z-`5FcG0eu@TP1ADl#A)!0()&JqR7uzINn4t`NyZ@$%j{2I^&NfQJTGUd)tOrK&g@o zmjS~%`Sdu3ltjM3{~YK32ZV_K-wt9^>;Z)^DEMDR-?Y)hDhKu-A9ellzRSg@PdoTa z()B4E?*Css|F5j~if)tXV?Yf8CQaF5NV*I-_xQ0Ywuonkz{g@PMK~lhW8!dFlj=nJ zLT=?C4$4iF_4c(a9ezw6PF6i4fE==q*v0J1{VdC&uw|r8Dpj4KS{$PKx|^!6f=!Rt z{{;L7=CtvEYIx4+w%-~C>T=t95F9lq-hm3bs&SkMegQe=ZL=i3VPD+tUD&*U;gTlY zsyq?ftHe3OKyWr)T|FS`G2oC;uHmCVjyq5dkV|e)1mb`#_VnO;f0YnTCRffQ8~JQ3J|I z0Xwv`79>DM>hwFse&D!!x@C<^MTZhJUm%7~%uBBzb!>BPZfqETF86gXqRB$ql0>oo zbv2jAgy$ibNhb&M>q(Zb4oGsJD$Kx@e7X)BTEwma1jHDu!;f}!GS6Io*xhSF!t#8jl$0C09 zv3~rUn+SDcCF>&VDx&jNM2J2n9T2yOs%>pSYt&e7^m83qU(o?qW~jGBnN)XBxGc7j z)gAU4X%>OifE-m1dWw4&C? zU3Zb-?6~0!H%c?L5##sWMm@SYEA9@9{6fdl>6U0w8?=%s0aEViZhrEm4T!ih9KJ{+ zXdiJEbw(dkNz-;+XtnXf3QyPU;vC#g%ATgV z`!ar123A3G*Zak}lXJ1BR zV-Y(G=9I4a-nv zekAuQW2cx7`q&1>=vxPv=HC7G4nTE_5PKrG5Av!_1{vylK#Bwxx|~80Hhn^0=gLh1 zofzyDH=x`?-GB3822K)LIT8{WJ|cbSB*Ubbz=LmL&7FNE10p^fBh<8brF$uG1fZBn z{BdMJopVU>W{=U;7kh_8xw&j6{qRlrLw>NYsuuHcbDEwFicU!4U#@FcQl*cz9h&0= zJ4p9XO-PQ6^)KYASvlqNCPu-7r&a$_nL@hrPnkwDk$$YZ&a_;VZPE$uPDC*T_u& zDtIE0IPEQ9&if7ZI3D~1cS{pg?uZ-?q+Ycf?WNQ8YbA;5a9bjat=9u(>>8$HvjCE| z&B6)6&vby$7P^QApW}mR&W~aNfUS{e4;_98FD+#)H;fYUd zTa$@x+qP}nIk7RZt%99L;d!R^M5M`;s1(6lxn@I;;&$SFz8r^ zY1xG!wxSH-9H>e0otx}K0^;QDuwj^>_c7MXv`a9n={39(5xC?RB(oy2Is6aLny=)5 zRu($z6}qOp^5xTPtkT(;9_z%f*!TZLf!FQMy`7y{ELf|%^QTwKTduS3J+3{j9(%H#$$UcNU3WafP9i&B4Bn@1a4A?@1@}nsYX59&5ZSKQECkdTt z=~r(|_#Vqz!(Il>nTfq|6+c+Tup73#L}){#J=&+r=(;cz37O#)bv`O3W+OSv%P?cp zg($A0->DN)s?dvN}R{ONXX@+L`Llx>l2mA@Mg<{p#LaN+Fe8>L9Q-U3S zp7rHH>*ycVwxzeZFME&~xd!pT@-R}|MP#$!= zPCzDP{HQRo_4*D=vs7Jti^HF=ZrN#>AXKa9hb`2nGITjH$M5624p*G?rc9jt#;e6I zVp+7Q?`Ek2g&SpWAbOfL>SVf&m4xJ{O2>zGCfC7>_Y2RHG9{tZ6y#$S@Z$L*1D|r2 zKNSp`JXnZG3}$g5GoadHaZViR*^(;75#Y%!`{r*Fj{S|TGlJd4Wa%LP3&|a}nRRI? zh@dgNPHI~RcTZR~Om~S}xibxw@1g{yx`Qwb z`JHWq@}1R&uO!~^dMNR;OIjcUaQ@|U%x)Qby?bl%6?#30u{ev2=BusR)QgKY8>jjhwxwUVVI zvPS`R@`74#tGvp6UW>4pjK92;^US8akrwgip7kSd^4RZ)lfQC9OMk9bzO=(j^Kr@rXvT)%|n_J#X|8cLp=UHE$~WJ`nHb=i*@4?{^pRhz7tA zU!8lyaXfCFd!~wJC-9QPYm9LcY;Z_|?EIJSgCq37qsOs+FbpVtL(ABDEs zrbZGudGhFw$6P(>$d^1NdmdFO&i^J#2i5(4s>1C&C`qKBGIoxl{^!SEd&Jv8oZ1NB zu1fg_)9N!LTOZM@9X-Q$yjhocKD*;69J!*=?0l4*!#~VUT<6|EivnyIQ%|Am$Bxkt zzNSf`d@`Mu1?ZIoA^UP`{56W?{$WCtrPb%*jMcI){SozTz?q-*PsOk;(biv;3hvfX9Jk#uZOZe1y%~S<0@8MSgw%KWf{j1T zQ#7EktV=O=eZhop$tr|<&W%pV{(R^S? zgC3PQR_{pPdWN+oDXrep{&rFu_~K^kN-=YI!0M(@rE$@@c1r>!B)h_0Zh2^i@K&!HPLL3c=Jqy&Os5oJ*Tf0SdcXq+y#p2kA|Gm(QMXgyf>BoL zQ{VV}Mx`Bb$n#2|ttm}$W7RbzYK<;T4EVvk8=o3Ns@Ge_$>B{>l(C@jV10hbn3+EM zeP-spsr2>a{T>4*pwPz}m7{402zk?0FB$Z=w(ZsF`u(BW?)hl@eU!mIM?0jbL>2(2 zPNzaHSS%3`FX4>(=u)Gt=Q0O!Wii|nlW>MRhMv*Lxl!9XCt=tt$4zjajw?7ky5dXP zq1GO2w-0iOlQc|5p6&S*d9r@gjy;l{kJA;?Anw&o+F}M0s}gktD$(tmQV@5xi=193 zy*Souzr~LO6T|`xQPv}2xDP2TSL6%hFaqJm%>Dqb_KFun?gUCHWf_Ue@nE>^*6W<; zt`U~hNr!yOo>Q+p|pde?bVt;`%r-XBnbgFF22LN@9#jVt6%SiuqOWsWlffn zjqI3MoMBP1{)DV^m_Ak87ZZ5w%uN9^zb51JfMVC~9q_K_{4Yk{|2)b6??XNz5XorZ%XPN>^`pp{>WqY* z9`Jw&NDr*!#{S>HWu>YcHW~y~KFD%j@7DM{0_}xIf{b;aYK|u_z=5W*0be^fGtj)X@>4lZ}7)t>k z&ucjh@1T)2c^n_ZSV(%Lav^Ap&DkVM-=+o`qWhY$BgPo<%X=vBfkLZ!S5YDdJEQIn zA@d&oNIEY2QigDLEQKxiGf&DM#h4Z!dDuT_8Ob=TC<$;`15A_JsgpaYAGFTCy#R&9+6sw!Sp#CXS3@PH>kIjLqtWtck|#va?K9Dbr_}^ZR#5FRWaPy^w;2$tOiEO$EbCL8H->oi#!nICNE zwMinGlMvJ1L?qH;&1G>+59hC>beecOw_qAc3mwp;WU@u1*N51C!bopILWJ+P^h2TJ ziICXKEF{g9Sg+5q$!80E8G{@cE;G4nod@$@{LYHMU9WQgKFxNVZ2Mka+FtWLf`7Z; zl>-7=2X;btzdwPg0xu$Z{|+LH?uNC2-#os(DFZ&h0;ZM(!}}lvVF!qwsE*%HNZv3$ z!YY7A4M5ZL9Z|9ODdbPoV9FaT&yVEn&wbC&>807s7vi!2)ZovsPt)`C&93R`PuIUl zOha;0R1llzE_q zbQN0ASbiS*vJZ#z0!N4{!M#DKiFhT#2xt4ww2in<=tdr^ysVi(WO5GmA5x31BGBSD zOgv^Y81rTXikfPLoF2Vdq~$V7_i~HKo>m*Fh6RZ6Z@my{E}?>jRZ6p?%=!3$JD_|1 zrAQsjEtb@))dCG~V4$$177=?fS;rxtC$lJVSd=0QPt{fWDmx@^K#nD$-Xp%hJ1Yxk zUUigay@Ik2UIO0c?P1GC;|EGm2QX=02KVQj+=+Az?}U=b_?s7lHWS_a9K9JM&sajB zfn1XXOI!#bE-R1mxEaoswqZFMEXYcR>K{u}#RxuOS80fvcR5Q3m$gvie*Pc0w{<37 zA;x4M5nh>Ns>AGW%Ehdidi_RtrPP_f5W*%7=R88gVQn2z-5)NV51VT#J%F}RX3X|h zBnUB7m0h|lma#2tv*t&}#BoUHkbQ#6W#G{Ljv?rnS^fkQr;FjY`elwIQgbC4jBLhd zkaCaI4l80Xw$MIK+tN1*3P{#G!L}|*7MzJxv3uUD5JSvXp*!gd_h$pK#J*o_v>`As z$DD^eQmQA#VPqQPsgUh9SHRig{(MG7Pu=EtRnK1)WaY!57`Kf8b5|0^MKUhsLeb|K zm7#Tbm2$rKLRi^y;f1L(IlZNSUwowL6++uAkC<6oEEZXt>+Jcu?UQcfMvpA78DRBE zBxwpHwHC`>jZSjax-w_b`q^7Fqg^JYNaW`DdPth-S z3e#j_Hnf-MuhQPSWRc@IAh_Y3a z$>Si&1D)2NQ>Wg724QpN(>+hIl@~fRJKd`EQ&t@%0@Lelc*XuEw2?eV675};=Y-=a zhijB-209TjBFWQ-$-vmPF#^}p&c7X8$cOvKEP4*G8wCz=c-qdC?%q>t+vL@CaPBF` zol5)&gz;{$s#>bNaTSgyYl!K?X2_$G{tT6WXIK4dGT9oc z?;U3*I*afw46p581660c^OGE{7SUY`A}%D1;B2eT^?#J>G=aNe!?h~8j7-JHuddwc zbod8nQ64vGe@K3ltE#d|x|N~sIMC;g6J97%xaj0vGD;b4mX}IhaIs}kFQAU9Dxg&5 zEneV82VV!N!o?myL*;?(>#GYgVVuFC%X2C!FlQF2%R~n*P$e-O!-JA_I@c9CevbV- z@-Q|3*GlV!=m5MwXn}h$$q&wUV%6Bm)@)XhwA~3_;lt#oTED*}e*8cHy3ms+?r4P3WJBl&JH0Yc8J3IT) z*)<2RE3pE(?-lb2(c!8PI>L#wGz2_>Im4Jhc^7dG;|a*&CmF=a9r-x$4%aoa#JQ)t z>8}*TI2eK9g=wJbguHz?+6~tgd5OZw>zp6_x_1Bs3o;Bpoy5;iUj6!Hs2;?OG)B=SF)Wg*C(=H}64VuHU5A&6S1=@hNYC1CV)%VAPs2S6$KVMR z7MoG|Pz!u8mY0pxF8Uz=rl@jMW&|Gp2 zlcrJez@Vw@#`3acr?H6MwTHZ>DP}mDL+YCN69i0MkZJfS=brw^lI$oLr);06Sk}~R zT)yNNpe4V!pl8~`!r~no-$J9|#Yt_3kR>x}r5>MV6F5%pD3E$M;B~u+p7D?p)SrvK z5z&VDfSkO49&!KMs^wN^Babsxj>=lsHlvhBe=<{2AzAtKarB_MkTTAF-r2Zcb=>nwQI zu7zQK$NaRJv;;zz;Mr{U85z`2bN0Cf;cXy?BdDLsSbbif4qpzVDJ5h-c$Ef1*Bz4@ zR(UMF!z6@n8i5I^e~lFWvgSn1yF7Fuo={}f2z038Y; zmJ5S?3w{~zp7w1vIELsd!iX}Cm3|ZJT=S13ih46!}VHap^ zMb|&TI)@R%4`*$l5r2PkT1BRb^EN`;FC)J=%{RZH-lN5Tj0&m>Lml?A z#uJtbw1D$iNZ#q#EVoO-r%z&tIZOdkcLe3tRwiYK9jYAExJ^MVGLctP?nPFZqQ|jl z?()$U8gUxdTjSMR!AZ%3wd>o+WM;L_B}1Iut z3d_=|1eapgkltEE4)ksWGf=Qc?0#h&BV)6c^-P!bkd(t#bPgf8)QPJrRl)(YDu2Qk z>dd9B6~-l>WnT!<%J_R&8S^%ax)*FIW7kmj|Xm$PtbrEx(|>NrT~(k4Ow%Z3k1+ii0GDZ z+BR*soSIZYgL#4*eyK7$#ngf|EZ#p<-y3lcH-YVt6}ciCLk5iLqq(HRcEGOhdXDSY z1wI$R{^h#$iuI3?Z9}s&N*UBQkLCOE1#LRV?ur@^0{+aywvgIO*aKyI>M#W`4WHrm z#a4ZcG67bjropp0;JyFwn-XgZ*=9P{ai7Xg!yD2WO=*GIeT;NT{I?U4z`OM9AHkE@ zie1hHJjFSI9pwUE;+%-aJAW)O&j>oj{$w;t~br;b&LIrTF!MgOFg5= zJsf2)$|VTGu^eNbndKVknEUE@M%KRO7sopT!wC=G2>iB-7FiQ!Fp=B>gipDo%iCCU z)47+T)Zn>>aouD8%Emi;aF0CW$R9<6Y{AwkJ=NZIZerhn zGjpag(6Pvr5021Y#o_`aN3%;wy?+>b%PZfy)$}?--x9`%YXwHydF*{7MLgH%mVCj< za-r9-ne(!d+ER`FO4Gp1_Pciz={}3PE3O_m9Q-a{B;nYY0qUy}&f3`j`e>BzeEsB0V7mpkCZX?<-v_|UefiiOfHG_n1i5?neDS>-_40rDAh%+wlzZPUIxNLwgZH_AndJZrDb=#Lq4sYNk(*RD1oo>Irm%h5au)p zLe#PVsBbQaW@vq5e6vs9OdtRyd&`<;YLOW=BCDzVArMd24H4)#r~>>Li@o{hG94nY zZ-UI`+*`h^We`=|h3u1(Ui-AECU6hq@WBvUBez`QNs9Nuooee>YFPiCT%AM#*X@aqwF5D`&Me zG>b!be(Z5(-6R4x*x8B69F7wJx%g9E1)1ygmX#(Fei{AnyTCzg`ZBTwHLV6`&-G z1p)d0!P)sNID8-~Fpc;w* zVm@qNhm$SJuswx#omDLmCLXY&mF54T2HS$ldpF^Hfe(6Jx zQhZ|j#%KvjT04*Z$iejpM5RK+z~Zw}*neAXAL86hdfP?{y&l@Ef#fLewbEL18WpHn?osa zd~w(+h9-9J*9{73F+ys~Rw}sJ3w4umbaI5PM7A_@&ML@zf+m;vZ9JhtL*&ipd(52h z(7y5gn~h}%8a)N1V;TTG>ic(=R+zCTmQvH#&@%e&1ue%bLDv15XrKe4A?vJa`D^xEz{#`4?M? zi_1Q}%XVM7Pxdk;vd>$zOlZ4aT=utFi2N?K?3X{!dzoHAU*wq<-C~0l7y$+r8P`6} zk6|!FMAh1GYRy5q+HmbLex6OkY2&M3_UmG0gTa~%>D1{}^K58r#uQ>EydXb~5Wy-D zovQ9d11}hPj-whq;5J?}f_yHDWCL^dsMSd-r||X4wX7Al?5u?xG2fn+7F(Ppfsxxc zy!%)>+}e}&MD#U+KU4|PZA#gF24-VCQQb3Zy#d+8q2Xz@J1;FJ<65P!RG_O2d~Nq($Y^$EbbWqzDlPuz zELW#ZW&i%toD-fMVKa1D0zv^AdgQB4J~CGBqtcrm0}*RD6NO}RuMIC%)F{wQbxic^ zWN4nb+6w&6p$<`aB+K5H7EfV1HRjfSSs~3HUN(Y84ei*?B5I)ske2$70KDwgoN9#ngcaZqP~FqT zN@s_UtnTC95JZ%bnZpA&BMfRJ{UORh&4!81#_fP{oD~jIaftKmJrm&uNZ^@v$QO)* z*uT5XfcUaiMJT+%7XNH7S13tvz))EqekXobd^<}6Z1LLVH(05k!kb7OV?51=-jOZx zK?$Ls_3##eMXo91XltgAxU8)R*=L2 z&C?d&XI?}hd4uxrRv|ICd3TDe9QzjqFZW4jXCFw)8>U>(KAU&KIfx}T1$~o;j(RYO z1p$}tk9-av?O5C1>Y!&mNq!J;adXPkM4Ru@%B7SyqnUXgh%vJe*~HDa!!cQK$}?T` zy-iU#o4nj(mOHwZK(w-xfWO=gInm-K4^)_dBFsbr2$CdcE*^{y#pKeA#Z#2 z87&vsEF2O3Z2%j z5jhPWHtiv%O7!M`6X0x*i#^ZwSK+wd7wslPVC>-KX8{o!afHG-tG*_kM|81$TKPMZ z-yww#)f%a`_KP?gyglCR$wDzht%u~tMBAxz00;Sfx^Pe@TF280qrZKNe(`Syh3Z9T zSC;@mYzE`64Hf8;iO)TmgL~S8o8oJ{_OQy9QEz*F1(~>Ni2Yi!0=d zWCuO`!C@TBTTKyH%Uu#4P0BCMju?P$^w?Hp*HmJR#+hf;lz7z`)S&ILe6X3`PdAh| zZe3>bNzLJP>W!hL?bZt}I!~0D;_jg|^1Y6BZ_Pde4&s3)pf*Qr|dY5wYMdL6efCnviy zx0Z-|cYr6;+1G-f&&MwoMBZ1~l1OMGWj4q!V_+}8BYO5IXI~;EK7Wel6#|E*I9*S5 z?}7HAg2vnV-G}T`0Wf=vfNQch%sqZ$|9qn1B^Uru^oe&M0zgda7kdLcAc*o2bwfR1 z3CJh+3%X$p1(N$k-Cz!^06vAE7za23JK$E3L1b0{2FNzW$%RJ$L)>W)+2MvG^HD#x zfnQ=sf&unn6Sja~Vhp33-_$gC{x0tWkOkTB_+Sz|U*i|1F%ch@EyT^RP^d5<1T|h5 zCzuIiPb<_IKnW-XjFKWq{t(3pVnQ>fS>;rzGc%e{$aoJUR2XFzCs2U{IVqEBZbfHbCGs0Zhq>=S-q@i=cMkLK-)e=^s0P<>Kt(dv1 zMV@9rSLpR6sC-N%`N@Q{!^N&h%`H(F}lI~&;?8j=`^3M0IxpZKrAh>CLX z?aZz*7zCO}p)6i5rKtysF`#_xu2 z6Bu&!tO8ouB-Z6eFn?k;n|lx5o;yteDEpY8J0z+(7EJ5{%l|bv4fx!R#6c>iz1S|& zOo-xLewzXpJR(g5b)ekSUu3hRG};8;hW?yx`D^?w;M?t$*OMPGq+13P8arY=js#&c zINrz(Wa3K>X9^9tP9BD%Z)%&STjH#uBtV8TrMJpl7;DNZA`GYJ-H9CPGzpiF?e&0q z&^4CiB5S6qV#k9QMvcy`UzbvoC???pXB@C-A({$pw6LWWrZc098XBx3Kjz{zWD^GV z#gw&L(HXNen7sr2el1({?G!_uQe*8H-4cxHnKg%Y@)Qi#u=P+kVk)NM?2Zj4tyMW& zQOL>v?CbK=m3}}Hr@N$0z#isUktDB@XU>p0(r7U*FEUS+HdrJ=bg9K_@^WvU#3NX? zAicW4{m}V`mtt}MTW4Qsc#7?hH4M#K1WqZKEI_lDOKtsN>~M6rmVDmSKL&ysEKwyC6Ib2ahti;3j4{ueiIG)_6?D= zm=!f%VxPz!o)R4tENb6=+&Vgm4T)|qK}Vn+*QIW8MZod<{heNZ9EaLnzI}_P>8CQ% ztP-40B`Y2fx!59O0e*Ts+O&0?t8OxKS1iJKsjU`=j_pYO_?n=Hz6FN7Y7*qpce!Uy z&8=aNY!I`7m8hcUwO}uH5e(8NnLt;hXKDXcj+NMMk2)T;p%K>ZuRa!Ee(bZEbunhC zV5C&flpCDWZXZ0VicV1RGzA@;BSZSecFRF!#vlrw@d zG9MO%mgofj0X`u-AF;;Aq7nI+Phyo&G zK3@J$^=bH9QmdB={@ATP8v{FvVB`DXOiq6J!yJo(S*7e!{Zz}c23cFJbH!d}#7E2Z zr0=hv*M!!UpS*T_=r7&u_1-)>8k-&57-bSbk?X0Py-x>Qe+57L_GDz_T&S%-Z?w1N z`tz<+3wC%fT3NG<-H%`qHMkSy7V>$ZC`6lwDT`3XRI|=+zD$JX=fjs^{<$XO0lGP1mPH}Qa`vR-7j3pLC$R0`%Am=c1dz+v zQdCW8ywWC4ocBrNP+H78dVnXp6AoX)x-Te5ysermhi=u8;cx~ye0?gOu}OZfR;)eu z`LtP!4KhNFOS#d9M>T=Tc`>cbj3S&a698`qR~g4;S9zt>jINNZ?d*yh!fKRnPG4SH zg|5;c+HskAUFrMxuHW_;TnQ6o0;FwA6&`2a2it}dk7z9Z^`@`P1LSB}FX9b8uEVb!FfdPutEr)Dqz!e6iH*0i zGZmMWjn$_@#>Mpd3i#Zi@CEt7~sNP9%uwOpkGP_GV^-8?Sknz0>RS1 zad^dHam2kSTdCz&5vo&L1674P>ZsQ-;c?qJg~gU1S?thsgc5CvSuJ)kY=g!dac~9+ zM;4ea{zOYrWC0q{)7=?N)$iA*=Eq~!|2ZLm{MY7|w*z!iOio^bZio1a48M5*$>VoP%V65Dg*Cjdi|H~76BfFYS**bRG9 zfc!_XDma!}F+=We>okQX>omzuL-eC2LrEah!N2aAS-CoX`W*Hvl^7ggZ~1!pa$)Pq2cesO%X* zXa}I2L~Tw+zf-nTcO{^Dt9X&etXA~+e~9%!CmwJnPEui3{jYFApRgOZe&J&3yj$$&W`0WDa54oMtYNV6E02Ck_g<Lc0Zz^E_k)!^tWs)y<1;~Wr%Zym|1treY^ zXz9%01!X`W_xpU?=(+~n_)o)x6WKPw5pGzk`p<~MFi@W{)_J#`ULgm7 z;&v3Rb^+tQDOSMA82#6%*SV)Yfh0_+q|1mXOR91FPo~=|a_-JZWECPQen<<3JG5P? zeJBFOO^R<7uu97mE(p{xsL7<3G;i4o2H|KS)0Ja&Oqofm=xzbPIVSmn8ZLkqU^DR}buI@jU5TX{5dN!u z-&{bD6Wu?@)@|7>nfkv7;R?B_lKoha^JvFVRLpHfDwac4?(n>p?Siwdp1FSp6uQF<^_D+MU3fMR}ut5=&(%X%< z-DrN7%Phbo(V6;Z%9T4=hG+C=R`i>A8Ks^dr*su0fRMunsG@h8ig!cG?s-u-amU)U zhY{b(iI|~@jndTKIvc+RD`4J$!7@HXPIm*6k7;)i=k`grvl^vU!4IoswK?U|0$~N= z^uPjH%;0zzVU%8%0qBeXhUz5LG>UN}ssr^KUv{7=9qXh{ERQ2u*-zb&`yKT-ehIrv zENfFiwKZf^P-p1_am8gs^aH0XNKj*qn09^Vd&{R7yH7;agD3UnXFurVZf0gLSDy^+ zvf14=ewmv4FuyBXP+giDGt)!1)UPtdB}b~BGbClU8fp)cVc1dz_lm&rRY+vv`D^eDx*fvk_o#8 zLHPB08g>5g5&7rQWJ-dcXwR@Y9P)Z}0|6w!O9hk4nWqCkPJd9j@}0 zY}pd`S0jC;caF@7S`ZP7u-WUvh=uMBJx79ymQcyd_Yfw=X=f#{*`a;uY$%!=TOXSI zoJu{BqiLa3Tji*$tWD9fNIP%b%~IJ=XO6^5u8?{@Z~SsBMc3xr3^t7)N=&>4J$NZ`lPiDK*m zbL?7E>|CH5gRnAB8@c%pDH81^pwkf>SH;q&Q7bGgoz{$SxV|qFC>$B1H|pNMQ*)3d zh#xjYhR+o?-m+IkO!+GnRL&HfY4HH%B-~w)xsRy8)$O4x(R5_Y`maJMGEY_$tsrja zCA{k=ahcr-2nyLp_;XwtBf$2dY|ZIb(D&F}42N<`S#P(1%Q&%c;{^i0O zz*{`v5l(k=e3y5v%)PIf%URCM9z0c_<~CT)d7gP$FXSeCx0=z~CJ=m^fDw*M=7SC) z$#X`DEj|E+d`qJlz+(wg=StLE9ip9Tq0u8zSVwHF%$YzAL2Nv^o|-#YX>xK&I(t|_23ZYZ=9hNS z6}WESvz41pVHa+{2_K1AruV~U&?xk^-a3&X(_2<4uDB~)(f9fL3%0MF;Ry~qT@B}S z&K+*U3F{{<@jkaD`T2yz_0H`xHb24Be?ETgdYaQTF5H#~=V6dBPkk2g=_>HxxQ)|nmFo;1kmXK7hfB)=hMya31A zTb2l80_hqHl;sF z*-nb5rW;IfwOvyhrr<37^F>?*E+kz??vB}hh{mMuJA9gt#I+^nl_AidpPo%r&R|r% zlo6}bE2ygf{&7rK$TFSE9BqB6iLqE>usrMGmU2ufKu={VnR<&^jf;fV2(ShhlRAk+ zoR~fBE_KTUqLAhm{^k?VG$SVIcjYd8mjcbEuH&PoSAFw%zOXg6t*&%CjOkSM@rmy4EV29?esI7}s!0V_p0Ae2E zRA8&D@!Rs_X(47jtp@Pc2$TNMb*hvbQxED;54unf;>8$4dy;#KcK$t==G8q9Z^06G zRLsCA@|QXf_ux{_en{@XZ*pddOqbKip61zTjGy~ejNTVDlTBG8s^d$=ugDV5ld~eN zBWFVlZ0fO5L;ovK`O~mXJdi4jB6pxdtqF+Hn%^<;)S{+0?nU&04GVivh7b!&X)4$6 zB@I1-10qXu6B|HiO?6H;lyG9!@G|)9B;+4?m2{` zuSjugTL8Hq#X#lFIlkOVv%iAjO++~Eh~#qsT*28YxEbfMq-+aRY_0GEC(S!cj+5EC z$W8GKyaLf|#j2FP`h{H~&d6MWZSki39(CEmzA2cPPsh)VeH1mk(zvJGc3*pGR7Aim z&i11B@A)vMKZ%hYHF#bT^ynY{BTD@lmeZR#dTl{F!Q|H~RBfC;ee18wl-Xtk$NRes zxW0d-x?|s+UK9l#ZIi;TuS$kG8xK->xxk$z^Qv~4Kbp?G+)Wh&gPGy2%|cR zg9q9N0RUJ?G6c^|(Ir=LLq@gtSLi7?X->_nL%cl@Ju8CYFWRLE)!8J4;%iIXQoyN9 zqQRb)z_gWrRm$xnQ=Xx^U$Hc^fi?3Vpwo!o$XIPBgR{#=;^}%e$BbVL>&||c%$Y3g z9`EjCui4GfvP>dt@}UaVi89KwFm&RMYIF)p00NodILRHJPYvZl4!7K)lSi0)X7XQG zl;Y}}eS-Nmz{ zH0`4rt@0Zv4!L5R?{86<_;#8HELgA?hhf>OJl*}$0ac+#wbH8wy+@7Zo^J6~b^Rs` zU&3voiU*K`Pe}BqPvR#9<>x-zoArM1bqu@OkNx10fMF<3n)FFxu3STi8N;8(FuhUR zbntB8_1qsyDF$^;NZWvah-?mp8vsCy0$fMdBQ z%JZl7D$N3;y5GGd&BU)(SyB40LgYw2TGnu#u}Wr*78nN%>)Cff0ogJ>C#r7QW3Y^H z&ZztF0SM1UVpil$>FY$>+zblN7&F@sm62yqXQM#v#wS--|DG zaa3r3pA3x*3^r*5aXUGH>keQl{bUP~hLSNV#pjZ6MboYQ>Sh)6jZ0YbOGquUrMDbV z|IAh59xJUJH0Si_OV>&r&I#d)=-(aAyKU}3aCcV$$vfpPeYtSw*`-MBhT^X2!N(lb z(+T`FVKSr|aiTU*fy4Yv$pC7Xpo4mcP6`E!4HHS_76cfq zd8Gy-Nq)NQbt3|6Gp~fNqDXAV#GVKbuP~3Hjj!b~!U#im|DoWX`hbfOk{#$N>g4C# zNx`CrPwMvFDSx%V*D*;6riNOB2Jf;}V-k*v7a!MruFH6-h^>Hi^0r>X%SG&x2N<=% zfHtkqNtIVgwU@`phdL<`vHF&VAjzgi=l0KaK?!0_JD>w{G?JHYLx=II81Az0(eVR% z;^o%$4KNd_!mnHNeiC2qW)eh<1`fmsLSsUebal1+OKB;d{iVK z(*lBfH6Z&H??|4gQ>*j2pZYcoI8!Hlq6O%ap&*R&lfP#lw)4Yj3zqj9HOBwy%Ejmc z{VClR02aLshP{&sCz_ZE4D#Ps+w9&)%7v7wvdK` zG~GXWuX?F(J~zCtJI6kA?6_sK6*C#78b`f#6upG3Ks_t!9aOo;eDS>`eW}#W>X8`x zB>!&}`u~mz>HfdbA{}DZ)O067Ci=E#Rq`~j|00_prn&()zVeVNvD2BS*s;=?`Y<5V z8HHg`f!1nzN4NnDo5n^iq5xS0h)_sXQdX4MtcmXwDAgn(8cf#Jf3}%8jrW;C2IUYu zJgtkD9USDF|4?GKtbSLPVyn?8j#BSjeaLYrFMpq3U;L>0%qo`8f9&+Uni@-TLfUyx zd-Ux+@ou}$Zu@v#nEb{t&?&%XY={cpJ{i;L4;*WQF-$)uvc5_^#uwn&N!e~Id9+8q z?5&scm&|(gYbQpQ`dc8yM=|Xg!?nri3S;(E&z2zdmtWgUE$jOA28Ss97Gt;cYB}_~0jYgg;mCiN6;ZtHG3yv)bUo zHgIlvaKjPdAClqX3TsKBpHW-KsGR~O`TTKDNh(wFtypJxQQ=r^My4Dk9b)J$AyhWcr+n03s^W- zlcSSJ%gLRWrq7eYCg}i|d0;ce^>;rb{6m~%LxKx?lxug=`%a3+=6j7&D=QzOURm4u zu^_D}#Jn;Pi}Y0}k&dlc*+o8SWBr*^d+%zzWI1+4_N~)|QFr$tyy&iFsLV{VSark&?^n>B}j zh)SorYpMI8HW%nS@*C4Ww8Z^ti6SWTcP~jfUkOI{!y5xvVw};D@XbcG&u1R7VF^c~ zW}Z4m`%$kC>&@wvdAzInvC=~xY!@uJXhwMB(gAJ~T)%X_;BZ+kVQ17+&}met&R^vr zd!n~neeSRyb8?_x-2!>rUxw}^FYl1WR^ZYtAL8h6x?2LvajI~l9Yg)r9uE51AWQeI z>N`f<%kYpm*O-u`eNDJJTi#jDpE@0DAERzYU7JHxn*2zFvbPZ<-#hLwMUxQ^Il`HQ97cu=>CY*ATnBQfh}-(!jod_A@3N#L`cKT6`5Ag7V&S z#;D&4ci|GqvfAGOPl1H@tU zMlMjorZ9|`*cc6Q7}i15>x6Y$C96dPKXi9a`kC;F^SbnpQeV^OkilUHD{*ENy3TL6 zC}>ZT3Jou4g4yy&$>H3}$}x+~OEU>|>e7{MRC5vFE9?&?DgB(Fq?r;koYlk4dkQw0 z1eO2#bkP>lJ7C(majY<#N0+}klE{}A@2T9$PPdA8xDX)|51SHB;6+!!6>4cOc}Ul( zYw5U+mn1u`ULaJPy!^|@ZQe`HPE|qE3X|xnbsZk#Oas0zoac$`zIb{+w_AIDoWl7a zEBg>gG!nEG65p&5w?0OQg$;gGi8v;XwcZh)>Jj3!F^ZCG zQz5S#Q*^7QsSzqLRZ~?t@L(|r48_gA_3Z#cdDTU88*akw3QkeR0+#T;cAfrfJeClD zs>yFBNJU$9uxf~S>nD_)-^3_fR-85u?Cxi)^N#2#21fzHFycPg6r4b$49b+Tp-KLQjy0 zQ~!C={15db7gpM~SKtTuePL$8Pc{a*rd5T*ql%*XCcJT}FZw1`Gv=U7R4S%(#g|G+ zGwNomM~|{s*opOFHAvP?m!3BRTx9f@t-1#KIOrUI-^ znb`6utHzTqI7WA!>`%_g&6As}d*|KLjS}1d0p6E8g~Dgsg%Dm*$~I0 z=O9uiQYr=N2AF+)R(9K7Q!`dMd%lz zH5$6pNEXmz4zE*0zs7&Si1EUl#AG|OYFgyq5?c=cp&t1|BhlX$?BAUHU}E_S0zY%| z0QJq26rThd6T12nh58M{LP)d~iwh(p(i5yCzC^mLrmGc=Qi6HLu8P6|vluIv^;KW0 zxos>sEJ&D~ELH@ly$SJjG{$_3r8N&{tRNo8u|49jQje2h(X@%b!f#2DoC8)k&v$ zKj%+!OBRlQ-;6(P%=s3TG=Dt)pf6hIV|mk)Ue@pq#dFN=3fvY5RAyYVU2ZE$7yN~5 zfc6!6gme4s&AQf_^Q=T#x_xgvk*S>?YGdqj`+l;ReOHmZeNVe=`^_?!QbePwPmJD5 zGXEl9*BU(Z!jhuRQh_>MUvWAnO_1Nb#N zIL7be9=~YWY&nQULC>jF&TOr(!tE^zmpnIDYG zPcN7s{I6c%w2qRLuf@4VUzIzdt8lIsIU z!sTJ6TbHzs(`hO!16nob){w__EX@@3c z-`hBvz>G~%Ip6aN&N4*Fm&cj3bh^o-|hN6%ua70Qm93X+yA5O0Ks{WlllIu z5FyZ$i}VR!=XGp*{l-$hhy@$I^ouQu4vS_gHnKEzIPFN&s$*(|rNFt9Enz?5+3GsV zEhno!ZsXmSKS;DkOFMBv_+RXC{?>&suka7wY~SnTw-=e-sow0nhnFqqpo<<~XB)HU zhXFYfi+zy>%VsXo&E{W}YDtc5LkoCR{t|$n6lN$n`U%_#+6fX%_)Qc!EXg~lz%o1t zpNE(So(G=?orioc0$)P*qpYN?gsjx)$5h+tVTBN!zakA_J?HPSKCF~l(<4;Dn=Mta}@CQ0T8XD2sP+el4^ zlANK&=$-Ix0LuXn_%NA`T2C?vtrN8gqlv=p!vQI{HTjHsMan67 z1Gy8$jrV{MT$j8~-7a+={3mz=r4!9fz>>+5=ztR3ll(+|C$;7-LQ#O^(n5AceLw>q z61ZCFsY#s@?%{M`w$~VN3V|=wJgv4QHdR);UGHps zbq3TwNjSKFp>7)1TbE|otQib4Ti`LHkDjsqY+uVweCQZJblrTq{2=-Ikpj}Z{6rhV z`RekuQEbn_4rS5RtMbQ_WlfBJ9)_+hvQ3Y$|Bpt7^GlDr(}ga;2hZDFD5cpx6obaB z_Jid6w>O0it6LYP`N5Q~kKxVc&P$}*LT0YfoMQxN8X~RWMt+;s5OtGTB276zn>BxZyn~5QB4yN8AG%M?caN+rMj{V$QE-YS#N_ zU$R1N-~c<;jwt!tCCyg)Oj{fmV>os)9$>M5tYS0+DvJa=Da*O94W}Q4Rw_ni(uA(x z-*{z2m(^W=XifDdne%CGO;eKVMo=2A<_5hax`ImUG>z)hXuttQzdN1S?9 z2}k`P8L_Jpfd4J_(4JrtBkV{&1vlYd1N$;pw8~l2S~oHkFAaAH58VL7k$3DE|K;n_ zY&w23pw>)#IGJEuhb}#sclc?v374AJT$S$eK-xr@3L$^YRY(J`5I^BbKRF=9KvpkB zZB-DjZ)7u#D~`zen|j}+?|8*OT=Oa3Kpq}RalUh&d^X;m_s;wo+F6AA?J2PnKjWsz zJfv6TN?u4OD}-0{l-@{igGoPU>A5=I`en!+0^?g|;>>FQD8avk6@5Jx(?rAzU}A9G zitko1k`3ls2MBpNTK?hq5L_Ti_1F)6`l_LFLAd(ZnEe8JjPgmeIW4?N=eH+b6C%Dfs0piI{@AVRVRtXtQStUA*W*^`N!#kj8}-zE7A}K zuo+~)D=pN!s@CkvtN7AM#QQpDtE*))QLnyM`y13AQi_p%xjCyW z8ZT@2AFHFLNEI)uW}V)vgv(wpEuB~xGY6k1n>sL>wI8?zHnyMkQ)2Vj?+2DP@WwO` zrHyqcp5p1d*BJb^(@SeM&ozJe3TPzcW6hgT5!}u1C-t^eiuUWPo~QRb5tT8td17^V zF7&Wr)vAbw(ui9HXm1ywNZ0DuiR?hPVExr4(H(t_5=@ulQh|Tdv5X?}vpy*sDt2|V zi)(q$AUv46k@ruxiE1H_!-jTW(?%NGn2g&@&tQXqO>a^ce@1Kd ze073&^Nf7T`i7O2y%GN7!y9ry5FbMT;q0adY^snOG5&?j{)b^LGQUBR>Gh3G0}NCu zBK;;88$IefM%A#gz^yV1wx5&0*JxR(M zF?@yp%IRXOtx!Av?N3%9?~A9Y3rn8MtLEU#kBIP5}l>2baO74H~H zko^%)4}Y!hEmduuoiz=5mw;C>R^^OHKNi&2)|Y0{Cyu9FyamwQKCO>yw5`9Ld6<%x zy|$56#avrtcn{!tA`1O&V&VijiuWPNFOv6B3IEP_(_iNLS+C~#eQ#E;I7BG$OO4Ci zj;fg<$Rxw0B#A96U6AbP@!vUdM@?hHuNBfKwkOqVkuzUai?>g2i@((VY?D4=9WdSU zCI&5cM2TJD%d-M%3qC^ha8$g$Uq_pL6GuPD3lyM@ulQhXY3=v;I44jIV=eno$(wDd z?MEcPCQDJn-G5BL3MHRFatJ`Cs=oXj&@tN6jm(3}L#l~Rf<}Ur{Vu!qvML~F)CQ7G z%5`W($`v&gsr~T?(n(qvRr1iSxQ})(nV=VDjN$eHyhIKBuV-ek2KAQ&m2J2f#v^(T z&$dDFqe&6uV}Ya%5%)Bn$?qCgeBeq`kUsQAHZ1wzIWPMlH=|^{^X_j7@Q?fZ-cj+7t5Gg@ zbQ>p04@{(Q=|K7AOv;4WN3k&!uI;e+Ut#_?A7~#uBaeN5UWXpbBwUQ>_z(Pwl3tZF zP-8OPo2q#qeAqaKd+3JYh6wbE@ARl z;i`ZTV_4wi>tF{EBt67t@*+y%pgWGGeI+1t`@K(#1mj2Pd%;}AkLUb*Vn@EgW%0zT z1Df8aM2u63w8OTWj?Vk$la2znbWcP@f{3noAelGi(S0!HS4P}19M^7DKI27Y+A)!x zDFq@>6rN2JM^rg(S~ED_&M9x9;d_`>bm@t7P73~-aZLiy`G;uQ!;kZrymhB6y?|wY zGDgfOBXN8NglItPPuY09KJxCmf|tqYfSp#g75YJ+Dm+eB2Hd1)d0;u()rfjx4p13A zP&DyUYEXPz(0HVl*PLh@zeWeGFVEve@-lfx_1n>_&VH4`VwuhpnJdJ09c&|48677A zaqqRPx%~z-6mYL#aav+NG6{x@%%Forea5>ayt(y_qC1y*$+DEm2_WZw0^gW~VunPe z?@8dD-HFSBxGBnyrBIqGqD3;3k2|SP?6gAvFwk<*9*l5;#WXFQaeNVI8ayQC)4s#| zCP#PYB(HQ(qm_ZfWDXL-yT=h$&0_!1(K<67P;dfZsZww+ts4`KcPF(S0*%D&53P@F zcuhkHAo*B!{1hy_D;y&FIwyMYtNBnN{ec#zydj%Ha?}O`bs^IM8g$QBAsOzRFu4Ku z>I3wJ`{YOrewC!mm>$k}*Ay$AzVK94Sc)wl`A%x|SzkW0b4grm4##(rk zbrVq{oTX>Wpk_sK6MSqv@3S>BhZE3MaKn4Ayb33iVheS!JLuS|H8_HkVc|-flZ(Cy zC?;Z_O{lna0|l1KU#Q%KrJpn(-e}PFB>#TNI3aL*v`q7CyAx^2hFJQ5cZu|0_u>Vf8?Jig&9WMSD&;aDqQ zD0x$bdLtv$V9H?lD0P#;%*n8Uds+Zw=_$82b-CKJ; z(yH0tV-E!IjtPxa!{W6#-Mp}mRgF~Xu!Pzj+J4phN4qz0*kJrKlv9oPetv+3ZPU(C z5hEbn?vVKX*Fh34aLjF_D&OS~(j$j2V4UUU{EmPZz2;PkAN7{x$w&G6TdZ-B>5kMy3u^4=5d+?1XraO5DjJ(k_DJM<}Da{WMEe4lbDs5&g z_&og2Zm5U>5Npgxi^XV3MRZ`z;?@~05}piaoNRQji)MF_7C}BckcgWig$!E6F{q+b zOY!TJluo2RkSF3&JFEcFAbEe!plu3IwxCFnI|VO1kf>r%Z4KzgfEB_;rf@pt<=PaO z%9d!|NNJK>K63IimGM86dbEJq8LYFVyv;VOH0|Ai?DRyOOLr8rllR}b+8E1nIN|(r zJv4=M`I>{L*gfKvV@OD#Z^}b9k(@0{?9GH~FsLYs%}putG*8UkU6x`{(w$3prPjJ8QOpH;xHvZ@# zDmevIhMgVHt(9Mkc#HM*v-m#J_@2rmP9!>&5}jgheZO zH<4z$PhPA3R5Xn>TZ+0)%_fUqGW!PMi+w@@uS=o(Z`L?{J=ijJO(#4*Tei zedflHVy=_nj)e>y=x9$|X+?`R*3KR&d0wo8;!4%(@OCfop5dazA~Xulz(!?(_C3fi;oth+2)2hd8S~%xqpii5 zc+CXwgMNVCHfaK@tV({t-Qr%Uacc?`PJj^0IGxTj_Wu3y(V zjss)U3~{dO%*cCx;lxe>xFy?!s0)|cKep}JaN-_B(7NH#~h4fYWKac}4BuWp5 z{NT7?<;N4L2|`IHPw8G|@Guso`V}h0iMY>B!zoQlFN$m9uV7xVVeSy_^#=!!09^oh zH|J|2+b3OGsC08}e1e=VhqXv%{muUG@66bguIS%HY_n$8zV7t9*E0)z}{n zgBngeBTt0Yp80|E^+#Fn&T? zHfOt%_xYrWzBu9m*;shjI4kV0`42bO+;79(+x!cAUMVm1jfG9wk?~LW{(FHF;ue@s z@sL~llzR~zGZ)A$*5JhK_ba)ktIj`}6f4pErVL~atEvTNV8NBxy!M#uM2=(k2hlE7 z`at>8r1XoF%nNRdwf^}NwF>V7#D9uspH|{#xC&EY?uc>#rYo+0Eo?hxbLd$<$1Ey7 z2H>C17{w2wmaj^B*Gd^vQ9jQCK{Y=GdavqEoTyFjyEIyQlWN~UcDCU5+T$~oqsKYD zM?9g#o0abHO*f)kff)Wiy4-EyU&hsz?hP4n3sc66?zRFyvUw*)jfTk|nI;tl`;6FeKf9DG6pSiQ><{b#eaj76mDgMy8IXJo|;d#1axm0bI%Xyze)OQU8CqH9~dPbt_wUH;b}QNmSZ8)c{dA- zn1Ff=yiO$)679ZPkiMb4v88BUI(+8MfK8W|Psob`ekBsnHQxW=<<2c1N?CkWAW63v zaQoGML}U+oM!NnvN&YHuOR=OfRmJ=eq8P7nb@6a$`AB7LF`Dgee}vq$K7FppC*3|7 zNsmCc@I6;NE)?gj(IJK;{sxC`eRZ>G?IdPx`P{A1S8iUmr8&Q8!cOYeZ`_akew!tlw=Cm##wFJ`aMSP{{18I;jF?>pJ*GYy*X;p zzR%|CaH31%hCBsqjimG9asrcuDeyZZKcwy2`mo$%d}|keC6_E zO|5!q%hvgcrFD%aX^eb9*t@xcpv%B~8VMZK0%aGBin%?5R!%%OVvYhWG7ibxl9B7` z>GecHuF7REVh&SnheuQgCD7bW(Be(zVvgxwg%k7MZDyXgrkn?RHG945lC@Q~>Z1im zpr)@{H!y^0v)ai=ZK>z)pAW64?zI@#rKfP~t7rC0$NoHBLgJ`bgS*o`Rl0#~ zQvcc)t7qIRU)v1-xka5`RO%G&j7J@8K_KTZ-r>b!0dw;gHlS9w6Ie^LTK}V_9Tn@?{0Q&Pko)CXy*mHJR2HmC zdmxA-rbFi=ngcRHjk(U2-1Erc>cJei53@%FA?_zttGJ?%9@~en|G9>sjmTtFaXyhiT;z7`ads5sY5%n{*UWa?T+H_$NpDm zSIDBsyb#oj1_d*-UxkkF{}m&r0P=Mm^q?y%n*#8u=wZx;jwl2i+EM}V+8@!DBIO+& z;U{V`*5)S<5D}wL@~0|q{)sco8Q4^VELbG=GJLwohm4&_jP#Q5ihKYXCfb}d|8%JK=NR5t$m7pkZ%8bf1#~~Wt zh`*B7!}FhEayb9~*&L38%%h46t7ArMV%b=SXOe4FCODL8g%m{6R`w&>$j_;~rTylee zgVZxt9&k`edh~u9bnR3iV$VL^ z-XBZsaf-bN-Ve7UOVlSyZC|bP8D$+>(*C=wzhOGA%+Y`{L?IeY*JZEhGiHjvi6W;T zswcxShi_a=4;%KSW&_|fFt?Vo(x_Iq)R{?*D3)2TolIcDdWbCatE2C*(oiur+z|Pt z<(YA+B%vCZZ2z&!TW@cxyv$Xnw>KmNCUSk)f1`erku+Ue`?DYRo<-#0w&s_%Jy0Y# zmj@v*YNN?v9*N?3wq##~^Y?E#Lpdt3_@ZJ9H~|l1NyRa5mjj@-dY{plLJHy$zr!-L z+P-K=DZb#Uw>O%S)_g20nm)%Ql$bUenE9K4Il|OUxO$Zv2v#7|CyKGS;Fe zYc+;WQS(&U<9NxgFX9AV<0cdJ3lSskIys|Pq(WXS;yX)KLG{b9rIg^RtI`SkCv(bo zs+&_tgYLy;x_&u&REJP>0<27b!78BDyXb^e2y@ADeTht71~Ibfy zd(ky-P^cAief@XS#~NXK%mh zAj|q++-#4wVABWfx+SE@OjboA&3vL3z{uRvLqY=deLs;~5kSSc1e&vz*B#SZK)LCq zmdz{)UvN+rYJS3OCi9gQag^B#Z|2jlJ6W%+uC{6PhY`x)mmw%25U5uv&Y3#FMYOR- z(fYhA{uYhovlD_8>!ZD$`@oZ}%rAUK<$L|}F7V>c|3Wy(nO`k05Q6`X{WDr@KU^Ws zE@uZ|6zne@@KB4)w&T9|;tPYj=-z;gw2NlhD}SV9c~0jZJ@sPQt9-;}i5cetpP0JN zZBM~;t%Jkk5^88+NlAT8tpZZ0i<0&sR*i8XomSsosr=`O(+iV?y;WjKNp~%n?VS2R zn9T$Z$el{FETG-Kz@GE2`iD(ZNMCso0|^9tavlq4t{u6b-xZY1QO!duMnnycOMOC| zaEFPOJ&dU8sbcXZ^_@6g5rrxK&M|uF_i9!{8)Mb7bbPqCwe#tO{GLq*678E&+`8)G zvywVHQ9@^v8fjTNR~ZKjbv|q|O)Jql7WA)bymf)afP+kEgUCzV3O~qnK*o-fyh49ANC8K^lFdQF{z3>FydWb|1GOQ0yZTuvA#J0v(W!3 z7T7S`B9h7`{O2OteymU_7H<~!x5<>hfo`eJc7gA^tLOwY1{jTpMu?C3eWNz2p{^>C z3u|tmuQvZaeTQ0ZB0Vnn@zi1c&Y?|eIh+c?JG4u8+^MZ*4> z_&)UWe&xLv__9S(IHPG=qHzf*Pry0(60lzuL*!newX9m z?@#pyH&8`HBi(OJuCOMnGIAK{qSW{cw?6UaN0{%o0xsCsJWdpmhM~>CQ+i4`e-{tw zQab*TkcH)yuv#g>DLQVh#P?KxQuS%@ z;Eyu(aqt*)m1Svp!={vZn!)MN=AZ1JiVV84<$1qnnm}2ipQkbaeT15HeS(^=)ad>Q zx^#_(9SY{x`uW4B+0=~>f5>A#^i$wEz>C1D;N?b#=BP4W-m=WSmKKkD0CjQ7Y>{IB zzMF-r^#^T8u4FhF#|zLNi>;3}FHs_vEeD>pOAa(AZ=$KKU~h&teY#$3juDX4nrAVM zSzaKL{0b`2$B+01Op~;7)OdoUVJ)Gy9Np})R@Z9ye9c%&IkpY9A?yf0^J2W)_#9-| z$hmA^WBeHP52rjvRoI)BnmZT3Smx5TYK0^zpBb8ZCRr6=?USk|2ga4i_D^TK7&*_v zP`;EZDX{*Q;*O#B3pB=j##c%z>SVw>6MCOBPoUfkR%g)y=%oz8>GH}Zkt_J(zES6u z$Ag)fTab%WP-lkHO1rcv_$F<=xLGH4TOb~P7R_4y2LUmf6@jfWFWLDA|MX(lV ze?uKRO9P5UP;@Fj05%h#}zn7)^PK&+^f zWV4H`WwK=ePB)J}bq#r3Jv7CQ;$EH_5P3Z5UY;)=&?9CHrbfjH3J7i{rZZQHiS0J&pA*a_J^(CW{3GZI({CE znzND)VmC)r!bxUvevJA%3~EWbsZfOPsM|+bs+Ad{z&*Z1@Z>il8Wb0dM4TJO8#*G7CfyP?cKXr^6gO;1BRmM~n) z@fueUxc4FjCu4j%(aiDFH=-Htgt1qCP-Aw{Am&Bqy3ry+p6s&d#G)w00w#)1-JJgp zqEAhKWD%Rd{Ft8)|6{`84t%&JYVrPf{NhCTB(Cw2T3SAy4TOKJT=1B4ZcPV5uZbrzsW{sfebPavV$kvSUPv!b2G>bNF6S+M; zd<)dp?+eHbmV^35iv~rDpP4Ttov9)z1SE^z@zr;6Q+~AaI)7wgbHONxclgX{Y*SL# zV#~&{rZ}Y{p!+R#p*%!8{qw9Tx*w5Q2n;9i1S9PJlXyzp948lH$4{j4#}@?ZPbrA( z^Y7wG5&F?Cych(>6d*M89o){LcOAHVf+%9K@B+swE(vTBC260yaQ{r^?ilKYBB+MG z$h-J&!LM}U=AF@Z;Gu|R2n_g}pycGid(#bw`r>tCZ8_s)-C5#Fyb#0Z!{5;k42U2D zVxO>$FBE70(05>=U>+)9_mib5%}W{2cgCwN+oa!oJz)s^?4=;?TwO{fs~{el8iddA zLag}cntp?>_?T5wdeKP#5^0>;D;!lCHHGaPuuA{(*?2{(GNHnRAux&IL_j&^X7eqx zlq-kzB#HX6g!v??s8sx%^~Aqi{?X|kpw7Hu#`;Dp?0q(z8Rz@0U^whDJ)v>Xy!JaW zn>vLvpS7N{#`U`ZVyC_Fi-Qf7qEpfXOsgG^54MEj@03KoKXxm$^TH|R`(N@!UZJNC zHMhCs)c(&co*LEV$dP_KvX=pZv#7@#UDHXdeM!xzH26T`bM|=*S0t&M>lbjYoiSoy z<*MsC{qALD`~h$93@RDVGN~5*?xiNy-*ifp!8jkyCw#inboAM*aaW#jnuGZtqbl3j zCCzBWG_`3f=!yL}(DxbEkHg`Z?pQt4lq8lQ@m;Y(4 z{ofPb|Nm4OqH~mn1^@1y2y{Bk^Dc953ZFPr92p5a^Q|0&*bZpC=Yq#;lig75BfCXe zv3>W>Oc3TB2x`H*TLTgT{`8;)XzARF`#qLkz5f?NIKo}BgRfyr@SWKfPC?;)1S%mL zcy_+LWnRt^kD1j_MR>s?8jr$7=+stD9 zkaW&>-*NJZ3fyf1RBw&F%*O+4iL4}QOB5N*LDl&6V+1f3Q>XXO3)qvP0RMgat~k~f zXyj^lm&>bdQ=4zq=Awhc`QC5)t^(*e{;?q5cfa){nz_wiA)dcix==bAlky7@3@K*r z?txrPG5gp#dOY(`bQ{V|`%#cjI>(15Ee}|DUbUWr&B}o=ti_b;ddxnvC~#9FQT6rp<;7QZ)?-(Wy7>dl7Y6_k8Iq5;YDxOXV~ zeZL^|C|)0C))2iQ zkdX`O#e6MJ%-d}I)zFLe*vBio~9D^1tW23D<7)m~pS2!-gQnwx4JC5md|3%(M5yse*i z1LSmE=R)nH(5Q6ES>-||Q!rs!CI678kt>lCaLT_cDpvnwr^6y$)kF&_bCUzfXU0yT z#>Q5cUWM|g8?e^91)JB?GKXtVr4b3&DAI)R1x9`um_qeu0Aq=BR=z{X;-$9*%k z$~a*66srI9mXTGhR4OCF;D8bke8s_D_sNRGgEI}4jg6H&g44FTX+CzO`{dhUHF<ifCEquytbwH1 zOkL%EKNEHJ#`wei>hx;1N=EL!z3K@!lt+o(J%qT*1PM)&qY&81>`#j%3L$7H&xJCC zo%#9{5&C3!r3i_|uzD~zKO&Zvy0PHkWe^izS$!o`M#B;CMoq+=lpnL$8c5S3OvvFC z7yh)dXb~B6AH{O-OU;lJRot}9?gKhS74P>mEb5U(7i<#{sM6k&o_^k^V-N0SWs)Ji z))B^Fltstgo6)+CDZnv{Y3#@s1HV_Q^_2_XE~&&sDTe-{8Is@mmQ^SupWg_62nufD z=NHK=PydQ3!S&iLnD{{DgS>GCGmnf~IJt*c)`(9zF^DoRjjD$s^>8__jS5^p@975- zQ;yzSlx3&)-LSNWopu%dCQ(g;?~aF9!Zh@Jy&-gAx+L>FC`IikJSau+!ML>f4Iamb zFs51%K>VY{<>XWgg4>hx0$|J}o*9JP@?a^cK_6gyWA7b25BQLMu*NUZe0VPfQ12NB zbKyk_5BSjTt@oVgQWjWuPJvJ{>K!T^AU7D0@%ix!sNZkV>QBP!E7lShg>{NuKLwBD zxV&krUc(3AULKzM`djk}_DWE?Ca-#ZREAZ&ml77$?y)t-Yi0IESU1Y=r|3aqKh z^G{e)C7-xo=ZvaG2+uRaRI@1yjC=GICjEN2KveKs8blRlh8`3fEH3pdD)e`WZRaoF zR@tIw2p#h1))+gtg=DR5r%I)P_j-|ps>!m{DsulxS zV4dg#<6*__8C12PHHP9d^E1yH`Q9Zpt%`1pbq6LjjczA1HBW>B^Z{i#ZVV zgd*u9HeN*lIiatRkV`b{UyxTV37(+?sR+5{{D&t8Y!KPF9f3>h8$49NcvL1gIxWwO z0z?5@Amt`B^2h{=P0XWgq)G*#*#hDZ(l2fe+7DyrguUhgbDL_52Tas?2T(p<8scG~(MV@8BNb;(& z!CW*TSqe}U?3SwAJp6`~8;{3xi7i<^LUP#m1N0(L-#ZPOH@&+tMQM3^97eIbIxMfd%Os6J|jkLpGja% zrF_bR+b8o35xNua-Qh)A0OpC#sD%KSyVN{f7`?F{TI@vsMTi9Dh~S$2Vu{75(M+@c zLi+#79(;G&qgr!tpQ4)~yz!>99l@cyzriOsPuJU^zXXsU z3W+zK9v}r+&Xhb$7&B;6Tt6I^ zTMpBXk-Tbeq8I~|fgS}Bo+(nb=yoW=ftA>Q?nqEzUr{cf5R7{==b3kct_q>FPnHS| zPaJoPwoUjy)*p)&r(R}?P{MSNz0e$1+%41!W3q#gVg%(NHZ1Mf*g+66f(j7t93Z=- z-5OzUs$l|lkjW1|&4^SD1MA3C)af-QVPkd>Obp6~0pSlFw}DhLVGi5gz{%VW9G_1E zB30!;ne*U1HAY_P0Fnd55<^Tntil$u@S#H^jEv8x8gXSSPHkal&3})C)YuNiv#@h)3kkposz>CS+hHNys2Yg+FTDD{QLUuU0T^4~23lVh8V0y# zcM?dSB{8;22NFoT4Z{8}nyxx7s^@Eil#)t!2!eEj1IiZ zr9ry8k?vk#+5PSJ^M2mH=H4?i&z$GX-Mh0p=bQl$Mqy1hE+_t&m6R|uJlcs3MggBJ zOuEvpWkKj0$~M_Bgmjp<-MI!{mCd<2Wt)7MaaotYAk+qrE0C(6d%9DB(m-r2SrAH& zm-DM@z|N%r6RBk>rMO{OvF$m#fKMbQL>;<4qct+)2EcX9dv+_S{9trpqV2;mhC{wQd?#dE`65~1DP^*0I zKpU=D!Lrg(cq?n8KQ*@jUC@sG>|!Afn|Co~7B_81PxkP{+!VgtOOHD~!7eS&jzGki z3OpI2lENu_Y%W{%bfE6I1*KRS?d4Fjw zdv1iV33+jE6*XxPMUTy4E1xb@1h*jiOwoLkg$LPTc(9wmoLt#Yn-2%{)l^`|kPv4* z%2Z&}P=bXgfIBi-_g)w3g`3+oG~TUAE}5K~8{yn{u3G`)XnWaX4ZZ}x=LWb0mOIv) zy5y2k$l79asDV}xX-?=SF|z2O?CH@#B4Fd@4j>WI&7cAQNuT%h{D&)!Yut&HTAM+= zk$~bad&uf85#Kg-aH;;Q-Akv3`6FAyxg)vFkONfn2Jo_jaDe#DSIE{d-bkeH5EwXj z!Zq$jf;gK(3I5Xxqo?%00=1{$zi};bjfaqx`(Ini9-{vg$Hp}tMK(f`rTkC&yeC=bo*{O7?NAwdZY=j#kItnJb9u4mt{k$m z^p}w7d&MPL5tVf-nVv*4S|@u8rhZ^D#J}q;>vVovMF-ASW^B@`@a4dO^vgl}kckq6 zjxd7I`o?2Ghs8^S3cw@FoGd{%t8nYq`znkvzJ^&b-o@Cf9at-|$>0) z8?5}4yP_pvxa89IGA%Z7lZhedTw!YOedMNVqb59JarvmK3n&>+spi_g1`DX96T(*u zi7~ItR}^P-UW=3VcM=i1J#6D=wQKI=>qc0G z$CujQ?)FS-Bsbt;vsBhsl@zYr({OMe2<=n-tQchA)brZzAL~^fk#BNk#|0UQyX9WD zjsk&=4&xCW@J`GLO=bPxozjdGMyjbsS&*D(qs-L9Q&rZ9qf5ln?~|yPT4kS6B%DzIlP7@m2lA%&N+jkP2%# z_5hXkRVlZPPqJ;i!kYb1!=Q7fQ~htc%X9l&}+sof4cmej1|Jc~@RYlV+qvfu=(> z*V~l`s2M#{TrK+6hdt0K(UGZeTO+={ep{;Bc>UIKKS5N&@Mt`7$NqibEx*%!_xL` z@mK-PcWt?KuWmUSyb~~OU}VkLolm{1g+tBXzpDN*Mw1+`&9hHva4DWmU$$@zkDpEo zLchY7(HBx5x6ZsSQ}WVv1dm*iaTkWcgBkl#3*bB z)|PeAkzG#my9j{!ekwF@OL(=k%`?+sJ^tF*Z}jH|Y8KyG&I9HpJ;E02ovV13$CD;% zYkCj}GtKLjy&|l0dji6vRr+_?-i$H83TotUEZlEqcWHM?G@5WUTZUBOMNRA>Y7P@C zXd%%fdUnIzDoWJkCKV`_F9-CE58vCQF0A9?Z!bSD zwyF+fzVJ7b(5>liXEeGaioiA1g4@u+CthEBWxhGf$>%PdkqIVGGA);1m)x&>@lI8C z1gkdrj)qkU*a%=ByT`wM?7z_lZei~)&zSEuCwGeJ)Hy}Vmkc!;-pt7;)%|Rp;?ma} zal6IQ@{RFy-$Oat9VhCtL~WT)t}O7<4ZMA|rzh_Semq4#6j?A&T@R~Wwz&$JEeJpu zPjo(O>~%qiyvu#yXyu6XjN&+$u!^_Z*W(%CR9?l&pfwtkg8JeuPQ-P;8yXQI(4y7W_M@05^7G&#p7>DKs!5V* z(s;!0H0J775q-%>-JuIcAjO*hhls#TaSg-v{)7h;#2c;%o=?O@t4N$h12@rAKdhpu z0dq3vAOQbQoZ?DCS(kV)U%c1*1p^mUZ*b|P$?9thhS*}f=+C$}#rXTG**rXGCO?T= zk57H^;u;K%;Z<=fFyB>(??=FMzPcl#Rv%)HT z($|>JY90u4a>vC1Wa>EiG$NsvoW3S7l*2`vFi12W1FNAfDK$$MUlt)w=gG^1C>!Pi zq7P#kJmKhv3vJ$SSkr-ogN`q9d_z8`O53A`cY(OfmJ4e!&)F03Os$P%7934GzOjb4 z^aP;Ag>&MV%a5@*_;T=T`jegW2E8Uws?Waj=82yO=3xb7Qr~(?IQa3*tVXfYuL)cD ziWJfOsrZO}gP@JAz=CjY*3BXlFWs4a^&RGnT?CW_c2Q#&Y0ZrKevrvq!eI&)IkaZ* z9L=CtUW^EJuVVSs*=y-~%FKWCv`RqQ~YZq7Z6Nd(8i zy3Um8ySWS3V$>^;bD>^Q7y+UhTK*En^3Gr|1w4@CG=htXcuy_eC^L`;X;kK7{HjYYGE1A zy>+M1tnM=?9M`!W6tnpgkJqbhhyv}Wb)tp^W$>3}Miwvc4v7PzhY%R}v)V_zM`o(d z75Pr@P|gJ=JYRUE*pp4bdwdvD?HwyW693FM)JNQ@j zjkrdStO{w}Ot(26;1E2Z2j7nxz=S^!(rM2H33Zt>0uGV#E66Nj`6J)HVnq7gR`uP6 z3|g^KFAJ4A=XYPIv6I9{J=1tc3)mGP@5GRIq9rckGYS#g3=!A^vJ&DN!?Nw-Go7-8 zNBB2Ew2E8#a-FPNDpUIgGS%TiQTHjPDT2c9>3AK(#iOpq>4T{Z= zy{K?+8DXw|lQSQ`YHiC~Ignj*)6AdMku}Q8x%MDPFV$GV>kHm;rFjG%JipI}LHM?1 zZf{|){qoUl6fu@$>$vHYCF(0Z6 zr0ptIaW<+GIWxAx`1il*o0lAYtSZML9FXNgC9ldDem?ikjuh4T$!<;(xelD4@`$xR z$ns%W17X#?=u&%%AKHfc^#Wx5cjG%y&@0t{N?VAYTORurXD|@87c2J0|LZB&ZC;=( zqWIQj1mR48fH)D1x9$or_1qjhE%tz}tqBmGp2Ysm_i}4E&hOZ#XtmbgKk{3DSnBcE zLOkgv-cHCtZLObjYooHJRqP0mu>;CIxnr=(+wd+wq~zKry3aQm=`8Guyed!(K`g<%MkNr zgZccfNXZoiRUN1P{R5*SES$*c$V6AIO%37r%X-21J0M_|^=i8ma=B7^#aOixbO`hL zR{06!86#h23&Yu8r%+FjT_tB-d zIVAJB%$o9Jl z@S>W8a(uePGSN5ijr3d(Q|c@N8Q-GfoPNfoMg0P4(`~uulk|$I3rzEI@@5rg)W%XB zzprWs3Zl+dq8^*9ecdJmX#_q+rlcIleor!g_#J*>|D*3@_ffX@;|l0J?yLj#bJYQn z!jbLjMN~26P?2k3|9Y#HuXT~>>Z>4^Dcd`VK5BHH_7#;@-@=e#bgw0*v#>w*reZc5 z6g2N;_xcPeYFrNr{?cwoztvz?DLn|e@ww^Y0)V@)8{eAA9~K2MmO|T$^Kj@TE28M4w z#C*r_Y0{-4K_H+;{@eHT>zpb6ovS)S@Mnq;< z&+e%E2^!e%j1@Hh5XwY)#=h`n@SeLr{)cfsWT0&< z^k0Y&E)Q01w!~~cPJhnCmtzZ6_K6!a)qTF(n&Q-bw_28Knv0y2`OCL|^p0O-4_e&T zQyOyIhOhljlxPnQ;`O(a&drdhlm;>+8h&w)TEjqE=Nae)M0#WVEFo99=+`SPu7poXrJJg; z{RsO^EA?tOhLd}ilhCOBtAuF27i_t)vpWef%!t0X zu%&x~A5-4;R-hgHoNmGX8vN<^K=z~4>zoiHF46}}b|=j3fRY1HZ75%Vt>}|=jh=>( zXDQ3^aU`xv`Hm0reYcb-^{Ixi0Ky^9E$*?4~ z=wC~$N;%(k%-wWO>Mlk5q^6d!Yd!oZZrqEjm1cbMb6d4|qNZke*lnvF`Pl31RsR?a zKyTWpAdl9+rd{SVjg#4jQ$h^S(XQQYiSrm=JgCK})%HxaT?uP&@$z;G(`8q}NXe~EBkTq{PB zKna`eA+s@l-~w|UW8L;stqBVuyDh4p7T6Ej$1p;(*AZRRk7}equKC%wfjC#MvnGFx z4fp5V_}hj}7rM(z>ht!$7HDEw(0WTqA@G#T3J{Z zI4MX_WheGI4d_^s=Z=p0l;4Lb94_uFvRBoy2{mFNwJnaoHx?!rTC#T_rk5?L-3mIoEspem*;J)lvKAZvrdWJf43mlKx7Z9} zU*~uj9GlA@rM5M8>QYFqm>QQ}j=&}|+GdURW)`gPfnIAW9Xk`fpq3fRp zQI05>&#$d%oG!^0=VfKIfiF5x&(Z06%Rj(gx~}D3yu9!LygJ1K!e@p5olV!glcke; z?JR!^*_tEWqaANXj8qB7p3!62RhdRc5C?NC@E&{TC0`ynsWk*a@|_63(9`lS*qTyU z)Zt3@;R;x1V!^{H#9!{7j#{nk$?X^oG=Gs<*-ikHBEz6iu!v9D)oO!A&D?M^%_4R< zOx^r`fB$(3d=&AH*b(>VOa?QuXa0QgY0Na5j=o__U8}bt;?r7*aXX#U%+uMG+$vfM z1=Z$ZoS)>kOCG)NSbuV?b|3Me+NtD9cXFf9+c+R?>q7PYIaHo;OZW@SO0dIT_dDI| zag8^8bkxEhqi^zGvfpoTe@5?+h?}2052Ao19MPc20vU@o&W@B71YVe8jAz@}ceab> zy@8Vqj_*41adceUCVrQ~QfXZKj;Tq>MNrBX3wj;!_U?=`>Wr%R>nH}Z4lKOGOWX3H zEGmS6@eI_!;76p>xn4UaPom1d>v(Py!hm4a@7BDGe^w8(HM^ZRFhSbm#(~>eI&?G7 zyUPxR{ z+jW<&0fGyu+33A8(3<`4Msmb-yExpyGg^UzE<#sw&eT_8YGwEf%q^vdN6-ib|y!>>lay!b^8*rNs%g2kho==c++VI4L;#ao>%oVbqNpw;R z)jQSFN9*H$^}3~1y=Z^w1+skEw6nt;y+Yzb)D7-aj&0>i8J2c`&t;rF@z)aNx4Yr+ zG{6x*b4=1M@MzC4DrM!?3ik_Lo`F!aWG#6TKm;yWtF0~1xVz{xOZ8PJFjOwu*gS5& zw|GDXACK3V+fb{;|HMRFZF{`?CrJz?0pty` zKd6%O41~GdcH!irX4EN+8QZ?|Oo* z9lUYQ&w5ppAM=x<=MR-WcWF%43~EJIjlK`7c#iWb`jai%;JqX_;lR!->R;~Z z{qnRebw1%@j(wNI9edg1VFqmk;&=_@artF&3sl~($Pd_qa{g+s;y2EFX3sXYU3msj z7`6Cg`3vs^*yk5c>t*;4{%M)Ek*JjLc%+_>b3W^hAc%6=0h1)N+b`=3g{|#mQ>hWR znpd^mkuHD#bioBmLGGwj^9AS@xRGJ&uN^3>kBFTgeCK;nvJw*7N(Bu9x+JgV}_OHn;5O0ME3UJSiri$5c zdWTFc7kv}`s?6>j;wAA=yAtn+b25I`wX|fb<6>Y?JyyG6H~Si5PI2?aYxC8}L)N0n z%0G#K=fRY=ukImr5-ET(@DAa7NqYQ+;|{bHSm|5#?0Fbrc+@~sr3=5n7jgamO=Y{F zi_A!&ir=(--gFfOjXx}h?Z7`*EpC3ulbfV{YV;HJccHOMTOQ#>-N)}2%M%++i;MhI z?()u;IX5Oa|J2`xf4Vogi!2zHmr--=gn2}=-?$mnF3i<>{;~Q+<7!_<$=A*=}_~2rMTnl_B_zepe(C-@*Llg z`F)1GXSYniYbR^HC0yDS8X@CtBgc{+8s#e+VdC{wi(64zLI2^bsk`O&(_J$N#d4V8 zLvEqh*H=T@6GCgm;{Z2Qaww<`?!dXiR)v!%43}@F7ZsOK$H+(are=pbbq|LuBUD8_3c!(bmb)C%< z!t*HV@mjwFh+I~JN0O5w*A&HPz3GFI@j6ya{_?fE>Jm$OKwILhnaIqRhgQsvTFj2J zq5b0Sb$@i)Eap0x<$N?@r~KqN^MP!2x-`IU(@Sl$^B1~}EU0LenZ=A@!Slf7K27$1 z_ZKIX;hNQz!=B}1OK#n0bu>p7uHV(qc3J0L*EaXq$ggtCy~4hw@Il1}YQ?@GVLy=+ zJKrz!)|-!Q021}-TLil?&Ed5O1LP7|2x>z3P2GY^4Po%Y{t1Wh!ml=erV5{gS4Z=U zdYV!emWTR+MjuUue6s@N&~Bk1ec5|m34yNCFq%mUCP`IHuh7%x#&z z_95t4>6!D#-?q;6(!n>+)u2Qh(WNjtUJMsUljc5>SpIHqRI<*7crkIm>*1#hU8Bnj zb7{QpY@6`)_~*x5>C|KojSr70i#t9NUjmf9tBfB`=gsr_*G;%5#&9=qGNgUW%X<&q zo18nVF$cv2s<*ueEvt8?%A5OMN~kS4-LuKv<4*PLr#G>_sFsCXP$#JKQ##UckwK&5ZwLgANzk%3jwX|g$?gf?y=F&IaW-|$yxoSEmfi&jt4_a%5 zOhdw*ec_vU4LPyQbb<8R2%z2Eoh6~j{zOWXt3v&XhA)jH|iBb z-a7a-z21&y@wUmAeMdf`TO#TkdO-9tO1g>mLu|WCo_O1<*aI5r+1#opk;v?8axJHj z&*XC_YX@SFvbVnR>)W(6F$r;$bUj;)M@Kf8?|%nWS5{XO$j^{1)A6rb_@RM5j;@&+ zfWj+#<7?8Lr@g_-utlwP5HzrVJnw3IYa8q6s!wbI)jzi09Odle^wvGQ3Q*I5`+x?0 zCJKV@;EXXk@RL`+_3;}P%G1>m?L01z?(-gOvtQ(`J&V`bnN!w#a{04cy{od=OF5cd zDd2>?_+{naPShw*w7>RFZJb)#r16FO7g#8>58}>2Hk&bkoePnI-if_D?c3%rk>h;+ zu(kK`Qvc|p=XL(jrXZRpIuM6`vt9O!vfEqsd*XP^b)(0uoKa4q;&Q*5uXsB~Rm{ib z37n~STqYuM*$!9pWOFB8fFkuV26XWLOEP0T{1o!_teo%ZU<=z>y{-OpRd5{m@0nCr z>HVsNM^jA%1|;G#<6mGtv+8=KqlXtV!fJU%ZhEvSpNe&2<O6+N#zhFajczVoN!B)rrb6>Op^$wZ{;?ma|yJ$vG{ocFAg+QNP2wbY{a)r^O2w} zJ`9t2S+mW!WuE$wqOI;5jLDd84(n*sSIy}d{90f}1PpXW-Ibb@n7qREBytrzac|o6W<784v)a$N*Kds%C%q%z zIXhZNf7R(8?d_QnAM}i-{>NyFV#+<99U2MnRq$^v`QaMc2CbHu4tJy|Y|eq5s&h@V zB*9@3re~oyb?kSI;$UTB@17vNPR3- zpznJRsxRdztDM#+&&9iAtWD;JQ|1~ff{*=;*BZQ5K1iCC69TxX|I?Y6fKZVoHz5ks>%zymV+TYA+;{`$4IV-S%QPmnu)_j)8BL@H4g5FW9Xq~aL??GO#?L})DlloEtv)cGv zLtx>hbHT6PpM37kx$k8DjXJ!+Sv*s`q*m%U3f=$k`HV`s0a}OF{7168(db-Onp2I! z2bj~Kimmkr>l;9#4p>?i>U*A*gUw=&zpi#k5+3%=^Ke`ogq+HM?>fwaS&8C)_=n6n zp3&Yoe0FL=H+ronBtspK@nWhFfSuPLaeTZ|{aA4MbYAcOdp7I4&R5Q4hZ`&A#eW8E z-dR6s^Rx>!0inqCe!p|-z0#v;C3+6^vRu&oHonS{j%d8>zh zWR6syk6_vht24!Q+b|49AP95&IoBt66{%yr+piCtOOkAQyb+nScc zb!Oc5>FoM1W8fGb`S`#p7L7SiNEQ4tjBl5{zG-{6Q86R2s?VX_n^!` z0{U57g`F67jAl!GEt~Mq9%i}TiVg-4(-n@jkcikKHYq@0VS80cKG{>9Nz)u#m3jZH zvluBWN|lsN*VEJUhqTu-QZ{v z-#v8@;+W+IzR098-0J$CTaoVZHH7RMi`gXLucZgV-X>S+`SJT3Yp-9FI~76dsI;L@ zR@xLo4Tw+~|M~g4gZ&S

8anO@#(3rE1VVwVzF)4Q-Eu%Yk*T7MyBh@GF!+i^jfgN{^Wunqg=-i zZTfn*lH2&pC7_iUA~8>)8=?qzZ@X8_k^hEcw3D51fZq7%dh>*1b8S0rPx*AX=Uc{} zqfKD%JUv1{o_r||;uSj87O|Sbs3RIL9kd8iJ!XCQWm=W#JpbHm^@`ejK`@j*_Az69 z;^T9}81hove@@@#EJ3OH@fH@4iF2kKq?(KGdxcV! zU2tx00mx!qkEg%B17CcO&*X-BM%e<(Hfdl2c5R7rw#1PpSDCDpq)Qlh35Jkb9sRY- zeHvwMLAVRyLMC-Pw!fr34)O>{((=a@FEqxxApj@tzeCaUbjV;U^>5BSsqyG_*Dr@v z{tzrsSUdeQVJ|Xa415l<-F;ZwqKsEuM_1Ga3`f@XKf@l?4{}b07w(wV-^KbiMk+4H z1>TD77@nH6(CwPT7<7#JBqvT#RuS^tgCjc>cf7uamYEloIojvWi?e4H6yTnzXx5!$ zSuJC);%Lqcp$%F(|7od3gh1;HS1!Y$JpNFcG}(`TyjcsfW*_L{PXO8m#DK?z=A$8? zf-suUT>%x`;n=k1mN$kNQa*diySRu$v&7dnGl8~-53!HiTp7yUTgeX#V_dDuozMtE zP-TLQC1+V6rLXR@2{4n?S@B2RN`!~0v z)jY2u>M85hH0qc3j~f>EZ?Nwr>i+<$K_KzMQRhHxvXhuJ<28jV{^$3GQ^FwC_d!BK zXQ4Ltn}I>RklwE_vvAm)#3Bm>{QB~7-+=3(FHrrmd~%oU0U00|VOn_|h}`9i17SqF zb=l4GXwA;b&ORp#tNp2`C3@3bZ%JS&B6Pg6+kkX58u=Ncs*G!Fl~yZMC(NJM%{TXNv`6_^Lny07d7+Y{w6wk zc6{r3V8<_2wKvBn#X(DJ2;~PR+iO-(spMns#GyOU!8W{xE|4!MOA<3qG8cAb@nArg zKaQWd4uR&;1!gV-$&$5{KwzC}*|*&-XmTElUa^Vao$sC|Y1az-2DWelFAZ@f9rQKFZoG}h z)>MsS0-)+K)hPf-E+%JqNR>$#11ZhOO5=6*rIyl@)BO9Amj3J2x%t|n#pT8vi9I zyRkZz3PrvdiT)Kpm&bbmS7~opU%JDY(~1djNDuqgkhN-H6CbU!cK$H%22MXOZ3B{B z9qrt>Z%G2A2`R4*uO^*CuKiuioBbIf>y5s_v+gzXlvkvWR!6-}oh_!Pg$_lW)g;J*$>wB_ zTJw_70hhbS?+==a?apM|J$2y4!t&2xMEg7&#L_ATwe8Dv1dCqcQIxl!_IGGn`7JhK zMVN_C*RQW<^E*-43*IeTuer}J6CACPWNLRi`*8x!d?e^Qz zVcF%!S~?nM`xD)*zt_=|`(W5)^1<{|iB|3h;J50N|I@yC;G^*3{ItMBz}4X2>mOwK z;~nnJlA!kG1j$P050g>G-?l_A`sBj5KVLt8B!y+0uaY4b0K;c=PbEQ3U3I!QaRO)H zva`RXcK~g44dS=)Y3oY$XWf{FqxO`h8RbM~$SsUy-hXGaVa*lnAhJOjChRqpFp*6I zH1a)sT72~0U0-FRHL&U)DmQ3fFVLl0+Tq%Hkz*Cgh6Bo#-h#utJ7VqCJA56&#H>HotZV8?}$1Mp0agKsOkI7oaPb-NB5jSVJ-}eR&93@`6)|<>i1tqQ? zoQS(mN@f}I(X~Tpo$qwzIhE_;{nkZmSJ|j=3CiwvE;erI+s|6TM?dB3*V!$P0_ph4 zugzvJXwaZF57&6Nqxsx)R>Heu|9}zM)-~aiuXL|eD=HY<`*jTI%--XEsEIfNh{Vkp z<(Smak!LNAFy?$BFe5%=%4M6y6|BImVdqVJ91bvEjFf*-p+EeX0+qA*s;=*NDfHM& zS2lj!!FFwA?H%j5Rjh>QyhSYL{5)~4;8An9xoj)~ip~;(1)soe=@eh&3HX2CNN2&4 zhl?tpRQ^}b5SaZEKo8BX*tR*4Qb4dNe!+UWI3$*m9p@nS(!aMUwv~U#w^dWJ%mR-0 z6rV}HGaF@t=883x$`E)rHu#9W_kH z*hp+k(Kvx@cu zp+OWD#{`tOAzm({y3<+FX-N*k_^-YiR}wW*!rNX%edpdxFi{cOhBBiZ(EnC);b2nAOd)a`{?|+h9>JJ z-pt*heNxcv+8hy?5r(Q?WI42lzT**qfQ(Af1t8r@@Gf)y~V<*q1sr$!Bj!|LqvfC zhdGs4SRs&DhQpqXub&w1_S;R~*K(Hci@JJh!5=-g3J>jOAJ2CNpAX+s0bG?E^;61; znjhXcv0P|sDpbi>drd+Y3uEeV?5eyuz+!B_rw$jLSa#L)YHfr{v7YV8ywQ=)U9$QXKxKseV*=6Higu_z6seEd}PhozwNAbZS%wS_yR?Ln?h>G%7(gYXNSvb zF$U5A(ezjhd5j~BcsBXONyE(&S5a)ka~<^dykZtF(%`{TaJaI|YAl2j#d{9ofsV^zZ6EDsKTxxqP%gWevgkI~72v&?C&XTd*d>Tw8 zxZIj!`wHXhUKfl8H1JYBl2_Q2t*@u3&_qK7X{cHdVX)U&kIt&&qiB!Hl^$r;9}0(c zkn!t$TH2&syQ`ITL(L^+Qzx;5g#i3e(OWa0kxv?c6)uRy;Ji-y$p_5%@r!-KS+NAF zI6AFFNJCI5zu+bW-hVe`JJV@&vtG>Ddd*rU{*Pa*45wT%yK0Ed;%J(Cm84M6+?_g} zjpySB69z?`p|a}eDd4fhYt*BLbb2;1&*3|nA zJ^(C{Ggn`|re-AMXI~-7#0Q$H!c@(e{6F8Q1qqRfs-|`dEr)dwmAli7Higs0rhmLQ z-3zA+Z(d3f)QXs5*7NV7G=zSqu`c#~$9XxItdnia{%grlI*wQCtw;lL?+4OXKxWCu zZ?qBL)}CLT#ie>(i@8|+46~Cnd8aFCAmsU#kHG5N5CtkxF+bJPE2~ty^!B=`=vQpu zmtierqQx?n+uO&AKNsAaTf<)oH!IP=mW}l#;vGjy^TnTe%7) zo0gV^s=tcQ#F%ECNiM_`!}s>z3-3>Iv?7n{(p9(p!8^^j-9K>^_r-cFLpq<8*_~8k zcfpP0`YSG&1I%ehIW3T^tn4}sj2N`~ehb{4IeJkuI`~7-ow8wmD31SexSlr;|&u=P3|3 z*+xZanp$GtJQi;DAp3u3YVhvBC%d9)ahA)*T^@Bctu zVI$p7$aFfTNZ&a0#=%n<07K{MhQ57BYJE;xPa=X*MMw>`X?Jn5Y9&q_e)^zp?PxRp zMprX+=V|7LL@%+fqx#%^h0DL&54QAKqmp$Gl(YRN+zpx=4*goPWyf#-(qojV;j)E$ zOFYo4WgZeXm6y2LVk{Z{tS0+4t8MK;b)9QRHsZ5^m8z9qlB}oB3UGu$%~5luzf9^a z(}!IuI%-o0I)Bs9dT~-{Yg5%qG5=QgJJBwMjI=*iSWiIe4n`q~?uQYUKeb68-$k`d zyZ+k1@o`KEd(9dzh(-1ceRvS#OWzmU*F3WYJYqxGX{I)5J%%SzR3Ra(nyDBc_M5`Z zG_rM?Cs3;h&j>0K@`V93U-{SXFtUag;P^tU94MqB%~)ERY6b>+?;0vl{-tU~-Dkw) zmucwHX#J`emBGU1vn2a4ZLLlhNNTI0!&K6_1;q_%B%;>*gsLoz^%U~+4Ut=Jj-O2FtLbex?D>5$-gIXi^uJaWN-+wop|2%wwxqPuoo+lx770k?{`1!?zK%StK zd$16S=QBt2L#!p-dJG%(BCMszgNvI#e@u8RQNly?Lgr?$!UUy~;l#K?+YuIdNLvKo zzNS+%Phzr!QCy+|ne#7k=bkxYF2!o#V~Sl+N~j$OeDpTv$v__X#TsT(^G`rL?*%4R zh!r%n3;#C(W|#sRDpn@806_^!$+MXk^}E%b1oad)>=UsNj4oT^fre+5sCsC~R{UzPt5X^& zv0!FFX}>%QsV|XIfoP*BOV|Q*HdmmqF6KNMDKzLEYEJkAM&<+Yc`m*m3bGmniOfDh znTI_gm&d9q8zfbUgqTA)!C1l-=tFS}u|iSy!<<3_AqPRXVVn@kbCmKAVUxq<1D?f!BRWGoj@yjWy3x3DX$a}>4u6_5<~A<{RE8hJmEWOlcWgHZKwkh&151prJa^_Vs+6PTX>((5mukmi|2bL_j8 zTIP|Mh)mA|P&s`u9vClCPcSm;0nDjvymxBsNK_Rh!6&Iy&+l{-{o45wJhV1LY7^BY zNJ~EJ^CnM; zHz)!mHnd1wLL|54;D-<^^t|?vA_vU&b9y=(MeG?IBsU8EJb)VcI6b_=T6%_@M68MU z%J1VmZ|8+FhCPChH4-w{xwvI z;9XQ@NxFY+ILI0CC%8t*oWwuDLaHMMkL$b3UyYP)0im~|AqA2Xk?r(k7lHI-JyQO` za~@J+fFTN0-b5HVW0e?XD1DHfj62>LF{CyALc(CTvT&)a5Ydn#>R=czZavL5DC8Dv z2~*%k<|BeN76&=!n0bP^(ofGE;nbJ^RV+T=c_NrWo6tg%u8hj-_|H0MdRG`J$i(GF z#xgC)56kGi1TB9UE%F(72oVA>fXk4gw-cx2n3wBW}=iRVec3^y6^SXnDQGC^}y@DWyb!F z0L~Q#eCSNa_S3MvuQGlm+(Tq6`ckKufbIDksUVm+tzaj@;wvJOA$nT;f_=*@dl2L} zQSlt^*tw1aldOc<-VGq$jBP2^RDAq-Nbxc&;e{!oOgn%MZ)*+hvX_C~GDKqx@>qUH zrjw$BpNJF>4xlT?ZSVtV#ytv9wQ;@;`)3_8l{uuwKP^ABEhek|jS)&`k%G&IK!~8@ z36qzyrKbsd82qwbBW)%n16&)<5Ha!*(j{}$x&_SD(4+y=`;clfXh(t#f$_>WsJLo3 z&EQYLeJz!;#0<;}xQp&jCJei!p?G9XW(>VzTY5i(mNJ|^(eyT@iKsnE}_F0bip`bEx#^IoJ;h| zSA6Bx9OU9~eC?sRy`GjPcI{X|{&DU8&C$GPYx4M*Gp{3CjLS zY?GSO7XY+T7s{cjlgkv3t0q1lBYmt())#+dXZQl*xbDW!u3H0FI+4EVFp=`mt|#71 zH(jmX9iOf3xn7$g0s@5^W=fJF@44^22Fi~;vWlRK@=b<@T&SZ$z^JG0TS?*9@R#B` zkI@NHf<{J2j`SVBbA9-G=dX#3!|g|cwvOmKfLbKMm+LF4Frlx`v(fU$FMFG$Vb2^9 zY&%aD3};rlOF?cz&|iOaiHkBsLCn&DzTmOD19C}trcM31ak#p;i7hoXSWO|P0SZ8;!a%vY#iRZ|4+ z?YFe00)|2LCqC2P-|N4FV<_el4$<{jzoCX4F%9lYP;#1Qt%^+;_PTf@HSVwp~>spp%0U7rDl~4{g$QIR_d>) zVJ$cxAgiRi3F|IB{c+#ySZHI+CPFaV5rxG6UOtQzr;N5_F0q`yJK^e*3<+r3Iz}MrTD@j=U_1&Kd+hpu`|P>@$CH;fGd# zhYyQI`oL+$kh2~4<@Bd0eW%jykE!|nJ9^{E`J7ro)#auymr@}egqhFmGk^ zejOHAI?QDiC&Q+w31LJC?!aR?^L63^a&74H&!nFSzCJ?(5clup{(*oA+WdBa-#3gM%47B(P*rjGu=yd6**id=bm zroBNwdrra!;v*irIJO8tBfVk4Blo9jbY)b$#lHm`JGNal{+n$L*M(w_ZxprBzvW8n!5GuUX(0!G$<2!EB=w7%^^%{6OrKdNJspRi`wAEF| z!{0EclK?vDv>`awhN4j;_~(l5x6JCv#BtOG(_gC$hHbNN0YckU-?}p3zNfob-({dd z7*YF`BL$QWTM;Fs4dF{_8d?K$@>e&L%4tA=5Ls<|^;vdg^oCrR!3ZPXO_fY{&Vw_v zM`dGX9s6zU7R`0MJ96rN@kIG8urcXLMX*wM8mo*xZ^pQAFSypB?l# z<|}lPqL2i8NebupmTnAQ!{`#2b*S8*a}I1249S6hB?PxUU9JyAzW~u>f&S0mHzxIo zP%5V%=Y2A@AHL$;sWRN)?YvvQWj&>ioOX!khvUGW2GN%VYXEi3yQH`sUK!Zv5)c(! zYEYffz|wgO%p+stLU?{R&0JMs1%`Ffry)@Q7WgT^X!=s=ZonJs>MT|--qBs#d+814 zHFLiHX7ZF}JhQ13Fw@@!aPIXCK}+W!qPqO))jnOGL%n9NcLT)tAkJ*wH;ZaV$5*&l zJ^!v(y~Rbqvuet`~}WK zjo^eJ>2uY$o{eo?zd46iKMYUq0sab{Go$a)`$4Wo;tSB3Ob*ur$(QOi4J&)qS4KbnP5fQYUXu?`;6Y zp{W($7&aP$stUnE;g(phQMpQdb9QCFXf_=%J2R~1ig9l4wz_zHqpkNDGwqAP-r_}~ zih{Xdk$izG?$csNrfFg?rX85zN$bt}@Ow2~@`nsOXWmvSRZ^M@B@D29?>JPp{QcK| zuy3Ve-FgUBHsK6aiPtS!-E4?=$bbjyrV@gO6=aDWYvY zQEv!5{DZ6W2P?c|wmp+TPSQNP^mdWyK1nG8g^qv5GN~u`U73R?$#em4>=Jwx4~FUN z#Y{!K`>Z0K(mD4zYjUYP5KK zL#!`zcwX>>jmYRKHq5bu=N{^=lrsH3gZVl#?F$@-W4N4Jfec4CKiSR2K(>%~Z&{$WcceX^B7Z(29$(#VzPG`{v7oXSe(9hToHnkOSoSS z#jCMw=n-XeJ+^H!sa3VfJ&*1_*~e_F8)Oz?P-)xpe4V9E)Rn3BVse9+b_xG(_#wP0 zA^)hA5}kD ziq@CSwwp5Eqtz>+d#DbX1awV-SN~^HQ zDE_;}WAfmvP7HZLL-)YN7>G?$W+v^(Vz=@~u&{wfX38%m+~hAqX%e_+EXb=x780qc zcrpy?s>%lB(X_gnkhDB*8W(9)*E}TOj3N6qct(-p;gprn!0eMOiGVloRCjUEiM~8#16JkI0442Z zDwz|}?_zWT3=TAeeWEg%swJRHKB0nI?X*!=gRkOMye{++mgBMPyJ}JNkJlews+IlU zstGkut7S6^4<5_9$Q6iR)!V96=guN;N>aK+rK=_==rs08N4_WpGUSz(`A`PGJpC87u5CMm3vK6 zlGb_iW_z*{xbyjlTW}K#K}Pxuq9-w=a~=}(j!LQ{Vt{IM96V3^P7J>=Qe?~cS%aR>?QHYb3{rYNI z41l^?R)uQ0a$Xg>+Dg3rvR-PRVz*uksuoWlZp=>n_HQJ8j;q8#@=ucA-$;p9Z%rO* z?76QSJ3kKkmw1(2m+DW}U3=Hq9}5&$*}QH&l$@t7ywsloL_IGmL$&Kb(E8iQGhp%$ zm+$?JURdNA0CPv`!!(-1==(@4_NWjAdKn|g>F%@|oQVNqHeS(#pSI`ABYsRi1Bd4| z!H;^Vfwb5OSM#N!k2HHnNnUtGB{{dHR~4ZQT(b{6e*L4z<-w&_dePZZ)HyhUR`Pz% z_s`4^w3~oS&rV|B!`ejCS5`6_kAFN)$PGqb2CQ5z-Ki@eA65kSU&rz|bzC>{aqQB?B1baOo zeUdJ!evn!|lYveM? z5+#IaXz|X>g0Ow1DB{c7iI@32)9>A*N3f25#`5aoc$w1_*oCC%=fM!0g$N|Z=S-&G zv=C%Ir+i#0L(P27tn$b2d7nz?;L-q0HVD8Aj@}D(k0`a__}rFSr$&u1!JQlPLmOK7 zFui{TdO$48D=YWIY52T?NE@@CVDr#@NzdW)59ChdOzPj&Yuy>$=g@#x#K#hv1@0I6 zlaWcKtF`*lU!<9J2araPfFROZ?DrL3jiK{)hrxAN!^wMvjjIoDtoP=FHiotzK+jTX zq_gLP3wuTI@k*Ckpra?aJe&4|NbNX=JmNNjTa zINl!KK?*#{oDF|&?!)E5wB1COvm*pxJVQG2;tO9rsBP?omdu+Bb&oi7A^RMm%+lOD zmt&{=0JT7%(Cpg+j=WOB7wagB`)HqEy}J)HMRA|C*iLSI(zXI}V%OT*p%wQbd%ssn z34U;ZaWH*W1>+5t=D5Cgury0v5I3!5PqJ=(FnxHo?1-RQx|@DV-Sk@R{Ajl>)dT+l zK)XmZN{m^2pW}5}C{QUo+!NyUiE2Zxr)3Zd<&f!Pi9pLdVG44XhoIuq_ev<3X=q#+t=yCT4@gVXR zGYj8xYh9QvoWqw~5vw~0!IZd=T^pVwf)&9sVf!~at#AB4?%-x)q^u5aD{5~mGAizE z6b0zAA18sYV9CMMaY=y<{QnX4LC)YhPR9 z0#_}BAoxADvv_`uY2xqh>e%Shmy^Rxq=vD25T?*X{1i)m(T1!2#^-xHM_wV3-O)gh zCo&RhKHZgUt^|%1I_dvH^#3SJ{rPT<358 zz=e?fMS;mk&UG*U+?&unDL{J9c_LNKRP5<7=rgDE#wH}!BW2Gy)=(ibN#lwZDwu`( zD6m7|Z8RV25rTO!xAD}Sq5Nfulmy}&FyDQ_Zr(n$vrTic2llZW{MyOmPUa6bkzq)g7Ceb$wbksnYXmwo@{Cc@-Y~{pkkz^z@e> zP5oL0MHGFX8ca6`Wf~HxBpX-97V8=g&0eR>Ihzn@>AUcB`gG^Bp#VW zz5nrGLbMcSzM)~2-?j695Y#uPwSt9rSfAFf(Zz<(r^;z^MS<1jWpvT;&Q4lyZvhR z4S&D?!vEcta3pP?-fNwxhSs3EqWU>y>-!z*&-!%+!Fngfqw zyzzekPNo0DZmdb}%*ph&`8HMfIuZJml+I$5KN3C(n?GMW^u}i zQKqAI3E>(wjN{EIA04dwi>xYB(AL$|Lia_#b1W3Od8lnM3Dl;0IjU+i57dTKen2EB z?pL#$JSi;&>EO^V22~X`>h5YbjeaV|%?-JL;`wtXy`BKz@vMeC>q_@YsLRuW^zms~ z$h698gUgl(Tz&=_&vZ|rMZv-!d07ZeU?n02WvazPDk*1y@^s2V3>FY>H}1uFQfRm; zWj=@6CX3SYPcbGzK~ss*GS6@(G1M>R;Shp^lH^Sz!Zfjq<3BG7bEqIulSt6=7ni41 z)L4kZ4r&8&M@uczshr0I^z2pIx649tcgR7TeObc(48E&T6I3Gti=vTmqnpM4M(G-| z%-2vuY+>rFgEhoLVr^!~E_xncyRc#4Z*(~K5yn@P?U_HoK=zsoQ_5I}Gf}Hrw}6ru zqdo*hUctDu2xbf}-`z9j)X|ojBy2@kh5OlwJ0JsLu3HHKGHzXk(Ou6r=plcK<$HjL zh_StZb%lc4ibKAzB4zlejo*vqDZR<#5NYCu+Y)L8eMZE0}xPBDS3M$+HS zt=jCysoB=B$Y5$Q1zas}1!Q0Q+?iA^Qf6(SMcqqoU=^|FYfns>75t!H1V35B_3&D* zt)ULG5#z(2e)uR?bJ(w@lnFReF}6yUQ(h* z28L|1HbuAMhRS1KSf>>dJATdXR%u0h)aWN&nzJVxXd6VxhmC%;t{^{_4*MH%b|!Bm z5&LRhb$q0PoPxFHy!_b8qdsrcW^uX){B?`jcW}{ctk<9Z>-E6vzWW~O`Kosb{U)z< zexNJS2g~{f%W@THBmQJxAowXA>)T-%A^JtXbLPA#9$wjVsd8o4UP9)F%wMK3FM{h- z?>_|aNgsK}6gHf~7wD+|lp9V`Dg>4a#%WursLKX_?Xv29m0zoJN^~pxhA|m01T?L7 zZ&A1#@g&Jw;TGrCh+gfF-nYao zh%sY5DmaRMQg|JQ*oF@Mx?}wyU!cGyV+?-MNW;LkTJEs0we*=2*>DwR%!1<~t>$1{ z&ILw`{lWc&A$3y~{n4d(CHVmKTNT>^c}ZM>#j-&8XCN2!g=m|teR z$b#@>V;SF%6)s1aO?5#WzAw`*yu})hFVil*Wr0-{*DSiFfmIdnk=0K`S()IG*sn*q zhStkRu8+NGfVqaf>5D0dvDu118Uc<+AdLjeVA#WKfI{B#)^Ruh2pVBYPMxfR1(G`FLeVqFW4~2^O1ZvQU9f&cnhen@};~) zbbplC?Bsv`b)-&UWsTs>)255=7|{i2s+OQm_O}xw(Li+!$1Y#(Vt5KpL<1Jl#zz0> z8vl)Bos7c?Z^ScW;{J^%SK1hOgsGlNGC;df@P`w1m3rJuXZ(=cDvwCdM^hltx5Ii- z`fS0|A||W5qWY@dn?ognElK|mGe`tD`Q-DZ8#<;TNuOOsz&<@aUB$YzikS`QW6grFrhvmW8Bi3RW(^4SjU^z@3v0KQD zd>!8yZH-Q`f4FXhg)*6D-W*8~6Z2CMPF3UjQ3>^B#Bb2Z)o`T7L0uLAAFHtiYYqNH zao{7#fve!Zr)rS;Z7gzi0Ldw zkGi;&4Pr{Fw4f_2eg#vc?0De-8$?#h+HT4;vUrMkc#R)AST4Dyn!mZw?PNcOxqm!r z4%_!skJ7_;Hl$>$cEV_rxnA;&a-aaLSk1?Aaz?3HSm zEn8g`*NHQJp;^9@sW+H`?`B@Gg5UJA%j%$0=5 z$!lB}wzhkNg=%sFaQyo|a|xr4`YK1>-dOv>Na;l^O(9A_t|*D)+QNFkv^p|$v-rU2 z-$D{2-j~0S*!q1|J6>rq`=UTs>0K{nkY#UOIC@85@R4HnmJKEaa=mWLX!xcG-_O1Fo7Ut3&W8WZQ4vmX8 za{8DJ?NN#yAl}=NV$cX<#vHR0JsTr%dog-xXA~0R&!n+OSftg@sj9a>$J$Hy(x4CV zsN!g~!2>Yj-+kGtDv(?0=zALAtJDZm@ax^>Bc3aX3!cYW&lvani(AR%F?7*kbBSHK zHbB%)_fXF0_qU;~8a%iw)i^iVIuVUhWhZEok>eN$h+!O5?qi=_VcR?V4p&GloYDA< zQEfD#2<9yUnqAvimhD{EvpZ~i7X0l31dD;HQ6~~Qq~W7NBCFrHMChwlqn+2> zB;L?J|1*sLW0(FnJK{f!Wk||B+x%|iupm=eWZw;AcL?*RR0jcK1RyzD#I9EuBY0hX zc|n_5DDQ_=Q_b~MojL|4dW(laA{jM9U2PlQD-{!AU(m%~x2i|o%j3PFQ`6skyYI*baL27SWdMiCHAPWPXS~6VM`IbF}9d*a${~GizXLWEMWxan&K+clYgRu zp%f|XH2fmREpT?QDq;SZ2l{&`&v)v2)O1So<9JRok^H^!jjN1tE12fD{9X>+RSX;}v*gJ*dnWqM_f7Fyu`KNX93ynGxII z(oK{R48aUZ#K_>Hs&@LLNHyZA-82W;#Ej8>Hn|LcXmoURMOI3+;>7ykb0(_DP8%wPW&Fwt zJ()3D9Qn35Vp{YHiij-Z(Aod*8S#d_3bO_ z;IQpZl1;|zhY;oV7py?mjUCv&+a#MvS0_Q5mGbO5Ysz$Il-1V4w&J9*zz89$gtDQg zT2fs-y~4BMuexFF!)%$6CH9Bz*KITE#<_%x^RKM1wTD{6C^e}RK;xn*OQH-aHx9T> zuy4)dhSjdFuo7*JqyL8kZ{v_tp!q6u(EjQ9(UIvd0SoP24`;Ih<73Oc-d;D7rGrXg zTARb9$CJ!shEW7Lz*2$k$K1Hh)uXCnd5T&+6i0R&6^cy)Q>A<;YR7>}_EDtBB3e@6 zw+}{K_OXY^B6xvEz=ZQkAt8+etKu|Ae;h67>F?f$lO#Jk?kF@=hYbbPDvDyVnD2DO zUn62b3aGr!C_|0PABjW6_c>0)h>{mH>0S{+rHEYjTty^v&-3yI~w)0^9@nupxs-wWb8i5f=&v zm&VmWG*S6>3Dfn7w1h4%nmuGr_I@{UEGM4Rhk31E5P<$$pHlM52>i?A0dOpMYBg)-bk+{FM=ML`$Ga`n!Gr%T99L3n{ zdQH@Sw%pxb0rdCE_FKf_)%%%Ax>Rw4Rch*cfjPC~VU`9#sSk^oiAjD|ol0wXBvm^c z7b=E35w3*TMCP_)q&wtKzpH{Hh({k9Z!%8MV=~koNyd=A?6tFPrn`sDAYG;(#%Q%P za4tFwCmh=W*&5Rllc56FOTR?Q%qfo=WIGEu<;}za;8Q0htqfygK{(AXPXdTr5EmAT zUo`afoGcP|SW{_5a+D67dtzgz8Yx+l2T~E-x|U9}k%@=|ROpYC&uR6A!Hvct^#z+2 z;*{wcxm{v3zPLv;M0O)ZxU*}lvN~{OgOq=n>*wE@L~5UK;=)!`l9&o^S&p(rvOIr0 zs-)@HPxiNa^4n?$qF5VhK|nF6k|e#C2oYDfVyS<{vMmC5tGh9sn8p!`U29rt}fcCvTd zTJ$q`hgSeWq~>>?v5(XdK}G^PuS(u-Txdj{hQ`<*Q&V^*;ksiJeFXFe(Pt- z@oVE>d2TLpCvN*YE|CMq!Ta%hz03vyPcB z*y5Xn$rvvp9hrT0kgQR7qpI{T>7}bg@eaB$9P`TdbE;3I|CH|k=+FPHbpM4&7ZQ9w z6E|eU0X;1u4V+g+D0%FV#u?o_ExMTyf0Q4ye+`r5^gzY_a5XqgB-Eki#%bY+h_K&M z=XNK-tN5fxx+$%$j5p&A&nic2i88$MG(XT6c~W0|pPX;kuIAUfp5cs9VrRGmVpX|g z+HA1o+B=3!e+~>W2%22bMowz4Smo7eo0wF828zjQesAD7$TEyF$m7|`cTUjD^@Kq2 zMStW*qC*nIw>Db5HP`r5!%vmV?|36Cfd>nBY4=#%S)@^<;-cW7$5wc3<4z;VqyXyG z=RjReOoipGOlv$epveAnIlGZByj-uA1QmK&%xGmtL_hORz`($JIKdZU&v;Zh^0ZMs z4$xGTRnj@)Vm!r1kw$4=$;)M!RG(YNYeIz5bgeJ19mlXhPHJw!Q` z_@SJF%eAUp;b&3i=}3Yr;=0Uei){n-D)3V$02_6{l5XDs#Q7(}KE86azag2!#-set zvd&OOLRrihkDYPfncH7CYAu1=f=I+aGld%Ib6g0{^^i5LZH2h}wHmey7jt{gY$E)x z-woYwP-Rv1EaT|?&ARM~u0zyeNHTCP>56X6sR>2D4X{QpJ1;9)I2$CWI>?d0W&=Oc ze<;AB!$|5Tb{z14+%?B%+K<1cOQ1FLeQAfzRx_HX_{KU|++`C(UcRV*3AGR1&_Qpg zoKRQOO4ebC>70efzpC9;p!J+ybw|QMzT3Jnt1q_3+=Bi1TCgCskRBmuuAbRXi|fu@ zw#Dx-X{H?irAwO4GhM$$L`X7)i|rd#W2W3d0-JmB6G zujNE&rS22J1k3t_s=V>%>TXykh2annmUzZ^h!iR6bsZ1?V0|bO!P>i3m&HH_#go4i z{zt1_xFm#93dYgO8Ed*55=VhN$<9<`JEkgsn;wgm11mPJzO>b+m~xdJvJRkg)oZq{ z@oa9z@pTz>tj6fznFlClFOE03ab5mYB=QmTRDPW;ALfvE!i(PQUrNOyYm|h2{UDp0 zfjncRaykWYy*xVNK~>Eh!DZ1L2xa-g#gqI)uIjWG8VQt6IFTYU6hnfh3@~&1wv+|^ zj9aHR=~M$EaWMqnbwh5+182FXJZ4K$=%ONjt<;XaAeTSA@JdrelI1vXZ)BkO$g`|Z zp)C~zQHbXSugGz%%5A!};$qq!4SssAS*`s3JY{|co*Xh;!C4u;@Yt1s0a);q*gky( zK1Fm9M&?)tD@k0a)p(@yU);Om0Bm?b4Q?k}#N^PH&Dy~B+~?O9K>K-D^S<)4-}^W< z&W`|dgi*|Zk=+&zl#kUd4dr*o19ajT)fIFVS58{%#-?9qsrhpTrEGTTVwd=C~40ML5Q6V!|~M_t_r`@-Ys1Rh1uX&s$*)J3@(Vh8A1rjs#-F z6IG(eX?z#AE~E;8`w9Y|A3}|guWg_HLVCrfUWox)>M2HhR>W<0Nobu?^I1mIb?nb5 z9bM=GWXcf;J3O^jU-W7aYPr?a5TyMV*>sKpco5w9x}k>H}74qBA6Rg(Cge@#{US)%otU z{@;ApYV`4hO)7Yt!WvLe!dAf%sHz|dhQfnUc^03=n^M8XD<14;Ph-d9z+tKNmajSm zZ7Q&_QEfO}&@?**lVTb)$2U9f_L7>lJMdP(&_ji1x$bp7O$h9_ds{X7buPm=LTh`oX}LTFU~?MjL9Iy3wH2?)!*aDfKF!Uj@+8yYWr|MMxG&xRVrY6Cm}r z+kla_@-2q1$H(?;Y8+axhokUbk5ee}Z3PyrH=u$j)5*=p!_fhGk=_Rr177vH>?R87 z+$G?i|B;9P!ZHpK!x|o5OJL4ENit-*iZJbJ~?p8oDK`N9uBLA>;i5GjXXLwcr``ruHm_e*yDF{0>nt z{sT78GrBCy0h}~fS6Lm$h zXn$};fPe^kPnQNU0NPJRIP5y*Z=U7kd!;OMNEklNNqAYNdX1H2X=wGvudW>8<{`oD=jYj z{9XGiE)$o7Z-$<@-H^{(qBHOz$bvyEL#dvmx;4sjZL#28pn${rZqw|(IY7&`ZPpN-xLXT(b7%n6_X2L+EBH6K`qc(|9oq>(@yz70GIt zdD-z!@qt?L7Uj3-O?AYX%ATuHSSr)cd+L;X3z;opfCys2JS)k!wL|_tyELqon(l|i zXT+A)D0?=#4J8&yJ;oZ;zgTYj0A71V@|G@ilb?}bp8>v)0`eBv(OwHbkowI z3$f-&2P#KbPMGXeC4Mw7@?A;nQZko>CCe_m6PHe_daWxo+ZI;@nPFj}=>1@{+hxAT zviTUYDz2u;Y{1x8Ynk8_;-X_&CT?#lS;%$c>d;FE(j8~?{8*M!O8rfbEo(TQk_oeL zxPbQx1($xasWC2e+EXL)>UUfm*eqmdO|Gns4v4su93#ddxYez55_9>?kKTz~HYz%z zmM5!`fT&u}5eb+tHST%bY^r3*7t^`Ct}^2MG^JwZQ+TvT)pTQ#Qn5Pt62>Z1p;F(P zBYj^Ql%uk5?(r-TREb5z(|Y`k(f;%AJ-T*{3x@neyB&P4iV6>YO+74B^q`{B@qyX} zY9P=;!vbM2F){JF!>VW6D5%B`V)maZ%c#XSXIk*ZvdV@2}} z(Rm0S`S|#w5_e-h?@vyB9lr-t%^A%G3f}V`BGwi6i)!Zj)QljcD0SAep2-L2B&NH| zUCa+pPQrumj>5f6=FeDN`O}3S)2Y)jZGZ*JMtzhHzc3=xwgiK$hs@wuGx;`M%I7x= zhVAsIr)tqgtIN-}%f#!xc3-t@R#Yf_34x`+pQsvk^M~sUUK(Vc{lB=l;=N*y~VI}|Nt~g3&Dy*eN_$C3I4n(bQ zy_b*03Js?PkZrp?*92i!A|V0^7gofn;E<@UcewJ`kS@9V&?sQbi1ju44JSFCvkv1= z@Kt;_!~W$fF?=0VeN(NeXa$u}!s17qN+*=ii1A?y0RBWN+Ph`ty(l3g0+WZwFPq#W zWa7XooVTKsO)P~iiT&Uv+gNiIkekLmX|fCZMQ(kSo{ob{iNi1*Ux05N-CCcuNRU!k zm>_K7ywGUrw#JkpSL#@LUoI;2gJ1&h@;8G!Rh@E0b{Lh0h8(3hNqf%O*9qZN&?v1} zI#16O9s{b)gu(tW?_b-QAf#$M^`UJeg(8#T^f6La5GJ3tNRE!(XGAnSK&IS8%<0$d zpWF9$3l+gz-KQ|x%DVhR;I``S%jUkcV0`@sWo-1UU&ob-TT!>b>t=@1?I)xJ?Vj>OLobzDori)JuM{E&hF@ zD+B6yd<|kP%__F741=ui21qcWQCC}Ci=yBh1~vi=5DIYp;iKE%X6%t2%8fmQPUkm2 zd3nec3%pID-`MgV=QcelOYj-L-%u=Qr!ktXvhqiYs+Q#L87-?w)wWvC$K?-_5ySOY zl8{n{lje8GPtMA?>pH|v{XMoNDyG@5LJLma9G`a%`^uA9RK#0R1f-ka?pklsOxLGS zh>odfjiJcwJ_k&({uX7*d!#E=ZtmP8v~}op$9;|3^760|khMebyEY?=vG{-?O@!gt z^M{~Y=@A4+T(Ale16m?`9T#MQo*ceinh0KD&?LWmg*C(c%;BC6#*P5!;dc_<} zpOzP*=@8N70A5>M(+xp~|Esv~j zyi=;VHVk?MnI>3c-j1!e4q&VDc3B(0t6yNh9m(V#4U)^;8k6E8Os#ip{cI&AYfbOQ zzP_d8+@fXQw#jdkF51IVF`$(GGQH~^a`~nMxv>gW4g|1kL)%ot%^0I1gz-aF+Wc6j z&$S8J|AzH5dN@ctZ#%3#_y(%(g!uaMaK3}dx7TlUpV>m~vrt#g*y#C5f@l}#imIt< z+}~-1h;2-*t&b}B{=C`2QNOoHm*Z3lyO4@9tLBswaO4HbDy3xepqpaVy(tK{wuDPIS#4Mbr0#T z2Nx|Vs~X6)_U>$NXU)a=+#Yy)xWE?1oHHy`R_^ma@=ZJF`?hi@0VhdH?WBb&!O@Z0 z>zl;RKLu9q!vXrHq(CD&F~?W)mGxZi>rQo@rd-X-36YG8m)w2UHysbt?7B1GQO0mJ z$F7>=B*&NbZc_S=$twR$Q8axMa@Xpkfb*3>IMW~o-L*_Q*4eg&Y^a_QE*JmeTVeBA zXR+2oWEsNQ`rx!vmh(%4&K#a9#THlE2P1$Xm@_ zf8}ssqE12yiMK{-A&)n{JUdc~dAB zuL^c^kq2rl+7~@08FgYJbQ*j|O0&p=3T^Gj>hAL^)C*&g+@z=qlVS4+SMl`5^zGZ+ zNK%#m#!G4r*8OY<7&I>u5hr5;D=U z#7|!t=Od*WAlW9SN(K9N{nv%4aWXl*CWdzDQ-m)acRvAL)&sFWUnt+oS6^AfEB_ux z{D&$j^+?n7lmTBMEd~We)<05zKkE+Bh2+L~^V(yMnXBVNo0k?H8j(rfK}^hmTy`|9 zKhgjhU!u+faH(%G=zYYVMkkeyNT>dw==oAQmVW z^=kF|ZAP(P_)?l`Q>6M1lr}C(q4x>a6Z8h}xL^OS5C8Nng&_enmB2PkowMQRd`1vF zLl%rxxj~27EZJH43yyw%gos=SEoHzFBWY6@&K~Z!GdpqzW)$2l-X$b;Aj>uUh}ll4 z79BhApVRc;X9oU%pQe97fo!OPm`Q4NJsMXkn1u%|(IG_}7Uf0~2&(&&@)?ToKi%)) z#Kr#!asN|xk-H2Z3WfHs$x;E6iHHROAx51z3IDI^0zUPao(mbs(m@u^oJ$sjhkG@- z4-^&^gy{%-)*!nZRB9S<8Or)Bb7Q zx>r`1?L)H9BVq@TqeMC-By$DwdHnFgpW`Rem9&?G`wtV${Hqb;wZru} zp04;?-V~sF&~(j*j5`I$T&?3hl0F*gJSpeJ7QSg)3XT-8U3-7XO14BoO=L&GnHZ%` zPp@_4^!)*c{B?`?!|v#Ba*8^3&l&B)kyOcK%K&!RnM@jBICutU*M)^i-6KAj(=i+( zvPg#zNWFR9Ba_2A?MLFsYhL_dhu*c&;jv(-Vf7M^keR~B{D$A5f0k0`If-}|H$i}UMc zn2MKKOa}%e&J$}@He3|9>`GF(vSJaPEZDY%`8#WmI zRPy)}fM~r@c#Yna?kQ?kwxoC9L}OB^x+C9dfU0v#*2P{wg=apD8U2DjU)}!gV`ST{ zScUHgs<|>2r;Btd%vy8bh>{rXOXHY4s(@DIBl@eMn_ZrGDvH*it(=NCb_yf zx4ViaSgx$(4n6+HU<){ zy#Q@LF$*Jk?a_|&lN%v78;}_yf-Om!HUe9c6^@`L5pozsNb3WuilFA*>UyuQSWiP>KlrI7#6z9E=Hk8oQHtx%WAwzO&KosM;jpODtM>f7cPmajjaxpG1w_gZf2<{eE zk2JX&Mj@LZLY0WOWAE}tn6I3UvBOu0+QY?g5>+@!)>*lYko>tT_)JI^Z8D2t4G=5~ zlU3@<3>JCb`k&QC2)JOYZ(Yg=4@Xh$5r0%^#+FL>8E$_WUmZu#OAFtyqr>$CB_e>! zRY*`sDow_yZ5DY~@tA^uWhc+ayxpnj-DiK%d-A#f+2Xn{jSyeC9-0<_jS_4Y04WZ? z3&)6s7Q4kTwlDa}MC1Qq>z$(`?Y6kVj&0kv(XnkC9oy-sI_TJ{*tTtUY}>YzPBMA# zJ8SO!W@fFbwe~va)OqTUXM6wl{z1$c^fyxnpmYa=;X=P2+PgR*x8ZVW4-JW;Zo2a) zy7gpg&*o>>fZss7;t@!?7}>k!yYjIbYG@uU%(HI)ObQ%J7Z1Ku`+ylO>$>$i!W}X) z1r9MJWS*4e8`r{0WKN1M5j2EJyHJ~mqGWCEw`h;S2C19C=m7^0_T}F-sK5CRUq5nc z0jh%(aw=-rewI!ll4S2XE<&?|0RvSCq#1pLN$o9op0rPn&7+#m)qU;ioBf(3K4m@} zPh~y`-f?rYjs89=R3#n2&B0Rd?{?{(A_!`4YL^z>GDveMYGOyoiRg=JgxkVNA88?K zh}Vsd#xG2k4^b6tjTgJQ!>r9Mmm?Sr>xza%1LOhc3=q+3y1!27RF zjhAxPrOH7kC42UI@}jQvy&im?U$;HmkgZK^-xBxM(RsQV!z z{QN2on%N>wD4@)p7MR#NjX-RrXk$zcX!@FwuxP(>YFFF%Q(14(uHTFf2$5Lztl2)Z zHgkEtq1NmaK@Y}ewpFxh7v}pa0+5&~HV^MuO_a>8r<}-Cqz;|`{oF2e zG2lny*Ugu|K9oQ;kuJapV6T=tz+s68H)krI)_h6-TPNcY-l%%%?Y-5g0|3a8hOM=B zH>rz~5By#N$*&fz4q+AZ?%ZPLFO!e{)hH$sTt0RZdAB4vn=_Clpx^m1n}5>>ABN=Q zVlMx?gfkypg6~|C^AQw*tp#gk2_Mkp^xd|)FJ*+c>b+SFM@9IyTw=- zbOroN9^<|s_+acNgN#uTD0+d-$h< z`jl8nQ^&4oNr`7C)KIq3yANa4%ujN+$9!C*TS5|!Fi$=jQu7ug4$xK0{JYRVX6PrW z6IO9Gx3W$ei&c&`s%R=xx+s;j>@9D{CRcpHC1%tu=_)gy>QweM&ovv~LI`D2mgzhG zf>9R0l^i=XpFz4=0(C6Xiy0X}whs>053+$`F)gP7BvXl1pHSAB6gKz`^iUXcz=ejC z*#3cmg9C3JEU&Cv4d8D+MNvTlm&|X$T?~IXGH^frmP(a!IBp;i4K68v`_4~b#~%D zELTdB$|IbNId-6t{8ODvZ%7waXr7IfG^`D=q4n{s&>OJ_X#V@8XWG{ zP^4<)y|#OB1J04)kF%rh$(vc<$(CN(v8$U>^B}i7<6@b#=1}ejA^YNXDDA+#c1(I0 z>$qM_4Ecv8__ZnPWktIgAmF2JAnAu!1~~APZjoreA~(b-2%ZjOYw z_;ht>-IP+UNKAIF8!Z0~iZ(n(_COfFoFWcw+YDSBpUnYzX+ne0{`lfS!!cY;4{6LP zD`2qofP!MAf7MtNRlV6ai$pqOW^VZfo<9P^xB*WM!s7RXJT13~`5B(t^{o}{aihZY zPpVR6{Wv*_6|CcMK?GBNu-m;hXeUm(W1gOZa+o~YuhwNu125NmDX3eX9Nxu^wgKUW z1D=(aCA$sa*)ngb?op}=k5vbfC4D;PtAGdwU3rS4bt=oSMtYAcat8LR4Q+eBH2M|D zUFFjS8myU0sF>JzHie?~N=h5;Fe?Y2?->F|PNQ@qAaSFps*`AHKN!*r;`4Ni5D(7@ ziUYV^N9KF=Vi{R_m9&RprwHi;lhypVD&+6&v~Cc}a;|#sx)m4hITw1U&W@Q%vH+X8 z60amYC;DvxoB8&y-`nCp{HyY^iEBDyzFmV%9u~LI2nLq=DQx8!PIH}Tx@P$07ew6iv_@6W$}Y>l9-Hyg|L~us!!{Gn`wmZwbb_fq zr38^aiF}8nGe(5Q)R72KlugAl7y=x|)bx^OE5R3E<8Eq(&l&n;B=bqWn5@n^Jwv?~ z4$pS%G++PPnnAsvL)1<~ubM|z03XS%*rj}_UOJw!m*In(Wt5^Oc-|Y>Uj#n_EDj6*#x| zP?Aqrmzmy3i&EaBRiIk7a2dRHwqWIBxKn<1L6TKkeX81IzS(hcY9d8yVxiS$-z&u5 zrrs<5*w~yEV}^3O^0XZFL#h!y4(alkRH_qDVB@*_;VMjfZM zX&bl80B$oYxh1)6=6gRJ%SsnlnJgShL>xanXvtP*!t%G|ks82IV{WR%C7iChJao1H zZ5gp|0gNUte>qeF@`N<@BZ2(H>ptLzzqc*OLM87P5-;9(JqKlU?@UEX2R`2XRka|E zyU^Fsea8N$*#WqI&=%*MQycb==H`*{Xqd9AQZF89KKmjTbO|mckXMtgxY6e#bT)=! zK}ke1-+RR69b-TM-8&F_gybEWhhdW28eF;j!MkBnr^PtHT|nYUH^M`HnnJP^997Q? zJ(^z{K~kD!N8^X$_dC|sTpwtS_Dt9pM4dj5*%u$|EGKGleY$rHT6&p*x^P*FkEi`E{?A^(hA+ByscHq`iGfZTsZPX3m_HTEu?&10 zT42_|_9$Q=hl^#IfC!%oSbzJF8ESZ-I1@j5(6}bQSV)<_KP55pX=B6rgHPPMV4(FB zT%TW`csL^4I%piB0nwLZQ@8!P%X+B2jKHCkF_A`duao_khYGfW^S7(~wQGSkysJDY z$9ke(-DB+zfI(mSk$i{Sro<=WCfC(m?hg@fv{v<7Np1!-Y64c7RWTC0?4Q$)5h|>{ zA{RWWCDw5;eihx=p#lSZ`P=be@57&9{{(&i8%X+php9%+1NH>IQra2FK$bQ7{~&vy z#LUc%-E3Wfek?rLfI`g`O*9MiPXukqzCT?wu#6=nCAo}^usSTz0m>EBq^?dWK0Vvq zy{Rzajc+Dc?Mz5nb4OhbM=j{nUqGH>f%}-#lgMGlr>I(bOis8EahV}G=p`qzAno-ZH${=Utef#U*dm>? z4EVV`(y}75dWsVAJKXg%iTQXlE`16`^zsAs4ZP%u@TX`+rv>yR#wEYVG9!2s%2Jtw zFdFU1RvCsC0Es*Ba&RuwoC;wpb7f@o`W(E`@hF0-2{m>YUqI(FeU$OzzEzzmvAn|} z|3^_4W1iQRs+o}Gg0ghfrkJ7Pc^nRFI0mzmpprNlS8h#UG*m-F*E|ymyWI&}gFGd< zT60O$M49y5<)WS2I5Lb1UL5lIy!-|aEeY!771JL%z!@?GU9y}K>*!t^&16AUG|U;F zsZ2>Y6~_X$$_Sa)6X*M6fxWCHU^P^#WIB1!#G<9K$8w&De?3V`tB;bh%aKmpawA7H zmA6vCo1G4eEO`c=BFGwBI@mEL?n-zNAdWc0jwiahpb_8}qQUE4aLv7sAhae^I$qvsGl+kv&U85s zU%e(0c-e70(|VnzAvZ$kE|GiVC)aC#u8eF3m8@e|zgS4!d|ZdoIruY{=? z-_VAbodm&VqJP06dU7eQvSj|6QZ~v=Z!dRLt&=RY$7>-}f#CX-X<20Rl=%jT?&5xO z(GY<_`cx|lFg>{em5w@g;dXE%CHEE_P$>(o+bl+^%E#`OiSBEu|9zF8R(98*BE2bJ zMT6pelFd|uZ>0GbP1VuV4+Tj5rgBQ=yTG8=yfDAlX>w6yycQIz&OpE47{*j(Em&+< zC|$2AHGJ2fR8_5 z;G}oZlfqqBqy)sRy$cTh*;B8JdNP&cUCRVg9+RViUp`YglW zE(rUKV@5!{>NKqOxt@=d|6<8pnnglxD7Nc^$>>b55$A$s5)OQ`x@*QIC3uA?1mxV$ zf+kW0Y2DLZyy;c20{aa9pa(gx0<;GzZkS_Q+EjS#2pPcFo6-L=`+Fci$1U`{d*Tf9 zT@zWAhYGSTNNQox?8qVCk|o}(O4__QjH2?B#wZ&EO+uBlMBEOwj~B$JsPVUE8`okC zlEnkL~=annQo!M*Im=22gZSe!jt8 zxEHkt;lLt$VqGj>hvgPsgi`X?REluXb}E@_RNh%VezYwS3*DQp8JE}twFcs~N=XEd z{DE#|O>+vHv>2`S*SjLQr+4XuXjN-oUp1>Misy*Zlc~5w-T0mrWjs}Q+7RXT*U~%E zEBr61Uz@FKUv^%%utYM|0su#`Y(~;*%u_{Lw(XIR&Vpaj`p$FqdJOgZ(w}H?KW=S- zdB!LIOI0M_`MX?3L!CvY-`Nz;>EHmmv8c94JU2C*6`0cFUGH@^^oRBB+InNFH!kxT zgo2ynYo{d-f%VH;cnRF2nU>WPqs0@SA}&nxHek4S=}FU1TiS6~CP3rhT?U~~R!SzI zD&>GpUuxWvUjNT{hrtqaO>LPK;e-dt8z6Y=2^k}%CbhYVPt53MRT`*4mZXk0B5(f! zu2g_;3Mu;42kY~zY6=Sazv||H712Mrj_07i1Q1_V&%YOZ|3R>RKx{? z1sbvP!UIL4@bLhKs$YfHpZX$FS|TcBIYp&ICbC+@aPv(%v>4QumOArB=Brpo?yd6n z?JE=K_5ttEe&Fx$UEKLWwQQ#eNPckwu76>pk2Whvz~W|r-cws!R~ar->w+JzFIeBo zmlcf$d(N~tn19>IcmXn^&ty7t=hGntdO{%WjT!vcV0-{Sj6lz%*Oi}!PnV@kR5x@X z;q_#{(js}-6Qt5T>DOK z-6DcRB;yR(%W4Ln!RHU&`b1S_(W%B&c~094V15Vy&SNQbZ)|sW1|+95KQy)v#0D-?sn%7gx|3j zH*6q3)?Xy-l>UtKt;2PGTk+itr!-OPY;uSoC`|^%T=2)~5G#qSLlaIlh4BoO9<*W+HTN+QV|4tFm-BrJ2<;WP!}ATAK6Hn zb+!U5>{eqlZ4mMQQeE}D=thFmX~j67LH*@ zRF0XM50YIhG>*z(5xgZp#V8`C_U=> zk%m&jV^!~%)ci1pRAl`v0ur6^5ts^Mw8Ot%(uXJBf@1-1fvr0LC7Xp#pStUDY&k zJ~&hNe*}~4bAnO%s}#&p8XHkV&?DrLkf$MonyG~6%6X?`kQc*S46#{&3fnD|smqox z+T5r(gsw&i(G*l-mOE4(1lqLvJQcp4mY2)EKM7bot|v%KksOe`{XH7*^4WU&eB1gQ zAIbiFJ6!wLv`avwmT|K?_Y8={${s}f`zXEX6U~)%5oU(zThV{&Jt<}$JqAMrDzPl zRRIpC1_*74xwu~Q`KI&)^Gn%4Eiy!)vq-kPz?Y*EeMXc1rvM`(|E<$&bapSTGKQrj zgtf>nV;%|5DscKf#T+oZC}tR(!)77EMn&4n!v54*PaDfcYpom1L&C1D;E8B`a$j1& zG4z}IWWdonYwnOAr^YR!d_~`2AWMzbcmSe0P-9kMC;=wkc|GNr5PyF=u_Z2To?&-g z$6h5Em%)tHHudTpK5AuoZa2Yeo?G<8D6lmapX4L~C*4LL83!;|YU}En2+~ll7O~Ng z%w;W?l9pL(xy;5@WwxS@S#&iscx{_%F?miWw~DsnHWH%PX}Df~ew4%8jEr8~+9EQG zLYL6=WK+bMtNX6AuAnm)fql*3esT|wSJopG#>{AL;`va-tT({S2cXm6rMzXu4tWSG zu(M>(j<}OBHv)K?mbRJ^jvM-HMgKC0+sVi2U$kO8ZSTRp{Vw^N4N7rJy=v%sRbGB1 zV(@9}KuHsg0S>;~_w_!YdD+CtfTGTnsk$aUEQs)kAXBF-t}JEqYqZ{p7>h~pgG;h0(onB&pW=_69CkF>SBT-7a0VdGQkby_qLyKt#qoZ946jc2nPfB>ucfkQQe4kKZ%s)z zkha)$;+MxW7oT6;rFyiXxslC!W_b+_AfkTG4IrXhAOXYuruLJ><}WakEF!F#Xdd3+gPW-NmUJ#g*PB(uQ}nRFRS9Z-SB|Mdr zt7pR(el4Qw+Eqi_xM_7n&-SQjP5qJ1Sy&~b67~U}2)l?G8q<9RScx&ioV(`z;KYzn z^J2pouv65TH+$z`rthJO7~@|Diiw{A$7}FxjGBs&B1H(kHH`{xJBq#MZ8MRW%nTU9 z_pQ&CU@KLdMRjnLJXB^?`#JBD!Qu4A?khSy0nzMR?gv7$l=Jn>r>gA8LU2bNMPb#m5pBm2c*TJ)sy zIvj@Gx8wcYVh-)7AJu~d*G2>z2(y63;Hn3Gy#90CWr{lBj)JwX>%r%B?zTV$)tN#u zkw6f_ubNAd(8JxujUNpC{6D;prJ;DmV;En{^WUOOd})o>Ibw`?w1vc&cL zysCRuJD!K3&k1WifowvIo8(vy&R9o_l(v+yF1?{S8aEG%RsQC<8*AkPdk)CQQK~L|-MtK1fVWQ+#w|-)is$c&P68KV`!qBaiJa zw!QE3-6?$vB;#NdYHj+oOem9lwAL5!dbDmlv=g9GX_yVQ@l_Ll)`<)Cf2bFsyGrsV zW_&!Ue%|g2sEF7%%U)vK&N&0xAnW$$+i<9bDWphL4dt%rPp;{I_h~om@(5p{c)l1( zGZCLakazwn_-dU^rm zSXCFVzGGgKq2ubNYJ_Y>|$h(O*$Z4-}9E zejx%{7yu3>L(Y9Ha}B_R>A5Ly{dGX<2yzkp>3AA!Gr$fksIo7)XI z%M6x4~#cfVuhCp79L@y#(^A^P!CEn!=^XJ&PXzD2-*RJj$P}^jPvF7>*HC} z@O6aY9)aH+1*A(pnpGHaaIjgn1Sj!q3P&eh`1g6CnD`zOdy}0_!QIu&6Vvze{r`Vl zZ(!;04tL_2k*07DkuZ)yXAhWjN%WUtXA6I41<{_nHVW(GSi-ZHPCx_zxJd*fCEUNA zop@n+D}~;$63DQpqLT%MIEj6Omu3-Bv&2U+31XIk2$9s%W)V?|j729SAA}BTIKt4- zh>WF!Y$nXDD;L&c;gdNMY$gnpXbsaH9&x@Nv}!!$dl8|y<>`x>+0r?#(gNhQg5C2i zCc$A(6;+5WO{>-C4>HIB(%ReFs+0xu zbeZfI7SDi|@>y%aR)xl+lKr4)n!ap;Hp|o$DcGwkrprC-=jKlL%mnYY`2DWFp;h^| zsKY5Fe9lRps3s?IDfd!xUgVVgs0oB&(lG-Ck|wicEMLp;Qz?M|&sLS>YN-)+u?54= zuCUgwRN{6hj!!b?30O)DY*<393m6C(2r7&e*mzhK!k7R>ni}kUBQT5E(S`zlVr-Zk*V^{^%k!&|j9E0GR#w)oZGZm5TsSsBZU< zTkglGUAGS*W0hM(yr{wfZfQN(gxZtL-Z+n3mMe6s3p_w=Vr);Hu+R0obvSmAD`Jjs9d#KY(%!GKuU(;Oe|NI#y??1ts{zfZc8ZxDh zu3iCs&s#^kAwHHgi8g_b90BUv9=+2xtF!4z@77FTj8 zi`kKha8hB3M_CEKZ>P_u$uYeB*_*d(j~B#}&2`>Dq5ov%rCZM_e*!3Q3z(Txm%c2z z$vnhuN}62#j*+JPgUoxW8U@CMn?F`j9UJI2`mE)&9{xf*AuRpN(N>b>CK3f5rWLV@MWtkruyLyLI3@BBxxarGd>0<0Nu)$@3%Zbxq9NWRfg)gGhT|`q1QIeWfe_8Y3_o&} zFhNyraj+D@3|8sRQK(aVZ1#`PGLR~?9)$aCvVbgcR2RR#Tu95~P@zD|)8qF-X_|A@KaIJPX~!siiZdTUNK~N>eVv5AKBPbU z*fCY0j_CmM3r=OhA2<{b@U=wDX|(+PRtGaK8AT8ugS3KqRJrrau}YLS-2HR)wvYgc zE4gMtS=a7ms(XPSu@w9C3rTjT&Pn@_moxT+fz(O}V`?kDdcCk!J0OD5RfC5IfAnEsEFHH4_-EsDKRHJn z$zuI8F>p;w!$!7P*wy=6-$hTO8#!d@OY#e?urgWRrjWr1Ej}Usk+T1lEdMKI|0DUV zf6@L1L;Q!NVQtK7{HmDtf&|WVLlXdrX`$hPHC;Sp&C)@%Q=tD!D=QGpVetPP_jH)P znfZcWCgTE8_^}C^bs;g~VE#R1kBZs&@A+a|EhG@c{4Zg+OmwagaEPtzJ!|) zHYOA>OB)FVu%_^ZGuAJjbC;p9qv1M;CT*DOBZf9;k(eIK`%#bzluMgB!8 zrlZG38{CH-%s3DlK4u&g3u~-)%tBHX#LRdf6kZMw=*_d^V{67ig-icSvBZ$IH4P+K z8+VXnr8Zk5Pp4sX;AMjk9GR0Sj+=)K1a#HdGY?>nchstF>ad-^`Ao2irHLf#(yen2 zZh*ZjOwzLyV2%f5ap4~&W$nGVjnzER{~kGA(@p*`8q?X;l_>pF56NPqP&7%3&u);| zDqgDtC@bC6U1P7C{BcSGMwdaq;2n?O(m!2uetmu?Zjd-%k_w-T&vdZvmH6Y7k)k$A zWrfsK0vN06SkD#&gfshYv(6$eAGHt^BHjm`=ExAhjY}p4?L@#;Qv9i1A3g%jJB`x| zq4&eGoZ-WKrC|{vowoYX>mu62i#dQs4o=tsz*Hlv>X53W1mN((<(@%ov!Xq+yQ!+Cja+Zd~ONtlT zB*Z+P!Bv+6?zgMl8r0mZ_qA0jKXrD2O}Xn`f-TjGKd&okIAa6K>{;6m{v_Bf`7HBK z0BV?zYji?(x&#%&Cg`fgvj8Ckl1z;zWMm7ZfsjrcTV|5Z(Fa?m)n(S5Vzp3Jp&;@0 zmOK%0nfkWk+!G;)8P_|3zqs((w+Rum<8tC>#9D$NHdHFQxK$1%Xw@$U)t;GW!V&$P&nwL!RgEWQ zAhf2^)opnUk_NIgd&@6p=z*=+5@Mb2XKU*9g#2j_ISn!&|6%0+a?$@XGV~XJsZ=`H z@_kttXkQ$xo`gcd)J&+41ONg5XRkS(1d|>F@gEW{l&LlQD+AzxmQ)Ku0GtlNqykP< z(4qp??m-KMeuu;2k|!o6-ulVF_(Q%1kT(7r<}IE)0u_j6~}rx!u+HgU|1++TKDOz}TuT=89IecUWv7=q02$k1fkQ9N&c&oBuk8(7;HJMyj*Cte1W?e^L zaQUK(-PF+rg)H%(l951MxaA2^O`6^5PWJO1Gp>4a&&B>xY*;x6X@8UNRmDDypo$!) z=F`)*-w7hiCFpatdrNTU@_JkBsLlc|(l#-g8<)M*n>nAXJ6Bg5Dl6wyn+iQ1SBn6R z0b-T&2pa|rHH+E#6+CO;u%Zp3Mk4w4mJ^-)nQ4vc7jnpjc{7n^VRmJOcf6wXKiI&3 z3^23PB`T@XL?u78p6sM6YDBR`ajwnIXT564MZnBQ8tH+>Tp|ufn^Y>Y*{T$EIK`|M zLweisqu3URV%H}2Qz`L#rGkpn=r#gI4)Zvla3*JDB;iLYJFu2rXCX9u;lRDht>bV= zcynYUey@=GFZ*Ivjw{DOO=;Dp@e9F9w5QMikjOOmztJuhv+GpE6M?1}M{l?4`(_-m zX!^)vmI1b=F-HOIxi)Si<7XaKu+GPsG!Ps`otcBDFP}!hWeQ6lwju?P9`8%w ztEcdqDK@Geh>)UNrm7`kf(g3aCQJUT>nRYPKVy)UXS@GGTDdh06Bct_J0R^tdP6g(kZZWho-E}P zJVnuEbjZeCzMV0k`K+^xmjMJkV_Hyj^+SY3_)I=M`ahf&oS?MFhVCSIU_D^ey> zO=vRz8Vmd#6hh;GTqVbz9l~@v5d)ywRA|_xsx0s=b2}_8EXdoO z*<_g^)7w&5TLR~Z$c_V*dfGI)31u`_Ijp$M!i?21T#$8`fdmGW@3;;yeG{g~K%o1Z z@m?}GfIqueAYQV@g?6;CX|?kAu2+E3^*PZG4`;%nfnfHr&@&;VCe4}FBUE= zGfdLh?|AxraEt{cQ>IV=hj^$CUE@jmHDrCijf04P8j<-kJJ@~)N1qEn0Z-_;Pc%nb z52KmLldT{O4aQf{0LGnyq5JEb zmX|#J!KK6vrV|D;qTfgE{wuhc&zBTM`XMs|d7(AnzW<@Gpsbq#aFr3`oMLS(B@t{K zDIKQqB?8P+aa8IW)d94i>?ZrI7$isuSiS!cFSM3YA;>a>u{YaQIw1arSiLpg>%1L7 zr#P+nv%mKO!XlUzS_5~CNCm#mNg-m7DfUpp(sh^6i4I=>9;a-U7%&DZ`)Rib3qDkb z!BztCq|C)QI`+~I$S@bb=fd-Y9?xB|MxREy)|h(7$$N!_t(q?Tw!n^whE#*)VpZYV zirmB3lUmCz0wO%$RyW{`K8ti*-394{metyvZ7bOaYXpHxO z_`Sr9Th6Xu`4lr+Il(d93mmc_ZGYYb_AnojkdqfqUhoO4K#t-uURmp#{0LE>iKSVVvd`d|>k|=6+d_GErv3x`?REXvK+!Marxr^^eh8eFV5|MWGCCZp&A$Hm_4ewi92(h7 zMsDu3>fJb86F>u?B{K%5VF@{@FOL>Yw;^LkOykW65Ia6z#Yb)msZ2Q4vc9+6ih029 z7p8JLH?#*@qypp&OlwGej$3zy!E?mXy+&)=!)<-%-EHwFw^w0arySnu)&3jr_6`Y~ zM+L@AZ`?-Km4@&JyKHE{Vy8u|IuFmZ}m68sw4{c zrTT#2KoU)8GT=4?CMvKh4ha^(@-M(PQ%&9hjRo2FvPe6Pkvvgl)I$ffK-Vaj3S1pY ztTG@GQ>=V5Gs;TO?)=YLHjski{1#mlxBy_AI78Uf=hq zelgfG=7fHqTz4zd35aP#wKv`v<|pDHdcp}{%bMSm9ZU%3wC#$xV0X=16sJCBdb2I1bpNf{IIX4j#utO8Lpu`4VXA+%y$V zlq@JQy=TqJKge!JXax7vo~pvpGG*NAm~1sYeE6WRksrQcInnfh%79>^fK!`t^gxQV z^{zyim9#V{_bo`@7x~hf$co4}ZJNnM1$hbBpw4PNL7dd;0A3z5Yv^*1xDqGi9wMO;Wl2Q2PezAFt@Y@67+cO;GugsL zk%gR4Ew8ay+26xyK-RG3;GOBg4#?z}%HukH*68Y@Uk|}dAT${T3wj9zuRDRn=JBVe0*3^$7Dh-b{SVJ95!IDVUf-d^q0A1D))xnP{ zcq?M34!bH834lDSPST0rOGNgJCc2!-mXR3<|3(E*XXm`cP<+JBrn&8|)9kFLIMr(J zWayQ6)Tq+SPfJObY~{ScLGO|n!SRT^vf_{u?EP!!1h9d}GVc;j!Y)E8YSPF25;ofh z-(1|73fd{Hpw*_qUAPjQ^2Q|E2NFCfoeqEIGroxV9JJH2zRwxRs{s8dCG3K)P@c8% ztvR!tB=$%jtJQ9Q>|=?h%4LsqiiQ0Gb8?`*s%VO6DO_M$f&lfTpFwBZp@C zE^aA)DIhdc0bJfdD#mjvpHScA;FiL}(#LE^jwYK*FnElSi18p~XzDQ%t>u!DW!^42 zchJeMiXjpWPU1BeKh9+NwUZe>E-!fu=a9ai(*BV3rGj1P)MCVDj8BHpY3&C8dp+T> z#u`a0N0-$OtvEO9kvz^T?z9>AXTRfU)`Ml^PXGwPfp|C~*8}UqA5t#!BRyE-NXxR~G|zu7g#(bNLc@H11=aS8_bn$M{SLbmsqx;o1-&ct zKFe9Tt8s=Em}PN}XEb`&N$Cf2W%$k}MT=XJ+UV`>71`fG*tf9;Dx}>NHCm2VjTtT2 zxwm9lMq{wc1aJ_<975c>wo;Fpq~!82xdO1gPC1!$&03ADTA{00IvR4poXMb;4NhX~ z*r|g%;h{LqBdw`yl5BZV5S;4xn+{+3u^J-M!X>9GO`udnPQ$j3^WB$zRe)-0>1&zxsmdbN{jYe3OJNHAT!Gs7)_M7pYiwRF zCjcwb!rRwNtN5rN8bBl8dz14t8>1yIBUt3fE`36E&q!XFy=!4vwJZ9SwYuBY z@tJ4V5IJczpcDZ9dBDQmzDEx}-EG)9awSlH1ygCvXzSCDt4cN-$mbYmM60J;&ecmA zpY+BAH81_8O>2}4n^Jf!o2?!0uA6vo=E!;jCRJp!4`B_7m?ArC3;e0RdIj22%GGj)bX7k)3?6WBra+|zAM_1_q z>B9>qkO`8lfIozNqG2*uduf_|4acS())~oJ2>HS<;zx8b3WfSYqDUvC@cJi-+BNZw z8aN_|66c@e$pv=;2WZGfQFMnJSv%(TMd}9NWXfxm9v3*OzQoeLUP;k>3Uz<=vOIo_ zgd~RqeS*4%r-WXwzE{Jj-kY`QJ%jH=Lpd+G@AAgc&&7TEl)n6glRntO_(K5VEct6) z_<&y`E$LS(6sUk2+%pP@pWH95MG&dxqQ*Pl^6{sM#+MJ6T!5d<%SSfktuO5#59RXQ zoCvieZ_GIXs*NLbEFbEbX3njFWDHUR)lLL;?PT(-#6c7!sR1xlr;K~)fp>DOS(;L1 z7|X4t_?oSdg7%QMuAY}H z)ju>*-o?J9LIa4==D%kr#KORAk{258{gh_)-%<<7LD&trAR5d&3w_wx?9w$VC+at7 z`VF?ve4#Bv8bTAK_ZZnRpD`G7{K8zh75wu&Ph}xUL>F$=>H{=>>pvFX|LWZT);Rs6 zjjKPZ%^1D{9_3$FANRNBHd+FIkbgCHNi_=@=)ZxFsah80e*+&v3@9)I|73v>fc|Tk zWB_?%Pf=q>V^b^FpUQ4#Zf5^liIo~#|0G&II0Z>oKoKAqkYy{E|JK+>Xsw~vh6?BA zGN3B=EHaLP!j0JDa%Rp#J&b(z4o6q>9Y@*~^X6ZDC`TWi+jEk)e-|}>jlJ6BKf3hj za`_61++6_x9|%UU=Aik@$Ai$EIDjr$WWLZnV^C#Va+csL971M3lUST4Z1i9|$Ry(s zW1w-2u<5b|CnLeFL?2QugE+hxW`k8AxB=FD>CY4i}9cr6JL_6n!-0Q~*~~SY3C) z$?1lKn`|ZUpb}PFg+err&c<1P&te#x<*7jwL`9m7T%Tit%t~>wna)~YB~&8rQLdho zf}vTB{Q#-Co?sy@a&AMJgN@*(z|^}Uqwn_7oo+9@;!5VJ=}~cf`FTl$W^J1eM^@~S9_5)N+QbirJgJEiHBY4l zFkHk&i+cs`gu>1+n~*{4D39>0ur^5+SJg#V+8U$jeDgB56?W=nN4DTEXtF8Qqq3o7 zy_C~jO8L?OS(Ui^oT3@!p6R0@?H^D`U;elW=hWaEk(cBp{&Zu9MSwxPtvUBF2M@p* z>8Mn$!GO)Rd^}**Z(OnEMBGr4WSA%BC7~|ir>QG@#kT3k*sa0|Blz^PpZRGc(l2od z7lpAj$)s@FrAq$7d~RLpSBya$*tXd)^?YT`d59>*9i=0;l>KHgy}q%#;n~)xswRC#S4%*vA!MZh6rn(+GEKFnWbmys)cML9x60R(}3% zKcV;T<=k>!h~F7^`0M1SD}+r`hCbHmpj-dK1v#5CA%A4b83Q^LQ$&IsgB;O2#nvSo zfFrE--NEp6zc}jqZz92d$L7BJ~XCs*|AS@YR@)IL?KJk=d=oBRim#o~hEOIMhpLx_YgOa~K^}l;Rce z$)RGj?^~Rs+vf6!FiAd2!_aEbxIFGJkPaN}OJSrafbwexuGxMDF+Y?umQ0e?(uIbMWDw z^DY7?H{eFQm^Qvlq>Zth*W!)`&9*8*UwP15ocq};h-qLAeu^TD$A-`4%y}2ywZ-n? z0-DB^Hz3Rf-Bqe5|LE9-96llFSC?MlN3p74d1fs8j!fFMRGK42twZ4g*j#i|^`8b@ zt;pt%LCX$*lSDAEt_y_ca|G}L0L-M7j&(Qc5k!N)08GS@2Yncs-GpSFYL#5L@ z;2$67zhdwI^l?DHoE)1a+MoXl$L{jwwtdWR-vT6oxr+ZJXEHz_fpG8?Xut~xXehwH zvT;xc#s}Tw0pQ^Ne{pq=!I^~Jn~!Zf6WjL0wlT47Cr@nKwrx9^*!IMDqKT88_ubw4 zZ*BEm)m8nWtNZ?N>OSXO=lUgYTqz)7iCF$X`SEid1awLnu9iGuz?eG=f;k0?t2>$5 zY$LY2nP6pgb+ve9wYU~;)4R=6DU~b`qgLHV+d;Rrb+=ZhcHP_NJLPNdedfxvnGHQD zfcVwd)8%vbIk*4$y?5^J)CoxCH5CYL1V zozXL?NIpt_9iFY6&TyH!t9<{HrZ2*x!Xk+HvOib&j2(lNnHt8ww(8B@Z z1B4P%JB&s6%Zcl7ILGxcy!H8l_D4+BoLn*g}=@Of)I_wH-yADIeV~mhD-X<2M~`d0uvNjUqJib)j)(| zpYM7Sv5{4t5)_5o|aOIH73-*9hGkCd6f1$$#T!-5zTXgo{9l5;p#PmN6KfUk~`z8**0DOoE4)%L~ z91)v-6vr@;QXhzk$k#?mE?RNKG8@&_J=5DRsal&pSLLF#xmxYO2;WWmcJd^~ zxmL1%!4B?KNf_ICi_Mjo@y9Cd5GdYIPzt2RjIU_9&^GDXE!C-KFLt%@6;Y|enCa>c zUn9mc$n&*@Yz!PN$04yipLaHv`FQrB3>&h;QCo}MB_a;>?Lv6uj&f3IfZ6I0+ z3Yqh+ZZu$9{6R>HZSK4o3acx{$o1#ITf{v4)_oU9aUxdQ`zm{l+(doAhgv{SDvXSJ zt)T0_@e% zD70H#ZCou^>&q=hZl1$Q#;@}`O-N*q~oVr-QuS@V~Z!riTwe!(3r zQ2RP+! z@vFsSts%T>Iu5Mk4O3Pm`eW%Rv>Mg_#?tlX@*QAihI0$5U+iMxks?)#?o4%dQi}G= zq(>`nXz@fNM)Os=TiM0+DK4GxF;N66B@eTg#bPKqv9Y~-0dZqyXg;c0MCUPVj_GMC z@)fm*wODJ9pC4tqv8|Ge-IfygusG$KPqmq1K|Ip>(yYZElx+VvbsDLrao2ciOL)(h zdi2t@wtqym1TH+0mle@|)lh^SkI89bYqqlRs2)efWEnk~ANT7fnv9Lj+ALQvvic!- z{iub%Z3cJc00^fw%vL9L`y5u)Wg(TcbLf4mc1ft)wx*eLC9H(fwS<%z#5Gj^9fAJay+jemqOvm_~#7VD-8p=Cgo2UK=fW@QTQTAHV1@)CbEca!D*X7%KSP3Rkaoue18;2+ZwqO08!3~}8cZ9k9MP0K+rFje zjcV<@0NBAkvl2zXub9{IHbGW|bVf-<%TKe9xY#6OU9T^N*5mMkp;tDKUKYTJNi%I4 zH*-||%LjiQ30#zD^?}i1jMM4jC|{*(*VJkiR;MZ)Zp;>cifV&-I% z%MZqgV>O9WP01y#JV+|1CxNR+bvnj-)-Q`3k)PD~3c6QsKcbp0Px`iGTp?EVa#4qC z9C4TdeLI(N$(&U9Da{?d4sBMnNvN!p0Tz<9;6r7mE*wU)D_UT!ix=V5S;AI`1rHlO79XwAoGkiNt+bYKNvYY! za8Av>!PSMnqDy=GA?#!pzkgdRc9V6XqH3%no9(wWO|Na7X=@f)z}~EscX^{|0ia2&RL}xZ2RTR@QKdXNN2xkjh7Sj*D z4U28n<16phss({pj)pOuxwva%wGgBF>EEFP_rg%7&ru~2)RhmY7u$rE_OYJTHGrF? zP~p6aK4hbHdU8o}p~($F+cM>u%R3$KWUbVVX-CrRkCsI-V-^=jpQe>Q11ukVxR%DU zY~CLsVHw(-WS3z$IXkGKqh$%$9OKJnYEP1u`ejQT@F&ULj6#w{h>g&koW+ZMuX0$- z+T0+034_hZ*bl=|3$ll>JOSb=i~xLr)i%8b@rMUi^O7jUa%2uOhP?cvJPP`kF?iCK z54_775n1iRvU4zu}4b@)oqc8F=?dafu+@{Q>H%Io(S1 zv$cQHfx#3OPPhg4$1 ztv~b}9S}oc2>U=5j^NG$*$x|a2LVW?*CXoddSJA3Gjo3%iqm>(IEBB zW@d+M;EH}w)0_qNQ=gfs{Dhb_CIO&~NE$Ro^s&+y9*%<~qe03dio^>ORPS>`QIFrj z>4%t@R>}>^t>E2xPe@~TXFwN3|iFM%v`Jcr6tgwl|RNpqq$GV zPFQjWJLB@7`lxb?00o>EQPb=S{CC%HjjWw>{3n5^& zH>$u2wgkl^VPboE9O8M5|3^! z2I=Ih0Wz@VltpUfY_NW8y=c2})u?~95i&e7tyV52q=;a8mpwOj%U2Y1!Jh~K$n&H81zIe53q|hU$ zM%B!Tk&o=@DsMzoS}~N4)qJYBd+D(knS`5X0fHtmyaa!G;bH;8F(_j3U7E$E^aSr8 zVuCj9h9JHmhpK)Mx|^y}?5=s1MHqVWiw1Q`2dw;!YET6+#z_s)WV>i=kk=UU{fOHhH{b-H$lAtkIZdkAykHXOrD~Wg+17CB5u9= z_3%$ao^LuNVG3uEJQ%!4&O&-2B}pO%09rl~KjubaO67B2Q1xDBubX0}+29_7KpMil zPPg=d{nFEhXFsN z5G&-iIGC2ALf^bu^3<$8;8nQ-UcTxgm0Q@VKc6Ev7)3 z3+gZ3`Ar8LuAjW?FZ7sGZtzELK7jlU4~a2E9L;_|O#TA9<290AV<9p`OpQTb;J`+X z_tsK+m|GUrjELFGg|G_zVdeQlbJ*hHpPscaFO2ksEoE3%36g24)!$erADLnf&2j<5ak>Q&Q9fD5QcdZhcd*WNH%= zL7dG^mV(JnG1UHhuF&LgD0IH;^OI6AFzkaa&AwsSL)xU z`X;Q(+Uhl5iPA*}K9A#L3j&V4?N?rJBHFTttQP|bT4Rq3S1WO_-#`|T)S zFigBZ=x@U@-9vs8pne_i&m1Y`+|Ecyf-9&VII{1vcK;&}`~wC!i*{_s%^yYpSwBVx zK`8QH;UvMo-~eoyRx?4>`8?MXA8?pXKETPF)W7?;E(l+^bl*%I zs2KW|Z-s#x@$(>I%8vEmLi+i^#ai6;T?g#E2is!YTzrQ)p=)wEYk}loD26#*{#q0p za$OWu3ZT`0q4(JV5XZrO*bOW04O_nk^rwOfO@nHS0V8*&3^m26ZIdRV-J%8R_($~Y zqhd#V8c8UxT+rb-NvB!YEFTT32(Qra79kw&A>D+iaN8)Wo2Y{48Il%_^D}%#-5yAt zo~N#>d@ZOv^gY2h9hX-f;d6hU2=wwJyEnGMu<$!tg1vj!pnSbi-Yo|GWU@^^emvC*nNbVN|hLClJ_*aI=x!}|Bpx6zu?hTT90~qw2 zgEHGgy^};YR5`vh0xr#ct9M??XN`PuH_6CzE1YJ^f$2bs!qm?@(sNiVYJ*vo91x~sA&4z|Ui z0~{$FFy0M)i1jdgvS7F@{ThmG-HCe|kx9M2QsJCG*x2Y$Y#hS^<1WSC;pTlXc#)Gh zmgD}4ro5*N&_pEiKUF{0MH5{rh&-=~I`YBNbHn|47D+eEl$#TeN2B`nW^`((u+V08 zDM~m#IE?J(^0uGF^*1QI49F-bimT`V4uCcS^N7lEx)Ws|x7sH--137dW4$n0hRO>P z3-<*}>Qx%4pI8zIM(WBkLcrBUNL7KAOO-KdQ^>f zvd(%z`l`$;?&jFIHMBeaILC8;u6|*ibUpPel0+cDx#C|6Byt`Ls5QQ%TS=^SG0bGg zUX|(m)Zs2ScnDJC!R_HjzLam(SWLCMPlVnxCE_@~y6ZWzj7vE^u9$1u34d?`b@Fi{ zd2)l&p|To@wn>;*fx#aU$@x353g`s4qQ2kb3v^@(Vs$W^O(JM%2&UxRbNZF~lXHzW zel0q>LKD1;k@EE34VJMq*AKi$W~Z>1Z1uq$gudBfDDpLi%*9-5`U9JU@?q2*ml|$G z0K(_Z@b-WNBuWm1SU>8wd;p~j8CiIlT5iB1S=`lF5#)2#r3?Thsl0>*~ z7P86oF@dge@Py1KnSWgp>IVzgA#Ji3Ph?VvAbR^#S0ny8r&c}_2->{0t@Au-|B~B6 z(S~^RfK;niunl3$gb<#~0;!zY>Z~3+|H+z3^h^jZ>tN{4WQf1fkTi`2y{rJu48pD*6S$* zqfQSfA3=t|aJKLkP-fh-vw<(uF`e0{`-tB?6YNz-@6W_l65H#vCLgQpu*vR0Qa$6< z&Fh;)47X(>Amg$Gver9yHR(%1SshdZ`uS<_>uTei<%}P4Z87Te1YiP8+wV(u++m^q z_r$|R`--IrV1O9nGE>$_-q6gZt0`%FNV^V&%rFA#C}oD77?u z*V++R54`%!`bHFUGh7D8%snPi24u@}=?!VMdRj6&$q&e=qcy(xgw?exv*uA>6Y@O& z#2J*2Jk}5Q;9XnrQ}E?qbQP3c0}K(pXd0o;4)q(^87=&8=db@Mo&UcD_Www&<&%_L zHUD)SrC`$o=jlk&_an%W+L^{+0wDgY#O?)rf=h=_zuC&^_YOJ|nl!A`f8%?J+!L|QEUpq-W}T}cz>pI6I&G)MhYJlKFhJo$a?M$NvSjTUEQEXMQu;YwSMd2ST?PK$~HE_ z_*#|5wIZ#8)vdKEHnnYSeQzq&;^#arx4XUgBHICf-zRcAT`XpEcnQ8L3Aa841FvIu@djIynbJ(HlnslRMT-AVvYQ)0sY(&ETM^$M_V`h_mZJdr}v~@%q{DZ`ws*4BMSr4JpTaTg3ebgF#XaNqN z4rpN4jd6T5rCIRD+FW`@J=yaU&N&`sdSWSdhY;fvcygxu=XsHNzyP;}?U(e{!>;+8 zhGR+i#}VfaDE&hS*C+MFJjG!|LVvTvUeJGaH3oD z&N%G+@FoKE=gpyQpNzo+Mf`gs%s#ws?jh4Auro`eU!5ZHnMQwS)HB^({?Y3WA~rag z)#L@zA$?SV_n!>Zy9}$_e?a*!1@0Z*Zz1(${k9=^Aw03m~F2;^eH=IXw~>RAbFZ>0AC{=P=)84d(^(7u3U>xGyQdSa&wnH~xqZti@% zu)teUwO^|Mi6C#q8>o0i?vl=79`HpMsv^>)9@X4l$Oap*s^P?fbqcD;63?=YzFjH4bYdD+n(crV5ibm>yLorY<;O-)w$w&?A)d$KcW-S*#EJ{R+9`W% z^@KgC;zwM2O1AJ3*jmPmnY8@WQv|4KAWQ@Lj=TMBeqI{4Me;SdQ(NK#{`&Bd{Vmwt@8Xo7s$!~K8wzz(2_poY=J^I^^-P&!<=3S&nlA2wRq7=%~a8Z zN7YfrWZ?*hvHT1p_QjHU!>fn0LAqQ!k3+ufECT-swan}YRpQnQj*`altsBraB(ON^ zN10ir2(z5&ngQ*h>CsaxIHaNQf!sV2E52Z<_2AOcknGYDx{28nZ`-4=+HIo314CuO zQiy9ob7GD_wOM69?ly)XB(9awyHg70Wl;2L$6w6kTGz&kht!Cgm%)&IE-Yr@Sq`9E zKyn>7))`?YN^%Yqmx*1F;seZ8Fo{aQQwpC5o49d$c`R(HS+iM8rnrM=igzQZhxB_~w6#AM>T&TD5jAE9hWTPW&Eu>G>Ev=zy6DHVxr zcwu;EWLP^=8!auU>kxV6kZ365vJ+LImx|}sV7R;*hkTl1f-`Q@zIglYqmgnyy>(OsHAlX(;zFAK+p=CxY*;5CddbJ zk76#HMkU>N##~Ck9P76^SV~M*@SF82^$YV*+tCzy9Q4#&0-%hlP2qGLAp~+0YQX4MtDZ zd=bBE4h?{Am~~+`Y9Dl8u>+k^4*S_~eWiz6FG#pofQ1KQ%%p$+0+0-kQuqju`W^vP zzBu@%56w2cZH_)M2x9IHKVyExM-lIb!+oW%xk!!115GfIkKU_5e2Ib!lct~i>MzM zc|sTgoTKaAg6msL``Ty(&_PWdh7ZTyQWz;A)z^`s)e^&RD~uy1G!JMzbwbAjYA7p* z*ouB)ii#JKhE*DOP^*l>u&9J$k5Gs32r@_z-Nd3>mP?&Ht+cms)@WTkOa+|opvky; zUTQ8gTG#9~IJY`vaw%NEwuZJ8(DIWDvt*e8SS2v39hG7zLW-iC4k5%n)milAlt#r^ zK|Q5o)Wst+N`{C#IJM;tbSmMPqARPYhpAw;=ixp4LUr~rt0J9QGv=| zD#MICqM*!8zg6W0aqKpxStc%p?_+IYWeoJ`3wYfYdRWu|_bzFEs2$X5*ZY-WO)lUr> zr1~h;49BB#aqb%jvMhMUbI5M zA)9N&hhH}3qU+!;WlZkWuKE#AC$ic4v!~oUopY_O_7|0|+QN^trLXEJRdF&AZP_X1 ziPy#Dj~a#CwU)gI%C7OynpJEWmN)WShiaaq#Fa(SsijpftL0XIlFJsWH0V z*bt0fTsga|J9n#PdCVE(Qd$k=k`kS0THhjd%<9Y?jqjs5YY5~<OR3hAS#4wh3o|7@>5QM|y9?d;h%k~}mNc*48* zF%L>fuHc2V;C`mA$asou}fMe_fx`3X^|@_^dOA@|9id?BXJwQ z-%*=mh))a*GYlJMmCp1QmD!&M_O+SD>o6MffN-8yP}RV(n7s*emN$e?qDSIPkH1HW z+Oqrv~E-!$pk2)ZfDb8et$`=k;$U@h|YQ{hp#xx8LloVZMSEZIM9Y5I)hIyTzTiiW1&iTHLDP?1w!G^R_ z0!He)&5Xph4lgMgTYikz--vM;OvyKF+7fdYKNlvlK?@Pvoq_fN_N9Iap0yLmk7F>t zTSS|j06qJ|**5IJK8dL+dClbu>!(zrW`j2Z;L=w(8UNV>ipu(_y3m#WT7o9VCMfL4 zE12?&IDB8OK~3v%eWdEDWbInUE0PXiAui`h1W{y}M~xysOIADXZSKJqgewL8vy_f< z+l)w;#@^1p(#FXSSfYG{#UA>MA;yc!){OJ+6E#L2)iwE2;x2X)JAbSE)W2CT`jrKN z9lUf+Mfl{mI=I~7*jU4isc!C~7Wqw_sOLxi6dZ{jZ-jHOR->mDk1q+*dSQYueOIKb zFIeJWt!skkP+3}uui8#wc_$7Rd$e@vTBMe_LzLeoBSK9JxZZ@+l<$bn%ja{$YA@(AoP#xw)u6n#h$qoy zihe(p=K$0x_60H_mNJXhj)hjR$(6E11^vB`v2B^mKPxzyMfrv*c71jJ(wjfu)4|_& zVSLDIo$nC=v~E{#%KRBEN?(U}4WA6ke%o!n;9~He`!fb%^QL2fSHx%8%!VV6dnj!i zaq52(meDt`VxpCs_aqe|2J%mWE}Fp&!py?EkPyVsh+?;To=^V6mN$&=pm=@`3=eUk zzIhSUcz?wAC+d$FcIScRbaw4>y)zjv--XO4xHYD40Q>40oEF;sdGLrvni5wyL(M5$ zd&6?1eYAf;cbLPS$l|~W(SzYDc&^UnAp203;=?#}CA_bM_l;5yqnljrAK?JEj@*Re z{#ZaP>e3+s36E=OTT81pMM!0f74tsz+*>x7N{c{hA`*OQ-NjP>4bIhT*p~_ACBye) z%dZ(TKzEi=>u=k*rx*HRC47-PkcF^2W9mhT#{L#|KCO0%i-ypW{Q29yherdCOd+18 z_imiP3F4mwUYGu2$xd~%mZYEH(W=@Q(cT)?wz;F37A$_@JZKBc8>hfUGQOtxHr=|5 z$yF;C7;S$cR=AoSGx@cJM)0H~l_pUNkjN6zS7h}62K`}}}fTdw1q z5-w1K^77m_kPo!FqH@|x#uq5xY?)(tr3-NkTZ(__JV&pFJJb5#fL?Ccr~o{^{=Y%R z5|w9=7Kzta){@#@jpO=fn}|2or$`l*PF@)Cy+somDcK=Cdi(99l4xPa13K|G2C+8S z01QvPP*J;Q(Sxp>LjmULcIdEMI%oTEX-Q&jjUN1{C;|_Xc;bUbDkE|crkJXpz zc4;oHT+L(ZDR>P&{);})K7`4rE{h34)~Gur;jq%FCfs``H2Zt- zfqDnVm&n*V7wcyG1kPSq{*aI%@dtihsK>#aS|8}G@+ClN09s4zkL3#t)u0hSDk^pX z;w3Y7dI7wmB-forOtvF|`vI&cORqr~t$rM&CnD15Wh08ih><5-+(<4r+;ai+v&y$* z<#+P(Hzom7?t!p#5uDDNhaulTc=&sxBZF^xjtOQ=z~#Qr{*cYtwRZ>TJcR*)N~{nSs1c6YU*B8DK329lwYo=@)O}m`EA_|?y9VRH{OU9MYs1D z^U96-NDA#{$pqRJ)yJI{o|;|l7z5+DVdi>BkoJbgrlsFLyvaC@o9*g6qhnU%cpaS0@6tK;mFMK@X#I2=5aFkZ*BJi|En9jw z&Ujwle@8@Be<%iO<1s+Nw+}B~XJdqqeuj>kyNY6wX^--&7Pa|XusK36kq@>xlJJ-D ziZpj5h5V|r*kX3EQ2ifBY%rj(4vN0`4mJm))4%7^n*OBMsppu&i-w7+# z#gSq~Lpee_64Z9ot^==M>W`Txo_^>h5DD%l@`1KzfOk)Xzt!C7g}Vh*V6Zs~x8=VZ zjZfJoxC9Ef6ev)5m14zT!rdsHqs?7>b$yetWr)f&!f_G6OB@_tF(Y2hi|DVwApqlP zbSBvl^nr~{dJ=0O^>r;#?!x*FEVVrZ45lBjAPXO>NqVh4ox zOgTFh6}#ZoyjVeO6J^eNs~KqPWk(Z>Io?~!X)e7%PWw)gBHBQj{_S&jLuHnd;-GSgbK%Rbs!5= z9CPg(kxFXI?(0$Cg>RMT#g9%>3U3Gkqx{3l!AW~rIZa`1p>1;EPHjlRFiAqu=sxWr zi&_SF8fqeEhxyU;+vbYs-5;oP)G@vbUyc+$CeUywD{Yzfsj*5;WF~a%V@nj;M`@&v z#MQpc%^&y49F|46OwWR;={=YpvZ{B}l+_MSB(24kd?#ClSZ--S0I10ax(c7W_RL8o^_o~XB>T#cHv)w0-4l{M`pQ3=cMyo1dK ziL7w2aRRbHdT0sNS9~xl&WxWY4`8jQLzyGj!eG|!9T2=Q6Q6mcb8ZlUUq3mZJ1qbd zpdCUg$6k3U-*78>Dz6*`1YzQa>1PMhkY6PO)o?v!C~?6D;f7IQfW$H3IoQg#%Wy*k z0k-LvdI)GYw!YXGlV;sII+st-3}YNuo{gLHt6+=c4E)I!@*Em!UM!s6^G|vs2|DjB z!>6?c*1BEE`EyX5aAXk*L5K$+swH6H6D!!je#)`)JpVTXlmobOrMcrbRF9Qof28^T z4>p?7=9%onH(Q^mOZ2H;Y2a2|L_vJ=vw(EOI}JQrvYa_(=Cm9s#w&GfOA|0{a){R2 z1nZt5*{(&Q%%s7>2u9E%lX@@bKdV1KayQHaVLVS1Yp7eb+Wr%?`Zv@&pra;Jd@4to z4VXOc4wSlmF$A#*J-wk~>zd+YD-H7p1%l6r9+{77 z4X$VGVhOw4+vQY#=!)QLWH$GFHZhR*64jcX4QEDLSNu*Zep|2m3M)Hy07vH@zGb`z zX7mMZ^2ID87!cC=LJRoU#7a9)q^9-7@0D^l$ z3Wa!3s{o|_jK8p)92wDLMW|NLV|R8aQJ#oOA43{8J=PGwgHga-Dw}&Ga)n4h1~-GTxl5O|1LmTw1+s-;FSA zG)_Ppbq8mDko5^ba@JT*n&4-Lh{ZwDQWS2PFC25HRpCIr8gt8Gk`*MijnT3|U5sd1 zq8`yl-)H`J0Wx6*p}!N8=E6tONY+bIa(^HVs-Zcb;0XmF$8ty@-kn??Qt19el;6

310X#LRpb~@;Hl)6u1>Rdb;mjpHvdG0$BE5m=?LPCX@i2b(N)^!Rn?qQr+6lf@Q ztj>c_?zkc;r+98vMk4>Df;~M6xUMr+kep>B$Q9g-0;uYXza03^R;z6CWS>zO;Y*Sc zC-)TyEx!E-S_#Fz37GUbD01sQcnCE(3uV&DWtRrY4w|Rvb+n3#&*_#+WehANZ?ZB( z$23h(WtLGB#!Xc>=AA}qmTp~XNSRU{m39o{#@!qRd+FhZtix_?h`#6^HI|B^z*@QG z80}rd06gofKKYamX>a?`*iEvi?ZHeTC=MElbat9!0pdjrc{GLJEL7(ezNu&O=i7q? ziosW$Sw!vNcw>S`vk_di?JyORU!8Gfou^w^q+gN>Hs@KEO7`tpn)bWN< zy^I-BqL!#>^lUals%BO+e{-;{=0qu&>fiqq0<&1 z;nqtzjQnMhsB_q{AIqrNQ>)mH7{yikF&l!`DBQ#;LS?!U;kJ5{R@^|&&=bNShs0D6y{M^G|zwflnz)bq9=D*py@mmw=AGtE6Q7W4%G_>SaL!?+6v>^;MY9ugNm#pMWMvkn($+&4< z%cVD8kpC_%0LkeY4-w!m!8b+xvqD+pp{$qo6-~}V?&NfLzR#`hjnBilLH|GOfhf=q zgi$n#{2E$e4M951^dG-;gTyOejA}9whnx~H$v6p3jwQW|><)rA2>X-oYGFrk5_FPe z(k=;ET9EWL;}|`Ok32ugSK^*$5}VxP6G#ER7h-Pt7bAGA@5>EJo`r1&w}rt9s{i1) zt)sQf{`{rHZ&I%Et;^c#am#1j)WOCJ3SBk}m!rduSfX8>tk#7tOGJ*g%WkV%bU_@j zm|=y(Z)Q{0sAbD~Wa*uydvs>2xYF8U{UBa*Cl+OHn%KQW zCEUB`+cSoub5GHKAhXTNo!hufV6b@4$<;ABkOJ%TC7i8eLR_;>5)7{`D}NVLOw~@v zXLo~dqZ5}M*IRSmb%JDFv??sW?>hrst zsuH?-dxkW(g`w_byDxHWlgkNerVcbXpY}3HTY;(eE-L0Erbs!~=zU526hWdY3Kw>} zBFa$+6skPx_sq0;WKyz$ZJl_n07~sVVK5QX5f7P(4ozPT(GS5tT3B3Hi!oh*Gma<= z#HAw6(L?^}&7IC`K`Cdlcy(vRW@L-bSeC3b;y!an4Pa0p+BknrM%~E+LGE_y zIbD5~&*u)^{_1Av!&79?pqInAmMd#!&V_taMpt!N17R>yR?TePtVvhq>5_!^V9u`Q zo%jRR=+M5kfqG`asSxo&(&rxF`RBC7!3d$RH=V;#m{C4^2Rv1pg!UqGym{MH*wJ$5 z{KQ-D^UD=d6Uo-o&eEUqU1=&T?M43`MaQPRT5lpRv)ney)|$f2xX^dGLZpc*L1x*cuPni!Y(JbPmO40dQ*sY^k0qm4^4z`O`R+GxeB8Bt zr$iALD8m*i{E^j<@VEv*jG-jCfPbNj(hOTWF$7(o^IQU9Dtg-vKP^C3r{N6nhdE4L zXz(6OUhe{}uN|xdWB)*keU1a|`t?(?F%i0;8%59QQ$p|iTOt&0p8V%inlPfFb;!^; zxd;cm(x;h{C@?8{=v-Tb<4~xu3w|HHfYeNh%yC6LHtKzf=_eu}UAXt~3A2twIfwyl z*%EgdN3MffFS4jY2eM+4wL-e%q{z|Br$W5G9&Avl0|XS*X3XptZ5wSP*bDzjAMybG zbRlH%J;~Hl3P_g{<$xDL1XxV8Eq-nuxik~H2j^4RdN zc&-NP*WQ`b@&r8~vpuIz-+7Y5eFo(`NZ$n*q%;#@QtdKW#7g!gxv@AIG`zzwPHG>~ zD5pICdxPgZhW4|RmPyHNSMBbwv+x>22dLT!F8!cp!z0C}Pf2(~yO(l#hTlsUQygwS@U zH`*{~DPCyiV8)BgL~SFn3qVP0DRGr$o~InSb{H1ESN%?D$ z(<%+^XykLwr8S!3bFF8P#5cb+Bh@~r%BCFB1xKVB`>0ENka$a>g;OM$;4x?@b6Cd->YlD z)phLE4bSpUXMJ;axghg$0n53!(FCnqCyRL^eY~ZaWKM%Ka3qu|WS)P#%H{UI>d~A$P<@6nX z%~2QxcVjs-35>_MWLB~UrAv<&bru0YPOI`{eWC!L}9&g5e3CbSZOrRLy2sFt9qeprDH{uf=;T47x_m!1c#q=I9PtqouI{JLcN+ zRS6v%M!E0WtiPU*1|L}^8nmitu*MKC+DyVp8vNm}d2b^TNmA6*?j|nu*qufI9j~?Q zSOkY@=70}7=A0Jcf}9#q3H1{PSxP7J#Qn5$i6?G^ZM9L(rX+xy+5Lt2)sio-adQl* zT>>esL|7GRd`ZJbR63@vaawDts6-;EJb^%<=U6)E=PXU_;Oz1K&2VpK3MOIMDRGGh z8lO+WeiDxtS_<-+6HyXDfk_DfqHw6JQI20&p320eG9`7I-NK{Q!l|>H6pL^v8EmIr z@-~W-s%VmI#!s{|vAD{nl;}4B`m}DpDY+dyI&KL((&j3QL~(HN-7zXuiaLEQOBVW4 z?yN%S$pO`@`-)o{rtlHpMl%j^aPX?3iv`xi{g(BFbD5Djji?3=8v8MTlv4LZt&DoB z*j-2Yjfm`YB$QiSx~|z2AYy+K{pToclh0NW{^DepT8u5snUs?;z)6CjHobXA^;>A_ zT;L^nK{577iEx$Lt1XG8%;8y?&fk<;6!Wo)dfYg&(e;C|%EW4s*P1PRx&_^w`l}=j7@RHnsd&?#8@8Bz4X~;_Sl+(kYm~Vj)CL!SpSo;q%T*+L!pjdc;h^G=M z?$$@YTb&AAz1RIA2sl&ow!HDFpsN9h-x;mY@HNk|$3u-?P^dD1JacSyvb2Vx$wuYD zuUCvy+r?i4G~QCJ8pFNI5m@NmwC)Tc3t`$a6-;>1%CMK5_S@7Q7D%WkSI&4Rg+WQZ zzc&6IurCR5Wk+l3I{xZkp7H{nWBJkB_c;ewZlm1LmAC7L0}SO!^2lJXH_-T0B*(&5 z3Z2hI8hQf6iWi#!n`4Kt;R(x4={dVh;g@pSG}W$JGb1{h4utp=(5r?W_9jbhP^uTl zXgl2!@RogzNNbH$Ok+E%EmpGND=9YZmPt{s`dHK}f4HC%sGI%8_D zx>c6D~2D$Tdtdn^o(LGFtcf;p(ER>X>Fb6L6NQ*Rh&)>(#c@pKw@)Da@qU9P6O> z2P$U2F#FGrSOB#{1+6FC%B~VPMK|To6BJat9{+eN7;lf118vH#QaF>-8If)mI-!rD zM7!iSChw&Y_c28&p~Xl1=Q#5~_ct`&s+<62K|Db@DuuX}EqfOdGLb z6RO%%fP2FA8y}*-l+c79bE+1x8p_D=;A0dKV3m#hp-2j%5uGy8%aZT>RhL26z+1B} zMVn;=GJVC=Q}xSI;ope@bS>BTJA(`vo-x;8G;34`@X9w%tYF@XrJZ@ZMLZ}&>ExK zL4+SH2U|B?uWRrNl$57qw7czUQFa$%GTgai*M&!%KU(EZxAt&r`QtYZVJ z$tmOm$jR{y%7Y~ORg-DF#mA$?$M+7^?nBSU%vv_vUh`GvQN%&8OsC3^tBAa@JGF+n zq&gKlxsaF`B93N$S|{KKjnulczOvGpuAkDz+H2x|Nus4|@J6fBk?&JuqG3y4oNHd; zIsAw>@0I?mf>7_qrY4`M?t1sJ-B&7oY4=J5U`~?IFMGZ!cj2kYp4SlUC;qSh2Bs$4 zEFNG-SN`1Ezpl(zcZn?V7ltu!Y$VQ=o@h?ers0?r-S0-i^;C;JF^)hwuB7H~#gvEB zIbSTE$M?VV={L4JhoS2B!p2H(aeXrO$Y=O{!=<4|i9DOCl}$U=`+N%rXM~fE$qM}z zfGoK7Z~BDX@~d%^J=XRwZTg#vZ1`|g77Tfhk)@4o$sH74f^z4zX=kzS*d?8=7yWekoh4i<^An(lYm*Er_m(l)A$ zlrbK~!Z~29H$9#o-TpojUR+u2?Vz4_0{S|(axT#(f13B1HrD5pM9-vr^Cu$ZVV?8- z6fLhgK>h4N&9GuCF5#~&={DEOJ#Rqc#}-6V{{wM-bJh93xcbKE%(^Y>s@S$|R&1wY z+qUheV%xUuq+;8)Z9Dn$-urd;?f$pN*yrqXe#|wN_L{TPmR!;MFZ;sQBb~wrcOaX( zAsBS)*TsbZ-k6wr`?Cd7sROgp_6)l0(Pfz*ExHnlJ>+N&_#a<_$2@dD-p<4wQ9$Pv zwa3-b+h^Vruv&#HesB20t0AB5B(J;Vetld^`+zHI3jHjuy@!Vs}sw0y$}+Tbq#l>kg6h_KVyeidLg+ucqeS2g+iW=z#B> z9e#>m?K0EIiug0y8>CckhTuUsHQ;-~v9GPJ$OFp;RbGY4#`!1PwgV5WZg*k85-RZ$ z0{OPWxd`aF+=ZhWj>Zd4QUFlZQ4KRnn(#JC5@eP?g%{Qsd_YmnE{dPA+{u{rNwb0~ zf68n_9@g4Sge8QwWXEOBy0u%t>Kg;+P55yypS$wFZsP0Pc(NcCN?^i%% zjy+gVgrWR2gg2f86ngRw>3k56z!WA86w4wjsFrV7jwci~xgXvVw3>5Hd#Ni@`I~>% zkv#e%Scj>dWX>)~6S#S|D_bKw)+q)F`keJnap2 z2J#{B#I+=2_~ARK*rHTYIM%Da`kC_uBPGJ3wP%-T%L04|S@Eq_UjaecEQ9o5#YXMl zsdqqvWwl9>f>}ZG`y=cG@3--bit04}V^lUE&h1h>z^c_@J;D%q0N@PXQC#CIP+;jC z*vQ{E8v9yza!rpJ@Xsyx8Ktm_zm+B zukYnuEmHo?qz&qi>{!X zT-psn;qxD>qsWf%E4FKI{Osmy^BqLY-e9Y2pA5REUp502Q?q)UIFHpw9UCsYd1O59o+f7 zL6>`&HrJxjZ#O@|x;-4;ez5G-?j6!3WeRn7`?tYxJHbg_)Q)ub9gU<+MHT%HsP5en z65iw7lFQQOkTsDoY=xpesu}N|$`D!Ms2KjK4Cb@Qd%JVCkE1f;+&ko<;D)?;H=eeM z3F*I`&vxUArwk~`!UT1Jbwgp7MNK3SIyi1lvl9K{yyLH6G!*hQfMuw|$tg0cm_8a|}op>sk-0b0NP+9J&m^X)BN1 zh)J;!I`}Ya+H7%;8fipJT*|+Fb+`pv|4F1ReZ5`YE6MJM`l1dQ6oEQ-e;Fx;VmJ*eV*36RP$D$dB4ym z*95mzQw{JttLA&1G?$=CxDcu@gawUT3eqfTmjnaGGTu-_0^M@R{{ogYKJZSv zV|#Mf(&P%Ygt}Z>C*CQuhsoe`)f(&Hine`kGq!Zm;LWY%}ZBnsdc*z7lcCZwKGf26XlE$0ZfdB=Dg z2>f}7hH6KS*@=|Q1jTFOnU{#ZFdA;L0?^Si5jxF0;NIzYroFU_{bX_RQd;Kar+3Ki za{!)t7n}Pvcgplxf4^fu?GrR}sCL7vFdI^0=>=L2B`qU6M_pnwdl>}g1z$+#WZ9G~ z0jpKvyikWBX#}ZKe^^RFI9nzfsm(TOJ$FD-TF-%k?Lc$~m!4N~xaXKMTz6+r2cY1} zg>}(%K*VQj54g-CYg@TylQvNOdVaBG;q+FORcAD`WHu6?ijf>%#`@Ki)NxeUqDE&} zYQ_&7Zs#jLNB7PA>KQc>GtsgfQ+LK7&h}k=ySrA$8Z8De4?B3uvgW7+M4=eRBO{qyOi!PWn_RMA?{7!CKZ!xD}&UDxL zq6Vp{yBVZY_t@nhzc!&r@HD%&(0CHn;C9jAc>XROZSa;IYo{-?OE?=NEhYD->Fc&u z{cGW?Pv=Y|=#fd7MxLnOax77+u(pBHBh+~^BbIl}wXS*OTmBZ=Q?hDb4%3q|Dfsr+*lHso8*eCi=vA~+_p)F4<97gHvzqvWMgQkum8 z{>WKO5K$#k8 zSa<(BP95oOp7gG<9pYEcg?zONP?UXUveAcuhw(Dhhb0Y4MaCh3M!)V1jJ^pCg!Ho5 z`K5q1o6-=|g9@$6D3YG{goLx7T3sytjw#(z($~mC*L?Ao)yc#?suOAgs4bDB4UdC@ zXjF6TVCZJ#Hp1X;v4}&%)7K6k+D~vQ7e6T{NfuXCK~0Vld(||$qh)P4a)r73I?I?# z<&6U5e*7@n)NX15l6tC>T$2soEZLJzPIKT{OJpW&HLbSc&1J@$!SFvghd4S3t5&hd zLf2`iR9L#G8|WLDD&mR+V7w~4rUSPtAl8lBu{bwuVV|Y&7;XK*Gsai+@ebLMDw~U; znRE%)!`-{z%={g&$pU>3bxbP*Zy0gU=*534bXu7gP4j0iKGnNsM2hGZ{8fkWmFpEy zxC6mU#33w7P>GKY8BYS4QQjtRivz(DX?mY8H2>#lL{xKt93TV$9D;%onF&nTogy_N zTX(x5V~|f5B8Y+4{mg)U+b{Q}*JF5}-G>ceI*Bn}sKlZvl67>y^OsCsgGw^O5{&QPHV6 z^WJ#ts3*@9ANPhhC^b%##S1Tpd@$?c%8i>L;}p&{qJx1ugE=g?=}td1JYVnFHI-Y) zcEGF6M#ECGJgK!)sT1g|JY~_sec;!Z6NM0VAd)e|c&8;Qv%Qrki@=*P>yiVad+a0z zr)1TXss;GkeoZqEr+nV&!4m++wyx0oZJCvOZHuLX!R}+jt4g_Dsq}PtGYGc@sK}S} z?eQmiXO~^c7{`)ppL^N?EOU}0b^`8YD|2AM!R4!q8gz*Q4K>bb#vV&txO8igE?0_W zxhS#2Db+8IdTDXKX2SX3}Gq)FQv>4FospXtU%bWh6L_g z%p2%oBAvhA+z_1)V-$e(fsSEWLlR0VDFRXYoQMdxEMoiDXs5tqL}%_WgP@#U#t})P z3@k+19qRpw1SGCtk02u>GY6vvks!S2h{t}mIs<2oS?Sg&R!b(G-)+udE*=ZE5(x=N zX2~c|#|m1d_nUM8@!koC^vz4D?<8jG{5Mf3iYCDqk#VWLYSCK%%c5 zl$_rLK^Bu*Xi~9m)LFH#yn6Dq2Hd;9-v8Ekn_Q`H3y9%Q@2)pB=}dL+vh94axf=y! zxx;mXx~T$s$lDQGCd0#P#{A)TJm7=b_6-x!*>>X{?Bc;$o)eNCvmbX&&Xdmgi20g_ zS2Pcbrl?w^6>R9cai6t*UiND{oNUy&Mr+mle!NV~e%n9t5%=VOe3Xa)NA=naufLA* z^VZ3T4CZ#jh!*SqeQ?AW+Nj&J2bFbg!02+YC%yvk5%B4x;M2X`)$&sJ^AWLe8x2Q{ zka#ZUVgKU~iqSWB8?YDWBG#i1It5WOQB*pFov%TZOOAqrXa`O^F$*fO2(q~7+7R$p z^Nh`vjn&yoZ41|G6A1xf0;nF#(4==(s$4Og>_IEXkA`RGvw#DE*~QLPpB*#$tZ1Q( z97zL!4_U4ek+L;UYq`8tof=z$_Y`c1$a9WA58kKIqWkoOQG|y7$hIt9F68{Ugveud zpPH5!flR|TFENU#!CYk-yW`@6RbR0YI{#C{WVE)ETz{ak#+l}b<*ycQb;E*yjXO2W zT#ZU>tF3tPpYqsxpn6Rh#jO~e^O-%uN^%K+guEn8g$P$;3*ivljPV0Dq;F%f2nm#; z15Nx+E_41vabO?RsVnMAi`T?$g*#$84Z^MQPG3LULrE9o0&8K*3>}Skn{u-DQvCTv zAqs3?(q_zgrDdz&TG>l{65>NM24uO+F0>%}H@!>R$`QO3{FzV^A`_*bL_(=C+YA!;)y@QGKXA>3yvA#_zBt-k(p)1T8+QquMW&$L6ycFay+#H&v#V302LW!VbL)Vuj#xzO@s>;zQIDM ziI4$}v5fM$%+C~+yLj9Er>$sZL;ojW&L>`Rl+?C)nxT|}I1UmNx0CY9W296R!c|R@ zETkHyh$GV-NXF$^4!0FcadGbcOM)d(o&(F8^+az_1**Wn*J87^`=gICvM}elnrM zq_3O}Lvpb1RYNFf-cbgNYARloxaQC)KpWL^hKvur{22>~YLOsk`;4m77hS1BFf+_{ zR|J)Q$N04|%y!2Zb#4Fj0z>5nu|nm>nn7eLy*ExdC4>>J#0|8X>NzS5K}Ye1b5ri7 zX#G=rq!xKg@y46O3tF|*O>-9DL$Jc^=TVw@nP7RZLyOs{(Gb8gLc3qM) z+Md7r`?)p@1mXdiaz^$BOxwCweD4Nh!_X%QsbLr99Cp^AaBuvI0epO!wZ4}Qfl7@l zMxdZcaA4}%@#Q5jOm|mRVOF>NIgN5f`h9Rd+QmZ_m zd#&o5SZrih0WBk0Sf{18aYF45f;L5CBwtG&f*J8KbO`bD>kC6V)p!l~!SZkVGh_oa z&FFcv04!Zty2ktnJ8r?uYby_T4Pz36(Y1BB?aUryl2qqm(nyhfb3mijZ)3Z3{lK?W zF6v0N!bqJHF;#O()zl> zuiaL`JFU2(a74$SJ?OjavQo??b`;cF3sKY%3d1)VB%xV{i2Py!XugQY!PIXB&fz+S z9rxC8P5)(_U)WH$bb@d#jtHw5>HH#z{4GP;Hw8+n(24GYaJb+~0!LV-2(wi^r zc(f3m_c5%@fWSTlM1Ql0DQC3wvx9{gOpGKkm|v52rdlV>Cd;tK!Z}i)g`X<5ru{t) zifXhp7L(icP+CHwgx=cK=ubWt@mg+^57L;oZ}7{h+;ZdwFmGdnQ2XY+b4a6c;V(^JpwL;R&sDtK>Ct zl#o4*ZWJo-4Oe0Xo)N4=huy*ntSB{Z6WrEoNw^`&8K5WS|8YQ<-+f7N%j7XNWXCbK zP4wQSxtfK!ifjo8*}qtwhy_OgU&UzvlqIbxwVF9Pvt&Z>Nc>jpsCtfh5 zNKyO?Ulm&hU@TEq#+l*_5e(x^&$}9W*GSG#Q&B#|8d~q880{%S%tOZ;%}%iQ(SB2C)Ek`K2WusBDW zXwk9ihb0O~2Wbr#I7j2M4{t$i4%M`8YXNr+;b~D^k8bNXTqe%y?=S#)x|x3p2X^Df z>D751@!usqN4DM8zahhGe<}DT>W`k9AE4zZxc+`G)K$^ zP(*%7uq1ApcDlCIf3ZbVPl~@&k{|* zX;^#U!sbNw*vtk*CQA zpH7*wtkm&<`0Qin17DCe-ZhGy2gx#E#?wDHWbc`KoKPxwlnsd$n#Zb#_^LMIo)|eL zMW$7B{KZsY6m~Zh9E{xNiXd-5=VTO@#S?bmbkSghE=9OfpK#9cnEjgr-&K7CfM!CX zc`T+Pe0jU+NY;N77-b~b$%(UJ<>C<19QNs*3j?!KA#Y~bHLCWBRekdSxzd}ybL4DA zIyP`-H?kQj*-4-2sC#16>wrnDD=1H;-Q@=xD1<}<9xb2^5eDpDObA2#VIgJE@6 z*5;+p)@5h$0zTm59r_Q~wKhzBN_Z~#9pk+~^sW~+T|a{Fj7&WAd%uzXK+F&TnVagm zmk2stBy>Q;=anRy?#3*3iDw|c*!>ygAQ}E6f!|xIW1odQJsF_Dp3IrKuw&jdEf#Owht<1}(pEMIW@%^`yvVD_>6+ zu%YaXf6iS=@XO>Nzz^1Z10^0%7>Fs)UOIunJeSy1mu9a)ee885Eqd}rGZd~jilG!j@zv!O<)(pyS3A`SuiK9DU9inZD5 zL2I{_0)OS^3^YlsxPunI%^fv{Ym82hx6aq5-Vs^CSBLfuw@-HdwV0p|IS4SpP$FSP z6?N=Vlv1ab9kH59ybZeKJ$4eJbl6L5H51r}rIpODf2i*udtOT29f7Pjvn2C;vy!&7 zgnjA`Wq2t!y_dUQtuzG~7=b#!@!5@q7|5q1>$fONRb*)n4DKk~mp9gv|H#y+>=3{h zp2zZ|TlpKyNmw!uYMsVdE^M2NQ)+z$Uvr#fNwA>FQd?GY16F3tghJrtFaa#kORma; zceosvuB9zCt12Gd1HA$rQyT~k;Xk(PEe*Y8bHj{9Mb$c<7o+Z7rn^}d%E!r z^I4xXDE6yN{2>*0Yai}`B+u?eNqyU7BcF_xO?6P(iqNW4ZF;#7-a$sYF@VOaR14f( z`V4_z{tS|>@Kp@hW%8=pg*H+;9|8$~Q---S!}zW4Rl2K#qi}=TsS?}Ei%Lx6Qor1@ z3vsE)IGWFhw#Qzv-N|waq4Z4wk2#=C6NIt(6HVycDr5tWF^dq?kou0|LDY7xCNs%T z+Sx*&=kP#QCe?lHmvGdom8)n^5?G;_^SpKkc+t6^%_Ik)(wobhvmlVebczf1I00zk zBcdb?r;z@S-?YQ}d;LHjlVpdZs{ArJttOLbWri(jYupCwxl>)xbeIkBdyOBf%^#-u z8!Gly{DwBO1f1Pa>wEQO@Ku5Pt6hyE`qegbsdRrMzsq0uxNA~N&Y`|NhuJwaxhGH? ztonb9BB}@gjTn@0vn5N~VI(d;Cn-c^QA``sMWv%o)_CT&=1w8B``Nx zv}q;HK3HPk5od(+B3NIWvn7)dxl{-Ln!9YCvv(m@PCmI(KO+3;Xu%T-+57Rcb?#@p zUF4fy9B)bic+l_h*3^n08*-my#IY2Q!nI2uGY!4zfLWo8JZfw+@%KN+#(Xf9llvP{ zstelN?#gbK?xSzq2WAG4r%ri!m!kFw34>))!sSvp*HSoniS13aGxY&c8rZ)fM*|G^ zz+5wBJ0txZrhhWffHrh4E|dJ^Bm)&L4%Wq`%bD!&Mq-t6R0A7rsajq3C?=CZs#9G4 zGpBsa3GjBxL}~XNHVTjxox1P9`wJ(yG!is+oYq-4eokC)OR>YH8Cc&p#IdXR(O+&r zR}y*RyEQG!P%S&B?P?Np33fs!WGsfvHf;x)PWR^Al$ZjHC1MD6F&t;=8<{ddHm(nS zU)C$@mZfRN*V@<=tJ2!&671i4B3n^h_lk_G1<-=RZ;nK;=>*$3gzB9@3LtxFgwW4= zM;UgE^;!$KHlPSFevKe|qG*sh1T=A3}OH%4_mMU+os8 zE<<(_)#UE=i!lW!=GxO&u1oA)hbW;M;Zhay#j^h@vTS4XC|N1Hz8-b9bwz(w7%LNQ z4q!q#N9J?kK|wofv4gxr ztW>VTxC*$Q7c=akl%c|S^r!t*G26qb5Nnh<=2_ZOpq8f;l$R#McD};FMNh)-g3)FW z=?9@)1#dE`GUHKS#Sgm@B|apogH;i=Al)tbg{Rlbb^f&vG6^Ns`pED+*9+qW@o+s-IIe)w?y_`wjL z3x=QAjR%=LwZH+|tR;n>hywY~Lw^&o16q@%=}9@0KY!t&0I+RsI`gG)u%{tA{mG0l z?MMC?5vQ0%P;MlmO13pLLwte&{JwyIJG~iv9)Dv_U7!)j+rQ~0ZFs(}J7(Qw%sIa8 z9WB}ZFu9@d1+f`677Pqa_m+#1)I)Md-J_sLQ8=;E=!pQ*7KjyCVLY#(bis5`9$Ubr z>(8=bg&o>y0_a#=$-)%tJ67*=`dx*&2ep`K_$1|p4>3+O#oPRawK?@EuPt6OFLm5% z)+!bvG(@KvLW;)BI`V+MRN3C=6sizbf*C10xK7mb^!KzJ43Zv}TVR3p+_XkjWXB7=p@2Iquv$mT>2tnJrMw%uk=hj0L9WY~7r@ zF(c@$$fl!^SH{&lo(lB^DUYL-jHU}Og{%-F*0g`F5DyC!QEr~!*tk+lQ@x^-X33(Q z1*~K3#hPhatyHvxK$B+nz~^a;&Mf*b*%G5@Hh>lt#yg}y!D0Kys`=}2J$fMk%HN4o zSOqc`LHrOpSns7&Ncc7zVS}8)PhVqiKg0})*eIJgLW?ns89^g1ZYFv#ZjxjNIxk)) zwHLh~W7CiUz~2uZ;?=4fL~{2>3I|t^h%>^;>0UT!Ts{bfZky;v7b;&3D|sXY zQPz&MQ7l-MOZ>m4kM=hXoyONIc)Q@wp|)MTZm*c4l1F=*_pYaI?_>t$>9%=})Xc z9t*23l;dM=nozw>#u|dcRckguc(kY?nEF)#z}w)GYE@?o(+cjQGfpm(M0y$t^aTE74yu z-2|6^^h2hJZ*Q42^mJ#OgND%sVfu^=saSu%BStj6FdZuti%N`n3*Ez{?YnC4Y>`@K z)!V^8DQ8^9Ra7tpLVWzm!p7$Th@3y;pKy*4pSV@98;0ybt~e6n!4h@#!kcMQ@;HP& zLn1+8qycR}rO1QE)n*3}(XKAhRtDO5Qm+cm^8V!NOkpdOm?RDb?G1MmH6Xe;l!X+} z?xAHtzu?Qtw>5tLH=O?)eg6sP>2D010KY2{CsRv6{svgfuBo8-NJ*Wy%Lz)HaM=;F0Ou>rJpy{wf6*=t=ccwBCCBT�w4|z>E0X?b->ThWyvPV& z0F{^4Ksw>}nB;hBf8FH!`h0-@(R@Pyf~kRlJxr;lk`rtXn@$5oRTVsIjI?q&mcuMum=8ql$|lMWr093uE4k3KJQ0Ri&=PzDO*u z9k9>b-rjdABPcwgkP=pPa3(rZf|W)|NMsfppaY0ypk2;VVQhd^tWNhRm7@*wyH`-| zEnu+4g1>*~SSzVH2x}{>{A=7^Es61^D)dQ@^EUs`ZiBTZny484P|7xDoEvTxW1-nh zQ^|ibzmX{zcsedAhvb>oT0ZA=>CLc>;}?Cdk__(7mx{$4ltC;}nJ&FV=^Xu(DYLL? z8w(Ic>>0xsXNWtD{TQBXuzDgebk8jEjFE;-p|feicCnngWabFZmTT(woOw2Q@)+$jY~2Y`Me_=w0of_N8}$|FaHfLb%%@ABj7{KAES zv}UI(2J~Dz4xjTm4Svx**`Ee_x@j+)8Skb|a&1%k8@Ej!WBEH9|-yYBkpZFHxt3-y=5J~0?9`0+$&uoMO} zmKQzR&6Ev@IfwpnN`@5U5Jx!Of%-{%Q1*Z_8;?%iDjolmcL<`h8>O>AcAHvbWNG>u zR^k~Af0qkq$J5IXU+fNpZNk6p#w*AI`!=Z^povVq41LBK7k;cS>1sifIz1H@h6f2kQ$)>b}ei|zZhX5|FKZI zIMmV25_gNsXn|MYu+L4O}G-3^`3_%SPp5#WU8UBFDcnmX&u?O=uIH zT~FSQtpSoML&5M@Z|isOSzFmxS6At#0LM$FAFL%oKLCGkk~w8w84>s)WkBVEhk%!T zWfO02%yEmI_kkz|F!?4&->UcRC!_CfI}!fi7YE!zwmxI3^H4Tt-x^ZGe$_ zVG79KJNY5#COU-9ZJU{qk+4U9>ZZb)eYYp~L$^1%V;;((Y>(RK-TFtly~L0zuv&Qa z$leu@SwCP2_!+E|GWy8)%6;W6DXI%S>Pv2Ph%c<$c5E+u|0Yx6o#|6#{8r%{S~H)wN`H7oNe6ls$WTARuqctSdk)PncqDx7-nR(G8wF}G7P@6S?Rnq z0j(s)ZAL+JAyrU$dLI6hxMB@Q(u#a8F*LqEVM0dOwINN*m4TsXmZ@U5EHw~6&mhWL zI|$gZa%r$P;!c&0NK3UcT2xWZm|&0OP7yaMt+{yD{Z-+7 zUaZB0H-%m2Wh3+$W+!W<$FILYDvWy)D-Q{8$!XS5z#$rCQQf<1ZrMZ@mt-Bpnu1V1 zW|}28lHff3lY)8AtjHaDpYZ(bNw*Vj%&;HvJ=Y}%vt3vq9i<9ZNB)VE*F zbX1&Z&Es_@yOt!_r8vALVF#Ch#9EBStQy^|K zVP5=7Dkig`if={Il9g%g8urnW33s{}>!aE|df&{cSaiHd#*&JN4)6@i-5nqcGj45m zqy-n1`UvitTRMl0Co5E424)_G{$4fI3bfJnF-96So(igwKOtPglPA*kV(d5F^JN5|!89%XQ;Dbm@)B3=C{Ka|IKDHs7gJ27< zWq`SA$K*&`rn=MZ(sQ}l%cubp-9I)b9IUqg+M?!`D-ZV?*7JFSQ|P+(^^J%qt)>r~8%~_K{IXq^0r&;&<1SZAouM0s&l zbAi?v*5O?$C~3gMZO3jEB$D8Z z{LFxF(NV!$V7GBFP6Y*!?%H@b36~6$0;XvUM5`Z}TB8lsi4B!W(@ou`=#yw;(yXDh zL!gO-)vN{Xa2Z=9dh}{0uz_`G2Ulp7<`JOWvkUFNVLwrTF#c1b=KwS zqqB~ZPtlJyCeEL2pse$7Avzb2ESeMNxD|bD4FCL@8dfBw(K7*#ndTS0f zHDztmwp}ofrnz_wFOxDoy(6lk<>#rFuW=q1*b2>ja=K`~p@qaN_g?zqm-!e zlY|{h5G1*qNU`;nWnq_caH@ZUrXln;mJDHsh(TYpMy)m(#B6T-18GRi9tpR(^B4}tU@B`EYgj7z9b>E z`mDB%z8dwLb!yUoYO*C=w+50~lUl7I2$Pr9;!@iotK>pkVj8H%pK5@qQTwilGPwTw zzBK?SbB*tNgV+@cxC8+{kPiyuUp?iI4R!(&!XVQUP=IbBO|WA_T9(CC*~ZwI5qc@N z0}-!kXWbs#q0KI^;l(B9d^XNnWE_nJv(y<#O4WJhG18*PF~$i$K03`F6z6(e%?Fy4 zs3-VQo$cpPD9(T#?n`+uvUA=H|9u_SC#V}B!62ho|Nc%R_+(gGktJBNf_-Z-ucy8R zQU$Y^HQ?Ym^agVJ$|E?pR=x0aSd&MObsP8l;sYYYkgUed1WekqESL~7C}WTT4q*r? z$dsq{l=v^%4K7(;e*Zn(t~+@wcNC%rg~<5aGrO#eK1^}!0BAd=><&Q|=|MvCB}ojx zkozv~@1i#7adS%MF$nY3*wOlZ%mOci~yvq!GbmPZ^DT4W;;CPbr#O8G^^rwmLuq)b($;RA% z^*w&922#a+a66$#+2_EcbN@szM3*N7B71`|kDd2vyzU8l*z{(gA z)Z{YO=IFQ^w&Bf1388`|MzK83fap8p*eGLCY+DNIl~E^-sm^<-SK~CX>m2ZvzTT#P z-?QRoC;Am>6CmjAnqbp*Z&Sz(LKJT$*e;~u{BGYlSlf+tHecx3+hSU<3h)QOSD?D8 zmb58I|3={m&+cfomS1a{&E?yaiK)+h4Y zY(goYqSjmU{)oF%%dR(xZ+Q}~)mcNs%uZ^i?NPS%0yYFh46 z^U6w&pSITJXSvd2j%GHC1!OiFK3t)Ks|_>KG?3j)5rOo2n4KY``rwHSBr-DHv~E#i z&{Om&vt34JpJAc}Zi zt1cVUbdQ*aexcU43qNwjbGaV1=|BAL!qTJtrwOcud-_SNotr;T<>ejZ!#2Tn1(Y8m znS{0x-&@n6urGEJ`>NTgmnl0_F`>!|2vC5kdIQ^KYM?j(J5G(75?=C&eUIwm;*~`& z&$oomeh*`<8%JnyIq?ZWnmjhA;WEVvtxkHz{Hn%k1!!pVk$QNe1i077->){5L<*8Y zDbds_S9B?fz_}%tm~DL?lZucBSG~}ALeqT>7>9l?AB2298;AOwe^I8L$u28pzw7=} zko+eLB-n)g@gocOKa#Gq7^fA3wmAlL?(jX_677IpLETMu0E@5bnwAV2!S^FF8``Oe>k+&9Pn_x$Mf>pIsQ%niMF=|^DgK1 zGThzOc5Mhh-Up}t9=iQLADG>0*RJ=u-c28;mkauWKl@e;UbL4y>$%lk4YHVi`<(3i z^~jIuTCt*sQ6N`rh5XnCn!#IgFCSaENgR_$h zvtz>AVOEvx?a)HW+OViEvlgzl{5uA~OLbe&9#Z zoWhGsUAT;Rp5l!$I;~{h!G3ZpHHc1Qx&;M6HT8i!b+qy=#OS+fLGx-(jKBk(1qzg` zqkKHLPLTU?a(nF-rTu7A3#Ogbck4hSBm{U{+laJ;F7#c`=EY0;n}S#v)W20gJ*!tc`agWiT}$ooP$E>Hz69r1 z1j0_}Yo0f?Muak@V}|)60UXsrQ{{&`lxtWB@5^n>Ep|88^(bme80WW59mpQy{w!kw zMCXf(L@-p_HM|PugRoSdh3eIdQOC*4-F%$v^c97y6?0Na&!Osmy<1sB`2RhwU;gZ! zruKNgpHs@RV&ZEvJ{bLhyd5r}nH%I_S`7{UTn;gtMIDY<^kvrGZtFiW1ilQH@zV3E z=SC(dBU;_OD87eId>kmz=`G>+;9Hjj_>1aWB=Hp{$A4avyNa)IZVNTq##0ik3aUQI z1c*}}nJf>t29cdtX53E`ir=bt<>0OYe0@b1nd&}tt;O|9IEXCkb_8iRD#Qk$v`5eJ z$lN6yJt+kPeR`~D(`eHQsa(f@81gY<=+jP;yBm+MI@k6HH2T`9Z{HseXYf-1)X1@6 z_4x;%g*T}ZRmYn;Yz0z&t2P9HcCh{8*@R4X!G1S?tFf&Q)Lp`;0rg-NR9F4M7bx7bv81a=EG$~+ZznA3D(`|XAF%Xx=H_cy$v59yS-)~A zbxtZTU%`DNw&OGM0N6mN**Oy}?pLdC-rgj>oa8g30J< zVyRY@dV*s5wLX75z6&Z}-Mg$Um`KV}--=8MOl&Lt`D)@a8=DZqHaS|t4Z|4>{iBL_ zIiL+PT-wwFh%U*6%){Id6KzG{Q*|89&0ID1LAD0WH|cis!cY;hmMY*Ku*unQiB_uB ziv>w!;=Szz7$@4W_g0l#w2!k-*$GqaACF`VFp3vJ=8X1Gh6~13&*z)et{Gtl#S)S- z%N(Wk+6Z85BgKd_>vbkdw$Vzk=NZgXH!l{PnP;cov#XOPJ8_zf`k0`K*20!1K{2@5 zM*Y>;se$<*F*%X%<^UH8D3rvauPaF!FP*OA+niib(Rk?VUzH?bX{zfw+^k45l~xWh zo49jk8ck(_;atkOYnf&;<5_al(x@e>;AWP|t@l_z56v^` z9o~gdGnTNBRSpYGP-dh1n~Ao-*38Vx${YyHf(WkgIt0nyQH<{QcKsBvoZpc<&14 zK(Ha^B&rM6sf7tk5D>L7w6kw#38fMWV)AB1!$nmnsOxX%lHAn2trza`M>)gd4^;cC zAm>3AOED+13+`|#fXD!5UeD%SxpMFn|1pc%X0apC_aw9hK&|8{v3Wk&-`RJd=a69N z-0$@@sAg-DVs5tqRWlzcI!j3r0jY~*pm(QcyDAz&MavWfq_+fLbdHN0RVa~6rh&9S zA_4sLJ)>b^Nuw=Mkt>SyQm#UR$$vuRfdyIxZHcWZU;%^aiHW&8k#Qn#GoOsT*BVmW!sUXFOieoq&WQ*~DY@v=PR@ms$kHaD z3}UeAhP8JBq5l3_DE`ARUjz%+F;@hC*Dk>yYUcsAc*M9hMDi9Yh>`Bhs53yRM?aVV zo!eJ>1;F!sNG9`!$R(D3S0Igs2871KcPtf6o;fDgsTF7;MxoT4056M0E2viY#B}AY&g!@601g)g4JX;`8dBq zM&atC4xa}glAr%bup`BD%%wj`g~06&3ZnQc6v(nWX6rA=a10b>o}&j@@H62s*MmtR ziHfFD#}JRF;@P6#V-latZL|JLf1=))IIocCHH~qAY1o(J<>8gowIIA3yO}=5r5TgN z0UX&`ADI_-S=tJ_aQvF7Zz&RN&2Z*!PUldN&xxd&;y#m2yea4;w!F-Wixj;z=*1-N z;Xs91ax7U>-XvPr&8jY%hW@8o>38leG22rj;fmucRFxZfi7mZ_e{Yb z2ep#!P@Cd>Wr!q3$*fy1+Etgf5lypi13+q!ZHE>j&Y~VHnjkH*)yR8}d2yPOdpSBv zcjUG>Y5eYnOjF$uRLg8a5xY}w7|nA{+I_4haE?8G6br|ulo21d{V#_@eZMn+=dm4S&Cr)38}$@@r$Gj}?(18MgZWr;-X z9D$6dihj3fZRQg>@Eyc8)7dOpw}f5#Q;TJH;M{S)1-lylmXT1S7;t2;r@$VDbAl(i#2#aj!QDOL9cbrB3B7P9oHb!X>xJI4U5M||qQ47# zbaC!!0njUBODr-NzY*wu$1t*xvme4uGq9$fu-zkfP5g;A3O-?MCYfe&Kxqs) z-S*NqH9Rt;t1F^a0I3Ek4KOb zO?~R&l@Ylc5z)u&kfz;}Gk1$`o`_UDB+WEC(Wi*Uf_{t+VPPOVN4-00GxuQVXAJ&) zFf7iQ$#{v!knTdsiCSk#%uGn+8;C(#@ujD*P)u50GmEn2mqcl8ba!f|3HbE zk&LZ$5-5>LzK?$YS}yj#tAL<8#ihExR7Sve3|{JVdD)oKMOCa4@BwLYgjdTg23Gt!|6TbqzpHd8i<69I6Fn1VJ=DfX*8(VFp|yxB%ud4OUlCAjGtk|Gi)1%&_dN*2Fi2|^(j?DvNVOcH z0yw2}EiPZGEWX?RwgmI3s^Y&5%pOVTa5T!}cALshf|6BjJy3UI~;S3OF=WFM(eD1h5C{<|N zpeK52jtu+UtJstC;78#0Nf;5#3Xt01XYbjTV6q1 zqbm>2KwQGiO%X7sjRz;nZZAuO=mr(h=p`0^5G3I^t0S-5VJ$0Y0?h5#5TN+7Ly1Hs z6)_Q0nH9>w5^oo7@Ki?yL(C2;OuR#UAIm!hbv02D95oR((vpuA96GR{rlz6E6nveh z8do%tZV5tA{K#STS>_(^!u7$A8o^hLYaNM!o}@-cO$*?wH%GAYehCmUaav+~MO9Ws zN1dTIS@7_-WsDRZpIF9i9%}+CDISw3kKX8ZabxAM&D%ukp{L4!2xy3C7df7cn1I2U zqNL-laSM&EzXTyKq>P)(eHVTR>s`4{1*&bNlwYk&p`_;lE80OBiLd-aX9e>;4c7>e zzHw+dy97AMg%LwHI$7u6Qq@ZG;kKt>%9yE|jrKs;=B&Sl0{OOPM zG$xBxlF}~ZI#}Z^We0E9)GGDmU}zr| zzQ79Y&|}O1$!dlw%|sx+dVZ0AFa3@(DuAI-O_`N<9L^ysPFm!8J4>)8AUc$YtCL9h zXSy#e)%;ZZ79A^l4CPF>y9hRmth|=07GXwC`YuLt$N#KQZ4Y=(Ma#0w#%dZYc zUf|$h3thN=IA1oAJUrG=Vo3p#WDt0RDCt|y8aS7jN9eO@ImZ{)LqXC~)<}m^wL-gc z_Vdc3gBBlt(=yB>@*x$dPH3ML`>3Vz4byq~QUJWs!?FO{`?%0a@^*J#?foiq$%U)34l$&$= zPqz}XI4WKN1F2mn)T}S`XVh+}4!64hnBDy8KlE2h*Q*tzoO`O5uY;`@-+rFt?SO0f z>sNbu&Guk9@Pp9-iS zm%uD`2OjQb|2pu2t_u=E z)dSMS?^XeQWv>DOvjHW<)RNmEjVHtmVH29zP;Y_RCnlSL$q6#b96Zn=7Jy8~I(%gP zYeg4WM+?~YjTrV>1rxNk|BpMKX@9Oh-{v+^OGn}!o;n1;AKM)Rd#`p5R!wqW0r@Wa)-+8e>v z(5SwnhN&#Kv<3H@8Y+8opb!V} z>7Q*OMkT%+pO`gm55N-JB%{*TZ&Q)?#CX8{m;h@*F|Hpm+|&+Y;NjKG0dJ@oUU+*8 z5c?zhyB@j5x>x_0Eoi8lrp6FzW0zRvH8-yr`_cJVYAgFA+NP&25{{|>vg!b)_7Ib1 z6;!UZe^f$dFv=Qj;f2f*em%p21M(}&P6Y(BJ1VTGIJ=)VH~>QSrb(-0PdRe~d!=g7 z)%}`*n43Hqdg9j7ypj-WbqERh^sXSNXpe> z^fT*pVbq+!?;8Y&e)_RL1OfKij z9d-j&@+EY40PS3j>&F^LOQt4r4fSGu=M3tw#rwlRk7M3)_!uEW*J z$F2}?#zCZZ4S z+vb$WW>F4uvbl5*s}Dn*o%JO-jnDMp=Q~(zKc?laozU6>ek52f;Znp~kF^dav=%6j zk>LEA1(cR@#aMyNY*VKj6d>CPVV{mTQif! zw`o)yka?BR*kNt%*{TcF{n`3Ke`C0GP+xOV9;mWIIZ1PxC+E{W&LLCRMhJc)I}$JV5DR?*2PMgeQ7c+C139E3aZAz+2Jc1z&vvYjAu=X=7g*A9pm`4U2!n z0N}y>1}@bS9kl)4mL?YqY54w%2Bp=PeO-OxqEI(UQ)dKj?Sfi0)=0wvj(2hJ>t3C=q0jq6Hw{Z`@t4LSuPQ!z zk0}0?mo#@48DD2Pb3eM;{lcEDYSt6&5#Y)&lU==Ig9u-G-Wf% z=ICa489MHeNR}YD%{lU6@tB|KDug)<)C~=%z{GpN!w(As=Cj`yIjzDuA|? zMstKHk2=+@ufrC-U}}FVjWY|2Ig~k>^G}M_NnR3G`cLE9Q)63Y6Bg3qI+BJ6Lv zXs5>vRQ9Q@Ke4wg&V~olQHu9ckkV(2;hmIwjDY5W(_2;*L`(KF-wd?+@5{qK#BBXs zNg(<{I8WC_p~~H8Y^2IIS-_%YmVn@?tUoxXxaT7$t@%5TO%!teVFGej9laZS;Weny z*YO1dL1oA`$1@#~c@-Z>m$QEj@Etoop>kXW*$L*iY44g~(V8eKan>R^ykoi@wfNlOAqBl`7qybT_IV?jM&#=ZIJ8=r2p$L51n zJB+MD$_~<5T98~mPVSB=*SkHd1A}7E~s}zGM)Ki%grjVbqk5jtFff_ z6jEZ(&uhZStU}1=u*E$z=Q~HSg_v3_mJ4LcMK&{O7SNWUG%K}L7XuQTnwylh?hhYI((%Hwu*tc_o0au{6uO#hW)OnXO_t4XP+#bTXe-w&x)OEbY)M^ zRho2bOCFy!H}cr$L;xS@&h@^0Iwg8CYL__Y*xkuB%K(YhbN0pOkLsn0T?zJ!-h8d6 zQ}fCni0uoYj=^{dH_nH)Y>`*4lzE>RR_9(L6dSj#k?ZZH8~0=`T)jEE(buJ$Kb~Xh zl8D@+j+uh7+^36n*~d@Zr{hmS%mQ*;vxVI=diI_1IgUHxy#Ti!fmsf{Vzcbq1*aLg za}PQ}U1jJUH3ldcP8~X>V?l88SripV96xLAKD0I|9OYJcuuvfTy~J zE^LiYYv*h3_-5^X*-@0bti`V7wcVR-5IbrL>~!mTG{-&x`i5pTNsR*dqlS@-8J20l{p2TiH?ALT3q28dvZom7?j0eM^miim- zYW)_g{rM`xJ5bsiEe+p`mijj>4HwX=JyGr{2UNNMI>6hd}Q9sYR%sRSr1Xwfhu{DYp? zyqX=5LBR1f_GL9%Ta+}qI@99@sSPAS6Br}!#aV4sa7rUl&Z=o5=hWh|p$=^bs)s^f zEm~?OvrOv+gj>c9^j%~!s?DD@=uCG@3YlH-%+37{Qi~6uHs8bOz52RO5o|5TmjT?! zW9pQne_9N3w#@VeCF`9yGwV>jk0{qR#)+BZ2LRPo7mcDZ4g0l!rbzubp8X`-%)D>% zX;SN`ffe7Z?p$|Nu7a%jR=7N%r`v5VkkobUx3CFBTb>4KC2H5{1aBtxs$q>UZ2R5Tw6j< z06_Hv@hjs8)@_l`-LEGE;R6D{`1_6x&kym8drfS&u!NB8v9R`}&p!jj%z`efw!L0i zY5=7fl}F_=5V)GYX1>ckgbj$@yu3|9YLIqApi2TLsMU(kTL>qL-fHlU)&}fH1Sg!p z!st$K1L-X~MKFUcN18TKv6%+D++^_&Eg-bSq8%5GjH=azki_X)vy_c*0FnV;PUZP^ zlR;4B6k+EFe-h834>xGio^!H4h8SIbOdDLUq0kG6-6+lr+vp={8;jAHGhOn~&Jw*QeCi;lBx_3X3;<+WpSp(V1b}}P z90Lw@gd_yLT-vqvyVVa-c>{pnd<(qifbK1i5&-3-_yCFGsQ2pmoc-mkyR=}ao9|of zIrr8?&?Djby}fYkr#va;F0c_a{c=9*4M9OpXw|UfG0VmYu-pK5OV(f=-EJMsy0(wC zeh@3+YAT;mHmfkAmcBbLpOLU`23Y64YU1c(-PjcY2*}wthm>#WJLke}Aoa%9OJFSh zr$nAXW0@EsvfxVlUfzWN*wBgj9rn$gI34)#z^zE?N2tPpYhuwH{3u?7afncdDmB9l zFYtsY0D;;wQg02L=EfLU@7MRpM_8#ib0r^2^5xVZkgxnfWL(+9dLbOa2DmKA%nypf zj&Ql&7mS7H;XoRIVa79O5?*+a6=;v(y2N3a&&wf;`=XcPClDZSUbM&A{1L|-Mc$rA z;C>8!!f}|azR>rEeBH?R$#j+)=Di);S$WSAg$g7;}+rHS*z%xCSxtY-9#m^j+! zv|e;XW}Y#f2OqrG&N74K1+)Z!GGL|`Jlc0xKPqtojccr!j%?neW(NP7yXR1D7Rq>P z)3*-QiO7p#8*2%1MO=qRL{4($!Z%31Ss$oTGNq#6&Iur)Om}Var(e8h;889fHy26H z(W!^UKS69kO?#}a55~KL!q|;fHF4`R+EfoX$PSh8n31Se8q>kwJtA=RUA+Y-ib#C8yneU+7x zmAKv=N1%%m*TET91e_j$z5XfYJHou_&@u>&x(XSs*55$r-3ITaP87IqA<3|nVhSBT z^BMTCm1=|^G3GiPo^ry~Ie|O=Zd20Jb&{5T>J9KEdSr_Ch#A{jxtA|KWa2DeUY4k) z(;VT^5rErsZ}qKj)Q*QsJ6_g8tTAH$HPU5r8+l-!2W+HWbd zXlV?DyryHD?p$ZN4JNtvy|dgjNP7XwJVh8Yq?KLRSjTk}ENp>Xu|TZH;)6@Ii0u=- zhF+`S1#h*G0|;fdVBD=9mTxNCbM>@4X9yf&ro6JMjf34Ib@E$Rk>Zd|VE zeKMC=*&V1~)AA^KL6V!_?z3F`Os(?u!CVV#SNKFtHp|`NUNdi%`Gof_^c?Jc(syZn zvhP%04_oE%}NRGdq7c#^!C`1y2qWO^+^4aAeUb0Gr0$eTM89K9sD>rTOdLhOl8TgjZyhU3 zE@9L&0Iakfmx*^wU0H%6{Re20P`!04`^hlRr0&Twm+4)j!*)@ok=Q#G-qfYu#R(4l z+0}|XY|Ij!xiL=976+J5zWQgn_D*~m1*HSxytEf5O25%+Pl80maqviOSabtdH*9r5 zok#0sAM_%u?ErWH)QB77h>+%o5aU7o9=seRz`#!&|BE9S@E0ZNyYhVmSo?e?V8F|c8>=F}Adit_T;YmW00171Gy=FXbZ9!i? zveac{9Ay+w8z`O8`zievK2NB!Wa9k0+LBH~_hW(Kk2G>(8 zv4ja|c>=ul=!*d%PiD2Ss&SYBT%`MTS^S+>xTMUGKtmErlstpBt^Jcyn>K&)awFWw zxo=4U1jHpUW`1o{A=zMwDFdub5Dx2x@{R6ny7jBp?Y<6^hL6mvbjzYCKZceKAm)U+ zzJZG-GI4^gR3^9@S8i{I;vIc$grlyUO`UKCI7yMDuJzCU$p$~zuZNtTR#*?(?eR;k zzL$}3q19= z5*kJ-!zw#`YThCnOe^&}2+IE{t3nIYh=9ZYS5{pv8RlF;`1gCNU;_woDnmI2Osebz z8$xQ`8V3YmS`%6w_te9e-*tW6h$9HhkeJYUqtXN^A(#YmQoRnuKN`ph%0OeJ(!{Sh zb$j`$iUgM+{7~Ccr<+%EpPK`h1@(1vRh^SxZ)vyg7Tw<)%pXfJ`Q%P-f#vTxEUY z^)EAQRTGVPLvAS|BJC+2_o1L1x^5KeZQb0KVL_L-Xzj2tz`;lZ#R zcfDM{+0Y%{pZ_e$XGz8%_EHpfd>nzt^Hw_wzdZa=IoO@E}}LjuQx2;)aW z0KM1V@Fd+AsIJ2y^;-p`h`R=mw@m*kul+ub;ccejtrkqr^$2zrx>)aFKmUQs{acOt zXO26-TaQN+f2E}VZU+DDujPHy%R;gGTe{Z;TYCtPLHA4hPtxfKLc@LF zFkvTZ=mT%rkQEH)mN}ZYTEYPtd3&0eBlVF1z*bzPtWPFbxk#Q-@LC4$GGDOzJPw`N zeFHnv!jl18lB}7;puQO=3ej@^M1C#o(g=n&gOiLMD(A*s5>ak8q0ubbCf;2}nmMXI zBVOK`q2ip?pk9gD5Hmaj-PhUbgHWcGp(TyM%FLk4A4@`K0&4$+S~y#Tf=q!#;g8`U zV1>=!)@Z;gn97+LIhNE-G$9{U!6t5R4L@8Y%5#9Yr!w_s8pZG%KR~Cdj>RTEZ6ybr zH}9M`z(Pnuiw8I22KpR{u@(J#L_Fxa>p8+*zu4#QZ}L~Q5b$teK5@VMls>?A(Qt4C z3a24iX2f2Rd0&GjG~G}-as_OdH_|%2`$6RA6Z;?>_@#*1Rz=f;13-bqrcKz8?kJ&UpfQkNT+M&3GBqAkB^? zRitBbEmF5R?ZJYIQ&T^;^7tNLON`p%)=V&mKhV&90W%gEXm>JA{JrERU8zJd&ZB856`{G;FG*}4`R{Lf_I z5CE_10|YINBxJ!c!xI`@tjV&o_!?9|nNlln7J;OL!=ko67Wy34FD8bfFs<*G>@=~P zh3snj6_K;z7|yx?!TsEZgg9CV$nuvoDxa^RxwNDz{3rBUWC8oMQq6fkV%mWU<$%At z)wjnULmuarD%}p5vFK$)3rCnJQca??RRkN3QUX7mFAI+l#0#adv>54VG7k98qO?}e ztCn#eNv=(2eY?1#gyl4PJPiwkBS#j@q;8S97O#ueSdIf_ZpDV7RW*kOu)}ZZIS3ca zmkXJ|&>qW*=R58@b&Q~2bgTQ&_O#E{^lA@pI2+{Ca>iaSg$R|G#UD43r-z9~HJyGs z)wxK}760}c3v*HA)aSxH?H!5&+-fuQWI=3o-@!Rca80T9WNi*AR1Vj_4+U&itxY2- zz)+ny4#0z;87#BVBU=RlC@<^fmmG_Ab(I{*L*4~-gLg_6m^5YA9GbI(B7E&9tUVe( zpwT|VpYUkI=iJ9|>z<-Gx5I%j zRyf4V4g0XdeJWa$<=bIqS=MdTnG_zgn*&i2W)%l|#>P3ogv5xkvqTG(2$Z($`j~#R z*`HW~7CLD@kMT6>!E{X&tQf7-yD;>gPF~w4X4`@1o_59CE2hpL#oxBIOyw>X!Wndi zY?yMxG=>N|%tA={lQhN+M9skv?_^pT@Y*b`@y3hh1DJ{Lxu-A2NBAZou4Suj207JiU>n5fh=!Uc*P9E=we=x*J*|=q?1X*s zA;n7&#Z%`-jI>&k*%k~pU&d?ulEmBIffbYd>JIL;`JElqD~oG$v%yAM>>vOq@glJN zN#-_B5vVZBK5@QrXB3H-bWh4#L&hAEBuNuy=jaJwa&A}}w0jFB_<>$qXuS-pA{dg(E*<6M%#yaj7M+*sn~HVT~6v>_Ur)4BYcC!*iR3?1Wh* zYh;o<8{S4neh{e!u4#r%>vg|{ZPwQiR6{ECQGET-xsySC7BHyF!0tBb>#@4Sdk$+V zY+VQ7%#dtwTQZ%teAMOQU8%0ph>bPXy>+@2uiI;wc^TU6fbBVJLWvI=>27%&ILOp0 zl7*!z#GV2suIxAe`h}QrV!{)OZ8mSMDFC52R^4PBCft>ZqtiZDC>9dhp5#!l?yCXO zbM(2Ig%@-VQ=l5I@)MjLy&jAwlj58v^mxoM)}6B|15!weaF#GPyjoK0iOlU1O;WXXwSNR+6?H z{91)){_BiGNKE>pJGZ^Mfnsf8v&%ZSzC6E(bY*^to(oThh^R!D%jfW$b!b~w#*PMn z6gz2mYq@wclk?+0vU74$LU&JiYB)E-C~PGS&zK_o2$+?E^=hY?k$7f9TH@52PH#&8 zQC{YDJD5U4eRNescp>cWXXsqf7nPzeyQ5WeFUOX6iMDT%ArmrK--j@fU^+Ng%vdLx zV&QA73+Kmla7MJ0P#7q!Z76A|gUJS*f#{2cw-r2={n4&|&nCBB6>w;8LwVc+KZvvo z%Y}4zWzMa;-#o8{StE;g6Uu>lT%5MsW0t>zgYalt2nq}#6o>)COCT1clsmv{xxgVJSyzxNX!8b(1R2%ka)c)wQXIb z3mQJCd0Z!|d&1E{(2-}wy+8w`s#EVCD=GR`>%eyXFm{b$ePJu_sa6g<%lcpCBX1AM zzJR9R2)XWvTW;kvCUOl>`A0E*{oM}L-hg~*U6c_ab16lW+=5$TWUVeK48x>s@I*i$ zi(6uZ#1fiWLPS=e1JzVuCB-6M5$@Z7VwNW(Q**@X0|rGnh}OyZTw?(&8{BImH^V+D z{a!_&K*>L+dnP6yvjg10tw^$l>tHGTh{Jpco!8J1^%8g-ByTtgfJE2gqhATGd=FHv zzYe^-&5Mr^bVw z-7}O`Q^MJkm!9~+x@Lgv!OMzCg3wFC;1T++qv^jq4>oc|^R*d$7hJ=YW5dkqi?=Ey z`G&7!Br7YN!_IgT)=F)nZ=%JRI#LP0Q64+U_oxx1^?i2Vg4H~3vix@WIQ*&Xeo#Rk zD8vR}LR=bIN)>$Iz;#hqepHN-m3fPnm`w<` zg;sYQI+038gJ9L8W|V#y<=C}z5M83IZ;UoWa%Cx5O~Di9)X9s_O;Ih2mL2hG7Zy0I zOsUmPnYQ_SmWABqfuiM^Bj)svJ+Uzh*HhnoAHRADioXF1g0qH8 zW58zzdD|hO9M2T)S%b&CU6*9!w>_z1RM#ilulwI!gSwdr68IW~-2E8Q{CF!(5Or^Dm1)!d<7kA{B_ zG3FuzXxiLK42626f7aB~+RLc#3n{fgDy~Y`~_mruxe<<(k zTmDeai=gkdnG@PWm~#iN_?jTI-?l8PrB%k}@8~fGYdvt_Bs&gUaYKx9tfXv2f)a>m z_-Oz(ZB~-0aLoK2+A1ZliFkR855FGa>^}LU1~l~j$xDgL5Q3*9K!TlrNlA=iOE$nk zSf2lCnMYb$)Kq5S!2& zx5?GXA*DDPe3N)7?l>DTsNo-@?PDC`8eeUoN1Xr<8pfVctwtfMd}pn2$%s$KR5UA)1jP)Rh-F{k3AXRE|+4=|n7& zkyKLR<;Jf|+ob&~tFrH%3o@5~|Kygh?hoMmzTR*4-*zMXgOB~!!1}*KF*4V`uGRm; zdi*zDw|4fB%0vGTxTBmH)DZ^iztxS<6sJ^pHzojBPmrBqYn&TX4hZ&ti(CH)AkP1v z4B&rn0>uXk2aNFFC(FyRR}dNyP$5U^d@mGXs{AenDS*z%+Q7-_-|+3Nyxjg5z|P#B zI4SU|yd#N-FT2{1c zN*LHs1aP=ax(@o_v}?$OgtKtmD$CPO&n1~QwkQ!lj5n=dSu7e)H0@wvcIKK4Hru+B z_hp+d0s&^0%2)@XRZbFP`nEmexgc{CSfLsJU3rk)lh zPBxhrqH2<8oozKFgzH z2mHH)v-n!nESy`i*W_tifGk|am5hoArXxr!W{RxJuT9JR$ta5&-fBv8HBlhAc6Ayw zx-H^uxY5M&N{tc$yLMU7&J|&wizcef1bge$ac+dYbz1Z~>jo=i#AF$42DT$H6otwO zd3i~(v1x{PnReOK)V#dFp+D{~sqEtJ1{?$1)pd5!V!uFkrlOQgOtdT&o7GlV)~>HD z^;G)`TiI;4Y?Q#;xNL6VWL~maTko!Htt~CBiROt4$=Xn0hIn6~zg;%9F&DGzi^`B5M<0l)kzX@gfe=D!-0mHO8!njs z5$Ky8$cs!*+g~5aE_w<=hV`ng=8O4ZX2!=0dO$3)yXCPC7 z-&&Kxa@j$!)AL#D9JIc_K?v%~mNBj4-WVkDNPiKav9!IqMtD4kHV{o}2Mo+!sEfGU z&5Jv6n!s|-kC97z9R|pNiP6gai@CiAo_wsuT${FSck=2-d%^pV-E(cix05pNKa zwVXUPi;l&RRPLn!ApNyzklUYk0}^P3CXNcg%N0yK=GR6zU0KE{H2V~+EBF`e4VCvf zW2hI2Oj*837MW>fXUFoW0A7}|PZeDSC>XG)A;W~08T^X5BNk>%RUh+0S~~N?t@eLU;iSUp;<43+r^lbU1Kcl+{QdSUn^Pub za}M7b@(s0Nw5G-BOLd&8n}emwwAxgSm=Q8Mku7bal@$460<**?i|7z0;&Va_GS%4$ zucM&1h&0wat=34LWup}uW5;pvgag4_Ry4<-+6UNJ1_lxbXOKv&KcxT4Ha3z94j#E{ z;^I$)D~*{%_a;h(0d#A^MOS-!SK;hC^zTB-9K`5RFrn+|Qlso_22>b2*eq}Y9lSa_ z2Tj>AVBv}!wSVKZ=ncayk-UiOKZ<-hcc_%Uo;%~HEon<%I&(8mX1!y|Bsg};si6gcJ@E1w>ET6(EWB zWm)XV^Fb^tq|}_r*$DUpJpg1#vjK`yFE209PB{uVC~7&<_znHxy@pPZdhu9R0Seq) zWVZrW{ZEHcKtjZ4qY}($Q*(lonz4};irPZAE+;7mCv@<-8V@uUTM*I8d4kgrCQy2n zeP&Tzo?MyZQ^)`fyqKK0<6aY36TAVSmEU}jnWzUp5s}KEadLOG$}K}@NB#FkL5rzo zBG)$Q{TjKT(@%j&oT##8SE1E}jJ~3tR0CkpZi`HwAG(^D7aKA(cjCVmjzM$vCUu0zS1AZA@Sb`U<&GH z5bKY)7YK1P;sSAlG<6$qO8>%8JV>Q83tUfnWF{DL`QdG9mt^G%{0;`G7WlFN3C9yH z48a!&IEDZCy*=GLBatGTg#+IadPAC&^+!WIja%B-=yzW|f8Vk7lC->pxpX-^47adp z5d&rfy5#WTGbC(xumuzqTrjLsodW{nnedZAAfM3ZTbW14MY>);Q_-jMN0}f2vFZ}e zTe~3QTp$;(@!a53m~3o>JpSQBf8J<)E>PeL;7|ed+daT4Uor6G96B(kC^i>_6Iv)C zsJV@?O$nxYpa|t|KVNiuq zSr?pV2A_nwbHj6rKgKzh&|r{D4V4YINr}q)H3VK>AOxP_-+VI>$Qzw_rq3w-J&d=3vQRJ`L3Pj4~F6G1Sy+w{Cb$+Nd6f3-X8{3qiIQc(K6 zMhKq}zP@Ie458>tmi5%CCa?2%X2C~8|EB=isBfpI0zvxuQ}{24UX0N10Dt~He)x}q zw%Jen?exCZKX-VCUB+{%b-Q`=QjZO5fU&5huMV;kM2ctc;2x^!)is|<_zy08wlkcy z+C_Hw4|Jc+zFp%ZjgJ_?-Q@jStS1;By;3!J&uHJgMMU@yq;A`}SP$CoAYc7GP3VK$ zA^dD#%emPz_&{IT-e0J_M~z}G9__NdOlPZ4nGQr&6Z=jZ{lc*8Uk!N;;*n`hxR65j!=^ z@(j^>-cCq7X4Z;r%1^GYZ12Qb0D+DR(#buIG_V91-%q zU_RohX%raJ&6VNtM|?YGhk^hsD}Kk%yN;87``BV*EN&dC%U^uL$7*W=(55|e1rzM% zUZ^;p+xEA?IVBxIYHbT?!(syOsx5|vP)x@chK|ddykyf8lRFO1%QZ*jCq&)Zivs;0 z=KIzaYt*X{=WqqZCHVN7PwF_&#E0npL9GO)awHe!N8&Kv#d`!_?i~Z5mgs zn?kpf%V3PrMiZkU(rhGi88zIo8m}AJfyU*=vudlvE@1ueAcjoLrx4ZdKR+#tplk79 zhK_<1n!|q~^#02(Tk-NLNtMBq?Rco=$(R+5TyS%1ZGOh<(y|G14-W+d?8U_WERAM%fgpGf(~O z!UIH7D-l_5oh2S`sUanMPB&NagF>I0oAxEf`Zux1)p<%7f(l4_K$^4B!L9^%?~N1o zcEIJg_N)&)dmr(u^VaK_%0B|#*mRcXPJq^OB4WMeUz@gU!0UGYOLy+*t@0(0V`u5g zwaTo8<9zAJt?DO_B8Yt6mA?!K<3{L$^^ELx7ytE1=e>Q#bh})!wJ+E8iP)Wa#&n_*JAbNaBbi&w%gOK; znN#|3RrPqesdLaP9I4=#bY5?y2NzOz{KOlqg~e{B>toW=^ytFEPAvJ=bmg0jQd8rx zjKsfZ2kNfVN|*s>E0qCTrsPi81FbvbfM2wN;wy`kLEgmhgb3a|)esaapu2L^0djkk zUXPY7s`YB!?lo^?{-|ueBbhtmhQe}n`P4?}Q*wuiY;`tplB&}*oFQ3-ch2r52!$9&WKQoB^r_?C8$f$AyY;+R6DE( zV7c)M=-^EY1=psTGhxR7J#6GL)i6$mH*j^eu^kXWO7vwK7*i-as(_d4%5l^Qk0=`- zsTKJ*%2HL3jr!5v;T-5B6=5H5sDaZB=(L*tlW5BXl;R>Za3#{{u9N3u|B0r|)tJWR zgd0M>3(>=5lwr7jNrR>_km6jdGNT<4fWaK~!L+QCeS60c(Qq;KHFcVLfhDEmQN#0% zl$DyBep*pAGcjwZ+~vH_cLY<#1Ws{jit*h0ZJG1PH^uwUNHj}yY9>D)uduSR|I6A& zOrA5W0)r!nzNLxej#tWBhPHg=@tK~8oS#{bU_fyNT#l}hJXNLhL1(3insK}Wz^@k= z=LDzq-u?TWSM2-~WaSn1T|>Bx%mntv`xPjK`4_%sziLAQt;O;$RqncU%8s}-Y)h^P zj~(7HR^c+VTRY=1D>N*a-lt120J`vDa3M%VENF)~Fjrb#<+pmDq&5_}%7Cy+npC8M zpj2I2%KZ%NT1!QJPRgJZwF781U;wWP+igC$E!6VP0KYT$;vPyT=(HS~e!;U#n$|s; zuK#i=LT{1mE1d1{)|ry?j49(!>LSaAq|-S}hTNt(p-rjfMbYw-$h9-;-_DQ#)Dkzv zB-@f~XX@*M&vQ-Bn4RjY57Q(2ZD!EiL!gvlXBCQn!Nuw<3cVqz1o?A&z}8B;*(TR3 zUo3rHwbz*fdM3#jOh-EC6@JU}urqPb(D+<<>OHr2`lfVO#44wsZ!-LsTvb`^{-ymq zNy-vCrdUeCwxqNtN~4SSqtMTE_5KORiM^EFJ}%xaQN~|24%pa2@zGWn?^Btnh5K#x zdV$eb)zjogg6@I%`3T}mKpGUZ2&eYq(4n%hBvW2@#9tvRgMmyZTwv*J{ElV(Xhxu?jqG5e>pJg4T80zIbZeDJsqE!CP>N zTct@+(OS?PQ&&8(9wjV2-xCFz%QVF?L6dHi0G{OYb$e7=ykXKSKxXP%;miWDIi67F zK(z8INOShHlaU$$V^0*bgTiyvk@6@j7C%Lpd5e1 znFLKitP!VAK=do6>7zx{9JvE2d|o1i120~G_Y(|$$?6@Z55ecT3Xd?q@<4Z%3U?#` z;78CP`m)K37umK4KyS3y=PZ~2%Q`cz{tlZKJNVll#Dze%NxIJ(#?&J=mu9j$rB(t{i=Jx0L3(dFM@yzZ{ zYcrsxfwt^7>ow_BIWOt$d@d)`U-0&)R+3ICaR$XW{UX0bz}9UZyqQ4%rx(sGJP!%l z$yU63I}5?U2%NR*#f~ytlvjr_u}rA26QP=uQAkSkdPcrEMT-Ebd0J+kL5)n{Q%G=m z+5Aytw#;60Dr$rl}o3bECG?I6u|$$Ra%gG ziG_fKu+pL|_c!ow^SE#LE7a*_T93r%9u-!f31nM)S9)f3g5+$=f_}e(_B>;lO%_aT zR+lG`E`ENc&%ji1O4i>7cqlxj}WsfULiB0+2Nr(oLxzxK` z>$#fwi*GQ;ai4s#y-?UMB3>Gt8CiNk-x_-LS5M z0ONjO$a1je6yokbUayx^D4q-1F-VjU+_wC>0- z`t+D)(j3U;kg&)2&;(wb44ex?k{%iI=5f;!aTm^!{YBr*e!w+{T}#$=08E}(TgjTQ z&iUy`u^0>&-P&Dc#0v^3c!VaO)d>%g_@4dMFH~UiP0mzsJ;F&}go__NNyE<^wS0qV z(#~KBPYNvp!xY<%HFVzgeHN7>54WSa5KD&dNDL8*S9n+&U2phQTK?mnsoPxM?Uu`g zXk%^GAD8=Vplj0{a_y9SnWc(|nd-N)vG7)QN3L(D`PkU@JpTJEC8U24MB`p? zLv!TWTs#soxm8~m@t~~bwAOqk8yAvp8eyo0L&%0}?+nbnJqbm28fTFxsdF8}vm3c@ z=|vG+?N;gR@)Py#tPlGgkA5j)r93T|3SC@T#eU>i34H#m9Q1$C-2aOkQ_)-fwf<^c zUjNnqwH-w=J@<(M4q%|RA&DVihtS^wbG#sv{2ko!eD(YN+vGfu9k%vuz)zOMK;6nbd3UShzLFTGrpQj~G{Lih*&XaAouO_RX0Uskq zAT+!V% z86#ZtG*nY|u2&twekQ?PwYz$d$QXyXq|hCy)Sc5kXD&cFo`pGk#zCemm3fD3J>8PP zUL!9Z<5P5s1vz~SZCl=CrMdkYOKu5AYYLKa-BfJ;k=28J<&DFzE}=MdLkeNIinV-DBiuRqFe4m>IRY-L&n%4SA@SjQ%%q#dD`6H$ zZS0HoI|hyH3Q!CM*}Zj=o-sy!w#(a~X0lX&N4E4&U7bVoI_xw!XvbNt^V#XAGH1F+-RQOe;2BDR#Jv&=oFWe0EH6k z*y#7IX#n($`l(V@=yY@!RqJBy%rU6S-9p&`Q@vW~4BgwXTM1FAhC&_JT?7L8f3zJ8 z@4!-)CxPh?ae7)KtEU(z6>2!8tzeh04&}kYztP%r(~f(~(t2&1U=r`M<~}T}@QO8+ z4|n%7azP9nP0zWNP;nFTRt!3NY=d50y;|bf90TSW(7hDp5=qp3*-X4QKfAi6;?D!G zeUt7%!sMBZSP-s#dX7*-T${e{i8crzfaW+n#}xE#qHxTC7XKWg9>#aUPHHdd3T(XW zJJvqIwKK@YX3TZyRgw2piGkZX_(H$ac3K@<`t4KnOKG#!!o{;{Tk@-K(EF+H1k+$s z;{a2Yv0&W1G3vb4q}^6e<_1x2p~8J1zV&Tw4CGlp!BAA%9oFlS_HH}Hwy|JY*G(&k zLB5FHq>%C5Kb6(Y2hSczp=g;Q3RTv^Ks(grN@}x+UX2DJ1aga2pg?z6Gh~IJ<)-+h zzCED{%#31u!^w{)_=TZgc*ngqim4_I@(GX@owrl=EaUxlA(0R$%wZswu&2deAi^Yu z7=U2MHXw`m0a`Wl3GQLZy_hkm+Dmpz)vq^@n67Mx17rNKHffJ&bsBA)K-7R8vCNDy zSn^n9l40_|6*CG8iIWKW&LJq4`N9uHoy#mrAIwr>8jNThl`E1ZQBY4pUyo|OiUVlT zj@hQQs-K$P`=0w8SrXBYvwVk|B<=$?(I(1SuGTEl6UA;B@kk$A1k)p``zr^G;Wu#! zR8chfn1(JLM~{dXjXrhiCw>tVvu8s`1p%2m#xCARh6<$(rtYr}f0 zFJXQvQD$;I+>7?th1Q*Zhe-})28AFO`IZU6ijpKOsw$q(l$>sEd~fN^1}| z-OSjg?oinVuR>*9-i-a1Z#m3S8LA}cV!o#JLfMsEa|@-cX7*ui`XWJ&9y7Z=hzR*=Qzy3>Nw;O z{MvQOX5^Ia0)3rw3yI%DO-5{lM&=cn*a_*O+%Mp{8U04S=`;Y&$xmH`f;^TH9PZ`< z!v!}zPZ3ZW_7Vj^G)leW@HZId zx>cA9kRM)nNmKsZV9opGWzHXc19$FM=%dKKgFQbrcVh7lB80IRPyVhg329ijN45W_ zaQ31!_%+ko5N-WkpPecC7`y$2B9LNlMYLDTd~u%(Es(BAaq@4j@}&lTAk!W)+Nbnr zZH>W+gY_+E)rQW&*9<^{v2>&QTybqZNbHQSo4$PEs7i)9HA@%n%*CbQV+*^S8n{KQ zBjMb>{{U98edHL26WCpOKS<={LPxTJemRR%wOwn6IG5)+Wv#LSzIjlF{JJPyY5Owd zSIEXUe}#;0d;THPS|dUFwzXBUjXI0a8gJr%eJxFXxLw*D9s=mu(k))uDrx4*HgB}N zT#DhfMDnWa%7oaoa(_d5YK1|r}Wh+N)$(eE0LMWSk%En=7o z!l$fl%rnY_^jRLt%8_c{SV|lOCFPq(FAcaKM;^sqsG2`WDNPz`uH<;E8nKWdpO`z} zRJ9}zo9Be{<01i5>v?5Zaps%~%*!^ka)?@3@u-|N&=*`ZRgmmP;uqFL`iODBt@Tfd zyAe}Eu++a-@6UGt0g{p0@IqA70UDF5xoO2Tjc%br4Jkg{6+HHLgTP#4eDP+Sbt2KV zHhL=QEtd_L%vd=u@L$)+D}Vex*6%H<9@WWNLf9KxZ{!h*nhf#xFP^bQqC|l`ffjXm6`Bm(+Nfd~ zQLP2dk*!~VF7-aLYd_<12x=*>HL#Iy?2vt5MhCEW*bMJB<{>F&0A)o$C*L62E=m%1 z33s%|Z6ZVIVB4g~ffs?G#G^5y4kOAE50i%7K$*?DM6~ggJyKC?RHhM@A~Nfz2lqk? zrxu`Xm4+WL4k#)T0!^2LtzR}vz$+#~mPMXbIh)BI4-$DycUYK-hDoO``V)@xj@wYn z8Z?!rDPa>Wm98lQORh=0pwYTqbi%q_)M^1PjJ8a*Ak{riyb?oK!49LM@MmcM-iEoy zBJnOa7K>xq<= zi&|X?AGUHBA8>SZCR1r#nhCpj`&pnG>A2NW)MY>98HQIL>DA!Cm$R`W6`YeXpf9S3 z8LG@1$B@7bh}qc2;nvHXn!FF*a}m*I&g)Keo~9v;ZL@U^P=p}6!KIPCyZxopq&%!EX!u>7TU%GcycVYw3Kk?S z!0#B{p=9iD=|I4l3_t%PNG6K}v<`mTv2(_F-Y^iTs}$XuXtxq0!gBmkLwot~ruLi{ zW5WrvSpiiDrn5fZXax4)F9HxKe0{3p5$C4+&>TFWEz-e2*ecbCTLw6?{0O~r zXkMC%%|mSVhi`Vb(Ko`1fm7$7AF8%izwquv<+b%K_Eb_w4DB}Xot@>cpvGpC#O{|> zz}*8S=-Y*!&Y!J0`lG9iK4Jsj6WYPW^#RyBpd(>?K4k~^Tz)G)Jdf2frsuPF zuaZg{{5vibaIJpYrmr4% zLlR$Q9>5@Bu3j{SbgQNxOK@rStusRg-NL?k+c8ajKh{Od)gfl>!bQ zN4~oU6zotUj{OU){vd~}E;lh+;E5kYxPn+1ajb*o_RfPt7OdoQ8JsyD(EuViCt z>5DQ>=$FBpbW5n5s7ffcOKPoMi{A>$<{p+c2BK1zY`b8L;5` zq_MoxTaP3C;qKa5&l`4)L4eSIvK3AbGRl4&TeOW=H%!RgQ(-MbmEuadaHO?4vgT3R zc6snQgG9T0oP{@y0e~idjRZ-Bg)s8ki1!)>GADe&-H z_78M9S!@E(sVEXUC^?}9M_MSxU4x&7ohATlUrGv*0v zQ+^9es)g!_2_JpQDj{sW18jXeMZhSHf$}o)MZWM_&hp@lf{m!F+W-Ij;e9c|N~&d+EVvTxtTZ(sYd>NiE{{^u^hf>UW+DXh=A@WxeapP$_w|(a#wzsK?an8 zX6E<<;9c`iZ9pd7Xs`uQYx!+*4g%MTW!;F>cbwsg!V^~*D1S2&6LkqWl3 z7Gt2Q1!4+J#~Yy((G#Xye54ATooUI(SG8f-4MhY@lVB@M4Jr}Dtrgcdf z?qgA14i5{tVQFJJYq!r-V3rxkC^nJz96pK`Lx@hBfP_?9jWezvLK{ee`zf7i8ogJ= z8MuBhQ|^i&r;wa}N0O9o{RcINa3HEBq>X!vrT?;&MX72Ngtp~((R@8C&~2Zdr7Fq< z;DKN_m{cNPgT^T9X7%Lkn1pn`Ym$`QwA43{(f_F%R7FNecE zLUw2`6`t!74Q}bFws8NpbYB-@1K?XE0QFRnU;25Ugf}Mqj2daKkz0Od57iepexA;o zo(}p$N6nesPLOCuH!7$Kjz4kfd+DTl2kamE8ARgJ*i;VhCHZ`UG%q?{abme{JUlhD ziKn7uww2)|yV;?X+@o4yuIGs=2^vomE{wEJXwabk`KVblQnz`Eg;W|sk;EE#Kv)JB zeDxzILg-q!-x-BC%2CtSL{(>g+hya*tV<2g8rpA})6~A~yd|LKrwsQF@ao+R3a|eg zqE&+PFFxv&+e9xH$c1-5_&-j2L0)x>#TPicqWbl$#V?w=ORhmUrMOpo@-2#(Ep^9s?BoPO{4T z{;;rXFBpt+#`KnsESC7fgi`HZFYnm%#~Wx54g{g;RLAL=Wco3L7xBgeWYnIE*)w5< zm#VsJ9uG~fB{j^D`jLPPvOyMq%+I*Ozo7p_3I7QN{(&*D{yR!Auu#nC0s#Rb2TiR9 zCH(5-jg1Qsw=i=uw{R76ayeLn2S{&h3oMcw+pKgO1kYn#i6p2&xq-KHTD6Le>e(Qm? z^$LMEE=|sp*QG264SBD7a@YZ{q9%`28o{-60B3vxN?Ix=uv;ivHA~-(Axnxv_6B)B zo9&tSnT&`^$ujg;IE#qg83CW@dI~Iq0!dE;A)S&o|59R{i1nV}6$r$V2$wT-xAQ@r z%&oGS^nuVnoA946{ckMipPadPf|vfSFJfQd7qO2Y1f=!44wfDKUu?ed^u<1_ujIQ+ zu=KA&UnJ?btDH0dNgI223s-X1fA34Rs(=!h2uc9_C`sW9PxKdJEP(BT2!g~qwP^1~t{Suh$Oj;E%x*W#b!SGYj@v?s`7EuiU; zWRWx>8>H5){pb_6>02RKsYJ3NqC6bRSWY`J3~5)a0G9Pc-s4W?$|H9fldCmx5OAf# z%HI2x)TBuG9)!|j%Yx%ItPVd!&+03vI+wS{pk0glIw|;4*qh8r2{WV(6F1~n&#|=@ zdVV^a)siM$d>O*7B`>h>Yyjq8&8o19ufAZ-eTxbRkVv29BneJXA+uz8u@=h!eBaX2 zYfK|kz>;?$ip>HK)du1^q&x{7^V|k%Mtru6QD$71v)`dkP~(_KAEWBK*-QDKx8l_o z7y_QJObk>uJtmYH8Qu{)PLhd?<7Vdd@BC!Uc|)6R-ai89)5rNg$~F~g<%%xbSS2MU z-2`L9$aM$N-sD-A5%dD9pbrKArO5xtgY4e|`M*LG{t!M${t}`;3X6gzfi>N@1V(0yPw4F&@0-agE*kzmOUQiTfG`FNo-_t zU%tdb`Ghn5g!Q*?)7GPc4=onso<8Fb=_s?^Rn%M=S#2t)hPa+nCzG3pm@G8aJ(z$TkBF|ztBa`JM7o6W9@*lsL$kfgf`E{h!dKa3SxlY<+JcQSod$&Yy&qOB5$P6nM`K&+PG<-jZ2w z@+QQE0Oq=IbrWyovuAtRJ$ZHw2JItHM8Y!G$zM7x2%KpJ$R|P8#pdGds-iSR$h)HS zY?IyhKbyQ1T=mS$CD@DhVr{^%U1$t>&}S}$gfC?Gz*Pk@*}cx1uGG;$6Mt&FH7wWa<- z{bQT{TY3KvKe?TXm=E+{eP~e@u7mr(w3`(Be*hhvFDxfoPtOYXFCh$%47UR`!g}eEZ(C=dAU_1-rv-X;#t^OHiXZQSPWX0?1bUyJyIUUue2%IcPe3mF@hLO**hKmx zU;`nv0qEvaD$6kqO9`lE2?}Kr86$0|BPcAQLAJrLbN&P)p|Hsr<0Eg(CW8##Lil;R zcc9>o942IfqX%M?_QT;i52!@M9J^*!e)x8Rj)PRUGU$#YrSfgyUJb_!lJmY=g4e?W zP@5BZ&n%nClAH0+I4@SKw<4VUW4_4s$0m|bfJP|({zKxUI~;r=$6!RITkm;sy@05Z z1%jHZubS|=(;+o^1?R6MoeO8@=eV|49$aqJ_CmBo*bXErzc=`8$*JG6xvSiO77~~i z%}NLo6iBc5!LG_%GX;yjD??)+z6IiA(vbUTjGf5vWr{h9^9~oV(!~+H2tKUK$OMoYaNfv-AjW|1K-^dQ9N1u|SQt z^5&m7aN~a+V?1pAEC!$}9yup_{K>V5f?=RTgHfQ_eaVtDq;m!fx>`Yxlk8HJOTdJf zV8t~YlBdikVk9bs;156X>H5|e2VjD8-j@T@)xMeFo-)_<>7q8qGcpOObzSWyn6R`F zj{z?J+}^)S`i^7iorB68-H2ctH_4EXZggX;9e1C7<8W)}xE)@}e2$jq|Xdvw;#SkhYaJ0?d2VmIeb=OVz z&7=GyaKdXIe#V0HcR=_su(X1R3HT zMW}EUbiXKL6C`E$#FU~FN z_9Jn~OtNb-PPALScg4P&I{<6E5X=kN!ywPQOy3Wk8>Xs1batP>UNse^!|uaI`WHd% zR8{!KHZ($FWGQQw$T!~bkF~6tjzE)Z80+ZpL!lZ1+C&C~qaP{mc25-HQT7#B;ZvEF znBjJ5K%(#fyVpl$fqcswvFkg+4;{ zyGZY#ks|`xojs%uC~l0>c8Mk9QA@3^U&iN9=@;qJa$!%d;S=-nD?Nn>=@`fH`gC?r z`!8+e$)I%DQ65OdKL@;^y4=UgCm}jxxVhxv0UZLC%ZPP(y{3+POjGND@AgmwtZI%x@(ECFm03=0tD9N zO9;=zOnQF=@Yu_3;-zTNkVcc>qkq%M=3|+w>F%4%UVXXu#~7O$t{fEliE+C+TAo3rbV5(;&Zy66HT{gi0E zLB7o{DwhL+!WUa*B8}S$ixYFcp4sdR}`LfHt9XQj-4*kyqD_gZ9>gI#jz6=3q z55&7**OMPmHdNsOKTy^4K08~$NFS2Ws*}LqEY=PO=f=_M{|J|))a~SkUg%@LFD02Y zHjo;Etg6i!@*U5`;e1S=*~g1>tcGO4;5?t~@Ttlk7!K-JV58<03ZDXJTQ#K|Fwkq` z0Pf?|*iEvXD1=ZS1k7D9}*|&WMvNnTvLeUlKI;?53$>wa`3s`t)tto*>R5!?*-S|Ctbf3|VtH2&`FNqVD@ za*N}SzH-O?jzz#9ciWb^{+ASx{}lh8-E?|Ogg{Z~fNWz%5X!R(HEvp#Co*-W-geg9 zys)kq+NK}ei*)m|D%R{$(HQ5X&hERb*R$&PbSw}YbPyh#WnHG8{2>l#A##7B<9|o2 z|M7MH+t2?e!74wk;e`tU1f-Y<1VkDnouVB@G(F#p2L>>u4ehP>b*tsXo=4tG9_PSA z4O$q1Akjj~rx0wm@STCKC0NSVXA#NamA-eCw86Z%G)A=}e-a6$Lg6<;z4tHXEo$rl0%HMe7c%F=A!PEf4%) zZ|WbU!vnyBf7?1JkZBdR1BmSR$C+X0r4W2&PtO0PA@qVUBf9o>4+_?(dC!jEy@PP4 zkBjgHt7mGiXF#TRA{_t08QrmE59@P_%A}Vtq-`l~!}+@#k*hHa@WB~{jNr`^8fnkZ zb2EtVstw-j;q3X(BecISRe0cNxF7dbJhyv>*fZdMZ$$r9>192@zikKjWBm?YG7oO7Z-xV}KVKI7K>9VXAs)RYyJ7M~Q^72qL zWM@FAb|DwaGBzHB2xdr8eFr1KrsfJ(jRmzex87Pg#6FyfWD^D|%J8N!YvOWB=yJ^< z8|?s^$Mbi*rNunsA87HFk%r9@3dksNSrjH2N*7}W1zTBJ(j(R(K^`oV_{wS<$>pDt z6y0L^3@FzA>o{`Hbh=IP^gIk{5jOa^jQ0Rb4-zvl2Zp^0m4Vk3aV32P$iguq4!qNa zrl3B?X1?5*YvtyxAf#qK(mqd)p)f5%JJpqxu5+6*tyCh?TDV$UD`se$w*M#ll|vnFfB;6|!?%*>NS<1#i`h@Dln6El$-x~>38>01h+ZADeEKhw>)Lo1+} zJpq2?EmUMZvpLCRAlnl~%#e3MAVJo4%$im4F3xgZx54Ncbf5yaGy2zB?d5ig>gy82 zc%HZFnN9xVEOizAr1KabX`z7ue@S|enETpfs*ZFiX$925SkpS*g;p|dH49e!1HEZB zE~8&*87+ZUe!_QeNwp0qU$v%{v_e1(jeJ>W%%FqH%vhOCtZky#`Q%zk!CTjX38G2lZo`Yh2g zd|;>wk=NGhP=f^}_)`5h1fl3}xNQU-&G@94B1()`bRr`ufKOX_O1Ltch z$;Pxpx^0( znH&}cd#VXB6>N7T4XA@lL+U$C+zWz;yFzvb!W;1=AFdh0zT%%ekAHW?{q0L74qkJu z?(Z+1#glO71Jz=TsIF;Jw5I@O(u{tUvZK~gOvKa213%20zL6HCB0KoMuziy5)v|s$ zMH4U3pE8(srZnYagZlzHU-wck*q>9%56j(?b@o#oQOZvx=N_N4!29OU=F(-&H?Asp z&9{(?3cGRJ5X=kP@d=A}=vwKQ+8C`nyj&IBufjjL7!J5MB8@Rn?6QEEF?%GoxhZ=X z_lxw>&Z>Sw%>Jzao-@Y^-qdTi89I*>xKKVX>pu;Xlau{*mbCbGOINa#H40^dXpQ6Y zrks&$7RSautY;kQ@59F`Hi`AgQ|WWD)R?%LM~#qZbO|~nbV(U%mP1w2mhfw^tZpvW zsKi(gd*Yr+lZwZ)Xu*Iw_}bCQDS1}C@fH56h?*og-Z@r2NNj2Q%$SEY*UV~Tx*hSH z7??262T>A3V!HmwTv`5Qw&5TiuXaBT2E8EKtrWb;iC6;)He$L&?efWYdyXL=INxq- zR1Ww$R$!{yu4Hl~ghaIa{^`-?Z4!Ym`?J2mRZW~NE`{NlVIIKn3hC_bhzL$Yu~%r=BMlKcVWD+@W)Gk*keQE)3C&}2d8*jwd&`V zbG^Aw(%DQpJi!%2Iz`LgQW#YJLMH~C?>3%SGymvOz0I>&sIuIXZb_kUZ`xT^s^NQ> zK8`A)dN%Z0#sf-{py{`7X+JyaJ?U&bK6G6pz3iqc#@3lCS()=GPnQm!i8=%5!we(; z3>8wi1u@+#j<0j{=o}VEpDZ>vOSEW61)Gc)OLRcP@MTGi6}Zy4cX?zh{w5-3v7>KI z1pn0WC|`UYoNu=ED&b1my{RX1QB#$&Iopd~e3ReL1Og(VeUdPVj%^yd1XfJe&h_+K z?df(j1(VaPo7#0*2yKfho+#QCJPI^{YUYX`6F#aJ=+enD5g%xxx2bK>HMGY~${pkkNcchDflUgB|IsMUekV_g#*#s+bpu^P~deKi_KWe;@6E&x^j4M~~UIq{@ z3M}{L-%3k6UOqNv;|Q3(!@nv>i>NDLo=0^=Tn2<^pCcTOvVZVG)0@VI^5w(KFNL47u$-BgsURt*DRJS58dA zXR?5#E&~G-7lrKZ{YFnsu5n6=3JSH9Sf9f`buS*ty@IyGnfO}s2eeZ$fyzUwhasSK zrl>c3Fs=o#Q%x;lyFA*$f6Hk|+P**Q>~B;6HU_!koj2x04*YC9iKZxCv0i0mKNK4$ z_M5U4Br@0#9s+;WB}+mF1o4`pN@9yV>7W7ZhjjLze~We@-(cHfn&Jz7bq6X)z@r3p zk(dN4Tn5JieG;Z1Yop!p(5NBTxP0`8tJqJ;X&Fmhiz9u6wU{19!yV^Fw8#OnDvIHj z7Q`$l50}#okX+Nr2yg0f78uvo?0PKT9uZa=*Ke^SOo&4=kwP+|!a0_vwknArJ&^z? zbA%aMYR(aR_TrotHNg?=0cy*tHXJt z-uexX4;8vcGyoNE5;9Wv%WQR*#r_tD(;ESwABgzx*U)^{4Dn?Lr!%?(d8E8OUW;51 zQeuIhB8MDlM4_ARFm8O|EHgrY&_9sy);OXfKi4e2sH7-SNc}oCckT3{-XKFE+T6kD zI5i@mL6D5SH=G#p#)-J$b!~1aR7m^eMq?<7M8iq4l@q*RTw*`PwdIh_YUDt~Zr$68 zMRHgpEX1@0BQQg>X+Ua?{54YvLY0ImAX3p5MbrR4r!4qlOw%1;DxXaOL{_M~$w$F@ zIwI{L)o-KX$uJVwkl=0U1YX4L`+>X<$i_TGgLniYm_+@Q{PHz^Df+*0RK3CcZpo)d zRP@72T^RCyA$q5Q{ULB1dcgSw&mDsuc}~&q>ob|jQxU!pJd%jAH#tBity*`IFi6HN z73#+0NQFCvk-*UgYqQ=ANZ8DsO(>`JK-PAge|_iQg^qEfbQ@@lGPZ9n2a|LREI!6em769?ib^y`Lhc zy;C0Ey{iuAO+3fzfz~dJ^;hKS?0?f5KyQz9pOL9L9}H(H#?e?5#4|f`Y)X|KF1O~dKLzJ3KsOku84A57irNvY^Nd4v z2O|iKPkxw8p*%SNfWa9MU89n)6lC$fDD8@_cM^eCNMoy>dPdU?+~d)b>(iT}BshYVxrOMk<|6 zE+-56D+z{nDDIo!OMZwGU9<3;=RzO!ik|XMeQGx!0fj+rQA)%$BD|5%w7hADvF>)jRhp zd!_`%57R*|RDMGL1Izw*tnt5L+0p%tD8Vl*_5&9LMDFYQ%*H9*<_-)mU3iz92{4uM zriQtX@g>*n-NAAno(l9%mtZ@5GiH>aa-XF^L<1RkT{2G%o1r3`tR6Hf zmt^)cy6cBhMZhW3lRV+RXGwC4kUJ$nxb&|?X>V(0o?Eki=r;(nk`Svdzw0G|6aQ%* zubYr~p+5&dK>5Rr{_Op#0=Dap0svHaV2g=}eS->V$A1zJM!mWrk97QzooalFaj*WWpj05k=}H19vv=N&}G(EaNo~j8!Gw%7frOyTsFAK zrd*c!$f|=wbUDci>(UcMD;^e$;^AWLA_vFW)-A=_WQ!dw&oiGU zV?NLIX~dM&(T^av5)0lF$ZdPM?L};lmaL36&ee}TBFb3Nxu)vthRhttDT?8oV4Eoq zB_}((??m9hJd&sDUth(re#>Nile5#Ls-b{9=PZP^j<&D@o4L+X1ANn_|9S5y$-R@^ zS5EKCXvOrzzy}n3c)bj zrk-363p45FR_d`4m0j5+;EaavNG=I#eU?iEw#nAsAmd5;m;v%fn>N?6>EB9)S!q5T zkbAYqWsTP0#l-w90Wz#h7I2<{i!H~?oN++{EDSVnzxx&qF@?bhzY(@uVA9f;j1^lP zqIDm&cJ|JeT;WVihs^M0N_j1(60oI9XGur_Rgyi^AU!X2@;%cWVEG&6U|eOgdxXU} z5D5`EuV?B0G#k8nFRi*^?#wJLDv({d|DgH^ai^4>LB8Ba;B)8L zo6Jk?a(7~8>oA*0`Cx@0qzPi{d{u{FB*-s8%v+u7YU_TrQQ` z*dK62|M1Pa?-ISpwxF-}g7@!YiX&^XxEbrNR&e|nNug&?fAP_Zy-R?8T9l$DclpTp6;`t0wwkM{m{3*dq**3v*12cH!m#K#6hU~WE z`vkf3D}N;yKpvfDdt^hQMeaLsiP{dvDl9cP0vIhgf%+1kpI-9fJ`LLyK^N*SVn0bK zH%6^@juZTigd8OWX3rnm>r8Qs&D*$A7C%DrlSvWx>+s9D)IAbN?53)6O&4g&8V&mN zn@1ij6uSqJ|43m>I!vzCSiDxvQ z{T)F6g+Hssr)?V`AM|gEGq?pLB(|or$$tjyIFVQV1g)`n09gcAiGxZf0ClvN&bAf% z8@-F;c=Lr*x&o_T3!)|wzwaQQl*?42O=E?tI^_g&HRVh&0?KUvp-=hgrwWmzU!D;6%B zB^nDinUzfxP!~mi!rblUsZ{KN*+FTEqk!=&rGhu|lJLt}k}S#kEwAz+Pd#g5*ewyW z6(f|M5aW{ob~yR1qp{v7%HPM;((4?Hku2gv+@_{zQbXzzOS&m1So%X{kDI|vb1Qp^ z^Eks}&k)O;)YqBCc{G5e@+=mA9bWXdI2+tUc=DY-txfUw*#2D*wNRV_cug&{M?q7f zcfzY9B3~jj>zKgrQlFux9^1rcfDhXJXP4FJ@|s~wt-V#KFw62;{{T#>aqt`2Z%D7q zO0;)iKVP7J9!cb2^7=~Pb!1da`^ZA(xq`1md`L_xBd9f0<^2WhRsm~UI391qzIX8C zs})b4;~n{8O0B@_ZBOvg{&t;u%4c14&XmpcM9g)OlX>X5AZyBb2Z?vIqStnak+eMY z29KZcp*W;Ds*k?=Hyrxk?#BOiM2cm6!3$AAKtf4DK$!mJh%mK|PotcoLO6(hRkwn@ zM!5t<`e#}{Y2`fhwEzM%9Tf+HFrE1t1v1?@7X0q(!hchdhs4-$_lrGolg=#0K%A~@ z&V!a79|sPT&VURH5BRBWr?jq&6BwtiTj%|YuFTEN5qwKLcN%^&=zC&Boie-0n=qVa z^{<9S(}qi`tBFB#M!l97A0gW{zc}-`N&3vR!+`soXP?@uy_4EnA&^Ouqo7b8raUvl z;fxq;!$*_EYzoK^Lsv$Zp@v|P5ll#tI2FhN0y8*C?jCjKae#})Fdb0Df$gCqXYwQ} z%)cdPQyK$p%(Nq)rO7Vt8VA94xB-DrXCXd5)(m*`RaH|Pc+eG3aJuX`RY$h;TB8z6 z!1q+-g&M0@t+LkmL`G)u1#6P9O3-MKrQRL?LuvWa^V<`y?~gVx#Gu zu|BhUFH4gl0dV{?H<4(lEzS5^$G$0{Y$>W$q_%olPXrXM;Q6;`Q~Rg$t0qy4X)QO! z=pU4su1a*P7NP54jNY)=hc$%1MiP~uEeldKD%c6(^@zy;S`3%$_&OZ!WvgA2BxJAO zDlw+;*ccXcEO6QilI6X_3v z$y$|MmlgGn_s8H|k)=L7n9na(LM5bep)nl5Aw5kEmZ0LOuR2MOX3`Shv%hf-Y{odV zMUeofFWly<%a+fvn%A)5jMhYYT&>j!@hpD#@|kM4y;2SeL~lPP>fey#{V3qulmZ~Q z^7n;<0ARow5L%qu!pIm1j#dc|ql=|si8e>TnPbrLsSe{?VSnQ~U^@qp4&NqJ>Amb1 z|L7}93nX&bU%W`C?9KlGLtAMzSP-JDG_1IVAlWOePCml5+$$tjVCLa)MS4CH>bH=; zTfS4b&J+2?zBVB#IREYa&n=?r7*pqTLC!C-2!Iw^u^39T0m_eWxsD)s=5shGLJ?+-zbURK|iSv z?4JgwGNz`x0@xn5EuSAQKEQ?HoB8F{U771j2YSN9;H40b#Ve>SjTuAs+z`1K&A%PU zRdDD(yG#315x@gYjiB}^Gvh;jcgcQaxC%YC2_4FwKLD6)2Gwv?o(Z+;V=Qnsqp9y? zRy;$5%aAg~gYh~7aCw6w4y>%kL+Y7Q4ie*~TzZsAqpTNgdPgNH>C!{YSz$E%8z_CJ z6Np12@5PmDZ^m{1uEj95#}dRj*!MNm!%m2%7JLAlfJvHO{zUvdtwou)5#yYXAgWENjzv`8cBv83((w@X|ZiKl1y}e*d@p{t*-~o=qYa z1q1|_5(I<;1SB2v5)&t#84M0JO@|k;^=yx|6b0g+b*;6eI78t765c}x=Y`#uO`L^H z-y;P7vY23S2>+^?(7>jI+ zv`Hk)i)g@D@zQC?-5Rk4KR851Vz7O~Ne)lC)3n3$Nv+ckD8}8lIJdUyu?9*nv-2J^zgoju zWC;I`qUZTe-m(b6_O<PnqmcVlPvCdTy$J}ytY~gJ%k6VQu}?lnt4Noc_kpA>8t%IVZBpt8IiLV z3;mA(qak?AQWe%oMAJ6`GwM$ODS}!D(_-8BD)F9nHr|vSZ&SVy39jsb(@jyF?l<~b zWJ)Am81V%g0k!YkvCRWEvybsn^Vz@XeWntdqG3!28#VVJCUpfw26g#F7GHx`sph2; zD0mgNjc=TTzDyv146vUb6j$tGGxWrmpip#;vSLVck#QS``VmzQvyA|PzCIbpJRq0@ zrtl=9)&{(Q@CiA@x%=~o9^AVYvUB=tq`Dtj9ss<%kyl_)6Y9thpg!*t9RZ-W@^(-h zI+rJ76n%Q+{mUx-M-l%^tC2xIzSI>#6{BwTE6I-^C%wLm?*G%#zm#xT04;cbwI$rY zKeC&++S?t(NQ^-#Wal11hoUs#S#5NSaq?&3uspo8H(Rjhi#ywc!0@N16iO8d#}h{U zcZ;&)d2pH_)!oH1ZbpvnvZtSO7S8agYEL|RZvZ{dUiUA(UIGyL zIql~91;$V{>bGS2Lt0uBfVSb5&rOFF>e2rAKV!{s70$Wn9Btb&8$U+1FZzby)HnOu z9S_4VFy1jP{+v@%y|3n@JV`=k*JUCLO~BFTsba=KrH{3(1?gd6IBzSor6RY|tOjp5 zpuv6-%UQ1v-#Rhe_fL@C`-3_O+roDOnZkSHk$VpZKKx%W|JJ1z0qhS3Y7JK72pQcA z^hA(7?;8#LSF3kH`A4;odFsMvklli%^j{DICtCe1_{WFVsmywV8(nf^c3dt;%=sA) zU~!dL%bF7-9A5`?=}W#|4JBwe*`z5N1-mRPl4-i?;W8Impl&s2rZw9pDr<911kWW9 z71vj9Et;{A;nBbYsf6r2XVNi{=?9-|`!NUd?$A>S|yaa{P?3 z=TVE+S+X%7JcemG|7B6{jX#i5pl{SXKVP{&^+}FR!kyg~j;Gm@3{{o(YfGvo9L%?P z$kt+1X)KL*QjYfK2K$>|7Ai~C83L!#oG4W14@atw@2GPpfZ>fS_eK#rxCNQOZ+MIS z@Ei-fZ-)h2wj8En>ZAj};?S+u3?m-7jJcAk1O z96xqlAi=%1^BO_50Sx1T%{T#&EzVcWxV%N5E?Q5r%|3O`aM>v!@XjS90E6TuCdO*4 zfuw*&EQHAd@Z%dUbzyj<$Ydkh@x2sT+O~;;!W_p?cJ^3|%uMM78nbCy0!gKU9ZRrz zLiH-Rb?fZVK}ImA;pi~k`j?Oep}+}(A? zjWsX*&X1Oa^;0-3MC2?z4QxiosY#rj&JoS@;_sFd`1oMJ|0C-g!z*2yc4uPSwrx*r z+qNclvVsXF_KGvHZQB#uw(ZHuy!)JQzt?wut-iW@J-53CwPlMhy>VO5p|P*jj2y2oKWH8xN?`36c(UhGSllZ0hziSy^IX>N3ehG-Rny zwCL}Mpc0+{)Y>DpXeI{e!V40uop-k#^yY|GQ{Ef*@2ZNQNdM&LV$@NKsJ>xWm2x3| zMX%asq|qP3-NQrwz5|c`ecLXiE`&&rrqOBE|h z!0VwgVDKD#S@MLtVUc1fDoT7~UMOAQE-miA-67`yWtqPeKh2rgF+`>W2ME>8N8c%c-JUQW?Nr1XXO+$<-!?-4546MPGl(0xZe zxH$6@0qF}BR&Z+%`M=o;)3)Uax@7%7zB@IQ9 zEQ_2+b=yn&UVnOD&>_s&I`m+oEOWk4$%3H4`1i8;_X_n2x5g6ubee<1My99pf;)Hq zESLRg4t}nfd&To13=`J|>p9!|`ay+I3Kc*&CqAsyH^UOMw$_`c$9_CuH2AXB4Et1} z3cmZock>|01BKVad0(xZ<^ao|m6k`L#$x>yls|U8PT7QUDL6HKN&@2!~xd60%ABLgtKY)E#mbw&@9WG#aB`)Vn^OL#abRc4ZUpT zenD`k=jplR@b!XNW`-BnB9fS+L#ms(;@h}LHr*mwMJB!yu0L5De_JXH>PH{B0!r#L zE_svBaPL@@bZoV$5|vb60v&54ld3BRs1l=PijJzRg-MG58-d`5i|li%yP_|_x?jtjBe-!RyGAT+f32hOSiIFQEx=oq{o>6Q2Y(y# zM$&>ZBCePNL&w+#N#hnaK{0}`1q3c=t%0tl*e|gPkvI_12#QlcB@p>EU}2)bg|<;a zyL0PV<$0<)jx50suSS&*zMa_*XI9tQb8$7(!~4Jo=+ks}KsWkbDK%qlqAACqEJ|t$ zpF+oa#Ubi6M#>a#T$cbvC?1UCbD)2w#iN{)7$WHDugOW<2i1aB#=j3Q1K=B&zZK!O zc!$fh#x{Xo?99B!YEkaZ5WuYlyd_ps#`gJ_84hVVY+LVbA@2war2PiLj{hX&DB{Q85;Q6GhAXc%6*W{S>rME0l$d_2qjyJ+;F(2cAW86T{ zz!aBlO@9Ewga%p$>{?;x0_w0n7|#QT&X)tvJ*2I|k7*JPe?YGCPRfx|D^m$e6ROvUe4>&O@#|T?DK3pObiZ7jyS0$LkUi!o z_r8^0%1l>BK5P-=iD=1GAqlS$ghVf#eN#%^+J=j>M}{7kiIUKSKruKscnd)Tq=h`) z@cPd>GCC5ggk{CpHbenGDe|6}vfsOyXnU?ED_PQ<9|y8Nv;c?8<9@2j;Ltv|-UeI? zPB#-C{6n=B=rcS7^Ub;>C*L|Y1MR>PV}0>KO_GM|RCSxrX+#iWwEO7k@?r>@kc0~= z1nZ*vwxl#Q(0;asET407T6Cx*?DHaBbq2Xb}u~69Cf#A+vlqyvrVE#;7&vNxNS;;1QX>JydIXq>T5FmLqsC3-gUK^a?Ee zcrQnfsiDd;b>tgR-?ehX&N9;2;&$WDvf$hTZ~V$wJMIyDzopiQSyl7yk=YT+xxDHT z;VXD6;Sq}8+qU=TF<-ZxF;mnTSs;;Om-N`hhfZ#Yat6@yq?#5rng9GGtsMqxS~5uv zZx9>Y885ryzGY*@jv#F_MXM*|dO<@WWs^!*Vd7@RKNfFqzC2@h2O7SZIm8T-bmoGk zDXm^%VNw%`BgVmC{tPu6eggfDvLiPT0A-R}!8PKf5H^?_`%WN+TcK-O;BU!7Gm;zI zkvDAOcmR;Ss-6FxyhR>?Cw&#K*D31|tG0m*PZw%?OR;iIzbAbP9zH^wP+w5RFhc_o zp+ToT@KYY8Y+qBzwKA_t?Po%gORvp=+?~8Le!t^`K25f~5{|}*CjIWJn>Zj*Ka?qr zxoS3bmp=@3`AVN=>A{8P7v^r9^Aqv_;qG`uJvAWTSZ1ij;zeBX{e8ldWdV;w$dacU zH#+g>L4291?n$q^*t!|oB=BeC2SG)b0zcEVr*-`J^7t4#dR80JY6;@X#rY*|duq^r zL4&3HjJ)n8on*^l?Bg})*FPevvl$As-!M1Mf5Dm?9&qP9x1v)V6mX=zLAy#>L$_3U=nX`j)4YWf6 z%=yGfzpPMpQr*e)n=suxm-fvNU(-2iHyvvWv=efzm3=xdeK zjQN82U{1KQda&pQGMOA^xXz4nRJZ_{LNFGI2ju}$pqE%48bq!W_v@D0M5(wP zG{2GpOVN(9-F-bBr>-Mq?3WP9 zjCn_eYIX0Wlp!%9-)-|BxQ5xc9*Nljr>(2WsV}36u05!Q8)oFdE~q#Ag7Gm4ljx3f|2qm$y)N1L z>@vA{*39t!Vc#q3{XP1iwY`2!I>oI4C&IVnggH}Z1>B$EI#SVVrgE&gUWV&yf`M=V z@JUPdYCG|^F(yNdM)JPqlDIQsK8R1HW@-pYq0Y=7cnw2s0RpK#vBMM3M5QzNO+?r` z5Onvsmz50QuRd3cqvd-|Ex!){h@3G0eyfs1zb{()CXn(6*%un^zPxgpwN6r7QB`;1 zWVEgr(@!ETmtWOaN7mK8#K8D1+c5QzMIgqCC4>+2P(=dA0Of%mZkIN4ivy-PNNR0g ze6x>yO#&NHv1c*z8eX7Jgf7m2;xe{sTPfq_GwAyPSL*?f^T8I+6($0p>miew+My{o zlo`8QEl^rx(vKNyh~_Mo>t8c+l4_Uo!x(hTRunP~ z2-M#@DpI^lK-yJ0(!Zy$z=;DclqQKTLic~Z@JFwfpXc)nwP3)WiO)>smKirnlRyfx zHVE?HKMU6^;rDzdGUq=H0~4?&3Yq-x2cVkPv|+Zt{>N<0QFes>&4kJ!0Fn8z@PUTn z^oYRZeGX(mf%>{Lsv6pdopXZR%DKdg)R%&Jlo>l2uyB%kbSk0(!eaUX46(-JMrimp z;$>CZXp(52T~Nf6AfDX_!oJA9$YML>Ovk&Q*dLJDZ=87d8S7@Z)F#`+?w8z;SuGlS5jFcx)UI7)Lx>FkDlN+iv;1 zgF|8YGJwpJ-l(onq&>J8nsZunTDpmGiE;5n@!Aa3LO4lVItv=)Y%vCN>h&JltjRk? zhm0rCr?Ee#RINv*%6-&}>}E~!ud6JN+v&$+$>84EWrWwI^mL}`hTv&svJ#d0rs_zo z=z)M)XOxGiqTkIqIt$i(CD^c5{J07!uwEp7vIcWER%12eu9(k6lc5~n*(JX~&ogO3 z9^8v12)#Cl$xOx@GnwU=8$qAfOpl6sn^PqrJ2B=s@|hhaJj>jAxoM{>oXSR~PR+1t za}3m$p{cWowKM=LE6gt{#%h#=`~0djU^M|^b)NAA-Tm@xQZUdOc0c+TMUdW+3M2b5 zcWc;93gK*&73j$i$8GIbG8#+P2~RzLMX=rxxeaJgmKm^U96^D~w6YH-O-mD$&4qkh zFwqc=eHiBw>!D$YFq}{kGf}J5VRsi|?buoYGua!sH%(Ph=P;v7>T{@%)ac4soDl_R z>NH3?tL{*IWvw6s25iz#b6D7HNEhs)Y8So|c*a@32&tJFpA^y2*T`zWhO6WeyR3<3 zwC2>|6>N|gxjCE}lETH zb8Ze)JAQnnPnd>3L3pou)k$;^3t2TGeblLBG?MR~_z**AJvt8;%pIcjR~G=Nc`@lc z8%jM=joVa%22L&|v`H53KI~DAw2kg*{?t`SBPeFOB$vzT6nNn%Z6YpNT~Nw>kHrt-rIZ;{MX2V;cxEtsh zI7o2jnDZvJ`Nj8DCa3uQ6v-MOTHaSB{&osA)ePZ@U)vgB_|+1P7$lAK7sl6d)G74( zFb!nnvL#}D30YmCY>z&OZ)adP-H+Z|uXkD`b}8R5Wy>LO>!DF3fCw<@*Es4&})Xk9e%c-j}gK@XoY{Q zBN?@NE3^)I{!sTApCq>TvB5A5AwFlj|ulhb((fu_h)|r7A zpM%&&wyp1{^ZcmQGm`5LfV4g$CIZAw73}QFGABz5ysHZDgEGAv5VnS(?n*nl?$-(3 zB(mp*HQ8*x>9{H)^WFJKVlK<<64jbf&D4N6Y?h?cpRV%h}S}3Fp)lfURC=J<@?^2O@3s z_^B`~`T~ZhRgVm%%hA66n=a0Q7eM5R^~smh8zANph@*I`$M&!;6!D`Ct2F>ao%;qt z`&+QIQ+i!NW|OUJV?B3mhZqJYkq_rcTQ#%&{svhm`8z$$^<_yvpj*EtMC%1ZGxJt6 zvky9b9wKsAD>9i4u))oVX>pErJlHL#iSQnbfV~?v9@W|B<9LMZuF|oCf@>yzDg3Z5 zSRujy456|N+l;mWUqWbb4=;{VSHyMQj$9t1JCTvU-V7{hx?+`T&d` z`Lh|X0p%a>x53JZ1hhKgga=e;XnpdAVE?IPyU`payS|{ZL$RZIWe4!$=!&^_j?U+apU{(-sFAw`0>2e^hF@tAk+aWB;12KIuu}N%mzg`n9s&w zpdOZuO3WAVBQc(MNpfVe-iN=0B##8QS00O)Ygp>ohnm8S7rhrj_Isa-hxUZ8cxP5p zc6NefHn8eYm5^=gqb14VH~#Zkr3qRA|s(s=}# zm9;pAY)NH0ljW|Tn%`vP(x+%AQb#Mm8zy776<;AGHAM)Ebiu^D?j?mgWfT^_@eKG$ z=(%02LvXV}Z`=Uy)<8gNsOJ0l!%shk+{K%2A#18`S-mBDR249REwz94wBy7AomiDh ztJv0BMV}UmfS_s8c(KQ#+u{u`=a)uSwgBHJ%iBRqz1-!abUFOPimL+N!t%z{O;+1& z{Fh*{iL|g5HE-F@7)ZcKPzqI5{stxGk&Z$7(uW!Lw6xprcRAe;UnMxRj;3er=~#}{+RJy) z(G8TYX`JJZ=@bvVc2q~hAL&cksy&Nrl}V4Gji7CLyD|VpqXiRy#-eyqS!L_Jx6NZt zbn1e^c!)SPu_4K2afSqDJIY4sRrO>R4E$Mm%z&&9@{Dx?;P(;p_w>QMK8>&S6$Mu_ zt#uV;1;b0NvJubO7CuCkMfgoe!4S$Vkw$LXwNvXY@)yTJ_|?rfhKkziD=zkT%1nY7KHCCbz0Q0jaUvW z$@~w76VlCho3mFG8#7nFZ-QHfdMrTKla%J`45Ni(dA(j!yAqjYbG@0_*RT-69lMrg zLgsn4`t2bczrZmZ^D7njqH9+8Agq}yY`D5)%n|^()A?lTNqW>b_^xhVr|Gu%bAI@) z{wg-zp4YLc4r)NHuHdy>+L%Rr6{=7udxw>hDTp(Xjmyc&awh zqZjiR-|d!mGW))nEBEb+$)C}KfUnsgj^*3QyDe);G{LV>DR*5kH#FWXR+ye~O2`dH z_4orumM4(fJRVGCv0FsdD{LiL&Sl%03Ro|~nC*sbH@c#CvV!?CGPP(^>8ekh%sC+6Fd!~N!!VbAY^Tl%ItqSVXQPz(3eX3yH?AWr-GjOrNn!Q5^- zN*WLkBSt`fX0^45)CIPBGmIxnro3HtH7U{(-i_||g1qewsq-rdbyYFs6?q?cKraj+ zO}?m^A^-Ua4J>BD(7}1d z8j4na*w&ZkJH46f@H{cx_#=H(pGGq3G??C2qdZ+F`y?3ne9`kwv=F1H}#|Q2Lc67kGBR3lVDKSYJ2*qwNOO+~F6`6~+UbSQRmyL#j>RD4*LTk!X$;fqQd4>y!@2cDFDmmKA5)Nh5W+wDqzy6>D~lHLvKi73p9G<0phi4$KAfH znV|KowK_X5XlYQ8e%J07N?*oj@_vghj-a4xf@NZ2+8|~cH)fYhd)h4Uxc z$8gb;ABI(XIOR3ZN8hGs^iu-B0|=P8+_hqfF+sp+NbL@NgYW8b1#Go`b6U@CzNpfD zSN7~FJ=i8N6xjUbcl3BCDMTvkkSL`zuW!FCwVL~VQG;98Fn0% zJQ}7I$t{f!=eI4*^5Ro@$3#FJXpoo(o1LqNf!e*xBgBR%GHA}3jW!CfpJpPb@lwrD zoi58&7$~9^EIvY97fql^_Zy@K)dCLq( zLm?r%w0bVR!AVlj?<2MMxO<8@H+q=wj7pPqLChv`eV|G4e3|3YoeGEDrzUHGH zfn6GVKw@adh;HU~gOmqMJNa=Y=XY#L&v5XCbAPa%?>p1y^gMnq@yl-hxvm=hIMnWR zi7a))W-zp2MANJ=CFe@{J}%v`V})%WE?S1&)mCZLg;>YsAe55}us60Qq_b>GkArU? z=IWGuCV$W~rR)5Sau;59tyG(6l3iBACVm?~IPMu$a93S+jesGw#~Nt?5PEb%w!ow{cKgtK^68ULFmB}te~8bx)9q*3@BOXuB8B7eW+x(!6aQP+&Xd9) zJ4Hju`3iao#1#UIVg&OJ#ntJqg~Fi?uw{&=e`F@7#wvgD~WXd=dIdUy*CU zrje6#@${Tgsh5N5p&MWIqDq398-2elW@Dg;W4 zC%uj=?2Y`7>D5Q_l5LRVvADlbuo9D;%&aOF3?&Q8c97@i&vfF77Ow@tsWR5MbS{tw zCfq|tTna4)m0P18UnpJ&DsivmyYdocEW7~s!ZfHlF3pwpYfbb)y}sP*jEfeKC;f1K zt)G#^%|^$8ZRgK|9mjMPzY&HmA>UO8omV)OW53`??$&qlCvBsL6c_TyL`^&+9U?IE5#|j%7opY!J$p2N z(uMpYZ4LtYig6`4p(J&O6VJj!bAbvt_MD-K-yo3+&u@_YhAb2BlYE4f%J9r$zlAT= z8Og@-qblAnSzT$RG0bRBA(AR(S&0gaQ;KAQRAHB%r;aHNHy^i%o14{@*)1$C$o>fh zrGV{uZI2(^)9dAS|PCqtoh^o;x0Y~cKB9sal7 zXtGR&dHM?W*Mu}V2GXB^{bvc(aH6<^|GQZ}cq`=pnQUtSZn;p90R0`YV1Wd=Fi=1Y zMovV)xSEavsu0?r!lnXDJNga5*i!Uk@Z^YoEF2UOVk>2~n!=wkV;A|6iOc5hr~3*| z3bJ=!rP!DSUOxww6$Z13!knWbvc8U|vTnAFXT0A&4l#Zy(~Rj8WDbBvgG)q#%Cj|O z7V0GjEetRV(X3{C;|s8G-A;>#?4fktjQ<1;-i7BpYaIB;>1wpq+F7oeNq=lE+;xv{ zX*5|(a9atm13z)vcUxLv&!J;h)sDTldqF6qn0J|Gpzs3qqK9P=k8 z!-G)I>$5M?=zks?5WK&Icr&GnK|${We!d(4Zj~jj^*>0YyMqYHfYK)RhgeF{9(kd? zT##G6U{@4ZZZ{R&oT-}9cts8`vd-D0dDRtpNoNk?ug$+&$s)s8 zu|;{W1hbhl$qb^k#&CxR?o(7lV-%ZjZ2iC)rL?Wk79v3PAHoC=HG zBOZ^b_alh8cPCJ|+W>)S7Lp7mwYU6!jrdo>|3^$A|33jYnnk7s|EHc*>P>+L{!bA} zE({A19D(Ok!n-uIjj28*JQ)lacgc(otgz$80B+iFK?2BRt?bPt&5S+=Op1B9I2oC^ ze0Tc4lN1xizN09i4R*oaR=ZFwY1`1Msj3o)hbf^}e6oan6`2Q{?cXbOnHplgF^{|1OWL;y>#w+)L(^`3C=d|r1}I3W1*{D9w0zm^j86P8)C8A}8Q zXxt2lq~&1YsASPBrT0!n>02+oi^x*6*0go5nno6yzQ^q?u>E!g*=PKG4cl=Mt{>CX z-#9gI7zBlwPdP6V^18f?l(a?=R3pYLl1syXUA|cnu|;0xE~!=i!f<(ZEqLNXhtp~k zLVnHwE1#Gd0MlId=HFA|kpcs<4(MUy|a#`{KB8pqfb>@j{27B2{SJmvY zXK_lPhs?!E&ApCO*tKx08rN;E4NpTZNSbJ7T9aCfxR`3tAgcy1D%*wJAJ2skiU%uH zddN9Gjx%f$#?YwWz6Q`{NoJO3dMu<@J?mU_4^l%qhb=c>5Ji~bazlEeVU2#+1i}GU zS#zR|P=*p$>-??w)?B4V*QlrQHB(s(Alk?i<&PG})Sc3)rju%q+-gjfC!=zy_}TcF zeygY!>e9-0UP03tNcsxxk?tdN-RV!UKY&ua{`7PUZ0=Gw$Y%bG+pr)LjQFvDTc=z&0M>tmEK^lW1yu;?4+W%5FNu;8vXT<%!I$Jd)+{Wc zZU-t1_2))gh}!Dkg>Ih|Faa?FuaG{C((MFv+3!lp{31X2h_GmrQnRuys=6+YH?q4r zeIc1bD5>#!tF`GGBJ-z%rx8-HWK58DrCT$^IV)YZXevwgTX{B^6amr+?{c6`aBCb6 zp~UaK54H02D7eIZf&=&TJq12?^R`O&9)`gzTMvb5>KN@jq$hFhBeB!^N=OdG+2ECX z5Em>d5xS~+HHGXh8OVW>N#-3_aE4{D9r;UJA8X}IPRSRvX%O3_Wa+tSi^ zc9rClfU^FNqC^98NkHN|&3sAO4vYYyUFFqQNykpyf#iU@?e}E_T~~W5)KV}wbs6TN z60p#B9z@&Ym)x`O3;dNE2f`}sc)iGLq<{_uF&3Cc=*9}`)SM3@n~(`|D_&e@w+=@!zH0Jn93+_%!m)LnUmX1A%fW9K#;OROEX+YoU18jOY2Q}U#Vo8>> z?!sGX05s~O6328<0tLcu*v~vJqg+fxSy|)Jei7O9l%DdR62(|4tjsass1eqwLna}t zuas6hcoQm+jjDh&bg`uy+ZuEF@^WX1nT&J7;+>HA*&`<1kIg^ET&=inhvZn{^j}!8 z)BvLqy*SVB9RLHI1v3N5Khh@>k%x0>PJIQ#jviRne|JaJwr7vjKYSf@}3aKkiFe}Qtylc%c=NYv%5TueBR&}2q`-?8b7i4ZG10SLgd## zj+LE!RxE*>cm9_-Ej?eca3dYbvbJc%v|}rLMyVQ!5koj^j%YtOaL%O$3WfrF26A1& zpZu0<503t+vzXLU^(3=yy-)=qaq-xF+=mDk6MP_MlJ? z*9Jw^aE^u8MEz>7YY(-Y&X*h7A85Y!J=mK(q*MZ*9Y!%H z9&V@skW_M%3F~zhlg|_G6Ils--hUpyeG%wJZ~BhrY&Ua}N)FF3OA;w>^*W=u*p?!bQ|)=aU}ozXU0PK^+>h?-ue znvqjXZ;}%vsg`%{Ml}vh0HmWC|wQN$cep>P8{qk2Zu1Z3rl49h86>kZYT6?l&DyMtK5`Z7(^aD zsdVUpyNmZZ(I&bMYu5wyyJLn#kO}r9*TpVuG-FL~O?VYvxnudRP4xHQwb76Wo%>G{ zSbi}OeNPdy2hCxRB|6?-$T&g?ftWB2`|GP7RK5VY-Paw#d!p%hiumo7vImkr8%U$a z&5)cKP@?dyX=N4(*@nw^88N5gnny+DQ8%3x(V6YIfX2) z_DoMU#rIB0RB7vcHmz^NHVxm%V_tO#i}+`31BZn4w~=fA3a7}|ntpB3)D6|BN-6uA2P zbelT{?A7IyGNN4*7?KA~9EoJgx5j}07SQ#cq=cG?%_t5Et^}Z zbuBAFV^hdC2hj-`>wQ)s+1v5I-5d`}m}>d#@q9xuVcIvM*{)`xJV~`3-c0Rwg|Vf_e6%AIh7Q# z+pvd1GrS*svN!p?Yr8sMo-x0uFp(Yw5Qs4 z@;vJg`yn{&;*K>K)F_G&ZGrkbfS(M?Zl|VNFGSQokaZ^UhN(8@u#<@Z*K5AI=}vi| zgg=fduJSx!e@<$00gDF;e(_SCL%h~3Bup+9Q*?fboL`K+={s(WPneZGR_;LBVG6)_ zIzD#j17SY@cS#On5nI2b{y=Vsj^<2kk|{P}>9AFERC@@>h2;|zElxO0o{-}gW}m|gKsZXkE^=0g zRX9(FAGOz4N*&Jxlrf17je@2F?E`=tzs(_hn>Z=~4iG~`z%9#0&9%-vJ!^L=?5Os= z?e2l4|K|2tm^p!A4>KIDuLZzfbZu3Yj$U&-viiYug-3`xfIivyE~V=ySK2@FV$Aw} zkbCGAjd%2osW~~#c*)KR6MS^ai5RHeHQKPdF&DBC{+Q+xgHK*`w*O&5{tECEHy{Lt zQ&*YFq$a&C2kV9y(FxLn8N|?My}2eZuR+g1qA&E?N3y3U!h|SaVI!moGn~Wr1kPCJ z+0n7^7JBSyv-8?{=%B|>L}fR(qYBxGg$tm09j6Dlx{sO3;uRU-y&H1Ltj{O%HBS)_W+v;t@jr}{qhW#r}k3v+t_arfbbQHm*yxI-s$BENsQ+swJ# z8H`~&`MX%(&_;?y%-<}6JObAe*}+4hHGQF1iP?P`K46VD(3V1yPLk{@As8IWptZV@PV_g4x-76EZpka_8RKO28NsvH8 z6sx|>G*N4$NX=98TUqtw_4A96KK(}Rm`44I@W=I;*+3_;Zp|9yb6>*7#)qkuR{>j_ zm-J=e`0BWQ?Nt(E>L>kKSJktEfWF7>NYNKMJ0mO*`KX^cl_g1%G%k2+5 z1WT@?0Nl{MYoiutQG_6UzMpvnmC++XyY?A}Ju08ou6o?;$6siZ&5o&HJj9_E>ride zIoq%(``p|aiG7&++qe4d{*0d2W26yZ zWcR#>JSf%^8X1oJuR&ND#=>iwZf$Zep-gh!G<9@2u-!STxS zIzrf7sEl0@Y%iI2(n0u*E)p9hf%1o_Ci#^oexzlox%q5neDQd>j=p)8=1CWMte zk%kCdwe%cFATHfT#Q8(DbtZ#t?mPY?SxYm;{~cB31yBRH83b|MN$tGX?$zDO#WOY) zwT~vEjkOyPO|)9T54iGodcvXrF-ujd=H)m>NN)Y@$R#=GF!V-5v z&4UM{+Dw%>3X*O;II=$5RH9Qu1RcOSWD*+}V`39(^=pixUWQ<<*DjvcsP;rSAQ7+D z4eIx(f?@%n&NrPc+AxJ{Q9RzUvd9c$o~A)AP$dx5F%`3nb<}Tpfp06JV#~Z^K^7Wp z^j1c!`UuL^f+(0OB`nEPi6g{VJVlH-y)3l{JFp1WVb?={%vZ&BxZvbij|ZQ7-0fx2 zcvbWcCRwg8oBd6!&w%|+u5WYs>Z(?!P38e{h7t$xr>EW+ALbq+J~9oQ9egm4fZ0v@ zPsO!|ztTaOl}-wy3*OJEBr^IHu$9|@%&g_fUr5v{X~wp6{(q=jd$HjpNYMaU^DLgL zAd@(x7@UFiW{gozJ%VJ|EB3Ys*Ym*jQ3*uuV!C2Yfz*cPI(aN$bDthBtV9 z3l<|kTRzd=R5TRX3eu}z*0w76oe(I7NaoPOh*(e9bbz|`h0&TrrBxw-nfI@52R}vC zOic3N1_RToRcd1GGs}p9xxGP{GFJ^;Pnq6{6l626ldh=kbrnEX-nWZuS zIT6Nuqp(a8zO!OK`DiAy3)-Pvn6(m5jA01o`&IR1QO#OY7|f5F@taP1DMww`0(G8; zQnMr6b=EyA^U?0fs}itm+Fd#owtda)Xos1mW3tE!rrdalZ=>krX$BuS)h z=WUD|84jibU4}Bx&?zt^G0}}HBhw)Pu4g^t%D>XdA6D6vN+LXSX_~Z6g!O)vd^x}E z9FnO$?G0<2TPXbF;bKu5kJatnF=A))Rpwh!CcH1Fpe|v-jf6QpOo|>L>nEpJDFUeSTcPQf~#Y2zeO+@Ie zg}aK|O8g%9M=*DGDM=Lo*OJ0-3=s8geu>QY+TaE`!EfK(GM5o7Q$DK|~tKbehSVC>?_{Fi!_D451YtMsY8doWD0L=y(;|kCg(dHFS#<497J}FK;AU}tyzjJ=INDyLS?DuB{^ornz z#>9)zQHJh1{^S$)$UYIpvZA{&<65RGB?i2sj7+ArQ>wr{8~2I)6Q0QHh%fngbsUOq ze!f#D*ecRCo{;XiTfzErO8+(9=t>xhHD|(#S{^jK<)Bl2>}U(ysWjl^kwV#nRoG?p zi`-fPok;>t+RE5GMkN@-PTC7%LWQ14(@P81*;HdEgS$oKx+`*oUFdPC`6vao&7YMO z5iS#h6?6u(3l3Ok`f**-0Jkg|VPM*iN)lq;!{-_30i6?RnP|fs4DIn(JPLd&d#Jl<6-+%oOnr(LfmnHD>c!|W?>zpcIg z;BZ|@Z1G+)*_x6LkKCw|JBr#&*Lo&3Ct~jJ5c&vZdtI{iX3QjY|5hQ{rWipk=xw~* zIRuJsJk@vy@?GT#i)jfPt|}v^;pLoqZGVy^2pq31u~^NH6mS5j>k+B=w|Cs!yHz?I zw|w@qUpV~IeF(-6%EYASAf9fXFXfYSe%Kj(XoERxYYGZF8FsjXcWad9@}ZI!FyXmp zA>Oi#>Wj$}Oy?Y0Vt%8B-b&@2vZqwvc<55oe+^1m^K}PSClIXIO;__&7_s***_%5z zhTPD_3*b&qeN6{U+Q8uLTR|F|FI8~vHZ2+GoaFT@hF-VpAdLN zqSqI3LP!;olaKaFkO8+;i3?X8nDGOHh1_n6v$sn)pj8Ir%ka%2KBhv|#Qj<+1KK8M zSOxZg@QfF|OLf^uIFM`KHF@m(%$EJibkYVfE!cG#nvTpLJeUAMr{T=*8>=X=S#ZF= z6a$)ps7MUp{uTkeVmSm}ho`AHI@R=cBu_ixQ|rf!GzX+!6Ng-eN?nWVGjP^bZuZ8u zCo$h3x1%;~TdxUG38Xo24tZZ;T?{aLhoXSZW-7gjjU7@a5qtiHhVO-4%H15-5Q7!%C7Z> z$)NW@xHf#9xY;fK-HDvt5bMYy_-&M1EybP`5Vs0TdgG%{;_mQWVTHjW)s31b+^p7{ zn>Qjh#fq2><#(XQSf=RjSm87#Tu-48H0r%JRi*aTA1$HI(7RXqDg?CL{OK9rAX`u# z5*|}Aw2VpG%$|7y_m`jUF?^bSX0wdG=hq2P;?qNBl6xNWbvw<5Ksr;Ij8WA3l(Udm z07U&P;cke_huyyG$?x%da6#U12NQ25qPCPU?a5q){all@^!#XfJ9|hQ>T}cD^JC*J zgiYklJSU`e+}&N$@!zf+_7NW08l=3YMtzob6|OCZcvY>}aa@0Lc|m7c{6R;f=LSZv zAb2(jQe_dJ53ypc(&-x1XpC#TPg%!v@%U`f479dc6cdO~~l7I+f^yEOoX&4k>m-S~-XMUKX7Zl;| zDQkmiQQBuxrvnzKM8k#wtPTOg0V1uyKmdu9h@hJA@3CsZ{&TVIKJ60vGed+94nzuL zWdbVfe>Uj4u2T?${O5R;D8zUA^Vk9sc>MuaIuXe=^hVOqy zRmx~aUt2;$<5a>Xl7UDQlw`ZkiA=A^ug^clc>MNE1T8t&|E;@An9%21>;q)C#ZGC+ z<5z0||6^9xYs+Sqv)8wGpATd~0<(}vb8~q6)pOg~g*)5%IamD#_SxZyCc8f_>f3 z)_I+|7ZiZokqodwCZ%O{9cy)sDD8zPs%8R6rN)+ml|nV$O{(AMSUW>OBw<#34TAM!bL?ra}1!QBv)$RQ$LMYUg+pS#MB@n4DIFV!66)=B^^`|NuaQ}_bLJ$_;F9;r8 zZpALrkd8sAMD85%TXCSg^d-dF>ZEXDaUg2qr@V#tJvsOw;KBMvyKCoP83&rD1vtUbjFOQiz%q7jvZOnv^FTv>7iZ zB01NQQ7;c|Ux@;NsSs}`u= z7G%HfN~Sd_UJsW+^pq%HSV^9QCyVVj3SmvfY*QLX+6OWhyPg?H&zPn42!!Cdv#MMG z*@JNBL}6wzOP44Xn>oW*2Uv|=u&bNF?+5!faptN93Rg}=TcO^aU~N`U@@mgQ%kIzo znohP@3K?mlQe!MpXr&TxL0Gu}vd|YiG3Oiu;$Nh|Hq}Y~gA{kDWCae0e-Q_h{60s4 zo=gzQiU52?|DtTsNDoIJ($6STBc+`$UG`I{qu!T|+5(hh8mlfrme*v7e8raZeHcno zFqh)(6A;M%upRyex|6vkO1I}X&%t{?k;!X5k>&M`|NZF_Mv#ykGB710lR*9?HKTg$z0+nZCon)21 zaQh@?*wBS6kV%&lpHt=IAAsB|_}r`+JHoUdUK#)`oZ>YFAFA>M9|eJ<8-ru^Vj$4? zfY|7v%uBK^ocrJ#gKU0i+k5pGr4j)LAAhhkBjhz*UGW753rB3CS-t0zusrQ2;Ua}( z!%E;qe_Jt_?%}J$GL09e%uwZ(CyTi3S-7H}P3AsiqdCqsb+*?m9t))7mmi<(SX}b^ zyrCO&p+9@bSZ2VUKqa+YBZ_?v`P_dwf|d7!@u~TTG$)!Ud>%{XW~qBDb7=+fM9fVQ zBnXQH`>&4D_67oWy^$dj#?1?q;;n$^Xn}s9lMy+H8uneAFXsNZHVBg5<#niX}YPt%(FiId@l>%v;Js;pgs zRpMX%#N$H)fiJt0vupTWzua|hF&;B-;T0)*jju}+h=_SpaK7|8>_a!Q5}|m-a(7*V z%$!1hLyBE13P9w`Vcwz;?&jKb2TjP4=#aTHvGvO;j(>$f@h)Je$@q2%su_z{Xk2^Y z@IFrDjYWEDh`#a}lzmsEW17oPlIJTiki^$#hjlgWCBxeG1!83l^5b}3oBbCq(U(rN zTg^WD0lGNf6;!{+Ju!TTO2Wq*P%ffS+^&_;3pz#-W_&s%3VeohZ`i1V(dpS!*!@?Y z>%Y(Z|Ak9PodAf%2O6v)KYbGXP$_JDP66-vyN2OU(*OF;;v=R$C~p2SA`?A%DjVlt z%)mGPq(IGu{1;NEL0MqgADCN%OTOHuAWFLbSm%)bU?Vn46Tmh81>gt&fCP&04?>++ z)mo}QI!*9DY!-+yl94yD;gj__;8*~~kLoMrxBX?km7N4wjPReicOdSU3>{K3L!grJ zh5QLIhbwohU2(?p_KjpOl==}!*s}i!we_4%$8@k=b+?X}BwB(P2xXRN9hbT;s+q&4)Z=0w7)pD$KW7ceoj{HGt>0Ww4w z@!oew)(hyZERN-N%DIMbaz=c8P60eT0WG|UrpDoh{+#~D) zI2NG$k_CU*_IRC}dKBQ22m+L15m2^blNu`M*%*eV8YV0Pu?0!-Is{(g8iZDPVzc3) zH0G%mIx?d&1eQPu)IhP)?u?dcY^@$a8=>|j(k7Pj4*rLgHkP*(7wX`^IXj_`IkG{x zk9TsPl&Ht>J~^`6QdKT5DUrjs@kI^}V0*aH>uy2UZQ!mr_>UBGpN6tncZLxXz0GRX zO`7$a_EY5M4m$_1*{mP74_&r;U5Y!u>vkFA`_Az>36Yy2aGL%*>C>m8WabYYWTD@^m4w#H7%`$b7_=U?vmIxgPZJ6ZQ=?fC%6F3)NDDg3YoG2m9jsMM7O1-3EhGt3vv z>7ELHXC^kx*tQGZx(4{f-9#zs4HS^}d9|oEhuasN;@t$%90Kz4@;qgJ z&SwV)t|Es;XdS*B3$}LkYkNMoxIG4owLn6)1c|?NbmySxciImv?hzdwEicXQ;NfB- zcLis1VTg8?0H6{Kse-f#5nU<(UhH`ew13UJDJTPiu#Pg|d$yn?|;UQr#bHAs`J z@7MC7Q2RC!16=8I2cLSDEmoKFzz-40Azhn`m^f;pxYl#UPUdf%{;dAY!kc%F_teYD zN}s=x^2=5Hvb7>l1A20dcTG*z_psQk zAS3iHT7^XCa5Pz!#o{@|7P7*~Rg14>aN|u86G=>_43kM3UCr6sHi-fCp6@*-cKXGR ztG|6C8AbrnR^9@kp~c8?B}s^%djteI5>(TY!^e1TtxCe4Hfj{4{?bg_FS)s zo6x&zP`*Z0O;f_G&s836wbdPvH06Z^aT_W^h3mL&3u=uvZL!vbnOe#Un!`<}s)`kd zdfW}PfE6Nci@JLSr9z@H?{B*GV%;}0{rJRcJftul$FxQ!>&J44UaJ`7CL(ai$~AU@ z6NfaMvlXNRg4ZKEvto}~Tsc|O{fYTJS6&p{CbUL@kA4wnCF22&6VtGU&Hq>B*c3*% zrLrR>g@-#pg^yW0*-yc_ zUUGAvzJ4eLukojbkym4vlW9;EGd=2bFIc_w)4)nZ zFl*f5J?Qfo@%7PPf-Xv8gBo3=RavU9o6=c~lU z3X;chgZII8cA5Cm3G%^Oui9i%X1VAUkT(VuvV$$T@l>-jE+JAghU7fckKa3h%^pq7 z2FwD2kLwid2Hii+!nk2)qwu6zMl2BStYv-cXDgC=06sYt{=sN2umq(%8=ESuy=m!6 zHUWKRA)pzAq5t0;_TGx!P`W( z7PNL_TfO{Ce9^8!=uBVdA;+b39IG6keUTt?gf=41K5-1{phT385{DX)#dOqdr@6&;P}ip%7}-04B?Pc^LRX9tpCpe9%7+s8)@a zNZ9t`#5~NdXWvUs@F7-`ThSi%U$V>ZIt}(lJ!p_({2vne?ySKilBX#X98w}SP~Aj_ zt3H2ET`7%FW0yAG{&WHWR9Vbb=$~ib^fc|ZzYMP&{yasJRvb`agq?jN+A%stcXP88 zY6=kMN_EfIrjU2WR3A=yp>Ea_3Lx;S_8?!X{u%C>jVU2wtSncPA~gDJLzXr58ANJM zH~T2!3BPj|Ou+;cdN*0eXErbELIETfJy7#}xwwD_mN~=$A(P`?u`X5!bL^5m`Sw?Bir# zmhrduZhfYS+sJbPJvJIQtj}jnZbMVw>$>)^Nk1TVhyYTI{ z!4$8A^_w{B2AfXSfx@|&`P-ZV;c2INj)1$N$vbqxdLrF2aOwc1Q6;?dbgsac535Kv zGRkg0-=@((|H(+L+Z92K(io(j5q+U1s$dk7$$W7{XF!oYOaY!`!BOWER_D@jnMb4N z;?Q%2mT}fWx+dt}v94hU?fAFE$4ZV3>G6A(8(rfL>v=Es60hovjKv%2owiM{etb3i z`;HLlsN6m;05f^a;4C!WBBQ&dj*6=rA#mG?e{I!O8>Oq_%r zita`U71$Z2y2%pjzNGuGdu2VelW{8LxQkUr+Hjizbp7e`<93%Om#nD=zb^5OdnWV8GQaSYK5*dm^cBs_M&VeUBj>I6i~>OGk& zl>xrNaCB4&h?wyddM;Z(7b+^Krku4ni}(_t|MLZYPhE&qEG!+>3?+pHUp}-4Tuxj1 z4zeh_g-a@%)abFEpiS#N8LG2;@Vy8wHI!lpTCl8AILUk9&X9~)?JKMtb=k2$*Nbw`$)tL9a;n6H5-y=wX1ToK;r338vUg6(C z!AC*S2A`r?_z8O>-SI*AzP18s9alKObp}P)fqLQEOPO(zJ8Dk>*n{+7ZZs%}ydW|9?KU^2whPbk}v<;!T7H!}|4%`PXS zj=JTqL#F_Zq&Xa|mLdP$JoSej{#|xYO$0ny{03hL>P+ zz5DPCAjLT?@;FX_ugcLhYAjXGm$$j4QWvu3H<(MllJdVJ%BKeNj*kHQm!5K43nu%l z@cY$$jzb&mv#76~@pnMX0@UTbW@2iI#X5mr4zIU(OYawM8s`~;j!}~stCwttyQ19( z*7H}7v@3m*NiXM?O&?7L_l+OAmtK}s-@BLeQTwahpXM_dhH!i86XC|eG%W}kDlA<}kj4i8Y z?Z7#)^|C`n9fiCa?wj1PINiK#YICF50!K*;xyl++aAw^@?A%(WW=mO(i+XASFZa!k z%37$B5S4HZ)u57M>fG8){(0rr@QM@U{8=Y!aUTo+JNq|{=3`iV?SAC2<8{ z?2c0}l#!fTAf;+1KbtD9xVoze;3KN-nuy>W6^(q>Em)>331<;IGGtxCmN_2U{agUn zSk5A?0pFTQHwpS4`p&ZSJ?Ykj`d+HxHOs8F!uGD*zK-214bs|X_Z2*&VW%2H&g+Y8 z-GjY$Ey3@fO-aDcfelpth9|v>9$F&)E0Q`~lXGN`dWuX?Zq@^?!sj+Bw8&y~Sq*wE z1x785eHQeNC>|q zL%4~<(*w$+l6r;)xXlj$*FPRSR5Ht&IbYu<`(zZOgcUc_^3BKSvW*XYA*X z9@#gdYohrz>^^J!C6B!lLj$59Vd|fLR`JO7erBIz=JrY4dRv5dRV*JI-GtYegNYhuL$U&cein|R3tPCor& zg2~WAA(fCMa?^qB2(7jVg0mN+YaVl#QUvb?)LF=5Bhnpbv`&sX`p&pUg*#c7FF3#* z%?9W{biw~m3Vg`H|8KfLsfBd2?}HXH21_zx_~4mrdGM2^dcctYJW5Av{IsM5C5AmU z1u#&ob%jzX)lV=eKb6Wl>`zPT5?jL;PGH|)^jVY0AL80l+#@xLp#;$6B zmzR%jB;{2s_hOm+;pS+?Efq}avmvWg(biO#-~mE`qWgTb89V3KR3ws1PX{u2lLX03 zv5O#a^w#yJcx4?xqu<}Hf=3ajAmOx64yWBUXnN8l3n@Zo=&r#F^z6ux{}|L7AUe#W zu4XB_bd*Fpe7m82Ou^ES8~%gf&)>v)46er60w5D^g9`cWfA|O4!nxzE`6FZ1Api7h z5L}B?VYsxoBwDY<%A&vJ7QYiuGTAoP{MoOjAr_?#n&}K|59luZ`auiAVEhZd!g6lc zI`EJfUZEF3ZJ84Wh7Li7t`5vH5#Ov{AJQ+-Z|1gvi!d(JW&d8Kw1Adwu{(>}Mut%p z!Y5~EY*(ttekczEN65RXkgpc0_V`FWXW}A+SlhC_++nm9wsh=62 zvnD_~!~VwHGKpYv=XXg~(k>-h?K_6LyyMv{`dW4l=L)Y(Cx6{Dy7lbt##!Z#T;R1s zrhlWRAQVPG%1lM=hFF?gVXi)mAAK>9ZFvkZ=(=P%cac*A3Adz z*<|+K{lc8X>*dPf#Iq%Uw9@RfzPT75+5n#2Yb59MNmvajqy3rVJf^~!0ul82w{7fg z!*1SSh=eur?-IW5$q!#aQ2m>#@bN7MqC~Zoo4w}ibIfGo@f!GwG#Y|qq{_nhhpRDp z+)EvHPf04&(Ik*}T$nxEAUNzp`7AC2WEShqjmPO>1xg0-$Ag9+hqNUl!2R zWWiBXlJcMt{GzBb%6!Gie1ccte~Gx(u|wD6M3b}S1m^H&AcIuEBVabhe`g8i#vwN{ zL_5}`v`iwcV8B$_S7`;yjA~#whRq^}j+|7Gs>ml|J$-ucdnlWlinH%Vxr)fC5EOS0 z^`Xf2)e8z~8J({Jq%FOu^G@J*JggkX>#WjUs3~p;4FybKe6S6QcNh1!ulDeL2y;Z3CvzE!xKn?Zh+y||zqXb6D`N}Z~RK^hx1R}$hbJapSV zV1-keY&CKYlR6xHU*T^yfFHe=KHh(n`r*4cM{?(i95mqo#F2@rHk-O4!o!P91-XUp zPvHwsqp)Xk6&#<(xSZ3-4qU6+4l0~R3|mT3%&wT^o-=8w%Z!(tq}5Ha$yh?_kDS^& z&fZIu9YvH}bB}w3j^ROa#6TQ!jEP@N^e<)nEZ}F;U$gSr6=nc zddFwFZxL;!N8l00U$ez3t36_Rc(aE*E z!H3(JcK6)T(WWw43Z(BBua(Ugb>Iw5C^4tyd&c?)m;XiI|G*{cr+*XELd%x5AHdxH z=s}fCc61@8Z=_FSJpubqIxR;Jj__}mKsySR=`Y7mm4eFfpHCZ1;W++&-|R=M{P$ER zkoXD==AVCe3S@XMPacmXmQD_bgn(`ACSe%(n?xUqBqsSgeia5H>hI~L1Dq_(f3D3( zL#YJvZzjP|C(la!pQLv(VH-G2{o-cVrvZ0bb5jQwnmU+VW^sBeWel9|kBi*KQC{%^jB%1;?}){;pG1itefK%ZA-q2cU0yO@Y_sVzsU!GBv3Rw<@o1uQ4sJk zPBjyViOyaj-;tt-LEE-&`pFTVJ%Tm^XLu`$tnl$DP^7epO>uH^mKJcN5pG8f`tzb{ z(wL5OZl(P^gtnzoiJVxJR(E~BZVJ*8t2h!5Y@q#XLEnDNp+NKMQB;&Vj-y+YEtqjS zK)0m~#Vln0*jW3-V)y$7cLL>Fs5@?HN}nRaelwjU!a$|AfwC`u25+rOC_;?}7*JB} zrR|HSO-!b!)GcJ)hKWP)Y>GqNS#f6%fE$v)@|o$kcG3S82@7*oYKAM3|1GNB60%NO zbwy{&LDC?Z1Vxex)YJJjlbc||hIC}8uqna@dZYiOPlX0Ge&{NC*eos6=0~;ikYPqo zxGnUitlwhRvFm|p0;{v?^%G_da8Sw#fyg(`tv#(nho{C5FRa$@zr-}LX^o2Ee5W>^ zeqBDa3RkYwU!HbnRyvPu*~IXZn_LY{6@Cj6Q$irU_IScP*=m>`ZKyjbv&@de^Rb!! zgvJ`LVTlbs5xLX~Tg7L|Hpy~-+`GYM==@$4?jlX$xbz?+-kkZacs*t$i!_^~q9{7;cYEy{A^ifW~kiY6tIf}({^pi7Xpp_`}St7wq_ z88Hl~C(bcbfp|<0vym5}`PGk}M)7TFYHwU3U$a#Z2BA=u+`AK?zv}?DBLd4Uz>W`x z{3jWYd{n8mw`wB9VU*Mfl!bOLF`CyAioNnQYj$JTMP={Bz~af=d0|u^3RwfRK+riA zEy~tTj%U6hXWYUiaB6v7KL5p9Hj7~Bz}=}qNBn|`8ix@ja!vREmv1&UQ#FEq(fQEz^f(q3TRnDnUR2K`t(JR|3M*x=3~$LrITr2s$HOl;BZ-;9 z+>Kt6N8ZY8jSp3{4!*5K(2uI2^aS-MHrI{Y@oM_q_$M=%}a7QOsCaNY;hs2kTC)LJ~`XxfljhCanc=aoMRoPWqGq02YV_Ar-MPHL8~e8y~Y zUlI1sOLyccp!0j34Vd2ezpFCHQI)RM1YSsU9X z#}}9Ik6te-01oUexY36V?|i+nXxi=GZ%v>my9TilIpz2kfSL6lUip4?K>2QEbi=a3 z)ce>x(Dt%R6~#StuasKrVMg89wdT3-ZLNH7zR3#2gFL-^bi1?YE?C}?rH2}PecQWZ zyq~?d2(rjI$u?CyZJH&*ITXcAVUzci2UDSsKC5#VxG0J^AFJuyVUF9rHD*YAzeH13 z;ND(1f%lgt23b2{h}!>($cEq~(XC@bBketaH2)Oxn0UUx$N0H`vb*@A5t4Jk{X=w1W&vNOA$`s4ixFN=QT{!O?8vP4Tx? zLu8%$IvYqS^)^xY9sH>G^ih59#Lg8g>d%U7-`|Hjck~~Fy;~Xw5BAJbPp{aW_;;{# z3-#=;kCX7&hF!`1JyUOTN4O_&=@ciAN=Z)vwPxA22L#qPg1yc`{udM|hv+mQcm5R; zPp^iOWrU{lt<#!lUIkQYe=W4Pe=d;I~y|w8xYC zYu5D*>*C}m)7qe~UogV$mzkrAHsAl;-q43A(YgcTArJSVGaG;YZGil{7!!Z2#{VPW z_&44l@XOQK{n-Acf5aU8|HK=3fSWC&f{meznZ1*(h^d3Aor$TPi?XYcvx}vRtBa+* z-Ctn=C0TnkW@I0^83swP*H5rp=`+GB!VNS+#DXERl4QhI3aFx`Tp|-@N;++Bdw=5kw~ObF`C*3*gA0w1d|wL0V~f)tuagzoDq$Y{tkroTIwrI2$A_ zG9%bAtZH1*rHspyvwCt|#CPw1$;@Jb&U5MG4+VfXB-mmSaNb0v13Y$bJB#So@d*kG^|D+#U;Y)PKzC5*`rCl*3%F5J3x~>HYeX zN2uPVezyJkQn~|ax1jLVzyFo8Uv#qd{kcG+b9ypD9VSwx6rf1Uy-DI8gi^mdOV|J^ ztc^>=XQ+oUdcC(hT1Te7CdmxFM$JKTI5TUHasFu>h@sU{2-8hyGWHF3fEbPV=FvDY zjWM7)z}};()B7hS&xV(C*gd)~8V6#hXF`4TsV5xu{6y$-JgXJ zGZF4L6-%=N0Y|bvC+rsaI^#98lFrk=`^f+Kko<4S{DcRj5K&eR|;SA%}+|L?ov~!z_l~FVW{r1u^Q10bB05 ziOhzF(RTMk&H`FcQBzDJhm5emc+H6E>@V0JQnDTOZ7^wy>GG0Cv(MOHWq{x)#h%}e zTV!j#DGbB4ncu5M7IF>;YZjVsRnqBozsHc{==UULU6R52=3;z8EaMSEU3d#T@APQIsqCNs}m z==8d-EvD<-zS5j*#hAoQXBleFu^FVliNM_nZ&Q_W2O(7)_{{ULhYYa2=^t(O%ZGL* zJ(^ULC1nmX9=_wJWM2`MP&>kb(PiHpms(l)-+vW3>CQ&pCSpRH?=+(0`5sEmiqTH= z6tyJX=X}#wG$EbOI5sC5Cz5fSX9_&4p$2XBD zKkfZat`xyV<_v2-5vu4~!fmV5PP1i@&R2Lfqakbh^cc>6!m7k-=O4nz#0A z^)}2HAT!`&a^7MmD;X?`j7`5BRe^=W#;M)F6ctJ)7K1X>Uic7gr6l0k&cmuCm-yf) zQ^OYZl#Au0%aMJSf^X(jM>*WngXhi~N;jd!73l(sl7SU+WbUSHVE0UBJY`2inl+Rb zR8#mW^=N4it@8xLIqKA2BCgABeUX8Sn3j_UR!F)_blh*v1kfa0Wg6yk0`V)c7d7ve z_qqafheXxNKzDM)`W+L!@7?azAy|Z;EVhJoq<{LJz%N@IMN($y)M7Sq2P`Ak8pa0V zJ^a3W4XD@9LJTW9K9@H2^b`v4SY?5}B`Aq0GT-8$hzOYFFsq|iY9(AWF%aVbZ4P|^ zga~TY%ZGdeEwck#HJBB2GL^wfxfbZ?47mfJHHzZC*jODM)|UU0;8(28=22?o`#z|) zTFO%4K&vA*oVd$#k(#`;2C`+CmnUJr66HBnN<_sNlR;yt>Cp`X^Cs?Go_zHFm=o#> z$*$?7ttVQsG(fr3N(4S}-hgkrsJdhaEQ{#4;j2rJr{{)RuSTFi_jzOqqf&pjzix@> z>Iu*!c(Z?eH@6@i<9VQ(Cm#2MarMT(bT$OLS{PLdnn|Y=I(SFM|3ki}8%Y%Q{ zIjy0K&8TUtiFM8y5A>EjYk)uX9R(}OL34A(`1dffTP>2;T=MQ?R(Im38)h7Fz`YQD ze--6AmVe4d8^brPin~pIkkFe3Tz~T!tVi7YVKUT&sh0l*>ERGMTkHyIyT%?lJWZme zLt@D;-$p+_>)c5FTYQV&7#DVl+qZBsz*W8?J2z7Tjy<$|B8dp=sV9#Sz-w1H`bs5XOyZhDA z$g5CcaFbRwI`Ie*I)=q8EZRMKIrs9OWz!Uk4Da%75ze#L+qZ1D^boXuf7SW<^+3&h zuwkN7BHz01blQ~hzQxTR>4NT@3W3$rlh?EL?tHleNbQwFxEHrkvGK~KrcIkPk)g$6 zaVp6XVVg5VGNd&h7#~XtNR2p?8jQ$DwOov8YfnERZNzWCLr^&w8P-O4vpl~kW;Hjt zx3&n8L9{JQu5#}~V~x~cF-!%oPVcGXUM@5=8^C8lRqF}6klCxyNUEiCPH?YkMkw>( z&SVY-*yrFAavPp7^U+CVN!U*;73}e)W*4&*2SGdj0y)jKEFHLGjMD#rLZW1Kx;)|`|6c~Jw2ze zJd{G9Rz5iP2i@@-M9rq#zzr0GGXP$<(%@g3WNe{-biS+EILcEd7jU zxn@0Fi*Kk&&i!aWxPCCCKV{c%sIO&;q0q%ZnSdiV9x(>J#87!qi4IMo!2&yDBFoA< z1$t(Db$U9mLhle-U2ujX#$^MQOqtw*%(TB`7exfTM4X+jvzgb9PfW$wEaCU%C;!K4 z06Bx?e4z%6xBYh%A;ArkKJrSzR>ukrrrm9wiS8ga$@Xpt#?cMeoqG+QJX7cRG~4}n zifz1a@{rpypG+Ea&Rb)k`Np(D`jA^>AsP<1rtvZ>f^+5Y1;JRpScD%LbuSuR;m>9~ zd1XZkdrJ*AW`?~>)DQqqGL?Jn*kZmO>xPfx z)5WP1@=hh&(gXsr zE4}Wa5cUDkme6XBpOpe}+)VTbXn*XKy=2AY-jrswRze(c$Bd^6+qKD&()CKg$hy1; zDIFZR`)8jkHB=>;==1p0vR-@-1@t8`J4KN|w~Q(T=os27ax7sB(btyyzi>t&zm`YV zMtY+h@6C@GvtMp>U$i~YuNUm8cUGJXg>6VPwbG6j)D3eFS&OK-XrIU{zUQyz)k|ciWVCM zCPs0tN-?ifD>6 zwO)U6whb0>DOHW$_(bUCQ;TetZu{oMxhu{W-Qzo~vD=A+e!a1gKRCl}g6r`i+F}fh zs!L0$VUCxgN{XaQ<|e3dfJ*x{u_Nj3O2f;P@0_uhIiz;@;hT7*i{;sw4QLs$~_3K-%e5<;>9ek4ewB^8qKRm$*7Ud7-F0E< zJssnzEjoj>6$kDpAQWZiH!@%FIPB|$vG-Xq@{$J045e@J!Qtvnu`<;3PrbgyJCBMe zVXSlNBaa|87#?Y=mo14=(?j}h(3a=u+>T;9ZIJd>`9#%m#w>;sClpnfOf%X{#w{hr z-U7XxZS^hXsx8DnEfp!5Q^1z9@clIt?^@hbMZNM{hKGC}!1%kV$(J%Z(eD~-Bfl@~Ya?*Q16^)Vr22$XlH2I{ z=*J9a*qz@os4getW%~()*eEk^cQ|UG!_-!7^A;aDIw|8FEq;G{?5sp4SZ7}_%H}O- z5RpJQ%7z~(&~Xutbx9gkr-pH91QM(@u~meak!!$s;{h@j#>8l1>y98$R{^K4@ePZw z{@f?%0nB|qIHgR^!0#z!*co1W-qA*t(suNVgFC5RN<~7~Tdqah!bk|~qk{#$y!I!y6DAG1+4 zb6dHR9P8ln6XJD$WYuYDw&FNqA$?e6InuHj0deKud-ldp77)u;2{hVYlrAmh|2!=( z^-;#%X>(=wvce#yWNal_BdVe%{f_ycHcZ_AMkfC^E^1WkC;tEXFd@$23jPzxFmyEA zIDT}dsKNb{{XtJQz(GT36nsL>`mh8GSd!27%# zWJ5+46Y~i*F7=aS*C&wJm;h7>P*hM*C4x8@ctlpV)M^V_hI*t5rEj7i1|w1xb5Nj? z$fnd7K0fmd4XWCylI# zs_t#BW$wH{ttlC`>eg3K)3{qZ^l??Kd4$fglBU`w6wF({EzFp*nophr4IPObw!{lY z!^=ZsDh~Xm9JMT_eX#73Lp^M!F+5Gh%d<;cnMAjZo-4My22x?T1~i0pKxzqxQZSQG zSY!}_p=WdC$9emh*pZAH;Bclz(znZX^8iX>@71-FO-J!yaLeQKQ;)$nRjl4C5YJuP zy3t*`ngY9c(bAMtMfba@kM8Wu^1w2=ItP74sh+#3?Adm~O=?q=bL z5-qLP0L!3#?JIaMq0M^0)G4uz?yA9qd3ihY;O0p?owLDHC@x$UtHn+2{SCXUB-O{A zpQC}DU0r>VgPpm6QqDli&aS-Ds=TtIswz!eA5OH(vxW`N?)LO5GaHWj2bFyW-3n@wH5`qy zge_GnlPz>41r}Of2v<*e~O2wp2@PB0?|Bc`|C4!d|fY|)v=A`m776P z7{3XDuSS5BQh*ddt!$=mqE^Zl&fqA-@|#8Ax^3d(zVgpQy3pnRGCloztm6Y zWFMl7*0eDI3)}0U?Sxe9lh-Kd{Yr|?_=Q=>#B28b9}VOzDn11B zX%_2he9)FiJS1*>ns~Ni1Glc&LGm0)R$4eHJ9;QSr7jpA_`CUxpe(n{-C~Q~4_ONf z4yC4AJc3TI87bS=(^FFJ!<&3#gcJ8}EhxF-wlKj!I?L^sEtQp{Q6WS9uX^~)R(NNE zW>a*zGxT#bL*9whqGfzyONvJvitOmi!}U%f_!Ca|oq|xYXr^(PmMt#D+k;lfh|@at z#~TJ5ypg6fcr2!PYrXYW#iK#_HHK&F2;H*lAuEM_h{RcLmgiT3J*>r}+bA*qN4dFQ zxi1cZ;?LyOLS6@AFldpueS7`d#;Mh&TnL|? zrGgFwy-SJ@SgiXh+qi(ju%X-Bg8!(_Y_|A%1)3-~{Vl^KPA(jm5nL3v2Ay7m8uilI zMeK0XqXw4kM`kq}W7o)pFO%n+2iHu)Cv7WX-7gA&?VsF%G>SyM+za079i*SanKu5D zzy(&5jYtNeFS+%~3Pjj*O}rlFLto1Ro)x@ZlzJlwvhy#gD_AqDJvpWtB_=~H+GE8n z55bVHhgi2QRQ&co1@Ea%n{1$;MYu1kGHweu>pEu`tsm?&iplI@stb0E$fSF()~9C< z?G=|f-5YSv6&4%vS&?I~si;fWzqUnH0DhQ>ZnM2cOBbdIL4854wz+TN4j%e9djdb* zLWdnI%V!L2&$6pIBI;eR2iQuHx-hyefSk^-L?T%FMa!O7-A^^Z?FOC&S^fH^@#*g^ zG~c@Ai=#t}Tbs+Q4H3vgUh!Y>a)WtwZp@zC@#_@4KPxvjl=Z}?L z8~QLcDemc5$>x@1a8AJt=?Kiw0u6fg(<_7ruNfD+pt0e>{h*%;84+7%`Sj0l4TJz_ zD31ZW_TWrLAy13=*;Ad6bB3g~y^bj!HhFq=#^qhx>;oh_F%YC@X_!R~q@R_=YnoiDC*IEKDC}wjpN48yK^()YV2ojtYidu)E(9%||l&>r$j3 zFEg9=watjL&hp_qRM?EOII7HC`ko8?zjaF_gl6E$t?KkULrBjf9G zcEC-O@Et7xyGO*rhoqT20C-3~(YU{P46QWjS)k6&HYpZdsZiOpXcW}cc3h)E`|#)W zu}m566ny&TSQu2UJZZnqC8d;P+NLw))NJ7y793Gd0$t_e#9D<4G50M}eQ&iH>?1Uv@_Q#fuR>gsjU&-VNxDZOkscdd~vuwdu0zhq3SNT3Rbst+1 zxi~84-dyxWfp)E3)l@(VM1LF)c|g?<)Shb)%jUy>OTrz6N}~>A5B5tl!I0E~{bo3J zJu#FbL1))l?$EY})T%IEo@^tGOz=$AwVBrLu525`rJd3bu`mVLs@Upw0*{CAdacWQ z@?zbeN}GIWJlu))fX&_E#DBvTI|SsPL1#SZ$J2^v7L+C zq*Cv2zNF2n-{K%pb_Zt%6MS{)`Yg@M^?FOO=BJ|B)_Xvzorbu;x;U{Mw);* zTT#*@nsnX8h|M#A&)SJw(Fh9ZuOJHUhM#eSW6~k-c|Q#B6PCS@y~B#~oz>Na63Gh|_13HUM*Y8arJOp8n3ZL?n)Z_aI z7%iK2ps0+fdJ7jgy5*%ENM%^Nyl%TSDF{;#sbX|u4HMQb!5s$_IPxVHd;XZeNTCXh zSFxDN9{U#;?+b{Z-3MJghUC6E+@HJz9ZCTuN=ZQu>n5es6CAKY-9Mrx*zvq`K;^$a zn6wE}&;$D6t2M;CM@^neej*PL-(8v^2u^r}Au2wcY?mK7FF~rf43!>j5fs`;^-zsq zN@et4LgJf1aBX$LXncsHRUW=1uFsxNz^c4g@o953c!r##;=RO{9O*xfbpHed)`$<^ zH4AuuWxkcp{8|3JF*WSIJKQf?P*4m#k~A|UkqU4qVd79PijDLSCJbGoGuvd(Z!hXg z%M+k_W*%66oo$#eWa8l`9&Y&*Bl;79kiC%#B>ZmW_3p+PQk9w7tSFZXbF*3!rqjBb zmZR{PeAa(aMS+k3h^{KrwhkFAV$BY~KRWfO89S6*IHQ^Q3GR@>_g5fd= z4K^TXa@YO;$oj_MOro~U*tTukwrx&qo6p3yZQI7g6Wg}!WRlIh`&NCmyH(xQRo$n~ zpL4qFy6?W=t?!gmSxjT4KrSu~sS_<{c^kaEmb@pWg6j($FmislNoN+U`d6*-Dj6{g zqC4y9B|z?v7$_VY(g(*Ii2q{&O;NHQWsT`a;HyzOo#ufj4tjJtN1oAo0bwQ*zs*`=bWmA*8z0;KHkBgu9>t)s`3u`A5? zRBpMm(gKJNq-|LE+^8$jX{&Dxj%*3uph^S2@HFMUkkmrsH<#bI-)4}>g|PY6H(8s;xP|Kl` zhpMZ_f+@9*&MDd^at!80k0`Cqkt?~vm6KR7^0e3CG9_3|i>j{r$GJAE@=N)%V(Etj z289yJb0-&ttQNyBz-X2|Y{1Bwcc?u82kH83G75HSRO{1V>+<<)atJ!1Cn;=6Xwo0F zSm?g79vxk0^ZKSv_*@d|w6V%hk@l6+c`M?udismA;%ROR^Qh^Tu_*T4#G9TK67ub1 zO=lxb=6q50jft1dFm$yVBe25ZIhlURBDby440B$(5F&xe>i4aJ3QZa&E;a*zu4e7L30hU>%v8X(nRi5$^OnSP) zc-#|E)8rB3r@G@NOaD3DV*oC)EJ)5tI#)OtW7qlcagjraJSa=%%_nzA%ryRHd&kb( zH4n2v=D^yqYi(4+=9D!!hq7$#-YDZ73y4pDt^QqD`+0p2*8WBrk5XFzpdV_JY$l4m>6UjWJ zrZ3XY1f`5dtO7`sMyM&?wja-C1tpq62iq1u?Bg9Pw?}x1-FeSQdf2|0qoB zj%v~2caUSB3CYV|5_%dAABRvKUv)ha;I&U|YC+R;!FtxYD~b27=`V#@^ADr&(-y1d`^fj7u)e+CInu z8Wp)d1q8l@aRn}Hc<1Nf_(^x+9dVzHlV7I&`*}ge*B=2nK~En| zTo6sfUpLmGkLOfAn1yJl{GfAjCT*x%PADd2zit-lU!Yf?xmQ2DDk3XPu{(2$CcCo; zB(9LmzaTwP%?*mVxhnM#`8%UjV&`(vvAnCZo}qWXQvo*O|2&40A*nE#Z|yb-`k#=R z7l~@gDW9Kuk5Y5@miY$3AQ{reu_Rj@RR;P(?s0NhG06`n#g+fsaFpymd?@{mQkq7* zyOI>L6%-!$z%K6ZD@2FpL7G)HHE!Ad$D<+6Eeu`hR%q`ifNHKRxG6_{M zn8W)EBH5UY)e?orzqlYinomf3zomuc5!bDd;MOQ+P}aY=p9$=;va17HvozG64Liv% zIwNygujkQ$Hu(;LN70Ey7dL-|@(Mf0JA33%FahvT&hn3Aw`iF72(K6X9fjCG8kT>{ z>3C0hd53xXK%BDwfu?7%>rK6BM(hWN`wtcCT(x3H&zVbr7}Dl_OBM2J-3?zCU%7IN z9wyqgLtUxMv4d=7=#$~Swj;|R{8t7DwOS&wr1ay?BR{nq%0EQCKWO9zD!yK3sza5| z6PSP}dhkVb*Cc(#IYd;>B^H&&1}m1&OCRP;x$;BXxV3Yp;suyDO=b#btx}2{Xj<3UDgkf=kmC3V#M^iEyBZVfpA|nbG_`j^EWHZrW@qE#VTp@|v@;Kmh49#cJ zJ>{(KMYYe>^$27(G0fQ&yJm{XMeqyUtkMAT5bg;XB}EMo!L?oTu+Y15CP8={K|%3T zYmT*2(5^X%eXrm1zpo7Z(3DV-ZkQWyfj#)HvF&zqv%xl5c?c};wPvv-N#97uk3|0d z?B%=!M~xm(H45W1n=#_d#A|WiaJzAEr0YUMfjZHBlS8ma0BYvD8)pywoD)spCjj6k z^=ms1NO%I?EoI?mf4`A0axQ^n%8~lW2|MP%+4&AFa|lzFS}_R@G+hxz=}sRsdinI% zA_E$v9KoGQOea5a**a>s2Zn&dS)B1`h*I^yiI0S~cCo5lAj%9wSmf6!50WGsb%3Qx zAm&uiKY_4AmcAL=f5Ke4qqZ+6useWS;0<=;E^*~7Hkf@Uo%6!Jf}5xvBPJak7~(Gx zz!KFx+$D|3g~ptE>GWA?oF4yT)HI8m0w!%OL+f*WWZlyM13Y zjA?p!f|^}%sTc=>7nby>RK&7izPvm=gs>czrgNyo?wR8qn>8P_iFTlbT0g+5;#FfS z4t8vY6locbu2&;87lr(+MmdqAuoTHqLBT-ytUIyM6CM558fPZ245c@*^iG{Ei(hTj z#*(>h_M{_4XI1v1BL<{9mE(@5Gu6+P)KI?7QAk1VC@gn+Pryg%+tj59(yW8uh2p=KqZ9=iXpdSaK@7)7yIXLVV{J1Z;r|v z0?r=~67H>%upzSQZu)AP*0bSS1=W<=(nbOX2dcCED2o&R;f3e<^hCY~b$z|F^J7_= zd&j1@MB1r)1fsM24=l6l{w8jIJr||iI&@X+@Kj)rWNaCX5ElHk1`KIo;7Z)Mp?La` z8dfyJB*#Jx%#v3bF>b(JkJ6_fmE4QyTNlwLsM26iers0b8?xy(zgp;ptDeREiWP>S z%)=YBRenO@3ms{b*s`h}Uxpdz{wvf>_I_A4kLx5jB?u3L>)CrcgddwiumH_uhIvK8 z9V+XT3i*IH?43A`nO2E-7q|xBDUXB|?I!tFx0L1t7tOgS{V4!UsY$j(M-GZFO)aYY z0rm-0ZWCLep6L%H2F}lA9Fj&T=@m-F2=m;N)65YuX1|C^X|y~%FC;ToRDNb2PS4^F zx-(dD0p!b}qLfcw9;2cNVy+^D{V2GU#GcGQv7$@4im#0vM%vXcozdrab~{@EBQ3Yz zteL<-T14Fw13+LvKH+P@FEYzxjw$VfjML)*#*}$;ermQ{0TL`xkD?kw%@~Sn4^T!f z0>Efx0ma2k`mG*vPZQtXzPuo#qX^Pao3iqR7QZ6pKM9$RdNL>yK)i)WImc&5H;yQ( z?9UZ9PxKIFdQ8|436RS8G!aTpqs{|}VBRSfY-*kw1N>8tZ)f&=>Qwprgylae^r!nu z)ju)xhaHmAb`r+jk^4P(geEXbpmDCz9fjX1*L&Hr+zXe>1Dq^W{@?Tp+Y z5-Vz~wmFUkJZn*`Hruoy@JXHu%!>*_se%$(rP|~>a0#`USSLYMlCWjkpYJOkTgJ<% zAN=`xGrgKd9&=9oNIlXlzBT%w6sgf{*@DuSePe4W7(8!Ktg){Sal~OjQq{6 z`LaUmx-eL3NvSmEkmL~2T2M5r9c3^1laEmI`tVup2~}iD)a?-Px~{-$@)H4lsyVA0 z;R7V`@~bn=*yO=@o4enu*Okl+3|_O*syI>9Bsrm8fg#(h2(aTlH~E^##`AO5hmB! zU*!gH&?xH~HG4IM4ON--DL8=UI=PTl%!6ycAm^|z8e;Iwo;rdr2Z7V;y|8^z=HcJ+ z(_r5+5$h7L{b#1i=JBEx!KP`m046hcuqGZmmd)HER-IewI^qXl>pAUzmy_LsJ5De5 zYn)r1f)1Ew&wfLzgFa;$4`tf&=5%ksE)2QaNbBKqDhXH!?&`2OMugU%rls zX3ySb+D+BJx;e=xMieXMV&8L{C0uM$@9z53@m0hL$5 zm_svTR;NmvgEOO^(#WjP(3XHrA)?pdm}l9nQ(`Dv8kd2gvMJZMDo`*(x6`Q2w@F!Y z&2c8*`hCY};tN^J>(nD^RU(PT$Y0t#caZ-;c8WLR*VY)TKrP@c4A53&H2O4)?hE}` zar(x%-cesK`39$++ME1L4$YVE9dGJ^UKBZeb(CI4AeG(1$5$!opTEWciA|4@b?kz#9p#x zTT+I_;d2M}g`i9J>;Fn);E@&NLw6AGer8c56xKXlGd;CecE{GWqS}fob zTcJW+up2j-za0GQ9ILkafSPH0P&R7?mlY2R_x{s%Yl5Olkk%0k`ZpG^Pp4d%@*&nn z)VV-7J?$eR12ABeVYdQE?B&f06A<<%)6z_FA0MD&sD`jFWLmAs7CBF5EIS*`sD3t& zv9yA2)Cm1`2rE0`hcyX*D}~W5g}3zXZXL)qNuyP9q04B!;+Sq2w3c_iVsxepWan#c zM-=Nhu?teA2Q5YOT!2tD9p~lEUZp2)PpK7Wd?wvo1I!`-VgQpMRmb!6pNY27L2NO3 zpje!wh$(jyOg+!m^x01@lI+wV>k~b*z~?@D)3tLzYKRM5CbLi$G3@)2?Bi|9__<=s zTtH0(fzolaP$f~$DfXGQc;Ju+*<^XnnStXT3Br~pvr%~d5ImaB}`9yW^6Oe@AZ*F&zu zi9fEIu|-4^(X*TOVN?Ft8|X($nbA~xPG*+q0E6USHHbJqvYraLkyMf!G24uomB97! zJSE{0Ej`k!Wxd5sg9XN~eDOIaQyz_=B3ZRg9}5D&%>te}S7K_j393-v|ZzO#$c} zKyiK;`c)DabOpPs3hmR-JwtVpjy@%_)egwmD?T}`LlIODsNS1yrSdj>n;QJs5c(>E zY5vyj*^N{h-#@(T`B;9%xvZ{2CNs8roQi*Av@n8s@ZU_NKC!PmTZT;GGYbHllw%&z zLB0DjO1}j;+59)*bo0o3>aV^sg$JD;fYX$FN8Ej#*g^&ht)urG#jv z9p1I+!}tA-v3H5f*mTeAE+y26qE&E2VU{v`OC-WW7c^y={^JQO7k3Fr{06+U3`j>4 zrWk|DFX7jq-q*t&L#n)+r(w#z@3lAI@bCNwZq9qy8ip^T+(z~Q=lz>Tu%ed-K;C=d zOB!s6hsDmPpJ%~i?M+r+pCJ}M?{|>J#Ull7=-II>!uSxn8P=CnVwC=8gA<+_woz#q zri8|YVBA66BPky3_X5{I#4LAMVg6hPToH*4x1N2*y$?yT-u>wdYUFCX0`6^c6d5besB?Q2_;+dg;eXVUNG_JWPv01%Omm)%E%}xb9%D+p!OG)VvS&0f)FKk({TwTj$ z=h*281KS}Z?>qrRsfh&{r;x~_K0Z34T_s@T5z)XLRY>wl@_YL}qMg6YD0EuMWVLcz zynLTb_q_DIbj5w!=X#y|9RxbRr3~l@MQ3#zT?3#`a7nPVYRJBvM${7ME4t%QZ{;qS zt!=pJh9FUGmR`*oOHWdje)da)=HyP!>P;Xdsr-hlbD=Vswjx^`&;6%pg zTBnCDP%(8m(;ZK6hB9HR-WUAp{9UIapg259^hrfo&n;VG8aC8L60S{!Xhb4YyohH- zE|1!78{O2sSm-CfYm%E_fZsJAo2%l~Aq2>+$Vff(sIh`CfLRh2a|#hsijs9bSzKC# zp2S{g(V)f`l}CCj$EVhJ5$;!NC(xAH3(a=yK{H@&y$weirDx4;u-kGqme&|@&8!h< zl6TqXmle_a9diF$ob40@Obw~)^OcD{xXLs?H823-#VWLEy>aD?H%H-E4r+qj76zaZ z%mi)5q9o7F`18Gn(h<=LS(+bUuXbSU$YNuyrA(T1qVM6y7l~pM5w=nA3sUPdvVG!= zO<}S~VJEf`?P9ac=|>w~o_y5UB4B7$pq&ses{M=OR?`pWXHjQ#(GmEl**Z>kBV>U@ zY&(|0Z!ZP{zjWk2u{=*h?dyJ!gFO|u#|I-1mJesW7iuoSnMiXj;dobfDA|6VoL*j%$MZF%D1HKxdf5?i$yJo7p%GKuCCb=~nG_jmA+afm46p?uj zw}bzLSzy)v^g(pJN{J-JD+jxx9{N?ZO0U0_1p8M6JMqmVnt zGZRT5;mXT{RjW03-ZLS6YX#77M5d*lmzYBqt=LrQ9N%h#z`I~!WiCS+9WbuZcIQvv z6ZDGLOSD9ZQdT!F%bJG$M((R^e#pOwxsrG%mpIGR3G5}x^7&>pdTz&z+8Is8YG*v% zKTWW4J`qce$P&e8VPjL=W;j)PEDFcp)8c+g%txmyiDgJhZS@mvR{Md(_9VJ*Cg zzUYZH%198P$9F1r*?h*5=HGWa4abPEfE9On!{a#T;~SB=DAEZy9g945w9RZNzS6&S zb^_q-B!vkIYnd5(YXSriyto@L;&aZ-JT>66Dvgh`>*VXP+wbiBdZ{YXJ zyMc25Fk@8B+3B=SmP&2B{mU-{H{hI!r2?;I>2R<>!N_s)G6j_N%;ObA_{DC8dC$}M zPk~INQ_~vRvXh15H(rTNpAQ%1ih2_CLGN>S6V+Q!n)7K?EHV6z^;tO7fAax0vxxcJ zuV=HdRUX(8-esdSL4^V&rR=I~nm6!pTDM-PmVG9YGWqOx%2~tKozt25h7}FBwb}G+ zySTDBp5R6X;R7^AVlS%Tsj;n@<+to*tg|~8$_ohn>z(&R8?u!;02zPt^OFe?M1+hA zvOT0#r3)?&xu(k~WT^CHi_{^RNxh*GPG6X97KkSW!aE@K(Ccn_#oSr;_}il|s#s=< z)44)@@91{ajm<7fkS^z_GhKSvzT!?4=_}CpdQ0+jYJ3sMK)#9!LQ9fCgusqH{15yF*{61 z<2PsqiEVDODqTB7WyN0jKk#G0Q@5{SJ2|l<3 zqBIu`y?VMsK8AKy*}r3NA_2QjcZzTkfIX`_2LK-O_aEgu!|su$QWD0*zDImH^#<{H zjyE!j3aC7xgn?Im7anWo5xTt3w%1ez+OILuRc4sNXKT#DsU^_%!9GR%6?Me@TDlcV zRoW_A(?eIpuR$Lb$UE#bws1xUC^?uzV;;k&beW~jVK3GD^1JP=K;;hFuzMYfGt+5b z6aasSAO?;TVZ)OVD9I~~2k^AjaC;uJk`Xfr$_=0M^#EwGbhiU`z>Be5ewf3<%QJ5} zoq|y(iBj#WV+M+l+TTGCazag7@YQ_BFI9?um4Tr*yUl8}4lD(+T{1K$l!XYrTQD8P zSs?N0Zv%LK=>=asOf&wE_44k_!R~hnK|pD%V5|c?vraoCYCFAv5`S=zz+)bd_wUGY ztwh#_8BwBvmh2GT-=ch#NG^&N^M%N{;@TlfGRoz0A`vXuX5(yfswC+Za%Ewxr(1Gt zROnjelg|9qg?npYS~bQy&~k~={e*G`D01YHIe4G_=LUxObX ziu{ucD@93tB%ypKb;TK3Si5<9CvdlHrDu$rx0;Q*|K{NZ?H+$YnKWs7)^#3XQxK^` z(LO+l!KKL18_VE}>cx@EfYz$u%mIq&2E9w5bH;UCsi#(pt3}!xaW+ygFpE*5c_Zq- z?bP1>6ocOj|7x&YFNhj}p$2s}6@>eQWzx<{7(Bqx$zCDbOiLMFfeHR|lj>)ttsi#s|45nt zGkmMooj&dA54JBDByo6_$dO& zCqS&Invb?=Dv44$e+O3@1cRZ0k^F8?4(sLaDjND_+j&QFl;tkZE4u^mR9$=KoSJJ}Vv3vo#w4DV;?Pp3D7-`HF#1eZ8kGQ zM=z%o1U)ti|E;jKoK`!JUb0|O*4~uIXleux)W?wKXt8aJ7<}nD(@g7#5=p|_2pO&e z`HGNF3U5`Xcsif!a+o=4Ic#i`E4I_kI2Pzgjp3gJyJf=44_*TRxs>0+zG!6&aic=F zuua|8J1HPzzP|_1xyghX$}JY22JXY#uuHon8t7YzNAz1OR=&yJ)$s3Dbxl7 z{$6LrT6mXmnt2*VU5UD7jq9(5qHqnr-?R?AD^_`7n5nznLk&& zz}YgDxJ`})kWCE|?l%1C5r8_2ecad5KsSQo16`1}R=Nv*-#y;OE-F;agDwKSz#HgG zl>k4(*fg(O&`%|IL6p@vvlmowP0#qayiO->|I%FRIcF<@vg{M_E^z4-)+n)xV-C_( z)h)N=Evbqc>Pogp6Ay9nRfWi&O)d7aD&9 zIIf}5>}E>?e0prrBKu1PZ<-ez!~xGns!C=~2bkQP859%q8}hBBNcE3QAFetm4Ne0R zVTzYL3$(<8Xn4ee<_rr_7BsxNI<8R;<@*J@zkfRg8B`n{Ir`iGI3WLPP3nCHTFJ+- zrnj+fnuHw&kO11aX@Yj~7u(kourX)aSlA;J**Jn(1uT@-nyS^K<}&NU5gu&mYAW$S zD5g%CcpiyXfcA#=h8i}aj6H^cb@bkm0tm^2P6h7-GUC7KrQF97VJT-8iQ!`URr%1#*58p;7- zYRLz3=w0=?e+Fs3e)9(h{>fJ#`bbP_7cr{R3XxRfixjMUO~T7+NOM!a!y>S-Q|Aky zRR7dwQZ$N-EW3lFt=(tz)cDcO84Sa|Rz+gobqA(FmEq|Yl)q-(dSTZgQ4lnUsCP%M zh+V9xcLM@A#c5Fc$R%=JUByd{vA4|)DHMIsV?1?DyNmYqnO|=_6^B1w!y*yI*26qC z{s^jn!V*@jhQ*J9_I9wO*I?hIqg!@2YKRRer~2hj+aNFqB@KpK)4qNS*aU(7+D zOP8>@b9B^embj{@jrS*Q!o*f|(Hv_0B}K9Rb^z#W7s!d2vACm88Ewpex+e=~Zz)-R zP9>ya@of$zRAxB7pI8$GqsFSeh8ak!oCh~Z*CNTp65GC+@}Viah6Eyfwgax5@goreKKb1zhxNXj(d@b-a(b?GmJQM4XzDbzFX zPj6kg`Nso7zX`=SuO7Q^$SH?vJx|NHSTe! z1{h*`Er;|u;lz_}4b>TFYkKYC{PX2zw>HNa2OSm-^ffiDAi&o_OQNTT~hET>l4#rslP#N=qE*$N4b;MQY z6Gs*>Zs;b+4DOwPU}j+c$v{xhAc~oOcMH$pH}XAMgoU>*Z;s!|*&rE~0f2103@7Zw z@je)}K&9cXruYGXkh%03hmy1vqe?3j_VI1zbefQ6QRa(D;fhj_PX8J-?}!P(v1CRG zeK0JIDqkCV#TFG^2PB%X4xwJKz3H{k`@I3xKV`=e5HE$8{vKd#a%9g|vAv8Z)sOz7 zT*jK@1np?O_3ub#rnAC~6jVO`4(TjwCcN5*Y{&ih_Y=KU=3G~GWX${!~-R+>LsXhzZkC@k^ z(4XmJaR%9^6Q_UZogL;OVC0SU zdXj&)#%Se@auAh&g`9HAqiw;nmxj)$(#)!6XCb&K`>KPPO%$F-s>ddlT;`sUS8Uu4 zTWv=RWM_ixjkonJ`Z;PM-up&!`eAM=dl?L$7%!eJyfX#WPhPBpyJfB{g|Vhr2b10& z5tzQQWB z!B2EWqo-`k%aV#p-2BoO3#x2sxurbbp{cV?t_1v&#NGkRbF!|;J#uODw65Si5}NZD zwhvv+3ETM>r}CW{+l3ov_?=1HMKzBCPjisXNdl%7gbl@hbZ4Z;opId$my5IJ(E}8< z**tW=KM5~Q7Xc>7o1fI_k;9fk5D53R$nuV=3^4UzvU>$6p<+fZ%t3uoy#w`bdU48; z=xVV?db=?CVYb;(bw2`}&^z|(-S;D%Xid7C=|P#=#lM$-)o0stC%7r5Zkr~pJ7IuN z@wSFtv31g;U~A->5xfo%e^m#z&S-7Gfrkb@ycPdy2?XQ^x`zaT-;Q>?QByQoKA;vK z5(vJjqCFIlegoa3=@B(w5B*R>Z^L%lWdsO+k6-u~UrYYc+wNHFX1dfW956f>$5`7-pL?H7R-(ak?pMclYUxD2tbqx*I_tzTdjRA+#JLPA%amwdhXxoVN4F zk1P&*r6IZ))^OlUh>7csAscW>RXt+Eo2qr&B_!yCW&@nyK@Tc2$ZaCIWF!D1&vgV6 zO_&mh+LaSmw9Y(_mc9?p?i9jh1rk5GfIKi3C0e+diy|UhSL{iRL)G3RccS4e*D}g| z#BR*QlJ!_V0(URG(|@y;=i}JRl1M$Ra^TobG*&eia+fd4I6lVwvjOIcWIF4|KG}(W z6g^Bozeb*P%KZ;{>zViRqp1Oq9*!#_Tf(67kg)A1zmR1gK_wEHM8zq0L-p)CaC_|b zsLvM~Dw1R}UaYRZ2k~Rp{2*RzuS$Cva-sW~@L{h#q#z{OyV2*%AXw`a)lBHRdsr*(BI1hoE`Gum&;x3U>y4AK@7Q@StP#|S0ux7>oQnm~>me8>8s zDAXj!%b#Z>EJ&u7$|Ix`1ZA${+~2~RhDyd==|)Z%gMa4+u^>uWZ?GGID0-Oty%K5G zWFDWnD9Sr8Kj8nfNpSx6R`@9ui~qmMvdudloCgBoKSd_dbH;)i3=j|;ecG-YDrQ=} zD-TiHbTd30phgQ?5A6i~8<7Z>D-uV1xCli)c(`-KGhd08NCE{342)fRsveH1{xW_` zdzS4iVHqc@^jXS#h`?$gvy`2l*}P?)tW{d|7$;oZ6W zy7}II*?#e3o%wh3nFO?a&?mkciHL3lJMwxq2c!VN+-J_F9y*Y6e$mD_iDZH$JT80S zy8sdNBl}I+{vmD8hu$y#1;`8&)R9sogxx&|Nsm)R+r#F=?WY8NRYS(uiikUGJsQ77rs(rXP3tSdUuqOFvwv{;5iKNwmHkc=4HypJpeHWPy;hoeWa)(Eu{ zG3r{zRF4*me!Z3vFuTRI=r*gfn2G`po%O|P2J+F zpy-?`fm*`n@Kjex@?8=wU@^KFpDe8g5UIAt6eCKXVkFmDmNNEM@vrCehk<4=x!OFz zBrhH$-kH^__@phb?1?glo(=ue9F^Voh2Z8)Vfuh)F8d#qdi`@!nFyQ+qt@LaFDo*dT zUiRbT_T|HFGF$EF@H@soGFWq>usB`VT`Pvr84DrB1v@Sx(x@aG50BU-`T8v~ND1P% zoQHy}i{kL^0*~DE7aBBRk|wxin->i{z4TXHtMS+9jd|owTE2KAJ)4zEs937VBFZ?0>72MZmn|bbXJw z_CS3Jce-rgx4hVWlBhQSgh!$rl|~5fD6#pK%-Ht5u>m87rw{^8taS@d;gw=$pDC0c zMuf_b_dJvb7)UexmPyKSebVj6jddI>U$FqNTSbl!MrchagQByXxv2-hA|AqWU&b`R z^l@Jvj{^m?T~Q)iJaCvQYD2pl%pt*va+~e|YKC{*Z&YhD#>WhA6j!@DHmhzgKsc9W zEa|0Tofd1i1de?yKL}2d1STiu#w01oHrqZmtEv0K3(ia;N3N?g_$lpmb{4-3^RAwm zylCnsunM0hqTHRTIc*4#oL#Kh{+t?Q2k*0itE0QlL#F0kndx+7tn#Bwcd)!#U0Yq8 zo24jd<-3a7jv?r4k$(rnViII&d7R&%cHoI+$)^Tq6Tc|<$2g&=jYiV3A#%~&Ynjqn zcv~iDlW`jDtRL0s8Q0>wROl>~M@Af{M~V?OPT6pA8h2Lo*)szm)7pE2RizxS2iGMo zz{!UPQ?p^rU*_;$55ao}cCNptEBoiLN^FUz@l19rG-NRykP%QkNDQ5<@h0uouv_tmfXrtD3o(Uh@T?DUDpBg^* z4>~6EHyj3?bRiknvhR0e6D2`H6mYFVU7CA`Q$Tj^(W{Kf_M{@JI zEVXU5b>;zc?b)uXi@wSgtn)6OAqjBwQWVK3%YTV31xy~Q_tb^H%j2lzSF*Fz8UV?gMAksAh zeiaXha9_3EG9Nu3eq_A+Cq4A@=-YpuXD<*CHkDdyioiDhqOQ>V1Bmq*EE$Vf1Lsat zDMh-*Bis{b37o}!N+R6Fvez_u=M<3FNf|8b8d*QeJ@qws=LkrfSs5%nI%+4Fnf*IZ zzxk~69sdAYXa%+^#T#lI_YKKv%Jgq<3ILx+mb2pe!5FTE{GygKo%uZk4!_yTQg6OE z)_HL01*Nh)&Ae;Ukvt<*teU{;D>sVlM|m99A5$sjdq=W3dQKzK>t0IvX<~O<(g^4vYm!UaAne5l4tjD&AZi1&WAhtdPmnU_gom)dG69({+IXa8*Zd-@aH=Qqzn+ z7a?^qUun0rzRa-Ps!`?M)Y7ud&?k8Qo%6BXJxP{?bPL(>(%Y5oGwm_W|B}rs@OB-9 zh&;#VFKYpa3k&6vgb|I@L)4%4;yLjd8wOuH%^=|qVc}4RiI#teH;O^`Do^Mk$em^_ zR;L@5X1Or6N|CBs9Axc~(>a>293y!u`_1PIlSd;)<0ddATEZvmZ7!>r(p<@}H`p>? znY8rOIB}Md*7vgQ!!o;DK+Y!_*`YF-)XQo*?wbMdHAi57(4#(v>H+8B^tNQcl$sGS zI-twwl!~Ot!JJ5JD$ybthAg(6-ovm@(a4&$LRy%ibH8?KN$SrXx3tR7y)zV`q z));ea3)8&+si&6T^m0}_Ex~7;A9N~Kyk7YX80qZdXCC;QvpHcdvWi#fhB2ra;-4^O z8q@)=s}%Hk)BFVM1O7Ski27qa^J`@f2#9a(nkPaChC{q1r)dem)PdZ1`@%Vl&q}|2 zj!5aYkaLO%aCXxl;h=fDHoJMlUTz`9*9a5$Q8GL$rqjG$oxkeC1P2J0-b@JA2MIkM z&`Cy9?7xDg?xb@BOCHKH##8ffSxt`Dm1Y16tewPo;qR?4Sv5X|<{JoxOqS2A*g55c zcK>>%hN79!*I^5d7C0RZnD#H&GxYuy^a`*}F4mJESwm&co#uvWvCZxvK}k$G&aUhb zp@RJB4OQqe3DKDyI=sHtTAAN&tPMF!P8jB2zJ}+(+%jK0c@g(Z&s%Z&I*)C2!@mOP zuGr~qHDJKiw2yHf+MK=w@U#z5q^C6TYX`3_Clifr1e3yBnGSbeZhvW+j`A3qQZ6); z4cHj}$)6lau1E1AZ8^b_cm90CqiwoXp|O_Y6t>;>9VPs z@no7@Z8yO}R8^yU+Y=HlsW+THE76@g#5_Ik|Bjj5sO7;41{uk$$Fi27fykm(^QBTM8aH7eJ7NrNRFU2KP)Z?#64C}=?V z8XGjtdp>(w#eVvsqt{EANRXp~Ntrv`o`yky@-J6*v@9g)zd2-p!uMQKkG~Bakj2{k zz3EJv$;DA_yaNULG(*B#FdvazLJG3r^C&Dekg_4dCpNz%E21_;g0B~VGa^%xk;JghW7``)hg`aYEb^f%WL{9w+QDCkX$}`foVD3Yr6O z4E;-q8#C(71;Q`17KMcq_tR@TD*QprGDdz5t}{`0f` zNqEJ1I^mo2hx37+XUF;$^OYkAVW9|OXWFBG6xX-x(KmJX$|9O8CROYwwC<;AFz^Bk zu|rPQC-njMg>tj9fAw64^!CvbY8A9o((ETQNJgg8+y|`V@1Gz2NvtN!%QK zz`wH*qZobw1c0y@E7-ZF2O$KBqPMAsjr`-S-noFQglFay;RFVD*iN;r{Gtd0c{yH< z{D<-x=lu2PM`I=7yrT&$c)S7~`}50Atu}2)Uum=w`eMTZel*LnSf5k?-FS$=ojppi zAjI=e{h+?l9nM#2f0$y4@Ie;ipr8UK61mr+dc`vUGqNja?3|UG4c+Wb8rgU8>e%=_ z@I@nI+%+~EW(w{n+qck(1>6_$_gKE-d>PyqvmkBjKh&V@-8`H}+;sdNvSEi|Ai#(q z=c$?Z((cLO1Lh}f4q$50F?(9k^79BRZkx|24$eEqfL|EKVi!cpCFMN&;1)Mji!mbE zK}ZHr{|3V#5k&k6ZO^f@oA!oo^pb*F>{tp_js3_IR1mlDzqoqG;LM_}TR67ubZpzU zZFOuLPtviSbZpzUZQHilH|M-xeebPXwSTPIwQBv?dn}DP#~kRgaxwc8VoeErA}t$o z=#)tx<8v%%P(#E=#Vibq3#hQ6a{tb_|4r}iJP^XY8_b<6QP~ynIEH2c*>nG!+ZC9L zoB3^e{Iu?#6xbEQ$TW#r} z%LmZvCET|SRG$|Dz3r=vwpol}XnEx*D{;DGF8^2$%-YUK#k@yaCC%q&5#I|b2Yq){ zxx%bd`ui6;Mpd~=m^z*cVK|uQs6O>yQ#IGJrR%`~f9Ym1@aNPCZII%MVNZ+GOB4_; z#vDU1d?U|FN5l|f;EBmCWR}yG{-$(RaTWmkR%#339#on}4w=zjT#Wz_WPHN9hFr|w zUuROc&Gdy%r(<12H1x%fT8eV#vR8^L0mI;Q2@gAXJ5mMG_=h&PZvfO27l@6q0 zgBZtl4HV?Vj9t`Q2Tcj(B|DGcwj*)uWCULbYKyoq?yp{Mv`yJ^mzVk{91fe8h=YJV z6gQd5dXM^8_~e2W>gVBiy`gDO$^;O5N6YxZJc7-JdDgFDfF-33VQ>N|fw+M8_<+#t z<`MxE=u^jZ>VW6$bAON$RxQr~I9*=7hSC4c8OMKC@W* zrcq2=&bCgyl26r3OfC};v44nPdUFGi^ero!KXOD*GZE3cCHvWB9V}WEqid?~O7Ywx|%hciMY|4sB$*hXb zgvl$W!)5uDH1Z_?*7nY7o6&=+vfK9B<3*$L%BS{ymvpyWhh+=8D{zq&Wu{g7z^9o& z$L4-s9>?kBy&zkVKRSjl#QP}Ved-c}RWWwW^T_YT^Hb#SDJacw?_N;a;~-7SURb{L zWNz7~6`DNKEU%Pm)-lTjN{YNzB6q~ONZTniTCiQ(FmU>Ssf#}z?JG;AS_!FMO`KD9 zF5X+j?n>}xtzGnF&E8R~4TcF|S!hDIMXQY)}ho<8mDLljs z?xOnDQEVbj-BpuH+Qgm3+B-_b9KG_Q9`5~`kxnu5m8x}w1tYd~JHfO48)lSM<>+Fr z#wi>Xzi;FK)8VT7zjGnBwrN@9o1)t_F!LgWocoC%BT9=n#I*gN(SNl83cEn9{kbx$9qq=E{UWlPl{~?H z_p_Uq@QWAH6xkVzelwqRBq7!aXy~4Fept$9J!&l=E@0y{sY8jHHlhDfq)rA?9GjgC zY#@C{)Ls79E4>XtdowF*Q35f~#-svkKC3)k_*6M(<~9d@DSr>VfhXC_6=wrVD&Ws6 z=aP;B*m2TdE0km4m3C&8N6{-VKMid@1q_K2wGBDdKg>WhnA1}&UFTz4(0L@`*cWyt zq^}!kp9**jp!d0(uk<2jx*}ycY99)89HzWv_aUd}c?lIkkG3K%?8A8g+@r}=ROmpg z9Gv-&?$N0Cgwv-~%2FaN5EM$CQnze*v%p8RwBkOeU1?nv|(Uo<`rzvx?!w@YqU(Rt5uXw_L;qca6->6 zK~FUkPBocj3FBgpMI{~>oA5Y=s~qrb5qT1_iDOc9C?j>ocG>Tpc#@qU3-w?QI72Z2 zPN@l{o9phAwxsT}3XXQPT)g`Wo!noYxxa>ja-2z9aa{(3FxOK}BQ5rktzuP1h&S_( z9JM1ZSmjS7m5R4wdO>yu8ZQNH**`Uo1C%lifL7@XEjyFt_ia_DH_Fn#W)%qOcJZZ( zTW9xt@(KdD#@S4Fo2yjybsFU2ck?}<0wq7fJjif z7nEyIZUm#6Q;^vm65Bn!1UiIVT_DiyUf6PxYG1QdUutIMLQVSJjqP3Q!tOLczjmp1 zHm06;Uz`Q_sa;74*FbvBM@zZ&u+AO3FX(6m3^&o!ZT1Mm%aQvAs0d8|bB5v* zvh(ReqUN9I`cn_S{a47`g+QX)*TijUcqz*O3?&&Jw}3pZgjkG^+%FofMk4GK_AsVbRx}q+Yv%*T-)M z%g!TKix)54AbL*~-&}%uW*6oHMC=QneT3_RGmLfqa%UN9+kmmc+h?t(lrD zv7z8pajELdsM%0x+iEhL$`QgvSSBDpbjL+lN5Ni9bZU<2ttK2^dnd>HXekXN1Tl`? z&Ez|0Oo>#hIp`l#v__$wZqz~>b{W^qTiDs9I*oRWzs8S#Jg|)S8xVSz#yzvDyg@B= zhng3c!=IhnR8RqS6%+#a&Ka#kON|(Qk=FXbXbQ+JykUFX{+z}1uAr2!4E3Y{B^-%$ z+0_dI_j7j`Ukk@)R=3q_yiYZi3b(2{h=A0(oZ=c-RXK*`lY*jF z-80zHj!a-MCZ=^LRi>2dQ%b;PN#c@b)}s!7c6-oNAt~4=#p~V*WzIiYl{bd41%prq zKhU}P9ZUWmsi{wltbu<{^X%dAVsOe;fHe@h2JrbZaHganrZ*Q1;u^wL55 zfFd;^aU6!6xR%t`bUWfBHNp+Vw;3_?A#wnY760Rw4YQLI zP_utQ&v^@<*Lk=<0gj2WXNuiL+-N_3q$B(x)7D8HTQjz?!g8zeYK^VuV#Lr*m)fIr zm&N*71&ewcqT8JjaI<$~GW~LNALx?(@IwEIB#_Vrpe@K5ZO<7$B0Rh!RQpun_6q@| zy#34HKe<0PV7Z0;%uw{BN;QDa;8?f=SbvI(zPqR0>v-M~Qh%#p{Y#D5naF54oZ4Xi zKxBR=Z2M#a{TKNUm2NwJu7mpfu-{5I!9CS9=f&jF4+6L+X`)6GocLHiLK*>+I=bu- zO*$e)oU1Y-p zWaa@3gy@D`VHVSnyyE;T?>?6R5KTb+((L)yDEG6ss$Qn8ZH;|vL&=k}Tw|12ffbD% zvzqL1+^_$Bwpo+$bd6$@eEH_Z#bX>a+tAuGb6kZ2Ej((kQ~)hX^@Bv%R7knN_FHo9 zxA0pSGn`uIxX&1z%F0{d_o-L^{JvwgUTffPCCr=V>ZKPzaR-N|RRNI$kgV2t5EQNwfjT=6*u+3^#c|LY9sMcxlfmOX$X7+xX&JGCn%MRxwT9D|RFROe zdk*(e(0&v&DXBwy4(keW@#m_pl+!fUqT*dA(1tB%D(0oey7KJ){0hW)TOM6xnRVGo z`9O7eEqn50-sS#5u-~j$KyB5j%E1;C% zcx~`-69b@Gb2b||7m-^-jl@iwXmK~}`?1y^VvX-5p*jl~l3AX=2g_U2o{UK12=0V@ zvB;43u2!&A|6OV<^t6jFEg}kw3vr_c9PqxKl#Jz*^;^e>fI?Zk zk`gR9KiPmOH~V$=xi5n!T7}-GKBB&CY0mvtMX3m_^ifCKQ5DTZ1cI3*{vIc z&)KYUt8m!npK!MP)svf{SIFUG!x*QCC#85LHSYrIW3W0Eimrl=j6vn)G?F4yt|LY0 ztto|~+C&Pn8f6N$E-CC}SH%hQaMjwS3Np?x;30znotWmF`H+Ey#M?}ZK?8Y=&d5i+ z9*PCNhfAXpbkt3jhnPs0C7?m`G*gz?32Bo_nPSs%moVR+^ z+|IP(Y!OPK;#rGDb42Y|1AEhdyR)vtNNg;G=S~^7%JyV_&yxI3D^M-jYmBz!tu3jeyZFTw;|SOvDA1?`gFg6dmAX-vGiS)?972;n z`yOR);3ArA_i7lH+9oUu6RR2sm^gT~Jpj?n=~1bA2H-z@?TiSHkmy?ciQ!U)wQu$L{DPn#z~lzI z9_NmZP@{5zz{tt5o`E{wC`90IZ9jVR3_!Ojs=%OA-%;4wmSu)=jAb25F$1RoAZ=#V z>820gCZ z>@Q9h)hAwrSN?l)$@cl-njsnz#g%vZ?+~pj1^E=DI}-s5XZb;%ksZYtz}!z}NgBlt zV>GR?-w`g#9j)FLWD>oWi8)pd)&aCudEaPBMbnt{Z`#uvYV$C6tqfc9rPY4@>e6BJ z-q%pZ{+RuWyYBT4rJx90^WY$v=iGnM26(L;_h-n@Ei?S398B}S<4t^*Y8Mbdok=gb zVe1g}fyzF2{-}={WH@RZ0xA%99u6Cj;#kDGac}t~`1zUma{g^N(m)vzu=S78_>WyD zH2iXx5sQ;jk%I^WjI zZ`yUXfreInrLbtA6tv=-$Mfqx^LVC`K)6q3nmW$(WpVy?eC3=$>tb)+V(4EAg*!Hm zX(Twjr)KIsc27tgD(xaW);*}3DuI3H^|D=r30{M=ECoe8L);q&x1Do4@6FLY8L&@IOI9*IF8}OKd&`v=hl}vT2R86 z#UD-fh)sL)!R>{Edy1_(-_UG*UyJp!t1t3srSoXD85*mV4`BFn;7jW-GW10qQ^5(m zKk;Gn4wlf5NzffByL<<{5nbhj2|3l|>cWIq^87m=Nq?}BUlx<+#dtV3i=wVcaHbB% zK$t$OpVo`O`PAi4&x!V*2oJdTO}y@$!NB8U)n1>$T8R>O+!%I_6uIK{j_J!janI9% zN@vPb@wTVoGr*~c5d$;sCWE28z;U)Eqxsaq2=JsvZ3CtqhmNlhVT89*@4P}PHl7je zRhpgiF*dsuRdt{CB>VLeWj$-92}!^-No4o5ycaaM3vkgROm$vlRH=P^fxZ}eSBRAt zKHH=1W&TaC%#VLZcO;T8qQvP(Gw2EtJmQ|wK!AiTB!FOuB;;;LcunzPU6r?8>L_>A zS?*+tu#RuhGzJ-kf33?0L=OVXL=Ly?^q!Ecc1)wpm$fhoXmnDK)P<_r_s zk#Qq|44fs3lfzUh)Cqb)&8L4XNMWBzCL&{(6^}=T_|f&yd0(3}yyDZ8_If0t0S?#C zaUG*i7Qk*VTx39!=oeV_xB=!dDeWH$%8HJ9RMFJ5WCokCM$Vj~58OiwjMz1G`vJ;x z8Vl{y5PCoLTZG^4B;$itq?$dxu@X|nM#gL`EV``|cbWx5Hzmzpjx zI3N(b|28`zn3fDTF)0%nW;4-a+`q5n4x7dj&rDGUoL`T?*H69TjKjdzR_R+%KkDR{&77_;y<7Fx6T2B;V4QW_uqcFwYRQC)@yq zL)THz)F&JnVcR(AA=Lt-NLmcBDF#nIP(`w^DwVTh&E$}qDL?NxNF(J?tASR)NHPv@ zp6b3-5sU)O(rbjWwWd9)sz)u1!U6WMDfSC-&e7t&3GVTpvo2X79jEQaGi&zH2>`w% zTbv;l&G-?#nh{~3>rHY(?3zseJe9MlE3i_U9v9VP-T5{D=tAR-O6=Tx>>_ZleyaY> z>zS9ZQJvfT(Qtp#&Q4NR5H2^ZIydp&j$_wuMQ$|WP9$J!F!H*4bA8VnIDCwoVI=TOO3Y2N?GL!lTP7d}I z?3rX7XErq80VCWK%`437%`drwZ7~zt@z5XmHAB{?2qQpl+Go8M^3$z=s?ooFwI}F^ z(e3$|kU0C1n?=Wlm?CY29iF}iSj(w4_=y&#nA5fA;F}wGC#|b}_|2O~Z^RiQ*++hv z$i8#U0HlSr?-piCVoJr|nz=3x`{#qdkK}1JuKT@?>|!#E@i_EW^EN|#W$lf;couF< z-W`wJG8M0TS!Dn5E-0`7zQ~FV4o^5<-zYvm$p!xDw_(~-y__TV^6|%q zjadgO!K09Wg;J<6CRiUS@Q6mkk~cfMd~Y4n+@Jf;Ua8fy+Wy3^<MtLVPYUZY7RKC z)Od1u%v6Uk4(L>Ud^BhPDbxQX#YL-Z*ewVm@nr=4B}ae`VBfT~ltwLC42IN?@CX-F zg+XJRmF~37RBK4?5P#;jdP5<7g(jt@NAv+A4F0=SNL^HZ2sv{(=`o#&pY`>1a>4)< zTgx3tY=`Sy0tV?W(;EK9jNvfTmmgRhNCsg(IbfEHh^5Vk4S18b@I8ejsZN6Wx*qJ= zwZe*M#igP{c;psQiZz+nHP+;AF&B}d^Q06ATV>^qM-wph@cm8#@YE|?+Owdw&nYsh zdMk%(oP7%p+Q0rPlteD{ww31Y z<-^YijSRsM%ucf?nU@p=8*AYgG892-FWljFB zyVw=_g@Q7PBX#V^4&LaUVqI=}`Vx$+ccl7am@Bvy1)zYq>&i)+==@qT*hl<68T7Si<~+(y>?7}-U_pt z7(j~Q;&ufXVMBq%SwPlVG_`WQp;>iD8$I&d6-MBpP?5%XLzR28p^%$BS7}^hN#m_v ztI!>Rr@aUF+G=qa+Mjiz<@f~u4@UnNEB`nD_g`STKVd@w{|`nzJLwBR{|iFp$kC3$ z9}tHAe<18d;6O~>MdX4=g?2_o0SwFwP7L#7MI|FQ)8>L-!86?nf}<#pAjxJ7W+b*( zQOu+%o6q}U%##liyaInxHeRsK>kHaezHWQ_+_Vus=k%=f0ACVk4&d`PZ=fQ=SF+uq zUC?v?lxJQwyq#%-nLbQYRE7?hUA`?!hh6Fg)$Q?Rn!X1bigW9>@1A_?*0xKP=vA^^^bTo64Bkk=FwdXLnY-deDwWnbA_M7Rg(A=2;@zI^F2}NXlO+>0ZtSjNRm4xk}#oPgPBg z%R)c;qw2dOPxwJPZ};-f00;EPKChZCb(VW{=~tCxOpUe?nvQ%?hzY3iI9A^n@7fl8 zmVVCc(jiR&lVYxJnKA>YWrc0Eql%A~5qs_%zUJ%TOkjJ@kJh50+2){d^qFoO(;&a3 z0`n*?XKxM&vbHIzWZa@nXuLeZF}m9Spni#7Nt~_bZ_W3{TO@X?8bGw%0>KKFX-SJ; z5Mj{ta}>d|7RYe+D4KJ2lReM|6zp=9kmU(DRzX=O^?K182b)<(QBM<@v#x&K!$ z_`m=6|HVTNmZMMVbU(c_Kw=it5e4?0l(m+H$NVHS?Q(i!WrD_^Ii*3;ID#b5G6s55 z^KTQQxlwlCHB?XTQI!#;n%b_FTt3gPzAnxx=#>X4# zs!4)@W2>6OCYlfzvQn9lz4H0{Mqic0y`s{$pu>s)icgv!SC8vI0Lukc7~$y8TbVcCF@3DUYXm_dT{LywpVituXhX6K4|>hEsPY11!y$^tEeb9Kt>5v! zBfqpL)|}ypB(jab-m3fA-(Dx_`ThAwo+eJh?;)4IJ`0T8*_5&5m`51p`Q?mx!}o&3 z5(8cvdj3ny|L1eZ{Y^Yb=VwRtQN)P z6n`;UEiN}L^ZPzMJ$0>HeQ0;AK6jbx_^MBRe0wvb8nvQYSf%{ScE0BN_xAntc`|)r z@Kfy)nQoVmGV_S^*oMLs;1H8Ye{u;|zPAbILIFF5$}G0$R^^ zR7jZ^=~>K}PPTP`Mn9J8!>>PTL*yJZiSG0#4mCKFv`|$t4JkFwiRd9nDY&14$Fdo? zhB@)%feMDfVHQ=MJ04RSphV7atis|6okMO?)<3~q_9qoqi>N{!zeX z@g|T;6 zj%&d06v@VfX&db`NIn^oKR*iYKSD|KD@$ZJn81Gw4(vJ3<-xBsxkc|>5b7|Ar8&A% zgE6SOWs_xDZnrMizR>1A?`p!=pAmNt-PqVKtFf}QG2<=+OcR#FSiFpRCjX?2Dz2f!xDE9i(P32$5_El6k5hl` zL5nvL(CHokRzS9@e7Cww$v@0JzW9U;STUCWPV=zS(+tCOsBaubg6A)ZV+z~czt9>4 zUy-*!@4lBGvqr+y1pl7%0{q&IWi+)%ZR_;?Hgu)Lh+^jQsk126dT@m`VXXkogUSrz z_%p;#zlWqU<}qPd zRErTXFZvaZmo!`3IuhFD)S>pBJRc!T6W;*F?5b+U(PB&y+BIhsY1KW*x zKt8VVQ$&g>yBUtXyRj!WrJ;nlAp-&C^Xio7V`G)FuH<(kGPT%F@p+CA4DkkMZM4Dk zwy|jYw~Rk9K`^6B(-rhzW$loJ3dV%XuS{hHt@*6VhXSC_3*U_?Cx-dL;Aj??068g6 zX1cLEJ{oy#KDY{B*8ps+Fht)y%=h3`0A$o zfUIc|`$Qtv0h-57J@mA`n$=*5N8;Bs0d7rG4f$|tja~ka5k~n+KL22`0wS^mpk@tM zEtgye81Ye}yiyxkDaddJjYr$qS(-y$Fk(_M*V_Z$ZWqQNl-alystUoPyVmR$a!}S- zrh!~-YHQMNt~L@U_=FoUbKFYZMCsJng1sTWzR*mT7D7W8@j2JA7~DKmmxjzWKGAP0 zwG*X}@iIKEXA^k=9TNCO-@F$GC=9`*cS~I-)lJv}kc_piO4&@9fS{OLkoXLJ%R#Ro znzXDtP%fLl%HOxdpz8rYr0+k5{5EP+k)Gq;x@LB)FVrcHU^*>t_Z{8QVe3FrJ0p0H zj0;um&bCuJGb7TNl~1-S$||KJcZvORA8nT|+6cZxsn57Z@^de9DT(Ukzmvyg6yy|8??`&9bxVzM1o^0pP6^u~-_sH8U^rF8Z3J!7n_xm+SyM{ zenJAn_|}~dvPA~-HJ6t*H*|!v_7b+;ahL1oV{jRJEyYVR3}gTxpsnXLIW_lTQ-U6e>q#5SNF!t{##&#m_r8F}zNiJpK2+Z^ ze07dWJOuZIl0ydpyDRgqob9Uoy^v~t>4;^I$i(S6$TObmK^c4yzG|%$%M}!OGXDMX zQri}*gqK>V50LF_hc)RYCCXB%3gcY~@>Qfj$kO?Wo>B<~#f7RzC$F_}CyvD_bTmqD z;C@O)xToUrciu1ngQL_})5ARzTJh!!XlrI}vyr)5X>>n;Si%oP431WUI;WE|G^Yv4 zV6ofaT`hG{fSpOI82F3rQ)MIGfH6jV>wQtnR}O<`7lUG3@t?BWxnnWW-F%lqV!T>b zbkr`M-B{Z)cn&}43hk|*8(Vog<+4L!&WD#Faz{(cE37No`AN)2Q}vz8RqQoZ*u1th zS9H7l7?#87;>MpA$dmy4yVp|eiP=!G3or5GWT@W4Ge(!T-%Y$#oX^vhNhl-lBvwHDL5D>qo1NAi9``?ezbKtE{+Utnix znqdbFH{0d*X?qy4Pi0ru5Du%oxR`12oL1ltn@2p3`FbwAb#jBg$1ltc{fLpCHM8h` zFh{_Xp9m#&$yci#bJX%QIoSa?b$Og!fmwaW<9KLpxnavSwTi8xhp1SYk+0pI(mY69 zp`G}5-~puY#x8YtH$PpN7`mGsui2>V%Y-TF%$z+CM+B z5fuo%80HJ>BsrGCaf<#2>2?@njRL;#Y9T^&@b{IAubP5bmi6-;(KqFef-doDn0HjR z@b6BwlNRx7koO0-hqeiyedSyMN3z)*_lqyCU0=j4ST8~^JLw^MQLXBGk-;kwR!cyb z^7Dnx4&wQu!E%E1#VL=492@xp=;N>WZ4?J6@V2?Xv@c39-zZD1A(Tr?3YMTrF`>+Y z2rGxHpTJJ-PR@{&3(4Gq<#Du~Qv|tqmM87U$%KVtvLZ<*)P%WIB$^mN;B(lk*4U== zFyjSktXc9U2NrBo!5*fnau1etbj1J`CS}OEbJs~S2pm}F=x{A<4w2-<*=-TT1FF-S zT$eAsj!5SvbE+P`q8ymctFkZFrH$-mh2ZVyN5AxATa9@QBS4d<*c7d(QrFV%hg!h;tr5Nuqwi(hKSDyW4<9}YrthmkTDjYR=`)w*JD)MqPk~ETCIc92Tg>;%J`uu!bY<#|P#{`9_%B8mEI5%AP zer-0H<@UUwI;dIE+gz)yfo2PyE=bpqPZp(Ir_38awj#Jv@d3*y z(a$lr2#wz*XuA)6Y z&L^==zil1PpMjT~G4}wf=Yj6wHM_R4Y?C< z`UOeYG!6qVH~3>V%gO~aZA-YIWA#$r8%l^>vP8OL)Pf=Y*~NI>64Ii~#T;wP)U0Lo zl%6APlC_;?@|reqMjBvAXx^wH)gf98KX=w2B#}uYDVEDK+kuo|APepSJyke?l%Kl5 z8?++q@#JJu$~NlXH0ldK>ZgC$6+P;mHR|pcbOH`Fc{S>mO2k_up4|c6cBQWQiSXG) z_-2FXw00oL6*#*ksm*P;b_&!5wqm-)EN(J3I#JB$L`j1SUYU_clsskv%ue1 z#}9FCsqx##spE!>I!}3LmqS=i{ZebW$`inP4#U-poO6e37>sUb@>;ExkBM4_Q>F`f z#u1+nU4~D96zPD|IkTR@DrA8yLnzR|B9Fwa8nLJ`YHRhyVO%%EVzg^SE>H`>f2vA{ zIHSShUSllbUj*<<>cgR9t2aGk=VL?_7hTG2Wf*Srg*`~beGwM@7X8Ab^alRzM-^3% z5VI3D>41^66Yhf~N*+c8c^PzG6WZrSvVRlE^-T0P6p5Z;f#{uxPeed0w=2~D9SQgm znQsOPNj`Y45^nlz;8<*ev8GYKgm5r8_wL&CVhh*oIsh;h2@7+74+G=E?bH5^B-|A~ z$({Kg9Y`ZU1Yz`V`6~tF{_)ih{?YaI1(y|ES3D-oX9DseR46|gNEz@_|73?{0&`c( z@C~cqEoPrVVMSKQ&o%Xg#`Qv;NoYTmMQC|^O`B6RIb?3we3_<~)bb6~)YKMn2u8pa zT<^O1F9gtVM6r8+<9L^3JC;zjf(5dV?=L;~cFjl=N<79ivk<~vcUoVtTv`~w@_T5tq@@!LT@ffhKVQ-jWZj=EhIzjeu7g0=7dMh4U)fK=^Dbmq;k$h51ph#>IR_eW})aR9~6xQEa3;4D^g=BvCkAz@zNkz1b{aR z)`WPH+G+Birw8C6Hqs!U>f+{#&wfD|6ZR6p_coF5;%Rv5p8%hW#}MRJxAx9i8`B5M z>p3Dh(}r<*xsG9d`TLnOcP+~@p&v2Kh5+7`HXeG*iw_ptBZ<0{qOEZ?!Z5^!0p zLuWyBE&nQt);u&OrQz-{HvYb46aj`7rjRs93mb-8^?8c!hB;SMXy4Gkt|@o`He?y# zChIkl%6)JrmAjHKca3k)gS7wlF%NYJVKwML-5S`l!hze~=348hm zx&*)>Ds%quX~;N%X!!Z7dw%QZ>F#0a>qovy<4Cr+g;yQ9nZDjJk2=e4LID1KJC8SV z<+jj!PnN7+9m6YqC+E2HyGN%m?KhB#cWTeL`kVQu+P?V#h{D@Fm*W2ZoH^m(?{BK= zXj*}%RcrFN(zL4P^tVlU79Md+vkA}wHs`#&G&+fesP602^fsvUHmvkEu=F;x^ftWo zH7Pb9GS&vv^fs9EHZ$ERA3(e+i-hPY6W1*$~ z3Rn2r8%9T59N~ct1`i76j$H?5^g?2`kO4k_AfEAbZifTJRy510ex}gNT8s{3$` zP~ss4bI0lwenCAXat9(=_*Tc$Y8Gh^|GH#W#%Xr!y5yB%Npc5oyOA@>)MZIz4k;M} z$xQ;N%UNP0bQL+%g{i{)@>jb;tPdV1whM4Ay(m>96c%Q|tO1y8T0Iv`xVekeUuqA2 zeCe6j|JL~ZUvBUJrv3gKGt$UlRR;z8PqVp>0vryxRD4rD$&32m3(8MdW*{KS)V&pO zoK&PfaAd#`7qkbOY03>**0p?zQ$kQ2k`&OC7}PHaa0n^M-eKJYe~8S0d=*O=Zaq0d z!QqcyW4sz<38fKZm%+Oz%R^sA9tl% zIU!Sbl_7Q<_n5_VF8hvF_xsd!`(kZ7t<1M+^jgP*OBm-T4A}xzuw5ACC~DfVTv$$x z902LCD<<(x4u8K-`t>=UJq`1!2ectBurjB2;UK=?mRVMNcV!`VOmF!haItO1#j!aa z#zzeO1Nsf)0&@B=AgBYY;F_7B7_ojMndCA-F|AJjn*6ojM>*`#G!V_h*9sfDR(->A zObdYyH*MyC5a^7ZIkM~HHUm!IISZO`M-1T5pB&w?>l_SDB{uQySpP`pXm)ubNpiDE zb{~}!B3nNsn`LH1QYVflgR&!^w=A4aVPUU+a(lw>F*GIwsTLpA%L6A-;=f z$GHqgwO|4XuW7s8^|3)1_m_)bD2{6TZ;^~-hn!9>!N2(kQxuTPV@An2@P#DpV$pzK zlh=xD%cR_!ha1F@fzr8B(3Sy$sE*-R=Sw~tNtBi;Oq_z!9#W^dyoC@U$QX;uS}o#W zd|K#g{g1^Im11l;7@AbiNEw^krI#DaRG~pA^qo;QF_e7)d3j+w!2*Jm*9isZ2t{%+ zMggBQZZyn#!#=f-O zJOm_3rKg!7G_l9J`&*KtY7tNZ>Zw=`cj2QYeq*v^{v1%6q~>0W(3-@kWz$(=5a?CH z6J$hKR2(^6-A`uHEEii}aa(8 zj==@ndn9L?Cwa`PxQ>Hm(s5d8B+c=$vw4_H4WI~%dVqa^W*xZkXMK8?N}Du&6peUl zJx`;@mgU~y2Qy}g$sYE3cL&bMP+h?Oyq}u#oa|0DCR=)nvpo);#0S%yfO23K?1N%C! z-5i#UZn5mWALp$|a%Z7P%sbhzQDkEc#dNtM&DC%5vtRLd5ni}kXe%_D*b_x`wsx5k z7^{=D6g7I&Ywmtz3~I@*$R$bc^~v@2Cuq+SYUxp)Uhv%rN`5*{!H5%zE*Noluqc8; zV|B3bU>3AxcIP(iX*~dL-a?>P62`%`%oa~BdF!3Ny@jc!lJ?Rsd{}z2QFMt4kh@|A zT1XNiu1dxOlmP95S2veXH9!UmG7%ZMp9*T_5t8?(a61QQDbeEcdeebh2t?MXb||$Q z9#0v(S4~mLbO5TCT7*!r+`O%Y>2}%%8uxX5@I^G~T?x5V-VPu=uI-N>FKFbarBi5B zu`}O{J&bPm4hlWMsueE0OlcnkKUSJwpg1W1FVnJQtr;IoR<)6-|EVgMAXBxBdwZnu zkf`)Ftr`iwDp|&xNeOhJTr}1hWAbL^x18fII_*gi8ntI3-}K$Yz$49H<6zck0xhFU z6&ekYEF?w->q>y(_CP0_20W$q=z}$>kp_WwF&D?_Z(y=aFAHK?US(Vz`;B83)wBoZ z`sy?VYOa37e1KV5UdnDQY)v5+cyiWz*b**0%{`JZJX`sgoXJR0q#}2JXHXnFfl@+I zv!zJAbXFNBM$~8^bc0T4#~~3PM1QM4-XlkRh}CS*;4)x)aus8J|JYb@N6%zS*0+7R z!%yTQM7`#(CK>;b7m3I)bV7zlnTCEfr~dE`$D&s(F_j1PaX`~@hmgyu7k?r}#)4K; zdk8^ZLY_kN_b+{t3cjl4~KSj;B!cnifrLILxMsjy6AlgEW%m%N0 zOj0p>dT)T@pI;UzKH!M+owR#u9!)_jjI9{kE_<=0oUk>Ch8!5P@w9sco%(+!?nY3y zf$XJkp?K;0iZn_4h$~C+@qsx2MVO_s<+DZd*AUx-;6+7$n)&8icu)R zTpV-SPm0HaeazAsA=D5YF9lw9DAC64LPIDpafX4LybO{_h#qu5K!8LTAT$PjvHC+K zE~No1aj?ome#cuZnH5P_N6e9mt#p@yDKzYwWARSxdQK+qg6nq;?wUf>FnT2SS;Og@ z-$ER>-x5LlMUtR&2797(MhByG4s87FPy~T5*gn0xA`pa3uYda};P_2%u>)tYeFEN4 zFb>e5Woj3L5oKPXQlxIlq{TM~RNsVON`nBgdj}}jBTCRCx6sfzg{%D>4x0U-pJAO| z5u1@cf{?;ZLpd~$J3%F!_6W*6dYzj6vfKOV92%dYjC{UqBLj^8hpcxD&MfM-wqx67 z$F{AGZQC~Q*tTtSY}@W69ox2(H_tg=edk5h{;{g|{x|nR&2^12o=^Ui(enn+ebwjy zS;jW=F_(|Z-}PX~Yu>|$YGt*C$tQYB0DzBkV8yax0b=F6g~L#r8xK88>uv}+0(!Rq zvY_sHBaU0b;pSyU?bd=6NIk;oWj&l<*%MFc>BdrtadlVI+Ln5H1A`692=idF$uUpn z+0CzXO!K4I1OZ$4OD<@SEm}0YC`ply@XKVaI0-;|Zs^IN;$F36c&q9eZB(^z)*{O~ zJbO1v=HAUS+awa)r-^F>3T?JxqH}Hlu^5R)s!&`DqgKUs3L*)}<{O2q0Xa-OCy#e- zGCXAHX=lK>!`{{j-(5^+U%p&r^1f~3$?xc2{gnO1U8ysk>-Nrj=OGFX%fIZfeYEv+ zuDa@`Xp(y}+X&a18lN0AU?tkjn%Sg+TNZ|L$W28p(V@stt!Va?gMHIsW$upwyq~2P zOZJF=jlBkZ%AoIUa`U*wXQa1kW)p%ZFc6lGdmBv1-dlp3b9fJ<{^q(jU4k9;``myE zk|*Du2v<4f8V_BwLjesS)vy=<+Z5{Kq(h+=E-??7&eb;d7@^%53N_~BOBmo1!yXBz ztCeu0ixN%fS6ZS6;SZ4p9g_zER;V7L9<(*Su}m&PE+mMY2{{RYN5d6tI{%G;-GfU@ zQm|_vEDB(n9JF-oyf^W}&X8_RJ-RIP3-!J}z7t1ramdu-fgXQCtGeb+2by=wWh$@` z&V<1Zz5BGqqs<^yMOmakV=k@!f5yvx^9E00gIelnYCx=o^j-Wq}2zj zHV{C&3ONH+_zM6keTWkVI4X|&QJM~XGP(*R#|$)yIFCjDX~T{&5ihb6Z#N7jxN(9( z?1P~UWuGW^JMmX_z%iY`xFH9ZkxefiP<3F#R)Dh(al{tYg;RLl`dUM_0P#>q8JC$ZWA9yC5A$)P@>;tcf%{)y#1vfWhJpSjk9zO|BAt9d} zRlS|q+ML6;!XP8Zi{BKMv445xh-zY4Xem)%8zbtpp{fQ*0B2r~mzW8$tw{<`ACfo9 z3%S~HxL3iQ;fU7yK*^lb@j|C}EHfFM(5;A9OcSN?IEh(|K|1wO`0%~KnMc)JWSS~$ z(GKkVkg_pdPQ!zY(?YI{)8qbX7dLm7^jP}Z8#k2fyW$S7AimA^zAG?`$vZ7s}0XZ!=E ziNUfC0+$Z)*^f6syuX1nf6_*fcC!aW+T(BGtDX9xJk5)l$?yrk1RPa9MPscPdCqK< z9`TrWbd)}Mo<agu{+0VkQ>R6G`Zb#&)FRxsHFw0g8BiN;qH)$e7T>^tyhJFL0?O zVn1iejZG@cjg9`s{S2N8jQ%*J^x(I902$`2n}Ev=c~m*t=-YrF`uO^_?A5FuL{U!?6E#axsAvF@{w2AM-~zhh3OZy|XsZY?)8=4kN_BGIjA+LFtRi!RTf0)8Y5 zZX_h|7O*ws$>z@c*5RL35pFMuqL7ZH00l`Y=T(3yEDkw>2%o9NsCuQ0I$_H^*&TDR zWk>pqnUvx^x14Z472B1B=b08mil~YA%KEWjfJcP)yph4u@Vo)X#S&@QpQ}y$3`Bdu zz-@?@iCEulVp<&WOfE~y7T&3dXHMZV} zFxw~iwNKE%bY)LF!pCFSB*!cfWrjmN@G$RX*D&uz;AcB#%R#;glF9=SwVya`+9gmC z4Vws|OTFK9STUtv_qdGG*qZyx9>4B(ISRwR+ho$CBI8!b$j4$KTEE5g1D|DIA9>&0 z27vWl6LL$_5}lTTt(;a=S8V=)3%jpQA(Ndn9dS=Gd`Xry^x8R2K4XnKp=+1=4m18a ztxxm(2z}U+e%K@Kr2lPw*aK{jkVS54ZtJI!gWhK=3UWKq=N@O2LJNNd$qG^pD!Ka znnKrT0(D^`#v8lVPePX`s9bsop6+q^HC9(d({e?DKifydbA=wNYD(zEC6K7geDoBJ zr3{ekzxeJ3;e84B<#Y!D0Qxpz)c@tcnshp)eqAbZ-32Mr@nw#3yusSxXJP(W0Whoj z<_uUOTy;ESbbhk}kt1ApJY#j{<^?X3zuoM)|A-`(krcQ4t<7yi?uF#}Z&; zo_DeFf&NZ?>DeVG24Zsw(WJrgQ>`fa%LQ?c(JsBoJH{=&iHMe?cUIc}7`=%oIDrmX z)gRDk05cQn#EU^)FK*E;<5OKBX`u=FjqpwUp)4{@L}XK5plvMZa*y-E0MOzxgz>!Y zwLHXF=6oHLDIq=ddWN(DdV9Qj9ttw0twquofE^lb`k3+R_)0|n)QuS#Ej}B087&@; zwM1~#rXR0t!_QvCBYj8swwrl@EcJ2&Wfvrv2_UfvuHeM1tb@jJAfWAgbs(YlCu>7W zHFO>rfpB{Mo_pQTpPCH)2C!XKvfWU0MBlD9>ulHwRrjDcPLNL24pZd398>0ek?$92 zGnsg5={juY5o`O1zTa01AltHkFD=w{WiUi^Jp)Z$mr1|9DgHB51G-z9QXLm+;GFaN8XFE8=wh?^wEhcs{o3{t)zKo=lHN`9cGrc>78Hq=38duSCF zX8#g^y+V3;JrPY)?Eh803GQAL_Dz|``@_?ItK~)R3an&VIMK4|7n%%og|(AUy<<%A zU`%ylOhIByg<(ud05GOTF{UUmrf@N)$}pzPF{X|&rnJ+|2YiHV`O)vYOjjr|(i!D7 z809q?A?eb~=+ez;)6MD9F=^A&ejd7Xb=q`wy7U#=^cA}F*7~{NcC;v&bUv3#;6p8@ z&r&me=+sOL_cAFU4!L3GqHvrT6gpZQEYL$mN=h8G-d99X3M+ZSPwNVymHAiZ7;m7| z!S+4)fmIK#Y*16cf8llid*keX@VfuB)E1@U-X{;7-hf?x8>+U!c;% z9iIQ80MRg=#hJl;rk%z4`{VNww~zY>cFcG~f`R`PYcTl#OG@K!>#(BtL;hDyiHqa( z2VdOP67Dn}NA3KEaTqp|;2E$M>(<{bhqFzjPCDAw3@#X$-pW~CPb6*hAWJ|5gL#EzZAC>_`@CJg$h--jA<_B2&LybyVicyeS zijk{+O}R&%wpF4fcY3oX>Kb`PwbiJW#F|zkE+9byljlwDJV5gw zr1}3Z;QtYm_Mg(W?q0!FLV*3(aC(ix1_a?hLsWI2uEw8l$)WxmyRUnMgH6+){INRF zU#3R@tUIl%py@g-XgO}MvlWna7oWy(!dEys+pz;{D+NZ-wx?z*!LT)uqPPZUJ02k9 z>}EjbdK|K(^TzK6AnxQD{Cgid#R{=hBIsJtde3gO=T7|ocz+fEni`be6|o^R97~rW z4wyi~Rvuw3A~O!Hrxr2vAu&#^CpV6rPi2|~gyCVhV#7wSGh<-qO^h-55gSL`W<(#N z_3+Nz=#0SZt23)%_omj5ya_(H~d5pdLJx+U^Qu^TCQ8q7|v)#SONyL4+w2XwqpM_gLqp$93n*eV*3)m*h}*|MF) zTF(w}=+$v?*>o9FWMv0TmE>wH)|QN9k5RFm{<^rXSEg{Vty5WP)u^9~&%0Gu+DTzu zrm^}h{#{Bk$?OqYjjXIx&)Mpycx>nZsH&V|m)KS*R<6(Ds7~f zPy13pSR;)!CTA;+r9xE}OcP4zW%k@zy5hKDCYT#NZ`a=6vVN=ioEQC3+?SEuR}|4lVy81EORl-i@#gR@83IN7Ae z%NkL{m}1b_sI6AlQxVY^-et=Iq<;4}1^qGir4gEal9h(n=|J^G;5mDFkdyzBUT98z z)f(^HZwsMH&QaiC^<%}sv)U>_^OEpwmI;vXEtU}`sTp~(mmV8=@nZmpOEHbPF?o8? zwDn_aYKNB9jnA!px#c_N$^=UF18Z?T`N`tsD3SF%9&n5GBjECXr~Vve1!LfqSjdb{ z_hl6dB1ArO;f1lY?1jI`WZ1b5BE|#Zk)@&8WoYUCkr}0dcP3(U z4J?V6FkeICO?NTe;9R!RyZH#($kgEu*%9n66WgKUV)pP1k%M(gU*u|Z=Xnwt@gA^~ z&6~j&nT(ci{(NPG*J`1q)>?7u=mq0$;e`74Mi;9M67u;L1aWRi3 zmGBhdQ`G&~JQGmJaXPDj@$xG0pQYe`ruP4SoARGYzj38pn1&yUstBN`3vvvzMVg}w zchS)UA}+8=Gkul_w{=I zjtGS4XpT_kuoPvDjMGN2%WM~{0auS^lu^)5;lf-0s)3$p*j{@#^!u2fV)Zu`scWL# z<2uy=w#Z0($RTdwbJ1 z)a!@o%5c-ie(k_REZ#%`m8H4=Ed%2&{~6@NTa_Bj~@{KsrP@XI{&kG*Zn_BzXB#oY1+@5 z`X8KL2nYzs(w@Q2)WyYG@cwJv z5t90alc_=L$k{2{9|hAXr>OMqTdvYUa-i5ujZ%zSLV6kHPR3;Ie*1n4G>0w) z4)F@=_m5Hn66(Jy_ET!`|K}&rhE0bJ!2kkU;Y{-{{}I^j6cG~vnzcL)R32~hRxhsB zZS;hpexV>hha)*5wFiQT3nReAi=g%yQ0XS7)zz)WbY?ktEJfBl&s$krNUK&>L`Ih5 zvgZHOs(w5%tg^lIwY0w0t~`3qc;4!In3yDHJvsatNZLF&qN+&Gpe9{J<5%`8oBW;2 z%3|^;qv@J@kw<21F3qCd;aohPX7(qGxvg(HMCaG0QOI$LBXk~V&g+j@VxH7)Vn2lb z3^g*!#Hn5;RjN@m)7f;7_f_kx@EA?+Xz$RDdM||8rDwnt0HeQC`pP>zP1B~cn^K{n zPwYceQi0N6wb>QPAajVLN7uPLwxWeym137O@MzcJk85&3!l6GEv%m_y@7iF{En^d3 zD}C@|gww1zLa+KlYS|RZLG_{dx_MEm^_9*_VA=9}kwms?ZlTyR$*!|*{mhKPKB-iZ zwN=+JFN^*VfPj5Fx!}*5)9}U20m=1CR%4{2_*Bg!EQ@vP;3%g{T6B$Wb9zCCev<~R zXF-;~M{2if^WYc(6QJW)>v&J`qGi}F(Xj$uFCUr8Pd5FyFqQnt@!Pp*fTR7@tVe8D zd((w%`)F_7^c$c4tLfq^I_g^Gg+4@JlC`~(W#?WM;5Cr@lwVe4R5g0N;&c8%ET1(} zyqu3&ed8%iOgD5;Sz|?AOKVSUp4q;POL=R(d-Ep|!&p9|y;Tj?TT93eTUmUG3||@@CCgp>fE_n{YunsZMQCnUlScEmeG6t;>;U4$E9I0&7mLlTsY22^KGk9_ zk$^w>C;KCd!w)VNeV2>8o0*<*c-jt@fY$v6+(LG6HJs0mBq?5TU9b_+8<%Vr9&%bX zTX+b!@aG|0p|;4+v4qZhYKF2!FN;VHF#k z)HQ;5QIy^N-8Jqfp$z(~fKw2L4tMBbsXYfXTwl9+q^&?U6;-MH?DLh5P4!?NiewJv zKuBb2mp)Kn!w#f6xr<(vKpGKw{@xW!V2>ZB37r!T4~p+HPQ=rAjQX%D37^;lokMOQ`r?6c0;f7c`I`uLeNmaK%#J^^G zGWlp&=aam3GgjD3o;kCh#TE@<#{U{TgNz=n~|Z2H>A{jGI55GyS#CC0AmRP{hTzE2(`fID4L(%VeJARH2_{=2XhEae^i)% zhJiVUFVSCh^TyH|G(aw;x1XWnL)H&HLH39FpFRJeOX;27R($Oges;ymunR5Qy&k`) zJ>|gKzormAWmGR;0K8(cW)FQ=eL&<7+egj`Ah82>2rEATq^CV(({Zm9)ZfW~(!1G` zci1noz<`O~J^d;dp3Tr;i#hhgc`3mkF%Q(l<4epWyz{pr#Y?lzE^N?6 z=pn?6*CX1=cYqUAwW(gS2^{H4;+t}W(uNUCNUOcP_?x#!vhmK&qf|wLym%xyMN2aw zO)pgFgZ=E5yp?$x_|1e!=+tcdA+%&EX>tn|D-Q}l37|FX8F!h{F!evi}mk5e!tag3o7`F%dJ! zE-DRRGmnYC7TvD-w?Obd6;)dSZzkU^X${(kYf-&{-soom!}(yzuS8|`CV3Fj=7Fi^ z-NoNk0a<0|Xn!`3=RH<6UX2p9pq|&_HRUDpZ{$xZhAt^8DxERTHGImudkwx)rA$pl zl@;}-j^0XiHCBhMzsRz6JX_)(`B9m=eo88UK3|Uj9`zozN3=A?qcg!@jqPu(u6FMq zNvSRVY1HZ`&ztz^3d@a+wm$_0UwfsQ9jaNlRcF-J*0}29L7Sbp>q^ZQ8XRX0gKZjy zgv>H117E?pCW?1HfZ{TDH*F$~L(5gI%X1jpr9wp3;m`->#f!c>qfj8;jk|gN9`j+q z^-|24MSo-+k?GZG6>nSd&3pkSyw$xt=4vJma5M>rcx$0D^@-8pZC|pUsKv#@8%^}S zjFeN_4Wl%?72TFxZEV#;ekf%$b|Xl9`x0s;z7`OWoH|R%xGT6WL~1Yn42#r z$4jx1Q*j8J-K2ESfORmj0G66QqHtKpyxdo3Y<>Q8@u4?o*^^sbF{$B;=nMuj`uat~ zu*RWvB-wogo_?;Kn7qk~7^@ku#Y5y@08(Vg%OTq6#XU~M+PHQ45b zM2c?5c*xhp8xJ3MGh#VLztGBQp1K28zp_4_UpjBF7O8#xmlJ5oiM2TZ4Xx+*nV}Sy z%#+>(HLqwaOB}!Zav}3XVNq#Fw?g`5e&uTF8zie!@Xuma`BTy~D&dP)_8pzHp*p!%jk3kG z6N2~#Nz?IPs|JbL8hs%EP6cYp&MeqB%1bIGNGkK4c+s>(VZs9FCaTak(Wp03gWqegLnm}s7S5kGe|Pk7j=Ws*&jr%`i0QW@Qz z5tlXICQpq>IDVmvYSbUAg%k!Z(a4$Ij9t9QNcizE$A4LXBDGnz znQed-gxgl!HqPKSs%bZBDt35rPqHBYPK5=-63@OeUg~mV%PCl3(w{m39 zqSm2J$@%hAd=zw2Aj^IFLB8Z`e09R?PjDWB{d+p!wg zHv8{qf<`~6(n&!|+^fSc+*q?vA z$BF?%bo?|5Qg@K6{!f6a)GpjapFM{8f+&IfqDP?XOGMqP&|9Tl%W10MB3(df7RY#- zV=qL`&`RAoxYsr8gt+vSQ!j?_Czj z!&4WC<9VB0p(sxby2Hj(kX+kUk&^ZYSx$8AjivTHJ$7YdJbR|@LfGjWX*aK!9@B8U zb72P6E?~S>&*s3zK#?X^;YqL47h}q1&@oO?7%|)%2VRB?&>t9=YNo1kXt#+ zB9`1KTjhUMyOUR??R5K=aJ>MKkz`IXf@be9*~&a1DIZ%1Y2T5D)lUO-$e4*iQIsji zisWf?V0xk~`3)rWC9hq(g}pVq4nf_y8?3eV_X2)3?*qEu-yY$ zC#p|#=NrQI4pnx$n|0UhIx+V9&AWv$Z6w?MX&~?HbMCi?eYiGk!u_~?R>;i*HQzuK z+QQQhB2yDFdBxYw`--h#a@#QO$reI&odi5L1_$-f`i{nq}P>Se;d8;rGtx6-j z1H0gOm4gUi(kbo)Q@~<5qdhQ;^o~HW>9!8dSM~B6g#xFIzo-Ln3S8e@z=F}2b@aR8<_V#4d*vKDKmk>-44Ij zsd9KMP~J~z5&8fbE=t7u;=_2QAsb@HP8@iXcGSm{mgxGx;rf!=FIIhT`%S35>bDdp z5Iw0^dwdVzJ+V*s6jTE%i!45%-6I_H+CD5mj;P&(tqYw#xZR_zi;zB$$2aC42;1ZL zH=W_v%fvCygt@a^V_;jM$w^2@TLs&|Am+gF?@U>N$3_5{&)h9=UUlFmKwp}$x~4IM zw!XU}Y2TZ~)YYi>SJ)f6Ck1zu2D0um0i8(tH_I}xj$-=0bpGs|NGz2QlRPV#M)=r0 zX8TVY_LMxOH&duD{?p!!vpba83?L{M^7Us4^SBxn7v3pZ|CW97^K0M1Kef8Xrt9>*?eS>$-9 z!vZB`&B6985>eS3QTB?Wrd^QaPEE$cvG&aZ7f8he$=k}CQoYwm>}{@$NySW)dTIdQXZ3(eGMX-P$OOUAt`(tWgI2jCq3e?Jx1>Y9z451e-%BSG zq|+!NWuwj)>@uV5^Zd*U$->@l<_v9EM)n{(X;=in{?}F)VZY1Ek#5nt4`5)ry$G=G z!_q0IdLiW%u!v%_#iuoMM&(3~S*EReCc^Cp#lL?CrCYxe9Uwn9;@8ozx+S5`CzPRB z>%@j9nd+9#Pv5)pyK&^igg$PLqQT18oOx=dc_E!8!?_+V_XHXS-pHmUSC~P(R0XEC z7wq+={e?S@+JdT~$@B(zay-s)|53|SdAGqfoZ;Wh{)zIhIQcNWn(tq~eR7`W96NJ* z!bET1!&fRk$lTU1KbULx)Uzx8+gt8`TG{{i%~lrL3yKyD5D*PUTKpcp3INte`4Ig( zZ@sBi(*6{6EqM>AJXDFYa0gD!MjFPo#Kjia(AgtvqXl!Jcx%IG^@#n+!C9Uo6ecSx zYw@ZeD?e0K#U<+`J8Q9BcK%iH=x>2DHzCo$OuMAERT9YiWtx7^=lbK0$Vz{McC%0pRaW1Kv-DsoE_zVndxIQC`ZfGlBR^7o5JszLSrlf&SeN zjE|<8{&>IoE!(P}u0juS9gXR`EMiZ{bi-RqU(47THTsIj&afKCpAmNS?SyG}KFZN% zW*>Qn9tM|{ikE=8_kzfC)2{@`MTem5jT197(1y2qIDv%SGJ+2{4#4=C8KPgT{+L5l zUyFAgXzPSSR`wSC$G*|^4BiJyP(!!KFnz`~+L$i$1|5idRoiHg_bm;2ubLxafkh}m z6n3huIBET`nanj5E-^?AvB~PDnazmKzy&#r7Cw`Wx3O!TcARLmL-RZ?d{`AIGySq{ z_vC{28L9+q#*Ge(5n$~Y`RGk9(&grjoCq#7Ha`>5VqJB) zSo&6z+;)Cc@mXrQX>GVh=nRPFr66GL1RcNvYXlCP>AJpQ0ALjlL;UA#+%X)X?=XYh090FlW0d{Hl$B=J9b#u#B zi~;>SVk0&f4iM-rx{n}n6z_2h<$TWC3AHr(`1{W493O;ydVa9_MTrk3kugiaZmJ2V zAD*O1?Y9e$71>z2v|H_}bud>V^x{av?+F{pz3|sJq$owOB6wU3Qm~QVV+Sp(J~fjM zdK4Tg8`mZ;pwxat+_}H9y*7N7ZXmu$N4hF@>AL8YGG6fOKbugcx!?)=h5o3Y zCJ7Ew;TiG!<@D<)XzCX;lsQk$7CiUN46=6_Mom!rKyH_8=SR30pq%e-)W z0HkW9lAxm&ZYnRsS5pa^@b8HAiu^?og%qVuE|RF>gIT`Almyegx-JAT$Ygs^r55sw zJMtb}y}R7GQaEzlS8Z(=N!*lK*Q7azIh<*I4NkHrl{DX!>1)gm((wf{H>{@Qhnak< z;Q(o;i5l)r40x6MI=X~dk67~ZhI8G`&ey zIaQdhDdmHqIfI1xs+F&|%r8z3;MM8)4R}^-$x%|j|MK(HdW8*9q<;>3V&a_`_5PjS zI=_AXZwzSQK+yX9Zn-(&?QnX!CLk2uHelE~3sl-_9akOV#qrQuW5lz}y&gC66$Kz( zX_sZ2fed{R(NnKLad{B%XXV}KC*N6m6@FS~g;Wy)vK1*6emoiT_>)fE+p9Nvr7Fgq zJo4E6yxw#}M+-NsN=~h?&WWg0n!@hw{SCG2V8mN|@jgbsFE@>ovHE8!oE!teo^3p3 zvYUT2?Xw(S!${sh(5$gcx#nI6jsQl`PFd}?wGG5I!omsc*oz|6($>lIZX`FsF-w4? zy|H34B97=-Y@X7q^`GMXK0Mx(n0Y)+tCUrv9I43bFd)>+6>R037>uOvSYy+&Jk5Tk zN8gyd9XN;~G-D+oK{DsLXQn1p<44?skB;2k#kyuo!^)SJ9y-d~-^bt#i~1%4}WkhO*0b@_XRlzG=QQ<7dRk5ROYmNnLnY) zQ(%M1Onb-_l^_^bkg$@3VgYCanglW@yLiEy*2pzD(h8HYkhz^eFtHKQJhaW{)Cn5< zWg{{}Z&I@b?dlGaz4`0DnsY-tK^HsBoClF>vJ$KVR12@&QRa945!)<|D1ZNTrO^k- z2+c15`D-J}s2J>D>ROsuBY;{82Tg2*qfBPG-LO|!nxNDkCB9(F_9eh|RIvp#saE8b zKDjpaGC7BH`1;n%8vWOTR)-L5V`5BW;$}2F0h`(7Y+^+ zOki{Pd4S!nTM+o+9(l?9cLZHYTa)k84QwE$stIX+5l~6ch@P)iS zuu6MDNS|^wF@A+S@X{B4&YglEJ&-v2Vt2a$eLldj&s(=;>y4UuVXDvLwuNpDJw9Nx zEUse}yE{^SOW_&MWdxDZZB%?m&}))>3PFNRN}Z?vckF8XU>v%v6}(uKF(+Lqnay`k;C0AXl!hgup5aZeS=+6v5*NCv8-dUp7fdX)-=A%{BLl9I7X`{42z`jV| z7QuBUD*xE-)$YL#D1V$p}rR7_iD${fLX`3p1$We@pvpc7+dpV7I{ z0aV@Vr(i8%I}GpW!cJhSjp;KqW>wM(JSGg2%b^HE2v@1SJSs>ENh%A~hsmg4h64*o zo(QNlTmTnD$pho)*Zrfo-g|~$ul5QM_~katTL z8Vs0o+XiuykNW??FaGEJ`ClhfbTAkoC@3hP|FAT59#pz7us}fGjA;yh#3X6@MX0oZ z1}UguF|Y&^K@zC)Uxlmj7VIpfR0dHDNofb;CgeFGNLEWN)Xf&!7d_UhHZ;^qJQ_7p zOLlIqr(2bq+G$rcYr{_h*WEJY_g5x*W-oV#Q{Bg#_j!juW|h9z5i5iq>EX)}DRyg8 z`mT%M4{QX!n_}CPKe35fGCjiw>oPrnLD-(j18=*^LJH3Zw4|~O-u+UoH~y$^|1trW z`8bcIIL|v&asksj7Kpd@sNBPGXL_&P62JZ={f3j;^pDJO`}B|0@$2-D+;PJ6uD$~? zat0>PjB&z@kBITSDBwp-%+Dp+s#AOVx6Y`q-BO4*bEc!(wo}!XPgSp&@!oX6N7neF z%-7&SuS~9?x1RZ(0FsalSQ0P!s|-x~)Xz#dG+Z1m$%NV}o!l0@hf)#jJ1g@rsNrlihX*J(i>RusPMn-B559q!<*5GV#B7CNMYCM;@>UMY*QGd|8bW@I%qTNx{=CAtPv& zxZHx7kdT8~zZTjWL%L@;;B04x)sfhn@MfMY>%+~ckeVv&Bz%`8YlF)Sp)Jy_EpDve z2x`dKS-~WI^G2+=8`PBn*e#%u2tu&=RqC*aAp=ELbG475Y`n{ifpoI2`1oAqwD)0* z&(1);ZAc{*UwAST=V`D?X8;|!LltWyoq57AI)LWzxKsf|u zoToKZ5zm!eT?%6@8Pb82y|KC?Rj^PScC0rsiGn>DG~_d2M1$A>vIyoCQKtJ1C@ZyM zO!SmGWX2r?3;qaHOR!z16WG850~5rRF{060A%H2=&nsxm%@m8`GO`yIn37C*Cu;mn z_w9;WqLQ6trePL|=3P&E`#oox7#UwP)xVa4zy?LVOEgkQ?0g*gcgSJ)03%(pTa}RD zbo57y_4me3=zeq&z+ag|{|_5S-b;fe_i5avzL=uHV_fYBskVJzX*!Wn3O#FC+qLg- zn?PuyY}vvQi2Fb_~|?Eo5U$KRwp6LxWlY$35kjzh%lwf6khJdWC8 z(8`Y7|0p{rn6r;c?yW5K5bl`$5IS&z_3G*~LYZV9Jx+@NkdN8)GLhC;)Oy&-*Thk7 zEG!tR(}{D5ejTv->3kICi2MPOR@NY!KhP|}zF_AE%KXGbsK%1XRYh1=@6*dB;gK_e zD1rD5k?ZPM5?Jf*R=zfjhyN<4Gu}~K8mR9SD3_+P5hGU%{NogRROg6Xq=sWgogK4K z<4i=^IBD4e;BJsk#noIpWaYu@dp1uaM2Mf2OCm-P<#8HArqGakW<$4bXKlZzGSR*q zqAz>Tj9g0HBrhF!k&_v81IZGrk_*vIs*qde2A88u+WNEKP|v_cx!!%JY_DHC8))1A+tt#g3ZQ1*q9Z}SQ;ZY(9u(FYX{=(rI#$Q|AmKh%n&hBB}k2tF&vZS*_>c%2pKkr=x$Jp#)*kPfqmUls^YMW}i3j#6$v zBB9&N{>UA=)?s6n3f$kbiw^GWegbvHwm8uVCCgbt)NrlPQ2UgeBkIj?Jx)qf6W?hU zwbg*%kWxJM4tnyWVhyw`uu?x1VAB;8-Rd_Bf&bv zE}|5F-5EJhqi@#6_D`K>RlBOM#!D;8F_M4woY3I+|~yEU9!BXZKxi$?GFvLbEw3>)XLXQgSfnWKmdiuM8aN! z;aLOD-;Bq&J^XEQZ$@<8?LwjqjiQ;HLr#pLO43^1-J1&@78NUxgV+# z8gwG^4?#^kNkKbk8av%>h*3fskEw7{jIbH!V6<;ID&p`AiV@qo$8^KDiCyj-!M(w? z3S~J(_!_4=oBN#*Fnn#y2}39!T(Hp)y)xTM+t}pZ=+qgX>WsT5hKN^ z^7jc_(cP~He(T8{TqJ+-qJ?~O0M2rYa9u-o?I zJMR#XJ-tBP41w<`AHG~g*LkTge&u*+fRkP_;_u{7EqcALox8`@?dmT?3%Vg3!TN9zVmMx9tXJ*d?Z-IK@w`r zsGKW7F_zw5ZLQh5|3%hU2GzAK>*5Ze;ZV^a4iLoLKNfoRt^qsf6VM^c>B- zN-4?}1@yxBC4toHWYSC?{{k?ebMgo!k%RaC1gBFMGvPj0+ED&!taR3s!k^rPsIhx3 z>migcn7%1tr+p({!_p&m9!1s%|18=UmjNmUbF|8y;-xs$Pp-aaGwU%>&nJfU6vg}M zfV-~D*R{H=gt9ihR7!hkLHN}VsmENNc0=)=D9bDh~J=xl*+ZI+GJj zR-uBsT*lirTytOx;QPBPe(Ec|z_-{b_FW-GS+mw%UMlCLmGEkop{Idmb(V=c-f#!_&mPk2G zYc~m|ij530*sEX@seGi?&ax~3F(4Q^$?FFEjmo)V*=r{#)9h-B|7nKXlH&gH;||UU zqlue~$~69**kFz97*& z%@a~$4K%01huchi?bbY&f#mBpyLmj((o^qVMx_t0yN2W4i>0ilbfWqimaH@NM@}l* zA6G@TX}m$>qh@nhl|?TOCz#lfGU@$*2Kg5D3p(+z54wx{gsJtt;v6$pvRzj({lE}% zB7<)7hI8srQKj@c>-#-odd^$8?+w#l(UG|PsG1vHg}ynMc4ViVyb zB!we_wJC6B&@ur96*4M2^CGVsxUsI9Pu8JrQ;Q$X12W=P57y#>z3BZ);!7siJR4M@f57)|zg?IK zNx%MNh(Mefw5cIVj0k@gR0bnO1VFlRd6Ix!7JMIWfbUQRz+1{+2ll86%f8m`iqWUW zjS`N>e^Q5-8?O)O2VJ~NMJVWAo{>^Io!IHxGNzV8fN4S}ix=&u&YzaClJ;v_7l`#~4ru{iIy)$v{0(b^z3L6VeEwCdd(lUGyy%w!N8(*q-u20y zg4!+T%2vEV{@BueRoNczRsQHtshd_+Bc>B>ay69fGhM_={I0OGODH8dh;6I>r`cF9 zi0-qyt=YJb`kyG&(Jf65IS1Anu(pBP@&@jxvNc4k^^g4{{`Qj}d>Vp>lcLcJu*u}R zET+KNj_3s(9THQ7dATH4Eo4s<+7L2Ten*R(4{w@sj)$LSjU6sNk7{nEwcz^_;^G^arGn>J{!2yr=M9I)eQ8w;NxU|3m)xNj-{ z8l;YXh4LI%^0kPcqaI_T{36T`DbyX3X_iK`Z(eZ{P+Di7^JatvgQu~g^>cB<+k??v zpa$YMeUSNl@g1?XVrJ6EbBq}ujoPfHZ>#)@VuqN`aLQCzD(^poU-o5D^V+nOhU7sl zPFUM_bh?=sf!GZRw-GHvV2Wq*2%SD>(uK(ZbZIA!u8Uz?71!2fnEbwk|1l!NN z;3iMm7?rG`sol?4%oN7NfJ{cEZ~L?R)>v7`qS6j%L;55&-K9gxy3&Xin6%DoVo};I z0bSc7oN|xfU$Pz)Gr7Xk-jK$J5~L7Q6#yCV6#2=M8@pio?4zq6|MW%wtI_|juE(J_ zR#1JK0oDwZAVlnpPx2y^whnw8dC-660+pfSQvU6)x&q4}7W}jX$VfZ{8N*wUyg#$eLQcupHXEHORt2|$enGR2jtIO?eP7oe1Wf4$bJn;(`yWOe~OQc7<3)Xg4l%=>= z6TnY!{8tI>-^&?j#<6g3TTHDUW7xg*P<=mDdoDsI8(eniCsx}Qohe}SWnMi0zHN1f|e^AeJW z+mX@-0})cBs@r&`Z}0jxR@9~-mVb|o5yiFufJz>y@pybsP9RtPLFYxG)e)jjKJ)6b zrj@1CZ$g=kK;~Q;=|3v^A{kN4_oG&I6w^wJ7MKP#!w!l%{Ge~RjFpt*7!VK5L%~14QaG$D5v2c5Ze#2D_xHfQdzoho?4XW=C<*pm5G)=gxrgq z3@&lzmM)e z1Dqsp8CL)AET@VUVplN4zZdIGnY|&)XFnWChVwBbcE{ zlN?7WZGLE8`pp+FS!J`%U2Z*mpo^hI04kfCE))V9`ypg8!P#6zVT+<&fcz~H@`JOh z*=`zo_e7%ePXyVLLl$(fqMV}r$iOqYjEQ4r=_}Rk^D!Z^M4k*9jn~2@uW`lpx^`bX z!e;+#`R2MabFXhmU%-nq@h61bn}I(wI5cPkKZUd`1}n9^+c_EA^B{5;xGNP_ls@6X zw+icT*#1+&zU4Hmm-LJ!dudAOiGx%J-a8e5IU{pHOCH~0V^cr8-xlvxR(335jRDW; zJUKvjBv;F7_cp5pSeYpkL}m1piJv=Ayjeh0m(plN%Z(jxTq^84-i6r`sz-c=;yus=hd*8Qgf z)u{LDP<$Hbnhe&+C}{QnIK}JIo!J?7ZZSP$O9$i7Wp(G zf?ZGimh&&T`#&P{Ii~Jk+JwKZ02dcnnAfK;Is8YMZbI2`GHy^1(P62vl$DcY`xn46 z5K$3@fpY5&$`>JF08F^?x+z^Pqp{QU*l6|emrgYYqUL~WA9@0a z0nawwqMd1Zl$CgJTWFFw_p+agl~+{=r3U6svYY5LmrQ21)zsKfXDrB!d+*^eZQg?w zV@mqo8{4}@Z0Bs5>|lW0V+iyWKLYXK^cej_AZCWtb)ij6qHR@h_D{#(B4`f}fQ7}q zUs|I@QU^y{WCDXj;%f-+y4k*toL;;bj$eE3#nu}nFfjqq71*Zq{;R4%SP{h3?uyDk zb!m|J8yOHsyhh{buWrE=;`HG848>V~Y$A7-Wn=0=tJ85i&0X)Xq~q7_)Q9-CU+;ht zWXPc*!h%aUr)6lNA>x4{{K@b@Lu3M`^9&*fPN#I5w8+pqy18rf4Cc@NKnHdO!ggM9 zV84EcgLEZ0;hYz}Hb+ou7TTg=c>wX~xC7L_L1u~Sm@`RnNOFHi2p@|lur$Wj3>;#$ ziY4Hoc$hYNpBMbkZlzs54 zt#&O`B?EO2pl2;jNeyL`OTOVR6_|%>HCZtKu#DF>pX%E6vZa!`jPl*F6*KyEmm`P~ zRLcj_j(#|ouJxr0YOU4|mGw0i+Vj_RI}1bO)wxMrpW$R@QqeBLT8%i^K&q?yiIo0*hnz&B*!ZtdDRF@cQ8UQ}V-jApony@yY-PVr3NBx8u*OoES=DR}0X+SdHvZ*u7M5uGO^CefR zk2fx(9*<^BVm`VPae z1%F_V?@!M)0eDc-6Nd)TmO|H$v=uf##ZBSp_Iz)Ml= zWO_((?3cOONPmYB(YIryc2kFl^A6$qV}kinF5uqLyn*FI?3N_qGE$)xntSLI;DuSi zH=n=+3Y0k}vOO$;n{&sUbk}AqsUrVCtL(`lJo}dwj42o=wk&%@Ea>ZQ0m=*Fd>^uI zR`-KHe!p~G{P%-6p0^aij&^5FTB-N-$Y|aZ49|H+7Hm}QHl*o&>ssk}38vGtFJ0%q z8iDZw8)gT`na9n_{hYDpE0vfb z-#NbT6!orx%(AXCHnl{7N3NXE8c5s|MU1)5n>Dho!D%hK4x_u83{7z_|yGc?U z*>dFK%j(KH%h$x(W!UHc0cM2s#FNzTg^3BL9lM)u;NWQtCJvv1DT7&#&El*t5e5if zi&l$g+JGNUD?YGq>q2olfjTlIvm4QqUO@oj{@1_I#1HLlRp|8j1Zx~Bu zsLL}wXp}PMQ^fJ3INb9m(7j_Ex;REesGN_EG?z~p2@*FqT-;Kjmd>>0bTG+I{Z=Cq zA&@=HP$#_+lY0(R6(?(az|(wjKn+K~sHJp9g%jnu9~eU@q2C0A2hbl-Ju6R{)a)Ei zfC?VOkG}g`x?Tf?J_AZPP!CPhtKN;Ber5%a%QRXreJQhDBvDKCtKZ$yHvyBuADLoU z9o~*Ey6?A?ri zhKB%fn=+f298E!E*aqzrz(lV+TM^LyXi*P-pWR6jVB!MWFBnN-{3>5(`H5*Ps`5u{ z8r#!)yZt;mGX4Ysf5GyP^UTckx<(}q%YrG-!bF7E(5^B&vmQ27jAagAM+{1*6CINt z#N?q{#Ry*n1L11&zq!Mos|V z$x9J5oMO{}j6IOP2}`yQ)~BQk7ibJ_J00N)xF-Z~paiHxqzIt8CLOEn#!$`!lf@+fRQc z^Z&cU3iBK0<6uBQd{F+w7vzNc$x)v8!hrJGN@!>7>YAqcTNiy55CH9wWsa?+q>Qpx zok@2ACzFy5w+v#VKOhd-nAtksfWYLOS=7};Vx4W9Ls8CQZX}OA!`Za45$^ zvQ3Ubz?tBJ;r);D>nwl=rwjG2H&@%sjOW(%`{$%Z$v^&oz(Ek7D}sOD{1VxSDUDSF z!o&yrl@7nD!I-irM9ftZg}TAWo4D(-B~U*@5oFxThow+P-;m{_mW*QX#aW{B#W+SC zt&5)wx)#60Ki_o6Vxij&0hqm&*|sq{XK0=Y3}^AM$cN*LC_ismxj zndi6sSMSDb1K1!9U!jbK>dl6SnFp!9{0o$yWT`Fdu+taI$>Ev5V?Zsnh+-I)tl>!! zbPI24$NE{S$=xTDlp5L4VTtK?u`AdjDwf1C$sL`)g`x~^#;0?w6~(#g@5t$G4S6hV&wQlZV ziJ1HNOmgPeVj1czgv5!M3apaZ>8XUDu`M>EMX=d^UozJt6Wq%5U_3ak%;!3Ew8va$ z8&%Noy(7tFky2&ZGUh}2L4x;yAw+Il*83nlxRh`vxc7!Jn|1~rYt`FV8Wz?4KG2trnHCFcRXDZV>h(_n78_RAm+1Hpb zt-KpN$fxrX={y8jPY1s2RyWzmMat1JN}Ca4kSdg{)I*H!Hd z+0=3|M-9=qnMq1r4SsGv6$-AJf98^piNrtpGpThmKF;j5(ey_409gVm_Q0pcpL?PO zcM@pA3#*>d1k`RV{iuvD`Q%>NBWiwWhn#W^;@C*z9tGx`b>ngIpXK9vTsZX+V0KW4 zlgK?ZMCY`VXZI;dwi`mY(`_iq*iYhB#9mIkVGve5<05blm(ViEEq4i|xP^pfW5_a= zb#w?Gl4&2m@dHlTfibCFcpR?hd9i)6k^bpC%wM+-1xIPQYxoDHeWz+-&PgskeI}=0 zgPYS9ps1CcrU>m~mgux+Yxs#*W`Rsu9|`E!*`zJ{3_6e>&O_H7C95gfI5it}L>C z)M8Fjqq3zgJVS!XxMH{_P`se2*jxO{Uav4N{tOeE7ZZLIQZ|6!8eP+1-K9V@{*s=# ziv%7L7n0^0Nb|F|crK;ExM~NGW3m@-pfA#f1w^ z=gg*}!3U}gmB$1$ezI6q%N5+#(P$H4;vN}82x!8p#dr(bU`&E0L;w{NWM;}L;fh^3 z=~Wt^&!fNwSAA=qoh~`@-Id%g?Q(T2FXFJ7P%Dk#cbgJhOq~B>WoFC>jO8LaO@iXq zZV21DYO_~tY`iHN!KAQKmax((Da-=AHesmHovidn#SI_W^yt0gjM<|k=FYANMny{_ zF0lQary5Qdf?r+NqJcYbv;S^KCv-MXlFA2Mm$EGtx+R@{9$=T^bma~~!RUYYH5SEJ zis%L+h+mpCzZhkf>7<|xbe8$1K$73U=q8u=1S@ORvg4TC?MBj)(!c|;CxiD~+D{Ob z0bi1Zg0LUNZS}xo`=BlruT-zs_Du(ZAGj~;AZkLV0U_Wa>YWcG-aYC^0wI7R_uIx& zX*#38auV_)d-$HBWfhDmym@15EgWc^+B z!g=E0k_BDiCc)Xp*j>u+$%RjHg-YL|RN>U=Yxp51z07gou2~N@LGC z-x=ZH{*!C_*Ms?Q*ZMCcc-U=c*^rTR2B`>q-opA_TwbW$y#pPi4o9vD?C+{Q zRi44$z7Fxb2`+jM8W<=j!-ESR9VUY0aAaY5l;v%4e0AmM0BX0tPeySw5=Bnv-Tmtp z?wYUr>85{>rHCm}dAQP5wvycq58M3mn&?8xR1MwFzKkQOs7~<5cA2##P1Wd#%LoC` zc)pwJUHv5Y#oxa9U4%;u7wg5a{g^@>o|q~pvyG-?w`ot(v`i|nx=<+ z9lG(5PrgP2R0=KJyeh3|ZTz(`wZ&9GrSGMI3DlrL-a&BDe(?K`l2n2Pn2HCtm1~?w z#w5D?!f53!fT6ynv30d?({JY3)ogVaOE#6aI)}nHiPRbRxPk0wd&`qIv`S{z@E!>T8iDxPOw zE*o!GD%;4VnyVy$vH+whT$yX*Jw>v4y>^QVLYkaB0i1pfAR zXx8@9DRc9BDp&O^+X+3*^5Q8(L{9dN1*e>%r*Nmx?%_m+*sn}x!0P&8z^m=RZbEh( z&r?3YJtrTpvZLJiD|Wup)9zrb9-6?QntZ!?gzQU|T|FRv@Y1EFMZjUAbKR|0e6Z2( zYFc|F{J=`3R}e;f1K02>~ZGVh-&ukn{B--x1~10)4Pn_8``WrOo67F zy0xkU3z-G;NozF+(2NOTXdciUhZ$kqUV|R~)+3+dD7DMe8?GJRUm-6U4P}VZTZoqD zil>RQ`-sm|cIe(HY?DqEquGMEeOy}pqb@!oyicH7Bu!wk3}2PSZ|hzA`UwBQyL(rL zAkTUWHK4RET~8NPBf+}sEbZVE7gXO>ot-!K*(gk>C6|B%W(oL+sFZWA^YN`aX-p=Z zUsra9(*L~CTuyBfPYG}HSTroLN#88Dyz?%SxSkTEv{NaYpO5Ilspdi2t&d09SY!_O z#)mW?NuxFq9)-YkmopABcJXzcVp=&=A{ljsWo5;C^97r4u+l2NmH+u`(>&}tde`0f zm>>U~c{^1es3_k>Snw{WQgd6^8AmrP(yh9FeCKUUslTm1Bo(X~Z(Io^^d~=+%q&UE z&bJfXOtUs`enjsJ@h^!%e@wbWBp?_J_~tVS3hk7rA(7&OeM;mNwt`UivM}^QU%KFR z%PE?#F-Ldk#4x5}=N+!Sw2Xp>$a2ga*#DhZacDmkcn_g(X?0De5T-K&aexvVe!z42 zL}f2VFl9%aXKiU1`kF-hC^yMza*m=qx|Uc6O|#ISrjit;&Mbk^qQ+M!g^@%nZ%3SB zlfe{2cLWA2N>n8OMJNa%L->wZ7?{B! zzgmBGHP5ZpBGf=UiJ&Syf4MPN|_kF+a0a zU~j`z%NBms$qN||XJyU}&jGj97MRsKpZ<*3Q;2IKkI(DqlXy1ZIhpU8?gZpdxW{NbEARI-R|@mWA*0f@-&YKTFoKEerP=Aj{)XE&3*~$XZA2~D zmi16u#-4a9NQ{^Gmp4+e?LhgifkMqcs-z;yNekZCon21@`^pbd7?H4&Q2}ZIC3q=x zxVv(Y1@5Y(LQewk*Uxzm;sMESy<1$MHIBJN%720nlwtIUxFv;lV8oMSj9mz@oB=TO z+R5`xk!6)Tk_)KnRfMF@(P~|{;FEFG3`~9^!cc;y4sc<+b!5`vWcuQzQUt226>2`w zEWLTewPK;xdV?5rX#LA`rhQ5pROv&%8a8p;rFq1G1mt=ovgBqwcIf}X(*Md3-zN^? z4)OmA8hvdg?&zo8Ps&K7K%~vMM&p78#-*$)tP3HB|C#Qmh}RV<%*{?00^=&$iPoiY zAkiTXYds@=!nGuhnbNrftx8a@g6xO-3a=hfdI+z4wO z16%8xk;&QsNxtU%cX)e)Y%S(_w#EsgAyYt$A#3fE`3?=r=(e8YR2pjUIrJw-^=5Bt z&s2*fsAFmJtw6}W9Qr(fDb=U{GsE{(hI*aSR0Pv^Q>2*h6OH6W^-I7r-AxAJk$5%=Vgp<5xJU}Px|H|i}Hc9vF? za*%mRG&e4NZO)J8-9pon82ai(pqMq&QgiRQtNUI6c0L?q`TACe0J)5Lo4tIYq`IQo zPy#Gx?@>(p;!7V%+p*JKzM&vECI?4F^!A1X57gmzy8ga+P=X( zQRM8UQVQ(y?>6lE-e;R&aWr;!0!7OLmo1}b4(|}>2E{l0=K0C&k)TM-KzR>=5+zZ5 zDNUWYdES@~y>onK9+Rry;1^)Sew`vwyKskKSEY4o^@_k5Wg17zpbkHjSrr!oc(#m4 zFbk`taZ4g!oF78#m2dxGQ2%qn|LbP|>!dmrvWfRQ1%;T{G!gTf>=*=!T5wVW9T`th=AnwR*GZmz3UNyQlAU9$`HScqB%f&! zYR!YT_jl-)30dv3!yLi>VmifH1_-cJT>s2`SA~N6JlTRp|KHb)y=f)k5!J6bnJl+ZqB``yr*~n) zi_atC(PjL1ye9QzP`E0FeY-I#WpeVKR>gK?nWZHt{Z4DB!)e3n>j>0y>p9hjdVI54 z^*F##)J3X>!n5fNOr+EY!i|3HqkivLvC$X9^5Q3wF2MESYab#tyvGL`%Tx?BI2w$F zGiLC3!JDw_Va36BcMKR!Cs$OW-dLz6+LW!IL=;3yFH8B@gczGtz$P0nB{yTbRhR!j zHjq`uiR~S{V(*>FOTPmjg)du;CLvC1BC3Z~jFMnq`6a5oc{7wv0Ykqi0Ye%Ntzgb8 z=AO4=Bbid`v5fwtar6w-(Gx>kT;f4k)AgRv*Sa#iUp7#9eJ?SRX+RMLw=s#rB@Mq2a`ikwjlqu|=tmvo=tYbKSYV&Z0oRj$XZ} z>6UVQN(`bU@Rc&DlFa`d8^j%6*)l1TyhG(hwPE4MvD5|;N06|N=Q;hpE%pfK7*5+Q z+xX)GL6s?CKBP(XNVXY*XajHIs3}2%3@7Y8MRHr>PmyV$4a7c@%San@@$yPfg4d3~ zOo7yBzy30ILBG>vjufk3>=tz3*L|~&W?0ssb1sf2SRan0Wgh6Li^;OS(3VWPRZLa; zSp61p)7sXkP3dMxUx%Thgu+M!Zrs=cr5La`B*4D!1CF54*fR?0$Tby?V3FBdY{>{? zCUd|r-#eo{PTy|ytz>Sb{3H3d08r?C68+24)PC{Ka|1F#(ji15cPvo~2}^nOt75rx z!&J*)usIAGW5M^|krRvH?+D604}w_}AK-tzjQ>{ie{&#LVjBuCMPcY2pT<8sqlk?i zJFSoevaRD8avK!l@5>e-Nd)*mPi=Ixh-9Ehe>ZJin$-cHCPM=*!;t;6fagX82jYv8 zi@Alli;KCLn6-fiyF!!Scn+IbAs|Qy=*g~8`%))RCBz_SkMgWRK^%NIdVa(6D zL8at;qo_TUFjgHPp$r*D$EZd}B~Yh}blZ z^HFCjjcc_MxU9<#4F_h&AV@3%+Tb>b@)`77@JR2!RiY1dY9+;~Le<B{Ar0W7I`a zUPcpWPwMKBKqqSn5;`rH+uCB+ShaL|kmZma$t?(b7o{!zjPv}qlen5teKRqIp{|11 zo!P&V&&V5!*k%!fg^n~XnTNKXT`r~G#JsgjvK3HPv}0~BO$b~+>N>5w8GE()=3`9! zt%Us{`az^zK-$M`V5-QA&V>nL5D(!y5i~V6DK=&1L0|>3qG*2jVj{(uQ(_5~4$)c+}^l^AoW=wo;wV^jb+3R7+;M&0>3qN5F%S?N=~hDt&quuCt0Kq z3!z9C#i21-IMZv1kgQ!JbA=Fve9c(eI9Ecmu9E)VKHT6?6@@x@0&s)0L$5;Oo=?xE zk|Xj~ohcQ6gbRM&@u1LtL+A|QXw4nxn4hw3bjHLLe_{s08Rb40yROVzEF?^ef60+H z&`hxH=aW^z`=EN&|+fMzPG1! znp%o?{i0>v@7l^0j)j4P)Fa0nD@S|MVn$wJiNSJPSz$&U$ugtFYb@*0nTHVXQ0_N( zp6M6v=MZO^bk~sn3|YMbyu+RC@w&LzV2=1%275M3+Y;}lyIsM*iutcH{kK>4SIwdh znYH53KtRA5{}xsm(;)&fz9eu#{omYBoesR8+H%$(1(uW!){Y=DrqFJX`7@zFDjOk# zYvHooC-x^tWdn^lkN5 z-St~MYFaSe|7>?~a(WsdKeqKvc+R|Bac}?rJ>yC7KJBTH+wMio~G zG|7+KUDL|%z&FW_!7C2M+6SbvP?;*t6b83x+}Q<-tm~T;XC5Grb_|(0cgI8z!!q20 zJCgtf#-khwA~K63s~B(?tI<32ZWqy+^^2grc|=i`;rIs@QvkUGMid3SY5mr4{4)p& zmTqw(AxO-V;_4eSf+@6y^-mC{(F3rAb6VmQ*?OoI@zTt@z{FLNT>*{i{iCn zF%(%Zy@~IWMp-cyQ8X5w=^~0uT@nIKdPSCAb}A>@1UVQ!@uQzpcQsCs6X@aiXH9A0 zmqwb>5Hy&%BKkWJG+3=H8^Zg&5oRo%KtqtAYyr>MZY(#fFhHg`Y&T}DQKKG`V7dn= zC|rD!p)m)jg(71p6c43@7zPxqPBFD1Y%I&UNh7q7X{1K+BEb$Sn%28l5-2R%2TKA* zE z@>?8^_N2Yb@W5)Djh*#_nyWZy(d7@RAKSQbHoIOej0C4huQj%ZN?1{$Y=&O-h27r$ zPE5TZ4}4qHHB5BT5NyIFxFN=MRc&3WF|98=qTkUuCEj}HXbjoLY{9K#zJ3{jkhemR zs2uvejw}pjdTXeSN;2!arEK7B!tA?m$JiF@PM|elKJlYoh{*zHa0Os?j@M67!-X2$*dCti z>;@!(6isp$dU?y8%N@5pGaC~}adDnTw+gnLNU9C*l1Rka6EP(KbScENSp^_UEbdpc zd!c;mfU$Zk!pIZPLt~+~f*s%N<-lQhRcoUx7YeC~+8v%^^HV1o%kVAWB#6jm+E7V}^xkyZ;7<+*eFcI6 z7=)|^mhzt0j}~h zn%BSbF!@$KLwi1rP5QN#o3`zY+hkOO0G|6a`DrI*^#@CsW2*Fz?6M^!X4OXXwHp>g zh6oK=zO6qMGXB-0zkyU!;?ygL#d^R@J533UGVFBc1@sy>_{k`iSwa>aEKS=|&a+ia z=nn9)RmB-2e)XRctCbG`|N4b5r!r}+i}Ey=g|uqU82$`nrZ|Jhf*L(k=uy@|;;#!A zikWkWG@DQ&dp0ta%_)}aOlYf-gjEk{i4eDa+(RwZ?FxDclLA#!%h()Fk%YjSJxY@< z8izvAi03LA2lT`fkEELDz7kppd`p7;Qe8lJ8WD~Tt}6QdR>9I?$N*?0ML_vN$N)u5 z>>&pyCm+d$6_tvE&;7YUQpG~RuF|RV%JH#XUJhV`#G{gpDzxz11$)kNhdc)F@FmTr zbeCFZOv08hI1RsWgfn9N^9N96IPW1$^g?A=e=h^cxsB!u<*E9C-aq01w;oyoz#v_R z?B3I_6lj}kX|A=t5aPE8mXeGu2j}q=+U&qKD8O+*roKVx!P;(ECb)oA?G-2vRN^aL zV7=J4JoQb>FBz;RdQ~VB6{&pSn>9e@R?>K=c_5iJ*qdf#^$P2+K>!+<-tdHQVt0@4 z&_g-0&U=dW%3QN}CG`JB5HPwS4oSfFo7zDunmDl_XSjl!vNR^NxPkeZK&mc~_Ax;4 zOdH~ettYNov!~oRO8i)DLinrd0pnexOws!h>s1E>&Oh!%wj_%fWhj6mmQ(?XA%1^U z?Uu{6BUjXGBC>!mSOGXz3t{QfgLy9*!tdPOuschQ?arhQEWn2Th z7@(i_nvKuVr#TSonjfuT6cKZg-qFh1+TqknYiVO?X`#W(+!iS6FR411{FCFFh7Wxc zKz6)$I#N}B6d(SRyk(1G@`D8&sBk|q^4d9aByOoKaHntM94LxVsD{fTNf)5pqeM;P z7!aVuX4Y(wo+H}!0c)-1>>aCki63#D=9f+v>uK=g3-qcF%QbI${06A!seMA-=;&;o z%6)CD-GMl|{47vr3PqWA9n*6v{1;B$6D*MMCv=dF#|T9%tN}QHDBsB($uRE&SpFa&VCO?v4w6uK;SGrQ|Otle(T=j7X9m z*8*p|B*jr>EPV;F4z&qL%%;YTDI>c5Z}GQas2e9BA22oMsUd$*;<9xFq)cqQgIx$J zHhR7v%(Vbxp~;JJzAe(x6t219YwSOux$(beO3lp82O6McXXD<)o#w~8w!@AyFI(Ue ztuqN{@bP8bS>=A}7$d0}S2rMZh)dNna@4AR6-)+HrQJ-dEKIob*zdb0gq&7B#DK-= zDRiOih=v#LJ<07^2&KC=krGk+@`0G>&OFtTDl;ZKI_Rcag;K7qaDQs2BS!yYSwg-U zwZ@f+7pvSrK5t1um`Cxf6GUg-8EP80GGh+!6=1=!GR1ZT^~$|LesEzij z8{`g55s9#l<&twy&VABUaYb6xN-}FVR4UlrM2fY$QsLO(<62#VHaAWaaQWepLf^K% zr1~TaeV}XW>{3OXf$wvjs9*s=2U|pKOw;zFoRSppY*Hx`Q3lw(!dj`>O#}-~Lr|M+m!QlPd#aWSDImA;1%B1#K=NdT^Koc=nA_b;MzkS zf&f))#e z2;8wiem==?Yg1d2eA=;%&)uXA3AB8gcK~n*bP{s$K2LL;$d~ejY-O=&hsTlVb7b3L zY$u-;LL+mh6;n|g5tK$C;3`Bw>OU`N+i==hS>p1@T!tfJSc{4muZ71G<0^>hr15M- z^D+u!uq(8s#GPDt^3qGxrb^`Qcvt|L_F9l1cp|;|W83j(Bg%^xp9d7j`81rd`oC$5hO^ z8HuJ6`wYWxlTOQzyh`9oq`-Prf~s;YVs%Tl=_Z3RS}~Ha(-6g)!<7 z9ItrDZnayM-L1ArQ4P?jM$rIj7}URmK1DH z(Zg2Q5P#hG*h^6=;D?r9RY+I%2f0}_x_@f7`=Y|kALjFh(917LIY%tpQ_nJ_<|Mdj zvSLfQhO|g!%qZ1M2}Nxz<%KA*vM=Thps<>Fi(Vfym^Unf*DQk1STqFkwU9`+vuH)1 zKr%gBiY@egpJh}gCul{xEj2HqI8~Eh&y(s)7U6C@$2_Hc=NadhD-n-&Qxvl1FR%dg zc2=KRSD%#t)D@N07au^g(+f{s?L<3NZG2-bm?}pqQqbHc-ZtGR6s9gpWO6gthH!6B zm;Bv+g5>2F;@D+61zG^>1caLArE`_}ECY`;5a+rq=I zU0ioy2f#OlufH?)4fni#Rg>=%MfAq%|KnJxaZQ{LR5X4CX9)k|1n`r>T9mXJ5lAn$ z4*E&c{b2jdhE*|{D55CxVpHq3NEcY}Wni(RkUca6Mb6YfTyYU7_+865mw=-@=7N*xg&E}M$dw33Whph zUl$Pg+Q@J4HmKUgzmd&m%NK7amYR)4z*bvY=ME#%4LPA{(77T{Bl(MlJ#gN_fVO1nj^(8q zc8Au49q%~klt*Ml4BB!NW?R61E%OoVC{@|^EtOi&!BelKM+(Yl922(8kX;Omvls|> z4GlR96EWB=Wgig7Cn@L@1BD!>>eNA(384!cSJ#7nBCd7<;=qn7L*huy>Y7E}4XA%^ zxmd*+Gbqg4Td3Ky7Rz`R_$$OyR?v-yN0_x3J=wEWhp$)SY`tLA-m++us^n6RB&x=i zf$fGXkf-nMaBz2>7R2rq>g;KEGMDxH=Le)zIjNJM)Y^sBFpKV@nkQD_v8Wfm$_Z(* zMNveLYr_Fo{o8jWG};Y>n3s5eDk$xO;D5FJc{frmdUN66qJF5spH*1SpSt89gC>ol zN*BbxC08vH@~4hvDoHJ@T^5U1$tC7w*zPSmVZ@Q=InQtE&47iG!*&!YfL4Xa!&xUh ze$%EZqe+>EH*MjfM9F8k)&UW!Oe9Z5_ElE0!pa}$kBNh>t;>5DtCM*#r`d$!I4$9~ zI~LWK9_OXYvGq2}^KliNbU$tJkW~c63h-|6+A^)?eYO`L3ju&PoAN)yQN?s)9g=(sx2<{`m$K-15BeGYk zFB}VZxyYn)^S)%$bOaoos@pA)t=XU2QND3ng`NhXSK$@y)h_So_8T$r$6AxEd+-Vn zap9Fe`I0H>lI!N!4mucsv=It!Xu;b+JQ$SYyhH1D{TkFU613;v%)H7WR3zX+QOPU3 zn_7?{PIwrDuSVt}Q*oF~Me_0)md}Y*vHzTg5Jz2FEExEu(D7nx+5`nXW=JuK>P9Sx z8I7dm=I$J8I30p6s3qWcdJ=cMX@&)SDxB}ikeP$i2L45VSbJcq#iMI*H+OX`N2t)j~Gw)g^GsS+V7t*^{+H6%>Wy6_i~cKIeYq{doh@uSSF> z?%$8RLMD-pV#VEhB+-sShsnn$>_iRj8zg0>a#8_r*pfK2-9PZ7IcrSr-G|e`pYn|C<++#Ks;k80;Dj+l1of7WNiFj{- zbM($O1#O*~u*g&Z!wz|mF~M!ZL&nD+A=B>$XsjHC?X(iq_X}hY-t2M{wBN8jivD;C zQ>9YWOW>LA%R7jiuS9qgkJKF&yM?ED|4%oRs7Gv!dBNF1VwKv7}BL&<{+i&L^Bu`{QJvN(mDtj`zz4_V(B9BJ3J zI}_WsZQHhO+g8W6ZQGtql8!U6jhWb-WTKOKp0CdPe&2TmW_vD8{vlcpXp{2$^Hcj5 z80a1AO(Q{in--%ykR1*lg)kj4K;EH^Jd-Y2y;TMUPpECrn$;5_;!9-9-_ZxUCeQ6e zPw~-L(JfcdgI6&fFQ3S)Vw40(qk@g5-#`V2LiR@)oX>~{UW|pWBaylR8}a-K=feW% z2|Tl54v;>xwxa=%1i-K82{m3KK^lYzXLI1hhIHklYJB5)=003Va2@sW5b*lK7F6@= z)spC5xWUMdTAZ;6j+;>7y#tL!lEA)1O{BzaT}KDDG1w)T+l(a+hFV8Gw}Dgihp9G9 zwz_Klrp|VYB9JnJpe09uow`!8$?48+2@Ua8@Q`a0BOa7;3sjc_`GoFqt=_^eu$e>Z!Lk2|$bn(~q7B1)=+o{LkJm4P(Z#^l^r zKDlM$7a0jK)T3e+n2J4wpBuvn!WvKY0QUft)`eY~F!}r`3ll`)V86|tU|_PYQRf@> z%gFNl9=uta@2;j_&D~@slm(J|PFiX0K?0>ZO+J-LN^dCypx{80!ySA_E3K~T4E`x4 z7}&E^d3kjm)5+}}2t`Oyu?&OG`t#yzffBuJHzP9VW2WP`e5SMe0oDq|8DZk)f<446 z=wmJHD9JY;eWl}UK&lBV7%V)z(eO)Bc*RX@v?cp1*373B0LM8jk^rLQmhE}tsoeyJ z7Zhf_OGniMPHMik%~_FE8xz~G(+y_fV3Zj2=9L&TOsg@GyH_w1z|hBxb!+!5HFU(B ziNs*C_l9P$FG3SE?5e{z>{Z|xjN;;G6l()#8nen=N}VaRK6&vNuVvxC4>qvBLVRDE z4YH37De2cohlYp;(*k;XsRF~iWPuxADd3J@`mrZIXR8K!aEd{OcC5V+uWUQfGOVAz z5ZXL_Q+h{yDhX$0k`#_}Y9MgJMfS{L>>j^x%^AOZ9Ns+h9zk>KIA(d+6#hAs$%_Pi z$Ao4MepTsW=vNgjX#K?cK4?ZDaI5yJ4&0fo-M_*S7#yV=9~gu4wuab{;#ww%!eL!s zsE?8Y*sa$}ijOMYM1?EfARbzMkQwda3vbStbl{o$P&vV)KAroL+R~6Dp}xStL`c1! z630qAmdaH$_K`oEzVA}&@CP;q@wVviRKNU`5JQ`FLw||zxUi^iqp)}90TWee7JvY{ zR1l+BU#+Zb;3(6O$%dzdFGZM!GoDlS)q%PJ>i_KNt}x4CA{6Y*Ol$MGT9gHENE*sY z`tMCraoZlsfwxFDlx9*#2OA6KAmJMmun)(jOYz}l7GPD{k;{e>iUpDb`TTuwcnE$i zk)pt_ z5`sjY#hG}|yqGYQO=Wtx&$jF{^nME0_d++R7ZKvZjJ5iW-V@!4O_D5D$tHFHJxiCo zuqHJcLP}$7&s)@$COq0N&lHRMzauk|<%!RvORly1Ftd1-j=j;aUy5i}5qpC1od|@M zvkAEJ_k-?;)O0EBmi0?JWX2Jw_uM*_$jNP>4a$dFo*7X!GEDOmyP#_1vW~J@Vj?oC z9x~KLweJ?p&@0B9MVy^onUindvOVV2o)e3RwCyGjif_d({%q#zFg0LOuiD{lOFV&KxN}EX{E#yNy zq6DM8OC0duO*f{acFmk=Hi zj(zZgv9s;jJr@a}9KkR{TlHZ!X6NkM(-u8iWjGI}*lv!gzjY_p)nbgK@F+1-oIibz zwN*03TsFpG5CZpuTf|UfeX?l`!bT$7lGtyG>aq8OEy)A4?~&*YH-zONr%+AFYh{a_ zV4#xUw8#6=|7zFUHz%I~_yi;@qFhl}6ec)gqX6~HI-6S%jk}AB!vHPbI|+eNhr;tU zr9LK|UMC}^7h_5ulvEU~wupsJ^2vRW;ezZ)>RI_PwuteJV}3ZQoKSOTP zi%6EJ9sS73j+luzUU2q*P1QD8QKw zbl&++AzTQ5SFq}qv$;4w45~70THd9~{$88#QPyZ`y}Q;0BugIcjfUx}oHT}-e;Jwy zQz!?5%0=jNfg?6?2+D~O>^(c2ev4rDVN~9b%q1Wf(T6I^iJAkJt_~~Sz1<&~qFk)C zI*?Pyw$_yrkTSO;4~wIiD*s@R{Y1(~u@~RZ3X#)%JZgj{t3c&-FkFPhH6UeGuFH=9 z%5gZC&t`=Kpamn#cfsA0-eUq)s8>Xdu#5z@4DE5B`Xy@h_P75bS$HG=^I<+RL92GG zZe2rJq%xmPv5+l;metN+G>-PDTNZ}kL8y}{4(p9!PArY(xs4n*KyQ;@>DfHWEw-_i z&%#vZT+H07V84S^%DifnMQ6nJM+Rn3reBLQ_~pkLsw&HYyi&X%11#yvS)U+fcKS zvp+(_7dhcv>Kvw$lu;7#qxcE94kl4C;lae)_tQ`OeZg{-0;M;_s#8m~vjVt4fOGyI zPaVr#K+b1YP~{D;<&d!K9>b13t#1>ru(t##P-fmohi9lXDVpk!MSkBQ*;ERzxm{|Z zpxP4fM{5XBl9eXKp5yMDRf(@hk#9r08Ly5G$!uN7vJhpX7`B@9jbnDj_RgLMu}nTi zymoui-@$V0lY5ueSDB&Pxvvfvy$Y({#62(>AT6uTl}4w4iV&;XtXvv9phg}s5*Ma9 z4m_FQqEm|(jbTAPO9K&0l^)j-eM@;>aloP9W;pCL+k2*HLE-*nLb6|_Aa)-71I{Nl zbf}&smC=kup8E?^8hC`3VCM^NGMi^yV{n8T_1nrkjP5z0(Ll1dV*Uan;Cnax-Z z7>f+_lSSvQA!={AdpLNfk%p6-tT`W%=KYc7LCA7P^#`k*aCCg;7ou~k`G=1&D`ZtY%p$Yx&QGG62pJTApeP5 zm_PdzNW(gu6sTfA!2Z7JXe44hhWdBO{uS-}|Lo>cGeX1vyW391?gRq%Pk%EFEqgNL zze{)?Mrsg*zq=Z}kuR{H-9@^P8N>>Vm>GfCJct?B7-$Rtm+zI4Dn;}%G`rxiODe*1 z!93!*d-SES@S&m7DiR+NjPGz3NCS|gl!UL!@seQA9$GdLS~lclIM^unPKqKy&X;(dSTs$G?GUR$ zz|3Zzp*J%JaC4k)+|jLX|AC!~J-M$iy{MPb19#GET!ZnuSFze)-fCQpUL@Y`hx+a+ zb3Ge4E+-vulF4j8V^^AvDb&>?X_5U9K4vCuC8LwIfj%pg?Z{rA6_afg>Fx_$3>h{B z0|6U{+-=S|{H0aCu`ai-KkyfU^6$Jv>3$ad#|Z^-fWua^BkO(s(bTNcHJiV^#s<{T zq$;6%A8~Ov3IXd-ycnHtc-3ipSq6TDZoUs$sb1q`{B#BUz??JRXpy?TY6)s?v)h6P zysde?L5o~a}iNU1xh~kn2Cyr!howBb>;{xI3sh(F1i8heQsV> zux+OT0NgE#)+iEYuDVK__B5wVz1|n@%`##Nw|=xL7<+^PFRSQ)>2koHNy_auozX-f z|2Kc6A^qwfoZrpE#Xx;df0}^o^N#p$4xMSdwNWum+fJ_8Wv&aq#0hmb9XHTKi`TQu zCvYm~)1oC8+*Pn~<&`5n)j4f>l%;=vG}9fE0}SX=UPDL{)PG7^p9btN4%~<*J@(b_ zp`inP9=fLmFI@SL2{`Oau;j#lYPM8puwj05SSx;Wx}P8I`U%`av#@F)PZgv z+Mbdo97!OgfDrIGf+%VgV?pdTt0gJWikxTGq1pQxMD}-p;g&>nCoY4n90l6CI3cpb z0Sprr9760#dt=rOt@v{n&x;sB5x@3(5I?tjc{-UxCea&K%+AWw^y?rQ*I8NGV;SLN z8Rv~PUy^oHAiBx#L-XLr#Dvt413Ub?vuSSC1 zL#mP0&-_lckR zbl-*QQMw|t>q^32nrxi1*T~0OGjBetV_?YiYh5L3`l*WYRdU-Edc1=_qDTL7lByt3 zhYcshTh#CS!7UL|>N3FMLW`$dTJ>E^nND&m4$|?QcV9nEblhZ-A2bkcW}i_L46>D-DjBI<_bo(Skm)zhN~Qr?l=ux?3>vI7db-iq z8+sK}xAdd}tGp5J#Hzj^e-d_e788#UF3hIvtVQ`n6|E>*%_a#zoX^*;OmU9Ut!v1? zy420XNd*TZ`fDLIJ#bn<;r}ij`C`QW;@7-o5I-|;3au6i{7ES%a^^qNg=^+{BG-3S`LdEa84m7Y}11Ue5#&60oj*V=Ldw_OBYs*>|J5_Z5(% zYFXzb!FQ#f>XTWw5P-N!v~pwI)26W6&~M!F$QWsJD;o2VL%k%TcLR_ilyaN?EKb&E zC2o6G5tq)EcE8p6Rl?@sGz0YKyPoYwM|AHbNZ<3eGRX@VEViF}ph}bLHc zLoa*JzK3cPR;=0DF%D<|Pp~i=_=%k!Ov$P>pt)Kgdrz~5Q#5eeDdSvYrd6^8GBI|| zqO;Ou&?FFHPSc1Y7v(0CtEy$_bOPiqv$e{EEMS+>aWqhx(^;iT;&e*swKrg?PgB3w znJ~S9{VOH^_tgJaFaBS7$RDHvarw;t2{ICQBvPPhfI$DD&>@DDoaOm5lx!T_hNbq6kwa;585_OKsvfI-@oL z%1-rAp%>mH1sSD=b_0kUSe|o6x%Q%jQ%y35ivf}b46?^}6o5H~W%m(Zd0)IS$j6t+ zG1b2MZ{BBSl);KGE>TV73P2~2VprLRCu~(~Ydi{)eteYoc=Q{=B`4b0!jn=z-kKVg zHgplDl<_a>^XyueK2*)ker8u^peXQ%_$I-sbkSKo2$U*ptN{#B9_}cI0m&6Xt8w?( zH6O7TCnnT8lq``cbjcAjVL0T*ptcx#wYrVWjsuHBgW#~3bIpx`@nP_G^Al=QsT$>0 zjz12Heh&(we5E&dS-hvoXg-&3z+q!wNH80_F86)^nt#%iQ^U6eYkf;ZM0aFdkj_%y z7jv{jPq)Ci2?qFVv)x!G{TjKT2BQ!{-C1{$S!DUu{*Kpqx>-_nzHxw&G|dJ~2dRh<^p|KSJ{VnWs1>svW3* zB#F^AFT3({4&>oW2J$%EClISS1XOwkC47cn1P@fkye#A3zYd#|Hx3OvI}gAp>LGMx*(!>bv`2IODpKwCd5%CG%g^b}%^?~*GFHs20nP6nY@s_N`|)_Ab=_Cn>mCk%Q7jaIvh zV{P}9#PuvuMMzT!-G9^KvXzSC=x9`Jyfh>4*)+Bm6J?d}v&+`0f$iCPr z_3Iaod*yGYab~cSNg_~!M{L&w{4-YYpM(5wk@`DV)M2m5s0spB^mlpE1(6DhJSX}Y zE3jmNltX;^B8d4H{*$;64KJe}=hI3xlJh`jG$irB0hTnaHBl|m0t|#ejK9P=3)_(w zbC3yba`s2l;lxE2Yw6iUm_d=IWN~E?EiOAO*L$>RoQ=MN@K-=6a#u{QyoUr_AfBA~ zQ;MQACyLE2YLExQEZH=d}`^!_hms3su#`sv@UaPW~pPry-I&J&T>lUOE(l8P}6q(xA7t^XWX%gVBwKBLE!)ynw2SBe7H{47_H&jv$an{dJ1FmG)GG09sLw^7p@bL&OL@A)kyIP+L>5ijbkf3e>L$;M{mL{OpT#LCh~8F;6g}q=6AWj^ zcrp;z4*gh;0XLn7_?6iN#)Z((t*c@F6A6gm>1}L{YhHP}ATnfYDx^_X;kImgkdBK_ZVr zI`pHpH?WI=i@LhHG&e5}rCgmm8@gBKl46NYisjs&BL>bEQ&m`lp2w$I-qUx1gD$Zz z6C<~H>>(x`Yk648&}B7t(+z>voZ8C+0>IemI;G_UIb2wD(lS%jOWY}i8d6iq9G)_~ zoGN{BQB2TBR(+;+@@LS2h;u8<^P`_F4U|=dN#VMC{WkN*y528~II}uSDs5KLw@6}$ zietCt+f@N3)YRhe+A3z;ZOcp@g;ec|l}KslKX9gAiIi&PX@TY9JqBvU>0*v`t$@G~ z=1;0b;(8sE{D$_lNdh{L}Z#Pnn|fu|D-XHO5Qi^fZp zxX6|JU8t*1Ax+sGSi1fJlSdU5iEF)L!8Y&wZ8+XitM_#FfdxcKSrS$rgW(L(Dh577 z)7l?x0|;=z@paI@K83Ho*HMA~PijJl0;a1uqnI3i)Vm0MCP+G)^Kxo1^8gtEPe?=9 zCx{)F{vGE`Pww3-@cY4;-SMjy!#t&5*QH4(oq@=?2RfI~Si|05M{hy`U%dKvro^iEip@e!yreIV+z~1x0Ci<$kQ( zR@((b=Fy>5vTx{6;XQS&jDYs)LX0xc9S_gGx8`4-CXSb$B5OPp=}&MpzcY2`XHWJU zj+@;zVw4FWCXhWZGfZ`-m2_s+iF(J~kE~)ZhcfY^Xp+ZiIcG$(i*6U6)YrMaYZL#} zJ%glJ>cRhi=j38Ws{PY90MrX(wmSF(Pi=tw8Cd zB#D+yL)g8=65+=k8oHuVR&6!kHkbyV@PnzU6K|SrQ}6_SCCLpBil-4JhY@DnHMc5! z+&Rb(j0m8dX^(Y0ZVy0$iMGn&_6LxIBO;r7(I-HH9lN=MQpxkZ=>(c{$%tW7(E`DU zA0eAUCwY(>Ro;#Kk{qR!68&;xTbCctxKg_HzGo9)!t&5-S2)!tpB>KbIPk z1fn!!KNlS3375z%-iYE5>Cum?H_!*fIqrn;CT}*B!nAmGXf8mnVpBFvL<;7EVpI4j z*z&aTmLtK{Rpn74s#Za5Hp=QX6}T^}O(kBb79siSC>2^jFs~S2F3BAbtI&hT>OGY< z;J&C}7a=oyRYB#e$hegJe2D2{$z<+rps!N_+;yzeREhVGvzvBLvc{xqwfXNh2n~3p zcsA8CyGxSVp#}gkdInVP;wTeO6=|5TnEDq~3V()_b(ognsyebP95AP3BRr`+&nz-c zEmu*tjBx*n#Y`~4*abDsg(qy+{B0}^Du;PH^d$vacn)P8s9$pWAoSMffoL zP01|Q>_;WFC?XqW-6Sn!4F}h(Ff7Y%Y?cvQ+|P1EMuJ63ERs#b>n0LeKp{&i!x5*G z@@ZAEL}Zb+vu&03jMCguK7B2x8J4YflRF?jkTtm7hkRnxSUx$4MTVG&*zVZwXqyo@ zD)0ongB!u@WF3!+Apj{D0R0?JO_=_fY|db+*?RM4Ixl|aBq7+K8S*?oa5_zPZ9&mN zT|q#LVDL`rtBBnE9Nka413T$wH6Ok)B^$I-Y?V()JZ^jk^zgFMFty@WM$ka1nHJ4K zsbOvVcGrQ$(+j36mR`s>0_(e3;AlaO7>x)$*jcc{vSsnmF@v&SAALtF)3M^r(vNOLwdRD={ATg3U~rYEJnMg*{b z?|a^UX#|9k2i2Cc0H~dAnMQ}~qan3o%>3_syYgE7-#_d>8sD{+N+tqg818=x$4`wp zU;Gx$=B-V7CJSe67=}H3Eq*Zyv6ym%aBn+4u9xVM{|Jt6WeRtM`=R@Uq3ONsYvEvw zRiLHS&kn1u*Wm;jgW<5B9!Bm27)7|88szqyj(dG_-=$yU1)yA7bY#6+ZiDI)<*oEM zm8-JGyY@Pu?h{zEaq^^~V2xvTU$qOx2izz_;mu%n>wSEs)PoXz19Jk23HrYDCa^O~ zW>^+b>VyVb3NYmg7iOZ~=|l^sx+|?p82UpspE`O?eUQ@nD#kA^pydayOb1{7F7$_; zg)!+ZZ<$RY=`jTqye~-nkYSJ226#c0JHmw|dqHDKrTfc%(yHQ8TSgC8MgIYwp64h; zr7J8S-)y0$3?tSulVQ{@Sf@U0+a-s81z{L-jN%aE2LK^F zwFe){usCB;0s+(b_JnYN93-0}6I%~Rcex95S&Slz>)5dz^!UY)$zW*STZa0VX(G3) zpVSWBT{V^#QKtyVCCg9u@II3h_W0E2HlvFX)!DlHp{nWOvh|cxS(&nAZS=MD!-IeI znxEpoH1dBdwkh@8*RP*lIkUK5zDWP|Woa|2e!`Ogw7iYbK6$eJ$2)j)rhZuv=Qs^b zG;tF_O9h4o6GP*`5rvA*MyG5^vovPG_p}5>>ZsLf)aiUS7bDN%*wsy-qgW&Rf;H&W zw5Jg~s`#z}36^!p{C9p>aC#Qsj!ygCO}%XY-oE<1{jz-yYgr)h)rdmY)r7!7rXSO( z1+`-r5CfeVql@Z{>!8@5;S>?dJAAKiayV(?)`7HoB8GFBbe$$fwr4;o(;W&3p6Zk>`f!F={j-H2iJ<0=R#J%={-C$y+UUedsta1&yDA!BBCC^V*A~ zFN$*LKpI+++=K-IW=_=PuyOS+GPDGtl{YB{V2Z+%dpxutvftV1C#%;-&p?TC1E(P4 zwH&4W6tDHB5542M|8kt{Bs1s=cYD}~!_peqCugmKJ#WiFLoZWgIEM>k8&_D)naX@E zp?gQ=4bRDVogR6z~Q@0XEGTzSpppuD?{Q<@fo9)cJ82M&2<)X^9jCl329nJ*Nm zZ7Y(t+Jp5Qn$6gX*RZ6)FCUXr9Q3Ut0HwV_^BV5L2|Sp8IqMJE?=XL;?`uq*!sYxi zB4V+!czE<%S*+q~(>5`g#?HuWr`ZgMuHva{U{_XB>c#8AyQ%aTmtR2SqcNiLOLatV z9CWJLa;UP2AA3VF$$F)@2bo4#%!L|U?VPm23m4WUY%wBICv}^P<4k9$SRy^T6WDl3;%L~Wb&(YMVM}hr z=g0b$a)YvPS~P&wED#$=#& z2Cyx;Npi?W_tGoNhxw^_8;TdNfLG*H^(5gsb6BWRLM44Bdab%qr5$cKzY;prf-jw` zD2&WCHSdGf7$Wr)NLK9=nFSJXKO<5|8{3rBRa@p{+J;kPmzbGFLex9v`MW@=Gv^5V z`h+!cuN|Vpwn20fm1@|JqmdC%9>qb~;yX8)cg_-0?V6MLT$dpRg&VrH0H|j1_+M_} ziWDWt9TKmbV+r@stLaz*LN(1(gQffoMr^(G2NTR+!34|h@r!h*bZdi&f5PXc->|;w zjeK_9VBa+LIJO!~&!W>P zJ_s=Pi~LXyo$SQf7y@F4-$^E6vbgz?x#g$L(O+wW@i?-WO4@qQ2+oT5)D3accd7Q-C;J~*ba$4Z)1zSrQfeh9}~0uZWfsQ( zp6BLUbskLI4)$L7?a06%aJ#WIXF}Hs||5R@R zsbc>DB|GH^9i3S%;18*o7Lm6vPt*A}6a4Y8$~ft5b+e`<2Vje}^tJGk3%*WTAH9;8 zRPG}qvai)=TxgCs+b++~IFqHiFT_LwRy~_S^W}c8nE|CZovu!{881%aGFrKP8Z^$J zfY-}W+^9MSxG15{c^p(|1O8B0fN^+Y<7}WuCbi>)AVqk2U7EQdf*y3Z?22kQq*Go$ zuYJ5wdBOQ-5ugzc%NO#KH92h0qQE3Fe3~_P7p193bgSA27|tb1@rZCW@FzP8SfGd2 z3{Nw>LmqPO{Vd6adJuT~ym0uKMrqp>)zK>(^kA?88130_3uyso^W&JqRX0kLB6qqA=arC5)s#K2`I>0) zIDhd=Qp#uo3e{**y8{Szxn~h#T>EZ^rsY_m1lZ zq&6>w;}&x1ZH-jh7p`rvPH)_N=+KHngH;>wHfo%XUF`4GjsdNw@R+Zum=NOU>X~I z2-mz8A@#(FcS2~K8%ld3zW&52mDWgnAu3ngrYYPs7_I-w2gl~x50;OaDXeNk8hW>} zciOKd9A3CM)#H?sLpZ?2DK35Ja2)LHiJ2ZLO+vfSODMeTurCiM>#h5nH<+DMFdT5M zGESehAeNKUL++d+;~gu^Ipevp!Q9th+KG?4>BqCi)C8J>An~A&`qP_y1ZhTdaJoT+ zMKf{uj4A3R5s1qa#%V&aLPhtcHIWl}_2txy_0^A6gfEhQv;V=5_VRGUN$@)a;`Q0= zNJkD{JsF=A>KjIMC@{!MhaeDBfE7TGnCUm1MVh9SKq>G$r`JX1ntwkeO>1Hh-zrq% z3d|6@u|OQbS;$Rl^_ZX_hu&tC;)ztPSH@=Ohjk3h*C$$vkqRf0XmN$0@vmn?-k+=` z%V80l?FxS3bL~Bw?N181gb^v3n^DwaWJxPw7yPsBrZk^f`D87oxTN6+ha7Rw`lAfK8o#GALWGpjc`jL%;j>wI4~!& zCnsslu}JS5I_eYt*pVlSrDsw2)euYoV*T!K6V#osvnPl1U%yoq1Oh?*81sK2?lbK~ zp!x~slV)Xj<8JRJWWDMO6I}wt^T!K7agl^j50b>n?jioPW7_@tX`F3O5e{`u)l*S& zo&RV)cLG7(|FBh*B#X%g(rA&h0 zS%L{{?;v`w;;~1Jb4AxIFweB#dgK zsRk^o@_&@QpqRZ8{w=}$cg*mgXyR{XJa~4~ne?-Ya`74c^ZuI|Z_MU|&uCnPKmfS@ z8}a9<+B(h~edfj)XcuX?Di+2!#nt*{vqoh1^+cAFg|T#Xh;C4+5Ar;y>%4WPUX>{v zWc7-p{?Zrl(RX4AT7i7BCmi)2c`GOYUSB_%mQ?wsMMkn?WwjD|`mtCE`Bt{7w!R8T%vFmEJe9dUPHI9RrB^&m1QQjYR8<%ngc7IHb z@48Wu8Q&w;Q%r45s5@h6Vd_nkg%P}=$-&Ll(D;T(QTQ_s_c4HXT|6fWBr4Um-dBqu z1I!)Px;4%=cs%tsFMxc0j9}lQb!UlQ*=p?4H;JB&Ir;FmWx=9W0qXmW^rMABwJL1~ z6WBRrQ0Q8*7$Qe<9ndwj11nmiPKi0zX1eZ&32qchXCTVkq*)Lr*!F)Ur;To%VcO`TqTdKnRvWvj_vf_ylcg5>UKGpWSI?09}mG0D&w@63Rd(x*Bqc z^k`C0SSE5jU~AHCI1q${vFpz4En16@X6qO|r(4Xe7S?eSkA<&MNPVnc$Qla4-a>q~ ziVZw~t3e6JZ|gjQ7kAA!mT=#gcZ~^5A-COFW)4`)OCWrfu)hM3Q|#IR+aB5U@7ZH4lgo?F_t|BQ1p93C!i_-#F-VAO8% zCnX>8qJ>s?N6KD@)0#@z9g}IUWw=?d9zY4_H|vvW>rf++u@+y}JyW==T9VC$jqu;?1|0qg~xC)=L(k7+acOXNCY zJ-q4Z+zi*wSP~%**K)~M)N>wiM`u=@FA){IwZaa`@JTDAD(S}N(hqRL-bN-m3IM#8 zvZ$~2N-^2lOPCiYp65RYJxy?^fKIwmLr7ZM8z{J9>qOI5x!I|8ga-z~8;^Nr=Q7j{ z8;8@ldo!=86(^ECt+hYMwk%-g;nb;1X6A3X4C%8*x!twTvv5;bx+**a&ld2vdJ~|5 z4_p_o+~J&=&93p&yztI4HU!-2XMiveo)Z&NM4 z;gdW%!EfH-GTxX>uWa)CUbVd5BW-&|*bf%fw6b_X(T!Fi+4U(E@btKqJPx$qXW0)Q zWNN4xCJtU9p>B1iS3~xf;RJnFaS&M+Gj8+Tur3)k_UE;ewUkMg?&(-IGc8UwKjF+s zIyI2>?%;w;MkUjs&Q!hYg?Sem@8LbJdbkLjv(O@C_PARq|n=<@{mg)Tc zTwU|A;;qtRZjKyR$pC9024iQ!%`SC&1$Z@alt+byRHZFY{P#1KSq`|_kp-xBRKy~k z!L+G>J+i>t1EI`MyiMLZRb1J1507jhJ7d1wLo1~zK3q;rCvRQ2G@#HCz3}7t1Jr*x zvn2mr@=OR22&JYOGx|k{{~M&4q9VxJLXFVlJyRn8H@suP70Df)Q0eX@35qH7(-pDo z@Bxx3;I;TNdj94XnBHt=C+gzu_sa}XhE#xmL ztzz7ak?LJsL(|p3IKYs6`r1q;q~P48MBi83f!=auwdc=C=RRdH?7l#mQY0Wd8%h(& zOnJOXA7bk@7%)L=cF7NS;80nO6|Yl#F$_+B>XQ*D^W<(pQ`*o-udr~OGgzE+md1kY z+iJu($U9J^R0ulOD5~&Jwq)bIahqXDd1LE2ga$>0w!v>8Js`Rdm(w}I2!Tzj(;qH2Rh{%CD)9=u=^u+U-3IdPx!{U>}Jus z=jn&hia7&GsV`*FcCbcPTF+1It{C}MI{zm;-NrNx@qkn60`K>u@?AjwQ^#B<#>DB1 zl~UsyffFGvxL}A=ioof25!cdF2?p^4z9}X<}Ti%jt*|-UT*(m=+!S2Q2){& zrw23zgB1Hk!jRA_VO;Z)lfh)OSwpFiGr!|_P@22*>RNVKeLO)4#860}$oqXEoba;j zks4?n7Nqc4*y3$D+T?dVoc@teVE7$WYdD0CWvqg!KpY2Fd#DjmSt0@<%e0`SU`!sS z9{wwvG|wn+k4e*mPc@pzDqH3$-k(C^a#ttFAZuDCY1h zgQ!VJC0vNPWON-s!d|h`(r>0Lzo^;L*)xMyvZG{Ltnumz&Z1)OHc%72p>47HFq{(f zhJh+s)xBPDipf)=1Pg+!jaEcimUoD0ZHh1aK4*+5d{ev5?D(kt;yAO1t+9*MXEL$6 zL(VMo(w+@?EX|qoKq@JCZ`OcMN;gjyW-Aeb@o4Hzq}Kr0jz1yVXz)!=b!&0TV+fZU zu96{OOEt1P1kjxUPBRQdtf{xd4-~>OXb*f7G}*_GGEG?M&Z=N@ zWf-pM5)w&6;nKSFG^S(<@wCas+V|O0$@96on<;o}N66*b62%39BAj&<#}MJpvUd)+ zoq@d6o9qB5#NXKaGn^gRFn~64rXL=)wJDudB}E=OziqqfmsSq+rDqKKw(bdZRV=42 zgtB#Q3ygd3^h~l8QsC|xg#pJ{UeMDD)C3VkT6SL*orKf zgywu;V}x@NT~EBRL10UaV?(V#F_~rj@bw1|u6+RzBTVrEa-qyIJ``0)FvHAp$fq>* zbGUUU$PZ_7pgTyu1Bc&<1e_y>MvUzGtLy&-yygGaGk&f>F^;1Krd$HCN|&kqnYsrvy9=^d z^Gk%*y5-0^Aq#2YugY4pdXHf2eVVAzfhg8^27Ogj$|sQMN1;hv&%W&Y{rK(o=0b<> zN5G#^!!Hbj%*w-<6cEfBdq%icnVo?#6f#&V+0fs*%>es+3};Je0=*&9sH&Yh18f#@ zdO&wjL!3_ILTaXC_n%tTZtU?MpoSwL85_c8v{~i89&{;Gn`uoKHADHDQKY6a@;ep8 z&huK`d4RWa@INA)(ks6x;x<6q9{oL6+Ip;!)(uK;+1hg?^7>nH))wOzG4OpYLC{NA zz@yvddH^70U1J$y%Lu7@mmVBuy_XK5i*RkEGw@g4&V>hh+AV*NU5G9#{`iBi$3 zk~AdPT|!)=X&XrmtK>{MnCRU$ljST#uX7y}L)QlvRMPG+`m-N{O+@k~fmYC1+_oTZ z3a36h^j*B#OKan4p814pW)~Mqy>3?JsP3kU+WmTJ|o;Uen4XsaY1Q>J8*USRUPLm?IvJmO? zwMI+*_OP~<(0bha%&=;PO5dBE4SmSG_o)WJ81SKi03J^ct& zO8}ieGH4xSH(B*Z%(GG@b4HS0%9f*X=Bm%}9FL1Sb#uPGo-=c1zaVMSnp{ME+1(bV zs4jm@;#a-aR?!IG$``&KzmUmF9-XYDl5lQ0WD-b-c{{IZJ2K4BL+SbI3-(EqwI9T3 z_UtGRAli2AGqJsyQ6?ea>Xz*h;P~OzAOZZ~VKbNbCyv~tIPs211Roje2%XwO3F-q8 z&k~4F@nBx4mOsQ!HbC$BB(4Nv4Xqenv@qyYLdI7NXKK`Wxb$Z(tCnP<|5A_+Z7@?Jrp}wYy(Dy+Oh3H6fT$+IO0i z7EHgr<13!B+^~NnrhoiXH~&;jvj3}@|IyjMfy(o)NI&xDrDpXx6VLm%7LE@Pc6YOK zbg_0bcC+>{|Ge7%r!a18YGwXE8myvg`+2jyD=^4si`mzx+uGLIhpEkx?;jA0M>dJb zN)d9Poo|@2lIe1`H1k3~75FtiDJlI0oE>6KVNYU{xHh)5n903zna(8?{2l;#jfMy_ zW%Uj^MT8OO?ura+{&;_Uvb#=-p)EE_PwVUveoIF!pEUU2ct*SI$A zSa(>~Ewt9s4>g;0NCH{I6~WtC^y=FpG~mI?WatdK%xo7Au52FOL1_ssAGJhoaDE7g9wmbV=Ku=V0RZHzM!oU|a+e#l8pm1kaC~a78#> z=ol)b{13niT56(bej{6x`=&q&fuOZH#||EYnwLn+Zr8r!6w)0Uqf_jgbLkiW;cCeD zaS!}B7m+&EE*aWkAEu#98Kho^D}gi}VWO@+_YnDkFhFba&TjyxDthZC-~uMQAj_eTFu=b$8qfavZG;nG}Zx4WI3X2v^P zSnZEGhJ+r|8~hwlwC}_ZmmnvMW4XFn&{;_vt=GQ)#>4+DO#czHzgL4-GGrR$rQqS9*V(n!AQQWz2a z!YF ziS(O-{K&+j`-S6){x-h#R}wQ=$p;gG#BH?r1C6UL{&U!bh~kYi5d_-lI`fCyfspn4 zsy{^p_aAMmaa>Omh<7r6_G7?nUSQe=tnQ7PtP3~#4^ieeulR4I6rB;ai?b!iL#P77 zcr`zZ=yA%FQ&eWPGLujAwm#5m((3EGIk|{N-kyS1zUgI zJtl8N&l)Ix(1dK?`y-$G@WCzVW3`ZM?IksmhR1ceFx$PK{r-5Fmh0ryFsjkxwwl6x z2qj+G#O3o@x@a|ENj*YbFx%}E_`UC07Qp}*vVQb=kodMzvod|BQQ-~C{Qk|lc5d^) z8X>ExIA4_5tSJ!WMBL;mkqnHjFEG!45+Jf)@+Y|kA)!qZ zEN6fM$f%$QA^su9L6%ZSQR;_?943@fWT%cq7G(-h7H@`h?q90sNgSE6xAq7cnGo0~ zsF$94ING45BY>D{5}0hXK=Go6YH4BFS=ur1Q1FB3Tn{%Q}G zamI=ejv@*4g%xS1Rj8G`utqIX(v3Rg7V*si=$vO>s>&%a7E6~FZ4AD0GfOejt^TP<6Hn7-@pf=Fe`lI!2+t@(S6u_Jl72 zijv2a%E)zZvSRgIE7@e;f-`471y(mY4LooP#B;$|hXiHZLUVHt9G3r--sPvTK(; z;MY_`?d+9}!yzE6Db~Ozt5#nv>wVXuGntEibyac_!&MlW+wM5?F6sMGeny&zGL;^J z+gC~+mPe9EmY)rJqJp8Jw3iqow_(LzMGKGM$yT&S6-HW3m#2^LVMnreMG$p-@VDaJ z0*GLP>^cf!*|NzWhgUoFk-&$hrdySh{nBa+?p&+T(d=>156)S2ej6-<74Xyo7DhGy z05T&N{fx9;0uvK#08zFTL~?byYi`u`hE1|2&D5V>r(kOANjqDL#bVB(H`Ch!B}{ue z>!7$>WkVcI8$XDD+|GOH@u&6h7;9Fy&%$s?E+Qdh7fU=o$;T2*FQCi2L!F+fX$TcX zDl${ypycOtChPw}{LdEY{^!f&@eLP$ze;taf&W&dAW56MgF{ctC;L`K z%_-%AO{>lOroYb7^7Yg|PCW1j^{y_;$o@6;_j5wGXz|BgFj2NPwA&YNh z%A7O{G5}W7qOD_hsjZ=_Jp!ifgyl$8Yqzs$tz%bZ*HY6}(-O5pFWu*|mz6bPN}iDi zaCF)8dd+^!InH(2y_+sWz$Y%$G7ibQdUB*nC1^iK2+oQ~H(bQO6v<=J^Uy`jy}TI6QXK-`zi!)892tad=Gd@bpTv zy1l=d>jr$l-0q(47beSGT}upvo2U=aa?IK?oT>c#A*fsU-%y2VhOD_+8kn=`^? zzX8zc1{S%NaLak1`4crpoF1;*-WdnEydX{aPC;}V8)*$E?FU*>uYV$uXerebS(~Lp znXQJ42F5ELtL2y>t)JYo@wf49BZZSkfW2+}^r^eIY@S9BA(h>J{-|{vRE@*@eZdjv zs2xYt(7!gG$1W7>*t~TnQTdQHThrjV=nGhh5LGOWJswB%*Cvo7EQ+~F5%=I+vB?z; zRE6^!qs2}#Dka0?DV2uBYp{%)=oOCV5lN_{3my;Z=TU2Yi9E@`4h_`xG@8tDYX<}!el+uAuL z9azRqF1vsIxU#?-KL&gvq6GNK-?MBC+|rxQq{F(Jo7 zw6dRH`@Q(<1wRS?5psz&T4R^aZ~sUvgkf1wPDcp0Oi&&tC3QVF%yTlxs~^CMH1M1PYY0>+r7m1p$)B(tg-;i(()U^9mW^6sTav6=37Dg- znP&w9N(4#rmtRP&pBTSXtD{HX#%4{Tu1(8fMnN@Ssxd&%UQBYF8NZwXm7DeN zE+I91Pz5I)uj{zUO-x9)hD{6%7&w1kNL@4qEQ$R=ZuvLxzqI-uE!E=k*Nr~WPCYuZ z*J49vx(+fOSS+DMdOo+oKyHyUqVd>CgLjRC>1;RU4={3Q_aC-4G^SZmahX)XCOKew zktXKsMkb|-aIb(jkg=KMbJd!ipEjt$UMWF0A}3GjxX4A3WP>MmwGUtq_ngEByNr7i zN9yp6pS4;RPcLSjT^d!R;l6lH%#LRZV3; z<>Iv@xXaNF7r!DPp(ef%00LoU$QSwG&I{7oV3XF0>VW|Dz&?}Pf<)F0d@>)2piFHA6f(_G-7IQY+Tb(xnJbYzyXs8;`@fELLraOQObL9(r~39w^`Gigz4di17{r zoh;O|)MvX;o0j$%T87Imq+uKXTZo zOjfBH5vf>5cHzr=UqObv0s=$)VrFuFSNrhr2Np2W%O&FSoR>w!%{=S^ffy|!7%mT# z2!t0=ofE~$FzOGqfE;2OA&g3gtmNbv-~(C6Cg@5mC(b_mb9Q7MA&#;X5e;%bUAHce zR~YCcc~}UTRzT9kMJ$I~7>Fqp9--FQAa=MGsh{lo(mpkUvRZ9u-h-WBt_F$SSwH-=~vO zRa60M1Nz#VMx9{GAKV57tOce}8+Kea2>3(K>riwJw8sF)3mjt27Xwm)GFnqez4?>- zm$6Io0uODG*O~%PiH!R^R{qA(jqKHxYwA<;&?{BU+LDO$)ZvhZJt9|5vLSi zX%YN@Tz3$bimU;yk#>^a%v`9AM*&+l$-*qtI~sGG@Ck6E3+X?f4e?OiX}zWZqxA?2 zoQC2Y%4ZDLc!xSggHZ?vTn;Nv<#_9m);$xHOfQj^DA5nK#w>=s05ct zVTn8}#$)@tSjY(7N4(IZTlP)Dk7Jlc^4sai(YDCCxKic9bTt*VvAEPPz%nPNxG)oDNa(HNhl)4kQu0|rW+C7kzX zG8R?Y?|iGuazVMfM&UlXTYs>cgj=l*e8UyF7HUYlQmpLTKR~=T?YQoRiLn;y0wZrF zWVXK4&DlO|4y5y5wHt3=%PM*|{oxRcC*o*KQs7;6wfMSYAS#UVSF%z8d%;rx7T2A? ztS7zg9G#}dTCD>c^|ws4UUL|N+g@>lwws_dTeB**&Ns%JRFNW?VDUY~voAucrx{uk zPWQdW=ndnMS&||0qSgI|CYZ;N>FH^neisVn1A5(*9<77PUTRpiavP<@k{$Up;xqG; z>bB5=$t&f{4pzAsoR?F8sG~sgUQ$W5GF6o-ex)dR12Yd`5~1qwffbKU6wI0zk5YRi zMS4V9Ml7Rm)rHGEj1@13<~Px&AE%6K2C`nMXd2am2Gs(FmPVVKwGug zaqR%<*om%0rqwIu;tdnFLfkKEUZKbH30ln>X!WYVl}bZDA5l$!nBgShT*x!3Azw`F z9qdfL1@L7@p^;vVVNDZ=5)!u_%n#~s#S-z% zOc%1g@U2(n)T7T7a*}s#)lyjZCJorZ*h4NC$d|OAUn#`(1`KT5`K+#**a3df=JBjE zlPRCn^?mA~FPCP3uN}TCMdq%4Mdni$c(O56Olj>&v%ryYR2CeS5{D)348f8$d0D84Ol%j3OoR>pW#UcX$9AIZ zQ2PE29G0|umDRpcw-HtiFElN)52i=W`^zZyhusl~mW~x*_Y#|s8@5yGI|h2Rk0~^D}jX zOSpKWdy!}BN>`s}y*Z36@C!yLIX}qd7th$4%9U;lwrc91wc9YPZkb5bZwDIhfVEAF z)Fh-X4zstB*y>G*)qVxSNAv54tXCoW)X?k2bngWLK2P57V+->Bv}VUqGm*`e_3ay6 z)6H@zBS(&kG#@2LmOzwWC*VRYHIlZc1ot%8Swj8kM())B{axC9)~Dd1lig&RLE_wS z^J6);it8f`A#K8-H=5l)FgW73i|5F1u{QhZeZXh&OEdvqti2 zZpvQVjP1jt&}ZdV^i|6M3_Em{HMoOu54Sm3A;p2womX_qXvgVQdBXoZ02mCvK3#x{ zPY_|U4}s{qqbqsddE%-N&NW=#J@wDOXsrKcn*VnW+9roV1^xr>pLCQ^47cC&yFNIa z7H5DX2hh>rpk~BH5d$?Eq?86WK-7Hs307p@3yR*Gv7Q1$le(HbYbr1$YWx%=&@W)P zJlsZ5n7f4z$sj!ZaqyJua^p4YlIP`4&}Sjjq79ksZs$NWjpSX4b0V{} z4+w9VhZaRWVXF|Y#^f_BGwpI$B!@*)K(w_DiLdso(p}eT(Yr72(8BPasg^0fskF$L zjXd94j0>8`H+qzs&Q-@d$=r+E@@@#y<8L`?P@a=tzXjx0sLI6D(GV#m`!}d&Ma>Q~ z?YRAN@)OY<@O^g-kJoz1>?OO|Y~=RH2RsJT&O1UCVbd2Xi12W;JrULwU|_ID8-_Bv zNdFG54f0jpCVC>MN{!SSTo!0MlF{bv_N!3)bLReOkT(yS^l5AQ<|S1q1N~zc(Z_DQ z!7Rr8>(N$=etdvBl&QY_>$7LNNse?-QXTMP(ZzC0T$x}m^dT|`3by|GggjqC60j)I zykz0JfC1~wOMrF=9V+~0D{Ja$P+Olhb)DEb=?+?SB?Aq2Ba)abza=SEbc5&xCkBXO zv0FDu3LCWT6)tcXpZ735hxmPFF)kV9Xe>VTK^%febYU>8`L?ovG?}>X#^K1GVxHSJ zpEE5uJ6rJjFL%TL>xIa}-_Wi979am{&^8ThDiILSe~0*UsDdEy{|tfaM9gZwyPbIO zX>l72=xLY9+(dvB^>+;v4b(rOuo;Nppser5FPh(BPft(193tTf{pS-CIUF8;Z~R_+-Y@s| z9DcMD^tB&IC_rJ)UydT-npN?0B$(LajzNjbGcV4KQ;7rQSt^K)`a{?`OhtFXg;EgH z;vI{SgXk@8V()rp5G3TACboZP!$MMib?W^gQrHdj~KWlYLyc_LP(;`{5-M@0DPzy2VJXK%7aH zxr`7M_dBJ?d`kt<>u}|dA#q{o@T#ua)I{PcBqku;y&u)9%PM z)9zRf)9#6J9Z#K69#0KHx?2zB1(bnVS!OTBlJipQ<{0r6Bo&$S=?|aeuIN`Mss2tb zJN=&8=nXW45@v7O95$qSWBIBrR#rZwNlU;)x;3K(0b=jQCdbOe8zM@yFX=C4{?vt# zAlk1nQ{O-ZGv85tfiVo`-eI!Q59v{~X6SY9Qxg$dMT?gBnoHk{GsqOl7`7Fxy|7rAL<&g?0(`q`(o#(O7bsLv>b=Qltk?tgNOd19i+_ z*QNJ9C6MhpTHX)n7+cj5Mv9zfu)ZL1{4{QwCDxlYW_hlV*so?cw9=f#`38-shiEjbDouf}ZJ$3sXsXhtG@H9m}O?s6&)4~DST{<s3b>3x779FRbls%z#6r_8afzGEoJ#tiM|?J7)~pJiohB-HeR)v7_MZ7 z4BB)W!dzOYNmqq^n~`} zG%uMXe=d_Ff_|+>D;N3@B3&ti+!REw6~$C`5PTq(A{;`3I|7|92zhv8jCIWjCyw|m zj83WBnW_zI#39cOXnX*y$PyX)!OF^f-Q4V;>g0y$ANNpmysj8jaW-(;XK0!t)JiL7 z-yv3CxVl-&Z*Nfehymyj_D!d-s}0Ov!%(W4A$Qtf;{#V=S!YI(n)L3W&D%J zm}V;yUOA-2Uz|UexI90a{xzsu;Mg{Ss5SwJI+Q|*T}>%K=e4Q;%(dKTiN&~qo`~Vs zH*J-O>5xPF+|Sues5cz1xElUNZ#B3}hoYcn4*U#JD-w<;JgaOGRfQBV)dyS(R;cPh zLOd93H;-^}0jV!JCsKbJXLKpbDAM63dg4p*NjHLbxW^pbCj@utvt-GL-Vj?z;NA`^ z)Zxz!W-m;$jl@kO&=+7c9q@=L(-GJ9i1#JdR1;lW9E^rr@hwjdvpMT)od;?fB(h5{dY_Izqi19Hai~V?-*_Cdvg>{ae~KAyA{L%OM|z7fJqxK z=fX+5!=<7J;3PX7fJG2cGN^(?AnX?*Ho030fk+IdUS;6raas7TzPu6`^a8hsg@XN{ zAy=m}tv4r%%Oa;Hw`annt1z86A6K{03*Am#!+?dss~CQ~b5nTh>ONIAYxu>reeG;; z@!XsaBknw1-_7#?b}JFi$9D3gfzIy$G%HEaHuOg*U{Iwb_?zzTUK6o*Cn2{+6jAm{ zpA!>uFPHcgb#dmS$>TD<@3?hp zWj?lzJ*%_xucfqIcFV7jUJ`tn9+{E7%-aj9vnaC=_*4XdF~dB{iT=W>f~XDzP7yC6a7Eqx^d)^{X^y^Pwa~(6 zw2Uus9`P4SWKOd;Or~Vxpdz1yIuTX&8~x;K?}=04ZvE|Aaj|Np&)V;lR-8!H}zpgTvT@GImO; zidNR3Xwt#jQ}ME{mg(~uP=f;J0{bNh?sQUX7a|0HXATz^7hE2p@nB?jS~RD*A1$x^ z1Rig{>h}C07~%LL^=XIa!yvBsOdrE%h_mX9O`&ajV-J`julS4{B4(Jg%4~B14PLY6 zV*5Bh*}w*B#MOifEmt2?4>}>dGBx0M1OWTkk1-e3f$91i_MA7dMjg<}fq{}e+HSC! zqHtA>S06@Y=BaR--FDigJZ1apFS^sh1&%knpjOay%QT>F4VK!Y$|e?5^%~DL*fmdR zmu5Y!rCa24UnB7!qpp(E!hRNjHhZZQ>w!jCpMp}JSRtLY+4_m5SgB-uyHd9dkXUZ_ zcg||Vnfmd}6hGogbaKkidMh>9OmWN0mXA>Prf%FOI4G-Cx?4&xWaZ?kHF>ML zM8tU}5UvzjO%!O>KMmGTPwPm|Q4?yHaojgNO`Q?ovP;`%&y^DTC8!hMgvA7WL@-sI zB4&WHn0yO~qo5``Bi(EPPADD^c9rw}^=ShYb%vdf%jinB=-g}^Mk>xNbfPHrUTt>Kil0Gw(?^sYU=J0+n6H@} zj9+Yt1RcGEjiS5-co*b2hpiiiq>$}XJZ4=ri74?d)7Hdx<`>R6YTT!x^7@Kxm6sf1 z);JOiHS1JP9!#=@b7nQh66O`F^9)V)j8$qNju7XoM_)kGO)3OiIa3iRz8wEJvPmY= zmpTdOEpG@U4_+13Kv_VJX+T-U7e^<94ablSe{Mx3)3KETToBZeL&?75)`bUYj6&(X z6S9=x5|`wdUG3(VdP+zH*}eO0r1m-GS!%13%&Q=`6Gb^rKhK{Y>Vjt}>b&XVV`sJB z-KhGpZ{4ZX2%aS!h}?WUV?7?d%4|>#V%ry z3T-b?(kffyrVjIKoJo398FT(BtH+{=BE2Qi?}vtaEU0u`+w7MbHEf?wp0mz2moP>1 zL`$cgI<`F@2JX5nI*+OG`+U8P>{#`*`zWgdy&k{Qcv)Nci#^q$RVB|J?ftWk{m;ef ze=lQgZE)<4KmL=TO;my(g^KPg_?*(U1MzZu?3bWsb=_ogQWn)VX~9{r#7 z4ik`)_zz?WGh~ibK08WgX+G7NDj1nCEwH0nu?iYF5>(m|u{u?|<)J~; zlvJlsPNi^En+<#k%N_xQi(G4DmuZ@f?6!OFKuC2_xuRiCz;WwXf6sqq7Xz=O0O0E( zzo@XmzT^5EKdmS}U5l)*#QFykGcO-J_eZ}<@Qk`$6#F>qm(n`Y9IoE8?SAcMQ14Ld znv>M)Zyw>BcRds{7%Nj*Ra(`Awr?(X7?^RE>KwmI*p7`&tku6|o}q(U?x87KQ#cHF zej2XlE_jymZKVeL_+K&Pn95IGtmdCjLcy}rM%@sAeUtwlpnK1 z5+o{Zh#>?Em@2oGdq^L=DDbYHUdG{r*6#RG-U^)ykdMTJrd1LIM%a0V!w6Pbd4UsD znlB8rI0N*SjvO|8-F6d&KmAr63h?D2!`yvW8FuLPCLBQ}87M}u8e?QrE zXP_FN{s>zQdU;-1Dg|A|l){2a|FP-Czomr#`py3xMzw{?Lx}ySe6S%_CcTCX1XRuZ z9}tG13It@@2{ktoAVnL>Q)i_im&B&Y7lk20Hu}nzJffPDDHu^w7(@%#_}4n3#Q@Z@ z(v^*bXlgWRQV8DUV$7m4w3i~~PxcZy%7S$#P2sJz)MK>G#^#V#K?4{%8J$#m!JoZ= zS-#HBjzn6%H%^b64h!=?87vm&vC~O;yq>|~ClN*jc4dD(0aO<#q`?xL-G##>+588{ z_5`~<{LYb!$`>g71fk6MtoWl|=gztKb0S_9<%lV~ddDaz!=k01ysD3mun0+>-;i+= zKUESCekGahwl%-+U(hD!n*f6YaQ*xCi-(}K3Ue-5J$>|e>7~bgvBYJ@4 z;elr!r_l5-z=3F7x&gU7GK}w`sS|Xa_P8PjQ%kyA&g3#JI)4&Z+S{8$aEZRslW%EU5v=_*Uh&x#n? zoFP>Vm8|s5>E#aDoU!Gy^b2!q7!{+^R_2usx(BjflGSP3yvB7Oq$5wt|nJ-*P=7V&*)dam@f^(Oc7S{X%V>aFzoRM_v-`&UQm zT+Qhyl!!a}Z}Orku#{48@GTN2TBIWf9sysYb}DeJNI2yFeSS9NsNz)UOf*{BzLFB7 zAi~o~fM*?jc{d&M(pDIF>qUgyL5H>Mli$>UjnM^&!lo&!fqB8mRc6Hn~h#_UP`ro4}2z4O3h&4tTY)I z))G@Jo+%Nha9A9hC$rGMx&x`;=Yze$xfxsCfb5PQDZY}@SQvKH7Mgp}x?Dunm}3b$ z5r{dHpB#z0SW;(v>o8r*d_~l^vzLlDSvU>%LDc6#s%HD-!t5vtdI>HFA z=DYQ@zs4Fpn>ij&MzW9(cF^Ui7qOE4DU+O57SA;g*ITOFI0`5l`hJ1O z0{kH^)FCg{RA7-&C?Iqm4jr(T*foZM3>LG3Z!#OY1JEXwJyheK%UYo_v26!x<0|$+ z1*hD&M-RD}W`fA@L@EXfR4t%u@ilo%OB3G^Wy@!0sK(d2nZZ;vZ8i+PbS4cpP;hUCjpA_khH11nEb9#d-kT-9vGDE>Kitetxe!T%&J zB*nZigc5;|W;3R3TtH?H2gb-v%*67s*?a1uPVpd4&7rMOEP$?X))Oeqg{dL>3n0~_ z!Sqcg$pSeZtNlEOBotY$421@6y%%ftbvl(K~J?4NSbM3X6D_r#gfQJ zt>d_AWd#K9(8e{CQS73_o50NP4x-{=KAA?om6=f0S>ur6sYM_&=-_>e-NV7mYA6}N zQq}3=n-$MCrc7?maBRh4uCp$b3%FrDW`q2ry_r+6(c0=+y{EWaY0F|RjW%-WyOWBc zz-;HQS1%ab&4`O<%;A4$ zMM;a8(t1b37PAs8qB*CI0n`>$u{n|T6e>j@V#6(h;X9>}^V2cb3$?T?)g$=ZfJ+06u&KQMM=Q zzA51LxN$VV!MY^ zf02{BLfLx=_j&Vzptfq5NLkyPD7XLall+YH7Zq3?_Bcbp{zBk8NBUs%7RH@Fv+|ZM z%8T@2`^}{yU~-}54G=FffJ}eoRheUYrYZHGScZ}P%BeyA6aq>w(D@$t2@iXQphr}GvB(1Z>*F52GnQ96dE8|GdM&|u3qglSv^aeKCt zzGId+dHf7>EV8j;o0JwNfF${Ps7;1-PQ!=9ehu9|2ACMvqT{Pc>-S=2nVxJlDk)pP zJ)}#dR#8-jaSg{&tDn~$w^E(Kw;pMqMT&zH1MxStzhJb`q@y6ViMyG+4BN`fuc~jT zPxxs^#b(!?R$8&M*DYOT^bDxK@cEP%=iMe}JmNLIEltGqoX@7ZVo`6858_Q&7E7Gs zS3e^S@*5l+}5idKZ1 z`kBmTY-Y9BR(iT^^js+9iugC>Mafi0MzY=rP;H~(Yd1V|r9dXwBbY+kD8@zPc{M^V z;1kmIjxkFTV_MuimY%}KH`I=u`Gjw36E~F}0Iu#_mv2ncyCSN;EiGRd--Qb{k*Ik@ zwabf5gGcRnDcKN=U=t92MI03-NfsqlB+3^|G{m0pwVJ273!BK``-FNvgio+%H&_~M(N@DvKgMxxzk6myjML6Hw+>!G6|=WDJ!EMDkUo3 z$P$4*tYpWe6H=t`o1)nkNJO$o!r%ZDn%+@VC-7ZQHLR{`UeeI-wA7Y&nrIV5BLoRa zhfa7(w!mx=xV_1RCqI~d!h=}j4ESMV<}%5c)#N8K$r<e*jtWjOIuaqAsnwA=PDug~CZ^Gufm+bw{Cen>Sb zBpFXzgyloV2h$DW)9gfSiwRuO%V5QrBoUWcNV#o^OOO_y3XvM9n0Ua?saxo)62!9$ zF#A3pM~+qSPV{z(v-B93&syxVyNN@zaQ^RH=#ugqw;#f zXs%S|^hMfNo3foC@Lh@Wf~o*|QPrH-e9e$~%OU>Sk9k&pVD_9X7%Ray@TH8 z5`4jNQxrMMJu!GYX4ec5-8S_?ga;Y@k__4_BIFD3Wt62kN6e86BO87Phh-t=KDgOI zcBj~p_ZXKuE3TVzom|ez&t_)VZLfl0wOY|PQv*q&wNErcYxJuzlFiwhu%K(|66fX# z)bmHRy81w;Op^WAfJVlz{u!d8t^(2)`=D+=?TEuvOaPm4EaH|YqqI8$qnml(I%VW; z3T$t(J1BBHC-kzvM?Uf(P)z{RrZO8kXex4F^r+#LSxu*>7=uY8JFr!TGiA!&<@R)3Vli!?IhkO zNszKXrc1RiKg}!8D`5$K;XLsgKceG1<&6H3b+uEBA_@Flw%|@GQs#%BHMi^-k7b`1 zWF=W)8sR0HY22R4*k3_l-4(2B>vWA+w^t)Tm>#kp_Ny<-tM8K9V@g#zcR0y$a5nac zPs6laY-QZ|<$x5=W>w!$WEpUeh}U9kLkx(Vp& zW&9I(a-UV^02LLSe=d}N?)j5f>C!1x8Ow-yJJA^95XsD4e39|Wf;c2;43gO}7&SQH zvfV5P1LO(CVy$o&EG@KCaLo>DG#)CRy5r4`u_s9XQLp`qPsVmNBmJ^CE2Q`)Ii(!g z%Z8JNJuIOMmQbU7y&A#kP(`xzcTWFeeLvYif7lT3=0P{YD1surz;MUkHZuE5InL)r zCt>9)&amR>=wD;xuv>@f^JnQDe!Y``YwDUR-@j|gQ;e##P%`{K*Q&SqwK?1J5>vKD zM4h1p)FLhdJHF~CuZy1Ip{r)7+tLV9wlFr8b0G!AYRD2@%DThOkx~E;jZIgSlX(RX zG*lzU~z?=+;$B) z;F<^l0`=P&aJc7{u22k_Hg&VeoF)he$H&Zl5JIEJS{SGoaqiH2#yR@l`( zu=O8k>#p=w8XdF*SAPMs_Q=+tPQo3u5sA?bp8{NbS=-7#z@oJ zwh`^pOX2@m$~Ne?j6rfBJJ7-NubenV6%8rMfMIX#vg=+l`8G*>2kHNq|8}r{ZXkX< zvqGD_dd((H7t+iTdKGOYNNHt21+`(NcVMn+Wop$Fpla?6RopV;Yd9O~wOQ+?hF_&o zLCXCB;aPeYnXFFL+c!CNtL@r0s$5w`wE~%J-iI#*VH29C87`o;0jRl5&dc4B=6ZgX z4(QtAy?|}2AOzUX2G~+y4#qA8Pa6Pi*_UteAjC5SPX}%J11z7&E- z3uL*55%40cZ+Y!*$$=j5Li|Ne&>Z`N-Leg6GxvM;KM&x!41n9b{+>-Q(A;!Ae*@Co z1taJ~#n9a1ZuyED@WOn<6k6Uw-DLyL3NU?}t(HI25BQ+xZup?3(qsAVsYo@Z(`DO8igR=lEO$2>^OLj~@FyZ%lAECFB!}5P&|O2cW@z zzkE+{Xe0puj~mcpzvGw??q!GIzWUrV$M-3-K5#se2&~0^su2QeERP&wetve-9ytKL zX9@mMUl{+JNbp{EHNFS@;PL&GHKuqE3!=Xi@qb~)_ZYC>_Z~?0j3xds#RI6+_}SfQdGwdH=$QC9Yo7L3?$7Jxt&&8pPmDPz1SJCu z+l#y)qb;IH)9A!kp|OS|chK@_C7G32?x}ByfeQbBtkWl1iSA40PYz&k;4eoa$Rfp_ z)b0}Xrn!JP8qxEXw#F8t+Vt$Fzp-VmIQ^I_rXIo!3vCF;mq% zHADN`^N&ckmln5{B{P)VmQ{xAX6PdHqH7j;1ztLNgOp5C;j!5Xnb*wJG~;6;J7FO$ zH6cBr^(BaLL%7(IcA})Iix4`7sL}u!8sht&3-6dp#cQt@%Oi7!)L;Xb%U;{g_3O^< z?(HSaE&m_VfT_FK@Hn)gxHujs0F1rcA^Wy|w8DPU_E&N^Pd1`cS$J1A<;iVX4&}fO zG5;@7svGx0;5;{DKK|3;Qzr!`w_CJ(FbZlX0*~HtnBL)|DcM*Su7M*O+1|+`8{%I3 zuJ^$ZnC*_di664P14lySd#28b6C%XD$HUs$hqFWQu-x}{yu^n*@Y(41fN~U&-pgTj z-?<2L*+|Y`!{V)1w7qjRF7UjgdsO60vgWw$3qJ|g>rosJou(t-3V!Ha{YD|qMZHf3 zqC$+Iyl0n_Cru6xIKIZ~`uYg!3Hy=rb2mW1vwddo+5zV21qu-aM0%?*A(&p`=|>}b z>AD~}RJrv1)iXhJbzKI_HlU@gtE#D}sVyvBktn}$hQM}<;OW&J_hzT5uE;_`qh3$Y zOEZRs$Zt)wiM~AgKo=b?W&^0y&p7LJbORxw@1Hg!#*z&_!_qdnK za!s-(Uu|!z4IXsuh*@B)6S^@{cqu)C z5-G>9R`zn+gsGaFZGO-VOwlr?dy0(Yk0ygVOP8Mz@gHKsCwYwG*JPs|~ zy07H046=Vr8kuekVNSTM5}r8!jN_O1puVd#rjX_!5;=)tdx@h>xQ?iQRGJG{t_2I} za6)`O%PD5kB@;E#=#jT*I~vk_yJmiqTb<;Ya9{YI6yS~Tk6^@jV_#+|>_nyjKP$Vm zMgA{M$%u37WYXJl?h_sCwP2i{$p%cYCgj~k%!rE^5vPzr*4=W1sRZ#UqSsm|js@VU z@^frXVOwPV5gX(`;Q|e60CYzs_qW?`rGk27jC%~Ix?gq3e?t3IX70?@TIgbUAHkD2 z$fAEymH@swAP#!8X%6Y2{`L>!Rwc1ZdE)$?{^|PSuav5PvFgnZy?%7gRe4chM9qrN zcER+LM&FVP|2MucOE%=hBZuwhiJ4rbb&L1~ zu?<-Gf~RK8c&1Y8_#j#oq0okYmPw5TRA_=!2rVGk9$LdQXk%cdGu|O%Ybf^fHG}NV zC3deV_@f_L%#5g7xiJPMl5QvY3!>sr|N$sWy8^+OjxHNO3Hs*0T>eZjo#TC*9zH zlOHhb)ceOB4qK*LZN8er7s(jN^x;$b))y||-jyg)Y7qeLdif@QO#;|m#lai#WsB1BP0t7d3UPjoPrdbxq#1EdRVT#&D~85M~>xlf)b;XTyI zz?r#1D|F+$IxpUmPgyF;k*r#T3jes?gHq-U z`TdLXo({$E&BFK8%M-@SgJuHIvlFNv1jZK`qr9M)6nI+?Wl*CwBTTp}MSVkDf72SE zndM%)3PMan!^04Rdb130k$7FLXQJD~6NeXq*dBMV*lsYiRz$nHeVhO-(I3u}bGTVO*2GJE*xiuBR%hsk1@5 z`tojjWA%TSI)~^$qP1&xC!M6@j&0kvZQFLzF{)$RwrwXJ+qP|Vj6eUqgZm9?)#R*q zP=i|goU`|{wHZO}niABgT@ni~D48x0pF}d}u;rR^mK@WZym6h*-N)U%1)MV8HrWm~ zgf-n3+BntbV7!**TjdNFsMB7x&!DpZzVK>oz_8xmuWY=f3XCz+dIOY75NG0e*We>= z@zy@!ASX6u?shYgM>yeNXA%7VGnh@b%JmzQpJ)6S!;_yk`Yn=n)^iP=q>ANcwY7Gb z$Sq6iAIDCdGZ}Iz5xM8g<^DiZ}d~8bhjvkab<%I9$_S>vwfB9B4UitpN&EbC{_&-V+ z`UljvpFe+o`;Su{UkM`azX6NKwQu~MU#&RG$umksb@zV{mXUlE^y$($G+7 z$pQlObbvafo3;wdhn;Dh#%-E$kwrQw5+emloPbhee3e`ns0 z8=jlMlBZ<6=3Qu?jPOqGb>MKwa{aQ0mu~VdAN|$+6%M}ZxF^%+ z$)O$1_2y2B+bcD={r%dY8$$197nct13E{5~9>K;~pZD9dn!5`hJjC3vETri)}9$_wo| zv}H0O247+FS;;=sexE;b@KmgBdUTe$|;Z^_aSu)`g z1f|Kr{8F2H^O!-&niK>!X~iL~jnthT> zaa`Qr!XlCJw=*>c%faaBx+E#;n!xD@r;C`z2-D_;cnVY|l0Z&fu7jeZg0M(71m74W zDU!cpA7){$XUK63)drOFoan4Bfmvt+L&|g_)Aysem+>L55`Bh)emx1n⪼p?1}f9 z=3VX$EV@TU_gYKF;m4&h=|XWetDDj0zOeSVk36sW6Gt8QazW|Hy`0sNG#IdF(uPX+y-Tl>WPmMua z+nT<6IDgl_w=tS7@oO>0AIkw;F=*D(qNAcZrpvY?tUrF8PxTH< zN7atun?N7l4Q1fCMSdy#DFtlvc0Hy`-}EoB1V_;@`P{6v##MoB*e4(&=o8I-ooGMKd5b0xm{t- zG=4v5Y#YkO*0^@tQDfO=l8~NMqPgudXp?gPy9}w?OP&qdlMvdO3bDAEb$*%`Y3Po$ zP?h#XR=MZ*r3}J^Nl7vEq~*^JJd&S|XF0E`F<8M~E~91$Y_QLpB-tX`fA1>_uUK=^ z2P5wn!OB&>7RRV1M_PxyM*=7VX0{bvepR@_wL>$wf}(-_$0Isp=_3>hTUac?ya)KZ+D4Dq?2hoa_2n?K74W5wiA z+T4EJS>$>^e>D*CR^{@82pN|O{Lm4!V}KIJUJ%cL>tn~FfHF;+?IpgY4U8HyKXAn4 z3yYrVE_VRH{#``RDby%ABG-B~ajY6SHHghjF`149hMV(34D1!mU-SfXKh@w1RJQH@ zc-b}-R*9I5=?AGF4sP9tYo7(CqZC8eJ47v)s09jjt_j2xVI1jrG+e1f@`HXIo)4tn z|J(>tBD_QhptinB0xh{}XgbnZLjhZpF$yk(VnYR%k+b+yVK`6Mg@u2y8R6pSL7V$P zO~`%mv0N02u3XTU%V3N_Mt4sC<$UtYPe0MxoFVG!|M{5ntvGj$qt`=V2o2CG#_xe- z6BcM+o{i~h_E+(=N%1tkGj2ILHMVTRFQbl8tFLTr$xL4*~2qa(2h-tvf>cB z=Hm=db(c{KS=tFX%m`TdXn)(o<1?SrCN00ht-JcRvvKsd(sBC4ZyL%gN{LXb%w-Pq zGoT{cSi!Z;GfL|;ZLf;Q&mOwm8fw8*p}UZ^x<0Mr8*af3LD`Lv@M&L3Kany9k>Y15 zYR-fVPn^QbY?dYoP8s2{I=RS%oK49MO&WNl05zom z0oFy4XG563O{=I=RaS)v{kSf$LF*=nN~U|qUvSC!oWXyKp3CP8#eE|FhjML%((gNVV zzN<5e&{yT)TI^6Oh%haki6bc89Ox{(N+ zm$XvlGIwO*$r_+Qy1QTjaq~`d(JutN2(DG2*vsI0_zRNa@%4<|!Bq7(bHuwPC?1pc zkRg{PL0oje&fB0$+8imnRxhc>&V5c4!s^u>?fum3;%8l>W|*pb8c_?>^Ddc#?#nr9X{u(@^tk@WqZ4v z+=bzc#Kd^END!H?81vSwkiVozIlr;D>IK=U!#<;=YjComSGL<<3@O_cps8q znO)M1)k|~5Q=L2v6ZFc^#km}SSVEzf=mu4>Pb%L|5TWdF`JKckkB*}dh{&*okXc8O zC&Wij2G+40q|HFEy~_@zvB3c1V??`cuKG^}o5L?jC~$xTl~X^g_)qJI>_FitBVOVO zM=R3=V;YoZtW;xuH79|bhvFdKMVQpY-= z_|#weGi(mKd0VobmFk3j+xmJ2vQ=u)RM!aMGnC(=L*!pdQku6H8r=O4t>-GuKJfL( zpKO=w3r&%>4J=-bHpP%vM^#PbgSTvI?yTYU+o%Z(h8%}sDC`;y`hoTz73Q@U*14WX zn$Fg-Bqrg|DbE@A&v&2H{|UkTPZa;((`T2j#OLNNm3aJ@T;9X|N`3w(1XESc@x{wh zhxEcWK>^~eUFy>(W>{$&WKLv?DQqO=3c*HZ_$^oU6*XzXWL?QdrYEFRQZESnni1hU z;FbAr_-ziZf1;ou^ngY_PV9JZL14j-Vll$P1uF?mpioiMKdH!qNG9EURXQKlV?1xj+G9LdjEd)Q)(Dqp zGG{3DJ|%k^^%t$QxecU^f*_Za|C7sD(;GB%$%5L|0{-HKu?%e5E6v(^CLWi5n`ZCU^72fL{mCm9Adr0t#GDwi$CrYr3 zh0n=Y;4$v7B<#WzZVG>BvxUj@sF@Wwbkblm#O5_tdn10A;5Atz#u>&~k}FhQz;P;k z)8W0$GRUH?{{Cy9eC(_iu*b{BX~eT>>=2k6hFj+NEj;>D22&e5%{DueC3^e3#_GAJ zVw1@mgLd|nP-jEjk~~JZmx(PdZ@L3asJ1#`KCD2u?`~|^u#6ZCTs>89yZ5iLW09|~ z1uoONzrVqmmNpU%Edj$^*L5zk5(R~oJ4W9`oN*630W61+X~?=Uu&TSsA~bmH{YrtP zL{zkOz;2#eDd0y+o^yUBNEKsWdZ9toluAPjlN;<{U_S5QUFaM9&NZDE^QR*Ihb?ov zSKxBI__~Q{c9u7?oD zAW<;OM{<$yFWAN{u$f^LQl;x`Tl5RaEF{ePN5R7*%RB6k>&z-bc`mj^I8Y35H>eG5osc4h#?F;e_*= zSq{ZUa7;#0=h7_d+qfN#ml&*>PM)<4UyI2s)rpDMXel%U4sBHx!&_DdAWfO$OVpu^ z#=5C&7ZD@56e!NM)?95Uwpuu+r3M^~E6|C%gk@79cO)HxrC)!wggi9LfdaHL8aiz) zhiZ8SU3rEAxy0|69ZZ%-1!$B=)$*hTlBb|3_o6|A&}fqcM9D)+=ADvGx%ToBa{E-X z&}s>b`P4B$ndUFaR9>zO2J~8I6e6yGE7opD%4vef^eX?J2g)Dqn*2nLIdZys0u5(6 z77K5iXS=3$#!_5#wEBB8vwi{X1n5pfFyTLkGBB8Tr4brqFi&;^7EOamptviULb1IG zod`)xgy#?GnX2;+xW>+rba|U4+m#K_r&SDd?VGIu{O7;~T@J4pQEHuG9OL6ye-tI+ z6V$Ds(NK-9{2YT1ER<+>KbYGcv2;vwvh8(QL=X zGf`7B*~*VL6#d=m7U}=y&dR7LFSzTY$vXWPTMER|=;8$&noeRf*-KkmMU}XdcBRnB z68@Y5VVdNngVPaASx4HmrM)q8Nn~ZhtM|i$>%VGH!Bk)=A{BXUJd9ms(cHsT(J0Gh zqOC_G2~4X-|FYcs7N}i~SW$sdh*kbD{v?#qo?`J#yW)ZV2lzsh}N+;mk|He|D2o%yG?BW|yRPrf5FSmGuw znQE{O4zfD7ho>p}s=_>`zMPJg4b$RxE}))?!B)>X!+u<=pbM!y$y#rp{Jo|Zh1U83 z++HvaVLzIiv4&@h-e0h)a#la$NHc~@nq~S}P6*vN8F4RYY2ynfMVDp6?WWj@!pWpy z$@cq#s%;lvu)fesn`YEH^szj$D=RN-v8KRQdJJsrQcfO|d>Lm%!&#K;RFreIbj^b5 zCU<9Tv2P>Cdvpd{-YRM-G;xYAb3!SB(hBy=H@YBDA4OBn1QoL3KJQEh*0D}-b>CsOVg`MXC(IcMX8o@?-7wG1pUX#+PPZ6q|c zppuIX_|B!w$7WX@ZA3J+N;5Nq%JrQzwF*xxTEMGh`(TM3nbs46JPkB3FN0r!*;UIt z9`NWnBOA;j3j@qW!${RyTG;4bzFO4L%s+ZoGrE5p%J#uK7{GZv@Sps8qm+%4n#Ga6 zOg{&np4bEd*5n;DnvPGjiqZpDJDI=P(pjVhJu?LM?6YTuq&kr|I5bQT8hv7m2XNbj z3}@k8*LM+0h24SVZk{M`|pP|%8+*m6FX*j31EN2#IsjSK>GC-{`mU0CG&cLQnUcgt3{Lfnz z4zWzSAX%Ag5nfb#ihu3W%85|(T@%k^)A7`}SPg(?lMQA-tf4oLd9{xNQ)onVzr516 z1N}1nbo@5_cnrPZ&-KqyywQ3FBD`zjZ%+K){o3`}_Wo>>_)Y!7M#e^5Cvq#@)?FTY zlI}5syhQ7O$(ne=Zpp}e)D}MIxSP>p2^o0^jFFratYMvi%P3As0Cm6Trso$* zguw5ktPep^Ud;USWMe`JsAUNkLiSl#LK6Aq`z39o_(6aOeZ;?=vR>_Pjx5V7>LOa7S753Sxv*RKTIiD7=1kH5@&Ck!bj zCE#UEiU;X#g-HroZ96sL7A9I3iq0Q%{=?A?%nBee6!3)k)M)nsK;q(KD3^RTD9pQ) z8s)@)kP?F+A^Jjw{tXb;7bK+rUQKy-`J=-FypS8D8bPiTwaUmVQ+elX2R6bkfQ_L` zZQGwWk?cm+lF!#}%qe>KeQUYln2u6}n68+O+4f_Z58$9#0={%df<%LqJo8 zrztHcbpSCsuuaArn6}~liD$@zCrW)>3VB~(rYi2`Bf>NEwXs2a3)yC9QTZCZ26px9 ztz>)X2cIr#sL^#O<0$buMF-iaDlbL*CRn!NTf;aXkz4b&8Q2xVTgG;$U-r-=>dw2L z-me$59jWt5I-(@Dgad8XNR!aaWxc{&`C54)_itUjl-*%~x_qf5!%A0}Rbi=ip^nNy zV}+xn7cU8KHe$A1=u~n4k<#uR#gk?#Uh(!B6Zt{cN6^!L`ZdjTADSnu z!7WLVPdFc7>IowDGk^M%80FFIDSTTGG+p|2gH=DMF1 z`F$_r!fg5_t-5d#YqjuIrIG_y4Qt}rf<`M9{NC}p=;xWWOB#RYZGMJz{fi5~d0>g3 zR!m|cSt@T)?4OlWZYdiNL#%=hDBb~>ng$iG8U>y^Ay&Ow!2 z)P;o#K!4F-8y{1-@%i;d-3Q2sKu9Tx75H)UFj9IjQuz0C$!Cd)kq~7_9dLhXPS5-r zJ4wORTAS5+WocIP?5zAZR;lU!Q~HNQO$h>mzhWfnzkYcM8n7S6U5yXAo`)Gs(fvM; zA6;ATK5*kJuiX!;R#%-5o|!ycc3JOdfjx1?fOsOY#V@bmzGi6d$T*4KDU$0-x!8i; zU6Ur1v4Uked|vQWMb6eI3fhO(Q*Q;<&JJ55S1VLuhcr#wG}gjh!|JK-)pYsIy!KYE z-d_XYDsM1xY~n=KEwd^*N4Nd4u2YhixQxOMlCy7#?%r9Hk7+-i$Fv4E3`A*Avyo|p z0ry7RYPrD1Tc|RLC@8CYG@_|1U9|&Rx++Jt`N3Ezy|C== zD$L6?x~ScG;@g@HlX@4kM(X+!5Rm<}T_bC|JeoVNmh$ks&$~~lAD@fSk z)XYfoaqCzoZ8Y1G&RHpi^O2_2*c%IN^Hh3!)uK~&b5+AQka*_u z>~-ekou$~^T(!8Deop&OrP5iZ>&mBl>}vF1qQ_#3SF5o0MRgf1TEl7C!js8OIMGcw zhZRG#o;IJ6$IX3de6j3Ew&R|R(xF?hC@J=q`Eyj_sv zG?Q7qeV6U)zm~FF$G_?7`OR%e{+gG);BZP6WJRK)7a&LKWfns{-oA5Q%N&t;_UI7+ zo?P|#qRpaoC`+M-H#0^YSC2Bf;romqaq#hQcX6~f)tqg|x8>9I?)CZ%d_LE&(*lOA zbB@6#ttKn8gu!031bWLje%}sEmMzu}cpi!@v@VkiE?Rh-#x6#$V%C{2!Up2@6Zn*> znkD0eY@68%<#ZxoU?Zw6WND$hbBr}HoT2lJfY@I7@Bm+f14~*f`jOKtGGm=d6P`X3 z9g#Z0JM_lHu@}a~+)85LOb0?y%+6ji&cq6wD9dFud-Z@w68wk8zS>0(P73V7<1og$ zuVDfp(*2fu>twuw6BJ!}Q$0A?jj29p>v|ipF^IUi_Y)2z*X7yYULgx~U!X%VMC~U7 z>%|}RUvL~P3tt@;Qx1x#T|e0#YYmGZD#DgbJq-~TN{ByeA}ab+CB`AfGR8|;L?za#i$jVOyfIUX$aYHv zbPt!AuELAk4YoSmH=+GYG7{^=VY0Q{Mszm`OxK7UJJFkG^6k31%}Bh?NGx$oGSH8x zUNY*0^#kLpc{LLRR`33lx^lZk3GSVC{@^YO+-C3{3eM`;e%^vk#XsngRyb3Aj_5S7 z8CC6dpSGO?U^Q*30H{cTo2pZ_Doi>7bi+* z?%A4+WSAmz-RH1F_0R{Pfw$#w&ob`M-`l<5cGvv#;j8ZH& z+8X%U`Digeyyzc%)ljTL$>!`x4?O0vj~oY)TA4kOq{Zh@D44tm&)*%3X+BeE_>9N4 z>=NZ5-XSt>j~!wQnIuDV;dPw- zPVTG#-L>NcOxadfJ?aH=**GKyq%sQ(t!LC{lHr9G0_2BswyhKWnu}@~g@5o}Up?qk zA|G#S;uoU^)yNBcF4{ZcxZLMd*1Mt|>m4IGxAqd^fupE4tm*SRu&y*&!v~XCVA-P4 ztl@i)P-ZLoxMD1gG!AZjpe70GOL4}~7Ff3+#{K*vWrrY1vs}_ze@Bozz&ppu#yf@N zKi2_YL?@!=cEeII;D=(f!yuNpXfi^ydI%g+kGN=oZLFF7YBDy6@i8jl=(slj% ztJ)LJ_*l%jr;)NKp%5YYf%$o@9493;-*g~zl7w`(JuHXn4t>iX~C-x~b zjxY$T#d=6I(K7o^S$F&#xO9Iq3>&Qhzf*SLJ9vTq%RWoqv{CK!bFAR!wBN!&#q$&v zHC;V}NwM#O0U2i}YX<(ulDWjGJF4*aMCbEI0yqzk31I{`VPvON>t7#KP_0(X*^#S3 z;GlwLZn--=ug3b6vMC7 zcRNrdGE^V1O869}KNvSlM}0e4PI_XD6;y_DD(nis=)9){^+O(Td2y*}b|bswm_5u* z{8xUrSw{1;l))c;-y-M5!=Bi#Vu^1*qT9KIwpjPRg_(zL8Jnz!1QqFzKr8pa^_@2V z#1elY6LWlilz;IIc5nqPcim7$m=u3NdnP)Rus5>dbyv8E9<4# z>Z=l@=mG=3=c#E!&(8>$p$Rz)#IoD7*#aAlYu9g@D{KFAhP5Q)NQRXUK)0W-$p&iF ze3x(T+*lkEvqO-sPqh> zJyna>ZLfS)b|d^99G@`h;22FJJ5?z~M`PiU_M9*8(+4GG`YaRv7S%bPWI{*9tmRJ| zuhZnb&1fNxrH2iEgk5R?IEk0vj$3PgAiJm^AkjaKTyY$9uk70v9+{2@C>(MT1H&u( zSLUjVThh>6Mtf5$hzE2%9MKCo-_)R~dlD1t6;20L@2|2@QuU&IU6qFQK%p7bMNoJ_ zHCCWBEGPJemB2KI3Go_-bpmgIqbEY?Lb_ao1$`sLrx^!xb>hWhLe{V?e(L!KK?s&paXxR(Pty}izC z@y`;HwG7@cR>b?~b(y`ZG%z(?NcOnmFX}n10AbZ@uz-G%sIn8yd+^0e701>R)U%|9 z=k|dEZ-(ckfT?iU=;Z7z(e%v^ER|;tz^?bLgDL*DZ_q4hw}EeKOm!J=J=`ksdT#^K zCLKG`YpP%D;93mj^d?WxEyamg3R8p&00CUx9CnD8G#9ylRvX+Khf9J8W&u z%UV~mhYk$cqT1ZwvDMG|xYDX9|E}-yn-w_B%Ho0Y} zDvm(C{Wv;)r)Si=d0?u{iNk^#+-*X7i0zf7_XTwSf{gY5pccxb7DB4=swM4pG?ej5 zYR;z)My#<$q4uhr!*=z_F6|R~(cqI_+9`kSLHg|$1vfRl6Uyn8U!Z4q4R;RHH2H_5 z{;jIcvF*4?_S|~QSAnAqh`3*Jl24BdWZ7y^Jg-ofH~E}5H)CPt{b}RQwnq47i1|-o z-3_ou>Tzjw!1tsN#8Q9Kt#V(yxfPzqP~}446x_AO>}>yYV7!8j2)Q#KJE$suzeMga zcf7=Y$5K14*B0{J`m;Wq>C;p+A(h&@+Q}3sGAB(ZiluZfcr_ww><|3z z=L~-CiE3x`v>6uM{&d4KEqD$h$3nkpqi#}*Y;bLSf2FtMs9_o7J5G=bOusm(kaWS& zKWlxD2G0Tyrcj{bW*g-y_iC9Z#x)Tb5{2WuCIaWbs#P<2l|!w5>9pFaq${ba>8&AgwS z>bwDK72EMJK(N(AFo-gx^2We|w|a!LPv1h*+MfBGMPr}MY)zMy*HR4B)qan_Z2B{T z>G<@RV`6;xH8NdReg}&WB@p0C^J`sm6DR+h1VBOSg_LVj0=m}EYo%|6Gc(c2Ky^G= zi6IxY6=QR~kHCa6YbQ_!~DrMb~Z_Z2`+CH$ayeW&$;#A^)C z!*D`HhbIma)NO*%cHq;fg1skDwVHjE)`LpaT=7>0GmA*As>^o!)ydS}f6CwMy#_^& z(ehl)V&noWarM33gLjlNVXI-(Glz;rNumxEE1O8sRZVJh#`z<{3@1PL)r{pnFk;{pre$#pTguO|i2(4^w!`5Kb z-0Hrw#&Ym^pEXXG$TWQK4pjRLP8q$&l(90pH|upLR+|5nvR7*agZfXUJ;QjRE}HyD zQQB4EJf)Q5oQppqT{M-hFw*gAC=P3GHTa-}`~?FT&#%A%TuD*Ik8+GzxxPSx4}o%z z%PZ($gQG3xi`?{_weGqyGJ8U7)GN=5y2wsjMcX#jykx1ge_S=}_PO$I>zl@0wAo4U zW7Qlc=-&k;UmA_YGD^79{ayNl>V@8fX2Q^ees~znu}-)lu~9{_3tW*%k2jDn!jI!( z#9G92CDZ~6sZcJOM<@2|pYtxUcT(>ESd_m^14`V^U5T?bY!R8lb!;hn-5iN% z`900%&VE(g&q5nq!mLxlR-}QhkVJeT%i=t+#vLd3lPv)F*ovI_6u}bz zM8oKc{TM?$J6=mx;)|mWDqH1q@rtF+m+g0OuzPc^!d*dIjX|fH zjMsPS$bQ=3(5c~HI2JxwFS7Tu$a0It zu=MF%#jvUiTrBO{Z08zSv}eCPWL0-V+-mKSU0NmGgkY0-b%ere`FJQz(pvjezu1dI z=hWGz>Qim{u@)Itk6CFJ;Tjad(Ioqnw+6L^`o!ADrax{`0h+s=uzi?+V)2{s%(aO( z_}e++XHFhHXTLB% zmF}QiILkg>j}R_Z!XnROAY50bS?!=)SG|X)-8_H6p-1fC&pYDXZDG$l{N0NnTxiX* zMtn2JQp^PPj}#AqR_Yoj$UAN9Wp}PS7|8SfMY8AF=G{`P6_=k|O3W*82YAM0zSoH& z(_1|*UcLh79mr8|>J>KNP{D#a4B$t9ILfX0Z?{0&-MjwjSDOj3A4Tp3E7UD#1vL@# z+wK1Tq(7)~FmD~J9|g=Xml`*ZUzw&^(l}D%qQTVLy=w!23sjpS`xG~kR&Q(^Rd>|% zN!qGu#Ugn*{bDupc_L(wZh7-edGkv75-suuQSt^&@_XLgY0p5x@&G}qR6z(07n52r z(5{DC#`nUxv;-4&fG>Sc&$m?WSsyt9n&SCtO=IP&tCegSc+mUcs(#B>`-lal0JbX( z&=O<}uOt9{7c;UC)qmW)VT+)@x={Kw)|fP3)$jb@V}3$h2#gtr^EUAh^Od{}p;&v3 zB|e@@njkMy%IQs(BoEubzRtOX4pMd(dHn@hgje!$b#G{_qml6Y=dZmyf(HL38In@| z_V>n)OX~GgQ;^0v!xS|{uFw}YS`{&SPG=XcR=NPa8K@1Z`E#qX7Y#+}bC=S*#S`N6 z#T(d}b1s&L%)G3RQNa?GvvK;0_2;H%VS@3UFoS4)n;ER0D+uV=4sk@U5M0P(P25)~ zqhT^1k{r$&{ZWiZj11DQC+Fl0pdz>bepDPiXzni!cuZywL?N6t026;3?I6A0KhXn`KRTl)9XijBMP@M&|7P|(+g;BJ}%k-5Cpz6NZPC_Wwq3tDBi?pX{ zxlm1i9>zPM>dX~fV@v$PHBh)I>WP~~Q32<~<`SzvVz#}*)(A-2K0jAw#R6}B6ujb- zKIc?|;Z;5qks=w&9u1u?)f}SfO3xT(i+3WvyP;oTgp3J)^gO!uX5o*?xh+dc$U8t#WkbJx;*&!vnxM zm}bM0gI!m!vsqqFJgMDLa3o$|r|;Xp@+#qlUAVBbE19ByTJx}5tdv6;Qma>|q@Kg(ZV^tG1 zW`PvB=x0*>y2EdFdFWq{bE$c9+yNByp>svW3BJkIJVF-GtN$ydr2oXXY8ovxB3fuR zjqrG{YJ^KZj@lns<~qZ<#j2rk9M-EJuV=CILva8z{>gUM^h`BG>e$WyYF`-D)dla;^1m`N8NN(}LosXTwi<7tCe@(J;Ph133Un=CVfT z``5wCUK}dPo-rWUQ8h4zia8nhE|hRPQ?^Pzg_Fbv=+qHW$flB_w6Uj*&a84PSKLr!2x=0x~gYhgYV{pF1`@mu+< zbyNN6Wt*_9my#fpZ$H4@Hw*ldy?bwUjWJ(vFTcPfR$YD*gRjwAq#g{7L0f;p6T=+C3xl$vbg)vFi<+B;7!Pq+JLtPdaNSjn3&ksiH@>&*Rq!`QpdB z@Wzu)r&!WXYZ(u^sDqP@>&V$L`7yyBATVEu3`2p6q=~__{$MA*V))t^GL|v-Tv{ z`L;ws7PRAdp$UXWWd>1hcSe{2>`dlLgc5>LpX}1{djt@HMW#YgVhT}WT_nsb3KkK< z`qjVNH}llmBvFsvE8?kpI<>%+vyBrr>8Ulm(;8Z-J*9NAZ6#Zam<+tr*l_MK*5*DR zt$k`0;6Gp{d(UOtQyoI7#TxrkqHyOmqg>!j=F8B>W|X*dLLPK#XfFlNVjB}X}*#|iAty4G=tS8><4TgkFT*IbHoAssz0{Cup#^v-vZ5CuJrsS3W3Q?zk25OkuKLU?0oZF32R6Ukb8JSh6GrR(KIx5!*+FhS>7!%fQ5pjh?bV< z_IIH$;5V4k=wVtk8`=k0)9XW!yyg{2QVHGVrxe&L$IKy+CdY>6Gk9QX%|sG! zW$4H>PrXiG#W9=z&I4jKhx*LSY_(20R(#+sx%q3;i~4p`{I#~X%2EhCzgmF}KSu~! zb3zEpfXSio^TB0JA@ngizaw1_Rl_7FWh27 zLB4vdZdk0+k?jf)dK89(ImejflIl5Y$$gb}2X&*NtN(aO@&OhH9?Y&Z_v2lb$r{(- zY7Uha2jL_Fp~mzBZt&#Spdael2x~H5)>r034&mS4XOH|XT)BlDxFh7Os3=dEWJWgEmD9B@|yTm+c)SOp6FdV<1|;J z;1yPpUP7PJeYn9s>nd3>dK(a~>F-zq+`XIawp=zVoCK}kh>JTMzm0Yy6PrLvsFjg; z``P4LXsVz_sUfQATL^m`{_vC+?U@c06|}Rv>oj=nE?nu45|#rQnYw6B{obF2a>%zpNSo$GTq+pGS;)Z;~R>+ zPskvji~JsFAz7vwcJu_^Q1(GZfHzQ_Kf<|Vbu`X|3!NoollBI}=dirrEkmvM2_wwF;3NoN040ha*a7&3*7XAM2EW zcLJ!P-p*y7TSg7#n$TKHrtc?w9-6b;G4g5$7snq&U%=u$exI;)qxw`1tY!5+M1L@H z%j)aaifBxoql)N?g4zQGI2(ubto}w{+iNj{%6V{%k86N;PtN@ z88-m!QzUQ4T@^;^fL;m5g(0xeA*JvO2?q4ea%AMdwI}t0JERoDrOO)lb989nu%b;} zGoLf9nE6;(V~4ryx`WrDX+9LnOhqQSAU$}G@t*w_vTYboXADh=lOow-4k0gXow>D( z3rR`BLU4TmAuBu*b*?=jMt~4`?!uqlUN6yQr4Swux%HMVAZS&Vju82UG!rx0@VOk&T z)4ffDyXoZdCRuFLURb!68%&7tmOgNRiizN8S(lEoap6!Hc|tH(D_=@=6iV3dZp@!&kK>~+EHJIO11c$ zjwcTIhtr@lMpUY|6K(_AIqza#h2;KzHd7p4BfzdRCn5YJwX)|5!{j)D`9&}R(241w%{ry=hd z6AzGAns-sqsghlOH}&U9jUsV}ZLRm$9%sbQ_iYw9mlm!X=a~MBhHTD*dR25d<~_+4 zE{{4v7L))o+)Q$bolv|QGd543@vNccJEaHZD0|Rhc6FVtr@GQ?pW4R{SorRxX~2WvJ)jYmDIH~Z#V4H%I|?G1KvhzSh<8A8`JL3 zvuEVr>Y;4>kJutxD92nOyxNxke&<}F-tSb;QKIEtf!*(HEb?@V-~nML76RFL-h3i> zO#xYAGr_LT|F|Q%^m40lz1?VAA-0;;n)0i$y{~fN;eC{M?Y2Cp7q#8tyzge%kfj`$ zUT6ijV2{T{@wzU{Iv~B(^6YqD>{paR$+$i3*DHSK!w7l(`V}@n1O#`mqyxE(SeL}h}j?aYjs9s=lma?{TrUJUiX4v4o`MyvE_co$i zmB*gnyu+uDgtjd2H^O=0UnHj=>0d7S@g~E0WbYeu>}X%IgtlO=RTgFgyG;@cZJWzE zlP{k$&uH{CwOq2t$Oi1ybx;aFEVtguESC_M(#kZOmA~tbxC3@4tFzi{HAwkvXc~tO z^jc-)V9K@+3u#L&UWZ|nYR;`I2AL*N2yB6&Mrc3PTDWo5S{t^pgjI*`UN#a4oK>+| zoEH>DzK;L=7{*U;libTry&ALnITl7zKp1z!W3VMmgfWN+%cM4}9FuRM7mbiFUDXzj zP+PULzKGVzYk-=H=0&3Im)q*uHnt8YDlsa9>-Wh*i&tVJw}4Iu1-GtC;+kux2DfU} zCR3X0u z!ERk$n@FRGj~&VBpZKJzo506~v`Niitr*ZA> z`f*MXGV>12BK8Z?B!M+@MCY-<=6oY9Xw3#U?!q&YrPab|v3fN2Bw7)Y@n-6JcC%EZ z^-UYynqdQ4s>-UtfU$C^u`wRyn(`n?iK`3T5dGx#ip48%XQ7@qTQ%>@@^OgGTg#3( z0E&G81-P8WT%x`czMA%dYMFI!@$H~p^-9Gw=F`0BKLk7;vNyv zm|27PWifuGoD%C&AF~wV(_^I_dDB)rn`D)-SZU*qBUwRKfQIl7t1qQtlPj&o4k@VroH)#0J3O?YLEKq$2ywsdjlSJJNV=4({d4N|66 z1~@Ftdn^h`aMM<^$&E(I(l{->YG+um!iW@R7*QmU{XeGuGOErWSRX*4=*HcpxVsjo zP~5$^Q`}uP#ih7=i@UqCad+2^ySrX~=iGbG|65kxS;-`MGHWL9h$2@s^Zg5sCKfxs z?g>D}(0U7I`J>aBWQiX=YIL<%>VU2M|c)GQnL7*feUyWirZhibHP zYDZIaptD?C2-DF_AqjpZLKRB6SJ+M)ujZPef7E(L;sTk-ITwwy6gS8v zQEzM6)$xk|;m2ih*FNCGoz#(n9?Z?9inNEiFOVv}Q#ySbO<5@ycVxtt{!-1Gvc$O1 z2G3N>q=T}S?CCaqa&wv9_mjenD8`_p_1n*~jca<;Tecgkh2*ih*!)&)du&w&P-_@> zH(da})n8z1DQxSK2sxWVm~x0K{rp19TCS(HId5ovbd-?s_qhh&g(!SDSB!-YioOPM zS4X~EECtwl_TK`T%YwY`U@>^twgqhUdGJ)-*Wb}uTAA;yvAA%Vpc${&8XRJvIumd7 zN`nohwyvYSD-)yl>)u9zpSyBN6~+t4M4zMi<|D90s$RwHH!noRRpNp}VOXbe(zydK zjh6+@Nf;)@C=Cf!Yd|Xr$SF5#0y5@(RyJAfPY^HN;Lmz0<|~`x<=TcD!3HjAf+P-5 zzO81pO4;qpi1nhVwdPj}(8w5(m|*9fO;Pe`}kUPY#4u>gCK5z-d3 znKh8ZCdwJo(=cCyXE!s+nJP6$woH9I7Pmj5ARYB6Hx?e7bK7{A*1%TYGKrA3Swl}F zvV@wJ%`E5UN1ek%z@sQLLOyFDV>SYBF-A6fgCu=@2sKV+wT%Bf;a@7|uEpR$&VZKe zm6_{Wl`0kt^6Nf_G{j^cfs68(t$5s-1r;DptFCLZl`fo-Lj|TbqFhO8Wl?=@ZmEsF zD;*0(V;8-BV#i3?$j>C6^c4CkY6j?^*^7On3+bsg70P?2QOq=qe`3y;Kt|5~Y=0xz z+@o_FW+MfdS0>F4*5o;DsNEykzGx;bs}{;BM`3DlU{C#_Fp*@0X*M4(^`B$}e2f4i zo@4O(-!8>!`dd*u=lgJDe(j{oWFF=w?N(qvu3`-v`hOIJsP+Enok7x+rh^su7O?oX zt@*0Qa+_zysG3e=HcfguESrQxC1N9=prSKNCDvmoO^-)kP>+&?MCJh1`&B?y#(o(3 z!(oe%3}5??iq&svc4#>jt0HL`x>hNm3D-q5y|G{T>x zndgCkSH@W1;KT*5;6$rdjG1PP8A2NAALheTHpw%Hm0vahDl!kybmIc(AZi)9aZIWb z?|4&IJt(8)SP7J&iQhg3GzI(e$w;mV1-BE!@oMgYi3H8cGLumKyYeY22Ugcx=gVV%yDU_%o(4z^nHezbD=xQCk-^We*+1#+Y#f4JTP z)M6?sjP6?$)Y{B~j^YwMd;2#xzY8(=G+}K1rSy5E8PpE5SHlo`#T3#Bv)B64`F^b@ zeZ_J@vq{z0q&3OYMfMhI8B-7e^sy&_*uX>x8b7Xhyt;>013Rzg6>R)ywpP> z*c5;eVz`_~_OBv%)IjpT0BkXQ?X6r z|F;ifua?qx8<F&` z3#0B5g9$gl)J?&+8+nLfPw#J=&6Ib0(4D7{mrwqaaz0dELh9bW7zbk8t80oa%L0&{ z6;)2}&ybfna0U|m2S6JLuMU!d_Ysr{`?(wb+-GfApC2#ZHfNu&*KwFOVYg1mJ2AaF zd2dGQy1VuoppOQJFO;{cJ}?j5seZ-S_@<&`#VI-xs1`4NQl$%5MQ z#(%8cJk7bldq4Mq@JXwlydZ$sRNERo`RpUvT9xxbr39}+`>gjZrUhO6OnlJzI3oG( z3G8#H1Z;Y4tvbct=#?K(3=MQ4Y1tFWvFb19opl3zi2K}iCyTTdpOrRNGplqan^-3v zQD0#~c|WRNjvn4BKC(SSM}MYJ0-N6xP@FrlYn~&ANh>X`!#B`VGukH}0}k%&;Getc;eKV&`rmJbKt)4m zBzfZ(F3&HkI8L&^@(gFEPs04#qDj6!3-Wn!OfMcDGKp7BPwRwtYDl7@3;xY>$Ix4t zZVP{Jj^2>G%{e->sScQ|a&f_&2IkC8kBb9;N={Ndp9F1EH2EKrqP>Ss5l&AY9r}us zDi5s74l%U^v|*0aUuRV9+I2>6e0Rx+-jKO%bGb#EUgz_onI?3(wae|2xfR>ppALAf z;(I60H8`1>d8yn#cOlL~&cCcVKBUfNS@@i(OXt6tI{3UGrx%Y8e~1G=I{#Y}6Q39H zG_WiWF6=!%dPDA3)Abf7mz4TBbhi;h&nLTNtMc%mqQ1r_`fxH~RsQxZXp^#Om*D&! zLrBbyW}1qfGHqylbpKonq#3^|cbl>JYj1~X@UfaEi)=b2VU_2nPA82EI7_wp zMHUoyFL_(!ax0W8{Q0I4`Czc>>XLBYf%h(t^_m!OojtHF1LE3UK=&m!ng)u^w!HX9 zcf}mWphf?gx%gC&A|jd|o8e zlbF%Bh`BM%DQ$Vfwp7n9AdOF)h&c0uRh-0N)+)#tZV9}g)#TVDD|tcrY~XtPh2OoG&x>K2<6?GZ z{8orH@^E>#5lt|~!StObaNdTvBlyrw@DqQ`vp1L>a+4Rkb`{w4Y74cM3S2A<f6Dnl3kr5iB9Ge08^i89hr}y)~1+qQ- z{7a5~cpd!RE?kIaY%*dbXuR)Qs@ljuB$j&qmKB!U34#;?JWxmsalgCq;fU~jDQAI> z0OKfo`aiRVW$C~;gh2UCL?#Ia!*gwLm0yNk)swu$!6xgXg&9S5cWP@Gqs+^Agv9k3 zNilL#H0Oah_~}&`UhV*ICzTjf8PQYWZGc}BXAj=uvpw=d-Jg@m}u{7_s*?c zDpX_~uQ$Wi=;!lg#;bb{V`@~c^JSP+skX9VV{n!jTw<(=6BpAkq}UFCdua#=*PpNU z-GBL1_2Q&&=!(m%<-r>Q3CpI;^z0dt2nb&+$pX`vSQ4#95b=&>WYRvGmYR|=aNH{RKp7VLqSr`DK_uK_MBLCb;Q1IIkmXVP=0Z;0P?2@R~Qd;{b z)e&x>=i43nhQyNGeLh7W%c;TvJfiok8GdG0P<4`ne|3xl4Y&wCW1Pqxjrr?qQmmv0 z2rXI%z(TbKl=OfCx0mt0`tU_l0Pt)&_vGR9NQquydafa2$_s3U%|r{2#>KngD`(;R%N%mVKRy7*jFDG&|E zt&}ZoP6&SUyVg=c=_g}qvDMXLZwf(}LQ-15rL6b3;hRM{x4gLCZz$-Uj=Tkm9&AQN zz|zlQ6=FVzC@caX@~iR>^I*RJanL6j{HzU*i=Hjl9DVr(lohmy#(m(tCe<8-Vl6Bf z@JtBi24O5$MWf@mF%L_f{dT}HGFoi?U0rQ~{kh|eP4G0Yv_?iv&&c4g4jIE*RiH05 z>vB>^T5%`f03GMPcrPfJxb$=J!d#d*Zbb(_d}?_*6h60Eh=TD{wwIaQH%{*e76i(! zPpqz{@cca8>6p)LnI`=_qG`CG;4IoZV|r9@*=OIB?k9M5F~Fu5%akC1FcoOHvraMy z^IbeBL6F9lG8nTnySn-WxW)_uEQUMOF21H#S8u-3d@|>mP+h&~Lk9v_M3cXvO;8iH zX6^WZm}Gf7J+bam%!pWM)})u@dF{UqDx#rGBOcfWy}$!$cc&qT5rbsm6^Z1p0h_QM z&|@Vq^F&o&b;Xw=@doOXL+g1jjH0^~n-7hMfq~rZo(VXi@E<@Lqa$=%j|U2EPpZuW z4<2K@8}q528ey^!wqpOw(4e!R4~6Ei%PF?ckp$8m07e{O8PF_X5nW%dIr~yD2r8%} z=<{G8!R8=EjE(Oea9|veUQiC1Ply8pm=#d_7h*%zJqT7836AsYr~JH18=_9u+Jp1-~hxgA9A?wvLqxu zlgFT(MR1C%M-|G4bSUc4qg8x1Lk*m zBni~mG;6)2C%!+vgeCS94<(%i=Nw&j9bI0yJ_4zTH!6D1_+AT9NTt~xXly^ z1ctC|z~7Piv?=po*lz;k9+$5aw_fOcgm-?jba(w}BX(7=d{bnM{7L=-sLLvTeM=P& zOR!-gm@|+wvQ4FXZ4@^WpVLC+G8b#;P%`0H z|A`#j!W!?*j)pM$mlc4Z^Ofg|(+O$X8?d>9#p*Cp>oK>mBTQZq?`BS;z4(dQstFU8 z->3wWy0zF=0q?WuXLA@nvoR1egvZE{LN%khre7tv-R$9!kbxTm(ZmfmL*YT=t4DtIsM19A4&*4aDtA zwdjTt$yUICp_Y#1RnuIg1RS_8QyLlKu2BLknl>om2g@8Vpr_w>0pI%zzJ&)o$@2le zKi%<#&*%vV8=+lHTzzqEx}n?+*I!PE{ox#=sLMRQCv^2~d-FaaNYnXA5ktt0LTSxm zd|kxx+qNDaCxnEb(Ub@t+yuV88gR;4f`yD+sGU47b@K(IsBJKWSrMCFu2;Fz|UtbCWbNj0Wub9SUv5-fw3!DltrOKfQ?e z4}e|^lb|SoPS^K`O1z<;waRx`w>TA$l3SAsXzI{a7Vz^CSq>1Wp2AMG3shd8#mbR% zcUApP-8xQZ)ZDy9M;3TIQN$|{XZ?51sCkPaM)xMMcwon=zr>T6v?3Oh<8se(`_lC| zoQEv1S-U`3Xue#U>kW{^6`F*E22SmXm(GSRz&EF0Z zzetLpQE{DDYj0jzEt(^$2m1AmS~_6&HA&Tgt2=eaU>Utj+K40uQbS4)Mr8>ISf(-< zS$_{C*x<~8<3|6*2lZQ{lUu|wsoUUfdJW9_gW~&*r5_V%aQm`VoYWDs<+B*-u%OFduD8wC2wfR9OFf&N!fSw^_ zUkOh3+_;Gckg(XbazMv))4w{wVMwMidAQMTT-aF7`L0w(U`c_<% zOx@MG2q$@A{_`Km9GocrKcv^|K4a1*S`P|M6&!DJca?!X;9S2+L#>Tqr4@bd1NY=0 zVmq-CX6Z>kMLoj!k340k#NJV6+sx3$2EK}#sPy7b-|vaO)ZzxuVoqq1b=X+)lp{Ae z@ygP9)ng2sjiAitm#X~V(|;D`y{3&u=|FMY9LCT|gKgRID&feHg-Qh%(z%%gUO_p) z_y4Ko+2v3Ee{u2u7csB=eNADcdNzCPGp#B!bW#l@ZptMBHcCoI1~+z!5ECDI(~=qM zJ|y&ibWIbDbd_KJx9Q`9Lh--NI!~@=XlSMXz9aR}DSrX{*DUy*r@NnG&yCFql(nA~ zL3`(Bq3N-eqeC0Xvpk6pB4><6gF{P+l`8O5b5%zX{2SF_)ySxC$`APt0SxB&N*1Ja zars$xT|-=Ya8S*Krpj)8VZJGh<;s!FHXs9V0*?x$8$bx-xq zv%MLC9PDcpLjz$rr|*S=2Npd#uuVEwS6KO_CVN<^52-64$*2@kQPm*1(5}gkGNj5cK!6>Ndg#C62e11Ne z{1iWJ3zv zy1s?SA?Ghm`g;Dcl8j^mU=$;FvJ^Asv%i34cb>hWg>b1s?2FC&t0opgUJUK2S+#g| zvp-Ic3as$-9fF4CO+>LtB&>#b0?r-Tuz(d-vni-9$3bcmU2m-T76d|o1>%1*7$MSd zOf!Yl4C4taUG_e59sxr9m&NygPW=-|uu%V(aSRO};y>*42N*81es=r8rA!kO5vFwK z^I)V*G@-yYu_VIDK%x8xng|KEWaG0Q^h>;S__>CM`prNBlsE+lHl4eRh z>_C^=Xeo<`WsC9UpTE}Xip;EWo_~Qzv4@}>3M*gQaC%x|uB^e_p6k=L>Kl}Ae*?Wf ziSF}pS(ijgS?eUFF4p(tGSsne7CF8zogEK=F2Uy`3kV9WsBQS{$bXocxy=y=>T-R5 zx6)sCO($9wTq9W$gkd3vFgfm&2Ol)O7-GsFdOKwV9)%3rMgIj z7JEswj7+og{MmftvrI01kZ9siHl(MTee(VD<=Me}5x4;cCSM}W>UU`9{X{>vQCE{z zW>JFYv3H!rG-aJ;9Dk=K0n}?x=77ab#u-9CE>??DHapk2el?WsiP#{n(yV}V&D~+#h zB^q=anm^;Vxy7zy>#=m!Zc4;Yn5%RTgsS4^`3{_SvMluhGV+2I+uU^XM;V-YF*Xq~ zyLRcyLw+=xZ0%tKT>lrq!@_6I~Wdf|JeV=E4-(jIH|5ffJ# zN)MKK$Pb?ImhB*8(J2$#Z=)!e?l3uoR6-dZYlZyF%@;EaHh*E03^Dq(nJUNV-8qWf z^}DAAHj!AQ8#p@Qz&F2J@@*CxO^i0FFiy~TrPIOI?(EN(-V!x%b-+N25&Yx{X$z~I z;-N<`klA)EGx!$6m7gW)9=ao5`=F!K zj1ci{IbS$CvMXnc=q*;#GXgLNqGtrrHRNgE$ul&K+n(bHVlzbRbTZR}m|;9teU>A5 z&hc|qynjR&p%4wap!eSHEJ$~Sl;?>>Bc5RapJlgbiz9!z6aEv)wSet}8Z~X}k{oSv zvFmYsZk%SSX9uN6HSd3wt|g%hjK(~K4Oq9#B=c=K5w+i{LbW(~|Hw^{Ir+X%wj>rb zkaSw5bZ$cVf|wWyHuXRRUrCWWk%pV{qASbjgdtHUlK%Az^k6SXvJ=&j3*o8b0{a#N zm*K+9U~65_*C_}8eVr!jqfF;xY@g0cXyVN$aK*)0!@hT5Mpj}Q^`?`UI)SHi;yssR>` zH7KmIW%YJO8T`mtaTsSOU*S63bYd0*WP2Xie?XW?Oak5{-7vELr|u8@|H-lc6NB&& z|4FzgW+@j*7zl{96wz)Ju9Q_qA}ruGOgQGb4 zv7i?CrI$Ur=OUIV(tFeuw+bN<($X^_8ZvqeiptuBvE@%e*6F*6`Tl+o(*=>)t4QCf ztaJsaETuKp=qD1wo@?LoE!-xMp*Pk-TRjPA{kK)9Wgot1d!>z~;%@v%SfH>Cf|4uV6*T9v2m+OWl1~FsTJ=g<-$&U)(fj!vUGVt;Qp2_i6f4*>hz|rPu+xoB%}JRH#{1a4O%XeY+3`zFE$3|d(y2Q z@d+f;+My(^_l;3lAfC0(>XWmcsBhX?U8N<7bjVN|Rng10Yb&%^fyQvlhe8_U^TK2C zY^`B0TQqYV`mV2HNK32Q!%;j-2wR1yrU{8XtV~ZMMj5u5KgQ{DaMx;=xiMJl zIJ?DkVF>hdnR?izt$3GRt<8wpGOpFl7%L_Ij2||K?T6SI(7gMnV=(V*vT~k_R?^RN z^|RU#5;fX#f)M2bcx%=xf8%l^XmwPSF5)uAl<3sCNVI;w}TgcwH6tM<9 z5JIPg){Ahc{b1ZT`U7WHrV*8S)P@B?7>uJSJI z5)T4`!uY~fT@nnJiCYONqEil6z8f<76n634V)Zyu{j(wC#KHH9s0@lfJVr+R%}Q)5 ziXS5*;er?ke9;Npz)F1Pkl4-Ikn)r}Q>VJXrTh>-ZT!o16~XfuN3-<6i|#Au6*%+m zcuJx3{I|N7w;SWoMA9LG_(#Y>9b-5k8H7dHO~fCi!x-hw7|2&)efsCNML}pRPLU2t z1bd^t{lS6YBJ2BBSnv{ygVLc)O(6=Nc9CrXR(Gln8_AQv%cNiA! z-y4#|5w087!O;X;HLwO&y{8RxNAh+B-$`zfc*@LMrlg zf?Qm4$!a9d8=OGlv~JjnjD+z2mAe0b*^2c4O5OjlTiVj$jK2J@iNYj?o%jE>X6VyV zcA@^)nz>q{Y$PU2ndY?$1lVw{?*?IQZiZi$eSJ)S!JM+|D$R} z;si&2YO@_FtTF(+l&L^&K-0T8b{`bVfAe`*xT7V)LqJ$!q+IsX5vKST^3XLIW}{TX z|1S>{B19}-{zv0wV#|OW_*9ssp%SiQs8cSgzrg{cRF~~$MX&@Df>QaIQ3}jL?O=nk zz_i-{G!|uwSWqs`o$vOa_>w{sPL9es`(o(LfH{OdDAI08vUA?J>!3eOo8uM}dOmM2 zlamGz&mQdo49y4qZ)oxY@Q!e(P&Tkj2-V|Le0%gyHrJSzFufB}`z1INSp~rf^y5xh z*aJZLvzZv@G1-v94C#zpyvCG6{6$PEIjag8)Cw;ScfrS`xtR`gJU9C3`QaQQHcr^5 zAekshmB@$wVXag}pK{t-?h%g4=;pW$18Bldt&qct1rQQelSXWtj=Jo1rd3tO_k`2O zaP3cG|a<7^%| zMY2`Qq^~SG3cacsBI0UBBF565BwQ=s%2kx3z?xl?K8Nqxe2QWFAY^CMHG0LV_ecg< zC-`+O_TrprVj@9Woi$$Ie3h@5MOiouua>c|isSm)rNDQ7t{32%qo2F*7X93Wfd@>u z<%FM@rhYN^P9J>U&sVYZY>peONb2pU5|Qr_X`AsE*u1eGQ=lX>{;y0 zTwFX;CbrN3DXc>%6e-%w--v+~i8`v7Y*>RIKecg2=%>g%B=?MU68Gp+$mwCkX+dGz zH#3E1X;h;&?&w`fV;+#CL^*Hlk6V#Ktl2i!|6uTsc(bT8b~6p$4=BHFq6h;aS^Znk z2+&YP6Xa3P=Vved;5()Wr$YU(p~#By5(CMWk;?Ft*-DN7i0+YFAY=nAe#zP5t%jVp z6c(@$=m0{Vu{mgv7M*!}%dY3cK~wDF3X03*DN<>T%*`jA^NYxFHO+YkH_7h-7A?I$ z(rIiJn+59X`}<`!(yt!mXUm$+6!t`~KH;`vU0lk0 zX9WEbKQNg$^#@T?z;Y`ZNKjKq&pVhNTOra6v#J%u6S=GZ;rNrI2+HBSd!bLv5<f9xKjJg1D+5uS=3F=@nG=#E{ zYs^qX46aa3z)b5gCMERW(IF&Lh3#ei=#=X*ER9}h*$A^({qBLVNdgX$k=ApqS@X}; z<;?xvcyK^|J^Xu!z-+ddl>5X`)eFD7q{-m1NUP~z(q$a|f&S}j5m)ooYJtu~v#_Bi z6=gv@=ENl27S&Q9yJ+~*FkLGf)=TZ#;5IJ$Q+%TUg3&D&NSGb2Op0}W9=5a_3-H!^ zO|G&je)m(9BVcYuupkX59lr=YPd^eEfUzky4hQY#J1RRDQXJ~!?|)5PF;i+7$*o(E zJ)P94zdQCf#81TeRq)H1H~e{?MvrFkg-o^fQTgkoXaxpP&KTkAlPeUDW!699ridY! zFEV9e;`ct{#LC|H$_9apNeDpl@>f3C=)`XHwE<$%HOJ(IsS<6}$r+g^q`a~)qC96gAOnlB~#0S-Db7Nw`a$GQ2$+*|Myb-?;91ye`SFF zyD!Ojer|F!pS6FCQ`lqxxXGuVWqvNdJt$n8Q?r=aaM(6@+OsEkUk4mLQg^lY#{e&Paui;1-!n+5zB5OAz+#l?M*VOTtZ?CDN zF`loflMaZ@jj^O=uUdWUI5&Bj1}P4xxj#uk~!9d%`z=c z2I9-o;{8Q+g&OU0qeB9}IVH0ccHPZk=V@zT083u9s#2D(vF`$MyFvUv zv@it~YIIH=BRY(8^iCK18r&R-F@YVbYhN|k)_I)HsEeA@_M8HXXZpM|FY&MR+jzTM z1?_R0er@|IT=_XN2n%(`pkD8A`}u^CGQBphlbxfHx<$l<#Q+P#`TO<8w$@)C@e=MG z{d@upF2GexZjMvZUIM#VIaoQhk=`v=vkHzmk4VDj7TuFhXU>*EcQ?A;{-mhvYeq$n z0UlunI(UP>y;+Qdf;EZiq1xGc3fnU*<+m7-+%YaSHL9XeOpo3-sFqZFj zK6U4I5bmXX5s;TEd#W$N&2fcpBCJl~D)EtCxU(?4iXZ&LM-@|%A50ZMY3(h6d2qT- zXt|xSt&{%PNZpwieqpuk4eu4TX%AiIldUuaPiD#u)6phqWKvO6wE^!A%ZD~l$=T@b z)j-}R)2B#c$4TT(!rc0mn&YvL4XrH1JhIB0q^Y|8dWznD5SV3!eWmRPnUX< zX87#?&e;d`zQ}lk^bJZDNx1|3VyY4$+uw-`xHQDFj;o=K21bt6u?dg0SLm$?RtX3Q zSLoBzkF84SwXKTvTa4yQkMC^9xpYd=t6_U;{N*S~i+S=ajH?|&! z%rjY|0VNjCZcpzMp9L0__VHRLFHf~CgQD&o{KGPFBzL_iunqx@gsyB#bEDIz>)?kV z?*LfP0=mmjg5aauM}8lPYECTUsWXYG%O*ng-z2%ky%_fD{`&VJDU24++GUH2m$pW8lT{~~Baus(YxsTH3p z7MT}uZb!Nbb7zeJ3;KQUU)8gO4(}~kC}1nj1#YZiM=L}jo5o?U*OPJ92?=d=sVRbu z0-mY+ZCA3|CHm>a4EB_+n4Rv>Da{{U#fwwhbD0sauXqROA8#sb%9!lKyE-<$w1)`m zPGg?uqlA}KMEF;OU3Io88lpG&v@C;WF6*udSH;$$?QSR4(&fdhXxR$@!4mgsxW8!R zn@z^@1-ED+gMfsC#h)#HS};2%v~^p@+g@=aI9dDFY+ZRJl0 zbqLA`PF4s?HF+B6)ap4aX^dFzo;$#M@SF61vhsG3QxhnY&9vQ}QKPy>PAHZr9RPAM zSRLzaCepUH4M;+@b8bg}A!f$7qI-x zzIW4RK9Y4#)u42$8X2OCo$Yf@98qfh?ioYfS%rZmV|UGUB>-aqU-f2C3*7&}Mfxf% zB97)#S5T==QWf*P%&)?78xHF?^;1|icQ}hIpJEY}XoD&BkGOkV^gio%9S@2P&62=! zvNnhPt=Q9A(J1NmY{pFKfBM-Lv`J))lr9tnKoLgshY-1nk~rE}x5p?!a}WP1EQ(WG zD_53^HC*mr|EeyP)&43W6#`q$%>z8kSZ6%wNY2Aq%{<1$b(@T!7ok0z%xEI~<{qfi zSL%7umaXMF(KVv{&zYp5C1!g;Vo)}noRELuo7gbOF(pmsy)BKeAL9O)@G$-glJ5O0 zw&Wd-^nrWmB7tPj4QiFe@o}0ZtgAE8RR3F+LEXz;6|<(@fv}t$We6<(H)VBZt}8)Q z&e<9W`y#1rse3}_mRDXfPk0^AQFZwOWfarkv^x+>a@JcHNKUJd9ho3+G5VV)-3ps*?3quTZhJ~4{uR{{ zEpNoV-El8uF_{X7+U>x^iQq+o7by(7U>yc^pAOlMh>E1?{Q6RZ%Q^|_%3ly!3Z!Kq z?~E^2Isu%x`KJT$>6TNw|7N}>z5UvTK|l?g>G7sVBvyO?sg z(Tks!tAi;_sdLIoYlM8NIzu_uk4uBoA=uiNP`w_kBmD?4`7IAbG-n}6UCT{LoVTOy zbz&$R?%7ED4L^U2ZZv)Ac71!6^;=}<7vMe!bVt%9X-lK0-^^u)FyKeZDU5#w0g+o} z?I3~=XMDk4CT_nG+iU63MC~U|(odNrX;N{E*=CAXGhGFcj(U=#?u?`*&LM&fch|&? z<24Es<$T4mCL1tAAFP^4=)vMMXa8={+a zkqxij6B>c+;4PtNb^*cn&;h$GDo0W6`~|u-tHz~N9F=CwY|__%8=5GNn6(n7)olY} ztKMp;={c`3j~fMl;UN@NRBObwL>WjCCUHfbQb-VRP;Q6i*@PC6#yYqY>ZL!@eLsVr zik1Rc1>FM=lfhECb4LBWQseMJ$bCsj6p9FSKQM*!`1-YF%rO&Y{<5xm>T&W|&CuF! zo)!FX!j`r0ejSVA%TXtph``WZh<&FVsLm+==OJXnGkwQ0Di$>a5bYz+YBLsadY@9z z#9XW;ltSA>)+Wy)N!n4&5X{J(W0KZAW+n6p)Ds8h65H3fySX%f6IjcYC933#uprq< z;}~K<0rHs`ten@~+tLjFT39=#OgOv_GI=m3BF>^T;HZ{8IxOdSP4`g2JP3<7??0hu zA)KH4C!Ny|b#zJdSQiapr#LL--H zub=)9laN|KN{Bxu>Tzmt@nCz1!6Z6j1wB7>nzNa)%&LX45-NK&uPcr|+65wKFN}#o zsX@EnUVg#)bS@GC(H-W_wHszA8+!C+NmK*Sn$Eh&DS*6AP zi+*dq!hg{Y%R_D>E?0cIQ~jiUq^?&qR~GWGKXC2Cx{fcIbw#v%hS6iAF>G5O37vs= z?w?_M8#7>HUPVu_fN=$F>jZmOmExv#vDt#WuSB$ewfyV#wyx^5R!gUpbB7wDFIR*AT`Ag)v5Qx^tl*cCK4fE2Eyh!&xPrY^qDKyMHN+mQE!*ho?*Eph4 zL>B1NAEu9W6b+TMW>OcaxL{Dc z7{(@27LAU&i1SvB;_G6n_1-PX!BQc*J~F(@r*M>nNEXYt0=VpY?@jOV*MXB?E2P|% zYEwTe13I$|45n@bdC!#F_8{|}4@wX}YzxUJA2(yt$!4LOnFR{7j{-!M@J;{R?V=7X z!F_|(n@DzcK{APGNZY_^ZY-oM`9pF(MaMp+4 zkA)c;LH-LxTQASnT#|S&S3W_#Rd=}YCzGbN6EY~`Uvo`zijnRZ?@zH@RlW2RglCHS zfM|aPfrUNowc=qu zy@gR&gc*Qbd_i`<^Q(tUio8=6^P+6JlH#_d(|1QZ%2F05OJW$Tx}#bcMQbN0ll*Qo zT}ngdUk{HKsiX3m?w>bpGnSApb$*XJJ3V7hg19SibAAse`&wrYyR|*uT636NPhb9B z&da4mzVW0sa6g_C&jr|)f9ryfR#7DuE_+sp<>CP%p{^c1%q^}J=^u6uscRRVE@tL0 z&`jWWZIOL;d4=8(gIrV6dR%~cstX!I!-@WD9RQy$A7q?A6N%K0zOM&TTg*2|fn4?t zXImzc*UuaMi$rFM+}a+g11YH zW<0_Ds5ML6U(b{!`%dPnghf>Nd5nSK#4gmR65x9Xf{e!b=@J*iDdIT$FYp;T&4o_O zKty$lo~-rLT|dmK7YJsTU<8rVGmzPS*N}yJ;nxuo?0ss9?o2;J?Z0%=oWDr<*BE=X z5dW@Jo{>Fw7`=g)Jx71FXhZ{oS*^CN2={hAvY@*CI%qb9C1^WSeK$mVz}@ooj>CAgr$)eKZKJt}(QR>ZW zCa8~4OEsjcV31JK1`*zjVr4)!F>kfwpE*Ty#(XSeJ2&tu4cxG?VOf7+1s_D{PBy$X%QjW50Ln-9J+)t?J z{0!uQ*bY4^X(-4=%Fc8Q_W|NiYje)8m6?8NanVhoak^cG>jucLK3a%pDe1m*y6xmLwY1o#7g$(zCE_ zmEGM}if!ZP%@v@nopUQJeeXUkx_M-E&LpSYj#7e?PxaPW-URCBFQmrw(uR2^pW)@p zi!J_4d`Ai@Jbq=Bb4T6Atl&6dYGJHGU918{{%kzO8E~)8C|SgPcb(*|es??k3`li6!d$~(fP?7f z@#dqFm%vBRDlet-)I8KP-krfwu@CSF2C^WG9E^!bmWi19R@?VZ~x3)`GFvp zR#TG_LBEsLY}$8uF$0&d4c4y57qlD8J0h<)g(aqnUcw0dkftjmGHFI>t%cp2pe9wm z#+j{tIpXYsnm@EP@FjDMj$eopt_CH*gn{j}GrCg`N)_hbfkwrbtxk_n=U~M;3%$%2 z(WO@)-*6NPN(E9^+}9=mG9nveH1IB^)rGQOTE*(|!k7}O`SAs#qNWH%?A<>o8J2-t zGBu7-mazUK4FiQ!eUajnV0gYKE=wBKyW6HU2K=v<7dF~hWAL_qWS$y6=o`ueF+52J zbF7T7TE0#8{muSrqG{gXeye88E1wx?=L5Z(mrMgKaAJ`TSN8}xUVa+G4!9xA#+N>< zf^=s`lE=YW0rCp{!fD@iMdmZE6gE3HYRR=*cz4Ms<*0U!LMIRdCR# z@`|dHjMXyL8WIcPqPp@kl*!H=dxB{DIx+oOYy(%S{ebl&cVdoWt^nPY>)682hCZoA zI{rCs|8GSY81>`^F5NSj@8uws0`IZ}amGv1-*)qa!>yX-xNDHp&^VQpVGh*;!~Xl2 z+;`0zA*T|~Z&3A=ArEOfq?u#tuwR6FduiqX?2=puJFL_zjp0L4PQWa`e@T?h`$_dQ z7F?!o;Y~p!A-KSJ(r~M`Zx>c;3hzFt?Yz&`LeU64m7 zF{H@erGSaj~3jLy3iui~t7X-v40<9ju2^;Q+S(aMdc)AM{&G3^( zIPsH)m#XB-C(&7D0&O}Y(SJA1wn1g7P44!(s~7Ijs+$`Y($`_>rTK7uPcmT*9g*qW z?smDs$Nx+M1i_1<_tjCL;|-1d5D4LUp1U0{RK2(2r0RUi96EX=Gdk@&<1CTopEw9F zvJL(2%2)$^$;sx4LZfm9k?(9V*6VY}|0IwZ1S8J5NWFsv1I!9lbtI5dH%YcF9gr(3 zC8)F3{_Cw)g>qCuHA@VJ%hn355?FHNmUSvHMIsL8l> z$W+@p0852pDD9l+4LZM7@pt30@0ZiT9#Pi=a{)-pLD3?w=Y34pV_wtd7}0&lq2s{Q z%-cZLR#XYn0N|}EtGTn=1Yk%o!K7(jGUpVo#c^%lW)Xk~9LVB`*of(Yc4v_jK;jla znaL6h+-)hL`uJ+vnz@|t{CA5{P6(-F&~U^t_1JBCXm;(}(V$W5>K-o~0Z9mFN**pV zKO#r8*uOSj@5199WGMo0#5!-dOc)PlOR}3l=A{Z2x`>cn31BM9e|ik-BGw<#QIXBWbM& z2Q+F`U(tJ~Ww;dkv0(vthB0@-F4B+|q@xihvxlAtq2ew3%5weUKmHaag*H~TUd^MD zZ4x4v-)NnsTcQ~U-a3pM! zddJ|*!fsnTwr$&X(w&ZNr(@fB!Y8(E+ji2i)3NQ2ZGGAA{?6IEPSyIe*56rotvSX$ z$2C`!%$$$cVnnE04-y32ZN}xIL2n-WY8q7C?M@x9Y2#F_Rt8)6rN{EHJ6?P3oe2L0 z+O79TWkmr+SaO7PMKZ>ih$m{@9!~MRj#x8Tm=l@3u(9S$dXVgc*wl#pg0CAEiX#E8 zW3ajj*@=a{4>P|Wu4XZi6OZ3e4hAk*(1j2hcX`t!wp@ANL6&-B^Kg9;H2M{G@t$$d z{c{JhugNyR|M)4)17ZhicW9qEy|c&7cU$id)vSO%gudXwpyg5eKf**WKMp{)g1IHv zfRx8KRP7U`vI)hzEw;O6|!4mDrs3i4xsN>^I^IH)c+} z&<+?qu>^ku4}_<8IlQr$2Q+4*?G4J~cqDinYDR}3$%rRJNtw_h9qP;aDf>ew28nc; z9`}K8j_~SXI`z^U!Sq9=_1YV;l|v$$)Gv1D232F-?~XEtJSRT929#F%&^M&EurEhV*$wksS2Bh^E!EC+T^(3Q z0laN!$O_vgO^D?k8>d{}0=Slag46g&t!E%{mJHj(;LS64R_GmRB*zOLQy!l4?`U{f zyQb05AI7-V8pbPfkDcxnT9?^YDPE;`wK~d+3{Uc$6WuCPPFJdp8~p$8Ym!%Jsu&|v zfp&<~eLIQlX;>61mfZO*cH-iNe;2%o`Uzf&b(`owF&z>VDveo+Dq}4f3ZnK8q@tk&7GL)}sm_k@sI;Sc8C@4?hS=TC{FFq& zXC#tlB*Kw8j_09xLnj#{Q%EdpWbV0(hUFvY+U<}$2GYDZ?~nb6b2xZdv2m?w;wx1- z?h?scr9p|06+N&bd-;iw;DR1bDHGokV7eNB&)|Jl77xw?JrwEDq6agA?gE`bp92;# z7>r~y6b<($Q1{xb*b>@~beY5uf%kx(;Xazf2e4T5Txj0d2@_mIN|occc{8w4974dZx)Mi)~NIV;yj%AW(%jDl4$lI)Y=xn89llWsB(Pgkhc;ruSr6$?yR2z97B&>om>-`L+OD;pIX*`F!?A9tqgtE{aU@UeIX=vo}+Fg5g>?s-^<1{4OU#WI%u)wu#QqKKXbRK-RM}1h{cL(Rr zLUqnzIEq`Jd_fUvQjlpq#etMja}|ug2kXTLRVYX+E=t_ajcyDND-3HAAkAgBk>vxN zhi!Y2<-o@uQtme&!^kOQ`1gYI*O>C08n@ywK99Cvio?wBJ($<5ekiz35KZE1+UOs) z$jo;1vSx+0?QHH*PTWV15P+FdF;*-tqvjA6057hOJ zSkJ~{Y}902>Rqg$j)KH%Aqe(201&c!GeXg!Xv1ttw}pLyGI4GW%-|1~SaM0mouyk~ z+ztlUC`LJp{0eFAI2au~EnReij!FDk&=`PG&%9g^pJtAxt{~}54J2+R#Xb+%wuNzv zYjyb2Q$_Ozo}_5U(f&;G%f~S&QpI82jP9WT{W@Q$ty{C+5$7JNQT>-&1WStUS{Dp? zs^c@YlQ6n5sZ3O0y^%oMkZ2m?tpQQ;p(<1HHQ(u4s@TCF1p5M#R5KQgeXlUid``o}Hb%k%pOkS!zDJAOCdxIi|U?%=n3A-zBoUOQ12m6 zr_qgqJ8S;kv$YG@@0sTVi)ifa+NVst;3Q%9-*$}C*FN$(ZOnHsg`S_4)ObCAq9rqz zRi`LitIKRul~IUPkdf9>NLP-fsVB0k_5T{jc{LZqlI@S(%N~bN|4wbSVs0D+T>PHl zc*oh}XAv8S{ewvRMTWO1XB2t*q@$WTtSHF(>p>8@uTON#80{qjNOrpeLM9ilOhJnm zK1!7)`fyUf8`7o(KtZP6lfFrOrj6vejfrUULLFgKa{H7^k-}igcXxBxcFhI9z#-uC zM5rNS|1kN28f0Z5w;JZ3dygmW7FTnxf|I)y$12s{pS+-bV&NYq!aY!I4bU|Le>WuxA!cys ztx$M7WW)3_lEJu#vv@1t_ktTS3otzn&^j^-FxrLPd~Od-?{LVZ$=9u$kCv-kBvQqb zT#Za?uT?jL`4LRqi?`JA(Uc2ply=l(8>oRpxPgat!Nud+XLC3cL3+>q1|07G zA{*HzhaUmj9*xgWFyUjK7A!r1qD%QfCW1l8)MSwKW~dfQWG(s)tWH2Q)iF`gNziA?aP0k$cFLANuk9Fx215 z$HB(fwn}fAlr%xYQQ?BPimDX|omcnbEjny5#O2H=0MF zBBoWSuc*O@5udk2uNfV#W6{gVsYl2YHz4WC zSAb(5ozKQQaG{#@+Vrz$c8#3FnACb;vP6!rsxqZS zvl1{xJG5cs?n;K$jX3dOnVgoxE3F*OZHoVK;-_u- zGw?jf?1XZA4pu|igR}24?~1<|+8bhCgyrquI^z9>8J&IcEmu4GB}tpYXw4{BB-GW2 z{SNmS&c=-Yb^#v!ouX)ZXc8Q7s?H6aXUks-&-YDWvY^zKvwYgof6zdM&L?2)-`CRL zQyL^cW9{nSs+F-j9Oq}SHoaqisw2}p=%8C2JctFl1B^=MC#c)sc5o}QyN42&p^tAb z1FNJC@Kpz25My|#lUFyY7}d=y)|Km>C@Jgwa^Od28NwRxNwhq}2mon7hAwEh13R=w zJ(~D8L#AZ-GF`gbXw449BO2zwL3dW)>9vA(-SEHn0>(VA?&~zGE^O}GL^em$d(J6^ zoigpF+TFBAuSdcWP=EWmy>@jZ!td>$`rJ>;Uph}Y9=tLmX$QPh_9l;2FWzRoO}qFE zd48HX!<&ri7*@T5989bO^AxvG*?%1GJ>7T|&$T~;Ya{W{X8SSkAXmSffW+S*Ma^GJ zq+=$HeF0{Kz)k%ey?DQu`Q(z$$NcEarCKS96LJ4cUwk=!-pl(-{ZoLe;P0Q_)T+Pe zeH)#vgy0AMtS!uzJBOc5IOd5innG`lxmSV2appph%!(AB147TUK&@v-<|ZMe^wl5n zKG*XjFJaf_tJ6>Y)5M{$r8{C^Z$tYWB1<^|sb!xHrw#2j|8UcUiJnufkq3%ijiY7+ z=07;a-o(rXfoQ(uH2y7%?Zlyo7+k`FRiTL2^qOGHKXH7mEn7gJp$7O~dveq>v82nn z6;~W754|<|IIiQufR{xOFNUYC5LOocm7)6(9tnA2c09suITSXfdC>BSB7EreA?kD9 zu%8LnOFv0JrL_joG-Li8WfF|z4TD!7;R6yrLze7XcD8#E(dia3u%094hOc&>dc8U= zujwNB=8;Ccj~qE8kvJVa6|O;?@UWWhdE=_r zSL@Z+Cop&am>c-T>KA&QAJhB~*$5G85a=L!d>w?7gRn=7aQX}4y3zAwi#@K;5jV=aWzae=IAK=B5OY#bq9f8ySdW?{AcnRIwO!n-eqcfR1Dn#v zWce=Z0_Z$iNG(gm@jV5W<)fXp|7^;yZXrSH6u7H5D*{wa%E%5DOEyxM@>xiJ-+@hQuz}C_Ddb}NeYvMpd=#*poFn@ZB zm2!+mV*0Lcw14Y0*NMdrVVCs|ztKwAl@*Wqj$oHU2+lK653~0N(BGeU4;=glTFfWT zkNiskPb|fn-BKvukX-z zOy^NoM8amYoB%IuYp6SUbH|+2>Mhpg^hvJ$Q%b8?*H#Q>OdAF^yq382WJ2j;LTXDy zX4Q&|t;W^|#V!qDppCF41Jw}Ps`5-~AlA@|W0jk^z);e$x3&B82<)$uyR4Rx4vce# z4jqiszRVQ~jp_0Ag^Q9&lfds$@Ek$IaP1w&bgs>)0ygsvd?c*3-sNIP9IU$o#juzn0UK*4jyL zl{n1p{l^JC0O&DXKqy9cA%eKQx_q151EIE8vtK2x>+Ws!z8OXXaCBapMlbeG8DSC* zU-S)1SUHMS^c0R5E=n7B6dXZ>QLc+BXx1$si!fwA1>r?E!{9{^kw*C`i1&bOu0)R_^?3pU9}(35*FXeyq-_)Mah!U+1@$SRHQ$ zaThnr74Al{LF6j`jFN`T(iK%31j0_#a#=eEfYRd3*68J>@{!#OQsi#3#RT4J(HUEdK#queKA?Et zYZcb=0b5WP&2qN;uClU4ZLy2BabDb{zY)yKIR|Sf#f(id0z@S)g6OYe#JJ8L&!5d+aNhTec)9+}quMA*f6O|lu zwqoCAhkka@F3WTwp>_+AIpb6p`4B(@_DBxBUc@}Zcpi6Yo(L}G_F&{^#}RJd%3Yye zfb)}KJ4Jnoo#AZO#3S{g?7JsQp0_lV5Vwmk5RT~4botxMVl(`d{|>=ijD=3gQ24MlK$}_D2C_~hC3lG)ChN8{1##;Se^9RIkDM>EdI=qR z+dcTBaa1;2WltK^jS<_xL1f+=Br5Tz0w<_|E7LYyC#VmxnyyF_SIRnrGL9OmE+w^`+ZAyUpa=8J>7~r) z2E^U$!YVSxd|qCTB072W9Ft)L0^A*=XY~hi57{T;xM7&maBuX{=;J8#3DlJ{fL;pd zKT-v$2DEQ!1p7AR(WX=mhAfRf5Q=jS{~S9&P4CNf_15}fTke~7#cEuG>K&flSwExH z?zc9vO(6#>v;m{va-F|wH@^6vzWP3ZPG9;PU)+Dk5ps;af0g6uff zuD3NZX0Lp_44!&nd=PKaCtuqpKxjqWA$_N;O5=rqcwg~*>Q#6!0Y^0_m81t^6Acy$ zZfMjFe@r|in$}Um-vh#kf;Pyy@jgbmOYL)Fa?a zI5{RRP;x^xwYBUbXT*Mv2d@Ksm|~Zukw8n-a`@hV{8s?s|1>3NIcWd8b@@+j?HJ(@bzk6CX-%=TtmLAVQF8QWLcUfY?@e;T)(r>FMq1-wyycGEcR+c853O&mDO zy0O0qBr*e8zS)mNR8OAuP*0eq`gr;WJbta6Zu9;|3m>D;6zJTs0Ev!=U8lsD|?`;rJ+he>;rgPb1ccE z;8pD!6#c0YiM5RPHAYlJLGdunzYACt5iyJbg^V;R zmk?1x^2{A(O2g3DcYMvK#lchLZ@MU|)1+pZecvcYte6dA+CdEFlSrJ5wD>Qx|?a%h|8 zosb)kLv%lDz6f#%S1VMg$!dOyxIwaW#c|%aEVQGPSbq2Eg`Do`3xu|c^Kv=lXdm3w zO8w>V6^)mcgulnCD=mFC(z))Sf}9ZMKNNO_5a$Lv$e6+&%F0aJ(q>>}==pPP2F$aYfV4RxDh7HMMTvP+HE$rJ9cJIP{#5dHC zn}F7UALK=O(4^d-1v|CRnE{|KbefUum|4MQkY{yU(SPWlp+C!aq`iS@Ybv2@GT1aa zEh>!X$RQddujqjo68I_}R^GDkWoo7x&t8RA8^qzKpD4JJ)oDN(ma2zW5o1j1tsRYP zxX+;jHOMcsX!o5TRom8JktPohBc?A?P`04I)iJwbzUVA=weZ(4yZpH!k77$_;$|zK z%bmEMA*~VyiA2bD=ShJf!K-CfiH&}jsE*z#FC65b0w7|Tk&QI%zF9>&7QHYO5lzGaIcE)gV zSj&Px^`)Dy3C9)9ip5XY&D;6aCDA9=rYVeq4L`{zu9g?s6QK9KU@EKhLsT^1fwrGf zz@-pJrr|M1&V?>9J6&6@1A{uZn2?aM#e1rD&&|%ikZ-EH2r<7H)2gF_v(?q5#?D$h z@g^qSkyU?iZEfA+aKR*dIkQ!u!}GrK71v6i1$8N!ej9^M7ztb4oH<4fxg+jmHVHht zW2v~jS}WzN#$gtiUMOBhw#3(xiR<=xi{VRdw14Y$Uv?AD+;&o6qt6FzG+QnH!)Z$s zKWpvE)KcG&h9J7gV|^hNf1^yWTAfF4mOyv!eLJn>rp8dKaI%GsB;R_MFS?&75@Mia z?s$&-lX~0_Z*2o>aos{Cmh;GMe;B#+(4VceEpfY4Y@r%B4BVU?wB~w1*(+yY;!9^D z)(U2fl;`!zVz{ap>C);+M&elBvfb=3ZqY>gt)j8)=%PFkt%yHrnZI@SrTAwc-z{vv z!*v8xlA!>~6wy!lxec>Oi^x6DsVGD-W`TJuUYm;7j%B+XKaQLe!l@|dXN=1X`l&eY zRYB;Q0&gj>Q9s7zeS*~@W+7=j{;vn`W4h;k=Dy!xZZw(2is)$Op^ngV?LUyg;dh_W zbs1(b!vZEee>@}oNa4ip_?0`0tkHAGL83Auu}?kNA1y~1qG9r!Jg7r)7_6wNGqRFh zD*ymwIUrgJ;P{V9FTf0pn52twe;2c4ubFBW_wpEs3q3evUm3m}0d|s?z}l6e)TRvD zNtw58(2T&uig<)O!lm%b9v^!x8?KsCn7Fxnu*nA@qnky5sfilX(;Dqwi5tfcii3Y} z#!>*0~E9l#rC!?^vUapI+x!syixa;p;Z}S-}(6dX@tW?a(iCb?B*?zckW+ zZTg?0K7$$ac^w{iDu#918rZ>4GIFy8BJRLZ*GX9#c)^@+AsB$qbp}-Qzh(7G&-W6M z)x{$i=xXDfOW(GGy0zoJjsJY`mBkfX-C6<%(rg1j=D99Kg6&^vMs*B|i=6#kzPmW~2TxRg{f z*_O;wile80%Ik^!Vsb|DZaWcD(>!}U03_M?~RmqJ$%_dzsxJ{nE} zSEIpL1KL5!?usfe5^{@QKL|>8Ju*8>b#6;0;Os^#f9iGO6MG~YixuKKSH)j^m6i8v zmB2~=gsT>!U^Hrk-87!Ok zI^3o1B7Js~mJX(p*KpX>T5=b@SqqA3!oDg83G7UbLjP zNvH-WOY$V26&a+JC`$_DRrBeq&2yX5Rfl}}UVT1!>}X5p`1ywz7Y|Y(4^r@)WEjVXeobOTvg$DO>nbTQY482R z8gl8AonUmXwlAtWOfgk2VPt0qMw^G|j9V__0$l&vr5m)Hf^`NvF9^_!QC2^F6BK(z z(!Al{n}zrSDyC}%0ra_!2T&EN&7l=O@CVBhJG=T39{kG}!x3vC0TBU#)YwP4=XB1= zm|Njmn2@G5uTbv6Z(;4eq_uy>Dc%0ODCfChM3zC#YBNDT_-CLpn3YiiTcj7lq>7fT z+Qi7IBB+ug*jYrXh6Eh3ti;$;A|5Qz8ULYFx5XWi)`OxtBJCA`uZvPMp&v_2Fq2X@ z&EonB3Q_I%%sU;*TL)fnWJ2fEgWnaeOYf%d-nMP~rEX9ShSwI%(5swSptIPhH+zIk zcP;i5pA(*7P(8?EHHNzZ2h1Aq3Q`eN#_D|8Y(VCN#D{IkSTk)G?Q@Dx1y%)dC{yub z!#Mpvl49%_w9tHOGGUb(70#M z%}$*N_<~H!2?N^jg7-2I2*LERp6}aWTD1*clo{O(mSFJadfiZ0TycA!hx-hK&Fc;= z4Y<))7;$?8sJ@w=Zcr=fW(x&48a-ef65#XO3pxC6<^MP zAZGRyVjaLhT5uNw&uG+%hAU9B?BMf3eUMELiSV1cS9h$FU7xup2DVL!UMs7V6#Y9H z8LbwkYUJ7XL2su5alPlwTWUZ48bn(fK*OFZLe=@ApU*!>r7uhyL`=44Mj>v<2gc?o zdmhaVymiFnczR;ZHwNXXoGAU%5?25hoH$9BoIbg)Dk+WE&5?SWZQ_=wnzsc|4dc?< zrUR1p9&s?cN^PGaGG)PeiGopTl_)#2OCay}3GX*(Kb)8Uo8a+3vg&_pum1{vYa78E zQ-1R{a1*a%sZ;gxxuAjBYG=ydn2gU~@URy~sF=liFpABpq-A+D5vgPWSSEuvp6MoWMO`Ng86wLSQl)>`!J0*PAfHFn)FU9f<$B-WHxoWtR?2V~JZo@HHHYEj)Fp z+Ec`-qu1M!%wMn`sdl0bzDlxgqY{=bQFmHw@!#|BXV+zVBV&M84!0GP)~aqog*wv{ z)q6!!)h!E;6VI&hR5Co^-nE*lLKo|4#QC@_2)6f{Q%Qu{!p@ZDp17`);+!(P4)%B9 z)hr2e##D%S0r&u=f~H>*6tZfhcbvW;kp<fH5>4fQ` zDEkT?*XqmK-mGHm7cR`O0(o49no7fUMH~Uq_D*{ONj1cxIl9^C6?hDpZBq`=i3rq= z5xX=UI5E3C1F$aa_j%iczeBv*rmsOv!4vJb)-Z`^z374WNp#f_G!hyf{sdgCL)R$tR79+k1}VF&IV`FyBP_9{9H4S{+%x4Wj^0cf z*`+epCh5`}27-~rRmlad;f&xHsQRd;aLGj{)!B!E>+Q=S zN7y@hiz6*!aIE z=o>fkKlIQ4I$II#Rti$zAQDB0RAFXV?9?3q0a|Jo3oml3SU{=qe$_{y1Tk+n3~P~_@loK5>t;ZhH{2C zhsVWX!xqFWXW5~F&?nn6hSE`w*oVTLsjM3Mk)nvW)et6?a>3P_0n|;`6QmNcTXw-Q zqNEM(sBhbcp_`s4sg~L8u=h-r89gfKm_daN! z)z}-bRs<6Y0B;5D66a{^1jE5 z_@#um_;Jzy3?-6F7MCoY6gb*JKOeA{pb7C|ixrEgp9Luh`U$|LH@?9&on~@)7RncR zNIp?=uew(xfIWq-Kh=;bq6w`NcMO+LX!HAyZ?0B5+<)J-QGN6I|8H&ncV+*}u|j&q zedYMJshlZN`$oSj`wf*LNbMz}LrC>wBY*;KYPstYj9`A|&^Ws@ppp1D(H)t{aP=n$ zvn80uU62IXC`Kdc@Q#Hy@L_2>Z^U;BO7X}i?eT0{##l$FB=byB3R6&|WRp?KD`ag| z&U>lp-cC97g3735Z8QIQ?>k>_=f2zOdfDQ(PYEUZ8vOuS8cQL;yB6B)(%%Ia-iQWz z{o=uXI?Z`=x$8s8M7Au~ml?r?0(*9V5qp+|34FEyA)fzuIy(xoBZArW7SIfe-A9MZ zMaU<}Rp!C%r}ft6&V@B3C7PQvZ+<++5F+kp^A_HXBvsv$A@#=9S9-4K9{}}c40679 z4YvQcWpDLiAAD^qINN$d7<_el{Nw_>^Z8gucDWM?CZ>Jo3DSNSjeRZepZ--gFf9Ur z_ye<(3ik)jkleRA8CZC|j^Z1-P5&;u8w>hQ>-PitIu+y-uXp>`Gyf-QKjpJIOz91? zFhHX=JFIA5&T&M(P0nrHl`kvpa7|8Kz;`CX6+S9Ov|pZcsM=*s9{#VL(B2v_+^w^l zF+wPi#uCSrN>X>jq#)W(Xm0_oy zzTB|u9w!o}GAzAbA!e4SxtvUkaVK}4Fv}08!^_Ri$0kmD-nwdD&pgJCPF}LK)(^b1 zI8F|ULAP@3D4uNCiK!x+K3-!WaFgR6uayTlC^suRP)xweE%8HNfiXH!8Q94(YhWA%4B`on)uHs(F+hbY5|fToelIu$oEGj2-ujjqS~6V`^@yWid9d~*@kg?6oi+qhc9|H;+TzND4 zB~ytkYgjl-!l@gs(h+&Hw0%T z59_P`PC9&r`c%w=4Ep1)<4ERtHU%^ldCs{>wTiZ#{PoYK@La9)oi7%&5M!ZEi`0W_ zf>ou)xvUR4n+{qaBi`{@iw%|YMi&v^)OoZyVMdZ`Lx|w=NYoBRe-QJuR==dLRmwST z4vzp69A94h>=TV^+u(}V5%od@EnQVeKKZqF-z)nsvU1)dBGT!#fjsmqZV{J+_)dw! z5mkYMrn9!HY|GxzP!5_g6#(s->2D1gA$kK!$f}BDj9UtzX+aF43q^y^uWuyp$PT@g zwe^Xxo!xZZ20@NNbxt0KHGAt{1e=JV&{LrUHA^U1v4nJ0S!c}M1}iFl5C5F`Uiovj zIzoO3^JoC?ULL+N>ls1A16?wW2pAg23>7IjH@W2w)`ry%RM$!`kthN!TA-RY(p7`s zNz-0*`CnNeGGZ)=#>~oj+rjd6U(q_|-)jE;4T?=v`4u-WzH_Pp=nDrj^w`kEatE^X zx`DA6me5BF?ug_`PLrG#ZC=ug@{5r9^sOzE2vn9LQE5cl#Fr3;I42Q0ll%m+kTlQx zwep_@Dwh4{9I>4c_@#Jdl^}MtR{)~!5QY@W5Qb!+sVAd3t}?BmNXo>j0J7N#n&&%j zfRBQ*`rF20Q!u8qI-XNa)`;$1KKW7vIve@%$RQzC{c4Rnhi&KXN?G(S<1$wT1a=On zD^7!S1tPD$dSlBh>FFgcS^Jev!Kf@+jZY1E*b9+vswiw&CLv>Z zX}o(Odo%`uLvg+Nj*Cb7Di>_6gptQ*=UkI^>bX(=yA*mPK!285&R+18oQORnMn-0m zuF+_*Q~Ki_tu!9uD#L=dkge%}vOz;d@H+=kF24V4e$X?yH(kq-57$^$J3B^dx1~n) z_ByURLn|jmfU~5AA7YW6xfw8>J!?7b@H#RiB56E)VMN*By!_Y9XE{qeG4w>l_o@7A ztvj!|d8ieFNaa)hxM}*(XuHR!14WUR-%iJJogM+u$WVLx$B7}EZm!GKe69sq(jyV* z>TK{crp(G`4&L)ZZ|pi4aR2rzcw5>9SI?Vee{j7udLz%v%+dP6)e@MvooqPArmy8V za@%8&VJzKz1Tm45lw7?y;RD>E%%70H+7X>{)WMc|UH?^jVOP&zCA(o8$3pr6i1wZ^rRjs~|SsYp|h4ypg z-g){lfmd>0Bp6I#pkgTnYqNcHf^OUS@?YYkmW5~2E3mT50*dE}xZMP(eD2va zU^7lQVpCP6-3*LK=QY);qEDAyH;(O|s3;-e+|tT@d;O(cr=MJApFbbHH2KGTxGhJ^ z^%Sy9Aaec&^K#PgG}@(MzWEbcNd8)`dEJsbnc`%iDFx1J4bVk)k1iVXS|`;ZqvH-4`HF&+W5 zZA-||0Cl21G8FQ9RL>koNb8Fzh}i4md?MRMI82_N_z@G(g;>0}xJV2XCH-1>sa#Hb zG4t25NI@XECWW};PD)ALiWyq`fVmJ1X?SJ(z`NF)YCwOQxmRA;cgkoZVBBIS_F7dZ zA9Q1vwaFh=*A1dCTaK8hi`K4o@x|ZV6sC6Hvt00|mu|`oGa}xRdUNDVEiEqH$89Ps z20x#;D>CmdpkyQzMSuk6EFj?r=PdZkGoyITLPCQobU)eq>RZYx3yCWrHZTlTzY#H>*P#PmC3 z#?T2U?7qCb^vi>;V+%x(wUoA3VSAF2Gn%^JdX9H()gON5hbRDT>-;*=yVg}^W*P7K zo&fRqw`C;w?S7vHqKrs?J99mC!#6A{RAADAOBHP@5^3&=gfYY>qV{QoNFT(&)4cfA z)A|xuKRr6O83e+0=)xh%%t0teuG!cDa7vq-C0^2Jh*uH=2#6@SLuQUYj(|Au5Uqw= zB;$o6N=n2O7Er)?7Ct{QaF$qON1qvljG>a*SV|0@skzqzW17X5=y;^Ye-tT5Mp_FD zd@+ru(c{9Z;>&3++yZ`wBTXm5S?G*9`JL)HWs}VL@W<&d)V%S-%-a>G@G*TH@bZRz zAnJ&D9X-7aanKIFKh!fJKQ!6>;M+iqK{#6tNf8&Uz>$Fl7miiBNJ>7R`_Md!B)P9Eih4qO%M=oxXN|&op(YUz6m>ye-cMV(%DQeWKL%k^?Y(12y(J z^a81``O*WI(Mq|cVC_gdHI%LT zeCD#Em9SV#ZD#y;=0bwd2#eV$uJOvB7))ElY~ML;p`Ow`x`(b;hTmjID0y- z2_W&1bTD_Wc)jlv+Phq-(mhTJ-A4I!3UylYxtDcM|N47azy%=@w?GYT__UH0jpYH+ zA0IHv`$As<%BpNhSW3_?@T*BAz0;3=%c~&*L3O7QdD*{+5Ovf1~BQlMx7q-k`#h2iRvq-rv!7*yRMPX=_|gV3lT zxw8g1K8OSX`*paQ4rC zME3s(uK!nxaSjHjo_ve!H7F1e`TwSUre<8RA|?7ILZ^0Da+9SZvhiWG#^JGo|2IQc zwuaqn^INN9pr^)HVf{yOlcd(t^TGpFTFDA}PUjvNG z&iP82zy5e7=F66!dpajdN!Db_ehhS*=6&m#zT0wtc$)rvXZ0g`&j%t&CA|zKID%lp z!n6G+%6*Y#b8E8xyQwxGj6|5wi zp}7bx-?@=GV_{mi4WK%9q*eQ5Zw))gU1ETh_8MdL&tL6YV)M8;IZjCvR{2I&OGYV? zCbfk8E^Mt-GJf$Y88uoT2-)Rl>&hox<0#{}^CtQOX(g4|ba&ipcMVRBlGNd>g?SjY zbS1-2dn_hXR;I8&$b!QYzSrUmDY>%R<5CGmM*r)uI*O2PuL35Bh!|(YFRDe3nq&HO zlx3_swWncPZD1%9YPgt+bA~2^88Wyn7`PEo)f!a@iL?loYP(GOI1A9zy~=Xgr)LPw zb0?owIi`wAaq6)(Lpmk7fzQvEpda5sNGTC_=qSnv{D9CZ^cLe@45-h`*f>^-r4eRx zY8o>3f6~&VwE{|~OW0)xBg~==9mW{SvqO|PA$+%9%58!*N#~==YHPvH8FUH4Zc4D# ziw{3qYuI95APVQX9~P76&O^r>7Le`eIjlh7#_$Z(R^4r)+LquJijwG|N$pELBwJ<2 zAIf4|6bee6bUG&K1p%4M3W@f`LGbcFk!{EU)bDiH=fDzB;*V%NMY@d&>^vv^Q?4zYEJUI+Q21A@P*f0y3tK zM=V6IJvC*{STKj;exqZIA|5*|aunEy^LZD?%ATxr;YdgRT7D^EzNPRf1sA4xmIcAXe>tza;t={LmE}w85h=mL<)9d3NcR)BgaJk{v9FxX9@D#CHX|2t;`(P#DbU{VQ#RIM;7QV z4#bX$gUE02xhEDbXbg8*CYO}qN&Q4$K`U@IzuK47II@ph@j2HNc42Fb>KKVT>8j@L z3Rp2QI%YfdBCn2Lx+dok*v5M?fs9UsPC*6s5}`=3r|0w`v1Mc~$kEq{gf$Rx)R9}5 zjm(b8lMl$}Wo8dbEro)*!3CZp3SFfc!%$9{vhsXz z0?W`9#&CCO^dAbQ-F(4H9Wt^mTLbYdg{XHYj}JT*`>`!)4M9NPe9bibSAoXv|MUlQ z$WKsC?m&=?B#ZX&b`|~Vbs(za1F!%=gZeWr#H3C{{UNZR(MV`pX2Y7V*JYCz0eiP%lL446s7 zReq$^?0L*x@97T|^T*Qf)3IMO7H1E&HY`zHc)0aJ4-fb(%QCQI@PzQ0?$WbbfDtV2R^5Jj>|5Z72xGnv zaRK8z+UW^mYe=}+HX2#DAo6AK%E$EAu4P*{fq%ES^Apk9>KV`I*3XQk(3N#N&Sn)^ zJHz4245H-%qRxygH(-^-H9Lhx0ZYqUF2x87yqqSltom(4R83y?7Qvf!?bt8YYTSEA{Gc8P<6 z`+P7e!}O)m`wdpEGfOLI4B)^qQmLS0cY&#frz`G}{-!ED;@GfhiVa48~->>Y0#tom~T*^&!P5ZNYmA!wh+ zf7gAHc)(@7xy^M{KyOUVLY}J6m>;C-#T5S9L+b2`EIb5sAE!HwrK+ZaGOx#QI`fLp z3(0p&!A&CaJl8Q@*33uvaBD63Rmka4blBFz6z)ro+YUiw0fD8CgsaV&AU@uUUV~ef z??77G#+AO1?2T?bIvLfmErhNqZ1LJTjmW@Z;d#*R2(-)kAW0ELmbjMZH6697B|z4% zQSb3ONZJ54gk0e)6L5OyxyCBA^SCyyTC;zfIV{n-BvV?Z1(R&0Ng6BD{32e?;X0hkGyUVHRga->aOq!Fs$l>!- z;vTOse1`OhTY(v*9+d16HdF5S4cMy{{KfP{q5jiK>9-TSPCt@pVLvvk5GRJr1)1s^ zbi*39X0!c4q-MlYlN1@HSf7|b{0Y53oVc0+(X#5niBI}4(gr1d;r1v~6j%jJ;Y-L2 z%Rkwr|Gj4a?_2qA)vU26RtUtuoN!41q`VoBkPQ|CkoUirFI7`VjX(>(Y> zvkaXUy+9J`D0FoJ{nX@iz~;P>r_$r!_p2-57u-QlL;YS~(>9i{$D!*$6dbmrk#OlJ zR$!2`P*OVT0K^S1T^Jon$xY&loJ{&nK3=2Yq{&u%d_5`Ih1i#9@H^%HqJ8T;NtOY#&Z4!|Qf zhvcBylR&yg3`zHj4x1aD`K2Wf?6xGGs%e;(VN|zY1G51Q*hzUMpv8L+mvGENcPJB1 zk32=*ctuXdJ{e$4v_esufvHNdg97)3m-S*k(;5j-+}YCUsW?bQ6BS^Z)BB@6TX zv6Ws7JU&FYk{H6Cq{ML83N!fuo=^upQITQnT-lYh`d};fNM!iAS_{8mN4yT(5k*Nv zf45G;X7(oR;9`bhA3q28;7ZKR+_iDFObgcvUd&7%@F}N1zawP7;dKzjm;tYK$MKaS zFwJMrYNQlx43>)>A|mQ#G!RqI-0Udg3)F4iqbgh`;8C`Samy4Z)y7raI>LA=FG>)( zpXQ$j=<%b(&&mp8ZO6Vw&UdSs9QtiC+=oGSJCBH}foK?HuRe+JQf|MM;t$p7hf@%p z9j-@3&h31F3d-!3+H9^((Sgdd0)APQQ0qk@F+?ciSvGka1#CLQzgD~Bs>+6y5O53t zH~5jJuL<}hTu&M0>bCSb@{?0>r6+sY`bH=>6eDUiC5eMm^xGO?BZ&)hTn4nbVYH(Z z^T<|<*CXpBIORV4szxv@7Mj$q>e1MCO_4C*1?7u}m1!5@cp;?zc0dEj0jf4VJqgek z>k>(o>~bwkD_gSrhh^ap)c%lwaLq=i{KTU;?QLf5Dj-hH%7X#tAalFtZrnxT4UW&5 zC`zmXCBzKW{jZLkKhp4M;(>dz)}?6-vV7eXn^p0YA(zMX+dD4^Q>M1O!pR5|P3A0Q zu~6MYe`6ecY*9ZTynwjKJMMwG8!WrNfn(^ue!@J{gvkEjK&+kZLPxkPqXcHQ7-yl6ATJYG%B?bQBnINW@LDr-kIyLZF z4jb;Z6W@HV-A!^jBgr06o)ZsSuQoG3$VozgJE!McBRG4-IDVWBaN#ts4{QCVB7v0? z&>HbGe0OgPR&q>Mxo9JWt=Q-XKM&~GM%14Cn&cms_^-G7B$8qL|IyL>^P#4#Zte3T z)Pnl=18rIM)9d*&vkn6Z$|qtY0Ff~9;(&y!craTv=8-Y}5838R2w@}GzdjTYqQCtW zB;t>y3S>0?QH>!xv5ZY9YbJh~mS8`-(8BzQdiev4)&DurJf62Lpz7Y(F&+qRP3jxYkmyRbWPal_I} z2V7Bnu=s+XF>wS<{>1p3HI6CV+Z?!Zk!4j z9s*6~4IXTctQCA}e(r^d-sFr0D<1g7)7GWW)0m9%HsJVlkuSWgTJ#Ky5qoWj3wotv^ zH10N*DLrMUvg-r??AUNujtJn?DkhCc(NYQ+KSq zQS5K90)H+6&9806Yx(wsmz8K!=n6=`(FzJ7d{WA==e00xG8@LoBfe$^icC1wHpmzq9Rp5g6pWfFHft)R>YL^ArViAFiM_oY(*iMZ z1XEPYnysEZ)%W^>W(_+>-|a`6tJmw_S5uRUX%;*>f`Sq*O|;1%-~OC{FF|2PO=wJs02z~INMXnKweeIh zGs<*nYdA1RD=jG{DWI8D*7u(mQ9BxanbOMJ8uRvp11~jvfp`ZAtU(?10=P1^;5{!} zxoLr6FPBS_V4k#mMWK)t>-}^11{^WnTofYyv;{+UlR24arz#YwHZdVwo7O4O=BEnO zO>M-4R6>q1QTjx&C@Si#HZkJ+yLLfoB?$BPHiU4)oy`o%nLy%SqsE$?Hi(kRFUrx_ zgunKh3SD%u&I^kKX}aX%hnuI?oLKSL_zImV<|@{uo5eB0+*O6|5xlIs5{tfb&BOox zh0YdK?4E_0!og9MS5agorj0^C0EMlk-I8de;K|vf$8j1Swx>(3qSbEDOXXaUXm4U* zrxb^5dy~6kEDlU2jtUD(yy)}N?pcp!HlxfY`ko&19iIkUx5|RPB=c*FG|cb$RJ4_B zwYjdFG;BHwI@nA@z(By5vzmiosl?3*L(;$~!=97mQTT^YfxPhsdvuKnQaUApJTgjh zg-LBb7iIX#qUoKy7;ApDP`vlZcchCg{Fn@|A=%O$EqGu+^$riaz`hCvev~BSk%qb@ z1FGOyuCE}8T!wm=I& zyl0r1Zpi9DFmvVfQ#N|K^0gj5(XwQrmW@KkAp2#PjfSFQ_x3<}SBW(1pam*)txXMpEKMdL>I&8`MUznTN!A z#VTiMD!%`|a~4OvPgV_JtTdFd)X`&#)aZg1bNPfG2JG-ct(@3olC~?YrNW~Wzp2zK zARGMFoC`L@TCW!)#!D2_Ha;n8xfpEYz^CSPrkOk_7lLIRmCpz>=M_CJf1>zH87@@$ zgn*k2)VSe!Gw2gif5Z4ty>`U<=mTbq#6ihW5SEuKh4k;LyLqMe?<%)P;ZPn%C(X;L zUIHiR#Ux>~vyM}Hd6U1hio;Ne1s(!mIU?@MrhRU@=;6ilqwH+LoXS7@UMiGfi?7G? zrJo?N50SkkFe^o|IMmtHCJVNj#U&=OwLdZF^UkjnZCGqlSOIKWXm{!=+E# z^$9mMq8POd9dY0HN(o^KlZVS<4#{Bs6u5A#VGQXRPHx7r2O}h6hY6e%8*$Kx#aTu|s2`t;Lp<{`t-Fyyd&4mio~= ze~OZ)>FZRM=v2NsIom-J$h^mBlZu3?}UdK6!X+?t1I_Nj&t_74@nOm()Jfkw!y_X$3b;B~PkB<};V&OC?Z)+W1Tz zj@O=RZCcfQiO5k-N$I$EEwK67k7iBX{p65-WUd-tNIRoY)=bLRSIumOj_5pv^OmB@ z#?DGpq`c!a1Qete)?wd+7jTdd={YP+Hg3ky^*YMC8(vlj6D2*}$G& zoKCPH!!Y>a`8kqWcG(*meg=4Pe%e;$DE#;P+}z@t(f0h>oMOhLQDE-LoTuRLY2kx! zOzzJB5;TUpj!@}NHa#{#a_zLcciUalHlt=99u$$?=WrhVnmAV%Dwd@J!U?cZqZQ^M zY}lro>MvGz{&s7x>`xYA)9pro4vt*2-p#EX1Xr`Ol>Qu3BV}>itvUj0X2@9r;GH(c zS`}yykbgjGMw+3bWWan<2a{es6qUV$+W!HJ*k>BgYu~HfrfXr~jFXfT*m1 zkIyGfUWZmSFn3{Sod#@laq|G?%Pa%OJXel7^GXf;pusES@>yBIw6@vW;`=dBrOE*? zwS-aN8dAFmjDV;;nK9tRD4O+mf=V`mSlT{I(Y27q*%+)g$W&`KE_iKp!dUP)NiaPM zKym_hX2(T?`x%v$jE;C#C0u@{%{?~#Y#MdJD4NPa1TDM2j`4I64-*IeKc^7(h`PauEqzSgPOH* zM(7OJGOtRpdn(Zp-G@q9K}o)M=FYWpn9b4Sf>*WzfH(5S#>g8N+MKJZwCkL*M1Rm( zHY0duG|5psjypbFKgbt=D0eceI>S4?rcyD=rhJ;T#pJwx$;3J3_(QD^wsm7Y+&kvB zIOfuVvpDK|6%`Qm3{HeaWJSiTwse0>#Qbp|zKf*O8HSwtn?c$b@;4Zw6=c3TA~)l& z)&ta5?35dRcGM%AAr3U$NyiRhFSL(1`8yW$2P!`pBIqMBfLcj6h%WIICa-O4@>IHq zLa!0~dhzX;99Zhr_Ie^<17!9sGFDK;l5=X;FQ84BRv>T)4Td6GfjxGLF>9(UV-(6( zpm|TSbf9tg=!9B{N-e+JCVL}JbiD!Hmm_jJA#_FJux773r=*z~W2HfZ#tC`dS5Ona ziKBIN$`$;$qt1-|<^5>&%w|xJ6~l4JbR!|n)*|lfgcH{v<&yrl?Pxn`f#i=GuFYp3 zH3*R@eJ@~P;mD9a9jhJQP+TE7$G!t)t!IhG@V&0(nC$UuiM}5L$MYr4w&(HGBVX6* z9s_w%GMOJQYh3TXc60M7Z^c?h=;E1yN9LIPeeIF*9=;S(a+Rh}+0&OJP@APEv8Z#T zU0iw_f-DzA*@D4YMsH>hTNTkjhw3A);ZVP;VKq>-BD?n)+)SIL@ow>JRT+bG8cPDg zpRg;srwgQK?l^>u$>hNHIOi{2@jl!AZf#hu&IA~{9zKVPy8b+7yTogLEcHDoJo^EJ z{SY=|J}V+eb+Iv^6F&ip{O=nMCY;h&(^Q!}JNAPWnJRxgYTBQHFwwo(k_U&3q&&ak zH5-7Mz3=R`unanms@KdVp$X-Iq8Vf?rpX!Iud<3aeF3s{esv2y2!AvL0@on4rLxW< zdLg;*q26#}%_y$I6=*7nrR;rwyg)Kz^G5LjDnX6Vs$35GE?-!OrNgRKx8@?UZH zo?;FW@=fG3crbi~c>#l*`;72gbE~ls;UH)K5zkcUouU3))qjQ|f=2fHj4t*BnD3T8 z%K&u1K)b@Y58H?oYt~le8bLq5qYdOTj$;TWkF>gH`q4 zVr#t;n+7!8KmW-o6P1QbtpZ~Q3AgaVfM%O{Q9;BkJTR$mV4tH2kWnC8ec@rz!T;Ss zL|Qbf)+g>@3I4wwILWcWfO56yVL(3zp`lw*i71`H{(W%k={VtUsQ<(<5=uTOq<P4_=#{-fa=N^NID5$N9n^@M6|0x>afvJKc>wPLhE5vY0dNt+pg!)8PM!-`P>fAd&xB^`cHN%lX`20|Y2viIND!$`7D!o#Ul! z1^f4M#?q|gz~?O+(EcfH|L>Mc0F)1i|Biz6jj#k9^8KGvi35N||DRPN0kBE%{}e8S zz|6ss|2eileK4~?v@c&^sX->3utXqXBQ#7myJBd}937g5W3}CBPPessQ**tedP#FZqsg|Z00CUY9P3-B zuDxgT0sZYw0euP}PuqeHF$Sl=7|&xWs`XA*dSRW6=N^}D{~ zXk7W)UcR2*1@ph53aRFiR89%8U8}M9J!PytxN^PEc>nJGS=MY+_Wn`f^HI|Brt1IK z;Nmqk^R*fO?{F^=_o*7k^(GJcF9XU)ENJ2F14t{&a=h#LyH)+lzS&6?X`Ai324}*lKUxjEHlmF;rHatGl3VZQ znvf?-${+DzY%E#-=2miyWAMHJiw1G*(X?5!1`}7+S~`)GC*e{_D!ANB)VO8XbXS}d zA1F^fAS1Gynf3r``G)jOyD`7IywdCwj?tquBta-u?L2FRG833{DiFlHf(t**@+|E) zxZ&D0XJ5fFMAJ-;1Rx(42bBlLaoK(4)9aL+yYukiaxQ7^Zk8jR3VIXfPm_l-E0687 z=F6547gjd8fah#(Zf-H^;Hs-=wjWx;0x|{0VKLc;!&>To)#PffwYJmw>Fz&~85hgM z&{AV>W7LOXn;Z=aWegy{N&n#|o^LM*U? zDc@3Bs3PLwNSP=PcU$#5tqvk_Crp!7wzmNq_vlJdIpkCGl?_zoq;QLz#8yG2} zIsv8A14RIJ`Msy0jtL#1we+=fAhO2TGlgTw{zjR#gDpLxtX|+m(LOpsa&uKXqydY1 zpnO_|yxm`K;Q}i0;I}t>v&0yQq6KNu+@m1ckDAd^TT!iQIOG*_d5R0H{pN!|%@vz! z!TIWZxEjf}odtOd`pxu~W*a03lEAFy3rG$>1Vyu1@T|p$-M>$9PVqme#bQu#%yzB) zdu_FJ^dbk;y|iTSayIQZbCT=4$q4cnnEUY5cEQ+(oWw9&&f!C0Fkw@}V=d45gGQ-~ z2C?cT-C`Z8woYB5>=-AGAB|sw25g| zv=eno-CD|n(f8f zJRDx-gtERd82?;W{N<$|?$APKmuOI?zA*y_y5Ocrw?e>-u0?wTs&KA%MGn;Cs7#wl zL{estDN(!U!t2*V@--GtAH&&x_K~ld2WPNQK)2L>SKG*zU_hN#8pr^vk9O2*PxADT z9CdM0FHzIL$CqB-tH4R=QL8Yhwo#ev5Fk3z-n^@gqK1TJNo0*saj%29#U(rqjwj=j z?8ttUS)k%mF3XtQu~BXa?e)ZOcvmg)do(&Rn(t?_f!@sDAA~DnW-8INl_oUen6;JVUZT8w6EKDFSB4a z{td7!4vTpKmti)ppJBC!#&0|Bm$3hena6W2cvEiDy5>o63FNW>!;L!5GeIucS4s#> zu@30#;y`c=TS!-Ba4%t0&+`GyBuJEGv*6%1tfU}}<#H+;>J&7&l01aK+$0#!@jl87 zi)n?bJp{%$EwwY{oJL;7oQzuSf%!@11gxZ>&><1$^U*#Ks|m`xOBencG~t)6j+0)@ zke$WKoKK{M0$P!CT9h+#9FVJ}O$?RjlPd&;g{I{_VXHri`&5|Ss5e)LR|MB@GJ>1= zFK(WwH#dhSFTP{@U}rLxK~VoyHSm?Lzh%?68--WXP3hAbFRmh4@fX7xm#tYN8U3({-oAlPkHC% zn;nKsUDb|mw61-2Ld!eRZXh+e``H_80&$EhExO*HU%51#jK&a{`l3rix>NyiSXR;c zt%|@Q0%>qxB~@lBjmxf++tGfj&cy=3NWDEWk<;Mht34to`mQx^8T&B%Hss-c#&XO2 zNQxcB12A~o6WjdObT_B^F7>8@Hu{G zD>5`CJDQliOlYBED(K|&Dv=-*xBYsHyDEqo{~(#(WtA5uE__9n53HEKn$@+DE#1>hy;A%=oDBO2Z92}t z@tsTckA<3;$ct}{YT5gbiTsooPXyvyi&GIL=9~=yrqafZMpitP-bJak;P)=u8b`u% zsC|JnLKqlm?FH5)1x%u0h1Fd{GIp%i5^cLzYD$B5L)OEK%|8q)og8Q~NFy;r$>eXG ztU&jfAcBi8w<wj@#96W%`;V@QEb&jJ4rSCp__jg`N*F9tb-O z=Kn-LRS~`oBeTPOz*dU08<4U`HVq_FsR5>A)7?{)0F9%}dV4GZbZsj>Zyd-JRv`si zGsNQD9OVVdwigX_Wx42aC<>_#_PP%~hz(w_fLB6QVu5FKXo^*23Yntw0WDHFBev=z zodXiF%Ole(v;(5YWo3sU&9o8?d9*;s@;phE?^bMB+tbF5nFCv#_7Ebvc`4u?oKAvF7k3isMspmL1C}3V=?&Y7TYKlr2j@IRck75g65IlMh8Sw< zHv?x_yF9CwjThuY-yRJQUK%3{A%IbTGuNzUe}e|PQda0|sMaT}wnA5%Il@_+=caC& z6u1sA3u&EA)TUHY&@Y2W=kESylF5^>P04P-b;-{KNIksePW}x%dZsAZS1KEGvN7Bq4+W0a%`9y*`s6;`F~RJP!g9 zvdy^?#nZ;O$BM;AvOFoDz4^d42Urd!+DXkR)Cvcw4DMvEA}Dc;id#p)5Yxw0y(+31 zfi5vef_?y{>gLX$#tiq)fwA+<4)n);UQnYew6j8C=Z1gRoIj!~52!QV`0vf|Im(gY z4CJHFx$i_{JPLVj$x_e~7(I79wLnyRhkul;=Ea@*vx=WqEB54NzaNl(9YO3if#x|~ z^f;F)^jZcDTZWB!Ke=9x9`z{Lu~VI#jKi7B$Ttt9^OQ6>T+NQ-Im*vJ>A7y^SA}#F z>Y*;{BrkiDB0mC0H*m-Qfsp%wHS=@uu2wp=%_8mK7A<`Sw5^^&b3E=RIPT|X&Xgc= z&Xk6)L+y?)459!{qXK)axcsh+ce(?w-S=@V+R*aMt{@@KW1sjp*cS~Slrz1bz#UXs zLD6A-Be_t5i?0NJb79Xj%6{-1iS|2k&<@;A(Sa-&cc_Zls%RePLD3kh`hT!b859#+ zRnFLN*y@pUOr4yqrYkpFYgn&S7omGzkf-`2F}Nfz_Re8<;ecs`dpgtj3kMOP!?M<* zi-ve*Ci?s3Wqzb>V$2si#a=BLMzk=U2`kO5=##&h8B+DVPzWl^R%1KmAY^M(#i$g7 zh^(P>dZeJ zWq{NWhQ_{7_)V9DB*-N)RVFZ0(f`bPaUYb~ z)TW%PqZF`5+~zjGU#r=jyRfQ07|b_Vk@W}TGv0FN!WCfYp@xt7$mK9$T=p+M^wv-s zeYE6|3xnK>PX3@jh`X)9?T~o0JHdcKCp~sm@g6x&i7;2DoS?{o_H@{${Mh;wFft)9 zf`Y1N$l1uNOto&i*#T$_1c)~4xoJM0u6Yv(H~>F;psBPjpr`NoPP_43%eKHI*CD$a zvH<(G{rmXpW6kLlBs6Qa1KziAjEPDn<@)iK&y1SmNY)nXeIDH#5lBgc)G{W~Mb7Q# z!nNo9+`podmlS`!;M({nH93%0XGx!F{#p~$^_AHd{Do`&3cLPZdotnS|Kp7-_->vE+a5#bWL7&}Ek1the1nElN8`fIHdUa=r5@ek$r@7JwV#5TYb zp31917#~(B$MQC51{!Ml^56hvt3l;Vwe#=B7(J6zn&t|Ayi2ZGkH7fXl@&thZgLu% zbUYZ#t;#1xCH+5xo?Y~yTm0jc1*K?7B-(+l>ggIBp6k|DU#iB(b{629UwM5X<8|vm zRsdC&0)O*TTNH=#ud?d}ebqnw56eq@t+?qy?K_k+Y?%Vq`=;-Ks%?NJAW&H6zG?q?h1%XkB! zl_X%;?6a>3XPpl#V_IC@eOtp?f0@QmfgcFk-UML6&x{9G;gOiSt)mFpt?&V)pHy%9 z*AHbxAL&FNZPG-xd$Dd6^yy~E$Y&^+w)-=XOq&b6KM%!d5Eyr%o0H<`-VOYnABMRXwY@FQ_?y1tPF2x zF^%@QH@|AOQ%>tYMKaji8hFIoW?_I~Q_||h)8k*lp1H;z?tplK`fQ9HJWB`OpAm-e zTn~YHhr(TJq~GFS-11c%B2A|9*krZZWG&ZayMj$u?`_~uSEnZHj!Z<(;Vs;v8mvqu zE~>Q|9RGIBDV%YS6I_m+Kkh*ESCmp*rBYp}if;5fJS6M})_;`pwQ)gDP_G|al- z)m}hQ^iK1XBL?jA|NZUqIv{YgEymy7WFPK0FA-~^B{ZCsjAwd0 zEb|?-!SA?W70hZ3p4!6G>kr|bYwp2m?#&83eD+an zxp!E))cd^nPFo}f)8w$5Ct^d=0>U(Xo-AN?`N9^@n~4VTi^;~Mo~F!^)^UB6Y|`T9 zxhp6qL}6Hr&c-ar)qnuLoci~F>tX+g-uz#b?h~Z@2iJk0L5g$zM2M6?#+kmqN@hUnrCFq47aiKLJRsp2_qRP>_ZHU{hK`uxen) z|A-E*I(qD%XxBI_hz$#y5~M81hXi^HWJ3UkFq0Lha4M{!HiIM3sn{`*4dQ366j9D4 zmCWH;ZS-b->(rrT)Exo#JHH%s{-HwQGyAIx{}kGx}l4(Wuf19b=Hl|a-7>*A-qBbJL2y|LpDwG8p}MXpr27u{%cr;-2& zp9X;|u@sLb2pkR13aEOiKN?}`VYoFh=z|V6KZAi5Yx7e2_#`*z=myaN+~>C@*kV5U z$rV_V1NAEjjM7o?zzyp9SdTimh~~#Jr@{tuWRG&IzfSQ14x!_4qAa=02;%PCEsa_k z(o|+rDzOT)y}gVrJdmqm6@PD#OC$kt@fsqNSG(bpHmC!jI=+*sbK08~de_-4 zwB8%YvDT!y+v(M2`E5f}S#4A5-h7O@uCgN|W;MoACJm$@cK6m$P9+O% z64?r4x63o zuh|BQwN!Qb?6K&Iuu{~02(DOTQ0RLWIIE*nN<-$^wsN1_qL?;zA)19}p+Xa$c~xXa zHY|^_wgx0~eK$0mvEWWGM96@l;+Qi(t$3m29F?i2?^BJww+qh1`4zs0(&RkoPSr6W zHi~7wnkzNqsa}_h?{3Fgg+B}nt;uoRvjL>G3E@l($#p}$QwRAMbISva(@UCR*q+8b zpli`?`N+0=<$R5GeZflvf{&AFu8Bo{sY+ejoKTFeouj4Xh29Zx=7t01_YnaK6yhf2 zn3h3`%zNnXV3`ESj^xM@SW3~6SW1%=n*3ZW`54ko7-O@O)qZO%*Y6A6irB`YA({rk z9@dh(VStI?>683vHL;vjk>mFKX`BSgN)aoY+2?Uot5!sOlY7byMC}+9wtkFWb^-ul?40TS%PGQj{*3RwEWb0wfSW)~N8BV5LuwC4S; z_3{}?!+}9lF&x`MX#L5UzgEl->JViTvD<*HJ}ew9XlGT@A~=H}#LKY*xfYc@nNQ}9 zcCie{ef-NkZV$-hGGAa;WFD_!UuVi@_N-KZND6<7VDXgUwM%q!c>TrS3&oE#aQzGJ zVll1j79HO(6(5D9{A+E5QiJ}|kv*o60V{psOK>zC<&It*oCMv77FNNP?7RogW8K!q zZ4AxjztI1bApOso{J(DNUyoxzh%FC>4EyPEl*1ER0zYe%UI;*7`s`$&od`-~5X&$u z42YGR2fEcu5jG3cjitky74!Vtj} zXsoVEk)T(Pj#6&2rt42DAP0xXMs50m7FZOCvVI|q1G3-TOnOo}uu2yeC-GbU9gP1D z82-90Asr%YOLDY7y#aX4&S>Yz0|K95Nn=+fY1^nP&I)8hW@EM_#S;+oQFtm-O#Ii}iW6f<6#Eu_X;j1?F`Dt85yCoIR! z4I~^*+7&B&d2Atb?wo&mJ6g&UlWgR81lC?iJGW^O3eJMdUCeu4)L|y`i>>*HUiwjj;4>AEcXUrFW{?0Z= zS!kpRRTyc9t&qWS*kF~kO)Vpi9CC{(?X55*W6-EifiMT8s|m!U1oih-J5V!gTt_0~ ziz51?GhHp49<7@tGqO@p#W_5(QN-5sl~nkh$=?Ln#6%~7E5L{`qQL?=Zo{vdir*zr z%5)q4FvGFTLY#%;$-_+_b_nXycr~IHIu_y~TA{**ODIAtsOtJMQSrTVpr?B1lh#Hr zWKn&Uwx?Weg-APJ2~0WP79o8`?rY6`q^ckoCHO_-ru@+rj@oD825zXd^9v@JIm$@F zUrewGeD#1ESj5z%GSK2|i-UAU#!>awtmK$K!2{lDtq=H(drMS4tDkI)alPUuO3iKb z300t__K-3AOUb?j!ky3%%l+g}ljZ7}X~>dARpk_c)#BICLVFx0|4kpBenWMg@!bX-as`O5gpvlLCWy0@?XW(RLX630XM6#3gsL?pt`5?*w;R%IqE8TZUL>xx^R#Q- zbdUE~?{HtgsNY=kxM&F~`Z#{4=757km0x?i_5JzicX{TY?3nX(%J8LR!URF6@p@Zl zw0{QT8!#Zh+sO8g!TzsQmXY3*k-@9Nmsl?)tAMi+1F$#bpC~&Y*8yLEy&rl{W?u$w zXb?!u^Pxt-aRc`|9tXZ<3)2t;QlSg?n48O$cro5UE7J-Z)HmEjJVA^mFkV14jV)Dk z-;ASPAWfy2BqcKqP%S5+fnn|{@d>E%99Y!z$O5zG)?((Ujj~+bfPt3`3;R%OF^jwH z9X`1=acW0vC$-b4i=N-1|X&Ct>SpuVf2NVjoFIyoYX{ptwx&baB5UTvv+_S z4vMs%Ow#RxZR}$IwUB1GlV6cp(NXc4f(J6BX=<<-8t60z4E1ZA*;OuS!WYRZ*R{Ef z^)pnS@%lf!taXICr=DposV~!bB9=!;?nN3?I&0LGFOC(}b%6u*(U?5f7|SutYojFe zvb76hF@jbXcdfk#WhmFi$0Ws)F&I!J2(p#E&B*82yZy>rLhzhLBb_+e^Ve4aDL|>N z=wPfq?7kw-swMHZ$nTk%8?J{MiG$-vS%4i$;>ijm$0$gQYBnj=!0w4lk zQAA?A(JjswLXKpgkAv%wF2~rO6t=PUOIW7!lR3RSR!=l!_0n8*7t89$>gI;t6dkJM z&Q=?OeV=e(qG$#)aq?(v$>~0A(4p0H>!p=GxZu$gno4Zh7m)OiPtx;fM0UzaTtSX{$D)t~q zMZmtax_BZ0U$MaW$JXXo_t`}z9Wh78-)qB+E?g&u0||Sq9Fr^+FSJRX@iTNKKc7#s zhb7hVWy)Nkzo@bDtt}{H8A(rUjo&Zf%P>X`>oY;qw}xk9@872_%mcxqV`w%2q}0$< zPFNh#J`~P{SBW+z%MuI=L1mu_gw={@>ZJ1gJ}CL}G4|NuntVJXTzuyi8hTX>_4MMv zi?F|tkwu<3D5{))hLRI4=%@swS>JDpUbS72h&Tv&s?b3lWx6Rm2a4K9l@3Rq@6%rT za=$7YD5|Gyb}Oiqp@0Wt^dtUt*AR#!m= zY0&Q z$!M$H4s!IR7cft;FM4@k2&nk?uNLVzx&Tv#A{lH%xGzOFh6|rEW9?T=_CE(Y?JJ%) z)MN0#GZUZ|a$gR;uXWj(@uV}KF-Vb3uf=QvDk_%`| zT1wXz!K9M^E4 zn4uth{9F5W1&7uf>>jfTqKccJrBR29UElU?&KoUo9GZX@Jb@DiF0D+DqunhVG$f5T z3Jz)U6Exrau)^)1i>ewA)nsM#AH07up^OH%@Kffs9CF;&F~wUqe%5O8T-IB#7-GlB z5S;YatgBiumau>q&da&2JDJZHXqKop@0kcxqKyAh@s`gT(LpidDW-b3RW3@-vmi3y zcereGJ^=@Y$rwc0dyD3(1N;@~-n(LtQ=(ZC9KB_GtS9D+)c+)^cuV#a{rcO$G?S#a z$}*D?<-TZKD!WK1@zJGvk0tR2&d2CImj*Jg+}Ut)ixS}(B27n^)x>V{>vqz0=}ilE z8BOzuB4YdfPR1mY9Gtxl zR&4k~O*4DTQlx~p(V_e4!L_mKanZa*bkssb&#q^t9{MGOp|i@z+1Q@itbX~d&jEG; zY?c&o47JmuTRHovXiK=l*7gj0NnIV@7N$`9#Aa9AN%3*~=Xz9o^$Ynlx7n0yG_?A5 z>Yp>O)ArKmS}ibq?#6w^f@tHu2dl^a#o)!$WC$j?Nkg?@6(ah@zA$KacbdIHMw7n% zD||T1JSr3e8l)YQn_L##;M?ydhu_RtdnmDi-#p|26YH+Vdc40-H;)99U1Uk9qbern zmor#%RYWE0F|41JW282S2+SpjEFMab8pgUK0BO zNSAY-?!6*EZdEsHD<4f|B!73GQ-(4r=3|Z@yEO*WQ$uOjQVUCsUw(JwG>BX6{p%)E zY6$G7deSHweU@8Ja?||845s9(zX{RY9#}--F3qq0T=1*zu5-7Ezv2ryK)zM;q;#d--WMOurf)j+J^CdXCNk%OJ+-IY z&EK42s%$gJiMH@hsHPIX@p#&e=}Y9&HRgP(=^_f8Y?iYScVIzV#&6=0~{P+ zDehU`~Hdyrn21Lx@EULF>is z)(7K-feV1DKDRPvaCyiGdDr(R*?m*6{A(HK*jHVCV^6pp_jR)8YyHIJ1^2bfLm$*A zUg4uzoYd`PcU%0$2ybS*{3T>n5^(G7*ek`1)2j0O&8Hqo@co zt>X*3gobHuI(1M;TSfCyFXiL_osuIzFwKu>wk~{IBgRcAYeAv27E+(`LHTW1!uoRz zl1eCnvY3LtLPm8dwTJSH=ROui5IU$m1@=-VO4yfvA2FA<55g;fmt^nU$=nUyt0YuO z?+Nh{%jIrhB@6RiOlA&;0SG*XB+(Kv6{5-Y`;Q})4fi1Uw2s#&OAppc4yxn`o{0{BUYJg8~q5D0WL_&OJ>N`c)E9QLQOXKN5?d5Xa0ruAbqV^*q3^ zSycTQC-8tmF_G=@JSR&zex2N#$4xz64Ze8p2~PKcoX7p1+iPj?ZRB1cy=0}UgMy}8 zOd_B=zxVrZaubqi9$-_B4^RG)cSm9+KXIP~_~8d#O8Y)u&~n1eMH={VPsaGqI7_0w zAK-n@(Lx#O*nSVuF+tpb7%is=DQHoWZtDzB`#Hx|yFD>FdAciT5q zyK9P&YqO>uA)7%VvKY_u`Mi0;0Fl-wtm=#9C#$!x&aDCGH=yRXH$&O$p2=h*C9Xe8>V=$ds5b;Id^$9G4zD)E9jRoGU5!1eptm^vQpD_S5$ZtL==ci zY}!YNGlLYm`(#MCI#7wmW`5g61SX!0#6Q`^d=ybBxyan|oyc`hM^0_`%(Tn75E>%9 z3JFgan?C@$gt-?N*T)ik&e^%FGOrB1I38(U@8D(OWt+DHx@b6zW{8Ln;Kju!z@|=5 zVw!NM{mRHliTkraX>ejrw$Cf#|2ekIiT%EO?U3AzaWQc!)vU;Qe2Dm#9oQ0o0UnEV zq5Xu8eIdcR?L)L&(WzyPTJH6mZGmuT@1?M;>I&GKv@x+>zW^ERp?=6LhfKd#&)=#G z({sttb1)8%yrwcAm|DyJ|B>~L;hA;Iwy|xeW2a*$-LY-kHaf{0+qP{R9ox2TJGuGx zKKJZ%&wZZtW3Kgg)||C!)Tl9TJAlje2#r$)cg!@qXn02Fq%`YX!9@)g^eej3QRpY0 zoO4`~boK7~RQmOkLz3~2>zAIDOlUDb&9h2~s zwYc#r?c>Jl9=rL;`H4RSNGOk>S7jE)|ErWQ*$u;2IA%ztO(1!4tEQ05*AboXmI~8= z4fZt%mZlXns4BIsAidP%3c}4>6|6e5#hoe;xdqU4OH>>|kNYJ$g0iYs*{;}vJ;pf`69KgwKkC$RMdhu`u>$;Unr^$-*Uam zF$+-55VOmNiOS&vrIFAzl~x%gHY9YAisSQZq+&@uNt0b9(p1=ktE6=!;M43^!Ps08JCAt0C9?IibqnBY)UnxA4jS-4}F zD=oUJm3@S3D$M^CR5~ot~<*)!~2n#^0nlPKm^wdLtA@Tes8>bE{{TgJ$gd3k{ zETv##p=05JZQdam>RP(%1-m6a_&58e9N1j@&^WN-s|wv@zHKB>)Go8kLOW6g5svmO zdixd5Y-s*>p-nU&2Ono~whLH$WXpOKysOD@*QDue;((jtR^tq|t6UQ|Bfm{QnF9bd zX?Q}d@8xX2NeKlz61;-5%d%|IQq8%=K|6WZDRbn@;I`4apnG!PL|jAs(B5d6ao+2< zb!&1GVDzRQ!kwGy&cQJqs3GG%+vqLDa@elNL;Vzq4DpNRvog9*5(y5SdFZ26@^Q`P`(xu`w;|Q z=*zh26HQaf4PiQ6?^$Q)ex-jocFLByN9Z}yVZHYX{vD(DVH3YQ zTldCzW=v)nzN_NbzW}WNMyVM7Cx8X{jbCXql!X+2^TnTllCKA-Fq41O0plibl3~pJ z3;&&J&vbD5=1Px%{6mH&O3vYC#{lpfT8cR6TO0phd3I@85Jn^~=NVxk`FD}X8-u?> zmMz#3p*>SAJ$-pkbt)xERV#GlpESJYJkOx-@`J$8IC*$HPMhOSo9Rt+Pti;4K*ALT z3=t=!CJHG78R71ju=E(%A})#YF3~+BnDp2v2iFozMm)wwg$orUB>-xRG$!?*Z~3 znMuZUyDhW&l}DYQpM|=c2y0@~)8_s6pm-YhcxaJvL(tee*MrQhxjZmAs0^MDgMk~b^Ww^9VzdA%p^HgGXA&n{(pAofA4DZ z3Nbz`FxWpE+$_~aZ2$uH5A5F@Mu|ZNj_}Xd`8FGa$nXAS9%AxT49LGa0AR`0o`}Bz z)~cwYXx`+|w4`IbAYdNhlVR$o9$1F5G;A zt*6P*-j98Wnxq&j#Gd_fnTx-Z;9L!z;!^8?9er-9y#Lx`zL^eVT?2;l1g&bm`i`u%L#sT0 zdkJ=$SL%+q+m&7Wz41qNO2;8=w6b6`R<1HeJB>gJ+fG> z()P-TDBI83G3sV3WNl8H7+%FT1zLx=_D<6!8LKHFV7UE&^|(UL{ug$knb4l|AQ3CO zC3mC^B2>X@2|tLXBkEm{$m$gUoFJ=6JMm2Eg-~YvZP_D^;2bsFXDVxA^)ildjDEHl zyYx&l3yEdY+^V5{+D;45C4*b5OX@%qJ|wfeJ?+I7jZ^8!_Vv4xA_eG-yus(Scpp z45m?PKl)|42!0LuI`vU!32hrAOb2szo^ynSNDXmEiZRzy`V(e*KIvm6X1{a&1mDh3 z$k8&C1$0Bswv^`aI!eb?OJ)i1B@9kY#dMUC#YaxLMzXP*mM4j7K{9Tx*`_?cj;acd zkKkn*>FihK5jjh4&;tv=Dy=$o_JUA03_HJ@L<{{M!)Clam@srx1d0mw8xjLfhFi${rq7LUL(t_1TjL zPmWqvjM_AGYCfhwn^-QQAeukU0Xru$9v5p=op$vQ3~mjy&g*tO^sAr0Q8{Uu;=3kx zM+Q=r9Ej}!ulXzeP6nuwe&G0pX_j{g{r&Ulu za7%z;m3lzHP;$MkNSsE`%(ynC*CXektHI!dA&G$DH^;3z+@YnVhgXTf#WgV(pbx zDaxv3eq8o9WNudPaGj8uH{xz~BPY}_{M^|yhh}GM&9q1%zgjFF>0~b$F%zuO<5fFs z)Fa@vpcn8)zM88KHu<>kZV1`u;YB_E!u+QT^`ATYe_r?hT?$jl(FsLT?zP{rcb6oodOk2>~Pl}iis zp@Ufmx)_IGf$>VTI2JM|^10Q@F)3Ufs5xvblkLDUIEKHl3=DrQAsr`2n;RS|*suv) zp1L#6Tqgma_0=Cw2dY3;J&s&a^LlhqfS_6ef38j3?`r)UT_#94cp5th^?JRcP~Vy7VLhJm055+r(Z zd>m>HGVoK7i%$xQPfJNyIOJjFB_N;2mPbD_B`Uuz^s*yW8GOto$Q7L#`+xumFe)S9 zym~(l{Dyp(NBXVJxg02+4$X}rR1@H&(h5zJt)gPke<#vR>QWw|KJM|ycsA1%KFRbb zlR(r=3j=pmpS!6!-j3Xb&aZ0nj2YgoipepYGT8GArG1chpvr|Da?D5m^|AdWLt8GB zE*MIK9hQ9lVI^u+bsV1RDd)QhbC4L+)nROcm|8`5w<0X9x7)dCH32OULP^p2iM##;c~fG$IU|Der1*-o~#g4XLPsRC1WLA<*DD#g7QAY~+N%3*KU zKM@Um=p6%+>GJQ7FQ>oS_KvA{C&5+!s$xcRzo3>^@iCnNZ>x?v$5pNh%Tk>JmVUV=wH-hh8pB_ zIIF}Z(;H6w?@R{Pvr_$JD#)DN zbp!0-F=Nu5Lt_{i{Rb4`@aZnk-iJ05nU~- zWA4GOzD5$Y9d5n*U1H%)WpHVsAIHNAwsEAUdLk3PPI=9cA_?V@?8%Z z^8AH>tj#>`Q?-88qP}S41Yt>}s(5UmB~Neav?(UfL5MpjAUuNF!zD@Ep(5{$dPwpd z8D;I1&O=yrqSi?zk7WN=ojaFwCvJ>lQMoPo4U=N-GuIGr2!_d7JS>iew%LW>V)R}F zYck{F+8R%iys$mg2xAm>CrXVBy1sfnq0hYv*Sk-HKZCXbx%PcP6Z|M&GEEGRl2u4p8uPn+mwnLO&v)x{E4i2Phk*G2C4CfCN`?CEUGw{i9#KQ>YzK>q~D-KZJtrBS?be>V#wPQ53$^QVmE&V~ zr8yQAz}#5Z`UBafyYAw)&13ZYOuK!wtv04hqSFi^_O8WdI}hxd%;%zXPO&=(w}j+F zu{>ckBn0-fxX`4&;u(u!$TJI0*>r9rjPj2%X?>l`a}|m#fb5Qle4Pi5<+CgGJY`Q6cjW3bS{~|j zqvJHl%r=Y7ta@X%F35yei|@vtz+889XD8_L;~}%6Y+{*Tdo^EpSGl@5(cYonujmNV z)%F42{%{`{;IH^!%U=Y=$B1rmp8-cFQ!V{l+0(k7ZiK(#gFVsPUg$>b+zF1y?tbeI z0&HXwsUaSf@2@F+NQU#5WV7Lq1;n(N+Zkb8gxDOS?bAQ@Q^4Z?O>v-!1#xXHMHiy@SytNnS~ zlbaDTg~XiS z%OgYtXNbatmn!*Gh40uGCH4X8!ssXKs353*z9}J;2d$=#Tk^Z!JL7*x=|a|fa$fi_ z2JJ@a$90?gTV3|a4&PzCy#%7>dk5?ZyQNa8`?SYC|Hi)X?YrK>Z+(OR-sHOly^WE9 zxa)fJhyR`L^Y_h%ASeKU_8SXOA@V6?{ay_5N$c}~u#*ib7v2q(Bg5-)oec3w9nw9x z1$LwTUP=7SO}4Xz`y33^v`gs!Oz~bHcdd(73{aN*lrDX|;(v$yWDfc21(XkcVYzXK zxV%^Ux+V~8DP~?l+Nq=rJ1RK-R#TV5xC|^ z{xmBDDn;hbl86&`Uh13KlQ-SyI)*wb^Vk&}Dan-9^qV}Xw~ogh2+BtzpIN``c(U}i z-npop3=>LNySxGbmcv#m*G3wcEY{3zn(HYAc^mwdZayolFB)agO-)+TVUsYwT?uPh zW8{D~l)f#h1O-k99$*$5f(|a-$XBy#Ai!>c>}|p+_8wh&BH;&`HC@%Vt!3fWU}oKB zPbUe^nrA!1B6Esp(8I|QMCa@mTp>Fj?J`0dq;8!KtfK(2(iQH@O`>e9S1MBk+}6U; zt(q57h365Oi%qwjE@-6T;fE`U)f%h~pFRXC@^ml4OP} zPOB2D6PfDhpJJXZJj!cYfis(HjHts@o`?mCL%Z1J&TJsBkb4nl7mwon>g%hDQlJzI zJ9nBCm1>JJg~gR=t>Q!WeB4?R=(3`CJz(Wyu(AN#2&z(qM6ygopP+fHI<;#ncdVnn z&##f=#L)*fsK<#_rFuJ*!%XnhQcY;KNp&uAt~BS71kf#eCX)k@R$GR%1keHhuCtgR zt*Kq{YhTOaosRS3xLS$T{tgoJV%Y=dF?50l5496srin~bs6RjW49HR@#9P{t{9opO z2GIiaAI5N2Vha6!6PPW6gfk`a(qY=4i-qPHid?V_G0El5D~ptKCb<`z(w%!gr36R8 zn&wfv=~8hXX^Bk;lGhd!HgE1Yod!_rL+wt~`-oikyp+pft+|`+*oKp>uP)E!K!EU; zT5*{;4=R&kQpbiYm0w7!aZr?GI>D(-idF)AH)h5ewEKk!2^emE5HJ)hCKxMFsHsRX zqayC``X$C&IGdt|f>KqAd`8%ds@n|17gfaIN~=?qgO!g8QKQP=r=!HB!>L=3PAJ&! z3O?6RH4o$T1l`V<>$2$&iJ~luJM)IzJg3*G;DDtc*04~7SHv@XyUsYs66?bYqtXKW zt!b)F5EP~PL8m>r>u>sf64^)%B=T8O>vYnt@EE}h-p_WFlZqtuS_6Jz+8rG$%uO)DkTJ4Hw93Gf< z1!{eW7?08wStVkiGG3Q8St%Ugx26NMM7c5*AQDs*SB=s`nX&ejtAD2%9x@rXuNw~| zD7x(8eTc70w`y`#iHu51jLHbQq}FLZrZ_~Ov)U70vNMMLOr0frgF9g}?#9z&VmQzN zA%QHUpxC{yv684x)6n5LNO2QFNOe&2fNPyfcq@#i$aA7)xS{Q`x=s7jAL9y0z0K*+ zoOYvhXG!IXg=?h|n51KFcgoiJheX(cN8x}0;5Y3y@n+Rs!eCE!NfB+@pMmNBt zAHEe@CqOtEGuWZS>NORiX*BNE8}p8WWug@HW38uAD?&-4O#z>9M2kYM z*TU}|yDu<2*Eb43&9tFLX~PFpAlOUk4F**%&s-){?h?mZQpNzAK)_@4f>&Ki;=mx4 zF*&78)mcet4k-*#-KBy2`Su9L9$7UOrWwc?RUki>C~Wx{Ep=_i%ffGcSuBM-2~#(z zD{$WZec7d`dy2#1zIPM7KcS>+c1IH~L6US=wpc20ChJ)_z^Ub)`z`n9(tq9pk{GIkSkD)o-#7`|D-?IGp$hl;cxy{X7JpQK}i;gAj8A&@3$eSMD0zsrZnboT8c zftj?WD>sKS*TE&13fol)X;w9*5+1y~N_~Z0YZTsOGs*vEAIcV)& z?(#Dr&xINdvVNq%zkdgSO(+`X&8Xc^QuoX9skm+gSk$Sf65jEXBmbQ$eOw5VD()&~ zjgpMNQ*d4*RsMjc70e16w{wQ#TJjsV$d_*HMjf$ zu$Ox(rKPY*k0;KY!NL#uL@H2zB4u`A!B!-*8(HToUt?p-gM0$H)^Bb5U89^|HH3tx z-wtwR@YO^0;aWe1%ecn9@PV$@BBdR_j2JiA`y*TMoNOCF#5#@C1q3adlXnHebF`Cb zr4mHK+7-^A6ZFMXHfeF$=SeS2mW{hD47OlapoLqDk&d+Fm_|V`DF3@TfwngrEXw{{ zPRZl)d{4`E-kJeWp@boGM!&q#v@ zCBH#?t)mfuyi#1=t->>>gmN&en&{`*bio$ga%89H)qwZMqXk_r)G_Bz4SW11D0Tl2Zemn2Zc&Bk&-pU{RdUYlXtBkL$pQL$$wq%qQy$?l1R}MDICH$7#{X1&nl6IwR(6< zu!A)KqqthxcsR?VBEd^0=lyr0iQB%aO2jyw%obC;C`otB3jCounDhMB>TJA=b_ozIYx89OGQ?~LBHWKy?HQw>Wi+*AA zL=lq9g5X3X6ubDPOEhJvqJE-dhKwlnQSZCIu7@|K*#ZnMB-&UwR8Le zcq2&5=F{grENc9U+y+}jO+1vu+xX!I-YBbIJLiqvXt=!08y48=y@AE%9Gp(Sy(mvh zA-30|&A3WyR&mGGl0|lbZxQp^^FoJicC+YCQPcQyeK?SA&rDeOT$5&&1AM_Un~g_Xq;`1cUAJo+FWl zsCbIOxnxYxRbjw5tBum`MxFWxvp6ffWx{f}KXSJ-D^y1S;1sI^=0em3@( z(xll%<^vz9vg#JD*B$k)cooeP>SsLB$=SUOCzs?R$WsfC!Ra>^xGTzztfLvFIY;E+ ztf8h6m`QOfOR9UWi53)e@(WGhBi0dEj`1gp(RTF7?wU87A+q-rx@wrMK;6rG<&S>} zivOoX{olYcS=$`naEwNB{*29$@sbW#ox-wdWP?*0Q!#cmQDJLLCp|tUlsM7g9`W1^XpE)XO@;P|& zI3#oTs{P4i1->)}+;ly)bL%eaj{E2>)3NK%V!2#vO{l|qxGaP{A;RcTv63fwNZ@lt*c&KEz*OSS9Y5Y6@=Gt~ z{y65r9x(Kgq{3CQvL}Engn@vzPaa)Y`Gyr7{cI20ark(0IjHMr8(q`bB_Oc6jVkIC-U`kNBRAc!1Dw! zY3EHd2ZaR>|3)l>mFPLl!_(CLF)*W^_DG6xR3_ltzmhq#_lNs|e@f6dfKfl#Z$lN- zDQif3Q=`c+gn>EYRb7#eCWU+8ezhjTSMyUerN9Z=(j*&TY&*LL`jXMsu-GBCVO}IX zG&mM*4Zd??r4p=#4x=K4n-~_H`r;PZ3;#r7w3HDiqC1Rh``bV3o#`a`8%2 zTKsCr3)k#-GEuO*{_pO4+u-^J2{oW-qOG@L7yE0(KbdX{e-O#w%&)tN2@+pPU zwv8j-9(D?#PcY%OK}Qyp-kx6lL%F_c27W_jF$y>im8O~-z^EAbaICb`u^mMp2 z+%b)e*8>e{n5dGcVqgAqQV`$07J93+!lfaO)cFkHnjh8cWm3%BQcC zwoPeO^mt`@OYOWm6=ta=AV18Ml_&U!wd2Rfiy%+s4y^~vx(^2YPl-KMOlf}jpWSrH!+4k*5>&`zd#bUMbqdF4PX0y+hFr#*aL!ULmi6C$$?w^EfX-Ph zlfy|1nG*d|m`a4>6Z$mKzdkBI=V{q7_CaT1829a%08i=A*Gl8G8Hejq{hdGlzpv>ZJxiOAc{3$ zr9mgg6?83$1?C8naAJG&km`zYvU0U^;98N**m%5ar>P*34XY=O`&aDbyT1yP_~oBc z?6H@`^uz*k6+eC}Cw8dX(f|GOu-f*-{nOls z*yCc(!30Cb1EKr;ZH?g!_bH!HquXW)fIFoe?VXWNAUu-sjr5!(LJ0|$qbKOMvPW~? z)u%-!S>CUBDG;(6#Q1w6-p^-ctV_b=YJzU66>*WV2OlUKYDT8nN`b`ZTW1f3ro*sL zU)rct6(Wj9S5@Fs9H^_cd?>ywYK79Xdo2!zNP-oW5y0lhiKga&gWld7%Q%h&5G_Px z*JUUPit=%TD|5q3+~vRsRdgSuoy4xkV%O(JztG!4)HQ~G+c>^icPAv0aAgx)5X}+E z_Q!}4lFhRKbJYG=m?y*+oB#*<STYPSPTebWc5}EuGI7#?8u4N^g)EZIjKc@Ia)~yl?sIw*;(r~=ACKd?6+{QGu z&vZjZzCpnmke%*#Np!wf&`fT0*PJc|xTi#lf5 z=YLeY&*!;zw4PkR-jo|~I$lkDn^>5vgLO3p-PlUP; z-sy$Y1QEctrjnAAxZ%MwvGwA|9?5h0y*Swm#TFbV6oiQ2D@0^`r5g342w|b4Mwk__ zg#eIxe#z5)27>Pb@IKyb4=0w8ZtlYQGejfqwf0OWAKx7LqTR=@^?_SsjlAQBHE%uT zBjZk085M;sXsGNF_zBrHqfY#3@zJ0Y!S|7b7wqca%CA=gpIKCrlHvn`?YzvfxmH|m zCjP8s2}}UKJsTjQ{`JfAJsYi!@&Eh`!GA;f4<1R?&fn9}PTXuiLje?w^^Fvr^qrg?|ILi6RJ4|zlSA{SKU=I!z#I~Cm-pn~_)YaAWCRL0cQ+2) zX+U@v(owZaZgwNJisZHDijW_P&E>inH_SEqZDTNNP_X z2FC$nkfADHcZe5sBSmgdB5x=sr;v#}{#X&JI8g&Y!;Z9H=&mRw&fr|z;+5} z!Y_eL`_`d@38osRGZ9A&x24XlW%EK@v%Z>SSDDG+$!txhe5(I<<2qD81`A3@yNHMC zeMVRM4lXmWQB_v*tekAk9!r7jZN+hIrA8I&e&$vo7Rdjjj#n#`_mVx~alxuenywLG zOl@#X<)Ki)T855nbsZ+Fdmm!?!P*yYWuJUyKX&A>mZy@nK6$j-mqov*b(!qGO*M;A zPwuW&Ib#hC}w%A_33O<+xV;Ig7NmK4SK+KH;)Uj^g(XIP`(G;m;aTlZvf>;UbWK}N>CtuPZiIf z_RTz>hO=L)7Dq`NIh3I@>a3$#j|obB6J>>~F9ygs?c5&W=hl1NhQ~X3wVzyChJ#9Q z|L&?jU>lrP8|IHWFD2A#9`}gvAnYxtdUz|`>m1z>Xf1iSX zaYFysDPa1q;7{24cK<^f>V{RV(vl5-5N!<0$pa7L-5X)>!HE}ttXKr_VyLd~k2C`6-HAGN{rP6A! zBjDuSm)2skPbDgRWujKXwrMopeikSN1r$$xWKjqV!F%*Nze-#zWpHnZ1+FV|Ypf~e zZu>)gTbGxk?l;{RuIe45nHEWbY{U+{a4McMV8Q1x&^8tRyhMc5M&{esj;58o_^J{f(TZ-NWVj1%Jv(|&i(NPCD#(b`wTkX7`zlKS2T*3m zdhMJ5EnH#`t87US>ngxg4<|Nz=p(M^)Wd2s&^*m^m_yY9jAlEoX0~^ofg{^rLN6eC zkXJ2>?WMt^-f3tw$_tH%V3ToOr@H#Gye zoLeIz42ebTs|T>Gk_xbYn+$=J0oaQPjweHKTy>k=+*~!*orTfRZInty*-XA5HKM(X z7s`}Tm#~{YMGN5)^Yvnixk-eVAjgcHo?;RT*x_N}U`bhHL+D@+8l&|NMkz=)}iu^^g@Z@GWtsyUE*0TY!iR^97Ifs$&_GtsrW7>^rxqI zk}p>38o?Hii2TAzD!z;O8&CZ(5jVrIj`d?Q%>PfZm4rb@s-ESLL_LE;cU5=Bk zAG?-->@Q*;*4aG~W|`8%23)%5hVag*l0$RE$!+Wf+}Ue&_KfEY9M1AI{l?FXH14Y& zD88#f3@y8SXkEF^Z!tRgPM#wX_$*b)?g9^8mmS#~Q;VK#8l0bQcF2I-ou9mFK9~!y z-LuzA--62i-2|EL~NA)2EL8TaZQ9NpJHtqKzl+mVD=xh1-fh zp#PWqHLs^G>o2>#>>I&_x7Z+n{f_R9t<5{uS5?e=@ak=L*-L2J^UsC%W{(d-Oz*I4 z_iLC>)xIyPs9#Ihu;4GrFbmc5!_^D-#tR)~D&0j5cEYGFGn4|U1tF3HB76p_(zWCO z1PO{Xav5nYzf^AK)K6A;5z>{VY^K>0BZ-_~6lfA$=|tbiO2YZTJ1qb>2yeIju912P;F&J!QDO!k)?LoGiZ{ga2EAd9@T$!rda*ziqv||2#(K~6fSnEI7CfY z$w+pdH|?m4HtYQ#rEDWkx4f-Y;6>Jtp-l}!vph+nj_B+L>%DqT%>^-qHq9;oPSg)A z@u5L>}iVG$nfNObQ{*G3CNeI2~khdJwYJ-%NrsfQfE#dRVHC5@|Ia1zPloULUCx*KM_)YI0l+NPa`eMijqvjpe?kF zYi?fGzJivQ8>-ovv7Htv{DNV`QC6PiK12fjTaabo?qMIbA;-`5a#-YwW<$BosAW>j z3++*yWlyXZmWKH+?MMYg63bO8QPA*>_iQ8)3W!AvrO?5Y!F_Z9rSC#RLpRX)koq;T z3i#G$OZF6x779gUaH*w+zYuB~TJ+H?Rp3lcm^4S7@#0&#F8ffqh(j!xWJTFMIEeYQ z563{XF-D1B&r&Fwoi_UX_>%|fgpy&tTi5Z=1@!D=3ewcht>BE>Hi^P*wDA_w?_3Cy zk$M|vqg`#Z$>?o>iiE<5thwP%zm<3+?L$ay3u9mMawl~r3lG(@vH`DAI7 z=>nPL35RxiP0}AF^vh2vy-qwoY-f$iLAs@rFBGc9{ptAu>SdDo=SZ}zGTVAkBv3q6 zBQlwdefXoF?M~)#KJ@L%a}h(%=JFE+;13gR4P~b3^cM~khM)WPQbWPlZ+?qg@2Sc_ zPp5UVk7v)6d_|JMPNxpP5l$U2Zwv_Cik>zYwnnbR!d^5J)(B<~Y@p9(3(@eA7oi=C zp+LXSQQI+K5m%UWnPY-jD^?nS|HELkqxF>Q<$N`HdOPxn8nr9<;qBOmTuN$^(G*JR3PP1L>N>1UjtJz?;%n8M zTGu{BXFw$mNBXbIgH5>td#8J?5UKHfEkkK{TSqfM5lmuN3_H~>qDfQExl*VCvB&_M zB;i;Bfz(Z9tQ>(wJG1e$j9a`aeL%HsX@9HqSy%XGQV76=txCvvzkSJjfgjwowX%$5 zg+4w-WX(XXB^QMsHXc55h$@mMwBJkjQ}jiKK0txg}mDSA2C){vjj zvPOUYlUQ?FaDEJTu>*{~#}pGfVA)(qQmY2QNchWsK+3#!9C*CxYsZ6sDo&~>(rG>o zuDE48ufGZyHt4jg(_?UXJd$RmQ1gt;%q@Anof*2b^|~zS7+l?0QC^p~?_`A&>D+M= zq|N-mk>FoGi;_oW*Yy|fYz=~2#7J46Z|)N_{s}nfyrAjH;9z6u`+I3o%rdl@y=^4&7J?8vv{F1A(y|RL_Hg%GZ7Nn~TPSZ@| zJcVNgOJ5Vbz+a+2y<6`{br06z%kK)D$!q5l2zR8G0ca@E6NP>#=> zgcjWM-+r(Sp-wE095;*o(efv4gu5HiL5W0!YN)H=m2e|Njw)MBS4nDI!rD1&_vGeb-*0@Z9D{JsA4N!~61`wSfhtEs|DLd~4~DtsCohn>hM1 zM3sNFo*Kz46liYL-XE)moh$<7!wg89iNFGJo0Qr@olhuaBR|tBU{c(9{Xzg-mkQb; zxz>J9)m?5Q;AYDJCLG>4(Rd9nIr%&^uu zZSl9c@Rh%hP4ygBbHcU*X4KK%EJe1FG6Ge;MIaHrI9o)v>3gxZUhHZ%LJX{kgMdbB zu=&=bjLlf=pXu5-@sz(GbpT#ju~D;q11>Kf5%C&iXgyqv&exHiE(eG#G6+KV6eApO z&PzjhN?&MPS|Ck_i|q}R1U|>bT#!75T@}_KT=mE01AX?PGrVEZlsN#Sp34d^dxI^4 z^UZVe?iym9?@4(40fTUeHro;FU}$GDy+jFBCwu@YdF6sU+6!aC9iX#h9%u>1*8_D3 zn>FO;6CCd%qhaKf?Ty?VOUD^CpssV^9xig^l{O`P9_BZ6RO9xxo>1Mt z&+8Y+2C@w?BtG6$*<8L5&Vgi?-Eqse=CR@NbXp8rxKcWMl)zGEACV)#5RmS9sm`wX zIvkZbG~0An;u-bo4bXb8OXAX{Ih#jayVhUv8^F^o%);9;=qkVQpk*9-VHsLzpSTYp zqBU&~&sq!MT$Z*q%f)OkV5K9>xoDp0(>&S0f9H*mDL4-HzrTJne|nlB;S;L%#Ove; z#4^V(TDR{A;qwHA?ua1u9>=fV4q0ZTq~{y6Q#vfMHEPj41ndEq{`FS~Z60Cj(0+F- zuGn>?%XW6docW5f8t4;xvw_~_bpVPML%U++eSjYSQQfzXNAidndQRYB3~kRdKEuJ& z|0cK_(j$$-mM30^*{^WFr}N6^D?*@c8OAo9b+H4;Tix~U-V^k+lx6ju6`C_R?Fy9Slv-Ynu zHXr;|M+B$m8g;UHbtUl_AheDq;Bw}gKC;(Qrhhek1Bwk%m&p5RUvmDv9k(UOLIF5@ z;Kvt1&!0+v2_FPzu<@tzJ~dB-<_PYTX0BU5;Zp1bZ$Pvkk3_TExY8Sjr7QjKaYAMM zKk5s$Pwh_d=__MpQJqieWw_C&pz2DpF~O37{HTeJJk-^F%MMI?@2uLJ?R}_a?>x*N!?_$;#RJmYVm12wY>@A>_$ASU! zULa(G*X5h`R$V@62toS?)Tb4C=1CV-XmI&1_txm)ZKkzteQW`&N3~5CcsAV30nVHT zXLJVbCCDNaOGs=P0ftbx>BXX61C-XBf-Q?HPvwq2zjj&yFqYR;B0j}fz%q)y*+r*W zMedV!kLFR{Y{~cv%RY#A{WR!VqosgXC7Fd{vjkM7s9P^YJV&<4#b(Rr;UN3hH%h*w z#&?k@%J${pniHwXHiY>3JMuXPOTM8|8>Sc@58+eLZ{5|OGf@&HlJ21;%c`EpkEZWk z28mI4d_%$oosXzu6<7j#V9vt(4J#6gGQCh{pE!vbHi#TvlbAJkAWcrs$MFE$-9>}A z1trSO(d|ps@+NSzIBC9Q^8##)feTt#nLMe^=M>*FWn#MypGrKl8WVAHH;_i}H+3+_ z(i0FI?srs_1Kjil{6x4m%WG+zV@;{EEfJ(G zedt^5qO;`Gj_#qFXD^6G!fn6>T*Q^Bo;TdD+(Yj$iW_E}FLZ_1*3ugri))6`59Eh) z;(>OlNXr6OQMP{hZq7eiqp60?OXXi}{sUEX~Bg75NOUPxz z3gy z{iodipEdbEGo*em;A4w!1jI7yOcg;L+a-<|{XBZj{7G>%HibwCynjs^ zgf;6i(vK|41`G;ne{aa(yN1Q2Hz3Y#rPu#^@-L--?c_#6B^%Z(_)vX;oW)$-m=-`{ zN8@FcZ|tCs=DmX=aanE;4!-OhtW-ZJ)GOOzVz{o;SYOk)kec(Xl=7ym+~gs!qp&7% zZ^XRR+7sWjH#REGNKHJOV%pr-6azcY-9g`so2t2;QOiDgS zp$yiTAzPm1*VXM0B&SRs$mMycg$=;2a==t0%T=YaRxYLT*3A2qlJWv3(R+}Bo5?Zf zNLudZ;ewCU8f>{|rY3zd&7#1{lg`jMFvX|yKM_T=+;ThqqS_q)6O=o*JF;C9ecI7WG>J#ARZ$CIo zI>U?+acD3(7o&**9x4PuYD&Lh(B&R1yFIpQ3G>ulC|2BC!USFA|KaK#gDdO0cH!8z z)3I&awr$(kNjhf7wr$%T+qToO)A@2g_j}Ise&@%k+Er`)+BN4KV_xGzLPq}AOBXnW zVo*nL{uNTkSHgAtB;(T;BX&noY1X`XKNYmkg1cC}Ob_t~7uVH0!C7wZs*d8(+$(dLqOevvXGs#INvsx-E&UBYH^NPK^6jHUk(=h$1w#!DoZ_cb zCB;iDB_NSlqiEShiLg{TVgEI zog;V?2);qL7%^d(=I1O<>M|{ukv|?no~SnA%s$)1NPir%V+;&9{Ii?>@0y4B-@EA_ zC>AECXL=Lt+gADjA~i7iU#&e6AX`mGWkC_?FG6=wupkRG#gIkW9JQi(53q0RP39f4?k&t=j<+Dk0pZBMYshkCa-ri3j#u#|y zt#FB15@VE#0G_GS0adUC5OeJxyEWlIuAP`@c6-7xhQucKfB`;OLO}v2fOgH@{rG@U zkQ_!)n2rmaW&1!QoRPfN{hv;i{sKIHu?pB*B~NZ2hGs;aG_*e zsc1VEu*5@i%N<5qJ~iIexokSSQBoP#!D-9mmSZg4=NKUl3#9J$iAvftho*wF9GaGW zOGs{|>?D(zM~o)Q`25T}z*Cb)hPCx_O@=$Z^ZoQ=tJoB&G~XTeb5u4T-vPOCEq25xVr7AEM+@xcU3An53K|^K-VA&H?kIQ~FeUKKIB6Z&c>(6q+L6ocin?YI$Hc@(?~!mF3pwjX)R5FCqYDAb zIK@_r?Mu3~TwnsBfbthQMb*ljRcMGY4#0IigU?T7tc_pRnH~VD0o}%xC>0s=m>N~* z!-t~EMSjx^g!LIwLKLTld|`}^gnJ!x65v9D*C?g7PzFr~sA@Y1ER|SJdm4UF1S3L$ zBaABiuyw{jH9$6^%zvGoxOz^5RoJsdifD4U5g zweFk=JT6N$S&s`Wd{ldW&d$PUdE9pkQIPk0c*>UFebDFEyg#+;M z_@Du_@Io8vpA}6K*rhr3xQ}+s#UDZz+^d(>r7uc1-VUlQ6QmcxvkUah7|%%aTq(+Qo<> zBJ0F+$7zq9h{-5vn&SyUWr5d^N$fXj(rAY1i%ft}VOcwA2P_=&$^u`aNk|HMy=j`f z$KcLMS^~e<8R6r(Y11X{A_9z)N^kLrb%!6xiB2&;{GjXB;^ZOg;F`VmoLI6nb_V7* z2y)9LMtdBsoww^*yzL0*vW9|S=ITn5p8L8luoc}@$)BnpdQD3W5irbKL+ssq| z3{nrMafXH^)U~z2D2+31z%tfqAkz?TKfR2Icy$Ax5uRtrrQx4A6du zv*yrt=j=Vlj)&UDxt*4=i+Ca}HoXa^N;&{=O7!9l1 zH;P-F?O#r#H|j`7G{L8sY3*^d?BdP2{Ym)Irnx_48MxN3@}4n8Yxsp{9JHT+J>3Di zK0%gH8clr8uu^S8#E3_@(NsIc)xKd=W;z40Z}~cPES|VlQfA^MWj{zv>X?}4eIc{> zODS$g8Tan*Uj8HzBezGbwwX3>3_LDu3W>hz@00np0x<{^fx{$Hx;`n8a*jyx$ZjG4 zuQmhGu7O2uNlcDLvRqybz6I7@yBY$>bTmQnH7okgF*WfLO*0D^Ncl57BR>k)MIQJ= zJ65c<;_7Gw+?#05N99G!wJAWpcYoBuw z5d{SH_k?*`;qv-o?#(JdZTXO!iXWGj!4@bs-tGK3a4R21DsPQj#}k7w>sq5E>5XF!qi+*ACE(Vy&u>FW+toft-i z)m+_19F}^>h(3g@t9AqE{^9%5vzeZ$6 zm!~sNOHF2@m@gO(jYE!LoooESAGQ9e$q0K|YaK(%P1YI5nQw>3R`bo$%SgPHq6s^Z zCuT6D%3R*Mg=ht^G=)Rqz^Kc|!!8yoLCwzUB6(mjsD*(f#vc~ zSyO5@w$=IzQD~7q3XkxM^a2BEvEoiV)7?wqe&n(V&R{IST%y#;$q zYGTfkeJw74>yL_kFiM^qKrSKlVsQcLs9IO36nJgr3(>aXEyeUjC=OUBr^(?g6PG#*P8Zidz2fLc)<*f}Dmo!_!1j1D-3 zZcFrqncn(GOQgpQi&S^zE+4wU=sU=Jf(fVy<{$-Nz}9;&5}ym)L)ACHpyCDHpKi8W zMIbG2+$+}A&5iOUCUR$ABybu(DM0YP!>GI>%tV`hw6JE^gW5mNh5B!R5PGRoXtf`U z{8_qam?(VLcU1DMNe6@O_6xiy=jmtv)j}UPO^{+#}wrQDeMx zyr30en{wQa;I8HOpq#Y~D2&)&4_Frx5;A299hvy>i6#Kw5^mlvss)dve3Z4ZYb zc^~KZ!e$w$tz$Km%-1X5!$!^9-ML#SOm(FV6OcPD3J0A+~(U3HIKbLs=N$l&kNa94)p1 z_*uASgD00K0`CHzmGLBl}O_A#}d7DQiVRd~fH_pC+(F_Fy6t{R9 zkkrDy^a6-&iyU(@l7alr-nysY(vo9^a{?mmkN}GlKCpG_sNJLK7N1p@_6Fszru?dS z;XWj^p}Al%gj@D_m_hV1_z(%JNvR4zqN;$JkK_1bjvS28nrh|W?g@`}QC@FCHx2dM z#qeia%C}dERRAv z`j*A2{J$cLTM~oGBO_{;?=;UY9@8x$_Uya;)%>}pt2>7muRJ)Frc2e6Z>fJX6bO9D zIS@nC=NA7|K8HE`HQUKG%g_fYi&V_)6g_}vfXGQFejUZt)e_@c-kRA%VnqP9x1fev z(-HgjDx!e^*{p`~-y%J%+Sz`926j-U>m7#pYwQcc4wA+#g!xM}qZ2N>0ZQ$GT-VgA zA3vn&P2{RbyE}&`|9*3asvtx^Jhm?NSxQ2Yvb{Okl7ddxhYup=7Mku+s*y7Pc~pI* zkCIIFP99lz6+Uj*Yw(#F^}hYMb#HJgmE~U)HvnhvMcA2L~WF3in4km;&jL6dtJgqr-~ z`sUkNob|tDi?+~52;^8F<}@ZlWo!hx6! znYtzNqBkPW4$mO6F`i-1am28!4$+bHo!e zro6}TqT%TT;d&G3)YbJ;h&MwH1$Qnh&BK?OwRL~>obA%`e!p@e0tDU_P8{nqUu2>f zAF+?n-~{)h^atVS4_0uIc`iH>6-7sb{_r3GB%B@mmh@7{#A8gP7Dvbdb99QFL}NP= zkG5Rv3hlLZXyXWcHTjUaP-de0@)F0({8^h+sJar%MLAJue;$R!i6G0 z#Z+amsHvEUi}CO$)T5SyxjN>nwcYOjS{+=(0GL7yPVX(Gb4$Wl&LZ=-D32ezf?o-sN}MY7G2>P&V(-2RNfXG z9O)iinbt=FzCcZI04-%-p$=oN`mhh65*ZahtEAjdk3=yq-^ic=5xE1qSvu$DD8tj9 zEyB-D&}HdN2>)gj&Bv@q6HSi9i-z9zUH$a{4O^HO|Lo{JK)4Y!;3 zN3l{Y`-YNE<6QWW<}iJA5>)Uy4?CQD)BBWYzIcbyBZIs`>+i`K6cuuU3v#bv(H|%R z6q%_I=>$o#4LekXA8aHbr`+dN;q@B?ShQni$AQZ^1=!uBL$ zCc*ZyT&8(_eK|;` zvRawWT1ZN;X;n_=%KCYIy}PCPy`nlT$7N??k~A0vtoP2Z{kg~G58rd|NzVO_%gJ|w zFTp!pz^iIQkv!k))EUrMPdLcp$F^u6;mDc8J*1qZI8{NTyfMrdg0 z-R-x-k9W@9ffc*~j3G`Bo*-xra?H%}0nJNAQx7>*550A7{vOxd9i7sbot2-&&`$Ec zk21j5Ad>LqRz&H(3+^S}^2G^=VV6SrLzN#uj{gsr3BGsnrV=P+pI^YA?lFq_mSg_S z4oK#PN9&Xf{yQ24@ue_S!0LsiKegdIU`+<9Ssb<5X`!KI`3dV*a#D6V6cw38cf&-> z@7_j3x?MfS?g!d)`mH4E_Evnf_A$5>3&5_S%B`?|txQg5PVW33g4zsV77IO?N=j0Dt3ugkc#dT!SoXxsQx86K?mQR9irk_@yrun8; zqo9!uV%Wl1Q5%_7urg1bv3OUfyPg8`9K&(qE^BD(ui*|_n!C^#*Kt+6QGn_#-bKX7 zpj6{Zv`&j0Pu8zH@yRr^I`que;3g*WDA6iWJV9(b3kYq3sNmr2lT;O2K>+3gv~;Iz98yuH9Bg;k6YJT5^A|; zu~j46RHVs4=VktqNcm2FwuM}iZX*k-@XW1H^ex`1Xjy1wrz>+GflrDFr`U;{hDTo# zJa(1xaTd%;lNN>CA7Uqt+fMQ6Hn0rW1}L#y8sHMCH>>NS+}P|IqJR(r3w)a^mZ3e@ zrd?kARKuUKGEN*u#!ndfDY7^^OKP-!O!y~-YNO(|^Ys`jr)^l6XKjj+uyx|2-d1h_ zPDy5ZUTZgc6SD3%mdjj10!$$bEd-nK* zBh}peL202x*#SDPT>wlDX81T{2H90DE16B^&Z#2tqQodnA`W|^$tFMAG~-Yxa`wzI zdUokLns~VNbVWbMlp7;;IiC}87TdE!qFQGkbb}PZ}~i_y;L|>j6SGxk|fu>A9+qil~olX*sLh=tjI@_9!xP@EO;W`RbOq zte4Al_}Ch9w0igt{1`D=qT-Yna}tagu=i||`1Jn21S|xu%(ZzP3>QDSO`0p)48D?#=T_x zYBtC;dUULM7XQYc)CTsX$}8!2XXi76lM5&^IFB8b($I&dH0=`gYMYkEG+m;Wepp7+ zMT@^?9hx3wm(N!YC{YRQWcgA!sxl+qy76*D;fA7SeW#B&7)&OCTg2V#oA9($X|Hsd zFEuRFWUN$K8HkkOFzhgIiKnAVt2ecH=*}(nSZ`%KMWe*^ z=cjT=6~YTyJ%8~9j(+krE6%Q+w zCyl7(#I&+wusJQ`bKJi>>QkS4eHIU6Gdv%v6#(w}v12?U=LF>29Q#DVu=1oizoDyx z$z#b@6!%Od9Fb3@Tt|yIfr_}20>+Jef93;N3geQ3kMWtJRpTI6n0rTh^PoBuFUt4_ zg53-kkCbMojgX~}QL0XBm-7ni=>xT`y+uqJJymioz7?@MVZ&F_zny5Q^(7PS&A;SfM=1%n2!7k#Zilg*>18TP3JEg@tZ_TygF!A;@QCv!s*Tl*?TsV6dP- zE69F{VFaGhE>62}J$@^IfGs!LZvZ#+vQT`HUD>m|ox4tdj`dr$M=40kwa=EXv|_L6 z|K1O%xw8vUdd1sIc+pA1-#75TxkA)dTL6S&2^=6y6d<&&5Bsi_5-;MIxz1(n59b<` z49biNL9|rBMWrWpi;uX-@61?VR#`F2LtdHKA%1<#393g!o01rt0FM#E7sUital`!5 zcPE*{kr-G$+8(n=o@wN!Nm1^$V0g%;G?E49`2|2iT}CXpQT*MA2!73->ld)ncmP1F zu6^bYu|waa-HuD(u2$_t6+7AnjO_vdgYl=Gpyo@pHqx`i? zSV#2M57~%~U!sxKccdl##_0fDmK)A_JaPUmPFj)}Di@Zwy~4Xm&Zk2oz{xVk(I5Jj z1$Fy&PQjnkoD6!Qm0mRxx08@cH#nKeN?k{cn5@A#FKk`ONs@ybZGjp}3RfSmFp*)L z4NbDW!HY;0+1N#-(uYADJWzVN`ojQ`w2+TW#n;mF%a`b^+!xxUv|d2!s3dD`O&ct% zbcu#&Fz2;Ua@{U47_^U*_88m@YFhhtcKnsZ(fVha8)Z`tq=xR8sS)Jb2hH_=wCcn9jqf*x0N1NR_hwdRv3^7_NRobB}*!7ruwuV2P(n<_$u!g`Z_dESkb`POkj1M-Rb4m<>dxyKGbfCmsOi^SbZhJb#FmXWZQ^ zeKY<_W!mK5XKM3$!ihN9Y4_zjD6Xin27CyeiFUy*!*5IVxQ8*nVW2 z%MpY3`7kR}2>MgtoYO@siSObS4*J2*JeE+n^yGEYz;OHpI`?sgQ{F!U;tgGWzBWAq z5cT+gLj2&<93KMbPBiX_x?L*2Gif_EG}R_%x1g$fxDVJo@pVIcI>Q{$bHpmWzqRrF z>U~Ypz1t`-39+kEZ)m56zUD&Ti1AB1*sa0p(^7HIvmDB4z)dGzF6^zB_{iqlC+vy< zIQR8d^+DKWV~4BB#<-6wh9EM1c7%%fv2zGy0lN@HnHC7B8p&br`Mb;C;jkVrUV{9c z1o7WIrUmSJU^tG|j>c|>WhJ>eg)}d2Pwi8Imi>V$cg8et!5>O^gse|OO&&?$K-|5| z0H!^#2AZ&em~*v^Q`Q?lcKBV)m5OMe8Y-MhSWJ&^R>PardFPZ;`2)4Y{^8%!Gqu2BW!P7A!Rz;xZ{x;lD-VD@y*8mjIZPvGdaROpPh0=$fq5nC zV490MIi!=Exlei0Rc;`?W{gWcD;Jn$2inj|uR4!=R8{?OG&?IHhQx!OKq!w;E#vbA z>)M^1L6uL?N$MU*9WJq-WcC>^|96DHp4b;(#rwerHPFsJK@UZ5gc|>AF8b8zA%4w} z#tkLrUw_~!RfrY9e+C@?xfsF&0dW!kuY2czJPBKCG-*u#M@>UysssDY(E$bdp8-cT zDOBo5Cp&5@*E%rSzgJVuM#7W$?{D5fQg60s@l$u3fKgJB+NjZ5Yd3&9fd9kwp#-Cw zQ2a)D2EkKr14%Gb75lkxQ(Z?n(E-ItTk_u_v7=w5S5KfA>p^AVI|VIACaC)%_5Qzr zLK8}Kn|5QMvDcyX>@3EvRd0y89;v$KEM>cHfPIk&2R+Tp;3+W;8`rW~8c(uz0Ml8M zaRxg*pqc~55Uq|rU{af7ef?GS*Cmb&A@9Muq#_2cEoNA+f$J4jD3(7NTL36!lCZ_C zVx<(>4>DhVj}{*3zhYe}#lG|2dtr!PS2cS4r9IWlR6qMvlrLKKb;{HwLiJfJ$4XX~ zy#Zr796Qa7a0AYd$(|%#v5G$WdSN~5G?=Z>5$I3krjjv!)?I zZg*Y1X|r|$#(z4J8{Jov-T@gLf{rxOV~)in1(FSeiH&7%sC@4lEzJG#;>`J;$6LtC zde~ydGtyrcmBAy5#oR6PmSO#YRAJS*3uxWp=_;k(Nc$nggsYN#ry+8$n;c?UyzQT2 zsAJJwb9 zJ!QxQ*|oT_X8Rxn7@=G(Ul5s%ejI3TP5xNohz1l0J4AwT8oz{bf}^uz3_y>#z`FfN#ZB@YXl^dA?8ds#OHh}V%%ob%fU2!0H#w?g=u(wK^i%+@bCQYzbo$l zUETtlez*`}$_#eVrXCJ;R1oA&D0%S%+ zE23{^PV%>3r(!ZP8AU3N3@7b(%(0FJss`2;1vEC)AU>022oNw4<*k2tOQbG!QYLtP zSu;W+Fhm#_1?sGN^(VUC&3sMILkOg;TFRAnv%;13r;=T-19Q4;z>k`xFs}P4uAMcX zT$iWU<39lZPy9YAUc`KG7PIhOcdqN*nqUTDs9l0*-?6DrWy(#K2|{WBM+6cZ4u{JJ z*UTIU2RBC)+f7Yzmi`)PRV8<|Y=(Jdx&aNzM2=>K}BO>xB-^Qe3Z^Rb(5jYkDQrvgD1PO#^C(CnAZ8J&$}T+<@ML!I z)E)6m(6@cR3mOWB;K{fEgt^*`DwX3^{glbPg)5B86~<>{UP0}2ikNEiR#Qz6s?1xp zrCe-8NL^JFZWktj?b9CPG#R_tW11bv8Y)t3CK=lEk%(1_JU5amEb8N3i)?gAr!r)c zK1<;_izCc3jTv0S+F;falC@0LY1{ywY+xH91z1+ObD4=yb;>e;hyV8CF**pwuQ1EL zOBT^MtbCk`YW=n^)Bj?$#=iZYl3VpPvh%FlxJsw?`@M_A){T?%g3QafMS}SKVaVTr@ge<2j^N z$p&5Xn^HVhLWo}h??^+OiP&mO^Q8Mtm=@f{6i?vDtj(??KPEIBa*m}dza+%s3p zy@Y-96BT>9dfIqfm9wVQCKc;>(<>!4vwM7#LH-nv!(($aZF>^$Sxk_VKyxVQ(AN_>2&QOD#x?%3R03r``DCIqhF=W1(vCE{Zn8y zzp?tuq6<%xn5O68?MK?M59fq@i3K&UE zy&4g1ZI0$(8!~Dab@CY02NT8~y!1%ZVv)GJ@0c-Ak}rw_WYDQ(zF?B9 z2PV2sqT7YK;+Bm^OQZV2+9Kevm2_LeYa6SRGQl9v21IBZuHfqPxcfg4A{n7=n})}< zup~ACVmztJU^N}O+*F4}?&ew&F0?l=D8t{qUk?R@cXc6a3%J+9;Bo+pu}ui+`j05e z3{jqS(bBNytwEfCS}MVAS6KXYir_(=1))B4oA8^2aWAQ(u0P3(I?d*d>JDJ+AGq`m zsCMW>PWx9vG2w+mS$)V(DAoJ41^Zo7JHNyL?r}w+eur@J-#Gu5Ha%i&LUbW%hT6QU zs#O0QVmvBQO?)i>{G0+egyiUtGpswy@D7{;0(k#z&VIA;ctcK1ebLq`!Qz1?HI2OXj2V-kTE#*Prlw zd)otH2(E~GYJWqxy}@_#)OXcoZS(B`iM?K@@aKMuqk9+#wj9AyVF@K7^<_|1d3sge zU=OwoApC#ruBCQ%X9+idFNEDc#Nck6<|2Kn6ad5@W+MJ>@SRrroWb}X&*uK{W|w1J z{=^Amzaq}r83}+SPYcOkd(iMO&viFX9q#z*+go`tG(ikt#xz7y5d(_7EbaqV6Qr^? zUI*qhiTeex&eSjDve+@$^{EH_%>`KF|I&H7IUxmtQW6dx-=GTqSI7MSt0Vsh@&8Zf z1vjpb$o=guVS<|4xBG1&@gp7>A~lttgAq`z`3cZ5L;um59ew}~>Z zOJ*r4XK}SI0cv89sX*0zn8dltK)%d3hb-ekDY>Qc$Xa{g7Tj*Hit83^?RbTiY%gML ztVnorE5vWY&2xVpOiR?_3_t>4N1K(Gz(nOmk#bIBB+X+jnusSsLmLzxhzQ4owg91~ z_mUjT(GZL&yr|

NIX(-@m(%4vK2R)}K9O;@&|ceXC%3zfA4 zX8e^_a27#z-ksh<-T1R#N}L1Blt4j^1n!V_D``fN+nba?puo9w9e~MX%D$1RY(grl zyCg%x!RAlp-Fmdq<=4zoD}DmiOB3gJl`8d~!;ZXzc1wrMpyX80A-W_PF!lYfj=;A? zy|e^$-Vfj1rS}Ob^}q=QBT8)FI2~`?(|;2BxR?!J#cQ(XN8x1|!VarMgwN;SF)mlh zrkmHu4@_~A?j5Ys0rpVal!9P+$S(J%RQ+a8Oa2RA>2;?JPdID(fvFP_8 zImHym9ZG@;_mhGN_gIOX)Lvv>n5GrErPKOph^M7F!?d7XJmm&1ULwLdNV_6h!P-7? z%u;cxpF;+_Yf|2n2DctJYGmvyORx5+V)iRVFVw32mDVa(00-ECxET~wTbQ))aD?i_ z9l`f6<>9$fFI?WFA0WMGPJtUAEx`tRE5S^_Rn__fJTLGL+sQXNAM%0=;`SiWfA;*K z^AT704J~H$=(395kI?aP?&CZzmtKvtdam?koo`n04t7*0!L%ozW zLwX%xb*}F*l-{MnOU-EzpXuCsq9VX6JnG{oYGeChV3+PyDJj!IVm*?v^LpE@6|Vfa z%RW+3FP#C)&G zknC6qM`iXjSzuAbzwxIDEpr6(7XB0ibE(H77gH0mj;-;e5X&7kD zy@hyj065Y@p5PGk1}4a;^twys&;b-Q4{x2sNP_=3Ry~*!_3MG(S-F)P^W_lH&3x(h zXl?l!8T>d!M1-+4kXTipfLB&)9q^G+**Js^9suFK1G51|()MiC5hloO&2+2-6h7|o zl+)nk()BNud>qEY_qN}k?VYT|2zSlYFV8JI0o+w;c`i|R*Ym7Y$(HerFN+j5ao*tA zO<)0k1MYsBNApLu-lWbJc##1p=h_l_lcWEZkX?|UJBZULty{9nvC5Y_@IvO-KhbKM zENG#zE#rr|L6lf&nj7-NPGIj-3G*~BnS!oT%F2c$;h<9RTh<+MkYQwfB=R&z>Qob}fTAW^Ae5F=Q#LsABI9%;+cX zV{mfn!~xvszrBKN{&V=P#B! zaj@zLkd`OrBXy!Hb?(iC>wPYajO9ldK`Aql;!Ka-q6J6H42au`B4Up)awjZV1$=sD z@CBe=5@nep9+*8r$u~;Go5$_(@`QS+;)fm5JtlZ1rSH`pTENg*U^3nrU-=dL_|q+fwp}0 zQ5!1`&GLFH*#NU47a>t>7!f_Go?ieO>TE2wvr#7^uoH<~{ z!wEvsGuDyQ_AL(>{mx;F)ydYH#1~&q z-Qe**!N|79dxd_n7ddUg)hL*31TcAExme)DU?)VBS=r9evSUZ_G(=gzS+ca9O&x;P z>91=30>Tmpy%?2e571lO`F$On315&2pP%~1$RlVTi!~B!%vR5#4MvJxv6QIlm$8OVo>LnwawXEh1#695{>i{$T&$sLBWzWU2{wGd`Hz) zjr&sda!V(Sr1$i*9}Bkfuldib5C3Q1lN_IGuPv8r6MgQd7ZsqASuTXlePP(1D`6UY zw_SD$ZrxeOP5`Q&Jgu`4>b2vZye~y|`Zdap9pyb-|&=cHcU!n1ciTq{KA9~v); z6FCbrCYd#_QnI{xR;7~XFkc#DTRlD^Iy9)-aKt*jdQa9}ouxwy!U2&=ZcS@OT3sua zpgluhawD%Ls-?A9g|AV(%vrHnbC!r-TX=Ag+JG>1Iz&+(zX=gpCk}}>-+X!lR!YCj zvSG*|+x?=2`BCJw8xJ4h)Auj;PN|RS!{IZXwS8PxJ{FZLd`SP9z30r!;083F6vW5p z#)mhzTftAoUTn_MbJ2OLTeqJH5uXKB)x823;W7%#ymKs7Y6`CSe({dHIM7bgw8yVm z%K%x|S3f6%#%%lySEykMwm|rp)%m-2YGOYj_o*H1GfDGg7e<_D(Vw?&!BzQ=vxLzQ zZ~L}&aZ4Sf`C-P(>lc3c0Sx0r5o=tWqtY5=zm*@BdS;PWuz&8`({7u&EnL|LXw5@- zC0I@(OB{<6|5YMsXn))9xi(xS)G}EzaR(TfE*Y}+gMy% zT?VpFSU~=b5@FwExiXHbIKQBfTz#9WsV7EqfUk<#c<*1VC@E2~*npl`#E-7`5bEXP zoN0=9Z0LF;pF24ZhZB1TaS2j}%Bfc-*Spr4f@R-R%^*AUi`wu16{uf@bVI)fmk!Xy zy$jJ;Y(WSSh9&NIc@}t`_rC@QAuPW!q(?|z~Oj^H-58QYWt*{ZbiqApNaSoUQ`w>=BR3TN+ zIuG|Kl(R}s;KZyHr6i2?<)y84_V-+V3{#JQnz0P&500wnFhsIqxsX}Xs0|=HjlH(M zCcmDKt1`K~Vow#g*)emO0 zyRgwwnznCeY-$LQxjxdL{0!!5Fq|@dYak=AWhjh7g3SRqz^G*DF8#3?^DQ6F z+XS9A0@d^BvT4q5&Ei;J9@l^y-^0HoI=m;7QcrS0+GL%rr4?R_9@{mZkSrdLPGepo zx`oZ=_;;~ZVG`#l&tpA-f;;2j@tTYTx>Fr66&dnBq8*6ow=)uA>-5Rnxma+$En1tK znwp=qUq##M8cgIEw>kPsmCP8Gtpk>@e3;qJ~+S@IhQC^Q! zWiD%#To`O%-a&1=PGYO!5Y|b|)XII1cCr`|S`56qR+vc)^qte{DDhxxpddf88kMF~ z?-Fwp+ux5 zWbolHT_o?Wl8GUH{8-)TZ5~$vf3N zx&it#-|dO9(<*i>e~qbwyrk?&^Mg~#TrzlN(as?qx;`Q^Xt)BPmag2W5{5JA&+}*- z>6W71l0MNA5FrIDCk5<6;%4Iu{nI7bIC~h{jDSdsBU0?AuN=&(>oaAgS282KBe=fB7@_DchJxp}_QHG)a z>vW%i7(MK9>I~q%a}iJD(A?0Ufu+~#gZgrqIh6mJ?fzZ;sJ61h+}Ix6U1ZJvjvf;S z(BH72$U{w%wO&bnCzbQ-h8t!y2?yD zjNyX~S&*vtoF5=T zy;^i@t4f#YLV)WC4Dk|y!x~f55-o|Nm&)2P&92P|T|C|x;7dtDtLgINwC@PPk)J5> zRYcQ_E5!)_5mpB`i+Cw0|7h}iJ7iNzr~Qj_2BTeT69=2_q$SK`ZSo4)J@D+@+dda) z-P7JS`-yi3chnBF)eaBHBle!aaszwf-kM;lezMYs^@Q0$TQ1fG^RCl|cb(qT+3E3Z zV{f)aP^B9o?!Z0zAdrvp_ZDNr_Jygg!3dv&cXI~-HAF|T&)nQIn;KQNhq#2fq_HfU z(wx=~QQEy=@c?b)PeC||!ASf`&K^DC2sTVh^r1`bY~plXn$8)4ul>SpG&;sX{d0vJ z{V~?4h)(RMHg~3DOLx`)O@S!VhI2^=WGA$YPKVzD5at(}e7x8HQUvUR@;W{YVW$at?mBThuFWE$A6ms-f=vgot!Q7sqq4Ok3 zPi8(Xvz$w##iHBq5lEtlY())n;pLI^X_XUz6rD@Hz@nhq;uVX}Y>-pJ>}|L(BBf$L zbofMy<;pfu3a|T)>CVRPbvzCINcI(ELg4MX46LjqS1FT%P32Mw}G~}NK}BKfMaEd^{H{_g5Ys2^E*3( zO>XM>FvI3t3M}v{tExUbQLfltA%|k;06SD;G=+pjO2h7p+zZzB`8x|74elVy_B38& zG>zoUt)Q+{HaSoJD3751=qn2YQpa9k@0zqJG-@+cy05h9i_G#R>A5|U6Kt*4Eix#KuTYr_Sq{|!~ zC3@Ad*j~(*GjB@ETU@vyvd_Eb+%^W+tJBu zGYd<;`ZaqdCUk_L7irF9HNz18XkV!Ig8~NxzTi4e@GVS_O1|(4`?#Kg5(YAqNyYJAEk(nNzG7OId%$IW}N^~Q+ zOHMCQ-sJr16#I4D=n{+RoE6i9;p-DCE&jx8@)h1>R_AOa5$xA%EjuE7FI{B z`jti0)hXnbDwYN8+0%38vzSd>tXevfDorw{yw$1GE9wavunXYp))^($`LNBkv$A$1 zQ}gx(+*k4$)mt*~>+05k#ivV)k#km-ePoFjBbt0a|Kqo)V zPeB}A>=N7aTWezsGsmAdih-%brH8*?YjaE^#il-;g#!Wd zaBxHRgnEi@G_CWelG`=Ue_U8)o|HsughG((`-A-)V zwr$(i#J0^hw)MuglZlgwolKmGZF7Q&Zq7OPx!?EP-@Ur~NAIq^cGX%{nl(sH-8~}# zsV8kksD_r`EA_TfwM0ki_EnP4iI$96;|m!4!O?b@uLHM@Xm5W4954waDEwi$<_@Y* z%*15im9Teptt5*q-@OJ@Gk?!-}Mg_1(x{FRXe zo>M+a?fWC1>Chl$TeB|Y>|EzN{FBz*9I%|Tf7#aw2H_32H3^NG# zFNiPi1`qN}7y=FsY^^DOX3yQa>fMr7fdk6wUiMlG$Tk_13I?TQSmRja{9W78mG*T;J!ZCUrt zzzC_U^R334@%}aVhc%&1BerWC8k}}kby7s{MC+bH{J~%$Ll4bWy_ySw;xxE zMXda5HlDGRKZ{whk5md0{N}6aN-(YSLPRVZqya!`pGvyG=zD}IYk;4`?oJA$pp+?w z1a9_XgFGo+g^D^6aG#&?n@`%rLUGB93FX;YMyrcp@iPgK?)0E4y>S#G6JCyq+;(d` zvMMg?mcSKCaUU*vdX!E1X>8YDRVP2N>PUGB|BSUxK8Q9q%!e7{j22ht{s;VoytOPE zd%B`fQQ0UOxrL(18I_unXHS`L-RHkLrTLro zk^bq9LI6Z2fk6k_F7cqWnRL*#LV^Da=VF3U|8Hp;eu=LuzC1H1P(Ty`Y(n4-KPWU% z-V_T9kej;cfTw{x`bpL5iMRezK|->~V3z_KZ{MVl%_%(JY@M904x-^6rEcil?HdWK zo5^~Y9;-fxr*qIvWG+IL)_cVGi`)(3Hr?1rwl5v1$>%F*5x{0SJ=ODe4>SbP8!Xq6 z-^)T>PSaM|lf>j|wAm}Bhxe-qHTW>gk+h`*{QfaUlcKxA?3``Pi&{AP?vEmAK-{~s zX<>wfP1y>HGpslAx}hXq*7Eo4@cNHXkD1p#Y{D2h<)PJjjMpF*W9A}Gthgg zbNfu<7OY-BbIY;Jt@pf55D4YLfGFVI;9+y<%myEyn$3X$ zTt!tGLUJSiY%m$)=8v*+wIC1tE%mIr(lMV&I9keGHO&8rAO2cA^|@R~1%1l@9^2-N zlS3d5;^})4YK)wly zpVB@o_u81ts_r7rr8H!b**!^=#`erkt6GV}M}7D^_OL9{k2+M_7x?sOKui^m){Q|z zOG0ye$1@x2!jWf&7s$R0&t}kzd;%}`Q*5i+T7QY)9W01LLEw^FCjA+6&{GQ|U7{te zR`SFsibM%~%pt5Ciq9LAnG&=6EqDjkn7}FV=lVSfQQO!ksSzQ-caN)pDS$w=2*1;C}$xygCRd9}y>6 zyC-eh@kT)8o6ORIz6vLz;lNj1!cb@I6uYNH3bt;3lX^odubSPca3+WnQ+k@9u?IHO zVd=H-ABj1czGd=`>IF2W4BX*bZl}V`z9HnSmgp{^q!hDfRi$sy#@jAS(P>PW?0yup z{H3wphl<>Jud2dIJn;aR`ZdYI0$9-Z1x>+I>KWh6DV*u<+KrOTd@YN+EFT+m`+76o zld3xr5~=NNo7e^mOR3TEPUjBz1U3yO{-8wQ4_N(Uh|Y6Yc|}pvbW`trP}4bHdk-x( zPC4khd7Y|_=EDj%XKTd{9nU=H)-``FQN0XxC!Vupr5V`;rs(CI5OxLjRzjYFD z*<_1;a*j+M)^Y>ST(uc(IQe)39LleiQ&V#aCEt@CC-GxP2vMJ7Ei+=c;tpME2Ip1` z;(jB?CE4EXKc(K)j(@3MIb&LPE0bpmc_$4QL@p{s(r{h;h!AOi4aK8&%>l_JW|~rT zF>vU(47pR7I?Di^4$fy0HexxZYy>b5C)_d42t!l-i_Z82xJT~P{7Vf5WKeRWGCA|N z@G-o0^rz48K)0c}+I`Y+2;Mf5lKae}%(M1qtt}3E~8Ef|B1p`F=fVWlpazjz3zN^f1v66j4U?XCRbFP%4M2`_DJ93jq+jiP))O0g{m zzqo#D3<$vER%$cl>3#0Xvf`qex?e&<3qy4#CS2j?{QH&1MrUUPx02zbDe^NmiMW^W z><9R}F!-zt|1U^@lB1^tVJTH>NP^bAy*;iXbQcCj`TG@e3Ee*9~s1BaP8IW!Ap@ zvNIsDvL~K1zm7yS&FB{{;=7*60!Umd5atl0dBWBTh<1{g!cA5_OQ)tm9pPDsUnfd~ zVkMx=IErm8H0;%ZZI8NPuZ_I61lu}qUJA1gZ`>89 zc8Az$UIwe?hE-;TQAR9I*I&q_+TkFS6bZO?CU1=T7{V3YA$$SJjSa<(lm?R#@(W1_ zenwQ-u=-0YL%LA^@t)m(TFnQ`1xItO(s)HiLR26nI1uO;Yy47DRV+zMz!5wdP>l5Z zB%HRB`czhnP+XKw>mHV+bW=m`{uF!~Ic~NtCZIq`?MN!>k?ii@V&Z=x5&fT-_`d`2 zzm%&_4padA-)dr=PiD#WS9A>q&goL9w{80)BZK}&%RFc4V!-|_Dl%L^ob!EQzJmcp zn7)E*-#Z~v8~-AmG01;NMGZnz5b&9Qs2_eJatsLcKkxY57eN5R#e(Vpw2Aw$k+q{> zl7_@MQCDLE=VEgL>fIRZZ3-~xG0Xdzj)!$3+(wG&B;O;E0@ypY;T$7shY&3edw z$-$;$bk~ax`a#pV{E~A9aD^frd{>@}^Mxg(*4DTo0rULoj;P+>pjzg%SzZ}#yQms| z3cJ2&Te<$HlbPT z8BJ5)>;bds8_DN*c;XJzidCt)S#>+H`O%D16l^c{>^XPg$%5d9T zzdia1*>sJDsa$j2ZH!zpa1Dm<2wn8)2vHY%G>Vo)r%G-JDpOY20~82@*mKO$C=e0| z3mGcP^EWKN*=zRE!iRd@>7`cmH{A#j9QO$j@CuU4XTY+I0n)*;OKz@I8gBN38{?d@ z-z7NB-AdKJ$7%fJiNsN?DO=|&=|JbCboHklwiNfVi;;G>;P8uQGV?a1T`|k}{JbBT z^$o2$lyp*51lk?gRIg(aoCxtc)wE~2Dv&2Rk6^R9IqE(p8`QNPQ9Yj33IFH;0|ZzU$}RXSd;_tM_B!qji!;4;mni4W`HSmMIWv9$BJ`@wO|T>LJ;)RK z0kZ8LIaiP=BQdl}a_>Y0l(h-D*h*$UL>^Sgl?9Z|foNUBX~MPSyjj--m;K*7m&MUwT%W%eJCl>0vVxQj>S1TpMO9KFYajFQnx z$OzTH1N!TcBVm{_f8W_qnPx~t^P7bdQFk9uvaLn*_PjHqU0!;J2LgoFs^=%3Qt*a? z0QV$e<-9MhRx%x&bAPI9_|`%@uePt%g`yZ$_m#as1^AO=*qi(FwekKX3W<}x`eHMr*Dkt}|5r)Uq{vd#7am2^garF6Ua~dvAxdOdXb6on* zLGVvtQU0QODPK8^|JRt|+dm;EwqGBk|H}w~|78TA|58X;J2E+#y8-Q8C{cjQAD~G8 zcizkwBlN^u!uW`%@RqzIXYmkq-YaQ6=WROBUQ~9urEoV#>2f#YQUwi9XC+jsk&z^m z!Yu`d4wZxZSz0QrI}e9!1xhcJ_{~G}$J#tl@CNGdB48HS)6^u}>&otoFJ;cdZ*?uO zz54XysZr?VI{y>a=-agf%0MGFz+<-^l4~TvB(|a4ls+64lRYO=L`o+)8q!dTBb>6b zWWR)5C~}u2C;7mcKxycPTu~0FBy(zgdL6syBbjVT;jL&gbKb6G^R7HfI1VT=tv_Ho zS)qD~;f195Y>X!v2NaoV-{qfSC%swnKFWQ#B3N>~iFTUR1ayp<|*;&&{^5(9fw=(6acXr!CX|x$zIkrGGP3!4aRfugv z>9`d9=66wR2W4V6;uI|C1WU=$v9fZPH&_KC$~gIiqpx;Mai*E$@`7T-fG=rPU1@pK zs(cphT#-w0a))OwOP3ZPt?zj*hwVBupv4Z3{|-VAzP4=<1rpAe9d*THuKht`g%7??qmAQH&Fa`I|*L; zgLsfzL*8JHyWPUs`_2eC9f;L^g4#X$S5ExqMZ5N3fB^^qnCTccv-t&!3}JF!Ps}w= z#;}A8J=l|RTaZBVMVeVYc}Oz63@iP3B>8KjuF=k8o1xLuC7!$G(n>f*$}fMZWKYL4 zJA2Z0>OYdfWbBUj4dqw&%9ee6jt1nTrhEbm2o-w!5)1M%jj;#-ID|H|buU7`luNl4_NI5(tQitXk5TuZn3Yd{A??qrLTKZ3R`1Z+ zOts`2GbMqeI9D)~Pvlnv)^5A-e{bONKInBH@Zop{7Y+JCyi_pK3Ld|4^oEE%%qwi- z-&M`x;A_^VCDV6ko_L507B4`iSMEYPA&BY$203qOit8vw;nV z{_b3?bCO~C8Zg)#xx^VpCTv0ev<(T;uPr0>b*%9<*!OAO6wjzvdILbr<(2$uyc{Gu zBS6*~$=$U09VqGU@WD0;Z-lfJ%XlFVb?zh0s3${RGk{JXte!2Gt6-O(^RCfx1i}u` zFM&~0eB@MGo9hL>ZQ0Hszpr9~P}Ohzs7Ym0*Od_CF1KqK+4mu0GYMy}K*=};Yj!7- z=C>Yqz*Tj?$nr8N|1qcmw%wWRM2FTRctZkR>z%4{6iodDu2KZET1a81zkd6uVJ!)k z&uhn6>Owq?nwCj*K-p@yPa6>ELE#9{xDH&~?GEKQeYWhOgiDk~^(dcf|lJ@@v1Y z9-VML%S(p={KGVWDAx~8N7qcSXBefAVdqyQ3kmRatHGR; z!=ItSqi%!{!vuc0-C^^Ld7G2Y*X0j#cK?k6|KA(m|7?l>5OPcvC(yrN)Sc;T!!&C9 z&CBWu`X5oJE5Ne%FE{x}YyuNjrjvD;G z8|l~S+(8lkd3KK0AYS~}ja|UGbP&?E*-g4Jkbln$o2dzp3asD04ch{ZZ$U|cH}YJl zz>5buA^?;h`qCqJ!1PZxzOHN-8f0Q&V&T9JNq5`TC8o538@fCLnWf(;aMltDz7g?*ShG_v5XO!wp|0p<_kA z$EOHFh(Hq7^*$;$<(`WtHIm1&tEYCRy$I9AEPlu`&9gAY60(4a4gR}y#c#nUT?|3_h`dj&Ko?i-an$e zj(|UWOaTd>hrrK{TPVgSZxRH68M08v+>XHe)ItD@Q!iHhU^&^%w-1}0erW8tUYuuN z@P`45`fovR$l-~4HgB`mD!WCi0|j#f_RtAflxWaCQ3V*GDi2T5=t(w_?0G4KAe%$q0^r$)h{fQKti{?bsfEH>#&)~q z!*yeK7~&3&_TiIcGK|}cyBp+VA7@SBg+}i7z3Q>4nx0wme1B}3fbQQ)<>e11L;>D1 zh7u|7$$&&^-8bZL3IFM!J{h^7diU z`U=z-1eMGFdx=0_n{IW!B{CZc0zli6Yi*FN+=fXdiK1(ad3VeL*$~d2{MNopK+f>7 zU^dY)^Z3jz$?=^j#KJqjiDhH5@67Qi#9MnJ(dW*n1pJPPi5ALag52{T)vXEh4PWoj zc2T`?do7lU`lEQ;?gkCL$%ON-0VmjdK5gwmz77XL)f*-py*0avCX3nHp#Z>rboMaA zTYK8;Z&tc33nMMD8n&U!@OH5NsoJ{7WecsBv$HpMXvvQHWo*s*xz3{5P6S#)Z%7?Ga!qINL; z@cHqp&G+;K*dQeW_Oed*Nvrdg39X+_M$Z2B=6?oM0Q>1)YL~cG)&TZj)IV4-wn3dl z#JqX>@p@~vgn9~dAU&;O`bqOe9T-J(exSD{AR%iLxk_CWQVE}&z)#on|1$i|%G=aj z*!tU0-%w8py@G8hH?-J~kS}iJdmJ>xXrI#psQVwzb9v>_8U-7$gf`Q9-o|-DjdTsA zF1=ah8F|6un?)l@ItxJGQt5|^R0j!E^MGIM51TZAq=H7DRygwkHA%}9iSQgJF%hQRb5qz6ZR{h|e}vCUo>Rtr^ZXcm?&g4z(3b%uJzD|Iq;adRWrD>)NbX(o#od zpd=t$x({Kf9an;45V&(ds&9-xO`QCk(c|7B{KP1|1@+?5bx_8mYaSMGS_H$8L!N?BGSQj%ry_3A z*>`OyRBT0FZu$@abBPsxUJ_MeQ}pF~peZ8Wi{vp$c_peEzelouv;E6QgP(<{uUY-b zKRfa(O1log(UJ_LY+QnqY`$z%k z$HRXWNbL)QYf&+hc3DLJVgEUxrL`ken`d1XtuhXP&=mo&4YZcdYb}XlDj~#Bq(qbV zYia(TXZvfFHdJ^{Bk{eg0RIV#v-YcUb#O5!!WG6pWR~uUO!^axAjX6b_VrauIiB3g z8FL0LabqB7LH3>FPg;=88J<%`K#q$3va%XlMXjZy6i1li>=~^Rl^1sycZgq~FvEF$ z9to=XYE={bpbB%02E_B)T{~o*1Vni zqSHS$<2}&D;-3cbpPvbDyb9K>LAv9yVUY{fQuGDTD2DQgR*2 zO!~#&y7SCC%c_zmQjsfkT3UL{rIN+;b7zqOp;6LnPF6dGX|%v@vB*)ZCf~12h4R;j z4q{ysDs*hjzfsrMX#L+4Fj{9AP{t#PnIb{Db8oW9t5DC%Gi719xKc5i|a?m?89*QJ0C~4AV(|yVcT9%slN-EHnN9Wlcs3%@OaJ z!CAVR0R_HHaDN`9Hr+nYjeebfq`?Y(6!r>s)Ew8X=02*`?AxlZn|hdw?TJ|cuTEZ5 zJ2`%uX2Liq=PwnUtE}&zs4;Gr`0@$>vpLow{StV@epnicaN%kVnovpVfFINGvrjiF zBG~E1WktDZt_H!xjS_Ssp*$J7;6-K*Z3)0y+XEeCehM$rk?wFZR&JU9HB!ZA8va=4VEtgI6=J% zWV0u*SJzZ>gr?p+D5q{5aeTS;$!&=%5d*Ddu71l0XQ-{YD5OlH;p=Rkn9lZ!0Yhjh z4fw6od@K~{33-D;ZN}Z{FMk7kJtA@A-u#HC86NGf{h4~Bt5Xs+uW6yR%&Pwp{+)Xkq0_nvzDW>6!`ZY8ml?lzV z>-rpg@v-aH1YZIE=0vKo|2n`9GyJ5G) zC*HS(u7rDSpnsxwDcpv7CF>mxmG*RsQz>MeRGZ*6PNDl+K{u*eqIKOy<)JEN-*6Ba zPk-jVT18rBz|SNbI53=V{KX%L6gtJo=GNx7i~LjOm3Kn^L17>)Er3ejV-8*V!wbbX zdygXTjdgN(<|ZZzt&wX0Fx4B1pjs7nxV!oAJouQohjC%N=M(^)N@-J(y{B>1xSy$B zuakOi9lbn^SGGd8#XhXt444|YrL@?iR_kWiVj1oN41U$R~x{AS+U< z(G%zFy=@auWwBO8mIlx_WRgqmCit>vY4*%Lp&AK6&S9#(1x*t|&VHjkeQDlTwp$({ z={a{HmsXI%WujwT`P))ryaqx?)>}Jyj$JKHVZ$G=2|thxHtPkGhPs+b8wU)}iHxEh zGa@FQNo$`}Lf@eQDPPlz^KjhaX?iGlP`7XQeQyoL-e%B+BK^yrlZb?jmF^;&WaZx7 z-J^oS{|*Uz?}4_EsJ+oke_*mHF=4(cFW7v(GO zp@Ho3rxy4~8n;qoE;dO+6KCRvqb>@vZhahGbj-iTLuIVu*$9eW_xH%xKGrZsWU) z^lZ==E}{6=6Z>YAqDseh6E-w2wT~LBk{i5`87}Aw!?|8UNhEPA(R6e63_T=LDg7y? z@;sV^GpTr%6ZAYiOJf5N<*(zvlVZ59J#M7;9o(ae|E(}KkC z+?)VFa(!!=(;2&s==4wxjyJ{q4=PIxjN?@NmW7ic6CgRI9S0&RJF!8>J|f-#1# z!yxV0vBtNpL;^LDHB)H8WX>XGT8+qQ`lU_!rCrdK=I6fb#=sK`aGS|z#+Ye7k~XjJ#?B5h z4#prbOLH)VDReIAwb{rV3$n8Ahyo@u681(k`IDU*hxK~YHSlL)-8sQB6e%G9=d!Fs zEag*2cT1b4!2mIqu!E9RCrzR(c_jSzQPzmyoIA@a_^Qu|Xn=(XlQqKe1S`qQ7*Un@ z@o`=vTOmf!CdSv~WIAhVTkfbTI?J312+aMcj{Ojk%wnA-2%^gSiOw(U;p<*PoCBvT zW6N|Eu}F0_NpE5|MIlCAuhZX=Vf3p%)Y{JUL-&M?t$n^_%xG`N_%W2G_LaZ-89v{W;5rRnmp)0(#^ z%$HMRZ5A#+MDDFde+eCkVKmq1nuIPFI7*V7d#(1P>)qlCzq=Hd@}x-siy;flxRL%~ zFfRmib@zwU%hg3gczN!yB?IM@hI#piOf3UB<=9gat?5;?QXbk!^1)G6cI1wwQ>bs< zbVj;Xh6LUM)zQ&`!VbedZJ|I??U?*z6SCQoZa%L$8c!}64O%*$$GwyrXF1oiEl@{{ zfN1l>Aw<7jxaF_7g-`X){{qOfwd1I4vKvy#uZ8{?_^36ssGED(k~be3nDo>d9d_<62^DQW4^GUWdI{yY zyRSK=!tCJIv*6+$8?^o5l=U@{D-c^42nMSR*HqDwG&9!6YtmMw5D2|M=$tTWHQ2kmR43pE2bgAX zp1^?Zk_kT}4GGh4H+)lm;qenG-8skA2te4bcy!$5!+rf}^UCz|6Dh!H>uK&<>RDz9 z5N@d6L4;Uwr`5#|{Y%e37%DDw#YVQoq|o1j8Z+W3y=LD*w~WqQns+IgVcYH)i0M+R6yce zL5_nUy9pd+ZN7VXwFN>sXxyOICL!|Ii;`5~iebXK#6sY7p_;}0ot9gLgtT85@Wu0L zofqXJF?qCL2DFJBXj%cAaEe+*-DK@9G0R4HOWM_5M^`=k!rVq3FJ#j ztc3&uLuU-+8_zE=q1`x zF3L`MA;I@{=uUAwGA1<=!4{xJI4fIyh>#;VD=H-Ej<wgPpQtv>_TL)bU4Qc7BM> z9qYXkz_HP{H@NUYI1;ldr|HC?+ac!zxjv%{Q8P4~V?t!8!d?&}o5fiKw2NPi#Nap1 zuP6wrS|atiiJZPvC200gLx^_4A_~mh9?$|=Mkk!ut<~n9my1Utcg#(g5J#$8=d!o1E>=kdps zhE7cT5rQ(iaEM=o?n47IyH1w6=0vpVf8Zi4DbI3}k2ey6cDdS?w6y(^Gi=co@@-VJ z0mN8#Gp zWd1}U)+NIbzgmzjTi)8jM%Rf#9Uy}w06kP~ZSUPbFLAo<*Y2R%gbS`$p3o`Ow5h6u#UZ74;~dRpw)IEhMO7=VMar+gWbc|~-}D4O2qfn8fMF6RvL$Up>$7n{ zqA0~Gx6(B`D9|G*PAzIP<^?lozKWLXCoGRIL@wKBV3c6+lR?iBux`R;=z@5t=}G~Q z*L26)NT6XZ`i5;Ea{y|*ogRIpx!sIw%1_MidAdx|^(ij1~#-U(E=sS1xifjT#JhPXTQR|63H zL7!F#h1XB5Raz9Eg^22 z7ulalim8Nr=?n;p9CVpOu@%SCBkA0gog-kv6u5y+id>F1XqS8#(G?f6L$_Wbhul?U zfzL-Kb4uVw%*QNFJ*XiAKF~%JH93RQTxM%9MA+mb6H+FlxFzk|{plP}mC2EZoQvC6 zUq`PxtXxyC%H4_eh52nl2%*XwW+-P*y~G_Uf1*WRZV5;t(iZ-iymF#OUy>D$mEw?F zW|7Dase!gsrq*sDb4!2nNGE^ewPyY6KwY=T7lr2v5)%wpn{}xV2c=hp^5+6zzZphz zeJZ5D+P^E7%L(&mu5fR1~VvTo-4baZxDj=cV$jmu5jV6tT7 zl<6n6J02jX)pkKqn=||Q>Y;m)!3pwxS`R9uLOV|u@)q74c?KI7?3bm!K-sznw=IXb zn)9R0O^JR3TCM(r@aE|n;n2&j#R@Bm{O`e8CjO}RcB`=?1cL{ z%VKt1JqSp~8|{fkw+v#vRGxNii>?!Y7_4J`fzE1P#pAj&ZWfxwc2zpt;Cq@XR}RkJ z?Gf#0o*h)z!jP{?qC-*5_b;KSWWN;_&6xAQ zhnoY;A?bb!uI~m*Mvj{~_A2F!tP5*GVKsSQh>@%dvASP*hCPm+m+UirAO0YPYO^+T zs`wS3qRJ!Lxe}eUSdm9_$}gK)nfgZ^?uk&J{?}TCy&jTK6txFKZbhThwjXX z9*w~)RuhWr48k4>w`=FjQiBfcE8Syl*V6;xil)Q(emuA4e1NDxNu*T1M4_47RjU?g z-O_;_5kF@bQBG}{V;dtq5?ZC9qJWyC(VhWHkI>X-DWWoT3MI)#pr}=(Ra;UIt@8)~ zOyS-MDXP%VR)!w@9vwXqQ+ zzeNU|uF>NLyntN>b{mc+6bk#Q$Vr2nzw|@KWkWcBoRMIVIk~|PaC5rFWbqBakc%h; zTsUfYwhlJ?2(#%UiHo~ccBYR{4ySm^E${m>cpH0vZM6|^n+{z&K9*R5ClAzLGF0RX z;=g#tvdG}M(ZtMlaM3Y^*fwQ_1)+-=B>I>n`kW*jq-Kiyjv%&SEh4?N+amHrE|w;z zQ*Am@y>SVOhrH+R1ifoy4sBTg2hT*;hdWpX= z8mlQaN9y!cIC?b`kQJB$!_Y}EF&aEcTa!aB^XM@e-Za0oa3h+J?dG|w`x$-<*R;c+ z4%f~q)r5fTd5odQp*L-Um=5PM%hW{ZIb)$mc8$kwz?qwXEk?h}4Dx>iXhkVF{q#cq z9nH`nVi&@&XS_n$I*8Z^<`EOIBh!dxfS%AN#eU>^i~UCLQLM`^Vld?pi!qZ(bx-br z>f2E61?wAA&Z5sBQ9OLlt((s?+!nX{^?cfh31Ii zxc_sHV1ssRSbtAogKldS@H7ndT=|ULi$$~#avO!br#7eJC%N*Rd~=&ShHe-&w`a?u z;zvdq<0z5zZ16k;fZ=-;7k;BoSMJcmM?Q|sAA-I8(XReR%82zk9Jiq?XN1ELx z0ec&ox(87irBx!=O|iKrhC?CF1;Aa!cqzcY>CUsZNp+uT7%8E zp@J@Y(E5q_xWO&ijCZk6ul#tY^kO~XLOEn>KrVT!e(c zPGpxp5kjJTWi_}+Z5Gq~DV=X0lV?AjcYoS_D>5Q=^@nF zCMRYij@oy_#Wy?#P?N$jhzwk3gLf|WN5pz*_e5K6f_MQT)S#M=+@`hD!-!PX6Qu&$ z5!d>4$KJ5J`t1e@ta$dj?LqTnt=&kkdG#X#67T$btrxdsX=S< z4n{&b?SGW<4zN*mW`k(?K8c*1G@eHkasqs_A&>rUZ-y$GC69s zj-+6X4oQ?`x-_A&;6?zh_WW86Md)$RPN_oI45;D^rC(obrfhe`fS09Ws7!0b{Rx4( zV2)OkaMgmzTlQnUunOfAclnyuDzy8TlV{eQ?g;WX)wY+@#;5j`4C~u0L2`qAh`){z zzSeeoy91MMfHg4@<%qIUy3LNHu~euS=|(%{!&7u=9YqC@7f@o~KoT?((Z_J^RVY+5 zL+RvqqXPM_VOm1?Oll-bU(Gx5zug}QYm-NawBUntxG?81M)YO6Q0oy(^- zagz%(@$0ip@SC);Ev8AtnbfjXdX%9RAg|$jTEw9e10KukHFP9JU>{}m6kCi&w*TPa zJEF+bv)<5SFj$d;TSDCz`4HWJUW%Nw-Xbjh)sLiqGPfyNn2}WjaYEaeA+%60lPjDV zK8abjRz~A57~9!|$Aei8+i0g*k#-of2yTwLd~mw*W?n%Pad)`v-w7`BfXn_akkOWp z$L>`-2zdKY%-HS?{G~S*E_$9L=?lAMZ912_WS^$WToX>H7}90$(vNn%dk|c%;nF`j z4xz?iSyifF6XSUuRN>Wzn_;_{d++zjo`Lm62yYjL`AtJMIV>(3^v3-GmG|X3eaM*)OgG|I!40G=alxI2{`1g2_I%+MqWbrg2B!E;Sti z1t>uxOg_Ac)ce+DM#E1wxx;fBv1|^}O-Hwj=MDSDh-I8wGa!!Bx7$Q0HGw;oTD}La z6N6?>a5Pb>O{_G?bLYk{lNXid!js1^GjLxSklFpMKJ|EK=}xa(rmj!-z}pL`7xgrD z?;O(&?KvrTBjb(JLto$Ld64#;)D7F43RvC&9JV|ccM}5AxS$icU}+pR_xzCeU?Naw zRFcNN@d@n&C!uR;+jr%(9^3VM>^kw??+nX#Ee}O`9Q6Q*i_Bj@2gO+o`)9RbL=O7DsZu7?(COq-jH3F$jZu#tfdcYz%``A)5MM0i<<0 zENbi@k;jet!Q5{KhwS8T{q;^12GTaCgs{ClUB<)mOd`h?CZtprw3;|EfQr~K^AX;m zQG&p_Uk76CYw$K{t5%fuCjxH5CxCCYKM3cbMCNZSu&3Y%tjmtDS*8=%Q|{o~up$-j zB$J!E$Hw2Unp}H-T&xF0Ja;M5PWLJt_DiGTDspIu++5&8 z3hG_BxyN!blCz$Ii-0LuyoLN7@JRnQ`sfQK`$U7qf{ep#6`-t;lKjQ*(i?H4zXgP0OFD~OfYX(ez zUj~-e=o&ac`=6rxZTjtFy&5CCIfe!OlPNjR2j=fM&^W^nqUoTTM2f)idU6u*$HbSF zYMy2#MQ7T(kY#Sab#Na-9q@ikJ63MO#m6$2WLwxffydF0R@L8?uFm{tT2{$En2M`R zN{X|eFnXU?$Dgn((;wf)VQT~W5>B;KhX=TCQSK^OcbHbsv_#2jEJz=w!mOVrLC$rb z3$KgfRshz6)cMBsLQjM;@ClZCI65m9APf7#hE3iZxO*puP0<^K4|pMXVcb>|e-_GM zcuM-bgQGbj+BJJYOP$OoA%02;7A!z!OGjSDX}W=B1;>q=aqEsw`9EBJV{oNSw{~pX znb`Kkwr$(aL_4-^+qP}nb|#t_^W}NoQ{Q{)RPDNd?7!VtuhpwB6gx2gVqE8xKSoR4 zAC zp_4#{Ysh})wfD`zg)Q;SDLx3PPY3B~aZ9Wg?0L}PkXtb^0(2vTm8yBO3S9mCFd(Iy zW*lNL5K}F-_dKh)x2hifT(yX|sviSnwNPy46AUv~Al)o{3{zJSbLQU$Bef!kZ%v%& zVnek=4e&ktmm~UchMngl`mSPryFzuXk~@fd=oD*%U%wcNT210#1MiIO<_G#0W3)%s z2XZ_CVX=1`1HyJz_nI-fhhfz~&>$)2jdmG5x7a;81k4!S?8oQmxF(}WXln!WrM7e& zYgEw5tAf*#aY{zpYwQBd!+4mg!At62C9R(V`z@zXyR$J1iZTb{@^Q80VXyN&-?Yl4 zW93fKv(lH?dyAvh#@-icf{rqlJ3T+@=^c>sXZkFR0GTZNKUMV*Z_4j|UcT8>NKfS= z92pneMzx`1iomvTXB&4&#nFKpbg)0I5A$nN;4L@6u=0-uO4yN|nsm}CWc`ik5Y4Q> z9SYPWuGq8Js-eCK>lziwed!y}bwZi&nAXnw)V2im+u=iQ6#e)c;|{SWBrV!No6XEd z9Q-`xfaJl44IvNFO25`7DGz;ZI6~{?+h#RTev8rph7FV?oNG01SdwN~CN1r7rj6G% z$xa~DW`KeIH8BNpmRLS-V@2fFoY$2W+DFXl{^O8r{VmN-9&y#oEy>Q9B}A0vyb9-! zu`$j*w_h$-gazed!rU>1}-Q=)~M!080~P5~_vkzP!Y#*YIt9AB{seR|rM;2W=DeHvcr$dB!U9l`=G=B{ge-HJn{V@R{ zYwWcHzrphTJ;qUtCR#3@D=c=ilm%z165D36%T{4=WmxDb*gJY@BdwJUY&SZb&fjzO z0GhwO>s}cvO174!qBk&#e`**bcEBZXQF_D@o|*5+Q>Tyi6eMGYg)6qgENud!aiwtvaMU{md!q_D|l>;}P~kRkyI;58l}H zNPptfEA;WRZi>BQzeerSeuv|*$~{!OhTSUn35~VfJtVj`_m+DH*Iyy)DFN-t2axns z!hWc%-dbVfGkil75+(=8}xR+erQnyMdHN=`l?w zJ&Unld;1Z7YjX~o@_>wlx(>(7b&Pq?N|=A|yfjt$86#Hc9K9OG zp?pgtky|sUjjl?vN3K&@wgEs;+qb1o#{*qG{8`cdBSt4}6aCaBgyt3jCSlv<_YP;; z4==Pw^T5#NE((bcZpF{RAws{0IR3(?mh(d>aB!{~*2YDm?mra=>rg)@JVW_Y=2~9b zZnVh#_=UUq7!3ciRU;NF;3BFL(H(M{=FqIJNfvYV%$f&dF!N9jBkOj| zFscY^dI{r<9aPDeEa$8>qDEv{Mp%RQ!Yk1N4j)n}w4h=xdDA#6B1D9K<2dI^$-uFK+Rd2{ zSovUVmE;OUu|l$+IU9!h5?`&zhw#2h-Zt?O>dpJyrt}Cftsf8JE!vPj3@pmc1fA_O zu6H6Nc3ZP(Qg|p)AwxsgN!MpWbQ^{map1oRB^q&HwiDsj$6Em)zz=cJI@=8?x2o5s zL2(V@jKmoBM~u3lhCNL8$o>*j1p#QMjahazGGB=it`UxVa|}YTktPD&_qW|7&~3$Q zOHmuQnKM44YL3i~ChMVfSNq3pnpd3rJQ<9QQT$+fc`gFszls8sAqC8vcLIAA-MB&~ zo6bZgpTlgx>~R2{WV2IRRQ2_*Ev%2-{nEC40?U~39p%I`8ZuTTI34G}oPzd#uWu*^ z0j5LX=17>d%a>R{dD*7`_Pmd4%bakFle0Zofo@M;97xs_DjI|hJd_t@VP1@YH^T$&huzjM-i zohc-+=x!o0i?oIYox{wyA+h8jK)mtyRqBJ4_(HqPF$B}UnZ_0ABMd&$Nc`0UAH7r; zj`4**-?@Y`Tph&w6ywy?7nRTkb$yd18c&Ac4xb11j(DA)fj>VHl;|(h2*f(#>)KUQ z@)e?){iFlD_?YQMY#}hjp0l(2YntKzX-EsoE}zoQ7tX53pV1rMW%oo&-<37^st<9K z{4Sf!t4gII(JPIw{7W)FsIoQ@YE9adlsk}zIy3f`HLDu!K;5q{qmkkV7?yG@hj8Z$ zGxnm{kg(+=uct{jEkz#l%5k#-%bFc!rH<(8LFjH7>p!J(sArjI+@mz zk((|kiZph8!FI;~NGzo3`D#OEThLm{D(T(uLENC#{#odgKjfv$1yn-m=pz#NfQpUOtBI8+|IasqKQd zp2Y%$3wv-0R&)JM%e*yrLG3T)`%B*z>tZKB-x^~|q=GcO{XDYp2#sKR?i~jYMJRjD znD%)k6uf82?>V~U?z$nL_QkeDya~Z~56ln@$Dn+p5vgCX!2b@>NqHp?SZ!4_sgvnM z%1U|Z!nw9Z=V2ACnr~Eg%Vt<{FZ0ZZH>?0)xNRVaZNIpOu#S3+3T9G6?%wQ=RQE|; z4Y5me>_GWGvX?M-zom-My_(CQr2Tqd{9Qp?@b~WW>b9ZJVfs6>iN>U(LRa@^7!=t_ z19A&*aO$n9aY=PBTTV74Cdy|#u|nq27bImn)^uTha#19~nNjgF^6)L1TSf9} zVM;av_w^vDqu6vKt`IU%rXLpCD2&9!2S6@l=vdjopm$8#;0O(<+w@-%?ayKspQmJl zgCF&gxPN9!S4;ikwCfWY(kAUP`q>jgaI_rQlxD z{6#wVIgIvqM_mKIQ-=5dks&AY@85OX zPxYGRjlJ=X{rk*muJ?6YZVo(^imB6vck`_?gyr&R1v30-0n)zXUIx5%hwxjx%nG?2 z#)7|~eq9X!_IqJ(F+WrvKjndU2H)xV_!Jsr%VvAaS3-kdB=%6^vxLUcEe(gNbD$*wT;CT477aP1uYhv~-0aOe@GK z(2Zhbob!cwf;m-7DgpJ_^rR-_r5SjW;};L8Cfa6}N*4nlT%tZ%6(u<(1g2Ii=N0xX zR@sU_j+w>hYZVHUBR;89RVHDWL384>h-C0lheQy3F7G9Mgr7jH6$Cys2?qyBpb6z z-)2^eiWRhqWfPYp!r82jK-d;)wD=xxtsB&$s~1hWVY8j`$#IvWqBCriZX#JMuQ?jYK&Kl$J=AV zqnF}XY({w@CQ}V7YE3LgTwh4OhM%{!k5gzKIX`dcvIOwB77JwSbN%Hg{WLsS$+?2e zZp0R$S69Lve^zaIP)LL$iG?f;L!xGOAyb|&L&cT?B}#HPqQFx}W~0#E5Kpa#5)p(l z3D$n>yaJ5}uR}p;1uYuYLW&*!x@_X#NrnR#v|cCb}hA z1$lN-TvnQpYZ76#i}t82#wK~@CaUXba;dH++f4Z2iaqq5b~z(Q+8X_sCqsW$RV&Gz z1+yR_RyRW@Z7>c#6_D~R0m(f6HCF#5CvH;)izLA8LW1SyOg6p4W+_EFhJCob!G?{aL=itfXnyck0v+v1BYqsSBLL4r&{~z9FLUFYi9QUC@nzGg=z!B+qtt+Yh(TbhU#r+LwJ^Y0|qQT0eVPW5uD zF%}+xbVFm}qWYBt7EMBSRD-qVHxsbjV0sk8%Em=uZh%fqo${c$(BfXi@2AXsY8dj3 ztEShuBCt&XZ}0Y5xSeXVl<7*ngn#~6&H^4Z7;1iX$z@2i0O`84!5W=@-XheEfg=DN zj#SlyYJ6{Z7eCTYIXrg44p}8R);g;GKwK`&u8!xD?Ymvy8Q-AY$!-`vy02&FHg>sr z5o}(R0JF7XQq$|MNE#!v4_LV(`BWSlHJWHA#cH{nupSxo!-xM=?98X0-AlNF4WKm+ ztmxG3v0%nvRSQueb6%Kl%d8Zca|Dpsp~zrkl=5M~URkKTpTDs36;7JA>5aO!usyGtfCC(~y-w$f`mR>oYf( zkF)Ex9IDB#{R;M;kl9P7K&ob7QR$RVx(YqY;BzZOnZYBnh7O9U|F9uYTp;6qL?^u}2w!MJKBv46 zABkE>tIqSV3^2M~1QW=5O2Bn_t=cL({DE-pl*l|ae#oh-%qk;%$@wF9@%X{1kzPMe z+vNlGOEZ;WqPk;?Tv4}Yh!61E;4P})OuNj6lMOsl9>3D5*i*6MD?1t~5tf<*eE5QG0v+3XEa24RQu=Qhx9Q@UM4^N1Auk!EOIKl=j~3Hg)WD|Xp@wd% z`LS957HxF86L^C!BP@WhRX9GTcK|^@%8m%W;JXQWpxa4dCXGOi`7+* z_i0#4`%d{~3JRq?d=^GTd0W^&Ws5?JKEi?Dc9M;))rPLd&YJ-0{;2)#Dj9Ud5Fxku z;jA4~(y0Q|F3-<1DLO9!cAO{ll*ab?%@g}4d#qm!iR2pb!D0)~2(L$UNNNo+9rA^& z{slD$)A z%~K}LGWA;Qd`bXkcD6jtKwF?2YX?s>-N z93{SzC=$CRCD(T9crW955mqetT50^P2LIdJd0Bnsn+Mw`j^j>6L~;FZ+Fjh z>%wM^VeXyMdfD+M$bL~ol$P+~M$1EY^tp#kqfBX7CI`FeXrXy9f;*+1aKUhx3t5Mc z8xXJW2QvV*@|OcWl|8{CFvXC~5smwg+2K3Zuc~RZy2hPzo`L}I)8=+L z;+No_UYg38ozad;gr#FeKyt^77wXuqVAEL^ZY#w`PY#&Z17y^{6g^wK^K*9U41H#4 z?KxNUM8h?H3>6xy|D3nVZ)+{(=nw4J-0T?XHFO3@cQXvm9AdA@DVGb0JXD(^JTfWH z638!;sKq5eU3F7)CrmgwH^k_xw77oM#=11&8j>J!Tnwt&+q`rwAxXHoD)%@~j&^lp zk?6%d(+maK)+-E1eEt|D(RnSY2Skjwt{Fin^bzRg4b##mM~}*aOF=4gM{$~-N#v3# zhfe|0-!$nnbCbk|1xKl%4xNB-N-7hXv=^7#*SsGLCR(-e4~W_#uJLrH`E%^1*<=3* z+_AbI9Y3SZv?Fuey>j`ok)+b?6-5Y23skj0$YZZ1MCJVjO1Tj};ET4JkUsvIX5#Kx(Yc~{N8089mFQg8(_zJ50+$a6 zn&v$ji8yYjJZcQSuoyOJGQ^=5pxRJ`Cd5#iYs`B-&hu`aBYKiCqe97tWazGy-!=5M0`Zh&kFxc(C-5OjBaW% z8qmk_Iy{(GT)Y+fa%|M9l3tQFwz>pJ62;(tgZ86{=l8R8CS?DW)G8A>(DWw5PI#~j zyA)PRxai(Lz0RU1vJX5(Y9Pm_;R-dg!6Uze#7BhHS2g@f_WDWL9STvt{H9Rx=Vy>I zSg3hq&F81N}JKC$WaO`jzCe|6q`Pnc8 zr$VtWqttT-ukYhALgWFMcFo|yPMR+)xEVYL#JQu}3Mg_#y^>y3=eoUZKshgMr2^7X zjqt$o!W?lg>+!sH!*{7Xn;9FREi_Ug6G{wdR*g_g7PX*iHtVEqt`=jSvrgYg-e15- z(I+wTk5mkp)(8N?_kx|@6imWCJNQdUFN(q&MTk5SNvY`0Cg=*^Gu*~JmgW?8cf=^J zM?TRjZ}T?*c#i`PxR(O#Z^$MYqp5{>*g8pKkFaVUmvIc9sj&3eOOs-@*sY!b@mtEQ z2|*>_iInS7o9&R%q}&F8>8oY<2gYv6M~vYO>!{L>W+d^m6IXy!`V+_00_8UHR8eC} zyF7OQ3oFn57f?3gzRi*4T4BEapkA~QXimB~*FE1CyuzWGMY8CYzjr~YC)6ZZBtE(- z4A3Rb^Px@vv%8~SxNv(en>S%arSyGh9S`Kh4OC7xB~`vW5{EFLO#Ma2ar~g`@}c;q zaFts6-fk@xVq|zikrEGL+DK;aLEL2DH6$;W(XJO7(Ko{B1McC@8Rm-tE|%X^;)6_h zZ?g^RhSBwNf>n-NecF6rV|J`&--LD`&JqeS{~1mDmsFKY+7**5h1n%g4*Q@Xi~{O+zirW8q zhM4>5$Br)jT=dH+5$h}x=c|wQQ5@%F7F)7&Mub&HPXl-;@%qL4BiQ>R z*}F@ekB<|Bkn7@PEyNQ6IbW%?cOl41Thw*=hnjlEepOGvHCu8hWAnt<$e} zIkZKy9u*5fu9hp1@5s4t75>oA%GJ9v%dV-UFQP$k4T>X!h@$}#geeq??N*?Nq2S^I z<7gR?PP9_xH`xS@8Z+s9_E2gIvIPCYyuq@^%(7;I zeX-x68WsERIKFxNgxfd8lkJ2*za5_m)7j$6BF#GiJ6gl7_pt*J6M`i$eDrv#9pmR? z#$aya*x1Kvh8gQZ+$*LDfA5&a?6G8f5{@b*dVWgZb6q5Cjb!HPOJS0!sVY|Kfv&an z2&MUm6G@)TD?BKY!djGxVoOlhb2P@Y2D9$w0j(fFZ!S{QPi-$jb~h)8?o&)p$jPLz zLQq=+Qp+czoDlFDqPUj_sH7EXPfL6@$N$a?`-?-LdO%>>k5%LKne776Yu4uBVWli8S%8vDlaw$iBZ#BRd-5X=URjgh5a8+z z#@(;QHRL_OesvEU*woKkYc~un++kq!Ol0{3nkw+IE9?q*Gs`a?$}dvjI#Y_)DVaD@ zqo$>&rpZi0RQ7x_0XIeMSfsxmI1L?R6_1L4QgAMdP-xK2U|ryEgIMa=iE# z^PdUWf5%teWPM3ZOC}($(tQkg6=^h?@)1Yx?HmBKd<*n+1i%pN6+*tA zzy9J)9~W~HyNa>$|V8}jlDSWRtGbRE6oqO^!}Q9Ia7EY%VkT;zPWONfTKFJ;$SG9uVSYeE`pe$y!66A7{ndotT; znT8Ek(!eSdvqngeMuJNNr+z{74wiWI9pocn2isHspa1Mx9w~0)eV_yeHAl~Lj&~H^8km#u0GM)BR z6{JT(co!TfdzTzId*>^?|0NUdwy~j(G1L-;Ed4!ViSC1SY@{*#L#z$UY_0AEXXTDtlOTER3l|I(1}ridw*g|l%`Ezl1!Cf znEeA-5-;NSV){H$`Z%#S8gF63uHtdS0ks*PB<@Tc4TpX~}!lulIv(BlL4%F>%(*BQU@bx%{VaVHS@X0>`q;48t0?5u)PP z8;@Sq9c9^LQvQr`4s#2cENAXYhzc{dRXGsHK}pi0OX?cJpE(wJm*T$^DqkTLedsD) z4uFl^=0xgDbNF@5%me-0mE6wPSbqP9>WTVnJ?My+$q^BRXpYL$>rvd|{iZ`qz9!8q9}2oReGA$Ta_YrBld|Qr?Wy>kWO#sfs>h&8rAQg+R4sQ^t9Z+ErO3X8VhsOc zY2NdSB}0zl9H)Q(hUGXoSl|w^MD{EN%Q(4_8>ck<)q9eU*c*v9Aoljr8RjnK{hxsM zf2-BEKd1=d|E**7z)RgS;6OlpFhD>&|8Wj&QK!uX|Bt^hlos>l|5zQb;)9<3C#z!T z9jP(sJFDUb;(y+B)Cj;+`(`*GQoUE$5n3UCfaZYyYlPgNcY~HHw?q~UC_u!C9)S#l z#*vNcc`J-^cjNF=V3n20$mVvx;d-7*-~4>Noa6vPO`FdrsU8&28gzvZ{Au^2kf@Tx zp9p^tOA);gXNp|Q8D|7LL3(mHdHwi^d0}jR0-2>L9pe6*d}iv;CDzlrdhz zJajKIFsJpMFsieug}MWy?@YOAyHojH&IN0L53zPn#C zxDC%DA!i<2d~f<2eK;}0fld(RfLkib$`M9%OLV+?6!FDv&FPj9uw^JU2sUU^qL?^R znhYIZ$`ukmQtuMYl7jkHj!DBdivruCwC&vCD`Uw%7lRP;`)!A@=mV3@lKTWNRcKHa zw3Sg(55IfNS-g$xH+EYzH_mUIv;kgF-m_(ag!{jSa&S#~(S{#z6^9w}F}5{|nAi*E z0!g5-==PD)xm@+X0i!B}F+=&X063BF;Y_EXm<6s-&6>8=9P@f2R7guAi%2UK4RFR+ zQhG<0adb9dir3y8r*J~A1Ex6!xy?7nUazLB1-aXSvA50Z;IYr#dIW!c(+Ks6gq zoM7L<>#TOQ;7_imbWkhSYfUw+OfQoyUn#0U<+O@aEstuxthu>>8T7P#KG{KVu;2lh z6Ec}p9ECEkIQX<&(6UDG&LUUJ$Y`#>S>Olq2konl&?4?*k)H|6>v$2!Eo9$(_^L2#&+Pw=1 ztFafnaH})gXBS6z=R`P!8U@BX!Wt!tfu3?+x&;@0ERreU1 zA3_;j{Xviz2ztWl53>CD+dje2rW5U^HAh06C5zyydtXdTNs=kbK6iV9V{jU7+zlBRvhHb7tk+ORN(%> z%WkHH`9pF$Q{uw>?W*{k-@h>E&9CXF(pVgiO3n@t&8*E2)tqgCjt>@Q1`FO`W7ZfX zoEK2S1&^Q=sMy0aFqG-FV$As_mTTrcStdL3ph;|<9Cm$Do9Mri8mwR8v|q&bfW%yN%0o7@J4Cy^-5_=Uy3liX zlv0$CWV#S^nS6;4K62i`r7KD~3Y@|8dhMZ(C1aoPBCd2g`M(ffs88M8z&!JfybszX zW1s6_`Xeq4?81Nei0Q;hJKs@>U;YgP_%A&FH`4#j0w4&Y9r~X_M}tg$_#ZyQ(CMb0 z%=gJa3Z6=o{modo)#fAzu#qy;8`~HLIfc7fN_bPjkFyX1_ijuetqX zZP4-$b3CiD(gJh`e5x)y4B|{R#Z|>+LkAh%FT|e&v%t0@50W?lEa_xfhWV~Xxo>vl zgJbYU$R?5l!7h-a@>nnpxFt^^F+T%erJ!;=;&V%QOxLV}>uK7O2)Ak&ej>|<&F%Yd zCO4e+oL;jc#I*A`KYMtU)_N*R+@>C7xqTYvti`GZmF7bovxLY!GW(%kmR~a;@ftBF zY3J|RHzcyl@4Q|E!f2D#@LuL?a)U0sET$^ourpHQGP)&~XNGulUYMhTF?0VA(^HL1 zXzT;;gU9OF{}{ce$Ac4Z`NLojVf;?J$BRz3Ja14pN-Sd$U5{$s^kZ5}J9?#b{q~@YyAB_Af?{D}->_3l&<)pg2_X<5JiHhKjd>fWh)1kjPit^d7Y(Od0 zNV}})N7|hF^^V7jVRqIelmYo2mcjk#%vkD(F#Q^^CC+S_R_yMNm%q#?{^r^cPg4^g z(o+%_Wrsru$fnxEkUTd1nVWaqLtJZk6JS=xQthA6!Y;-tE`r+Xkm>PIbSk-1pM&aU zIOfZ0f|>n>sPh^%kh{p5EZ2;fM}qeKr10D;{i)buNqb0biQXV<>~Icxhx<=0`tOeW z-#hf5T4aMX8zuMcU)%ae5VQg8U%CyCTkme@xAp8cXe!_tQ!o{2ogEq=W@+PM>ZIUg z?`COY>O{))pS{|wBrA{0gybX4PAdrt{3Zx1sv{u9xwl|lsaNnraM2G6M!cyLn@q0N z?U;@5Q{9#!+4BtXEVd#|S1G7M>Nt+z>W-(EW0v6kwDyJ{sB6d*{aY?}izP2q736_) zfNl8H%)Ze}l!P-4;3KkF2S&ZXw{7$KA~JJF87}`ZiJvKA$?x=Ymcn39{3prMQ?%i# zR3WXg41e?ugbq5kA0H&>9t|QrwF36HM8|0F< zCDXU~@A3FcjFF5(R`Avya-6S((?ql5-5|Ky8`+qOo38@WBuqVrZwA0w?Gdl(6D9l zL)@s2x9BXn;(zCAcIF{kH-^0|Ilc<$49gAZ4h{WNU?GAehm-xz8Vj|V98c8jc{uT3^d!W1|fb?ai$zBJc(TA=es4_%Gskw`Yv;^*g><yn$-i%Zo2P1*uXH>eNx{&A=y7MS@kCmVPRm;s?^L<4xXtp~^E*fK zrt9tP1KjTrD|t>i8f=O4A|7Ldi6{ovQ79}?C@aC?XABExC+MjIUyRisfYcf4NHC|> z9D||2fGBh-)&-tjt%KLpght7veJ7kv4AH)qIXsJfwH&rPS9@cV1_REP3kRLtOU^WG zrbR$zjA8nq*!=NRkZm>AFFO{QSez6qr`7I43|1EXqm~NoUp1BBtGa6Hzh>(Hi2mMD zn}mw_Iq9sN<9>Ep)(ywh3NYZrw+sp9?^~@dHqGQ7P@rh@$yTw=LA=itoj%59XlH&< z1#yMhC>Yf;TQh2(c*RDGV7OwWB+30Qh@wq0%btmKiGLpBBA}M>*iOqQ+QoBskb!b?Rai z^ph4ta@dkr3G<{)X8Fwq9D_1rqZE|QK~YhHaFSRlj^OH|SnIdz0RfX(cT_b@TknWH z3KyeBgv$kHGnI$!;D8$CNP^c$-cKT^+TD3|<+U1}a{=oPIcx3(ORHPy%{5mM{am=- zOU5OKzh!T}A@y+?ezD@b2F%IRGr`bQEnVcZts=bfFzFzc9c@d2Uo6Olp*WDb9ylU5 zw<9Heb_h0B!~(=bTJY~ z*Kw2Qi=@x)PX<~CWg8GDJ}+pTb%NqT$Qu@7{}1l|Kf1mzxc}|+|E>SNs-O@5>EWe^ z$gUE;Avg;4&#B6p+NTN%mr9UHi>7(A~_sBo&d6_OF=7?J}m`AO9pZ%X4erMAKiMZurRG z36^oxCHA{LJ~QpDoi{yiho$;JTYZ6cYcFF$416oOxa4+P9M01Z<4;5M7PX+~ zL-i<7=JL-gZBUBo4(|1i)RR>Vp`SEBS;JV>LZ!?mPwewM>}bd*U{fWtcv~hb`CY{_ z?r(G6-gvymHmcuBsG1|Ke(Ws6t{c1Zxc*_S*tZpPPyn;VRWl-@(G`zn4w#s5H8Wxl;^2JxPECNP4C7o|dA#XS63OOY2 zV8V8mX(2{MpxLiO`RDTz#Gsr!_buMZxHgaRtY&5ZLOJJkDS93S1lO~=`Nf12&ZiGV zu07*U@n#**HA6-Mgr8w`vN-)*{c`N*#c7OvyCD}}5X3xlJs4)6rJUYxY8JWl&6_0F`h52xAJp(os?A9YT+f*z)cSb%PylocwCC`KI%W0ixE zERP_)Fy`p~Mg0fh|HbbA1vs*8FHpz#Ro?u&Us6c*&qLx%?NbLuOs)FB-1wKBEDQe# zJ@Or)jv$+=I_^LEoK z=-a(s=E-B`GPnkW$;VsH&ehZvNAC^CZZnJLYqj$!@YTR2c^WdS;6NS#S{TbrHjGVN zlVxBhBpNH&MhKpmXC#Ut{eW{sHm%QuXeZQ>#IFB+{T3`15$jeEb>&c) zhtFm}%sYs!^}cP|FSgrR%q(m%LMe4CSZ`5(u7&Bf`TwPTXAf{ul}m9?ODizTrblgw-bx9pBvd zywb*9#qArHPTuS6&uQHqF@)i2@!`V)UfKy^w_Y6habbA8l{S)wf{2C}!+1w~a&6irj4NDlf_;Cf<2Lc*K)Fi+FrY$-#VVkV1rSd_d zU}0lxN~2UV)$At3NGC>G(QDuU=(SK)*C{@Fc*^*`up+rNuOi^yl}G9ML#3wRpkzZw zk%*ramY@S37q05!H1Nr5eHyF)N-t%!;0m) zS#aV*29>0U<%53!Gn4t=!SBC~aTyl&5W^?N^5dAPD}X_;b&v&SGVPPn1d_JcuyCvl zCRV{t=u^XBh~__ZSTTGU&AwlWFcgkS!COob|BU zLWj$8bHCwe7Io!Z7b{>(Ip2Bn}&28FOX9At~@(K(>_U#}8|i-IaChoOT_qG0O{%a9u| z@bKiI-*R~a;zyl9vR)E^IC3!OhCKC0u*nyv4#9zyJ>ube2J$_rQlb~~uwW}AyN3@e zK@QSR9M$^ZW|+z}nOlJfdqxf~L1qp?V(o8VO8o5i$$)o9+&SBv*d#m?bD&QQsO9il zCXttk&87^)$p@Num~oWQsBtkONW{*FLuDhSBWN%HFo}-qmt4TR!_PH-OT|wVN}X2E zo>{UpTA8IYipG`ei)T&C%N^P5?$wLUjq}~DE5{Cu!3?l3e0uwLpuqe?v6SrKA6WeR zcgn!qFX?`{hnOI~;p)FWr21;_a1r3svlFz`^5|l$A`9Z{;7T>>WLLHAxD+K%BwoUz z!9Fkmy{NZdT|QvzeV^KOKUUH<2$mjNr5Gv&x>OTrRl!-vxjQ*e*?w0gya`DHg@mM z-)M~2lNbk79E&JxN{526iKRE}>>ggbw0dl2yl--QXY;kzF3%dCHc(?8#GQ~x;{x~q z(>nd>$J8m|)4`>+rq^2uKNpW91tUIYj>l`SXglJ>L=h;aO^aCiY*p^UJ>qu{f6s31 zBY1jpC7HDiFDb13RsqRZc!NmgCyh^$x;#cmHC?aqA=9L%nj(BvTe%pnXObib-8hMS zg@{?$=Z4iQIa+Jzrzf&kASq`UgRoWu6r2QWn{z9lRfzj&Z6DvbzisPvG~3EmB}@K{ zLy`<T=x9c&+N8SGv16tT6R_)s=ML&=~pJvX?=D%Zq{%uK^OxEU(XQg z6R$1YKrL#@WMd9OYW+-3q16mN)x|}(SI1t&Dg|0k6g%}BPH?*0<&zGoL6Uj`Aib-S z_%(w9j7IFO?&TcXINvx`PK>&{@fY?8*!X7~my0gDMPu;x1nSb!Q*-mLt!zDgT=12F zD5(!Jp)XC&?23dW|8VJ(0Gg8+^kKG21sY~5mOPs0+KpmD0(^YFj(T{rC7?VTK{~fZ zXT<3HI&9C28&Uk`M^=)BHou%;z%hEVwv@~3;-20d7;YKcJZjUN2&`S6o~#hsdmZ7N zCx_Ksl{@O=eNzr9{gE{<2TuOt(yxSz-(+CCywZ}}nn$+bB)uY-p*2N65Z=y9`^}f^ z!R2jnoB5zD#F2M`;m`WWGdAIR#Vi3J^s?ow|Nkc${+MA=BLE;-%_3bDHqJJSq{5=6 zwq0bTZy5&S108r`2PSRf>Mf$oPhpJ$>9^6~jKJkTvG=0(J^q z`G=PsZ)EH8d5t}=)!73}bcxP4RMY&#yiPYXsDi4?8HW3yt%riGMK0gw zt8Htl(J(>|ywI2tm%gB8%O{mUu`FMA z)(AL{KNnB9`YoE?@S+F&?4JHh*GZlr^8;SRoYV{>b6~0NBLspXn4{LLxaWP?;3b+i z{!mpw-NzXzMKkIry_U8Bt=znAb0{&LMAnY1GmqtNY5TgL-oUzl07reW_-C`%?vU2)|B>~L(RH<5yKQVWw$<29W4ke$G`1UOH@2~3+qRk=+qP}Dxaw_Lb=)#{$_@amQWg5`qYut%N|4`X~cH@@2Rf; zBt8F~Fa0ey`5UX-=zf$wA)tU0kuW5{EC4trP=AdB3i$5w@fB1azpgy}4ZW|E-rA2j z_=QFhmKtUWlOlNR06`1GKl!JSshwScORAGOnYpi2)h>o7o^9xM0P=d4koJ0HY)=0MV^X} zTkT`gbn=ylh*$vRjlL!N*z~O(3xd`Qq@MlOQKZ|uU7Kqp$hysEP;vIF5!ib1t6f)O z_?eo{XX^Z@T1bOjYW$H3Wd&8aUEG-V!|seH;JX$4Yp`x2ZD=fvE5vS-JgD+*{keup zsvdd6w4&YojAk~CKQ|?8HdNTAH|P^C6EodDS0)Dh2s;ME>;9bB@%X+I7GBI#+_*1} zNXQ#so&0P&!LX^eLDXE2NW-W6%Gz_$rXUCPO3Z~wP`y1Alk%Q4c?A6yVmW|Ez0nb< z3cg*1AS)gBC&dR^gVPQ}6P&dQJM@)c&=?})h5n+fvUx(ob8>~xnV+KImM?ebIezDo zI@TpGe)t~X@<~S5m?!wu2Zs4%kJis`X*2k znw%EZA-C=W9LKVFx0Rt;MT8Mvv14B6gc{iVz?cYGYZ$s--1p?rdpm;Xq^t$>HU0Qs z@B}oWQN?Q0PRCS5F*lThGSrMBj|9Po3_Ye=g$L#UJ^ulez+W*&s30Tj61(6$X*k~} z{1}^vH(3r7C3Utao)muES_#8Dvk``f8x!ip?>dVN>M;&RYAsa`k(=ufEur4zuZqqo zM@F|i`SyO)*kW?3tU(?np_5czgkJ~V!2V|Q|FOaUZJmg@!#f`nx;}k!{Agqw1BFb$ zX@M6LoP9Ap>ztsp*Th9m8pkCZ^tRPfy@oV0!-9#HfD|O#HCnx1=Cw_lyby zm{3RxDakSdnP~eTi7~Nrns1&;GGsc3Y)nHf@dB)9hEb*g&C6RhV<@v9ot*6!Q&+y_ zt!_ibYhgQ1L%awmTPAGC;Yo=5wkWNLZtF9P0+rnYwd*#oQ~pXee8(scqB_pDoafBb zCA^Al?gGR8C8_Cm9p*t4&E01PPV)lk1wGCH(^w!IR^+IoUbRi^dhy0 z)w$_=iUBbImBjoPt%fGepD^;k>44F)F-12INb)DGz}5Ir)$Nc{4TV8PvA}F!iU;>v zy-k}z;`ZQ1`ZV1|j)%|iXy(8g|6TfN zs7ybuOlp+|^;*g0Dw|UnkY&qv8Wdop;!qv$U_=FArOa$j1t{;V`PSAl?Iia$6-!>o zz`#!Rxul6f;XEnSR0H8>h{1ELeMwT6BZiB;Lhq+Q)$pH51{Y;o2<-)@Fr>`53Qjbd zWhV|QXBKyzSwD3G-DW%ToGQH-h_Q#71r0McB2SomzTtaA{DB_JGwHy3s`P*Dbu$AL_ zmX(;FkD-4g%3Vb87T#<7`XC(s)aONIq{&JmSs#KI63zp^foLnY8Wp2@b+=%NH|+js zh2N1lgobt$cmGuS|J#E9-&$0-R-FG|Oobrqr%!7C+ICkT7A%ms43-cWg^CReP+NBR zB7wqB_5)MO5(UaC4L&LBHv^KTT}2O8d_6wzKp!!*ygz0osI&gy?iytM3?6`HLB>{HuknrtY`*f`Av;4a|lqY>`QXd<6O2G%-w8 z9G!kD@s^kXzePHmEj3)nhZ)ICfaU|A$kPvvhpBFLOdD1WkCkEg_Ho6c;;zg5IYxrX ztqw;1{!>2n=FZc>2}Do&``dYh+?M>D7dp`1W`>%?AnKn6ba|QMhHGJtoNp(R?dN=n z5$>cd8L=H}2;c<4#MSuq50NJ=Obz=9l{ciI`v4)lbIz_UvOoi%4s%*DfaZryPb(sh z&1Im(fGa)9ibs!#@t&^T0=fSmJu=Je57v^(A2L1p;^u69IDk?!NLJ> z`><5r!-HguHg!-dYLes)(g>WPh#9+BEK6$ z1tzk@Km`!f(2Hm^%-6tVKXzH@UFHu&AK$P)++ z9}x;6s8E&ds(r-+4ByTUtsYI+yT2MZC;K309Z|95pvfnB3WOUaebUah;4r3c#Mz4~ zsB^r38mAI*CZ1uIJ71m7stEkfV48yUP2tGshJh$&-AyNJ7B*HV4{b-b_yxC&?Hadu z$`;#QOc5@QEi#tW4iC#bjX3-}bHKfiC<`TyS8>c_3iAUvfInhR`0*16K`=+-3G&1x ze8Mw+^OG7}6s&Z6a%Rq>!T`>JX&!QP_G794zR{LL54H$y*p5-io5^=3h4TC8G2y5& zFosQ*m}E*fY?}VQ1ks-l)au!*>aFPeE$GwvS3d^c;GRz>mqOCcQqGe}e?E%78|uhZ z9aC@c$M(VEQH$)e2H|p?$-ct>-KPHAU;pFDf0L+2_gPpq2#8mq5791>B?$_`M&v_0 zM#mozf9>+*=XyPO%4~VXlxD)DMiiU}yqf*tQZ?=cWTPJSqowb;;7s}*W z8qvh-aReVSIihI>?u#VcJMiumOL$9t0>vxO6a1 zB{7Ib7O609&G~?o`mgVU2dV98XY{69pDo8KVC_>U-lkjya~Y3Bv$u!Xh6K{z!}r(C z-#wBP9yrfCv~~TuUn0eK6M;;%2_Z=J2gcD`{k#kU z5RTsolQ#dr)+DdVIasI!tt1bmWyF8wEsyK-5N(_~VS8>J*VZ+R+uVE?RptYjwXTHV z2J%p=m*TD7sa4vfwANd6O7qVWGhd6f_eC?ew66`9d$%Y4$U7TTZUof6__qDz7QuDo zlH-s#0pDT+$6fD=^fu}wzSDfWtdt~gA&OZDHw!g$5XZ9{QR#e zdGk>Yfc+~{Ez`AvA^lzdlSQuj@03*^2;e3WDh;rLh#nb>_22$;x{9vtG!r_%O-*3E z0sX2PkmiaG?=u=u?23*G5XNGI>o}aW3x{9!-5$>czYc%JxoT(w|GEg{=u6-)iM|dr zEx;W}KwVnYTk(OQ!aYXPSHs*srxCj(&?&wLJQ*%9A*9Cz(jVJ+6=;4;0@}k~VKjPW zItz(D*`P)>r#0N|Jo0+N9z26LZa@q#*=M5?zOH!;E z`Bg~XxpL+th*KfC)s25K-@(%B9HxR|09@9-NpU>k;?<9D}-L#(*!H$K+y#Esu1534u!9v4fBe~tVCEGDA zBHKp!2~{2XZTT!40K?91)OI7FQerlc%Ua4hl0+N4ocx59enhEW_{9*0a{slI6SX@1 zkPcN_qk!t*_a<1G`|1rLvBK#%zb;JsBAOTb46JjA2`~Ec4p!SONm3)T=wPd$|E`zc zE4u{eKq4&STo>PHYI8SO1ZDJiI76;IjttqXslNebl#8^xQejnxVSzug;h#`cgRmsS zQJoq34KgKob%z~Q9WvW(9vgjT)5))+aa+iL!`*+>{J#x`9+F!AZ&I!UL_WbF1a9yX zp#j&lI1w86rQtfjVE@_(6y2cGJg`Cz&Kp1~AFNr5b|X$c?z(o?!O5EP`1t&c^Sj>+ zx+yZB$CK?Cebvrxv0o0_=CSG2FksVs2Ecf2G_xHinIaj+_l^?LB%e^F^iw$|S}<87 z#(3($ENkbY>uOL`BXG(fDEp32NqP7MQlxoy&Ge!nxR+@?LF+_bA+Wn+TSSZsUKbE& zS3h3v^zdR=k$Jt7RD+dNVhHnOV)3P#Z1r{P_-27(b*=)tm3gl)tp}DiPReWM%i}iQpbeYiCJ{}&vFd*}zpKFC~Xz5=U@ zzJBanU~mXP04Y5(;9rOD)%5WEKAZmj#uCMwtkaExP0WbBlOT%*j_r4)j0Oe)5&;WD z!JuU292_vB90Lc_>QO6Iy;S-EiP0-jT&{&8<@^2ALcMV{s8!o#e)aTh>9nJ@)w)sJ zrL}iz{d^^Lkepn^==EeTigiE3W9Gi2$Ky>7S|7Z) zw|N17n}h;mdx<-}WUpM?4}^cGJAV@cMOim~ppNAU-)1Ao-{e4@truB%gfBe-+PfzT z#t)hd=MTu7-jVO|i;O#6{VdOnIB&}V%c~C*ZxV#w6d+A)B>zn+h8HdHX)52;V272Y zfXsnhz^L3W2POiC7JF86&rluQ-IjqF2Vok^nxe02cH?MLLm?Xh!2=^FY3__+*7Z%`4Xsgj(aI?Sf`Dg{0B>BH3IA zPTWC8TUva%Wy>Sd>J0|BlK5=tKpEqrL<1iL0DQnCDeA=sQ2*+{p#*~lYk2vq(Wx{} z;y_0}i*};Kuz7&EdULg3MPg6_?rig<{H+y6M%VH&t$}de7 z_DXT7F`DBkxrp87x#Fq*z4a_sb4AHnoZ`9SZ2}+cD&Y_YOC^D`UmH2B7puK@8we2b z07sk^77LRrP3=D4BK%=*tIbGpf+;KEMU|mQ8g(ny@YnhsU6|%pzi^_3XrwXK(pZp} z+8%}E=L~wy#(1&MhO2nFgOq*0E*L|&P$;kZrxe(grQB29tGPhtWCx>GBJ~N2SGh<3 z-6o%eSI*}d~}BJ;8&rP zwStp~KrBA{Mb<$vLAaUVTcE*&(#(K@JX~9ErtmA^%eH@(`H#^f*Q{lx04$VKW59BN zvA23tZK(kG+t*=n5Wtx@DZKZdVRdc6Nk!i!izfj=qU1inqhvY@j=3DPB2GOh?$6{! zN9#|4HQ1>Hku;Hp0O3-Od_x_~1z}z%mRpVs+hiQEYZdh91B0jc&HVe}bD;gPr7UPc zFQeDQYBfau4g)G_UOix2oJp$b62PjTl2D2%+bf56I}mWs=5$YfZ^IE=>>6(%4Yidn z9_x~#?XWHG@|&J{$d?Dvm^2fX!LUQICG=|Xsp9(#92mDj;s~Y;GtXEs>Z$B6`<~5( zi3vP5_gJeQ*a$6{I3ChsQD8A2SsiUOLREc&@opaLB;w{mR(E>=tk`34J)l&?&A*%~ zaR3HUyhuvwPsQleDE27rbPhd1xT*Ht-Nl|KMQ&0SlYR~oMzzM*Q@`lt?rc4Im>20& zitpigHqEgSu$R3f7t{LYR`X(JRy`eZml`fy26Xg9j{QgI%h4p5bB1RWBQ3<-YUUDj ziRd@Fo1$LiiP6!6Lbk2?WPm7tnW4Tvb>1gmz2QFiIxA6SDJWKqCsddzu)}Khdm_wg zS`0X&5a-77ajmC|5+-hNd`Cjb)pE~;(JW(iUNkSb8LJP}Xf<%)V5c7EwoP(*8?-U` zw%+h*7k8mobDHS1`I>EB=GE3PU`2ULIUCk{CK)`iAJ$REa6hm5=mVlxUB`Ss`2G8r zFHHv(KD)}F`;sKW2kcH`^HFID&Gjgvz2M9?$;S%`I6 zOoG!Op*4%l-x4x())-2lA**mNE{fZJj8?;Yv`GIMI=c;SQS*V_o=BW|b%Jl<5*HVNQZ4n**dUxy4 zpCrv@#$NDr!kh|uAxN7I?qEh_qZ8o|Tqq?a&4n5QrsiJ)r$}~^wRI91k!C{Hr1Z+4P6N*bWCdC5ms%I zDJJ!KHw@W4CpBUzBJz1!Jwr0SO+?F_hXg(L9XiPPWKVmy$Q6Qz)2_;AnijZ9^vk$v z7BX%yq@P7&xlCa>w}hR|9m+UG$=?qxdG!={cgiCVfcyYAC|JUaq3GRP_6tt06P`)+ z-kOC$hp6d<`Tb7rI#rLM2=}3EK^I=>NVPLJ{s;v@1}p7bnr3f$E}{%UNYZ?RP@Za);7`h)FU3a6#q<@$*>!{rP8x9gagJCtDAak z`DA4GBN`iW%wpZU8P3_os~&|wk*ayF4_U5w7bhC2(4oo)y0E&d4C6}p1mKxiH+ohr zE@U^vbEX3;fJibVdqS$`{jK0qxBQ!YHP-t;hu!`2tyq#00bi>VGZrusy(OETe6eAo zr51nyF=9#th>sJ^IHClUiL$?nGm6!S6Hz1aG(6Z=`_=|!`y{!)jC&f=OR7wLjCjmx zZN941=zMz=I65L?Oy8YW9k!cbqNlRA)jC0V7&H}OUaUi`Ew*e7;SR;w3JSrLr<^M! zo^Udu-`9xu?2|1wBg}{=*c7TFI1PO<6mkYp7;u~Ut zpUw4@=+3m9Ab?dh&PBsK&yOYjbKHprL<^ATg_~LN_O}aJP6D zkxYcgcHIg+%t`;nf3(<)&Dt6IEfcV;hDj0yUr@kgdSCRgxJQ?1HaC!HH!<^PBj%EN z7)QP=R17D0DtjbV$VH}Uc>x!(_SS3^@fMG>%p&n!#%Pr`GXX74MTEvaCOvnn^$d)> zWUM8-j+^J$yp(&a8C!Xx-k=i`zgS(~11^&QG3ShR87OFY$h`)X@O7xP8c} z$dVstqO<4T^%-QkneH9B{PR*|CEm>oyX)q%VXZ$1&-V&(*A!lkZm zkI;HxNQqm78HaleD`C$M>eTpk)NDd39)%Ck8*~g4V_maa(aXp#f>+{Gt%z}&9NC<+ ze1?;8i?4JPG(*T5@yp4XFUJ6xd-eEu%q4ir8!`ZSFLZPeG)6D3qissg*fBCqv9lPj z#3%%BYRYOTBdFqm-|8wC&3k*t#rn-#w#*gz%UIX67E>`1q%Fo0G!fNsZs!il3Wisb z0|M^5tAasE=Y+s-^FpT8A3VRiRj99k8N$`;$JEp~pA<|VtP7b`dKv*Vmsr&>U95E7 zN=+AZrN?e+qwr-9m6xh|Jr17Wh%5RPpy7p3^(un(@AEnap-CMk+D|Fghn5Zbm5V02 zDkeXXkodZ?_$r)oQ^r*mnSY7RwuQK988&)O%E@bw+|^z98w_2=2A-USceYfF}`YnS8x# za#L$4d&p;FU`=AP3!OX^A$d~j?PTW~z4s=tw4(&M;?hfK{GqJ+(Hy)o8j0^E5VaDn z*{wI+hza5Fijk0i=N(U^upF)O`QBP=#@0p}Jhk*CVCF=sVF@3=?h#7TaQVU%si9U2 zg#2~c8rPsDVNCS$Hs3|LxvqJ9!;^`gTWm#Nbkv`3>y|E|IA$BxKKA+hyoFw&`Uh7x zbfw&_g_p+0<=KknG1sp%k$R48?wzH!$bDvoNsR-=0e1q*x*8Hh+Aobnz7hmqy)lY`ixfqGTd#E(?!*ru6x(JFKOal1gYQ`%P%(!l$)*L7xx>Wb3=x=Pw*E4 zn2uO&n1FE03sb8Qq8xa@n8Be45}%-c*IVf71=+7$A_t-uGRQrXK%>t=6@{9NIpcBW z8VUsl{KyeJIi&i{_E4jdz0@$Q(j;1|m07tnzEJYKlG!;(c`~%NX5!5G z&OkGaj$RM}n3>SHwa|?fLh0yo#)6_=KCUdS{&LV#i>t%kvmTdGK}dy&qiz6d zUcRzVI~0j;CMd#!!lm%QDlSD~L6e0my{^{D?r4{Q_u0-mjAni>@k{t957di|T1fHS zDzz~nJk8F2WqL3GcBc=02;})OlkuuLjq)dkTU8|GwtI>%v0N`b8={Lc>RG{IQxM!G z3{?#1dng$jxP&>SUalr~7Po8a-WGUDP?VW~x2*9)($Ju|_fRgJF{VqElJT#5FKI%J zF~XH`R^IgoYXptGX-g%-F3hluq!}x&v8&l}duM|^AFPn~uB0@9FuoKwOpd{+N=It7 z$o`6R#R_e*fsQbiFB^LzX_vHINY%(Tn^QTS?VYi&MLAb&4SQJu;HdM_^dbVP3Bf9Rte zx!i)sz>LS->_}ByL#g+q(G+Tf$5jI`@HJ~PVW5!WVuHoh9MWW3H5>#-ms47ubhH3- zTpg&Eoq&syGaEdKdy*@@A~bLGT?l&m0AE%j!Ot~}RTl}1(E5QMLp9Mbjs@zO_!r^Y z1IJqRl}SBq|0gM0)*rYTtmcm5st1zl0qdLSKM6a!OTJ#e>c}oC41#%qY+P$@(^ejr18$POBHuLc*+!E- z7{PZj#0_IziIkBgV4O)K`6$9Scz^m;(C@V{ofhP{!?$qX5zCNmeQm-kfYb*ohF{+} zU;xO0({KsPt#VSy6%hO}L4Dl;k=WgW$UwjIz^)?;#i*hasC1C_8C=Knvuh)>&3Fs+a*cxpK?0Pa1X)%t&9Ed>gaE|Q1h2W zSyTf|Or;p*Z@-7!w##tU$6B;1OS_BFUPyqhvZ&;MRPhy-pZQ?WrK&4^1kiHOE(B%F zT>X4DflvO~vmt0d;K~Ib3^HYCehO0GFaX;%iil&h(hdj$OYC`+2Z^O2K}Vwf^N{|0 zLoG!4qd^GEhIXA>2=(&hy~9=$7)J!>ALl3+q=X629Wih`oXk>3a8|HHbA6Kvi;1r<5j5F| zlt}0jq+;nm+RDr1D%oZdl=9n*H+~}CBZmn_eNwoCe*D?bQsaO4vFOWMxuPJku8W$A zx}B9kHQ?hb)`~zAHv*B>o*P3376*9Hp^RT={#UX%eejw4><}k#mVn zg`7kmifrU_kS%FW`k`zHLD>W`k3JW4sUGJnw4@5SgB*SKApNAeNV+1HS))vYT+604 z!k@NL66v*961wxRCyd z>rCq1&&&qB(Nzi>`k9W(jlZKL>2(HU)a@hcB95DUWSX%eEC|v6)%HIs{V0zHeRSsj z-`+{%ks_)E6a=y8M+=ThgdPbT!t-wz&&&li6AbCEsDIbYV)RifCxrxF`jBA(t5oSR zfCAf`$UwV2P6VJ{G%75xLy;1yvBnqG7VO`<=f7c(Li{V9ic=c>&(*XP)Lw{xo|$(0 zDaqmEnX!<68#V;Mi(M{wAk#h<6foMD3#xJAfhG3ePK`klg^15bNmm%=FHbiCP+y!L z8JPCWf(cl6L{mq9W1zG8B|<8WGOZ>;C%&){{Xw$bV-~KFi%rxuav);Lk!q>Yx@epj zMhS?1nu1Ob84VvhXa&S?H``4{81dwRs_y%__n*fe$J*gD1v;NUZK#(eV}p$S2-Ri7 zw@ZB1FN1W{5b*nRA}uMMbY`pBe2*y+R-!)-^8jp)G$!rCgj{$G2E4L!hnU=hmQ(r4 zo=8`WCCi@#kk$EGSNE&~!`L~gpLK#)9j5O%`3V!#_2YPVwO>M3FGU#}bc6N3V~4l2 z?k?wiaR?lmM8(9rr;WHBLL;FNo3_TjhrrX_e{01bnf4z--Hra4YXYpxLHnD`w9cO?l}=_9xKu_4HLpg^Z;0^ z*eT6t@vsuY>Y*9tlz1vO{$9ky|riwCWG3*)dF zg~ja6iyb^^ug0FvE~ydKiEY8g$a)h~JaaQUVNKk(Cqv;`b?I zf5v3CB(fh~60t&F(k5SvCW*}az?FSW_yZRi-eN0PL6;Ze^5ct`hW<4+HE_WS7rMn5 zn$Up%nd&b3`vYT^Wo1_HJ4xIeg_Ja>HH&aG9`OK;uj7pzabo))v(zzod&!FZP|H4L zqwfJ}G-E()FXY!r0daw$)%hRARA&CapmMbcV?T8l6}}rNX>ix6eIwKFmX)n zg)qFeQ}SHT4fs-+?wOx1OS9UwWKhu@fA^fksqz6vpHDZ9KNQkn6| z&=M5_z`pG8i*KWkj>XCL=J}9w!=@8H{sVsdkB|O$Y?1vNMY1j7_8=hs5vzuX;QmMS z`GNfd3i0n%3abw0+mEnB1Y}yKBLqg9a-lX_A+X|sA(4H65(Jjkyox@YA095j0QG~@ z0vF-~azF_I2PDjag8@DraD1F`Q+rpz6GMNelX4`-kDp3hD)~eE)4{V@Eq@kDNwO=) zM@_i+)OL6kts<~}WL4at4DtF3za*AR6NUn5OzgnRVIq1a|54Y#?PdXlLP={`_I8!{ zRN{=V(}*dST{5A6xZXE^=O2MNyJW36!R37-QA=Hg_wMY#bSY?44tFVbg`Ij zx_5aCW+&&t`eElY!K?taI|9AsJ)gpLLlfmwH}vh-`KKO?N+{1xLn9XltibIDACH_Y zetabPT5OYN;jG`QX2Tn4HK)-f=Z)lZB#i@33u z*cXwye2L`IKUhS6RFCT7IdbBV5r2WA3GcATG9)fJ5w0vQvcKv!mPty?IS@+Xn94zfn!rg?1J0c>fhv9)Z<_{|E;&j zV}a$Or-vhhMfNuFTe)+p!CQomanf;R8G`z9f4=hqqNl>^yk!Ya!85iHg1EV{^P!Ym zSoYJ?<-C`}9Y4EDc67|8cC+WRG?xd#gqTR}Y^jIv6dN-De^tTVYT%dQKzc7SITkl2meq2bO*7~89$TelRK$1LRG@hbQ z&1&%}aa56F9_f{bTpp~s6j#=Xzm^)^o(d*|$_|s|fY=%Q-IBZ;-w|fyR~1i0@z`)c zV@k?5?$9oVqYCEjX!%COEItSOwuqUs`Tw#y{l}~SJ6!zr_eQv7B3tNxT)#&=`+EBW zF`ND6-voiF6%`36W5JEwsAfs|3I*}c@=I6BdN8EFFd^wGhLhco)P@Qk$Se&@0OYse zA_bJIS*m}L!1fN*!A!)!5EUy8ocr0-tO~E?+n|$I096H19jk}W5ocsvE8}8{al&}W zcwjnSJ#&|K_s}djbeBE~2a{L8|60X*uCum$t@rEA^L3f%Q^giEL3`6`BsW3kc4VL@ zwEPA@!pfB#;Y*kjE7{UETek6G!8#=(5Zo4B8gFxak>&uVL>_FA@2N~RsxeG** zA%}-rQ-pr_sK+lhJS*iXtVqbd_#zUM6)ie}9r{TN00C~Ds^`^#(-z?&`r5wL8ZysmQs%h|`um?h)r!CcS%NlqmRTA()&|<$ zx|-&^zU$=;#}EeYo>Z7)F7Cngr=hsIB|y z7JR~Y3Y*Ta=>r?4*l4b46TV48C7J>xH5}ryjmvu!kxLM~KK|a&NZ!5!;vXeJSwrn(6(f|xB)H^EyucW zG>m#h7)NG6-%O|(GZu_vHj^dO0&70|z|FpP)Uev26(W3BV%6WS_Cc<;=~5czvB1>< zC*(lzM9x%p5*)#m{4FXuI42EI#&z{AjMsqT5bAXC*fq*E92unzt;aY7UF98HJ~= zEygQ}1sX==bk+4k^;~ZX3?FeH{$OxRh;SLwM09QAPL#_DU6j~Ke!NoLz2$(zfIw2PE>FUzIgy^=#qV_SQmmIG zXoO#OXQdg?_6@0N%RFOlr5@%@#`Z86l?S~pvgM-m=h*!bg3RKayPY{tCXrRWE|qKZ z$b^TahAI<78(IE@0%HU~&%I|NN+*GBw9tGuP=GvGz^_)cQ-U|jY*pOxeu7b71hi_+ z7~>kSGcL(rxC${gxgf^MB!ir~%oWL3Kpp`uWP$F@OursqZikeQ`X%SR+9!wUQKWKoG0__p!*;n+B?x(3^)YbCm0x*Pdp#7wKj5TD&&JUtPT+v z<3&yg6p&{nXoU8mEQUn-`*XwHZMZwmr%(3$K%ZqyLSTh8Hy&VB3-$vUGb^}nY_c}8 zLt#YHg_8Ig%wLKag=7xKESN3^C7Ob+LS#=!oRZbvq+40LvE^5FMfnySoIREjS~Tqv zq!#^bNj-yg`L9WahWh$2Z?lefxAW21!QZf_=g+SlZ^ZYjF6-R;8Lw#`ar**ZSLUQ8 zYLu|c81HeO;5j%sTm`za}0V>UJL1iq3dpF`xRbn+@$QE!8-19}EXK&9j9MtHr$OM0-4f zh0ek83!|D_2wmpKsBL8`M#pOhz0Exny~FM*X;*FfvI>B`0hh|$7}`#)?#$>u*P{++ zX1_O8WkN3|7PJO%kq4`CUo9EZnCnLtCl~k54o7leI=;O_YaAm=M-EGmQDGAT0j+KP zK4k)DYq3w`&ei^-a6;!sf%scao6b(b`gJcuOAKSoVt!4pF^i{E;4tJFTta0$uykc+ zM4k5$mlFU;?%dj`o#mR>a`*9F>XM9l@gR3{OFEG6%4x#Xgb=`SVYi_4g?i{Xwh z9}-Ij|MTtkX|T3BNRG=8*9GBfIQK0Pv4yGbr8YYrmXGi-f4X}S^SYI zw_dlNo?qiqviIFjV8Gd+K<+9M%Ub*`I&7qK;dKupqi<%4Yvp3vZS2^`o$C~&zYhT$ z?|=vnw%`uES6dWv)*?}+MxsgX@pViDYo$ltV4JU6qZ9?r{ryUsj4T)aB<-&jG9KNH zSBbqbdRULk5iaOf37w@^_umDIu1FF($J4HV6g~SGGcv~qR{qw=dIsl#J3Ta))?BW( zA%GcKM4R7tD&uvV`VC?;&5btFJ#qnr-Ulj5{y0aT6`OadJKTI&8KrmkSSb_Zrni>N zuqn22-ux79#f?VIaa80mrm;d4!3n`pIJYmo>(HM=U+jPq+#AOjsV2iTuCxmV^-yLD zKLECn4vM!$wS}n+!^SIJ=qe+pElREB1GF%fynwTn1Xl9qE;u$T+0jPkj|l+k2Ud?1 zG@y&XnJ83j;MfEEV$D_GT?owvhcO;gZbt3>#@*gK`jbe}4>mCY9%W=ZQOAi-D@yML zPm)|noIZQ=DMkh}iHZ{%4qbMCoF%EwxJWx0OtdB_8EFNBlF1D?)>!E9p-;BQJ^0C{+ z%tVCTIVx7^IuU6TGE@9aad2uhONy5R$2U^`X-T2X+BtRz69^_QSPr$!DbGoUR*O4g zmliL8j%wb`5XBa38gEpKqa~u z*skW)I*k`5bV|y}rKE(+@i!wmH^)k*A8mJ@X(&--lvd?Jmz_Bsb) z-~}gef!aClsmbuUJdxP|=2gXg+-d#`JD+c0Js43onxRXCh}hqz;vk&>)c)R5u|`Pp zqT-YhJvYCH#!FvB3aSq+YLUuvUx@u6!0(0p`n%vic0k071= zaNCC%Ev!s~`s9C^JMpe&Po_Ho#a*U3A{wUoaaX^Bkd}&-sG#1MJM$*JoI1rhr#ZiS zYi_GQP~omhYfnO&z0mG&6Z2Iu$zAob-B2GQOaF!zU0M=x*SSyek}Fr=&b+c@X&Ynf z{z}2JM?cTmD6;Mj$eQCVFX+}u%e%x&E4)m7VT}Ncb$;V5yVQP>!G5jPW>uP7#_bew zA?DYXTIU^IMvZi?lMzlxFR)R2RL6Z!0r+ufOTMdZyNE>)O|9?V@<#xm1Fq^dw$~jy z`T$v8vI*uabE6+hrPIc*tox-e zSP_C7ofa>gZ;HlDm&k90W}82*aI_vI?}#!R0HD~2=~~+~-NxDCSAj9%Qj<@0@{(6$ zY^7v$rkJ*TTVmrV)3q^|iQ4qSSKB{%Vswm?ce|z zitQG&nNBePOHGq6{61BL;HE;a=W9-@l_r^TT_1n2QULlKy4c_(dg{-4bZAc@C5*y? zCVhWlWrnCb(@DPd5n zy%y2dHzg<;HbgnQrZ(@vo01eZap*!;t!?&}eyP|E!pQ?KB~93x1QrOe>4R}I6T2)a zD>6C*$07@Ar1AE)MUR+9lfLjTCbCoU2B8E;3eDL*VKtw{+>ZCsTXW18W1DYb$3-+~ z3FUAA74g15e`ek&q8UgSd6!FhiM=PH>D#d3hFI{$N|@^m7?-dptBBL&vJ-Y2vxLtt zfI!C7z0lui^o$XXfCb^bH7xQp)A5%t#p1|QBC;T5|6{ayr#Ykm9rByNAw}}a-Vzoe z70N3+Rhs#DsZa$K+IZ;#*!D2B)g=wSqmK_5Q%uH!lFz*wi)?lH?>+@F=6&|g{;y-O zw;|7)Z|Bp`@=e6GS$~K;8r{&K7m?w)BoR(d0_VPpwSLA}w|xQ2=IRAq&6yd+xk-JG z1uKurx~s>YZ#qc7X}Ej9jH@l9f^|~4j%s8-a|}|QK(z^jh?e%Jah$H+^gbv$oC%fTOKBRXeHP5Inh zoq9PmprzrD4_r9=jKkc{B!IjUNcpjiSRhA>;eGsfb90RrJ1HPDT_u}3D@K8{9L)Re zROW6U8T`#CEQxeYh(`_E`>V-UXOvn1CqqT0C4P}C4i$^D*mQS6@7L6>b~9@+E=(Pw zIq+ev(Nub}TFx?Q(ZWBsqyYT>;xrEK`h{UU+?^AXmG^_mK)Y}2Bj1;ngL=>xY9Nsq z&i2#hC!Cg=<-T@34fGj)8>&Z3No{^hRGUsRtX^bB_5Y$^`};}F9JuwBsekM%ATeFL zv8>~>K1hz$|MTikU-EC&r!!YQ^cD^0n)K8uv0%WoStAG$mX1gagxmh7G{2*y7!Nej zc_Y09b`Crx9GIB^;zlZd=e(Mf5GqbXGbGWtQ8Y?fs+;xfsfvDiTI~iJY+G~P9a~0t z@gamA7Wn3oZw$53!%sV=yRu25fZk>{?7N`w8`7YkO$O8pzT;%Q!;z>L43`4%Y6;Eu z>#X^02J$9>d{6C0wHTjeJZ#&GIytSSBymigB?+;OAj>HKh^jfTFh)je@g*t)EoE(h50YvT>h|*y} z8jenJanx}tD6DlG(~nrXf1W4z>Gw3rXRRCAz24er# zTb1kW4jqt3q8Gwc1oTvJ0%#YlB<$X}OQILOLS{9?9WBHQrUnKMC%{J1t@oq~H!8-g zq>!)Mf}T&Ep!dWaAPxD>3#|Jbc~?J#`V3yP!poDojQhz~DIR2Wgt#}NYi3ta4D1iW z-ets{9>E-vj?ke&!qomV`xA=0BJ36^w~?+@W5rqnzL*s!v^^?-GVE-ur>@1%X&d*n zfh5>Qs}cVny-BeB+QrZc4`TjOpE7d#2WF28f{(K! zz;o3o(=+($F3!*y5VIsV1|HWspn41js%YN{RNGYoo85!EZ|(@?&wHf3SIn@zA{@GR zv+^*D9yeSUf$Tu*C#1beDDo&8*tb8+-{NOmXiSy%)MYb5M#N~g3Y;kvkKnC=dUkp>8-OcMYNA) zHDj0HDC#k!ExyLjs8JV}D=KSe=QRi?HW*Rb9I$=R#CtLA$7#&cwcJmFKQs_IGTI}H z+7;i>CuArenkOI?-5|WNmfp}2%Sya)Y;=mOv#)}@Vfgk9-NG2NUk!4+4u(_|tI{b? z3O7sdRSu5;Hn>V`HDTg|K@7OMJH2iHhpTrCkF49ehGU~++qTuQZQC|FNyX{dsitWFArn>Yh+mm!>Q9Lm?Ey}u5dG0AUG z;GGrGG;xA9;stns-Yt8ms)M@jy9v6+98FV>E6C<|3MA9r3&yuBzeXJyV8ePmUI)cy(5mx z3-=}!=$k)Qt}fEolkD&Y73ifajnbSoShOZ`%%$Ry+1z2cV>gtZzHz^^?@H+EZ<&33 z66PIIoPT^a_W{HjmK;9Y^G*=W#G&JLmU zDm86dES%av%oPN!GEv;GBH80f_$GA%Q>bfY{4=9RT>?O9k;r}o>4VaoUXt}yDC{Cl zgT|Epz54KxzB+pk9FvL3nW_^(kIyX2<^oE3d~%)m<&b_%X#3kS#)RG-Zj^wBNLY&( zU1#JS5Z(`gMI}E~Dr9jCJWZDLH9qW6^DcLTn)WRjcJ#7|ED2Z z?ptXhJ19WDSYEm+k=>9g;|Vs#f?NZ7-!iBcv@h*`rf_0fs>2#2mhcAOFD?)5T$BoI zU==HbA)avA;Itt)VO1W)>Il(ip1wlrl1CyUJ*HK6M$7qo;nwAEMZ^ka9YsSm?YcNj z+Q#}FZJVDdaCCIpc|(jkjd2;Z)YKJt$oH>5T{HppN(uoK-~EG8?{z0OLIJOYxPTmq z*&HkWw{(SEWGl+IU(`Vcu9o9$XGb3X5<=mY2!THUpcOQuH|Ev+0T^|^iN|s#mD67CEjxMnRgocJa7QRQ5f{k(#Cs_B z6HFgrpoL&w?-d9#xwROl-YAb7(yEjtU#Jn_4e77Er>rKatY7Hukq)6)3$d=Xg=w5Ge~6|><%v8s`G%B zA1JVqbmV0a8BUe3bjjC+)mk6>4mJ$X?_60xv#XZPlfGtmrqo=npu_)#VdEa!(4M(? z8N}I~unv$yuH2n=L}2C;mx^joGuw802USPz-sT2zksBLCF@!%H`Di z{+UWNCk&U1h{GQAWJ%-?BiCGwcilEm1ELUfg$z=;Q~sQkgHJEFPzo{KD`Yw#V79y~ z%B~iT(3wQv!T9@Icmmr8T1Nidrk9B*E2ttJHnPpr%=+=L@rN zzeNzVnVA(8CW@T2!643zjRnyFtn4)m91f?l4c+G;z{`8uen`M9vp0N$RI`UGM~x`? zg?W@uOlcY-KablqXXFZ^u13XG6D+={%F;3igEU7jm{V-xkJ_Rv5N^#PS&DPpFe6^M zn?|w)Hn9F>?^j%*3+Xha#h2ko@pS3^0UQF!gt%isE-KeEiZp7}^Y>jv0CNKlC${(l zVZ3Xa@I8&J_n$NgI!P&G{=g-&b`JV$ekbzGoWOji_X&j?Mw!RC`2C_U&GnhTk&-v0 ztLO5sIy?P%K-JNuZ6AqP>+1>59BPxW2xH z?1nXT;uTy>9TNkV8=D?^>|7nczcoUR=LlVC&Fvl#+oEoUJayx206gIN?daqsk=xTQ zb18hW7Mgx&TnV@_UKYwe3ARUGmYlyb|5=m#a@%nn{wk(pMG%&=X6m(KybF?{RLuvN zpw8#NOk<%=@zVbhz_dwxe-0Fu52|-fUNncs8U$@(7_l zr&NxtDi^(wB9Dprv(epjYA3+6CTxyI{tUitAjT3*4t(vX9QmQ05XGACk%;ju=tdz! zwAY(%2S|ooIf9s-seKJv-^pDh&3|q`7J2pE#>vRZ-^U!@2PmR|2w2}WLa^OrdPhys zva(6Bq~Z9)-|PiGJVn7htKh6T$z;u|4Jw3ks~4J(o!dl1yGCH(@eG76n~x~Bb(S)< zz}2TD0#jB=_}R5_JI9)I`;Aq zT>8IGB@F@T0HeTgSq4&9`mQzDF2>11RyZ|@5|(`h)_sX@v~n~z(72eL5>aI$^RyZK z53}ZSI1wT^(3#1~fvhAky&_SK=CJJX{Jg?Ah%qK^V3`BY53|5DT;q#zoWl=PxoV@V z)eNk>0K>NWhC_jsmbx9{n98g3f(rDErp)RMnP3S!K+)Qp8{*sw5ZP@mnG&?3$=eQk z^t3KLp2GvYF?S#viVr73fg3;0)jfZes~x;3NP;$nT_as?H-+|wM%6l!HO3~{MkiNp zcr-sU)x@4+@l`9z)V_d~Yy}u2wsMzWFx;#*V0uI{;Y(e1Aka$GMPuusBV!uFK_ie! z&mdq8NFNC4M!ce2u-E8Tl8GhT)9^kUX7@eAPZ{=$G{xJF*eK6cc+W0YR5;47F$>{{ zImKK2O4zr{uveA}&f{C8fA`DOm~XDO!Tzo3-ClnL?T=lIil4nub1HmtVaXJi6Rf+v zKbD^#U0Jh2{XV~f=4H{T+3}y@GIIB62W|!{b<7?}nX(C`T7o=4c`KkT;ec{ho*YpU zX`R{)K5HSS+V8TA&~0h*E?iaMQ8CDabJ7fY3sIef4D6!RtyeYCZPlC0$YL%A+lJ;s2D-h zBlfas*osCzfORs3@QNGD{7|b?;Pw3mN3?1E{GP~N_XR;%k}?ogryDj*PrObFoI#Rw z)cs2Bs#K`z(J8N)LfFb%o9guvI;bKRfe>ec{4%A?RK_I>gDxQwc3OVb`*`u zasf+J6WLs%%-f_j#co6uC1Dkfwdpg7{EeOB?bRR&fJhgUVRGv+ksom|ke!we(BN^& z0~oX|HSG;CM{cqC6g5Kd{IKoEJz$=$y;q#`1{1yI;#^`yDx4!#=SRvO(95AXwo;4^ zjSQJXz=|H-QFwq{ZB?|aoNFS`8$|H^GUCBeiA|>lJSu29nkH2THJ51nDFFInlqZW$ z=g}Gqh(fnj{S?b=H_a{}zEnBHzI!-9&>Y4dx)5)Y z#Hk|W)X|yPWDHz66M5N+MizW2({1NbzJSkC5+J!a< zio!`WRk z%br25ZiQx^b=|tpr5wH^w!F7J)ynF5aam(zHI2Uv0P9MT{EJxM(D|GLOo-7UHRAY)XntTJqk{rmvy6xnE#EeVP zhSf=mgF$0tsVfisj07sFI3t;3z^7?C84QkVsVNJq52B{3FBJCCwdNX$4_0`|51x97 z4vPPj#@-+SOi6~us`pKN@C){tIU%?GDn305JAJu|f&O6N#XnPeCI^SM*mCdxGHG$6 zm3n#T#f8NnuvrHX-00-c3_r;j@@cS2aUVaxo6q!_*@r!L84x0!=D^D^)(t!4)&pdS zc}M7ilW0t(V9|!Yig-7Ra4Y>$bg!O#FIL8~&8Dw^1T##8KJlGxpzoeIBtoU_y!9y} z@VB3G6_DIW6>8y25Rh3uE1!jooN*_mqPX}0Ygy!;^eYm;zbuNpbl9Lvzh?m-g%#C6 z;ZK$bYT}eCn%esC=pe>TL@uOI8TF27s~Mt4tQHKEnfX$GjpSN@9*QWc=PbWEcDhl- zduc)0WNIi6Ef1>g-@p5sHNTa6mNs#J1?(~SLTG$}F#XLt+I&b5)I4@gc$&e`nrI%h zP0Wx~&X5EkV`wTg{_qFM#0#A)U|Jv$oF3j79a>kx8W!%4*SXJu=?tfjYZZ$dc!ct( z1Ib$A9c%(#jPDf+<4D`<5UYLWs#5bzcZ^gO#dn1e^7)RKG^RF04wT=0)zX{4mfx>1 zp&U$%SSQX(u+N0gDIeT_#(KQ2@6g-UZX0o3Dqgq(GI(aVj!GO+N-pCIFB9^Jo*efj zlaHwujx1sRKyt1Ji+XO))p&*b_jkAv5k17mjk8~B5l4!8p&GY-CyppdqZ92n=AV3G zCIsYptV_n|e%X+OvmRpVpF;_`B*AH)j?-7+8>}@Gi3w>7_2{d%X2d&EEwV6O0iFK>iqy%4x(eu;N};>Ua&j#BQB`mZvU7+|ak$T$#*)yA=wu@F+k=mAQ0gJd zsz=)unb^kmGi$A>BVnMB0O{QoU3K{+BarQE4S`MT*tCFiAjdaZ8a%TIEb*1q$f_v! zacmb`-ur+W6ezzW8;FN`^R`%9I%t<>`oDUmunp{B z&=nxuCbl6cAJAqKTUgTEshD(#dZgK4HV*^;xPaM^*=?TeoA;y*uS;W$!j{Kz?@4s z^3+E-)E2)5I8{ibzb77`yy50P+q8rRUcaMa0(;<5@qqX-Y;b_&bp@-$jB(noiMRLDM#u--}@Vu#*$U!4AoO5LKP@tNzx$-s_!N87=jm# zPYGg4o_AL^t3~nhxI(Y5O!Wtz^hsvA)M2Bz!OYw)t$DPo{|6V~iIMD#WsHDO8j&a4 z(JcosfPPLKff2WSidwQ+FmUyN-P{7cx*GU)v2Pn|p=PLf)u-Hx?CA^ZF@F_Yev?#x zdDFpsxy+o+%Mp>73W;l8O2~3&8t|^%+8tmCEs)vctp_9S|Bom9@0$O=uI=w`XsDL? z#Hm3*?4UtfnAYJa!T;S-@JOiO|L&un;le<3;E!8Mav<_9<(C%0DavH1e~rYJD9b?q z*;8TTz|6DHR*s>7z0<^)z}hef*uUEjh_pX!y@p$+(1zgZ#I{r3W|^h<}S4M5vwMh<|@U zc+;dT(r4f4z&d#x3}9|F1Oc!rk`1p#9tQ^PLCRo zhOVyW%lpsQuj*8pZr6(+3^-DEn(S?dS>N7Y{BN7MgkJCio^LXYT!{4uSWQ25MNhow zzf^YhVXd3~(S1y_sHK>(meBhmylt@QZ|FXQbyDiE*0f|7*A`DkJsQ$ja|kB`O_pA< z1(^+>l%cUJl5MoX+%ku>IJEkhZZ^a~jq5^Rio-+c{v{UemzD7KjBD?mu~SnR*-P?M zXpr(^MwC%&fha@yDO(QfsJPoX@WvzMS>~Vm27|wnlYuHExoJx_>jR2eVg(O?oB5IY;x8I9dyl&3!KfI zr9dR%K=CFX3BSZ}tz17td;Xm$$UYL`3(&z5quyWr(k$en)PtUX^3`5jttPM{%4|23 zh%v7q!NcXoxCA;zmqK^*wCtb0mTq9#Zi>=-v(U+l(fHwxs5`Sc*yI2R-8i$n>&++(GeOV<&gmYTg!}WdzMJjKh&aolu!HfkVxq-?g-ZSu?2<75DuQKyQ~(XsnsQI z!Id|wbLXP$N**g=l+-IAcrY58>!Zzzv^{6+;@C_I7#CE={ass_2&1*D>o-bF+H{!< zOf)=V*)ABkY6%2%1IZRbVOG~b zlAX$020~A=us0}7(}+Ij__lrin2Cd2re;3;<4O*7UQz=rq}ksjw>`zxRvijHa+ElM z4(5{JOS}3GKhXe}oK85Yw5*pbwQD+^z)@qfwu#V8^v&jMN9SFOH5(&H4O;d~#X>6m zQrw;1t%KTwpBF|QD@Af!eFWY{;W|1A8nYRuCnOZ65P%ii0*xSP)u@EWSI0<3?sIz%@;~`Vmq-6sPZuoy(TTL{K-?_z8hF%Rk zJc(p`kMnzT_cflKs$hPZ7^NJGC(A^iydoCNFBKq9lj1PA+UY;pAuB%%tpduOV~7fS zJ+k=9t^2LTyd>XPn3=`G!??&fZ+9|d<*23Qn0d}r_v%)08nN_lNPJ2O((qU1-_Ao} z8s*(&>_P!B^Yp3|;PmR8f|<}`brhOsd)SVg*<9}rHsa+l$LFdVC)n06%b1ZpScc?3 zSf}}?;S|F#z)2LPjQ5`10EB{c*hz_tF2~ae7ygFK4xASny-CIx`aOvJ;cO24QPY+a zR;kr`Po(R@kVSD6=GK^=t-R+W8K?DLFcH46Os#-xegk%9l%!;lE|Xk{-Gvidx5gIs z)z&M-gYK`5zp!9`g`W8bF${QsZ}!PTK2dhblU2iqdqDHo#v7fO2J~m~hv1Iw> z-%LeFfuSgG>E=M#CHsO;d~kMyguxXBhJC&{jEqhu#Lm2#_-M{4E@vLm+(Js=smfnC zIHCmjv5tWE#Nw|K@a5ac%Wl;djzt;#(%+}AMgckZE5V+=r*v-z2ql!LL zlO`8N6s7lsxVoK|0m6N#X2y1LwHq4vG32=Z#Pk}sm1tbkan%h~Y}{^g^MO!Vzi|Hn zC8J!dQ3G!g<7MNrJ#A+RPBPRWlV~gQj%WjjEOGThe8m8`?lL`f1V%a4iM++3o42}A0yFAwEQls>EVpm3ZXjMj2$HIXdCM`itS?@jw1`G zHB24sGbopVD;i-iElL}^%fuODH_Or}NIs}sBwINXhFw`YhYbxs*n3`LRI=7)r2G{x zLoIWX>ZVj-O?>T=p3BZL8yZ@v+waa6;UHezs!U9%fO!_nD3f@FoS#rZl@x7L-5^W8 zF-V?SUe5Z%VLmTeErS7J>{q-Hgl8KVj0?@H<31>&=^WlYyh0Fjn5UB1#OqxBh$A9wN&z^lSFw!a^pM>{>nc zBX+A^cp7~Ht1q-KXog#;2_q$)HJEsSs)b%t`j{#Pi#z5zkkV#7*7{NqE?7hjgtqeFA8jKYrSm;K-ex>}V@-J$<(N^M~S8J#0U=i`p{rj5!WJ9|3hl=7#7Q_?~ zwk@!30_gBn5AO14LI;EK5zFe1QW6#J%5fM*r2Bp8Zk#iRehv=(j7e1md|{@a_@trR z$!=FKsKt;wkVUN4I-RQHH64B?HYI{{>MpLIEmo1mMUpSF!EeqP;A17HZrCVNzBa}p zQN<=A8;Y}z&J4fO-^aP6cQL~5;utD~h%JOT2KeTXr>JI(X=IKu>X@h{%2+ctnHHDQ zH*#acrW9to%pdth+!@T={kHefZ;XqY7RfX?E_GgB5AUSOs9+ZgmCpF@fhphlTXGXW zMdA%3dN7XTUI5d5*tv@)mnw|*Wk#DPAir7W_%p&mx;F5e`ZbWyZC=^o2+$j|xa;TZ z>Z}a-_XOcB0lKjzM21!luEmU{enMT9PO1Aj1DePHhz7--(~@g~UKG6n z^Xo_SFNW8T)~_QGGS%O%##r_=Q}Li4@N6GpA@SVc9v2) zpVnD?G?Cwxv=2^ti4=DfJG}~=?sutqBN}+744m9T=KuuW*MD*d53P^rQfMM3wJ_#ltm)Mq{>D&!QkqmhV+t_*aiB&YrtK%qi@U8V3s70 zSP4cIsZoxJv#O7o0|+${S+#Wr$o$*H9&Q>{i$@jRC77oYO2;9?;U9VRmq`kPSO}-J z+4(FF4llOwm;h5{=;_YmWt^`ck3gkv#slr}*dNHucr@MBWoJo2Nv!RxEt)beL_sAG;d0j=q<5YnGS&ajFNvUa66}8FQE&<>WO~-vRTp? z@hK1BUXC~{pu0EiPr3wM;&iJIoZ>3WJKJx${mIxH;p^PjPX7IieF~S^T^tl;(p_l% zW1#f-c#gQR-_%%y`m{ipvpO2I(hQ}ai}bBL5|77)r}}V`N}0$kNJdy9t+=}tw924) z5Mzi&Uln-+AU-tN4W)R*JdH$_hoWWxN;}l24WFSNm+zU~yeicPAxpLR9SxEZ_YY#R zOUs{JtEl#1H#ZTN>NKTy>@~D3Xia>+$0b-r3UAs5wA8_f;DV>3cC!yLpyvL;sr3e; z(E)I927!}=AU!0OPoM(`Fsb7_b1X*pMRP9Pocn-$kPyT9-l#3Xqp7N`%C#njb}J>6&Jqa7uQ%5 zbsg-9rv8AV_bXEV6J>9mY8RtHX=}mAI<(k)qp!`4_oIt!Z!Y)OSs1Bv)dE4%i(@5( zl_#A$LLL{InOTij6=<=GO4#T2)LZqI%&TZi|_Xb(<$a#iHyWVZcLh&!xtkeFz4VzbI?rn+^qu_~DM2r5; zG;o-&;I^()b5tHPTrQhN96IBAx0Eji5>oG*r~>7$Srxf%I&~zJDLuIo$nbaA>It&s zSw$kM=`D{kX3~sUfC-^j<%N`E3eE8da)mE+5`n-oQPoaM9Oqu=5YPb0geC``xJHaYQZ*hZNcj`Zo!Ynwe7E` zPp@k4?IJA_0;7v1k|Hi8vF(Iy;|_j4XE)qvo~q+X*bFUmbB{;)PY^?6(0joi0A;;F zUqtWk-myPN$grv6OzQckD&HYeM_CSjDiV__&gWA(RqzM*)9*_zBMFea(T9|rOS#7q z6rB9ho=_5?E>0i}~ED&?c^;X~(@xlK_N0mOcF{oE{QKjM+u~zdZQG77g z|9K>ZUkZ1tK<1Qpkx06lupt?s0}#CUTHd6j7L^v7oogmFUTr754~ae+r8T;uGb+|j z2YF8$*0BF zc~_ce=75ir{%%)3NJpl6eq5Qn$;CRBlM+bZ(-H@Q?S0c$2qc<~g=8H2I@7hLqpS8{b~A{pV^V(9NB zBca=$n9rjb!&a#Mz2fLn-nccb&6rklbH!?3t1HIT7jHPHq7HP4nXqQ@eOmaJ7K7MT z`31h-4ZL6ptJV!hNvOBjajw_CoiDx|$vy%dbOP4DY>WS5oBx0Ow*?oGaTNMrOx4~F zRRZ?kB7QcT>VFVYL_9`5h<}h)1yc=)#!nb^8R ?M#=#6btrfm0#2yq`i${Ks1I z8e$jy-!|-EAnpEzhb6bf-=qIhI>3PrZp2)`>5@-)R>%>)>OUVEt3!Q*_y>d~D{PX% z`vh|lA%KBxsGo^cjEE6{7dE^wEdh;e9AN)8e3?S5{~vI8JP=+19OW+z3`~*!e)ri& z&nL1h1ax*I#sl-P2Rc7+q5)LhOE4c; znN8gvPk#`51PS4PDoGoSv?&T7BqJp|C*ziZeg?KC{BS{# zkA^0x^{1US&n@DC4Qo(^`w?E>Gi6eolLYYqWsC|fO)DxNYLHflE@g^JMF5GGo*jQ` zWqlslBa=oHYbFQQL3<59?SHZIalZ1QtG=}&5`*IWLhrhuSy)iEfWxs#gXXP;Vx^yY zl1HgD`V5wN?%)Ynz>AA}3GWns^jf}*`-TR3t&!X2b-jVKqr~ zwDZkHATYm-1M15Li2W`mUM6XXFz)YJP@3Lct!T zP;D@Law@@+XRzy=4zzUMr~1n!`M(S0zhU0LOK7royR-6h32^{>1=%rxGxogrz!N-3 zIKb*Bv*?Qe9-p(V9`5gC6~=v1Lfnv@9@}ymbx>I2Tot8Fu7@p$Chdazc9jn}1CZ+; z=qu&#b(!Ib=n!=06K7{xJeEvC{{AmWTQr0){2JnRBW(?>E;7So-1QPCPfD_nUA6O; zcY$&eLx@CzI8a}}Y7)`BX-ygA3A=w}DFIF-4$~9o#%M3xtbUcvC95{)@T#Y+!+!^7 z7-yw1lkjNdLOq(|g5QF-`{*in47%|;ts6(@lNC> z*%~C3+fWhh@(pZ5;e)zQy)w+IfNrb9R2>c>@uN+*3w~Nd!f01z4g`;++B9_ky%=Eo z0EXi8U8J#g@D5sB0?bxALB@}~DiKZo(uS-ygId(F{5a%{zJXt(Eup#|KYTa8o~Bbt z{@69?BwvjZAb-(10+A0KinkY2jv#4hBKdNqdE<5S?Cl;^kk)Cm>3;{gB>Z0TSrM%ZLYjB!FZB z^hRwF9ZO$tK|d**m?tI)kK~oNr)t)C-91hpVg(RXqH-p~Ho0BU*zu)=$naS=N%cjE zwd$JchTjOEXxX$hp9aFHKJF-oM9g?@d+!=ZOo_%iF<+&g7N?L_XG)O+MTU#rjVB8t zi|(1_sP93Wd3{61av=yMVDusoaxjsy^OElCc+F7YR_hA&u zQt$~8XlOxbRYg@4r1IvG_6T(%mG~ckbP~?g z=mtptQ2dD9t06whje3uCzZnJ-R8@Se6*%qv^f&fMfh!H1b~yA zUQ9K)a0J}3-{UtDV}qF|X8-2ZtYKh8{&g?WmLu!QdeJPH$%uue$CSWh_6y6)+u@QU z$U*G@Go!vNY*k$4SJ~l28F)qKVI6k}&AxBF$j^Ptm7qhl1{;8aqv-O{Zj zxkUB23a6QsyHo(0(u@a?k$F@?-gT-7E8WW){jQrg`jXfqEaWF5HN0z7WoI}`F!V*t z0lmxxnFwKiwCO#}qK7=ifhZYJkh*Bi{Q}DRz`in8xqV#|jYcZ<&IQ`thzwC09h4t? z7e)&M_R%bROfMN#DO{i~ zSPK2}fcCmiU4_|7o40)1#Kp$xX?+Ny<=P$%x$m+`mR2p^TTwz>J9ZtTgof@FW*3Yd zoSw~XRHD97lI=gt%Lc}c0GeL%(Gbnp_hn9>`BsG^Fu%!p9x zW8d6L{-8t!#m!_FTOvJzE|TQY_~b0iSn;kJ_+bfO zL4;(o^uc>G5%pEoRoUzY|&~-xuoX1>XYG{#7l9zinYzll5qL(*$*~~T}PbP6HZ8)4vLc?CMp9z>C~()o%8& zCPHvjezDosE)IFS6Z*9?`#S}1D06+m{P$fGmH_js8c5!c$o&pqRwE(LL`mRFv|}N9 zlt;*V_!?BYlVLGx0dTQBH<+7m5SF$M?@MMuL=jI^3R!c3N)gnFm*{rPT{nvIwjXM^!rrb2^k-85v@#(mcT!@8ivDp)iKJfNgTv^2I zh$%1k!9HtqWlerY-3b#A%GG(>;0TzwurOHOmeKTfReXLIclRv2ae}td?amDIB5K|jMvE4NI3h5zAW_83BAizt=E;Z>7sX-YT2ykWT3C>&7XWcf z&wnWqR-MOe(M}+f1wi6rDq6XlF;vWJ%%FPj3hSYXG})zL4T;dVA5rTy2>a3p*-WHI zu+p0orvr@HCPSM13ZqfX+#8#*AJQ#&6B6ff7iww$2rp7;+p{6zF1G&`z+GH)i%C!4 zbLtS(G5yV91Bf}R8NIBzepo`kSW;Tt#1%$wVq<}4LA*u<1FE%kBT{$%1u|sw1H8b!) z=6gtk$SuX86Ir22CR>iwi15U#Q3>l$g6yPFQHLwkUJg>4IZ(&)jMit_tua-nm_^u9lqJ(ytLf2=&ep!LQms^B()KTP3w8R+!}8H> z<0sdqxJmVC%%Q;QN_)yK%lu&ncs#C5eFrFzrk+mC3<_;z>m$B=2+c$WckYXR2DRT0 zAr`^G45f3ho@D+6^=QuZf4(u?BOpDY^jxmYyL16LXNslMfzVV|54^^Jy>t%DE3AP- zpb(&7PQGjGQJP!QHo*pX3`m)4bsu2Xl^ds;9tHEoG};re!_lSGG*W*IIay*C0rKMH z%1ftZ+g@9rUXf(Ar5)24=+l~AxxZs!kH`1!q)#2uJnP#tYFx^4Lzi`;>|L2)Z}vZO zgs?W-iub>Y^k+5{b5J9fqZwx<_tm<=303v(#ihD$7zbTkX0<RdbcTJb46`kmysUW=RrX~F+I5kmy5^;` zZm3fEar~=_9vqo{H8@Pqjc+vk$J8MKz@ejvY=V=`+|5i93Sl@ieLZD13Hs^msw>=& zGv0cek?)i|hM~ApOh=WbUm1sWF0g1%D!?8}gnF3cyAdb*NN_uXrV%JD3t;#|5KH|A zu0U^8z_-SEAFO+Q@P|1**1*)xNxLWW3qR(Ay8buJD5ilEgF*L_HjP-KhMdC_V;mb0 z3U#^{$MPn)hoyO!R%Hppyui|F?@>e2!R*I=0q}B$`x41ES&VeS?l0s9Uaf+ntQx$5( zg`LrzdV?DyrwD7(u>i!Yq8##71Q$CZP7`bf=Fu?lg{GMZ^jq_SIH0zqj_|z$^)`` zmV!H-p9h@f3VD{Pn@^hOF$}@xB%V9+kInH+H=;U!%&OKLfUscCEsso?z=*HF-Y!CU zT%WZ9)*&eJ;rH1{yub=A+Yz zkc+Wg-sFzPY#T$IugTV4&;n099=rM*FNRlBABmoTS51G~z8l!j+=o&*$E;fs{NULW zY+JV8Q2a@jJq`0lJ+4nf;s{S1#vfq_PYgI$e5_lKW<5ls0YV9&0Lr`Wcu)WZ_`3wu z>kot{NfL0dpU9i}&Vhafj}9<(Zh%MpSPVe|5gY=Puohvt6tZT}wa~9Sv>5>ZJa2(tZ`i_B7p|DzqRn%FI_<5@a3tky+{=MwDy50V3g+WY@Wkk9-Rm+5f@+E)BY$UrYUMKN9jpv+ zwJ@Yp&mtB29)S(TT&jRgs^ibZkOAh(XVL|piH)@Fl#CEpx{L)^e0YzKhrMilHhO>^ zD>-@qv7%P+Mbw}uEEXBg&0*lZ(mt~xOhBD;?BKX4s4drR=9(lBTp0f)O3rZAWukf^ z(8O^`YAN;6Cz@pj?@&H9OPgRf*?1OEOLq`@I-xJ&=W>dKDj372oYg0*d*WYyrZe_V zZ*0iab_=D>pPjMRW&HGe@4zW;Kns5aW!jw_S z+i5RWox<6l`>8$>^Q=nT$KgvJ2_r}w@5_ltKxFxbJ_#lpf;W`4BU_Hbsx!d@&Zhh` zFzBmkFDIgB!Ih3mCbN8cxj_7q$mqsVO}Hu_U6%GlSzjFlX{zUM#d{M3u8O`oS561# z%6<|Zj~08Jx@@rpU(g(@uds}ONAJj0&ZB30>ZOF}T6<1=(@|0%8xI6l%t#Qv=4LU! zN1^qAl{BVh;Z?`q#4HZCi8I&Mfe-&5S??H~S-@=TreoW-ZKq?~wr%H)ZJQn2wylnB z+w7bD?R_xL{a0iCUA0E7s(Nb9!n5+ETkIqGK-7?Kc@h1Jz;ep z*M%=4;RpR=tvyjb9XI%~_r7R@IX-dst|nMT2#nlYPRySu!GMzD_m;@C(h}huE3M(T z50582JD)Ee3)suF`Bjc-mZD>5czz7--p|(!oXv%34HTHXv<8MGb>p)=WN;`p*wGUj z-yRAo{3ZdSsxK_0D+Y^&ZBd~Q8t&#^qT(++t}B8u>wiO_C+f|BgUFUptSi1%WXd~+ z0va{}{OU4oVSq+hPpDq(*4~itU#t8+??p|(HlB}-gnCRORsn4Ycf7E<8JfEKUA;G@-gu8LjG{W4 z7#*WM_i_uP!fD;tv&;A9G3nVYC%y;)45Kv_8xC3}4d%qP{vP^!W`7?W{+|n&BB-=drQE zda+VqfcLBbQ&P^L>=s47R>8QjGzRfH$a(IL_KhOwmV_;Q2<#c(hJf^9$U0UOK549S z#MyDgs5(wUBF;oO>Y$N31jTjjY?xVF&@Vs94v0k}|FDDBBXi1bL~kl7j#*Z*McC64 z8EgxY2fx#{S9*F?sUF0;FM?%l6}jGfXf0?I621>xRp)bR)NFRPs8>9Cz4$(1?CJMJ zD9V3$j=PsU*z9Xp#KW%r_a7JN|CG9)Q}o{jKkun7YM zbc>y~_Y94ic5D4Ztn;5EV*{!+U_4Nl(7taj&CJ(eZ6rX~emevJdB%e(A`1V`O|SqW z(_O-57 zk^INgxW*d+`r&JGGQ$_=jsK791YSH&;2zW8a)5$yP)214Zo{rzVhvzcWz?P^^QS6Y z(fSkz9lHgo9z|_W$T%lAa&60!I=IhPm4@27c8=p5jftVUJFV7jdWo5|LxZCjQex09f9O0nRWr${xT!9<%A6V$qqaNV*U&z-d4Vi`qw$mzfvysxi+NnFi$26bv7(W^7O1pUcyuP(nfIenx-SN-m!h3g*|LL zm7*quu8tEDSdCuo3SCmWRf05Bdc}sg2(LU87Uyr9tJP|@d!nJbp>#FL33nxwl_ zPDfI}lw#$-x8%Sf?gvXu!hWZTb6C_!Z|`i7oSg_Gzy(6^*8wv`u(1NDIM# zZO7FPvtgGQivgJ*;KCUvFm!XJJ)h5?LXj?8q=!`4$PC)R4JOc#&7eS@CWI%*pyzVA zR{Yjfyd#_|HN-!LKpX-NxGxlh>|oL*6|K}PD_u2Kp)COnN)Do@^D^Rc3-=i_^o9KT zo3A!#5uHG!F)`4HBbz#Lsay?ejtMozE9x2fPPZc1k+`v9mv-$hX|V3kc;lW69gWw< zu92hWsSQPNku*?)G5FZ@)dC2c{^Y|eBTw?{?7XwM z;1^Oxu05Ppe0DZ1)+UVIqOr1B^T>VVI!kDPutrl{dMp)OaMg*GT`jEoIH)PS zDWWf96I_1!6y4bo3`thDi&Q2C{OTkL&X+&b&HtXCpllL$U;l(IG72qU%K|UTTdGvH zg%r(4i_;cGe6n8q3@$rlU6!@(E3Y0h+dR9GrbAsNpqpwyol zncI8-vgnM9I!)L)iq#ca*0LqA<#Zh&%|wkt?JQpdaVepDPEqxgjfx5kvo^ABoYE-^ z_v|*QGI|Dbe`kwgKY_E1^efO^xK)IDZ*qT9huNt&sj+HB$U2=@ykd7t%$jk8vIgcv z_Fe)tt;qddOU2UPfor;EbaET#2rk~pXvv@etQdUIoZ0VfFk2%)_QgS%M(_ywN$fsu zht0@m)R#@o*tH_!l3qdOqanPs*O*5xqUnzFq1nQ|Er?cHTc^^WuSqV-Sz3&K6Rn1c zV|sRLsx{(z`C@&5liK2_`e4LKCE*u=&;suhUZz^C+UAT;)Dgrr^0^Qx&p4T13SjjC zhAm4nBexfsdEj3k|nw9uFL5tpl9vB1yUoQRy&O|CAO>3M5Cffw1t?U;#+; zC0GDcfv_lvINY`f_WL@JD9uP)kqZ^zB!#7kVykwnS_@4o&QsBIhkpoWVRRfC_P5>! z5=0^mS&|I3Tf>>1)}b;Vlmilu_EJy*65Jex4lI?i>J8`%e&h-kXPr^m!q>8nB}un_T3LIjpI#|U$0~+1T6a(S7LCrV^m9#>L8qv>JjdGWLArR^gHW5 zVm=L1L8bYkawoy_8|IQ`%AQ&OXh;;Ggl0LB2V}Ft&ZI(h*KYYTQb08KQX@+Pc9Md0 zVPTBAR4ZR;22t1&OH4&r)R(xmTa_i@0BO_v_OD?CrOsw`zf60Akhk`(EHc^V#Abmx zLu$p~?DuRn@BTfLkk+~N9Y8p00d}nJtK3aFxJDkAGA<5aLx=4yTU|)T+txIcO<%#gLV`s2W~NjUfNsAAJ!3-~8H~Jm7(_I`$LromaWyk(f67ASG3LIXI*y(*x5FSQs4R_uqg2 zardnI(}c(JQrJ*sB5+Kpu@%`)27y>*KRV#c*$XEvHSuh;W>;~$&k2y$aczT~{AI2h zhQ=yhqh`bwmqKG%2!W>)_U7`&Aoox^Q~|60i&rU`zRbT3nLlBOvL4HDet@B;@(s-u z?l{~!#KcH}8xyJ)T2!IMSiEFgXY2-YUylhxfF^mWyf}{Ehl4of5*uu)SG|cmf9!vOc8Qt`W06lf4k|y zF-!()(8EnDklsuP(A+Fv*5U@UgbQyP{t+Glu_{hLN{>%W?}miY$)ww46tNm~WW=Br z$*x1ZZW!^^CdV5{w-00=6UUWqDcJYIA6f}{uMaV?dm|FJ>Hit0G(}bgtl*7!=odyi zuXp)%CAGc#7W+N#q@RfU!XL-8xT={tVjOSyr8GXMF=F%qXb8;d_Q$#enKB?a<^5Ib zk%;4No~-?s2n)Ic)qc>yGjWPGsBx%sv5_T7O5IT~uL!#1XLw02g*&3)2s${d7#my4 zq{Vv>oE#S8Hd8Crnr?qfKD!2N0=x$SP;iTNrgCL6ywbq;;ndqWSv{cJ z$mcAzwf=(t+xiRg#z6r4#n7GxzG=_ERE?761&V%iWoa9$2ZU3k#`jgPSgJqdg-B^V zw3Pabi0#nb>yuq2$pO#ug+%0zz+_0+uiO|SQdpA>h%Sr=sMz zz&_mmhftTl0N1N#2+1o8MCy77yLhcgmb5ntpd#r^n@gQ&`^0iILAZr?IA}f+11u5& z1x(V}_)EB3~)B(mrrE+JmO-{Xsng^0n~A3It z*+}7-x%Hxk`rCh7)%>r90QnaC|Lq-G?Ji)F{s)~xd_n&8-{@3JH5UE1MY!;c(@XxZ+?!RFeMqF5!Gyz~v1OO@1|Jux_B>e}B3H=u#EUY7ptgJNC zWT3#Ly6FHFo)s=7@rydrtJg8jjmErHtCr?YL!$b(&Mgoxs?lq>qJ{yox5t@@(^@%Bf<^spw%st6aCmbXq0F$)p6OxL7uHybg@m1z`w~Fa}-l z((sZ5lF^9{EaYSLrC>=%9%8eHYs{XDR12vMIJ>sl4?A=c?vt<|LfwccJFbP5NM+aYH5*bnRqM6UQkv@CRyC!91KwsWu(Z;Ij;#(2Zx zW$12e|A>cEx&^C#KlwdE6tI|0bXsvIxYUHe3+#ncF*g$ zsf0Hqlc{+`lY7`(xODo_`;*B}5Lg7=;2S;H)UofM_?uGrdIc7{e1QVK9%=0S%0blM zE?1!NHV16;mw7{P{hz>6`}}8lG;%@moV!}Z&X^uz=l_L%lr~u$XD#xQRJ{q)-shq0 z8se=94K$2fN8W}vr`y_2)r`pe_tyG*4mO%DW=2F3@AkT%Nz zhWV2T|3lL$>RSI7p^;?Mc-FD5RlU0kVxcI^9Gb!W{ z^kSkm={6d7ujSsh26j-h?xn_+@vXmJxFufU<$og)2?3dW#NMU#(aZuFmWiU$a*!gdz$cbTz()tF3+r>(~6Ny#E@(|KD%-|BT=pf7a`*s}ERNzd-*J z686!T0K@-RSmTu@+JOrMgu^G&u2UJBNEHqu}k!>sQ-Lw)5=U%)gJB*4%NJ8-Z_D|MK-! zAbx3AzsvlLUjV-;=v}c)x-G8zw$<6K_JxWxwIG=cimkT=Ti69)ElL{t1O(Z8Lq&gn zp=m)a89G5NA?c&R;sq`8>Lc!INCVL7zm}?qhFaB&;z5)pY`IC#0tKCF_-RnhldkU? zKnG=Vr8FiEx`-$i-3pNt#zE1|;AJr~XDu+v@Rqj8;s6=^ER)XZ`_p8*G~WFMS;#Lt zPaUj$(qb1Wh_bMj)21a?IFP$K4?;N2db-~mXVe+)&9UyLhdU-0aWDmVzdAx|QtIW$ zf-epSdndy#OfSLL_L7*Nsjxm=fYKJsUEXbgq=PGQKpwf?yQba(r@jk$dr0H5OfHXs zwuE?!KLDmVJ%x+vAZ^j-J}#AA-beM5;;rK!&p^uyc*M_J4YzVR&!7eiy39`Q@z(|3 zK`sPa3_H)~kIg{riy%L^JinJh{w7ft?_rTI@qoR=iD3?Y!r2GNBPCw^sPO_H<<7ec z_hAv$FP&lUeHa&wQj%X@*}1p#Nx!@zb5k?KK>#k%x$|dPzjS8v>(yBgwE1}{Wqh?u z&0W1>cP*n!lQy8llzHvQPh7>6sgh>*ISUNxGo(a$ri#Z!Xj+ja2ZlV;{rqDgCuvO{Q~$cR5-4Y#XLPg{S~=&Bj@Ip(NkT-mHs<4?c(Jv zD-Yma)~=Ly z7G0FYOd&FlTbgRz=AE{ISCN`$So5NEE-V44M-9~NYq^4+ynnQ#kJgsYM-){F4638W?} zs)5~9Y4E1z3Uk(ex+py1D4m%!u>rPf^U*3HuRSB!!4r3Iqq6AubaAE*p<%QV8Uj@3 zO=RAm^Le_NQi%`qbFvHv+7LqfnIl@7*J4BBYI&pf$Z*&*s(-Bdg6zg(%V<)W>GT$) z`gL{7;rjvDXt@bWRz>dGl|OTWi_C3ZM`son$HVh>WUxn+zOX(4=>Vb|EqcU9 zJ83>BvU(5eh0FcsmJ#kkL-he>mo$b%u-to5>m$7A;Z{=itm#5|SbjS@G+)NX>b5`A zydm;Sf6I?-pT0`V0 z1MLx2Xw-t%saNEs+0_l}dg^$gqW!6r*23~UH|a!;g3~u~P%KNI7FiiKk*ygU8b9g+ z4wp}%tKRF;mGmcCxM`x`4Ogu%eDTnyGx&W5*`2^kQNaD$* zvXCRPjng*T+jzF70UsTO8<(p@vN*5tF65XE>v=>LFAx`KrL?=Yp(o&FivWjA(N%}o zFqTuS!|jst@<3J9hE=r*V_yB@?H&d{XeFdI~E#Q$@QR3{ydB z;Rnarl5y(U4G5$1B|z*Tf_wT|%4N$**43rMh8&ZkyJr$}V0y9G-rrqiz9ZR*1>Xz> z;!0D}`L!16^)+KHiOZ5s>5xRDVwWJK>7C3oI<`G`tMFz#<#-(S>hSs!QMB@a9$c$fy1pT@l{f(uq z>bK*-?s_hiF?aWvQlhI`hub^^!^D-mIu&58qgz#uf^p~a7+xGB0lO6GOe3wGIyLg{ z?>nb7bpWE`jg)Wfg5i2}W*wXqKz0&h*2q*=_wy)4;Qk zV$F&^Jc3s2NQYArexPsgH8ntc-g;NmXEs0W^aK354`2)Kb{FJFSiMpFB)n;|tBK1| zr>!NwXBpcI-W|OH`;L?lfN_fl^0LpL^Szk=uTb5^Ybk%_)HTP?W=`G%98Ix6P#{I^ z7aFn{?7j_Lm*CLMA?tTYmZ=#8%yb{%AS@B&sKfz^hl<5=Zuay99DfuATyI1dyeGh$ zbKmr_7SJQmZ!1H7^qa(-5i})ms3pBcHoWPY`4r<(8pJ*l6cC_~{YbM}v{(!G4)?X3 z@5s8H|8TT1d+QuDQxpNN2Q2;4N*ur;V#0911yb7p>Urr7evR6H4Vw1V%W;bE%75`} zT3RZ_-EBDsSk%rAGwoSZ4GL(zrGWSA%KoN|0|;Qb{91}^OMlD};r?37@14_!%kJxi z^~(A-+g*8$68tB8Gqw-FUL}d{j&d)aWc`j^fh^YKadpeYo0pqWT`VaYie?puT|6TI z|4Q0sxh1rh7pB}HWO##{IDqI0g>ZS-50XO!!J>v{x^B`vPr~C%j5o7oC1$bo66x|%Ax$z&-RTKuuBdd{58k3VL=MSOM~ zc1#^%S5@67cnw+DD%3ZZw#-z3#)b_!;?W(ByN_(B$yMpF3DPt( zy|6W%ca!s(SiqTP&eGDPRV^WJm1U-+1z4NsdD4mhVeU+1Ksm82nJAQOj4p3>g+`*2 z$1$42q$ZCRfP1eHsk-`WV6bbpS!n$hs+Nz2t(Cz3w@+^#Q?@R899K?S-uz@Ccw#qu zk)_E{E1dD}Qsj^IAe?P*d=9p>j1mmFqalX8D@Ed&WQ6iDH3g;VR3e?36Uj+<2Y`KH zZeX1fM1MfGL(Rsbs5!AwAcrcHU5qA0983HufL)@D8{=e(?IbK3)y9(mF`KKOvot$H z;aN} z^E#S>Ftf+Y7UqfVe2j&w##3do@uT%S#LZrVo*z{tDVHan^WMx=&vsEXBe75s$}kh^ zl~F{bH7g0rm5jM>40TCsF9`VR5p!rzj^=R6*sZ7nt*eXYH6BjsLN6J ziyUH8y5# zp1O>5+NWO8((G}2dsbP$1R#STueFkQv_@?@>uw)$LJ7L`hjrKxJ5mTsd#F7j#E)MNmVc}F8}Qp_*j6lw%B#ekoEExe6(+qvg+&4308ha zM$dafyMRzuc1>sxG2O_PaPY@Uf)O(Q=iaV~{Yk@v<(##)p2Q9j4%lkXNZmT0#sAtp z@}@OKI}LF|vVJ>7M?)sh6O0T3O_B^+jmOK;I&2_Zu9z^G><1$?K4aM0fLJ$39XO^O zv?b#m-DwN9T~mbyJZY<5Zn{lN+zQ+94%KAmNb#0Rn`ceTMpx*Ab6K@ylxr;5IG?jo z@q4`r4-+!d&i3QF03JTCQro&O=d<=fjO%Q-Qi z)0#63c@ z!&QdukjvK?Wf4P~{YTzo%JK%QjCx0lx;mXQYe>)01sKKRf|&oeFu;Dfyq3sp%MSIMSS zNhZAoGnkamn$cC5Z)x_u6Hleyy^r7SZNT%OH{ve?8HO=m2B+sZdTx57&#qnSOkG0r zc>Vcn3z*zg3iKy-BapuTrUjeBlRT&G?Mx3R$1r2BXJBDc^si9*H>2xt14y3jY(fZO z*&Gx_Xd4j)L3 z6VQ}vj^=(BY4iyZr%6#JyeMzbMeEhS=U(IM1K>Yamop|D3`br^X z0w9k2*dxIdjStw@L%sifhZId-ra7q4FdM)x#stXDn{Djq@kgRfukTXlx>1X|p?M8$ z&T8PH)bIfU0~_>sCCfDm=nxu5o8a?UI}UzL+0aBQJg3S+=OR&9x0K^&5AkSp(- ze=|s(`S?wNXp(dCjc_Y$&S2Z8_wC!I4p{5?<^2^}{`JFd1zCQ7XFq$b&9|I*riPa( zK3jRB8oLwqW_ItPV74A3ju2FvQ7E1qn`>ISgqC`>oe)k8T$?i%UcgF4ie1r z1V-Rj&6G`XZo|K_p!|cG{H&$y-}sq98cV+s$PDtN$Mu?*%M}%I$Gj#e;UQ%80|3k} z7`3#cfch8bhj-?7!4_2tf_1291Gn!GYP=umlle+o*QiSUTT-IUIR-%ih=d1Sgz*SE z^xl;L1VA;9k|*TRX4Ptlp=ub?p1%SXRJ$$5Y8ds4Fub%YuB(5N&#ix68$S`;+u;b~ zJ+Tb8)OZ{estDOpL1?G}_K`6<0B%?Z+O&RsH+$MFH0gOx`C^ir|mok;5HOF(stg_ zc0$p1K4`d|G!B;kW)fvgtzb+bFUTY}rj{KJVTF5Q1&MJ<00u;W(a=a}?G}^yi7M9* zDWxNRU?-Gj>Ds0wmWIjt6(*Lt=ku06PAVtuz~e%gj5|l7(QGU`)oq3#rx2=0c{vD( zNsSimALz;9ks95PMt{=)dW9RAg6WS9nA0+<#Ie}&v$2^n(9LqjP{YcpT7)1)IGUVt zHhRxFR5C4j?kh=r%Ph;yR>lg|pV#|qx-k?-jn;IR+Xt`tWon+rtfj^l)h>>bsJ1*I zM#{#)oD_UO13tm;Fld(CE##n^7{wfR|oP7u*Q+OkwoJJyj_mR@t5&_t)%ZEIjC5YFw6j$FQ)@SH&ibGe9=)CE@q5XFNW7le^=gy z`Yt#No(P158T?S58$&8BJ~CO?D&i`M6pMdz5>}uluOutfRWCzgb=J4inrs@tyE}Iv z!g_j>IdAiAP8-i-*=cguvNI%?UI_}WGQgbFmQVA9dkz!XQ`&zSvM$3z!9jiG_F#|< z_RPg2lISP^PLv@Bd-!Q#kJ?}IwN7=E(gqMgd({A$lYdqr27 zlT(>bV+!dBTP ziL<2yY2mAJPAeY1T#M)#@C zEU%oThHc4D4Vsa)2bjovubm#UcoewP^jwR8x|6d`ree~=9-I_I`7xemIfRSRqipdg zz1GV_v$~2yFN~0!kY^Bt%0pPZT* z@}97ddOeCFwOEB|N^b1kb8p4NRjTtFW-|i7OGPeX;;?p78o!-_M!Pb zeel}KCwHxFO!2SbmE=QCcj}sV)aY$`QE$`WCVMn0?I!Sg8>W?}2pBzKfR@;kJ;69K zA3b~WVqDZNYrEKuSda8^wY6%M33J{-jg!(zNCn9=+oA6e2Gx_{jQ!1xOfwh2-jU`U zzAvoNo@UM9S34m>kaA{i>R@ebXmOxemUHs=Z+lB(s=TR)8@Ays;H`US`@s-DX)%}l z(qd&AedahhdRy_r54DoSM%VY1vhIowCUb2P?7(r7YzWmUf~UxTuo(Qc-H_B)L<@!xr%0c z=+1|o64`K`<2Tb4nV(NcXT{|&hfCQSrXsIUVx9`z8c`mrCU>xUN8ge>YAGN!y0~hvp3My9Fxvw zH0A(0tb+O26WG23X9$dQ72=T1bibjdl#Jxv-ccdmb*^)YI?ov3?edw;{a{xPo9R>H zM`C$mcg0UcWbyXn^KDZEopNyepxWYhE*WL87VYaZp=$suhGhsr^Z6xl`M^;>>rG*e{HsBwE5)k6@CGq%h!;)#Hp}oN<2p7g_@fj}6nBzWR6b%nkqj224k*3$ zM=!SPENuYgkCFqv6~`HaEyJqA9@VVIP2Bby$HSg5DZKVL|5_WFeyBDsY2E;-cM2CS z&|dCf<`b#a_FIQ1BDUR76>oICtM-0+K8vc=s88K|S=x~y(BV2r(Okifv_W|94Doqq zOl{_8E5&d4jWhSgzuxH``gMQ3X`7O&Zg98s6J!8KXtR=%rhiTnbLCJxHXbYryRSxV z@k2$aA?!L`cUuH@Z2{@S3jbOb8iFZ-sOf+jIHaro(NVsK=`3sK$A2o4+BfLa1@DJ4 z2R6~ZrO>L4BNL}gwx}3R7lb<@rf@B`+JY>$t~h(8QJdeneUu0=?~?0TWH?KI@whiC zJ{kgeCM%Nh;sAMozL>IS8h9Z4e2ou0o__)+P*pqY!jOv1&epqW5`6P~D|C$HN5S>B zeOIqQn0}#(Ufj{W%+ct#%_XPj!mk{6DBHuT zqigDFRILgc<2MT=uWg@JrQlD)$Pw6XIiCPiu{d%^*Rcq0Z|fgJ3w&5tPo}^n!#_0P zN@)Sq>i8A9VCm=7A~}_mlZrDg(dOi`1?AD6loP5)i#L?I!g77H$K25+g3O48XSlo! z`Rf6iE=SH|X&)>*zU?GoG|4jTQvc4%g27)g>TLqCD>ovH%5c|9a6C^eTWr zv^;K~t0)ir!45+H36#$4_%a-Xf_78UX;Ro-ISA$Maw)!q3GjX#Tsb$m(yy<>xZmg!5+#`PZ*R-0Y74q)&yjbt z%t$hy2A`J%rarYBLa1p_wq7qi?z1184?XTHdA&Wp5Oo2go|tXbnLOkr3QUeDsvz#N zCWbg|=DR%s5U`7*ymcRns}II;hV^G*vA2Id58HI*TllXL}I2$*bg zA-umBA+xXCXi2Uo@Q(1t#tyoPqM!lCF#0_P9{QakpCZ1Kbl}}cZj>2Dix}ILe^#Cg zTf6Y)B-*h$crEV6)W01R)ZZ92yhTvhfbmE1+vVx$XSV-5GAk07Ra4;|3#Ncq}f<#b$a^;^ciRlrmh9)Xqf$HFLe_WBwrl{;R7H~u$2EEtlr{}ngY>BuH#2E=B zd7tsl`8{;VNu!rim{#kioMFnC%*v3RN-^N|Z+`Fjir zdBrthGF#Fwsx$0L*o;b6-Ae2sZi`>L2E%#`mo3hxTs$~nu zyfe6{Z}rvy1!yc z{C8TRx-Q>{qX7XWv84Hz(f_z;{iR1q^RH)xNF!K-g$AfBEeV`V%jhwCYKDU%3n>s8 zjTs63rY4jaQwk)amHb_XPQk)(Yx1LKZ)mM+)$CRu)zTcIbroxCIUE9{1h!GpY{g+) zxx8$p-nqJdbFp$%bo7z;vE7wH%JS=S6k)sbW`^(N^JeYlXyc#Xts$}1-im9k%5Nn4 zq*2ZLbXq{1lK{T*A=8q)Vh!c4R4%!I%F!akCvQ%rYcA1_`Q!nv0NPzUb0$@hreDq8 zgG+Ap!6V!~G(y z9wkdu@QYDKkH!hvef?kgW}k>D+Qs%}e{7F-AX`9EhTGd^ykaRq5y#{^8n^qKC4cST zw*?`sKANL6yBzg5OQM%CtzpMzARUsTX6DwJ+3wAg6F)uuKAczPpQYSb)+-%z$L6g1 zt+wg7nWqlb1@Qfae;znzne&+ZmQTQ^8Tvh;eZu^*P|GVdmTi)_2EK40BXLn#Uf7$J z=Mcbzp57S>i|nf_$9?QX&hs*{eXHTxHupjXNen8eh9c`t4po*W&oVLu*VCr34dj410C{$54Ik}2atjxvrZVeYPv zwu)~dMPB8Aq6V5&pN=KT`UKvZSzxjXyLRzgf6m%A0@S9Et5np*aZ71VHX=ivVjAEv z09(xxRtVL|h8mgSXi+|32D~c!Jd>^e`v68fD1Z?aMVT}m6BeSyKeRKwL^EN}q>v6b zCaXYiG1_IwXnys4jGF~gcKPQGRt~OVl2|;mpta|I()7thBgvS zee&7o`l8}T_LIzZiv?sdSvrSzqA&o&tV-U4CB+qH_bDKqY-i>VNBt7;@rcf*czdET zgN+`}5Pzz9f3lVnw*?2(X4jtoDj4Vw`MmI#AMBh6eFoZ z!VFGVXE`NiTuHO|Xlum<`B3dqc~F$Ag_J39cnED0Zqqelj>?Cs`m2z0Om1-glyUL# zt_g2rk7(;vwJcN_ip+ja5&@tr#`9cbxOkVS?c&!vLooLi#0`MNVnnrV-It5(FGaJD zf`@}ZPK^w1Y}DYO(`ZRIA67@mLdr_OVrkVF!(=KMP;lYm&pw@z=7ws0k};|vm!1g~ zC*+Zjv5z4iEI49fsjDU-v1FW{xcZ3A0e#{s)e4>hG7cS_$D;w>69h0tEL0R^PbGxU z7Fdj7Zx*uxV?2fxUg&_KicH~^GcTrYB*=O1jOp#J+6^0&9*y@4CNK#YH-cEzSiaQ+ zN91A5j=#a^z_7@qpj4e!H4n1khTB+b@j0NG(EAA&-nWPi5(Q|n=Mp7CU1A`&q5@OS zVfc>(6-C}y#2O_S^a0=(0~Mq&xOXZZY-Ng#QU6Yvzuo8U&NB?^QpdwDoojHFd*e|herUv>WL)^68tYHzE-t2^DoY{ zg52iqf@%`x;nyvH9K?`<>`D;!`&%zQnSuo5I4~dKI|NVoz;BM^2~Xv=nsAc;T!D8- z*|DC=`-BOuLZI9MU-8W@HmWqS|fhvHMupM zAD3;&AYt|j%oR&AoqKtu&mmxar3A>Hgu(iy=Q5vu_`u4h_m2(-3ltJue1`YypV(u5 z%Jxj|ioY=h5uDg#eS!iub}3A8;0|<41(22TP`rS7I9WD-eqTSD`gitjus(g?JVtiS zMf(8)s370r0Gl~y;tMXn?Olvlic=U^KSeV?>bSf+=>gCMz|8E$XG4GOEd%U#@#izm zcXNMlKk!elB4vFIK9zrR1{S1WyfX-Yz%hrN8tSI5U9oSz`_KNpa;BH%_2m;H=${0D z?Ul_jVf0L!Gba1ofD-*pnSJtv-I%?`I1=F9vZ_w8@5j5zhSuI!P|7E&IaL1Q!rZwD zd&Fx}vXE-E`_VE|$aCq(ZOl%RQT&o!6f`=|G@6JyIbY2Lzp!=zuDpoFE-$l$vNH=@&WkI+rNS({y}X-S01aKj=mHJqnT(= zG?)vS;A*H@izMS>JZP^eP@8#Q&s0jplXA=~w0Dxhc?%8P@l%cF68nSWqc1>C4&KFx z&f!r{PP)x7OD*v-=TwDL{5oeTc}rx>($dyxR&K?nl5*}|?%Ed*v)m<&(+;AGtVfn2 z<+~n7!S?dxS082iqBX@~1fuMUN6rq1AXK^Juo%{>y*!n>C?PWP%d|9HW~1l~cLAnx zB6gM#eU1phDtPOIf3VFFo-ztO(7xOK4u6>3% znKN0b3&lf7A}2+5hwlnk_*YYthJC`%I7t9EM%!)aA*7l}x~9Vzdc0}9inIk&m@ACN z?{l8)f-XvkPIB^&+Wtza-p`RVqb%<)+kvdM2%4Ph?c|fn&(4nCS^D1>jlX7&MYo+Pg%pjj zON27X&8di~QxpZa3T*00RdJ^Br^R_IjO(u`sClGEAE>y*M_yDQC`mryb63_F21_9m)Px3nV7 zdY-ez%oFuN9@5L>QxHb<$<`d0-P1p&&bHDMo1Bu(3;cVOY@agh~U+M_w zm+J)6@IXFs=H)4%4%429rqC8MOcw-Q`PWrJx7C+Q_$TOgVHWT`D*b^e_f`Nxwqm`? zq2Ad;9~ccFWF#x+?q$e1R;p_IDQk;OP#5h);)|H7gJ z!bC_zi2}HGD`}ybhC%-9x&s!AtR^}U7K?u&GL;BLa6Hf1gJX|wi2s(J|NrCa9iuaA z+HUPT>7?V1ZQHhOvxAOpW5r2_9bU0*+qP}n9d*=S?&sa(d-i_+)~K=8pQ=@B)_Klj z?(oD+-bxJ4)?lvSb<%+#p3T-I-OY_u>Hzoz-67WA=g52VuxIeF6>mcI7)S@}ua(QW z4`ZD_ZBeg=L5R-F3!VTOG!o4GKfTM}|6<9P3d*SCBF^5Es#VU0@(7gdcxtfoNfaVv zk+SutKJ)<7TcY*1JjKPg(3#^m5`tZ3SNYP>l9o#npH)=o%;LBci(s?s{CIJFbG=i3 zH^!c4WV!x-Sl|tCHHb`6&l8F5p~M`54M06f(}ALyaOo4F0q20x9!1KY{LJa#j2W&k zWWbBMj_lKnsSBfrfSjlcJRV^vC4LYlzPqR!B={d%t#hzt1g&mVnX=+RDfjjyD82oX zbA8ExNBj7^LDVAEEts^*d0)yBz2a9zr6dK!x6Ft(7apyXwq59~FqJWhEg5ak;3>lN zgzle(v1YX{6y-p_%RucPk&cit&A-r#F|H|;EN?5vFSUx78c2L6FW)1BsJ1zVGHNrr z62PdoU9k?N!xEA-+Io*t>bKdyVdxx=_Ye({~hc^IHeU>q<2*}Kh#F2CosCYu+ z>x(({*+XTM4RbF*PU=%6aCEuXkX&T7!Jk|gJ;ahs&X=a1@$=Lu&^zm)}EvW1?n`x-qg5bQt|9xR9 z6x{2Z1$<8vzAH;+`xM%WNn{S`x#ReTsKQgC8dJ6S*kWaB zP1BJSHV&i?{Z@*BWs~5WX8d|Ox3WfkO#?v@P?YqIUc#<<+qpRpzqBHCgW}BmxBPCA za4}$1C&giP?yS?K2hio*kQ_(+$8nv+zVHc}4vZA{0U>GYtbf6Ahsl-UPW^>oI;HHM z-M=@zT+(9jE7M8YJ-)+3>p#7c8{wV$LXaB}ogAvz)A_JjimZP`%8nyTQZeU3I>M9h zP;^FROT!qIb+xR5oSSPfMvvQN6s;G{*jfN{PG7{rl^{1mV&7~(e+2e?G`=7M-!T=_ z18%SY$`X-cbLP3zFz9N+*;st^3%9b(`_MDe1{KoPdW*#^`OJ%UJh^8`urQnZ;~7c_)gisMl;Bo!;L<9w3R%ZFc*%-Is~IYq zkOe!mbN==OH}|9P)xxaPrvf|o!OgTP*h<18h>rm`H(Iv*T5Ky*-}eyOc14+09q0-ROp3&RLzVj2GY!Y+E2_JBA(jrmPIB>_2bPFRfPh9PJw0a*VX! zUmVChAkW#fx>s_4;bTUG=o;=~>v5F@zJaf9*;$U^d-#d;2AY~U@_0`0ol<+HL4hINe*+X@qlQ0|(e;=XYK=h{ufFqw^9@Z=6NTR})n zlBmx%~KIV6Pjy61G3FIQWNk%w64EK0xP7AbTV|PZ54__plVqdB6ncAzt*|ot8_RjADEZ6%6 zGr$VFZSkNx7Z0^Nsu?I)^2{QS(WW2$@Vi&j%q_lSDAnJ|jNy<-Q#tuWK|Cm*@)mXE z!zwex>6dRc)#>b{7Vt}^J%XcK3Fe|)ia+;Gmxy@(ENcLr+FYWih1}Xi;WF7R`JHlX zh3q`H@$|$BoETKA70&9pz;W^r=#e)1!IJJEWc+5oTk%$J;{u-Tr3E}1)!$#fG#TF^UJyX%mq`SJIBUYgg1 zihjLpjG4za%D<3WRhzU4fGl9-EpT z@cT+E(@5|fn**Qw+*hIatu8KvYuIcE1@mxbP)jgF3o|j+CEe=E zTCG8NK^fNm9xpHVwN!5xoqM%F4_0W)lKxF-MV<#*@_4Uwc zV)AkwWXk1tzs7X8eAr79EmUfKMpXU8_3tQEO398V;QQ+^wdAx*mBgP?Q|Gt;Ocg3V zeY`BkPJ>IyO7%)-g?Vxf_VW&6^;#xR-2e_6y{$w`yPHH?{jb6cA_zawn3IP* zepR@3L0~dX1r_b&19oQ8tme$YHf1ldM5(PA=9E{VM|KVp{dP(7Zw;`hU67n{)$MySgp zmKhc`E*)pD%>B@~Uq|C6mZmnnIsr|fdg_~g;6mQ+9?UcD{xHSZf9xo{EOxN4ULK&C z#4~_&>w+lgvH#o6Wg_Ce=_)1v`PfA1R2{b5accWIja|(wT9?B9Pb8~lKoBccso%JGP9R+nspv+a)B>qx;qVqkGIiG!$C*csdplwM&{%b^3}XvK*!I>|i_3B=rQx)2-;o6y?z zNnx3M7b-&bRU{LL-kvL@H8h#_(91Ugfrz5d(97DVp@HedK2ctN&%t53#6A(zRQ$vn zLzzfdds#h4ZyjhB6X#!%L@;?*;{kNoimWH{viZk2GwAI7NHx1iyLzOq0l+ula(v4M zFA<6uZ?M)T%30r`j{1pWKlwe3euKeNpZui4Z=NCY z5#gKby{p^2U53iO^eT|0VASUGU=o}-<}zIOMa-LS=iVe>Q$F${T-?jS+s_F7uev3rf25)vPS2NWXV$qP@xgUPO+9@hnR%!><1!`#g4O+@LMD<^*Q34}&i>7kFw zgVMVI!)#B~-phHmL2b7%LoL=rMZ0#0o)qXG(N3E##2O{I<|ZXiyfFi}uV>$?8JF0R zZ?<`CF{@Uofa4}G;kaRqkn~|!JpEwWtG+{wK@$)k-Jugv1Zsa|jmz*l;Vh}1F7D+` znuCINZ%cnaFL?T#=?T!)zJB0MJhm}&hGTncpELjFp3vd{L0gZMF2s(!{trL7QX@gF zPOR2v6kqaD{p=M^x5OV4(AlWRIrurXBlYBuEir`DYEcq$hKH;_Uei9E=3NA@7LMY&U|Nl1kQhqPHpjs2vd~9#!qN} z6wGXlL?zlpNr0;d;5AewBUOi&vt!0k?k9G05q7fmsrGc?H~$j_dynnSW_w@J{xU3dgU zNZv$tWh)8H)v;gXW1eRr@e9JJPv79Yo6(Tfrmao93|rn`8M71o`V-awA}B!fNS7>| z{FYHuNbtjI53->TCVrZtEGOsQKTzO){tIaBR{gu!Q?RT%Z)q(Emf1TnZCL*WoWo3X zjJTkMgP-Vyu;u~kFWA4w-G9%t|6QYGePHO(_^fvaf`lzlvHq<~5`$VHagYEV@B0=J zPvmXSxQIgWDxv$xpRGp#-g4`d5z;A9Se7`%wGlK%(QLjx4Gt@(;$% z%*;&Bqtw*R_dn5Tt6=O8s6DTOBC-3ES@x`&!5p|D6#7 zURGX_CqU^eXy45P33|Kq>6jDnO*n?b<9mqT{r(5%19Yd$Z#c4KnjW8O;}zhM>7q;GHLfdQS(H9^_yHEn*GWKC&qMh2 z6}?x?HG8-==WxC}ftEl2zQ}}%q>2xF!IbkpBoBJr+=S<5Ray<@L3iLpu7s=tLe=deMu(ZXtG1{;<@z=i*fS>MDQrt1=1rEC(KM{)4X z$K~>wY?OAuJr_r9=71_ijxc(Xzc1vCQzPuKRt6~jfzT-IeU7-G&M4%J zhG?3_4WNT3hHHuDh426!&`mx?teZFk|8$AhH%)7n2|DbvI+_1&Xp|d$5)M-uxDd7y zj15t14kw=|D!@Xgdw>5=t^EIAZIJ(a9XjEW`}uy3wtZ6c#6CkPK+U)~$p5n(7xyU9 z09^-EQ?w6`zxrz~)JV`ANh-8pa}8{ThNT+?i5Mztfep#&f$Jvg3^wljW~w>*Wr&|B zApxFw?ArirJhK~NHb@hRj9y9uyeyyI^&k+GtBcDJT+HoTx==a2D6I6O;PKnB;IZJ- zB;{Y98)AQo4!sDOi~D_}0q&(y(089xU{AK05dDO@-|j;Brhu?;WxUyYd?u{;?POR$ zFzyu{&v3t0FXkFA(J3UBFzkj+^Cl{=UvDow(ENRdDTgSRh`NVUYJ7~LKSsOlJS&Pk zlzn_|Qfr7sVSoo-b-St6IrzyNP1^vR4mfK_<l^k~<*$`su(DfIEpc9~413PBj5ze(g|=cd80uG-1LQHT*3^t2 z`d{FwwWX!I%7qN&p)=jsfS zmMGs+C6{wW7{kh;lM>m}7U@}cZlYa&lJE6|+as{7&{k+vJYp(TXW{qQ0>_MY6HPXp z9*8rs7+7E>)bWO4&?%WEfKCqC6E(hAOsZ{X21-cAGLN6>ork?Lw_s=kl_Jh^l;}}# zCE2pb(9oEwDd3OW(^S-U5{YN|ZHI2FZZbV}QkQb6kLx}(t7GlXkcrq!Y!Oz;I`uCE z6gNkUn_F@uxy7X!$$?o~aba*HXa#?>M;)ltew)?{f{O^)tZ5yp#E zzbC#Rivy3}(;Q_c#ff!5OIa=4f+~?Wj*AZe(PGBriAl!3Qp0C8T zOhuqSv{5mU2!$3tP&qCVz63e=%LS&(!WV-MC%z6TUtL@3dJdV;%ChoccvW8N!a1>? zu_oO2C(Gs~wj`!=6fSKXbF@}E=jd`U$_qFr zgGiQ;>Af#>f>KqESP#TlRo;CM`E&KG+bYBnDpIvj;T$R>pf}WU<+Ze+%E)1hluul1 zhjKWnqQn!_|K?)Yt4kG2KtxAHUQeaV|1%^KnFxb4%HM^CDzhzWaMDQ$c1)V##q7%L zBd>6t!75qPK!S;E4keMX=5IW1BCncM>(b=lS@jMvA?GiV=2V}Nc;{n>tsrf^J44c` zJ)Y#54{<~@!1C~yr!ZkYQ-E&}%_Du^1ut98_3VgW56>g1&Whs7gr5Y|D`fjh^@(x! zF7zwD@daW>-zOCQ5mVo|`R3~@E^3}XYaXIGjNq^nm*PFIm;o7|?6Z2GAPRO>B#8h9 zxev{lzRHt|$WV@wll4g0w-t{lrk^LvrwLu>;1BU?%D z+=6A~gY1xi2eZoc)zKi%4K+zC(kab4*S zwh{1`W6#Z}9Bk5~3+(jq{!oO(aQ3gvF69c3-=`~NqNc1 zC2?vh#xgql?(MZhe4pliVB^qis1K9ZK0N~KC5l598RFs=#ZEV&90kW*owdu*DpcW{ z&DB{9X>8uH zOdR%_ePF_VnnqsjnMi4b{Y%$$#SFQ&u4{~RdW!G#ha_0|jc>LxK#Gi0j7ZZ%%wseG zhL{FJJYH2gSs{_CRBvw`wFAz15l$t4bwQnmHVlfIZ7R&`Z6ZXvCAp^Yn2Ow_NuoBp zvxxlfyEumy>z4MtJlj-Eq(Y;zQ7~8HN(yw@TE28a?+(lBLlQh4$xrk@O$ z7C$cJQrr+&EDFds7)OdMOzaO=t(0Jo+`JnogegFgZ+zbmj@8`EB;>=EW(PA5cbg=& z=XWYK4q2!(0lxGKq@s3vlPf`B6yHr=ZHk05fU9>D8FuyOSzQYPl31 z(qpJkxg*SHD~pXA*M4z$Hw7p%}7=P9=1_Un$+H8)Uq7mG(|ht#p~koZTh@-&-n_AUtVUJ8?CIDMXk3Vf`{WCoh!W;2 zmw}nQyh>wwiJ}naoM@rn@qFmYBBT8!X;O&VZ0>q$4@=K#H!=2SCG?eR(Rd#sbn#YD zS_uXX?4Y${Vn}vHh0d50E~*4ardq7!n{fU4(DAr_7hpNw0vBfDZ$R8W~2_ z0{-E_0VMGlS@5VI?x?6V2JT0_Ne&5bjP6gfUed8*V5J(G1X|MQ^7m^LuY3}f6AiOb zE+9<*8j6Q?1>1f4QiEr3XoI^LUMVG@P+g=^t+U8#t;$X#xRGK*!eSLE_Ts#uBq=4o z_J2vJeeNOp&e%6=;p5{(g~!i<{c{MFQnG}315m&F8#+($7ggyVF~a`=1}{5zT3Ek5 z#wmyRflVyE=(3;>@shdCMB&iFxb^ySsVdJ1hPCpvo(;a)nDAY?<%{~sF**1QqdL_4 z9U(p8**gS3!hAT*i!JP9g%GU1_#(Snw*rDXms}n8BALB7y_?98-Tj=dt^%0vGnpYj zKG3@voZx}2LTF4{lJB0wkEJIpf5}^tS<)5JHXt^s4uLXW(jCtkhr42&dzcX)ebqc9 zRuYX-gy1*u=q`!4%aAM35*12&kJl26aKG_P!J)USWWu3jjBP=BEqn^C#A6~eh#?~e zRhWY`9^*sD>j3}st2eiUb%RAtF}uz~9dN3@TrnaW=V6JvW36$ zB%B;y;rf8(aTaKDm@hK5(U*X0qm z&v@Z#KBcjBF50yG=7Q&VXGul4PQ3nCZyUyHU&=~(b4$lO?9TG6$k#syZ%moDA9(1E zLASw2#4szHwTg|muFX4GJl-rBrcyw$p76qLK7JHYya zI7gXX%V>BF=hzi_jgrr&pB2tWSvxAoBwj0kvqgE*Wn0q~Z}<#Kw#+N7YA6uT|E`T@ z*`DAQW8uuufK7v6G+Rn^cWFjx3j|mow#=~A4K{E)PP%YeMvCb=*oDuf&Amo9+r5S;#CWWTm|wV z?zl(#t%lPp%DJw^bdF78;8L|TXR|hHlb|6I6FTl_u%N0j;e_NoJX~sXizF|-j*3v& ze$8bN!}7^vzNoxyn+N6jepT=*LER;jT{N=_Qy@u+>8+HTh-Bedy zI1v?b>s@UFYvGbhPkO#%YkHr}WgU;9VVx8tYwwC3G(8Alv|6w`)_qXPM zi&of3Cv5`IFGL(IAT7&u#fZcNCO9%Zy_`X|FFu_)5QY|W)Ab8QBpH%@zs(m^3RYd# zdNd8~u0Ny)h|lqu>^490q_y?OWJ*o{c&iM}PlDG5v+QWtKAk!%JpGw+f4S#-Ip6y7 zPw~!xBT_Za$$%$yCqzB)xnW#vtr8rikVb4;}exL_h!dUUFo2zsPKFWBZ zC*4EV>i_}mrYR;bM)oTOQ|UfFJ_sA)g!Q*$7R` zjg3OHkP2yOGR`$jIFzCs=Z|Kvx|v};{02IKv}b5jB6=uEINmm#WadnoGvpsOiAaSM z93rEU+l&+Nen7n$5PpVx zG*zc5%*>JJ6rXh#9)X$L+@G;R#7BnbROv`5#pm_%b?}t#lMOMDsF5g3gMYQjLzgNC zD30x*0p2zLQZ>cis7TXYaq;bHCOny^k5|7a+m%wkOH2mw*B%{?P=>COPu(Op)zJj?M)Sl7yuOI*=yeUt$?D@vCIuf?SJ zyF9MpkM&66)}Z-dbnbDHuj(NrT&F(722@l;K-2XkFC08&6K&6npSV(H8uFn+xL5XS z>KJMSRr8;J9*fQ0G0RNPA4`^lbMxs$D3J~)hjNc%%80+7Y+e`Gi^9XK`je5Hyv}YA zc_Uj~CT2=Gm)oe%IlmANGBb4usR;Qor?s>CS{tVFT8!Wse3>R&&M{qUWFP2KGcUBb z0Dfh?5$wGQWbJR~MFU|BAOA4M(|?iA3Ixr-QAy`nD#Z`TVv2=icTW0Eos#(bN$QS}X#x-n z_!~U_X0Es2oijmbl=U}4`~w&7gO}zF(WmdPnh##?d%x5^*WuL8jN~K4i*%mcIiBh1 zc97N`^ANVT2t|=9yCv5jEL%1e3#vK=RB>~MCjA=LGbBSGuQN18{bUVu=6Ts-gRe|A z-(}aDMg$(=h{<>4Hi9LaRLWoY8x+jkxYQfMzq;u^AB)e3Rozd8^#AIn=8bwbRLKAE z97)i%xa&X3o>5;xaTExcAb39>93WcNRuxqf&9BOHmAxvosss9G-Ha6nvOHW9O^z+J zFo`hQFKFAaRk~#p(^XRr+?wqd&X>mQ*IWhK(?o^$t5_@VD>Gc1=$O>eR`cog-xuP75MuzUjW>*$dCm%r=x72EzCU3K!~M8M!m>JoTFxj0R>XOCQwXWNi>jA zgKj!8PBp6a%YhNxCl(ru=q(au3_j?0VMyQp*o&KKVcU253Xn!K2(DZfJ}SdJvTFVw zVR)PA0>d2Zh0~+f$NpAuR^`NKM2(K~+F|=IgFOfQ@UC0enYKW-HNWpzXW6c^{pyXI zITlHi_Xv`$*27m(pCu={zW{@%93!Y6*MzZ+7b3Mn>l-k$R_IW^TXz5T05bQmF?rHZ~g3@l0DuUSxe*B&WQIklvnJ9 zwmYm|{+K73!%TrjFRTS8=V)K06uWQvrSx&Wo|dI}J6QE%7XvBGQ3-D;U00+sjJUs? zOI;ce9!W+%f=vPy3}I31c33QRD8Fe2mWkjZ+yqn5L|U{Z7;Orc*d{Q;i2WVv)CPL{$z+GZgTNbF+xa<;Rc{Ii|YC3ibE&7~WB&g{e`La44yE zIMA~wY`;oR%mG|iI~WmU$IM|WLhQ={S1R;wr5NpMku!0HZQOGShJb2ZNpN}8PbLW% zR3l1Ho)G%vSM+1QvlvPOED$(fAu7@ybKgl)#v5>FHErU>XmsbCM_Hl}uKloSldt$5 z9!lmB9#D$UPksNpK(lJ@#P8~dUfv{_dPXnm+B-#Pjh=W@b?F^vL!tebi(Z9yB;HSU z?{ngeX`6WhF)HU><$g?Mj*?lj zSiTIuUp;Ys_A2bVPq}xQZ+X0p|5K&33{{H?-~vLtR|s{jjQ)zd^K-B5|AgMA{@93x z3e50UIugL`5q;KU7KjZq^(ezCz71!K{dxl#1yKuT9=vEQ!a~zO9EseI%JP=SW-xba zy6UNsRxS0rAmc`T_m-Xby^RfEgj1BhwzTXb?ze|+HZS-CWto)}a$NA#!L}WLw9fJr zcmsGVyZK0_U7tN?y>$}~Al}5s5W?=pnj8(Y{6@9ZU%H+D!L~za`eM%GSGAu+gmbIt z_g4P~AP+F!GS0oKBWdodul`Oe`D2aW4U>o7{jhHka`wfOi2t{A-yQbjyDk6Z zdYX3{I>UKQ`~%FqW6ZK7q67*ItNwP{`!&FlDMhy@mWFoJ)M@FiYpuB6^o9S`t=W|} z(PaaL8>KW>66MRoW{fTQSE8wWP|$<1$U3BDdCflPm|!LzLX(H6%!ihZeo7 zVqY=mOFDU{?K^_^O0QYhfw$_M`>u+IIP}(|b<9dp$Rmi%6VRUjaD)^Gp-UcQFlU}K z$dl>NCQrOX|C8f0C}}&+eEBFg!6)%?d9TVAA0bA6%!LU>I;`_M{~z)CJ#E08>c=3a zJjF{TSV&!0MiX$VtM<(hVRdw~ zJyHqWvBTQ1f3BKHEbyx9QCmh--&QhPB};_Mc#T`PSxnK7^v~bsfEIqFg|! z3X)$f!Wz+45i2qjzVc+*F-+|d6<^*(1fHP%AT__#R+=Azb6VIpBatXl_6o$_gs3IQ znLgYzbZ0S^{Qeboyl6VvH4jI+IhA^y%w{4bfZ>9};R_G^o1opC<5UlD?$O`FoUvRQ ztTHCbg)&(oq)LaHl zYk2jlC7c#GLe>Qy_1>=lA-h9x^rzK52A-9rVvoiKo2P6|iG6M%0^W6QLAL(0@h`#1 z34mKCa99-P@tW&mQztr{#uL=n{@dMdbuga6c|T!J9_`AJP2pjsF>Ky>Z83#d^I-Fv@EA`}CK=gEZv2?u-df$9-&mEQ zugaiV9DGzQ6Ms*(y^>FF;+-|t{qm*A8=qayn|dPe<&GKnhxC0Q+TOm(`44=RZUVX0 z+@f2EI%e+PrZbu4pxgSWFs(205O*F0wF{#R8EE84mN{F`zjCx!Z6i@6?>1ge(2;Qv zPUp0?F7QQ~zII)1AZ1z0JJQ*SuC5SdmmUQ*yIDi|v*sCm?lvj)>`jGh*=ZPxGN zi-UinvoRLkBhKi;p%O@7&m}QhyQCsC{}C^%OnIrO%3w)q-x~FF)gOv79z3svh+GvT z%gSFL=UyYXg}K=*)-yM8z*w~X;Jp@ql4{|I~+Q5J~nL$~w7 zPS$Jyrc46vv?0MDX%6IV+1F}O{h?$FH%#qI)db$ALnzdl&_06mD+_LJuFJ~=zPOHU z)`r>6WOkzH+lX;`N7kvG^y>FqZiegMDS4ni9(pXc8Vm6vO!s>#2)NtTj}MMinYUV` zEwo={i#N$mlW+~6>!Y<&ZYi+`<`LTnB1H28*Ot!`x!z3Rf00BoVD2@tB^=NO3GTnO zvxfTQr{pt5v=TUY>bq$t(mILjz+_&2Yr51@V680rd1jGyZjqA7=cl)edk4GipN^dC z;Ctb%tkz5<1dcJm?P;(#LMdmR)mN5=JV7X8sXmKYlhbB~MqcIMrvQ2k?abF2nR7$~ zh(or{)|Bb?tfY7Y3@;P3_#nDHevG!%xKf$5gEZRXI`t=t2SKw=rVXn+Iu$V-);SLJ zXxCmc3zwh5mYne|CgWy~7kXUX7Rs)dIE%QG>+7h27Ytdce)c)1ZEBOwz7K3zy^hs5 zH!qYDV+145N-(wCV!oCG_!_n5_bGZ{Q~WTQ+QkgNHsh%XV~MM|@5|x|z10mm`-xF> zTJ?R-EbdZS-xhKOj(94Rku6#|`qdCR3+G>mD-d8=jl$!8xM__NU{dSc0I=c)=B6dnl-nc@ub ztkzWJa;f3nZR5fE!8-SEDLFu(yO=Pye;}(pMo6i{Qcl?;cxQ{W(%vCqD13r7g!IFH z2?RsdMM9U1k_a{2XfDniYV$JW=9Z13{uX(G!OK%&7Y)DFlvS{|Bm0y&k6}JWm+)uB z>vi1irlNMTruoeHiyd@H+l2ruuDRmu^kkVNK55p$U!qPQxy`E&LN&xu!LZ@wnGrXC zD(-L<5({EvR0>^-4(kJ_2)0e-|2nn2k@@5p*dTzw{k7v%ekm-)hMd4&VLTi4$^riWL%TH_)m8eMz(0F0&kZ2z^iLVrZ4{W%&h|c^_*k*k6;7W%rx(BzZcJ2 zG)kSL`hRbH_vfhTJE#7n_~{R~mltp=vLDMS!a8|!vi5k&82AIYt}O`HyF9ERZ9!gA z-*~k710UM0vXWCe>W`#tu=0YM-JEjYrmV z5VC=g^=guc_SQxtX!VV?nNh0Ef*R=r-}LhOeDSnVdG52jGfIbtF3OpfYxp#orl_5? zwBeehZe%_0uggGmPOJNK2Yc3!R~SPiLw%p17~jz4h5Fz}XRW|vm~;DGCm&pNkAU1; zlH7NEn_hQE`hXUMZAxfAp_Gn+$cr$StJsqYLR1<-VmAGu%V%ogHsx^-%L1K2hn65`P9tW4o*H0A>JM?~5Y*;?FH824;$HR5 zaw{*jT-UKu#s1WCtq`BeqxVT8&|`(z9AS?pu07)MBDEwMB{pf=T$`qEgj4NCe^Kf8 zEV8a{ZtIRi?+KSRnO-%}$>}E?;y0Rps;*yHE#D%*c_hAh5|NFbbUdZLuX6h#*!P^w zv%@g-bO0m~gi9j}xpI@5cZd@7gL(W&CN%Fptek$-={z@=#W5DDK zc9iOA6~lNV`LCk;uMYb^iVo&K&Ex-x=8&eMvWNQiuUccV5GZf>)Nx^7{zt9p-x0%s zVo|senzMLumcafqN65GEcHk2WGQkE?Hp9ULea=Av{wKwy@;})fi+{66|HXn36IZ}M zgMkO91_!IdrxU}VOqihz)yr)wt-qJ4Q!QykHbkiB1Q)2)F8*-0 z(!K1mdeV4r^SL|@D?HwKGGmdZ7(G!0UbcKWpEA6c{&GLXY(Gz9W`otFY>i0pavuV5 z`v+kRv3h^-DD(L!^Af9PJt*s_42s|~*a9fVEaD{+GDixj@0iu|MJVqkV@ZfJi)Qcf z+WgJ9nB&aGN>+Yiv&mviBtu%UNfMCPjq61pje2)U6yE9oitap!57W(Op_D1n%MTIJ zB94`b#^A2?$2gur(J7l{p&YYlO<({D&_$f%DoW$0lj{0u{^(WC;!q~ft>Lw$l$n}k zO0X+b}#iD zP?fH846t0K>474I@zVsJ9hw|))@Y7>#%n}0?E?q3y7~MHF0hWAmy^QbkE8dm87>H& z1}9anbFMC(2?9c#7B0pZ-$o1sN@l|{py$Kb+obbjk?vm%XP}oM_~OEmi4WLJ0S4<+%LaET}NfxHi)5J4>VK4HZi5sc@c@8}Op zVnHNY(o{MI+iKfQM%LEfT`!j&($_c}muxo|RCG7>mt|T?jEc>hhfNL#YDG!s~B{xT0)E zxN20g3n{9!VN{t7{VyrdHBJcQ35#0G`jcMjsAh=AR<%}ac?4T4{pXaN8V0!)D$8w+ zm7Q%zbCr#*^q08Mc&Q_mFp#9micDUult`>BT+#I_a2<}t-H}RsB!6)l16z+@g4RWA zbL_uY)T`og3XFBUHr4I%=_>F)CsjSKCd#Lv~ z{K}3M_(b6i$#8O!Q7vnY)&Y7{88X{`Wx+6+NQ2d;Iy<(Ab4=4CeVu!9E?f z`ayygY=dphxVvuJQdCNAtsAJ1XOv;{nMWDD) z{g|HGm+IQSij0t}G%XW1L$HHv^kQ+Y6?@=QZenP@i-rX-NsH@_w2y~>TQS_oq1#_n zFeq6(xs$|lZpCtXKQ3+S)X$gvjHhg_+w{qG{U8dvX_$+{lEsq8_tf0P+!L!oNz3^o z!XEJJpkWDh+AeGvHQ}j%jL3Q$`P*gfFFH*js$7ox#Os&f;W`np>^@qCSe(rve#mL@ z{?`iMi$YyMOe`+QI=Ykrq>f6DMnKEQGSsaUCkzd7vI|>eB#8|LO@vsuABMV&n!uCqR+Tb`hXPv$d(_x3jhBay!)< z(uzt(7&awZ#)v1-WWgpyT|sM(+mI5NLx9lvCI(1i)u^`r&@uc>G3mAOE0XdSz2f7( zk{__&&0$8+;^9f8*xxU3plk6%YSngz;|Y_ua*3_r1P`VBu|TQm<)*T=^N`_#N*=z9 zSSk)|U^q?1f+y|q$1abTs;g*!GWNV_q0BJGCqmhhWb^Ze{xBeS!OiWLJ54sZ&;IkE zl4~ay%VXO;kjjX9RC%p+sLYcMmd`-(yj}V%YV+b&(WS@XcinE;%$D9#p}yt6Sk1LA zQd{G49O%BEuuPwrvFM0-wREezQMym)@qHL@chDI2eQt@ty!{oUI5*q^gV$tTIeYpO za&K_1;)BPoPm--ov-mZ9)P9J*luGs48rhyycRO@_zCf#D|Jw!wu3t>AHI}&SXPFHb zUU6E9Z2815lOX3kZv`wWmKpzPZ2wD(@G$XPAAh;HC) z`-goQ!=6(uTQ!5qcaXT6CCna!P&U*JABVqmUe!gd*x0c4F9QXag2Buw#KW0O{0loz z;XLTXGSyF_c>X?IVR{ba`U1;eyf1mrhiRSJC1c(aIdcm6lBQr_u5gdwr4J0h+ep4p zWbIg0%{QN{^Z15MIelL~ZNJY`*v_vvhA`oCJAJ+X(g(}jsU`e=X-ic#FMD67Aei{B z*U!I(7U709i&O8(<^z*$bx)edg|;n$7E*iB3vp9}(>-qXG` z#(a4Wj)A)UhqL}mRY?rt(HYQZ1T(ok!cZM9xD{*7?xkM5J$Fq;O}KpW#Pco;|00?B zAqE@~08wA^6@c7sl!##-KIykC`9`~uTvB;oP%M~y(3v5PKlXbCE7kY2uW225ansB= zof1^uyq)({8rIScE1seZD$gE&ns7`RJo10(4P%T6e`l<7ySj z(h;Jo#?xQXzR%MYmKJv9<%jjc^3JNN-{sq*(ziQ5icR8B1lF;*q^6`6Eh)3B*+tdK z-3Aip)S7Mjn5OrCHv$N4R=8WL@Sq_+kFclGdh<6WuT3h)#3k|JaPxb%YH7UlG*0%~ z)y&!*e>Gz~Q2aHRSxiduZGBgQtN!(;J#}z>VJ?xq@52Fbc8z12awoXIHC zs|@8Ps@kd?HZNbar^Es6!;5c!d|}L^hdg$AR9KuAD5m)7+ynFkFFC#z!z!|j=}I9( zJ&mcY$M`kw?I5w`k=$wtk>*9YnWwe@apTuY{^Mr@Twf8pQ1?iESENkpH)!b?<&m*Y zZ4!qkg%=2;uz|iX(x_cCXy;Zwcr0+SG(nZ+O;ppJkrR|?O!D*iBgmQi9wM^4|BtM9 zjLHOT+eRnb_GH_(ZQJ(b?rhtx$=1ZFJG;rYZP)I3-nIApt#ALo{$0m9*NFqqF;2Sw zQk5i5g1k4a+TJH-*QGP`bb5at&?~2Hy)uW0Mo(@r(P;9z8m}?ZTd(cKdwbNAk$);h zCUp+5$VVnsk*O1zbgGEKhOY4C2~~p0jlun-XvnJF2c!vWQ{T*~A~41~>{3saGd=0K z^szCPQmTRrLk0^Pt5hE7q6@-!5sI0tb$-wdgjS48;R*^_ElwtNjMpMahr90Zz6Zu8 zoTZK**DDj;w@4%Fr-%f)S%{5x1!_MOlMhEvM>svdqHZ)jGYtW>UR#+w`8iXGD-yWz zj~`Bd12`FNyBV}KTK1UUVn5)@RUKaT7NHT2xK=~dA(HT#+TM6it`+z(x}0&asVA7> zE+Rm&<4L%qZp)>n$hIkhhB@@6w}}2C*nCR(CL9Tt916yO{;}S1?$K)`TihDEYuoun zh&6Hn-Mci}9p}9j+PUyX=quNK)w21tvv)cf4v>D>bnJeiO5OHh0#xQydh(?oJ(EpM zE8Z*-)Zan3&x=zQ{!t*V0=lEFP;R?D5FdfN<&C9QI8!DJ@A=}y3kHH}GE(I942DjTlj^@f_g$X8!{5<6#=Mo?K<6O$-TOwTYpC}(hHkxQVlI~9 z8SAd8SEA)#=vK_#pyO%$TB$8pJbkhATkf?-9AC0Ou*(EfmswROFQghnH+CgmSS@BX zw3-oTt*M+nP#Z{veWByE2cqeB=~|~D0MjNzO0Q0+ei}iagnW5H{)bIegfhL9&A z;!0&8c@iQr!nGj1SQq?66cD9FjZXvgqc$0gV=N=f<4AcHlG#;-r{EBOZ(I)D(qv-J z6(*qEeP`7oHAG-&y@gp`XL$l;`Yh%7kORCMEb_i>+{5oVc^H^6t{iD}r{yfI){bU~IbOLuVz6cJ7}EjLn!6eRNGynPZTCF2za(o5+nC<@7oMw5dE(WG>t} zYqTy@mu%lwl^Jq@;Et2wiwyulx_yjj5jc(cYb zOWx7be&q}_`qN=$n4>~5cIZHo8QAX*hmK!#*+KYtBz=%EsC@FrRMHGHSJP>wzOZR{ zn6nm&c>#UYN;RL(_hz8D&_L< zET^nlG&8(jsfJ}U#xLc|O?3HVo2s9*yUI5$$Tmi(5Pj3_A4%$-1@=B=W8N4|Df~zJ zZcN*6;2`S_P1*f{Q%So{owOMgOSxw>tJ5jk?qZyDOi);Nxa;pZ%*dt@!Bge^EgPLop#aiUQyHy9s@wrK z`@R12sHC#F&)QyQKLgb@X2_PUSzxVc7z&C?sxuNtqHy3>k**2zH9J>fS)X4v#5Af~ zjrbgq6{?Anh}h8f%DJcoZ^I70qa~=C)LwW1k+Jj`48&{zXdC(CVMqw6huE&z&G^`m zvRkYKFKo1NE(OMezaCG;flk}+b>pwIQWmr+%riPO_*@m>Xx0p8d_=D0=y%p2diz5r zG(Z`h2zRwYaQ6miSFM>VXDOLvWz)#>eVQ9EM0pY7x)&6v@{{Yx5l^^KT>qk--$w~` zJC=@(ycf>^uXT<4pTE0N!Ku9(GRg! z3q5JZ9c+dyZN`i~HsLtt35(bHWlZ2xg?aJ5DS|g6F25}O06inIE|b2iVDtB5J|cL& zD5h8KK)-ZMC^7_!e(%VoHgnROx$U`kd%ejSBXUPG1_>ty$-+0Y%AqCb7h!X?YevCN z>K#=$K)-aB7S~__Tb2yUU6)l53!(pXYv(C#F78^l{&j>oqR_6p-DJn~T^~`ao5U&b zVM{MlFcfiUhyFJ`p8)-$J^B^{Mc+1U)p_VIj3#&Uo2lOsk{w!KmHu6lLG4(Yjsh{# z_-ovl*%X{_UbKp*>WChNqOsFN9_CDIz>%6UfEsC-t|kr^#Emg6k&}Z({wVRm&vEyJ zx7XlEv*Z13h?b^&j^ZSq`k>k&o7JJd-dq&ma1RR?4ezW>rxON5eRKy(A@ZXVQtfXh z0l7|RR`C@N=5=`=Cr1774>$g|D;~c7E6Ec`J_Sx1!`hwt-c)<5q7#XsWkDyI73FA5 zfW)<6#Xe6`K=yZ*_-B8ber&=*U$(#m{X%nFLUGsAlIzI9JI$1Hwdr(sWHqO8XNcrV z@bx}>m&4Ae0vv91DY8ryUlb@O)<3Y3M3OA$WxBGpu%JD8s)G1}2!}qo4tf~;XtaNr zy33A%riu5Yj%1G?^tKE)v(B)D3o7=G0Jo}F{#p7YCNVGjb)VPe;KjDF)y6VnY1)T& zK5Aoi1P0Af8uQ;VK0`FJ*P$cEeOD$aokV*lAQNR^RK<$LL5PjTL}tVONd#}ml)Jy3 zAoz2HI%E_`ot469^qn7gP)loYzJQj-b2xiOg5fK;pK0`mj0Y>dlm(rNQ5d~T0K#DB z06Z{_Gx)L**#{ITvI1W2f=&h50ggeGgv74m8+ezo*ZN?vx7gybydou(HDovFreo!a zy}Skae4vXrye$_iltO?lec}c@kV1gYm@I*-iCn4gT^AqYHi7vJzYzy^sxTlp66=$6 zr~Rlp_)6zjdR=06*2y_uZ$!R50CJ4^+%28i>T>m5LrY+x23DnsZg!&xhjAQ#rk_qg z@p~0ToE_()M*u|UiL00}<;d-4c}1-}wq87SgRL<$(B@8=7^{Id(r#>*gspVxmj+$ zvB>q2Eo#mv0O`baCWnPNJ8iZVo}mmA!E$4-IBfWhuJHkv_L@Zioej zXby5JY0qp%UxF>Jd~wA9f*u9H<5}NZutH$HU#~a!mc3wi;$o5`@Q3d~z!WrU=1m+L z(ypzIaoQ@zmHU4yo;e8hW+>j0Nvh$Qn{t*!j_yc+j7^hU4HH^4r*FM6Fa$vz3er|T zfu7|$cDvR*DT_HzI1J0W@fu7`lwrP-JeXU2{a0D_Kigo!|Ez=mPpZ(R*t!Dmd5+{2 z%#R;!80j~5R9LA6BoOI5*K8O7%?}^+1q^>m-nT?^R1RAX>X)B0;LCQf6+Z_=#6+aV zM5Kzcf9w>yHS$VBa`29y(-E~$(ZH(Id=$_SiQ9gs+9(OI*%i`Q*;m!r+n*tQ6!t#U z9Ruec*Li_vznX;iy^?&MHo2d2o^tN;TrCAYuNFK&=msqf_^yV#pYxCa1pdB6f>faa zM2$Nk`C&v2I}jXQ$3O`|gMR5&t^?5+dWEJz9HKCp8`GSJJ@c;qU;ev51&-cR(V$7e z2ji4U!8_xL==?Y3@$>GXLsz?J2t5DMuqD44oESsq8yKRv`j;*wZN|-O(iqN_S5Q2L zSEro5uKi*2t^*)0NACq-u;d4|=|~t$hQdraoYmP1{Z+DVQ&^T=tZjv-#!_m^EDhdj zJFer!wkn%BRdLj24b5nXyHjewzYLzNq}b|O=w){gomC&T8un7`p}xiEj2s#2(M3BM z{+ZPpid1(QirOk`(K*c;3R{eZz_`F#{ak*(hFbSd4)*~LSmYuA=on86x>pJd$M|UV zZc#G~=wU|x?B-`*pnk(cFxc5^>;0{$T!ENSEcFa0my((Yx{`y8+x`7=rvZoqAs+}i zDGW&7FZNR0Du{?3t2>JaO2Q@KfK zEl*2gE`}x6tO)L3r*S%jBHanCDK!fz;+gm6)z}k%R>~ zLmlZG(KV?-e|_`I`H;!?Qt0v3;IKvxUKSDVIEhSxC8u9vLgnS){PMPDOSx*p>cJy} z7{AC+L&945cM!@D>wIlxmxb2!Z1Mv-t{-vW9jz$1ub2!0|0*hXT}ou|a7L-jR+80J ziifa60eS*|#aO^rG3Pp0%DCY$;F7bQn8D2V#@fq>ns1^yNK{QM8%8(padakT7Us(X z#+;fWNvVHJ=uYZGimYU(CX?a9@)ABFTxskM(O_B%o04TbM!3a~W-=#p>_*ha!fD~; zITXGqaFdh)%rS$Em=kE}21}xH-#}#~)xIwc?|Ty&S!ExGPIr&IhfaD^0~4?=;gLA# zxcM$W&t8QE>bj9aZJr^or_z-&x1pxy6f}W5@S>TmLlMn zU8I_*u`%XHOnn+a zqomDwL#$f)2^s=o7ikI3)SDeVs)-67Y=&QjPx- zkhnmQw+L$@?xsmytT`F(4|iI@isD@*!AX^_2O>cXj^Bc3wB9=V5Fas#$+8xs1iuLl}p1v?!S-;Ti34| z*O8czpuea+2VIf%(Kz^6QAOjV!0Cd!`Dn-wY_-HpvFn3w*)pBy^oF|}DypW1b%?_T z=B`5qB8{}ACJKFJx-&a-OJ4B+CFWQ(#ICAOnLOv~3&zOVe_nX(h#6&{>uT=D#Vx_s zT@yt~{&FgxxU~muJ2uS_karVio#AwZd52j%!g`EMZvUYS3QXvP<_k??5$TmgDXOxD zHRwjDBV20;238$i>%A{&AXZKU9pwdKjfJ-*=Wy=i79I&?0WGc>3WIh5Sk)vcPt9JH ze5I7omZ?X+^^Ie$F#!VZJwHykAeh9r{BaLW7$rNhI`LW)!>C8L_N9R6HmnK8@VXhm zh-4xX>k6_7-s9?6uiA`oZee9+W0bdqJFFZB}kggr(#^|Ob@?0Y-fQ@{jI*vq8I?R3~< zEg(O)Sc-Q_3cti5;cpY)AQJC^#3u z%RITmZ*J+1%t!3|Yp4r;!#rUtpW{UBJ4fv|H9z6258WZQ%X6#)D6-4sIpnkLvNU+D z#&*JPw_(p)CQk7-z_YRT*yj=Th4t(GQC)HFrP;s{o#2~+z(w5ViIh)ryENnMX~vpGcW&zQBoLv$S&1~aMoK)^Ag z8v%mryQIAtkuRfDLdFr#?+QqPH5xowyp;YeP2o)VVWFm_so6qg?Ga9TlAfrO++whI z&g^x+Ei14Se>)2zYT+WBO2be7!`C6q(e>Viw}9wndBS!DjC~+_9hlTkB24hQ#jHmo zMEZ^b6klvGKM-)Ye#2dX=>V+Ouo^cHNi#w(T6gT$fhveL4>D*0X-<*uEKTniMb!>b+8%MOSnw!3|{57td z7`X)Iu*maIOp~VXsYfkez4uxDgl@g&y#*}ze^2wq?Pz;bvM@J0u>?s`f$O?HpPJM4 zjQGF=t4!VX1pJ^Ykz@Qhd&D(ywacI919G)Z-WAPF|Mg#v`Tw?u|99h9ZAmnf`^`d_ z`u6w{N;jrMz)2TKgw9V8DzXG3 zYR)ptoeOJMJatS7EsP^8<%AtYHh^?EtG=jhfJPPlNjr>c$(H5!D2a1L8yzSW(?#)g z*yD5g=cs4%JN4%C$2E97X+lHkE(jrwmio0+Xjlx>3X_JKj0FS!W3R)vpO#`#6Q0)Rszk+6;>>m3E(e;FO~cQSzgf-KwEhlrw62#gnoy zry{>zwTw6B$qk6N+@(3fut=`#A%EDs@)vVolfZ21HErYrFsOirVejBW4l~LI;w==!Nybrrh67=c9L6K`KZ1V-I`*7^`#y78tgDnI zAT0Wo$+>4AidSO8?v9kq}U(DKy&c&gLL?~}HU=6FXVXKB|D zN(0CW6bvZBfN+rs9tjlW$ol||y!-{ut%=Kt*SGk(tsm>+uDFP`xD9p#QPI?TuHSi4 zvF$qrr`u)0y6~68>FVU0Z4W4-Gm*P#sNaPk)JKn+SU0|-Lcz0qO%#w^m4`0P3((NP zKS6*T9tfvsFMld(#$}rSvhbqYyzO}xzh&V;W>E?1k47TFI0=W(w?UK4Z5^50J>!KB z{k4t;Uy4ZXy9rpr7`5F@ZDw7H6lL)6M8y%@qRS{|_0%0mYbIAz6?QQFho-)CLUCSF zC$%W|%Y*llvd|M3=eG1lxb5kM81Nf5zg%g^y5v_rOgGjOezkXb;pGmU1%G?S%{Gp5 ze-RWB0+OwjQZhwtY@wY>3z=qT4c3MqicLq@>|#}q3eZ+7HzB!<0tG55+{_Pu?g`NB zn&PcX>2}SG@X7CDIwAHMSipN^O!cqXo3a4K|L&`d`OGno78YOA2YVs4j%1a4NAy+CT8VYYRKz#THD<-}T4ZS5W^EJilb4l$Dlz4rON&^Tb@` zk4$qIP9G=g5EDN%l-=iBPj#| z7(Kl8eh={%b}_l#ENG`O5^coB)&X1d-d~-O z8(}(|DN(T~VSofUsszsPgi8Vq&^HvQSvNI8!jTy7fyYR%?CWmD*2#|PeSZ8HW)t0w zqt$m$H)hw|IIQzjhE=bAd5lwf+FCYIM3Jr7YTDrnga-ce4eq0Qi!;Km>rmlyxZ)63 zI+Bl1P+MrLKvB4u`2&S(Gmd~`!BKLLq1k^zJ7}a{f-bzWXG}%asOIhm(AVy!U)9ag zGjJ}(sSH@F#LsOCZkK^4-A&-)*CLvvMDFZC1LFiGaXT>bTwA$4BR$28#j8aJ7y6fec^XpI4gh~=Ruh3RRd zECJSM@SX}}&CY3^Mg#t>W46B9zSrN9Mi`&AtIS_Te9EVL@s|98`(s*XF%xS3$u3DJ z3UDu&W<-E1ZjPU&4s-SvPER7)N7v?!m;;+Q5DexQ6tXJ}_JooE*hHj>Kt&=ZL19m! z@!i_>$}&rzWhk(08ZUarSKolBhM1F(=@{-|v8;m})=_^>e@C6Q5 zIx->;AwbR8)z+1a^?zu4xvFO}c*5x4^|z^`mi=LIan)w-0h1UHRhNcd1csho41)GW za0TANJvTHr@>m!8l|)2#kVqf^i9Bxw{Jc)&S9&i0(v`2P#ncDzf*=4_<6D}ii3pqF z!4$THhC5Lk%kT1fk#tkvqu0N0RdIH$PJi% zrwFFfDCAp5>A>^Qi|Kb~4%EgX?`qrmqcsG}&WRz$rosYm$gb3uw?bvlr2-AifS zi6IsTqMWBg1cdV481}I%`Kp2lMoIJ1Y3L`+Kw(P;ln&R;n$5m}D0E zzx2-KVaoUx#2-JjNz)SvzN=RN4i8d#Jsk-e!2NTY&imhSq&PHjAb3A?f&?lkNSt;o z+0Sy6>;csMvQehH0A>@3OfQoL)nV!6R%zVd&T9iRcJkQ?YznZ;(57~Kb_!>g_NZ;z zIUKrcY}Rw+*1M?5U+Z33IMS>p5zp;c-9A?OeQzYk^HXzB~gv z09GDfj*yz?oY3iTqaAMjw;)77B``nqn(uSH-|+a3(fzQF=4RbDY1g z=I^h;{@$>9{I4>iEjVL=PyH3ZRzK1`U{BEPRq){=ocJBYMa*gzVMtOc}% z2a!T&h1C28hT^4S38^H0;6}O>SH55NigqRj;q!OQ06)x zi{vqcR+}s%7RKu?%IDnNG@X6S362NebIjcL@6(5rk~9|vRWe?j%7SZy`xu^AnbXB{ z`D9Yg`7R~}EfNP)VReI_BthUbm4kP0bf7ASHK?qpi-mWjOl9TV8I{>a>^owv!py0x2glDDp)=+f$D_qO$P}Yw0jiWK68agk*iZe5&xIU8%w$76J>GY4p^d^UT?Q_{v3Hm_Kw1# zUuRox91m$F@dJhfbo=nGb*M#@&G$`K@kp=Ij;yF1<`An*Qi=UoqG;GdLL!DD220jL zhb7uN2!HovgH%Ce+tIodK${xtE!@yrYQ~?{)KIcn@#qB?bUc zI%uXk(|TiH^2ZeSa>c14>NnIa7)A2}jx_gdL=~d)bcWN(6x8ks*TRVcS(3v57s3Vu zqnisGE}!*q?%!{Z=aS1ACiRn?*0c%Ci7fnb$X`iQIU_FQuFLy1fSjhilg<4|6dEgh z8)Xp6DVX^=cz8TM(T4p1SGFl=8FNxcP_yntv3VSe2;@fJI=X2xO3EQI1&)={X7U9! z%Cz+{J&vh%mbCL7NfGx$O}^~ob{|9}uC(*XeWbqAG?PSKJA;}`99aj;_p*5NmkBwp z->pMlEDqDhiKLYSfGj>a#Oyy)d91a-1B4oJO|?JH4EK@Nk1t4PaWs>EPkTCq6k#O7 z!jqL>O_|3W^yAw{(KiZ)lKuqw(I?wX1^_?4WgDE*=_UCeBEM1yWD$qa!^?CVu(bFR zc1kD!wq{-MDFio3-WLP#zbpoBTqJAA@8~Ly7H@9q|LPCI&R|i4=MDi@i-dS?hrV)~4T+2`9z}Uy@wyMMq*pemMpb~w#lSr%tgvPe zGtc=0WntgG6@cc{F5WgTpEGq5*4$+82jVJrKR?H#ac?Z=4g$w!43vGB#3Ackx=ycV z;*b1#<^W;5-Vn`|<6#93=vKM}$VU|j9ZX$Z^(n-A?Z%ljaj8bXO~S&S1`5s<^o4-)LU%C3z)-^0V1_1-9!qM*VSi>O$yw3V;d8=C4V?synwo;YRlpvXi zN2lNGg*tz*&9RKeQ6Z}KfhQ^g`HiL8X_dxxSk<(tZe#2tY37wGu-3Y1veEQ4s20f@ zCZa~N*?{J%1xovq1dQs_R<}xHq?3Em#-!BAA<};iqcpl5W-a8lcG}TT%UrY`jkhbe zuny^+X1Vld(xiB1#pKy`D|8-fjpCP>&FC}Ye$*T*lv-w6=b&bBv`N1j4;hR&*T}d$xzv|D?1r3R6&_Aegm~RtYwXKPakLr)?B$^d;RUn zgKZ(PE|1Uy zHKuWYYKtlQJII0n5%-~Lv3|Uvk&c#d+W~ymrug3wyIU>49xv7)k1y%PdC&B|7|J7m z-__y!3}p^dtc&8#kL~((ydVrqKaiVDS!=whRnN=|bSszObD~-u=mdxCDq%c0v0B>+_Go;ax&pUlfp)20irLT}g>Ig-HFZSsk(DD9x8 zjM1gjm508n^2a^gL;T9x3M9S0^)oDLZxGC49CfWQCZ6Z}J$Lk0Le`j9;cB2Ajk+uv zrqaito_!h0xRp?b%&(aUg{=BK3cKGOU=i0u^_estCWMHjGKAwZaoAg+HULO`eCBQV zU8=ZQDL>U8euwhJ_oP!#>d@;Y^#P$i^!RN;U+8d@4n_EV>-d%vZX2@YWURk_;jGUL z{$^oB5{#R{wjpA-s&Y2B*kq1c&E(HhDe9g>f2GZMqB~`fE9!0g;V0Bd5#KXcj8yqP zIuOaXYox*D(+k>vmOuDDdJi}W>R)m5y7&2`yL4#SALLqI28&=QbimF=*bx-E^q|UZ zF~wt+(2L~i3?N*3AzW%-p>!SnL~nRdY_Oj-Gul%5+d6kRFzxAc$-253&*^zCFyZGf zuoN~t-nC%B+G!A1_ z`WuAXg@WYmDJUl~R>^AGr_xf;JK6MeKomYtQXOx$a^&=qvEWUz{7tfA&Fl1&nQc2B zhbO)W1FIf1J?63n-4t-=q~^g)naE2~>>NDX9)$aLUwL%$gsa$#3x=a#6ot#+gE>At zlcZX-Dt2t1H-t^&1mWt$POEl75MDunMDaR=&(O;aCmw&~-4qjwHfXkP7RG)4o9fVd zVu01_l7;qctXG0_>{@sDcN9{qAMPFHOIFUr9Am};V?-OSCn(_Az4Cz7PT=Ha8TM<4 zDb;2H4Oet|+H%dDO>bRV$-?1E5YJOX~-|9Kx|ehiOU zkx(qR3kKm9lWkmgmc~&uyNmS9%3X{(Y>NrE_L}WJam~>JC8v1!Fxd}t^A}hXch5j* z8y-6N)%LTlHYs4yXL+@f>Z=Z0)Dlq>ZRFNLZ4Bs@v6?sI8+SF~;u}IkLwAvd_PRDy ze~S-_1q2}^5Y(`@+4emEJ7OxH?BeGNs|`->(%6hH-~-j2Dw5d@w|V&?{PHtvUle>7 zsQr@)I7b$7P$ts2Rgf<@5OT#Tw0ylGlp5;6e0hru4GzE_z8V_H43F(Su&eC5+V<#< z!r9eJpcQR4dMFEXEw}RlwPi5AfDrk0R2x_285(v_6c<+2J|6xM_$q|gR%?q5d=2hs z5pddpv7320QqnbQpik%XIc`cAj{E`F7=sc>n?bMmkG84lGN({Q}%FCSNbGJhgF# z+q4LUz?P$FRmRZT0s!ZrLH!rzYuKLyBwpN@l{LY%&0bc|^>X!S^FzVL+h zN+t9JPTtDjkG0i{_Qd+#5(M4ruaj;8&-UM(_Msxikr1qSGZ@zRz8T2zmcqPoNK7ly zUaHT&dV(g~cXKC?IM1NRTU!!8R%?yXXipET z+5v0f!N#uv`L>NsAKVi*Plkh-jv5TOl9LDVL@41zXyN%YpZP2v>pvj{fht}&1TJX$ z0;6!*3oG^7fr15S(4F-*ixa_WXGq;l(-Zi&pw?lKs1IMocp0`0KvT`>GRz^hzP875xpss$4Ke{IzSB8Z*u>6(oZsh1C+Wa8STsyI9OFnSP|{8 z)vk}oGTqxh0nreztY7z-F|^Jw@m>RXo-8!|ux$wX3=(;>{bM%9%F!PZ(ZkwiaLl)< zQ^$lrs~83Ch{Tks;{xM*aM=$vy{P)knDOK;bvbedlL>H>#rWTixj@7w(0NUdctGJ1 zKHV=rkS|j$rHv18ZzABty0IlsYN}k1zOUgV&6+u%Aas7Wxa{Mc#Tjc?N&pr{4cEr< zP>1sH5$!llJ@_^9$h)7~sZ&q&?$LjCCjUF>{@*h*9i<^hJ^A;rto-8#<^MM`^Ki1m zr~ggk08cM~MTbwf;X^-?6=sVZd)3xIBZwf)<+hhm- zF;VUJW})|qeM`QKXV+xFPNDOAKzRPak(~b~kntJ>i2&Hk_97nWJ37foc~|$mMDaBw z`tn{s+WVkH{51vG*ED-q=ln7l`ZBsD;&-F#r5OJl-Cv5mLBQ94f{2i99lkQ%+U0w^MFabZuj4hqIu}6xv=3S`qrO$${WN*+}5Ss#oS3lWCiX6$N+!<2k6Z-SxCtDWRhoI%4aG&s2DR9FZd z%*(w97*OSsi(!r**?CP!7x}Vl65jo^W;dkb5gwsS90nz>QX_;rP`0QPA4S)QmUk}- z_b(YsUO;tE;zKRz?608-O`_>Ds;KUnkeKcQ+HwNI=(&W46w(?Ff}C6GONMGRMtsbZ zbV?O&wi|S~>Lm&k+^gpJ*i>^Fepm`gFX*)C9BBx=7%(EYIH(^f`?vJfobuZg!?rJE z-fo&zLLw0a+*V4o!NU5E)MS|*)3w<=me)r~Sw`#Hb1Yz%5ifi~0_@4=_2&M~Z(BBw z1q%ZJuGYbf_>bY4sCXg|51$rg)EI#?DFuS4D;%Zds8$)M&=T*H!iGR}5NPnT-zmHV z7~cJMWhwun8j~t9#uw8x7gFdAwN?5C4wmcL(BUP0`$l98MWeTVig23K)73F(D=D`= zmNYKdR*l;%GU#il8ff>Fm6Z8fWBgcTBG3nD8?^jdxosRv%EA~Q|DmR=q0{8{vU*_t zTwBQ@MwBr$@G7e@PdB(7gmHiYx3rkk-c|iNP>KPUsRGQ}uAE6bO)D!43qVd^3FewX zB>*SBtOt^8Yde=YHaW^5Z@1w2ZL5iT%A^Da$l1duqT*A-Z*1z|4BV3V#+7kW08Zj)s(Jj< zbLx1oi>+*&Z^4PwJyPY&O1!0Fy_g{)2>yo1cVVl`8j@LlD1Z-+vWD)PnoCHMYzUEY zKOx-yng9MxoN& z?xB+G+=TVyOL4N`V;R|ZWD+|A;STit^G3f{$zYu*62^xL5_NP>s~ zjt&YseMVST-ow#Okg_3=(^`i7GCx#b8lHQOhS&y2kGZu)boO2p{&YZgGOuGRS>`F9 zYHmCo>K_tShpGKOpFp;`%Unz|3zBkdIN4Ud*t<<(!7?2m^) zC$R}fXTwC`kh}&}OQ%KpQx#}q^P@rJ%tXR*DJLQt!E3f|Rx^f&wCaL`8IOe$FDv<3 zNIy{=ojq*y8Za#41q?tsZ|N2VRv-F6J(u)FbU$v>A5=nRn0VORzhChrIaYi#J(3xN zK8zN3<{})E!Ct-6+e1EsoR0ki^TT>};oTh(_orYfloLwF&uzjA9g>cMf6UtV!nwIp z0m>g*9;4pQTO(nhg$&B|NF(-1p~<2*(3B9JOmuh|>cJ*$&_@6qRsDcfFmmiiJBP&c zAF|-M!i0<{HPC%voRa5xJ9zF^$eQSuU6Kd)C*~o4fQh^dufUyC2kH!M+Hc;oOR*TQ zzOe6wp)fw?n9F6ElBj6x;>3OVs0ZiZB_ZjeeI;C=CD90-?F^2xjzUhM>?&pF4aF#G z$5SyU@)}S>k?F3n~zUu!m>q+tA`_-pT*}A$0!c0o$0K>4L*(UE(-oxi_t#^xe1&yW~WV{7g zf`n!4Vbml7_%$8{IpiZ^hpI*mah7pvqE;$LzLms&!vNq1Sqe{cL-n3s|We`}R8arpr18%m}Fxq~qU-eWrk)V9pWpusuAIm8onP|k1r z*l%M;bQjH0B%dP*9V_9^WI{p=L9?O?eOKN$Ku2(@0Zl08Ntg0-P+a}81Q<*FM z)jEAU<&o8|`z3aAe#uDpO4>c-riVow_!F>hBTJeF#l1rTu>ZTXotha+(jr0z`Xe$V zs@pQ(wi1udy|@D=^eqwJ3n8l7G!%@}LzG8h{%Te*6ShMM;xt@TgJ~nO`cR6gCN6oO z!D}6K{MZ0v;VoPK0ZewI(g7Gu$s8LECInse#Gb;oWJ|ZlPFah6%h4J_y7}Pm$OJ&A zTfTHe#K<}gPwU7v9w_$4eFSaLuXvUq>3_imb>D1Hjp|P85Z$1T@7zXpWfQ9Ya*+Ly z+AyuqaA)MRVWg`fmyQs14w)+ILP!YX>}nw2t%0+L(ebUAAPo5tj|SE=tMXLwCjSsn zN~nh9-n7#rBcsZ1S3y)IN!Oz5D-Y;domx#ta9R&|f?&@Bu#(Z6c(rFtmCXX@-H zqx9}!7oDnERs8aVTqt}jTFXAf07`T}PNeǷfY%m-stW7?xYoKje55RufTC==g#d zLJLKxcK^&`<%r`dqj-r&{UFN)cqr;vQnS=O zC4Xov^)L0ezrlFdm;Hcz-v!7%vqV?@87fjdmXFqimgvQJ-qJqBc&PM!-&6r0&s?xP zN(z&Bh%;5v8UWPRjGS?U%k!5gK+z3U0gax);5AS42GNQ~tQjmmNoUdWwCA)#_iAj> zns>O-9?%kX(aT?SVan5Z2MrK1l*$)&4_LC@eXjI~R0)ezuT1hE3jh_z(dORKy(=Al zNbMP|Q~L*n`G-?2{eSD{YZ@RA$i;cl@M|hpl>n_%eh5Kdk$kz;MwAdA=wNH4CFDGb;O%Qtk--?5vO2 zSGE>oy!Y~{{_do$XTT|Jv32?9Sf(C!LdVg?!8g4+vpM5gmV7MZ z)lLu^%-k5`^SjVj6^_nP6~hXj?YgeX9wF?r>a8$p=YmbpD?m~xrB@=f$SWm47u>=p z7f6F*<{`Ek2sVwjfjd`vOfT=gd$&uViA{WAEBm2HkTQcsGcRL^4ZmW%$W3^?LcE<& z+M}GS!q5y3S6Drh*>6Pyr^{ibgoyjcs$=R>%IhX?UQ6#NFt4zW3U8=cRD+1 zrfr-Z?c-x)AAs^yrp(kt3_e9i7ZMNl7?Tb^!6H`{2Xf!0-OR6q+RIrLa#gsjsO;G9 z3__dz*}f*e>cboC(aRQ`7{OCHBoxvE7qOdbA{2jap0tN6$dK?o7H01F6Haozg1?TS z$l7fwk?*u>Q_^a7%3(O*W&r-N3+nkn5nkrP!P04D4DiZBd?FE+OUZJ9e{5i_lwND= zyats5R@5X0kEA`sg4CN>@YObqXN-RM zl#0_Q9ZbZ<%lZW~{pnu|YH@L9%Q#T!c)(o@xHN~M;X$Da*??U1i z0K9h>Sbx2PfybU^Yb*84JJ6UJivH;jyyy?5@3FYvAcz?3zIXWR+@^XcVc0&_i}6h< zeqG4j%}Bp{CP6X07?P-&u=v@f;-ltMOK>ghNWs+h&;T~h zUn?FW?R{5{jh5B(tICVZ=pa3j%;_Zs> z&$7NuwA+$2VJqR7r|RX3t`ruT!UfboS5e(R}3eKy_-p>rZy|->dL9=7-;o%TIxHth?>NA28qQ*5aSLl zUZ}rQC3d?z#A%P9HA~<&Y7yBu*q^I>i~bBhq9wd36qpQ3G`?$zeZ8wURsbOU$Sft_ zS=UBC7DQV{4IR|DtwXeny`$<<~6z8-NwcY zAi5C3TwyFLzxZN+4Taz7yn4V4eFso6(`==n9Kj9=U7(Qz0 z&}Sgi_y2J9j=`CPUAy)qlT4gUY}>YV$F^W>L1-* z)%~Ma)z!7m>s;qCxoF7DKHwcS|1@vEg;AdhI5us2Qhc+0wP5y%F=K=qx_86^%9^l* zOc!c*#6_G!bEVW2c{sBs&$B8^?35lk<7R`pzL`U0OR&o$Z;9*aA3J9LcuVM+@0I3{ z>?!%@&trjGBut>Kx#K4hV(SvXOJ~8zNTbk-f}Iq$#JoBLW$Z}gITCjc23j)qvSQ&aVf&V^PxK|3^QTepB-rjZ>Lch87$oo8(CN-Z%Z z;Sm)b3V-6l4BtYZ7s{nFTKM}k=RAP7CO1;EJb}9+%ZxYYZ68RVz8hCGE@##fw@R+OY#(0*pH&EJ&8>*h~Acy%P9>inF( z*1yGR=tjrcp0c$(A$h~}Y64@6gYd#wuiJH^Oc%QqhiuBAkv&nsXb+yBb3+Mt)^%aA zLE+t-Oar(%{E8ip0<4dwX9Z(sTF{ye2k>Ll@OTUJYuX70~Jpi=?k_fnN!`Q6%R0^COHM0Pu9PS;5 z=PL>e8W@OP;-e_wSBxG>Ia#(p2PY~0od)SS-#9QebFsDPDjv%V7h-y|6S&M(snZL& z%i2PR1Br$IM5A;8J8z|yU^cMq5*j3BrX#QVvu41b=(?Z=3`^WjM6)NY9hfva53Q!p zo7%5fZpLWLAH7}nJh&|)oUcof^BPGlVAji6D!-{g$@zo?=B3iMg`zf1>Lo*)tKsW* z%?huwXX*{#b{yA=xEM+2!($WFrbNr)yW2pO%Bz`Tn-K^`210F_Gqw(%8s|5N%AMDPK;@9Zg5r z$()O}B>TM~#kEjJGP>Z@IP1??aDLlj5xPUl_2ezw#N{pRpl0>f68~<=))FHd;u2ri z8HboRIKpS(EFp79QBf6>N5~nZKG|1HEVSQs9~qI~CC};RXjK54CP;?T5Mh;ec8etJ zOA@);MRFqHEm7!S=k&p3hEkKfF6ezd=yACig*ecAq(UU<1|t;X5sK08>#fLuOAhEf z9u=|~ETn<#_B69+1G+JUr`Z=3pEQ_zoSX~kZFu0O@xOw}4GK!;;%i$h{95hGe5LdP zZL3*(6e@P)o(k%=Vknm`G}Xo!M-N_^y>$+1>Iem$qC!?H&$?1G*!6q&p1=`{Q)C=w z5qvkort4Crp~l>0K;V6m}Ov_1i}C=99NZ`s~--P zZ`{t?8t&rhmZMdwrygjIjl;Vs3M~q@{!6(|U{Df$9*KO^>_A>T#vK=HTo66&or0vr zt(TM|!z}H=ItUQUzzm;VG;{;(FDyb*PXoohBe0k^y(r!5`>LDTR#A9_FiQY~{#cdy z3}agWnYVg5reL4Ns#iy7H!dSI>~ zC}&Ld79jD@h0|{JOex71{KJ(Y)^KXyhBhec3mxuM>O4kI!0_MXCcoMO$DWYoyXM`u zM`yB|yx%(l=(o|%oIBDh_jDccKoRa&H1`~QdFmZW{^O0ZXrMU6i>qh&O_ACQ-5bn~ zF!0FhjJGG2Z$I>ud0SZTcHjZ^9}vT0kvOYAmT>awXl=8`<;e0S#Gq~s9C83!cdo_v z=a_a+i*q8K0G2Ye-bo1?g)L%g{v-4$;)UBlDcvly^sONTy^y$!NQxH>{csf6=?(>M z)zJ1a$6MMT=2sEif)V?c#vAor<(RFGuJG;wBuA@6b(yd}@rPmB5IIh9ap03I7nrh4 z4Ts2rOpOSSYN-B*xvQZTH!ltZ1xWo!_@lDG%v(Z;kBnvUn2OjD5g1K$MyO?=2&^=g;zm?^+yM{ZM}Y0A8QKI9GfdmFoP`e{|~088f@-#(efi(2nm zy0_>K{Ria){Br`tbBfnK*plzKmB;Ysn0)c)Uiq5>`16)N2(yQC2}XS-v0s?i2mb}D z00n73SZz-J&9%Gh1wEU1ht(X)cBoQzNMJ5jXgl+-#}<6TuFp`hcjqCU`7xcDBum#) zMd~UEpVnGZbQgy#05_b~8O#-eBbz@QrWSlE- z!A^g$h$^Qm{1w%jP2%X~MGi?SZ9pfmlfEXQr!*&wTvth_&>T)6+@agLLp1@d8c!yZ zdkMCzaq#ZOmrkNcqT4=;(7wt7%qHVHyHK}Nk>rpoQF*3}07KDSP)IASB~^5>@?%Qk zEv>b2#8*a#6+5tuQIJqn2SL}c$h)FK_+)+1nvJ=279@q8Y(BeNvU_%johzV>D~!y` zGJ=FG9XC}A58C|wI2V7)R!3be9LBMY>$Ztr3YB%y&A@YcLLv9CHe%ZZuUb=iETYV+ zi=0_%(x{S;0D~yPfh)^-j^U7k2~d1PYrOlHe0ceJgl?<_rB>M(L|En0ZUniDuHg}< z1$2%>nekjP+LpgYvn9mJ4=A;!3LGq5wTBt_QW&FFd}9j@ym3zTrmS#ampMIru`s#{ zL9z9kknu{gZAZO|Ab=-I0bKqxDngVMEVhN_;&KAVK)!4_7dA&^{34V@xj^M_B_Ru} z+aFd3GaKQ0o5FvpnOhFxVf@4VcP&-i@9X|&8EH&9TuUWK7$$AVmMdrGgzsJ8NMvXx zBrbEa9@(F+gl$Sa(VrZB+X{&hr+gTA`!iZ$=2eqWKESi;AU@x44&kAb=jjGk;7|McYl1wqW zQ>h9-OFJ8ZQyJC#3t?i4KCA(#?r*=7oa@hJS~_*N$3(f%Todg}(52=?XKH)ZRT(+) zwhIV053?*#pn|<_nH6achF(<7D4qszy22-y-4J7`pYPP(;y zdva8gEo`aU9xyX94Mi?iOLd6EovP@tnoqcSoGpC&d=UbOL(M~`st#)@KCP;14?~UM z6Q2xu{kySv%qs$HY&S(Y&37~}EsF>DgSIehHod;vAdQ+WaBv-u9os+KYy+(G$@>7G z^lb7m)Be$*B>L&h_&9c?*JA`VkT+LgNjx3$57nM&ted!-q{VyJc&{QWbW)H*Su!1V z7cCrh4TjcuNK2(NoYMiF82|GF1Q&4F2T6=m)}#_!zus=Nu5FN;y2#;5E%Shm=!iuG zi;9j&(+aOY_wV0F-#tW`Ca*~4lCAkeA&NY1lg3q{)1us&cXKYCJb!-WA~LZ4AeQA& z7$gaf2;x{qz6|VJ^nXYu#J2LoLHg}J-iAnv?#K#bFIauAHsbtcNh`ceJGO&ztUXO= z>uT9t9^}rNpv`3!kk?OUi43I#K^oHRernYKU%-ZED(D%eo*7DDymXH?ve0#W`U+xX zi}yG3N5Jydul8e8-aNNb%nqP<{lBpULwH~!$ZA~bL_+nziFrM_XfJ(5gBZ+hvOYaC zuWdA}S#c@Ag4nVJs(?%3g`r<;-A~QpK z?gE30-`}Ok-~Xgogu7uA-&wQse1#(yEA1_N^GMG%DimiFqCN4B-TZ(VKd?bAUH|03 ztVV*o^ZwBdu7_ZJf*f9FXhATNGq}`<>dy16RRL?KrVS^(-0ekUK1)ff70EnNKD-)|vhn74i8{ zxa846XH)7$W%I13vbzY3E(Wh}KpZL>#PJb_vP(^&he(Qkk6RcEpQ6CDiK}6tnizyl zO_4h}2;+Gku6C1l?gn^ulYNF!LvjCGJ^Fnj=v$;K4EYZk_Q?gS`b&DGyq93HOAux$ zlVP;=6y?!PvvRw{^N}vQe7jI;k>8@!GuJootC-V=2vRpvQKSu+5$4Dv*Vzxo6;oo7 z(|L+i%7Qe|xx~m5*)wByq0fxvamndzt6f*GrWLScUAB&~=pkE|E*`!!tDZQ+{>sg@ zFWsDds>Xsd6p0(s69Vskm^w9`K1)|cgJFDpEYaN^doA1|J>YBs1PW@WEM16*`-EMd zBOh|$3kLK5O5_7Zm#;Epo2*f#lxqldpgvNjdn58Fbw!w1&QpBhPd8t@eewcdRePINGOO(%bgJ5{WMj3B0X9BC(yq2 zK51bbtE9RZ(>;?V?hARz@Prlx`Ke?xh?lj{IX>|{w(SNq`%0XC^-zt9D;#o%6}2Vy zpN|s3igra?Pn^v$Y&-a!j+Hu3bcTW{u`SY>O*`QKb(H;>^2{xMak-g&Kn{Ax=Xdv zrP?7kzLO11LL^TotnZ_~U)Zu~G7C8|3>5Qpo#Tw#Qw{yuNkO6HrogKkd!BUr0PhKF zpqblviAsz|B)QfSyN){M=mg$-_;AR(NO*C5U)t%qDP)T|z9}5L10ZI6zT=ipF>YP> zl017ZBA;rsWYr`;us>tO@jmMs^4GgCkQ^oMYwwcKhR`hRwSMD7KJW zJK=r>$I`VX@kb^9j)LZ8!4ztw=);gKVT__74hCQjSQIb@Ku5*1EQyz54pU^Xc|P7Dn*4iO{WE+*fZ%^qs$ir5Oqu@^D?8;ol6<99US4CvF^_V55V5a>-~I9 zljec7Q9}%dYBVuaSlw>2gbI)Jx<@!*S!(D!oBbU-6QiErvE?AOHm8Ke_^LAJ_;>=; zIjT@sm%Q8*MvnyimYfGm>Q6Gv5-kTXS8*dbsg>K8%rypHUXxxhtvWhJWZLawUmQxz&m`8uDQ=cqa-MXWF?=pKo<7a8|nA({WagoIAR%E)a5=Ep?~5;uJJC zW!goPg1qKwq;;}hy}`c?73xnk0w93A&lJ%TUSCB`C)}stoI2CnauE7vy_b1V{XLQf zJ5qIHsc@3-DisCrZ;zW*gWP@f?HaN32Rdw2p>-C$Yr;lcr>*a&Zg1XfSEw96PD8%X zuJPdtCri(%i!y~ zbEqf;Do>@G*yhkIqw1RlLVGs;q$7%(qunDkC&`^`%o$cO1m4W9C12(1WUu~!y3c-5 z!+FpdgRsB=LJ=!?x5oJL7x2e@K>mSS>dNtrhAbKXA4dIr%dI{W~hwuNK8 zRt*;Y&*=L9=MO{sHi-T22r+@krYyUmul!D`t~ZuBun-K!4chk>;-zqKty>=v6+wfE znVaYSa?<#ChRkCi74U6-1=DB%iVc0eg3| z%Tx^yM3%X>DR&#S4UkzhDmQ)%B8)6Scz2rrH+eTr7r3#2iMn_#McQ&6=ufeID=3>2 z++#6GM_<&=F1xZg_}dyuW^Tem1G=L|CfnkfF!>avnFure#@nBhKtjmbVJ23uH)Zo5);o?hPYG9jJ>t|< z?}LvWs+NkbEVPtUV*(YICmh|8U(YKasPzhJE`-bVLPl5qdn{QDEIL2Lm58fKmS^!M zXIB8?WgZQ>vsglgOf++6KbD!gqNY7) zl-R4}_R=6HM~I1V5lEl`yOD&;PP8XBSmgw!R;w z`IWj0!it`{D?tyDYIjEmlX{!^O;KPLV|@kz`|qeZodjzo7}$R%^jabP;ky5at?(U> zJ^F(9<3|!h>Rt*tdTOmF1#0TQ05}MsCX~DOlFL^t%X}s#FTG|oEqw#+#=Lk-*XS_! z=wC9eglN)@)vCg|L>6}z=E(pX9D<)X7xY8Aj6ST^s!0u!}bNn6~*&y%qKvU*3-QmfHS-&n@+VK^SQmREPlm6=lN*J zuaRhmua%+L9jOaki%&tkoAHC#`RDH*m^hYy>?Ak%stPdLvv;}HxG8*CKr=qgY4w>% z7s}&!7(a1XWO3+h1l`W!*vY7}?SXY>D@~r7NH*$Kz(dAq(a?3jMNte&j@qC~HUCXx z@Y+nH*CA!%8Z+ux-SGS22;qm9WYGInwMZlTBHC>Y;y>E%{mpN=-8+7+ij^z zJv>-)(ZUv?J(Qs_SB?Hg@D2#kEWm$=qaXX%dIl>$%*Wy{%9Fx0+AD| zrupb!nk2C*LOq(toq)n-Tb9T?mnK~SU4ljQj<(uq3G6KQNtwqcg+ic=?Pc+lVGb>* zQ6&S*;T+pK-A83mz5)r`kplD8ta)iBU4hPD^SQF6XxH`XIXreQ^LT*Q*6F@SLsq0t zQ|yzKb2v6g#?f)C)R99vAhq)hx@`+eDhaZi} zlNTQww{m{NCUmD>$qw}N>!m?@oiEFlPda5NKlf$?Qi$7Qg~ICbphf(q!urh5$U}_8 ztqC9b@W=X&ONkUC0x+mHo?-dgHc+hmhVB*)V)Y6n3>i8W(4|2YRX9-Dw(4R9v8!t| zRpY^UXaR>cK)~*E1=cMDI4f+1VF!69s-LOqNS|5Wx&7*{cLxTIw8G-+WrAH^{A~yZ zW!;mfkZ)WAj+)o9d$~G_c|_1ISDBR~cXo%K3-eS(5sYF(f^W7vLDJz-G>~QdR>O~g zFbb+bp`bMVJ1n&2t0$A~$qHr!C`H8N)R6+z(EpCW>SAijl^8d%Acmd>0F^HXiQBC{ z1%cMQ`HUtu>p-D=br!U{5GytkO)CJSNCaGqXW$hGU_rh;bhiwlxlxdu^(_$K)YUl^ z))WC~5i}Y-fxEl|eEhX>;WE_uI{p2Ov9R7?$p+}3tdB=-qC>{&OMGR4l)=xENRoNu zXYepmvSsb^S6A~%-Q#B9x+=rV!s@bWNguuMNFah*0-hA)=uBFX2Dse7NHQr_eUSkP z2z2ic^@=r~BXwYi0Do=)(beV0xGxu3u^F=>+x&)|u=u4FTU7Vb=HVVb5E)$tsApYg z=7Tfv(OgR@@j41)9tw?UDRmZ6xQ1pyPP&WgN}OC{$8CoWuE{vL?06;~LzJDuR%#^9 zv+Hc&zEfm(=rF+JdDohrsZ{urIlRdJAO$D77V>A&e~>opD{0ZXE5EW`H1nsu36pMpNF#x-m``Ck^q zC;xaXJ&gX&fKmmwuq2q2)3a)R>4-ff0QQm7h#8ho0RGq+F zWH>R5jT)&4EshUSPeX|cC%7c%jW+Ts0DD|}$)yi+bFFEmL?)gP`Z2BDT1@DZ^&M*- z!K;Wg>;^`hH^X92+u&C&+y&qG z{5M>1`a+!_3gkux+=F5}g_YptR7P+)hKEvReK%?GM~)nFdGK~2bcR%1=WSi$fnm?L zfP?ckLNI#AuZh(XZS`Ub1h6PsyF9vi%o|li!y}&lBoYNFn*NQOkT(8Vy352+ygaqk zGiN{0zo*CH$toD3{XE7^wRBp&HP#ZvvVzcKux(6~SRRw6uA6=c_Zp1lb)0zm!tbQ$ zr36$GZles%dc2Bv`;yT_T%wdEK&7mjxibRX`7;K*{l5eQ{ifloPo?P#^hZiAj!2nA z@$x}dmf1DM#8^J!r5C53(YCW^1I~ppnL|ppdk{Wg2o}zXviQZB%qz{(b=~<3Bd2IW z1{sQ)g<39e99ty^iEY`>MxHUbOdF+~|KwX2=2Pnf= z>TRv|@&$5EkyH4$q?ppVZ^zcPXel6f`AqgLO4e=65Wjfl+#Pd=kj5M?c)X)Mj0T3Z za+GNza$LxOcwYstYVO4OjmL-3>0%{&-!$Od8+v(4Vqhv2EDjJn4FHzsH%&J=+7SD(!4m8^i5^FaM5y4RraBK8768 zCDqjxIJ_JuP(TB&*}G)qYRxgMRxx4Hb-H?$qezS+_MM@h!**?qpmO4u+Zt{=+og9K zrIdt?jD#mj-f*X|wK!r+YORnS9DhUwS8X^#SRGf`I(bY80w-6N4&hSYg^$K_j#79h zhH{QDJM@-G9W{`F5%wYXzYYg~L%Y7L5!@@q;#i<{>CQ1WaULk$m6tOkMTCap0+A+a zB`=5NDY`_%)DNp1#8;sO4}Nbr9H;H8E4r2!cO!m97DJNVCOp z;UMY-ET0VyMCM=P@$5euL+2~b;{*EwS~K4d{|YiGI#H0Jk{Os}SVU!K&=3iocyavg z3DqKf8kZ;svQuf1{Knaf%frvOv^H+%vzsMPm5zz{Y#?g{EG4YsI(9*HC(Mu`+?ftg zrw*do?0c2THmZFcAIPDwYFor0q4FeOmIdhLe>p1fc1-*dvf2Jqm*>jH(%Q7lx^hp} zVh(GEKSo6sKD^Ok5`s-N@1@<%{u_hd)kchZch)8xct190E_Yowu^=k)t6vcw_ME=U zd(wiv0O`;*oo>mUCt?kf81zpxD5s2Lr2u^T`4j}j<8dN6B)bDQ4E~WFpK;ljRuwzG z^3tl!Ik!|y$)Cgf!>~U)3+d1B-VwRE$F;np{=yF)6%}SE(w8Gei4iBV!Gw{yGWPkH zxqIJ2d(of{2Xxu8*&SyJarMjrOvHNitT8q22n}vF4PofcYt;b?nG)h(GZgMO1rF^c zleDOb;xr2V8={=Gx=Y2a``mq^3|cdh|s`}#MC=F z@N-4hNFi+He7K6)Un=H-R84}Y>v_;N^1)~n7F#pZyNb7X40|S5LUcdDmSDOJ)#^L( zNpl3H*H&WhJ>w^)$~Uu?BQFi7?tY9zD0JBUUTC)i^iA?jQZ2v9t1@J2`J^qxwwJDL z0|!ROgC{wotSb({lTr@r)NpuXKoK4db3*w&^B;HdFliVeVh#EUj&G>dXJesFJ@HgX zIco~t5SHY}9HkuJb}7pwEpM}gnw{y4(@X2jD+&=Kl{`b!x>YGhld2PI{^cLu zQE;7Tq-}l&?_g-VBrW?6F%{&$h44q*_{AO)y9j<>_UsG>Vi;GH-o|>Xe}Tv{x0s2K zP_sb&6DWScn;Lw!;PVxyz9rU-#9#RJ5$c!>e@9m#*9P56qqdiFM77zV9juj73ACSZ zM77nZ>8G7gX*21FXzH1yWk2|N+~<-)zPEgf3a7U;TexS1)pJeo1u>b+SL``A7>-5V z``z?j_a^Nanw6eF89y6L5x2~oRD9)DN}Q5KvqB|AvXmfwjbIkMxYOQuLe!;j^c{Ls zyv45s8lGVfcKH15_WglwoF7X446HZ@Su`=QTY@5W;iGaOl+$pYRz|!dyLe}fxZUA- zg24OaC2x1bIO~L#!o0X=ND;e(JHl*sMOhv{%QeYqLX?`Jv<-LAwnsU#o4svY7@wEB ztxZ+alnK&3hLQ^F@qxyUF^b^LKG@fOywt{)bIQHIh^Y8gZ=u(B>}$<<0(6*X!tGV% z^^1<^T|Mvgf@GK4lr+Z@31k{;IS%$&)$O+b5lu6!w1wbfhfpA$Hqg8DoEc($-uEQP zLLGxE!xK00w2?mKT61@4q&Sk?9pSrupQc4CSv18DUlIAMjym*Yb$;N2x&WmGRKZ>j zDJ^Y+<)3fNuU$DDXEVmkD`0ETmMNvlfso^_p<_X`He}ofu#yMI@TND1Y!{Fbw!sM< zqsVt%49X)Nd|GVCBTDL?UE-)b`A3UoVq0L{z4qe7WL(H^;VcnHXYKOtg0uhEl4Fkz z{mBO9ghsy+*$HJNReRm5y=*2^ZNVmbKm2 zRaf33=lIr^Lv4??-Zk}bn|ZUZ=%lIp1{kR|v#TMs$LOkfS2lhguq9_Wqq_l;!K(rT zQJ<=ol=@-V{Pwxp+WHm*SM&{d0G}-beN_$z$2lIG>Y4wg9By}DhZ-jdP8$i5T{@X* zlR1UsHI3&LGa}tK1H8Wh?=m+D`1wJ)*eJcS)vL-mQ8cfNejR#n$PlNsSM*D|5I0SN zTd9g|0`zPUWud=m;qqsUGehj;Nw$?+GLn>Z(Ho^tOza+T6SAX$K|DR1lG~?VCJ}XJA%qDPD4zR zJ)Yy$o%=M5k}w&kUyW=2Gx`l7YTHAL1(ec{72SMd>J6jmXJ-SbKid-ij1|<~Ni;q7 zv4=hR;OzdI83fXNlKruwU0_Lp^~AFEL_KHr2gacAbrOK}&OL{({?T zr=9_KZ$$7zNOVb3_1z4ETWE%XD)UHA<7T+jy7xr-KVUGhmYiXpN0pS%Wnp#ggX2A_ z5WR+_N0qUEWw8C%+|Z;iv%^D{kA!x(&l!~*Tjwq%4u*(8MZz^7nA|P-BVkh3vu6v_ zDf#6;{)2}%r^!;iI{r_jlZD#4jO{`22dl*eySi-ONU1YgZ9%)jMBfq0XIvbENb(OW zOtGdjJs`UNudk)QwV5Fr<2X`xwH6*`@@f0}!NpJuGOqhOx4aWGs#+cdYRj_xj8%AL z;!tloE7X}!e9AYR#(t(rBE8?ar zx9eoQ_j0!jh?Af8CAoj>k8>URlJut=Sgm4B2Z0~CFQQnh?demmsV~VxT|(IpY5kHh z?`y0h86CD0FSSidZpLhn?Cqx5;e6c8550Sac16cklWD+AG&YkTnQo=6mtb0z0Czch z$mhNDgIOG2dt?3>IAU<#G7;l5^l{mz7OXE^!#HRv1O>$YI;`J-EU}usXF)NvT$+B4 zLEg?|17AI_J>?qUBoKGmtM*aXtUFcnX6X2W{y&n{{|ZO{w;=f+bx9pX@H3T>A4MFf zp7~bYTtiy3AM#eRFJ&$u-mHH1Zp_4@)WUpqX0=`VB}a_k@Y%!(Gy^C2X_Uxo=*yDaI&C*}&AKSoGd+aWfMDr^odZD5aszkE&5J z8cm&o(w!%JUc+GA$>-5#21C9i;B_^$^*J6F|YD zO?WO8CUvs(#Cwy+V7BpW%pegaIF!i8RcPr%E%|3Rnl}n_5-Ukjq^yBv^O)KrfI>+2 zogl&ADM0%b@OFZZhJ=^+8Qq2glhOHBnAL}4QN&(|LP*8gn& zAL;Nvb+Fr7q(u3SVWWfkA8Ah$jU7AH$LSX&FclUPBZN&}ATJ+IT}oVW-cZF_lZs$& z_IB>?94Wm{;ux)T`ea=QH^mnS{s;Iccu#>;{@ZRv?H9%4W+vklsL;jVqv>?7nKiH6 zSnjc0;O7&!A7vG0Ta-n+wZ`0b zFa{h~6jzVo^eN`ZW77nKj@x(pKnUl!u=~UNQ$s&&$Bjp9k|C-{t5e3B$^70onOU2T z8&4133Kg$P3pLf%bE3zlJyUge^x0eqnR}Y=JCmU2=&#SlUCT-T$&AM>#~~p(;R@X@ zy#xrZ;%UY`ZGI!tz+>b`tFeGwn_4p%5QxRmU%^pB3{UA8neQc|{&bztT-u~)P!0`J z`w9{CL7-y^)rs}OYL<&mw(YL04E@?J+((H$UeGN>Mw(OC()zPU=0O2v;8e?B!tsxe z?bW?Tt!5LB+j8*}T*(A|)lcdh&23j-AM)GYlm->KbsOrW+0PTtQOsK_ubzin;6k+s zGQ^|Yi(tNqCM^P8g@{@92I3%}>#*I|&Oph$hT@pVv{@1phO0V^juc1F2!pdf?9{;y zQo+BHGpwxu{LMQ1|6ji+yzf6*Qg{Su60eqY^FLYldj(Rp z8HZdX3{K8GGV87>dZc-K`ckcLNs0L^bOu}^Wa77CTNFR+n3~#R6ozNXa#0vUo{1GE z6dJ;%kIhbd3j%w{+l`#@SQBEX1JX-^kl{d}HVK4Esu1jzNO@$7fi*@mqB@Kyu1^@l zXNq)r$U`N=7;wfvk(20rp2{}cA#c1qtx~BRfq56CZ;rKdE^^rOODDom z*M$H?h@^i+te1!#3S>#7hUyAYZf8}QUyhsoMO0Q5ec1&KJE|JRWw%O_8qoL&D)d&c z)^S4D`-S4GMHvKCO!sna&K>p_5tCK9Vkofq)U|S7|IPIK-xcHkUeo`(c~I;2%jtiJ z%@!m~PJ{?Qe!P;T)^{SKr``s!V5ZJQ6JY=g$;AO-p+7(>lY@Ti|AAy6089Ce1OO$* z*@x}?S*3AB_5LHN!(yw+gQb=&JTZ>T;b1VI)i$s^SRMs?ldM=Ki_9UL4`(GjvB@Ig zc&?~c-umy(+S*)A6ZP2ZE;6&*^?x=9zPxnZdLPzLazN?;-=_C|W<0(ccYD+s%NL;3 zz1hJOpsM~{+8YrjHhf!O7vUYQB2x-+C}vi3(V*1sl?k^ zxUI!oJjlTQSF*jM77J91#V;E{;02oZ#j<>aV)H-klE0LP(L*`t zz1_j98v1AcqKB|dzVz8^_<^*tbnuULlLCOPOFL{8Q3u5y2~LC~frn0_$E$s=kaue9 zStQRA+g4l_h7hYs;d;;*p}SHv7|5$wK{mlcxl+_0s9R2x!pWp2qiUgel>*%0IUSJN zls{JBEm!EeUvku~UmOQzZIz#*oj0y0a{=vU#$hYGA|ys*HVPp67p_lH$LpDO!1?UD zW0%FGDFk?Nm%5pImjQOk$n%(IS@0XgQHx-mp|njgm+<ihxHZ;)J_1Ebhk!eTJJa z!j5QhS%WsDVtu*aCau-9>;W^royWy9%G@ZA;IX06?nIW*8O&Wp(O|c+*;` zbG8|-{=Y%Gtv>b5Lqr%X$4K#t1ft?J+Q#B7o+Ys)tl-SRJ$>&=f}IU}`CZFcFlm?N zzPhzU3?56sH!+BCTG>_X18|y$2v=d}HIK@VNHSWjBzVn_$Qu2JkE<%$^*qZ}j5wU| z`r@x^#pilg_8KoEB~Twdn*j;ec4B`wqLU;?mb5H4n&#R@-Jp?gWm^{IaJ_}z_C@ZR zHG~Eu&5K2&{}ej2ls1%%VB0!t!K3!i0f(`4VlfJO5d@ZMovIi1Jjzwm$yg)pMgQ?8 zoq^41TIuu-D%;E;cw`rInOeCJ$)DR*T-J}{dJzhdE;rB44xpm|9u5n_Z2h|FgUNVG zoPOWK26`_iBFn);seaIlWU4@1u6M<>!Ig_)!i%ban#{xj_?2WIa?S0kg%(4?}2 z!!=6JLs0gy{Ne+)Lt@GDrCd;QF+^ZJLAH$-2Tr)wSpd9t_OUTC50x8)kq;(jbl+6g zv4kwS+gxD1GVrg$dlGhOv_20lE+vdvP=wHyWT)*thTI=q{j}gTPIWof@)MCoo|<_C zANNLfGqPzk%%}OFNO^CSjCR0>IwY8NC+c{~-Nso_Y{_}(lt~I% zdt~y8&XDe8xqvf9UqTQfu2`qW4 zJL<*8;z1M{T*ph1LL!+?hDmaa5r-^6V+XL4**KZU;#bwS$jB!LrFyzjboMbLF=E6< zn)!Ue1kln##k0zw&po?rMJvk6WiVb}kI7%cjKUa?UFZ@9IdGYCgH=>Mbs65aLJP0# zKEo^|EEjWCxz$B#>-?9@@|k02)rxT-9rh7?^8lG@2-F_$cYrrhAH&ae#43qQy3D>t z&u`n*RflzaHCIf3E%6V$2OH!MpF0ZX`4})C6<~3Y$K`UFihJ8M8<*-yaYE*~bGuSY z`H1V|YQ9wGAFCCUK86h@tnyRw*PsEwA_ zcHl%w4F4Z|%LcWP(z`M;-j0WHK2#-O(ZK;n;X$J0cx&3pUj&JCjo>@I|TmH z9R^?F0n+vi6`Kk2Mkez@GSf+1%zfJ^w?vBe)GwQeJe^+)f&|3nl#D+!aF&mBJpz%~ z`<-U6!r!OfX%S=rE?mmi^NPGa*%fXbIKX_ZaqJ^L+?zk#tR9JUAvjAzwFDlyOpZ3w zvs=C_Xl5(OGve7TUbNHEy)yOo>v+sR%UK?&OP^dY>sQOYsWUJds@&SpWIEBcLfZWg z`yS2Mrm2~*#w;{XlDF4p%gNRm^yTb=)YNmO)I~esHp8a(e%s za-QJP@-|%0 z-@(AYxB|EHx#*=C^FDWGb*=L<*y|BoN#J|&C zD68x;NZPgH!9Ejxkus;Md2+q6lPkQyUrZ0i-P+G}s@ZQqia7fm@B%(e zQ*XRO#b?Ok4{$4(8(%z(9F~nfOargIbNjX17F`0F2eLFqIQKedQ-S;uUF5VGt+!`; z*sFn{&Ijxq^NGstKYGVn6K++tgdloRY{F}wnEu}9hFGD1CxYV^_<=~tyCkfl6#>Sz z`)(?@lW70y+FtB;PO5Ro^!t}6m`@ej%b|IK77q$TJ(7;$=9{$0v%cBzm@B02xrN#F zG|z1$V2}CC(tNWo6o4qNbOTP9L&UQ_2NakXqGyn-7-Fw~M3tio9Krf)Ie!*qa|cwc zhYDU*Kv3B^%^mxDYx39LFj)c^|4A;eCPfvx<~jCJpy^s1r``~uMeE;jCcS0Q=U-E$ z+)k)*2Nd^|Akq}>dS176IS!FlRds{ec?^)=&XA|hsjn%oc>^&kYYI%jDp4U{(I6od zhE6Q7>T=v|2r#e}0;kl&Y1JowZVMp0CH$r9@q&`U@8cBWfCY6-_?^|i0kzG|aZl$B z+dHHBq2rEoBk}X0>*uKl#O!^F*0Dw*O5VA$4tsmJ)WA)6NUPO@byAXh+puiv;<<8b zoa!BFbfJoW+c7Y}DJMAP6rqSyJigH-zByW+^TfQ$VE z@3tLKH#cXlWFWJLas4}Mh=_H+`iW|t&4P7(pOh8cXced(<~6#cfB{+0uvMafR+NH1+Gc)TpeJAoSh}H6mjf$%{da{=$PKaU^69^u3n#H~Th-|9KBTZQWa?|f)*cKMpUOP9qRKXT5p zy1p+AOb_6Dqg-vUo}|1#lm6du-sBKx!QpV;NzQG&_C}S{?db;*ft*(*h;ro zHdq6$eeIb$ZuU7jme3t?vO-Tzi<<)}NWzpPkm9Bee{7jHH>z zlH5}S3j&VrCQi+Dy`egz;A*VH>uGuCR650NZ0*%LBl#X2+Egg`6be;3=?$MTt{vmM zZgKwE_PAdkH?B1t*V0SVa)r6n13`Vi*T33Ewaz}@GnM76mAYt+ay77bQhbQPU|~f+ zgOnbKp=dWr#BJ&!cXekTqGEUD@!PZ{t+Hz~2lY@<>0Shf#h_o@eSD{raNJHJlM)@V zTIH%$@siE2i>HWWx1s9DOPrhVpDN0YUpd|WRB%AgpHA8<3Vnrj(?f|qeUufk_rQN1 z%l<{eeB(cr;MdmE&W*TK@CP!snp-V!ynbP=CAh>I*x}J#;)>k}5Ob5rUUdq96iYiz zq=6>BOqmS_DEE;rQ6ytc#=okQHJ7D=%N8<*^4iR|p0B}wVSa7?ZBQL~Qbj9;QH_Kw zNKjj^-II{oiEQa{=mb@!hQqe$#sXr3yD^)v>(=St|E^0?Os?pRH=#)JJZ->nG6_ET z%SEX~rOZvGd_b=rn?S_?ej{PQV7QWpa9_<;a-@W)rhzoXQwA6ovaM(co zZzLvPV$h{2CL~l)Yf32P*e@u>WXm`t(1fHQ0;q5Taq>(F*U^T=%V{!!`1Mdxt0lfV z=y`OTp%!(8GeAD*x~f*Ck2(H&=+mcfZ*I#@di$C+lUqN@S!`Cf(>3SL>-B%gA$B>w zM~j$5nk8Vu%EGzu@?0X7<@_R?4?xIPJd?-rAK7$yKYT)1W9vDP3&xyN-ly+Q4nRBu zSaXkJXX@YX!OeLw3rbI6K;8`fa^Mx9e8PKY?4Oxwa!l?_(DV%MRM7N{?rbW_J*1K;t`dG@$8S>W@!W0U*rm4(hm`!62Mfv5<<% zV~7Uk3ayo#2}#A!foBzy1)#&ZvSIA2+#yNjk-J$p8ZNk}qDwTt3#(B_t*mZ4>;hY{ zbNxZ2aTzVJq-{IOydI6#0C?Lff2^TE93Gd0DC1l;$K9knX~{B4cxN@~9y+H{EHCgv z(KJf#6f88kr9h>8hy?|^e7Xh2TcF|DDb&bTyq{7pWyhHC%Ux~cprlwM1QX) zb^C>SZ4bKMr;Vtqk<7Y1+>-4%SfM#qm)-`gLxuVJJFUWz9a1%q^5Z0XCcBZ(nDy!m zPhFw-1Y6Llamr0Gu~@W&!FaFLQuIDKAwJuV1YVnyL@^FWy3gc9#-CgLRW+FG!LmN7 zTZ;s0ym>%5Hk1-96sREBJZ3$dz^1hNV`AcbT&#JP$x9Ao1t}D6@*HdBI_C7J6H48$ z+hi^jMUg8UB-YldE_h@Rs;T&ExMM8dk}Cr`t&veuW0O7#j3W`Rq%x6gPoaE$rUjaP zc;4+EuBEwr@{Fi%jG@Crh9MRVn|b1nxk|IjNF-a_Qm5`3w{IfItbgz<>nh#h9%)ky zJh3A3Lc(fN1?yqsEPSHm`O?)D)E1jK>}wWyV6utpGM`DFqioS> zaxy_TDy`;3CB2vZp`6_yOL-tnzd(m?u{`8L-%m;MWEVBY+2H{|xb0Yh+;E0Go?N4B zmN0L^n53P{Bkw@ZTsv7~_bxN;UQ1izE}U&NFZ-e7RVCTh%ka3i*Q7?ckGoXPhou7Kwn7EJt66aKM*e4^&v(u#`!4>+xKy)OL3Z~Cm z!$UEu!*M@O>|O!D8wG#uzwe9;HKtFVsojv~DE9Ms%^u#xGbtcj9K%8(X&tVHN3 zW5B-SZVIk7qNy&f#Herp;dKww*1aH<7cB<-5nQ{`r`eKdt!&Cr++oCjl`fy}wZsE{ zN)~fv&~%G@sae9=syTisJD4GDD_`uV##W|P9MD(GuCp&yM*JrEo&f26o9WQ=d*cm#~Ms1uX(R2d^ z^5=!Gt2OR1wD~eKD`-GEWQsf2-k3-U4evmBO6HY1(uO5p9)dPeOp~bao*lXfW8VoW zP)((kD6jygnf$W+6JQs60pgwpN`z8KaSY$~%B)EBP+0xUf_X}?T|OZ@r&(P{_$MIo zt`F^HRhzR*<%_cnR(rPdZH4VMhPDUCW?hLBq1`^U53dx{UNlgbqngH0IY1o)ok|=E z;dJb)GH+v_`+#q!@!#|_m9-A5y+6<=L=9G_=q3U8wy_jb0)EZw*NyyzfqF{8j5a>b z-&+nrPf*5Ept=NC#FnIh=Dk2wk)e8U#ce_~%0i;e$Wf4bZy~vnJga2WZ$f4vco}Lg zMYK6S$+i)8qI8%oo1uiqXWCik>~B-fnB9^J*lv4>9O7(D%O~#pk!bJur?!loMI7;q zvOhp4aCed~h!Y8>#zA=ShKJXpXN!{^|GM z|Id{W<9ToTQ+MEX^sN+4PEtwvsD<*NCCwUW;KFRE!m6Th=C1_>WI4JF0p$&RT;DN*igmx~} zI$=PIMow$HUDLNkd_UM#42k{Q;UL@VTTRI6SsW4E3>?<{eUNmegw)6 zA?yB{apRratahx&h00Bj*1;P@JjP2+vs)1Z!p0Rt=y&55*YNF7%yUS6Iuk(jNjoV% z_yI3JcCRjzz|YxZAI^2dysd)(x5hDw8o_-w+`yK;m?y(Lg19UCLY+a|riFW`RHC%5h5YbMNjVlb&2xwo=C4b3 zO_Bj_6JS^J-l-3#zAkA(8{cY~12&Ojoky+|f&!^c9!XP6niFH5&R;p=;m%Z93MSdP zC3A1A&QrQj+*@WRp&%Dwnj>l*ISJfQf%b}KU)nA<#gr^&2!&Z z^@Nn?OSw}U1;5SjY?mF&{}D0$FBJU`q+)`B!~TGVhW_y%sCC>gd$#*dZeJr!BbH~x zO8Z8s7_Dz5kcBY+CnBm~n_G_h`;dD9B+b!~kuYs54*@;RpO^==RmqI;{|AwyYYoaG>18B&oplhIiUUzAS3L`^+Akf?~ z^FbC!6!=r^!v*)7K*Ng3u`Y-_q>to4_IIH@$hW9x$hm1#>!zzJW7a38>(T&KO*2eC zg_i5-rrkTuLvhkUs*9A-eLEF$-?ATkPqH>{FA8q};B`>BSOxeWeyY1ipdL(lyV1E9 zS`a}YfLp^dVU@ui2UHRu-@4#38(}=3I@0s80@w>OKF-I;=!^L!RI~9F<#YuoV+gHf z6`BJ^f@Pi*cRa{4q! zrClMt{mDXxX^*<<-fv5S@)1ekKZ%j%vZeJNr(HDzl`||dMm~@-^)y_6X_J>icKx!2mB4~?yzZqHvt@{R{0h;m zrmq&9jvDMR3SZw_lI)*0j?lJrPA#AR1sFqxavYhb1bRIN7qwo&_y^DIFsKVKv+~=f zr(!zqMzZdbWQtWzu$s9#`8?4_Y`3hhI;L&5`Zl8zy!^YgS-&rx->%qA1XU3h2^c2T z&0}{@$-OnWm5jz=tsq!An$v9gR{fxI`r}MN)xdd; z5@Pq&jhV0O;XC)h1N-;zhn!dIfB-`ETYE`2e|))`+1v3DD-X}3-MfyMROT9n8gkda zaLAzUdli*mh}X52&V`{X%{l?ux5ikJC_{}tH11CZyly|9veAsqmpgUO+?UQ?gwgXb zUB^#H!|B^FH-pUUt{J-A0rQ5wPf-%QI(?1}L4Ak~7Ujq1LMz4yVf@L#pHWJU?IxnDl&sTqDsoG+VEZ-qN@miz=*y!^9#IWXU(C2tl;Z{9v4zK58 z*MdO(85kqv?ahux)b47sJp@sT2r6<-*UMgwD{LYZ#f#l|6;!C@0Kl$>Q(T+Ld&EKn z`nE=(!oJxdJ+Zhm@|^a@tuaPYce@pLXIgp(Jc~OBiWLWb6nDd-If_~2Ls!Eehu`uV zn)D%3i`WmcpbTY)7L)ur^g!=)prnI`hIN_J(W<<{5#cFz{$>IWk$obOQOMG(pJ>tq*lk0n1o44?2IL@%~8mjZJ_1 znwXUGN)zsd`+v20ItE;-RgFSRiAynrqG5}3eo5jDaLpJ73aY#x zld2zOk`Iwxd8erFQ1gK_g!#uS;f%Hd*ES{9NiEz@>q>5AHnG3QBRwK0E`FvDr(HIo zjYp1MyE-V8m^gOMw^1&%Z8dT8Dvc)8&FbS!xOs-6H?S$oj-9MpAr) z#zR+yStq)6aUj2VBRLymzeoCyxcT4K^8e4dys8TLSbVcFWodI?xNNOEb&wID|3l-9 zec@LA-%_cQ!~g>v_CJj$b1y<6`2UnOb0i+%VE?^b9YnbPzsk*}R!C$}l!EU**=3O) z#|Z!9M>zI(PDGk;B0FIkVgMsNpjZ>;o6f=c)O$+bnkFL)K%fhx+k+)$hQ=lrwt!|n{h*V z{$`qREf}Jn{BYX)>uvVt<4e%~{_-vi5hR^-LT5F$mZnXve?_OxAPlS((CGkmog9@7 zd+oGq$K5>|g72e?*oprV3ieFXHXEYUwgUl07(d8)bW0Dh<96T9i{)Ln65y-6FWmkT zi-wh_e}Czow-5q?@370#{t+J-1pEC12!|u*Ivg;;ck7RmB!4hQv{AZkgDYMoj!v&+!9_@SjC?GP9?Y+V3x$Kd@_eBuxODZ3lV(Il) z!s<;9D|OZEj^0c03p&nCau(%t5_n98z>)v7mr1#4yH)9`+%*R^Z~BLa$=Slu%J@WI z-p%!AetUCybHCDFe|dG4L=j!H)bAZGVb&zWKQ_(h zl7Y*DM8?8O24&_m0%AsDUJ?~Gv8brBGnmBH5=AJib(?~C3qkY4xBo0OYsWQPRFc~( z`OohiY15IdNEJ9=#W%bsRNQAG(6{yIU2tS%q>;H=*SBp?eT1W2W2DPN%^MCqda!i~ z$ElU@KZ-+l5SG@g&!@orMQMOqk*1e0)!o-}&g22lb}7w%1aSW2fcR@iik?!(lgM+x zSFWO|oL5f+ZexZITGNCGJ3~tl+h^Hfs%t#3!X@(WD^6n$O(meblscY`jOp`m1fn98 zPq)OHV5)k7|3}CEY3v!E$Q+^kC`F87d+uP@dMu)}%m@mvnkX+~h674bq2Nx-Ru z7II|jE-Dov13-X`UgZ5RYiH104MV{KdQ8~i9&^Ir)H4M=1zO(E!gtX8Hh3)_`V4nlxW?Y@t9`M#(dmj#6bV zb%Neiy6^td?VqV-F58mOPj?Oz0@D%6xG$sZY`^)ZGgZ7WG*GbYh4iy%fA%G+?-1!R zVcJA(6M&w}Ypy0qCzkp`vjttlvQZM*yKg@~v7`1b%EM7zLS)nt@o0@;9_lJ4_?W z#besoZ)u74GtkJ*2zR zq3mtUI!kGss|ZtxZ{t)a$7@1Vq|-@DT$fsj=N$$C%iw(?X-X!B@Ht&%`yG-?3q4f`Mq1`Cx?ECe zN+Ql8)AJ*hnVmJUV=7n9gj+mU6bI^a5i1>gX2aae(TTS#!jC6xo(q}RGVNi#lA29z z={lI_x}_DIA}nF8M=~v{tn0U{C3#jW0BZ%JcTKf6m}&5W*o``?@;t7haW?UsQE0xv zjA`A@WqLVi)vQ#p2JW$IV=*+cP@PP*W?rBK2X|)=waK!bUs#XC1q_~R`SW#fa>1qj zb|jMP(Ln+#3q9tYIN2(^5;0GgbOVAWXCi}Q>hPdYY7om-%j-E%d!6NN+kp`kpykSJ zX#?8d?B(Dqx@d6<Nb;@2ACe?6P})ad6KM+n2xR#j{Y*uyXv(6;FlT(RGpj;V4D7esm+h?BI+WKqpCAD-MB!AhNAamfRBx2ljbfd}d!Bt?Q}8 zQgD1#EuYkSraEnQZxW(y>Ja6%P+iOyB}$7m4R2yChAK`m0aaoGyJB++kehi_*?y6r zn>J$hZs<~x?Ch9+=NAWz|10La5yf~zrMD0>p6bvsI^Yyk3}WlB&2W5H?_2b&N z@r%@l?hC!sgcSrAm{3Dx*yo9_cXQIst)Dr`tW}>GorHtTm2W##3&y9{*@C1s_qYF1 zfTRYMu;7NA+VW9{xdv$fxCv7}Jp6;S;&-U0x{Tz?J0eq4z$djD!snb|e|=p-0>{7+ zv&OybpyO}JMRcSY{#qtZ07!K%O|Ov~#BMTz&~ViuH?u7X;(mzhFW*I#*Dz9>UpRYI z{hVwS2vo971wm&+%2UilFOllRk^Hf7XiH1o70fQ;W9Twn#5fCzG}$>16l7%tY`0XrHcT z9S1}1fW^W`dz{!9-7NYbp7qxqK18KyRu>&aTz)jEt*hei*t1VERx+XdH!v<^lPcoV z5~H3s*?M-V`qwE`K)YZC9a_a)=X5YODyJ-937nQk?mZ0ME=gmlXn-^S7D`iBoz3tE z+HZ4=Y-%DFyno2M#>zxKjpvg{^i8T-Tgwr=j&|)EJf-Vy$a*CUCZDNIDOaQDakhdb z*P~4!vb8PVm*6koXYk&YYL~b^BSX9?!V5U4x0dT$17tn005k`7Gh^DMH|=YEN&8xg z5zb8JS~F5e~d!%{V6EywUNtL)jcr&<`KYSYw)cETikULp(%d4HByh^mz%C zyPk#u92OlM_@H!5u5jh!si}X5jRBr2%j&lLBotm%5GwzeD<_Uyjy7ZY;+FPX{uXwv zpVvPXlH74s0+60`zf+0C$V1FE(1IJuJ>{GCl@Qgq>D1&~7p_(#P~-O9X3E_>b&}WQ zP2J)yejGO_x9~(P;}O1rOc}>*P%qg&tBvcZl($Y9y!zE}Emk{v z?N;3j=+W3kA zU0Q5YJgR86N;uBe-Br(uQm{TLcXgZ>zVa0PLOd8sqi@ZOa2eAvrj5{e$ZUoKI7*Xr zhD^h~4Hz)#z<^*AK-|zOYh`wHGhnbYAKvVzaKZ9E-nxu=Ry@|DSn+*9zc%mS_Xl$K zhHM?mUy@1+5+(!ehDANnGjj(G@l79Z_L;lFMql|~gX<2Y5~4^8uod8ta!?v?tI}gm z#1Rl9w7aM<56I^tG={wJi=S&F?Ybyx$ZT2l0GPjj^@cuPNc9Fh{=V(VI7G^e|3oJq zxV|n30Tr8@M(Wl&zI4L8SEBHY?v(%60O~j{8HcNN%1;+NNLPTcI`%xKlMpADwT6Ar zc{+?imkH&FCJ;cI{|9TyM$av}FfO-f8s9^lm&4htAHF_N{EYIRlkD~0G@|ZTZ9yD_ z5dd$}iB{$;(}Y%5vyG$rs-vmpqMbJC&af?Ru+jh+i1+d(NTuXZNQG}w>Pr10>m{m; zv01d+7)W4^dckpnJwo%c$$S`opgH6F@HNeSV|@QJs4*tq>>009wf_VPCct-yaNG=S z2CoRf9U&+65@<4`P!sm)NZOw>$K!OEYf6)Nlilt!(PCxXTegvO-O2C}bISp-J$~BF z;gc~Af!o1eokz0v3KxZm)Ypn+j>QdGXF{~;J%8gL+~7>*#z3vl9P{{r_1{?Y|Be{{ z>ooM=@X>l1$-V>%p8sFjKZPI&{y!z6A$lkn;+*KFJh=pu8}Qpq#)Ua;=ZJ(C;Gw$m z9pP@_Ie%(thzzYo2lgG|j$Myq+_x%%D)Eynzyv11_8t4%Z=(>$^B6%w4|PP^>1iL+;^G)YX z&R352b-TIe?Y}pD9ykMb{-dF3z>5~)Vs6GWmJ#D;IkNZd?EG7MJOJgsR^(G|tQTuv z@HTh;Ed;H1sJ7@?Kjg*!J?_zu{*Y6|@51E1=5ZVCo)_VUml^NGGrf;gY~J}7qvS6; z5Q*$$$yZ!UTnw(g5)mACo;g|Z`GpX4IO@5wCjW`7Z_8a+6q2+gGi6U=K$2Dn8;~Y= zDq&RmOqZmn!v7fdf{*wC?lxEg4)(T5!j5uzVSq$iEGnujGZk8^Vy=OtB43-Bkk}PB z2W936SC>Y)=1-Zx$>7QCgBz}8u|y@C_SF!fHfds&Yk@dNM-0P)MK+qPOS{Ac+$k(2 zkA(z?EmdAj!>gP~4SItXps+Id!zu<>%w905pI!vl^i6Kw8nZ!dj9sJ!#uBALcMOrZ zTEq?35^bKLNOX*W@~TAs%oV0x{a0%|Va)dxLYY6mK|xp-PdNt+S*M^(2syyit(myQ z(l^fGmZ4-PaciCtvH7O;zasg(a z{=Cb%jr>z;c(cQUEZ&Dj!+w2z5JMTp&+Rce~iaL{D%aL33c~> z-pw&+)(<{^1Ks;qBBVKOfLO|D03SQ6pJ?dkgUhOhgJS&?`(7*bh}Y(RL!NcW**2No zQW1BamD%hwkMWP`D*%S|w>=9Y-d{gbhq@kegAAzYcrROgrP?UaW8sjz934){sXq>B z_J80dsnO<({8>m}T{12cTtWdQFuFW76-jV7~$)jfU-yqAB0S5O0yq;@A43&!S$CXf4d zmcdybJmOrwjiB~rf9!?G$H3OR6zr9o)bLp%m65hLmbH-0f5wGPoFdPTgw}+~<%mW2H)hxMVWJ!a; zL?NwqUI1zx7s}~Vh_8a!g)u^cu`4SO4Jdw#(aCa+OJU2)>L{$`U(0y;fsQ zMLTxRmDYo*R0~@D$d0TO@M-B%#`-C-FysZ{1Wf%8-eHvB=|ic&30uZ8zCI0fE?pPR zg)95`5^>pS*hw6UO^kPG(`?z>44&YRbrL~6wO#axVMsV<3o14+RD;IIy z91^F}W%tCi$VjadnN`P!!DsHKL#~d{Z%V*hs1P!iI=kXCF=||onf(No zo1I|x;DARU^3A0K1?jX+e@uz5=)e$v;jS$Hz~dLGXepS7&Zx164r#5t?@MSb|k3+3Lx$%X^2C4N0AMK)6|bz(D-GZ^yA2 zJh}Jm9C2e0O%M(5q_pTWWQ{0=Z|-dc*8vqoykL7*vsJl8E&8KVS&Oc00eP!R3Gl0w zNsCAPlqn1{+ip*#fK(*(tWLgF`i6r~sI7z<`|cScRm;xX`dSraaEk~Qu|&^f)iziJ zGPgg-DKu`Ecwjy1u25%BzAY6p0R4-^yF<5-*X1Abos?ebLiSKcSCpowsY*BW8$tGU zzPIbYviP@CKLUjVoPk6((2Ng>A%K0%8+0~d9u#-4{6Az;5x(SQSexYNn&G6(W_rSY zTy7+05lA3VUh>Q^J9dt=;j+-u6YblY2Yrugrr9G=%`wnE1BSv?{5`3Fm+vR!u7E+u zplHd;%4}@EEyUIYYdgX-x<*}K9p9Zba0B|Wy|Y7m&<3K!o5@HVUJw}U2^hMF3--N- zu`mv;HN)CpzT@}NALV@sh;6-f`xlf&S5&sgjLC-AO7*!Pwa+Lq1iyOy39g?i}Vp zvbNe@cd)fy`P`b$zN+4ue#6VmY};t>t!yo9XFWprl_r78h8KPCI^0IQtZ^2vYUBng&A!jar-|d45Z`tM0UzFBNGf zXw{ZocMY3+LVf>!mU;}DwaeG#K`M;Z6kX4Mi8)+aOLa*tRAg(e8kEhB;;QJ&;{0qa zKDgzXscN7fc&TW10l3s|?bLUAT0dNGP0Iy~u>aX!UdId4#RE3looBfoIBWjiI^c9s zo}UUmjM86TV{fh=PfLgR0~zfA^c+(Dy_d+unMXKbt(;>Tz=>UFl$&N=Tn7y@#{7>< zc&RE6c3g*c@luAFc{qIxudZVl9vXw4S~gGIC0l=CzMw@%0KoPchXAu3RrhRUwSGpZ zk&w`~I3*>G`^BX-3`%k8MWRS+JYA_YzhE&Ew}JI;lr zvKM}i^l-Br3^>_%8C{AvcMd4fWBC%>M;V#T2Cw2NlH6)VZUt~$Di_SgMwmx2@Ymz*_v(Oe0NWx@#^|P1 zI0Kz#0pll%;bcQD*HcR&Zim;jZ*(n z6CO0IL$0Mz&C!u7$p?xFnDtP{h}B=*D*NEz;Kb^OD>&c8-Qsv>$fxSOMDd$5b++1} zV<=5!Xwp$QlNsjkzlaROKdec?jgOt|!};cT16oVA6-?*-1P0!|hmns>O#S1yc6XMZ zT>tLe$PQ;4Tl8UaN;EgO2s&V8D*fJ3KPOk3eQlEc<X_Cysi%aSjQVr7<8ahS==iWZW6(OgiQH%dmOES?S0Y{G%N=W))+K8mV$ zg=!}m2&dW4o4g-_LO6b9bz96eTrE+yfTwg8cr2&{<0&cSk@ZTOdU3b)Q<&bh&PDJ-84(ppC~b0ncbnas|NPpc2aJ$*xo0N zvm(a1zVpr;H{$5I{SA5=(^OXa{19=J=C0_jv;u?9?qwMHFEm%5R%Wl@v@Gs_0n~I~ z=&%ZV^Jrs&AsCcsdW)4bXb zQz%}hH}2Hl%Ntabx*C~EU6fui5dfjsN8HZ`*5#KJfQoqw!g~up++wMhG&B!nV<`hQ zQq3fA^+|&qHiJY_=+(JJ18vk*fHq_&9m1*--sN6fwHzqzFVN1&{u+6(r#f4UxS~`W zThvuVNNYK;r*QhU!g6w2S&lPS|(r2UNgCGYn<FH>x+K&K8x{SLc^gsh#Oa;ZkQECL&G5! zvI7a1-=D<`+SLV}_RJ7am4C1*8!ZMV zulJedt~&C%*?LEDb6)&1bF`&DwDUn87a^D)vbmElGU(3c^USYm`BJ&RQ}?T$>tluQ zV{I{&oAm5m|M20R5H zIsuH>6?~fAxkh5_Ng0o~XY=Sio0Xarnx%RcvP`G<+(6q{?Dl7id)jNrOaUP&A4_Z# zeltDofqy=o%&J95$kmmW5$YSK>5nWvJHa0hL0^iYQpSSIAq-@J`V)oJWg zEAsKYjVKG4FvShWoHIUjFoy#O&kiN_)Uw~H{;5hv4X9=ZQvFjiw?&6iD`koVS#p%v zQz77w1cJ8?jnGHz$Mf0Oi?2E1eo{d$ZQ>|_s{%$b#1&na#h4dWI2~(7>(X6ctkM72 zD@otn+j`U#tG4c|JB#6d886s@XRDRs383sx1px}kHJfa?~7=u_>+3z zTpLZq2`~x!`G1(8|D|*O-vjRXZ=$DFtcg7p4CTMFy^Lb%;`?fr=(qm}UmD6B0uDgY z$<)l$$;s42*u=!i)Y)0p)4^2K-qyj;$~{%;|=7z!C1vh2E$^xL1(@yk_No;8?8;4i;~S(@^rK&^|%iGRtn~UQ;uA!Zi6ubcWmj}VD$uPJ=R0KX;H{hQ<3qy zO|^tgpj_xY;cb3NQc4Yq4v|x(vX5gM+pjjR;&YsmYit@Y)<%&QwanV*YJd-Ab;=%) zsfi^P2~%Z_C5gI~kJ-(7>Le@jN$)zAOp`gy)?AUlRpa{d3yhB#b@W-^+WK%i=-c#j!EZ7i&7D z_@QD6;7TPiT>T`n46WX20z}P$pA^D;a%ApjU1+TjK1e*Yd2>I>WEz$V?wENvd6rn) zl#$dl+MZ_7Ua4|gz{oTuS82ycOPE2ZD)7fWXIllO)*%luDrt;0z2|PA$$&Z;8)ko0=9hd{Z`gF*2Sc<= z>{U=FE{RH3{s-|NwfO(;mH(v{|6>jrsjNjH*na#NE=x<;gk%Bu!WgIyzrJv?GI|2H z(vUGZ#+`5yNZ0VkFvxX6k$)ni%7S7mI!bZWHLM}5A%s0IsaH!js}EQHEmCj7o(@nQ zwyJ-ssph}Z^895-g;82f};#tW_a7#dFJSds(Fc`mQWms^UsN|591`sU$RQ|4%B6z%; z>w5R`?g(|`C?WtD=(kgOO-jCUy%k7#=6(Gkn0H^Q>h}i*?e(NAthDik^<8U%z^MWC zGiyTdCRpX=VQ~wN<|`1bCW?*@F3`GUa@kEINwH zeRN74+LE>*RpuD!lFEWH7|w8`iqlCEco`~cHSr8I0fq^fb{qyS{j77N$n-Px#vvQE zDaQdq=CpMUMJL?IFfLt;2OWpZGMC6#qW~IAo*@!SUM;$k?b)APYKTwn9Sr$J+88pr zbO6c4KhrGIsVWVDiP})J4iPCv804;6Q>5xD$_lf%s`Vq(Gw3n$b7Yxn^&DWQM7$3y zlD>Y1+ zO%Ba^_ASO|7aT=6eezTh+3xOe#p*e-Fu=-DpxzlThDofIfGxdFMNEt`zcqWJgDOH9 z)wv&ZPEJT9`BHIM9_wA!gL*B*ipovp0Se<G`c28-jtfXc8csA=C61uq7e`edF~kU-S<=D z6G9?V9Tt^OrF~Mqx5S+vl)uBd3#WN1-&ZP_G*-Q8)Ov3<0$^?n$eZ%Jyn$~CNVV%# z%%7O3i1pGyHVlE>p>N@<4xgyr0h*HJii!CTOcu`mRV868{Eer{G0s~G9So>Bj~B#6 z5L3JJgoIGDwbcdbvaN)HW&^1SZ;F(&Aj-I63zA^SY0MfMzK>`eTib&N~1;YE_wN)@oNJin*L z6zyq2v9-pG+N=T(H_gt<4xptHR3|+jwA9tf>VI3Z+V^{5_0Oi_mSU@=*3$WUhc+nB z0%#LPekfQz&fKpH4sIS2{huu*w5U-{5+alD1pR;P&>k~KGELcIVo>FN`$VazH3|nw z=rdoQmaZ<=ffQ-Tz;SL^ar~rYsMb<%ZmHqcqh^40)D#aw7n2GPb_4v>$Neh{BqkU_ zLrv)7Dl5sX7&*rPrgBu(R8&i18*{8j8h3ep%!is)4O`(HGE`gDXm4vado3vA&&_8_ zK$arVVtu9qWYT&E3#4Vh9@Ei{kTvDHL2TqR`(`c?U9M_iMv2=qD_7t_Gnnz?kC$f- zpVmTe=`Uhk-SdyLvjBOU)d>#P?iw0D@qzgo#IAO*b}(;$|G`OC^#0VEsr-BKE)E^l zYG=yC;7R~#Jop(aq>=}>_M71B^wIs*IDugSe@q@~@&t|eijWT5ia)ccV%Qd%n8l12 z1@cOfu8y{4o#;vCw}u_5ap-bxh7M7`g+#Lz;6j~D!SMX40TR%HeRC6Q8{-0D{Hf_6 z`GW}CcnM@m9wd*(l{EOwhP(jhX9;3Zkd`e~egsn)0tf&KY=;T|Nz!=wcfIy+-J&Ih zsr7ibBZ%^P+#!&R!a`!L!&EveL^p7ft@@>YMJjCZEuDyT_ZN2*-oSvuy5Bs5Iam<>rFjg%ooe6S8dlGrzl}7T`MjpH zl3fw)$sSQ@IKwQirQ$(I650fj{ejZ34F5s8}a2FR5<51;*#X2QDCCkd8fS#LW$29c)aC))6Pm z>r!vp) z-AnuHS;9~vem74Emo4pY!0ODe6^=$UryqW#E@JXWzo%^F+B!-7k1>_VNYSG?iP;h2 zGdKgz2P@Z>a8KE+P7uVQhWmspLw%v(E(iO?VF&?zJ>v7}Wm;}3Q#&2D&AlpaRj^r7 zf(7@Zir_7IgLUVVuvKB;bTln4s#wt@V?>3+)<|{<0UX2iL31H=1Gumf>xN0#_bAf3 zRDW=R^*sI(*@4Pf8AiIwD3PR8K?loy-(b%0;dM0}-ra~1&WDY#R7x+HnI2{hVmtw< zc>WWmr^j5X7_Pe+CkM!9!4U-0EH-}|R{2860K3>-HYi`ilW6ig z{Ek{B-H52{lrzsG>T$SYbdhGfDC&@M93p+0af`q1lJym%ETCEBl5UBmIkk}?73qM| zNG#P)f)s!PM&3P9<^DHJN}bSG5-*{VDBrK6(KF$D3t}R^@zQ$`90$~r0Yuj?3WrAp zEE%Dp+&CcD1F_kLLi55JYf0WRwC)^e9m?2vvX%ZuM#iOab5%C%nkqB&Bra(t%T^8H zY%=+7(97%{Myq@OLdQ9zP=I3nEd^L#57qtE6!zOrP*Y$5j*fZ&e?BMr*#siQ_F$ zBkf58^*Ki>-8Ez}2|tV@Scx=5hQjusM>az>XB(znxJ-$4bc}l$LUoORC*+ zpwUDcVeC)!pJWL&tH0wZls*+*PS2xY68b@b`w*6n3NvN3Qb>QBh-0~i48N~;6-$jQ z>gG|ogBPBmtmDFHVL>m+6F>vjvl4hkzQ;6WHH|dDb9^3ycxSSh;;=uBp7JnQh33TW zE80}n!RW@h)ms0tQ<=XL;#1%0RG&F2-SCkSl%mk2AM9n=^C}(-# z*wu5$&GW{E&i_H&5OD(s+@56aY9(=yS$~vreqM=4fNf6U z2nc3odep|u?AX|-Um@=_4j|vUdXzS^7V)nfI>d9<*Dk|F`6$c(%~s&_yqkvY+;x-u z8i#?(j>$aGEiUesY5)+;bfy9ycKBqW-)= zm@0PXwX@(H761}Z=K2jpwic*TUxSEW1555#C754~LVwB;v>Uf0nH%C_8x1FaM02p4 z*>d${`J6$Ov(C#Ok3LT?y>yCnT5qg~6*xtKD6bO1R!7t>Z9e5vO*{R_lu~d1u@As< z-fTMlyLwc*<84$FB+X;CZd)wBsSJ1uO71Y$3!((7B?B1LRO^%apTBrtoOP11txZq! zh;c?Z#?}7e8j};jJYqm1eYqq5I&U4d>noj?A4_J$4DBf&kKdTadCVPiEg+jA_YGlp z(yY4SOc#6y^4;?R-W0T+J@qBG53& zhR?&Q+5uyy?#lbJOpcs`sResExns>YS@|*|P3#_;Q$2q_4W0y8&j-+0Dh_n?%?r0H zx4*1yZ`mnBNhk*d8mDiN7`|HRkG5_yKy$E(+fKkV=DuM5)PWu%NB}$E_o#gPmj9Ug z9BM5ri37u7tB5;I9fJkNhs;FJt*-K~Pt^~s6#>FTy26QOXqPTQ<$7t*k^o7`+1QpYCdWh`o@I*WWTkES#0*2$Yj6M?Nd%LB2CZ0<=}D>&To(ub7ThjnETaP5EpT z1nnqLY+*m{bXlQfj+Mtn&rpwo^xRK6&6HSxiQLIk1o%=AF(zh-2GKr33iy4N;08yI zW0Dj0y9eMc;oLm*S6KZeh){Q)I%hH%=V|4OIwf&)!bSb}~Jpm<%e4*Qie z->F!bCv9TLd2?V#%VGO+vl}uALX?AEeO@ctc*xDIp4sG#oNEP*Fc@K3+fV?D@BaF< zkEGpN#Z;SAvZ;@%Thk5ju;01fA*k`yM%Xt@vfnklV2;R^5L76<~;sA`w zXdrh?fJZPy`>bVpw44E0BP1cV-&lrS9ovTSG0Yt+Q~v>LL?MW(O1H7Fpy>}$jGOz< z=~Fj(Ei(M|>PQW?JAymiXt9E=J*Uf3{0L=2O5Nsnv7X;wH2P>eXglUiR`Fo>Xl`=v zLhs00dATb24~(35N4Ye5OG9x(aR6p`{0T+K(v1hx^jhfXg<7^JsXH{;RROhkRWv~~ z!BXI8*$WoDw2D>ev?J5vZ5pn1exYbdX|b}3nX=&O$Sg2@eq)MB-J zBb;iB4q7kP&ipFNRQ*+Q)z2I^r3N^hNd~g(PvLhZ4E;YtxvQ19QOb_diP9YU#=O*8 zeVA4@wB%8zZp=qBc{&(JsQ|^AOnMWhHI@j35YXT6`4u(vTLdgdOw&Ss1>!pM1n^Ql z3Oab(n6a1eSRCs<8Y*QGf>FJS_Zo8Va_`xv4@fbtjf2UTLRbz{Ugh=9tL-gL^u>e6 z4pegQD(_KVexk+~t6q?3{~uRh8B|HIbjje(;O_43?(Xic0}KqV7iVyH7#s$7m%-ij z;_erB|9EdB_S@KqjPA^+uKw56S(SOtnP}$P0@#5#ALtc3X*cJ9!Mj`KJOvRgg8=?= zo^ske7R8-N8CueammhF~(U+o}lB#UqaVgmy$XTMwD>8Rg>Ny6>S~;4-a>{5k6RY5h z%xe;Zza5eQqLtqmCmL|n&9!{h(5AEEmzKCn5m(=}H12froTN;(O%U+rMYQvGmcEF} z7L!%ws)sUH^fYm+XdNx1Q+TgMOaQu0*f|CB?ft-Sx|r~idXz3s_U4*{g|uFB-6a?_ zE4K5NTut|OV?36fB?>0|9F}Ae3#|*YJe=tg;CFF;8rQqMbSWLUBkEdPGZ0e`00ma`eEM*AKE7n98fOY7j>Glz_<0gIkO%riWD#fU>dUxq?jO1ZM3&63Oto&!%VN~LV|AHI!h zCA0^BGsh4wmXg(3P5)W_lSRZj)X2!yupLvdOf8>SO6G-^$~x$WGO=2pKi(<}Hw0=W;RtxO$5Mt(+4AliBsaF@VJL6b=0jPmqn&=SO(UsU>YwaOOD`L zvO>wI&qZKz3guvvDyFdS?kD#gFsZ>)aTLqxnqjK>=v-eHaOwO+2yRpitE}+9kTT^# zqrWTU0wFePVkK(6B~W2qqbZh2HMYh5l25_Y)GeVMPr=MotEgN6Y6G5i@p4C51q{M= z@_!}PM<%wLzE`xhAs)o#Rcob^z}V0nG=8I-k!p_xLauq3%TO1B`U()3nM%w6`PWUF zmOG)?KjgaRL|AQM$*+n^&R?v`_q94Bn3eTgj^g!dW<3Restjs#94O2sbF58CUT-hk z$ra8O)yc0sUq5NtNdh$AW*GX>QcFW@VSVfOIV;BW6m^HE)l%*=f2Gd#1*H7&MHg1y zV_a936;bD^)u+MMNQoK2PwZakEy&SS#UdAQ#j^eZ@^_UH=;PPCE-1+sKNR9z(vs7e zcwTSB8%R@99Pe$RZ!3SwEXkI?Vq9w#I01&kO4Ow9n zx5D>^ETThIJQ=uBSeY`ibP9XVTMi5E*xe_j4f(^r9-z)-#ggs@lBd<8DrUbOVqx|2 z-Nz1l`OCrC7zF4oxU(w}N(QpqWHN@go7%AVkwFIAjvB5FRxi=UoI4YWcUC>)kzFM0 z#?EJ3bR-#LnHmVta~B_k1D!OCkRvaNz=b+iBh zOn-h!SFIAAl&48^DG-{n>%7f$GYn`5=b&J2xuYAH;nOD<-M+)C2^LD0U&W3zgE6{O zCE1(*N?@*-2&o%H~l>)4;T zb1=LwW*w^UW}o`Zv?oUmOo(5qO-9p+1pCL$S! zpG-V!@xsp!vBSmp1!H?p8v51y7V^UNRiD^GjRG7?Mp*T)r=QgIo8}3Rh6{7Gb$RuG*IhmB6RFoh-V%cADni;PC^pNs*h?tvm z2W`hXFUokR>+cOGAd$;i1vj2pcdBG4GHXmJx==LJl!$vt`y$cq{LRxi%0EDvT@`In z%w0Lv#8tq7{?5CsF@4NK^?YjTX>hlUeg|Od_?rrhS9>f+GI^dAnQ$x{|MSzGGu$i1 z%4{!D`b#Qs_I0xhwz~zU@~ehel?T~3@Hu2!5O|j28}@br_xl^ZJ47YYkO+7QE9uqQo~USo=jn!=1`3R`Gtb zFUtS3U9{UfDuZ}zQoas?o)BGyuTTFqnw+&@?UCp zA|FmY5q=u6Qj>?pW!>33+zPIn{hIpzQNju6TS0$c(4jr|&k>CI8zcemJ{v(L8d<*m z;=dz~bQbDic5zZPqGNmas$+{LjU|Qb&H8B`x3^31`a;E|pg&&%sYF%Nl2m{*W=$Ec z{k|JEd(|~w8BW~G!<8~J*^V&0T8ox4h`l0-(ptod*iU>$eA&A^l*YnxcAP3s<>Kl1 z(kN=3h<#blo9NLSU8mcQYvi)$9r&;x&~b{|_rWJNghP>b<{muuOvs%NLKl#7(rK~t zfhb2=_m{>n2iCS-{*F@WZVmv4)3PQZ8N+PCV}qHBfo3%hs~lB_dykXLX+4PB3fbOz z@FJu3dwRLi+7ROKe9Y*DsVy|a0wawxnB#&DLu|Naq;~^mq>q(uZr{twBd1g+yV;Rf z!JXU^L;* zjVA~$dq8_G7@VvqFJ!>ujvo}iVruTx;ph*_Ean;npqQ7T#X2+B#Wc%BMf5A>kY7_L za~_h!O?xzB+5x>zRcEzD#eP$4RqC?$Qoypos!PBV2bxeN+-25^-b9D3bf~UBR%%Nv zQoS!ZzrNA5>CxC3mHVI0mldl9fBMaqx&VfsY407j%1ND$DV~6KTc0n7{ZDxGD>7E< zGVEFM^7$_U$5krw8r4FhiB|Up!kc#Wxn4zn+kH3g4iO_}jtgcp?5rolM12wF@2tis zO5TPc%GF>dx+GV^0S6GT;UuNiMJQ5LOq^rVX=67>Bp%bV!gQ|> zMDK@2I8Eg+Y=Ct~dsb-Db8#$)i6Ac)JS*|gPCAgBZi(p&t;)91h!pEc_^T3MkFMrO6|1(qQ#qPjA?8Rna^0SL+9CcR|bme+7^Kp4Goc1M6E zLs*ZIP`%xKIg2jkjFV3y6=5@nPVgOfTaR`HL@e~q8wAp z`H-TQ}~-@z1uOD9Y%#c*FCJ1{S$`Yqh?J2r5(i6)#YGbS%gJjfXEQp2Y2UOaY8 zE{4#c0P;j!tv)EfZj%rr=)Fz@S)YhSB5H~nK_m+H*L3eJqge@QCvKkg0!ne?gk_N71QvU}!|;>^e%n+kAS3tOx2{DrcVQo}j*|6Y8WV#<8Oe}| zoM2wbLm2{gi&zYD^+Op-0ygQfcuJQdCZ!A?0PMH{n>1a)C;_c(^F%`C{s;l>7)=Jb z=4cA9TIwvD^yxNNM9E$ocEp|Svr8RcFOgMl?+>cPb5yz_#RTZiT(a8siPb;~!@5T1$ce%K8_BktCAkh#od6;g|G(#C z+jlAEFxTQQW=V@6R>dNVW!fje{<0uffF4us`YpCZd5VjPOYZtKCX{|$-}m1c(cB#H zt`9LA2r0+QVa81}eRoF^EEn8HaMT&%-mj;?FfV#w|Jyx@!B5+=#;nYC2HtPzdiV_4GE4sj0+HCgPcYqyQw}i<%Zm| z5(E0WzVpK7z?%73{^6cphB5T}Zg@FKe;$lu+&uNT-+^$hY6U<3_zf} z?ec=z>BGw9{zuGo8XUE3?g++o5-Q*2<>#bp8Q8K7W-ygEcXm${hF0^)+iHel;nfws z9cg`XA-8uiW{uLP$#{X{)%AY0W9_#9A_pKco-h7BT7=oggNE5z#Zp!h_Pql$7&h%P z@)u0m3V1J+^I?D88G^WC^_9V6$iWy7p;MTa-td8Q+p~1#Uo+uD=sl@~z{*EntcT?& zgmy4lGo?QZLdFUuWPsMLb8N-!x^V%CJ)w2L`iHH)YrDgB_A-Tf%&ft?^U_;tn*k(T zF{Olw%@9_?S?!)-^<$^XkaLzZK!qR%cS0$w<%UU+)WUsS{=;U-pHD0IZ|%iVvlqs_ zj)Pb&b@;$@&v2ojjnO|A9u3wjf}o1Op~9m4xA^YFuk?Bu+$#_QNrEF?AX1|VqJsvY zxi8TxgK^)(?E%Pnv&Z{M?}0)%9B_Z5wKPo*0{a&rIQY{lcQ4nHS?0_Qva@L1`7D&T z-`jMm)+oIYM|@;mYU9d0cDE2QgFGsg*2Q90LU1{6zwcWY|GE(9 zHWGY|amgKy6da6wMPHvW)+FQCl%bKYsE$oqH6i>Rx_d-u`~e_N^w{SBa$bUX_9X&Y zDA!Ww2QUY+oLTy9bBe?HHTq1>8U-~IiNiJHf9W-qT>q^!wuGSLC;0@(S;A;O@TLgH zs1C75f0TiAQwY^eDG$G0zQmDLn%fUq-dle_u}*Nv;ph&-?46W&1b(YD@O!IT33C1d zfT<#W`=VOcvjj|+fo|!1j4YYKEIJUm@1UG!4u1$nhn`vRK-NZrs%>4rMKV~m%sgd@ z0R)}qKZshr00pvlNO}_UH63NVKB(_6X?e+hm`aw$X?O*%BbwgX%n%cZ-zCLVEFUBx z8U}&s1YKt&zMaJs9*)B6^~T8CavM=*2ig>k0$rK7jDS|4Jdd&&56Vi+`)QBWw%~n6 z;Dfy+S^A}Z{1{6Q4QL!h-|jQSqo3ePGsx;+un)FAmfsCxv9W|vLElEcN+w$v0bBR~ z{?M3gX5%vaQUC5d{}-kK)be}j(y9C&hQQXpf}I^L z@ft>%4p_|fLw6?cv-~Cu4n^nV)!4iFbLTgA{m->%AJg5|PfK=35%Rl8#XWX3{D0>2 z>9fstSwm}Rc(;gaLk^%0D|1IKCNGYB=VpgJMvxDeU>fBEsE+XVaRtR5FE<4U{4vji zU3G-wfVWJ^^~YwdHLA;QzSZAYY%j?HUwW@$7C;)4g@&)(N~u+GCI8jn`tN0lA$)2O+OzFxPHs>zOyzYOHsv|96Y zH{0V|8xn4i^ieX~vsxP>1RZh3C@U8|tTe{}di`xV969ig&QN!98Y>g3jxNo-Y5w8^ z-U)D*8iR*gOlrS3223vZ%5_i%9OMW3#=MUO2#*I`_MA*@gm3?mc;jOTAb|GI?p9-2 z-LkJ7o+#?=`YjGr&OodG2D6<6u9eme>jiODCK!L#T~mZ|u?1m=B2fF!)oH~v8?mi9 zc&vjYYM7<+Bn+(^-|7m%@+cH2`YntH@K5qljLEvhw#E9?o!>E`A+|>YbMoEhf$z0z zM`B7`UHR!?_YN3+{<_`UdvWh<PN)3tP;*@`X~;M`i;OJd2BKAC6AucRaW1k@x%rd^2sIT3n~5fAmi#! zkuZhwPoD1|xsPJ*)azWni}0Km0PN%qWA2$`?%74=sB}x6k61Bl3MjUN$owLq=oLF5KtKPFMO3ED@v4D&?%EMzOSn|ya2Q)b{&tKAwCsMf zDLD6BL8ES0L|hjLwjNscYNIpd4@)Z;Mk_u(dQdA8C(@1dOa(34-T}O!9^@t_n_ ziRd*yW6f?*R&EKp{7?T_Dq%xo4pOV){8BXS(OVDgI`*{vJ-a5LKOA-5??qjecc0uU zEI;P}%Ny4BimO5A=|@n5Yg7#5O$3Q;nCtmFRh;c*XLL;A=bCF?cNToaVEV*&wcWERHujE>2_ke=lr~K?QDNfI*j_O2RxwE! z39uwXKMjS)m&9s>r?y9DI>sv82Hy$G8qj%a#ewk|@O(;62Lrr@5-sG!NC8kQ4|shs z)Y`zOcG`}~Mu_gxvrkxg=Yu(NsckpSov2X0M5TBqQ>l6x=Ta=T?WZS)O}2Aw6LN@x zrHg8z@;Z?Y=!m0EU$~MBI#k<+U#S#0c+z2%UZ`faw-tfJvSWrqk4F)1-ACKcg0(;o z*gwYS{9w#1*??Js0Ool0>EQ`e@9=B0;$6)llpgFyqIXT(lW(38z4KSQ-=5Hh!;Lp< zxC^xjjPqo_^Qp%s_0?>LTvgl`N05wlj(vN=3%{4TFBE3%ckFf)c?(0H-Cso0%l}Rt zOh+Xi%L6s^n|7qMj73@@cz(cDht}7rgeDNF0AmXCUpP~;vcu>^9LSn33|=0|+BQ{UJ5qAe zIRb0ujt8KsT0i=rsDn!5<9ZOHKOx?y8KgfE&Q>>L_A=(h zii~HWgv?14kwQu)WR!>zao^kWx}5^~gKnxeVqkOMZ2|D3MW9C$pma96^?H z#KQJVqJYcU0@G12U*Z_aA=42_01ucj4#TIZWeFfI`BB#JtYN&f{Q7&8{P?lkM1@D* z`0UTC3+Y2N3c76?2}jJ<0lN|O`*`N19tmeE1uR7xfk_te%J8-VD6B}NS_%6Q&0YJy z=vjG^*)R@f4Fnp6v8`o|-{xhZ6pE2>|N$;<4 zCyrsjqsi|u+oeS+!u(*u(hbpvkCMlEbblQ&=_Y^B6B-S(W9k<4oj7~fUj#22g%9&q z%u&*D#2u<$2E+dhla2~5n|Nq{`7ZYG?FkCN+iQG5I&|V;6O`hoZEJryn^yhHY?b^W z#fu$AMDrmdT34zf+Kir!3U+VcnMKr~Jq|k?L~cbL z7Ydxdz`*X=*d=Qk;?=oT+_pr0&59i zyAgFUJrGSwK*@~uoiQmwxY#bu@3zic1kx_Evom(zi01ftGy3x>vHng#=y48d^rah8 zw%lghrG8=?R0h#9Xs;kB9Ngl#e{k)WcP38yqqA)WM7K-;&4qZ5`EAJIftEOdzQlY> z%RH+~s&^$%+LSiVfU2Inf$@ygR&YS}Za>YnwmyfTF>=hu!L1}n33SYN#@@~I5uM9w z19gLFUp_;@xU&e&WlGQWkG1ZZabwlwR(0S|YT%H{$DTqpLy{Uct2mW6KTsU$gK=*h zi69;&F;~)xCd@r(y|eM#zuST>K-XY+!rUcF4{@s-TRHx~n#zNtbkRePt25x=3GVkl zw8WAnz8D!dsup9?s=hHqDc+&J<2ykSomjbg2ZbZaDBkWExhLkj5lhaCSI)QHA+5Tg zQ2XnXy~wP%)qzPjwBS}hBU9iJFwf6j{xzCZGckgwFzc8!4y}>{55n!L%`tOy46%p! zGGJDy!33$EZr0$D#QG0p%`rf7dLR zp;b&pS1-6iH0)OkKIztMTkX<`mvQz*rz*Ho(GAOsk?lqBenoATb*+OkRy`&)zX3== zm{&%XHY^@cZ3LMa+ZJ$c1x!1Y>+b znJxGiiS;ghg$lSfZ`FOnxEQ?YdU{j6mOnd~iQ zyW=v`w=9d08-vv?s^^K61CpNk>xabYiE2RRASm05keEc`GT=kyq9$&^)UCe_)m9-R zD(Ev-dvEEKHMo@2JxcV1XG?B0<0e=0Q|}#fTL9c>;}nn<%(bvTJzpgxNND#Im-l&K z-3`Ut4l%RIAt*xr!I|-JLC=$$*L{@OMV`O|21s4|ZDSm%eZAdC=fdncdhuRtSqrLx z+5Wk1mT-^-bFDmr@fl=r4Xxy{8N>_UfYZ}fayr3@v~8g7c9>btzRVHef&`Hc#}53_ z={0g^Q4NqZ-S>%*r87Qbm1;%)T*@MUYi8{m6uf%mpJ|cl_&~{&Zbj}BW*vE7Zs4e+ zXFYQJ*@y$3YeYR`vNiC$p&x9Xaz-tO6|HXIlvvQqL5Y7z&s!^lYl(Yl=F%B8dCkN% z+yNx`ZxC~>1o3mu)xOTM^SEOugWPRyhd>Iz;YS)owQbq`?a3pDxy2VO78KZrmeHG! z&nmdOqxHC!ZlYkU&Bn8nhOFQ$4z1*Si;0p<^NcmEM}tZ{$C-n)R+~zfgkMpPH=H5KKAvT7oB}__5-kn6p)pRh4?coza(58G zaCtnjQcG_eI>2cWYEcNfvM=m+QM{z(Con;L-H-4~UGq_DP}ZLwoo0O%!2aaAzC}dV z!n@5k!00LvqC#dp02`l?wxir)81JO%CKGRcMOtqufJKu^Qcaw_r~h@GZOJynPN!?G zXK5BII>Po>?XX|R@|%L%q5cr(3c)hKqC2By%!qm%bx3-a$_E9f3xMN89r#D=0sI}b ziTcvQa7fP?U&;_HRy=ew_bYPc6Qkji*8B5@)_L*eE8e&B^(!;nZ654lG}}IuEmBzd zZsE#SQ%CZ}3U`$*gZp8`dn4Ypeu1+H|#q{lSw5rnj220uYt*)Ol8&M7zP)RrQ3~a%oUz>umdT zxZFdo9#mC2W-D?h;tbXSG36R4^#&Wb)gGy#0d#ar=c} zvVeX)J>!IhS!~H*f|>Qv>R4%#^SB7TYUiHz!AgcZr6XBMM&6q-LRADD2N?gR$F|(d zyywm=krNTa4ba2CrH>xczNv^FLf=QvQw-N#NY!04n?$)rpH52l&iMsQCd9t{$BX692l8%{? z+FQoWj~S}r7Q>I(63It;@FxG%x#3|44SAve`|3lM19*pg^8BN0!#7=?q<*Rz<@8ch z%y!(sYTM3Xyl#HudVtGz5JX}Y3fw%fJ+$qOuvVT&uf_U# zb9_kEshevmWSgJ%U1uTDRH(EdtIkHeQ&4aV=clsr)D%muie(T$DeO5r@^6|8jLrvP zJZBI51pv$g8prlsz z+4{L`@->@0R$q08#mFW57mU5*AnS*o$| zmTg}SKL4wH`@i_3|3clqcv=5TP44kUbrfh-S5fyRqpb#?HhqPJmB!M8fe+AkS(Cu{ zpfJX$h(ymSYLs0~Y_!eMClr$pEP`6I#S?ejE}4O*!S~Afn_?NqAU;X{2~Ol`!Z)8L z#39g#Ym$Bg&VC@wnr%os2E|P)SMSrvcl0{N`+9yjDE#jY*#zD|8qI~z(ZonxRhn98 z0kv1!Oh;zstRR+>;Y_D@m=7Q%D%-EQbST{iMKCiiS(?Z%SKGHPrZy{NyR0$Ip7VEUh_&eY-8Q zYRb*v!M`oGoEQ{`ycI_V7c5Xxv(ByhwYr79C-Y8gh((ueEOwc{s@Vbc0#G&C^+B>GWbfrn$C zL9-Kmf)AYIs}n0DOf_5Neu~&@46;uTT}je5)J`1V^~ zsDu|TRCl)CsX2fqg18Rkh6Z}BTRP4bzVWkl+VO_<;kT%u8oph&#&pH-R)TEM_&Mg` zmgIRRu|cf*C$|!vn~-*UN4v(@q+zK~DekHAu?F!jr+4mS*KF`Ir5+l!Q(|q%Vz~c~;u4rHs%;J# zJzdqj#bA+JSS1f>l0`5>JqJRx&KB1KHK~^YLwOuU-44IW@1Q{!tfQdqYZ9Fp0io~D zp2|gdMFfCBmZ|xy-<2xt5^{q7$bR`Wa<|9W{5uxg$2Eg#04D0B`F_^E5$QIPRR!jK z-CZU0N-dwUDYU{WQKO;Pq53d44HG+zRnQ&YNY2$|ebzmYY&%$}Z2h*K!Tq`juF zkdItlJd!B~kVxJ#{ou1u+;hw938(dF!C=E(5^&)>B zUNh5pRuld^4rufI6LiF%hrwLC7is{Ah>8MvvM#5wTooJm>h_Kk)WR}rUjgWmb)vO- zt}C86VT>ZQSz{{N++u>3`Am3?ddZEoulQgpd%k;g=WpM`Z<-9_pL-?|{%qbef+5*7 z8`eL6Qj~|IosNCQ=!}v6*|MO%6}Z*Muan7-*-4MNwAVuD1iyLS^X?fJwB;;9?XTT? z$(x7&Zya%RoCSu!li1(5_Z2``n-K>0@J;G#Cn9?I)T<%(Ya}q-IT<@|f!N_*nQ{}Frru^Ic*=^cjuV>b3D+h|Mn6ZKmI;b-6dn)_P?;pfn#2k5Dq z@Y8V^0K69D|Co>ctlxWer;tA${sdnCLw+OnHyK939hHXnuiU1F)x{4I1uQqmFV8RI zf$pF11-sTdVeRBewgE_-FM>O@+|lv7D9DJO;;@TV-6n25_z|EoqS&Xm#cq)Se0a~d zMZ|3cM2(hXh^k+rgkoR7$jq>drE13W4F~-tE^d3^+|2CyW0f505*4a>%-^6K3$1LUH z2ct+eG0BFOfUc;omSwbMxHFvyogfup%x)(am;{^vg4WR$Fzk3SrvIxTsh-bRK0w)|J zf_sMU5ZEEK;H_E{+Q%b|TqcIenb-r{_Z$P*~S$Mt*$s5nr_bvDv-oL$HLO*tG|;`f^fjmTjKr#4FLZ$;`Do>(E@D-)4JM7a@#JK zdH*~{B7ymjeJGV`0GKP0ZF_7&k)&^+;2bBPpD{`M$&P5;)=!*tEZ8t;!J&lcbQuRD zH9h9pshKPJVcYvq@(@qwxtYj_w)q(ucLV4QGhhzt$zVG~O?JV}QTns+k2a4rCzEh0 zxv{*WK!6ITWD#El?EH}v7@VZ9g}!1pOO_1rj{Xk96IsP~hGI*6)XJD%AuZDi=4}UJ zqV-cGH&&7eTHe)TxaZay6g~1aUy7g)c*3YX<+f1A2+wG9#jQ_duUu(D6-5k)j zw-;?q08g%8Uu>fA!1t*?Hcf?_xN(YV(TMm`BPX`;_K;1<+0A)UgHoy`;q`|1_JA{8 z!N>YGA&7QpzimOdLhk*f$~DYlna+u!Lja#i_>wa4TNgIs3*;>jS0R(XDy zdq9AXZw~9L4DMI7wtn`z=aAl2@l(~2FG(p7M3Xie-fLjoYAdFE3LqfD zgAcQ)9F6R=&73WuXKk58C(13fybtf(yzWPkAI{n?4LNBH5Ir#6+=1kr{~Z+{%ipPl z$Ia8UM;|pqeLE>zB28@-r4n+V)S+Z1a}fR4lP#AT7K*5}A+x}NSibbc$dRYm2v@*< z9RuD*0OK4ZO$jRwCpe{-b|nij9?)h2TtT;&D;TzNabM&iC`d_Q+Oz$$$}PYn5twYQ zp`~%g2IfAO*aHURdlS~^iMM!)Z z@|@}mML=Cr;9Yx7=3#bcjxIE6oS7-xQ5sx0X&gvn;@Z_U17%x)7&zTi0l56sF4l_+ zwpm9*513v4g7>14OpXv34jxWv$cM3q)IVrC?ThCdX2naS_GT?8Z4KaTap9pNCE5jO z<|PFwUcfPg??*E7QHEkU+k5FegCs5xk;A>ae)#%&W2YnN3|bruVU-R&3bkLAU5P&> zz!$QV^qICQhl(2r3=cLK00jC6)vsFNg%5TyJq<%41RngDE{=A;F&PE6XyEKDi82u_ z3~g%dzzP)gsaCxrZ?Kl6A25-5iehKBe@bgj(5QrB?_krN6E_(Jy<(7Rn4)T7%GU;% z1&WgT|MTb(w-RKT5bgHzsPXvSAjP&5kyMUm+!``Bp6GNZW0J|$4%k=P9}iq4 zix=*3I3(I@4^K-I%tyPZE-P#sDWg5ek&xdvz3ruGvFj^@R+<=Geg!TNSG(OnU_VBm-u;xo{=v0x-CVT}6}Ii&-T{rzwnK)wDq>n}-6)tHxPADUgHu#w6u!-V10x z66f0dEGg-=k|nflAcFgMemBB0uf(;~STGUt;EtZu(3M0nNVaMwF#gg=puahOdP$y_ zl7!*YOkjq`Q2iT8*1Y-E@x2j9T9$Sp0-V20$*3+N-Q2*X0)%}xG=an%ERc}I=EG!F z$NZu9^=)?i)=HP2-enof^5)aHBP1ry33L$tn3{ur;(1FJaH~=goc2i z>qJB$-5-+1FTkEX{DL8?NrsN74}w4Zv8D92PaOdjau{e;{Fc+yh_HLe2<>#p^ceg$ z(&H3MhE-Mo4UqL4=tFZ({MwuQ(3-piX{t5SDKqFuJ?fy!Yt|`b6S!3m^!)7RvO~uB zqD3(H?&4Oi6Sx9PNi_rKl})F$D<)Z~Hb`4wgo*SUBX0s5H%U;l(ke7FU)dWf!xlX8 zjm1okg;$V?8{0ta&bG|y4W45dQfD)=oyIG1I*y;$3s5NoU06kovV3^^>PPXb&?Yf_ zlxLzu+f9=riW4@4>MF@Fy@DE(-JKZzCyDd3hSm;@Z(nwKWE1GmEnSw7!}t&RDM4jS z#vrz-xrn?NpUJDnxRSi2c>oDGR?wx4wA^WQE7?^2!hz?ur#Vb|sO1XF8G0l&Y7+R&_N7t4Qdo}7MheX* z*0%wQP&*)AY=}S|i#-ogMTt8%@H_m)-Drh}_wIfC)Hoaq#eO2;tm^|i#VbCoY&u(` zZ)g^8ZyMsc!g&d6J9fy4gzr-s{ipX^4@mzPAHW#h#s!vVsr-XW6W#;%uf z>mer-7U0oY+O*&5<`kWxKN*V`yUaZ@vgCse!V{rM1$SfjzZ8YcB9n78J4 zmAz!n>^+*IQ}7nLsdHwhJ=YCLX5$dWuG`9i#itp&7ZJRLDY^ly<8ct6AJl&5hi*;os5qt2l0fqeQ;D>Ng$LzSW3H(12I7 zGS+`ANb>`U({~9TNI94{qY83#!ekGLbK3aNNfyG)GievG5a6f0Acz=D&;o};65>BeVE8XPje;yG1(KW+{K ziB*9F2Ml#+zi%HFwT^_skb1Dm4}hStffngdoxP|i-p60GF8?w&deK?JS0}!8+_0CW zKi!9Bb?S%1WJ0_w^Xe0OSSI62_YUlGWlFDK#~=PEO{->igs8}4BE@Bq>0E&=i(Bqh z_#6Hn;Dmx9DrDRT_E!k(arEWY+wY%|aNaVxx1UN4@xqz7x1Uapw5h_9^frK0mpEmw zEjwfUqhEvk>K_!GXYxxN?n~A7?h%XKk#o6ZQ&eu3%?BLsC&hg(+g)zS2Rm9#l<~O5 zd$!Rh;-9$Q_v{yDihJV_W0}@8*C*q?8ZfqB&v**&l#s9RopJ{?mT|d#U}|HGh1a%YTdQRIUK}qx|NchQqb7Qn*W+Ml9f`uthNYGZ|W4RH4Ar8#+~*}qSY6Z;_jzQA-$>romjwnIuo=TS-k?$f5v3n|yO z=KotDOcFmBH$1O(Mw1Piog1^w%K5;()#}H+P`VJb%Y)deYC2$TFOwVome`GdJ$B^T zaoB++941c8cA6%+Z45;`bj=wDtCgghmeHAL&TR&!)ikzxHkwp5V*WRlx&+Tb<0Ei$yQeC+i1M7v*^Yc>o?a#2468X$sC)~kj z^O^PG(IClR>5Q8E=oo6UNDg|8Q^Auuj@zCU>+tcB^+A6#P>?^6i*Dbz93X~^1B0UP zI?zsPNb8{#J~be6=@GONmKU+wmo}je*fbi81Q5@>83vYZa|D;bbyp0k4B|xbndurY zi~wE{2Lm7I%2-3VH3fc9l|dMlVP35`a?WrXJF>NqX<`se!A#}T0|&LCLtQ%5P;}H< zBD%*<>zoq=TBhe3v*n-?9BAQY|F|u?#_vG7bdZ%W4=je2C|9 zy)O@AFs9;FoYOBn7W=yN2=FNVCr!=oA^_+5J*yEO_#8o3nP{Ldi3h`QhVX( zN4L~vmlSk{jD>y=+0CUlKm2>a`r$%Rk<5!5rMIbc1(zD>g^Xc&|5{*&cM=Eo+4=jL<7-2}fJ&BZ=Vx;dn-rPxxZQt?>HYX*>j(bBF zlx+%qA{W;Aha_r}q?B5GQ!bv;%GaGjd{dUy)(Y`g6H(j&?gtOe(z{91KyY~TK9P8` z{5;YLfK3X)T5z~kH$-_{>4&SY`{eF)H#hBou!)G}vTBA!EKimrIo?ol1P(wrEtzf? z6@C;&5s4m-^W{I{L_iaAw=>YiER{Nqvr~0WOh8mAu9y*FooF^ z`=G9Mm^zJmKb-L3^M28&7?V7jnnfmTS$>iP@P%T0`V$?Sp)U`Z~ zr#w0PaJe+lk^$S+InY&8PYZn380u}tlC0e@ir82IxAs5UD$5^P-*vqqy!L8tIgWCBj zzDQP)^rqdd@N@%Q*R6jFvv3@=LUq z;msDBe*=29*4(Fz%}UiN^Ea$dQ=_432SV|LP&cswDL zz&je?lvQL7Yh_toG#TsrLXu)>ZpxfdDf?sZUE^?a z@p-5WqX8B7gE1vh>ZQ%C5splI*woVRa9B00hS>yr)H=T9C%gN>+#T+hcC@_}s~$oeOThF!M8JgZoZa{wK~4^bHpKhl2LB>K9X3vySNEYVvf z_t^);qp`kV76-(ss$zz>sAdOwhgI^Yi(*==WiQ9IVy~rsr*8g(xn73WfRAnt9%yW1 zrIDfS#Na$VgEX`W9A7W8-yF-97;b+(zOxyoSr;t1wsj|}GV9M9YPWXN>2obFbKoKe zEdtyqb8NH;A*0Uh%+4}{x7`hYB&e@c*I4T*jU2j>*WSYVLhM*+{5@&^!{lV{w2Io= z3%>p@q1tvQWmRZ9*$TS$X;$8#IYP+cmsWlnMo z_(^4$7X9u={#K25xh2(X1KJYmYsvTg)CX8-n>$HY0;ml?hPJD9KG(>d{)ek`4Duvu z_H^5}ZQGc(ZQHi{-^R3U+qP}nwx%`h>Fsy##@^i`czq`@?@U;J$h*WsFq*m zV;s=3ArEdL$9v61sKr|O6&0AT*(zH=&$zNl=ox~|T?&xRl~K5HR}~%bh4Nrvp*mr- zsrtBL?Ah(mB(|W3AaHI96PB!sgS~T}w&1h!A!mbX=4JZ)&x4_Id@zTAbRqQHgWdsa z{qF6+L4WM9ip#r#8DO^}Z;<%lj64(b4MKKI-wcZJEbVU;0v~>~J4!9lpcP1SA_0T#P!+Tuwy20;4b?za5#F#Pu9@seI_JX!8O}x?< z4C0f(-lB1vQU15+!Z4j41qNUEkoXIQ%1UE#ZvNdt<#= z3TnsJe)!f+{aUYB?uL{#=Od)+XASMZ7bLYG@`r(hk!FNLjgj_C1rjRFFo_2f5R<0kiD!mA_HB@K?UeY0 z1WIltr(?~DYgzm|phPkeqswS1)xMFRc*@ixQMM}szNtx01<`XqUrj!U2!tjdWK%eS zBeKT%4-kxZ5$0u#emSRi!R~oz-1*0>FK<`I3_;>q#us4Ecklblugt@I*Z*a(3aIrU z2dQ?nOym^~Ab>jvdC%Rujb1#pL%bdA)A8!n>l+wVrs6f+Qp~*jX z>7~wC|IQG1wG2Gfg)(dnLU&u=5{F5D3vpns6*;-CFuqTlb)V}HI1528cj z#J}cXnaaK z+fJ?sRK{R~?JJ~md#WKdHLc#Oi?StZ!CX40EWEtkmY*!^rpa}p$oxoiNJ zsi$n0vCG-2d$!XX;Z<;U9L!a-)0Ws?Y4T#ljaQ2t$4HwMGwC=EZ_)jmU*JQFWi7?I zYtUx>OwOs+7u><}<`$pj-nut)m7}JB3PfHqXK>b z1{^atBuF%}`>>#)YJ`T)7`eFpF&I_#wkGK&9E*~M$@)xF_5o=N_K{1hs-kM=bfx$a z1+?_)Tf=9SPg$YdyHlsCAWr70efsV4eXgp!BTSHl$!hiN+!ESB)Th=U?rZ?sxg8fe zjBVwryHpj5_`qV*SErFxX@U92i!e(usx9*4Ctew$ZMn)Xel! zFPi0V$8e6BaZK48Abh+txKO|itEu#GeJ){<(`LoOt+VF-!zAQJzvnK?O0yatevg!G zzQxhXexE+nZntYss&tWMDS!y+=1RW}=$_>Qy*;*;_#K>MFi!53siJ%dVauuK2DKky z`xisCKoIWDDllBUbzy2SRw8n3kwNd;VoL1&bslQ-JV<`oe^Uj%b{fhnS^HzZql{j= z1M8uK-~Kf<@J7BZ&8VBH%j&j|qIRI}b@0UFSk$tw?sEX=JhKvrTLsfL3Tv?vD03Ll zyh(nbU8F|~npk@j9c~}{usgvrrM}j0l@DRVZY$%arHpn*>O8Z z&?xB|mhq@W*g@W=sX-B4JD%UrYWE>X6PJ+=>YXPUhq7*BHcPz5easVH+?2moMDkwlYEOGAWZYLB^aY)nNNEKr zpoUQ}6ElDMqv__dv2uB>r~lU?;%g&pW6Y+}L!(|$;(8AC0~Writg5N*v9V9`nvs1v zS1Wc$!_>fF;2%sw-VYlEzR6}~WADnW@7FaZu6bVh#IDM~Kb(d>`D+KZX#uSqzu>el zIb%ST(cg-b^)Io%G3eKKNFR)mUmEaDS1TKwUK;?pYe1h>`j@5L-znO4fXsCt>EAuh zY5!`T4|B#3!icXp_$CWx`kb+2T(_>QexK9(UYJ)wC75s4lK#HrP)iSy<}Tt*>JX!$ zCCV(=yAa2HAB=_(*l&%XZynG-VekM6yH`o@&pqgWgkaxuVFxppe|q#^0mz?yxM6c$ zUFPbw)BrDWjrTn9@E(WH5%k-r+>+nmx&uX4jYn>aNieU7er=y=;(r+!2EOanBO2HF z?XL+q-4T0i|5Rey5;mmLT)=7xXD1^0p%jMaC2J56Mbz-77k5QTb#~bc#?`*YZ>&IfSS`E|cHKRdfshJ#eTtT6A2r7kGi?(Xy(|xo^oPdb#CuA)-!1B!=npPn zR(G894f!8I^Z)+Y|IaW0p%$Yh@CN~}2mu7d4+I2c<&c(cfJ=}@K+S^>$WgO#K>3lr zS0T_L(AJSLC@)E+@zdOVFP9cULqWw6B`qeDgTOMRonAF#9snFDEFfmS4iN6gvFv7q z;$(%=WYxQPoNl|!Wp{Laem;Le27vM*93b3=A^)O@mb1lPXLgJu_B54VnqYZV37tX3 zc~IA5nnshOmhO>PTSwOi*ey-#E>zzCg>I93=1YI#rEi+5*KE8-aOgwBpStZsmRm~n z(zih-(vDYNf8)4w2M5kWlcS{%J$hPN(JvytezKXe*3fRa z&(%g^mNt6+6W*)^RGJduj|dd@aL1hNy!JTp0f{_2I5>!hrP;y)%wP7?iz4(E>qz>J|lKrkQeWjv%Ke z3Nou!?EQV%Uf_W~AeV@;tc()OuJL=y$)+E9jOGA!Gn9R>V}U2pd%GVU2fdAOtGNVy z(Eg}ZP!>SI+_mo0`h@!*2mXKE^uHDP|Fag%#9>Uw6BY=_2@?p2IPG8>4Kr=OkP8y< zU+{(!4If3_73}ZXoh}82dXOmM>q&}JU|6t6Bs2^V;!z~2=+Mz1WW9CR4bzPgGvWb? zbW3AG0UZ_g1$DWqdm(LuUsXAq%qlq-f;qi}OJ}mYOaG2sOK$`#QvUV2-0kZ16S1;1 zon`yz@7;a)^*!bc#R+;}L5em<0(Sq*L-3rI;$$8l5-`NNI0Z%dbhOU}ACk4nJ2?qr z8tflN>H0=W^i7X|{{5jT_YoWqgl9k1hfAMhz3>f=6m5T@GxSd8#W^``?n;m5W%Y{3 z@m@e9s?9TF@=rWtGt73LQ`hFXFRXR^3yaEiT4-YMo((2H%YKqW^oz>%04&1F^DvGJ zI2zJ9Nh*3-&}pU3FI_}-{%aTz;#X_q%!qD#G)yA$(6FUKmWnHJ6&jeDgY)pjJAFrS{WR z+&7_jjd_0LvF-D&xQBya2NVzn($i;dC&@ajNi0Jdhn3;*an zR@HfZ9aq0VXJ}(R380qE!FGr0;`sXk(j?MSz7B>WAGcvjYR%-xt7vWCQN#WE(5&?y ziXtjFDElix$&V1wu{e>FAgWr4n7Bjk)45ooX)LKIWu|sEFz#8*Kp+LTg54u0iO*uw z4JEVTQbdUrn95BkZu{tJGR$Q*YkB(GF$n7Pn7xg5Yd_uK1aSEl@*~<=Zfv)qR`$3G z;0Rc{y^xMAfq24aZoQq!Xb5~+vV<&X596?I4y|1&&y7-Nz-j*p-qtJ`cM6qY?N%&# zUs{|NW=A*}?8^jVP?DD|0}63=e(8YMj2qic6^Gy|SC~o{@owk~?vyK$vRf@rht9&9 zxXw#4T0OD!07%X!L-8DNqehB&j_GLFGYesLeQ}K(;3yI=4C#>Za966>aEQzGt>7O1 zc#h0` zrSRE~7hE@DgX6>$KZzbYV=CezlJv%ok4NPG0;#Ie1<*~8XtR3_3kx>)96Np}4~1?N z{N-8{P-bh!leti$-5(4FR zodK(rZ=R(SuL(P50ok+m+X1)R+{adyr7-nk0Jc{H^dQ=nn2ZCfO865E?463yHV&~i zOrwg5Coy=9(&){1iFcqgn^^?-F@o)F6AQln%`4a` zP&}7&em&}0I?gEVjnqi%1O@Mx({Xz!zY{1#92(`z_&)^6-q&PofEf<6=wI9QL zL?0x$ITq;~kjxv={m*79nh=dOaMy%&!HRv6IMDW$wkbeH%s|Wm!MZ~wDDwfQ2SZsn zG);C8`$gB@Fr$0`?&~=l51H~o$1MXG_W{PChDi%fJ9SoEeBeRPD@l|=18c8U9V z!k+ZWlJ+*Ac!|BhYl0ZDPhX&EJ61?2z_ z*uz6evq+N9#Lxi`4liuXLnRp?pb*R!xp0KVANg^HhF|hQ?w|uVN2esGU6`2+N3-rH z%3>KD6Pzf-E@7FJI#1yek}Bc;@?EYR;g)HBX)Ea8-ZNwR*q!l>L*{38{qT_()cE2& zUX=@hx^kYKsxm43S}U%U%CAkE02M(%B*-sWbxu6|VMLZ7;do3iGZ6}aPX)l20&*Tl z2Uj&mK~s`Z%AmgxKK~>#sl1@R8vAE9&F``1_X#rIvV>vpznU6=0g5ym<^#v}`>nL& z0Pvg(M?eo?0esZ>Od*@LJoDUjl%#r~)q{-qsK@$A29SHC+8i=Irgth(?@QcITy)-e zrwtREPU-Gw{!STPgd6~mmjOn4-G6A?V>u-$3PR4$*HcdOLWd#`hYcfjoT$TbhM^T(;}+FPO^&RM4c~Y*l?co1UBWtXd$k^!rs^#msNmOV(Jk!UP%$nii7uQ z)1#PFQ1*wffN;XX_L<4iFj`y}U239ZtQjfDkUs{4JfVAqr83|Bx6$R1EluM;(B#w;Tbt;q62g!0&oj#`;n0;Xu1mp_+tgYjpdX50;6sup7oKXwe3=gui5E z@Zo=^5m%l|-JdXHunD@f3_|cQeTQ;wFH$&v)M(e(G@F{r3AS|23dQ8qKk8G(1{)mNZvx`(=|9skd%@8-( z+8>u=F*PDvgb{@oHCYAE{vBL9I*exa*>;v9UzFTe4g-@ib85$3N8lEi&NOWZNGJAePa`s+Y+ zeV2GjeSMNzmr|3@gxG1DR5>fb)GG7rb8nMUVBj#b6(C@uPyZg{wW4V>y%oKW{a8D6 zJKAl&IPG<_$`>*LcSoPhqTx8DIY4g$-lkmxI{^jwH-a~DLnd*Lr>~7bNpUzAc@=kR zn8}+jYUKVS0PxGfJGRc(1?2#k#IeL2WzNcQzsbNza>71q5~@7uIZ4CM$=c(2B-4Yb zCVet~pL+t*G$#vFFdz~TbO{cCJM+ zDDw;ON{lqC#7tBThMI?Ra4^Nh*;nluhKqF{}7-EI{2v&RX-rS9+ru@@rEZq zGbdU|H5TiafZz38$YtDv^EYb?y)kdpM<}D<@|rh4;0>w7@m!h)$&4EU1{0fV+G$w4 z5GdSbu(4T8p18BH+&s+sep39mik*WZ^OiEJ^P*LQZrpNJ3(OT^N!Pvl@%#CIrKJDg zfc^gu<=d=L5TJn({u9-2_Ff2G{?OV!!P4S382?X)$7}%?Y8w0u8U_I5V3h+6#2P0M zmap|vgfT)o3`3gqzTJsqLft&(ntdFgre6+*Lf9`sY<@Ej6G_oh@;n`DF`Lc3`}O?s zp#U^@zaAJ$jtdV-zU`qn(ij8DLBOJkt1Q(sp{Q@=hDEbwq`)-otsKcYF;*L~`)L0w z!He<9YyKdC?L(=)FdV>QUVcMOUU(BQ){uXLst{I-qFrXsdGR84Ob}v3eoO@ zdT>>v8dX!g=i55VR5P6Rb;~Sf_cJEb)_Ozi=x{?yH^Xgv3@Uv65}n~Q>2Z4#UmnGY zb;LzA#J>DS$nt;Zt>^4@EdnZ#V|^>YqpD4c$X|k9!Wd$pm;pphNE1HJZNsplxqD)``l=~YG4#iM3Q8%ArZ$&akJ~ghcp)k;!s1hXNKnu%Y z7`EHO7kI7Khz=kHGs`$^L!(c~c?(<&chATv^i6c*3b!;xdgikOGwQx-7us{k9mtZ* z>DIvx-E!ewKAnv?!V#10Y1#Z2HaUTnxQac2cgApQ<_;XCwVbb8U6bG5@#EI4M!2?f&>o7}$R`K>qKppf4g$ zR6vZ1j=~SZ?_X(}o@V7XZTWUoL419AoGw8=j$|bglbmn1ec4UCTdJ)E@t4LcNhHNT zAVBi{w0$U6OIOx(b_VO2?@!y~^uO1)XWjq}&x8KK!WeQ$ZaQ8n&Ku9YVcFb6kCPZ?@QxD&yV=Q@AbYIud6sb9zaR6X(dBW`FY#Q<5Q zG&(0pXu^y8FOJ%;Hp( zN}ym(nWJzybGt>j3K{AO#t<&3?3FBn`RyVENe*`QenJQCP!dQ;CFFxOoa78~YLwF%7wz;*HU!I^%rPm-gg2ex1Cx2arfa2VV#da03A<1O-UD zt&zCAA?v?D?4+|6OPitXgqMV2tfUi_BOYh2uXiu)-8}{cJ(^e_zj_@`eP*|J5x-w^ zhJYMtHZX~BCrXI`%SkgrmgFKZ+-r#VLVpqtfZXj12+ydGKZ(G`6_HL8U@fnN1vf4! zF&9duEuyK&a6&4$f`=nc!j{-N7f^6BWlrkus?aX{8nuRFNITFC=3@j;+oRLa&SWx3E_jKbs`p*|QJPD9`tWGy-P6ceBI zH@a0crAKT9D3gy9IoRxS`3nS4W#B8<0i~nwNzljG(K-Z*r}XU+o~^983x(Xt4#GQX z@pbF~x@`^Olx;Yia(}D>---J%|{b1&3b9^~V8g2XMkjxDM!xRT;aghuPw+1N@^@eAuq%Pxo zTh*CiDsl_K7!UYT3N{&!l+@`{R&AklYSepxX~MDDK`#Ar?4-C#29-Qzfp!Mjlbim7C-wO0`RIS*>N>&!-=lD?hUtP&q6OQp_lDi-fDQ=m_!v z(repU(yxj$;MZ3w6b^Oz=(bI4LgoKn#F!Z%cxWaY<|KgyR*w`kg^8$8-A+Kw-+=w~FTS209a!{U7r??| zy@ro7b6YC@#LnH*WL}tMDaOeHWCu6`Jt_nsIOtbqP>0ZlP+)R9=srh`{fN5wb5Bo^jm?@b0CGlS5d`xV ztekL-94~c>2Z5P2`q{YpDpp>`;xoA!5@0bDQy|wuFF~dCCWduPWf)LyoD0Yu;cTJ! zMP0GRuktGE%*~K9Ek~^Y_K=&Q*9nqP9x!V?Z~jktb`4{-N6e@`n5v5p3Fb&+wfR}i z7;|M*K6L61GzaA9CUjC)qZ^!DR897&(xZz&KMg~NG#vajbiuX5!8cvLfxng_6zC%= zIQ$wa0#--J_-jrafnLp~PN3DTlJga2%oXp26>PcGQ;ptOLn=Ff;(CmTlcjZK7w`3@ zF&`54Lt*+gq& zKOwyD_Ds7eQ`JsgweP>gd@DFU*M#_HDO+gSYh`l9EL~g14RE}y8uGIKf6tWPM_}lw zj)TIzkef6+TiK)l<;O?w_%hr$HJQJVFRM?t1smm}NbWJTv*K*Ce8Gnyl{gocMYU%v zaaEROZXVHbq~DB2=N&`AIOyO zg+mN`Ykif)XOvS2hQ6ucI>zH>%;Z^@pLYy!{3Gh0b$+cYpgyi#?vlj6ggZM z5EJ&Piqy?vI_PqLc-iUbIgU1@nshse;o9ijiLvoyPgs>=P8qzHEHgY|z%|Bah@EL& z!}o2uAO$4_bBSSf*UnHQ;i=94>2(!qXA(G_S8oZ=1LqK*gV4EOMyYM)#!(t;Q;7D; z+ao6Mj^}6s#M?}y|2(;ej$4_!LCprWU^U5hvD;U<-pk3ubkK_4hQ(3k!3xiVYL>He z>v1Yq%pxYV)Qb_ngpBtb3v6EnxysIIvOdMN9#M=nrjnBzrZ3vTCh#|}fLeZ1%C5#E zgT0b)8W93R4Y-p9uo}rlWQ$@LtI z=1H0Wlu{I%Hl8oRq#YCIufUj#vUde3O~mf|`x!0{DI!Byf3LMuXOyO32(KI`p;l=6 zChsuf2JX(nxE{FDmYXG*fOdhd2WAzn!7owbFTyQ&YrY8v-pAZ5@!@)!uhI?7>#Ppe z2@KDB_yTCk;ZMkTR^U z4WN{Vs{<(-LR5#S{o04~edxDNm(l~>Vy##?@_>iOyT&zv{20~jKP~h*dU;9sYkNL2 zC*W%h$`}g-Mlv%8$_>rTl-mHN5=61`hfbE9lG%^L3+n#^3?+^rJmL@4LUc6NZ=mwVmAs;A zH0EFK4_&G#)GZso(Qe(}@)ln}sI`c`a+I0rD0V{yhi+dxyNXsD3@f4IO325QU-y?Q z(FYw%)vyR83n+HT_o^wJos&>PS*C6mu>I42|2GeI$hU?5RZh#ZTOp%Meb#%GwLtit z3iG$J{Xue7ecEm5H*EBt}Kxw@rR($DnJN)iQP_=|{mW^LBuR(^=U|3tn(zs>K51Ww?`}IP3d? zh#HpQSu#YJsD+U-1unI=IFFv?F^q)Y!tvW5phDHO+Ts`PWmo=vqSa! zl!aBUM^9lH>@{`)VqFvApa(1BlMt|zN8!ohkR^O1DY2P)(8HMUWH#7|LP)X~^kg0kwTGnQKUv9+rKft>j_?)s)oDJc7?BvC z{6i!*Ba|#RGG!!##AO!!F77(y+t3w= zx{reC545OVb%s81214RAKwJTezm3y~b&?GGnXpcpA<{F}Cr!g6%PUN>shOXUn9n0* zR<$p89HB;kN}^<57m*)2X%`yGeLImJc}(%oo<0W6S*R1DHbpe^GRQTDZP698_^KoU zM=#z!UOe+>fGT2J1i>uMDo!9_kmQshZRsT1A&F^T2ER~mmqP(u{Fv9tkk+^hb?V_% z41r%DSQWQ0NO}zFB})4J9EbQ5B|#Y^Uk3dWB}Et{efIrhn=wRq<$wkxO{VqT0s)@!5Yar{jpO}^*xvd3fOE1JD*=A9GRl>0pDlikHqbeW7H@9GZbi;iT zBlLpjO`SWF0Akz(fZwS^3@YcSwfO9?&;32q3zjfWj(#M*4}3e{R*CXbhd>0j=y6TP zEC~0~9i%67Q{(;eg0BI@s#m(eW!)j#YmOI$DJoawwbp)c1dK{osCd4pkcj8|BB>?e6_%W9jk?)_p^KufC*2q z18U1NUdfJ>gV&`$H;drxO6I9eeY4jTy7M!&u$xoeIjy;Y78^miL#z=-h2AR`n2`1! z>kr%8+Hj4~zyq%m*! zR@0az0A|Dkua0;vB}^2 zn1ml2nepB^A@{e>??9V>f}@Y$N}hTgVgNui_6=t(_6_{2k0@eU~+wefV&ieR|erS0_0tZ*;|Ch`6yrhz8p0lZ2DH0|%~Sh%>5vCAuwA zjzYqrJ!euD>)&lNI^!zXoIX$SC#zGxDF1l;c}aXy34Ia_1jHBTF%$zNGJBOIzvBRl zPcETP-=@2GT?>pzi{Gl50wl%X7FO@-#9v*DPclFENaO&W@&61;E;L|&7*BG0#Kqqo zY+u^MUp{{v5}*8p-vMc*gg=i8XZlM@e#==ie*J-4j+VVn;55I!}8LfkiEY|@IoxA_j*qn+N-Q4{sS-M(;&h96lVVp(6lm(!*EKe zyy-PB&iliU(&X(sinxj<1redENaOp+-!}O&TGMu&|G?T=fobo9ELwl8f2Tnz7^w$( zg;wO-Yoz{rP9Cj)%|9~hBHkbYdmz~V?Sx_BvH&{m6J{+>*BNr}i)+?KkXA$YH}jc{ zKB@dVPrCCYioyQv`wzAxK<~s`zqMns1d|i(!et?fMYwpVRkBkSkSy-K0)ElXwsnudFDCy8eP+NtdT@dOig>!_Hi;$25svb2zW0=t=)c{R8_j&vc=2dSHiiPtkuEdNIey9BM<+X{#q2U~Oo=-Og)j|zL`?8OOCRtK zkE=Li^A@KuD_}e*TaBZVuX>a?`aP0U)gdc0Q2Eify!P1M#RyL(jSI_r!IVAmBf|df z*flSB*0KZ)nftGDG7j4bG!n|w@JM*w2{-j^HAWG;V2YeFgiY(|ijv^@VDr`ktG@Qa z!iM)y?HD$AzUT~BF^%YmX5NgzVn)3n{eG!$_M|^uw4%Oh+2FXU8xTO9vp9T{?M~V; zsD2TkR^6vAZcz=4`c3X7Ufw@)>%i(`faa`cJh=rJSW$WA69w_O0UmkbdhHHflXgo^ zqo)#oD6Wf14Ww)i8=X^`xf0X9TCi#Gw-adb8i z2w%xDUrk&bPyK?UUty-NA*uACy#^L@TO|91>z3QA>eWz!SVCaa5265 zwk$=nx!|8#gaueiWjf$zL3a$M=}pwy@Zu!l7FK6mP{O^01N=2 z-q*6_f7yk598$^C*wR07rVSVSgP~8ZK^0nps(nPi*)$LTDuC`mkGm6`ZlM;eTKHja zzAFUxteUOLTZzbMj(O2@#Q9?3%G3W-7#5DAcuR;=%F{di_i?~RdB~E~9`lr1Z@-Nw zbE4S>t;LvR(v$e)bhy;tEpB+=?sX3^aIFRspSYvEk`jb$&(;~kyA#s92cd(?_Co4y zh78iwzzbQL8-Cs7`~Z*O3ts{)kB8vXzLL%g59i4=N2>+`X$N1a2OCQ}h+Zm-s;H$c_B6!x5;WfbojRh4C4+2HT8u%-Ks7s0n1uhEdR|H0wsjxUDtC4w- zp`G2%Z*{vCby!uK8gvh2>xe=p%ocZ*Ds`PS27iN=nwBcxb{NoLL&Pm^Y$TN4De@oUr&3U1248aRA2QeqF;x5X&6 zES_OWqmwSBNu&34TI%#OwpQ}U-Y&%%v@_kruAFme=79sSImZWL8`lEI9ot1J9>( zrQkKQdvD5>`wW&|;wa9^VrNRd@fj`O)b4D0gI=O14(x93D1E0z7=b0jx@ZTdMeJ>o z889pDk_@UDvbW15{VX<}605Y)Jh8Q@M?A_rz!Y#WZ+3G=u=8~M({`g-8QWGdvy_-^%+b*lH`ycHuY$>ZMw}h3j*_aH8O+-19qi{7 zfP2dd7)Swz0~=cI*m1QTm)%k#dK9j;I#Y}#-&og<4)!olMkmYmWfC`c47{9NjLhMZ zu1sp3^=5sAyPe-x;)*Qz#ovL&14>VWk%07^|uIa(UKDM`{=1S;gJ*O0%KT>{o6;(L+5 zL;md6h26b`M^FPqX<}noba9xgr^i#xI5*?<&Gl2u=qnYqP3=tc<}G}9k7gPBn(=*R zdE3eM$*z1l zzc*IV(?ZV1W*svUsMZWgaTp+=>Cqf^IV9j}o?1d4ANiEtcskwBjpn$n>VR=>p2D6t zFA&BJ!V`J{%_Il;b`+H8-^QjZOYpH>S_Vt)` ziePGV;ACufON?tO>}jhtM0BYWgH2gn&m#`k`@2G#3a?Hc!7egy##xaPQ0CY+0c3bE zton~_gik}Jd};6OpRH%8$CUleCEP2r2~l`;{oIx{Oh(Tuzl3f%WB1CMHYtYPyq#ui zi4t3U+b=B<6gRlu&HOI7e-KL#jbJ~0pFTr2zFVEX0~mYV0&Ln$aBbxRM;XW{5@a}k zA=_ADSM30F%wt>}=UsF3`1FrDJ+f0a<#hx*o=&E#{`6F7=9XeXf`4R0du%$h1f^z z5$2z@><(<_s8`v~f$^L)WLK<*s&(V$*+uMYxan5H4P475wtelkPU4+W=sVFa&8H&} zyvY!uen3QtVzcZ8wT@AByNqXZkuw=IJXeKv^BVB~N~7uuXmwrDah_^ZcUKm>)V(bkN~Y2& zinr`5=C-VuuMUZ$-_{n|km+U*f{eG09S9k47)Qvy4@O?cZCCq zI}GpOdsH)W5P-Q6&Z*D4$F>Z<3EsLeMgwfPYhB!E&n@4m8MN-WU>zi`Mf!^mH;KPKURFvR!x2qIIfXDnhzdpM!%y$H$TlvpG}R0!dzgsy7{(PYxQE;(M2L)H4o zYe>Vd{$gD&2G_@1^brR%r5^07dInypn>}MXFJfoL&s9%(|9k z*%_HHs8y}RT+$08h zDW}}QfIN@;xHs@(pWyTM?0=LDonS0ydB5rWDQ4zM-oU?hMi>_*82lpzT<)i;%bM@N zzA;EMX5=xy6BUp84i7cnTG0L~-t8UO$d&x=OB@YNrpy7y{_Kp}L(2KaxnX%-WB3Q( z7fB}TdZEpAckCv^!&Ah21i{&f_>rmE(MN`d01z6Xpx8xEc6Svc5;dsc)XbC=s+iI# z|CPApR{Hx$YE?{6`NJ73CQt)6l4s#8M_JROk$%(4Kaa{(iK_VVw z>PD=KvvG6nK&&|a80f%^R5)DFn1|f;zGt+Xx;`aEW`I`|dqcYFb1VpAT3M3nW)|T^ z34j}kCU_zyZ>CjwFAH2hC*~&h6)0_{BIJyQpDH)^BP5xS zF2yv%58eUg4$nyAoHqN$T)N=YQo4 zZu~tiuqEdE+Xon2ME@qNlswB?y{>kn9$@u#nWE29kN{a_sn@ZSc|cQ^g@L~T)&MZi^+F8k|>~QWmnx=YOr2>j(n-8sgX8e}#m7Qd0A9D4O zpZNdE$Gi4w8|Ic+(UvbHF)9W*Zr0es4PZSW5(3@JS1drdW2impe*0v%IEn+KS+vuK zYBnV=mJ6>g6INm27wQ|UnYVhgP61@oD$E-|7Ie*0+)pbtg-)L+UM-ixlH=+cZnf}@ z1!%X10c8)gkwJ-3(_SakjGq|zGPctZ1M|f(NPF6T$iow>p0y1Wmoe^Q%4Ew8Pv(X| zoEVH1by{2NVBhhrnKJihiOE8S_;O`bdXl2p#5cr8djnU^W(yq_*@bnj7naekL8*PI$-&9uFha$OO#~K@3KXA+xSxQ@dP^mv z*~+xYieKoTG#ip))0yMf*#W^wA_ca{I`9<}zm;xx`AzZitG~jjeL$(Yc6!3csYBev zxG)t8`O|wb!R%$Z{!n%1+V?;5S~FPCzIV~T^D>jcj-l%uNBvc_Z|Hh4pLBkikN;Xk zL-$Ckawtn!4?gE;To9Nn=0~co1YCjuAs|`j$r7I4~qXIM-ld}2j(Y+0xeS2kY@vh|0l#?56BqkDYBPpfBPRMbhj*^~e4LS|pj7;Vo8l-g zJ@!u>MDlH&SmY5+m{TjrdbpM&;n4_(ayy2afm`$_3H@pXjQV)-<=mV+RQJ7iP~W?) zzu!&l4HDyo!nlQl@GPeu*ZHnA2|P9ON(|aLcV zDre|Pv}yCow!tksz6j@0kh6Xnf#3)GL#>ds_!>-}BqgN*HXYt>T*F&FXOpL<5>uP_ zt={yc!PXZ1(#b136v0UJ%L_FG7T{WTKY#Q&V#-Yk*}Z66bUu^*T%8gcNZRs*F#nzsk`+7#FBoE1BiBilqGbY^i+G8x(mIR z7qi`nCM2WUSau|+EKrlN5DtA2N_+GaQHGc!MQO8RHy6}a^6#d>Zb_!^)Hu50CT3)jc}0D|=YaV#yZI?|XUXjr@!BizD+A1i;NPbR$xf9~Nm3w?ehOdgZY zU^CEv;e-x%)A)cT>ftcgsh! zg_tRVQNnBaZ%Jn0brRIM0Y0-b32;xS)YUNn{??k3bkxe~mF;PI$U%oY82t+_edwg& zza{^k6%n@4qm^1Alg1PFA{+feh|8v_p7tt`Ll^NvMUcNhk3y8X7Hu>?8<17`L`%bT zQI;_~%YtP$tEb)Qs!w!E)Yk88NSft6M7m&XIjWmV8hc>TeyLQh$27md(|9z11En7W z^bGxbyVvCK+bU=sjGo+^Pv>3ad#y_%JZ-C*DEMwoMfgB8VRVSQaH95PDG%E%7`K7M zeDG9mqs>YA_@!F0VE5oN2jON>JDTr_kau+1?}tmnvvaC%;+-k@M3!&n8?dgo_svkE zq;h{rIb7ih#kdyS5mjt?D4TM`+*=R=I`v!MwJPDc;yGzZPVyv;iP+Smyg)g^U5fX~ zjqTV@imAJ)4O}Y(>ZAK=%^CglUpY*S0$z>G% zHmHj5WzQC$yxIrGT$JuERh)1@V9hlfW9>G?qr92vs+(F^wnIiwm}32h+M`FVti1`latnj0~EFjzAL66W%cmT2UwfVZ=N1MUxtds zT>;y#IQtf4Hy~8cS6a_sS}$07yTH*0%F!FE_Z4U4p}8RgAXfeY*WkNdi26lz@P>Qv z#(TNLH!Je-`?7DN{Po(I=Gz}obqtu-{R*)Kt9ZU6d-(MooE7xD<6kY>Ye}3NR#-}* zGvz1r98*b>Ui19OZDcUJkJF9ezi8 zlqDVcH}O@8X-v6#Fk{Pzf&9oXIQI}pmHwE!e3R8Mw6n^YM(qOvKVI#xJ^7=xSjsla#Cj561Pn^ zUGi_EWSdTT-g_g)LjEM6fUHsDfqTfODl<+M?H}A+=b!Rrpkdtkz_J!v}p2uh( zJ@kRL(1fc@%5#c5dw}cIq*-m?99ul2(Bq06G9O2bOB#KEEW-mRb}rkKMyaH=3ZXz% zYn`R|GjO=%@If)S&}(QAR-}aL&)ZVQ9cx-FN!0r|%^n$oe@iDK*v43p$FNn=)ewNP z?lXvF=9ZMWJHchL6mD}^TA)wWq8_UUJ7q|Ctbt;dW{QxktuhO||6w*kl9^>yN<1nQ zIa#*g)hue9#aIKVaw=IB%dE9RQ0#jVhuM*5QQ~ z)7xzA@Ct8zmKmwyyYujCwUv^TjxD*y>#PgW+R!0?76o{3-U3??eyXOHc^LGRfx06s zwtjqU?Y#ovh(jORZ~xYT&cIW0fX+3E${DaVRy{x9%qN_|)jrxV3$G{F*uYADT8%)T zE2fFlKGA+wi#*{@#A;t4KQkpPCg(q6Ue?KU8(R_26_hTz%5~{HsoFm9$SD$*Z%VDM zc_-v=M)%|W`{ov(zo}Tl4LOcDOnDX?|C=dzJ^KzIMuSH}W+x>EF&5qSB_BQ^53soXw92Xo&hK()u9bw`>(7?aL zhWEe?;RX}d6%QW)!$5c~7d)tD5Ds|k4M+`oEcLJbLR1G}RCNDY(gQiK_I*_AeLMgD znY{wogg|~nRuYqFufMUCU#lrm?(fciwEqd7LYWhBsIXzAma3#^_z$ID8?mSNY2Fp=7&*N+o@?v zW&MaraG9uaEaZa0)R&n6g}q7_(TD zOGofZ*X^nCF%bvHJMYnFQdvmjQ*h0tBhqfvE$$v7e&MW{9SYA^NVx~YjCzaRoefa& z2dkvPPMFXhU%YGs?>#&yKYUP}=n#C1O=^1kC+04CfR;+|a7y;>eD!TIFYE(OC!rqY zkByKmxuw?+IOm;0`XrilJi$=T2`Cqvd##BMX7k}ko`7*m^UV^mVs*Dgs{E}>DuUuj zzmaI8T0O2Uz_w!F_sN_gF+RQOPSJ=n=B7ACYGa}9X4rkD5;6M+?(;SmK)mZO`^)?r zK7VDp#k~BwWEW2aGd~tsBXk4Qzxg|N$%0Xw|FYQ@B69c`J`Mj%%-5|iw3Q4yn~Zm9 zBuIg8Icac%M`QtlGLMNVXO>rHEz07VI){eh>^sKL`fiU&(tD$f(KkIqcBMQso@DfK25}*&pu$kC2pFn+Wi4 zC>6~D^Vb+ZJ%uD?3#9nCFk#Gv6#b&h8QP@HwoR5&yWRX3F(kmtFW!XPDQjQxCS;4b zsZ1`jDV}GexA*C3yI;SdC|4Q!hy-aV>%U?sy9pkJ z5?QrzI){@yS1FF$EOi`$yma%cT-n;;kAK+M~}*3}d@!yks3tpd6bP z?~P1<67{0IL!-;G2`g0A9J23MFkQ?YG@GzQbL9zpy2@AV$os{3cyCvR83j7A|0L?M zS#0B24#@IW;Xqbd9(lwG$%=>HY~(4X)J>@y*f*=H91&-LHWcsIjKIAGf`P-x(suno zU;5H22@->LLUU(;R^oXVxbEh6WQC0LK49F(f9) z{J=!ehrfZKpvDam3YMOm^6utbYqO-&MM+o~a??VZBedQWn^zh8o3o^qsf!EOyM>X>ZSqe^XRZR5Wx7NZYD-;!WF|RZGv_g&Qd^^{z z2JsXbq~oHez{DBEW)%`Tn&f=*qd95*mtE)4=%}*3*GvUiya2n-^lB^zccJK%QvR)cW}RJzi%j55fFe@BWNNGRY?@|2h9}MKMnb4G z)6yTT&C{mMI`QUquD)vy8x$%|<^tu4Gj$=^U|&ih+aW^EBL@{RRF0s-d%9!-rc8?d zF%d-*b#A&0T7VnpLFFZ96v>)%YjE5=u3fgSc{H4EIB#*Yt=pm2wpe0U0PsoQ$=^_g z-29waSss5))RMz%{p}JYfwZ$0WWF85#W(g}PA$?jP*J4Jd_}B?6DmIVcP?We3Tel~ zzjHZGbOdy(B!;mK&j?6$I7AVKHr3D?tBs3*$)?4}|ofxgmSbX+m7|laXge z@#`1KuV2Zd#$yp-OClO`QYp;2d>2oeutkDZTq z#Rh8;M2XoukhT-x5mNSJ@~T1HFyxXj)M0QahI&=po@FzpNls~UY_G{e#vyzunpf^ znS7OAfX8(}ZP(n8kDq#9&4+eN@#X=E-4owHCW~%9iz(2O+hhv!;{i}jHfh)zjfZiX z{m&uBN;nB5L^$?s9iuT07{|VkTI+!pB?H_Vn}2NU+;IW3NeZ|()O#5=^&y1P!wliz zR_`3rG1ceCn$}OnuR&d76F4|U-d_vNIfd1t$yr?hN`kbvG~l6)XC%?!iTZn2Dod;@ zSp(wB@v5$3SZX+dD{j}W%oC0V9dL9NWU_<^u2%B0xvc?7gMzTV200vcTx*@f)N@S| z&n5$vpCO_LrECRj3mx1zhh(=^!Lel7F6=AhSlFUx5e1)0l?ZSuy)g#WVZ9I9iO{+9 zD)E3}s38bQQ2^U;^vBaD_0(Q&oeQeX=Gr~jegmM2XTP=2h2Y1sNI8M-TfSWA*XbPa zupBYUG>vu6<3!HfWhm^Fh{2AATGH6oC0~hJhG>I|4am^9+|$!b3)lLcFsJ?dwNBr% zr=$W(DyO50Mc%N4u!O#aGAAIas;H4&1q?8{wVtrle#&oF^-J=d%sc|_7LWb%0c`Gg z1ukzV=ud`x5f-GYj8jH7M8X(1;z(C_v79v&>7PR<4DbYaLtJ?Q`+fd|8GZ4;!{d{^ zyl@-DN51?%+fKn*!T8(Ku6=BaPVROT_SQli=%<_>N54)*OuG!$%%XLm*4)1~PFw)+ zGSGhw1MaRuTHs|lW+6V)6gc-d8(|e@!LHM*ZK%$6R!*mz zs$jFNHdAs_-w4Nm{~H7PKkL|kci{h2IBp6{0O?N$9#66YBLuvwUBC@!HroQZt{Z@G z>T|)gQz2%O&jwQ|l-BF6UM`W|2gjzQzR5Q3yz4^?XP(OVo2TT?rOK%W{U(H z_!QJ^Z?U^xi`H8M+b^_XV`Sy){{Cum|9G_jf|tGO>DGiqTJuMOMLUK^3rse5ok@}R z5&QZeNFvbhBL}=gB(%fsnN%*#{jEX4@1YT}ngN~%lR~dB9Px(1iiVfMwKRQCAP|aN z&mnKk#{K~ZL0$;qU+n)AiX;sjC=yG6b=My$p(vrj4`$km!ak+5>eKjbA-9@7dJ0)` z*)T!(FGf>Lyp&XVCI?-e+Qe`AnYvwUhdT+d9E~8AZVy1b)HrXvFhYN#j-20H)P9B5 z0%9YMh2w+NZZh}8xGg)wNE$X=Qpqv-0}9M{bm-|mw?|`vQ^{w4KzbV_JJdkSH1vW( z^}D!0vp$vQ)=)!K-}wAED`wJ?f;)(@lAYI&)%u8#^vQ67>k9>JR@~|1tz_V3ANP2xZhGF z$n?iNWk?5tETDjbml_dN*9$(k$c_m{fYe;aov&j?N>t~wA}Ivgo{oDTFk!agCAME? zyrwCCuk|eGo@fLdT^+*t0bRw!RV)=ejq4VMLLj+SgNI_6U&GyUdMWAP{_?%eKjIi# zOwwW@X8cCD-CFeef9Y`lpJpKbXG2vG+>pin@DaECQ4=RfGK2YVv}FEz3MPQ^kC|El z{%fkK&RN6!cXgU(~xZ1B}hhhFdOc034Fz79DJ=9g0#8ngw#S;;} z)Q+5YoJTIy^w7XC#VlIJBS`?9={Q1gT@4u8^QxQ5p5yiPC>xdb-1~ZwnxL05H8kkaI-)_-b`2z-#AWTfh(wc_AY4S7R_pQY{vS z7|qTVDJ_x2{h!~M1VWO&nzYGWOY>`0H9%_@q30S}V^^m2-fpAx%VyM<6|03xbwJ1e z9t1k>eHR-UUPflK$86-wtTW z28Yw@6pNY#YXXgGq?MMh+l;44%biF;4IRMVE6_svr3q_P-p>r?mXilEv5mq>qQ$Y} z0KX(I(F&oYAcZ&}WT1{-bP1*fn23GoOlUI(1Qqt2urnP3H*mlu{OxjxTf!SgaTNWG z%h@(Pro|b1pFWv!o;)eUj}yId8)5ocd>^Rj1_A2i!%sUTW_lJWn3Y**hKL={!Cg4Q zD9nKt0KL@ie`+hNEM|dbj!V5jyQjz=TuSeh#{SQdxkPWrVvbZr(DH`tKNG(HR^fm2 z2@mxDLXYKt1WG&~l#|TfMIiy85xKz;piIhl1o4uhCq4KRR{HjPaKLWZ#dpxCdFvA4 zLxXg09cG%q+{p)wHSm`{M;dA2#}qJsP538FFOb_4j_t#W0RC zjWo*0g?dVpe7${g@780#`+tpd3&6dTVT?lDbt=qM^6l3UW>450V5+w z{zx0(-0Zaq?Qsyc6a>yfVOQ`H13%^1#-b1%i2MfOuXf3EVGOBc1+5{+!XZ>kKnzH| zsJK6A?8c2N+>rOht9p&4e~)6YBA#Tndvlnu&^k{QH%YRkFVW<_Ym}c;;;M|X z`jd6iCe4o@7hJWRG#(0&CCrMOv?<{1U(wit4lL~a*DRg1(g!V6NF|gfasny|DuL`! zx>e)Wx~6f5l&rLHVzY zT;~lC($LMrcIDqC(A$IsFFCyQJ{6RX>$E{PXz#@aAW-(AV1?r1+8aC}-o+ug^!729 z$(|krAP3fJ$7D<{Z5!-r9EKkB&I9g{Ka7QO7R3B*QRXdcqxs-rz!dAiL zo!4g6DJ$?yo&@V@KTXntyQuaCL!M$Ld=(SwFtJbfw~Z6td@Gb zv!=a=w!MaWn6_PBv+kaEwC;kP;=|{=+vk?+crrbe&2DSruU^+nu$&L(uOYw**ug=N zts_yZgskicL^b}JZ61*)Nd`VBiFk;DgtW1&?1W?tVmS0LKV1pGYu95uQhD6U2)EP) zwZlkgJ(htEwL`rEF3AHTlvIb~o>B}5iMHqmwMIhxFMn{xm2?e9D5)=L30o2=o&yu& zfii<6VRb|mMBjk};;H~4;$8ryL=ts9aUroqoI+o*x>(YIdR%h6AL5X>f`Fu>qN*P# z7egtzc;X>T(!#1R42XsScel2HZZ2^(ar5YCx%zwg;lXs`f$JR_92^b_(9cj-ZwH1K(J^d05kH^}QhKkkv4k3>H&c>!65e{eq+)rc7KK|bVN zOoo><1Vo&COmsag9nmM!CjUr(#EF>ah=pSi31AZ4D}lspb&b0SzuX#Us`en>dS+h| z`+)+)Lr{RX5?_vT1fT?YhXL}6Me<7Rp((&yOgtT9wmLIi@6k0Q(tGS&5zlC`JJFV|K^~0QIkK$UCj1Z&`sXv)5~s1CXTd5`2KZ|MWgSq&N@>q$G;6 zACf$Ul(FNQNDftI_;4WM??#$bC9^=s*4(OucncRsE!418dCkdRajPwB8pLNF)OvPd$_Mvq0BgpubtMHfaOzcezBs7WZD=uCN+)8RGQ$4Nh(*1O9r+7WZF1Qft8YdTdRl*UXQc zcfh*+`vrlB&aYsz+FHr8kbBRr@N??f%)r^JbkTYSX=fsfq_W?pDLsbqpnJ3Nz?rK9 zHnaao;|{$8xPa3itG)}Wb-=^9)w&dWED25U6cSR)ZvcM9#im3VG3=1o^(>yG$x%Uj zHZCTr77&P#$GL*7Z4`FTE)^*t}J;z z=7L-T5~yeyMadFubDtm-yWmAm*OzxGN)&M60?2Vgbvo20PA0|2VoYBp))_H~qR=e{ zZH5qLMSgukHx!+7d}Z{_G7hZ{iaRBh;i^fRGShOR-!*_DD$N(sMF}GrAZHAAZ)!Z7 zZ0z-JBwd<95$caLgs&snz^r_Yvp{YQG5)PlZY+c|*(P5YaB#8uugr<OnNumhQXX+7Qoy}>FqakDJ6Rb-Z(0H+CRf7$sid@@f zFveknZOfBMB%(9sY3afW!LoMKs%faT!`C=7KT(hS%7ub39^XXL02Av+8Z~ji5k>`D zb{?f1eAitp+HJJoJqrR9okSD|} zPz5d7*t8&qLx?msDWEnauG|Whj3HVJ5z<2Xr+Ioarc-Z+V3lWx6ZVxa86f3aGe`Ec zur1JF?ERw6gc9jQ;F{AAF|~jNlMosd|C7j>3B4W*XT7RwNQGR>g0;eEv*Hrz1#mhZ zE&Zbss}Li+r551Qp(BVI4cP<}Ca4LX2<$XL3$8#qZxzIvNoV`p1F;T4AX!MHchO9f zI+~dZSeXtasw|D#W4`e0#C>?HZ?j~wmIPHMfA3AajfBEa%LNcjcrs^xdeK^kFe}2v z=AouUBadl<;GtBSa;K9cx+ckp13-xuMC>h{adpugv-)P|YOSBYK@R5# z(oCfzEnkWB6xlheL|daUhF%3UQje)4y^v#&KGE!ZDuIVm znLA_+=Nd~tTk1k91b)6UyV`LwSHmnsdT!-MuA?=4bf~d4(IebMFC`enS0kdQQxFAe z-ZR=Q4=Du-Q(B~zMw-VW5>0bCcUYyFeYUlrjDzyx!o5pjbXqwM0=o8KE#1Ex475j5 z=f?cH&t}g0F^w#Keh<;mw4D<9O?{^WAz41h40{+2ofZU*nD)Ik?Cc|HkBN47)Csep zT2vu}a+a*dF_G>E1J+Zr(i3Opt%i^;@DpmCu0`s$K2IIx2;pASdsxX-J z#$+bI$jZJPrp@ch0HLYU9$MVmy3S?|42|n-nd2*B(xZpYDyMSmA5LEfb(6e0Zo9Y) zstr}8ilcm&QR=VqwG3I$CZyiH$*$_j>C$FKhX)~UB*@vWL&-=Th4iS{)qzr^Zc7}f zH{|8M+9RAA>Z21hZ9u0TYZ`R4RajOQL`)tSd)(wuU=2Moc(kF@@( zYWKGvz-5u)%k6PUNckRCrVOk`6>q4_K^)t{E}T}kQz=kmqPePdOQ`jO6kP{W=Q2*3 zp9^)REfaL60GYJBQr%ju#wrJyNuA~!U8`9&+W<(E{ApGWw#yXgRBb^*i-#goHO8r$ z9BdpW6;jf&KtDFyr^Wh3hw$;ZiN#_bqtI8+QKZNt5S<~UN>jnpGM_oSw3=ijr}(v0 zeQ_&s$k*B3*tw8(v^mf^rG3dCl+`iR?ct5%1W#%-Ks_mZC_z+QR>e|5ZRVslL20Ts z@pNxNQhMhy$vmMMbG~$@gO5uE!!-qsaxGNEjP_Vg6ni&eZfY81d`d!w9qksR?ybX( zAkA92e*Voo7eN^ddwzAcW~of4)ZO~k*^*l6PVON8HiiK=MYE(-%#sRW-*BQir8ZI( zQ*z!JFj}a#d?x8oj(PD0)l{8vwetf39-ZZF3n$`nZq9gea{hNEkpDH8l<2S5LwB2C zy&mmqtrTTEm@gGM9GG!+c!J*~ctQEV zjI9=Wt0NKC3?m67=j@=EjX`3^7K#=QDI4Pepm0T~90uf2LIz6Kg+$6wD!U`2IndF= zK07FjREx%UoC@#kz};}c^1@Bq-+&}}p=8C+e`0qA>4t+biNVdBv%e4$T}e^Df}Zn^ z%o4ot`gWd=ys+?fhbBJs*s*(puM#+VrsVfOgk<0zo-mXp%_Brb^;L6l4xw+(VDP zMp6s}X;eW6&Pn;hr;q3eHlic5SmaqZ?Y3KeRz++M55R}%U4L0Z$qF(oIm+@qIaur$ zz`GNaki?ElQvIuQI!Zvg=!#XN)=szqU~)ahB8?oRj4l-}7ufwq$Lg%hobCq;Qwm^% z@|QcHDC%572o>aQ!8Ra-^kh+wE6R=X5+n=vr^dt@sv=sylgpB*$&|$&le31Ndn+;FFRp(A3!jXdRq)qn-OA%WLbD24@XM;lI&94>|iI7Ce#8GTZBwHhL#te)U z5)lMf=NJ;j;FUuiIrz#|n~mcEO3QJj1)ND#d!QVeQzr25vq>q4htisXJd_a-zy}!N z7!FrPCnR>QM<=SL8ng|nV{&K^5oznNFQi2(iHK)iWe_~&oGW*Cu!!t;V$mM2RB()| zVC-|#uDiZ26l8Qkexq+oG>gaIx&L5oC7wj@Im$g>sg@T+8kF*=Mr~sNGubv`VAwEu zg#~Q+Fon%Cd_s0f`OSHs`BD!i*f70x9GFG>*pc$i{v;hRVBpL{v+OeV%%i`OQ;k78 zSf>%~Zm1)pWav_z@1?Rhe59FHXMGzRqi#%sd7) zYD!-yfDLm>x*I=IjE7UiK*H+=v3(6S3rQ{LXffxte_GdW1?T7h@%-z=87{}G1cxrf zTP#eJ{NC&E!#ffa%K;6&D2S_zn!!CLbZn&FyOP3$rZg`qrO+BxKE!2~r2R*0M^fy@ z*UGa8XZ(aRbE#48eJat-mcff``ZKCX&lcN? z!-l{z8S%e;9?j{1?DQI1w2#?chV*p{x)ZB@&S2kZWdGHaCPk<;!ZZCo&U|T7$3eka zgb>k434p7nP1yui1BK3r%2)+4V$_ZFKywL(t5wo*T$6(st!5)>zLLhc@v5;vH5Hx= z7^Q-mOWlE-^g<$0;BML9UeX0Y0wTQZ<+~06G%*9N;&EAk)S~*+5|S%a_FSQp6BW^0 ztfelWn!UyG{;DG<3LFb0hE1=k6mwtxyhk%5hhoVfl9H2CY-w8ej(rPd)*sBbyltHb z(TRB7X!=XlWaz=flx36YO5;kOdM0>=a->3K%RJ=CwuLi(S5cc%Qz?}~xynUau8t&6 z4L)thKqN1Kr{@6T?(k-R8lv7qt4`_4*jPq?5$pt->WiXk{7uzC@sVF!TXl|1q?VA= z)mvWz4v>4WfQB3xm5*Ez?#*zYsFMYu3SOo{bW$coyo_Z-vI;ohn5!70pSb61n zJw#=J1gq!;;HyL^SYvKu@qks1X#Az^vgO`Ob!wxuOd~`R5 zL!Ipy2y@tRwziHPP2z6+%25F+MRIyVUCgCOL91i0&T^mDdlrRhdine?r%v;^RE+2< z!p?w{C-G;MejEz(*|?=v9X&IxX)$vt@SisaW=0meL*&jX1~WCX;MQ!L3js&fC`m?s zRf;#D^{S!p_@Q6bL~**^OqEnkvC+hdLEGu9&OoNDZY4$h*lJ3gG+shQ$S5tvqbz1> zI`?;eBdS4lq&B z)2@nCPKv}W{%g#z%aj-&c$(U}%aPr@=L}kAl=jk>XXHMGwT$TxYUbIOdK}!?6=z1U zf?g|@uaT=zQ<#8cnof!Hy*&Pg&^(hFRz|L z=Q%AEre$GB%FxoLmcEaCsUygD6`^cQvtlmCN-sClmX#q)K8o*}3(!5yPS19IliFu7 z>S;Cn2{1!%;3#jV4FE1~+dHoBhcE!N(W5LXf2hdO+l1Ua#d#{Gq#>qgo(zk=?uiqc zM$pKaRLqZI^oJ~ zEKrX--w@Q?v*hGT$)H{(qD|Xur)h+Z+LTEfxiK4dbsN%>dCqELWHe+>gUosxcaYDf z)LBs~sP4}K&!qEX<;rNX%yma}e;a8aW-;*rz&Z2+jvo7c|#qsX06> zK1_3Xa(zNQMA4p#+qlSt$0Bh}muHEVQX4BNMe|q%qp}+mVcwm8r4fZ=)&1oz4-{e zh#nmuWF985JHthu5uMA~-%yV*rG%faDuEMsZ5}SMy}_t9Wwu9W|DNbM-eA079U*Lv z_a5(HxH_HIstg|mICBoqzV8%ce?WauiRKzIiOKcJX`wiSZj%61?TgOH#kMCY9cvw- z%b(qp9X9@QmhEGI(1b63`?f}2oBUbd?lMwov{P3RqEUrofJ5y(wR(`5_9i;V>Scf9 zee;Rp{nfn=kHOV9E?Cfk8gJWybJ~J&3O>?gXbUzaG`3nEwv%0 z2kpmmc|h-1L9_?Ryl68G-F&U!^T*vd>+A5pNlL19D~h}ZZWAt1M;yB_#qjsRYumxi zT$SFbBkiLW6idUm%DFIrYmDCHs4q^)4F|$)(-_xzF^)tp1r#Z5}q3!)Sleh(p-kQie&P z2?PmAJe9ThejS0rKm>ccp{JsNt8O$n#5LMB-Yhp&~Shaa|HtCbinf z!s-6b5in_WRmo?;5ZF7HKXwk}FQEvvNx4vswKk$zK{y5xB10z1w;7|tVd#XEjz`I$ z35b)}vecRzB~y_n_QJ?5;OMXic4QzKxgn3wOlkqBuTX7(VfBRlCdDMR?xXZ=o_Jk{ zjigbf4x`ZaVf@Yt%)_xnNq0Y!)jyWJT7ECn$u=!rSL&5kw>>bC)7mM7oThM=(1NTs z1oITagU(GBuZCZs8Jpo#CahFCzlW_T!6s6*xFP}O&;)IuR{W)GQTbRycPiGP0>UL< zihTf{&L@OGX;<;}hndW&s*L+t;{@~WV=&eh>he?T@MG~42lImWm+vRp!iZ@LCGb@{ zs_b5;iI?=L7p+)s&Q5JuO?l^pn(n*2)~3XC8jQww7xihEqj4u4jl92lF6*YEX_IAt z?6A=qZ8I;orw?zdEf5DUs!3GtwNf5rO52ZM0FOlPtizB{RQbOrdDGwLMlqqkNs9n3 z#R6!O#r?VBkZkl}=ZH;~;S76W<+P6f2$x&MojMOIqu*D=Q0YJ-k)!NJhfVjJo?vFu zq==}F8%)7|kBf9?Yh>>q@g0Rj6``%)PqZTaMLZE>?#zV&bFhNcW1X%Le-J!^)8n5S z)Q=8-YNMWVnw%L-cKWn}-N)-am&yYed|bf4UMy7lRWD&XP?|$*4I!d%{&g}mQd*6XHnXg>X|gS@{ z|IWt`l>6NS+Y-UkZFtJ~v=s$-)kSRq@|`wJ?VDF0zUkBfpziW^mwpAxJks?!5q8lv z-qAJRRX5yKhy9E!w?ltsdEu%6=v0i?d_Chlsy$zs*j=dO0)w0Z}sq zOdDYdohD-I1!i%%0=Yf6>)Wzf^UbbP=>%qyJxQqM%6WR>v2J?fVXrqswtp1waQ-dd z9a9zhu4q`ygVzErl+%5JSFAyL^2Kxwe}%|61~wnmCY*C8q6Lpx{zc-}!+NK=B5@Va zdGKgY;sJC)4P0fH# zR+vrh;bBcUgMGGoXSs5BKRp<3&;IVFXjcz0czo5K;{$+3_+s}W1LYffDQpTUFF)PW zvv04FbA30w16h$3NR{nZp5;{#SrI)N=!NvSqt#;74dZ{%PG|@bBOv_7chiklxBY!X z`UD7w=?>f`D1(V9ud!n~n4bAnqc>0aiK}b=*b(Bv_D))2;>KyJWHWTDcR9Oh?^BMT zW`}$G{s=$-2DflP?nfP?KHQzJ0iaaX!tepr8Bk5RzhG-i?dCoOAp{DO6|sX6OrIhc-9X&1_+eP3-f1@s;l43-~>SYzzrMxC%Si+td`RYkDP(aQF0=x zs;@S>aWK#LV!k{#`wDHamqj7gHqUyspEl;fj!AMB|EZxd-o^-&jOzovWErJaV$&B^ zn6WR`#v1c!SK28t6sp)1>or#ey|${aCbmQCum4?j^l7{wX!yO=A} zVZakdWHIQwWKm&RFDdo61n@C0#(*u8iWeRuqV7`8FX>ca_Rsw)J7@@ONRA_|JAh4TOM$ zM60ybdk||J;m&vA><8);xhB{UCi@yVw%kK#_0Y!YpsUdj?;H>83t!%0qaESD1~#Xn z+2Ity@6_(?b3Na`Tz>S6exI>7#%D2iqT-k?vfFg2C-C(b{!O)X)*M}uN{IE_2the>`=6M0rSC|`* z&=0Uruvdof&Zy6&-wUtR(0f@_83DE#x5(OoofWi^S%G`}XjIdC3nbkMkCdi=5iE1c ztEmlafOJ+=QqOQsbbbOikLTmtiR0Vf-JY*Kf4%f}eQg@mfBL%zl!4vuRAdJDi-Eom z=I;_~&(d*0tb@I)<@(?`XZ``mUKq0Ou(Xes-e-%(UHk76b?TTy{fX-eBE!Gbx&=1& zrh5i%d(+*87ky|3si|M0H1ZCfnAIj;8x(rNkET-~QPam5K!9n1TB?4Pkc zTHhu^A-a4;pH=Y;kmsiUHb+Z*4`qjsBuCBwf?diQ?+jNDOwRg^=xHRt+fFl(P3}sR zQMND9+fGpQn5^)#{o4Q2`t^zv%+5e^md=LN)+o}>hUF?{bF$112_;zO-t|vOEf@yJ zQkR>xCs8iDa<{DTR|(a%oir~*m+ zIwtaEMR%iMhct&wHHuQN$o}GzpTyc+D=Z}^f1fEgwzZ9G-?CgEk*$qrD#u=jLECTm zy{{=d^TcpZ1p z*eA(ZzUM0^TeP52%1@b=vQes6LghCG+KSfi;`J~_P0>y3xRy->bJ30Kyj#}iDj7j? z4On=#EPG#G1)J_|t1EQva8HkAvi%sfW^7L%E>>Sm^lp#`<}Al4Co$tZI>|9=GQR_Y(R_7)@*VnksMH-G;h^5VFZ7t>lfahm9|LqT zDgD-zzI0}P3>BPMZa}Un$PvS{0FcFB6vbx0t)+1S*y?|r8=%#LACdN4ie6xIAEpIw zjrMRIuvdXxyZtwy172i~x2o=#TcFuuA$C2R=;E9H1PARg&0w+ZZbmUKd_g$Zy(_!k zC;q(u%s$@%H`HJ7S!s9#YQ18k^j;AW{llY?HxQ10a7j7c1JHM&Jr&F(exdE7s*_0B zM+NN03;!b%N6i}e_e=o|#ph43vmF!NGmoy|4QrHVHnM-?aYyat>xI)5vfHPoP^L$^ z?Oz>(*Rwn#w{w1gb^|lk+dNbL3hT+VU5Io8cS7O?@ITuDTY&zShPqMm#QGrfhWUu= z!u1i6>#P1roa;UX+hu#mPvL|97E;@7eqg%cdeeB}`6TQR|3>}d@0J1owMc&XKJ~KlZEhg}ovl)f-12kk+Xg68j@k zwQBDHNY%tY4k$+}jkAdbtCFbWKjR4dnBk8W!JWYRIv@CxEuD&r!vZm>F+O|DyYueMcZQHiHrfs9zwr$(Cra5gJ z({p>C=eytg-n-WM5i8HBTD49^WJX5D-aCa&LhBJWA%07)zMflB&JAm9;*tVlt*emA zn}d3c@T)0c4N6k<5k)E4H*+zKU#!@W==9R{fTkWvx5#_vtJR9Kh8J!%iZ6daI_IX1 z@YV@hL9TA8q&hN2%Bw;R>VZ`~(SE^I9_2_ioR^}y*cjXDMl3C_SatA`K}(P>_B(jp zxfnuQ7XH_jK87*9WiiBlhQ)08cw#5$o8JLEf{6Qo&m0p37mE?QeUzRx;`KxRc$R(+ znJa*t9A-g#Op(q}B$p%6SYw^x1%G)_LJ-&pH7xpyZlYsFMtZImvq{Apqe$hoa1i7I zx!-&=o3SiO$=vLPXzWnnk{M+r8A9uWHJm;3HD!ZY=z~7xU|!rIG|EsC1ehl>vSyoe z94-$qAX~`XJ$ZybPHb`D*%-dD!5e7^+Yim(G;c%07E3?4Htxuj(J>$H@53@c^7_<& z#Qv!7KxOs>X=m~!tmcg!Hp9a_6bU5cBNiF=bNWV2seHV|kL0p(=a8m`$e)lsqPpb> zLKOm~6$Q%FytqU6se{|{mHQQ{bYiSAb21KaWLTn>SvEpK*#Ku0e|B+FXMDl~Iqr)^ zg5G5>Ms*Wis*e6$z`n(~4 zb(|RVfGQQ;zsga(3-r2Lm|6mK5km0+zY|0i-lJg3kY!={1Uz&O=i-`;wX&_*0t8y;dGbiuddy+;Pmo+?7W~$%!BaG9RgLGXj3`x&gnsEx+z4%K z>7dtN+8(DeJ3zT3xD<>CV8Me~PU2z%`gy^gZhja(zn{XJFiYr5ZC`M~LVhcaC7)~||OVdciXls9 zal-qE{&q+X688it@vvk_WofMK=c zXlyGMUKxOgf7TB_LAOx#GISS}k{*i^ zqw)e|jWqofM(YOZE}}j4Z4?`5PK7R^lywd#Tnba* zo0K)mGDa+k;4d4@oN<;te%yPJ(P(%V-s8t4rXYYiSK_#6v$;gQ`;pXrtVGDzUFKU? zPiKy~tsW8hb5bDYctwemZQzf{3?iSohD}l^Y4`zI&s@*-3#>G*cdX4{Ex<=VZ!=0lG?%Zxug@10=G6+`VjGpP&L6P;I>e)x^x51ii2U1 z71#i`-G1lVy*(JUGPtxXGb(+V5W|mzHKr&e_MGKRLz;b!5SZ<=vTn8lL;$1Pe7MP| zx~)GgFbxWqImcR`s{Lr`cd)O_!XlD?pJ264^UxOwCDY@XxPp}-Hmxo>t*mpNF(um@ z?8;`BD4h$ITcX5XR9|}dOd_g8zBC&KLv+BtK)$9#`(dj;OhDa_4)jEP#kU}x4>(w^ zbInGd0JTQzY56G5p6RdcC1DJncBtD`BIcCY8--iMT>`fZm*kt5iOE(-Ov$5`lVr?D z%#n2mbYi9w+5f2a{m0$@e@B#qS0C|4{PJamx@C?4e-!cG&gkDF__d(_%)H&+r8|G2 z!;=3vYT}&^3Z`&|KnDok)i*Vbjp*Q!kr)`2DkJ5QL|k1wK&1pZS;W=A$jFf%<;%zZ z8ON8{o6;Z7t=1t+B)6sOt)Xw2%XEu$i#X@TjHNNcY^W&a8Yj?)Qq>RPXGpx%o;%GH zk4xb5CZlJ0VdY5Xy+J2XJT3)P4jqcuMyR;gDRDyfEa!_#b7kHuNH|XPOO$y&vv zJf}C;O)GFzfQD7YZY+GSJf<7gG!`wKOx=7KZGSA$%>E@->Lp4tU%_b4W6L}f>F8mQ zl|gl0WzkB}omZ(>M{K9EEw{>j2a<1p_-@R#SfQqCS!v%*-lg9c8i%x@vB11{cV>9&Kbi=_i0>e{yx!n7*@sjaa@t*_woerL=U zPkuwuk!7v7{Dteef+oljm*K_XNL$yUNSsj+!KPlDdfr?5c<>7Icl!H}@BFvd{cXLU z0Dp&E`10k;k564l+5hdg2VP5HU;)TfoLrq`EQ=E5B9Y7LNY(DpnS~6b zT}@tnyg~I6wDU3exIpiQ#=pEj1as{MU3=U$CSKhtTQCGQNmYuq><0s=ZExtBWY&1# zKUk0hIBBOsO08G0G-*u;D6P9Ke>fa!5HNfWq{|EjsFNi^& z2KMF4Bh;5K2Ee{@Qbu4;4m&2m)XB}%N!ZlM#nQ~u*wDpP*wDtt$k5pO-@l_tRoWR( z6!}A}Ci)Np>d4=O5T%c}9@W|ujj1jWgqFRTbcEQc2zJo4F-{?9*O3^27dQ&VyJ090lVx0h55yIpkK zBbok4^{r7a#v0f|7(S^(Bz)hl>-^>6>0qL$YA!sigxw&fSMYz{s-RzJ5u7yXU}zW~ zYh~yjE(yE8==VYR?{8SS3wza91*rbCEfdJS)j71*;#2Y`-SQ`_GO6awMx%%IQRE@H zt;JSu0asG(DFPq10yH{yxwn)B(670S;nch|yVy(8T=N(d=f|MIqC>Ku(X1*8szSs& zZ`4_K(df2NU)X{;Pu+POjx(t&KHob0v@74BQOK?;_nV1SIkW8G(|pPw`vbbrc8X`( z7jYI{BxWobn$m2_NOq-i5wjVeLfh5cv;^~B<3gt*H4g?-2aq$@tgYIr)>@BlYzT*x z&PmMUa8`ww$ex04vICDX=hi*Bm7MIk~47D}FpPT?E z4U@@9#sz6b0bqB~&21!sS)~#2BUVu)KBcf)af#SI-dh}w_Z$V1+8X{H z@99qkSLk{8&x|SV@fh&&HxI-r@NAc=(MOYmbE-21yoLd+%d(&gU>L9BBE9>iM4;43 zbAFHW&_fy4S=OyX;H%~RyE^>=6W;iz?f7K36nBU zT&Uhbt32%StL@$=jXOxeu=6{Q98yv5N^Vhl7dv;m#|~GiIR4pVe;!g$%ypw*fI5!O zYW&8jK%4WB`EhPP(J=T39!w13QH=g&K`o66tgJFyTHcS8zA3-VivlGOpNQ5~Ng|A3}}u3X^vddn-`-=U2yH zC`yu(C6bhPC!rF6=uMwr+}Qggw`tlqxzwET&0a5I4G zU+Q~*Lfo&pi*L-&*GP&ev3Ky2rY$CYi6%IOdvrd9d4n4q!A(1(CB@8dePZ0^Ipr_G zQ2`4c>jKQC2IB%_y%5k~Sn{Y)N#=i8*lDMgAv`ZNZ^d5XkC^@?W%%#N{(m=DEgQ3> z1|a{e@sIkrT~43jUjPN%B>=+$GL0hQ0p*I=p#ZG^MtYU%Px}Qm^bfnc0ZA(~xw>EJ5QRO3T6UYH_z=QMW7<1%`uvwklVhW3AK z|1G~T?W&wcOobkIewTH%>D}Xf*|xF*_~ZSD_=hANrUPh!XOsb*{I;av2ze*v=vzNP z23nb62h4}cOZL~;o5ZZ+t}Kjm`HuxoQZ4@JL=O7n)@weLUzJQOsC>!SIDvVr_8svH zWoy)*BfrBR<*aA|bhYnvMq)O5YeEIlIsMgP-tjIY>P2y!SYnP| z?ObAr*}gaT##MvcN}0C^N~U+!qCh@%)Xw;oOd4w~;RM&$Cp=zy4CO4H05F z4o0kL&;LEzESlPC`Lo`VVamNT_w1FHjnFlqIY<-c$aGRL(ecZhsfwfEFi)uvfSkGT zeCMA@}%Wx1aT+SG@kT^D`07XpTj*l?W6R!ZaBX}?YYc^>h)*0;y-<_V2B zN72-ZejTtS!*PUhN6Crb2`OcThZj#S5rWIUIiVOcmu{ZGywpN#`kx1KgG_DChA{d} znVzSuSnK0Lh^S7b%&EMeJgIK_S@w=CchwpKA;Ag>prX@YMcs z6QGK-oh6IMb=d;X2Q$@-j{JNi1@AkkZ()erN&R;2;bEnhbc@WAfr9=XC(lbb5-(0E zAInJ=wDNdh@sgHR^P-q^hH+^p=9G>q$HT8qWc*8M#*?P@JoWn&$BVjLfEyc(-1#{V zI?|l#8>a5hvOT^VYS5q}K0;(@DRBFJf`0dKpHOtqUj3(DLO!Z@e}C7TLiRX0?H&?e z=3>qKv>+*w2AC<7WFs%67mdPNhU`9sQEd_tca$kImoSTFek*sL@>Ygy4tAzawuocG z&ktOZr^_B^h;Wp2M+zPQ>VTLzdBH?Oj{Ru&Slpr>xHP(QIH@jZx|Hvz1eK^>tQJ?cksK&t#F7%n8o1~71Orl;kaUc*_`kb{>={l6P^Em7S*yg zhe!qTPbx4P(E7&|;>#Bs1YjNkGYLS&#@UUTMT3orOVq=~)Xv1zeF$|~$(G*sv5U4WqX1C}LdgilMg$WWqvl1P`Wa*ipAu8R(Gj!a%qW&b=AE2n>ZjOZ1H>VppXGm60G- z8%Pbwp$9!TA~y^=snfB*+JvkoJJmf4tTf=17;t3@;yK}8BCduUImQwnVBzeobqr@R z=eV`zJOX&QecvX-ozpyXxIMFH+DvDN*zpT`xAa1O@lDjdN5YqdB%>O`$NMW~_qLda z4?H+ICUkHa&le{?+KjQDGY>Qk`Z{3Wl+f=j1DmA!JO6y;wdM58=s>kFN#SlcKeDl} zK3b$uA$`&{P7$Eo;bcwQdZU4Nlo_-(@oGpifa{>(@lsHo+5L&vq)Zp6CT%Q~6n| zQUh$=p`+7ox?WBz5~#FRkmx4U@apC?T0uN5Q=SZ=8TEnv=FNbnR(U3LG+gaalhXHG zuINQ6y8E`P1LO&A3XY-i%l=6dz_h5h|2yjkE;)eM_k@MgpA!f{T3Dh4R$7q&NNkPjUgct_G3+?2o;C#L~#;zTP5F$ z^E>*|=V{7yS-=ddPYk04Xtzj|7x+XdF2i2J*N2#|q7z?3un{@fk4n>*$E;=Ltos3N z-UVv;J|m({!xi00V;7Bw&zr9B+`LwJA1Y(?db#+-)&oL*!yN zl-pGuJi0@OJaOe&ml5KhgACfN1E-5@lv@x`8#!zzugmRqZ(M&Vnuf(u7P6kEyeDXq zByiK1sLf0_zp^loUM3?b;$_|7M#lrjIyMW}WP0F8@sLLU+~f+eGKZkuL(YlX)3U0G zSTD$lN-T0^bqczxpD>WgB}yBNEw(`W%DNM~EZ_>KF0+!D6hZKEinw#D7dm2#e`!Ng z!^saLMOl4^m{M|fz}anHu9+oz0g1o4{+52f>l-_p#j%WS5Hr^JG0iD;RPRP&sYFbgnDxE9n!9?CfG)jZEHfypd+_$K)@_me!q zFQc=QWQ}qRh`3H3Pm;3UyUjUtT; zSxD~FC5lPvVk_$xsBn5*Cu#!#pa`g@`vI$0%CE_^5cFv07u)N{aPsXIC6#H2xaqf_{#JNRtGhMA?$rxfI#4 zqjPY_=_unl<^4d|xb}`7sVo1Vi@7}HgAHUGy{?Gx21)4st_a!E2`Ws}3pIs+3I(gK zgz$o~e4&SY-5+Bt{jwsr3}oG^hJA2VJN5?2kY@`N^7kcplT$*q5Dg6zW}SUr#4O!R zj`Z1jgp2{N$P@?hvbBfmvTWFg`Ft&rcbVEC%2V3n=h`vSmX6r}{0-M^bxQp^$WnJ*zD3wCdQt&a<&pVham; z8{HdD%toJuB#OoUw%@fQuHrCw!*#IyVX}t7F(O-J>*3SmF|$<~O9=Fjn}hf8hX!a< zOiE?!Ezb-{7%N+~f|u2`2LvB*`j+f2f=~@&L z7aDQ6FiI>U3G2JntcfSvu zNyAJjysDFj)J2*adANpaz%PBmUcwV;lEAg6R9DwyvsU$*3jJN*tPcv*&SAyL4X~>+ zX{07Jm0XGyZJE&j^h$3&lo~d_5DEi|;@c+7rN(wHoRY(mr;?f+_4ExR^h7`w#qaH$ z?v;luFlOnR4liRn*Z8$QGechTA#5NtM(yfZd0SR!JTDU;qslJ`Yj_#6hVj_+z^?fvd5=Tm|7almofI0zRWkKLTq8E|;>7jIE72 zI^9Nl&!^CU-#}Q+n6aJTZKJGI+7@ERc#=*2Q@h`$*6amTxZxwjxY58&{oV{<;01ch zJHp(>rTyfaD5bt$>r&ZqW6cZ_+!^Z5y>0Pv<&!6i4<#E9-xH~d`rO`#wM(EMa)zCE zlqQ$Ud%x*A!BH}@UNIS3UkWeAY~&7FP!HZDmcGD@{@T+x0 zD1$DWu^y=Td>h=Rq-wd_qFE>a*WZrEkUF2nk%kiwbOteDP7c4AqKjp@gXY8FrOp!k zs%Q_k8@YAOZbymIh{kGsazV091}5{OSEyGa`tU-s;!-`$S1HB5HP1ZtE+uowp8)og zYBiz&J7oe0YejQeFwGz*6A&>#Rf^pQ=D$Gg=ea-z9}o~qgL$1C0f`BKfc%Ld9uQ08 zLFRDsg=3bXmouYNXrm-CF-wbdn}u9+b#99ZGw>dN%dX8+1jk;{c#1W9aVwW_-KQ&M zvD6!uvsqno$0xute@{Ydzu~zXXYwqCz_+{_h2gjt<#1m|rSK!4iD&L?IM@UyJ|l-^8!-~yXdV(u$Jo5_PEkIZPMqm)cC=wl^bo7B z1ts)hdY#V90Z6(1NIJrOAQ^`#+2=-hh0Z9o7;*4peWW*XsNMj<^EoHGTyncBVJq-X zC_h;^CtSnWMKcQsd~KcS_jveX6e6h%qjco*N=)kp^s|#a)xycrARzYbFX-V5CzlE(9d!vk!T_zc*oZJV~l9L+f%l`8j$xzV$8^8Y# zFVx>Pv#@y-3FH3PxO_caRUN{VgSt zjo_elSvRxGEa&O%HWv5Cqna!JFOvq;FbGem#{(XyPt5VXt}+x3c*?kh79OR$D8fFH z9zygiE^;z<%!4wT2k^W&yEitUX!bWx^AU1+eKAaY_lX-(iy*K90a3P8@6I)HtOa=`U*e)n(*twsO+w{GNu8S^Lf3# zWDrx2z>0c0tJJx80i_YTY>U)|8+5SmziwnkK(kpUn_T9xz2w~%*VhVyndn5Mmda2( z0i(AGD*cLv36@QF^n~&uO^=z7AvWGEafZ#o=krBzP4$-JM*wu2KawEn^9TiGoEY{2 zSL%y~*o>;FZFZV&Uf%l;Ze~l<_DAehou*niArFsB>OM=X3vE2Y)xRI&zvqw?qm;a5 zR*SZ8b{rSAe?0?@hCkIj&U%G?OoJqt1(ZdvWZFB%YyF1)A$?qwz5A_td93U^*CPZ$ z4sPq!$KLRV!yB_eDWLV|?AoX=Q6E92VRgJcG)G1ck@ey;^2=}S)`iIns{_BN9k)F@ z1h#;=boZ!rLN9R(eDP1Cv-wUoLkI_si0{KJud-=z2HrbfJql5XUbsTa3+;Z;8^q#)ldcQfUK?3Ov ztU{ODk0u+wHcRMiA)BHamAY+aHCIA^sD|gvAiDzaO0vvz1N-5(fgID6V{ZcUQTGgm zOp!>3(UFq343g>b+nKEG0iF;*969UdBNCTW>BkrfpQ(Mr=OVS1uJj48syTU6P5j@2oR-Il^LNgmv04kplGDNRl*DR z)$7=nanh)(*Ouo;s6wb8-ls>9I<-f*V&G}0JIKoEt2=&*F$c(EyFbUe|yWZwJ^tC%opb!YjZ2;!-GGuOE0R*SmX#9+ZKW?#btRL=2g6Ezuc+5U^fMj zG$=3NiRDCKmA;vcO|oPHAYq1(toj?aj{WmjXsYo{txP79mVKnQvKjkCW}qu#Bkqxl zbP-S6bh>p6srUgGU=UsTcp2^S`gufD_eDK3nyjy@+>yS(V-#koKjU|aN>lU2er-~& zR9g2Ma_6+~SAja-=W^|eh1@DM>Jb2PWe#jP9>Lc71bsdFY6Zx&k`1j_PT%6;6KQPu z*s%r3uRQa^1-3jCC`srP4w={B@f%_BQ16dl&UdfM-mq5_NtcCq^x7kzjYOcrDcV(t zzhloqVMnNuKOIOkIpj7(luEJAA#!5}chypku8#=aW@m*+@Uum8u9&3i7@aXi0GS6s*J#}1 zn8ZGMvB?0FEW7v$*^6;?JSPkM#;&%_b_kZ>)e5Tpv%dch$+|msYFz}Hg&~e}cLF9BW zUk$(c`^N=IhS80$rnO1phFsQlP$DaCIu2nFmEHLy~vSRrPk09$^z z!#=EY{c|)+-klb3J9g&}#zc?K_}9Wiy!cP!qEK9Ww|+T9xa(Y~PI32v!VWIaia~Nj z?YR>Z#GYJqScRN14%o0h{l|+4b<)zb8q2vE#KN3%@}<%&3DkzcHryCiQg8)Ba-w3G z%T!dWrFiWPnu4Au;7>_S)Yn0uIw1;7RsJr~X`~nZR!i&3h151oXjf4ZJ`F)zyUI@U*oXLa?>)BZbX{<;`H=c zn350;PMtOv^^*mtN?P$bv)|YaGsH`&6x<|afyE1Xp6IxS)ux#YX|Z?#WW|LL#IknC z)6$4hhx^;*;|N^XbJU$!#)53xvyn7vs zozTF|g*;Ul^R2APV{E^@GcwX|db44vFfZM2?l;dq+QjtbR3TM+Zgd9@Dkig2jw(aU z_oY+=>=;NptpRDqg^ELg9uj>dz?6A$r~K7(Dmfk3_Euq1@dSKIfC^8PI5aIr>>e{j z@;+}Rmtw3~$(|!mp23hTHdi}Ol{QIQ!WqGlB~l%NiHy=m0sv7yCGakosO=TQlf)k71qEf z#uH#3@Na9BXC=?1+>^GnUT($;_HA#9vsCEH#^0JmgjU_gPRFGnXr-Mi&G30$*^;Rn z66st*@&_UyaXY#Sjg;_8zf+2O7cba0a3tk1OLtC+$DdDa0H|d;q`B#w=GjW_i@kmN z>t$#+a;PiN&pLNkN$WIt@WY#iTUMF}Nxd@)>LjuPAXaAB*_(!+M78#9n-CP!8M`EN zr>Lh+n5sx!19IhH1Lx7=_)z9-S4P%W@M1*Q)7{2NyOJgynti7zjz{bq__SF2e?O2` zQ2dnSkg@xz2AEn#BOQUP#<@`(wW2tYiNRTesR2uk-BU^mT+JR%`6a=rMHId08F9kq z1kz=y#0|6!Ddc&@+DDwHBPk<{^83;A&XaU|cooe^_96}o#_6_aC*(qnha{jtV+`Ja?OxdZ)JQ# zo#D|?4^R`iCDb2XAD|)m6IE}#S6f3C6^Mg9wSc^w8$l93 zuT|-+$rSXkD$qvL`?Xv#&?n*!pWO(}l6}Dk;a#2T95kkb2Va}himTZvD9>S0cNn!D z5|o8;0>br=0kwEg0pB$iC}7~o-^V`>tR;~cX{$galYpVg%?M}fC;{&7q`|CgBoafU z;+um1I~{d@q;|l+`WpYeO#CPLs}tUo)3MAz{#%A90HUmdVFMLdnBjmr)f^Ztn74@I zVE>$I)YEt~D1ELxau`4moo`q`;`Tc@q(UlkRnnBB z-S7>fuv8MSBfwa4wH*nU)pn<^Dw1995Yr+@RIs3?59ae|gxH)Jm7*TAN901}J@dQUd4@N(g8%o`3K7Whun_^fAYfnY$Sfpd z<~1HpAqQ!P9orB6y%kx2AWBQ;z%M03y6GAJDF-PRQ-8-+-r-J^zZ+cuF8g3iIqKHb zF8kLnT0x#b^R^cnjv2>0HoNZWsXM{WA6S}~MJmD^KRRLSXaZqq6&5sroH zljJM{xp)&3@95YVa_u6Xq9k7KK%2>CB(VxpX)M!L6$BRwB~|%Ia>vkdlA0RxP4~P6 zS-@Ej#%nvNjnxFm%8r?3*`uWERMbdZmozp{;0*Yyr!qL~^EwE`rGm z#lJw^v~khlMWf?i%Y+yxt;H&94A7k_rM7Al=E$<4p=&6;S-qkLH{>24A2Q`)WbU&G zWGME+5EcLKBMI9C~m98JDmJ_L-o2}0L5!Gvt z#6`aG;AnOcWW#MWMC^r&FHl7|Xz80tozXsh!)KBuhmAjC$`NMb{I}PVcm^sx`Mbqc zlv5tG&#TTU9eusTk`ca3*B(^-cq=!faV!p{m>Hvv{49XY)B@&~=@M)YF`D+2h!e>f zI{>66q-&Xeb1u3IQPgE|CCWCg$zQb#lX*O2#b}@<)MnHrV@Qm&#>+tt?iN@OoHUWckh@U87soXk~zm6Aaitv^A3P_8&? z`sE|gBO&d{eGNEo-AG7iOsY%?H5QO6qH5px&dm+6NzmqLZ&$)|kaj^w;%bX+*{PCc z2l8a|h0O3|LG=@_?n}59#Ea1f@5&|^jh!Bb3+?ykx`L+!Rw$bNeO2ZKG3xj+b z#4XQ^qKLv;3Eaax(0`KX*HWKXpkdkw)7g+OHR*U%NI%hy`RvB+* zRw@P8%9}vfavYIU$@Wcmo`HtzjBW+D{UIDx)THXGq>U>n{+2=ELO0kF1eWJ?QbOXs zohdDDTR8HT4kYQ0#bA-2D$;8IkFLXqbC1C;IaNR`UX4y(=D4zA{}_y)x4(UNJtxd% zX``5@=$JM}#2su`JQe>HLtVc1LwO>j!X~AwZ^*ZJJjbi??}}HIol@D^j52bxCUe3l zEgaS`E<>HzT9me^{;J3?Z0y^7F~0XqJ<%ULi}`5{FwyB;d8xdnkf*$vv{T&+@0YuD zR*!%rr4ugC7OrlnDozn6P(IV?t08t=<@~}&Dqk*e#^@jI7HBxGFlR8lUcw5v-j8Mp>YveWNkV#;-Zig5V&nW z`C_%4^%~be+U<8>>NgV)B;@0@`DCF0A=PJ^?X^b|K&(-2J(@Y++0Sd4*DiaYh4K8U z>ibAZmNS{n*&@qZOpIwGET=N7QXs&SnlUk6kC@V@DX4OrxWb)T1r=!>oOF>zOn9;7 zi~UHS>G^%7X)0;R3iQbXs=9>(Hm(gMA;N4PZ$&HQ%{4B=9n&C&({8(rjy=lC2FC$z zzXQ$+w4D0E`e8qtyv`N!O2_A5k%sP+s~bV}7K6?e+6vU5t0|a{ag1=GEg%#+p79)| zHmVDBE9xZE@U?Nq4QWC=1Ln(px@9kk)-~-p*-L(85dN(~IKR@owdN4-C1&}c8ew@) z-kq$cR*?2pW5i3A}{xtYMbK2J+7)YTEl->ngDq!H|nf=_T)UW zi1LXAAI6BCHYGJoWsExFRP9b1w)d&+5mghbSAC#Cw`jk!O2z zZ-bOCR}WWzyD=S(SnN)BM1dWQDH>uVyVe3wmU=ATiEKsy(&amd8GvbuwQY^ak+Mb& zGUkSSkc?-oZD$!(V?Q39fckJ6lsr}^HmMGqN3|BsVj(kIJ|()r%>AvF8uVSs5@_?* zVH5?-h)B$AT2)I&#G@69IYBMx%Z;tZ)(IEnrKFtV?*oG3jM~lwfBvE1|4#z)zuR9F zXy`BRf}3)RUIaAia9_UUVggZku?c_z99%HKo@x%P&mFL*i;gPJpRF{GJF{X&>5{ad zF}*z2EHVUe=!B#%Cql}~TIt2)5VrbedI{o`ERMg5$ihgF6;uTLVd+tgAe3q)1nULz zLYAvGZO`Jw?4|cWr=mw@b+eL?}2s1ARc+ORBY?18li1D~zj@<~|%2qE1 zKsR@brrbSR6Z?ca+ap5_3A-l_p02NXi7tnEX)aAG<2_d+-=1H?BHpsp<>chPJ-_*w z5*hRNRuD0r3vOSk-^4|P`j!q(5x&PAiF;r_01Tpz9FZtI{1!uz-Mp@`^}p}*ea|L( z+JQluzS;eXSmvSBi^ZI87Hf$UUMR3TL~qtfVWzPxASqF42EYhe>TjHM@$S#s%Lb**RI9I;P>ZggFqOG|`Y3nbcSr_UI%hJWC)0 z2fQygUZ4+cmg2Ca-xs228V2p76t}RFDz*?tib#_2!K^gOINbY5z)Ui$eZ2{1vr<+c9Er6M%gcXGsogfnoCE zY4+`c;Bf%IkxDB_?hoR)FsvBM+WHDg%&FwJriK9dEfh;p|7Z*x8pC?u$*^TQ`Kb!! zYz@L#O|hOOCfb%1=?4NY*&I_bYOjQp#X+3rR|r0KTaZwk0___<*vri{aEQgERsc#2 zi6{1wu`b)$cLuh8pUgjVv$)4uur-NhaawNY7sHSk{3Li)=SX;6DLAp@s_&^r6-(z4 zCu;fur7nMd+Pdg)t=ifC;jSPbX_)PtK7r^XM+^Nmt+F(F$!z>>p%{6SI=w70b`f<3 zy*#3X%D0VdPNhyPF9H7ap*|lM{ z!X9kGo$hPHofb|EIppaa_38=&QUh7v(fQFhAJEiY5C|knpW8yzuW3A8)6-2t7{s1Y zs~2knaJoZCan<7o^6N|G1LVLwH6BS7b@Y~(eGLlf z>(YqwDRvxzBm#R)8z5$7()GMTOzXO&Uf0e{V|K(43HEv>h|F=Ae-K4tK} z)HaztlGa_VbpzU7pGZDEr7iICVBzdi<9B*pT=v++aePhx%qQ~8j{(G~nC9!zp_7|~ z-?(RyuUvBAFB@nYvMJW;ye(X+dmG44BwLXc{vNUCJ`3xs1psmxtpsjPsNxo#aCr|@ zpgOfCuEI6wD9(kk@uay=GzrAMv{a*vFw~f0x5~vfgA8AaxERKJA*f4UcicTtta>vFUGCB7fBvE~Ez(&qC=Nefo4w~($w zhG~`LG@xQdw4Vd_5j@m1A?WRN>tT zee8XI6i@UknOHo3Kr(T==tpjJ!!(A&oE;KPpcJ52Rr?CDol*hW3;YGV1i^d4*4uiP zT#Q2Bl*C_Qj!F(1BNjC@K@q)vTdwR0@?efrX#J9W>q;2&QXJ$lo`tB_=v0jj+n0PhTzKBEVhnbRDxdpojO2n7B4f-C z9~&TSnSoej`#q`oMBzJANTDl+j>wr&E2ShIFv5*ED1u&KT2eeO{7JDO?&=uA;8UL} zzrTee6tVp>V4{ujpwbrRQal3RJSV;$^QHA_l-z ztGh>Ty<5=mmg2VT&iuCP>%}sth=Ig3#Bq|=O?hO0pQO99!JblnBQHHsj_AobNm|n$ z)vYRQP3i$HvUrWm{LDg3wqhem9)UVqckFysCJFyFe7%R!4c6sT&MMU@hFqAJ`mAg1 z^hKpIm{o5VR9-G^?((vCc|bNyrV8K)!Q)E6u2;Jrs{2O^^KqFvq^eXZX(fOk*qQ$w zYfiHfPL+7POH`E{WTlxovoq~SOt#0WmR(L^kZr=80RffG&n&2fjCvw zhUJvRMzL|^?~_*Sd>61-7C`V-iAu(AJJ_O zpx1cMSRheC4TT|$|3omDIxJ;18mt|%-6mOQtL)@J?W{x1Xun>3B-D^k6byehy61f{ zW3Mzx*op}8HtNuG0aYJ&K04qJx+y2kc(6DfDu`n}6$Zw)1TX3m+)oIU3T7vnCNW(E z5fX#A0K0B-L@bhTeW6X!Cl9goxZyk#M1e=@C%OBYqC~}n3Hq=2q~9(B;h+iL0&WB` zZQm{KDC@oumNU>WloXo?r96mq7%`L@Ge_EM0(a z$)Y-@v?P1l69`R#1g#_G*8AeCew;GXnfAow4Fh1SCWal#dr~lsIc@Pvh|&?BqTr?0 zv(_d>R0il%^3@il?hYqZtfeW9@NjM;$(BWLF^ONExd-f25V%j$|;QI7H?SPjE9Vrc46*^Dqwhh8jDE9wHKm&2MYFwREdqE{){ z3u$E$6<_P`^K;TCRU4I{z3OZjEB&CVDQ1MKtK*lFmu3RJ&=~+XAWiTnbPipinNKGv zvHUMKJ)#HJVOA~{<@z~eztdS??o|f(BJtdcU4J$F9;Bc$VyPG91~q~kA0%I(v2XL) zD5R+Y6?bEBEW9$zFwW8)=MDaNg<-YHzQIP(&Wo^JDD^xKE$dY$|Ml>1jqyLJ_}?rY z{{P~qd_IeeEx6pw3Q*wxm3h13r(*_~3ww9L$730a_%Kmg#GZVhF^)y9hP-{=nr=Aeq5c^KTB{@d{`bDR zFS8jqOu^?*VB?SK$_2)Dz>)w3urZ;4+aa#-#G zRd#n+n=Y*#rqyq*8(UfiVGQfl7mUpC2l7N%?8i=P6g<`e|&JY6KN_YS6_--J6b+BiwRp&b|{5;XVBNoSy}~ z&ao^pU}b2=hT4!m6_2UGwxn=hyh<;?H{`i}hIVH8xf)$;%P4HgqhQ$$Z5r!RvHxj3 z2!6eNrBb~TLpxQ1kUA3s{<~6ZC8>X*Tw?1u(}Vt{iq3)~i12p?!zjbrjbB;C7b;cX zg|FtaR*Y2s4_V(B9a-CL+lXzWW81cEn-x3hs5-V$v2EM7ZKq?agT8s+bH@G7J$H=y zv47OwRb%Y+%x67o&b4N<^QYshULY=a9hRx2D^C)EC2uX=I0H|IP^EOXWuYv6M$5~x zP2v^sQrj-c=%^w!TN<1j+M3khOellf#E%NEWwI`$-c)2pSyzSnzwaz6rql|6oMu!1-Ki1XkyNNH4%3sU1+COtjo09qA*sH zO@4=X82xJO!$+5v9Jr+1iZNBVrD!R1b$MTDg10JPJZ!c&@NW01h%_h5B4sSf37`+xF|KpE?qoVY}oaKw6c~e4Ptp8uuz(;oE79Qd(67 ztzI1Wni4cLM=cKKFnu%&6A|ea@)}FWHJI1QcS+U+L1EqtzrTKe!M+~r^JpbNkQN+l zId^+a@g4IV%lQ61J2v<_v2XWHYXVVdkj7hUXT03|XOuxl)m{|X)&Q{qFjAjg)$ANVZRFg*5n ztl>dY@@FS#P(sSV5>Lyd;u~S7Vr}XGY)i5zGFfogM^QziO|kh4T`=cq`1{-D+QYVil%l57rL<2?ywEK!XyMG*-r zxA-$1oIGIH{R+A!6E^>1bR(To<3Yz{bYn$obmM8-MmW4!0>lY$yR7~#K=dfW;i&s- zSv21CHxe1q2^?JVpMVFyUeNX1FTlo$@{Z$f>dh>rt+CCx&M8nrJxcw%W|b4?ihMz1Yi++pkk z8TB*Fy1yrX!E(|dNk^!b&{pv?cAkKQB}Tth_1j7B!?0>Z2rJ+(0DB3yJJrwJodF0vN~ugTGB;+pTco9(GBf^#q=( zIOv6(%2ct>ZKvp$(Z}2{tMr1c!f>Ejn@3 z6~mxV?TAXi&geUV+v>#zx0om8Dh3Mowkx~qpVCu$D1?_Q-*)+@`BMcaYSzB#1{rl2 zwYK#Ktt@E36`MILaBO05JfIUTkw0L+WIRTy=_kz9AN-thqIJwN%0rrVwaLm=pvQ*5 zN$x*h(!gwWU({@nVqS2N>(AogmAND_%9f3HVOw@=p`u*|Xm;%&JYh9(j{-Z=Y@hil z(yc0Vo&JiEntSOMQ}NnWsO!*e8FMbd_Rf&}0Nn<#iPX9a1W&}vmSeXPr0mHIwb5}y zUs)FIGkHTq7p$TMzf4xtVq`G0!S`>mJ98(bcgQH#>ltJPp*W?R&Uj6dz(`(5kZts) zGo2SAsq{j^iBAxOyF;J~X#r7i(O*&wO9osm6w@!;EN%fZkNelC82cQN>A}H1SWAqd zM6GG+4odxSr!f2r4$$!6^*BcksYX^Dr^UN4X*gATsPK21GuKr7`SMX}DDcY;di}@G z{=0p!`nbXSe`97G%bOzEh&qS%)%=OHVjF3P9rI7Ii$;9}-0OttYJf=Y`mPhpL1qb+ zwjs3lZ6x#zTRHHX#8&ab)ytWGW~{(BZdT@P32ss)* z{???7w@R-Dt74VMCC<|TIW*SPydbj0BEC?_|ur>Xe|F$`xD%NCBs+$l27k0nuhvbD6tTcG!c7k<@2X?h2Ye)DpF);8dwg8x;*by{H^6DNeQy+zNP=-Ad_NaLGACuV36YX8!rHRx z_rZ-D2NA=p14ktF(+=23&2qgJUf3V`N6$In>Oxkg35bS$uZ)xJLnb7MAWI4I2kIlV ziDM?xhtxhrIWK*C)a?+od`RIvXjR{sML|F`P@S@i!0JgT&i z<gv`44Q+Y&R4INWGN<3v{ZFhyc{I{=^L|Uj#q`JFTL&SgV%k0xJMlyT>2AKu$&5 zrZf#k8GVzy_+-zm`O;4NI`|28kKtN?^O-Dj7CQvHn`@uqZ_NIyU2;z%YFjQ3T+_UT z`*hav&unL}>D7;iD|_GRUUWcIF?+^(_VZFvR*uZ3Lep3e& zhlJ2xlM>QdKlCmgvz9T3s9loa7;Ju{=fqu>;4`>v7Vq9`vVILX1B&Y?!M6e<>@9$w zX4$d7pv)A#lpnrIFxj+5@=+m6`Ez)54)*BvLlSf1_4I{VNik6J$R^xe| zU6SmHX#}at1ik0ht)ZL018~V4Z(=o>EF#ND2emOgAHAPe$|6O$>UWXS93K~vD>c!o zw^bDdj2V-}>4vlj2W)|vrkS#Vyln-aNAVvAQt`D7&-*QNe^?tGhkG(EvzDxe0un`N z6W=#|c~&|sOYVxEkGg!!L*YSMTJ?_jPOC{AnYk*D`CtIf2V-`GIWT2<*2b-{z7K!F zyd`LdU8}+wEQ1l>okNnfb7@;w)=&L!V-Le*D}KV;9JXC}VzX6&q;?wRA}ti27v*um zkIE3`mU%e8%Em@x(0IjOumKf6Nt*8nJGSbM;ml=6qalwNSET{ji}9di!67=Y#Hmb! zsvEKyj=Rn*MLNDA>tMc!a3LN&&D!jzTET-gwF`ev+-%DL!b47uE8i9lO_UbqcA7-@A;uE1K>hvwuWeEt-{{+7p6(Gdhu;-=Ft9JOP5I z!n&nr)}}*Rh+5l=Wu2%dk6ebku-Q$Gqdgx#u7l>degj3uwTgYdimqni;>dK95lIL) zUa%aVQ9u)BWL=>bdPrCCN2K8G=QP6mqvnzM3~X4pplHT@2q1BhaLu3U-!yH(#}1Mr zIGEs5HcKJF{J@T4qc0#fyuo&y;7@dh?)aQ$y0yd@&ISLI9(R}wUd2PEA5NeP;tv%6 z&CD*_+b>*N*MRgplC#du*LI3>yKM(2d&Q1v4TVy#a)=3Q8ArF189+ zdi}}kp_|$iGZ(PuWoj^B&-79H0`!xKuLaE3mP(TNRzH#kjSRxgv!u(^h!*Dst-eTj zQ>Bf%mIATf>r|g`_FzgsAe1F;;u+WqF3SUjPbI&X?h6^rN~^Hd$Pu}A_$@%gy(%|2 zeJ>5m@f&q1a)-Zwshsax^3$$tQ;#1rxY?#~gdiC}%D`5INPwBu{T`5ybq9a6xY?ve z#I`C(P3VGaMxR6@N5uB|+j!M~loO%Vs+y<@%z?O)a=RT2Z25LOZb1u$5Xvk<->Fm} zl>>UU?eZHt_Sw0YX!N=)xSGMG=`t?A`yxYoLbUlx3>V7U8?=RI-Y>TXSG6mzzo#k^ zV~5GN)tR=5m#3eVO-EN5x?SrLRxW4`V}a8uN=-UMOxxtIE^7>;p21(KV`V+aeOiSw z^#OUdeE=2V6ojOx1x&>ATiCIPkRnJ5SCz)g0V_A$=}iXcucZz)YrR({7G)mXcs$i^ zT|9W{Q^;j2UVXLmrz>9lwI@GNYasmx5^{2@jkg1ShjC{jh)|!s_Jb2D>j25|(xXjX z-6lPGH}%BaIoUH$Ri_XGP3C(Q2YuPW(pfXQu9oMRt_&g)nYgUyj}f~DMLXXAO$+~n z+5a2Oo5zVU=D&eK{*xf?MlnTy1^YKw6mlaKhWs~IloL;oeg5pS5yJ)bjzANE6k9n+ zLDi;+;J`)v4?A;a`(#WgnQtbFxyqwsKMZ5R;$me-W{@aRLkrM&CV1mR5>l^^;E9yc z8_31Q&?i>Kl37>}{=|*4{Hd=AAtYwtZMq18Eba5P&Jpx0jFW z&9Ciub7#{TkTqz3olKY>TNQ)!x!U`K-aU%bvw=LD8|~u(?T$SG0WHH(UN`jMSLzPg zkRcA;{{2qgg1*r$O1~h<`jikiJYq6BHRimtaS?alVF(^`um~OYk-ZhOd`5y0^QVJM zIIDJ)5RZ0AP_FX{JV+NhpHoPm9T5ro_wBsDDU2LapnFRbXU9e;$Z#o)O|`(j>Umvr9G;ZimT-P!qbc$`&$x@F$r=%bVF{Z`mHapg zmaerUZ610GB^o{Rx$v;_V-?bxiSvH?Izyq4E(o?ziHDIiJC9ee)|<-3f&L6o2f!z+ zwo#8YCfQ$%r2`Em)%LjXvhKcb79;u?#rjuFve38*;A;(Gjhqq3$QpmFhUc zaeX$DE(gSy5Oi@+u9j#TjkM{r4aAU;a3`l>!6-#!ZWCxFV`ELJIVgPa^5@_9W_j_p z0XRf_R$)qpwbqjrN!LGO-9tyG4uDm92enPh3rpngaXB&(V#)*-TEsT9j51AbI5z~T ztAX`FlRn?LUIU=?<|hU%rrmFcyk z%3CyuczkJFv!h?og_a@=c>#fq;(Geruyvah5oXV&n!(Q0-kQV~*0{lN zZ*T${*RWh4PTxF9q*KB*3KSXkHK={*s6HU7ndMa9FuGN)z0i5CA%W!+MJh*Qs4e`>C_fZA z>r!BJd4#@Z78HE+PL5L40-zkJku9b~L9!L?>sT#{$Vft`oQZcS3 zeUq#R>hLs=>AZkxvI9OY+qTfn{FvdawdATPOUi6PXVd^8oLR>QJ}r{~yg*#E}Qr^;>L{Ii7xE(b< zEX|{540YK;TU4-J+L!%9 z_GD;tG(UiY=(=>MR#FO6>bvH~W2;sh#=G+M&+Bwm)^oaY$_@^+iMkQD z(|zph+hz`N>n?keUF9mA*j+*Y_XFs1QXkghpMEjl)sU_UVno@}am0D_JpeuRGkp;= z{37q?+meMxG)Cv3{6-wDoVbVPB#y=a#MvpPjLKN&s@eylexv#nS_8Vi31THN^!UKl zb9qGP{%S)MY|On#>|Fbxg=tb#gEPdbF8(Tu$r;gyL$wr%6H=)im*Qht-tomfs=j06 z46jEPn&k}6+$-}?f#eD=+)5XW=2C>IHyn~tf|~gW3+)wpMJm7V?8nYIYt^zEu)yPP zx>u(FuV0Buneeh+jYna>3}+dTxX{wdQH^+au_!;6uCagP3E|Kv7&U$-&D0Uhh}pAg z-Fiml^k~$!ycqN1gPq)D)xNA`3jecYM+$Y0|sZx5OYds_~-dhL<>IV3#&Ky<}=6RjsRmIWKl{`r!mnb5Y4PbhIZ&{ zEgmElpjtDGM-=}t;>)zQ2gboFdzS4n_k=6OGE>#l4gn1gp+#tQe#aSAb3z@h1PeQ( z>y!z*J4c_$D1u@P882Vhy9bTQq8Qbaay7hzN|ozERgweEb9hqeK~7b*(ma`LSW-b3 zx-IidAQRS2#%B2E6{j_@nC`Y>)lIuU7-e?o=@Di8(X}Nid9>K_Voy8~_peBpe-+yvN~2Tcz;|6a7-OjE+`h=kyRygMrDq;9A4(+f@c;NW(s=O}Z) zt_*)&4U2e~M`1#eIdXLEIlTW%Q2ZxJ{QvSxTRTL3ScHG_OV4ZzFWpZT0{{jhav>uE z?SztIfeee7B)|=>3E#k)%Zr#YzJA37NtQ6FezgZ>mM}4q{^y?m{op0f%vAf&r@=yi z;(SO+K#Ve6n4oDPZph~BGN!O^Q2(T{5a!kOAjV22KcMfI?LL+tM1hI|zhZp-*zAReS;IYDeBhRLI))Ig zG1rOYKGgCZ0bL7|`@0o_8H-Gnzg&W5XbJ3I;0+n2#)-euv1H zQi6lE#Ri6FJ{wXl&=W!&ls1sWKz^?>^|zS%?84Jmc|=ojg)8($qfuxub?R!OIIye; z+TK&{j1;5IBDAihD%Kt~svv1(+Lg$dYt6+JRl$zt7LFm=9taP^#jvT+qRcbhC10Xj zP_)pL7tZ(Pv0xlhWBOsC4*HkKZ(`Y3f%j zG4Cl>VKSrqE4_*@A6Se{@UDS?+KkQqA$gOIIX?OLhK#sZQ03M5$M@BR7O`ZNulec` z;_|3GYyIT14bELb!B~6@M1=VHw}jIEO08Y8XE$P7{gu|z&&6h#nW%SS`*}7%>KH8K_Q9FVE8<0sAQZ8`c*b>SnVKtc`EI0<(41W#@8F@DhYsK?Wo#f1GhTkTkv=>F-;(9=6eAH`biL` z#shn+X<|Be9xnZMQFOP&f$m@3GoQIGRVJlj-(yK$=dNhZq`MoMnt1U_AyWz8W%C}_cL{uT zq41KWENvmBKl>FfB|GXd!RVtegx>Fww+CTY_FGI^Ogj)x8ICVgj!BemdEb;OVX~1; zcYURaj_Iur=wX;*Mdz0-t1B%{b#qYSgURNutDrmL@ zpY8I)$LEzWMnn1h%3z4ZBXYZr*=JrQIcvdp5=S1!52p~fu@bGAvyeQa;xcv$d?rF6 zQY=7n2@#A6{(=1}OgUEo+fL6mYg-|aE{PLj2wD7I)E6%hJD||l?GQ5|s=en99hM;m zT*2h~vLUg(vGXr3{wKozXITDw!uE+my!`&}2|H2}XH@Vr3`M{}eo3DvY{5!WI8Y5Z z5@fTWEr#WPF*c3(c<-a1-*NjF?I=Mr1o$YVZ2vujR(11J#(?yLpbxcu-#PAyS6}dUl~)c;WcLmWm2DB)!imQ{5BAaL|dicI=wVizYDLEvLpA zfUW`wH(P(}6_MP(4cUnD^cWQ{m-Ufnjq~WYtL3!mByP3{XXO{#!Rxws-0-RM{F*9e zC7i8uoJG|6vl*z~-1;LrY#h;@2FY)@zr!SH#$3_>b24^?m+7Zo-7`!w7V|OIH>1_W zN)w2zVf@_ee97}xKZ{#B8FhY7M|2j6AtpKLt0I%NDtoSwc3suYI)4pn{r#wLv|!cy zovd(uQnmYBgTXHlYH%i6gD_jnkp_Cd!X+f-0hXb(UK`XU ziyL)OWz&Ind(rGmqN2Fr?8uAa#&RxzZRA&JCim-Krjc&Mw+Q*Bh;cEGdqhKiGx2!$ zaAQXKz<48&+ek^`(vNcpAr7b(zp4#HO8k{TxbOoetW{3POe>M%Yb*sQ5C3z@6IrJy z$*d+x49W55mgLu{_HCqYP6M^I&}PAP#f5-T#t#ncx?Fo0oT0vg-ygyE&;KI!eXM!o7%jh3N$-E{^>`i-yJw)t*U|LQ2hUpFPGM$FFu(){XWJQ2J zO`QC%S|s8mpQD(_0Jv>|P0#>XTU3-abCO9X9Ij%nVv6@hpo)URl@Ckg@SG(2F63sb z=iQ}yw|g_k-;XPS4`@RqLa|ssWFgooj{R?S6_qt3VzIzqtFqczgJR$q0~amY8=|Fru!)|Fm~VkE;4RD23;fBDXfF@ z(KUL6vvF-EYgh%9qu;P=rv-(TzhB7$2S-WomnMO6-b3-SE_ol$mG-*S^bGS|wQ4$q zH1rYQ$EjE>&bV~?n#x{D&qK+q^LX*nt2&7i^HF7fhdxYYEYkzRDw26XIo{Z<&S2YE zXL2x7r8W$|A4Rp;Ax-8V0xeA(M5tK0sg3N5oAg0d@jj0U%+`IGP+VuuaKrjOB-7G&68>M^I_d%f+|e@>&zb `~y<__w8J$&dT;aVALj$rfDN1DL& zfFk3p#-F^^dua0I9vCVp+~dU~nzK7LH!mN=`_u(T{8TbpzbGnCt+o6Zur5*^VvQ~bBO61g(32t!1FIW;>UnIEDB*@O` zs~I``abe{8vbZVQQfxQOb;cbPj&6ZPsf#D-subqO^0fQ{Zca{4##vTH;_vewTdlR< z0OG&Z<;U9wedCu%d1J?Zm%=)m_tr$o{Bb;sw43E!K}oO5_{}@qUe-*H?qZRov4>G2 zG~&2}w5H3beyin^F{i>wUIiNdhRe3o2bjD4$tPii6AE5at?3kDEilZab@l)h)B}5S zm~S+Q(Evtu-UlUqNutL=K1?pU>Y;RJ-oCq8iI>FrY&Ok^|FPu+>y|uv&B&-!{jstapj>T z9Ba2OxvVh55(0m~0Qr@sN_=G(IonFWJy$0*E;TvKgRt%sUir0r41%yKmLS0*i<6cYrIJL-l z6{8>W82a$_19?PSnIgDi7WqL*vK8F&ugXWPQ{xBsWict|$mw_S0(;~JVeXZ?z0#2y zp$lw;M8G?~iz09S*qLPMn&r@$2SH&M4Gp9`Bv*CgjUU1szXbm;K4C4E%NwO^ z3r`FU8=@^Y^4lQG)I+%Y`Kv#2HpjHF1g$}njyTpj_JRnV0gC(hvn6LLdvbYR5#k%` zs^`r<`<5dljG}T=VA?m77=X&J7M8B4IO%@__?Wxc`Zls2{*cH~?4q6*8QSv9EG}gt zv&nZiWzjI`@p-uesf5=&UpY;;3MIQjDm@6>ul*Qs+Vz-?T;^vS=I$~$vglEaqF)y< zSGrn)tDcWNN&@j`L!@f9QQGqPDkAvTu{RGZRO0wiHZOj|oY;1f@oM*(90tACGBm0n z#5E~>u)Ayu>=reJRmbujF7bFT$?>1nqwg04YSQi3SjZ9@m7rI(OaHE%e$dT1^YE#P znO%YBbT#m)viPp|w0!JIKI4(Yzi*mzv;TlSP2y@$D!;8z_V6n08hhrcEcd5w#q|>S zj|SjB8|?qyqMNC2aUdeBTX6bWdD4g(g%+CYjA z)PnZFJN)S@a3P()iEmJ6Q=c3eMHf)f-v{wME|N|wnbt_EiELFIVWmRd1aPrfwbBo{ z3loSWOfM`<970376DVi`C#jw@5L=L8t1Jls%(F_&lglMEw9tX!#Xn`I>QzSTRsL#< zA)J1DCER}T9e(rN&9vtOH*DlR4<-}CwPPWP_gyZyWlC@?bdy^e~ZC&z6plJ+?T`j zjcMNSz%5?<5NNqU`1mc<*h5ClMei6oPuCiMER>ZON0Fv;J6IUD9#yzEE2=RLWRV=# zPAwg!<&eIC0F2ql9knRYgQrbA^5!L*jUNi?CJSCb))Tt=@>R(ivmR)fZZMrumQTV~ND$TP82qSaG% zCQrHavGQIrDWhDxTz7@2d~u%_$mZUarn4Uhdmt=Fe>(zMpq+rTX0TK?8Uaw(DjxNG z$p0K*-{=!S{&3pKWPr0VN14tFm@Cj?EYI&M&22PN(6g|3&6rt8 ztIrx}sV75XUfX-$6fc-x#y^l1l3BTey}TCtcG}E$68QiwS;(FsTP~NY?mj>qT;0vI zcgDfpZ)Gm?#NSV-a{iVGT+p(rW4M2L9}=wKTELGE@zjp8Tv(RjzmwK^d_nl6RJs)H zKih20!O@-WCA^X>Q8B<-xQ+S$eyDH%qAP3Pjm^V@3du{;6`mYjR&_NvDY^FzCuH_l zdsu28RW9Pk3&##IW}Ba{8;T!uF_ifs?N^%J#6bYpM4d~+@~1WqxQ-^Y{%uT9$bgmo zv%Bn4X;?N)WDHJzFg7?GO9Mt-HY`)g*E2f^Sp~DVRf}>v(a9gJp$lwmtX^Qk`KQu+ zsDBx4bm|0c`JJVFPKcyEJAg5Hhp`1q{E-SfhJZ2!Ej@G@fnL z&W5~WtvuMbs|e(B@%h67YNnR3`#LJA2He@NMFK2sBD+ftzyU_SzIu6}W~%$?8)KR# z-tMK33Gt8m4V$kkv``2i_By1MD&|9drC~?!!p|uUsk6UOQSfs;7`67?lGzpZ9UfD4 z6Dk)nGgPIdoZ8K`H^iIWhMz7Er{F_OhJv5B+6-fZ*!|4SgA=fot~WwBnO)l(;2pQK;>NXR>t{C?4~!rW<0_kMevQHL{F<*G#iw3J<4Uu^dhVZ*BzP@*)gg)1}9&D1UU!f!R%f_2<9 ziElizvf}q8)G&!}Kv0S$r`H%mV3PD_F@}#I9Q;odzMtiH1lkWhx^Qh<^cSx z#qTGR4cY}+a?UQgju{&@WMTPi+>iU{_&!j-SBeQTqJhxkIZ~@dRtvR{5$?piNsKD^)xtbk(#?_YJrh2$5#fJ~nHj ztyb?l|T_ zL>t@~GQPr&6h6Zq*^;4N7peQ{GksU_xSoeJsC{vty;WlP>E^q-g5G5k11yFD5;+dM zdacv?s%XM18WpvNn_H5Y1R~Ua_1HVGjrWJdOJ<7gY2$nfy@ARwb#YRyt8NgY8Bu;x zs4MT6I?b8c)h)&4wH5v4HN|c-6Onga*J%Z;_hI&P|BIm*Cwx)R(W`CABshm-fQh-%Bvt&9jQU1!> zTuuFTcx$P4=f17``4KjS`lsWMZ~i`~ZF#>SHHPL3tnBx;WLBJZaH4-Li* z_=?UWxkw3b{i~dHjKA5KO;^CHY{4;ytDD_sz14J?O%=tt3ft;|Lh#;K>sL2!1;&^r zHXIN+h31{y0vo&t05G7VGm6 z>!d)56bKtOU>-h|G+_-}>B-M?lzmp?0q?JSZJa)VsX$7@$S1@B1QrtTybH7*=`o&S zH#6Ziy?L2!HEgPwbvt38lPdgl1t~`j(GPCXXD*j1sokuh&W$tMLLuKnlem%c>vwEZ@^4^Fu$`q9%i2g+t{YDqA3HMzS`3O{D~<`?sh6gDoESWJ~r3+Xo&CX81{bG@e>-zHM%7?>UMS7KyPDz$4h;cv7q9&<3S?20KzcI(QfR|iBSuZ?DQRBRP#J+qbb_jXrnJ0 z-;N|}k(~>$4pVH1+0!33GW(m=)erE3ofA9OGwg&2wB_pVfR(mVb!l?yNMe!J;hGe{ zQoW#0iuDO|0>tYOK2Xa1Y_tpy5G0n z`j(mWzqINCeyw`LN}7*Yc>O6F7s_Z0r7!;bnmaC8J=jvz617}(A^9cD%fY*qZa?)X zYObMR7LDVGRlV^df*p>g5---NWncA$7w>2nF=<|sf$G#@3)5u&L>XqV$((8fhlNz* z{hH?1mTOu+e}BQaX$|1t3d3$m)IaSrKJTN}4YtztV}GtmZ@DC^dksB?k{~?#%}%^m z4z}Dcdc)*-LR!kFAkZy()itIkK(6j9f!G}u5Fa2CAJeb=tK1a5D%o7uf_P5_?AFfn z(MnfD6Sy60tUHd5321fbJ>IuyfhH8AKlFy@*f$X!ruO`e z;hmwax=*zcu7HH}QqoT`+Fu%*Ac!xKJ!FAK&D92zWFbyh&Ta$_1nwbXizF-FPu%oyd6Uv$2H-pLi-T2*O1~b?<&py;lNXLP>&7bv zMuFkl7=*#m-`8t&n&<%`KBpu?EI~2}#^w#Md8A|te*nJ4lsd~Xo?nwC(l~XNAV98bU(@NL^ouhhx5P9k5BT#vwmW z4FqN?NO`5SCq877yP?#2_uOZskeH3*&9!DcLltN*>b3`1kChP|(3d35r_e==+xBlL z7#$ulH^xgwi+=aiuP^J7nF;_xM+Jkw0y;*fC(e-K8#+G4@U)2FdsnM;{U z)!K6|?tZ64!wprSCzEwRFY;GAcKMSb*X5veo+Y1>stj)Ptb@jm(9j_VmOJ9C4iJ$g z?<2D7jaQ-hXofWLo+|Mar)SFCK}x=(qxlr+o(4A&h<0SHwf@%)#Qps{@2l{CN12-;ScBNQCnjd_zX@Xsyk z3TI$&Cb~K)D}f9~NsubyGkX$O*s4dG*a>#p8)^7;XCjPFylOdVsXBC&V*tE4WI<^E zuqB|Jd&tHJBhF~*hdKIc1_ypn78Xa8w9(=drsW$ZU;>l^WgxlRMLFsd2TVN}z@rM$ z++aJHurs%@Ss~^(og!(T1E9<*PWwi$v zsFNR7HGDc0FnfNG4oZHROZ&OjbOy&MzfTK4Z8LL?Z!2qmtJl}>uQ&4O!h~*Vk8ahI z+?nGHwyq|OyN#%oxx_$qU#*O%n}JrjfuBFd!wS z1KIBBWTuM+U(neUyG8HJ;oXPfO-eP)h8jl5m9bANr5~Hyo9etQH?dov_gtrx{;eA$(&b-6 z)ycGulG++mBUqP_!i!z`4hX#Ul&8%j3|cTyp|{4_jt7=(T%XteTA88A;v z(V8}CWDWGxS$ZS-81n88?{3uk_`VIahj2mokmtT$Tcy_uW@tW4XxtL``cF~iKSh%N zSycIlsZJHbB?`d*^5sq9fAASslAOpOtOyKvAZmQi7gQ)PF?eVuX(nTTSU*93Bu1fc zLP12Sq$CNcjHzFG3)l9P8x70Vzk-$uE%i=u{2P*%Fm!Bf%iZcD8pog1x_xcC3*R^H z-lu)+_08)9YtVc*+_&|nJ}toO-==-;_6K9K-XV}Dcbi=ierM$0aT0QIE=!Hs950NlEJi@0`d~`?Vy`LhPexIup`!zRfI)Yd7H z>SqtC?5lmV4&881f$E(k->zS{1(pjh!UAe{R-Y+RBO7=1!>qAgvTa4eQ3S7d?fC&p z+@dF7^@ooU75f-99e48|&OTcLJJ1sXRo=N9pxI-Ej7GQC!vArb%z7nny1v(U zNZoW#pJ08?h+=Q63f!Etf(Zu`=;UKIG*ggV7#)5uViV_SH;y~6P5qu6C4{gi zu0t;?X2KF93(xz^5)E5xaBS_`C4YS82Z!{4oa~q53?+xdOmnDxUFZ;4n7dRiv#i(aF zdSP;80XLjQuH(wf=`OZcN*c`oNwG@h$faz;p25Z042gljZp6gnIa?2FZDAeLHNb@E1{kJ(-C$fDXx@GDUTzYGL2AZ9 zY%7wEvK2qzUulFj-AFk87p_-o7@Q267Uy7N9cOjrb9N)|prZ=wR@#cE5$25eLCV{a zBUk^8&f@Jim9srMLDSO|?{F5_2_M%9TD2H4aJk=)3`wmTLiji8Gc8u&yNLp#WwBun zr#-IB1i)yf%?Acy?#W@QnIm(}w4kha{N2mm6#=OWnUu&OFV(#~1jPL;PJ)-d8__(I z6+6ZYMw}l~VH`_YQB@US!gExnQZGfztnG9B#Wf!Ho0160^6v29NQ&EORo8tAtvM*p z($Cu{KQKSh^eBh#t1L9b+DmS>)$ojq#y6fawqG`MR$jOnl|}Og;m;pB2;q-yyidCl z#>(S7rOvUY*(RvIKw12odiX(U(_+r8V=8a*6gJzQ#R6-~>c-r&bfy!&CtV<{tBawB z951p3lOE8oGyB*Ng_yKn;ap)~MR#=<)he)^m#|i3(_2FLi&+e+XgV($XMMHXpV=pR zJp0`TF6h(t9ipUoRQ@C04aeulj^T~nc98C0y^?mqV=d_#$Yey!WqKA1kd9IJg-eN3 z_@2n0_esoNMnTtqyx~6!(DnDdU3+GC-(En2dZ&Olc)vJ8mS@`cy8hUkLP!A$eEp+Q z1V14w`CDNg0n?ks_b5SkiND0YCUbB10fHN4>X7%uLs4-V-@9_(5cokR?tAau+gsQ; zS<_yyFAPfiXD5W9UO-JMAUl6o->t%b@&Mn`=68=RYWu^UGkJi?-Q$;yz63}>{Q1We z;|cJtTQ)QrSPluq>a#tQYreg`gKdg(*swoCX{7s+3wfsl(1a%VF4ptA`9GtOw_~42 zA$v*ax@ijaMWeU#-A7H_IA9lE*n@xp0SMAb5K0!T`NKokj;-BXn2+4awOpn#Tl~ij zf*ATbpEOw%V&k&b%BzTxrR>_(+hWmkc;FMFmtmMOU9PK*YF#%;itLxsO?Z4ki%0{s zL9N8_dV!rsZGkIvE7tdS+RtW-M|*fbVX_--zs231znOK7O}-h_wx%daq+2G(_Q|3R zyVvO@RVT3aR}tvr>@texNG;vWN^G?jg+h4szqC(;k0MiQ3o0% z&j;jeHj1@~YI(`Hj^SB)o_a{jv%mnVdX1u<^v-mDnxhfT7d^c0O-DPTUTqBbYVVKi ziPED*KcJOq*B4fG8zVDgkJfoc!5U~2{( zHBu-=P8!*`4BoV}^cLncjAa5)Vf9$gR#-b>uW|>r7(4<7bU(w1lQ(er9uJesOCdSl zKoW%Qy3QqzGzL0?nxWC78sV5;VsA7r7Szb#5OMMPHn`fT`d6K;Kdj%l=#Zq1_D`Y) zUOIlAt6V8j&5}xj{d9HY0&u^f$=v-Cf1hB+{3~N4J4Bk|M%As1uV5UTpq+h^=lMl$ z%{o2Sl2lfVmLf7O(`RuS-eglDUv%beR8PEFOfJA%h|3Rs)uQcdx^zUrMx!H3H{;}i z$~|?>{W7RsA6AC1brGRSEbhw)?Y>&+@F%KrG=JO-;QZcKqqv4B^dzu(i5j6HQiG~hD0!mQd^9YJ&l2(HX-Iaqa3G%D;0ommGUm`=K#Epv$hey= zh+=`uY0gYaiL<@TUuj8LMmt=hP|{;37X5 zjw4@qNW{HzMS1M!#FsCoPv85{pJ;*bVoC2SC6;b{y4dthKHs@pv@|r!%I4=h^_Kh< z3bbx;ITo!D+H^lAak+1(9SAkCXQ8MQImc-q>nItwtH3Gg+@^oKp=je6p6%jPy@k{z z#Vef{+(S0z0|0l!aCay5m{?kLk8_)Hm@PgVXYUZ;a}iMnVM^;sE!4i0((jYgt&o#Y z$J86iZyujpy)9UN=$M(K-K>*AEIFNWczmj6{YS^Tr1sGAmViiLQ2{mr9+p3I!1g|* zuqGSV@pAIitxLDq!@071I&DFIgdpRd4C2-~OTX078o;+0x4kIsI+iNY5OOksE>CrV zHiVVH5FCdLVH?C;fbaOmC)2qslOyD<;8?H+Hr%fl`Q#Moj^%dkZ36wKZeDlYu-3b% z_DAU<29SqO_mCYCX`$Q}b76%A-V@%KALbBOGzL4P%tf3PGLNYwoq9rXlULuGrP6vQ1f;jnxiQq(1P z{}MQa>p*Ay%v;r)%oS{+JGm@Wra`D-M8Lth1m4<4b^Z`!5Yg%w*4D*xY-9!VGS0yW zr{0^}*bqXP7&#W}dra!#D6wLV)pK}H@H?d#n)p8cbZ+*`P~Ah~!B=u`f&8fcP$97u zo5Nw#qRkRyT&L;6WaUpx_rCx*k083e$B->g<*E$v<>%Tq{Bl>&VZo#GfS}>lt_NjH zz>+F%JRIQ6CN3&ThWH?9#xrE6RJKB=Q}xw6hLmc&E}<@#S4tP|zMn7GeA{4ULYP7b zew_DA){AdhO>$c^yiJlMFA8&zDPUz)2|8=cdMoLJpU}vt3Z4yBOp2P`v|hi zG~;8oBRo2y`J&_;=`9iDy;%_Qh4;qJ3*UqD$%LJtyfW}f)fODdHg$>0F937O_Y8cp zgmbJH1^kcmelx{>ITBEwJibBZ3(oH_JVS7&+FeOG$4B@0dg41r=)ST4JY&0nAiY1j z_rg7~dV95Vm34=?x5TeFy>bG_t*2P;h}|>a?xpE}_Rim)BG{sSsAbC8Td|(FYjJ<* zWeNjxnDe2mzCM_@uh1+$_eZ5!9OY1_;t|^yBga0$4a;aWk?RS}7HOt9m3IR8=n+bUa_IBR z&RKDk;faA6_RD@Pu2=*u`1Sr|X=(8Ybtt+Rah&fFdkcx2;4GF+R_jaF3<}dOj#DaG z{iy-jD}Q|4ExfoxZr?07>-Q5Z{MiiP!;;}354-7kFVs-Y1AkF6O3n6)B z4M@%5-^TQX=VM#4QvQwtw-pQIAQjAvCrVl{A%li^OEDsom<=Ltt$@kw_;(0Y(Kx#E zo^4-a#wqOkcu%cazv&o$QdTi>t~yn7msl?gcZ`}2^Bys~{so!>U|huZ*wyusYPEig zn-Bbe$evl@&sbG~4Ledpymp*&4ZHv5k^>+nphq# ztTF?kGN^0`fu+PMUl_T|FCvB&@I{igVHm>AFmQooz|xVRr)6i`vbk>4-b`)Dts+}= zq|$s`{mrHkQ304L%)$HSLf}bPY$8pbZ1)t=fp|SHq6=lga;mAza-ezhSNAoi7`=DI zYTz7Id7bsOd;!(D554in;mTIJk=tyW9=J9vN(Y@|0YyRcuqI4zJmxWsQ<@L#kfsbj zLq2k<67y`yVp~+jvkFh|Lthu%k`mm-^q$3bPKSJJuo!a-kr)a26Bx0&q z1d_aGYV+*r6e`8YRqL1Q-Nk!_U&c<-xfmXAXW>Bl2-0oF`PFX&yG;xyi0qDLXDGLr zD`&*H_9_!6{~5FsE^#Y3Y;jG5Vy6ft;{NMGQQ1Gz(Ju5?h2$g+zTX&ALUl}2=CVwQ z%|oZ$K+u2`VHgp!(gCH2S+eY7MTT6_QX-?Aan+{KW98u45?dcc-I3!3b{_)WQMjac z+`3kFg`w0_@i-F(mRdquW8M%3#ea?nt_mvgao8W+QPMNvY0J$wiH#bB$x@ym;2{Om?{%ZfP~3l z0iN>(!fZV!uehzUG_B0i;j8Ow@$iFm%D|EwMBt=>3*pOx#)hpe)$F5WdyY<7m@sz{ zjxV56`W|CSWeH;Pucun#KljQZ<)u&RuRB-bQ6^3=|CzAQ^8^?< z1a8s6RPJ$9vAJYWbJ}i)(eE^5=JyZ*z*0ad1Rf^%N8glxX|mWDEy5%ev7@i@SR6_Z z0!Dz1dM?@o3?wgc?X3X!0HT6@{%MF4czH=;N?iP3iUUZ)+R4D~pbs#8^^1I+VBrXS z;#ohf2oJ|J^ddeSs#d)|2izlq66eKB44g}X5Wnt^B_iRHlCUT&e>yI$ZaAJtfXd>c z1Z?J*ge?6w3#vNhmBg&V9wDt6Wqn78(wR2d z>BsZ|qSjO8K?d<%!d_OlMoB+qWtHF}{0rkjutR>7Bz%kS6uH+83HvQhiy% zTST|N?7PO4^832GhdlfNwNL^-fOfp=yJuFNaZU&Jov~&gwC9a+1PlGJV=>+z&(2*D z0<}#~C=rrh=9Ee#=*aQyh8;ja|TT zsCYE^=5Zb(csa3tql8x%1eon{xjhr2pUV}KXRcFGy~#kLkK z3TlxXQ)YIme&~ad1kBooxmbs1|rsqfUS=4 zxEQkLkJ1Js$b`n+0nn*mqV^TmAz7iK#e!h>LiRnodUPCvU?UNa0BHVywLzk0N#*>! zu?<|-{O8&Kea3=w&-9Loz8wQYU=7z4khN7Lc^X>j{( zhg+w+oAdz@Sap!~Q!~pilZnoTT1#4D{Q$|AfkQaff_?W1@N%P8X-=~5C$0e%iZ{o)iRS%BQ8={35hYYf zewOb8Anp5CyP`D#$ck5ao`^57gJ;_88Q>eom;Rv(P@4P>G^On1Gx!mP-hlez2E3}B zT=B~mQnEqeFJ#>({3%dJ9s)`{PiixAx0k`HtlROHMx66G0%qqZ{;sW?=v7Sb{#y|4 z5~JNUvxT)E^8T^i*;Jmj1nb$!5na1ba>ou>^nzZt?zThCGFq{|@L!Udy43dPb;ebP zw}Ts82owN)!H3q1aIluqh1&H3f?DnO;P7H#@&pa@-;>h+I5&ngfn_ztsr^ja7Slpn zlQ<{-ZpoL_b1pdD;Ue^(bNMyZNw9>N?#@>>z7;CId}udVX=OBX+l%U>Nz5W}2=qwf zwB#eNLE=6UtESx)c|$v(UJL|b;p1Re1>TC`v%-f(8lzfGXH_tZB5MYSPxmmFL0UDkd>g_PGW8yLa=mQz$#P|)MR!k zs1+GM{c`?#Qb5GAA9=?pIQ<^#1UP`3zp%FneM zo`j+5k73>a@e;N>s`i=CHl3n|^BVeiscxs3mn6`>dtcM5kHC@L zZK@1&(C=NWJ-i3Qf{-k&?>JiGD^H(;XOu8X-eh;yb)c``Nbp60FW&RTu2Q1j;#{rF zP@mgXkh*3KNX=N(^7p??l{-J555rbTl_KE_Q$U()3Kg8|77JD?dNk26u(*0fdIEX> zZO~1!EpVh6gI+AKjUR{pD*jF)IZwTw0*`}5TPiDsEw>N_R&`?3+HCZFN${B_{ZF?m z<4dV*-lO*Qj2`PRfW7o#zPPX|LPUQ*QX8?1VkS^wSNYgIJNJU2C)^s2!HPX-B_+I!(95GRZ+OK??2Q2!I)l2QkkXx zXH^vCAIq8(0a(_6@j*Ao_>ynP9A|(7tuT<%!u`duhKxyt4j&e}$(9u`M6@pFMmgSh zCDZXEF_>nhvc>9L@f38q#R!SYK-jjiY5o`1vemNJ&XezsIk56%Mj^fFvoTdu-kTfj z`{Qo<>GS$)yX)HfdV})w;lvn3ZTFuKM3Oy96PBL0 z@$b_=w4?@TYRzzXJLVVmnMXaOWD!~HEn9j?i%YxGIMSD-Syfidae<~Hn8jA*iz?=!m>afy_w~J)ak_@NEUTKP^wmukD)-uE zXIa85=b4q@IBnW2I5RVC>z^xBW09=n1nzPs-Y%Kk%DjH5jpmtM*{jI1Tj5IL?4&y6 z>;?oYWY$zRAyJhlNnuy#?EmShr3hOx0%daK-zCo?h=YxGx^vJ@285w| z?Oq#llF>~%>r=G!mkHXlSy_eho90`ncw^AotrH>Ou!E&1rd>>4Ja$jyp+?Yp=@)Iy zy|Gx^h4C@RHo2io=(~}gR~KX(pY$+_g_lDRbJ+)?n7O4%x$Kyk_A%5i;VN>9T>gq8 zR8S{52xIXh+p9{|$%)d7Q=Zk;1xV~a9*rwfUugJW0K5QaOs#C{E}4wb!ZVQAVw3mZ z)=*?ZR_P69%TGlz6@S*wr$V?|jhkgOjk-y;ZhF_eyBF$9+8ex(PE;=A)r$^YS|atL zA$iK`s7jKHww`{o{j8a^pT|;h=e$`zW1>4#%3$YjphK3|nHVMumVK6l1~wd-g84j( zgQek^#)(oVpDm#U24~5a{m+*R|0H(+a+OA)7inLwEaZjLNWI zm^xZR!w^W{q{Wa$Y75mpyM?odIy%tA4nd^xy@o{ThdO}tCe$skirfu3b*(pMbfUC* zbEx(p`%N4E4&JGIrpH(<2ZB15S-jz;V`Vi)s@^-xpEo9KJ0zZ-P|skYb35~y2M9~N zK;`bb;@4T(C8ga1&j{gIuOOL&1@>l%-yz&$zSx65+Tw>hPl(dwW1KP9kb#4$_aDMK z)5afJ#>_J<}R(l(O(}U#0P|^xTS>!fJE9uu5Qs2#E+a+ zg2!;l_MfG>&StF1JN)ZzQe{&_S|ymV7Cg4_9jLK3muu2gofc6ulJkD-9rlricJ=#? z{AF?{q1m3+lu-&=iX4jgX>1W%broB-HtZMOd2{_;6<$tn_~DD3*%kE5Vp~IHmA7T@ z2eP>5)GwcIiT0wiz+|Ym5LMqu^2*I6^rb>=!&1_k zWT#y1$V>aS@>jcTm0UZMy!5hmi6Y?^XAT>OEsVS@@Dujz_nl8|J8v_<#;5c9yGM;2 zQ+m(ri)ibFUijqg7K0)Tp%#b))qYKI4d!B4#&=xH(#lIJpoFq7%K=x(qo3+m$3TE0 zx_fp(KZy+mcPf#8q=Z@^Af79WtZiYEQy5%v2xEHZ!|%%Q16>S0=E%c=(6Bk7t&tp);7I zNtUF^i!_OQKzup_4!8aqR6vx>1976hcP12u8%+mW?aD2AbWB339jKF@ZUpUNiqj`uS87{A87L@&@gN6*(cuvn5Q+z~W9-m+s;=Y|JIwbS-tw2%5 z4rj37YJHU18XoHov+J2ZX)U$6(8CayQR*Ophj6-@Co;si?5bw(oNeIJxa^eSiQ8TJ z%*VLwQcvjG)G|AFgd_~xLDRimAo|Glley|d4^X;lE!CIGO}4D6poK|IO?tz}O^sF% zs^Sk~zi_O-{3}Mo+h;ftyfHrbLm}ozx!z_751t^}6$RsUzU{Lf?6%@WoblnyM;j*P zjawF3G?c}C5`MMXFJbQJ9@yHFHr0KeiN@8qXo~fwntto@sp2}=+QD;MX;$ipZHf7k zF`)hL!$F%idkuuW|EGB!DnE{Wf@5~nxoH=UXmMH5*Q|lqN|4cUKvXMlw6sargQdz? z+a3;`P&{iavae5-Gnzph-P>uW5xg~3Ml*UAR;i-;aU*G~8~XVK<>3&W_he}`YU%i( zxQ)F!&ujW-S=~qc1?^+~SkGC(nX-%*G0+uFa*(+tfwwIGk*G=%Wevdv7AzBj!J zj~5uAp?AIPT6`29kZnAF@!_$ok2U&e>~D+AP6~6JQ`P%Z6M|}|#KD{CCMo6(%^BIG zzc0Iiy5U#OX^Y<~vrvO0+Rb?x{QFVNejV2iW6kcSWQ($8>qNFw9Pf_8e%;$aTUl)5 z)Ie$gt{nfNWGh6Nq&Jh#BC&e#v1;S5-P;>q8|>y!DW)4f_$2U8g#Xlq|69oYpLOAX zz#vw*hEvA>C^p?NsZ0fQ|D}PTramu#(g5;HKmU1^RMB^8I|}x_`3`r3K!Drll~sl} z=$N3O=StxeYiTXuW8}yzl?e<9{47BAVd_bL&W!pi^=?{`_!6 z>(3xVsSxIqm9hfx zW2-Qmr!L~dUyA;7R$v%PU^ee)M0`S+%sfhArW_JVWu{sM-N`zJpuQwG$-P_;RB-;otROgw3J# znI-lX?WAj)^o9Irk#5>eJ7S?6(*Vd;7dCM7+P>>Ow=kL8jABYdD7~i5vNVN^3raq3 zW+CmXPlC?I!NnR&@F&!yR49s7=`KZ z;P4a-;GmLUr`!Jp)qWH@(5GC_O`Da)_nt2+BtS^$EZOUb0fS`QTwo60Pyuo!$`>4= zgn54bL$DVZqr0<5ol_YG?rlD1v{(`0KQWaX!#?4NecH>j6qz8xhk4CoXL}CN_s{jR zaxQ+CvgZ8#XhCjVHa=F+Di**zMC)!8&S>fJbDIB;H{2$cG?_sgdVwt!d}?4TA7?j= zG(r-L_vqM}F~drqN;#AI0U==4INVz)l(VE^6#u?9XBDrN`a>*V2#QwBMDV!)oUF?A z(#4%+P2%%Dv_jA@5n>rBBd`Kk%+|fJN!2uq;KEJ-p4$J5*C>l=%TM-BjTMIU#*Ps+Y=(`a4!w1PAdzLUj=8f+V|d z+2@KhDK@;?C^gVtAxyiMw1qAtnNX{;C|9xb7uO&SYLp!LUF95x&&btLAE)IL6bS1M$p%`~M+kM5)&+pin@z@`l~w zPh`Fgoh&cDO`|((QA?81Y zp!5Ae2t(4ENnQhKejTQ-vF4=z2TJyG{>JRja{P_vu+rP9!E1@;S9)TKinfiu0In0wx=yzjCj6MvXb))LZFU!IN2a=ea&dB z`XvCq*lQ%NLd6^Y6w;J=s)Ggn@~@!&&_)+wTM=bUAa(Lx5gS&2oXSOLOIv5-SUjr@8ggDD)SEs(bROvIYXF@bqOZ!J z+Rci9t)Q?wKpYH|tN%uDwRSo7{cW%Wy}vKwfQrLWJ?x+MYbYz`#scrn0{;UiG#h@9 zznQvn#=tI~)ehW;4V5%>AI7q@SAHH|{xi4lXLP=s)u+B4Eo=Q6DM6I0G7`-){*#Sr z&ccJ69*=G4oB1qTfd@b&Oh6VXl&QR~w#rx3lN3?5d z{%tMqFuB$fZ?ynz%0uxvnx~9RVm9$TC^ZjgY)=6G7nM){lKDq*4@vfwkI1ViSR*(q z@KC$tZ@v1JfvyWoRevtl>H^~8oF7z0|)gNf#`xV*|9jIIJCi?$>{R{r?+5|3`5BH@0#mo5l50qpC1f0Uj6~ormefgy8X*%5`aY3`Ib937o$V^=bsmKfE} z#RtLS2w8jRm^ce!KA{l@*3LzXVSXUlWh6h$?cqc-Gc1%db1?6N6ZtGvA4RU_Qsj@H z-K94{IpNVngfUqkSer92Erf$rECj=l@>gY>pB^I1!wdF)U;P=Fz=5}ciKS`RwCZ@j z*&FO?kr_D}cd-(M&6+sAOv`{}+?B$qOmM_d)2oVyxa)MWAF&%d&FPZ#FqOBOYjv%6{k6m+g1cnf?A9`?kHyjUaaB*o>4j z(_?k|duDnfA|wFt)OuExY1J?O(p9@?DM;mHsN2V+eXaUEs4m}Fu=5L)_VKnVDHE1f zZwxAEGq??A@^@t*ihnh{n{WudWU4$JPA3c(=JJKSw{QnCYJ#4TWZgSOOgJec+p<2&f;UNr#j^5B+LsUV>uNSxKv0a#$Jy^lp z#lW_rwem-mms)3cf8*N{zw2^Koz?-lQdj2NK!8EMzm!K6{+3>Dm;y1nsNiErH-F&q zT{*s?eV;yXbsM&7*W*utaHZFgw7GwI+d1efSTHHjt}R^}ynPhRTgX}T=m}Q6O`w`3 zl13;VxJsxN(st>`_t1gk3`Au+JFbxwJZHcV0C5I*+X<516!0gPx;M>Ht zAf15*Ss^YGS%s11LUwx_Vvij`S2Q`jZ|^LV$3>=fsSb%lZ&-%VEGOPi!ZAo$S*&+x zMHgI)4gHB~`)7C8ARw=K*(S*CkG0_5QFj)}J+F|2^S?w($^v*lM5s z7OG(wGwaU?7l9Ho30G}tt+ANrmszMq}`to3H<0Sx%f)8ErA4< zbAAG3_cE$e+LWYg(=G^Z$+PO!tB%duKgxb_{XHS*XFV-ok`>uq`Pqo;=3Snp;uT~C z1k>O!3Sqy7C9t`g>NrUti1Tu;)gp?FtYyiO;&t#?dwUHQp(-=e>PBo87BTgSrbVSH zFd#o*Lj+?Ck4ofeT}72jcQVkyMTA4+O-K2fxE)oe?$l!v1F0la;pHmqlbmO&&h=w2 z<9YzpT3MXBoPKSX$vUb&g!V7p^OveEKRGL=zhzw#i` zfr5jAm}{nhQanQW0bnk&?zyc#qH@K-e~C(jH~NABz@=zXH4=uk6&w3ENm~&+kTP*3 zE%+CO6T%T4BAoGj0WtT}P1+LX9^%K+vL1uMG5%x*7Lx<`aeXezd)hfymG8Wa5KWjCV*q;U*%%S00_}K5~b4FrzD%ktCd^nC7-Tj{_TCVL?;R zbhXF(H&oZ2op+mdbJ!-ImC3xSacQ23`6@O~4MUf9 zz_FyA!?%sWjhiO0-U5WSoXh;f-)XCD!dt<*{ns}ix%+%5SqFMK=)VLi`pj?I z!LA$s_{}ExWQ%`IFZjl0o@}02U^WM_dx|t%91rilD<+u6zCDXsFJ8d>E`jpo9VQ`X=6r<7UA~|j z=vKKJwwae)W8g2Gw-Dp4gCM3M`Xy~aJDRh*kxitEwi}3jE1SB`snEqh?jxU|3GDN8 zOm1-n*Ifi>X^a>`i0|r744~zDCe4ou#8VZ?PfVI@#yX7b=vd3qkYXoW*eD2$m{TR> zsna@BS!3WwCtDehke5v)^Zga=(c^uAQrtifUbP@s$3G!%oLfq8PvWbrtS@e^sPDkc z_ocC#7SRZAfB$_A#{8q;w$0M20=T92qDmatB2e;{vtcfzmh$0|S&*{i-=#_S2(Ed0 zj6cPicedUC5III8Az?AGmjFm74z~pv1?-(hRm}$Onw~Ok_h|$|;aAxhMgXUT&aHg0 zzbGy~%jR+Br^kdXCn1=qxdDyTR^C5aqiHEDnz`o9YkAcRE05FCD4{b2!KA!-!P*E<%G^uK(Jtmw_c?-OXGet z<*}yFwp6?ock{h4sXR;oIMeubekOHRhE=QAKP!%;Ftzzh4N=SjZQ&X^st-tL~D5gX&_(u<{>Ai*}`w? zzdkDaM(h!dKM}?bIitATv$yqYpE$$y_T6Jo>=#zc(a}Dty?S-;QPs7L^_qB6KQH~WgI^eMRRbWNk8e!E{vj4f4c1A2ZpzxiKKR!LV(155&+(=n$H70H zT#&&9l7n99;crR-OmGfrVU0<7=?D4Upe&ypkQvMVW*GYmd0VBz)+GB=OQmQ{igiRu zGg3eIGUGz+Vba_Jer)-t+x5wWy+x=c`hWT(esqFw3HcTea)V38pv2+7_49}n#Dx+i ze;K zmI1SX(!^c=AYleUzHVY3C1FB%&Xa`2HJU8pPaZ!Bmk$`TU2*1u2~LZ#FkQBS8&K}L zOfgh=SkO#FAV4(qx^#@q8=0WMwWLT|U8dn%HiYS5T;u4^s?$TXh75V*wCLuHFbEO? zKB3U^wWY!v>@6T`{8a3UmF)a%`&g-_LxxA z?XV!nlj(xIFw(bem&po7DZpwCRkhbx$b>Z>kisJ+p68IX1XsvJXeggJ}Bj=nK&g~rMIGnPg&0S0{>zL6YmDmfuvLMi~38bdA_ zy)2#uFpM?{LQvwkmuhziNrKua_Gy#OfGJ$CAwDv@%|$5c@Fm+Ig9RUE0w1&ibLfkE z2gS~3M=uNu`QD_%aUkw zOk6j~^oWCgTJc6{l^a;f4mTpRWe2GdH4}~sAO^a_XZ9{2P;m_8yDuD}WFiUvcdge~@vk5Df0SDup7&+Nls z%yRU1CGLCaQfy3KEUjQ4va%p0V(pYQf8q_$tqQ_b(o8qMSjSJ0Y;#Fd|5LcK#628T zU|~~!Jc|6&rNDdn!zH{wB{mW9ui`8mf=R`Z$!yD5jdGf`_9+^q`oL*1E0MGoSo;FY zSZYe5Xth5}InBuFBtpgHo_OGy&R7tkvQq*)(vn)u!k|CK4RX@>&1{Df09M7OTOMeG z(iL6@s|H?vK_qiL!wg>Wa zHzYU37_^Z4!`vu8r`thyLZf$%lhDJX<|o8h>as3UwAdtaGH$@!?Kc(D8sV)DV&$hQSiW-}hMa3^>_dJ%3ar%cn=V+TVu2FiN% zUDYi=ySYYh6h$Ys{%!?@x*(hJDLq!OhaiH~DAGzBlXX^gtS_k#~nNLkUFjGAo{4_%Xe!F&Zv$o`<6zVAGSoXTnY{{(df z6$Uszs4<9on&Rf>!ztOfy^FxgV-L#=H`iEi=#>%g<>@#|+c41uTju6|qo9J}8|`im z4fx(g>4M>cDt1weNu&Ia?QlL> z&k83(+c{UTznYb)a-Qv;QUHAG@^}@BdG%keEAqzf*tSjq$kCx&*xi4uGJNsQ7-P%> z>=0)KxCcng2YDyVX?+#GCyt!5ouU6CQdLwOww4P+6%?Haq@g?sP_ z4KdLYyXWjDP2T}TPwuZ0Mby^-Y>i)_=dm|6h?Nu++pUwS@uwM3{dmeW)?|FH!CdkK z(%R#ZU#1GJLwPmXf_nA7(CY_-??@31gycmR4?{`mu@Y;5e2iTBkp2mpl9`;unzP8- z9Sl|CGifhksZv-X#jc5;rzXiyMx~|~O(D^q+%(Zi6wUXS`rxmA6pbb_lfS7$`_eEW zUwiQy?Kby(e;FMs=#PJ>vM(Og@TrgQtzc z{D2EsyyO$2!6@=|euB0sbLSG@)q!2H&Iym?yH?x;JacK1k&M9E0^H_qb%TX>qkUpdj#Wa{Q62*vvfKEeEc5vVe(B;C>D<(MY6>Egw;?+SRmJveoIY zpJHwS9MD4}6u;h;+sZm<0pZQ)D?{w(|yh;yh5NvvI_} zB0?W<7z;r0<{PuBN^YO#Lg^fZFXYz3F>(rGzwJV*1Jby2l*`RGw1Gwn8ar9D=MZWb zJ@)a?;;49XJPOmg=`OS1Q`XE@>2N8rdr~;SsYb%T`e#d7I5vE3-;s+Fr>ZC|oR$Hz z1R~D#96uD!{fv2El8gE*X29KnZ65)4l+R=`ZVo&ZG!+#y4WZkCx;{(#11-r(+JPa8 zf_xCpgSjg$DXJ+(UHJ10>_39{f0ajc5D-1K|67wBKG3Ob)$7MZ{2%7_IS8i&=0Ewn z_uGG5H~xo14{-bshyMQ)z%w%wcNf^uPeP>gLn(BOmMVxuLdpUQRk)Z3R6S3Gn;q4r zF;}&@)1!+Ca|3sP3M46fQ`hGcq8{oL%U@|98vP3z^7j#gzjr0Pe@S2;xshBVhYztYQiVhM2WaJ?V$aoxDGii@1WV;!;CE+ zF+K)&gsA;D$WksE+E6RzNTj9ol`DVFp^6?s6i;LIWm8>0io}iKSU{vAjC1KG!U8Dx zep^M*ps+I<7r`-?SyJG1glon53|R)=BC_x}m6sFQA2d^`}4 zQjt{T3k>|!xMgVA)Kq>>#8g0*8xk1ShVnpPZRS6=au?S^AOZ)&1jW=X3+%&&i$Yld zgQL-aKxQzChqRowQopDHX&+&)p=O969wx6(=5jQG6CWn)x0G}Dx|VbF@+^0Y^889) zD``o1&+58fRR5Un1-1x_9|0&q$ci6BTf7c3qCZACPd-bA(iM+7vra?B}7|*Vr@&^IR*}&?hbH;p=ICu zW&8pK$WoDhoXYtrG02-b5auSVEIPe}>%6i$;KZQ>-ExtonzQj&{}s<(8X=r4VghvZ za^fc&ao%A*i<R#6mo66{K`njs$DX3UcPyjoEH^i*n#a~G z@8=6o2*e*HOb_(Yy+HL}_beBLYpLVwi4Sk5mmZxMu2ZYefZZEm%`@?Z*Lod4l5T-y zcY(wexQ3xw_RkQy8)?H!^}yXoCmg9`{mJv4v9=wloof-lUM&7EqU+DJeqdkVhvQ46 z&ZjuRSH=v07UK?+3#3j89Nx;~ywt*vU6X76HJ!f)q|cLG0DjuYu8P0hC7by166%`9 z3U)u>?K4rV;61+?_Hv@aGoR-C?1NT)?cS?HSwUx>b|Z|%4ya?qt}CmpZMq*i==QZmM2cT9{CmJ%dYwijmDLukXP(zC|fmTGvil&|Sqo zNz?Fj=0O}tv;dGT*sMBranqP_e6E1)3wcZ!uhQptWuecnLud50knNt{a7c*tbA1oY zCOb>3%j$~kw56ju^74#2I*R&yB@;TD=c6#Ac+SgH@0ID)sU?=M;-U?3e|(F!bvkHp z+!;;P_0Rau>7KknuZk2AQj1j~K=(p9kk$^K+;Pvs(*ViQ9I{;}x@W<8`kxoe&AVd@+6#BiGYmD+~ zk?Ysf0SZVrzkf97u+3gy-krobyyE)nB>4QL1L=z(sA|Cdilk)Wi(mw`c4jFYXa|{= z=6)zHV*rrtR;%-0oI+x)8-#}bfa3iyPa0=Jd$x9xUHAa(1E=^@%S$Nmm@zxkeTT#? z&jUwT$AGBifmvx(5?QK25SnU6t^-)Z=t~I)?UtRKPSE0u2fl>emHMIEVX(8lYvB;V zwkAp=n=5{my%%8(#4kZ9M7Ceda+G+-T@Zic90NA#gp?%Sk z8+}zFg=3!SZ+0|J4Hcvc9SUA^iLdhd0vI0RiQ9dnghFcZ2xr8Z>%+y2mX0wjg?yw9 zIAHn8)*2TbBF`$NM=VK#xEhBP@3|Rw6-no!(V>=S(Q0sI!VXlAAB^uc;MbnX5)nJY zC2EEx97*VhU^$hn@|g@rmmiO$kcCk{UvkJ!>&|8kb98`!er^R1%FaWhfjilor_pc9 zPDwLyqp-2aNXLM~e#~eRljZ}6gGe%R0RWpaywQZxp%UXP+~Jv{w1mfOSArymkroTN zz4ZmucyXhGjko5US^>wRok+S6f2}b%{iNo)$_=tRVfaSry)4Xf;Pq6f_vPv_v(H}- z9{Bv_L}lf)^m)pro3XTirM{s1=eta#i|6pt{G*4_6sTs+2Vq#m$|Tv)X82hN>j30y z;oiAx*v2q=$`bG~s(=NC33VgSR_7B%$%p=W+WC_@tiR|U!o>FS0XNjfAq;=Oi!+RA zub4-YqDHKgi8i;E(7{wUvJH(HFEaPAwdB=lrD^a3B4mROiWuWrR%AWD7ByRoi}S!l z&;v=L&(`Y>6TC^}*A6$MDW$H0oLhE=(@koJ$;$mwn2<4Ylh0b8; z5~yvo%^@`NX2PaGqZcVA*p@RESvxgrq#^r?)JHV|W{)0uozo*@%kzssD1hMc3G#CO zdh3P^DC|Rs_o*tC3pRhVcBWUcNvw8GBesI>|*&Q-tD&TnQsc9LT+^sP|=;ch5a zd+G1fe%J6)?oY8?UG~^L*-0M0W;J2!5Mm8n<=AsZtm1K5>&bieruTv;+hN0j)AHMo zXerMPbQpr0vPfw>yN*7YX}~V*ET6keEd#Qfi&wB*LE(ZbhOd%3i>09QT6=nt8B5j) zbm|_Xe6)zv#C@d_llG@Nnu!Z=@g~w9)3AU zPMo-EFoW+fX&z$dDKHjRm|C^T2`Iy+Eo%pZ zG$|dC;zio#ad#0d)BsQq($5b=ca??4WENbSq|hzO6&CX(kf_-!I^!iL6M11Fl26N3 zs+N~heabZzsAb#)8bFV2mcMAgx)DvGIHRf{%9xzf~1Oh_#Um;fz#6}DEpJWSqz6xbAeg;wjD02QA@MyWzOX4XXRo`=jC$r9YJ~0Dtz&}x@@II?N%z_-1?K+ zsrQuHx#3xCXI3$^_gtDHj%mfu=ThosD z(WpOPrx79`#8)gkjpy*K-?LBHa|`&u19Ov;_cYI-bOD!YW?|{-9HDKK>Z4m)-8}+9 zrIh3Be9EG%W!FEt<@}t$wPtp!WPu2t*dhk^DCpZSt3?PGXmHxTT^Rd8wWeJq| z1v-zD3V?BkpjuPmE@k1FQoUV8u(%{lG87_OA5#jMc0Ms+COCa@iIA~P8|nLs@D|w| z-&t#T7)BB_AuUbXmn)#N&q5L+<)7zvs$SB=>QegT|FBY}hto`Aeit0iMs7-O?fOK} z_ul2?sX`wEP1*IBC?~(4zby{R{j>mtnsR>r1vtN3g>ll~fU zKRQUxALR+bl#y90Xz(BlJjR52n+5KfuU%QtXd6)O=rVQp)iu^N)pZpItV@cw9qbD-H`V2tp)^%ma4X6XJQ7ayP0;M5Qa*BSUN=`7SkfCrkk z$+W#mO2cuOM{;sI@}cqsWuBW?I;ykH0Mwdd_c%HYrLCVye`)C3=I8ggE1mTGv%SPb zjB(^I;|I|LxNuJlT!czjg;)o~X|+-n!@m`z;7@Io&xtr4qeObft_=hdFl4LCTsJT* z6f=z3E=eXxvIpU7M6`a)h!l%_4N4{rQ+Yo2fZnCZKj0tqS(N{A269P0}fEd zKU4ml(9UhHM27kZ_Oc($)si*EMRCpzqiNQW@Hp!6b7(e9j*=}?4pJF=7@b>)u{6|m z=QkT>baW5tYHO+oF`D^--~2dii68yj+wc++#7VwY>9{y5!HIgh4o}CBM_*{B09$`(V0NZp!jNajnm`|CSUM#R1Kat6lsb|xqF z^#gs?$L37F{bF1r!Al6@&P`0oT*?Tn>w- z>4R^^Yh-TB=PRUg#HCOi>F=rKa=&QU)bX=FO1!ObacHb)DBr)Pu>!~~-X?Qknm{I9 z4<545Nfi{ozwd&ha-TG=*Kw=a+^t|!TlVUh31n4~#!lW66l<9x*;(HIR?o|5IE5rg zWv=7HH0Kk=E)KO#DQhn&0W{T>m4WRQvi#L3#I~-Uo83me6LsIH#>m^I;oJ!$ndm@y zv%ktTTk?SEma2AC`|i@J*`_vv8x6G24^8rEsfeG*ca4RXPmW^J%EFKq@}A}%?4~|d zsFM2o9fMTQ;@-w7OpQsI4--wT`>%%h-HODGA*9OMO65tM)}0DB02hef>!5wHhGzro zj@Z~Kzd2FJcP@q}e$m}&R(qSd_d>OJbwrNe&e`%0>|kIZa`bH{2+S0H?4uuKeja`o zBDOaevyLVFPNTW7vffIH%5K-9NO_9m2qRJqPnHSKJ7gFgnC?TOS|=CmJ1T;m*30?q zr8S59LOXRD(TD4b28_42>k|Co8E71YaF2pK)}Z5Y@aSld1Y+&+_;g?$HZMVbbnL*& z;Ww{*%R803tl##Yd{<(ph#P?+Cu%R;h>a;&(Xmw zq3%d*fKPbytzZ+{)J&A2C5%4t>r| zkuapV<=atHNbi~1RUv5#Bom=Xj z0kmCI#QaoqljX1h^OuRge!Q4vt35pynX2xpxMj@TCx5-^hAknfxU-g_fMr19HlqZ# zx!0JgvP&^Um*`silJ1{qOLLeU))j3jn%eqJbwvs@bby?i0b9S68}He!DU~n8O?FYz zt|5BsuUz>#3a{xyC;ZaOMlAbUY=vL%tflA{^}p{louj56x!!rNGj2`he1YXp;;zuO z%I7d=_Smc+$=iH9E5v4cgw{M{^?~Ey2jyxXRNvE^JoP{{Elw?|@$%2${+K6j$SJEi z|3ucca6s@c+;o1yhViQhSl#Ew-|1V@;7154zM)R3&+T~TD2V=P6LEZ$@U?LBn=JGl z2Jzx=DxsoB-Nez7&{`rp15Y8RZ_w|4;N>|=Oka`HC76P5&eSFPvp+V0*?O#XWhE|; z=INoFvph2*A}VEJ-Euob%-TDJ_G$5U`SuI$GJwwX(+&+Pcv5$eHsC+8A#C090zmT^ zPL?ONt)L!X3I&9F&!SO&HvQ`GuQ~&sst-C3r3pLP;O&Ffa28^gixoEtG-sMS=auvJDZU7$Bg)2>{wc}`(2Uyw=`6y4G- z^n<7LP(-y0X!0#0${z>kuEPqpP{lha!`(36h$dz-EI5Sob0F7MDXs-VCc$3F{k`!m zp*MR>zQ2s{P65#Pb-6oOs_VTDr`F;`hyYPN>QT&*zbzuQ27i(@VfaKQkOv&Kfm3X7 z3zNE731R58_?p@6K`5ej#jd9Yn$j?5;q(IE9xUJ)ie)>|UXPVY84CD1@i=R8r9H{< zk5wIQw*&5y+4W&Mjn?r(4{(1jMn%xeKboW|y9ktGPd@jvGgW9yq1md<$d0R4u>59ur3{8Pg0Pd@Ge;NdPHD#ZFan?TU!~{$dp2p1G-hw&Q?RDGu z?e(J;4p>EC5z~T387+iL8w`r(8<86z3S}+`$Im?4D`#655 z%;n^NG870b=*zHZiTyK9k8{wKIL9NiZ-6_h3GJQd4naq$nXg>l5wqF90Vv*8L)LLt z6fZQkd-^Fx`~F!D%_nV63PWOBSc!aO67_9t&RM=1`toTzS(Q^lAXt7`0Dt(f9N5k` zvgiwvH;l7nh4pqzf7xs;TscU4l}aD>OCJ*dBSJ|=kgJ0*FLVv^$54V-2-A_n_NYK* zV#I)v&k>Rp@!@m8MG&?i9{Y>XMrv#LOyc9dxBX{LPu%DMN-gs$gmvkYTwFXdFC60j;C44Ees!T=KHO)3Sur{ z2%^g=sP{FfXxX6}uhAa6IJR}Ini3Y$39IB;*w?w(n3Az439AHQ0z~P(?}*xhOg1E` zlRlmk8c_^xPAOwT&$-53xZW*Kpf|6+8Y@yR=^=+zUgnJ|HJ$yBji=I`GPvBUCUrSP zI!x^pbd^3$bU0c-q}Kw6TYg{?q=k;SbjKUSQ9P$Y-3e~@x(>SbBn&-4)rT$fbUqAS zX*UHQZ+>2p^0Sby0F%ui-LN@!6KjOrwP1yNl3c1{U)t{xxt9f*z{H(2W)FXsRZFQW z=;=V#SS2I&t;BU3?!E;gN$DN+*aJL#QrUwuS|{YQPpt!NoB;bOkQRVqx|T(_i5V*_ zZ#bLC@d^5)@)z{{d+DB}GA($r#t-loNJtR`Y(j7c9*Lk3AmayS^+r&7=sgv6q^X=X z8MjuCl?`cSsGMdagv-1)Q17X>ILaSIB}(@WS%%C#w`fB%j-SidG#lR~LwWj}^_hve zXY;q!b;(ygfv=0DfNt}4wy`>*lJvS4WMRs|XkE$E!qJzDC}Gp%!rv|~w~fC~(Rd*f z%ktae7Jp|`0atk98C(N>$4_ogvu!hIBi_OCv(lPF_uO~9!YB#NW|+Kx9gAN!(Pt6G zxZk1pq^f$Ah{}WLPP}Hi%8`pYpwY zDj#3-n~5Q7paP)%o+c7!W)>j6aoXN4T>kX@J?oKFe`q=7{zlm9-H8y-0&Rq%3L}HJ zldwm}OeJ^_qdz1;J;IfJGDDr?I&CNoKTsiW1;9pYnrt{En_23V-<`7=^foy?F%xhV)JNCkZ%CL3-0Oqx_d~^2-b}=SU|pZX#*lCutZrF@S@5YOz3gwxffo zLc@W33oZf^cA1mIstBV+tu6S^7rSYU$KIZ`n$78mgk=!FgkL5Pmt)GuQig<+i zXU?EnlypRvMAqV5Ho%3&u4UH^=tcX;PJmI+zOtFyo=4IvkCphb3xim1ih8oWHOFp; zhOb4$+b-&fHtK1Uns2ntu9S+I6yPaFxHTu8R&q9doDbYigPb)k5dthrsXLWBL>&RM zB%@VgHfP3`u}K|bwWPCNa;zBPq!=<@B-cbl<+L@3tI2|z8Sv+o#HJDT_!FeR5xvbU z;R{`>3lFzI$;H-`oQ;>1o3xDM}S(6)lE_i)`Epvj+3^{JvtYn zRybu#l^8RQDQnns zWd>}(0fWrS%znf6xHyCfB`dbPF?*b>!ihh)D83X|E;PRtc9p2t7Lw~*XAVj9?#^AZ?yG{Mo4XXrw#8gcyBi-8&RHqqH1ch~cu`uBg= z9sg%>@;{~w&*hHGCO-x5K?oqAmenJ~MiBV_4BK|Ct5#tX7$yltCTp12O>)u1N# z#>zOQI`>iEvRf^%sc`_G=Gm}E%f@&z#cq48QhQy!q8{!Ihc^)Tv)&r$4{7t?HEqtJ z_!c7Jy`3?23GKEiz|Y<{&we2>WfyU4?T1ON!$k-t`jAx||C zuZnV7&$tkE&HP;_?Y4>i@af&!+ax#Hy-%$#of!y6cI3vq(sXY zub?UR?;IFWB?q0)05a5gp+_Xd6w!=L5Yk$rDQsVbI+fH2dGTf5%n6dm^k zNE&AA`Z4h4%yF+G+#J{tsYd5v+w?hcArbZL0&uVsmPga&`A12L6!!S$%t@nU{$UpM z1ffA`&!+ijvW}9aqfsS5hw*>YEJ%>XMF;M}CJSrFrW$!L0IlRHQ6;3PcMiWe*b{(J zOTbamR;gmg1b=}MBkU0>d*l0(W%mtGksg<#*1Dv(%jDRUYaz)@9O!UnOHQpiOv&hu zsgT9}tcI3}GjdU_oDSj~7VSC`#z<|_5X31cSU^Gttp!8q?8j6*c#+WRLd%N`W#^tA z1{#*yxQcjr0d9Kg!{yFUxBM9;$tV|)p3vaTe>W-oJX5`{EJEN6pV0OLG1foy>dbJgv{!J ze;Q)TZ`l*A=~1yF@NCV3;1e0Of9>?S0CIy-4TI|q*pr0v>1v{!oO1=N zOyWGqP@zSfljn+)yIhrIXk=yd@y(X*L!m1E-h`VQ&!UHhfRrgz-Q8}A*}ar_8?zYt z3))y@wrm&curk5t8Hd^Excr0Gl8=+6P^Uubl7+(*?V9dgLZ*a>CR$xpB4TfJiIFXq zRW*mQ1;j_0T@E75EyXs4m#fyTnUmOFoo1;%atr^SmCOHo4sE4HC+^c9raPn9euzzhN^c^_oE)|fhN6_=?38N2X&$?L^`J$X z3F_|y6SHQFg#+8`nmskXuZcIlg%~(kv!LfoOKHDE!~w`sJN>2-UkcAw(=U=q1TjB) zkdgbh*o)@pU*zB4NKQPyr66MI99btK~6$CQ=X0fk#i z6>GkoEQcZaP)KF&_z65Q!p{#H`It$>TMmiEmQuP{$HPjk6e_Q_Vt?NbPjWI|r0g8& zCkWZo&5&Vi60=HZW>xJcy3bPdhbkx!CA#gGpa(NGE38E4@1iX-dd(1~agz%j(OVLe z+q<-tsKuZ=0|9LgvEm~r?IG&R2GzImxCKjFq9^ayf3 zfus9}G$T7Px!!B)()EMCJiXR$oYvx-pKrr_4oE^$oWS@$hyN3 zr7ur_!}!GrV?APHBz)Ki-R@!s#XU9ljeVjb#L4`a2+eU=g>Nc|g7}BV^AMebi~$ugDjfxksX`ROTBXL|F}Z%Rzj^P+248dz}_nL;S?(DL36ZnyC7 zfz5ky1tVH~RoN~wI=Vg$gNo6Tcqf5_l72yhRC3apd62r9Nv~w`(moNblYm>QFux2G z8D)mta|FAu>685h8IEt4s2I|DiUc%reKi;CjZUQU^gNWR#=d011H&`$siH>A4ab9~ zLtsUjGL=KmI_-;510*G=^lUNX`gNr;6*OZBn2~tSo~S(&Qa!Gk_3ib(P_&>NbMDr+ zbp$9FWJQhH4(KQ@aIHw}6u{C+wduHBd*Je+?XzCzlAdcAn`Naqobco;yJ&^V#5gN! z#@u$ySsQpyW1&TUSLBp2RNvX8v)U3&QNwXxvF%iOD(!NXvEtI;p2axl7#AabWPy@M zsX%8`oEm)u1CnhM$EyQ%l7y|jDNbjBE4!$)5qax$Xbmd#q9Nt8G@y|HK6pCV*L2+> zMxfrdsAO4v_srrz!l$pKLD#xOY@)a$+k9z5<~>cLapBi`s**xQLVqIDE(vQ!Qk!Gup=yEl+@NgX_xvb>2krqw^d*TUFb*0nl zK()aS-5kThJ$fk~eY%vo!w{D9{P-bFSY+lM^t)ng#5{Hc5kN~JCTO+T1qg6-Fp^Ud z;N0_d&;-YGZz8(7@8eFd^M#XhQnGDHptVRe_Omy>JPFNc#ItLI?(CNRnl z#UZfR>UQc+0lN>JLnaCvCp4$&Knbw9jRVH5BrY%fD+Vqv=xd^EPAp&@A|_5f^S9$o zQI7R3rFnopG(e-dui?)Fdn=GeU7z!x6Wm)^b`5YJZ_qD%`1kNe4f7bY0O>y|Hn&-S zCfGGzb?f>_=-c5j94pud?_`8GXV`mBY#wd@%xvtc1AlwsUV?qX!hgE2?REhFGXniQ z4}SNO1^rI3dDVyb)Y2x2-W@d=W_P;Lai%bs+sSH_)d zRm%4)qXr;f_tA}aet$3E%Qq7RA#7J)&p8~u{;#&0_I3%+0n zW|4klj=(nJ%dO;Ea7LJmXK(dUM?%*ZuN6G4n=q8$E<9;%aphCHuYg7Y^CpXTWrm%YVbsxZ4;mf-3nyrNIAA-h00>;o#9n3Pz zv~&in9sL!uFQ(9cQ0fVs=Av^vuVh#FKK?o{ao)G2hwY55GS?Y|t)HOX3JrxBgYL5t zr%BQh6CY(+$%>{l>=T`jl#dEi%#c@3zt=ua<>^d4Uup7}+z%!1@idE*W1va-Fd+zw zvOEA8!vW~=%a2Xi;P=Rs^=sF`>Mht$Uw-d@hI*hF0%x)UuIRdIbc%Hs0=hmz+B0buza3-~EsxFn zUb##@AA&~@C~!esG7QZq`HU8uTeb7V;VA*!X917I0WAxq1e9c-TzqzA{J_xu-5Y;T zFNZ6>3l{qx*U`^A-GiQ4x$jjoNVlNYP=iSUzX8zH#8+y9a(^H63ah)L?M{pv-G?0w zg1qKw8gK3(^s|R=UUnxIn)$D)RXtjmpk!nTMXPvDYeqjAub5r zAF@%~Vo@`_&xHB8*>!JqBRHJa^IQ7`q3Ai@y$nX19IG8NT?+vpDVF~pcW0TOusL+6 zP0w@mq~kR^1cX}s;O=IWFu%a6-ZSYmeVa_4mg1MxrPcKR>*0&-E6Q{-i&1VC(GbJ4 zb9A}mY&YceN90vA=76hw^yAkEphVg?3#$H+1^RsMy$|oN6hL1W5%_`P_unjiRy1qC z&5c;oF^q3Q7G&qkYn+EvQP|=51R_+w*(5^r%uy7F@k{0~TYW9C9rHx?6X9HOHVUch z<`3Y1OS8^89-`=OZ8r!B)+{c69!bgh~7;U;K!Z z5Q+i@+E6q&DNh_4gFn)G6{YM5-%8T6$Yz^-}U7(J;CW^=1DuCnT zzTKRcHr~tbm7^85qfSggzG?WH<=j5CnaU(P38pwmGFFc|Z*-WXHt&7@MQHLyC>QZ1 z|J`k1t-r@BbfJO@6GJd2PF-rR<0b?5jl^!2M@qPRg@@pfH#+OOTHgu6iwdtqm(8ql zZWQ+;L0y#4UraPOb2^R#haG|JF#z^-<5XAVnVd6A!oyeFR+{fV*Yrd6%**<9f9S<) zw)H}}ispW-KZJlex>5()=6z4l*06bG{*}H$F_LS+k;eBF48diElG;+I8=eXhZXO>@ zdPDNS3+97(8;2!clOS+5h0;-7$8~tIHx>X z?YzH!%D`U|0S;3-~JVC=|+8)OhGQR+JBB3 zJ2+orh{7s$WH^$`(y)xW+kEcOx%-IQL1hi-r$t*6m8K2*6c5^00YVknpsy`3xVP#b4%}Fo2oAfa88mb5F8!f$h2f z4;VayMo^x{qtLBU-*#F>C_YVDfE0Wd4S0?%%jk{NR7 z$X7!|2ad`Zl0gzp;k-#@x~9dqObLdUyNTA!64ekb`g7aQ091x9Y z*$Sdw@md=kvr7z5I5dF69~@(y+6xTNv1bNgl6vS zCqK6r;XIfv|8je81DJ_CIrf&qroZ(S8n6W!V0^ZEE$np#bGKRz;jreUb%3v;x$G}q zxTW&Z8X$d*-2K#q&lxyj^8t=g+OOW@fsfgXfPnyw+VhZ&0j%D)3IiBu2UcR?I_u%lFBQJ4iHXiVZ~=;;|4{$EE|(7 z+OJ6I^wLaO%H^H2E0==8rwF#Z7b0eBjj-xi_!r_|sT03g>mRBeq*NOYT9?cLktAE? zIlwbqLPHPQPHiD}6M^?SIPPQ?vfK$0n9)oxtkVseTm=ZI{T5!*>PcpoO*}Aa`Qv82 zG+Qk4#zHQG43O#;lb~Oo)y1Wu29#g}~%-F>yj-M9+CY>er<^CHw|-52{~1&BF2Ph!Ogg!6SFv0x+6G^`*F z5Uj}N2Njj4kv8!9hrO7U|2r7i%MJr}6;+~^a;`tSUk+zO%()R78xEwNJdqKl9d~6` zJM{UCPQ?AJB(HFWn7#NVN0jf2BU7+oy9bQznRaSLl1_3^VnUL&&OSGaFACu~a&b{P z9@*-=AJF81e`eW_p%b!_5X~`bL7JE{muCdUp~N8e;OM2yg+1$&IxNA@{p7NOLM6|T zcSe@}0>BL;m9#a0&ggazPi80j?Q#HIh;Saj7XtAtO9Ft%SZkS`ltXQ7KuMP)$*>%u z^C$pY!}h;SoJCv18Lk4BK)2}$j{{>X8-=T6MJC`BOc-UCHleZo$NQV8@RBEG{u7ZB z@S-^52Nd-cyZKcB+xp?AR4Ae5s-Q*9s#MpRj_b)o)oQCbb_KrwY`p)R;M!(Nr9snn zHS)K&CxVs2tyvQPaE<9~t?J8k2IL8i%yh=CfM8WV?1W)<)KBd9i?fYs4;(QoxE^}t zh7hAIaB6SL2Z;XmUmnoz1ZL1b6QYAT0AXBFud=4AK`H5!(SPBM-#Ay797+vp>v*TI zHX8(nlY=daxf8IRY4Zz-Dwb&~_-BH5E6AAVa>*pRx@y`a-`Y;sHxAeze9a94yD-lq zp$W>wTPlHea5mnRAY&pHA0?m+h=t(cBfl!-Nz~jiQjk-*$3LDQS~GvY9^!E)0Cc}j z=5!(V5-?m6ns}(|ujJ930b(2z_+tg#TPCbLt{6y7=7S83va&Jk3^ zmt@9#?fX=bdZ;*J?RP?Ga0d~`UuI3?2A>|g|85tsepBGw6n%SJj6jlue`5Y88UI(x zg#-eM#`xdKSXMkn>GcQMOal%C#0dlhWNFV}*AfMeBnI+dL}w^}kpAeO;ZmR!CIA^# zio*;SOvef6{qN1jGxQ@;B)R z@VpyJhLG~(0SunfI|Buoa$`jU4fv;wD~k4ymaHunFFsjFHt)wKbnjPTqlR^vMmM;U z@LX}e(O3PZt&@37M`{PrC;Ip0zg>nbCQete5&_kXJ9#I(9XlluxwzQ$w9b|fx0eUc zo2%;$@8=hW9^h7x5=g2}h9NpCv!c`lBfW5NDvY90at9D;Xe&C(xREQcYQR1y6%2Ic zt{h6mtHOXd&NcI+y`%de#wN;AE|wbj&ZIUR?M&_%GRvmfC>ox?MHu7eSUUR)@1EHS zu2-6OGEMnIH)i;iL;7B#4%)9wa7E;7a$e^UV&bVC=n|U{Fg=sgB#1iO_NLx76ccT8 z2`TL$?c{?bY^4vpiMCEWMnGiqXD#la=0ZzgTGK$S6L|-1QMy$0pwI@i7(I*5$Big# z{-Cwps)qJa#(s5vA`}Py4752`_lyffoIQu_+rC%#)vVYG7Rdwa@T|H6uVYDb8*%x| zYp`^30{G71HBJq8{SK>`2{qOu?+=AimjKMaeT4_c$|RmSfi|Mn6`+@q#l57(bxi6{qH0;vh1bcnyD86aVBcE!x$mdapa0;kGEc=C%nT8XmqF z0rt>DQXHFIw`*DsYv6vuL}8nHq{1;RZXf3!9C>*nkh-DC6%f)r zC_A69OdZj}e04cuXmP=eoBq`a7EPE0jPAvv<8)+{Q3J~9)dl;&)dd$cj+#ULX2Ba zd=tx9px}hK>Y_h`%s3F@nMSOXPlCOFK)V&HH2JV-20&!OxNb64&JXoY>GJi;kU9b* zW-nvW&xnOppy_m{?8GVNHMT0cFUE@B<&tdBPDnh)*a+);3Mo@aI!`vYFsJ*dB#c9* z8~Kg%b zVvUDNZS_nZUZa2H37x_e233_-WWU)$m1*sS*!L|YH^9>>*9BGOmtwY~V3|5BYw8`7 zO0RGM7gad>s!M)>7L`S$8|m`(UugRO4mJNfxCxt?SlIp`NNrPgA(5z31c$kx0TPCG zCN`!{WX%6h>#3@(j4Fxt%?&LBRX{}2T8XJsa10VHhRG003lVG4D@{agKUFBJ6_U0n zgZfl**}(s9lvMg^@YdO5X+5!!P)bXl9$WrsJN@?lA(yl8H3J|N03O{_M0_^Z;V^eK zCZ7#)?&6HmVW8hN0s$EH1A31L1|64rO+cV^7W9j~9Pxj8378p;cOiuJ19-;DIUXW= zQeWQM4Qj*Z8+7s(VJn8W!byE5o~iqrt)!Nmm;LSR{oGScBrGBYU*Y#yYE^^nR1<#R z;|&b4*J5zskzYi6bk8;t^)8To`9?4O;g|u2h5{O<6cmOSq zRKcO~W$>6@OY_;&4L1rxH0-wb&+zaNt1%+7H^OkHKrkE~1cpoH4E>Ed zkjmk(FbDzzVVBlg^^qSYN{ih(z_?Ki=6mbrq{e$JVo|~zyric4rt1_$OJ+UJ-Mc(3SP2J zv*AG3raPh=={%vn({a_RY*#Cw+KP5Bj3XwY$+OO*TlrVURMxM0rtNPIxFVof3XfNb z^<{oN*E)`-m`kFrJRiruAQLVNZq3gxuYygH^DtR?JvXYHXm>cJFqSBu@6g(|)r=bf z*8e3pFn%ev1&Cu;KlJkJu+EV0JFR&(!t$c^trMCV(g{eU{ixSM@6N52DGcXf@vMix z$0~xXMO1+deSW!s1;LC~AzyXA87blI?2KgfMhIUbeT1Xf`>5A-A``uHT4^ z4+h7?NLq10U}h$Hz+hUm(qi#JnqbdRCW7&CnY)I^z{ZFy9vIME^#_A;S*%i6w}UXPMI;A)uWwH!9-75C}Ne(9&5_|Dx!RaejX zz|}{0faCs77IS%DJA7cL;|}pv9RA;-7|t#^XQ z2OVuUHdNc`HW=1kiav~Nh8)2s1Jf~HjrECchYoyye(*{t;S@~-hD}pQvrs}ed?}?U zQD18D()8Sw%M}ohuA-ubal1jy_Je`@wPX?{T^-kyo@iMwm)jyvB34Ot88Z^qn%hYi zhqdNKge?~+)<0~a%xVGFj>m~R?L6-gdISJ#ct#eqyI|_lL;j( z!4=rrDofT#-jc}D$%OG35r;)W{ef;34gWYC`XCn>Iv79_&_W(nNUU+*F#*`rPmu}i zcRi+bqB=!@7yAq|i9F~|JLp(2g46uG0Fer7Coa;^H3sk^KR}`sr!SycuU3HJ-Hk*_ zzYj2Ua^)G5?Q(D9TUDGts%w^bItG zIcY@AJ`wXuC9Dq6GXwNWD|U!auo06joCic)Wzs6mQ6rSpSg-2d5G4#Z%Fn+YKYx}@ zLz|Wu=y2%S9$up-2Sk8at)4=bYo!f$<(QYh1F3tp+O4g&_JT2uUJLj#<~A=3|0yYmX=`>dBe6eJ z(V;1BbsKQ-xFN!KNEg<|WGJTh4dsA~akz&f%U_9p zS52|4*G;%^^^tHpbqcs#L>5#S8F(dfj-~H76vIb^8E|f)_YN%9%sJj5K)?;STvrJj zlW5pZK1ggZU4$Y-MYH9^mQQ@aJvvs`XXRER!2i_(szBwP5dGmOmEhaXO2S2$$bt(+ z{x;=*T%qXaY8h|V2On`MOhGs+yeR3j z%Up+V8Z?^2`m3z5)jTWsFxjH3iosPnVLL9}UYPNmP3dVhUob&zi0CDfvJr%!UG4ng zE|b9H;+fK^ox-1|<#)D1PV{@3HEYRKFy%Ly=79@#d@Q~D*j9TWc|Jf=B?Y-?eHpx> zS#-=VhMptTY0H{MAn1(maCqBL{yNDuu&{sBE_AfdfeJo?N@>U!bG&$EUe zaW^s7FOS*ko?myHVFYUm)^T+xw=U_3^@@^exA^0dpN>%Yw5ivMm1uwvG^m%TpW#kQS6FQsGY&(Db~DydrlKIXBb)L z7KrW0PG9C$M2tU_R+pRKY9VG*F{QtHs~kbM*5L*NmLd>$3mF^??`qa=M&AVkDa2%V z-gZ*g%*nQH+Z;avRpObCtK}gB;X^1u&jIjGTE@sg4|D(qyQv=x~*m{vWd5G0L)K>jJE_ZQHh4Y1_7K zowRM+w#}-vZB*K^0YlIYj`In6M)#XR991DefoEv?c`J zj%9d}9O5L)4Oep^EmF41edQ<0^BURIb+idR)s51o%GzCZ9d#8o4MjyGD=SNleLY=e zRSji5O+_4KhS5v|$Amk&u6w3v}EL*$!2D>dWR_}w%+xU9&s%RsA@ zm9o8>*c^`=u~LE+wf+kbVJGfgm1a4)DOFra3GD#?C~kx`gcN6y(6P)##<8nWmXR^& z-en6~#507ZMsNUshtVIXP3EgJZ4EeV&AhlW>^*?4CWTj&FkVQxR0ZdDvw{bM{5?5p zp!Kmo`;orMQC%Lk^AX_v!j=EaX)3-fQpR2JPm)}RI*{fl(Rh06eZy=bZpW_ zyAuJn89%`~XF&viMsTa|D@Sl^F<*~4K!9HFJHu~J_3>|7^1sc^yz#qYgQC`{nWOmV zPxk@ACK`py_kmG(0<&$OWv-)5Lvi*T7d+;Q``J5wfd zN#Fe0T^k&<(xo7%xqM(}BZ#?#`3EC^YJUPyXV~w9Ef!o=Sw<=LONehP2-9hbqC_?K zZ}Eduy1b**1*t5s6+a@;Yi=t5Wu#W?QNky(mh5rg$prj6o2XM%@zqA$G0H4!)Rm~Wx zYULxN_&+P9Lr`wMDMmoB1m6_rKcth(LN(Bn4f~?*e5Nfrk1KvHK|L{D@+F<^BZ){fH^6Yt}EXIQ{(^ioFY?kS(Mw?8X!~Q zhnXR>X=kcqt#iX&;Tz(@AqbL_?+T7VDX*-}7%s)dl?2%hiCT%9Cyo;wV<uXo~7CNV(BAiR>KsvlDmPA0{n=lVh;1w`bW3F zbzA2saw}<7PL&x}7PWRJK*k`aX9E&R9b!(;c0MqQN@zY=tS@|6BT9UU6%z z0jW=xO(00lONN1e!r&8+`oRPsWqV|-ObuVAf?G-^UZF;2SHMR=d@biSALf_|P16FC z(Dhnov2cR~N8mb_9EWJC3|JeKaCREE)GBK{ptUP@O-CGS@V-y7LK3AfZSTi^V^EMr z>4QARw40gT1g%UhZ;x0?QCLk67{rvaw$*n`=GHsJ9pik8CgOmbfsX{h^U970%yi#? z*$i>{dBca@@ZN8E{%L}VK8%_~VPFLA6%lx0Uh-QDwq|(b4tBx`efurx84-Bn0zJ4u z1u2g*7>6Vb;d`JMm|<17=M>M zQ)V<}8S6KKD$!=P9)$r|r~pj;xr3W_%e-Zc?ZInVFbe-1t>ch2A|O((i%DwNWDG?v zO!cC3)$s|HNmbg>Vy+a;@oZD*M*rL0E3DWzj-`K)t&Zhf=Qvt5=}|Ygr#ItJzh->0 zVkr_bEr(L3b63$&B)8X$&UGE1^K%FbuCHJZ1)E>eKFH!_5qTcK7c0L}$uEUZtwf{Y zkcp{*Vu3F{ftl*6!s7CzJ+MelEjIkHY4k@!I);s0Y-hiXYg{{2#(6}Tj`4) zv)=gRV-~IbxGs%Sq)IVrg@KRns2n)96TFp2VYo=1tm%vzk9Vhrs#EvUfkj>hC*YT< z-*Ekhqu9PnNvab-Hjt%PW*N;?tzH|i9$lsyO#Z1Qx~!`fb$4p}j&JP` zY2OqlD~$xez;=ewb@M95@xgP(7Ofu7qAXNPFBg$2d}1LgXsnGuBstOEr9&yt}%ai`>aZr&a>jVOl+FKu_8i##oM zo(m38qb6;Gql(&N%#%Buu}DNmXDKz$$;v6Gh4C7Y*c_8o%386dFZ@N96HW2EIW}kg zV7}fT4yRM7++_$~${;4m%G^o=RFisZT9B#pC0#X zzUR$1euA(2XG=h~CeN6#8Pl}zUIgp23OWicxu#6R>}tO! zI);Xc$E^z_iY7W2Itwj*WHLqtVIqC}S0Fcyg0s@ZNK<9)%;;DY3W^m!KVwxD$Vh}< zK;W4v9|HzxWkxCmn<$~-kDv}LvNScY${;_bDY-M{bWAh}X7#jCtA$4TxT$FaQ;iwG zsgoIDnx2M$4Q3((I@6{4Y8Yc+0Y6>UX&2}^`eX1B*PN?Let$j?UG`$*S-}27dnQ5B z7z?JQ)E`tXXO(H|cUw5{2P(MSXZrN)#HLeN8>e6?AFUiU6H}>}IBQKV0UY^g48(#$ z1k5}tYxWX!=(}R*UTiT}&5rm)bJ#Zk_>yRVR-vx6qmIJkM|L>MHjz6WMm`23Rhs0r z{x3B=A9&3QIAbdd#r^j=`r;0BYbCQ~l(~RPr2dm|tP4VKSrPH8kl%(ZsX=N)WQ9Yj z+O~l`Ox_*K#tt*WTxq9O`OzHy8RPQ?#N1kJ*eu-6ePH*M^wydIuSH86ON~+hy&SbB zP6%O3)oGJ@K9{9%(2a7j88ozD2hvPl2K~eXEG`}_0Zux>s9v<;al>NA7d^raJmhqB>RWH1bht+Y`X; zX|cluGj)GOi!0a0Pz5^n6xux?vQzzlgr$*mEsK*oqA)qRJY#wPCFunDGO;{nS$a{} zn#`&Ug6u3mjMOWXUb;C}falaOg8fD#SU_9zqKFAyUNU)KFVajC_Fh9E%J7qya5mRl zkrdeWi-CpgBe67tFW|Lu4G#F5lJwp&<4~1r>KQU__ZRBDl~@YQAACLl)$C}A^>4Qj z&~?YRI-&sle8j(;Tpk`ToMd8AteCb{OT_wO8+N|p)#wo{>u_pOlLeRgKb*F~&Y66C zW&Pg6-A3#Z5YKZ-^sC?CBR)CGTfKAN)sHC=&ohcLHR9OTgXYqXDKVqAh8Ja_bF)Zp z!D(MAa2iuYevR!OI93(`@C4sMZyx8(&Wdtg9u3@jP7~PY$%%``kf6olj2{$nKS05* zlu-VPuG7wwypRBGb=B)I9sX43JY<4Tn)0a22pE-Q)LHz|blo`saR7%M1O<|2LSOH& zN4Hz4W30Y?UfCVesx^MCiTd1lj8KuA8H^%QNK)Q%y{X#=_>&Xe(H z1y#u4CTJafY>5hGp(5o~18xv+tcDzMs&c<4wzIhcn>mfqDb3itxk|YI7i6W4BP~rD z(ggQn2}_x){hCuHp#6^vgh9r5SHpUkT6D{#qOMH=sqp9xYBKB4u$8kWR-eU%-qU78 zcXIs8xP-9%M!bn`!i^a<5tB0U==vYZLqR3~#H)bd84$5XpttGYl07*|n(Cxo6+1CW zUJ!}L`48O?$ovJX&S5mjbFqhD>ZDNaq=g{AqZBB!=)0mzOuD8FM5$<#?fu3U+^ z;=6%Mtnnp?)pDAT|}9Ko#(+M z`vXkyrP_GoZ)C@EwRO2H>q$IU*m&eSo#%H2@A#AXZv^M990S{8Ol`74M22NM_tS?~ z@i5=Yf#Wnr0Znx}Z6D~l>lzbu&^)$2f$jV?KUB2ARisJ8_P`z3Na?I2!OExnu4(zZ zFs`na{3RZ+@_l3PUtoLy5uAOMKeo)c4p}$^dvBnlOLed;TnfLy|Ma<<{;>Z!F&sT2 zI6O@27!-BL55o#rN#9wqt?vu!xvog?>mNE5PyQ{B2H+JP=M{C(#Sdhl@gHSeA8Kcp zySyPM;2v}>@201uyLUNQmj{A8Tp%Z@tWqf!f4l~ z(zZwhyvuj#D36i>TyucJ9HQo)xgH z=6?MC7L zq4fW|I}HsL1q3=L{JmRG3l#+XpQVR7urMh6?DuZdsR+j5_mBY@$UnN!#A$@t_z-QS zie%8hQ2#t8w(Z5DN%NVd;Z2iM<$`E?S0)n$`Og`@w8>0>|2+c(MT-kRRyWgVHs%dlgzyxUo{)otF+h$N~faX@P1q5_T zQn~^v$nfUoR0TpMt%erpC(|sQ%#lmGH5+K&lb2nB0{+)|tjnKREnrpqRou}82(bs8 zu?LI?(M3fw{Q1Y*F&|>VL>X@l<}-hDcDRmnyYhBEZcu(eFrp9aQMT?$a^Y1BDsxho z)l#0#;SDrEdKJ#jR_$4K0G@uIwr4XdiZ}@W>*^c}8P?TQxrn}++%5jcTbK7$J?bpcio978}aF^`H1HcP%z+{7xDA`bO z@DI=edQi!4$Guq019WLCi0o{`80=cAyotehb;b;J*J%g>>^@Q_e4-TGb`u#K%ZYl5 z`QG{}+Ym}+#!W-V4$_Nk18B6;91L}+OQroFATb&6rE6fV3Prp6YdO-vWUG!Pg=T)Y@VjABM7f|>Qs8A{8I!@-7+9VjJ| zhTb3Q&PL-bcv&5^G;rRvmh!j^8R*8+C+OrRCtyd?>S%Km(g0EUHavL@H@!Y1FEBo) zKjbcBM@w?JF?*iT`qH>d z1cP=quSA#h)LnS7wz|pzC*(BRsm>vf(NVpzRW9qkc%RMz8fUkoXy?hcCA<&eIBcO~ zb0WWZfi;A;MgjYwG2TRhX`BteA$q)?+<%{w+CX^^(v)w6;84>l?~HMXP-fU3ghJye zEBoX^zNxvT0&vw`mn%`$Tf_iCaynIYp0N!3CrRgMnnivj;J8A^L8CEpy%%k_7$Nr5 znrz#ugoIg#$Bhf-Ui}w**19L}=Rs!3_H!zu)wFL(OW_deYkaMt z#}`U#a)qHizM+u50jOJ}RIkVf4{Uk`_P5}^Vb7=DlM4v#LIOq!euL@{EWHIhFFAfh zF@7bKA7Wjn2&!6J3<1>mi*o33i?t|k8O5~5aO`FK@7AQyi;BE*w{*Tf5(w3t`#`S4 z&j!;)c2@vgM>u&}wqNw76Rvt9eB#?T_^o>Cr#hkS)6@`X7euU;$zUDl;VYc(czd)I zei8u|UqAkd)&DoJDM*8WLjQn-g#7VOtVLM`2!4IX>N&_XCMhWLG<_dnh_(zH3>r|- zeJmL}`RY_^_$J*dfvsW;nppIQ$XW z00AJZvh&H^gaz@7!?u)`L^VtiCWYVuc_>QM7+}z#F z+|~BpD&=>2+^_ha1>(G?JK}ooo-p)*{n>qf!KmH@IZ=P+XbT+f3$(w;(|<|_e$j^! z41C>;eR&B55;&hz=s^x@^PZL=cVC870U#HHe$^n`r8rHDh+<*b(Ua2i>Lfew9Tf(` zWJE~~j!-ku4_SS7hejRgRPPpsMrqNjG3(+j)a)-?F)SZoL#^Dcax<{*dtGfS1n_VE z8X40 zv8yB@alC-TygP$_8V7tD0_^jx0ny{eTgEwjp92m2fEzPVSVEQgXcMW3lHrH>D)qY0 z9|f&t8#_yUoXgkO=lCc|A3s5YAkV!>ZkN+DrE)BD+R9r$p>1NKB652S1$~wz&7BwGPuj~v1 zcRY(5B`gXB&xej0f~r#xk>*alkbEl^UgyHVJ0wA`_tiYir6_5`1;0T-KGVibdhESw|y?1%LAbGq1Y-v|`j(P)LN!mAMT&}hU>c^A~^uu!*E2Y6)H z6Ow@wN&Zk27Ck6jD3Q59Zq)o&34#W#9+N>%D>IrYE3}m%D*~H~?S!zRY60!$rUIS6yQ=POO#_y^_(%POn=?5 z%6V?R#!MU1jLs^rMNC?lXO-SCZg}iE0JjCeu5)(_NuhC>odSkG`xY^}Q6fjS7*9-# z)?ECmIikVGi6^Zn0V|$}H2(CdL7S_jgw$zs^Ou2-iVjM)QZt!vAq{afmAs(HID`BW zxnVh;{hu3~I!f zifph-+d9SP1iWE17biw4!Q}Z+uM35^kNfg81C zX3xa$0Me>FMp6&$K@#L`T0@2seE!}PPJo<_88f)e;98NXQcd2@#92%V z7&*${P?wR%&?9Bj7Sa@U2J7^b>bY0t+!c5Hr&cGs8=&8;Ny@A|NR?(X_CI7@(wa67 zn|5HnRXqwl@kr=rMKnFS?06QxFA|ZMJ)Td4zwTUY_RSZ8j0zGn)Ku zZIIX=Jp|-_6qNgMJ%b-CM8tbh4yKiNZsI($qf3SM1=>$I@N{Qs1L%f8hfO60vo;G| ztz3%j44kN`4fCQuI0=W`ce_||>VtZ~BXPZ!i&jZ~^uydyXE$g9psH{Tn78$l@LCx}nZ5 zcJ!kroN?|BLQfvp{4@uWu)lAxbrKk9i=bzeh~>tT1QHacZ(=Eju7?QfE6YsMgFM$c zOmV|#nB@#a26>vuv2cKF>#GqZy)nd0J<#;TR`s$C*{@AXlbA>2(a)KEQldE8J#On5tmASgd zv#{Jb#*vL77c(wmDJpyVYtjuN?<_>)%>dXQtDGWkJ~~2;8f$$G$_Ws})&l9&ABDJ~ z#kfu|gq0hAA;&C_?Zy?r|)n+=hNU>BVS;3ivSiDEELc+ z=Jh_1dnF`Q-{WaWHNZcs;6J=4)_Q0@4!h08l3~$ZDGe{J5*&E(g#cCn% zmp>TCW?8<4uJCkC{`YvTFV5viNl?K+zow!h{F&3nX6 zI6IHf&K*9@U8C}NsvU;q(f3JnrYqr|qKwcB)eieNVNYmrP=5vZXd?=gkuCtp*Y^nR zGwSF=oO3{g*wMyUaS>vkg!1m4Gqp1^m5dD+Sn@AF@5aPoPL-BXO z@lENc^j)|NUaxxL75^ZbBM1c?`BX6Wj_rY4E$9Y~^Hi8)JQ6Kkk+1N!=VGwBE1s5M z=M#(@{3u*ZWTQ?G#$1X~#P{$aQXLSY&^1+?l5KMVNCvK<+t=XL6Md+szXE zDM&>?Md9jsaK_w|=(&65X7CxNmVh$w{H@#%AQW@w`~i0MM)(dWEWulu7P76yBPe+N z34h@syn2gN*fOA(5%od26vx&7S>OK&>7z1swTRa#Y%n9FOaFj^S}$+rVH*OZTfl$> z;y1~)s|@;Lsoft2wp-ovTsJQ2lX<4^eDTrrv~3^oLAx}=_4KLfeXXB!9yX)wdy9AT z&#$&!sW)I9F91aMm!kJ|-Av0tTDjlvhGzk@>kn(EhCboPs@R^dgBvJ3@3!>3QM)TPxBIcZG?!%G&WWz9Upp%$NSj8Z=UBN{?1xS`1^1j_>7mYoV^kQK zdV!B-<#nclDL4v8VzazZdCkF@L2zc}eYB8|RCtx*ZGb-axzFBX8I7IsdY?aby%wGf z?~T2c$COVsV4D``H})`)9itdxzpq>X<(2ZPC!V{mWR$!q>eVe6y#%QC0jLY#H62)z{S@MmL9aiUXQvF&2bYdPTnCuOw>8#vS1-u(IB)=*Dvf%+cm|b zkV$L=s$7$Fd!p;FJ&<)_ShZb8c3^97;2(mjig~PAURjY%uIo=@X$7r0c8{jpqgi~e z900803UH*WuKwxI@k`T6+k39QyXcY;isgYhT${L2&6?dd4qaC!Y}O~45vy`tw-)0z zXps9QaH%n4&pdNopAEqJRLBEF3m3mepqpRaOU>PaZA?_rx5DY<_Econg#GfyydZEOBgR_4OCAXsF|BIa2q|<8>p4B z97t@ykg5Nk&_RB000jttL6#uDS1k8R9cGZw8;b-Akj&N%rD`;MH=dLAsZx5ZKbBVaEl>M}SxPf!E1B!6%1TQ$w;f6onARZ8&pBaQsp4UEJ-g zfc%$7fA>4U4bgG0V?M%LIfq0-)C; z2t#GJbFeitNy~twnwB`;FP&?uhaGiqixBjWCe<+dsy@(?Q+1Zh@J2R}2H4L4 z^&IjqlJ_Kj&l=2vg<(<34#Is3V|vBUraj~gn|eeX_NFlS;3$M(9+4Z6Mf(09bO&?{ zg1>=rOP0R{37)$L)VmCumE?UvT0T`7|8iiW$C3;nt{r0LNQssArdYV0!h5PD<73XA z^XyVnUO8EeSnoMs-F^Lg3F3cs{r`n@|HT2S5)k8n|AhovGchQ@;s3!7UK93fA-~zc zGC}?svi=v5MM<;s0Y(7)D|&?}g)?5Gut2V3C8!fDD2S?roB&h#^KCZF;-ET{&P&kwMka%3IYkTN0v~Nj>g~dm5T5~Hqz8{8QlcoO)1IoD`N7^C_w z^ES$N!G{XD_5t8Jm+d*TSuaJdRDp(kk%PZRWs}@sOhb8?T@JX~7r5GQadlWAo&}uS zCk!}JkQM264;nC8P-CH8A-V=vV6DeUOxppI^rdYHx$Bes2;`+{V@Km|=cGFFCt8)- zOJ;RDXFX6Gn0`L77RXB^Lh!x$L9Dn&gSbk80UBK>lKcE^UZ8jE?sEzR&m;@N^pFf3QvR(fAhVe1d24wI_ z+HUTFzd+&sF<8f}n|JGPgWXHhKSGpGBV1&M0JNy?*v~7Ve$q2?cgi_HXWg%q+5l(K zY`pU^6J(x-mm^S;zq(Ly%$9VGw8p<2_7z8ro(A2n35>dTFgS5##7UQru0le6^z(-;e3Vdh5D zgj{&lv+iYd%brwqjN@NBZT)Gi5U9I`0xax|6DVQPU8#+1#KxBr_C?UQ@feo=)Ps$v zTuR=&ZXei(t<#3FaUT?hXc*dwmE1UI(a*y!%@K2SKQLgInku0Bls5)iQpv`-M*&(8~*h2LMw6bAFa7y5Vh6PZ5j z9!;l&OYExxOeTpnj`DZatc22R7jkh2raIu?F#G?S;rBEM{QuzCf9+cc6y*NDzVNKg zybtzYw|K@s2mOz;@GX!*0sY&|npSF|!v98Ld8z*ses`70P}B73K_CEHa30Dlo_+!o zcPU3On8rB9u$aQ(4Y2H>!hui&m{?Je8LQ00tPvf*R_?%vuyiBxnl;x;HM3ZCHPcL3 zaK6_CTNcvU%cHL9^R}{IU3QjTwYn>8TVKo7=dEtq^Da9b&u3DpWYXg(9-PP9&NH4f z-o1VIIV8Pzujc#^1^^m9g$O^N<*5;Z4oBELl)4qNZo<8L zlHTLd=i8N(7ogA4sS$*c{JxYM5{}H<#{Soa;J&1rXS?b#0f5AdI0yS~9_f1%4xNYQ zV4sT6U|;84BmgA`paho2wLr6fy0i5{^*I!iwDL(ks*#PpT z9-@Am8LZ#gpxpfqTqpgKBI01oq34F=C~I2i5MH<$v+Tq^tr(M)s#ueCR<8HWptvYVLw)OORb8C2csNQ}gIrf7+8`GG zNXEgoprwadQ&o&((IdivjNLW3*N&}*d?q2!jC?9fI{K#S%Zt9G8h8;Uw&dY1GUuvF zQT=P_*+ppyg{u!w#Z?!IMMjhq9lpv@xJy5(MSMwQAK>H7Tp+sMVj&zOo+8n8k}o=R z$zL^K4|_t|-uVVBq;D`Qy_${BTPh+>Y}L(&ilg}R(wJA>o81#%`=IA7zKv%U?cotD z)VtHi@0x!w(LjHvzOU-Lq-fvR;plHqaf2x!LX9}J(qHtp(}5#P4n~i9qSO+>Bbrql zY*vp^2l| zdq?p>4Bnc3lyA)3wq}Q1=))0B+yy46{oox z@5zbvNB#Fat_6WcoMpREQ5v=dNf0THLt>mtjkG@}wW7$En%(hf0vtQd!@Ig8a>jUAWf&2tCtR6(UG@<@2|13@PnzTuxBHs!d=2hG;JE=}4|2<|<^(M$Yv@B%X3uijLRn_p zQnv_hlXYQjQwz+b6SIugIkybE0(LCJ=--@Fkz+Uf8E&&R$XnzKa{F}dX)3FLo;LuYUlces}@o7(f@N5PE9045e`<`)9@ zTM^J8wuA9zfQ^A&{2h+70xcwih^`NL1$xYkE9~KICumLX7HXLBiv4ALt5vu~a+vAo zbdet?SOtY5z)iVj#7#Q0LQVLK>-Xg$l|7M1iO(HrXDNU>8-F{$i94Xb3v zsD>?la}x$MCJCZKz4ynd086dbgLloO5Xg@3f!;}(;d#L+R2dvi#+DKHQxO`~T!q6! zXVYdGViIU0FUAGXChwgSTp;L(dIWuSn>x!%;XM=qhM~R-7p9! z?5J%PSO|N5SoUK)CcKon(+7UjyizIYA`~Xvl1LG9zfnst_x^+s0-h?0q?i+bu}egZ znsUot*ck4kXb6{*<|5y54($cFWK2i?aURs0o^a_42eD5%wOz~*0G-_6=Ny}@5ef`~f#)W5pPk;e^3j+CL*4Q!|!ftc`dOEcR$S7&RK1d?|L^ejE^@|h3r5V+|^f#yWlI3e^Z0w@vRC2gRC~GP8#eHXc z4W*CS6&sH``l`)>E(op@X4hah&i=2&lrYy*B3YIqd2=LfRe*h@ji?Ua$Y1sX9%VS% zix>Y@|3adc{s#LiD{2?^xZ-vUYGFvK3V2LokB2|=Vh+?)y*ST4!B50ytYcUpkx0H` z?)>rYZ@$}t4yzkGJ1hDp#TBAw)T`*Ic{st_ zvsQaC3)VEyz1?AOLwi!En(0?o8S5^}iUh)b7}e=K16c3wZE~b5a1#~xB9_eI=(Qyq zad@-Z-8N#dWU-5j>Onx1((7Ra(d%Vq-d}NH2T7=TY%>8{QlQ0_8a=Qfy?hhJpfiD& zfuk~ukvj-gz-^5>(J6?It5RfvpKJ>G`^WGoBS-VY7VI0Ll^Rwe*Vss0xH@xU;^aME zH2@oZ1t6&&_xKL;bCndbX!DhkZ%B1Ao+t`eAKh1D%hmZPZp*Fyg7Q9I?L$(M?u(#o zr@=>mmLq<1di+*S3vGk7r~31`fjxralH7a5W-=B(aNWbYW1&lfu^qe_4${1?}_>=vU zSb~!;FYG=NPR9_|!yR0@4DMuD`zG%G0sgxA<$pQ3=oBJ@b{^1 z|LaL$A4-a!>g>BoqAZ8b4JCyTFEu*RcYwvhJM79#e7|=piK8^dt9E3EZNgH-AA5#`b7nrMZIrcZ8|Ekj0$Dc!i|T_ljA?O78n{G($j&eW>{KIH9*NC510 zQ+d5B=FmsYO1%Z}qivD~ZxER&Cv(IZ7A=|15d#~Fg*978t(>>bE$7rOJt5x0K{mgNEG(2%!eayD+K|EIX%rh8MdF7QV zS>8ah${d=8T*<2SG+pgMI23JZ4fSj_lD~gvDkP0=<0tP>(i3bMoxJ{(+Zty8_za8% z?<#vxN-!0xtkDvxn*Q(fah<;y8aND%mcB@`FO77bSW7Rq@3BG#EwTo#-~b?q@(3Vu zhdHMx0!9iT%n&OV_vq>}AIdIF$07kyj#e_n$bj2c!L_n+C{# zp5O|56q|R3FL~QTAndz)scLFnY4vt-Xrt^KEA@=-|Dw0~`Llw=#d~cyvpX(W0)EYZBBdUTVXXt@8Zx=p07+CdfEF zaxMjgZjEL0TV0}B^R3jM5@)WCbHLEpABVvU8BDO*KbUNsbEO8(rqbv@rHbgg1GKSqq z2q$+_nL*j7jK&nI%(2_1ciG`F(vTjCg+u3z(RpI7b1JSk)?sNn64F|u98A#L5T=Yc zHpcMSyh~@7*h6}hPXM{2z2{9gPR=ur4~1=wLLdUiqJ5EM9x)gn(jGV5+>(QKPJ8^7 z0a6o)1zO!%XSaS<^+lCox_d>q#le;ItA!o?MCS-_s=B-VkKrZMP}8L?)>_5Q`IXNN z`TTYj);K|snC?|!F~eLTAEz~|F7G6x&5gGgt>hEKUf_B4^#HrpeMwWAjcjY^9vC6j zKmK0WdCmE58qMff3X|1F(7hqM=turpyX}1TDcusw69H9ZwRga|P4%tX+QumXOU;(h z?)0E)DIgcR7YH5UY_bD+J>D3^Plym3xcdT}A^pM%9m8L4ym>H@qC4u;i$R{6c+_ZT zK6>R{QbA68%7E^QOdRE;xv)N&K?!gR_{gZC#pb1ZRT54hb?etMMu)j|yu7i@%uj1b zeuT`C-C0t2`N;SED@$0uI8(QUIThuWt%p>Vg{Qx=En;HMaY*!@&YkMPk|e;U9JVt% zE<2>5%6m)6Q)aC~4V6}~j)(%|a|ig_QCA17@0emQ*T$OsM7-l0qh$OLK8f-W%X;+p zdRM!!BK?!QEMk072V#&;PTPwJA#GicI$`a0M_$D`?>iFrGw@BXfPbc?{#RK4U)1U! z0I7nsS(D(K0?P;?Z4VEGIL*Ke1RAg}o;RzE`Ce|eo+EytpTbEL*gXG^_4%D3R?kA%RSZ>QmUjV| zPUCA_VR!@HPh#ZO796Ii;kRrm-c_jXapc5}Xnq`{ZRn(>iZ+>NvE#jjA60IO6{kp` z8G>(hZ4)&*f!6v4lFsaL>Pe3^qR2c!8XREFJ6;u6=44Dpvn{3HBP1!9Qe@;$M&i!e z|Cn%}UNsCW>#|rxa1<8NYU0Y4=4=Djl#Arp(FtTNdfcl%09p!W+5!E;Zo@Qi~?n36Zl6Rr{27|zgtI8l1}Q2 z?AZM0PB4NU_Kx_;NiaRW&b_f25k_KP76kbsUFb#&~m`KqRP zd9EMm+N1@)G#s7(YO1*DUF(ghM6+7hTC5Fh$dqi0tDN;1_b#fwYKX>KWQdo@#ZJg^ z6-e{Z;xQMFkZahLRG9b`fV&=o*RT^Ru9er|ZVy`B>MRrC=h-y6AhFS(Gf(FHAv_~3 zKB&)s^s-q7D478ge(Ja+=+I@9WSFOpO-EFz(Yn`y_Oa@C3V%BG5O#J((RQo|9NrFx zc6%M3C6tB9BlO|3%h_bF@lHViDf|e`N2CRg3V@3gWTFI;?pIQPP7OFR0HXvs$_K6n zt?GBpN7g}53u4QMt_C~r_fmkf5~M2!!RZ&PhYA+tTZIL*`h(Y_*??>ez|TUu1?uKQ zu?B4!AgsbX1u$9wA`y7-cKqKm}CH5+)ot`j>QOv57KA=-fn=t326shHh|^8 z;z8?y>%j})gvrfE{Y|I$BJxW5YVwNus`ASE>hcQvD)UPFYV(Tws`JYG>hlWBLvSmM z!oYlMgu=jdtAxV9a_fXr$LLrUp`PiH5}}^u5fibJ>5&t$VMDP0NWN{bA49%vvOh(> zZM5%(V$XKVj$+So3y!j*8eJfmPcm25K$ujTbxP?Fg95#OZuHMd*eIFUtmwlfa-jRJD8Wxa!pBNq@ zr?=LL2j+1#g=q`w4BMD+MQN*Y!D!2J#c30-kK16fIeh85Ec?r9uwu;8#?z41R2ENP z7*!z~Wx+&OAbp~wC6zXJv}ki_lez%inFy#YGy|9~*xcCE7O1+=U5)6CJ$Y%#T3gW7 z9Hhh9*qeBxY!GS5b1cv`2iTQtoVaxc<=L8eQ*4Q5S+HzPF?p|UOFz#s+7=+KOKouW z=3K(BOLkdsY{?fjhds^n^9pz6U4qyUn0Wh-o)A4DH2S9F7yNvnmZLZ!{bsduk?J8FaDF%KxH)g)e+IIg&#iekZM@w1C&~UVmHyu( z`ws>#(Ysd2m$oy6p$;+Yw}1 z*Ri1{BlAG;qAgvse3QWV{^66#k;W=Y@1!js`?W_G;xasRwx|08QPGaw+$i|XV^ z5qv>o(gL<_m6y^1miy1jjzNarIJ@`RHe42tOApCbtG_(h z;MUNqCeK7S*5Vd2u3FUaSZ=b=iLC0)JMR{pLobU)wg(@57dXeD5lZoj%+qX@wpJ%C zlfsCzioeNLp*F{IJBS!uLJuYbzGFe%V&;W#Y>T0moR39h(%Q|?Q1rzXHWZGHowE#JtV8RQk_tZ=V zrTrhK;Uqsgi7%MD9e7Q#sUxnhJF}qBIwdhyK@R!`Pa<}jQL_+lKp%;f1X)dWn(?NQ zNgXkpkiw4LnJ4n2bnz7ke6p_O4E4_V)4z?o|Hjf+#!u;sO!|LkTFBG)jby$&t6Zso zGhHwwK&1sPe4xV%1svdin0_5!mQ|_?xSs&_5&0xpS#VHrxcwpu+fucPzobesvV7J( zSVvbZ?wK2n0qn3`T=m(ihs6qye)i2gC93HPRJjHc4p^?C;OT3V89Gjp4k6YF|hd+={{tN}()^GtO zU;C$g?ba@P{O+QDfg2tCWQ4l%$Ilet4u;0$C#OWFW>BHT z3NFYtMSKt|ApcFeM;^o6v(zlVTACux{$th%uRT$)G{Qu8Ej+zOiN05)*dWU`p;na( zZCfkZ;UfNwkzs{sCZ<-YLLWKBHQlVeLegIL3yg=sf-@w z7R(0=D5n~?*5?I3;1KZz#Q%FtFD?YH`U6T2nLYy1;0$6VK_2OQ{psI;)XzpOeI1 zIns&0#4vS|ShNC-K}x(N$HA`(ixLw0^jM3hc)D!L9Qo}r5*2(|(?iB+)kGb8WPPUYr9SMioMO zM{t6Gs(pm9NrqRoc9Bx72vSHEgm3Ld);G!_&MTDu3jti-cZ2PH1WV~{Lx7IR%Did6 zmsr{ErKgU0NuIlvnH;GS2~Z?b#4g0CrF5Ggf5BU2pqK|T?Now^E-Ci+Om+e!Abe7Z zoD$j4%4m9Re(YF3VQBPv3KePCpHqgFo)IDCfQ3-PrQqxz?Cpc5l_P9u zBOd%_CY+9J2=D~y9r}%ylR*JCw{F$U*}^Nl7LaX=sEcy>N-{~y1Z67GmG%uY=%zN# zC@t=$)JBiRdJ8S^H}(CA5>nOhfG8-Fi}3Xg6+?p*{z6)I^$-b9+I{1a$S4#vLH9C? z9!gC7KM6@Luv~Jq@&#ZiKHtGJEmDuu(*tFiYMc0uZyH@WX zyclK%eB(t*eq-b=SI{PA>8WLimXsa!VA3)QC5e=jvI7qsLz>?rdy|7j0I%#S!%47~ z0ljK{q?{IIYErlAK3Kn0EkPN`vv@QDm!}(JK=ra^-1JP^cS=%c@irkzqIi{U_vLrS-bjH$;#O`;=QPoOA7fUS$Btopz%L?v&zTwXfL=yD!B=I*B z42{#m1t&%%dx~spWm5tRu3giDRV8$!xQ(+=1!7Z5KYTZE_JOEF;%2t21PKk|(lK}n zcUeutW&``h4~5L6fcDDY+j}(oHt3>$lsgH3{#Z7KlAhY81VOL+fZ%SYRy6Y2J&Y{+ zp9?lT1jb}G{#dLnZOOCrh5r>T@vZh}XZj(G)o)p3Tl>g#QY2WqS&cX{6QPpGDf0F+Yxar4V7+fEdaaIDW}UgUG}s->EWP* zQT9gaSw&1_{X8Q+M(UXjc2cl&ork5`ErQjQejr~C+>&^~agLkM8HPL8*qrI;>8x^w z2H!^dU(=)Yn(WqQHynhIGHC}q{>)mgIubQ@dsw@xdIzx$NW!qxyn;m!z5VdfATh=u z9e$95@x+KK1OsFv^D}gr z%l=EAu=PTPpJAnS=-z;fqfylMv#CL5V*Py5&Wt(s6x5L|1Pa8Uwz1lKBWZ3ERHVdc z^Ve8XPv~)vClK)>IWYgumMALqoxOu_;K{#2{i6 zW$e_4jrU=Fr;o@&o&vmM+c=dS)8RhUa}!nq<6>-v`p`WU{gBW6whsuL0w$Fx(Q`kI z#`otnTEH)#`V__QPB+ex7TV9a-K)>ie`yaFjd=-t&^pu18Mo8P^&D=Tz5X;8-hCUs zJQ_;&+avHdt|o6z2AGdhObnZ%(!!4%)}Y)X<>=cb?6S6_$CKqjjU*^JL=xx^7ZC8w zP7lJtV2Y%|u_tX}d@SPegP8yE5Y+2Qd{>eo76o{gPOZnS$(SpIj9tQUBX35ggO=bt z>+io`)$8r%uMhn|g35Z`UPn}PAXX!S1V`HZYPIvmd46S)w;_|XD*#h{)--n{Y)?_x z%ZqA49kj9OGqRE75!B4ZFGmsQF#0`&)L2N;q&-EO6Ut*@ey=ZA6aLITR@qeCxRJ5q zWeu=8tt_oS9-SD$)u0Sqo)gi2x1V>c<-~n0>!T$x|-nXeNIQMXenp zM1oI((q{&WQg7lRG=s;2b6bOqF$e%uq}szBp$QCa1jYj%sIj(AVRrBdIyd|A2uv@1 zqQH+Gp~lb&@eYn9)-DfA2d?`y%{pPe1iyfhfljJz8D8lJ5m z$GqVbA|Oj7JisVt42oQk&%T8cPJ|o6`H}^C_$?nNT+$KhpFeFXl6sQ|fy=y`LK#H~ z$?#$^P=ug65;TiWX$OJ2a}*%@dj*f45fewta;i-RX{Q~u!icdgVd}~eVvVM_JzLiS z`Qc{+ViKi-TrNNL*bmS+u!VC+=P$J6jb#n>R6WTY-SKl^|*;HndQ%@^)5P z%?z5_g>UablAe0Nbg~E3?myfl2mQK@1&&qrfh_L-Ikqz8HXl7mlqIBZlO09Up;xIr zEHas@GmT1MwwR2^Lh?xfUk$9ElvVqf)_lXk_GUBz4oobb?Zb@Cf-BiP zc3+E}DDpFJ0ik<@Pl?o;=yoEm8lEVMWXx7lW40jMp?g-~Pl1PIu?{=iTr9zAwqpJ{W1)eqooMbme| zTzVL*3F>;BZkRQ^!^7(0S_c;L6HDftYE_d z^fkX=QR`f=a?+uJ(;7l~^IvV%KWW9q^R)28e+G~W$Cp(mjZmbsj+;!#QTyIbd~Oj> zFIB1QC~H=pM?HuGKZt>c3Y4H$V0rxG@BGBCXWkbrw>z7^azJo|t6*)PgxGiU(_sn- zHObQY!%+45Bn_jty&$CJq|>l`W!Q-UvT3lQSb@^xaL0<|YcNS4S;3yL@CXSE3*lCt z3+ydD(;Owbkt6z#ojcVuBPxOyQCK$Cs_xu+|c7M8VKhp7lNWfeC~r;6k)8ovm|fF&F4rntrX+1jKE-HKv|Dj}2S8lAgb-d7M`duk zg+_(3sC(9i$jB@pRu4uGlmUCaa{SiQYauFK>YIiutT(IE&VK_8Q}z@|74Bv?<@w%OA$4`d7eIfP^zPoPMOu^ubTYPshH_c52Z4yp|B67q@E<_ugXs@* zLx~oYI0CJNo|d;IlMZ?hC&K$xz>zrQsRz=Y@JF_~{Wo4cRmF#YuKE4w79$?Dzdk$E zA3T5*5fN#;2SfzmWs6aPrNS>R6_N#iMLaE-D8AH;bv}-gl{D*7i1e^4H(fxO>NQ1X zj_t0l1m|a$VX(lX(7?z#2x)kJT>Y>kQGq-W(8x|09V)`RB{Wn{Bx3* z(7dOoOC_dpm(FOxH0lC+Rc9TT+D+JX;-jBT{OlbR`*5LvNw`o2WWz20)iIA%ck9HQ zZ%x(x3c=fayf}ED2%hIG8Sai5KIp|*8i7irzvUe!K7z+TM81rNlpm%#lTOOJk8Tb7 z1>_ey!tz}sk?i{=S!6Q70NmfxJR%n*Ju1COhku9}Zod5xBQeq$j@u!u24G)$dK-KE zN<+oF`*j5Rt@qy~>3^4||G%=<^gS5`0rcMvkuz$`XoD}2t@7Ky>lc8T`G3T<2`x__ zJx$k-STlBevpe#b)C7Ulz2OM-I1}w!8}SGV-5QFq#lI{01Ci$t=ImaHgLo6++D2_f zX^k`xD^M9=MNm30vi_+k@T@8hQX=RoXh#ydI;}!nxyh9faF!?iK93V~QmLleGB40C z$L^c!lP%`YlRS@r$=hn52Urk3^8D=?y)=L>za4?F4o96`xYfZ`TIkfu%9G!T5?hGz&gIH~>3?a~={5K0Pt!*CmV>ulXx)S)qseU**eZy}!1n7RC*l zl|#G6`wX_}tdIQ~1)HzptS-s{PgLR$Q)Q2?3$LLk?@?AC z>WH5cF@T{j`ahTYEd#|_6Xxru%P!wjHNOw`rt|jeQ$XefGRIp}C~?b7ICj=k&C7=v z$7eytuhtrLK~)F?WtH}$FFrp7)rTS?Bky&s4omvSbcE(wGr7Z=z%}KZUu=gr8zCH7 zeg?t9-(d~FS;n4Lk6V+*UJ@HB|Rg@xtk1;BAR}ZPk?oEfo*QnyTed!ZaW=8 zc^uxzUiym~D?#Zd5GJG#e8hvBcZ2k=MOF@{h_rFNFDXt0&WvNhrj*=JK5W6c+^iIg z5BwfkX$=e6k7}$H6=QO@GbG8r5sjYAcYUg$M~DSL=S$v)&=}Z+7&9*(iSn06Q1A&= z;>huf>Yx-URt9@w$!ShcdjfHHeJUI?wb(5H0f;v^E0vWf=NvhC2YCuPJ zRA6tF!~Bu7d<@QY&z4j$+kk$J60;LI4sv-7hoz!IaOutmN7O12EdGoM6&Dp{V3LH- ztD6o$RFI>Ke}0pKx_ONp>^vtrF;KsH<$GYen3V}Hk?f#IO}NS_ToCMyXsH-0rzx(4 zQil~jT)S$|9P~~PO#zb!B^rFJ8!FleYybp={j>Ph4axq2t8tCTuti)y+Tc%*(eR?g z^=&D!XB8yai`S6&+cdO#ZmEj|wOO1DqBv)ODYZh$2#c`vJa@`~V%POca)K1x$vfTa zi4ZNCa)}lMoLa>3&tWNc6K_5k>z6+yx7$)t8IIHO=Z=%o5HZCfBcUNx`I+pK>?YqA zElR!}df$G6t@x!m{z#Kef-UE|%Q@ z`uYFJ9waSw@1zYAOF@Yyt@l{6jiw`W&`djMB#FrLXJA;W$i?79Z?lXpZW-)fr&s*V zZ$F9FB@

`Ay?X8wAnwB7SA`5Q9vOSU^%Bo`fbZ18HiQ*q`*iho7m!7dWqe?3?r{ zW1_(XggtYdojr&+mP$*V%tjhanVMh%=+hep#Mm9kQKv>aN&evX))IN{%$`(LJ-5>C z3p-wkRsyH|3!$5eO7W+HJ~mi9KUa{S*!30Bl#$f>k|VcEFDoNrbZ0uzG)wz5yi;Qk z3z~H!ertY#byD~egGCh}-QxcKhb;yr*~TdXjs$w7JnF5CavH6^x2eYf@TCuCO=xHI zipEfJH6{p0tzT0rV^8Q!hB#VOfo*%PffIBNwTU@9&LS6PD_Vy3)L1dB<*H8yw7q72 z=8d5iX!E9GBCXMpd1GnO<%1=`yW&Nzm7J@czK(b^j)`)m`cDgWwP<0nVb|eS--{Uc zZoOT2Xp*XX8#JNHyX)ow4dVf0^X8Q21CJy7QeT)|6SKDf2CS@*3Zh7GM-+z|}lQ4-7O&0(vX_qmHz zNBn7%T6)F@1~0fuv0u04NG;`Nwhv;JY%f!+AP?9M`2pVt}m{`%mqM7`k+A^K6S zuU6fErBmqC^!RY+QjZMfsGFu4IrTEv}6!fPB+*dp~*0TMLUZ=QDl z0F4CY;bWPqY|qb66suDa3C+8rXabPsacthT=yOVHmjS*2fZ~~03NCW;R8whOEZ<*8 z8B5MjzY1DyFV(xSrg4G@1%jwrdA*R3+{z5#lVLjK5=?>Stfx1HX>}@;h$(TS?{tuz zuWI~RS)T&t;-XBx)xUMyJVCb@tT7#OsK)g0Je<~7sI@wwLcxR1vjcL1v{w;e8mr$; zptA|0$<+t}T2MuI!J_*L_uMOuWkboXmR8!eg7B~9mye7?1{5!jn0gJGh*hT7du=E7 zp7ga?1iW$+;JY(gYZl0~RH+8<@x~{Iq_dFE6?Hu>Sc35i!==NtR$N-}1(QGJ*p46GO$aPTPd}<3TE^Io881WzXq2v5oWJ#sUyR z?w%jkW_%YC({I8}C}c3{l9)yqp)GAF#yDXUXV*2LZOeQS(t8^DrKB>)}I+{RwPoo;^r)s09 zf%N}g$i>%QH+5vOViKg8tWGrF=G@}GL)*F^)Rp{IC$eO4Qy0rPYm`x7-(P8WyLa_) z9;3=&uP!jkA30t)*aFzOd1jyNKC~RfFP+`AEQA%@@Elob<#pO2bM12-CgkU8KOS$a zvBQP=dg0|9yqy&q8`gI3daf<&9gQ_T!`ZGW?a}>IWb^hs^UR7gR*tE5KpJqS_QWXS zu=L0jz*cf|fjTw&(>Zu3-NzM$I3eDdKcvKH!C3W-K$|1pIR!W?ROUIO%jp@!)dNZ_ zAaG7>dDS@wO|gYdm_14?bc~$YUAK%^Zyp}FryZ(u0iR@Y#M7(EV=|nnmDU=|gi}VE z7lDuc1dZvUGD<^xNk9n!=(uTO&beJ=W4hJXh^8iV->x7ueh}h~2-id_n97b(O@jHzQwhNCxK5upWji&zqEkbQ7E- zLaAr20GUSbmQM_~pG_QV_KoSEP1YV^gSQT-?he^H0{)v5{fREhY?zLRP2Zt9>CLC( z2VY7`cX<~F)$1K+}2CKB$1xHxW9o-oM(grVx0u#cQossad(Hj#T zR>9*YaNzYadnwv$xux1f_QiV)+D(tT;;Sz{ht4-)OqG1{cgO>VyEVLMW~TV z3jpvpt$vlEeSCA9Fa*%>Qp!3qy8x2b>qe{rD0bEBs-MCbo*4_jgA2I6!*n#xS_6I_ z{mA5ZIgVEMKN?3Q!j;7mzE;^4EIe{`rb!<|1=g1TS9pF6s-#7!dVTvA|E zT%Z?o#CL>hmm@F`A)0gUyh^0w>2LokOde1mKg^$L5NaC6W-A6uWUGKD@w= zv+U=Kc%fYje@sVpCRoL}rZtkNNp}>S43EeK0}SVc_R*&jGe6F-M&CMOMv2`Anb9B2 zC=>UFSw8p-4oZ$|sSR#`oMT3oY!^5s?Cf_zm`{rD4a?5=RQie5TP@VBRhLP$0c;f^ z85)MF=uL@>Ds`-FQbtuCGGt7DuX&21MmKqsC8s+bP9%%63DgbC;j3$>s1zeGO{j(< z)*@w^2wWS;_@|nm`mvPB57=V3CzUp5Zz2}&w5T0(s$P%#k`y$b_^zqx(HsXBsnWp; z_PhaSN+S@|&*zzPU#ahp)5R~z0Lv?6)8Mk0BzaP$J zkL~ruF6)oP-vdo&E{hP#H2AEmGQ~d~CcrwhUyGoSmaT56*)Uzqz(*}LoUYGQ1tYcB z8g{jAJ1%@kwBLj4u zSH->(D79pgj*NGSvz!#g7*ZxpfwL;J^?8*^N==hRj#3#)7MP}v^TUafiO$;r+Yz7f zoez0Tp0MB(AZ&U+Jog@%5zIBKc6kjclc0!f2(#C3eK&c$-5LOI?{iEbj61g)Yu)mQ zLk_ZU`)UZ*+>r^KPH~Ni_Ysk+(ttez#qWs}$ zGtd3Vsi1B*Sp=J$y#0*xm2DJU-{&`tZw@|= zD?^U8oD(2guC71q85oOugR0s88ib&acnjwf!bi?Om0P)Tk>8|K-$ja9Sp~dZi|@uQ z79)j@XB)$goaMXe<+9=xpNqDocSMIaak!oh@ z#GibxrNoF7Qi9RghP*bX^*Xw+-CbBr@F6j!BiRxd0czB7?DnW7ndG3y+$i+7 z{aOi*<(MQy=9K4N0Lw0Yj=;usmw`u_9rfQi#yEdVwk&D4$MSKZ9&x?L+~ffrJf`6F zO`*uU^~TLDia0ayjk+A-h?)03q0lrw{;~*cpA$`CNROJ}cAk$VAp1`?2u-UBQS=?_j-W${DXfLLmw7(r3JWpWqsvL$Iyhit}RQImBs zD!YUmf>OLe&*)jC$+=;|@flSUU~KY2UFJ~R5!rn1wWNO6xOzDmfpujceMRKs z8Cd<_76G#f<1@bKmvj5vMg;LO5acke)^(lX3{C-Nc;C=nhs6+C85NKczx~Xa&9tD< zpE8-U2dKut+6H+kJ%gUGxpdW|Ned`!FldAHxJLf61+->BmGI|bAAd#nW8Cl2QAk4Qfg2PtRS zYbN>*C|2erk`jv>Su8@8@2!92{ccT&uhhi5oLz+A*#T5pF!Ulc!NJwTOKPWKr z5E=|J@*9T6HAuPs6Kxzgv($eTjzfgm=ZK5%i!-=H!B4rk8GqG`yze102+IOc9c(h; zKkifY&>3Vy`vUnjB9C2F;N;wr1Rg@(iGcp-F9nOx-b#k^fxvE%lnXob%uC7|1+Ond| zLHN*+KeQeBu(LZ;Z!->tZ5%AHvgxw2Fv=Ebt@s{O;dX>2Hpc7{Oi&MS&IlriDhk<< zkem2^|A*m^-umqF>@rPo#}ud_Vjs-I?Fs{jmq$s)pN0mq6P8esY~%~awcJgoHbpSF>aaC3_~!Lu2Bi} zbN(C}*uICByL1!E0_90>G0^lc7mI`F)-A5O+YRCgMzo$x$7zuus{!dTNZb{9!{d)Z zJ()WqtD^|g5|+hNJ1wNWbJj@D6n=r ztZWKqGS8a!<8FdYscfVhWjQGt(%yrUr(V=J)M%erHcXHY~rsP16iWk&eC5KD}R!4 zCY-U+=Aw89u7FS-w%&PY&X3eEMvW5@qewDsvnq4*q(uc<*`0WF@y>HttX7um*2%$Y z2HR3bd~@ZE@;6-7!^BfYpRJU$jYC)XDjBEskZeBG*Hixkz+WP7FWEAGY6&)=_8{&m z)KhcNhghh-0u`X;QPJ%QGop(p%oHi3rOhEBgH~-{1nq?<+$EVXHeDx~vjo4e1pGtpv#G z6LO>{7Sf2V?1BmPA9TMkD5-uu4W4)L?*KYO*ii>PyC8Wh6(k^efgSdM_@U!*NPJwT&S@W$)=8k3%@Ve)bU{h?G)$U zvl^pIfVzC#A?Dr6g+-ci`0d&p&-5%pv?DHNU3ju+5puQ(BO}FiN^|9(ALf5erynK9 zN^ai+04aA1ddQ(n-9foWh(~P#)IZd{?PMx1zb=J|JDAzCJm-0q>E;lP({m0b93&4$ z62YyhsbBu5^1XY8bTQvE20kGUW`MT>5Hv^}jd>wvJ(k{)ssw1)U~k(uA($kZ!|zyY z=vO9nR(XAVHX8Cso{=g3!KCA%!L**7`%O{Cv`_0>ub6DD1`$#rAXDkL{v$BB71F^snicjH zE(#vDff$+q^Hy$H49*RDge{hD@VaHrp)RTl5fq@>ZE!hS?S8oAZBhQhYJ~$}BpaXp z#z4?>uX4%NpB?s>a*OgmRR^`GTgw5o5#+=%bE> zq^lbS$`_Qon+Bf~IU1QeI4(xS?}AmAZ?HvT1{PClB?J}!Eq09SrLtIKU$bzTb{ZA{ z6K|&5PVEXxX2YM&bkMvO=~Pednh@j>aH+zNZ zxuaovU`1UKSY7jr@4@fYsvdP)uEnuoEHN2^m|t`M7>wc$yS?*9tloi>xF!-_y5{}f z3;7n&$@me?b%kw}_9?Ro3h0CQP>5-YTQRr#p$(#`jd@>5N^n8Ov>^&07BMJ{j}H+n zxe3l?nqd?muD~&9ye$mJ+@^EONoE$@*CRVtlOVWkx-AU!%tPBFS^f;L8&w-YqD6I> zL=0_KSj-3ZAZ|^F)BJv^jb5?xaFx9y!4YnxWQxsDem##ZkYdmFa19U z(#Uu0+W)Yh@;t${|AR!snTGxGA2?Lw7D^TblI@qBO6Uq|@8}m2O%xo+g85~^8mZAb(Um%iB2U34@P&Q97L4E5AA|h zOT!_~P_u4>FK@m4nr-5mLos+8x^k$aiVgra@zB=0_>u?kD%M7;6h5@=Ruq?((iD;) zf(tDzfD^SEawD9AHDK;vTtvp*5$Ek=R(0i9CeCsbo8BP=(zle^J3q!l$Ig@OvJcd5 zD&r;;m*KLe3c}bZyA=ZbR6sMS-m+BT=IZpjE`)lv~y+9Ov9$bF({Ckkx?s zUfZvxI(!J976Toue(=5#1Y^~Z(l=xI{Aq7FIx&n zXkg86y06d#gF{H=p~ME*I^(Hh_}P7<&%@VI(Z!Qh(TQQ~560P*1(!gSPmw`iWuK7~ zv`vM#0f(j~ zkCxWU2TO0r69A@x;i*ZeiRr1>r$caQ{q!vW`C1vQsV2uiG+zZkM{`p{iXZ8dzpWae zHR5JtFZ#`B?*TG@UqK#DuVN*3?O`=+*4Kon(SuF(T+7UQ)5X>miK*bn+y!if`nf_+ zLe68wG<&v=Y%64x9sR7^Qc6e8x+C0vUi;y4Wqx=4Lh_=rc0$f3mdDSmp{WBlMZ1^# z0BuJL=cA=lc34b6BxZ`@n%Dk^o^q@g_O!2g?l)_&I^%0Cjxl%rV~@UMG4XFaFxz=& z&kS*%mtMuLE}Lhe>RA`*t}V9ioZtX3xseZXjuS zFSca^Ru8FP2t7~gU2R6HBm)`+rc%uKtX=h%F1>0Sb8J^T8jrQ*S1nDTUG=9J=W9dx zFJ$PZwSVtJFhY7Z%9SYps}BDnE&H}Ih2H$67isBd4}><#)4BYhc^J2_gQjYCWTVIY_`Giwp5ce zh>9yOkf%NPsHAY%N6^J|pT(t@Z`tL8kY0`+a2;O+l-g!+3@W}Sv(3xmn_C+~()c+o zFh#<0+-`|nseQxVOL{&;5mZj=#+=o1@I=&Af3~TxMN1ME{0_PGp0Q}waVA#!w;D9tgCH&0`cc*-b`E=$$Sn=QlCEyA zf#bso=Aa=TRV-X6e!_PmHcz+&pLnqwXp3aVk0;t*a-N@mka+U(O3IQ*X8BusXP6Th ztv%>Ky!aq^^GjJfvA~X>Pt!q+Xi*qr0_kV0#M5EvA-^qm+QQ_M2f4pv?FXufZq28H zVO7MO?Ct!!mHqeC{oil>KQA@NH-nMhul+^{XsC%w1VrJ%L2KIV1wR3U_~&WUD;%6R zDB?fE?A(z?+^>l!z<^}1eZ#*%*Z7(Jj!dRyfVLST6gTi{oDD0TQ_xts*?@%vro=0jcS z%ge-qUB1wN!ZLYBn`?v7s_^~}aF8s8tf8ndjJN6)V_E zQlmT43~e`&<5q@X?Muk`m0c?Fx9`eEyPZ4{gIAo6klS0%_>@T zT*v1L(Rh+-VpLy-S?=A$Qnl&`CS0d-GoeGd$Q}NPcM%prEMaMr-eRIH&E*Q7ab;|w)O+lIGdBI0dAuhUB7?@;;{k$}<6gn0 z{#ZV_mc?0Jg7o`UsYS}qK$jL4#o&#n~{QceZVjw}6q zN;{+;&2~?;pDbYgj=nnu!{=_Y&678*p#GdQdD;gSDUyWdg`GYgm{pT($EQ1yp%s+F(VX^&ypKx$|z!PI` z#HL0r9463zx1}Yp%W(BCf#~iazaSi}}*fcpb*x#s< zdQY89bD1lEIDdIZ7tFW7Lj)fh5b&uKhXYUBAIw)ks7?a^h{Lz1NWV{+%<7lxrI&Xd zkRp$n?xpu5z~bSKg2UN z8=f_7KjbN1T5_I3kUsVAUy)5;e~n&u)j!JWKc#SXEgp3Fp88#+`vDH_K0y3}mSj){ zPC?N;MT@-aFw7xCr9d(Zf2&0f6vQl{7=RF}?1kJiDUpUg(R3sy%LvKbr%9W!3TH-; zQzTIXA|YHoFqdW0A1F_IF$hyn9m*bxaapr#M-gnRI8BO4FhkYg#vjCbWg7bXj_Xs3 zmBLiEdxYsIwEm*Xa#B{v9Xt#MIFRBssl+6h%t!ta;7+TZJ=S8K`GSM3uI^@OFk>I@wIbJrq{-rkP~d;Dh#-sDR{AhvjoOSc)#Z)GURbkU{tWKM0=hzp8D zfIHAjM0t0cKA!EU=+hw2lrElhq+L}6kdBCvXUb?t$QiHN+N`c+mm{XzN~Pr5Aq+5H zi2~@?^F73D4f3g8=~ov6Gl$M6o_)RR-rw1O+@9l>^Q;QK1mX`t z)l)Kp6l)5+3sVuz0J9~S)(Q=5bm^c#!;u+GdK zakHuplTCvv8o{;c$A3}F8L}e?3&{gx>EaF90c38oOut)`H*r~kPTZbl6C5W2O-iw{ zuN}Xn?aaAz)WeD$F{3)#D2eG2o}4n_um-NlYq%bl9+SAeL8oH=|j87h?2eQyZ-)cx64N@Yw6 zNjwbufQL~f_E$4n>?e>>)5Rl!Q~oZ2Bca(_7OFLIS*?EsXYSPgre&5v=}n$Luz+%> zc}WaU(jy^_75UZW3dt5{I*G|iHB;)FC20tFyTXl|P25nW@AgyZeCHWJm`}LvS(=%l ztCTJ&lH!5G+}QC|f{E`_L|@%hra`Y@qM_Zs+^V84=njdlwam(}Z&XM1Ko(IvNvEfR ze__&wzmox%ET@z}SM{VDLVX8IT>Uo1ESzihtxT(xHep=Na**Fdy!<7pv~EA410 zNtNjbB+a82I(RsZn0qE**5CaaiPbcKJ;MWe!+Hn2HeNH4Cdz?0DroE>>qLCCU?h2< zR;QO&ijrLL}NYh8G!LCxxl4u452i|cE8jGroK@bAy=ZLW9#0*%s(xU+VnW-UF* zNXtetB?|T_wxR74$>+5fk@NN{_ibrI^2e!!#b@mzIN12J#-lN_*8hjCcL0vG3mQga z+qRvJt&MiW&BhzsHYT<;v29OwW81cEZ@4%6zVE%?_y2XPre;o^r%yj;s`|7~cfH!W z&5(TYnjE2WwYJ=HV3fFo&oh6FEX^>9iJ7 zQub>$VR8-U<$MI3s0OSt;uZ*iU`kYu{N&Krot10qx}IpG6e~*ne%F17$1E<_2qWl@ zih?Xuh@6q+t>{IiSqEP=gM1?D>B@!3W}iwE0L`%+eDU0mmUw{?)`2*;?6QFKlWK`NIb%P&;S-Je6`b;wCf_ETw-bm{^_)c92v(?m7p1J85ws2|PK*YS? zM%Vr;*AA~=af#4wR`1&lB2yshs6&|0>~ctNOO_tBLmxGu4!r+iBjK0P)fiF8qc?@k zl~IAbae2cub$+{mJ?${vPf+XLLhIhE3pw^NgrzopUo(W|I}ulBbeP0BF$$Mjq3{gk zMGav{%>`D}hQV(|CACR-Cdpkk(;S#Ce*@N_#RnL&}tnm_a zP}=?AOW2A4ph|B%l^hh}-bQJA`R%;pwb5nrF}hZMKqZSa1jlewJ}AIUFvowJ4?*#o zjX_aAg+a}lapAMGf5x^LT;h};eG7C$Ev%KrnQ3|}B+jqWA1N?`;

pkc?oeF_wH_ z7jEEF*$~=6txI;nmrTRhguj^~FZWK);JM6>VSd5`Pz8L~y8k$;nGmEs9vl)YZMyh} z2XsuandaE#Q*urd7o;AoJ4=PGtg|$!mx4~%?b$KM3&Txy=RR{xawxxrg_qBLK8d7U znx#ENyZp7Aswxun(_;-cH|hKZ(f$vZ|1+6GC$I&O+5!k_0VJjT7TS3h1wVk~i-!4u zyO4bXIMCM(*+C!c@3Lb&Si2=z+dXOPL2eCq`TpyO>Ht1O>CGKQI;Z{lI$$UxHBzvH zQinm&G3Q;`AQ)r$8vtVwj-OB=U~(50y|kjlX7AcFsp zz!O%ylu+F(Ss5ciQ5=tw!IlJ!FEG-f=adf3fw(NQx$M+FfN4%!O(ZI$M8e~E#M*aI z=8tS~)NZO=?ahYj#mKhg)By{(qR5_$9F1a($XJ*I9`{PeV7qb4R^9EEuqqg6k@{pB zFi7ZT!Qv4`8GPJALeqtOS3HnSyTtP2fc{YsD)s?a$h=i^mVtD&|5bAaG^VY0f!{Ne zU6A_Yg75rr25A#5ZAt|U#7oj!8Mu;_1`m1`TAUN?NC&E9DDaEoJ;nJ03)c+%~x z-|%AZ^bVfEc56A);S*bQxksTgr!*E{N75{>*y8t2OkG zQ=b*Dmz0y%&guChcBW*l_q3;+K1c|c{IbpIIHHU7xUZ99pwsl`75UmbdZn$q^-SIx zxbwqZxtfT^hfnVL3YLY&r*8-BI3+L9zdV(yahH=gUImD>jS_00Q<{n(1!CWyy z#T}(5eQBl36)ElEgf5$e0KrhUKMriBz|PLMX^%Rc3USBoi(7?iUWRRM4!AwmQ15&q0r|jrzhQJ#03DYxPl~(q_YX;BO6HJa0cK!Y` zrB>q$x_X-LUFv(4tEFXW!Pb_p7UT*s z(O>vtFlN^}x#$LYCWzQX-XAg*a#4DrsXosY^vYf@?fD)GdQxeg@pS17__QOvkCt1&`pn71^VCNc^Nv@ ze?$2?^eu1{%8xX}(sN`KC}=BO&(DZ4s(!eMyw3!f8qlzzNa{Y@;J%P#a08 zG$ubYte-a2)_b;{Z&q17UP7Hun`qQKiaJjcT}_L5TOpm%n_kFvYJdqE+PWKEcDfW| zo#cDu<9WK>)C0gy1!a7SdCJY-3}u69GruuF8*C4|a+4W=2aqc4U}+=OP&%wcO5xYk zd4}}EMK1bW872kO7>$J@qmCoSI^&jW=Mms4E`e>T zOQM^Iuqq>Gwz*nac3zbY6y>f?DfD47Drsubk&%LK2K-^oW%qY(=N78U2=8t*Egb5c zt*7S13MXW6W!#5s#>qLGF3u9?hKWpmqwnq%Y>xtAwM7H5iS*IC72vy@xYfld z^ATLPJ%Tr)*AZCqB)Uz0#4FvV@?-z#7hu{@or-pa_NoO+!VdD6ZV4^8)`eXiGxGMC zw6X3&LX+-LPRtHpEE;=PZt>fq5oZS-#5R%v08IgepVxnhXI9!2=7(6ETBv;GQ~QXb z_Jp{9i&Z3P`t;HCc@s#oFpHz3R$Bqr8*UrHcG+~cVI9%-h)uqJQlLzhL-7Eo%ek=3 zDAru{;2(o0!a=`I1WGLIOVs+6Rqg<5l!u~qNUPj4!4W0?=nv;$)ju zeg`UDF*NV8`G+abuzYd(+_gq|V!_g7{qLuESFsqY;>dMzTH?Je`vbbiBP>8d=xf?Eo=F-G_wE=8~u~}1acF+ zwrr8Sr(R~EvO~0=^c%j(BLX%sVmy!fqbI!yMyIM|CS!^~4$Fz$WXA5==LPQr>cf$) z;;#*(uhPKZUWo5^P&f%dEI`5wK;xhiS6+d&eD@VslcP#%?#k_|#jF<~`b|EPv3FisA2}9;*r00rf#KJ1|nj|u_>r{RWYEB38@|)0F z^>WWSCpwOPK&>1GQ0^BqqVMyfS};;`mlR|@+>D(nqiL8ru0zP3B;Y0%OM^@NOhAt7 zd%SAYlDhatmSNA29bd++`1U`!QUmA~ZiJf?+2XydN=&8(siRiwo`l&|4Gel-D3iZI zv{?wUg!>~>p@+2|b$)(ej%f%j;e4| zua^pPj?e5eqM=hy-wmH=%z}DaKeJ*mE_z=;SwYv5YRW{ z9a6n6#Dorz8TOb0SMVp-^z45)cD4?K-f*HfaOe71rvqzJ=4LMEb_%2(r$pJ`h0$M* z{<(ntKaBr>ySo2MjM-Py-}PXiz#UR}aHzi%ps!9eVc^JrhpD6?zjHnWQ!y}a1D*uP z){2e-Tt4AO1)1?d{rt}-PP&550rii+o6%Axg8BnG3>JuXhE4o&Q;`=P^cI9C2Z_S} zfsEC=+&1UK({6zPB+p_e0an}aVFNdB$;H5dZ{#~D2uD=GoB^jVTzsd2~gQM2Wjm)qe*zh>r{6^DtPihLR0 ztX#^20>@8&u3b_ky_c24cUZ5qiJz@-lsmji4IGnB6CM+zmu+i@%X;rm$J~CdfWpx_ z=$C*}$r2?UG2I8)YouR+?l_b1UZSNW@%ISxv|l@!ASt|bQ^Yh4!Q0S2WAl&ND?N~- zlVF_GGD!y!qimUj4Z>m5Q}UEXwKz&uI(^jT6)zJoG9g@nBbHF(w`5$*ik!f66)ejv zO^abn{27%vF)j&j!%~h~^P5l-cl2KBwnh(baC)xkxj6S5-y*vl&pFy%stscakA-nV1U8IXaOpM) zDAQOJ8?UE~S~LA@PHouS&VZCD=tzE*rNWK*M1GtGgXovpgsMOj+sE4LqdwOvpB z?wjkYOuiDjc&WvHrF%NVE`vo8lRBS7KW7gkVP78wZ$h?<9GAZAYt^I8;Kb~Jri3({ z0r4tZxe+7Dd>x?|guC(}9$)~Hw|W)=PftvVS}HeH)_*d>!%+(O}62P9H_Ixh&3%EqsDUY}esaNXuXegCopHVPp}j zmyR?3b9k{?B4eZ_1n8`9dL?>MV#Y;KQl(Z$`pWrkBS8#zKSpefEOMC~pF)^BB>$Tt zkMQcxR+{j@sV85ty0XGyUAAuTF4JnCnRURqJH!z@kXT*<0H)?PEc7v4U0f9z>=>7K!xTX-ws%=4tM31ECNPz9vXxB)i3DAPM2b929 zEyP2QO2QiEG&3ikX-_4j&z**ADtd<=`o3@O_rhQNvB<#%(H?Vq zb;mf|vx1ZYnn@wFk#Iop+pnn<4vT-N=a_FYgdN}1sjT=cNnoSUPmbAOD5_q82PvE?R7IUB8|A{324{!V5 z?Da3#d#fktyZ*?s68IZq1hRkRL;|hi@lXI%D$b5BM$Z4goUbZz-1eWq_30A4ybgvq zVq;-m64|sA1-f{0E}WQ%QovYfQu$+ag9Q#Bw^ogE4~N6HpD2+#*ajG{{Wc$3g9Dc8 z7q4*MC63#iYdbTkjY~kMFQh#x2v@%#Gc@pCkf8+--z$Z}gJ1uPb=NY`U@%cx2umyW z#D~NcGVjE94qI5#PL-2rK0{HD@I}0jT?8Gr$9SFvon%<)WDp7UaxB!`hL~gNWsd_( z5I+JqoJ%=}ehF2W)HZb3&8|MvhdIry!%^AT^u>D}E0Z@=qN=5avVW_&N2&D9Z?G=+ zlU+5Squi{=Vyg(8wAfO+qr_Ar^7&%8)(g*7?|ueJ;%SbXU$#a%^dh=(>8od}_MI5_ zJf7MlGo__Elp|S_QZ}5TbB@8Ax8>b^MZLT5DJqyDv2{|{pM3i)4cr@v&B5iAZR`$0mK@WA*=91@VQ0!I?m zKVmgr0$XhF!z?|9`NyOK*|Z?wL!u~utfAHzggg3pk^&99>Ziv8Zk_TH05dxfNC61@ z4Bsuv@>6i=^C=*9ickuW7%1T=?Z|b=;%HI}=}w|zHUnb2qe_2o9aWHyT%Q#umD3fV zE(>`*WZnt@$3d4l!jO1U>H&P{%IO;O z@o6N~eS~hUV&3Ga$T(0=wIuIKc4ZJxaA6h&Q_}DaP zDN#0g&BrfnpEYvEPAp?fAF713T=G-wi*>6`2cTm+aF&@%iFd1F7qk@X%k}fpeE%_6 z7{PI*;sD8+yQUXHj>*>S?gC(8u-!=sSVb!MvnmtPS2O9atD}~*s2!a<)(gImBzQxn<0D*eydK zRWAyqL1Z#i?&i-F7Sgmx>C7a7sQ6e!VGthfN3kHSDTEO2gzPxUxCg+&8f@(kUsV6K zE|P%OnWn4-we2o$G(#GS$|xGsyuiEm4nqEEWEolg)Fdc6(Qg>E6{(v1i1J<+#1N$D zT7^QXo6@{~{Ss=4VZ+(^@qFe+)qBp*zFZ0d3VG6w-(=Ja?8ulJWW!5e%=MSV%z1zc zouNAw1Zk7-UleGU18o3eCX?u9tYP;ULT9SPtg5?3r87tl3SaGTN}@^Ju&tGHYw5l# zbx_m?s*(GMQDrSlbN)cRkMxSB^9rb?1MqAT~C_VTrP1s$`|?~ z#r~rQo(k}h%mEAcKh*!fS+df{xxfaY7t)}^{&Pkw(PiK^K6hX5&}-r0fb8brxWLJJ8VmRj#d1(sjCuz!7 zjUV|TNA{ z)&{DM7d`KVB|Pt|`Myn~e0IyWhII&b`XIcFk9$>alkJCNW9QyaZ`YW9^m`cQeQ|6& z!T2}_YLp2>RO|tjYz6R927=-6rJra_)>7E}~4-mktDw5M(eCQ$YtbdH~awKt(l`MK48qwt%5j7gJt1SWB>&pvydabuQ{PI zZ5rcMYElO%)jeyD*8L_5Oe~|WZch1X_T5988^5qkn@%WuwP%& zoBpWl7!A#Hh2%7h5qXf<24Bbgt@#MA`BZyiB4XHWJt4{h+Ra{F3^N_Ok*p|Jb!^36 zQcS?pWKG4M6g9|$ZZ|#ve{96PQXl0O$>-3jC{8Vt%_D2(K%j+6R6@~u z*d~q?o{&-F4ACofuKfmV{kjOjJ|va=vcE5I5>!BANWbMzOR)F{RiHK1<+pc`>C;*v zOi}BYl~r*6dDHxPQLxE*|1*WeLBP$tasF|qC__80XPNVe8n24LR!Y?jHJ<-r8j5-j ze3;2ApkLXmW`4Gds*gkZ51&j~(vgRsV3W_0b^#ZcNP8kX!TAxHGwpcx%&!=4H`hj_ z6kMAVQ6Z#phN>gUAM z;1-~lkP{aO$5ing@nae%qt*v8vwMYx7*QE6sbI8S$ALJ$|3?1QfjNolc?>Wl>6|zk zlH`9jmkGOfRx$;YdVvKRoblVnX-)A>YeQUH{r;zs@jo>DA&z)I8X5ne)D0r)p!ov@ zrT;b9AlH^=vIKAyLyG1jgX~Kg__lT^G-4tydiUQvzph^LiU5OwN=xjj`GX+ z>mL#Wx88tyj@yIr#~6>>jTevocC(xhrZ|R?iXAim*`*h+8vj#Id^(B{Cs738$N@G; zbU(d%zB{s!SWEtM4M>)U(Snm48I91gAwH_Q70xk0lNEClV zE^JeIUaKz74x)jNuwx^`$6#;ql`6CDmRa{sR9TSTEcqfyaGbMqqj{io+_8>&POQOBVAJ=EdKw~R z!c!>gWL3g-Ui3j=E*vBDSu(h+MvRIMRV76?9vrRvtjCtT)zc8*=4`RmApyQ4WKj>l zc4|@f4z)$u;^JwIcR7;_j&~?|@Ahj44WIV5ID0zuf!aZ$aW?O2uqN+dbdd(lwHDrk z$A(L!DUvl=Ke*IhKv1))9fI_W*1e9D7ZW;nkHFo-l7i4ibCZ?Xn6q40^!<^29&wf} z6Fc_CmQ*s|>Rt$7G1H}0*=AzGXh4qEihkPv_Shw)b|1y?VM`pB1x&@Y4#DZ-@f1c+ z)5OQev)3lO=_9vkl+)dbkctk^!&E zm#VH4h@!o|HY}j**rQ3YYJAO4#wkNkM*`dWQG6)BVz-y zz&B1CQ=UYJa%EDgOw%KudyB(thOE`^wNAA6p@gIZ$TQPCGaP~GQF1)O0nZpxQk z2W(Qh{fLOckzC->WWR~j)93A*pLvj91(K$N@a2cG8r9T(tXu*XXP`> zCm8Rh*8;uvZ+{1gdW2j}Tv2;!?OA-=gO|n__PzxZrtc<^Bdto7rX8;2tTb1*N&k}s zoH)kA1^nb3Ic{pQW#Ya)#$>v-AP(!`Oru1!ELcm8g~m8cL(1e2?;PseOcYahjO92s zf|%g?s2j$qPNG_3{=n9YBKXyw=0P;cgMwgZJSz>q5gDC1e#jGA`b2!)9VN$Y8!X#x zS7R*x7m2e$b7hh$^h>`#>vHuDPXr?CNP?aMHNa=OLzEM15me#!fN_!HrDx&_KbS&) zpVFVGNBGl|k5*S1`JJjw!8?oppsv}8*Wl^*O`)_lQrWYr`j@MT@@U zG*8v9ogb~xuNXQCa-C!*=XLq7b!e6|&wjqh(RNG@xaXf#Ij*o{arLY|o!>~9!k^&A zm;>6;vrLD=BKk3H+gOC2C|^^bM6}6vt+ZE!L`)gRA zZl#-~$)5LRg-svTXm)v2t{L0NYrQRu3~<|?e|Ol6`}1B4q;v?wdub3fUq}{je{?yb z-)8pzC5YjuBUcG257&AipJ2 zyM*H4&fFT>ZA|6BC*}V=11GYO~Jx8}I)+Ep5+bG-J!br=c_l3&NyqIjH9|-w0 zSJ?mqfnM<7;X;(DNu-?Go=o@|BauABmWznJhm$GubSB(0F{$$ni)Yh3SSjxhLMsb9n z)_O5bx~Bv76ml|poCF+&Wy-!6--+`wnw8{``Hdv5(L_iJwJtxI9du?!tKZU3BS?dQ9E3b{Mepp8*Tiy-TpmBLt zIfQemX;X@Xh3VyAS-Dqp8~Gp<_p*r8%Q(c($fa)`-Nk&QPnHPv{*;&7uUReJb)JLl z5%RB3Px_`R74h4Pi>O5^FWWNDwWrwiOPiB)tt*mQp@CT%Ax&kvD!p4FbP~WAz9mMy zMjc{76M~a}b)arez-ctM+jD71-qtw%^3cA~Y>5p8q zbIWGQa!d8{YEFNZa+q)BFmeFZc_!mDEG3}zq>@Nn-uow+zKk+!BFa6M(CQGi|(I~h77%h z0Mcfmag$TwK9nB(AN8o=DJQ_KZDZ-LO`!Cb-4V#yD?LaRF8T2zc+pDvs}5Si?C^}K+UgoIp(!p3Cw7Zl^OZA;*#Ox zHiA*(LE&PAc)T84(W(KEAwEQ}i~4z^uwtyjUG$`0^4m`FdNY2fGvsyV7jz?{dqSt; zHFK1`TN1qq>g+Jy8%H&h(WAN}+-#vHHpWNqmh>z8O!2Ez@;}s9ENeEBSA1w+j6I%r z1{^kTTT#6(_BAKl1t~j<=|-HOIz*too1uTCz`ssyx0q=D{QzLkU7u>OhQCu6tMhu4 zplUl>GV%3)LUUsb;q+!_ZTE#(G9>Y(I@}%)Cidk#+>ZYEVLIGq4kms_61F1FE-{a4 zMHghzOW=)iay!BK#GA906?jW8m^IFqfat`Dd$92%^~KR4#?0ZS>OwJv@S0&0qRjAD zi8+xTUOyr%8=%z-NsmbK4|vp#zYkKNs> zZ<)$-*j^p#l|$_X8dkTj`a~l1Br+HTLLwEVTjK9`pCn>;WG7h}GA%O@7b0;Vc0mE8 zs{$Gi7yX|w1UollL6Kx;fG^|k3F&Y0<0OH4EbrYayMW)DzHL@28J7(sCY2?K-Cn7p zwRK;L;9X{+NNJHu&327d-YyPM0|RtY~LDLyv91iGSglv2DL- zOiz+MrgQ5QJjA>(0)-jquH`%QPk8_9&7$ zYUTDCiiNbRp#`m`cBmspx2+#DbZXLfADOXi1;wDs9TwOwp$T-o!lPnfFwV4w8xv_5 zzyCM4;U6XWBi=7b_`j9ve>CYoeo)cS$bW~2JR7Wk9YH%lGgg`x(c?$V$Eg(qlGL4?nE5fJ|><(GxbCE)*{Dt`cIqkf=D1J=po zV*_)epb&vm*Sr`YKMV{uh<|W>_f(t#=8Vx$r#vDcfDW-RIKcnKqKuCL`=6E{c8Y)k zh4S~!f?Z47#vcfOzyLRAnMr|u=QN~1qGvqp=3Zv||6cFh&zu1LFOD2ajOqVoa}zj< z!T-S_{*;>b>%+pPhD{Md!UA4d(P9CmevqL8{$BrkXF03SaJiL0Qa)0gU4oGPg0X7; zRb0ZK&K1*;Tg>8=m#}`o23T|v&TQ`#m1O1#^LP*I#%C^h7A9aR?lS{%7+Tn zV#;`0l`0#G^K~*sLT_kmHb{aa)2lpKL?jYjAF?}GWHeb}P?>M5)A@v7e3^ydy6L#; z+QpV3mqnpGNH-9I)+2y0Q2O)N>YX(p^V6LHc6rm(_LKx=@MODx%viKg2jVs z#`-A+1eDJTdActEgI+`kI3w%Z-KF{jpE*yqOBA@LiA+KKW=C*bzvr&NILOBCXU3;G ze=SVIH$mNF;d2@nutGVCdiI_=s6a#C(nIM};_$6lxHJZ9h9Dtz;hH<2AY1@Yu?r!B zNG=if;AS(5JmRH03qn*A9Wx5#Y^yLLK_ohom}xz}KXp7Iq1UV*b?h*>p>O{ zWqY-jK_ynR75?ch@8;82je|U@1k?FbS2Vft@p1MsDn)i_V#XmWC5H_E>zdHf*l^@t zI@}6_OM+0^lW*(ivB4{4vUEOvx2$M8*W})^6&qgSkH-|t1#slPi5B~C zw{WwBN;9)&IR+=g;m**c_P+*atSAmt(zo^{C^~0lgRIHd`=cEEDy%+2n%k1}G+8gu z4IJzi+r@&!z}D2$os~&ISj5{P?2xO&DedQ-n_W`shtpFaEZr4KPe6}ky_r3UWPX<_ zb1s!{DJuE z#rvP5{eSbf2yrhgf)B>ZY6F&Xl%k=@HS)cBBxUX0PY?G z)0=)rOx-B#Af@Fw;d}IDdEWnfV3ipTUhnDz`=0^g*b4vKJOC5m$NJf8&TUAjL_`Tw zb^n&J8$}-1o|w8fG5mhSfo@`+AaX;Tfkr2eIk|K=W|j~;TF89g0y7q#W9v8B_QAO7 z?_S)tT`Uxl7!p#bCNh!&i%|+y2cdIPqaqlMgs;KslX+}nh92mrp2|>f#CF_CqIn_c@G1R>g{Y6z z0bsT4G)0M{CY?T!pa034lK2zWZOcdnKCPn7G_R1;(EmIRpc*4{eN`x zHz4=_#aRA|qR8YXAA}Dvat;mj|A|2i!e__lh5T3SJT*hv{3pePp;?~)lR$!SEGz#? zu^l3k^R3L%tfPp z{BLyz7VXe~8^orfRsL^w+!7JvKjDiOL?-jE;G86D9Ub}DjUOx!93LMGm}|w41B|ug zM*$>%Uv*slj-^A}T+ru2uqd0ZOC~Ro8{^3SVUDwnS8?|L^po#m2P*FwNmntL7%#`M zZijC_Kr&1GfC=g0h6lg2OkB~dT*Y#n=1jwkc6nDw&ct@@Y)9}6J#imqCw7iG zl_Nxn90*b3b4ho}Ko%xXmUR{d8dh-YIZk8w=nfwS|M@gT$Og6*YCzF`F`roGMF}uH z;mn7Lrv|0sQ>lT{i*v+S4R^#Hr6S^*(uWwW@^(zE_}#Km)Tz@TZ_Om>ZoV%m=4067wo!+XF`z;R$R_&&pW_=G zR`XO$U`(%v8vZbyg2_squyQ$tnAi8C8ckvB5FtLNHkI+N!VS{S*sK?sY5YXQD6>t5 zyl?&qraFFXUUEjFQ$O1K{$7-su;rTeu623Sz9DXCVq@@uR7?)~%pjewjE>f#WXW8; zwih6&3nJa>^;5Nt1b7FWhqNXNJ;zWU|=jwkU450hS+J@R0k%Ei+& z&HkBUO0TugmCtiEy^a!3A@msmv=($_p$~h7Z-b|dwCz-#zem6Ae)SLQ`=8_ff3rRu zo55s+4`Isk!4w(6z`#KIX=I8J|0vgWDT6NRNW);1iE^1~wRA3J+D+B-D*=qQ-}`a8L&U7-7ur|G-z(=A7K$>es@ALW zUpo(lC(keuFAwjHZr1kKoF;@G*BaKmdcWofz3Wh2k0ks z3hW@dctIwnz_!JYcPpu9)eJ7KAD_nX%EFC^Ot!8I&~Ke@d@3`U_2@upWDt|!@TF3Oz1tK`iIVlYI_}(cL7vuBaN?{f( zl8t_gG6oY_Xy~>zpMyMrBUu05Uff=7An9YH?WN+QG8)lzUWI&mvRa|eWUHsUzM$5b zkIy(;$|YWa^wj*@RYB__nvH_Yo~N<7#@4*rN^_k-rV^6F9AIvtLrI4lT-2}vthNmJ zlDAfCxm;fS@GGVH;ozB@lS{Rxs?5$#=VzO*pXd5J!B&&Tlp`aUuk%Qyq&**R~e$UH-`Hu+D-F_}KF==xvztz{`)+TlECeryyS$?Na706^a*@^98F7qw9F4V?fYY~=$TVP8GQRWA8fbSrf`>DH5i3@lAE3 z0PNg+cn~YA3QG7i4iVkm=FiG+Z5|$AC}^S^r>vo9(GEr~YTX(psZgVL)RwmDnLEPf zf7KHjzk*-V0J=z*RZq{6jlT6V4BQ(b!2I?{u3wos?+pTJU~V83UCL?I?vs=HhT<7E z_78EJ+X3B5n#$!ns`br**|XmgCkok-0O&#TkmnP|Xo;p%auNKH#eci6S#tXF**wnx_ExgY)wacp%zVK?SO(>$z|>U*0BA^b|i^4P8Z4ejV83zWyZpTrSHq%N%uVr4ciC zohZN3>yz-%qb3m&tQp65j3Oq{C2r#lo#NiBozW(wk(2s)7Dxw<*uwY5Yz!ksd*16g zzTeKLSAlTgz`A}|>~-WjMj}@j1fZ1bAXSNj;t8bA=Izkk=EWNf@Y!^UKIXpZ(09HuyOgDLprZ(Ix-?Twu2m3%fW+&O3pV)VxuM}M! zJ5B7*0u!(?A*nm+7a_@(1IDL$;K*4y@Cdd0g5u1LvQbwt4O+)V#xDR+!9%zKcMi1J9@tEe1nvB=|Utg=Z3}y+h5uY z;jth4t5IaK7&WQ@g-12b8z58Hh_gll9McP5suNT~ix=^mUwg~Bt?hAFs!P@m&JyX- zRo&&Omsm9=@Cr(TghUTIL6tk0cbmZ~Pz|cqfySMz%pvV+z_%K1gh zna!iiwMWznvE3(Jp4Zi$-sXYASSFEcy$#({D*~p>9Q#>^sM9;CHX`n>>a!>@Hq$fX za%_r)zLww}8p;JIF2J+Qo_!)FX7g~<37vkn>u}FG*{R5H$w3-9hB3#6;GO269?0C_ zx-!jq@-VvQg&q;LP=VL?6@7I@*008&>wjB`e~EBZVK4d>9RpQ=O)xc1{sk2%*9@f#$M)@HN~VX#f`-4HXMWQ&hmr6)mU{@ zzIphn>Dm_!4kkB)W{ueYX?jQ|4ocvzI{L+r=}KueMt`ze<1PuZ`zS4wS&a<^ZVn&C zr8ZuyjDx7|3m>@NV+tHre^E*GZUa0&4R%-$qoQpGyEx!9Z3?D^5e-7>)wcGby27xG zTI0^2li+IgVlh`GpwOgpcNx>@Y|tmQ zvFj6p4vjqMi$?u6{LOqFXO%D(LP+M0Hwex_gL7_k9sS0o4tEi=c9%xwGjC9Q{vy`u zXFvQ^@GXFXp6iwl&J{FK7C9&O5_@%wd!I@sToi{XCwnXI`Qld`Qw!IKuksG)pBS$t z9n$DoaDZ%`>iT)P_;?9n6bW#L6t>bEAiVI9mUJ8UnwH`}5_LG%x#PoW(aZx@by~;O zb;XOMTM|MJ?b>TypIFwA%uY`4J_lr{5(%<6U@ibQR(r|K#1ZIGxC@4*1P8HrBDMXr zy_4PadJiFeqrp2_%#%WOK1;f1(zYxcNm7%HT6i(=+GHT7JbdaX%89*n9SCLW;p-0c zj+0*RFR&B=%n6gYpf*xqH1cl;GkgjKdk}UI=@s6>Sv>B7P)5f%omhHse56qgDM+}_ z1kVQq1FvAV(AV2Yo6>%S;#KgZJgd1mlPR(H^SD4TbwG<+1-aJkC||=d-jS;iZGEe9 zkJ|nGP^aSFD&3YhbVCNL8Yq*EraS!Id+MF1za|=3ziL@uD?%Q_{&1PPLAyM+ zw2OYLk$KdMVdqvjzPbN3TUz1|?8>l&T0Sk{3~MrV$7|=s_PWwNhF%CZzL>X@H&IkO z?f@rkku(gLx(JlNrT|*@qe3S?oE|Sv*Aye41p@Lr+>SChNt~mU}+aB^|^Frl+s84Uz)9 zNwu99`|l8|{R}w+yJYXOLpKvE!b~GDvXj@>TZ*0P)o#z~n2_zBN|8MzXGetTNt`%D z-%%mfqL6Wg|J+nm_Wn{)oB-h1!))Vr#y_J^+OwN~$bp5L?ol}AHL zOcu@o0k#e%XC4bsHYyq^z$&C76O<#a*z^PLKm_3MckQR!yI@g1@~IKaD$p{}g?&yG zT-g9>&$a%mKbg}lTyTk>I*A1ugZ4G{iLq>I=(_+9*d{E4KTLk#WX1k^9T7OSYP>WQ zb98129-79T;#Kf|w?0WVVK4Jzr9=NxB7~ugDJao6VTKc&$`CJ=UXUwk_Oh9LH$8tb z5Fok|il1wkIeU=QQVyN?Now8%xfTT3Trd&@Lvmt1E^Vg0SF4R|mmywc<7P}(hR`bny6h+wF)G?ZJ zkE!R#P3thJz)9>(TXAj-ojJ)6PM>)AeoiqL~OC+I$BeW9&2Fi z5XPwb?*M#hj+VvI#8@hen0Ecfi}U!Vdo`up$az`&XHZY|j-FRDSw%0R0Gq!D zT&e_@c&`ITP|Y)0H_4Cw65QNFwBvW43Uw#kYAqLyee=J(af9gN)vLuJTCEc%=8~B5 zO7JgP808_*Myxq@l;XKTVYR+-tFLc00jF#>TqO6!$@IBGSad>$54MrFmMI^WE|!8V z8+M9UKQ>lx0TZZgt%Lm&DsMrC1vrf~vP~Lm&GJBecS3jW#1WuHgOkujkq!eD4O9A8 z(>R_m;3#bm9Nour+vy`D%_-QKd3zamgJu`?b{1kC#o_K`yzPYAuC<*8#+n6J&+-r_ z4Qow4`H;bnsk5uGFExQiOpy`VnmN~pOwl6AW0*yDT){GU5=f3+gA-%>0~eIJE+ZH9 ztXbpuX;L%?skLDZ|9n!Y@(}|ReAnksf;dFB1)_nvA%TXQuy}=BGMDL|kMZO~P!`LA zjEph;mdufJrW5^^4S9PRfyB#d2I-8Ct644CCQ?(HIqLW~u*EUEAc54EX&dk43gu*j zOA?L&WgzykuaIHNvrhL0aB8OdDcQY+`rv8M(Ash?b7y~PHAodNp_x(@Dnrh=Y+@^@ zPV%<&(3h~JZMMVXfTTm5+z#4Ec|IGW7Sb`~|0Gjk6)>bk7&16n-fX{cHihCv5H~0mud>APe^#ltSlH)v20Lbw!=R ziT0pnZl|>|t*r7P^|4G3PyvA*R1zykWsq7SKD985yrF^p9^N4tH}L<~7b;m3O*qcY zuAXI|X`6ghI=^LAcbu0gKjSXeavrP=FU)c9ryFK=hVi4Xp7(fk!6KiK&L>UaaTJ=I zzaR%v1GE4z6#vXGvgn80x=)B5i=Iz1)_cKl0z{j6!=mbFHLB^f8d=%xv5GF{pO>Q9)B2zJ^9wXY5MgJK5;F28tLIEj5+$5@x}tnL`5Vz>}4SKDjFz_q_kj);xxxx z$$eEj%xb8 z_?8b`nPJ2C_vx}JO%MAqMb>0tfFg{D>7Sp$yXD)=F*wuqn9C35f7ggbAJ<0{tO;V{ zkF5|u>|aBL^Av46>tj70!J9I(Cj#N%OK-3qA*wU2d57dwYNL# zhXd=hQL?QbL3vT96qPHC^~#FhCYjYYrZ?{}J1oSKQQ+ZP`*DkH9S+~rwro`rV%H#~wl-7DIbYKdUNuwN>d(Hd+z<#* zm{1;-46}el{K+?RkgP6)Va5$}#+oTO3dJ+_0a93Y0*ZTN%u)ko>Gzlf_b*x0;t^og zSn(2?%A5~fl11pU>u$?a&vg%nls_!ZCtON6803#;X&5H9XK@%N_GfvPdq0|Memi}$ zv5|`Kd5LOz{oTFLQl5D3W7_2h!_orW^JL<3I)YlXCM2u87-&Q<&QEctOy1zls^XWT zPU>BQZknAjtJuy=>eOUeK9>5UeA*DFnP2NPd4=t!fzh|*F!00v%Fa& z6uwG=#mY?YqRZJQr1>~v^NZ(bF0D+mbtPWUmP!??e_ZCEIK&x#Hpn!Lb(R3uOalh4 z#G+@1dBZrqXeA80p%L;*-Azr(dZT(3>Px2Ut}WCz&lnn|w9f&sEKAGj^?xcT*^s|BNXEr9!ysF+f8J2>d>y)b-AyqZrS}@ z8b68M;5AnuMnWUG%E%H^Lkm>@5AKAso{WS{NxI z=UH4>dm?ZGSL-nL#nzmDbbN?c2U5kd#?O#~wMM=M%VA8)8h69sUM7H|;*i~&CTZ75 zQzbvfdyu9=0`^VJ#`wX|(?eonI=NI&Z_f;mS24f&ftle86KlK(LX+Bu02co#=t0^)YR zarsM}^&eiFW~MJbH$4HQi@tX}D(^quhb#1z1R)Wc@gyEqCND(Sm zo|znb@u#yKinWvLk{Y|x$A#?m)9BNwz(bqw8Vb5~W^O809utkw{I^#shPWDf>y71Q zmSell7!L5IFPJJAC&sw(8T{T0=jcMshI6i2KbydF&Q;E|VC0qy1ys7~FmY*%r0Y9o}5SG%f|1__* zR)k30?ESl1G{~7VC2GfQUBGR8rCTy8kSuge>y&3hx{_Bk+RjJ%=P|;ykr3rI11}a> zuTmaet9>kG71G zGnQg!+RP605JgXR68tl$j7%L&Z6`4ALGF#a?y0?B#(tY9wCX2>wCVmuMqHp=o6pQh z?p6PrC+eG=yPngXg}dXOg~1JM6b@CRpl(+k<@V*3WLnSzxK-YyHj!BHR3-dFC?9{L z<*f^3tnRInN%Vc`W*_knkkkZ9q;+QIZbef5g)v-!59C3!FnZ7Lm0-h52o8wt%!_T9F5Y`6q zg_oaSIN!OPpMZAj{%IvmtLRHfY0L~2vymQDIFmDV9#s%yO@^jYQ(mB{cS;ae4@uII& z-a~LfKfkDm_?4C;%Ih8J@z*p6e4-6MynZpB&{9j%)*b|twYN~FY^~?ODehT`ZfgZO zjEp3(x|h1&_5QKQHhjjD`zG@=Rj8vnH1)Q+9OvU*`-yUXp2Hg7g;>H!2kZ57g8M{; zDMJ%Zp4v0}J?U5hV`rD&Uv=yN$1VXag&oh{VUL6DW7Y}lXCY|857QoxKCwviLrHoj zbeddeOnX7q`$bmVFK&CmbpmOXHBgGOU#5vX?4YU$#KSR56nK+g1wsz|8wR8)@T`NO zbVci^1?edJVOI6r7?64|Fft>%`%!MJ8%BFt0UU+gTry1;#O|gEoqwVtr);AOZTEXz zcyalnEScsoY!gV;XN79gVEVN?q##EEVE8UU{R*xXtFOmKDd)_-7`s@kv}X?0)zCXP~=P?~AfFOb(X7*m|FU_SE7=chOU6X+y zqsar~i|0D1ZYj3ahzv}aJ6@w$ixuJLT7bnjMB2f|GvF5tsZhLr!#%Q!x=o3y10C&~ z!#CKW;}zMGC%V_vbIb>)%Wu6STa9bjhrxZviTa?%_4U13r;VO|oU8C<;1``Tys%c$ z&XSKEY~VO`J@(GhuTDOpTdV5MYghG5@S~#k)oI>g_sgiIw*zu@uhk5mq2JjoRZi(4 z;6Jnu3+0HaHJ^DPKMC=|***BphEQ-|D4WBa#>W2M!CH)@TSI-Get$n4c~Y$yz?QLJ z^R_8=a~z>@;dL9tHwkYuqA>}a=DoA?xz!GKqLGw1X*wy(679;{T1$UKB#Lc6b!d2l zVqdv>d+w&*eg+_~e1w1zqfr65oRNBo#cgn`H_jLDq@K^Wtsulx>^Iu6?a#N%o@m1> zNiF)Tq11>>0j|=cH)&nB#qEsISt{Q&>b9xknWPP9{3`M*zMi=w5l8}J#cxw8bkMoA zJB}GJS>~ElB$T)W8imHkgj2Blhj|{b(OAFL39es=nE?aDZ#1$+da=M_<*2gpUOjE- z_uJjKkv4`3I@f+HbltJv3w*lHgI0Ylhv5r1ElGq>t|wM-53`afNXc*iTQ{Kf2S4f^mw&V z=@f=7g8`g}2|HD$a{Lu^6SBK*QTRdi939DrN_Cw;a&)D-TjLCGZ8Z(r_JwfzaChPC zhYg-=ncfiOk-MT-vZ)Q4h$`!Z(>G)$?2zmJ$y9rfOi=4GOL7*WRz;##jT3!fB7M@r zV*(M#gb2Gq;<@r~2uS96>?){?^t5QAjW;L~1HhGtQoAeZzw@gosIv#Lve4xegZ>*c zR`K6d3z6gXUxRpUGrs_SdQiWl!AVy_m@6$t_U!kBu>oit+Hdl);by;3nbpU7tYWdO z>Ru}8i#F00p4-tR2P~wYNf~$9!xJmmSy5a&Y_4&t^lpAs;QV2;AcfZ{PpUa<+e6*V zBm?I5`FGC^PW|h7LR3Z1c)^S(M_5JLgk266AGn zVli`Uv?crh2yFt%OsJMsC*tE0Rb37`8UixN-&%|1ATY6!S+63S8YIK8oxr&F#kSLk zD9rlUPi`IM6?)jQn+rjf&`hYb9OOTBbsZ}j9ps@ZvAfwt$0PS-Oup@wPYtnCd|T4$ zW|tkaPKqk)IULi{f*Cv{i_yZJ!hR%E7>jqH7Iks1>AtN$J~R{O{WO^=lyM}URtK!9 z8HU3fU&v+(q3aPW{xt|$?^@6YfYFm_yVqK%n6^P!;QBY}(~S!8IvJ_bjR-{qPlc~% zEVCHf)XUBk#!Yj8(Z_LrkWP&sYp|I^i2W6(Q(Pe&V}7!!=_P1KtKfcW9px8|Dz=a{ z9}Ur+nk7iq9AKRE9_K3QO}$iLzNApOj>#MAWFp%&A0PS8upzcMpX;i79`hMs94lx zHFz-qPEZ2RWNKIif=6fe1a5X&I+Ex z6z=tF@3m45E^%AVj;aLd@g&*UWaSO)@V2Ku`tDzkQEROSka90h_+)-Iku1dn7O^~2 zEIaDI^lkC`G2{w4V+Vp*4J&}xxWJbjD5_$1RKU^mi0|zD91l&`vl@EshazuNVNAkJU1=Wo?M%!#A27Wr=^fk9*47YC};RXH@C z1!mq%%L2mB`5C(rA~DrwR|OsjxV7JSIo9Ajz?`}1qh=dYdP08RqK^SQ(G96|6mv$( zbsP!oLYp9qc7KhY(Vs@rEhF2>4++Bzl8{YCG2ZQktNFIzUw+ua`z7nnP=U21 z94Kgy!z0^iIm_ubJ)|^)Ms8?QbT|4p}FS;22dSE|3iIz z`28-~&zU7>${UR{5Yz~Stj6R+khh~AEE(i>DtI~}!pD2>Y})7D-&}~D)ewwdhO=(j z4uu%A-{$SMGt>ujvpgoPq4vrhkYtDc3bSDtq;SlQF4^zc-W3>4eiU_hH4elL3i0L| zgDyIj=Dm+c1ikV1=nrjQ+>Fn=_tdge#ao9RQ|B>+xSZ~L9f<@IyE1RO) zhcJ9ww!8)yLNy4`1<~0srH{pLVFt=>WpDXkhgH%pPctmhuuWf!b83gNkkLOmKb`jS z-YGv?vkMb%Vh{RdLTB&`o}XVWAWolaUaoxEknVTXJ8S?(ZV9{}tWhjOdJh}IsjTX) z#dLy{`h$gdxz2P2qL0N-&x6m9gV-yBEpeeyy)+jEx{(`-+!+~aH>DpWG=2@pK4UCL z0%`fe<;da^lN)Z=as1eQF*8w8!+rw$8le+!7*(vYa&WUPL8v4DtU6!{nFgvn(W!nm zyJQtn2|)oaW`>p$-?2QPGhwD-9B9HqX(*%A8A|)z)12n$*!`^%rQxlJ8cK!h)en@y zX!7Z;zCSPgfGIX~$-_~2Rb}2Nk~#5IldvyFzWYN{v_*s1Q6=#kj9nXThgUUfhW3*! zF%!fxOeF#KW61Q_m?j|}Z^+VLlPZa5BVQ~V&Bzjv7-Q1lXbz1`!l)lkND`jF7NU%K zEL@C#NvysqG;$QS1V}d1$y+VB-_oSQ%nf&d6Br@7~X%Z6Z>U* zQB`uRNO6FS5bl~Bt(d>{-Rt9f$h9MSe-^#(3s49MoiP6H^MW1otnnZLVpV)n`j@tL zeKG-CLhWRWgrGmafpU6ol8=_`j*+gGr|dhGvp6zBqgd#HVKh307{f_Hl+T6{)uF=Z z?bmphNCZb+pCl)SN&4)<>1ETr1B2V(e3vD(L>6CshFsl`^g}r*NuGT*|MA#Kt};hr z!IaB(sKA&O>u9OR9^=`5cZ1caU#(fXr;vb6e)=va2}jux*f~+EIV;ns8PQ`59T!d5 zcbOw2#FpooO!$kYx2$!vWJyLO*~04kXq+mlgAZvJXdU!quftT7+`0$*ZbmMhwvM~mFgMMZG;h@MO-2+>nN8fC;EHGxVRh_k7u z#HC75Y{{M`ctyVzJX^KDyWOU2405&c+EB~WBuprIKGYW5Y>0xx9E^?%v>XcM>JI&^ z)~4cguvDB4aTL*3C>26rB^TF*G9(5F3j$7&o?aRc`%DDFyCsRSKv9nd?Ggu2UWv}W zcG(gg(1y^b2kRsde*PLd4OHC*UpOf5Z5v-28OnSRyuryob>ZtO5;_9TjO>YmBu%A| zwKKq6g$Qltg<`KI4-`y|!y^Sr7`H9DAX}aF`Bu%tI=;_rQDAV{ik3RHsRIKo^_u!v zT}By*%<()j+^3E-KZc6sd@&hzq|=JC{?^SuT2^|6Vw9TKP%+Mu+@oZc`3W7BBwMC? zg}4erw^Cou_7$ ziQrlI7xOvbI6U#Y!l|U7&KJ00S)8NY_3~1S%PM5}5Tc3w8P3?L0JFeYbJaELj3ac) z6;+#nxNbW-xWRt1-T`&di5I4e=8b}Kq;OgMFSQ3(OIF24SfGVv`58QNj4HWjrZ^|AmuWH_8c>|k?6qDG0zUIo~SZF(4KGvf`% z7x4{?6!831aRRb^EI`TTb+aOThLt{~a^DIG=5?tMSMvg{WB9o^kfi)?nA;kbk%yKz zd5gG?iuc3*WwK_9Pp%ygir!rBW{?~yJ>yF&LOZ;^uT&aoWcwI09-I}~4lqReaMJ*- zIm-t^()S}8go_H{cfiho(HmVI@WhQMh+5h+h0$i4zOgxQejvI(^_DVafTq7E!T?kC zulHaQ^;I#$Uhy6wxbtSo_a^h3mr92QK_W1Xiy-Pc_IjOkBk6+C8K^FzHHCiLFX98= z;lV$hXgM793883aa1FbkeE{{{Cc+|1|A+Y053Db`3dcK>FCY%(sKE4*_C_l~#gna{ zM0i@GF^e2L(KWQ(c7pKi{;b$(jUp@;9?c3jL;gwr37=?aKhn}Jey@^1XOlfx3cR#XygiKptD_L5M#U|!!FQ;*{2kZ3p!*(ZF2 z18xz)95cyz1Dwd@#awWNP7CM#%3=vfm}Cq6DZPQwo>Qe>DZ4}%FYuFQvIyAma z!ISn=k?PTq62E9J<7P#%c;>*#NA3nf(9e*Gjysx`5D2cg$zEo~;+}J|02{pPx!_xrxi3{Gpi`x(P~_vd!Po<7+#EaN(w+ZI zqU3@m0g&UaOx*#kl{l07;*C+Uu;uE~@v$kS^v*9MxU|*Z6PvtPJeNE9<}M+qWPU8^ zNf=Wrn>|fbmY;a;#RFfUwa;NTM`VnUh~-=|t|?T63tKW_j%Ar+lVe# zA}?SOOV%=TMogbB3sJKZWHecXzmipb8h%h_6|gV=8y2NR0ev=pV_HMcC*#nw2@JE;CL%@r-c+!{PrO;Da84E{$2yEc1PX~>cXj`vPKDQo&e zr!A**{faCgM{#$Qz&a!goM;WYiOgi{Z}yjog0&2GWRBus z$eoLny{ChW8|79I?J>_pHCcb7M0aDlsePB+`D9QvFDXuAE!tPW`HOC#FRi*@Q9#Z8 zqxe*r7qh^HLHb-bZFVIE5A&fUX;8%V0r21*SFYG|O-W4ea+vfhrBS7=$WjWiYEZ*b zFJ2tXUDBE6ekPz$FaDj`*FQ+8h711{5tJw&Vx^*LYnR0p@D7Qmm3>1BOT$q=_d^fr z2D6YLtNEDGrN(d4nLw|swPCdrMhA1AQEy8qVJ^r+YGzF}pS~`U#9pFUW>9UP1?Utj zbY#ca#-Z!JXIpFz7{uljF}44@GB}}Xye<+ES=FnsDN`0Zoo&7@fr;HTcQA=>W^xx$S*vSDF_%F;)UtJr;7QLk}r4EZ3Lh>pQJh=JwdYsy3()&E<}9!Z>?(>+dG>e?Xc-WNpjQWEo1=#R_|99(bar-a9 z{w+`f$G-dVEsz4YOU&RF;~uN4Fd}M;sMs^2l4UcOH`4t?z~_?F5BBqh*9&H=C`xcT zI_mL+e^2p_(&y902PFGOV~3?&5|4D)7c4(R!3u@Mt(D!z=it|@a6ypVsM=4INArC6 z)3o}xttU>Rt_=8_R={OAx~8NmnVw@G?-imbhlO5EQP7kIiTrxJSL6;GX#+`=O#58! zU}m(q75e0NYm1^Ux)y5ri|~9WM%nq!s%_0GzHS?>`r1%~w{gR3OQWE=H&X0upBs*T zF~Lr!{lE)>_|}fcR+ADhN)^8Nxn}zV{(M&CQkDe$UgklbG=Rz<9W9I(hle2UiSUFb zi4f($Wtl*lJjlDwHextSHzbsHG@Li^$4C1ROO6&lq<&FJ4F6AV{Qz^uq&u2QRbB#v zTf&U#AE(wy<~Td2$iHiNc_$1*SYFf2Nd%6n7N%HYW;@;e0MZpJHkcm?#r9EoJ=!;n zLD-sASY2=FB|v7|5ZG>whEyYw^)clz1*b8j;21!KYwUz;U- zi_cmz$Q*%#UuW53%;3s%IHNuU2E&ap>m~`Ww_98=2H@YjbQmN14%7q_IIvBN{xQ8v z$&3B_2ji8?!eFPHC1e|Pc7XBMoqj9hhXbG6sE61)&)!b+OOzKcca57%{Lb-+dx`se zTdL*ze7&Be>u}O$M69w-IV?A~NA;OXS2X?+A;w0Ksj9C-3TbK=i6%E4VRUI1eqm$r z%aEZqKA`r=L&q3i`iYmcBxdUEgXNib+7(v9S}MR@E)Z3U8aB{cOpGumu5{7CM&K$A zKV;}C|K`A8#f4b*@2|v#VJn~2K0XT!;V);FZ7g^#?|v+&Wvo&~mPe%zW}L_*!!Xkx zKOB$98zt;NGPz2DKZvX%HT_*Y5D!*3g;l#SuYoti8E}E#SIf`)ni+l&!}Pc2NmQ5(;Cz6<|HfHne_5Z? zbeh*R>ljzS^X?=$Eg#pLCsnbF{2BX>G&;(Ys2H)$n?&@*CEU-CrBFuoO0&eM$oYyB zS*5WOhik@YE6e1FL!{P(s61ym%`c-C@&=ICHze0iK+c6)-H?5=EAupv;NH^g^4(7; zyt3^i(0pLu<2z*ti9bIhJ1Ow$CE*I!b_6a__D<}W!!D2)To}3970ZjD#QZynMebsj zrK5kz9;X{2;-iHfGm$iq?rMdHYn)K&-tFxQ-qnxiUXFly>0eH$({N&jW5W(mx&RL% z%rLpOB|8kv&S6VfLnXR&XyzoMz)EZpz#GbdM!C)VoE;gFjS^S6Y98GC{4;nLMGOCv zKe%uC*QKhB1f^9aUejhjxWVQMBwR1&PkTK~_{J{L^2^WvhI#!T$It(^5>1TWnxBAz zG?BK$ivGtzg!FKC7x@MKqDM(GVn9pwcjd%LZhhy32O2bN-Ejxd{RWdrW%ZN$Aw*t` z;C^j4iT!reMhR0M8vgx-_IM`evdsOrAf``xUfM>xmit3rmbxIJvAvIR`-&L)h)Zaa z+UJ7sw5f4wypo(dX^%$QwVk!j{eo|*!})R@|MPZh>jS(SNdOb8dZV5+Xj{1zWqF|f z9Aega958A<#(-&5kuu-tz1l}w3%uJ!0u!y&kV-e5(~eUkf9M;t?6N6hwGME(VIgGT zI!ZmL)n1};aDe*zTdG=kfug!7>utKW#pnV~rHhOmP0~AF_D_E-+VZ*V6i3AE?5*um zYPyHAkX)mwckAc<%vDy{x5($2b&HGnapyUe72w-sCg!qhuiM%75m?FQS)&eGT;$z^ z^IrP$5Z43DdKb7A-XlyuEK3TDtg?Z<{ZDmUtyT+VM~bUrL8EPfv&(LQx0p8B1=iWe zi9kHgitc*`XueVLwb=xbgS$iV{x@d7zf5i4-qN2s7b;>aAUP&BIqh>``K9}>`~xgFmn$;$+<8?Nuzd3p%sZV3O!Feg*xbY;|rMIwJ8oOEPL!Uam#>|rW z+!u7|WHR+O)<{RuwIYpEnCe<)Y9fWE+`-BP}YBgCf?R7RHUv zGp3x(>;ksis#JpVJ*6kyXOmHfg>Zc6qjh32Sw}1+wyC;yJmv;G8Qion)$fscPrwaW zUk9&vYh_A^z+;U_Ke$HM_wjYG?n&Xafs9o;iO*bo<7xLzA%rO^-^Hp%Ba5Zj!U+^C z`^{vDBkAp9dNPF+(kU9wEMu@FA2Z=4b@l-Bi{M1pxRU|XEKu9Wk~P^Unz$RXD!!eO zP1l)w&4A;$lZ|cVe8%dxPsfs)cmUSDf8>1KPrqg2En;SErkb~e+kUGDHtL<*7xsfK zSo!c%B6X@XHfJL-&IQa5?!qpI-eCt}$i{glyP5uOLUTxkwTweBzcz~n=5It&{zmGxOH=a5E$e4{PcR7KbTtddW`le!>rS5lFeH?2W0xaUzzFd3pSI* zPWVIC+R{M$mej86n|{uFu-ZIkEg8?eca#3UY}xJ#o5_5wM>LjmX18;MxU1(2slBV` zW4U?Nvbff-(YHhKh7l!M-v^18#=+(8>?;Kn`@N}l!v!lkCsz9)HMTDag5QLSp-LX<;wUq(cX}qJ<%S~?~sqMx_vA>wBG`YL*TNop~<9$;!SU}%0;YuzB?1Sz5 zKQ1S&^m32LA6VB^cY4}4u#KV~Y}$Mn+%q!(`cSI`mG1j;hDOtrL08hJ`Q`8){*$r_$s6vf{`8q%7Zi?m%@RbZ+d zo(DCL4H~p-aeUWZu_)z6ZD6Tb)U$;1pn+D@sWK%kKcedIue=4KTm)Vy~{qWuc zcybfN8t%I{{~Z&J>HOT=xV+xv+5ChlpFnetJSe-03#CtulfAvDFd5(WInQ%^DLs50_pp>+UF zX6B?sOT_=`cB2g6_Ws)_Yl0ftSn=hvb%#s_75KX8&yCZx(LpryAHRO`qa*g^e}Dt0 zPT0HuG5IGKcM~lj{}-RY=T8&z!k1g000snvCK>b>15R>r4-q;b>-^=^|EMI4t>B4n zWeX8NX=t{zDQYlA0k46sw-lAn?6hH-yD)j)EiP7;h~u5;GVH5_P00-oz-~1$fYwXE zHh}bl>T{?h<8*u%@{o9Ryx`t?o}|qDm;Lel;0;1zoMub%TFwwQ2<583LH+%O)CL!0%(VnN;gknissf*_&zwEJ~yO?`ldX%uG9anQpBz zg0cEiYidv^uqeC-Wx)b=?taCDmRV)XNU@9~JjOT#6JN9-Tm9^TL+uzy$o&e3nXPQV z){G*ye1)vZ-c-t@%}D1bylGZ!{u9m1DFe^Qc;etzF3Jv2)`tr!M=>}`BZkY%m9>_% z{R^(=63|G}cXwfK%j}iTNJ1V;Mf+pS@%#HBw$a6(3#Kp8=iSMi_ixzT_7No*2xG2J zF3sTWJDyD>ISi~x{EQ{IAa~Wh+Km%|{X*~ann8EH4Hj}b7OEu7ihBABs=@;rdOJ-s zNk@9qp|Ny8=pxYvera(+VPMHXS^@)8C8b9xwk1Yn^(Q^+IHCg)yq@2$Zk;=5DDejL zh3b~oO<4oAK^TU(v^G8SFk;@D#5-8B?;Szzro)^_pbV{Eg{Wv6- z)l(EV%~YWWeOkk{7)jSm;{?iDM7E(4FVP6xC^-xOBP_NPv7zBf6P|qGKbe6*qVUzRcR%M+Gxob%L#>L)bmP4b2rDXh?(Ri z{KFI=`CCThH*}TORKy$R)7X9w?NxkhNk8X|Qe*+dtJ9~E_NJ{3=`wtxjV9Me>|#XK zeLzp{$|GtZAlW}=W_o+N1x{W}dxh@?WFz-6O6_`fZ!=_?Ib=!1KH}aW>mT@B(CR+n z#cvUNGiGiL6_}g+Xtbc3Re6D%okiSrInW1M+oPMZ=?^+3#2q3%@z4nm$cI=;Rmn*i zZVA?gi5n06D*y^KWSb)?@$GLYPl;%6QD(2Kv1_9;> z+XKo(0}~y3ma4gb^itF0KAG4+OG3#!AuWYjhGJ zXq{yFsSAnd=w04pZd>j^$*B|YyIj7_R65W=neFKDi}B6ZmMw4d!^4d&pz|Bkw}_oT zYVusYkATAb7qElOop0-)u!~`|7yS>l@u+{bUgXDp@7g(^xJFJtgY*z}~s z9@|d2IaJ>?;5bAcN4%I89y@~63#YL(mpU(F$Qv(zl+>ufn(EWq-JOJ8NmiG7zPqYi z8>4UT5u*}_K3mRXHbI_}z3t~Z#eyn*KaA2uvK~o(rhIMs`?WXIbUXPrW2_-*=nQP& z`@*RL+@3sOhq2O(U|)Gq(>?*qtt+KgL_117RODo4$zsl()YUyrfV;uu0Dt*!dfGe( zszcABFAUBYQwgOPB#1Z_qlvRw4;3bp-2q>e2hy=m2u*|hg%omPtAURPw_?=1dBetg zP+~CLNNn&WC3g%JCGTdzXk`XIG}akeODnt$D=`virb(Hi58F|?k$p3(O0!Nt{3WEqwDN$L7r&IH*MKOIP2;V zLf@<*h^#|@evlL(nG*l^5SyfJoqEOS{ny`t#{8b6ClN@gMNW^h26**ta%csWM)H|q zbw+5wZtY7%UKhZ4;A$bNo(WoIHYO_y>SCAHdn-XfL4jp<{6n`A8k-3zmecSai<>%D zHOmy7r*b&@3zM2*S!RtRk&s(N^7eP{x>l1u`NTsdOt6KOI^BEo>~~1}rM7~pI!V`H zq1-ce-@B|d!VYfS>96c?$*6Nr*<41w9Q$h^^Uu-p;!KFZYH_=oWAjhvgNve4axXFG zdX0HY-H6NOzeG?pN|R3gGPd@T%{?+?m(I6gly9a?OScPY*5Q%lloE@M!DaKXifz*3 zOF}vdd;aU+UP$p{bi}QB?>0m2(o_JO@vXkUHdc2Gw#8wy7PEBnTUh+wyJ1g6ACT_= z6xIhm=TF)qlqiXJR=XVz{%`Mp(a?^I9N`RoWO4_GDZ+C>xY3FF#3Bf{C#&Skub|b7 zv{T81-H~D6;w>n>b7Wr-3AW>nnSJjOL4`U;GEziNPS-?xr=~}-pkFn(QL82sS*zYC z!9SlVxF7Z3qcdLNxn1W;qUPYKQ3DcyNiN=Qxw@E1RuO`^r|)$Ek8IGfWfJXKBfZ2J z>DJa6hWluw3&jYOVe;wW{(_gi9Go;B5lBYu!~q1Oy84?g3EeV(rAsF zy3lN0&UL+~{8A4Q>FY{fHf@VGO@ z67I}ala@ibQ{{AhOO+B5v?gWSYz!uHhxNAduLpo@w2<#24VtSC0h2T7tf5T5a>-N5 zS2dK3rR-iMM-i`1&WKPKm{NU`5k~a1c#9n8NpKLnDC-lzLn@{{L1mFb3rIH}Gp(FG zMe_Qq8Z?4SLyfbeDMSCblO8TEWAN|Pi7e2bdF&k}0S9tMB@8ql-H^J74evS?U#*li zqy-2%z+z5~lAf!TZm~4LKCmEyvzuh%KLSru%Qv;DDVcZHyJi;Q-b}&2I6cxYoi6O5 zipp@qA7Y~PF1H+{J!RbInz;f|DAD7S`lVdGJVZf5x$g53D5O~41w4aIVtw{2&8ip*{m7gxLq!-YT zmU)AbjFxmn#vjMl!6 zrxN7Fobvci=v2*{J6T<^G=;pyqzx@!a<_(-yqD2eUFST%6CKVvz&e%}^`KYLBuSEg);^p|_Da5hQb|^zqQa{tp@4aU=7`r0tmQ9Ax)!&9`a9BJ_2bSU1HqR>MXEd*x47GpniB9TjhH#H_ z3g;~nC0~HTNEJ|tFd(c2OY~NQX8vZ&X77IOfYTk)UP3MM=qjdLIrv!ZyQ9A6#k34@ zq?4Mddyt5Q<;D;GFkSc$O$lixKOgN)S#UB6zI{kJC$t-hB10xv3?COr_7CStk9vsC zq-y$KFoLu-8^}cOfvmS>GAcX8V!O%RXY4Wa8VLP%^WomlXF$NGbVo;fmL7C&Jx*{? z`;7@K+euNUjN!x_u!hGFOw084k)OD*#5WPbshk!@E3TC!PAFYSm?+f!4*0Crf$f%%z=f_fq4Ea;2x?pQl5T3D+s$p`i zdcnKVOoUOkN4-5u4sgMproT!1kUtn9XUIaIk_{lvcOiJhA*Gqa`FFV=bE0g> z|2|D1DF6ri2hwJu}X-3AfVL`h=ZEbbBFRx-g9D^=^2>qPAqQU)qr=mi#Y@rmr4Dv+q zc2l1$Af(#+s@tsYlWaqmbX5htei0wdGh?EKju%g_?6USnI;t|THlF%| zIwumG!C)~oW8Dm9odl-mQ^=dAaQ~Ti6YX+wZ65nxb%SPppi~JvA7xy4>-7$C zP^*k)O9g%Scaj^_zYMQdLUw38zD$6II#~zQkK)33z4rE|(rm!pKXs_dn#^IbMcJK5 zDnxH~jekG>|G0X`=uD!n{j+1+wrwXJ+qz>L9Xq)@#vR+XZFFp-<8;&=+nhYl%zxf@ zX056ZRqNE3s#<%WeVu)N7lc%r1ew^{+*rKmpmV}Sz2l4cv`@${SsrjH6u9EZChiyr zF`x^gEGTt-ieIt4RN^g7ozunNWh!?K^#lV=mf{O4%6d$!Z0w>E+P+=i5R6YEDh+<} zCfcW?oSznpvpN5hXQ8hV2{Cq|5sBW?G!;fExiUSyG2<_GEbnw$1zk}-)6;a0x=!6; zdU?Mh)I*fYr*x`9SxK^}p(t!Ds8j(v4TVOZQ<=cyTXp7?XdGpor1?sE;d92!c%Wqp zk;Uv0t=fEi()eMc*kdMnjy~F;J{6Ab{^4*%29cZ8G)Y$D5Y=UYLuzktp z6L}ZJtS=jh(W>F5cuw~GvHg>6`R)-n!Oia^ImnS~@WZ&xU@q^$m_|6M80dR(%+9zL zpcARR-!w2cOBKkfJ>0VL^bfAOUKBMajUKB?HC<0!C@hMAG5&{!s_EfPnubD?50ARS zR6TJtsxO-Y!Ef8D2Z^EH=(-5ZciW%_oQQ}T*^W}R?6N&OT+B)jnsQxIjT#w)RP691DE2^Gizj0&ilw=~}znA8x z3)|2-)DsNXF5z6~NHYJB;n&iM{ySjE|831zG)vKM++%&IY)|;@SFmqG+!<=7o>*E% zJIQ0j>SsK+7u$-2jdGx2a-nQdOGGirfGF85s`n934DrnsqsK=!AOm;|M&?XPB#jQ< zt)g)eydvzH{Lo(ORL++ChL@UPeJIr`jbYk!@du>RsGm`#ZdXDVFt>qY_pEz3O9gatV!!GKw590@qT)m zi2j3Ajt2vXsjw+6+Xgg7ETF6OCIn&+e2OvV!+@Lj%CdT92SSa=L?z3={O}Ufm`XD5 z+FmZ%vq{L!@CqaAn&dB7x1*aB$V~iJE%1IEn8+Z>Ac1X3Gl@;1u`Z89fxBF{ciFz7 zc=w93uHo+i-DyJ+ZMly(uI7kR2MhgaC5S+!T`Rb_kc_g`-3R1n)=S*So3>TZ|E{-u z$H5pMjg=6UM!XMPZu(sesw}XBuChD9YqK9j$Y zUUa<>J5ca&#m4E)%%_wyR?G^aQsIlyE@YuqJVHfY6VmyzR2Da!VO18_imYr`&T_`3 z^%VNU){NCUga8QcLuo3ZqS*YmSfHmpel8`1`QgVRfi!c^d=2@7aNzbasq#CaGiDjL z_9mCP9;qI4?0LiU@LiT^oN|jEKh;>PM63L;PWn3I%k+>p=ZDcrkN8S^xj_w)SjI@W zN^J^Cfto;2Wo(pqxoiT~OSGhJuH1f{EClu)KUQ62 zHg_lNp3SC%^4#FWZs^`v`MXKn;B>A8wl93r+G6@Zt|*c(w)~naU^rJA!M7j!90<8A zWnXv?9t8y1NXX36|3=Pt#qIoF$Z-)_BEaxh7P}QSUtzsOF0RqOYdgjNCcG!a?SY+r zje&DdGOgyA)*nNdnNqSUj=A*T+lyEU(AngxyO@QsbN@gf6ntek^ybxJY+0hr33r2d zq$I>`*HE`wrifEsWvVO%v#8o*I>-YVGjr44(d@7z6AQ!(agr}7k>~aC!j?5^n~Ar} zhE%!2w(;&cH%?`l3pfkfP}(}>%6R5N-~dHn9EHJj1W~;G3OPr>{+9j?E4RgNjvS&i zXyeo-q(Vf5NeNcNWrIz-G!t*#KT}XKm0*I9Zd$bDvIlx?>o29Edqk1Bd)0oVR z?;pmgeEefbe)s)TZ9So#$v_-fWqox}Umlj#4DV7!d!Z#nr3?L@GA37E+9$MxK?k&1 zufk2us64P8yZaUq%|PJlx@j55^7w<%b>R zg=gu4)a1p-X&7lMOwi)ToO*>HwI>*5O0;r4ygI}pM)T$N&`T)Xi!S7cHPEdZRN3zz zq~b|-OSyLD6sVr|0YPD`KqN}sQx2r}HDX2Rjq^Rr?pvOD-ZMGFu`v+$(%K9~ta(F4 zJ`2=ipgAGj2FgL>j7Bx%hSUA9At)W_R(3JY-iR{Wq_z|?cFjt`pH}fgXne+>*u4+= z-er*P2Z>?O{@do?5zJkeh~=5@0GF-*p@!QX!77)v_I8}aVanp1xQ0jJhXk-+ZXp9$ z=Jb~|fFIVPZ&<9Y=$9#3iID=SPt<$@A@h%A_>^glj(< zRl87@8iF5%MsFnePUP_$I{cHR{;xJ;u**VYf}=me@UJA<1Fch{vD&4J(LCh+I^^cd z-{oGpC*>u46CZL+Mh#()->0>$7YsNv>rm}yN7%|6l80H3n%Pi=>|Ss4v1WIn*cnh5 z4pR$d^cnB?&D!18+X1(EWp&^M&*XaEEP{RlWSFH35vUdSXcgz*kw}2QLf_!^hw3Ca zb|UZnaJa9koY?ww$y1;XA#L@&q8kOpW7s!gg^oWgtY(plrH9Ij-@X}c#*o+Sf15~| zCKwjo{kC$ET&!J-xH}Bl*d(Tl?%D~-B3l+mZI@DLPnvVK_$`nBKBf7!sbvqgR6XwS{2xqOms~LQN-=rfnZ_<$2 z=g0UKt0*c{OSbbL@wL)j%9f;Kp_~Pp;8m`qbw+%&N%O|uB7L~kOdQ3 z5p|$cheKH0w9bR;vr_T;q2!OHgr$kGndEf-67TGE-WDzVQO`J=O`!zm6ZDN^qA0We z7FO}zf7R@RGbIGJ_jo~$jVyTBKE?O+L`cQ{4A7bJhlc0~TR2)4pM1FO5L}jj($0mo zkEln+F{Ec0q&rB$R#IxzGp5ejIP2D7*GqtC6qm-OREQ{wsRBjN_4= zZU+l|vJ0v9@EBMuPV!_lcKZ)?5N@mGlR$7o6ZwaF|K8HpX@5a9TqR=3sV!S6MPof?Gzz|~|aGGp#{QR$l>i>N0|6N@U>TW8%)-;>;A%FXZ#`*0V&;QYX z)AXtE5z|l+$&k}7gLx6tW}1-@(>_y~;eqDfI^zTA36hj}sr3lZ3FF{!<`fv{!7^SM z-M_|IF|5cWHca)>Yz^e>RBfY@b?4-mB#NvkoMHqd`i!d|({l zEoWE+@Rl^(2YAaG1_J!DCGzIV7=_L5KO3I}GpOf2+wY)2EJ??NqO@@RzyPvgENx2fH>C!VV>bb&EZM@xXD!&`hW)S+0NJCaWFSe#87tK( zQ#KF+fYHR63{-+`(iDLXqQy!k>(bAifnQrTJh^!SH}JSkO4D@+Xez zTo?j?+6iVwXC4f-g^FoV$v@SaU9dh^g<=(>`ZP}Ine8z#wNGpy6ns(^YnC9n+vvBU zzZk!%)%0=V9$XWKG{5ghct54el;<`J(&_HoH*yG~pmgpUf-DguOY=3lj6CcsQw22e zE5%1h!FXst){*-b_o0C$Q$Yij@`p?=s=qGXuuN!|63=Fk9M|ezDol+(_neSBMUh+^ z++`a|cN4b=qHOh8GzmZUs1OO78LV>yM_>Cfal(R0O^FSFVw~=x9q$nH%g+3`>V?z! z1)_&tZq9?x&>WE?`syqpL&^UB>bWHD6T2hu2_@Hh=GxyC86kk-LzR~H6ejTfmW^mR zKEnmPZM0N@4<5sIo`%R(4N##e04CZ)YwT}2I(og!1=iUO8KjZJBFp$QkVdwr<>sM|X@7>2^gDj5YA6^*Eu zwjVGPEpBNY9ybG@Rc}LA_TcHg4X9s&m5LYEQeo*qabsNm!n(4iF(*cli?$*Y(YuuI&Ns)%_+R=#TY*A+c;Kwd&pH z#Fv(9=PD=N@PG63LwZA@Qc?!*>@x?0JA*O0`aAF_nV8p501E6CnjUTL{2OmFiK_L) z!UclOskSfGXK+tdIm|deUe}NZ{}R$_OzKs!QFsUM4f>s6bs$}=+s~!zm%5Q&=lsh0wl%|FYwRuf7iJIX&p)gS&FP&JFuc9%mge*GC!8m;b^Kk0=y-5D zPpCavjcp-jirX{aZBTGjc*pkxVTPnag85C5X0wHHT7C=kYxew@344w$x`Cc>>(8Ac zNk&yY@=- z@#t8L6h-*)$eEIO74$Y2XC>S3G^k6hL*eS5V{pctF6|YScNy&YcGhsRF@xb0zg}Th zcxe#o4i6iFMSH`gl{2dEXS)!|$6EX_7t=9Twn`9IW{`r_tSFGVSns~rwdnV^wqYOj z*stG#e>AO(s|X@=_<~3?``HQF0s%=#;#k5HwEUJ{8AxN_s~dNR|0tBUD{p~EMSBJ% zFiwaD#^4OZcckCH=KmVhhgJP+mFyH;ITh&?O`t)yb^+%_G{H`@b2}ovxny;NCR^yC zYKeB32$15Daicj;KJvV)mM|P(vwi!cO(V+)EMu0{s4q8f2pX2d)?}>ISyD-pT#_q{ zWvC(@V>qFDp*Em`>#~kN(xaVMtdBjhjqIHkbToG+^3Y zKN9$>+4&A1dz{!FF}kI+g??Uo5g;xnNSsCmAIpVrCY>I-iJ-PQyyH9+B>P7e* z7=KiC#}&za%P`(d=Ep(h+yJmcOZyXT@1!$SAd;vZY{pA}n!byiJMK~15nR@k)8Y7! z_1t)Xwk3gle(Nq#0_IpU^1{qHeHhJ0vasl`QDU_Hk++9=9%J*G5`=LVBtdiRE!!M% zukRDKk-N9;Nc?-+Rsio0T9{9~Xct~DFbm1OX*>rJ-WlZ_htqdVI;xyGhM)fi|BpMy zHHC%U{jS*I)nij(uO$|w=8_dL4f@1ss)3|CZ{HOa0uEfZGG;H?fq=PZhn~M0+%}ke z=R~px-=CY?4OItue;c#$6ARrnL-o>Bb$q!~U0d79i7NYPUeqEh3vch7n##ropnla# zT3u_}#>B)4S5M_c=HVeNjmAm~=hC_}L!+kVGIxu{YGrCmvexpt_V2VT_k&}cbj%nO zV|%1k22LsWs&lL$dngFQerE1xVXPZ_OH~qE+YM)!{c~n}|C=n#%UhEwbfU;P_ttSg zYQhx(v_3yl+n9Crgr!Qb)8z&H| zzpY;h%>g3Wx2U!jzgw`B00F*qr>rsjKWz)OrdO(Db-Dhr9=dSpRQ)|(#;oDjt&`Jr zkFL$C)Eqro=B(jYBFV1RyuIH>tntBqbjcsNCEQ=)<0eOx;BF-Sb*usXFKy9FS4q7* z$kKS_)|!f`)RulYTU(82d68aM+=EgI;*OvqIMbSeYmvLzP~Pqu4cHUnp&~D1+!Y*n zJ~GDqbaInV(s2kcw1iGoR+jUzDD;YTw+m73`(QW}OyM%Y@8H9dQ}fI-$^WjJIVX?d zYooR(>?7ssDsyVyOuW)v)0(>|ug)v_I8Qu%Ojh%+@E&^lk*r32zEcuG6xI<;_qSaF ztPPXIDLfM?8LU?G3~+HTagE1gUDzER&!>pn?L}B_cg#UJ>Ai zC8hS7*oUA&<2v-@SqjTZJ}y=pX7==*h-Pj?Ph~$wz-LCa@JR3QH*0!f1=LA}8DdOT zyUB@7a-BTQ^V?7d3Kfh5>0|m;5=@=E9cld-x>T%pyG5-w z(!P9KsMiAl?CUZ)mKLv%3cfMc!jWq8!3nqwE6GFdKZyy}Gr-(B5q&e|+GDlZK2Z$T z6B6lMrg-v4O(n(;TF!!rvvYQQbqvT`cJ68RA}P^o+C8>^ z%8yD8yS&Q|$a^iD^FXgrR>ywN9GU0ncc7NH`gJ=FL~-POzB72q{otxsnWA@+6}wOE zCjyGP;Ru3uz9QYf@IT=Cz0mIc=<3W*LhFXor&Z}Uc24~Sss>IB50|Vse{#!{)+&?m zHsYp2S0McLHRz9%(TWp8<6!6Y@^83xcffKygE`s7t)r0^aOFi{7k?NEuc}ByZx0Vwq5^PydFE(Qk?lP>7u{ z%?o6?fOpiDe6nD;Hg1Skc3VFb>*&8@Xl}vYulheMU zK%)&S_`-=Z!RSgtHos~%!z}66T~vzO{zN8dGC+$pLVC-Ded?QABkr#-CdC)FVi=r1 z`)(KU^8pq4A=)FEm`$t-UfK?NJrfvV9am{R8m->IPgnzGeM*rpR4V=4&Zeu#n+e|h z0Baq0YORbb-N8*b4b?o2HWz@$Hld8a|2uU5_2*^W&(EreCmJU)_K2lQr#I$5M#NyB zHrX|+1T@aQ?BAplZd3>zPN zuC4X{Et{kLR*7m)bs2Uile%%Ez_xFl4RuVzXTV85~ib>IAKz((?XNA zxNTN#V6612wC_EdIW8!ASyA-y2K-NZ)c-Dm|3@bIPkCHzvl)-~wIutE>)SWJZ{O0w zpzzVtzQgj-0&4&WUiz9Xh5|P1;|B+_S*gXL-)*u=ed{)TH6>VH>Rgp* zZda4|GB|hAxL#X6-s@WaYr1xJ8V8Jt;Er;piJ1ceagwC^JjJ>PmOq{6KC#uo-bp|r z^-mNakp{1n;XIx0kv$g1kBPlc(6Rdq0V`w!s#2%KfivBXY+59gKE1K(*g$__gd`gR z4G29R6+TT)R4m!KVv~#>w>*(S6WJNejOk#49DAAZt90E%#ku(rBffNVsfIlX&>z|} z#Rj?~`ohh+*hB)DnO!5WBuz__xFa=@RgDy~hHcBNQA>~;M#xt9D|D*&UwpLe%o_rS z4;i@|+k=Kv7`Yp~AV5_BdrN07kR5LRPe}}rANDk(9H2Wvk_^Pe2npih8(JaXby2GW z3>WJGnj5Xn=4VU^KnG-4`hwYX~9T7b_}uo-KA!*DAU|((eq;)VYDq%1DZA|>P^+WO1o@) z;Y-m=z%<5Pw4JD^XUDVppn<~aQMr$+a<(iTY|DuaJxf&aK7D+vGe>Rd zI+#kWPLU5SO&}!BWA|abh#eLKoMkwYiwsXP&eSRkCKV%-?_v{XWbC7LIf|}koU_eVH_+8=;5`F^2D`B zPf2+uoKF+Ijd(=pDdy}2{TE&8{*tG)gQXgpV&D1#IsD;tP}kg70c6~-EzRM^80JDF zcT)a$`N#Y^g{nWvKX_4J%GjhB`-wGZ2TOKY=%g8{a_ErP!Okx{e{;P6=X{-1i&Ggs zv00qQ!Gz_t1hvY1;MsgBFeaSKT-592ZZflWx5`x8B1o5_R zzAbP!-vj5$m@&56!mGpf)ec9tHkM7%>?FotM+sQ4nTEQIk`%8ARDskh0_4!80StsP z;nT1yuJ=sMl8W;TU3^Ra>CZ5zdE|H!b)Og*M3B0TEPMwqbd zIyg0si_GU61}7cAt2L~memUmclh!~h6 zlt4_lplo=e51)RKE&z=J@t&7R%Q|ydhrf7k@lJ75#GFj^2JgtR3B>~ zQi%lojbuYnP_JbrS?DMhvlQB*DirkGViLPE9keoi?4>=F4$C_vtWQO$>?PRD5h@}a zR)*A=!g|mAQX%HK7>D@+`7Tl5tFtLBqICrOu2xXA`TFf$p}=hOi%+3nfT7PnZmJuC zihy0FlWoyWbBrw5V@bG75Ir>VELn%-JJJHIzy793+SSg19dY8PUNd;x%q*PB=bxP6 z7aK`rX&Rm}Lq>g#RqC29OK8a-z7{1ug=<$zHvQu80_FfnU0g*X6V*O&>OgO{L=JCc z9B85n(-Cp$3?ugvcq)>;#U=kz!l!Pp^n3guq(w<6b74R39S5v4EL&?nQhxPNOPuNeO4ia;-sG`l~n zX6^c{zu~K&6I>&pZ(wkKA*x+fS5)Uew+My3A<*4RoK`ou7@A;%JsR`^#z^}9UF}1z zK7V5oLfqh<&PcYd?fIR3x2baQ$>|>uuirh7h1A9-vQOt`JSjDSH&R@iM2dXux{EB_ zaV)!Z_RS9vq8*`GJq{a@k!F4ZLEc9@dx+Pae~LlWrv`Jq&tyWsv+ZOZlo#~Z%B#ITj+rJ#N@kAsP!);9P=a>X>(>vQ4~S+UN{?uA zt)wq;z$E^PLXvfQaW~z>rHDy1inG9>xS=SN8&%2zHNJGs`cgiRdiod7nigKLKb+zP z_tAr1BB#8qz$6n}BRRV?DDX{|{tlTjoif}r-#9U<8zMAml?NhLmT&?>GD%~PsQi0U z5J_{&9!!kI6J2hx%Ns9E#ee`%GG$c#$RJo3<9vW+QGF_s$p>R(6~cNj!#Z)$*&g5D z90_+znF>i=3=X*wVd$Yt=?84X~W}S^>L^&{3U!h|ips zIcUoi&)(FzRGI2}Z=!1%sFB`-XHNq|oxDRIA;T#=9ciZ%NsI?vzFZiS(muH<92^1cCD%@&b_?22ml|3$M@Ra8yR* zd4iXH+??K#7Vl+^^Qgf^Sxa9v~1d%vbk33 z^xRTqq616VAh@&~kDv(7Z>2z%OFWr%)~ES`lOknqJz_=C@)!N=G>0NVEWVT`rD6bI z7{AFCr-NYYAAsOb3OBL?+}07{c4is*`A$N|9av8zr0~mMQa8pxhVy=@h5D4jt0TR7 zj~~)w!V$~9D7KGegrjJ&HjukthxgNK+0|p+pKI+SgFAXDSJG33x>1PdOe`7Z9*q5) z==jXtfNA)OwMWP1Ez2_|i(~RPO-&}|Pj7zh+(AmEShSEIA+0$LNJTaT}TG`wVy4_c;X z)o86S*p}y%^4#-GnX8t~$L2)7ZYu~z?Tty^p+V?erLZ9(1#D!-KG4n)PyT#hD<+g0 zY{6J5c6J4p6y<=euOV7LlCpy*?Di?^G3;_kZCQi_GmEaZ*UJRsR%+~n z@^@c8i1pz12v+=DY<9(5NE^GPXv$UZHg`KCv4(L1zt-}Kmdd!UEdObrFm z`QvwADv0&s;qziYD8sDS2b%nzuP1h+8LrSWGntyszxodiM=y8E33DDeHrzI)Z>Ijv zLFWeC_I3T;E%p9% z_C~A#^i^o)J&v^M&f`KI&Iw>*A)4TB2LsoxTU*~ptR z?V1fcH=H^RKJTgTVW~RbiC_KD*nDVN0b;STh#Y>oGYQKv{0MvlU*z1tPFuTj0Fd#;# z+>e{q=oQ#&mL?A&w3C7Vcf8`J*0< zD@FDPG4DS3#{AxR-lDau0=C2j8vt z7r~mYv?CF7VSrWKD1z6ieDm+s#Q?2~;QNQ>yyUvM!^5>M*81H z_5Zir&iD8ERAoe=ZkQ-|&TRjR(Q_RVfIstcbgoo(DvGgw5y3;aN z&M7<*!qqQ2EfsblVZn{dK*vBMM#rK-MiJ;)nXDe=8dVV4Q{$F3sjD-vo2(UeRB5vv zdytA{pbbh$ImK!zlp2|kUsH3GSUa@h(TIQSG!}iQZ+;D4kPrV09OMl15=Bp(nP%swUui7HFhAJ z1LxF*`tGlgI|;wj08YFIO$fieh)@^3wQoJD>fY)U0=C}k+0cB5M9)m zy$-8EkIp{ItwxF_ob+dru~K8pZ>D`Haok$*Vfm`4QhC_$#7~NT(tky=yW=Hd5NU~p z%fWv3T}Nb6jd0#gQclfW$<2S4coG@bljI#$(Z*8grIcz~YI_W$Fu)~8$OwU{&IGD> zB>Nb6dGKrze0bvRr2EOJI#Uz86hp9<=6+%OXa79f>E=Yz*uriNHxr&WxL7D+QS6xy zx!CxPCZ6DpA%gOzx*lx3$Fv%JkO zab~1HLbJ8BI*zH*GQScn1_uz{-1rAIR#r}WZiX&z-c0VSd8w5phBMrI!8!NY+$5dU zLh_1+|Ku5Zm@4}OdE7$uI{~Ww-p-zZ)!-1KgK;lzDA^ZMl~HB67n!F70MSlwIaH{Z z$auWVMO#}cv>FEye?{-fe74s++pwdE^g48}6d=pr%Qvu0yK`+BYz1_>RF{dXX_BUc zNNpvlf!)Q`?q8>~75mwOUq2_>81}4W+yir(D)H|RM*D#B+(2Dd)QZ#+EB{p{L`zfb zy6n4iVeX&xX{(S~vTnzHqZYGs^e10n&~JZ|i&T2SxEkW+W~^6 zJi!pN8h!_qOuTe^4Y*X+TXi|Fj$vZA4y}6Ml)B=GSpPG~-ChTM0aQWhgvNLO`C_j^ zvjDr@kWdRJ?f0-84_mivOOG6^IXSGDGS>Fa)Yo6i4Lpm|`r_i&FZ&>a#y-%KLS$W_= zc|RjkX&XB%H(R?*@RSyG9Vc5mIn>boZZQW}cM?8lcO5>b6}OES4VSgo@z)8#ZLQ?6 zRPTnvW~BT}Q$F8U`|*PQ;VrBujsTrk%zM0W<`4*`Lj9$Z{!eiJ|2GkJ`!8j#e>Ia4 zq+OIS3#9!D;)Ma`sGBG+e#Z@nT_$G0f>x4H$zZo2FjY0<6#H&KN~Qpp@7I?Sj5W#C zs*M3mjWpeZ*|!2FU-x(*uoeDU-@M;0n-zNO{@U>Q^n6e76DFNyU_=v!FbSEOb=Y|; zFw`a%epE4553%(EQr10iSCt_`kgDb5&ofa$GGb5LrCJ71?$TRezTxVABo)rnjd56& z<4b4xrIrugGj&zGk|yS#)^982n`lyuMf*%IWIOrPVjUh5-9l~@(nXG}unEOvmeFre z_Uz1oDN9c0w>f*V{F*FEri69RwLi2Lxv-jM7pPNy6PNLbJnSo)s+~e2C7X|ID7O{) zz-y_Gx!VrJ64pdo+|WY(h-DeWEi0e~1BcUOMkT11{94IIH)Vx9L72u(Cu6Cp zSN;Pj=~7I=EK<_?~U)S=x8uNLa*s+(jpbNKQu@Cz{Bv_Ojhc$X+?}inskIAb%vBf^GVOlhGgq z(jp>`R%Ph&fN2M76Tnd4@YS)kUMrNP6g}v)X+kV)XCAWzxTvdiq6c*gniBb0hci2K75Y>;ek63elj~A(wSF zvsW5SDB}#zy;%_ss5b@^qT_K=+G5)&<~<<6)+wCnVxL(GpKK(w^^Hrfsre{;`qhd@JemylL#>4Id8(&=8x##He)=(7WH@V=1OOV zik$=7=^9n*wGUMKbMsRbuih&wmT~N_YJjkMDo=++c3a=vgV{cJqC&lkn)tX@>)=Ve z(S~-}Y0o^hNsRl8oHZcGq;lo207?7U-09u8yaOLGaFpp~qWCy?aIVZFP06`=pS*K? zK|E7b?Y$%b`jtr6o}}R@u2_0bezx_OK%l&+KIL=#1X=zE>l0d`s1pfPpZKGl#LB|F z_>hc$X!c}O2TZLzsY6G`~PvizHUIbw{}U#dQu zU5z8*K$HTax3VJ(zk+pJgP*Zglo;&~Wa<9O8T%*13?&%z+SZ?7Iduv8@}7j`n(L#G z41EI~H)8_PG2KH}uVY#dYydo_1dDu&V={T|lsgUiIOQRiAK^eSfqeX06Ih*WnpWs| zaT@=rQ8v7j#bloFhVR&s;pl5cqUWdmw{mlqKp6Kqd0zi^QQa`5!H^7!B{crSBOt1G zV|2nLv(CskyKFCA;-Wc2qC?;eRikD^HPy><#jesXkka!_@*#hXL^)Zf3%yQq&!2 z3Qe^dteV5NMu6Qj%d%fDId)vN|4C2(Bqr)7l7ECFvg#~lRZSsED187T$=(HVL2rVD*AgOE!COH>&%TFQ^ z0Qz$_fJ(Lk+zK|riO+T$dKl05BS!)2vvCl@g0aD8CvX?S$D`_AnN%qxJ)1F&4$&l; z(Y$T`_Di#gGw|W&h+{Hitdw470YWWKks&GPop{`w8^rL^ubP8%=(~Y~HcU04JP)Pz z@-gcI98xXQjE8ew^%2YUKs;`e#V9AzOBunvclwvFl_!2D&TTwGHRp{!* z;mM}c>DmV5*S^R5NFbFt2%*JhN^v^qWSWHYkSrG3Y&4IR$LkMu9*IyE@QDb20S>>E z|AZAXy^VC4x!Uo5?&Sz7P8hQBRFzZ%<^p{G-Q0jl!3n21@_-bi#tHPy(hhhwS>K3L ze{|am_k>{TJ?r>do(h^4$ayLgQNwcsJ|bu_BE1Xb;B7a-S{;+vGA<$#lt{#b?X&cd z3BQx#!&u9Xcx0LMd55a`KxD3CU~y!0anfG61%O?R-JTx-UG6343*?j)C)rY5ljR0G zqaZfDuke6)#q?crW74LA|Z&+3+VBLOm5C@>> z4^lQcDOUDy7X0?!Zrj=g5ux@K^S1hxuo7x#()R*mj#k>IpAN!9((p68Z|T$~ORcWp zD0&}xi}YpyvC}2L81D{Ox9aR$h^3N;esyEFE2OGNST(xLSx%nrDxyoTh(AeGI3Gzrg2oyus9}0d5 zLkb?#5N_8($-njF4brUq5s87GHc|>av@IYviuHmLHQSwZXI1^Sk+G>*PH1Y(;hdiE z=JKyMj~#Uc-{?+pj%jVh7m5>?9y}g0LYuPFvT~k*SMkwI#1V(@AjR%*n~Hpu8a2wc zQ%DfPO_i+d2|n9nGNhTq$4)z9kjhtn+sgo+gzXMp6asOgg`axdEq?ifmWOX%$~p%qp-B?IDN8A#$vp@r88#hyAXN$h0?uIr~y%Qgsvn_+ltqN9ee$r0em2srd1FkE6%-@WEZ;Ttrb8pLR=(VOZGr=+@Q zMcZfx1K40-B>9jd;nPi-W25K^ah7@MY@KwcMs-v{uH?}OrgXwslh+x5Ko&`gUTjPZ z=j$E-h{2SrHVSB|U-ZGU)TeZQx%5G{!l&=i52_ln#+4`KaNV!o{-yR3QZRN+(4$$M zFIe6mcPBP4BY^NnZ~c*8%mk|Umo29{H|MTv^G^=rn%nFBzDRhyQ%BXC3HFuA)$c-h z1QyK`5h~E3JTe@1LM3_@_Uq!GV!DCb1THpDH1~o_9xK4l5L+?-3-4s!p|F#t?$}>? zvv6qkW5(O8;Mv3_1&jq`00avX$eI5|BtRE^9K*>OV_ef zC$PW_;Ko58SXlbbf(hTCl_jx$7o;1Q>S8CKDVEZTsjJ65P5@Z7UzlbN^{H!p!MOv) z1{|gF7sW!3%p7?FLx4kg_t{&DKffsvQC}XO`ObLHxXySReSA%acfMthq>&N!r+ixg z(?{wDF^ERr3ic0j5pTow2YZSGz3`IA^haZ)bhi$cG3y+!S+^qaHvdw1ig}@U>UhMp zHFaxxrNEJa2&>>&D1Wv+qyA-`W#Ec{dT7B7!ImKVCiJr~BrJ#)GX@Lpbpc@_F>z_x z)Wv!jQ(Q}!C0ZD6dM2=6e+Koi5t5FIEL~GhPK!%c5kKh>Jn@gF>m>5GfkW;4?EG`Eo75K068vQ+ zq#o_@!zZt(>z6dlX>!UBU^O*nuHD-Sy8OCIOA``Ev5HE7Fm01UmWNV^!0{J?#v8dFU&~0E& zgb;NqQ_U!ImL;SSwAP|6CGOrjjw-B2*o#n-T{NWN9=%!3^#?2-bW*RWeZW~Q0D#16 z_}B?AEEF#<#Syk4a6oE9Pd&ILxLF`^vpudr5@D@qvWb2?NP(}M%n^`5CwB=iB$I0{A9Tx z4qw4mA+inE19lm_qx_}M(SQB|8AXk!tI#V8%ZCT?dgS5f{Do1(sz?JoC7HKMm>6L> z?ZI4i8~88A_M<5Jd0o7URZz>t+sQPemQpm#+wvteBP_BJz-1-9(n>1LEzGc5>;H?a zw+yPR$+kstcXxMp3U_yRcXyYKyW7Uy-3oW7a44W~ch`b~g3H(4@Af(O{n!y1d##9! z6?10h$SGq?+WoY7EmpxnoyJMWOMmTA?JEMN^QYStnrBUOa_TZza2rk`c~i8Gj;Vff zL$P5oU`|!5t=8$uRMJIWh(m}Uz*ObIW`|uH#NC9Hk!P$641SZren9nw@^>FrW1jyp zoM$}u6XBVl{lxMQX1D5DW05ivi=V7L%F z>vw^LzJz(+Y)w*Xw0>bjsRz1eighj2k7#>um)tQ$2{qWXArv*<_il<2<(3Y{vJFx?p z%JXf-VH+++qiM|b&O(U+fAxy1b+5*GR)z{5_Ki{ZN*LShFBKoGj{k(CG2%2KXvp-Q)e@M;=*maaSO4 z>ZaJX()^qheu0%@Dch)%#tN@-HJjQBF1^8aIS3qGQ2g(;KSf;t>($Y{YO9xk+MhWc zeYATDoCpe@d$JUSRd@H)kh4Y1EI5C{`@@ISO0WF7_lNq9Dt<<^?(WpynDlQ{Xk|$| zK}Kpfg*;a|VyVxwF3&7h(amjb&TsK~KER4XY!tW{S!5QW;DWJ`p029C!=mgnf$kk{bJg5-`6q0xlVh{n?Xd+2du9VX65! z<%o!PK>Y41}$$IQRf@8vT)GTjBkV?Vs4@$eiCi9GmcY?B${OUWLYPR|Ms24~&<8^C% z3y5Nqxu%z6b_Gi}`sdjH$9C;svCZbM2-g2T_S+?65Za+2TE4nz+l8a3o52wO8QR=> zJb{0KfdJsbCms%P0ErtJxQomK1!Rb&#sdh;1%rey!WYdzF4OXtpMw`wN*9T=R*xKv ztx@T9+8x%4`0N9Jz^4oEgTK&>)Dr5Frc;b9`dV6@E5&{Igf$rkkA~7&*XnQ0L zW{2_ZssEO6_h%SVMPPQ| z#Ia0&`FWiSqqL!2aroSEo-KTJPpMT}cGHKxJ^uikq#%7ZRV!`#<(o7>v_c={j)*UvB9`+%- zn>H!0408wnz5+2UT+&SQ7@(Xq?2MB6@KZ}XlFf!5K#LqeuX z7!G@(5)+LX-zwdn)y~{0$RD-gC|s804%8pRbOosnC5c9sq0Y*a|0H)I&I|DV{)cct zK%G!q`+L(dNq9&ogX0>_ddG~hgDm9}7LXu;no&wK&!C_9NtfSuILWU#4F+E@kxYb^ zPXV_>)>^>1m#9LiuX`)OeL`SM`Z=-l{3Q$~4U5?0j`_#FWS)s(H1ga= zuc=n{8Kxv#ZO-}WEx&3$vAC*70o)cmFVC;c78l^&U@InWwJTEWx4B#AKj;#lXZ?QT zjcl%H2p6eU84ch(GEXn@-%WV!tu5ML0HqNRqGyH)MJt3a>5|4?W+mgJ=VDGpZ!wh@ejNa%V*6$J7`H z=8Q@7{L~7+?BoabKNkL%8NXJ)%-2QXe;c?xTZfqHYY#z$`**{=a}+lcF#P)K3h+hX zj4pyO+ZrCOS{D>jNyFXLD%?xYT~sfT1oq=eW-iNd+|#&wYAsLG`=?MLV3ZnJ@CE!` zg^2x1R1eqsN-kxF;`M%p;_6072mm@D=8YeQ!hb4!k&Ka8=1n%LMw>5lryoO0TV9sA zV5}L+gJ_6iop*$VZT&)S#s>k`1;{gj7@~D-`YtLslgpURwXl@ljA(nU969AQhl|2< zjTTkWLD-&&BIQCZ#Tc9A(pw9xYc*(Oll%E2b6UpyG}w3+Ay|>wG1Q_nmMMc2B{o-Y zY--g(u2GDY9RCS_KS_9)79wO)ER+z&QLi5A0pbrAT*j_X+ps}`Ia-DVJm8#=I%)k_ zP>2LYcg!@wQFYZ~fRw2B=j3RzJtaN)5a9AK&_j3Pn8mj!Yqs4r;jfIMR@Ez3s3|u@JuUgI3inlr zS)&9~h0ASmQBE49s59;mUJpVrQiEtxnyE_6sw%^vE6ldEorEr}P2CV)aP%jh6Ih9d z^(t~LV;r+a5rB6Lul4|pgE#T1`<@7#`HkEwGtg6{B)Ms9wXVHd8gK>Xu9=t1AmWwY z&N_4KsAdD7_f(P4#3FAO#RCCOR9_BXry~F+t(IZ6^k( z-6oPm;Z@)tVZO(dP*v&`3kCU04MXSA=*i8Jo{HkqK?`&1OT^D13G6acGVcPBJ@hVd z{2@H)Uk6hFU>unYfaO!#aUagoW%yOrZsaSFbA0C|v0k+y9A(oNOdPc+)-8;&LNxNi z0co-^Q#Bkh**Gl7F}{&s)^KJxlKgR)(~l*NiSfi@H8%_sehkk=5Nygw{E#-kg-boK zC0YOQclgQ5W{03j$@(d%#8lFLr=`Jc)NW!bYl5ehNKaeGR812kFJu0CHrNrPsw{Fw zkM*VrT!g4!@k#UVSXJGQ+)&3=Sg4BUX8Q@z7}5;(B%XY z2Y4IC@eLdv4h%>i&OrrGE?m2u%xXFD-{C%)p8Z-Yfnb%!Jh2Sr@|hFAzqAk?$`#~y zgSjBV*{LWo5YrH*@w_LJU~||efVh@+s%!Z$!^mw*hxyd{eidi%+rQ8PpC}pwX?hyV z)$PL1HOr`nIib4eW%j0-#JZ~SLW=sS1n$UqE8zi=s|GO$D`|k?ErV;hUd0d_#-u@C zMd85ZBM}m7o>fqrK2cw2b-5S*v>!Fqp&9g|A^E&gzN$T_;mORVomuTob1Z>6Tw*t# z6M0@aJ?T;0t2=zfWjf&CZV0dJaVj(h8!GweuY$DS=ajQI=f&m z^?ws+K`w4=V3+{ZI1zqf{qQy)6m19>RA-nr8yqs@Z)%L9gjZ4_-HmNGGlAef)puXC zj>ugx@7~;KFmyxG`h6m7xh~T?4oz*w|_ks zhT;gHvf9#^FZuDqn9r2}6~g6^Gc*~4Ws|lwk`yF$+~XDCn)OHX=9JqO*^kvs?EM0H z?2L?Ma2`Bk>MT_hTN=AJ;IzUzp@*~d6m;733|{+=URA19)+ES2!ZipIA%&1@c(OBTR?#QBaLhj9^M$N6fR+O%kZl})M%X@)h5V@+bQGfahH z39`*M!s*^7%?XX$CH9x(q!Kx2_Tbg8641kZmFF$-VgF)MJ%+y_E^*fn?PjBaT){%& zWGZGZ9|E>{2?{88&OHXakWDD>kpFD&|JTF*f9JPVk^{=sSD#QYD9|L51{=s6&5a2p ziRHm)XG)@m2K^T_%b%`L>yoe0WT6j!Nkpuf|be4-o(xAyXwXl?Fx#3 z;-b9{m;$r%HO6{1tTY43HkyWXkp?;C?L1xOd1^~jvtxYwy)UXi$R{|nI#@LFZV8GR zbrKy(GB$(HSpHPj0WRS4?g&4q;unm{*bGR!D26lN2*EW(2flH9(PoiEuc7xJwDdxkY}ON8{cm7!LQIX02ix&(Q0#@Ybl zQCMnVj4pZa_>(Z~($U)FSSl1{*ZSgMb?z>f-P0^9V5($O7SrREHeCM~P;AD*aRR_` z<&;n#b768g#ZCNC<7w9I!ck8;`m!+0EYq9gXV0o3s4xWKOu{EHtCQ;8})0a$bNKc!*|`!pEy=L-o{1Z=Zq$ zw%I^3z(1T0+r1i6aCyf-nlamHI0#maki$NcT*PH3g_A)&_qwGO@q`Pv;8gSdLVzDo zzM~hRC$avBHcwroCZ3&~@8=Bl9E}7SMlrShLVzu!DG6H2oXXTGrJ5!eC}}*49VoHa z*yS0qKTaFA!V~5k*LZ4{QOyiv;}geGbcb7JlhnVUeWz|sKp_>R)pY*EssFnVpMv?d zIj(s#`xEvbC;Go`@P!8O|2IguWCa76LHSCRvx0z70q^8!$bki++)#iz9awLS)#Q4- znM$b=s4(Fm6s&zVQ;H;%U%73`5MU-Q+F&N;aRS*kG7b?otV`r!P4S2cH+(Mxip|kp ztM&xc@SYR{zjo~j2vQY#q;j8#>^*y>JX_Z2*Vr#O^`5#aDwfZQ5Vvmws#hPjYx>WZ zYqT_OT<(^*=Rk$Q`_ust-Og~SRr2Mmv{r0+GLWgb^7U|ojDp+j~aa5Jvad~Ck^K)OaqN!oTN0v>!zpU;iURu9`ff|3>l52&Zo_Hi98t?2dX`i zLw2DZ3=kR}xlNk46S9^@lzXmzCAYt!cNsde&>ri-(sp_5!x8)Cnr`LwfgVvAy7u1Auf@-|@Ru8~aB03#ul5KYukz3(-N(Ld0xjb{Z&;|>fzWLXv{zACv2!X{D5 zm4PgbdmuxtW8whOCGsw`i}_+djDRg;^|MtC#$b|mDm2+<9wM2tw6w4{R7>4dZo4a8 zU9C17{8sf0sUeXtS3A`fNe~r2a~C&#Ia)>=fjo5z{vejVCcZN2uhu6&GR!j-6u1Lj zr-l^qN;|dhU17x+(C3R?cnxakT`BV7`yTuv3Xdz_vk3tXAtsnj&r&I}qEeap7V*y# z(vjW~)!|g1EIR$a=RYd!dw9Mh8pl$0uL3$A1ejM_YfG*DVvO5|0m6 z+0IuyJvS^Ze>V{cPsDLYfMZ*JzNaJB_#3cpQ=(brUYZsthD}Cv>481*+UxjSf5}lod4_de)-QIKQtv>D%3 z|7bjyk-Nb0TR0+#;zDtz;4eSBUbQIz@W~a>jg?p9HWOHQ&|{@6-|toO6l^$&*kwne zbU4DAM-e95ZAs|bDSIMgszLo|B}H1YIDcv+=UR4EW-6ffRJuGDSLgr8LoHJih@`69 zf>R)^5iX~DL!rf8$`;qIzeSo#qdLrNTfxFB%UrP^=}pO8Spcq>t$#oqoqV(c9H+lS z4x6S33!z=@BcPuk9bl{;=AG#THxqFxI%g#-lxmj(mFqp@A(^ClI8sY>CpOlI{3R+q z7X}%0nwK;k%pu`YspdBQwL5rMRjXZ42jkJUFH}f)G&i$x=AU71T%JtsCzZnzoas2* z`|{g7^J70|vy%cwxU~z8^rv$HcK3S(CG%~!cqpDn!N8?dx2bzXrdc@sA$V>3cFTNV z8^VApl=Y`Q)jm;an4Z9si;bNZ-LgCt_Fn) z<@K|M3-4J^>$v$h5U+dPzBwbKBC=7JsiN52*n$b?q&1`=E%R@n)E3@bBu*2z3OFey znUl0s6?^dfM+k{<_gG3kITbxKYEiidv?qInR67}a1|Pn_{yuO3wz)TQAN}wff(L3J z_1*#7JJGw~%`iIJVxT`vf7+c)+b*hS#9Q#QNwrko0DWN0;y$CTy@kf#!Z@V^cD6tw z!Q$%nwrU4TFM=|#3R@G*B^~jAtK#1}xF0ZrDSP9!3rc$$um!1iJ};0BH%|7?^;{Itoc9%fXWLgGD<2*`7>v+`jg}T{cuKz;j`-w`2E}k-WgtbFxS{Z;zGB z@2QSe?6&I|#|dPkJFX}ZmrbN7e&7D~te5QVjMu<}fH?~&iD=%AgsU>VQt@-!Im{U) z2uz+72t8jnYz{bv9akyL!Uk)TjBc@a+c36WJQPQpMxzv{L6j!OIu$T$78PR9|zK>n}d94fmIoBP!%fN^eP6G&c%; z*$LLr(tQTJS`9M&LWD730WEhchYC_vEGRBJq}7$pD=y1r(v^KGE_+mTKtG{aAmY%I zeJ(EZSE@^d2U}3}AXz6rc8dA<*%n-Y&S3M^+vtk{#sSty=MNV%WJTJ_uI~&N6A5w_ z*^}x!jwnK+&kk`gyl0Zh0ztW{ND!jxNR1f$SqcIGWS-i?pgE9&BC?=n3}`gw|KSqG z(XFx~uMNZBL(OI#MUfdI21hZn2g8UFaHM-ipw;{ek@92I%!t(>1yEKGt_0|=-y#F#V2;qHe~-CT=t^1&1v7h@$rN0?7pYCs8A8jx8(z{{%yGZ0P;xLop?TDI7g9rTrflnp_0- z-2LmKXTh8fSGi3HZWjdxN93NveW-&xzPq4Ud=kP7<_?y(yy?U)HUMhh*SBzHi5=di zTNzV@pzqP*i%y7kzz5E_hJiKhv!OA6U+#fEm=#@~UDT;CetykH&&ZOU{2PRTN+Jfp z7<^!fw$J0#%7P^bLPL#0aDxOD3YJ0h&k9uAvxnnXi1HshP8#50Xbe%h*korxHnG}3 zndU)~$g!Ps8bYVomKrKgt}tL+m*uUG;C;uRfCJT)LKRU8J>a;o$OP?IYzYLn9HRfC zYo$|1NIhqVIkUifScpm-Lx%r;qHPP%it)Cs6u^XILeEqR*-3+eqWrc^G3fLPb6j&v zM{A6;O)amBBdN-RZ-wB+E0ff)fna0}_a{4wwjnF$!J`Qi+JI5bp?w-`m~Hu0&oUFE zFbUSm`jUUfHU{N4#uEU{`g|rSa}3GPE`1pkNC&@@uY<2Av2o+*z5i=sK!Cqp9%2uLdd5G*zbdz zmq|g@gb}*bE0lVoa5~zg=uJ@lK z>7QxXfV|XCtNQnfI_a*>7^@IKe_*6T3j*{Xt^K!7JNXj)R-gkm-_wf{@&-|({To#t zcEK-JrJqpL=hSA`+E`DF=;?B3NOhx7=R+pi`#}<|Nc0LV#<7gQOqWn!Vj(eLfZiL! zRzTi8cme;6B4zJL?4O*9mHVuPiaw$S7h!a`;ODu(IR^YAc5Nt;-2HO^``3Pq9G=04 zS$(T~s&~sL8vJnUz$~S+H&T7?i~&A%O>^T**Rcr8>x2Ng^Q7sVj!vK39u0c__tp5uLq3* z*rm(t29B3Lc(VYty*sTj(@+#EGg7uAiH*wwI&ym)H&9v$BDce_jl)6oQ_-C{PDBSa_Q#HL3F>h|JWNS57Lb;b zA>~sRR9SF6XvY5nl%qpgppggsd(iQ2NoohW3vq8Ep1hIm$HnHue-AnKAQyY%{JDj> z9DXxF5rURVa%6fTAsCI^h4jbii%J}rdZF=0Bo1}jJ#dC92=cm>>4;Jqgnr@h50lzM z6Qt*|1(%+eqCTK`XSJvU)-@p4GBZiAQ65vipnntZ)|4RyoGVFC<%lgj&Yji>+h3{D zl+sfgMqut%ET%rcVz#{ZxMzJKS@mqgl-)UcEe4W?woCw7)c{C%XA6+BSsLDe4eu&)hO%VWsvOA$cAwG@|?CMnn*42=^-Cehp z=?}+^wPzix4Xa~3zV*31Y$`k}7SH0L?CPb)VVfW#m=`|>+}e3j#xOuA0QzA6j#wS? z55xN>olPn4W53ksu1Uw5&T+WZMn>jbC)>?*;;R_oh0@&6lTD}cnD7)YLN9lfkS{1J zTsGuZNZtR~MGKGiFF@f~X|EZ+B~8yG6}MB6GgB)|30AeE+-@Rv>zCD}@d@+eT_`9h zX5kJ*HtF$FUk_3^b=dDUWot_i_y_4$qCc`Y1Z7T;XBQ^)+yTh20kJHH#iNzf1YV3W z2ES_nBn^K>*UVsvF1O`yA`c8&Z;#r;VAV^y9eSXx8VEKQ`5;+q^0iA^lcgEMCXdiYzE$5PTh|>+%%~~hQgzEkI ztYUo;$@EK0`|)3tJpVHiqKqg94gm)O0|Nu{uVm|L2DLgk#6J<=_DN@SC@`3R{vDce zga;KKFhEDz`v~_MJ0k4bzJLrYkB-F5kXR;UVcK&D#tZ`z)k)q2-wg3$8HAEHq76L8*PQ6Y_c3y;hRQy?|2#QIS2xlE)6h(gfVYm&Zo%W-*G$f0*qw3yi~#q>+4-CLTuR)q_EM`SBSRv#yGf7j8S}KkR=|n>5aD{yR z=a+-GY_mYYOKrx!=d%kk_?|QgS7PaG9P*W75Brff-hRMnqP!SG@V$ebxWx-9HH%nl z-vB)ya499z3vHsa7l_TMO;mU#22n!0< zYkg+myYo9>hmq03(u;9&kuG^JQp#z^?_uWdcxL)RZBFnVC#Yq#n54LaCrHoP4utT9 z1+Y!EM-IA~0GK41*&IAn%>STCyiRfE{@fmB|IPM_lS(dsZN{b%{dN}=22D9F?M)M9 z@$Gzk_}wpVl3bn$(Ug|x;A%XJfA$cTirDHNma4$qLy&TB36wv24x)*u0I?8FiS;!% zZ0^m)0mXzLYe3VVR65nj{CnRBe*g3#Jz5{0ID3MuH$Zo2^{W-ks9}WrRsnYB{$N8S zcMh(0+$-~{TfT4n@J&8&;4o4?aO7}YK5*!;LOzgc?*uK7bPovTI?eLayt}?s)mGN{ zD2i>H%cQExz}7X#6tPBTM*335HiM4D6Xu1>Cc@GI%Nohc)-tt7cST&yUz-%ASB8+& zqqoX~1Ykj#-hp}4;AvG=7T7K<(aj^x63fdewv|PMtDG9}BS?{bInPpm!HZrdzJF^H z&Q6;=94{t%GTezfo|JKcjHJ^KVm24(3pN;L>yfCNZId*wHuRJSq=!x~Z!J4hF!({^ z0P|=IE7r6qv*_UW@uytWhN(5QleW2YbZ;J$9I)_Hz_}BzU6+g9Ub23mlSnWRd`Z;^ zM%2o*>%Azo3I6A_$J5AZZMTw87Gyv`TOYh1BV}Nayyt71{rSeJ$=W1?%kFt}3cRWg zcCMEyHU?bH4t?WpVR6Nk^1ZWiCu8|jxV3E9Cp!b6Fa$EPFivM(HiUhxr4_hb z)~)4`Ex5i<&k=)C#v1y$I-fB3{(KpHrvwBGCbB=R^C%(C(V%J+rsSo!$)AMb4f~x? zPc)c&LqcwyuRYkb1T-1QA>0&|$a&IEf`ieI#W2-nPyMnmc%P2+u)>(p4SgGyxUD_h7X-nkud(F|{5@B$bS z_0fX&pr1XOGPz{Z^=t8eXpurWXQLKE(W~X-CDO8zlN^m=Y;D6ZC@9CkAl5v9)(7QN z=qiR~l#m)dtKy#y7${!5bzIdL%{Mxr=rN6#lJJIpsFTTc#tv2sMe_rN&cDHAu2mft zAhzqZ3iwjgPN$PysQLMxk+^*Kjt1}%ly6*>r#?8>jQ0yU3TIuJXreTO4y4>$WeW_= zjJuNeU7&VekCtl;+xhWA_Op4^q6I$9RjzVE>He1OTOPLNT= z%9eB^Izm1&U>JF=Y=Wl?Q!Sc8S)v-b;+YWXi#A%=XnMnjjpEoOqZ-asT7QAXd&BEci z7ZWTD@shMOF%ywv^2wz7QuFbkCn-|`rd4DwK^ebmXrc91k81FbDOG+kPqv+Y^yU9* zb6Ci?hW->H>Ec_@;#*Q)GF2)Q*FNeO)N| z&T7U?gt^sQWrR=bBbO>m-Gq(eC@= z)bo9C-;tNYgiYEXO<0?FeK64Zytt_G8URs-Z zDism$P`OGGhQfwjDD+Lq?;Vqx^oNBOKdU{K_me|FQ@WT-{Of88Td*oRr|fYp=mYBO zLtABib_9Mh*4MlzFSA#!qfd76ThFzpNILM$-dVNYk_Vf-F;t}`ML=4|!Y#C+>(;`o zA91aNzR3Z+68%Szq=xl2M#EqDqHDTg`p#_SC>ufIMGw+MTLl~EdKOZo*R0hw3|e8I z+5U2I6G(g?K-YG~u}d_u`DA6dC=gM2y0@vVG-QiUU()6bT^`pt$CI+{Wc;3uB1}!c zkhiMjCT_N}>V_k5d{TStSvdJ%HJ!qUB3`Y7=e^4Wqyu z@?2dE?VHa1u`c>f;Zhc}Yh-{?uuj}Sj)b|Nh`uV0`o6mRy^XuFD!joEM&e&NbO6E{3)n3jJ?%KTb{f?gN{SF;=PN=Y` zjzSJVd@5uIi|L)+5sM8!e`LrE=C{`B9Eqzx93tYT(yQ9L4=GA9!n`ukR|VfWOSC7Z1KO#tAHeX=Lt>QNSPq6Mvx5*rcLNM`Yo3890B+ z!Q~OpEL9^O!)Q1A3-;OxoO%~h_L#(4^(n*{_~V-~l3`hFJI>9ZpF!r#wa+V=-N{dY z2J{|jHL{eO=oHW;WR7Viq|uQCnonE7(VCGY{m0GVCfy(D1TuJn%+b2{KdxChTQt8V zXq35*zbwGB*@wU_u-nt2CwoxNU8u*~vRPBnR0VOpnWH4FHh-S|5;q~57f1ul3aTHg1YzAz~C2L8sK(35Be7g zs*f?o5hlQ>ULk{&G}xI!n1^!>1U`y}f-DEVScYrH5e==*y&0EFjEmjGlZ#YWujaWh zV|`_NeRxIGp=mS<-W1i>TCb+JHS&3$X#LIc$|1nXl5OoWP>!nmJ-PjX@aA{mO(yUr zXWU5eFARvirQ4P=@^9=x2!3S(r5}{Of12Tm$_x@0cIla3=p5eh5Q(VoJkXTpuc;?5x%F>n$h4t$ zT!-8DYT^41LMk>J(elghykv4o?|Mb8-G_hgL6<(rlX9RBv`_>}?@Lhx%I^cHDe}qh z_@fI_UT~ueQePl(0l_1&6oE4PS`>i_``i?Pa{FEsflB+x6d%(2N=z>;@E`jj{cO%p zu|-8uc$NdYIJG#*X`ni=+_5TKiF0|2!I>D#C4(XgF-XUC3J7Nk9JVY#+nR3NL(WQYw9phyo#b`1;lC+Z7;og? z)K;n^Mx@&-Cpp@v9v($_ei*CqvP1a`FG^C*uq_tSy&XC;`dVC>HwJ1_;C2k7`PNTW zhm_0JRO|hYEh*+MpF(xY`4)m71LmrY`ji=g;6>fATBh4ls4&etm!Jz!BPXLXY3K*^ zV=UTvq0~A62eA~*D+YgRR)iQ^s>=aJ<$ECd}toXAebKtZ(!1=bN18E##c6 z44nDKTeA3B_o`J)VT#pAJKV=mO7Uv0&60%ZBuynnT)@)oGVd&gF(9*OGfMJQ8R%Z2 zpl(M8WGCy2Z6Y$J6zl;Ic6Z!_2r^W9_R!kDPj}tFGin4ZC$2Jay_NyjSwo}U>wM@aEKC0j3$JjFO)`8 zep?{5c~f`LRu@%`q#~^iQ#xH=sm{oVlg5fvCy1nZ5%Iyn?IHsdClnrNTm3C&W1&Jm z4Mo5zx`J|}c2G0(%^`?Z-q1nWR|J-b=6E+Ef-^^y(cE9BiQC6o4Cl&tm2iX%iVlC% zUc^;%hq%ZnVe4ZPlu+?lu;qY9hjW#UG>t#@<o_d7+lw-2-qt?%!%~TF6+AZXW zLvK>Mrju57Yji3W-B2+#Ic_;q2P(y+euhk4g_DEg%&-L!B2I+uUDNTAfaczthY?z; z$NTk15^xEC!5efl$xwro#H{3&;$C=;RLROn!8zr=pp#MmXK)*``kw(GX8od;N{6#6Q07Bvv@x+Ux zbV0lb8G;wOpJS{QpQ>svLX25YvCu4OUTAQVMaKXJTw$0ILcs_?C{QU~7<(jxJTs;W zEwQ|;f{&W~vFVLj3fw?V8`%s%E(LJ-jzzg@QD9rhu-)M8k(T|zWDkO(FXKj&auEW zUtb=8IR*L+j^8aB)yzU=w12D-DB2~H#- zZ3=%W7hG+dqnz0 zutjbxb}59F(zlasI{Kzi%X{a~>MXClfMU8}GiPPB@zb1`*HHv94ucg~o5`X5ZUUUt!9G+ZCn6KQ#MG|4bZMhWvCVMzCWtV~h*}5MMO7em z^H9e8-$RKjRZQeYI76PqKodY$eEF$K=RBb9{oJSvN#=97LFzSMV(%&HJW0-r@O_i^ zh;jsG9{1G>_Vf&8TwI!tCON;w60Y4JQX;TJ@?KWyU`Mho6|l{9xv>Z#!S4f5ymt4dXs zsV4tWBYo&XY&U|BwZ{uUc}w@*Cmk<7$9wl~^cjc?J8?(BJ=cwhkl1N!c4!=v9rr{a zubyf`3XQ%K_Kces&#;+?btI~EBy;;nQP>^-N^X5*4K^jZAxf(6)HHH{Ra{=UTw1&p zXS%>)ADtE`ns)>Ccf zxAVD}BVS)~RPPEz>nfIZ`^gyv^B;YhwsK=W5o`r=pgs+59m(;$eRhisQevJ7iUkUw zu6l0;!4YrB`C9>uRo2WT>#;WTf>C)vGY<=x`8pqQE8D&yNJ z@}D)W*1Z?@ZH$iP@o#Kbk}YNNKY{w!!hEb7QSSJ}HXE18w^UjCsALXa$Eb#qK6jNK zgzNl+rdfY(P&@ib&VR}IO^Tu*tqWDwQ`0{6H z2y6qtemNQft_KAw0i0@5%4?LXe_WATZitP|d_8BnA*)UkMF_;UMWh`9cD}H{Ir5XWN-i zR0eAYsaMH(0kvKM?s2SWVr`a8$r4m1t`F@n2x`&p&-)? zb!n$gwvz9uPT%ORo}_R*8v;+re)nGnz6Uc>Fz!lH))EQ>w)t4sD#ksATGxx?Ic z(zb;r&jA?##1G1=h=-H8sYt++GBrRhTdVA~EiOQ+yoV~aKS#pkBZH#>uIC2#y(_W} z+#t^1X#OQw{%;Y75ZlXm^C9)nOM&c-)#AH;Hh1Z3_+Wq9Ndv;icYc7`ZNJZ{?uS<> z;G&3znDa%5|L06Uy85S@@CQ)-tvB*B=TpiT!14k}bo@w-{G0}$+HXJ3zr6~Iyfh45 z_2a&0@P9IqpOpIOWqX;Q`P^!u`oxI2wl*9RkV(6 z3LOVPU4@EJB~`#S<{qzFThM4<6Zpfex`^pv-i33fq`U?#OHi!!xucdTo$v>O#7FRjxQNql0R0J>g7jk*b0G4jEU}FgY z3C4qzXrFPo{J>@muV0}d^T6Wtv$2a&rkz9sI7pTZP7PoHC)S zVqm4L^QiZuUF0v@nePD$!n}WnGGnp{7qmxK@#KtCXM(Dt<7y2VFgziqJ$uPsgV-UQ*E*F&KtF&C7pfY;A&@rTz?BrnO0i>j(_gS zn^)t8P!4ZZzZwiRN?aytXoGV)IG_SJs!goK-}ClxG6hT;b>u0BO=V-Dpj{xILG&#r z`d6D>#SB1?&ZuiTUYY5$SQ*yFu1FwJ3Qw+E2_eE#;C6+%_uJ$$?WM zbQTyrnX{L>QfV1aBa^QQ7l=Xuj!lAS*9hW&mxqOiPMeR}FpW@hJDxHdJiJ01rrP?K zX>S#e49+8l6XTry{C*<)Z3@^|e!?Ifw!MLgYyjGr;Uh}+Tg#Cgm65DzTCA7}ik?Gm zMW>*Rz}vgXHwkU^*0;73XcoDhB8e(dU|uu@zyrs?mF#A~r{C)Wwpa53KkJ8+;Dj9_ zdPyG<_*gL!y4cO(w`^niJnXNkwpPz?7H2n=;j_wL);&F>%n|>&fza%al#DpiV4y?qySg=mfDD~LE`kvKt zx^DBcrAOM0TDZ5U4XADa;Z)>3Z@)cW^+ZOo-~)14ZR*fC#vmw(o<7<95v3VdcpH&v zvJ;>fQuE+=p{=z0EPuHIx{)u@9)6wP1dU{_^um`Tf>hu}elreML|uid7sfJ~F^Z@Y z9-+zOdfl+)t^2)zcPAdF)e9e&s?3Pz3ia5;X#db)|0^fPu$!+0K*%MqhW9yEA0$?Y z*n&C)Ye-;joC`Ja_p51>&r{@gH6oMW^D7)@5_drduG)t+gy)G02n}!fTi-|I>z1NOQv{RLs8Fs6(BFbCIW{yAXmMyf4w1)dz_yPxS zQH#X>01Xt$31RXgA}b$RDXH$rJgHZ#F7v^TtvhGeWY|sHj@=nrenL3x~5LJ%6WtVCm zRZ*)xU23hQBlqNlUb(OrDkh?5Qu$o|%(5K21-y3s4>FipKGLlc;K z<6nZkF}^kI!zJlLd&fAD+@T7O>@U@-f(_ZjV;YqfnZ*KaxfWDxj3$_L>$D9!XM{Ka zhnk`r$t;H&6zPvqj$kpEe_(kSQQL+z!EspCC}2UXk7c<9rcMc+Rjtkc&M~#(sa#&= zQte%~h0r8D`iR?Vr)!P+_l~9!<^GqbrSTYl_Dozj< zk}HC#BU?uHQ3iz1nxl+i%7LW!gSXG3Sb4%;oEY>)wqFAj|$T}lvL4gMRjx?T8 z4O=owqAkew31NdRsW30U=_#y#u9(F-6Dp02`g05vw1b`S`3H3tjj2Q2(L>Vrx2WLJ zIgBkBEFN-cIxLOUZ} z%3;9(-fc@&L6QnPt5BPH`8Xs1wWWZlu%E0)Sw=2 zU69MXto2BzvM#n|RqRZmbZJAI>h2Ey`JU}LbZ0&>x_;eBK0rw3k(zxlL|Qr_LYOPu z`;)(5LWlQMPlWO6yMuIdCECDR!CCqj7n~6ZSyi;}Zb~mJvwMi2(c2^KfcVB;m~R*l zZsV&Cd{Lp-SKI5if2Rh>TA7OI1zf+quuqkU;MqUwHOtDgS6xd+QrDUO?P(>_k2X>W-FAJU zSF@da?xU_dMNyxsK2ufCV3V*m zoAt-%2y0MID>!{*^6uip>WIkhOGC&H!>L5s*m1VD@<5%cm8=4ksHf_ z%2?s*e(M0g@8|)1q-Nv|nt?yZ{oV(2sJpI1$2)zQE^$HLNvwq@q6ZidV2vzAw)%l5 zmebMjofE4>4emP%1I|C3*$wyNjl{@3j%U8sCK)N!$Lf5Z)`kMjy(X+bNAUah7TUyx zTRu~b=z)dFgSOI_YkqvxNX}G|0pYq_on6f~Nppv>gW-T0mDmL|X_lKEX?rH{qFTxJ zp+SE4I7TeN;A?lSlru;~YX8%q3QqewlfcGAqpwVH6Xwz(nsSuu3^*QLiZ0l(@onyh z>dmF$&@mMdK&z0-9Lg-|bq zPV2t@Uk89KsleC5J^f%gIsK{{bHlr$`SoCmy%Q;vSj#Pv&ZoM^_stJG+CgL;BUn^N z$zJC)SQ@aC!vLrkOpe}H5jw1Y2)^|vHe6<~^Yin9lncK;r#f1CXND&4qwU|zxEKNXgbIjZlX|7xWlx|%&dNqm-ysX4 ze!k0Sea+Mi8BPJPY`*h`+x%v)8yjb!^XUR)g&XNal?ThTX43h~(zyFFp)QP^y?J(z z%qX|MNXTVh14O1PN4xXe1>3Vg5v8fiM=t@wyVN{B z?-2MN42MQP)e4}z;s6Uu&|O&yyZQe6nhAs(C7hqOha5kvAYB+R)6}x&>I=4*tCDO} z4|>Cm6)+D1$Xv)9KeM6t7I+6tMYP2360^fveH>6(mn<1?s|#uzjg1esLq4%rLS8XX z->w5=@Okfvd>kB^O)HpJ6wQ6N-}eCS>f1Bll$~der(R}3r}g)li;~-+?Y8NiHvISB zC%!WDoFZ4}C4eFTq#sH4E<%3x6bxhFo!&19A;UQ?s6!`y@6#V~Pa#@g(eCE^%#iJC zs%I+2xF5IzH?HXk;;MCiDfdf@+-wK1;X+6K;J_7(>mUT5v z5i>OOv~-sRQhciEaQ_5BqrQG@rk!(e?VjMnI-A{zlv!32Gst1KZCd()LcYr_v6{B9@^WEJIhZ*qP>y~=DouM{ z_=9PFG{Y^2u2XOLWQ4V;BEL_9N_gx<{0(ngG>+VI893|<6yX!n_O0ZZ z;uHP-1Ku}i;l4X0Y0SE{+svg4dPl53y}xeU*j?O)+GgKYh!&J zwekvcc|rh?a7+N;Q0W^zbkn^> zov3+X2Y_)M`_>)%lKj2Ux2{k)cTya?rt3TP96R*V>4#_96n}@ua1V>)8yEd*SU_dy zVoTeruIm1`vQz4Aiy#WHCFB_@G_=+t0S+^qoEvRwmZZC!WNp@*hlX^R$_2o5@8;h+ zZjje6@b_^I4LF33`m;mSZ=?9|&yt9(e^0f7r6xbmsYOL0Btcf|HIiR)?${uxU=YaY zAx8FzJ7h#N7@Yi!ts{DSF5_D{dHIIEwi9nejsxf!PyP3OatxrynKP!eKC09SBBe

dHU);YYJDym-FPecH$r|BE0Ej=_)vVxR_}Uu z58*vLFKrnbA+IdBithXUA|k(FNg%tvm*pWdO(sjMx)D>IN96fi&@=JMY250*31Jzf z@Op#`RnXX+!fKe73%7@D8#hM3QyhMXNH5pRL;;(FU{loRQ1(6q-;vuhi|(im#cUu_x@8 zs`-J1X<+Wh!oLr=>h(357_(PwzNG{1+%9S}l>{sj*hN!zj9}PsiTG!*QkgIOx zKaR&1mW_aI(D7V|6tdbfUV9*Gn~HP}MR12;GTZ=H)SAC?&5<=e<|*pA>yjy3a1Mkb0KEALlqFGSRh zXBNPt#YC%yC0q8#@7wrZ6(%C=rqa9udSeEirZr$guFT7SU<-Oa)Cw(9-$^si;lDNcTOR?^M4PSc+k z5n#IR7{>cE1TfkCj)<4mU5z(i_q%kw(L|OpGIYT1qoFpdK_p%&T=*8omsaFp!?3<%dOZ*sQXjE6M94T5 zyef0Pcp^hqf0On1-T>L`TO9S3{w@;q$lWs5SUX=e5JH2iU4x4xs|G)_&Yd-%rg?3j8?*2^#?0bGgpRsuT@eED5xKaIk}V$(&4y53R#cU^m8KBy zu%yA7B}PT1ToIk8IX0;H`2H|+g(PgHFqmwPXxZ`xgrmOlfCjgy<2OL-2qW6-q-dSFji%~=u7xHAR4OGhGp~{% zWp`~2h~Fr$+l2Va$~zdopit{lu0Rlv@6Zth)>ETMn3Uz`P^Uf_5Lbz#X)-+3R^enL2JCh zUeh$EDmrC~hq+7mLq-(>%MOu6=VR=*WUyeA0-sCStUJ2bWavz#rXS_Zm2|0A7e-5L z* zfvH&~f|pwiXbEe`!}0A2Jm}&B$(o^rEL$a+`9~|{M?T?tEo}sn?J-L_P8UfPpBK1o zSP?v;Bcx%x(DW|L?#MaU76G9@ps;nWFVpDVEJ}a&;q}>Z2nu2~HaWKuw)B3^Lwh>@z>}h7Y-%+=3ydmItBe8!+d%6~g1iLQ&91PYR+u{Vd6!TP4gJlmNYD7Q@X8j#D64e9#0FPYwg7bhsRV!Rv?M0YLWgBbfgN z|NBMa_|@nFWlZ93vsaTjl(sPY=-D~F#|~i-M19v~hFgTV$#0}B@LZQ;*m?L8XOng# zx&WY#{?*$YlANn9w{8_hC1A&_mzsL=T*fuZA=e~_C(P_{e*Jvkva4}-?u%QX-YESi zA~m0{r0n#*j7nsuKb`U69_WpV3Le}>8$VuC1l;NoT0#gF|A!+AwI(67&c%Lr@8mrM z!vj(#(v~L+(?I4B=!D3Z;xm|T-H`RUm1}G<8&QDa z64v8EX>DM0E1Wff&AnElZ^z6$q+R3PHCzKaW4|Onpe7b z$4e?#_L#_86qYMwmhh&-n9*VR?PMJ?B55Z8u*E7qrq;-K7 zrtDKly?K|iZsOUCYh9E=^Y75~LNTIFs~x!~=RMwk72}Qj`%0b`Me+C5teQ)HCGY+6 znE2gu&Tq#^z@Z^8>cXP-^FzTa^$YgKUOiGrkKs|uPklQvq2fwowhEGk<(v9{fQA39 zO#h!XYexcKMJdc5@F{az0tNj4^fsaW-!Q)!`r;|?E|}Ch)~24!C+8f=|S=S zP?=y%@jwAc#t2A?MicyrDbx0y5z*DJ1r#7^K;aD7rjg6jJ6hSTZS! znsl?LJf+f)WK@W77ugw>JVKMAblC!;%&bbYa}E!h4qt3zW)oiVi0$lI@}`a_nnm}O zF%Mlb*zQfudPiY(*+kUM0qL~IUG3SJFr#8DHArwNIE=sN@fc(ZS@i$?E6Il}sx;Lf zono*3ZPX|%Xl-g%5^c@i+wW)Q;7}KJ-maes5rfOFe^@x8dvvh8U_dpELs^&NmlH+v zN+RUgu;m#MwR(?*Z71o!RMEVzr3{K%k;P%XWqQfGNUPyh%nW9%0$_>xXO7(hg{Y6M zM>B11<+IQn`R8*1jLkkNWmXC`nSxAClZaH_Ug3zoTTM^Y^t&ml?bZy(0h60i;b`_A z9p@3|TZ7_W&8LOU0I7Oa(xoPQqu|!9u$@ubr6;QOKCQSzsFY*!go1MS;y@?cr>gvL zsxX2P_CZ7UL#=mu1kiTBn*HEkeu(hXBHmYUD^JlgdCd_R<#VMt4#iCV1~NS@9$!(u zflGCVyLZ!CD7kh{d-paZK6m|uJ^@_U*HJ#HU9JvPB1>I0m8P*$?NB}PBM=Imb^xrxBAa&s8ZXR@i?kNg4mVZJ8hq|=FjUJwe#`U?Up;egCc>8%4Y^@ zUKnNm!qOSO-qvnjJ7=YdR#chW#6W;L%p^_+7lrlK5ke7{hXvD|#UWay_{)(wxZ`g? z*6F-ol3wlD1^}L~RN>*G%R0mr*~{;uu3pnKwWl_BYn#PfYb3*31K|stZBgtZ!8th0 z)wMhv$oIybCF9#iT4;n?PElXNzp!$29~ol-4~SZSK?Wfju!<&z{FFj?L)cHIS=&WqDsP*TL$n#iKTD$!VF@^k>v=*;j)2kGk%%O#w^JxWb?itsc5LD~&O&}T zN+DiF~SQ>b6$B5YtUe}F9)p2 zKj3+|h%$P^*O&yV)j0D9tXp4|n#CE}S?+h5tm?q<$h_(1h~R*`_#~piU$J!c~DH#S^Ybi$Q;35d<9EGkq;tQ{2db%$2cfx8H``=r8kgN!y zBKA3i>C!`W_>x+s`YY|B>j6-rNnpz_B*?H;>F!rF8z00I%U^?P^GH^+(<0-pA^%eG z`VnJQg2bCRf>+mX=puU}q#}2h?HTd3o&i|=&#hjdJNpBvGIBMT>*xr<+r2PaLY=_J zI5)=U%)C(d`O0#b;=-GkkX#<&CB zVq7)lyc8~L+E?*o_v*xLNnyK~P}qn3gypwVOcy3zqn(ROT{{?X;dQG4RuO1r9{^R_ z0ZQ=?;%5raq_+ZxX7wqBu=EO%e54_^4Pqfi6=6k5(auPq5?v+Uu|xF)YwMpkjGCL& zp&kvSo9cp`Q<*+1O{bNJqJeUuruv3u0{w7Gq4M9weJ2kfJ>`oc-7?1xxAr-`;_gg` zIlN`sX9z7su^`Di%cohclmx{pB!J&%vK?3YIag)ZCl5rv8v73CK7>C#7cPkw+GTu8 z6^q+o3YovHt(UDZeP#CLbFM9PpWWUe3SzjE1jk9*=*kggfZ z&zt_zj2Eq62FV1+w4;W^Ouo7d2H*{M;?a&wuY;j`7*Et)&?t++A)a1dl{1C}OjNrptd3E|i zRvEDJ$eK~ODdPT4e?EJ_5CBx%tN4Ju+GfxE*^t;TbNhl82nyX9mEn@CI2<^Vq(#Fl zTV;Ji7(lzlp^NtLC73WztoCUmlQ_0`EnDD)l$n@dZ*8s6EhQ`CETbFaPEq;AXkqT; zlshA^e=PRNc=A!Wwma+DHrh9L3i?$t%6R(0@*R2lN5z>kXXn`9`~kq|Cw9CW4(2yi zXH1j(>5nz=9+;Qz<|p^tV1L!kPh@2W&ycU7yn3fG)n6&?R9IO+K*W~IS@S9DMK635j0)d>YbYDcpkg?*X#GK&+` z*mu{(z4_uBBQW=R{Q*GK1H^4F`FE{_~KEf zU^@i~^+LyON3v&avzK9TBnIMbm?1D_^sy{yLPMq>=st`IvHO~^@f;?8TEDo!8D~$! znSNFEj|x~%4HbZ|_KPS-b(I0;eX@j1@=r~r^16oKl_7uV1n%@ZPGMdwN^%slnYt|)JN zJINgFoy>TRHpTED%fY)66%ivjo)IT(#nT(9&iOKRxGUgpLSZjbj^Evd>Z^lu-km9a z0RwnRt3xe_3v0m~rKcdpE>u9?iWVf^!>bzQ48Z#v20Z+qfz>Z}EY`Zmj-f1&ffX1~ z4UZ)b80t=q=8#OfRl+15RkPF+dsLtUS&?+voqY219tgwVfJR7ro*`ftDeJ)jNSTmp zVus{d9%8_?8IgN%8K5Br#oYs!rc=G*HdQJeYrU;$5MMmR#^Lcz%y??)sg<~JTrt+Y z@^HGss7hJfXW;vOJ8*IRAA*+HgQcp69J1xk$vY<(^D~$M_9{%E=~4+5GCdxN=q**= zo&&}n@n1G#b168iQ~2dJ(Lp22Qj&Ub5`1uu-2(u8>tv-w4O;GshWA$M2>!#fO6lu| zupV)qQFF$Fqo)uSIBhU``52Crof`iI~p3pZJihW3u;R8l^q^$VE{D6TjBh zHXPs=tBDP*1K$1GQaI@#r*~$mH?lCQO{IRh?3pUQQEf*bPNu3AV4^a|RM-)WlKFNH zFU(<0?J##UOoLSMk7O;~S6due#yasZDfTUQJNO-AGz`+s#7h6sinuj*VXet!-Y|M7 zw0ri3viJh^l}V*?4^b>o+><9!>sh>R78j5@+ck8W>1@fqEh}1Nm8}Yg6_XShpKd?2 zCf>IH%-#vQuUavp*0*@ZQ~j0YYH{(QmtM4`aZYuxj6BCo~H5r&jJC!sW#vbjTyuZl{%S21@>$jv*I#Ge--B`j0YOJ?Z}V!6lH zW5a<|k*jCcOJ0$(dVM-M^KhBwm)J+4Oe1rOfNd+OqjvhO1o2}Eq?S4>Y{;0gpZp^$ z+>|$ZSyo-;Mb(ugo<=4hj=EAT{|6w;FUS?<yp3pYfKc>Flw32W~f`RIb zM{&}_D^b5dV;&&-U9#^=L0KOqk*}~XJ5$PfYV8$ADTE!u8#O!Ng`MUC{suVIoq9Z> zyhU55aBCIWL!=x?q*c{92%ItZH5;+I4A#J}Q zk9!JZJ!l89Etw^_U6Nmij%FX)@N%Mb7qsw`)%D*%>XA2kgKb&#_Wy#k1AW{S`bPiz z*O{U-67>Z3q7eV@0S6O-J$NrXh?I8#XS1o3+G(ec9#AW z&{`13DBw0c#}-Zl#CGw{6vkj&J`jYu66AcDGKRBKQ$LPo!8wjUo|8^e`O!(y_zqcdXCw+Sw!lRwp_9nlrI7c$>RhV{G%TypL=TgTI4GXeK`fzJaZ^(sVL+{TP# z0&HL+C@*=CO+u4B2o0bk{@<#6w(Uq!UePD`=R!0N6G<00FZDO_q__5~>sKbI>YSKo z&OzdceI|q6ZcGIUAearlQC>oDN}TDu%Pp})s$*HG^+EK3aehifSHn`D19TPLO{u_B z&xiwEhU2Ca(sA3}Y5Uo6dk0njINxBtuIBs~5F}i9Vv!IOrTsWsB#C{5LlcxrFfmcP3)vAUgz$d1AWXVl#H;=zF+3HP{r%Et2`vJ z@icIl3nC`3@bT|O&wEFAh^v{=ZZIWy6^q{kLNqNy*;CeNmg0NJ(jjC=mlvSdMJ{*| zvWP-hVaNdlWv77l_T=0`2<}458IFkX`7J0y4|ujXiDXZF;pV*uPLFY$eoCKqqE-_& z!lM_8Px%O?8I*47NzE+s(LiGCCalI6*7ZyQ#p>CBt-OHd=@x1u>9m(8^ti#wuANi=RjO^DG{VOKX=<|d?#zp)$NW%9C z=P3T&{HE~-Z*}-EN+THz6$eHoN($@;wR|yjgAVEm{ua8N@D~vMH*+Whj<$H5j9zcj-FR`It?@L9O%bUl@CRUXXny zzaTBbu?2C!sY(VmM{MA&iJ}xEI`as0B_~C>aSFi9i7l|5!uUf!I$|9I6(uVhvLe(9 zi1H$Qf7r~ZWrD*-W{vP)c4#l~QKxeOSa0wq&$tT0GqJYVzKTVr%^HGaBx3|prnhU!T_BYfY47Smh{;uD7=?+$eMJ*ie18t=bTk6}tj z7&*Y+JX?~A6)}NpfgAOSRKU`A>5DvgC}Ogfe164)yJ_YfK+mS0Geb(H+6C5Wfj3Q% zHi3W3D8qq!J+_Egf{`Qvbe5nAs&_Wt?oNVR2FnZxL#Sl5$+%6YdN|6>Hd|AL-uR!> z7j39Ztl6{|ngwDFZa6&;LKZ(%7UymgMnKTbF+f(99Zzr z`Q;^Fr~%{X`it2=j+Ulp^5XflWVcF>-kk>64tAD!6Pn;=q=GDcVyCG@9Vu&q~UZcX-nrv2cflz964FDz8N%cNK z*shxZyGv(X6u)tJ^DhD+_0X3AHK%gQ00G0sMech5Fxg zzXGR+$w^t`!3at~TCKe9l(>V^hz=KRC8$RlAc_>I&rxx@kcxuYtaAKc&^p?&T|wH_ z1|3HIHnPt|oad~|*zTScah-S3m2b#N;mYQZ>9j1$dD}A**CmR$1g*AUjvb<4-LV@S z(~OjCLL5x%Bu3O?#qfbT3giIfSnTyMp%Gk_({o|mJ6|F+Jd(tonnTOrw55cZ4te{M z?$Nt)Dy3yS<|$;R6s{_w?2rY#rgu8Ft~EhC-UhoCVqTP6(_~&Oqr*m`)$|7kP~^A2 zGspAFnqFbF1O31aO0dQUUYiE0c?(f8tm)J#25(|XYqP3NlT4@^o&EsER01b^hF-?n zA54->e!booOc3tExTlTz#QA=`Tx>to1m12ptF6^14tk$>vFAqLGB}0XIU8@rcHG5s zE#gC-vZQDwTV`w^Yv)!3Y>ruZN3fC_3F!6a@dK7)X_M@CrR1sU>*_JaMi}*}qi$g} z+$#mCtRy-?J;PM9XKVnM@tY6{oAgw=SCJDn5TRl|@K!$bB%cVA`?yHk4u#IW85s8( zdy4K05WQJ&J0i%vc?|p4i*DU%_6NSMumZyPA5_~Tdaj6$TVr9y@;^m!M>t4v)`>V} z#jOIokhHFU04x~L##o;rRt}02i(GazG`Yr^KDuC@PL%k{=PT;RNpEZr1iDZ@;@*Grks9NCELf$wX=ivWgC+! zPYKIpt-*_N$T3f4&2$;b+Pb6?VOA2qhNPc2;Mzco3GqD1upRjsa39V$SqL&YMTsjR zR9!_5E?w0(Npk=!8ZJK=d#_-Y7i^r_StSGknWUF8sD#;}JAk~)3wZE2=B;|4sM-fh z^DAxen8pdo7d-o|12Ta@5l|n3gAa73Z$|Ctm)lw&_IsNNfbz-R2*A;YEF+$Dn}@a$ zZB5OciHj?==452Uaypftc~GgDlo@&TYEsYbT9FseIs%wrFvV^KmLez|=_jjW;-3&v zUie;iBN2;$$ca)*7RE{H?98gYCYlcDHvy!Zm(^YsF!I6G)XMi1gxUL~n?KFTy&FHL zwmh1ftFO9mALvt3J&%c46E|1)8)bPRs3_v0O6fKOxOVE>J+S445w zK;c*WBT89L6S73*Nc;BTh6d!Q>L{QJp?#IoSL?MZBP-s7YBnTKDbrA4DpBQAh@l7( zcx2hPxjN;Ldjr2he&M5v?gz#3r>A4ZeBte9Y_o%jkeDU$xg7oBI)0z|ll^-4zE26X z?~xc9szbvvUgCt@R)Q7(FE)}FqC`D@BYr8g1WPGlq#Q(VAU%*oqzbTG-@Vlyh%)s0 zT6x^Hbz=Y(w<-s5S%Fll)7o*IksH_rR}y)6GWhW6yKU*H-SXHcbZD8&lS{_IB|_Ac zTK~ZnI`M)150V2**{n=Fc!M^TDG$7^vb1Za>X#NbP3*uW{FSyd0j^3TRcOS~fU{U( zmMf?b3u6GQ4L^r|?F}G@v!(M0tk#G@$9dsZp>y7@cPipAk;!(fX?P{_lp{Mcw5q+d z06zz(v-nql6#TQJ!4O(0`X9^m5m0y+kMcd_BX+v{Y9r)DoYZ>RF27K2tj5F`dzXfi zI`uZWp|Wjm7Cyw_cyelwKlvQc{ff4kCRzy9FwI)gFWLX%>P-<8EB{_vw0Ixw<&R=_e7 zlAqf-99~vxyGH?AOtQ06axpPVd2HJ$^ZXB=j4-|r{APkY5UJCX5cw9gLIBLp39pD| zu1uLIZ7eTHb&t5=&ADTi3H3Xnf3#kb*RkDmQM7{P6Sl6yjcxyQ9paObHEHnt@rhos zhZ->8D=aO!M9-PKoWorbQez<=K{v=^OeKqO=|%T(mxguRA1M9ulV2i6hU?-}F@Q&# zYr~kUMgNIwiV@l`q!)&@5#kp=E=Z>mT)|RXMe)<)eEkTvAff)>Q2zfJS$^c+PB%ZO zz!uQ|^juP;5g~ISrR_GMLIWM4js7$%B+H@m}cWi z;#-cwgw>3PmM2Ss4>h2&P60wTw=>MTo|4lbkn39!`G~k9T9ox10cSN#)UZ&`|AT9 z9Bj5Z)icrAq}WLFyqs>Ok!okGaCRbPv=4d0gYa{rUj8F4MSTP{^u4BH-kY`y&OY(6 zJlRXa>ea__E#>vsb*ZV9E6c#5UM_7HV@$1G*xpWaM(7k=x+N@Gl+{TPI%giCQv1A* zxeZDUbLMI6#Xh(3bWt#Zq#!lRWI|ik@*HPE&CGazp@`kFlR>t9b-bQ1t&PKOB$FJ0 zI<-bSkaZpGqf;&$r!apkENN00R#gQz{jC;7vAsuw#mi}A39x{klMX8JzM3gX*D@^DlJu0G@jRx1+|)B8V6$^V7} z&i{Ae{0}-brr!-`@<+xp4*9>8&i{&eSpK(1H%HA*1y>D=FOBR+yhst;tm<>RB8&^- zs#2{@M}k%%hAP-r__u7z8MoE;%KQm@)9Hiw3#7MP1i$|*+V8^kLoRmTg@bwoVv~w> ztLvTjIM+M3ZTiQ(`o`lgRsBp7CDqbhl4`^4b=w;2ep4HI&OMn9I2-WBv41eaxjs)H zfW$F(+Zyhczd>ZDvuq!U-`?A~!AQ%>g}UOkgbMwdG6CdMzLQhaoDlE>RCI|?nc}4d6V6)JB#)+5!be~D4oaC90Bb}zJ&@+Ul^#oIF zjtSJ&kk=L_06v4hE%^7kbm7OD^$e5 znEsWi#^yv!GAb?#Eu1;8$AUZl`VkyK6T4URa*Vw zKQ6SdjyS29TQ`=lWy~bX;faMB+H>ysp2jB7(Uo8-yPUr;hoj#}#F^m!-O5G3M4Un>c#TpQUOOi}nN#ql&u=J8Rp_dt| z+*`=Qns?;lkWnKy$;>Y5;EJqUiku~hN=&52|M##1o0w#^?JO#Ct{F%;CfaB~%yLM( zPorN}#=G9A2;du58r>?%?fW@NK|=j^*8iXDLE3W=3bgjm`6%j#KP&hj4HJ19p%fKV z8hz^z{FPsd3L4PKK!gq!X}lb0M~$XbrEDCihAO)pWgL^Xc)7+v*JSE`rr-mL`Y22& z>L9%|40*%ugSem3+JH<&Vcx`Q(R=@%)6^v3=lh9Zh|W>eZWy>%7=bEjW#xwj>(y7B z)*Q-8A!}MRF4i#Y$b!`S)1P|-I+hg2am`f!4o6C^00D@UQ9HDkfkVgGss8-9dm|Mw z-A;(@98r`^YkPIcO%pzx(1g+1_O*)Q6ElkRRU2t?58fzW^+g67kxdz(@^l+uQvY4- zv9?k3!!TLZ75-;7TPn@DEOJa>Sze0D^v726l)6^*(~tqeCJCwU;Eo8%v{U~#Z;p1) z^9d_*iPYf2OZuUmSr*Y)PS5YC{i?K@%Kq9` ziCol{Cg*gcOZV5Ej@+(y0d`V@)~$9g{m;wP@8N)rv=&6qXFb|UchmjfQ%>iDZA`Id zu%6YL?a^wXbK1%>8g#nLYn-)ORDZqDIIvc}`jh}{2>~@mP-Tz#)1iup;Q|hp1c3R= z6>{}LkCI~+r#g>A>8|AWg~QG8T@hyx2#MwP~CZ zT`M7td0(P2mcuV|5$0|!J(k3gwX65(rF!2wvOi9Rh~59hb3o(*Ps@#=giEU>#(@Ip zsVoYj@ok2d%8_b|4kL;-TMPIrD~jqMQ{`JiqGRKkdq5MX8cJr6p?fF-V)-Lypo&;y zXTPY%x;9~G2y>vGjJ%)CueqKtcDg2>3G@0uPQ;V~g3pn0?5R%1T zTNeGKFUf7gMUo1$Trp5td2IB5}+ z*RKqP(TUUnfI18BI?2zXDg_2^lgkbxi*WZ#shU86J74w7Eyu{wV>lU1R7bEz&9+Ri zxG$tC+T)-Rb}C3Bk|QBKB=Oxe*E~{9%s(k#&MO3~cb*#feC!}|nYRJbezW13&JL2? z^@)4TY?O)KdYao`;U}$s&H8U<9Aqwbu=`Rz9OT3tSmjpXS7V;dy~`ore4d@1A^09r z;}$4^hP3nt9{_>oz{7L?Oqq^}yweBsRym^IlmbLI&;t(f6kW=vGxTh(J($2{mbRiu z@px2;QRxQ-AwCt>Sn_}(bfgvRLgPX1uzR$LKB^7J0AtYZ6w8BPg<-sS6|2^WIfx7) zp+d*D=Fs?N1}N>yDaQ;VqH4}sqye=?HKAd?)z%d9@j}1K)6A+nRhtS!tJep!=BPT21pHGOS_^F0DW&u@LgeqkR46wNeb*ouHR?n!eVQUgL zN8apMg^L{twHRS$6P^Z+iS!vbenF8O2W1;xv=zW^B~_PyHpHCZ16m>=2;I6+dvA~~ z`96;vNu-x0ar#=1_VqsozyHp;|8wi8q(k&v|G|ap{!CTApMM*}Ix!P4^nWI5SBg1-gUC;g^gkDnq+#2}oOCvvsv2!O(q^U3f|DRZs76&9QE;6%{tUSFicReGoR~)o zX!_Lu5&r(mVM8Jm9Ut83dOqLsKAQP@^L<6_XCPR1g16D%f>#>pL2`E#==bu61SViv zq9D1%aXZ6$1GOV9%6`X~5XCB5IGJX0&xmIL^fo?+V3*KD)i|l%CKg;(l~6g4%wgk2 zMT9FGjm{e-uGbl-(@a+ME{J!n!oBN6{d%GN_k=7$-b!k{j%QiKk47nyg8zmk;%!(W z87ErW!{?%|Tj&Du6{~IPM02N`(9!jmZQDY$G#fqiAeE4Ww_c25S?AK*_M`3h7;1qh z;5k~>u6FvR$2kHYeC_X=Xi!low?sHOLM(y=Cj7=&;XQNn5#*DTM>UyU3!l3$O>_r$ zLe5CORmX*%r~XBWe!zAK1w!<{ci= zWY%N`#SD|+UTU2Y6rFe2uvy2+cJ^uA0lNkCH(K@iTy^Hcy&qoce^%@N8PT7C7X4X% z|GQfMXRPaTRu3-I5D9U308IaTg{b*SuPh1vq*rQmPBAnw!LC8nwJI0%eD5)&NCQlJ zM4_*nwxM~7W#xFt->wKrjDl_l2>-^h!y7&HMS2Dg{UN3q4C7{X_ zhvZ1@v7I)9qR0_f8!n7TM({C#hekwlA*6j4;4W+bkdQZj;Q`1&Ym+65V)P4Ka!))C z9Gw0yvfeQ|lep{h?v9O)ZQDl2ww-irJE@M*v2EM7ZFcOWV>@|s-!p6Gne~3X)~YX8 zovOX-{Lk6HMR`$tGb!Z(<>0okj}$Y=IwZL9Bj_Q&OEIMIk`q5#e|D_tg18s({^X?s zk54?HsG?C9f@12WA!M~mWmnc?2IU#yvan(9%oJMQ&4WHtK+k!C9rMj5Z*(sKBZ%bgcw=+-$tG~5eoB;@tBA3@Y@7+GxZJIZ_R_d# zf}b8vKz4cJH?29&4`K_R8c=0sm1&|puw?LDTf=~!l)iO7@hM#>VLFX>n2&m`lVE-^ zqTJh>=v%FSAu+07FtIhQn2Hxc6`eV0D80%2WYj-81ebQs*O>v{{Wi=~=s^ogXA`s7 zzc>YEjM60#n%ZBSf}{Rd;hL*nB(*1dC>~xvCzV`=*a~FqB`4NVy%NkrrjT(#6D&Qk zX%O_`3VA4D<8OaaA#W zr!IPWn5C%{ETyF_2GP;~(ls=QTE>E?I#VT9!z?WsKB9^xVb!rT+)qWj7hm^1`7sT- z&*Lyy6URvz?N)XVY#BscbD6m&)6v4DpFO8N`fMKgOy5quT~Akce?RW^hJ(jsXZj%w z#!;pituE7nEiBrC#dfU)z^S@#h)A6e8{yh68Q?ZA{DW^yV`Rb-#w(JMNl!s8!AW{N zU$kbtaG$TT+e!vmDmfinS6F)RIe{G@AHI~0jEanqmxm5j^m~dqdht7lU~zR-74tjJ z24Ul}Fte59*lU4hO-CtFqEK$?CY?#~T=q4fsBdbI$;h^9}}jpo{4ToMAmM#g781jif>CKnGpQqy6O|0(PJ zotj$V58ZjQ6lH@?*Y~w-pvsCAYJ2l&F614eV zrZ5kCZeoNUal>c>OmibII=&Ivc?=@pPJW1_YDMr8wk8}^-nrEriY3wb{rWT!ufvK~ zKOS?=2@P@+KHV=!){E;kAdipl+Uj0wE%UVASJbKkLkKe1mEH6mYNyA*Ro^HSY|vdD zpaA9RWHqhcx6)6&?i+R(&Zb0zos{Hbr&V`kwtLYfG$(jTC%9Ho5scnh=BWJDyT~jPq+Qg5jUpJyuq?zLrjxus+ib0Oi->=-WxZPa zr7$SV3D;HM`ez$TJUsYU=Uj7^_MMKb_5(DXyrWw(&<=H!By#kXGlLSlIslP(%Ggq+ za60CaBF}nA6=`JOF~IN_mNQh2Sa0Z-*8)ayX2|Jy(IUeT4)nYM=9liioPCr{o+UdG zyTZFNaHxUB-_g2NZ_SpueGt(pz1Vn0`d-u#0FV&B$#O=U5My8k1D z|MzY7KZWo=pDk%f)!)sNZJz1*0ZHlpFa6WQP_}B+dc5DyrN%x{y*5ywRN+&B!Jl`o zAvuX4Rc;g*SN{3)@`LhPR!MXq0D|1^Iq5Oj!wc~IuFx-r>_N+=K~MZ^FFZiCZzWe{ z4i~xZD_w!;=p>3T0Y@q9*0-82Pr4h=v3PFAkD%#I;;-kzL~2z8bYYtkKzt)45Cxvx z4qvKE7J=3BbJ>j2naz#9fFB=omNoXuOAp3+2?6T)RRa4|t#S>mWU^R}GK&{%cS)ks z2+~T2!$kmQptK=A8E{%_zrOU=`w#)ojVf&8M=2`jL0FQ;3=l5S@-KH!oi(aGWm0w) zgV0BT=J{*^HKb>13hZhH&eKsTCVZPtF}5n2Kdd8t_Mc|Yl6U&uzUB6$j>mEyHizU4 zgW=0VCHxlsNLB(NXB>@X3}UXYc#6qyq~RrTViS3|59vxe#!#A)u&cM2VAR{;9d3(|J#|s{cn2GQ`5lo+kdPH&3`-M0L%aK zCy6TBcJqv=9~r^H!81uV5j;2~HBAA9zphwQ0%0WLxz!x7rfvUKWidkzj*|MK&$Wa? z`hF9Dama8^!;F_g8NBBDm~x}%f2iIP@C9)XbOkqAx8K<|1*f-dZHWZO_|vjyFxsmN z$uG=>CwX4Y)iSK`mBD4v*ALuv&UsT}cF+}y%!Rcq39(eTnBY>z7G?X}4_HvaMA2~v zoqbN${Ctxyj5{Ik-y2$3n$(48M?s{F(^99t=H@y6vyXQvUXXe+L=^4nK^)1e_Lm$d z_})VI`4rWXY7{jRes9SK$4rt^PqLreZOq-}%Y%fU6vE_dNH;dq%>jzMSM~(s_|z@O zx#1p&@T_1tOrpr@1H1S zzGp>|1*U!ASs)j#9zr-ZwANKRd18r`NRWI~Y_CbJ>vv;U?@Pun`dJNfxC(kxbAJ6C z#y?zy*;wpimnK>?Em;T53RZZ2c)53Y59%4O_Cm+lKdc~C%afDomRU(i{(Fc2f3D*{ zPopl^d%DqANh~ERT+?GF>)jBcBwL|z!vcxQR`Mu{NPNk*?Q13;i%n3({DSA8?Z3AJ z)C8?$CDN9R{R*zU_6SJ%hs|qO|?5XwGtXVh&{A7$~#a( z(aRI1N?Ww^9wlmZ8lOaU*8$~C8ch&sg0J{&+%>)A!F{EZ9qL4*Q+pldhz0wsViLI2a9^M-W^cr&m7za>jc1ia6P}m54H9*cw`B961 zG_-gQR9O`-(@fztk96$p#i}*Fr*vcXz_n}<1i!43-EA%T0)r0(N7HfxPv%b9l>+E zf?1j8%p9bn#K#d(YR^}J|47CEDZ&3~iNU}9r#hRjF2Mn?|2DOZuGas}e3fv@*V)~F zxLP^e;26o}ci>1sv+S~YwO+oOD5FJgIX@H`5rf4ap)w(&Gc{8u7mvjOe^kd5yi6h*bSLLk;#xM8 zC5;Q5vPqDgm`}{*eYaCxH^QFOA`6Q^qO%(*IjgI07P#{h$|FrL1A%?T!-Ob==M)>p7M#3#avz-;8*9)cHbnZ6IL9fS zB74R9IPqAO9i9icJ?6Wd3r~S*^|5A|)1pIQZGhpH^+*1GP z8}3{RJL7+XhHN%4u6<;nf1)TT(XgAyU>{^|vn%)pU|%b3l6lZQGw5A4{X<0E%+7x@ zq4#|h@^-=Q{^zr#_CPuP!a5B%z>#nmuLEJ!51K2Alu6p*m!7rNXJgkcd`G`}%j%$YX z3GbDA}d61$$W-zMmkQPnORt%XK7W#;U#a= z`?u&`Xi){83JfRmUby@27*gPAu)9Z~XFea_>YcB?pZ>>{J3`0hZP%sKGvD%R^yho8 z>^B~JmOx6SpO~1?y9g0uaR@-~09Wo5Y$CFyOi~3cmmxl4E*_#kh4 zsUH$Fv|g!+f~og!>(T*~rg4|)l&3BVm*vtIx>}=;>w<0cL&@en*{zI?f-1PM{v46F z(MosMXy=%~ZRDZxiT2}=n1mS)=aU_|+EEk{cG7W6`9;L4=AgHlm+#zRf4`0q{W+f& zj#U>}NiqF+x`*I-81k&-pfG3!CtSdRkFll)QlJUg8k5>FCglQ#e0FeckdX#r=XZ0l zM1Oc_eqBc1abN!#I4ID9c%NhuS=>{jABDTW)h9R*vWWaok@=>XsWL>gAKz$1`j`T-dQIHWPi04y#)P1Dk@{c; zgXC}vc$OA*;bK~V$+_gIE12U76y9wQKhoy0h}OFu5f9rH<~~UwIdy>Ho$na~&LgT|cpb+J(;9FmsC%;Jf9%DhEc@J**5ZT1!4 zmuQ@U?F~lVBzKbwe)*~^+yfr=3*(O5tyW_gX$_n6j(q_Lh4{;UwsWd_=j;S)yu<&1 zu~ZwJ(-r+v4FC zp4B_3Jr+*mve=ZtkQ>KjoLq9(lAp$Xp!3BWim!=9Tg;9V&C4MkH*p!lX;FFD8M{=@ zT13XG5@Y~v$%cn1)%5rFZVr^Hn!#CdLr&sKvRY=!Z_9AMC~A?ZgJx@X2kWZX>YEHr z?k5C2$XP{qt^8plj&4kuG;)6gcKqJ#TH1%w8cLa38m;>zE-XQ_W$?Qg z7ii2tc!p20*F{2Sk>U{SwCcj}?2rqGeDMwuxOJFb$d63nL0;bVjz3qE{REL+h@Zp@ z9$;16We@f?zoSooy3Ynk;R<0Q8nW!JjB8hxz`b65a|1_Af;)GCR~6&Cg%S(6g-TRa zyG1_E-}Mb-tdUVQkwiOEQB1sndIsr8Cu-XUaH$#kk6P!tzD;^zwYRFDZ*YQUt5=m3 zE8-k(QZ)FI$+A1?8GGeBYIa_!ig<*kV|HpS5E|j;r7C1Kcj>fEU{zp*mR0-!I>d` zrkB$6`Ba87aqfJoJiBd|vqwZ#qGFvB#wAn>waf4A`?-FJ^1sh8`0QHdH)>B+Z$(jp zPt>7LNn7_Ev!43XU|s27Aa6_K-HT$Bp7R<34&oV=LUj3qa$i)i-|O=#%*jH zzwHi!6?!)P2LTcKEa~?w;oDU?;Eq&tq^ur45W#bLN}4m>3MARV9JyI`@1e#n-0^#7 zqluLMz)(%MdjYC^J*7l!u+L4~?Ojx@_e!TN@hXv>!rBeVy!89z(`q> z39;VX;r_P<0IgxM8i*1@=w z%)i?=zpEdIsef_Y!3p+$eE47PRJ+3izx+vOUorK?niHDj8Apt15L{B!coE2`9_Kbg zWLzOx8lv@INebTNJj-*IIDEtNf$%w{ttFA2*BJXXrXk&gXHpV%smNnD`YpYs-(5Ow1Vc@7+q9Ho+c!cU{>AdYmj z!$m7u=1|4YEmL@Dm^O6K<`Hn+f(Q(dXfk#N-nNE7GUwFGp3Jf0#(u;}0TdYi#wDyL z+#53x;r+J{iN5cCKDWarzd^_ZDeU8L^L~Z4U$HlR;@ixZ{AmJ0o4~zovCG`h_2gSW z!jC^1;Bym^_Su`)McETf4(~8ch2cb}s3Q*{dxV87*P8(`Fd75y&d0=S>n5l(; zYB6wm_c6g*SGWKA9*c#stIx(tYxK&-DqUP$ZHQw=x)^_xM4}|l7}t(Xp6{1kEIV+K zw;0;wZ$nZc$sQ6Y-xXNBtqNQT{KtR(Phau>_Uuc5wrUaFZ{O%dl55*hFp_)fQSkqF z-od06w71IQqQFttXvgTj2^n-?4IC2L_`o+1DZvD=7zwDDZ%ifIq_-fT@u`d~-}AD) z8tGNFZOTPx>COKQl|iSd%}}e^G&VK5lv{1IX*5>S{BxV~vD=i1lZ0y6x_mr%9M`^R zf6Q?DH0ykvn&0AmgZd_ee41$waG>871~S-oCEOeY8n-2I<)ToOa;`!nWc1Nd+?Y`7 z7jYT#$Q_MKQn@ovDb%h*V{wLcPwpRTQn|xS$@MBjXWaaJ(QAG4hB2*l{5+~&ih5d1W?xsKBZ=_!0%7sb>m@W^ftG0(c zA+u8b;alP;?9rPu&Ig?A(YmWlZ;o!_Lj+iF3d0!mDE_TX5G-BIYuVmj&vPf~@9rH* zOEW_-rbJY>_z47N++nLR?FM?R91cXbjE1GxW-=UcU3n+n0l4np$DN4gJ3_;BDt5`_ zH2jkt$4#hxhoN2EOT&`^9z!mwY{o$8==CQ?uKRf$bS z&6-Nb1~gGefWX%1deiLzJ=&$K0mbxAG4Fwn|*tQH`U|d-t$NLc2J8jeJZYl=#k-f8_K+Vx2LskFo4ZU>&}?gUd)~Lo9vJ?-mP6%i#~9 zasfwA@4e*mq?pujG3J4cPz0Z8yI<+vwH9E`o`>i4$y}y&0e!GVA>`#()E(n58!uc zO-W*W4G~x*#;WsRZ#29_1l{S8?C21NH^>$kcWTg*NwzY!n__%Or3et($O%H|gpKsO z7FyjS^Vq<__-}FK2!FoPnq%`FdO22OLOLtnb%Ehi_!GmlEeY)QsruRe{PBPOvZwc< z&yg=h!FNdpBF)@9LPZG3a?;=0*mhDu!|O>=t*?VzN)aOn_c15lvwG6%Oss@^Wh3Am z5y5m5X+pIMb9F{Hwg3lx74HB_QYX6FYF`U3^*NZSjA;Gw@TWv z8Pl?t#*-Go?bMA0`h@9gx!Y(D;H;w(rd!yM+^|H!$qC(GccAj28hz)pp*65fa4vkN z2hReUHSH`(q{T3;W-R_Zy_vAK7d0t_qUxZZ3wox@Xo?PUAPM$tpnysOr0-R$x!MpYe&wgj9_r(q=2?8Ryv`f{}C`y2ky* zNeCFZ(lwrK<7%Sbjx}Yc*v>2;)>&7hl*9tsh(pu8BYla5Mr)JThB%e`Oh6+^Vkuc4 zQ!DqGX#H6CzeS2req}X`OvbSXc_Uk<>tPOiI_z=Z1mtq1(Vg)`5B)-0HI;UU7k~E+ zq#XZIY$~T4s8yfZCt?gEq$jwQfN3Ad6%=;9Z=S)R8QrDuGbpBx>G+mD45A4-0PG1k z#)bCaL$ZZE`J;W`GXt_F_240;>Ec;XRaq(-{AAwm9X%~SkoT|JlAV-d)Y~tz_ujKn zoZ8*}ZNX{{0lzPnw5#B z77xdW4QErgJs^GqpyLmpRDNsw#syac``B>Un`kZio_bDY_;$2QR;1J55U&-w(FwY=@~Bp9%DH7`6<_>%*`73B0a z=XAXE!Cr@aw5q{)Fux+yy`uBh?ibkOq3$Z*@p|G<1!S(cW?ww?C{am45PRt11$w$m zp48z#g8}txVffGQ(H2k0ytzBWS73!xD_3dk-GJl_1|db0cW6P6JnI{U6E@DN*1_Yc z74HYPYda^9j#T&0L!I>a1`N=tGj@6|rFQaoBMSd}u2O6M#PHS}oV{y?N^2x?N=&40 zAS;?aeR;shJcMBm)6}43BH9*8fofs5IIfHB8>MglMCMz#YY5*xWXJlJL=OU{RKO+m zjv;fHdT1qNKwg*yMqU9ex-t^ab#hXO`%O|KdDvvm%|=T)w-+s(1`^Q4ZPih!X{F?E zT@lo#8fnE|^8!`A;Qjlr+?s|dr#lGcJC;xJ6xY>9VlRkB!WPRQR%+BhJ``G;W*_E| zhRHy#&}#=iK;3o+6Aq+NA`O@By35K8JgO>|gb^3%Pe3>f;-+clu%Z^6C@La5vUz}@ zkOcmWmupR{v48{T+a#Cg+_k)ClufQ)Mbj*A@B6toDJX{B*?_T3KIaopvG3wB5p%Zd zpXQg8ovOV>UnNurrYHn%hrIK*YgG6uSTY#ihFdf55v(tfRs(1e{eqsH0pd2;$|I~` zUjws6v#1v^A!)a4?%*F)y;fJ;Xl(D8SS5p^uwN+_WVRV72pl{u{&gWVTBf(Oec5sX!s{M{!28j zT@~gy8VVBfu`w!i_7&d2f?wMJv%Fyy`5|7s;?(x+?dm$EWd=REc}k$Gs*1%f9Mew} zSkXpu^e}8y+SzTZS{0PWJUNtc_bQljfNS_STE*7x2hgO862W?mT#vn1MsK;1r`DUMP;U@4n14o7XY(&#R93eD|gGi{5{E~TQLPmhsEpAAR7 zX&@L~gcDU2Q7euTnHTo&rgwx52iz97H4M6>m&M<5vo4?31esiQ?8cN= zauPON2Y>)KlB0p6VSHq;9gnVXu4jCBAjsg@=4gWBOFK(-lC4Q1hDPg#_1|&-Q4F8? zDM&hmp9?uYhfC*Rbnl59ge8oTx3z){>SNT@fa#1E) zf{89Nyqb@Ak-rLGu#n>3%K{}$!yJw8oQYDt37Bl~73eM}*8Yt5P{`Bm_x<86UEpwC zskd^Tcb-attXR-b^3PpA{ZyMxtecjp5`jGbN#z!aFIn&{FfaFF{YUy<@B%UsSB_n{ zB|Dq%QBsf0X(Nv`JlD0fL4IG-DfT2r)Rwnf@Ltj*1!9An!~Q!2v9<-;xv_r01(dX` zJD{cQvdKaxOKRRt?&Q*Q1RdDQyPFfCgt?{2eiP-DO%9eAyx^|oey_HY%{m_e7J1~3 zQYd=5D$F?hlfpA_!&0{`9)Hi1m}h9;eJvKBa@q8IboT^-PW{mEDPw(Z+9nxwp*Y*O z-lkC$m=WC#UwhIO<QR*MQ9r zd0ZoP1pm4kMq=Sm;@=mQmQ}MZ4AZ<6wk&L$m#`3Pq7jliSEIeYSX%*2d+U!H!F9YDr_;U^$H?#&oJSSFtVD!GWJ!UPY$z z&E4^FX#lPp_WPd~9CmPWt)HH6JN(G3mX8$=Yz|@0B5$!+40*rA*;m8+#DU{*iUkn{^MJqBQx=jS+E!XC}EdCwy~b+ya!weU@O=w(T72x}KE z;MGO~%HO^tR;|pV5P1aJu%%jd^tL)cstg7Ur)_!#bG#8)XM7Cfq+)Fv=w|<7n#x}C z^?<1+**D~9*U;Owrf#My#@!$W@>M$X%ZhAQGt#6SOG@9597RA_P*j`V^8iWub}sFB z+*zOK1!1kazQiZszTm<<2cg|T|xRqgIpj0Tyks}N5xiGy~<&orw33Hb?29we6gk@aDhPhlZm;X@=_rrfgl8Mp~wa1c~0q*cz~n2mqys1D+9N6PLNfMu2i z7nniDrrKo$mUUfqBrQp_JDcpe$HG-^a>meM-7rKHL`DAVWu4=Uj0c*R;0kZ>_j4LQ zqa~gghj~HW5Ix1#>`KT|dFOEa6DeTi-rWSa1Vx3;)gDqf3$%39Hd!aM8X_?bm{m$6 zB#k|6gnxsQ5f8K^gc%E%%tM3G1aB z+T`{nn;h&poI(2JdjSMA?-z(~{_=DCJ{)JUkDcDrn7w9>&ePlTe3scz?EDS+v#%#M zWS!7$(Ny#vP-lZ1Ip~kS^K+km*0>4Z2yfFi@qtmblXZriq(Q)Z+k_Z(hm7cH&kgzO z2oH0`^2GJq+el8LGP>t`{IsQ~a2v+O_exNhln;K14G@yZ;s)S7CTiw86<^Hk416kN zN6SaHejA}-TxXTkYfK9eYpzHPAI8#Iz#LG4;x2OHOIiQZ0YirQ?3W{k zOeuqFvTJWxyW~rxJ(mu&t+E>pVTkE1*vb`B!4AVHeYU@lH+J}yIwKkYe|xQ|A@^8+ z#l?vOz;b`!gFJy)4`A##dP7zXEN@eLQWz(#g>QJU`zP4zb$B^L?rU;O6{oZn_>t=Tg&(O)#1Zo* z(lP4wf#^jfhP5WfaTGz=59duGy2%xa@W%$2&_Q-NS18H*g8JZ-LxoyoPDqZs~F#5#_NZLz-|a(om=NDrH}8Y6c#LF{fST*iquCj$PH6+`R~wSAZequfMD z*se1SDQpiNx=?FL@UsyXE$X;ot%jlICg;se2?Ikdp{CJaZTbXhLOA^?aP7a%toow< z2E;LaP#s1@JxmD}2aF(mH5oFdgT#O%UGto5?=;MaCR}`uG0hZ&YzFA6Q6lQ730xD{ zC=>WBA(mdySMZ9A`=pU8_r0mOq@n2`*$TYb!{|g_sz$~4Vhx<+RAt&5DsqqB;>RM$ zbMVAE(4I}b7pJ8JerQdy!-%|E+)_e&0q7IHnVFczI-IG0`vbkxs6*3VNNvF1X)OLG z4&kSso)Th(C1K;yYMbG`FK05zo09M)zR_Z+g1+?--RkfXwDQqXL|4Qu51Hn+zXMDuvNAwLZ?3{qtbEjtn?vIK?=ql`6a{VLd=C{OCt zGb%29AhiRs$J*dF8b`0E0}Bv1pw^{%(ex__zZ)PrZaf=GwP&p;QQ)wqA%D*g@tcxCJ0i=5-@)Lc7*x zMqJU~WVh3G=j8T*V*%)nr=-N034qx?$M%fWE3a6b2fjH%Hk&=zPCMDx*85T8g$v~d z8(jll#Z>ur13mr){)ohz4Y5wR=?@@Wv#(1|KhE9adu~!} z`r%*G^OA1{Y=t52kX_dCByXoAfgSaL z&t09>BxM9m$|fGNqi0Mpk?i1<tL=}GU1s1(v6snt$7r=2FF@8ZRf z24$lPd#hD&J`xy+f!38fo;lnpq7zoI2Jh@oam);Wz`r8tgFq^qB? z*KZ7N*h70D(dE)NF6k$$YJ-2eLEVjm%QPRXH2LeM$^~btW{D)4$FY?(kPeHg6<`XrQ}q6Z>~c(rQt78S?cvJL@PrTpjFYz zQ3Id_JAVYSBqC*$2Ufyyrl0Lug_E#rK$Xs!w6Z>Gl->3;chD!wiyfrGnWeQ}$#}94 z;U~nbA@qZVi-j2SR?-A2h7u1JCzmLq8--X@ivK||r1B1EkcMGVBrLbkwOW&P_eoaq zGSM$z6PdRyw#XKr=g+?LBr>MLh}C0=VgcUFX!E`qs2N)GcQUP|~@6hpQ4~(xEaI?dxuiE3!10CU?XIvP?p1*s%Mly;%6HO@E`WH$;Rw&r|L9|sDwTrL9M2^|i zMD;+$84w+~LeaSOsE+yA2RI5?s{^impksO~g=hX7W47)ZtDy3VTV|z~x6VN9MK$GX zg_apdgkXAwX7;SZ0n_SS*uCp$bMO1&x)L2%EZyRkty7XSU%yJO3aMbppG1A+q z$UN%1NNm2EtFx$}Y0MdU;(N{}-kMX~eaf7$`x_kV!I3clxl63qbilsK478K1W{L%W zQn2hw(o87L{*;S}kc)R}ZKWs>&EMp2A!=eEjnBB@)cnY8=_H!mPnhi9U_Ys-eE`o_ zOEYfoZYgk6P`bEeNiL=e7Z7&xKo_H$hOgnk9RCZ0)NcKu*fP3+GPz=!DN%baCP!(J zK?X?J3GC9YT8+y+gBnu;KtaX(+&9g3jCOCMWOUtULDWmXF}acuO{{Bbb5R3N^`I1!a-B&{+bnR{}$YpvGE@q#e!!74$t^H$vjgo#9 zx1h>6#bkb$m1XS*;MjB``sNI!$H*pJ{&;5CPZ<;4!>g>(d?Ez$YF+Zzac>3#Vh0W~ zinKnQJ3D4%rL{=n?}iuaCc0U!E3Qen3B@>J01@{u`wB_3u-3}moPVteNo`?NSmW#= zqN6_(B23ajbr0(%{q{I#kngX<3RR-i#-KH3qUE}DsNoZ*0DQ?KPG0_4UQT1~xm{e& z*_+Z*$BS>NnrrRbRpznX7!4n!@RP9TtYw%r3jf-mE(%{4jd$FXrP_y0^9SRuzhK*l zsgJ)s^vyQaQy;l+y9PQiQXlc$sm~fH=Ep2}KGMSbZJdFu3%8kf@U~A>{a%69_-ou1 z`Zy{!K2mfin$OSwZj$`pv&sKCC-t925}VFpEL4VX-%bMm*CNT+q#Sy(pbZK{@)kM- zB=Fh#*qO?a%3BvFNGKTiZFB-mXoXaeHSpUkDXnDB?{HyeNwGeejMN$p8!O#5jkU7I zaIZ$G^SqK6K?^_2@};%~%SB7=Wb|@f>&8EoqR-h*TbUeoO|&K1TSxlouT5UlT#xIQ zd|yYGmsd|A0-SK)RL2J8IET!X6qEc!xd1eJ>h-%RVNr0j`jV9M!0sC~kG?15YO z)Hm8qIazi~?|2l`+jioI_p^5~EZ&Uf z4vX17uwHI7sp%brU_Wuuk}cnuDS(Wiv7uc@>Z+XGTe%PL4^ha^AB$aa;SBLL>a!c_ zB~R37`uV$-)K#-jjRQBQ*xggmzIvtCcASPZ-C3S)0Yh#)M1s6g*EnHQvI8%s*b&;t z8^y z*NJC#z-6K3BT6qM?%PkuE%9F+0`Z8LFL(vJg5&}t)sBqO0jiuDkW5d0Lx@~ET6(FY zzz9%q-7ohPO!T$wJDH#QHvC33H{4&4n$7tnkN1ZBV^@(H-{~g%dbl)QVEG9}t|eZ- z`kErs9-}*pUqEzlUggan2sbNvB14;2y2eH8&fE2%*2~!yrS_E_lve4G8MG^TqC@M> z-$kR=E8J!K+S1>BRC)6Y|E1EAdo94dLkTGF*>UX3wj27h{0KwW<{l zPh@yn+;|S-$FIyUzM9*gX3#L>sv8fUQ-8luJ2S9stJnCcH)RG{fRy0B$&s?h(X?dBWGy9rhP#pMIZ!NS2t`BdfNDLb<{|bQBG*!L13T+tQO_(TUe;#!L@Dp4^e#?s9v;R1 zMGUij@atCt0^8adI<=!12wv?zbRV!j_dB5gjw5U$_Kz^VGQJWTh{RDktL8uwB0KQ} zaa4H-;}R1vdn_4=BBC{;^D}!uJY+gVLU-0K!btM(KQVaggjT-eOC1!U!NA(y zyM2h$YcCZ@F*a+HKVYvso|rX}aCO|CZ=IR_P1Pn6=3Alk2^h#AwWq(7ig;#*n4bPVuOrlq9GJOd`US{4mp+ zt_dSNH9M}YWn@XF;ovN)gHcjU%C)|Y>xa3w0*D&4Di~urA08PLAKBI8->Jrnb5Riq zZVoV$@MfUa8v_wP2TP3e`FFX0H}xy-u<`iW(<0@d+ssr&2%^d|TIE>kAuuIt$01C%($6T)*YS@_%~ zZZSSo3Kr+DIV%QwsbEjFto1;Noyxi6WmnM=auX4|TKW4Ej3&iWE78@WXlR@{=iL`h z7xy^0zI@P`VNx;@j)`aho-?(t<--yd?wuN4^JwPK>FKK|Y@XbW>KGb-u>>&mUGN>w! zznCCwPOh}mM=4>QH&!_JybIhsA7ho&2&hGz_1<~#$^7S@?scH(N=GA2YY_}E;acZLas<= zPmY^^KG;6ih}#j{CgK2B^8VhF$Nu6B*LQ%wRCNyP>|6)!-et6sgTRyD0w69zhYw;r zGjXe9dkSkd!ew6vUQZNxR^rhRnq-`#i(zh9L{6qAsTSMgA5HgZ`KBo>@7 zqt>Wnd%KzHYPQ#v*bbr?{rkm+W0D_AX;+q#)Ya0;Fjx~3#2Ix}Bd@vAAu~}{2Bgmk zHg_f-o|kq7P90*L*9QOrB2kEdP!vf}Ymfsg&c4?|59gVN58;o$qSAI{+|XPXN{Y|> zq(7LyLasrbJ;s%|^T(1t=pjTUvy<*$y?BCrTHz+4uP6T*@;_ham2AXIzNMM>KS^ln2Z%OVV(i`I`POjSYFWAsoZ;b z2$SyCWF{K;>&S!#r0P6Wt&VgX)#ss4qxaLz9ad28m3F6Yu1_zN6xv7UmlwUz%_X#j zXauqUm{R@~J9`l++QPmH3DH#&3D`W%K+#IXqT*qh5GWeO;Dx{|UA6>rxb)iibap{W zF^%4s2#*k4+g<~YyJZq~ZIYNSe*}IfTT-EfGq#hh)GlK5E>i5{%#Eb2!Fe(i9vXu} z?9InAFC~3@!1yjSZa84Zu9d9zJDM=1ex87xOAgQIFon(#OxZTcOLLQnrTSpoOa%hy1ESodfyi!a z@7BK4%g9bmPAw-BS9bZQ@$uJDO?DJ1@X|N_x`#QI3e-~n*p}e_!P3=Zhq22?)aXs!80sOnC7r|p${Xj(lr7z7FF#lk*r8V!OC$sv1dc^6F9J8{LY~P5r({vaX`>LOSCh=Vw#9BuA8%S>ZND&xEGY=O) zL_x2KM(EmK%M~JQWUA*F18_J? z!=`*kd<*KoY(1Mwgqi+w0(2X*BGppr)DUn?n;kiuAlb&Ond*__E6w2Hv z$QUyi8c~zc)hw8pTP1GNFw-~IpIg;xk~1@@&@mf_{{Ogo=itb~uI)Fr?U|Sp+qRud zY)vpRJGRxaZQHhuiETSMd7k&H^M2=4?LY3S?%Gv*_qyx8*6Ou>7i|y>a$%b9!w;}o zwD?_^okPHR%apouy_)J3!csxK>&ljMQ(EQhsu{o~YwC0C-;x%*w?(0AgeDv2#j*3i z>*~hVdOHP`JVROpEAjmw5y|lU%?k0%M!xp;mDmpd$vVAh{Fzz!$pUe+L za%FiH`Q;>SQ{3QcOJ(9V&4WxPNw7+A-9%E3ze?UO-UCsT9ECrL|?h} z>8pTW+-Jt9ZS7Ivzh}>gGHd5nIoY9#f6ErF9miF^Du(`*J0K1Ei&++VbT&p)=qDs4 zD>NyTk`+g{6yl?I>l>~U6qlW5tcu8czS+kGfBlcMP9+CZJ50FE2-5+*OFxB z9mt(ALF(5+T~ZXem}J%zBP0?gpo0b@F5LkjFK)hf6KSvj;&jQ4sGuPwJGp_gkdpuQ zBJace1sg1Fg4bb^9+&@=B>KMYwWHNmm*$-*tJoi;L<=VMAX1vMFp9Dv#}5?{7*Nr zcQH{|ceNw~HuKs2D@N1cbH5@@X?cipn6uf>95xeT@_4VKQr(Lm`9H|ElhS#31I6?`(B1BXZ_O%SW$6&cJU4+mzM2;(DoXCMUM)aKIO_ za-^Fce4-r|9EaC4Jc!PD zNQKYARerRd%?$BYpW7J(C&b<+EZ(A+4)>EHYzSCc`&Tv=>kKWG)A)Jk?hyH&rPClHS zrT;#gE~~b-*6p;tOM7)r*Qolv*<@Ct4~9#2Jgf;*MAfP4*?IGzVkNs}G|+X{+4<2G zc)o2P^0Zh7b$S|^DAlJbaCLlE%-2`5(y5I;sRt}P0AANO4ph1(R0TZZBfMq&&>;2b zTKsng?9D{a_K%;CMUa)>x{DZFGC(c{O)H~io zByKF>qY9dKsGZF}i=kOS2fjI`xh~+Rf!V@E7yOI_T(Okb-p1Q4B`}(|wEpzdp0#ZN zX%`1icT#F&GUubBz)@|9bw<)BA*|&tgu_h))R9fqbsL=+;4_jRe3Ooa+qKd)wy*lQ zw&AKJd(<`OGpP64--u)41aGNskRx;~|8J^KNriEwXv-(I|L26=yA+XEBPYEA=_vvS zrCbU&a1k+R$zf=f%@IK6vRgoPDf4cb`0P$oKMvcHXjdBY;!u#cE&%CZXZ)N0)_d+G zTJ6UHqwzZ4iQJ>8!t%#Y!IkwOxr;xyW+fkw%K3A<#P4@=RGAe9{xZA^hw6Q1xM`pA z`YA?AET}7JH{!3eK2!Kfrdm7X3i#eHvaE)Pe)a?SfE>rg@!*XNqMhlHlSKcMV8{^O z?2OBvx<&QKONljmID&v~uz+S_>>-sRBoK-jmynh&hU)K=Nb?B1YqBzrd59EV9t#Ch0$?HLhH?8 z0%+rdHh3jc{3Uy_g6JD+plGDZ-kv%xz3Ozs2V@UW^c#={&+={?GTSxAr3Ge82@8XX zg&05~1$m;FdC)-R+Ekai2F%L$j+id6;T|xmhCgTZ^M-2EU(&45VT8^<#`Z$a<{Nc` zPDwnht>~%rA`89<**kE;T~bVw#-8DV0H&k1oqhzuL8<;xcO1+w2@+v!*ToMUi+Vdj z^vM$WQ&zS}@}NY;Sf){qc}eo{b3)BPBu@IaT=)}Gkg5xYyang&7B>YZI|$bvR<);4 z5fZrbM2o_s@6TpW-p~s~Rns8$>7se%nT}mMzj~?bMmJxyyK0u;pDu{U9iBNjkQhB%(BYRg9+uk}1?3)b{Z1Yl%v zH(FtV-YJbzn2=VP*vd zX*Rl8w_n?gR&W(W^@1#Tvh92W8bDZL0l0<6TKy9kz; zU`|Ei;Yh4Yl$4RMM#?x$q0BU=S*=-td6&c9SE3ygmT=DsBix}W3it-$mG`EypFqHw zFRxhQ^@nMc*P066#aLEcLNGUKbsNAl8H{Un63qBmLKMYOZ$KxcZOl=$@dZcGx$G_O zMc2m%|9SmRRdqu{6w%8hz|7t z8|qal=`hd5$Px+oq%8?NitDR(h}0Ke>_C{j#xJ<%(ecs~O1LQt0~C={qz!4!>~mo5 zbIktd0$SWs+~m01 zd;vpukxP;kt0n|M(VRKIMd+)uY9IfVg{{6GelFGou81v`Dwi{1mLp3_fI3DX7L7Az zBZlP1{8|Q^Xu91<1+HWkkz0Y)7HGOgS9O&8C&wC_8&a2IOg(2F`kJBQ!!GfOZ$xCc z8YZ*L^7yh9CxGjC&f;5#wGcd9sYCQ<)1Bu}YmP z!-K$hj2@*i(_fZC-6Wv*Hm6d)T?eGTJ_lcf$EaTH=TQl22CB^P;WGjm!Fq}Cn>k#^ z5Wfr}VjbE1ePI24sQtU3gaCN{U06ch2txW#UmVAP-zt83+B;HE$CTe)EcXd$LfvT2 z&2U0nzBvNmLV&~A97={NIp*K8|2Xy7P_a$Tg3`547Z2k)pNxgg9NjxxDmq9k}jTt$Kkj3w9+mm;o`5H{3i#w2+dv)UjIbx`RrT30JPbQr`Ie{>aD0zg^c4!1rj)#2X2aSj zzECyVqS#!oaPTW!n$z~rv(lzqOl`(=<1Pe%Lbo5F($R{ihL+L`X7(wDTB+ZB7<5>% zVBF-S!y4XTz52=yB)%_mFAvy1^hKZ6-CwW%6IO=OHhm5wb)$EwV&G}kD1MMV%W z`5J4s(~7XjgLSk5I|4?|eKm#ou@#wboigHRg5tOXUgH6>L=e;{S5o9WnFWU{(S+k* z!vC6AxmZ?Af+q&|ReOP2@=F=>mS?#<6TBvOt%$znhkyMVbZ%yBb1jHxwpsWGZ^bo$bU}WIp3ruX~;f5AH zadkxE;GqzwT>x#cC#AQsYp#Xs@0#j#?%6{5P^0PsOBFz9xZYJF~wGEympFeTkK$R4iD{dCV2|?IAZ|3cn zP6h>bF~(YsyI&42rjHzfTy}wF(z^j(T39@fiX@RVwYvt~C?GL7g2@AFlJWDQHuT$} zxvJcqd3K*4ZZoWHIxi@jFR%oFgZ+}N?Y5uFduy+D4y5LGIysMDb$JJ@kGV+p$449& zk2XDk9=Fb3>qlJq%c81mBBqA)#5iuhlU6+zg=|3k;`%J(0DlX`j9<9BF+6%mIR@P~ z!OCfs8;tc=*-3meWp{8;y*78&8s%RUI&;(j1(+3f6+Kk{T>;=Bl^&hqkQ2h5z+Xb= z*4c`;BT8mrD<`(oxK?UD%11&;F?55U4bK+Cy!i0ZZvTM~p-2wM1c_QR&i7%@@o2)cK~dG*pSkP7 z;iaL2_{Vl&6_OX4;?65&Xt+;S55!gv=I;61#6(PSn6ih4A6MHQ)@GljXWU#EnIQm# zA>?8M5!(S-r^fMh4*X=hndlx}Q2T|P%9L_j?ZYW1? z*>Xu*kSu_>8pT^!q~rHNlWOJJaE&nsodaPI zVHD>pj~xVuzwNRu6(0n4lZ-?J!5Zi@#)~#KZr9RJ9j}4P%MjhCbc+<(DLy;!sA}j; zJ73s$x|H5^LnZ-y`JD)VdF(pfNT(pFB?^Lh^mwp=! zujP)~f4a5{gRQQEAYMNGMArq?Z$Zy(Pw?K#fp@qIFPb4tN2f^F_#O&ybv!Z$H1s-C(amHr+adu9%yn z!+R#X`nqV8S(odocgU3N`*en@hFrcSH{p=r|Q%J(9|P7R8Unb-oM1@gG*3b4b@qt-qbTaHL@4U+(I*S%9p_c*I zm{71M0zPb07uG)m^x;$A+c1do5WNAnXR^pjHoIn4IMoMpc1mQ;L?;pQu}H%CI%biG ziXI6gm>QTq==;~{sLsvWE%LECgd<B6ya3+|tqbUsDir44=CL6D%)}*+7Bf5R#zir2UxoEt`x4s3;&tJEX zA}lsZ4>RkF^2!l0MQ0`Mwfs2u0y(<W~aL$>xe|LvtyoocLKGAY) zL~E>%X1v(uW&sph&~#-ry;GB3 zjjf(Z-gdejb};)Qh+0#jzLy8M3Zp7LvB=c4s@1JVRZG5WQKVPHvi~(#wgS637&3f1 zf0!BPqx?j0DE;|K2X{>XcMXPd3$%Z6#|?kEn#t?(uYYllh&M|Wo{X>6BaXyA3>puT zoaB}dTf+MtjB>}$CHN%6bd+kOmq|x)VS;CNleURv7?VnB0rN%y_l5<4d&96ke%DvHXxUDs-v#HWd1!On=|(!h1hc7wt}-3O*h zEaCY<+!}f#4;-5y@Ql+Yw(e^*QTKf1=E))X1KE5Py2Z&GCP?xOBS_1Hp2K{d`uz7f zJvS=27%VgX)@3sDnePDLFKbgsL9#FaKd39jTLP|A;?!q~_!$b$+gsMc^08pMz?-z2 zA_%~z=OzXovg=DQ2)mago01dh_m_N}ixS4O#M#ceoKO{5sZ)2XRDoL%WBq0eycR3ot$Pv9V~PSO7;Z!xlTz)qK0HZl?>eu ztFc9(6JjU75aEqX=XjL3gB`O2cd+RnG~DKNl(?rMqFXC&yy4!w+l=TbPvCT%0W{}Rjoc9cfJ-^Z91YVefmjoGU+NCCIHE8Kp)C#HAI}%M!fS>2E{)8{|XDNAt z-fkM0QRBj7HH{PNoUN?Ud39E|B59^XEfpduF8s-y^r-|Y)1*VTnD+t_In(fl4#Nie zVZR?8d!gBMSo=sRLx^-r1j12Y>8=$QQNg{xF6=hI0E+lGtdJj?c&6A)&b>%b9U&Gi zPmcU5ByA$RVZg1{N{J3!deV4Lxeik17-rF@dde%Uueb_;|NF|NrMMjMLUEM{?6(V< z+$|`|?RMYhlxMPz5;@nQE69X59*$)l3#{*E#2f62$%d2aY`y0hTu3lbm?L{ellP}p znF16@%%Q%xpv6@a$Cay~xhL{bgwZAVWUFL|af3XODZM~xXP?cJIRJuDAr2;fmW18otAJM72O<&xF? z;MRSEMq3w1uKSQVSrbTE`5aBW&}WB!S6%&Q*{}snY*h0Jqw-o39<;>0*(#X7_ymY@ z>B;7=>!-DS{Cdlrxo|uwMIuxFxTqn;%6|swN zy30;XanJnVj}jD+#W)UN1rw9oIs=>|pZMP9Luk_|fF~YH-Qy@m7Z!VP#+9+dm7TPC zlf{+r%QkNr01^wZB=wa9ASBkQ4cTA>B1T|VcM6VO;M@QnlnW$;C%L=h#0MJ0td zFAK`S-L38k!5fCZ0VP@N*J4xL zG3-w+znK?LeKZ5Fi0qtdKdw3dEc2xCA!`tzcAJfg+k6lZKR^hd{&<2nqAwBj$Hb{t zP0_~ewpLyaD6{QB`U40YJzWjGamt3T7b~#5npn3y z*ly{Yg<+1FwI*Wkt-bN(@4r19Rj-;_1*bYflp+~-jULm3;{d2sf68zxmK;=27&vTN zkyY``6-4Z@VS~9_ZWwT|D-V%emhpvb0zIDzcp}&`KHl0ym7)2yTScBnJuqG zWM!)CT2eTRh=y>{D;ukb2I*Bbv1XOHqf^u%4purd2EI%B=p+uFGvXkpq5nGK&y*U? zEd=ncb)%ltg5tk8T{{=l_{>u(&hHv4H%AixZai!DwgAJ?TM>j`{4Vc$=g;1j-1O=< zfzghpZZFWvW7TQ5%f7YlCfIcpWRVN`1}XrXxDi`vlOy&Wjgk%i4wRTe4B;c~6gH^NBYfA4-}lZkT6&SkKj7pInM5w*6?2YxGGxDzN%^`CgEjHtXYRH3QP> zZpulm-vbh(^zpU!z^U=O`Rj%l?lv%@00MP@q3Z7&hQzVDT=Qlz;A;#IS)njrDy(Yb z!SFA!qCht2{mkrJa3aVrWQmReI-M5IF6vzFwy-uQ#WInzCB&Mcpt*`Ql#z_WdgYO| zii+S3MT0Bh-g5UGjxz<368eBzh(lrAS1C0d=IrpZ--vXWL)e6*A5J>w&rVij z<~>_Cc*0vpmYh%kk?~J5y{+D}cDIj^(*dzpoOkrm=>D_r>sJJvuXY)|tHfm=_f48= zJG>cRfg7lyb6gykO4FxxmS+I51nery=YM&o|J#lKfBN*{Kw!RsrcB>J(`J5hHYaeT z|19wUbj*t=>mGz>k}|d2A4P zw>5D1J!+>t7Y+*LlmS(^`4(ehJHw&AiXg6FdX=Q!ecpaiB`XP7V$&`#E zX*3impi{{Avurp(FqU(Xe|evzR^~a^2xnv(+I0 z`1H(G$*iteCb->>!=0~4Sl8&7dNh2O;-1>(gTzb(9fsqcTxtFB!0%oZAeNZjE^gK6 z#bgw8TOiGSuUku>@A0N>O(pudP{wxP!5v#xM=h%c{I${OfPBZUeB5Zb$KP*{QU>;r zv#kzKzJuEI$YJF?-RB=A##?heGepX3cKjvM$a*2enn>)~1rl5fSe{_~hm|^1ekS}; z^k+wE;k~EtL&aqbAiIUhUi_<`Z zX(%1yhuY}29G(UYnr8fK7N9MgNa{G%V}N>$RgHRP{*qi1e`6UaQ~l-dL=0}o^QFaX zJ4^WwQCqdfhFu%^^_J(jHjd%~v#m0gz*4#1@kfuDK;EABc|vYIAi~RcdSs4k;ad2| ztKTzLF4HTrK}{HWY-s0PtLQGysIe0f=25g!EYaU95=V;Q(}KIeg6aYu;&BfDSD&;Z zU=}gLu)~R2UpXQ-!>`DnVY4Z_Y&tPp9E{>uWVXt!Tx|!E=D9btu}i(j4FAF8{cjNd zH;~~W{wI+Ci)nu3BTSbBalA=?o8Kfu1BvAsuz|j~+{nOdEDStgiYN&R;1^K>~>BO6E7Fe4@ z_=nOwp?-_&JB%|w9{0WtR?*#-g6T~Paa2%;-S9n_a(7ASsaq7yl4zN}3)QcDD)3H_Eb9?Q9Pdgq(3WqpHss~}XVrcj;I&F3*KvHj zp}kv0-3@LkN-@p&ab*&}m#Dr{xkjcE--EhITtH9MK9zQ&6zL~^Th*Plcjm(kcR91bSVCTWdOB1=eRuXp)=(8I22Qoc zdHhD!({u`@TwY8Bv>wj`U@t7f5p8)qoD-=o zOIlTt3aCHO7mgzOvjVK|uoA*r*KpCYH44i27{A0Lp1{tnN}p17I>hpy$hEz1hDJN^d*; zn^omi$JUPfhOHg;AFe|b6t{|7GH z;oCK^exh7Cy?p|&PZ7_$;cK!zqug~ccU|(MEQ3Y;U7z&+Ow2GTJ#2J5<5%=+LQ@o| z05;-hD22ScwydWtBE-ZFQgbQ~&jw^8%p0dc;jr`t!7afDZOaGZf8O8!`)aHdxW#QfB2k_rYwfCZAH zxX3}V!n7Drp@Ai?n4mEe8(Ky%Bmnn&855>DqwQJ@PVd{dNSloBDdXsRvQt7SOBc6(N7- z~ zW_-B@T0g#Z*u-9VBow-M|7o1$&J9+6w*gzf7Z-WT!+t8likEn=dTe4jkXI|xS&uiU zI--vwkoVVqx^NciU%ip$Z|~+2*FZ0gk$T}QbYs94`mHyy0LpDgEOuZA)qaSu583{( zun);TIkMj4b}`YrY;VElTF%l+bx^;DWbf{wfr*_i9Z?lLeq5X83_%HeEX4hB4v}-g zDW+Rn<93!}fr8&$L5@WfD*Hqu+?rKdDdv)#BS3csYbrP{B6Ce15;7`|NLR?c9(C#C zrW3H~(I9e$6sIyv0Be;i|I2D*#SkmyLtM6(O>1K%=gI0OHr23{3GI^Ym8@#pg2c)( zT2#og$AnwE0dyRmwprk58yl{}yKR-*056U)Gfi1#ovyViB3-;BdTxOZ7RTFgGE(B? z?ePAEEtQ^={m%qjSK~vIxFr13Vx!B|QUJj0L)%YoMXhGcR8HC8)S5-8r3h65@ac96 zN23BX`sz9UONYzgAB{A=(&n=;5T6oo}tmnh#L1*hw#Y1?BB)U69HRUB9+xIKNWvipLbtUPnw2 zosu@6K$BqQE;}ssM`MBUp~n~XyS3l17O9ZQOffVS#u46WMv%<52`4v}b=y)%qe(g_ zSTS_7FH#f}BECtkSxv_I>?vJY9X1!1PAZJu%y8!!>YP#YAmBs=uBr9hngcKj>*nqr zJ;IwqZt1{c#BdRwMu=@!p(%4r%adXHvzQ4Z>?FRX_G&f%%=YTya6}QCmSinY?1@oH zWYw%pd66C8`$MXpC8SP%Qc_Je&0)MVeIMelz@L%Dyz%%JyoOE}_DrYlD5lHmbId&o zuDG_P;fa=(Q>$+pJvnFMr5^xay-FrNx^d)TV=VD+`J>MJm~(oO@t+h;1?i9*D3Tis z#VwN&HjCMyt4s}PWc6C2YPajMaQ!@?L2g#65+duVPdBOsET+B*`(5RqG~LRzSR5r5 zV)zFPUa(3x)=p*5h}?Y7mEl{XIhfrVv&d;*;GZSC6fbMw;ZQTVy9$73mKc8~hBS+A zg_yGEh!NalwcPyWfNmO1z8zx)F$i!P7lY#GM?8ipqYpc{qBfmC47w346KTCqdf%_u zQ2hkSuvZ1lufA@zfg9g&aA734R>&V}H!fbHgBQ;=;XFYbztBU1HUi$%I*@;$-TL7E zqpj~`UZ?E~5xs~7=@0?r*kbPM)y~phcXoIy>=XAzI0mQX$5FTrim{sF)fP}g7HOYdD3hvt5tVt?Y+Kh2 z{)xEs1Su?8>eBQgXe_Z_MFCJ5rq_mdYOu4hfW3y(Pk2OFPu&B6Q=e=e*6P7k>TFA& z7~}GYPU(c>@Y#f+d&Mw94sKj!J|+wR-yV&+NB=_0#peYqo3&F@MznSzno}@nNdQ=x z$)@?n4>DVQ+eFjXy9u!hmJ$|``n5-dYOSrUYV?sWEh7dH5)=}bj*#_ z9yG!E#bQI?EDHf3?a=KE0POf~qNoCV&Nld$cksivY7e_94iBe z!eOd-^Hw#p)XwPq#gg6Xmb^8TXxiU&t1h>0aGYjD+jc|%AOra`s_bo4&N;?dj05K= zk-SUrDJq5|=RpctneaK5rc3z_v983L@86=v_?E?nu0VBf1l{+NhCgJ=bp*d>r3Bj0 z_~Fd)yU74-K_d(d>H971nX*~L(e2YbdsIr8oU*D zr+&It5em23n@1a9Y3ccA44mgfN}%`Z1fvO=U#=@kBOfyAMx<0>%C1L~&CMcX3ocuO+dHbk6eD{lFqrWwn` z8HbUMEbc5-?T+v!c+5fNhcG6brRS^s(nn@B4K~9zG#D#vA%4Bk3>&V4nnpnL?ca5{ zB6LqmIW!u#gr5oczf5RYMri6$Bzq6jRs1E1S*zM_^8<$b4IJ5wDn&3vdZbWhTf8Bj zE=vW>7O3UNI2%{DL%Uzx7X|iz+PG`lqq>L28P5Tq#rAtgl0&|&X+i?_H>EEpBj2zN zcbXjTYSw=tchqxT-t)MpShm;CjP*|#TS`%x9+ygS4s7upPqd>+Q7K6GO)QW-Uyxnz z9%^zUPN^nKzHVCA{k929(-ANNC)N7f&atsnKx<4eEB$G`Z`0JiX)m5|B&` ziEmTa>g#7d13{&9LoIlRD!*E78SE-c(40#OJZ^kOp6@1=vKWMIFhbC_5LvRwMjr+E zj9-DncB@=7j1K@3wI%AeyBBXBb9mMcQZ0tV*fSt6%%G!|cMw>y4CE;ArsXrn{9W*$ zfH^n87s2Z3VJ^WI=Jk1<^TKFhtIY_-}cIR12XZZ=d;%DpN0dM;Is z%kkK5TCg-|9OYzg1cW)wY{r~93XSbak=zUY0~CBugBx)mMSNs@cSUaUKBR zZm^AJOhNpjaWGw4*5gs$m)71UA5?QOi&e=cP`aZ689L9cADYV)>&=YOvm@4Qi%fSO zU8F{iS&ZSxl-i=5-??xT(Q203N{pR(_^rK-Ub&V$>UFMq8@h;=<|0+vQCGV0OITGT z)4c_AoBa|ZWw;fj{u+FO7-nV&4iVt_s(cK#(t4nCLCF&U9;d$WXnE>fxlnLnp{*4B zqK#NE=A&2QI{WpR(M{5HT1G!Wp?+fIX}}3ji7-h3a?1*LX32SvW_D~%O11zdHf?zo zuOEFC)w}z^WVZWe%y~oE@`KJJ&L(2UK)+VhVVH(){uG3rxh--DvVyftVOt9a3TqcN zELzeL;)ryrgD>em3gIm{HdAxeK-j{BP^V!;rbYH2TQ$bfuZVvEd{_eCX#_OPe}B{O zlw$S&;{g1hv?92RVp8I}ipLHGjN8NeKWT+v0w+ZCPkksSFo^$rry0Q&kq->wzgb+0 zAruoB#D9n$&D~NAv!JN|S#lpCV2pg5AZ??8fKUN@g{bg>TS=TKfFzAKeT-$yFC;pi zaYmY|Hc>gJ>sC4{X)o41aZyV_jPPoMA15scjBVddU|K&ciT=!y$=w~34zUlVu*amS z|H+3^v^07XN$|PlSRH=aAOwo3GydS;@I0Dw_v${%a{RVG`|5lk_oaBB?-4g~{-#IU z<{-4fq#*yRfu=rU!5XB`K@-)Z*D~ODaq~BAe&lWZgJb#`HWn2~EZ8luFS@JAWf!TWG)bgzytHMpgCt$`(>K_$dyI_b!OWgodsYc0!w zD8+W;;C{U|kpamg1GJ7fS?kT4$K9)%uc850f)Fsr$iUA9<}WfVChS_T+mF<*NL0-u zte@JB9B6sYpX3xwF0Iu#D2VF{PIklO!quOZ(BSP3&|K+i{WF|?2pXB1{}e@mlv-)-1Pw87dIfD{J*I5EfEUQ0u2m_G z>@fiQ9+4pCz0ieRnv^x+z9ralDQ#fY<9L9f82lIBJWZy2Qk0OzvMVZTz8|BYKqU`X<3_Lb#a?MM^Bmqgq2b(yPuRty4O{?X2KTTSGCp`6RoBE zmUe8eZ}CZ^FX}~GM&1aVzv<959OQz}@X9J%9IaAr9ABF>&nnnX4~{6r8Ow4F|B8Ym z0207?sF92*+GZvx40joAQ|uik2IOFOJJ84tjaa(!_maaL&Pg}e)BTR?-y@S`P3xjf zf6uWjr#j0#p_cxakH&y%Hc%k@c5#FN##X{)=09vr(R3mg~xns(Zo|3b=vsI3ydnppX>y!6?=!}xW%a=tVAIrTj`81Qma z>OTj&>MvPB{BQQgS3iLJVG)=?o^Iz{xJ|hzQ<&=yNIJUl%xU*fg+tnfQ$vHq4agc$RdoxkxLh1$d$wCpiT%S-;(N3t1senF!$D4A z&G~X|NFq0-`N?c&gRj7>$gf!&(ZEct$T5W7{LflvJ|*&r2`{+0h4bim9{~`&RZzTX z6gP}AH&l6w0%3hLs()>Y`;nAaMC?4J41Z8N-RN@QUGpg)Q~PhJ`fe5ERhKU6KFd7? zY~l7CJ<<3w3=@ylqh*ucV&2b)1&%Rb#|ABqQzn_SFyOdhtz%M^uMxk#Q_R$^yrFW4 z*|-~F)K3}07O!F}Cg2Qb=mFYF!qsC8FRAi)=U_kUxxf`1nrab7N$kcWI+8?w4+;7X zxOE=@SVZE@XC<#l^n|dJG*qYLa(+hO!X|LYvngFOI0 zT|k0j4oPgiAg`5Wa;+mv?&l@K^!!8URD^7=9;!KYs7R;ke#ob)odhtSGXCH^e2O*} zy23Gfrk1$OGKzJ*fRQC9>mL7ae~`Z zX>(`5Jm{S%v2(n^sgNqQhqh>224}X$^DKlA2@)?*L+U0Ba|pU-(Hh=5nxgN%S)v(=+y{ zPF^ zlep2rf{&o-zZ~!b=U%N~^ompGdCQQ92ANj9FM}#VlcE;0Synz@Ybc8rMc>9#L`7Wh zdnk6D;jwZWo^O}i!{-L((~tlDhx&hup8qY?{zndp_7Z4IeXFhJd2^_A(Erg*jD93l z|5N=&@D>;0f&>BK1?HMF-~g*Vs33u-cw7X{WC0{9kg)$zSvqrJd162Uc?MMg{l&5R zm;rbqaaRs{vtK#*JF89;ilijg33T-o>*GT>KMc02u`>z8oh`9D|&zrlhR`GZF z1T@V@d|4vdg7$wa zd1gcNOm#_MkwUHd^k3)A@VdwotUK=KZo+{RBbN_?8rI8gId|w%qQF%Gpi5RV)1cj9tV3$%9?xeYNC$t!aRM%9+j$nEq;b%9Er&N=K>1LHO? zO53-Hxp*nBEY%g;bS8^N@|te`;eO=7rywO*MYT3N?3bE2gDpV_L%4Gv-b`dV(?C(n zI>nB;yT?x^SCKrvF8~4nCrgje^_`+@`HzTzSEF8TvDxhZL)AM*SJpM_-?43U?4)Dc zwma_FHg{~>wrzCWv2EM-oBKY``#;YaV}00TtopWpbFMk7>bmM}{*-eNg)mLg!Andw zfQRe5VvTkj6Yr8!$Yp|gjv7pt>rEQ9L%z{pS0i#W_xE8}odHd2qEh!Yasb>3oqrya zU%15?J4h=v>Vt)?T{Q@XQV5Y(N* z+UQ6+6&sGO9Fr6`z&_hZXMLExIVf=kcYg2cc#T;m@Da=d7fe1;>cbugT%{N6X!S3kw0u$M9hl26LFm{aO zQU6FzTD*HWoQNFm@d3Oc{KnK$O|)+a25J$i3~HTFxKU1ObQN0H4}}<#SX$zdH0qni zqsGNO`2Y<6lHm+;@-nH4Mx2+Cdv%<$bZB&a3_U7i8s#^1+oZ~7^BUW_`{n5PBc@aN z(9QNOmgsmb%;*khE=PbBP_XV!xqC;^%3kDE%`4mk zyVRK#KC?Bfx z0u6vTHo^`WhqAY8*k3JA=tWbUl=%%pgxiqS(<|xycIaaQ)y7Ugmrj4>MZ&Y#SqrX_ zqzy`K$YU=199qrrwEw2r^RQvH4=@t=s~T8XrC8|yqOSYbvo*$<<@4XW;Q#x}|Ic0U zpKS-t&ByNv_?wa4|BN&KcW?5wfj~)_^rqwj4Fr>Kf%Tyr1jhNF$bSO=JA3f&tx!=A z?I*~;!;x`}Wa(VLsf&V0$zzAdOqq6tK>p4qup^~>g>gZo@MyC_0Zwe@1du)#T?|mx zSpq9Et%-sLi0qZBXFK8!2st0~aZtGH@uG)!#mm4@5dR1hnOH-btxgv= zH#Tr{Z?QKpH@w{~wGaSd)Flr>ua8eFj1mXb!|=j_qvy4eJ(g3$p~DBSi`Dd7XROh$ z|M42bZltIl`!HF81AyQ&loEJ1Y#lX+bHknF&3KL7z@>!jnD(!-K~J-$*ZmwKr!o3r z!!1}hsjy-@j#@G_?dgwEt2*r57XsHb3)Xsz5iomi6>i;r(E5Jj3->PDFh?evcc5Y7 zI(Cm&Czluw6{G)hEEn#nELvBdERL)8w^qZViOPvlhv6O>ik znfRx0^98(D1iy>O%w*)Rc@ZuaG&@aU{E~rl{Du2gGFmpUes6f3V%DEKe&NQ!MSHVl zxzADF#}9HyuYh871i}7?YlnkJYL!Z+aeRlZ<7C-bwcsGoa$Xd5m6j&xc7uB#C@;D3 zRURwEpERdqyTE~vEIu!Pb+Hc-xS=;P?D-T+h8cuyi$y+!RUvQPXCw>bB*mk^Pq{R$ zr|+oZcxu#L2v3r9{=T|W(m&`Fy2O5>C}tKh(u^C$*MIw_H!g8&{@>E7sR$mQTU(E3^9;rkf73dwJvgse9h2PtgJZZ5{wt z3DjFp;Y+9BE4J*5Y7h}@C*<~w_2ofN>0KiHD|r`C_^gNgS|`R2h#UG!nvp*-#rfco zAo9?`u}x%TwA3bTLX~tB z4Psb59m5oJ5?CK(HHOhdLFnw}z6Clj9G z??ge5sKi_otH(YDz|ci~NfNZ#HPOfy70@h#lAIwQq>fq(=uBTS>r~rAAn)SGXOH); zv!gaHOa-YMI4+GdvLIkUy|5?A>aO8UU^Bch;jt#&+xgWY!?0Iy@!>M;i*oF5MvcsA zT$ol4DHl8yDB9{R;XLx26q$fgJZ({SM@%lT&ZTlA-ctTI;CD(|t;lO-9UmFl?Y9gj z?b76ym33=k|4Lc}zIYUv+2(va6Q3>qt*@W)ejbjLcxQQIr;y6JV=jkJ-@vxwu#p8L zYm3F2qW-zO_(VX1NFEp>1bh(+?9J7h7ngBp)`WQT;C{GIp{#?S0YeVQ-mD?-+CWB= z8Z``y$}0K_pifO86+%)1(Qjz6RoggpX7_1lw_j<8T)8^0tpF`7b{iQtzSR4QO={Ad znzqVnGLSU*r)ogvL1gw2yEGq$B<=neIj<9OLNxghW67+{!?2@=zllY;BD$;*ZZhc* zp0WNNdRga;v2LYx@EX&J%q&poc(Fut^X`|Yw2~+h@(Z>~L@1dDaAf4@)K{2}B7*R}Yx- zzK3-l0gkE-ti#K0Jk}Q7c>4sZuaiZ7e_0*0=+e*m0nr1HmrPZ6_Md zU;>#Rrpp{nQg5ayGa)tNa|$DzEQT6NYfl0~0XU_EVnTJZAD9)12(h5uHy>8r)5>cz zGl)nFH^N*JAWc^Ah|XjnF1ZC!7af(5K3}#TC}^dyuTH;=TNsPKoZQAR{An)IXAX_~ z;K_W$lH9!}lg0thd*eoo*DS16b=D~}B0#VZmx?UcuBJ)rS8k_roX$zjW$!se)5kmu z3FtpX(CqeLr96MIdrz}j(P(p@Sy@o3Ow+WCs!X!3OcJ}rY`WH-)SFppQfA=_6dNg) z7?)G^7Y>tQGO%JMgVczX|DCj~T0(KM$K{>KS3DPT%$PMfX9!j6E>}4R%~T{I!5Hk% zevC%;$0pN!N-HwXjy(CXJ0IM*K#qi^9NDPmH~T}zFJ9ldWu z^)FI2CwnBvalRMx59X^JO`+y?*%B7-;4)`~lbFf1JS-21a94?&TC^j5*v;M#ENMYUoUyyMNvf#tww&YXL!t3{{u(Z+gC0qc>_ zf;-|By8oee$}bvjCeiziR>^NWEbcLctfw0t|L+c$@ahEk?(#Xu$o_=`$|}YrY}^D6 z1XM&^>d5}qV@^YIrZTbqw3P1_e1K%3L50&MJvktpZl{>qYwF}4PG|PT7Y*_xe(34g zD-?08&aGoe+=*{=*okkFqf}iaMYf@fGZjNF&6qMv#Orb*wXT$>%HpCz1WI~2hE1rx zLGegGS^|0D<~J5{$>0o6?{#y6m(&p!!Q6A>q`erHn_u+_4|%XhaCWjcAAt7rgROX4 z`@ZU?i;Dt#Nv7e*eAr7;x_1jIr`b4FZT!^ZkFoCYb{}bwqf8+=v$zT6^jo|*R_cy1 z`A)lQwQJy#W%ansidafVhrzsr7PhYv%6axg1Sy;x{C0NbfXdsB;Z&95gmcVq2gzT$ z(yeC;o^!C%T1r@_RQ8T*WB@YYla^Tm8cXwxxqkfJy&cl`wjy)Ft%437M|Ta_8r~!l zLBZZu#%}I6gf{bFIHOZCC5zU~UTK3D|d{JzJO&9+{B=Q8DQolw%vSP*Sy)x1BZv;R>U$w#@OZ0>i0aYEs^rFv<~_z(srfi0 z{Um4U3UowD6d)8)tqYmPBSwB&(q$EMSoBIRbF_epA6n3^b4ilDAa$YT@_{Q_$~F+~ zr%W4yvzhqaB&bCCUTmX(U0Df$)ul$lttgvSv>SdGSn5A8FfG8Y<#h>L*^0T`sToH^dlj&L; z*tLfc^Xsp>DRcpYV(n3Eb0Fm7@#i>H4YeByn;9V(U!i{ z{2sh%8G0Ezda@Bj=$E9vn$`_Wq%O`?Zy;=CG3*QAq9fvK#$&jsfE$f_m$*lW(ihft z`GPlA57=S^^LyzIR*$n>F~m;bd0$H;odGQRFX-G~vy5w>mPycIQkvu^!6_$0rg)H0 zrJsr^Uw}l8&j7UfWZRa52(9dbTd$o|>DFcBPh*d1j7>~V&M^tnjTpD z1FawRdPRTsjMFUYw-1zTr{a&~0TBnx_hb8@1AOl1_YiBiGVDTb6@|cnI}T7AiMMJt z_x!1y-%sQGf!L2g-)0bP&LtSt^{%=68|TCwwp4~k-vQM2j%2+X$k zRs^sTdp1;EX_^Hd_fIeGVZC69W#_uz7itzFJ01dGve#PA9Dq4x=QWL7V8ulM-jN}vO0R=RhCV7PWh&AD4RR|2bdyJdhvuO!C@6uPP zmt~oMfzLu}3EIdaU-VQuX$6a_uc_duM0N&md;q`d_5Cd=i`NbO>N%e-6N!D*dCgT) z$r7%dBsZb4pvcz`BTjJMe2_y^In>O!>d6UOCU6(&tE|VQe98!__fD?o{+2Yk(*WM? zD0_T2y22kZ^UmpeG>id%6(0kd>oGOnI*M|;X||KwUO%auKUwbqs3dwJhCZMlTMk$Y zt@mAjcbX~Hw>%TO54?<7@8z_FC#I)8W~Qt1iMq(Zs?c%TV`=Pmb4a zdp3JdxobaN^CGS~oaWE3&0L%se*i+x>|3HSEbil^oV?IVCU)=ft(I&ZPghrx59U`f z+J(IQ4qJ2zWVZuV4%s5+4|wGG1zv>hJm52ZL}c%k2g?RQob6G_&ueLv&a(w_=u>pi zy{;r*TPH51v64ky8nugTinFT9Pw!j0grrYDY7~k{1zV*18dt>g*v5)6%>Z#Yu#!Cs z$Z#6PImBE3&VeyQGD+GZcgy0==OqV!F<09H7wxI651Cdc4=jYwe&So;7W(k&+nR-R z5k+~#Hh3%G4M*L3J|C!SRSnCIY@bgl@I&4sdx7_ywm{kxL<)e&j2~*>(x7-3lSqBZ1kq-m&SmXhLLne^6iiZWsEX#ki5G zh#$~4-tS6v90os)2Ya6kSTX~$jp|L$LT#*Y_IHt=&X!zV)6mw_z5?Xr=0UH9ymv4I zamwzWf=Uc|*Lp%a_iq%j2^Qh5G2wlkiAssDOZgzgOQ; zBk;Mw;vGJ)^tHr?eS*CzcZN+~TT9VyOyBEMmmD3tyB(WUy3$WJ{^+(BSKYhDqM44O zDMI&4IFYKbEjIXo*8~_IEucOR$Fuc3k@7q`Q)=EG69|u1ol3@_L3vKK=pnK#a~s0! zr(CseV$QyMHCl62ubH;Sf?bp+07tJ(Hen51a)~!3Z`_dkJckc}effUnOG&YOy9_$+ z@o)JhL74IFsSX4{!F81RJ+-Upo=d?=|83FhVyS(FZ7uV*cmUXVJ_wBEOLE2bB!H@n zL+be8*+ruN;pIoDl&@HfcvSQV+lRldze94N`S{KEtWWvz*QrST;ctucD42cQIlC{!gd($e$_&fagS&v)OB~ z?toxS7A|wja}V%*7=8!7FTSE)D8KQR-kurJ=~p=~;ONT&q|uTr1|V+1AnZ9_Z?=(G z|6B_2TE|5>+L6cO*C`i6sp7*_>Q>qwj^qD}$W>PyrpUDVJIbWpO%H1H6Ph25BsTW7S1@8cr?9AzK2tOhC- zMEC$v%)C<@a>c-6l5XMvs!_?3W13h(3rfo!e%xmh#DFt0c?hTiPN|m4*Rfd12v+MJPde50_1W ziT)l{NOQY-nk;Bn`Z6R{PsB8yt_HZ@`aRJUyI2VP zl?bg72wkIjwx-N zw0G*NWZm`3^C{c$a%C#b>!3&P83mbEDFx#BM>V5cQ*RSrHLL;ve+lwfO_!v96{uzp zV#VIZcWUneg8j9XN&vjGH)aagcvAW2nDEuX~ z0=6EJtpU@(fwK@GcoSrEumgN03H%;%6E(NVD`|X_sKIsA&e?ZsaOrQIEABhK`-u%e>kD9u19|NC6QHIWbiQtYaJDPNGhMgVFSnSV{3Hzg zG~;dR!QKw{j!fr>(+;0rzE^v1#xK6TkTB1JJ`xy+xa&uIeL&Vn@O;_S13bk21t5=% z0R)H&i8Ykl6ddMuJ-v{iXe~=2U_+Q-L;Bd+i8~Q}{`BbA$UaiRCGbAv0_t~)y&sk1 z*|67lkLRfXD>uPMfJUyjXf$A9SoQY8vYvx3<+**lo2bEd^jr%9z%9Uj>ngINdw=5^ z1WNz%?CJ1zh5HUl@HM?nfclin>~v{fySCd|TKePlb#)PbLW>Q+W@~0yT48HKt&m1GPezMxy=6{Tek!15`g8t5L9+sJ+f1-!p=%+0uqDKES7y)ZxGT&~UDdD@ za_a*?yE7?4IfL7fN{93AT;vEZVEad5ZU(tw_XgcOV=-heEJ=fI?Rb}GYPG=t{S!1l zw5daMZgif*0wKv{CDmT_&*Qup_s=#R4g=N)n9IfKMLOwLY->d4rXT}-4Q_huuW+kY zptah(&rz2O3gl=pppm7$-zGS7@lZ`yUPd&?rezFHBF@9G7oI!h1L)@OEC3KBG0_E=I66BDC{ntqN0 znytZ@pMv&SFT$?rq{r@w$rrlu^bsEkq79Y7-s`w(QNpttWE`c%N?Z)v2f}~6>HT3@ z;Drpg%8=uPbZbmqz?1z2+HuF7U(t_Dn4=bdmRP$WBi`pEWdjGi^m`BDD{-V287QfT z=yzQXBp7i5L`Zf?OxZ#g~t#k;Kb|(6wKN!J^C1DS;ng)V)8t6F=+3ijLfY3 zB7BG_x(;u~gl2vQiZD+O)|Wil$ZZkpQF}8lM;I(?V14wgDn1e-P3(x z=VRu%z4(?*|03QtS6}2Lw-4}YoGA9Yw2^^KiRmr(nPeQWND~!?+^^`$)JbaWcGePb z+hbX)s7EobAAVtMFO`4kDri;<03!qrlX0mFidW|#nByCSg!-a^^uL{ST{|7^Uyd6DiR^(`pd61C`Soh74-Ur=eZs^3v zo81C_R12KWk`T8#Y!OpmJ1a&uwu_&Q%0~OsBJ5zerzV7wAEj0Z3W!Drp>C77?eYKh zIwp{{01CjJ)v&B>!`y6Kvw$7${|jzG9A`#cc*nm~V_w2%2jNEEDEzCAMFzGIBtFwh zSY(rn?j`YlnkAMj43(eP(!&Z9x89skFq>7zfIl|B#_G!9v{E1f<1D?FlubDpL*)*< zs<(&z1GcC%)J6(>db}4g{CydWKV$MoUYNyMxX@C*uA_(phbp;Y z&@|an1;BnX=LBiu^egs}2<>q9=bv7r{b~Nx@l>@&M=+uUflb|`V#mvJ7LE?m{1safaRhUZofC>@fHNC8pdaICNZjX`#V%pLf4T9H(8uXr;z@)Wbx%jpM zL^z6Z;TP+q3Se&mOc}jAtT=suKBHVhmTdtoEi2#uubV8ZD#qNr(_d0oU#0~(ZSkyf zauQBlo%Wy&5clwO^I3a*NF~*}GCAc2rB}{ne;HP-2|we=%;Yo+y5|8tR>bC5k}{@+ z6aP+;U?uEPm1&iO@;;R$i|!Hf0De^qKZPa*lZaHQ5I}BqLvv6p7JLy<2?nUU$&t0Q zZL4&PfAJ%N8Nj|JYXO5oAwwnL+XV$XSkt1Md4Vb6vamy@Nn16;BItO`e#RG56jQCd zK?Z*+L8eJ6E6UBF+)*W}GYy(%B8iOXIMt%5Jjr`bnXLfY>h$7tHB=kfZ*ar9oJ*Cy zA18rN7?2}t-V!ZvCUkQVm4`1}#k#=t*awe@5JE+Cipg5{%PgjoRW*QXN#zLoq9_-y zUk1B4!=lSVq>xE9;dlljZetwTAY(*%K}h>_oagqvVzUQC+KO**X+#>eTrrMm zIZxT463o@M514_u31g59rPi$Pbr!Q=;A~{9V(v3ux>&DiR5dcxD%m;wj**7Pv}^nM z3SbV%Tn(WFBmg&TdzlVaj@Trnn8}puYCj_jbY&@){IWbglR$iFoW?$EhG&6}6y>yh zbs1gCDq^TTPJ~ErR3wToZe*?qJN-sq4FSBqMaS25d4Hu`tGhyMPKBVF24K~S zx6t6=8Y5ox(PG947;Z$$$ z;D82qG}l1^rcr_ZGkyiMUt0~cj}=y?Y+FYmHS`K>#~SCZUnawWDaeUVxFq+)Sf2v1 zzVj2|7 z#4joOONYd2-+f5+K3@HcL5d`!2O7h|O}T`{Wr^^oCObIj5<*we{TR5=ApmO2Kr5z^ zshb?L=>4@t9b#-Y6^#`(W}OS-WmB=ER!61@>{XRSM-i21Q%(macO+6u7b_8rXA0ch z2Ofl>adQr8u)WVqUN-*3$*7f}>tC7@ZH>51oWFK69yzUM*PQs5mx9kpTTbah8wJO+ z9f0UA z+y28nITHtqM2z^mpE$=x{Hx>wrVHxp2o;rSvtramo?C)5S*-{0D}b@U$W7x=OZ`5W z6x`mUq}sxb%Ro_^RL^H8$;}U~6o1AXaqbp5(S(C9YRZaP$~o@hcQijxdd8%*b>xb* zjfdog9R|Ov%7{?21+%BW5bTLsdsOOUd`httZWPp%{2bn^JHVDiCPw_LBKNm3X!OdQ@U{YIDVcu zuzVnFnPjNEmzmhcz} z9atoVw!~PS#jo^P@r!yO|+?yvel{`_9 zl%~GF13z(n8%AuicpLl&j4^p^Qn1&upI7%HM}9sN-P66Y{)&EjYa1L6=Lt0RXyOnB z;Z#N*SxX_Z?mjVfKE+8y`ZfIwGUU5gSFi!URKlCK8Q|r0W0R%KO}lJeR-V(eBC}z$ zez=VC?mEjnr0a60uRtyhP_X;N< z*s=T1P>ki&Bv{hsq+52&9_qQwWkv*+COgD;TkGhsYL~!=VloGOJx0e)8gfn3n7omH7oY^d@oUWyD0K&s7P9++!%`ti9uEV2*E388;?PdPquwH`<|<;snI?Z^*rH~(n)AR!bWe$N1QN$g)Pj3n^MSY5DX6*(@s%r z?;N=Qh~%o)Ap7Y}?~ge1^~dQ8;;}n2^cs=!fZrUr3k2#*?DsojmUsk7eenX5NVPqfOm3WEBKu2M`ebZ1!_Pt@lGzfYl1xg~h4 zixtn|5TxZLe&p=lItF|1`O0j=1qt#A`=Q$L_GZ79@~5V!4Th9K*X4Zx9$=LD=K!XW z>I3f{=N>WO(|nqS@LBKq$u`q1Y93dm#(MH^{S0HbBv#0lBM4aP3LlDjzwDAZH|GbU zS`CcPjCD79%KgrV^MSnbhAuT7ApA6UU{jbdI(@$_cqe&Djf4*)@eGsu&?# z*gjkQY8RynQ{7M0aGOi7gq^U97^ei!06)_-JGm4xJwe=7NL_@!8sPY_%XNGRO}laS z#}cBSIu?D+bSzi-ab@sNh23I=Rh!D0E?*aHx>>)pdFT)#@1q5Ln&>B|ZMe*inVvgn zN`}AHi=J(0Dlr#T;KeSICVq-?B=@`2#0uR&41 zyqBm3_;V+6&SXf!382J;v(AN{7{!H6tOfNLsTWgQIQb1o`Vg!+ z1bwk+XJYbQT4Q($@S}s~7)@!Mx*|=Ea7J73PhDBX>LML&ae(?QPJ8xL5%)6>?Sb=l zv*aBIfm)vLo{C#=kC0LY;nkKwF(u58$Bv2ERnTb&XmIc9&bJ@#B}T=rbp##f3~68F z6ATyb^_HDe}lSBXBIyxrb zY1L=8)Mp*tFzCBD@OKCa@q(HVdId2Nol8jf&01j71;YM@w~PlpgBV6KG5cjoVSXkE zhHmC4uBQwRa?2*tAI?{zO(*h@uC~Q=o@K~_F(o87EdsaMm|Xt-rxkZCkV6887oVV@oU1&30c=0&3F0pkWuni7Jai>l|7t*$0vUT+&Ah-e^%>{YIEj9m|mseiFh~EHCH<=)xrf@Mr^3pNVKZGh89{NqheVRk0 z%e|4xOu{Xpkt!8{HdD14Vmj?AGFzb|li|A{cLcDqhVgi{%UHO#N9zv+cs@9i=b!p? zHf60Dvw-uoL!UY`|5&GG@v{|PUX^BkkxQR&C6eugOk3+}On>U_^^RV=^Jx#gDq!Fj zK;{$ZcoA$*CMd>ji{YDyAx8HADVsKQBH)*cvF|zjAlu&d`b{a;dZ)z8di}GI9wJLL zYZB1$_M01#a3T(d?PQelKhIwYlYB({r5E#7sN!I?I}_?u29rMAzvftemhHWcaki0J z9w6PSrsO9wIuHmM9 z>%@1kv;9onFtT?z-GU9j#HSRvA_1n$3GP#S1&?q={qDgu%sbuB7(Bu!cG>z+!35pl zQwr&~aPQKUpwBK&U{gSAPkbaM?>t?uwPi<;cR3{z3|~k(LG|NEG=LBm+oiwfS@A>K zI#KmwDTS_NC+S@(2o%bMW3K%2sBu|x74s3KU68$+#aBrISB9$qVk(6ibii*zyv(cb zaYPHcpy=~6xmyo7yX0u)L6DMI^C9Vy*(aAfnA3H7y}a)@mpd(Xh&=(G&m{G${ED1k z*lFi_$5x*-ReL>(%sr77Cs&ODUxMm){3oFAFy4V*qU<*nPOv>OA5)H$R*Q9p{~4#OE^oQ?uc#h{546 ziS*81wP9qi?)0eZKKy%=Y!%4f|FkZZ3J|H6w|E2Z^8z@cUIl{1sP0Hv%138PU#i8r zLY;xTFL3h}lgbt#Ll$b9iC&Y+G^HI%)fkQg;C&>-k@$&2w;s9P=h%4C2~Z<;O746VxksY6DrK+&D6zGtPvPxD~;L{ zdCnGe&gJfC>SRty)Flj)mM-C;Ev@k3eQUN;0zDZl2q)wAEh=lPTUb!{j7Vb=_`f7(zMCJIzn$m_h zKULsaB(tQM=RIVHP<*q%$fcofrgPZ}q^D&<2-8-4ICxXt_5rqn7u7176waFb-W&z} zMd>3In)R+q?$BLMcMB!*8c z0daO1zo4!R|~)>qFzWxxNe-Timj?_UZ2j0?RY=)VQN+6Oy@;BWO#DoBdp1vX}KItNXQAUJL= z@W1+t1uV^noA0+>|50<~Zht#~&|$Mf0+JH6Z08uzhQDlt`|Bf$+QJ*(W0#SIOH$H)Nn7#{8T@HV22gJv zW3nCfksH22FO|i+8#E@7c;@8kLuw#9V=aCz^2J}dl_3<;*Xxuq(PkYB7Bj!a?p&OX zmw(AxTSxRt%TQyd^MdfuY&{u&raz%y{Cteuwum@NYoN%;rU@ciC)LcR)J2OQf6!o# zyi_uN=$vP%Za?m&Q9cynoBZMR0MtV@bMK@V9CZdOtB!N0_5|71`MAo>ytV6iW zdMZ^Xm^~!(xChbVRBu&gu+L2KkP~U`*bzR)}2f1@9^{HEB!0w!zXpzeQjwgzl#q*v!yWgP5 zKt;j%<6ptUe~7^QM&iGzg8BA%@Tw<;`}*!@(1%Qk!^8Qu$ZUg0O2H@Lf@?AKpvMM9 z_$OxDC%6#kTONc2l|sbN`k$Ds8Dhv5JV;zhFtC4OTCB<`1)#z5zBLprRExwVAW-?= zKUrMiD^mE@iNyidmfvj=K5Vur1D3d!O-P)@dPB63ubkWUb=#lsp>B#^*q)+fiATbQR9nBP8zmz05?^pENS$2j+EGfB?E zgVo`+=eFB1*c*Q_+Rgo;{x)m(J`Z|&ZVSc?UA&gnEA`%d8e!gwMR@YmcE^#=WZnqn zt_fouQXvrTrU@BEW|jYHq25I&c%4zk7Rz%{4Uljr&QvL6i+K)3)nc{t!uIIZR~7+1 z!KsB?XR8AQJEI=`PN&Q_+C^Z)N_GytxZ9q#8M&Osp6A)=P%SECvRBvZC>bP|f?LaO z+|ym|@zDP&Ahe9=t%USb=-@*X-YDrytli*8krTQ2yy}Fr1Pr}<)1@Ilg{&@9CL`dO zMtBUwR$M*ccUraJ&|`(0A~F{`bosvByr1o{97qDvS2zxIOEfE7xE4m26|=cuXc}|e zSJl;i%^J>P)Tj3TJmD0&iY0|D^#bMQ-?qzI+xw@-XoZ7V^(Otbd#oN1nWrEEZ%uU+vo61BD znsq}PMm_%3pvEC>&25s=2pxY{l7CMHvWz%HL?D9?st9BV5UG%5=qe$N#O3+7^Rz5i zw$$}yih2VE3$ubdt&b1mFYVek*CPLJqm?Ry?d{8KIAhb!_5PxOq*_dHiTlYI%o$?`exIyA@7*#}jUpy(R0f(zU3W zrv7SxRfMa+>cvBc=P23*J5`zAD-0J~k1|jhUv^>c^m1`2do1iHfvB zNlo#6xnfU9sbW*eJeezmzBKtzulpt~vr01SZBDtl<<1a0h$A4q_*R$?P5bOg(o@Jj zxH+Ent_U3PtLe-Y@_Xw*VwOPuf=6DiaukPgxpADBqI|jH^jZ3Ov3;` z*C33Yq!cyNeoOapJ^?>BUlUhHG$4`Nec5u>ujL*3om}Edb%-?{0%E!GNXfkK6)bVN zJ{aid%6~c&9ZZ=Ud1cw$BY_gisX-H<|BqA9|E{zD{zCk-0_Wx1H?(KGJKezEL&N3 zV&#qfax{Zx@pM}Jbqxz6(aZqBb_~LSo8D1oTidr>_zQZQGn-{+r#i-S zuGRldmgV}-aCl4zoUWd^aSPU><_?AZPdDOw{1~g^dtKY}kt!*-(n!EXQwZFp3yySf zn}YcZ9shFDO7ybq-v*>&k~lcgVwetxb{9SD;NQ7TSnXAhfwu+q{uUXE>82zvV(8;z59&5(>+^F<9*Kp)_>XF< zA1seIlz$8>A+8w*Yu!05Nn;$!s!&D_-hr$2^L&399?NXHLVV^Jy(sR7RV^_zk_VZl zG9J};z;%;5R%6#Gg?JUBGy`k85PhAp3p2$;r=<5{x<{v>v=HyO$R7x98CE2_u+sN z83>3ug(w#dKSk>=1U5i@OBq)c?b8M&9tp*)R3O&NPQE^ULC^J)Fg`)JK` zO4sM>?nn=K{ST(~Gmzsh8$oZ#?h9cs)GZ@vY|M567REJ$e+(eVQE@$apbDv%N!YEg z7S1((=S^j^wgJO+BcQT>thsG0%W8(&)U@)}zqvmpk=_-nxk^JcA>XOv?}O&KK~RcL zf3pLwDHxsohD&%`f+FmCo}Tg`F?=Fyb$?Tqwnq{Xi>x^o*KP11CN+1)y2c`#?5)$G zwGOj7>o8JJ5;I`m2G=3ZW&ZoB!9r*lZAh&ao_bYK%;l=QKa9- zj225P`B3f0U7uQ|3P;;>Ckc970H|Qt2)lyH%Q&!!1+i!^h}utn3&DsMVZt zDAX{cqg??XT|IL+ioBmL!d9qPljfv{o;HTSvc4`@2j-q7Mh+=c7SZm1VjtU#X3ys5 z=ljUwxT0=lii~jD4{xdF8lM80Q&ALY+ly@z*tQBk{fr-P74x;`!coWkPStUPF;fyv z_sD*YhM}eC)O9zW^IArCLpQu}(T`kJ0Fl#D8_xoEPqFLZF54=cQ$KeO4PUV&{Zo{N zpZ`CuzA-qnuIW0N*v`bZtqCT!ZQD*JIGH39+qP}n6Wg}!%p_m#`+e%gS9N~us=n&_ zb9S%YXLawjPRM=VuH52CGq5nHa)BXOx51egAK~+ZEHpV2Qhyn6TE~jrc&;Be)cAFO ze5H3a`hVbEwDd8p&kHj7M{@D^AHgsE@f&o6YULVYA`pkQ&y70A=GIT@%mYNg$e^IBejH<|&~Yd zTwm2mL2temm*eB-Y|1Szh2;Ip@(D=IJu(=0;Qk`=2g2W4uud5Ma}ITywShOc1SHPo zN+je`-7!?}1NYt=&1WA>tUUD^mO-k3KkS$U&#*IW$pUDqX}jU^pKXgE7)3`u6RcgK zOhW06Ue-B5bZvGC&L;K_!HefsQjydmnN-c^8*oFa)n$Q`C{MAYKca;nn?%S*VhH2Z86`4H8hbB6#)m2y{huX9sBbFpJt;*k z4yc$@9uQ#_Ab!Qw!sU)wkR*KI5@8OBz>EHYmzQ;~2ydw2ge9nk!6!aLLKA}m-OCgB z4K=r%=ad+{yx$ZLC10?Zr%$6{}zs#l)2?<(uimOR{k)iSPY!ey= zrIYgOJ>A9zad$-Og?pm9NRaXh4}r0Q`SiDdIk^xJtw7mz(!B|#0Vl`qf1~*=8-@u| z8GxOdQ>obb3SOz_k6xteuQ*7eW_p8uGsLcuO#J?1B zwhPhYc2IxJtrBS|*IMKx8YKQWlRT6tMu_K1RXIbt7I2~8^0@(BvReIBb#S{vxbIW? zHV$6_E(Fg&c$neyxRPuSm}kfTxnFV@0O5+Rq(aq*Fob55wvM8`i_gHbZuJ)bc#g?0-jg1TA1KItX8GB*<~HchG-5n=Q;S6z=~9&lgpa!s?Pi%4aX>2WpJVv^~dQ?D!dmRZV_HiqQ||9FgK;h*nZ z*mG5;RV$7D4i*^CuaD@4L`*PU< z%DP^zyMKxJhWB`V!UZQ(WsGh`HtPnpeA=lp5cKHcDPpS=JYF!}=~@vJ%IMMz0TT5U zhPZ$gex`sfqqV-@pAx4LOYQV-bX*#jgXgaZ=UO2rOp{b*j4z zmY7QTtId4OwL*oL2?KTHn(_pBfV*Da;tlWQamqkDPmrV3FqYgIR&PnrlZ(x6Qk|b#yN7ZoDoWfLRP;or>uzv=7L^m*C{i} z?opADy+@SX!o4QyJ1V^(kd9t)<^TDKeu1mdNoP6!C5x;!h*vUCg;#P{jal4@Wm(#o zzlJa$HUY#ds4T}zG+*QoyJf)x#vXHn?mZK)5E5wgWeiD2IF_(45F~N{^MlZEgh+9t zyoWV?h1hP7wCVM2Bukw$gB z*Klt&L6HK19zycX707!*!Y^LtSE=8Q*2gklUe4aA{emqP93aY7pDk3V+fTRCqFte? zURo~qDEj9dt_96*N`|+BX9KlNRwS{aTNZ_Wm4-%(V!F9d(5XIF{+6*(NeAQ$cnqoV z+^}e-jL93v^_bApE5RV`hspd6TWK4Q?w{COuG83ypTY0wCczCyTa7=jli!Ql~ zPOz-NSQ~3Cq}BC-!a4rz5KaF(E=sH2N4dI&}an0<}d3* zW_~nQph7kfEdlND)={X`RX}uJ7q-lvqBd;ext6t!A4L;ZQ(@>WT*!HccET~+>l646 zSWu4el8HOIymBnd{D;djZj@*S^SZXk?G?-+rieMF$VYt+@1E7^4*{zt#P;yo>{#{8 zzbnQ;Z{e&y#%QIjqq^6awRT#SkvNuDUG6z?~29H+nslgpnT2K zt%)6OvCn?NV-1@*B`9;j`6|9(|Lli<-tW%mw;b*NpdFlCh19173O{F;~rd2oy9Su^U~8EE#D431gkVeP=y>d%w+8k=M|W zQVW@c$RF@6E=TfEba6rdZvwDN8|~*LHU$zpOzaDMs?~a(RVQB=KRh9~@i2i=XeP@E z6+&;Bw-oC9kg2g0(fvJ{RE{Yv){{LQZ2j$yXgGLriWX)ea~4%INgO?Ck<6c!tO0;vY}yDQe`4t_H;<6IPI(YpnU6m9BYFiE)ndo0ukfRzP3oq)x zGWiL^=j;AZ*GD>MuT%zX1LR(wS$a&p2d^hSj%-jq52+7vGD2t2!yEPqw`5~4j?I*d z+~x<<6z=c$oNheB-k!ama&w!|N7#>?z+^{ub++JO#%- zx*A%%p5hB$u#%$vkuE5Xl3$Ap`kr`vQGE%FoTkA$nJ6M;EzQzqKm}kbCSKe2+CCY$222@W7C4 zQiS;XOWh*x7{IxBbw0;Y+ONZ-74P2zmoOUkh1_&}kzji<$~P>ZT7^!v22bKhz0mVN zzWkG@|2yga2e|)9Mvw~-M09*jeVe3TQPhloGm7v%l`e!qTmf?e_+M}4_We3^*750M4*IMN} zB%A2mRncJldJJ-!HCOfLKbh70*4S2RK`)>S##os1_D+&~W_C~f?qXA_j zOePvhl{Wpk>=Q^-w288`v(bh{-9zwu#lLKtNO>5 z5IZr=Ej*MAT*Qd@wN6rH#GBC?0;3~L(;$^nuu?q2Fffi<^xHY}92~Giey6aT2^n>c z+$~9IwU^+}<_UUa1~0vzC2mD6Vkh|dUXa*tJ3ieaCOI*b>tcmLs`@u#V1;SSFoh~x z1y3yBb!u?GFeQVoXi|yLMIoL!amOws?c-l>q|Y7ypAG;2Q+`kOoY0{+Li8AZI!5b_sE-T9V6MUE+Q3q`boA(H^>WN>#FEUM*| zmGp@oGqcpxiTniqi++B33~ic0wZeI=LTCOE7Aq<`)zGqYoa z!Y0RSa&!2}ckAh{^J#MJs{0nlPxjfG8|Q|3^ZYI!<_$3A-3<U6+>4E&PL$Kdyd3b;D3M4l7hkB6_FFbBqAE~j-Z1EF z-65-YEBg(vSY}*QWad_yG)!8rNrbY*njp}?PDyy%gDhmn`CCagQTTo>aJQa|M(@1u$Kd9U$oRA2_c5B3nH$lo*AOXp0oUm z9fW%5gQLOsVyh%v)C=1AmbW@dTb{0CAGdc%-_lA^6kHEBX`lJ!Mm@B8k*zdGP6y+&_*QQtJ?BRV zP(QQ2lMsE^Jm*~2NV=8t%&7eJnB&)8y#N79FO8uvFOkMgdcf>fG+)vGVo!VOhf-7Y zotr*q5|b03p7^O_Uip%#EhF~jW*pvg2S?`q7 zQX?bEQEG12WJ!*~;?3&l_8M`umFQ_Jto{mGe)Q#tN7wW=B<*)?mtfn1&`i)W38Js# z+PH+{W?RhgcSws&ZW!fnBN>w_VJs$S#Rbf4(Z{4!DGw&L&0;rR?1!P`Q}11m)N#+t^u zfk1J##63&Rf?%I+Q0b*E3Qa|cP_w%hx3DrkVkEZ1=3wuOK`l~s5V_)fzEzZEq_c{~ zxT0&=k2&v=uWt?|+N0mI)uW{Hu3|k31W-=SFQb%yB;8Xn)*x@VAS(k&@KGmxcgzTu zSR~ow9#L=l7B(3y!P0gcnTMu*f?U78#*b;sZ) zL(t>XX9ZusH=LlZDHv>Z;MJ=PU>RbwrUP7Cdh289_WD_JNX@cY#PQ)5z`8StK9tM@2KAK%vcrZSrMf}&cHk8i+@VV-T#4F9QgiT=W} z3qjWqbpjhlb}*A*0qVM_ExA5s_V-o~_{81#mUN3oG7*v%s0+VuyG7o|IkiF*cL^MM z`;qpj z=Us>jA26s8h59!VE2yIDpO3TOPc+vhP=88Rvy#5^Wo9|QSXN&wn=$0+!>E{Lb;=1p!CAg&`eYrhDvpYr5 zDOw!zvIDR`oZh@2<6C0uKG4Z-z-b2JOE9vhaC5DPyCHPo=jD;k3f$&}rmL5rz+1jM_T_hOfTH@e>%aDp#raRQKh6_@INNNS+ zoD_LtRb&ntR37~nX2UFvo)h?)={c^**w$KgxNHH8Xj{4&GMf)hu}d`@HKM!FgWsXn zW0sj46)FlGjH;^!ZFnr#_ee)pOY%mr)=NoqD|)Z1rZ6JFfK>?*KDJ!wHMCHZatVvF zQtLSdW7s>nty%cd97NElrO318wSKDl&YT8z0-Q`>EeQTGN_*aG>dygie zg)(m%Sud0`-HI;mH&0|fg57xjPEuF|2AaUeUuE2F*dKj+JLt~d039iU*vYWRPLtrI}C%s520^;l(h>z(V$HKxifq-gs`O*EacMm~Rvtc&Fw+n|(S+m}m}aT`G+H)=7G{m7 z#rBU&aXpXNiSwTfG!&<27|>y(gl8>VOUiP_vkN^w;RWO)4CB$MoAz03T4P%BWUUpI z>nZXjcd!=89Gg5dgq#OFg=JQ_6>YLIaLup7srxTZjiI+VFC3OQid)fSwe9Xx4x$dH z9M@32)nfF3Zuldn0D>|ts}d&*aw9CHY~$;}i-vK#zM5wV2G-up1P((~Gv~fV_PdNj ztXrM$uYidN{ArA-Bc+{H)!50hZe_l%7j;h&6vmWka-w}*{rXIxp|<00vv{GoYD8;IMow4;)_KZENkDo`9J4$n$*Ih=Ni5umkKOUt8Ht)? zrb=(=;Q=2a*+oYFouF`1ou%r@lLYr^J*YINZStyp%3Nh`IbW#}Y*5$+bm{T;_OQ%j zl^doumGeGQn!c7iBdApj9o1UGqA++mh^8<)J(cYe2VyKam9>Dur~3-mg*x*Jv?f|e zGr%<~abkE{F;r9(ac;IA8>1eX=I}Z$EcqtPUtN_qXi23biu1GU+2Jhi9F%&vf1B*CBQ6ID6k{Aj@$16+}8W7$|i}{FB3hF{#5+e70WL`#FSGNiGxMrM^NJ zVS%kT0)F7ozEgxZ;Z)y5D#uMt+JAzN^~n`kOytH1N<4s0)Nd^;iQD(*o(_Ar%zs0t zwl!xj=cJ^56`Ww|PCDZz-djVV`LS?M9jOP-%EKfJ3!pV+ zl5Zy6s&Fs$FYpcv(r0#U)s?d2T(p=l-$S+YswJXyCrvofPE6y{#5Ssy^x-FEgXBJI z?Ys6(=BquPb*|b|`UN4>D z!;-9gP|slmM_>u^sV9iLqj!XKy&q&Z!xr73t{(=${_m6B%$utb!g~666xuHv${wrQ z(c2|oMe&A_3pdRA8c9BiQ8|7kef%qM{=Y%^6Q0rk;|c!HNTSmI?!w6@hdG~|U4qUJ z=>F4N+u$UWVM$e@(qNxyB3UIffST4?B??Puh(%{ARUr&crIq>C8qc0uGYjmcGy0rp zS%p^}e2yc!dS1%#fWt_ZjQ0QKT5;dkUGzD=|kMuvOZbJ3x^` z(9*;`lLV-Gmu}NyP*;>bqS}UR9+AyDM=@UT zOJ)DkC-Sb<*7MCjkjaBwHa5QoZ9Q`J-y%OtJ=8i{NCfKT8cx(fFVa+^PArGf+W?7% ztbokyqpdinu3-~jG@SkxSAk=G|3uT(k3z#$*mn3(*Ci6eujNhK+809vz!L$E7osME zs_vYF8^To5TNU#T;WB#QApbafGkrx`yLmXl0iU77O%+{GEWSaDx6ECun);tjj$7@9YYW8*j5=vqw){-tz0yMiOKGkFWqi%OHcG)SW^cQ_`3VOtN3580hGj z26)&?T$?XaJ8Yz{$9w!+i6N{~hADB1ZX$zVWN4P>6$Zh|x}B4@fb%?)6{VU=L#VAT z(B3$*66z9bkB}?$b1M(BMtY-AK>+LPY7OTS~fhKdtFKaw0?MoGXTz=u;j;c`f?T0cjL4HwxT*-G6W#p541 zhe4UWMbYFha`mTMYInMApcQcf=ceYa?{rckJA#<~yI?bS;l*cF7n^2NK;b17*?|65 zl1i#(6>bu11&o@+9{;x4rKGya&g}|J`-_lj9ve~%(gOxc{6J>+9cdG8UxrS%$c^Ef zX0kIjonPr}KdA-$R6u$(-;W5ktb63LTyZpNZyf5mcDmo^(lzENf+dGPrgq5q;pPV% zSg{0#Mbj8(XntKYV#@*lHa`9v06yF93OLeW|G@_LPiz=XN)VIy>}|Lqf`AAn?~J2k zCksGB(E;i!%G#=*Wm}v>x&xd`zG#nw4ullj2nD#7R=Hjr*gD6$fm48j zd!^UL*&-C->n<%1{h+DUn}K-w9>-n;qpO&C#p|}f^vtRIW#`WZSz$By-9T>WJx{>{`o%mg8W+Ich6 z8X2^);9zURG1bwfp=A^BmvZcfb1hZ!Z$a^-q`a4Bk%ON`O+1x%i~G6w`>H?A}dNNt}d3-fCh#v1)2TtaRPA1O|slJef~tJ5T=ic-Fy&K=~VoxB|Uj zdP?f~T@EsElzWaxw*ZW4FMF7 zE6N%kE68BW6za#^uBRxwC7Q=edNNg6CB2T)T?Yz5Vx84NOV4nq3#wlCWmI2~Dvx-E;P)Y2Al9=n92x^(J>pc$aU}jkuialc9 zafS5q(8HL?9+IR{5Af(w4;i-T@Ix40>OJZimEDkt;+u#-b5q{nBhwDQujF`%H>i3^ z*I^OrH)#<|H;|d2*u&RFfz2iz^d9Ncbon?0K}8>qi8HbZ<=yawoa0DAyMT2yLJ$U= z^xa9Xfpd7^N=o_!x)C|vAxs%pdg&p!Qx)ODioAi;DJHRN>4C%OWqG{&JF3jhJPd*T zWfJ&Tjb_-z`2v2QI$0U$G`8Ifqu<$@(I?_bl{wE?lyh-S{Yul0x&x}RM~Z;v4$6Fst1q#(YF;Db^*mqR_23Is-@c0 ztY4=0G!y+!j`m*$2X`q76TM$mjqtJTa%VskmrEFLJ>-9S%8^bf`@%LM!7o0{V}{v2 zVrQJw!LD$tV4p+6ar7WykrJn2ljhPzmmK@Es@4$yXoM{k=gW#WaIVP?%AdnInrpE} zRN|Di9o|`WL6qJc+y*Ee5IwDBP?J|MQZjji3!SNWBlE=G@WL{OH~WLQ>8xE6cqa2J z<_(qTi|Jo|Lof&p3j_HIAF-7SWW7T?gU)Ml-(P;4)l3oB-)Z!Bedm6Qt2%@$G1t${ z^oi7aM0l*Tcl7fKHhkn`djxYiMXD#ffqwhl(=Y#yPt@I6@)H2-TowTu$+Eqhm{1`G z#03pC!0@##62v8Brv(tSVa9&sQRQO9TdnafxM{5kdX*cte~Avv9BSUNzn8G+kV$t_ zl&O$-walX2J!_mWS`Y%!HA>VC4ESqZBx>%%p_fR>~|KIxDqQD%{=c0k`lZu&4fu4l)V-;_*Uc9#Ylqu ztHDY!I_%6dWKiWl|JsNB@8jlwNQ8e5qK^u(_>xa*0htB_gb*ZocN`rvnNbi5;y)u{ zg_@tTjR%bal&!4pW=?2Aec(ZoLL^xIF(7r|2*fx-Oub||Fa{uS@#BFJfvH@n1{S(B zV&y&yRvV?2(zd0TS$Skc3>vks6&suv?|O}0>T9c=uH;uhm)DWmNKhA4H}{rrSAF)~ zhWFjTjCqI!ihji$EkT-mHPCkA{aWgm4fEXUrw{#H=%)|!-01g#eyva7w_xsou^vYzq{XR?XYHS+}loA{sjHxxnRqoD#$C|*%4e(h0-B!^8-U8vl zo6#F|&jMlMS&~u=J@ToRaOecS-^QFlLDo&fXaCCG-``(HF0WP`0GqXAMmUv_rOzH2WQ|&d>qRl=-N&gdU@LdW7`TCJuJCYQTP_t*JFC6gP#%+IVJDgeh$q`TP>DSeULp1+UkG3&-S^Q_Ki1 zZ%4%F$^Wrya?-)Ow@ss>T?{jswN;yBEx%VppLbk^?PjuUaq(bb`Enh56FdCMY}B%Y ztcA8E=Nr~Wmx1jY`A!tL+FUJM-DUXWwhkBgF2{~UMSVSI2GFB=xKd<*q`Z2VxQqq+ zMAkOHys);kiMCE~X-c!s%YaRP2)UijcM?Qa*oryS*xq+DdAV#V+q;S1B3`6LLI~AN zuy7IPLg7$qZC(S|@}frtj7L><9V0r+o4MP(<*mj#ax`f^&AYe{^ZdYQ_UY_>w5wq8 z7`=pcbNMAa0mLp#iQ>4)u!}Avhqy3mV01Oedr;x?shfT}-mQ#ogw}X!Fz;SQ1oyK* zsZ0_m(wJ~t>!46c;gWXdg>%PWNrXwBg$BFJnvhvN-G;&oSFU>G?f6b7exqDo!;PXO zVH#0IA^)24vz`VulB_H>DM`*K!kE5H@O@$f?xLC)2llDISdPD=oidcDo*db?C-oE+x$?6~*N5q}tCfeCZ zQ&Q@6vjdnmE)@~PZ*G#~!5VS64Cmo#Uxka5EARf!S9|cr_QzB zK(pmIHf~tR!Ecz!u><`EAp+q7-ICz2o9>b<$c(K=Vvy`d`zZ}S1pS(3a#$8VY#c&3 zCmwEMN~ohqBJEhGMRj@+_TU;Ew%RYMDjNPZ38V3eJdC6>Q4})!cQ`m5$8q~T9}qJx z4SyPIEsV}lqTF+ItQOD|UN3`N3aJ9EPX6=yMK$0D49}hXt~=xg!#bZ_Fg#9yFb^Yp zi+?gPcC@i5q)%oR>oN0>nVgTDWcaouJrT)YL`n^L$M*`FiexG(N+zOTt!WCh<~i~k z%kMNHvnLY2B%SG)Uc**#SY*fiEdi$QSvG#nTV8W}R|yKU+|h%oJ4XIADqYO!>=?~4 zzQ+&9g4pIcPEZd|$U3}(e{~mhVcp1e?(9WK5zFGzXRu~FX0c;wNnJYakaN&An7+G5 z7)Xb?nAnCpVc5rGr&?$o+*Z@GzlO9Efv%$SPt4XULJ#(0*yYIN#O_FKCITok`+_K2 z8iy*noy7|lS23b>&62Pt4dmsyU~`j_qg*>{H!V^LI>-qx1%B5xyHp^Rkr=9UQcF@V zZfszk%SJo`?HN-zsc2wA-xXTT=<}THPDbZR>mtBBDd)^y&TixBD6yS0G9}h~17WX? z!D~G6D2BonS~FuUMx4Mp!ThV(F1`eUx zLk%NUmV&mvjh)QH^4$b31-%5BGz_?Q|=KRV{60ZT)7ev(cu- zw!|~_>ZeV1v(I`pph6bN zgTQX?Q`0==of$zp1Q{z>!M~o}@b+hR_4l8p8mFpV$mpHl9{4xbzsj0$?HRdS(iB$kmW{QIP~O+XVU(^gVhjZ|0$qiboIJp^?^O zb7>>dbkbIt#!R(v6SJ#m1MkpA```sP?F+f`NG;oOE3XPnZtcbqAC9h1xy@47ys?t4 ztSy9R$JaoWm-!lk?-C_T{ z1ZLNh`~~*PaFnpXzb9U0Iq8B&rXI5Nm}q&?0%@CpJRzOA?bkY$MBS&kYT=p%q*a(u zTfpITCVEc00f+nCN<*^R?2+O3r)h|LQRu&V+a7*)d5nPJgW$MtYM1}Vn<5DOjPS@Y zQvT&Djt}l0q0vS0J@j8p*IxY-BHjE1GXpb3gzu%;T&|E-v61_ri&AZfe9ctB{1B>f zr>R5Ke9U8P3uygP>U_-9vJV3zcx9A_2P%Bb8u;jT1!p(TGYBd4b#lH^sS%$2gO3Pn z4*kV50s#PCKhZ+rT=&U6cBY-li}*e0gyFki5mUEt{np^GqQ;E^oAdgjNJru-WTn-_ zNK4AK6s#o5o;+ueZoeNj1sjX;3f1r$L1FfPp^%uKlh;=)-mJQSz>U!;KVS1ZU_`^Zi;{Fdd3<0b9 z0~IKJC9;E5T$?z&(;sTdI9A#HZSx)59OD~^)@H7~zT+W<`@&IL+H=kpMJkE-TjEuR zb$h_U@R#>~m$dU4;n`P<^`=RoLd63=T5z?^Sot&|f`}f_tc;-AL zrN8l8wSFvPJpNm~@JsGrj#k!0us`7W>VEHl`F>$`KdLV%s;_B0LSPeBIK5FP(SzOn ztkMXB9I-ak&#O7M)Xne|gR)C}oud3oMi5y|^-cQN8>Y-InKdY3@oJOQq~ccl_F%k| zx7z$;h|$g4=5Ej?Unr-|K;{ns zbm&c7OV*y+?bH$A&}a+p%DGpuXH|<1ajiDW&$?8Oki7CWZDNf@oj_AHcmN$>EjIc# zwVKFg(0fDV5CX#TJNAvAn?3nS2fufMt)Y55cqzKDiV&<(0;tvZiotws98(Xmg(w5_ zwMANmt?Vi1`~IVs-^rs9R-9<-j7tW%yDfi2001(f0BJc%IY01 z!7)fr7p~W~zsYWzQiSuOS?Aqw>{Q6CH3@0hLY6OtEx^+ME;lB!T`oV^3I+#`K{8OO zMwWz{{Wg3gjMJ8GKw3M}p!*be;L$xYHK_fF2nbO`AgU-hMg~(8emlr!39)i%zjFjA zWJtq#;;1?ju;02^ff}Bz$bWV0vc-&ihB=e-Sfe2)K^cc7CqWo$5DAq4pEH9inm~ z&pzgNX0N|Ms9zS&-8Q~sP>=Q7?^}Dtp}hs^vwI<`8q+=T{CSG~>HPf%W1?hbtn#II z+F^5{7rwZKpmciBAZbHQ;lxhzA#`RbEK;Zsc1*2-c7njJIgO_SOyLcHUEoY@dZWYT z>37=_A-Cs5v-48Z2C?l-Ouxrk8kDTS$5@E3^Hcq=Qd?B=jQI;)TjW#|J*v*ymrh|* zH#O!LKh%;a6Hf(dQn*&nZY423_BmrIY|`B>Qqgtg5yvlNg7d$w;t5TPelH=7opr&2 z@4g2Hk$zKR%%Ku|xBQ}!G7S( z2hk$vh|UzdhL>25E#{gW+JAyb<>tBn{2NT8VJ`ysOcka$XDkR7xGzD z^odzL1jOnl@ck=AuFR!_8v-HDp`Xhm#$Q4`o%*vlLq#9rop7f|<9ZBlwsNoSG4 zL0lWYb%T>$1g-2IE1dHaG%3;|0ug(PP3Y@1uBcTm5zfQk`Dz=@X$^)<^Rk_Wq0idPXq z=mYvP7n-fC9p|p^*!)MtTD8e61NBJC56rcQso$V5l}g|4o1~VU`HSVBc$_9XokFZG zDK?IHe383-5XFAmfRLx1Um7EV~YrIdvap`fe|?#D~+knseiHJiGET*(-+qE z@N2qpML#$@f$jQfw=GX<;2qR8!Z|DKjfTIQei5Q8ah8ymSt2{7AeE@XYEka3KyItX zC%GDDODM?RuGo3L4_Y?^ORRK|vK+Pg&m$(qd;|mVD3>aK_(>Jy;-2LUG)n?VkP^2S z^JV8p{W=34ZDFE`lSYb#mkUoy&fnwIbNUJ!j_=yYW;dl*@&v2TpevJ}zcDJPRXo!7 zmzk$?Q9??Kv-ojKJjesW<-nn&YP zX44+kY*j>VB_h`q2qkr?pC$^gFvx>{7!o}yUXQu9?oU6eInNUTq@ zntW)LdsJpWd*CsB&wvs?lpQ=n4H-0Ea9ZM+gC3utTP_9X55kxM!+DW0Qlm@E9agi@i&V%tEyg?r``W7LvtgCQjrG~u4>_7KRtAKw$E zp5om&6eLtGI>p4dylsYud%!e6#nFXz^*yuj5<=ghRNCZc!0jIEW8}L@n75iT-Ez5T zk$E0wxtSL?=_3US>l6USK5AFn$BI<(5=Fk_StSQRcZb8qh_mb=`Qnbg?0tY%6wEfY zj0@=@n&SEw`WIH)FRa7JDVBHd^l$!hXVhtXOX!d9e6+WP^##`Zn97@s@xk4RBv*KE zMR;cvNE*+NC8s>YUCtw2b4ktmUDg-a6O&IAF;BHH2R;#9#<>7PvqJG#JprwG0V$Nm zmzsq9sa5f>Mru(!aP{F;SkA8t33GC1Y|8WRSc(B^J~c7we=MSWQ=Pb88ja665r^XW zLN94_7=$KOw>WSXnj<}jpExJJ>AA^Ztfhwd6!6=atkjZAM_t=*f341q@!~s}U1v@{ zEZQH#+cm_KQ_cp4Dx883rZUx9S;0ICH8xq^gOm2vXNrh#A)^%TP_)AJCV8+&Ia7#7 z$6CKb#zY7NDn9xXw<(CtW{1Bc%K|QaNbCz#zeBgc)w{*B{(x=5+2^wTMVmMjnDc&z z%VGiw5B_j!4Fckuf1oP&fZd5_#t!T)F%ci@^=KaL^JsvleAs(_;yLuqV*IPwbYpO&Sgr-yCQUg zDq0&b8{xe2PWD9lT5AQ|{wsI&zd6tU%Z93~AOGC{ypYrd-hBU?o$(8}+b2cZ9Q}sy z0E+kzTgtWa_89#cUj$1A_((C6nR?J+n#Y$I8$l8N3Dfrl6MXp$BPGv#$0cZ%8)yHt z%=stK9P$@o`OAMTLGP%PVE_F$*e8#yozDS*FxccCH7LyF_c$IrAX3HFc})cQ^Ui6g zso)+6OeL64;fS1as)3f&tirIRg;sn5o2;~FYV3iu6aE3Z2{+qEDAM)d`1Mc1p6JsW zFCG(O>C|P`;-dnetPLN#LSXa#ffIzTSDr`5KbS`+5COmIr+6R-3?IdPphEC;b6^v+ zE!q#M<8MOW0<%z70rLBdX##pii?_IGX(1N5*#fCz=&i3iiIBfA7(8mq1Si)kE_Mnp z9#SB`^61o5_9`a7w7V@0eq+U-s@>pQJ8DqpbnYd-vNmvQF^9nfFIID}>7Mc~U$}tk zx&S{LKccK?cEhP@?0v6;JRp6OUE^=UC&;kMv9&U|_8oK70J^6N_X{-h>%5z`Fn_Ka z$uO^T><9E53I&vD;*Rl1DZ}^2Fh%=hqG;^)-K?PG>%8>fas{Ud>&3QPxaqA?HXW?) z_N-oZnaS07Cuv;9#7~ZMNvrL%v`6F1QO`rm4%0!(mYCuq@S z?jidfp5W5Qk&{}&`#A11;@Y;$^uj%Mt1P?GG0(ThRAW*U?M*=l-+{*#ytpJ}Jp-t9 zG%m0D+9I(GQ?N27`yM0(#=dCCbw7D{rX6K4yV*Jrz!l7@fv$&Ms70fpQ#Kp|L8yf- z^pTrh_Sqkq!QHa`%M)9RXIJUcwv(Cxx*}GHzR#ORKx@RwsP16%BkHt0qSHK{#Wl*M zn&FoZkY7I@N~ph(}u9idoX{sP>Q z+g~An3Q5ikZ+F9AC5T;aKi#l#m1m5}*Oh4WBPWfZm4D-?u}m5&W`U{RWy@)wB$VPL z&_pKRfk;h_&SuxiJJayN$?`_r9L4@IdDLpl<-fE>QopQr-aahJFY zlNKj&SsB*Z)@m0iDNpx|>c}pu6Z#3V#Px(EHoQ2mun%O4&#S8y>PNxX zfZlBZ5PMH+D4`;Ok_qia)5nqrGe8@=PfAWr8S`nhR+crO@*trat@5DY+V%_0Q@`~8 zarKTdngw0EaQC!r+qP}nwr%5X8`HLJ+qP}nnD+FXex7skz4_i`rBZv>ubsV8wNmQ> z`ytA^ka7dIjfxk;MirF_iw`;%L4wWwA5ClCs*b0U+LF@md)$hJ6bR-@Fq_4+)+l|5 zxBpW1|7QdE|FE5~kdQ$C*)bL(pVg(()IY_P4N#UD}X6q}K0%u0Q0lhdkvddWnnq0lU z@`!Bc8L^R^yuths2|sImfSHM04j=tu?>ezH{{|um`fMu22Iaar&XcF`l>A#O?&l`v z!%uJ4Woyp8$+)FgvMakpAV@v$_YUy!7&t$}UrQxt7fj;{xvP%~n@JDl@K_L2Ukz3_S)hLYCMY!}D_kX}~*V`J*QOACljH6YGb` z7x<$G@&7;kTCyJcXn$7xepclFtGUErZ|3USCi@FM_Sb(8tjk4B>4l&3jS3`sh?@3h`r30ggRAK850M)IXs-V7$UXoa{TJ<_{*M`x z@Ahj0?Yy>zcn4gd_%avjJ$q9!sJZv#eAZ#Hg1f;Bn+3rBxl!~|#7fPpE-4qMLYHux zE$q59)l1%2@3w-L{dk-679$vVqn*cnnGPI6{vxFX1D4XKGEp9bOm3O$5*l=i0GZp3 znx!Ki;@6W>nFsCOl%@>gm%I$e`1J&OPNGzXqM?xWjA|UbY>(5Qghh?j z$p%4|&6AcRs=hGv$-05?KLHxkO4IuF4*c&?6>|VwSt7%&%EURR6uY;wfWK}p`W1Oq z()ZeMFBW{hB1cPLZ&NOgePMIzs9*D&IDzn?ojw5ri9M7I2h9LK`MspAhs zB5F;~(8b;!C1=U5Gy3zNUEkJM+uyz|#qU?Xy@?5l%;vv;2A*bvuq|-fr1|gOIA7`~z7E-_6()&4Z9Ex9pSQ4enW|Bi-z?QR(N7fk+lTM>6#e ztNSO#8If2Ndp8kOn;g!(JU4mX_`EZg>tDHat_Q^30T$8k{m8lRJ8!_%7QexFbtaq4 zcv&BvxezcO9g#rl_6sIrX&#JoYYvUr`t*v9$mDI^Mc`%u#|5Dr3?~ICMyrq2u@*P> zy5Uz|9f9HS`0S6-a$m+Q3_M=%Li+9<4O!z0`UrFMM@BFRX-~WAxpqfH{Zlu*S0`G( z!x2p`Ow_@acsv~_rqE6|=<#d!i3S}$z0gmzVk|&-uTP0)eV}s5yx7N^_;{~>AY+~m z0t^DfLn+gMCpF$KZi2c28!z`9oVqatmTfa$+k*pVj@!=z(7F<&JF_LtP3^9Rdku!xNza-TXNm0!39TWF5waf+r^m~wJaoC8WSOI zy?N}ouubsKAM-FAkS&sCm1Swigqc9k^fl+D)NL$z#1Eu(4lxMh21!a!CrQL+l-njt9q) z3#dtH)J>#hds=!ol#y{iFE%rnIOPMI#tCD7SG;`LL8CBKFG_)_H%tL5Px<}?S~RGN zqc?N!p6NW6eDV45du8W{q`mTJ_lY7fk0vXinIn`Y?-X5UL?qCVau*ja>}A2)fj3*W zE+Uxq7}r1^THT^;>`;iDWhJek+I^P-=xkQdSd15ceXecXchgsmPfk>UcI)#)QU)>< zw2gh4Ao|V#33gNj%6Ss%?^&(X3%c?wyh=q(Sx40U`VjfQ*_zAPBy&6-lZRrcWg%+7 zM&|CM#?z`AWRhhARVL(JL|BlG$iTf}pI5}PZZq3d7ZnKyR&20)R#AxFUS?*;af|`W zT1~?_Jc>v3@#?lySt;y!vheS28Q&SM*lpfQko7hP2RJdL=9?`ky8%!p6>jMEB*J18 zPU3>|xMO)d1mfh+OrK0~isA+$J@RaTR|{opMj#z7*;*7h@pEpD{3PoV2m0ib)MYh> zbu!D6OEwa1PTEmWT+k2hj(6e;M`X$8SAx(>d(CCUjs>`}g|Jt)jm_YpA5`>fhh@Se z?W_9Kpd>?x4_WemWAQdck_&}tn;Cm(oj8MS;0Cr=Yf+s)U6DUME#mbNAl|LPJwBLR z87cZ{=xoTikwE(S!hokH=hYx|=xP0uBqogZ2w9ttH zT|=1RvPrbcfa27%7hl5M9#E{DMS4#rc`a4KUwizEXfWny$93_OFWRd_nY$;t3Rv!WGk?+KhdOu@IigIYKPaqF%y8WnsZq zA|C+_n-J8H&Y@Ei)I!ZV&x~X`lM3%&>jl|SMM_QST3ITRv@b2@*1FFwXwroPrku)Y zQS;2xwuO9$-5wU@uTgHqWOoJhCPrlw%?^-adee zBDiPw4iJwb~F0%?kbYJv%UxlfZ7DxTF z=qRm3^;8o%-DJI&cCddCihCP>kGW{vLc-qb9n@0DoMSbRsZ1)rJ5jybSTLtTbE)`h z1xFz9o~d_+vWMgeg6TqO00-bjwL^k@$4}y8v(bAGCtcz{6emXJ(gmpI`NaT6d&IZ}Cc9sUs!{f@shv0SKs_|{HyDpS zTsOMEFu{*FWj!|y{9wsH{+fv>{^kG;jYmA>eFOdH3IraDftVFN048I|ygcM;N%)dB z$g2qsQ-GAZLaN(w|_Th9RZm3uy#5!G~t$$VcOV7ICM!k6_sujnc zY{lC62SGvaC5P=9qFT8$TR=-)BUKYMYz;}iH-b~VOjRwgNy$&iQ922TwTi|YfnWS< zTXh?a*O6>s+XdsgWmJEH7mN&O_)FLhB}HYo>_EH@!WZ<( z$kI{%#Z&&^DSq~lyto4(`o?W9k(;+IP0PD>hQ#6xHf9Zsq`IZ2t{lLdUddC^@`9|a zi+5Q-Nw)Qt?53x#=YHbNL=v~qE%IDmOz3X#4ej$);tmm|-(m>Mu>mJ5x(^V8q1>{N zW{#p}UK*t*To_3G!kZ3^VxUhd#FsAg4l&fEh@ONO?)ayazM&5AK`vb=J|;Z=+!ziY zxTwP`Of&tP;m-?>_HFH)X~IZjrzfxgzAlmelH?`;ib82ATTrhg{!&i(a;9WiPM*U+ z${_F63nRUce;97pB>B+(6Og@<6s$&BzVow!SXNI4u818EN!M%hNr7xz39xp%q>xyr zI+6GtNJ$h|#{emw32vIFd*CUfu9JJeM{sms8E*O9jE6YAD& z_)7IOI1dOaE2s30E6r)jW_Z3cq;F11Tnw2))j1dFw>+R?NL=Hb(UwRwJpTz-)GSw| ze)y^*FXre%d)A|RS$@GWcWu}tLt6>I)bp4KbcO2p8&EXq?x!eyWJh{-V>-f%Kzj1}8#}l-R?ku3I*efm-}><#IjLIT2a&e)*9YVDci()UR~!JkO8F^$OfyQQK@LfqR%zjTN%YelEx2TqTXkWb8%9gtaDxOv3*?*bN@ zyzzxR;rlOUHDBqySz{|@Qj0+BWM*uwH1vuXa!P_Q8!U#Y@n;oSRzW?M^gPP(5*Mgm zHh}6}AmZa+4AFTK^q#ynQ*yi$T?VK{9Tn~N+3R|1iTbSH+B@IN@?$4!3vojIH(aPA zxcOk0qSOD;@cP6p z##p7JVuT$BCG9DY_AG@2(lm)LF)AczwTm%*jTLkZIIXk(Lu^)*8cjjs8Y}^ApP|i= zy=vCpOvQ?UWgrF5$_LUAhN=yM)&+sNN7f9R3A%tQ?lcbiS3#?TjrJKE1^6pI;$LLk zr_`_q$q*G*90YBSG6j4vqCJ7-=!_6Zx+bGMxYIcF{82Hlb4B5`Svl3su5ODl)3X&P zlqEJ|zO4=8ZW}DM4v(*EVa4AQrBSz!ICAZ^@#YN{?lqXtAoG~E*um>W#>Zwl^MbUx z4a#kgPC6;qkF=I(F*osj4IurJCH7Cu-}rz#{FcQDa@$Sbu|@CNkhY`hVoI;qg}YOW zWz^wlcZ31Eo-(4g56)=MnO=qYgNeFLN5LnbYHp&&O39Cdng#4XoKBQ(S=Tn7F@ERB0d~N-AfN3 z+`b~a{S}!NRbzWD002gLAcy(*71}FUz@S+9{)e)1gH@P~Bgq>6w>3YD{R^OI{Vt83 zC(987U0=+6XM3VVNvnXRrQU%CrKwrc-1Z&U>e@Xj#-pFMdr8>4m6%T_A-6{IA3Mzj zuB!^JS>pFE&IfzSQJ;usj4>(f1CJuh7MBq9JCd4R1?|6&?SPh#Qe?ZD;XDWK5kz0d z5D{L<@;-B+oI1rbn&o%sKX+)CZpeQ`QP@)KHtGdr5AMprK1Ds7?j;1yYP{ zi7BN=k`aW~AV+wmqqxMk5n@G{0EpZF5<1h8jb*<|jUoMBAy*X;FDXU~Fj4*hsFtrt ztyP##R8P%@z5%dQryEX`rwGDK4kge6){fJIbz{nJf4bujZyRw(cF=8{IO36E_dP1w zd89^msH6?{ik|=YX$H;>SZlW(Ar!zED2nOzP+y{JMs?zoX~!eiKz!3icjCWAS4()) zis=Eb?;asgre@yKsV(7=;yD(mR9RQ!CzRfNOwb9_DM}z zs%!9+M3z?_p^TJzwM4!Wv%fS{CA#IEKH-=L9gOI|M`tUh{ z=lV^X0J@w~Gi~GmminnZ*mCjGOzW$N%*%Mxv+v|1j$l+Vj&8VrbUxHgwx2iwZ@M2I z``lLqiJM4;qi*VCRU-YswY}62u{8W(wXqhFcV>&Fbfv!qsn>P%_zx$AR9Fmm3 z!JqE$QU`mK8`LV+D^+dPq+2~-3g1xYs2_xL0abe8If)eQL)*0IN2cD**H)w-xUkx0-$SvX4jzAaHZT>3UTMO(aYW( z6xr>5v)2SDd%XlWj_|&MY7ymCO#@`G+7S7|$3b&5ql|efSkfzIjiaN@=6*R0VGaCn z6l9RLrqQR<`9vD-qzdO2@xw>G)omMg0L#@T!*G`KGHz%K*UAmvBZwi3-Z`_|HYLSt z7yFC6zw2fKobAXjnzkQF)Ow7&^c(c*)@wCv*JN6~Mn7DCCuM#tn*mKs%^C1L8fT{j z-K5rD@SPd|j;2@O2;Yr~*WT2vw|O#5RENT!Nh{(q4bGY=@M0lX(iX@$=p2r?2l$&G zdt0q;_{aTAOnNlNc|g{+#hCsr9{-R&Ay3EI{H|ENl7K(gl)kII+;BQwc&nFC=w*Hy zWGuZ^#v-er6(|yAc#XGd673pYO?bWSfhcV=eF6W}u+$Wrd3BF;$9QdBujG?4HZ}5M z{JFDF7&lV~bqVw9-=^^tO{+^G3*e-|H+zAea8DB!rG<=9q6(sK6)3v7M6r%8*lg3; za*;kn{f78_S)CH_FyvLP)Y_DA{8Y>;r&!p_=t?8!O5<=j%FEhk=RU<_wJTsZ+r=m+ z!DD_tMD*t&B!(%ag4XJ$US(kT1yfgOpm?AbuV842Dt$U^oD0Jc4$2D(43J`;UaJS) z)S{kVRij4bM{ikP#Nv@ic&c|s3e_fiVV;-{?wKl?`3}58$y^@yel}TbJW1u%v&5{` z_fkyX7)1H!G#8M;o+|w8jOTffxfa%Kk zeFGRqjp9#cHT7y)otljFky-V<1>vSUra9ZXe81ot)cTGa?x9#Dkz~|but@zG=e1v^ zOae3Z3m^FBQ$S@i0v_0RoDMh(*NavR3i**}zHIb>Y)u}sCTglp7a(OQ1?f6ZY(ouL zU-^U-X?~T)=szK2lsaVUbr_&n(5g08$igVH{)^%WmFgUQ!^11&akat2Y%)Ws$I6E? z|EA@9xoG{Kl3h8^ne{tKTdFcXe*+*bnTJ-hQkFl@z_zvG4NG0Q)KZDXSy#;Ik49Ed zWmM^2bE4uNS!@BIK&r#Yi&LfJUO{!JM0UtRbqGUMNDUHUREXpNEy+OMl|r9;TdOi^DJ;PJu}di%!$tdw(am1R4k%KnMPU#L6NAgY7g z!i)Z?_UCr!kyFXTOUHXC<8^Kp{c^|{iQ^@ld{5j)r&MC?ELU!ikn4^A{>vW!pGL|5 zZoRY(_~MKKBmJjMtc6&BkN2Yi1P%uTM4jRUkCH}1!G+y6qej93_TP4kVQ|=+2MiF9 zKT(W2omj@0jLeC$8C5wyn~JlfS&&Ov+RAE?y){b^IsZhGYf>rBIk^b;gi?Y_ znkqD-t!l-)g=XUZWa|9J$KgAJgT=vI-}#}0OouoCX0%5VKn|K7629XYWMwdcCh9MIOhqg}_Q*mk;4#F& zd*2LXcQ2!SrBQqbsN1>60aov4aZTJ?e7>{1yR&+R;2v#a{V2luChgaLYr^>+6oef; zYFtD~Ql{b>>2I<#nY+!MAGL!u4=;>-Y0s9AZ?#xl(EMcQ;>C`ekcI$p0Ju`pm#{=l z8y3y9&TfARnma10Z?=&lGjTPNP`5J)nmp$WrwME`z$rmA6o`X-)e07h!+p>DJ%atNfuG^kM~D1V>Zjnn0r~U;>*=}47qUjYDY;xdKc`CN=7f^;SHqb zl-qlAMnTj8RGUX=U%QE?VK2*aEm$-7RLU8zT>Y9$#<@zv%kVR&NA|J%$ZX~u$^G+5 znhm%qTgIG%M`Nb&dIk*V6D>aLiDeS-sd#BUeup$N1~OVw5(0~0AE=pvyQThG$Q#O( zm_!KdFNjFgC1E;wGq+B*H2W|hRG^FLJ{Wljz>}4gv zJJAz~q&2RyU6b|~f!cW#JF(fyn%eQhsw9;nxVd_`Wc@U|Y4a=9kaZj~V!|x~Sj($#aCBgJsz)fGm zLvQA&OMS(Ox+G&9a*b-BaFOd$n#?FiX*4tH0_P4n8f)Hv?<9KUEzDeF5)*ILkto&u?Oe=2MT1O+$d_bzF5b?SsU}juw85LeRz$Y$+mDLGsIzD}RPh?% zi4o!W>lS%KxLqrxOKL;RF@799siCl*Kxftp6vYcYE3Pz1$W*tj6G%%yIqZTZyK6TS zhGh_uEJhn3^G!BPS~Q-KtPT+OP;Rqp7!yF_sziHvp<|-rOIH+UAD6;n;6B;NsC-yi zUvyct&FYMgoWz+RHCd<^l>(&h(Nj&gxo5UoKv;)TP0k}fE6eONx&P`}hJqXw&mV0> zI4^1DUTTD+oO1`x)kWE7e+KIL;tfO3r|E0B#Fi{UqgvB?MZbofCjta2GkR`}&ESzi zHM@d2*U{wg^uerGtVY=WxjkXY)m=c3VcW8y+~}o3>*Cd^=j-skVNbr&*RSmfv)Go9x$^rnw(@aPo$JlraBPJj_ls)k;Q#}cqQ7wG<4u&ZHlVlu zsN4ufnaQKP5#1jW1Z)c0Y<+b$c0O{y;?99i<^d_lcY0^1cqXecqNuyTwHC=z-_1==Hx z;H>1N&Iy}T>3mW%s2O6ZK08#~=a%b^hRz3XPwgULbR8|A#^5{2F4#h3GwZxHTTi|o zaRX<Q|vFGK@1NW?U08tbI3RPRSBRd7==Y-raV|lxfe#B3ERNY zhlzq)hGkh^C03-;DbFx3&k@UC{M`^($tH3`_seY_y>b?|r4DkLNT=MJC0Q5rvYu|4 zmSy>ySh2`)7~J~BkGxzIseElic!o)&1V)&t9bUgq2+lYxzgr1!UlYzw&aGA|9cNq* z<4x*hl|+gFsTf5fb}A%8lIH)E}68b|uKmkDr& z!=mX6(XSdcQ!{yClidEI^gMg!<7Q|*!%+JAUtr1~WcM64R4xdaJE46kAA#frR0w}> z(y4X_7CO}GE54&jIiL&p}f&7!(i2VRK+tnBowu$6{v18;1$Sz^LVSR1u zr03O~1k_?so1sx=1tQqZ`N9f=yeyASF~vG?<3Vw>=Rd=jyw9;Y%i~Riuf)XoU86*< zgLp4Wf`BsfW7karMtXt*;iDcIN#7X0R{_>LzUa5=#Q4~C0!Bu{oy2hx8Oh$SyOjhz zUVy9G@u`%it++pAffkVX+PLkCm$r-voQ#fm(!HDHi2NN46|3nfe>qLWrBjm)}+A{gP6!&$!R zR6(278Mf;c!6v&(2|m8AX};BGbP-C^eK+Jj8CYl@>2^3BaHC^q((ksCWmFRh_BAG7Z&3xq0D8PRWR*B+Z^!O8`Yi=_15YD5sML~?;) zPVKfB+NLv|f+V5TQr5QX9@wjxpn%n-9dmrtb1(vw{M7gSz)0``xz2rZdicC~Mi3VP z=-S-?hD;x}fvV@xIQ0<38j@gF==_vCNw3l0VLbPblKG-BM_iXj>zO@xd zO;F@UE*-D`g{A*bss6tU{=TCq8kZlem2BFA3JGN!8XHye|HY$s#CCr{(f`Qud;MQL zIy^f&LK-a-JS5;pwqxXdZ`#Mfqsgm-VbGpX7ugx{cfhtZ@;o6hsUT^&krcY-c;E(H z_@Q50i+YVZ?>%}|c?5Q$>P-dTGXC>o?&|91=66-=OFiYkyDScemz&!yxR*48)0r%e zryQ2O*BOqx8xH}mTQDF_t*z7Wv3Du{xA{4Wr%GDR$uhuOL;1_oj|oPbn?C=_*j0~j zCPD9D+2{L&Zy14ZcOd?3u`xOvXK2)h>U4oIJzu&+%e$(MU`wcKw)BXVD(z##f~#F| zVPup!fr0EA*~XWvKlqAvxu*0;hJNq#82OtKvP1r-I)lM&IF&EhPId7iEw#GRg7}Cv z^{QfnItYN%Kim37HCt|ko0^-o9aWGgE+~KNk7~BaNQY{+g&_}obC|nwLtxbJJcnaF z)vK2$#y!!N8B(!t@n3-%~^f5qIU~2s5hw$^xy*YwYgVIeBRbJ_lpLa!vPSm;i zhX6vfAHG^dR9U%aCN%ID7Ij!Tr|bwLnlR{`_q&UKdk#TCRvjYKoXQOt17HEclx>u@gP3eJ-TEFnnjNxBv9v~dRkp+nF6Q-+EUR&$_`gHvZj3!WgO5pXb zB!GKJ@%|qFu2BA>Lhuhu&*AYQZ|RR40RS*G1~@qBTc$@^xr_|1Zqm|LV-) zaqs<2ho2n%=+5tYjNg5J3`GAMf&j2J_U}CW?jD=~5RSfOe z-`N@9=x*WY)$a>L;2+OFh#w2&HvzvdgN?7fF9H7VguH)g1Vvf#f4@-tabe&bf11gd zp57lE4)6$!l+F!F*Z}!M`N#M$v3iIog=U(fM#bAwD~cV5f5( zEp=Qt(Ql*vkf@*}rE82@yzda z@}S-RneGaQr1BxcyM;q&e{>FY{k}!F-QYrutVj*oo$qgTe5JNv*jT;22BhuaY_6bg z;$KD$yB8InN_-GEgu*^orc&8FEH3ZG9-q=hKZm9vyx7AcRKr6Fq0wL6LI0_SgA+a1 zXTNNqLqeTvM;*klix~wM)?>;72LF~!meg5T3)XC@q@wfyj4z;;XC|ilsXFS=F1OUM zHR}m2ISyv0XMkURcxS_r-*Be=LvN<8ut{8`9dPnNKjQ8fJ)#H^6DM9x8GwQ-ZSy^Is> z5(Y6=m)sv|tf=YZxVD^ya&y3o8P&ELgx&X3_h`2v+{U?$W*Uck1&BGJk9ker#XgfQ zWv^Rc#|_F)4e=bg&~?puS=d91nLOweRh9(1TH9WZnFX$W2ihA6R#t=Ug=-V3(v*AY zlasv<`Rybq8W7Yv9k1H4(!-LL*jZS9zBsnwjvIDQ!OC6-gY;w#87u2pJBV_Vq!#<- zTt?=+%>Dj7 z+%Sy2wJlJt?%1>7r4O+-M)TD)FXx^-to9noD zx{hwMn*g~vc!m)6X(>iDo%&{dK4@V{cbQd9qm2DpHS5HBW@Z3hv=onVO0k?1ktq7q zXDFIv8;qF}2&gUZIrPn`v{u!6*&}eH9fW44w<-yHh&usWc8*#5yAX16-c9nnN;(xa zx)##6a1I%L+xV2%;ZW^LhInsCCyxiXb`XaN zT@A1w0*YYomP62Ns21!DJ)W_(o&{&+T0#D89nZIKc)+LdMJsQHSXZp%R+@i_hiuFu zx#t}&1H>B;%eAKWXthupTAP|Yqi|{E3g>e7m(8tK95mFrT1FimlSB#Aeznpj{Jr6d zl$&Wx;=5r_v}r3|ML`M_TkeGBRu45PfSee%#?fi%WpV*M7wM`C8kwdqB>G!D%W7+0 zI`VtLte7CpX}`G`vP#KyT+$@+3KG0U4NB&h37|gL^6y2#dMcyca=A^?3b#TCo%TP% z%ZHG${6>tsOK699F~_#E6;Au%?NR#3=W-$$u-pccXW5H8df5x)hhmD7W@7S-!)`Y`PJ6anJw2EJ&J}! zm8I1ROLG`kZI6twrRq7F&1H}5OY)qgY9igneJ#dGEk6*8KvO&H^ShLwm(hsRVY;&MilK}FB z$F?oK1p(8LvQ2Ho$IT8$Xm;|!Zt0-RY{FjG>v2y+0o3qjMIwZ|BEt;Mq;~#YUrq>IT14{c9k->rT&AhDxI)vD`SaL zHpboTyM$vA;{xwm&bpOSVLT1B1YTnCu|_nSB{I?|K4#PILoVaEz*U}m2C&GSzPCd1 z1o&o8h$n4}U?o4fD_yii`JNc_W}8`donbr>@!SiEKR~HFWnyFCF&*@|=MRQ*5)Hz9 z4yR$)>QsaY9pH*HJ>8y^+G$CN(Fs~q>GQxvJaUyRisOeP^r@9AF5KHsxf-^MxVr?Q<>$I~>y|G~liVR1T*(o`! zU8iq{3x})FtiZUz1Gt_e*BM&PyyhkT)IDX<;#grl{_L@BWXkr^+mj?GSl!#2?afrBEUwJL6dtsEXXE1M@^zsAmR?y zm0SuR)ETB{W36^=tCCfymerl^=G}swR5NO!U88PctB(%jynNed2QO znXotK$hcOgz9qEuP!n1K4Q)%m@k=+?=lUYZjI>o6YN5*OyZ*%d)OH zKgW-=N&cEyb>q+4-oUEZETm>`W|-nnF1B;lwN5W<16oc>TZMI`a?O{3tSz8dz=?M( zVJ`->EUOm8qkMdoWx(UD!Q^dlV16o8MTcxSJJPJ|P3aR}I?k$|O>7mNsuq^=4Ad6d zIyRQX*ju3Y`AVyn6jLQR{eBoGT_?k<3n&NpKeX8Eypk56@JqQzlye6&?+qIa4-mir|Tf9V*BX>qka#V2%FNqGxLy zSvpbMkIPJkZcF=@;;bLDf~MAyW#UbBl$FCG%?ktD)C;Rk5pps4j} zi(I;sAvtOLwKp{(5F+dPQJ81#Ar4o1g;;I%jl*WNV$x-%leHH7K(K$FpFNZgiR|jc z0Lvt%IY1h5%McO6f#qTuEYuMn*pz`P7*m5>q~ zMx{c5Us53}8og;`GAm*{a<-n?wnhs1^l&w$YR!$kR;$tUff{vMyu&zJm;Hkn5Wa~s zIPK&_Qjv`*uu zRbSN-$ssD?!cMk+ofJ?8%rmgIQd+F?aBE4s9sT6u<4M@6)7e$~&Cd)pfrX`Gf_J>G zvc+>Qv{ObL^X1))MKU)N{b~8vmBJQn8A+51PpG(AC3{KEWT&W3HK%JjsD&!g$>&Ip zY7|3ByP`^3T{tX3(i-a)z_IPEn9fn?ueZ0oG&Z2*i3=o)Dc1#59UHbcMBpg#hkcc+ zJMIl!Z;^Cl#loAr#}5RrzlypB zpgAw}oR52+9_Xzu-RP-%0?ag90mAIG;l7%{1V%Cy zY>3}5W1ySA0t47135F<;eQVk!BsYP<=184feux*7sGOs7F0ddE`&l6ad}#Eg!N0 z5e0jch6&76{Sovg4l-v3vC$p>`Fzz&QTuqgwATDm}t3T(+~ zOo)upp^T2Z$(T2VT5}-9e5AVikXJa5XEJ z)~9F{BDEkS1Jn^6@DFE{qpj%A3zM|;YTO77TJf`&US&LcW{4P;QZb&YPP#$cJ=F=D zqZYk^xnS;#&TZVJiBJq4kVcd5->2EZkn_a#a_PWvwkRU24Q&w&NW z88L?tX+XZ$B1Y*F;X3`eyPNOWf(2@Nf=8*GZfguc7`TKgm*-C>C^kY#KOiPImj!r1 z@%2x^|2UA#Ch#z|14*mocLxo3UrNh1@GzB#D5gqJ$Dap-X&6hRrk@pq&WUZr5g%;R z3bdll6C*u2m72qpSL#qP;Jw%Qmm-}tJ>1WDw-I-srur!q7G{wR-(*(AnC&=9-| zX5eTYGR;c+&Gj(;U1GZ2w?cOY#)Q7W1UEAw1SAKiI-=HANH+(sV16tc|7L;U^PoZA z3?yME*2qdo<8(+M-5s3{HRcXY>h6h%7>8uYw`wDdkas6uP?U`_&A$2B{gU1drJCgD)Qs}UU_gW@V^ z3aQkA^j$T{X`I)qbwgRwV$nIDE|M36rY@NfANHGQtbYBzs%|F$A?c3OBGld1ef;NL z{|TH<9+cFekCY9H5`=O|Y@KV(BMt@54xkmKOhJ3T*1{|spv}oXd4p?;LGBs$1ahX9 zj`r7fW02ft2a;z1x4HJse;_DES--CPU{e@bnO}7OtISVp|j2wr$(CJ@Jlh+cqbd*qAsI+Y=iTeL3g; zIQP3%{iAo+uKiG!tBR8Z`8H6qUx zc*50dOBzk^Y)EjYr8dOXtQ)&ErEq4%Bu|6Ql3`hL)K1{i*zDJ7aA9EfueSUV*8N#T zpDj-p& zOahw2aS{9x*aRXzJ%kr8>G-@wCZUyI2misWo^`h|RHLR`;5XB*qr8-?TcapfXc13D zVBGhDySY~s()B3^8)ni3625}~sgz4#%E5qK!ZEq1KMFD64n9^cGgjUxZqspgb13jx zWuoOYZYrI8e~6OiQ_S0-svp?R0@gqQVb^%TUCxtml`NP4sD-*9D28-P3N?lY!6M1L z8ABBLcNj;QQ3mpzC&=Fvd{&n1WMR%)r{H*;{0d`fzw7|wv+P@eCg?pt^Qb#}*SQk+ zGC2F}iCWPn*?@0Z^z8@eA2#?YMY2HeAK*&0b{*iYIbb%nBuh4}jPU(?DSoIvNSl|h zaiu9NDB|`Iy3)D~A%1Bg-C&mS7S^RdJ)qOz`~Oz!7q^*rkGRQ3Mv} zV+aEs5O47CjXGFPh)p-CvF-}QBs;rrsEk_mo(PHGD5VSDZlq}+i46nSe@ zl8(4^ZQqP|Eq*ZJ6rybuwPg#UC~wlw5S=v=ia1`A3z_?N6IL0N{jGg<)QXkk{m z{g96W)<1qEFGfZn#(+8{r6w)qV2v2|wu>V4_@=!y8-ybNgovD63ynSd*kRn2i?pnj zI>QE8mENR^*H|u_tZWK)Arn8 zO`DiI=%p=S7pYyLWTHM8Finv{(}+*im`h32$S-1oRr+J}yQ=Y(@CsHe^sC9=I7XS{ zxL;ufyco}iR<-rmb2?wwBPIC;9vT{Jw0!FPrQ)6EXmt(l1Lo%ORC9CWcT(00?a%Bx z`^!a+oro0s?leCfrcNPrkX!#uCudg%$@4OKNdv&nu%2Od)A_i zSL~#XWg!lkYjLYs_S|Uqsd)z+`@|n1&|hLtS{=jY-lCb(02rHRd#30s_x9OU8I|VL z5(`;rW%(i0iQw17qsjBbY43(uRIPNV+7>$cr%K_OywHB%tKsJRC~b+zMkLtkZ`$qt zb!I&PZS4MQ{AaD$)0QYJ#HR76KNr5^RWLJo=V{D^y(Wu=`xX+VHh;Yt&*Qnr)#1(u zA3SWp)wkU3Sk!i;so&ocW(Y%6K9FN}Uzz>Auv71m7jGnuAOZBFyU~I~K}5@2${jj? zWDV1Iy$^J^n{s0`0CM_Rv>}-=46ebS8c+ZZ^Cz4|JvzoH=#^Uu)I~F?It=O#ygOoI zZjA%r}1?SFZTEhZ|)>qbkdaGK$e_YFU{xB80)lCxQXrW?J^-YFc5D7i!8$!Kj(Tj9)>|eFeV7h(aAjfqtC2 z>_adY;k}Bl-!Sxq8^rB_ejwyrZi|@%aGHCv$UBUKWFPnyhugv2Z|YWu_5HXXekVJE z0SIs54XK3Uhacb=udMA^pLhb3hT%Z=Gvltu2W{zrZw85)ML}U6nJpCt`HY`T-fIfq zleCx01y9VH8+?ys<4l2m^KI+l4sTc=A7P-j8+?#je;5nnE1k`=>$x`uzAj>2myRGj zb5z1{^w4I|X@J6wGgF=OB;-3{@KT!+Sg$0^8|cLII-;wUBlg<~F+-cnc?Iw^)BHW8 z_C1Ja!Vdhbh8N0y%P7l@09a16|1Hfo@*7jS>-(yFg7*;TN!sB4RYzX^YZV|GjSisp=*dSx{?> zUC^ks1T3_fF_%zEM%voT$m+$&212QT8_Dim)It=}iY_mLhRDM4hxkuk)+XhM8ob0< z3A291z;nb3=wVPr1HhYIp-kyyf;DKTw$HCn72lm}@2EY>^k%P?-4p2vRc2q_Z!Gqmyz@ z7M6I)_LNb*A}%x^L#TaRgl#+&*LQ3b2`mgf8YO*HRK0ap+}}t~H=ZeQHfM;AJ(PDm zt_gyok><5c`*wEd*n8|A;H>!^hWN znDoCpaSh;1xX&cO6ceR1N>Z=24&LIGTm=*{(mh}!->Q zZ|kI1cdR)JOmfTBrgOSOLqG_E4qT(Ma1;BxSupnKcb4aKRg`J|1U;vOJcsxayJW6J z$s?g^`TX)s9etSw;7^NE#GZFU7+k?jyxpb+f0BDlJ9+N04mfn@hQ)eBQN18&quIBK+L|?^KT*cewidN^ou0Yt()ra8rg+olCQv79aePF zRiUqe@Jm zDHuc*eME3=34eTq1g_=lgR$38^D)K-SlUDK0it|Y#CcCBGf<>EUaJuCqk+eEfus9T z$A%Hd+Y!ft5y!+=!3{F7A*(v6}!`-c|d~4bQb3G@k({UnkbhGqNJ) z-c5YK@h{RH?R-$A%Y%F0A=Lm+SjoOS{>La-WBUc6N^7_|EtwEB5|n{56Pj=F0$?YRAJfkpmmzn*@YJnAC;aV;Shxlg}qH`Dk z9I4Lw%7|=R3Mx5AzsJpkJ44>Qs0Fwulq$-jwat!y1AQZYO$4*@nm9ilClLokY5H)W z8;I%q_`!CRF;)^)MJ>hWlq|3vG7{GN(6xJh&<@&KjYuwg=bCs@^FrXO+prM-IH=})jB2)y zk^8wqeeqtYUo*dYkzF9&uQPwf!=xx<&0&Nmlpp^=4{h?GIxCYggbg8(IP)g7>sv?q z?5m<#j;6Lvv^U8NrE<8FGyrc&?ixw7ROoZ&29b$!`{LN&^<(b!gDmbVn<&NhQP_?D`IRJN7kO8WY`| zQ&COFwcnjn)$>p(gtT9}_@bjjiEcRin*Hz~AS8dRLZtAYo(=zQISbT->i1%*KWj5R z%oO6*hqJsxLZ2A+rTUY~%{c~nzguwOoHak_8WsA}vYi6RqM0sgJ`2huq5s~Mb%yLpDK^QeYwAwr3t!bL%JZE zO@>j@XaR5sne$u`+|xh`g>c$2m%|$0-Y_!Cls~vQL}j>auSsuOno)F%SaHp`S~X!< zEnM!QRBj$-D^`p(g?a(ifE=$z-te6|Gm*R> zQ?~qRiP@7$fxerm5p8;rE5{;3lhzRphNE1v1e3?w^X=JNw>M9SA80H5As(L2k#NXy7?7f#vk{ z`QDAwlrz{k@6Z`=_ff{4MAT?Cz1fie3^DFMRcDWzzJuTQHa@ACGlL@~(ajnNro`-b zvo#TU%JumWo+IJaR?r$!aHCMk7lkKnQ7j1#o$TeuX&(R^dFzgEhj7n&eQl-AI0KZ) z+8X%K;G9P2tpFxZ0B`@!J-6(S`J3bq!QH%>y%p6rNND5%YZxkb+T+K%@UTH*Ft!JKt$C5Hk zBOTVo`WQLCC9dKFs_WzB-6m&5AT?`B^Z`+Fr(*&9&Xy93$Ht0WmhX2ax8^QRcHy!cT;!o`A!8bKy_7!iR-x{)1Q2^XHs z#tBF*?caC6?OK)(9^GW)f@0^wV&_6KXM<>v^R3%B(o|dKgC=J;3?6+lf!4lL_w>cile$X_+1&`>$z*Dmtkk4kP!R1r zk^CU5>M;5Q5u0MdlcQ+)1G;dtkHjbIU;}LL`r}v~2R{5AZ{;%0@4}ZL3=};>98?{9 zUl6^P+eO9{b$T{Zxnsgy5Uk1;Q|8{?QQI*l(-Cx2 zepV505K|PQ#{1tK~wDa z*`uwKjBnG9Q|BvHTDj*K0NlL|VnKB=;ce=PZpr2Z47&gNzqePWp`>NdESY@|^gjGc zL)M!BJw0KFM@$@dj=4Qlxhc2}(A9(RXo7%$q}RFb^C1}iShWfPF@K)uOw~i&je*&J zcww4&(Z+10L@|EtR97S7UPvjpPXlcG0#Q$N} za(Ldms8IX~eesAoZqi~}Ue8V*VVqS8G64OBtIzD3RglnM4 ziO6HcFFNNPMlSXjc@@}oOF70C&=B8T`dM*~-1SqlEA7EgZ4BkPpXY+JdLpL_|7@% zjr$g&!0v3BKqfzM&ustQ9sC^ussq*SgT$@rF?@ry?*lQ`_Zj((>g9)I84t(`RShCQiN!-||Y5{C?>HJyT$;`coZRCYd*;C*9w`-ilJqfojWC%QTLyU4TylL{xPFe25R!L@J+|qQ{vu%QQ zL!8u+G|~;!o=>L};N-17jdCqZLkP+?u#KTNy%7gqfHS{%{X>T zp^V}pkp?PwW0|p=iYW#Ad-hh!3yi?X=$SU8x#_ILKIeA^RDFx)Q=ZTC=C5D=&sR@^ zV4fI+CrZx5{kuhRvh6jy2-Y}?MqrLM88Kg$oz2C?gpShrZHGV`HGaLR&Kj%>&qDMu zbG5O8Bjz_n)+H&Jd9hmYdB%os8%k=Q|vYwv5ZmOCkA=hk8YtWoU&r z(gUL$boG0%2;xAGA->n@>M-Kj9M43RH3XKS- zmIKc{@Av(tYpmKPwPnIiM%^CLV|vpTI?WT4ws8}*xMT2S*s$jCMF+FY#UO-mgarQ_ zDbqWYOx^GRH+_Rly?W?L_ZRkrrsjD!GvbEx_&V)uV}AhQiY9|cZZr^mWI!%)M&gj^ zQA_mVY-)-raVrI_3KTPe;RE3& zBUl&jt)iod38Ru6i?L0`ax_0lQOXBVLAnLii8?xC7ndWXw9Jvt72Uk(<=hEIpYbS| zQ$NA-c6lfVces{6Q6WuWy5@5}q5cyy|L@8CzZtJAvZ3I3NQ__iRm~rZkiLum4J-W` z76UI`$PE!0&_t6&|CEPjkj8B6601U`w~4S?q*;@wWkdav>?7C|>TKFcXsw=)Ofl?L z>%2N#Os<8$&(^=&A(>PjxA@!4{3VOQA?hDl^o`&R>)jcB+NRc`l`czo#vE zyjrJR#{*6}Rf{5jSo*iq1YTnPCkWPs1Mk32Y^L_KQt~4F*4Y8e#;Ga`kF`y5S=EUo zx-8mz(!4E_^?}G}GiZEfoH3e7804=UTKHd>z!1172BX&`vX_Lx7`5nl2cqBzvM4*f zYB(C$M??{8gTs8Oy*U1*@+|}PC5n_Z7k!|KzIFq~q(ucZjEk*%zwx;-+E$SL5zZ-{ z!H&o8aqdnNThcB?G@>^z9%^yJN>?KZ%TEz1o7|u1sYG(+@}nwVdT7cB)2J=6LFhP) z0MTfJcUD12-JP)bRSRS;-rPES)ooX8Cgfm~bWkvNxe*k?0-}ZItdpC*7PDE#*d*6C zSqade;F2TI(-SLXY{f*EDtFv^wTeA~ECXDl$oe1v*gr2EEGK0NUJj{tvVD5JvWm&} zpf+>8FS7a8ur|!qxF;c@)U(ovk4By_Kdnr4aU+lfPiDT|2^y( zK-0q6-s8nlJ~vw7A%w+Q;&)yxX658tAUD;zJ=rXpWVR;}=D>J*G{_>S^ss;>t9ty* zMEouij|A=KUgufDqwcIpjRdbWUl`+nRc~MMc6p$x0P;gndSQynom+mnm95+wSZy~y z^2Qi+yMr)KYjtfFS)g`TqJZhhpoJ8-9QBPf{X(io9C_O4k(8n)!auNP2pnLCNtJx#(?-SExS1Z%vM$faXeMI{bp`O(*s!umQaUR zPoZ+ODzNXX&AhbcK5>%d^VFH-CCEmtOp&0ho~&uMVx%don=70Btu!Q8C0Ru!WLD9W zlUG;Gk!p@hsKb}(XI05og6{od)fj*jQ69KnnV)l&m!@{awu)Yl)9Q;0*k`j&YKhlR z6Ai^<#YD-`w%S(w__CMBvVyI5+@3w@{0C zGLAc-Qsr?b)r^s9!f(B%_bOjvsJIj=Mjm4@EwUB?y^1nVVMigWE9C(0BV<8C-bH0x z9b%8lGel6`fX^%QJfd_6)X9d9DtV!8>E7$#vz9okB_4ba2Na{PMvzwU9=9aSpBcd3 zOVY0jd3>Rw9g8XCHp%Zb_ibNbX5ulY-17u3cjW{=Lv&D;g=h9lM!>HTG*5X_1$64W%r&+rX!?RYX{5moOdcLhYa~HWFwi0_TT5o(4*sd3tK1KscBJuk6beAe zr!5y|?^G6)%>Fu=r)Qk?z?ut`RYeblzEn8(7+(|krUH`pYL#$3u^qA<%wKX_knD>N z^;>k1UMw$vq^c24{uAs!(f)ri{r}F8{}b)o5WNv?z!3k_)xnsyqDe7TY+Mw z;L)bUR zUjmj>^fMtrKq67oQTVa3()+@gC;`pCnrNEnpWS3k#&{TdN*hAr{ovwyHuP(l#XqHx z*&VE^5sd@pjKdTl)7QSLL0=u76#ZL|wRRR=*Zl8tJ_O%&UVW08VTg&8;+~JDe4gGf zyL)}EekJ#QzP}@ZtY1e*;pU_cy1IDwfZaJOh~rOB?gruA!#dnkNztCD7Xkwb(SET1 z4qb&1%O+H@4xOY!Lxm&FXr0vHcOf98F=ThD#`;v5=r|dx+k>iIJme7>Csdc!yU0on zHLjxqmK@f@vK#K@M#iA&&T(1J)kaR4>Pq=W*}*lAG0uQ)e?y|Z+eRYc2#xJXCq zO8&B~haKzN63kv0Ogd$=;sDj$TwdPCOPgp4%Zi}uG)g_AE%H~=G*%mKDnv<6-qw=? zCDz}=!=ZfOpW3%foz0c!rO%e6H`xz;8nAC5<^DDK9cd`vO^h6fzc+2=FwYOxjW$5_NM$GAu^i7~!J9O84l3Ai@y|XcRNxV#=jY$v-|Rvfd>s)*vjBvhW%^lUagaf}EtFl}WSM>36}9iRcf%TJUWWy7Vsw!kvFFj7 zXqvFg%182@)3S>sX|l9* z(&m{wF67hs#m{!02q8+CLw+)!qy=VKh843)t*$!Vkj+7c1qRrbqi&Jd&t7ADSX=ce zaIY8d>UWjzLT)WzgE~yaBC*@`Nd(3&(EbQq^F?jdO`EINr2}Thq@`x#*?eQN!E#-7 zR&fRT%Uz!3PM`G#7pzpTL&F$vQp5aCkgZ+?Vc&DizAs<1_axHvm}}UyhhBl}SisYP z;(B3OKNARAJp(9$)>6AIEsvHoFcDoe%6>I4AYBa>m-28|iF-6y6%2k4uxu$(i$P@c z(64)XO+VzN!eB!6iIhurLoRTwkDGQUG=y)!#=p!kzzUmT{3G;m&rL@t359WG%?x^X z5Bc@_Wr2Alh}a_gm#yA|!t>`#pyftc$9z3gPBv0D9eDTmG7jbw*x}J6BmP)2!EN0> zSqQHcSufGN@EBRRV545uF6wp2g5=C)J)MjnsRqc{dt_V5(K4lToa|=WPj8whU85Mr zm$B_+&e+>EpmKo2VQom^Bvs*7!`8Ql6wxT-k_ zG$M!Hl-8I`5AKOf57qc%e^sl?8h#hmQ#$VqxW74VO5)~A75x198dfFX0?o@6?^GJ( z`g5f308KAhioMbEje&%eYEocAvT$gxmpDtaqfgTrJlYw3(z&KDHY;ZhH($@gZZ+r| z29~FSjCvGycYAUdR4cDQlo#;tD77$$A{34-=U{@G>SWqliRy;Sq`rONS?Zh(BT0T1SNFly}U)#K&JD zlI4G2ly=t@?=DbyZMmu>4VQ%+Cs43S4B>;p7pa7<{K20f0WQ0Ho)lCLD+01Q^3DI} zi!qe^ujmu#s>=gx#;0W{*d_*CCjI_tARn4^pGHFMsm_nUDs#5M^4D6`Bd(Y7=cYoz(l5CO=F)=_= z`ohYR=FtVVmx1p!kqoJlqd1P$mxDNG@HPqH1}QDFprhbKnRPLt=gK?x)Y?qY{=V%^ zd#fk?JR1l2IU*1so+L#301R_-`7=8MHkl>@8e_GHQtHMTqg+D);}IDR3v^j6uCor7 z{2A|>=l_}${crX9e-}$MD$r)%udPu7e0pyL*?-qHqV#%NZW5qX)Wi2)#ub;Zxvl5pW5N9?>a>gN?AoCZOz|=H%GgcXpWbt;A9~cGI zOP`B&hRFvn7+awzQx%cHsLu(phVcr+KbF}TTluGLfTIadK2y68b(*0;t~!q$WE$3A zTT-{e(T2kzuXR+DRz1$t@*BMz2U3MIMcZ7!*oo43-5W3oZA6);XFoGgv@TO}rilzP zIdF66m>?*i4G)X2YhZ0hTN4vU-Uayc9(++gl-0Mn0cY`LNxn%B^w^OMa*LCy<+!Yn~9OQp2VE-VbwX2)H>GMK{N*Z5{}s`37NqeF{Fxw|HWB>s%fQ2DsIiHssYCyb~>=J zPUSh?&~7vBB8%tCqEESsic;=YfGNK^cMpVEAkTz}+{kV2RJlezo|G40w)2R_luz=| z$BUh2iK;u8ng5EYLpNE$)fP`sy00^MhfmTRfnO)d6UREHF4ik45F;BnuOh6d$3pkf z%OT0d>eyXznANOIv~?)InBa6z{Z?J}2=?Vn!>N{6QY`l8$eh)B-X!n$eFelqix*&e ziX%aC=2l^~P%d#el8{7!QF-cN8&qe@@q!msBP_u$U7S{`rb4FpIL-F7K@m@Tr_#oH zqy{%Ol`5$6jx-)s<+4tIM^HB1gzPo``6ahl|LlD?&0tV9?#zB^&~`J)%0-{b7~$|H zi3lQ=XlB!qf`lH%>^HedaD<@;3V;XDx8C+OXYU`0@J93EwITHFk$x?wwKls5@SvA+ zo`sVzs0ZuEYyM(AAxAGO^;SMn)8~6{t6$wNZ_&gYP1;{{%>@$0~<`UT?O-Tcfi&p{ATuSMA$LmekI@XE9U4#>MgNe{xadL-AtFJN=?R#d@F z;;;H=E*Xm{W$_%k4F@$!FJxQxY29{gmw|>V$)AYxCoeYT^OS26d~0z&)QRbH*$CZD z5j~WxYAqfAwiio<>V`>FTY#(Cr@=aP8@_viI5dKzqH8hJUMUHhk8&b^U5*w%6yCfR zfr0APv}aTgi@ZUkM6hf6x92#KXM)Y=!fE^7AzdMv+8FUfyFCt|hRXPebI975%GE};DJvSzBsBP<}rit;Pz%rgY zDlbK`JAgnAj+_hcC8yg+Pm~I2f+i9rmzAdZz|ujM4$u1cH@-+k)K7I=KM~;)oon$M zWn0nIl7dn-I32?WbwYLELDRh>t+%Rr)Q5W&XRL&z0*V z|J`1%q5#|uqELe37 zv*oKYP~P?2LU_0`32cpKOOl;G?FA6#_X}_NF2##_F2>L7-o2ffc5&0~O2MoK@E_0Q zKkdUkn}8S&qe^a3bm_J9BsJHeygF| zr@wIc=*J0ijU~bpXL(t9f1Xw{*xTlK2jP!rU|ub$Kz=&7d;b<7okG4O!hqRi3(tCO z5mVGtWG`R1IeN_du;L;3qssfBCd*=c`s=qjyg4(!`OkZ6i6F)aCI3Pcc$Hp(l=tyo zZ%sKurdTr|2=k_X9!kYqj`e{6(%p~37?=0nfz&>MoE9yzq*D7^YRy1<+KD|uzGzf^ zKC<6*O+_XPYIjgP2u`ztY&leNC4quSKT()GRwYMy`P8Cnj`V2HqlZHPIxFPAd-UBZ z4_q-{iilUo62UcU+?=Q9r93BrF)6rtW zAkQS6aZN$1pK>v0VP+<}yw6T=dB1aNW!&Vv`+*DBAaj;(2w@S<;G#5EtUFGdixH8- zkQh^(S7;EBJ!h*;Wr?5lr++nX&jx;YO(=-irE!vl&#zR^RXo!BE$dej2NM)2@55G! zwUq_HVvggzKt653#Z8aTD!#&~JyQOlMPZ(Rh}*GnvTIY`yPfmH=PCdq(D>(fE?jX- zk(9;RvZvG3{hc+grOfnR-r>ItJwdbUsS_nrtvVuSrGE0!iDiYd{epE)u>uDbmo}FR z&2n*7p;)xzo_g^&fvZg=6Ldq94NkBnvw!7)#swE+P_@k`N{h7W`Zngy~2? zPLiN*0`vF5U+DL&T&Z*8PAj?{tgLj-pcIuY=TC1 zWq}T>xbNNrdYR0xBhX`^4RbHQD_m9taWLg0g{zWWpQLI0{?dxZ@;2Z5V>j0xydIkY zLMMmjMsKB+6QEf5_tvVinfksU>tidspYhk76S~jl@Z8=Q1Thl4HNPN7qCTMTkF<&o zr*Mx0df7*Q>zh{Yvxx2e_(t!m<-aLBShVCG#Gj=+#lkuY{hAc!4>wO{d%*K6g&{gi zrv|9yAY#BK^VJ{VB(qDq4heCT@l+WA+Q|6GjkN}xVI-+ji^t_?$}A^~R}q(`QfgL~=co)|)*{bIwN zDebT|XR+K&Phb7{f>IR@F38Zwg0RjYlUPMT%Kh-?#2Bm_(=Z4$y|CC&il|=zPtj0F zO#V)J*9%S}(f+Y#q*me$+?&cs{k1JlZ~vqk@evvIRCz1;o5sl1wU3G12;p^9jNl%| z+6Z2kNDt$+^>loqJ?`9wswKtQGkIa*aO|dc?rQmRDa3xPwTlVt1@Us(Hq>FKXUVaF^ro<^FI}smMd< z$Y=kao8uq4jlfXZw{tl3w;FDUUxX2-|966GA)AlwNW>GDLzRz6dNoJZ7Yn|JL^;G6 z@%64Tn&|>|ag>1wug(HMF1UA;on03tk?7byI0%hAZPW;daY#K6iF)Q>b0F*<`Hk>L z(Q{0FHbk$!SaUxC7)R-L9JAjX4m>h`3F2P~~AJ!G-V{Lf4ZPP_{>;Gxz`}0%n;zU)$ zDoHXSE%1}$3Jl%3WQ-aS?9nwhZEO3Er8h6w+r&VIu6Do!2i%$S0B(V0Y^XZDvMhvx z^v&Tud{inV%rS!wfJ_qxJ~DOfjKDF1``t7)wsB-c%ha?+7|yFvOsaa0hG_}9B`JYR zCtrwGcdf}Pm$|1IWd8zlgpxi+22tXNuMhLLSE;S&OiwbXe3N}uJKv~S#l{# z%qWV{FloXaGa(5rVd)CLm|r~1?{7hk4YAxYh&31*VG#dFh@J0;MfOhkMLJib)*FNj zX~^OXQ0G}t`t|ls-bGGJI$RKujQd3$06I&Yc2Q6F+`~VbFs|JE*<;lq9fgUwm%4E| zyesu%azyEB^{34V>4&6(qDy@7-1l}va9}SEO+hVdW7*dXlk4&+Q_6u|F#4Fc6#apc zQaQn>#G8>0fnCxK#4wS~yBMhP?)eF6+IVa*94_2esCZHgjZT`vHQV-$w^_q7z)&s* z>?@eT!Q19Jd(#7BiZf7}d(J-a+=k~s!oi7>ogFbHKXgQ4;)>ST5xKG<&yE5o2f$`d!|$z=H!SHSEp#5H*;nSL>SD%s~GgsRb%GMh5z`}j0TeXh6wZM05p z%m#u6R*-dj3Fdgkz#c(*#@vBMx3r^;E?I3+qghmq$X0R2dQWy6@r_qBBMYDCpZ-Z8Mj3 zv_Uk2!x(w}L89nux2{@F|Ijh*pc&K-QM&%J30IBy+Yzc>96lu`sOm(R9s8OYH?PeVT11b&4{CSHk6c5z^(ZO>3yS{t+1qao+o!Gy4i2I&3uUkULWhSN z$6vyeJHF4)JHF!_pSv|(#%?epoR8Cf65pX%ApBaHQy?t(?a_G4H-|HSXNuD6E&?il zfSAE=Eq0J=FOV>Qw@uRPPJ7EYj&qcZ?{vVFz?a}F1K20v3uN@ZCg&S>%-@@GhB2%o zKI7tIX5Q`ByjExI2`l%ybBCFcdJco5WS(nu-^K;b0ULO0pIKc zw5F`3uQ!WM-P7_WEvhbOD^QJ3$w=%!NtBBc&Nso&8PIR$skr-<_(@44ko}SK;mEeT z!dbzeq4<@Ql{Noso?3sizTWJ*W;0(Y+?_tRC^L`!)JZ~~s*|S=c`Zfu?;3%7bJ`!N z_fs{RTH295qbmmU`uRLHmEenHSE*bL^RgQ z$#2Q$rPsTdIcWL^NnQ$HmFX@cqG%}*Qs&+L20B0(?nnb{5iQS zPbhWeiJ!xuW?|8jN{$H4oi6HW&zn;TrqxWe|DpF?j@p|+bOBKxgMto0otI%|C9)bF zb&}>U>RiMzn?Cs=+s}daKZAqP^u(uy$lqzIa@PuWOLCBG4WN#@WNg)5CY+*(IaHhB zO_!O3mQ$)#IYor5y;`oD74hZHfGiOX1jF>+sa5!6hsA5P$hV+fKtuk31@~`3qE9Pm z-D59>cb)N|WmuSu74}hteOdIC&R;VXd8gEu+Ou2s!<~P_kA{w%ob?$>E}AjLV!Ph% z2c7k^J)5Ue%c+!Tk)5|m=_Ro;9KRTIadygUS%lmcSM_0ojCWhOaH=~{fQ}x->W^MW zF@mi`7t`BV==aWA0lSL5s?P2MP$1cAdky7RFjn2qWZJr@?$ZZ!-olGghrP>@vD%4; zmhO$-yu4ch12^}QanF}^D~Her%vXF-RrQiN={8UN(tX-Zmfy8<6y56Ziak?{x?K6o zIFtm#ws59M?*Q2tZzuE&KxBP_;AY6TWqW{N`t!5jNy~&WlYJZ%ke5Aqh4V5nju-8tgjR~G*OU;#Dnc$u~9jsZt- z2#&@7%PeG_bf0eI>!x_0yo=5hkpJw0knR}34)i04{ z9`^BLi)%O0871BG4lamfKixC7Vf-1xpMIb7xi)5d_sQ)a4Gm$IVq(fQ6i)qqe2DHy zzLWMyzH=rfHk<@C;9!13<&C{wiLCw2m>>VhaG#C55?^WX-`COry|vbqa&DZ3rf}eQ zT<|Hv`0~g9M=_V0%=fSYcY9@2d0oyhgX*k;I|pljR=SKb8tfzcdM)mhv^&{+%-5DC zIY4`fm9DX*#wVbi+%4Fi|Z>4$)l$h?>Q9FHe#^Rn61D)==j= zQe(|OpIKPI&@>Bk*DyS?!pOQwC(>5L`qf=Muv|Vt^HZHkXV@~X!@JY0aQYT(sw9O> zRPyifF3k)!IeX)l9)c~D`sR)S9tLs*n=U>Trq)$a5tZigJu#%@Hou?kiwysil!`~= zy^s4`Rqq84(4{7K)--I;8|!!u%Fo{+A}3L~aQZN0Vp-$V_5V@z&B2v^OV@EGwr$(C z?TKw0ljOvg$+RKwysdz-^}xYEUVk3TCV{>6{3@3!KF!TsGfJ8h}Q&va6%Wp_E_r6gGc> zdrPfPQ=BXE=QCf{OW)UcC93!*n824S5;G>AaK?6zQQMhc_9ZaY*otP|kyO);?t<4b^uTse zIg&9poCr&S3&1x23;qs^I@Jqf;Us$82YeNx0QQiXeK&O7nV>$OL_|?ESEk8$w ziFu-gt*L4P(Yl^-K`zmY8nX?qMZ>VHpRp;N_3z1PWOGvkOI^1A#sxo&MEPKKTC{DS z5vRar(=kh)WB6LLob}sA8B%jD*@j&kd)x(k1D;bS5{rK&7{Rp&VPwt% z54degNdDQ0W2g9~gaPh*g)r!7#XSAU#c_a?f+wo(kzV*wo=6(rr@oyKt8W^kFSaLh zvrg6OsgG5&2vLI)G%m6XG>qmHbeonXlnP#A9CV^&bcuO-lh)G&MbE!HfXc`OE-MR+ zq5^aUkVu_D?ZK`>O9KJPR*}TbmdQ(m!}D}mN?S@cY#Z;nC+z@9%tdBKphl3)083Ix zQUN>sLrmz461D}l%i;el_s?Ul-#VS&_v0rs;U|^9i#{;e`$zPU6X6kKWFC+lYL;W` zU(63EX`v_1}aXf4h+ct2XSDgY@FQBSiKzs6m~?N z%s5V+_P7)rtQFGjr61w_uF#gjBVzQ9t(U~Dav*k&^oRbB;glQ3q9?8kbNYGpah^ox zlF)yI?c)7&^1RU-rI|W~R6nDgKHw($x4=t4O%h$w(y+mi$%}-yvFqbs&xPAkNdcj8 z>_qg8Yy|`ah~GaNNlD_?%SiI7YzXg$@qK>|hzFuU>Q#w)GRK)+BYS%VsCu#jF-@t# zTYpDm%MWr|aS>z%3s-@&9n`}+!A?M6I2+Jc_If9*rK=9eP3)7KIiaxp2~S@ikhL=+ z=U__AOdI{@HF-lXT@eUJJu|@0NX4n~H|@(6OTri3Zis#6?C$%IvEILN3W&lkLA<;g zVM%ez1t4j$D`mq*ZS0MH2K!VMP;(Z{$i+Uy`oKyyrLT2R<%+;c-f^$E!d^fK(BcIF zIHhr%9N*Cr2R)Ia-Ma9u=N{5rXkHj|40MVKkzobQeI!bTN_87F1D+ z*v$rV6o((sbU@reS50)qX=ZQH{v%fi4JXt-n8gcfv#rf>AvdVKhMqg72bzmCG%J;` zJ~7E^ifMfzqdQ(}7}3Ny4JBcCrp`!R(MjNj^;CoujL{wUdtUQ^H%`E%aLd|x*J?s^|vl7d{G+CM(DDh3`Bpcb!O=TRy1|5Rv29=s&V;k2TvRuQq`GuhxlaQ?_ zAi~G0NfCq!y9(1Hbyw)Z2+)8-<2-uO8kP%u%B!dA)m~4@T3p(%O(Sxxv zQrJg>_z7RYnvb5#(vosH(JyVb;*Hu+Fj^6XMZ9>tN~CZbd$A z@sID4P`|bYgrVL7P(%YuZ*ecrJ~@~&{}$Xg8HY^KBIRG;1b2pAEIK*0Sb zEttrQsHEin45%kP3pSS;w|lJ2pW8vb(>w%=qWC;XU>#+li?W7X*EKdaZh5n8**p$U zGi(9T<{cX&8slHSjTqQq?s;q*_VpKqQu+5DNz*(D6KZE|ppGQ(l0!uSRFg}7U)lxA z>ar8ZpjJ3i7Rp(wI$c6K*Hl*tb``LtE0b1}@$;8gaL(+n3(cgoVZ}pLMF%{EROcN6qUNfLMW+uw1Ps?V@6(ULg5*@c2C27GpF~bM(X4+hI7Lt3 z$DWoQ2uH3vrTLX<);G(9#Ip>pvZy~=J1z$`6(nFOd;;v8p-2RNYKAJ={OMJ5x@bKy zDccmN%eG{!QEkQ6A*3RJdND! zGTn@tGi>W#^Isn$?xOHu=@{^&L1@I8XI=q2Adz%5TId)iPd zoyz^ub$f$A0`tE`RSvR%&RaWCE3lbXY@s-t~C1F@_*~@)+Gt_9bU@#CS=HbBA%RkU9>>1WzR50S^=?X zu+btVg9@qU6s;^33aDX?h9m6veQB3zyKCkl8)c5#O*r++p=0n`h5eGK4@s^2% zo}hb?f<&2go90uc%8n3E$K&3E5J1QC)Q$3=ZD<5sDJGj*D(sfXb!s}Zu$}Ew`rN&J z;1<|T^HjRjqv<&HIE?@$Z^MvmHe<6g7P=6VMIbCA_+_eUO`7SW6RQT;d=QH4INg%6 zCC!ox-OT%tfg{{`xPd(`4GwyQGcirhh7k%hS*1qGG<5^!GSE8+hPD#hWB}>}qfOTC z<&P7GZfvJRHfb@Li!*Do>1L&J?A_ZJ&W*OKRJ{ebd0m4Q6&%t`G)8PwRs}k;1j4a( z>gmVTMzOVB)e6{55E_yko$kw1*!El?y*!DM@2StITzIcQL0pO5v^K{zT!!c)8-}Z~ za^+TPiI&CWzs0T)()Dor|Yf| z;O}uUkRo2u#=(YP8LX(@Jr2j^X0Q*ga4N9c6|CP)W9JYFvfa+P>;Y};0&a?oHa_>> z^`WRb=B>8?et0$suTpOTVJfA+6?T|l3v;Wpy)$=Ud~dA{5A|(eOS2R`ZO3AApq5O% ziB5fkgQCdk4{M%C&wfCc(tKv;E$Tw5f%y^++A5?!Cm$WG9ugP-{MJvTQv<`4FIErs zmgc#EimyZ(o=Szw!2mdiB`AGFOHQ5Mv5IE}6aVh!=nS=B{WG7mW5P%m`rsMMW5Kg` zs}@t#;lI`#^0&ym+Dz)v%>eX690l-j-wib#fJ;OzvxW56_}5xSMxx;(_=@?xW@xo? z(yUf5yOuj4>A=i)Y$gl42D&NF8_THFJ|mm~Hhzh05yi>zs0L7UZ{^1-Cv^EKwGb*G z)O^9-w?#O&At>rYi6uq3`}0q<1^sGWEf-RaAii3jn?y0}_=c!)%jvw-!&&AEy9w z>Z9Acpx4ti^(7$joNwQPpg;Eh?g7~B8YTJ`lC>M~trQnCJHcj6(5T1xg-BLpCoG-s z?;^|}>x8_24WTs(ov4cQ7@r#OS3zAQK^g@uMHMnzd}RFW9^&^r{xC=<{DtDks~5ow zkklj5dmOGB8m_8?WUt}SzHoqq;C>{+i59kc4mBtAdoZA_yIwAC0}PeCg}Lczo2*+| zuSAy<^X{rPz_mqPb*z>dGLx^~G`RuufC-AYXG$fAxSG^G<{xCM5bxXAML|qZM0_-`vclplbVf*Bvpa&5#bcz|#*+`MC0LA@!O77+75Bl6~~SyJF%O)0M^GPg!V(*$2y zraq_^Hnbu9RWOb`V(jb#dNWnTbEd77`fPYX2`R8p|l<;A{xR`zmOH5RqT%BFy4DAffO`S-Y{yz{ZPM%i5RYM%^ zwok3AgKZBAA`Kx$4jiQ?r9~_hM+Bw$;rNo@ZXo;4Ho?E*Z)ldCzL9q@7>VJ0T>$x= ztusp2(h4@+A;MrtQPb=2xUsK4#}5H_GwTIVyIHp#b}oOM_GxZ1$(;QMnT-AEY;-oD$~SW#Xh-R^D?v{Kks-fj%CSH71&Cj%m< zSxE_=z_pR&oE&XxiLqcBXvXcFQi3g0QGK_PYqM($wa%5YwQ2ZwAYAQa3N<`US|AOP zQ>wqi1$TqFG38`TPGeh|I$K+%@{}CVev7+*_a{&qOgh1IDWyWD5!@#*AgW}}9bA6a zTi$iuQH}&9qE|ce%uJ2#zTotdQW?Uit#LqQ^SMh<=Fje&XfSF`aR(4{h*gc*ZB2D% zCCQuFVd`?xp_e5pyg6;2EX-0b_*TfeqIXwjC6V2nA-iq!dR@gF#~CzB;m^MS_fh9l z2n-#JVoqAtshNPHv83k8+GFg4ft!X)YaZpw74GbMap-g(EfbvF^#zl3=gR>Dn5;t^ z=U&{PY`C_O6x9t>$&0qts=Izy=NXMq1kJoww$3VSkPTnU^VKDE3Acz*7_(flfN@cV z)S~R#WAEoum6N)nrh1k&C%Q=h^E=X?oMv2i`zC|c8Vx!YUt~2~-8PiuWVC`a0_1nx z$(Yn}2z!60CID4V3w1Mj>37Q6=j&_?)*(yvB(cQoC;T86%|?Qn>UNKs0}>xJy|JPY zxEeTB5!!veey{7gB4ZD6f29bfx&z8IWotABYLtO$ZzQJ5HS#6pNP`CeD-pWnIwHUl zoh9#=3)6jt%0y(1-3-z&^dE0m3%?bvbv4FFF7P{AbViBEOONOP`0h_^MJF6KaR1r8 z*1`hVxR+Lp$%>|OT+@2>d!8eHIsUAKHhSw%2Hd%Z*4DD8d+zqJBb>^D&@RIbLPAjq zNCGCGzf&{LSPHY}XkIt~k)Th(j@mEtur7VXmvC{U`ve}NoaEBr%2o&ud)dOF zf#luBoVy{xUt8z{=ml#Q$j|lgPc!{gJQ4iz4#jLiSwDpNp@S8Rmk8szB^!I5T;pOz zh(JRHjf6%h@)6sp_)SRbveErJQHML5e5L&)@(J5);qKYmR|z*xlyX9;Gk)*1;*lOUGmMlebw!C+Z(B zbf~vha8CW9qCD-Va)2#!?CSJG_!|KE@z=EC*S(PS4=$XJ5kRo;A%cGvjPU1oLoHko zPvRdh6ky23;(c1RBv*KkWT@Aqovw51WLJm3XF`YHJ|Gn-z76$%?BahL>VHh@euN+` z>bE_W{GYlEL|QJK7Dib#q5m<$OnH#HAaMVf^!)(EH{I_F4x>~gEoRI_MFsFgn;@uE z(HatNfUo)@aqBhcJLQcTwHuK#)SoBvKS%NVdv_OjJxtBEMfC|H(3Rv>nk#jRMs<9M_3G&OGW@7`+`$>o2r*(1an#%&pHbi) zGhl8MjM#!=gf&sHku)SpdG)A`w8l&+MJ>bJtCT*#|M<%P-M;V5{=fB;-hNJc>ECbn zQ#VT>tpWdz#?hq_Iw_&`dSQrLQpR^){V-AH4TP0R^TWiWM;~1*HQ?9GrqRcznQzbu z-hn@H9>nyc8cWflK=3ou65Y(r&AZY+%*+IQ0bjs9xaf&t#*E=On9_ML9Eq4iVS!AL zSP;k$M>6@cHPRI!Sf9T*cd3Yck$V`?37XC5!{cbXs;`S3#p7(-h|JF|^#AdAal4N@lizU08Vu zLzC@`@IJ+XBl+td3-)A!)>!2a`%*&INfi%bY4%d~qrH~ziAvs;7j)V(wk?Q&es7#f ze=M(&z5BL#I$MFn1)*-Rgd(ZM!B!j{yhb-yYzhq zPOZ zpzJ`Nd2n(+O< z$Mw5j>;E3)Ur{1Ry^ugaF=(lVtl#MCU%4hJd0b?G#u)%r1Iw4Z#gIdUL6Iuyud!*U1CI%Fdv-b;l%CevZ@r4&dZ)#Dr{I;pi+_WG)0Hh& z5WMd~e!QpvKFLnsrexjB6{g~ziR=$2JadnHraiYjr&a|%j>gV_ylz5S+pfz|-S$RV z(-rOj*DN;t2yHaW(3 zbnIJZciF66kw;fjv{*$pQ9X*G`Sw{-W9!LUeH_F&}9$e9p9%6%) z)V?LVzgfK;qy`fLmQKXB?onB@$EXyi|9BMxjL=cnIlRX4YZE?K!Q7<=Uyl9K9;F(F~oG8E_~M zfX~L)Vw{MuPsURiG({88sHZRMw3Xl_ z2;-GCUATR(iqNN&Jg6=xd{ayH=2m|O+y@(LyGNvVC@@8=`a=6k@ICuoWYQ!3ygN+e z9~RqO?_vJgZ0l4AN3wnyRL;+2;|@SHm$9u>53$VT(6tVMzI&9j>SY4gPQ^23I)lqwwC$vHyvU#K ziJJCYmo5y~;a_@K_?@j=>O}2}HoL5C?8Av$7uoRvZKr)0+aeApX17NISvUyUPtsO7 z?P)mU5Il@~Hd`^OhkH+2tfQ|8@QG2W;b3ZUlmG4s6W1lMwMaMo)Tc|}1Um;dm&B9U zo^P!@cn1AgwZw2k=2&VISM8Fm51-pFw#->%8p*b}LAJqj=&>kWeEw=?7>8?db+H{- z{Q!3{WJavZB$YY#YejLb;{@db{zkViwVllZm?GR$llvJ>N6 zcJ>t>?pNIGwIOfG4iExDo_;@&6E?&u?m%1!QipcGVU%UsfFm3QWAP5Dyd~%4VW&rK z2#d`o0x{bPs)1FSp~}7?04gP<(Z+0bg|o(K!Mb&-13ohXBK3ps8M`8F*#RB~d#dpT zh`-E`VY6g+8QV8-OIZLCM@&K^CT0eBc@aY*OM0$lz$lJaSRoeG%|KJJGcIN#UsA8Gw9s~wsYYjh*c`%O$ifET-1!T7KXY?Cyq zGD++!C&aM@SXADzzD0iGi}Lw@m88vIPBA{Bq5WxoCZAf>pQ{r0$D1Xv$1z=3#> zX4I8K))W!+7QWm2)2m#|i>dQ%a(f|#Z|7RN{76nA!s8a|yl6vJ2rb3!)*G0XyeUn*<_Le*q{@l0;rcW~(vRPrCf%_|x>2Lbp@&u~27j+6 z{(Oh?GSB!tZYG*G2^_gkV?}8z8@a&|dmk=(PA7c_*b;wAQFWyyeQArnt0?nsRP8^@ z5r1M)ecB*@x*>g4p$cRMZPD)LChlGoF*rs(lkqbQPA@*%>j&9i`NGM`wC-^?M^Nd6 zDK#xq9)n~!#H>?QOv#|W90WO&s%G0D=CRFXIOVas{E?gc-4cQHJw9{}+vQ?ba(UpV zowC3tfXr?P?ZkKPK|7{fdq6*N_vDa%*dFEVB?MI=)%OqcI?2#iAs9gpS>*ktH}bSw z{LwVWets_gtQS6V#-3jMF#}Ze4H0#Uu>A>#y66K4T6E-^OtsgR#dPNJr7s#!O@VR5 zuP0_V`uscYHhH~*RhXZEXKJq!H{GEcYd(x{08#L-Ss^idc%v;bce3r;foG147oN$F zNV6~h>}QINqfcGy;%wG8PJZN#+P-tGlg%M_esF~u3zCdanHh340yU`%R5SvG*#(LK zQ{P*_KX~>(&i=n)z&{~{B7|%d@^{-G{BM9j@vq!Z;!7-kDkBp&Bp_Q=TLDD~^|M43 zBBff4dK2Jt&8z<-()Sqa5wX+YGNRp~2& zDQ6X3I;zNuo#&`wE9-lnuldo&bZRXy?2Z|WOO~pPmI$m5-AG=paUN7D>v?Zp1039N z{GtQ4h%`^XV~TOw9ssw$7j0gOtHzYOMV9L-bwXmd>xh1}%iwr&t%gVpu?p*;#|?Ho zS$*{`2sVn^EGVCea@Ci5d`3kil`@HhxBm&Ra62@9RCRm2xhb#`7UiVv_;vKn46}gC zJ*RBwt{K&RT*kSNCc_!t^Y3j%8P8KZ zE<;^2bFhLe1$2!}LBn@ehe<=tZ>(9^*ZYjdHWCeF-iy_eJ{0xXfOCq)0=(s1)2L6q zYWKHnAT_$@Q1L-)7aazRb1E5H{To`$2`uxpf4)XB(BNQ|TGjOLQ<50Oe~OU@D<#>+`MXqvp7WgQh$ zDjGh8g{;AZBmNqX*rm~)89RCg9_uv~P13`GOd?HtdW#90uMiqB53NR2s_a@vN-+H2 zY*~18*GuHGYgE6$`07xa-9g}vFtm6&? zS>+Epp!{*{;p96I_TRnWSeB=f(v|ya@8{V#96}j`wrpclF~A^c|xC{g|Uv z31Xi<5>WGt$lNF0=R-&lUP)D*VMD%osU$Ccg@Wb;rpIt=4zN@SN^LU=+k}?bI}dfh z&*1;usQ74brug7CzX(r12UCvgajE7FDC=agcw5cp`~5f z?E7%IsEeushM>$l5|F13{YdN;v05$%wVGf8^I?tu9Vb#cw$Y=VeiPB5jCWH!i7)S7 zUqF7&oUjHAejh;am`$Wwwc^syu{S>{zZ(EW>Aj*bp{(amRNXpvN$x6OY>Yp1v=8Ss zQg_KCM@}(OhRy=;0v{JDum&$3yP_6oU)K*wqh6^GAo#GNf~8osF0&2;sWS5CNxcZG z)r1olEj`-hy%tN;aamfS0pzoN9XGvaV8vI}Din2PN}LlEUS^8<>_4InGc?0Fq*aRU zG(4Lbi*w&gk*W=U%RorTI36Y+#&QZriW_B_4H+wg8Yn6isFY+c(JWxA0*A5vaI!r_-BLvvyK04vi`|!^t--o=3)T>F^HxTs-WSd z0>-Hz5@qb*0E=2s9;%BE0!O9^9O=9bLVkwvK)}N20Y82K&mszf5Q&oe<)Md;$}lsh zhqIudr&?92S1(ng4K5X zwXGhYxq#XR4h$BY>?Q@%TvqMuCVf-g4BJ}^_wOo<9V3D3f?xi;>`={qP22}aNfMIy zx**>96Cnpk@e-=7=jf9*=)D~69x(Q9OHZF$pUBl z)8QcYuCoxEITDNBnPV;OWyx0~d$^w+Rj&b&SNrWc-W*$`8^4O)#4xV*GgUtmT-^SY zC0?n$@Kao&o2+Gb42$V@==NM;ghA@O=lKcj7gT%KCK9;ars}?UrWON#ydvm%o=qBHU?+LFqf+8WV5r-FgTgP-MTm8h%pEF3ua7O@}GxqMdn) zyS&@r3k+{L+;SXz&`}7~-nOWH!Y3{v!k&I-aX64?No2YF6@Rtkn^joehgt=Av9?v0 z@SGwnJO?WA;od`7f%OpB)LPa-U`K=Br2(=gE7hB;s`OV!RM$QH-5XoTx52Y0upu}P zLl;5=;>$QNb@$+0&Sb?R(HY9MH&z;mny30%@3UNJV2t7(;3j1-YC&}Jq0AhXj$=iE zlg@5hAUkI-85L8>GTF!=R{FweKrgQ9`m?x|DM~k-Dj<#!HbD>DFUq( zzr5?>d9BP}#ExA?pToO_XnolFUJd!_eaX^7CbQ;fSIefbrljwut9jieyzc z4Ef!AU{`Z7;Q|J9MhDLFqt!bn*YLvFpCY-cd0jtt2`3bDIoPTPIZs8Z+s-irGun7K zPIu6i!hf2L9KluHgutO`2L3P_fMfLC=o#Igl7z=3-q6*}cTlyDHm)OeMFcGMu=2*Q zH`ic}66KUFw^#n7{_9kXLQ-}(J90-9B+swc$`V-0;^|60NfA-69*CW2zX zrAtRfEPLnMv(XxD+DwWgS&FuO};>5EP)CvG_>M}Fe>k3gB9IcEk4^R&X-)UVkcqg2|s1& zKsA-nt#h?qSwll{p9D%D0~A62o;(AE-CagV7biquG!JsafENHbCs-bYYuaf-1l_?p z8FzwKn`XeEaL+=EM274t>Bj9mPvKH7=W<@)I%7m@4HSlQ$hcD%#AdlcDE)+Wk% zn=+Jk?4g3=3C#eur#6Y$b`Hq$Rd;!CsDCyEfBh)7kf4~G9MtylN{v^pcqd8LSAH3$ z{t`XNt$1fi)>nTyqW%h?dRL-3{khPmVg{N3+pE^ngEm#n0vAU0{*c*n|(JQi=l zH0#a>xN=?SdU$lOE+K)F-j*aDrJs5`!-1XEi+#K}RGS*0$X5gM`$*Lnnf(yTHPolg zvU?B;e3~`i-$-pw|CXj~>23R7bXQAx_$aSaAJu}M&L>ag% zAvppuRUId<3;Tzna5+TcyxB^1q9ZF7T{K|4TLiE6=ITT)TK_=6;$6a5Y)=3xlVmq` zX`8%r(Fh&z8^7QUBKm6r5H2*x)T9^?7eoSS0d!jXo-qX2>$ZMJ@YUTTc@G(~zt!eM z-$Tw76tsVWIPJUG$R6Q1kDht9ZIjpoKr38bw>N{8TD;?bi61<%efoVB--h1uL!-@F z-#%v#VIwGg>o9|v3MeZQimBkqGA);&f_0+1gnt3jhu{*(ESqAz(!|ZA16ltn#9G_~ z@s&|=MO0ABM_)qZ-pBFk6%kmu5L_%BM>eSxz7@MNXi3-X&K(mzSl-j}mCnN4^T~~( z+^|{*VHHrl7_leD(upi2ffOqihMwT7QOzY8nU+<+&L<(8wu&ejFNwlvg__GQ zqm%`AtU-q#kLNORbx>h!BKvt7d?{^6Bb|P}L@JkFC2PoiG=L1kLsP!C3wMTf7cDgm zR!Vg+rs}C`AziE-fh}7|K($!GET>{6I&uOSgH;pqNXv|071VG*T=NR}q;i@z$(XAx zToig6vnG3HLSM`2b(w1r8<%OX(q5Wdio>UZ3n?gG%1{nUpJT5T@Z+s@Dcs!IIEN%M zO>yAWPA{Z(PCqo+ZM8%UacoU_)F+Paudt+w_sZ~juaK@TnNT_G|HZe_+h{JATj>ZO zQczBwXEpT~4HOQ+%Bji9r*a(=c1iY%*3An2!St+~P(eGFG&qJFeVI2+Qb)os`6Vl- zS;0C$Y-!|TUV%J#TwOe?SdRO{*B*#W$g)X<;Ws_7%uaoASK4@25$$}^fplS$x`lLN zgR)u%+X1Z?3fnZXb+FH{u2xKRx|TM;>!w)ofqD=We9&Nm1zekyWBtP+y)S~|lE2Ny zK`n`>20^M-Su`T~C%)zmw_HXH3C(YK(+oFnEZcxSZSKB&I`RxO0!uT3JLEU*{!@dW zPr>Y&vi-<=&0adh`%W}`QGsb$KW0&HUEa9Qn<(HN3d*_lJ+w7gO?ugWX=^(I4hMzQ zNDCrePP#Hew$0MgOb+o~_WVjIK(rm$Z_Mmh*2;}pc2h{h>8V`h`0zj3XN@}y3YA4; zGjXwH7O~ZbqEnjAloZ-(!4I&yw30X^iIc(UekkIOj74J{ca=GAWyP+BG`D7+XdXym z9FHdZv_)F~vAmW*cXt!3|FU8Qn2?Ig9zeCMGgVj)5i$p9jD+jw=?uZ? zm{jX`5t2z_xM${&q+kW9E}j4>hg6+QQn&iy+h9yciX)Xfk!;|QsEcbTZ^pGLGp|@s zrrg8PD8MlB8jYt>%DmPBD_TkIQVhc7EW*)6A~l7ynD?;e^As@LK!D%?I=kf5iPvY$ z#9BjEAmr!;hWI9D8O|TE^(W>+!a~$Z)$2BsNFl~7EN+_?*cKV4Jg{;cnNZmTR74v{ zvG(7!Wzu81yBkj=wHYhfBPR!%>Ui!HE6xaVSb8md)_;9e2oB1A=9+=QfK%Hg( zIF~IB&-hu~a$?ALJbxy#AtZxj^%LQEGYr?%c8Hzg@5P+UmVmOb!fk11lVL4mIe%g< zkq5~kFb|cjpmb+D?Fm6SLjL=WCP4@EUUyg~B}EaZ3br>1X$%$yKq#jOMm?R0CTTz2 zF>t*-gK0j|$z5wfz)C+$UIyh8q9Yp@TM8ovhN+FaAoIP3u`667$+P*nuqy^_liozg zGz2Y4Q%?h;%JjjJ;5FG9QD#vQv$O$&T?x#^y0h1UH!OUjsZjhlRlkwb0wxZ zzgjTX|(He0*n?EFj2$RD7^yhWdH?JwA#BYnhhE_l0FUfIG#TH7%|z zx%2~!Qm^GwKrL&NT1ge*wA^G^3?dj4b+`gW%|bj&(Pmx(5Xp2P`}7lzhas+gx^=;B zo0o%4;-}r##1Uw2GWjS%l!Kmxh!Y*6#*pZ1WzdSN(>bL)txv*Iy1r>f|L5efA2D*L zkbSkveAbGU0gtM$tO?6TaH+_vetUPWr;b2aQ!MT3CB$B6EdTjr-}42#uPYbBR9fs! zYRXAMAaFMtfM;JX#Im~h!^LSzCdUfOU810Vbcl7Rsy3i7h&wt?DL$mI73uA=4~RZ& zdNk@+^4vZ$`$MVirAOQ+>a<{*b;)nnggWye5l2lQornP=#2|?yQ|jZ* zqGe+%@bg3J1M*m?De~ZQ%|pPZvpdVL^rzis^o`GgamUj<$Nb7F|PT1>WM7%p^mCv&7bfzJh_X)TW=mz>%n zmp+LrKv&(d#yq=lR-V1dax=wIG zesuwO@Wm2L7f9^Y5*oZLg!@DmT7^aI*myxM6qpMoPK-6HOIp8>)-VOXaQJ2lVgN-E zgf>R&SgK5>_cdK{M!v@f53!%gXzHT=$j$H)KsAx9xD={dr0g%PaQ92~7I;$q1gq|m zmJiZkc1k%;5%@3oXJlTskqD27@n21Zg9Yx2b$AsKJBgb*J?d4VJA`=$3ro?P5k05O z!a^@gR_;#L6882!!?LODi<9DdwM!f42fMMFNv0%k{YAa7CJ>|MS*esUvw1{b1`q)Z zfJmN9UW1G!tWOqsZ=cn%mY}}G*@;)xAEZit&_O+6p|(NAEs*mYXl%JHKV36pH4+~b zI!INooa#Ax?cz+r?s8=YpK--ng>Ipf)!9}w```;mEl00Jy4 zNUQN+vFI10VCwrJom+SZ#!gWI2zkQ;-Z8fANNVa~2ZZE@nIfwxZpp+aEN>XiRHD_w znNbZmWwb+(SE!GL8HxCv51XceFSJ6kc%n#OGr^9iFI>TQTzjHV7z4!J<-pbzU!pLL zcDzd)_n4WShH3QGq7U7bK^TY&KrwlTT>DIkuEjik&>_I`T$89%xSP3^PITb)cYc>} zU?c9@uI%&HN=L@q!y0N1!2IP5ZJP_jc#$b*cwMN%n1id~wQN0mPd%3;*;}8wGv|n@Jc#J zl%aLvXT*`rOm;n!CZ9MM1->>Bb#0l0Ifllg(jtngk6(#et8;{P)aE^?i<6k?4Q!$F z7;wg_2%wSrm!%s+c3a?~>d5E8;|y+)bc>hYp*pXy^IMr4B--Olhn5DyWbNdJ9LK|q z<(wvAHcj38{a;>9O~sw5v%>El&m0Hpm_JPNYL*VBoZDY53~g(Iui-Q1MIL@)&zo4@LU!LR54 z3@9O*p0ny{hgS*&g=nk-X-avRz2&#!6{Mja!p$r3QCvcQ-l|mGsJxfhv&B9XG9aiGrT94%WFY=*eZUx$0vS!Yf9nw^Z$&D>y67zrynS>+0khW(shBeb^$$_uhtQC z-vM-N^$*^oA{1X;GrltY?cR>R*LE`|k(k@M_RSu4ppV@{(ur8|=zq@5%Mbg|yuQnJ z2y`~ALpIgrG&C~y{dG>|@XP_mttcYOXl`HJShY$QCuZcDgDEnRKGTo|MK}JxUkU5v zYZjVQK`95a7U@&>+*wOSG<1o8%VjzY_Tm*&nfl=rqg1-!sPvAj2G3NcL<&dw#6J4W zK70|of}>GaR2XNq*I(Gv23&|=zNk)SV2V-UAT5tX0Z2h!{BM+UDJ*PsoO5Vd&4=}o zx~1+GV@i^FrhW83fYC&aT%;7?t0sI(6kj4W6R+RB?m} z5-=%epyK@P=yVk18$dvH7gzMOGU0bL9bY8;qw9KvB}0NYE#qhm21mHjUbv#vLTN+v zOb>FT!y`u`5BOSAr)t1vT|1jL*B@Q^T;R#&FN>W^`3&&;O$;~yje^1^UQ^HXbb zYPX*&cGPQfdkFI(uJu*#-lV(a$PBkPflW7#_a%751S%>Ii3@Fk(iORROJmen=xv=v zSJgF95AaHvfv*>~;f{6HL=?5f!u*-yx7^snXO>90GH~AbY_|~Hk<+0}HbuYJr8m~) zQyn3FB0gML^aWc0J3c&+yG<>_OPly4>J78VeAF&jatAhUT2R8k2OSELs%ltoz;cR6 z*OB|7gqG{JTDNw(X+s6sw=|5-^-e6}WYFEJ{z*`tN1OOoq1Skw`xt|Fsk`@=Ltj7N zyNMkN-Q!w9>4DBm^q^%EBOXu#o1w0n|0W`+9C1VD1wc{(nl!jIKeFkKD+m zmD6eq$6vevZNc_M{9WleJJWLr&+4ye_N64=ke!pc5+x243J2|;?0|TT8+pC{r(3d) z%-VxDPac7h_GhBYgU)x>t1|@NcjVgy{m5qvwC!`VN=AaytTZ;U`60Q!FV!=+N9aEz z5lbySQDlkuD&CF+y2_asQz7Z-`VoKsX6tGOFLcKO_;XZwZ|6CYbBLmIgEM{m>7)Du zBje%YXR$-jajKDNen#)?@b)p@F-gW1`KpwIStcaS^;IQ4gHntB9!K^KQKF9xVII1b zlFXO(rntU7_7h^8S}Z!)HLU4?D!w_jM)og2u0jsSveYTq^>)J$uNE?*_OwvpqO>xy_N34{ zRfEx)!N-dOjdg5Q&6{s(eVP{@r15ES{N%Z!l+wE z$aJqR3lL|gqZDiekvv4y29+$G%+x7(*#P7lXJ=OtBexZBVg;Jw^hyhuho=^6 ze0HX6gN@T}lzd(DUp+o&sZo)(Ak{Tbrna6TK_!>B>Y1Vyu?IIxCAsEM#;PuoE}EZ@ zpUjq~lT=k#LAi}|bz%R#vnq8oqfUCLG3)8VXPdWn>`i*5mjuDTc6uA%hR;OLy21D* zy&nzR!J%n9es#U+p)|!noH_~GIKU= zj}uc_SKdcF=7(o32)RJAJ6b5LuoV`)anNY}lbF3_O+!VldNq2Aj5P&FBG3)g zf2}il%vN&-WXi0qYd&5S$Ry%1NdSS5*AJ+LtnRcSimPK4h%N8Kh9?|cuO+{ot!u}l zVkwg?&s?Z&H16}hVz1d2m*X zALQ&1+p(?0eh%-9@M$&U5|!gSn^|b0x7GhHvkzTROFV0q*M{KULepEN^#ZCwWGt2) zu$-~A;th@~>T|wkiD7b%ykO^De^lhmp3hV zpkw)_IRj=$|dxtSJ)!UE5tb;b{8R*mT_p9 zI3WdtWharKeKEszT%6mFhZ_YYuL>ovV=lR4a}b6lsr_kXD~TPqeh0uyAz{7MLhnpM zuE+ohWk*0msOS@ZhCsp1C2$7+m4;^#QsE60DsOR=+ia}43xAHr%YCAjPf^|iF#$8o z%l@(1MeXelbyIwQkm9hF;e~V*wIt7sX5>Pof?+fILm0h5-=MBfT*wD75HA=N?jFGN70m?2$}PbwsusLa>a4q{PubNaf~B?AoqQcIp2gVec4S zS<`lncG9tJ+qP}nw(VqhY-7hp$F`kLI!4Ft*mi&E`~J>(p7%XJ&KT=QjlF8DwfEXp zvu0iEy5_XbVz@BibtYP_KKHv0|0sO0UcKw~+!tnX0S#CU$Y?)Ycb)Q|_ObkUxU}W` zg4f3@%tngZ=RgN`P1hj2Z&W!3xieY_FcQQZKuJ^QBP6>Pfyx_*Ho3RH_M#R_od&zs z?xIWiJPYn;FicC+gCxe@L*>y^ygMPhuuDtqhi-^kSsT5`spiKT0hu^^MUTFP5$-2B zSc%h9x#Nq|Q@aC+!(j7N5JqVIR3DW`HS_v9GL?sod%-K1bflrg&tyx7Q=bAz%#9;I zt4hrcP!TqFq{wNEZQ{2nz9|i&(LB^zSB7oR?lAygi=z7RoyI~cv=3h)sBvebz*9n7su(DZ3s(slaDgnb!VO3rD}t(qVHE$23i45-Q0mvK)}Gy zTxJTD(g-IC>tYFbn*;_X|A&}V)tUO0B8b=~W_stPL%B{XQez9uNxK0QOTu=@8)Vn< zpP9$SD_K9v>|Gm-dKmfTt5CYOq_iaz`g9EP3-S|NEg|H6DOi8>rK`jN0^12sVY<*% z^0UomtFnG;^X5`(`c#OS9LQ9X8LzHa(#Mh&U=+5usx`^|hyX2@Tb6K%BPVZOW~4Dt zMb^Al^;vICVZ*eGcJCz~>ZYEMTT_T|eMH}7$>pgk(VCJwXvC zszUUZN;RevX3|0|cC7^Ds|vbW6i4OJF6Uz%D9gj9Re82uDx1V>zlmktlrZYhov z+>YlNVX84jDUB1{nWkYlZ42TAdzHud7FX#d+dt)tNl+%2(SRzX;a0V44<3u_I@fBJ zbu8`b!q3UGeD2t=k`14MZ=q`te=8&M`SIt$6pR z9LH@oYbWkzE%gyv#vpdIP(y==ckG#mV#--lV$_Os0KZl=pj1l0GS*570hHD{!mX22 z!G7P&bwzd*lbcwkw_$%UcvK!1GK=J^XLwCw;8cLwE+ zDx7<~U4Ifx+P^LFESY~y(Aif5r}}V}1_Rlx2-ZdN zS+Ni{0TQONwp?vdMwV1~9kHu*f_OTjW5LlO>o^G2BE9+4z|BqyqgS#skC3Ocy}V=R z8i6RDA?zqrw|%nqcdUUgQ>I9&`(;i#9qgd7Z$9HqDu>i;v%fWXTJtXMozRgk+7zaX zFh`G?2w9t`M6d?H6r)gu@g=;Mq%ZDORmxHZ0oX*A3EAO0{_!jGYH;OTzg3y5cG>b^ zOrI3*-f~Px=g>biuXFmaDJ#4(_MmR0g5rgdE__Q~7gN6GQtUHZ zIN`BBrMq3{7oGDc>pF>4&5QUor5V+i|!;Ruv1+YN({G?m=Rp>sEm1jWv6LF z1!#up)C|n~ps%fJD6yk;!&kF+1VOf0#n%{j1~md_sJ?BXp)2!K(KT4(;)6PMC~{Scb6?Q`!_h7gr1QZ&g~uFuz>TM|fQ= z9LG8FZ}Za($TN>?SdPP^6z`$xMaJ|T8pe0N#dW@YMIvR@&)(9fG2KVrDg_6G)H-Qq zFK7vIp`|{xh=cQmUJU??>|til*CQM-A1a9q=9jC|x3$c#>S2=H{p2eXv8St>l$FLee*4YJXPa!q!3Om-7bx=+PS?~6{w zRPTG-NmTHip=MR1?kc1azKx3SH&yO+*_yGRAHtoJR0POm7sXTft7_?*_>}(nhkyTv zI{Y`O_?w93HnY05{8_-eh?qPLiTPP%1_nH*;Y0wGXy~i1X&~`~2G=J_JLy`%r1+_r zO2q`upjqJ%6%&&uj2OYZOPN0hfs449XHdUW`X31%iLxZ8%dyTMwD*?JAJ@>u*qbSx zT>pHraeq8p*>J6Wf9wUkK^x&g8p9d2e+@R~8<9b(BzD_@&Z-?W%Y{gUDf((_3|2Rl z%T^2ESoT95qVR3O+SPsV8?C%tc|#gueH#-9Z8cl|Zr5N3mSxYuYv&?tC443WRZ(gT zJDBEVt<8!WR(C!qR@&B0Up4erKz|Uu#)={WZj8pJ%Zr3kOb>nDm|3vmNe(xJBVi9; zMh=GYlWtf~YYDDiWqlgTR{qD)ek4OyMPV5LU3Sm%7j~Vij_|-Xw1on51J9>khR<%~ zvLdaIGE$yGbpns^0uC;Pf`9!ED~*-vB#9<9y>vveo%n`t)qebZg~f-`K&s6*h)B^D z6N6uB!=-f&vdb`wsoj8GSRImDTAJ!oT+f3yHv1NBHCGf0lZnppz@!;;Q_A|x74!kf(JF}CcHIOmysopTb#ty|^7(P1(!bjdf3r!U(4JS!Zp011DTfT;vqWWr1z4J}r8 z8~eI4j}dr|XH=WuZKrWMoeHn~Ywjoj9_=B#ze!%HQaVB9H@sW{+@LWP$c^6f<_pg>LkD1L~7IuTfEc_1qMD>k$O`xmSi|gY6>uvW%MHFXf>K z_miH9dvZkrWiUMRC2M?7(RFf-Qr0h@ae&o!UiShtFot==g)d}&F&JXn1b2>Y(v4p}=@nX6FgQ9zS(ApV{OF-xY=3NE2sDjtGGx!G4qY=s@;TXC> z?wva}I4)WRN^V+(Wsy@ZiFG&Vmjh4xYoOJiZqUsOE=hc$_O5>L;*8{8yegFnjBs1% z{*|6BqF&)qi!Edk0J7eitIwj&FuOF-9#rCTVKGnG1$B7mjQxIZq%IgBmrC^sQrSW> z$qiM3`z%r=`&t$XIP=^UcPEE#-7ezOMI3e(=sb;TT1YH0^wo40HCUl3npu*6=Y-U8 zrkQ)-TyP-NCqYu({!`lo$61o*TH9l1WJA~^!GWPiT$6v!20bSZ2A}86;SKcsfG$UG zRtkCp_Zl^#?$##OqSXK({=HG;JF4eZhN#hMveH8EOd2I|2XOsqT3@7q3;Ki~@tdW( zng}`#@lPHU7^FUZOqz3*|>CmE;ci zsm{Vq-kk-g^54pwH(DmU74ugJxLa-6w& zhP?;m!q!f5{Kr>0w(n#RhVRNFtZVcmS~1HUIj7G(pXv6lrCR{MXBZZYBFKwF3!)^x;TwEnKqL8TD5a{G&tP`XYP zrtI47PvQS?e7 zHG}ApzGzS5o__HKoXN2wb^Hty9>U-?bg1IK!p@eaiJB(X{6lVeLm{Vrq5w1-Yy6o0 zaIzM~AsEwR$hO%Rvd+z9q(kt0}2@HZv%n_^*rSdEI4e2ME(RpmL%wcK~;PI`co-)|E``@#+q_&LIs!7ep{1P&vZ_;m)4=SEGKp z7*V^W;7yy)1!NX%VPUAW_+``qbyU7joP5UF*D%hE5t@hEBOP?Sax*t2SNMu?3}yvK z56JGF$QlNH#xM73EKPG_y$0D|(79tV67nH)TB;M{jh2oi$*Okr*#Oc;J%1c zs-?o~Yo$xxdcfGap{y~J{lf0DyN$9S{O#)5B22osgTlrU>}ioqipeJ=8? z0IJN1^&R#d`Wu;TpGlPslNlhMq!%p5qyG*onI^yIe8q(;7RBmLr9-#zt&o~}u8nJX~S7D)3CuO-k>_@BCQ=xA63X0Nh*1e>yw4za`y z+Zxhh4OZN{%6sbCUhyT090wY0-6Pftcb)Id0F)}-g~jN?-{W(_^LmF0ix{~mv$=Q1 z^ylJcFLunXX4VVGS34(@K9c>7$wys>UccEO@aq@5(L5X4dtx8jq{;aEY0-CHGyDXm zuPA)Si^kv2Yr;C8CIoE!h-pnLmJncjtjS<^RBideUkn_DaA zYlb^PVMc&(GYx@pnsy|N7%(_wuf7Gz7PAc)8f z*@@7xY;z$2*FmNIepc`4i$Dt3aiLNNRyy}>&hzv8>j#KaeBwa;K;!`RPFIU@n$2s! zB14Lr8FiuAY!(2JZ)*uMc{C$4ZuhNH!^LTXPstG zzSm`iIX~7<=Ez0f|0(J}kK&+L^!2c7&b;$bqip}4Lk0_7%NKX^{?5K%ok9uD1lHAXBDSVMkrjRlh6;ZQ2+@|}iGw1{eC{wv!ucP>K0DHb0+|71 z_&}ycE~M7V?|5b)|D66tMw?pjxdHhF^*5^d&uRQt%?i9uFa$r*bp$V(Ft5b>0sho64%tdqRiONE;FKci7^WHt*+hIX6hA{|P$dXL zEmlpTSZK$jg?}>fFk=qXA=Qc0k*1FVNu89?4E2O%kh2GlzhEH^1Zi=j66I1DA}80v z++-6?(?01yF@{`!u^4(gg)u<$$W_0B2j4QY$`#nVDN=~zg?lu!xL5TV2e6zi^xfDH z4oW2Jdbc_)Z(A^sYD1)1EMHWYRaa?`Uz?PMEue`;Q!eIZ(W7s0JG8^eL&8lsA+ms> z9#28zK%C=Qw(**3`8Ba|PUblKuNJ3TjZB>!gj`^mTrfgx1Sy*x{Yc2b@Z4Z64|4b| zcTwB`mMp(vGRho2?K*K=2w+akMH#nBk;$@3fqkU3X+H<7Gaa0)B$K{$sWAjUpq&5K z(8%P={jQZr&PwM{3ekNXVY!fR-|p9cUO6TYw6GY{DzM3Cn?IyxY!AyA#}aQ6NcYu!f@+xbIKmMIaj9J}6L!4B2zaxK=)gS#_an0vU_ zG7a}5(+OeWUI@baCj+;3z%o+chDxQHCp7NSS~B?r`@vI)G8+}_dzyJ?2jMypTf`K6 z<9e>B^arQq%tExd0^pFuoA=R^!SP(R9y5tgM!k9${R|~JILI&D_|w$NdVb$~qZ`1V@? z^246j{K7r8yO2)>O#6~BQ{`UhwdF_14<%RmQ5(xeg_=%+BY;zyfS^2(oy~#Q~-Ff!ey$4&Esmiu*dDLE_!_&^n!;E#`cY$r+x8>1$2XxW8-Ib!i*Ah{4 z5962~?Dfo!bSy{lrm;)fbz8eg7d~_nJg*6wyJVii@J37PhNWbG{FEjOKV1ouDA=V5 z2$n@V=P4;M`2q+w8!tPGfa?^{*H`5rm4a-|TNy)NEzVxfz;zunHkCAhS%F;8Qi@jr{L+6l=a8ksgiWEk3^%Lo9 z@L0L&ap@LMZXr8Q)C%ZyE?)D-Y?Ac6f20q6X)b3E(guJxOFF9-XDSO)}6r02ReX)&)dr#!E0POgYb+0aX9$i*s2#Ok>j6<`xnknp~? zL}HD$NNrNqt2<}141X|n$GFYbeXG>zxLr@3=6%LvE}4w3H7;`RCsJZEvbRdPz&;(l z0d;Or3_ZYxJQvGgI*lT{vqSj2t~ua~wB4)yO(2I)hWk7#L&WzZk0EjK!DP~VKJJI{ zb`iziZm)==LAF;sP%EU$A4`2G%jPDyS5UKcdJ6N=u2`=TYiE%=jmwBq3Vp25ONYVm z$uHl$WKp#Ipy3y-RGUt*BTKzqXsw6|<(&rBAtC|3M)VGRN?(5wt*@-@!aVY?OZu-% zv!SpcNq>L@EhhEc-k z%O9=bmyy6eNCv#|&qy!;C!_3wQK^ogG0B^0kpPdUKC+DYky*qWLiS@Vtc>4rW#xk9 zLOS4^{ImdGF|iMbjT{rMH36)Isb;;miX0;#@eIk>iyCwVr0woCAt29dOC!L(ep@1p zv;KWaYeTj^3Y@n8Z?z%8+6m%-*u6CUw{F3~CU*EVTooJG;0zfr;F1)6i{jeZ?1!j}a+ z{hTC#@g30ZrDG+fF~xaA_K&W(wIAQ&wHo%d8YPd}^M5(K_0;OSbS40nYkWv4W*|C! zm3_j}yk;C9=ylEkcMfjeqd4u$qecW6OI~SOY&-rO+FGH03+|4Hr%@lCV$ei)4*&j= zzK^iytgJqSUr>r76p~X~;g>+?7bFHyH#~u2xszYeh9&tUOuP20C-`G^bv28bB|ELA(^H`kyf$2@RoOfwN?+{{?pvBC{b~kQ+>Xhx7mm9e3vVR zf1a)_zsN2lm}1BhV|^Bx&g=0F4mav$AC_STecE#!*8d!qK>}17;|(SSviBJ5n{4!v zf5nA>fiT$a`%T1wt#;`#ZybS+c$U@jd-dMmcGOn=^mW2bqt(`}lv8&qd{GyUxW|$6 z93^m7e(n5fFx95}q?&nsjPGpsk4Qsjid=zmsh8@o&Bn^b!bA4b=|e89ierkM=Pw8K z=S@iUS`~X%vuMC#n7Sw5Ws7?KsrfH!ZE_dY28Ttx)9Os$A4Ok$@uv%HELm2ZT|<{Ysd{9^j2tsNA>H@Rpw+gwFd7Ldq3~Nt*sC`~(*7l(=mY7)^%B7`R^Y z#}qcG4W}QQk3bl~JM4MZz|O70Va1wxD7;!==e993ZrdEKeYx=MmRf+S(t2DlFKKX) z6BPyX>(FC?p9~ywYl~ARG0&>lEJ25_>DDz&%|@wi1YtWl2m@jLS*t>8mVB9kg%;jt zjApCJ5dRs#lS}<}%S=#&$thc_A{z6U(}d;Nt%ds0v0W=&A0{MlP4xq{=@kXWO-J>> zpNaKXsL4Sm)TaXacMFkE4N_Dc%oqJ#DlX|8upp6&CRpN7>2cyXRkU+9VoNBHrD8=M zZg8mB!;z4YM%9p!TUe-A%kMC;hu@K1G(2Fq9=y(<`g3m z!-^1tb2x%SIf7GJG_s_%7g|5E`u)1Se&k2}$jkTZ=lYR5m15>j8N1CTUyBP}s}ti3 z5Q<5HAnq5yS9+)PSMU)GN2%No{*aynBE?$}izksfitTdVI}h2;}<4jOtAZeIfMumP9*Yn5%8M1F8yL}tukBsnRSCw-?bqUf4Qi( zAbX!Z_Botl|1jGp(Ar(zuCPuo(I#qgUFP&5U$3CBu*AID1Y$Yiv>9));N&yUpJ2vQ zr*>DQB>sMHvd^(&eolyH1*oM|EPjwSM&v#$0BvS*&OyRE&Wio-0+!!wQGQ7{>+imN zQT*751HaC%nHTj9m+`^GboyGm&98RVL*=UQDT9eUnzfMMnBu9yHk)o>3AiyS=c|Zx zd+HcizUX!M5zdY^F-;AV$8$)D=ld!^QXfeWn`Sj=#%tb(Q`%qmny8 zMJ+R+TBgW3kEHU0r@~T_eSr3UgT+*!g9OtIK-*b?R%P(PXHG_g}*xp8anM*n3aSA`wQjgX&7$sqCUH%!X$t+^-MR?~&4S%qbs71RWc@nc@CM z>6?;jsycTemb)w(@LCi#MzK5nv&}igu6213 zS4?Ie(HOx}c>UWmq6@yZ>{u>joWtUXO%j4DPE`zBVejX%c9K4lL;W52tY-QIiNQ5d=#NiXyFq9*DaS493#6ETfc zWO7EJ6F(;u@U)Es4~PiI1p%nm&~{x@L;I*Pm#qJ^Hb+HS4daNWPMg=zUPc8im(GM} zg?GltETg%}(W)l#ZbEzHyu$P{#%(E^N)Vwsh5dO|gf*2KiZ5diNfFRC(SFFgMq!CB!6O#gU@hyuY-!QT z`Xiy0#Me^sEA)*Tt^Lbof!bGBIG0KH2l;r`m}c;5cD*s)^Q_l(=r{O{&QT>RV$iK7 zw!!+?KKJ%>Tl1oE+CacCEqphaeA*R_a@|lUB5a zI6aQJC z7u82L*L1?xM2i7Le0r5rVGYy;SvpIMz=MCL(I9L}PyCdJ9ni+Mh1neLh-nCdx1(7e zXtCNf<=XO&N82wLZ=0<1nA<)288SLMcg}ylbfGW8qIX7D7K(3WoRzf&@-fXjGBa_y z4&@h4f_`K3D%#?fw^}mhc|j#CsYr5F=oz=8(6fNjNw5rP;X2)Lyasst-j3dvlr7EJ z2kYKZ?RzfZhlR|xIqgYK%v|(fto4FVTDJP8to(6KVG_@qggc513aM%nzN4Osl0|!P zi?=$bi#6p6CpeM3!qc9bw*uOs+y{p`BGbwd2+`>A2zB)#BzR$wFevecfD!qJwB0c$ z%~2i!1%?6N?yx1S?VAEeu|@)&$ey&hP+&}mx5S1XL9SuJPf-1m!;g%;;GQ|?@8WW# z@klcIRK@&+NqWdWrMe<&6M+O_BnD`eyP049c?TE-Cbf;mjk{dv(VHRNLK7qS6nl1vRWFW1e~0KW7%C=Q#HRwAY^c ztD#)dGc7UUr;MTJ6If7<5wq6%1s`1j{N%Am5h~;+v-9Tfj; zjtf}!vwWCSViPt3&@NoidWm-3*FnvOjYZ!@zNp38tL5wxKCI*wZdG)2$DrQgNucba zddFXCpDGjjm$hPzE=DUB3nbXRP>X#uQP_9FF}U)Nb1VwLqjog=FJP0+ioFj-uj7Q& z;m7a$-InX6{NfE)xrXMP>QOy4eOY0hK`;!q7|~=Ei9r$MJX9{Pm~L?xcgxCXmi z@~fK2O4X+aH{9MdUu!@Atj~}x+@%Cfc%Yxm9n$<~di((2vWMbBrTofu`|@M*&b^WL zANql$^&6i_r`y0G(H z5MW7SrgsEo?Hn2zbj2EcIWJHx(KDPOhSJR4!DEyf9B7KB3&X4Iir+; z2F>FK#qwJ=e{sVcx5Rrvy++_}*$n>fj?~OQgybxf#Wgh}DEI4-8;{&N%)-P<(B+Oq z`$S8ajsxfQA4RHvhU6zhi~0=Z{?DPEM3nGx`WZ#%_(wLH7>L-#1p|Cnv{>e~L*dZSn|n|GIIG^n_rtEey4!Gi}+l&UYrfGrmgQ@OHd$?zz%voQBiIRe~> zBY_hZProbguE^IBsmXaPa|5{#v;H2h_j{DzAWlI1=$_m7gF!D|v=Fw9he!DdAN~1^ zsCwrlI_1Mm{HrIno0ED>((P@FHsJw~{N{oY26DNf9wg%H=o*#}%g60fWpFz+%;PO4 ziKS6LEp#c@sSXvq$m9ey(qu(a`CKkkgB|ZxEJNKKSu6LdP=YarvC)6!a$?Jj@+`&` zDF5M<%k4287K&xcv?VOhWc!8v6!JqYB*tx>FfU~1cau1h+KhCVMg@M(z11^dccx5o zS4*jJ3k#dFoFm+KTwGdk2o98xJ?#7s9SN!YQp;i>*2u->Di#K#7W6ezB^+aE726Og z*g|>vibeVaPoJqV!t?89V$#s-h}e*O<3y5H*uAulI2!a}NDRdIOq%SW-o)(66L*HX zr6SMxx$b#`Y`i7ae6q>+nO7RXjG|tmdnB7eQX(Dy2f_onA%Zo-;S>eKJYF2A(lDNM zm^WQa@}o3{^pqgC_zrDU+10~RZ^6F*$5*w5G>=yN@Uu$BLM!scufNQ#rvDEDF zm}AtZldkv4ClSCaaaiqv4x1TDhUJs>+y=8Mv9NznwPHjUV`gCis^BEl8LI+SFJ%ud-iXC==%#qnUGS?eVs3%a(QY zkUpAMT-COj3m&%LtGJY^LV2wMI^32wR3exYYYOlPJtcms)#faw9_@EGmsqT5-fJT9B2;1%ZH zIxar%K-E^f%T@E8D=g#;U5ze($cbIc<9EUnOd-bo3kpEGUme|?uJQ>K9bH{dvpz^2 z+z?QqB`V+F5mj4PV1jB$+g7kk;ZXjBu{Fzrz_ka*`d$_ZN2h>lsv8>CLfbazM4ey= z-ijTk7L&G}Cid!`Anws;a*@}}dumod|3u>L6!Vf>mm|Od$SWj}^>h!PYlyY(n9^;E z-M@u7fg`Pq>z=@$?HO5v69n5tKuC>AnFesHCt)wV?u_&Ef9EhS9~O_N9XubDe}+}o zFxxqho9xKaEq6=?EjdGs>Q}lIVCz3XNnrL8|x|{a>;S3;) zon3v=DOcK{(a^9d5f2kS*z~^}#YO>aq=1BdojmB%pN}RS8{sVm)l_qR-@*Jh-=#hr!k! z=IyK-`|YA_gzverTU@tw6)HKVe*i|WA+R=!H~GSjKuj{uYK<~Rv0g$YM7f0RL)^Z> zcZr`z#$jxZaBV-3;S^CuMlxu`ey$}fS>8|&U486Orq~{iJ#dU za!M5Q9BZ=QQ2&C@UlbsM`rOCKe!kWGpQ}|tf;-L2XQ>K2I?(7h5;0H+j+O*?T118h zc=bfPNB`s3m?hVM-6$;t#xX`iY?sz2lAn(v1QkhVhms{k)en=rA;mT5!|fP zq~zCPavB`fYyzthbpr)f?WC|aA70k`ptk0^3MekS+g$l^Mg|r^`4Ilxa}aR(zW(_0 z@Xz}`&NmCa<0R{%-k}d{s4}mRks!*P0mP)& z=qwd1Clr>=Rbb2FP_tv>HdO5#EPD|6+O5L7(bgVQlQI(Rg7zEFOI55?Wv=dJZx$VA zmQWvq`tLMJ`l^(0YEREz?4+2KZ$y<~50&c^C2-tSv@ebwQwPeZE4stqab1b!^=12n z7N?8VShiYgmk3s)=x4AOo1bbz1ME7wA=VZ^$bl_(;wTa-(yhr&8ltUor1DKw>acs8 zWC?LU;}(C)V>_OqU$xHZHkgXcFHY%|8nS!VJAgKG+!_oocwOSXXd8m8fXtdE!dv71 za$mHQZwx&;pVFpQy8JC4R_^?Ed|xbR@|hZBm89rKHHgq5v?;n5%{G$J0Enlyz2W4| zfm+ST&w-Yi95Bu)rIcqcI?;GLU?OQq$!tLjbkU!Dq&dd>0o~5Y=Yw|Ch;|fGVb4~M zuog(VV+jf?&t7grdP3H$dS04jj1HBbq;xNAD`iwuuEVVga3dLNC1_agaY@qp%~H{5 zZhFUDtT)09ec%>uwmTlw1t{09;Jn%Unp2Uvi$%dZa}Q%Xrq(Db!85#+DPymim=Qwm zeA4|2m;;|+ZbQ8S4s-VMcGj?X|urX>Wk9%rL0jAZV-oAmj^A$xH=3aw8 zsi;pEmDw{I;gWTHhe(Met!1|F%%<4NjYd`_tn6H^D52Zv$?q-YTNP2lnCX<%V`dRT zVaphcVpQtm&X#A}f<1A(iCN}*sFVtoO&GEQa%Ip-~&R4cp7uzd4Y3qBdJ3A|N z^PM65mwTq^0W}eV3K)25k>zXy=Owm6Mg)tGL-J^D)eQ+92vKFE*Vxf_b)-4vVs^{G zbe(XHpHjh*RFqum*HilT8#%gqs9-+;JkFWlOF>63kt*j!={Pd}z+Ob!=}fPf`T4K7 z$?kYmZVl3>bz|i#LZ4h=)60gMOYbioA0f@Aq8mzw0fZ=6=JDc3FR4D3zO1<}n4U2P z7Al^_i$v|DWG>3F|8CJ_C~l`4xNEILDn_PJn$XYskdSFxTyd~HMk5&7^$2s(K@ck;>TmkJkdiCRX5@FHvwFQ9 za|8u*2)sz+^-tO>&+De;Po3g=$4Ckt2g&t7R6vpTjGoMH<#fkO+I7L$^4mUW;GiN> z>oJ3-e7Bu~uQ$I37myQ~{VRQR*ocxUoBKCKae@+)g(8KFN`;Jb1B@=zNb7G>hoxV< zZFGouXLt0bc!e^1lO)r)u6OPSjc*1U@AMzCA-x)5yRdW6%97t^euoWHjh{2wq(`s9 zZ~zVyXOPYDTt}TB>to&h5^N$bN~qnw?dDrmhZd6!F^}zfgIVX&8szu+LCX$L8eV^$bBG@If8Sea!S|(l2x=bfPWIjbiEO19Pze6)0$PcJt z2}oHGI0`fv*jh)p)5JT%EwWog3(kgfa<+fsV1RD0=_gqb8Ok2i-zAMoimy*Puy6*U z-z|hZkia~Uh@Ygk%c8{gVtm0%(aoWQ5fjyPGQ4BCV8jR!k%?E8F}#m}Y9j$%wv4`V z3HVvfVtf91`(ilTyi6UN8gCZ9ZGZ*WdmYzu9M5v*k;f!-mhz%>%FEQvF4eD?V8)i^ zCH_zt8*te-OMlC}8CFJAIcuv?~^^25EG$a2Q z1*Y#Cz9hTM_zS;rOFuH>NC;;Tbi};Ox5XE_JI~X1tvPio<)v|(kNe}hoB}jLPBl!a zAF?S6K)|REfjlzoN~WPtxT!6@su&;1ppURNHijmVPmt*KjMSumLItiJTf=_#liZP7 zSGG?pvkIY;{jSgsPQoS2PY4kS>KH3>ow7HhkxwWJ-MIR9#*!p;*zaoHUN(UO#jMtucLjp^e~JNx=laqhGWbj>*Iu5@zL(sd@fY1i zf*&vEo?p!Qm8Yr8cTT=ouDydrKBt5JBG{<7)%(U%(gKlr@2nC(^RsJ=60 z8}91zV1uQpYhGTJI<;>aE7`V8esY?>!SsN!oa_q3_-`zAQ$555@jEK<`J!_Xfd$^Xx|Z=@ zQyu%F#cFb6#o$OwH(dNW@vR4(p0!r6DCBAjI@d7}+cs-f;D4=nv`Y>Qxqb#QwYxXs z{5bHeRh)U;$op~E9kL!CI;Q;z09f;|Qkp!Y%v^%=1Hyrs7!qNe(hKraz6Ooh-5&B_0++ziM_7mAJh}gk% z?m2PC*G0vSArv=Zx@ni^Gw{*=jk2k#@}r*c2J9t6m_BB!?${Yo$T2ip0G0ZfiYPdN z&#hl(6ysG%P)C${Re4){Sk!0D{*pSMkm@i*kBT!s&frUiCI4VGDKn<78fpjf!9y<} z`phtJs!#z-hO!$|x&{{lUA#rD;|G&x6hnF~GvD!OZM;bB{UV%UxL=VPkNH^qgYYbVATZ6sinoN*N694Q4nGXv+qe~0+H%_Dpvi4iJPCC1x~1pRhIyn4>GZgZ$zOEW?AkHdx1~^`hTmr9KUxY00pc z$mE=T4!~7drxC2%0bfElhPgc$ztM#_`0B(>Sm4dX%xfp|p%>C|Bi4+IuHHt785!q7nT#|Ft0ercU0AITt9ojOH8VWI=~IT`^`AtN|4jD( z4FiObm(T*}p8-LJPp$OURvUZ-7{G}lOdFOqo<^c*^*xxZi!0<0XNGu;IjiJdWwV+_ zZ?BZ0sAo+ zRS5{5-_?-ZQV(YPiD~c(3yCvdP}*P0xv=~KqXZ9Q_?8s|5_%H1?BCg{^8hk)D3HXg zv+VoDJU8<&G-dQ`@ht-|zvfU@6=L1+zn>mc_>|328I9Z6F4)_C%FrDPgr2|O1@MWt zc77{;lP*QGJj4s#PgM^%BuafqB%otHW=R|_F#8dXCf51Q0P_g99E|T$8AFxtj8*nn zID07SKIEyfQ1Bida%>(r&yP)o_d)T z52);|ZJ8B>sc5hgNU5fkMfK<49r79yOf@>L8?Iw= z5w0*Rhpj71i97xRGO0a(Rx6RGTo1m&$>a#sM{; zMQ3orwwEDkLI{MoO5Q2INvsyj6MqOu*gX(hxiN{MNb8~{q>9#_(KS>T83_7z{9 zK%jbFKj7Uh3gYPhu=S47kuF@fXvengbZm6ev2EM7U9mb=#kOtRw%JjK9rI@I_uTV- z=iEES^CM$arGCw4F3mO91e=6@30AV}Rlej8dsMe=s5q=3%<&*rMzHUmSk2~FAyqLI zYu_#M(lL;{2AbdTnlE+~wiAk|ZZO00`9hFgXvX0ac%et-jTe)M08e8F{L+FCStK++ zrT?;dUDC-Ouzm~88-$_#!T0N^zmQ^=_ZHd|Dz3t?XF_g1;jQQrXrb9L@KdCz>OxM2 zsvje61bL~8t{JgcGV^;Nh~!^AY|13OYlFVqMv6(1TiB!7&XKP?)QxwZ}eX~qo`l$u<=UYYi;^T+f$zz1PFva72vVZOeDUfPrc zf2= z?ry=w^wm-;1k=#D5)B1gI;W$1IK5=2zkiAML1MO;(J1=6 zLg>XgsjLM#qelSmMw)y~pM>)hH~tfk<)nqmxhEj!opAPDFs~N1*G|~d8>Sh>o3Q@o zhslBC=)S|rIJz`Rky`r3qwAXlnp`ux)HEHp!zGYmYLT30 z>t7Ax=T&-3Zi|y!b!>QJu5oRfE;OJ5Df-(HbE!5+t@hV)eMWP=*v6_j0neD7!?F?x z>67Xa051)~mt2bM4kEdtb5u-UrJDpg*Hr{@AA)eQGc=r&_sD!V-|*pUdY`7}{1CsO z7?g6W72oCi(a_(fJM_IQH8XSv;z<{4TsQfg!G(xIK2MsXY+V7($>O*>17m!O_v)x{ zp|Oe=@Jg)v=#=-0EkqK`?c~c8eCy(in~O^hu-XSr9}H>qF9D4+(wbTLCIurvU0|0j z=ZHAV_v6hkTh{#sza4nIE=p*%c8%Y>$7NG;JK#pcF9QezKT%n17`LSCi|CHLyHwll zlHuxBz0PtX6l0lVd-0C`5G<%E%K?~BLh-RuT2@Myq?@cT)%O=&Wx0%bcO|HI$8=f) zyk4uG3fFg#+aoZFPH2n4g@f*-a_jY~bl9X*%FjqOW}>(fk~CaPmc>KT=3nBMoQ zTj?JMo*s+S$$j;LXQhtgUG=kV66-H3W$Ra70i>q#{ine!p`{H<;kv{RL==s$GZQ1# zTIxI3oImJ4bu1G-p=5vKeGPH|N+|Bc@?unjS&i}TifAWQJ_Wkpf9J=(G90>>b+YH{ zb{r)N@m2@+Kx^K4kMoYtf=NQSYPt5a<7|0?kX`X{K1LA}1A-gSKN0z0j%ADYwVc}3@Z3?k}P4faof49$_5IZnZ~ zOp*2TBA;#Hp-$lMYkHN5R||p@v+k*C2oM#NIrV*u#HuSw=^Y<~TV9Nyp%YuVjizr} zX2KIbMMr;7=zHuf+c0v-c`<>+>=v57N$R)iz8}+3dgZGrd8@Zn6R%yesBiqaL|P#w zlkFfB=2}>oNJhL~{c;ch)_7R4{!gqxhiF=j6VwqW+7Tzl@})z(I3e{?7WypK{Sna- z?Q`apC*CSQ634&Y%&UmatOM_j2b-YI{GQJsx$k`B$zi2QvBXmVY~R;`#_(7r0-^%? zH|UF9@v>h;1>pSEcUzU*m4#LI^AB}}N5(?FD)*#K?#Va`&{ean;r!o+@tcOP z>@LW1c23P>{pFK*i~tDyZ)&bt1sZPR?$lOglqInIG}ZSIOwMrBkX;DyHX2~MS7zwld6N_@4s z)~)%NzJHbQS!LWjdBWRkY);}uPdbWV`h{Y;Qu-bi|HPA2!jg`jpN;?iNM+0gYvb*p zMT!Fh?e(21s>f$_@n?`7Wt(cNMGMo|y?^9hF<&&HfUZZ*hBRxyN}}B}Q7ztybVcMR z=sz=$uY4Tqe<$Ss=ggyJvfeEDYsSdZ(ANl`od6i`OHS3&+J*G=pHF7ulhLTY z=Bnplz&$8rDxknuZyF$BXlG($>O{`^pTa&_dBbi&0GT%x^`O3qY=rvyXXiD%1T@Ja(r)sn;ADHe5CwVZ5pRRJyZN+* z+rvnW{m8fS(%;OnA^1)jlXHLS^VQjOR1ebw&tX&8HfL)Alh>msoj0T{JNsY*aPayf zb3Y4a-m}s3_8@upaNntLRicag<(Rb4&VWYx4_&>tmc}T{nl*V}7OwOi;3hmqtO__9q?sjI*>~JdK@VxTTy@Ov$Akt-_h@28S38dDZt~vqKTGx z6K{s+c|G*vUNWnET)4kj%^PQM?K>4FtWhPqUo-$jJtvtKD%YaB+Amxxr+7EYR%}A= z8^Y!SYmc9GhUtZPmETzZNLL)$XfJY3!&S)QXus&7ijZiPDz%SW zoZ(2onI7LS+tjrLh@!KrxR6O8ZlJs@^_&)&9@Is9l$m559FTzRo-#PzB0l&<<(0O! z?>C3WasV#6XVRQs$`+rdY%FN}9cMBDzIjW534Un`zy5Csa7^uZh$eV;s-Ka;2Hh%P zHNF0LDF57koWkDKt>DoAYh*>$iFkygL34r5kD@Zvm&_$k7W z5O5r^PhVYbdoBR#NVTd^$_xrl%T^UB(tq?E5rdIb>L<9tHo+cngXfVtCZ>LIu z2Oky8SaJh5;8y4*t0|RU+*FfZ)JM1m2U1>$mE2tKM}A>`Fy$pHJW;qpj83f6eO7%^ ze=G}>5ILNnCb!YDstiA0GbAwk_IOi>ACeBfpQFye5Cw_uE`O$9{G@I+f&q>we`7Eq z1T;mKr?ECr6)yIAwI4S-^=Y}^{b{=&>1hshgj@I%a5JXa8a{*l`*UT(Sfb2@h47J+ z74BRfW`@bxiIFklf*3XIjlgwg*6YsT@Cp8}9X5=WYUl%Z+50Q`+7F+^Y9g3Z=bGVH zlC1cm28vAcUxFuMm(F#>E5Y#C+zV|!TNJ+GVn~ow9q-=659Oj6{JIo7@>luap7n^d@Wr>|n5O4`)j2|PvudWUhPQ=b?t)sVZst$i6O`mVX2_`T+zRGP zBYx{at-LQPa^bBrgG`YhW&)EcxAeoHCR8ZOYv5a3gb&Xcicrs zCvZO@l93j1AbD4qP>mOHG!G9V0AG`RDCqwZ$^T6;Ewe1N7XMKNeFG*FWB{wfFhc;u zmdc6?pnHz%awl=7z(3RJyB)E9NN9{OR)ZEsJ{-(`NC)Oj*_t%CNlyk_CJ&RDTz@m1 z)6!yhw!5L2qSw$9uq8<&Wnc@iS(&nh@YNAyEF@VSH6SSwWN=c=EFMF?w{vYKS`gTF3}jgDP?oFdmTg&D3y)T{qc&kMba@9r z4_5I~q!iZhXu^bZy6gi{A~4YaYDqB~2yQg}w0|RI)2H*biv+krqz^(_%FhzsSj$P` z(d!&&L&ej~T6tE`M=i2R_T`xAI3Y?5Ri5Kz95Z%~Q>L=$&*e+X!zu(y&h{kF!6hxW3Pn?1h8~B4$90=UX(8ZrLm^bdB%^>dTyLN zWivjX@iQc%+)^HhEma#jODht>Eo916Cuoj;bMgnsd4(iJ;)H15y$wkbLY43QMQ20p z2uSY>*z#HCCb@8@;!MZ^#>P+vuX&c7@57WxB{;D_CZ|(l%qTS6o2MI;=Z!T-Jx3*0Jweh8Oi~ax0yqN!!d1Wb0*r73_^U4*oPBG2i z2?l&CnpL#m z?c?KfN$~mfe2?dkX*UAx$Swx9LaGzX9Fz>rNm?n^I)==bq}4tGzZ5-177EC<^$Iks zCTsH0%XA6xO?Jaj#pI@xUQ)Cwni(1DMKGC;<5|#JZI=40qwtbThOpA#hU61>mW`TX zKA3(J;5>lIfBjIaAcvUzgBP15-Q6AM_mslwqGZ9w?_drB-MVIG!-;S?xSU`muzmy?#(u3p%|%o;=9yX+UBke3@vYAs9G1l{@M@*4W5)IoQfK({ zg{$+?O__+N<)J>KRP?;XfqC~9}SHNIAjy#!v)B-o^0v8q~ZaN_`@$RD0!erD&a ztKzUUKg2oce$~Xs$%=i;9KrYeUBb%u7Va~2WeX5Ab^5{Z%N}{xRBwo`nmxw)s{1*VUUzKMBBR0~8krL0Vv;)A6+sR?oot_WRh<=OrL>mMTf zFADx4GMxV@m>iQZs096jj`vr>r1$@`9YqOfQr7)PU(1_lsYMUx== zl8b~%G8l|*1dbN?LfVj#w^qNFyj3O&NNz%cfcy9cNb)fM(%B$a&Ell~e4OSz_GA9? zb$W+106PM`ZME$a8S5qZm6TAw$?h-{SREA;z#h7!hJD4pFG1<}C6K4xGoJf8d=4<9 z31S91_8ZuX#!KLm$ScRQq-t;J^=#X+;cUAFu0{tZ5pAKwhUQ7=@DsxB5J*1z4ODrT z$+R+VNS%*~TssE6k)m(|e!V0a=ILJXU9s&<_2ab8Yz0Ee6;3Ht8HC&nl({ZgAEOh0V z9Tbm)VK!>!*G?HTx~ibvXo2lxQ45*RSnY$ntf_}f%5PSl=NT4-B7NgPvseIrbw$eT zqm~*OaanxyZ-E+h31>-`1ZO0q_>B^2L}{0^v>S}-c}sbG2l@g-5f5Y7YMH)utlF6# z!!G^Wb#QGq>IfV+3U98ydSyr1O&+;Au}&*`0+P?)$m<}#V{#X!etg^D57kQ3&N=kA zqt)bYVaid>F6okDoIn?#M}uQn%*fY!g82tV{}U_!XB9YnTTt%zg`#vwVEh#`0dThw z1QmEC@dEe*fsN{$VOMgo?}p`Wj_?NghR}W5Ku!w8oQ0MJ=)2?ZO>JyG zhRj?y8n%U!6>c)MZ(6SPv&ytk$P~~XtvA*Yuk+NJ%5wc8yoxv`IesEpbcVNFYkIqp z3|-TxjNHpR(zrz#2r}Y{oQD;5%0^^?=b#hkf%B5x>)MsgAQeVfIedV9q&pJtD@5&D zsq$QjPeF6AY@1fxXvp~mh{QXAZlmMv^S;Lv&eteU9A%$j54l2XUqFj7(V*#GmtKco zw;k8qip$4sJU;tOX(%Z$d{+9Q$nO-pB3oB1^2A+Sws>h6G)I*Z0FgO>X;$`8IpDpmfeJPiHMuZCfz^4Pgv;G&AcKw=U-OHQg&_FF~}* z7`3``NPpu|L`F6TI3NxN{xzw1CuNx9CsRNiFqQgJVi&kTUce^jgP+QGELvhLZeJD1 z-dn=R=(B*%i0wv!0s{OL)}c<;M||wJ-}QQst#6W|kHGkI z0s&ypJTw%bNk#Xc)9$&l)1xD2MvH>k)3Gwl+XtdDoZ1dpU%hAY0fiQZLIvCey@lI5VV%5tz{XWaOR4& zaU)--$Mm$zpi0+ZQ#4|7Z_{^Bjm^_|1QmxZ;sgA!`z1~7Q{pyT6T7fb1JC@Ift=An%LYp+oWs|7oO zecyO=nU^}-eFFiddmkZ(NBO?@=mQgpm-q5#A~!pvchG8o`b>eu?DfYF8aUN)7nYOo zIBS4Cy@k6^9@X&Mjds_v>XyP_yRyA;1_?jxltI;l&fJ_LDVk|$zd~^Jt&bAzmK`>1 zS!N#cve_(xZr$(GN%PogBhRoar}b{t(-*`ki2zP9lr_?0R33LIDow6} zh)`F`mC3c9UAe+Fhw$s-qs75#-6+s-_JNeUa85jjYWVw@uDXcgo?wXG(a5KFnHUyh z7ds^{H@qcwTP6ddzgIcBR|e{xJh5~GSG^UJk$khtHWV%W#Q>GB;67R#yo^!?DKQ{Q zKb6!w)CYWsjp18e5jBpliac8hjCzDz0ZS3@{zj1EN93XZ^nBv5{pxTI$@nCRDJ0M# z4Z0>StI&_*F_=mH^-%SUollqMvo~!v)g+i3dca#X8+8>52^_!qJB+f{Fl z3W;(aEFm#t9F3nNhP=!@k&Es_N1`S@y~D zyxhH(>7;0JjA|X5V%()#lj6+iwn4!L8=b zCOgLNp1v!|pRYJi&VEnVFbq#Qi85?G^VnB&HkJi5u)eUQ>8Ec#0$eJn^tU?0ImzI~6dCmC8aX@zA}rCEU@bkmJK zWLCxLj$s8@f+-?yXxb>g{!^sY=1X#tQpEzl>~^j6njarseI34bVPTxg)d=$C1A!P> zZNEj!K>(Cta!7>DHJ-i%3(a_kc%*KJemto^T<#Lxo_$^{jd-S)W-C1K&I@q2?D@k9 zCQbK)X6N9wl>d`r=kRd2)4a`30;8GAWUB>1MV1b*pAv!g!_o+wj52$G8LhzTo+l>b zN64d0XY2-KitJ_|6s6jAUPP?JN^f~Y>`g>~ysS^?Oxfy7hZ_8mIZRqP*@(3hMuMHv zgHjHRfxX{%q;B4yI7Y3jNK8%q)kB=i2OoRywru&g@{`iL6$i6x z7T;a?IiP}#i3mHdzuB3$7vaU9ve(lzZZb!J*a-lZnyFKu0%Lu|k}omKCctz=8+%l) z1$m4xwi$V<*oc0xKJkzDAQAG)w_%H@M7mYusD&5sISJg78`NBN*pO`Y`aj8tVsckw z!++edpC0tB{FbdIIfTU#$k=O8fv6wxPe}nGe>kGnG-xj1&b_sGOQoKbZ<@#cVC)TxF)^0wnTcy=QN zpz|KG8S1^7nF}A7ae1p^cUGX9B9gyRDIK$QMp0am|1eJCw)*3&i%N3KZ`N3^Frt>^ z>ki|Y#71f)Ijw%kqPV6%2(zi@gK1ciB!2$8x%pp~_@AMT>#I6znL?%jfkb@&vN9VR zM&&wm@M~fP;e1ec8#i%sepjfq?(xEZ6eP2>J%`uZ!GoF-1Sy zuYW261(M(qk|y^OA_IXVoQN&@OvqvXF`Dz_%LYULVy|#8APO%wA@G_96dGu-#0A;% z^nu9_3iHp814m#axPc$+pmG37XEbqaU$SWSdN%Ba)K$6&5U4|hI&g_N=3hXK2>KvX z8Dcn`*!5d~9#mGc5jj5?9jC*7hy?6Rm7-CKu*_R;B^8<)^WhY%Hss{P`STV1o-1GJ zMxb`;DZYU&`u z+NO*F>@#_>qxASrf;Fq;pu^X-RGI26$?HG52szeILW0V6w|Z_Gecd;&!};+HIBv{Y zO}vWea$LQJ#9HQmx7L7m=OE(8eU%*l;2dhCPPq`TM39YfeiHGSDmRk&D|Y z>;9fj58X^%GLO2Xx(>fkXPadqI&QNK`BIr^NO}iH6Bcc5^bi30^IC=(-2ovg2n}EB zz*I$HfqaAPDwNXFdS>4@ep@POtz+y@*%98nh{@VZ3J$napV&Qd(=e*@LR4vA zsUp|G!XnzZtvZ=bS?X#d!rOdzVj9R*tG(+GgAkt&2nNbuDh>IK1J<+{pfWZLw;87(qn28Y(TY!0l@*= zb2})6MFfU&5nx_gd*qS@CB#6i;*N9Ec$5b0GsvA?D=Y!(sF3`ECFw6GqZAVYD;H(tZ1u1q_LyS4mF=r> zjRTnezGy?3<1+K}%q16C>pW9IbY+{`{Sc-Ya6M||;^$>|vh4@dATB9@=N+V{h(6GD zq*@Z&4KD-!a5dIlvf^ZQ!t-yS``yR31u{aw?Oewgtqngb%(mg&C8eUpEV2x?jCKrB z=$%4qvOJY9d&i%7R+u_l=FUWcT02!@8(rG7ir-hvwc_Mr6%rdi-$B;z1Kh`%klKbry+O?(Z!lW2p&oxmF(I{oMW!jZb` z5x<}K&QWeTisEw z6m)l|0{D*3xsgZ1+7GEh>@FXloaRzaRMThb#+&$>C%vuW-QUN$BYdlm#Y-z@(L06S z64n8EA2AICXRzrxWPK0bF85WWWxHH=7=OH?n(xaPn*vid-hCPCyfLa!HvN9@xK4^w zHa!-EaAu3)t~>{2_Hq&3L-kFampX6qUuY+jeXw@Ra*gMZ4P)vQy$M?J_DpAOcJ%UE zn&OXJKht)#_D%OH6FEJgFhD%eXj#-Oh}!{(3=6L?(mAl*1Z|axjC06?LTikWqCEo# zoVqoguGaO;bFQ*t6v&$+ePQ*CK55ol^?`|K7G~ZkxOGiv8z}kbBj3buL^&NGoJ~$i zLYKL$zPsaGR-is|Yfp1)$6Q^6^TV!|J09BXQo-D_za;m}5^{A$7f)0TfICvd_x-in zOkAPx_A1(Xr~RSu>yY?1#x8u{B)q(sO-_5+sbB$^7fX701Vx_xe51GElKBLRtbSs; zra#b`!~Aw5kQ*NJpT_aOg!+{(`oaHC#{Y+Eu_r3ZdB3`J?})%Xf&b7kI52LJ3mzyX zfQ$h+bw^W0|3r}eX_?9poB~UYt-zGUK%GHsdJa3M6ld2=^+UG!qSplVd!mpgxQVKR zKC~Ou-jQu3(rLW zxEOMcddw`A1=mH4>Z*%TK)Nr#?A8?)YH-d3IJolCg~x;CG7SJU)hA?;WjAZ4FO`<` zWCrp*DUrD4rc+`i7-V@BW^8h1`*Skf%y<`y-;*ot5a#_HmK`$xuqGF-3(A@C&TNNI9`&YhF%V6XCF_So%|;U;Wvaq zev;BMsbE^>-!3L*O`{eSY%qM^MbmqC6KhxVS7x`MzHuH&YSgaOqG#2a1p^n;tqtvk zwSQBKth5|X^5=%Vtz;<>aUDvHbH;RGe1DMdQ5m5k z8fo+!!b_eE=LU0#h;}mWfo&$v#$13lPpT6apPwb=yQeF!6|Q?mbfzAu9-zy^B}-3j zXk}4mil;Fd2jyv~wG61V*%p3DC%nW17*gC-xObRMVp8Gs+}EJEo_v@+)r_Wg5d!!q z-{aDwQe@7pIGa?>dgTTH%|6z0lN6a`tFZJ3ov+b6H`1OvU5iaWW`C{|;hh}G8^R>f ze7D^GUQzdB)x7x+{&_jGsjTt{mK03sA!D-lML|-G0juwwwg7}6j@k{H0KiD?=5B*l zmaxkD#1G32%{NVMfGR9@S&D11b&eBi&s^IDY3^H<0JtBUA>)M!KpeTofXLvA8(N^& z9lmLNn^_|bp>8LH=~PF~It#!2d$6jLF`{X~?J&Mi!Cwo(jTeUSLoJNcDvC?J4C8aJ zQddH&tT3(P{M*ZK5T^y+=?Yk%Ok=f4`iN6`J}Q1X$c9ifqw2$2$D^Jaepu4Gd5or2 z`KwyrHP5vMwXmLt0m~-@j#U@UGyYUFU-k@c!GZi%c!~JTE2+I#sn(=t=~O^K$mjPN zfsiSlloFjw>POEu&!y;MZuCl@z}`~)muS)q_k#85Gf7ZFa#~?eM^$e})eTpJ?>Ocd00UXXrab)RL*&eXb^f)Z zrLPMDp3yGpEFO^Br;Ke;9@!i8Uys`YbP_0&o&ng!DXlW8e|!J8VrfW+$@6$&J2laV zs_N2mNhOwBA5`8P5pOY^v~FMwvtbACL@yYp-!se^HCUgk$RR$ilLtfV_L_aJudOm- zp8o78DINjffl?}E;PjZ#Gd%S7?%!$E)ec-+L_0SRZ63XA2-3@@&CG#cQzFglQMM8I zdilx@d^aU?0&K#*c6peZ=|ebd`#k-v0yU#_u4AIGm#AZU#nkOWtk(<_Z#1qe?3w+p zG%E-)`;A0$W33+j%ZpsLMr3R_R>@4tFIeR0UFe{o}TMr%(p$Y;_8vIubehzb^q~; z`ER88pE2m40^+-cIuYAf)vydi(PAM45;K9qw1}0^>Vo{I(val?<@sNghLt!nI~d|W zaf=s6O+WO@e$5aS$Sh3%-!cLYh&A;CwZ$2QFc;?ETEhAQo_Y_6k4GpEKox^wD9S@` zah#__p%fDkT2BoIhX<#qwmbmGl>&@5e^b~g{e?9sU0qGg}_sk*@Kbi zSi>G@d{F6Qbu(Z4+Z2cokZ<<5l||pFkH5~8&BoHJ^&B(P9*&8+GU-H(BV4V%pofFE z9KPf5HKn1DHej>vP^(sC&}hOk>m!-bXjWp&vTZQ?%j1IR8Rn+DtG8TzQhnq(@{)I& zf?rXL6;WKeQc+~=JGIV<7XvJD{8;t1Q_VVvPS2_H*LKyzN3jQS9-(mbp7xdy{S_)V%Fgo5AeL$ z=ZRF$3tGJl=>5?bJGoMPnR*#O=s|5%JG+XN`bSI4b`_CY2*ug=58n8r+s3os^k&GZ z@NebRw(iDz`+9^^z%j>8fXLruu__DORh{shY*mwn%G}5qhvX@G3`w7|eU8hr**YpF z$IyQKF4_Zc4DTnwq1dO0a`A&#`R^)k@QJ<<9HJN$3b1u5R$M5BU)#E0 zMt8FHEt0kS+!<_3C3+ufNc`hrBkW^L%+c{Q=Nhp9z+_zizrT>J&kicOgg=>Kc9b6CU?M7v=|sV zf{5>xstDB?cXTzl;uF69w~rG@J5(s=E!Z}A_<5+fMCBVa_)N+SN`?9F(%^>lEL`LV z@nVi(Y{lbfo^yU{@JJE>Z8>DA78(*kLN`H6 z6_FdlzJ)DT;m%NFDiBatFi>zq_-P!;8)`G;>9x%k;*q*-&cZcw|1lg!3Fy}5PBuTX7urV2yE zzd&0yLL6ErH)`DZRuSh#+X9>frF1*I>6Gr~wkg>@CO^08$RjwpE1=;C`9rHcJS(t{ zjSv%L0XWJ=CeqKZ|w`lBOjB_zzn z4kt{L@4@9K>Yct}4PtK_6T3?chK2j|w#|eS1*4KX=g8n^36Q3X(xm@PCh^TAiJoW` zdQ%^uO6g~3F#?Yw2cBGe{XE6LU}1`%0EFG59Di2oCvr3u5BbT6P8Q@bd?IYZ*D;$& z!1QBM3K2a0>mv0JirT+AU2Ok{2;$$4Su{VW4+O$Le}bGw9w~i&i5}P|1^sXAgKZ&` zMfUq&T{cfzLLsPsTWd(VchlfsL9!48Xvp?;HG!A|6ai?*P65^Oj>$p}@-J>Vd%=J} z6j4ww03ab^(76L59M+CVX2Dy2X2Ln!`SSGL@A`<=-;yViEGEMVM>TI>jT7BuA6r;$ z2-*pDShZf3Dpe(yc9hk{9>;Cng`lqmLb%wZ?aLu;b#AJC@5kmc+|+x--lEMVMtXJw zJhNiU8ym@CtkIe5_bnIe&Bb>yTk)D%rM+%@K${$#(D1Zo3PSMrlQH9P<1Wef5V zUT{6IW39H0_$fZoy8}7A_kiSR^<2RW`vb@+EH8@vJA%$~RuGw&yyR)>+(2541#JZLP|~ z^2KGlvyj-03GHYt+3gO6@+dXLbcR=#XA~8N#j=Ruwq+9*Qd+Pb&|Fb3$J;D~A>`#% zAG?Jp`D=>J6UqArZ@|Qy96L1Kk2atb&>oJnrEj95UPv24`BGp3)p_)WsQ^IthfQcj^8<3MT z!x$*QC7MWxB}itbeY10=)@Z8Y)2@Kf&TXB2sfA4@Uo3BGve?qo*4DOuT`#YazOb%v z6;68Oe|$AV6(a4^e;?cG?#R5z{>$%vyZD{oBRKqbsT|V9z8JNdWZk;3H^3vh@3^Cy z^1MeRii2WaPmi_=4)w$mNq-cJVwM%Aab|bMlPYZmr*m7l><^H73bMt6YT&F<3MI}c z2;EkxaYC?9q&<=G!C)tP{vr;I+cY-LrK9j9WehkPR@;Yy6G7>SWS_A7TV%T*i0h*X zs((qssQ=}N+8(p^DS7p`9`Ht^>DPq5g6&fk_WK4G=OZwz<(30Ue?&iN`5tm-H&3a1 zZkN&1f&`$M`Vwj5lt|a58TpxTNe>BFqEKps4I4dKF?p_1^3jdlfbZ4~?IFp!s5c60 zyo}q=AYHfj{JI;KaCs^6!Ewd>G!5Q8Asp=!(&s)A=xwsD7u1s|0&U6kGV38U~o;m)swDE+GxUOMGI|%Dsm{;veHr+^2qSmv@7Sh{{gc$Kkrr zq|Ro20`RzC)Wq)}1C-u`B5R;sdy>$yf&1#DLHg1+La2>Tc+W4|bANs*y~#xa;Xk?- z0myovD+7Q-e#!G0rB1r~`=6S50K4`G!By3L_N;BV`5zYk z)MZwty2cCbOGir;<*D=Kj1Ts=Gk>nb*3CFX2Kp2>K#Q4^b@L<^i6W~X-0ut`)=qbn z*0#~sSCKp6bsw0E8YTE=*Jl^DwlHEKwU|3os67*V$W|krD-Q&d*Z9Jbi$j+Bs?+iW z*t||B-2kOc05at4wfQAni?lVuXtx=inEdAFeF13~NrBLT=b*K3rr9xqJSo}xdSJJG za$@Ghs5dPES*PE(%!=4CDp0i{BldJ8%8O{x5rXrV0u;0xvn*|XAj61!Ej`?UlWI{- z$Mv8&k07Bsn|APT60-`?*B4nQQkan8K0y5puGyET0FgXkE~Ag&M^gvsg{irG=fVNiAI+i& zUsf@+Q@2ipD78cABDN7!8<18a7H?6~d;GU58I0Ayn@<7x&V4kESunP*))X&=1@#WJ zi*(NF1yGoU+nAzuXS0f&aTX5Fz!g)RZBc7%hCC1drHy8t=;#2;Q_BN{=WkY}^AryU z1~kzdP@$h77fE$(8p)rVNxSs*cACldPDV->fXvqSvOxJdxH6&Y=8DobN5A6W?_x@| z8MMNUKP7OT2rs3i2((bi$*=_{+A1^I{52UT1jM5=VP2GnQPr_Um244>=}Y_Damnz0NgOF)2NzI8&ME#vA=eTyHdRNGlfa?jFRnbexSGFA zMfi=B*5MGhN!OsQK*@(hYQ5Gvzf%PS!NI-*?X2#*!hW+4+7X2h!()Hq4cZAKij>CR z1dwxpZwP%@$YYl@eU8`~Yj_Rm@ucuZ^81Ul4+O}`3Y&_nizFJYGNemF2)E|)wr;9s z|K zjxMh8XUKb={XRf~Gl}GJ(Q->k+`JIs2OyZ55!-SVKdQO$;F8U4$tA?(<4?PTdSgXV z1=!`oA^MOzRziYMz*OB5^j#~>`so1FX?wACLmv?_I=C=5c6HQHT5(`vsn4h?UvlfA z)s1kMmJab?V{CO}^)=vQH-I(=XAOIC%RxQtNg5!n6863sKZB0r1b5j;he0NT3b;as zf_KfSzhxC`CV!fmo))M$QbKd-4PhLb+CqjG&llp9C?8C=j8$~JEwBm1!hXv`j1d4k z?UyJBc1+dE%sa|S&O;;X0;$BRI`1!*M5V)}Z#J;z04l2_Hsht!5M(^wv=*~Q&B@Rt(%i)Rz}9g#V6C(f!BG9<%FcMfAu zm=pa(VsU&H4`ZLvz+;v30Z18tnTwxu*zNngL@9l92t^+LuAxyUJ}iJFKFs-2`o>;z zo`O*O`3H{79~2uE$s8%jB{q0umg($jwIv(g`tWTWYVvFp1}$2bgL}ct*{32KIl-2I zvZc`|ddg(?atX*5OXZ8F6~j+Hy1Ia`zY!emHD}BHsCaTJj!4FI1EiUlTgFS~Ri6|} z(5<0!rOv|0vwK3$u$_+N>PDk*kHE?vKuuG{9LtW4$v(d*APsdbf1UV+YkR9nGStaC7)Vjv-vO%;tfrX+B0izGtrMx=` zXg)NNQL#N8R`Xl4DI*`P!W~HGflQIgL9SNrv)Ichn^u4b1*F^6Ye#t}+~Stcs)&|j z3vQ~gF3KoR35ic}Ea9utkvYq<%L|H9oY4tSaW1WuFRH-hay22-dxiU1%nvB>2*svn zjuI-qlCQDK71Qc7ur>DiZ9nl@GV&~+%Jc-5E zDJ?NY9%H<;e9(pZh=uBf!($47zS+OnAG_9*xC#|R)S5PWf4ZSO1 zjeby^2?e+3b)=t>-g7Bep>?lhj57*ZVE)!Fye?DALlYISWm3(s;(XT8GjxhZZ-N&J zv;spza0@ALlqhc3sA5_aQ+BgVxFc;bM`+({%#Z zjrPAb3Lr*DW{Uy$>iQVk&CNBcpnVhr93{ z|FuwJ8kWm$e>n3J!bq}?Asmh)0UsK$RhC{bepF$lR;jI}y+PGKiH#iuzrTrqtJhLk zzTPrlV*cG05iBj#H&0Cx*(DP93TKotn0P{p)?WweVs2Mk8zPl{tm=KeC}-)9+sD{t zHo#T3J_zM7J`Y->239jo-SiLp2wK$Q#*C!^w=hH#&EnQlk-nYm1P_fGy)F%W8wf-` zFAB)g_n9Iilm@{XO$Ss5m;J~&Q=2;U7xHR5gL0F))t?1<}6t$MFb{^I;O#UcO&3h#)dQ99>7 z3R!eba>D}N@)IsC@YO1%s-mLCwz8_SD%3d3_3Xu|Q8H4Mjiq2Lecjl$ZQHiJVs>mBD@n(;Z95&OW2a-= zw%ND$cTU}X&c0Q*YSye(tJeHy&GFV8?;OAJJo_%ejF*({rkL&G^woHH6X}3hKUF(v z14ql`5h$Rs*wg|v+(Ds>+!QvEz521tK=CktN_>@F!sfX=_Ir z+U?8VM`cD(rl<3|IKY5--aS+x=2R$2&_Ebyw<7cA_C@X!Lu1}8AXbnsuH-ES1}QA* zh}tm=X0W}8A(u<>m;WQ;jowT_shd2-!7O89tUB>ajlVuNE{_V$%BVT8 zQ|;_r$OS>rg_-7WoKVwL7LhO;Z-AN0=h)$E;`npoJX+{Kg8qz7xM^ZE+O2(0TYSzo zf(vm8#0{vNfJ|OaOGyCR%3)b-A|11RjRvx+i+WjCyqX;1jd;p>iv8iE>bj_$G&6Y5 zrqG0djLvrVPmVv?X96+@J(+bsQv~~fimydS<>8DF!#E*i#Y$hz4M%09OTr;yY>qGdGpD3wf;ix?R1_%c<22BEApIY zKJ%Um=ZM+LFh$}0vP4KAZT(SC1 z1wLi@?IQ2stvoFRxH4US4B~<_pr(BLRy^{7=WIs!*hY%xsnt1?x^#Xqu|W`UBwy+`r%z!ijgvRtBp3w49ZZo;EDe@i+VN_$=YY&r5e2whpj zhB@_VftQj=Kgb^1{DDxnBf8ZES>+8SRdW^J+Swcy5w`^*WrVVnmT2hrtrd5uBI#HX zbOfEd4gxQy`0yH2w%7C;&|?xKJs9M&u|~o}7iq8WZBtU6B(aw1n$jqee=0NHD;sm&Um54C)7VlS)+Gy1}nE*l}iGEN`n7bm-Qdj?Lbx!=T|lzU}* zM)TE)ANZuf>OCo%cM^xEfK<}xjvo4B>AYEQF4Fi$SF2MS6nqW%-;6TDkMytZ)GeH_KHi{2Uq}+Y zJ0i+TGQ1W2VeowKoINlG{BQ5CD_<#KS3MF;z0p)+z!#HC+4BP7)8%NlAR^a0evP`5 z4gfBuZh0kMU!u@km007~qCv~zx}>1ef-yXhBAq-4sAhr;99ai~@paey?hV&|jH+?R z0ONpNb>^Ee?c&LzMgQfRKg2qkrhO=W!Q4QxT)QF?Jzu+WhA7d*ZF(vd|A|QMXmaRG zr|=$;hI3tK?R{au@QOxSdjN55oHT_k*8UEX+?bm~xrAjT ze52^T8JR%a-X=Vv+@h!qVN;vMVFMH=F3A@Z+p9cB8fe5T*2HoBJjOTP^hv!s-@xaUG?cX}PzVSDCHx!xP-`Z%-|vT#NJybDh&JxNA9$73c? zOiP>|EZiaNk8so8Y_2nnyX8O2#rrB7O-mR~3Q;7k29@`G+`NF0;J7(c7*pnGK!h^2 zO~la(c}qZ5P_wvca}?R@+LhZfikswueF&Lu&X1S!jVoG|$F~SH6$fI@NDxBZKy)*% z7xBPwkw1ed>*OtgK(V9GcB~}7XKp93EuG4l_SwHeTEvW(Hr z3*-c1)Ozo5d%TZ(4yno3j)0K?D*<_%6QR*wo_Iqu&>`i(QR?x-d{wN!Ic4`i<}yuz(st~2^ zJZbEppHK5r8ah3Tq(*#U~YkVILqBHIfo-ZXq!NC6)hesNuFrPwHGi)1st!V2o}_iOpy2*a!e=>pmP1(X`b*Igi77k82f8I4(#jH;_=3=ZR zQLM3>f%}Gu?)KQ;9mLiVO=43y+f$`;Q7?hk&Yi%OX))nL(lL+ZIwvTa9r{?#hL83L zvM1E6icAhLT;L6bS(V8lK3$|^!!-?XB|Y$NQOU&x){Yh432cL5Xq5H;lrbi?B&T9A zb4m-Nygy-f#NkzFz8C+u;9WGjNE^~y*y>r|t)X~IRg{W%>wClD)gsNtw|1=?kxtn()<9yE6vzPLU*C9 z`mEFNb1|Tgf`LSMYJQ&j^kcV8VY&u=PYu{(*}BM01=l-_cJ887?e%2AkWJP|$8EBM zE+kJ{cp7L7!Cqv$C{sp__<=Dt0_pNQ$#$@=X<6K|?fRlY#PPn!kjS=l87ppc;;=T4 zwn%x8W(-ySh@Y6Z6El=DC=mw)#UEKu!a@vk|9zl#pLAe1a1xwExs_QiIj*-oMnw@! zpFZhUqdr3uYWlSNf>Al!$w+_RbeO%GNltAmrFP(_z?($2%Z7p3pv}XZ+lCD@Xb_qT z^~jW&c_J5@8^5hFSo*;?Ciyy8lVIPNMSg*^;YRYRBd9I%I=kn5eKbSy9}l9k*wM^K z$zfplDHIiJ55D|Af2#N+Wf`FcZ7nnTz^^nM$zRjdW?k&D;*Xf#N8Obisy?h6$|1v+ z)Y=xn2nP?DDt-F3HXQGUMgPEWd52*}eyexT_o}TNufSdl?ge|gO=s71^FN|ez;2Xh zq6ef`Dfr@T8W08hB8k?_kqX9?8m3|dcV3|1+lh5mXINRK?H=`S5px@SK;?z;s!@si zxzD)@_bin=xj}KS7|IiPKp^a~%@p~M^UD6KsHO;{RsvXJ ze{%0upv67l#p|26-Q|p#*D_6%6_5WcvC)8zsxTCRSyU!nZh}-|PwYgx%y)pJeKRS! zdc}ySDA=e$P3S$Tnq};Crh8Q@63VgSWhq@UY@Z{9?*a0Dxs8%B z!)&P@PmAN9rAu@W!a#gIPNTJbm~>W-?@fyD@%o0YAGN=GI!fQAoBLu!SS)-<(sqo^Cc*Gr^x&gF*)EwrXY5q_A~d-o*8Ri|-p~j6mdnrrGk}$8 zl-M!3V`bHFFV>6;$pYHzmjaSP7(6RA$F)0y)2&md%6$y$0P{#@m(kS}$!J2NWYd0G z7c{&)lax|fa32EcTB~grHvGAnNzF9a*cbPLDMz7KSIgKFsa=lM!MBF#%-}E7@|$5k z*#hT;o4G*niQZVR?|{!gAn(qtU-&~98=WPsYI^C2g>i6Jtw0C)X{(C0@3Tce2%EUz zta#CB>6CCqA$rbMeM|LbSldTB-r2QI7E9__?0Qq!(J|+`own5YE5YY7Qy5MpYD!-H zlq07Fh!$#IG)_*P^=LC&8!r@2XFLQYbUd`zZc$!*TOaiWkEi)IQ}2Tp>t~Vo<7M^r zC$2Fk1vCGs3t{);Wf4zn;7JL;c`}B#>qcma1(m1#m+$nx z98Y^1-S`&Q=lefudUEW1&qVr56hi#B(1{wp z+pz2qf=C?R)nl_%Vh%{2wTC za0s*r1?pg(xEdJhTFvemrei>Tf2|VnMp8=9%LFy?35**3*gJCZV_Y^Pm4?>R)?!~< z&-#Lrt6#VMPT~v%J1ev1)WvuHwvcbAA4sAAJ&rk)HoFfeaWfKvc)i9j3(sw|xR>D? zGgmSPJH!;KdlGj=OJW%429e$BK>r+tLtDG93d`^61d^m?Or|ljjqQyZm_34@!!iPjSb-7Pb56~98!1$SloLi zYH(~eT11^8pOF@e`L=D;MyM+qd&e=#;DNf+UawUp>haszzn-s(l|f;LcHH0s4{*x{ zs|}`g8FT~wjZRS-=av-nQW=p#oMfh-;s+VN3twLnn_jEGxXM0Ony&AhQ0@L2$@A2r z$_ZxMGTJCz62|~_xN_C9o7GdKoHzM68hc^ts`-9jY8q)A>GzqP=kb-MDa-mru?deh ztChD3JI|gkw78!+#)gx7Jdyx71rw-_>aHVn_Pl%&_V#K!2fj38O06_0a+dE6@`fvX zNium#a77H=0SoVrRZip4Baka6G3>SA(*Z=pl|fkbS?NGc#o^2Q@vem=`m?sn2x-O+ zMtfu%(D8D+;O9;DCxLx+Sirx{3=eU=1`ETjT3cP{+XLu!26e3>wtxcqL?x*jMb%oz zhZ$i6tqMCThR5tvaCsL%-AM#(w0R^cW|=;nd?qJ$;v3a7*^e>vGK~qgG&%LO$)hq2 zS-LtR@9O<efsR9&NY_G>va3at3N7}M6c%CiO<54 zG{YrV-%&2vj-ylu&|D%po4}4bV=iHf$&d}Dm<_C#IsVuDD=^XFFU)1A??!FHRLeYS zNoc>RBLpipt)(zFEhXS9bf{`3DoSzyP?c9P_o5gS>CncbB25;L&ll=6uUO2p?YNJq z&N%L1Hy!sE=FQ2%m@~Z)jJQhmY!udllws>KWW8&TBU6^RX>SqMtmWc2Yd(Fv)w1oR zJ%QOS(VI47U5abuH>?zP6NZ&sYk8MkcQA1oufa&jXSWWQtamJNIP;cKrF*)3l7bwi zM%oq8RM=p#)lBM86gKsx*8Y6Oob$!q+RDky+#%Mo8KVu z57ncOFbj)y1ZrlJ+S0llyq~Kbid%{nwm@D}=i@@2Ml59~W6jeKbuK*)xW-Pp^pP#Z zBPM7|dnC=Ah&_%^pMX2G7Jy1(zsJyJ3l4IXxholAo2OIsJ|!c6jpb>GMIZgg(urfD zTYEgT-KbjM+5l)lYJ1=F&~TR88P$~%)`298<*q7>G7AFvaR&PFJM|N zVKyZE236H&N|Z$L!3w}bR4JTw8t`M{cytu;&@FPqD|0{Co~}M|+9+ zYR%Ufnhlj;^q#8X3qBgz^7Z9{2|%P#nuX1NvjHUuw8q>a_jzXQzg~Sl**lJF;ILty zibz&hNHt*qK3vR4qgPGV5gM&|^_CO6SBE!YGmgq#8gHPN(`?iN)Dy1(a(8l$1=@w$ECA%PC)<~j&ORq*1&+_vBPF1v#ZB-#XA7VVa=)Ru9KJGg8dnH3JQZY9ay%qbGFSot0VAU zjL=@no?zu+EMs*yh0;wd2~=0K=HCm5eWc8-HZE*HoX{Qq;}# zDB{i8<~CM-XJ85N?S%o)3QtnqSjXF$)C>&%Q@t+u0gQGx@`E%=Iq*jm^q`NGzpV{FeqUuCRU1NWB=Ja#U4Lvy{1z z>d+U|Ts#B)K38mwW&%?1NzN8cxRlQk=OMhIPmk$zPVy9In=>X<5NS;^c7#QQ;45oF$OV)G`R^u@YPTH9^9%!p1zjeme4?~9!A6F#5 zXT2mBI-~V!j`Zn*JzE}_PH#L7tMQa4;(h~TBWDw?ihYk*9DqaL&S1UE5|c`(SZONUU^tri#CzLrVuwU%i#{&10_m8Eqp@R7sv$+{QVig5d+9gJ8Z zf&TLzrfyiO*ZOFI%C6s{Z_+c5rZG6Yy)M~W3?UDvk?E~2eeRFu2-N6GMhb- z`GCNm6z?op*QD`#Hs{~dDc+&_zb6BM(HH@7qMvxsLqKFbIVTmSeiNOr7g@pUG|qNu zzPo@9OyUF~Kb7mD@WYZ5;ybm!Rooq%ZU z;XuX^m8hgl`aaFcFlvU+`gtQmd_9`h7cBHJK^YgRAW-1gm3h2}pWQdUXx%`yH=v~b zeve7c3`nlXze8`pBZK(0xaZ3g!f&h+Ovw|oLqf2yByVnDJ%?iHP930*o?qQ{Yv|&$ zqkYcL@9BC%bZsZ8W9}{crc=Xp8vstsax1!HUMl~`xG@MBvaXKr=aHW?Uh4@>&P-o8 z;->9x8-gDM&rdmUVnIY;cF@Q1EbX~X|7d0BGB9sv?*_d+zgV+kL1Sp zz>OXDVVR#YJFO#b$#xDUtG(M0y-7AaR;&hd(*}!E%p97OiIu(sm+ zTby-Tal*lZ#x>85z@X=h+7^0ooM05~G~a{>K5p6wsx)UKfJX@oYg z;GvA8I#}d{8aWiTBfpu#TJC&lE8i6sKA)BeJN%Ue&QqO~eF7|%?z>fF^Vd+PluD*Q z6N-QDN|3!}Z!BJjZ(netW9UZ;PR3nUMmAcpi}>5j6|V5PTMr+!pG=sP%YkebbL~cx z>_1ZYrzoSnI|lVfHd$7Z_vt?c^AdlA90Kw=>oB-;#VmcdVEjQL?0##UYzwY9YgFQx@eI^tcdOGTtKrq70^qUopi4 zSut?7f7DGB)8LxNFA$=1I2}%M%8~`mqjXQ#gzr;2l1PNOc~^TL8%R671$Z~3<`OW= zO=Vg0;2>XZs4&y`ST2Ltt`Uo0Xffy+$BSC&4%Vo5?|Gm9WA*y)M)!X<#!7P$+Y$e^ z!>b~u-vyClr;9LwBd4S2@Is}l=3;f$NUrm%w+JPdKuB^)g z1Dw`|_0m|v{ASL~%(5@*W@wP@n#PwQ168Xq&G`qORV zd{O3pQCEv;YkTFe6@eylKCfBHuUo5GT?Ej(>aDL3Jekf;4^5D~c*$gT-SK|O^Zf%% z^S$o5>F2&_hwqbX+>XS3O$w>5nf;q~1aulr@90>Mq+PurImdJ89hhzvxCK!6hF!G3 zb%tEzW`sF9Jt)HNG#zXx1dImV?E5Nh2Hvo5#0or7IXL_oVQgQD^Z&d|cghBDGyM$? z&eNBK%*RR`-sB98#QuVu(~j=R>%@KSyc_=;mm}H={E?n_S?9T48Hu5eU91 zFt$#--y7cHJrpNR$P+dC&(?wD`k+qTBSg9<%O5b3k?&H3pxhjl&wq!I*Lr~cd7nc# z(;SGMb9(^GA1CXTDEXUWFTLi(oEpqaa-dpiabN4Q$;kD^R;fE~2WGYC?%IKqbhkQs z$e$Ik@cgQ#OK~1O#KDQGUL<4@0UYY|7cDMU%2y%Xt*NhpeQ#T8fvARbx%i{zLDyR$ zmP5)YiXLHFB}^o5Bd1^sD53=z7j||jBih=e|CWxFTGDP?FQNK~kXBSi%~1NfGYHzs z!jqS>5)pBOQMgb?-#Cce_Mz+$x6CeVzO(hG`1i=%BWg19r@)%yg3-TJ1e%hX`Y|f@ zM~M#lq-bktD83HdF>zjA%(V#ZFtxJdBHyUAZuN4tvX>;++61R+S0YWb&3gJXg(1~x@-*+5osy0+1aLBZB~VwosuPS&7!GE4HcD9Z>9V+=|t3-Mf}_kS-cBa|L+q|9O5^F#|^-b288^=RKz2RE#1pOp)tS%VKBhW<`P)6R-GY! z#im9_Cr?A28{n?0>3m|}z4|M4_m0s2UFilr(n4no1dB z&}o;*qp+C#5jU0`syCH7EKqjr>I%mVKP0B(mBE(^q`4e%);WZ)X25dZ`H?71L|bgm z7){wOgC=jTPm?x2K%e$ImM0Sx?(*2CkXSSg%9KCVs>C(|oc;RVty;GeA=Q_L^BH+5=c0#gQvQnG2!46^DNdT+Z3YE%2I2Mt4sv(@+av3vkq9IMK z=qQg8IF6YYh-wsoK88DWIr693N(8HV$R-Fe`AJQ7ZMKM79Mg97wyBMIg2t7>yf$WL z|6$Wl3GKp6mFcyQ=dp%mDrsu3+X+pN^36M#8e)B%s*Mo#Ft!&GR{#Dw*NZA(SC-hb zNi7n_JCUmt75WB{JiI8j!pdh&9l^h?*D9?J90%8QbmAp;w#$`MGiZ*|-`~=0^=skJ zP&~U(H+6fYea_?Y+eauyd$=wdW)I3gA_TPfzQ!hsCr28N9!go_!D&S9Yv8f7FR?`^-kd524vM&)v+!fZy?=YR|T3_ueb%Mn1XLcq6 z;k{FckRIA~wzE1c)@-x0+wADKH2D&J=XMco&%A1$s+Q3$vu(hp!A$dMq2J!V^YC0y zW{4YIq!3@9?@F`bt%6yk#%1qq_INBi_OwPBN~13wiHP+rCd@+ObsnK~@GAz?kmMQl zY_jO8!NDG+{FEyd!pKnS-Vt!j`Wn;(OjtsZ(%5BiHh`LOhiA4qD>Sa9;k!vwy zs?hmU`2Jb~x)t{D0M?TVU#D7v)`#hahV=|bHE~)yx0|d>s(QD3$<0*?BbS4mO07G* zggSTTjG$!Bo2vjHKWVt}8X>`MsD0pE&ZN`Pid^6FN`>`eEUUk#sJ6NM z0@po!_@R+x9;aw&80b87HZ{|yliWdm+sYqA7hcIM># zOt^?f`e8F@qF^f=2s|4G9~+-8wE$tCWOR@hwm|H2NAW&d25U>jc`@}6J|+m(+qF4F z%j9os-}RUh3h~Aw82Nk-$eE9agbW4O_5#Us$=>jQDThmuA4&y6sBiJH21?pHR1!yj zG@60Pe>Bz*8|G>o)(FGFAs4BXM@8)Sas_u6ZC?K(7R}ob0)p%qDwv{Del}^OYi4z+ zQQNmAMe6v_@X_G(`5(4_jAJ6tx_HCL-?C`#qiXEZFb0=k;WLqr9~)63Jz!(S)QTj~r?xQ89~&+nC=GWIUl%-<``7iW zpIWCE0Wmrfs{UZW?f3Y|G^s?+>?Ui3GPENJ>5HWRkmK-|xf?VGN_`Og(3td`2^@T; zsFx@2Xp19q>aeZoSi(`QX2@JGQ>L;{&jbRq^rkxt*?7%^bP>Bbh|@aaGwHawu#h3kJY zy6lrVtUFV&5~cOFgRRvfkW~r-MXc=$8Rx+Oehyq49`c>|T=)gOx3OfqdZ3to`_OMH zxsC2_BXi7r2dBV8o;+92wX*RtDROqf4$qgIowwXkLC?3V`WqmG6LoJ486;;E&b+D> zjtMK?Fj`FGtWHcb&j{Ut3DArzBVr&BR}yQ~+kA->I=jsqi(31GqF$*n!|VVy0n$obL-VHUX79`lNAF1oUWnflgnwNfc zKdo1ehja(xstPC&G)GGv^?3^^*rQmp2$~qTa4;{6(#j4M4iY>jjwRAgWwZ${7){S1 zjx{9DQ&FM_ua2-0f#jWqW;CJ2=vkr-^Qv@(sr;lSwALx2wk$2lzGHu~Hmg^oLSlOz zEZGw#;$u|6o)y;D^FCqjV#?rZ(x;18MPOCqpXhbwXifAJ8iKu2Ta7&ygD`fBw6l%E zf~zufQySQpH4U1VN*~r@>gc$Rn#bquLh%H9JfC$|=bLaPfu&1MH3A#wJRyLE2FWEG zLMXKB*e6Z=42O!eV-E2#u7~rCkPlPHN`VBt6^$<4>weZPn)4sn+5_JSJ`WwR__Z%n8_{3XJWk$AN0u4zs;*~w6=o<%Vm!^e!p;;IX zNa~UHS)xU;1V8sd;B~L0i8r2E=1(#r0R1rlfZ zl;b^aFT9z<;^^wkoLz5^*0lX9QF>x5V%9mRi>Vp6mcvFAqNVh-XWSLdu4J7_=W09c z9Maw5fV7%Q&AHbwghGp7t5~qpe?VDf0Pfp6v&E-b5PUb`EeGBXc68!$SAC9Udcx)brx_xP%*HAUwHvV_?kcd8oVkMmrPB_^hJMEL=pcVL1m2Cq*+ zOOL#`;D7ayBayUo%z`~$3c@r+3cjrj&YUvP%225teqEM@vBELR%?MdnM zAh+qH7Y3s{y}H*!rpd(cn3(PZLP7!z1aPcKch`UHGDWp_fwcbtuPb)5cW!2Z?_Pqu z21=`(u8ivG=9+~?7b4H!7=ya?KpVu{pQ6k@aftqEpXJxLO63Wc2-ee0XNC_5A452g z>Q_%O1_U~~q`;&3fp_;qo?womq0q-s%FuAQ7=&YQJyN6pF7hz(H;Skt5mpj?YU5Te zHp%$KZ4kDZ_X-7HHpjC`lp(d?^nsBH0fZ>!Op!1SF_yL#iZ6y8tpI1&4uaj3`fI)( zcmU}w1ILC!PbE&B#Nlbet-3`%Z-D-dPCs~xVZCIzjr< znP0pNsc+wBoG1XZwRHjp^nY)gDvC%#eQtEq0vXF(6iCo`prYD(XpxU zk*%^;BljP%e?IAez9eYc{^`e+q=Ex|gh%^+9@~4L>PV`72T_y0;+C~i3@W+W!8;w2 za@F6|@xmAlaK1IC>iwasl3AQe%`?W{v?IzA=OTbI?p*Hl`VmYp4e^0hz7YvS9NWw~=u zQf;36Mo4EedjH7jw<(w^n%6=58PwNKJp~BtVn5ZUc5I(q_!yYi`TEDMXj%B>*WcNv z;t~)21f;%Oah3*DRH>#WH=;;%$GHkqNXiPQZ(~fy1Q9!(O(vL zPVX<9eW$*}?c*YJA(X`HC((zWIg(VLbM|cQ13K5V{?>Ob$Nxh&vZbtJeEk9*KOFu% zfxl;HJLp4~HbkrZTD7iPXIZVMwVuZS8fN{2@@IWmEZNhgCO%S*XYpy?v0PJw?nG-W zKUF%%EB6FRm+zzoMJ&kVT|O|x@M96JRTMfLu%+2ysI)Y~Y)HY{ViDkHuNHm(0osH$ z@deQfc<=W`f`2=*i&5-MS}_Rm1ysW$mt1%#DbqHmPS0$qEAp<~{$& zbPPMNGy2C=qDU0xJ;7d||tEmWwW zH*2+WOfq!l3OD|fbORf*uZ}xK%N>VP!-u(w>@}Gh7 z4`*`qPgC=M521}fiawElpq@Oi^n}ZQ`KsGhE?S_JwVj)}i;9b*hqalx%m30G)v77V z4~n4pFD#c4SJv+MXcqRgpDvNf*o)Et>;M8(XU6IWQc3o;!tw$zaQ|hm;faLV?sh}w zZ+3jS%J!Z*S=oRN1h>=GW(sOhlR&D*T4tz0zf`Z~wSiP3a8+nfTF*V0ybfeJeJ=#g z+f60o@%rF6m2^9e<>BEz6cDY&ir>UdE8I;r#C@rZo@Z?ZMEwFb#u4Mh{V8@jN9N_D z(9ovdkr2iXG3;TLE|d_d07AkkGT*j3#=)%l97HA9j&l~SKXJZLy)a3Gk0L_QJ`Ktq zHliFgd>AqNnp&PmC_Z>vTkecN(QyG8FkrABut^q^_5r)6dhW+@YwRI@?C!M8P$r^K zVGXGmBsyHuG(`|H4AiPB_?X^}G;dsNGMaCMZVypY@J%zcJ#@Z?WF?shHfZeHXb>`G z32JJ4`2SVQ|NVFW&l|C5_IiQ&??y0zfwWydVTb;2oZxg7N($J2kV?i}Sy$x$Xkp+m z>A)9kp7cHdI6^v6HUl9LAn*83{rSx$+Z-#qgG{rhS|E~?hWB2Ok|Iqk3Nf#iSRvC> z5Zl#K7D>3KM2wB}3GcHzQHmA}4ub?!_em9!;Vw=SDy|_`a_MDh*_-($(ER=WbwCY5 z-i|4X_7hK+r6f(BU=XipHa_8B@` zg*iT(#>v>BLW^0Fj2%j5&DZ>kG)jJeb*Z-s!P~ZMvLS`jek&=jt+pQ4wv>i&iv&m9 z=%Kr*mlnRxzDC|rK3y-bMStO!J{wIc;aOE|FHJ6g9L+fITM3N-+Qfz|s|17Z@-G1w zQ{zd5=({8F5FZL4Jg-sqcV(9;mLOaK+;uIk=HPWMs~J9SWT75T7=tfvwXs%Bs&|!s z4cbq^yG!*((#N6(%_r#4^7&ge*eVWf7cI;)AOYL0;)-RoNzpcjt@m48$XwDxNTU$* z#CLRwrpfHQ9-SxvdX)1VLEb^gjT@(K6d9K)u}})_sHp|0=_hHJF$@eM*KNr$a7K`= zQG09r{W9IKA7jcEb#WS9gehIj6NfYSTDULej8I8a=9FmCq!c3XG;~9QP|4?SsUXj@ z0o_x6@u+$5Ll?7f`=FoCA@JC*4%J{f;V&_|viOT*1N2^JK&*0Y7w$EuLqjDw_PqRA zc)aTido%(N{!YTwG4@z(_{rtMBGUmv8qt&`Z>%8y`U$JT2>nZM4l2yUCrb(iG%3;S zl5y1+VtbY<5J{KwuQA9pPrQ=P`4+nbcBg*ys09P*2}^lu>6QiuE@e$vd`T6NXv-5Mn^V+5IH>r;!p5kK zy;Z?-?3}`UMXI;lDU0ob+8#cu#qo5XXMwP18iwF;$&5$(0_3i3R4FP36VXrQ=+Uoj zL}^v3-p_-0`XiH9RP}T)624s9@1o_~|$5e2D+AmIQF7hAxWoO$DJj zV8a73{8L<2Tmt(JOMnF8Csc#X;=J&(8GBNyIX5T2=xzOsajMoElCCaZ?LDX;&19Y; z#NP-Ni?(0oU2QjK|8Uq37oH%d+E&9v1Pi-f2+lcH;$VYNNnK8CVHPn~?-~fw&Yg%@ zQhLr%ra(OY>V97E6GVL$r^G_f)GqNlM_2B_o*ydvllyUxS~(&t2o{7p_aSN*X$Jdw z)cW)Qy$iAf1jedE2=QVBPp(kevQs$gwE)#sP1(w(Ck+g+9YZXe=}u<24wPxw4myfV zwcabo&Y8uQ)+^Xi*F{dKRI8Rdpaqv-JZTmgF%aRd<{Z0bCC>tKG) zEf;qw)tJR{JVv2I00ts48Zpk#Dh6(Wf}W^91j)ry$=TOCF8#t}$R$8>{m01M&=^;3 zUci7BuX)z9-3l^+qqR~+k`af~_>lQy=IMYTI^ikDT&xhdmO_kau!&9ixIK?j{aR5C z0Wkmmlu6DxC{hzP-c$zWDq{ zvFa@!$|^Dx?iG{`J05kQq+z8%-}vQVoVPdk_oQs2Y-DRJrA#gIvFetlX`Aue-Hhs_ z+%bZEO90$aq@xWXa$@~qHF4j8#B6L#-qKoDM%66iwVBpt>v!^%4R9G6 z=4m;R(Zi&ysN$~-&sKr}tH_Em0mrY1mbMFZxPsI;hE6*2NB83J?`v`fNjLP`yTfs1 zkL25ODC0}v!o(lSLhALQbChg;sF!V!skax#DfuA!;I z!OrYwLGuxN44sGMEnIlycyB>#6cA?+x&>e&r84Xg_Ob4(e)OW=byO9@R^lw%(_3uajpECX5 zUhLZc?YmmgTuxiULixm+qbntePIv{|D-AMM6Z#=5>y_;w&DStx=V6KZ9T(Y~EfXX3 z3jV1&!mkIxL2*==eL0oO|B}c1GBsMiBOCyhG3rOeeY?qZ1O|2c1BIFOtYF`GLNwLn zGy>Y3{Qsfqn`3-?f-lFmZQHhO+qU^WWAlw|9g-dI`z+pA>7X%VpWNm@W}X)Rt&X#cczKgF2iWCXCYLQ!K$1 zkGNfy=Bn~xda=t!RLdnOxMRk$VjErP36^cO{se5^WLGbf8I3YbfHubz6lMBdXF98w z+L_4-ShC3k1o6Sx#mZ8|0U<1%9=&!A`AhzR)!s+N{$6sr#~`iBn+MWsaJ?&UdHRj$ ziNVr3m&@qchDdxl)3_OUPp62od4 z-IGc7q1h-^x;Fd$?7SiY?v}ZSl?JXpz}SG$&A)Ra>RGG1X7$DTAUTgi$#tFyIP#?7J05|*07T}VKh=`uG%ER^DnYNz)-|(23i095c>uQRxbFXT#sBTas6bFq zP(c52W}am&FZukWLz|J(X5*lU)A9kJv;bXA+!gdMgcvpvs4Q7BQFsn%>B%V!chk#& zD=c7^;`@?-zP7kKDV#{=0XTdKjt0~4rr{}mhjWA9kH5ofmozPFAbvM!ER+d6B0ddC zzvm=Q4z;M5z{s9-%(VXdnihD^eqG@Nyz>SGzXV4`3qy6`3^ul@^)0IbF+w{iqwe>0n;pEU8h1Cvz!_Y?P=v9WqwiEq=ksR_vLT+ZgB=K*->z{u?XmPMJv}N5gZ~;W}sQlX1lAbZ zQ^ZQt7pnI!J_}BNmKT6Fr1ee#>d&qu@dp+u%{KfVe;4Ddw#Y`hJ*6G(g?5*R47g8= zRdyHPaf+_$g(#%+r6dYb^acGI`q2To+KGv?Nc#8}e!pXJWAD(7$ct2qz%V7qOilPu zo%Ol))TL`fyY`mL4{1TyG975|8n5Bh!-B`O9H(x}M)rQ>EF7S!-Ika@kf`~?V7G)G zhbDg8f))JPo;edd&VkYk9D#qyC9uQ2gX6ZbN=yXF%fo(nq8j3-Ovl=Sxg_@hE03bW znxEZu-9zz2&Lh4_AlHX8CZ+?y+o;l8PKxN`Hqo^u;TXjL;n9G}4|Br|Wa|_vb4I0G5QNup3%ZPQV1#@? z)-1C&{*0d7dSmd0H_kJOpa8-s(nl=f;j5Gw z+Y`W3Vx5lJt>U+~J>5H6CyC5Zcl=oCJvWVpvTQ-eJHFvQwoM$BZoaJbsMjPYTh@(MNBjsu&=R*d-@|zFO4DIn0FyIe2rzbTH{dL--Zv zSLjDeWF|)cZHW>Yt3KHSgJ|Vf8(f`3Nn-_dwH|}6glH?{B`N)rvH6)Y%g|@tAL^|u z#CO6h8l+|HXW5=b1@SPu;Pt=3%8b!B-lKs|yQqs}U)Pc9YzoU9HAXX_8I!pB5-*&Y`lF)U2QqyX{xMY=#7G%_UvD*? zC^bRQBV0G=FMgr!7aV??=?C5#tWol$O%Ltw!`}an?*Dr}|GyF8e*m;KeH*$Q6wKj2 zsX4eq=w1jg`~OTP1_~=su;2fg)17p7!2e_Vr+@;3ApAGl?)m_V@{?%WK}-vjCHvvN zzD0sh!`P3uu7#0K1uKSl@S2e>iG zpx2ZA20{jt0|B9-l0X}Zqmma9GVKd0G`ayt_r2-<7besGow4Pr=pZ7r^RMUo#M4wXxClSTA=AupL$*y3k7K)7`K^j054buR92zJYyrs55jiDV^{j~#dnrURxWyBwub1KmVCn<~T$ za#iDn_qSg#e*^)H2jx^OpEb}AL?6nHe0IuW3v@Q)2}($DImyN6^^RF0pOc@!qESF? zIhf$>jd>|vB^>os(FiDz6bjE8m=zR}jA_&&82e>79-r-xmk&T1Ja{%@tx zk&)Um!ADM6TijawEPKsVcKl?jA12TlU0l@?AwyreC4pcV5s&Zu(3v!jusw|YkzPblCq+O`K?6kqtuA1OpMf>X0uwMpLw&)5-kA|~+thAOesm;N z%n*!`GJW++?d-$(p)@v2Rbwtbkk5QMyMopEMRmFy%j~ys4ts~$^J5ft+wDDT)Gyk&A?ct2WcI> zvWWvHh!#K=BWMUrV z+j6q@Do2MnQj7V^_@)a4wF@uY^x@8tra{m9^%n5Eu-$^nBlFr^PTJXe?vjA(@g?a5 zH}m$j>F?_nv{B&SAO{F{&T>hW34_%d>>d4`k6bXHj7)y`e#*|SkM#LhSDTdx_a*C+ zp`k&FhAzLy>@%&_oaq@VMncY;AlE6bn;*K2DW`*s2_18%8L&Mi zg3+@16iuH#%8#4u;T@`-$A}qkQy)AkWZJ{D%04=GY40z3oBPZYCrM568WG6P0GoW= zdoPrxFbFl^$Jfw|hoz|HzctH5_JQ{o!SUeGCg^DB6fB>@Dr zl}oXavwVa0*d?bQV{y|HP^8Bk7_P7P38dwH&!(hN;Ps2jqNT}b#`O}RW@z%5THDGC z&9v6Cy;nV*G`vwPtF(_Oh{r8cRj6C4QljP}&ps8?o3@dc6jeQLv)f^lN_+|lSgb`v z<>$W5{tjBN&Dj0Q@MR%*>ScRX-2@c1eFNAz*`R0bW}BE`pUDyQHoH7(bF9-b<1efdz0KE@0+B1s~~f;(K{G>;-NB-=}-) z0cB8n%!mxXU3Sc>Ih2hMF!tqNBK__8ZRM0rV{r zzu@&i)gv#ig|mGcS}h48d7%-&_Weu#NBe;h^m&UL40R`(rY+uB<_VzYI$Hc)(PmBE ztZf?Ar0k}1W~WIcm3m?ge{7KummAfXtplP@3?^B)bQW%h1tL1ffgh%b7k2OvQGAm4 z!bfw~E-3JmlqLwCk)-6u++}SbDqV=sWtUi9!otRM_Wj6$N~#-jPR|Ur2iuQ9*a&n# z_2Rrm$hn5765ZY+Y5|-RP?Fk#p%^n(V-H~e3q#|An=e4bi*Lu5^Gye#1Ug-eGN2$Z4y=ZCFeGJmYildOFe@O-aJrxR*N`d^GOT^N(3T# zO6;2%<`iW>R8ne#qExwx&jLg4BNZ8v8%7=2PSs@L#G$MRVFdUgj52ychw?VeAa@Zn z?xBmRpi8t>@gT4`#vJyGa7BK{&nv1d$T;x4v>6@$9rp(k1Q1UHrT(V%!(=)r$mXs? zz7i?oS6?d`xsM}-$(~?Cv)Tm_(hN>%dM55Ui;4>d197kzPI474%z7BIPZppc5@$_I4)ft^yFy>jvusAoI%7JByg3Sw??9XTq+w#K_;h}dx3F}6lx#cJ zs+OJWmjV{$rtXt@Kvhvr?(4U`LqEH!Jhz9vyZ5hMP*=~cU0}UbTt$H_eR_|>^0!|- zkbV|Cdg=GsJ(BaU#EUM#GGPLw$cb-%x{)1rlusK^wLNYftWgWW4va2*lk5jFCa-L=FJ$IrqNY-HkE{Xdf;A$>tkEcx z(nMY=GBltLE~ZK9EF%_(wwIs3|fYRygKtcE)afv%g~vQxLn$<#`e45bb3 z1Rw$;6*3LxO9+2cK5QxO+;i)k&k>q7;RPMv6ws3t2J= zjdqM_;}Hv9LTVAwKPKg*4L?+xUHP7Hl(?+LtjvBG@;k~FmSUt|%5Nkb1h-U1!3nh- zXOt}%WzYZy4C(XaQ>+oSpT z17n{aoB5o!&l6r)PPgPn04tfu9n3!|BCZ`-!KB<*}hv$i&@qWu&=I#k`XdczZa`N?#AYRPTm9a?(;AHm#LS_|W-e zw+T?48n&Srxx+kVFsv~Tsh0u^08B~UUnHJ#9u)k7Ws}fPi#o1o;y^DZwNDjlCsG6$rUC_JVoHl~koCF6I=WrOzZw=;SM1Jz^eWh^O}O zmaCU-1c5$Ws1^EsCt008`!~Hpdw5t|AVMZE-K5sSf)6qtvbQts`s-q=73tz={+VMu zMc-*Ly1VF3?=o%9m=F^%k_linJK960f#B*(7gB~V;CjnOimKj$wi`Yemy|lTcRs|B zTqMvERmKJ33rzAs!Te^!0h!77yB-uh=O!C{kc!gxQtA!>^GA;VlQz5VgB@S3d3Wmt zpZ3Uuax0&l-ulF$w%dGvT`{NUE^*`za^_W(oj9@{{EWJzivJv6UepR7x>Ua=_U5~ggzc2WAPI){d$ zU?aExEbg=sqyPqJWx6a@s_!@9Ev;LX0LODyF)^kqUEZ6TRc=)nGO@w#BdMs8)#op+ z3U_+~f)CRRGKHTN1%Ffm=2r`N8=rRVglW@P=A_m+GOchybB-V3Q55D3xgiA>{5$mu z)p}&QnV{%ID=;+m4L8;IM~S3Yg8B^t{asR0m* zyUV!<*;|D00&4!P*oOHfQt^&k;*7|0Y+yF(KN|6#__}|~bG^UOmU6?2K*$Pg$PB&1 z0xr1A@!jut3;8gxLkIM(9U+ZjA&?qWp@z}-&7zfT-oTa`U6DFzY_DZH)!|DFdy-M0 z&53Slt;GoldXPQbx;)~K$Q(Z8kBqUn?{QxfKtH)z9>{33*9JQ?Jtrg`fFi^*ei(Bg^ zMzHST9oG*E@hUFPZ5Om17sa+A#>vNLGz*42wDF_obU_z&1`1w z)#e3oL5yh+^6cyI_K{b!>c!0rq76*F@ohEFFX+?qZ#AoIc&94RFXN^LjZElhdg@ok zlbLS42HXr7J2L>^ue53I9Xw7@lL0(*jP7ilNP2jQ)A8@phBf=cgd4~${;PH$9sr%OW~)C>b^N70*%nLAaOCFiWS*=3Kd8oQYC=sE^L zb&?C6NQ>u^riEcUDZcm!E=sj^RTmT-EIOwu`~Jj#x6XDctsB@Kt`u*)`Q;mlT zz1xR)iO5sKxb`=|Bw^&#M5Sbr3}OH(Ij1;Cx{a6PR-2I?Hy38y8$|PYMs7fTP3c6* z1hQ3;g2l1JP2){3I8tZ3FLm*H$+2*h_V|OYcl$?@xcz0Gbgj=qGM*n^Q|DLc++W4K zLSacgz<7Hv)*5X&$dti2k1H$3IqsdPLmi1p2C+p&`ES<13ILnvw9tL|*%`o98R&LZ zQHrvMUGZrtuh!5r6!V7>}dWdXEGKNLD|n|bbJyf7;MqVb#qRb?Li$l^#A%RDpEo?fU9!eY#tINMZ&H&> z^1g&kXB2#E72z*gYR7REk*@$Xf^kcv_(P+df?!Ald@*3(a2kcU&`M*jr&eqT9$>os z;cmorFH@6S&T_bK5jHQVPT%<$-$dRDqN4?!^C!=`p)O?rvm%8BYVX-@W2B+W?AlAt zEQP=bwAJrj2?YsM@&G<2Zj7}U&fijW?3nN=KllPqwmy5&W_0+uToRCtc23G}fNhaN z%@xmhx-C=lkgZfNWfM27z;%~!xY$(Hr#gLmliNg?Fh6^51PCzkn;7|(VA znfPQ#9`Xt-hbU$74Wuq1P)d@Pm=qmMg|CoFOTg~CT||I;UfQxP=vLd={zjN(bLRqy z@aP$&Q@H7%RQw#ny9y9P=*~TK2>!Hv4nYt}E%y0tBDg%hq5N*QMj^)4n%Ox~FRc%@ zVWljH&G~|MkDapp65w&xs3*J)0rr5&Rac#(pbd26fQ^k?o6VjIe70k7GcOY*)Ib2$ z0>|q+9)R+IK&u013ZxG~T_s5$pANCMW6mB=-IaDh$hDR=rUvjLR%(;;4Nixbt+K3+ zF@)&bad(BhI@Y1zqIQBk*G;6Gq!rcpg6p(s7MlB_+sIs!(nkBV0PKeV2K1Vi~p8O;#yCRPXh;C(|h>nXk9eShzaJJi~&q$bYoW;Xydz$noSvXUN70JFnzbqN4A@W>sqT#*LXFMeOEF=y?}jpbovD9 z7%U03)h|siFa|f)ESE_yU2Q4f``?27$+6ybLe@A7Q33B7a3(vEdzZeW4NzuuloU#!31(Y%diTDuxhy3hy+MxLfQaG&{nK~_rdq2X zM7_e$mu#ChnugK?`&xa`DU}v~5L*dx^;`9bt}T&{phYi+xzVIG(qr8Q`qpFFezjE3b+AjFiei=CSkQCqqTgo!JZ>%ZY$;Ad)7Pm4GMxXFP zSaeEfMYl0H_spG-Q>9CfAdgc%(8ZH{S!`A%&@=6vy)EduQc`fO<>G0tiWpA{k6StX zyK;|)6kgM$Ipt0A{a?Ijbnvol8wj;h-5yWmH}E0H^N9!U*z?~uekM0WxJ4%`JczFY zx_iK#hcm4~(bSa*Ekj()7KatWIyZ9$rs}$H+FD;?4xYGVHvXSvI7SfmyFWJg?r&GP zGht9Ofpmj4-nkZ!^`)FQSGu@uTkJU-UXDBKfAMS2HizB+tj(%5<_@+W*(|H3Z}hwW zNj!_;ocKig3vOnLsk^7Z>iBL9;ElG5_p?DJ|K)P8elhph|vit zFky=pwWDHVb8DfDGm`x&jw#`yzQ~6>2Aj<<*azXs;}p(E^;UstA*qC8p_yUK-*3L- zwAug1L%kxLxFPjVL#U4BP-E~y)(Z8ppWtI@maP5Z@Y-gdP?aQ6vFNO4dDyW!Qn46N z{Hquf>wE0+3!q^8JwU)&9kkqrJtchSfpm0LKz#$Xx6OxG1SGI ze4FdUa|3R9kCi=u7%b~Ws5S83FR_j2a2P&t;0M&zA0MFkD)bV780Pqb^a85BFCY-3 z?Z<`cM;5)?H=j;8rE>?e97{NWFdzyzh7w4P-9K2p^8=whLS3u);F?NY!GXBTrg=b- zK~?p{jvHL$SgBx9q*WfqKlSxJ&*wRVy{7ZX;a;0`Q8Ft|b zCVV;^H-AGRG8=|&1x^rFP#yh?(JY(hH328w;kYEA2McG+-Q0uGte#Gi%{CW+17%S{ z+`qP+Wc2HmX_^}T?8`amHHCEU3(jUa?G0L+@Fup+wU7&dZ^onStU#A(n)|BKq;5-e zDKH<4m~|GVt)QkN&KYb2F>4^tcYw;x$Q(M%T&NY{Lo`An(k1sc_Y-5HJCm&RiQu{4 z*;rYwGH#DSqcVol4mb~_Cf5TvzOa{daH1(RUHZK<+XobAm_4^JF7gvwGwV82U%=ml zDQ8?rE{tGnLBp?4pQEV6qlIP6Wj2!{(eOS^tF?H2K%cVFB-KP`_$cj)OZrk42L%hn zNb*W+I&~mpZJ{&wf4hG39!r-Qz7WnwY!X2h&40hHBeuvR$=NORRWt%_0(H(WI|mwb zAp4j{Zz+okGk`0)(C%O4tcERxeShJ&?KK($<-rbkW5*mFIpO^Y!1fbJegJo(HVlzD zmRtQh6NLL>(w4*+82-Y2OD2e!dvMa1)rGQsaxzfnhwV4a7?SrQUYBJUBV!2;dn=nU zz-YPl0kfQKICTFivm6gFoPu~zw*%`xs)5T4{`ag=j&k*<|-4 z)t%sMl>hMYKqiTK_VotW?TTVIIQa9x*`(r!an<-sYC)x@Lx#z{QbWI74kZdjwYG3D zxr&PUh(<=P{r`LYs;#05e^|PaA!?A8iaUQL#36m1Z3B7hRrz)=$8x3*G9)DEM~ym9l zo95_Xm2xI&OUEnvC-Nxu?-q};u)8pDTxv7Y?ItTXrMA29Y@IgKp{wxh5IuHzP;);g zMY2g1Vq<0=7u4)kR*gwos1t@JIO7#ve?!WP<%TKjhfjre1;WYo1}*v?Zh^c}U+~mgO2N;;ZuoTeawKU{wh`oN+-Hi=uP4}Lwo1D3lh==PC24ns z{Rxz9N`Lhh-4;`J9|A`B^$Y!pfnRd%XPxnkU!wKF5ziVarx^mnpVpG5F>QJ!!5F(O z`$)If8i5-QxR#pMeLeS|yUF+L0x+m68srgxVIHj8Dt$FB1BHq2<%&6#DxSrp<&TWF z1#u+x{YJM{a1Pdl?#3P11|r*x)SB(OG@5}&T5%j-vX6etNCDh~lz~IbUl~Lap))C2 z4ztX_-@>L$>BwiuqLd#OPP_59fs|~-4Pj`dg!d4Pt?8nCpke{P$FEm4Mme#yHe?%-L>+iMerR4oqP*af5lro~;N@7YA%Pe<2I&d7TD;8~4`9&|; zBSuntE>8)lX#p5+zHUpq4^5-iZT;0kwLJ|HT09(8VFOAlyY$$ZTKFz5N*l+gmU{IE zI|x%0BcoZe+J#z}XuCCS6kjn(3do`91*~TxwN(jy7x>xH1=JL#ZffE# z6OF$Iqu0i#ppCoGQ|qd8r&`6>nILs(Hem#CWq*^%LITF+vUJV{>9?=EyF7gf@{mO2 z+8HF}wg4sZJ~H;Mjxz_%r_?zt62v9BL1kmAAr-7m%H^wbI(VR7MFiFI;Y4P2q|Lvj zhI%SDUlaefL?k+>ItC604ZPCX4v{6C5Cxlho zj|-4t*kZS}#&bvLt& z&D$Jk>)0EWohbM&H#e>Sb z25S*@jRH?`pewHoqvio)J1%q4-!M#9Y*zZ+O_w*#BL!sD!Yfh892zj5`QpR*{ufKh z79KDzhu+l6A)*Ouk7|pGXK5dXT76M4TN!w^97{#2?o{gzt7m6_XCl-|a*~cU8|YMg zQJNbNW=R6E$ScN_LbMALA3)0oQac+65GhN=`WjmNDPlxV2OS90U>b-8j?EAG`3jGV zpBloAY$)J(G0e5Y6M4#2a!Dd!c^k#;17KOy0pE2xgV{a04Y_qXlS9FV#9lj-h!*GV zRl-=?DmIs@v`Xn9V;}abI!xj^Nwex#lO^fH{rI47%j_KXPCBrgq=PHx{sJ=3aih4( zq$MJLOu1op)_7!{Vr4W;M`%_?#K)$3=O+)tb_aed02i#flh|ep6noj@)c#6{9q@ywxwh46AT|SdT zfP4A4YE(7?!OPFFdRE`3z=-zZ%sJ{rcZ}8Q26Pp)UOt17|2!bIZkdXAGN$LAv?ll5 zuWWGNa!+Q5Ip{I5S%Zdl5f_rS1Ykm}Fe62$R^mTyWc&PRem^UkqRYjDwF`@TlmcE( z*WQIP0~u+B4tn#p|8QmTtvZNT!SV}0p!czq7I(8c3-ckoRq8D7!RiTkzojYG{UZ#x zfX6cKj>Ndqb&7i8xbE|%3c4BayAW-%G7NpFiD$1l*|*6osL?E;{vBJn4N&T4BoMa5 z-T_fz7ydRU_)PPDG(4UIX!o}UR17hE&fEp@VV)iI)xt+0W(2&r!9>iMd;h{mBFvAU z(Ub4mgfGC!%31!cXR>Bp<3^@l8dp62NK)Z*ell7}C95AJ)A0nrb2j%+brUAcLUT3` zwo3h7VOnbtsHu-Zt{18)r1&$Rix@>lFZ;eo>^vvqEvD>nDo9`MM9KS||wuA;f4 zu+SG%>zX zOES2$a;L=WuRjYKVuxbbKJmY?WkutE(y%7>F%05MH2CUOI8k%W^{e3|xlvh*LNT?A zo}L@gbLt%(^Gk>ka{zYWJ{d@`D$pV~*UbpS4cLW}+@CsvEBdiM*LY%fVXgd-5V(T; z(7J-E+diPZ@HrB=12GsmlnN&C{1U1{CiUVzw?qWiP*C$i@eUIgnaR`aOy{7zgwk<0>mqPe62w(;q9u_y~mCkW;9c z2!-3OQ#i&*KQDQ=z*aHN7bUlFMRCLzmB$cAamE+r!ClQcT#smhI>SeEIlSx8Q-%9} z=dz9uEa7PO#MZ@$&UM&1eZ+Q$ZB&Eqv@V}KAMYI!%PhV_!WjY#=&#}AiJf$pS6o#F zfa&;=qn!TnI$%Hq@?xE*d0bCs?wliS9_~f@!4Js$3w7#2c(Y`MZ6@3ZXnp3!YQuu< zGQhLDReFkA%+tap!r?db23fB(D$K0^%=`Q}GnSn*n%JPyDB3`N1r~z-pwub3L|?rL9m@q5m%Lnn>!6qxEk&REi{86S z$GWP*Qm#wobFyl4S$1W&YWX^UD2Y?aGbhz%Q}df&qb;asC(|;Xub1FnpgH+rhp>@PPc5^oFSIQaSh?HUO}uY{s>MNCTSV_q`EV|1@?SK?Q6^+2(1%fHvkH>| zJ%wm-C}sUp%&5Z=Yf`iAPK8|f77;Az40z*0dxj5OE2TbcCTJG$E*0zZ3i2S zV;q#FPat;T`y&IYE0=#?TJOM>gz>(ph(S}@gL4Mr^h1&LE< zrY@o`J7k~w7!jqz<@Z6B3v^xgSxNyd<;c|Hz)d4rF99~A9_?xl;U4%da)tiz#b+kU z;DGh~BXGB;mAmFMXUcBhotWwvd!$Hck-2oTWC?~Cn|q_-mA~U+e-P@OAwNBmO;G%n znE2LGqf4w&Gezb;*2Vusf2bW6s|SiA@_@E>&c|CkY`D#@47U2Jex{*5tS~f^`ae|0 zy5-qU+b-l-pf0sgX(L-jACu2muuQYbDF28PwY)|icnV_k&G%i}oCdFzDyMS#^kNC5wR8^Lf`TP7joT+0R{siekr#z;6{t%74rH zreRwq@7IhS`C-&o(jP5mRPCYZv$W( z4zrW@wX}K>M~4YSUNB1yRQ6MdHL#cGD$x?9rb^_`>Zgx6Aie=cZ8@`MaQMAJm=Auj|Naf`evML7>06UAVn)7 z=Ys#-v*c2xBv4L=x0qmO3SkNfIupa8PV8YhBg>)D&VqG-DIlfwu{m?+GHD8hwxLX; zX%?I{!c67b8kMwROl2=wEmsHXo8WiRFBiVL(EAZ{U0M`?W(*2 z_!$Yz{zQbYmF*)>4;F1w5*(cEJLi(;9#0Ljb``CTNk&+{GVgN>i1W=v4@Ub%dPOp# zGCt4_(thCFQS>Tw4RJ<$z5qP#73VA;#sBeNEevmw|OEj~_Bt!J6Lt%N3AZy9ZF|<^~l`$1w+vgsRqva)X9nd#NdLDh!FI zGd?S}@n?JFXAH?o3E6wk>%X4_;g4C=UeF=Z7thR{)wiZ zC*}0Y{S&5IfzerSlDJc%YaClI%g`xTx`+;31Q zMYoJ3T`L1ixA`5n%FWn2-U{Wm7G;KIk;Wq|*^QuUC1cNWoYUWnq}wR|<@EORv+Ep> z1!2$Vr>d8@j(`Bj(|jcfE`+jCZ3D)^$6lSA83Ft;*-D45k49FZa)QB;Q zSw)U;;s$WIv>KyqJ9zO1_~cO!Ce@bk_6AV8-#JDVmvAQ&3{LHrFt%o%Z3fQ)yk?4C zKzF7r5Q<|1yWVS%27}JqDdE1ow#*16CiNVoeRIrIPoG#f{+WA&WCzW=KfD{o!6fJFmLC09 zkzAF;?*KZH?T@!-Z5Pr8w{rS*x^($oYZs{ZNd+w4YpbU#)=7a`0a=G8X747TFMO-K z3~e@}yw45omFAiAn7HS)J_*WqxaF1l85R9^y3olgG(Way?MjY6k0(MLpEP02e^e0b zW>om!Colf4T7GW|_Y0ur`cvG5Pbb9!=iPu0^#Sb5y@cZ=koh4#TKK zBuAy43i%MCj?B_Yp|h~^GiOf;^wyD;+_<_>6#xvohJwvIon zngSYr^OL95a~^?Tq7Dj<9$hMWBcIHF*o(pSv>p1bE)>(r+!4JvZ6rZ}OfZ zP|5NfP_gV8`IFuWgwAZLUP+M=b>nrdfuY(~AfC*{5)pqrE~Oxr%+x1lh+Yt#hPa${ zL7Bh0WGfl>pwx`yU6$I;nZXNMp5v?4wyW!(Sza||@q{qJ9(6vL;96fnqYz0^SMI^>8%Gxkm}0JoT-tTgXJ zS-!PG1lFW*sk7jn6z#0A?anEL7VBm}4)|)?-FuPcgZsD_M(h<;)^(Eb(3lAr(Xid;O2@3<0$z7O%AVqz}9)l{|$1j#rDEG#u+kp4dmZI;;e_R~`+(z9B z_fw4RN;aTkZ1m#9Dvc0e@!SUA_K&!H12uhQ4)8bxa5h2hcH)NJEj5jKtKw#aS7zjv zJ`;F)%i9|vwJ|v>#)~5I)FbA+$ZL}BY0K8ycXn%N(mEzLZYOE@yNy-_> zhM<=CO8-tFvH``wnRfls=d7i=bX%YOx(SiX!}KsVl-4}l(s#m^2}x|_ImNJIPTo} z$k-K&7*q1BLjkWw0>(Pwr=P}GRs}pop-tAY=xzbA1b|q%3nC1qA_hpd^>jWQf7a5P z@j4ENC$8vC?Nh|lr)tsfvmYF%ztAy#YNbPH_ZWb>O%`Sqi#snTKipQISY(DR`P3Y? z9WD_?jm9IAT0bM**Q<6;j`By&n1}8bmhv*h&Gjs#+4{;IY4K|oa4mG2c(#IDd<}

|JzNa4wucN*QS@XAPBLFva|Rg`z<_PK!F7XQhF+H1I4Cy zs81(p{Liavs5YO%R77tiArboR$=P=#%!_NCc0|T|Kg5Xv??~`;WvMhVL}x^N*&b_o3^n>snWaN*;8Zo5A}H$)R||6!iz z5g}Dh8m&rB-FPNilI^pxJjCTECJ?M?f@eGmhyIG`;Iij!`V*5+)pNW>95Nr*vvG^y z41hGqPrNun7)|;o0_#m^PJ+^Ry@4!9<+(TX2HvIA3)Oif3wJm{r`glbGoh(1*E1Y8#<^o zdnxwHK%06~pKnd8z-(Z_orVGMpe>Q@p2quKGy1O6$D~_(+jFB-~8w^d^19bfow`wm&4;*1kUC zKN8qh9DYRt_p3Jf02_L>9^W^YxpgUdSsOn-p(gB=dVNKw`TIv*xJM)O#RC196Huwv zBQO54Kr^1FdiH^y(zl}&`$AdC{B2;agH1DGQ`k_$Kr?b4?v-z2gI|-WJyhAQUaF&G zS*fwCUfJxPV`FQtj9*hzndDwJ-b$dPUp=CxXsV>9%zMss!MZf^rvB9Y zr4$ZZ`UYD80hNbCMPid+RS6M|0kDZUtqhC$vPtLCfX6JcDNZkwh+t)QtiT~nttc9D zaj(`Y8IFv#>1xxLNAs+b-Y0j@_7cTs>`_xKR-+tE;8Et#pd904S8q`patWYGl&m;& z3Cbj%P_NJ{R!BLi%D6XSmo%t29)CJ_^X{cdimz~KmsKCtT*AJ$Y!>>e0ASF_(yRPx z7876Ou1wUaT(5divvlvl$xp37j-V`uU8=BY&#Yk>)}dV#kI<}`P)q61+p4ZrQmm?4 z)U_#jgS+F@x^L_lK0eps1$Pe9vE|gfxb$n@zOJclgk6N_6jNVoT$J$5t!!ab&bmyo zY3w%8chs!Fy_~8X?or;d0_@Z}w6s>}UUn|kJP(|6yi2=xes*;X{8G1X-cj~kzp3iH zbX??k*K}{4`Ez~8uDRiVwTNKI>Hfzty<>(`W$T7lhkqBhir`UR1MwYZ$+So1sm(Fc zr{A;LXWp~MXS`K`|Efx5=EY{Qu?KR=wTE&sx+iT(w#RdEqQ`KN7Vs<2XDP4J;K8zT zZz`N?ppK6b#Fh+sFOb9mdC#yRmQ)q zlqmoElz9&@`a>!WWO-9KxlzO`1QWeU(j|4}N?%(VZaeahJFtbnff@B{&EgF=IyFU* zfvK&Qt&5`_TK#4sxc$Z2pL~P43)c#gw6?ja#^Hs`v1*o^4D11SYAeBau6YjzhJTSr zgJ|MBcCw%)kcKmS?*}4PmlQZx0>ZI8(npt>+tio7ffj}1f7nv;+gRzV3pp6r`{Gm` zW|iW)@YqUkHQLijWonG8G=$w7hYp>WGaxUK?Y=>=lN1u-GZ(UBSrVL&K;5=A}}v3V^Qx?YQGS;^y25{Y*SPM`Lg{ zFmIxaxQMg$IG5f&1ahK&SLylJrGxy_PLV%HNejtem)* z4vid`K>VS=+Y#5d1JLu05!xeK;Sfgi`W)fE%dXLZta`iW?LE_njm!0i1*>gTA)2GV z`(2vT%8lRlJdc-F%EaMmM(lmV7>R%5Dfr+i)Fn8)A?TcpDmDH4?m0R(#!$Sn0J6&i zl{hd?v?BY~1=&Yme6|J-Bw{~@#GgBBBX=d10{J7Kf7y`lvhO@OlsyrzHfqSH93dUf zJZGmIQQywuSQjs#pgpZ<8*U30FA&_$99efRz^yif_K1G-sIU9)QvA-t{mzR8(>Cq# z97fL1=vUsNbY+Ula=&?x``z&+x=X7U_j_I7%!4Uv+CQ_=P6**h3fqk@!bhwuRxX6r zW&r!YLw254Cs$^U_JDbDSj~5}kL@|stD={TuOhp?ik^yttz<7m>*j_p4nh9KhOU%q z&;h{(zAD_W)wtOCKyGP()Y_`bG&nJ_$7meyTNd|)(oYy^3T4>lLQ+bh^_L^%Z}O7( zf@sH8-x$k#N{i1gmor{TgwXC+!4xJT{edrH_@O-FmDqB1F8x` zT4ic?%2wH@p~_L1{XxJhgKWFc@{e1~U7mBED)rq(mh&%0d-I$6$WAfN^$|a@wkNPl zw8b~HLh7z~U4lG0)=97Ep}Hbkn{uU~3MQlc5Lx^>w{ALAm@T}A%%`%MVN2^0Oq^pK zV8RHE{`81|n4PSc9%Fj+6!wg{_VqbfYvn?+X9L4y-g#PE(&mtm=A4K|eHUsAW2=%q zmMAbqCA4a)(@)WM^0HWVQ&ypKfDY&%gva?iqRKBo_ZL^KDBV>{#t4BXyq^n4iHCYa_wBXFk=j*9|z$*Tm&MKY}U*d$EsnM83qAxwPpN8(!f0g}w;R zsStq~8AqpMjQUtEh4p-8Cai4{iX%)uW$Jnc?$oFP37e8>IZ3=ok;5RKB}Y~CQaK@` z-2Z+3u}o$w90@vY4wa=O>fE`FLAT8?o+VbF3UNskqjEr|e7+ItuC%Ko?5R1sGOUe6 z&&d@a+{ZL1I7Z{*p%(x#!}~=Xn8Sr;(5KSvFMDQdw|*M z7A?;vuP=2SaTWUht7ll(4}<7O{uf3~3l4BZ!On{N6u&rsY$NNs!=nwY zTdsSqSN<1WLx;M*t*dsA2KTz%TZSW-jVrei=k)KvU5PzrTDLRj3a_x=8&fQunq1H6 ztHS(~Qk-_|?^>LGJ+HOu4xDdPADnO0*xVB@%YU!p+Ss;CS$hDqaaT8L7|w~8VLr>y z{yiR9Z~4sw0i(Vf0jtlw0fWzd0rH_|BNCZ!$)inqIBYNYE^Tksc;+HB!|OBA?f1-V zI{^~X2 z);yxz1g-+cx>ACaXsgm=-? z;FCqiH#1pP?j@07n>}d$iDscfb^|?Xj5LDcxh% zR~TXjXdh$o_e)(n6&n>Kejucq`y@A2!KLqMoQ z!}OA;XmZ{RPg|oC1@<=n35AfjAlUaii4OTq_!kV^&->?IW^`CQE8~!oxs%1nK`KBK z;45?BC-XV#a3HR3-kjEgc1bSad6D>Y8iU$3B-R}Y@^RC5L z15I#}*$_^w-?6F)E?O|yT76p(QTX*Tb@mwrFM7u` zzXM6gs`*K|jBYd=+wetX&jS1AHo3%ig11ws9Lnu^aH2gzJ>mVn#7h}4mnwk|HOU0h zY<%G3b3GgU^D-50fr?Uy>qfgLAgWwNqrDRla@3XAD9Vm_%Nu2`sz_hr{fxKl8v^F} z&hD2NFWMF_@vt|V?Y{S5hV!01-R~^+^}s_PjsS|+GZ+cT#e1M^SNsL{k?7@d88>vt zpPbsy1m~utp3^AQ<4R%;y_=JJFs)DWg@}7?B|YbC!kntxzGZeLl-g{2^a=U^&~Chwqyd)KmHvS=#(JE2 zIgbCH<6Tk&$KzNm?Tl=zdDZ7QG>59HYp2dy{-qm&J%jv+K%RV&*2GnB>N<{shek-IwIc8fhhTJEC3fjr$1|B-Y^`KKqJu?3m24;X4MAE&>rET05~Z4|Dq$%zrh!yOVcEX^^(v25H*9yfNFp<#~dY3kRyv1 z#e`6R=e#AiR?LSEN`=xzw^qnU4r+l~N4HkWhYX5?;zvK0C(9GX{d8b=r<~0ZMTHQ8 z2T`C=&Sv$RKs}=!%jJ^>4ZwqFSOa^}q2gj@D`9PUrGbGy0+NzFG~@_dUipDNg1A}d z;_xC)qMAKCK!i&-uh1G`z50_8ex*WL7ieg7cU; zqy^u-RqO$;X)B)?X3K8YA7;yG77|9tdR81p$bOa*M#y&73`WRt786FuW>yVG$YGWP z=A83bCnT5S*eApj9~3OcLIC*LgkrIL;#qtP18De>apfE8b0#J&W5g{m6Y=;1pBep{b3K0__040fu5Q3t` zdI&&SVm*YQ2(jPD&kOlINY5MjLNMoC)_Gzt$WfFXh{1Yy&@*nL;%>V!rh9xOA|aw5 zeuyLngv#WLF=b;7@W`xc?GU{x|34KP(lOEbwl-v_MiO$+WyxMg)Lq zcKNra?XUTg^L5iZF*8$XJ1%b-POfVQondy5465JaAxiJmNt#8d!&X4@H4<8WB-XmZ zeKyC@%~bpO`_s!UZlHpo{4T_Wbwl;o))x%g;(CY)Q8t-O4M;nfW#m<);evxd21}N- z>Y+|@wTo68H3F)xGt6F3wUl-wsN+DldS?v@Td}(`#}y&PAyPCYJRwin#=#9oq2y%;%WKi2PO1Q5LqHO2 z9EDv5kx=b%c*&gGhE0X$>{O<|`Dm=0xwK=e!S*64ImL{5W#Thy1 zP_GBO%}?t1=aC|Hcfc+Y{~`;&ZvgG#yIk$RfT`YancuqXiIT(L5K1F$J~^HH-9bza z-+SwweTS)Q_X*Ay`8viD<|2W$T*`;@l_kgv`tUysbyYuLG#h4sS`moBM z)GDvgV*&w~4mM5sC!@ZKU)htTcDAL6TZKmF>ZriiVZL$T3#kP1A~z4%OoF_k=Cg(v zeWntdqTx&jk81YW(wlu_el&Z=l;08msw<8+&iy2f^qEhp1O*_7r zqopc&=qrXcr$4UZ(B6V7;I>i0lGZ2XC+knhXWdCTV^7<}5_n|d(2Yg~sUG!&9#v6= zO|SvVcjVy@4z6bTx8vaZf^x|AxDN_PFL$}o16LY4$+uI%|2&bfaR2kzec~&}Kk=3S zC11XZL41;caOs%L5TMAueZEDgA#H*}@%sb|ZGbmuGJKv~g_uT8NJ5g9@P`+>bw+_n z6&mV4%dO|P>?aWabLQEnb736}7+5??nivZ_Nt&nz0#<9H4D}4m|13XK6%g2bUYQp9 z|2j(6YcQdwu^#clq$ODJBBwe3o9_7D@Yt$@T zRIAbb_}A$wO}n|O%!(oz-jLMw8N^dI^E6!|aQX4^z!WI68ATaH=@MhoiP^=K5#rsENuW3@E!TM* zNuZmznW@Y^lvamTtj7nYMj6vH@;g!plO=AylKECjn&x$1um&={SkW-r%x|u`(WB7H z$z9D7BCU6Kq}|SNM1I${a!MeQ7zuI*uGmC#gowF>piJNAC6)yMFgbzw30^p?tB()OK~YzY8_)c*6lvhnCg9YT5!u z3^CvBOXyc1xvP?srg9qei_%7q=IxK!C|bWd8MAc7fEloJIX2zTF$V{HlOgR@oQH*P z7q<&tuAP*tm%qoRU_k@?B}K7*OM5OLoE5FO;PFe=XL-Kufg_oqqFI{+b`_sCg-d@( zj`|le|EnqhbQ$V88eYtYHu*}vk&a?AWVB%V_p)Z8KS6u3oj zV42%?D?^!z$t6~eGb#$%BRw0doSLoXxj z{!avp;Vm$C;)8*Sd`p|NW1>z2^!Z`a;GZB-f%FU>q_Bdw`p6e(=2tlJW-=+>++NbS$-KZmcg~WQAV> z7vFF3jN#Cl1D^hDwO=w@?OgZ^dL1T?L|Vr=BoEWHN4Aq?=<6Ss3JFr;*Ks7BK;%+k z0*zd`2g95CJ{dG_w_BW9(Zya3_WTmmic4+U0l39iUk$%>4LRzL%9$^z#>#Ordn{-h z{@wp`qenf3fD1?lXm!v~ zlj#Nn%~_8NrGszKf!$S+-PZ(HiZgE9_&4VU=k`TuO6P!rMW@<-WXCoi?+p6Y92$MxY>xC0}Z^{s^k$}WMVC+7D`&_(Ls+%FXJU|U@TfWko? z$H||*e3nke-Pmt_=+sutEb}BkOaZ_4I=aO(#FcMdDx?dce=Hbemo*l-)l;*#m36iB zG#8Y+I|>@AOG^(qEadg4lRFr(V!n0t9s4@vZ^6nZp|Zwq`zmsF^!E&`bj}|^^Op7_ z0M?eXxUm=6?Q4YgD3ARd&?*b^I(&@{mp9PzMJ+x90ZwMUxR41dE-hYJUVvpUp?yw! z>9xI)8deoG=S9pj#Zq^TAuvm7&xBgb$IDc$f zTHJYZnsGKE3Jq6#1?yk6(aQbY-`rqy6_(lg$y0NiCD}TQXe&h>i=mTDu%2PS6`Z!@ z>Q~QN`P~lxJEXgJ(qi~h7J$ToBDg#M!F+76vbsGFLDVm6^L+wB9OGOCFt!X6!7w1Z zB39VGhwJ^mKd9bPG`50s7ZFT0V7BXIxT93v7(OwsKJe`u7j-%P<+SMN#$>TWEWDR> zU%D??dU5G7N~wt)Bn9VSZ}MkgfpIOO43*vCqN`~d`Cr&apSWMZ0aW#@09&uZd4$QH z=xwq^g>!K!Q_41X7!n)q9s;=jfs?gg-Y{&&W@@Uj_88auaz4*G35-FOh^M%WbP?VR zE#RERv+LB7ZEj<@Iln%&H|t3b^hqzC%D{mO+h{B0?bfq9DdU7ouJJgo5?E0(FR_-S zOVVYMC_Yx48KAHNx^CY$q5@m7>;GlwAB9q2cRO^0#ju`5Q>>DihMK9A_2b&dI)+oX z_Vd3wfcMlNTHm)mn?%Nm(k{YGS*9yuO4STMa99u=yluIT>|)?2iiQ(c_-?&yD#17J zX(b`~^framD|t%*W7W^JE$o47O$Oc0a8BJ<+{co1f+b7m2O>mNA(8Zygkh*(rM1JJe3P`?>gv~%);T^E&t-ZP?~`k=C$ z1|AIVH)m(c^rnaqNqEx=`o8JC3ftac=)7QK7!2~@3~RVyq=r9`;|Yf~zwk@_KjePG z7SI}IX`IJ`1LkhP&xOe!hHG-|jwnmv>#qu>lCW!lP?IF4Bb3GipnLscvi^I}KKP>f|-2Z^WW6Bv@1C3#ZkD3A75AlRpR>5O(lVx1op#PZav`UHF!M&dnZ` zcMFID3!RCTHf9y(zw?KsiBSXo@#A~u#<(y_rdmihmUgktW8L%I^IHw=z9y5(8Q_ve zB1=%`P?pReOCvRBMoDAOe@%P*Ifk=i&}ru#Z$I)@XW{ME9}L zGyKO9qM01cZQsm~Phn+$u*#*ZvX$*6$&9!Z)(IlmjqLB$*1YHQC&{dI{vB``1HBM@ zFt^u6my-^MJ{)`#fs3}RL~EnktM{T&{D_6oX^G_}seQt;V65m>HMJ=IMFDXtvraz( z^nZ~Szc{3b4M%Gdtv*7^YVC~8OJ?1t{NcsQyVTth=tUE1*^j@TXbN}0QihGBcVU5Xh2uTxfajL0w2i&{8>dg4xXzR;e8db&A;N~uAL7{)Kszs2t{ zZ<51YAm`z$YFLLh8V`XkGK@-YPSaKkES-S+Dj$UKi%csWgNS`4k7%B<_zfh}P^Xh3 z*MozjYI{;rTV5ckTeToPII+f&y$5Q#^>0BM*;Th{uS_=G$ZlgA%T>1vuf*tx<&VrF zRkv;wB~p(*Ioo9-9K@tZ(1?Ess8%}ACmc9}Gr{wDaZwVoXq0U*5yj7^#G+dxH-#~NX;*1>1h*}aBOJKeCj2gqd_^&=eX_$rq1l=$dndY#~x;#1yY z2E8%laQw;xrkn;0U#g@I#Q8sf@SV9QUL6qg8yd7)wtK8z=WOV8N6(&X$MiJZRTzB4R9-B$9oPDump zDr39tV#(H~G`X9drOL_|trYDWCe;et&Ti%%it)EQ?(YI#s3EnXWy(Oo5h?#i(g*bK z_(bC^zTS(IIw&M=#d$uROm&<2k|!Rh;g2Bw$DUX4h2Vm#G@b*_eW&Y}{v~0)xTnkm zeOEsuuOuJq$QtaGmxTCUVP3hTNKgaYO*P$Tp{HxtvuVE?{&r)vA zp@Rv#dq^h)cJ94B(&pK@%a3~je&EG8WK>Z4*@9rHHI7&kuPWwan$5hwv9@fXYC<-Z zrf;s9%aXTy{7MH=T)^W$+d5z8mX)Vq{E{gxbQ2q^#zDZJ$PSv;vxJ zv@S=Yx^I3liwFXdXtkGrTw9#^ybkrhE_PseXvi20W&Zpl&v6_67CHMmtC=nhDTO`U zP^5s0Pp5Fk(5Wz|jECJJaJ=ET^`6bsgtYIZ467Qz=;hLtCqEKc?&*|IFiYi@A(eSJ ziUVY=X_c}*j$9_I(hfYI#<uT*N01n7uG( zZ>}FfjW^*ga3^+>YT>%w+)!9P1K=?!VBXKO!BJ)T0oul=OL@Ua87iq`r~JiW$I6w$ z#clUL{P^#oGfkkZV&o$z--3lDb&&QQ@{~Xlls^vr*;O?mV6{x0BgwU40&lO|Qk|{s zQl4})F3tei;HkeWOGEG!+9%S&^&Jc=%gfx84(VHgoHeaORI5Y8V0s=JU7G zdT;BDkr;Y5N=LvvNC@#8xxnKFoHoE*< z6aR)npp;kQ@BKowZST&OD|f8A)Dy$3)uUycODoPCca;^^tTne}UDv0FKK2wjnvy5{ z)N9hPRp+T6y&P*)NL4(i+JPdpr7e|LcKVt6VeK~VryY^i$;t$N{6t9-5;-Dw1wjx< z$XKU=&$@!qsr5F*AXBrR7*wR!FW{cQRKuVssTfpquo>3*8L61~2wdHKYL)M)YNQ{y<*KC~cx5W6dJaa>xEV7UmD#DSPCs z9jr4r0eOzAMX>}P9;rZ27Gs=Nu{FR@YPj>)3{871emkWTCA1chyjK(QPvM5xrWO^% zeCChtmO%z#X$pxmfhcWDwMLbk7S=!{4kkIDLy8F3oVEuU?kF>Tmt$Ce!?1uojHnx% zMU<$PYl8J@t63D`r0FA^f5sx`f835KEUcJV4>am~34^v!1hGL>{DIpc%c~4cv7b{~ zz0-wRCpfNPruI0?xQ!|x$C8b~f3v@Hx*2g|wuwR5(x-}yqeFE;d(4Oz2GlVEy*?OQ zu^JO}CA&zm7x-`BOiRbxBm{t*XKZ)r+zQuoILux>Tf2(Fv@~Z%;hdPoi!`nI;~Kdj zY9PY^IYXe_D+6YW3Yrxmjfcu!UjWKG-{YSl7vT(>r_Y}LD>Sl>Kbfn6lQYI12!B!f zY%Lil=R7^;CdacmwWKmnHRsQad2C!1GaCoGt44FpF5Bja7+k+#^PK|{x~eJ-_KdDR z*(6%)7bx*9K5B^zMI{TlQxfYEV`nY}(GFv|=I1TBdHG!_2&N%w`#EMfM1}cz4DJhO z9l?<#-n~Te0|$cgZqZw77tEO}61QQOUxprR;KD+4A+!JOa=`}KBkWHzUJ2mpvIKKw zBgkn(=#>Xf+ZEg)&w2xhoLeXgMyLkcPzKuy547#g_!uhOLbe1ZX%NVI+MS%{s}J_C7TdoTPAt@%<2P31J#?Z9;ow!`i3xk zh0i$=<;P#JjTJRjTGo&nimsCB`p`FDj+K~lpI4K1B+7`sxm9uw!QPqxh$x;5{rr-d z+E^=%?i}!)a$o9#r)nTS?smJti=U6bV-?~aoVl0LIi1M#n^9r&@O{W>c>dl>wD%qM zV!LM)vZ}YOil_xba~?!z_Yr0}e9)0x$M&P`W6~yRY)%H5>Ba1E7(9%zPGzHWFdK5T zJ8Tk1$WwpNSe{8_jwH4;Q94v`?e>d0O6k^eCBy`|5KkLOIL3>g&Q@Pz#ShHw)f?W~ zzmV&C(){ryx{rJNHK(rhFF+Ri_l&lqwzrN!OIMPn^esLR|JTLmr;wvkloT$4*vGgc zy@-4nc3G5jqZ&>6JXfJ7WvuuMVrfDzfx z9_W*gpDnX>w6Z;=i0El`B|E($+d*$VhztO#%AUXWRh~%&06qwqjR48vj$ns%`#vO% z6%-B=JljAJ32J%zjhrvIro%wxO2Nvd_(u}k4dgSqbq>tJS${R;q9BSdYA???_#Rd7 zg6L8+$PA>Pe-7b%jrU3+E6_V3+}nkQe~2AZVyq$L{T(Ub5obF9hD1#x8|Q_M)(1No z56=)1=2xDn?70}lf_ht5+8$@!L3^4t9Uxl7w+sYG>r@-6YGg%n<3#R&Av+D9Atw%? z=znV*%s(T(Pw&8uT4Hi#yeQT1!0afeIAYbn*+`i_ixVbcGqfe+50>5$C-FYee!PTg znd4McZOFCi<)O+BX9OW$#VjBdPU|I9A;VMajq$^~Dg-Dj^!UZbktnt}(ccWca?CSCxnrc|cNH5; zIVBB?@Te{We|nkyicG_oAUns7s9|LFYl^PCUjRMY*K0Neu|*n6dN@Uf&y7VpNO(fE zWGYs6t?gE?O zHt1G;lEd>hf^n6+S<{EthX?TP+)Cek*kxo)sbX3YqMn!@aZApdzpff`gz_ExJri3x zz+GCD&VORG4=L%)1Qs8@^Ck_jn?bJI;jwvkNB#P(!RPQ2s2H>tAPr4~c}7aMDgx-z zcWx&x?yrJ3m1youpjLWEZN?!ojcXKQr(d#eW);R($j-&B*)m#>}}xrBuU(3S~v@ z&3%X~EMsAy?!`(PpjEdXTPY;oYz3fXrYMs#AE{Mf+*u*B6!0^bz%YmWoWLG8ZNc~OpXvO}98}pX(ZpoVCK3LwgO6>Gl)%ULBO;D?%L%(6T zDv@=+Q0#c4`_DGC1HtTmr}(UAv~Ng}dAYvvNs^yUgwEef3HK=~ha@6#H3}f=omY)9 zPp!ef)IZzbH@CO$N*soRACk6TV>ZJW$co)dH7#yEGdF9?dS$yD^9+cP%%HQOgCMc8 zW6!S$YFrAGgVA?k_-R4b?lokxu6m~V9x=H#0`4`I5npBOZigOJwHL|-H@cKO(9D^# zGKaA(YN0R1(4MQgSFIg;+XvwDk~UrGVC}+W<99YC5uOM-79zH1Fq@DXjMOyc@hsHv zifj&mcf^N)Xd{lH9gBFJb00A)A1R4WY97({PG?GjRCB>u0oAP@tR2ZFag5XLu3=;9ILe`D+|^blkR!rE`-&AZDL_!;V!i%G;x3`_ij0-n?7S6KC2-# zQZqtuLq^~HeZh}wVMAjC$t8TYzI;n(TKhl#&}#YP-oIf}Jn-5-aEKVr6JiX$P?il! z%VV(+7XFB$UB`^vH~q1Xk>80`*oIWtmhJk4z#mjsc;d%3b!4XurEj-nMl#8-KAtM@ zGaYQ42|St16b}N>#-ITc;vS)+-mpnFZ+HVY-2iB}%nf0wDsXP=(kEH|P53z)RnaP2 zqM1E1h3q->U)^^jk)l%kZ~`0PftU(leWhac1MBF4pWKcK-cRL~qvVtz zt{Tp0$OO|-Xfh8V=KevYaH;i^IrFOA&O_KX^OgsIYYcOJO|Pfx13zTDdf_d(#$>If z5u4&Upl>|Uq~q7PCiAjdi-IMz=nbnrc7C<`fW;@IXhEci7qOsui1k;fyn-SQYU!kL_l-|> z@5lzg0#Kg=z@G!q(f`4^bmUl_c#_C^inrb>w-I*aMBcoa^vVZiDT&W079B&WwVLg{ z)07L)&mH15+|6Qn$i{8a4s{-X17r2|5j1%Fd*DtvvigN+dFV~QdpL*FA5ch(h=Gb~H% za!%sM539`y%@!3mPC{k^tRAu(ciO98k4|?2V4WcugHW$kFl79)2dOq?DPlL^zj}qH zqnm2}=tOg4zd%^{&JO%d(iSC$$9@R&&No)6M#c6kG^UGGy3Po;fMziD(iqN+k-~UJ zo;Yl2Ri`isaB$W7YWtMnKjDhg_~3S$Yq$Qwfq}syKe6siUrJt}w~K99)8NlsP}N2z zR-w`Udwjf@0b8@C+j;Y6=wr1_)^>Ji&g4`2$IA3zYN&M|-5jvwKT$fRW%ozu!m(Z` zI$ED7L`QFuD{Ol;#uwI7qrcf=VQg*lHqI4xl38bS3Lk5DG^OXda}yyl*lc6-isTpN zgb7vU*BjxwwoPM_va@+5TlGfv9rh~fp`2ES5wqNrRjlvju}2lC5~CkQvRJtN_uooG zG9v5mLDkRTNUjvMw}8g z;WH~Hx|=KqzoKZgwSEODU~93`4W>q+cb4oopcUb%u`wrvvt>3ZuJ`ZJZOyj#D09?c zcv<+>ni9^0>ZlCit|qe1^40jbT$Mm^RW-wKvJ~(r6RrVA7H5^9hVH_-DcdQTsTbWw zG#`_M6xO#(%YAuWyYCV1Uqj}iRBq)Nw0MmjQ@&?~QVnwR;CMO7;K6rTCsm+l^PHDw zjSD{z_~TE!HW%!SOH0NKd6)9B7lfO;{9&Y9xNo!Ls8YEN)HhmU)M7NOFQ@G@>s}YG z6${`ffb<5~%xj05T{J74v;MjBsqtQ}T5~-qg*|=5`YU^ir1-2@NbOIlAb)M!2FQ}KQC0#h>wkRdshLP8^4FfZpW^wA)-u7!bJ%07vkMB{}&MJiv>N#P|`NNqw* zM;{F_LD5NRidDP3GOEt6xvvs_3L;|adDWDqR;NEs)jURh`E?@-%gj8!D7MF;%k83U zv4ucc&P59{qfKyL%x^uc^V%^@>qa*&?^i!@#XOde>D->;QBI(n6C$}aA9s^D>me&oG3a!3ew zrup;`sL;IVM0c0U>1IfIs`Dorkg&%=GMBFje{cEDb=qRtNqS3r*doUD`ZyZ+N(}7h zx=*B@cai@iM*mfu|0PCXl%F14dl?Vv*PqQMGM^5b46VFz2z(GI|M{FQ&Te1&vlj;V z>~O*3phz>OhC)qCaKS?XwCvO{weSK^;Q7AjFh`mGQXkS22RneaLHrS`PtWZQK4`0V z-k{TOyE-%_S;DjN9(|AMUYciHHdHX(6mX&R&wX9kiU$e{qd#Wh#Qg7Bd*_kQ<>mLA zk)57*m|f29$bI_Sp?X|3wk8u@Q|fTm(!$Ml5-A2tD#!9mSPUL)V2#P+)dY!j&n$9- zjdM?tC{Hxr3`LSiCmrK9ik0RUnF+k~0(W-x2?EHRk;Wn`w>4XL_S}BkUG3kXw8i;X z+eK!5hRll_ZBZdwH2enp2|^q=st!x&e#%W;>uK#4pY}Fhs}7E5{ak_H9B~ClRvs&= zr}LJpdi4kp{u(;}fZSZSyHCP}5_Jo@$2cbu&u>q$$dzNqO^o9$bckl;$uG`UkWL|5 z4jg#|@ZGqO`RF6}lTzEemL2ZgroAL%PUUQxFh(rokX?xVRMCxS^v2F69Ikc(ccMy}a$5uxRC>E^ z>-+-aoRxF^f8(153s-&(!!}Ah7YhI6x&s4bw=6z?qM0E(71a$!G{mo8QsM1~I&lUG z`Hh7On$yc&K%rcTrlEys7s9!niouXgQtr%xN;m3}hTp4pL$K0bl#-8RfAZ zW^_5NYJ!J|WH(w4PDTN(5uOUQtGNmPtIFC` z*;fl3JmkE6j#NVAyaU8Y7B0bIEr^KLhlc&{>bsz6y}I7W9Ox}gT?c$nH8sm{TW>c) z`iX(@3AGtKcfltjBdg+87}#S~f1H{*J{$_I8X>-8W-Tep zpT5QQUSyPO!!L-C`21k`g z?$`K2_YV*=C$5Gkk1Lb*CSc+{fNX(IlMS?8jvNR@s*d{L(X`SkrI{c0^2hLrn18YGdicSb*joA2y57<0vFiap z)3HTrePFT82b;;G?K-sIog&Q5+!N*K&1n_4i!_!97Hxz~f~2r*px7@>Qzj_@${&x4 zi-8bGV^y@%KLq08jfUvfqx;h|7G02l5r2Vy-k9(^VEi2k#T_n_^Xo!~>){V`(U_n` z@cmNfpOVT%N^{;nhxIse`i~jRvZ*9{E`QC6V3mvQGex#iZ71R-81vx$+ zB{I2!qws%zQMU;}hWoRH@C7Cf?F^eZt;T^5C(YP_4-TkF0h|^j@kV-9Nm9f>^xSAd8Q%0t=0T~aS~6ib{5 ztlE~o{lt@{KcG#UZpb#vFz1_b4YH8!mw*?6lV?bKkEWIZ8e5K@9dDyVUsk;}$P!Cv zA^as)yg$ld2|9__q+1gykv-OF3AT z|M&PEwk{hp0kQisRAcGx>v29VKi(8 zDWMU4Ksd7{)X$8~8-=ZMYaQ@&j-lk>`oGVq04|fL?!M}x)ZV>kg|bdhyK_nE5;^hTHAD9ry)2A-IV2(3*Se7c9+7*JK7vGk$-Lvf51ZZ4cn2}vc>0O8%M`@2 zH^awbc*MBLKBT!TneC>A)+w(uEGaEY6kZ}g3zI;9<_5+vd1ZVfKW*0MmEzoS75=bW z9)+KjC5s0*J`(@WCh7l0#s7{$mCty|14pMvK>>pVb0JML;e^9YOE~2w0GQ3}P2Ak3 zw0-=tEpk4DlQMjcyubgUNNRDHl~bUWQBx$Z>7%wGOUF8s(THp1aQd01xDD^tR8g*H zyyY9s+<+rdvy$(voUb5`jYz*Hfu^pnKeKeFE#akuja@N#dE0p@U%F5%^SnZxCwO@) zJagH-BlvzJjta?Z#2Gdl2E>p5F(Y?B$PbT?*d6L4QxY!NOCk`){t5;Q`Pk(5_m8U* zSYG1HOnCmDA^cN2t8hF^=(WUV*`6|-Mw?N$AlZp<`Cfe}AI30k91JUU9;yv5PCtep zT@1^CN#q-LI}z#ux*)xmQ3|b5QmYXv#swSlfk}0L&-kC6FMkin{QyWMsH@^|r|{XB zkee&I$d9JOXC|4$4U9Vh6Ty&6*-l{P${ucK0D!voe-(vzRUU}{+J+^9azOK_ge-z= zLL5-NXGPw0(DP+*=FbgcYtQwrD9y4i+h2ZxP0 zyQO4ro;)&Ax=qlsA;7R?o3(bfy{x}1+oJ2aEWaVypaFr_)jT3}tN~{S?Z@+Hl8BD^ zU)S8K1Fv>nic}|}V+)KuWtQA+cin5i)|V==0mUy>k>#dDl?c5-6kgp&Jz>nBVD)+Z0cCz)&pra@UY$UP

_X40!DLWUURFZzVaVlTKLc6wAV^V=BJY;f*6nALRry zK-D zHD6^^$W&zN0mbbU~bWHybSMLC%NwBmFk8RtwZSL5%ZSx&-$F^;IcE`4D+uWId&Ue1J zH|`(N(UDzI)!h;G6f!IGNvhRPBTD9Xm`qECVxfP{tAkl-91i1JaFQBP`3bVhL$*=q zciB>dIUWAPRyHHs)oTeC1_l8IEZ;?Y0Cqlwy1}H1-)G(>4My&*Y1g*7_|?rh_r1t* zDq|@&%S?8)8OSfi$}DMQ&%dh^CcKjHW|HtgFZQL{03cx;WzJ{qYj$HmH*4Sb^ z*MA{h;fQKYSX-e~72JcL&YBzAq6wJXx*>Yl-b`1Y$dZl&5_Izj3`^>phJuJ3m&s<2 zZ8PO>MzzL5E{#%-&>QJ1558}GIz9uz+13qX;aIb##B`eZJ4=;D2=9W>E);lb`Nr?$p0 zEcAh}TC^NiB0oRux6WB30%PD_xE8+)kwiuByoJeQ_!L>VV?Z{9yC8{3b45<1tQ(@x z3kj_?fns|eUmGlCIF|Z+GY(|HASsI!?<^mL1x)r^{t=c-UxXh=?>CrxO@!9fB!|B=6Lu@y+DY!JGig`6EM@ zSA;o}X3i5%T@#INe1w<4m`?f`nz+`%W1pD1L#}?dkIo`qxHWD)XG>FADZS`u93WY_ z2&glc%mu}jV>>NW8U&lpsReR93QIbUfSp-p?h@%srg2EF23y!uuFHU_{JU1h_;d+3 zT)#k}EzMi30=3MM>wLHjk@`-aT`2E@KU$his$L$Z0t~IhiY}@?9>8rp>>~ZHDQdEY zg|D!7{kYa?)K+PNaJjBbnunqMPzA`a7mvhvPg*<(wbpFfT3U3rSRqu zrES{!wSkKUuD#k7?x0+u@|5e&)zQ8OM_C7MMhLCC_={}3`Wht}qWs)q1JR^Iuq z><+k$Jm};F(o}&t+uBZQYJHZq^8-Rz;@JHx^>l1!%91SqjI*1AE{Ow<*qA?!X-C~B>F?vO${%xmwf(lZ6O#TnNxGl$_GdS3$K3GBO#w{fAaa)QD zF%L6!PREy19WS``01Jc4< z!tqy*z6^3Lt9gzSL{9F`}`N!va@#c;;~ZFGIK#5J;q0 z_L%;p5v}PnX4R{B$|aNXJHvSp3(8IXO|_w4*JB(s^`Zu+96B(Y z1b5lLo^&=+(KziM${5(uD3@96aeSLDL*;fF*^2SR*-P%DA68^rtG>B zw&q013kbOTN&6*Un;H#mPoC{f{?TYgXOBM2QmXlxBwXO{$#oaMq!qwSzyj8T(BG8rnjbsm=>%P(XXOsKfq8p_6z ze8tLyUYp;L1;h{Y1x>Ylk!t2hK2@Zl)+HKk5BIl+T6?GTY>zI23W&XI%Dwo4$=oty ztSRj3`l7(UvT4U#fNYCa&lI5MSycQ3xr>YM$BYCJM$ZweXMSLBAkOxMJMqTugFemy z^MwSzq6h?sLwjW-#UWiN_4>gZ44Hi5YxpNrY)6W(0%G`+3(_h$`b{5_+_R)>fbzJd zy;zTWa>h>nj^%)!kcw-a@+UvacSm{=4cK*`o;a1>=#)PRwwrAhw8kwKpB(&7Le;F^ z!>9v*$lW$1TK}X%#IawvjYE>%hSm)-d<&Jm!VQFGSsuRzBXulz#(d@5!`w@0_*fR< zzgI_<16b(!XQ58`RSVRBco1Wu7HPVSZcINsP?fZ;LAx~6Ai0zXk2t!-{_`S+y@`T! z`odyxJg`((fdd*zQ$ojxnN#)h9@6?<~~cuA|-jQ|V|x8Ml#haIBv1H?Nf4ZNJ^UCr8os#=7Z#RzTsNH_h+ z;4?bEBhwj}Y93S3#pmj9*?peVX1Npbj3pi2xI-|8i;zx96uvshQs}Fp9g%QC&eVO&GLwAx^ z&)4Uj+&i-1M!Mmi+SW}jodtw|@W6wiVH0e5gG9H>*E?tDrep90riTIa)%k^kEq`1B z4gN%ED@3Gr1&c47xwkEVG^)|Z0b&Y7>))4OQe)_H)*x@VZO2VU-JCmMrU_ot52whJqTj!PfS=X-q zMaiy6nG6Q*V>zWga-aD|tn;{Pzo{l^{5~?ZX6fY3#8+&DxZT^oTr0Q#tHH76hK5TZ zltV@&T^ehj*p*Fn@6|@~g{)g3K~0FOhk0|(jm9l_v3XK*-8Am?`9C`5|7wyyisb)a z$Mi!t4qy2(+q|UB@qu8a)ysg90hHt%7KD)evbdbAH#wS-^69Dn0nHOw?@Fr$&a1;n zlNN^%d2jGUH||y!J>m2T?)3+vS_~3H6@LMJsRU|`1gQ1_6Ph2#?BMpSy*>Y%pbqe{ z^eoS}AE*q+)pTjN?rZof>=)J_7A+TMP11a*WN<1rd@f~^7szj%ZZ}NB4A9LGh*MVL z;q~!weShvl43n-}xZ=NLL&UmnHp8Pbn#?jBT_Qr^VR9zhUj$QVVJqYWhpZE2=3x62 z5cSfav3kaeim4F@Uib|eWYkNaa#~!3THy0ZRyZxfi!g>pX%q5LY7)j$u*lWA?#yzi z4L5qt+_4T{3pl^U^21!yQq*B3DJ-??r?ox`At0NrcM&W0@y2d9DITG@SR2Y-(`LWJ606 z?yflp-BY%o;MdM(21NJo7et#`*O-P}2m=4d-0*)o{3m+w|Hc&l$J|hA$BdQmQ`)9# z#<~lF@Sh^s?c&YTA9TcmrXl|Z!Tb+Apwf1(IFZwKGU#yti$ZAp9-Q`_Ymc<*O-Y{WUKuey^95c(eborj9?O0lI)n-oHBHlb zU2ZnHPbPCdo^DPUfd~pO{ti#^SZmrFf>*(B_S=PTvEzE!ZnikZDZ)Qu7c3!~7)aIZ zy!ZX0A}9R=(5RjX!I5A&8crt}Se24wwoB9{nE-WDkK&hi!Y`FE>@ zbxx1xmf1$tCMWFQger0@L#<|L@-EJfpSszpINATW)MP+9P;ttU3`?) zt?L?Jgc{~&m8{nROj~I(S@a|T`4iOgg?|C#={+#!tJCSa?-0?wMe>476AeQoxGwDo zhrGekd>FHI%!u6Ko_c59l1F%I{oE$I_|;A$Ap-mBT$skc_%vnjQ6C;*3H}YU{2Pnh z1Surbo_rW=*R8QGX^?@vDO^vKKcax4>JC5X9EFY{=d*z3Ox!g{ecEXT7oPV z7Qo2V)Y;6%g;C7Q-pJ0%MEJjtq7L@{C1JwG11euTq~N9kZGec9eWh%NI;oXUK)>@;8@czB zUr5=0gndvQs^8epEeWBmvTr(a6sK9h2VWbc>M&8?n3)m+Cy2`~6A)t_I5e{i3+!yX z9H6g4Ur{<$gi3al-AHlY_0pwj5*69uYW1RJJF&?hn7^>!UGr55!BfFp;_xuU*J`f8 zop;wY81132eclQS^9lyMoAm_*0)qx?vek!-K&0M@4MYX1sj^jLd&2OnZVtI&V-2{S z=M!+|5#wgE;mcfSIJhI8;txy?C9?0)%1h2Ly?b1}Q83dz9fk@wZ9s>`ZOsTsENBnK zsg+=G6BTQDT#42ipI^V(cW>lwp!t6cenjo1WBBve+i9blZHEIOHZ;Q`_iZ&O&YSP- zZ?N8V3WxCDr~AJy=X=UFHxn*&?O(Q z0|tYRX*}dw@rk;w@%X*bjXZRg?hXqbY(yo%{}pxdQT##v=le6p7R*&>KtSmX zX-E;QglXT46ew*-d_)uf+rBanOzMB%n*Km>1_A${U*T0C#PDSS0;)_?phuwu_(A)s zE=&nz!`Jni?A>vO#})A16sMW~LSYx-6Zf6I8k1;WCS!QQjY;Iv=vwKWHYA*G5xq4n_j%ks@9ih|z z!>zQ!!Cq75tYRl4x1VSiS=uR<_I0>?yq>!L6ccIH6O+rVrl?P0OPCP^U`1IxWXWu5FD=YomjzCRY<#x*XOTb_K0(e@zb9u*St*celE= z8^a(pgQodlVg;Kz^Rrs|3V=OwRDmtFzTuU-m1-*!hgJ^T0JWdhCmL%FT5E|I44U4qzh%)>0XVxWL{-ly{mHZu>3kckvmwm8{Z$h9-A ze>%3>Zlf8GUjP8I(S)mw>0qPNZRGB(^Ft)-PzH0m5>g(80cItTLuu(5C95X)6q#dn zaB^KO!`8-5mgU=w)IENzCp;L8qT9wEJSVX;X%+ zGKvm*p*p}CR{1BT8k(h)ZB3{E8}Jgo{EfV3V(#6o>YnoOp$U+##)hsAWEV!^M9!E- zd$kkRt^g#gy9w81=savFB-vwF{??nM=gKH zmi$D0RTC)_yNxi@iDho+)FClc-Sd{9s2|`bxdH02C$co(a1DPm$}QUHR-;Jb3o5LD zpSig5*KdrPGwfSh#}z}(VV*|+9#7Mh?u2uOY@XNf{HtM9wo=tQRQo2&vCJsdN&~7o zX8W>-3QIk|-Cx|3LT_IZ_Y9iSd3WLUkx~?}2izGYekg#Yb@@=k5(Ek)RM`820%!INrRb|CB>!XwnM0QWpsI148jaSqPUIfVF9u+?YsIv+7M1@X?OiO*7_~F7j5SsgsSZY? zxqI;lFLA2nE)(C#FiZ^3Gq~F<7L?7gtUa{g->-sOC z~X>Yehaa^9ipZMD_tShaV>;k2xMr!q&Mx$oY*vkJ+Tmr%sp(GHO))MRU z(~2nU)XY27;s;U9J;3q-pfj3zO`Tn@5D3_Yecxbc$#yTB=LUuYiwpLB)|lx6@3hMf z?!7GG7EO2GygdoxfFgiU8xRst8lbg@J93#PL2l0a&#tK!JsC)jtba@>&&OZ1CiuxPu3)ImsLfR6oyJr+;4v;Z^6KyN} zhPAo4Je#O^ZSG|QVrkJrj@|AqgxAmjiQ+{<*oFF!OS)Ql8q!%e9^0v9QEe%hkOWRW zM*69ENZ2@kf`+eBo~;1#EJ$f*{TLH@HAvsLEGd}gg4Q@dv5i`(%}lA$ZmxA$LO5yO zt-yz7Q#88_GIS702@frJ2r$cl?J!ll+&ZY9hlPMaMZ@Fd)*WtE>3wP}hi(y(Aj~Y1 zJ;ohD0)n$jjzjDnzHMflYJ6}pvchewfsJxx%;pidcS^pt7`9v1qM*_?j9xi}`;Mo9 zj)Dqy0tLbyR1G>Slr#Q}GLbmitGvMF`4TxMWMd}&89AQ$hXyUcDMPQ818YI1<63F z36$v!QQ!3J)v8n@HC{N$-K!lF#R(}Dma69k{7bd{Ep0BwAtboE$QzK|;Y|a4jpq8u z)%>v5*ol=$Q}47V02yOUjqrxff32uyK{H~Y9#vzvv|Ade*AjP~fkWu$Y9?qhriLNM zHnhXMfL~PAdH6A%W`=fUCQ8*y3V%;jtA1*oS<@6-_LQH@1#neknkhqwSOkJ}h@~b~ zyFlopqQEx`5^*?zS=6mur>gbNE4}qmd)HsnOVN~5rx9fa0Z629@s08(+?lG}HoVhp zql*0=e4%cN;*g2wNY*`rH&z{-HUXJ^yq-*0sH*Beez{>&9c*IAQH|K$zaQ%KJ1$SB z(c$(F@J4?d2y&1K6d3E>Nl~dd<9%b-t@dtDO?_Hwkc5U_f^)DnH@9*-Y1gjgE}gGB zFzYaPMB!Ig12zV||BU&if+IV)DDT4)-*H2R&wNyX328;yWGBmwH55L?C$jrV(^AdG zQ`k$RFaD$fDeP!S&0q}yuXTtK3)($L$($bcRb(nlsrlHHGrgd^e#w_+YiWHFcmMUT zW}jo51qHsVgS@~MHpn1nGq7Fbib#E)v({E^uj3lY1Hh8qC1hVr+b02`sB!V$c{DYJ z&>i2tg6&BNVRxXc`>1J?|aiRbPfrV>4X@qvYU6sv=cQtyolMO>ubN%ng z9Fx?=RzL&%U8IFKL&@WHrO3X`KCbYcTi+W{e;7tB4;v>(vzb^U4w9@^UiVW2sEwuU zqt@J6EEm-q{+*r#sPM(9z@dX3;t={B{T*m-lx=~54huY?UBpPrBU9CkU&vtGU|a^N zpi!Dmmx|S{mAn3Sv{K!Cj9Hd8O3TyIFCbjZWdO&=;0$(ytT^`+T`h40X3#$7ewsa) z>egQr&U&((=B7=^)zP{k2hUH3El&~nT?>Ib-dNmtdhC{`C*r#(Pd=G$GNk9M8u&Dk ztFHiCczl2J|88V@yj@AtapWafc}qf3awHCUB{7CIjme zMI^ZWvI)=@4ilEf&72Z-0ZTAA?oczK1n{Q(Gn_UB^zl&XxKgDM?Ek zmxH4@^ib8wO;J8!0jQI@A*oHj4Nx;w^}wB7F7Mn7!b0ch(M&{UF}~JqUJ|?j((@0T zK-+q>gU_^gs6D=4lFl)bw~l#CxGj^NkqmQOoX?%PJwZe(=5Fl4=Z?=>TTYQ7Wn#Ao zeUW?2?qY2nbd4M5H9KCHScCa?NpG`?b&+!A8ANZYi;0Rc9@FWvHShrq1K`#E&dmRw z*jw$SZg)=0f-F_tVbLG}vIbp7{qUGQ|5k;uM3>c4?ri_UT8ijv$;D2_ITi-=l*9xT zi5Me6IPu&_fLrL)n9u>NvNj`Uu<(%O0#g`rX}e5s;AVqv$?+BX8%-TS3Xdbz%b4qGL5Hq^s-ZB_EW(9!>HDnN=*PQ zKfarn><`Tg^Q%t*9YC+W2(-vo??f82w@IK`PbEPuraMQ&JKp74%uZ&j6bC`=lSW<9 z!hpDLqY6=t)AmTVBp(+8rYD~7@~`d9#bALlCTc0Ou!7@-b*@WdlW|KvD%RMtOpa)a zbaCyy6u~{3=hkfIxa!fuBgBi~0bGbWKmDIR{F%x?{V2S|9e^u@e^pvC(yOd;sPLPf zp_0ys_${)@NGv!8?-Jo?89-aFs51pAvBjA`?w?Y8l%;gUzADb&l{^S5BmlifFUihm zSPu|)4-lo|LVZI;tDO$)dMWDbvr5Hq)OspJFL`sdI`_|BEUYvHmG3Oq)>3>6!<6jZ zXBM_hd(G7?5YSMy7V4ruqh5jdLvD^*{t+=N@an0SRUu_>Qz0{{!fjVx32j{*l^v;$ z6J9^pFGvU;4}|8t2g9s^JY>lP{RXn07?%GhJO0tNP}$4k%Xo9|$6MGn#|FXwM`VWEG(T+=6TrwGx9}S78UM-tg~7GOCQLj+ zudhG4laH@DgOMN?jb7h?qsiVngy|P_r(E`1~+`R3*^e*xN)q3StE#Rz2Y|B~N zl|rnO0HPB>M^6WVCM4%2A42}(EPzBI#PB!Lt^qi^GP~-lz%Q-U{-E zSn+;cfy^*W9BS1|RB<<;dNf;RCfs&QElw_v$J-_AeSbB@+(Q1H zqJdrFP0c{N=_R~uGT)f%rQF7<>2ol&w!;XD$W7$7LF5*0vpEyb>Be#72yVUWs4nI} zG1j9edPFVIv}mA4c*mEw9L)o3@zx!aYkN$Ta=bTvpFYH0FM;Za%INq7M~_IZj{qpw zYb-qx!~sKr!co1^b?6`(=bhlKMxrbBYQ0{I}) zIO@?C%h7M(Lm&YS)+ZSgqo&|kxrT9`(?PVZS^F}|f#Wg`gDv41**o<0Q1yZt+p-gN zjQ4zc^e--eKI(J$zs=R$S|%roPXKCWFG3+E7pzxA?K6_z6;Y+OPIG>px+*(0Zq_%2 zAFSnbnb)HH4n$g*Fmx{{`CVUZZb9-`p2WX0<_qk}4<2)0P?NmSCT|4NKT@zj7R*G| z{k9bS9_+}+){)cZkpjXyn6BCW>C{VW1p8}3pOSjT@8M|s1idtcCEmrU=K$IkRoZgt z#xh@^G4&z^#;sufGXY`2q-2IY#wUusiCqgkXoJWfAqFg>tFmcos7k5j`bTG<|N25=DNj4cp zrrzU`A7vskno4`ehFkw!0C`TSZ`SrlOz?jvIPV+U^D8`w2uk7IU+;ftE^PQ!SG>z> zpG^H~C@*YUs!zNd37$>;LMSiL2jP(`8yv9U${jqQx@M6MV4X=8c2D$5-wHo>kOyEu&hmq1zVkq5G&l3iKGGDT!D zVb-5d|4{MQ;XGD?WRsX6c_>Dthm|_cS7IHe)h8i1sipyCaGDG01rl_+JZvR|8LyD& zL`tI;s{E`VxvjkVDqtV@DLkd~pG4`c8T~=PJ420568|KnPrAZ0Q{JT^f_*G7WVrQ; z{^00ec8DP%8nj@`WU68(O9eEgmNYt+?IM2>I0>YUP*R8IzSA}VxFSjOfPW6m?=|1_ zMI(W82Ysi|hL3AK%8uSHAkoBY?ZfbSJ)4-*h*{=2Gts1Q!9%ocD@qmlt4J%t7 zg>4tF`LXK1F3lgXfsC1$_#n)V0uEVaGkPn;pW!My6vghk`CG2zkyU<`IcQAqgSxjS z1od@=E`uQ2RDeo$h2M%=02E(HNfZ98IIz_m77@V)JYt81@o!H zH_{5Z0oQLqC4>BFp z0-nsj4^>F&9wSBfqgHpn_HW9(HwQNt9E>SB5-yZn69C&{#0tF$@2teh4(mIxZm?az zn^(#nL@Joe3tLNCs|_$T_%8Dd?-us-tMIF3*g>N!NU0*yYa8&tpw$T$VO0BLHb`4FAJgj z&&hY0kW7N5dBtO;Nsd31E?VJOSym$psI|#FJZzn)d{~g6X93#f!o-8tnCP5tsIZXw z+7aXw-UYRwH^Lf1BGN(>(0cO7X5)>>^wDYC;(%1AbSa*JpP)2Z@h)uiR3)R4jUjII z-g^92gxOzamVT)9W%KnlL96d)bAG1ch&K&G1cZDSj7%*8p}We}%t_8evSupbvzlf; zcB{MXFy?;01H->qXq1dJB95cAqL4u?OVewDYN+H;Qf%8)jA~wS(B7@Y%4hn+_#ulv zx6ITzi!K26IcjB1fw z45{(Qw;-HZwns8d7w{9OBDSD#9l>-f&na&Ty-E}kvOyX!POeD1X88)wRXJsqIKrs4 zEwSV05=BsVVC9yOV+sz)#+E$Yvxf&9%Ra4 z;G@f+nTWdPaTze%@^12oIRm;Uz>4*}m z3)wqPlBP$NVa|$~Xpv5v94MJl&IL3P)VbqUrO{%mXt7|he48gXm}f8(1bL>c;bbY6 zvWQWwhH~Cl^M?tAd6qw{92@1bY^5W+g~jdeDV0Pphp!puu!_RS@+`guWhjd{kTR$ExzN_YFZ8V&JAW$4W1$NL^SZdygNsFO!b6AdH|Ro2w8+#KVhE0t1Net>P~w z#aUzr%z#ry1KHFN3dI}aQ}cVYC9G=jM6%47uTn+Nk}M7AC8-F8sZJ+QZ)g!V2Hdapf;2O*?`V>$p{et{SJWV|`ovrO~?zIWeHsUJJit4)?Vr1D#RAgm8hj5mi+49jxvf30DN+Q#x9KQMt zC#T9yUQst3vzIT_8YxZN5Gt$U7w7d@F;A zaDPmrgE~Y9e4%@?|LCCvN_%c*Z#EJ5Ub1x<9ZHO=r&ou9Sgz!~Il(^4!jxF==^#d~RpN>PdZV8bMLoBemT* z_9#XHmhBSmHCs`N_B(_|?wi#?OUy1-lg3Arq{Le0pE{Kfo$NAuOK zvj!bYpzx6_#CS$;s47Qg)Pp~B@4@kP10`XN3+U#nlV$QtL&D1`=rnG-ffvS4=&h*6 zqL}&c9n-5-Y3{UE8;_vq+S!cDM1Y{9M!TqGwnf(f_X6XP3+LL>OvlpTu*@ua}C zUkT7*m_O~tj~7Ia*OxN?>8cqXXrnt-+}zTGU`*a}DH>V7U(hJ4YG9zTFTnm{v`|H# zUW7yj;DHybi4C&*=2pGqjl%;i)q$tL4ffwPgCv^#$%9Ps{{{Sps(4n$BrI44e2>W{ z)(b@QFOkB8v}smA<4awMD7-r0NdSJZ6*lL}e>&+o`H5QX3Dili^lzC?o@^jMug@90 z7refm&pVj*7LHjx4$O#WL6P8imD3vsIIhtJ(CjdeVrLWCO6w~TY5?6pIVZ4Zj`igl z`Akc&nhEKCQZQ{|#kZ^I-jQlaSTm0@)iHu{MG^3)?EsfAvbQhkw5mlcPfDj*OrelH zR*ZI>F5f?YvTGNqSguzR9HjvEx5FY5+bYwU>_8!NnA`!Jpd1Sb+X~eDn*>G;i->Cm zAVDUNEk3*Go5v&o!RTLve&|r+2SK6iCw4kRC%bV*>9?-jcMdh9FTc;;s$8;0sU}L5 zh0&4}UQj|76d z6-(v;c7!uaQxp=0U7~87JhvvnEyp%I&t`%*#WY>WJ-pW3U!G4tEVdr?>~0bzfC_(j z3|(4PkH3TG0y2(sCoVTnt(iX83l~dENkETCYIxwA-%T6(!xL9dVzH_omObVLKs^et zaTfO?>#Ag}wB+H(X*KLBGTH&gDJ5|YIgr(&3*g#@UF0=jL&(xgG8N$Wkn+rUMF~+T zUxr^t)Gv*&DqqGB7nv`}54E*PAt{AN=Th;_$Vify1SdN7B045Hcc2R^XDC6IfFDG& zYHJ8O#-_FEZ3x4_wXy1O0Xce=V8-E;;VQ|Amh_OAwrYW{i9I(d?+<=S6u;`E2CdedKQfY)Ke|i1_tI1!?7TYeJu+Kb5a-}-B9R{wql>XhFMN`-dm)yQ~YGcjUIoLQ9yF3NEJ zEn^ulCnEY=orZOtXn4c2v;iM&N4RBJzLL{j_%~w-$AO6}`}zuQ+QJl;&tFFvU-m;p zSLG028V%1kdmwolb~qh!H)Hh+yqy^G-~}Nh>;zSY%c; zZ9FrXG8JR@Ew3i~GF{*5j^NMXW>K^Noc=`d=Z26@7RZyj`CH2Zz|$Zw4PA4gJj^yR z0=-Mqsm8)Nb8>D6x(dLlEy`Va1)j_f^@VEaN)kFpo-u5l=ggJE&i#}BIL--%-b&g# zpx7zCF@|?f@#39EX0Ks9_u;-DQRj&<~U zurPI>GpTm9kq3AMunYn;+%$l^za1wwr5&D`=FetL@O8S_{Y|R;K&|*-rTBoY_;8Vb z%qcWwAv|^Ra~GPr3Ci37WzU0T2S8bWK`nj~m%l9_icST13h|w8$+0Gj%MZI{P|_mc ztq+Mx?i-+vJl|b7V^zZPAs)17Uqs8qdJ?LsS zgqk04bk&$J)S%V*Pt{@$SkC^|+_hrBafl)dGS{yP*yq zdB-{RA#(JAbM#A;4-i%DKaszl6mIJPfq~xe410ABU8+Gy>In{`$aan>@T}#x9v+HNmXSZKYqJ9eFW_2#jK0p1Sq4h zErXg7AK7{*DlOY?fb{*vTFJ&H5&Lw9>>yU(xG>zC1*(8_TNs`ZHp^bCd60$W6VKxh zBpKp&F@ty$gU08P75>)%`kQH*4h*ehnSO~`+px1x?sY8l|)9 zkzGA1m7g464JWTSq0nl{9m1V%sZKpVneCk%PRcdLB+K8o)YsN97lL^6<|Y~5!Iy(M z_dCDZpMAq$)w(>L3YbN~?aU|T_Krum zn|-(&^C*V6&PHi@X&3~F^;4Sfk&O6{^pFkIP{Zw8{e_R>-I)9Jd9|%Xq%R8m5KaGVZv>Q1|n& zh__AG({lqaA^)5gr?nO18wlRaU3lF&pclQK>g)^d+c8lJ&79@)ruCN2H2O=*cPwH* zrIuE9rdm#ibx#%qaHly>d8u0F!e(S3W>koXcn~b5!xA$-oRU2FYCABL)Eu)|vuglm zF(;Q?|LTg4Otw5?tt(`?NtcEx1;~{*rVHF9Mrnje%8B7~TVw|v`Gz~p+~dY;^JTid zD&YY%4`qX&5uHIr5QumOKNj{AB*o;(9uBKkRsKPd&Q4H##|1KYsWvvDIdr8#C*za?y}-pM;}$ZG z?QtCZgHqE8SJYR>F`lV2!EmaQo}LaD&xyJgX(Vo*s_kC5g7*@*^b@6L z(8y=DB|=Z^a=xgHc7qt@Tl55L5*-S0{{khf3f!QB6wg>eyR;98y|=yL^Z8RN!1z1; zBE7V20sD*{fbDk12w7%d&IOix?bXTmEx*LEa@V%SX<9if5_f2S$`S~OIz*azu9^@v z=LMfSP&F6$F?S?cw?X8Y!fMflTienuhW_LL=cXNpZJj{Q;6-rTxJ!Z=#@E=2DPl# zOLbnlf{mT%is`2`d$R`c2~o6F>BR~#jPPN%b}rL$H#=&?_*4z|H2UcBun^=II9EqC zxlqxVk2~>9yH5^y7~=HGa38-=+4>V=oO|i{W8TbV;PFhL%=EwfO`Tt83Kg#xi@iyq zFCh%0_ibtxlf4N2kz?rpC}YLW3+>F>YGM8?NkViEgTmv+<^a4j)ZE`hxpIEOwdfoh z;#&G0K5O9}ZE+Ac%;Kwqvjpi8*h+&;BWB_htSesawGWPZC(JH6=EshiwKphvawUl~ zYSnCQ4z7MF4$?S!4l9rBNdT-Zb#(*R^_QsmC!{_rStnkcz}Q zJ*4e{SbW)>4FCXc>g93<3b9uzQttqFE=p`4-~NumRD~RkO{9zX1=>%eKbKBjx-Pt{ z2cW#I)R%R2P$8f_F;J?j;XHLPS)B9n^;^RwWm!I9i%*Kh+3MpJmBk$i#?E0pXnpQ{ z=i4VrF9LC$FH$dNaUW=MZ~1TeQQGuE-FkX{4STf?DS#8t6mH7ncSx>p1o9NG-_ElB-`F`93+TVEWB220SrFt zHxKM~^=5@wgWX;~S#cfBOS6}{P^LS+U8OC}Xt7*;)cVIs^{FZJILTEP{oIMM?(My{>w*)%oC&6_+;KP0HsGfUF# z3R1XpbHL8Rwl~X>>V}1k{hc86pXu`=0wR-({f1ntO&A!O;Ju97>j$U?whp3P?VQZa zRYX-Q!dj?);XE&m6bSUUc3G#ix8S|2MGP3P$^bAkJv=#^6b9#NF3suqrE|HkB7%fC zZ!v*$%Ub?Ig=O_R!kKQW3;5VZ<^~M2lJaW$wp(cHWX`?$Qd^ z2EZ*^3{ye$Wfx{*maFNuRe}t`=1jBR-SRCdS% zIQKaYA!a3Nk}KPWt0&3B^pjK6B%`2@8Ni0pYs{rUEp2$qlzOZA=#0}Al1n@ zqmk)?rv)+Hh!O?f&R{7tSnKfc!{UPsRG$pKecs&qs|p$znXWNU%qd#BBw$mgP9*^C8e48 zuAC-6MS_*FuY%VlIoE`4mNRTIx}M;f-fFd~upE3_Gu$xfrlj1BFvn;MQRWs%u9+Z+ zoJ;zlCfq9H3CgZ<-HLU6Y8iSc2H=nO_>tg#baIkILL7DrEE5307X35gm?VuKDKVrl zIsryc$4e|oY)Ob-&>%Uha;T%Ky-h6G5=V;=&SRi{mU~8QASV1~ zXYClnJ=;RV4LR4MV$?e}OlzkYdAaRO_JS8?3kkby^dw0`YzH{#yvOkk1%TA6tzU%P zyCB})tPye`^y(6|E3|qEYIK(zUEi+buTFY7iY(YQ?&GfJ{nYo)t@sVD2mmhshAIE< z)L1kK8_anmyOyJO8bc&x73L2nnG>E?26oQRML^r>n zoIX$ouh=6tE|A*(h;D8J0^0wf?msh1YdXwJG`DFxJdxQeVYCr_-)B?wr6`J1^xeLq z#wArB!}B=>z(^O3_ll1U^e<60e`l*B(x_|n!}2$WPh1?qOIMlvY)TCDJY8U(Px-OB zUZ{g+3OKB`M_)#7P#vN?ERm3KD4%pLNZ^e2=f@dti}N3~&eP-prfzb5Keg^1CYiVj z+^huZWLL(JeJW;9@1r5a8g>XoPYd7uqtY`SjG6@CE15<0^8LtaeEZpRl^j1j&P*Ql z2t3@mH}_IM6F*rZm=diIHM$>ysp}t2N38Poq(j}g;hQ8EpnT|Jnn&ieo;@pDX9|~| z*P54ef9YI}B{u`g=E*%<2XglaN)gfeUPjT3bpZvQ!-9#0QiF`oE)fez9g+8&Tawuy z;LJ-$+s}%6$Zq-ewNGHT+0%AAv6G2y+jcUsZQGnU>0n|_*s*Qf zwl%TMi8ZF{krkU?C7VL7ZV3OYoVSEY%J2%J=`U`N?Ucv9YapZg~x0aAP};#OJ%j z_!%R`r?Oe*oL$SNwOLdZC7T2LE%lN&4w3svxr;ycW$jjYjOJ%un)oj`tBTGriWM2l z2w#Cr8^%Jj2NrDy#$qpTa=QkN6F9{#pI(iyXDVwtJf084C<^l`c~JFQ1S)skYMzM5 z=CiNgQs!^z(^3lV@l)wrPe6yn3ni=RsV~dq$wdp%$PS~T&3=rZz}c^O{{!* z#~N-q+F;5*jlVu*tl>tW1U-*scE-IuP{6M0SpM9QbmEoTJ2y|WYA)N5r(9zU`N2yp z*5swp>&dyii?n#YPit?bV0#rFdo^&4v}(C~w>;2FBnHSBiLOE4fHw8f1Z}*N&Vl0t z)!Rls28|WM^Z;+`A)SHPfNp>SUxfqp4as|8XIuTlmuY7p;d0Jc0L}IW5YBl$080}p zfKkoo4|CU3Gzf^a5o3EEGF;(HX@kK0KAj1>_eK*Euzw28Qde`Zj_bp|x4Er_b{pej zZK|nKA`TEJxKNDdE2CojSu|?G1Hhvte7Hm?yS0DiI-Ub*-fH6xZ>gk++L2&NWG8LpUCQ$b~ zjjb!sP7^_%R;avVvn=${HIULSy<&{XMS1>lR6Nv=oPcyV3zT{8;=|b=FE;A>!#2n1 zFAea4U>oB6Y5oGK)L8u!BfoPU^ZR2~cBtM_Nn7Aou#6gGgl?bGhps+;Et-gRac(jE zT}AMZw(hDDh2#O1zVKJrh5MKWyk3n8g}o>~FeCVCCnZLkk!rE)>R5Im*v^obn}P)# zQkJEbpslKT%+t&QI>PPluGqSW-x14tNY?kSe?d;oAkV6-o zLj?YQzFw;0Uc78X))7JN4X_CZUZyiaqAYYh!{5~*p6r}Z=G`P+D+zi7k6vB^MyC?} ztp4&&0f#q-&z$3JV3~#fRnOZ(7N%d4ww|bjSS5Md;S~QEU|Z$5u+6NKd^elTc1Hxv z-@eF(wQi)aF}}-<5q%hTglNi7@%Ds(k$0*Zy9U*LVNqcaKP%m1W9cGM3m?z8JR4dR z^Q?bz@9#X3&)eIRmU)4r@bWUx4G}kaA$!K>_YHi^qfsU_Xn)f`d1#YeY`9s9?+r6<$IU~0#C#zSu9DCjxQT_6)xf)nkTKAN6`$l4 zk=Hv5K+^#SF(0@2625=_ruzIZ<;CUyWmX|F`M84vvQO4>6}L3B~%~%QCPjnf|>@mXHDS z-`<>3)=jX#nLE8!x+~Y8bW2lk;8y{7G~l)|91K9!(%H<&RLsN7LxOuJM!nh5o5>L60Ya=#-w@}kJlkvO827?rS zgXGz>OtT|(l~J2i^dMDrj?ZG{Pxs^%fR`)uR!y^IJ10imTF;c~eohhp!!w3 zRy9aFx?-a)u}h}l%>Dz* z&ix$={}HvyXT<+eiU+)a>C+1N1j>~r^$1=8^hqDDp^BkEcNcXa92L9L*Rz=kY@u`30 zQs=HN$B+UR11%cdA2(e-J$&7}T-@KUKHl!HzQpXH27nbMG!-7v1B%A8tP17U5~}Pp zPlRP~np}sYY=-J_ehA&7Zx+$_XUQo{>Be~~h)Nn>eQZ_{lg!l3YQ#E|IvykU5%3ie zKZF%Yun#~qA(@B1Qie?H5n$s_A?jt}VUHpxz=Sb_Y#6lh3Gm&o@Bi$%Jon_jG4p8(S zZiushy4wVIZ!Y4+G1^a1u|IDBK2^N_JfB4E%1kqGUDad}r16Yp6ma*n)X;W+$?Vkr_ zu1E7&o{gd0<%){IdEdv7$;w!sA^(|?q%VIsUUfnrgWI&D@@r|cHsA&-`PJ1mpE8D2 zQbilde6Di5;PT!R4xmezoF96p!bYE!o|_hqhXn)!gF2pbjGTI=tyCt$s7O0}Ne(V|Usxmh z*kbw_vXF-UcjXwQu`g-!Yr=IbrZude7QsQ1Z~eyng4 z=&ds3UL4_1`n21U6#~MvN@PV<&-?|_>n7Ve7sc*~F+e#d4mZdupg-M~aLx4tj;lB# ze!do6PWtRftKZAFY6nz{^TK9IE%{)8LYRS{1G)L=rG~g6BhIDtUU(n6r+!X3Z?e_PjACic-sln85U{sJim^7DG3mk z7`_Oec)&_gN`{FyR`S6_W>^m4Yf5LC$dE2HNr+@b}qU<8A?13z^J^1 zn+P633?XqHylim;-Go77DQX)9?~R036WP$vy19P#-?9-t3?=(%{p3WF!w7U`{f|Y~ zs%rgM`skKkJ@WlF1Hu}jEDqRJbl}n`T=mEjQvjDyt7tFvamB~X{5=f6ufu0g#G}gU z`~?jbnR8iXHLJWVuPxTvrwd^_2mzPFG?gLEDMrFmi58x7wt?E2b`E=#FmG6b=*Mq@ zienFuOjmP$sIlK;XPR%aU~dH=bk{W(l08kFgw(g3{Fps64r~Y@CQ0VL8)mdy-o%8qvNjFX>k~1v!t7CBC7LT4*YsI4nT;zXEjw7p7 z9_MqAzLF=B-_>FF8EWdgmoL)^7Pyt)4iP@b+)KD2NJI;EEn27o2)ZQXG z;9T42-y5Q||J-dEFj&a;gk|RO%4_59cLba|4C^)64R6LegzHgbJIv2kI1a z5u)q4KQH%Mg1EZERicW%LZ&QkSY_mPip+#_RcoYn48q&KXyyP>-aGm>7+h#_5ddcA z!`}(bNc(ClLPWL0a9M4HBH@by&tLGOF-KQ~Ag13FvcKsWJYCCEfr4rBg7OlAYKDpv z8h)WTnRiR5cc>$AFrri|4jdxt>XR2Yll5bq35s|IsZ6TQVvlc*-=9Lym}DTG9sLDc zgcexTo^lEMj7DoY+UOsEnnon3g}F#KtWza1N#Ey9U7V-Ib;e z*kH91QDJmroNy!O`y`8}5uulzObVn$GI zFmD9iIeIAyG8MnDm8(XYepP(Z`i7(_G+!|;edEhRXQxw%#c}#!_@* zJ2gdUfNVuJ0F4f7`nIPMt5)XLo=n{-NZ_7t5#i47NSh@gnLxg`C185U*7L?tnL%oo6^wQa+M`pT??VhVMaPHG%}e{vX-CKID|B!I&97Zir@ovvh6;GBJjZ}Wx)0Mm2X zxE#|Rb_Cv=7irw2@(S8~H#Z0U+ASl^))VY`@FYAH4sObG?xcpSYnYr)X z@!-u^WvKX~b_zQ9kJvO97q=9bhOGf&6*Ingr#&_4n=@K<`MYrAHA|aW?vc5-Ku;qo zmL#Vm7lHU$Zq%Y5thfWgr_!Y&0LdD8+-&^E_D<1|MK-kL*zZ4Cm5alnJT!-WZD;;K zF{r>-E=UEt?~cftun1L zkM3{sAFbc=9i6*OPb&bPV0yf_;g{_g_h6|XS=6j}vDcFN+*=c<9jydWJiX_S{s8nh z_t1A@G5C~F)qlB^@fyoZ8c9tr_hLa3TubonIlA+ztosPL;WLNCad0z~Nro%{Bo=2! znHCV!CNSLP>NE<&FuG2lLsUk6B*TOb?IiYxII!e7>EXoc&BOJqysKgeLiS-)r4fNT zS>3FZZoxXp42uinGx>9g!GtqTAplKhTjBsec&y`o>W~2^P5ff)DAEw4Cvi|lqm`bo zQ%tVX)dL5ks2XGHlC*xSmln6apL^lqXX; zR(LjzkLjy0Cl0(1@m<{P7EMz|x`iME{h&teJEED#Ty0ug)UC~>iSRv#uwNp|nk?s7+ zR7Cy#gB1@OQp^=h$MHq?#w;?|Wz5Zv8()9k8B{D0e&B|Ak?3k|KN~RmF|8ye3vaJQ z0xe$u%NBHl3Eq+_;nG?wF!i2hp{%#V!l`(|R|lbf(J~LA8RyY0J;tvsCr@$X7n)!H zTsukYy<_+lM^<#dlLR{!t^Q*3fX|aM3S1TKn^q`P0Q66C8mAHk_>f)k>Fh5Rky%N; zg)zM#?>4Rjk1AWn`{jV4!=#m1kII!A zBfZR&lD*{BsB@K7E{}8UpbEH|F)iFt`SBrO42~ z-}~>0c~x_&kgpJW=Nee&oM|)N^Rj=yM9&q{q5;ICe~P#PH{ifuDzet_%WMcm7^1?F zd#pTLktwtH;Q2fhC!y1#Z3~>BB;RLZUPPBOK8fm*zM%&WmifWxxMy(4%OB__Tft>Er`$tB>KV|@h8*DaVs?%R=H{S(3Q^|&69{)+s6KmMOV zVHfHP(tkyU<_a2y?SFIoYBoyOe{*}`DHQvE60))PXanm{XNzIj|4qp70BUxhrWR;^ z958frN?*%KK$XR$puhT*tJRFz;KJmHD;217rl;4mozHC`j0u)@`!l{PelIf4bUVk) zJP|yV)p=c?l=iSo!ghYl>O9Ig;+t-@_;@-3ZhgrZLWLNXqqKwDGLQoL&P%5^G3x3x z&*dl2*zM>q0`K8y&>-ZH0M98+uI-6gqk?E5)g)y(d)*sq3sfu3%TB1FpH}IAG+^_>_igdyR*H#3%psToy?0@r&2PBE09N^+lY=oCnV7Rk zk+0|Wc;2&`vC!I*p0nu26RxRV1-f)&ecCAKnjCwv6;FA_gtAjKD+Yt!F1_Cg&OBvG zEY}Kj?FVlM^CA!-Cdqe>eki%g0nDVE@BNz}zB!Bj($Pm6nhl?3#~WmZ7}bOAFhB8| zb;2ATh_8X`dw^1P1;o|p%(w1@J8_6E(XO3{&9lxm+bS@^it~qGdSwU&9$XByMi4+U zUzi=mDLN9zZLCHI-LA=vUKDY8%@z{}`$=EQqzz?b8IN+}w@Y7|BSIDSfpHZg?^-be_Wu)M1zzG>8xu{nAUBWsIQ+TIW;ixA_^LwDHp@`0@Cha$^~0;~L*xY*0;~_`zxD!DoX*@^M5Q zMuL9hR`aBI5)jB$momo`e!{i8LVC5!4sj3HreU?hWxxH?He?+tjl-sR7sFeonQnOI zy+*~>KR z>$5X;35I zm@B1XqJ7>^YUODab^d0eAS_OO5q;5kq~}gZGC#KgCxP6F%#Qvzx7qSn)S~qPR*rHQ zZAFg^LIAQK5p=jsxh<_9KT{n#|@ zNOH(e%$6Z->^`fvwTQEx=oi*#X?PXHVv_3C=YtqBnhxjQF(?7RgGd_+_CXN zJp|aoSw+VsWVg`hfRFenDvUBUN9kI}R2UsXR8{XBsuY;mwD8{JAh>igGAs1V`*!EQ zQ$J88_&UsF=>-q5vzK0%Dy8~Fd%H|N8jxhmb&Afr=sGZT1Y-BnJd91|FVL&xd%6)fv3qe)iD(xpsjfsV=$`EL`WgD4W@Z=obYCnR6ePjI*ZM50AS2X6o7`U*q};)ZJW5T@k+xBC0A z&I%3kcSRQ!ew(QGb2JJV@GC7U8qgnv8wn`j%LUzx?S{SfKf{d}PXAdXSY~OfeLn9A znjDA>^I5ec0^_CtZs}7(H4{lvCc*w)P;M+k?f7T(aq}{y@Mo3!b16h7hGzg;TQaag z7|7ibrZneUGN6Kz{b#^`|IE7V_VVY?`+|T~0_aMk&X3 z6W4ML)uM=Lzuw}sHx!NQKE%p;AJCAR(vsrxDdE<2wVf=eVTH?BrxE7ZVhrw3zdIjA@oW%(UpxJsX$t3bEORHWDOFQ9`0VwUr2r@L& zXmXhvV#lL1bTNrB5z)oH&w_d>g6p~v{hS#NC zZttx1STJTUr|qNl#OO8a)S_D>H@!ztWVzrplQxs^V4=`$8lS}$6K9cZKc0yEnW`jt z2W+lCvEB*hHe=bxUm%cT^$>ks_kj%IZgUp`Yv2sCafn;kjCApbwQZGYu#%~RO$LjTNb)=$gBM_`vSr5xbwN85KjXJX8%LNu^& z9Y-~>4shZED(G)t!c{8=)`qm$87`|#tHFFQP|$%;0{*cg6FhOSfdXvF=h)L3+};lr z@3+g3gkRj8vU5>Og^iBwr8;=f7a!;^YqMRrP&JhccGN`C}FvsR*1e;V2( z#y%xNF#|(;&t?H}(2dccqW29i6bbRM#zuRf{V%DmNE6`wm(Rf~nne((P8CJQT%&iI zoxtftui+V6gg=Y3c316=HDfh=2erkX(j%)M{_rwhD21Dd&#YYC46Gki@o|0=*LjW5 zy;`R-RMkCw z0PEQ?oiKvcU_k(1C`>k3e(pgQ?>UHFMh7!EPv!7vhE<;UtsBiNZwUUHn=M_Jw}K>UK1gNm(drz$u|MczFm#RZ3AxWaf*sTPN9 zcsh=wi3(%X8gq?1xkP+5{uPq|w719>hI~ZfUuE)~mmgspn?Fogj1-L`Eir=<*{Tlpzs~Y6VNz z`(ky5MvVpyy_7~e>5r0XBTeOkbO0H``{o2sq@HL{oRBr$mCw~v=azfNRp;|zrouEE z7*@1+0CRw27r;t$>E*2j9s2OKzetcE8mB19fPkqya47J!WT%`?tDlto3?8r!D-jyK zNDN-5{<8G+J4m3O1|UBu-S2Qh6mY(9{MKUiLK7H*C}%i>a3Vg64w~<<@fF_+G&O0* zc^tDb!xLl-zjg$SlLcH?{W}pyg?}bp`}g1B9e&C^@BmE}@S&rJA402f;rb_vGs8if zBaWhj{GerQGKRWHT%pls^;9^V5SLOAX#qQp@ C4UC9EF^o$zh-hj|3*#9oMl8KUT<2{B^)t zrePnYH$Z$PbF>wUn0+6n07`Z|OneV{JSg-41F1(c;LnX@;WW#bPc4}eku3xC4|t^C zoXOFA?(Ks#fcu&_*1EXxaJ1c zVHMC$(As!QHaz-T#&1W-#(;4qS}<1Tbd&f9$yRi;Mybdq)& z%`c%Hs0ivi&_kB5;G7#;8cFcjJA5u~bO9D9jG;VeOx(-H`E%QtlJ$Kt_DJxhUIgmU zmLc$6Lfw~rgUzn&pw5Kqb>(_dH$-7t&Re3OZGx6h6CMLWH7>5Cpsk$nd+-9U^4iJ4 zzmV`N1o^Fh^w~sN_2xh&?}rgjwSG+~-rc;E4`#EcUm6vy6-eseGecx-&s?iG3k2*s zt~U@^0LFciyVIKONVAd%1SpyAT+YO8rILsnQZ9t5a+YRCvU*yHFQE_>3|^wub_ZS}6z=*3@t#Y)lf zx=naWlbu>7uX#G`!GT#HjaYIf;$OKSJIhi9ryPN1I57fuXtA_UWV}y-Khb|v3~|l& ze+1-ye+#?V{lO5s*5s5rFQ*S+sDyH40((^&Z(bLf*Tmcj?!X=5=|-+q_yd3y_wvVL&EEwvk_JZ z7tMQ>5ae6N+b-wNAjg&@WVGHSK(%f5`68rHy@Ki8r>KKz(0>@b?)d|7CR`QCPk7yo zt-+)fOk#u(h8*jWZ?gn~pMSxf){=Q!@pn;}1^X0G@+Q5NNGw`aC~udr+f?aQLU3m; zB4{Qoq^rjw#1tK!-N@-oyBB1QPHiK}o0Rv+N43dc(8~V#bzaUQFPb4&XGEgwewX1p=pILIM%{ z2gALeC%ZaQy&Zj*kvOJC&^qxBmwnAU^Og?IS zwB^>(Iixv!^a?MZ_#SeAy*zEFH9^W(rLVdLJQH%WVhV=yhj}~wppFqy;&6d+gdc8g z>@<<>{DIegkPHD3uKR(NXR%|!wUhXD%m~lC4fs=7XEs)U$Z$1=xB;y74(IpDceFWs z*&O~9F>hwzLvF@HzXFuuoN$LwoKa2mO()a=JhV--Qnf*+0}Mmmqfeqy zg1t}f*pDE4AEUov8>tzrHRei{PQ>J-5!q=)sdHtEFw7VlY5bBGqqO={?p@VAlc%WD zj}gpR7pZhCo9i+ks2UqIxMg1xo2|O%zCk4#%nBc%EYf@!dnU!8%rb8-^K71pkL6L| z2u-qOps*zYv@Hc1<9nUS8)tc}k7O+iCZm4!rD$NgBp&(orohnwrc(r253601S$D~f zzTx|hJjIqsEtY6q$iaLEZ!AeDQOoRi25BrsU#P-aD5)})&0tQ)Qt5Y*z)CRuHI?#+ z`8(;)Hw7@Bdrl3amGzCQeCEU6+0QmdVaIe?SjvQf$0!z=q(t9AxT^Jy>u~Ym?e8v zH=%Un9xL-Z`n;3|%wp?U$t-(lnyyUgFSXI9BkV`my~ldKfhUzH!TuC*lTD)u2W*#A zZCBw+fR{ls_d=1N<(!QTSaTOGpJ3>U*Y=ezD{Vxo}ph7+WXV7UgNNnP#H)iV> zpdvjA8W0K0D>Wwz`^_eYHCG5#DOCI)mt1@?5Od(C!E5ld;kQ*h zn&Qo;Z_elWUg0B;v@-VTv(0W=r91ppKsv&JPd8>5*^feBK%YSMVYbzL;w%h|X^VZi z_J)D)Ptsq6A5aHuY%j|}G6t8Qj?ehKgOGeOF*i*&boji}2RSt}_UMm1$2p&G@crH} z1ofe}*F1oLL3d3b;Gh@=P}65R^y>8*45aZD5^?DBH5_8>8x)#w6PNOZn87eOKi0jz ztGk#I8yBr+ZuVgSHK-Phv)mg6KbB1Iz9YJDl}BQ2gL!3#hL++$YJ-a*N5DxnEjceO zPl7i#-PwX6GdX$#H2D`U&(IJ&cX}a~H-tz17z{wmUyxi*ih8hM9^D&Fj(N?B8qWbc zFIR>zAAb=>EaBBTA-|59D!QzoROXmk3O^n`I2*rkd4kH6C!Xji%{FZ~zci}Astza9 zgN5>F|F{c;z<9pS+zl(lHDgaYP1)MGo6YPOkt&2oAAM&s8P>Q;86EZ#YOo4}0;QQs zPZQvpUe7hkTuxLvwW*xJ+sU29s!%0)3_5J{Kud(YvZP2>30f+GxKesgOPwddS~OWz zO<{1}w~mh@7Ur58C&cN;yLyzi*)}-5Xfd;;hE!6+jU`~}T;Zy@40TV>1go_4Z7dya^nM*c=6@Vwx%(cbsla>}e$_lEEZs$nd z^9pS#SV_s+CQLcrf8zmnpqb0aA~tPiH8#deks9y6sQRcXpLE!|co08Kcp1$r^!iw! z8Y&e~kj1L(9NPz9%4psL78MkR1qr@h?=PjRcHks+!8GKO(7?`-5ML}SwmkCQR|F`q zqFVK^kXIVUS}XFz38iq+Vr8QVU0oB=5y|Ifb(I|p`Sw}=#;7OxJV^IC5$S=lComCC zwH-=AFr<%+?-46j9AT0hPXiRkUeBg&j~fsjPRMfqa}c|a{hQ*ClBHawww_=2xhqle?9}r= z({tiC#Gc=9x{+^`&hK_ak-G=4zEN?}7=w0O8kQpCnDhCp&u5%98Q}ew5^^FqZ{(O5=BX@uZw(~(^yb1Trbbsp3_`uAw$D7S=yfzK+~}3f$klWJR$zGap`ps|l)T$j0b=M8xovATk_dyG^ z_ResZuWcLlJ45^B*zmnd-Q14zyRB`}bQ=CPV+rIX2Aj<`gJE=grc*3y6Sqk_d$N;6 zgZOFa(#pYcd!-ZkyYX@EbiIV-14c&!=y~*nu;pEr==}FY{5HhtTyBrEvD*^MiGE-x z05OS^BpL!7wxkV`8Ko5fTkCJOT*Ei#fv^Z14S^yg{RK`qI%(<-M{^SC0@g$lVTbna z7GVpu+t4=l5Uzg?R)IZCK_q-@_VtfU=$Ow6qh=)ju}fg|9P?6A!tDXDB4foQxX`wq zF7g3#*_rcYqd(BQ<;8e*M{~4aj~r+q9soK;LdJl{|P(@QD z(r0N3pBSukq*JL%H;3QNIcUQc#cB$)9x`FO)vOm)#i1AtOIPfZDoLI-MJXvKw!fa!?%RkTt^ZcvZKC)xQ1sJ)xct;W7nkLQ`I$8zv6~33_LswI~30I z)I*h98Rk0>tQ&7%T1%6#^r(CsGK;7USN6>KQ@0}3l%EFJN1iK!-e5!a$)L(2=D;t! zqkv{idPsLQMo|z70>>{tK08pu;)>3IkmgNtLnX+&?=-zfi`zxzlBMDkUW%ECGUD4? zI)(p*)wr8Nke4$Ye(A}txx0Z9kZTA*?hI0y7uLxx`kA` zyZr}XX~{+n`nns|>w)x4zYS!?p(Yx_Hf!(&v}uAwxhJR|8uNOFP8)WGCsGc=HeHN% zOQ_d*PDq@)3)Ys5GCeG9()F>=^+d-q)CwZumGH0a^8a9+{~Kxjg@U$`i~^=VOK@BW zU%oJZA}A{d#^&%622AjOkkb_$u|MSBaLGbk9i!(HE>&Rz{Yx0gfIlHQp@2^V*hl~X z>N4_272~hpphLYJhN$2J!pZpi%5xa9(7%yuYHMqwqNTJg=&KI~&-CiGpCzvf>|zL{ zVhiL%(wnRj2nrx@Oi|K)+}V3=9c|4iShy|6SLhj0KMUt?Jeqgi`raN+ru&`&s=E*k z1^=84gZPxeBJIV|bl}GRJ~I*UX@do1_gcKO{pQAqeI?$7$$I7M_UjF0{5=AEI*=mp zi-_)?P%tpMVT-iTfi&TnFuolqa}xPt(4 zUqoyb?zzDf3hyhN8jm_0?@U50#6@ekmeH`#YC9HA{ELesm9;L{@`hQI*(~h}RnbXv z7VS37|Gn2w49TX>iXqg5HNI%n+T+1TS$=G1#57QMU;?%Hsc}5(+!7mIaj8%Nd8Wuu zEJ1L6bqO)M*L8l$K=>`@Xo}G7)g{MnqCb}uzbTl; z>gIZTpPSp<4bNz$!2kq`kE9*4D1Q(^gg2k}A>e+38e%EL+tZJIfAS1aX=pmI+fHmP zU8|*CXX1W$JwhCtRN0JzTNO+eOcV#xn|T(}b5Rjo*X$##oS@F*30|_`jOs}#D^-)O z_w74Z(^n-mevbGfrbejgCbZO+_T97oWXMz@Fk~Xu=#p3}m%1!AC^R)Sy5`Qowhffq zZzx_63vt;XR#yJHGEU_H3i$Ckv_bbNlbJ30XYOj%p&t>s+xXWLvwzScM~TL_Vn(|(ID#RZi&yI41M_R{9))9vO-n71m7 zF(y?eH@kMPMsm0suEEn8$wXl~jnmXm6LVSDA*%AXM%0{JxX>#DB+kV~m+xFQV7Tdq zPe;wlV}Zpp(;A)`Zh(HILyxk!Hlbw1l;wp zF^)28l#u+jrwXyd+|8TyQAD%tlr7Sb@Os;ripnJ7T+zD zm5NP+M4MzM#oJb|=J&T8jWT{ zcyVJKCJ0-JY(K8t^6X!H~mvks={Kqaw%;6j|A4!pb3Tqr`($m`4lZ4Aov!E@|>y2wAlkk~IJu0Wxs`SvisML zq!3_E+b33C}Fj7Zs_o!GNfGmw@H0=uCVV-k{<4}ExYEh}`F5P9L?);=s zt#j^T+bjk2#UDP)tL7O2!%jJ_x2xtA=}lV)r@e7g_Dm?ioRJJ?8eH^m*GOEuqQid9a~4i3@1P{ZOj&qC@J+!pv^NuBPAI8viW^HNf{DQKCk$AKLQAp9( zSz@w&IZ#5fkR~E?8I3YhS0$;V!h2?hPcxCmV|;`$qRCk_qwW9QZS!mVDIj{`Q;6vP ztO^}@4nMduOxv&8NaM(JtB4++kxN}PMD^blI~eb z+KZu_Jn;eZT2~C+PvWDyy5!~W!qk7T&VR!tPd;_Rqt9v-`zK`L{s%4rBOq`PfkM#S zh`^phY!u*hH7abgyddfa$UhKjU6Jh@D&k+or?2+aHkh9mEKK&Jq;nF#l<>yb{xd7sB`NHB$w zXTsvh`t9qZ;91Yk;R6N+id7Gw_6N&4UgJXxXmZHZsIzbLMdTk5$Nj@-kz5rFrXDk0 zUU)agyFcFVk^HBxNkU9oijC<)zFtZ=q3zL77(<&V!xMxZ8bzfse!DFJ0jPLX?2cox z@gWCU89gRhc47284EXVfVDAssd8f{i%c-Z{>2b#LyFT9mo0w`SDBWxS#l*?#HFAy}MHn|e=Y zLe1OMzJne@3L#MLUBp|)2$;TBTHE2?SfuPw9%bVKq2vtUUuLRDs(8bjp1u{SwoBJ- z=5Rxq6Y!2Ji2^0O_DJn8kHuCEJq!Nh-HeTr+|1OEza#7!r32AVGnZ@>GNRVcvVI1C z^M@RW;bElM3`=3rlIX)(=9FcyPB2(n8KR+#gqRN4oTtIfL@hHkgxWJLm743)iV&kGJcLcC!h$0<)cR!{do82}N z>W9!>s0njmr!&&(H`ASU#F0YZvU(zzW?%1-Ll8V5sg24%&zM0AMM9`aH$kF7w?3D_^P&>}MvfxS# zv-cu0D7SklT?7z_$=I0~KDXe~+Q0PhQ(}nUMn-V+KQ?H6b~S0p&wNEA#QofeyW}`W8p#b zjuVGo0+75_Aa4=5z|P|hp~&MEskg8yY6K0WYXyH^lrOt+E9K(mev9s~ii+0i=zG_ZwXOHpu3ENq3v?8w-MWh}pf=PY$ z4puHuX`&*R`^Q%Ktg5pc!hMUpR(k;3efmHujT5AX6=ni+88kk=k`*cHa>CVMT}n|G zz3-DoNn*Q4;xS4nVAr7r00Gkr@IZi~L0*O+nF8z(y_~ezLxruC4cN5=>u0w>5PQwP zED3ibms-@RS;kdyZ$;}1VmtLdV$dhfS% zlbbCm=W4;XrX%`ciC%^pBp~#E$a=@%%DS!#G~FHBPC91Cwr$%^I_j7wwsm6LwylnB z+jefA=e@Vy@B41m`LV0cs@gx+*ki3Z=A2_jBOpppQ?u8*tuRRNO8B`U@`orR1(nho zdszgH_zSHb{{#a>^r`I5q8bc5aTYYo&ipJAT9W+?_+9v;Whp;;*blYXr8H_Udl-wb zTKEnKluuNXKWPLG&iY72YDpPw%5XAW|9J_)@D!FI!0#rvwc9&9N7DSp{x=v#p(E^p zHcBd#BIXl9Jk&0qSYd{7=sVsp>b*0`fB64jCPw?0ivRBc&=h-smHVHFSfNVxCpsDk zNF5V!d-5xR@I~inz@J%Q*Z}T0L~(KP-USeYIA%n2nl$4?e)UvP3P*|H6bR{*@%*}K zbDh!!%XQ61<+c*EhGJAHA4RRw#(B$%HkG!PdK;(Dw@$e(s?E=}$AiDdj7%}3(O+{L zUoUtL0ZAScEo{-BuOASh#gU)I4k8SJZmG<(1tkfEqyDBrccnb2+kjM}*5y8?7S%&e zta6}fQm~|JEAMwJ!AxTYLk7coEpu8s#wzsPp`miO9ZcGvMCT(N4n?_Gp9Xor32))}T*iPQyGd4$+;?T(? zLpf-%AFLWNVs&g+W1Yi38!|8rD%Y2IRCknlRcBb}>e0_&v+iE^ReX5yjzc#%Yfj3T zu3=mac$Fnf0^X4#o_4kJVz>2S?+y;MP2_rVj~{Y-wyLot`aO`Zg;Aw2?sRs!wyb3O z{c}{uugG&1m~d_Jt}dB=gvMZ19~gLD^V``jnbCZu@4{6*77cY&=Gs{e26s%eH*ur- z|32^&SPl|6HHd&#Ye5iwyqm6l_^|{suyi|43->(e0jN`dpND{zt0eQXSiXGc>5*TH za1Qor`{ons!k~g10lsTG7;Krc7QMOsA*>E(!_w!g?@Bj2uJ1~MbO|eBXsX}M+_ye$ zPSQ&HE8Z7M6&By;>O#K%62^d_bWgy{i5@qi%EZ!k+#*;7gJ8H)WHm)QBNoIw=^m`_ z;GqA?2EfEgfk`NxRlR-@4!*yR?>O)XmNUALJ~#BLhYU*}{nT$9!@*`uW{m)v#fbR$ z{Aha0S@eNV@q-DbP1)3VL_-+v;r{^R{hBKny*&Ork zv5D>Uh`XX0QYsnk=7t6!Z*S$Ox`r;B48Bco0mQkH1J6Wy)Z47D{X_|Mn15gL7Y6+{ zCc9ALK~$K?r{JfG_9rX6Z9@eA9$TqDz5#TF8{{PJBkpvSUv_pzPiaHwHus7h4nbZ( z6O1mQ-Q-T@#9U=G45D#1om<1{H;ph{B_aYcNz+Ir6L~)L0Dh z15oY*{XNzjLcspZjKJmRzEuNxyIEA4zl(F*OR7hr*w0-oeP=eEts8$rn}C=Q(ojGTDE+>9%&@;m!f)VFfWf_NpB zcoN*M1`|VtmNS@ro6unEIBrvY92FJ3QS|vcx52IG=ti z^Icz9o4u#!zHMqNhKN}7j#*`VRn#TEp!;BOW_rYqRNcVZUjBIZdBB;Q+f2zW9)8yMjcQkhn1{nkh*~x7764X0mGSx1 z&X*Icbj?UD2OSWmuZAShXk>?3-uh;S)neJDfZ`Wgjb}VMHBfBd8A#G+(t8D06|wyc zPm19Tiv>eJfamBD{Q7O;rr7z2-W=YFgsXUlq2beA9i#i>6P`SvY?&-^O*sgK<$ zvx$W52<}eQe}hc;qx&br5qQosviq9?zDa6Qf&8TCV$jhz5U%-8Jg^y#$;7tA}dZ_ltiR#Ogl9of8NO~ z!;_~;D`o#n^x-UYZ@t`CSl-?&h9err2Mv(zNvKtiUyXchPsQ!tx~X(iKsZi;B84EV zJCNv;Hxd8n>kiJJY)?{-quCJ8ya^WISw7}>Dle%MM&Ma{O5j-s3jUXeOyGRodaO?Z z-osmUjyOtnN~KHR2LPH)+u8&yXr8-P%x!PTTlf?kMJgY3kC zR6mzNrQS6Rco&7UC%$w3;q&5BUdx!QKYqaw><8RCdI&jo0S^hwbSi-c@*2vomY+bdLF!Ez&+ePMQ3` zKn{*Dox`neW&rU}0?EiVT8sA69ydFaiV4Ed?iSodxG21)S#+pwT-Sv`>7)aIXuw1= zoa&za+bXAlM7wB9@dSSw@%7(OL|wg%bCM)%tPnB(?T!&?rjKHF>W-%|XNsG%LT0`hYBM8kHy!RGe*L818ES#KqFggTIX z{(x{cDqCoAm(iz2mzhHh+_g6P(k7Q_~}Ylda9=(Wxxd!shx0 zm_a#5HDKw0#(!PtUFWpPT4s}XH0?#^420>7jcZ@uoFg8e#eNC7OvUt?n3A}RiEXLz zN}MG3LM8G-Mhn6P-s$)0@$toEGtrn=7d;QARG0LUB}&=fnz>#M;!Zi(Y+B)=&BeKc zgH+};hm!|fX)cGVVXwiTsr7*5yhybozcx4PQ5i^4dTG90=4l%-P6m6Gd_JZ4H!v4Mlhn&`iR9Hk zVSSWWOxO+rF4V9~RFS*+Qr@&MgYNs2=s#YOule`%)m!9lqEsFfCFs+W=0}+>lBW8^SPo$L1Js%o6KlUf<-2BxKa zjNG|It~WdW79JZogY8Yx&vj)NZ&><7?Z4Z`S%P~}2@8=J%?oh@L&W2iQr&kzgKdLV zKAoPn>4Uvn(ud|SWw}n=I^5%M`Q~mbPL@f0Ox(hRH)h1QnZ1P=vZn&n~(g4uI zfo640$0TTOY1CnIv4_7B(+T>t0DgsY(KF1px0s2U*^F*USp?_0<8;|+QximZ=VX6o zO-r+BnEe|SY2cVitJX}M+}j{6e@i~ zf@&DK0thkLiwEj0RSZRe;mcPM%rg_Jlp-``pPpHq{?Ze1i#Ms z+?{)0E3k=Q8q7ll>VqLQwpKZ?w*R!}HFD1vC>Lm(U#z;M%$GyK|e*E3km zDKe8Mea9-@K^SfaIL^N)lv_2_^(!j>!^R`clEUd;WYma?b&QuYNdcPWq`YoQOjWy= zo+5GFVc`>o8|gI>TRgd-jz&_-BgNu67kYD2U~gu5{zLiuK$X-FRMR#e02YQoRR%WQ=nWRBKdjq?9317a|f=4#o z`dr=o(3gA~U}1g;`|FB;^;(Gkg`Qx0{hX(LWt$U6tOwqJ3{pW z#8-vo!YE2qwcApdFHqDwi2i|DaE+;Y`&Lcm~`Ua0!O~P(NgkW^XwHdgehQ88Ft#efcYwzrNi;B&elX-q(6Fv&`$ylvIu!a zeaQGN0OdNBE6#GeU5!O;K&q-031he_jYT_ABz1xQ*y|^(#K-r^PipRKTj*oP@St9? zQZq^|cRxv`4l&dj!#%d1Ln0?NEbkT zJyM^Rwu_=j9ZwOQCA%P}yB6EnO%i%>u876X3|gh+;<<7~5S7l9*k@FsBv7Q(kf0zX zv0Fx4mQ$`kvod>`JM!SxQFB;jZy&235K&eVY<0w#2?aN=7;|bKa7sDqfNMS9d$6fg z`+h6DGa83juCVwiB{r?a%_CgmhY}>@2Opsc#P*pCFoy*-==AuJ8x7 z+^h=rAJ16H?viy2H!qy4U!?xPX@H}y?0vf*_7k$^XhujbPkq=TwWmY%V0NH$0eyyP zSvqN|Rz?!i=*T^t<(V&d&}K;LqR81JQe`M<^&$d2Nl*jI2HILy`u8;$dNdu)$1N zdjl%%;(IDJ?&=7TBu;U!T8|1qU7(j!M4ESn>ZoU@ZVlq0p$>|_3qz#>yg^LX&KS%d zHjv+=r>Mn(T*t6(MPjdE2LvSqbt#zC2~ON!8*3bu8$e*337a;RUO?2vX0= z(O5Y&&vZthnd;=HS^1g>o{=q8&tSPIJL`V#y^^1iGu8k>k?d+3IS}!v+@xZr7UK%l7dp>|2Yvtk$>WILZh54f?l@(Ooy*T8yU0U! z;ES!Y8VbUY3D$kX!|ty`F^sU<-{B4819EcO+8cqIpPyvE@blhN65mNR(3^<=cBGqJ z_rhps3xDPU0A{`whkBk9ni0?3*W$IO!j=0Y#3L}U;VHuNEc4oG zXr&j4cyt67x&Q>rZ?pS5V^0Pj#s&;}6Vca;(BYr|Rw?~gQ2n2(YyCf+qr-|Q$#8IW zUyqgZr5Vs)?WH4~n` z67p}`XvDcP8jWHIO_j?;CI=H&=6%&BpZ9n3JQV?~CTYxO9p6wcvWuO}P_Q+&YV$4M zZ)rK{5yro(uw94c&*67Sc64jY40%yu7pw||=U`lM!;=jf=!DclNuKxIiDjjH0UkTk z)W2VC1L2RWq)6l8f{G^QHX?bzJ1|}nBr50LM;n_)3-GMM z^{v$BoUs;YhAE2V#Rsx9^)cV{Q>TB1+aW82nh$vXuL>-~Hqv7t23xuZC zDmL8k^AGYZ3^d_pz3T6U)~4E-1C%h#)%n% zQ6~;Lg-hY#megh07o&2$7>AQ zpk<47N!chR(@G+dQ=ETUhbIv0THe6QpW#*x)0u>fQrM@o$7eh{^|zr9d_0p%7+R}l zG;{8`n|w8IG0!jyE~mP7dS8_Um{6uyiY={LidCkDa!IwvQWov~H#lo-k$?2h#k@1q z9&;8X=mEaGUmzg=g#_9^m;n9%0VmlQCH)8dKfo!oabR%yTJ#PMGzk96z9PbN;scZo z9IPDv^Rt&eEsp#5u0Y?d*36w%&bvG(1S^ux5cwbv98;3R34WQmhuDaJs>jWm zYUhqL_TY)wjq;bTCwyc^y&Y*q6uKmf&*=ef_-`r-WF@@+=a=7`G-EnNu(@M&7ujSgMDD=S1`eMB@ zFHmrBK8Ro=8w1rUiGyA$xSxV zO{Y~FC2Os<$>mRXx*HCFLvu(CE}1qUhuy}vfF{ArnZA7YCFilUF0$g z`O8xKX6&28$9;)i@szs~1k~pn+kQMQdyH43T zcUiq?=E-E|ETG5%X$4i{iJipn5eYr;artKIi(HrlJY(*aKLlO?h^KvcB$6z0Q_Yq%d?7f=k(P{*EWAp` zFmYKQ0dm*5u2=yhf$+^E1|ixvw(QJSV~AE=x_B zfCh^W$+R;EXE8|a#U8VKZXae!&t&|gptSU$TCb0@eAGVGbXQ*83JAOLhrO2|HnMc? zA=)o|`m@oexYydh_fRUpg7tot zz);_Lu5Z#2zz=wG4o{ zq5?>@+Xzk8qi-%Cki;poD6NBN&q(7aRSq?qiDJ4!&r}EI0pL)hpUu;1Og9*yDoo0; z5yB5NC|J;b>x!pPHxkEVhGgZ>w8{-qjwFy`)wibHKN@!GXN)KT5O~@ z&zk~<<`p7or!rvSSb8KXF9Kd} zN;E%KA)+Ki3ScRGv$Ldl4Qd3tE?POsq5d<__+Zf?X`TG0v8HgDNTTU)dCIR}RalLF zw#uI2mzaJY=D!nG7Zu!om8;|wsCY*I#1!RqB*DbqJcn&9`2DJ0v2SuzCKK3Gb=*39*!Dsj7 zGP0)Rl|wb8W*jAC`{*a&RxeG1BV)5~T_fr;+**BB!BsDYl8W?TP76fCUI$dnAH+9` z*u&BW9RY-fgcPA>H?)Wd5vE#QJ{i)zsz9%uU)Z_8gIForuFnVa!2C#Ss)OpD}8XncM0+L=pea-b=Sd(G{{AunOgJB_$g& zaORZ6NbN7b)mo*$wA-&5WfWK)k+&RbgwTci5w^)1!}yb+%m3U zf#dkeaYRjcN6KrqWrF7=brVkL&-M6b8SVM(Tp+Y;X3X>A8HeO^-8o1Ic z?F<&LBhW1bzqZW=*?cvAT;M$BLieQ%5Nd}^8UV3@iZ?-jpf>>Utw z{*Ws?&zJBOMcj5h|$|gsV3v>Pwt(mV_byy&|$^@o5k2ZHBi@QtZdt8)Bq6 z3brfA-vFKF{eF506r!Sl_{U)O-Pq5!IQFp))S5T9W4;`k=g)s#FaOK;|Mw)QS<=fi zX+pC`sr~`_&&kk4mrCyO9sZvy^8D3gVAIzj!Tx1>0ffT<#&m#z13v@Uu$!FXnU}!+ zUGv(gK^OW}wblp%j3xV8Bkzg=2Lz-rLjWnAP)b1my=kxO5!!4{le=;Q=8V3(%sQGH}xhY_}{kTZ#9YopW# zP3C|-cwi+{vPX29wQur|$s4mJ) z@eSoNn2>>G%+YbyB)!6B0Sxt}5|7s1xXb0+AS=ycWsMl;qUX5|In%qpdo~s35Eb5e z;EFGV8CnE@U>0Wky@7$F4`~6_GeQnE!a%u))Mnv2(htt2Q$(eQtdWFU=?binbSiee zQyh&lxrgRk9Y(f9;HusrBa^yLfl@`&52Zt3?)ivx*E`P9>MM`o%aH&+c2QHFi{D>W zfc)iD8dC)){HiUxvT;##oQ;su4~p-VkF(hAY4u3}-8dH}wtIsY(6uK-0MR^X8LC$d(E{5o zlTY9da^|P*lL~_JSF*KgnL-hiDQeO_Qk9VgQF@E1J56SXvp`&7rabF}^%;^`^mYdc zT$TY~r29LQ9tJ;Y=H{kGF@f-q7}a8LlIWR_;YtgF;DkmN!jBk_NGU`AplDLD`iddUym?|!+l9eZVGZdw%;v|8Dlu^zHaY0j({wvH(cBP8{1m~ zOakK`B|ewmT7_$@IW3qo-3lQdX@r99Bket4j@HGB?s1B@UyscDs#T6esK;;{6Ivqj z_SuR=@TntwX%Vw~7NkZxSUt#mjbxngqwu;$ir3lC=p+2giCB@)k!cNbi1*`5JmqUQMtAK~Z)AAwP8#4l2Z| zFRm&h?3P9ILyJ*bhKuytEQNrK7cui)VUX+GCPX1uDlKJ#hv#uTrG<&l+nevl6{?g; zbMv{c(Lx{Wh27fimVT5`=I*y4A@Ie!-1^4et8$b9-=slL_}m1wGN`%ukR9_-{0MJcbPYp;Tt>pkBfc#=2Z2j7YES$yJ1S2izpvEoQ&wPjG4 z4u5t4FXV<3f{D3#8C~{c%EEwZ6?}#45aHOneZ4uidXHeowV4-q5H#|9E=#{`>{I9H z%m(xj5!&g`9attH1HBwZfd4k9dV%4o*rRE{3WcYj?UZ?jVYutx1-0q}wg}G=4x_oY z{%@=Rdf9i!9zie^e%@Nkhz$wlS4}H7*a#N6ZdnD+V23a(40qki8j-cdmO{Pi*HlvJ z{c(%6+$tdq>}Cn;q>p9_dk2ng5z zTPV%$O^DTW`GFMxj`R;kq>JQJYrZg2@O7}L0=I|BS%CDOgmi%Qe$3Mf47wNyl%D(V zo$A7o=+HR{nme~6?tWs6IvgIVcX|!_Xr5OPPkCWX67oGT>Mzw?UlF&b!}9WK*CHE8 z5<{98wvk(eQLYax8BBi19>){X#Rt-alsHtA#R;2=XHE0noM3GyW4 z1=ntO z|J=m?I&}ZDy)1bLC9^>MAQT=TO)nNCK&B8Xq$H3W{i!$z1_b<_yjjI z1uHe53^YK(X-C^(u?hYPQirY?pISqq%2a2XfElS?o0Q2daL9`PO;VpkB~kPOxJSJS z$k`TkLwdx-e}_v#$y2`ZlxL@AY`W;6F)P5FRQG(E)jqty>T63x>Hlo%Y+HrBpMxz) zapqgRG2y@$e^rG=tr&llA)9NgZh15dciKewu}OlRMx}rnx7M(Wfl;O4!J=guN8K4^ zK9;~5kJBPo=jyoSuTw)_yy27qpj6s(KMZsvW`ebk-$gf!f|QAxP2Sy7-)fFTm-z&} zXK?f^6^81FZ^u3x9iNtm2W}*2$_3ZD|8@>iWuRK0@|;O?S46gdiH7@dXo?d4olsf- z(JVqVW~|U()`6dNgvr*8NOa`*#t$;XjK_KM%?p?2#UrRZ;SKxP`ElbLK>hLKo8N`n zT&Hq3tAw2Y77wSfUa{XQd$aQ8xu!%Uz*r(k(i zBRs=07#1k}#dP~gXPjDn)#&v^7%Yq3z5g}*`*)+izT@hz@A!X1Ko-3z1|RmTQvw?V zGF6tigHn(QEVZ@5;*Onp=vrelNr( zgr4i;n!_c}A>SC!;Y9eS#|}FZ4aPZiBxrFgK1@uu96ifu79fxNFX)lIyC2_B@}mSF z4_5ZBqaV54^`QkHPv36cn_CoL*3E21`{6T)SE{c5gxS+=ZWjE@aox8UTrY2EFW*kg zY`SUS^DP>mSK?3=<((6s7cqa`ahmr_lHex%6!!}}pmQwi671;p?nj|_@l>?y%`V7$ zfOqNCWFddgB;frnobiS9or=G!a1+#=>E}DmdsO0QCLchb9{@X*bu`)jNhkfDSklFC z?^5{DGsl5XcVDFBL}`@RO9W(&ujiV-bTZ<0xsn zuya^p88;<9K57L7r^xCj#^24wgN3&-e9v^L%g29kKJu-cyD zucBSxts-3^V^ga>l1a&j7`5m*4TQ$xc$aN1s1w2V4*)(whN zduF2=(Q1YZJEIn;8-rX#Q+S-Lnv0ll&sXs;LPc7;Aw?g}U+Ybke%vHm^??w4)g-=e zd%4e81LPo~1F?r>q7W=c@@-g);s<>Xocrch9m#EQC>IT-yl^T4Xp4y~N$w}^rfk+P z=AF~rRrRYF@>hS!PY(%W1Y>U_4d7`hbR^<6Zto~dP}COKw`DU%ujg4U?ZJqU%yK1~ z$eQ$1jxn>Xv<{k$-MuZNa;)ZHRc3EbrKC8V0B9-mHA-VQEE=q>o4Y#&XNZtkS-!f} zQ(JWE;2y#caYaT=UNJID@}vsHa6u2H6^Ovk=&8>K@7Qs7WH<@)KxHLiVW^^8G(ByD zldxelpZK-qT7nWCQMdcJ5Bt#}`F|6|maFXlF%Vk8ns9%*aX2$rU~uw-2ca+lY2Oi9 z0kCMHK#fEVzfml_GSKYENpzLKbx#>;xv-B6J~l<*4FhKlNN=1C7UG(g3ey-F3gB6y zS9fX%OXeg}L~-%&mEjtenaK+<0IzW2svV8i=li1*Elcd4o1ub-^YkLf)vyV_++lff4hy+5G)@Q*N z-*L7Yl!rVH=qMGW9f2p>574c8@e}$}@g-~Wt8~4u|1Hf#*RpxW7 zZRIObj)}6es*eN}qw)t2cnUbpPCWxQ@M5ej&m< zJNOa_mYSnFDA$A#E@ZK~ma6|sMXeD=lM<=RzLm?eqY{BsG1qJh`pS-Jm-(VJZ?~SJ zP3MN07`716Di4Sl1`?^30(K1TLnR;WBR0mKRHxs=)GNYR#+M}(RLV_C{Y#`$PK+(_ zlysyk%u3Tr#@JbbO3_rBDPar^zG@zBPQ=m$)&WvV98sNCkN7-M{mHZ%b^`g-7!V@$ zP|#WN(DVXktQ3Bw&>Qq?$lOnjR;JjG2HPq3bZgs$Uc@6VJF7IR1f2^&mGM)vhE(W^jb*e0>=~nmE@{iW#w_<)ofn zX<&b$4U@$ZM3*2UkAFj5m#5G*dzE7Cp8O$H7?RCWRMElk95yZO&>U>#1CJO=BatWM z0#Omb^pH2i4_LmD8kIc62XQu}&%j>oVIgSy5eTX_X=k`R2;;+WSrSy3I(ZRUw8AnZ zLgRavjU(qoqS_zf3xD3N>cu`fe$!Dc#nyD%NK9=uw%mv$AT?Q8KEr64|5q3_8{4KU zyR^k?w5AV-tD|NNr>z4?Mk$KH_U!)29Ef>p0ezlL4G^D_9Y0WDxbCVoB5^h?KB7S5 z^DD{Lc}g-QDb5g7lXdP^+u{d8UNOOsgYfOp%jWR5YDQt74;~euOWR3&>)|4l zH}hp*lKY`df`Cs1b}F1BlBeA2nt=ND$mUyFbz@>aIvQ6RC!~xHtDIv$ftNK#*_NZu)_-RM49%!8E{pX_{ z&3yx$*;WjGJp*{qZM?KoM|F|Dm0r&+`0&BNPxHtyFNp-1@EvDhhTzN-a{0zCv&zz;+wxd< zPJ@s#NS&mlji+i}GK5!r ztCUxeB~O`4(n$*g>{RUd&7=4Y^X~@M*E}x=S6eSXWR~AtNT13W+Aycp z1BG>dC@t=$>IS*i=ki%+7bw@q()x9&DseTZU9G1xc1l+FpjLmQ zYDnyq*dST8VdY4J^LukM;h=+_T-znPI3Hw0n;k6w%>ukX zaQU28v)mad4kUzGz?~;Y=m$+|pg*dQTbK)R0>T~{s$!o6`@D>w?0p;PG{$Nld^*D4 zn)3~?^lBbr8q$N%$}QFVh|%@R)YY!lds^;p6j|G1b*1PkfA29y&q60e_}zB8*_fJ` z8d-Sr0(BJ-D@QH=E~`$(d(pCrjIvslEL*Ehuu730OH&-((T$^qNBw4|nEno?F+@HWV*Kv5`e8~I+> zTysbN*Q2lwsx+_=of5ay1zOb#TeEV(5&F-Dal-nLzVfH0!G_1`#On5&+PBDkz(B_2j!SLRn_{F}9p#MC<^q;ebX>!sQ)fSNu!$QMj zh-w2<8gSsH{0M}HA?7T_RIt1MtcWRggEXc5)J8XNHBhLMN<*iQ8^;gkWN_@A6O{WM zGK>7#hi7lq8R{Iv?G|fum#!LPjsoEGEwo6}Dp^KPp@kVE;%xgf;RxA7JX`S#dT}j5 zDyk`IBfAR4R>}588A9nvhxUXV>u%OjhT3GM(($)JK~iX3L8?+lJU9QyTBNG5(Y#Ga z+$TJ@X(=6j3SD|!dUm^!Eib;NTHl-Qf@H8&epSe)YJG%5FB)b6g5poFJ5GQ&vV!=j z(Zkg8;N6WZ5zqzR)j9H`YKxfSDY^b95)lHu^_f6uptsVpNv7An!A_XLB>TC^ z!aIz>!M6y|9!bRVb6l8CfaMhnU5G#4_aT}uD`k=3MpyX!r}*I)BX%>`Tj?MKh#j?~ zJnZZ&SU7}ff81|6If3PhkqyLs^qMLjN45#*Lm3fZ_eI6LTAT=yBA)=29oQCMa^~lv zEx8H8;O;n0b-pJfo4gV2#=36f!wwVVFls`m{wVd`yKI8 z^BAfX$@#&bZK|a*J1|C=YeQeN*ZN=z)7KM$L?4Q0d1k@B9!XDiI@)VcM+ZVx3O$_e z7IOahM(xT1Jw4fv0z@W;>Cia3?K4tRSO{AYjYnt~yPQKczwuKnlU>Iz6$4D&s&fgS z91HHX-6Jo&?m9&=s>HA3%r_8&C=;OEhNQe7|E-bxFSY$&v85!qp>Pc-y+Nx82>OmQ ziG!F;9fCSV0$OY+ER;Iun?1kLR?^_^I+4Jh_@pkL#^1j6<0=R7uzn^6k#2!B%*&y-w=PCK6Z?DX=3O6?3^lr4EG1RsApnYp@Kf{+f0 zqlOw;KAZs8kmPe%gdn^wyEV+`NoLS#U;dC zs7t+0GpN5_GgzM6+5tLMqhu^;z-2Lj;<}k&dV^@$alhiG+aeONu1KHdLH!<(uA}5% z;U5DIwEXHqNe`_>U<*5$a{bE^4_UN}HVX85Wq!rZk*pF~be_)imLC2)^6cOJ{gT-v z{#TLPKT7NUIE}f%mo*z0%GbYNT04-nfd~o6KSxXWkDv|?Y=dJ*`bToJ!%@Tf*r+T| z+`CuciXx6#LwRgY_Rtg#`i8#+%V>!WC+nY{*vtC<~W zI661-W^tmBM1xAls!mR3c{2^Uh|`gnxnBM#2PdDt7{GLx%s*|8wi}_ z{jQIad6WCC>X9%6z!tI{$!m^@TCYyIvt`J&C-l9~uq=E##4nH6P@D?$ItMlgfii#K zicElYAm+MPP<1K_YFnX(Q#TC7wQi|Xq`F;Y%7$uS)vnI3>9rdA5;4IAdbAC6v2Eai z$M9a4p(sT?JCsoE5i8*BSm znGx$4b6mX;y)M^q(JVz9=lBoiiiBsG&C!B$&_Xxz9u7FBwoVDVS@F?4TqheBspc`eWmpr4$Ou;9tU3G{3un>>=^zv-$Lo)|&tTQCYHsVB zdefT^Wi`e`03nweD7JgF`B1(1!c2Uu`e=xWOgxS%iF^7`t?DRAj-?`|+|3W|aIxqi z*OeJ1@V<)MJqla*uFM08QT|tvsGhSaL^k3K@?RS%c$O}|$ykVyj(@T$OUszhUtd0~@V9*<>{1qf z*&s+!rg^`)f2uZL9CP&L7mc%Hbkf9+{I#vT#*n_8EhFJ@{HA84!)w2qST|?|dI4W) zt&%L7rHyJ~^7THqz6z;2sjbHIJIwD2mhNcZbr>H+5H%XbIEc+VgX7ngjxZ6N zbbf!#X%uWpirp1ASjBjsQhc_L@>{1uE1?Lji<0)&mtw4;A!ZnMQ6IAYA!etw5hS4H zk3%~a6E%~sGj!+*it4EI3nEyPmjO{Nm6wvm0h(25UN@=680NGoOMN{xACxctM>Q&x$?w}qtYWV{W ziPg&6cP)(}N*~RkRw=ulkZ>)3-1zeJ{(DV-QW3NPXlxOzN>cc=5q34%nKR0&-Nlvu zA9nzoYJ|YaI&<dnP_e-`?KR0;huW>WQ`e%x16tuwn z7C6KKV3^T&PFCY@x#*jNwpk0Rk_si)A6AdTa1$QXsK+!j9~u2~(s#J1fDjp<$;?{X zX?MSL!7=XHM=jF=M(!kKx9FB;cL&24byD4g1G{{arw1?F)j!Zj2%rL{7WionZibv=#Jf) zxUQzM0KFIniv#LoAzjO;?du?H6aPEvY=%JS8WxHcoyJ@9BE86qpUdln=cV?g;WZdS z^EnF`0UOhR_;UbhJgv|7a$1ay{ZJ|HzG{3Ka~#7cqffVB?7q@;Mxx3vp-QyGIsSB( zNy);@IC~k20XH!6s`p98U7Wdb%a*>{MI~k^k>bkk%H9tR{R-{}zJ;X*G(T6d1?6}> z@_7hx5q?0h(U+45&7->`Z1X(I$}*H8`giS$1eI7Q`Ml}rDdZf{vK-sArY!AKsHONN`RCDjtHb7%PFW1Mds>= zgrCFWnfn50ACUeoWB}$zy`YvkI6zy=B+iLR6O5T~`K|CExLVp1@7j*a_1j?>Kwf}% zYIBFa3jT1LK*Cd1I3!8c2g3K>Ka-ZP*l5cgjT`nHri+|Pf#w&4)VX+768xvr2{nqd?trNmy#}D*joiStE0qbNyp|$}%;>taw+#D@46Z z3A)Z)HzQ$Qx$hZ?_Loh*tY7NZdEN(S&WJ=j1e<4Yx4Wrtq(6`#@u1mkBd~=9N8lkJVUjL zI}G2&A3qBL4Tyw5@unV`uE?0s@}aLaL`yBHD_&ChHKRN7Nfn)#r$gGGBufV0Q0Qu0 z+eYnT{gA4?@<(Q|Xs*8a%^$J4jw?gpO1Q@Z24r?mm3!P0XjBSAAoBa^{SsDc2xN@1 zFsPsdi-(DGMafi{?Qd2XzaMVUGb}r458KskMTBd?kBDAvwM; zOMCmOe;lt%{t91NY>l3jJ79I}+oS-XdFc9TU-aDsbgOfg_67_*mU211P1jm?_PK(j=vfZpOxB#XgQUrL#rzHI25+GuXKVmvce8lv8uP2W5UJ$bjxX1!JF`)fo1#lFYbA|`pswZ5#Z-H9Mttx4`Gs+zTnVQ7qBUYU(sCrZa&waz z-ywn-#6>yL!@Qw{&L@8XVC0sH=yUKg97#)SCGRK3V2&bKFhU0=DIdbXW&woA*hvwd z8-hMn3@Gh1Yd)>=E|?mv`61p@w*PoF-SUKY--mt7mGC?f06K4N&);Rgx(&Gg^2A>Y znkNUHd`&Fx#4mRD8#PEP9UeR$ZFU-T`D61bQomEh(W>rcUrwV1K-`S4zQYJLl9gEJ ztu0ylv-Wnkhgl$T&o4j7Hz_NSx2`z2&*kZN;N%9TU;iN8u;PB5$u3j_4UpZeAnp_f zM~Ta2pjQbKCk@Hp)=ckM2dV1Ahm#q&_D%=^tm;?w3TrhoUhkkiAMMroa4Z`iWFvqj zC$&Gfq(1`UEF7Hz){XFi^+g?z44YA^p1(R#GESf;)JGa2=ksyGdb%lXiLS#p;W|7w zU#%RZm~;F)v5k;NqItsBsofq6k?;o|$W3*m_Rr*?Ico&MSacPZ=8_8h4rDEtXpYP? z^Jkc3v}55GBs3r_8tr^`F=J_th}pUn)Cu8}g6el~!F8_yI#f}ID813NsClpAPf%Nh z_Yn;{r?OWx9R84nFZNw3Pi8Z?q$YyXTsVg&SIU-3g92e8$vgClgV*$mbCKh3w7`U- zX0BK_?8!vKGz#)8A!ZwdZi6SLFiFSVPlvbQRP zeBNY~pFddvN`L1{MYa6ITw2FF@K)XRW}kp&aV6B+>L&t~-JGXO#G<9ZEslih=;#_Z z*pkJ6SWv-8y@aw^AGK$_cYJHkOU+8KWMEvwAai;`#>ul7?>33YURV@lBJys3-C`p zty&M>(9k>Q{fz_?6*!U+n!Qg2k587+`6)9L>BkS3B5=1KcAS%fCYfVMIUxxo%s`Vr+?>$_@>5y zc{x9{Y~@h8D|dxxTeeXlep~B3isYhOkkQho6eJBq639>GajE087f%=i<&|l@wta^s zGZ<|BQyMHr(^84jf&gxoX(JSBJr0VA?i!r71lXev9aU2%xwa+pjr*V0<=o3u{~V07 zsT4md>x&|4=@-qJ^{0%t$a0-sNmq7?2Z<`M?bpQZQg>QnZP!-6)WteX6t(U_F zyqUlT=|8TB&Ft`W3u7%+3aY8N+u%a7Wz(X;`L_B6_kdCAX#fsUQ(^CHZe7}2=Lg^* ze|4bI^U9vjb#Jbt$lS?)+KmMEAJoT&mcwW|s=APDVI#r_IR-n~Z7StDsrdF{p3%kE za@?j+m~s%^g$E}x03@|DlACNwMbtIL=m?`D(cL$H4J<9dHsT6IQpS}rL^WY zC1-0+^oHtt0zkslphGwkp?%| z?Dxg$j04p4OQn3yQ)exa&VzJ|8YwP) z8n6B2Zz_bb%6IUjyVb@?#D5fj1SBrihsu0~y;Eh@gzmUELgYr zV*T>4TMx+{xaIQ96;B;MU$9ZsXmUCtlB4O-mws|L7#dRLH64|x?7d+Koo1X>5Xerw zcg$uveHHdWJaXgM{j$>p$y+d|o*I?0-WH-mg_v8EYvrf=MC05$K0jTe%5=79{}OCx z)c`1JH!PY*la!UfW!_797Up!CMG)Jca{&WK&iRmJ9#P_WZjh!H%rjcL=&fc*wbP`T zn3ye8`^2gd)>V_2N>E*}$WT3N!(2W88;oS(hymD7jvu#cjO<+?Q+!9@*ee&yGe=Y7 zC8wS*tut?x+G5j`%D}xQ?G#&4re8?VAq^-#Cs;&KXq{~)I7e&n)T~txk&bQB7;w!d zYf}VSHKmDD{6fw@5*D)4Q@f`|>x}cVT?j=<&4iYTu6x5yT`|!9EZ6hnHyQ~|RY$M| zwf0Kktnuu$u-)QsSGDUqj}Rd1_!%U{z1Zv{?3HEQhAC@pOe@mjjtkwS57divUnAh# zn&x$Rk^KS-@FY)J{J8k3o@;=vs)tTYBGAm*2SLT&<0}LKaZ#Q;d2Gkg!S}D1)mY^N z3{qUv`s}eC#|hFx9uKTvzobgH{-}1PCCotK>QM6V5FnmCLU~F62F{-56clvYGy+-2 z&YU`AiVF!g6-bwkg!yUoAG+{DnneH;%2~~VF|s)X!crqz{Q0x5x9G7XWJ^0CyXVx& z39`lETu)`HYuCbec^J>;kJPfe356=deg|i>i12nzIUxbI9OW#&7oDahEo38usrwIMx?YRfFcf@W+yOxvoH`0o7|!GXR5hZFX&v7V-w2gLUWi;Ez34zVpzl*&eJ3Uxzkn~ex9(~$d@^nKV_xkFd@n4-b zt~PRYWlC3(shIy@l4q_GUq1uRYAj}d@ih}59+OQfciOzL4BD7dNSm2F)Nf?4>eQ2P z2CzMa?MdJ(J8qz6R!uM_t(^6(ZiC98moXef z-$%AK=W0e)no32!uww}Xad2+=T2+W5ZS;pt>e`?@3-EjF%{B6@R$BomU~+%q@(*v0 zE*G6zc{(6v<_9cV_@q!?{OGIs%97Y*s(wwKYgvM$LT5^I=}zmRk_5&D(NtfYF)&Y> zxWc7x4L`GFdRmKidYQfN9RxopTO-o%U4)L3d5lVPKo!!x1``*bu4Op4t99UJs%v5C zl`1Tq&Wo~ONp_Y#6MF-F!qoL|LB^g_RW5Ms%Qp8wS_7rq_%mT7dqX{;Cspz$NTwOYzQBjwC3solKqkH zlX;9+(PF556#g)vPIyaia>Q@gRKZo>DZ@cOHxD_A27>mpTx_;YV3327BI@Kix6YtJ z0~cBK`O1A+(a?u};eCob-mUJDfiBikDF~6tyS?))-XL^y2s9AT-tC$m#c19fnles3 z=n&`*WOja;>m`m=iLc8E_;c44Da4K=w{#KL0a;HQ`|A`C2W=CAD@yR*S_pZ)Y%7TU z=rl0n|Cj{1Kb=qu{W6&vFV!WCjN-9lh1UPFH0q2XQram$9c)N_nAgTaX8G-uh{!sT z--ImY_YoG`2{747EafNSqYqby$C-4)TPnN4s!L44-iO$gQ#?Koy6+wzt?-g?@2|Aw*}H=`y=%;JxETGby@ z`7tyuankG;<&aT~YkHVRy#k>Q(u`X$oY_LUd=(Q)p*Y7|$>0bl14&+|f%6Z#Wo{{j zWukts;FjslgK6~0o%qowi+j@#A^%d`RSxqWH@_VKUcx6v-t`Ua3ytuET9fofh+)(x zri0n!bQP7t^e0#B0G6Q2J06`=%r-NFY0RO3QSDcyv=iJ9pCL;$4ad0bF1BjYW0nyz zgq5{dE^ zlVz$WS{oX@^M!58`{8bQt_Zb*ihGlUgD=8d>1RxQ@nPM%_UC-+F!g1mdUrLPbo}8% zkauZ&x?MLXXlWexuP#H-Mxq_F^d-X_D^-bX>n{Sf+mep;kP(| ztUVB2-sjjvxQh`PJod5W&(Xb3!n9Tx$mO-JMI9@9hO8l!1-DF^dP0t!-{kg}&jP=j zQhZl%Xy{#{21SMrWsm?0#@BWo9;07PcegZ zzSeN|)T~ylR%_x-5X($B^pn*1r%srx(JM-GAw4CsD zw5ol<4?}NUq=GMwCrt|@^^!OMCQl>LDdFJkS zl#Gf$x3_z3ncSy5>HwES{hrbYWL_~NE*QvZaW%iZ>7rNiq_SV4^#&$Xi&6>{=?A`8 ze%g?SK1sp0QPUnMtP<;Nz!|~fCDF)qQD2SJ825Z=YCxP}m;$o2y6@fqOo$C`sX7(# zb>3APT0gl&9cTCN37S{0Ispj2^{d+9h51L(z3wM< z39be;OAWZ{pWgg4i3&F%z{pZ;dP!3~VY7MtTu|ybc$EgkN&ysdqE6PWV}1((L;^eJ z4mM7qr0@ynn|NH}3_ijqGsyt?6F)lYIiy5k=g9p--g1mbpe%K9$@ySrVaS!06-U1! zi5ZRG3NZX!Q!pFM0>_$;^s@;e-BnS-_&n?IUh%tyG~;x(sz8*u>7LW z?HAo)+H<|WZ^F$4K!S)D)qP`Q!uLS9>FSF>t@qUvy0J(N(Exmy>-E!oS77Gy=l#3r zmpA_JfNwtmeIMbKq;GwE1i7=B{%$X7r%3bMbv7?Pggl5 zPQ>gO1tmv|qbvLqQISf?_W_frmyW^0woOiLUx#LUQG3=3z}YV6d_(WDO8_gco3{LJ z7|69dU>A7v(Ch2ZRkmE`or+oQ zwWhd{t&;4ht?C<8E9MLV_lJm2t9l@Hvv}zSgG2e8O&m-2COn-6bIP#~%TBPUbl3i` zOVZHr({F{O4sj{9CR4rC5t9c7vn^2wGX(1)KOWm-*;`CS{6EB~dS5E*bLp2Fp`3RL zZSf83xBBq|zm!L$vAqcFQ>*9?zyDpO{O{t%|1MHOYM+uoeiSL;KA21j|L_+o5qY@) z)tY)Lm}*#WPObTSbrBA!N?66!@)HycWaCcxOeH|vT$Xu4JO_cPCbS3zt|!_TXyF@G zHetMgyQz2cm_wuxd8k-;RH6Hg%d?*u>#RRT{GootTM~00&8tPfvDrd0!FX=;Cji&^ zFSc1I2l&PDOdT~xgktIk0(yjhu;rTq)bsdjuUi@ga;Nw!G%?WJA;jjl7*ga^;GwAW zp!_Q_A@s+Zq8U=Dn`{L3(FmFmw0#wONn@+g_-(CIMIF}|%7&_6GTlffnu*BoY}ZrT zv5$5A?|Vgk2MU6bqZ=F?E9v#pPU9w76>cI67NjdwRi`bK)8wPNtme~l=*rvxzPzSR zvK+rKDr^qjB9jY7h-EIkzFFYsH*7SB>UglEZl$azHPWb4akLQn$qvHJV0xw4VlxjJ zHoZ<|lqJeQ@eq5z{8G=M*FZ&VIGV0Cl@|q-$PHguaM0JG3l-wO3#B{Ljupg9i9~~+ zP@6k{D#@((;Uz9jJNvZ&)8VoT5JjYbIdRz%6=JbT=V1%c3mP_G4DagaVp}IAun{c` z!a6|Wa$V^P;=(~<;+UHpP3pbPP$gGA{F9U%b~iY;R>A>q$7d@7fva*DGphG~K8Nv_ zOo}MiFlyK)r;Kyh7EE_xt_aK;gRa1q;0S?jQ`CCMZ3dPWkJbjsQ)D=xpc2!`E4G2P z^OE)PI}^E_^Y^qj(cs@#2eLO;*Bi-yc5{`sx>to>ZwsYD@F#!2SJd3>e1} z5D#Xh=lDEC1z6g>o;16*Yg3pct@*d~f<(5A#mF#LWL)_0-~*+PnDzG(w}qP!Dr6r+ ze+c(uA6@=%iTFP9iRTd>5bd6)CU%#*PH~SGuIv`haGCTc1?VU3C{lXD`Xd~4dw-9B z=@aYQ<@!uSrE)TKn#@fj*+!tT>)p(FB>nX_y5SIa)Dbxs(of!7+lZw_Epq8F)Db^P zD`osR?-m&dSTx!j%u@gCy$&7T7((MsHm5J114o{xPZ2KDiITKA?EJoqCPc(GkAKtE z{~IL#8$QyTY}`LSMi%}+f=n`z@jz()QcEV5`5?CtrnD2le=av(lM47>8j;pNG zijBGSc2MY)Mzu&x2~hj|o5xeRp>Qra805Un;&gp$mxHM2xnC#stfyRCloqZRV!^as zRs9_ufM;)q-@OjnEI7@}xVwokW;EAiS=<|Fbt98_XhYu>#*tvcuaB_}ntE^V?A-$} zh}2%!Q~cg%NZvL5oo17~_%U@!^1Wsulix_`$~iyDjGBTwuMPe3PbWkrXm^lu%MkT| zaA!aQ@{-P;yCrs{Qt$!cYIS%-7iLLdK&ufC(3&|V4v&d5G(a^T1FmS=(aXP0w3kuo z-v>u}2nq4lUiy18-X%wc$epRf`UZ#D&Bs`S>}Qi&=~)+A43GJ9#xEyD8Hi<9)WWSJ z{J*9)|JvD!kAsl+|53yuNN9gMeLP(~QNX}t{^5x_S+**5a*f0MBSMgH$|Cw7u>(~k zWKZzFwVZOjsN85jev2I*l+XJiWbl`k6UZ6$6Dpus+ffBm3@xxF+uO0cD>{P08QZ{k zE!bHWy$?csE)NHSTzb-VvkBHR-mn(#`*3X_meQb&*~?ix$Fwn`8QpeY`1*QImTzvx zYR2aC-Savs7-c%Dc%RExU?SPYP7g3#{hBTUOqyE|MSLI~SOkOz_C`7K`^&L9afBX( zZ3Db2@^GWOMz?1si?lV4Gms<<)|z&M1*VO=_S3_BfP{Y1LF;l5x?Jw!-ts)wy8Kd( zFSQ$^r;_RsD%H5e2E5stzCxwZO!Os8JHL%+{r5H9g6Zs_0lchHaV$s_zNA+XX^PHA z0$;lFq77e$o?v)o-5~PK9_Ly9hsrKUyYS**v}xGCZEPt&4bNq%XQ^q)&$e<5}#}3jiqm ztmej_fSW|mUYKrpR(Mw^qO^T&YA&syC>DyV;ArhJo`lQs z3}Tk-CQHX~T&DU0=j%DI5-Q2|Gyq}cDF1X7GQVWk1@oP{r|jAhkvE<`#jM?-!XzO& zcRLd)5Y>3*BDwv~8aOWhYx&^woAYTxUL`k}9sHLy%dq**+zV4uf}?2CSQ`pCPB|of z##^U1sGqnCFQE-=3nFX}*g8Pr4j5*xrFSMvvR_*Cml&l)Q`ireSJ$`Fb-^tp*k`dp%?!C~)tkXg2hX zeBV*(q@%SbB~gFThEeUN)5+5gLkoa zd29XW>`iX?I))_F!)&3$Zoy0Ut^29R)9;=4U}rn=`&oI=UWu+^j>DioOuve!HeZ^-6acE9__@&Td7tCwOEV>rw8HCjAHQISeGd&^h z1#~DYXiuM|spiO*hAn^1pXN6mO(&zVW;RtR@GQ1vHE@+FTt^8(CzUS};?jYw09dW4 zH-KPMZ3SereC9;xP3Jk>!m*}OTq8}1IGNm)`*HfaizeyHgNu*p^&{3{oAB|LXpn{& z(vd2b4NmGB%Nd6y860&_LhZc#9PWPTOL(bc!|CKD{-DM)vfqF1O8u7KyqtV$O$QzB z4$WCfvOLI5s}1hlnnw|8w89LO1i*EhVUJ9s>DMJ!)leV!lzW9PPH^z4^r7+G%dMzM zf3K;;jM}nR#wyVq;P>LLR5s9`-Yy1(0+}^9X`T=>g-fnkzJPuya4DF4V&t!o_T>H+ zuMb5KU}h_gZ;mo=*4E$TbRGpXp`#2FoX+tkwjafSM`)XGMrG7En~j&vARvlY^G`w z(b29)t)_?Nx9FnHKlgn?@)LtUZzKIky{KPa&9R^Z0hXB@fu@VJa~Z!LH<$@3R5Eh4k>5Ty(F#IoJG686iaTR$ z?u?BDG9Eg}tz_%c%SQ6Eh9lKZbJ|in7|wE1k8(yCtIou%gSUL~(1PzA-{iyctldD5 z^R8jg7(Gu?6peY`Uw@2?D1dk^aJ{mfW&MJCNN~|u?&|Ps8GPju832`JO3%ccw~Hgf zkPzwvM{A;zw1-`*Kyf-VohqwR<#_54&={c#zGaJ z2+GVr@FoPOuFzR4R-xx*g_mnMA9J(~IuKsa%k({QM1Y(o^6B8{1eqks^Y41MyNxPV z6KJr}5v!wyi1gFDRRI1+!U}+KiGQ_Xdxv8Cyw+RRg%n+eET0Q(U-VdFAMBWK)oCOb zAlxV#E|#gl$^!g!fbc+VT`)d&hYc5<<)GdcJnM4(wn^v(y|1RCCETF$g4P+MkDsUy z>>On~R7htn-Rn2RSJa22=LC#pwdi=&@c31D5iwNZA$ou9B0yc54#g3Mg;kV=l_AIH z-46WJ_uYh;*EvdK@!}Xr43XZQZ4s{r=3Ju3H22$fS{Y_zqQz8KO_ew&re8J_8hqD6 zxH6v;3R&r|uzepgi2@|zdT3>MM`uwL_@ew=78>;9%((p0RVCA9HWaHqOAbVg77MH} z9PKuJ9sZ^Fr~$yYy{kC|IcAVp**e$f*34Iwb%(?%2e%XPNDIy}UZ-~qV|cbk4f1i{ zBSt=^-gzb8c|}tL)`!(DA6(W$bW`a}H28X{-yMx)f2!-IM}7Jw_CUwx)TxifOX{MXSqX4|AvA1=Q90oEcl2g7z&V3 z2w-q|VYIWvy1&YZ)B0dP5q~Z-R~iBF%R@-?<)69taEAe5S3EsVTSa2G&N|Y? z^$~n%MnQTNhm+nP>f*5ZfXg-5Lp#|6neEQvyE5qHAal_4nJwZlQiU*!Z{)?07GzR0qeS<=< zuIfxPYGnE8cw28QW@YjN%IUU!DEI45j$vn>5?^qS1x(s%UC~}sIA`W`9cZs%ht$ul zlb#TwF<|#@8q(1TfTA1RTNl93N9ozRk~6>VEL2)C&JJxN1(Rw%fdFfn;9<+V(Udi} z)31&bkY=%5R5)au(=IDxvfKN|LS~KGH8YzqJ?ISUX*-NME9e|!RlVafldpBdJxv}r z*mK^?#~$LI1(;XM7^cwgX875O1Z%O7gh#Ua^SKalxNl$pAn7G8(n4K{p`4Dw&ZW;$ z_5!6H?pxvN#Bk&!SFJus$C2f$|4!|*!vU#!=YzKT#6t(?x1fs&Ux@% zMYaBj^*TS48vtZ;{XddQA~p;Wzvy6S@DI?RW2=|1C{V3mC-O$PlfK0*G^9;gv)Z@g z#Nc48;tVU}0i2Q$OD3rbOus0|{$^^Z!T9OGR zb>J>1CX^pSJ4y$&oiug=g6=HoN=D1DZ58=sJpsD9ucTGkMJby8^d7=DoidT>?*yRo zRHP`QKUL-%$OyU^RWv3jV#*{{>>56vIiMkE>iG1W4_QS(&sVc|}D{70iah@JL7W)PC{BtCTJvAY`)NIHC0ybl9>zQt$U zAKsr5ib6_Z%3(1T)j3kc6GL{w{6RY7WRj7SoLups&&3`MJxNBebfOdMm!^eu=kss2D8JA7EH~@j$R2 zg}9QdOoU%b%(Bgyjj>8!M6DZyifm88o7A$>i9O`{T;HCdz66fR9&AK-jA)K}905@d~q3b|Mv?Z!e z-LBcsBerL1#JE z%tT18YG)T{R<1FNR&afff%|R$2fHKN!%T$q5I!q{*u_CgDs&DqL1eYOT!UrBZ{x#a z`9W>Fq^dV=pMY3+`#Dt zEet*wCb45y{+JEbeZ6S|(@fj3yk#=8?SJ~ph6OGnNYO*N z^4awueToxlNoJv11-nq>B$!L-Q04q}8mCdULB#Qn_)@>A8c#Ci{?n+V&igplt@kAxVTht^P_ z!`4{Dkx>&AuZgoc|bu%}Xb_ZzwS$Hcchc+*n6Y*#miJ9h#eVx6{rmxPh zD|yc>x!E+r`AGeV?&F8PBB6wwCw&okD5@R@do0P2B_w$ekgog?cj}99JySG9~*?2}t zdG;c*Ii8kPD6$?^~eZX_-m+?QbW$EGhB0oeaV$@{t8 z7Nu>GWa#oxxK%|5`&*}RRW4j*CKkYH;U%*@Grx?EK6bYni6{Qz7gFO1w}>;`ou^x& z)bTAk?K;M01zx649ET$X6-;UK@(C(~(O?R168FHKMzEN2R_fRT3!Zfl1xPcetzztl zy5@JIW1UcN@5Y)xZjHmI+&*l&ISC?TMhh~@`S-SBPQ>#y4Ebb3nV9vv8ZUs&iw+|O z)bz}YQm}O2V`GK*{7WqBl5TvM%XuY|yAd{4HL4VZ(;{{`dNlYT)7d7qB)3wMYrjGV z;6^9|MF|=26a8^vf3XHlw{KV74<{b%X76N-x^elhsV4Gp>|8Xk*+mS7w$#FwI$^pD z_-Rz-L)$h>62S3gdk8nZy;cAyYI1^3ygtUhUQ+cH!)tsc3>^R7q&3vfaw8m;N|0`k zTu?D15-ZwYrWDZr0RD1Nqv9AT6IhU0NKiOaXl{KVIou`rDvrrTf0SrQ^r8|mAb!_x z+mS)eIZDr_byc}>7!5$0St!^=f#AJ|m4IWpxKMG?>Cmobo+n@B-7 zXXof|P?Z`a`D0TpW~WV^*nahnWRn1v*5~I>&#SjOba1Ot| zfI2>x7%gi^P3d$nKR;S6VcCvsYSr;4fUPZ!YTkMY&w0i8n`hVPJwIX%N2B!^Ts;)c2^ z+2SD%jH(d!7cdCeu8r7tIDG4sa%i|{kl8g7jCpAh<9~8G%U94w`K3`*L5fI+U79eY zOqrF_C;kUNYUR+Sk+m)}Ad>rxiKoGRWwJ*c_L*SwOc3PHPe29;9AftZr(Kou9}gVR z`y()2FXEE>8i{zuTFJGZAcmb2eOi|-UyquhxtJ=MLnZ(FtblK}AO7Uah6(l!g*JMs znVMEvNitEs${3+5=5cQ6W#6K%M2Eet2;s3F-2{U}f@<;|sIy$E#dPRcIlJSe|T=Y6Fv zz^=5k-wk33=|joya?oOn{3QP5H~pICv=GKz>rB@cX%LzLm_FnjBUZ+{`(**uip^A5 zWCHDNv1$Zeh^N682XCv6^!T0}V!B{1W^|IQp|_UGgK}e~kzTYQj;h9CME^}hF7W{u zQ5|TX>5{EkuWNy)Vx4>cME+z{xyKn(+!>x~VU4QI%{1ptTG)y)$OU9Wda|QOFoAL{ z$9Z(6$IBrHbYuu1Yrq@y{}yl;ZY3UjXqUi~r}-Y-yN5h`)Pgk`H(% zogaVu1PKy5$Nz8t6Ds!qIG|`4DG-Wb2_;wQHP+BH4=-IVI2jQ3#nwj@z+qs8+2kYl zsPKg7ZBF-37d2gm{keoolP7(Gd@7W`bbxrGvhZGAefGV`5O9Bgdwzu#g>;U>8^{iU zI3UKV-!?&4;7N0=S}Ehe@?h08-W)~**7N89mZSQFQq`%ZA{ePZz(xPLX3B4FH!rgN*QY_dt!)A5#VKLs}=H@0812)j_%NQCzT?*(>{8)(en-A|v z2~qBvQ2%Ng3>&06ZqIe%2{c{`A!Ltz>iJ;xK$aK)kC)06kKOX+8`Z!!wQ=F&L;)ijt(yn^JuUT9&qg3#(?dF)@TRq@TvOZeH; zI-xry-cXi=(#^rDE?uff>GK5Oh#?Njzef544{uv-5@J#JrxLqlS8tk9Zy*2G2>5r8 z|L>u5R(UxK`?%0Iz=DBs{Li8D06AuHA*N-1A_V2h!N38G+}9 zQ(zb-$el7V^fBZm@;Gw|o=j*Y^m~@B=PI;{Vv4Yk06@|N*mPS3SRjh?G2Bo za`IT~rjAa(`|Qp`lGB;*fxZyi5H+jmy}pPOT6}GJ;n-ONQi#s0WhPoGTP?2eAjhr{ zZXu)A-!*_RMhIojD9r}^aW1X$uf8gG5B%S4*1(HU9Ls} zaC(sbew1d~T`tGLS%CT?F^zL&%nR_$3wpXETnnJzIYwa~^f*w#6h6wsZ<{77QaCs>WZk;gJ*cw`!Mt4lZnKmG{mdHv@q=W(*dmM5^#YDb-}RrbaF0Shc0%j@-1qd=T}3ZAef6MT%JBy zSK?vz-VsWNfBli$5GRT0*|&qWM`U3D_2VBjbxRk@w^f8tLVfddqRsB)CyrhAirl^$ zlG|}>uC2qSHCzdbDI_xMZC4dU1Jaax%bm>j4T6K#TN8zF>CGRC@uq%vfDE_HMKFxL zBebPb*bWH`cj#3FlI|EoqPAZRz!+NC!^+PmZ=0`3VCApreCr&DDUx zfrlsi2pB#%S6LXK9ZD=m0GB9<<5^J zVK7nz_D=X)Q3}(+pb%?oE9=1A)vKrVd)Ck39`&~Qp?AZpA!oM> z9}!`6jMXPTM8z|q{)z_Vx4){YXop4-i)HsC zRLgjQG)|bp@3@p>06$&g|3}t421nYk-J(e*$xLk9wmlQuwr#s7>e#kz+nLz5F-a!2 zPv&{|`#s;@r>eWE>#qA(Us~%TpYUE|xv*Gh74ll3tuJaX_Q2jiax@s*p9KvA8<#)q z==v*+3JRJV!u_HsbbXTo>1??i>38z@Ar2`MGEO&5GVa`Vw*Wxp>QmWvJVqD8T*IP+ zN5Wcb_pd=j`j6_-*N;3)6&YY77%Us*rHO%6^ciurYyBOlsFTwvfCTztnO(nLdsjyu zhs?1?oL;upPUp$p4XPNlvXXIWdKD{Up~Hz&;9%<@(Y=y&tN$@@7ivwBYd%22jddw& zK&C~-J`$ri;z*j-LR4x&j3T0LG?y-OD}CU#tlVG4mwdd@A_z@;7LEFhxr%YAM@6Q+ zNOvR?Q~Hy$Sz)jeK?wXa>G_XFe8LZq&(52FRqL;=GFNkdBGt7tej9(FnyaOR#Esb(Axp_R3y{$08j_@P8)d|^$lP4ac$(~LQdf8CBu0>p6isI7xm%!4O@UV@ zp=6gc1=}q^a*d{)`REZwWJk3USNBrH*F`o#u-swJN;?i+!}XA^w(-}|3R*4wY*J0~ z9F~;(W9PMndRqk0TeHy zqBILO(6p?1g=wNxA~?}n^mxLr`E&uv4%T|MkMYz1zSxfL8PN_aIm|IF91o3anW1}U z-Z^_x;ckYd440rS5-Vsd1zq}Q?5ai^-{f+Hu&NvFU-UU62?>0_D`6$Lx;84Qs&N=S zCigwxmHP>{R&!7C>gu(oz3>=^Shl7b%OuPu4P^TON~xrEahfQnJ}o?Dxd2uUbJiHN zXozCKpTmQQQb*r5i;1Z6V#vvDx-S>{;{D(I7b{FgOblwheKex$f$?yW5L;|#2RJ5k z)Z)p#n=rAA`i8%HsAR3*_MAdsh^o|bFsq$tPRI{nja^79x!LG0{k!@75e49-l}%Y z114W*!^Z17AT>Ko67F5lCX$ikE>J#v2_?U@L1s=WYeGj;K7EG!ii_uA+7`K3B{N(; zau{FzR_omr$sL)sx8 zvg{}Wp#(DIgTbcjHywX6h@Tk-_^Q6*7;-PP%F|X03j&KtB$P=?NU=MO~emYM%Uz3YB8OBBc9(EJ^jEL zcm3Z`hVIyyvdUMNVvHq7qol1T(sCROW`nD?aqE*et#W7C{ za1K$VLXnsp$^mHV_iSsRu}fx9ri6uF!E~2Mui`NRK~FF>Up>^aUeF(U9!SOWL2Fr%~k2yoioj2OBUb@!187AH`P{xNOY3((lE0l_Bm{tM{X+WrIJk%)-xw!92xF8n8Pi1EF z&(kK-ceVFQc$k&z0eplXAg5U4OdyLKoQn^r(`P~Vpf}1r!d-o=Z*Zn7K;Dhfcx z2}cCchorJ0siQ730wn~Oa2wGgh>T3%+7RUx$sdFE3wz=hd;JccCJa@YdQGf!>aH+i zdECBK9A)@c`&?)wjIX{R-F^T;G?52!veuCBJ}zrHIk|_^tR=pep{?#W_-$x*8SRm6 zBHnZn{Ha>@{@o{9a8r$Rqce!bDpP>Oc)hy*5k`FrF!@nGwvmiqoxRp@*TgRuNWh(2 z3=baIujMhlR`hz#jZ_xkF4Q`@r8ahUW`kz)OhDsjAlDijN7uOd8sfi&EkkTB=%pg$!x$v@|Kv;}KI>9;?@^A($4>MnrF2ycjL z1C>!V4IhoI?J(G~b7NU?M$3_gP|7zFHz1w)+GC3aTf6kTs|rvzCu2~TP@gM-Rujs& zyzuv*ljD;V?C&!!(p`}ZP;6rEOgzo(3AdC^3d!lKhA4P;gR^fr^bejvg|4e*BKO9% z!W-gZqc)+A03-9SHz`*ADQ9_JmXJLoB=oVc|FT-ML{Tsmmh$gsZ&|;C9>>N z&C12IR+bu_^Hs@<7&?YMb+&uM1HANDBwJawm>t@qxLe~9D%(M3 zPIR`HpLcnp9|62?Mzf@)l?4Z_^$p}L)t-E7TBQL!%a{mYXG*s+yS#61_#|iOajS-F z2k^+QPxx{7#$SOxT>*duJ_B5$4L=J4gW-1w)53u9DeN1ExFbrQZDJUNeE9^MV0$+O z|7Z$K6krktBZPS*jf{Dum5k*Xu?3`xkkC^3vCg$Y^_Bk{O`U-JNgp3to7=DvT7)t>XZ>xA4NoEBQSQonU-j ze*$raN4Yx3DpkK_31&W^nY-Aq%DH}06UkP-x!VM3=*W6^N<%ze#QLuYo8K;NJu0su z;oQUC(Z(m=@+lzT%uDoB_9>iy$(6QeD|loZ!RzTG>CKXSgyEmliF-#vb`j-u@rf#w z%PGLj`GiT6e)Q?)Y2?|6%=L3*7J`@(87T=3mS6;64O>>M%>yr@W)hF*ROYUkvu5d} zrstt~B{cDu9QqOe&r&v}@n-13Q*2_TcVLD?{pt34^v0c`)xnnfzF$$riP z=@g>^lYU6HQt7s+7t&25A)D{Tmd(_>_xbo z5jJ|o+~c2vMq@f7WiiWkx!1M2Sn)a>tCq_K9p7_+-DtrQgH_@aCqW5_vPLLk%5SP4 z9kpO&U7uwH3{4-X6(*<6F;3+O)Y|Nz)1G_u84V{D7%lfbAi&j+9uG5HMj2s=co0$t zao7F$ZpkFaOEa7FPTgD&Vh@R7nP3mzV!E%)4vd|<25M={dMPt213F*5X#>jUcXOLG& z(SNzMg1`7~+%ZkL)wB-RDSOLQ- zTCIp7+NeVA?$j9Vf|IRmeq7&U1Q6xXfpIWV*7~)mIK7`w;?A`v}86M=qaIT z-rGeq*kzRP)J%G(F1%&Pkl5iu85sFcsMoZ@mCVJ@2_DAC*oyk?xHv-^`!5JW=+CK#F-ENR~#0(vg_TF?O?*Rdjp+ zuB+DiR=pOzg=tfvQM=3<0Ar@BdR)|x2~%reXGw?W8>h_}6&|`O)nvx)y>RlFyhTed zW9?1{Y>*th{}*7pg%Ja63fYl7Q}3$9AFNvn=3Bf?#2KQa56J3{8oiZ-{NSv;(eLh4 zZY?LnO+b?`+>(%kvEih1J#_2rHDF_(jc{$<4WS0=GEip}3?oYefKVOHFcd01S+fE^ z!wjA}=?+^?N%82Nxt_|#0%gDJv#h&q@iMs9c+h*9HLLdZkFSpLYgH%nJnLMTQFxxK zy(w1FO-=U?QZjIEm{~=>j21eodvUm*V~;jKX(mN;K0Q~F?_^ERibBSDi} zID|9s3sWOggXf_mKU2p~$S`xKLe-;aeI)XSZ2!|Uw7CJUxyrw9aFq4pa-Q0G^RRUx z0HT}M7DjxYjI{kDB$x;vD&bYV&?^}1E~>7U=%V}Ah&r|qca~=PUR9bj8=&zp7bS|d z^|+CFlmH5MA?f_YJ4w%^q7dts&P4(i+i5Y_k1=?0{my(?pA?zreb2gTuDY4p(D~xo z(s4cT0O^YF^QNjxv&=G~8*e6Te>x4%f^PaF-qNuBM&&Iw*mW4<8Tcqr@LQD1^~(N4 zX7Y_C*gLdZnbGIEQO`B{*Z}6kE5a{atdVV$4gqq$)?bP6#oYaikyPcw5Dk+7ud^qg zhZhGL8jE7*D;DRjJOq`hE0El~NO7#33M7hd7h(C~gZIHV*J~z3q;7~=ZX}07*+#~> znECk~T=Nu1ms(ZYEhb8?>tP-t!X(MsIrG5)M7BTl6BW|mh}Rki-vM$VgS`x>zWX93 z{8HNBS5Q_F=o7&*{1?4CC+l3%!z6#sYhVEW z4~1!r*Rh*pL!suIZU3WTsm9kM~TH`*a>=2-NbB9W&nw2 znrTs+2K8;xaALh!e3w*^<+Z=LN1lj+ICJt~B%@~Tct;28JU!>dj;Jc&u++N<*Zbai z=JR4317%+^`GQ97_@tD@R}Fz4n<$u8*$K1_udmFE33U<$w3p?;n|}l%KB#4yK62Ll!+3!IAxu0Lym+?_Nd@Jy@UOWC?~ zEiSl@&NsZ&_PRfwPVs)eEKdN8kxX|s=ly?JZs&v zx9eT>$$o*2R~f7PbdyokgITglu0cZ~l3&^oWl;GG(ui{{#%lesfE2&N=MtK*6k)0) zYIUkK;q@x#I+_cYtUW)k@B(e^PP53)_kyWv`GLm(7%TKas>h6>$o{A2$qbwJ!MgJl zb~S_Pyga6U^;&051EHT$V4EN(ey3=};0~6%p?qd(_zzTlRs%KF1SkIab;sD%9*9%1 zcEv*o>pwX%d3d|5oWT$?WYd@@i8H;94+kpcB{i17`O6-Vn z`_EVarBqLBV@y;}A}gblXu93J3}riU%}9*aBW&-{u8R$?MWRGR3s5Pp{jMTF?*1*Z zC648q8 zr8hpHRB)9^6-SdjWvRKGjbXADf~D;SQ)vXiO?dhsx@VF9lFZq4uF?xquTD65UrWoo72f)I!SM9gluWo@dS=6+ChR|T{u`#P?jpYqs^&B^d zCmWMK^R%w)Pm&Y8f(n0rnRXID`2e=dx^z@SW1r74ky@lnFt(apRQ4iwziX=T{!P?y ziy<(UblXeoiUH*k8eOQGj5mJ^^^F9iYQ~Kbw|WO9j5YeV(swwwS*+Ei>`N)`8-~BW z=N||2SyRaWybAfR@5%dgJ;EtvE!dv__@PgD2|Qf^FB5=d^2$@g9s>i?&mwr zR&fmVVBoVtpFcweAPc_y=jYKdr4AU<-=_>Mi@qeEgNT`MARyeyNHcI)$@D~=q|LbI zsC*FcGoNqJMge?9KZhEvXp>{tAqkS@qq&g*3mQ=F+6yl4(WV6)*nD=^U`8Dgz2U5J zF?3&;`e>WT_L%C7wRB@5q4rYEy#CBYu!dvw7?XIZ_r-z7eJ6yPoQ3!nC!YbLLuk=P zL1!^rBx5m4CY4+uwVF2lvCe@V*);Q5KK8Ki{Ce(lk>zy0GS%wVbv4G&4U!|meAbKrCa1eNh-fF&FYP(KDprCz?lm%=Hbq5ig+}imJs}G;tUU+kw z?p(ZG?hxF*i7jNjEk|c>7e7Bov!w(BW)}cWmkiwRX@lH?J`YjPS(jQO{rIuOPE!ai8J%J5%*IE6Nu_qK zmQHDO`Rx^YcE!!eLQ=!CL%6olh^K&Uq7~Xv8+NO%ZH}x-02$6(oy01w1;+A@@vMZM zq%boU1xbly0X%{0tp8z@dl}cUaeZ-`(~5f@SJpW9u`%gP1<2L@&Y}IgQ1FiyV2Mv` zL;N_jYGhtfrZly~x&+5iTl~Q?-H1pOZ=HRJ;`U+G~5cPBL3+Y4v z8(H!!SLHiAI;jw9{zz@jEex7!euF|v#l7^yTFlP}c>rNHvVq4=vmF8;Y~ z%zPCI(g$fp(o)hF-FE;l@J$(EwDFS=ip)vviysu7G>(kD42^iD=A^Y0aZ#sU@A({F zq*+v0a{J6amP%o{?#xnfbeK{H%3I$Gu{Ku3ZY3Y@(3|Blke*0T#Gp5dzonX}@@NDwwuC=mD=d<`DPK$X zuPC#M4116;lM|0$ZLF{=@extKY;35_pQXpi*Ji5~Mk0l}Q^twR*T}@y1Rk5Neg~2}uW+t*OLMry@cDlcq+xp^}mV zNG=N+X;NgfgBX!>RnigUXN&UMEMhSaomfY19fpWFft>XR16Hb{bHex=dM36pRw z^uJ0x6!PGW;~|5S=h%+_LSZW6cAd4>gm5Y;b{2jZ{~FD@s83tj;q!k8N^N z92cXkRvG0PEzfAJJrS-a zC&+o-I7F=Hs)9T1aNusyp0R;793C{gN{BQPUv>w}u5-Z9ZQLi< z;x@)f2i7sCMRG<#qsvajN1||!+$ejTWkmYMm_q`1-yn;f7`3=)9~~Xmn^#O}K`x*_zgA`IH0x)W?3Vvp&!@`M}3{*z32BvR+|mR(I@4 z(mJSYS|>-+f_DV=a~ev_C#BdnUq;RKcJlPMuWALjW!S==ckeaG!n0oY?rphK)Lnt# zH(UV_@tXpX@#6ZMaAR8{hb)k}!jkE@Or>^t_stcMJXJs?q!gQOCA#S>jyd?0Q&|_w zIJ}iCjIBpb4l!gDEWnd-4L6((d#$NUDvcPCStC_1-FcB4$S8(vxcq``yhUVJIZ2AG zTXoKB4rIA>*m=}S#N9Nm*|j$5FKYHPn>qke=S>b@&yDYz4mPf{EOe|%KQT*nhg?lo zF$sn5x`OW-cYDC`tK+3GuNNt%9DX$EZms?-E?q#WP;YE|NCfs=(!!%#=;Uuqp z-BAVM$f^~b5z%OQF(XPGVcu|tWykDwyRqN{T`l|=dMW&d1EQrHeAOTyVFCgZSAGCu zfop(^1huMMX0J6jvWtI%5O1zWnXbKwhCC!(m?4R#Ai!5Ji=?T1WB9Q$KUiu_nZz!tBC6S zbv0f#q#?x7PtCw?WJ z?!@TUqTk4sADdGHIc&H($)&@KO*eg-35Doc7eUG1M|kmS+6j{z7fxL5#5vl+`3Un$Lx$Ya-JGK-LGR>8 zL@d4HzbwVE`C>L@-GSFJw{XOu;bRFZv(y-<8NCj`CdAFa>Zy0mX-#kV*{2) zoMb2Bh0p6ab5J>FB&XoEluJ`hW#P2!8IenwO;GI~`vw-P=;yz`xitgqPEM>(i@Mzs z0*~%ticO<5NA}0^x0?)u&d14QX!Yt{u%F3Zopz&%RF%mVvZn@hd(q2GOjr}=4&+-U z!yuU9QwU zX;*>csUkTvupUVsC<;K{+rpo#-4lj%*=zfx&C^O26FY^CB1NVdR;+NcZLbsh0XvfT zfirxkUrzu*uTd@V1c|Icc;unuK9AQ+VZ@7TRT6wui5X(F4-iV=? zZB2$FH1X?9F^`*00t*(?7&elrBU4_#cvIe>H7?tE6a&x?)PxN2 zJ31{mj4TTUDXAw#edEer_$*3q9uv}RIuhJJGlK<0Bh>(o;OM!#aJEV&JDup8W;<`_ z{Q3q_K?u}SRjvX9M^NiR+9&7}xQUV`zqyv>a6r&1N4?ig)LoiOX=_%ay(oTUvEe@* zO2qE9*dgzqTn#r%3*UR*u@6NcZ2w^44-AHziQdkqSKyNH7U+hZuIQEJPipiOS|?-- zk$=FG;!>ZXd$CCsqD;4C^jiBM{UY6tFZ7pR*;7~2qGga4 zR*5U<#r;Sshc3zEteU))s`GgjU${zt2@0=2x~$cykDf@$kPtd{n^y-ecDtAJOV5q@ z$$2TXd%IUf(Yb;4IcVoi+#nHaxtT`xS(-_fN4a2kK5_xudrVNIk}` zqXXaUJVk-nhqAG4pD#-}X}V4s^i z&}4HMN2Hw5`}*^NcJVG{sXTaw8@Llho|LskNk|W7;t= zZtCiH@J&4`Rt&^HvZksbnf7V5FZ7vB8sKL;eS6a2O~g2az}7^o>o&loMUOV*LRosw zBGT6K+90e_Yu;lF!D;FG!3gdlsv-~)iQ~V3zz0MlT0%|-&X9A4-)G6&e@hRT+8qQK zSeF*W2a1Vk_f344QH^i)ti#Bg)V$=X=s&7O2oN358Bu$IF+At+|F&}i^~S|qO{`J! zEmNr8@!f+vQ{v_kTLH}j#=|844cT^eaTpU%R1Bgc+9*kMfxXJb^D+Y1+`g>Ro7pUI zLVtm&VT;)}IV0RT3*O7f(x2>rRM-JP)yq}t9tM9$R=T%iH2GjhxgQe&v!?o11EFVZ zUQ41s`mXYd?SUuL*>-=LQ7XPr!Wle9%iBqsZI-?BVFuH8oF3L5_69s1z6gyl1d}D;ip=*0 z6R1<0w2$SQ8wXKxbUmZ^#RD%~gPvB?S2^N`3vRsLQ5=@ZpOT_QSr@jXVZM617MpKg zY{sS+1OpQ?0l!N(#JcxZ?;2kn-!=!UiR{svZ&^!sz;wCy5#bwDvwB6$9IR&4ih7%y zEYohmse>KXQodOc)%P?Hzh*N7esoHzc&8`5vS>c=(6(=x{CL7&+RhwjrgFRu08jr2!FfwL zzeRz3m$SIZW^(k^-A~H}dm(B&8yifck^b0kI|PfDbUsV$TRrw(ea4p*gQFsWUZff=b7N0bITcLfQI%vnsO4?(lBS~6!O z#FU$re;+0$T9ToaEV13qhv`bAXdSO$1f3Vn7W*TmUMO4`VooyBdh*WRsWNm?QxMQ~ zN+xjL?ebJhCP42P{PEAOynoXV|11B24DvU3(T2RU!}Xb*=lyIi5l$w`L8S-$FgA8H zadJ{}b~G`tk+yJh{%_5>PE@E;*al@su{JTuvxIj z@Zfz#@zz*(dHEP0kkPsw?mbvYGgzH3Vq9G^U-C*w=7iW^HhnK%HoZJn?6w4aKp8`{ z97tp32!s5)4yuVue)JVxF1SdBXwEXjDd9S&~5m{l+Z_n!&hMFDenHVQSV_SUNI` zmGI`uQDEFR1OzkBFqp-{QmCfqD((3p=*Ula! zr{>m-hf(rD`vqa1`*hBq5upy!m?^9da7!``hS8zyf7l$pK%flG{Ib6_vsKb9wVaY` z^A3gKdKx}NJ9h6IfXXob{jRa#gQQG@HkI7rU7R;cG*1r5fUN0?9*TMH^kb!4<1ybR z@3-gEgq4vM7Ny&Ktm_UbK`>Zh4=QBXzk$u$)H5u8_`W53$r^HC`OC#xYIb^`g-3Pn z%aMf5;_=Dx`;??Zf^8=%rYngkiKvh$M`_U^ta#x(ZQRE9DNtO}d7{;gLZ`^hm-^0f z{S`Lzl|Np9n&^;9Av1N6{`QzX#!9H5dD2}?G_hgZYLU1TCcNv+1>cPF2*uZy4&p&jT1A^zGL9^l2DrjCnWvZuN zv940tNV`nU#Ki$EZMlo%HFc%4(3#3vebZS)rF>$5;(Nnz!zFYPV!60_oK$f;M(ToN z7HMS5r?!m4|5_q$a!ncs0)UkH z-(uJ(~AYoJosy^hwMWNiLCsi)JH0!0PCl9X(^R zZpug4RlZdIO@hypf2#nnQyj^Z^eCt}wSmLbe9<7rIwUDPNPm5lPix*~EF4u#fD_?H z6h-q;v2LxcgWd64oc&YWrzvslTEJXY=w!Aoo?ciydnv5-F1V(SJ(@)UWrOHy)T|bh zk(mI-769wC%aLHPlcxtv1jr!CnOGlzH3^K2pbF%WO#%rD%dG*o_aP(olj08J(>8W8 z76SA#zqjDwNfW-Hyf6MX6o${jBx32GxZ)W}(qyM2odZeM7YUb7Lxd4*b}K z%&ppd1H?(t{xZPHTP3sY574pLVI{Q-19LekIlzQ!Q9JCL>sgA753hOC*xfy7oAV?e zs#*Yk8oqdZo*Te?Rwi*W{>-Un3Y8sc`UDAtvlmTsi5xMoMOE=JWk|5~0^@UGrAb)7 zK;o&gL5p9a!w>*jz`pD@$9G`nHIGSXP;zg=mN3}O7riHKAZJkl`t^D zi3C|wz(1PtKl1++|G%0sD+IeP8^Ogqy71|rX^{VBkMWZ29Z=!_djm;L%ME1-?OimE z&4_8R>Kl9%T%zAkS{&h7(!#D7#FcsGK}2J4*VULGY3g70uvoCM;u|Ik3tRP$$zom# zU1mAyBm&M<7A3cyr<1!LY8l>8bZj2@S&~qRygbOR0w(#7{)|1G)lcz%7SldQF~C|- zq#_yZ+5LF{_Vj{EN`ubeMCy?ZhBUt;^&yW)?B(~-)P4v=lVQ~}(+k%i75Q`FGh9#Q zL*px5qM!q?f1vml>HoYVh?Y;FK*cq6lXGk+c0V|xeIxdldL)y}R*g|%q}ogU=DMvB z^^E($JC)8r>7J6Tr+gXbpN!U#x2>}29+|HbwXOCDXpGnzeyogGB3oxJ5y(rFcAKUy zYS8{cP^W#A2hL`r(H76#XsEz2J3kZN#8nQ12Bjqi$769^<;8St$2roWA{}eRk+qYY6@49dH4MuFGqaGq95B$21+rFVl^Yhy5=)u$Y$!o1oMECw1Xdm z7~y6DAPZD<&9Y~U#!li_P2rFo`!~8b~483i6#1$FZo23z>aLY|64;b9TO>)OgSZ)~c-9O3`1jCcLUp21cC2 z;`3TivDtd~*Bg(1Hft#sC|zRR#6y461DU>L07(`XLZJ$8)sZy?!>iqNnT%WYT->zW zaaCZyi+nXli>wU+aTeYhwhl-5QWcl8hoU6BPdu}NVAwoV)0oJ+OWge2)0HdQV(*(_OD+7byG zN{8JR+S51~Mp}r8;%cR2Wd)4w+e>VUy;ejARnY8mhSQ%4Z!;{~C)3ScdLO8uHyI?GDtU`}5=b!n1pZ$Z@tj<^>8$msefLW-BPPvVDBOj0gnf9R zhQ!pseke;B5~IGJ_TbLj{WfUqU9pSsDE#Y-dd&38U3qMP&uHsoewcuR;Gjd~5`dqi zitX6};T#0-2AVF8<9?t_7h2Q&A~bX{Ci+SLx;D|2H)q!g*2(srb`J(^sg`o*-o>vt zNk`&shQ*9361k~TXGyPCqjdeAElcqPT?{^pJCn=d&+^RPo=`yX(2$s%8Bh_+7jN;& z^QwM9LlrG^p+D9e$vNHFposr31$RYY~ zvfR&oFj5R;cEO$D6b4gmwwZd3%6E8Ix0-7%TWW?S(6>fSJgUs0>KU4daNgFk-I^MVxd`=?wMwGM<>$d3vH(*3jpL4H6d@~0;|CKa=?THruITp znvmzjW;$A9eNwt)I9j%<$eu#x6j|e9YlW&oR+gFpG@9^LLGQ0wg-Ak`>5K^9t^zHt zPKE`xP}yhZE)?W0F@ik>+j!E8VupJV-|tQ*F{EOx5@8PCBV-DMWKw)D@>hK863in= z{j-Ij>4`2=YXpX^Siv6hkpNWB5Xy4#rU9&V$wXUY-NRBLq9nC}`n!;VD5R{~Vw^u@ z*YdBK6HHijiz!(zLTuGj%g^u#@{SB1jnUx@Q&WJsNB>|vn7mpcL% zyUCkG?a1a%m-`3=GitCt*jT>WRnqI@VC^7_AZH6Eu)`#ji1HJc7;l>nRZ_|rPC_fOL;ukm{;ZM z%Fj)v6>M}$95|%T=>gx|DzhZGp3;XWD7hjo z(cy=Ovf-6fQ?GUy)*`rt4`$XPL*H+`SNY9O&k~Mq@7^q_wmdS*p5u64JLm}N9R~bE zWb|2To=CSS%_FF7z$);hE{8=LJFrNaLaQkL?zkr5m&6aAaHSN)&=JG9Fdk|Ci^1qi zqwBr~9Hd8tIA#Do^tah!pV^NbXjrEj=HnJZyl3j;DqV#Ul>!{p5>@(ic(8oSMZhm5POFOmv^T9+ALSYStXhrl`r5k2jmyYVX^md#g zCd4gNT~+l@L(N9ysIZk&)i}S}r=z{NGUjgVWVJc|Z269RUTwTCymFsV8jmiC_lbai zmW|j$Sr39}>&-?kekxm)d+H#5^*Qe6KIlJ@Zup}!*hLeAa$|nS#5|DHnV7SXe3Gpa z#Uql|Aw@g8eT9_&?|X>Yx3F|J0;q z$#Bj=|0y$9D8ToleBy_RugMB9B(wmQ{~*Us6|K+h)Tr-8bQKLW3E`9&poQy6V_ z!cc+(%JM|KISga2IZ{pS6Xy-JfFXdsAsJ@=?|gw#cT*}1nebm<=l*1@a2>r&rrI4k z172TmA$n->Ifx)-;P)I;V)~6ysf!G!neh5QkH9ch5@L~U(Lpq$tt;3Al8JO#{{#;C zEYV+!^p86N{TdEV;4$(F)Nr%19K7E`_r4-g0Zl`h@krbvu(-cEA&d6-La{;4<(pO8qQ80=`efTU{NPiZ>Unml)s zoi#;WSDOj_O3iIABEq7nJTOz12X{6btH@Zqo7->$jn8zOsn;3Ce>>+>uB3Rvs*!Aq zGg;Nu)Hx*6%joSJ)^GqL`*nEQH;gaVKdq{2J>pD_tGSTaBExmVyX>9{tH{1{Rs`(b zPH+CuM`K%;%I3@-P;b2IZL@|TKXk`m4pvc;s4;A8KCwl4(3-ufnX8w$U0lWB?eUU? zyQ8sQy+^QHat{5{cG(i^wbLp>BUG|Tmd zKRx_C{t`4nADOF%45)O^E*$bkleSE>y3BD~b797Omt}lXxUmTm&LCfuz8{|&XKbPz z&+Z#i6dj_AarbAl z$HfjMKG;8oRX^NC;riL*$^h}#uvh@U61HSNEo!j4(1hlc8V$5G2MnEZEW46&(_9+X z5Uf=Eduva*fm?x85sR^@R9~XqUZ(_3BnJP_Xk+55@us5C!D$|j6(>{EBc824cCPm? zzn1Jk7BAhu38~lQ8usVG`N2uC(|TRj>>9wALj*!7!?`=4F?$ zU?Ad$c6`fFJYENSYcl(HIVDiYQW@dVHy4o6B(sW+3^GU5?eRpZ4;3(_P`YAFqs1hL z3OX$ti#6Ghg$~TT>P*wV%?_88vL#!ztfdOY#+LdZRl&+U4jzEv@kb0Gtid$^ppi|z&z${U^mYHLK$UK>o*C;T$Tar{n~wX?;C zcAvT7wv2HTSa=Ks+4GQS--D=Ai}+u7-{xeVx$H{Jdws8^%I)Lbr#2xJC7M3V!YQq${C3<>dG zGev3fdu!h~;Pvx|wd@#ybNECUjt_yGdROENw=W&-{uVkp`*oU+f?p_{%h9luP$y(i zRULLKS$pD|1#_h;n&kOsTj&5_*EWfd+Y*26>GJ zd6NeD<1GwUju#l2;#Lu0k?0O4;V7VxJwv}W(oerIQd@qsITk@Ozy=o5$QH-;5-Uh9 z)i*9r%5&dWKWueRW5Q#-mygg&gMvG{VKTL9%2mv-G)IJ;&e0-PBAiy!c-sF3Ld|o5 zPmAdRCB8L$y!XPq9iD64l;t8{noCZg-D7=18Qx)}O7Cwls3-p)b^L#Imc4DiZupZb zQlNl<5dG7N1Q0SXGB^28AZS#3QNsCTi}aoqG3Z!Ef&om+jSj?&jWrvr?1~o3M9|91 zs8AH7Hpj%32AL*mv8n?FGP4Dd=?xOkzRNxW7Mjg>B(*ZL$NR;`MIYbIu8W;{Wu`jY zX_0BwC5p%G368S7Cf%~!?xsFEo)AEWu1s0r^8n+)&;%8X@VVH7%vllJV5*Oo%HVo~ zYQ*l?QEsS~Jg$1e!J-T9U z^}3OHGepoUr(gNJ5`tVU-Rh>e3haOPCEXc9_U$7yfi15nMT7KMXPkG`RHH`gij11v zHv<&0II$>^lzN+{?I_=O)Kv_WClw9B6e*M>jg=YvLL1Ybg}zEiDOzx%R)9GvDLVO; zs;E#q@93Y)RZNE&rb1T^iy=^vVXVrs2ej0nxvJA-x-)6SWx!5!*QBB?>x{yKvODH7 zq*8Th^8l-{kb!oX-RrC|(2OZ4SD1~%Z=%{ZMii?k%M9-BXk+a2K6oABa zjypFs$nL4}cVDH|XR(5(n8ILF}s*Wg#vWfG(YS=oQ@WAdw*K zZ2T$O_%C%u8j_a<2z(_G;4a zVfP8tWpU49RwgqQC%-Hj3uqwN3QyCCFShq$5eCJ%^@OQSR~#$3J1UFWcL6;MO}6BV zOKBngDB#s09(9hbKkh=y@=#)4`0Ty3X2$IDT4b~w4|INoy*pY9dixftTJi7U6V3-exbz<#nKsnw<<5*W0Y zpCKcP8^HYRAUx>PqE|E(2S7olAA6+oCEqz>_Vv>neMIx6+u>mL4c}70iM{fYef^?J z_8MytE%p$1g)KY$h@y%G&o<3y{F)l>4X0*oeCB5>dRiY4@Dd);y=%u@wVM_pc7@p( zA-ek`v`m-MV&0Z*u0{g6YOKX?>6mL3YMTklZuoP#7PGC9%;1oH0wC8a7^lFp{-c!3 z`s^8%UoDJt8kU<0m5;xWW|8ava^`uy_tg8HQ`J@7KcK7E>b355UyUHvf>hN)$)y=Q(=r7+lm2vNt)Ck6X`^nZ zZCuVsTRBmgAoWq&lq!r_vFF0xtWW9Kr^8Lx$nb%@H16;9)-|RCt`}@^6-{NT@|9TO z>-OR0zB!HjdOBxLk!wO)ddz8n(N+>Kim5e34w|i(WF^^Os?FR_p%TV7iGDuERu3MU zWn^N~YS!<%R^6oEof_&}4qhmz1`B#``iOF&1x)Y?6H6hSILxZ;Zk6&@B6e;&{AMWD zEB?Ca#;EYbl^Y)}JGXTPxrx5$TPL!mCB(dN-|Hi~VZXp$2c5zJe!b82Y3_`Q zdbfEjCxv{?Z0-X7-k!H}9WhV5X?_TFd&?`Z1wPs8S_Mp*ho3q(Put=2Svm24Yzg`Sj-z*5v0R%RPLZRc>Ez79=xWWjlWY!jD zZNVZ$k)PEjp#f}wrW@Y}3O7E`>!)|+_18ycU1s4~fapHHm}_%$)N;0%M%UG1Oi&OM`6z7EkI6+|omrJg(E?IP8x;9=}$ zuwix^*A$YYbx+VJe-^R!TB-;1CMy3M%A--_@y}VVCJh1$KBT5iq$c;#k4hLcP)lq5 z@fn4QT4J|uP}Q*5(mtv6X5fDg+Cj2MBit~nWx@0EpE)G4)IL)~BR)l0!_0bNj#M)^ z9$59QC^?q`P5}5z5irLM9L;_{{yvlogl_P1mJu%K4xSH=+4=-R=pO~QmKT3mEQT8o zT56TScqPoI(GIYvda*(RjElTbCeta514$RoOG*e-UTDHVj7%{&(inM~IfMO0)>s+6 z1V2z>PbeelZ2zRNA4rbx{zx>uS3nVPmxsAPl235P9|lY$8lQx?-&rM_)d<*NO-NwZ zw29%MQ!*^0n|R0wh1y>F3Gl};v<-fV{o#}lGo~Cd7ACbi-^BE7r}&(3g>%mNGlEb| z|2e^ynm36ysjf-W3gx4rQ-;`5g9@&(;RXF$pMiJio> z9KOdb6u;MG$gH^j;VrWA+o6FhK{5gV@NYD7CV_x{)@yRFYB10UVpp z|4N+y@4)}R<>P;i1tfVXUcyDdKY236KLN66+7DPVD!|YG>G7GGh}pTB+Sogo3Yi!> zxR^Tq&xcWc`yW5$A80NyOHh-M82DZN7Zr(65c)5g!>}?sHVeqgY60!E17XZR6n5;Y zJT3ESOXc%DK!9MLhlM#D!Rw5r!G-J>Md`dVyS4w%ft;mNk*Te&r|f?Q&puCElLi35 z*DIdCA+MDv2R}j~bgPF;)}(|Dzug~#4O8d1NniH7x|ifhaCR>kcta;SwtSpZMs&;< z*9Zd^g!Yl49mpW8)YYoG(FQG3~S>7uR`p-Nw^9MPmDsml!STti6$ zaT%?2f`@vt9ZLe#b&CnqeX@S7$aFsDG?`#T^w!OFB0a0|q&41275tr<3h|d5 zQAO`p4L#L)Nq2m4MvDo3vAO+eYTxyhiT^kz=ZTr}>G)CR*wLL8P4%(u6$rp?oH>>` zb=D-Eo3tf;xeg!yiYa|4yPiOUNpMp$gLk0ojs`(zdLBycU_@yX=^&RPr}|^hmc^bn zqeHK$3{6Wx$XNensgPT~{H>5hUkcZ3Ou16Ne;~TIqIG0>NOi5m>}&ft+oF|i=Jk2{FYt+HAZJZ3PCU@B zTv(Bh_DqW#fuOl9sZpS{?joBm2lNJdEOUOk^~nZ!ic*4D$y)SGT|*}CTXMBC7VnWr zH>X{s%pQbGTvS{xc{NhZD?SeQ>}JK!qSTvWY>edTXk;>|*>Zl(c>)wvR_e>3OSkmi zycudK7a=~U<%P0M2sU#Wopp>~eUdcX6CbN+byUFN% z26T==lBVEthK2GAQvyKETEyKl7aG3bhtBLfHA37k=_-016()gOuJhYi+$u1@n2=%t zS=*}W`=6cxlEKBLPj0V|yPTeRzywI5pM0H0w}{~@3^#*N96ND28-m1cRCQo4Xw|PX z3N1-|rsy$F-{_b_KbT`hoKTdsqm;~q`@TVenGewVOoe+s=Ky}Nz$%)}ZBOO_OM#`) z9|$kTTx#)@hwm-dNuB<=CYyI8O^9O!~*VC=*9as#S(6DvJOLCDRJG3UDU<8 zUat`PR`+;>G}rv|a&}=Awm)5bC?&eVu&p2p4cJKy$oy-ns&ZXTGTbtqXY)4bqF(^26l(_)hC!ST8!^BsZV6Zs-Xvt&ejZA~)_w%I9S*-3GX)I`SO zezBEzuvcRY4hDXQ3m#1a-Magv<_v+wc$J9|_c-gle0R$<=rCC)M5XRL>5RJDB!; zl#bQn4yCO-O1kbrVH^5T#y`ITyk`3dW{DOgeHCYP&PbHonO&bKkUVAhR9Rfp!WN3W zPrK9mLHD4DHfKLb-)E;H`*nf;eE0tE_xpdJ_WzXr3Nwq(fB(lKAgvyej3CY45gi_2 zXJ~8s-!I?)tJ+9O`e$(~gdVz0{Q(>&#Sj&J7p$6L=p>HNPmYhrL`y7)VAFNr*o?hQ zG*ovtmwHy%@*u%wh+h7r+RwNQ{Y@?E*4Su1mHU*za(wardB0ETkEJmvbMt&k%a+|BzVU|uDE9+k-Br>jhe++dto zQYZK<@n&i%Z`b5bvK=Wc5o1x7p914V6Ce4O=&;@o(6OFW?&?2`f@d%I^aPsu+Ru^S z*VN>JB?pF77TP_11%rrVttd$El7F`%;u1TJB&0)Ml;6Vh93-3;-2vR(H>y2K{3atX z#X%}bQ^nIw_P5k~iOiaKg@n_MK71&BS17M>7q(B5Z5@uQ#wPd1q%i{9x37`sRKPtO zU8PWuZJo6F)e-_t0s%PP+clO&J$#M2j#;H3x0GKW@h)$|$nW|{YIvTS>?MlhyX`UD z_!So?H0+oT(?j@IGXMkEBz7iRy+(r2v9Eq?3h5&Q9o{t2wWKNe6R#~snNY}OjZCjD zJa?Dc+IRl}5=#z-)NZaFp|B46f&rQ|u*u^R?hwEhdx^>H=|auaOEBdudU0LdBu`R9OwXiU6$<(!~OCjOq-G%E!^@ zN2BK03z@;Qq}Rc-YS!CKOk?Fk*@uP$T$EW3(ajd>)J@?Y0wmo97H9%`<)6YG;?p?H zg*&?xmn*r1fw0VB#Sje{$<#z5Q}JC$@e0f``$G7SC~s1Iz!d zO8lS8NcitI{>wG~zxR}NyFm#0-*~tH!hajs081wkdpj3X4;M8TOB?6^lXIk~YdO!W zqkp3FBwSK~$s(i|r*D?@nOT3MG#ypn$P{D>fa;)bfq`o|kao(fa|xk$F(|zhF?>zb zurNnWP5m1YHsw;nGST!w=S%!iV?!LJ@_VTJ>1@NfyOHShiD%jV`${wexMqO4DLw!J zD1L!Mg!D4=D;Md_VIb_-f`m}$ZV{A=1f@j9G!AGeRvBT^>MU3h=j;U0(b;?0 zO$XL-#e3L3ARc&xYDl~rq&D?h2_B%MQ6-QPmxs1*-SCJN)_JZY|1lZvBhyv#aykiB zHcN{QpxNwlt4rdQgVsT3xanJ+YL@jvGAVt4z%yH}LqLIn#M8w(RMH~OmUgZLuPxp< zB05`Q;iX8RJib)ox^5{4%fd@8L+Sn!I}))KL>=j^klsRO<9O(l zm%de)PnAoUjK+@T-eAjKjEuFz{bRq*CKPD1-E8_*_sW}mZd9C>c<=+j>Kf{Mk%J=BXj7= zH)efP!||0NC$szpLGovoSO0u&@+bKLIR2b-CI*8RM0ueub3!~yPO-nTsq2#-YHYRP9KSQxj5MnnjwTF*Dz)b?<&Oh^z9DX3~&$ zgysAg%`EnzH%gZ;GKd1q;6}w-wVNC~XyIH_d0<^rdk|rn_yBiAdRhDjyf$o6*D}L0IhgrXZn(PBQM*4rf)IGk#}}NE z>-GW=r#@*@+w9^>O81+miU!OQHK}GRy67Ui90i>_qDeNHV*&wYij#jVY*K}GNGgMy z)h5a+RF^JmglBOa&EJ(^o86dd(SGZ>m;r_>rg|0G{huuN#T#xw@) z=i`@M%`Hj5X`2F2Z`J-t79p%F#n4jfznOq@-_h8{^6d?SvENynKnl40q{Va&rbO=f z&1oj3HC_tg9^rlY>L{tC0rd&%i4_EtVZ?1h#A@ZVG-xGPN#;aA#)I zr+zVnCYc`svu^ksdJ33g+#2K28QuboH7h@1SKzj%;j)W-w>88fs<3Ncw7G(%q_E05 zyZ;Ws7nCErB0fhymngSTsMs*OhzZOUvf6+VIAj2zSmQajz@?S@LVv6T%1GWV{kjvA zS^M_%@KfnIb0}|MYY=or;R^P|yiCZkzgLsfgv9?5-3#Hhn>I^5dXhLxw@;1Z{gR|f z>0USJEAMEVO{^@UHssBVYBkc^4^izwTw6NvOO6pDU^Tt(@E-}lJn++-Mpm$MVum@f z;LnrTY-z4H9g6cg-VJn`f!G0Ma826>>D>BAZ@`C2|JJQzTHQ7z=MB++lAr$#9smEJ z(rSB(==L4L?LQN`gExpNkU!l29V-D@R;YB}z9||1D}bjsBH;r56J!3T{Otq#TXpc` z%iZkn<@zQlyG~H@`|ea9oj5`o2?Q*$;`d+?5}5|UpQ{5G;z&R0_SDW7wJ~flZY3*M zNUg#P{MXi&tdOf)nyqZDb>ayBzN*n>ejW2%UydJXTYmNAja`5|S+-yF@4Jq6@ALQ_ zv%^a&s{H_od+sIAd|#g=#9KeEGymYBTt_^CKd{|O*nOh+b_zhB$lYRH`Sy18K%Lz0 z*;DUb?-6ahl~R9(m41#h@S_tsUK{7W+F#}DZBdpyFHmxWvudJ374L}e#i)XvCO|!b zHqtkq+itU%e}9u&n_peE+)+5z=e@Z=x1$Y>ACm*%a-4#G24zk7Pb7k)yP{GHSHhgD zQq~qHIQ4fv;7n^km)}g5G*cb>_eNeB40LmOzzu(XNMz{=h!0qWLs877iV!(cL zkm-*w8pErO7w^hEF-hUKF=5}D-msO71!V!K-LM*i%Ck9`h2c=!sQSGkw_%Mm5@ixa zT_~#%ouyuu3*r>49k&I04cVAdad&8(r)0Pt)dTY^-499ehQlGOEh&J2CKN96Bx@7(hHV6O57~$=%2qeC z`Zn>%`b?9jbDP*q(cq(866vm{&2W)AC_|MfB1DHYKU9i0pTEIVxtOI+lL3%ULS=fD ziGhJPmde1Oqe5O*HvGC)B57+aBg2WB#VUXvLrxr0{8{ljnV?wJOipXDXUUM=Zg*uv zJ5{Bx9}%Nblip)^m?$Qbs?PA^=MkJVoqeRJalWLDT3J3D!nlw#1R`&(&12{tr0F=v zgdW#F6v`c1Uvt>}Mj7xyQa*W-fqlVPf z86+WbA=6!Mhhdd1RcO0XhMKtGv?g{tgw`q;Tbo|Ve>f?c1?B1KDPT`Me*-7}D*pQS zZuLRoW}RtxHlFJVy4}lrZM-Z1uZgqsJb#``njWO|)*rU2Qfm$Dea@Fw5fu@|)pf2D> zV6fJqjxe3tZj^R^!rD#rn6MzKlHj?0DDmP3x?Hd&OC2Cx+pPa}dk)iz5kF!NMXhyV zSd(3?y15Dog*~_%lV>4R-cCBf&O_J=uVRp6w^+ly@1f#4ixX5)wHq6i0ilu)rT@gzKEFz!r7 zuBbwyCCW3YaGgF$nXAl%5?=ieGCVYicf^0mD^kK22KCBt)9}~^p{UssI~#l0%1wDH z;|L18BSem8$}FJ$G3ytp>mv!H?l(bo>+JKCgtb%$POHAOg)SXY+yO+TW8NcxjIc7a*9Y#8?sI#7U%;=Zj=TH~xrSP^nQ5_bBWw}`1DZ=YVJyz6S|M* zf4)sR0l9EpmH3%_<_+ph_d73&2EnA^;NJXn4?rb*Wm<@ zLt1zMiIb-S7dy?os~H(-7qmupRz5ZRUd)@#GMS{Nvuj=uDCOzMHfMWlgfm543@XC! z*3f!_RT9xUOOL3a1{=ZKQN&VTAX)nG$n;_(DwH)7%v+Vg9>*z{#w_Lt%?hQc+c`Uo zkjNV}L)k5W=(wnbn+-&8rgWvfxgd%ukBJW8LiAbsv?o0Xshoi#4oH~N$x3LzeCey> z2ZCQ>@&q0l2hHnjaKLdQNQP&f3I5&_l7W?AXKP*{{fk86!EDRlR7Y|a7Z2w>hM?j% zoxII)cwh1!KyAQCzOd8){?bOHPH}%ex=>yFv)po?qFR#rVp30sjdQ6IYUig47vv}a z`)VCO)|KGMuW->d4 zZ+g}3&qP6P*h)ltV8#x(ed2kU9qgWfKPj@K^{jP}rSvtm`s-4AC~ILV65z>FGzR#fJl zMBo(F3ED4un+HU(bMU*DHbIlFob6$S>U?Vfn+R=VH*+)JB7Szf5ey5y7~AqVw_lR3 z=2#k-<_?yE8f4;mT%Ha35e|JEjV*m(rb#_U5{>jij}1X314scCCFN^hZ zE}Eap=LRYqnrMY2R`R-YROHan3X(aI%%^a-*_eU$o$E^|`8K*U9TRfQdy3CE>Wfh@ zFafsyw}^;`b6I|ssq-)M`x+|9n6T8tb5;iYNo29JyGvsZ zr#irK1+^!>r$sav_9f=Mnw)09N2S|#bbEALr$98@G~#q4ds42P6Fx!9h!+>$9;sFH z4*8BiKj293C6{~~n)pQ`RGrRD)2P`W{nKPrAc|{ag6+`Y*}a=wARf?su-*33`5s38 zH9Glqv)^-IkN~Z{5CILSm-VxVpn3Um4PLYq&LQ{0bRw%fgnwxgdG&2z%Q;-?tCd_~ zRDt%T{CyxfCzX^O;-B$&IG$e<(Hm^t9$QVBt$nCH;&RH7!c-pDFN4w~$kaq>$Nd0h z%sb8raX+xSd1T2$inX~p7DO3O40Ox1wgpr?(Qt3YNIIiN@})?CYxutRnTl&vjZ!iP zo0SMkY-Q?Qg!gWm0NF`vvH>Iz;ep>C+L@(_==tg$-86${QQ2}&TV|tljxHsinJ%Rm zO4AAmr)?FFYg0jzI;I{(H=xZ!%ZAREQs_kYigw+oaS>2@ww$27$6P;*n#OV)I@Y+r zXhFk{Gi4l;IyMX#vI@qvj5r@eOl?z(5q2;wpRFR>Zd|}-TS8S3j9B1Q^WPaqWw?74 zRtsjFP##pIVbcehmBU(*!lVH(R6W-Eh_TnsdDREwf48#$;jK_X$^smu>=A!IV7086 z>+-)0JDu_}1Hb(U)9;%#c~E5y^@8GqvR216190g z_dX4XL`gvVXYa?C)v}sq)-OOd9$kA*I6(3snF>Qme`eGLB>AvSg^2UK)%5voz+~)0 ziJ(C*S62jC>g^r{8Gh=sqrKyM8*7~jG%1@D|&Tq_XD9Yz6S{*!Ow zTLxC4e!SsUau+(`NO>4D3{2vJiV=|A4HE+RE;plmus2)qBSaeGVvWvW^86S*M4T^rc9U;}R^}+AusLE^&wbMzY+aqQnLiU{LO21ACTC%( zM&Kmtc?&MNC>A)Fbm8jRu-vVw0=fe%e(<_u)N7NiYq!m$ZhgRcHB<*pbXc4Z)rh3& z#6Q@TN&rvOX|uSX05Hs5=MA7~+~68n%SQ)m!*@@?L~imRs;JeI)Sv^z+2@SKuo&vH zIEsZMzTc~Bc1zna0(~K`CxNe%I;bV3ixOBIwldg~5o{^L)_*}^dc#lX77GBC2Rd(F z0ir9Prud?h)&`o-)$od(j@cn32Q%{hS#wO`M@VC181*#ryseWBN?MWNn26V%35fz6 zjUDnwImfW%P4X>$NP}9vNQ@>buy%g4DkAgjbv#2goPCZs zS-9GTIvrN7d^-S&XOal$sYk&5n#KJwMl8=5!#{?Vk}JzctbmEwoh6C27@pFM+W4u8 zQ~LgJiJLC?9NDgI%Bu<$zjosr19G1DZ`Po5zfwftu`*9fW0J@b43a)vT5y`3Hj-1n zBTY`69!(O_=;s_$mSqc5-9zoao9U&kNuY>~Xhxtwa zNOx1PvmE|;YJ;}UB53?qDxF^UcM_jG=|x{|w{Fr=5oT@Yt1@B`vUjhTstZ8HhY-K{V>cebYi}ZEFn|1DXpWJk+A@EG%=yGwv59GR7?%f zp#Rwi%>zy4yVH`M2{Zu=Xl&I)ljm>qBQK1va&(*@)ZAN9dHwC1HOenRlLJ%-tUc`y z%M#V}NgxOwzr@jNwrD#kp0{ySiLFnxbV{~g2p!EDM z>-26Z{#y{S4~ac>#Z|B>d9( z^SsUUMf9NXfCY$;t2FYsC&{h2mA|>j&mimwn~RILqMVjuUK_z*6WZz@AbJA?v&RU$ zmcm;S+ZOjde_$)5cBOt+GjW9oP>g~dQs5kvDx9d8CpFe3QpaV_?Z zYZOJsx+_X6q^D5=570tyxT!LH3rrq${cK64r|$Tb_9_vfO{jCpMX1J3feRvh#C}pD z_$JdRUX{zyeCLZ3_{Pgs_}5pfxv0qJ+7SWl_b{fL3dT+{Aff9VR#sz=nZ{x&i>7)1Ow?)q z{>JI`)uop(c6V|8Kha3I>5QkBAVS8!I0ZR@ya17Y7%w|Yh885R4l3Xr$y5#S^XLnd*+edmiU!}pw{k6>6Qc*e`t zxNrg52x3(y`ZNMD-dow=F{~kHEfq}W)~!yz4m=31V6yM z5R8XKD^mL@+8V`Jg~$7Xg@`U609QbtQs||~gjkzg)Ixlj+hlAyNxbIL0c?p-1c`=dTn92;{8};m*rm_CaQ)PQG2)NuiQBeg;f>#ezWY(F`vR zR2?2kChDxDADVE1#!y`Vwv}y!SLC^}0^q8A$e<9>T1TQFV_hDouQ%VfWaYSD$&Zht zwk9eoiwaK`7uGJez#m&K1LgjAI2-aE4mn(zBKoYTlC1LpL@z5$O^I zZ9o(|iD4ewyz09HrI0%2L=-YlS6n?BE7Dt6ZOoK+jmtl7(i@LN2bh

>^Djtnq_= zbwSXn>CPxkrUU~w4R>TWJV@&h{?yWu!i&Ee$C|1cE%PEw&UKQeg;T=i&qj$G-XhPu za2%y|erb}e=2I(x5QP6gQk)z3HT{OY1ODzCPN<0WqJy=tf{j#Bn%!u5ZFES!8 z(}bj{5E`^ntVdO?dJ`z!yO>WNJXKZ#rx6Akk(HG}Vz2>?=D)Zwic6`Z<`>JVfdycy ztbvg*Wh-zr^sfRXf%{1~NErWQ8qYt|lf03OiEBb22>~OYIWwlUE;VFsz518IkK2b( zIGQIVm_h)K8N|p8BxWHdc&@BDD&kN1!L3Hal8Y|NgqeB712r^n(Z5Cp!UNXfT6Na= zQE3tFQ%0JsUHvEdI5_K#6qkgVY@Sd05ALSQ$5&k}&J86O;-Wc8Y#1N8`WM=`Kf7Uc zya{5^Iewa;6b4+ry;931TWjlXvnBd2^SVEYE=&VXB3|h>m6nM=-glEaSL7|D4GCB; z^9iGRmV&Nsy>8o$0vbJH8a=|#p;b8lay26ElKaVY+^OJg`NBRR?)J!4eBpeb$0$X{ zv0CC?QL<@-NuAuHxG3Tl*|epl6648P6!+WA*m5omL;Vwwetve z-+lmSq~cl6uWC9Dje8VXo$IF3q2NEZxtE~bX|v8WcT>9BXBAN{FXddHmuIP5ys%Ly zPiW11f9I{_Ugy+3lk<4U?Oj7JdNK}A3UGW~7PllFlH!d(-=UWNK}jW(T@<{qDa@C8W@^@qqrC$*PP30J1@_s}5{T8U$Z1m&89+q&EjXeXdvN*>FLG+$;wI-3gj2tfz z6gn>*5EM7KWv<^(@FJhzg2z98A9+>!^Fp+r`^9wie>T|TZE#B%;-)pR6Lw<204yhYsx9sHfQM=%TQ+mH-nmnR68= z?>?*GK6AVUps4Nu`!ty!U2z5rnACBeoo{jykhS2c)?Qe8LFQGzE{zbQEbmwt5qDaY zuWX$4OkT18Ev0%i#;|z^%im+ShSeGAVfjlio;z|I%?D{KZFwGijnMT4+lE%v`ycBN0uq@+EcI!iY{4B>=8J zNxGUbUXCOu>2PqZXBQ+0rnV?8%@%I$^A}E~tB3N9<2hzwJT1siifhTNRQFK9+AY8N zkCV-IriXqV2`VJWD$Gy`RE>sssR#giDmWg`FN?3(PUUcX8p*=t{gHPzJGsb1YZ4*( zpii3tw?fzZ=!pZSQL<~^C|VX%{3L-#a+gi@F@%FHU^?cWH=P^Bz!f}2Wr`vz>6!YORz%xyGMKfTa8`}+T4f?~fZczgM zK_}Sib$Psrv)#afx>8u}EGs`F==RG3T2E>jaKcU9=t!nZKB`q@f>S)!&Z?O@@4yvD zF#^Y}GF9q)*zVG;E9lyQidz>?pBiNmrYo;8rsfh;RbYJx_Zq1y)jGQ9qF9svcA(8o zvpfrb=i`OOxSfjE6rO2VQX?AbonR60sS6*5g25Di!JduaZNu0wt^Obz{#ULQ_qpa5 zV8LB4zxy2SQLkD>`f15pPI;u9!L2f&f8-1H9hr7RlZnZ_oVq0^5bQa^L0MIL9 za6YYy_ckF|ur1tGWoa*dNvQ;3Ub8&{0C#|y`IBK_cSE|;1l%?y#-%*^Cw2)RegEG! zg!Mw4U#@W()#2k9Je!3asb6uDWTVEBGMpi4@8e%lMbG|oJ%ri$dUhUstL2g2VTAc~ zI$pWsOTj3sm3q*pi)q}IrCfcU#8*{mPr-FOV8Xq(4Sx0OAv43KLcR*v8sz$zfSZ#4A>mx&uIUAUQ*95>4w*#pIO|XuayEy|ZPsrR`JF!s5@f#1s zCJq+OMvGj-J1**peW>#Y*^!hc#m%1N(>8hZ+^$_#=qx**LF0C|fe7qE`=T=~f+dn6 zw=d~MdSg8sh=|Kv`G$2PfLQyE!2VGEzWjGs)?IR3Vgq@_86Y11P*BA}@XpPhheP

_s$+lW)aAnKHa+ zE*^WcyAh=l9(SVkXlc5^ZKC!nDtx=o`5W|V2EyR}r&)@R21=Pe0QiBAO!1NJ8H*Qo zjPdPxoI7Myn(ch9JH1TG?Vy3}u;eev^TRxpg7*;ILj%jwcf89KKRZKM0^dh`KcnBHNqXPVw?FL!4rTV%kdDDA8RTh^lr@ z*xcH;qoX{M?Q2bO0QV_D=;k^iGdciIt;8Ay&uCn=^DU3E)NeOqur#8qvpLV; z_q3};H6j`Q!cBc+twja~p#?Sm6l{0TLCe--uyeQ~#5{#(1{Cb0aZ{;VMcbDyRdObG zJ$F<)bMxETOrFH=>Q`hO{G%?7vZ2=Jd zw`TM&QSrEZGN6Q_TZRv;Li)r^bBgcvDGPW)Ot91g`-DJ~2BL`zEi+6myt8b>7202< zqERxo%*B<6QU$A44r$WIP^!jNK(zWkxFEWgPciRm1-DsNGB;Tf{I|04g#H1!g5^Y? zEv7r7ewfisHmBOuk&jaw=3|FBB<^U|%dxNxR=P2~0`TbThZ7ck8g)* zFxv`0tanO))fyh5?KRwtUwl=MpK_&FsbcgM2eo$`sod?~!(3W2U{NSkXQ7eer5d4> zoOG#F=c1v}@78Cd*Oiu(%u! zx9ms+1aA>tQS`kzzNBK1RfR4;#4008A5oNM5Rj+u5pO~5*s7wUj~Ke%V%f}jq8YFl zYrB_A7tG`m+SuYx!DFp(N;P7P^cmbB>&th7^%U-%;|%h*sSApAu$)H;<2m7mpd)bM z^WtzJu+p1WvIdq}kCxJZV>_poSsZ|u;1>XHQC^jP_T;tKmxh!ny!#C?bs-c@f-!HG z1J0jOfFxu+<;Sm4s1vEvY@7Vc{*rCzy`2Bb20TKxae43j(bWP702N~$iS4P)g zmt{IsKs&jzhY~@h+8K|c6K8tX-Y|TB0f)P)M=qf_M3e~uo`jdy!Hgb(sr#yRK)Z6- zDBrUH|9LFZ>zUN6HlaTI5inxmCXlC+sKm}x(v_&V;>|MB3f!Cs2)_^nOFi#}wf(r4 zg|xk7JyCq+J3h*^%(MJvXIQI?=C(p|K}~sPkd<^i$JS-gaVH;EMPG5piG41m%A<8g zB^M~dpfYR`!5X(xQX=WCL=xhwekz&7}C2@%Z?6z(x0ZqmcRAvSIX zwEy{$HBJ9XZ~=Rn#<*Ea$+&bZe9tx0=HBW&=7VJXu~&BD;Ff-FL~u+2Abs`r`k9&Q z-eI_*)@IU3&k7nQfd8v=tbJcxQ01LZxro+qwV%g%gNxnFL&f)mB;EI8yt*@Oh}>%T z!;#L$SvITC1 z`1*wEmEb4ojfy5b-81swBWal%N40*>gyHomckIcZZ#7-ov^6iKd*j_YT|^8Ftn%4I{cxb*hT_r3df(L8bt!fSoPX~#BC0)qevK$Bj8f;yH2>h61# z)oqc7^mhVJHh?*p4W?T@vXlUld)qMxymyA|!GUKo<-;e$0{ySbaNIsuZ{IxsGxI%u4Hz%l zHY`qK4OsXjS36rH!1)d-5|KO3vXuCkEu+bu!>FWX%DwGt=}MN{)1Tt_V6(bMI#S_4 z87yaDyvu!cJLA^Xo`TO=v1na^Xy>#V{GX-#w7-*^&e>lOeLQ$F;e#7=dIAzFvnLcd zzrzf{q$Hmp^1$$QN7_{r$2*Dw5kBzT`4049fAV)CxY~IGP>qAw$k6fWh@A7|8qnLA zeQ6KEtK4@+n(=>rMtu4M#8(bm(`ngLA9lq^Uuch$ zR~i=M`=GZEfIE3pdx&%d@3PyFw8u3laF1q~-;T~YO}@oQu-~8&$9)_0+OI4aYLl^H zy}~ry2kLa@NVV^2y@I=#;nBH!rOkf7wS53RAlbSPeTx4uzd&D%xCVwj=+I(J=MQ69 zX6@?L&F%!`DdPZ?Sgmft zf;C{ye2p@m34trcw6*jX28hUxCIX0Ke}VTf0HG%;@44Z%GA;&|yk{A9@SsaZS&4sq zHaA8q94Y-wk}6GbF!7TT4tMq-|FX?Jb7+{gc@gh<^XbJv!@a zpuZPj%_e4v<*+8!`B+Q$w3>|6E+ARr7#@ndU8FSBY5}@sJn#ul#f}r|maqiBSqaW5 z9Aa8CW!5Mo?;KjlCRN=H5PubadIBe`bG) zzE&)`#b3GC!!`7Y8^Yao?>8VaJx2%!e|6j|B@T1B#xVTHqLl}op|H!pPmtJTi(mtu z{H~59@Na%962Aa;h55RFQgcfWcl-fg-}wZt4-@yG*P@v|58&;|nLE}b>3PQM-8y2y z=0)d07JsPu;5nw;w@5x zdt?l2WP@=t$c=u%ZFj@rAdh#--tHj%Ifh34Dc-P$%DlTujo2B5{o2^s%HC)5xS4)E zqXojOsn9ZM>Zl{7_@1{T1v^Wosb|tm>El^i?a-Ag=O&a-cQ!R87$vQA^*z9VYX4SM z$@DyTMNa_!sE|~TVX>;cv71%Ngd5r7966V%sVbi3rThTu)O8A1#9{q zM~?P)RK9oUIizk_T4)nXI{vkBN+ZS5%i9fMuGv2na0?Q4wv?u0vZ*d zvFVJahW>SF$G>*j65K=`(xf;hkyK(!^Zh%WBrI{ct&oh6Vz6z4C03%`hE4n)^s|!K zVC^mJS3vjkHzXBq?*8YCAmmu&?$IEHv*`Wjg4m#3Z}-Mxa%u~nuBVIkb0x>^-iaEAd=(({|{I16r5SKh3$4a>ex0qwr$(CZR_jU>e#kz z+xmizZQJ>?cb!vb|L0=et*Uh~=c+Ml%<(*L{f;Joc7i3}?}5;xG?#Qq^VFpA5GX8a zhw@|>O8$A7f;u7+V`OC}mrT>xqg2&oI8H2xTNx?_d-`X?S4GrqoLRVyRgzqKrplwe z5Hrn%n{Xwvg0vfw7auk0tBmGG-=awawhZ|)was(+`kZ4%kZcjYDku%cN}N>-O7=WJ zn)qTkjb*@qNjh(FI+wRwBEGd$$sgA@S7m9Ed{E zPM*$OdBw$zeU*Hc`yR*GT^51JgrfNX@-pNXnc`B-^%yi!0cn%xN-w==9I$yo5yZCw z)XkJVIq)E=bVPXNiL&-Skfm5ntJC!`Er$qAV+(pROc{_fl6M}eWd95Xbz?BCCMh!8 zjT^!@{|JxAxQrj7cedWP+NryVc__ponZ`sN^?Oq zH(SbDbYfDz<5x=G$m&W)J>0;=;aq2HNeRqB_vS4?X|)-))T#qhElsXh5&F1PO}^!g zQVGx1v!6F03t?Reufc}hhFCxK&_!{KPE&K@M2IS#(n=H?;m%qp-6j6Jx=di}t1**R zQG)2kd{tV>3?s;A7CF;%>lXrcv#AL{N+L|>u_jU`6WtWdLsz!#j!hq-wdRU@g~np< zp!Ej9!56|Pj>SZJPs_zi4A>uQr!de4=SB)sP&=daz-wxQ#jnf%3`2*Z>hzm}*X-02 z)KGhX+P^Xm9JZi#MpHv?_H{sS4yf{Ps$YXtQ#~URPT+m^9YoDG%1+w;*epfF zQtsGl)FPsAF)$3QLPG>p`EZX!=ciLV}n7TQEPMusD*4E%9 zXU?NUF}I>yG>z7DTu_tCuo5c+aM=*_m7E8EKEbd!j%aEQ<^d4dkGjM3!_9|jj8ust z_lR~q@`dEFE+uI%(Y{G}C-e!1QCO9ONwZb*&`eKhp+vZKL>qDDMq z-Dh{@&YhfIgZ`SWy`E&Zv?gZ&RHozd{~|!0wdr4*pV`LZ^mQb4slwYQ1!tjXkCWIGc@%fQmSaz zTkYQs6a)k`6J5GFeu}?QVv>Rxz}^AYHNAD3Wg`OQeeWKaU`zP8#-ul79P2C>g!u&& zmn|EfJ^1V&S9jQ4@!PK0*ePTEY??xZSbAvi?}uO_afF5!JH3r}vOZNA6Qe*a7M$0qq!0k!+Y$#vHY4zJ7#34(bpQrJMX3 zIbcKQFXqm2qGW;G8o6Vt3vx;f$7DjGS~!R8Rq(#`8sdulKfVhS?rjsC^N6{$=EvDy z3kUcL42jv|e0%hVSU6!PDvpo>AN&yrx%8hUYw(TlF#qA0{=b6vzZ}zlN+DAA7@haG z_*>;0rfEIt<5K5a>NSt_;|J3>(}a@Z_Xvmk{}`Z) z)Lxu%%(49VwCBff*}6|zsxQbaW#UcQ;Vmuu7#&TyEfb%Abpvq)mga{yMK)E{D>#Y; zCKO|m3s5P3ubUf&5MaZb=Q6uI2NOK!Q1Dz`YB`xu_9bAwZCrW89`W*jyao1rf%ZDS z6Zw-kXgOet646^ZuqGmh0s0esHAkbt*c_Pq7K`I%pgkq0`IaL{`RLc^Nw1mokZwhZ z^pS5(iS&_f%j0UGU(y)>x})qton-s2L|ee?DzMiZTevOBZNkmj>kY8Y!6z0U{hhkI z7-fdLh8SLZuMpFWm*9XT<~x*(Y)-}Q@?3M9H5YOiu`-inRQRi1Kn@iKrD2?}wn>il zA)7P!0g!Pi=enC5cp(qWq(pG8|!lX{tdtaO~| z<|JnL&w=(|Xeg5Xl#yJ_B`q}c+m%|g=DM24C*KbKs&^;G<=*3_t5oLeU$_LKW!mPp zx~#oWbRH6S-(iXnzTxr55NlMl7*fcu*n4qOU+A+AxTRY*fJow)3AVOXT)xLNg=eIn zm-f$I(K3pvdMj6168m;6Zu?D>B2$n09WA|sLf4H}PD|A2%mZ607J9lYgDvalypR^w zcZoh6ZR`!>eF6rspx(!c46(fJ(N4nhLQ?8e0t#5*m^eiaya;Xzc7BX>kLI9BkJ&nd zm<=@4R%T`-Ae^#c6iNgJR(Zzf-(#h#^FJz08Ng~Y-puuy%4%;4J?7)7gsK>eMv}(f#L-ta)Tba+gk&t;i)+gcWrG0LvOL>q;y4qgJ(JTU;I_dT^bO zF|Q(x1L^&i&=YaF3P;Cmae>5?&3CeVjbh3kzStUPL8qmsku=2RM|K2Gk7BAXLsFA`wZ%tO|FvGD)pJ&iyMv@*Vf;$xSeE6HOplAZ#AkMr z%S_(1`6pk2&`vyr>F7wK7FeWpbpsNg+gOwgAW}o&88`Cx!ksyuG%J;B*Exb{Hu6(p zWBOq__yR@R=lDp;Y}WeRy)*Ul%G^$*qV1sD0^QBBI9G}>jCS|ULWh1`hjD}V{o?fS zj~!qjH8QKMD*1bk8LpXU!yrgfO(R?mZw(E`12Y=u;8uwEvovdf+=&VWtmAT!;b+4D z;GK01&Vrv%cG4H)E2n>f=Ziue}&fl8W_Son_;&8EYo)at}iVyJ#Lvm2YC> z^A8@uzz7HcEfw!T>xi9$QxSwI#O>!Z*ptzvltuNEyhV5LbRK}%38w0b$ZH4Q=ql2) zYU!QP1?8n+P>*#>n{g{LVP66BsYrz#z-3|*v?TV7CNc-qM+7W~l6V<^?l5Dt&Ck(~8-`cv-O&n|T=(tX}ubrb(4W znh-M&;g3IBT*VBZWsPi=q_M%i_Kt*mtjp*3a?29*{u0CL03%rzi)|m96L$y&I7FQA zMI$&4!#NG|j*Y$0Ovm4OT8&;m(j2zx4@O;WVp;584WMpBhBGJinvQt`-T9<_N8W4A z$_}Ck?rB`Ji10PD4yd%o>~k-0o1k_DS-v2Rv5Q}x^iRWHCcc{{QjCw-hkU&Qgkf#9 zxYN#Wl9QoOTF?`D8Bp`ZlIcnSz(6+{tYnm(I`Yy9i6(J`sTheg8vQv%$mMKFslKHW zEZ(h75LrwpE24S%YV)j~Hbjbhq96|tNJI~}ry$;4pypQ;ZHj&V9l?MZVs!LLL8Wu)ub-SBYGVNK#xQ2m}H_V@%?UImm@fB9^D zzo10@C8hgY~sYv zQY-~I^t(UQAzyq0wn+W{^l40deLF26fBjD>#{D13#eeEJC|$;gG0~47c(VTyvg}Nv zVx$;QaDo57K@D0k-|X=sAM?MBj(_VxzYT?xZ6t_665w`e~k&LRL7>CN-}(4h@F$Jkc(r20W$o%NX^KRS4a z8*o{kZQ_#F3>*Zf8$=8Q=NnNv{GDAwj98}|OpI9TU1>U=jDoj0cC?Co9ZbdJLC zn&R`*voq&-b)RVNp6Q_k9o)m+Ul{U5I*?>-Ziyp!3nH)(2sT?MAdxwi2k9C~&_bZkg<{Pr7t|4E0=er}i z8vH?u^VKedp+x)oxAtb4Dcg(1zTNyPx0S(1=c^+nm`=tO_jmz;|@I zPybMf&V63oVDOLMlx-4@k{OMG?Jf?=do6Ny#r+h8Fzs=c>p)FbcjF4Cjo$#~eauT| zXU6168M^vb%#q;G5>sm@AMA^GPRn^}a)O2a*>svfoqd*+Ogi)4+5?$ia?|tsoasF32@T*U zi#rYsN{iUX=df;8^*Rcxy9$8stnsQSumKR+X{5w<9?WBgjH5hFjCQh8vR_WLHI*Gc zwzj$l^S=WIAp|vds+(E!t{>a0r||rfw@)7*)nHK;1LiAM2eRe_INIrESJ7nrX)%5| zxqu0$%|6)NJ9z(zVxGs$gC8C@pZGE3IJ~n;*?x zgc4TY#tpQ+O}tfe9i?mQS%JuQ$~;l*Emz{*Hi(_W;2M1IWB%^Fa=~ltfBq<9r3}XA6=Rai^dS6t2O2Dq50*U z;t$jNdhW(HOXCJC$W~xDm+90$wLup;^7EEX4g;JaT&8fcK>`!&X8z>eBc2yG^9INc z#HszlPMDo3Ub0uV%2MFKjtTXqRy zu82pjN~$3dEQjT{asaVbq7niISm?o6X%ShBfhA?>UrCjLauur?sdecJ92#qj&77E; z11fb3*s!*`7-4{*R$I3SL^F|=pk5EV`J%*@9|c&^pwn?n=nYBdpFu6|gJBr3vW#Zg zC$4h>dr8!wcvjH)6=XAz6+7F_LB=38;CZO1>M2nUV7A)8(N3h0EY|b^PS2nNqrLjq zJ1wkmFax#VRNlV>wXRexY^H^IbxE0daszoVQj~Q5k#2zW!(YUa2{=*#miBwiv_krsrnbu4<3m^MZ!_W1)CfICg%(FuDi z$iLXj?#2K_{+vrD!DQdAQCNdK=+`^3-^p{MB2}444MnLf)YyFwldS8}7P#Zir4qeA zugi^u zXz>0fVk2Qr?NjlD$2HR5qOc$&jDAdm>5;QG>T-a%3ekf6MHETXn6UREG0ZF@exK4X zRRE79hB~3c9s>^!_q#hLJ^pwk*-nb7hH04JfJ1-|BUON88(UO9nVf)5&+6&8w4&cLcZHqhf3xC-XcLYkID%LmpW?gdcX*gacs!SP(k_wEbReVriL1g4;FwPthAxul5sb=)B7tcqzQkb7P#MPF0EuV zUf&h}%0jorQ&%-F?WVwzg^H6(x|y-RNL{uWKM&4Oqr=A<0#pBe{Y_=@^v``5V4k`E zQvGcraB#CSS2k%XO5>(+%)ii>?MKpJ1Oco`QC2e6>alBX9tx)fga4bRl$wzG`3DJ* zy#fvT+EB#`(P_ZPxeB)%#tj*Da4D7a`(f)#ZPOvo=PDBhDhvDH-!J^bu~UT%rN3uV*K!6Z9b06atL>#HsgsrE>-SO zm^`+dKvX?UqO5qZk4Y(IxWdah5~L=eaV^R_`VpQ(4{&2Jqn%YK-G|wQNm~9=l8zD@ z+0SF}izlg88UG-vd09|>2OU#qvZ$;n`a4kv57<^p4O}f`+Mg5<7iU;E!QWU6;V-Aa zcN+f17xQw`_vk%m!S&5OI#2LwH{lH?t?d-ht4-o(>cw` ziSa<4k%rVr3d_W9ky3hC2IIB9#*_Wd&6*U^VT@5_eI|6*EFiRc1V^g3Ju~_e zXcz<4xwyLI_~%Z}?hEC_kK+Jxp=}-!kp(*UPL7$8#xB1?_({LGK4Sc878dTK)e?x|9hk+3MMY;Ciu=t66D@kB~Z9-Ri<8SItNv5s?x(06LqlObI~Vg(wIp8WA^Y z<=Q4yy>tTGejuuICLHZ2C4YQtgldHSU240F##em*{go)>?H5@-=_jv`<36pItDX~$ zxjyZu9nLrK+A2tKWfa^uD%GioRGSNl4y#~olS_Hnh0tw`gtrh^x%pfo8Cx<9T+f|| z?(_<8)xrQbg6`=M!2p;?8tLj$LA!clVWc?MNkX`^VJ)1nR*LZN?uOnoU7KZECM?$o zPaEai08J9;0_!y~m^~W+7^=U1LTOps!Zi~*iD|VzS5G-n1`H$YRAIIZ?|6&)(`oOc z(+Rb=is?*XcM7Gb_Xn9&q!z+JrY71NWz!9;FUJV!{GkmC&Id>(9G%q*>lH=P)v12l zU#C9kKhUBgeRRb<3kYs!lk(M4bd9pe7_BmbGM5^t=nnGzBPUu{21S1aM@a=0NEZC~ zPK+1Ri)^i)z`G}3iG7dCeMfTmu^VKC=AMiSmJiPCHl5DfCX~SK+9s_x642^58}^Tx zOucs-h1+~;lM`Uh_7o$1p2cf?G>OFw;X0FscZQ#_V*6<7u&IC;$%5AY9JKU0ZM3? z_h%t|uRe*D^5dzaZh4+Cq#e&evlL^T%#>%QKX;hF`4HO7Of+`5*M)5Vv&anucK94?^Q zS=&RiRsaydmhwUCr$|pqqSFRRa67ru9n#Eo+VrmFVe+;%4j^28jSyMrT_1`cS09g! zKqUoxqQn^5`Mw<*AuMv^;)J||wpcLsEb8S3XO;1`KjLUn?V`Y<o6vw;>lcoG1T3LD3ADmMpC1UVm{+;fQEg`^m~lxt(Ux<%PsGyI+ze3tDYZ)U+p zC%|6XR=U+DlI(Rsa%k*s@>3s>nQOE;a)3rIUA&iOEd@hwu{93()|wS{}}nexS^&ln#zjQBjJ z&+MJ1jlKLj&Q@^g@qcUpU6r7zd}N>V^AmUr2D!k1Z0upp{kQv#rP2T*s% zwU-?sOde4m-l;w$!j3N9i67pvI&a1z4(quJRRW`xb%#;MKQhOzcFI( zJ#oau;}lh$U`9W@*~s387$=@Q0Xks3EY}%t3lpvJrvQmOGkBy3{ByStN^To=vMx^_ z*p2IDsa%^^BNI1Pa&k6uF`VS)y_qQcw%0K3%4XFgo6_6VGspe2JWs)wWDV~5i`;X( z6Q@%xUUwUMPr+vP{)^fM&NEEhFjdq2lMU;74erSF10vp~VFWT`rH6MefC8=Fgea># zhHM^tf=hFumnywXDF!L<4&@XD<5rak9nj)1>5tC4nSb>zM zlvHfWDHa7)`H{`xXBHi;(bY&#b4|O7t3=&$IY{IdB_82!;RFk=yR_}un_$aR`}q~4 z;`TeOR`;!{fj9GO_s&t<*xaCbR_73f<#omqimYx>Y^ug32I@`v-*K)#ATbu`9e=QT zK?c=NrP(iZcwk?3N4l3>)8*Ls8Vr`rd1HUXDtv{>DPLk?(_i-j_2+67cYOpEC_7m^ z<>GpXT2iFc<+8k@ik%eMk~Y*L=B3G;MLQ^y9u*)qz*^iJR-OGBIh*-`F%(BBQd}P) znNEmpve(^&c!zy{X zD{^>t!|K5~T65_%Ei<=5mHEUzL`w0kL#Ygc847C~~eCn$M{Y1?dm9=`TjE0x#<7zy;5i&RV^qlC4vR;jM=le#vVZO!) zX!%3%YJEv$?TsQ3cMJjxKs@IG3&9^|>-i;%Co_BL;S42-UpX4EwB5L+q|IBDoSYL! zCc;Id_{)+BPzPo2*0>>=33?fXbEz@=o@hsw^TGQGrDunrNs zCCvP^;EliLm_C&4$B&lyT5{<4#rZboma3nrZFgh4!fL$o(A5{tb=7 z$Xzc^%}?7-_{rSG*} zwLf!IAF0tdsC?CuT!N79O6u}ED%ouSz#YN(+%}}K=K}rMci!&CI7;!*w~axN%NfOI z1P2ZFbnQ~6-2egk_53tYu+MFsadK;^XFU%ASKuL z-XK~$+E0~1i7@;O#`YO=kmf#wT5gdE0cYF^xu@B5l6Z?ydkaNPvnFzB#N)@=Z_PDb zW!9P%*FjMGOi{!mI|1Iw6ggOYdLxTXxD7il??Co83Ue{k^#o=ujFRX|?sWwPwTXkj z6dl*64?Zd3gP37BF6yU`pt@`XFcsHCs9L#W(M=U9GCi&o%-Pr-b~yP5jsNdQ^f?Rs zk=;)l0J54{%sT{cWC5}CplERz9!osG&L1_MOwnYHNEt`$0v-?RQ91-_In`;UmqGTLi)Lf2bN1|8IY5SOC&=QZO zt&Ttaj&SA-37$}FhE{e{c&DHM5q%#pGco_{`-H4FjXqSodfD5oZt7LFn)jZ=E7vA#$_x7~Il8JrE+1Qu;z@fk&fCIO2PR!OTYQqB+ z*;?4xYkid`ZER4ebH}AxD>Jl^;q@0ONEvIwkio1(E>n~*cDcNSC;+^dH!R8o!#n{7 zCza}5CC22yM^Q}G2%8p-*`g^Gsw+nrXJ?Un5SNtfXf8TGzz*q-M*Fyzl2Q=~a;-V2 zWx4)Om(-Qg;);?DpnO=(P@R?kkZZelUWUq1d&T-+|0b9=#qVWu7BsSq>a%t>w>%99 z_s!7obf>$2Y{w!fMdu&J#Y|{8v<>sf(@?45X(66bV~cK+<|e8z21!i(7^QCbxN2V2 zwj-Sze-&v-Dpr}cDqCNkHOC}QS?YfIs4FIG{QS{@>Su%{2Y7Jel2+M+)s-;KUviqT zWMEVJ?6VbfWeZ449yGhfP&}U#=5yA;`Zwvw$iK_B=V(JFJ%TypTaswFRzEF3a^g_* zD^3ktnbmDSss`H#fS8s;GYm$To9)X>Pss*PxP`U905$5>E;=%M5?VFFl z%ktFI%Xv9a1@KK!A+X~kCS8Hdb6_YKMu)dQEpQNI8@pe|igRoc`Q0IIXV!&y!!k7S zVkGfjx^%D)3BGfM1q==$-eO~uEcMShfUQdW-bKbdD`Jt>5jzT>u=HOMkrw%56{lH) zEX%M$xEP6U@{h=5rHWydZI!k1QXqPt``%+Pk=LWcy;BdOJ zpINIxD3=t;!||`JJ{k4dSARPFKN5A1MCZEnvyl>fdY3asy^-#~~O!E2wN% z8=8*v3efVhkfsk_DbP{0yIb+m7s(XZ8;9l|UxJ1lZ|1}vcMzo-e{{kakGW-u>nzt1 z$D))sk4sn*&ML~>e&uN%*>3JE!^g)uVkyn})y2{U4%i{*6jTfz8B)2=U+m(y zE3ZpGMZW!Ux+)PL1mU8@&$o{ZypQmR(nE;_?QqwY$Vx&~{znM|nfWuFeex9b{SZaL z7J&88>&;YOol75iw~cl23p_UBZU-E1lo3Bj-WN-%BJXWb=XpuRV`(PxC3WZLdAlr) z0Btl~0q^clpo>+$t`U*d$v6$Hh|~COa*0fkVdX)e7Q}|sEN{^;_XCtUVJTmEiOT^i z98VTp4^&OO-TGDb5}SE*3Q|`4EG0yB&T?=k*Monk z{H zx?AlLiLs+nf_o;f8R&dC6S>5Y-XmK3)6Oq|=m!q+sG3gf7xq0h zG=*3LdixG`xl__8(Ch_>Q#CcHfm3C*@cG#E zY9qr1W-%BTL&GG+SPEv)|K1Rl&VK>ZOZR-#9N-f{IbtwlZjEpKMCw>v;Z*9>Dw$<( zxew9yYyiR+FbX62!KMO#s-d<2MSi>ONY++~MkSgQp!Ol7L_*K>RS_v<0pQ`V`+8VF zdIXfKAIhE5WxJ>fQ{HNSq`=iGLvxH=r`gcX1vS`ScgwT05Od>Z?infkQ-~sdZ)KK9 z->(4UHS8}e%RCnj>sP)`f8cln5=SbDC}p*k(g|WB%<@(Y<|AwWcWS7`^El^2ID@jW)F-hkJ(L3{NE~qp^0U=R8k(^e85{kDx^6jWWvNs2t#~uohFMOFUoX#rW_k*Ak)4~I-$9yIWMSr z*Bs9oI2r%!7$GGFEYF$d0M}V^p-~%14^Bda=`_Vq{ckb#eSf+wAJ=Q%lzIu%IA}PH z>$BXyYT=M(of%($3}zNwP=3%0^e?zHhO-Zz3_ zs?23ULRJ^}X>3{anmSZ}#^8RHe<&z*ETJHT&&gL*2OW}=*TCXeS!X}~5xAA0rF#+IZE@1bLG_lB57dV^pX3;M% zfYKcQo~? zF+2Qd3N7l>U$i_^djGjK3x5s`6K|ZAw!87Axt&KNAoV(?oAr z8mCy;B?gYo0Ym>vXf=HBqW5|l*h~+$5vq^3j;@N{U^7~`YE#}m?7m~sY$XhptE~<6 zl{A)nHF9&S{ZWSA+aVrCNL}GFISv*d=0EMU)b8w2D0w|K%9g|?%C^AsJJGV=+@uoT zD3k%4AbEbt$pu~57ha9_qHk_5ZD5e?_Hm#aDccm# zCf_gI7XIG8vlh)v$!o?PaZ6|xc{Z%@cpdj%qNWaqg7S@zc34z);)oy8;ih@hGUG=+ zJ2fb=jHdmY5Tn?8(9tC~wLCS(8Fh7Lca_ z7s-4HxZUc(8$IyDyaW;H{ZJt!sLkKzcvz+-Uo3{Hsz?_qY-}ON%ws&wSSod|Qhbvk zQOV?Hb%b>0+uVO|kRaoSoM7JF`B{XRf0L%kdoOTq_DPxUoQdxffp6qz`hJOU)?O#C z*029-up87@Z@nzrooEy(D|m{x-(*A13$^ZrF*9`J<{deE!{c+LKl2OpKO-LhJIC>V zLkmk-9CAA;vS{#JfU+=fm07ylhtO!UVVLMSM2Rq4M$G^W9oZI8JPor+7mPrJ?Fx@7 zvSG{t&}W5V=8C-axVZC|m+8!_OkU=rr<@$~AKBZAaO`uH`2lTIo_FYUt*xegwiTDH z8ZM40hEuV8qb&GtFW&lA`r$0R4{!fIyBR*Us}h#QC@#fs0R~%Ok8UZ=u9;Sch~6 z%#Cz4zB^2sEq3Q!0~Y;;t{gHiW*j#6Yh`L#Lg@OO z;Cm3eBCuAwkkc&;gU_j|PL=CLWfXVnX=i={$LQl^j+Fe4^8GK{^7qd?`(f?kTp6Uj z!V3lKpf!QiKoj)Oio^fL*!*{qzsGSAzjM_8@8xG#@@$0p9?~rW|B;f*2Pq7=vR+U? z<*kuTpjA^ZSvsj)&}`!GwV@(*fn6`=DCJsTzuvQcUF}BFCzVb7MEj(^iv$H1yiNcl zdgv>$gQp0=l0J23IM1|YK6&u_`F%of)1|Y?&}g*Uk%WrkfUnmoG&v;n>BLKgRKUJy z`KI(~Jaq&Z7Skqq;CG#b8H57ddCDi7eN((Uc;dWx>9BVM;IR(fApEyCFzbjwcS5WS z6e>k|Lw1Xmh&7&`9VmXi)YAHjhcn=EW|c0Z_%n-jLl>#@h8MT58+~&^ACc31s_D zv9|xzd7^$Qy)g6zPBNu+b+CJ?KfZ8Zdgwf4nP`_;p9K|v)sAgG7WAUY7j8z+6WF>p z@@3tRj6b~RseG6d<34W?)5vJHPC4*L7TeWeDE4!PK>nNN$|?qQ^eRX`BpULG&ItwX z%oW;7vU;@@)r3h=fk^{gI#Orlv-OxzT+}F?DVQRHhly6irJQO{tWWcA;7Tay$$msI zqrzPf=egXWS2>|x(2sW6)eG+<82KBATrmoB3PxkOxZz_Q175;()#;SjFytM|8U?9R zMo8!sMV~UF56LkhDVfv7naOzWT9HSftA|ymoI*B-IC?@Bf~Z*sQDk>$TxZf3N`IgI zckleq4*CD>9o)R?g{|-WMKkIY058dZc_*+Ggt~JG;x*V` z15(oHAjF@ASWs9|rig^vMri1fw6G91p@#Qy!}wF91xT2kKv_$a1hyjE_EyRBVOQb3 zc3MQldP|2rmTj(>y=KSs;*;!ExfGVj1f$)euePfxVj`$i;A?8fmh;SQj`NIf=l!Ph z(GMC$_csS>J;2>I4j<}vlaNi{mY3sfsQvC$;TKKK)4Xfsa+6_PR6}e=*mtIFGmX5Uk7Do_5^#ifqltLp6WD$?CWc$ zlvl)^ZV{eQ2JB1TQDIp^jxL!r;lZ_z2cPrMmrmiGtzxfKpg3nA2*i_Q##bvHzue^R zw9iSi`Ijy+cwbm~9{xBqP_{#gkjeZqp47qp&4_zx*|S)RDX|^=R<6A2Chz25M?MOe z4y0^QmajoX{#0S{n??5mZ3Ffz*-E?1OW0}y=3$>sMITa5!uFF7>ZZ_(ZNkXHWqYMO z;UldZdr#7|Z#iw1R41oR4bBCJ9N|)Rs->#(>dxM8cKowHkGp}hhnbq-i$x61>zRnk zfM4g*>1Hz^3%KWDL&)(C5kAmg!XM!pH38DanxQn+teeE)9wr=;4aHjhWrG4N^YeKX zauIeRdjXLSA?RsNHBrL_Wp2TbA(XZ3LnTyhO@{;lyC% zSSz5P-zbn#YNtA+x_WG;bf!{gE;UEKoWc>cEYUS+r&Mvo~OVKUTMP?dOI7}^kl9Wld!yktOlnW9iXIO4tHY?@qw9tWbeayTK*+3~bec=}r!|Y=oS7H{U?;va@onHn zuPqTMMz$Lj^0IyPC0_cWOnFB&wh!J4SwnaS>DCOceAP{n;DVQc2N%7@OEpS9v-(DI zm@Aj*vPJol0yd!Ql;!PdT-|_N-+ynm>Y*vj=RQCORYN62A##p$JnWO-jBYB*hdZh^ zH5mOeNYvFpD=)vCkyT~_a1oH35%)HVY4#~fo~nBJMT?ViMA>l8+E}*y`|lE**I8nA zN{%vTewPe~o6cC%?PGd=PV@u9M+hAV#5`IpCr)H>L1n2hy#DbgJBm7)8W=5OZ~G}0H)h?3}NarR(8U) z{o-)%xO_H*0lXnrJ94gBQEBtKG(mo#ISjM(RrzfZklwX8~UR4V)t9KpPc zTE*2a5@5UODxRw~CeLB+db@W(pyR^xeM(F*9%gs;e6t4TG-`sT(#x{DZJR>c6=@TU zIJdT%;0`|mS*pgu0gtO^*l>%*Y6`(tMg6(Zq(8N_)T8{J*0Y{+V5vj#M*1c{ z?~*VZD7Q$f(_>Yt`wVvXPHiz-;D&Q5=ZZ1KyIFvCs3+MKiFT+f+TP3VnauG-4zIBV zQGY+;x6m*R=*cp^O}}C%d9ss*6;+=5I85HBV@^|`us%M;kG(;A^=k-tr+K+F254fF z76wt)@6_*=s1MH*d><2>M6lc2XZUWL=3XZ~(CP;DCL3NSRm3}6sHaP6Ju0XDM;p0$ zI4p+KwprL^2p3I-o0;#fe{4l6%Xlk8S%*Pew)*4&e8Mx21vb;7;ndS0?NVlN{vxey z#nl7Zf|WmRQDd3#+n#-vmMmxWEe>UU6l1(yX|g;PRbE! z)YsAhCQ?qj?${Xk-A|!Yv~Mic&KHTnnm3q`#9G0MFg7Qj8YtQY>nB9VY@v`Axau|6 z1qo#nQGPGJelKCE(;?T`%%Fyn3!?tFtaZ}!g!JGo{C!>zu0y)+1v0GGn7GxO0}o%mi4j>-+g-PWy(!TFX7vhqXTc|^-n9q zYgmvWKld_g)Jr_Uo+B2XzFA9;K-m(-6&L)hzVEz^gj6=@J(h?7Kl23u zr3`;KR=Yl~NsS3=Q3)gSvHQ!0Y(c6kwqHzgbgs_`Vz}ghN9BAe5p}22>HC@pXjwz-G4YM-!WPbQO)!PpeiC@uItLV z5F$=|kRQyERy@%cfr7FpnMlA_UXoORw${kfufr_Ck2~^G9`e65PX*}*T90Z*5KDQu zXPosQuE+gCHX^@T46xW7O{l$oB_SB%z%>Mx_kdCg9T4{2Fc{t%3hk*|j*?q^mRJrW zr9VXKcxE5ILCZc-ML%I*D$^ALf8zEc%6CmMVV%G>kki6=%_GmE9WVK%ujOe2AXl7D zmp*LTT+Rr!?KY58Jsc#23wn#)%tKZ;aJ~OHvpV$1H+SOm9QO&om+T{?z9_p)+vpe! zTpNDtvYL8>k34_M59E)JAeDRJ!c=tsdC9=5lIv-BSx=y_e&n7p>YE-fX4Y77UMY*| z`r0_$JM|D7Vcmz}*@z5P5LrC{cz``r2J+;EpM)P9QOZ*kqkIM=*Zp$l=oa4}CVtO{ z=xp>0r!CSZ=o25ylCX9d)cT@Wj*Md7_P`jz$5l*GaHQPRCMZyttfI{Wy5LjTxGI;@ zK{qAO+LdC98h12k)e?|MWqFGsp)yk1XH3kUK6X&+5IchBd86lLC7I6xfX*wj;aims z2N9h=f@xwE_G9ycN|umtqdBA)0dRPg%gkF_VOzj_dqM))()s+3HZM-TYFWKdh)ZNw z7NcslP|Q5;f*3nnpCWB>R-O6bmL-0x8vb!++CBCD8@||EiXu3A?4NXbx5P)ZcpqG@ zgDJntB6hqvg(wV*jZKRHca*5ATnJShl@u4+sxuBNd7`unBkyz=et!depck$!p>KlF zt0q_F1PjuAp!}p8XeK=ruHm6AAh%m?xY>aCg^faXUJRJG+%vgtWv{U4rU=f~p8Og9 zQ6hG0zAIc-Z3x=2mLY`?k`ulwuZc^h!-Ds(^5#qY*>im;Gyr>mQj8Ya{&6h7Lku4? zrTDV!=v)~>YQaxW>VLRdhjyGLlrA82Bg}clr${t*t`P3o{m4hzx+Qf%JOA#mFa<*! z#T~k~UM5?wuW7u^$KEhJ<~e~($CD#DC}Z^dqjQPz!u`X_={3bsxU$W-3z9boVMyj= zJQ`=Qrv68Mkgg|y+Urmn{*hGM@Y|mV$jfjFEtk+USTweGY937*Xq(0VhpcZ5&MfG< z&BV@xPi)(^ZQHi}Oq@LN#I|jl6MJGC6FZst^1k<0eO32Xb#?VQr%wOc-Bo+-z1BKr zJ%M!`Z#1O6#b!dkH?nx}4g1t;N+bwE>va(OERqRZVdjMh5YK|$6G zvBIB@Q|F6CYv=2c7Ltw060{nlSYz;AY&o_uOwlYHbgg5R?!?IjTJkxFWSWn#$X~3) ze!uoC(P3vrR;l24BZvd`C~ou)Wc;?N;9^E_vbC%k`s8fl6z>$^J{TycEQwScPud1W z+hUE0n@GXhm4%Ge`Xk0m{~UZbOEITWy-wt(*hNU=xm_9;XYR^{M6YjhCOuU zlygFkDKx{#A=Cb(U*R4Y2|GEvs&>z8_JqkYv>UdI&mM=}kRb+w@02ME2}1>r>m5op z&*(km6PaNK4!%aqn{AhJFy2b~WUZZb$t!ISeY5u#!?%1+Giu*CKR)?dT9g@9kFjHU;(^~DI9 zvBm+Zk7v9FWRl5w)oCaCq2Znur>P^xSnpqpXdD=I+!r8l;BFGK*MkJ5K6QlT+Xgi! z>`L{7lSWfbe9XQyh1!;b3SV1OL4A3#Np*-y?OBb1 z7)^3e{o=ybb$t}#fV$VG#0_?$m#$dm%SNLLN z#YH=~o8hB(uPz}mtJ6(4&{Mgum~YllqVw8a&Yb}Z&HAeAI_aZ1%tWc2z=3ap@3g== z`SGLUFQctReNFk{BU#rD_`CV#I>*2L?H{_~lVuK!Sq#vV;O6$nB$kI);2zTjh#I94 zm7}aRPLwU-(nir}EyyXh5a$|Nlu6Vk-@Vsnd@|Wc*{#H_TP42op-IqOHW_$>a>kyK z@Ex#n`kz2HS_BhP?pSl>E|1l<+S2Z=@$yglB+c|y@fyccKF4V|^b+2+2*;l4`=iU- z65fWk{8RCu#FZT*mX%KrO2w&VyP&}ZhN9KCFO=*_L(vm^KecZjuW!bxCVg?FI1D&q zcx<3oKYezP$&tev&0HG^;@%=bYBRhr5q zbtHI{fOYPy(JNCjKY}HqT`g~D^3d<+FgGfZI#q?;U=nxwGv1anzMdbSW53bCP)yRC zk(No}wW)u$mG5~#aItGIoR75j)TYD#Vs6^8q~^abCVDyN5i8XsVm8OylV+ak2qf5vHA4)PGN;z73pVtp&eCYekm6BnN1Mesqdl4fokE{kMGUw#%=?Fh{v1v3ZnytW_~&qz{kv$Kt_p@E<)4dY zq$Qk{&pAaZW0a&@ZSkkPIAZ~0<_!_z!tg^EdeiVb90@0H=|$7>oVJ{x>7Ay>m#e;5 zAuro|HAZxWUX`lru9F&|Y}lu#)o+J+z(1uFR7oU3a`1$_`-HOzz@U z%P)ghudzE;2g^jxl$5Hs%4R84^d@sAG~WnBpZ+8!W2zn5Jmz~9p^VvaOTX*HEg6$s z<3EVFkoPnn!bcKQi@7JtBeXT*$aW8U{FGS(_#883R~OVtxg;IQvVT*8;CcHnD2x=* zk6ps-v9x5ElGLd+iKAVEq&3N0PDTcm{u@hQAp!S=)%yQ`C~jLh9j7Qf@$t)P6RnQ z0;Fg#4hpwVrPv*3-!NiukFi?w5_yy zYoWf`o%pF4Kc{299lPGKsqVD2KA$&oum^yWw+ZgQqF+KwwcKvN>gF%`(Wx>m^t6pV-Z=Sw4iy1>L{p3FY7EjX$fa0!1I&L> z4MHo%Sz6)*u`4_ZqxID;F2h)K0d*@H7m{9`=*5JP?NpdPf;DNWN@5R+Ceh8{wD1)M zL~0juNN*}C;z)0cSKpR;&@my-28h+@6ix1%2WZy6J5XZ+Dct?A_qeikWAi6f)$q>d zB{SBoiQJb*Uz!`%xry}qg`TVr&R!vH1?QJ2UyVyR*Op3O@@4tr!?R=|yb31o&s;N= zvTCRE*X1jxSzFcw%MXZ3tlgSB-2DhK?|at49Y~h3GS*Fr^yQ~=${22KR+qFC(x{|n)1$i`1SVkdh(gd?tt%NcS>B!vihbCHrR}L23NYAK7!gq4yjb+KdscBk`E~mj!=bJ(P#nWArQ@Bq;o2qupkIAHb8$cT< zJc3f;od_Kmwfi>`M5K1hiYZjEkD2$RcBN6-bG&HyJ5aKPt9)a&;t8R$M`q+JJTWny zRBkyjua$2ZTi!V^`wAwX%f0hJJF2&Qw1GgS5g7H4AdryiEeR%(3d@hTW93^aw81gK zEA*>bO1-N^!DYds4Pe23o%%;CXtv^o5A$DsaCM&KiW1pHS>og3_?wOSCwpVy7qOjN z0cuSs_A*cMNeK)y23|2tGAutBv4B{Ndi*;)Yv`a{SUd>ht23@xbv_)#PCE_@LIi%z z@T|e4Ss*Y>hYD*c2|wCkb@zNyaj0nn5&Q216{q47jF>qUr~BvzP&y*>IrO8(Y7aLJ zXCG>Z>8CazexfW3g9lcLUK(FEYoS)ptIqYe%JA{@=v$)$&@ZIK&b#n+Ug*ESI5-kJ z8D4c8heRTqsx;LtQnz`#I**G>;q`z*n~A2@DUj9HvMZ;dQ&RPxwD(IUp@ z=XcQ)?@0B5A3*v85-r)J2VzjH>hfz#j~vsw=5Z z=_~CkrC2gnNvKz#)7XZVSdP^dtvzd%c5ahw?@`Yt5;R}BbJ9dv=nONg8Js@*hiaW7 z0ArSr-9rq{aEG8(#F4R}=mk)bAe@O%41eOMnBD>?R^acZaK}V?{=13;uB4W_xUx^e zOt71ok|vjGXJ>fjdghx8P~-7vvI;JerMt4UvLIQ+aC@z!)B5hv@E$v|p(UkU*ha0& zmXvz=D6x|V5{X9$6HjxNuCA)k&O?p8y0X$sEz{c`RSJcKt?L}^GL^Thpxbo%_2_~e zJBChPWjx!9DzDp9_MrHkRQ+f{sK7tsi@2&EA2kXy#ZUj+pz&LxS(u_}RcY91yXemUp z8g+9g3RGbfWykbYw7NW9DQq*hmFa*I5}2RfDq}^l;yK+ED|$jp zP#jT#*=EJ4`WYHAQPt%o{#0fGQ?e4*DTgiJWhgj`fJMhj)5wNlI{HS?Zd*ZtR)Llp z#)pP_1{yls-!r;RagTX1`t5uU?F3YaVxRiwtAF~|fYhrSbc&oYeOUG9?yDmOTS|Rg zS6W>NJldRey6e({SHFiPY65ih{8WZb{5nr4xFn$9V44QG!C?DfNRrh{zgJYKdlxSe z)19Bl0%K^x>XWWoeR?(*cS(Q4)KK-=l(QU6(rM23%s*mUS9cQEHZY=L2xO~h$>HEa ziI|ThQWHr?ET_*dk;JPWjx0$}PlRin#$1p-#x8qrlxuoD_*UhFLz4FWmMCBuxA9oBziN*LCSm?e<=;IweA;+CYx)yzs+)O2C@IOW6ytNrHe z1CEBP-TdKY_toH_E4t%Y=2)REd{)IsJ6_VLNm=g-tQPQ5v#p{t)7rZGDy~hcMjPsB z;kcapxnCLcZjC{oA+*Ltu%OSuw z%GWg#dZ(@_b*ty}RBBeDw`Rr)U+Sri0gH-?t2#IyHR$H~vWY6HZN)9kskVz-DYtUv zu|}(G-L>L4TT;{I$cUc<=K1a#2B9n#5y(>A|=y6vtkp5MH7^AaaP;er*HjW^l)Fi6yu?Y}D(>P(`R@_jCzq7hOL+KGeUBQgg=q`OdSoR1xT-F%Cx)@(VdM z8*KfT*dHsst-$7}v+lz;Q*5_DxOcGaOzp1LXgzE5>e!{4VvIW%+2lfk^Agf|+eE** z;Pz88kj{=yA}E@!l4daFulk$28sg7}AcDxwcUHjFv*{`u=XsHJ4_-=3V`Xn)ckR#Q zRUEp~gvRwrWSWbd=ms6C~WBa;p%CHTqvK|%$b>iPFeNa6H2z@MGb#+?0EuOm%@vE1| zRE6y@N`klUe=Z6mWC6?ndd$ixe)@B|JgVT+D-7e;7i4!gYB5q*r}RoVl{-}Tk9To1 zxztd1;QMJ{4%p=VE|i$ngm&?~T8|u}exoHj(sT_PeyuvMK^mi#shZao>!L&yWw4?Y zuVnWR#TbaTc%5EBv?P&${mDg)ATJ+W?I5Ja0LOZIVmhYlQVV2NtTfVSDHMsdBbI-V zc4bMR#rha5e!(FtF>1W0St)thL=>w&$aiCvX$XXvwXmXCVHjKf7|!*w9XKD>4c$`T zd0n*-2()D?BndB6`$`*HYf)26xVZ&@~- z)HF6wpAB=aPyyRY+<8~bZ+J>O8*(9LNGR+%p>(rWSWR;<2Lux{L6L~HWGO?|a7 z23cXR3@vvla(rtwK%tmZE6SO1Nc4c(_r~lMOa39c%H^Qg#?U`SX}`IaQnc8I%>V?q z_<)kc(zTXY4@}gFT^7Xnv09X%`zG2}T(aa;=P0AvJ75#^6J9oeMZJ_mQ){+7yUQ1vD1q?BQW<;q!RfnG$d9KE$h@CT$_(cow-?Xc!ns(Kj)h+5|6DWsWMOH=y~p#&D?#zA6nRC@`4$amn!1 zn%6V52>d7v?TPry-b&2ViZx$TPt6mBw95G8CbTmlSe!eXX_{_?2q60?RF5%E$m_-> z?nO=|p43Fk@oE%cA2g(Wnm>XFYz5V@k4X1nQ%JtAgH8-H z!#>4oYM&oIPq}$QL)&a4uYoKI^3P2S;>(CEi%gccM8t`q`hV8U_W@0C%2MJW0!zM6 z;0H_KCev}gOu2~M3haPp8%LqxM-fCxn=n*tE z`p0O5VOD4jmU5`+Z^J0&9L-~DX+2q9KvH>fB*?GKyVM-RImqgMR0wf?o;{X=W+p zYpgL^Go~RQmugQLM#*Lay@l(pPZ>0{ABC#B^uV8KuOsk5KH{48gd23YrKi_`DrKqm zuavE@Q(o?pj(%QEQ~Jp>2?%r?!z0UE5%)@rO3sKCt&^ z-Q9g{cUE5A=vU9ZPx%Y~;fcUaZuywpyl_%E)iQ3yO0+G&M4`IO6yo~g1WLUt-8M|U zOI-i_MB;-%h)6y)?wE;4VZ_)4$RqHTAeKKayE`$|v5ZLewTs%{-TQ6}w@jB}OZRE1 z4tag}^Dm53Egoiww?t{9YpFQUZt=x)*M~8d%^kVZmd*>9BIQk=yu8Z)Md)kCMGy>m zGZcfJA%i?YWxapdC|x}U{03dQH}ub@)|ZsF$NjM{U2#DQ31MH{$3*6tWb&xtjPJ^)TrvA-4MFz6vhIu6$K;q($An zY@hesU&y5^az}XNJM9}8&}_cvM&?pX?eZ}?z*LAej*;*t8fN61=>|RNn+I+EBl7+U z+FvwGCe{CB+Qs`ELVfYdtCNB^(Gk*2&+|utw1oNlB7fC{gv0|-RCdIcp1(f^BSe(^JYr5BMnaslC%PxZ?pQIuumDtc9)ikp^|g~`+++63Qyar=E8uvQQjU7 zs;X$uUN?{&0rDHlhb1p)(}Gy6jfv9tS^Bu;WmV}2$COX1SL()RE7#V|b)TM_bZyt3 z@gc+!Q_;x8%Oad?gx^}?YpB|1UcNK)MjSG`X)Z6aN{8LXYoZZm{p* z#BmmF#ht}j^3?p%;5le)V)QOjn32;*Oda!$qHE4D1fH%C@1=qaA}T&(Qi=SECZ|3N zA0{kM(iV}MT2oV#4)@}Ixoh+WiJV}SH<145@Q#m*o6||OX+gqB8p{b#m(jQoU!}^m z#u4gt*sqEf=laA>_eiwmWfP-vw`M*$|4FIz@G?QC#kInvJ!zZ~^}e=hI&~N|IjV%m z<@3P)4*a^!QoVNg+umzR62x@%T5STZY1&#Y=S&@B171PnU&YODZSE;E!mQFQj&DlY_2*$ zW)o}-YluV73c@-H_3x{TrW33!I+*b45glw6M zlb~FQrnH;byZXw8;6v>%d-=0{xVU2_?mP{#$0y}Zj-H$og~BQ=S&|%pA*`VMRvoivyWlhdhY5*TvN{;5S|lESfsl=S#q^Tw&haMfPP6rPhM@*E zy~b#0%d7Df7&$@e**PvPk=CEn;kq8y3{0~%Q5Vtlr><}jk^CWwwT~ZnogA)1z#XTa zA}+}*?{`g!iQf`T-H)*t46+uEczmvNo5T_KqpeuQ#kEeo99Pm7%c**!@NYS4d{y|b zYs{sfr$>oxNS|ZI74FXER1F%sMxHav=1*?b*XCotgU`&ag6Io8QoEX_-6McrgwmY> z;#X6-o~vWo7z6xL)ey}gjNGOuYOarb798Y5sz%k*57p{}`SxJ?dify_yrPcWW)eAv zn*&(lN)7lkN?nH4QuR08KzaiRp^PkL(O=v$i8-2(G&6~>sEdWtcHUvl4reKIt$p(I?XyYNRF=GWYBZm))Q-{fTbKp$UFeA*E;ng1r zBT#swAKy^TAENq20m&GuRvZq~KEF|o2-A{A^SU5k@$lnO|K z!C`FhgvRMVa;}IuMhMCO@1W!ebFkDx(`|0A3Gc#;V7mdfMR08X`e7Jleq^Vs4Eazr z=NR}`#X7IZhE6IMrK0}(`VcAXkm)J19Jyu*mN;G&A*ZH*eYNkZ_G6@J zSPO%@~D9q1-d! zG+6}tt`!3K_gEO};fL%g3@QwIw7vWPj_+GlsCOLN&sCaPYn&xtiti!8`FN^8c~1t| zDqmm{!2j`o=VhD{lU#5ps}JRv)Y~5tCC1A-s&^HaZc)=Ky{tT1s&t48b61y zIg58=5z9KW1R*1l-}WI`c;srf*Lf;Uf)ellaGFEC!K&igMW}1e}bC_(}<5CA4IOVpLuGuzmA}2 z--oEVn@SYnQdT~&A(05VkO+quB+9`}p(`$SGB)g^$$l`y}9R z`x=AB%+?n)a0(za(Q!}=AcU*c%CAz(>h&ZO)rV5PQ*8kCEKf_(hU|o z!C`lzH_jm@UxNRG!*2Z!#rnb1O{fBruxT)+@V!1UIaGu_eh`ErYoWx}&|JrJ;v6`w zfFX`S8M_Px6TrU*V#LG>@O?ADdI|&|@3-c}plm=(9|<4O=fuZqP?efewri?OZPlZAw=G%R`!32U#F&FV zGN7b97e2q{b17qACMWqOV)u@8D2LUO_OwgGp*N8%V2#%Tgs~pbWrwzhX7!@%#8)Tc z++eZWSYK^viPhD!K;+KBtd~3}b7mJB%ux1j)J8e|rlMnSZaY%SAuxD2KvlS6#qFNE zpKcnzbpN9l0F4DL{twM*?i~#qhM5z=7lJTTUGN-@Iiz<@gy@O($sW}{`WJHct4v@N zm+`{pI>IM7kTOoC6~KF<*HXm%3p`8FIDWwV(qnhZ=$2+iINMfY)n4eK_3b)@(e+q| zOlMU_yn(gkr={7PUpx8HKl^%&%xxQG2q&_(op7(Cx)P@i7c_9aAD*tLxNQ{FH6^Dt zO#Kx-h_Z9GafANW{CaV3no)0918`m7WzVeILz*H;K)sNpdR$JEnWmiOL5B2;%l@6M zYQiV8KsOAES2_D<&enH`)_25>PrN+f&}jN|RYR6X8NsIkJfY4s10IL=eXUrNT;_91_4jX32w~0)I3&)siw6 z$&#ubXenEh2j$QkB#KBO)v&nw!ERpk281$D6q9NO==h@QpvSWk%qmY+Aj%@YWNm41 zLitHM1g^L`*n`I@z^(dM^nUAE?PvdnxeS}`XF^{JC;1OeX^xD;!`Nib4S(jK9-;H6 zRAE&G{v1nGBP#igsT23^7K&s)pJ{ZE@2&g`kXZcS;jin0eq53dy6(j^_O?+J*r*Ht6==-JUINi+HUUd_L=k%btvR6kxoP$YPeHsb}>r*<2ehojODM z_S)YMZMKc!!GA+^Hch3D(hlPhUMI;W%6-Zlwi-2;%4On-YyHZp@m~6tquGQy`o}Lr z;0yd+Uo*tP^<@1X46VKPE@B0m8FcG=HHn7*)UFK{+sAh)KNehjoAijHFo@Rneh?1) zIGM3aV8b|Eq`)p6aYQDteb*Pju4vWzS*9=#ziB$%*;8~PH zCVXgedOJm#jwugU%sFuRgDGKjZr&IpD79poP_@~;Q%Cc1V_$-zQ!gCh8%lu$XGncY zasB-~JytjDXG0JG4?i&#dMMtM=!xq(BA=bV_pbsomC!C9yj$DL3_kZgiB7va@OL0i zP>gmTVwDl}<7C3;!Y^$GQL}l>wapZdG9!vPEfcV9KI)V7JOL2PFSEH^f^d9!1>OcK zme&AywqIn-uv_ECo`{n52jwkFMOA!3ao0HR$y6qo=N=4vP=JCb2Xm(Uy*EMJTB81( zODXH-KtzQXzwA#)#Re_reITYGa8$6a>GvMs(&|R=oyWgF@Eg$7X^`qNE%HF`hQ|-4 z*ZA7h`VH^c?7G4JN$}#wqyCfJ@Lz_N=$;|?wJv+GZM=}#vi*AoILP33LHaPfQD^GeRdYGsLDIKlWPbP~RX7h|v?wLA02F zB9z!-Qkg0Ovc;PmX>HK29t;r5nhB&G3XCA zJ=`=3mHdOmI^-HB2=CAKvLWf`c;zm!vbZtA5q%NM>HtYepeu5*cV+F z%oqbOShYxfRYxojy;_n2v_;p1`S(6v>x914fV^-|)YUO*;B20XWu6J)BcS;j zC^r2Mrt=aw===l~9`P^OG-XCi}u6In`0 zgfUH%+OaM6@ahsU(YfQO_qu`v z?9+C`hdQ;+h9;N*X!~B`U;_;Mw{7ZF<$YgQzUdkegUy?LX}uDdE-Oi94#h(z%^Y1i zzc6enpAAmh3_xPU!JH>r%%otoeoJ<=Hb+I^h+XJ{Yr2Q@s!36wsn z_=o+14d?1@bsF3E>7zBKItr$@2|=Gn zJ&)19$S0Q!+*YH$5>Ae+-h(e$Qf{9TtrRBJu;EUGpCtOEZ+{ft^cAI?6$^h34|?H? z0q}l2I^(tNwe}IlL1X9;jIgqyT=cczJeyrg>9YNlBt%8Ng1GgTK7uRI&Q->RzYfFy zQU#5I1>wu5hkz5MI+~Po03-EyPCk|SK>cIyV(>)3&~dCHPsDI=LC6dq431)K4)gRH z94KoY8vBxQt73&gX^G&tphuY!hK@@wb07tagRoF6sx9mwA_L3A)KL~MX{-_x%o_8rNGlk`>UsX(?sY&8S%!jv5( zMS`{s<@?uUDMdBAh-Bi(^+^{4^5L&F98y#*2}mX$0250s@L)WHMG$uD8%Ql>Db9yp zktr+~#{?F_Bzi_KIgwgeX#}R@j}z*H<9-4>;*LlY6EcJo^MNdBMjPF4`qd3zde6Ze zQ5P20y<(VnicC;)h?tEJad=5-(HKCzQfS-0swyTu_V7w}Cf39(^On>@nurws4=<*i zCYXY6~=Jy5PW@YcGLJ_u| zE2it#77HTnL;3hd7s^ z-_bFcG>~GZrOFs%2Ym5mS$Uu?$ip9x{myPrwU?mv=3-Ab5Mndt^UopI8k~hy%nN-5 zvPk?3pm~;(GJ|c^$U>9{X|%rYmv$v8&Kr+}GB==&W+N9$tD%ohVAbSe0Qm`sQUb%b zDwJZiInXOSYd3uqn9dLwo0K7w_B_tTtPM?-bjzDF&zoFu`LB;gwP5L3Xd#>BVJ$dL zpRR*o%rAX@V8+?HPt1{29ZU1)#?sHj35Q=$56&5AhW%tml7$0chEYuU!&I~of*7m1 za=wBzJ(>H-!k}bB(9c{3?wA!I7aydR!pH{dB%iXgmk3NRMCV-&_oL?kSRhLBjh_ks z4wBm|Vfp^^kc2-Iyn~SaF-pH@AeiHTpx13g$sD6dVV7y79b65hK?OzgEjGqKEjWj2 zv~~Rl*s`y$VXdnEG}OIzDKFs8B@DeetuLO9btzwyQ}T-^?=Y7n2*|`i+snp01@FWV zv`0Xra7lz&KA$O!!zY%c1^*D(^72ltD<%JLjyH2G|EWIN&PE$oBy};5R1U*XxlBe% zBis2~P@!G02L&`G&>CNkj^x6{vmkqy2g6UgW->;D_$wJu~j+*!-^GH-{HgcM65MI3lsO7xuSn zbm}836L_JhZ;SW!F?~N40#So{Lcg8F?-Kqs^G@hPjK9$;E>0un!p-lK2U@f2s_03^ zPf$Qm+5yv8C&CS|8-G?fL}U(@`11`Y=qJgI4>6w>d&oni765Gk`eUc8nTvW}h%M_$ z5V~1x%((J2jph6_3f4}pi;a1k5HW_v%74Ae?)rLaj4=W-J(Y)UOqoXi+SVX7T}k=? z5u}oy5+kQ$7{zh^;mpKv^-luhNQSep??2F{*Xx|ayQ6BK2{x`uO^E|#ZOonceq8Rs z5^wy-XWnBsga-B&<{0{ph!*Hb>Cp=F^+X<%{fwQ<8gHmf2M_Y!R>4npX$X6gfN8;Q zbbU?jL+SX^T?hDs1-udwAeW8rIlIrf;%t|;@h-ijmET72jxxF92tC7T16P9vpw)Mx z0fb?4xzR%hUuW71+wK$7;ok`|fB5KL0V<(`cjn-MFR&$pXD~=A5s!#{B%-3nPfnb1 z#KwjB8=!-1>R)WTe-!^K6Z|DR;{ML6h&vPJHR?F(E&hb~)Pe(CjTSzwr}4@xR8y@| zzlDNxhnjpERiq4hLKV)O+I_$9^og;O3qBfj167C>ExBV+KMrt=Cjonb6b%nW!iRi* zHBx9<4>*a^;U*&|mCTA-Vt-l$W!%0pbF4~aZRTd9&mW8-)@wHDl(8O#q`Sz#5*W`S z3N5LNtk@kFS4?TGlw&`(bgMkP9v55au;W$Oz+pL^Dig>@RC9&N>Xcv7wxB!{NDY6K zFEE7yJH_7e!#Y@j!nfCw;+>VHr1>F`nZ`i~49F*ny|GO>^t--4kcYJ!&1(oeBB_)> z06nL?1MmhN8z!wRT*nU{G9KeZiT6qR0FNsYPnlI{AqcHCBlP3TEzR^C;L(dhrdx<; zb}1A2M+2Ma0igIu?Uj9KCST8$Z}e1Azy4=doE+Po&fQa+g)jijfdY~CcG42PcFW&}az zBh1J-9U{?P^RUIXXL~fOBHtR}oz<2^2poi|s|ckhYgfi4Bci^Jzq&f22W@u-eKK;nrXp_^C_un!0Dkc<$YGk89`@e?7h~PGD5wY z+-`f^=5K zcO7N&^ek0a&g-T;ZP8PG_Z^SZwLzv_yQ+?aUiRlY<$+;A<8eMts_#V+HE|&-jwpn7 zoHJzR$X?&imvm896ut~W)lQ9?1bOfoApUxJ3Y3OBEv$Y3Yfj@Ty&QQ_FT?QyV_p~V z-ko|@>R$=V&>{-6GkuZ?XVhg-1-2k}uZJbn?zzvMqs}7~WVI6d+Ta;Cd&1HFEn$}G zLQJ(y8-jNrTe8;b_jN$GU#JbOYG!b5@oYJqpjX#Zx2s6Uof1)6lcBv(b^o188$Y7Y zrX`s3gGbJ)q@QcUzgy>4CE|h}D%Exa^Mlo6T>N@lQ_Sp<;~crtNeNz^71#^uE_y2F z4bV$PPoZDLkA@ge_V#gF^p=CKeJ)o|3#X=F~cOhVI(|njjoS} z@?y!{tcQ;PqrO5uS*N)QAvrO`5nB0e~Avhj2_Q0Q3G_Le#oTq42WKU|a%X$Xb@eItqCz&+`SwYCK75U?K;&EmTHAv-XCvd7)Xj zp}CMIYbdlp=#L=Lh^WCI6@#G}gAfRz z04xL|ePY$i_F@=BRhDt#7`(rwGY-jX^wetf`RibU%A*h#{t3Ko`jh~D$mR%{72J07 z1eE5OMdSK^gWkUiOh*b}p9(%~)I`uPOPjtwn~Y)<0?loFNy{i(`YXUM+o**1dw{)3 zb_(^sO*4}8#t>Y~x4v1c?Kf14%(}V$W(R1B3GAz)#NPdO30El&VPqwiFc&Eu5BDSm z6PYJ;m4G>A6rGa_HEACYk|`u}ZAPcZ>`}g(u{VjS9Fi?{pi5}a)Agr%ijBy zN6z-B+7^zV~M7{!!9okZ`(*23P+jI*#_4A3`0hSSWND^-Y-`deEE&wlvB# zodJ4?lm#+_8)HH|YPy-0Y5=ASaD-`1QBzuxM_i;=%F)(g;^y73}s zHpRd;Wk(Z&Zn4CFz&I$#O9z7(Cig?>p(?&d6O?D!!QmkFUD!F{6n z2^^abB@KOAwC<7?2gLhSklPgN@*|TWsNKqRkpF_k1{ME^WA|_j4uRNOFcUT?0@j~(kx!er^GD$iWEaS#k=A@3hOm8Ay_&x-F` zkvO^$(;p8qv!Kij?PQpKll3K$^=)7{0-7YuAQx4>lOP^Y$Vnp~amr0Nq$ZQeNy?-q z-_QG}C%bm&)^RcSCR;r&k${U#@r1YyQO)|^u%u%YK>4SwZ2W^TP}}$B)T|f)$Rb(F znkdb>Ora(=;e|_RSw7&m%qX%AJJa_8AY&-GL(zxx;Y`(2Cm5>I?ha-yL9U1MK-Zfn z_`^l`Yy!IZ;cC%)Rj4piju;F(2zHN zXP+f$->6LBv^s4vRLROjr0(v*2Rk5~!))~Bbeq1qpD;PNHAQBtS(Lmn!5WckpywutRKL#WTSu|(yLK48dJ^!!HOG+cYoVX!Zbp{ zG(^JWC1m1+f>R$MuO`O&mK%-h8YfG?VN4vsnx@APPm$1uGuDX~=8-}sZEY66JW5y< ziz_!*=Dh?fq)~^g&2qvX13<6{O}Zk#X7(*N<(q&b#XZZNbhHT`qq%?wsvUvJklOF4 z#+yZ;9G3R1hDRDAoSE{uWBnF!auGm8oJjVNGnNIgwmPhbtaIH%Y`&a^|;5{~g_x&`cn`u1M|xcmQ~j**q7zti={zz0MS~ zya_858e*&Tc(aNyJ&`E_QSuG>)0++D9TI)Q@$Pe7=cDBu$uK1>`5p#)Q*!@Q0^*4a zVG?~J5`5xG?+L|vxMQaLz#sQz%(oN&1sP@o9g7KMh>@bKUC~gNqvbAPzMiEPawQiO ziEU=TMxkN0_<~}AA%A>vz&~IH=dq3CTI%YECV!TNq?o)TUKg~M z>thQE){Z>LjkGdx^!;sA^`3KP_Mp>_l?J027{PzpVSdjnfa&jx zC%w2xEH)eZ zj-w!-e?YBCh_b&@lMudEbB~M(gWX$`#KMDQ{rfH7LOzzAQ@z*Cj7qDnuVOsW8XDwy!;F7*YHIE&hTwE%?ZR0g#pE zG>ZN6b0S0#IS&m9W$_`V0sQhSRQ>D$q*Gu0j{~FutgJ5Z$!8}4RYyUu^GDzG7?N}d z)5iei`n-jnpIqxcCb9$Q^}TS0k%WzlOTAf);>)mF0te#*VvbB8#f_kjd@-x%w1#v8kl0U`>kaEf zEcC628qblNFr40pyhk>0=8~9_$IcQ^IMcY+^|!|XwI=k{ic)T?^Hw2<_p3M$$ zywR-I_JMJo3Dp~gOo@rzCGf{;Bf2f&EQTqQQ;1zVZ`;ZpYwEGK6`EIjR0H*{JgW7U zc{Q;aRqK>JgMDep*l3IXhpcxB&n#NDhGV?3ophXzZQFJ_?$~zT*tTt_)3MR9ZQC~f z?0qiIdA|Q{u2pxd<}>D+qpHU6OHeYC5zm$s-#Lr~j=85t6g8Dy9XkJ&kI`o64suhr z$4x<{<^>4QN8$YTz?b=Avb-G|uFVx}`vlycsY%?=&6E@s%1KGMEY2dOJ3^|foJoi2 zr6Ec=z9QB+vZQl@P3%H)syRhOUjuEiiPVr;@L6p>t`&Vur;|dpAmxOt{;iM}E8H8-vOO>I*gJ)yy#sXsyjEP(Kj~k z^euz)JO0VhtILGY|sdZ$^=yOi+ZmSqHzwixHfIz)q88+$VZQ5(Qob2=cOW zSN?Wg^PLDgoJdLH397R31n1 zl8d2xa*5_oT<|`w{aH$X$;*v2??usXbF>Mk%pmlMyB*u!`w16PqT_ieTJHCA7IMh4 z0t_Q&mNpSa0{m}z(Zv9r3%Bf{Qr}B*4dVk3ssi5_xexqEPbnwAfM1DK#%|nCNuT8I zRZa>EkDx8N70CxDz{Q_R9&wFW12OfkAZOTCEh`pB8c3F;e`m9%^~{N4a^wovOi1@v z{>Jt*rIMma8~>2DLEgs=%#UP3f5q+38^%Tyi9(UQv*+Bxw#b|ZJvObLGY~VE%vBHh z-R6Vo60t{X$=!L8JeKxlg&*Y;9FI|G*GDsJt4O0Pf=}rT2kd$H@qjchWO)myZM2VB z#g2Ve5Ix9+|A@N7&)^t1{6JoVK&+QFIJb5I6sY5gMK4r%+lJnXC_zZH+IJbl^R-^< zlSIE&8X*-yiaFaHKlG!{ip&8NcEKAI&w!lX@V??QpmNvZ;Z#M<#PXxh;Dr$22bcr) z{(5G}_fY*)z@xERH*e86Ad>)Y7+t4OWkA zA^aiULEUm$keWCIJ08up_Iu|@SpZMef~(~$mat`^Au`0in;L{QRP6c@iT4m!=h9qxZPPg&*OBqQeUcScv!F59_|P@A5dP4wJ4wg@=Bp zujGFR_jpEXGth8-D2#x^_g44Nbihz5wspB0y&^Y{JWNoHN_xJ0zpA8s2j9I zULg}K)V~TSM1Fgqc*Y=*uywtwHok63CJkZY!6$YbZbG*V(?aWT0&LRFz-4YP-OYRx z4OJPy_gzIq{19V7v+|sj?ij(TGO_w<@LUICh$tx5Ev$T)r9v2v?elhQvn_JQ%@%a( z=&}6O7BF_)OhspnKAS=HWy4Mpm0|9w^j-4XQXYQ%X(Z88y(9qBU{`5M^j+^z-u}SCqj$)3SJvZ9u zH6d&_zoizmRctq*wtBeew+-|C%izsHTh_~`K6_)NDE7Ne6)1m1wZQ(gW+EpWJ*HD$(eVAz zKh5fyu5?RzSv{*PDe{OfjNpx~95wU)LIh=Lu~DULm4!2BuCkrTo- z2&-wt3yy_5)h9M6rPB{3;x$5{{cqXeY#Qo~U>4z&U>06;gh3s4L)kan+o=VlM4IAs z_}U>MJcN6`rZ`84!wd)Rg&_D?Xa6EDJ5?M5&;Vn#_eQClpRrB-vWj}Dg7$6-6s78b z77saL3*}?fq)E;t%4b>fBA&YqzrUqx?eErH=K)+DSmF9m#s^eC-62^OSBZ9c#N-S3 zl$S+%I!q)enhTJ7*$k8E7;LJ%U1BrF|MWy*rseso(0P$r8*g8Bm1IP!PwEf^CT@cL zH6mnz^yw2CR1-2ul}}Uv)*S`_zN1PXOT?LWfUOKF%Hz_@gi|ifcNB}k<#cQ=MaLiqvC7!L zgd1NtDiCl@_gE_sD0T)p%Gk{mzguRM6MHoi`$bg(ElT`WN)Vmokj6PW>xS=p0W6S~ zvI12cDm$nw2>=j6U z=Jx(lQ>5p#Phe7*U?PST4ADc z?L}(Q+C}tJD{PTLv2u*k?k*D`{N(5@cRnYxn=tq3w4KP%jvGFrU`5+AUucURQ#xeK zdTpW&GLMwVPyx`c-b|q+7)l(dBOl$Q-dU?2g!|k(zj^O6DFB0hO;lPjMY4wgJPg&0 zqh}Yg^sJ?LE`y!jbv+#?nV$HJGeorCR{9-!1bOb%C05dqx-6t!RLWvz7* zeT#9{A(mj1@xlIJB7}60^K1yyvwCwPSLFfN%Gk zL-L8$)~zHG2g1ngFzW3XjH+Jf-~8@8$6mW(GxV}n$yPPE*J!dfwNNs3;eq4&9sN_JA9e_S=Z6~Zb}=vES)Gv+&8<$q(_S2(yRm-R z(^_HHw>XT!w0>L){~mm0f19FPKBqFM9p%IMC}xLr&+;@`&E3N5+&Pli$rN1%g6F=p z_D?_CEc1e33%8X;gthyu zU{Acz@WrkQqsU{Xq2Lv?;bHFO9atL6DQ5{dT0xBv9sLukhlWwW>r*RJHu1AS_8 zqAkKnodttLE@5O^g02aD1z5izd^~-J*;i%9-3G2O=YM@mfSmhf*ow%dLh-cujWfk{ zWJUe`yIAr2z8QDHui+qrRj}QAX5ilY1!qGnfFFocAX4c-I5h|Ete^nnev5D1+%RNI ziNKCfHl7UP5H5hPt%K;r)K~A!+(fKM~AX% z8susM*Fq2@>?9Me=}A&&1)P?(b&?AwY!*VNLgp2qFS}jeJ{}?9%T?ryv`AcTJgP6^mV{;s`}(W9xF%gs@IDy6Puy*7J+J}!)`fNIgVX1! zV3HrCJSX(Wu?KeBwq4EkYY&TFR$*XP<2p z30-pCTJbLjYn^$Ln|jgDGCGkQ=7?vXD5BU@_Q<0-75dr3`{q2gDPn2mF7If~!mhQ` z`KIP+5XmJqx0Agp;-G#z2O9Xn8e$0SPeqs70r-Qb0i5L1eSj8&=J!yk!0Wm zQW$9*7Fx@DN&8VVXQglG+&#GD%{yrDhs5#83o?|~Cq!{hEYL9F2rbRO`;Ira4|U4y zk^UVU-RGVrMaHHe<Pk8?`c^|GcCt1ltV{P0%odn7tWBswx^$6jBJjcg8Ok)? z_*yqNz4+VzoS}D)-G>!-jfm>yQN_1^_-bQd_1~9TzHf|$nI_fn=8!J!@3%$nGuNY` zbBx2~>3kpsrmXGp(67+Txb^ASHY5UR+hdXUP+ML&%C$GKh4$_URbKRR*7~f)S1FVRhmj&2ib_P(hkBIU6(eG`UFX60ntJAHI2+`@P zbE?x#&$08V?Fn<_VY6S~Ytq$rZn4h)vGErNQ-&dn7_>$vBzb!`Z zJ8(l6^(p4K5vi^C8ssyCnbcJVSxl4yJioq_7T|5YTx!*MM;TekB&|U)v2a8=z^KWV zpjh%~w>nZ9ap_uYygT0KwsX4RrD&a8YH=mZ{B=~R%QFO2YYd*$(V4y_Gu|HCc=+w@ z=;@g5bp<=}kTV24sMtkIbb+%gzXV@cTXfV<(yNc&YOsdtJ@UJDg+pF0+W&zLym3=C zM?b=$q%Ff*G#e&atJM2y#BP9If@#?M9HbP)zb<#%5#MN#0Mg^Rx z9G<^Z?m&Mx5`9bBZA)o71jy+F-IFHV6DD|jO$}`x6RqXnbIq=xPs(P6ut7Eq=3(-oanqpqoIX|0KwvHlBn!Y1WTzk|$PO+rNUI$=Aiih1Zq^5SZ=k>vNNm?=!q)*D5VB=C2Xka6* ztI4%v0~hFOR`!bjiPQTdYUgdWyCu>({k7`>(0fTQfY=>TIgbeGkq)bJ zAY@4L>hToBAwhFiVO;2rySIY6eJz>XozY0$2sl?$#Z5r<(L_OzT1cU)Sq2PLw{%Q0 zII2B&ELamXf}93&98jma0zf5(0C!Gj8=B zWHK+MKYqs@P+eo<5Zn2**w94~KG1mx`0w9uNqDn=_A&WKkyP1Ch9bcF`l9-DFdF=a8XKuS&Se?93WK@@#>#b2D@H$#?vGr zA4|NdBXx3&nGpvG%SifwCIy?EPezoS{%SqiML})1{A<8{R8wRQXqc;pd2HS?19xx! zJiKU3SWSrwnzuW6X|H{V`iAH~1wDy=5mtWxGc0YT6M%JSv(zm(cGYk$P$aN{=550# zcH0B2InHjGTtKu?Ii*i)7NmnSmV*YSK@{>zb1OH_H^7cy^Ub_CcuH&dX-7*UQP9Ra zWK`TOkFX!IL+suLy0u;lC-oZfMNk7Px^K{B)&=e1>A}rf=;Ul&)JebH3hj;1+UqPd z(!TJz4m)F64BsFXA5XUkNYiq~t%S~kGIX00Rs=_iIOxS1V^rlxErb`w{v;qgCUX9j ze0_a)Ib8=kB&efhzb42shyqG#R}dDzF1G4~(kaT5ih>0vyB#5dCK(YmtE7U_AL*v5V_Wc+9Gu|@@XT-D5#3T!Hl0(7 z_MI2yR^;vSc6wbmxYBz?@O!qt$zoNJ%zag^Q>}5Xd)yeFuUy{!RXO~~h%Z(WVm*Le z`YX-7izCPOjo~bb1I`!Ne^fI61IS*GARwLy|5tJKzW^SVfCvHlKL8$=i1-r%;y(c1 z+?k4K0fzXWCH0@4fw*s*BzW*t&|l10iHZuSKvhF0D<@?aTU!%HGA4RsYeOfeWaTmG zenE7;xsvF?ztLAu?QqowmXI6~cuMFM0hmv7&UG`K{<0*`I zKZ)HY`C5H4sBm8wUyyTI}dQS`A$j_79Y+krR6_38;tZ~LH zOQL}Pvif&^&Svx4OMX26K0xv1_EmM@0$68ajBd;>3%^pwrY-J|Sh*#~$ho?O(9+>L zmC4Z;X)#Gye0HBjp^nY|=46Jwn?spIy;QM4-M=f~NIMfe;{QpMws$;pOlWf4sPv;r z(mdC3LNhs(6~TW7 zMf}emnBc%TUHn$v&4y3yokadmiXl>?=s0nK)oL~>cxvdrnC!4p8b37ita>ra$YH1d zs?~$}fC7RupjZXznifo2$<}P$(kH|NKS92tCYN!dV~Fqt-U{)jR&+5lXyZ z-s`&?#+dql-204pf@`l?{moFhU7G0)iwu@MWLhqTx(c+~OYCnKD$*I$CdAep#Wuc; zS#w!v`#VW@qz`2;&Ql;fW;QWUJpm!)5sex=eJj(hvTY|c%WbB%7BRg?+sdvq;%a@O ze^1`}&`l=Ts&4V4Uq_q0@IlV9{r+ zfR|2*`cvG>7OGj>L^b|mrqVF_l5mE+cTNEIpf1*Exx83Vq9Ho+tmhtuMD7et#2%&5 zR})X#zxvr_alBt;Q_a(4b=IVXZnDP4y5veGi)QjW>aKKW*-pm0`FHZqOjKJUAWg5l zja{rPEq;qpeiMgXMv|`hbNqtV7=uFjcK?XFtT7vGp;=~rU!3= zk&wjPag^tgHS%J*AyFO_LMz)1VAFlZ-n8)u!$Pp$ola3SM3|sXUop$LR9Ux4&sSh> zC=yc|BI+PB%7mJyNVvgILd(!8ODZQLo?B6#p*F+FY-T8=P9w#N-%I$qw0OJ+^JPM= z(Oek6dhQ^))UOd(r5|w6)O$goFn;x!K$2psU4zz8W1GbbsS6>Z0Wj-^f$e7i{=vTi zZ<@Osw2@qxZW!zR-e5d{8vhcZri+y`IRStznW%ds%W%(#ewaUQmNXJ0Z8epESqSOnI2R z^Ym)!8-cQILG2!r^C!X!kYhmayR(x%;hcg%anK}+B$^3+#-c~U5c>*zGp1VRK6?~x zxl5{2)Jo%0V*oeo3tc

uE2C{NEBRZE@?LA`E|p_t~J_sGe94ZjE6?Hi=`2AqO5^ z0gjbcl#`)nspsD;zTBt^bU4|es!sR~@$RRGAl+iCzx?$E6Xer|JCa2I6xN4yB+0{72t-6PpxYm1kSRCb(!zGe4!wKUW zXjE_2Vvk9Quu&hNMLXJe$Yb4z67(o{5(rzP?%w&q9izMK6HEAR^P$=zTg#z5OESVE zB@Lsw-8V=lU-IHOfdr_fH?X!S%wqMv5jcI!8`gqP;9U5nE{tJ~xdrFh;RmYJ^0By9 z6dOe@k?_2al2^~uan6in1n`9f@C641JS2(1MMsiRkE~%NiTt9(rGZ6HeZvSx3+gG9xJb^zthe5vrPye{j)uU>!C;R zJ5%rG2PEcb4}8kd-nW;@WWI;8Z0qsH2oHOK{{hJV#lHU+AlX@A(w@@356xV}R6|x6 ztW;!LF5KqHX2cfA|DvYTJ9{G!8VJZa<$qE0KVit!aYk-HDr5yT8gPCg{j2MD43Oe0 zZhs1A@`ncV2r|qMnU)!O+M3oVg!Y;2aA+HKsLtBY5+o<1cg4C{^^-?zNB-eR7*-X@%`d=7q+R2+KU zl?CdJm!=+@q<4}uTmj!C3pG0LwfQ}c-}F2;LbHj6ACXKRNqb-Fw>x%eGiLs-^@8Xt7jN~zZZLdsua4hz9vt1l3V*$&)D6bbRTz{+~^Q}#Q0tne&v59 zPkn`l2@D^_-o7LbduXJ+YZW^Ed5#pn8hPgDm%TNNc4D1eSOyl?>wpkOcQRXUPkG?J z^`zaI+rf>Ub}t$%WFW<_x1Jv!vpldvaS)g2qNKUgJ3SWb#wa>fP^3e3&jfeLvg6ktcl4RhdJm20i=CBs z>tHl4hee0-o*FL4n>$`A9u=ZSe0HLik0!i{2; z^tTb|WCaD>bQKJ}F>b>jS05WM-MQ7lkj3q`lBSTy#zvrx4MH`YEcwKmNV_ma;qM}r zKc$v7RCN14@S06+L}VD3v$7q7jI`H~t!s7kI6&wfdl1e4P8{*>@1*k}vbuS>1Xx%O zj`W4Qw{WCK4a}*zOgg)iL5GgRiSz8`JKHh${k|RfM!R|5I)_V#Gi+4VdsC3a3lkrN*42VMzEV!P&-u9~qS3l5dYC z8?U3@l{2E2dK@+^S8yXKWB0Tu`G@W=8K3&UTG28EH#m#biod8O|tde zNrO#5vb2C2#SYR1j8&BKXcrP!A001a9@))lO$|3HT6+e4P#3OqJm@*use;(5(`(Vk zd$I(J-$&WwlXKIYceq!-^i9h3tu+bNZ_=l;K zlypyA6$Ap)Jf1*}F%hB06NWD$g4{H5j9OZOd;{EK6s#&Un0ugfv41-1=nOX`2*8%K z(mk>%RIy;g?qb_JIB?JW1a$3<6Zg>(Sx$C9gZ;n4tk22e(Nn}O-0d+3TN=aH7V+2F zykbP+FV?ltyZ4zHnwf1{&F^E~U8V}mc{B6^Ix;RJb~iD=v;80pXcMZWi;dK9HdW1l z2av4=O;q>|$9EUh;1YeK^hlNma1@@2JHruW`*jJ{9j2qiE^Rrs*N%!u>}?{Yna5Ds zOE;d`9M{Q;A3yYi-{3n_<&kE1iHqy zLGAa-oMNXe6uiyH5SpbL3!AytP`A^p{iJnFDcWW>%}DMA!U)P;O;EVd0ggPN6T?;C zFKyCnx4?+UnMM%VY`#YABR&%6ZjcTz7+JC7Q>Rn^b;Ygksntn8H4v6_cb&7trl}e| z5Tb;YbuE$>TY73}`@>1*8=nVtAjO(qB=+`V+bJj`CfMQ$S@!ISdi|C_IQiSLc6Ls} z%G|9-NRD6CBZ8bw^o8Tw4`(Lem9k`YwH^9l^kVQvBZ3RqNMf%#SWVTOi(_*~dBa*M z^$9ltvY#o~ev{r=%VGZS!-Gz4{eDLE%1IxiNS|-*_Eq*E80#%F5W#E+3&7NU$1%9p-8C_q45Fq+mT#Jz-5Dbq! zcomxg-L<2sR(Gz$;W10=4qpllJ-d^r?upJ{uS?{J=Y>A|>hQZqh`fzWr&*%Uf=4NP zi$kk%YhCGpBwKP7@zoy*F(~@95rsNC{yuJT1f8cWv|ezQ`{EoyDv*g84^r96SHZ5$ zLg!);9RlGbg|;d(DJk`IBSYCjD~QeHqU@JnZ7O>k)*oKU?s(4C#2&E+fAC}-8Onf^ zJQNeH0|@?kFPzZ}V3_sA#e3$rn5Xs0U9G zFPljI;!RD)^mdl?<6s&0!g8n)vFoEhZAhxv>(m4hgT039!3sYV2$o43i8C&7cNuVM zi4|y3uh1c~ieZV>7n3&TH-8sVVge)}is{Er_FvMZLlZ1yfH}51wP6%07Ceej4{Drx z^Wo`ACh52h#!~yP!OC>;d25XM(f)%4?{>>R9MaZvcgCj7UXk7$9eNjl2S@TLE~X`^?-Jo5{%he41O zy6zN}>cp^oXWGHgu7hU=Wus*CAi~6AC5~g|#g3IPIP)LLiRdq-@i#`CX+x2QvOrff zCI0ASb+m^1S<#C6DcmQmJzuMiNn9jN(p4*?Hk3Pj;49}Mhe?&)AZ~R-1Ao{;nEei%D2E&?oP88CM@fw7;mi1n*v>E+ zBgOrGr^_tF(4ehI$kEUE2v4&gY559&5nIrb3Sv-?x~Afq+HTMGYtElydE`7kmd7VJ zR}JerktEuQB)kL(s^@C&J9K9;+lSo$THSuiDGI<{(!y<;@rnRgt&`gpmJPUsMAV13 zI;4cy@rsc~#Zy|ShQsA1vaYTW7+0=1?~s_}O+ryl2%P)d;JA~{!TTj%SSCf{1KNYi-EOm_Z)|J-! zV^;ai%59hnb_~ZIduhaD4GzJEsz*m{hej#(fBYyo+}Y=rAXmI&R(jwXO%+NEEiJxb z#rE=l-iJ1S=ajq8Es~=1PgM2PoG0`P08-?Z$XNaYZO)J@cYx-#WN#cKJ9LFb7eb#ASdP>BHVVeFGGvH)R`X#yq5&uBogzBrT`2FWFY;0dvD{#9p zmr!L?U)>-YEEWSp_4+rB5H=;jJkDJYx2OP2FM45@49+HvVWthYE#E5yqU4M2Vdm>VV`@qKa z4rHHID9hd~LhsCu|9DL@g_-ZP_n65$?w7c9VhstEW5A7hVR}0}Z~Sux+O6c1aEM<} zvB#>xTiptl;~M6TG1!sl{PW%J)8H7)9~Vxp(LE}|caJ%ph+;$YH#pW49#F=5TN$sV zYOAdAXzZKz`EXPPDR84gQvX?^zxUWJB>X{V`^A$X)m`dw4-IijRG#uFE2Qbl*Qs!< zV-9_!oT2*{&+Sj6lB`Pdkfbty-g{6IdxR8^MB+B;)$H*5Zq{y=H`PJQ9%3?fFxk+S zK+Cv&y-{^$`{)K)bdFy`2JqAw^c2ck#neGd_k|YWF1Lg`{f9FR&H0a1bnePosy|C? zHK?h=_4viYebu^s6Vjn{I5-Neu(5?;gbM$7{ly)AMjwo@=9JFB=g4F524dRd&>cEz zk778JPpOvXMxxOR`gvG(lM+XM}an`usA}kN9MfL z%lgNtEOnc~z23KP@Qk9Q9NGu_0}d67I1ZL>AMl9gKr?pzAX$e)y#lmHN9q$DrsP(^{}4zTu7N*Ib3!qeyj;QKw2S&z3V z(&Zk^HdPK98EUoVgT^m#!{RtJ#o_imH^#}8xV0)zgR2V@G{=?WABFq$H;BtVaM2<3 zi}8_5{g-&=WI%nXL=Tqay9L(BW<|D8WFtfEpc;#x5@r6)N^ow zm}fMba5MZoj6~!W%sKMvgVpuEfI|j_sy5io0dBB9IxcERfI< z1AdGF5;#xWkTA zM0<-iNzW_(7;N)3_Mj+h+S1&zt2Ex8&9q0@<@p(V6>zo@z3PLcJW(3AA|S`|!qReW zsNMIZL?<9f^Kxh3&M=MP4N>z1qdctyxG9|P1;#2-0>eF@VT%fVlLq&k5ptb#U|c=W zT{g^DGa&I%mg1CXp zxD|2r2V|~Kty}!!PV!6GQVTR(ovDT(?359Uwo}FlUYSgeRhL~$%EPhqG==~Hxln;@ zU;&*1oYb4)D1(DBuP7-s#$d>30q(Xnr}RGzio8Csj$FZyk7dIj75RdqIK(?-*U)8o z!s1xCUDe4Hd4Fc1@SPloJ#oX&_!ZDzAT#>53Qpb$u&>+8aJC9x-cfG{$NpyP2OH`Y zp1ruVMFSf^TVlJOUS55WdUs}n1ze~of);6kt% zupq$grxEnpg)jB){-sSdW8!T37D_+kR4Kig@J%R7#mgbGsTi6(t5j)@_yA=<>@H1NvJe$8bZa5 z=S=ZGSA~Q{PuY@RQli=b6K;h5``z+7xT(Mmd1dQCt0x{RmRrf=qN|FOz3aTrz{yRU z(X1^M*Z0`&|JJnrmw)?znY(FObu6fRL=cc)w5h$ruvn>#!-#mP-MQ=tsbiWLus}il zBbUz{&9n^@%w9*yz<8|@ZuMo<1X?Xjsz0q{{varHx43nC@M9pPbnR;!T3VddPK^#V zB{mr)e-c+6b|Ff`b#$5x2=GZu!?U>M5((w6*gF&MyO2s9v!1?a$G{ru435gkZ>BPV zn_r)9UsoT;!B;+)S|D%7Rd5^6S-{Lt3nt#e?K@laJuN0a(vUHuS1wQ=A;Iqa?MPd1 zSdDdeLVfk?O!LE==xsW1AI%t+rotC zioT+H=DB(IVtNmJevCDGriA>e+SXrRjs#+`e#-TBw>yA*DMft^09&uOo{9V)yLZ=Q z09|L|qjjk-<-VcD4cjfAPIh0+E5`ZO+hSSA1c!>ogrU5BWe(x%@G|b z^KbZv!9>_fnQ#7D{GFZD7BRN@c;`BgkXxoq@3PR&mW9+e1{Tj@44+-3&1Ad~J-CpR z)adS2o<*C8iPoxv-y(g-lf?+TI)|0K&RS0}$=P=Z5Zcu?BI z>l)HIR}#5M;Q$vyQ~W>!hJz}S*37fJ@u%~lgXiucR}Q~nRB3mROA5ciKP}kEr{6_l zJf?IOWDhk-(MR&<1h$&iWo!I&8Q zr2Z0=`iz9d<1Da0V#ind+|76F0fpq;zmXT+GEfS6xY*4Hyda+-?95n;Lha`EO57C` zxT};BeORHVN}~)(ZAFE}WmrcJ3AB$xaUxT7`b_&U>1OPgfiSQ*uxqz}R@YC1Unzg3 zhU(atWwQ+)bQQLZ=wM#c3@O_x{6J@sMv^$(n0bzy9zi8Q9i60ih3-4V;0wtMDx#_5 zra4~^ZOs--b3qFb&<{20y~mDhHfzigqsoS}$aYfc=?ImRC%B8>w2Hx*A(Lq1NpfH( zK9~NKwEs7&1FUa3V84{Ir=aDteE>k59uY)^DfW*jk_>KX{sUFVYfx#NrW-PLsyS&^ z+TAI}BiN~4w>c6Km!H%g3N#dRZ44no>mwUg1f;Yvf`Go&!5c$vH|nY)}II;62K5E~EuIop5u?#N8Ec&l)aaGm|)rn4Oj*xW<8fOHB+3S8Yfb>+LP zwkqWz?B&=!Jg`fqoFj~e!7angQ#it?g9TP6g2S-^VWPin>CvOohm2KyCC6|4oY-efMo0cmcRj z904lus8tl6skQmLmjBH>Ra+JD!dNYxjW+7;981>>8cL`RYAS)@U}#oU-L>Xjd+_$I z_?wGZ=8Q&wp0!yuOKhoF9*>Cjy0l(sqx%A8=#~STV5-UZ1jU4}bb{HIyf;ZPW>1sC zBWItG8P=|h&bIPmS|UxO8EEO$3z)R`&B$gUfS6f)b< zB$dG_i(Q%XdOp!QJyi1MzbI;V832i`$12T#l&Q1o1k~d%SYN7pyjU6(Z=Q;^XT}i8 z3(VN+pfXT}F3mgKNA8M*;~9!5VE@5BnPLyML5Xj5adr4Lb;uzc-(A?Li(IKCFDop@ zY`7dGRJ_&ow31c$hnvfN5Js!iLwrL26=mCFMybZXalqnS9V2 zH1`>^)PaR!lsu_hIk9E0zG5MGkC-Fm0FgDk-XC3EAkl1Mkw;$2eeQ`ShLqX%4jG<8 zA7f@SgSnGHUsK8xF1>~n^ib$o5DD8DjGzP(Av289Z^;(&@O#-35G(4@KEw0~_cdAk z3nc=ll`o;rd2(mHboqoT77n;o{fQC3AKgJU0-O2Xn#43sG96)jGVZEMmcNs1QQOq| z*UD*tt)~JD*7hw`pM5LgK*pzL0)3#8)4;lN{^PTXC$Ly+z%;r@!_0VFK2t@K#4gF- z&`Y}efvtT=du8VwxZ}Wxq|vDFAt(Qom>vjD;zaNQl(&d*@UBzDwp{lKxZnM` z$(S|qdRxbsh~c`ceum@J4@vCZXT)lHnAl61$-=hyf5$$ibWzzF#K zW(d9dTi|5U!i>*h+G_vI6GZ&ZH5!pqXX}8+wrB}M_|ma!&G22Hl^ouSf-7=OFS&J! z;0pcwwgt^We-(E^zHui*#^z+h(PG8T9Yj3Gr}?H>a!PZL#0rEHp%xMEi;&=2yVPHJ z(4dgwFvC;*WQZDE*eo(+Xj6jE16kOmj>@;3A-cf^1sQ z1@_ZVxw;-@{Lj5~uadv($jKtISt8s0z9uMYop!dF^c4qmHop4q_j zxSR59PEK=oh)|zpTbLB58IeyxoSC|iq3ZCyV$D-bHod}l#qOYXfrE&FzA_o z?vxI2p4ew(YaP&_Sw>#aMK}~%oTerZA@C$m40mX-p!F#vwm4 zcS?(K0)YSjh`}m6+qs}Xzte#h6l@8?EqzF);x!=hM9n@oz>UT|7JsX9+Xt{&yKRQy zE!aE7g%9x6>t#T`fd-2u>WDBR>>k^tZ#rXOUJpgO8Fm%p6ZlsWY|XZ`hbm;=Ni{K| zi&F!b!*z-9G8)|dhX_pE(pYY5PVt8s*94MdIW!zf?M(M3tX3qT;}Qy1 z8(82*({31E;Q=WeM8h1%$ZWHP=c0Nna{Nm3RfV-rWC3)=lcDuFsf~{p8NxaM2}(tB z%}l9CH*56{3;e4}wA<#$SgaYQa5N$IxDsZSe;V8K5tK9K9)xyOW1e)GhOsm~5;w?A zGp47+lbOyu50RArA7Aer9qG6I33rDb+qP}nM#r{oRmZmNif!ArtqwcvIGKF!y}$R) zytCG<(y3E_KF_Xw_TC@h7kYktoqi&G$mgluQcSa0 zg1scB2(6eByo-fbE+=?L=#JzAVB|q@ZlPIP;SsEabb|4rUQEoZ#~UR@cIc`m!7=JR zro@O?Owh#Jk&FjJchut-TaBRGA7MB1$ z6cWV+JGLGb1>k}q0C|JO9XN7|ORiZEd5_LJhRi2sKS1UspzQKpVPPN*C~@uSh01b| zNxvW#!oc(fJ20~c!%VIx2tMH$#MRn+GdE;D^X~Bt|5$TKLIWhZ&7!}iHi>NVB=m+* zrieL|$R`TG+iCy(fQkJNZ_7Ri`_^x#=WX2FA_BNdAc2cCYJwgjSx7q{C8||$;U?!;( z>;z~Ap8McIQ)84FFv81F7=0IekTywMovJvU`-Pp$>!d~W%l_=@FY{-aImn97sh!Z^ z_0u8;l&X82)F&ej5ey$OB@RhvYDi(En$$?S&`s+-$TXztyd7#`@Z6x%7h|l9GEAa0 z5HHUMG|BUa0O!+2j&4Any0)mY4yUqbIA5WqNxFPvlnYyGY8THgX^;bzxwl_QxpPJp zY2qao@sF(;R5nLuk`8YzJL7Y;)9fmEKgvxxPMsriA)l~zOJZ}oN-yEdXNx#6ri->Px z1p{s;*X!{1i!7CrveVtSUwzGv9|%6*u2FvR+*pD@)H`k_^%}3%?A@SmhEcFaks}Uh z(7^BJ0YXHoIHO3gbD|E)z_e~$KO zf}@SC*wI1@hYzqf!FW+wc^MF$JAL$CK+=&mCR&+)$eT&~Uz*r(#C2P9V+|Q}JWo0V zGa^IZLs#Z9;+bzY0{_11?P}(b#Nl$8$!B7X07ySAc2%N$>i9RB0wKYxr5LvC1Sk=) zuAP((w!$vQZBCzB??)ybC8%9{N2?Q ztks;3T{X4iu3mVl*d(q~)q2&WkKIO>K&BnF?_*iLpr;wiky^T^CJ^JsVotrr=<;A) z09Surc0XAyAxy~td{s>Fb!;9^Ez^9A8(}tAYcv(d`K*L|brAPzN&+W%j}e#{RsUe- zeBlm4ykjMU*ZR7Pn(H_Lt8`f~85NA#)uKOBr8>?%YtZsPD!J)U;}-gia8++fYRy;) zHU6dt*TxO-G>wJOQY50vaTN5`f?9( zmOQF-wXasH%gDuz)rl}U2x-M{q`t9_n)2W;Q>Taog?eKxhjCR7n|0Ks+rY`wRxEF} zBo|rZkdIFeYCI0Z(B4aR=r6vSL&EclBZeO0svB0X6=(5*SQfeYRG{HVyyt=;z!p@q zD!W%Ha_kPoxZlk=#BwnSv)LLfabDwFdHmm_JF?7^nY3%4fl9LCiTH(iCiIxFz2$_3 zYG$X%nAGd~7$MlxR-BsO9+Q!Oq~@OA)SYXvQV;UGJ57?d01wk}oMv^a z9i9H`LtS$^TAcoJr+>u}c2mR-U(SCvphMR_)?>^by_awGGwI_$0;NJ)hTW4`EJ!0vTJ41n0)&mJ|O5uA}lJsrwUUJ%32&>p>yDp zLAnW`5vhd&KYhQu72^nC04fv|AJK;ROtkz@a#1bFT0^&t4BF92RvU;%)t9cs)>l=r zFZ44Vs@gfZWXjb;jbV{oaYT02s_QMt5!`vEMO<9SGrJ}3z(u=7?kg0zr~YW&e)Fc{ z2!LA+b*pAiEuQt4&nwrxwx?)smDrLiOp0{=(~{jo$R632je$7f2AC4c6a$>_e$|Lt z$MF{Z;Eng9%JDmO#}bYBQfDW1$0w>L@Ydc1cwa{BVuLDk&a>RzVlMFK(?WlK7iBy+ zuPUk~UY?~%)I(v+m7C#YI{e1PpAOMksqC#Uq)^+woKs%!@5Yx$6QTa=d@}86b9_$& zF>{rJ@B(zmGSyi4LPy3lJn`aY4w&E>_xq2Gu|Z_Nzl!6Uv?DMnh0aBn%tn@^-r(&PnG+Q>W)I=LJy<{17)g|!P0>U%+5u&OzeVWPr4B8RQPuH~S(9LBvb!2VHrt(M`cdhIo z(={mNMUgk}0FXLgXzewovPNvvJu6}5rJ$mIRQ9EGv~sDkwCwUUTNSe!eUp>{xS)CD zJE@3oHM4IOs(v9r{R-#&sxx8jrQl{V`oSAY1dC_MmQY?nncLw%b(c6cN|20SB!rW)PK{^|0F1+A3uJ4`zJ#gLXnx6eK8cYQs+v)W2Op{GhwvA>flCx z{}+Fu?(#FB4dmN5Uhu@5SaJZ@f9MLTwh9Z1sDAAj?a}?@rJa-@%zD6nhi-pD8=Egy zX13-?n2^Z$a;>nf+=f5)XKe0J5TfzZ_Yca0w(MFkVuSgd6W+(|ldOdW?>{v=0x{o7 zW7aH>xL`8)%`b)8Mpph~R|Hiz#Vsg$+hnxh+-lgp^O&HQs|FM>@ zQc~p?bx-0uGG0{7F10;Z(xeX?ZZ<15U9vlqXDz{W?a03LUaiGaNr;UUong10QKze#9in-16E{NgUEx|)kiiR#cOdJR zC+`X4D5;|^Q}aOr6^C_nCGgC3u3lEA7v$K3mU%DXm$htbRM9U~7MjVT^fAIQLsgc8 z#UCdsopa@6@^s-XGLh5Hh_@m6T9&$Xw#hHZmL>GImtRf>p##@sN*-ajTDIorS`D8T z-XmoVCBNlT7}?P_D0Rd-1^IL|C(MR!&B~}#8iMD!L&GbiJr~NYLkc>2GTme)+7%Oq zU_(zML9;gO_nb+>Cwg|HLe8M13)0pqiw=LTcYRjDo7^euP1n5)D3{fz%Sd5yrD{#d zFenJuq^i^axLn#lG8TjF(*rf#KiGH^wQ`!^nHgdij!+dvM#x-6EhJwn5x~Z_;OFs0qOkcFzZjT7;2$ zF(H2ANj!4L3ZbT#<#?IHg0~$-*cpe-hVqyk+{4eyrXj|E+X{l>m4{b!gzA)pfBXJV zK>YW<{-1#OpLoE5b0A6p{r1fSGPNg+6eG1agb^Y2UJ?xjK>E|p+0e{X)ydV_Mb6O9 z(A?BX-O13w!PM#BQSlGo14SLx&#u0fv@&cKL+~6%8SeWgVbjmL#$Ug}r3|pkU0*CU z?MBz9?9AL%4iqeV9!!Vj^-mkAjYs5PiwRx|xqYt+U`3N+77*6Fui8$sz8*QwcaH+G zevl3T{2^8^#`u1sEENJ33HoTMtQDr@VeQtq;SG*SU?TOZvB;3z55JE+xrKw1C_%9@M*~`lNFwT?2ZRPFr8ACR9n>cE02L z-$D=|;ic8oYHmPlPn_f1bz4gbUa55@8G}+f(PHJS$1cfcBb0f$0;w4{YBS19v)x+2 ze!afsdXuL6z)7>3zQtNS5?8lG35zXo>k@YKOuO5(+SYG zWiAx=I3rVBMNP4tV8k6easnA)z7(l18Gy`jAI1>sq5$DwA!q?#&F4$A6Ifqpv~^Zg z3V8b!{PbY?Q`fAd-R!UxD6-kxpyNE$R8UT^STi1SCK4-E#BOq*^wN-SUeX3&Y~Klb z6m9D5X{ca=aOO6 z8qH~q75%&DOA3gA(~sO@_3PPvlqtGK8}N%>DdUooXgB;O;jf|0NSuPIm~hL~SqDer zHHS4&8TAfz>5SSutAvfQZtMcsmX7&32>SbAO+S8BSlx4<0XKV>d~C|pRmVU%L(CQUCcpm4KY-^I!A@&4P>PM(A(f zIB-)5RlZ8P?j|l2K(Pjt59$)$hgrkeg*-WQ#Sfx9q|y*FCTdCJfF+$#CopM3%*Bwg zD^ktXi^_%uA8N4ckwTNIxt+J z0miB`}^GC!|c_1QqK0nt6So!F5`WE2k4w4wHG92@hg(ysTsP@U? zBHls7hz!VtOV$>cK>SmSlW=A+xkaV>|?O?I%s z+)ZMrrNc3xRV2f4U`W!=G)pd|MU`K$6gEspXXR=cAEbwk*ZtVtg|NsWnYKOe=b2Y% zzV#xgei!mL=jY`XvW|MQBxh4sA)U9DZcBMnL%#X*I^e-NgG2JVYP%fH@&So%U4Q3m zjA;Q#1mm(0rQ@@bD%HVbKPzni0-^qqyMH(}^PPZ~v6s46Dqmz?ZmNm@vZ{_#bS5iWG|T&3ms0Z6hBttY~?iYaxM>PUWE0??g+-c zfg5*;2Y?*d)!rs_9{oFjA5OHL)hlvmx=0ua@-k6Tg$1jQ7C}eLAu}$9PONo&(eBng0-(tz<#Y(>N^k)#%P{vgHjqBd|o#S7!t9W!56ZGZgB1Gh;3{i3!Q<;7-h zZd0nTvJCm71yNI^o(kbGjME2c8xwt#0id2SDw58d4~@0D+vAqcgru60m`L}&y#Xdw zSj$Wg{S!nuTC9#C??MwjigT#O1sjT`lmTJmotH*o2IbZGktdu%^ALFSwLanjv2~AIF zE{5GrAFXHDUiD24wr4=b=&38HE%C31|0g6GU{DRs*MK42{gxYpTif(aYAE*x!>&V6 zRkc!3a7h0qB53_$-q9ybSIAP~OW3ZbtQ8_80t`ZRa43^3B!BVR9VruM(1E05p>=~p z9FO8GzlmvU5ll<`*Z3frsb63OU^f?J81;ueS|AmF3k(U6(H#1m_xNNn9PpQ}biM6L z^{(A`hMM`Kcz@?9HtMt71MTy`GjElq6IU+j_Vxn;~!>q2hxLKUlu^=~P$>mZ?ZJ~@y<)TydbQU699XMpY#!R)w%#DcmF$N)Q^=V@XT`% z9}O$CX|}MhW85Cms^9f@86XOe2CNPiMb%Pv=fMx|lrJNCHREiK=SsX6cSL>Ub{cm( z9>knaSec^5sgNjJkY6_JyofeT^i%{rBEXEmQXQWx)5TsB?JRnc?Z2 zijxbbh5+2{mG2}_Rf=CkPk*}2n;~X?RLnT2MYbshUr_= znJ`Pw+T=68cB9j48*XaJ%K08jSJEFp4-fRp{c$TlzH$93JN^x;+>k~{w5Ab4Jb#&f z|8m_EiJXXHPpbc1g_Bf?AJ%nI|ED`98f2JC09*4VbBtWv zm^-)TG;t)?p$AmD>ng5IrbeXeulK)Zl@ug5%i(Z`JTOhGqczAbj>|AB_v&?$!{nK7 zXjHe=Z9!$bx83UloW%nQ@Fnc+2!`u(((cF$Ew}-Hey0={lL2HV%Kz@7en!oca83n> zD%thqTnQKIP^s{SV`WmC<~t9QGwq?Ekfg2YQgwtfRm>geX{nuB-cM6@K;}q4Gg8xR z;hqgfJ_;5lFn#I7rEEkd9~DNI)8)s^V`1ExY>ELpYB&7K!}io(U)f(Z$5qbcUtjHB z)13~q850kns{z$T!&Z~Ucjc#zk*C$@+%{y{OHv#Uvr=96as=$gIMt+X2sd7JfzBPA z?tCY9{Y#CDfjqUr=w5XgLOiJIb8UR0SHP^$8A{Wdf*~tnT{)b9S1)EL#x!a>piM9Q zcDKv*-{QcQ8TfRWxA-765c%t75&pinL{kf763W8VPXO7t{`83>>ne&6Rz3=4D1}4> zDCh5yHgSj-sPDOC#tgZ;^qU6@e{oA*X8#;M2m3%r;0vf6F`5z7RtR&pP390ZX6FM< zelN1~oO@(D-eWkk7)|j6l-qqW9v_0(%zV2>CN!wIOF8)Fg4jD$5T z_zpJBp9T=;4rV5X-1NYiFl&kS5SNae3tQcmPp(^?Bs5O$uXbfWar!f3=&B1rlNv~b z)aSq&2w#!@L3&JA(T;HT=1wAvwbE zb|+@Ii@eT0=gf4tn5WbJ2P}r?0@6K@gfj`auq&!4Rj1iHCJ(iQ znGZlEs>!ALnF&d#*(=gomT!&qG^`2fs)YY1K>h{a+m1g$R9BE94XVao1iL zmyJkIl*6r4@wt7(QbfW>K{o+d_puL`_!2;AXV!P+y7^#NGm zJE7|1cAn&uGUF1@FH<{nvi1<9m%h`OymBu#lpb^PQ%@`Fmd!Ky3xry*$zH73<8Dr! z@3)gAoIT?*m+$xapT%^Ej!Wqs6$XkG`{~94o?Bg)>9(m4T&RO1a&)X=Mvdjg62*E| z(wyDQfj`y7mSN^|@8`cN1&P*Z(q|7efZQpJkb9j3FV8~;lHLIug)kdXdl)1-3b~vpeaIwyhgIb zn(WN`hDJQ3DMo?OHl?PLy-+T6x`)9J)vLPSr+I9*PhXQ08t-o+0mbPFoR>*%{jzDN z_YA)Y=5p69`{RmEuB6$ArPo^z=B~OPd9S#3p0C=@d;C5^alXwz$zj@JMjF5jlW+zU z-U1wH1MCfip>LhIvHBdd;tEGdJU}Q0ID^Z66nH3~@sT~qhdM&+Bq=8wa|gImiYf+a z1U67QPL8$=*U<$yb29LqD!Z$3vvbPEFc18#M^x)prq4_viSVp4DU}~W4)hvpWHmQi zHd|#(<|)xrRcW_MQuZ8~k5U#8i6o;}4Fb?b2e~R~9Gq1W9+@e$>|1zIm#o2CvU9Rz zY|RbSa+IV~iEvKqSc$VmwAU&(SCx+Hc(% zhVr5?Lu%{Wtu~n7-z&muN#vw%Wm2^0BZ&v5KdB5tqRIAT7pwYSJd8}yKp>z@>0PbX z(%cvOpP8C2j)xunM+Xd<6@!7+8I%-Yzqf(QK;r>k?h@5*%+ z@A5oV2UM*N-|&I&jIe_X^df{4x>FcrVGIqF^NFO5@c{;JjKKx^6rd=<1qL=;oc@R@ z_suv4%eep9)dlZ~WBNc=M|*?r0S(?7+yP+w;fM>W}T;(sXca$%uaW);2(`~`lOizv5E$FRGQI_Emb{rH* zNjO&Ve!1MPHfCF$SjFy~m&g3k8uT_9oFj8wUUV(^u_uz#Z`6D-=?Q46!Mb=Ch{HBr zZMP_i4=^qD8mCemn5Br02XO11V?(rakbJKEC@;=r6{>leA!s~jyZ zRu9Kmp2x!DZ$vx;)BxNo(t&ylK$mkpF#o486B`BHU3ERGwX-H648=hvGRw|hg0LDF zJq-OGEn4Y&=f`G*)SFGFEtf;f%<@8I^XPG5zDa7o`~cUB&qwdC}dd-vxZ?rxr#j{y^rnX>Jp-A^>JU zW%VG>FJ8WN@3e^{%s2Qa{Q?lm&s^qUBTIr@NwynbmBMV9^wq*p&Cneb(Q(xC01%qL zeWI~?7WRS!tQ=mOT#^w#9nIe*VTw@4eox2Nm;HL9^v)d7~tlS=x>aOzbN zeO40rl-l!d-2-rI_LxhR#6H=dcL}HCu;a&(&Cfb_mm9XLe{cI{KmQV>{FD^X6Ro!? z1d(fqnR|eFx5o)1XzG`w@dE;;1F{E+rH2> z&6uFCF4xC9uT$7jRPymm`9rSdx?lIXC$wp0jy3i!QWL=L{k#H(0vK#h)aBsseYvUb z7Nm7B2F&*K5UnG^eW3Fg`&)>#$BsOKu-^w8<;Ui`j0h7H426#9+Zo*HzjZZid{T%C zh*lr~`i9T#lrpwQ$TJYVeD@BgG&J@{j@T69yQAH=+)7SvdXL#Mh^=>w_CO+%zj5ywOEuQh+Zb+n#9az!JQ(~c zzMGH@J~1Tzj^Y1&$a*3846p4hxw1d)Mhv?J1E>gGaEJ~mv35MJ!;KV zA#700FU28nfTg*ep^K}NsgRAiy_2Pjg{_FaotvqX%l|Z|lU23tztjXjhQHtjn!R#u z=+OSa#Z=Br7}Sg)!~;^DGtw`io?_}{6P9|6S4$#tpCRpHBttaO$W9`k9Aok@Mb=U+&`FDY7z_~|NGzR+?RXQD{si_*9qj;Bs;;@o+T-j9 z=8p81sA*0;meOTwB4f0Z{Ti6^XBNyKf9)XRC)-i5o$0Kx5xpYSZf+A#_SNtIEi~Q! z;myow`>QitbK?>*k@i*2GLlTfo%byV033EfLLt}>3rAoyF<2hd)G1CPKX#ZJ8Y`#nG;Tf)xJcPQS=K2 z89xB_=|oK{7iTXbh85W#-ZlIBeYR#VVD3Q^j7wE%M$r@X9-F8+bZ#ySkM%qWKZM(M z%)^46j5ss8m(=abx>WN&nvVZeS65%2)mY1%D?ID7VGE;s&XS#7rF|3}gWa-Y-hQ59 zxqub^EPulT#uU8**=>5ZeR_eki>GT;acn&ff{8|z1zzu*$q zwW+M(yO_~8v(HR13l`0w4j>d`}hh!_1|x8%|At>Y;8#rYz_00tchM&{Kmwz z!{}nWg|6HOdpzNcU2wwDA+#mo@$!f;3g30rVUE@4<#RY{xo@}%dGewG;g1H$L}osa zi4l(h$ge^+G^G0)Pb!(Ldg7?+rRSEo{jd-0GJJZ)>H-oq?)VjyyArg!TztRjHeoff z$ls{mf))mqMV)`Ku!8RCBif~5a!XX87Tma}aX4)EQmEKF#y{b#k$vin(lTZ#1OL(!${dvJ73(}vLH-lR|6g2woq7LH3D7HJ2Y;onh#i4UMZyHbNHw71fNI$} zAOZaQ9BukGV17XK?VGh%qGAwkY8*QnRO)mX8YDmm#z*JpmlO_<`|-4_UBdC2?D}kD zQo`|E5(u4~3{kec)HtvSWzRdm=LN@{*$1KkRWx~teQpO&j?CU zq`9WNiqZDg+V0h#?n`1~?K|#Q*(~mQWA{Q8uZ2DOSA36~&fSmC)1O(+B{QAR$s^wl z{BHqN3^x6lT^=xgy@3RS2scLqI*&-*o8zgmw@1eKop1uv#R}I)iF$9S+h41@BlvEg z;lSJ>!2WzT_IB~!$)-mO#`raUb^8h^!N`nv_=_+0)%-P#Ug_}b*3-A0p3zju=UXy{ zr)wfOg1`N;n^Sm?7Og8V4L1|m{QJRNJ^(<{nBoMt&uH{(>zrSA)Xfo7w@)ww z4!`I>vDO~5Fy?zG*nV3uFgH?gJ$u2<*HQ?yefCyQNszh2jtG7*J(qpv*F1XVeknYD zD=;m4@A%YwBe6RN6OS<0xi^qH+(mnLK7c+Lj9rAbdlLG0m{h;X)ZFVMF*v|kFo51i zY5?%iujLcYAaJ+evp0eLu?d5*-*Y!WqxXA2o$U?5?;XbPi7eG`2>8LK?^n2IZRe35 z`FT3@`6W7hv*URBsP|_7M$rA{&j7gJ&%FjbU!L2&HDG)O0FEYbus^1v?b~l^wZAjK zLd{T)Vg#2`!qb=OIJKjts?VXe0DvB>L+8OX$sm5qUDEl=MN~r>%tJ6MRAhR^s#GNY zNlrD&`Kq_3)cWP4H`3w2r%q|MpD0_FKIzk`*xgzE3})BG%Y~DwX7RzvDr~0B+FVmG zE&F^K%n~Z{60;j5NF>JykP_liNMPpH65iGb!j>aPeV2X54vfsJn<%lt0OCLyMCkXq zQAu9R@EU{=3Amty$eElTk3@mmYWIb4HGus}JAcr^R1Asurv)wQ&f2E^;Qo|fdr))G zOQ55%X{`W$^d+g49Wh$uV@NoSonR>@V>6hl+ty}`Vk4K) zVEFp7feE3_V1|Tv{}dkyU{92$0Qgb}N~-KAL22X?18>W2)YUg-NFudD0FM>#kziN% zGq;lJA{_Fyr3wmOo3BJPibmE_@=t3WNsNPDb{RfY;}=M_5hSVVA*u9fmLP7bZ}z<1v?7tppfazZKf`hiYC=(jAeXGU!o)}h?bZM@*UY#%GA5UJdJO%J@84EuzHY8-Kp6_r4wNLnLMNNog+W4q zqP>j^X*W406ufdkt;6_@tC%KlB6xq9=cxbnL=Vb?!mkp2ekPK$qT?BXoK7WA=7(XH zSouFb{r8jyeob{Oxk3QtLep|d+|HO=i2=jHT&Q%I0IHzo_p*u51lv<=qRA{(%R!hA zCfz%C%tL zIon!PQic>o=fi42J6UN_QBYF#KZV5 z-;4Y-1~@P&OKi7#-qW_YOf1*nf8r5z{R9+a=fZs+e!u!;G zBH#auCuchgJ@(jybk9#K={PS_+W;$J&DQ zgyOVk%l zSdLL8UEY`x8v&eZ5E1=YtIw24xSk+ZUb?}5)PT~csG61|)^1IhH04he&V)ouW5~dq zA7IYdPOI_p-X(1-T+JQ-oh8{OH_T+8B{iwoO8hP-NxU;ySgR>rO>$_sZE5_-Gx7^1 z2hrv^J_ew7G9r~-w-VvkfxKle4=ehx2y#-3+OOsMnk=bzkN6*9n0m0W72Ag=-I8QBG<9y8PiQF~Z6~q3`I@7#sN0 z9k*C*#NYKM4=KNr9=Mca7%0Bn*&h87J#hk*>n6FcZu4&HImr?%65uv~)wm39IW+ej z7Z=b^A(3Sxey__OKph{m?BJ^>la^YM#@lyoR$?92GeJ_kxB>I;elEel3rc0IuxlN3rdSgw#TfdGaQi+w`kxfC12 zB3S-s-P6MX)d6xi95&M&AglG03f!mo9-%Nh46lR+fR(2!u?49^aimriAn!P1Qu%ae z;b;JwgST92Oo$l^j20^A!#NkZK{VK=v^W<34_oc;R+y~ANxObG{6!K&V0{-MG5qGzZ zor}T?3ohlAj72gzJ;N#nc&Cy`&Ts+(vw{`v6vCWNK+9Q3D4&`6oQpo^i7Y+kZirOL z&7$wWx1MX1#2RZ8pD=o*{0Z!>vkeQ_vk4lB5j~t#Sd=eE_UYfi#*a$)rX3nH`(mF? z{rrPJiyw59i!U)hm@+2$C~45rTWeItg9IWp5YWzO&el=CJm#zdw0#KpR+%);Nz*LlJa|8`A7#339Z^QX1Jo7 zR%QAi^u)CjmE-jH2wH|ycXeYw4fh+Eo+zaZG}Q=E^vPth{q4~koe;7|9y@-ODW7nX zel@Grfp+vdQr+DwfE5{=(W*?rwx=v6M&7gy{u9joEJ9y1pXR$M8J)Uq2c@CjL9n)Q zV4(=uiftzlxZ%J>0ZAfGIu|^P zkIB@Y+aoT#4+wktL-jlD1lgYcEr*jq2-!6%a}^hPqmC$eT?va0RS;E>4b3V?3tF()wIDW2Y7!z{!19j{6+L! z2-qQ}4v}27$!$Uw(fE*4ty68YkX*}LM1w>&P8aG5ILel`0l|@+iK29r!&*wd97p4B zKA@(*BxJ3o;Ym(zcPe&DuGX1KuhCSmR;pfdq+ecGY_Bf)sZwpM_Gdo%JlUOO@ucT( z-Up4B7E~oTB~tFZrD9H=k_ah|by-fmWF`A^FeqED<-V9YybZ!r#( zvMz5d0LT~MO*`z3QfAX19*=$08^i7~Dc&QczFpFpi&#{>IZ0Jb3nSWu+?bABsn0T= zDtm}7_x$ao72#|=GzkI4V<9}3 z09^ng{CKsoIG7%0IM(~z<*}o5dCmew1|uT0FiF*haK>8mZk-Q(Px&CtZYw9v59rGWBc-{CJ)d3NwNp;i3_Sl z``2of%ZoEotg{BesTf*#l-xD8$txlcz+#W9oP&oOCAT(Xnk`eB9IRPMn)KSL{9j^0 z9Ap_Ti7g#3H}KH0zd??f^@f{K)G+mQQr7nHCr}$52XmrNY9_`__&3KcwV$7NI)DFM zqRd}RV{+ML;@RWyMb0^E_?~Zw>^=cP;CnN&2-BcOz+W_q~oy z&y;mE*c~`^dqa)P-vSd;*alfl@sj<}@kEM#5tn!s~LeFv#G4PqDux zD>bw9407qy_lr7S9Xe@HY%~Iymy*w0k+f8nigRsE=pV(OWTPGz5Vg+M0W9vog+w2A zXejFmmg&2*dT38DeM;8jkgK(r`6UNd$zMI(uVp!>DFH}ChJ?uGp*7QOlu~nkg#zcY z7P;W*f2lM_gj$#yr+-8QOCv!s8wgb7X8QSY^{s8a;f5xoxKNq+@yfPE2=@)lsI`Sa zj)M6%VVVZ+{)i3?@JJF62UtW81@Jth@H+}BCL5A*MI0ZUG#$pp@6>Amg*Yi<>kzW!5w8po;7XPT%aJD;p>XG{0ubnoc`?0JrKEivU#4o4(t| ztP`6_+hI}<)(-q<+~%djL+j)GL-C5VVc}&V7iR+t&TbW0Ht5ZO-i)j5Zn=-tLePYDut_n&{T}}XJn7elEBt9|K z^hzDkTHD#`6PzB;fYKSOwoxlhR*3P-yi4$rtFF8&^jn8q;sj&^$cHVd_v!%e;LYD; zZn>eaf@q>Cipj$6Ro&l5debj#oqPl?3apFQz`DOZ!o*s1=f}t-Y2dXeO0-y{otdcbpJPe$reC?we{XntKcVvb zgz#CoeZ?r+(Tp5>|4178L(H7v0Jj1A7y!wKKVjn53_!Fwr(+;rMVDzFLf;StGoj9* z?_Ee)!LRL9v*eB@z2+L}&D`T=Vy#1kuEmu;&2cd-1|*dzkE>7}l{Fsj&Fj&?(};<3 zi&5?MOJ~B+RONV!;bvSC6#iYy=E}+6XrbL2mOKM#S6?YqFnxW=s?3diSfC!820wkt zmIJ-UEiK|Xe}rcuttj?eQtq)T({Ea=!S05e`Qcd+43t+glG81?FU54!F_PnBm;2fZ z{~N0Wc)iH*qYzmzr;{Tf?P&o1drb?X;%FOG<#ZZaPYOiQy=<5DDW`ZBz`DmOmej4_3a?~T{PXLTU2s2h4Vu5p zn%_+>UD>Ujq~RM&S@)u+Wc1*g`LYo7q6PneQ2gbbji4=wtxS+A%q5%O-t1+Q7iVWv z0HbXP8=)dshXKqR*ehIXGI5uwt9d$CbmJNFzClrRyZ)B+keu=w@hCDRL?j2wIzI`h zfzF&VqPr%_xWos3ug(0NF}E0$Fu>=BhH)gIEmj?MdqdQ8#|J-!xfXGPQV@lTY?k_e zJ~h#g=3~R86?!)WL~?yfGkaRH-c5^3e~H@uwN;e z_RNJ$q-t(UeO~>n`bDO)(mu@wLWA2%B7*|ow}w|Ij!McSUJ!M^|ENV?j<^zTa!f6 z>eAQMQpX5G?r2v``oPme>T~;{7yO9`x)=AYlC0n!hB`grBcU~UO?W@V+ftr;rKat`LK(iDun)H&0aT%`OIj0?l2g2QxG=`z9%2=lAAY-=G+}1^#D3e(5U?wv zh-nk{m!(;f7XEfOuUJ+uTB}74(yUld-KgOhE~1W9EJ3TPMG2yYR{Rb!N=oq{jN}*o zx2^6sz&z|-AZ3-qR2PM{1ZF$cSS}8+Qv+I$b~r2i#&hyW59N3L3BXvhx033KfEn!5 z*sF7KBCeT-EeV<7m@s`Z88F`dG(p(KLZ*j&pwMtVX(Ic<5eQ`lGp_B#IL1J!-;(AM z=e~v});TZv?eo}0&wYIvM=U`-9?U{)*@g#eML?U$mE!@?AhvD3ha6&%8q^3$=Wf9v z$q5lUnIb>yYK(9K9)OC{;Fqk^@N+EKJQ=pWJnHL1$3so7_4-XFL>|!O@&YzCz98W# zTynVX?Dt~u+-AF{O#eWc>XjtJH(rc?XoOBlEKks=Mae@ql{(6ESUBAG*zC+!E zApZtu&T}S2+YYYp>j>|?-`SOex%Q2{=<_h#1oJTmg7O2+A$?M7*{FfC5Kq(NU_RXe z(p~j{QT2L)X?eodV5}e0HmXd)nRL>uJm&pokrdxNW4sjc;10LREnHX!uwtHP_cXqV z#E2m}XMwhGO$Tq_P3-;}VA%9&`K59&-*$j}7(HMJa~LfX$3ZW&-3sRU&(8nx1qM@{ zP&!u2!%v5^Q!)v2HeHzskVjJo-sb_4SCP#^fj3L^XN`!Ol;GH14E13 zO=#%5#(WD#%2lxQV}Je2yTg~Ck)5SDKHaVb1xs}Rhy>&!XS*r1+X6vFU;iZH7+hY+ zXr`-_ka?IWgs>t4CAhs{cnG`)0z=vMSx+4-N9>#`0 zj@77}DKvg$zqDddh~9irlizw_Ew7Z=p~DDS(PZ(iqfPykwq7K%vk_yoVt^iPtp$Ba zM=vj`>4!of@QN520SHtMHAJbLB_6@%kvh0j68i|q5*rexEf5F(9ir|p>GF4?x^2DsBpyYQh$E+h*S%0iqvrVM*q8_q)BAs{(Uw_{*U47 zCjBXz=KZSo1y%$?fsV0~T?-{fnlk%U;LlT&+@^TCI1?CBz#0f|a0+1rV(T)ojK5hy zUs12ztD!6R>B;3rT!Kp-sjiLGq$^9jI=juDCVHY?WVrmDOSI)dj0-%&8t3nXKzgp< z9_39Ol?OJw(VVr|&b09FqGA`YnO3aJ7lCYOR_F^o)T>#h>?txdJE;;fvR>&LC#G2DOFQ% za|Tr+?F8ur^H{xXJ#7Q(A-CBT*i8o*O=7pCJ!glzBKZ=6pt}Bm-)~@<1s%q6Rlei} zR^awDsDmMaX~6@X70Skb0eZ>v@jiO$%+dYLLWplIplecN9{RwQ7Hp6saW{&1_5jc5Ohsdo;}EaY1rasd_D^3->w+<~12Hva0cqjQbYrWGF7-Pwq<<*3 zKtbZHrSo;7SnPM@cNj|=zFipJKsZbl)PR5*=58agbsdHZk34;6#u_T=Ah zgGPmF+#A1nVTxuKr^db+ED6J1IjJd7{>E(D*3?*7TsY{@qxmX`4UMSpQNXN^qdH~svI$G*t%1k6fF_V* z$et=+R~l9jF>B}NRHk0jIBe-cc@49zwCD8O0r7bp!#kz-62hn#c5=nvR(9(vMHi%7 z2n>Rs)g~=8eIaY9HYqgq6=_lXV^?h;;fY&?IskL#p4A20R^z{G*Y(pN5X!0a&FDUW z^bcw2apa23;leKpgaX|8!X$n3bj}nXHYf6bl?MZ!ymVIUe>pSS77M+S&8IOM0w2G@ zvEYjRliGywPJ;A&?txR1dVUO4eWt;(#RIOoz@aWl%0cp(3EANt7V-D!4|$W&$h9o% zXfhb%_Ml9f^uN$_oC@uotF{_?%!`4DhgpmC7a(>#uZd~Ei5{OG;Vy?V(}97d_zmZ; z1T$5x$nnz({5iwJmCTyw1~!YpfTF1;)tY&Lq4yADL9@8Q{9e84*v>1C3MP~DAK=^0 zAD&7d$g=Yx7Pa4x2?PG4I}r&m@{>H=RIn0=Ea)qcoOpfm9B2DbSSaFIubvQyD&ozr z$hwt5m>E?n$>s%Nh|A8Y8U?|gXUzx8#}->gvd`XO?B8Lti?H^20XWrx9e=XJ-z>^c zFx?x(ODZI$dr>AWf_CNIaq6L00)cqd>y;7&#^mZJC>9%tZ+S;EDSBR$^sjEkV}*gX@6 z<8>xC+a~0X1JdBCRD&qgqt*scYiE2OL7jn>8tj1}WCEkm zHG>Q15}b>h)36;UC)X8Bv_s-H1AWedRZ+{7BS>3Ps;ZyVtw31nvn$7($R%4RwzBMm zD;%6-{tOlLvSHg75Ap)Q88C>-YuHeg{yHyK8w;A^2PZv;LI9K0RCn0)Rg*>ENC+>N zxE1Zh1s3KdQ?3bNz-+DfzpSuN#I$N)tLYH|y+=G&mo$PAWq5s4nAd?>r9uFk7eETmMBEdk_P1=<22(M- z(4ty{DUp04P5osBTB&g7Z9VTf=o#u7AR3++JMee3!ZiBFWdLbV?B)6CC(<9D+^e_$ z!I1qggyjEXPv*bSVlNLsC5SJOwhGAA8=WMz{}8_2n}>}B0{%bFJZj5q6p$DK6+0-F zn;*ITl8=oJ9O^%w_WCw#7Z|7ji7(ShoNWy5zh;6FKnMK4=KCo29TducI%vfUH1mAH zTNq$LUNQ7Y$M43apM=NwX%M_0TvV}I31oC^u3xJtsQxR$OHoFL+opK zmnEWR2q?qfSv#3ujslA?GxuprrfK0;7p769!_+8FL{TO(WuZaq2|IXKq<8XJL7 z3t?A>gKIYz!M@5h>T`Y1bzyXxB<4OQ>L=vCSheF^qszmQ$UF{fFXrt4j&6~E0Us#e zpg7QnldB}as?qdZXTn+W2GBUkBP2SA#PG6MrHAYM5%4e4e%-YwD*M(GV=;tpUwCWI+Ke`O{ID>BeIu*H& z>VBB(+$N~~fU|Mu*TDSwSv;sTx#5QO*t_Bh6uMHhAYp44vOWd#d;yjb93C7SE#!>{2 z(xr$Sjv+@Z{M(@ji>6aQiQ0o;W^c)(I(k4!Vkmu@&t2lGnUvcDB7?4p^$3Cs{BhLcC z^F$OW&xTSVkubuY7gbg%%~TyTJfM^oNH@z=vOvEDz07hQ$S+HhHcFDkddBYqZ_8m1 z!$2GQW(Tu`;WO~F2c|RoBkDU)pgS^CKU_p%4i+ly?*%Fp4s@*$XDp?R3iiNLl=H$x zSzeE%?V1>Qj@zrR-k8R3;JpOk z1w1`hup2Ij)~P7RLHHIdSr0d+Y5N4Oh-dtFNC>9Bd&7+P!C{H}{NGZsbfd7g9UU6X zKUKxCCYjiC?+)yqaC|i6ic4nGWU+eiH7iOd19ru(_q!PBnBTf~oO3OKRV|4-XjqH8 zr@}Oa52SHdO`FiK=0FJ_OI*(h1zGF|=DycqQS1eaPZrWjhrg7~Bz&0< z-|yxBD}(*N%jo~^L;qP|P*5@TYh-OqK%k?5Npe*D;pRGkS7RmzCAmhXkIQAR^hG6vBMr*(^wK7?&DmPq^!r)r z=qGoz))HS8Ot3-KTds8VG_}20q4LxHh?NnJeIN5>clrzF>c9OIa4fLx{kZ$^ulJt) zd)~7OmXVJ&GS^j%87zs~%_SWL&@ILPc1-n#9U4CrXz;7c>cOHrlMFk z67}{-0w7EzMJzm+ljwGySoQ3Kh5QWVEL^pxq2@5U)bqmWEL^hJtj4*IzZFmqQ!s|h zJAu2lqZd#W;b&qT+OSgt=qukl2lSzj{Pig%*glwZzB$_Hc~Qn{-|Yqf^EGaSu(z!% z-Ds!x+MFVYhsyIBk=F9$6i>9doq40<^0?Te;n$ch6p4=hCY7iVKnApc7@u-7P9NFs zhljG9L`KkdjUpa~$}Qv4XvUDNHA$zIWIr177VauM#tyFBSO=xpYq80OtQNQ!hfL>F zaD-rQ{hDm%k3Whm91Ng<(&u{C2DEi|8A6uPEJ$9zy4b^P@Lz_%KM7v!qONDyD|YvO zOpq=62KKCXv%1!%O)d0%;;)^s9ERxt*e^LHzUfC~6M^{oHOM+^3yQi<_#=d~rWfr~ zirB~a@Y`TB#tL!nvwG?<#g}6A!<*IbM_NW>X}^F39^Y2LTc#y|h!vW+hlCQ=cm@%b zj>Hx9w2d5%{gf1<&0RIAxQVLkzO1n9$3mqKZJxy^W`a-@DO!cTZcA7 zx%7lxDwWSt&AF%n&%$^H=g|jqi&hc$#7e&D>J3d!tJY_+Pv5}nV<@;@DOul^75gT` ze&C}w{bF=P`)6j^8)z!KojVepUPqp9-sww&&@6_4GglGm6y`N?LxI8HlkG_S16A>q zgXCH@3a{HKWr5CQ^5oFBn8rRXeH2!zWvNQ|GPBLRD}@ssNP(g;$mNL`Js`qL9IQq) zc%$yR*I}-bE#*rh?qF5<=OCaJI+4D6C$%-fwa9{h^d03Yjux^^H9t;>(3muKksp0g zGwqs`^@RM~uH$x%GPh3A`>s5)NT8MvI%txa58r(JV8U`g1z%wH6xTD}NzvQxL>l!< zXwGHLQw}B`m}@-Q4N=vD~sv}sWBF6Zdrv({)I>K%KeFLfNLPyO|b z$>EX6p};Ligi$bC&-78rdoJX=_y`NvJ3@o)GgnXvaL*Iv!vOi`5UJL?+|XBzbah9l(Q@pi; zwek(3_N=?XPK>G;>4?^US5y-BisZvPjHtWO0y3uw)sVByJonaU^F2+S1!qEpkZJt1!8gGOX7ZM?B8%#NOg4q>aE3m|0 z7PlMSm(O$j-nyYkd*stJEarjsRx^-tFRUGMI@AuuoY z=D9@Y-*5r@s6^`pM{=V&BOXQr*dCD+WmK!*&swj@X)Smh6Rk3)pdWsh$Fxgl3hA5S zxIR4(c;%;Z4YT6dj5U>Gm7h8~ct{adXxExH{mYJ2SEQF)O1*WVT4?%YKmUPyLIToV*>G0_EZ_ma zu@u0MY@3VzxpvJ>5F53nx5*Mvy9+Kz2Vpq0aasfL($IbARmRE*j;u6c!F@n3fhvCWT}#q|IHq2AgeXvu3$k-r6)mkN=AG$WckCauQro~;bwlRLBwQ0eFvYZ0_=a+Vn?UME*>+dqVPi<*E+KGBv~PwL0_QOet~FXZf<#_ zQ(g-3O^q8abDjJ)V65~WNc%HKC zIYn^Nx(2in{|!K{({VL;1vsh9?IyfFGq!06w$2rsqvSPs2>+xa1jec1jH9XIyrmoJ z{XW0rdV(*DE9R3HSuk20b_XMF`DgYsMxE)O%&)YoZ&JECt(Y5J!g@LpCWrevrrCD% zn7#OdRCXsrgSW)%3Uf@HKVuVaNz?<-x3;ft7(Z3D?*2z?_kW|G{~KBTCoal_Z*m0RIhZ<<~Y(_pe{( zZkXT5`$N#hW#Pgk6wzeGF)-Y+pjhW)hW>}9CPGMDJ<-kpyP*vy)~q$Mu1wz`8Ap6r zyN0Qyt*!O3adqp-@~PW6K@mFxy7bMWGMUU7sn={VFx~;rGuL| z3iq8%Eb(P}Q~H1rvw04tUsIk4zo836`?Wa(%jVHAL*r8LhHE78HqHJ<-h8mmS8&#I z0LU-lcg6Kvg46Oj{?%-qcOnsQ?Qs+5gew>LpJ_pU= z%F;fF;v*t$h&Pgn`Q};>2g#1YUqn=M@r)8=G84Mya;`66Ja*-MyYEM97{fFE%uL%y zxV=9}cteBl>?KZoX}#0VdTn6jL%4N0EO6b6;@6W%-djNG{rd(I=>SZ_1GnU#7(VLl zo0GuSP!bT8ZP?%UG7@hy@ZI+(B0K(3>uV^wQ1#`*TAzzaK52R%eE1?)gXTWDq!P;Ov9!-@)Bb|9 zc)162v_2oWcy}znZ{(z3=`UX}_U}2JU*h6S+V+7uxh_Im{gf5YuGyr-J!dsu@7Zy4 z{H7vzSLezd*gpqK1!?zrEZ;NYwli*gp~{5umIqhBxiA(e*n?onE4a~q>kuPI4;6`u z-f2~#Ba=IO9MdR0*zS&JP7tnC#_XIS)b##tYZR#tN0%r3C|WGUiATwJtQ0d5MFjat z2+sfweq+Wzrl&jYS^pgmL4c~@k(Bi<`BX(SwMCe!nR#j9XV|$~>i;v{Wq ziNH#bs*PUCSES2{Q3n0_&D(su2Iea zo$y1fj&HSF0nxj%8)7+?GP79=`R!%nq($2Qy_D>j+_z-4Y!qUF>cTEWFj6XMi$tr} zKYLjiT&wbu<`xJCDiy!N%VNi$Lc~?krv#O?SfvqIc~I-a(3|?ovwF}c{s2gmQ3?X( zUoEt%=AF+lmK+l^V`FKXGcGz=a89e>e?SGmb#`gPE`Rg3)uO|i*Q8mK=KjJ>W!ykx z=(pO84OTl1f|L@rp-VC_FKjKKyTzLdcfG3P^ZFti$)U$18rVJTkgoAvcOXK&{t2|K zs`g(IOX?;AepANc{WA9a{&+n)Xvze{b4-ljY6a=$r}wvxsy60WSv|r{f1P!3Zgef$ zGVKeb(Px1O4_V!DM?rzC)u;J@oZjyy$PRa~fklCWsb&dr+qAA|1s3nYaV+X>V)d<7 z{WTA4r(vn)q<02Szj$a*QbFU$cEO8wAH8z4`44J@QQFIiiksNyXB73C$kM>pYF!| zg$DY!bGeOQYr26UQkU@XGD&Zmqyo0=I7}z@r*=Fxg^eoa{5NoSb5o@FK)QVe_D|VS z5dAs%4I+Nl&EtG{)M-FDn-nl4)qhWyIcs>UG-9}k_ovlk4Y){?fU;|MBl)ZJdVb2- z(t|0+oAY*GriI1RlJ;)U<5JeAY+xe;)+IKHI5OGh(<*Ck;M?-G&`iy!ZnwPp`X(ok zyQMxezrttsXXIkpE97$RYP-JsZgSkB&iLYA`;t|e;y&FlwuXF0JcZ)P}~uIa08g0TqGH`qr|VO(p^qkv=aG z!g&@R_ply&3hrheyKw3Rcsayfty-qG-xJwICzu5b5qQP#Kg6qDD>hNJ0Z!dN+gS?Z zBh<54i`dG-68YOy{Y-!^PBytXm4Si51H2ETNC2pPwGJ()#N}4ow=|nnDNDAK6x%YY z1}>5+8gy%#!@F|HYI9}}gS!q}MrO>G&OMvvzfqb4Vz z?MWQG^OxTl!Ua(-sn^sRgX5clR$aQJ_v+zd0~ynj-)Pj8!(@OO(p3O#p-SY+lAZ~J ze)mY;fD^n6b67p8g{IgiN!h9s%#&ybCss7|Kq3Q3@-|ona*(QCAl8VayxHqU8NHS# z{D@|kw})#>(2v8DOavOC+Q0EHL;8YL=~$~3VH$#-8UC%ZVUy;}2QnBo1T=B<+gzE3HtkUMyo%%w`MN-)=(1{)X<)_K14cdcR0fj- zTmdbq;M0OQfQaB&R+d1QJ7m;Nn({j1s|-h%yw4l|iV2zVSO#B2J1(wnr)jI5sG(G% z`%#tgV#V$w;JJAxcOLX3LoJ;(8cYGjUGYy%zgUMsun4f96uSc!h2DiFKsGpW7b2#O z64p6=w~@ainNf1Tses;vbdzO&4*MPY6Qjt7{epHW8%qc49s5&ybQ^F@_ldImBkhpQ z_-uOpUq1NB&;Cc?BF9fKzqw-uGx2<#C@cx+- z0vrR9IRGnBo+l*y1|u4-D@m$P&9Z5i536kJYwc=({pu3{@cl3f%0@%>6<;7> zx`@V5CQ<)Wi>}sw@mr}!=eppT1^Qon1N}ADr_8`wRs##**+h_X{~uNWA(m~4Ca^@d zRh>9=yHxgJ5doScv?OL(l*yJuh?RK4j6iQFW&^o2Q%JyILqGd$!(Px1_eDp=Xli!S zF+H!mfgGPlZU(`M9A7i~Vc4f zYwK`cOHhbgV)41}nNam_^&_moO?MrFl3N5JO-kJ=ZHnM&9#Bl~f=Xs&s9nFT$UT(i z&7;kh&`gnyJELfUJpy6sQd&&Y;nH4HDw@NqB}qsx&B;09OXpdiM+#3I^i3I*^9nSR zq;8h}X;wf;cR-5H%=C^{6|h$Bcl6pJHZ#dnmuq&i4b?9xEqScGV`WyU3m zU5G7Mw|_!@D9HFTR19FntILMm@9~8e8_6hmD(uf@x3y)oEzhzJ>zA7y$Y8;A*lQ#Q z_XX5aT;bJ>)0?KcZ11Q;QhgJ)x`1msbhHLPj>Bkl<;cd_A;lu&|foemtWsMW)bp%MZ< zoZiz+`S-H0^5wJ3$>vwST@!f-mgKA0Y>rkQbB-Q=3_IVxPlzDqZQC~7oL0N>+vq?F z6ip=84=NO-*(jX4m8;T}fSd7|tKo|9n>)!c*zIMuiSSKX#MQL3NzF>*E`zUS6@^*+txb>&gGr@oPQ*T<=F216j?;4;o=myov6!` zc)(_H6}BT;VP>k`7OrakZQFacL(^(fgY#|L-d)+8ZV@ZqoO|B@hO}IID6|}UCbvHX zu98bKKQunTEz=74RGDmX$e0!8cDvf9+bA-Oa#B2~>EJs`_ebF!I;w_N%?1TnDnyeY z|4V9g$lUVG+ZpqgQ^64T1dH>Gt@bno$?d(_ZENr5pLe9^r$O&>(1hrC_aMQ`-;Px) znh(L0E6Tt=sw>h&+S=%Vj}#(~>017hzRr^J_*(varaFHh$4f#KHw?AI3g$T}ET^OB z-RN+utJ|VLq*O)&Egs_yWzMw?1f$cUV{4uE zxqY%f%~NTBe_U6uqY+b=grvMYJ`EUf>AC*pt03O$NgM1Aht%xT2Q@hTy`io$^kdF( zSO`_+pdKJS8X~?|33(cPBSUt3mWTPi_0&)9rw~9(*>@HC2EIE~d_{SGRZ`M-tomkt z4JIU_ER-0E&3Ny`NC44!LbWb%NbW)eX}yp_v<9yV%n!LYPtw-RO(hp*PYkx4V6`Ae zlHeY+&8!FJh2_ptOC$*(Em8Tz?da`b-sd8+3+XJ7SSCT&*j_0V!mZA4E0}U<_J?ww zEj&f444QtLkX4S{7uv9eV+6>nhFjZ4OGQLl_~=HcXpmxzB;t82V_-#Z;fHzWi+aAY zH44!HZLAb?%{+ukhhUn{Jp$W0?IU(b7PmvM@gc3Mo?$5d-F@RT3;7W}8xIjztisw) zX7V1_%**8K`oSC9{E{-P+q0^htKmUp71#JipP7YJ#RsmQ5It({l4gZYH6eX%wET3N zvcKa*;nJL~5MjzK-kHAja<9xF3|)gaVVXGvB+63@AUYxmsV!k;88d%@m=*y?%wHaZ zM5d^B90PEm@;eJ@1P?$9d+)e19;vgQq zagxP)GBM-<*OSl+atM=fIzgFcAk-Vw&o__!?B93n0Vu*M0V<)S{cQTw3zyr$aeHQ}eHt^qr}Z1WvZw zcQ(56P2qljxagAaU!aVp>yx4fFjVh^d2ruv<|2fWJ;?i-trrj_YCQAs5M7-#&k zn{IOXTr4w=gds{ps?Ow={1lJ(DGuBOw@efP;PsSFf54`8q; zQNs74qV(s9D;sf|(D;g|EGF4X18I7|wy9AtN6Ify2A_7`%ql{udU2rxVRs|1Q?3;j7V<)4wTBYIpvvX`azSULvW9lS-V#4*FvI=XMEDfzY!tqw(EMGp zpG<}>eW1aG@EwQ}cr9}gp>s|1%wT%!-GlrY z{M0XU1O5zB-Ijq%6-U?s2J6tv=2BM-AylJnT0JwoBQL`HA;0Hss=_=NGb|gT;v^1= z^h;e^4T#t-Pn;37yV>DuD>DsER$1!uwSJZvd|i+mn!wf0MOqq^TGnC5i$|=?ahV>$ zQB1i7faD*HO*5AKwQI@$lIAe_@MOt>fK2$%Ei^u*ikRX5ucggXsE%W8lr0!* zD{{2?FSs-yx8ybZ;C0C@(5JeA4F0TYBsphSC3`&#FVznviTP%4*hwl_>9LhL2GxvX z8KqfdWQ{h8!mY{yN>%GKQ4m=n<=J2-SrN6(xD6wgpt#aRlVWDEePhyX+@msqPR9oLw|0>d)I~VpCFWUrN1|)H%y$`L1E;0XePCSqxm6C%$nuXZ?OkHz#lE2 z7xk?II=c|(FBKz$sa6bg5$p7N__rA$NL)2P?~H(`{L>FZ-920U^bVlZ3{_i2s?XTj zkCFMqfu^p2i5Cxij`)*ocoCwKDZNs$t~64b5I#hI)wSQ=nfm^%dWA_QALJ(?uK!T&2lyli1n}mrXj|(?rMX_r^k`5z6Keo*GBciCK z5jshw&7_&=MRxdad(VLb@mf?*@fH-Vs==OVGo1a{9a-}>FvtevlEy%^_I{;<7v}+n z9qx-Te-{D4?*h~8{Sua;`A}`D)267-xFazBKCr{Rd^zE#GXwWW-~8Ntk`%R`OuJ0l zV{rU0ITscKGxKfq|2X$6yUj*GDl77YKM~*bWGZMk)W|jL`ZfcVKx4JQW4xcS-HJQ| zQnp0L4wzrXeH<>c+%Z+gemb;2P#JD#xet9Qr1HF!*7;+2 zcx=`wd0D2Oo?R^Rl>UwYf^&+c6`iT8zFjsgN6g|{vK9!2T8t8+Te>jME|nnpA$7?V zPY2wR@Dtd_vF8KbiRD?gLz6tng4JA9(Ye(J1@@^0jl-Z}(iD*BPKgBfggLX~2B64J zX_8_mqEAJ&6QmENiH4%iN5v%e3hW8cjL8KXxAoYe+6DEdGpTZMcf08#dUM_v;*)?XwyOx6sGdFyEqi~s2jZA0lIv`}Hh~$Mb%@Th04`lCO&$CB-j# zKeb+&`L&(0s@&Q5Py*LpLqUceWj|-yLm!bVkhNtbYCo|{f6PI}<@avcTF?7WvR-HC zk2!dRupw?(!QgG|tKL!`luWr~3Syw6`!^A3>M@vhPDCsDX2*p8!hHTWrSyN3QLtah zDBil>_hK+Guy?Tkf^>G6a9}{+;HhDO>Xzn~wqBMNU(|H&o{s-NGhKXq6T+Yb6GqreJj?mI2zLW*i?#Ply{0D#AkLm>5fb^A62-v6WayrP|orBCm5O;78x8X zm15a~R^TqBf;KWOxcaF2zy@Gxy>37K_$B8R ziIXAin=5bR6XK%m$W8A*?fh^*wMs zS22+T*nz#HJrAy8COgmwiDlUEEfzARX(igxhbECvVW63KKGC0hhwq(tE&6!(1;My* zXO4(9vriP^Gxw+A_%-w1*a%7w6w8+oHiGfxcg3C%py5QK3GWTyy$rKq)E=v66};pI z;fv;H;G8Hae(f%i5U+~R_rvS6Guq(i-m20&utRUafXId1$x9r3>#W~o-*NL5Ik8I! z|EgaAyJre~>DuYCGqPgye7Mes^y$ylFzoe!#b(3?@%{R4f22zVq<&Lq;?QdA#pBYg7Wd?ro6_IQol3vk23;z^Ry}s(16~TJ1z|6ijdH+hdk>2jhk=!MR zkT&B9t6ghAi@1y8$qDtz(r^b%^gQ$20q;bVbt6q~_SK+&Di8kS-ywP@Rrp7E`Ob}g z4-_A`vUuAJg+;vu>Ac)f?O*p|0&ZMk$w+oqQQv)HNK@2c;WNOb3L)zU7_i7O@E}Bj z$QGqSLP@yAHK=IdyCvu-Ccp`$*uim77U_3A5VY?1!9SuwX5T4b_;#ms zlN5m=I&%ARr-HU{?Gd-7NElUkNBs!N6M;i7$8b_{RMgldDUul)aX98!Y#A~Z0z2ev z36b$C7-LDd(L|qu!N&*eje?QmxX|yJ<06RIY!l$;BNmaF6y*Rjg25zc{a%I(WG0mG z>IXz@j9eI4GE0^kGJ!$Op7U{xe=Xf zB6+G~wn7r|?oq_|8s*fa2>*cmq%V@v1>N(Tl^26*A!|FhPzGi1I&W-0BbXR|C-_TM z261I1cplT;rJnGzSO2C2%%LTH2~P46T&HZ(`i3hzrFdJXqo1Wwu6lisK#x~3$Sl2;e3$|TEEKfKAp2%3y>CTn&h$@Kq~gvo$Ma{Q z&3d*FJ(4cLv?u!KD`!zXTyT$i(_4ISIc|F?x><=LFfsetEuzhWRoTL+tI)A)in`(7 z%rVemG<^c}Ejw)Ma4Jm+WF4@B+NZiFv9GV4OO|l#?vw6mMn^>UfGNeNlH^4GnmuH2 zp1;+^qF5tjIa`FpMZ|*yH^pVGu?A<1>z2<^?Wp^@q&`z{aDS=&P_d8hNEzY7e3kB$ zZ9V%4W(RdQg~92OK4@{&*`&hH161ZQZ((Ef);~gyn8QwGU}M0vA`LG+E#xAMc_;P% zd?~C$NGZ}r+I?rJ0@j$jk2ha1<0(W60EFzTaZZ=;*#%f|5lBC3 z*A3Efmc^4>f#DUi1RUB)gz<{%Lw%mYz}PumkCgCJ1|<$%vFs4!((sus4?1gsVz4T_ z!p)S$z%VtEf*e^?JLyIA=Ibc)#=|rP9V$orAcLqS@pL2+Jg3FDf@r?OLrl6^TKo*v zH7@h)t0#XhP-17zh9V!btYOU@F&&q7A}I{&?2q~$po~=p{12{~wqg#eLby;m1C&z& zP5DH$rH0YhOnpP~A*!-iE;Cg!nSC_8gp{klpo zT#QODEhr%t8N9Q*NWllk*W=&JHH^HRh`14axNV~d{(-tGxy=bs2_9>I5$+ZtDO^?N z6u$m}0(GN{Vb*-Kq7S#%$!5rx zeE!Ooi486~A>55A6-RH5Qj?akz#8QHmso1)ZD5qbPqt26_DckTMm%EM+G~!6gv^0A z+h;R!8RZTN7uwLNPWYnE2`O^CDYcDD4PtIW94L+=aP=8vvUWex_4pu`KO~#Lj#r7< zuYIY1e&Z+Z*kcFjzk>O;Xp}9K$q1IJe~^+pV(lRouKu2cBV{(D5)QRrNPtKv++04l zmQth3JS|(tF@yKeX#S)!99Mtlzqz1X-ZOIOWk2xkv1KBa5r?nbEl`}#(z7nrOf?~o z6BzAJGv-(BuHx9^!sL7^nUOUxm5X=Gelt>Bs^`?uq#{8Q{3TQlchUjur|2RF>P=?{`$n%1!$sPgBbh!Rz?boLf4 zBU3=RB~hK#nY8kn&+^kUnS{6<`GmQeK!=)QO-9WT3bpzu>S|p80Il4hip=k_jzV)Z zd=8z*R;x*oQD+3fwFlv6=&_a$c)pHgI-puA9)V?0Ax-*0ie`N9ruv%JMm|Q6R7Y&hkliQBqFSq6V)?afijA?6uSumis|P8Sb%y!FQb>ux zwRW+U0QJLK02(+_t;g!?SSzhua@p>XYPje|3VqE5npw>ey*&ph2h|b2JwvTr+F6|u z=sjyd09nwiejqtyi5_m%o|!8!8o*2)K~~&oQNJK>6#g`988>|huwo~yN7Jd^!`3CEXam9+wY1ggoiHonos z0D>r;+J`SsjK2de$=J|SZT||4-~c4+oF@ptE-}1o_DEb=2C>fF6*kY~P7Ou^)BcWW zeN^+JBQjNe_gxaleu{|-BS0vHBlHT{Nqw^OjSr%hS;i=uc z-S2EJjW%{N8d~4skxz3V?TWF`BS3Q@*tdF}nme^T`VTqkssmT$lO>$jSA^Z~1HZsq_>&7ZNmUh^& z+9ze@p8kfDKF$>55HZ98EnX8J^DRrhvy;#i%9;J9I>tvD`)#CI@zGgJ5MD)l#C^{@ z=D8Q}NtUlOg1E=8)id5i(WlEAPTn^x_G~HDQ@j^&Qw=bt%ovEgP6im`rg_v3U~1iz zKbOQHx+cfGTqpt)#sJtLvoh+LD~z)s#5wY5*2*;shZ%$>$Xm-+AMtoiC7F{ah4!RF zgmA-4={l!kiqF!R_Th~Tx3DdB1Fd(x_i)?KnwW|`+u?c%Q}ruU3;diobcfmBtjR;8 z0&#=enzdGAclACh=vOs+kT>31@1dmsu&(rO3)tu@_e_99_`;eaGJECPs)N^y8V|4|Hdm&_@kw(tI=?uL-r`Mnudc$S%#-d6hL2&o0`| z*=lR9Vu{=*=@jxhu-wHCI$BKv*J4qY7|jW?C~*rcP7RkmmLqmNR#ryj<5eoPV@=ht z3Rs)Fcg%pO#_wH=bGHz{VtiF zH+Mz9r=*GN*`(o{$IhUfHd4y{6BPcXvHQfM(RL0*;DXdfDIw@TM zJ4Gol%Rke_#+D#;5NvGL%R@x9Gqt~Pf(<^2@F!ruMB#M6ztw_ebktSd*V0o-&e9ft z@WvJRD;D}>BvQoyRnJ~Q2iMJ)?lqm*xG)3t2+?hFe-X(#6eorkDr}PXPus6_#S>o} z6@7D}K5bj;uvW6@qH8!0W+eaKo*1^G=j&6Ar`-VYQk{o@s>5+2t~Rh3?3=9g8Jgl; zC^V``-(%M9+H|b%z|(I4IR=|Q%Uj*4HTKoON%@RC{l5N1_=@ndn3q0~;*9kDbpy3n z`)z104sKft?`m_qo^fJQezTQ$ z!`Rhqp&q2l8#=-IRC?x?hnsXRIER;%WA($7`GRg>sebh)?HsqSRlLb8OO-oUWT`uF zevIUTIrzHD-vt z(7Daa=NgubZ{Om>O~v2DK%hve(@L&kfKyua?Ji~E&0JaO)V2-K%MLK5*K>;c;IJO( z^(o-W;6{&Mdj4dn&RAn)F*0_w)6-hH1=t@Kxz&3VABBFnyG=69aTPfZ$6^Bg?@Dg| zw8^@3u1QUshI6u+*$C3h4I3ur%V&Vw#_x|B^+`5F<+si*Hf zWr8mjD3chyNqr;?2+jkt2D%9na)YGoEkPEY5*TKiEI$40@OO15oqeO^c@8Fxrj{(L z+qf=?r_lAk1P^_Kky_fatJ8s+zt`y%+)hFvNBT&(<|`$Hpe7LAwIkQTV%GFt z${3ujzCbaL6WUZRU_&6XhZ*3_HtM?l{X*RKYZOgRa=+;f!(SKDU zy`pxdNPIMD(+Y3%gxn&PJM6#t|FQLs!GT0wyY|GkZQHhOCllLtdSXs&TNB$8+qRR5 zCw4Mlp65GN@2R&=RsZSg>L0sm-L-r5z4mo?Fb;X3F%;X8A=Rcqi*;OpW-`qrxy$~F zp|Ped%YDVrFf^#T-*`2Z6*~9T@dw;Hy_~wKkokrg+(YqjWvGsavoJkfsgKcw=zyQn z9^3n{&t7rOW&f@SFt#lg?#1VH<73G)rkp>mlS=Rkw~}USB1-Q(G#xu9*pQSQ$RVFz zrC3kt+f%yVAEv6d6TQw88=Zdl`B_!|l`Cz!cPUYiPFrx+98K5~m#n00F>{E0Q)CvS3kfzy@F!? z@BA8)xQpiPliP{f(6Z>!=@s~A|IFt90@o}UB)%)e?l>_uLnqyM3F>=AsH}-3zYZb8 zM?wN1k=ITS8SI{bm;Km8RDKc`T3h&y26-Zo!;D;j21&A)*#HGOF;kHfFPZo7_Y|7X zIO1hwb~6ZZbQNZxMdtofcIGBteunSFo33~O6C292555ggvrk|*%oixYX%VH{g!(&M zZp}M?&E=()xDENY?@b)FRq)4eG;xOyuumA6h6g6nEy-%6NTZx<8uqXes>{I1rB0*g zU={w01|VEtYH0-f96luO3$s3S$3y||mD0XpJV=S1`-U8I5u!2~P9LJMy9rhVbCKT7 zJ7tRE5a^cUzxK=Z&I9OcQ^aPrc zS^l%w5{S)#+7r(Z!eorRL&%QYmA)Se?DEN9j*6Gg@zPCxS>x4dmQ-0f3e)A%NT zL%AuR5iii<-VYBV(E^WGZy`;EPh})T{>LId5Du5K5e>_$O= z6iEYRlo>Pz%Zn(Avx@FUTTZ(g_xeNfQRy>vcnCb0`y135BYB@3dP4hd8F^UfC3DhX zUEQdpzFu;g2&nd?k30rwYuZU|$?LBm?RA(5^m1S;h|kK!oDZIy@+x>d$e_*-&Y`pi zG?R+Ed@A{7bylz0q+_oZ5d z{ccIq$6rA34t@ttaBKti`!tgNf*rI;GF}cFi^8Zm@&_8lIMo`ClOiO|_*gT%dx?N1M(lX$s>rx%bM@#ZYUSlm#XdUolk?LL z!~d~KEb3jpyNt@o5$~ZU+R1^ydAbvV{mgb=V$Mu=zKrRV3uUqD_jLROIJyj$0-{3@ zi1*7PMxt%;5nG8baTjl1gx+sB-A4zqQ!b!avMmrasb+^jSy(q77PeGGNIu-u30l6$ z69q98B3+~(50c7=GCOD>vWotC-)erYzIf(9I?q))Z$ylb5Uh0%d^M`Qj^NISNHOG3 zJ+k1)*fob@praFQB{#&vwKjg}f;0YfZ@U+M>VdqQF$(9ton0S~7vjVB*t=`oFYY;> zv!KI>>ZaV^GG~yUALh@PjQyTYECE4s#h2MKBsE0nyTpgj3y5Rf>D2WDG$G!&b4Fyk z(az8+iPEedrjTN@N~V37sL6*dP3m$Yav_?BL{uhE)3=Q_-E$}XtmMvl?{mX>VnDBo?8d0fJBE5hSy`!%p{mWR0)=B zC=}0yuIx`qpK$u<1~Zb#tzxvzobh+7V1KFmAA^R#sqI-3 zuEY%95309W9Wca;*WyIBd!GW4X|q4%kWaiT;0e+)67e$*_N;e>v2IKm%nBv@E5mhG zq0LW7gxV1m#cIj@Vq<)2sM~AG;XD(@6%ub=^g^0UhDkhX$XFf zf3C&fFAdW8W`0nnYt=#36tG|GZc!W`Uk3jC<(;1ESEAn<3Of6+u@{EemcW|lQF+OE z8D@AHl*PMyY>(5z{^rnDlDbGoGBv$z;|hvS!@7r`Uciqg$7+35jZTtC{%ZZ0t8Gbs zYIXOv($$igpk9Y(r$-I2>3<-TuPbiAMB*gF?X>Ww@t->K?maS&ot_q#{;-Un1Rb0r zZg2fIIm|rN_MjZe;Dp+eNWVTh;6PlFCpM7>msRf1mMdla3fhLo6q@4&!5^*F7wyH- z6}QqWb^i%k8~WFY4C3OvUWlhk#VafMLxwzW=o*r8*DcYS0)rdK_NGw%m!x`nsV36747E48U=G8DNfhlu7!^)Fup=?1E5}Geg8c-k9h@P#0Lnk@^ zgPc3~lTTXNhakm@FjTB39448|4~hSf_-4W)cYrYBpY|kkcVo<6hyOJgFH`r4-qCE*8Vu+#(Pja`r)X$Hc=L6N6ii{#9gbki#B+sUpcZ|3v zoZY1sHbaz#kr{bSI?4N0UiU+h)UOe~zhh#)lh+uo45Gz;5x}xcQWxLsmM`6WA#hTH zm)c6$QhF8;-o6*X4(7{8CJH)mDubEHQ1Q*BF~C?F9LxY%VVQYzVSboWXVMQ!K!)MP zz;gI=YiQtq{7p$^@A{FnR=ssbRm_UC1A?(bf!S?Ii?L8NQ7W}1&C6KYpgfAnMk~@X zPQ$uRT3&vbn5McIU#?uPX=boz`Y*tc-f>@jBiDEb$vQdYuJw)VS)h(#fN-k!sV`VU zHmrdt%pn}Gr`d>1CQ$DbmqPUyTNUGP;uSL{<{Lxi<2BNX86>UL_$EZj@Emgeiq3k? zwttTmA8nY3^$`X7u)>V=PtN+7OF4Kak(VGJ{EN2@lU!9kUmw@dk7XXpG!AQ+j3KBW zOBaCwOTl308mYclosg195M@lecC7q59D@rD0|gC$M6u694kMD#$CbsG!dpfl5U&NE5N7~c~Qbe8YQdb5elpJG{r3+V;596BFyZGFdFK!yt zmE}ZHxHx19lSo<0MGWcd#K4qQ3@UMG&dc#7u*M0D$ij~Bu9k##Q#M))nxvQF+_}e- zcToZyO0g!n_(S{|+-7O9<*ECfh|l1!Q^}Ry$<~7Fd>0l@QEc-1Dn)fm)OH$CVx!A8 zs1$p>0HsNls&zh-mUq>YhG>)Qs(7EV)}D~p=}!({Uyx1NDr-Jc+xcsBm}m8IL>~}8 zN2VV$l)ZSd&|ZQixF$xZH4?2LULBL65x4-(>+>esS!&xs+ zK6C43yEnRIXWGk3rm8&OGEVGT?88Mmj3QVGA8)p(m?0X^sOL^FD_H`zM1OGElX%eC zD;cqIU08hlcP^L}mE4kn45|?@v*L^MA0uSP4PcVR7aIsl7eDN`xRF-k#|*LRF;W0K z2_rNl5h8wsY=-+_c^6Nmgx@&we=SFOs1|&My5HvBbbNLoUw_*jeE5!Iei|v2PMW}_ zeBfw*8G)?Qm~c+P)QwaJ-uK__lrU2cNuQrM>%TWmPdxvZxAQ-)@Bj9LSLcTWxxQ#O zD6OmBNX4Kq|M7@loQf?u(vU;Rg8;|a*dnCY!&+zltdN#q?)kHcRIDJiv{9J;s&GeVq?xrgBSIe+c|A<42tir8ZjBwbdx6lerU?AaJ4vCMOAa+M(<#^S++?+B^m!zsfINr5Id^<)6 z4`%erQgpVvSKhhFG{G&$d8t9+@|eW75%>~TNewjZPndHI0zk_tOae~RESmvUTzi4Sqh?E#tn(=w^O!HbZsfbeyR=KkDt zuDp$zd@O$O17yH@{pYM=p(A&K9~e!2T&Y%5(yWXv+}byyS&3i!W5(Hi-o%4WAXsB( zsHdWC^FJqYS1=N5_i<%FeHRx{o>WHT2oHJw&4%9>6Y7^W!vf<@2Y-$76ycZkK-hw- z)FM~nw@KZ8SU?&&@ci(j&XXm~Pucrs3^VhqD<=fnuOQQ}gx60Db;s63TEj$|08Na@ z0Ww8tPHT`U^gH3$iIl<+1rY=6*;;k4Yig51KX1b3ziz^S-#&iPvh)lQDo+_yLkZWn zNi6F$S zb<_mU|33&u-8c>B8SdM+H|qc74ilsyr<3EPS#onB0883%Ug%30pN1W8suF=C1Rlg*v-$SfDQ-^%=-z{!olhV?=#t%6tcg$}EC~MhX1e3VZ-+zoD zWxPj>#bvyQj45Tj%lFUd;DapfV)2ve{2r_&e+P8?9XHDd9Jihc0%nSY+E|c)LylUx z=OjG18%r3#xrh)*x#r@+Jox*5ivaV_$rR?3X3y~bZAbp|KoV^Ga1PM9&_j)#M*`u> z4Bb6V^Hv{p9g@eF&z6@>mzI~!XiIO)h%wnWX4OSC1($S}IubNDP+v+Z7tUAs3C=FD z0<=Q)S1Xi*O$0M^-;9JQHPEic9 z3U2S9rHILP5^5awpxcg_XsA87LQG#*&qT9!iuKM(SHdJWS7bKOZm*PxuZ$ZQXI5B6 zgC9AC<7>RM@|=FbTGM;iIO+9qkj`xYKzpqpG)V5iEWf}@Mn2NPL}SP&aIWt=MO%Hh zRxIUBmfuTC%N~A(=O|IphC5H2bIGP7Ze5?&?(rmHU`5T9bQ4PzZ6>3WibyJgoO&EHTA)X@{tptj7l@%`Z$jupGo*~QZXSPp?oz_q(Vu3aw zSnv1<8#vU1L8hXFEM9?&cgt(6Dy(%srngKj84;e? zqZDOp?p#fVDuryiq#cg0Db`TaMBUV>0?M?JEYP^x8bBVDI9A0lzc64i)rj#$=b?{= z!?=XVD=W@De^Hk2eP9oF$w7mXInT3 z+s0u=HvCJ{x3`gZ)M`~p%`HFnVH=V+Vz{}w64bF(#V%L5N(&|rsw~6@R>%<4RVMX~ z?D%sMs;mQHiap5{B{{>$SV~0TIHYhnCqi~FOnf1ZH`Y5%4J1n2gHd1u^J^s810kp+ ztyDYA_ql%Hs@Z|Fw*2)PnAqSP%{mm;UiJaCN2#WAS6;z}gX4yhfvib9<~-D3mlJb! z#UUw%ha+jXpOaX0R|S&={PW3&RcELoVP#^_yoMF)h~hZu$+p)_Bn4e?<-pIv2HxNQ z)ldK3$D;9J^;PH;HMmswHxnQ%sq#G`7iMI9-$m*Kr1K^$s{~NXS}fYf zpK`3*v1ayyL?^P6Kg&^zArmXVA7!?cs?}~546AoAaaie!r~N#TrC=)pE%C&H(_5UPMRs33E}gy<`uWyr zO4^WKE<7wT{S2iHh_qgWES<7O2u~s8*+jmKd{RM6msMSKth&_bI;Qk`+h^I@DDeuf zTXm}ywin@&MPd}EGVd{VU>O^vAhZK{pp{GX{7TNjxeFs*r{BVhqxr6LN2^E8s+Or% zbiOo(dRG+4Wzp15s6WVuEFzTT>&3J$kLX>8@Ym8M4JyqP_ywi4`vZ#4nIb8e@M6E+ zfWH-w$lRqBYUOrDV6r|<>di@bchdM?=wgvpM*eBn$C>xRxP>Iu(nE~O!;vy8BW`Mb zY~E6tYahk3j)@VIEn~6}lImYT4%SL1sWU@zf@;QE+6;7XyGG-F4ui4~JW=LQD9Zbb z2Z{HAj_gMhfLg+y*{3VO{}pwO&QZ8bcI;HHPY+7y2Wn{HQr5dEYMM7N_9H`XkLH+V zlFHiw3TgupvGWrD2Ol?j?eXYlU*M}q>;?h;@CHv06Ph;ftsP?|Z(wPX$}z++{>J>q z47r#}z5zq4hY1mYSS3p`?9PF&%rUMFWa1-cuQUfLGv@Og*p{Gfw$T>W zMungL5jB)FmafrDW)ZuQFqRm4X_HX*UDW24NV8<&ZdE z)b6mUj-_QeIN$Kvah`l{K1RIEa93?JJwk__-A1F~ePg!X+ASQ-+fq%=ce$wn)JHw_P%ex^i=oSmdk{{k*SW zp=gq9r>_~jM;wpcq9w|6wcFCs)N53emd}T7ewV$OAM+fobMS_^vSpO}*$eo4@m(u! z`KWn@b`c%iF)~lOKF1W<73CA?At{I6XhY=#*bs7CsAd$M$*?$s&r~c0ww5T*ZSp)6Bs*q2|CqS@bD5mGvVr zRgx@JalwQ!f`gc%2Zy3ssVhQ=#hw>-jsq?nx8p57eszYU$tafxi<-%+^R)`V>w#qz6}{qzG5oO z^#?29KBPVH#XP~bG@?Rq!>gU?x1wifrXo)(i=k!Xmm9;vL*W$plO#yLCJ=FxGjuqq zHt$6-oK=GJ+CC-VTrh*%XB;|f5lEa5lpvce)X5xn!@MbsD6r#e`KMbncDLf9PLF6e zC576u(dQ&LxFw>nnT*DnT1zW*(=ZBudEGJ{nil8l83n%<22GDAL`HaUksxe@1jdDNoc9ATTb)e5&d!dr=8 zXQrhxzEwpYEf9;BJo%65DxE93biW}Mw|Mde9c-Fsa4XkA&$jWE-{h$~L@5gf?UM~D zd^~3Xrr~A{p};ICm4&gfj89v%^v1;8K#)lTjtK9ZW~DEx;TbOh%-1g$y73s}%Wub2 zb;N<5OMF(BI!Z?C#F!qMcie%E8;~kHI;YZ~y6S(i2luCO*Y{n|a@aku0wDG0bx!4X zw;rZN*xi=Zd4C?so&IJCEPX`<%w_O=eD{SaD=awd>11cZFn; z`-${VQPWCbd=LzkS78l}VWIXsQ(DM8&u68R?xhx`#}*#)iUMxeQs8XgYQ!t%b|Ac!mzU8vmc%05fLj$#m&SZ=@H8w<`g65CM0A(QGINtNb zs56h*H~X*t6mW%R`8xoTL5%!M2RCBT45L;SYX&Rj5Q?2${sr1JoufI;q3ZizP80Dn41!cL0+h7tE-t9FTL#RL|DuybB>5(WzM6RksIHy9+MREQ zEJRYEIy7M=vuIXQjDhHq`{WK(%&=lZ+Ce>ADhXOFH6}p?{u6Gj?!eBkH4sZ|>W$nz z7m^l)Up5_#o-zA90Udu;jUpx@##?ql!01>8#g`CSL%h>Sn#silcHq6QAdh(iZQ`u{ zU=grEo5{a0L%meVE5G}^ob8TE6oOp*(bTesevHb%1X{~93 z=B-D}U(<>;NyA|Cjsq?xPsS{*o8}XY7RWL)5Va*M}~4? z*2=L4M#-$#dLiuWN4H!E@7xUibjZ}BYy7_)ovEG$E6e=m^L9QRmg@DhIV^RbKByZqN~J5~Qm zmH9SvzjW0>ptc`qT*3RfC;L0X7(Av*V)_%(8xtWB0xa$9+OPZt|6;qycU7n&hb zPN}1dTwxI_BJ8`hKNP(bX+t=t^}30=*<_>>6+yzWM8VNSNg|Dr z#JgnkA3bCA2gM~?`ZN9`?K2Voa_T7U2eXv9tRzm&;u9@I|7Bsr&ZxFY4E=BmS*Grg zrtCj_hWL+A{-@agZ=w9>PzT71d;iB52pNPmm6-rCO-FzWqBWGw6A^K+?{iM^PZ&2C#4&TK<|k1I8-lY&Lno08^Vs^9@cpE z;EVOcM|0jZw%TFMk>3KIY&?2#ASYTZ+w>#s2P!jbCJ2h!YV)o-(Xmlk3mX``dV6Kz zSyY;X;f=S=r5RvUX1K(1I|JJl^ViANUnEB0L2=XsCq)8m5uXqy^bWpwF7?BH6v#c+ zk;v8(&zn^oH>^MqTFz?B<+@4go<>VbZy9GHd&4*~gDPm)bOUgCdn8%i|6yq{RFI^; z)+leebZ_78NGNGuhMlT6%El(baB_RL$rXCpE>WxL*_N*_wq{DFZ5I(X1D6Qc2{hqA zD&AR);)~cpZ;;H_EM5Cw!5%;tcU%weyUS2Um?l2beIICjE@a@F1^k>%vyM|`s`Z^1 zM&Rr`4_L4?907#^^egL=ehGv>a|H!xG6(|c%T7cP2y9-+BUNEM{AMS7?a&A-vr=HI z*K6lUrX{RNdadd{jN*W^g%FEVo(bXwFIaT46Ae9sgS zPpryM5GLaB*QJ}ES7m=WvEHH>P z$45?eWJSP!B;b>RB-K!i)KEO9v{Zk{Z;n&!$T|UTeVQ`sy5oGsh6+$s$>2p&F1qWg z@ZuB}XUzVC3>#=fcTi;^CMe(tb4avF$N&B9KN|U;I{)7qY1PYxeuMt67=DEYCtH2( z8dnhid)ENWL7;$Yb%U>616`monbBU$I(mnwX*9}SWI1R&csPU|nH@u+@|oWNvcs%% z<6;ukpM8$V>&#m`6ukA) zO|b!2{iyP$a%C&~k1lxf9TL=YmZ`$9nG*EG0-i(i%@!uo5ro#zbGQ4}X&b%Dh)!B# zn+oacs~<0ox_9mKNQW#=T!3h) z++3s}8UuhjYzk$(>0CWfbQKLn>pM1kzrzex)MI8fepatf)qZ)cPLcz!ln@>=ss&51S3e2CbcCbm?K_LCcg>AzUK=A7@D z;uxTnTj>)b3MKhMg7}7=q9~mAC_bK|T>p|RhDn;G3pdgHA@K%L=imdSf0M)*QOgN~ z{OD!?E(RTTnnODmBrql5 z?T5bfvGX@;D`)BoSyWmy)in^B1ZoeJhaZ|GP?&oX1QLDIj_sc%M^tlx+544j}H_a_v4Hi5}BNAtNO%pSiirqdDl8 zllEK~_1i@R<1@431;H?2B4%P@;{BOb>^%q1uulu;*p~mpJ>bnZ;0^M(e-ab0JE7cP9Oye`Q19m+=z9sv?`ELk7_#grxJ04O=TaYqv-Tw4b&d^I zmlI<|()eYJGnIA>v9lOpu<6&K0aHb>RP9k2I)gJX@TQngQ^ynd8k6pQm;2(Sd*o@{ zI@$eVe{M<=iTkKx7q8jw764h$-9(^AsFn2EaYnkPyqjEa26X_90 z^u-u{JSlHy8PkxhR_F?f_45>(kqi2TI#tuCwu7fJC-4ESsqXdDG}G##GZeZZLUK(} zKcX4!E4BZk;)&0O{*7vUZb+~URj-%9FxC)PpNq|33L6Nt0yIXkq9IJ6$$p6@Xft>9 z*aXMDig84Gk>Zxa4ex{CmP9pLiz+O(($o>}#x+3}La)&k26E;JtrftYxz0QdsgmIr zjJ`hjsi|PasE2^vJhw!N^J7;+haCi3tzuQ~f`l{{E}xVqp&gTYVQOeGme)9HueABS zbaafrhAy4Q0E8j)=upnguMNw>gc&Gyr z!}7q=L^Ej?a~xsWSjuk@lj4N}=tlmKv;);}WGQ-nzy`7tM?bNUCuMV#a(2+=%ER{r z&41u4TcR$k8hUF?kC>-oQS{3q$WqmDi7!nQq6m1XH_M}`PNq(3#EB712w|ckCiqxM zFgc^Q+&IRTH$T)_i5c#tc9S$oS1V6ADry+95$WaZY+%Ik{255j#;T*A{R>ihtm5;2 zD(d-x0j?`TU@YX)s+}+YDu^{3$ssug{Xc)o`Cle#xyYlB)5NN2ODRWDC5YM&MSJH@ zIFgE`tMXSE9wwkGp8QEpUV8Afu(9sga1UQWh905dYswExw6XHjpq9>2xHOxygSn6O zuG&6BYGR8?=cyw>-olT_1fyeBvB1)E9hM(40vzEeOz=jD9($q?8c7`X1FM~3zQNdg zLuxDRN4j?Iq(#hiy>~SFW%*-}5Vcx`k8*HcWi@?Rn=x|9te+F{Xu;E4w9Y68{66_5$*l^Nz zx)Gsf@*>n12-p(K9{#xfs{^8?b78(B0Qb`<4i73Ml*Cy20Z5XVmMyczye8MT=EuMZ z#$dkcYfkXL#-MNd8ps^&)d`_nCd{HIqZO=9X5#LOIn6WpmV)G0aD!dD&yEPVS1wp@$f5;3wBj-G-NF)3iS`!#P{h61^wxofD*cf z1g8q^2>MaUDFqe}cl`Z=M$h0Yf|^pWVhIjhMOlIGjLZj2sl-;P zT&{GMS(7u|WIt^Y(=3>Q!tdD!05bfVm&T=89QEQd2Siqv2U<<02DYJKuTR@^B)(yg z#LTwhUK-2e89H>UHY9|j$YJ(PJ%N}B4+ii3bTRv%P;qIvWQqGkh&67R?X5in434cv zy$(iu<*KNNJXBR#uupwuSvf+VD<~wE0+`caxSptdcIVG?uj%yF8-*!qfT%`Bfn;H< z#0WEjE0im2j5(3cQEm8enNhGSnm-$<0Gy`RWL}P`m7H2;3}HmrNh;KA#HR@Jx0!Im zS?)XmL{w8elf~#yd-h)(vx(Zjd$FEK;q`RPi z))2jt&tD2Dh9gsUv5y8^&K9Uh!SNAfo3~a+;}l=6POTvxbP@!Z^G-d3!3__2th7*- zw3!)lbcGz&65I7`?;c*y{45lc^JH|8g;H(8Ql)~n1s-ObSTC(w0Jta)?yXX2b`h(x z|1cco6kw6zjkV;JM1OKfsVHFcF8{ZhV7GRmP{_XrSOf}CuthuPc@#f-JZo@f-RL_d z-@x~FM^-oWY*v&gsc+r=E;ByRHC4THD*Dp1S`+CyR*XikDQkNQ~^}gs02ZCTY6ZOu*pv30)sz|xP5@?K9MlWyrRov@aEiOmVoj;F~ zV3t&%ET9ZP-)k3(&Li=m+s?=Ap!vRwf_~pFr6))|4N9~xSpX1p+^T}H&%t@nt>?!Q zgjHH$7|w)j@x=LyLb%PngSIrIF_?vPnh4qfvB_p1Z(mkJza-Kz>Q^LQH|N9;Jb|sb zL5^~(eO>1g${MtlU;B^>o1-3`I%1t|G({;Uy5V7fM%xe83icry(U!^eE|;gg#VOW} zipZ>?R(5ANUhKD44QIOk_dxOAHAL=f(>+Cz8_@KKdvc6qZ3cc?7+*^yxQ4hH-V&=w z%!k`LGVTd08qC3s#qzY~7P*+O)4nKVH9*_J^0eJ2%p|`u?`&5>ds>JZIT!-Vdg%+XA(#IXtOVJBX;bZ2ML4-k>Cp<$Ee|mhKDU= ztl;cZ9$WnmbJbRdl4dScdzZ$5;{A5Nzj*F1u$d0D6*=MfwtazmM^6NePvVJ`*#>L~D2 zouyQ2&@SR_U(+CE%mBi1RC^p9zX%9qKCS) zUO?~Dom53vs7Eoq{=$w`u5t|ZP5iQd^Zx$XacYL49bbT2F&r0jJnZ%?Z$ew3Urz z4O#9=jGnDoSQx){OYE{2lpWWN!!ZAl_lGC=HLk2Mrgp1QBCHlh6PmfN5=Q3LLHJq` z^Aao~Af}RnlF6!yQ}R{LESdWn!#3mGU2RX;V-lLCW;EtiJr5OP)C-78wgzIg5H?cCXO_>E~b2&Mu5$aV?i|IZQ)~g0X_F|6wDgvqPH4wWEVA}5L=qO+aYf< zt4U*z0cxemDhmKH{t@@I;y8_x$0QvmS5JHf9CG#lWj4gHe8rob;x_@ zQ3^+NhOB$;-Icm4(iQJSn0|9l34eY?%{wApKc(Hef%&-Mcng5w>i}JCwJXzJ-5kub^mCPP0Hz$R8e+B2Zw%)|GZhyA#kvd1Akeaxd)lreHPoLs5&nZfH_&vuAl({jjzbT2#_2 zO7K&hT{c969ylZs0a}YhaCY%h#?$F{y_BkH@Xffon_6mWS_(RBUK$_}lI4~jcml3e zc&t;@2lfYu4>+2i(l6$+Q2Ad+n$u58-K1DNo~QA}{(FF1|=rMTtIjsnF`rX=@w^LOS9? z6PJG$*UmN6N8s7PG($-dQFb3t@c1dXL4LPIRYYc^?8$%5bs`XZQDPV(Lcm!2h-Uw= z2_DIu0!UIktfHVyLw) zg&zB8zbr6~22m@8RA^38XO2AOKSp#(wx*gZMDzH^ZsL%K6o92h=c^U>(d__PRei3( zL~2z=xE(bd^zhk{3^NTr)(*vCxFV{|Wl)EWZsv!lZsqrYQ+J10=5sht zH%gS7sR;FJw`%?#2x5Ds zIKj=!YXH-gc7VNUHXfhv4`AJY(JVZS^Q%rx*sFDVlU+INC%4=ngt*> zqJ85qBWxwh1j_|>p4s;;s8|kvpo>plTq!))im-`doUgC??#RcSaAoqZEzTdx{tmk1 z(3EWUqBc*dl|Kc|+KIH-aN)Z+$k+7zP>k5qJHy|~KNz~`H+K5y7LRWu3D$ZLf2Uo3 zeL33w`{t-Z_pH`ow-e@R_Gamow%%?O{Iu&Y{QM&~vsZEEn+`>Z`O$d5Ebus5Ks3K3 z!O$z~)KB{ix0P}NwpV-o<$R2Q=XaT+hIE;J&5iH>g^V(sYs^`vpm;M@mhpCY1i^mC zKD+>HE@t_6qbJlc1;P~eWA-1~FHs6Ob}HLv?HoOwS$EeLVVJC$5`0SX)Z`tToa zQmZl(Sq?uXlH_+!xEM=8r2doR4`#}h4G_~2x8AnR8@B}ziAn!4Al;vtlp;vKB8?k{ z6lAClW864vP@6a-)2@ME2ltz7DSEF-#|O3`OGLT|1KJ5G;->0FXJ@^0O(;Uz*zm9G z3K}qW^W77{m!KwAF9Jb1wzVz)TM27QIKekF`5HD#q@pn$r9NZ_-rcsG{B@5&h4z`` zWW+00xHemsPM5V5ixxh293r(JiA*tn@`3YTNY9Hw8(71 zoXgeF`^u~_4ty2{<9L_m7!JmqmZGmkF}<|6^2*~~f%M$wy0>{Z6i+&VtIrNp6>O)J z!s}P)#X?7XwO1G>sprPYk)1GlIW{~$VpLNQW>R35duAyT5Cbxg$S{Hf>gF|4W@--V z4U+#**=H~~{uG(w8=3MJ*#H%h(p?Ao7>LA6&$({0ZUwh)1!LkEvo>dZ^6spBRG3k+ zaV{UC$Sv{;JIAmhcM;)Po+nktL8DgB_5Pdfp@&~QWTBE^o}W2%QEbaZmKr8j7X{X3yh$WW_2>@S{FW?m zxBDFsfNvWTcUzI%4~yR`9OApmcyR<}j6p(Oa~os=P`gd|1;#f@|D=})+Zs7)PrTm=I_?_M zg%=i@pFeUBY_eth$b&k8;|a!+_x6f`RsFrVLJ zJ$2f{zL(|I=f)h-sYh+O(Ka)(EfAb$VedaCDf6nVrs^4qycJ9J6KA`zrW8boW4pEn zP#eE!@{QGS$1vP;y!Q3< zJ7fGFqF|nUd2l4b+Z!c0KD5Vh_Fn%~;c$NIg899gg0(O{hrbEexGT>Mv@$dp$gWhl}m! zF>2hF=>}5ueS;qL5SI+NV?wJ}R`%yx9kkF7wHqCu_{R=_Io-X1#rftOC(Yc;3|=@H}d0qPCb+U>Vqm04ED%7z|Sk=D{Af ziW!Mzkj!*QO`sk7mbw&VJPJmL zN99l|tb$u;sUYl6zjLy&4s4UZg>gyOW@6AQ$lzftzD(i~3m(PycNb$%vQ{H9eBWRk zL!5;PzL=3x_UNw>Z5yTD2=2{a^Yzd+L?-X7cnccMIz0g9^QvX?aoL!;L=lf$F zChBd@%a;slZqa#A{eho9+}} zr>Ba}G`(9z>}|=3*Hh(91pRC3qzGlV0|6({GP1Xs!<^JPon!j6%D3CagNRfz{fF(8 z?D>jL8s>w?Rh>uU$lI`|J=(7+WDx`lyMShBnjLnHc)gEgI?N8`|0C-ggDY*;Xk**f zj&0kvIk7b{_rx}KY}-yIwr$&*NixaI+bRlU2O>fZJKd24m|`}A7t2~(qx zuBJe-qyTR<ujVA5Ptzo#Ll0FlC(1iF%+#1% ziwT|~2v@`!toE$0R9_YG>)0(jCQ}dSCBpr6SS)$7u5)+Vb1xS$Z0%4c9>`P?gd@IU z6Vj67iRbW`rI@+_NiiteI=wP76|6S(s@{GocW8ofyz&~hVKQ52bk{&K@4*`|62cG) z#g`vIxtU05&%Al|I^-R#LjyMcJl-I^Mx(s?s=MszwvHWnBTQ~6do}zVZNBH4mB&M{9JUIO#P+fcdMg$`SVLC&q_z)KPNySd#f zvBuC{20>s1kh3R|vu2R613?9Vd^Vu4SbI*82X+Y>PMys5S&wCRsw=ZXp0`vb$3EM5 z10X!f{U|@e4_~O~Xd{qL(yc$3KGP5Fb;&i$@TM`W|2a)<9FlbC5d$K$WM`eU_>xMg zh}P8FQ8y<1kJT-q@AXg=&(ie}{2$9$@B=Y|F^X)U6q*wwG2f~Lw@V77`x;H}PqNjw zKr-vZF?*MC#!BZ^P=Sooj$Xja`*FL-w}V3oS`1*=(a8512aX+P*uBR66B4+#{_bDQ zagEzwk>+k$lbs*%u@=;)$X{-G@H=+C6r|30TYQa5a%_$OU31D`?aVmDTE4h?L|X|( zydrS|ANWqtOup%eA@-J1|4)PIC5UarOTfJ6xNmBN`{R*xfKQM7)tQYp+nfgTEj>1V z+YQn7)205o_x?t}w5XH2@=4n`%X}*KhmkDL-a+=jsGuiGn0idtLaC3X1Ar)dc!v?DwK2l5?k#6PjptL2PBsOYJHBy8^ zT2g}s9XqijrZgD5s+v&o6HdJ1njlws4tN_b_%kG6^Ed&j)7S27n^RSJ z$2T`@4qelx`sRW)>@c!E%Khl)?Na*i4RaY`aGpl=e$S9NfhZ5-SdOWR#e!jP9w{#Q z|Q(kuro*RS^u>bZbS4NDyj=_!#u9$DeNSBSIHE_YIc674}ht45Kf+J-g?p;xs*;L$n?*c2hIA6ognk*$jBUlB7 z3{+hGG6n1o^81kOcV+g31)o`cvhuV}xFlupOj&sp7os(&ChS5poLnO_>5LrQ8Br}W zL{M6;(`Js)J|RR?^X&ILq3;OU>`tpTZT~9dPRk=o)0&|1Zhj&?m42L}Zclc%e)9PZ zYk>0@ZxpYz+o*h_4jD4=Kvy&S3k!!ACn?1Bgcs;4U`rOF;`daW|MOn3jVQ~%&z7ii z0=C~HW5NNA^Nqnfh|Q<&;UOmZWaXWV;kSDLi^PiSRCm34TNKo}_=i3LjxYF9ZdlB{ znfasaTe$X?vCE@j%WP>9Xus5Uz9O@*6NyEIqjT(R?K$PMDi~qP8J-C*Q4aj(OG$1m z;v~=%VKT_X@=VdCFGv81cV$#KoPG(_z*n>@|D3{gNTtG?>Kz`!2Rm>HtaA`fQCYY^ z6kpjlqwC5oi#3;8k!E1oC?B)lYE5RoUakwCH>99J#|spD@9RJ!U z!b3if8$npliSyZU|I?+#3`MPx<5d8elRjx(CnWq}DuQsZslE3{DBj7J?d2%_OQ?e$ ze3uflR#et%=FLpyjg1?#xH7ZRqgGtyMU5Mo9IcIfV@VvH4`3h1U>_}zfj`N3Uovn? zf$cSA_uYw{y^-RV;`gFVnuBJ#qBr7pAa#a&-%~W_Z}}R2FT@s3$bV6jldHRQS}FHat`k!BAqe zh>*XF#$C4$H8#%5?RFkdt4{H-oTW0FAu>WU zFS+WYR8#H3cmkpvDShDB)XUfN*K3zwncvSlKQACGYb0s*LVOhM?7jmOG4I%t&Z|eB z316U5QV1p``M%IR{8OiGSua}H0<*NL1o&bg%`o=++m5DAVbprtb9t2&E8p0n$4 zqob{NnR$Lw<`~u1L2;R*?-|hZ1`c)NxlZK; z#)-mXs-S(i3p5*i9!H-t!ye`x#9tmUeZ!-~679?vbrVmUV`O8>WWsiQZpYDQ{OMGP z<_Z%_O(na;z{Y2Fxy9!X%wfy9XlmZ;l~lwMf&?PR2K-)C)*HGY+MCqol+G*+k1_(p zpggT1saw#r!7za=rEN-B$rXQHzsq4^vbmPwb$HH8_^9)Fv+wBQP33f$?ONth#xFTI zyxktrA>Z+~LJ+>Yzf>9o;nzH;?VNdQJ0HZE&N3Nupz{!SQ>?-g(_}DXj_y5gK9d!l zy$ZjR+L`2qyVFQX|7=7R5tE-K=#z-hWsA-?SfRB}Pc*2RdW$u)i?QAzxB=mG#AXzL znFEpAV?Y4=SP-v#M_W<2~d-_jlfm^kF+} z!A=OvMx|yN^qsWkHxc7iCG1rnR#>GM(+7l-6&=lvUOe`XQP>7l_Mad0>!eTdDRa%G zK-wQaT$_*OvTM&sShapC#*{b~Uw1A3WG~Mg;jBvgB4<&N1n4-P0smR{y3E0Lc`4Q@M5@x0=u7fqqq2zq4x_}nF80FJq%c5J*Kd^C z2)&U<5|StUNteA)KAc3BXK!PLC|iVkPuE4AD5w$3x33Y)f36WLaI6t4c&*{C^#!OP zxex}SBrLUtvdHKH25Z0jyhw{@uz$us$O=^B_ovkNXbqR#*CVQu>$1ppbua%b$@qg| zn3%IPq!*hLR%{7QO${2f@$jvld-O{hGgO*W3!Tupy5}3Ur}oNn?#-wh?{)S~+4~GS zyYTT9n&a~Wt#QT1--xFEy!43-X%%2Yw5bF%AvB~-XUM8K5~bDvHH5lRd}(HHFn4~0 zrTBB`h1$i&^jAg=F~MNKK>QbIgNl&wPsNZaK*3YH!6Q>=#r<{!g-VW2@u+GsAvigx z5@}Ea(7fj{EzvO#B`C&sLZ}E}H z=oCsn1oKu=q$NCEwy+4&$a?kmy5(Ai@k1(K2JemhIDpa-_o4HcR0*wlamK)f3kPvV z3@yya_kXC>|GToQDnR~Umg+z9gZB6$N+fWof0qCDyj;{#OyqyUzDgpMj!*Rt9teD# zk`lzKz=sQ(Y2!u&No0TKPXGCbe4t1-_@_?*5)p{J6YW3jR(w#nG(Q%wSJ(E7p%(U^ zTSlME3Hz+@vQk?_$W-o$RAr1T^%QQ$ENXXFPV$ttbcma)X$AJ1VLR8-%4Q>3T+ zL|D=R8N>%RXwnxm(z>$~LWBo+fy)7qE5jVI25ZrNN5AcXt|m`1aoKKd*bmjkSOOu4n9ZVxZ-q1| z5<$I{w@y1q4g~U-+3GvU5K_OnnHu3E6`&E4dl!bA?1H)&4eG5V&4zK~Wu9FMcrvf+ zksS`4-L_euUDyeLJZ-v^PZS{4$51bwxyYE8 zHg-%J(&fbxj6|1JxZgyr7y>QIdR77Y0v*~}hOaKo1qG6fnz5O+@I-~$;NxyyZ_lz^ z5nYE;fxjqE@s32T8b^`?SF5m423(S%r2KWBQ=Vx#aLJe)qeh_Y!Cy_BhoVgxB5WbF zlnNd*x_r7}m*50WPtXUW)~pmtJCFab{!%xMW{pg7H1cb7uz==vR5zBt*2tPmdvyH~ zA-!>sFShHo`ocF4guco{lI`k!1Ny!=Zfw7T%5~ID&m)ILV}xWOxvvrQw?kf<{#qFJ^+Ra{vu0!Op`<@r(BDByGX|-ju?$IY51i?M<513|qO_W! z?W3A?IC{KezUjofQXi00+8;tJsQ_Vd?e;IK^~k*^8ig* z3fQ4}fCF8nDp}ZH1`1??bI3p*r)El$q9x#&jH;aeAqKl%={2FsFbF^cR(kzFYN<>r zKYeTdQmP?|81X~WOizhRi?;Don7{|9!vO>o&dHi($5EnKpF&kJShe{`XdmgX^73-T zxoHViJ{#oiVUK}8Tyr3elT=&)d7khJFslb{5oE7l_*88VJ!(AKF*T8mKiY2r*BPB? z4Pz3on`4L)Ry4zTgk`Vfv&IBc)_<5-w1|!@3{cFi$_Qn)hi4ij@?;W|LQW$b(HoCx zrCcGK6|N)Q!5$!E!DUNv+B_LQq-)!d9wET)b=R!hpvQdeY)X~N4Ofn8y1%iA>8d=*>Iwt^7$+Ggt5S2!sh>+q-4EL4Z_8BO62~W* zUhpJ|+hi9@D5Xr<^XhP_<@PJ(7-$$Uo;tj5Z0K*#9q#O$ot(FJ7B@C_=@L~MA%bj!>3J&`1fza01PZK!G!+Q} zyA?GBwV?TJlCO111u}73UQ2|j9TGli9W#6{FvR)fJ4E~MlnK?jq$8#bWvA9VUvwzg z({^B|X%(8g%=EjJc_s8UKG?HwFB@oIYfA|k9pu~${3I8hcnSJOf18&X{n8f9|W4*TNNQQ@AGMCreH=Wg z0XAAKhJ@n#$k0v6J@tdl6~+qw z*tG3g{89E`&+Yib-0fsDHyO32koV$@GZUX(IC*Drx!xG-k6EcY^l1?auWixf9BGPX zL2J>%JIrO_D1AXGp#9w9#}efdhs~Iq`s|AOCOD>VMU*n|`m#G8JQMzK!vXF<6_BMJXTimXZO~ zf|%6mG?=Ff+BPe8tBz>RkD>LB&Ehe|)>2&h0XclzOtMU$%VK7gami#t}7k0@#5NqyU!fx}jB)^7@x0>j_#ME@oos&V1>U9<$nX z7UGdZ30~GYU>H}O8e`Jbw5oKm6rkXQW3^*yJN`!|}Xu$%Wq+OHcMF7?M+6@{fxeDv% zQ1S70BZFpEtWy`|yKWFzt49&WkHLgQ-Kg9{!RII~V28Bkk-l2URfQ0(oojQ&5R)0Q z^oag8)1_;^P5*B*ViL5t#noJ_oq*%0UbqywD|8ts{x>Q~&%3fbLRi~YnY!do4)FVe zY2qNC4#Hl^Gf}=$|5RV?+3Aoiw_r+a8Z@j!AI-Y);YneN-Ta@Fw`AbFVg?)0VLXF3 zf{bzczy@omv@1)U1E(X56BzIGt~!!YjJCSHraDSm%4`Kdchjf=3M&2`1y(dSfBdEz zHUB;)C{Z?7m!Yd6X>-P`n^QcEN=&*_+n|$unN0fBb>63o2<8XliJ*(G3R)I#K#ux@ zqPCE-n}0CsgP3oma@%#1H(VV+tIquX9-Su`m}btKK!d%ePW=JM(P zH>P^@EkL;IL)$>X=8R+0PA5GSFXp7s2g?JXo1)Z2-)A^{Qt}sDmyDAU$5b;0PCLe_ zB|0E7S{NaRfsGbp5l2|zK9(u`TCu*&S;R;m#TFbJ+ETYetw%J2>qt3>;RN3-89As~ z-1a8~QbCw_MaHImrIEr-#Z=H175^W^e~#h*Jv9GsrskjHw4tN~U+kwRSM_I#<}>y3 z>B$93L}d^F4k&MlqJ{sBBUX&fF0n;~lD{ZSTl{uOuR{wfo=7hhmW(j$YMwAIT^F}% zPf1dQx+I}l2}@Fd8peO+auo!bbdtFTn0vXK-2~qK-3C85-IBqs=IYRGHr*Ktk5CG- zb1ET6t=EgM!Gh~6p}PpAc*^-2G@xo}DnGg%j+r?Mguh^&G>aaLVq5FI4_;mwR`q90 z`=hNY7SSFauj$l6fJ?n}Cx4kI%A|1kNp9zYP{MCB>|+_;5)E6-Zed=M?#b{=@%YJFY9o5FE5xgosG@?bu+Xat&Uj& zyIKBAsfzg%{dchz`i(yoV` zSOB}P#WMmHVVNnRU5}JfRhFqWl%y`%cM$!x1_Fo!w(=Hq@*NKkiCf7Fg_|PiwU`@k^Bu8qY}q6)Cpj)Hv6=hJMr* zE3vi3@}i7M_1JeNTq&|o)2c)Sea;iXlYM0eO9-v*rl@R$T#blLu~$fFvECrp4wxf7 zNdd1E=T(_L5a6ZSWDXOQ+L45+2d|`3ijgil6|$6vDvt5UBvhILrt+b!$p_B-_T%s`3?yR|#(w4-k!5)tu>Ld-V4Q+Q8S?UNNv6HtAUobTY z=aU65Md#vD)Kg25vkoY%{EoLUBPKw-PxYPe9Xd=5oFBthVaWhFUfTy(EIWJ|vnrTi zdBWRx0P$R-0U{8ea~)M_0N|}C2E_hASZ%DdYR0I>sA9ADOyR+{c!q#lo z5<|iChF^aFy)zdufP%2UJN}Z<=9Fo-eKKxaOtEOz#qCd^g` zz_%@1>d&1I&3(54#>!mh9Nz|ayqp14{mDK2i_v~D4B$LwFiU3fM*c}j7g zQ#9-fkfvyN(Tu)5Zj(x>4_CY2H4hSf|&w=)f4`LT^s1LEv&uoWE) z42uqgjt4~y+U5APtP#SeLI>(0`WvkH`~^0Z*emb^j=0wFqVgS^PRS7^qXp-}#$$vG z#E0BahRSH@^?6EKOq?e{aeuMjkZ(_4h_9u*S%*oT=Q5JB$;oJe;x{gYmM zzoyspF>U=-j}{{NVCVyoIwM@~S-jQ}z2?>Q)gSC}ee{PQ^r%JuO?-=B|FEX}qi{;< zqZ&^-_3(G~K1KawIfVb|-MaL(bNc)Iy$H$l7~$iC0*m&M4ng3}fOyZrh{0=|Dj!Y5 zo*nPqfO-#;z};uXQc@t9chtC=G}ZC3(ASCmRoN#qsx;q@*x~SXn2hm0)E~lGi29567)GkEOuM@)%f9BK!3RI?| z5dtQ<($dErU3n!E!!T0Bpl?Vs#`@Azy)D7|$8M!axXn(sX*oa>rfM#duFzFGcRI1V z*pXOCRswnII7-ur^jhK@fv#UK?(HVsH0w*fz2)`Iz^hYYVqyconL9^=_8%|k6)CI$ zyaXzZLR~v++AMB)FdDctKw(HvKD${ZGk`m1Q-Kz>925S}EqC(7e1n+oJocwlCsy+^ zJw51oJ~7E=!yl+Hxt>GP#ZE%ZiQyR9)Hm+Jm?}^6>MjRFy<5IP9)T%NT6dWX1SG|7 z+aYb9pYsn2l4q01$w*Yn3byH)B9F>QAF)wS2d9NfYKase?JD zqeV_L0Z^dZ3E>8{RZ^ny0Ov%@kwVFXk{eBBj9QU;7*5Ma2EN(tWtE2mSJv9~zVR7ZG91&c#-nlP~6Z>Fg8?|Ria?){w4Iiqki=^3MTy9U! z0tg}O#UnyGZI)AC=H5Yw)_u5!z-ZK%RPM1*5;%Ib6X&lbt0|p~lfYd=oqUZzrO~;C zNLU_)B{t?AVpk}WjM;_`qPKN8B>O=Vr}+|QZmvw}bCNnRqKnsR9mAHzI!JA1;+!R6 zKD>@EC$0PIr!HfjdPT~i7XFCkZNn%;oYDwve)f*i8_;V3J=>YpjV3o*(tr{#Ek9NI z1_)LMCvzlM96)PHscFNpQrME|`mr?3?RE}tX`3azUeWm$To9(@)Jhe z3^=`tl<-oC`9qXSXTvc>C(H_aeD}P=ik)exdm$(kgYT*!mv?3+_)b>e5br)JexPX~ zvbRzYei7iBji4aiy~O=}C~<||z%k^FrvqFB55dH>h!s(;)w_Bvu) zo3i;(o~?>QP8r>0f&n4}puT+*N5w=i7mkn^{i4S1iN6ASuqGb8XO!5o(Ge6+zFlzi zg0|EsqYZHHl)mA2EGvlL8*}6rrWhifYCsK-?{*>d`zp1O(Jrbh{nE zFuGN+r0vIlapR{z*f0AXOzc^0QqI((6^?=e2`qMXN%e3>YWv0GbYE|`c!AKR&U>@W zhU7GbbL={_qdqxqR=PzW{AWSZUz5NJaDf_8vhit+73f-HYCcbz5QYr2t3Sx+C6@ao zacJ;IbDQ)rL0```EMBYu9(Z};3*PY;2T%&2m$_umX;pE*G{A&kcz;J6@ljta#o~>N$?BD ziY(aTy_~mzDERY+IQWf+_2Qf`vbVWghkrkK>j-jhC&9&8yzAtyPFt-m9~;2+{) zXpx?5-jaUFOJ;E21j?x8`U$puF)UZ>(kl*CmOG#f^Yf5@`c{8nNnC%(MQk!QMR2or z@y+Jzh@S$Xp_SVbF3sVzHqN_3v>y=T)Z~@yC1k{V5*ki<_C05b=EXk+>U=Ldl9Y8X zlPeB-FB+K}Ka6}(7PC=xd2tkFSEFlucQW=0-CtVS(MdIUOzJS@W2ua0LC#e@$rc}lH0;^Yk2!IB3nxGj z13di>(ib}VCK$459zwj}p3L!_CC+kuVIpljTZex(705{I_jO)R1h&;S>@$Ft)}G%&Na0Z}A&@|7zOB|=Z^QDZQ*x8ehe4LWj%*0e-7$5a#B9E&;|J{rX2j5s<>hiQC( zftwtt0*6L+!vj_Q50{^mux&%B&NdQ-vYJ%U$Z^mKSU|_n$&#c=IeH}x{-@)q!-727 zrZuxX?3t|AEmR*`;};TR66oi{{Oht_n0$_ocDT`{#;6(&EIA7VS=LmN-KkU_rRvm+ zz?=CR$YkfTzSZ;_FFw|vO7Yc&;|)DJWzy)H)|ncDOwG+p0v z&gC$y&wcIBy(8tYI8w93HT_rrl)gB{qfL4Bz;k=x<3e|o|GRmz@~R#`n<;SIk941f zB`5Fc-tP5_57lkx+jkt`vHPT3RKmJIIs{N;II+7d)I~)f?6Dl_P4+;bBKdTYT}EK8 z$O_kf!ejX-eEa4{HKRZWqmM?4r(_#FN{slq{MQ9yI*gs2wr0WMoEQS0m2Q%LsdBZx zcTRfGd`U}>6}+meG1lg;a9r~Y4~$mpjMMA0ee4%zzF_=vwf62!FATR?qUIZ+2t6S9 zcSu9;*c;zZlZDwfp}4q@&~eF4>aDhH&2ej)P3o8#RN8a}91w_Y3>faGyJ!f#d`7>Y zS%h`ZCiQl-S}Sta8IC#In=U^Xt2UBPS4($dn5524(n`goW9*w=d;x8BU#*vF|3tfH zH8H$ijW=Sl&!QBS!@_QEdCkiZZw}lw)l}dl4u9c1Z6?OaVKa|oMoi5S6&m`OGmBR| zvLU-hn1_E=F()VOJ)dsUw(dW}60s*4qd4?BJk$ow{a|!D?xm`A%75pZM#>po+`(9l z>o(Y-X`tM9i!1yrHm({2hH3k=b9CYQ+ni9mB}z;B>>^6xppGxm)XHd`c@glJgbRs4 zFn`QGo;H!$n9WZ6rl0okLC%GD;v5~>as-bn-_iEHeJmF7fd9yv-c`{p# zqX(%N-yP{#=_SLyTmEp@$>5NscrumUNjay$Z{E!ey7sW@&9m78{;(h512lUM3;(H-6UKI0*}BLN>hDG1 zmKT6S+_J`-L;x)F*$)Oob>k7ip75|&n5=uHno*IjzW%v|d*Wz&11P|rsH=OUy57Ab zsE+{8@%*oUe&L^0o9qjJB&RIY-@^sQ&F%AeW1YDmoxR2VLCG8I=AF{qrKAazWu{n^ zEoZNyMqNn(ovK4KmCJX9R@orVl-y`atSvDT6z~Ub%#~)_f)f|ObYx%J%0zRtyhOl%8&#|po;E1 zL&kszv2v&}qt1n~Tp8P6#f*B&6IpnmbYv#E*0d4{W$t}{h@9vY;~7O}o2K*0C1qAh zw-&ABlX!}YO>=eDUK1j)Uvy%aemiFJ)am!Ox_6(K9XROqHy!xUd7(G@0Y(xkeRkcO zt!MCcNDlU1ZxlCp>cGsNwqp3F4D1+9`*_6Moy9e+FHK zpwP5x$||}o3szW*)PbiKoKjB$E3=%yP(?1bH>`aJbrmq*hp<|1Xs;J~M$PTQ$D7mt zClURud1ZL9b0M(j(6QzS`>rpJWcBdO@b0m~_SFwLUk8tbYk+Z*k9W42e|OL@B3%b1 z!F5MucBOy1t~-Oc5^QD0e4Sn?q*;&ZTf9+bpq!=lrCIDwMNwW&ms)KqQtK}WF9Qo} z&AL%@UcCZ>e6w+05*9GNnM9VR{P~hk*9jj1m3_saEfwJ7n#OCBRvOX>j9YG!xE5s> zM|ClI_hr&oVfE^FA4^ZpOby$2&2?dq5-`la{XiAB6^2H7&<31esrEUz_BD2={BJbB z7o+?6i00JG){RGfhf?Z+MQ+BNTKt$W>$fYu>@fRfV{oDETq#x))JnOAiN!Na_wj^$ zoPlCN0Uw}KqH&8c)r2Sq&2I)!uJ{@AENE-ljPFb#@WF+1Wdquud<^~Eye283S^8Fr zDgdJSTzL6GFyC5)c#_>XWX63+66)zPEXT8Dz{=zcs=-YK%vdWyUot77$w@t~quy;& zV8of|ldYj8kC~3!Uu--H(6iYNX775N>$9w0ODoWjNxuMAO$qgYA}{-VNpBj~;S`el zDM;|29z5pFKMqw<2WP>7--M+m*l473-?r?JVD&y@^}dhxfS&dM;$IPZ zmvNReZS0N}-(v2JJPw}nLM}feIWxXb(zb6>%%H;b+EhhpePN#Q0c`+R^$(|U`(8ro z_n7>UKZM+*_RHv>bOnJ?mm5=?S%6ErV7O1FO!HfVuDOs5(XW$Xn`xU@Y93wo@3W3+ z83_%VrbBoCF`oQy(&vA(MhoKG%3a$_qnl7*U?IruIP_G^&`|TAgYD$JXe{7R|ID`= z{UPRnK>261Ru8e9^z-%x00Uy%VkAzN;K2deW5VMANzfj?#UhYY6a^#Dw&seL!m?{n zjmGdR(y;T-CSH6yn3W{$|EFO(3yt8J$3_|ZSQepvG4n@0(ucIXcL|X@`}@+~)%A1N z-t(nA-+31tPn<2j_dKQ$^J}d1JKjqSZ~y!pd;izO6SOJQbO2)rqg;A15K9;)G##QLPy2MY+AqJJoV14N(95Y}%@hZ5 z9MV|hn=xih#YK7kQrce4wpuK#;KaSvSZ->KJQ=EpM=#Abm*}xGA}FL=CR;{RjeQNW z%%Yl82fOi||C%Cpm0Vq`08^{*^(qTFo$H#Di?oe-H;u z#bRwK#IEVD19@l~I{)asItDM$xPi>We7$Q{<9Oh8Q5{bA!2y=Xn0`SAvdbY2#)KV}biw>xT5 zaKHy7Fd@gF6B4uF@3$H(Lz5ajMJo<9i)0(LiF6$t?z+Y~^|-)z`n>8q)g1>qPCq1W zWg3Vpvnr4~n>itOhIBIRTiZy)ZZ;w-Eq@cfrV$ZcJCS=d{Uu;Mjy)|3U^${kdZfou z1-!TtdX3*5lH8_<^h*d*0AtI&#r|O2Jp_FKgpUy4bo__B9;Mwi{IWx+S-!y9LJA2z zD;8R74=ZE+As>l@n`vaI=q42M8wZ?I5to&RnB^YJN?ttPj$}sNn*4K@_Fu^SzqqT$ zYQ0JQba_Ptu}UzJg6;(&;6Pti_z>Hbh0yvT{zYMd00brYe?m~9G}_3&cqAmn)rIf1QGO&4-FBdr~Y|+m#b-{u_THmgjh;P<&P~{P*DJ3TOuinCMNoZqUlm5 zE)#%ogD{Zjxrhau|C>*^qBv{;ANWmV^1&&Y4$v}vZgF~hJJEZV*ZoO*|NZ9=WxCG&YH4v*i z9X(_F|Lk}gTyKVpDt*)&+Tc+nb3`3Hiyhca-2B!|Nb2kC>PKPtTSoY#UY^! zcY0W=X1HLGkDe4PH$LDDH4VS`D+DDU$n5TSmuPJi-U!?cyrZoQgP~8NT5I84^6WGx z)b8(*z>|>q`OSJycn()-P4IMUmf`jozYFX4<3;4H&7Y}WPXv)+l?8py1OyzDrdTuA zsT7`)66RGtgI3}}pmbrsmJpUlQML^dCW#E`24Rb@lDAOY%_Ut10ZTD+@y|51{I~0{ZAwCk$ZN6#Rd6*#ZItV~gcr5@Gs%RKZ<~>vm zX%jU@FpD-@@!$+snNQEil$Xjj4=KZlNn!vZRZCjK7&kAPtBy=&j?Sv1w`<6%7{@0N zQd^`_bECA&)5WoiyhN5<=tcAv@fLtp{4M%?LYKA7F>c8T6hqw5Wx#KZZDZerq&x)> zho&z03_lfpZ&8A2fpxMyhTqVLquYK;*ASr$_YmdBIAZY2AOcejZ_`7W#qMq7u*RxNv` z*4CnxHoNK_ht}2@M7ZC1FS`yCNcfw7ghl>(yMf&1gr;8d+-{MDeU1(G$cqq!tJu4v z^jw#u+d41gwSk�v9+uC$4!Zb*{}JT08<9ClFx}cKIsw4hZ|19VP~>enrUZUubj+Z zml*TT4$PR74%G1(Qda$K)24=DCC`_H36jCZn?owYqeJ~Vwb79>4Ib7LxwO1C*Wa_kR%LRwGY0$w5 zNnhezqEltIL5J1}H!4#YNrnM=?HdWd8eICOEfkOP&e@Rqu$lA^4rbVC8-=5dyhBwsVwz0Kf&dZW~nIS z4d(q*1)p5^5U2BH8gp{gh)$D}sqIhCh0K^q8frHVL&1KwB*)%!_fB^nSua)Yx5$i( zVtU}TQiy-pw4>&mJo@h=dXv&enhffi7g$K6KfsLLg7@xc*;M$eam<@L0>daLGNQrA z1iJ5Y-wUaUqAv<{i%Bk{ez14GKie$t6|+3Ogs2>q{`v9eF-Vt8H$I4-TF1VC<~RXa z(!C6E$r?9nr$A&`;}&v`L`Z!F4-ORZtn1LwvqpM}aGCTt>jK)<+a-+o0a`LJknhd zp#B_d-F+o}`c3+(O=<9)&tQdTWoSRW*9k;1k|2k?(W&-FUP}9$pM_tfbPnYxi~39a z&#dkjp&FxPiWDwAWAi`U84hv9?c`P&)Tq|1?2&u?PdZOQtm>H0GR6Ff`WehWh(Zw7 zn>7Av&1IetCSdG+=1(@^n5_F~QsI7sV zI19f6`j~T;`d-WQRJ?JhZ%Ksc9$=&W#~6k?B6o3(>uGd(yJGhTTDYxAjx7Tm!87c4 z7G;n78_nCKqDzUIYvA+7Y-gUa_qj$|tXfFN#8tS?K8R*)>#!=xL<2eLDyTHHg#p+x z3Y>)k#2x*$iL1ZN17j>FG8i$356_gFBfmUSuhkNt-;Op>g|G!oh)I3ZeXF#?ZjC&bhM5B1Dls%Ip*oQvlv=wxj z%LYK0Ww+b5S1YgbCyrLb+MuP?>N7qeR4uFgkSqJ~cdU`D*am}?+7kj)fshixS-zYZ zp67s4hXSMKIXsv|1nI>tzfJ-CWl1~XJ47+N>>%9l8lR!S7^@b1NAD;Tz!CGu2?15rxaGz)dkoO{sr<=<*@7-ie^ zdlipL^g#KOY^^@%HO%YGOEwM^R-+#xUK^K&0qJm3F8a%|UZ;^Km@=He@F}rpJX3bl zNZxvFb74zRUL6`P$Sc+WV08#p@3BWDF zlN4S~dx;|Q9~%~kMCr9B>-)m33}xlZc_>I0RM-LoN;Pr#3u~=EIiu~5jbizwT(NE2wr$&XGI1tXY}>Z2iEZ1O*b`5Z`SRTN`_}W;SJi*I>QtYq zuHC!O-fOMBV7qjS?T(uxIqe=@L{Wf#v7>r#h^a#L_ql#pvMjNE{hoFo620Lzs$nn{ z-($cDQ~PS}wO$Ql`)FBrWVAaKj;=pAV*6%aNMP}$u8Z$KKLSRP9>B+?#?nbS-g%u= z-tcg5_j7S?4;kdG2%{xxcdvdSB=8?j-!d`kNgFkSY{w>4C(eXUq#?PnCEQ?1oZf?Li zGJA1*o0d#{n`}=1*+5cLctaq+R9tvQg#7Hl;D=rgzEpLoKqqU)IXCSb!ioF4ceO4IERBdtm zXr8S~71%OyN7besi_2~PgL3R#?Gn!!J@aD#RA6NbY6_EpFug+$jSxns%BSedv4_IY zvd9=NQEs@dn7waDXVT_cZP!K*|K?Q2{20MZQOnRF7z8DU!^NICkG#mbwz#yBW9I zmezG)(YZaOh9jICb){sQnT{P_kS+yMd!elL)0ucPdXcR7z6mnv9n`xUOAV?h);|*e zD7~G%onwui%VW8VK+Rmm*Q64x)la)RdTU}y(HKqK~Bk`L| zzWg+avj)R0E5UI;i2riUqN`v?Yl^*=<?BTzByqgC+*p zTRa$7&9OMgb|zcNxI2WGA}Wq+7cd{G3X`1P9EoN8;Xt)Yo(RH6T@eC>PMf;rLYu~?T4t@Vp_X8`gTp3Xdr^$I{3B9&MPD)9fp-uY}AR`>?OB&##mw`+HHY(s8b$6&}nQ($lPM!GXV@ z9(7{IB?VR`8ST|E*ZR|%Mrd6ZFH#uvBmeIewqXCr)52lrh!$hjJndE8_~x*BqGZOe z^rfS6Z`hR1uXq$jlV~T;+KjM3QJwzGFq?N8FuQ*y} zTM||*SecZys@ev!g5{=E=vG{>t*Vjdz6?aj{90tb)a!+l7fo4KuznhgZk)dYuY= zL{$?V;wFVHDK(PLHK+-t?L?5Wp@wA)nhYjs7-eCj@YD_K9jO;Q5AOiNmX5VTCc+Ht zQ$g{i6#40T)(j;UU?kc!mG1QQLqK}&@5dN=uGj%r=<5(h<8wa{0j`+hbpLe4&B^$5 zK~XA=&7NlxUK|?!dS)l0nRVH@iGF!_ZUBMje2Ym|YZbE-*37P4xq20ap?ky$^>VLT zg(3z#IrWKU0XIWzku|!+1o8~pGP+h3tL|WczKzK)A+4Pf@bMIjck%uAAht#seA>j> z`F?9`nO2oe61QBpqFcVhV{{08uL&Ck2WStjuvZ`c_u3H{5u%;Owmi&!mUkD(sg~Qxp|SQeTweH`Xazc*=vgA$BiZ;jZ{KXOie#@ zA_@$sti&B|_#Fpi0pv4?Yte3gU!ph5@K9-C%8##S(5PhZPVt@Lwp{BC z_8t6R%-8VT(&Y!>`|z@4^k(2Ar*d{16}zj6{}{)biAdvi_R~U;Km~JN#e(IVqqHLn zB-=`*)J)~LQvI=-!W^#!C|l)GyEF!4pJZ*6xe|3tlM@^oo0q4L10y0wUf8{RNZkqt29l=J!JhK4_;*v8!Xuswx@Iy3vp$W-ynC(X)*i<%9;K%=Oov2>lRaBYD{a= z`q4QNL$hJhs@kI&4dOhnuuvmFKLsu=M`>vHWLv4$)|PFCSG1M(Gsb(hrrjn}cCK4W zEmdsUFN0S7{mu7SKd8~O($G79R?|fmaY-~4k#e;B(3z|15#R6mL)6S3NQ<})EBUi+ zpCBvzTumLHR-TA5UzQ1fF=bZ6^u!s=S`P@O2%J>KEwY8?sQM%?yrurpvpFC={hDFtrBfcCyI3ywfhq4b7trHa^72Ya*$c|IJMP%>=gqR1^;*hX(4GJmDd^eY z5fE2{Mq-Ti8Rk#tNt`EMf0&W5;DZnd;%T;e_QE|1aHNwuM|zYBaD(~`{t~YjOJ|+` z$ovAE=Q`-<4s6;4g2wj6-o=3@caX#DfxKWh5EgV7bGs0NsFn6rJ#wY)s-EA{eu4}8 zfp%z~=Lb-u2>WCxj`XWs-Av0XzVf@F|B9JnjsGVc=!81r_(lba>GKm=duX~|A%#7) za6dZOmdK%^CB1ZDH3u^#)dK30xSv)#w>Mo-^;|wQZ|YJOC_I-Mkf)~{>Z&wvEi#vj zfGwl~_}AiOm2CIOgloY6fX_dI;JXlm4zENA54R7@v@faein@8z$-aLR zWNu3&{IF=}e%U~pzCrbd3yv8TZM_+GMYk{g{NTVn{Cj57n@@NEeGdJBV1E7g9AtB@ z=#!MSDWDDyxVmc`IIBSCJmv?DvP*+m?x|2wXd$Yg>uR96-*^f?zhO;F`p5SN%;Ib) zv?EJF8%(!+#mYd|x$>5=$c%%qL$0s`L#Kj$18Ce3Yyp9M(G~oos&blQW`9PSiwAoD zgPqHo?_O7Dek4n0r*D*;a{kX6dnAKIzd#nIxZh!=YSX8n@u@sX*0Dmq_yhMtOcA}- z+xrgekz-3ch;Msw&wo58qAwh2Y>96B{^JDuzn$Fwx83_6j5I8%ss9^tu}xQDqTmFw zGMd?&xVg2eZz-awpc`0K5b(kG1g!f=Rch-R+ukE)C6tLLft6A4TQAF(OYw3~v%8Vp z!TARP13{gTMy-SWEY9S$njfi-FV;FYwZJYb~x+{X%gc*KYwG`&@`FYh$_H{lavFlYWftI@aQ*J*O-t zDN1Ks1w~3UG>Qa6W?RDr>BR!buY8XjuRd*(cW`lDS~0#4HlxxPYbgz|%_`l(+q5-H z#$z8qD?hEu%9}g)!;y?LMfy9+yhH>Wwh*^xyU&%=@5q>}vsWP5d-0}K^<_9b!S*%n z-X>q~W)IHOI4WOHhz(_dz%hqY=35gGHikq^+uPElSy@Z3Zlq<~x`GP~UcDUlM7Bo8 z5HL?*z&z*mdfD?o7xY%BGmBy5gZwceB1<_E0oJngo&0!&olt3-5R=uz&}~uQ20^v` zFF;hk(4d~BDNMfYRe3~lhk26blH>GQqS%pT!IBolg5^0uUz7Gjw94)vQNmuNf4{(b z5iLUJyv3XuN{bMWNtgm@;xbZ%b!7>r5_qN)1|;-JKc-C3<&FTbD60Ajj0yL;T<7%H zq|9kde&HV|R{v&=Fj!m5g8gA6#?1=M*eDJNTEy32Si1L=Xl{us}ONemY}fb z1m*D@n$_EPP+0idyr6&4qs^ZM-){&_a&K|skK$V&ig{X=v+ccHPqzvA`+q|1lMr%a zi*`cb{qgB!i-AOn~WO3QDt$PBB@HKU5$81am5`?t-n4R(F{|gL;9KKobHirl>tn|I68BWb`Li^&R#JW7=<1+%<<97W>b!6V>n*Y9_#OZ~G1}aKF%0yIWhP z-lIv&ca`Oxv8oH_r-*-FGdwuSd2!g=C$x{pPbQ+Bi#`r$yGIqIz(NRcG#}bWxQzK= z61zpVAQc}z$z8t5`NV}ytA@=7nwZT=n(26SR_44^9rb2}scn%7zgUz%2qOWmhG8=D zpqb@1+9x)o!G2i`IttA)z;E)rV#EQ&36?gLhcA1azXK<%KUZ-6Y+Mm?D`_26`*OBw2K+Yw%!@%A61 z;r}=2{#T#+Kd{?rVPeYXx5U^H9Hd=>gF+4LzlixfDk~@`!v8E7iC4Mx{a%orem_W0 zn2t;Wi`2fQgq-tVkp0{68_W$F_CFvx8Jc52I!QW~9DqyCrmjxF)UZNhbh&kS1I zOqN}iGwCow?PmNa*xvNSiD^%{#87#c9gXu#+@v_FUR z&AjL@#%S?}|Khlrvi7f$4^faI2Toy}>&S*21g;!c-2nDnBSssuj(6UrsPW*32yP~M zks?E{nM(+iLiMW(Mt3;RSDR8_x`!CjtHU&`UH34FHbr?%i}U@dq}X9c^g{BktcIDF z&w1x9#Jqc{?})S34YxdpOB!(z z>~INjy+X#IC=*N?X3vvIfVhXDUSnI1>);u@pe%kk2v;*BOtPl_`M7&Y5=^t3mtyd^`8* z$FLvpxD&Ot@#=G_y;f9fP9j5UVJmRQgALA<=qG2Bizou%Kb%TgWRI|C+7ieS5?K0z zdp@HA&&l8y)47M|Ns^3A`zdgr*6F3n2Zt?^D7%7_OjR*+v&~%bV0{8n z++5CKHm6C#gz}LoCt$|M%q`chLhK=uGosd{oO4S8wX-OpgUR%t1cM9g6yU$S4iy1bzyHGq$tlJH`#;1Fdu7%p zP>8wjh;EhVceSP;b|7x|4hNrc@V&&x5+r*ocTOX(h9Gcv4hTZX zI7~2aFRCBmHJ>UpY5QZMhDitMp&#PoK)_q2^QRV#f7t2oCtzRbhZ4PUVpdYxW$4~!4bFjj+I6^^mQ&3fjhs$YzG@OSL-Zbdq>VQlt^WFqn1r568mJw(~aZ}6NS#?|Ia+}Aj zNb|9NV|9_+2;j)HxZbifz}UY&)n-@^*6hn zeLdB`%(U!0e+6AAQ`<$4eG_xin$ljemq4J8pdC4eXter!XQjJ&EX!2Gv;M|vJ1xtz zhadG=c68>Suv<0O?hfs~%d|OYQlCdXj>If052|B#XajxPe(c)9-L zsLHwndG@xg_ zj@5}Jtf;bu53R+?aASx7`DmWJ6d+gDC|l^~6$A>PduU0)9;P&%XC*trsG_pGDIECm z>^SZ4bJn=^uSuxTmn^M7*{;T#g(*kHkem+@qd|qJ42d>HCbzk2?MW4U) zp*lkzijL{Upmryriq0?e*Yzj0r7m83yI6;&#lhr3U>PRuRX1_6+gi9F+Y0V)6~bvb zODposBn`|k`b7-2)1c%qS6;PhXN}z`H{L@wA72baPA`j?WZG=2T6j44&(&&C6JPpN z<8xruWJ5-|LJx<$?1Q6Xl*Kp~5YjSmM*wa472-3xK&%vJ}oiWRx zJ>+JuI(Q!lT0Di%UNWZYm2TDZSzB+VbN^U_L_4TzJ9^e^Syo>IIRQ>ji(|Kr(*zXS z7DSJH)>U{>z`DLz+Hr93{)gZvS4+{t4GJ_}q)VasYa zGOj&Hi%WjwN%Uf`qa;^{++aRZkP&Dr<*4$uwVWW4FqT{GkE?llgTg#$!j@TM)Z^1C zYPyO#zm-X|hXrsfKaqO))#kHn#Xn{>MYORJSof+-YJU?kt`!Zb!>tvG0Z3Qq{*W^L z!EPy@O9~_y`+=_>{o~P?iHWJX^>powd4+5?~*8SRafoDn43Oii+N^ z*M3ws=wq>en5b!XggCrVCGb3e>*~tEtn_J>*P2=ZgqEuPXr+7)gIcFIugIQ@Gnc4} zbC5KkHN=vkm^ovpW?$(CV+G3fpWBuKY-vj>CEKKRZpyoCRpoxPMnS|MM#TM+QE%2| zP`X`Q!EM!4yZq5=*eWgWw7i008yq)w$3l{ydexa&yK<~hcf>qvec|4e4$FX-^o;vbGZS==`>$y3fKirAw0-L zoK%Hbg3UOMq(Oo*e6HP*G%dd-9#2{YlECsg_i-gOM;7} zur~dr>FK=3o80*Y6_@3hr_%RE+~@|l0HjtgvMw*3ixWN2x+9`F$KJjPpS`iZOSWC7 ze%a_F$we-=5;&m$EGS%1{Ll-=G;CfF(8w7*RH>gzjEbIsD=|&%OU{#x-F|?-#YAZD zhcAO;*uWk*W_{La1wgN=nTECr!=|X>6&8JB-dsXUZ9$QCldY%7P?Zxbu86Nm-#+mM_DFnsuxgM$Vf{^MB8*8HNLEH zk$M=&J1}MIm=DdCYx@{mSk~|Ib9+EH3+yrRd8q-0fIp%%Mv9EAaW8g=a;;OL&h7FZ z+KZ1NijT%D{)0pcQTx7~z&kvf=aw9a4|fZCdu56N*0L zRM63SO{tNw2L@~3$^4~lm_18yYs~%nsMjMiDc9nWNjLayG0;9>J7hf!w;K;A&lJ8x z6IA zgsu%kN0n%Lof@O=8DLdBz6AZI-aj+wzm~XjKF;WAo-a-veJsUBp8hLo%3$L6wlna> zB~<%oANXGdHF7TAq@OrRLTTa0NEY^vzQ;rOjfuQ~N^T_X-5)Hhq?MD48^=J@>gPAy zUrl^Z6YwhHKx+%XQ`)ut-V2o*WJxt}(0c%H24)QC(Xyu>DOizwrfbEm~l#We4b zehmPRN`{-rxo#2SrrMO|s5;*tTPA?1ih)Cnx;<*gTwM=rN~cV!U+E7CLMY8qG?8}= z*)V@|#49F}V07?5XoElH!$3Le{&Okvb{jaR>VVomK-}T|xZB{z_ZtVhGeH8$<_G|Pn*DT?;B0yRq-z%I2XpXOch!Ix2FEu8!=K1)2IO$Jx|JF% zF*<6qXP#)%FGJi4)cZXNTX6+zmv!iQrcylugv96x%^aHk=3$Qg147{C#!5BL2OYq3 z#+f7s#g4i_nEyb}_Pn3o9wcm?;9+d%)EWCt3G^wbW9gU|LMdjSy4UJU=E;myOTYvb zvQqj3_sm*<$NfnWKWenF)Vv;Ov~=Uvt(*hR)X!+B)%5#k4xUOA@j+2aLz14#BqMaG zq^#GVgFA)YRVvxdBv7f7PO5Y9MCjaH0Rez@OB{0A7?5Js9M))Im3hmVl3#pKe&!wNB_k~4b2#jN(wL&tbX|jlQAOekj6;9+<$kt&SJq5h z$B4$p`(*j-Naf9o1x{n1-%fPs@En;(`=+w zB!xM|+6k)iArR03XLN@=pBW!u>Y-xf3c)U>;}%LU0NII&by#0(#{P+NH1!(w^p|TL z{2VmOi|kGCmZ;SWJm0^b+W2Q^0?_~T1ru|-w+NMf0O?*zg5tkD{r|Vx{of+|e_BPj zs7Z;Z&>$ch-=lIkK|s=>^jXQ%{dRcJ+RydrL1F$|aYug;0{&ZO3tajZuYPwxl|a)? zTqz0CnLSyN(!Ku?KL5AArUTO7Li|TW75FI(Q~9mGfCT)f9#0qE=SGJXw=i=uw{T4t zUgAUt)~2bb;c1|c{o_iYR|po?r6w<`UFC$h7S(IzaA+hCCJBJtOzP=Hjrfhttt!7VsE5lCLtK}o&7@iwUH1g z0XN#>R7ybIZsW@+p{N(PrmlBmwg<;Q<>p~Omk2roOMbIaN5GGaJvnm?1CP~{MJhf5 z^jNX1R?+w>Z@NgJGPtKdzmncNaXG_Byx=^D{hfBboWhhq`>XCDI?kSxXQ(ymIa#Ub@pCgC?#13eJFPgq!SROYamzxdVHG# zEy8-f#`rzXYr9)qJNXP)P8A;H>i;&U3^mKU?r%5n0e=sE{H3*;WVqy7*DVLMn#a^G z@%$qF$$1!YZ;^~~^mo;Vaye7ks#P&b0zs50mCL$Cg6zH9m=zdO=;!l_e9s>V0e4ht z_)IxLvk9_3#7quag&kbv;Gjzb*bov+6{~L_UZ@)N58{}x^s_H+u46%S38htkwLcSVPGtNX#BYRWsf|PAiAI99(5C(zAJ1 zlK0EIohSJ5J)KD1B^>p-A6ZKJU%P}T^gq`cXschZF3L!~SfX<10>cY$Z)C%UdD@?$ zhCe&;7UUR31UwApsEkduL0J^^;Uhg|+*n*LY+10^YdLqpM`1e((s!2c+bdK8*>$ka zjGEie(sqA0+$RG6gAM-gSN;EQ<{7`eA^-Z;f7mgl&&R=#q!$3eG1Kz`S)tPBN3jrq zJ-o19Y8tDj$32gw=X5QhqH=H$XVzMPiU^B^I%H~uwoqMRyDa`q>8#~rNL}?|hacNm z`1HjAQL8Q@8vvN7t&Z6a=A(|-shGcYM>iGIJTDYBs?=)0SO4C6Pl1>1=6p9JulMx7 zL=XWouS9h6!pRs=h2|1CGu!0O#m;8H1Ik2^8qE&WWYhy)bu^nS`#`B0ZWw5~jKM$Z zj*HmN5g6g7KBW4wLp2ZrIpsLr*%QSm{F8^2+1ZKPF|vaT*3J9abUDFh-mG!A69+we z&Ti<*o-IM0n&SiHaNj=na|)nbp?(f1Lougi>p^Jr+zUa;==`HWW4d?TaEg{d{xLo0 zC`cvaV+#3cb;q*If!FrMpe1jUu*Lw8nCoG2epopMZOZ}I-txhqRLz>f%oqzyv(`U_ z8~nM+AuHVJg|;|N+zpJdmo9?AjLHoI0C(-Nt#ayh>rhDT@_kXdoWW00 zI{IV(Mhr|hU=^fN^PtUo$^JPI^M~$0q(V%2CLR{ojrg)NtXzF)dF{4W2TyQz>2#ph z04Lq+{8;*X=DG*v^E5fUal+d&8cWC&wmAP z>xB_y`|{Qz+_{DWZz}>>FUh=iu4o`8efO@X+X5o$(mG~zwe+K>nY+ZddxS8*4$@E2 zi3Qxtk74BbPPCPq8G~ekrG8sQWkda@qEf1#HRRZ}kEw$1=Q~(ASBXqm5}EN+YWeaX zR8)Ph%4tJ2-L3Ix$|4qEP~E1A)`VT!Zu6fj4mNKZ7Rw0}dYwq<^u6}sr66ZhCZt-$ z{He@{BM7=~x!9W4@W|9yFcIaC6rfge*^)I~)nK!n z4B9tHZ$A!a(#xbp)?Gs48q|e@f*3QRq!I@$@l)8_q3k{vqrv*{4^J z@3}X&Giycdi*Fk+tDL;Dhq=Y>q#2lr$ZGEaisIui){HaF@^0zApCVv+64Q`+xpJ_L zJl1E87h27NCQWsUaLC851B3gv@k-UOtsHiNK32`? znN`ofa&{qS1Hw;%2NF6?PY@`Iyz)K~N$j=725#qJ_6QEh7>f04ecerBYknd~I(U^e zGaD-I$mg{3|3oc`HF8Y&-2?_W%WsNynzn~pFeG!X;tWU=aoj@Qv9B0KZ4pooV%71qn8Tu|tXA4t6jG>F&p9)?;nrzi!(JQ$1;2%`TL_#}5Zt1ceb~t4 z>AQi<#=7Gm0)o81krGF~Un+=+c6Tq|78&o{IXK@;hHox9C;p680&4ke!Ji3pv$|5Z z-0MT^--wX)DtEe&^=i%>ouU@ua&~Jv)d0iqp~_5Ec@%%GlLc~LWD_qCr22-{+Uf2+wDU#alcg*W_i8Bv&rX*{o4utEpY#x z+Xr7PuMb12Kk*Iw()V<))l1;$9Pl?_S#0B6B?KMu&O0a_&ud*6W9+fTO4RE_Ga`-A z%%2SjWcv62#tb`!kYQTYa3hmQP1To&AJDAv-g@6Wd8>bt-pIw07uMMFKla`(y_iv$?raGAKPTV*0gNc9qDj zAT)n|(PV1i_cnw+WDf-@K6FGHRGYv_$~MWnc@J&KPFfh-@XP zn($o#HCz~WEA|y=uLj&{`TJ*rI}Ag^twZqcEW>;(4V>i38x6S-dLQJ!Y)Q7`+H7&n0mw7bzSmCn$Cg9h`pgFD+b zVV4-&HHVnnAQ9|=ZcQt%+BO{2TOG%=BR_8 zx%KqN_fgboMtbf2oR(b*V|~@V z@ODkS9ZhkTC`J-hM^9QJD{PkZa=T{G$XmK*U@ADR<}Hu?5$kYzC^BOK0s{Lln}hd2 z55Mj0w(a8f&B6196ATaAY?Ey6bMEYXnn&@R@2n7<#nmo4BqQp~CVx+B znJDac?XfUtd_566tziGjABya=;Pn|`+`it)qtA9D_u#{vMJq0RYhra9PfsGjj1{(h zUX<@Wi3k5fAP21=Jdx^d&gE5v)UlDMD}kvU#g_G|v?cBc!%S`GSaW0#kQ$m9IRTvd z73Ob$F|FX};~v~gaXZ~TKC|HL-{JfM2AuU|C&Rp*UF&$%2ytLk>6HKYahc{MBo76Uy|0{+j`38 znDX#RonmU_fq!xs|7Nx=*N0dJA_BsevnyGR&sEa?0{S%X@jlW_F9wM9*mo+t?Zcvw z;|K~~PcG%k=#BjpcZihl`Mfj+ZZ$qYC)@$=0|Ss!Jwo~s62YG>M%fI9ZX+_`Z=~fl z$}Z%DPcTvlHOR%N1^Kc-coXH1%Scg|`fqLz)W=k8y79Da%+C^nR~TF`b{=g8rMacv z5!vk|6#+GYv!yhtYHSMLrqdIG-Af^=m?+zmNwOdq6OWrgGO6V&{lAG~mX0SRkat6= zxP8pFbaHgx|FrC?t5)|X_2$^4XubwMk9&%se(iUvkJ18iV5Dz>=>UcXM&8(%uE>uZqPsro_U^hb@7R3==sDIa&L%f33w4NzUlO-yq zaHpMmC94hg^Th@>`wC+k&-{9WDBalxhx9A(djsX9F52swbd^$o6xhqTzCMhWUJ!9U zCTLi;bKkfZf7)p~`-<2@nh`)wAYP`Xc@9lYInfSKXZOsk1jz~nJ; zPWiK(Xjow*AvNG+B}b$dNqqV=k3208v*GXd$4@dF@BU&bt0B^^o4*naotuV6Mg|+| zI%+0NwItU-=oSlIV(fdD%$i7j`oKv5A8<|yCJdU)?K=Eevk)z8iv{q&M}e(_&0fE% za!c*0&ZV)s%4>^a3;g8z{qOT$j>dC&bv9qBQ5BZT={dIBzpoZ~2IuoSal|*Mlrfhu z0D@wYA>3&<(1?ibebTJAe&-fTMXWZ5ufDgj(%v7ap+V8;cleKlY8aT@oD@)O)SndE zB=_@%)`@yumSb-qro!kOgZpUox*)kA01tx0o*5Hap!c`uU8Z_?MYI134SYG!OyTpZCQczW(gYYTrJ5fAoT=Xd z{*ECm%%F7;A=ShPl6UP)AQwO+JL_wxFG!Rw>t0UFFX*g2Y_`70lS|W+OPgL2OMkW* zr^w7bSuTf=t>IUXGt5tdJE?^|oSR!f{T`6u<~C5Fs1NG~?epaBJ&pN8b6xT$>P{np zPsNT%^P4eLCUARtZjzAZ8f2SZ*z+Midl|e21TF{imvxF0$lzy#-bY9tj4upJ)B-p= zYve-kk4o~ob-dm~)vP~GlN)b*11T%usvbz=WWN15j4#5XNYs4&i@ulIT}aaFmZ(4v zQhkI8XN1T@UFq;k;^5V0=@0ptMyOe8jtv#trs8Gu@XCV)u@*|WE<(hm6=^>B)Xcu% z^*ylF>x2}egtD8E<8b475A1C@^(^C0N!#?K4@XkbRanw69GD_;^Fgu_wk9(#bV9Pn zAF&psZOp0M(ivfux?n^bX%H4Dyn zlh@3qaOT%R_T+p7vO_J2Kkg%PnQgIR(gM8NQEpMTKMM$kI%X;K&CcyDv5tXiTpiG( zwUB!LA_tD+Hrt?mGJI9gaVMXIgle`D|Z z{tJ%I>`e|02KTGo2F^{exG)-JZ!Xf)>@UI&!G&hkphj5GMteg18^gzb-h3s;Sa%P8 zyuXNV?8SgqY|_Fg1m(cfWFY<)!o*vwIrzglI|`HqvYRaa;HZBj3*J85Q9srbT(71u zQ(Um%Q^EYNgO<^V!|ZqMKj*%3sHVnjXwR5;a>aNqP@@VA&<(hhoo~w;{Gmhd70~;Min#xxORQ@hDBUse zBkm2TabxF=9^AoO_BV4w8TRY1hokM+bE6mzqI%&>?PJh~uidkIp=tH&d0}>krw{$@ zz!mJjJV5Kl6C46k9k}@cdI!_@?flSs2io_cKM?(gKkt$w!J1`6Hd2Bt)u1RdSQkuj zqa({mTe9MY$d}CJIdda|<(r)wCgLi~l38|Iu){OAxwlIWV3LufvNR8=mu&GJxN|&e?MqI80iXqufe0w8;#FFVpAWQ_m9W z29rzi8Z^6LFH2&znXU)k77aInG0*vt2+(v|lcz-LP}X?T3&LoZZsLOFHHR2*y;kbtKgrA+e(oUWQ7#{y}QJAI9Wci)S)T zVCco=Hf!XPbN%@=f4)LNmhX1i?&xdzrRdrRTq=RdDW55QGXVbjemKCD+PEZ$O;Y~Oc0nR)?CtJ^Y1Co+6?3<;Zb)56XuB?Nsuo2~E2Cm%e0$Y%;heZ(o|?3C+!To<8XAme`gViez%xGE{8w z7DIq#H++p3WXV2g<|i?^G@79dxF=@*Ni zKUKK0Kx)VF5u6fRB{+>ClaIMiZ@ zI`&|Yx1r|UNQ=&N=7$~`5q!pgn~>x?f7aw3?3J1GPiGKvro=KS@8jf<%ZIgF`Ob=D3P!-Xhk^fVl&>!BlyTPhh$^Z4+}U z81p}Ia)Ku~oLI1bn)}tC=rX9LIUjbc`A#7IAJyyvo#zRMMb4N0yWp$5f*aVK zwM5jJjh#{Z<+QBp{ZcAyG6NYVW)DRo6A7Nd##a>g$V7v4vZ-%bAa{87AS2x&HXIam#0`_hDP^6)bhP1Q7Iu zJpc?pO=+6%!T2C;Oar~UOmcjKMq}}hvvy)4BNJYoW`eHEVM~=d^bdS9l8;5EI&8|| zFrw6}L$BwE_E=W9x)wo(mILVvk(yTYKassfD+W+wct^EMz>oNO zIHabph$@W6FWS*joE+r)97&aDrmF>Rq9#U2gC*7UgZ_~sogvA+_>`kVwO4S9@Tymn zoCX@PJL9o%fLB%(*_tR2n9b^Oki-43-tlts@Eh?9YwL0(^m>%&@EKRW8Xg1%uayms z6z&MqEkvJ(r=9v^;BFhO>@?Od7EzBSfD7oTe%qK0eEx>v8g0*0Yft}-*>a~48aXgF zz)0Z0GnKZ;*Fn9l!vMA0%;%)9xm;zbie4Fk&qs8`?xcwdQ#9=@Q(!nx&|Obbz$l`K zLKC|{TgLowl)Kt<c)Z)W&p4R<~nfN=dip%`f<^F<3mpoFN?y zx76{_HfSfpjQor<=U)p#0$STZhkh~ebcYzP!)v>RLJ(8bvlRCh(jq4=(D9iZ20->dQ&U|UcB(jLY9@0;Xu9kgl@wOWVXN*eS7Fe@yVGi^03^U zlJFCP@@~>O)-UsGaDi;ptXaB?aNPzKXsu1fXs)Z+`t0iBn;LZToj#dgk(nT@Lcdiu zR{!ECS>9Lez|AgvlqQ&z-%;fD!n-&Ysx~6G!$H{eh>MqM{&7M=lldBs(}KTj)dM&_ zWxb76ghB5ZU~(Z5h0CAl9;=9xK*t~80Z-@LoR_ImbH^gMz1VdzQ~d5WNHwZ!HttJ& zz5UFRN5igsff=7nR7h|hm0udyESWgk9Sifi-C0=LVI#9h8|dR8`lP(gt%>U?>Q!Y* zr7%fSY4H&@W|LArH7`E}-e8W#zYZ8g4FyP(5jiZ;rC6{FzgjOJcTh*D6tRSrXEvKB zFSV+2L<{pJ)&???=?%4p+_VyHlQ2koVw#{-e`Ag4459bM9%v51=fjv&4DzJeruC)Y z#_s-pN#9*4<3ehL*vRJqHk`@5tUt zg;bj_F%w}|vF~PjUGMts-O^{y{2I>RP$AY#5v9Wx-CBBs`?i4+#;RU5=4RakH|y!~ z2(K6gCwHHi|3MP1Dd)7p%3pKB%z$72bh7p%|Hk0GrmS@dG*~CZq-%mE}SW6 zA>p~N(c3sxjI#t*-#1PII#3R8%ZQ#)vY~Bb6W-Vq$xpDzJdUBBYm)$;;_e2$z=B>y z#Vz)!)5h2=bQ)Wne!9RDx+9!S+nc3D+*J9VH`?{q+UFGVT1rna;`)=Q?_SR2F2fTe8`fg(%M0W` zlNSHC=D_{e%KPOn$U`sEI>-*{%NH_~7LYT{HK>0O8f>$fG9RB8mC%9PEi}YHrY>kw zppeq1O47$H=jVy7rjIMyBKF6|sVQZ;9F#PfYv4~5>Glp{y3z;=G-(l)!pv`|WE4wd zrtXPwa^}F56~5AGngy7uCIRclTQgmq7p2mb0JK4YkX374-IvDutK5Uzp7(?8#<%lR zQ_dRxQ(*-~h4Z&Rzc1d}H@n|+fd&wc=_15Il4PjoPmG~qA6rwycHu*{O5e!kB)I<_4 z3{#b+A8G(nmOtM~d{ZtAla#`0WIZu9w{kb>dRrW0bf4|27d*uHhPC}P2Oh+HbB)Sy za3gg;MYV?~A~?#i;VOfLapos2Z7U@O^O(^x>z2~2Y<4;H?^)H5i_)_SCe=)oC#{#^ zbsMlqkohq<*w=EP5NXX9W-6I_H2eFT@e$*#KqCQZ?TQV<{n!wZjx`)JyIIYXS;cMa z%n>>n{GxuTbuwA2IgMiB` z1hfF=u^FrppiP*1M{$g$ys%6Li!5$rto2kcgE*%E6t4`c1J5g*#y6wK4a?0m^Wz^D z^W-wor|J?Zj&n<~j$(?x6w-i$=;QWoqCzW!_qK>4Ez@h8Qum0SlTdnQK6bCHk{5SZ7b>}z3j1fXT5Kb4>I;?PKf%m_n+G|uJc}+?Dh4_wehs%;d$KuW8D_RBy4?%!jzad9HOc@dOEJEbrgTDKiRdDGAPUKDbVdddLVH}I^bkHY{rON@#c ztDadq;@#MLBmSAsGKGdi7gvzjfPq}GSM4Fg%ea`H9em`Au}9o@2GL;y?2n&bVUs|$ z2NwI@VatfNEJpVJNnl|B%la#(&6wJ*_Vma-1UMUZ~CKs|hOjb9l-u-^e#9D7XO&4thKetF9(J5Kh%nWlz{SR)gLg99QmWz7TW z?PQaVf>malAf1-7*sJ*%lcuaB8HU+UwJ>hU+9$S^wki(Cm%-yrz8Upv3Ab+w_Y6`Mi-GTP>+i(Dot;pgbT}9d zp1t4KHX>bS6U1!#t()# z3&NgnRXa0AjTWLI--}wF6NGdPjUGz5%@6AYBR-5zKv?Vs-r2|@LKHbjW&|M+-1F>0H>|TR|+DT-R z6684%u)t}kDTQ4CWH#)pG}-+?PoICneLiX{#bQC^u>9A)*Fnp7HcH=fMQyRl=5Zx zq2(EwhRZ|-YAZ{q+pJBOKySlDrih)%HgkkPex2i6MQP?h)O_j5E+5xoxL@cz6k8&L z&mqU*I>f>6utz)CjV0ceI~RxLn>9DRwt36L%%QL-M;P2AcU2foFx#2nNFm%fQ`XHhH80wPi^Q**rX!t_#4WDbI-DyNR z5>i>_+Il9IKqiJ2?Os3G&R}P^w`yv|2keZ3zDsTbAk@T)r1(8Oi~Nw|_6m)(G}j&$ z@6f$|A{a_}Q}`abXr-6UVoub#ZJj$Q3*nuhPxG({qqrBg6q(uaYYNFi@kODjm)9h? z=V4)c`!mQZNt=`l-q?t=sKOE=s@yKIa3|fW)xjVsx0Q-u?Bq&;-VWM)znEWI)OEye zLOGi|;0F@S6 z6gRTZIoX!F%D&x8Rofb}KVRhv2&TW0#+fJUis#@Ee_W~j-YXyNQisWZ1sNM< zzYs-BJ`p=%} z|KDOA+Xd~meLnEv!2=ztkP-lz8-KyAANo)$NPU`?RLMJ#gB;Ly%MQB2vh}SDbQ$a;|4h3pwmywhkY`+j_|ac;W`V?-iX_em481d*Q-6N?;LbV&9`^- z(XA%dx@MU+!#i8ZR+hvLGRdf3*@e^GEmMDf!!ThDpgRw*k17)###=&TIg|UK^2P zd)9vfDq>uM*u&?+tkILRb zIpR-a;2e9Qb^&zH`c9~$qx;EE{rd5vCYHl`2)dEqg!UMs%CWYT3n?Y0&Z<0%CB61T z#5oHc0So3sokhYCW(DfJw+nea{Dl+?We%Dc9`@*KmTB7UZVXnc@9$*bPn2r$JR-{G zg6E4t*-fbHYFM7>^RQ1H>E*JmRx{oTTTQ$JIGdtQyU)737ecs+}Dlqx% zG*xx6fqSx<%o4fMBp9x9F&qx-$D4OlXJQd*w%&|O^KS-cu(2`zK&l05Y0s1#tr<{LxeL=$oz+yq1xz?NgT^?(G!li zEm;7};~CFoFV173=LBev(88deE#esn-(iqD)XY?7c$&XDBt}Ibe;g%W?FE#a*ug46 zyGD_|YYF(|@+=2RY`w=!105b$Kj}+^XIZ_&gKkopyhVcIhFquwf`=6e_OdN zGJ2qY{$~jP-4E2|>pzc_4IQ=g5uZh{9X6%zD?YGx2ooFloK1%buv1-=K=PmBfU*VG zqY6nTE4IqO!gWj%p<>^j+(NR3f=9!7rWv=;{YDl~hyDoqhInge`XjVG`>4_IrB1MP zzN^7F6O&cQ;#t_EWbwRV`uum*c@4n-4gaS}a?oKfrSL7utU`q}xMJ|(kQ&l_8nh9H z3rwPLqme)ecu~kbfHl7stIvfFdZmQ8sE$rebKp`0t^!5?wVV;Xl3L5+aAZ}(YLVrJ zT6b;V45E2(hQ}%`jCF7;9?qT#EKwts)%6E_QcY_IwtbCFI*D}qTxvBsC)*m*MGn&z z8eCtxJg&z2UYrXOqosNC$XTlblXvG9SMejkbTYkPIPc39Kt4x1X* z-_%}`$|1{nSZs$u@q;!U$~Q zGi>P`?Bt`m-(#mGX_j)FbhgVbfvXcdXXDKx4JwWT=wc!D^U~NC#2f7-+H1nI0_K{c z;B9FhMFwe^9m*b24E4Xx@W6V)8qz*W47NNf42FA><*|cfqx00$a5=+V)4(vP*_ud- zBX`iqX~CH+%j8BqIK~@23*PgGeXmf@%fnP#;;L%0<`hHtLp0O+5>#Yt2 zH=|wztmbu1LBbEwsJc}WC6*SflGrXpmo5x6sa@NlZ}+{Xfz8Q0_PFzM9&eu(9PTo zAQuX6XL|-&CvK)4yF$q55#FvQWDn~M9n9}n%v;cmjfx$wn8hvSBjPNDcz;o`hGVcA zSV_Ph8hL!4d+`=B&k@e!7sBth$yfrtU_)1EgeEZ1?Yr_a-Lo`FIGg01@h>d6vX23k ze$IDD$U6dB9 z*&c)ZQfTo92A~xKkXFwjk=iu!3FPmm%xPh+3Cp*%Jw$XxB-}o0FVujZS^mk#xyM~v zGI~I6odrYkC-UD>hX2&6|E_I+yJdObd2x+Ds~8j%&}fdC7`UZD01o^)#R<^@vkDyr z24?qHa~$UoX#^DEuPK{7$JX;XL1l^bk6aXx=^P2XrM8mp3KA^wujkVR5)cgGuL<6( zS?%;WXT?blMCQjL04kYsqX6UMKp}yIKUiP@$%~)*0Hzd%EKevRL{dZ}M937x#6)AH zU)i9AgQS0w1@uBHB*vSXvY=9;hOFA3_3Lyjm7Cf4(Fv5x+qhwBNei_b>uOdstyOpE z68d?xte&~+E^kyZCcjT-J5UfKa>^{-yDN5fLQD<8?serJ{hshsD{e z6na-t6|mtF#$a(Np8jX zi0*~OHfYua#umG~xUZ-99G&zuO1!6EB5^8tdV1o%Xv!nq=)Fzv_lh#A6dm0>CaST2%(sk_kQ4 z_I`J%VN3W;xXo<(^@YX$KG{OY;M`5KQign3kq+=-FWIi&&gX*~66azUfoGPb4&`gJ zO3SWJClLPkGuqtzz5MX`fQb1mvG$z}y`;C*^Y@YJ)EUXvrR0rX$I_&6d&i(`g%~Q+ zj&I&fcqRY`RzlK{hDbIAI44RJ%t}iraas}uzV4O+UaTWH-nytb%aVEJS`hh=$}HdfUMu1R5-*G6{M^4q8mZtNwsAh#1mcyCTn|q+vkrFe(5MK-a+eO zQku>d^}MbRHZjaWeb8s9rV`Knci$B(5b-hNt+$w#vb5-MWWgs)g<|1TZ{xV!6ue=< ztn#5O#9!Kse_QsznrGn%`uHR_DH_CX-C+qZBzwuGve30HU*2#ubar5y6K zK4Kd=e9MHuIQ11mA%$F4dhq<3TIK0+U|dl^g)-E2;faAGtGu-%Hc{-Gd1D8%of{>x z!k;X1C#KF)A%GGy-|(qqgf3<@SU>t=SC>MODriV1u8mXmnFk3zf?t18%JM!LH=EZO z4TweU;}A-H@y5|9+;)-Uir?LDUxe2LITL5w;iP(=y0+=vVv_vV=Zop-P?;dU(YVSF#(*QOPz}SIl zyZFNSK9OIZWe*!sT#gpn#^_sRiixBg%1ZVxB*kBz_T?3a$zXUc&snXwIYCpAboTDv z{k>gkr%~1KGrcv9`!S0R`c!a>Crol&s@dNuL!tIj!eCbCm4PV~m9 zdf&#KwE;#Zn0UC$homtCx*B6Nx-_A+8opAQw%YVIwtR8H&6WEZ&iPr2?@jw8zH>wV z>y%nvI*TVL2Ns=Tr4VXKqjtXX*0y{h;nL^L3u8RJjZSk-ryb9Z zzm8zSDopg>_y?H#QZ(04lHDngb$LQ0owmUo*Da+c(Iz&rO&vX+ca?ioqBbX0-?Qs1 z;{ltKP3=uFAJ?qL2-%JEL#SNl(m?tRl=^)fZHryz*A)5=*N*($zPY<=84}!u(lsFn*nTWSVkKc61bZRKUeL>-B!y}_&IV0oc??^ z(#&UaM`Ha&t==ss2#rb~n_Mkoe;pt``IKfi8&tBgM0$x?7j?cHec+zG&v2hwp)XX&QGU;yN* z1$NyP6BNPPN;E8k5~VSJrTKzvtRl( zm~UA(t9vR!&!H3zzpxXTe~seE%?|K-LCO9shjp#kYO`7v5loZWNjvT?`giU4xQ|S6 z*VsS1W)ptv)BPUEDhLbyObakqN1?@Z;@`3gQ}RYBG@L0x(#ZviHc`q1J~(SmMl+MV=eveWQb|R>o^aMij`NLLUyCr2G>&1f4t57 zFR@I{JrUBrttGm~cp>S*6QMTDK!+R{>9aS4CkW_FV@$1E-%d!4w0k!eCk$G4{$6bTjx)dbwYVXAZg_SH0(LEy z10cdsOh3;hCMT<(XnyOzu;_zZ%I@}v!r+*;E}&aJAME9hr4 zyEocPp5l&K-)@@DSJ;xN-U!jt3k5k9Lgut42+|60rxOY9Rm;#K4K1S!Et?oDqHoff z^@Rfran5?J{%UB|8YPya#uxjon#L3HIv8IX=Qnh~v^2G|+4lOPw}LEJNInm#>@8gPmkNDost7ENn6p7f`#Q)(a|{MkJ1hzp7>$fb{{oYYxkvsmKCOJn#3c-HNS;v z7ivl%s?ADD+8JKZf`E>++$)&NVy#yd*}p%H8i=LXt7SMK$T%!B4#5m)EBWcd$p;C` zR~+B(52gLAQ=L6g>GwcX1pkh{dFboj*78$z%MVH;(4!4tF{_+4Uf=!G1q_ykRc4J! zWvk8m#YpBDO(9#|8xD32hBXX{D+XnUtUTze9HU^ExOpp`QE%_pbu|g9{a4*Ae=ynB zIL(&_7hqCD7RZIaoOFx_!vh?VX7R@FM=dCEN_iXz4_oSdARjDY@l>H>o4YzCpPCU@Da~onEsQ$VfY2ms?PTq^a+1V59BpP?xv!xQ?$M$ zw;IZx3+gbXJ;#82kOF%cBRB=dZws_xBaR68iW3ZoUSkoYiawU(4F~Xr*NoG8iWv-k zeZocP*t}6}$1- z+7|?nTVz;nc~x?0^?dE8vJlIc_el^X8f`BEfn4dTi{N*q{T793H9cwT)vAD8_2%N) zX8qIh{_v{;fnT$IKITB9eQ_$={Bsa-!CF$0wZ!?>2qyR3SmQBoBy56_QEq5Y?Ik&^ zht6YtXa^C-xjN@J>ICLeTV4ZhFmN(EV@qjn*W(9!jrRrisr5EIr z@R-}i-MWl7*R!|jzkV>cbE20Sj-FB+eSIBP+o_y^RZ5!yu~l`OEEpYBFRH~JF*c+s z;S!f$P7k<4TEb%#6!H@=2l{%CD>}nmX4{k4b2~r!MlC$i%lL+~Oe-B*$qhH8MP2|@ zp1sBZxvbriJFj0ccEht?cA!YAIcvX00#6PSm1rZLp@W5-qIxAz-TX3!&Vs` ziT%xDcylWhEv6o6;`1`JlUN!cK!@b){NN2Kk8P*?!8RQyvw~dDJ3}QE^F)5{=P3d( zoF|`SY?e@*#qQP%OKWF*%k#HfMBG0!vhgAR)d$?RR zPP5C*=sm=bn{iT2vT$T+ONL4GT5EG+RSYOwEI%#vPKkejavd74o?$>07Y&9jHeB?* znpnyik2;XRQ2da8%1m-(;P-b#%h;U%Avn|KkO$@vM@1?V`$?~s6)pORTMg_}>RW4k z*E%t7O?`F^6AUC~Jm^kxn4SXMYsd>;fb-T8b4o-vBnT??%dctZ{z3OdSDw} z#A|Kj>iFG9EYkb)1~z!%&k&Mq>;n{vv3-}V8_}&Qi&;yMq9_VFvtGV42ZY)_olBZO z5$PbL>80O%p)(I%QuHb6{OL1H)NG53Z6AH)6PrY5l)`W{Lb;3~4efiR&f5)fmc*yM@q{`G-c!iC@ z6OB1QD{|S+qSC63l~~LIGT$wQhxF5$OO+a2#+7YlmqE;V{E~ezRXvS8p@3cl{DV%B zi}kQw(?r2Yhr{(C<98->`@`9Mr3qG-Bd%I1UkINBCC!a9Ci?W!CkG7008`smytowu ziiT7D65NpREQCNA<{)Q=cC)X>C_;KNSImFf6s5QY7^9ss>5QfTx?``+6OTRVIq5@u z1@A2jlGGm;oRqS1vRygmI_;E4$({3eJ@Lg`oTGjQ*T-iQw*2dg8%gPxenK;3{Ihro zDT4fpr|Sb#e385MSm%Wh{EF()IlXufUszx`|Fm2wr*Z*6FO%8v`^Of(ZcJTW@2zyc zdXCxszyP982+~RdX57@r@@PQSlt$rTJtHy;3vF$yYUG+XAkwtPhCX3ZJ(i$TI%G&VH(w~Z^7!)x#|YxhATsAkz17nx=Nz85TaXR_ z`HQLgPs`@NyF{MV3?Of6W@YmB;U41l-qp(01{{nm58S`g?qnmYA2;CzSICX1>Iuvw4Q z$2S^%!)2T9$l}vF?|~R@nQnZPTBZ?(--^>#ixzuxMCT(irm8K#<#XL($i0L|H8$>z zcPCbZmMhH|H~UG_RVKaXYgTCyJT^Zd9fD-7TBqVq6T-asQtEy(0kp;8NdY64t>Mb> zeuv9UKSi^`g>PA1a~)z8ycVl`*0osNew@wSck9EC_&$HcGEL-Uceiq-M=K0{N{ALq zw}*GRruWN=uJa0c&7ky};ZbeZZVTs1*vMaTLF$E*u=v%EM=Q%NWV+IkD)Wp3RvB#{E*+ z!TcNe{|VXuKk(_53InV@gSi`Mq=`%bWCDRfPSxgt1lAmIL$yd>&;^6Q|4j<#_i1jq zPd8oQ73rs>-WES0VasX;p&iJ-d6LP1ybbe@qZ{Tnh~g)6vo&y*59G5cI?saCl1)kP z{`Fr+EluQg5HKRK&y1sVKz0NL^ZnaY)e@ul4ubI4+|Pxp?)?1t1Sqg~3JD*0xz7dL zq8W<3`}JSXcy;=7(0@ERg7lETe%?_bD3Do{{`1avIq`w`jGRy{!CEl5Apg2Y$YFJX zLjC;%#h|}IfkLVj0)S6_3t_DHAIa79(P&Bt>d3{$h$A35;LJsVmUGA8kf;}7W|o-1 zjdtGXOPcP`NXyGU&oiGiOJB%cK~ZX=h^}AUuQwl?MhGuw4+o52elTT7vnO$xDI{q^xP#VG{x+CRz zElEESatnws#F3Qd?ZlTM# zm1R$Mx(nzwmfxa?eL|QM<75r^cws=crB9&DtY_mvbm_QY3NJ$Dui(1}tqc$N*X~4z zD>OnXpKKuPGh2f@D5eU>Ln0MWt=3tc2odG@21r5YDa-+vX-0>(loSx(^Vb%hmspEZ zsK)mV7QO4*iET%jyg6i7po{Yw%n$i1?d-Am12m3~VyLPwbc**iRo`_TNDJb{rROPU zQ&sKepfO3;C=Bqu250PVsK~nC_1>XoROl}^%ad{i`d^*!P5n2Z#pc~3B^PnTbN3Vm zCtIu$`)>gz(t-J)b*?)45CPup*taIug^aQGI%oN_EXF)@>Qv`J=tJ(vw7w#1KH0xy z?SfDOFIt7{2+37x7zzkyNWn*$giKTq@GF_!BB^whJM#?|&7yoFSUW`>wn2^AtSo+s zl7s=H=&M(zY?uY5-W5e2-9p&sO((Zd7zj=HwK}v4)Ek81tc1cUj<|!c?V-{gprkRP z42^sT+UE7oO(<~FKq*P&9m~5UcCo5W#D?ozg{|{j1j?Fr5dUWK-#Bjh410wCC2Jw~ zH;W=HV9>!3{+cgPo!r`=bZy2Ez`9b}&n!CNf&~WLfnozxQKiw|d7#sgAVY+dk{9zz ziIF9)7%)>xI}QVk!5o?plZrSS70h?!hG~57H6L3UUye1b7B)UlbN%j?Z5PW+EJ$R6 zG@h6auJ76&()}NmZLD*lH!DLRO?qm8Fu80#8RMgcE(lMb_KCWcGjSt z$1z7qQ{Mq(flO073k1SOP1{gewvHeH?e&xW!US%*??zZ~RDOj~`*Kgt=%c9*C&I_G z#x&tihqT*qs~1x7Z>yCYjF1bE;t?NULDZjJ4QXN=LlZYQ&L%aD{>Ek9A8|oMACw)D$QPqec0sX6a0PXhvo2e!_l^!A zVMKb%44{&TxJQA!pQB9)j8lFUSyeg=;13~9=mWz}blV}WcOl)UlD0avA!n0P<KYexu8U2wR z1rD$yB(S(4rb0|wQ%4`vG9#^chtvkRBkOyAz9nuvH}j0j@$HU0(@&vzzAp1;u_C#% zH9_;qL&ldBbY8F~U>!JM4kVYG`*0;I;6n+JaAyXA{ha& z)&d3!xHU!y37k0O#Arb-LH-2-_D^0A?Cf6ZwxEWNM)eA1G~&6nl#9+)A3 z!dR@t5NallV)iaT1V&CwK$Duj1Ip*j9w;o78U0d?O4SkutI~0P-t1MH~jv zMaqOXSdyDec`7eaIE5G=UQElH>C`R6L7G53w=$4kKOQo5w`0BTlR95#c<9^;t*?xe4oe|MAib>g@b6(C3fd5i?e}DGQxP$R4A_#S0%-^9w*ob9krL^;fm3S)eBeilu8l`d$1Kd+@01Ab}HG2=g|0N+q#zxpcXxF z@h;ng&wHo+D`Ni}K>Oef{K^1faf7aVt`6i(x2>pw#$}{f!N)CCAdGG!0SktKG%Ums zTPri*-BGXMZ2YLi8C(?y+uDhR1X9y@vDXsL& z&wW8FmK44}mSk;?j2dGxSD*y)sY)Kp{h9bYOZM zJsdz8>0O~?BeZ@*e$X2f8BJy!B*-!viY{$INjgBNfZ4m9zc_lFVPym66)o56Qr&Ro zG4JA?qHt=3BM=gnm|`QH+l7~PQ*L^CbF&*XZ9oOZt`#`o2PexyBCV6CEX^VfM@(DH zEDZ<0i$lRC$5PtRY!mJRceVXp9%ArzEC`TqGUz@Sy=pK0h}*fB@WW@Yp|xSuPd5iw zhP-n1Yp{W%{&pq;B8af0$>2SlX2^EF^1l5Fa<`GP#<&(A++dg{n()t-!Ndd^fFsHD zjPato4FV!{F{dwzez`gg8eKnzO$zvMK8ncnH0O<_J(iUv%AteEOYpQUYh&jDMq*|Kxz zPxqE+l1gtgcvhZOr^vaZPmLQM=fpko^prj5KdyX(pSnJD$4tT|QwEkCaM{=P{oCiw zbvmoa8Y4`8-_gwRVd5|C_{kqQ0>1i(EH zA<+W@T{~WX?)j|O_lstvGkR+ONQJhnQa4nS13oLR_Tr>#=pQge`ASno2J^Qs@UcHs zeQ59XX~Sm5$T+C6FWzRb0u63LNj-^}5T2Z80BJejvHThpU5|-IiKZeb=L`Wq1tk9{ z6j&G06-1XBcnEPapjxyBRos2Uki^Tl9mgY;Fy!-rEA|bsPedIesa7 z-X~p6p&6;9YsDsb#c(St>1cyWK{w_mt1> z>V^BA2*AD>kl&dmF6cWbjb#H2-tXX`U7uP>)z^8ms>p167p~x%#){9$C+@V0I}0Xv z-y23)wZ4T1p~3s%JRgY(GQoR{I1mM$F+fDp-a-W1gK!TQcn4wxIYJ>WEd4V?{r_V9 ze~ISse3oT-^sxx!PjE{2=a3iUKT>4C(J3Sh;JGa~X3I?&1uYoZU$>UCTr4;+=)WfW z(f;ND5C8{5((9p{CQDM5>>PN^QlP*G|7k`3%GT{h123M47T|d?C-lR&^yR_ zQ1E}vGJ{4DuFtF!0urjCV*zVr!JvSfXPnT$PZ0`;7WpK4m4Cl5Q0k7o^K%1P00m^W z{EGS45v@hHg5nnRpWk!K)(A8C+<;;s0Fkl4FoDzc6qrB)V;%@VvBsJMnmXRQg2$Sb zM+s&#rw&!B?uvb2u~O$3jIrhvnek$98lt76^OdvYBf0c+`l^5ekOGmNXbCB{=Wov< z#^q>K6cpA2Ap@8nD7nzc01NYWeQ)Aufz!6!i-+tBk95Ds>9^dkwu5(HtZM}W&<`03 zl;bTCz-nP22M?71#0ZeH(E@eOWC4X-6cTU3oq@Z;SwE$nj^zWLWGcJJB|weO?3mRD zwuNU0f~5kYn~E$|P{KfznCZ3~!D5Ba=iKav*aqucKZWe_3xLCvI~gpcR~++d6su3^ zY!*~AF=|nnaI^M{5y$Jzlm%A0iv{KOE~=W1rCfQ(WtKod%gmYHuTdpy=U7^di|y3n zqSnEm3QT*bm0y1IhjSJkLdRy#7MW+ox~rhFx~{P``kg4Sx4pjBhV;l z@hlfGx5RT(9@^Hin-(>JO;+7TcrG!hYo2L97`9$v1q+&!SvEE5E%i9p^jp^c<8s`_`{@JS2rLrk*CsLvZx9*+pj7W#oG z>ji7ZfB_k3l>V_altna^r>!lf_%$#pAC5H?=$vnsFpShek|IhwyY~vh{g^Go*v{R3 zNQYSQH_E!?Z)Ar>$sYN#nmf%r6k^P6B78x=C2Cj!KI|5#u(?#85E5$qyR1utN;%+! ztP&FJ_8mc9%1z7`!TPaanphsas3>q5XHhl=T42vHjo1g-<(HCoBJuVZu4Zh;(Uv4E zOW&$u+_V@?&~(ifOxVBke>==#sob&z~(gGHGdVKN%6{hrE1u}e2(CPzfFjsFRS zHRuBeV8eTAvf-LGLft+%DogP4n1+OAYcTr}5{^6~0m<}Ioy4);pft^VW1{y~^hj0v zC|Lvs*~9K+kCt*MQii*3(C)ZP>{y>Z0Q+3jT$t$TUrebo32D*3mhTrY{5YUdRg1S( zjZ7+o->yVDZII{b5L0@VgpZZ66mNB^>#PI-R{a?jos&+S2w3OZ{gP9uBY`H!qrO7*1SKfr(y)@p`QD&g$h zvpIU*+0bXZ70r*`2ck_Wicm>R^yIPAatp!eM1!W)x#sQE>bXkFcG?b%{>iuA#hWjH zgE7rqh7R@1GM$b$*HGWi=^?Rq}atv2a3EuEG(7<@~jyVutD6;&8 zgr!gxu+d(GK)2vkJJ!TUx(JD8T@eGvC;?rv{ZQFq0Gh#)KmsT@ks^T8MBfdFGN7ew zJ~`|L2JXDDUJlYv!kj!}ez0v%Jlw?sfamS2@g6irI8bdVOl#qFbF!Lar z%%P7BTE+igB!$qt|P^4c>2!F2t|~ z)7ul#+im98kbGm+B=Tsc6ZS=HyMTO&2)b0CG+k&>SIoOax&gB$Z}8OiEVEh~liKm- z?^7LPH2ExD*(vT*{^Hpn4O(31h*2{>6-4QgtHZn%=jFnd97Wos0>)@oRiyq}G3trD zfi{c!loI($j(!SS_u%56>1dyvc<#t6Qek?{BMLDg!QuoM#qd7R3$Wu!w*dZcHs}+U zts8UrtCh#YPn=rH4Kd8$g`W6s$_HcJC`@NC5gbk4+v{8H586sT>{2ERz9`U^y>EI3 z4Mb7U&2x<`OIg}E07@qEjHY)MjpeB zYi2nupWc!a)=JapZf6|vEZvbE>37p^j6CG{#hKqop9BMz2Ad6-^#%BT1EG z|K09BB2cJ<7iAT8QJBOn-i6f$>KqHT-%WPRTm9>+WPN)0qiiE(`rE(sJMpE(GpU9RTasGg(0s$>r| zC9hDqiC5J#*mSU8Hhmhc^D4ovbG}(VpFOyWV;OTpIc7W!dWshoJ}+NhBP$NgS1n2}<_nVB=P2JQX*hduvqsNf%5Ags1(tJBS9T>#|pMMp^? z1K-P02+3GBIvBJEaa)>|=A1r{EF&sEvkTbDdWfmnL|Fz>!e@#FSiJnG_G4asB&E-^ z(%(?0IB9Tiae{DfA^Gt8j+1>(F<4kXp+QZsIGF;wAflcY`uJ7h>HZw~@gZfr`4hwC z^lKH_0Q?Hh8Z-(aTg&H_UIBiHF^c3MGM#@w+WZqbKqL{5XEc6q)G127L`slVK*H7; zV7BX@%Uo?DjteDr@oMLs4iNDtS&zB*WY}v&Lu$JrHa{+(80lhQ%ZSvh#fd(_u(IB5 z+uoPFeg0VC2bQlT{`MnykZ77Db%L24IB95F6GhlW8ZVaoCf%POlnp?^B&|C6sVAO{v`#pheM47&1@<*aSzR#hRLNetLn8KWQNcR}#k8L)jdM=38 z{AGx>i%aM-9xa#2bg5yg!?MpniU!@65Yj_jGMP25yC-xC&z@MgNMIak9BTER&}ph9 z<|ro#`rCZ~jFwVg1r@+pIU@tT6icfY-w1ORZyH$}7=FQ3r5tLEU>Kh^Spp&wz$~L~ zLiMvBZ2D*%>_Y_djYL|3$3y`j=8ikupK6t#bCRR`B+=}{^{m~ZUMp1KRs{2g?7gg#T$O{9pPxriOLj_~n^CraOJ75J_hog+T(OM-DTtoyj1^cOBXz-qb%aN)5Fk4$piLqs9)eHj9W-kvXSl z=jP`6Zb$H%4_hjqGo4PBt@MZ$ zD}Ho?k@v5Ws7!zW&DSGag5Nn8KxJG0SQ=)AntbAXt0bk%N9>_9r57D$4uI+tDCu8&9dU_uAuM;| zXb!t(C9E-d`%|+zsq5aY$qp`6TTM3H`i5P&)gm0-oq;;-cH49>@L0#}OaZ(Qm%Dj^uNkb01AK#{ zH`rKM!{Vn!3xLHe;#v?k{7d_cZ`RoZNpPqmf7H)~U`|coVc}dpXQr7P`+5w#!}2iB zRQgZabdG(`5j%7|jsds2)b6I0&RogIuAR;@6c+_JC=N(sX-9Ay7>J-e4RZEPc6-`` zkNVa21rM`cEyOxGh1mtaxMSuMU(wHgjs5JW(&Of#vH=XuTP?u?#V%sGnz_w!1_*2O zOKM{`jwX#`KJ?wH_(U8C7TBcAF~cRwC6Hd@V<(B4l4|6PfD_H5krk(Bg)O6P!{VfV zjhu#d3Tu-(+9H-nwhxP!I+sAE+HOX^LX5hjoC35Wn?69Zw1-T938GvWi?)l%%6M+xMQ+G|}1Ld+K$qN%89z5l7;YD6PfYFK-HF zrK7>(UB`*07Q*7Yz4-0cdYHp%d{|+>J2NsB8!^rF2XirL&EjQw_;c(c{W?i#f#Z?L z2_^pgs9{?%m*R34Us}z}iq5-W)QX-6^eoz*|9Akm;*Gbsa1JpT1h_%?omLJ`z-U$& zMG-dxtBQB#9YAeWf&o94wgRX2643+>mg}L1)rHlMr0T(JrCfQEl*CLqchJe;l)0hGHhF^xEyMVpN`R7=ZzByphhGtExp@~EQx;X~V`nRzblCK?^j zeLe?ZzWt!BBCT%PNXEBPQt}=>Xu0rK3eB71ShBV=f5%=3rZVT-CfEM;wcxPq?cFVQ z|GoolTKf2;oRlA;ZO-KBNAeTz6!Hra|`Ak0eD>EzK8Q`h>&>hvjnqEO-8}v)C z2ePSIyCBF51OR7qp7Wl+<1rD=cL3WrvJY-)lC4i}mnjYj4Sw&A$P{`SN?Il2T`V5L z1*y0tCV4=UN76HX=_Ze?u3ILvcx$!;qPXKsg6-Q(bPdfa*t^&-@#b8z zvtjn#o|Km3E6%^7@qcNZ!dEGQTZ5EMKV6p-PULLT0|+{D836Z6`tyS3O1CmLxXMh4LN)hL&w+ zp&s zQwZS@i{Ndd7p_$+w8I|I@MGbq3vR~eQ!;l1F4PX4`aPuI+5>?vrEz;eBD8;5lBTqi zO$BX#EyrV%iS7e^As|XfR{0P@RC=Udf;1D$9LM1g!lNUqEQ7V?^mxzHAfV*DP+*J` z3yXS`Yc(OS9Y181hrzkR^P0(^#4>l=^DRxWx`zvCCr~N{E6jNd4-S?eeeC1Olq%WE z+`mBl`Z5$t{_2v5fHH7h<#5-HR5>D?MBXflexObGicE(k91vSflTH6AJ-drag_A_egh&2F)LAn2*RE7i}zc}8l zS(dYA+KDVEf=; ze~XaNME9V7i{Igi$k6{em_12h4h;3z`I#>KwDkN*3i5QqIT)C>*EI@OV1&O;T;`|~ z&wY6`7Le&TA1D~<4Pju==|tGrsOk8ztk8gzjjuX{uX$N2RBLIW7(-v2i0Zbm-&QFm~Ta=@1O?ojl*FZnZw(!@vfH9-`OrLy6LxuTA9kaO?iLsm2MWVRWg|Wj`|D` zQvf!>s<*YKr?0k=pg)s9(~S?NT3esDU7)F}LpO(oYuxLH>BNTM#iTkmitw<_PE3SD z^$($eVhss_q_Adj3mOQD(yxFMCpSI$f`{otKjG9d%;*%2 z_BY!M?)NXtPB-}LB#nk^WOl1JuTTd-k!JM$R?w?>n(7&dYa@j_a^7S`oHVgm{h^qW zQ3E5C_*^SYcYbSrF)PYQ7n(G{22M|!o3Q^LDo6?sCaWasr9hc6ZpR!8nG9S1c~0fJ zm|w%OwTv{vSof#6ZjysjbQFymtK7>t*bP=n=d$Kuy&WYgquPE#`rX1E)tCViM><-WR&5PHJ&5C z>h4bZpkZ$k3_iFSzVb9|_4rOL#GIZ%>)T?!Ij*D|a>5H%&vrjf3B*R7r-i z(kfBPIz|2o1aU$b_zLUkKHBV*GZLD2?eLe;{SWv5TdSZK(=2Sh!p1+*fPmVj=!r%k z{!ym$7nmO)|A;UX^ax=8h!lj_{vdyAQoe{=UBy>)`~fH(DT0+S9q|t*Qu<^KJ#spo zI#CDoU&e&sXGUw?8X*!Ba+)+n^0_KRp%+aRn| zUAo<1Nap^FMPU!qVc$ZDf`uU+YJp-r-5q^V7=2rwA%@=T0){4))NP^sCQyJqng>7(EZVS-0fPz_iK1S7cq zKGoRK@C0Ay0zT4P$AA@e&@~q^VETY(;k8~g+M zY!Lp?->^TTgxNM&BaL>4AT>z8<#et2l0k2rYfv7#0TLZpSEUYi49`<}736 zZp5MDX6)i>wmqr9hQ1Jfy$_1Hr;gy0SR`dj&*R$^Ceg8rrt{*UZ^n zGtaw$+Y8_nx;rDgM-uNNgU6b7ey?zmdTEQ6^`U{R!9x#F6&@u-y$4&vIjCN|aGLu) zUT0^{s%_)44FY6WtU%z$UYI(|nHxahobg4D#&2}VKrcJHZ3>#EqnbBU(-~yKB+B*A z!%aG^^TNKLAaDcnH4PiPYY;vftD7RoHX&=8Blyu>?G{e?#Gy6P z-{Y~J`_vose!a(p=RYa-p>9^QQgYlxT&U=q#RZ&jlT%;|hN15+u}rnE8KntG%D~`d(6rh3wS7f~OEYFbOP{sl!As9$ zVt@ju+i9R`Ao)vx$cli8iGff>i9kaMSG9r^5f#j0RRC)jI!qiwsB?C?8YrGet5+|b z-UvRGzFWTwar2kI|3ZA1=5M`NmrqCt$@4b!xSZx=eVk@>xtbybeBR;)`16swo~ciw z)pZg4(cOqRDWkPjVK-(CJ1wK~RNsjWW=)5NGmo?4=JsrY5$nV_gsp@u|+_i?2OLjwgFjk!X*0X6rT8yiGa zXe`C)=8>VejGc>E46}nY-k6T$sNotuiwXdB=PG4cV}-uYo!gCRWBKC-O?zTRi`ofb zQr>H0Ys{Lm6)bE;auIK*RTnr0D)holnY>AXgM9*Xq4U{ zzHlSvP{)dDAE}^t7b=^b z34pm0&m2jV7;YQSXzRPvViR1U^U}1-;EaAYwuKG!7IF-P6ELP%*CcroDXPHa&7`J; zN6W9AlM1W;1}8bN%qm&kMv&qlX?G`Gd+aHLTUVY7sPx(S4x7WdN~TRwz7T5NMC7(y zNi|8dYt3!#?&pDw;h30v0pcNm>H!NSn=3|ec@eA;88hvTi`+$)wWl!;eLw7Tql+-t z6XX-q5S-o~-5;8~jm{8Qa+zMuA(u*uQHgD6jGM&spm^wuE<7oXW`^}nml}<0YL9%I zddkr3jhC0C{u=mHV>{dgQbWnKX4>y=DrTgNqCb_P@`Xv+F28GnH$ln*+G#ZpA?BZt z+xsJ$$vq9|)Klq71a60R`d}o#WUQ}n+D*{EP>C(L|L3q-a^JOgZXcCv5WyG!9TMH6GDX;R& zUt~a-@ZoxD#%ym(+~;X0ALPv~Uy?a(sHywRLiKY$`^SL?SG5>Cz*icw;<5zZD{LL{88 z0rcRN<<3s>S+*X->LHZCprH>-pX>E4VGS5izE^wOdtIbFsDi3rLoVF2e5(jCfSIbs zm_(y%KjvAcO5~}jP0f6l1KE^diq(rVDZ(r2U}Ny-9xN==MzL{rY_^w96uXv*EJ|e! zdtHbk7OfL2Nl9%P%l1fn7pX8LPBiRecab~dpx%1G%TX3s_A166%yXO4eJ(A zE^Mz7losGGqV&7%8Thek*MZbD{ow-Dx(!WX23F8DP=$^!=6v?w}_&NA<{l}T&ey9?)?C|?ZkatF4o8ftBZ ze{f@%Z8P>2_EgE2Qj!caX8lma=sY=bCcW(0+J>C+E0>+RrK)Soc2}$6ztPQFyq|?t zjUD2fs4%Go@_yXacPy#9Eg&4d2K$`s>goF`r$ogRj-vzQPDL$qRTQAXAO5UQiqlYmB%y$`!gL* zmbFcO3A-b>YP-qewUxa=*jVCwk91|qEi4@S>S>hkwD#60YFfyy4?pzs6t zg}y0i<5Zb*_pT)lLj{&Ls^%>Q4O>v$WVygBnyL}(CbylN!(+Gf*H;A%W%D{zu1i{P zCzU#n#ZeD0wGc8JpE_01rmby2ztBBaOxBUp50c!S1Vk5FS1{-oGA#zeJgIh)MW?uH zjz-iWD{wsD{+|mDFBZBlX!;roj`VXc0YlLdbx88l$2fC~F?Fa*<}Ffe_;c}6N&6re z%;9{?4GQhTfDyj|NGg-BQPl)IAowq($4YrK?L z|JY5IK{_2PTp%Dvwsa%`IJEQv8y<{wLIDOOfa=n{z>%5xrYVaM6d6;1QvtGwIubZO z(kO&jvmu7DP(<>iEDKj=WU~;6y<4Srt|3LmOvLRN=M=gSU@=V=f9mjO<}jc;Nj){?9TPokm1q&OE0G2-umA^;|9 zKuGYWkWnh<>6WSLQ}*DdqlftD8S4NK^H=xe6~$M!8P}G!#(vYXMa)gcIE!ce7DQ{T z<$M%zvqHHstqjI6QSt z=3#xU?Qs6_Jy3M(LF2j3iaK9;Efsk!fSsG4ytLLr)#*4(SxXc!?y|(-uc+3_RI@LJ zwF5O;m|&Sy5ng%IdE6=+1#0?wBOC|&AQ;*=U{KsR2Ks8MYFeA9d?h9Qb=8)lx|*`e zqRxtzZfkPXdb9D0YAf5SYI|)xRgI>qN_RCSD=jM(FY1Mmm#SNpvx)0<)pbn`fciv9 zEv99jD*ZPvV%+CweTJgUDE(zoLDr6*E=w_+4u~BYkU&WL#De3gl8%c7*c^5F=N6Mkmy59)g*a|y=Pw}$#B3eRs2RgW@x`qiS*ms4=GDP zvm%ow`l!xr-j|0X)Y&pHd3f&0bd)>qr2VBpa}XcCL!8v3Wy2A2Qqy&2)q6UJ&x36p z{^qh^4&4p12VX*1I4~EZA9{<1!i^_!?36Xj#7>Tq;A7b-Rk8z2t|220P?8X?V4TIy zWIzCgWw$A$)DAC=6O-oFij|8Z#Y1nB#qw6On;}?`qrP8Y>|}ij6Oy)b#u>!<>jI&b zMyG|CILImbDK=unjxwZSYHigD%x{o{8QF<6Y!{NjeI7=e+M-T|%aQ1ggeP~>G(K#S zUe2XyCDu4FafG`vhz8#k0Ha2H(LAErpsm2~rfGR+5dr2|8=R(sud<9~fAKBH8}2Nm zMnX_$w15aM=O1MMyq432S@#^)OQeQo#hdM?kTbUn-!cTUS<(1= z&AF^?N)w+$`&}=f+X&`t5h28mwMcDtMH=^~pg)hw(op933#)i!CiK}1Ua8ctN>G*o z35&ip5OmfiSs4j8{|Fv@w)klgM`BY__z%lhEGM9d2t`$lkqj2d!?R#yJVL8`++G;K z^<IG|SjCxa886bpNW$Igxnh8!DBA4#F+-UL(7iDwLC-)jXU^HlQy9pH zK*Jl1bn>cTjS8@F1%Hr{8`<#A=;TliY)ul-m{%h?vMz(ynQ?0K4Bm}27Y)SXo5ewJ zYd5nH#+!rX>6v@rd$Mr>St_K9q0=8{ybp&_Jx0%9D=>7S@-3Sod}wdWGby*pWs9Vc zoolz(3?m!{K*Q8IAhq{8S^hkY6N^R2{a(o7?#a9NSg))l@EU=Hj_`!Mab9rY3cDaC zmOKLN8_s21`^ObK`u&A{*H1Bx!sD9L2SKmO-(oYH7Sn`zx7sn%fJ@(X)`Zm{qP-CIq; zPo5H!FQArdG^?jhlkW!NeOTCI&Vjj!`#gRC0L)_^pxXZH>X;sKL)5{fjvXnwh4w=H138jDP;rQh{gAUL&>pzWA6?c|1YIQYM!=8A;}e|n59PJl?Di_=LU4cPd>;|!@#}ZR6`DB~$&G8zHklKCqJ-ChLptXO>9$Mj zL?v|gz}Z0nkderT@fwR^cJ1t@v2J9m`W#7oEqUN-gPao^IM?6l-A70u8l29!DQi)$ zbr}^S{p?xC)`Jp*iu{Y;#Hi^@f- zIkBgrbbP26)Taz8BB?c@b2v&m#_U`Q?T*897Uy)y$6mxxc$v!rq&X^WMHW1vfdb83 zr^e2#zASR1P z+z<|Icxud&6rw%xHSQr_L*XZ!@zA3JGfiXiE|D z;)J?ntUl-Tu?h$eIj{8n|QYOb}`~b z_JEeY>HB$30%@1pc5F@nkoyxGr962$7L4DoAO{b=TFw$Mae1 zJrk$z#I>2&{Cy2KM8DjK3$LO^Kv&}f)~a`i`)uO4=I}?zwm5)AA>818ZdAtzl?xr_ zAkU8+W4A|liO*vw?*Go9X{~Jp(r)yeVS0Lx$GWvGJCJ;dq6*1u&?f!C1E8Lz;~xu+@(Y_+U`<9EA-vx|qGjgdVI$uX86h*e`(S)qJzCRmd>Eb)29ZOVkJK zLIo3t_e>!p0TZ*a{*+Vb<(q?&6e8?v>4c3=nUWFdx8y>D!ZOMQ5S382@SJafB{-n| zjE|<>{9$F-R3WZb&_T*NKc;cBgwUH*?=;Vb+U}o0^xawyptO7cHA3jm{U9Bi`5Eu= zJ?hqRfpIT%n{#%J#&Ob04>uiS%YdWZdY=2Hz_5>4Dmy3d6GZP%upX}Hj3j;xqxRT2 zC^ME$I>xI zy2eQ zRgy*cTpKH>8xsx66pBp*d5Doe_j~zr%hSAo6zi0a z7B6y`!|w8QQ(0A4S^K;3LDg;Yx5|o|Dt$j|c{{!Ltgr=~m9H^wB|W|0D=WU2bgz#Q zX2qP4UQaQ*vKSoHAcO~i&v}ZA*Jp;v@;sx0JOF$r^JUtouC94X(u1%+t8k<~aGi0_ zqIS9L=Pwt}cj|GT2;aVGeQ+<_4R${@{rMcjt0zEf^#E*9$Sz(~ zRMgo|Y_-NXb5cA)>~K08?&kS5?9Yp1NdO#~i*xM{caa0TQxQ2FV$h6Q2{VSRD7cffiIPE3+Qeyfw+Cr_<6bepQ3i{1wTwTSO!(D6=`{rxweiYvranJSrca+L3#fF?n1^ z=4>i)c~^);?^|Q|ZLqDW{lU=eM!<4`y3G7M^VNC24Y7Q(4M8+>kQi@?nTQOAd3^sH zUQGrhv@El}jGgMY3#|J+t6~mt-YDu5BzdxIg=V+$WopR$NwIK4eZS)Z1mUMqz7Wu) zmvW*(;;UzQ(1rV1mBGhbw%T-%@)B?ri!zKm<#rOvwsa1=a_rZNYG%9Wdw^3D8^ysg zhqklz>+w3j>tWf{X99MutZXm2E_jAsA&3t%r$LFyBxSkSXT2cYtS>+ zFh~D=HZqUS(v)(oamGZN7ESJN4?TjO?MrwQ_ zd}-@%B?6s?M)6B#PFj#fJAiNww{YKeBTPMmHA^=wo5y*1>69(WHt(9qKB-`?^re)O z#zskt_+0+6j+r^G^xF8#Yc6jp@)fhQ(p-Y@i5on<4au%HrAaecA2f_LiA-o|A0jfs z&-sU}d2h>?h)T&l$Wp3jYS3GimxFH`lCL;=)IP_|@e2p8T1ce~M}WY*r*`>a6 z&vD>%we`oL0km{>YJ3rfehM4!sP$u!XLbaV;hhh^yB5mTgX7YCpF6)hkz!ge#P{wXJ^(b)nm8X(B=$<=;nZkIz2Yv4u zv#Vmk0rif$ef<~I7C^^vsfieTS5kiIk;mM~8kM;DXH)LO*yEPBC8z6%Y2Mr05yyDa z)#sW7tL~WJ-GO@+UEFJZQpN+jWl5+v6)I;_d(L}Y?obx3mOtqXCro2m~Br*uH3rCAgs+$?*_HxwYiYxQ6*~vcC44O@!d`R`d~YHZa)xp^bpWGf^d#;)(^D{hTVi>&MLF*3z|Txd6>+UDq8dFebM zdn7&PDP1|=Q5$aw6{%w{L#M7xC^ryg1p;v2LM(#F%;xMhZ;^jpD``3)=F9l`;#?eH z{2swQVcwK`HF~Ffa`sMgD6)9M@Xos`zz@mMl+;) z3{OlAefH#_>5PiG@HN)p4A1_>AzC7Tgiupi#t=J4n2H6FCn z!Bo}+sdA)~Q*Zp9=->|*0Je3>^PT*2eB0VrdUWu3GjQRmO_+ZyeYrGR2>t3i%iNh!^t{!K+~e3v?}i75*N z9u{9*gLbxo1i#G*I-SJqTL~4&W~f@*0vAA~iDsHTD&H^e2C#Pmirkw=EN$}72FuL_cK-cQsGK0XK+B2Sh>p%D1>W=5Wvxlg`F+jbKP>iv*9 z0&h#K4W}4iya?YM5jjq&sW>w^!MFgpXH`^fWOetfXNXBZ(w>lzvr=v(+4GxV)}QTB z_+DO_Z3XeP-uT(BPT0ZKOsaIfQ^1ac!h&Rjo@K5?Y&^df-Vq>Vgp9HnhAki`H(N1y z^yTyTh2L_Gg1qiok%z15f?KM;L3?O8iJwXvE6Y8#L`lPc_ln_V&IaSXIA{hmh^7K* z2m1$bC7(weq6vh>9j@NtOGF;nis>StJ24b780;n!H>QWzoxYZEBEoJQ9 zaS|tM3_c~4#)~drVZ?=$ez$dh$IWrpyKex(+LSPNTD~Q4roj9`nYZdB)0WD@H2DIF z!Hn5RpC&yerp`MhwnbabNfE3wp>JD2Zyo-9h&J>(GH#>MZ{K!uEVnVd{&>%(`Zm90 z(rz*56YRLa4AY=3w@D9>MpQTVjP}MrdaE7|* zCX`Vmw@$w}O93vbB#OKKRfSbuFywlWaL?pOZn@4MHT-Nu!ra*s%zPQ>vJYtVFCN*I zAdL6i=zm0D>FoHJ=W0iTA~q4U+%%UtV#q=Ot=Xn$I#m~YTa5rbxgcL0&1v}xI6i`y z&5otqvDeQ2EXsrgH)Q=$QVG3Z%plLAv=AaA!HvObA>f(@q!rdnpOCWf)%996=@nPw z1?l|iiReuQ8I$uP8~{TqDEtv#LVI|1K#28G8$y=c&7KPtkb(Niun)QTNWxSCJZMHF z@mRSyBLVFhLpTCh;Ja@j{6<+(4hn-6w@1lDTWYHX0Ao}!KM)SnDTA|cPm#3(sdFP| z>wu|i$2ot}&T$7dyN;c$g7mva5E|>5Jr@AmP!PL#|C;yB69^EY3@nc(E>F5Q&Gmo* z-_5;$YEwRfyuRxHU}<8tpFH6@LiT5~pENLJz~1AX3xWX9IIHcs&By4>^-eY9%uNVZ zk4BCd)$Jn9WElOLm{C6vltg{FnU+{>56|ainw$S-Oi`BPh(+sT`#hhUc|&C@v0@LJ z+X$08ho1Y)+R2kVCN*Q6iug|4;^NfL;D`9bvCo8%%?^{5bJoihoTk`!3+}!iD6qFf za31_-&2RvKiM|_L?#Mz_SF-&@q7vmm`|GDC`j$dxtbw`k<8w2xjMd(<${-%d1N>3Y ztyxzzaB-Z=EcNO?HP&P^P!=;nUY_{^5ia5Iv)UjoBGb)m3>J((hWlJ(e-4uARf7KF zwSjWU=&S!zzqg{|z&}f<`P3GX-`4OTpK*_0nx&vkj+WqI%4((lR`bOXX*{($=)hN# zf}LOsIM@=3qt^QPp*VCUa{EeE;6MQeBqoH!^l6;6JmEn3Ys$WtLH=wH(Op*lWsl(l z)pFckKmEat@bdso*Jf4V_gn<&*EhR3_`$ioXMp}oBeN^JP7?p@=;x;d(N)n|U@g>9 z&s!*jY(c@h$D~>ElMU;`(yp}c=zKZ(MOl0)fXiB)?YEzHR8>N9%U36gMpY&BwpHX; zq8^2&NI7LR5K}(dGA|8=%#=;E%?9;#EXx=@C4389>MFE?JsJs|phi0;k^IRGMIc>N zADRq93HyV#96jPKl`6iv0`*5-F^s7MI=@u47326WtZKC`N38Aif#nrOQrYnl!qQ6s zkGJ_$XmLocjp-80(m2_@_f#BZI0$Q(wf6P<=U>Uc|1x*~JNegUb&Qh#Z$;>s9s~71 z#S-b%4=8(o*JZZSCJl^#B?o=L(~+US>N1(WS|rj9K{>J8+QHb)LH=$NctkylPw$!` zaszx8Zw|?$sa_N5n{Q5buRdn@dHZ;RHN>J1#3lF-wf5ei(u)WtC?!z+B)uTKg-r{2 zksHg5c-BrQXBJG_$)JT4t58@)|09Z__flR=jo1>5SNP##!%B|4{7JQ2AVlS-Lb4y+ zLc;c9;2j*9v|@{P%P(SNASObHlvQhheE}dQUa>~xPhq+mkXRK(wnxUO0@L{>Cim;& za}P#h{c^nVM*VCeAXKzWjD}kZv~V61Sy0-Xv`&EcvO`X_&xeG*dyi_j3b%>tkokKI z{+e$jx6LVoC{rANGA8_KW`|kRT8Ct5vr~Aej*6CLM*AJJFuWU)=n1Upix4)qs1JXb zT~C2!56+?U_jE~Clcaux!-kiyzHLaTzh05(S9vSy*Q@-WWx$4VA@OWqG%21i|0Gra ze!+quT!;WN=6{M@Yt?nWhLWLuEMwt@%wvV?h>k5059`K67Q&aYrDUWmBjKn}-O=pB zr zAaMz^YX1YbH+GmFYg@q;tw|~J)nSo_HnVZ8HXA1p)&L~3o)woqij-h)lfRa=Eq?e; z=dYEp2w^GMrW;inFka=|vppreaTQTx!W zS;o9(LyaHx;`b=p_TQlsQcObz#e9{<0IXn`6YpH3S}%$cLsXeal2|ImMB@nO$Yin0 zZ_9Q+eDbWK@{%@9urQ@4D`YPxuv8PxIdBFKROTwUf;q)NHw>{V4h*=gFT3qv;mgq{ zBl37?ZWc|y|5Vpgzd^8%`aQ*>%LlHG<^q=+rgCS>gZ`&T^J=#TCkVG_3`5(x34qgc z-(=@Udau?P@CaTd9_Ft12Tv|PBcT_#KTM^FMT0^?SPUS$)MfD zfQdwQF9DAk8q3**l@)1j{Ys;DdKj}$9omiT4hzUkjm>O2v8oo|`s%a{tf><=-z`ur zh#ZDVsaE}~Dc4^e6<~x0IbPAyGH}s5|0Lfs@+MwwPEr41Ai9B&!yPqTUH+nd^I9HO zxEgiqL!*^bbw?^}Hk0YK83qMftJ;SqLGS1RN$BzFz}z(1h`l{3!$1PAdziDyO%*yEcbc7aG_$r&Md ziO|?lvZ*uTc|h^X+1q#4F@A>EEuRx6$u%Hg=m~Bol_{J_fEO%+St-Wmmt&8We5LNM zm3CGwK}nJMMj%(|uNcS@>L)YH5mO)i9dLFPKq#qlg2fm5WG>Kji@FAqRRiJaU!9t^ z#x=On^Lo5Vmfu0D2m2bj(Nm~BIr;}^fEks|_f7ZH?aKT9wIOCFyf(|$KIc%4^H|L@ zM!Pt61B?piWDvVwR%d`_%1NW3P`J!;(xuYku%H55C?muD@hyQ- z576{UyZu3vpg|$87EJvKC!*KVSx*s|=ET2x(IEi-{NoPh<(822TPByAoKHE<{Plp( z_YaU=Rz1Y!WXVPsyLePNQ%Nfqt;4oOy0wH~1*4mWcGbzr7gOIv!fOV}6HJ8kNg&>- z)7!s&7qSk<`>qw`M+O3rK|a2;>b$iKUSgC6FGI>$=Wyjap*$~2X(LrxUX~{gwm>Y| zEUv$xlb^#@zJp6@{-JhByLYs{N~7nY;LL(Rd%`SEim4*>F0J~54QNc?upmx%4@QI?i-+Y|-KK+82CG^bO=dDxg7q?bV zEysNK(Msr_Bkxfzsq*QEQyAda@t>upS3onMIpxzYGZQ#e9fw!of8qQWwEqidOO+P+ z`G49e_~;g^`@Y(W8i4-SwvBH@2bb#a#SWkPRm{2fpDqwSY?Q12cA4x3)6xI8FJrt8 zGw8oT!7dNW;(v!wf-DIC9pVVF#QnET1e_YhCKh;vNyuNa;NO36f;fun~f! za*3G^hG`MZD8Q>)=NoDrd=<@7bzp%w3I1g-K_$ zcASiEe`kr!@YfE2D%2_zD^x3#71kPanUQ04eCYSUtIZy&E1P02p^j#w6w7szG{z<@ z9yt8~c&Jz?S*SE-1|}Pmju}d-d7`pT0x4UBQl>>+1a(81;B)KFI73)|j3J>gw=ibV zBtukwP<~c^k|EH5S*Ry0^ngz&t&Jn~*DXF{qA}YTLoxtBF;kda71@j`TPU^E)HD#thZ$y2v33r;UF6PGT#RQEjgD&dxcuJAu&pTS0 z#tCa`mJlTKQOMq7y5swObvfATHKAa%G9uH@4Dy6egMYET>k7gsIVmfsmEt(MivBS8 zHVh**k)`dk;ro3HX-Hr3*LgooI$kmJZj?*d?m;^se})>R>a>D2JNc+m&5HO>w}5#_ zrEAnCnr2(%lGp{bmKov(2P^diaxwwCps!EFnuDO|4dKojE8^pt8EmKC(hiRN;7arN z9}kX|)LqoDY3Yn}i^)Jpf!79*)2wiU1$9h|zgjQ&04-Wd;6PH!Q>)?!l>SqPeX8D@ zhZx#`_5^F58t0RsMkPE3D3G(muS#x8?;pBOwj2dE2byFjP9Otf_C&vk)1lID^C*MT z?JXn~7&mR9DbFHj3B$7ba>(>W;Fn&6w83cUU85&cC3(kGpdrlkVm7}g);*411Wr4- zL+r+GmIZ%bOyxy{6!UrgNfSA?y^C>@i+uF};3{@@j*c1QmBg2g#K*WBq~I2_k7eLS z@q(?wA{n2hNU@+Byp*V2$A_KB%7`JRLD|Vmw9zfqek4S}?6yc6Flki92pe-y#w>Ra zDuD)fIyzMXUK0Efk++#c2+T?oWSn8NX{~CkaYO3q7Tab-Ylnf<%XJ_s3(-OB8o4tA z=-3Em6TT7AqxIpwAH%eZ7uEuXZo!0vsXg>K6zEXH;YnH&gtE+uPGF^tbBayH?)0-PL)V#wW~&U zo7gg3+up`Ygb*8)E>h58*M?FL3G9CS5;)b62gtBDxYp_x1!wmes8iePal|fC7;g96 z@3Z;U4>~A?1lejINr=v;U$8GG5A+GtcL|dB2m5h!=b}aI&Jyj>ihpb999RM0I}9)e z=P7ge=Cc0!C(hH^xBMoGnEKMkRKS^KT67Et6Cw5dNJ5^Wk##uNvMvX~ce1b_AXbPa z6^6;eMCEyLgCIZ`Z{R*&+wD}QitT#F*lhU>!qCGaq(5S(43}kGtzYHss>cEakr|1543~p2$nm=qAXY(bG1lM;zrrnl?S|6Pf2_oA=mzQcu<%m5SXUJM z(g}v>{?C*8jH>*0GprJd8`{E7jhhK8rqHtrOil36iZ&P2e5SR>Xcg_L6G#B)V3B^V>K;yTA*{zqdY`LZ7D`rbK?c9RJoV15P4MV43Ps$0cmv+I* zOCJpE%Ipe>S6r@N?=|B9fE>1Ep@qaTt`+^)b|1E-D#J;t@AW&M-w`Tqr1yB+?;pa6 z&+yiMv5%7Xumck=5(Vm~tiMadUJ_4aU*YVAB&CH5nD1jnN;Ov`Lm@0$M6Mc^`I&r1 zuPFz3U1~(wqpsEm-xEFj9|_zUFJIoBN@uS6bhTr*(P}|d-+)Q`0R8}Ts9)vEcJ!9X z+<}B`BJ$djar8RKiQLHyn+Y47YqcK)zGQQHuQ~FJD4R7J`Lh!dLWWZ23g1EVU9u5yu!b|`jQCp1iLU~nRuiZx<~G* zhAz-}EAvECM0Fu|0i-`?>b9l~w`e1Lb=2YTVM2)J- za)U|ZDvQT2>Q_`Yc=bM-&*u3xtZn6eW7dyXHk+_{njSL%SS@)*;=cY}71I9H@%gYA!L%fUNheKmHvuXI`H(uF+^MtqUJb}l{t6h&xcHX+u4_w~9zEurF zdKa!N%gwE4dqY}bBj?TDxYb_Pb2!fhm`Ag_PC-^|KdT&9men}V_FU*T+&0d6;oRJ9 zdGXMNcB`uZb}ntW*48|FF`->9qbny9G=Ai9OXh9Ia$9kS#IGh437G$3R#A0$&ri{=`@MliywT~$Cg|MwNY*nk+ z>ON}qSg(AnlLe!7$qtFxC6>@gPjgZsn9T2fUC-{czPR!=!skY#BfIP^KHEshN4_j2 zx$_zD)fp$fi;fu>tE-?2{+if?|50@n__YuuMQt|Wa zUfM>GWw!Z7o^(`th$NvG%-Ez&GE5VrH=cAqhaGDXW+k(5bVQ)15|pSuUFJ zV$G4pciPj^V#JFjC0t{3%%T=+%(#Rc7|Xbn=RhECaf=rqtEG8-<3Bu|XN&WA#Ju?P zM_RUn67>F!B9##sr_d4&i1oycx4WxBb7!rg)8JHTZQ6`0?<~p9b28Ag$j}95fOZ80 z_41b;5yM2vz@9A)!)D{hCj+#Lyf;O31jDVN>uK?_^qCVb z!@_e>x-Ei|xfmm~Es~UZS|hOA300+nBj-G++4+iK?WZ~S_$O`%D@k8B_4_qOQC?P!Wu+BSKh^pzNg}WBc-IRve=4j<6gcuUAc&<> z+vaz&o26rwo}>7vEke{Rrz!ob4Z5H+3)J*9)Bee?(B-?R0dOt&z&BVdl)V3BDEGt3h$&D^D*M!D zs9=xyz(1fA2N6F9wY_d;Q`|V1UCtHJ&dspzBljS-wMK5j4QkFMn|n;FhI=v~F0n>w z;)S#2moB(~BqOjM$nGC)0k}r2e}Js#$QE3XTj_!7D@Xe{lUR8pt?NmhZOE(Ij>PzO za42rj=UpU|tmiEHavSOdoTD_oL$7a36kL=SeSc;#s`Bj6)aNI=d%Z#B#4l|3tKdK3 z5rdlwA0{PVOwS4$2$$9uZZsa#e)h%Ayh<;-;2AFh45^J>$uD=c0<;^n>7Khn-1bTf0Z4SgVta+i-4?2=x5 zMcMc=Rcn_I#mzk6&Bh_uc>JBeko$??DHma;Yc1;y__-EwlUixlZ#?)-3}!QR;L>x~ zkFQnVVEi5ZYevkRH`C)y^#KU(R`ufEcU3Q2jmuQ+-An&1KvbVEZ!5QK+vmLh+xfuE z6ZK@9_ihmSCVNrud*hhTj!D4VQLQ&vt3B`&n(}22Q`(jNY)#m~_Pq6Y7YLtA;M*~3 zYma1)oKsTi+(zu7VLddf(4bTXXZ>T`F5n zHcQXc&X27DzqL_T(f#lIkBBxgBy_|v_q}R+0-%RtSoAiFycA+bO?D^mxKHqe4~&hF zyscmCUvv93T3Kthi+8ZI@79s0mG;@B&|AnkrQd-{;rY_L*#B6~+wm1%3YuJrnp{Z} z)?a1*EUFKwScm9{BdBgoj{+%CeE@)0jzJ7KUordw_(AgB2fXc}wgN2ryB2_-4r<>> zT64cWmCCdMOnToB_S)>(etq!xcmQq6m-N2Jk!^m5+0+l((hqz)-}PJn@sc65>w$EE~HhLdo80HVT&B+q+=3Gs4M)sw4ytls^V@gQlbc( zqIGv5n_$Y~Ro66E8e?2LxqhNNoUCkSdp|K;h^~e0JHLA8TovOt?Q!p8U6{*=*VU^m zb;&H!9DhIOyqH8XT|81&+UDQ0bHyQJ{0tZsiB7>J5rMZxYy13%;Nag=l>dFy^Y?s( zEC9^O_Aj|Q#J6umsR^@$_^H)#oZx^MRc(1}aSYxdTeRpTg(Mh~_*P2NV>5Ep3>cD4 zNg(3_C&m1NQzLnUQL`1@I6?{4rH|-Ggu`Nl{A6IOq7r$PGy4ju85;N!%80;Oo$iU3 zX&+~|KP&t_@9=pL4Nc+1ri8jd=ZY$Op_pDYKFFdVUe>edL#3^jduniY;z$725AGB8 z0RdEZ$C>`T5JH#%G%?U=I+HN;FmmPr;-vJkm*TL(0T|~4;9pGGk0se|Htw{%J(u3v zj}^pkU}q|h9t&!*YtX@o7ZbpHiu5y6>CsN6&G#m%uXkaW%FgT4yg79+N4`4kLy7bg z-^LZaxO+8FEhyR^8Slz;sh9wJLiO*oJQWZ>D(`uX%sR6!Xx6)#a-8yO_0&ZJCNPhR zb;}>zxLWKws$J&mcRA&}Hlsz#F-JK~aq_`__HrknTf70x##H4OP{tFC>(lO31-<=x zB-egavr#5w?cun%{h&n)pY|l>Czpk07aaS+DFbWn7<)(sq3saWtTqG?mpWb>7&+*^ zdI83bTSZ8hx1itiVEg5oetpDFdbDuDWT;Gr({#4+;RdVDfHuHE<^lJ8^V{fi1{fVB z{dXLSV&4r~G#>p@9eV6ujaK{t^EA}fM|`E?K&YgbwZMDKlePnS=XwJ);I)CpK_o4s z2|BXFg}_25De~J2i)tc3?y8a(?;P)A?3sI$K2b$?I_B z7D9P1x~a4Z`jQ#CTk)LR?&ohhOg$loj7Za#`fb+lVhv4VT&Bftr1Gd|8;2WyDs4r{ zNov(8-PYWvZ5IV8@_5g9+|@1UCxtG{MR+$n;_E!GFDR`vnQ^s%Bid5_-*vr~>ULn+ z7!So&mJ+ucUh)azvUZ}sV%2}N_KBATTqX317MfQRl?{37Ju0**9ZY=j&jsf5FY`6B@?4 zV)!FCCOgCNd1dRUj*Krgz7v!ju;%|IWqwT!ryDwVL(23EWlcb0_U1@l8#qm$_FZ)C z?!|N?-)M?Etv29I@eR$ldAi5mz7)2{?oz)BK8?oZB6bH5-4D0x*=O|EaQTI-?rr3X zpvx=A>1_(}|1xVdLc})!R{k@4wpJlZlYBu-; z0wt(JDoC_pdU#0s=rN0*)ba99|L8#c$01?-$0Pll;r#2XT5*g0Q~%@=ssc8YPy`Wq z8-i&_Dg{&oZjec>=q$cnfr>^8&gEGU>5ZovGLN6tXRc@G`9YV*c{$+qX`^95pvh1fa~){B_KK*l&9{x)pP5wgHFeZqkq6kA7^ zrF`H?N=LznB#auuLFULP1`bI5kv8wBIno$Q4eLw2Dd%almTc6nb{Blb44aaB^F7Bs zcv4;Gt}T8~Du9qGm4=_Sq;Ry>+G{QqkKMHVa{ZBaV?o`53zYS^=ZtQ%Po#^7ZTPG{ zBq2mLnLUL~GhFjgwG}N)3!yZt4o8`l|smS|nL1THgCp;SN zTRbhd@3{6LyMl>|z(U+n!Yo$^(cYu)h6G=_;n5%B3^-gqK@z(TWN+}laf$F}H9w+H z24-gQs|%>-w!U`pMzVcbjLKEU)VIj|%~A2+i`Q7b<1zqV*?X3`N)H!Mp4T}^9y)?* zS%N~@{VX>mQd_FsWE)Xf&mlztXz>X7Pz>AwN26@```_GBTufZ`Ch;&zIDJr9#4a^C zgL)aH66^viw+19atgwFk|L79^wK1hHs>i{vc;H{_YxrUT+ee>aQTnPnr~v!cg3^mp z2MquB>bKRazKZ{3{uFQM&{GA*P!Ll6qB$T^Wf`zo0Q>^8Gf1$32!8(fq-yfO&~J$W z2t7Yoj8LF(6t$CR z)_%7|)#AR~YG~e;e*$f+?5iobimxMl6(ZlOXQ-{Z)J@g}5{bhL;{=U5ImkHtB$9SM z@4yNTl-&gr()TQlu|CDN`ZM9Wz1+U>?3J4Z3mB+l`L(mNNJ$!m_xd=iQItLE-zQq3 zQ2PKLdFMBl#{{a;K~KQ(BuztB7T%I=N9qNm2=aKQs)4V|q?l&vit{!=v)6R#^RB#0Qg zb90`ixrUB}BpTwe6kyi@&5X(+a3I}Z@9C7#q6r)C5t0@6B_%uaWv{>LbK^nOzKA?{lPFo6 zXvDQJF?h({l(T68-{}OBoSMIW6M65C3chmXH$visp+6=GN!&p^(1~$Hw<6Tt{utI2 zhQ8a3R-$*T?GRst={z;4f`w;*4nRi+WayFUqRUyRhVVK>mL=B+VBEZ97Uh}1DT90heWdM!vcPiH_8)EK zleDH*G<~bY4d^P1p^(C7I5>rOAwyGwB1{^6O=78CC zT=;=>+F}OOEBmxg+K}_-FM9US9Eey{+HN8rK?u%Y9MvoZ7v3&X6Xyl8R%nz_>Md|P zzlusaRdMBE0qXUw+hU5N!l zyxmkjW4v54aoCGa$%|zS3*zz;tJD~ASOzvI- z(qy+B=_#q_x1b=AvlnE7h8@pL_tj2#Dp2xnH23m7gwHpX~@g;1J}LpLdOY40WWMcVZ)n+t(4XoA=9|7dy<~ zB&%=Jdt{}0@fz)(zqzP?pwk|C39kyHs^3diN|07z484&UGLBS13K>5nb4u5Pay?|k zm%=#Za3e)+M}{V(OA4nRI(Rj5kPh~4mfjTMspbcN%5boEi=Zw<2JD%g%9&27v*p+6 zty8i53~B6()lW@{vyhCIlo?{P;VUupRibIJS+&f_WP$T0Y;G5lan&ztui-AE3yR9` z5@D`(V$2%G5gWb*{PTkQuciH88~jBZg*f@;;&8s~koq6>FnT-F)GRVM{M4B!b_hU> z(uN#1BO))EWr__f6spi^a>_SC37c{1?-H_t6{uR@sqkKpZD1L7J1thk_SK(ZNGkO3 zp1<)`oUKMIR>;qd8_rZYDH5ytk9q%;%&b(;xCjh?J-C14PLq0(wgS{7eexNjI|oKWFq`z569cHu>60nw4h`StyREyY3;m_a6R$*S6TJs*neyMs{aNt1B5 zaR2f!d+oBj%;s~NWj$#mkeecaR2yF`KUA}~2M4?Gitt!WCu|!6O{>MAh2oD$!d^z% z*n0p(Z0l}B=_xpu$e-BoCtn~w8Qkw_Pjx@#N-iI0c9FJ3wOkB_S4yiMq^=?U?sxwi z@_%EPe;jgDFnZtDVvzWs#gLj#21gB$wOiP|nQ$SpZ%)Wcwcm@d`$^CT#@GQxC63QyzjOC{(tGyXe$x2!Kj^1g{6SQ1kgIC zI&yk(gofkAFrrk&?uno~^k_U1)d{j)qeu^Csu^OgID*tt7fwVCvr{WCnIER%3cAR3 zIdkYc9w|+=K=-V!*jHl6v?_~2w9wqKT{5))p}hPOVJbAlz4=G1efEDL`7g$oUz+Kg z{KGm>&R+myez7PMr5e(F-Hwo}nVG56f8rTm6 zn{#9C>Xx3xw3ZsniUNuQyrKD(n@=I=5-T?_^S?5iYcW=5V8_5tM-37Pb24?zC1^TQ z3{L2!#Z@vkGW#pb4MJ_!s+_&Ey_I~$hec{heM4{RU1Y^U`>Hi}_G5`{8cs)dCX3Aq z2|GO_tvv$!N1OmhPaiR6s;Y1GAJ(Du?rx~1#AO}S4HdF)&pyyE@cI#l9sd(aN z1|!YOy^Y)E4=L4^UAqU)nO2KqrD1)S{8w`M4!I3cur95iV}EWKZ9KT$r5r)+@OLdk zz;<=wswB^da7Y*?PG;%2xoO9-AHQ%XGk}4OE23dO@O< zb-}wwzJr)XD};dK6&M+aMyUQVFu))NUsfnq8W6I*{9|>Dph@SIbBY6KWX>^Er|h>IxF~{k|9Je*73EgS%=T$~gal!z_kl%r%h(#4>XpD)^VfhE9{` ztV7pD;3^UzmJ~+_R{?vzTMa#0QwXR-`qIymujt6$& zD9$>c_Jlgss>!JiW%-~vDr-6$6>vr;`B{rYl(Q}G74;T}lgl1HNmyAKFirM=KjpA? zJnl4%%qOaa+iXD`R1&A@)gpYa-wQsB+ErAHs4NWtB1-R;;lj-3RkrW_ARrBGAW6!< z;h9cgSqHjz4VY%3GvaCYsdR1&S)DV@mfK}QZgqm)MdpJ7qD$(6yM_^^+Bs=M+kcbj z6!605rBB!tzx}p;br|Wslyn}5{dH<8^$YYp6iIu@;UZheI6R?ZkBfBNkV%T9Km_GY z;(`#s?dpGz=k*!-v4DO5u@HCzOlehc#TqerF1Jqe@NIH-)2#eGQ{_`f9STKJydoIX zh?uLm=oeyfC+_gKbVOb2{7+RFkQy%KaRqpT94+m<2-;8zM~3XvGfpJqb}s`$Lv}8g zDbc*3wLTD&r#wVxd=1VG)*vVPmJG{TX`LA{U|Hk?M+HFfU41FmG z=wB0Vu@b>@1_S;3e%WGyNG}Ei`FD6lvnihX&IG3o&*o@kgo6c4`>8-cYe|)A#tg>; zI3q=M_$t*`=lS8VRBugJ-O;*dl$>utW~gM{);hGyd*D@M^5jr~@uzy3>5?LcgvoIA zGDI=2m~kW^&7obXFLv{woXPj2+1JM8Rw`}npe%?&!oyazUlrdp$s9O0a*z=aJ-(YzhxmFnk)I%%%6baS~2iT4{W{dA)?XfYg zHA{HNgn8L&aKOG7PAA5tv|!cG&rn&}vcMHBnIew(LN`>(u?K=|IaLbduUBI{0Ii15 zqXP?n91LMn(XtKO=M4WGZ=a3kFFO^GIe}i?9{UO;d)+9%T{b)F-4-{k+ z6ofzD{80C~wp1c+W~C@1YifLF+EzyJd;C71p?3K$vs34&~Rxu zTaO3~t{+=rn^A^Li=Hq6Tw5fz)cFsB+;|Sf!8ZJ-M_dPWv7sN#)vrn>U$P~uD{&YV zUh#*5Vfl_&9c0a(gz|Yci3mbuQW&!bBBWl)dtW7oR`c^%+OTdv9>IjWq(SK;pLi2& zZ*z(g((SOE+lCqjpGSh4HK8qUvsUXRZh6oTn}@3yJS3V*^m*crbsGaE;Z>T)n{ zUjRXu7q`4}-}zMQWbQ;RY#@0h-r()LTZ2py7Or_@`W_SpvVvxXL1YnVf5smgs}JvH zir9`V$-SA{s}E*~V8C_J{lJQ>-($+tiog@Bia|)Nb02^`}UtE^bLO9XdLU~Uq zPPQm``;p`%h{y#h6^BYic`Db(9K_}cYoh0_MSs3`Hmc&k0=<`27(=KANYJM6G&r02 zm>pZZpFhRrervv74hj0D+F7<=9Hp+|q}XQ-F#)fJ^AlRdtvJz5K5x;&4Rjo$T^!gs zK*i;UuFF_fFkmY7@3dasq=`Hdn?yDmIp~^nB zSZom=PJDg9a_YG@_vV~QWCZrqE)PzyA9BU&mBre@;y&%Q^HUbl#I=m}FjZsv-u4wa zjF^O!q8SN0KbsQ)N@RHpzGnH-guX3({U_-E2j?#zqx%*4{p(}? zu7JmtCk?-^Z_d6v3TtX(2MTIxZweb|Y92HyTq=Gd2TE$I1r{~f=X3AKaLb7WmJ$%j z|NhJWI~g=06utacU@7%IbuWb+{eNbHO>M))f&px4Y^kG&qkh5#(}Zduq6XxN2|^R5 zjjQu3REjRDgiDo0YJHKV{~RVE!v_7L2e4|I@jcbv;qh(o{bnoA%kyM^V*LhjUsqcC zxV<%%790#($y<7aV|F>c{npXrZGJp$HVT0o!`M5d#P zcreF-O^T|c9FOX7IcMBnC!bf}a9-qZ%)wU$wHSNGNc*FktN5!Pq?{sVF>aXH>b@vr z@nsFq1*5@BD$iLZ4#FtIkOq{f!HT9t6a>JC1$BCpz)5QsJ$w)`Fy_Iy>#)26|0OSHLPG9M4MPPd}CqRN1Rg@%I538XSwiu2iPX}9fii=phWwK1ED38%T5kZ-GNxxC@JEvYWUEpJQgmG9VBj<|ZRz3944!(` z`{=#b+I+F<=yIq#17(3fCx$+>#n$py7b;UO1Tw< z6{>`rxiT{IQAuGSX`rX3pS7nNrM=?E#9Dz#xk?6*S#RdiQb*(8V@s2eF5&^xzx)m& z3w_azJL=$Yfwk3+`lS5W=hKWSd#C&Z&+R4)bMd_>A6`Vh)3iE*;NQH(wP79}quBAe z9+2@BVmcq7*>yNVxPx(x!$=!=k3~gczs>Cp^Ll{!9M#C|D{R|Ml69Q9xg0qA5Y~E6 zWhOSb?8FRkD2=N3MQPY?q>5>O#5=xmb&U6H z-2dElR_G`~E)HxX;_p-N*cY(y5Xp0&!YtH1X^iad75GJ!S>%y0qTrcYCzP4^rHeU6 zTo7(}|2>;lbW`=*sT-SlA-w^SQY@F?GslYcKGfO2@HN+swfE(hi= zb5%rmRt&*wOgLflsn2#CU>^^qt1dmav>MdMI>{|Qfb{<`HOZ<^$y%nYFtVxY1*t9 zu?EACL`Lr*rR(r0xwA7dRsHr>N>6WW!urn05`Vjx15AI!CX^j2;9PE(eJBk8RCe&$U-JY~_112rcI+*$ zEsgp^BsnmFzib*BC-Gq^FK22!ZPKDe*38*GbTTb>l*VY`Yn~(LHc>x&<>kxSRb5TK zObV^zeod=}wnIh>QrGt71SzwAjOv+Bl^1Zko(6Z!7=%5P zZIf32dw-h7vJs(5&DK{t28wa%HFDhwPu3IE)~7^kXGW{Z%6A2DkHFtUpK*VemA+;= zVV#QBeKp->KF#~fJ?Ckf@9p-e6o_RPzs-a9(y0*2*N%ngHyJ>SA0a_UOU#q4mwx+D z5Vk!t4Nh;E6tv(M>{Q&@r5LX|N8-Fbc71wq{>7&(ePU0ii{|gMd8Fwy5Lx6F;=_DW=53(HC zNt>i#IAZ`vSKT!LY>6iVQ;Vg*e&xP5p%x1jkr#MPtfoZ02DR+LdhRvc3S;CvdHdT1 z{EB#ZB2mX0k$s*lc2l^S4Vs2t(oOX`9gw**i^T5nTN5CB9q)?2VEB<4^@~(m1uD1$ zm@#IBZEkE9l+C1~z$+3TvI0l>tuW|}j}`GZ2I6RiA&#c+nPUL-r*1%>KjkFa&lUFy zLsdie3#GUPyErt)#k2}+T##UqMyNwqETCe0cY|I~NgyN#iZ0*@D=Txlt^URIZf`47g-wx1rc8s`{$a(_q1SMswS;#*jA)k~Tf28I2i@Jz=y{@T-rlwo`y?D$HiO_e)Dz^9VXpL~~G2!tpOs zQ%n!gFZ_&9_npIVG75yjCP-qOlpLqO7Fnl$s>*4T?}35BrzV!=4?( zFI^T^&@~1+Ol(F;(NoU_(TI4dH=rt*w$F*t^v2{f(_a%jg1x(AMGyjVMAR|0dC>4;6VbKKfSbrv$iSDNGa*9F1+ zpSCCAON9_RY*5 zfh={lR&0vRLtCFkeH82TORp#CEO@}jWI)N25O@m75BB=o0cQoiBthm}vo#mH(8fZw z+o|B6YlnoFw`-hbUHGao$EWf}t_pw{j1LPJe;A|CDD<`YL<3@1UCd0rDOL+HbA{sM z^sD_)v}*FVWDgoKsdLO5Ag+cJhK;S`rxw>{7WYQ`tD@WMnQ?gu>V*v9mvZVZD~9P} zXLH$c!%ljyxXXY(0P!M5i*I z0E5{tts^OQ?||uybl=(_O_m|0?A1iYuH;!RD^S=axvfJ}K?CopuT}w;WVR!}Q#PX! zZ4j-r_=NUTQ>TqF<$gt7TNu_mVvCFO=*)y7P^?*Sm3GzgO*`|tiDROs@24Io!c;{= z$mkCuZ}+fbO$PN3>Qg1ur^0~L2%d(Zyoo=f5|lA7eHQ1`S2-xB(hx)2Z8)fyWM-~p zKAZSc#BqF+R$fXoA!5TzJmxwJO}j5BV+MX5(5Miq*#*cl+kGc*6JL6tL(cQO{hE?H zQ?iH}AmV;dJHP$^_;hQ7T=z<@4Ab7bQCBth3Vw<_yygU6zl!a)3P=MWn&;AFW%Azk zt60#qZ7hb}db8>{UJr#}v}_H9Nw}f_8P|G&7-A_q?6BlW^s@_1mW;~ugm2x-8oSao zI0&2Uu;rPqX6alLgmce4PS?Q43%)G5)tRCmNXaf+I-)JI!%Bvhb#;fmQMj!!45<0h zd`ND{^SUCbd#$lPP#=KvvOL%kyT%O;?31ySBMlDcY5DqRQspR{n^i67*#O>UKS%l- zQ)&^dB9FvGP9d1I=FUZ_hl^`oK(6nuk*S}KQL*F*rRP{#!dy%k;d*m>b? zBpN*7bdW9cI*YD`Y44;YoE{U{?;^MGg}1#_kr9sDQphJ9~bs-plTMSp-^LP(ehpIjt#7Mdvs@%T-vMxb-S{Qa6ODsM=lJ-wt` z!VkXRg@?}ju#$kQLBEWIl|)V#JcwCSuNR(n%VP#}2*>bQN6@Hv<*zuL0)1tUsM>J5 z3fT({5!<(FEgz>aIV#86cV=~)T;KjNX3xwi$N2^kEhSs1TeJpCxJkerskmrF`lHnp zLz*1fp3;f3amgXF*Z~Uvb`Tmh0{lmWn8y&-VpNj`@iHLOth(8S3{KNJ5bZb`&W!+p zuQR|*(?sO^wgp#!?6_#kId~spz2z`&GbUCQM=6Xfv(JRGamqK1NvR<+;MS0DY$&}K z6o|Bx-xbq9RlYvq++$a2q9%f^6pH~OeUXC5W7s3~Mr(*tzfrNprVxMG(;n5ZC-h)f zmCjh3djL+ukCqi_dLn_nQ&>Z}hGnVyIDOF~`Rn@58_1|BNW~R$`?gIx7MN&zoC07e zip^GpIj3e&Oym%Zju&pVxzXaw{eDEkt3>dkPIg%!Z>>_n7wF#SY2%2+t%eO0pfDEf z2;YZAd_d00&%QEhEeiXo z#tC8D=2iu=ly|hW^o!sZDsYI#jV0}G%e!j}4D+FnDr$h)sF>U z#7Y9QlHN6l!9qKuG?#pQ7iG*zo)~Hx4 z5q#6A#VK=@nco>(=Ey&OkT)^0v+jU{{xjdyVht+~jrb3z`ZH#^9QnJ&GvD_qB*_j2 z$b`vw2wXpt@hDNC0JOi@9Lm_}Bv<1KXk9WD*mH9UU4;&GSJd)!ZAmIKn~1JP8_ydS zXRMIP6{NIZa|#8Z??wHTHVXyf1$2-sq&-lCJaG6NpZj-^-{Eo0s`eYyH2;RhI~}@- zyIu15y0K8}zg`73{V>3OX9cpomLd=TMYgT_tOo9#e$7qN36S3Pg2eDsw|Mw~dj^7^ zA|U{(cIk#ku0hf}B9w;n6@l{|@8j#uzRp4J>=DkohJiN@H22^TagPq^3**AR~0ss@jsn_g0IOOlN@OV2{?j;DFLjm&7UU+4JfaBlNI3%V^4IL^YWh&=T z;I>W&j$r}msKx6?tDS(d>QnB*PL-9)=cKbE`=;ywxs6n3+OE?T91C>@T#Z#~8BGj` zCJ>b)@8WYfin&F|@@kWoc%Refdfn?cw>re)T3{FIGpGV$EGW|Emn6cs~#l8nrYXU zd0dAb;1p+O6_;{I@Tc*J2fMs30;gi3jLyt3h6~la-6Wz)`;qoGoOS`%q1`$77;nx5 zWMz0nlezG)?&qWKH3YpO1W43yAMs&|#WoYMl^3Umep6+Mu}d5K!iAVylHhg9Q4Mv! z8D^h%N94f-10U#VG^3M-l>OdE1M%Y;AWtPmi_#$J(mt?ba5XEm&|1WXI_Od z6Ul{G;YmtKPGw&)Y?FUszRt3OMFZRiJH=8-PaI%LyR5W-UK4l*%88=vh3dkg(EGWa zqdS=z9FFVAOUL9f5KCEv*}UxF5|j(Z7IhIZkQb6UVs%#LL~iWaR<)_Q_$Z&$foUZI z_|;3b-1DY+G{b-bx46Koq{3=mWU7m!LSo_^^k+0xla-co7YOcBCL)-lA`Fw6m1vhc zFU4q7fJ>G&q5yWRH8(7{!i=+7ReTo3TU5NTg76|)$(<3)`omCm;)vsz!!WU|k*lYt zgZEGRg3pu#`P-EhU(Kr00ny|n-GIXdz{bUO{SgLw3;Uc73|!1|{uXDpFT}Qrb?lK2 zdccp%;M|5mH2KfsZ$pNp!ehbAx+;~3q;^Z;xs*W7b7 zj-Lqp6lh$fO0}SXY(EvvLy!o$IWt@93r{eILUs1OqKD5_Llq(8+he$ zc0gk5FH|$8MV(;}FL=J9I&Q3m8IpswroZ{LcTcg)QbyR|>1o-$RzIN%V|y0y932a8 zXoorl`)|UpG{FEj+655w? z8G+(?LcQdQV63B(eb7BUA75QqoXd^}H>z&H^tMZ^auv@nu1i@@2SjI#m~6XEHxQqQ z4$u|(UO(Yb=CuewtiyKGvV?u~MZS4E;#NOTzPy;c=p~@a8vNs}-8qoZ3kw$n2igtQ zjj%phc^peC2bSGAkq-?KK$8M~%t=s+UgU|ZOm^r259VC4(W9IVfU9V#|6WNZJL;J3 z^dOWk4A!AmVON6Ag@a35{-!b58bE^k(jS2ak+$qK*Y+z=0Bzo}WQ_A(7<#6Cek!6DN zV#GCzs$EVfmeyfc0DY1J9v3_P5A>KvQJK$Fu0iyCQq4%u!2k@w)R|k3XJAjLOLw}+ z)rJ>d4G%;l@IX}PSnoF&>1g0;Bh!xGs}aqgRwVA~wC?C+4>lOymRBdMo5}W9V-+ZN zsuIuYY9NcBIud__v~5}Ah)uzpn*NAhz{(JT^6L{ngNZRz0`vltug?!u?o!^y1UW7d z&7iNdBWiBDfh)McX@!>3YhH15=pkT!A=sHTiyfP}$6o^lTL-zoEJ3p~w~IC>?tllu z!mvZ7V{DSGPhP|A2*X^Vcw%gl-ZQP!*cd(=?$`zU^zyy3QZBOdj;d$Yn3;=A00d~{B1n9l*zM__~Tzi+c&tSTMn9iKu z*Nv0=zTp1}JN`Rz_+K&PpP1r~l;nv1TUlXFMi_y?_$PZ9x5-h0UJ?lQAKfLyFGpSN zTXWe?-U4F9NR}RD3}$l=wk}Nch=>$ZaMhO!kV6t88ts$zllC=HihQ=6 zT{cD(lG=uB7te}h&h&q$y)tF3u?nETq!bl59o@fA&Uwkb-7PiY|1e`L7apo)apWSi z=MI3mw4gY_=r0IgUl?(c56I0O&*9&(n?qd4)D0ojcG2Cb<^b?OhzQh*H7y|d>UN^T zRE)-IB~-!SGm=DEc?8f=>%|Mbq&W}FUCIeRSG)HS%I6-gtgmc$syx*)W4e+k7F*5M z=^zG!CvEJgOxG^%cUp9!JZ|0!QY5uik`Cp*Pv4Ngh{YQi;9%nwE@I z;txh!MEsYysT~{a?Hps&5WBAIh zF5dxa_~4l^!eE&HEsI0uB7CAT3mbIhIOMswT*TK&h7S;)yo7kWaH5aRUC$p^%-2$G zGvi#h)qa>xw)4u>QAKvjq7w(<^B|kKPc=R61GczUb+bq;aKRHc2|2k0LvsGnX9i!P zA>lpHU8MOQxi9WWMgi$eHX;|F*#oALpw&Zq8~KRL=szY%0Y9A^m+S3A&e8G>gmxASwam%ro!WO}6k$$MyiT8q!WWJM9aGzViqF5CG zOIz9!Zkd>p#1!!@QY()wGt!o(#CN-!{EE}J75D#C$6O5uuIldBpt{5U2dMu8xc^TN z{s*Yh&p`2w-#{%MGW`ksZ!$^JS3Og5cQuwOAYvm;K-$lqwvbSOPARh&nn;xtMS!62 z_i731>D-_usrES53O0d$@2>=OD!+d4YsFAa?gAly5lNaAW^V4IjG0I`z~}2DNVks) zu7*Zk?Il5we*lf$8pUdJpMk9;=>wZ^tIDSNFpGb*frgb7sovvOgXr?EZ8j10MEBk& zfDgV<6FvZQBObADSCE2rAg>9g1zjqvX6KFqslH($!LK}U5oGDiVfA)n=t3f7eJ;x6 zBcnW8K2*T+MiO0IWnhBqyVLKD>YIf7>Yy$ao1_^BP3G-m!~KTt4lGX|B^lrZ)WxFKQlD`}rKz`d!wE9%lmYyzMB@qmt!)5A^AJ|*rA{Q5)7 zY3oP|Sdl?UZXu;_v#0E4M=dwnPL(I-jMIuJH zBivYX_qD}iyx`YHmsfucSK}TG_FU;|$+QUBgv08H&EkZvaynWN)`kw+czjVZpa_Ty zYQMzS3Ii(iyCUlxv-#sO73% zGaOc$qhs3sPuh&ab{m};;7Mdp`T;EwO3kl->Ggj=0rS0L0euU+|GU!tvzDD9rq=d+ z+mM|wCI7+wwjrzbqyS6)ZTAx~8Mgro0s!f*yp;GEL-RH=3u5EfLl&H}kwg=z2|{MZ zXw8sfz@V)}G&ZFnA`y)S0|{w4n{aO1NET~Wbk;1(4I68~P+(bf!6Iw6TAw*7!_914 zTvs$#KSw)9W;&a7w>dXgN3&)$q7KNue4ONc<9u2_TiWD(Li*iuugKndeKq?Y0$}-~ zoB&APZ4F%S!eV}iqPpF!!Xg5D?ks5iE6Ch>c1Sl2KV>g@ANR3o385iEO^zTlPR^I8sNTaY;hb)rP zOea|;r*>|@mcA|m*DJf7Mj>Qah!X2z2`Z9QrkijMGHad@VGi1-fmcSAf= zH5U-8({)$NULB!5B$NJZ)7QiBnVr6g4 zPK%mVotP*Vl+a)xZ5)z$@0A3hB?zI#4m(L|Pf!*#6+y4=z7xe%(1+_DK9KFO`PgkSW3SJ!^%j z=xk}`MUy6ryD?xlbTlh(&}w4hH8S|66JgO4*f=Q{>xw8=#~G(x|ILlb+DTjB2G-6E z@ZFejpvhK1k^5gDYSL<9MF+=61;Q@$#gFR0r?L6hQ7-n7Va(w~Tcs<#tu;KW5e~nz z4GHx~Xfxm?wX__|xCX5_SlzNQ=N^UEdNB?n&cJ<2Qd}X;hJ3OVxU8x!uoAx***++s zjboDJx1O6o)QPWV;Q}bY7Ff@Zr5K#0sc0xo(s2Foc5^jY62gZ#f1WNDkW5`ynGy>} z#iFRG!$%o7@ig*hOk>^dh@7zfhEx_8lfd9-aE5a!?kjfa-ynu?ej5gcjo5KS^-Q!B z>gx_!WRP@7OnN4(4B56gS{KMX_ZK#*RXK%lK_r8djdGGDK?9uO*)tUys^J?1r@xeE z5at?4!6=($p#L#&{teJct3%i7FXMck&_qt3=6 z%>#;!G{#<&<1ZlDG;+WrY-)glxnf$a_i}8>kuy|K0P?BR4}-irI442C~O_e zgI<-9%jDTxc~vnv#@_5;_+>?~GKCdxXVq{#MSjq>CF8@@v~i#p*YkAM$)hS8eCqCw zK?-j0vXdtUKk#ODUU?RA%?M-4G4!mO#rWK151gbR=@^hy-8x@XcqgJASR?4(0jzw9zgmK2ymusC=H0W5NT01q48w1KRtpA?V z=@220vk?I2@YKST7`V5MV|iMbnr>L6mvYhyJib37mUgL=>$O!l?GrPpHq_&Rd3gxw zb~5rVZT7%B%2ZXD8|o3})-To#_}AT(fkU+OL=mfd5n zx*9A@b=j|Id6{^~XW5yrkS?`m84?Z}2X&!hi4M3RQ#tZlQg-;EGxT=bwO-0WQGqjf zffLY|2v4=~`%QH{d<}xtL(9aIS6Y;vN&TJ=)ot4~@o`!f3QX0iI93Rk%5r|BuwBZl zRC*xfx+J`~;N@p$Dbdd&tRf$($-Hc9-$c8?K6F8!DutMm`s_TaffD7y%ZOEed7E3G zL^}ZCrGC+L?Q7weEFqZ&}Q|lu5*(VevX$)d-#${I}wg= zY&2Aeemg`Sv`TC8D%jR`f{dD!8Y z$w95unV*+nA|OyxmA6L(75z!|5Ue4;ES|VdDyzKAMyBK*ftSYYBQ~5~n#Bi}oK7e( z$Y=(?^xjYnNN=c#&ghJYuGZ$RY*K}UD3T{{cgO!c zOzfOsIvHOtBwu!);9f;;qR(42>)@HTGFsAF=#%XWoyBC-OCW>y>AW}gG5z!iv|~{B zx6d@k)r*-0w!Cz*ytuCpv#XQ{h#O)|6URks<)xq>T z4LXdUU0m{*z~#9qwvweK^RWe;vbBJHGB(1Uw9`WyuvKd8J}+#mpCFi~VhJ9WZ5hl~ z%|jax!b_ZJ8dcI;;jSmbl5gpTh}!wg?a2&IflmWuxY2GhxR~KCHSBoOF6O+DaE>Vr z!1D%fvFik&<|^F7Y}q_CT!Z^&#mgDK3+a68(#D6nQ6ILw=*&A6$JE{!pfCUTsHfz~|Sd7%p2 z$BIUR~NQ+yRxXipzXfVJ%9yl@{XY8ga<1yYu-gAoAcwOj-g(^TFJj zDMg0#T>OIoecz-z)zydP4~&@b_*h0p5yoH;BbaXJ?;3nMnS{Kop{5`91aQ?l&J`Zv zN8Wxrwdz%*o>S;>9Y+A{@XakkAIX;ad7kRs4!7Np_054gGY-Y>j~Y1-VoA zt0dM-zX6ItByWMOQsG@ZZ-i-ap|JZu*W1&K_^gb-gbDY4hGRN)d;F}j=`Yu8X5})< z2C^WeYeYa#4>cv(MLj0DWR(O}>=*eF6HDR5^k>(GNV{dL2sst->$jE&4;o^OXumPi zpFb8;rSWZ3Bu`w%6uX@ZZov;OXjyX#J`J0S=8UhVk(^OT`rX=P^6kZpi|U6wQ4V}v zO)~meTOp9eikDtM2SBhWT=g|~ELF^wvqs{83CS+Z$+#5YV40^!>gZSq z#($m-qOo;`&Rf&~jKk)w{%3@f5IaCJPE|HWu)Wx3v;r?Ol-ajOiqN{*hf$7@ntu>gyE!1zl^>c2FFdvwvGX#@*%du z$Gx_wgM`VY`IoUWH$ z`zz=i<-9|hm!NGAk~H0SFiz2K9ieF6h}_pmA&+B`d#Z!CP)E(xb#0wRgSQ>CK?hM&36gSGmm-+^GwsMyEb`U@vX)Ytwl_0-RzpG)%l=XMyfv8b2n}o0Mai! zpmz}Ry(>-4y1*{JF(YYjb`2u7n!M~qQS0d-m*qg)@BMKvT3+{&!i`5JORXu)N@sW1 zk$vy|!PQ@{Fq?_unr9BG_g1IlDhHgbJB`10Fj$==xnufCT+U4`i`TWkv*UGu-Z0)| ziQ6!yk{*m-ei)LUQl4ikhT+!$XaS*|&xp{^#x~v9jrqANNR^sROee&`^_m%YF?Hyu z$3j@S^EObH-rNIQe7PaIf(KwSO4r_Dlqb;d+!KCwptDH9@d8{n5?blXN=c4Ep-a61 ziT!%3^C7B33^DRkm>(J!u?$*$wAh_$?B2iI8fcv8&FCB!)i^-RYpc)!$ktOv9rWFn ze*BT6GID=y?q2#=4MpXN2kqaK7pJE*(z5TC^zCP-ydRgTP?*`=D@{dv zlyPr9TKelg3n6@rnVlYByyKki{%NPRLPaW^_6}9+9Ocwc7H9X!e$%;oK}Ie|%ICBG z?qtP7m`!RS`0u=sW*N8ymrLhxzEg%V%sfA%Pus|qZ4!^Jb%8a&6&Pj71q0GPe`h#- z9Q1f|AHL>caUsi~KcAD4wif2d7|c&9Hx@nL5@)moCf<_AOpz0SHiJTwTRtP8dM>9- zOPr`VQtqdT?1XxJq@tBVri_7UD&APpZX)IinSss_F}KJqwo1h`@P+iHCl|sN#U@>! zwIj5a?P_x>4$b{oymOa5beMVMzzf*gwY}(5W4mVig$@5&^8w>Je@e$E8@*ru==yjX zw2gT_dMxu^6?F%2rI9veR+L;Q$;Yan!C9AhI4^=5}5~Sumz7 z8&Xo4A`pF8MmZ2vJlJIna>e~X{puX&Vp4j8#$n6Lf6btbKAU9D(MzZ`Ki{;J`87Q0 zut!=xt>k{J6`3#Q>VJ?TY;F5exw+f+&)8qkOAo(-fHgQ&?h~J@ih()>#fY%^k7(ta@zK(SOvtmE*IQr&py*y zz8A4DeScqBe3h~Gw6(CvaSj5&UaUWTCYUDU9RnTkuWP|OuTu2%%o#VK>D$h-GWo7# z%@J7E0d_eQZ993`+lAPSVUO3Ha%-mK24P4wB9JhSCBG+fQ!XU)osPF?zeE_2_gu_2 zG%v9+wAWtFEy#se#Eh_Mbrdj2>*0})&9q-MtL}&;pH=CiBAtsI=yhUt$3{F$>1KmO z{a`gwW!9Q18N%kLZ(By@GBt%giHBVbYZ&fa*CAi<~!DJb;c z!3t3&Jb-dsgtSyS`eTi!?Fc# z;z{V2I)3j^?se3N<~0wR*Hh-e#FVkvfnEvibz5-A<1Jsf6qrxmAyZS@j zl12ko>us3a%)a{?I8Amizuhdg3IA|v-hhOin=J@9?v{bT9Jgvl%4;acsyW{X?GE=I@NL0{f538W+^EO_5EK;&bF0P`g6$J_$b3>uwJmAnIg=L8kpg6d>S)alN(cU zskub`L_Fa(1fWUJG$oSX=E{S@!P59fg!@T!FIaIqdaMTP7f2^7N!~sDX$~N+x>$S> zDJKhxns>g{9?lueBB8iN$j6PT0X`{XGlc#HZPc2+CZs~juiHF>+^c78JxcE$WK%+D zc28_sJN|bzLQcNl!937A%-IP0wQT|X(Z{Ga0S?5QKERL)7jB%k4DkVEo>!#k;|kF| zM`V5%NCp403#O)Hk62RR1_H**UVj+xPg!+JH+%mLE;P&3PKY>-BuXAex2RPa&wIiy z6A=bU`2FNDmOfdW@V=}trx?BpY#){wiwJ8Z)9efK2i9P;WVBxwiH3$#dqZ$??gLS04|sZB#XK2XOgz zcdWc{b3(iwkKZpppn=dZi{wOCc}B)-M@jpP#8V}tt-C}<1T1LhDDY0@g@>_&%WzBD zSIm~5u`|;5C02*0#ne+bda)3nhnu9R-=Yo8ZQG~6%pQyY*||vRe2EHY6tw)Uds*X; zBJ`wFIzuEx8|bOGkHeZbstngpBJl&I!%mWBUKQ5lIFUmZxeGQl z3yE?A2!*mGAN(J)^C(#RNIp1o_1wbcDd$WlT4x8G8rB~;g^ThDU8vnylJX$lqM7?2 zM9smP>tNxD#U}Tg!p0)cX$*Y+O_QNlFon=6RV%$h{yW^F9~2lb2;4?AbC7s1Kj0&Lp3g61jkS$=r`LnU)bT0h9bQY5onT60rYX z2LA!Xp%>D-c$x)oq=c38YU3)!J4Te{6lNKOLGfr9$>h z;ZJrvZFpU-OtlD=z1=&SctnLM%xJmOhg z^s0^)^z{0yJMK}kuMPmv&4SN%_X^+{LPMSQPl{&}!e(f|_wSPNsc(<_lMtPYS zS5F@!N2+3Bi)2xkm$By4@wtiu2$cq#P5_ zWw-G-V_u=vt4r32r!_&wm3O?mc8cv;7Xm_Lx|EByNUuL`U+gS5^KZOZeX~QV_nz8b zK=4{;MIHtHK4K&JX!2u*kMptguXeDX#iD-QPP2N~rT;=S<2yWzfu_!$epX`#DD8E~ zR2jG%bcZr5H2wtiHV_h{ttn&2fcZE0DU62(zV~PWK?MG^(yjWJ~o)N4$G_G4|~~V%Ps$+frKYvGG`S88wk9BU;J*(e~aT$Y0|& zNJIckq0X}I6@+M^<)1DMjLzihrbUyeMw1#zX|CKnWk6*C8yNZcY!HKZ za{B}di^#Hm25)yG4t#4X%j7#iiDl7QwuBK5)=OxRwulMx!S)sFFFGx}yWpol(3nJ7 zY&3#49ex3T5J^qSEPC$NRr{m+vnM0Z&513MkrwOff;BH^d~MrQLRx|X!o|HW$JDaO zvBVPU)83VgA>z$O%?T_IS6EEK(hxiJXC7=vZ8u%XEOsa`mC5~JBp(F>1=g?Qvxwg- zrU+CEjzt=ac^qc_7%)STj27zp&5Q`ZX&$I6u3<%hdle!Xr@zYA%Uq!A?<#s)(*?rv z*hBC-HS{)NgU)tp(PVoG#(O@M*5V}X+B)TUr9b@dgOO@R>wo6ir=o?j4>I_hIPRbo+^wKuu=L(I=q0V4#X!GgWV!Pz8*+^} zj97&OqW2!a4J)BPa{0oF<$DGXF?ySRFwCOdi{}&V%!ZY)V|KJ*J@rQpx&s0XZiagQ zI6Ae7be!Ci8ozbBbRm{DG(t!K`hOH^?#*>)<(g1Y$j95egqL!8y7p2$SR0nEv3#<) z4{-*?qYYv6NNDLyE-UX}xmtY`SXacOKEJ9k!q%}G}1@u3w_ zs=;04{w9|_{1`hIs2VZcXr3ZBBA?O%@a%@EHoX5x;HB>evS`vyy>8*#lpXgqH~-yu z7oihTi9Vy?@GrdJ)9Gf8s;DPdBGe-I{(CBKt%43!2Cz*gBMPJauB4*sUrN02o%&6r z(4Y2ptbsy1DO*%lW`!hwYk6}x$+c@s=ZcSiP=&*wGVJoD(@-*)e)!sN9YqBB06Gvs z_Pu@a!(H@Q!m{|w=O&yZ5L$Kl_`xOXP+qaNa_^2_pn+YB<=AIsITXmR30Pf(*_D09Y=c-+U!^zP(2q&M|{er{4Hm)i)qq{KE``_l(9Y z!F>>!v6J9H6Y!FEHRo;WVvw!+M~IeiGeDVevmX?-b>zI6DCavYUTdl)a!WIBJNlHP ze_oLvSqgQ^ZiuZXHN0pSY{oY()*`wo0-tg4W@ID}3rh44bh}@OiOfp=My{{S z0>(TD=kelUEjP89oq~06cLNF>OE1o!^k4FOOU`%)dt5%Dd=+<)0AD#6M{egbmTJYN-e{2&lx&@8bYWPb|3 zw~-UwXVwt)knQhkXz$|IxQNrsUY}ugLmk!v7L(dmUPr(xxj{ zELJ`=DMMY&tnhFHk1QfYpS;uMO)a*@>Q9e@mvkOXPCBchfY$Qe*+I02ABc$})(7Z1 z%+%*AgA&VICZ6uD5pNmX@1J-#tb@UZkh7*E_q~S{+-RScJrzb%Bx5gOBM?1>jf7fr!zJ zOx;S-rIV#b0+zTshIo*ho&y{_I3PsWF+uDFiHWqPffdUSX?{|=EXYIu(M;JY(36@) z%iuk2kgU@;u=?vRcG#pruzV3CKqs$1N*xW|GPecKr<-2clP77_?}%KY=OiQU$pwZ& z_kakgNX--fr-cUja|B=Bxuic!&WjTVV6vic%g2K7omMOt#AE&}*8mc}V)z^q$EXv| zkh)(xh%8$xBiSh3D3YKrqylUcCQ_*G<9h!jeo(r+afW~U^wY>|e_}qgQu&ui8faCx zBJ3;ke4DA|KtDqrUN%^bJwas>6K~6cCtoU>NF?-?qDL_Za@&C+`7F{YbCuC)SxPVU zu!;J77{Iz$Il4=}ooCK^n2i+6UqmSiJaRRLoS8aFMkaH_#Kz1C5>tSSE#||Pz%;=TpDK(i2{GDgkQU{Y==$7M$CDmV%-N79G zMf_-(-6~^&6~YuAezFz*Lb?>?KM&u|M|*!lR{^Yg^sZGS0Rf=Fz)&K$LNZZBJ6Ut& z5(y)@vB#iS_E*>9*fK-+Q1urucYm9E&4z<)YZ8O*BhU3fj!@O7g7C}vg+yRSwCOxP zQS~Ru819d6;bAGa(_qMtMUXX74SY^z?c0H5IO1RYEez}_EK1ndUB9+bvQ7ta2+`zP zl>iQ8lLUp}$)iyt;H4vu7PqZ@lE4i&5=kdApR}TU`GTy)e zp{J*|LF z2|OT`80q%YUj^yY>apT@95rvo&#eg=7>KC&gqMRC>o?s4;=?Pt3EG@ORryvUvK|Qm z0Q?~id|pLxGdAFf^b_RgS)c<#`VFb<{w2M4oqk%Zi$0zzU$orBeTG_9JR;k`bpX84 zcf}#^aagNlzAYPH?yKc1jAxHSTUfx6GTo=&iIjbsxmY4D4~f=m?q=|y|UF?h0riWEvlOQF>2jyc|yWNa{?*GNDE^WzLleA%xn&@xekL;@4Oyd9eUUQUb{ z2YWjxa4b&rg98e@%LWgQApp{tEs{nJzbq~O)OMR(Ys}?TJHW72>&bNeOQ7*1V)kh( z%i$})`G>;aE})Kn&8+i~x8!2*ZS2ndOys-%;BDwzT9l+|Nk-8|R^o|$8y_LRTl*7M>0btq}`2I@g4nM2h4ma(+N zv9CMAH^1d&^fm}NF@;3rve_OZK3-r`yq>J;AAO43?l>BFqT%k5NyhC^ch(GDaC@*R zzk1%yUb%FIwiJY_g8)!ezfTai?Tr?2T2vZku!^zEyd@}$wNpoc%FxT}|Kw+Qptjt{ zs|d*@vUf&)shmFVioMGz0ktR{G8QftNit#9H6q(=nzhV~-q>lxUy8~Djm%-vB!yp; zuhhuO4?XG{ylqm};0EnFf@(*jmyPs|LKgTJ+FDq_&oLD}o&x%u*{p{9oZdFJG7h9~xV_y5}R|QqYEo9xokVBW4 z)?d68+H?9@gGWGw|*%EyrV4)L3e*K zl|`!Cmv#K0>Ix{-2dzy0qpR10c|%0;2}F4jWB<(GGXjY5`y9K+t>6u~sk=virTj?r z1;fi4dcR@>z!>kG>SJ_?eJONK!(T0np%ZA}g$(@4KgoGyjl2}sOP-qtqp3p>Gb_oc z5k3H=-~*~2Df}XO;)jH%>?C_|*qM6Uw^k-pYu*HotD+{_)_mc4cw|&k*;!HJ*oD zaIkXsFa&_bn?Do7^2)e-sQEh!R89Rg4s=02S{X=d63YIFVnXer{+xm1+&nK6bQ_dI z&4CAvh+kFkf!X^5Czv_@Q2&}#(g~FEcDM1Ae1uizNsV8u(rZNhgQ8*aAZZ`3h)t|y zI42X)1oxtEzcMN7EQ1wtwVA1@{{^!3J>=Mvts~sX)?W8K1qtoj0Y_b-BU_%@C zv_S>kFzazH*nc|TKUWd}Z2^rw zqYY?DdG%m}y~SybKANr6k#e)gguNqb#SS|1)sp~k$w2Zhx(FaG`SOMQH6bdpPjQ9F zKdKJ75_T)9Oj-+l_C%=U3%gLAV@=z9r_hsbZ%qmJ>nn=NM%+dpSCFUU2W32;#q29? zPCvM{MAmZ$)`#zm!>oaK!Ifg#z{@ITP&z=tBU=Q{p5n{>&#cl5K7#jQ&l{|_)5F zZfCyhs|EyFsDqg!DXJsv#Gw||fvHN5xiV>Wj;CI!$_WU1p~70FLLteAt2X3?1QMW2 zPSGFo!MvPcvM|G&uIHX{A6B`bL4cHkCsFlUgjn#yashOyScL;GJr-Q^h|~q``=_75 z!3UT@iIGk~m&rgCnL8)?M|`uf7whdOs>Iy_yg@*`*^d*sa<~-w+5z;~cBax^REI{8(ypqjBT z8&Y#ndJFP|o%pqBTIMo%UmGzznE+*id(&~7UN9&8K5ySKz3!h8!HgLOz;lfAT!!Lx zN9p^j#kf?ef+a6Sfyd2N@b5hdZ(dhx<4}&vu%p&)8aWvzrZ1&>!uT5|X&6!%mxoj= z+MCeT&nYIpK*J*d*4~GvbcrjeCCXMwl&#ts8%%+&^7{ zE4-x5sA4uSb!l@W%n=@81D{3>&aN+O=gXb&0!OuLjUc=<%i~*Y+mA_Et8B)sXja~A z<9%-P*T~#CZn=d@clxXdhdbZ(+=qee@=2{b^JXCYAsqTaVJz>S0{Uj41;GB(6nff{6+D>%mlL@uS^|X}_+Q^Q-j?~b(c}$ddgA2kFD%riuWHz{ z|J0=fR7rbQeskAW$@C3GDgbO#jIRzql2{s21;0ihS|L_V3ZU0Q*g+vJtps0<*u_P) zWD`he!+>;IXw{*w?lyGsE1csa2I3W5JGN_nDi5y~swssYB8N!dNEP_V%~S zH@>N>7vAi)?2Wf)R2BLs@KpGdqBJ^To)FP8&NgBxS6NN{fkCWQ^3Bh{<#_n3;?x zwRr&Oq!U1WEaxfR-T({pGVX+|89NIa{E=+&K4jp98<{wq5`0#f4;?!@rKZ7#5tdAr zXfq@&PoX*(-{g>m^Lzm(N>?6ZIZR3*)4m&{NlLE{`-DpR?fOkZtJkb?~lZLu=VfX-$}_eGK$fqfEJw$ z{i!2AsBVzGy%`K3l^Nw6yYWS&I`amPQeAGd0-1?P&3b$s4F1j)=Zd%^50Ltru7=X6 z6x=F|I(tredB8h5mf?`Jz8{#Xy|6^u$ch-3P?dO#jnsG3!`$*(Cd|X;My14z4bKV{ zWk02h#-hf=JmBJ+I}%flKhoBXolzD6rCT%?z*{noY@M6>xM!UXFVg0J|IGR=wCV08 zSR>FMg7G~Aiq|H&+NM&2tAh0OUfoZQ4iGrolnJ0tGyr7gr1ypaZ3U$&1tFkw*3Vu*x*Z%%|>)VC; zJxlZ$MXEOG1L%-B*6^ZHCIMKqAA@YD(?x15CEV~syP7ag!tElB#F05La%-ZC+5Uqqy&jodJ4NeGrs}DAmeg{pzhrUxF;2BGH7u$Ju!?)KU3B4MmKB@LyMnkgNZtEro63VTjJny6bPzMn#9I`$D(?|VSv47&qb()o z8Gwg(1LD0a4qnblufJWQ5V(p%#^QDyt+++X`&eKM+xh!RxZc?Z>dOlPp?%f`{qvG_ zp0-qXqkOqR#Arcu&{wGA8MAUXIq$bKN~Fs+0vz|G^EL!ncaSldl=qX&&ZQ<7x7-_5 zsG10F%+KREvb_Cf0^VQAIO%sdXG@$_xd30y3oTKDmcX6r{%dy7fh*A+3wj2dWSAJ# z-c33WNjAZeLl3UJ8JZQHLu$`PNwzlC-I;Ek#p!1Mjj{LEa{?QN7ZBE6`Ex`7Drt)> z)1Sph32Wi{+394>7|AVOtJu@Jdb`iu)pqs$jN2%TC$%2W%ECvK3Ihv#Bmkb!)#lk|BtVCimrrPqJ?9lW81cE+w9o3cZ?m| zw(WGxj_r;+NyqGT_@}>f?m6e)hkrk#m4>}sXb0RNcxRVrGFT{GoQfbh$@=s7GpJln@KIw7jb4DfBg5gHxG)9 z(?BLnk1s41>}nVbv^~BOABpSyq0N9=3!c03HPheMUjIqs$p2d&{+UZ(HX??A{cA++ z35MtM@1tEa6ff96(`sXQ1~QY+M`8u(0WFAzfN3>74fK!gYZR}Zv=rJUQ3jN%KLv=& zf;I!%2yGHHEf9^Wdj6sC{@2fgJw419D?;V|G}DH=Sn#4);UopdZX?+NXB z%}i4$!<5A4Iw!jJ&HTyT`FyQ&cl##v)ep9S2p&q8c}0335nL*f^~+#K5SKGcq@16} zIzU)}^68u*@7r{<45pSB&j{_JT>wX_$& zu<2b%Hk?%?s1_Qxj)w0Cfm6yz$ET9bxCPBg--Nfjjt(_^ehb642VkV?A&Q~MDPK!7-Zuhy}R%p_MAe5G4$QcjB#~)RSxjLrmykF zCSPJ!ZrE_~yx__}<-FYZT3X|2sb44KsnWIdl$|k^bMI0`qhg!?r2~WJPCozMUqEK3 z@6S1Yiha^V$aOInEx>M+v^&d0BKw=c{ZlU`8SjOU%xqQIyC5hj`u><)Gcnj}8i`Yk zCR>FFL#ze;I}cc6-ePuzw=m!g#E{rKLovuvIl^&1^vWqmGuLKToX-$q>70BIYN-az zIP5oS7yJ8{*x=FdHA@(DTIIO*?72B`d`-i1GPcy*axPAX88t?OJTwJMGWs4 zzl21n+oFtKQGWTtS#=Oa|F$c-5&SfxEqF)Ex?$tJo<#a(K=I4K3#T4H-EI&`EPH9f z?x->RL~qD@R`_j^y*}uq9|uD$7^%|QI?mJDthwRmn(pCR&qyPPDH-OD_~IyIcpvJ} zPUQR*(0JanW0<)HXH*sz&ekwXq1$3nNpDGjBm2xx`~Y_Pr&Wws##Cvb7_zfhNID*z zarOjJ%B#@CYdyiLvnU&2{zsg(W5|6mW!;sR@{Tjww(}iXDB=FNh<{$Evb@#$rhn5ymjKMV zE=r=mbDl{Kqa<_$u`^gBxTB8!A)%s4*hhzhrJ>=3u%oKo1PAssN`xII9)=oz+Q-hw zI|?ul#UjaoLx1mNdjo&Qy8l5FLAY_Mw@w-EMR|kpXyv)H>9`Z%xU)(1_PEo03}SJu zfaSwNfv3PZ!AzQQIS67gm@zWe!~?(^<;h7e=2`b69f3CvEopg3~d(O|Apaw%7y~aul=UEZ}!O6xnR7YI|); zp=4Oxp3N7d73j3hxIMhL?+lvK+XehX5Gp1iJh14u??T6zzwH>BN^Covy93}C-g~UG z-zCDAy*8`ip>K)w%wZU~CH8&z?Kk!%AAWQ>xjuJEWJzzZxY=rWD$uN^lAJOP*>#(l zxE>;SR&%yGcQV}JD3)CP%I>l_^^loNHL#Ss6zM3|`O4KheUfUV2BZ|@K2tXkqt8qd z-@)@$kYPI<)YxpmpKX!D*#o5K76GfZRc{CtD^G@CF{tM2O}-4#3XX9=PSW-!47!(K zn5l2|D`3!dl>|82A2Uj{3e+5+ehAj%zi2{z?blfr!rG1f}x5o#>%$u z;@rCEb<~-8O#1tn%f%05*{av6K(OAjC94;19G^$+ZT&tubZRr-D!``3W?h)P?II_^ zBcXBl9%j^UNn8vwVOuVe38$a^TLoXFkO${3Xn3O)_y@a;VP>Oq6&46HLmb6y?1aoz z>`pYQgS1)b28(K_c#9)KRT*382^+nlw;vrBuSsb9t=t=4{HWQPvW&5LhKqvrmv@HQ z9J%6D&@1I9h%LkNYXMLm6o14pfKc%|G67xddsU9QS=Pz^dhP6!G^y0Htw&v?3QbNE zFB5H;%C+kY231&5UTT@+&$mdWx|nf|8seu0D4-IsJ>^n{K7l@6zeSW%0}UAo?h*FI zL=opta3@H9y`Ua!8bzsXo=D3(gir;DoHz%@scy3$A@8l`go-#L_(Wzy4z>!~*`K%v^~IQ!Od|7# zAcW$11S2Q{KmqMua6E!R{Q1KKgDEg-6jQ%L{{%4$q5dQ>CY;}zW&a)gGoZXv$m&ul z>m~Z<`_H@j!BiJpmpM8j$R$ z^tm;2G~d2nKsR2Mu1Nb8T$4V+WyC5b$cj2td670)bg=uPv479m^0E!0H<^hYg{Aiz z{HZXedF-e{)Z|FQ(zBec&)dV{@wom<2!wyo4avVFn@VkhEW+1@tIqHDi}H9e5SE}U zPKE&4jo*yA?MXx~lw=1}ZfnyG5)d+f&5ttG&5s~4uDl^fI%tkk1JgUvkBblz1S15a z*Y;$r*YYY%?=*2*j~ISlbHH=UBi6;+I9f3Mk_|y<#hd4~ylxx4%ivh8T3Y7l>KO|i z$2+E|aLKLyTW9fxK}p)%YMn2KymKXZZJ$6ZPZmil^xIdVl%gN&S*`}bG610r67kl; zU#jj4e?pNgbw0wUG`VIb&UC;r$`ubg?bY;4HLGd%!@Pm1yu^|L%U2L>+HUIAl)jUB z8%Hl&!YJv8Da z1pD(4%ZYKeJbCCrW1qI*e1Kb)`%S*VfWGsHIJZX{%Zd%FydS0-P;u=7Z1Z!Hr+*HG zYo*}amsxzQ-Io<~-EY;nCXg0RStOWso69_6mMk|)olas5-bT2^P}HK^=O7)|X<;8Yw3D6S{uOcqtSssP!m#1%Z2ZQzXX z=xgxQ7(`}ljzxNlJZWqhxncm099@dsy5rpH^esw|AVmt;f(k4FN0Gc|6fJoXTvP?A zq{(G;Ml~Jr8Cag1ecbR;w~-InVzFcYiPrG6tHbzv?_PE@7yY9Op5yN%{#&WrTxl_2 zZ)KC|N&G#levu9fJMH5?CJq0+r2i>^pY^ZklhyH9$mKQv% z#PSjXQnfC$&^53>4Azb=<3dP-f-8+{JK#^kacQD4Nyk@$e}VzkInU!JPneshXMoz) zSL?ny>!(a7InPBo5Z@Ok%LVU=mKzmkb94PeRGeos`F6r!YVX#o6avJeQqA#GT=OiZ#AVg3a!_PoFgLYwCP+&8UBpi9A4(*RSuOYZ>F{|4O zYq{`nNXbnrU%Z8c+gqD-k~kZ7R*2uVl}SmZ*u{`!EU|{095bXAwm-MjH!NY0p(!K* zK*-SO&J)J1^v5rh${9G)qY2_VvwUQ;2sJg8N^80Vr{~Ux5L)-fAwnGc<*Nt1R9Uj} zb2V#O%yr9iM?*7J?_oz4hBvnF*V%A!8kh<*MY{xuC_2jmR9cIqmR)5H+asoPa`6`2 zmRlc7d=;}i!vxke7sf(W^clKvb>ZOwEYy(5IEvu{MUA@QwwfEUQ)_;j6?I5@EH)A) zFVVc5JK$BM%D+j^Y0#h`3Z29ud{PGx6w)Gs5DZc7-NHmzwL0%-n>>&~?Z*-ixLduA>d7zL+x; zF-4Hno!LmvztqYHq^Y`JtIAGqWLbaNr!#dYuRpb3Hd*6z&}W|FDV~O}=frW`XJF}J zI3Vbu)Ax@*@W&zA#T}qH5c3-b+%(20UDJrk4TuB1Y$SJ$q@dv(aq{y(wZEDJz%DG*eI>jx zOms91FEKf+DY7*XX}7j*X{EU)Oz2Jo*lS9V*DP4N6pUwO9Uq&FiX!6=3SHHd1?~ps zO{2VAI+hk`6=fE9H{~;2-GXR4ftqSE2iC$Jp+RC*IBhm6CB;;3BZ5-3XBD)f66Vsi z@{_p>F)cjlu1?%T4T1)MarSmW>^97A_Z5VyE4sB)0@&p|L9BwDZf)HW#*=U4Pms%? z;RL^BR?vboXp4WG7RhX|*!kxb3h}!;&fETKI4N^1CMeJqLBy4aFZsjx+*Bo!v62XM zhkj)*Pm?`x*f=c|kh~HeV_ieFao|z+wMscb+}|fBlV$4^j-ZkOWIw4mdYH$cl@{1v zYKz)S9h3UC?_phaeM%^UVy}7Z1XUT)Owlgj2D0I%C4DnH8jW6qT?9zTHz32o3bTSk_bz5G6P+0JUmmEg!!1YVPF2 zv^@7p4be6&*!b52`j(l-Um=^|);$}HZ;@~*(;FHaol)WwoE4(23ZA3~Mi#M0MV@0W!lDGbT9Njk@|;CoGAa}c!>ExG^t7xz5-MIvK}wmoMPoWL z*!aZpIfn1ZZhj~BT3UNj1$&N6uYdgtz+%9cLd6IF0QpD)u*VgfJhYQVUMgdKT2GD( zAA@l+6&+c?cJoTj8$lW#iNkb+OYhHkP{eXuffSmz1Vq@o_nAzZU9b`=C_ZC0aPXXd z?Qc6|ZQIcwH8p2An{Xpm+>oR$jOTM$3T~Zu-XifqBB5f#c|$DtDz*#w+o$|D8H0cz z{nrbn@UQG2S|42p9O>^7(So}L@e`{gLZ>HgLH#Rbx4*5C@_QHlCL9irBggxPHX=Q)!(lkLZR znM6W;0nkpEiv-jKvxG0-On=ymr`&dyp*o}O|e(o;&q zXx>hflF*%&9~du9h_ZyJVD5p1&N6Gq;V!Er!MQ|x$oASn0d!pmT}ReHMu=_4%$#ZR zx4CqsZ;-__-u#%xv9`Z5m3Ql4@?9K^wnQ+{5De-_G;J7f_MepyRwph1ZtkdAmXo84#b*>CH6UJ@*Q8#{g_K?ZTNGQA{EGZIe-W@y zWhDhP>t(%$t=SI{)iVdbu$9>b?M^=AN7{o)uxcX*e52-;Wyps7s>YM8 zQtJot60?MLZsZlRl-1ft@@55|Y2raZuwp0`*RnrTw%&$Rz9#d`>!D)vBiKqw7#%I< zs$$RgxtSy8Kn!f$x@`l(jXuf}k~fa(NvS}bRRZ}y2A`5aYA+0eC*Ysa)_+6b-yjGL z1m+V3Eqq4D9N|T3!Dn^co<+{LZZNdX5M@XKQ5&t zVXfSJFb=9f7q2d26LzK3UVXOPsit7+JXI-5+jdBdkJCI>-<(2 z&C0@j>=`fbTxBsl4S{8186UgprItc;k6T!(q7b9ILp~0oC%XLBWGb5T?lJOroYz%U zyHXD_;K(@)Wy5n%HNj86yH)o#4Ew-2h7Iq7l(guOOkIhCh1%i_?lY;Y}HDi{Q*WA6L z=P}IA*6DfG`8&t=(sRateCO?V^BsuKwGylfIfcB)!G1;P1vg+cY-Yp_1WHxEL}F_e zRW0;rEHD(r&cRZA>WPK1aNYhxb=Vn}Ap_$kkO&P`_og5e>Q#1Fj5L73m?0^Ef9iou zpa1)a>1PK~5IIKCasVp?CBaX#&jo5HDLaVy${Dz`Cl1m#rl!w7jU+HM(j34~weO9> z1d|YfD8>QpJ_2Y`mlO8Sx3Kc)3*2xT3kBIRqAi58BAiEXr)Trm=5W+PIwOaTE7=DM z7G~hK3lz@gEsRT=iOWeKF1J}0m)Vrw<1EKeBGB=gX|pVstO%2e5U_t`!jY|zN{Mms zZ9j=Bb*Zo{yLkj@nTK*s06-its9*FR)@( zxvV)Vn8)GI=U%it$zYQ&Zo=o31nC!NM_5a$YF(s|?5zcsIam_&)rbi|Kzhll zxF65@9-0nWQ+wVJv*`@IO{A5@o9%ghSZ=lz{%yJ9VLY-?Qpr2^Z9Ro)`3Uxd-bjqm ze-yylC78E>@V!W2?7$|A!+VL2)o{cXtZrhG-9RfDC}>29yl${+DUMJ3z#PQ=CY2pdq@c7dS4lt4p=8^K%#Lk&0 z{p+5dTkhfQr;%{Ln1Pg@%RcC$x&;rk&H=z4*<9FzN9R`xTHMSrX1TLhd>9j0HW(`$AD+{{zTg48e$dPq`Zh6X`QDM4HIM%! zILTvs!XVDq%vrfWU%R&#muTW>Rc|*Tq`HOun5Iv4Ut~k`&5Ux*rjm2ge7cJa&lxb5 z2;Nef>ZZ$4v96(5Eq++4>NxK;dsjO8a*;w;!K+i+iX zxyQ2)LAL)>=j!6sTEQY}>j&Sfdk14%W0TT7Uh$3OXycjU+F8%>X_#&9!n<()<;y46m@ikv9=1O?EPv^n0Z zAAn@f330Ok4VoyBpTk>knPu4)p8a=3 z(b`{BpE_1f^&6$Tv8MgWrja&lzkF?-hfqf>NEiL*#LOz|sRTfu|Cag$5&ef>pW@%A z9O_A`v?J4oMfG;ApMXVH8MivxvO9!K@3fPi1TGO@*xbkOZYLD`wRLtq<}SPdR7NLtXVxg$QPj@;3)t}y&4 z(|2pMMo2<9Jcya*qyV&j=5LIA$MYm}=iv7!C>i_Os}f83lAcqe?g7x3l+5@W$%asm#Fq~)!eUFuaYXLIdG}2V z1S71HJ{+$KwoC&|`@0lN(H+OA8{%7=??lofOi#aghLOF8+mrAf$n$B)FAD^(oCq%X zcfNKnuKv>dpbY2L^B3qo+xUfF!ngSgzYH%rvxy(RwH7c_`W&h>4o-Kgkrd%Zg5jG`a@n7jCaIuW(PTWPSi(jkAm2;yz#Qd>s(8B3Iz#eGGM&KtRGnvP(i-Gm zVOULPxwc)V8i?&mE0hWXRycdq!?S9VyRE#y|Mb|V$bO?550y+ZWx%!;;8N1`h{-~- z^-Whzg#>V=w#(6U%K!QnyTawRvS7JU<|$ghgPD&BQPWCeI74Nc+8#8`YqJ=A3X^2CrhwSz9N} z_z8Y@eN)$W|3-6m+8J%?qY`(d(@S;jC18qo-CqaDIz~`3V*Y8X+a-|x-KWla_F?pF zq#ajo6WTm^B{|jER)^)Ei^Ktp4 zS1bswJy9o~zJp=8uTg0@7Q!Z;drr}zL}ehxiH(_5K`fH@OpE(uA0Jr?5hl)_KcYUf z%j6v3RMEax;!Ve7(e*X4KTW#_GoM5toeZJ-^8=|kaw$Y`@KEQzNUDQmp4*homdDNm z`rF>_k{EB%2_l~6DK7kKzaiWZ!DUgZ{bzv6rQk4TZ}}cKq1LrOqENp(Bp3VEFfs?e z36s#EJ7WBPLw{Yjn>qdD=avmn&pLhL#|QwHtK>Wzfu7N(k0SJBSt{3wym}w{qd3Rs zhN!)t<-=mEzpzH~sCgnsV@8S+?_&IvTKG0ri5}XeWo$+`az@x<{#fD$#38fi3^P$% za=dJB+xCkF4>JQ479M(x->Wkh8R6S3!r5OD$6 zz@lQKV*iZ4oCeoPO5d)SsT&IAI1LL%Y-&a)(i+S1YDA(-cc?Htph4I?6_hw}BM^ z>FGHow7s7?9Vs$ChD2JbxpKeFXAIDyU$pA9m@bOsI0bCoj80-gdivF75xF~>;$u(* zZ9jn4`?H}pv+Gm)q#ud}q|qJuU=OgRHIi6>|j zJrq78gE%W9pUC(;X-2yKQQ5>H|9q1}+&%ZMCI zs0#6A4=BA;jZzoURPG~hD<$JF$bzeeQyb@7kR<<*xe<`XqlgLdP$+(vKTn=E{~EQ# zH{iMi4sJCI3!?+|&PdTKh()=($?SDP>N&quHbEd1VA*3lVE_K+7tCwJ*A;gF4PO>SLYK z1}f;y3VE%rg02r89j9Uv5^aL2m#@pc4hiz!hMmN{09RX3a)& zj2e(BFmu$w)ZUzPZ7!;$*dA&fg}viZ{j0~I$qACBM|RIp2W^UlNLag+-@9#}Q-g$D znsUT|jKRfaHEWthY~=x3_;bo&Rs8{4Z^50yO;{lZ+VPU~F&xKSk6&PQeCDs{*2FyJ~cj9c!!Bisv`5qUGhbw3Be50y!_NyLL!iUJ59s(NSZ-$47p>s%3B^ieh_ zpa-`*)_zISRhOzy?$XheC#yxxv2567FbkP-YvcjH)9if_Oj)-AptRY{H-rv$wx61% z&giTNs>Up`^GY>R>eFv`bG>S>KY4Sf`%wCfHH*9CBZO}D3FX;Z;ywP-uz7Gyb#m=_ ze{C{H>w*x4r)i>BC5~1o5&d2M^W<$MU@H=zrQPWd;$NKc-~ai4amL>t#uD}|mQm#I z1RLUisA0B$Q^RTopYv;UfyEWZS~>=zeXas@tM+lu>qr(Bc=D`g3`FE!`*t#drd(?a zOS)H7v^NaOf>G#$W})w1p4L_@K>Oxqw&OhCpHttOg#i99NJikXVDwuG;l2{`MwO9} z>h$zjusV$mHib5YqsqTtjE{0L!>>juR{%N2jWj8Aoh(ugb2p`jg=e|FN;rB5-is(I zr2`GThc<+j4YRGAX^Gto#px`oQh7jy6^)OD*%qE4UdfU>h`WyFivYc%;FG|_*X>jNWQyygkyu|M+8@CC zuGDZm%)V4fkH3{~xv6i8Zja(#ZZNS5ykwgkUt!s=Akokj=%MCJZu{KL zmVKM6qAOZtG^@5|dL`afQD3)WZ;;-Zg}{azPnndF0rgSyKdqizV2eVPm}I|8kV#BZ z69RQzQA}{|C4Px2Kq#}CLPPP2x5hn&A)~c$!jQI%A}_+4Kn7)Z3pKi!s+9MDe{inle=gbod;hTXspV-tWfvTuPVXY0QEBbSK)ka^%2|zw)?VdC`0YXH|XcoKc1WANo24xT%ZFSM7NFXePBXke3vQF(z z)g+e5Y}^MlV)>^2ScJFMEf%GdtErUPY-#24?Rz_^f4{mC^f&2aW1DoBxvm_~aqYd^ z?)&X`>~zqI^%blE>jS0$6-Z>5gJ6VDo38*f2LPL6nZPOsyKlrn0JQ)C)dn?l<_#Qk z4xcTPRSENaz7_|`zF{PSg5aMhSNd}u8CVYP#r1=17iZ?-hiVV<)p0&D%u_)iGUTCn{`5b=$j9Irqgeh=NyuM+ zP658&>{a`{;+6w)$8Ku^9hp-qd??BA5av6Iq&U`_%8GR4REmKX6XBGVDl-LEnO0L2 z!}dX%P+#d2&mEQA+*tS<$nD0<>hjWJt#sLnb8jnBG-?&6<-bFrY%W@Fa3!aa$f11; z573p_tg7PeMr!|54I)!*ba#g*8~B48_ydM+w43q^`2$pW8F&OoFcwK20ICyKNvRxW z;o%ya_A^<2BoXFxmF1)6lN_5q388dw8M3V6E73-&XH64iSMuZ$zP``n0+^=MXLB5j zN$j@5s?9E+KLHU%o5DQ$8L^mihezPRVqy8jHs3ywlBx-ftZAMpwn7|s^dY4L2_PLl zxXL(h8cRbtl$*s)f;EmmRM>!4POD-bDGg~;yKMv$heSA<;h?((@!KSpm$WSRw$p|G zn>fO1U9I#1?mpg1p=3p(nBRfPnoITC*G)pqJtXdaml^L)41i68uE@pxPDQLVP zjV%JlWE+CFda9aci(cF4J^M|{Ex<3MjbUM5?R>CI)4am+kQ~-IwkjEoD{a`MQ@YK? z+_;?bZlqTGLRT-8%1mkxfsqN;l1@%n)uu-;tEE-txFCRZH+xAI!Y`T^LX5c5GB(9e zdvBB!k2e@`DTmUsoh!fDIYsSWC}B3?xFuOlMxm}m|+qv@jD zhx>UNS^K>%5+4)Bv3gQU2JMElj@3>#(+fptm_xQKGTljKINr%WtVInzc>Wm#>$)jY z>)MoXNlE+_z5ZuN8;4u23}7O8%M?@y?Z%bRpgMc%i+frEd|%oeno{_nBeq9vJB`zg!v3o% z6}Cqm+P6U#>j}#mQ$R^FYnzO0xt7}y@cNhDkVjbx8gyhxpxtJ{nY!{+U6f385^;hM z>`TCg&ULTe=Kfnso8K}KEis!qaeTciUJCVwwSUez5lp@PhnTHQYZxPJep6)UcZ}a= zgwexfF)|jx3sM>j=Y^#D%Bzp=z~dGou0Q5UAp>G+inpX^Q~;(Ol3Ome)>fxIf^Pkh zlFTf*ATuYYr5#LCSaRSxO}rVK0dC)S=|-;UbcAj=ZvB38o)+`ciO^(ln@?xuZ2Yx8 zuW;YWmL_8-vUe{fDc@IVWPrYAU3wUtUhhP&ZTdu7=Hw>-P###^sLj`_(`3i#wr+y1 z?>CZb#|q&tn}E8iR~&SvVbw2=DG#h8{ERpoCa41Oi$I5(8qUJOn*h@;8NHymK{8%WaGwSMZW&I) zoFiX`^b8d$Lt6J1XYx=wI5 zl^oGkIARO{N}~M{V=krxtQIZDn*_Sez6yNYQN{@54N zAGRgU+}e|-QRL791KTt75nTstFh{@7uzusEJ=&|OIM|?ZElZa@qLU>J{ov_l=h+g9 zD4I-Jv+#*B?B^VWQF`GyFK`S6wq&asL=cOdF@pW#3_fNIHf0OCAr@hgG&-kZMpAQT zr=0=lBne6pXBOrj!y;k9=rAcK9#>A)E5kaPeEArLeT5f$;`$Nvv8&P(PV#fav<9E7 z28zZ5dAi*+WtnU!tFQ^0WgsrW1L+$XL|d~Pb`k`45Wn)oGYj4TDwI>s>qYquN)|@4 zlGMZoEKr&({&Sy}ccu1&7vHpSThJG09$rE%3Ie3nYf>#dXMV{(Xyxax=cvTnq{DFR zaM&mbO&{eKr3m7ViT09oy9t3RcGVr8XPBy4jfsMfLVscJ|ALGEwg!LcwRXB%j(+fe zwFbJ2P*|WSf4?O(vrXy!#KP#G#2eXP(hUdjPiH{R@_+E~=S)}xTVT}lL@~n#4cj^PKO+CC)y^`^}K_yfEDbduzkT0 z%_NMOIcwhFJ|x?St<6>>1E5>>Lfe-Ar0WtdlFdNk0o&oEs8ju+Zttu$j~(yA7k(H; z#Q2v5fxSExlwOslb234}cF~B9w1!SOu&|JQP%+18UWVvEUqfx=?^iqFqTEH_`>mMN zjs|!=Yt_OVgT3u?wI#8+0d>Z5#u0gA5=Q5|9KBMEnm|2roCSG`dBX1rt>(23-hNuF zG4EnLyct@WKy%q}C$S4-1TOqObZ=L0iYgi&X}o>`Ag?XoF#ffmX+9A-Tf)$*PC5wO*j6}_0HJ{1X!|V>R1BKC~If|+Vyfw zyQvqvziavvO%6h)&uj*n=Pto+vHOXKj5gm*Pi!~_nFDg+nD5(FM(N*ld6bdoh|){b zYZj_PETCdzMwbq80BcxTk7bujp#PVnv4C5$4D@`gn$h1?TL3{#zv>#J+A znKmmCeDgt`&P*%XWfh{<0@48vDmo6T^D`RkTT*}$QHod%P1DpV&ypXxByY_Ur#i{1 zKlN!sND!oC$*^Nn%MOCcTQrrJvZ@lf>qpl(GMMVk1pB?F06FEnClmjtMro%dl)+Z{z&h?Ql+EjJ&G&NR)n}lW!tFtSj z$x2wBY%0|M8n8RK?)w`S|9j2<@32_5)gc1=)Prh#%AN)PqVaT$c|?r=KjvA)%-G4z z+~r^Kk*levg{Fo6uE>Z$PDTpEO3|#U-6K_@1BT0jHbRQEK($&q4w;dY4IwB}YM<*G z_@4-!sJ*QG7PN9?cW<4fB6?o=Slskf9EBhuejwqqd_3A*ZMJ*|JRPs@faVNZ!n*C{ zTDVbNGg5NMx9)ynluJ?|4L>D!=^&lZK)j+z?B_uA{==Uf7M?E2h`pZ;CFmZT}xqM(rdCmxSn5wLPW(p3!JGb>)i!J1UY?oCBU^A0IQA?|M;o z&yiY5PAq0jY5y&bk=qYAh*8z`?$M@14+KxaTx7q^+}V3O0HG6xb&GOAfw=pmHo5u~ zv6zt~1~BbZCP_K^_Ggh6oW7|_Y4bpCphYLY?o>rBEhBVW^2|nQ;2>r(kZfe|r=zNm zPaQR#+$UwBw{+AKgZy(WKo*%% z$T$Z}#~aO_^Q_&ln%SY$&hE5{ROeej@#o7pKs!X2Eiur#`+oEkF;|~TNw~bSGvO{+ ze|_%&Ft@Q+^UEBUp&n5dEFEv^fipc|b~LtRy)gVdzyB5>emQiDeZx?N?70$s#Fb2DD01=2>7l~`rB7;l#cdi>x2q_x>+;aS) zrNS0ZjH32>fm`?@r7thAH&Ta^*HG4U?t!MS?9#?qK3-stnziGnpf)F4mHI;d8g?v^ z9k-9(SMia*{o~8vTByrj@OZwF&d=dQ9X!#&C=d+^o z08tN~!@qR-{wT&10b<(L!Ef3IG+MM8_<}z{iMUja&BH|>KyWl~{a{|>^9=$7OX?U% zQps56EC>?mG+ZvZfCO*kCxXGAF@`#Jj-X@eZ}@%(#+G61@zxSOsc(Y=P)uQGvp=fh z7m0ykkwn)`ksk6wD?jyQuF=pqlUFzc(6@$DZ-%XX<*wMvS$pH!I20o3#u0B7_$sVM zasKNA$G30b-7@v?p!X8|2JZ3hOwoh3t4z(59rqYQz5+q9x})31*)z;}QZ3wBa_N-8 z0#~CPrAHx}B#Z`0B{7;k#zL9{TY8+CmYZ0aC@B;yOB}u|Mekb`PFE`R0m*k5J&sZb zRSGo`ONgCiqpTDihv^s!+eE6N@oeWt%mw38WOgaQ&;Zfgy);iX*#u9%@=5ypKeWgH zrqll>+P}2hxct1s@uzw7X}ZcJH9@*j5fd~(?EhgRV^u$we}A%X%@dr~sI8iKwl%d` zMX^QEq`f6DNtzAxHzbA1JCvIwnXI{q+xGo{+kt&CkpblE;$gmIj(Ia0!Skuy?%z4C zv$=o%oNXe5Ff|wi#n+-@ z_jk3aL{>B6eR)r<0%&>paichg7V4H*{il`JtO;1r`F?6M>%*5~ZZ|ImMZWkIP&ead zMOKRu%K6SDjkuAc1tUp(zjmVH-!mnou8R<1iY+B<(~GA0u55o^p8 z;l$#~%!U`+u_6iycrh(pOqgn`r4Y)p(omp}P<35a!T{$hc*Iy{w$mRqx{ zpb)uD-*306-^Zj;O9?2K$(9FtNOpfbft}}oA#-S%B_rws3R^j8`Zb`-SA$X!UV$F7 zCTrRWyFF$Q$~7Dd6@ zgVCiuwDegNtGA;#g>4Qe%O3`NclfubrI+8=GJsqLh{|tBS0`X$_0)NMnKmir@3aNReE5H<#FYy?r$6W0OI*KLd=0e&^Kr3wocY#w_15)gy%hjI3vM+W z4;D#`5*q%4*v2VC=p|SiRzSL%jP(*;Bz&MgK#^_E;@*z{2<&wGm#sGEk{e0ps1r8yA(pjNIE z;4aXsDyEvQrJ*uMaWIEUcX1=bRLZ(Z`Lz8-JB#~m9hFi@!OC5P=yn#Nz#*pYswX}@ zk6DzhQM=J*(J1d5?NA+YqA!g9<(kjG%V3+hp+Ze*RDRky+$7C{DpmwA4Wy9gTPZ^o zGud-oQP{Rdv5Jw8&arcuHUzC?$Ry6Sbo*7-Mz#!)ZhmY)JufoT=Xk72o7q`ch)P9G z*i_grn3p|eR+zS7TZ4PnXUTSaYM-Z-QsAcW3aogfd;#J9&Bz9 z5ohIWX(|<@(4$0w5vvV&A%-ku%fxq@ew>fsU}r&TY5^NC+-}N!L?HdNB0HRB=p$*# z?V2_!TwEkkwy_2e$6ZC$;nkT$fo+;R!NQ*d1?bruD|?MAUNOf*ldy+uY0;TTIriUTmc zRl6+wik$K(#k;V8tsyBT+79qoxZdz`h@TiGZ9l6zwwsbL?|QTboc#<2xULl-U!1U( zeS$2j%`3%lMs05xAgQohGD`t?N_0<1c`8mx%g>B?sWSshw`sK+5($j~sd1@tzPEBc> z{u3#QYq^zInnM|2-+2PG$u65+x2W1MblNv{ASth?feChRB-6Dh4%D0vR${J67Lv!m zOjxP(RB)95kH{5ik`JrX-)m?^HTAutT1Z{rHr{BKi6g?!OO6i7I!Rf*1Gnh6A2h$8 zD^cDXwl`a{*8}Cj6jwHFo7+S<2v_a0d=c6Ia7vb8myX<7R@dP~Hwwd;qV$^=;;+Yy7*jT z*2C1%(Xq-YnNd*02ref@J6J}vEB>eG%`!C?*W1c1RS@~J5Dw3h>CMl#5 z>0b2tjR&N8CU>#@=#}hH=M2|wjajoHd{%dFcN`us(*tblylkn0<3;i=bbIwGe@MC} zwJ^Y6>qM^>>}qEr6y92-)?dHaMP>z;O7?M)Rp(hh0I9!zOa8IIBMbO@dGTkQJH`)E zmTx532MkYa5F1QfVZ02fcrp=6H>d%CR?HwWsC0!jv5>~({i);p=aX_mUVmunlk>D+ zQHAxj`@(o$1tsP081@dZdZ_)cXiXAI#P>mehpA_SUFa%UNT}1=X>Y>hVLZ;sa}GG%M+H)yWXcPSVsKWR;ahmEsu)jM7qJQ2uD(9#E zZe1d@D?5TdFs3ZNYX;fFG&5%GyEzEb)I zh~FxAG=5}Fy@Cfnqz)aM==9Z^pO}2$^!B571proWwLkEC2Yc_lkoyRS;64zdy{@UG#Swp@BRw@cjXTG zssCH#UnK$;UB5LG?B=P;!O$2PTJ?@(>42^aO7N(z%_xGDj2!dd`1s}I)Ssg{(b$C4 z8E{N?IVHW8lsYGh;yN$RA|?DSSQ)j=++n6WsG|j~r6m^Xs%!bxB?6O;c_l<0nyOR{KCqj5k zgmV(M+i5mwY7FC}+;m0ZyLW$)zHFpj{boQ0Uu1Qxlr*AJ^)ai~ei*#Rcq!7NqSlqQ97V!=8AMW}8f}8(8MmDQB z-mm&eLahCqRR5bAbawV&a5PCp#3VUF{trku)54(ppaKDn|MU?1A8eFblEc9WXx4)M zUk1^n7vl?!Od!A@Lg4E`m~j3a{j~j9pfF-3Et*;2an+ENP~-8&DOI}Wjir){LqXCRsz&I_#5GR)a>nXV=2#TI7F*(pSKAOCnYQ>;1roI?{r51v@^4qVu!J#eKGj1-zbHx;!G^yLaSceZyEV5u_B z;j?s!#toh_iX^_NGA@&>=`~@ETg<|I2l0MW&f~PHSyu^85>%S(oU1>NsAG=wjGJU! zO2WESq*Sw2@j|o_T=h^o5J<7S`}rU_?$@Ij#t?*j~tE`A@7m52RPV2Ti9CSJLyulb7QcPjLoM- z;hay^2O(dtFIyCCkYqau4=Xg#pzI<^4x?q8#7G~(tzeyP2+rIyvFqiAUD-@C%asQy z#4*gq$$E80UCYo~=VeQAdj(G{5t(BleuEtzV~4e|AIfP zNKQM;{Su)el_=JKG&H8tq$9l@ImX+FrTPci2yiYM;>szYO&yRzkPlyf|ij z1t|KUp;r@^RWlb_+x3oE{94yIKD!e~ktJQOH2(aBQuhjM6fd3gC`=QD1LOr*lm57I7mJ3rWY~MutBsx6$O#$utqeU)f4O7j?kV^v>qd&nO;%M~>W9EW`S=LG?HAw{oE( zScNOCA3JqSRcNG)GAnV}4hnPi#E3g|*5O7x>FC8$c-T0X8l` z<3*a-(+@EJ;rw|2d7#7QkXO1uj~83Uu7Y{?h_h-IN;g(5-@)@#7%@Dbp`>7_aqF&( zi-Y$s%0l70Nzhyr(%ZO;Fj{FuB28XDOLP%6cbj1fGPvQb^u+Jz@1RC58gw*GdWt#x z!iglGnFZ^h6TVM+2dQucUr;Mo0z9yvHYjSjv#%FPhw7A!A9v^wsh_V!^LALL&~;cR zhjD#e(YAFsOQq5)dh45wHp!-5H3%oBX$@52@#E!GlAPg-V3E0((mS}2|NO5PhiFz% zpfku@vN9b>D_aUvqpz0|>!*@7r`SHVX<-rDx-eeobQA^aUCf)bUj6~}15jjlsr45{ z{N0Nd!yrS!iw$QE0-stTW^qE@DoD^sNsk)j>R<*~5n~-_gdvm||!bO98$zSkE|Q ziOb~3{(&)H?x;vmB%}*;07|u|*$Y&Byol48gX>JnPQY~e=zgC$vgj=yW3;PYy>!vH zj$Z}DYH^qhM@+Rv zk(iTL_r!)sQYO`^?>iU9VBRuoVu+o$A*@kf}~8ChvTO@RX~^#uCXzgehL zZmsMf*&3HN8CPj}aFt2L)p8a_a=p)&&94i~28wPRe88Cc=V<1d#ncD0neci93G4ND zxP~OA+spIR)c6;Q0?@4n#bpict$#DCxp6abbZSU8I|Wc5h$hjI+eryFGqhP))H^TM zT*K8?&q%nn#)mCrl_F^a)kTU;F*sJmKj@|(DO9~g&=t4qGHv`$kX+DAgmFeRd5sKV zZ1S755*S#^#I|V3!q2))!%ed5BGHy{sU7ymAcKtJE9`(+gd zVyYW+7B0S!J{MQJTrMR1v~@02RuxV{-)I3-zNtR%6r%DN65-{%q5;9kTdbni3n+T{ zBu`T}PMxFDu`%Ps$=_9wTSSb}Fw|D6yB!3$V@2zE0Om9n(B&&OySZ)Nh)^TvHYV@E zGtSm6*ssiP8}qWqJ*Ch-8J9A7M&w;pIJfa*O~ihw3sO$gdnVqYgDR>HSQw$IR|F0_ zv}mZd%eTTh$%m32D$A9WxB69XoYuO;C?WNJaZ?|mG@L9 zz$echEBqZV_jkTh!tG>UJ)wsy)|NbbqL7;r+``z=8L{h}pwF(hV=E32J>Hvgu~u@| z2}i%>hs$}-q@?eksNZ)$Qs?4+r8oK~sj8d@4sZf&p4@HE2YfY`L7{N&!kz11I(XgO z>cY_#(-M;~(E@zxS_WMbRGua_?&_Vjo2W2wB6X@EdtICs!`7D80T{q*jORzl9r9IL zMDF{^fB9}3$E?d6HiTrRTD6%VoyVHDB{O@Z?{itw$}6{w_}3~9tN;Ai{Q^$!?0W0KiG z(@4wQ-m*c{XgU0_#Tknap+4|KnUWt$Jxc%WSMg18M8)D&t2C~SBN~)YV$1(gW00cx zB>PavOjh-c?1bC1#w92%L0BZ0_As5x1t3_L%(nartJVmU(<+0tLLT-MTgUhzHlu2} zPxm`d=e@AG|FP)xOr$BaXiq2BmXh7{Wk2HD_|2xNspz}y+b$yN51Kh3+OP#X@r^>& zIhEqxdQ3Rlf10vzN!jbKpr$l2C}gmGYt5+VtO2e5nh6fCDH818m$+{f;zatg%G`PaZ`Dr1Bb_1t6T zYE?rWR=onH8IU6oQ`H0~Xrp%?g=7Wo9d)AiXO zlBD;bI3X=mI+X_pKVWhB5zr3X0HK`c!uXAQjDUWX2BnHzZJOc~Rn;kn$Ndko#T)T; z=MjZDWFIJkk&&N~dZ0v%bdvu)U6JOw*ti)M-nX>jY;> zgIjjq)akes>hOWqtwGQo_t8D_PWCf6jJksh;VmIbIfko;>nK#d4V6W7rK_uL2nBYuoqjt)rc|YUE_eO?ZXX0*gcz<%XXs`Ci(9%}z4}w42}1AXo4b{Nf98Q7we|abX=`X6l6n2s zJ>vFsg*X)6Wgh5N^>h9>-+FnQJR114_jVDc;tiQT>A51|ejK9%oZja0f;d~=CaK;g zjmCBOluX;Tgv&q8kMbw3EF;mk28NN{_%k5t7DA%CU!arqOe%IQe4_Ke&bQR^nU)xy zIw9=K8K-zveNjF&SgiN;E{k4FoeJZg*ha*xCW5H77=^yVJDz>2t4`*S{RZ|8W=n`z)TxBf5LvO1OSnyE_Hy=j4)HzYun_wBilcMEM|B zc+)WF+jbdzt-zF*n0SrdggYUB!{};m7Qd?As99nT`AIJUv zg&Yad{G}y?B*=8ei|}D-RC1*v(fsh_i2qMlK}n7upk{|c!51XaCQw9Nok?=ZIXR?M zHNTRQ&-QB{ZDjUWtYX;-p~vZdhXI@Z?3}^uy@p!-rr@n|+GS0Zq4GdVu}@o8`S&*X z3%u;7oF2AQW=PfM!EDU_7E=piU~ z0&dEybfu59%S5y2)2>witFjvg2=lb_wSria;)R~`Ck5R|!Lix|jjOqg>V9Pn^H#5@ zN;F<6M0mt0NXM6QMF1Ecu1F(LF$uyy>N3>?R9ZrLMZa2As#Zf04wm5Dkb{c`_SW9H zKJGKCMqZCP<_atJSG&YyMEJ=kKX)YHzS%wN=XeTA*xM zKJm6rV=3GKs~UgW2-EbvDm!F)n5kc`%6wnPzL^CdHI?}by+w(R$fFbz&WvCt2|kku zxJx6^n@WY&AV#g1@?N7}L`pteqhg68%_i>7e0(b6mlN5Le^XmD?ne4p7Wc<>ZN?SwRR^(m2PbPO((sKDgzcnkhWOBDS_PdzWr- zc_mp%)uhknJNcy5Amb}tzV9BWrpS{SR1o#4V|~nNA=Q}Z;`JMp}WZKD6y$65#F~ac4&0{ zA~q$Z2J&RW4QC&@pdlY?gKp$!_ICg|R1_`EF7;Z7k`qU;t8c#wbDHJ+a4Oxo4(`s4HF(bwhW8AU25I;!_ z+O6>w$sPJT>Qt|38*;5fpFLVEJ^mR^b^7$bq!<6c`Mdv{{&V{gI$CbxuOg^NO{w8ufQEwjL;{A|CnkdqwlEaCdC{Wlc$t1dQ1T7aD^U>6??)(_ z)?JB&YJHQ?o$h1%H{HYZUri0*6ZjT_0Z+r-wD9Ox;0eOLMpM19QF{MIg4DT9$fafR zOxz}XM!=sg^5hr1Y&H@10bD+!#AZO>p>KBR!sWYxoSSiKeqC)6l$+1a70gCY)p(z( z&_&K>m=+y!N2uUlnj>Gg^2eW6Cd@x_?)-W9NHI)hgB2xXq&t)on3C#Jm990!BKk z4Z$vazIkQe8Qt8 z@HMPI8gh!GK{aTMF_Iz4D!*e9-hFl9U_ncVZXMR}{weEyN#Wx@5^6EZr(m3cZzJF9 zG(=hZUZGqD}L|7V}le{+8PQx?(pqZIysRs#&sf6IXwV9;%zIxT?v1F7T& z1dx1SgAKt2RnX?$?|=xZ_A8hm3ZS9OpeOn_l`u4_=qy@Ey(0gnQ`l9q@Uc zg55^n*-avBRu5cby_k<2l$4Fm04&DZF(h3j(?0r0yynuY&Ou5}jf8Na11z{sWH|k| zdXn#Tz7u)!ysIdD7R{E0&wX(j4&1+lR}98n&|3+(y>SqkI&Ake52Qkh>Wp1@_F3dQ z^|(@gWT>%Sf^a%P@jEq(;!M}R-4K~!;bk(pNghi0ky8xIK3c#?BXDdn0i7K?j!(5j za&dXB#WeH66sLvgVP`=y5wufy-GSYr$8gzyu*;d%pv*8qu!00QLYv##MVXpdsAuPz zG0n%-4adnsyT|gI6owIqCJF_fp2mL1x990c8M1nVW}>lK&$r`9Vv_hxA^w-1|DR$1 znGvd=ve^H3M*b5|{%=NF^`7DBz`_0tQ(^8YAO0t54HCj-0Y&()5s26|H1L0ZNrWHUUMipQN!h^} zXV1A0+m8j<40r}23zP%Xo?y?r58MwH7!#NtLI;54a1)9sZVUuh;K4kR8YlNEf_2 zOe#n0Deu{qceoTJ#ehz+je$alpK&ZfpK#9Q0K+3?2K#RaEKrFz? ze$;)meb#+=eKmfTeL8)%em_3)e&Bw=e%ruaKp4Q5K#{;+ASpm8AkQF|@EQ;r2y2ja z3;+Xu0XhTlUqBkrWuQx-0e-4}%YG07P<0S>7;{kOAVI*$0xSmjzhJ_E5BxCtJo_N} zMu`6$WQ2E;!(>ug2UrOW2Q`Fq$t;-`AfURW)%69@kykPQ$%U4dU9%qWM7?61Cq}hl z*~de*W!~pPm1EtP`|l!xO3%LUfZFpLu31opEt(`pUj`VWxhRj-`=~xLX*2l@T|WfFM?Ox^eB~(5k?L(snuGlw9uo}04ib` z86@=zJRDA_pLUvTlI9+X!utx|6>|_TjhKX8BIcaQ@o#x^QYXO!{or)6G=cNO{>jnK z#e-cEx_J(gziZHBOZHfm9KFsTwu)>hnPYAoBNI#Zelr;#eA|WyZ z_S@z}n&%=iL2zk6r~Nw=yc1Q?0ILQ40zQmd_%(>OMUC)^C~{fo!G@8iH8yI3ZN<=R zBUKmDF{M)~GpVzKoP7;*7Y>csftE?DF=^+*5+ao7O_2-x9a>8voc?x12)3F) zZ^+@v1^OApl~ApWpp|(Scrw^C%K{OHdn~v6(<&TYeFMThtxi_>3xcS$%@v+fb3WY+ zOz7H{%09&dOB3N=mMkMeK)hKAYIamTbx4G6q@m01Y+JR+}Z>)0Et&6xBiKMfm9`jfH)M^46=NS8D*6unNKq_r6&H71FbC-de zx#9t%$M&@nu{swzhpv>mDfcm|u2gD8me5S&+GK7${{v3o{_-8f`I?3i)U35UpyU8e z{=gr}+dP%N2-2}cRl;01eEly`vmH|9FM{%LabffA+a^W@@vDM)B}I$x?4vnc(EVj} zR}T^S(xf@hTbcZ5K%PTq_k>!TMq5Odz)`yU7Oc7`iL9LWbgXz5iN9hy%5@fY(U>VO zr*4A zfiRG7Gm18i6F+L##2fuab5)sGm3AH36bGNf`4}|4p@C@~6H2_V(Q_lfyqgi4 zs5q3N99V7+P%zHsMr`3VqaZlZ`--}oGvz|GbsH6;*2A~84PR`7hsd}jKP7^R&QAUp z%r3LsgyU$Ua~0d46RFIwDw;vX%>?q~T7ufLkV=Ie5>lFFQ;aGkwcs)isr)(VCd0Yg zBdWyfPL?k#0br~s-T{K<+TWa&VIK z(@%3Y7?<_C1MxLZf+j4!&hannbw8HBI~Xu%+P@L6HCv2q&aq&WE99m9iqD`wSic+a zpO+i(e+2A12YU*{uJ$y}g_sOnNe1LO*2u9G_?FQ$31uR=K~>tLg&MktNbCJ4c+~sp z@{?5oT=hV_SYbwguM;HPt0dbu+U2t1CvY`$9JLU%nirLK#+m7h85z(9t+OySn0h~SpQ%Oeaeqoa|fgx(&;kKmF3_8-flGkB4ue~$thhnRkG#u<+LA4Q7BAE0s_7Kmr?>#iTz zo17SrK~*>HhS!^nA7bB$m{Y5dVtqqatCvfX0YCJU=aRf730`)Rd1p6`v1bW4PzklXX z1n7Qz^Bu_F-$*3k=SAT*3P+EdGp)j4a+^_!77iqofIB;Gdj4sPzYi)9*pS37Mq@)Y}SVqHYyh%Hw4Qh@{u56nqcN z-EuD~t^pPnu4g$Z4#^5Za{ei6#ja0e311e;2(UoaRPp!?&!lCmK< zVJ$lmhO8&762WZyZfFz1;o4c?9r4@L>$yy+X=frVr=ynzbxfJevwTGA&1M)#rnlNX zN~u-c1M&8(XSc?ic>CWbps_{%&Q+`}?iq#&V#f8n8lYMAN-!>XM4 zu+?cl$uMRo=1D%2DeJ7g=@|qAJUnpDrg)?OO>%*B0{wi|lp&a=_yf$JJfI*$Lb{1Q zY&~Rz_CfQ-UIG5D@MRon&<7IRunfyItlkjMpo@q+LNX)`E)66N5P_@gV8j(ua%c`= zI|Zf!QUTOBWGy^}-id7p9C1ni%s50W0B=ygwumGRBn|8Z{0u=0N(*@jx~6_<7N*q) zBLGyVw1i!t2L@j+RmCn62!(A(mxnw@IuK%xaRG-M?8AioTLjjC*8o&z6KV}?jnE!w zPoPiGFAz91Oc>q)fVF4ehwMi(tQf+fg=Z9IH0%(vM*C$HhB_1;vIfEdoNNc2MJmTW zM0KFjhaJM9oMjUBF!bsN3*_vX-v(u0PDDRAZ^F@m_oPQu!#c!vpe4NJw}?9*l=%z5 zvWKDKL1WO^HjU)Ae`?#u>*qD>6S78q4as91Cf?_+!ebf+pcqQ@GqZm~iTQ!(5sRx1 zj}UvZSv@vkZuVEYcR55mn1omdBzN^heCWB@2hwZz=zCUo(L_DLKD+ymUL0n)a;^UC z>L1o&hz@VCd+N@Ic7D6Vgds14d-8Dpy+DA&5q_|aA9SA0rTGuo@li*BC=bg3ND8hy zNoH~L!XyN+11yg3u=E4XOLgs+ez1Azw!;$WobrvyY4=k$oX-z9Kbj=cM|gGY{=w%u zn9PsBN6J3`3#7}?Ey(&nxiI_lsSui;4nDzvsIY>FlRytqy;1V8-0Y`&%+84)USn*bL|&T|om4fS*!&;zELZI(tI%85XSIeCp*0^a~h7_f1$Gp&pOl_(FOZ zsy5_n*&fk3V1U#>&Vimh=jPP+&6{@&PKkWr#kP%*(!kT;(?HkI6>Lte6x-%Ons@-# z!eDYbW~Yi-BhC})9nAhIDU_spiK6Xu_3r|=}Kv=#R@V=opYbeQ{a}$_r}HbFv=5*(Pgi%%z%Of{@_E*u@4KW zFO`|(VCLS4eWx1Q1n>%MbvY*;&TN)q4K=MiRue9(2qM@eVs4NHoeMU;R>Ch;4Q$>&7U z+1a9&c4`%+v@U`e}{}cNiQ0%A5QV$nm9-u3vLRICVEV{sAoYMRYJnsxN;Mn<~UXXQ28q$ngc1uAhHI zICJL~|7w)?0E z(p_6_D_BgH{mhmVhsMua)Fv`2lWcfJvnFVtDvbS`Wm>bzTWYR1c>rhh0HK zn=KoS>rJJn08No(!x4#ew60oVBr1C(XDq{-=P;hFmW{`yMZx8Jm&V{Hq%;~NGtJZ- z@>#P{`YCZJCenyE(iZ5q?7@prfIXm?=7!)pQr(K!9V)}%OPOexkheuLJw z@V*8x#}XtotF&G#&y;`SI-kKeAI5#k5!!T9^zBw(a!r=`#)C^{qkLmp{G3v(H%{z= zNYgBn;z<&@)P2xXlyoH^ZF!<*(@y(DPlH}x)DrRD-m@lt9T}U}<1WB6_B^b1hg8~I zTlLLZ`OR4Q4fQYRlEUi%_ZaoQ%zbKKovRz5bP1NZQEVzz0M$B#c+^I?h?ha2L)a=b z^XEk%4OwR&<0BdjU3x0jP((1-C@4E-e=@Ld-$o3Q01NwGKa;Cu`| zS%yhx)^)Vj(|1NYh*Kb}-JDE+_=Hz2F)Mq)Orn`eq~}_KGk70q`+AHn*Ev(IE3yF? z)@A#qekC)jRjspoIkoBCV{YcCx|PM&@jv|ms(v%VYn*6od z(8pejKslkRg;difd|9`{1#Fb)#4{9dlN8sPvcJ>@6PrzZ`#0o#z67dggpOB?W@(gJ z{$NrL51{Xz$C?swsMUHP!tAQF;_#l|c>N~fefijchJ{<)S0?WUDOS46N0co;Y*qn; zfSYMttSJNZ-E~zlr}!O!(!|gXMnfx$@TOz3@rphgJN6MQ^|g9{%hs!DyJ-j@`_`uK z*P&SLZB6*bHv_2ry(v8c%-Wm}70Kdv%Z=}jtP`eQ*eS^6Zmx+J4DlOrxwSU6`}Z_yWIgGNnHGBT^vVZ-MIG2*u>5SG|!`Gl>iO7)`X14A}Jj_kxTqlOZg2;Z0W_fFuzD#(r8_bFsS;a@ad(uQ$(fA_%V5f zV)Wfi;_vH~#3z)rSpXp6>LmE(c`4H0tkp|1M4oOl7Rxb?kS(FjD1c97Y3C?i9xAPj zEoZ9bhez+q38ikon%An%G(@K=YS{LsV{8f6jkfB4>h0vQ%KB1!fQGm$UFv_os5sK0Blo|_>0#c zS!#2s4t4cr&JVZM0HSJutM&7ku4RfbH9L|@z^%CpMVH$YRW2pXx6nkd=d-xk4x#qe z*9+n6q5SOE>43IRf^oCFc`)C0BY(&J1jXcpM^VE84ebsgKO;VGV zXYV20`Z=Gmi+=2lyYoQhzpVT8hI;`ef}^J^l+iDa1F{Asxu4i}2ycA5Va z(I*u@k9Fk@xT#{5M;e!tX;$4gDq&74O|oc_-F9O?0U{ny+Dp~TQf^-+;8z1vCtAR| z0hhCtdgPTgcNGx~;0_h3lNHFb6@yPhsnX;e1hfsj?!nv$L(L%i(iw!23BGtUvw7+r zF3+X*E`F=iJl^k;Hv6i!$HL4}nh}Qe!CCJqBHz0i%<;ty7we}2^8pVpJLls-c$}I`%-&Cq2PSn3URc$Gf)ru#; z1HJWGMRGZ@ron&qr^~#KJ9aOFD<}Mjo_u;`6p`PU+$v5L`$@anOPY8?N zpBCr z0JqL&i{h(?x`h>*tEOban15}}t?|!@n(qy%F=Wu8Xn~oP4XvOYFA8D+q2l&Ejq_7oG0r!^CyeUfZfzb z#hRv!DV3{3$7a+9wRgW#OH6yhXg;D<4$<}+&9QFm$D*Xtzf&r=PPe?iY~NJr{NkG5 z#NA?tdh6a=DQT_vwbopAxQ8d;Ns&YmMYCgXUW}3XOSoMMqHwwe45}HtNmIUR&mRfa zrWamkUaf4ny}^5MrNc8Tk)Ksb0Hc#*&i0UF3$`X7xxbwKfSzXIWOqZyjUJzr@|u(5 z4mCy>-o0Y_kA}bs?hpmHu3xJ|RfTkR%~vr#>rv05ZU&md`IdBv^IQ!22WAvl{i$04 z)1+-N$T~*dge1497*TtZw*60}ZT{#@bZ~V6wQb;tr2T;nP&LNj`Yx~p0J#8hW8^9$ zaD69O0_lL!A49&?KuJNbOEhi<=bmc}o_Mge(MWe(&^R&DVI+Iw3fblL1VJ)+>VR7BgN|x z{S1X%Knt62z;lkaBh9)GznL4epz``OnfXm8jYrU>Z&LH7)Z8sB=hVZkzxFK^iZ_=u zmW?quzcXx)KmY?X20Iy8JA7R1mS?R6IA<+Q>&rma#Hej@tK>00K#Xjl?HjdqDs-tW zHf@AWTmP)DhwfqfQ2tot*vW(*{W#+(eKW_dOmmkyLp5Yqz)@wSx%t59tg$|eu=yh^ zSEV!6Q<}N5%AVCRpXfKqC(T$pd;2Hc{e@>J&%m7(f$#7sYiLGdaW{ zacJlGLXyq-Yj+6j)`#5F7sm&Xe7D(!1MrLZ3{-#iKQH+{Ek9!wBuuK>w)M=BYh&~QEx-5zOyhbLXMA@4b`_ZZ8pQPq z&iLfFyZn3bRGxha+n^uhRCnb(q>*H`4&ZP+{qt(1# z6Q3^V_N?^-M6h1@RnaqyleF&PI(5tbIS5alIXnYS@5ncn*ZB#_UNX&Xns;#H2(LS$|q&( z-KVx_{cq3TB`0upk27%HL%2RL!geRoJO25po`v!Yv1$vOUWKX5&8Z%RC-#9mPb9AA zg1LimZ!vn6>WFW0I|a+MCPSFn-P4$gs$x_f?vs?_0P1nw?sy8y@$IhoQJV4X?)Xuv zao(SpdYtzsrX26?iWg@o>}qXwU>4<-bG>J0kX37RduE@Qcrr>7?%KL&vab>*JhUdn zOn;^cD3B))E?^BlS2v7xT#%+xFp=`FTUCP$c@*4Tm zwkOD3CJ9N5iDg10nEKm77w6k__$^{VHn7Pivg`Ujip$G?9~D_-E;f~xmP&5Mf&ZJ~ zk!0_eTMRpFcQd_a-`(=sy`4CQodqcysm0gu&m00Ul7{?Y+PV~g=Q9(o1ckiyz(v@w zxQhdA-Kzs_Jzz%{$Fo0hMRzM}Tl7zFlZJDH4KGCS2G&<`h}9s%)kk^MW5R3PZ-f5_ z{p<^AdN|3>V=xNGV?HR%&O?irixeZbaf>e4OC3@hzJFo(5)ync%<%~}I5gxhep^=O zCItuBZQ5^wy3l6l`TMKWOvPJ#*bVzqmb*i%BJ(%9t?AhM@>E=Fy~%qAoeh3xCr+l8 zHWnW1jk!)M{ala#pOu=G?*7mI2y@1k(gK#`u%YLrMN2W0hY}n7OzjoOmR4uxwIY)Z z!!dO0m9S=0(LCZ7BNGB*t-EBHvSamS;V3^kBtCEtQ7(knMt1E zUPYgDJkORjq@{>h5yI>j1`EpcyaBVt+{nR-L>IAXkMB*lOj84u(4T2TPw zG_FKg8N2Ds80uYS0`U&w1v|W0!5AgOn;5CQhVAUN97k(JNh~TSY z_c8u#s(2|vp47i|sEl(_hH%p2F-N2TPVE~5)(*`PTY0R{ekeMMvMQ{B*+o1 zzlEq_OAwc$Mv>uyql_Dh8X6LSa1(@+u_`E@D~9w;m3TcG29~r0>)C<|&O}U+rONyb zIAms^#$Ej3{pl-!#Lc`Jk(pP-MGiX1kM|vs^HFpA;pMDHAt+V~rQwFE`?b&kyG{r1 zFTlz5sFi!A?`{X79F+%Is3h(I);=#mLO0@Yu#LZ&B=o3>>yzTczzZ#i8uk9l1;#Mm z6`>Y#+zesmC^__pN4jV1XF!8{&}2d=XL@EH1lHR(m{1zS6%pTIZHF?k8U!hcwaL{k z;jWyTvU_B^L;Sdgaza!)8wxW37N zZ)&H)F>e#c*%40Ba`lQYmOFZcR;`?Uifi6oe3ENc&)+O{5zjqiYeJoQt1d!n{+&Z& zZ8YTY3qmOmH9`8wEg7+yB>47)C*N`_MlQ>-hUFtvs&<_Y$=@ZQR=ZCC!c*^9Ivyt~ z&e7RuJ}WKNA-#nD6t0Hn9->zea3!Pm_A#h_MA%*!HTBY{O*<8%+0#T1wGiZlEv?3MQ=ZwaaUIn3y7v zD~a-@_Fmrlq&g*u2#u%G8s>ZXawf)ABRwaW)qpn1aDIb}PFoYLkB-m4*+{ z=KMXf=~v11xbBj0RN_^*@&Y~breHD!7;3n>@K^M1yo(S%yO)~ZCAMbrTWV)c;Nt7q z*sZVNnA;)iFH_0!85vfy(IZXB4pV6|yIbsBlz>giC2wtE%CMFIz&3!x*)1l>9s-BD ziw_j@pXSKiWYaG5^elpc_se_Du4BgxRTctEL-aV*rZ9_zx4h@c$%?5Hn&@RO7MlbY zbdJIxq!Ej(EYt*5c#>&*9_q44$RTqROf3{W(4&z))nW3W>Fy;(>A0#tfU7G+YX6~W zjON)sWCb<%xvpFR&h#$!h2^!++cxbKKh&m`oMDDZ>fTu=De=6^;;hsru^eMH){w1c z((hL32ignca>aKvnM=ozKE^=ZnN(^>QW=Ke^a4P7u!6m*c(x)9^&*D1LYehKl0~d6 zcE)Io$8e|X!XSoN;@U1wV?|WH&dkis3PSVAAqV^xlpFsKWp4o#SJSi&p5%hPFW zn?9}28nyGn;*T6Gc57nbbHWVBLKUQRYsnkUVB@303=l$@G9QKmu6AX}8jO*fWHA=& z$QsN=N+$x95@GWgkb5u7MCJjIy__(;w?JD#ewkmJCnt(It6Rh z9k&!8PwAZ9uCu}_V&6FDEV09B3`!*kIW0bHWBWi`{UEq^bG$m**r_btw!1Xr-0qkE z<32dwah>GwUEz}XGmXoR-?41Y&m6%@?tUgZYd)ek+1ut6Vb8>ym?k75^wjJyoP5At zdIlni9{i74!Y2}=tK^%4=#@s}bh}=X@2hj;Qz?2zX(b1-&KTZ?WW+j?WbZ{P1m;|s z&qzC;BH;bdQsVD9pFIC?q-8jnrdD|5X`jIIA}1jN~M*9>%eH()?iA0|*UC zAKN*+UE<7G%j4mAq#wPCF;Y(3%^e1o=PhGMNIQo(8W(i7>tkJ3;4v9WvH{pU4TOMh&=S11xY{b{rgYNl;y2iqsBGyKo@Ap!JNX;IeFs{yZwFo8| z{1|Q4n`=M%!Q1S8sv`;yykZ2g_+bTd`GR=Aia5e)E+6-xX z*pIR)rF3`e$UmQ`GZ)fws4oQEFVGA57CYg3Ps>a6cQ!cTDjR2hxV7kl)J#`<1`SnC zr$@_Un~&4Rv1K1P&P=u10+RuTX(i!aXEUU}IrL%xc`Z9L)m=g-W%jvnqWIR2eHQFz zh#{-Oo7mm4d+hvno>0^edMaN>Zcs86jhs4>KWocEeb!D|)mBk@A31!K499hn*{kwP zvh5*$qLL&xR;#{0^U^*?S0Rq|kgSW+_>ecr!7Z?*&qn;uxmN%8BB1_-ON=)iE(lj` z=v}|e7Ae)9fuul@DxrrmAqL_S1BbqXiY=p}g-xsO+h2~IDImq+IcIi5iDHn2e5`b0 z)Sww6=BKv2*3#tG71L4xUpoCSZ~C@iaU0m7V4(TA3Omu$XPKTZs#!T{--so3pY(I5 zw_Z4&4cqR_YtC?JJ;2s8IWbOUb;|3OfJ;K#VbXOKgYW!1aayHMh=My4r5ewHIy>?l z3_yvnORi&@RVA?R=1aU=Mu)AMOBQ1F=a?;VQkB&n%7(ktOTlpl()B=R0v5WhduH?T zqs$HY?4V}LGpuH|kP2JKNv!OV6c<|Ep{scv!63VJ_Oi5bBcMKi4x#lsPPLeuD<0d_ z3G?O)QTL?X?=4crW-M-*yh-HbiH``rBMwQZ_CnM)D#hEzUiIN1|3KvHc5FQ~o6*7A z}t*j&-8lOb%TUUXapQ<(H|GotpSe*l1=6%ZXSmP?7+YLJ23obgh+t^ z0%DgB6t4iTUMJ;(1LeFy!v!YIHVW;HjQS)FQK5`bVPk!+8Nx^r{+5fVi`5iCjsOY8 zFdUC!#*x8}ZRP|G+t{Q7`^Hkgys^yJa)r>ucUJT3V_Ws2ulLE((dwLpZ&Ul9)-|@Z z4{%UVe#@{&7LQ_;NzkfaY#Z&2crb^JRdN%a*>7VWRn09sETx}1KMBx%e`WdV@v+7O#W z+RU&oe+Xk?X;QxIUYVfINMd0hhgI6UeiV8dxjeZISjTMoM&$PENRY+Q2V>H)?S*-i zs+2hw*X*N$O15b}e#KwopnSO{=4SUBrzrAtg?g7oA#5 zMrY$%CW)@*=|2|NfCMKCSn}U?7Hnq<^z|dO*dp9?Q`=L&bbJ*`VEbqzbwu*?-NM+L zVbSxXZQq|~+~USozn_ByJ>&87#foojm0NAqy;1dhL8u=E215|&I+VUzA(R989`Wwv z(~E1(OZaHQ4!Z{!LnsI6>80A>OH8coAmu`OVnQyzu@3hPs+rZ7MCzE zT?XZD(EOeLROzQ;L2b!{cCu}w`2F55nvyrrWYK-0__Df|iG%KR$pyKmW)ryB%}zBQhe1bYc`afj|U%U5-$ znY4ns&a;kVh`cK@iRiS9n);lAzx`DZC&Q9uehoH|>e4YThDIC^YgR+rzMc&CQ|Rp~K3TySze>wzzI2Qy9+twe|>6CiyiuaK1fAtwhUkRr*Fz3>iQ_& zFHX8kYGY!caC5J+9aXS5?&umx$Ty?$v8_gY_z;;qD&Y*|iJprw5(^Figg$ z`_>HLmS1OR_04opw2YUzbbZcai=h|l{R8i6nM?14jKYJblEK-<)lIZ2mwuX_VuQn{ z7~Zg&_M5Nizu+I+J~suKFv#2mDsSQgPzSu*Ob~@SL>vg=4PvUQFRuhwS^6P`*1Ea! zq1U$eGGJY!9WJI-eK2T8Cc-?v<~StpE_u)bRg-;{xOxk!&W;u=h&70Ua?r0Zh1TBn z*;^cc%X%UaTI)aKBo1rjMZFRc>X6L2P{r~dIomq$k`YQfS^8Qm{_6vYP?sT1 z2k6X{fiH-BWOv_ZX5#z3pO6q)4e54R4WTdW-rU4@^Tg&$L?NK?JQMFDkM)tmX@tZh z&?F)nnp(v--fG-4QzXmSIu0RS%6vgMeeYM+TrLsd6eW1wdBD*Y!Pi;OT%$tAO!ZU# zA1}&G**pC+){F?AWtQneJ%dLeyC32XuI5$ zHzg+wG*v7Sa!E9OY4iF$kX!-i9pOfq1tOF_Ma*YwQCR5{RXKhQ=Gn^nm=k$|r&m?e zwiz{59c?KKFU@CfgSEGfJ3nx3X|4H8AkRRSxVp<7k(_>9TDB57__N>Z5&PW309URI zt#kZTYp-!vVjqnNbRK<9U`kbD8qx)GSYGQT8u6~rebg=$OVAqwZ$Fj>we+?!8d!v} z_vSX%+71&5;hD*r1SXe3mZMj7%dbliQ&D?K>C*r*{hmOQNH%S#W5%j@I^BQiInoB;Y`8n%yy+0W7)T6d36}qs z__Go+XRE<^Nm6WMd9r*v8rmJ7dk9bTre1W`hhLjU6D!-d*xy|^zrEE$YafUb6nR{h zQJaM?w7PiFbe&#wSh$G?I;DNH?b&(YX~%VUXgD8iYk=3XYnb$|&}b_6?C zX3s;!FtUns^p^D#TEWbF=ZY(vOuL;1(V(GJ7m`OE({wlci`^8Iu$5)(Q*RyEWnAM{(a@6#zK@O?s>6UmW5(k=YTqgy;@2_QjEYUg%Dlg3kp7vz;6sS`v z91gCwc@?}KFrMS$9}7Paou5{oODGd4mN>NS?90HaYNQFjOLD1M)MEhVdiv`XXfVM= zLJ>-FP01rI`?#EBTy*(gt`?DTUGX7}GMffnAjXqBCo{4J_6S3374+yKutwDkN0p*X)&$Z$_HOW$$y8b0qysgrR z@Cr65t`0I41aYR4t}p^5+rl!^3#d!bDT@H)Wv>p5py%?2cjIfKyaL5&5<+AnJ}_9$hiti3=X@n9nkRi?Fvmk8DM$ zh;X2UNVMG8q>VfmNEmnkS8YvKON1t9XQPDdO|ghtv$BpA1~C$ggPVcfsxS>@gchtBb9?Pd@x@{qPTQR$hsVk5O2Ic+^SEvg%EA z8@9@L+~Pen(NkvV-3I9Iyz_?qGJimy2Xn34u1YVhLsvsi1EJ5qo`rnn3%%Pblo@R7 zo!J(^n6A0l(NeS3tETJv6fB!l&qA7$zQw9UTx%i4<)Tp5p}9~1TD4GgC*4jSvQFww zWK3#Vy(|z@=~j;Uu}z+?rc83aN){zz>a85)1zSvgoC=RXYrKf%qtFBO=EsVo%V51_N1YZe^RSpVhPTZqoD6S398B19C;K7{v7jx^)8K2iZ^_`#i|_py*` zL5`{DkZTbxJ^E+4byOi>`*z6(DVas`1NGKdUBfoCo_n5kn7%d-K6Z0)yhg@ zxM`qCgk;o7pFz?(<{20Zibmr%+f6HP{2wG33XJu{N^0&X5g-kEXMfc}-w_hsrGHo^ z^5>fv&?#dc$kY`*#@Gv{pdFzIV*N58k3UC}_sJ;sLkP9%qWPQ7Zw62^U^7WdAG z&NWev^S5De&EPn@NT|0NgdaYfWzA=e{0254Gb{t$agS}jtB^AXaES#9hr;BcAeOUH zAy`P)bkhe=J2fFW2#0Xx^`fs6`h_=<_Lu-aFkfTqbqc}8r@TlGIw3Zra2dakP z2p9p&l^k|?vJdMp(?4hJKP}e!!z>q=j9cTY9wS`l~Q9w ztAkqIb5pWz@JV)?>45O)SKjA;DA_87$U`0r0+mDj{G@tof50{TfH5hCu7k1Mb|BhviIcjlFUW7Pa6w=@2#6jP?p{J6NXw4@oBF=n0- zGy*fc6d_KT!;vL8cwlSABpaznHf9p;vm2DmYVII`Cdk7^Gfs>Cm}?J7_h1Eihzx8E zPr2aM`3|YXz^f^n%|T7+TSwPnwy3PT7|T$9McX$eM&OM(_^qnc!`U__62k+_17>KV zUu}Rqy}wB9d!iLCC_svCWImd>aO~r>`|g(GHnC-)YMIM2bfQ$OS|p=L0TM@X;D9UA z6^hO%yA`J)1mW}qNV1Hw%xnGw*#IyN@mnoNO!@H_*~$3u)k0&bM;dg#x2zHK529{q z*?wq4^<2M+s%*Dqt>_HXUq#_0N6y6P6H#~0O%us^f z1vN$q$tH(zW)VjQXp*`;lcGJa$%(SRHR9HVeWjm~e0=H(w`3;aqksvCKOXzor3OeQ z8o6-om{^E- zUGgDbBxj_fO<5!Nf=jO<~qZL+o#>7CFccARh zxB?j^Ej!};zC6tWOq0JbJgf&Jf~$Y`4o6}&M8L_OTrPJM#2%Srocy{AO#p6wEmA`X z1A{7z&G-rek9PQq#Yx!4sC#@|L2i9wvGbd;ZVS${K*Lq}5uVfBhTdWz_|rgi@) zb{6n*Eqp7E5g8r)&17Q87E!4lEh}&8LIs=-qq1dnp_*QVLX*VfQ#@sFctg{I-hf6% z(h^|(Ao&f7YmyMy*)L*KDgXDf15_evH(b_P5|nK$Sn)WEC2HIvI-h>tG`k!xc4M?$ zfzy$)1jMSc0eevb0)WIr$m#mSkPa$Ps zCA%(sgvB&C$rDPfMSCU4mw|EE_Bs;qWwT!!)Jo!Z zP0LF5O^xV3{zQ?z%yNZSYGqbA1z842-ejvxurhy{+bL)o*F9vI4Kg@zGuHQG{e#Ky zd2bkKBuMuTX?O0Sa82T^$lA~^r~)0GbB2^O@v)gmY8}joW|fZ(w^povF*Fwzn3cY< z#TR=UL7o1lcM^Ib)vD}3f`p}`naBzGC)}d$WFwVO{`<8WPhQuG;zXmPBEoc0#R8fK z%dkqt*9%@ij8xO2LX3V#10g3wg`@~L12v^E?L0hp!1KnrY474sK`4t@;1O+HeU?(Y zFpEhNa)l2Rcj!TqXCU(X9G^e#7*nFL+C^1}Zj=ozl>2C2vc`~a;(kznFS`W05V~dK zwB?hKqhWemY))~0X{_TENobgC&sw}GTQ-W?hAtIE@}7>{*UE}lA#t`ML}dmJxWYGc zv`bE?d5F{&w7@PwxG#$(gjc>)cwN?_sN8}QWCexAJ zmOXO%{0a$P3(IGoG-cScs%Cr|^al4hdsNA@-^-K*=g1KJ(tZYN!VvL`GDoc>_uG5A z9Bh;l=J|xmHJ0wZ?VCYbn`A;iik%j1+_O{pqM7BcRgmry&A^09>bAgY2Xw?270`Qo zafT88oK161YDw{jfcBgv9O_xt<(?G5aaDBm7GNimPrUQ#0OC*LX@HzyRqaWbwH4fN zK38mWg?XLqnnr$O8&4=5KcvVAd)`*sl+mw)=b}HA=>bK2pPRyBOo@|kiSaON$D@4p z7^Jd5(OV|%21!l^05>yz4`6zf>zMo&2fF(D$MQL!20g}H&h(sjb4XSt8sd))5O06d z@J1hnGY?|w##1(tN$ulajS}Z+%5U+0cKgUMo8~zz$o?svFyuqb3?{}rO~e;(MMk(o ztV!T|6{_eLoiLNZl_OgF!{*k|f~RY-^BxSjSv? zbENTZwn?1TPV#(4LyRdUit1YqRDJ=b>CqVDsJgHoG^o|@^h)|X64&^7V`6H*`R#Z_ zu#F$yR!Qx&+l9>9de5fbJGU;Hq3tDD2G|$M2Uz1ORenZlw8EJwR&50&7Tdh}5wgf+ zJ<~f=K342*ho)aJRs;=P^x-X>{UM;kEKsH|Jz*JTsm1CoMkwrp>R=Teo?e zuCG%vudiI9`R*1@z^yZvrBp~^lf}2s)m&esmP?COw5Z>}(6#ms2p~nzdZu4Jps;AV z;1Z6fRx4x4z3~LEX79S{W{5_cA7!^BScmhya+H~GI(`x7HV9|Uo5gKCb0t|-)voZ`JB!*ElmD^qobOP z2Q`^AB{INS2dH38?yE)TKFG;RNgJK@KzgI2&3ilg8)&YfOEcSfs1Dt{Q@;wH4sUf; zJ!GkW``8{cCLbmw_<-yt_KBY$*4Xa{wX=bGiRNe)*32yPPRb(d35r*smu6N8)Y#;X z$V6XBSB4uf!A4WL)16cPhr3c5UXj(J0+LM@hvHG)gqEN>Ltq=*LUp>3!Pxf*@4+YU zy6QBTz3Dl1BPPzgLyblUZeJqvZ_aTGLX-CIRbxbxt<_3D6tFLCozj4Q%G;ml*qX;} z-35qJ1IiXlpEfhDnO$OR#yI0=KlL5{EaL z6oftMEF?+caIoZpN)hXJb!n*yZg4d9fjQLo5P$vNP~R#*8Sn?2dzVC&AD*5(JU=`^ z)dcMj*QH)g##xa{jC!QruGr#1ZWDCs5SJqB#S-Yj26ed zhbfUoRE_Zb%1p<6qvdFx@oO|yTv<;p>T!J={4x0FZ)H8ExczO+v%k}7 zm{-ngFuMDU{E3ku@oL3^6>|Xm%Q72$7dKLa86c^+nbo?`K7(>O`?{ zNjmRowm>cherrUogSRPGv%hnx19q;gmRzBK?T_`)x`r#5537D-v1<=V7W{Fy$qgx? zqP*P4+HVpTg1HU(Np=Naz?9Cl!(CtDv`Dk&lbQXWFFv#^iQC-^uy0M~-UT^*&+Ml1 ztykkf!FOZO>0k^q{OcI!zhm{k&4#ZkYWoW~h)*gwWRGyN6B^;`;XgDBcx2FEDGxTN zdw~TGSfpX63XYZMXJ2QQXx{IiULH;%iArRX`xAsDK`tzh+6fU+q+rp$u-v`r-e(I~ z^tlAN{^}=K^25^U0V#Z+crKlC;d-C@nU#{%`D``OW&DaC9BTd3+V8txh7g7uop7*k zC^__|90@el^paB&5F{@uBWu?a$HwGQ1zKo)fmI`a1~`3!5Uu1!cFbsFO=SBULg_kH zoq{&*pFT5syw?S^X1Or5Q_(PMrml*QVPnr?1!X%CZxq`Zl!YUSdAKul#E&bCu@R5! z>?Be0SErPFrDUb!ejIBE;p@dntH*nLs;=J?@2~-AGiOngquwYcOy#+ZY%bwslO)Td z2A<;BL0vHU(_Ld(RF1(6`hwN={l)`D4T zx%^oV44i1;&c6)K$rt8dYBk2N1s54UHoJ>Kz682L;wR_2F~Er?f)FJ2CtnX!k;clOGUR6NKl zHs=n=tEw?vM2XE5y)Dedj#>T5g^>p=v+s~Z3B$_y`H?k_+{@wm$0Mz9HQJ#PtItL? zT4M5IN$aT?d@pw_w%Gyg2+UaY?mi*Z5khB(^R$*!*v0alpp5C8K>tfVjvaJYdZ3BF zEpNXi!UoP}C+S-hLi0}eZ8KbRPpF-WXlRORQr*8;3h z<3bJd4lR;lRb!|c6c|2~(8qn|9Ly{rjW|4w3=Gb85LUpV8WJ9qw)d|DjNpE*9j^Av zer}UeRYRaRO!6V@Wt9j%2-@h-4OjI;^dIL6e}_@lg%NMH(A3bCFak?@XhU4+=^svi zed6eU%X0`HX7z`qfD5MwbaV6P@3jA)llwm&%3j=I*&g>m6yQNXsA++~Fe!eltE9;q;Syw|MuvjvTQArfgWGn`|sFp9&HM3Nnz)=&K%{vtui=CLKemQBNc@ zz_wIT`FKZ3sW#u_>_mIDzTM*Ncf{G<%F2@7M+3w*;T z+G+hclD%2|=9~*@f!$zt!vPQML4=vA6V(&`5~w)%HU5t5Oia-K3Hj1tFf2Zfj8sf8 z;0gZHaIhqPf-Fa@)87a20qK%+n`p2({)}u*%+LP`;Q{fIef!N|b^IDxm)JA>19TnH zC2d5Q2=_N=u)hn@OR|S}C~5LSgv}7}JKEBSh1i8SDsc2+uA#00t|6{Lu3^{y*TL6; z*TB&005#!XqPPCnLDyj@{wTpHfheKJhzrPUFq=UrAt*s=qM>4v!ndJnB9fxF0nvfc zq0s@+A<;p{$P4IoLfa679)kh#L1fru@M35IAA;4ytjQYVW#Wh8>Ep%7-QvsQ`^cur zrpejKZsOD8wc{7!;RcBZH{x64J>x0Ku7KpoWXR-FqOpO*s624aOyAJ0f?>$yG2Yf8 z)ZtH|Ad034r-+Ki<4NOV1vH4Jh~&rPQ=!QSr-*3?Ylv!y#0C+g^FY=S*1)4zpDe9n6bUY;PGV`X=lKm$M0xHfR&?&+u&$KadUw(Gf6T zQusj}4akp0t!F^BSSZDFq_66cH3GkaQ=V(dC{7-wUgfI2K5wkcTvKKYq&3m@zFs5thDFwF5QX z7sy$Zgy~EAipG7CW@8(ZV*$pbm6@ujvf`afWf-MJ4$E4HH|$#tWnDWmVa_fE(c2(~ zh74Z$xFP?5zrvHqhaB!LTk?8yba5yh423?uo;xmMY`@!MGHK9p6~lr!N3#7>~FuVHG zZ9i7WP>srjezqR_pfMYFGh0Smi?RYJp_hC&U|usme{?10<>AzxR`1J>$wq-h4^_c+ zx>2F4l|^4|LGJeXc`zB6)`~7MZM<0ua5noAVXuNbE?J^=ohyjxApebMljEl@o8|`v zDv=xT8*4II9#g~qtAyy1y|qvi=9w|l=-fY|4_I(2(l_I{jwX#f-nVdmJoG5|^jJ4G zwverIy=^o&&OR%A>oO$4JWCh>i^(2y4E_?9$g%qYqP}Jw;BzYg#&+g&tf|eW=4@5s zQf#BMRLM3|ItL_!>Q}A3%Nw;8^`_n=%7@|&>Yk!g)0GBlS%2=a%}y6Lu}YLyX0p== zFPcDlsGZS@TQGgg?*gF;qslI!gd<0^UONK;KTNm)lbVUxO`8!vQ&aNPNN_cm?qQ=a z7$E^I9S-;9VwSHG$T$}=?akKMMT_4AA8?*o-%*rR8qfsk^J~UgyVn!@-Qg!7bx_|9dB}Cb#gB_qcI}mx`aM(gN39SlCQPU!SvN7KFdWzx*sjTP#m=;sABZ}lqKn?J&=W??u z$@rZN4%wSmX)i~gSou_zEhE#YK~r&+#)*TJ4~a5Sf72oqewkrq%4XZf?s`#cT_1gQ z(SqVmfcp50jYa0SkA2jn^qQeTl;p$nz-M;;Bk*b3ydd+1tPxRgl!}J9q1w(au8}wW zDR8=#r+EaAKwIx^^?SDQ9f^k64@=B1RKtNnPr)4PpV)SaJ z2l#c@LawW|3{T|f6<>j@&MPY+CA$_KoR|i!C?ydfzlpw6x^Wbda*cJC0?I(=&q0Mf zgsu8i71{^rv5$RsibTB;9;{|FlcSbY!n6{+(S&|Fs?8jZ?7O?RWB^9{?x_X@>l=LL zlGXre#_^%|E`$Z9hGABfrSy}sn$ImfCutMbOVhl{m1)?*@>Is73+_-B0^d{{IuK9H zGX-~m6YYQiUTMX(A0d%G9X(0{Oz5Fy8Nt8v2u?O|0FS2F1URUA{D*!-QuH+^@P`hLsL_qO^|}+*Z5d zgrIeam@RU3AF`>b!?;5iinKKdWtKsQ2J1p#g(9JH=faO~D522G9THM~yZw|A!Y+?B z#~3=H7uie1918KZ>rQO}25LdxIy+Y7(FFQn9}bp1^WPwT+5AYA9Wc8!cfDNTeduEa z@aC?Xp@cB8p&p)_$Rx`xVyK0zhDuWq{;2Kz;a9r#wWh;3uXc|gG>uua;w4FelTPp+ z_^ZfQN3<%2&^uJ1qLynugU~ywqq6qXyb$@B^)a;G@2KCprlfV?hXR{p_S>=nL8)C9 zGDC_H(BKnpRcG()+%Zv%Z`9I1j|EXVWI{og$lIEO4)JT`e}bMM7f@&Ht#PvZTsX5Q zkU0znqc~3gJ`xz@Uns<@ND}}HiKhsrV5e}WVm{4zpi3)Cvq(FvN^Rk`R}b8+|K?cD zLH9=qJVqpdB0wU5BS2e#XVckwy!_1}m`z%GfvBeD$GVL%C@#%e2&aOa_KnuY2WeYr z`R7F$Va`B_O*_fEqm8=%-m)h`iP6YGS4Q42In@`GWzY9O7G5Mi=)_b9Ezm!y#pHxmOp_mxTL_N_XTtIjIim?_ZV{2~ZQT0!1q`rk zzp-jZ^;=p8Hvhf9-eL?=bgQQVo=gmJ{8$qrb$(7LItjkoR5A=UxsVaECoBl3S76N|! zz3}c9$29sU>8OuuW6)0B-uzY4CDaPDDbJ`_jzE~5dHbb|MqQTY?x*pEVdqaM744v% z6`2X=`KZDbZ4xVLZFGweTbLto^dWS>=MFP|cKl=WTO`u~Tpy=f27M6QGp+e8PR$(B z{85rp?82P0P^2J_FCN<0upMgcXJ(-5$@}FUq61Tu_K&-@=R2t_cHW9@zCD$Jzn@jS z)30`(ktP0MQ{57egqRzOwJKkm%-_k><$Ly>)?gZJDWfu)G+H&P+0LoF_m1mO%yniG zxb$x~5q6t`nk=VoX6Y2G$sunnfmO7xj<`jgLdt8LsuLP4kEfW|q|uH*;HCgpGDIGi z)0yPd{J~ehFuW}bDX$Y9f|mV8oQx-x{r0;>Q>{|0LcVW71946T@5i!vw5Xz+`;g(z z4-M!tCtYecLC%wYX}t!s@A>%ncAd*_a3fN$?-T^K`?#>e#uDm;PtKI3jaOU4p6R-3}>tnz&UO%?{$BJ>>b-s zvsQNO9e(>p`lL&9;ST47CE<3JRPDAy{E@Nn8K`!%x|;u+rtG&O!=qsSZ&a1XoRi6~ zIp$}aCybfQUqg=VWePm(hFKV74QY22@|UTLp33u&Hwta3hkaFs9T;SVfV4Xz`TSHx zPgVKXpHxUFhqthhzH5sq5UdyS%O~lp43jbp3)Akv=C{)o9cw5c2W6(tS?YM`zh{23 zl1FxRuJaGmnGWRQWJbZ`TjR6B4xIbw$&9D(G>(b$UHL=P7v)s%$RFt2BZL#|(BA?S z0v6@OeUAh4P4l2Q#l?Mh1A&(e%7U0TjmYI%5rho+evTrOBANk&H2HpRB5o@A3=%W; zZ`8`LIQ4`th~KECVR4Fz1+nOzSUFmKKDbK@oPpd4boToT^3_bG_)bp)Mbjg+1SAF;j;r3Ges5LRNkCv z4ZQa8gR9Bo;v8`OaIZ-HCZtVKKc_O}ho`>t@k2kkS? z)685$1AK4CsJ6623*G}ix-Ti0os`bFtI6u$fz7NabVRRj2SdH`?fx|v$l5O3yV5B~ z-A0TZXOE2bS(;&7`LzH2dHE)aDy~J~v>_!fZM=jNL**t)iJ~Vh_dKFQu>ywc2@K`^ z>Oq#j<|HQ)+mvHV+)ho8vz9F%zc_Kj28X5<5yO)D`&L48`(|Kam85P}EV+T7_X3=2 zLFKymq&8D!g4zzDX|fFysfAjVc5geH{oT%^_SkG~FfQfiFFM>CSIJe>#I~7_j*UWN z{P@aS4*^rp!b^jM7KxR`&ou*$Ld0fvG_<9PRgDU_ye2uD;{e(23k8Gz>}|6gkz@Ay zkEHn@72WH&Fdl*JUly=ukLeZMiE3HK)AcJqk{+5lP~GBc*W1h-^U``_*H(|E>sNeK zbgKgw?|xZGyf@3RLnBLE5gI~ssdJ#dwbQ;nbAt1ZOcxj%>dctGGaD0B)2E%zI)04k zU_Rn4E4a}dAjvdi&pdwn&5t{(1)Y6<`rg$l>;ChX)gho?;5=Ww(FE)K%^~XO2iivh z6QQHwY%4pXLv~xEk31VJlQ_TXjuttmYXizAS_0BeT(98>x(?blCmOg|>xmitI=rB# z*RSv1%`SSMHh%ou&faL3`+`)(Nzj=kwj&?`SE@CDXZhevm0{dM8>eK7Vcd4d&!np5 z8}OuU5ePWv8U8IUAf$I-oxOh;#-ukkw$3p9a|dB^p}I;J6#Av<-PH{HypM(0Npa|} z>Bc-bUG{lQpH5d@j?5+r%}~gmfCt=o=l>7Dy=Ktka z8yLU%-9wKAsO4d^ZI!Y}dtA-YgO1v?$v=rGo1gr6bc%`EV z^?n1>gJ3b+P=0LxkP8&!<(Cv7X&*eH&?AInvTbWc;m*5Vsw#7X3EqBZ3#@6-R~VsK z89+%gZi99>-^P)4?H)@1E(a{j`&Cr8Wnd=F{Y50N40l$hoLilUI&A}x{b(D;e0d~m zFpBOJpuP;Jf_rBzvCNpM@UEnVi(#v(Vbj3$B$e2%bxNi^bkR96pZVgJp^}-;HXiFq zd_*qH;e6dZnOkm|><`?8hk1VtBf%f4D539*nHfdKZf{};=k4kF`8vOoHUInFY5$56>4mT1~`EV~C|B-9YxNZVD(b-0rlWDY-!T4r=ZZ z87A}9OtyYDw&_}G%aIyt?j^*tfk>A&#+09Fw<$&`rdD>U_K4$7ya!@#DtYDDHgZ2w zjO84wuzCD9NlGJ^RnoDcOvEeZ83)V zn>j3(QuGfl?C0#ZaPKe#jw^ukC>&;u1TAVjpY-;>JVBRTqC0Chv21}x+}|WOd~K0N z>CjQ;@%TRN^r<$%u%~dXQd!d$yH*v4C84UpkV4-a*zAtcwYVQVcP<~fHQZ0(m%iBo zg~O#ZCwr$kBk&GWyI${3GuI2HybbOhZmr9o-sY$>!j@P=mI_G1P*4Q;6`x4Ld-$%N z(txA;m`g1ws)Zejhqooj7 zT{8o361tXDh>Pk?W?deiCg~*FYe={?4(d_ISVs*%6HfZY%shW|D+5FXyJ>^0qtd-9 zrS)?-Z}8N$tFhDS3=Ld>PfjxSzz$26#~se^!0heYKNycC*PyxkuSv_#aFbofpt+hZ zRrLz4$Y6QxxkLRYdr!3tiQ?nC`I;^%+*x@FW4WKoD;4jK#wgU?xZPC}%d6>{!kIm) z8@}j1_&Pu8eoJkVIXTn3$<>wk!&i6^E%e3#?@>aH4#=T3c;H)UwljZ9uU-H*(yK(*Xum1S!qRdPG#7j=)&IHx_>AGHJM*jV8PtqZe zGRiIblJzNDaXED#ix$14ZGW!ej{QPm|I>4wh}9@2Jc%(F-kYPe^ZVI2A&K_nZlY|r zR7%{tdf7lbNlHSvET3+INP@nP)N&_}pXPHLuNZxXH1l)5IZ$W{i5iH{dAHSTWOT45 zwOgXyWv1Qy=R3<#_Mlm&1<~0y3!k1U?(yjOjY8QGZN1BHI$%E=i5-{lPTz4EmAB3l zmk4j}puNZw>CZmH#}_9U|NXW5e{O$W>wp7wv|Ge!g2OfIfIqC@k36W+mr@rBFbBf& zqhu!gpVdlokWe_#I1mV6{~3RqS|^b*f$9SQyfA-rpo;(iJIvMJ6kQ;I8RqeCpb2mp zA;={VK=7K44+QYNl1u~w#9mW*L0}TrAOOQ_HX#L^^#}qxrFnq@*NXoqMF@fp{)g^; zFhJzB0-PcQ#RmguU&VOI&i*HR5e}{;3;{5`5*x;Vvt}WHcdyx$5P;lkYUgk2r60gh zFo{AaxC{SerCw9Rp#a6#)Y~wC{A=n3FFXv4_d?tj29S77y`a1e2cx|7Lm?a>`2^na~BtuSEECMfB5uhaIL1Kd@ z>S$mw{;#GNabOC06wt;ECNAUutoVoA4onP@bUp+&iVSo~D z2?#*he~U{c!K=+nu^0wOWCWJrUxV|{QbGvjfsqJ_g$<%e2H?J0&ENF~ss;meF@XIB zWS{&m4qF(Y+8@~ehI$d}#nkXIL3#JE`2RPf7swBopbIs^Hz3}We;{8}1$-&~x9Pk{ z_tK~d6QnE;_vT-a|7Cc936>onga85~zLdT!oxLFNRABWYLV?IN)G=6~9wknE(DGm6|LUa&>#x>uQj)k3{>A-;;$^z7rUHmT)~Wxr`aAQXSg=9H z(l~^mnp6PBE7(i1)&D8ZP2+|I{YeENzl!_9zX1DX0Q_4~VVVA4Q}EvgGlvbryrIDP zmuUZ03_Q-?TKs?E{T2K_HU~~%{3r8wO7r1>L?r038UL5P{fFlNYG8!}Voe8-{9kht z{Xb3q!h%3{V4~OF{#6>}H_&-Hfaw3n)c^Ij!Fw`g(Pf2?*)EC2V0>9B2ngo?uz`Rm z4B*8DiDZDkN&ixTjtaR6q4=ml#&+;Tpiu@M;zSH;IFJE7H_8jj?+;9=_Tb|!U%_v? ztN#mSoCSpGiUt4~R`M{zDE);prs5)m>SBLcLF1{o0Iv*&|CZA7yp*no!*haaGQmdh z!ds95%v)6mq|A>?1OnFaAc24k+}LD)!NQeJJ=nkpsi?rQ!KnYmw^AfAf$t3nRK-L5 zLb;=u-Z`NN9-&4E2=0GTvWbHdNEqIM-m&oFze@-$S?Ld~z!R6#_!=Urw&imUxe9&4Qbi<`n z!r+Dh*nxs;tgWoFHejy_`0Cn<#aSC&b=L%4*ADC!bd9w!*0sfM{eClZxiIYaJm2T@ z@W!3fbLLE)Gyh*MbpHwsmO|;e62ZT=m5Xae$sXbEG|o+{a}uiv(5tgs?<5w~GD0Q# z%k}wRfh(P>Rm+X7R;}`AU6qr6xI#d)Mrl8$4j(v-zHU_c`J&5~O!V{N8TGh8eSp5r zbYAgwH$FZ^dt?oap$!rGb}I_|-Pt9tyGo4TkMFX&G=wZ!=tFsQ+^jBe6kF6am7lTN zzVq8L!%n6c=JxmioFw+-7zXN2&-ZK0s4Pz7YRuqi24k&KiMFPnz3l0H4Yix{98BCT z*66Srw06SD&^h{lt8&(81$@jOYP)#}|60-06Y!i8;80H$%up|px6dp3$BOb*8Xx1a z%tA{F-~^i*h$e-x8U0I0@>GV0fa>bnCFvN1`BoBsto%Hkq$=4&$k@B0C6Ya7YhArqS z0CG30s&h4u)!%qE9kfP*mTc6~_I z0ncrD_V+joQbgl1lu<2I4nixX3QlA+KrbQdDig9)sU0}wgFS2R=m!Hngu!M1jT%@< zl45IfOKPQ7>rB;h9R|&pOaYL+E-sqWTDWe2RVQ>rqEpp)rWv6JRgld&FB>YgGr%mk z7c&+2ziw(s`7ujCmH7r*(h{{!cc0E3yWd58SPKc|D7iGJ5>;!B+LsTr*DEhgtp%UE z27d^XZ1rH0fVZuq@lke#SsUK*jK$uBw_;VETs*~wg2!{w_3zO2vd8vYA;eq$ z*4dVh4~X@k{6GY09tv{mYCR14Qs#E-zm(zT)^sF2)?fH0YgfG9v2-tNGz&J8r>|4L z38Fy45@|@Rq5^42@99eyLXvPemOR#8tQSZ;9`$TyXir)7)E->?I?8{hjsU_)tbF9a z^JbGkx>-|YL2GKMyp4ZlHuKw!o;~@eXiIQ&F{6F<-d42s3Pvyk2)cc8)+C621w`2a zNBO^1Id0o4A*IyUn$hmvYFA{fAr1WAjvo@B>sz;~QheIQkOFD>4Fe1M*iGeNVGj78 zmj61+gxAkdx=#ghfX`_V6M>d6+;T6LKB54Wej&QrM=tZthq10I({;9BxN;&FHe8XprUDe zj7r#J+;WBa4oENoi#9p9^nS-ceaZHlofpM#$AX+my{uWQJ||(nS{O?$Bmt&=WAFh) z^;LHC*9lE9hd&qJu>DzR`4EG(N}MsG#M5!!RO%d|r0n&Gh>SjUI|Z{Dn(5l61lm&c zQFAjITRXyvW;E7$jQjR_N1lV~P)N~5l47j2z|0-tqVR5(n%;Y0+7P}NtkDV!&VHE# z(|*$ZN@o)h|Cv#j<*VCwMQ}ZW2qkYaUfT(3nWr_;9JEDj*pc61`_*7ByY?Pz;g~uJ z8w0KBa#O9V@Phq^5zG#|iZ*y>L%J^_F)chOz3g0C6nur>)6X| zr}rMVt0iga=OvJecH&r-n=Pvdqv3^bB^eGEsoZ%NOx#xRqZi2JfebqsF8{YGCo@pM z#~;)LP|r3I?lk^coN%=4=jqpu05So}%6?%VDv(N5DmSWg9NO**)OE`{_sgFr!7?-d z)7cHfbV$x%vhFCB*<9)aM`3~3NYPf(1fjmPV}v;+t@XC2RxQed*lng;<7feu`NG_j z9?ZdhgHh;_Zc^`8h`0`1<@tDbypAJtq4={JPYz$O{m|g!-|Su*OUm&}%W?ud1w_?vc))U%n^i92%h=F9J10p^bw`|>#cWuU+;Yoc+WbvF#W z`Oby;qblC>nT#mH^D<7Liw6x6?aF^+-Kn5~+J?@a zP&;uI27c+g>u)^Ny+5F;%9g`*g0xfza+yzl*%9lF&VGYGnSaBa3jU-h)^+xwMPEL# zLZE*T>*dYui3vJ*<{ZzP;ZUon7(wQYVUQV(S*#Lav+?x3lY`0kkjmNc*3Sao`kP%i zwf?tUw`s+QRX@$t(=w)IKScQCHAg$M7bPa6ec3$GuGp{gqG^^vPP8sr=N`r%JGA-* z1|5Vj)=+W}%>sc0Zx0oY$lT-3-8o6NP>2Ozd12e&7Xi;B%*Q|N9LTIlH$>|GdvBaM zn0y1GJd$@T7Dxy^F}yFjJa}dq33v5_3(6CH^<@IIlo@ROzBn(wv^y~>(DWS~C>~`O zax^SjN#_nCpjn;5N?6NmkyC_~i0gXxvp?-G!Ucnd{p@!HW81Aj{M_S+DpbjS9#-3T z&TPng7(FL@=fuqnG@h;uvA3bPe)^vbz9Xlav2A&LeLziv zib;~WSMOj@VN`Zr<3eX!sXaOTFK^uI?FBqr?xsla-FD0H=5*tg3wFoY#Bq6U_G{PL z4DvTdM>UZ!f7!<{RkT0U(2Dkb@HVBfNos3zw%ZP=+i*7(H^9=x@gM_o&ufUyAKSav zIb?Y3xHlHA4XaKBZH{tCQpOlsQdtcj8)`90J+DCoI}U-Kp=^hWEgW#k7Pk} znrl5dlj!dDTaG~He8?<^(zTxiyy80zHUqy$*z+mMm`O&3&2hAK;hPS?&%%d(x~Fmz zZ9b>bw`oRXe+zq%vbu)mq-?3hI`uRrCbkFpu&`n1Qn*hUj&dU<3Egqtd=>7KdmC#B zn_}!d8GMsW|MR)ty)PXsiI2annkeFL1XGWzA) z6<1n+3=0r`=26-fgLZZ(1MtbaYGF+U+8`^sQXT6$-k#VF?ag9%JC(K#vLsd0KpU}I zXC_I!7!^_t9`*uujFY%dj~7VF6Sz&{w0d*^t>+EJ&xP?mj+D_o$edcA#Q_XAujNUM z8l%PnAZD$rCAB_@OFQ0nwH+esz~-XuMn6|~D!PdAzzCnbbjvpjeeVOa$}0!&Bmtk0 zqzx2gI-dfIuRrjgp|3IZ7-r~21HXeSGg4~~MMZnSfWCfJ2f?h~f)s4vOl9gw2dXnd zVMEy+F>Nvd+XX5%MZ*YF&|Pvg2_7ggGw)bf(z7I}&Gz-2UBiyk!}lS+v(YbaG4~9s z5OMrC3sdUx)zF&Wl&D>WEfwcrj(KQ2WVw zsC{{w0Eiju<3{^SARH5bTK|Q%6*FaIGxBYPopIDwj4GygXV;r;Ekjx^*b*L9)5 zEbm}nZY;qz_ZmF6aJi0ce55-M#-c5@6Wad#Rmb*~p&ga{j2n#XPOvm{jJEBnZ;QYP z*^6nWrKG6uCLKZM>0`bRa-b`hu}EiP?*6rLe!Zr2%_AF8P|P>YZv^D@ZTESBbn>! zeFDihTVY1ZpCX;9pe2S3)9>PfmDhTp;}<|&Ir`r{!jQtK=%}kJMgJY=#NpRj4L@ND zpFfPhMv^_Y9oNHKP?H`SB{#(edON2JXnSMX)J%epI#~%mw->^7On^o(aVfIAT?(OU zLmPRD{B~B4d2qe2fx^dreXpfU- zaHuT^>#Zatx>^Z|ix+iM?ap=D^?Lp2&j2=ph_ZO@r3|SSrL<8w(5SY$!P4#7mo0q| zTy`SL$(x#)xApKARP?WN}z-kl^M38fo$22 z2F7&GFgTGSk0S)f7yyCwU!9wO*|ddM86%oDm4rQjKtDJ|TbsI`j5DX)I~G>tb53LJ zHwde#zg)XNn-7d+oE^*a>{KgbGWc8NP1W100*vRf1)N83jfc^AJ5Xz{il9HQsoee8 z?+w)XMn6F9PjCV`ju*IaSRYzn8s|pEpA_bFbrmL8erh@Xj@rX$tfw)}v9b@NU!Btc4zix zVo1Hzw>`%SqOSKfcGS1CDuN^2^W1!85C)1J5L6O^L8l5fcKmdaBb@gg^eh!lAAsJF z*Mfg_;X((JBEa6dy12g=31Lk3ns?qc84N}tU?gui=JesliKKih^nlNka%H_sVWIy0 zjpxN$y0PV(FOBbPZ%bLFFqN_qOz3w90P_bh0QbC^FmabVswi=!zxy9_{;k2rbo;I< ziau^tyNlmn+ylzCpWxvKd;=(Rvb{Y`j|M${58!(b{;+r~M+&9n_Ler(YHggWXq>?s zCuea?9}4ZPwWUkuDBM>ggk-1GDP`qUNdx=dse`qJ~RmiCm<8zY0YO)J~` zZ=yQ_Nav~!)?%tLkw=mjcq6j z%M&i|=zs?Oj)KD}cvqB!&~l@`ixgDf4fl8P4}cVapz-}5w>9|yzKTDY%b)^dZ zsI4s+Y*w9WP5-6rCvnb3S#ONzokRLt;UHY@g+{2III$fO1CIU*VkR)Ue5G>aogUtd zihI?xqP}l5?kSAw-o8oS+XIRIBki$I#?|^|M5m_>8#XGsuQ)S}E>G8MG%7uHP`_S@ zB73i4MtXW$uVK_C3@*79BVuq5-!Fe+I?BWfH(ctd#BYf6uk3@!>nXxBgS~6fq;4lf zlUeY0IhwR~G9mj@mUXBAbC5+(z&;tg`jrD2JXbjz*}0mKL8+T3)q1XSkgR8D@Zao6~by*j|H)Cl%f z0%w2;&6#2BLz%Bs*3`YP%7b%yG3m5&FWJA>I2-K=F(E%YhvwAtrOMizQS*1sn?(j? zDi@>Y;W~E7|5%z)t1_G*F@OfAPwm!Wcg9RsPTz~tZ&=!KIvi}=>f3`YDW@;gnHj^u zB1un3D3vkKq))%NNSwGEI!1GNLG=NrV0-9r(!SmXJe z+!tM8c&Gn_@%1?RdeL6#OIhv?w)E_s#)E6-*CetQ<-Ce>G0Lb@!4t<@BQ>5>GF0u$ z$zJQ8Gc6co*P=h=KoJvfLT#2AdsBX}gB3;nYK&);hLv~FlaCmxn-e+X2(r6S-8m0} z3N6OHLQJ02j9Xeu%bx~2&@yz0u4iAPu}P8%jei|iQ^^#o*=6PucIFd@c`ZcpzVLSR z%~%ycPsgeqxUhLEe!F1@VGYpzau9gk*@VvhVC81luJ4ev)ZUGMO6xUzSjLc6Wbjn& z!g1f7d^>9#j&Q%RGZ=|YYB#9yT2-He)&Aaf%Vn?_0eeJ9VD&7RlFqNvIqmtsa`- z<&b(1dbhVkch5Ku+lq>p*L0)m?=fRAS8xw{GhhrvuZ{g-dkNubHiFAf>K{0manS$9 zv;*C@kXMdZk0(Ojk17=hOAfxR^nh9~@FyP~(PZ?`5LE-=l1bU{Wv8hov?3xTj9mTI zHmr9u4%)9`p*8}-%Z{*Wn$8hY7pb@-*u?CKX$pC6L7oIjY};8Tl&Xyir&a^B0fJk} z0Ii47pX+tSCLFdxfv+~`- zfy0)|vWQC-auEXz;`H8nceBQi0Q?9-%Kq=W1V$UAji9ApaHqrifd&{sNay8T_~umP zrLr+%Ki_6QcgQ!P!NXNSbgWJkk_*^2h2iY@`(gM-2(k%NlN^>et>g$w8uO#O6}`#P zxa)dxt#JrCdN}r_RSurlaE(3f(uSm^_eyJ+F+6=(`^?^{v^h}aEClR70+F(5p$YA4 zT+5GsAE8omn(MPxsQsbjHYh0vi0WH$sCYmLXRB{*N7;?w|3&{Rk610zq1VQ3aIu>; zz4$c5shw_l%PbYn!!8Qtt%%W3>vPQ2>>v#d|jqMr~?&QmKoz zJ7t8{GNUgBx`DTP#`ntRFu{44KwjGYuaq@Ec2s77ydK^$pO~bfk93VSGPFYP zj<@6AJOuj^bb!1!IQZOz68lDlQM{kE8Cm#gxO1P#`7Lt~MsXJAP?@p$OC9ZaL$w*@ zb~LeF(4#!)&oIP5QNGs!^fXer)0dMW)V5d416l^8FzSr!}KhwYkxEeiz(P{eQJ&+pofsd1l&34jNbQNB@gpwF!rlvKFtt zRcPVIVYfW0H-|kT_#U1iTPV=XR9NV9hE~Kl1p#OvRP$tT) z>PhPmBm;1!&zHNKQ$eN{H*Y^9TaY`%`mrDx)(%G=UPKamb5CLX)N^6A@sA7Jiu z;#2vWMr#R=z2c{9%&^qixcCR#Z?E9`r{VfdrI@)B$BdiDtJ*35Kvem#FK*ghnAZ_J zn@eDRF|@)(cob3-O`Z^KguL2+8Pa zbxey3VD<4>{>nk9hDt}@OjDhMns@p(pXE;919^EmNvtnG^Cy`&lkaj9DOahT#+3F=Q#qM4Gt65#^av$8Aus4;6P;|O5G(gz6Wk#i`7u< zik=QqTbm20?$cEfT&%%sVw^uiq(T3gTGqAHfxh%}q~p$7p+<;r`lsQ5XIE{qalW@eZlEOz*v%%K+K`rdPH%9xZ~S zeXy%m&0p&L>g_AQ>*s0Qgen_daF6BMO~{w8S9sq^TvIb$3qOARPK&5_$*QXJ26q=o zWfN<;Q*thj37MJy^cdI05svfFoS`|jHecy^N^kBZwr3;1kz zEv6Y4Cv)oRfoR1&if=+Oc5Xmx6r|JY~ z^zy+SZ47#-?O`<5U$G5Tj;`WjD1wAl(pWcW6#^zbg!#aZRZ zNByU*+5Tid50}sCwfc4w%1$?Rpe~+BPhmu>-AWr72qCtBuuOFPdOZ?|0^Wkbm+VWy zhfy--A%o)UMgj2LTkAvH@e4QI(ayJ$U&3^GASg%jFNF*q6ViNDEC((1I{0UE@GkjJ z2<`oc9@Ly>&d2c{<4v|obhBccV9Q2yo`cqcS}C=zT$dB(ud@Eew$U3-Ew8#R?hrVN zU)1K5oMq}qnYrk-6(Blc_{ftN(E}fG@FMq`^GQ1Au5zO50cw0~{*O>|N>^ekUCAU# zUyLY~d{zL&q&hj%zQ>`a^evzq;8{LK%fkx-z-NUf-1r?3*4u*G$@R33;6uYrY=;{+bNi*?=YM<*(U$H51e`ki-{6Wy?c5b zV~X-q+ta9AZLpFF@#D_Xd%FN>9T>@z+}wXFA)U1ez^Mi_=14tWT6G$LZViU11bP{ zDVIlKm22jYlXHpgm-nW0d%4<&x-VBb_-)2wpz*5e)rv9Kg!Hc(R*#vQ(Si0c;dDI` znzEC)v3=8CvvZTb;mXYnP4eBny$wn_t#?U@2v3x0oGw+J>KgdMoND_IiB;9V)EuQRg5 zevLDwo@+3bF{I8m;b|=E`ySRhKS<*4t09mSv&~x3z90+|Ch&?^WBPwYcd&Fhd1U=t zo58zLSu|41%3Qo0=kM#sUdy1<{%1 zaS@!!@rePqz)Q|x|hP5nwEfffp3IL@;2$MCynBv>L?Im^7i@VOn-I*-2^s~=jAm&R-!*@JyxGg z%!Y$+thNVzcAyq6(cj;f(GQ}p&%&^7+=3a2(T{7q-IV2_IRRe2?Fb#9r*CCZZjW?Y zu{+L%W3HL-V$wTcb%OOks(dX|hV&beIxMYa+9*nFsS--H8>mPiD0CPx zPhRa94b?L>H&D~rmOmRQ|Jt=Kfa*^0#f(Jn$_)RhTrS z-N2zEVrzD+9iu5z8fO7uvtH$Ckj2{FD6<4Lp50gLM%$feV@(9idIxf~lZ1zbxCCuE4pPB=)vm*_yy*Y9U2dVi&Pkb z)kBFI`G*PHuxV3n#eY2Cq0!Z3MXC`2LskkY0TqavENh;_BQ82uu=WmX_3e#$c~$SW zP5h?7to!gMhoGzL1!#7Z%9hI3AqAk%ZY(XB9$Dw=k4!|LG(((`kERcA(Ca}7F{)5; zT{L16VQc z{OyzAco7~ZudFh*%19n`i{+*Puux_Sf27-#dce#>@MLK^o0g^)A1s~v(_q9Qn%X3bb zBbAU&*HQa(vqlHCsp$+@R&XVU@7c#H!7tB71)JJw4iXtK2I9MhvY=U|E+FpE5$%>AH8vPTXWYPUKr zMZ$U=lQXl?Y4lt_rU6#gc!y03i{m}Sq*k*}jmD7U40yP+V<+%EOrk~A5vZHbvx z?!Nz3qc+hv4K3Yj;6REU;f`e25E;A-sddj?LtaD3%`6Z}eK*=xASEn8O=Nk&5-Q4b zbX88(uEtFr=vl4W!iH6sVAm9t*p&-vN?9#YFyymCS1{x>X1%{T`885I8x}-z_`Cpb zYwFfWR}YnUtSq1ef<1yQDd;<+}>Kfd1 zo2+gx4upiy@F$P9X^l82SE#2&keN|puK|?P7`sNEvcH~HZ8{El%kvzoc< zIz~`OEF&68q76^hf$F-?i5?x-6>~)Oo8em@$s%3R;quTs^qoMe=)QVmQ3S;;tNnT>dvd{%C#H z6`Nf|N%;_G(HH@me+UQ4Wu*o-A{Cb@9pjy(d;t2cSnbGj;E{QHB;JW;wDvHT;tc8Q zmifhOM*sMquxP(nCsYE;_+n0?jF0zUop?+PI4I!)XU3~)b6?o9uValckO>3-W|D3W z`2rN<1qa%!xXXlD@QNB3R6Qli z$9@%*%eKVfK?1MEKxGdt}m2T*v^m?M~*McX-8%y4gc8<0_m z;X+Y)5ehNavJ#8oYd%`C6;Ucis2nxh9T1pBM^y2m)+7EPq{%%ZeojbKh10Tw*flU_ zYsTg#KZ5aoMCW2$=>M(CIe1jS7ltZ=DA&)>l*X$t*D~IZ+Gf|RgP~W76FE8VdL7qG z9YNOt!~SDgm45Yem6V7q zLADXM*rTJcX2zv*-U@q)Ii_+qW-C*w*A@Zk>SflZt9s~pQDa3hsjj-Qbq-KlFCDsJ z0gi5%6=iGU)h8}t%2)J*q7yK)zYp@I9{Ug*IQs}YiyZjWwNqPi>9(2-xf@dVWc!~HC70jbu1-1^mRfm9M;(^Go416hG@%TMGIXYRpJOeOj+fEOGT!s0M(uW8Mh|_RI$p?ggy;Y zR4QcnZowFJZ}g%qW*RvLxjYw0Wyg?3nxL*>MthR5cH<^$!!rGEz7PO04I`~tg0>wg z2U>WE+6oSpRg8{0^QIDSRA9K1?P-h)o<2YBGVUx5j4W89tUuv9Af~joCC+rHbIiN$y5}t@0N= zIj1!wC{R|kdDEi{<=*lIwHZc>t5-BylFbkwCy}+qJL73Fa(k{sSCPVwrgqUd3I@d` zfG~@#pkYg{=i8J_> zG4QzbAA?^t+JaEOE1>F>W&1NXVpE~~o)nf;Fkk1DsHj0@NoEa{DWzf*@t*bQj;_%z zwxAqG-&neG11j%*3WsDWPa2vv*AJtj0qoRDa!BuG7A&D%MT#R>NwPyHF4jx0&R(tI zy(zGh95;8au%H#)HHn6SwZ&l$XI^$4J)5;k$GrXxWWTWATlS7RdKi100IZ4R<%Mz` z=bK3Ndup6G?64+_2AzP6n-HoSN-}o-L(s)@M!hn6rwt(kHr?kWBiWNxBSWfI zxXj;Pp_d)moUzcWlJ_{JUEt56N~mN z2c(uuRjf(JbXd7JaY{XAD+--hu_W)2JQNrhwLbSBve8FTjWw{1N|X+?w`{k(FC@Cb ztSw0te&Ii;n*veu#|_@*ItGK{5{HPT<=5kc|K)f0{g>siv66sp5;fzi6$VjIcV)0E zUBOz8>-Vs)S%Y}2n>P4V?WQr;MQ_u$3&@gVuB3XAdtyAOv&Vzv)o{#19f!<28aJA4 zk2faSa|4`1oz@?wuY;$eys7^?xszPK_v#3sV-3~e9sa5 z$Zcf}T{+TRAT*~A7LDMG-@$j~8nvq*D6>7X?F+~Vq5G|c2`O)@{X~hzwD#B#Py50E z0_kLXtsl>He!lhd+$K=F8Vn|fn`+-IDDjahlx96cz7^ZR*EDHtbr@>D27F^l(BF+M zSuq4I>Uy!9)Ox7ZV7d7m%l?4Rg6fO|0*|~2yJcp{Dm-{ygDqt?wKS#Wx6vhxSFPyu zmb0PX6Ghd{L^=KNJF&Spij=TY{E*B97XJ}7cjRoD)4eU7x@ymX}|_Yk=**5gWr)5N8| zo)u*X_xySl&V^y4FjL5fWrG_EE715}*kExB6y1OY`Zm{@?sG@2746s9qvi)MVhL6= z8LQSw6)Jde!GAbg{a`7C9)vkVw$I=Wmb|{>TkY!p{B06(2-%s{Z|vx0$?7zs=-Xyr zv4rJ*Gc!4kMWEA2?@+iM|vf@&umSr#>MLInq7 zvgoG?5Y-})J)ro^pDQ<}_fUC>#+ZKa@ZD2%jp_X~ft<^tCJtRXgNqh^&`27V@1|AA zrl6nu1Upgv!^qxZ9ufFfd)^I;l4br?FK!OaLf>`Px|8QvO9%d}2d|%~yZc1FhhWnc zY~;SHxx$hagm@XX>y=4<{nUOOzUA!;)mkB>F%!${-Epfqd=Mo>qfCW8l*SRNeKYTO z48)T#9m|0t>^F|!O}s{Apj%DH3aq1pUFTcFzU`D%*>u>V!$R2!5;h;jIfx>Wb&p}= z1G{uGJb8@Cml-5<)4c1J&@Tm}RPN%GJ;KpXJ@Q2^J09WTbsz2Ly$=`)cw_jp48LbT z4454pLL&yKJveM;*beY+hJV33VF?pRImeM=0ky zqE#FG9yn&%h|6He{4h-F>+F;0>qKKE0ar;KInN>(5Q+Ez65a z!u(PInml^*AyX(c9tOaQMgMPAj?+C$T6Emao3aWJIzP<7tj3coPW6T*Gyb&{6&^Ne zSog5s6F2M%d<68K=zW>c%BPmB!UHK}JK>8ghmL261S--}R*~{4me{wS=^*w$Qz@t` z9&F+yjY_W1`T)J#J5@auyz#OELBq=y_LL~i=-gA3Ej0RsF+U(AfNY*&1;Vl&=Q&5L zS_QGfpr@B!Lv|)$iKmml$4d{PAY~XWDLNg6cyzidWpEP zdR#p^m@?9|E=E_}{-e~H4XElXd0(F7T>2iwZXp93#U&BcWng(q@wzgs_8vs>-+8T# zbtMVLy+GqblpvMYy;zZE7|sZ|ISK~Af%?Z;>B~f{@jMomwXE@ItLwk9{D@s_ij%r3 zpiUJ@t*G^j;L61%g9n1{>Uugevw{}EuiHylZ9sw*^9oN|@KWu`Sq%)>np_5}-$gu> z*WJoS95#Sv4^xLz@(`^~@B_m^C^yN9ZVp%1;YBD%+Sec2C;)q_mQ@Fudt0&}j;h$! z>3EBUX*Om5gc*n7RL%fzENQlD1Os)a!hXn)vYDl{q~~wSt(-F&=W{P6Rt2~@8h1R5 z>gA6ge|wBTSPv6*TZ!uSDFUfzj3$ok-jy4Fa|X^w0_KXgl(Cv1+P}P}0!46E)|Ikv z_hbSpb-tcL5M{o{kygxLm0-K>z3b?7j7rxURoB9=Ng$QxC~MHIemc`FSb=5etqr0H zdPdXOZ#oUe9KO-|93=0L0oqa0;FkhDDG!RuRfbT(M{K;g5AFYm1DxlD48Vg1&QUsw zBtzELA-i)AmTZuTQ5-H24Ecj%Q2;>MUvaed(@-SFF!(lqKIu9V%vqI`>~fRwZZ7MX zvBtRB+%`b#!r^ak>#*owz#l_2k?D8dUWtC*CmsE|$9;R10)9Q<|CbSWdaSh=;`(QtsU8ma&lYj9b;*u#HzLAt}z)v?rS)U3p&mzu6rqeGGD9 z>)sJS%g3mLgzL2V2G?=sMvOg|HLmW*j zlI(joeZ1|If3<4%*q6$SB7=7bo4T?PBvZY%5?H*JutUr5FHI%iQAoM0;Xx}#sXR&X z&B2M1y{r|a-H-X3=}4a@BBTWS3IG%&xbb&`d|BJ)Hk)HMV_OuBSrQ{#|8G@J9OaL~ z8&;X)kXg~ySS38}pTLG&-$MVlFqxd~)ihF&FB^?#j2Pvd8c!Q4;hb7{oxE;Z9aTwG zeboR*vblF|6?*{hZ`9t&W(bL8NI|r|J5uMzKSNFcW0L9dH$VTLho~;YPf{~@Y*@6` zHl#2{@*so%HELQj1ITX%!Q@NU4|Rl4MP}L>!fJcR!)yA~6F;M(B9QkMCv#@bt$Q{g zaR9o>2OKL}=-GPF^DK3BK`Uu8dO(8&ch*J!o2-fCWZwL~I%~L5?jy=PNBl_DnE)DUp|9WABL65hmO{)_Y zLw?LcGi1f{RcO*Y*znR2T^lRri0}x-i7KnmU;xdE7ay$)q1QlY*_B@)`->4Q&-C{o z-^m#N45`+4tH(J3X(f=bbk_e{l`|7bVGIdX6(QUxh3i^2W;L~@h-b1xfBH#)CY*2# zpj(Tuj$#I(FAi{^!C9g$dMYx05**bIJOO*EK};UmF3`9b{*f)(yR#~ZvglAe8JVM# zEoR8oiQ6EN5v&xWmyGrK?g;`ZdxF-2=g_qoo;Ey<+AP6aTdW?U<(OagJqFe_*j7Gj zD*hkGjKwWjHk!_39S{OY8=Yd;sA5a|I4+C`@TnHl7s9o}u#Rhz4rE zW_b|rBK|@oF;`5fKm;=@`o$GXBgX9hIv0mC5U>N7$zgZK)C!!wIa*=*FFB8h70i8y zC>PAcKiFaoXK04j9J+ z?5lFRKzi|bnUW)IG=#MnQs%Ya`m#)X_LQAGw3_5IB;C+DO}IVb1tigP)E)-KRPsOO za|@A1pfq+e!=ee2GkONa)>n>0@&=$M2a_LG>xD3*nc@C;USkscgQ=T8#;);KAU%P) z@|JYmZvu&zlH^-DIgvU*Hzv=H+xCh*f^Z7Qdvd%zzg8gedXlq?usnP2suo2kdo<41*WKI6j*7d&GORGga{QXOc39wi#8{JK|J{=u3571MR6HIfcDy)-G0JSJnagq# zk|3BIGUweDjNsci54FItWTz(4c0BEnA=1}5SluuCM*vlys|pbjYJfKu=|A5WKcl1s z3HKeMt!SM$<^hKH*PqKWTEVPM(GxO5lLwWMP&b0Tj2B)uWJs?Jz1vdR6qJ+|JkkTX z(4&;v_OxNDF1F?6nvS>&%54!)Wy)8d>5)uCMam$;B9>83HCS7~Dw*0t964%uyc9?! zBTemD6$wgbJ7I!|SK!Qvm2zZtpV)3GSee*WjSB8>tSPqG6qAFm+9~g-4YKg^5x^M| zy$HZWs`zY82WD#V_RT_-lPK|+$tpOuSECn;FuwNY^A2SD;r$w${23;5Fks8{JR7e0Cgj}P*~icKkhzl}v#cZ1;aBC?|yLvm;3 z5f=1zA=U`2dSh00+lR@3KZW?xKw|R5lEGs+VHIFQPXaU!c|+AMyqdzfdYjt^A=q;8 zkbQlN3tXL#VSW(NyQo%m>RJ*1h6$Y0A*)flaGFN@PvM6FxDDN`dw*egEKA< zN?8f`qE)cGO}NIHBV{db{_-^zO=~b2$d{*mk_3|Pk~nYuM|b!M+0mB1PQQZ@dmK|9 zn@4(}q;?9v)YE-mm@Tb(u^A6lFwCj7roUPV+i!4PZtgd!pWgSj5j=cuPF}wT@GrgW)ne3q_xCv&B)=S$N zPC;*6cdr^^R}$AWj)UEbsPm>D#NCQv)<_aJC!b5?t^9wa>&NozWNxfYNw4ahgV*L^ zQP4yp*7a9GETEzYAuu^JExql~ffc{BLRSsxmr4ho$2nRs6YS2b)83nkf5*cwD}LwJ z#y)f@@>!hM;#QMEnA+ymKLJkei1G4n#&44i#ja6@QE5Lf1$i&UxgWP#Y#3~oQowEI zs;{)MrkE{QHFLW!W)RP}aCCezB+J&GO2fd9p${xm-#-LD-wq#=15EGjIss2Pge%3} z0HfyIX8QdxunL#jiNC{BXLW(Vx1GK%EkY;MV)d(DxT}2oE|Fp^l?le;{Yuw-|E-huBmT zs?|BCgTUbSroJJu5&XFe0Wq!?XdL`YCtqPLjhEnF!hZL=2+SBwAer6@z{9Ah^WciM zTWCl>4D7^JjY#;|SQByWj|UYHw$!Wx9)n_+e0^?2 zKAeHLP!rKrJ_~sD*oI#Gf#=bpqEx;d_RhXfg9f6L+2d!;q)z_#r4ClP4j;lxS{tYP z5YM%>(C5UPipCHN#?<54WRK0%W)iLt%)eALT2Nu%aO+AR>-nmIaRa_`2&Jfb$X$oK zVp}Diw9e9h)90)jz1QcP4lI8gqS7e+lW}!5SRG{>(u%d%23FKg`1;r(PgH{gJa(O= zSe@5(7BpJ}&E$i%$9QcFn5=e$6;RmGBaPNw$toF|4|x`I7B~li(^Rs?RWrej*G$A{ z@pO8!RvRWN8OHKT2CGT{xxHdtD}7!jYAY6bldUTO^xLWm63EN;Vn)UF7}teUzp@vn z5KM!6KkPZdO0L;%prItqT6`jr>0Yui&N=U!moX*vLxOz8S7%~2BK0raNe|>l*&Fd3 zE+hKLe!~f8IERI8)w#NcJBQa-FyKoy&NQQ@?v?1ZX?FdiF~alU{LLj{cH#U=m{U|f zBu@?xrkI+TJs3agydMho3@C?z$+x4}W$AuholD%^pU$6$?_b6_mwbeKJ3_*-q4@LXj zAqt8h7lZfwI(zq07@xgE6C%NPNz}vJlEnc$?8?%Thwf=Q;wfl9hmByY=G8gD=q$&#=&u1$}|DG95Y>)PfwQ!<(3(!LAdVb;cmOX7Qh>*Ha+D zS`KtXjpI!Qe|hk;f7r*&au$OQ`n=cM={&<=eJeh z6b5Bv3)C8;=fo}G&zVeV*=%Pv;5z`mgT$mhUX5TD<4jP6f;Dp^t@!xO)EMo(KLBir zo|2Cm8vIWn!2)>AxG~<5Xa3Xi`CrAX?(`Qhl4s=369s%iH+%$S=LwwpG59~DE+pLn z?Y8juj*?J^@W6zw8=Hy*f1Z5Z^8lPUVk&P#%&=pr?m;|H{P^t#D_4MZCc=z76oRMg zSo;_TQ%R@-<;Qp6MlAU|ZESLlBJi&b{_>)(KHiUD{1bc(18G1KzPZ8p4-Ym?a07$4 z*xR?1Z2-vmrTNt0>Qn7ZY;MaZqlyOi(VZ;e&*6jY7j86jq+YJpXF)hCa(dJ`SBOA4*-x zmyxV!Niq&xSf+6IW513Igp#LlvMG1sZ9E^rm5eg7qZX|#?CIez@L{&1m}@=j&Qn;W zOIX#2+IxK^q%-Z1XSlB!+tLDQ%cqu;hJw>GAj+K9?yZEREMfI2Ep@LyJDl3-wg7#$ z1f4CP{*5`T(`mS&OWtwRHD-Q~ava9!yNAuK?Qz&$8$nebEW?iKaU75Y0O{gS#kq2o zGB<{fWOr%Du%eNVKp&R~q?pT?R6b`aJSyJ*c2};SZN)|lq?7Lg;x6ftR5bo<9CqoD zta$ogzikJ6A$m!c?DS;;pTLY=bQbSQ-zki9^kqF{6>_RSdxI$+(#eZtd~#|d+w`_p zIg_Fl3Scd}`nb?@7dw%Z#{1W~?QfT>gB@F=w3eJWO}W-P&5sq$~lkKzUb-MP-YiRq~PV5j76GDHM0K$=OHP{+IaEvz>5eQ zpUw0?+t8+qaXviJ`{cLC-WVMPupXk>KKu8AzcV7Vye(RcFBLOB z`Im!yMD`vV2{fh?blUA9+6p_V>j@dAUBk~Cy>u+v7%8aunp!9JlQs~003Kmsn^saR!q?9ujwZ#`Sm^KUJBRZr*{UiQW zEA*A$38c~zB#Asnzfk=RcqNX#q%Zn0s}%mC=pmqZd5m2QGHm%{w=97;?=FVb$6lg6 z<{2i3o->WC>Bd%F7l{1d{p~SB1PstYeZ&Lt4F=+?87!fgygH!mTe@gdmSJM-)d62t zWsBV-_5SI85IvCTRCVIGIb27}(9)VR9$>vRpf)Zh7}D14whQ*asWM?&c}-`LQwizB zow!gAzi=J}c%bVB!a#D@)%c%)S7`8E_U@i;x-U%dci6M>4!hy*X#yYx|A&e@B3F_x z`*(FhTjDIyR$9;3ff~%kJt%K0{sHzhUa4>MsStcgRPn4F2+qD=)==u&Z%A2TV-kWy z@&|Xh;Hpm-cb)_MJHX3=zu2UQ$9H6ds@u{B&7`|<=V#RwK$yq??)W84LcC|Q{e zAN2$+==Fm*U*5&38xA?Kj}`ua&7C~kwl}16D72({{qf}6vdPtPqfir>N#GInlsv_2{WJiWx2;LTnm@BE?$+u&Q7KZTyTi3&){X4r6YT-3XL9kMeIr;c3L(c+@+& zcloKbUk~&(&cy+Ry>$<57=N^V`GU_w>e1~Ucz3R=zi2CLX^aoDKEzxfRojFbY|ZmmM(;O=J`w&}}JkN2<;$NKCd?bop*0{X+=uHjX$ zOfT2dVfDz=SkOQ`<}MOgZvKFSvN^33bUOo2o-O>f{Qci(mIa%b45XU1lr*rHR=n9h z^~j|uYbf(qoONqb@CbqKb#QB{l4=^?8|Zc0C|Ig#%}V*RU!LLgZ+%;ZNUmjxojk3| z`sR+y>}I$%{Hv*?ZF+|aE}oR~67vJg_n}iPF?MMu1xt+O&yU;DwZV8;pEa)kbbtQ? zP-CC`kyqlwy6QM1*VdMD`@k7Ewb>)sQErdQtl0bLqH#bPbjj(Z&}YC_(w1jyHH%2{ zT}B0EcL@ti}-()nis;EfmX7bJ0LKUhKMUaEC@Pkt=t_0$kuS7!ygo6$*@X3i|I_{q!C zc;K^a1bJ}(Gz>KGt~%}eQ5z}@QalpYavP!hUAf)ZmLciC@ zwCnUHE|haIPRz0V3=W4859Nt&-)PuvfHnw4PjR|j-zE58Mjqqw`^K6lbLlc?1vl;GESSMK6juc7zOS-i0Zm5fI6QK!CIFHWuYvy;Pa zLvJ?ywv(t$nX4ebH|kWbP#$u3jULa}ktpn=b)sJQNhmw+EpB2u=J7@ixleZRq-Sq& zJkO-vojR*!DroPAV|SM5-&x8L>QUA^b+{mOZXCFrT7lokYT8hYcZeCK3@JBhtB)|p z?O!=kE1G6%V#SkuIhT4qFXyuCs4K8b7m17S8XcFy@tO$1Ka3e=?Biqk$;tK&ei>9_$!br?eawPBb2k#zt+V?pwjEgd(!?Liu zfV&Q*<$3baRvm2V0pxe)j@Av<+n*C_u$*01p(rOqMQ&G!r_+}i>M}Kpf8M;BRoSu6 zt;k+#uoFUmL5<(;f!Yu*$mJE9?>_?WI0TUc=-gc#Hkg`yMm-?Y``lEoRb2tw=RY#s z-=mY^YdLI==b9mGbr}mEZY0?{e;=pYhLao}*DcoyE-<%F&r2kFuyF# z>sttbtfmAN0ay>9B7lojui;JW5BA8(3#)qNe)KL!2;&b3aoB73FO576bGbn*rQU2T zf6w_XhxMfKNZRDE_cRaAu*b|#;}0t_{l8T?(eHR9_M%%OF_veVAY!m7ywz_BTl!(Y zHWtwGLGAAM3fk;q>O>Yf>KG1N-DmQ?%^>Lt;_{)&R3m%x`&o-(|A`@yG<~qv+MMNP zeoQ%DMu|BnqF-ZdPpMY;au|DU$d9#U7Y!=w1e0AusA{2?lW8@;jKhYAm#uxWRbj>l zfj}*@5%}s_+2cVsjXV9))7qM5WG+Oa*#PxM+f`te|YhzWTMV+v#)>YUx3N_zJJx-R&iY& z!v6dC7HY~&U47La?`o@9-Fksr)zI{?aYeT|QU2gsI7V>5=wr!!lr#zRm{~GNUiGU@ z%O)X!vfo{9WNO&JW5XEg>Hn?D2~M%c9kh*-)~?1}$efGPMSEjA((H7|HT|#))ZYPTeU?;blG&= zS8>75GN#+R*wX>~Xm={U_`vNUs!Wwz$& ztG5_v^)jmlmo2@J_O#NvHkGcx66yCQ_}*oP<&jKCbLv0RSWX8AD^*Rhn5}W-u<2Vi zEoS8ep|FlD`R$oH*djZ;WX`tiCepiHosxxx9sM_Bieq)FvcC-Y1CD2GqoS*m zVQ419H-`@n$f2v|33z!>W$)0*d&Lxc53a|f$%reTcIjjboPv)Lvfou5^IX}|A_nrw z?$&r;h4Jv24O4u9iy`dy+cUzP4opQ(!OS9V_Bu4c$-}-*SkrQMoiN9uI~H96ZC3Ls zPc2!0>jb~Az&8)MLSN@%>Rx-*p3*01J$Yah0ikFrZ2kK+d%7`IRgcRN%vZ4*it9kpWg{*+p_n3m7L=zV?Hp3X&j#Bl5L$?5rk(8EnZrw-_ZO8Ol9 zhmS2sx@3@~e2CZMUwbOEs_jI_gF|g8f4b ziO;ro`5VlpLIZgLaq%6P=4joh{!F~j!$&GZc-Y79FLnaP?wMo+%MTR@R;0e-XhVfg zC_-jf69-K^!}8^W;ZX9}^!i+Z70x@RV@&cKf7oCkCOtW*Bz@J{!P-gV&0)JOiu-dA zr1=+g<<;*+69+aaR2PAbWtHDpS!2U~$iO~=CvP~Hm^;v%Y^)+uT(F~B88pC&lAr=9AH>ok36*Rg99Fgf7NqBBO4u7%xt`G zaVsnu$ynvQ*5AGJ`V~534EO~}9&o`^hn3qp)P~mntQ#5+*42N_b|6PEmSj_TcsmG} zDO?Csu{p8rh*y8i1K|Tu1dBiYzg0Qi{2b`qBoxx8I3Ue?xeK0>VF+^v99en+y&M7J zGD6dU$_NGB@qT@S2)?V3>!D5XW9j1y8-DFv!w&U8&{JOS znua^j3lBVDl`z`FhU%N*G@O43VfuVbAB%8$e{Pt+KRxS?Z+j$>hq)Mt|A(*lfU5d< z-iHP0z4zWb2vROK5V0YO-6#rZu%n12cEzu~VpP^HDt1NG#I9gsZ`fkT9%Dfvw%GYU zvwOL)_DiO3|ua^7RI0%+&^O?kBCl_)i_N4I$3UcVwsjNc+PL^y0^D{ZHm7pMGn=AD z>kQ0UUwee~CJYT{=%3-(W+wD~2;Y@;aA~d;f;2j!H4Z!gtzA7oDkxLN;r1)@42Vo84Xcqc}(ow zSy=)e*xRt38>LB|my}^@OE6^l4M&UbG_+-4Z-jxPxFYh7pl}3??Go+E z^iskRvHdqg1D+6pYcRv=>%AfHW@Bg}j{xq!RCA$v)6tDdg@5~8NIeHz%tQOjUix1) zXNcuw{v9V>rv+RySV7(ddrakIE^L-H9?*qYV`WV4_ww^KML&$(8(nJ;^|z;U21j;h zK)7$U=ILE_n#}>j68qZq{VfNHC#|u~zO^*UpKDW|k=o!MXn)1YkzC7d%egIGS&Un> z2d6FWH}_yJcBnnDF_twi&*!u*?1591w`h;EbFf%=wM(=|?p*96s4l&qJKg#XHliT< z@|DPv-^}=OT9ZdfXQt+ubM(+r#}$U#W})GRqo>H*;yruKSWJTieMSm=pd zXH>%Mtux0G)@nN#Z*#fcY~nc|ZN2~_J<7`J z#`nRvJ`z14UQ&F^U8NqG`N76)%VR^p>s&EfkurPCUe&vXMq7wkO|I^whc&Cq)4mde zliBz13+NCNFv>TTiaGXA4F+`&<&$+%Ew<=hMqN~60Qh9r{r(!$`Jr$Gw_2|V;O5-A zExlzCXtQ7;*>(H9f!~#AHDE{fVVELaZdezE)5C#(Q0-bK`k>UMO+J?Uq{APXXO0m{ zBM40;kFwUEYMSQ;-W72Mo8c92&p_}**i-HR3%+tiY}nF#g)=+60K0T5;qu8AzVsHz z5Ve$Vo&QiXC(W(difdZFl#S4UGqB(7d(s2PGN>e1^_+%aut>^E@0*^LmHx9bX-KCr zndvO{9b!6CiPz4A1kzNu0k)DG@1Bl18+O6R$trf_i2g&f28;j{epEV=k917W8igQn zMJw@InoJn~ah&}cco_|avWSnE+3GC`c4GSb!>w4sb3DB4M!ThO$^HnU4?u~x3Xl#F1Pcj*)u34 zH{!2OFzICr#D4Hlcxn(WJBN%dbxnT`Pv--CWKZ9$YdTeuc? zl9RUr#iHYi5t$g?zls=k3H>w`YiRj?@h_~3P2T#rFy@50i)f=-LBe4?E>!|rbX>?hpIBOKYwgKF>2SQCBoLGBG_@W1sZ?MV z49co)hZ*C|*?rRxHx`l`?Z!(EkGRlJS+XZnmb|J-2=XKQ!=p%A_O*srD5)Af377XU z6-%qF>dR>l{BKO_5t!cU!kY5hdERQJ7nqy{R_>!SmbWCFbzhGesoQpC$#g-<1r5}{ z+OG(8wt}_2CBOG9Pzw6y5~J6wSrm?mTyNhNoT@9H?e81t=9t)Wgvpm6yNwaHGzN za$_YW=lZJXeQG2L8nhm=90SC+^AnA%*g1;&N_WWRkK4F!hm7y|C(n`|ffj^a3u0?F zz&J^u?L1@Rwe4t)pWs&V=n{&At-u>`z>qbE({SP8K|2zOSY7aI@cFB7Sz#0}BrX~> zRgaxpg+%~;D?V}moei9ba4fPg&E{N~D|^?{(iZ8L;KNkMf8!f0D~DZ^u}YIqR;DI! z2FFD@t?jB2ZzpD{UAG4PBD8;k)D}0AEC|8Zfc4siEg{KvJGOmR8_WS{vL3-(io7<%M~cWIIB#k&#O0y}U>;I5iZ2K|}SIpZ%4W*y0)H?7{?~(BMMmi zv7&B^hUU8P{8SobXqJZLOLe0sDA`Zk!6n568w^SGvEB}imAYnl0_IhFL_9G`$&hqU z+eiQ?skAd?Qyw>lRac?|$i2(?SHXbruzk};4j#cK)&p#n3Ta?h)N36o^%Z=w2No?A z9>`L6MR{=x#~OG~91YLg8&rGobYi8FcK$yAit*FtOAx*wvFTeQG~bV@ah7jF7gpiA zkQ>L9$247I`=)7gv^AZuq7c%g=!oB1(aPR)sWx zw94avLDlnUSObnVWItx4tdkb3{WFIUk;5XHmTBkc_sKVu5|OUPmN_558&Lk%u@V@r zx=FV!IE-&hBV&7x;pfj8zt^3xEY2xwET!O)|qX~9#}bVxl9 zR(Mnq%!RpzB@Utu(GV0ax5AC*O4^lb0F4}Jh+1+%7aGC@>t2gm_Rz9IG(#j-65m!e z!nd3MVLYLrw7rXQs!u$_6i0IMQ8NcPk33E-{LD>)AYT*e>9f3UK?w;D_?mewM#O4(7oV{@}t2}JxnKV%SExmGD_Zcw@Fei)FjV6l>{ z-|<|YbcGiB3;;4`HW*2&xlz<^pXx!UyL3+?kG*%D1QSndk{S(ku}K_4w~-}d0!?-4 z)!%I_CgoGbdM<*A(z56!rj3qE%S`JtU<6zHF~Wvdc$eouZvuAE>2)(n$u6EMCCwPp zw0QHPiniT6_yku8Q{WczYuc+`)kUci_-L~`fW`@jyW9XTDcu?iDxMn%8DNZ=@oDrt z?vZva@CxyMQ$bL;7a-%oEuuvtoufrxa?`L#Zv`DYNNVAext1p7K*qIY2hdWg)RA?j7RY3>lzi?#@I%m0Jd%%mIAo0zaBe<4CnP77U37cG=`H_2rka~P zU)`S-T~g9}oanE$%@bknFi^|%2X$C>LsDQKjN5p*wU&TJQXy~Vo=W}%f2BRTk5r?- zf&>%Cq&R<4H(*<@DzRauv!9=N3J32A?c_~8&Lu@jaRE*oanH^6*Txfy>)3KeLp!;F zG9mxx+IoT>hY zzsF6*9Jn6qrsiO*`peHg-cv9U{D|fBK_7UL6lumWXWjw*D*LmcDZrc^9$uSj)q77| z!`;Q;Mf%?SFy{c}oj1mnLdz=Q_-}sp^3j4503v6W`PoyS89e~pk(hd>w_SP|M3e8~`8dn(pOWS+d--5(>dF3SJGrgfGHrm8mc z;Pj^--I?7Og?z&tDIWst*ieHW$S+Ht)q^wTw;Ee_9Znagy4Bvh{ahfJc>hVc9_g%+ zf$^R-x(TcIDHh*==Mtb&@h#~-9J=Els&ATv^)eIv=vOc+TvK~JfL+VcS>?sXh(DD~ zpH1|c$;MD)7GSK4tcWzw5$Cc`iRru15^d0)ky48kovgtbv=FH!48xs`*!|8pmmyAe zlmKYgPt~BxJQ83daL;^@|8^TSntVYJBt1cnh(r3Omi%2I$6x$!Pu z6-(WF60At7AT<89CkJzNh8z~{lqtp zV>_P`>BwbWyd5#@4E%&b7RIV1(WK!TRD(W#r>QGHP~h@jk-VQm#P^DS038fW#n{BZ zoqG<;#)k55xi6ypxJ52M)d)1ju=qPPmUmz)>I*6N!hkgz@B+IAf`aLUk6ii=8g<47 zKtB00Y9#1^MbqHvB+8#fcnGXouQX`YH&*psm#s`gz9EidOMDBT>m8^1rp?w&2{hnH zbwMz2APACL)*#r3U$`>cPri+v3>W+z%_EPP&04AGG0dufZdSpO_CjhKWRg%1ZX>8d z4w)kS&r7Ub`GL*-K`60Td+`lOHB;s}SmDLZWo`Q9$sRa7t=vMTMj6#b(CeSW17S-- zsZlbBqORS0`#{57@U)faXZ2v&UC^JZP;J^K9&H8;NpG2+hD202anDLaLU{xQZx1P9 zv+u2;7+E$ybmYKZRH`}4c^P5PlN@pSOPiez=79de5A;Q8HRvzCM%t6vYcVEMA#4J{ z;tFv^N&>5)Z}XCo27&S|anZx?ruH>JW$Iz;6fae%bAOpLiqk*n_-x!Z(0#za5Q+ZU zU_oE9+slSc!jT7mePMGvsG=Oxy0%B{6>DjRwK4(4Q7>b5x;O4>2*8o{Hulf<=qB{U z)LkmTZJaehpq$v6XTVKF|A$eIBV96w(Vgme>l%p_+WifHU}6QvAWrcO86oP{n`EuZ z6oJQnGR0+*u`=75KyCr7XketH;B*~^ldghk7=b8*gq+rat9$7E@^W3MYKc)u9#OW> z=3Jp%4*@oCnLgn%X#1fV<-Tb#kCzQq?TRRBZR+~K&xB{^l& zSJzzwIi;Ad<UE^jPQ&kZIammHaX|NwxBW4D zteM&a8+(=#p&x>G;<(kJLhJwc=I6(_2j1jWdz$P2JN%2_aqZp*I?#oI|^z zO(@nKvKe-n+OX^)h-w?Uj+dC?Ba!JNVd8o4PZd@lj@c^c^;+bt(eqkHYBdjwhoWUw!@&TH&WM zrL-9Df}u2Iln+HhV&F1j@LvThmKzbF-7Vg!rD z#MrR*w&o5j=Chg2k`zqz+?s9Fz-$&@lX=-~W$Rb9MiSK;nrE&Pm|p+%>`5VB8twYOohJ8fOr%XXRwh`Ca{SABY5z{GB@ zxEStK`*P#lPoVD~?*g0iF;0&7$%UV&YYe!g3Uc2WMz|*zyI*wHv$p%D)x>DZ=gK)92|I zc?tFTZ4ocwVt{v1T~j8IALUZngAbE*0U55*@W%uVO#C!isYi7o)!{W3-@RzEw57*TkPnGOwq0j!@f~P+u zTx(gUggqeiM*%W}8J@S>YPfj;RF5%$Xqmksv~)5;=#~<5FV@QtF9zxN4s%W)qV?8{ z9~3-cYAZlXGxe|%CE7JM!^Y=;wgO7ZyVt+XZ3WIT$OwpYqII40eCSH_yD3=2$-3Rb zV_Lzfn%WCMPFNHs)v_^d1Ioz#P8Ab(5ZX^L9dGlQ6?AALNU9%wOFA( zx&Z;z7LWC-{0}2$5_XgF^idDMS|9*yn8`6sQyzhcBir8Rub8) z4RBFZT;E)WebrRT&8ic4fg*hOMGbtVSJaRZSJb1O*sdAdCK)51D<|_bKQO2aUDfiDg>w81NQeo%Mfgm zAjqQRU_vlf^yGDRVgtuUhltiWhw6;CIj`r>O5fjch?0DtD@cZm4bFDYWiFOY3B45j` ze}3#R)Mgv}vYAvu3PWv_I5!;Cd4ho)&bZOxjyq8G))?%)C5EQUls#?3LWRJ@&9m6G z$p^y2AY3+8vsJc)!m(q8j&>HzXtq+D4QBS<3=FS-=q0CCD|-o#>ra}qK?IllBMh`LgtF}ZSPWTgfK>u8b|0P)Es$AJ zQR>r1qxd11PS7({fBD%PH{m%(D?5Z+Gc{DUEEz;C=GLsP(D`rtlke^(Zbk9GDiqvq zvwBY0@*Q^j1Fa*k7=yQ?3Z_v_Ia$~4ExOTB@=_Ep?=*7?mBp9LbJJvHi|q#UL(skkmbpyBOdps`KB62lu1hX9QG`$Dh`Rm6jcAq(mx>{uu>B>c9aD}Dk6QhzziL}$)sot9)jpoOtwL; znZl~B&v9F(7H)^TLo>W8CUJq=H`||rI7)*f4}C{YDrw8jl|$1c`j)_ttQ}!5%`c$cCr4Ye10OY!^~nUI5dUas4!3bA5g(Uq0QS)5 zI=18Vt+{un{1Ygq1=N!paMKl~u-kJZ{W@8)^DuVSMQgfA)oOo3S&s1nfXY8xD4}b&e;U|)9_GEz#iFNl^ zt_@BA+xpmwp_jmxn{MoAv=IJSi4~LF=i@4s3;|_9}GRMuE8|<*C=~VfAC<+O$YS*qi7rX$Tr)TwV#b6TN6v4 z5SG%?O^fI16;MorMKGUjun<_B=%&k>EsFBwX@Q0|oYRbkrEkz&?(~;2{*iq`#-jHT zP?poL{=+BB7|PNR`@qCowILl$rDanT|GM zKiet-c;+arz0>n7!Fe6NC9CHjEjUe#(0ub2BHT0C^Hu(8*NJeg1sECRu`$0P*~g6? z$@H=2ffmHN&6qM0&A=r;RrNu-mQF77He|^&qLoSD3|Hc~eNJ z#7sKk7=h0s8M(0SmKH)lRZ1gmwmCP-RS;Yx2|u-#3G|uaDupFi`Io14JRYD02B8Jy zHSE*`6_XXSutF87#EIiye)j zt&{9TqM`CV00a+l!QzY{K<=T-)HkA`npoFn*xAuehP`RCoeQ!1zCw4k+t*@P+}j-# zo=7&CvmTH?{=1$hrr>rx?FjSJfCsqb2&YB|Ud-tMwYH)p`RILY3a7QvP0L7Ef|^vg zs6M`z)zsknm+Wgl(F*IM)U65%=PF<&RHG@E?hbQ@6TL!t@=&{NsFJI2ImVm9Yg$Ik zdxH{J)~2!9Ukl;oloDI+nvq#G%d&D*;>y|P5o!Ih(lS`$ z0(u<{)ppn{n>zq%Rlp4LMzhnnA5{CTRiq_1-lqpmi%n5df4G2LA)iUyc=rE)SSeoc z=wG#$&@1a9C|n+c*JUHKC!ryKodZjdk_Gp_y|V(HD=H?_YE0#{(HzP0sVq0s-w2bf zd+o>cBWKtNB-P4|nB)uAU{)gNi+1CSTHB$+(I0%^za^-=H+tkyYhNdhHsTB}@eakb z@S-?vvbEHf*$MnrMqZSFJ?*i6Ui1Rp1{E?ua_e9dT$L#^JY^^4=~6=$5MbZ+J!nGiZJXnm3h3 zz;T;U)xO$3?DS@ZH>dU2_6wT|CD!9mLGBvgw%8F;tsZmqLT4qp!&-e=^$BvHLytsB z?)?I8LO)hvi1}xd8f47IKtxz?2N%8k#UM)8^)F1dQ z6W1zLU0ksY>r%v}xw2la=x0=gTk)G(-@vYPJJ#O5QpszM@jBbGr+!$f6RqW4&jqhh z844#ZugET3vt##?v?Ho*nQD=?>lRjHb?s^o#7;Nt2!-du93vty?dvFwU-wgyu>w|F z04CXEIzCb=V&oPnpo9P2sS`tCB09kWa_yi0ZAZ8{U-r@)ry8X6n2oJgmqDf;Mqrut z;6L11Y*??@5K{q{9R-GakNjH_e&{it8#ziA`T9o5P_PY0W@mBoQ#5(c?TI*QH^K8Q z`NaM9H#o&Mg+JSP)ZW4*DM3x$qOL_99UYxI{06ox05j$BhHPOz)?i0gAbC#Xz-VP+ zqm|n+$JK2@-x@Pte{7pM%fIVzkNtbB__;}7v;)6&bvmkXcrx09p6Mn@+TM*LQhSCB zX>l4!bnW|k2L#`4Nk0SE7=X*NkT#autVw~w)rAN)WUo(t3bRtI8E;8eaDreeo@t2? z$$JrcmsC12CG{4?>6!S$P&P18(5Lz$)#Al{MmRqO6iSB*`LAD(FRXn4lBd2;;zWy* z5RPGffFg+NFfXX(2ufIZ7Za20WBOgH$Q{|cO?VqdD!i|pINA_iKqs<(lFGmL5ag2= zX|g8%fN;H9uC2p@ap}U#d11RF$`N&?{+=u`+V!epMCQ%A?vZS69KaV8Z%?#C=?1jJ zu@p|k9*lO-XVx1dL%B9>TeuB9i^kBP#l9qTOCLdh>3-gErJxT ztjW%Y;7L7c-Rj)u9(}NCU5!yq_8{ABf_`#dM2Kk1!@=kg_xC7E^hE%U?|BhU!nq#@ zymvOlS{_=;#aizd^t>~a?#2UfDzaRu@f?FO%8QXCU~Uc!jDg5&(U_bG409QppZL4^T~^}*^zCLZp<(= z+J%?arXXxwDQaqgjdLqiUFj@f$vu^dE^Md2LSLgJ-DY3TkF?}=8dBPSh6O4|k$5^v zoX?+DmxhiPz&5Xou;)C6_geYVOy`e5PCg@=_)L@*aew2vDFGJjPl~DDyAmiZ-DyI zJ22U^-l-9(HOmXY^W@1BFfB|r$IH=Ov>Tbdiv}T0A^@N)tezPQSqKLr;HL`_4D`7? z?J|zp%5lCSFKHccVL?nW`xHK$^y`Ja$!($M5^7DU^;>$V}qoPU3}Qa3Ds+<6(lRumEoz?Z+bp0$oUGT!vZM>rm9{ zk8nYdxf2m;0JGF#*LPrwqb#l(mx$`pK6KM7jcZUD?N>OvkOZwyF5i!#A)o%>l`Wdp zpue{p3*!rccm>B_JsQ1@STl#6ihrfJ9PK`3VO;))KoI+;$Vw^tb^W-so#Fa3_F=}y7upTsybcTNbodS5Wh{#)5sRhkvcOsEG~ zU{`?hijE_HABD>oihcPV>K7rkm(q|{423vz(HT3FhF*YI=3wa;CmChi5J3+k;_aZR z4(@4OJVd%e&S`2j!?rOvx1v+YYy9WKYjAQbD^Ft>7ZqhC^s(O-VW&2asqI9aA>NS9 zu~qZs-mgKJR+fYB>W{|?DLnBXFs$AjzUq<;f;C?g{RlC03PZPiqO*Kr4W>rhBb1p0 zkL^LP7@V#87A;V)Ux0#DUpP(mvjH36Mj3e^H=i=I@Jx55Ul8kjaY=8Cxx_*~kOPW5OR@uBF6-!;!FyH^PpHCj7nS(iR)X^hQ-^ z@fRc6U;S53s>(T-)JV;dzoQUEWfwr)mBliRON^Q;&o9Vje>kRnX%bY|@v6PKj7FL- zr~%oyu1hgRIMy!82gG_MhVzG`%I(mq^(1|4w$$L+hep$G?_nr%GEm(Qw0k?5VxzqH zf$_!mT8s(xfg$4dI!Fl+Eq7Ia2a-zU7~Ec%+WHI1r^`%vy0F>D$r;Z{IIW3}3G=&v z)h!`shYbIzM<`tKEOd~E*`fPuFeNp{Q+%TL-cpkL358^%5MQaXpAXldpW|Sq&%QTS z0`-z+^W=Mii6_@9Xb`DWH@43pmK_h=7)kDvcXxIHbk6{@eEye) zWLo6UISY-g*@PqTb*@R9<7n5+hcz^rdDPF4U*`y;Sao_{gofIV>9e6^luHuN@rHdmg&UI^)_;=_;S8)Hh-~#~U~1&}?4Pgwe}1d&BgtU`To0WnC_q zOl~^5u{H70R?KLYhaTH<3wU3$)|;l^TVxnvW$ROWll%FDQpcuI3Qp>I@M`0maEYE) zwXcY-BG1dQsBl(%1`iYuH&v#Ud_OjK=SVcd2-K#fRDHbhP%DmdRsn8xG&` z_+C=#tX0!@>%ggtD#`PtKH^kSNoaBd=5jdFPJnRz;xH%qSdvr^HZTsGZ4$#8DI)FE zZn1w4MY*t$1OyaVsfcvqB@B<=6x7nSPMzWjwNJn2+Y8-#olsaXo%2fnHt%xDJy2N} z*{K>ds#O;rwwD*Oa6{V2pTLw1ACM=szmQy=s`2jk2$4KA{P^!vDQM(n3}VeCTd}4@ z>Pijn%1Wd0E{yx0!CnlEjXF@-zN~VynilK27b^g&c2m{xMBP&EWthYI;nZ*61 zNWN^m2kUwIvij)7yBd)47%IqKKPrW);?L0aoOz+?r+PfQhT2l(etE6(q&I2m!N$g; z75s~kLx@DrNmgH_0b|a@qO7%KCI6p=CGzss-1%vbaDcll(L*=&r*F;-Eo#XET4Dc7 zvYxJZG+!P5h;n#F z)17zJjCnac&Vi3c;JG9gdyPEgnjCGXHT-_4?&wl_SKIB3pid>RwW zjV(e`p!~I2&(mmk>TaeX#sgg8&r2{;$_H`n3aOdAm|a3t6j#^aY%k+zjFNq@d2TPM z`+6@G>&{r?$cB86q0leAODMNOY8#1kQxTDJxVAMLeIiN#n+<tyL$gA<8uWA%xU=eC6G$RWUS+ zP(ji*|;?u?d*cT6emC@0=Jy139Uee~8>e9Gvix$CJq2ZVlFA-vMof4Bb}=%>g=B4O+|R=oWPhP2 z%B_FLLXaliQJAs_oF9v_9dE!+1M#Sx%5IyAVpH%7mvwd7lCu;-r1cLb!jno~?8MGa zA|E|Z)M_!H(X;xXr@#WT6AiW(^dZZwwb|f{k!Bp*Uiml{_jeZ}tkl0ai0`RQJ+Td$ z1tmUQz%e9MXzsC|md8*T2MjCnG2{VPV)9`v5-{QOAO+{!z#p#puQk<2+UI+Cju zKDXC0%%LWAho2-%O6{v(gK5ki ze`oS{&a<-XpZ4>?a|k^0NDvZ9iUqLyrEqb)Iu}^{(!~otBw{=}F{m~U+*=_kUiQx4 zk)<3(e9<1MVaf|6#cpJhml#DlQVGRO3r@Gec)fr{o?tL-YGZiK)hrV*%oEHgQ(L1%+cm#Ri zS{%zOx7ZI2o?xtPC266_L?L_#xYrl(e1}GO5=`SG%jpIS*IzCKB3ZE(Lx2e@mh(tg zla+UivgLsqd<&YWFJP1IVxf?`LdeQV#hTtl&$&j72FIkwdOcA1WHgApP+ze|Fs1gb ztIuZKS9k~uoK1yY`)m;3VjkeEW{|0u0kbt%I@Q?Nc$e7B(&Ih4%dWkoY-Fd1-e_eE35)tg{xEf1wB=T(Jx zF~?s+ah!)EY~i})9Wd?pyCev>Yt`=<>BUj!^-nEoOc4pY8`gfEaO|3D>NaOXW+DRK zy{kBrrcA;tBn`Ifd4~v~;kr-_mK@}z&qk))9Bs?PK;+(n zGB+NIPdWLanrtUSdf*xUw3|PC}7=DV+z#9DM%#jk-dVUZDNmoNn*ts{MNHX3Ho0xi)Z^s8+vK;012FFp5?50khIvQ?ydf)H^hB*JpH^ zY%s|SAKh<)=~AMeJ+pp>{UvD@sOi&|d@vcB$v2e)D+N7|@^G(vq>V1oYXzKXe}h$a zkF-=Drq0qHViQ&1j35TBd*FDS2+kc3JaZAFny-KD)z3~1L649aNL_m)!i)=?oH%>H zSGM3D1U?3qkp%LX8I9YRYBQ&2c$G&|S^N8FvKDw)@el_Ix%UzNg6E`&p%@asyU@?J zta5IIChPGC!FYSc1zL%u;&fvF%yG-rrl=#?L_RS)luK zZm>zL)Q<;x3Hquu$p$QEim5rj1rdkot<%vh8u`|wQszE(76it9p|DYqOkmC`tgz?i z{zM9}*8iZkqe4_vMr__cSf$gqmGA2;rOO#gydZl;lW-M%b;@NecIjg%f(^IE+ifyr z+EUNBZ1{Fl3~sW2o3qr1=wy6dyz&c155i~j*bu8+t{fZg1SH68_gXngzL1lrT+V~~*7YXymd z2>~QDYs75${4By&#B2z9fu<~KucAUS+ZRy|TM+4T}4~+6{(Rjqk7gxowye^X-LUlD_Tz@P6)C+d6fIVI3_u-jE@Jen*DFME zn3kc+t!C2CxGycki{ke9IVRtB9i@MwbZxQ(%h`gIhi@q96K2QP$;5N8Y|O&CQ2r2H zAYN5px|?q^w+7FB%62^ARirmB=v#mF8T2;T7N(V%{>Twbh~5yuiqoP+ew@{g=AWB^ z-UnTwsZ>boMb4_!#ojDGSc#jkI5a5eB^W4ZioB8ezQmdGB6~57SIRH~r#IamIsvsF z40?HBj9cbRNsOBcAfq288g^gdEYSSslQkSTnqPdpn~68Hij~eR=ua<4?v)&wJ9qKj z?rtJ3Ld#Xoto=2;xaw#KAGF5kT|uEg#$mX<&bda6i0WtvdHwLL&S;}^jTFVz`HOp2 z;A7ktXXa{b=2x9PnS(fsPvvK)!xoDyYEPl9w>q<9S&9TvjbW4n*?t!<-Gl8OuCV0n z)6cOLWzdw~D#{*udN-$yu95k;Nd@d=zn3fN%qH2InX>r3*48ycSOz&c>rkA61Iw$NXX%UL>MM{M& zSooK!M9ueyVOCt{q0-QvH5sZeQWtZ9o#U7I(GdF(Q%GA$;7%{WgwPZk%y_pMl0!lZ zQMVlfVWL8RB0$LWR2KvGdLR%Wq~Tv>6DvQ$!6~bhJbQQ`C=$_c`x2$Z$iE&A<)+mW zKf8~JwBr)kl2$WmKVJ-M%lo_=2zZ1b73!lh*f0d>(2*h_aNY}r506mviE!}Z2toCi zpN*rHU0-q5koyhyGNfqPyo0T>K>85M$}ZVVL8N}{T@~p+dk+rU1hUgGlsp$%G!V+? zWGnFIp?;8tS_Dg59%<=}0)oOrY)w%BwUwVr6B^Gg8Vjyp_UW6m*#J&)g$(-YSWz;j zC6={ypU_0m7ggfHX-EcUDXKuPfm4#~;EgZgM6#t_o2%#%8(<0^tEdtabIwOGDW@9Y zT(%94HM}Lw^4ki|HmCJXs)JbsTC^E(6`EbZb|*rjk4Q&AUrIucAPM^Hl}5A;?|J2& zb&|iL+JB%Kd?bd9!7i-rFW7>%)QmRgv@4B2rQXGSU13*ytSTDfBC-KDeqmY7FDR_lCoPYar6&mQrz;Y`muw!a?F?^^1=2+zZ7$X4D$aFCe(`A+cV?s;Z6!L$iecy=Po}EGI4?Pf16dMP zvcj`Bl}eZc*+YlTP^oH635LjzT)?rZu163>a@h(F`N%hSmJ2Hm*7O!ZB{>pj{B#W1 zo94K1EO+iZTL24vqd49?a(%Ct^n`0KoLCmqe4ZdDn7A=JYlwy+Sp{QR{6Pogk;6_= zp7|>lxDagH8zHS-wb-ovK>iW#=-i#-59pnF0Th)_AT@TtO_aE6EUcH+*JX|Lf#V}C z^M4z)xC*Z3SdsWjw(YsInhThyvw&tdL@;XsUu!T8PB3E^G}i^vCv#e)4+me5|26^% zZ`Mu&X4P;%v?K^~buME<7(Xglul5{exti2oq%1k;BGB3?>yhU5(g?(_A_j|**UMX= zPcFhjPM$^{7Q0Y7*8t{x3%aegR$`djERCIBfbKF99aA1zW?XksA)k!}oR>-ndmmKX z1x+&rYRQX&MWxC*oVG-(5N(!&=fnx1JBTjROfpx}19X=KiuRl={_^)bn=v@_cdtD- zGFMNF-r{R~!~S%Rq8-*ejkMXC%zIH3bRl2at-gd6c@3E^XyN{fX0&`Y~w595A4On|a1-6?HG>zHo#Z}P& z#?Qwzf>ayB%<$BIbptyc0q!GzjG3_E-{WZR^O%=qe;i{)^iC{EGgzCaopm9+YlE_t zEy}UZ+6Nwam|q;g%U+TP9CCC;eGAro7lwM`+1}6j5{)N>l!}pzeJ7%tr*SThOJgQS zhj5;vJ0Z*)^l@k;*{hyLRr55OhbQwo>fuIg>82<{nubn&UU70a%AhL|dB7ReK-k7# z!wos2fWYU_x-Y5!D>6elTTza@shj}xpwgCjaGHzL5E>>%T6v{Ylu24e$kTOMV?iIo z6w#ctSC8G1hoQ!gS@#NK1t|d;MdrOFmk}eA+9TfS&6ksqPZ=_0`(BR|Oz++yGz6h` z1>QBu67Bx3x%lSN9@%P(T$RaDxxkyJychg#gakb38khi8fBD&G+6gA0VU1Z~fSVz5 zt75fAS_F8fL=<|07RUPXlzFR{U@Fdy&|rCb76v?luPz<6ME0JTbf4uc#<}*rshlcz zag;wtgU01Fo9YWw{DPXvO4!d($|H!X@epKAtxhMJXOC%R%}hZ6Oe)8ia=54~m%8PW zk2~!y`oLDQ)PH8Hq?)kOxNs2!LUfdp8IT$@Ul2TK1?L9F6D!pf2=vS+mB5)G7?2|f z+O-OI=eabwUGBQSQSXa-9R8smHcJG(Z?>TWZyip*E$hPRkEV!=1s9elm0P)j9`I31 zRyoBGAvhfIf`yFV%_8>OX%v!yaYCND{I>|Ea)&5ij+q}?dd-&wTSZ!_x7#K@b+^J+ znX1v4O?+>IRp}EHFOTC#b`mLJMoriWE1YUlp#wZZhr1dBY2s0Po~XZ1NUoS#AB*C7 zZU_mDzzm#e_BA{2*YJ{q&{iJ&_Ma3?CQGAq*!{(LtVD&(y4u#+8??vpPrmo8JjLmA zmq$6WHcN2IM&xxY7mZ7VrMqJS!0@E{%g>&BMu@*OAY31Y!!{^qH99CUt&APMDj8=v zns8sagEly?VzOeD>k%W2;$4XOwZIx{bB!A+0!#K~qn5pZGn3SH_Y{fbz%+HsN68N# z2&RxF2opBhHCog(%-QgLDHNU#df76g9;ph^MuN^75l^`KR;Gy+1z@CvJMuPj`{Qax zR#AyTJa1047iU`WaDh%JI)MfnP#*ODcv6F@xL>dxhvbVoJKCVmonF=;fK(i<7iytP z*eIL$RmlYX|GN#GUdx;gtf6kza%B#ZfvR0Vlnp$3LK`9LawP(KsA9XgP-f7>nO=!a zF^W=37kN6Yr^Q^YFpEt-U}(bbUq%+;NlW3T#8i6q$`wkTl>jDrZGKZvX0qhi1d)Su z^-|Qdqp^xWmt9|qlSt|&x{>8?@?mRJ2yLqJFxs}nBf~gpk+j^~g6Zwoa^#^xT-Jf< zaQ~$XCweF5Qzb+cuuY=^tKYLR>Lj5<$?JOiH8_$p)OKY~ zzR@CwV4tCbeJ_(*pdOY*Kt})_Fx_G0V`EG1Re6QkiWQt7wX5>Ov70sDSrFt z^H_o|e}x;$qxGEvuDCS=blIF4Ua&ys8Ng*ZSN&1Ji+5shbV9!mmx}k;<;K#y6iqo< z@3R-fu3`i1(6#o260y&Xax4Z{;$aE+ouLsjFDiuL8Ljj*IQLr&5u?}?W zTstPo@=I>ax`(@awcL59Hx_%LyCh>!!vikWUw(GsQ)LSr^;W`>vo>9{pN$rvd0ifC z-71wWkg!eR$LlsJwM|(pXwO4m*@bP?-3dAE#>~u>DB{gQtpL~v zMb6FCRgqZX2SmFG+)YppLBPKeU|7$?JjRTk>pu5cE=cHo7ifKcXiw4oNOMw{U^07Hp3DgOS zcIOIgT5G9ih%quDzP67HLfl_YPvF!6K{)0ih0mTp^sp_IwLlxm1N`X(6}=(L`Nv&j zQF1d2<{YZ5>BWa@;#NcHabT9myzu^& zUwkBP!q98kwd#`wo4hH?gInuq%0`EHNJ@ma#!Bt-GF^~QUWAv;TcXT4viRt@%@1<( z@SJK*|0-d|EIJRcJ2K@tjjvi)A%tEqCrZM?2XS&%Tm`CB^I(4lM0tu9FbINr{~KIY z9&kmSEqz#{O;vE_M~KY_*iVFu-B$f7k7Rwg|8aa?-Vhyi^E^hq0$j-=eUEZ~h5 z9E#dxO{=aAfSOM1f1yfP?}rq-BUvb9JqnT6Sl6eE+Ndy>Y?oHFXql0X_O&ayQn zX)8|$$z=Zqn`ltJ>n<>fT+WcCHOeUpz#TrdZFa{-FBhPkax7isZR+q9q8viwW`;Uw zuulpr-#_(&Lpdi!IE8Swr=gpWS_pC?_+ZUNasXog*ug2dprm`cPBJ^WxnN zf*wdueU@Ah>B{xm?Kdu{19WPOX-OVFUTzWeCcE*Ny7e^pC|!29AM5DX6%%+Krbl@Z zx^}mqFDw9Tp03p$w_bVi5PiG_`arx&aT=z39u=fj*f>|#%@^0t{GPtvez@YU6M}#v zDXU0$P;A<>=pc8OpqHFS z1p^H50}?YN3~L7dNF^o=<8XS}4>US#kUyYtI}a)!=`8DbYCMJd2}eJXm-ELfgi@Ia z*gK}a0M7e$m@>&3>)8le&wXDd9mi<|{9Drquc~95jvSMljEv7Y%RWW(om zDAlE@cqT#EyYMD+kL1DH$Dx$Gp$$`KyA+5G%ClTHXbhkM(d!>r*WxWYt|mI8+;Iam zRrGd3K7kf8-<36@fRGyS0NJ7wby;j2qVIA51feuII^od!jn_Xxei=Hi+#Bt12^pvG zWX0EY@D_fLA0TNZ7=5Uaf}c!Y&P4&=ZEIg_OgAAWch=j{i656AW>My*11q^=NR;

VQj%7oZxJGIlu5`=Id zJX2yxEG8l{wN~GqdkA$y*En+f%pw@K7_ezjZr{0;@XZv zu?J^$d_-zOlD}}4|3BfFN6N5`kQX+Y-KhQiS>FQ@N=V_w<`@SJQ$^j4T15GYYBy|w zpAYsH(tl|Y<;tz6nQQOx6!YFYoJTg39HVz%A)$DkhZd{)FuPXw%>~;U1uHkhQ`%T!cXKUCl^PErsJ;;$AJYi)nUi*>2 zxzCP_bi`HZCb)t;TW4M(CXNillmA48660BN41K8rvmUv)?H_nSP3@O)yr{Ktqn;LCaiS@h!7qq(`o?C*q?V z!Q(&fdle_B;D54all<7)j+n($UDb72Q889_83yV&3@*m|aLP2x%8Ox- ztHCv>QcfUjR@|`&Tb_zGc>k~WH(*>F3U_WTHBneS!Nl{|QYu;vQiaWJ120R865k4a zOgvf03oAnobP*-KN(4cNhN==xSnwO9@ur8WLV|uQ+V@c-K@bxg8OIgbI?H&IJtpC| zaQWtvBJr`ZESydInBY>$RUMJPkndgNGIloXd^1SB)|5z{Sk5)%v~U6DK^X{%~;ZF8AXq1;zciOAywbXNS%O777Saezx?c=ek!T@?0i>*h9H5?Z(85a0{s)HFE3@I28%*IWm!f-Lpq|AZERy;+D%wWRgug^vn!;6cmLoN~|l7wJf;`C%F$dnk1|$?x0h*Yu94 z;d9s8J*($jp_$7XT|G7=2l&(rB@qt7)(N-J;&w}`33A(F{=a?S->f>(E-M}O6Fo7d zXApA73fLhSQ`_R|X~u1Z5~%2U4Gm~gSS^&LUyJhLTKX8zp1l=|i3{$v7kKu+sp$1t z+Yz4j9HAx*-1s^KhW{Z5s?lW&t}CVfZ`Y9Q2XKAD*ELi51C7qcCsa%(EcpuZ@9`G< zb_Lxi?Ud@9BcE(_$?n5eRD2!8impajs^^>+1UYxHaB;qdh)A5-l#pJdOXwSiRBEwv zfPac_745K#8Tz;2|i!`es zf*|JGQ`!A4cKF?w4%r*xtV8yO_Wx2Z4rM7pb+vewQttCxJbwGh8S_Qd$6?`A~B5#mzf%h z8udtl7A{T1H#C7GEBS`})a7@us-kK=-{7REfET|+)#S@#yuPSzx96o*o;;pVQ(-Mt z-ypCRz^BzYGm`(q>Q~9%K`*=f2R2oONy!SwjTA@br6!&w_8cay*i<5~5SU3<9 zJ4*nqztdC%b}Z*E&bQ2O0+-W0-g|kbd^deG*}>Ljqb|x@pMP_KC7mw;duFO!`FWGljmM z5zFg}PUA{g}-QEDX4Wl`kM+;8-F3#S1J9_|L2Q zz->>C7lOIQ&+!X4$qn`LnVm7pSi0Ai9zRjg1AJw~5}V-;m1G??Skd`CTl)f)t3*gI zvhkO0(FW{DSfEY~FjbF-sOC>OM1VZw4G`Z&*WJrhOt5z&8%q{+0nfvTsc}Vab88sr0vP2*>6BG9m_q)J&|)r0 zfYy~;8SB)5LZ$V9u66VN4^lI+n&iimb17GE*oOauN*r}^RazcaxQv1Brm(2&kPWx0 zIPoSS-bRS#O)#> z67G*#?ZH`R6Lx%^sd5uaOy54TT#9@37Z~OBi`xNWYAVo97yd;9X&zFE(PjlHf#P6O z0I$9c&^}3yh)shmCK*Y>sqB#FxNoEqqvLdL{|_bJMu}mPno}{W6X}FqK(@uCqB9eO z>E+_K7c4Qqm|~9>B+>Urd=By>0I3Yw$^KDRoW4h^N&57H*&8Fiye_nkbXaJ!l*caXxGlP`6i{&5XR z>EK;9#pXMLoTuTv^#b95sOJ=Flvt7I6QGyZVO#DCdY+oMWm#m9=(R%!Lf;3EYkd1Q z)y9w`WI66Cc>LBOls~>1$8Q5~sxbjikn-63RWp?0|NA=7Yr7d(xLRprDQLzYwKzJJ75smFU3Fa6$@3SH?rwPKkW_gB z3k+<wUax~#%{$v>x{GY6#H!S?EJlF_rrsW-`9`V%RhWRGqW?Z zv%T+~4hF%q&l_NN*AKTZmI0^6=}#>CzMP&3Hp2i`Q*zn@EP?iI4ywi4rsG9!y785C z$fa{9(E5NjK%#AdtKfF0kSV-#f1Nq|=L?b+TRg}6;Ka3L%&Q@rVY?Ql$Xw;@e&pH! z5Sh4zl|=wBQ%B`v_5DE}%nFyUI9Q--^tU(dQOR^c)nC%V!VTBp({VZIdodbq#i~ue zRYLQl9br$2c87Z~%Q>iWVd)QCaUH!7>n6II&!!VDEjB1v-yd5^Ovf_)9{K?fmLdj{>gZ0^eJ=l!_nA36xuBDHiYLt{>&GcRr z6dV479&hTDm{E=`8H$@4B!k1z_H@5ez*N08&W0!I3##1iN~XvfALq^A4Lt)9;BtRD zs39u9Pq06$^B*4CCslr}G18F|PB%nv;43+Pa#FCs9j+OSh~sK|4Z)?B*Uj*Ij9WPC z^bL)Pey?Bt-_dlYb{qfX$z#h_!2&;~+ELCq_pL&|?o+*CbRskrWkyS79&4@8TYc39 za@n(69~ndV;KS(Zf+gA`?Lhk#ua?BbYn?gm-G&J#{a|-fbRV+aD>?^LHrC3V>}Wx! zH`EVP#f* zLnQCTfR-q6)$OBj?mBoX6A{-%5_CNy zm>qnlZOl5RtDKlcCiY->6-^n5VcdBbJC~_TV&80W0kPzpM<$}qE|Y6emAUn?z?0Gg zu;s}aZ;!^ue?~*<2TdDxZfQjm)_#mm1WC7F{@71v_PfxdwUyMUFt$V*Rxo90K&^ut^vJ-u{j+?^dv8F53| z#0&*jdFJGQJsu;J??OF!D!TS~Fy-y7#YNh_d-H~{8xi)UCvP85aI!$Q`zwyPxp&un zhXwb~DP8Zw@5L?Z)pzuyb1wwXwbsONsofl=ZeNcY9}3AKlB@Io4xT#z59@Pz-kFfM z<#I5qoT&}rjN7^={W=d?6OvjbG5&rfn0*|Gq_w+_Vr50_bJw7*r(WZIfydmtI?;SG zr|+<%NuoEJT9_mu^*SF1%JVEyyfoM4_D5@@S>6V1xI{baR`A?pjgC`I^ZIk%DSV|@ zqx>bRn0s8JEAtuV1|YyQMKShtB)Yvu_tE8hBAK5{tYsi;Y*-xI&kgZ$`Xh9yZVr`M z?Q)t3UdrLeX+9MYMh!6aZX=aa_zZ)ip-#;fUj^zc-Voap+MC*4`B%y^dbgA^y9G8;?YpOD*F(IQ`yH7Z(sn$phxdqvO^}dV0LNeaOv< z6^um#;E8IpGcc2UuNDOP8F3Yanw`E*eS=VZ1K$ToWv{6$NMWEA`<_8?vTQ6P4#6w8 zD(_w~3<|733#cJ66;;I_$!ntEL0i37v*?Wv8 zwrYwuiKnpc4`smfT29~S(8_9i;lv7v33+I&+E8DH39DO((ct<-jio6$rNXwo*fk#< z=Ulc$Gd+>wx&G;u0y%bzi@#$u;XqB8Fa##lm(-cnLU5uwty#fCH@rD_(+#;$#^H4n zj#XJZ83RS5_JZpAWL*%y)ZVLT@LNOJM_b17(#EipAT6-Q(yGt~h^z1^!FY$F2d3d9 z@Ll?vWCw88gF1C3ReB~1PM&-%-9~4`mQ2IA)G=4#+_9jo5AEexp)Zl!!K*HUv;QQ# zOg{dFn{thT8jt zUnpA_%}j3l-Ny-f?&nN1T+rmQq9#Jtmhs|S(l8`C2d>nD?S1Fwz`^{yFIuqD;VJE; zw1rgJovS7a&YXQl#>{7r5i&}kL*`6HzwKB_36%+eE_Wv0zskj#v0njL<4hOX9ou?f z>V?uGCD*l^TAHca2}=|9=>&Fr$dZct&#d=BQ4JAca$CZF6z{vBqo>#iCHmn$-Zu$T z9t$3MB{F=bT#~1F)quO}$h6V{Z)#0LE(cd@CDn7mn-vIfxk+@I|H#d{OVYAi%(8 zlDioh!1-f_6xOdd1&!VS-(Es{mSf5GrQj*dM8X^*o#1OUpa}K44fQJ968Kt>7Hn|{ zVmnr-Ij(bB!C-fqY5s`@D4R0kjUcDiU9M5epg!X5Ix2&`A7|eV>P$gK zu%;V}*zVxW@90qIJbQT{<{9)#1&_S3lKNTjklB=TQ_Hhkb7LBO=>M(}wDUxbCQ1dJ z{VJHGZqt%&@I^vrix*gmb3JWrF_pST?Ftq92cK1TsEchr4d z*acNU**xXOF~l}hr1|aR6z9S>Pb}x*#P%J_~WR}9n3cyk<4pGB21c`IwpeG7SNrWd?&b&$9{*C>t0vV$DJ^>>Yh(wxlm&Nk zv$JdHe!zDa7$_++B#kSA7q$Xp9r(Rif$p`NkHB~_K>3KAy66o;Vg*Q+`|O2>X5|?8 z_bvPuDv2~1gi0MBhizAT+=@9qQ!AcCI-gPI@H;fmXP6MmbAx{d!?u(VUpB)H`B<+l zK=wEah2LT{pX)>ET(rCdsf5@}g#ig}-I?*Ph)H7T>$L7rIaJLa{F7Vui!lmAfveV^ zd+)8^u_vD)v@>XBkF-X1Ti*$B6?_R=?2z5}tt-NIA38sI5cq8@tjzFpVP11_kkdLx z6KF~dN7gTX$j;hhUnOu_D0}4|7sg++=fWoJ4?0r^E>r(B0(G5%GF`E=-N#+)$}!EK zHLK0FUgO+=mbO}ZwzGZof@Z4(wjhqaFKk;B5_VA^v+80Km{xyNc9^lV2XZ%bGpZ@Z)| z4`tcCj9pm&Rsc1B=wa-@b|)*lnQ!U_Ty6}mO=P!JiB2owpRs;nY-Fn_D~@#L#aNC? ztf@noR6&@?Q;FkwoS`DyU8HP!cDOKk1tDNQA_fyA;mK>bN+E3HY}{+G@{aT75whay z%T{Nh4ZTGo+u*ofFc`$}S3WqkfC%9MG98j?No$98x`6&FzgaODgYDkYlKKq=SZM)|bz^f84Y-5XCdaBV+% zqM>J;K!|O$!r87DHqNNfY3+YKpn_Y^qU)09oy$(5O9X0}<#-{RDaqKq`$CsAlt&3l z5|6X|kSwJkxn@8pdMIqVerphP%pM)8brNcNMmbIU0Dc3e}uqp-`N= z<>$rL?!kT|!C_x*`}-)JPBetN^7#DLI*c{F>lewqyx=#Yz1U<|lN<2cRnW^VXg=O> zd-abM?qe^&5-)iL9=5vx2E%}vzZM~oLa@yy2ge0Rg~7~26U1pNcO3hj&Xvle$Hh!j z|Cg8T@2=2N%4JjFdAsyVOYO!6M0q%~y))2INVjnobCULh$p`w$4dtqLiEg+eaMoBk z%OS2|!LKq$!=a`KE1Bz^54e1B2dD@crcH@9Id#cgHp5TnYGkJL=zirsuco?0GaL4iQdbi;yxIM`mmb5Ei>R|w>xs%r zv;j;W#WAxdbLS-1g~_zvkRX{|Cmwn))dsUBrY0V2BpwFm4Byu!&i(^h+lHZ5C5CT_ zVLv4PO$<;Fye)42n@yJ?_fr%}4F9#~CAcu#ld(<$$xP{NJ50*m_Zt|JU_y|@5Zwv( zb~kZlw-#$W1n@R81pPp0GHRX>;HR&kvpI$A$l~LeS@>wtMPX`u8gG9T%)*MCP}c10 zV(dM3>KaDbcYRsqB^ZC{_vFoqD@A>JhAAh3mc_>obMWz0mSWJHm$4otDDJLFlT_DS%aqxQdm3`R?wZb}xiQ_G zI3MnTxc8gd4Q8EJ0&GA2;E{Pz$^qjJmlR4SF&%N9hQHRUF*;ssp8xO%(ayJw3 z6b~f;FJ7+3aDYkAhHKqSNv^g@z!0{4HTs%S&y+Hg$~ghjAA*s92u^lP-cKl(E@;~F z+%)k;7&GwK)KSq{b@~HeuMKF?)2vF9wRz3%Y{8ywk-3v6i_fsetI!LvlTJ)5o6;$# zeGmlsD>PNb&z%Wyee)Cld5&j!R_L$bGNbOqhHi#=G^dtBblg`V1bA-?wrmk@pwh8B zD_DeUQZv5MHy5S~(U{EMSlOR>Q34K9V9b>{9p|H~m~3BqF5eYdJ5NAs;E{uP-eFGf z5Mr=}Kz#Y=x0*}Qnqs1-=NVTcIlas1r5>iBZ{bk7Pwa$?T97}RaHQ79Lh)XCu}(+p zJJtw9v=oGt?shNOm+-XRsPhxf8nY35hRwZ;KPlMf0~?k6(DS#Rji6rhGNuP|FMeKC z%)_Cq&g|O|yxc)6CA;~^ zQ9;HfH3?wfmK0ZZK{ymGs3ZjN9Lz|M?87M2v9KgE0?(>UVs}mf2wnU=h|Q>=Xw|sN z3wIqN@fIpfez_tMm*lCkIMdQUuyszAf4|Ad##7M_roj@~iYM_u&?DV)1h?AgVa`h$ zQRiYMC@<`8zCzJD%MH$F1jr;2{3RwT};Lo>~y1{tV z5!#jhFE87pix6K7=$_p$4M z0=G1VKWpfhex);$6Y01X*og|Du(x7p6u*8 zjncm~+>T~DdA3lFyyR6eELe0`WO3fk&AV?xSb5ZjJTj*&&}(SVEH?moM3-Wjg$DBG zETnHh)oKuKbp6c1t)4gSms0O4552t?3)0NZc#q>8N@UA6(_z6<@pJSRyJU znZ7k!uKyM)ZgzIuf}dOdE`GLMqjnQHb^AY59+iJxC3eJ4_;?})0@=?xBpM}6qqie5 zGvMVSdvDtQCz?sHnic|bJ*OalDAC(?4zp#Gw*oIr_cOzqw|qpq^7l}eayQXnvn(Gc zqVoOEgPSl$9^ERaveu~Wn9AEmBpz0f>6<|?4|zzb*kS@p_FV-)9nY&n3iO&kGLjz0 zr7MZ@ifa3Q&Y2rx<0dYxb^!VKQ?>o9P#u4PMP5(6Jt)X`X^k16<$fICzV>Qp#Tk%1 z3r$|`n!6q@A#b9yac9$Z0J1_w)coVWxW#bC6HL?P%jg|X2wS470Nl7|9j<2+eVa3D zZ{I^N>xS@<`>-BoO3H}8>uJP>1sPZ|)p`s(pDE%xZ-awY-4H^TzC&v1yy`Ycy;cWj zdlY-UXF0sMj5d*GTe@<(-T8rOX;zpqv)Zl+~Hd)aH5dtcpVz^$96|fucPxLL9yXc-Gm<6uod8yyN-PE)0`Zk@YRc@ksR8AF;bqFc zJEL zptWa#0baJO4`opxU#NOF+d|Xo1!xkrB&K@+v2*`uG_f2R+b4HV(nQvDGp0d}J;gW6 z?>0Vocs|U8{bQ!6K1-GJjh-2!H;ClmujM0{N}0bbu^7G@xdkDy$XC%dQD@51x2Og5 zS9MSQTOQPVfPZp$-3$`+$rZeudCoym#c{P!Rkv|1kW8@64e*866_Zmg?#ZMcYXX|J zJa4=gD(JJeseO1oJiMA&#uSD$KQ z1yQ_Zc|Z1aWq^H2a?1#Vr!VQ?b`5H{wIo>=Czt>Lb!N$bBIGY7AORIg_Az;BR*h{u zgoipu<85v#|6HQ82`g+<-h^lO5@A~Z0D1+t7J@)E6S(DH51^Tngq;hIU!r@^u{iRS zNBt}9f1uyF4_&5J0j52~bav)khY&dIDfE>WN=DuEy4n&1UVvLiH?P+;6kem1xxAK` z+g;G3%|$Z9y0}nI@;wXBR(p$(iAIIU8F+jO!e=llO)$*fFVsv*rz)J$MWiLd|5%T62lKZ-M9xxM2Y1X_1f5?adX zF+`=Hh+ih@qBzXw#7QeJx|z`%s>lP=`SA*UA}}gDZt9#>=P`d-wDLjwdnlWBkmOcs zJxS0bJ*XM`{tge6Ohp1(F|k>A1kFBavLHBjL>(-CZgd<`oHT>~Oc_EM!w=$9K@;@R zfine%t*@C4`|!=#j$MlY5Qy|($Ia@-6>kxT?I8&6O)r{ut~Uy<89^ zA5({NbXljNn*z^34$ZseBaHnk^s>xZQ3c>lubkBQm*cYjIt`28|E6f@3xKT48LcAXBj>92Li7EF|Q)8CU4E^2h<39+61=eiMG2Arc)7NR_jjtzV0%LZhdRe7c zBr1c)zdj(yRRo+PPtHc+fda+R=FGnll6X=ISK*v5WKi_qf?!#B?*IYSAb~a=-~2^K znt4A($9c5bi>o@+jc2^{yzfd&ZjTHgdgQA3SAL()i>A12DY_WKaF zHP3(@e1=e4)ZN z5hQ&>bL8m`liF&7xzsXuk2avQ!?mzQp8Z`$8dGA*Kjs2xSy$vaO=_og@F6=~n|rzX zA#O%Gm%guoN1&eprJB?fnv5Ay!Hq*Uc{atbJq^ydvW+PKw7EvelCkXRf^ME9-^dCm zh)3luL4C-#OMj0NOohQdc<}C;+N>O9C1k3K8c$)V4_z;n?AXH7kxa;^s&PXZPL@(gQzD?!dNu=(r4 zE!fQ#(1C!nhwNhNn8Oh&3}N!Zvwz1Qn2?JQQPTk99)VN`FFkJUgz)G@ordJ?o5>0j z;IO%SQBYbd3^*Lv+P@H8QNu3cyQ;|6n5`{ThjDqk2K{G96Yty5yoMyNX*WH+74x_Q zdRkzuvA*M)kMij;Ub+0gdkT7+!{OqN5|Q6{*6Hp)5TA$!CJ&IedJ}mlOa2?{lIeHw zf*aAZnXR!>aqqAHhI6;)n8wn#l+8hGABv7g(Q?0^HIV27S^J0R_gA*TK+k1$Bw+1t z8RFaRt%jCJi>Jss;bb{-LoM9TpIrLHdnpo!kgWV#K6r5B94 zlAE?17xVzHI+x@-JZ3U%(gAiW7NHU1y$X>5@Wp)u`Ll~epkx3H zd7zEtyAC%DHn%E|!q=je$;Rhh6Z9gF-yQAJ-C{YqDkt$mF9FogQNGyi=HxxWC58hyYbUPI>3#0jg_FmicI3MmDYym)E-%XXXTx#^ zD2PqYW&c;ZPD`L%9VjPnfZM1Pu7W3;V38g0IpC^>_JPXp=bos|SoBk^yJ$qqd*NeH zc;xIDb63W3qrisZHN{>4K~c*%^xNmiHn3^{f!=rI;za7qs*T2EN4}xj7+iG4< zjq3W0>XMIswG}{{1%zMQemW11fLv-|7iJBc2jCc24wLh-aBo7naAwvIhqz54ca&3? ziC3V*d*~onMYURyCFsheXP9aeJzJlQlf$8j`fr#mrdG7z?=5kG7Ps|304vVu)BB__gKN$BrG;oG>@^-|o29afYrAMWuW@HSH>Y0+(t#9hU zsFZHq2Br=g6jf_@T1vlOU6t3zadMxQ+BJg>SgZEnS6|kB+;e^z)W|?h21%-%Yz#ki ziL1!T{`ILJwGyB2;9rPDw!c{-oLFQ_25e55>O)tr93QLFolx6QVL8tBwDtO`~fHV z0=P;c(((gNO5Hk?d{oefl=_d0GznVivjDqYT`;=J3z8^w0e zQe@!!R^)2^1K@H%5#qVYFt~0>xgfS}5Lz<XZ6d>5`$dXq#hbaUQ~R8jmFt<&Ci(%&(A4g z;2scwqsmQuoHNlguNhSOQp5g*k%XCaXXz34#_Z#04ES8Z`rpvQP5g_+e5W}t<$;^> z4f9L-7QNLmD;V)ahe7gkEag1WhqLx?urp4iQx|oaJ{~bL8RYe)IC*oCNHwhLP^~p{ z`B025vmB0MUtOke71=X`U@ulbw-`)SmD29{T@~NX&9e6>l|iphlkD%b^qz@cD_JTm z4j6M1s2Yv^*ZvzJHY1XO?yl9e>|4y@p$li-09ml9&U*Ih%HpIt&eA0 z`&<-v4z|hLQ8$4~SBk@lrrx%Zy)cx%2Lv78<`%`_!LeRSzPE$DI$Wjmj&9Ig-jBcb zn#kSkTJ#*4n$oTIu+*-DGtvh(O6kv&=#Uzw$rac=@O}laJoW~^<-DxmD^A(=57&y? z+v8AqT?8y{kDf79vby(sL9fqv#|C^+JMa{pZ72S@UNnaEsth5rM<4ZkX&1mYHAQAz>z*l0Q6PHpd4_PR`FZ+T!(DeJsuhTly{>yPpi)9ll zS>u5n!#Fi3o42!vDjGVEL>$ZG!&?U|l={V$XH^H|J{~V>)L8T{zqml%E*5wtty-`h z8|J6AG9)#=5>)b>d(>QaosjG^z-E*`>f|E+1xiMA_JoAj7^iK<;VuVBUbUz}_)*Yk zK#RG9{x2{4l2;T{rNsqtG6%~-6{V7)V^LY;9q%YMzyOQoYNLw5+MA5v+qbGHfl2h~ z66u<2j4LnPkn+F4kbC}71T&0fw>t$J>%YxC4?B)WMzMsAZdKT*ayomC+x~aQ6=Rle z9OrB}F$UjyMn3`2CNZ3ZWTb_znj7_DJ<=C62>BN3a9BWjzqnsw~`R6^~jpjjPIfl*JP~ z+^xyu@q0`S&gsGCjn@T<-`0L%xQbyyONmE6=`J=TeZg%%o4#UP_ ztoFg>BiWQL%@x|@@|J+UjKl>=D*W)r0ss2JvH!vpc@EpCqf&7Huu5KR$^fO-1{`is ze-v!L4yLA3P5j+Cie;D8b`r{OGK7L>Q=;hk1yA~k~3*MV+P%-lRKT%aPK z8)@7Pma(S3z)de3Zv^BhpkW;3IN?%)Be9ip0kqhFHYsa;Jxfo|wKicr9k4JVrr}ju`83WM4#skUPg2 zO9c~LZ^}B4)p&7z8hlod?+4mEtexeZ+ixob{W&9T05g1sH-<@zF>}8gCcsJASaZsA z{)>O;mEzz`>?ag^s_DkYKF&Dm-GVJVc^7ZQM!ix_%x5Xa3rgX~fob+N_cg1G9nXHDUxX%_?)a8o|a)eOBfUDsw~OtHrM za#4KSV5$x0h^4h@gJOmv0>8pExj`+uDo6>jRIUZ#sS?68j+9$((fGmj{)X#XAYKxs zCNlCy2@e}Q47oo@w@p_&FG)q!^oLPF68)h&f}W>e`wK8haS6WjRLpcnJ`mrKcc~^% zxg^Y3Gz<~_>5=%xVT?fZxwEIkurI;wTxt$GU0x{63PRF^O)|iQf(ZJITbtpA_I3&* zpxn8AEX8EX`dQ#w3^Dbt+xTid)O;4EAo47v?|(n20z{<)I|U}5jWWS9;7{sv^U-0+ zA@5)uEpl#pSn5JKZLb09?N`w}%%Pb~JH$L%B=n>ehJfdNnW*(BYpIBM`OVFTM;w>a za8=TRO_1{&%9Z!)dO0X^qBDI1c(ieb*p&}gQ#P}FFvtZjaZ(r}4gqWYJsbBN34GYt zc}I>n8ja4q^hMOGF3|*<@MaUUaG{oHgS$-ndJoZMiy^SCRP%KL6!rV(=&V>@OKo}1 zaKYOU>4=^w5NG62 z`Z(|@Mr)ob6&^8Wd#toxoMCJ5j9(~EtqYW{B`JMa!x==b(G8FKv{+yZvXIi#x|Nlq zDM4x?t8T6M%&^r{Z$pTrOYIFLk@*d|&l-+VG@kmOV1A zg$RNs));^WH2cww)fg5*Bx8R1tISi7K}*{jl8j-^OCryX#aU{6Im0CH;G#4rHyBM? zE_6#vg#n3E9oeuwm0Vc0%wlBoNjnU7xgDa}rcP0Q0zb-8ED4h^`Mlm4zXRlie7rbX zw6wBfQzj{Jk0nC_2GTMq1}#HgCN)oi@BTx@09r1V=#Nfb@NNM=KMg3o{qV3G4#dla z#3E^>j?Xr8C!^R!gQHphvw(~?spx3J>Q7d%RsT*s>3a{ImKqky9bM0%(QLqV-^%PS zQ&bn}Z}z16Q)&-e5jL`VTSi5*^WpBHxrZH`S)=ioGZ202IUP>ZTLoJX>hkjTY__0h znWGVoM0j3yws;!0z<8Oq zPLRE6o(KnK`P#vfgO9|=rehI9%N9nnw6Sp&1lqCRIQ(R?n2UB`i^pLiMX*U$WgaGy z!&WQ5N6*j%iftGAXntwVf*knuGxU_}H({N^kmQU6k*9nB5{!4k!kM^oXFy}!h7yTd zQBp@c{5~eXsr=Nh@Q(Kt}O%99kr^O>7uH?3x^_&ZJVH{QH=!_LK+4$RtnD=z)gRT|J z3`<_3FXxEk$x37Y@3EPNcClAtyo{H0xcrLu`+9((J4ch%nzts1?uSweK-t2acko6| zR1D7=DY8K32Ovn?KESVpGAj(gYM2mS7;DE62HgvG%RBAoK0yXS!Zx^TRON`4*{z@! z^$ouV#znKv3pCE+_fN<$X?U#+(3d?NE3>4T=m-dK6+V5;=tT6zeK0o5o9x|;V@Nih zNG13e$=+>$=MLRMuY#JbCE0H1!fAm(%!{(OQ?_RDKw)Z1zlOaAWr!-I;vQNyJ4TZt z%9Jg;V}rXQf}`7pd!SWPnNxKYPF_NJrIsJXwN&c%#qhef65O=bsc z=$x=SkG%7XwRO3T>zZ00c0hq=AV>CJxnR6tPw=dag@^@`vHG7n6Y0E;I&;~^fnhOJ zFRpA}o=(T*ZuxV6Fl8R06nwJWIpK=jQ5OCtOzRJ{6$w}GzQ38CIS9tY#aD7*D&hdb zf$)VfTzI`I-j~Kea1bg-RwPp+gcticwOj49A*p@S`=_!yGc|6!q?HSPdzwrQ3likM zsb5v$Hlv!@Mo(o27HI`XrQ4emOb}qz5!kY$BjS0@xSI7EC~$HatVj(l$)YXi^Oswr z97?t)H}+|XibYn>tUTC;$CxZc8(=ghOj!7&%j;@@;{eK%hc}aYD2p&*&x)+<#dDB^ z#92FLVQ+i5xM|nYj$784Gk7xB5JV}_KAc+B>o^Sgf@y(lh(TtI0G4$QGO_2WMFsAt z=@JYLzT*%v1N{AvZy2FKf^N?J`l17BiaG-RVDdMFS}_Q-$9(&A}-}n=(Iu6(quW>04btrHhilEoctSHCXY%fMr z#zT8V>)CY6I)Ywg{?u@tPjF|bn3oy{VN1s2H3DMzZ^xi@uP}WX&TSbo3%MVUmjcbTC?7`A&*d@2nM9w?XLUDFM$5^@Lv9w)QUwi@v%Y)fWHbK(wML0UA?@=1N!~Nj3Nf_qk z+N<6~FeQg-T-di|YD?DWZ);OFDi7~WC4uwAg=g5BUyO{B&doU`01}>T*kZM!)U-U@ zuXx?&2O$Nskft#!7E>X9Z1)5NY<-hBC*Wd2+4b4ho~+YvIP)OkEbTXN2X|8R9_DG! zkmS*uodJQ02*NfO9HHr#C3Khf+2$a(5c~jb?9FH!VNkQdX=xm5AtYB`4QEY4rXg}x zas_m+g%hH2Wgg9+d*qD9=?T3WvKY-@9&dlabx|R?9Ge=U4HELctb(RHdI@=e1-Y}o zc32~^$67_+pfBoemC>a>3ob2hL%N>Hh@Fm(GiOF?OmOzQ8JuL}xrVJ=-k_i85MKJ) zLE<>US)7&9id|oheIs(Kuf@?%9TBZBz!)zzu~oAKQ+!a23oEMQiA;KqF}D1wkf@O9 zYvGxJxJO8y;Z&9#kR$arSwi}3dj5UvF0`^LXz4IU|Cg7oL&78B85=ew3dsUU6A!EY zzOzt0-Ox)l*AvsUu+9=u28SEHsq5rm63{YjP)heyo>`PLci**r$03xp8D+_%S3BG@ z6lEE)32Sh}kjG0trQRHh8-}EN@&$h%g8B=yETUo7)K$u-P&`U<`<}lgYZ8quRL<+Z z5{Fp{>ji=BcDn1fUTZwtW||NCx*qL_6Vu4vg_3&EhS^{ zz|@4kJ<>LXD_;yUAvaqTNTDa_mqwk5{ZFa!}Gd5HcBd)7(p2dNw z3#{R_-S{UjM?*dp^8nkb!P|6G)W{q~<<#16>k5Hxp%&TwDUyD(NE{LX-(Pi`Z&<$vO32i;44@P zg5(Cct^8Xod`#b3POH2{10R=%kLyZ44n^8|5?H9#2D0j>oh*66BhKWutm(FB_{ zE!dqiKz;EDd!^#jTy3U2tU!93SbavMS>~ zW#KOLv?RLCly~QQL!ES}Bi~QEQ9&?q$Wu~P13Ol4x7LL-oi>@5aUOcmVW)fwvswKE z)5#_Q$Qpx0q8zLA?||7&B2khnN|JYPUuXmq2RPL&z{{wFZDMxusI_sG_1|n*%XmDt z&A*xFqu>2jMgPr&B~^2rlmZc~1g*{~}jaYDVj9H{ByM^4DWv@&S1Uv<1^lrQ| zO5Z;A&+9>_Ua^?p)|OP7G+D7DUh5=MJ~kPTK#Mwr$slpgj$768cZ37IO#*H7e|g!_ z(*$S1K8+XKvtJv*>06{Qw-1<}e-0@fLTzUVdfRycZmfqE4;gc+8ZX_rWu!O|Qr;ehdp*s=H^rh2`;n*S}FP^H+!ZOf-uxB1jMc^xriL*vV(6ScG z1XWQKz*KD$;#6Eq-Jy%w|6A(N<^MyfAN!VrDJqwGTMIAm%`4W|lBsk^L`0^n9Y1bP zPpJ9=gSf2fp7sAjs0SO93Lul!uL{g;@2rOZ8U_HS+O2duTA44{vy?hAVQIV2v2%4S z55j#`mjpr1B~yErv^d6;&1mB#aytqqPuT~hUtbnfsIV$rVfWWJH*JBkBe49E4{D}f zS4!NY4#=I11c92#?g`9|2T45dJ(jzD2?ng;~-x2HxLQkG#QD`x8 zI~w=mxB73DxEA$CSKniRh_Uo6J^mber)Z&%%?XKdDqkw5L#>)RdM{ut!ctaUSGE${ z6zGh6c{b&BKO|T@vjZ2l>XqXk{aOB#7$f%m4+Bf)y%ULtNaU7-Cto*Z{SV`jkQ+vv z%!kFV_A1Asr=gmxx52Bs4{QB=DPv$xs~(q+p?C`vUr&n4*7p^&lh21SHg~IKtmz#KBk2-ADw+0sV&1@?1dzkT^?X1{=(;6ezqW}wK zBJfVyeP|n2_KsLY%5$Lyc9qynttoDox&(G)vt@BAw<-+0>)rfP;N zqIn4O{1YWr2*o4H-<*9}yFZH|Id*{L7^a5t-~Ct;{^v{d>)TqaqGwY5(ZS1eokih_ z43&+U75j2q?chrVuXWt^d=46d1!7AcM!cdqLokc~3)7R0wb0Ucp1@-YT&>o-uIC>2Jm6se|t5S&$MlACr&RR%^&xcmL_zb1#*iD|)S;ZEY za1zCzPD0)SD*jsMuc{vC9xpqV?sI$Tek@^~Kj1EeP2gPL7j!)R4T0c=kwupacBQidON?g>ZBds9hu6^6FW@LD0M+n~ua z|LX8@0ScCH4c}_O3+6sHVmE7IgN+LQ^|V#lNw^2l6Gu7n{A65X&fv!mG{Z$^qTMlJ z*5(2T{~0dSyIYA>I8loVG-Vc-aAk+MT5Kp{Ex=U`)!tO9qv<_<;(F0c7b&V;)f`cp z(p%A?b920v!X4#PAA8G(`wGrnhYE_L$oQi(n!X$fuP=nN6&0fbjANl@RIL!nm!Ul3t=gc$)tOj| z09masAXQA!N*6np)do9|M7!hD8uMl_nc_oUN;R6I@bj#WRs+4w*_E?ud*8F*n(($` z<=Lq8ZrBd2ueXjyPs3?HmTKO?+mj=zxTRCwR+h1HXC;Ks#6D%g)IQ*FgPR2v>Rb<+-7lYhYlMKy~rzpNZ6whSl?n zKPnk3oj69Gdt{$^vsrYt;R$92vdGVS6t3cob|Ks>!}&%{wqYr772)4Vaz=l=bdypy zyfI5}iea8xRQVr7<+8JtSWL4D0W9+$WVGaJ^}i0ao?fZM1|(sdXhUn{Yu$Yvmm!ma`W8SxQD$bStHQsbJ z>@z~l$@Y2P#t7oz6Jt^nEdgh?H<&3AFNtx>C2(|u6B>}RA+r+ zl^$~3mGxnZ5&Lw%Qg%n5kDNBdzGnKsfrI;Jq;_ka+NF7V*FLEkEUT)GBLy&5@uY#x zfjTg(9rnfQO2v))f_7LoD1pP?IBkc?y~k26;Q;&6@ZoqInZcY8Rut^x$Z>D1;d2*r zX46rL{YvK-)~9$(f@Si=D%4zI&AFxZX5Y`FuOo*)={NS?F6iZnx&CBHuRk0Vh5{d- z1oqE*TVFP&Ij$1$y6t=eJ%y{=kJ=zJByQ8(P&)EW7wy~;ZFQXQz57mkY0{`8Y-Mp()2?*GttDp~zi)8~^YP{J-W@y5E-kBLO)gAOD z?F!K$%z{{>gRM6Vz$8KwSo#WCK{d`=f%*Ji!I)#>_^e}F=e)`l(P;YOwqt^1T5eJ- zn`vPe$~t+tS~64YlW>Onn`%zV!8jU^Qqb$@|MIf6ofQUcn>Yb2$#gi=g09&xOhDxe za3Q_m8f)_ahTMe~%-SBaAwK5PM5xDqPD9HhaOvDzo?g@(Si z7kOO0mH|IL&^6$iM18vNu$ar&+ZpDGapz z#})@ih|F!A%RTJrjFI42@srZ;CdZbVN`Ai$xH&&(pLQ3)a(OBdxaCe$mt)88YNEtW z(b`m_~GFECiLoUwMu6B&ie zg~Y|&yQ4B%EmSDlb=L-S2mNMv%Wwyz(W?s4`e>vs(Pb&(QJFTIv*|rhs~kOM+X>DI zUlA)HUX43xoH!ipv*})4Td33pg5()j?cWvJ(Qgfrv`G&`^K6gtzuX$-Bgcw$YLAp! zR7%&)%Wu$=G@ht8Ify@QR2YzY5%bf3@wO9JEBrMYzTXz*xW(5$({N(qZ_c?>7af`0~Yo!fh)=^l=aE;@;LgO-z6fHR5EpMZ_ z1M9R8K}n76dr(o|vk*#oxaH-+m6!Tv<0KqRz_vzCCnwpVhu{Cx36RNuh&Teaed&u$ zETUy!`XCYqzgK>b?~4O#`aSmXgWE@_f&UcO<3DrSFlN)dA|GAFDMiXtj5b@pqCOI^ zZp~T?Y(`U(f8!af`};fpUBz6VV=IOKe(nt-q@t;s0CBl*b$%S_mR|T_(0yR}r zivNS=oKl~d(1B&9;qa714PM-8LRa|P^{3A3g|(Wp6NWPHF3Mxn?iIV=cm`LF1AT~8 zzm`j}ENbO@9vLI3Re_!R zfW#~ z(S>la_pRFWrN@9`(7wa;HB2$qtS<~AOjF)}3bJy$PLov&v^hV~UPx1DkG)kZprk$j zL4~vd%I{wwL#WX7J=%4d4CJWlVo{#We-9uinHN!}p+7;;6ETaF7i(*Va@q(x_1q8!*175@QTl%=$}#I_Bj^F)JLSgkX)`CtOXyF%SJgqU4ZTtfT~Ec6u0ISmVV%9R{@yEJSB9OsmfQfL->? zfM5rXQxpUdgtRMM%DG+H{_a2?W(7sM@C-s+(;<1=eshAR)+kq=!I-X4xEg)Nlf=BZ z8G~T$$ULSX8ma7wnzc$EjGklApSKne=iJ7${eXJpNZL z;0%Fm@HUkV>o-=Zc9^1aR$;-D>Rxwo2r zRMDx)SX~%br^4az_PubV)gL{!T*YPnglj2zrBAg;OV;3(qE1e7nHWmFQ3VCaW9adF zYWDT9Dwr+(7l`4w_t?jhrJ)DSwWN$p5y5lY0a+U$vp9~{dZIpd#olvqa&O8 zT_b9ITlrasPNIa7s3&-zw68w6kH|C*Z0T_e%M4_;~B1@)1x>E0#3^x#0+1$;-3* z5*srCeb7KmZRB%Qlh10(8yQyuLF5dkF;lbtf`YU+DX$_w8EXiZ7{gTK5!Wgcy@He} zGQtdC>nv?$q1OI^sI0B#8bbE@v9AN;yu|OL2jTarmiWC-um{^d2y1RK<8`^~)^yMH zFO2zeG%U5!Py)(erahvd19QuvvGfL{TVjQY z%eznw0Z9@#I*eaxzUy+fQZv-*8U$OsBw}BLhOS||=dxg6nhPADL@8!!phINIHWRWU=up{vs^zqbkZ00UibmM0^>RM4=5Xe*T|CGHAn-lrE^?ug!# zW})ZW6y+ttzxNQO57FkzNcG(KYP88OB7G=zJCUZ!23(TFTlXQvd)o$`wd zUHq>56YT+D?}4xgWIbI$O8|(Q)sJ-;1UkAwB768+Wq25fiE=dtckj-!cVYvA7-o!Z zIJFrVLct&_vZ%UXV4c7D;x3m(lsQ|BcRB?KV;zAo>eN&y3TJ5j1xCWg1);aUu3~PE zZB!UbcTvD6FT-!vAz-CHBA|}sAJh680=C2w62^j!LE0_>ZrDw5jBX8iDI8vbH()&fm=MyL!4NXh&U@1>ZxMxd2Km_P;2NRcMca@ zYnYu|Tor+BI8UjpV%ur-md1=&m8J#PDGoju0ZPBl%As56+nPl+LG{$f#-`kE(^5(p1g+=&(+)>4 z>!|Bex+8Mw$qn&k$bIF;()uDp4u=KNPS2`&f3IJRHrd0W^!>cL193aU6}P))Oo($6 zyFytx7zN@+pyUj{C|1oF(@bhI#xt(2b-)C$8e{}W&X_iyhl4xY@2+(bVUj|*rCv`H zO^Ge>j1vf|iP_6qw}M*=Q8RKsKWc`C68CWxSr8I-ja@Gk7ZbW4=D3V;hcw#Exw(!7UL+43Mk7Qk5z#Q zM&OYn;lH(ddL%O(7vjM39hJ4NeUc)8x{ltIKUVtxKHU$g2om-Jhs(%hn0JyXP zOVLXWD^h&1YN;;7lxjWZ>-v$*#z*UHsC|ouPbm)D`r!2twDwx}a)P^IvXbXl+L8$$ zHM5f(Tv*4caYCm_5AS^)4Wo15A9;4#&`3*NHqHZQ>5v43S|#O+8ggNATq=lbzzBIt z5NoOx_@Ck|Q(*!e)ZYiL`fjCsT;`(<5x;-&fn6Q!eq{Q3wK$;_B@jAww~5LY{@Q?| zP%3&+{GIYx0(hSrixU?VK<;qbJ>lP07IhFl=r%PaQY}Cw+RfO;XLI zOQC27m!KnBxc)CMyR90=2k3m*U_YIGIl?fpn~P8r7d0hnV6Gb)=p- zuL?J`1}pFcEqQ$D$E|g`pjFehpxg|r)YcLTx}M*n(wIc0V9_s1)yRedf@OayM$@Cmuuk^ayk?LyH_GP+7n729m-;8te(hnnO|FycEkwZ< zyoi9>X&D21vGWnQB18rCSo7e{Iv8~o|75`@jdVBh zVydOs1mhr=Fv@?me~4C(OR6n?KlnR(hnFKcJQ3KtgXc=kMO5=DKIf5hBAIbmd@}7rstQ^p7+YqGP^J&#CzxMk8C;s>xkkP1fqD8 zrCPQlP)W5_VD&VS07uO+Nt2z5gi^w)j|u)(V%vk8Hewk}tGE$COYp2+%P?}%?^_O? zUwjeT*GDfQ_YmbaaqS)0#L6a~KPcll7n7+{h1CLrjc= zB{#I)s|aec93v`M=0*_O{+CPqccdkx?wo#4Xtpiz1E!KrSl7$-I^(96-AXdSqfZ1T z;xg8+fl~_KD0Nj>CDgq{nL{JC&yRssa}gjdBxTxv<%OmQ8J~k8!@7)4RMWl?oP&|< zHyfMnTMV}2_!lm*UoHz;CEFU00b@2MT!GAW?=of61Vq{a6fC>)jG<1Xjnm`0DsPx_ ze;r!hJT$0ysf3M2yo3<;=O2{l=310-N#yi>cCgG!Iv{IPx_>=krW29#1%Nu=H+YxD yZ*Pwots%J`{>hWu{VKeW+cG+sT_5Fd?wr)X2!UIs%n|b==1-AseULBhk^CS2%qdR* delta 1824389 zcmZU4bwC`;@-^=6?k)-L?(UM{5Zv8iarXrlcXxLP?ydoXdvFZ|0$*}--+TAFzx}JG zy5{uh>Y6h>y)$=<_uxA<-@pk~JSL@cN6U0(iardH8%dZ` zJ%kip+;!X2Xv5B2oT}O8Taez1yX?I@bZ~v^6-cYdxOlK4$ipgoZ67Stm+gF&=-$7J z6qF|);#QG|h5P4N|6~CS4(zWi{7S%oe?S&+TquoSsR7D^6GJsJ{xWRADbY25dHx&o z%RmP@L13Z$59W6}@CuO}bS26U2RsEwfcncbgMkn9gMb6g$g#oyJDL^Z{Xc^U|9OA} zx}xNTMfyEeze)bZM-}Z47MUb3Ebs*a{!jdWym?Z*FhCZ_zx(~|t%gDQcbC&Jv>=gk zENEcve*%r6X@O`^u)y{II8Gq>{{WvrzDMJD|GyFb#eo%`7DNV(0`m{x|N4!-Qo;ad zpb-9KZv#sBk3j}4_{T7Y=J{h}Lv#KyextE;g8vikudJcN@crSS3?ur-_)WZS9|9WK z3-dSrGz{1OntFvHq5rPz-|71IWYK^`Z6QfPB3HZ!{~-QXS0|9afN%beFe z^zV4(-$cH`qX7ls{!X<#oWdU>nQ$Ne4C^=9St3E`f1k?=&;2LhH#iwR0unG6{=cK4 z#dP`P>iCa*|3!}#o(^P$f%{LWzw-TS+CT{*zvfK^;cprS2!H1z`&ZNU1}fA)djIQ` zIRw5xy`v)X{voV@$o9wht8Njv;DD8gf1_O?vj1tS$HWJUApLzs=pr%yfrvq(`(vn} zl7X%a(f(Q6e|7e&{y@mxXsF14sm*^&*cci=l>!DDM3#n#_HTYh$O3;b^N=O}7{|!6 zD8JGEcin;N+YyODWGQF_c>jN4{v#w19T5v862!*4AmRhbFyTOsLNKsE6tusARME&mDs}w-r(jjlSpS?hgFz0e zuE2)}*8S%cRtS9H209#Y3k~H@$n}3>>oLhevk^$pKn3)_;eW+~vhC@SfPnvau6{B9 z?E%dy2>zcL{R55j2lf|QDmwu*kQn1{uwPVuIsVVg{AM_-EC>&b{ZHU;9sxx#|EI=g zFgX613;aI^(=hnJ1WZ&AS~$l4A@rw>_&ii~7Q-LIdkOEDi|0iP!*dCfas|w?e=NX=3VfqrK$RhsO*_w?7!89jakI=+m&&NQ(@aL-}fZO)g*Rlb1r5FD1Tf2wrN-yD< z%D3Ydo-f1(uS}?Kz|2eMTS5CA_A1F$7l9w|0rovJQ@3AV0y0URZz;L`W8Tca0otAn zxoi5=QC>pMy`p%94-PF*Ub1-K^IRd)&fD7#zh521CH5SP#I(0Py7uatj5gd!*Pc4x zIh?C4LnlKPV<~o2Kg8nV3#_}%%OZqD%~slRMrc*n;0gVPm7=72fJIVT=8jOS{z%k+ z3BchjSSE)+sh1GhAdWvBdreD~JE0F;n=5j^v*hUFMvpJS9JBN_(U4b!PaeU*#Pn9n zbD_&PjyXNKE1y>2BRGv2UhrnC8RGK-XTJPu#}-!9s|60=0V;gEzRNe|$X?#_E1mCB zy`5z}4>3Q6quo8sP-4zbiwooTG2>2-04OH}S-UuYu8~wYi2@eI#6~NfLYSaBC2?f# zLG`DfC@9EFD<2Q7|G343{^slY;tH@Ip53vakv}a2j!BrCIMV>RKn<4U3u6l66?}{- zTa9dtCUJtR)VwZ&tJu6soW!(RCf=!bj6B34?bFlM~X9;?B&tXU{5`-wu*^wJ$)Cf=#cA}2<6 zgiV;rL_tVPsJBebd{a61WTHqtz%4{Tm^gN*fRb-a3I-T?V$7U_MrUK6-`03fOW_+V zKG!Ccl}@q(EVS(k5KLK4vT8pg+mfcJ8bYxiqnpW)w>=rNjGF2!dsb}yPRr)or_1~D zK^btJtsPg?e6?s`M^nk2qBNPHC@WZ8>N8-RTB2JRc_?VBAgL$Jn;iWCV8KyZ=Bt~G zvPH?_i0)}{$7V=q--LNU5dPsCni~W+*INhcLkV6`?!wNKEf!wUfu-xOv2*ek41&%w zU&_q0>RSy_bMa!Rl^$q!PfUYEQ>jpPWT)k1a1_^h`4fNesjr3D%@MqXYQc{hiGJ`ba_N|Fv_yqq}Mapv#C zMJx3k=v*%sEk@CGbmC_}es@VwJcTVRr-L?j5Xg~PIE>`UW>BefFbKm0b1*wgNoAPW zjRcza?+fm4pmC?9A5MBrD@U}wbd8PusNlh(<`f4 zg&vDCR}?iN*~p*+kdPU2*t0{KtBB&7n>O!OH~kY`wz;A+8_SF0U}F#pjjt$>uTn=hw;0T?oK4Fxw~osE z^p(iMAgLUR8~qri-T+%W0-3=)%>_#j3uvMbaOJW#P)dpm9>MEIkxAeaU>d?BVhStKoC=B_9{L%X7E0RORS?58z?Qs+(o@sW zq9V3zd~*>Cpg79W$Id_-`%=fin48iBq?%YVr)@HM%3u=ySX3G_2=&g;REB>LN7EL2 z9nQzA_MA|-Bpe>Z96cln5_u-9K$ToKyVf7&OOZQG=F4)bxR#=zE7X%ahiVOfr#dWW zeQ$zWbFwzh`NTHysSX;}bXD#91qs2LAlvL`2hXDjfJnDn`e@A&=p#-A#0rh5K`hK~GCa#>b?Wx}eFn!6w1}O!TU%#U#p# zo3GtE0Pm#9d^vfODXGxAjGCS2)rm`0hN^eW8055iV@e<9z5>cO?4~-sfQ*|7@i6q< zG)Wp|45RCvdv^t~>>)o8l*xpNTU)yNX2dvVwa+>4ygIQa+HlYh_A1Dp>&~#>fj0;< z_>h*mt}18Ts=Bqw69m|m`JuBCm zPE9FMvbAc#PUzCAI zf!6>nm}5&*lvgx>#7{QcUb}s&K;e=^$f#R0q%X%%tw&0Zgm!~RYooMnnaV1|G~?`^ zkw={Y2p3{5SL%j-;F@1oaa0PoRW@g7ZU7WlJa#u`YI zp4S{32OhIA7Vt`2)Dgb%{g1o|qnAKf{{YmT@HSH6g3Q-?*N-Y;t6mFc3O^&eRuB+C z-!U|4HjaaC*nC?+b zXkq5l0#lZqp~%6&7lFfdw)pD!kU^#)xl^Vg*L9|#{4g)u379h-%7;%v~3> z*fs1zg*J?%R%4HoioG)yic+yfqRqIpbm_*o#Ya#k`6E+AjCi7?b%O<%?ThiP6$vx)yCfw(W6u+h6 z9%~CjXj{o?f+DiOvd*T)W8eXcUqMWC3J=3KR!S{$O07MZdYDTzMNd$h9^W~F*IVm; zmV5MMqL|v5kfhpK51VZLGr)GYGOuN4df|iY>Yl-(3LoC6#R=eXJTRu*{(%IJc#{xK**FrN^|h3=w(zg z`7N~kMV$A*uf+MCRy8$%Z-eQ5e-XP{-c0tw5!XHZs|n|Qbs=Y0@B&k^h4Y$eKDIh^ zale%{#+}CKxJSl~dy)e^wh|89x=a5EQ>b`xe<@SwUTVLMT&P0>R!$3+Jp*>nk{XZg zN#hcZkWXvz42S%a6ynqOkK-du2j@69q;<);b;(;snT;Ow@00otlV3* z?RU;z@O=t1Yn>RB<`&lP>|n$%+sFx^+@9}zQteEI!xfzA_zC`g3Eo>U2So$#vOPc> z^|AS4O{AJNaa@?I6}OA!`z#}yiu}y@H)fYM29?CZF5Yo~vc^L`ei`x?*^kWWDwWU- zZoJM=XB*A=BE_cTBPQB;4>6{Alq=~XlDz}No>NriZtvmKP-l8EtSt5OF7A=5C-76a zy&{|Q-FU@MqmZiEk*X&MbhK^h8t!k+E8@_E1NC3946fN9hY&xI`v*#0%QGClFOcsF zQ4A6iA7sP>lz>*|5_4Y*GFLM^B$#TnCVLf)fc`F!5773$@CSG9O3Qv(3ys)o^PZ^x43=N1hzR8 zbzV0{fYSt{+ceg%>?6^Cl+hHCNqWy9M(!`o6qwLn6(1Vgp3 zLad`htaC!O*udL5P}k{88~G3hDpJ;k!rKCVHK;RjF-x=V5z?QC7+6=V$w*CH!d@U>O!NW z-Ix6HNEZ>UP_~aT$0&a90U>`lWv;|FQxxFAx}aK!lG_)`Aq5o~dZj_48F*nGW54ws z@OuaLzt8bhU|@gtZ+`DX|9--61>wV%qy4qr+WJkL@CVc%Gw~+_9QYqI`;}o99OZW` zqXcq_4>TB(65egSHTZT*oc=yTH*u|wvrtl1CBzc53`WIf zU4?;4wGh6~wim|E?IVny9+y+2Xx-MO;vE&ma-kHZXiKP%O#o6509;osB$pr; z@ku1s$%(eOOg06krd}~sv9b6XF#En#^g&H{GvN-irpWR=>3Abe)28Tq3h_!EF5q&O z@%4*yBs(4b#IF0-#c?wO^Q)u1Pn1uKrb*-(Tv4jg{L{{NVdhh5U*l3I%#-hCv_bib zDy(550{kVzE}_=~;nJ2| zd(myuC#>p*5b7{U4T}yqn9mAvnviHKavAnK6G+mn9_%R?#$Osy*r%)U$DPDn$`InZ zLV7BNgk9JT^ysYOSnbkG`S!SUMi3di@q*~Rs%pK#r~{Fv&0|_peSn= z+JJ0W4hSs&*+Yig=I0y{P8ua)Tw)UgqBt_ww>p_>ovJTJ9m-RckB>JDO3m;3W?Ea}l5~{@&@SXIpTs6FRzTR=+k0 zpIe1^hmVZ&UH;j6)Df3mg#o8SwxGB9?&_qcmf`_$WfwC&@=iu5vruS?%H@dbjH=`K zv=-Py=4Y`cIRhrqY11UFa)yd2G4?hp~!V}tR4F{BfzBIbT zlo2McgLu2z)Yt20F6GRoo1gKWHp{7QB0RCnXI#g)r|NZ z(Rr4%9FrqBu6~A;lxcEYMm2s;aqtg`vXipx^Bm4PpE6LwPQ_w}CjMS>hIP_A@}nPA z;?)3RO{q=6iZn%)@h+_6iN`$i+tg@iQxftIa2NIuRYo%e28N+;=xYdl-8tWedo)Mm zXAR&kzH`-X_Y&}F&FwxDZ$dEtv<@sXZQd@nt*o@{Xf^P#H@~8F-*Iql6xR?}9xb&R%JvHVTk8KO(*Krv5RoL- z?O#GZKH0gW^h?OGkU-!izl7Y;jS{&PN`W!}`mbiJNU!EgC@?U+Kh412YD7?vJQfka z#s$+1%b&}tQ?UWs79H_Gt*Y2cZB>UUNUsUEH~Frh4{i~0eQMqG#GKontD7$7_>ukP z0#|rD220vv(P8?raM6V;vu?v4N_c<2JL`xS^gXK)@bl+{16auwOC-5(axd#wrKGlj z^-d znmrfz4IBQNGWm$=X}N?}qG~NnYWTvZ=e3fS>gCW)M^K)>7K{({ueFxLv2P zK~#5oTucMqasnqytJeh~HN2fvR78`6sUNI0Rzh6lb=gmfgdI*ZJ}wdke}6RwfOJ_$ zA2l}XH09e&OA9_ytMgcQ+gD6*8e@Mugz#zDa?LeB{~@Ky9*eh=Be9B9vCp#wm?^pv>b66+jbyCQ}bMto;l@9xr-={SBnx>=$l6tcF+iHO7gd`_zf8L2!oR2+!qsf0OOXUUy|#o zcWf;he&(?IRoEwM6ZtIAIraWj-MQ~@ciD((n!-zeZQ+foGj0 zoVCFvU$N?2x{H;pK3wu`yevbfUxDu2Io|E)%Vv2Y9r91DrpJ@g>2u0bsf(b7{?tW0 z5CVY>MA+<24s!8S@8gKR1$fH$LF3?Fw*<*Ez1xeLys|r^^Mt)@Xv;SWytJpN-KJ8a ze}piUllg2JfT1Ml6Q5-j)nKmH3y$pm5*RY}w!VuYWw0$tRW{?+Y|c}Fb@lkw1#MB^ z!xHMs6h3eIjJc4yb9R<-QIyM$aVcQ;q&te&2AE#94Z&BaYFbT!3OGi-Tq?|I=4jo4 zy4|YIY-z5&1$C#-XGm#nEHPy#4|^$mHY!Fff%I)(3ON)vjO^=>GGQeAop#f{@|;pUa49MZK(!Of?3IBshDeao&Fo{-OIk)`-(v6sku zwGz9f{5zP&FSb-UG|C)8_`_QfgNDZ;%Awa!G6KJLGrBN_KLbc3KQF>>;SS?cUUQfg zUc>f3F3G$sarv)}l(Gqvw?qd6wpG%04FV{ZO7d1W^E(U?v^~<9wD~l?BXv7Ac3`{i zVNV*9tdy0n_9AbIrw0a8g*GIqBK}VH%;B!0W&m2rB$m4A=YoBZq6l->UkTB$E84iT{?>s#w{k(O+Vk3_>?W#rwS| zCI+zny(-q)aKY5XdfT8mDOrzAi4O?W5TFkRE$TFG z%xCuB;Tg!?0kU^p<*_T7x=Im~y50?`!Vb;h8v4%Tk`hY`e7|tll^3V`WZ`|e>N4|` z9q%*U{`4a^UI=_PwpGKnHzG=%hZ&Hnif!SjC8~AyDab`hWiKd01-w=#utS34J7+{t;t$9O|^eI@CIY)VV<8g+7srAg-08F^m&6 zc+s&Es8*gpQNFinaSFiF>GZ~5o;pd@CEy^oZyOm`SI-JEJEHgeME^K|ijP;^XgkI0 zowy^{km}Ga4*{GX@oF+jgXFNa@IN7Mjb>N*tPeK96If4T_|Lle@}(zQ*=N>P_rNLd?b z4u{Qi5^}~Sc^_f6w%IJ}AmcmDx*;Nv>gKjLA7PLjN*V1dCz_KX+1CVs0E!Z4aC^>i z+Eo@Ag&B+IbQTs3{Inj;(5A<(ZmzA36MXHMNb`I{SWIyu;^9&>z1C<-mJM-Ov!@1CS$bOmC zcbR8O=%j2Q%>*u0l(Yh3Gm3Jai<8P2q}&_aS6LtH==oZ$#xhaz#j?mJ=)O?jn1pmL zxGN1vVI9YDxEgB}I&sT;l=KBn&2~QoEvgKfA;JBz+(28$C-ktp%Ump2}^Ff*f z>@*0M6S-&zN_h+(7ZX#~blKrZa{DJl8^2d>{gxKADsmV}pPg)y5$8JP*pjl=(XziH zmo~Z)Pr4rV4f#1>kVdjbikM%U%TsBRTu$nqn}W4+W&zI$Cu;F^^KN?9(z%p=Aks)D z?)a9(LhBTa0IsK zfWKcL{6eD=!cxv0UY1us@(Hb6zlw}2F``G>fU}U4cC-+4bX1UvQb5!SyeCiQvjmc% zYE%#Tn8P&{pl5(&jsHsUWMkOnG0@b3QvR98zbLoXgC#iEGI=Zi=_mW<@Ow6S?fk^9 zDE%+7bmaGFY8c2xm^WU@K(g?aIVr8QqP2QZ7uKRs13{8TR9RgS-wB79fk7d6S?~iM z&avBZfY|cN)9ACTGuTe&ep$tKKu|CO3M&mRh5!-;Ah$z)Wxno4qYr)i{o>Eq>Yy|7 zJeKw)c^}f82gU>Zwb0<$*PohKqf^)^H^TSqXFwoSnGV$?RsV~4J)a+^LSL(GIIcFxAYz6yCa9T3wobEDb!;A;dhfcw+F zfBRtm4nUF?4STqYmwZSdYax??i60-;WM}kK%p~pGAp7PKE#sO}s&H>dO8(DwCiS^q zEv%7p@{{ufY@KkRM&&Mp#GZW#V9v~GzplI|(9>d;t!(^Xqb&Zp=*%gZyi8UA8)V{004cL|J3T&TPRUl0ODx>28X)G?_+?wT;S?k(e`qci zii&CtQR`{3G0H?FkA6{jAZQ_QYuU*IfiL*r_T_MNsdrN ztmpdPedw!;LjS5|RE9T%C;lodfFr&KbF#PK(tn5Vy-XZBCu4uwL*MSe4Bu^HV%r$G z@b=240imO^Gvcv`6!H!Zp-P!h_PhoO<5Y)wtKnWRu`}n2)=GJakC)Xom#-60tp`ZT zGk@|&2K@{zakf)CQBw$>@Sl}AU(0n<{|o@y-o!UGpV#PR<@D?jVl@&r zLjkpo(Mr#Ho&W0>9e<9;g#0l;RZs?~rzCIB(pr0NC89eY3a`M0EV9%vxLCPIVyO{r zLBU3lqRC$ZD;I6R;;c~J20s>u2&`Y3==Gey+}cz>Rug9C|`Ne~h!&stS8uFwJDD zeZCwLC8GG;cfmqVS>%RAEQ4_YCu`HEHCK~Rg)q`T$yYBNlZnpF_hkJH|9fTjzw7DW z7b6|&Q!S|8l2VXOE>olvobZxO>%)3%=*UPMJeHk_+C&0^_^xh7#X=s}gqAiHbiV_PD;?ialw#3G?X!mvCC&hA<9o8tZaQ^oU2`l2bLlf;Nq)R_Qyb~vy7VW;DV-K!xEj%{Z0&`t9%R;C5)-|b0Q zumm{C@>AjxUd-4oZjYO{Q0W`Ls0NafJSMVSGks*y!Os#bi0qmL#D7xSu})3?ftl2k zM2SPxOh^G|V-V|N@2>A?*3BleI~ABISX z9!78>;<I4EF|JvruLqD{;t>D#;jF{{5v$X_cPK*A!^1< z`pV#jB|%BcH<~8;Mg=Hs^;p}~cB##Kjag(Y^G(ICa_4r7qc~-|*i0q-EER&?{t&-w z>UZUcLVyv2$^Kf`|Nr~tp^RFhoWH!Lh#=NVLViGs*7~m{KGCaErYmsjp?^pQJ3l~X zZ61aRO}B1-m?9(QkciQTk}w$a2fwo9L$ z!rHvDw*k4A{aeoK-o#5VL}I~-*9sB$`#PZnMH~emn)G}9i5&9JCkbY{uX!SdyQ?Ly zwO_rY$dcV})biXG8xA6$D%Iz+IZ6@KcwGR_<~uPbI^Xoj!(Kj4V1D7Z<{gjyoSemZ z?kn06IB;Fi)M!9{bt2DzL!)QWHxBFNO2xM#=caGJ>?H89_`M@<{jQC*zjsDW7|54b zKv+}!|JT#yOufo$!lYIkBHLf2jLRfr99!ptmPfh_+>7zWe2H7y%?ZCtV>R*Q3Ei5aSH zT)?PGy9{g}JrT>7OOx#e-qXzXcGgC`UJ-GMfEzDlTn@d(t9PokI~a+A4QW8hEoWQh z0UwSuP|hV^(`=fG(L&>NS0+%JDYO_+T{%#!ZDwhN34?4qhzPUmsXAZ=c^SMU5BLCm!*S1P(x6+gGaXu8L02ZrSar~?0JPeS!pK9yi!M(;H%Vz*i6ft@YP#zPYao(Mu<-UthT!^ZJH~UUYJ@^u?;MMAc-ytmvWXxZmINJb+$2AK zX{d5wm096Jlohtf>}e(guAY?vMC9jO1@V)c*vv8Ti@y-eR%*yS#YhC_rp1_1k2sg4_oz20?}Z=T#aws+7ntA5MWgLHNvQr zh|{qZU6;{5$(PZf?UbM6s{z+07McevOl(7MLm!H5Ayx|Ew5mNZJ!;C$#WNh|w!}}Di|K!}r2%}A_|C5a_{rlkYEr6vLMb%f5{@sS}J#)`aUEa5@7 zs#xYtqtdpm&*^z!%MqHxb9y96Z+P1nmBX&ql|2%|9cwh$nK;t($0^7vTj7>ZVEQ}} zBt-orU6DdaNiKL+NY{mSc%^NbhqT$@g<1C_bs0KX_F-!NdC(%5^} zNW%RssyCFUE#MpG)8~XY{wF}v8?F*8G6Nd|D*cZwQLH-RiE3{d+cTuK?+M2$-SMg} zxEPZr{1^Ga>##@PXnQXH$x(;v6EofK+k_^Cxaj8BoXXv^=@Z&tCbIi3HMc1W0G|rO zgs#(gPY52YzSQo3Sf?pZ@UJYAClG;ZpRpi^%=ih4kKS}+!*i!UWMJoK-bn!t1UAuDPg0}nk zm|F)lvCN_VdOLYO_LHp<1Q?hgC{dM?7BoFX2n&eSI90+F#de@ntP;hxd+NBMMR8Xk<5exwEu%=- zMbXSriBxNyK8H)m(mJMM31wHHXD`RRIk-WLZhu8AnX_ZQ?b19ZZgoy+(T8E~XlTvc zAGWJXHJUZPtL`WF5+0NT*Pa+?54Dkl5eB%b@GH+UTbEAi7S=3ylVFR*k&p_9>q?gRmwHEmI<`wJUmKGF3)NZjZvy=0c-T{ZxW zUfT!ftJ&g@o(lS-XB^@Owrog^xRYVV`}9%xacP1H>qy$mg#mTqw4C}Pk?;ve3#YJ1wdNbpZ{u&Yb0y+e-4ForZ z_%#fNAo}A84<>65=gpz>qbHRiMii5=T#A_`cT8Jh73=cB}nuH3*W}-xN%a=0O zr_esW&;=3L>vWNayiMHE=6-VcWd6XC%9Eb~ld(i}QWe>=po;hDQ}-B?LwjTxbngWG zQTCgf57ovZq_Hglm)|n%#}XdOv6a$!^wK@bn?I}7>#rl04wv4NdG!EZ2_p+5vt(gy zKD5@+I$kRqKruwtRTh!7clU+(hQ+-x#lf(>$o8U-lE<538#!ThtqNLXD!)5|8Y2}F z7bp&iy|adp;~~l(g2qynea^J1Dwte7G=N#B%YXQ5I{ugQe``pGX`CZJ92OXu2PYU9 z-LH?w@IWg4d@z9jZ!U&(;C(fgmV{~3_Z4YSF<>AR;LHL7Rw3zhgJfiapu{lJ(csE> z6bzCQ#}u+chl-oEn%-$v;`G^T**W0`e@tLl>u73O(yDH0(dz7Y-@Ig9eDIvh{cIWr z_HAVg^`diP^`hOPllyibK>ton9C$%&lA$pwbgA_cDmEpZ#&(U5D$>{=;!x(& zh}EVJ(-m{k@+qWK8yRG^OO^qa<+_<_~8I+7m0ZL2ftDksf;`6J(x1?b%sg?b4f&26u0($M`0fcj z+04bg{roimw+In46&Ja*1As^O$<{_kCquh+c}0@`XfSV$gIqw+p9Z?R%Aj`S(#ca# zrQn<6gS(A^Ztn?hwh~gCj{xzsQmG$MzA{!&k)am!C@UozxPaWlwJ&v)w;SyV^UqWo4`zVqST$D=I3La6wSOr2+bQADiv`OMLwp z*P*1tn^p;Qh9{hgJMdhxh(PwaKTV4_;1+*(*d)Pmg=&9mz=GjUS>*%+W zvQXRH_&U|*Vpl>iH~?6-4ti{>6scIrf#t`R#bz09RS@j4@!D?>n^)*q)adU*31b?b z&f}&k&${^-*n<0U$&2Y;u=nvwf=4_BWZPeqYgr+ncNJ?Td^9vWPITxS;v?^MS?dKiB3( zEMh01!*QCa69pqsYVh5`KV@*S;g_Tq8gco$AWQE$oGo!&I>*nDerxgUN^&MZizn(Z zq$pfz<6rhZ?*{BJw^G+HqID~mMk_E6_Blh56)#+X<)0%MPI7>+(za1RJWy!B;-+WB z&kQ1}qg5@hDz=cqgRIDH&pmD6mo6iz-Q1&0fqeco!v34-eTB=FD4B;F6l3-;B>@Xd z^=j)q;}Zc_yjC!~taWyd1TkTh=dJ5pf?)1+>$K z%sr=-ji%{<(^x|9GOsMdgF>-%*go!T-aa_{hd5S?#% zzoF(MTj+c3M}*K?jYpe+$_4Lkg5=7;Of$FMbLliommgUXbDZ|ABej(qv`>N+8FVOk zhFKc<0|1RjugVXho7VjdMj4>-RB1MvJqnGXJ`?O~%)VSFIxki+X)MHTfzJ~wyGwl8 z>rHV+8P*NoW!%0Y-aSxYg<6OXMS_v0+eM~6dkxS!DJOW6C_{{Rmh@}W`!@HlY=?yv zHhMuAgyV1!y67?MlJ7FSxTVw{oMQDD^lYhE;Q}@a=ii@bA73klI;eQzUYck4=*5-J zbKbd?&{I-rrqmRlCXB?rYp!&K*RPmAa19;jzTwvFh;lD;t>-YLOuKH?c;x~}?_5^+ zVgi(lD^Jw)64lI1Q)*~!eP}oG<{KAl=Y={94>GM(4s>T{!H^kzE62BvNXxpUGu9v7 zQvf7|BgG!Di`6&TzMyfBLhy9|l*^rLhOG%7%^v0LXVbQLkKNHvOoOK>cs%5DvmiN{ zB*BXU4|L4sHttSA-3=}NCR5EP{|k`0$bl$H66RX%4Ww^?)xdRekj#9=s`YnkdiJT9 z!XWjh@RvLe)NG#m)CaYe6^sMZQw?tY&x13E?o1G>g?w`&ULel z29=`*h4&2UUa%H|G%&bh$yiKkIDH+$Nx>>jJ{VO*M2;(0zR15~n{+d88*qCpivdz{ z3b{4?!*VsBcwafgUb^Iz=N9NnrFv=@UV_Fy?`&1vQFox#S2WMj?BQy@62IkG*(L=H zY0z}c@>TiAYno&&T!`gc-h0?Aqf{@DI6hT7l2lTj33fhGr@ZyP7muDJaBbgMiG`!C ztR*8Tf02GwPs>G7*}GqBl`oDWO9dQ5d=Z4&VCytdD&i*8EZS6moMCtieT#Z5x!D?A zH8@z&@Xt2XdO}9;2ixJQc%k_mDx@+C(9OCc4&zshZvWJ#rvuv1+&f3OBSVmUsJ-!l zqk8cme-wHX-vLy-fZrEZ)FPe>-5`Bp4${Zg{y5VKmh(jZhIm9625(SdYz(-8!wi+j zhPPu?rU|7awIN?LCh^xdso-hxjss5?5pl1~Q9VHUkM`p&AwMNNVc z&6O}~LS{ZJ&f#SXF6}|(GXkcB8s;AE;ZwVX8DvxM;o6&Qk|(!2_T1n$HOlfUr#te+H#*ZRKLSlmd|VME*3Qb z!6xx_fxKe)G1`bB)ABfAF8H}_KWeume{5N2K|?q9CX$w#ts+~poQYwdfKp;&h#v8! zun!g>p6E}Qo3U%x?zOO*G4L72<>qdwnwN{)>qQY9Hw@8)G(X5u_8VY!cSpo=Sjq4C z`hD#LP|#YFc5@SxBsdxE(qTZub(08nb<(xWjZQX*CMZ%&ejhJJ84*&v7ZRhde1u7JjyQ_H&EHvX+mV8 z*5Hvmy3|o{j7N$({0!g#bnCh+ zTpk;kT}sZrUz}=YF})8>D6Hc;JQ1{mu=wJwM-Wn8L7v5!RBSt8s~$ksklaZz>^(mkdU!lkgE zfpI6aFo{T1(#v^o$-RMk^&AhQcU!C)BY69&ukPd@cbou&!kGB`H&abqMW?3ERZ4o+ z_b|S%S0C8)-*H2doBGJ9ne5ifWndA)W38Dy~n2A^FLMBOT zsF&FlmE7=Ce-I5yLw4q~_j6glCOY%hGbySj(JLAHDc82;?tVYU=1^Ci@8BKNtD_?} zm{W|=l=TJJSDA~ zZQITynb@{%Yhs)K*?qQZ-*>C}Q`f2faH{S;H-6W3tiyJ_;N%K)+^WX|TTrSY|MR$_ z`6B5SyR$kXcwONtj?8$Is!;kG2tXnBet}$j&p4Dc^sf{ol+V~z&0N54=FeoqZi=Pf zsDy%MLC_k}`VhJoZi+oYsP*U}lf$uICh+b9#AZpFEEDM>MZB^@eq*$MOf%$OsoQfG zP$PSJrHP^$uL`sH$2JvA;1?8*P$O?4%pPV}_>zp2UafScMn6mm3pz+}w-wN&r65Y1>lSXY@}?-pWE>U4wH8lh_Iz-9>kS z%doypl;CzUSogEX)cmIa*wmbrs)DCW$ZF;zZXZGNSPU6 z!nesOZixBNNtFUO!_A^;VQ4M9PKYrm?N2DON1lld2LvT%g}kuOgfHWfTN{}GDdsW! zY+W4aDM|5b1!jK+Dq$unQAIBS;VSugZ-EjCHyomVoDK9T5%y)HVW>{TO-gC>haX8aUqB&`$xLfvG8WPLp^#-6LD@HMD*VWd zll7xko^M5BoOzu|kj@o2-D6RDO%(npfJktf*gN4MpBYBPml!uOOOx2w$H6WLB)gNG znL#`lZAOQ1!{!wQcY>26RdWq<&!0LMsyhU!+dIe``S%9DdLIpFMd-Zq1T`2^>qx}9 z`R&7ki9wQ3;84HH#)ZlZuN>GAK=VLzPdMd?a_7R#wEimX!+4NAM!$D+i6uewON7>`Ntp??j1^^ zQzoQbDv+O@8KmPH#hAUJ%G@~^5KH4-cI11tu0UP_y41|*S&zZAqS(oqdpV$Plq}WF zPHwl+Qv@WtV!=pj!K|9#XVoJEP+fD;E6*P1*@pm%{)lQ%9&5Fv(zlZh*Mu9-GK+^^ zUrr1%U31ciNp~$Hv=@=!Q~3=4pP=eCMzU_dxG)qSpnJ#5kju-l%ge#c&V;RnM`xmx z&?>6t6$h%t6Rbrlv?Ax~38Z-FtSh0`@~&j&Ll&@f7I|VB0%j ztnLgnCxb-Yu=QZQl-YI2t~JDWz86LzFPH(pPz40C>y~NmXGzAy^L?BuyhfT&wd zp|z5mP|I-P+<#|`QyRDIS#d>-e^17ir<5^d36igDmJ@acArbC z+t7>tzk_#h#-2m5b{^`;dI^qNeOMpl$6Wc3o-hWt`{U0Maj$9uKX+}6pWI-!ulF;b z)j_`RhzYD++X-|9UP(Xa1YOBIg9h^20@xd;%NXlN>>~Y~%dMxDCB3zc%la$}Xwod@ zIQ5;Z(sR3*A}a3#D+3Gt`#13(Ujx^pN+B^QlTo-EPALU05zK`ybk^%$t{XEUb2Y!U z^7-KBgtdwLtMLuSX74S#+6avz*lnog$M)b$Y2${(r|7Vf%Br#JXPt&oC(PjMfCs;M z$LE517uAtwm+GZqHW8N?mWH56w%_aX{G&T392O&KB`A|Sh2_Ck_Hxwo+}UigX%z(u zv?-KZtf7P!@?@e<>LQ`Ct({TIAq2q*sR?Iu5;FAzZR`a~jQqilS+x5vJjv7`HVi3T zhK$r1%oX(4KDtlp#l~S8h2e`wz>%U_qBSb{#Fh$u1+hQmpht0o?{IuZRngOC&^U+1 zT8xk+lPz4cLsI`vb3HHoTUbekVVicF)l_K${NLLN_3(`yuLgmSiH#t)lIV=}NM3b0 z5jibPj;v!t=_F`n)l%+A_#bpA%}q(6H)6?DVrsP%?{T|-p5{+AP^s zsZV(~A)95DHC%NzW69q&`xMHoSa?7Ag*76h!8YYE6;30@SYfv-JyK?A*dih`7uER* z<jwplH7_t4%iyw$&oD0LmFP?rT5k#oU}3em7~p3{NH%-wMgy*O%ZwZT5+Ucvr|_ zPz<5r5Dh5|vQVe=8>gq<2;s*$L4PBEmmR`*&sy_&0i&JQb&riQ%1XF_jM)0HCclQW)mKtKwPjSMmiMwb`8(o9Yh!z@g)0#Meu9BCuIVsW6<43{R}4T;*`M`F?Mm&0kkZi^DwQ^1HCDT}+dzyW%lU>ruC={_m!VlJm*7I(T80wi?UX4>w}fCHV12@s7cm zSlxm)ydvv~vF2jZ6>w19oaX6p8J2>-lT}U2MDI!g40wxw@^6jg_LSWMq55H!iZQ!& zh7hEgL58$ASTC}{csy2kz1^`LOkEG-d#)p|S5fKO6jdsg{G8MNGS5FA*5gB$iY?ZB zj7HQjh^bP4nDv~>Kp5a1cd0?e7$#9D5A|}gck3R#F!VSU3s&SyQDN#T9Y%M3%SCJQ z6z9UpZb`f|f&JZ6`orP54Fg64Wf^B{hd~;3*ZE-wVXM?Po7ZD$-1K>`91F#%Y7l)}cLJCo!acYxmpXEb+h;# zVza_UrGtu}mt2E@I>8&k?gtKJ%Ans|?B8GDU##zV!g-VP-W!~I;l1r7U)KgnkW-_U zP?Q)SOhV-3HB-$7m5%I{hF_wR zxTcpeqL8MZT=Ge@@Llb>*F-KlVpi_btM_YFh3Ex|1|TX29gX1l%hH{0QO3WR{VGEp z6bHG*WN(c-EX_muOH*=kHg}|NoTpR-Ux@niFI}0z&ns2N@pVZfTsZ=`%XvX8KAqzZ z6-S3hBMvSfj%?(Ju!6&{|4KyuzbVH5pKohc_|1azA5t0VTf#fpKT;X{TdGnf3njq$ zpSg9m+J6X(6ioFse^OFuHO)L@npDUd>WEaZfE1a5=YF7{WoFSV<$MZkDD4@n>z9KO zsnYu{__N{=>$-J3L`2C6m&;Vno7vIu^gqBtchH$to4HI#D3AmpL}kQ~JX&dmMz?o` zChA`$rj5JoB$8`y?I;5PnM_XW0|aP^P%P^k~I-8 zgw5;U5?prf_*b6LvX;BB_Z*GND&Biw!;-$86(?opp^^tkqE6VzwRrq1g9ejPGj+z4FH1D*-zDX=&-h!3%DMljYH7wOaB+`4 z9F~P!h2TtNcuX*bdWKMWj4Aek^^>Mpf}I{Y>N94-Y z7VYTT#H|+IpF>a>TRAl>(xSuOtd4c(mM4|l2)ew-M{$^1_%Ef%mIv-0f-c?^1{H2L zVvywATHG6RE#fpZKT`k$@{eH%m3C>|Y-x^23HLh%tkZ4a#CmqP=A=S_$nNeSJ5vZ)a`2?mFrpBhYCVjOCSz6WEQ>4wuv=n^ayE5L)XfD`d%g6;RVq>ef#Q972s zB++^f7*hB%B^Dr81kM^Q<9Zo&VLB_T|52H{$0mj>-s^0AD8c6pI<5UuE|iH~GXwFaRrwZ0oPsoABcw=~P!LLc_uCx@l;TSpzh)uUa`B zH2?KNJB_tZcAS&$pJumYt>aM4zVxW7LO;(LQ*66}Ahg*RvtqNlhJD2iljX8bdS)NR zX1y|sqrGb?m2YZ+>vZ*kldcMf3nh`M)=-l3k#;iTikZ-=vy5!_y^`AosI2Vz%N>t( z?S(DskN2GX%it+xQI3{`TDOr8OW+&$e)4t${(-Du@q=9L!q?iHSAS2m=d!!k<Q5b4pE+;Mt{8rAA`B5K+kuI z=~Gw4IzqL@L6}`quGMMK8|nz4>(j+J!YxA_?##naZP@s3l`gHl<)uJHK@IhnzUo=N zdAtoZLaOa?4&g`~m9qIe{k{xLsXpgr<0Ge2JqJNI6>26SfezC)pm+nHjXK`s3ZwER zB#hYCG)y4C@6k|iHCd%2N^gWw7n$*+y#;0CtIsOu1A8I0n(9O>kj zPRoVU5+!EsqehDW;B}Fl^H1lUBCnvvMZ2qSX+-;ADKv7*Yl{XkHOhIvchD8u3n9(z zFMv?c>%R7y3TkN}QCouB?knPcFwH#N9F-Prl9EkAHRc5Tz$Cse!vktL-!=kyQ;02b zhNNKivn>0jY%jxOOMCeGh`PW5ZIav-Yt~kHo1~5T!S|o|C|0lG8J^*tDF&IaxO)Yr zFLCDBd90@(^O&x$X0{hLKUSwj?it)~Kq1KehV>HVGhqAX+paI~z;+PLc_}Iaim)FT z-y7bgbI#`oq`k;VRV-UWQ62& zzJ@q5%c-Os`u|4A|2HuIZ^+_aj&9frfPe@l{ZDyMn)tQ`3#cwWT(^#Aa7Il(@!j?1NGwmx-#6o{6wrA zdwoq^=Sw?PTkK`3^=fk5lqE4}EjHKlsq5*I?`gwj+V2X_;Q4Y_0@SeLJrM8qWm?zc zg-eexZ++8s6%c7gnmZ(E+mvZ!Y;&Ez;IFtSUF%on5ll6hxgcZwq!y5D9Nw^n1a$u@ z9KTC(99);ODSlC!EAE&6Zbyea!O0eoIdtI$wnhW&u_p~2>uNK9A9WY~{XTc7;p!Ql z(<$4ip{wbgEbcWqdg*+({8DprgpmCu{@0-`KS|*70O&5hWoUb!8r?3reX^(C{rC6$ z@!)=ZTbZbj@_b+$eh-bgaWw0G1X>Cv=hd75|(l`Ouq3U zlmX1;|BJJ`jNw%WH@h$rke+5TYk zc{k=jd-J)&6?7#^b0s@=y_mvr{I&_V$bB)c#iQBc0Hz98vLm!(F*R5p@}v1kNVa?7 zHr4sE$YW=)+rqMZZM%pvw8(5@7$Z;8O?o+$2V&MO^DnV|u@?oN`v%Bbco7Yh*qK52e#*r#nCzuxDzwJHzy9JS&Pxijq zd!!4i3 zgvaKf0hWVu==SLz14kyMpTtO~TH>&%4v^7ww_A9iS zWJ;f#8M~b8y%v)91loH1yJrVUpv2vnj*&DPp4ZktF5v5|eJt$DS9bbC8D#qMmjv5$ zdyiiqQ-Zq;UEtW^U$oyZdFie<3A>#69d85nfs zMF@%e`pT(o2Apz(wX}3qBnahzHKZfc**Q4S6k}#)zT3qMh8Y58-|1^As|+;#gcmc+ z78}))Z7{zZ3=#hZBU)BlTv6@N|b7NoQp_NZ0}wzFruFzX*{G`rZ1)e=e4uDzfTdbNw$0YU68 z0FtCTsY>;Y$eIBG^LXPW^FWwMTZfvb7X8YIO_z)hhR=ttux`)vnveU6$R>pueFC`bj++daRrK?JiiD0W?h2pYYL+u z9PAyex2mU~IhC+%^iN3vLCs+>jw1q>MZjLB{C>peAL(sp3;VW4>x*LGyCZ8H>Qr>z zN1l970B z6nt6|!C4i=j=~8Wkeq%$*sm_|x1az7ew`~hxEFUboIBjo@K5Wlp+pf!t3#}Thnaga zVMjcp^Er~p8Zz)NOz`<4cdvfza6m&SD~)By*E}@Rj2U9*MP9i&VaID-#nQ{xJr@|F zG6KL)?`=%@AxwH{UiwDO7@P8t$*{k4x`~KB>8d8HohbDX`{K-rkde?n-0W7zgL-v& z8TwHJT59)>X1Z5dkD?Cop&6*W`02Gy=t-qdg5d)uqTX|qmOm0lFoor}7+^hJuX@o& z^zFUtF#mO!1~GdnFVG=*>I{_y*csQCmw+WyOJ(|KSUOZ|*v352y4SQ0Jt7isLD<=& zxEF$w7=lQIL~E^jl*ddS$4XGuKP5-8;>4KBx+2~!5eG4Xs#Hi|5nGsxuO?6sq1t@b zbOlLL%?-=SfQ@Bgx92~4Q|OIK8gBJdv7K>tFy01SXhC8!5XzVfs^#c`q>lP<5|}h< z_^O90L5Q($hzn_I%WG7Cf*%c%!MW!kp|k(^Eb}EwQt6jG3ELi8-8`3Q=9nQQ5hypm zZb2}ocX!F#%xQ+FtIs03chENG7Av7rU7nrWU&tt!BU4#qIKXSlD~WwJ;WMy9doXhx zLqSym)$4d<8?ba#aeVPs%v@bQ0@(9U>}mA~ZLchC;Nr(EUSbZ`hxX{S*OwE8t^y(C z&9YCfE#Ku0;Mt~b(#vt z31`ZNk#5I)jSiNI3`<{FJ~*K}zbZSWm8hZ>E8rl<*OMUmSx$|2oWy#2<`pj85TblG zB8%9^^+iyi3zU;vd=l-pPSK!><7{917kkaW%!^5VCI{RWpwu9ci0?}CHq;9eH_^)x zPi+&Yas?0VP>Q&ZZ6YAp0ppcV4E_d+lD_3Xe&EiNH($|lkF{O55ls5M$n@e0@ZTjj zJd*1Ix>%=8<8bDM56P%+tqP=PmQSx3Vw!G#HoubL2~V2C5s1vIH*}RXc{Pf4yJ)Q} zAJniIB>Fi-Lq*G1BB9zZNP*uQAJpGJ-YPk1=6uF^+*^46otP_L06Z%gf9NskCJB10 zu~n-&i7WF9)p0Iq&pNW7(l6*ldKFmUR^PsjWB*1MQ4|XgM7!@~;@64@49zr2HaXSa zW0lr)ZJ6T?$#~93AIeWN5oUb|={y?A`}*RQ~BHD)mg!sHq+gP zXVG+a7_h8k=<3r_`y?`^{($to8a!Rp)jg9ZBg}k?GSh^>9AF2d%Xd3lQ!5@ElPwDF z0}VR12`{156IqVNLHhxXA7QBCH_Hcth?wo7Q1Yp86O8uDZ(Dmc(}jx^UYWO@|KEed(CzQj{$;K#bJ*qm1I z$5L!hG66%G=(I-5YXOo!gk=tEI}VqYHyrDc>{f~hqXGpzBv$D4D~guNq)+n)X2Mje2rv6BY&p)22WT; z63LvwDyb+MIZbhWEm6QxR87;BcFLSi-%)(Fb%2y_*u~pM@HXy)bqVFYG3vJ-!b0;O zgyMLfzwP296xUPC@*`rFVlZ=Ycx4sr857r)SdcGEu&G(XsFiI|qga=<_@P`Cu;Oa>w3f&Mkh-dpf1hEmYy4!uqG!jsqTc3}&b3dRTH zOB~>lqtg5h?*6)clv&klF|(ucYO3;Wiz)^zO$tOFdjc21i|z;Pt0oxVmn8IjZh*px z5iJA{tq~&65K4U}9c2}F0Sx7~Tthx$3R)sJJv;ee#N#HjD~CPu3S-0xTQv6%{8@Cg zX2}jMtJp%$O|JIx!L&sZko zPFMo%?FZltewFa~Cz%8}9@L9^u*@5#66c(={G6!Jz+TyHekgw4sNp9AE+5 zk@!!tm^R@X=TSL_^!8P48fH2L(384c2mIhtJ;HSk!Y?1^6*(GLf@;s)?B6NlFWVFTn{~Md{a4MKCmojG?_8>gniFkXUcozNcB9z|C`P` z8Yl556yXu1ebcagl>o{+&JkzxijW29Gu>RURRwfL7idQwnbWf38V-Hkv&S&jgZo18 z#|f7xSPwY4whUwIhT*&9hq{| zgiY$4pBwH=Z;2>afDw?063|Ljf#*^qv>P!wU<>S=^0ON``M0A@$m`Rl3V0ph-_P|P zT~(93>7)ZW981Pc5VVFAmJR`WktuT`>~qwR_*?}r7_qt|!M*%Oxwv&1ZQq3(4q)nr;u`Cw;v&o6 ze7bse=#J~6IL!xccTR+&R`{9>JtsP%X_RH)A47QK8D$WDf|l2Yb8-M$3UQRPs&~*e zX815S7>gdM;a!LJf);MOTs&rxt3NI5r>D5R8PJ0pN{Co}T5 zDr0T$+8%0)uqy@raJ4GId7%*2RQS!l!`c+{n`1>?xYoah?U|-1uA2;k3#$zt`<5-^ z;=#i1Zoz>x^e39sFC+_qz}~L6@kzF!sTE9`aP0$SdW|86y<6a%=AjB9RQ~B3pPfJZ zmuRd2K4LImJVI9()k2Z1RlUVLR0D=|4s?Ywmd(V-CYs{TOrhPAHz~>aUkyd7rYCFj zk$JO75*;(eR1%S<_zm7MMBH2WaA^lo=Tzb`BgymAK|WaRro=WtXWSFnwFzqq%&t$L|{@ovFd z$!PmK<(t#J(^3kU4}xz=RYiR>m6V-*SC*|gIyBX1xUv;(q`W0+A=XnGIKx`^3s|(9m;!@jtZ^v00U$h(9JJDuMUfL(Sk?}(esP?d`Qupgtmj&$+5JH2Qz)0=3 zsuIFpQ_+%^tk`#atIorKd&+o!5qkGza9?rPeHxDg%w8|0jku7k@;(>C$8!rnH@A zx^-Wq@^{k$dG>?GTM783Vdtt+t?aft(LNnj6lL6wZ=0xN)tE_1NwT+950wOJ!q_zE zlYHLB`52zSLKmuVA7Mk)GL8@D^bZey{ey?1BUnis1KGBh;kt!5oj$(qnvn}K@@F&{ z;lW&*L30mO3vJMa&G!B&QI!b_5m7xHh58MYjugt&VFIvF=B_=08G(}xXWTtXRyT&lWm!)v31oOk6e%j-MI z+;{0N(aX{C5T9Ucs&Ng37hHqt@nwV;n-{1+snR45xTBPGOjnx)VS5U+7B^29(}Y0) zjbh6wOOA1x%dNgHZbX?%P1yQ{J6usMT&oV=3$CgF|+W>++@vTOLzDrma^ z^^6&xQZTkA*me`sW?!b|r$;MldtqrB+%keG#kJlCPkaEd3 zL7wIq`DozaP#a9*?UL-bwSh1c=7KN?)d{@8aVG0~NiPS4{ORJ8| zC;YyLvz_eX#|+FKp`aN|83(nbXxe}>J|9&~BreQg4y?gI&#q1322_L)srHknDMbf* zIp)I?u0(`1jc2Oc?8nj(L^>7VX6=wTd1>;-WwMb-`S9po()z?mN6@V7HunoMoc$^M zr&nHQMPoF(>R93xso0{cnP+@3=``X^DkMm59!IZ*)jj$$W(6VkOeQ^f@6y5t6kV%$Yto8sI96`Y0!=v{noRc6A> z!`>!ENnbKk)kQ|Fr55w8+n2k_lsQ6)xr?cl`TA;P09%s^k8{;hpqw7;| z$XfL8-FvCByee&sqcwaPltr1K?lBv3+(G|EeY4f|+T1gFAIoIwwh))Y|=Uc{E*OFlK;73J%8Y%9)D>H zM+6w>mXx%w)cf@2;4Gn&5Ap2w`%induPGokX!R!{W(*g?pHt4VvdW8=LG)`moOn~) zwu=@+dei^J!rFBjiqAJnC@QKVvGAftTyA3j;R zM@FV1_|VNl_8QGgwO|VF*Qj~nSO`*?)qZ2d=Kp9eratim+(dk{Lyx-U31Su0sOOzL z@3wv;-ZgSS9B0||mHmpzC;Q|di=}dfs`Y2(hMsF%9u}NseYcv0`azNlyW9!syZP^S zJ9C}aE#6%?biy^L+93jgU#SFxe6lKCp>HK zyOb`x***?pKwkg^S#DC0-k(I@S3s&}I{)rXNze4B1f{3VtL-b)*U?za61!P)La)^G zp^0_grgUP4U#W8QX!MFAYVqO4XDuv5N-%g4q~y5;Q9Q;ene^6Yvfvf6A+&GN5gf;I zsP@MJgXXIX_O3a?%o}wlI9ZyT|HB7hrj@@oaY{u8C>jX~Q5ECw6WtX)L`$G1Juypf z@3a2f=10t#lwqYl<86@He$zD)xvmt-2#1fQU64Dt0HG`#m5nr`wr-o0Wi@q%Y=in1 z(81#BiV9RBHhKnw00hSleTn&aH!rJ@wEbh!4S%pD&J__!zHMjtQ(122s5ym_=86~g z7qCtsQGZ02GMXNG_HAn$Hd7&)j%ah8$@kw?hpo!a{_G2@4+N(w5}kgYVq$8(e1YX3 zQw*~|al_4POtz1f;ZO{S%lLiet&P>Sx(jFK?#{59+&2~838>=3k$TV=WD?{lX7yp@ zO@wt__)!xRP5}+E`bdRnlUbgb51t z8&e43WMwejvF;$Tk`I?(S>1|B6yd*u7AHV5uQ8vJ>VNV<3(UY=dd%X&1A2Ge8P{I` zdS&sKNP5Ha&my~s<8dRpM#J3R946?$evr(Bb7t#N?YT}ihG?y?k4?YJWY_doDjDu`A|<+Wb897*8`1YyK{K8 z5UqAvvMP~eyj4myd-bQRDYS}P*-GAV+Gg!T_D68FoXM%)n<=%2yV|dxKf5Xb1H;C3 zqhu`YL$NTTWUR_JPgh>e65k8arEmWU7J8jbx;UF4oW3!T9Q zcq~8Zel71T%Ehj3?ifDR+T8e+tww!>>6QB3sc4uQ!md1E*5bxNBffdT-8~ALbC?eH zdqeBm)&%@z)>*sWz~rq!*=-2`-c68UZ%N_m&}QkSL7uR}rpcHcv)Oe^5thp_BkHs7 z(=PfV!G~M??T;s#xKQc95Qf24b}vF~YkVL=&V$*mKu4_Z>+G@Hu<-wIZ16yYTmkJ%y+bKmOgjJssm_-69d$LVkb=wn|^uEU{q zY=s)?8ez$9z(H1gBFmnFeV@!I64|9 zY!C8IXg+Syl>O2wgX}N~K=Olx!NH_i=Uq{{GORcRZS14G(-hHbL|?-a+Z-GYVaTuV zvX(nH1{yV@-|iKI_7x-cGN6Ew`jSKrv(dzX)XERe>f8={$d9NWw*P_#(Zanzlu4~U z(oGu;t!K`0q!`T|1(K#r!6)#9}^d=LRjWm`lFt{5##SKNL$MhSo zgy=qI=pk0(NG~b#$ZbSxhMnB>i0fXKrvGzOC#e!B+f;{LO?P z^vQ!AykaQALcVEs(5;^?1WGfs9ut{8O3QgWG|cjSX$w7Ni?xlG>(%idV;W8<9!F2F zib)w!C6k}Il0kFXzxhXA?mi22B$Tj_qeuMS@K7u(1Y1sG5L7Bn-(zr&)HVXS;bezL z#Q~M!r0nY!vG}x9@n3rL^S~_#$D44e`whZKc>6?n=$GE!Yke)$t?|fxKK&>eMKH8y zbm{|@#Op1yn~o<%;ju7}u_TJ3iLHaAbPR=b*dkdo)G^lwAYb;$keZbT#!Qsg43or6 zUT|!I#0--vf8wx{rvJDQIg@3gMD`WjDM>3wG5f*-Nhf=G7Z2BlGh)~2M{sO^mfp&x zf8Vg-l%MLxwIF_VDRE#8Xkad^RR;~!avAqL@%q9FtOk`;Fq`Ux2LSS$B3#|TYiLy(KNNyLP~C^=qe)F^CI&<59{yxgx*3pTd<>tkqqoac+3BljN3Ef_*S4v!Z;roe5i`cKx2lPT3iwq*k_BCZ6! zLTh+Fkf9utX&vlpK^B^HdV1O|B)2|%R&`1shUG;tN&}fL(HdUO8?+Oq+FI0n)}{e# zwtr^{sFTji4d!rc&}}8c9Piq{2u9y39^x6YwTZP@*~dAj_{V2Q)2l{{r*w+@rFR-f z`+zl-s8l77->~w~BEE19qAuuFtIvT%2$NEnZDeQ%a zOE^+u@6HC1g+Y+<29oi<8AM%YOxD^kY?Gz}Nqv5rV}@8UluyFGyMCHWkz{gm_c9B@ z--RgwoRB*WV@~E}LT-)_G$kAJp{Vncs|tDykEA|q3ullN4)fo~AtB$AL?59y&pe)> z-lBs~BJ=1THFfVft2Q9f5Cu(4P`nN*amAL#UhO%{Wic-T7b7O+haN72!pXg+`;Z?1 z527qo-yHerNW#G6W=bk60&`X^+U*}s zH!S$#2TrwKIBu-f7bw-wl4s7?n+jmDV3UYj%6a{GFig zCA3U}UzqO}3AHivQG3qkEb4}+Accgi?2*!+*4Tc}jhWB)G z20yJ3O@@cDABd5Yw)-#sVF@3}9m< zq=s$C`f{lvP>IQ8?>rL$TTylKD0Gb-@CLfpSJoYtOiwr^_*-ZE%~~rCzSWNT29Z|o z-CD>uH}Oo;0X|1c1Xpz|TZUGwaz~cYd}B4i5HRYVZqoP60009U+5tEVf};2P1z6~a z?2g$9msM4xnU9~uke9_3No+Q_%<$BU^J#cmA$;TMK9k<~gC?Hbiq<{mMiweL60*L! z8n)%9E=y5qdZy~h&ZEJ{!Sxq>k2D@ux5%GzjzrB=Fk~diHXfnXm>QoV#9sqm0&y&u zmKh8)Jib=*pjVk0R~>*xszNi#7l{gW8Gu+Jq3jzh0I9}E-0@O^XpZzcMjziPHHJJ# z8>}@*>f1MF>7b4YSsq|}{+$Nu5NVRGGGQE&HaRzlpTaP<>ZmjtgQ{lRkv2^k0Xace zq4lTCLS>MJU8Twp!x*I!{fBLEnVFn1j85p4{II*lPW$fy|7;-oiRb#0rK6kh$~`k)LUdZ?Q9g3h{?ZpC1A zJC1;5rTF}6Wy*i+NbFq1wv>*B8#IJ4MAxz0wnswN;%jDA!0nNF5&88wShKATsn;j_ z!(hzu_rgCRi#0H|%yeqAW1rAmK{v$n!Qhv@Ne;10A@Qd*GTG@WFHVDsu!){lCn$$~S zcJ29boc(*f3Hng|zsuCn`@va2AR=i4zB3#ugaqvN+- zTEu}1^*|!_6+P!Ie*Tp(yWG>|V9~_VT}HMGdg774 z2sl&9eowRS)2ppN;IBOh&;qntQQfAhKODiiSUA{`E8E7EEq!`VU4GQ5WEhVmeFM0W za&^qfJxuUb{~uTX9Guzng?*z*Cbo@TjNhYZX99Y7_Dnntd?5i? z20g=jGB5boU-bL|)q@tYe;8*-{KBK2l8uY*odC5(yWW6aj@&n`VC=NUNaB$$(=Txu zX9PA{br+LQl_!ifbf|)Ui3r@fArcofsF4(Ww4-Q#Y?8H^hn@QGM+5IQ zzBr)zCwni?BhvQ%3?J|Ki3J4R@grm=s?`;Y{zHX*-yo}dKn!6bx@I?yUY1VBO9!4k z`J=mz?r*XgRB1#)YNL(?WBeJIrsP!Q{og5b_$-QTNvGgODcn51*aqS`((lxf2>6oO z^hp47hzK?PC`kKl7}lrXJ^~-XpxpkNlWl+>(ZgBrGe5V;dn>NP zMEEgKrLZ^)L04HYEK(5jYw;<=dQ{Vby0#okl1Pn7W@vm;)SBokSc1YWUs&x^3Gvux z6u45wvss)EZ2Kv{=cb+QJ0f&f>o9=4U|Fn~3+$z-mW(gR12Ucv=DKi~h~_8FLh?%{4;Qx#o?yMD zzp|LBCgd6bbGOjEi&W6se)$t_OTPfttygcKB`>?aQ0KfI_|ceQ1eQhFgCm#vp32F} z^n%!Taf)b$Et$}-b;#1DD6%PuM(LSgZq77xK#6{9E`p^|9OStAzj{-l(RKJ!zy!+0^e=K-~L8*5H)9o z?iNNtges~yC{2b{VnonHv_k0zb(mq@6f07`@GQo7caR77bl(>?%bbU0s%MjTl2rjw zATu{n;1iWccAa2$f07StyAr^3*yTlW+{-qP7d&O(2_bN5-Reha0rR<5GR}|$@FM7# zrK^1~;r89`N<&M0HE?4aR;!Ml0)jIeNkj|+)(+}V1w8x*=cjN0bqrNq`W^hX*pi>W z{J+UDDTJeKM7#D<7QG`$QIRVK8WxG1#GKpBN~TMY4#RfwZqa4v{5JqrzSM<5veg2& zHZ4ToG@@@_o+EC#jr zG%l7e%v5Q86ReIVIM@;g*NtVTo)3l>wHFl{f; zia4zP(K}^MHGM=QF9_Qdjb4sdk(YoD!lwLl|Ktsb?xbJ6HYZpe#15gq4(V@VI^^hw zl@A8AM~+BeB*|VKHH$Ceg3ApFr&St%DJ`O1C#-)~O@?2ujhzDY%)&UI{$;HR|Eg!x z7tBZ;L=5Qd3fO1urdX;l3ZphaUL3?;jZ1b~6F-134e&JuO+020gZ#@ZfuRK_7#U_` zI5(pg##|vLB*??QP_(1yymydQVt*Z)d&htsDx1c5U3*{K-;;YJlE6NeJx%4qpFiD=`6B5Q&Jmq-$ zd*nXsS7^_}d_}rv;-HH#lUsknEe)eq@NsW1WZVN24r*z@&yZO2$p~~>JLQT7salqy zE9{^5DJL8AzaJzaMu%_mE3(4iDuZ8`3M6{vnN)=@v*eSA%Q|F+6rWN(UeU|Ki-|?} zBg^l7Wm9}pqkEX}0*o*nlr9;4>7aG)(-elZHYfrtEcvGXwAS|w?`zA|WF=70D)FFV zEcrC>0#uy zgFfv3-Gctm)OY#VGk#6f#-u;rUJ22f#BJi`bOO1QUnz{+ri%CBo#XZes9ePI-+)x; zcbK_v6tM@@X7(6>q(Vg}^|V~!#M4q5B#BHzS7&>MHUsB=lx~~=Syf-oG?oRXpUlWK6fsQJ4cRkXx|?xg@TI! zz$Q#7msPu=r~Dbi$Q1iikA;U&I;uo+4%MX71=(Xj3%IkZu|{aGaMd|Z3F%s?sVyTw z*`tpwu_Q}THAM^|R(fg1jALSwJ2n4s#QY9Ftghb|EQ@6aLN%SGw*$K~-~+4d_p~Ad zKUDJv7{(>Aq_In!_5&elOoU0#A8vT`dL~2wp3)Nkd(R&(8$y#cx9xJKWA0I)4EwK* zlENOlbWjhg(i<20A5WHPGnEJ-$8oIygE{fOa&7*OO0S)qHEhlTg2B4RaNA7v9t!J_ ziE@luq~_=PY&Y}gQD?Y)mKqLzExR2KS^RW?H>f{g*%hQ5ZXc@~bev zgqc&=U8^sHuoJc45&|hwzPj^s3&mLutUFQ=(cF7i&yCjc@Oj6EYlT8j@sz{g584s; zJCf{OM;xm+$-{8U;IYlBj}V$6LWI?D+|;im+A5{EY-H)EA=mlXk~IrJ=for_H@?v5 z&wd`3YPhPOiL%KyK8O-jgVlQQA`LQzsqRkIF8WETe*yhoh1?KgjWiXjRZ8Z`Q(awri>2^fY%u0SoB0@J5Lx? z-$=QmTai=E_XEpnx({OwE}{r7USBKXvkV28Ppr4dnHhb6*J}lALr#yA8{S(_X3AQ= z(e_=3HSr{1VW=EVF8aBG7`N(QYZeDbJSa!0{*q8Vdh34nF>q43{z!jt?n|vxaqm5> zaT4J27A_O1c2+3@z!NdXM+VM4Jr9FBkGahIBZNLdnp$=XC46#Fq~aN89oO&%(cv44 zC6VuN4d#GkSb^M+z_kj>?%_TkXbu^(XIMSGmyDVq-X&cnJoGe%ou;^aA}?GA2NGH` zBdTHI9~Gxikpm&;0}8umlVO+&=JZ=lG-27LHP-as}8e zD5lDYaeo+=T>c@Pq)$j-(RSL3d4g$aP)PhXR^b9Swns%fNKS!f-?wD&ZH|Tq=AQIq zj*KOnFD}BEnZ*$HVH|U}rDiCzO$%yQWlt)%rPjiQ(s&o8|1+;5l@_MeU4@We8*#!i zmTZeHR2q}{)?XG_ipvhUk>;*kAjPFgRiT}6g9BKoQPk20GGpKH8yp1ZnFi*&g5V#3 zVO;Ro0dOhDJ4~~=BH{foF8BmG_j=oVzYS)} zVX8Pl%xnt?;WwHj^$Z&-IVPN$bI2US$5Awx;)poG=A~49>kyJ{RP+UC+{#F?*$O94 z!kCa?F5<~{Jn9Uce{$8CwkCPYKM;BqXdK)hI!Ai{2+C2RG$OHbBd(;nJ?Z>hkAmN} zIwH>*a^&;-kevjn?t_5?Hh}S)8e4C z_uEytmkW&;G!sPQPE(an@(e!%76r7@wekqR@g40e6{y;8c!T`hF7m|Us=#@?22w9T zwZq(Z^Be@Vcw4?%xY{nImdK&bp~y`ZJbDi|>6GMsB?c?a$`xt5ge}|z=kDHl#EGrP zQi3$fr6S^+5oUKWyuK3KJSD8Pk+^@BEN@^l9Hdi0@SB7?=lYJ&kdSRPBASJzZ8f!% zsape4@7cXRf6s{rLgP?XRiFRhv{4aoVkz%D(4e8Iy+RA_P&XMj&F!A15am}tx?)1@ z+B*VfLLR}i(&7ofL<))bb8|K@0YkJAGoyFn+Kwo6H?Cuq9}cH(?~<9U?clvYQFdW+ zh=`13g7gPx064&SgG+5#9N!tO1wVs$-bsBTczESl_1-&+#5n9{VhsF>1@$R_l8T;e ztFfz7^Ur?qZ2Xt4`roOKan%YeH+*up(~NZ zuz8kE+;sK$bWHh%VAYi7np$^8udG1|k)*4D$ZJuL#J0sU+3PrxMK22l@6|Fn(fI>g)@E`UYOQ)&i}k92RssgaLaA+) znWw;gO37`WIIOE+y5UVnbnVaDn9dzH*Iel}qDL_N7)_sVUwOg5WayQ~Z9Y+lCxvgHE+M(I8b4$Ra?=}0A zwO)Uabz@AJcloAuYVHzXQkTBkSqFN}i=*Kl{aAVW{v!KWYb+n!wPBo7^+aGtnh%_` z)*U-KZ4B`q`E!_G`E{4vw5GDqzQ3C_PX@t2TgH#%t~q0=0?m{-1iJNpsHLiq?}yFp zLPXmH>8x!83xdk${5VO#jwg&VLG=zv2doKx z^{#M7z{Dpgozn-Y_;QYA;+d$}&(HwdLSv_xJU$J-Y!X8T(_1D7C^?kLg?dVwI?;8s z*$H)&@n|Wiw7>xK6CLb84H)KzlNvPNMfX6Y&Fy=*=X#vZyR@R`J3xspzYx!}6##}` zl^D;CuWM~gcg_jW0n;wa7u{p=#J3y~A9D4ARKVFKBuo&2^K+f>XM1y*{App?uiMEK zs14`?$h`H{@oOg7NP3AM1TiM3qpojDrlB0F`Hh4+mh=udS)Pnn=Jlc0Pq5?ijx{G3 z8a^CVO$Fk^mliJGmCvg8_!uGI|1elgvdc<~oJN7)?<_t5g!Q9Qjp%fUw;y=LjD(P2 zb*iL-rQ~To0up1HQbiCFjNZ$z0M9se`waV z^8OP+_%xk)RGBHN4A9-gGT|IrS->+-+7MwKi3U9!Oc~i>HUx#YF@i}(*GfvE3$Igl;e;CvHsEji9;u`6fA~Ma1Z|eG& z_5pYyVh({yYZKgKw2I`cDORSOuZJHILcT>ocR2T2P~8b7Az9UX0>#*LyYXTn1SNCT8WX{`8YrLjV34SOjF60djbpF(5ggDZ#D$GHi@Z z11>;9e^(s~G@+Xy*(ZfdukwM|^ZnMg+W>g`sr>HoxYNT-@hU^;>ZRhpk`>qpr+3)- zMe(=V#_^#ueB0KIp87jN(RWf zZful=zt1(S8PLs?Wa9sW0q=PCe(`bwdTx@yTBj&68B4hAjjTU5wmpo=E4f+_@~-(9 zRKu7-KlfCd4z+6mwWAMDZQZV=r|!?8(-BMW1%Ks-%sp{TsXYx8&pC(E*=3v4&y+yKCt?D$g!hy4X>+aX)A$ojn(%FU2>Gzl0xDY^72 z$RqK7F(#x_=yPUIq10$wwPhDEG*joAq$$=ki&yimWSOR8#{TuCmI}d$ z;pKaG+xlTi7t6)3foMW|;23uP^?%Gt{@(-uMVTMquprRT$j~5)bnmhipM&ihP#_>6 zs6b*$WGvuMF)qkvdNR;(C~&@SG+;CC9s(CA`0{@tN&=J}DB^#i9`la@PDl`tR#;$f zCn5n*uoN8zh%A5)*?h(gst5VMbevN3Y_R_gB9RJ31`f{seYoaE8T?Z)M4s<(MOmBr zDa|){c2w3K5m85{dEJiBdrkY7r%`#4o{|u;_=2Fx8l1usemAcsJwO==*0R>jzInjOR^F9CYHP;r`>s50OxMO_9L;erSJHs7^ zg<~E7MYi&#$KtV-fWUOg*0Zdy0G7_@NWT=Cz{p^|zb0qtQqFxoJfNK8Q;+Z;;n&~L zuM+;Rq*%aEcS#6Q*;eer5rF-+gXsR&!TN1`@d0pPg5Y1V^iPH0E&S|L)y_-E&%fyD zQ+M)H7~yMtfKl#aEY|C#F&yju7FA#UzM2XkU469A{^7yzAGEyq?VET7SU7Gz`?!|< zs78JKTG)BE?$0QDPRY9sJnO9VE?(&4v3{uvpZBL5hUpd@VB{a&Q;gCbRkw@T6B=bK zdOIjn(C2?Xyce&PrM}sh&SWZ_iRGv?pk^stNumzyC~`I6#+4pK11o38tVffZRb&BJ zQ9_87>n!Qb))iWPo;}(j=nJW?6ZP+6U6b{6H=H`<`{0W36rSZJ^7v{Py3|NTo%C{rX{yAZ0#W@mwd zj-15Po{qb=roCRhGS^VnSm`C>BBKJ3aEV+a4y3UdE(?7+YD#g_S!%4fTA7*aFq4z< zVO1!R%!? z%QvrY>tss6=Tj)+pnbhmg%m!s*`fqnz<^MddudzD_16ftD%j-Jml2mMU(-W{EuPCQ zNmpIR_$)XoQ)9;4|EEc{R$>RBEGbC?Kn7v9RaDi#@9|s^3NeX9F=3gwH&l&UX>jR; z_a}AI7QRkhxnE7mwRFTqJ6Z#GDoSra(u`Y(VRhU?F8v{d4~_D1bVQL`gOgSQ4=wye z)*EvPa@#DamUNb9{VECEIe&Qzmm+7F9(Y6O|2tAJ7g#nW^f+i`3$g?p`Kt3~;gWw? zO2ZEA#<-RVWLZa&HQxA*>O<^1@v?Mfuq-B_4k=Kkr?W&`=Sr-~rA27E$3#dLlgx7a zwGSnH7xA(PC1b^I3TDld>*TrT5Xd<7Kg@15wF6f6k7Br5Z98~rRUb~G9)^TVT>~YS zIt-d93o6Anx7T=-215Yczl4T!J($I;PcWtI#l8GjvQIW=9Y)^5&$B zEuh#}lK(tBt-&0cX+0tCnU*!DVR4A|?pWOvg4*d%r>mv*)?5nj%6+tr(NmR`cE>>^kGyb%MWZZ`pTmKA8+#A1R>$DNGdHCC<^I<4bvzfG~r*#e8vcBp%vlhNhcvzTU7JZ+apy=mJ3%1 zVtA;XRv4!pSryqs$ux_8r>13X>s9(nog;n2H%>CE+a(b@T)!xVb_%WBD>i@-_+%)s)DbDY?vHeP>WR7mc=M) zVJ~F~qBNj$IH6EA%vaOO7&=zDoK)GZ2=yU<$dydGH%*NG_0K|vb1C__X)8JH!8f5f zBqnijqWUWQL3Mbq)LLywHO#nOOK8ZuynVjZ+5(QU{2l*xswPfxnNYc6yq`+VPg_s(Yl*`;ZEbBGS0B_6p$1z4X0vA~ahOAdL|S?5MtDLqoQR zN`Y4~$={*MbhA?!@Zdh|L+42`MVTQi&J}^NsA?#(t9Z;D<793d%i>ycSsH%ZzbzTd zR3ATM*UM;Pt1@V<3t=qj7?9Mis^Bu2U`2CZKrDr?tuPOCzP+$n#W5MHkM|~4bU`$| zGXXqJsei7)#qz!BLmY05j(<+{WS#RLNb>*tk|IxJ$duarC+VE^tWOm_p!u^DsFC{0 z+`&^}Clz_ZfRJs4f+UKI7m`2EG+w69Q}|cZB(|Oew0H5b*3=TOG`$K2osHtxL}8jI zUj5JoVa#xoSZc*_W`JnomWc4TbE=3^@0&6H1q%aN$ekMp3Sa@PP~`?UFf<{nv-SYnDCV@q!L#-tY3*hOjr!MF{qN$w{K? z+5^so?P-$GT`Q6>!e58tmLVgm8)?RNdaT5!XU~Du4%hAmo&U1JC9OHlb$O}#ra=HI zCLHjtAXI{BGzu%UDgi({E)GLda6rL7zFvx$UlvgUW4DVq`!+9TZhzl(9=&qPR9x}g z*9n`=xbk(>+1fU%*3%Ax)ljMRhi56K;MOC0j3jrMUl56DwxjT0*e=OD&eSE@s4++~$;5ivZr<>9p2)55yvpAcnRs~dIzDk_$@Z{np_DFT?He3) ze6g*f(trIU0+`Us_JisvMeNBWfv+ou)QTghx!@8kD;Y<%zvR{hYeaIAgxvui#>*?^ zG*{{xS{X%{_~N_Fg#nqh8U)AB;MH%4)!x<#eaZ+b!?uj-~7uYwQbv^9{Fs$lc;YXR`U22-g}UN56BFsqKeRe zjQo%WKZ_C8DhRnjj+be$f*$)}V*B78x;oe{y-_ay z2Zlc~1K(Kd(sb#d=&+Gek>h7%wxE~KrBl)(9%$j7g&}SbQ%MUfA}ypL#27q{FLJUq zU*uYasaQ{z^r{*piU5EdS#=*mB=ykHD8@R}af@(b0n;&*EzDB59xMaVxUJT$zCOu? z=z3;7R8fwk9*7Of;qv5}nMtIpXw4hYn%QZHOjQ1BEXw3@NNHkS)f``Zvi9u)>1Znx z=4#UEntq(0pDuZ|dG-Q|_(_;O1qYBsGTTpXt`&0$33iIEjJj zVJ-gbf@6FjSmwbi(f!A?=RfjnfQaY|o%GVUGBs zZ}`M4NE@!#Lnwga`yt)t!HISt_D8@kgm2z+xgfAu=)}kg6Pn6g#Fu78GU&m=m@<21 zdRcMlDfTw#1L|c8#FiE-h1jAsr@cgkSEsjR{e{ACT(`n-A7*&rBv!HL3TeBWKd@V9 zll7I&6e-5hi<RBGqs5e9e$lE-?cG()m;?U}Wn$(^)a z^s7d~&bjrXr3(Dc6TIO((m<67Hd=}(8>==E|EcK)4oZYC>c=eusnG2zh<1Nhw1E8U zUWr5~Nrs*oZ9uqmo&(O&fEq54XH!z8_z`il9nRzYMp5G7F(KEdv!{INsST$$wjx0V z*(>t&lOI5!*;OU>5{kFu;3!7r@j_4DEdp)s_&$~zH2`d)mKnJur`pf=OqC%rIb#AI z5&{no4kFWwoq;lgDulkpJESu6t*+fzAl(NkU^0Aw!mn%r!LVh0Z}3K8b_T_q5MjKR zZ_P9HI`kh3|FNR*2#YCNmRB9vpfis2p|f>U001hzR#cls_)M9xfWb}alA@vwzFa}& zE8O#Ry)^2G^R`45D02UJ<_sO{150<^-RBGOsIiaR)G%9a-+XjQuhs^ z_oxfbz>tpiY3NNZq=jlUCQQn!&Ac0X94rblTp*V)VWiuQicZsd1j=RJ-~hC z#4M~lto}#zWkvc}C?PzeHxuWdFwKnj7MJb}9(T)Y8r~4CsD@&u#I9?D(cc;CsVI%x zW+f_0oGzcwWUOE$CEdr8SKO8SBKtK%%=WD+*$ZWNrY+4$O>p2^m1X}VXR@kb6<+C_ z{Lqs9Suy-+8Iq@Z5yRg3!RCbUFq)3 zN@U)gZKKWgDHN2_^%JwkYmXA$wktL*wmhQ`7!WDf$eL6@#F9Je$RyJaFk9dZsUQKx zY?pbBKjU1i{^k(ycr5P4hpBZTk3^hRA4Ra z@D4looimqbfaQd#DxR`NEn8GHhM0cXDUx@P<)roCPo@D31XXp{(5IwEP?65;MhuNI z4TUHDW=J#YitjS~ArK->-yU7&KVEHXlPd}Pwxc*(eVQJE0=abG_A@;OS@s0$bh{={ zceE5eSVp7c4db*WDH=`z9pdXBp-o=j9QoyeR<%r>$yWBewxwHGg@crTODEr#Ib<+-aPVEU#AdUuSz2f?iw_4)P^DBA@qH+^>)L)GC-<4Er8_ zhTryG9d8^;FAT7wfFVb&K2pMFvH!LG7W%Orb%Fe0mz43b{dIekIOp@CF~fFm1#}I2 zzCbnphEm(5GE^v{viDBcN|Bh@)(wllBf!Zs8mL1$T8WiPQ-XgQ>6_A@zhX|dqT9^x zYQ5iVrp)4lC_7n9Ycav8_#$XecQyDBx0M=do8O!oglG|@hKQ&&YSjv2tWsj;6IG>B zLfAWe?c@p9*h+PEPa6Q=1{qM$|Bq@z@&CqX{vRpqnii1)9`V1>8Kk*D{ac#?g#aoY zKobDB-FeWPx9({@z`&*dQ<>oTSf4==Zoe}H22~D&zBB8A0;7Z^KxAQPD4;_SFAThh zgQ=MbnW&k$k(({hX&9dp!2CaoZlkJ>!h#_BKl655qvkQNh@Fr~Rb9-28qej_RE#`x zB{cM$D4ek#MEBwhcj=o|15*-QKPLY7{3weyaHb75r+L?-hs^42uIUl}BCjF_!SO75?aOA!6i z)M}-%W*=(m8r(l4_x)j_Ufwpj(?Mp0QuiJE6Y2J>H&5zUQ_4BseMNs-QgX{e z5%y-2{3~1wib6ywm1LCd=^Jsx<%L;W-m%f>f6V26kVOQ~ZSjwNe55d@7h$Jo!SXSN zNF78%Sf#T#{@FT3=o&4pydOjA(=2-w%RS#M3*yrC*a7@q79xaz@@p~LFACCWnPhVH zZAF~g2aC4M@R?R*8}tFyIN!QYM%c9pz1^27AJv3=>$cr6@_hS(HT9fVZ-n>r(e~(O z772T`-kZMVJff_VHYeZ*l{P_x>r8s-`eVX6fd+8un*1pDkD5tag9&9YjSyos1G!{# zc16{3eFp5pi%ln}iBUYd0x6q&$5hrwlDd1|B&*=xtR_*Amv}Ns+jD%u2SFeM7Fd#^ z7}Ce|gj4ch4;4my#Zq=33AMI@=|Va%u1M)FP%#}*fISgN;f{ZdLmgw>gU)Ca{s|6S z93WOtGIPM#D6dBM@LELKZI#Ox^g)f47E+AoraBv zUi+pyk^OR`O^$tYll|r;Z2D)Hi-Y-{aG+F%f&X##@w0zc;>XUXgFjr4Fn8{*2Cjcr z06gGhsDCnJ?p?vkpLP-;b$n%a35O0VZQCDTq1=;le|S9m4>%h8XF+U++3L-!;)=0$U0CKYb-Pm{t83JC7bP7 zMulV&q7Kk>kwH3aA)`Xm-&*ILTcJ+|GPa@0Ft49hb)!hGHtLwowm3HdG!Lk$S8UR) zm?inO*kZ*T8OY!?U-<2<9VA}nRyX1_-qZ6NM2n;bI2JH|_`zb8B2zC+rbwAtjSj#) zE7VGsLJ&6kv1rr4{X^#zme?cu;#q3`B)4(lpvaBTy66!M5QOYnD6V0(kf%f|^ejdq zUcrA6I~>hPY!M`2E{gg{@K3TG#dc-VFR-3`R{jtsrzDzxSM2T(^OOWd!aew^r@ zGLJdBib7Un{G4<*`vHvTj{RNR&kZ1_bH4nNuCBK@)8VGK8Cpr-HHiRzhXSj?UQxBa zvM@HkY-?v#X=At2*40`%xyD?`j~h<;5kIi_&vCA%z9tjgLl?P{-J5MqbYufL(jBK( z6iETuIzxE&>`%PpgDNXtZw*R0qG*u#-!{_t6K%~-aqUnXX0UKYHf8RE3sgXggl}As z+dqarF)PhcQL2gGeMa}Xjf@GJ-F}ovo}@$*mN1PHAK|7GKShT;tWTlP+D4%pJ;_ zJmXwTx}+1OcWlAq77lsB-fV!NG-$BbaRC!QAG@D+02!MstXZ(tKSxg78@L17L|d0p zd13eBZXUB``rOVad+*X08QG>moG5_L3UnR414TRyZtZYO0e|Jy6|vVgUi?uQ(HN1R03l6)k8FstTcVkn1Z+9r3*kV_a7X8%p?zd z!->u)SJ+54#~(fcd!dQp;>$I}l->UhPS0Wa*u$ z78tuh(E#R<%rfX!iqG;W22xZul7Q@`)ZUv$mYiy{dP7$nR240|RXLu|_Rpebc4k$S z@Vp}SGT&4MIT+XAJ|O^FbnGQ_%JS8-ZWn;xW@BhqnaF5~s4Zh$ zk=&$Ch1VMmH9Gl^#zGaCPju$x?CCYiA~WC<%K75qLq)mN1|t#Jo8fN@;Iy*&=P=(> zRWiD3hMWS+iwK=~GXxp$1jDxdly}R95CT?ojS3b9C#6~{i=NDx1&xgfySJ>n@9A4P z3+d=ogX18oCIn!fbXL?-Vwy=}XSFWAs!BN42vb8tu4}ngStR%=-tvp?AAN=AN5F5C zSm0ESK+cR4CJ&blKMj$`sM&dT&bNtt+%|jo2&Ji6kSaZT*&3w|nFZZGCDi{yqBm4J_})_$yq zqc6&v0(|nBJOVoAKo<8^PnPb?8G3F>%zJos!egQ#5X={gk|E2O ztnh0SPeu@}x#WL*{cxOM~*ojst`)^?la=f&=65ro|a%Rlc?WbqI6obErsWL@Pe zoIOw*+q~Lr$v)5j;FizJQRF=%QJ+hTqLgmJBOjMOI)A_+u7iS!<}aTQdWe&UI{7P~ zlMe`U^A=5?j(qe3Ct5sJ;*}Xu(9zi3S*#?+Ogs)R?S~_*QYh9HsHS}!_$F&VVPjn! zlFP_QgNZyVwMrhr3HM;R@rL_nbq!?B?LU}hAu@xyrfTI=p3stPSD!twAH~MZEUT$? z7}d?!o(t66{2ReaNE@H&;Y)`x^gMnCe9q!}MeQNrw-XIGP%wz0>@hYapga)=4op&N`w z|0o47v|IHcgJ#tRF6^Q zVYIob_Diu!r@D3H%T^|$@WO=xVF2q>mninw<#MF^4@~FHMHKnJG0HxB|BAwr8V|6Y zndso)qgYSJhxI1H-YmS_NOVhi?I%XG_Y)Zq+ zx7Hi+mIo&p%M82dm@LqZqU4$?`p6z`IE8?R&Ltm|_$j41WQxbd-)tLdIsgdm&`g)x zzDtwO{YZxehTyd2jia!rJbr%0plV2u!VYTS+=2|dSA8s9$35qadQq6Pa<^$FkpA$I z=1yaNe6k+3sGE_Njns;5^Xhj|fUV~-?c_o{GHvXb$~iG4@CYRtlU$@tROxh)f4Not zr%PEU2^34w(|2mB;kx2^cmOc;w-}*+l;S@tx2vUOrBpTPC)a@BKnK*I5gv?3L-){;y~=z~MDU(|N^B{# zcow@pTVy6TD|Ze@WB~ASS&SZPgvWPjKDW&F0#o^iHTUv@_v$V0J6_rZAK=Y|?41}4 zw*7t^8AkT~l(*Ec9$;56+{!)tc3}C*j{UthVI6)W5ArTMoDX?n%*SL!4Oyb0L4730 zP5kJ$Z9u&C<~`1cPK-~^SnR|^p5yvn4X@bVQ7GxP4z<-)zZS!LC(F-NyWygU#MJqr zhU6!cDuljrz1ud_)eU=#wQ4a6u+)g3Wkx&37lQO_wGqh;phhn!yJ-v?zx65#uxZI+ zptO_>hU%ro^#O_RNl}PeED70Ey+f28EksC6ThmmFD^S?+di@scyd&?YFPVjz zP;`e$aEVx`Z#yNN0g;}H;+(&2uaeNNm+-@OYo7Zu%&8^f?|AF)2qq34aJqOzqYtT3 z$6v+2p!3n3kNP-%E3cq)JpZ-3kwaMZ=q;Y)89}AV+@W2`~BrG4$Av@NEsKTmsUuh-bI#xc`J#`Ud ziF2V;+B`bx|Im&uZUh5%3Z4&wRX4!FWez*uf+mf4%)*UmY&V&dP^WC|?f8E=EueEo z#+o<<$N=aG`=WSy7r;J`Cf+)ax;{){wJ!=nEYLPgvM z6I)O7U|rxHMs^Mv_Fq5^{u)*?i{Wzh$0P61L)+y|=i-QjiN+H+WkGkQBQrg=0{Kp9~xmz-0~#wmU6h7jV0e~?K~ zio=k(mfa2@HRfgguW(S5HGj>&DV5jvsgDJ!UJqx9~c> z0Oc>M4MF?5f&c2e8nXE)j{*!N^lRsJfIbx}9ZMc>*G{L&xZ=tq7yz(Vfz0I!b${)~ z7Jbr)Z&$>bEjYG=9M@tsKg~k6`lL^vjR0PbrR-iHTp_XW9s0HiM0hV!*1hmh63VrP zn32#2!C|^H)Nrjy(QuZy8M;?-HH)bcFwvevs^HHtd)O+ek%Ra~XzD9>J-IpJ7;;)d zvdyV~gDLiWUHV{kmvsVYn{QBL<4oZxoh znYxLTthaGd(Ey7f&*a~&!;pp^+dBZFowE_GIfmKY76!eXh;Vs_{Q@TPPbF%ibv$z_l@SIH|`^}@Vo_syLu84;2H?*Rcu49)g zMjT%d7O5V7o)g{*>l$g&(Y4O+^!>B7u z?Gu`hrE%PfWdVJf0S*ArWUQ9UQ#F}458^MfLG`{?$& zMs@B`Ewd@cxNELZ2mIT~q7;zCpit=QH8xiWcfC{1xyr*i(iP%Rx|0~Ry~BkuAv#~_ z*ct1GIDhVk{{xbu?mDkoWl5&=wGoO_O-*!TK-hHmbkVe94v{$jR%r%aL{c5{D}UV#VG{+7hz5)_Q}qaPia7ML);l1Uc>HL9WZYWPUstE_G+cd4 zRC{s{zipCz=xfLD_O!-;{GMK^q+5N9{q$l{3|zZLnnP|xr-n1?plgobEUxv57TQK&wbCOAG0alM}HTrDxnxWa6L!>H_pjO=vEP{yF=GW3=P zXLAPg4e%rRcf+8+7s_(*GMYulQbeA=EsC0*T}njYkMSn|7dt>)Oj#t&B;osxt&Kc? z=L)~+z<4M0hE;z54~2sA=+&~L_3^2eSTi3|=~D{xYx9aCotp6D>f&{ka5D;Z$In~R zWaO$r0Zr%4ggw{DG8oB2M?EP(3-BaI=WKuZHPeK^hT>jd^v z$t*htE5}B2d(i24hMsEov7Je?-wow$Z}}pnnw_Y)t;#B}9JK#ta9Ee-u)QtWseU=3 zor|}Q=^YCW3p?P}Hv97?$xxxXRwDcbNS!oHYt4ali}GcgEkv zu-&qS#gz{;PoGLG?c}MW#&|8olld`(aR}4J&eJC3Zw=|}L1ZbAn%j*)uLWNrI+W^W zr2_%R)3?Ml&BV({=(vYxxLz~GBEr>CnX@3S8>5Q;3!rlp-P7B9dK=MxnU+W87nbw? z;p&^BD{ZrGE4FRhwr$(C&7F#~W1AJ*t{9c1V%v5qPAWP1x_g`+f8Xtkci+v0xz>E< zdXF54itvw^r_ytlUw4o#1GLhsZ?>@0pHT0W#_d1LkFvm@VGk6AC&lS$o3>+Qy+|F= z60{)!Tot5njydShwfmFe*0mRJLgM(e=j&^K0K0g<&UNu}-ZX_9xxWMb(7|`bB2D_9 zLQUMtEH}3NspVlyPnBntAIN4N5cjh`BNNB!$Enx#kBxmWDF1n zyIRx#4y5I4HCic$r{~rjOhts4l>a3*e_i1IIxGf46JXIa?qKutNsL|Xj!q9C#u=(` zm_hP0fB8GK_T>dBEMSzskUMJW-E%OFQVTryY`hrBsg~{23sdau`~8rwnl~KZfQnsS z^$yi!tkzrwF0d$(mxYDo)~ekX~z$1vQ8tRRW@)B*oSb;4%=nPs;hfj|_t> zMh%IR{=x}Nhs07gkP<_wongxDb$9EC=f?vbT^<_?=D2m?%8E2X@e)`n3JE^hRm zY)Wj8OM7A^VzP%*{Yw08SkxPaJtz%#3;iK(`%Zy+slY1+Cq>$h7a*JE3ra}i>lM^d zT=1RzjX>@9kIq_9!_~(qzi!}jlD_bmI8tI0w^U40drX~ko_m1UIZORA_)8o2NK(0J3X(z}lCKlm7C zrR5oV&Vcf}@p8p9%!ESV+tdN2Sh4gH{HR>(VE)Ad-G-l|x@|EILr1B~Iy=AK<5MaM zj$p{+j7z&#HBQfTg*gCv3_hS9 z)2^1uX0fqrKl8nZ!HV~`uH33MwhfZvGS4ij?$Tc1sOU@=wh?zPV~!bKL3SS3GQmpw zQb@?c1RB!*836QTCcr5ZFFuKldddqs9h$>v^@Fq?mKONuiTs zxIY)hl{2IUVwpu1Zs}OSnrKH(umm7{*D7y)C(l}NPa)h3%poGrRF;6WbWxokw4?1X z?`kQlao^1anQHKStMOwUUe0ywoMUixE?={CENJc_y;3dDKW?i1>63;E3}2|fwWvMf zK=RAnI_H{83SRAJyJ?OiVk#p_n)ENQkYeTlmzbb9%p^_ZA}nRLRVCw0gHV8=GE*pc zi%KKNk#q1G8v|X;9y+o#YV53E)`+gQ#a)8tnEesi3gN&uvJb`Z1!H&*G499&VmodI zqba;x1ay)^F!|J;zOZwTLVW{KHD=J2$V-eFfV5WC*e(LOg|nzN{>z-`GsVolSpoqA^=pU&53y@#yL*SP}|Nkyv+N8%oEJ12Z-=?YThoPfO$!RH+$)-4^}IZ0E} z-;_^B7dair8`+sey+Q#H%->JJ$q&#ER1#Q-_rC`d#zYki&sNQpQeq?ROGon)N=Y8T z;KXBhR<&qJT6kC#J{FA^;66y!pZ?s%h=2l5=WhG!v%In1^n{&v7FXuN2fbzEk?hpj zTBoyB>R|IgyaLFbp(Rb*)_ru`@%C^2sh(K)`QmER)7@U#Hby~Lau~)*A77h#zu4HX z?C*n8v(j_Q8M<$^x)BB|^GDp`WRbv83y%-4wb#_K{ay+Y!?q5mQYT=2gNtMNKrajB z8F8ETh_NVhPGG`)S$-Qul|djXo3C}{O)k1K*QFipH!A=>$HaoBO|mScQ*2%udkT{) zBwQ(#RHp$YzhPs^)b=2KrYd~huHIu2*+v=Ps+p+FFolZ_l)AjB?1)+T z=^xli>=85!E@w3fCRN%wQu74UlT&*jiERfVm0YHMZO!i5NUKBCF4o5&*r57b-=LK$ z|2eC>FEwB)cK-alu#9dJ0k-$sp*x+G_!S=NAPPLWGmcgtv-An3MMDbZ3Y0qm*B%MmT_Vgy`~>X4-{b(g zAq0S*K1QfBdYQlms?}a{{s>_<98S`LAX9{|DLE zGmifY&vf4vkhgy|O7m<>>E|Hdz7d0`tMJlur4ymh69fKpFjmb#2~7lDVBWy9?VL8U zs09d$31QaRS2Yy6z)3;FQ5rf##kQ$w_ z$cX6hvb+*F-g09z>J9h+HAZ*}@XlZu7;wOH6I^{~zBcWT2+_gvGEm1p!e!+tr;sg+ zN&yrtOTP%a?{O97oqG3qB(Z{L%SJU*`lLn0^sn1{wQ{Ys@QRgTP!*!;_9DLNKNL$3 zQlCaqU)cTZ#pImq;Yi`#O5f9%>{kDfc#1S>?3fLltL)|8cWL;w!|nzeex@`PDV>_I zoA!&f=Z#PCkUeiF=Rte;l_jL5xqHw!n->sSj6pX`h}cQGd1L^_zbQ+~=;e)@{pVzE z>2HQFBdv?GX5aYST|0&Mc_Kc>u2+%?Zj_{bfh<6!ymQp(?%RAj>JBPnSNrJA#fAkU z@(wA&&Bk}SDtNWhFlE;kJi*JWC}}qOk91Rbq*0YLOBTn!+J#Cx-zkBhvXbDh3={w| zcAe-eX)oWOvSkVMWPGW6ByK}Ft29h zGN}^>l~hO76mR+d$!Gro1KJlF{ueQAw>0#|p#Kt0B$k+X=zQfa{q#a#JnVGjV?MGr zDq#ZO|D-!PRRT+pe`mg`2R0Irf58kiPBLx(D@#scq!&Jek)#_3@_%JLV?Id0Pwi+U z{6&n9tHv&{337gL)gffmWJ?`pIBQdIAPG4;F^MKLjg1=HMcf5>Qo^65SuREuQKR>%4IDE7oVVX=_J|U`8{oqw^9Rlz@AtafywON4K8J z+a)o1g+0LUaBCI}SuXHwzVgT9HWXqb?Ik=o+4sAt_fst`d95*2s-CXD8ax<$+}CE~ zaco98M_3idx6ZE`enis(Xp_Si9kMDiC>tKgS(hY0W~r19i}5G#S6W!q(n+nMW|@#L z*y#sPQLPc2Y>x?I$v{5{8Ih^+nCSzUlkw*-O=$9@A)yEGXYIw}nvw&lOqA_Lm!8H2 z!cBRgL7_OIbPD`4*VTxTSA*c47+(JF3U*vOH2Yla$vBX9HJ$wii03G3n;1g~nYjBT zuL@u2jQ%1z*v)}nz9>2M&y<427<<0CA)2Rhr>oU#I5p8LGpE>C=5k-!bf!1SaEY&A zVyq7kZSt7V$%9rr8}S}6kw;)zc&Nz8l)+1<5z8r=-jmDnfDk&|V{6?#Y3&=_?WqZR zTH(W`;YI#a)XiN1P_X#H%bl6pqgYTwlU=Kbrxn>dq``df2xc?Aw+WH2St^xv_^d9U zpV072`6W}>m5hoR4;Tv}R!}&%N`^dV$|8`vwLUjT%G`kn#0bypszul zunRMOagIIBwl#p)l>eO!(4FV zC*~W-82wChqV(>NjjhUTuqH;tO##piTWvO`960#QqqfYN;<&z4YQUeg^kWYMU;PFM zG}_U?*vu%zfOBs~_cs`}d#0G5Sh8r5<>^(uwR@K|j_(3fYa00xP+AdhXW1r@C&T*l z=q^O~Z?2XfK#Von6L;i3yfz$@=DLm6G(*J&%&GF3Ah2$a(ozF|lmWtxRxnHl;twt6 zO&U<*U1XT(HUrpZ;RXh!(H*i*`$1j-7%@VB*BIWu@%nxv8J`IBx<&dP(Q1!rI6YCy z&Eff7Mzp};50C!Njb;$RA*DD#?JfE_}bH@tFZ+)qeahqg(%uj@#^7yu$=i9X_Xr#O*T2Q z9x2W!K-4^jKEc$W+~07HtEu!7EkTG6q%fLooM9M-O*=c8O$0@q7X@5RevZ>rrJTJi zhzPCktw1!bb^lZEz@0j;!8(lRDxgq1poCkT?vd;_S0LHD|DQj5kk5w!QyrX- z0qyyK?#fNUI`2R^8NQf^l~%WV+F2dU+Ra_oYe-1+c>Zz%6s(dgEhBDr*-kYbOj~|3>5j&Xcgn zH6%nMLLUvE1dfTVIs`_@LKS@F+%U04>?hdf1smV_i2e*txDN?F9G4&oF|eae;t+0l z&#>c2xst1;GNz(3_Gh`^xo*(to;QNRgKyGjf#A*$?i!^Xp+!I$k2-63YOPOU0ni({ zp;W^UsrVeJwBcbv1%IRJN0eQqRTg zV_(;2fCWTC4ws)w|D#K&bKcg1^#XUu2h50a@DlS$BbM)|gC~QdS|VOF1#2GKSJQDJmRvF(y-oGEpjONoL%||-INFg~;qqWCNx9;gXvrp47xHM|~05W4lda+XbB?WuOXEH)k4-raf0;{*s6N^70xEywP>@#=o;CtTDlgCsq z3jNE!hIs$KAHc)--%i7aseI-5YjmdxlV111V#CbPQJJ*3k+#KxD89uq8V zz?I z2wKzchcWdz3dW#t0MnN8BnQJ`DeXP#D(w;2(NX+kZd!Ptft7Ld$shk^)>iEM%1wIi zwCihhPK2_p`ibO?z~3&FXno!y{oUF0CV*kY*-`x5*z=GzQ5;DO5_-lW)mlUWzR$hEe8 zBuxWO!WLt+VWTA6xGaQDz&h@~E3p>t|?vod9EFjyTpSAcr&8kcD=X?-o@8Q_4!F9#*BU%uPZ)R~H zb0qZ7ulD}@G`E@97e_i4xK$=l#uN*h%d1$$L){Fj{+9;bzz6;@!4p;y8gH|(J(5!LA&D!J*=xMAFyZp{@v{6 zJ@f1qUZ9I2ko0HaYQ# zci|plQ$|QKGGB`-unbw|7Z^EZapfkW6Y>#PB+a&LkNXbh-7vIjJ@c5T)u%X+OiEyI zNR5QNi!iBV-8@Q48eE@ta0{fiYv{oMORIB`V`aDmrnY!zFN0hUM2$obMSg^6Iltf& zKxOThoAdzFbr+!3u?ONw8)tJ$^)6Q|dE@$bh*!h^DX43mB33Oaq}lDas?h8YtZUCs zD^1mJA!1-kH{${~Ct%`aLd17BJin4z1ts*5qkNS<-UYV`-ds?VhrSt79ke+U?g#8m ztH8X3nE}B5Ate9J1z%YL;Xl?0|CYG6X@RrH{fB47+CvvY{flQ{LZ5K6e{qvyi1a#} zFX3y@=9dVZc!viyy+(tV98jaa?l>=uEPz+Wq9BhaN}YgdEk|A9$C4t&$b_jO3{r`@ zW;LqAp~rJRZm4x=McYqA%prQy2X&(osl!Lg6e>yG%HPDt>-KnY;SKnB{Q&Dj%+W!2 zR?4<;&;&9!=ne!dDDX2kSniR8_?$s_80E6C>yo;YYn}brf8wX{0qnGAI{7x@-TXrM zdK7j>BPK(YD{f*m^fm6V-!p_Pj0rxAYXA5(#5#L-9Oim6@>toy;3BW z-xC<#n6?j|$M3r35l52_I>05p!e^TS8oqDqNJ3~lzRO^8fD;XsE zo}qP4x!!9y=PGO9a@gci8MSah8b2O=RgZA5kh)Lc!F3Am^5cqJ{N2VWH0ys6Rk(=kt4) zalGOJKNc^vDzW(f1>qko{uf&RTd_jiw@Q@1$SrcZvL^^(`i&4MYWn*b9}1vE>qQH1 z8RG+zTr!dc5+ay@Lc|ai4nzgMVI49Fo7qkWKgLBSdo9G9m}fACAD5@JriTSz`mR`!WD?JQ=>}0Ys8fB`FEPP&gCzT?-5>pJPdEqrqjLH!gMk zy<;w)7rp9$>|rnM?twiG%x#NX0gP=VGT`z+d(5(qA6Y2T&7TO8?y+e)lK04bA;ev( z_jj7;HtZML==_Ffo@k@`XNqV$`r4-FRA3thJM4Pv1&W;nwQ?!CbvPi;O_8^CW|>OM z+D28Rsr_W-b+aWWIJ`e5(44Vm>bUQckfR-vSEE zAQ~1_f~%{Htu$k@jtS0B#E8?Hg4LnK-Atv8?S0(>%90!nPJv!lUH(TV_?=&fITJqOm3mKxYIZqDjjjE5fhFg(~zu4SDJry}4e*jVUV^+0gKc!4=In>Mh_ z1ec5Bx2V>PpHf(j31VlsjJ!>Tl>^}fE~d6#pALc%D8aLYJoO26zU1HwyjAYYr{ORW zdw0YWhkp zh76OUq<>6BZup7J!oipa3o}%l>$_Dqp&jY8AtG$MaSr4UJU7+i{KkLlrjeX@E=pP{ z($nYS2y$?8@+!4xmBeFCP=@hs`3UHC#8-6k(RL?kf$ChZZxOLKggTsspYNd8sCc_& zK2(!PQjc`#_$>2Zt+xoaDTU-s!X{6xosR~NCIR$F!0=-#Z0r2&NR?r^bpAd_A1*J< zPTmVhCgVTcOZ`>Y4nuKt@m+Fpn|CAnDFmEOeCPj*Ta@c@Z!hnTRbXFqHSKziB^`bk z%`JK|FkdX7y%7ll531@1IgJWGC~({l0iCeR`x*SI06xBph^B7@!F{9`%6BxN(B*70 zRtp$~atgYDq%f>*pC5$ly4S`VV7|0GK*92X6&ZjOjq04*QUITDkt#H;2Io514cE`B zW?}3t$voSGeI+QivNLruMni~Np;AVhRYG2KMNW)6o)O;M3_YF|t(l5Lb^W>Mwm^$k zCLL?ABok`@Y3-46f?@DDUI6ltII^CX3<>CHrvBctXNQ#`7|48;MQNcPWV~0WEX2t6 z;Tq*K!zLIEGp&6aGuSpBGyRG?PwyGqd$wfYckgh~8B$sj*&1c=d7vDI)<&^eqYaZ( zrM_mvFZ{V{kD=6sRpqI(0oy~(kZP&=77!Ia9+lN2ZZqSHVen+ad)_K zS#Y$aRWt^1IMl973GJMO^T?(bv1GY!;N5Ffjna5K9l}&@$TTD?OP7nRAZ+ldBaWM$ z;`OONp(Nahx}{9)R2406CQ%ySpab~(yds~iz&v82be_pYGVT(DQj7a}n(C8@F4H0F z4%x^A39{%n(_N9DUw>d_)sblfs@~Mtr=1~vPp;2A_=j|e1^mIf(Adu&4MT9fk-op` zS!~`Q&i)iJyEZtB%%&i#+$Nie$0Gl3$uDWZFMZ$6l%>4=OoXp`n`-v^QUXvXqS9Rc}+A=gG{|9%;@>;CMp$t!)J*r7!!@%J107*ahHk*Sa%ZPbWJx;%33!ujxh z&aLEFN4ONMH~XAS@Lx>i=nFH0$JLKt-}iViPWykENeGDh%4vbb zV>y?1J3FNpdv;3~TSN-I>OTRLIx8G@>?jN!2^bn2sT-EOVv<2SD{-I%%wF#{c6oJ$ zi#V76^wf6QxdnjI^r}5A9QjTydpJ4NS_Q=Bmrr`HEpzK+--0gQ|1H+tP^#3rrdtfG zwUgmEwR6>a1^iQ#|9|O4|KFlKh~?kLiVr#l_OHGWR{Mme=!;YmK&1!Cz!IbrU+|%% z3qRpQwcVwX5B>WandeCD=Pz?99@6x>wXYfV%s&m}mkSS4IxvI)5kP`04L<=bGCQBEbo=*WU!Jfx8D?zA&G}=GTgo5Yga2dx1j4?i8nu=eEH+#@%Jrj|}2~q(L z*jiNSc=`k3gp1EAsb=T zNEdIC%0xFaz@bzjt;O++WOLZi;a2f1ce6U&wRdyO*0!6JByR{g`q0lVr$p+FiPKw3 zLi@_`J5pwwO43@CLZ3@b8L>l(S54-HS|HBnRAcuji;;odK1l2*iW7a;^)`X}+vt!+ zFMrI|PDs|V8uKeKCYD7~?i@y!2yn-}CAAV6p@8=Y2!#d7X)cQLKHIRC4R2bo@{m%) z_&Qqp{$<~ll`F|>Nr2XEY=9msRU;`{eMjyQ`&ar{U;KvMg{rWg1D8vy)Q>Z3tfiF! zsE1}k=9!onGu}pa|0{fzwebLVqF#H!)zz7fwQ>deyEUyJ_SHC#KR(0;*4BpQJuh@6 z-EmF=VETyAg&nh{^_G%$d14Bz)vG$n(1K}+RDfZQ%`lGCc8J#%QXaXDIeo;`9%CYg z+mIa1JFjQX;3=E!yhY>c@Mr;Img!AaQGa;`nt zWPaeEBk;(HY+SHbpQMpkn9C*r?YP$8PWd$A+(aK!YL4-C>oa9Fa(tHfE#K~CRNWDe zbCEMUTJQEgBz800G@RXG;%j7zlTF+JKI$hfOSemb$=o?{m^pQ30-Pu9;7BNxmQK4R zW5o>$2E%h$^DZE|9`*y41a>!y1X%l|1Wt*2qSZ?cin;{G_$4}8hP>`tl_B#kWR7z# z2%L7_hND8aOax4kJ%D`)0;aL!32%u(fO|KPjMs2bjQ47UfcHv5ey~8X?~)hbIKm)5 z_+tnT%~Nv?we2?}TLsGL!Jfpg*mW2-2O_xA43gj|2Fu3eSkw!7@(@o-b)p;!{|t@! z9+zaCR`6hlWqt#mX}EcTg*5T<|6Vr1XLn&+{kn zxt3zJ2Wse}p(CFaV}dNZ9Eyaign09sq&g;Y1Q0KSJtG#E%a+=k_~!z%?M0@k&fcnL zm2lk6_m(ClCu5XxJo@OC5H4(*^XAKR_8zQD8@3$KxMp-eR0OGbR<*4v?<<|f$EUfd zwqfv7e~bAO#KRAS7lM}oXlCc}*ev>2&AOLY;uz?bIqM8Nnjh|t7OU2^&}c`;+vM9+ zjzaeiYc2`WxmxP#jgc?~)9hFIzQi8ddu~~6$ zRv2+_IW7hl%Tx8Dy{g-<5v*)ui(Es?m$2qVT-gs#>+Fko>=k|l^j;J{G4b@0C)Cis zRQm;NOoJ?#9hZ!;K+RN)=e$@PD!6;XMkbA`Y{(G~%&NO7)%h|%+|dsZ)J;b<`CQ_i zt?+F%M*7Kr}9w3%P1 z1khuIts*`o;}~(Huwfi(1yT-Uq7|!UMEqe!3&r>))=>QPZ8@i_sXBI@v3gHPaaAK* zO-sX#u{u0HFD#s}3Ga!*SyB*VN`nc02}Qb`Ln?byAFM`zdgV?F_TjC%0Dw!2e)&U>u8wir@q4^A;(p&G zEVjThguRh_450na{HMHfd*wY(*W#nDP#uJ15sj1Z6pt0<4+OVC_q=?` z5+%BnRr9&zRnxqzmv_=ow-tYTEZUH8g=DJ|+Q2J#AQjNFdjxI9eZLAKT#LSART3&q zvF?f&0NATpt}yCrUMEi;rM{dAjOMv`PQ{$P_Gu%hKp-k?PCjl2WqVKT^uB3a4uA7u z(O86gM(=(PPVog{NLh#J8Ho(3XePr3v;t8?rqc zfCS#nJEOI#;9nPkI~Ji|k-+hF1csh(ri_%81CZipISrq1`4yMk3gj)5cGH10ObpG2GcSZZjb0 zoy=cY)9Q?U@FtcU-&H$PYYWy>0EW?^^p;8&7Ih0NoLGCZjxU+0ZDdjCdWSIbA;YX( z0LvYhVxAa(Z%=7ST(u+2`+g~lOqYu>v5%>mn(=J|`yUP)uCuJ*xh{sC=vue>@q3wk zDmS>mA>#TB5FbmTe{u3qsySb5auUy|JG!Hy+^n{QF;qo(4`{%WX!o4?V~?k zmTO)7<^m#+Fi2g+8q)|=lBM6MXrA%!4#5-sK;Xa0f?sDiU@FWN)tL=f9_waF0z3)k z4Elvl;gSm(T1d~Ql1%dCW3#*@s>u8?xG;Y_m}V31nVT{HqY$~$QMp!$M8>4f5p*W7 z2sFa2BB-q74eD^b=zC7sWZ~Gu>C1jxF=OL|V>upYP#5-N&U-XP6&ROainn18!d;7( z@JEb3B~F{&hc)~ap74FS_&sO)OS3%avEh14^LyZ7T#RqgQ@_rX^=-Qpmrh?Qqu zXOQfGo^rwy+aBQlG&}Rxk!+zZ^ z&-vjb{b?6}0t>0PJ30;e3@`J+z61`x1fq40ZIL9Vd#E?{M;&pwlZHNYo|!>@xA zLu)-f)hfQGLDNQ`=Se0fGb*_9Y_DkpAN!3Z-zLBjXZ?6`P>P7q-_hLoMvB0S5w6k7 zU?M?S$n@wL#{HEh|Ai@oHSehD17mcI&AMRQ6yk4O<};GE6hJgIPYo|~+W7bf`gMQJ z+5P12H2ID2^+isFVF|{m005)1lu3Rd$p(a!GbB-O_*2h7*+J4aCIOwjRZG&OhOF~6 zP4Co71P|iXU0{xY1r#v+lBO$B##F#JPhcY_x zk-iOcy-SC?gy?~QMB{4UcWa1t^A$0sThW}hzoJl@`jZ)r)?aN+fuK?`zmzjnPJkHW zWXFmpBN40tuB`qGwa1If5$ukAdLE1#*V`Rp2fK3k>BU=HWrj`$NY4z&R>jG>2Mf`x zJE1Gl_66tTIezv+C7`%+Xg76`$47%wf1)zS@F>e|lSitg(Rmph@~hAoP;k$4(2N3G zaF|;Zzsa4`LM-x>&XEcc?xfhmceu?mg9h}@U5rBAB(VAPoPmsBaXp#6Ami}E2285} zb552u?g7dkfJh^V9ledgrvtBsIjYa5A{;!Q2)Y}D_mJLvMoG5Rfro#|KL1M;|Ce+g zR=rVLebLN&m~^MPFADi(CF1cjKtT6>B*e=WKUL`a}+Ka-qH)KENhu&r!!sL5V3d?NsBg*gbKY%^iG9WY(>t zXK$1#Js^*S0Dlx&54jf8PMN{lp6@wXLWyfD_OvR{utE);-*iT7iAi6!>)pHLAxFCt z+tp+jezxe(7RPe}jqmRLceLDzS15yFnjFp97|ebF%^PtELNVC9D(n&HMJn}5bwGDt zIS_ONJ*Fz7PYyzj7MNk#?$%s7h$}rz_q#k}_{%!xb3GZ}B@t}0$Z&yDc4FfO8#4ab z(p1Z4kZ~QS)*RkLM&;ff8(XZ{mrBw{q94!pQD#}^uvUu%M9e6twUF5!cFX-@h>)H` zne~K#tSWkA9a60%SU(n!p@4y?36N<1y2#$GuedQa8{kdg>3l`Vb<#b_z+hA|sL@BzI;t#n`QO*UTLTL}2tF3>tuwQb zDgo{7I!HQSC-0%GJA=E>@(q8`VswivOWhpv~g+1u!EW50gb@ zeA}K;dcRa2LLN>hqi3@$#usJHp8En%^B2McuQ_=}vP#~HIMq4aTjM5jkDd|PqCHZM z1A)jT*c~*U^;9WAurLsgiu^r@KkaYSD*_4ncAau#uw#)497;HYI8H9mqP0MFWoQgx zo6Dq)0Cx4zIt_D%mZ3<26)?-lU>0;za6k2QEW2nqNz~u6&+4-P9~;;ui=t0v1GeJS zOw&#XHSGW6=vM7jqu|i}2ySKfR`X$k@6}?X_<|+BE37!bB_&>dpMRF!qA)Sf^!P*_ zbWtg1%PPb*^x|O;XSPe6>hRaU!t}qv_@6LE`|?%)_wsN8Wdd*eYfX3$`H#8zmqR)# z0e*Vi7zlLwR1G;4pybQs=}Y-^9O@$g(*gSdKf>#U*1BdaeMB&wBwC>Wm|%3F39~!2eWOpGsJ(H^ zQ#TchWn-^MQj>{cmdAfiv6W1_WPn?zrO}wuogRX%>$KH0`lSb}$CHra#DMo3K-=XeqqXmt%9#zmUT%4@ia zYO!+CKZc8K!!bX2&V?F2Ok%^H%c{;b@O9{+HSW?702^Gl-;fmag}Y$T94L#t&5N+i zC`VXKb{e8&_)Q}7_1mK++C_79rm0LL36(k9nq_Un`;-41AM>`sr_Y(GhV?k%M8PUp`jEZ8ZrddTxhv^zw@#)hHb z!Uq}%0o>2bkNeKID`7+)-woNn{SSsma5?sDPJH=wGzU9j@ zWni^r_~0>Cp2(Ke@Rvn=e=#q>aBbkiB0Oi%0VA5<#7D2m{p#zOSGbxRuOk~aiM5Ov z#sh1)Yxi(1SWmX1u`QyLn{h6vOd)HN1*lO6V^7cvPq)R`WV=Q-lBWE>-y%**VOgHB zxoFC@5&?q6H1br(<)R};gcDl;dNY6AH~ zP>d`~{Yt0Z^Lf_a_LtM0 zzKz^6hoZ!A5`$0l)j)!?V2TQi;JM(T`L>(w)P9!%QwvQiy0=%+PBonM%uN-~5srre zD6lNk(C(INs8Els$$z}9hmGZ5LnGXIXgbme>K*JHgnQFeCpE%p&M{U}lccmR6BNqk zu{*~*GuH6aa1#B6AUhoaE)$F8l^QTuS`jHZZFcUNG2CoG(gfSl-Y@hjbcb_S>zP5 z3r0roz&E*4AIgz#JOXBXUu7#qKI_m{qBhX6;&v5=iUQ_OSHFAgUy%B!A&52uazsVA z&kxbHzga02l3b>BSL@feWUV1W#hk$*$Y*EtS4T0{=d3^aWIXXV)bwKheCQL0o5IUr zbHVsS=}0Ue`H6Ij5?&RwdwsyI5PE{+O)9T}k;_z(^ER&2OfVdKRKMDu;+tovuM=0c zlmVk5k1$IFO@W9dHPSQ>fNsZpKhw|o(5G?crcw>2hRA26& z4~gXPKD?~t9ya-{P}c79k|kpDT-yx6Y#(XIw!?+o1R~WbUe5%@Tcjyz)Q_`^b3fhL zLJxlUCpSTkLe7q;w~`&G`F?l8-R0q3{)6{Vp!^s2|0hts;)EvGiNgi!+qZ7kZ|R^I zSorD4knC{jM%z3HfEHdDZ}r8+<8Ga;-v<=(sFVqz2_hlpWFQMcCJ1A~kW?^~An?}v z3GzvaWAZs+!w}7T;cK?hHN>>GbPTq`-(=7^mezC~YIKJgmp*E078!eMY96&F?(*HX zvoq5Z5_UfS?)p7#`c3~l&hfj_*}l1pD$9ci;(2b3(cyI70gyXVuJ8E&Zte53&j{(z z%*q^4Ammc(7)zhE2|>>p328FtUx&2O?h83(%`*#e&*}Gu?a(4OYrxp*L&ZzmlXo2V zHsRrF@?Ny3W8cSjoKpCHe#39*F9=4f>dZK`zXidsHTLsmgWi}PuA?n7dstIDTCQD- zVqX=f%0XH70$_&FYp_ATZ`pw$TfYQ!)Hmsev$$U&4)&nr70f9Xf?Uw6?+>TDB^A^s zGqZ0rW<%0+VT;dR+^9CWVruP|`g%Wi?otD>%+f07eWVH(7`0sk-w}gX2eGWzfUxec z5LqMW$Xi{H!RO%K%!u=s6SeO!#xDc{jDFft&_a160T3IvOtz1K3IB(Ol(QY&4hK7x zh4Oh@^8L?{q-0lA_M~Z^2&-emccLboKngA`mUmmn+J;@t4a8P?~$S~QDfq~n~T-eUTLF2Ehmfer@B8@9aEd-aZ z+Zx^MozKj*uSc{G*zM>fExhP4;y?ROY@LXY*y^*04}b<_^E(ug2{dWYwFWMr@JYy& z@qllr5f<&=Gq9ZFyH*#ap_cpg1ZrOoCtGTU4q2auyVoKUghp%)ErDnRCOCtW&}@>y za4z|>8l`ghc06@W#OCpxaah_Yb6vjO4tTb$;6FD%S}m_%ztMbDv)ubUBdVUte=p@+ znw`BwSbFyM!8V!VMn>2aE1&E7nXSQg{|q3TrTk1CLa8N2Wv?r%6m|6%X=rdH-)Tl^ zRY#-B?UCHZU;(TwE0!;#o`F6hnQ%@^#|jO|!I&EEAie&GMwSnw+mNUlQ&LBx2!$a8 z%xDpv8!E+#VrYj`EgT_KKZgJ1U!shiWfbbjNxFq*b9ttbpSwnYPn2y*$9(yr7YWee zYLM7~_GCp3o+DV@U~DCBhuf4HKZhYeG3=x`)5AT_5JCi$y!?8x%S@s`MR@DpbVLOn zASG$W0&zsHRK4N?Tm^8F&Lo}swy$DeZTH_|QCugW;YOJHf_JZSZ`M%r(v^K-d&}kT z)xj;(zE{vyJF!zrXV-t8gL7vYL(xTO1z8i#Vo`US88z1R2U1RwV!NUGX_cLs zQ*O`M-A5z#n=Uj|00qPi5!$j2z&ZEaHmMW3LtWOZ;&1f(2B5uUd(lg=1%SDc0Sx+! zC|~C}jszFjn_{(%kvru@N+z7xnvrF+$Cg=@^;MFQU==`{Q;X0Z6|6?Tab#$LtyJ0- zXp&##ZB#B<5+^!lqLHh6-0xPNf~d^2jxf$Y_P8}m72jkUXAIm&Y?EagxfeHA*Of9G zhiYJVDZfrHQm}ROrB)C22+ZII{z*!#SQ)MUol>MrP^#(sh1>HDr?w1Owh4RGH+0~q?@_G4SK45xae zTv#NsbOsk{TWQ5|ohfiTh7SMOUo!KA0_{3Nj4-!|)M)eA98biNZbv#Cjl2r|hV7SnbvVIiXQD41Yogijwi z*!__pg=n&zBt@~u1WejH9wk^ZSi@PvK>Hx0cGl3K!?nGnz3_BNFB=8mLQi{&DyU=(O;g`%519twlYxP6jh(I_-;9<0f zWt-h-K^R7)V|Yj5`a}1Y?6mEcgXZ@iqmQ9`6A3EsIFWzz&nTS@<#hZFdqwsQcc#2g zW=`rIG08iA5Jom23HJgvLd^UrIf<<@l_B;8Y=G=QqEoa%xGR}{Sf+Ay>G0w%EI<)u zj4tjHz)$OsIOp{B@NKqh4*}%TX_;i4QZ1clj+R8k4uIjZda=3Zke@5Yxl) zPO_ZNEI;@!cfn3_HAZik7xL61vI{){6Cp%?@Duto8-T+*lxdf<6VV?xU*$&UIW$A< zN1yzqIl+ZlE?cRT<(KO`%fRs^aTi&p#|Y?Fz2^rj1k^1LL~s0ba#~jisV=WwcD7d9 zw96xWKNYh}gUAb*Om}YUN6_Lv#Itxm5i1yl$H(N9GE^e*wq)6^SOU^X;l=tO9Bk&7bO8-?@e zFN72=iM97YjZy6BDrl?NOC_3ljm-)+ba;CgN7yhf@to}USw>#AF>2Y=MP!INZQHhObK+#uGeO6;jfp0t`-M@PE z?pk|y_fxg1mflz0@K45(*?9}Kl_W{s^z!x{l!gd_DM-0OamZ}ItK z6j!bg)zT5X{cR90Xs&z$)xhCfcZtJFZt*H!o+_*J2-eq8bzxrdp0l}Qkn|Q=eIDT- zbGn+A37&w1T*OJ~IF0(0YvbZ1WS;yUqj`AO8R+Kxuspgblf8-w4rLy=Hx1iE=D?zd z$=X3oao1n)+P9d}i)OF$O?eYj<-k4`8KvTKv%NX-0mEqQT!rLjyw`F88Zb6*-Pd@z5NcqP`R@9w|zJ=mb_UK-m=V924CV8Hqp^Q7u!^=l&iug8>UE z4cK8o-A=164cWU_O*K5k&O(XMenj}#$ZV*o&TY;^84%G&Cr^yae_NEE`v_Gd%AkR3g;E}Vqx(hT4e zUU5&UX`OKIdUnem(@=HdQWtjhU~LvXp!kk;p-DHwNx#X#dfHkbA`OON?Gv>qK`ETs zcg9+vDGpeYl`7gbL%bjOxE=0LzK*#ak~viPh+_gyg+&g^?hz!K{unn$lt2LI01f1T z=znNEsA`N1VgI8Tt0mYs&3c7Wo(Hh%i%jT_V6=+X9yyxBv-k-%0rkUfXj?7Tn>&Lu z>|uQ61>>w;)MvA_-EnrK(e~nZmnO(I;yVh=dBN49>+bJa&teB(%*f7MTtNJQ$H_pe zRhuqnNAQ4;>#mQrZ?=Q4VC2K`0MgW>@cJY1(3W@lGqxU2ICtq@zfb4ApBTVLRT{}C zdn}bBl5+Zn3y(-nQE3v4f@x+|BO^Q6A=rH38 zZ#j5CHJrlwy}RDPUL!>p7Ic7_4OpKwXMu2t*)-Ub6U0~`linAUZvAR5Npt_!5pa)0 zm~)zK>0@l`dulF`rqrIuz|}FfC?Rjz^RXRKBHoC%7-ef3hH;KNQHzFONE8%a!^p0c z{;o|p6utE^P>V3n;c9d+&#K$jc|my(@O-pp+~|^t#e#O}v6n;u;eF1*W+1dbi#mcJbQ2SP*USrwdP5XM@3#ewy8Yuh* z{;D41TF2EEstXkK3vPf2@+7*)B&B;8b%v`iQ1BB%HVGCB6H zj`-xSY2>48i9|MnA4jCHLUdN4j9lo8BzHJk?~4wFmW)FK z--SHrZFd?SN$=^~*jS@EU^>b{GdXzP2}PWvdLN0jWFU z+zEL$x2e#!A++F3sx{PG9OI!#I+>+lz#q=%P2hE7alHkw9kzrV@_C3EjEYU}2y44t zzjZ`d0L4s*xb4anK|U2-Yy9P~XZFue%PN$t{x5vN#E$~NZ78e&Oefa1kkze0Pu8}; z#%{%{vP&Akx&NXQQd`tRZ+a79)Q;*Uh5%S+*Ws4^E6gk8W}wGq@};gXu3vO#u*a3@ zBbXl|pn%p}k)`(~4qYHdWt%Y$Um$^^Td+XyQ7R2KTZ(mSQ-#`>Vm#;*9eq2c4q8Em zJwdQSIDtW2dL=^Q$c9aNRVm3qJp2~DPR7F|tPT}`e&morv>uQ;6YwX7V1VwZHHGxf zq;f7=weXPH&>JFi#+FOo8w{?X(=_*%83Q|ptc&1MxGKKxlxz_5NMK>->C!Z=BXT0C zaAKNzWS;vMm zIaXmn*sb6t%{HrO+AVXNL?gLq%Pk6^iTrIS?j2GvIDZd&lVp!x$u&-^X7~~g+mE{U zcg%;`V?Deh7#$yEGgq(CNZg2gQTbW|jI6bH+VCy6nY_~h9vBou_t<22T3T8q;si@^ zxUqvn%S0&YE9Azs$e6GRFw^f_hua~4Oq?J9<>0&x$>O#L0{gQ{_yQxq?aKjYJr)cR zPm!-*^#bPMw+ucqg3$^77;U?~6t55xa{2pi|3K{+TJpi@Z$t=_L3n&fBY*gzXpI=C zE!Z_g=qG|H^r$LnWU;)zk!Z228toYk?=b%)KC?TF$$!Oo4}X>`P1;v<{O0UONQajM zpv{4kYm!T*O+f`83$?NXZYoda^nUnM@92YwK7#$o!NS}?l=r+*K;Ou7hE`O7r3xlx zYmpYBl@h{?`=Jr@<5)`Qjwg3a_L5I=_ldsySAzRJa_TF}#w<&w%gP`ntMmD#mI#?I zR#!N5(d&HU^8#kQM&40z&B4Wi7)BeNcD?>Pa(8D1P<4Hgv6fhV1xF9`FD3+u=b?xhid*1pTI+=clt)+l1 zr#@>T$@~ruy-lSQ+jTHiyLYj1l?k}Y@T|p_L`q;fQ8u4LTq&#YJj_RClPZ`Ip!8hE z{uM5D1R6)`NNXGntrqat5yd%0*;@P%;0T>(2|(p7 zm2fubBgXHXkDf+gt&w()*Ym)vqLz3;%Pvl@{?x#<*{juXqD|qzDC3gefBYkXQ< zuTm>AI!z`F=i$REL?&j zz>~56^jliQN|mCA5fw0m7!H2u_EaRFp6;W-e5qX&cYA#*;nMJDw%*;GTxSQ8!OfMLRA%oH zYRBp+A(BT+j@@c9faf@fGqy;qJ1TMGdX{c98PTUdu=|7>C~{9-EL+Yic60b?+;!Fr z6+uJV&@P6IuBSk|u>4Hk3kX{AdHxs|@K>_HQ7G#=-7*A>X=x5MtS7#_;{v`c*oXxa zsKj0n8oaT4FCXGX;&0N`xHEVx$<)+dt;?1kNCStp*8EUGfP3~Q#3#tS=5gbVo$qpj zS~cr|tK-)l@jgDpDRoV-l|Iy_8U7b>g>OIe?g))hI-o``6}S44hM&HJ3%^p@^#Hov zU%}7G9vxoQLMoF2tKk{Yu#4n)=7_;9{F|qM)N{KsZykX~7o^=Ta#VqlCD8IT0R*rJ zZ)p_!0d9GA0DY1A8*nDo0joV%l%aaGT;t>7Eh*SF2ja|G6KLzyW2*g(xEYWbpfy@1 zV=CpanraDyjzsH0p1B+*gAO1aJLOj?lL5&+$SY0nG*s?mCVn7dWvX{w8g*~1bN`pX z?0_L*Xci(K(42ITYBD2Jye-qNA~K_IcrmQ-Rpf|G04uDm5iTQguu2)w$|$f>Y^QN( zwwm}~=D}2Bv?Wz3vh7}EHO)%ki3!bP!Ek@t(J-deKRYbFe+P)G;V}RaZlij>HbIt_ zDCvPV&XzhVxwa*5Ac-RIDKySXUI=R`$f(E`_PWqNBqoq2#BYgFL?bGLP3GdRj`TF> zmS{M43DldlEKa|STHxv_UYyCgY$24&Z0fOK3*<@NPm(Wx!2J#E|F7`<|A9zqOoJh( zf%pgThVO9iJM?GP>mxZ95t=Ny>zxlX`Q3&at`R*0l;l6u8{RC?Xz+hgZz}7cT|f{} zKADDq4WD43pXA-s$<@VZ1j$|XpfJf53((NXdJWL;;D1Ln8Z<(8f&VK;(vT}bApX7j z>oGY${w)|$a)%;5fc&HB%1#&?{i)ga zPxJggniP$j)u0XkDP{FTGyJENS`Uf=_K#BRw(7{hCx?t0!QaM*f0U4uS5|mFDPuJ4 zRZ%t31Y|*fHkJHfBw8S;fB+-b5E54-`XQ5^0}*Ib@3yRG=@5XKmD zM5jN*kad1S7**mN1Z)Kd;bhGHnE;@(Xa?l;l0?G0unm(9>ygIiwOZq7vyH@Ty5Y1- z=i`ECsDs3!pw|8@hdkO;Uih5QlFz$IIp=tYr7k5uhv?-c=rp23!#B}Ikz$-t(;!3v@(ZE7o-gEk$LO>Ei#Ba zY5@<8v*EM5idlRivv<2?fQ(zR*d6}N*xp77W762qR&!9KW*vZ}ZLW-%ZQMrUxwriP=1mFzf;rZVAqYfTD`mzS}J|AI+G3Oidx+s^8vtBM3 z56G!97BRWcTu?qNE>B#*J4*Z8Bd)|B^UI)*z?H!*K|_FHGrW3daF3uU5!#3d`}?9E zq9w5>H(a^}Rr91m2VCjt@o$$ z?Os1r8TF%9N$|!B{nQrA1?5SfW=s3z8qcZFFIorS>t1G_OT2IerA9h#&)l{`aM4$$ z&=w{L#6RTaadx0IFwPcq)wM?ogHl2^!>|$R--E|7LUixpJB6&sdB|?iB`g>ICB<%4;vxdsD8_l>kpQzzk8iBM6yT&&D~vKC9`dW&}nBU%_(v%yPk;!FxE7 zZS-8OQe7c_Z4aNj$nATfx)SbgaQx;|Homd*NFI&UQbk`4;^CFtdZrn=&Leg)jW-R} zd|wG`OrEAWYDW{Jy}wtRFBzVNp}nNf&@=-;7)bU~K0cu1+og9y;=Chp{PnVx5OX|B zQaTu@cBA5>@fH_zoGh;P-#dIv6@G@i=ZF{^1DnaxN!VY0;$rNKOQaVBDacwiRC3p9 znEZ85)1rDilis8PLkJF67<^p_8z=R1sv?D-9ndxgRfj5yaEaa$C3Ac`S+am}xO=^S zE_TwW9p8`?b`agt?H1lIQ7Q91hb5#XuYCLE)tx2#ci3^bNMQ$=%JE{5%((1X^PD4e z;{X0!#Rku?p*;Zr@^{z&pAGMSr}O{ryot)R#si<=cl)qkz9@WYl%9rWg7|lj{SC*) z4GO;aw=~8elpWefjTpdrR0MOM$)x=tlhfj@(2*ITDv{wEFDHV}iS|Qx# zL~Tt>C0u@Q5O_{(h9t_P$!fz7XmA@x_sgn_Kqq7myy7$_B>6 z=gf;!M?mto_s5q*Ly(L?PG;AgB493-NYipyBU8DtSg#cxp^2O*2arl2?u2k~8sLAW zAxDJSOXe_q;0~`kG{_vutMVSDmmW&$-+H}7>2hWeVR>QusMc!P{P!=$bX2a;RPus7t+CIphh81Te%Vh zv)v*QyTbd`ErrsO5O#L-c7t+8ypamzY)P$(@&fyyCr}yY%YZ~up+)$E4d5|j0e1Bk zfsTxWIUU?ST*bR&0jdc&VrIQOdTO!CCod!03fh*+GcZV>tR!u@ben|E_C48uqv&`3 zwadKfRL7eE1aVo;)|5tYh1x>qLhV^smjVeCE4&TF=m)Pf!bMV_xaW$;oY)PPdx_0n zY>BjUZw+<>tm*LM#oUFBEWovk4nn%23Rf9KaCYB_Xm-jjr`kHt++px~ztl8LdDP>@ z_WClSO?Hs?kA=vBt_5R=f1TnW3ZA-Ci!?awL|Y||T(bc}Ux_IlEP~-5&mWB1Ywig& z_>dNEuG>|Js1OL>c0xToNGY6AiNx@@CVX02WVXBl2(;!7N zTv^l$6R*(NnGeXQ0q1vNo~KQi?J7C8sQb}~V1Lv1;s|I!Dp4gu)^!!z?| zD}_}rzfRNIBmxq8Cef=q0y1~N7RW-ChxA#yT#$0Lz-Xot2 z++FZ&lg6*8TJ1(!#tz886Hz*KOTHPL9U#ncpJ(4yjG%mQZx-S0Ga{SfR;s=;)f=pB zDV*!?WYw|2?kBz`8dTQS9{hUXe)*zlb5EmV^gEucaT8!d_VktB4XCfeJ2%KhG@P?- zyCi9vwN;Ygn7G#2v6y+=K8qeAw%{bx4Tq;E3iW{N;sRS?$0$nLpkTr);Ytl}l%VVi zx5y-!L||+ckgwp9yxl?L>{zvLSGE7Rf%ue5tqt#BboG-}c*ZB~zzAcU-DceB-KxDhjHEBT>S7)Z4-Jh2>T`r0w;`lB(baCwnGY z2Tq9XZOy5p(Z(d!yC;#7*rMQHyZnzi{RFG^ga6-l`5&80!^%bmg7{aArnPn={Y0z{ z0ZqO-{B(}0_CCET!*8DyuOW;0{2>3D=!jf8zW+qHQ99p$t+h=mk5_l{cQZDgWQsL9 zLBK~GjJHGM7v=7=0Bi4yBQLLsZxj6On@AnL5KNJh&VA0R4EZ!_YsWx;IXu&$QY>MG z?k1TA7>IaSR|qfQIeo7}dL`*&gV9g|-CuNY-WF|6yx>*u+-5=MjI{A~k=bua#1zL# z*%X1V%5v>T1?div=`EWjXMfOjXx1L+0gKkcLrdJF)qSSG!&HTURdTL|0c3h)G*QE0 zmq(W>?fs2X<$7sWB^UW#16Dp+zs3Y*ZRCIkDJz3+4&UtAL)T#NSyeT|&QBn2VkuoG zPFxO5a6uOrM@&!@Prd>?AN&#c-j8dGKfj=ReQV9wbRbIl?c+JkGz`1ZM4z<=076~>7H%+9srquIcxSkY-Rb&#^lVKE`5%d8qTCQ@dcl4IuL&IBiRhcS`Asg9m^ArPs z1PCx|(@vNVXlsj?Uy#_xOro}rj|#1%wn4|0iz6+hq-LLcRVr_6#bDGEb)2{|GMooxp?$-RW0-_a1ds1#@}88J3L+mKoO4@PJO??RVs6m!qu_frCp3*u`6O9A2r zT$Pxib{1w{p^P^%mYyhcDSQj5Z4w5Q_kE@?a_68xD(Xs#gJI8#L|33YHrC`phw#&n zTEO9N=~SIrF~LBRz3r(F9y*7>7L!G)QG!Q^zYE}>8TuJ%u=#%&UH_psMj{6b z1N(P!b0QM(ga0d*JFD`U zl_Ei#XED4>ATcVnwd$PlWjS29%uTV)anaX|s8UG*4nzAj1gHPn$d3ti>E9jYh(tXN)_yKIBBns}VnIh_QAh=goKA{m*r^_bO-Mnw; zg%m|=cC0UYFI)E0?loC9~{rq978CYBBdzf^o1Km@#B$o+3av-w|~jYjpu9 zJ=&CjT7oZ!LP|e(jr3To0THzlohZf=JbEg9z)jvQ=HE&FKU3s?Tlv4Um0l~X9rbet zlA$FV8zB-W6K$YFB%i^+AOUP$_0-Yd9V}QpSemV8bEQ{P>c9$PQl(iC3JVJ%q(yS6 zrFD3%ZDs4*Q`g8E2r`z7>ju&$b~+(ER@70mL789HX8_Z4-j;8|~fll)Tl)sFkN??Uk8HX@-wG-JtQ z9@33dI;LUVlLGd6T;_L@rp;i+U5DKB*lmO5c17Te`O^=0q>;-QF2GW_17inCR0mBj z2q8gtK8x>o_+T5?pMJoux+kdO$N}**7hNxkHUauEHFnf6%V959)aYg1W!BYBpwN2@ z=BCnBdDtP^*hY=JYMqbh=0qg^+nX)t9ktd}t?p*NWK9xwbU~!CsiRrGgSG@ub=u6y z^9$#>-Db0^WgS4(V{;$&)$6D!3oA;jqAnGyNK0OmCARuDupS^DZ5b(Df1j7uXe1?E zH7FA2#j! zdP63}e&Z>BzZz2`w^rpBy>VfRE=P+>Psyt0ekY#_GXp@%UPu=9aPY9HUgrF6pT6d? zdvBKmsYL|_J2`j#P6qu;8&9NWe_TE_zx}OhW#N!9GPalB90?e9j8h;28LAgHu zZaNiThn~_Smhxw&v;`1sD%?Y!?tCxmXU?zE67Gzt3cECe{EIp2ZP#VVco(gkN+fsY z+{e6OJ05_Ao`p=%e9~z{s67N6rT(q??p35TYxio0yW^OYcEXGkdB?EK^d&tOPDdmP zgr$6`>)`G{6~ToRlBHy1pt_o>Q_))Ek5m|}d}>>|sSFtlmIEPQIwB9QnmT)RrCxV2 z^o$nnm+dO-u?KoLL7$PeG&k7(Khc^=y$INP>EjhKT22n2$Wa+ z8XdrkQR@SF#;u8RY(ln~HY^MO%q~25E=@+J1i9$rg&+_F#vLdexzHh`D^i5@KPp#o zKtfZIUT?kJTuC726)LvD!c;f|!uO>dW6}-EN0ahBzPcHSqU~a=nM>BTyq!ZFzn_}y zuU`SbS<)b=s!=e6Bd}ppLc8V&w33Zz=nNXV<}9>&7>`0|{0g@hA0vT_m%1QUL(E*K z7~m)7e`1M)V9+~fFMVExdkL>7xAo*N$9eO%jS9R5jkucf$Fta8v47t%$S`Qx%!a3~ z=jpL4#?r9`d9mJ>TXU98#hyhab1a{!vDN}K@=9}7WE>OTb-IDE#zsa2AG9Fg_ zxMT6f4j9E2nW5J&KXEm{9ra3IyYGwatLlH*j*UCXA>>4FLCaufmvU#mu4iXmpGE-q zK32H0Fx2SJR1L55m|UC7A2C}P3tyG4=Azw27~4~fgV{%&pN6=*p?ZoEX!y@mw@C@B zb$|d7%2+?N811sOnW&JS+OT=pM7`zHs$}BJjh-T&1S4^K?v@+y*ZL-puMjVT_vI{sEQk-P%LM>#LooqBd z7ST0_1eWJ+k9NxUV+{RXh^199$~vcq#{^B*=vwGJp=qc}4NtTK+ujTb0WYwKi7$aC z%Lkf++us>n@znW^b*CEX_Xs526d#cj^_Rx3@NghCX0~X?dT=tj^Vu)0EKvbkF)=zY zr=b4JBEO#7RH00e>z77)sXPTW1)418GyM(wIc8SFl{}Osx7YqGhLl?n&XK0L4GFe3 zmGBqH<^7T*AC&Jy6Gz~_yImD}+0h?eb#4n+gv^y_@{PRSNI=+YZoUz-!u!^v>W^yS zAu?6Aw_m82chGUIjG~_&9+P>^Me9I*|eh2Ibw$MV_d( zz}%~0=53Snl~lCQ>s;ySPYVn~_xPGNidZpgi%l^153)s+f*hbcx}*yy3@zibj+L6K ze@&d>3~~pbU2oVO8rtk44k^+LJNX&Zh2*R-c=i_+?9tlEgl(9s`pbl+T~x=9qOK`7 zb25`BA5J8US`5e71{DJ)L>(@L1AE=%;KK#m2y5Y7WrbVR3mosMpf>!PuN;CGDcXv9 zXiO~m=$0K4*f%HQ1|+z^H-1*pD36T<#-rzQ#Ff_B znlt;V&=M4DJX$8p27N7g=A^N;c@Gfal)r`>_>~TA>8wn>xT92U!pCv*6xXVMw576_ zUmS6r7ciTkn4}Ikr!K=wseD!W^vPhk zGk=75yw9I;V|wNDZ8xb6`gpi;pF$Ykf5=zxZCrXU9Bu_5eIfv9(Y*xS*T3TqcGgaR z1aNJbEq|cOUNR=VrgKCIZD>|2Y1aHf6=MmnpqDlX(hr+=QqL5DqrLhXiX|9~U)9KK zqzf*!1nJT(9{HlQ?5U&_SL^HK((ZqP9xw}%Wy176>i3HAN_Qnu2ny04REwqv)0ITh zg#nt-sI&`ESm}@kRT78T6f@@-hS&8ZSaT>H5r?bdNXiHl)U!jSZ6x?XF-p!|zQVIi z_38qHPN-Ihb0_neIp5g}z@zeqCK^2yqUMXb+gL$_+{YQbV37&;RhME(?hTqfsrgj^ zUW%7*(7|OI_0xct!T0H{EZTHjc`T6;_CwquIowbV{SZ&QqV^!WJ1Qo%E$%!rLKD?V6TsC3-tKs8v8 z|0|HO+4lGRiOBg?4ef&M+9n1-5`mbOM04nrNo=8=zne+TwZL@{;3xcWp62#B z;sgF!HUw$u0XTe-grl<{vVdqFQQ_sHJ^{G@gvLZ!iF5NI>}I7>SeU+Qse@yjr=b8c z`kl<5DnT!WL{n&}osjPHS`^?4hqQ`3aT$I`U1g z=z^>o5v89VRW>l~_g!@yjyP*AT;W3SE7 z9<>jiH2PM0NiV*q0jdbNk_$J7?4?mX>9zx%tj=RRgfC~I`?x@ydcXB>Q*}~V*MFUj zd?rB-TcD}X7hj%~T zcVfs~fbsJz=#On4ZHQ~Pz9EG|9iz=5cw*@pvPdVC>Py`wmB0(Qb?d+L#ib25JPP!Z<8z^%81uR#BZ@I3GL#L z(=3vfRS`OCHk52D6C&%l>ApmauZP+uEe)5&aMpZlI{(Q<=;HtX^YZ^;Y5MC9!4I*3B_rSyXalh5a8ZSfmMOH5gdkT#1CW{Hh=*eB zzfvP-V1~2hxb;=GwYoSVKA~_QZizDd+7T)>QAXD!e|Rlw^VXmNp->Y+T%Da=oqfAo zg27d>B<=+tG$g9O-X|3NqY{ilJ{ZskB;b*VY+oXh`x0)MCE z*rJf)R4p|wK~;URqBu#KZ^f5n@tkU59`lo^8B0*IUsot@n+;!2MeHk4d%_8CG03l%Z z!qU5fINAbRq-e>_TyNz7Yp#F2rlb{U&I2nz)**MLKG?^EKRT$GUv8!iF}h|@bB908 zEuu7^H(xbsG8{9+kh$f09QrP|h}>NO1Zm4PMi#{ZG8g2O7=y;c%HRm}lmgj($B;i7 zqxPX&%hD_Jx9Qs|jPcd3`?Q5v;Q>-gztS8Wio+`HQ&yLO%4I`E@Qleaw9}=WP|Cfb z3$8xtY{@+m=9P*Cj|o!I4qtywBmgHbo9Qm#AaLPpKL8t8&xp$PYtR!{M{$vQ)^4kL zV3>$Vc+?Kpa;VZ~87x5fb={=)lH{_Ar_c_+vglmt!6V{74ekEd7UHKYVt^?5*-&hx zV*(;zs@D1Gq7{pVXM3nCw&Ia(3&O$eCh?+8Di*_l0#BUehakBgzsv)`nmg}S=OMrW zmaFGlVoH@TDcN5myA87r@=v%?V4H-rE+ubqJ*V@RB&m)-MieiY_k5478jm%kOX0^S z($|$Kt1`HfqGU(Cn?rm`ipPDT*}ciRecI zp+xC7J3;Q9O)C-Yc*K{z34Ulz6p}j@P(Ij5L*W#|{x6+CXu@79!a=Uib77e}6;~p1 zXMAQ*WFjnQq_{m2VVq*X572uk32c6Y>t9x+?fb+~{xAwLh+az85CFxmAux=o3)eQ3 zrYRy}d}e`!RBd|{tiA+*?>F`oGmAH%ARh(=iiL?6xpo@uR4p#H#Q+va&Rg?3zesiZ*@ zZuR&DXXKCtFnVJ#)K9oGiM2Z$nYHgebq3fHWV&Kr#(0;#{{vO zH@J>XR2B{Q8C>W5+$i-wE0?St86C`Blkvw9;F5)|U^p8y77%&0ywuF%Cw$FQzB>%JlCrCCzgP}-fV}e=#Cvz%i`u=_MGi!pb2!Qic zJ@oPeoJdJx3_=Fl)7eE3AqPh6CMXpzg`%!@F);&8LM)|{@MH%Ygp&BQ8Di|uR?uWd zSLkfx7_8$gZg&i75NC*}Vkp|_tm6$9bXV-p8r=2ME3MPq+wEW?U;Ga7Mr&f*>-hLG zZcaWrKU&*1?;ejdC;VW$c;=CX0G#i_gD`^6U{u}1Uyd)OzN}t`st6FwJIhEi{7}ib z-;HP!q>irj%ZPt?`tr&}oe(|TuQ+_{-24_u7y$Cw+y%2$`{W%83@LGj-=vNfu%T?kRjh%uh0IU&v2Tk;Eh^dg@0H{j@UsZz1 zk&W7#2YUQXjHnO2k-^T8E{@gQ?Ux<|FFXiX>gC7)`C%{~}F{Ce~iP ziDjmK(l=l^g-tael7Cr_gnen}v|xyt!^}dW!p=1K#$mbTKmwwI6u%Mez=K>x<6+6E zGjZAcp0XMBRAz8&FVcd7xN}{eo;LwY-6~VXX-KX=evV0kqk140VCP=o>`9j@ACZ!3 zXR7;Q#bG*7vs%9GdN74S;J{W(Q8A@*P({B>;uacER@@V}>_!+du=}G#jg6 zUV+9qFJU3BczVa}16wFvJ!!165YKM<(wItGnfg;6%5)**i=1hhzS3|#D?3Y4VO|oq z9cy}W%8F76olNl-0DUk{d3eLUbbOpEk}pQu>{r#-aDludL`549}^Wn7XykRt}UY_vFAX*j{H3xta8%&Nuf zW{=}7PDWcg$b8n9!YBX!6KQ6Pi52?>Inr%+{a#{!uKxZN;2s`j$eQl<3S5?1?5`+D zEab=vvH!^F3&GGPtxCEMMiqudk7e)QISv~--B!Ex-;`l4wNC=6am<#R#JSGq7k(o< zi@{4&I?V{5Nv=}~SSB_=iD@kdNn>?nOAT>8!}Byn%oj4LW8Eio&Q(2#G@O}3)yygObr zXrcztCsz7~5#~QrRb2(Nzu2_w@8j>}#I>g)mPTt!N$qKnMV=l$9fMC{E)&uiEo-P< zI=lO3%bFxlhU=koi`hGUEEOFqp0TJQW&km*{_Q3UKoF-pGu(uaK5B_{$tRyf$)6Oe ztpGoV#(1k4Vh6^*RH(#RDEXkp(oC9(O!bN8QS!dn^|4J3#<=VMb8`nHNc_pagYo=R@PT{$H zTzRzP03pGReiu8V&TE{JV%MO{4^0Dvea%9-9_qrr;Q%#Bh&etRUbr|QiyAgu#!p^# z?saV^!PGj)aWh>DC%cr)oUZFp@N*}QOvDFD$S1@H-EFg#2;D0u5_AH0xV{~2oBFM9 zT}n6eprZ6@!b$4PuCci^PJK#=r8ZwLkIZ|N0O9KOxguilj(y6a^tthBk)`wVDx&k- z7GREp%n;CZgYazxya@GjC`bEE!HN1SqDwp}SmRN^@q)Te(FWZ)*!zWL*vMQ4RBGCJ z8c}@)o=OcA?Fwc!wfS0Nx8mCE2==&Y3Hs1Gr;pq!Yz$?{s9iAMlKL9&&K~w;mNF_f zz?SllwV%0vK0NK&gk5)0mv9vuhgtvq!xFu2_|naK zTN8WNG{d>w^H?kjZU0k#F4ksv!&;wBtVV-dr73`efBE|zd}|%Ij4_Zu2_af)wrJ=$ zHMEMCJx<{hoe6h&mEjr6j3$lE8k>_BaA=nUUSH16=!&zp1`SV~iQ1Dd!n#G$` zPP5P*zD7qo7uaHmxm^1&A~`e!g;pSRx<$(J>dHi6(SD`DRCAO5td-Moqb^>VX4}pjz@zGX zlEW&}EUYAXaQ2N^hgH*ChQ%d|U_7ls^T2VfuG!vWu~L_(q3-N-xo%V;zudYwI6|N0 z+a$iTIWtXKeM<$H!ZQcj+3(szX>MYdscIiKR*uf@7W~w4a|2#REJQDxmsYcJJ5&61 z!!=TP>I)$r)Y_APH+1Dsf+*xEK+;Wkefs7a`RgEo&2O#Cr%Hc{%q)9%PQaNz=C$*A z^ulj_Kb+$Nc`wrJYfG>k5S~gQMor}nQv7$U@7K(e^8}~lnA+=s>Gn3AuGXGSKD2V@ zA%TZOh7M74f zH&Ty~B|{&4ydm@6Q9q1ZaZ5lU#W4Ta=V2~t=c>F)1ov(zwuNH>O8N2|v9;cRoo%V5 zI%HCw&6?GC)t`t*IkG|vh+74!uQiU`8mFeo+NA?ChT-3wnTu|z1ZWS(JEsUiLT=fG zPe5W;1E0}=3gau-#p*-ehKb}>p>Z)4)H5`m(Hj%{moCaMaKyTdbuJ^m7ES zVu<#cY?`_n?J0&X2Y~N#ag0nVlDaZtj2^=TrnM9QmOx0ljoQ67I>V-`_h^gGt(+uf z98zJ37Y^a>u=FQSb`4iF=N1fJ*K9$??ZO$j3n0PLKxMI|?8_TQEo$bE zgYQof=>#=6q?DBWBU`OM3@W^jWVKMJ(ql=X^e*u7b5yeUVj7uF7ee;!w_(e39Glu} z7@LmS7yxSwIEPg&zu!OWK6OHtYl@a}1-v&lPvM>>h*rvd(q0qadsSrp6W?Pv#T4f^ zSM~_AJq7m0jR6&U)-mI>-j)Z#RT@69goHoie>VlYzqXo+=W>-MInU^`mSbwIF- z*`P;N$#o!{U^mm_HLPADTJ$p?j2bdCZb?IB=MJtE?+{f_e5r0kU99wh9Ih@WMG=Gs{75=GZFv3DL_UmhKToc)&=cd{d8 zRn*rLJ=V8r$;%bIG=;sF+{uj(&!6n^?Mml_-!3Rmqu}PaBp&@WvyJ3V%q+?j+YLEY zPZX6-BvVznZT!+!zSjtyr1`?nF##Ri>lc*1PBjDkC~KEcR-w(y_Rxu0-QMkfPHePI zcrYPnBEFH#qli1W=(ScI7$h{8Y!31qat9&Y_7{*ot{`_dG+Y)G)i)e)wbfTt26NDj z=QeA{r!^TLRgl42t4eTIlYPmyr1J%l)v`Ih6vrrIoDO7UE${5fdR1rJwLR@Ow3GtG z)9(YcZC>eDfp<|^UP=xoBxeQmcb*8d)|6+~7PX$7ngkoPz4~-gX$Ywx6*B-3VmA;! zcKW^xi)(J8N;hG;X@abuTOP2;bi<^N1Vv@4oeFX36YaH6&+{aAqoF_NU7y(>ax1Gc z^+q9k&NLc*tdLr_415q5rL|i-($6|7w1EK7!vcLDXxdD3$=csDx;`0+-9e5lr zGFaK3#(2rx;OTn)u!yjT{T4SRF?@tGeB7_GYMu3aok40y!ixjBUIbanYaNo|9g{y^ zLEdgfG2RKQ$3L)eFHxWOybwVI$!i#^O5P$+H^b?~1ubQtAmFAMwLSg%6W-ZHZ#V&- zga#HO23?X@x)u|(Pdgk^l`1*D1$q%a&e`kflLv3!7#H<$2DsTCD}L)E9BzWPVZkmB z%Mgl1OWpp}IUql;L(*bG=c(;w1_S8qecRDJVbIAj3;wiUkgKU13i5*!^lUc&s@fnh z;C>CO_0QZ%Kg!MFgUCg8LTA`>!@B@@^d8AHCOqcR1Xr-esXW0wK3)^MWw4Jmub*JV z<-{idb@?6sL@E@jG`kw9LuQKG{nSxX;`H~T3UKWTE-GF3hpDWfcR>!t&r#iOHj&0kv?R1QejW5r0&iLN*tM;f-yQ=mct5%IQ?|aVc z+c)r(S!hfZ!1xAL|GS?7LOF2+h;bZp9I`)n!f(lMq}1@E$dD8XsWLw+@KjHA2O$ua zqRMKiF_2M_msM*u-O8I>N>$FPD%UVYlRq+mN-nOFT_mpfmBIMhzk)9&`rrQ|%5pY&0{(Po}PVe5%c?l9S zjGak`0Z+_bBr$3TZwbG>CVQ53o`T_qah}WsZ?HH%Ft!ur?ON@_{p{MW2(UirK(+_F z+n=wqy#YN_-&kn4Ya&Qrst#}ConiX70R*QSSxkVMQ^0BWh#sFNwVUy3TqM;_@T2(d z;~=>7Dd}#D1x}yWZ)C~NsogT7{bK5OnA`3@}<1+eSYfv&AZbV0OJmg zbj#=$!RQytighx_t^LaiLrS*VKkw1eu`ERgp;>1-+%1m8=icDu3|7rhD zX$mhdb9FWE%V-_CM?qUjcLSL5EXaum#L!PH1XM_Mb~uX~-eBmq+&QOq^VEpB6=?Go zAVX|M;+a{>Gam;!3g!E7%MYEF3@M`nHsLN)SOg7Psj+0F;W71So6LUOlxqjbsFV{l zcQ%5fqA=FcoKf4Bs3K^}VLVp0I7bUj66+!hTldtPzYhMqMR?A6c$c|2wqwOsqE|B9`<4lf3Q`+-EViax!##sdg|8yyi|( zP=iD$=VFc&2d`K$d7lW=@6rbKuB^*E7St0N??F*%REr;$Wo%!(Rdv9+2vMl4}FHwjuHa1ZDFAx;&|1!B8% zzaLv!-*J!>&|6+DF29zWaX*q!uz>L!^2dGqPjEKDCEWB`DDNwo%Uz?yULd6--% zJPSma-%$JrqmYzyxOcY3GJ4rVm^&XPv7lZFMKSM2fX>Fg-EU;X>yMZH($)vJ1S zJs7bbqkZ&M_Hm5L{YJjR(p%e54V~+{w%B0mBeW$?4)XMYxr;)4mwa(7#2dtYH4*?{gi z@^jc4W^Zd7XfW=R`jL;mneq1LyLshoV-|sWi)Tjxv>ls}CK-f-32nTGTQ!sdJ~d2I znfFHLrdG#4n5=vNjz8-#z*~X)8Sp{388^la0Y6Y1}np<3U<46NnL>{yK7DX@VBxT%hH&V zKr+mIpEhQ!Kj*gf+YYzCY)eGVigVG>uAzf?9@cQ0K^(yjAbS2A%MDXwo6`!}oax`D zE;|NFxzNZNP%3<888=2=5&zvV_|3}u0y7PH-VMT-fD%U>r^;Qly_G3A$GXm3=t#AC zY`2EA{Bf`ajy(+$@QH<~RA~)rj$kxphww+0ZISys7Zt%=HLT7$exXd2j%!v#B-=Df zR=In=vRraAvN$UuM2v}~ORprXGFFn}WJ@@h;3Gl{WX-w9i6$A#K_tIlyH?fBl@U=a z>5`PzCcU#g7%$>ml?iQhfsrvGy4Cuq1#DwA+*By)@akdZXYxM#fmbr z5VmvxslAXu)aQBTa|iKC!M^)*p2JNXway_2{)q(C#1*diz3K?!*c&0#6@yE=ypS z*c{*5Lqk>x%Ptzad57j}rdmu9!lw$dj1&uj5>74F6(`Teva3~SKFG;e=j86LUcCc2 zxcjZTA_Y!`Gd*l`c>9Sdq@dG2ex7`ps9Dul+$4AB;`FziC-NVd(ZNB13L-Uo+@#-; z#oidxv$;xLl}qo75~!b7~ot`RqItOS*nkP}uJR zf6HQ$WfHu|f?QFiw3_Dh$XWjq1Q85qT#l8)u8~WtEdSFnnZ-86tC*~n_8y#!OQ=x~ zlhi$?ueP!*xtUgd9?f|gy{{F5u3xJE#xdT#G0c!91&${B(kh6R(LmV6c!G7s7o%Us z8=RgB{mFWanm}JT&UtWVpw7Ij^c2|%2)lL71y1k61xgB(6)^1@Jq_fH%eMJfv7S)B z6iTYivq8fFnrU;J9_UbGpnt4q8mO1dSN*zQ=D zY~mbous$i<81!}T(4Zmq2*`3$#e9zJZ!-=9 zk{mN)0k*6s6xQq|s|B*oEPXSs!n6nOeBE*;e2R0jB4mip6z}20{luKsB#crECzw@e zGHn$Bu&zWY2Z|lc0n~1Ndz`P$UVLIg^=H>33F#*p$S0 zAAj@b^ZzDcf;OLT;ZOB~Ry2SBT`!)|s$=Dc3o1eHyVh3HxX#6rZb}hh`w#~ZBYhnq zzD9GM&Uzv?1Za-4$M5G@>tPJAkne1nKiC&oC7ZB}XiDor_Y&M0Cn+uE>xBwy*jEGp z?fil6w|3gg1#u}o+R|>Lyk$-GwH*4&IUZB#`gs zp_kWR&^+T3?!U|7!E>E=1(1^))>9?-N@LF*3idv1}V`Py+hMk zM}^@I!%o^*hNWBVAxz-vhMviEs1tJR-7}v#3QM0NujLJf12=r@zTN*73*jVxA9Zu#qdEmcLZzdDOSK&F*t`y>^(jLA>DCkS6lFNGRn*45^r|&a~!dyGbr+3wP@-o|kW9uoB zik6zyE(R*vB904=a8-E}mycR`<*Y?q6gJDyU^0+cH0a{kkgzco!xoEOm9a0dz9);F zjrX}DihWoay9Q!5QJ7xax2#?^<$ND&$-RXj5|xvZbU%y_r?cXedw(4GRwCb<=H9xe zFx>wYs$IuhcKhrQrf0eLlN4^-E6{E#O|KW z9szJYc~M%$t&Q03VX+|lAoqd+V|QwVykV(T<@ zs+y*vx@N7hs^asT$AKt@=&ckAB!-(ag||R z^2Av6h#8bZx?bNF50{XMgTaX;h7&U6@Gad1_GU%6|Hx#|PUvIkrx6khXnUR`e?D_w zt&P2BZ=5~np>iU+ydh5s3fT|GBHUr|6W*ycetK17g6NN!z&te=sP(+(XA=x*sc=A! zE^`p~50{q-QILD5G z|DvQyXuBJ{T@6;_`{w*T>cgevPgf0$US0eWX8%ebPd^cR4CtOcgas_G?o?N5O(tUD zgZ!F?*&k7%O<{xR5({2!*12SAI=%p0YXgK(PQ%;TjH`=e(yft9${f&{2eFM zay*R}jlk5{!AiuS84~G*YcrwT%H|1=NY|gJR#M;Au^ltoujYP9ZO*jOviZJ+3MVCx zc}k*|8AQGOqYP$|?TIc9;Zzr)cyN4+2gULr1Pn*V>^9;9xg4rn?8h|+`->&;wnSc~ zh~aR4$%X(@wX{@5geY_V5bL=Xcc~T$DPTs|cY+{}^5E%N@;2`~#H0&>PBU&s#bqcT z_iN08z}s1>DJJ%MKyrm_RKWK1I-&Bt+?UjN;iWHgg5$2IesJM|d@0YuVeBIwnzz|U zrF>%`m2hy#BI`|3?UGyUl1vf+4kp>d_-}L%kzPTG0hd;n81|Na0u3xD3Fu`}hC^-Y zl@dHuwWR7THH=tB-kjS#h}@U9ww$&sU8AgI5#KGgA3oPHb)&uFj0ZuOI_x2yFDrvA zU0HWHh{I(p$QEo5eBH->5PConi7k$e$ zca_PnMa&7noyJn>o;a&SuXjfd$KF!fcW5`?>n-u2#Ef=nBIYb7|Z>v;tKzZ|rouQeQxa}_2lP<-KG zA5!=wLoduEeZCvZ9tGV(sfMJyQ7ESy6lTVz2q&1+{juONRJ;Jr(m7SOF}bfYU$4Yz zx&8vTflMRgo^sK`sNRxEgCC9XHVh1o2H_*Ugg5>~R&3(#BH!p~<+9vyR@})+zAi6M z?myS#lt#Q?$eP5BIPniZKc%LBvtRKK_2H_oLI$|@?ZDem!p9BIL=y8DsK2H(z*iel z!cVV&^r!dgZ;%3klCocG!N5i}!R2H@(6_Ll7u-nyWdq(05SE>3#*_4OoH`6yMlk#n*YhC!h3c@i1i( zgjiUK#~1+LhxNI&V!wGruu-Sej(cmuv*gbutU_wUn2?7+Q$A(#TkHz14>PR6{3#3$uJZP90wx59o3Sg&#U$ z#Fr0b;skSKBnr2uNHhe=^smpe%?T}2C6W|*WB`y#XCzm1M?@XqN)nU6y4 zV1v~^^!H%5*Sk+2Y784j)AdBX$$l(Lnl9-w)@kH|M;VQ)x^4Jr8;@QnMZSNXiZ4dpYXA@?k*mMMLQI`!_Y$g?^Htk|qwY&Hyxa&u>slC@1udnI% zZd%-#jDxlRz(wpfEQ?cAiG6*C+yk;T6+}B=pbGl8t%E+VwCEKbJD6@)q|gG7O2E~`$y`EgT;LcO%MI&-$V}3deR(`U+U+c zt4#?_BP0|)obq0Hlq!gCetp}>N?H*V4&q#IiVnevc)IGS5U(fdjC$P9*Ui$}nscJ& z6SZM{WU{Fv9gMp_HpL!*Ox#P2o%{qkJj3@}R5_mwp_=eNayDC_U}?R-dG4wvpSWU& zA=P1RBhK5KvFgGH8@QuRDXJ!8B`NAEi+Rwdk&x%}W`;W@Mcy!C$u~`AJvKgmQ|kUh zjykTyl&sF%x^3ctb3TJuJBcp7XBrlsah}G^S7;s3Za~;e;_#ZW(?3C}@5=(Pn(!mu zN;9vJX7yEBLEYH~)cx|Hi|M*?ZIWeLFW17;?l4bZHs~+k09n0p3?Dd$^*QnM(D=RM z^y+<%qxHR1xnvZR)W+{SsqPBAuULlyvF-xOb?GO|z&d(_J7zV`GQ88nPm@B3=Z5$S zzM52);CVi?hQ2gj&S1Q0RNn@mZ%*|JCYcrwru~-bf}amF^k6%M*xEmkRvx1|!e7VG zkAP0UJWpSAsW6UfJ+}?xE&Jfjg=ay+E$9@opgq2QmEFk9Q&p6>fdNqr(~KrrE0-M- zB_=2q;&Ab?@9u%kC5|l{?Z9BS{HS+}vY}67uO5#Z)!G4r2I10lNMaihnxb!V=q$8) z7b?{Csr+RZ*&t<9m_*<=?x`^;JEc4I_bUlnKTU)tFoRg|Pet?2*=p7RuDI4J(^~kT zrNJD<8IA@DM$?Yx2e&s5>t+yf5Wdyk_hDu^qM#Pmh#R&WA{^KL)d9cuc!XJI{@sn~ zNVp)M9a!lj3~4$N8hjccSre1a2xSICdR^ZJ3r_LSUap-hbKVy5h{`FHws}JE2MhDN z*~R?yLDih8h|H0wF$T`D|GD?&AvdYBZVil>M;3bnoKb9fwb=t(bzeD7-%g)#_&p=JBQ=%nwhN4P>=yrX?)mQXFQWZvP_vws4)l8{+=5 zSQO{R&6vv6sj(*^>j&;8ohXEZ;^6u)0S1i#jQg`!MkjCd2NZga}*bR!&{z*!CW6V$*?#vg4}3H!AaD^rxN zKVHLbxi9|7eGKmbbUdd3*=rg!-p-4}gK)9KA?y>UnSlIHqBlM>9(f^d{y~pB-LIO# zPZ0j`+RK>CRbD8w-P;-vJ^!<7h!zmNK&@+&7AUtg{ zR@1DyXLjAO3U_o@QJ8Zqg@wql)hMgVnRu%vFg6c453@#quBU>{@iG0(fWY8XkI-Fd%=J{ zvsIk5Mv1=ARiJsXjXu|v&watTc*dPoNck&ZD+Bi)-Et0U-qMG&ayBNn(GxQ(Uq0v0 ziPAA|=kKXy>0{8&Ncsh=Ic}dHIt6O;o_rE1$5zL{uXoTkp-*y~8IGC87qK=uyCS@M zy;<28f`y80+2-TN*^8I@1`QzVasg?MFWTr>cNpDPK8G%P{sd$gcI~hwxrv9}$qT8G zTQ`?|*sNjvn(q;5wqFj3Zx6rs!rZgZ1ND81E5`Rl3F(5j-0>wdAD5|s z?K1ogVjL4Py@PD!RMiW0B(N9!;E10}Kb}ecAKWdgc-PBHku@ z1h@MfpbTh>4|!8R_$CAfPWasrwBP7tc)wBjBEiYO5vqP|5PKRk2amWBg&|`h^audD zhNI5neH_)Ta-wNg#{)sp_($7cDDLVknRp3c=vp2_p%bJ>-dO@P%Ya1)87YgcR0+PK zDcO!>1C4F%?Z5H!Hy#TP?P!I}(v1rY@(tZln;$82#ac#cbYs8Fz~k#yi@{k6i=$zR z!m(_F^bWZ@r!MSWXtd{H@z{G6JlcUJPb+ ztNKy=!&A%JDoo+J4yv8yu<^<4JS|aSp|Z<|+lV9-otPRl+nv&0WgjeS>mWR=!qfMi zqBA>?=Acc=xs}9DEK?0yy+*2a!vwC{)rr__!`nk`Bc8W0tA;*0Mc1lCH3Q%XBaXzV z)dgvTB3#xoV7=MV-&MRyX)F8p1=dx1ii zR~)Yvrbkqz*A5}X+}Qy8)8x{z;=~za7{Zwk<1v^-5zzDV1vz+LoH2mkkNoaOMZt7V zG=B)A4~W%?bW7j|gxdV_j12JUr#BDm@~=T5%{9-6ZTA{A5N#EKU4ak;`Zz0=6wq3M`C97nrF;#|w5fF{762EP(Lj{$BkKM=dO2 zRII8-%Y9PsZ${TLD&E})`^pxVI~V_&Hc+4@Kw7|4>hNVaF@0Jhy?!v_md^}c{^d!H zSv*c5Ne#DFySD1UOEE$a?n1H~#JIxXX@hbgg={HiDw|jDpmd{GTkj3p}n*i z8>J2Ig3^cd3VUZ`v`eQJDu#qdA2lGBSkcB$)brQA-)RW*z;F(l2-NVBpmly9A@3tP=dD z@t1nf`{3FS`(RZ_Gs}bULN0hxqL*DE)s~1!euVy&0i(`baJBMdXNut@UhRB23RO{g zM%CTb2bdvhmnlGYHbD$-NYnK?Z{#`$FT?;FI}R*E@YDk_Z({)8EfvdrRvuNt7c-wQ zUJqscv^CX6L;|}Xb#oS$=N9rF$MrSqRM{KqQ8U-2+*x{n@88JU+I6fL+;3?un9c?j z+E`@x`BVKwRU$qilbb8=CJU}$9HaVU|6F&FsoDTStzG~GJ5**isAX#DISv2!-|tqW z`!-(blS0tuUZB(,WJbWMHqq6%7Y#u~!yP#3`=&{Eu)Q8iMJ)`rVJSTGU^YZ#28?eNjd^kzz}sE6j=fv6Xjsl_^QaE)gx z+P1%ITuCeDZT>(O=R^Nk=c}$|C(5}^p%5IR3~_cXDz1mw_zMu+~91&V!4|Fe$tqG+Q+PKIs+J@@3{N zSk3~rrtU7HM~qynI4iRh9iCYE$3hwd713cje+b%B;hG|?7Y|1>cXc38SsD{mOowa7 zbTOa%N=?!)25Np)hfLb5EwE~>_6E2zWE}cfr)zyzT!$pI;enqeG``&cNvjxy(WJ@P zr!QH4=+V3|VDA^-fxS>{(e*-Qx03JISyKXL@-BjG*6M47UrBi@_@Ly?r~5P3#;>v; zV7&`EVf8Elf1kB8`=4H5ua>|d>xC?SWoOW^<-Z-yi?1Kh8RM}ArC#L|8qp|kU;6MP zyV@su*-CGhU~L)SySw+b0ocd1A+lZWQaJ>;c|M41#13w_?OJ1*t)iai&vhewPbA9nAxJwPQ9^<6kWDB`5co zh;8gqOtuD%XH~gpDYAWE3h6?3YtwjF0kr*7r9q}+pPm`M*q&$ex@^$?`-PGW`C#;o z9+-$&c4nzHi4M%_`ppi=ROEi{(ND^{&iAM{MXDDBCDwv84l=xlnTx9mWZ+@AL$FyL zuWIu>oMvU*R_F7T z&YUaI9huw^x+A$?o@#`lVcdiqN^nNJPFu{cG$weikAOYu3y5V!cu_?I-)9-6AW1 z!+ut1U+gQzT8ur`z6A?5pBipSey?E-srs+%ctuR2$%|G~i}HuIe#O^Cb7NOtzZDps zf{cQk;lB$0%QCu0du+}~ghk`n5#!M*siSKYcqNIuj_!n+zFjn^(z+RtPV|Pe_JLnP$E!d-`X)t>zLJq= z?{uyQ0_`zveIlZ(q;}(ZRBzO>dT5hbsb(f-s&)hh!Bn1=m8N9xYqEXp_WO2DvG-)B zLEByW?g&98PvV0miu7na3BO3U^sb`~?&GgVC4%k|RVn0FEZ`?l4ibX<6aoLH0!bM7 z+AR3D8GbBHkPU73=&CL=;T3lz=$qAgTR}I5ozlg{6-Vr!mX;lY-pzH0$DWKU`t=c; zjSqs24yLWZM^1^(=?f^9jw|L$f21s!E7-+f1a|Xo2Hcx${jdM^`~LsEUn>8ne+>HD zfBZ+yUyuvA|8*t3S`k|O??<~~avBJf|M)9!B8Rn~zsyeDU(QRa6vd^l#F?BbOz>t5 z2231Kl>eN9X7_W@^2-Xf`<2He|DPkI0VaAQYXc{zMipth`Jc!iO%D4*E*4Y;bO(z< znwbR)7US(I^oqGCAc%h7doA`WP}Qc|R^-U!-pFHrq9cZ~2*5JA3Mm)TA&oX3Y+kz` zW~`>YFA4bizC-GRA^lRQEieFGW|i4{A*)kYt&k z6zV9y+bM#sH#G#H3NMBtu&J3TjQt&F)FBTj-jow|pZ4e{o|IW^Y*9MOCg?&+lv0Nb z9(E{<)j>+3@Q(P0_EM&}xzA_I4QMw_m__j!KxnlJmpNv5v0WF@6p)YDbqr_OCZ3=$ z1S?P4N)kGE+B=wZt0CFA<^3e=Z_lT%rL!Pgd6-ooACoU{)tN>`x~QqR9oDZ5vQ2!A zZopq|HaQD}Ub9si#+4z!ojOhxQyk`ISGGQF30jB$A~Q)35)imK#55UO15rj2G4zqU z2R|?r2YrIYnfzE}S{1wcFRs=`vQ%G77hy=r(ue4zpPYlyBj9x-$@pP@jOCq=WncMI z#T)?41*hsh&u&U;A422AN6En;ZGIQ2O=O&-1ofEi4S-&3wEHS0k4l)qJZIQO^~X=I zTLGF7QSLp;pSitrC$U`4z)j37OBAccbi8ujqeA`x+@GGCKTHqAb!oXwlI%U0Q?s1w z&3zumza?O2u{Y~e@xEFn)zjX@Yrx20^>QM2d4`g}$hKp8k`a4NgKKam_CbBrYLHj; z`VP#y{L5Ks?*3>Fpi9C@#CE`Dkzu^kh^9wKUi>#U>3_x|;r|Rq1!*vF*#F|J|1*$l z{}_W>fBE_BVg4t%t49C~CS}u=3pE9;9*hUL8-yzEdY;1~axMa=C3O`mMw)seDs{i4 zFE|KMnNOSOJj#B}a_auKQQ!W2Q619$El>Sn7hlZB9nA=Jh`eDr7sr9ALj{VG!rNFU z;i@U{jQ@%8bg&?G8>JtCp{PRw+UZN<PLy}}n$+}(^d)b#;q zQgMRW3of<;Y=z2u9KBXSqQ?8j)vPEo?<@!m}4!P)3o-&_**%nH;l% z!8vY&tdEp5ctqn3g4+tx=P~N76^3Th(V_}|^)rnf)^m|3&M|{!kj0ahRi!fpm*=i& zJWRwLw~)6PNG(1)dyr2?CB`p}?Ee9%jnQr7n3QX`+?bO{oN4yPoY03&4ppeH4sd@d zG-x5`Tov`hbA&_D8cl0mYBmrIEK$mm*UKPAL@v7T!|ylhC1j~@B(V(323O52#MhOH zu!>HL#uw$Dw(e#ZYqOk?XETW=xr~v}-(YWeVy4TF37avw-%9Wd(a|!(FR7E_*qEoF3`jHc59CA=&x~Sa z*Q715W-B!?k*ewLwsF;F?}{qX9V-XtzYL9kkwzf-jLNbP&IECquj8=!P0isd}!|CgPi=v%j|M~{6H_-`;E``Bo}O@X5uZ2JvU?)d=r6f7H7?iFujZCC-H?k$Q{UDDjP zL#G!GXOEzdmL)i{|lf)XkPbDybO}Gc?dPVMf zjq!ZBi_%8h5%CazOtDKSRD>dg1VhoM_PB^%0kN(pdimGPy3KIW)hF&+k($|VOK_(h)Q%X z)SO;OsO?B$$$tOJ_k0q+JwYHmEuvyI%qs^u8+{89sr9?m8*x^TPp>Gs^dWxG6?hJe z_2m5Q@yNS~ly=uK`g4K79lE8OSTg^@@b^31K%IY)*mUu>2Dad^(KFCoGQvJY@*Q_# ziq*e-3166I(zh<%0gbM~G5(%J^AqwvecJzThWS6Q+5dUVMjFEifgt{88k(o~kjcdO z_6AOCa3)-@OR(n2xajO%V*| z(;N>%`r?e!{QoI};3?l_$dOa@JE0+gN{#>V3=51u#2tWP8{rwjWA7mwt*#)`!$6|0 zkEn}`3?$3S&_~C{TqHn`&c;C(NUjAU$GcvvMZ2)zX{;BL7c`K|S$)Y2&RX-0>ut6c z?M+(CCY$8#tY>YN?SJchKDorj>u5E^J-#4fpW>ef@7w=g^xh_$yYDc*0a_OQ zE}}>ozKFWmZjMmtGRtilYwm+%lXEt9Ma`Bxd*3h{WY8w~W{o;Xk$X(kQj)eoOWl~L zNMn;+5~Z-r#ZRa-6yRXeK#79*IJBSe9?p&vWezL&AVwlp!GpiRDoR>0#{S?Jm^WDe zK%mS@$2rAVxO)sdwV;wVJTI7QiG&TyH<9mIw#Y^eB&m~O+oFLjjJRhkTVI`^aYIjC z@&i3*M0d#S#T4@3r1=KqOPJx&)VWPCy5)2UywjC;8xAA$1jq31uWzJ$S-OABhzvAx zs_h1rSG%`a<}S|g^f#ExgB0*!#n}oO_PCfz1NVAcaHD`b@WmQtsT|Tkj3^;ArrC-M zrN$k_&SK|gT*ZAam_gYsSz&D#7CPP0Vli}JZ$*(D6PYwEmH`(wIK-bx<&7OLVjqqA z%=Ij94rcTvdTTf{-ZC2YLKz<}`#8F8ip1d$Do%_%qbx&i!KjxAvPRjL_N7m2i_^Dh z?35HOOAA!)r?Sh{@CEJMWJ3Q)XmcgAH1jMot*ggYE=^}B8Dz=Ox~Vpln@zK+<4gBZ zU;I@iSyIc1gL^9$4b@NW)^J#?KX-8ji#X7*{CWvTh}0AmaNZVN=#^J%)Z0=BrQ0on zMp4tx1tYKCX9~9oFmjf@LLbR3nS^ST#|6fRPXS(V`kA;md$}m&8%s6|Aq$e^6VYq6 zKV&h-^QI*0Girzhz@a2WJZii^6wM}HDzsa|MYK2(PW+a+hIYH{eO>1JH2T2t7Oo{& zi&7sZm^89eQ|8idPig7C=1Ub$I`Yks3rY8$t~n*J7>c1GckSiTmboPCGxZG&<~hoF zdtdUONOPho;t z&ycG1J1!6s5VI_VrydRDM!2{^R8>e zvvO&+c^TwuDYkJZs%ymizJVqpqSprAB1Lki@g-VCK3@sv;zjQZ zH2e5PoW(1fawhy~gt5o$Y2MBXBtW-}kF!M4Iuki5g|lwP-c}l~iI-umxeWH#3%iUzv27F2LrJ?V%#jiQK{j8)=q=I$Do;Sl|7u5N5-3t8^9ano{{uHD zkBs@1D*p}Q!hxB9^x?9^K2g%p&T^EA4>{^3FYY>9{VLTk2*SW*3>oYi`)H;r5O;}V zeNi%j&0f3gIsxI@ynz@$GK!`h*El?mBD_TghMV3Pu-8TQ>^H!qnHYX{w`9Ya-e+OX z?ia1PVk7P4byH?;E&_xS%TftOd&u%8{_U=M#0}uNIbSTz^_Am_)Uo81b}${~9CEQF zR$;4?4~fdE;*e}&EyCa5=g#HXgs(VX`MuIjLQFm#< zFy>u5Y@+RQS=U{LiTw2~tO+4|Poj?pIy?|ZebQ=ebZhk&ruSCdH(2GrKnPVa9M9Uo9xjd{!)B~(q!imIdT7Y^KJn9tsM zCAe+r8i`Pw0a5$C_ChCkX#bSA?#BOMj3LT0;!WUWMNmw4Tx(WYHvbqiHdDvo(l@HA z;Q1_@y5YlC+LtmjZlKJZao=B8RX}XkCBJNX<AbpVWM|Ps5TnRfe4kdRJKD+Zxsq7cmyZeIXcqbCkzxLk zD0a!g#_n#@ee@tW{mriK<9Et)!Mlr%$J$_MQst*)jj8Vx>-yj;0;QwCpM(yd58k5z zWo~@bv{b%Qk|iubm}5PpT|Rh{9zynB+ao5# zs__gpJ-__cj|eI`5c zt$;B|$|>4ZK`m|VH|Z1lzkgBeh&zJvr-8Ur z&iuV3SY1XN9C%0!$W`4lB5ojMwd{xX6HEW zp}dRZGiWMSvZt~ggLK@Y4TfsrUK{2LEP? zU(>4%N6{@ci)x)Q$HVR2aWU%E)+%DLQPDewlFOkUzt zq)Vl*<{7!qN?YhE+bEWpGQ40v;sO3xl`To`K~77sNU3pM$?tWxu#J}~3T`zfnB_1m zC4Yv}aVk0!n-YnimsfBZ>aBbT?w^7SJ#a9d5L}rA^(#g7ZE!)LO0q$w>OXM|u1OEc zzJblFyCEi#a26@>hXn7^r~{K1E#U|ouAAs(hclcLf2*u=Xbx=2*t8|%4R3>(n_akSQ%>qzePMg}nk{Rth|BH8K&_5Ebp49r*9ryS z6_KO8dZ$QZW8!2F6nA4ZF*hc)pT4uZ|JcmW^;X4tn)U(H!TtQYxeLD31+rPYrwMb@ zr|O6pA0^~yhKo0Z-Zk@b%!lQwVIs#2^0hkWaSgv?!+S?l3eT%Ha$K*5eQT$hLj%vQ zH>!D75cMG*&FvQ((5D>f9X7;$fubp@|z=D`)awAslArvKX&Fp zgDn3Fpnc+Ct~R?LOQr&E5?py{Lhdxv&yK02j#gpXF_|?7a7}MWK315haE64!3ebv& zxa4g=0{x*O(UkJ(==Upo5ChGsy)`BIO#S|x*eAj-$qd9R+|A#5;MB~X%Bg)?{2bow zi1xJG2l72a&_wAI4IfJ*NBtfwNGNM;lv;smfs`v35-z}TPa6MrNKUmnW;fg7xELxu zWN+Z#&iLU7z;R~{sa=Ji-n;8`==wskq=Kk3NWkEK$jHkZMLOT&gzKi$yUAb}wCF5f zSK4QZE6Hx7x1wA~GZQ%dSAajL2(LhfDnCRqFVvM*E zn%P*8zTk-eJt$)r>=^pLcOdN3d^P?>RP=#KabQJKN#Td)MF2Wgr|s}W(LaXTr?@|o z8VVsbixYqKXz;AwkiwMC|FojCAqyelf~9J{_q1=)nbvGS8`%BLc`cAi6Qb=I23|P; z(%T52?@fB1am&TYP`8|Sc(DBQ+J4+}ADaGrxp4bNl%FmF`NvXyuBO6(90uc$h5Dks z7p_I(fw!z}YXVSkLV$3F%Wo#3VcWj%u-z;g;dj9#Q@PrQxHL*-ZF4dCkGTY+c@)Rg zvS~-pbsgdIA{8No7t5;3guA`SY~16oa)8ge*;A2ed+?5+z43WQ-iiHm4fGK6LqLuZ z{!Q>d%@~gv7I%i7zGj_WJYo7O#;(0B_B7K7wddw&oGSoI+ZoV1|jqeBaYRXqS;CVeObtnM>}4pr=rBM8vp3FGKxdwN(;wSY+1Z;gM}zk;#&b z`}exZs|V;miMHUW78W4{CsfVs#S^TB7PB8Nfggebn<-g1a5BYHZo^xNR;>521SQ}e zg!P0+Gb{SmEZvz+Fl;!I#a}3%g++OPxU$t)PoP))bkeqoYC3aI^>_HQ6}`HEY`mV2 zD~<)an%b@I(v>g5Zmo{d>GN)1D>2SuVh^V`nTG57d@1pGNrP{ zYP&-(e7~(HEu0yJ#UI>`);B`Gl4mtWW0)zA$zrh~Odf@#sK4@VM=uk++z~dsOW(<> z(7ztz>kHxGF}d3;_1kIC&e^SU|Cl*Qe-5u9l^SuJ1kkql&d#6 z%?E(-5c32}-^tEFPY4G@bBxSEz1rh&eQ($r&&E zm=oK!ZQIVop5)1WpWjpG)W2%)s;hSQesgvATHju)SAWjBiDpBK2t?Gf6@yT2N9H7F zgIwX@*rXX+LHFXuNsb(%>$*^#!pv2$gR_Iq*;Gc~7<}vjJDrk_tb2(9gOqwnCcZyH zD57sJ#(!G{v(d+zOe57FQ@>48y`d%9Ry?U( zrIC83;)av?DrT0-{b1x3J2R;uSQh37e^(tiCZ{Us)^Ylz1c3izuKv5!|6ilgeL-Sk z@|7uU`!WQkARs`oQ35L9H5U?QGgA{m;s4nTB`*3|h<|N{ONT#)DGUgRE;dk!n~oT0 z7{G;@!h(d=?1#-B1@mv0ApaZD@UN83=@+Q=8C(Le`44CHbCay*J4PiHbi_hgBQZ=x zqo21FtUncpcM_SXMY$aZrwQ@J++@|BO0j~^iHc7kz|TArR)lB1l<`vb^S6BQyfY`W zqJ@PEi?hHK-|@!XS5{~GD7G8$3fW^@@irSn4AbxH&O~C=fQ;#=(U+P^>dUsT;>dDU zd%0CDDba3Q z0S5E)RAXX`Zyh;dj?F@+v$3$a8~K-Og|z#Nm>+_ZYp4GKdP~R|o6Hl0SC=WM73Sj5 zj_H~uTR(<@C<0MGu&)+yRMO2+uI3P;_HL{mDMTiX;(9$D#lE|c5J;=^sW&!ay8`;j zqs~Hl1sY3l5Og@B^MlZoDi2$mQly8zp*9ub9$41>f~E1qrUa6Pp-$WK?;hw<@Maor zl!=QhG|-vv%UP-!qo6C@4$deo@+C8YN~OjW-lQ_n?Cy;&Ji_vu=n7$qeDaWs43W&i zi?HKNqk0YFyX+`mA|?!+Izj5lt7w}tNqAU zqn4c$0BF7>JCxQu7|U`l2v=+jZeY~Qx)xv-!GuS^GpY4|(14Kc{Td!C29yXOAE*Gb;aPIe7%+3{Flq#JRseksf!DVu1@UpY! z=z8+TM+_EWl(6&GyJW=Tu4?uo_q>9A&d!v5LvywnM%gT*nMn}WWkl&Dk#e1V)-^tB z7<#mayWMncqiD2a&*1AY+69DJ4==%~I))r+Vian$QmVRKuAqNLA`#AnHe~U461>jp z{L;85vwRE)-xA(`Yf!N;V_SxN>iwnHSkAW8-Y4$_K0*$u+wYp5=HByLY^Va5`6Bs9 z@{W^D?~VC^o~^}v(c2!po4tH(yMLbhgnnkB*BuSlOGx}TM8bGYq#uBh(5pC2fb+D= z7@q!%Nxr&Hl*(@&P^{DObY(kq60B}F9lf; zR~SM-)&lKbNe8D0K&b?8_^m+FILcEMf|hJ`fUIkft*bbZkm;NSnXG}Ot5?ojbPL8U z4Yd|Fcm41b(lR(KhY3(QoqNRyFLPpfdh(1KeL|iV+knPh9HJ@TE0i`E*7LwK1pyGh zo8zDA2p}Wd<;&dcjb5DronltDGg=Y0J|UkX+zEhUob~kMK$>g6dH_Oic$uCo&j{{= z2`S`4QgiDR&5Mosr1yE91o(iLN7C8tqhhN62BTH>M<=d(9cQS)*|GH?GTNcZ+HjOH zyC(d{9{$%x{jWXzN4_-^EWup=2l!O+&fG@!b&sqkDo_ZS_5YNdpCtV^phF-JEP$Ey zKj$S1HC{AyhS5K=8+oO8f8yezk`pJ-FJP2cfYkaA)RG0|4doJnLQ)nYSw@)*Bvw3` z5XEHt*2?5~n=5k6D3V!oaa<1?>PzJ~wOVtr2b=(AoL`~tpEEfoPqT^ecD`~0;0(T=!}`>g8eW8SIXk()@s%m!g(*9^rQz?d@| zj9_nIW%qxh!=iYnqdATRBq6NlM$Fih2&W?zi%n=UjgNBiZ`coF^BVSRc`t=P-Fzn` zIBMh9@RztQfsDQR9B$okR_FDfcHTAEnWucY$1Bd>>z~KFXT{(hf5`!$-tgvh8;XGI zqunGROg`s_F~YPM4g5hse2#e)*}qD0jGaD43L1)qfA|Gca>hfQlcMLSmF5E3s4AOw zfmX8!R8*ByZ!qG@HUn!Ky>T}l5-4XDi=IsYF_P^;{Cm!cfeI zhsv_p&cCvvQm2&y26D^LzGR4XB1 znk0$m0MP75KimmODqDpdd(5Q>EshzL;_DUdpxYZK(MLum>Q>wkzinL@{E?F6*FQCv zQVI0p8$!GJJdejk%S&rIyOM4*@}Wjc?!so$fDf1lhiMM3PL-RlUd-YNIKEeld_ zkZJ+91=E%`IyI&|aM9JlMMXA9yvFHb*uzEX3b%L6t4;){kl*jDlXQ{pB6w>IJc)W6 z^dUJ2^?l)RK)0MG8B+Ih4vl`-k?nAh=W!k6&&H^7Q1jr;GpiNupXA!?`uWo3nq4YA z6#aB`A~x_<>C;L5{GMC&z~bN5d6~OCe`#rb+uW{`p^fC?k-oO>olRkoG2Grg?p8B- z{>wR7ApRP_!4NSbso6t5CI}PRLT4=12jWY!Ywj-5*Z$lZW+3@Y?n|CqZ$VvpSh%`t z=UBoGEx{HWfV;W-!?`JbBR(2AN9vi~mvk4FAxtynT8AK<^O^_dkK@^s_cJ)*N0@+8 z<#?7IhGPX)oP6U{FJDzr;;3i@#+}nA-Hu{wm2)$of|nQtai5VF4e<9m-^n~c<^>nO zG8RV$H2QFX08ryqVF;A0f_WKad^^hweo$$2mE*BWXXfJiiVw;br>q9p@V19uo3m zTk#NJ#r6ATYwk)1re1eJKTo1@2oXH<@FckB$`p$~y*qd^K`=<~dknHAj|Ckus+Y+a&>P zj8{OHzLkdk#FbhYPp0W;y%YTb@t@40O|Qti*C^Kxdn@{8O_RLTtl`dA-@BlSIv(v( z{#(X#Et42Z3QCV&*Gd>`*;!5L_c!H%k}jZl6ca5sA`TnN>n1qP1z?0eddr^pk) zZeejWb)SH;7X9Y_@m{C=H^qoB5r5LaY=+NoPvXqo55pK@J=&6Xpd!n1YY%gFcTfvQ zxF>&}-h?~B|0ZdcE2uvDNZx4u{Q;AkMAGv!>}^QM^`{*jrRo za^PSPbrjlP(5A@;0%tspDcAmQ;5_|MY1*C6N_dRBJ$P2@Qfb<_R#7kPLO8YVQQeD+0*_e0{$@ z6j1&u8=k1M{4S=w(4U2A(Mwefy}aB=jsz)?ns2z(xpkJv9Ah)^G~zfI#4xGDOYk&2 zQNQ+Cni!1MEfyakFWlW=)C%BMEIG|;%B|eKv14z{7}aGQ8BOmD$+VRA+i)86P2!Vv z{6cPqc8BB%t9VV_w7|sNWde|tTM>92Ia+@sBpjW#&lsjupL$vo|8-hAe?y)Yll;}x|5w-GCyNjxmzryZdJ zPLLnunYu(M1QNT)|8@V_6tO?01BSw$C2z2XWrv@+Blffs)CX_M4H&?6m~oHL9p-aj z{F!s=T!=@=PG6k$9gT?A&7Ig=V>X)9^SNqHqy*Tf5f0c*{2@JpBdJapqZ9`Q(Vg>{ z4n62`_6nW^fjsAoEqEx^`3KD2u)`&2)yOitG@#OXpsNPi~rRi|HNp93lCV_zV>7VU_i$q0%<^@ z(yTO?5L#z}^^sZHl6oB9y*hsbvXBt55Vx4q+pcn3>SVD6id-YEsf8k5hMUSRnGyD}F!YJySl-@3RT>J*k7$=)?qAnjHhux@PEp z6Dj1rcej2RJ3McKaNLZ;E7F2Oc%}F9MK|jVL`)|wMdFSk$7#@~4Q>vg?ttBpn|x0m z55@8QJZLX!V~H2&XlIxQsNy8hZXpCkUW)1`056{R5gKlbQ%Qg-M6 zutx_oJdaMl4%x0F{xg`Iz?>A`Z@>)TSE#L0V^agy6z8+kBmu_AB9GnMMyNm?5*Fc# zMh~F22)_3RW>NJ?MJ$v7?_Sju#dN$u_%Ci~s4e3(O z_325TgxYn#u5xtr?PcQbJ$w7EW18pXIClFlevkbPB|0YQ_te;2)KbqFhEX%2DtdtU zHHv=9B(2jc)G8hOj@k*$9I}OE>4>NUbm=z}Bhi|u%!p*@n4TmBy_Cr^8@kns(wcq; z5{v$}2wG&kz|uaocuWiI9LX5{%xoQQ9#2bZE>_Q9#+JXV9EAmCVc250er?U$+UqyN zii7-+BU($>mu-7ZXSTM(N%GRqon!~FgtQu&4AW8CueGwaWldI}lskDT5wY2<6>*Y$ zKRL_BR2<5rb3;_fXmYHoN#WSw;YO!_Kr|uHI zOxYb`RZR@9cHi=RFN>ezNmJ{m9a0hZTq1mYdZQXfpgu-aQdv%7BOH6B1#<@Isbyi) zG$PRQt6xng$<6poQ+bN0wYnI2q!po(PfMbugcdQL<9c@Nq^WzaO?ZhY101x{(SJMu zNFQ%dEQt2eArYhGbEg_0*R1n@ix=;}ux%n})ahsgm86+aAM2+dq#`s3% z`11R#j<}1EsxJOGKTTV#e{BPNro!Sx$)d3iwVAU-s-i)hnNMa`dO-rUHGEwkKWQ1T zU+t3r)DVtMOO7^NapodBr%CRD?&-X(-GYjn_DDNzbuW*c{pvLN2&=ns9lh zyqR(yL6_|f9Q}26|HDsJm1deAvcaZy6BJS(Vp^r=@FVKCPuZ}m!m$a!!=t0Fz*r<# zb$lJRD+ z287tXmXP7=mXLC)1PBCj8!LB&YHO_imXPC{fCy`xs(l3iEu4^JJoB|$=dSq9KyoH` z!OS`i+_!M0+QP5UJoyE#rT+Htl-l=|PE1^Yvt?veR1+<)VY zQjfDZwXvTa!CHr0Ztjl!^prianh-##eUDJ7sbgNhF~+8-wOgzGFlISbd6n=tM}$_* zPWj6Onf=2M3>JP{p$mrBMggsaQq-gZz{e?^!_e=?(xCulj)sJs8<-*>g(_`O_ zl7`uIJ}Bi=7dayFpY6?2obOd#{#?>X^fNdpDkw7hjIfL&f_EF6!l?)^qkg-Kk}a{e zq9N{|aad&UEc3GlXoh4nqISAt7Sv~tb>|= z?Pi#c?jmsGXRXot!xShB)fnfBNox|-8{36ho&0Pk-SQ2KLFOOoc#3v91I zL2dIfhH!IkQP)Lh@UMmf-_0k{Z+0n(=(({<_;4JmJQMexVC}UwGP_? zF*ztTe=b2E$hXtw@99iYW?dCcBtp>pZ2aNd^&|X|m~jJIgcsP-kH`y&-k3K3d3+d!3rvOC4=~E<_WMGPxMdOX;=ISC zhmmyRJ}9W~wU5A7jA_T}>~o!m;-ntI5(THcCMr3r)12tV)$X%*{!T-jXv={9a&H$^BHcMQ_KjFq$R8)1_$iRFbr@RNMM ztruEB3BYi1w+?*>6@+%XPM;pDg79YsHf8>XY z!`bw9Z9Dy&uerB|+TW!E3>ej~mS)E!QYq3v*VfElhes4>8*}!lBzb^g z63U%D_k7co&xG-VM_|iissbQ~68CVvb_H_penRMsw+a&E@#akDPtKqJlEMDy8fcu8pD zB&3~!@AVW^j2_gITh5OY7gJF~^#TANLa>#-Ul`7*(?pTM@RxRy9xA=`)OIqONe&Wy zdXO@(SIieyOJ0->UhKX_TwO=uOz*i5Ez7QAfZSDIjTLQydxdE>nSAz>4ut1YDwtE) zkr9#J?>uEpzujz>Z!)hTMZ8@v(A<4;h(#1yuLt405s79gi8JJ(-<@>$HC|@O~fl}uyL=!Qda%+i20Q^&? z(b8HqUPENGXZdPLQ>2&pJXO;~(cm@ULLi^UcgB_XN}N=bwU9%uU}1|iOxXOFRC0#+ zd}UjbV1)ur&Y=yj27T$Sjvh$}XDutEb%Fu%B?1 zG0Z2$;GGsFd`N=qku;4Y4NoT6#ENd7EV*_fPFCU;HDP3j#w;98vybHlHhRJ2=N=`K zA_<2hNaT+gT2*lxnjw*FCUNVKC9t|`^a$k)5SL4BmwRqMQq^zrbU6$5{6pqb)o$YP02O0TS*?%mLABJ1NS%ekSK>yd{6_CLIF^eSJ4z`Y=s24wBj2q7r`w$ zx&(w6`cr9th|hYSdp^k7gAYCWQGZq?BENL((^yEQ$!}m}H)Alm7oh`Bc;@d_T<06) z{-gvLHuy*jS@fh~-Y2ryGAnDQqh19un=7G8abfqTv*@bC$c)7SSzBAjO`|f6rgi$F z%}*%$p8#`pGGz0$`;4|3GsO`mGN@WYO~8UdzBkpJ)bShqyT|?jrN#ppvQd4vHLZsk*nsk+pY)e#8&Oy+rngfaqWLOzk`ho1 zEJ)Q|p>V(hZ50U~V;X{U&7i`cFeZvBQ^5ct&CaeN3Tdu4Zalhrnz5k*=h!qe{G2c8 zy}X-O)9GSwtX6TNIcPrdwpDR2b+xI-z16o48xxlXhdy1oak<8k<%B(=Qc6m*8xjNJ z1`uS77gSL7uG)w5F5Rc)UA$?GvSsX=y`j%Oy@=d|ZPUk>7y8?)RX{?LsG!`5BMeO(P@)91{G{JcHo8%%sH zyv76)aO!W?2}6Iwar&e1^hH-==wn=s5kSkv@fMmxG?9G16J5GIM4G_lPpw9#7(;hg zhHP!fQyg{a+EY1NgBrB{tYHFo6G?Yy^p7pc-o5KJP=~hyN20al2)zoY-zE_mv~B`Z z*_AUHSo@jrf1LGxPAWWtLk>snQK8x9T$=Me_s`mJFlypkxiodJ+?Hu^28vTq;sdDN z`O;+xUNx*0LbP8ps=nDGizf2^?cie@{W)qc+C=3Wdt=mGu*j2SM=A7N;#^4+Z%MPmB>&4$`SJg4}$CaN3tSJH)wCyY}kYxFf4DO#;F)I#7 zEAB^5D}dl8W7vCOI(=q#xRSR&>v?9$9p%|L>VuOpoVjTY&eCb2WNY{v6bb+WZvY!U zev8kTTx##5YUJThK8A2^qyL)z2F;D9xL(*cClAMr%U^FO?mb2vN*A2y)B#A@x47E;dgZ#`E*Z5d-legsU|np2)Gcx)U;G|a9`w2W zb&yG&k}MDZyA1bB4bdjWVIk#mff67aT26>;HUwWUigkV@2r2QoMOcuDcR+_#Q*Fut zdP8)N9QpC{ze$?^n$iEY%KsQx3nfSfjxT`1EFRF23G)k};6#DdTu%<64iEiLbHnVJ z8E|JFLm$xC2u9-D=N|}lV2E0UgXBjc7ly-3MgXiENtjEpC7uNw`lq^ILp>Kn)i0uQ z{^a1fdz_r`%GjK)enJAFU%*Q(Sz)Rv?(Yf~fs=qg(kvr0HRKOOc0%A}Fd>_lDP#Bk zbP@EXB!CGqRuJDaW5$Jj=m{leKGes%cIZi7I0hKZNm8=pUGY>Yr;Yt+!yF|c(_4-u z$>TOjfVMh$G`SEXk<00Xi51UN)EDFj9bDB7C(Bc+{!4OPS*81kx$j#sm&iWr{H>Uf zQI)8w@Ug1mAxG{D;#}i+VGN!UDW{2127meK#hTR6hIxB3+Kw+TCaBt#M_BRT=DKjI zAq#jd!;^k33zB|qO2W3qMLeVzj$Mc^*0t&KXD_EcNN+^HtzIpWsg)IGT4$O-w5g8_ z@CdfuI~Vv#We7DA*JOk}C(fmPC~FF=h{wkHkz026FJ$qb_pALCpA!Fang4Z{|Gs%Y zmE?}E7Kg8>h``rr0w5wZ7LbS<79Ln-$&J@+=8v=l`mc*z|Ix@z_4W4+@c$q%zWN{n zriWB^{9*f#lkHcVv*UTZ$%ImKedsoO!;%QlD5DMH_k zei+Wit(v|so>MiKx+TvT(iFWox2(iaZBTB*m}-=?t$&lRb3YB$dZ8wmwc@SoxV(ja zbBXLk({!bVkFN|6!e1FepeFKJfb)7|wPC85QAO+Bzbixp7V<7z%d>*1K2r1X#~+&@ zlPKeKldPD-btEf%;#8`&Zl2q+L&Al2Qog?Xtz(g$1DoJOfo$+Dv}ru6#Hb|Dsj+R= z*8btFs8y%@v3x-V0fp6I!5SXBVEw#v<80abMeOW~&Xor+&NxDw!up#zbl${yfS*X7 zHTMSL;zZ&R&N%0g@H0X*{|z~OGOQ?!!d#U&J$;r~_z0UDKiV?lEYZ$u^E7}9+INrB#hi(l3o<$|9e>?*ITam1TAS*l`$LEEu6utJn z;vV-OQAYOPIr9Hbv#S2i?yN7giYnGWstg~ENrr-AO(lu`A46X zdPAImA##5`#wkE}t9(t(4lv-&I^7qv69p3z=o7_--MkKfum$~3cUgZ3Z18_4s4eLo z&-#~!lS2ghNx|X+zq|7w0he#NkpYG3GR|mUB97bHoI7^v=Pj9Ef&~^uRuD3DM4Tv* zU(gr70Sr4kYgDc%F^^~BJ{L{j)R}kU&LKM#j+bs>9uQX{rohLr zmk=@jV9`UjuD83WMo@L*AT~&nBgak@fkvoltW-y(rK=vVZbb9lk`$<(D@z4kIjRGz zU929RAx`pBlQz6QvYh1*Pf3x}x=J%7aax+2gHtN~T8=c)tn40m^zE2B+0m&w*JG^VK=$J`e_{$d z5_@!Dg_@Mr-b60Nr_f=J;&4f;&CUAnO&3lM5!=O+e50Qln-1kEH9&SlbpE&EbysBE zv*)+`oZh7PpgrEE>h+?-2}x)2P78Bq=f_N$%U<_S6~4^b;#yi?ZY__^E-4Kxt@tT2 z4RIci3T50~_ii2JMi;v+T3B^Qt%V~nH9GynRyAi0Yx5NEjI|#gH+)==6^ZPX$xc|= zDK}%d9;6~cuBwqqzX2%CFsYZnTjBeB!+*I;d_~KoWO2`ZVn@%biU?<2DT*AIn3b)6 zyXO30#}OWu8z{$Z8;3cghs6-5K+*t9tIo%Lt8Sx$Gib?p)#AyK8&#nIYU-T2UYV*O z{KZF*bUFV|eG35g1*a*~cco&>N_dtNQnP)obOE z%X8LfXch4Ad}E3Dbk8OirveTpG^o~|z<1oP`7 zH{4auLDgq50#aP2=i9!u6FtX(w*pPtt^{M^JBnpvfFserPebL;LFW!E5ob3l1cLJa ztO!Q)xA@OI{f0G)8&c;+bR-bg70l8UOq4VZyXVWseoSy~p$#c-q5Y;C2vNyn`6xg5 zuEIkdj6*mf!}jcfoClYf?LjUGy*YR%ndac9gA4wyrlfAE@sr z3XX{7#4!Nlpk&*J!=Dz$EL)?$-7s(udWnIy(*VGJK{hB(R|E*C#(W0S?+6InNtCsr zAKD3Q1*|JM@q{dUHxs^G=Go>pUO_G7nE#o71yMUNBo139;rFNE52oN3QQe@nLber3 zZhXm-35+NZiloYnfGqY5ROu9|{~_LUf_k{U%>LQ+>MP#gCt@*iM1jLOyTH;WwHf8$ zxd))o9y@*R2#e{0<$m=hg9YrrTL7Nf$e?^h?qz~Ro}hfiT+(qA&%eg)Ba8zwmdO*z zw&M0cu3yEB`WSu zaY7-4O_5Fs24D!4Mc8%~5?l?RRX2S;hq{pML@4X2e!azy!c3(PBN=parGl9pn$N23 zo{Xz^jLmP|)i|#VRXbu%sKNg8@oj*A!&LwE0srHGAVL0dG0oT)^ql|M?N99Wd>8%F zJH=4I-cH0X4($uhMF; z9LvdM)9a}4t}@>JikAU}%=ali*>r1dRyI^JO@qx4YGtwHY_iRr+frs0z4c2lf5y=%IOJO;2F6RPj$8vu zS~Jb!W)#VkJZ@Spql;0Cy2Y})mokkNMMBDIKLaN_^HxcUUo1>tu?j;66sqCmy0DPf zbVi)=(CxUPX2p+x2`^c=&2D{P2agwF*D3EuB4tZ082>wx2}e*H zaui8glFZCkd~$w;-|A`SrutG?x~*6^;2A`}%GvTd)((fH)D~BG>dG zI8?fPu}Gx^hxSa9L>5a@Q?)t>3o z{&e%cE0IH#&(*{ht#JF@*%!98wMPW=Its00N;VJ9BSm%su;d;CtFK!IZBYr57U4Jj{M)%?f<%K@ktTY2SYM&N5bmOG;fsI6efDTrs z>sbubkqp))Y(~4;F*aFlDwE7+&8s`f&Z2U@+t?w$|3EOdZ@lW-HnBMB&>6~kjcv!T zvS0s20>gy_WKZ`;Z;;8=I~)PoBMLAqwF_6t5_cZ z_xT94$8!=#Wj4G^>AGdp#|vA9(`gG^rHKsaC)w)&_uA3b&UX(4x8Z2x&v$i4-z>X` zstuBk#G~!!-0MQYIyCkfJ*Y+oasJ#XhS@JYV<0u&R8r^ea^$`LivDx07}Z^IlSQyL z)C{|CP@A(oKTq1(uKaBG@s*T{KhozxrtrJskM1tISx&pZRU9GI!nqeK_@GKGUn8_d z8e-}KOl+gsJxW*^cctb~dugrSCN!cqA|#U+b0#e5(q3IjgVVKw?UBNxo-e-wqq|nA_b3rw4wF$OtL!!|>?4fSe(g`C3Vhwidj=N- zmbY{}!|kk*%m`*TWb<%Im5hFvCkWUeyMN{?Li4Yv=F7vT2 z5KyLlgEVLv#f<7_g~G3u0}Zo&jhjd{h6s<@aO&N+QYV#dZ0*d*sNp$4n$op;uVOL; z2<0+=#;gQvxkg{o)AtZpBsDeDF_Fy@z(JU+L*&;&f^DxJSjU2Oa0-+3h8Kh|i?#+ER-{C`ME%lwU39;4dZw9h~oZ#NUc0 zgvh~5c7IXvcXo%Rvb>%-o9pNnJ4;ac1A;vwLG7#r;NhUsi|J$nN!e{n?;EL2!djs< zpm%TpBKvRbThj-%4G{g(r7#U4-nEa=Cu{e$xYEivgYM#x!t36>g7t@FgRL+!)|_H} z>%s4kq)OGC)DX|Vjre>5`Xe@kw6JhYw_A*?GoxEn`;)A23N1W;>=xI#arIbC0xoU8 zuGd%ZDKmGI)MHF$@et4wuM+1(dKi!K81%o0L>N(41X)5A>w2-_2v>@lGZApazbWCxk@WoD7p3> zWoE8i&Er^6p+^VfyjC=m-!?aI>(T-KwoHo)a$P6d-y6o77ro1cQ?>|VK+4U%h_EU+ z=oJ%z>_%8X9nWtMo+>kc;1+vQ(UH>vCeX|PQrWy@{!*L)8>Xacw!(Bw0EqH{pQ{oe zDls$5u*MjSGtVINgM>M1biC5F&=OjmOFDB6h21?8VhD|@VulXH5Ku?~QeI>xNkl1S z2^V%HA!$S-wh;;aK;iXn_a*WpQy2(7gN1!xnEpB1Ki*i0ta&1PP zBzGpAA1T_l(9f7yxx{UX0Ib$2H`spzz+ij!$+@ za*RRPPl&+e#=H{{fC~0J0Z#4!DgH1j%@nC^u|#Tw2*KO}W1HNrb3$2Pur?P@Y5i)nkyub< zfhi9}#IF}Lz7~Q~)vcA%1QkHxwG5&`hc^ZDN>cfOch{np zRB)&AE@zn?;&VXdy!fK?ZX4gv%Iy=Hz4*fOULg;hN9P6rbLL+>;^zw91LO&3a?)au zf<+*mq7!~@h8;Fw3r+fIB!NSXjG~imCmxAK8_d1D0p7v003(|?+tqim&!#O30aPUM z4x-0|k*WH`Yy!I)?7o7>Ik(fPMfW7IW$n#)0wz25F-H^QO3QY$nYu-Z;tp7I&&ObI z?7Oe7(c3dGa=tRh29kFi6@J+T$F80->2AM0VzW)tnOOvoSGpHGcpvYopOu89h zf7RK{d2E_TF)d@{mrr3n%BKMPS4K0*_h>ZtfOoxPz_q~cH%F8T9;g=3zsVm(_0gBy z#gd5>lUSA?kp4H_#F!sQpFqoA#pm%1M;w?2i1&D_ZoJ|5Nf-i88$7o}4~y_>bcL^N zbT9u8k>hY<6UO~y--*x=BTFAzTcbw@-iDF{$wJc7$W*h~mNs+;z8||$4P=A-xThRV zfK@{j4jz@>ura~aE5mfwTN;^Wt{2nIBVmVH%eF~d86nAf4;P&>xlnt@p&jN%fSqwV zzb<3Wq)Xp5{|b>JQi5-q_(SE-P~;&R)NW_^wCjrVQ|3ie@^PFLV2*xMg%M<%>7s0~ z1TcKJsa493c!`-lO}n1&yY-stsdWYp09rvUw>Ukwm}W)=@_wOLQ07oIhK0teBWgM; z`+~M6R4^R6Ql0{`kIl1*Z<)X2NhMnTWE9V!`@0*75E0IZGSX zR1=WJ{H-GS>8&P%0MtQrLeP3-8$gGfJiX8p+BAB(Y9*Sh)!P39>|SjzDKDiO23+jQ z0puq+NH%{12?})yA;ya5LnxNW&|Tqe8wMyAN>Vx;Oc5Uqq21qJvyz>t_|*}~Zi85r zSK)^9JO2&`2*(Bm{t?MNeQ9_*W{7bR2kxvH6d3zT76GfVXf3Id)%T{>9{?#?8vM~? zIfcXYwY88|gnHaKvWF%JG9og=kJIn-17z-F=jZ1)ROz8~dJ`-q+`Hx4MpsN!_%%pu zIlK+oP2GM>OmdiBkb<04(7xse6n?Lt6^6QzGrf6VI1`Nhm+GG z?pNMTKLh_8NK|nyb->tKLkGBtfMiTZMd;`%oCVtIECXb&DTs2BpD7lutl?a{H1lpQ z<6N{RQq3=(E!`e8F2Sk}`MGQoH_F0^7L6KvMY8Ye-{t!1QKm((w6lQQV%ee8?2Lbi zr7y|;8i_5I$m95MM$9hh<>&ZSCNR75Wc__15MXS&V7Yy#M-sxpR_@yY(IM9FiD z7jawmt;pW4}M4uo1N>*J@4jvttTE6F>FzI@ZmV>zSWXT^*dvmVH-5KbVymv?Z;-L!4 zS~~D*6;&}LiJm2IUE@s8KF^JHbUI^>e_81HN@EMs5$)$raqo)ZlZbO-X_aqY8tfai zSgq{0l{b0MZ)=p*bV~+%$7Z#k=E5=E`)3qwHyG^@B?53`xqukabYrzPWj=$*pE$LT zakc|;c~pODY-&!=w4Yqlli{}sN<%RVv>^0E()w9F$I+%W1i}+&m!~rd6LzDeX|N$a z+a7I)SuXMi@wHBq$=-ijV3=(-}jI#03!FGV^@pE~B?*4MyHO-S-tDh$Ua=~kZ z)EBM=k+#%ZXqS)KIk}zDxBG!AD3t&VI{xmQO1Jrow)b%T)Mvcv`Ezbb=SbePY=-n@ zvAbaV_PsZ@0dC=G7;|xc#A+2fq4~6N*}nN)IC7qZSjjoDiJNwHR7=#d|s7_PzyR| z<|o53aPWAc6o7FwpFVdr^k8`Y1m+H+=b8g< zNw5wQ1~J#M7*TWbjUk3nAPj1iDaC868b*?{PW!$<%pLNzqP&hj85cUH?)PHujjem4 z{WD|k@vMm;!BM$&+hO_ii-#M`&(dPR8}1p#Zn*LBz4i+%w;DQ=D1Hh2 zN;BFubxn(r*JhYB??_+1l)LflDAfn(Q9=Brq~U|13KW| z07&W|{P}%QH9G?R%Aa6ipF=U9R0x~UBL;(aU}p28Ybo{}aC}!P%!j~xk_y0Dh?>1% z^>*k7<ti>{@V};_JfxQ@%^B>qV)s0c=$0bZI}^@^iB*{MaizK zdaCiRaWs*5(&mWfgt0%=Bk;JRptnWz-S?TVw{4nc*_vpzrdf9~P~;`jUm)Fw93hpC zZATWxhZTzw8u6Xvf;Rw==~qf?HPJL?i5CWe7jATQKg~o_A?^&+h)n=lCR!Spl^ETp zA|@f2#>tu;g#mOiffLd%gq}3{!32s88UNHL-TRN(qq*B=@ath;H zpUX1oqY0fyWJ;4yh5G?9qvp-VaQtR-fW;k6sfcbqm*!3P!j-{s zC%QnGm_(-D<6fSdx*Y&eTC<}Ky5tZ^O`Iy&Hd+d~);FW6OqC`kl)_%m>)P&5%S^$> zozWi-tG#27+AKITT@Q7Eg@5{f=&V}<@H70hD1H8>GI&Scr$z=SbNj2$&7jm&{8dL0 zYhpci|65-xO6~vS>Ybx2Z<==D*tTukwrx9^*iLqAYfqAiZB8^3+s?$cJ#oI=_w%0b zdCys^|LC=MHP)|c^{Vc=>MG2Mo1{UE??)P0r^_JBS@a23 zNiWn%FQ8ztqNn;jr0M?4Bg#^)eckMnZxkn91=uk`PeCaZzV<;}~fHcwSOAG+RnwVJwg^QtqF_YJXa_*kcf4&QQ_Zn4{cE0H%~@&%Waz>G4evv0NoMkXa`muGy@ z8ffZ5TBgxIW$=RkojGP(*KCgBQT^_~oI z(b0?_fR+(fkl!}Kdmt$yEEUsYhCyDDqAr4tmKu|YAJwyFV*5zlQ2e_qTiDNEjH9>7g;i3^PwN}^(YFyjp9Bu{zdAKY_0C0q#a3PJEy>p&21Kz^S zERaQ~acw3l>^zt)w_PEl+LpijW5@VH)&cYh=}0m|q$$bjW*kg0+6bxx5*Qbw5-zY3 zF0?8}P^tz<3GsvO3LudiE&xx0c>(E>mwm%_J6jCh~DQ zNPPQ+T!<_Flg4?lFR?(=N=sjM*cHn)m%d;m^`R~s6S?|@MNxIQ61KFuk+dp0W1?;t zs-gJgU2aZtC9z%mprDl3sgp&YWH|9WUKWWY5fHjXO_P%s<{}7b+m)%3sn{wvTM+}@ z&OWXouxulM;E$%PbS-Y4?m@FdfZ}VQws(A7c=w7w%pQsxiP3V)%|Z18TVd}zwTo>7 zYW0usspykSLZe~GaHcwW+(`JLokq-#Fdt^y%5rvDj~4=OK(5*|B;MdwXqRzVm$6vb z3qUl)@kqBLCo*MY=0X7DGQ!ci&>sN7qP}xDKjSt-!`HiT!Rq{pzCf66<4UmNYZJpq zIqa0SYvfJ_g41b`TN`-i+>vZvO&ICnBFt^;F~s_=Q}J@Ly9@gX{SQp&zcHWxHyQ+f zcUUn6(aKd0K@9nyD6DP`1Q8g7-aql%t2N{yP^5plm6a)C8(%aJ-Izd7XD~wG_69OG zFi9308kj^&31y^_=yQw+jOAz6!nXoO9+sY=}- z#a*6=SXsHQFR3Zi)Ym`N*QkC7et>O8QEOH)UOYA4p8nYG&L`#wyvmuR3TDuKI<^+h z^P3j*+dNu#oBsWo*SZCIE&9e8Ml#e_9!@Al$YxoNYz`n8=HMYEYcjQus3N7Nq3VO# zX$w*cb4DwJxTggbWX!N8_ePqwdveDWLYKmvvU#dTx#2}*j(Wc{^#?UZ*pvK? zF$Cu?o7Byap*`TL#ZVV>FfO*MP1bx-wfn>g8erJZ9X94<`;-yk2A9EO#kN<>*Iue1 zEvnv{9t~J2Jg!XtNqd%LX$)VJoaI@eu_c@0%j}!fN+FcB&LKl<&FP$(XD_vPOl@R)WV5T>uKcik^NY_7KAJa<21u}oGQrO`@CA!nQ>NSMqLxw z)Q;MoO>xuWAonPt?zJw#fIupXM?EZ9|3wXezCDmv&>nqeuvIWNX0zwoRGZX?T;~+P zC<%v?;Og+y#nzK=PKwKpZR;U+&|)6JE<1rZUsRf$2unUSg81|aHLyB=he)qG!IyL& zEU<7pn5?A090c0qcW~Y*PZ%kp@cfM%4xdm0wa1&7NOxE-RpFj?+Xz?T9vTz94U?I) z^pK0|UTx-TKT1}(W047MQS+1z_dE?CT;B-`TYVpO;RdUgMu(O@%)fyL7^YzFM<=jk-_9ZNwKUIIN2Cp!O8r#MD1MxP;EQrUbalbrMU^Ugn&-EJ-aSh{= zDh1UJUxr{G^ez(Fsbb9b=uNSwXozLPw_>PshAjp5^zFCZG1R2_NpZ>tnJLN(!RX#v zA)((F;O~BZuekzAd^GpY@|FQDs#;^W%iCf`*2%-h~gH zSuA|+xX2CjA5ZKn7zeLcW}__Vzh((A0L>+L-B(`f@oKLu-Er2;g^1MxfFF*6mWq^L zpxogdTW_Qk!VH7=$L!VqdGQWh#iEkhW<(83SxMc+dnU%hHrSZfT6drbv}MbDltCa%P8 zSkUKS?mUiGC)DBn!-5)L0OKlDJHI5BBXRz)cqNIGKWK?{NJk=H^Ao5I_50uoUdykJ zy~sYD;nOYJ!K_PM!!_%Ela~d*t=u<~QX+K;A5I5G{SId!yZN1*+n5d@5t-~+8LFcB z6|GXWic|9DY}d1e7FVf_`fc`i&BU76CRS}CRH5xpF4&wIONN~SdjOjJLBWs6nTdjs zNET$p*&d9HXO{97h^o+|dZi!qmJl& zJ^00*Py`()7m=#_l;Wd8UNM&1ijvP2ZaS`n znvrxs<-(UCk7^zb04DPKKHdv;dj0{E6mIwh5I@6!H zq)SwHt-=WzUXDUDJ=zbc0AQY8s4yQ96z8AV7xoZS{h*X;LNrIM#&#|I9@*kDceKhN zFPENcU;RzBYVof8TU#9BgOjM;k3K2l()!WFUiQ=baKK5oF&#u*ib9X+QZI~_+ZIR_ z3Wo3!oBJB&;E>pZQyQoW1qIkas!A+EB8(hNm;N>VXI~cY;1N18sSp1aWx-HqyPgBA z2K1D9EKB1zIOOLUyf~;W@oTV3JiMab-=m;eZ>_qie`GFM=^9nz5wv!3Y`?EC4se=^ zWpx8N76I;;w1@mM_@~9>WPFp5(pm5c98F{hCo)a$qs#@SCmqfe1Rwe;>H6L(;8WK%7rZW!% zHH3)a(C-5XRtR30|M95)>kxf;REYn7Ii`PnC7@p&#NU6tCg-VOQr9o9>G6vks|=0| znA8Y|47^O_MQrV=hlqpukCPQ!0$2VYCu_(BI}PDWT!Z94Ah|Cm3mKTt#Sht<*9gY| z^Y4`3i6fr!zL0UH(7>A`%r78a5-(Eg*&5XdIE39lE?d$C1Rexp(^pqP8eO~V3(Ktn z1l@+g$>3tb0~*l_!n8tua4v&F{3FqtFa}2lgY-`~XYJ8)*{nU=G2{~V{H_2}z;$;ompN6qw zE-_O6stV=F_q3uLjo3^pwfNo{Jn5!uPY?5}oZSa}w>oC!@62@x1nv>1d6ju!=d*g8 z06i}Rs1%vtJ}HJ!xns;ub5pESjHpa^_Ku%9u*>QTOlVX^yHpvl56>WS$pZ1|%r{ty zk<@=lC|{O@P}-+uT%3NS1ZBs>060MnOlS-+TZzd12G zTDrUY0E5DiVE<7E3{bg5;Hzhe4Fmj0;e`fHJwUJk0ztZCrftpQSp7kS({9#RMRwsi zz98~j?MGLFM}PD3tU;D;M*0++Ps?|-a*81lowD1CdJPpXGj%vz@^%{7{h8-Y){m|T zDt_n`pFmqr?6IIlr;ec)G%no0AgbrWMa&fptN5Wxuw|UcN4S>#nl`boOkyevz>Up0 zx1dM?2nxl>u%A^0Nxupt<6)1{#$f&~iQ_G8#HlVAOA)b`4-f|AUM+TuEEAY3I6*0vt7-;Y(x-i@x?$2i2lyP!R&gTkRX}F=3YmTX!`7`A8|^X z$d*&RaYFP0^b;sMv48r&cN)K9<A;|z_r|V*d(F>-E0(y8X%?fSdbxU6P{oi%f%UAg!Zrnzmw_!7Wl z9iPs7PP_krXaBd>|C=O)_GJ(G2g@vN7$6`@0w5rCUr9nPo_v%*;dl z2pAVJnB^c!_+Ul*T9E88`@VDw;KrmQ*9!kAm9E+DZ|#lwdBJ(|c!{nDYHe-pZeQ)r zm#wO-Kc8>wvLHI~PTu**05<_=+Cw>>NpR1J$--m-$5OY{=}81fD_8ZT_;4Qiz7@yi zBT(E}3I@+T0%V1LGAqtMh+Z;0)8OnbLfy>H;yv+{>xMKtWXKqMW`Vl9y(MjN#=u`Y zE>?PDikbKczZ%x#F%>jM71QyxubZ;6(;aaphBVul|+4! ztNZXi)Ux9}+JdbY<-I{u9VUjL?9=LL;kiwlpqKWn99v}C7QyjZfZy~5+7Mf<&FL?< zX&u(j_pN|o*uMF=YfTN{{2MF-FPD!d*q5fQh6A0Bht8~kHVS;^L1n8wOK)E8yp)-2 z#EWI}g2cQ|g>sfPK$rdbaHr$F^`6lB)lg6v_&TB-&TWrnF~-;BZXMnYeTQmHF;15t z$yuf^6lU;Onn|7T@>cErEBS0a_bD=@SGYL%M^p$&D@3SV8|`+yL>f#;lQ2OHA{m4I zLu6&B9EWfDYU>|)ir~{ja52%L_c*)8INI@NdkT2BiNVq>22bk5p* zm*6P1q=~?b0*q~;$I`8(i=Sl5QZ#YrhcOb?(Kc&RlG5@d#@N7wlSZu% z7l|!}BlBg*G`)a)hQ$$d{ab$^_LQQD4^cG&|rCmwr@^Jxl$7)>y*!q;dVuW ztJ1%GwkH{#R!sYZC-ZKvZLrI@VAbN%dtDKqo6`Y4fRcgz}{N3B|kbsXsCK@*N0@<1Q36In)lX8zH z{{qqOxnH4Ye^Ta()n#C<`{y7)|L>cwQhigc2v8JW-04t zx%81;97`l~baH4;*NmF03k$aoJV)=_<3%1(B&ZC!aA{HZl`NfoSIgG45G(s*odK8y zy6R;UsTNY@sj&Kv^1Skop2$vJL&J)PY>If0ENm&WNR3n@HP7*1!LP=QjF#}?9I*QY zXKtXePFy0x_=cUz!im`YG~-<7f9A))-XijTPLZ@rW+bE+Z+P$wKc#8b?-t zWP1bM`548!|2#hby{{PJeBmw1dIF&Hh2vwK>hTZjSJ$3j1*p#~5F%E15cW;*AY>&< zKIZBW=On5NaSpu&8#Po-qH`Kj{LYO*teci*>99DE6SG?rovq%r;^ej~KPt^x+e0s8?P&KYA_i!*HSZ&Oi{3$QyBnDL2YZhXLGb0i3@+ z^CJF={0@^i4jJ1(+Y0onB}`8cg1ECvglJnxV{KgI4OG}OEa$3Oa8GTL4nZdtIqlQ8 zFi6+8`ZE?mS}DSJS&de0l&Ptkk3# z9^~}C!X-j?lGT|`AeU5d>Ht!wv4n|ZcKai7v`vaV>l-Vg=O1x-!}=D#F`dLmR2LB= z3MaB+TojFDX$(>*^oo*qc-u51x`VN4>f`>lz=i;~;}EEK5Ee(-))QLCpCEsW4>9lN zXmE{tW2dZQ`%i@A0^b?wsM*b$VlKL*hnROd(Ke~~-iy)Sl8&b9%K?j){vgC@*QB`J zeaSx=EO)qQ12%O?kZiNW&;m7w>UR1WPEm1*2kX>*Yzj84n8V%cxU?hwZU?{;?=Aal z_c@JR3lJsWWP6vzZG~W4_!D&JR3uleP=2v>!s2#^Dvv#3{CX{q8{WpQo=0R8LKFt> zSTAg|KUIefTM`VRjsfP_a!rDZY!dkgelS@0CkRgx6W>}8i2p(Wgfv-T3Ne#b0ML|4 zH7=1Zks~5PD1Gf=Us4XV4V!)!Go#v=iS%JxzA@!W*=PuAq9I0?w`K|KCtfn5{v}$P zpn0a)Lu405kBrr5PP?YWH6GHo@W(XXJM>%s&D75efji_)&I}l};8aL1*~7b7x_@Bq z{o#?!pK!Eyw6bynqu`5ULO0wz0rMRF7pqT@`NF%AV>GNqrJ0k*Sn9%$sQgF7E{|8H#LX>;*^5SRcKscn9Z zzzjnLAC7W_lWv#8u~YIRt4FN!doFO~p&dyIo<=}{1j--E&)gP2ib zXtKpk^SY@;%;@L!@nIgk!FQobp7VEOqZuz9znvZ1+t}H1y&nU6HPmW(i*9)`ftj&w zH)P({)z~stt9;RIQxuNl?A=ohx7E*%umz64V>_bFB_5@B|AzR#?qQsTC6x>eqEsjP z4Qkp=V9^4MQ7I&ORD*(htuGo(R6KCowQi-eprMNVb(2I6OI1V`a&|lVYOcMK!Al22 zzzd83c~6`=ac!7NpM(ZDS#@zX!jaOeX`vDqFfox;#?dk7 zN2N-@7yKb_uY(szqt#*@<6VZBs;6kT3yr`W(kDQ5)!)7o-N@ifAyAL!pW>ctiF!QZ5esOQHmYJGZQaHk&x!EkTpaYw?eHNOF)1W$QQ(WC!&sIvl`(;Ym# zWKAq*Mg!^TGez=9(6AnS8CbL^Y&2%-#A0k6qZ#_rxRWzB43_g zUO$-1puMC?*Pzm^6M}x#+$t>6;<;8B8?FNuyM1j5(ye0%$`X=a5bNRjWTKId(Wb{o z<34RAo5TF+@NyTE66_Xo~PyV`a9X*9)$o; zKTfUM;@2n8#^{W~f3PO~x^t5v`HSHr=^GAR>Tz@w-*c~ifO&q5IwD+|ZK%Kix_VL* zN>GmvHzEr^(nC4pz*42aU8Mj)g}2nHzX%MRuiVd~iE@)rl<;5fu}+wQtv zRwY4phFol>C@ViUy2z08+a)aEyEW;7C~1jA3ETHO4{rtb(1VpZyhlXRt0<}~Hdhq4 zG0X_f3BurKEH+}!k%>ji!;}f9n5%YLHaA7KZb|dR{#H(jI!Ymea=&{R;~rF_XF3Mc zs7punWM2$9`}#6D$z3|BUGn-785LeQ_zM>}YE>TP3fAo)G94+@2~`6CWiDq}%i2jh zp=V54hw<1br-&`@Hmur{tem#;>_kzsaY8SicZ8LoyiLH5e20PUAi)ms$0Su{7`MMY zUN&8)K@f|kIj6Kar!=ZJxq8sG>u$~Fgkaf;r~>#XUj?45bOEyjFB~_15zX>PGm=n| z9U6f4IX(^QCwks{jLuGg43eNClAt)dFQa>)wJrCo^ z6Ov%^QUNssA)nk^?mYV>r(SZ?kNK@Cyi+Eua(Na4VjVIl1sDSwx%6&G*g%B&9S6DY zD;3D=Tv9tQW7$wz)$)*=J zzb5iwnNRS3lpO5$9Pf{;i}t7m&qyW8g=zQb`lNMe%1DZvF49zsT3 zN2vT&C1pa1n9y5!c}w=e16Cz%inxP~dOTJaWUtosJuMF$z%~^Q_5-@99zxC;HEq{S zCtBVFktq^U2M&AJ*HlN)rz7pKQz3@O1bnA&JWRn~>0}}PVj<20t)o-Q+mbxBRgLt9 z4>!wMq^uW|ZCNNp7wtD`Z?@Z44E#=2Fj#RQ`|FjYh0Qyz&j+T)fS^VY?95B!H`pEQ zcj4eLk)yjnfbo{$4aY*9h{FNP0^z}5)@6~SAHMxG&-Wpw%QAO(_VwhKi{ba1BSMn} zO6YyLPB4D4T_u!f5h|Vf92GqUyq_a%P~`$?e(8(UTho znCTc|+0v%5hzYPB?j{2PDp9J2K3P-?;IrzQheXmriuXKfJj4j7`Bl%Pg%@=zFH5VH zj*LR?fGo6-j?P*LXd}pWZ$yVWJH$1+79y9XLDRLiPOI!z?YH`Cq?&CZZ(xK@G87{` z53eev`RF9nwO46SxB6aP#zTZ_?%b5$byu!updA0R^|jmNZC(n*qpl<`ai(-ZI5^W8 zgngBOTN?7d#rvaH|H=zQGZCixttqKuwrJEeU^-j_jR9i59rB87)Jq&`Q(UP7`*1g) z9^vZGjx!VOo7Zk&NLc@tI&wKhbfTeC;fqMwzm0z!BlMkIEtD>Z5X0<<=aj%@uqWdmUtV9_%qxrRFP0$`5Y zFj)ltr;ef7ow~`-A5HHgYZ9a>Hh;V3~Km$Q4>SF2W= zNNE}mxx1#|`ApgTB>WKH6OalXer|{$sftJSr9O}z3;*RU%vd%RE<>r}7a!WVMdN-g zPyN>uJt;4!mi%Z_2+ArBx2PKMSD?Bd+e9#TVYVhL=}>s5doZ^K^%$xUd#ObYG2O9y zHb0Y9w&b(sD?TA7w!yVx!VljF(fx`-#~J#ejdSlenQxj!HtVJ+slPbtf%M8}9o*D& z1H({5=UiLsl8)W(&Ti0w%guK)(*8GKRX*u7bW3U!8xNXpwr=Y;v1Y!J{I!2I? zMU*rHy&&Ta&-3r@$oRw$jLAAdfSzmgd>w@8gd0=0369#yMdlbn-^!{(!5j}yPPD?? zxk-?>!p!{P4|o2Wf&xB3POqi=#_nW}!3vKw#y+<4_1i^-Z5(H&B-hRT;vZnt1B!J| zp!{6OLD&5!dxYnoc#(8Xx05_w8r>0?idJ`TZ_%$SOs@~qk;D`A44!Vs%D3b!<($Zl z7Y}^!*PoDZZB2W1#N) z?Xwj?@Lx6)q@IkJpc-max82FDS+5(%bFSUlaLAm0f6dk$1aezNuRCdXMiEs`KhuU9 zJ3EVU-uD`fEl$!Q>p-4%d(stsj;DtswqbRjkG;VE6JGh>&!7MI!zkR5dYzWRS8@0Y zTHuu+3N1j_&qV#mBI<8fkmu zM3LgEUo7VkJ)=8 zGyr>`^*RIzP}eUEag+E&4gN{cBli{?3_vvD{7~$E`xW@?39{Ae75YTZnc!KPoPMY&`53dPQ=wj^4~VwlXKR zdFDj>;f;d8EP=9>*@*?kY&FEqzKmO4(~LE1Gkm4j{rUOtCa3G%P(kq5SEL65j0J1a zxrPRfU3Z3!bzXsq=Vj5O0Bt@_A-T(#;3pZj6kO%uFB;`WkFeI%CV?!u+^9sDQ$VBr zuzJOW4izD;!(~>MhrUMfUu9Fc4mr#PFXMf83EFIEwb@s5+{kjngK7C;r4lw)YuWm0 z6jsfPDC)h;i;%__k4T8k%LSHJ{Y7l$p_gH0!o(72@?mrX-5Fqv6j~(NNn00x?B>KM z&yMzFyF8q(x-fURT&=S(j3@BC1mH0!!JJw9w|$m&;|S4$Z1(l9q*RTUR4HjI%wqwkDuY|J(TWc*ed#1+ux9X5JIe)wH4< zpWdRy6@(evDd9?B%ec|hZGbsKTddGPCl%B(iyK*h40svDUT|bR868P*g zTTuAkOZ)T&q|ITij0CAGR0XZg^~}mdEAGfL6|#&ykI?5>`&V;{-YTG0tHa=I5rJxIuakzm*rhdBL(6*{(yY&{=7 zXlJy9Jw=KMe@=9JWLAaiDq%5K=Ga2QAX|(}RW!8A;sE|lQ1rYvI5UcF^T&+!0RZd5s|s2h^5GP5_airnrmw{78O`YYapq>OY5S3*{>e4 z?j$Fs^ld*N%j2%NW=TV$VJ*vUno1~|n)8@i!C;6|97mBBmvjFst`P27M^EYb{a#+a zyPaqx+YaE3%Pf&(m#B*w^-xg`)!sF+l`^^G(8^fJxHMGHnRBQn`Y_?!b7q}U47BD= zku52MbW|!WAFuVa)4fu*gc_2T_K}ZAJW8q=*H^L9rny+DEIPrNpUsgS*60t3f3C|6 zpU=_kfj4BOHFqHrF|3<|>y3@iv0k93KWu|NQwFfyG!SZWt7~;%!szcjE%&r(y6b?Y zG_cp2JLZB@tt|n5?{~byT}24D_th-HNkv4yE-1{p|_(6wn3dKIa=xbuHZSB6 zzTTvLNmr|f&)ul;y7OTuQGu3QkK9VqOu_I+(PHn7#hg|Q$A>Dn-Ji*HVb|JnrQjU? z9iGYHxii>=W90CW?9H?H+idZK>&cx+l^`L4!3sxvSuPpI`+eRA8dVFn7kZw zyr!ys6U1z_3&CcM)sUU(r!svUR=n`#QWxd?NSSS+!&Sq_vaXt6>XNuJDnChAmByL& zYG1%yHH@clC1JEXO$yE*7+R6RN^?%t`N~YUius(@&7=m;#;D@^NHoQY&d~-13S2-* zRIP#Y8L_8Th2a~qTb32|8es!^@{I%)fc)_4f*9xg{OU6Eky*Ory2TvvaT^IlXu+m# zTsPh*lZ0oY9oow@c(aJ-YED1xS0MM64{+BX@{c#`9?p)w@b4d><9+B~G-#n-QRrK# zG|0vgHxzF`hRZzy>?E}1DHr%k7kog5E73vBw>xtj*4PYfUg%ek;)m;YrgfI!7=i7b z2<5&j0dQrqBuuSP*Myw)L#&CidP|LNtc~(vB>7`B{AJY2rmviQK*BCXcg546y9$lF ze#AU+NJ^uiR?!l}{lY(J==Cs2`7vCReN$ehNBmrIzg`#8fARcEyh9Q2LtF;{41XTZ zi}VGoN(oE$nal7Uwr$J~!>Gpwm4s7E$C5fe$|PV#RE3-RC^6Khfwv%AzJWvZBhuHB zeMe>qf$O3*sZl%ygJP9CisF_ zd~$*B8VeVWXGeO&Ripo8+@S}Q3lNkZy&mWJ{88a-&i4ho%R}OJZg_*kQZOliV{8gz zD$v4H6B2`h(UT1TqB7Xk>k~kN>0II{4HShueB z9?z=}-}coq;P>q>9*}JF%-hGR4{k|5&5&G6Vhv94YS^Hk5s_TiSpj-A4wZxA@EHmw zC-W_^cdO$YHun5um0o<21~5_Wha@ z>kr~G4<-{%p+1gRRr>+znc%GhpQivj^0f+OX!A?OCti9i=nY214t+fLmi_*~H-W%l zwL??vO@%za8i%aw-(j3kfulY8l`?-(Hf^u8bB|cH4Tc1o7r;bn_$RwX6m& zY!Y@duT1^DNXu@Q_OedgLj^&)yP~+mrZrv`Q_(vepV+1~Gf6q>wf#sL zH9$X`d!NwuPOwPY@z<9$`vgu=s&|dlrf-git%1i+TtqyJjh9AqG(n={Go^!-gWm8n zW4LDq=b`5&($9HdThmVg&zgnCbjj{Txdo>(en4;cRZFgC(8icKWPxK7ew0WZrjW3!)6$R{Ubq*6 z9^2fEDxdG*?3t~HmO^7e4-;mO(-xN_U**C=@+vQjU;bEqbB27i@`T2>4o!My6UvEf z@8&p+^~t1{Pk&pe?gUb;#P3gQyxwrA2i5d%yw#3o)&Q}HYL-9x#oaF7TOL=*y2EM>oSucf15)hfc)ZXMjMC-Vdjr zcLt+Yd)5z){r-;yw=u;3)=WN9r~E&+@&AozTcmqc@Z>1(%K0LPefp||ObG1#LyZAM zVirVbRiWZ)g+%&CLO83P_uH2QGCYt~9UcSd{0heepdq6np&>gJ0q++_BvZA3=)G3K z5fP17bPLZY!1wdTxGf*+)6A%K-=V_isz}2lOx3YGp^B}(^-5vOa{Hw(MO28M?xm(j=7$9@;f=BFnp;Lk8*XK)|S)#}0V z+*oh~*xjUAO%0?rwRlKp%fUi|Ml{ANamm?5oGP2sz_5o2l0@`1*CR4=Qk^%OW)=ll z<1Rm|s09RUD=N0cApD9*87%;X`uW+^FFcBp$*EvH(17R`QBW@Ih3OQ}5K^k+jspg- zl|}LjQ_;_Yc!VKtZ4a+P+(wkuA*)YqH#}kosP<%tF*{Eni4sJ36}qekf@!J@p?Fkp zB8M;~EY>3P$8?r2N4RRBx!EOSo?DN&F-&vCfR{(*!!1@RbSgq}s-BCO3h^LGH=wT_ z&|pcTdXB0R-t0}_4JfFfwP%Ta<|4%FlgiCCT00OJ;*xVR(okl>&@LS%FQJLD>*1yW z{70qzG>HUzVH7emqENdJ>VfH45u!V7~EoHbCa>=XWN9{}l7*#R6om_>E zN;NxHRzus8l^&uogomjhc>0`r6p|ljv=sidLy=G zCmMI0m2dZ|vSSgZ(%q&py%TSMwEFjSFzrN8hu%DCDT8Aoaizk5Qd)cN1_?aHFzjMv z!zf#GAu^LTMRp`z`e<1^>`~vNNB(!-MZVL!<8B;E_g$p;K+&L&Any&gFY>l-sb}!> zazOa)RzspGu3P{Xbl#3k&I`g%k5l#X-m~d^p;B^C1~vTyLg`L;GHEnGLVFPuhnkE$ z$EIR%2sinNenTFyB%*MHR3O3DOfK99QV9&r_GKNMLb7}?GVu3D629Mh;yYcY^Xd5d z^2(5^EBDw&?v7lVWnfzS@+wUW`pD76@|1F6n%4~MOYXBuC`YOn-IORR?4WjmgdjdK z<g^7wK48KhT+p&kh%0`=NNIWmjp)FDu$K4W>%W=0l4@Zy!;)&OOuajms zfM)_a|CY4*7T#vBU*;MW;g-Vci3aNlXO6>o7yT(%c~?3R^6$u_Q1>C#oG6WxvySK; z0EYh9%1O1-M4}fUsrAV$_L-ly+9v?VbWo+Ya|#KO*OP{Qe@n`s?Gwkq$nccGc2MjP z)vN+ABKL-A)qAlz4BLs7K!-UQwvzfw zcUD9F&=?bc{`}{S^}omO|2n!MLH;?JTUoP-(Lnz_q^syi2FSh+X$C~#^{=l(I^O_= z04PEx@Rk18K@8WLOGV8F3Gwfkc6F3)u>W-ZwNU86{~aDN4$MCP;;=t~0;(z^U;#;F zxbT4=#E>Yhq)o&S;QtY)Z6QX5K>TMCYRGf9A77br9z6RnyLzZ%|=l*B0zwGb<|I@`BBewX@wwXFm9uB|OUHqSQdp`K^fio5;P(UPN z0T>_=12sbHQ8Ls0{|poohWtNseE){h1rGDiW?DeKmVv?ikd*)I1ERG@1~Lg8_Mboe zqwG@afDQ&+3V<~clmK&nmt7o~;JI7fRaX!s&eM`g5zG-tSXxEo=%noym)3d{d)vC? zC+>CMlaf==Xt{ULH=2QJ*%E^}@W_>Cf2U2Iu3z2n@Y{q&=$lm>GQ~^Uq6L}+=ljWO zhB+1c-Uq2~R53b{)MRV#Mo~H^RV;$tD48?g4ntrj)d4QFAvzSpy}*x(xMuk5de$sg z4DImth{H`hqrP()GHlMiT>vDSl@zD#YmTE+8TuufP;;L5Csjd530jFT+WF#7aN@o1 z*W3BB&iBz{?aYWVp<}i#%LK$jubz%& zqwnEw$jb)iI|jPj3?|Cw^k>}P-v)!jB?I4w$@C~Kq9;}IZAVI31VR3KuUc(=H*SkK zdkZrEWZ=e$IT&6F>XVpeQJ>?w{y}aR)e-?b;M1z=n%s&m`AGd6F)VjYtP{!o{XO1? zJSC$3F!w;MEmr$npFp>~zn?&PVYb{C`5&|YS9xDZ`;RZ{_-|<8AGMwb=^!ORgMiSW z08w=jFn~Q-yg02f3Xql1|C+KK{c#}H*T5}MpvWmC29QJ_61Fu)i&_fwKPE})g;MrE z2EFab^&RYAQ}JPLl==kD`l1K}F4X2#Pz3*!D(Z)bh(cgUNYXN@^)TmSKBGV&o+=Pp zRdzeh+qI*hQOWY(vI&9ng(R@QfC!~^$eyXd^1P6*OkQrf{XJiKd`)ED0#PszH8oh} zmG|BZtUZ)vr)E^yjC62FYHDC1Dz9ph{(oe>b9ANO(k8|Rb9Ol?#ip|r~(r2!_+|l|M97$AW09c#L^Wm_C>!&8;s}KLy%qWDk z>gnI;^uEGKOXn@nEb(g}Gv?J6Q*Hg~P01ZD{8sje_#`K;6*T zzYyE|)I~E6opE&PZItW-CJ_Boi!rFNhkceb)(s_;gIfqYX*>FB@QIR9h7rxjRa9s#SgSfawAdl`IvQ+}D5SW@J{V64t$}Z_R2Hfz(_0Mp zic$!5`5mT=sU*lOeQAxSkMMmHB?BP7xbWFTfguoSPglMY_8pV{uvx#7m%=VP@^TAs z50XqekgW@C?41(16Hvxe0X%uD`O?xnl@}?&^HGf$yte1o*IkEh1r(L&PnPg6qXjxn zuys^{h14kqt8^!~T1++@^+70*#M&}*OJafp!bJ`9dNzJEw66LV)hB=x9A4g;GRxpj zk5|=q2{Ny+&538iSEj&F%s<}!+i!WE$(z1S082=qSMMG{t38cKWy1)l!K9F-V`M3bcO6e8tCaZVqxn>KE}`&sJ@0nzWwkHwwMvF-&^CzA~}M2o=)3 zO+S~fzB+z?k(+J3ud@GWj9~NEoVTrq9l^2zNO78WjyB>D8=TyfXg!j_@-EqkKRi#i zqcw!SBLWY--G8*hr2m~nq38i^Rp^IW=uz4q*ZTO=gY%5e4-D)Qk=<$wS0tEn)mY$Z zssFpdgv1h-yToQg;JG#anmB2P4`LnRm;S7_8|dxcW3OKpi_3IhIw(WE z0EHB`n02YiW<1l|ukeEAw}Q)pX0%_V$RP(TCo#OnVwr5^`D@MK?ky9IG9wFnUCi=f zbpBw-R}Ud3B<{>R5rpz{yh`GmYMZMc);{pGL)T56 z>U`1Fy5e#iEos6e+FTd@IMq0hnQf_avh`X#E7wqTH*=RzmCM>y+#1JFHaB(QR0(V-qE!=; z1eW)yDfq2DGp{iOyuZ0cMkH{hPOj4KRvo9V7V3U;1Uv*wq99KFhl@+IwkjCOfD&)g zcx=}w^`(RNR?`f?~=J-mMN3r|HKSUH|KI zwn>5C^R-yGf{jKncASr{8}*4bK|0tqBXERxF0w{7ZY(F5upd9cfYJL%so>V^T_87B zY77X`CB^KJTX!!uMq@LJS$#VoOeuspW;t?g(OJ2oRzGLje{BIr>TB}2)?QpdEU>z|? zP2f)?UZM})x!l@NqAqYsY}@{bLB$Zzh3Y@nhjy@jf({2Jq*(mmWvw8dfPTZ_vPq=k zWl2QOY-f3tH|Sc7KxA}q2Vu^=&W{^|KpOnrNiDARj0~j!7qv9`HVq5~i;EzqY1kpKh&XeZPPUFlNHO^$lI>S6ZAcW)Rvz_@MQNCls$jMiA}&|fCfwI$EHRx>rgP=wc*gMG zpJ7rWe7PBm5{Ilz0S;+eW;bcl`7AskdxYLtu}U*(s8R!J>wX>qT|?X{#D0k^vYJl| z`FJjhp59KT7fn$}}Iz_%UuKNjM;oVCDp*^tIfVKpAvX}k!zaOpCJY0s1|V3yI5@xILV`9Yn4J(hXsqSnxhNXwb}YK_psv&gYskL zx%^4kMtE;XIm@;E(K|iKE>#UNi{l+5&Sf04<^ASCO`0BVrCgmvdUsyiW65yS*z5!yNdcmi*sqFtmZ`LoDZ=d zUj&@%6|Y_A<}6u`Hyy;zvgg(e{PVL+5TbEaIrXcYT&uUCGJkJQO^2k*K zvW(EhNw_Y!ROs9+@9F#oF6eWG3dHH!fX(%ZsXxqf-VUY3>5<9ljnA&M zhKH(^oe$_Kxed(lRk6d#4GkiCOB>a&G~A4*7~Wy1WxmuW$Er!YEk}oblMcM;+HP&$ z7DRIH!u`bFOtZYxAs5w!`~>@NM)26SfzBvrx$*X5ZnqQk&!5+c7R&r&W|Eq7-9jkA zgbD-^08n&kKe6x@#&jhYQVbvk8E!<44x4$1>c2M^rn=~)hW>+UV^1ShOD5S^EQa$x znH2DlVRX#ge(JDfbsKE}+0MLAAmhBl~nv0ncTyTk0euEvLjoM9$ zA-YbzbGVlb!FJ8bcAOP&g)*5JX@q(jOn{5I7eoDpje^RVOmgXm$AuX@;`mL<{X255 zKP;)bp`I7YM3&a92SR6!Wqb#PV#u^nO7u#CSeagIDOt2BL)k{D-r8lX-0ZCNm4_&u zfOsbas#e{qsSe|wf~iz{)B?$D`U9@4h8TMv(J4=?9_j_&Kc@wcmW!)_?f2@rcw5PM zKj-$6PwE(X*@{otkGAddQ3b2|EiYM|%+Q{AZnM=~kZt575NG@K;Dl`33nEmfgrU!6 zsSfCf^h0-vuI~}0_D1<*3vG;4u8$cD09?AeCu@%v=OH~6JLP-DD>HmCXTx>L4N%S} zR^A>qpN?B3o>8p+w*uG4rM3Dv+$Gnx{&od+Hs-J<+Nlu34r{2Nf6i=GgStVgiCFx3 ze~#3~5LgLsvVCzFl71+Kd97quIXlo|g|hkw@i<60AQLOKGRGdh7-8U*Q4Q&Y0@zgJ zA4xXHp-kxS@CQ02-|O1Yc`n zU(@g49mKayKBRY)wmUyYzd63~1V9?`QT{S;bPW1@_I+Ld*gpi?eIsD>3FF`sqCzMD z5r6UX4AaILAiyL0itROUa`(gCr(oO+o<-F-@K!^G5W})0Ni;Fc_=&Z zZpC&lE390OtiJpvKG#rn_ZRZ&1P#1Se2Y78)IKt-hat>e;p2*dAio}s;+_plbaz)O zLXAzc>g1`UdXn947|~aiOxzGFl-q#`D7chYg8ERo0*D?@D72+++8;l!1dA;FL2{ffg-}tbm>Z%X4d+Hs0AItP zZl_L|(>qQ1(nfm|Pm_2Xe2{c|KyTGN@r9R~9fk|NqnV70x&ry)nV&-hEKD$47#o1W zJ(J&bL3w{sXFfwbE00425G9ReY1)7>)Dm*^0V5K*O-Ad0l>96*bDT<9I-<##~ z#5W1IRf43Vu)g^Lq`a!70%Z;ifjp>&cobqt4fE${L^DT-**q;Wu4F5jwB-u;32_@x zk$8Ve+oG|n4PSPD)m6RIE!wA8ruknRW>k!&1q=x&E7s=jUNr<9Q1+p#(!FG%o*ki{ z6*E6bQyHN=n;dxIWgcpRjuYugtaX+tx5ha<5^4OQ(q8BTke_Op{$vV3L%vqiz<=qw z#nM+(KO7d4Y}{Wiy^b{txKKXg$MpeKTYMwC{%zA;`*g|rV#2q@i$E-&!w(K<<;jI= zpEx>@E#uP0W4M33?R7E2=+@G~@6M&cUy3!;=uUC2IQgpx!-rmL;vTK~$4p>)#7?<@ z7(v6|b*E(xAl4`c@%f94pH#lGihLJ)KeCAe{el81vrgSPPt&?jst!#p@~lo_-LJX; zCVN-R6=6f_m}ws)H&j0nWWH|ab1 z%{DL+tZak_&9k+faotLXz{GSWs>vBT{y@mQA>j^s_8`2?qi;!-~{O zo?s94-wUn^5xM@eK!BLQ`rlab7?Xr3>^CA&29{d;8-_h~EsPDPMehKE@jokVs}VuK zk^Y4cO4#EBt9;`YrqDlrP$n|MVW%?G@gM_|Rdm!)#4&xr+5RFD5k){#6{E545cgd{ zDztl=f0(B40pSRy2aU;)?N7;R$)NBkNoKH-Yi~9n6{EFSnac*S zTw)zLL;4?foRiTE=t{^;ZY6WdiFXN%05Z=YA&BLOVwva`CGSjlE`R-Mns0z{|78}HGx-}3}!9MRE>sN*CSAw_5zEVvY~ zBwoPBod0}J7cUU-VPQiNL~#wx&({D5+@Vhv^ww#HX*Cea7HUkSsH);DCwkzYyMhw3 zu;jtNfzVU&&#BoL<1p)l@GDZGDo;QZ9xPaMVQV}Xdw4Uj5!bpgADxIUx7kf-DUR$! zhhLy`ae)e%)3r& zDI|HvB~s^{tYi+_9@)t!zmUk5>&w9$w>jd1X}b2qfI5nm|3TEM6k-g0kwVj|M2`o< z8q+`v&i0_SiTTMn?V#oY`z^_;`V*vc56eBGF4la=YKb*ORPMpMA#RV9+V2iAfFx2Y zoop2ptj7|;eqNvuhZIHm;7ll~3peiQ7@`(mz3W1x%DM)*IfPc+26Z?Y_pw5;WNt0q zi_{sKv}$pJI0b#~Yoa@%a)~AsHJWA8%EWIg75P?SXLYJEZDP4*T$le%0q4xt7H5mH z+K#Uuxsg7VlR<+P9TkBWx5^kMBTno4CEE8U1tvv8xK1{ek~gM_j#-1eXq}9`$kRvk z`~$y2;L3i)rpAz#EhecP?F3tDiTkdQHYjq+MDtUYB*xiWOhkP5|3L~BO2GF*Z z4*1#DA%)D^@`$XOlo{(0s7ye;YEh|St;?Y-%%C^xW@Jz)0X*k|Yin<_Fe>dLWlyip z9AZTkJAW$rQX->eDm8vLR~>ae<|$@NX;Y=#J)lv#f4JXjA$bpQYO%-@UMSSjLGBD% zTP#0mS&y-1{q(?aSfuiTY7Vf;d&7FN>pKRR!9O86=4(`ql6%t^ zxI@8N2@T}u398`@Q2lC%ZLmllG~?6MnXkQFiV;Th%f&R~T~nXN-h4$`-NPfQCeFD0 z0O`oh>uC98H@5;D_q;&nJ6~~4@st>ZDc}qtVOL0WEM|`#(M-*nVf0Mx**rpOhU-4i z>HDQn73}d*CeC=s1jXeapfa~hO;{A@k_(|LtX~t``k`|%6zqi!7YF{#3ZFjFJi33P zbvbK&M=hNANRWt};2Op!Dy%kg6tFT{PfsalUB~<$$g~7-I8)4A>RSL0un}Zs9$%=b zHlW*BLtme$w;+%Qw}LUoGg$u3VB55xlOK2Trc}qXTK|kQ?y@cPuyfBFsudek$}#hX zx)G5zAVd0=kVPHEJXQ)Bd;eWMj3^pNgtGYJyWYt6Gt&vV#w15rYMBPiqko2L)%N-r%2VqSQpm zQl+g9$tmM;38t20N@;3NJ0Q>UyV-PAkx1;LJNw-1q5F4dA0WMGk5FSpV-AqPV}W&J zt-YHTmAl?-cUdN}o;ANA-cx=~E4dN^BWa>IBrnn_e9Q)o?A9R4rqsC7!Epi77Rsk$ zqh<(jzF42@Wl&_jl9j-5f{WV!<@NF5vRrUx~OcX`Dhr9AWeXI9v~t0<-P0JQ;PftG468JD#u{U?ir;9~=}K z+X%tA{NZ^)u0U~}K~@>KG>qn`C4Y-AV>MX9`em)(ko22i7`?gYA2im`o zsn{UQ5&6|ZaeaEd!_hO-pX?9NsQlcRq2z6`l`H^GOmY+;8w)v!rnAJLR9DZU&@tJ? zWMK-Ah6faREBF_xko5m z+~u`7h+D;08^zilON@*8IRBGN+Ev-x0E=z4Bhr@9`ICu0N~DD8(tB!ziOZ}%%HA}v zRbGcu1riFtX0g|hDA)oRtj!7OzYT|Xls#p+`GIiO8cVy{bHp?>BbAMMsO?T0BIc7p z7P6?U<)SDPGBzFmDsZGa*srvO+1f9?xJ>%+M`|yi`K2y`IF*rwU%k?MIt)>{_a{xf z9xT+yASpCcoNR3Qo-)?rf!bExXH%!IP#Y)6Y`>0f#Tsf8rQ(UgW^Y%UyB%GP??55_ za^oV{g;#%0N)>cYvJ2R>v}xciOa!_C+}zf%Y8zSY?LR$!|1sjXY5D&juzlZAgtO-R z8&;6ix)J!@){BBPB_ZPg8)G1#-yWszOQy2+AN& zQU2#V0v`4=>uAICMwv!n#Uuq`|(C8X+V`rp4EcbmT6Z!ZVbej}~)*YfES ziB4t;C(3w~7k@0%#Y5Yi?V_`J^4*G*=4@*}%(sOhJHzDm%cC628;$`F&q2uSJkzL2 zNvaN|!j-ZrvRK7$;Ywmxx&k=+^%RE0O7#O>`2uF&_==qV-4&eAceFbqeWXqkxXPX0 zH&V#UjQ^R97DI+H2xWbK|Cz@Q_@-0f*LKior?#KM1LgZV{u%?ZGhE1ym1Iq1DOeD|E zRcqmhSb=%0hRChG^$!HMZGf9jR3Bn}baQX6l-c86v+dIIf74K^eZo4sGOOLhc@#Ix ztz$n7*Ies z{vdsKPjlR8R2+0c3Ex*dUTLKJF7?(m;)OsJ^YyBR#>!(Q^$tMhzth9y*Omer9p8eq zO+LtaF@fD#-R+F>vg9GNA-19Prp*1vX2cy{(6k&`C0l| zOuVFthX4MZu8tg`@g=5CiXGm0$3wM5r|&@L)t7_mpBMOewx zkSkUPv3cr$ho1)c2%LN}(2E;TWIHh6%)~V5=~vy?JfhJLG85myc-Q-v9scLE|F<1> z`J>c~8 z;$O4#sLGKM8WfnuU#3@_YPFoI%K5&3-lFw_?PJMex~?KC+LlnyG0caM1{*mr^IB*H zN9;jL9|kY11RONwLK7YHGWE<8k<&GIwEYT(@Te9EQFt*Dx1lZc*mopBkpx9L=DjE{tKj+DzFh+Qj5+3^yKU1W01xY%DQi_auTW zOeK6t#iP&E-fYQ^)+!*WBHFP!E`(R#{p&i0rR&?8XNf5T-=q{A1Nta36WhhrURMbN zbZ!PY8Y-4<;NFefRJp1!++0A(pv;y7({he8Q1_L%V6Qx4fD^up8N=+Xn6)1w3i_7# z3H7sm1hG1JBK#L_DS)9YhD_2b^ROQf)Ou|gpFcvL6v8~{=5la}D%=?y!!^gT9NfSm zCn=4V-XRAfRfQ#{DQ_jXWH;x4l#04FE)uCJ9k?3?_f{^S(HzbYUG9R_i(=qm}Zk?dI5MKgGnj7=8wLFgu3(%gaja}j#WkeKVu}(&A zp`NmyV6|R__)^n()4N;0JL6JFtt`G%#{t@Kl>uyOzBjZlue?tjgXli1FjnV>!hOY0 z!K;Q#uPDQ`_HXoSD&8dBI?9#z0qSPK?EEdv-|Zf6u(Mv(hr>^}k)A~Zey&@hM+X%K zM-WmxpKEL(&g*!-vGQ5s6a>84K451O{yYqUd(LlTM-$wF_oN2;4xSNy22UkzKnoL^ z#gD9l4ndi>m_Y`jWv7tO{eb`1H4w%p6d`3)5X=`I2B6uVH0)|d=r zh;WI`R$u!S)l0Ke&H!??0q854YkA&U>z?JV3aHrLV4a7F9AUQ;kuNDFtb5#M!M>|5(H z6)W&!DG7nJK6m@IR;l*_?yY=yw>f|6W3py0eFfk{V0PgPOWLN51#Z?;R(#C*HrqK_6u5 zj?_0#T27D`GS!!!2OW^Dyr#IOimEe$jyX<75)y~Z&61X~DivS>$r|0ZFca7ALKls~ zG!=N>xUwZ!L&-Gpo7NS(Nr5N;rH%I2zRSjz-W^0nk+m##Tsm3Cvx zag-)OG{d|S|Bb0wBK!W3gn9)ZSPyq5v(K464EVHy<`z+#bwGUj`v#g3(J}0y}4=B z{vxb>#NG<;O1hO=a1-iAp@pyZqipM~)94qNb;ww>Fz+W`h;Hy+=PuBN^~d$U?XnX|v&Lv*yy;CgV?y(b;o*aU%D zXJ}{Kb-VF0wh91xuw1J9D8(KbkH>%Kp4h4nPoe;`Wk0<*Y|MLz?;0X{P{ih-LKubi zbXa3&-J&d`_F#9i;S{7;##Z?D*o|F1f4su~xUbCSsTOyjlc^!9(NFBrpZIgRJLv;n zm>q~HFr^6XoGTr#jNRyjghIG3Q&zS$^LYIz$7}qG5TCC4H__nHQG(3GnE9%+H<>OB zK3f?eLLOWmCJSMmLUfC?vTUrJB}T0Lcxia~^SJH&s14s<_5y0v`{mM>w zDWXyngF9V;mbfaJB;O=sNO@7g5u;IdPBdnlOGg?EjAmD9^%ub-GO=}9!eOj^=taS; zr<>|=hk&2ByVgb5@xWhpmB(A8%^E5@1OXhtqigZV8kZ=IsQy;SW(qMrhST4xI2PBs zl8gQNE$PCG_1OH`H$E5(XYbOHV@Ksy9P}lJ1HW01Q|59Ra;s4L*n*!6A5WvpF6|ki zX#>P2uRnI_9a4%zSbu#wEKE%vH|Z!6Notd&yc~b`K`@kI^AbOx6eW>vcZ>r-iJ26U zafg3>=V}m;{}UnpcSLH(kg!S39z!$$RU^2;Zvh@h>;%I8*N1oC(FW=J67t`vUW?Zy zOdRBY&{9#%81R1rQ2Glx^WC>jqZ}g@hy~`GrQ6B_or+k>i3(^^w{liiL;IZKaZDWD zpWIiW@K|9=m&BCPiXbT#_z_M5l!0j7J4e_SXpCczjXs)bV8`_p@NvZHm|F2(1GdpXP9r`{=S^x~0eN%4cdn8_?~5 z*6T?F`AmExKLL;_E)hm4mtbK*j*XTfK{lm9Tg8$s0gok}xHh-*Cc$APFyCNpEPrx$ zXTpgHF*3^M6_3dzE}xnV76wj!F#^{HroQd0Nmd1JO5714@g}h(gV=k@*t?NzAZN48 zK}O^Sz6^WNPLQ{27Apjn1+2vhh|@hw(3p@J?wFT}^8<`&7clrNda31kOiMgt;L|Am z*WPWq#vNx)L^-K=D1?hKz$2u$1)2nc`^{qDPcT8fKXaOeWINS?aC(;lA?&M{HT zbSJ9t;7D1lP$`t`(Ab37Y887WhFZvwcPt`y_|-W&Y~mT|ZHG@4tAr8-uJx}J(XLo# zh~$my=m52wJu=O!A)U{Nwl{IbS_ewyQ5*E~i9mm_ZYW37{eOu55+$cRO#pHQV`H8L zmPU}st00&Kb$%_;xYS&FMHUpT37i_b)v#)IJ-OGKg$q|Mgy`}5_*mb&EDlgAJpRkC7=}2qjZoNdnj)I0{6uGG zpHZb%D=kzfUY(@rZ=?q9WxTXS`_O`1B>El zdTBZaejgozYXDp+*C<}R-h9(%j%bC=O610I^29^D4ikAKz@dY8h1$v<-)pAm;(@HP zodhriJZv&qM)}5TKki;>cv!ByDg$)CYm}1m)N?Lw14Gm(HG(bPwn{AY^s zFNWGPhK+7$_0ov#MZAu2QOvX-H`nBQ2DdETp zEBU}IRQm8>X2XEGH0Jh{e2GzG$ZcVrSQ|he)&XI-Z$Z67Ib|-AgknvZ1Y6|?x$L0^ z=^c~6Zwh9DU?_wHeNi>B1g_5P+3U2`Xzg`ob0_HegoF7iPg8@Gc$2=AS4nRvP4A4w zO8-M!V$Z%TzK*?xFb_r5P=9gIPR$;|(Gm=^Yrp-c2Cz^dtA4PQ(ZpcUzVWQ0VP)C= z+D6^8aut8hD!x+5TIxm?I_(_&o<{g3r^n5gT_^k~ij1Pg#3T26u5}S8TObBW$#2Gd z;?d4SkuBX$bJ8jHKbk~N0b`gRH?)AW`L| zl@^mZml5$_>H5ertIK(|;cbEQSMS>TGwGn(?{pCr48UQ(tz6j7mJNlC_oY;JYZ{NKODV63c>g zt~QS#w>$+rkr!p!H~uSX_oqBm;4O$jLd_(5XJa6h4v^*T3=m0pdb7mHnPE7MS3r)=Ik*X}NXN}IBM7m%Y(QD`<~Q4HSA=;YQ= zr{-0f9&;IxFfnw7r~8Eb#5cy4 zXOAkZ9V%X+g-7S8$GA zBwa!7LOJVd@lch1c0U1hO3}E;RlPZ>oB}09PiASKv^9b#5QMHU$6bS2?svy!Ng;mW z(nyygP@3JUe3!;G{n5iPx?!b7YExukJgvk)tZ9~I@qBFc>!%rR=$9DQ= zU01FCiCX`CSFT<6jwtMzrprHo%@GB`!s(QewFHK90tc=;ln*O>`Pe3K4IT41ZdGJN z7n`lxRR&XuCI=Au3|Ks`DsmrrNt%?c3^DHFjwim`yT1iJ;+J&IDfpOgs`2`SnlKD5Kex$6UOq-r18QH@U9oKci0IX zjVF#AFQK8cQt|Rhah<+-!8vUO`fVEVR|xr#19hk{V;3OK()tPzzS71k^ZcpY1+LOB z(&9epS1}E2!W!kVs$)FfF?f~A3|Gh{yejG|X06>bQ=KPyUq|dy6I;BwRp&QNopw?G z94OOLx@N+~Rh9q|YkHzLHmz~mz!439njvU?l3OzC=@#$=Q(?M)c@N(65oFalq_SCb zRT=eMng#e`Uj5V_62fPHp0nu9NAbsfQb@(sKl27ro@uZ&SmLqZ`9;&=Ras{aBHSl75?ZkZZZE2T`88@Z$S z-4MS-Srk*{{K6am>5up+vG#<@g@L1DA*6Q@YNiW!83lgS;A8}jmyBC;H+APR;_yobtMHVixZoV7cx@2>r}jhCwCu=3(7OB}Ks3 zj0P}8l|{sQ>vnJhc5y?&OXbG{rE-?=aR>{$Orxz)XIk(lnzbSBw|+k0n8bd;dMDQP zC6&JGux6cNY5(a7A*O%z?6=bNQ}VL2nz!1ME8tqQ1j4sN?)0%f*(4dr@}A3;Q4}8vwoe$1Fe)wQtDM1qNYEl+rTj{ii9IYNTajm{d-F zMXgws{Qg4UNq1icWtmgzJEs&PQ|;ud%mJd4hX^NhnB#sW0rp?q;)no)5_Xi-_xQrD zDz(8E)&j%r&lWv{|M;4ekB4IdxM3m^6We zsF$Rb&Bac&&7x|hik4THc;J2JlkOt{6itv1v)$$d!29dzi}~&k#1lRUgOWhrAN&f~ zj9GIz6SeQo{Msg-UY$V90G>8ZOZS#G0#GR{ElcbcZ0@KrKP2nl+T9+NC(%=tKY2*v z0Iz8<83I#&Gbg7Fz552rbWAOnI{wFs#4G3wI`&Xe*q$XY8c7)}I{IS_ipv?p%?CSd z%$@b)=LPCI1neF%1RJ-i1cmdEnSEuwX~X%&vm?6(e2dME6P0^UfT}WjYiyg!30g~A zoVePyV#SjKh(xJQvZE=E`KQ&OSlSdB=}&X`SCHXDzrW{&O*SymvIA$akB~`gFf^5& z`FSnWRaxaM1^MtjE%lq0*Z~R+8@I$g1IN0?;8A!DlR|_-Y%Gw*P8@Vh)Yc*&`DDaG z_Dqw0e+B)sI9xs$00bl>ahg2klT{BC%Z*`QQes9y)7Va?bg(YvVa7Ar5)?8U15h&X z70An2Y;LtTNLerNyq;y|Zibgct1Qq$vS#3{9Yx&_81Rr`1WfX0}hi$SyB+O_ATgz@cq z1=aXDIIfezUdT@o684&&(XdzTv}sOQ4(7b17{|+@*1>451(fDF@U_-p+lf9R+(HJ; z^u{zC2j_{_vI?mYMJ?|@9JrMkwbf)GAd^NL$pgZ*JwE&r6l*b}T5WX?9}N@#sf`ek zVJ-Sc8!9XP#bZV`ji(?6pX_ds4Vn!K7sb{V76ZA+2d*gzN-hU&erBNopmsy@xN>79 z*ebypFwKK9kZ{q$bff7lh2^P(49V=GZ5Im&(GUJ5pfQ zO$SgI5$Sj$9VWqM3qVq(_YWV?v^mO&SV_G>6P^7{hRLlZYArg(nY_z`t2baDn7tF>Lju=|$(NkE~W`6X5g(_;vZ;6jI);n*`ZT=*PKmBU)` zq-6^gsmjjUbo$&SwSn_4sIp6!U^|5f5K>aU1I*m?z7wu!&XO56tXu}?z}uenG^|Glmjrlk^ud*q!}RIQ$llIB97TibqL&Xhlb zNAzc2TUDh|1D30pDO~9~9~?YDX1XrQ=om+EUXdpTHj9Z`v63`GRZ2(abUw5q{iq~X zE3|`i0qrp(@?(s>ix87#00u-1Kw>B+KUAMG#og5&cP-T)`2lz)V+^qCR2ke9zs-j5 zc)VLw$hv@ZS*~3bs!Tpz!OP^2>A9JD1}*)jU|LF7x%zX#G)x}YLDOwN1O9qtF14=E zJ>iI@i$4~uD{V$v=Yjj(As$CN1f&ax5LURM^IFyJydP}@({CF z%D)R{9Dxx}th*bAVG~ng^j_kfB`rE0J9*@!suRD&+f#N8Z;Xz%<$HQT1a|gLboN$_ zxucL>z-R3PxfbR;%SX3FQ^?btm!{x^M4(RN*@}pJE@^Dk4q~?sm}!Fwku?ktbjQNl zMKg|6izn9=_IJ;&?ebT=A-k%u+u@xW7h08+MdTdvh9cTWJhnrvB5DaBv;9rG9(czS zI@1JkV%Wp#Sj|x!MW6N8n6;p-0D<@2=UjCkXQ*ZgNey?H9KTB9Y@l?x;>c{-6WXh; z%MdUz1l8<4GUU7o&>GRAlie@BbZYf?9mLg&-&T(I#v_lz#Zj;vnK`0A z7i({xCiZklgkB!jUC%;~O|;p>90^B}BnAPGRdi0ZZ8c^C)EnbDl3MxkOf!+e5k*c$ zgH(vlP?9K|wzrkc zuZ`yv7%w;gn&Q62algpTlqC~rw5)D;*ONs?&4Cd!pQy){lP)@CVJjF-Fk0%TCn->bKuo*LL1%;g3chcKixlSZ}IY zQU2!K4sf($XyiDiOm%~lGP-NiNRI44dW_qog?s!~0oiM^(XMJP*LdP?nk*ScHg}}F z-8aFcO-1ofmNgvnjdI<|aSko~+PmsbtGl9HUU?l$cYc4+Zdx1d#4n$qKAH)X;Z6on z8B`Yln9tx3CJ5i1*&9Jz(c0mHUD4*+7eP}>)j8`!#xr?ESIJB-Pi)@|ZQpQON((ln zc0|5!tt~!KwsPc61X_9Hi?HvSG6~hN&s-BJKN-Yzr}XvwRol}09tw1tIql60Ul|4E zeQ@A)J*_!{sDh~M)VE{t`-u^DW=-aK#IMr;`^ChiV$?G+dj%Ha*W5tui{9L~B_5Xp zd*av5*h_fV3?4al)jK#By)1J6rU+NTx_J(B**LT_kZZYnn96HohM3mQ4D!5QZ%&0+ z^xLEvtkBX`w4x36b+c=j115!mk-T4`zABKDiq^?mJ1J{n@_yyB{)$+J3TGlsQQL-q zzroV`vMi&vRViwQ(eE5=e{|3l+h$}7R)fVyZ>@du`XBB_-%SlaZvUL@JCFj;u(E*uA=4t87#S87Ah7?ZbgQ z@baVl-h(UcRufXcn@1aZ(n8raoWCg{Wg`>8+1Ukb=L*wocGDLR+3n~}?Rvmp(#a9l z$*SA#9XX~{ZuV4fFbE>1u1*#0fK*QQQZ^gCb2nt(QFFFD*Q^eT##YN8$Q zkW6?8&g@W;e&V@glv(;E-q3c{GaF@+ zokC;tP&~8zxv6VIYs!U18^5PjV)KduJO8(ff#WrZP}K!{u;sVysyCgw*6w9Nw{g62 z-X&)HgNWNN#@HZ6Bu-%)gif;vRQaCy{hG_n7#)|T4D_eb5kMP!ooI-qKYlCi=FJhI zDMwc#oA6b@@1KP5Pj;1D9nC zjLM9PjHq9!vt^ye*l0$WFE9{Vd(K&u*L#1V`EZmGMJGgPwN@JfyBV*mD2R!QsSJ^n zC@(Nn@8X#%uLDMC21YLel^kG4$~zy4aA7I?bQ_%Cf*8jeR&4?l?|U+fZ&rd=W2eFn6gMdes}sWaz52 zz}aeaB8}yIoZJHkF6j&@&FY+6f~ch0Vh(aRwFzA2xB~ue70)gKy=!&B3^Un*qRMw* zFm=op_C{Y*lpJiI$wk7$oJ120drqM~t+?eOr!q z$G=6z2?+x$dXtc7z)U*Dj6al5!YiPewA)p&xU1L(l|`;p3NnfliA*(SX zt{DomB1-*&H^Cj9GN=S5MoPEKByv0w`hcAZt*1u~VH7a;Gp2mqcpMqrynccES3T<= z|M>s7ddKKWzh&(^=vW=wwr$&X$F{jTcE|46wr$(CZ6{Cu`@CoGecmzFhkL9w$NEw= z>#jMgu3vpuybHf^8~-U#{c95?cVKXV!~gThU}Yja_`QIWFnOyFgCzN92|aQ$9xoys zAo<38^3OwYc|j=gB!YtCyn;0U3{9)kzG(U z{MzQhU1Bu+x@VHWGeU2yUO1?RyTM4R94`^(+*V6X^t=+M@XE4ObGaot5r`-cfan~u zn5A69psESkHw2Mn{>ie{^g)ti`9$rn7faFMyt(x^4q0lH%hg{W;x?Xl2db9y*V;Nu z?#ZdSt*;hq{HSxVn3i~CLR_kXOw?9JkJj1Wq(u9wqMUX=C3##U&|to@f{~4wJVL@& zF*j0{0#z&Po>|z8zsu{Gc$`%<0IHe41tz%;CVprvWcBrx8l3Uoyh?mqXN^bMcjKE9 z`to|jdB-Qm%l96vCJ*4Lo~oAx%c?@7+|DlBYYl?$OaZ5xzF1xFyJJ z0NO}NacOz9+@2v`@lQ~$Y8K=kIw>SU1lq7Y8K}bBY-oJ6Ikt);`{%PrfFH$!Y{)RX zlu&GS={mUbw1Z5T#EC4ZyluZy79Hj^=1u3%12|cw2j>6dA2b z6KT&A-m~4NoFAi{k(^+00;nrZnAT6Um3wz4@);~FlhQP%?|}nkq?Vgi*t+K(SR0lB zGfHnxb*_9UV+^D+Ft?OiE65~zQBpJ8d|8hq!~4$PGBPQijLC_~Qd4K$jPv41H}6r+ zL4m>H()OBN=AmWE@W96uu7rYhuOW?kCXL92TtUIephuGP=T6U8zy)nIhIG72sHqO) zk-50E9y7Ok8Sc#Nk(8;O-tFJjMy(oSF%*)SQ31i#R+4(!(B$d*KvVLsmn1s{=;(=+ zU0DGkt!m7x2;mr9)*|VP%PW*>-{V218odI|%VbC$Rj#of2rB#y+r;oL2H*9nQwc$yV7EswXN%*Z1cGcEW@5o>NDKKdI2G`;`L+XmSt|)~fbh zsg;oYBr+WctctWM4X9lMiip0ot(GOjHc}Xz%msYeH)cJV(BvP!ophti(Z!Kd*UFe1 zG?#4^+3JJPoe4j0^m~|K0W`xH-Pw@_G^3%!70FP;wxc_u7(LdT%(22Qq66nS;|nlRvhjpWgbzIMN^fZ$|E^H zbUFId$R*3A0C>77;6uALJ2NHy%iT3O`}5BV*7j6j-b}SadHEcG*(` zHo;sSqx2(Cul=VqnDQEhg+&%PO~jFq6uI|ZS!Jl*gs<|bGQ<$a78uya_|r%yT7D=k zesKEPhss#X0;Anx^D*lIT<5`&9UKyN7am~i26@jX0J(KhjxcV&LyR}(`8n=tQY1}5 zn~N?*x6(uKQ{*v7UHa}j>0UOKNqV>Jr&X)VJVr|-h!W1sMpwjRUKWw08T*V5TVq3; zur|d&31KqZnT}dz8E*EQwm9&aB|#DV#fX^Y9(q*vd;2mV=CC zl#@?zz{FS#`Ib1cY6S=@JfZQhgMGo{1B}#RxhaW z!>9ip?w^2{76=O()3xR5j?J5g?`-poQQu#~XY2@IQ@cp!#-WiMcVMnTdZ3tEq5 z8$;||M@c$9gl?6L{Q_e~jK|!FL#`b8ajiKXaQovU@wxt|hgW<$z`uTw_nF+Lv%e@N zv~fi?on%!QuU;}ltSCJ~ENJnxFoSSa6JgbY$9nEK1!bnO*@7~6!hLpfwVu);sj1Zh zYi81G8qc9t2BLkqiH&`NVAUUjR#cB!7IjIcExl2qo)lrV+-xF$;N8ad6FL7OAL8pT zfIQ3hL}aFUWdaafpZTyoMQ9{$r1!Vv!giH3SNAddM6U4IY%3?D0OPu}vC?klmj0aM zo0f|I!9FWW-V{|?GgIL#07M__ojN(7er)a2&@ZChBa;5nroi#RHcPB4kE5-)#-@P& zg2Q@1{2MRx+5)(9-~LkI;eOVi5K+u_PGqs^Qix_=-4x-Xi`!qV z5~rZN-5>*eVI%gsrm`CSBm7EVY6KSEpIcSyVpK#~=hTBJ(lS0>S?CuZH(=xeKTdk3**P7_Jj2z-zbko=vj#o>jB;+&wou1tm&L(KDguVT5e9IMYjATtJA3r1Ska^tMhC2v8k0t>x7 zAql7et+b&+TK8;KZMD^^BFi@`D@*I(E$eNwJypgOq~ty2?XvA^)9KT!<>;^1T@+yJ zivY3VHp@9vOTr+NOg4yRt{`6Fi| zo<{9lx#W==4hdY za+~O#$f7=WCWSC@s5+}%ahE2fHEdPjks7J9Cs-xRQ5pYoG3z`WB8RC>UqzpMsAU0Y z-R}cOu-msqu-lyoY7c4}U2Y`0+|fIE@Nnn8ilO8#XpuX-h^uPj-9I{W@eK-D^@#TN zGC_R0E6YEk-UWV?rYDG^oX8HXSNDA5!P);AKp+hFj zy@t^vQW*>VWixziaBX~SWSrUpVw0*dqwy_nuI>!fiVHS8ZJG8uf74%T52yfkqkr|| zZ(&8jf*EDAlOIueL8Hmmp+A6yj7t!8@*&EL)HwS-`ECj7OB3P2Hv9_aM~q6_ahI+F zWMN7Kj{jywID_#D@ap5Un&}!} zVBh8cHF!a<5s$A7Dx)5We%=j$%N|$Ffy0 zc++5Bb|)}_!Q!@{c^_{0vkxJxNaF9~a`@nYn+6#hI8L~_QznwN4toedd>Jni(8TQJ zujTc>7+4H>J|I}LE--7jo#qOg7lP{E`afH3MkIa8g~@t=?yBXjAxZax$MfNYkOw*( zvc9^>wyJka+CMTNO=Oln%ytiNA!dXe7#6V=AD3;!i0#*C$_Ej3An|`u6!hjsg6V2= zA-qlOxHBN>1#OScPgmOmcydHI+`nhG8o8!00$4b3qD2V31Q`}hwRx%pCD374wBt(n zuv2fa)e%Gz+$f<9*Az&Y##NYoZH@#n^04)R?{vAkus@FoqEFuO`eOw0i81QfS@i;H z9*3u^>G4#28K@(Cl~}-hd>I-|mSMF6$BSFoLwTXUby6aWI|Gk^*Y`l&S}w99#fKFT zd}B>eL6BIy5N(5Ho`lHovpPzLTNxlMElgNL;U*aY2iOAF1|Vr^zanCcAgLu%a0sxS zS{7OO;+iBE@m(^$Dy@_|7}!X}!cr*Ha=YqvrhS+zn-!*Nd>-`<61A5%qMY9XwD|Hr zxy5%gtW`9BPj~AALNz*n8ZM?-GbdT5dp~LV>SIWp3Lib4zVj8+0nw6>W@i@pS}f_BAoVL{z#sXIC{zp4ZVmU5%P1nE&=5o;R$ESNDuF*BiSAA zoF@Siiwr`>Xmwfm)HNMxjfpmzUcoX=$M=qN5-B+}QBLi`9!dfGxjer_6z_Df$ zzgvUpm2B^6)_R7Glakq(yT|kLN6Fj`OtgGXrG7gGK*G!M=<|F*))u_vR)f2pBSV&2 z*}B%6Zsg$Booqi8`+3yN%e_mm^GQwp`)T{k0oH3PsiPpIILKo2=8%oyxbTaN)csjulh$H zWUxUNU=OfHs5hS7G-CzJB@zPX*>G=LxP*Pr!fT=650B*x15h|Zzb}<~2 zVYZ16%1JoUXRsf$@cx#X%y#l_n|#9bjuUqQkDYT~%no^mrQ!{^LifYV5qIIU`UZ&r zOak!t*Wmbuj^KPly9}?@Ukn1e#N3!zIT!xWDwLaeag>fSq2MJ^7hx;RzxnKZ6!%2m zu)}^9w)5CtH@$F!1Jq$(c|SV^qoMlnuaUvOVgP~A{oD9s!}9qrlwUHTK)vzrfs^u> z?<`;I+c38mn7)hKM-0Gs0E26KFrz&HM}c+|>|-!?ykq}`-;O$cPqfxRw6D#zT|dxN z7;xs^mSMnQB^%r;mmpYI9zW2&yJ%xj{wn8e=J>0H=R5vu*<=^{Gs1g&4ke8Z3M=U*IviS2lL*KR#_g!IoqoxTZ)p_KM>Wv6xDGqgfF^p^>~ zv)rO$r=i!%%g)bD+6!z#siQxie)9sBW+xqD#WCR{nCw0w)jWpSoN(or>a6h@~; z$Dx9io!n#0<3#JsbL^>owz zbnJY$^|D$p9R0oIOmLL~=*PaPWn1-PM-n*%XF@%Ysli72Y0fba%38Onj4r%n$kfs8 zL~+;#eD>r!Ry_oP?i#*~Ozv94ZxB;6$tc-X#Eg+CMmw$K-@#D8kqLvuc;7U0vPAL} zNoZjAE@aCcnY?+|$S%9i5X+s@;$gl0Ij-+%Ol3~mss=t3m`;lWz~-kfz=h6Qss^M` zy_v}N*pr~u--dCc;6U%jT!m8!Fk{!!)huI-isL-)m0gvi(?B-_qO7&(z+sCV#@QoT zwYm5xW61es&BVa8Y8!N`Szr+B-a1(@ERq7wjyIA+A&5cZFy*g5Nfo?@A8lAa+^OUc}b2M4Pj%Wk?cT%=q}IrN?lhTWB3 zu5U(%B-_L{n<}Re^|Qzfuqew3h~$s#$nZMY_calh0*HM%`f~Kih8S3b1AcpMmS>WL zfIWEUeRv~xVA_QIUB1>KKVp6GPWPniy+ZTk=4c1)kXOb7v|L-?vSsFi*eQGLSls?C z%8fU6w~|+KxKK@nr81&#}sVntkim^}GrZNjrsF9&p0Usne^hzlFaV9e=&J9%V`wI@}Os_P@M@7DeVOD znQH2I(-_@EX`%!E@)^+U8PmIFWQWTPvKLgitXSzff(+4UzENz7ywllbra(WY!FNn> zNZs8OpWE^XtHLgLUb8s`7P4d(oa$~Wt2hN)vLZ=}m)Hh|v#)FvIz9bP3`v^N8lnVf z_vo?tVgI@E+X|g66ppVu24%_O3v6+lp?XD8TNgdMX-7*Zun0J@KatYT78!Q`Z*@ytZk_Hcrm0H>16#R^;+i=4w)u%%b-S5OKDp}Os zgRw1k6~Be2wyZyv+3uoJsF~A{yN0T=h1W0AnZ~4Pk^li z-_=*QA=>Ufw03vh@LBRmiAFv9!)^Wz7g@agq!Mw*z&<-$wvJXt^}+tznO+DeZDHF2 z8RwVVz)=w=VnD@_bHoJmZ5>>E*8_H@&M|{{ysQv0KSY;KXbN zN(&y&1bhr%n3g{X%NJZiUXhkp7Umel`!<#3fkyt02i;b=NRYJx7&>uiB1Qff4Hh(hNoIld2^5v= z1D>)3Ozi8AtcZEhxG1CV@m9m{OccJjI$tymS6rgFM@qg0w@}IPlgcFt)U)r!k)4uT zwI3AG%_wZXI9|QCr$@lD-4DXWldQE1lhdAL?H%v1x@D3@XB4$qz2L3bmebZa1(yxP z%R#rdw~DLvv!q#N*=dTaG!sdRtF4WX@aNNMtLjm+jTsF?-#Z&XNa(=Y(voCqkGpU~ zV|l^pS&anD;I&3;jMWt0himb^?Y zvP)EBPfeo((U&ithChv*1fNYK4G~60dhQ(yz5%z}@a*zODVkLFB3-;>$;CR5&9Hgu z!GX%I_DBbcH8KY9XY+BQw!HDG~&cCUb8f~oaSvam2U#jyG;gTAPu-hp`;7TZ@&oDc74 z3BI7V2gR^yDdUWG?~FDG<1x8w;;)M&ZPjo``su=^v->$fA&W^Op&YM;*Dm!%po*DE zoUV8YNXJQ2EQA;LR*5%GiOTd&asgNd7lbfw?vk?W^F&emXuB0j1pmeLs)1Pr)s=9) zw$$`p{Tr+nD-9&8INH)UG8T+dA@ zX%F^_ZQjrnHAx<9E7*P?YHfL86QU6bmU>U?i5vP-@d>~2dpD|?{&4gd^74x}SMJ5B z#Cv5Q@&)%YKk+z)2Sq9oBCMQuK8w%)9=IUUrGBEyqty_i>cBc z=sP-CqYjO$5pkIdR+;soQ<4D6p}UoG$PLLr+mZBSA&eir9|G<`kRgr)?JuN6^ZRsL zkH3MM!y#0323Q&F}zp{F2F&)bZ zKe59DhTvRZw1MzETwP++h;9X+r*c@p!QcZxLInLL1dcx!3%z)~HQ)1CG-Fhk2;||6 zjF<2425s^ z6O&6J8t7vxaRacoWak>_XK*gu;rm7{6603^)R@X~yz+p0P&WuZsqH^(nha?)7#7(- zN!*?8i1XRCuj|E0+~A4aE_s&%J8@5CIYlnx1$+`rJ=5_?I~)R~(r6h+<&+gB<0x~e zbV|DG(4dKIPz*EtHJ+3+9)TtnI%Vz}?&iyRWyTuBPb@DOQ4qF7_&(5VX7P&78tjw- z0&l>g_2jds0ma0=33>{<(pU1w7NH3~_xi)T9vnsPCsWs+$x4%4R5X->IoI^`O2Dge zxe<^gwl3zMb`M|lq1o;o6tO<%)utVm7Oe~W@(PV%^xKpV3vunNjhau=P#Oea�BG z4b(cUkdKbter#&3@${;fPNVFl_PJz$_ro%?)mWJ~D2vspVhOJ}sfHr+v_~iRUa->H zOi>XZcsvKc)0wB-E{V5-y#mqfXEPUWAc;S@NyYKaB>8A4F;z`Z+8u%87LkALiV0%O zBuiNsZ|%aS-Tye0bmg4m07&R2C^8zhOgI}`j;}P)tx_ZOF}j+|K~3Hx_RyjMvwq{b zAEz`CYcb28i6#zn2Q+lqfVlyh>IW$FJjDpj+>_?H!OQrTq_rYAtCrJ)89~FJj+`TW zK_D(7hc7J11Io}r>34OZ+nw`<(myg_YQwi%y%F{DL@12q)r35f<&Ue0RQkox92{dt zP>L`V9p86xdX4GmCUPQV$TKwn=)+4Ua=NNjjZ_*L1=DmSEjeB#LUF)FirSJ`@`&&p zO(Lqlhl_$x3bA7nA7dV>R((G~C6BV`(4X#z-&@{=@V4!4mhND_zB?QsApiS*^*;q! zMCvygv<-Vsnr|&3<#*>b>AztH&K}7=ew0wj+d@3>O-2fcGQd#(ES}vR9fe8Go~73W zBt`t7Ow0vBQu0I$x@~V{pFneqvMxf(3R0ol_4|h!W5lg08i8Usc9v0RHaY&s?Bnby zItxf`x->h|iVRk#s=FT!ncZQmAIBCuh24JW_k@BR1XE2i+`37zYdc9)OID{c)D;Z) zS%VVMWiXGVNN@Fed~BIWny=#ra8UtZn)mh?t?{EtDE(f&AnpXKkJ!xcY7jnPGYVuh ze=%Ol8up17vFkujV%VAf)}y)4D0PKcVam1d+J08y5xPJtJS1F<9LCC4yl)x3%D9lr zsn^3I8sC2Oq~NxXBG%GPH$Q-)+8zN0jf%rACtMQ-9t%7T4b5Sz8x%&`z9bZYTr{XK z1=32W-ga5oME;)82!QRtq*Ars%1}s$K9oddDCg?XiT=BBG5kd^EBCXXNOHA^nlo-S zJkKW;1;4~R7p?DoxmP*kXwaefef^2D^`lud_iT^5XoEx6gYeZG1COzBUmQ^hLGeB4 zE1yZ`!GuQ4d`SKP*r4$)T7w)AFil83*J8v0qgG``D@d-CR}_wwmTp$A^^75;;IvYj7|g&FaJ zvUIghXP)Yqcm|2?V0^4u$Sz}9Jq*~;a@ZH^$T>r4w2J>9XZptr@L|6FMfd;cPycZc zA!;#ry6<09l51AjF_M49b0H)PEbt%z7#Zn|tPPx;Dpa=YW))CAp@W!;dx6-eORQ^0 zTBH2=EM>?XW&%qb#y05>Or4Xo=QinHFZqo@W}dh4xA9GQnpiG-vMp+VPPRPVdvCp@ zzTGW7;RE62boqlcCb(ydXouO5x`u1?^bu(tV=S}&!d#wHXg1c^>m}h{gh`Zr*iHt>f8_6bLj3<5H@n}p-UyImG?g4 zO&&patfFw6a!Yv{YRi|w2LYT0>+~INQIh0r(dmSe(39IPU(W25Q0u(N^p(f6-OYbW zOKDkt>0`EiYpVSC3)9Ag)vuQTGtT`1%51m8O1PzH88zLiPyNMZwX7*sp=22DG_ahy z_(kRU8&}FOb%G=kt}18)$o$YazWb#F+Fw81R2vemVRqbokHhbb=*T~pyF0@`on=)F z(kLh$K-pyz5{rK=h`?c!Sx-;DmfRIIgteVBI@oq$>e`36Hh7wNqC?dH%oW*jo51^S zM#`UiCm*8SONJCl@&qaz!#Lp`8s0ka8V;RaSvx-)@{e)^u6|sAIVOf;qt!J`ciRx~m0(~Qd|dt_4!g!_70R}q2=x-1XC}%de1aQ9wNhj>SyUWjF*J4Z zlT4{=#`fCm^O@xuSP@|so5{lvAGG_&}Jr5Q=Zg<3H<~;Qmm!f+FV9zR5J?B?jGptD#r1zmX2fz}1LY z$$xRU5dj679`49Xh+h?qi#>(`!0dhsh+qJzZ1V7;aV8qmqb8kqdmvVhGo&??EO*>xK|1|udmfKCC;=mem;`s zmnRRBl`fi1gmcK6QH?}1^D5riWoSQf#zrjM&M;Qy+qDs}kx^WUb$p(uvA7T(1FA0Z zXK~@kNi`K_ur*}LQsHq_nspjCHnKLUX90lCl*L5ch$LfiZV(n-F#dN*z~S2JUhLxK zb-1Zxp3QXg@7G4J0vn|~v3<6R84NWSVYy(ZIicS0f$+i<=^qIpxh`kNh>P*{6QyjT z@kXhic$^!Q55*)UwsXjFt#VS-JWW9dhiXA=Onn`Q9s=qMVumB8y9h#JL5Gp4uH?ss0eW%>$jg9`x*Q+t)f+0y#u2R?|Q%aU*GB3mGfAk~Ydr~;e+fLBS+J#Gajl{-s+hUrQ9NL^aGzoNm;RZT-XMjI}+ax;P@ z{zuHHoIS%rk{LX|e!L%(7Vn@$&RX^?2Q@JD!(Zyy0ReD zKnzuU#}W0&h?9#|9hY2|Uqk7}sEk%JF0Ne$Zq>-8nty0c*^oM#K)j<$sh*}RJZHUz zo2ENJF>Z<~ACmvgwOPkBo&qcm&!^2g#w}9DN%@t*cm(y=B!KL)8`HMP7)+f-pa6l1 zj~0{BAIB)Kd#F|FjJ)9)S^c*F` zgLBoy;bUB-S=1Lk2cQ-%msHqEAO|00xCoDk_zQn=JGC5(()2kZ@(W{lp)M}y%vSp2coT+3n{U&U)tE1jT7lgv_z|9DOf zrEBa=HGmODz(b4OR|L=thYK1U8y%ZX+s}q%MKmLH{mj`z#P_A#Rex>>h4m%gZE9x( zEfP65tlWV0!hB>rH{`5xOINkaE8+WQ4hB ze+Yl*Gw~YirB7?h0u7B$%@(sZ4Js{1XMBnhwv;}ZH zZ*AO}pjsPU^?o0}=sV08n!WN(Zfnv|%aYsFPlj*(Yklqom{M0<7r1P7dX*}+^E3*r zNyGgKX}#lRi!+C5;u>S+fj|J@LTfQDTUC%fjTam9dPdO+HinTv|Sx ztF7$W+6!N8QvdzHn7L0{r*0jc$GzsBr8&JbVOM8~fH^?7DLp*FN>Ow?Paj719@P8? znIYpQ5U{kl8aVz$3GuW$5j&XYrs!317gA;R3X+zx*i`tdq&mUhCg%Uh4C7 z1ngrC7~uB7_@1s=+=3Y*){fhc8xMo z9xkqutxQWmFV5!dOHP?bIf#{IR~b!7G%W%N4`UDBj-97yV1km$+xWAlVI zwT1#v<#!$AcIk*ZhXS7U&>gm_gP1opIW`iSXz_yblC(HJ`c^Zx>(dF^MwTqqu@`Nl z=4`{yZG9?SU0|18wQbsn)-rscs>XJ~&EVe{!iynf!#(1?&t$nB&AaPcyd(#GN=;qS z*aw#d`b=cBS^*c!f@#DS@!U6L zv$XY{V`X46kCwouC$zuVAD8#DzYZWgJ8v{+h=T2a*&>+e_H-0_y?4gwlIU8BC1!NyyD2h$wl z@uhDG!##f?WRmO_F~Uzx+q`JX3}OPt-jN)GL+_%B{HTsAlBD6>v2_ba%V*XsRBWU0 zauva9WcsQvySZ=Z-q83qEZw?`%Zis%ME6j~_aq>}VmZX2hdNMf zDRb?^P&O|L7`j4*)F5y9$xt3`;&ALG;af>Ubm9PgbR&q# z`^D0^$`=TUIDJqd`9lQ@f8yaK*GX57gHlW_ibj`*L1^PlVy)H2p_?)$v_36d<} zN{^n*kVb(4Ao>4JH&NffO=QNOK5}d;B-U#Rd`P3>!fk#c66M>_yK%O8q4|kvR5+dj zlY=DigOzOQM4vEjlfMIh3jJ$j+`d*lWt@A)y~ zjavJ@Asd*>=jJn`_#H(KZ~8i8w(X%DmK-#I<|Cnusg6~Skb`vsv{4y^DG`RFIB)Rm zorolD9Xjlbnpgj}%U`Do`i5ogPV9*Lm}u0D@+aBeh0vne-MhX-gTeSFl_`T(1Yr~N zBnu+ff`=uYj>#2NF0roIxgjUt-V%XwWD_(cc?hEP)UfE_=r`cTYTWEdsoy^*fDo<$ zF$R5wBH?Ei^=83i#*NfLY5f+Kvp}>gB~BJZOSaCNw)?A5pLFk>K6D;zvw?P`z+h%qV)0c?prsW2{sLuTV;~$ zX2a%nS~MTef9@%+cV(Bb9e*Vp^Ve7cHacq>AoT?W-k@_hZ;gQ~u6`BqQ-2o_4b?xf zdF;NB+mqqZpwMvMBXMqY#Noi^Yh>rBe)dr@qTo)6m&Z=Vsz`#kS9$9UbW??EyLDP6 zAhL^FxE1F7wICY)ZtJJ>v+)f8vC>R&2o4k<+nfR8qkGFqXoR8e@>x}0 zY$l*|138m?!_ujG6EkHVvzeL~+>|jtFB2J!o>AweEB*YfQd{>jP?6x=TvhUdBF4}T zBO0W@jw2IU@snPvYTK>@<#e4bVDY}K|7^Z>Z)C!~ia5Y4cQ4M=aHA=xjTS_FM7RJ@ z2+5do(RsE!dwUsXpDYrQ3off3n3F$OFw=wfg&mU;9VEQcjkK_nblpnzu?ur*ufF<5 zjX>0b1n4+#4dlEH*zL~Ydm`;SqQ4H~J<48n)bUUbcHO=oDd00SKDCuQSqHwYJSbsw zcy*-ygE5IuPp)_dS3(<^l*MB)>Y59{6DITivu(_c^mrM3jPRDI%Uf;2@90g>QZ3`S z77eORg#l|8eL2+YQ$sH!V>l`TOid<7XlICnOqSMc@qMg@$--(j>FrJPTO(Uc^zG{| z45M8)FmK=d$*_+aTuR-ZajJIISx*d_#i$}iiQH$56zE-EVOEfht)-M#b=41Gf<=~e zx$|g+%Q+o=!eLsFPAd3=E?y{13+>~jKFfg;-HPaZwj1DTC`y~*(}JoCHx$trgpJ&? z>ku4i@EThEt+aS7@K2r35K1m*wX%_&;dLu#vSPkxoR~37BCZnqahgL3eCxr=GZH3- zd=HMXNm3GmGSaQ$@%gj-9a#x*5D~I{Fc&iIOX34o}yuwSMH18+}B(oT@P5t-a8s&rn~rK*dWGd2N`e88WN{ep z*#Bg9{~ZMWE3Lt34pY6rL4cb zA|-6nkWi1eko(72U&%^|WgW+26$ht7CXObKYb^P5-=ou$V%ye)kvR#Qs^CMgN%qCh z8aqlVAU;kwmD&hBr1Fvit^BJ(yn{y4at$0HnRV9aELU4LyC;;~c7~F{)1h+!qO8kl zIj5I2M_qCtxJZHt!$DnU2lzpzr6p$fWL#CuWo{C{le;vEu>iBs>`D~E6c}Abcxk}c z?*od!KwEwD-QdIXS#)1VgD(vtPsdD&2i}Ebllyu^yn3*fsHNOgMENxa=Vc{5HcctE_XS)Rfm>)7rN z7u7hhj6v)6YSVhR_XD%kJNFgh^=@atmGAE}Z{sYZ6<+Kn^VGh&zy2PBo*ChHLJVu) z@83UVjHpnNJ9L8`H(CD}{U6JYeJAbN-_ifSjQ(wN#UrpY-ES}v-SEcv2AJw0xbCBBAIs>; zdofU-I}EIe32gv)UC5>=#l`qo(vir);zg_GUW3y%!~+U==6RFxQ^Ke| zNPWL!P6-l-nvr4wa(iGJ)yTNa?pKnurw~MczNgrrA}_V6xbaEBu83*<_`)EIRk3qP z+dBZXRFtGZl|Dtq3Sw9>e`cSP;BdNM?f|U^fG!C`sUn1I-{+Pk4Jt?~Kl?EleHn77 z#(fsF2GSuL9Y)V+$)@ELPSCWgMQmi&dx|mXNR;e0Z(1|~W(vtD{rOPKl_aZ#I^y#* zLCBq>)WlJWAniIoffwJy)|<{F#gjCQ?P}x}#};v-6$z)R{Z55;QoLJ_*t!BfgnUPi zQg3+=KJKD}+op%}k}|ygWpBYOcWn}w(H_8Nclf?72LbsXEBv=55vgI`WU46gJixz8 z0qWm)aYT&d-zq%l0M7rdI<47$4?^I@PETotXDmpNZmAzoAg3hpC;w3zs6!Q_B%tsB zd{gVt7*%Exb?#!p4-Ul#92Oo9!S_2_Xq>Bp=zA0-l;-BjN`{l^# z8PDR(G+R$C$a99Ki)bnq#NEK5%%#fI@ zmX=XO&!5@OU&U@=+oCs}T?6E%n-_E zJj)cPpPuH~Cpkf=ao*>n7~b*4v@NR^UFR3!8u5^MIZHuG!iY&j`^G7kFthx;8q)l; zaZ7!dp|(yjP@7rqW1CnD?_CxcmJYEdLMO-o0Gst+IWf;W4m1xup5ti;8FCKIuRMF2 zX<4Uk`)^tZJ2!Yt!A0JhMow3-6N`3{o=B%?dEn5fTU9&>&acPp~&A&!z{M%9fPxknc=5pd}$XF6{w_8je|xkizV(vjeA|tP7r0|0(>H zjUTH#A~Qb!X5&LnSIp#I#M0d4vw|Dd{`9HiKj|P^JPKk}V?u)X(tn()u zU?ks*&A#((t|l;;iPocenGP93%6tx^gE0{8Ed>O!LY=`%EwXsnj7^`q=$T#fFOA3b z)Fqbo5ZDwkm#DYs219UeNTsYdZjpsUV0mDK;p$acnPe_)-8gS7Qm1`m$}rj?;C~0g z?XLZTWnOZMZoB{+Eww#mD>zcDtKlWmRtvL#b!r{aXQ6s)Z!Eij*;kmPvK-7*n+sZ3 z^aPxPcKBM%+k9$SVLSpHuWVAp3tg&CI&!f5|RFq4fYEil`m~KzngV5_Mv6MR;~`R zL1de#o|hHQJIW(jXNI;V8fRF53$|~s_=Q0X+jHlrcZshQ4jaE@$tNqM?m>GtgkIZG+mE?Ku-e6HEsqq0Cxk+Lrcd9tT*D z$vq;2la#*R-YqbS(HRTh_nMlKuuv z>zMk-0of}*q5G)_2Sh6Qa0vW03V7I`*g3cz)C7yIM}tt7y!mFvsw?(?BI$n|{QpMM zHyL(!1IyZE-F8fQfO&(ZmA1Z8wPO2rh?Hs|5=xMH4;nwTSMoqP@6)y|^1T^U_$iz#s0Rbt*dEfs+sHMQ&v_NQ2AaTh1m6A7$?4H`vYTy8{Vc@%2Txf#e=a9 z^XV)S7H!54F-_5nmI!PBhJ1y2t~Pg%c|| zwHf&r28zF+049D&Vqo5pP>?cE^9qOB7>)cw4=?rZMA70yx<+_pwQM*n20H0_x59Ak zLsE^YU6DkICJpjY;8^WyqCa`|OI3%!e@z6sKRCWaC#<~rck$I_nCX^Ot^1#rH4QIZ z#dKXlsA+IvOjDMAusG&m3`s8rO3J|fV&hzW5%~yKl(UU#6W>AXa}$*`jO-=nF#H`q zROY+SVyc1u%9vQ*_>P-mCbH*$TfO11x*OLyd zrnP0E*_DpFrmNihwIfZjNjID<`SP5uUL>P;U$Hhu!)c8CfIg;qj{~-Dwz*nYi?LNi zFfaHNT~7L-kr}D#cWSXcbO0Gl8R$(41?dxS`%h8~{IFB!6$Z(ZA&hJX z1T?!|6!`bHzc-{#b5#W5?{3{9Yme*hk|%jpgV+ZE;1ZymuDlrGDqUOM87WEQU?e|^ zMNpHOXrYo#U0ahm7#@wcijAS@IgM4mg(ri=wQOu2K6r!*Rk}@ZTp%#qWA{oE7rR~? z(@pA_fvf*5s1mby$Sa=KW#^_eeRUzBX*7=(#)FMoX^F=Db`HX38i z3_IyJjSG9|w806SvqbMXT>U=H?_LSpu%8}<0o=}u<*C+;#BU^fF>9jw*2hP-)8uho z>oEvE_WdfXXYXpscwbs+4TZ#UrrVT)lniCS#kSwS#1(X1A($Ehk5+6fqh(-@8)6em zB-DdQF{ndq;`VMKxo}b+5u^sFre$ywm}LQzynM4VwfZ13Tk>8>6kh`4N^DFd6@i1h zZq|u#7qmSL~6Zk5jVck-i*Hc|BeCwuPmVdKkCnaDh4jI z6o~BKbYKSke}D4t=kdU1G$K($X6Q=(TZjuqr4|7DuPA^%#f<)s8&C|I5v2W36#QrC z%ovd5g9PNM=-Q$Rqy129m@Tt!uD0(U;t!Sv*9bt76Ej-Jh*XQnjE+y5%_!ho4qw-v z4=iEA!u$XMQoTG-8I`d`KigQ=zS)*p zWY5BeWlc$Q{?Xz?lVwNx@9Afo&*hi2>`B=u+rHZrPDzI7D?F4_&lWX+8NqulPnmH@ zKM~@^T}fg}g)$0Q=Nx16!m;q4ZI!P%6FTa>#~!dlN> zJOF|0!35puv#&}LPLeKdT+FNYDFQ0(`$R&?Q$bbC@gmV|UE*gomf6uYFh5;mYaBnR zTQ>5GP&XNVSf{FFB+l?E-(TCwJ^mwY5%-m*e_x8#C0ipahAo!ivEHHqU*paJGnRkC zR8#yR?V^G~2)0aDE;*^zt{~9ANERA751{Vo7wzh{S^-SPmlbiyT9bm zDcH$8>|37d+a}z=E%&*(>9C|@{SC?y#?BA@8Evq5cgs}PGC1pyZ0Gd}y54iYty_MD zU~y%1fzZu=8|VML^Z(sAH3#LC-2REk+<)0nNl8o{VE@sL9bGTT6u@BA|A9#n&tijv z{P*CP$rG}d{98a06v;peXZo+2B*>^Ep@szLLVKzGeil6IpX{AHHYbM;1;Zi|(HN=+ z6BC6(nY9=)QCg8cC6AS}IGh2CjJDFRZeLw>pt#k^>2A`J{?*xmUegvMtGl?$U}LM_ z-Km?J@4A=6nPP?la`X4|$M62u_jSkjwrk*%ce>Bf>og{U06g%$_&2oYAzc5=v8({V zF#c$|R;+u*W9VO_9u=YFYK*jvepeaFP@&w8cYRYy4|R3UkABKa3r9@eSv5&R zRUG{Nr8R4>p#H!{iDm+ypkHHxuel1~Z6t*p*+-qz6K_Nsm{3B*J{35m;Y7pR@pF+S{bM7jF?!mQ~08Hs| z`0fYI!!(i|j=w(oVP~ABt3APi-jlZwi0L#NMq_XsN^xY&JJ6rW$g9>n5|jbpIe;5G zhxSK7Nz= zU3K`H>x8^J8rMJ1?yv9#ygLi2z)Q;I?^H-ofUIwuWu$gP1;pge%17D*d;^z<5Q?8%pp_ zr(`CpiUcXU{v!Trm222Mrj>gH=%8N-zqE_jca5KZk-RIi6R1eF;a)K$?z4v@RV}1D z;!vpHDks(p#8*+;ZekB$biD25p2Cr;fM*tq+sJIU{E;ovKWps{o>7vEVGb{XlTk0i zs0PlR+WjvmY<4!_Js!`6>3NH?xHRERPqB@6(Tq?wS#}D|Hu2X8 zTaX|Hu(CjODF>JyUCy;`UeY6$ePl3Fqj`ENKAp~(Lw9WZH1=D-ZOy$`uutSEXVypl z+3w-#uDN{B>a5SYpQL?Ch>s*ut!^B?uq!;WLOf_U`$1Vr^)eEio0n*}9wHp68Ft9$ z_RmW*drojrT`XK&Zl&blG_EZfSle=96w`O)aB>LwMv^wt@}p!&f|mO7R_~zFx{=eu zY%6(itJk;Kra3sElq^#g?O zA$6g=zLB!=w_?rChB#w7p0|#3kYyYiKbG0zEdxd#9C@?x8hDqH6UuHr*$S8#yNMss-rQ^%%PPlP%UWh>YQMv5u zmR)?ey!J5r+#N9L%;LugPUB2VpII+PA>Zc z)Yws%Q+^ROF)nhCfu!D{z3!!lJcs5hKh0+)*&f_Q&(zqOU#E{=044ZC=_)o}>>>dbM@Vcf`|HX&k}EaKlxuaO5G?E#(H;ExLt#fyAfh>l zK3zmSg}~v`#y+i{U8QO2<~gZx!bhzGds;r*`&$L9JW2toK z2Kmem;Xauq39?*SAC~AT`{(g&zU*1o_n6JdtC9e-WQ2oayxn{8ApY#2|9t@O7I3;V z(*6$c(?(MFfTqnc6PU@FR*Z=^aJTMHPxYoVe`W?^g0aM(*73JsIkSUt1J`|4H<$}! ziI~h8{$_VGDu%cl^2O-|6|@*QSg-? z8175M|AkRhIgasYxO#lY0WlENZZa1u1@A9A*z+p2UM&ADP_TRZ!rrtqinTpcm?qUT z1e~#Tm)w;+EqReReFK|JcvG$>Jlyes55G71I-x5T#t{}`0Qih%XqX=kFsokM!#M|> zm!^iVl4vM(_NX#?TEBUIMTGwy0aU&9hZF8c-~^0hBs`$HuuFFRygzxBqq2RWp;BXw z95jX(91KjRa>HGeT?0#NXKa~IdD`!Y^$()QP7do`ck886cTEU)WftZgpXkx=hz3!6 z>UZjoisA&s8(Q6adT9+0gQyd;73~9TaW)SMDeU|z3guCuSA^EXs-YvK)WTFz|Lvg! zNlEZ0vnPqji_`qW{^9$HIT|AoXoz=Hu#AAIo7s@an%SBZXO>3cF=w&ZPm^jan@4J{ zh}BQ1j-tuSjjpLN%cC-+3+6#Pn}g5J7~;S!{Gpeku$*)Sd|RIfOfUT+0}fjTJG z@S=IRcWakVZ=J$8F6kEqK9~`aN;AG6Zpzs1T_X7CIV@B5aHW_Rp4bxDd!cFqGYiiM zO)!w@^>lPY?;>3|1!O45{h2Y=+glouOKK>ZdhuLY$y8iH8n-rCWqvj&jM^v`3fUVn zWcR{GPyt~U2@R5rB`lQVPWA}1kn=dc9Og^UTIbuE@gIU;Q@IwS**qnR0gi&Qf zec<{qpX{czU(0Z=L-ogr4!vH$ytB2&EW$z|dJzfX9-x_JS+Y_{hPMEdrBs(lnO?>B zSYv|!k_Cz>(g{&_(w**7G98W)6)?c7D;Gv!n6**UOv)&#M`+Y*SNBzw>!MUgt+OkQ z|2k!A_NT-5#4|n@3!_$`+KQK0Ma$)}Rn;dzKs1Hx)}vpsHRbE(59Vqihm{QDZ_8O_ z@*1^ICMGF$>$r6Vxq1W2An(Dxe8@N&Y0cKN*_(uOqm-D_J2^=4JB7 zQllt`wq-7+V`oo z_+0c0lX5}Js%t=tw2&zi0AGt@a=aq2t0&h|0w}Zfret*311JE}4+WeXbVOf-8;S%n zUDPuc(dqc5LZqc|$0fA`3O0%9!$;EZ>!h6I5^mNKE)0M#%UEK1OCHNpv*CvNU7VfiXVLG}?pP>y>x2Cbs-y(&`zte8nAT*LUK;U}<4HQSHgIrQvEM zwM&(=+xzsASlORS9-{U3?vMcQU>=ZoE=d(qn`7ksswa`I=kzn|n(m=wskm{A z&EXx(*qO2izdfv*Nga_H)FUProru=nBdOtGsLRJZ$=pT4$;O;?*2_ z6Cj%ed%zU!Glype;}XfKkjD#GPBUt1RM-J_@G-EPqPuQhy|34)HnU|?lShHzYL=lI zY{VL04yOQu7@7EG?w{~0`tjepvqE+3zYAK5 zoBB=6&Bd2-qNhs=aK+=_P$rj1`!M4llozZFxd6%Q$C*|=XD?yl-V0E7-Wyw{AQz?U zh_by$aRKWSv6_lZ$QD1h*~A4EFcM9>5L2jIAcOH~JfAWycxBJ1MM$MG-^pzv26Y_!5oX~-Xv?*-IJ$VIwgYl@)aXNe4z{N2N9JD?Km5XT$# z;&Jl*jruOsbc?01IQ~3H9Ob9@Cr0ZT;65Xu=6wZZ7{KHVPcB)aPe~)xE_-KSg50T^Uf#%ue8G;)xVvdiZh=nztyZP>=iJ8~Wa?qa zMq-F%IAE7RP>glZB*6G!S->z?BB6WfwRPlGZj^)IIrIv2z|c4h^Ro&L6g;3i=|Xv( zeB!{cO!0upM6>*+i}QOP1Q$S3vitp~H7Fk}O*JO^HMq|TXr)-#i4Bt<>IDESwGBH* zvSlE(NqPdmzz&WEbR8nri%W6v-I#FM3=iE%NC1-P$%mqHL&Rqn?~1@roCb~X4jer%B zcN))*2YQ|hC?(n}F(9*%kOh$RVNNVB7_xWMF2Q`-hN*TP!MBacy7Z@r^hB5k_@|Rw z1v_n{>PJ4A97zq6B=TL>SkemVm(#XY~7e6=$Cl^XlC7@QHw(9__AL6;9FowZxYWYzE;Vt*(3lHe`M>UP$h9R`~_?<9&fi(~E zoxpp+IrokW5st&(Z~0;kkx1P*j|bFTzu<_sq8t4v`QQc9)^9^C_q9dxv6I1GQYW8W z6*Lbl-_@S+zP7=7vb$RXub?8rAP_>=iwzo?#iS{5Dj6A6Cbj`@yX}bahK_PSp#fS$ zanp6h(>{en4UkK9n3VN*A$Bo0)82S#N_hE@yYEXFxf zVGcVmmTu;!me?u=-+A(sZ3(qgerpK2jn73M2gE4zqoAaa1Wj2sB%HOPyT_TpqbTwxim zwY#h(Snkz-$P-0-D++i;LLjr9DR>k^xC|Yq0-5`e3E-TGFuEkjjUjN^5jZVk3z57` z@Bn2H-4VP_ExJHiacn(`(;;>fff_&&uI>U&R23Z5|0 z3S;-zIqnDt^eieT_s%+GmAm+88M0MmBeInbIh#D(EKayYa_&~*mz~Op`Ctp43Ut0q za0N{AD2){IZ?HvcCULev*_B^z%CIfs_}s}pRLFohC5G(CEXyJ5L%K>#T>4XaO4zt{ zmk@C2kjV}^sth5l_FVY()L}M5@n9p8#1W{kRguDUqjZcBD72&!U0FaHuofqs84PFD zE{ydMDk{q|J@}IRKmXob_7C++f-X6Bv2$SW1yhj#$|kVFK>~8N8MHi07|c(A!{#)D z5;Xv0VF^*2-ctVpSjzZ@)F4NssO1=_6k!N^7TXLG_k&C9o}c@gHR|fyNyA%2VMsMG zC4~tJH^(~2)eh9@C4ei!|NUYNra*SM6L}Rgi$JEm>3Q(W#n(B02g~lQ=i0^HNevsEHbw zAwtqCLe`AfLJR89@2lS3s@5q_VF{B9mUqu<9i{F>@I_!iy2&)Ex9>y{3{CSz+yc-E z6AVCLjWnV8y0D{}D+OSao&1w?laaGzQM2|Y(p5`{^yFp74mvS}K=T0$;&i+$omri@ z*M|fPBYkGnui6+wJM?LkL%8?lYThJYVQA=&x9Zk}!2loQ=2OPhF!H6I5K((S!q(?s zBDCeLIl^_lh~>#4J1elLvz{UQg<}0=OfJ{QMj-x)y0`lue~O;D5fp5Bs4Iv8 zt^uydu&@XdVj8=!@gbUnZK;}|-#xPF5Jg{!vRbzkA~(b${7_jYjNUoGQ~%upv+kj#( ze8?fbmr5@zV><`8Fy0BLcm1W5D@%@G``7eoe zhA7Jjd8AY~Rq~zkpk;1%7Jwfx?L^vlkXL&Umj@Kf#Rqm|)b-SE)7QMb#h8i>ovM}8 zEs-X74Tpk&aiewfNtfll-pGJ3dIXWE9|9_R=;R^n1bT!q#w4S1Xl_f)=2b0zCGcY>XRjz{kZrH1wyk}-UOGD&3>Kr>N3AUk{3l+!Xxc)E)8bAi*nC1nmBg+y# zgHo6+oEJfAI!efTHioJJ4HBUxYZ8W*sr2b?Ue3HdYwB30!(K=dxiQ>ztIkYt3(Y9e z?l@nS>R%GNyoF&gm-KHlDU~H6ap@p6a&!vXi}d)al_SD?MQA67netGgD%Ar+>C(~9 z67U2l5avPA$>K$dV1R0tTj>6SK$y4a`R`J4)?%klT&Y`fUK53D;-^aohAGmg-oP1J zm=84-)KonwwMVN7>%^)~BFQWTRJt$_^RiJO6o)D$tEPS=f!1P{8_m2bwzUMh5#L!X zKF;X3W-A;Q;*);%(pB*#Q2R5s5bY&I%qN!8r4iR!^UP5L1%S5vuUr{fCmecVeluFj z#B%u3iwh=QZA5dK7VvemDFaej7ETLkyu#9C!M~Vi)r7;8<(T?H-L4Z>9M7rBoPCvM zgx5sVMd!XzZNuHIV=@;5Vn&7(Mosk79;oYSqhjZ=k6=^msI>}ZquLxS{nOqeU8ywc zT4Np2Xh7zq@_=)^;2%9H4YvuEO>35%$iMiB4R6LxD1l`%lysWyLSZPOGUQ+tG{%*h z-G#y%5SCOs8ppbFy>Dm&B~fZ3&1~ul{W*2yw&f!aiJ>JMGxVs`T`VR>pgXw!gh@(U z2*M2BMWRfgh^nHbzrS*?qiD>l-6K6*OaEf7iWD%uO95`lL5$#U^z4118fdMg8}R1Z zZ`f||*=}U5HZzud3=puV&|C90$B+b2U{NQjwB*^=5Urh7hsOpUSpMB)$);0$B_>V9 z%2JhoP~9hk@28`+^DSC7+2=dDttQCQ(VtzcL`*(^!HM7PUA`(k)A3e))H1smFzMIQ z)O`f=YXd-9Xl-@bdONZTH+O`FDWz{fs>WK%AXp=+P!h__W|SA2O|8B$j7wOj;=@pZ zSAp-c7r@(#rP?0wlC_HoJasAKj;`smL?xX`P!wY#l^5!PRzIpcQLP~t8%9V^aZa%G zgR7?9CBwh|@+-qsP;&^UIpH8#xKWZdM@_R$;{_PyJ^FL$dPC`?30tXt6zabN+iAZ1 zYMTnV$5&~OV7n&Q`ly`sU$MYU`&PTvBxq;JFp#3KaCTTipwx9fBz6T)Irzy$rwc?I zCzd)P&#lTQNvZac{&(GnUw*8|tX`q?o4%L{TEii~(h6Cz8L=OOU!56b{Ai?lJq3Ff zPY{p-u(Zqbz~1HDG7$>C5!l7oljNw`wk)D~Vhe*R#RHP}4(~l|Vcwo;pP}KU0HF!Q zfloK;^6@15$l}I)k7;OeRXB)zg7@jpC!I?SVL0)YQZ*n#ykDY1cA9YNV$0=k4hck9 z%KC22Iy^yJuK&yl9+N$J=pRimZ$}302LW76)i8;_`^Q!!@H5AHeNdfHh{&} za^cZqvTO&K^6brR8vQWK}xf<3nAp1IRxQl4m*UGD#kaIxwPt>n(RVx zce!ja8*ulO=2CO9=L>K-a3?z>`}7lgN+q9wD9fG@y;ySrsK{wPtxJeG%wKIjUF8c_>zQ`Z#|5s zTJ4CPEUWFxoXl7D!WN8U-b8g`I8;b041JDJZ~qAXXG!CKY`_2Q9VTUZS`{z?0fEef zWf)sx|JO>!NIztR0Q~lY_WS1u-gmwB&*aDvXiF0z5eWs7u^JnKih+Vbhc=Q021bgq zNJ9+CO~`3&W9r#nt!fml>Q~9qGuSPG*u?DAw$-hc<+QsutTMFgr2gf8-SK|1m=6B< z3^266-MZyJ+iux$o$DJ`6!blW0+K^p>)hwREh;C$&uLnT1(aCu_2mf7irG*ovu=R@ zJyg~6qbZQkFIQz9p8V+YA}^?V(&j0+3C1%Jzf)X(zhF3@uemUepF8U|0xFP1WalZk z3l=^d#=by0hOMkB8xO-yoW(T1=jrDPS$iV?<4_gDcr>lw8^n3(2fo;}1YYu9?G`{` zD6hYE?pFu94fv{s>yIyx7hS#j!o!R`+J)1ayqvG7#3q9>7h?e;MJ4nXG_NbdAcn`da}}6?N&(*xjjk;%DG007#y__Uw!N2=QlQEs&bWjp;65nZJ3J{!TEl&T7N~~-r%mG(jwIq5#dc6AA*lgP$$qiYSvgra zR#;BRp0#48oFXqD&)|~uL^4nQ5#y(i*0~S5g3J7C5~8>$I;fj!Wt0daOx8ch0#N84 z#JRA2d!=`~g|~fa>+mO>gPF+&6xi@=8}QGjJceb)LfUHU?%KGnj2VQ3F%g`5Ztrls z=O9ju1s8C9+BPD(-Xa*QkZ2g#@o{N|)YbLRgB$z(9&!ZI@8mV!S>HI|pG(N=2GM(S zR0%4AuUTWTG6~FzUCLNlmEY0d@8b*~vfUlGTes_cV{pBWbUX61QqI|+{1<8Y=xL~erHq%!K(9ew?;|&mRH4zF)vzONoI8i}aG$w0x17AB zGZX_tj>FpG^k$y0e(($*n)a^?NS0qGgd^JGO~&~GpGb`+<`cBbC+?eOow7@ z2=!B`K}9z8+de@UT9O74%?qD) zFKt~^SU#)3cDyT_nfGY`R*sg)`g+c)O5~+W3`MalV`+#MOJXbb!myr zmFKATY?6AA3!a&^Y38KRkf0oF#5;1Zp~;+@R+LZqen6cIh5exxXMMbEt=8ZUcC^eVbV3w0zb z9BZO1;~U7OYSIO#LjFUI*y9X zn$KcYStH37X`~0b$>jv7`CVdMtEKxIpQH4jWY+?OyNmQF0rd%_J`j^cE1cjT!Ifrc z;qMO(7;)60R}FKkraa4+!)_3yx~g8PKEqpN#Ue*L79vu%#i<}i;-hGf^;mSHlAl}_ zo75Cwa37Rc)iFh#3>y_8-7M8?z|JZkvZx=(6Q%n2c`W#;y)tjPIw;ByiV6~{^TRMG zRCac$+m-ph0b1G{q=D5V*i#SY>n3&%3f1jp7QAB_l@l$W2y8uKB|7GKnd(|P;hIw`r;OMY6sezVgx9E$ zgJ2ya0_@-&9ybP8+gGNm5hbF_#LTm6jm}N5!hs)GWt^sc+wFB9s`PO@A@u2>ihoN& z?4fz*yVJy&qe;%k_b@J4DrM^+g0QIbNx;L3N3DSl<@H=x%BfYFwk2#P+1;(X z2p~S%68Ly!KaR_#{|W5S_t}M0f2@Lq;H_LWJC!&!5V0$p^NT3P4k|%|e+u3LDaDG~ zQHZL?+xk-d(wEAe5*X z>X(;HAD0)fc~1;2C$EBbgKtJCY?6px0>B;F1$K5kh2ZpT{Tz}SMcT?x&--cg3>m=J z6!%${8L`6s0`66#$xaM)h94}K?`NtP$m@5=t8JS#YE`vXfHz`dLZ@=Ihu-&77PjJs z`>Yk7Rz96vY@r-qDI@C#kFAml|>{BBm)RV(;jq4b6JnTwyCu&EHhN0D#-{ zkv1!*bZ6r3ltMpe85)AcxvDZ=_y-ne2@u=kUY0O2blffy&{5I)v~?_aUVcb7SA)(z z$ycd%8{ZVUX9$Uhr8pv3llPZoG)W}f1i0w|z*1uZaSl0un(t5i8V2}3EOZQa(%3ij zuo>sCij6jCs7@$jJ}0(nCY*Zd0O)@x*siG+XNMwH`ROlm&m_0uE6~_pLMc*;eM`8-il&B2F0rbyhP(I0Z z{lXk=x!PShV~&sjh0tr%p@klv8E#?Z$K1R_Xf3v7uu)>|NUTb5mpYLbRnKcUbiO8E ztSHQEpWUMi`YII`+O)0`Q*oBhlSveCf^`iBWWd2axHhDyF)tq8i%nr7ky007|6QiD5Axt=E&J#<5ak6SO0;A*^fAF-8ie(=AGI>r6Y# zLA1}|l?UnL=C2B$*FD5STz@`zF z5dD!$xEHj~^udoNIZy?Y-YG10L32UY_2ugbh%IsbeuY`47kd3v0G3HAmVG1N>Yh=E z7C=9iS*(gm6fw&t8*YruuNmVuQykMd(j~;a$u5?{$C)VNqX#~H*ache^2Qi#>wpdC zjyZFX<%4P0V`bKeA>FtU#wJ~bH4z=BSS0rMD3@p~CaN3rB2o(04Ei#>2TzV^PAV9LG});go5exM$-N7~nC?s74Z;%1x3gSw~Ei6TBUxaiWRSBg1|Urjo9IT39e zi)sZwBB|tmC$$)`=jExDUb1^uuu#P}mkC7Hm)4c*k3n4oW1%q}@6o3x+PK!WYpYL3b7qtK8@R6bu3_WYM{FKfMB$nw zYRwZf$efr*v`y-WSBrv%D(21A?d+s}Ji z;dn!%0mEQCO4smcpCqn1*u+kB&M2BWhK*-T$(PSoj=Xc-k*}1^4}MV?ww%zuHcP@S zQhar+MtGBo{0+N#`8xbsL<{Gs=qG-}A6y@P*&e31=xE^v>H+(EpJ$G=G9{ z_fF~VmDX?cj_K~5-7PxR59zbyxFcgt;u5Jl$tL!t__NJh#G{nVG^Vx^dVin?L0SxB z2-w=awS8{a^lbMSeS`cXreQ$1cxSR{Cq!-;qK(YJTgr%D3x(d^p(5q%k}v#RB^=3J z4zO&KZR$!EhA4sZgTX24Y+npXPEk87}&7C;bX~-_x@2z=}6H_-J1zQ z>Yg!;nePFL@bZ=3QHeDb={M01q23KbwoiqFlhOIA(?2CPAMx$){Dp)W9pmmCMa+Pn zVfDzt8*SHIDu%4yLHl^I8%Pa_TMV^3$)YHjW?hR7lP zIigc5hw3%+9ccPaahX{+Iz6E6yV39OlrdL~cyC+M$-X=<1)nw(d_6xe<{wVnp4Gvr zvbZ{>_7oHNg<$2aNo0gx50F(|LVH#1c)Rg|?MQi{k|WH=Tqbp)SY5n@Z4KWl&8QV$ z4ezFsO+~{-4Ny2Ko((8ZGYk7tW6LA6J|Uy@t_NloC6lT#He_5}&&hc9__12iPqkR;En!@)I? zcwH5dfS08FokMkBBh)#$fzzU`Zig?Ju<7RS$LfJjxs4%c#LAvI(}iTRSki7zHDva> zkEmvTw@O=?83zk(6?Be)BWGknt=LOZS|HZk~l*!mc3D7gS(Ndp2a?|ZA!#*7ijJC5p8SjmAFs^!qrj)?&%Lh$vG9j=k zmTLD+qocTF$Ji>@QNT0pgQkny(Zj`fqTkTi$Yvzck)1^DTc%_shp?XQhu1&X8h3N# zY_H1rRd8&-Npov}nXN5?{l6 z79r+uQ_i#O$u9Dof?}UT*n$={atc|tow@ZK>`WJwFHb_!rowvoo^t;tw(4m(-$EQP z=IVv~3j*;s91lwhoev00Y>U&a3?I+?L=MOlXV*17d>)}(cm5(#J_pmQK>m}^YqpG^ z$a~gg<1u^K1#rTi@SI`#s2#)}05nsygWxSFQe)H#JHGA6(@)#ks@u$`CVjno`3{33vK5(T{}&xsK2aG2vhpaLX$}BlAVAV zp~h7Zw9JbXL6|v2lqokG8SauGEa_yXk5I-VWcj)ia~w)~gUHF~N^;IF)NMUftxXi$ zP9mDu4xkpRsa6`46DK^!su8WIU_@zDJ2#bB&Zkjt#RDq|)q6@rV~T0k$q#9~La<;b zg)gLok`5I;JGaW_nwjH${ED}^Edqf;gYUI^!Gjxsh_L;P6dZG3k0E%_i8(f0wh)9NB&u>pRT=bj z5WwDu)(=9vud`C!Q2g^RD~5NYbTj)-f_-nWF;w1?)u|uVxVx1$=m_#4mKOO+?%LXj z@o5VP4Xa6zkZl|qtFA?xvQWsfMe!F@o_bN}N7HgD)7Sx#i3JWbl>rE+bTnK`NHJ9e zNA=47JC~>mDCH&fqqED`o335;VXMV)D!|3`G5&are1==bA!Gk5@6BHF{c@_cf(BtI z)nzHZDrS1eEl;B=4YSx-QPRiqI{YVWGoW&W^UcP>+jw)?_hPhGMzZJ#X~v=vH++g9 zEPhz2O(&l31}S%L@H*$^55Yv`%7acNVVdFc^sCHhR`eLS)iLgI8??^xZ$w~kcEF9X zK(|WZR^#i9Rj6m7Zzn9w0Ra}oYe^>|5GP+Gh&NP`A7Zr;BVC}R5i`yHh7%I80I=>p zlU>N2ECs!gZdI^y>s9CsZD^Stfev!j4?>JHFWv7w<9|{&bPEDu);u-B zTdoCSAox5v4K1v0?f+_82xHO>(0)%wBHscSL{zK-8Ln6q{YH}bm0*X%yp+8zCCxK7%9Ke@aTbK5Tp%-D zebjbM*|*y4BRf5(s?$ziZSJ5tZ^y$?amT~4w$kL?x70Lg^#@3GAJ6+4a3~_Orr~x{ zmmWV0tfN&hygML<|A`J4xJN{57>?ovj{j4`L=X^x^b?8l7nS9bT??$Cgcbl>)CVf; zg?aLtUuM`s_l;w;++dF}-LC!7HoMedFEbDv^}&9!@@y-u@=fE17tTP@hjXb>!*Y?P zmz{F4wx=9SSX}W9dATMCc-R?G8u-~M37}X1Xp5!SB%>4kh(f+7HAMd`qA(Z~Xr~vu zz*zVfVunVybEwgMh!NzdR_%c`*mJ+7pvL}PB@*XW7rbI*;MI@Ky7 zX}l@SMOy4>t#Y>{*S9R4A$6A0S;m6Igb3@x8bN1)Y+iyssU?*ARcJvoEyB~x zTss}21GkN%C6F5hn?e}U4TKAG)?(~LO`E=LD@?FL^X)n$r9~m09G>q$uN&k^wvdP6 z=&6(4sp^t7&3o7kNc0>rhsV^a2o~`~iC&r0Sr7=}R^g=wci&u5y0nT-zH~O6ZGxH- zKNCvL|3|`&OdKrD6D>>6-sD!Zv968ASYVlxNvv01Km;A|lXuqo!{TGwmu(oQb2j5at;d!5Gs5Op3NdQqC~*7nz}qy)*bV zQIuh?jAjAH-}GSDzT;UmcHUViIc5(gGp@r~lwhk7Cn$=ta^cS!A$X&)h3K$_gjqqv zB*ycwh4`?AkXu2>F2bc3Vb_YVYQJ%*;L&HBQM8lmT8(aj`lM#p*srt$zyLlvetIoeE;pQHIgl>UKc zqK%CRINu}~QE`kq8RrtkZ)iChQ89EjtEMO~KdCse^%Qm%57-og*)G*`kp@DDrP92S z6Mg?$qzFVt!*9lQy5uYd>IWvj&Y5`2NqXopf)SWE>(JS41k)Csp{UEWN9Nu#d9X_v zO>P<%p8krhT^LX7`(7MV5TS^S+<8C#($%T~xMW`;@r6~L3Bc663}=lSJ)4*YRUj~8_-p`zvH*RkJ4Jf~T2s==wuYx@KBmU`npI={$& z7ImpeT1J8918Wu;R3(k^B$5uaQMRckhN0~Q_3ta_+1!fw_u@WX_!_bGV2-sw8bSC1 zkT4n}lrRTi4PKZi>i0xnJ22b#d+e{BC+knRa2d<5>|<>}V-oX9a}Z(`#-$-8tM0lm zVCU6gj)D8&aa+ItghJ7xeW>+%$v-&pc406&KFq3id5;hv;y#CAhgh&dv;@-76-Pl zG*T7jyici9xXphSeSkJK(j=dE#rM_`6I)3+b~x$sv7#x#m`jN?Xd}PLM&6;E1>cV3 zFRXDp128%RXgG+RjAKweoN!C_ub=)(!#IsVTO5J)T{O!C>LM?4lx~Tsl_7NjkQM{h z-GDZaq5@XKN6n3$v&3jKx5#Ip9%N0895k42QUwb@Ecj09$x`p6WW0%>PZ&#ok(5;! z<;MWFn9$jc>s797>S?sP5`>xZU$a{8TQrrf3`fO(UODi(Fyi&}l;$3}3L749MM(Zac34h#*JW~6)cF^47ex=#tE9sl3_2xs{ z9=qfFgt;U2eDpzxZWI`UeZu-=b`%Bofu^kYFvayH)a*hk4gVm-U6aBAT7O_?uTA0F z1)75Qw3Wy}>0pvMwEZ^7K;g?*IVFPvKp;%+RrwG@3KW%W7(T2+nky;L{m)e03^f_9 z2=YgGBCvP-!Vh_%8B#?sEEft~bD`6Rsyl&fb< zCW~NC8lSDYeDc%?J3(0hT?pWOJEtiUSx@m+!oVgyt&Hnbn2PcAs7?Qet9J~ptn2!{ zW81dfv2EM7ZS6Q6yJNd!+qP{d9iwBMT-W`)=ef^YwLYy{dsnTRbB!^F{y#I~=B%qa zyW^*Hdim#m6C#8S@+O!CI%A?8r~R~ zxHHWY8D1yzu_#-t3;Dk^DFav-5CdU*)@B9tKd{)fv23qG<75N(aGV(dV@#{M2!A8n zvubvC%r3Oe>T-#FVZdV_OzVE-*z0-YyXLRv6Ld!k*lFIfB;&>O7;g1~UeSzrc8A)r5%nrl3bE%0H)5hUP<%kl|6G(P z{A7Q1%3!YeuqNgWGS!vG`NJP#4eGP&6D5XD@Aqd*O}VcIunh{K4ZFQtU!ryyc$>0b z=r~3E3=^!!gyOLDO}-o|k6%h>DX4v~013MYayqJ0d6KaBVryVrKX!ZDdsXx4O=`b} zuk$1Pp+|Apo5F;ovBsQ;(R$P*AUpi`QF>?;{u%%ICGQq?Hf{%w>s@&C2s0l+7XkrR*Uz1<12dVe&{Y<0ZGwl$>^RhD6 zXdgs)qf=>DiM?yS(>o|7f^%iTQJ4>{>qThVvdfb**;!O3EoeY7We8-6%%VA3WW{M! z^%(tzC9M?JQMvSr`7;V*Ex84OfxG3kjgQ9uRLt)>;M_B6HLE7c4_F3 zyFP_K+QF0H;jzo@NqKjvk2QH6MB>kJcS%q-Yy;mj0|Od4>rCR3<1ppbZ8r^J$!alz zXKShB&hhpuLr0T->7aOnk-6oELjnDMO%h@3>&M79&gJ1b=y3?Km!qJBG#Jfa0PX-W z&y-LUK>5lf?OtQ|m%u@%YmPh2Y_Gthf?m_mJVdLTuv^OlqIC2y6MHa)!%wW?|_VsU!0@$-`1y`z8m+X@KLu$HU9X0?UNIPBMb0%g5Jncf$c zjVP?wA}DKS|NK&qT0pU3Xj{Z{4`-9e?dKK*i1DM?Bn3_;{O1jWoeJ6~>9fcT^riABLv$`kjE5xHWE)-g)gH9V*Z%(-c)2Z+~`tayX^g zk#PB9g5EismK9Um&7_{&teCy%*AV^Ggq>oD5Wg&@sN=t);O7mPxEDeW-2WHK9}h>Irv}JHR(lO^b$2X#vDD{F?l-VaXOImJ zGy%nL^tm08g6|A7+MEyCH6>oyM#3pifhcWDDHP>ZW(uF!`?=F#D(S5~SvLqoWZmfV zts(`*>d#z5WP@RXVi7IL_ZzT%z}be0)8LHw{sQ`K9O>;FlR0xlF7WYM0-Y}Q6)%{_ z+;7(wtEJ{F8QBJUHZAM&AAR`3jtaE{G-wOSKs<%$tv-%c)BFwso47NWBLry&(+oq5 z1T3TDU5L1GetI(u12q&K1;MXvE>F8HYCiG=&K3rO_?th5_`Hflvce{L0r=@!F&Rvd zY@P%eI`GlRu!faoq2|k<7qWI1GHEA5-yX=Gt@iSZZnNPEZ7gSm$ugCLkH^SOooz3r;}Y zv<>no)S=ME?2)WE52_>vLLqawZhs~8PMN}TtCP`WZ;YH<<7#Ltdl2v5tMGsF^S&N?|W;fNq!Z^w$HE0 z_!yw!3z;$Wf$?>$^F9&I9R=Y3zQ$-Nu8a6aI}VuT9DBV1UdxMzJEo2~dA4oC)K9X% zEoTNdE68G*(@9jt?oJeLRDkO^!u_k#H=!OWu=+gapa|rxrDjS%?j%9tN03GjqypcA z_(}sNm7H>AatfOazP9$^oz0;D4TA7D3Q~Tgj+cpG0nJ^&6@v z-1ReSPh+8qyzdBC3DFlLB8^-fTO9>>krD6Vaci;w7&V{C*6(}Ix7QT-U)sNKcj5@Y zo{A0(TEO`AVm#6h0P%{97+>P_@1qbyW=YSvDz}f4y!c--eH*Ft+n=EyYIl!Ii;8i> zLQ8oP5n}n>`FZSR>&{`&O~wLvQIO}gYDdg;wb&S|aK%ur5>O$k5gaO~7fH9fEXt>J zfyT-J9_2u98Uj^!#5{c&=s#zkiZ6Tt-n1#x?@U4#jY7MN5R^;lEo(f~P_Ih7LPOM0 zk*(b=YaQiiN1nWuO4MlN&cFD7Q(YH_y~^?~-th~{v0W05fa0$VG=2?lYvh&u)|}&n z6VQ?auK>MS@U%>_Q)Z&0_U--5fjY2&8J;-+7)?sT9EeOnuEjN<3?Ww6!^aZXeL=pJ zHo;{GaF6lA%}9f!y#c+G5RB{Ms$AfzCzx@M7)NHO1$i>TCL#@-?IbD(fBeqVC!s|f zF*-H!WKf=%{3zD4VuwAMA&eRBt@qDg%AaNoOISz2Ehon)C$IY(mGJYC^pP<1(K;m_ z5ECKQeH9v*>_R+_|0vyyOLw(Gz3x1?3ACn73b(A@i)4o$J+Ho@>JlwBgrz3$+|T6R zl7ihPn3Y$^%=Y2tdv6d`iC_i& z{!0+{OE6Rb|9-*JoWe#DTvdYff{^wCpo9LH?|folG?TJAwa7vZPcJD#J3k8{wl0nf zFG7zK_Pao?eVwIwP7y7C4>iTppxP3PPR2Sij804>8A!)U4weKMU$Pg2YEhUS1VRT4 zPCYcu)hmeTyIEjW{bvFhy7HzS#qHNX52|{>GeMe;3Y)8|;Buvlm5x(tm2%Z3AQ=1k zYQ^oymhe3T0fzIOPeB7OrDtSJc_~oYr@FO|yuW7IUMXflGkwc#<2jK*2k@{J zM?cynMcHCEf=gqjQNBAYt96G_jC(~Bda$WhkiN}PmCe}>fe1kyXk1ncAb^@FgQb(@ zzM9wZ(1N}<-5p&=+%mft- z1QAh?Kp*y&ThnLuxG0h?C`uN9^CE^VB_hGhDs#4Jc(vVNDc0{=8Mlr{ zt8&AdvwYtxA_d9anq#{U=oyIk>V856z)y`Yi%{5|kbe0>jDbg&r{p* z{>*2)E#4f@mfyd3c(T7a?<5-A_4vL~RrEcXJ7q{o*bDi)FGWcUKj1+kF;*QR2vZxu z7>S+%%EHl^bGo~qdS3)B3+Kr7ud|1e?8_R$D67;KNKt0MA^t~u7U`%F@7=~-G0nOA zuMR#DgaA@5S(YPrZKiCfg-Ts+oMSDCMXbuKsmnEx4|-1CKz*lkf^?X{kw2uiOe<$t%{N0 zyN#Pqj{`gI!PS?*a=Z}uZP)lAPlN+%uCqd%=RanhAxCZ2jh=x(OM3($Dx7yFKA4^e zA@_3Q`bGg%+HE^OoebIAlLN5^&#w~i)!0u=GT#|Hkt;eg4|{~jCgru&u@i>YNA z{EwvA>*o3};9LAr8S+1hwTevKxPbrYAE&5pIB%$;>5U}P>nM#@$>r0_tqYn=kRx=> z$w)%)9hYzLVlgi~0?Q|hR4ga(ChA&0WAhU+PCl?Rp>3z_6=E$XplcL`q0Kz&Hb3gy zTsO5moEQl706Aiuvk%YJ*6IcZKmB!HK6ak&_rfJ-eK-n=iF)oGCf5Zluh3#tYilAi z5tjMWIo`>icl%P|1`@J&`Y=POQrb7FNq9vNLaeBL(de0c`H&N3fE1$(+$5<17iw=D zCjE8HRG`rI&$EAYZ&rOM-uyGmyamIzww9bwbDcgXd-YPJYn?!=C^f%hBD)4zWX+?n ztq%PH-_}K6tH?*Sux9y5yF`%20IY-3;xe8yewzhU^1x28)J9!V+GNbTZfToP)zl52yrg| zovH>Q9$kU;o(N<`tN9)&q|NhcpAILD1Y=BPJ{U0dS zOwtlF7etDt<~Jumf%~d&`E6(I#)>9so_PKWd)=glN7l7(Am9(gN1>~(-Df!m7{YC% zIAWh)zaPY&eEKtbC?%7>vQ5(=Gc9CDTME3sfI+=a`-)HA^d@qAaz|Ujao{ z_pK48O`SNSZT227kcRD&zQtiofqoK`xy|B+1Qh%uNGMK#SOIvBJgRR(Db0$vG&c{> zwB+A2tOXQV*l6nHW3L?3pkPs)O}9j`saBrKl4P7z7d5nSwMip>*zQriB=qQy1Wf00 zw6JotwDGazn?)N#T`nLK;OiTAst-fN43$stVx1Yok7mR(?dn+Pum}4;E*a)70omHV^CP6I`#BHIX`B_@Sj(8#_6qT+}qI-}R8Bzi`_!PsKg|;z|DZBmM6$ z`5(1^*k4XEi{` z!OxQm`{pT~BoaD+IKODuBkh`u|L|RxoXg=(tmE0&H zl^HKzx|Mz}Ho!x8N%g=ESi|zBAizgiffl4=|91&lqz7j5dsb(BaY}4!PGSS*QRn!~ zW5LE~8F-+g?owffF&6Eds`xT1Iqs4rdHx?289=1@pc|hDdkv<`0UC1T54j5V)G=D; zj3BXSAmO@aJ1?nN40twNxijtGX)`LOi+9|lrEq*b%+m=xF;g-ipi^Kl4=W~V+Um+; z&dhx}=xU5u5p^<9Uc1gD&6Ehl$ ze*q=h3{s?wxa0fd$9iiTWq(zsl9X^ocXbEw*X7A!XW;F4%$`erR0OLZ^=lL#a-`4w z4N3$g=q%^Z93XMt+OC87zvY_h84OiAXTGO2l8Y`iS!u~wTox{7*W6e1)>#Cg-Jqk9 zmduPdN?$5acaZBWhZ~bOp|!GDjh6GUI0L3HhEVDk`R~Ssm8ME$&}keC)vn~sEo0u) zg7w<1KXOugkjgyKto5M~TjBktWV(Y0Fb{$#)U$RrfXlMs~`WyNmQE zOOsh>deYi|Z}UlDZ^WfKYVP*_;$F2HMTFj=mYd% zdT~w_J6-P5g}DOXHlEOFM022aC_KZ7)>flvT0Pfm>6UYNO%yP9O5Iu zvw!j9SI@g>w~;3mWi*lGXp(Sw&;WjLku$YlM$WO_X~&4~dRj)g^oW^YJ6wV0E%O!f zdlVSq)}W^O34Qb+=vDl=@X)a0nV1G{8H?7Hq?!nO(eeq!NipZYpnpmA_^pTTr(%fM zGu>dH@+|Pm@726Q@Ol~-0=f&hL~_kUocH{hGdwGdu?v9jSSIQw ztqAo6CY{WQK_aaieo6 zePX;}*fme5R_M9GDBdt1_#Ej@r~XW)TLi5fHBVc365C_(tsa8<7L*r!Vg5BFt{@}a zm$De&h(CuVZ87oTJ>iXE%hG^Ts|*tg*Hve?ybZilb{$q7rA}#-G}A$fF(DG1yj00f z0i%&l8&0n4D<{emzPK~DwyQ~w` zb1Qjv*}32PUXdf2a!wZHC*;z%&I4#NSxqXuMh$=nb?!aEc$LP}^mV}2fz$H_$1~3) zUX2N(eO~4>To-jpx=PGAnwUTok&OteC@UWH*Tz%2c3sq<800 zP-PHR@;bfBXe0U`C}>V$kl6SDQnFnr=vCoW?<9uIRAY*)^UwJGRU4wtNIjMW%2oOs zw)U}A(%RVA!5eOF&UN<08VWnQ0T{ zRN?#`CNGNohZLWz;s~`HW3V~Gxy6rLVx7s){KT|$s0gj0ZV832vPI`O&s1p3P}fb5 z3>A*0=q5+IS37!im>^gAQ<0y#ntdb@7T|sY1up!&XihQJ-~9~xB~D+cuCSuKrjM2R zPdXO4>r(-cMF+$}ysL}$uUQvwllUklY@sY@UcYSls~0*HQ2i!L2=h5(lYe@n2g0M& zVDr|Nd&sc2f{K3ae1{Alp#RK+{;$#hZ+GyIY;3GQ((=bQs0x@I2#6X8sKu|HNeCL= z>HEkh@}+FM`>i>=mRiS!f}d&_3jv#oA4&|BY8yk%1uy^u|5g#UVAm8K5)0e(IdPs& z_i-@q_WOEF{%XAtq6ytl7b$q|^Y7fvP@DFyV# zc@>DJ*d|%sWx}~Y_^6(WXu*UvbuU7wcFD~XYe*kTIqVzA5)E3aiDm|8q*fjVk!+J>=O(d8jk>Y(NIxtM?C9(n{$YJOI=|AwuLP-|Z$z>*(;7GZ%5;En0`#zE)`!G2wb zf^HI=Q(5mgOg-Q+srB@WPJjMZo_Ytq;)&8H+aO_GwV+eltb_S}^MIIWhC(+4+L%A2 z3Y=kqp`M}zr1N-vUU4cl3S@qy1KmWX23599#H{rQ(>jOXx@6ZG8j%gZ#IE}YqW+}| z{@*42f3(~GS?m_4IO?4LL71y1Q-9OU|J4-!oegmX{2#EnKq3d!|J?P>r!EBkmtLXG z>nIWZEnuArle)L}eV7O+f*=IgX`o7=`BQ*lB#Lb(7?&HWKm$Y51fhvC(S@=jNmEf( z#Y~vP!A{t>cjUh|y~DhK`m4*(|HbopZ4xLiEiLt6{~-iT=D(WV;c>Ou_SEK)lil<6 zc0eD1>xerW$g|HzukQhRIv5>q4r)2jMP!_oln165hn=5}xKB&R8s7-O8eDm%_{sTe zaIpSa_U8;b^p3P9JW&K2^p2EfYHi;vhBAKmMzwwVbkOSQcy+r*-C^OPJ%dtysIx!YCqRfsrN ze<{8N{lw4da|O@#+94PK^jo9SgMPU&Za&p z(vQYrdO5abH66?y@L~h9AtJ_k1oug#(RquaaYK>9WRs|rqh^|MyZ4y%D#OBh}ve2(U!;>E-86hT7BT+K>Tb@TMqZ>#(C!bMuIf z4R!5ditf}~Y=#NI=`cBVZh?e2B^#X54K zqy&i6TP$qo4QAI1$<-92-GpeqG-f(eGtnDV{Gr4ev4LIerL-oNHooyDDE@(^KkF(W zV!J<~29_RKMDD{+SspZ?1)NmU`TZ4qNTA1#AvDC^O2^{?0}^XR_8 zm~)&dZGDX?s+)5hYk%*WAP+Xn6%JJve1(1p`+3aJCo1UDKt$z07JDadOy8cgIJ*$V zU*9b}58`{Xuu6c<3lar~--p@e9+EL?^K1~m9X@jg(34nZjY1mBE+X~u4fO9#3NdK6 zB8mpSlhlD2tZYS*t`iN~ek%S9}>K6J4Ua@>;sX^c!AW!rifnv6eSG`BM z`+?)P>iNSn8WwJ5Hig3urQrf=iuDQ4D8nxQtAEzOl23FuM9RNlH@N@CM=Cl$K^RvY z5m!h6&=GikK~b_sbye>~SGyru3(aeqB-y~TYPL|j2Ip_|)LG<1oy|&))0da}DfHt5 z6mAcPTtK#-&hU;^DN$UqJ77JhI|newu=grpSpb{=Gwibq>u+zD*i-k9Xw1kTf2tt0 zwKx1l-iXP*T%K+8^B-NwL+-Vo53$=T(0sRsc|th+BcvQs*=6@=q=2>O_g;p+{_%nT zYped-2mTWheLO%j$bW+?B*6cpuFoX*EwL|i3Wo-04A=(`g+Z5&vaO24L0xE^E5bm5 zyIwFW68V+g&J%UlAS((-F>C=yzMY(JBqOogG`BFHIpW#Lk^y}1e*x#kdShUyPu5!> zP)M^!dfFOIQH-^R)`!BwqKBQCT&2F|b0;PkYFf!0K?t9taxBk#@S%G;VMPjd@L;mf zU^4^SDGQZjj7&@~|9T=p?Of6xb#&a7T}KUvT85JipM}B|`H#&~D_pSUOb!!>wC6uY zcz4r-&PBwvllYEw@Jk)9j~{#zAQrz6KrMvv2H%Gjguv%YB;?KfDaw*|YoDn66BTb$ z9u&sxE!SG1p=-0d66uDg?>N<|Fby~4v@tSG2=TruRZ|1wF zafZWziwykra@@7|KnyL#Uam%*|CNBD$`baHaLptJcaADAEqw}_UP z4I}%7r1CILq_7T>jU)NJql>*^)Yo{$emjSi&yYzQ{aS{@2C+yhbPF^!jr7`tNK6Ve zeJErn$_BPY$*LPy>kd$}lJXPr2D7lG9CvYk_lIWY2$|etxI*6b<1V!_pz9>bgao~@ zXvBT9Eq+Di=q7PtQ=sIsfXox2GJa&LGw63Fq1{KeCojSxr}O&!mnZhWTkQY8`6S&e z$^SWK*`)67{eu%Lg1|@}lfe-H5Y7^+{ji}S7D7y+@iIT@`rC;=HJy|F#S4A|GBe&9 zoPwgeD1%m|-Fm!V4gRFN=}|X5dp|;`N>U=?e?fa7!7Ro9Vmw|bY5lc|b@{X?f$jdu z>m(eIzAUhb(bIO8d09{3r(rG@*q2mK5KN6B?WR*t(;F6A`VBS7L zm6LTd1Y7apRmU)hk2(m6@=%oyD_0G>TP>=K=QDe<`C7j79ai8R^B~Qi(iICANtcy7 zH69oeL5$n;p~}D5gT^7qDHy9YR{@zS956H;E>C+*UNqu}mAl9fg%D9mCps5#oD1)L zBz~?zWW?^|&u=Y!@ntvxz@_x8^CYFCL5mz%8^dL4@czBKVlk(bSY|WRMGdFf=gvgJNq;ZvFj&hYrk|g!m)_b=je*(;kT@sF$G$@Yo+30X z?HS*Rf3xm+_u!nKw%~MN)+B0;?_(TIm}XY54d9Lu3SQ>5PZTDWTi_Z%dBM{PoeXZc z-p2@Ao&zk|Ub7#8B3K_1#;ZlR7^y@)1Xz1EcD?=!eEh%b`2U=({xL5IW~CIVMis<@ z0L(DBkbHPFBuh)sWp=W_Mosig3@|b_JE%rq*D;wDQ)gHe3YN+}gh)UyBtQrdSaYhZ zNT~aFiLf}w$3nItV7A@2&EqH;pyva@5XZ(zcQ{&?KZZ0gUH)O%2@xFs zsl)PKh{HH7WyHn#MRvp&Z%Z?tF-q?lfG5Kp$xXm~x*U7fzG;L-y`jH-B#d`l*!6Dw zOvf~MwNXuPjxD}WwMD^>-6&QwjLnd0m#Puuo$_{WzLcNcU2l;-U2?W@$G5&X7~3`D zNKj(I?YmRIbs4%l1bZa_?b$0e+64P7qbyj8)@}|EIjtOY{2=+F$DYN#Sd#4n zKP^j?ilCglw99os^+OwCdQ>+|yl@H6eDmF|LYLb}_bBrdIlGg&@15i+>Bu(Nx$UYc zvRz6-n1CaD=xrl#hUk3Fq2%iXxYEuI3VopF7+9vOPXtHN%%MAP1g(zDJkx*rtg3RW z&~VMXL@S52qA!Aupf#T`!x<2XxV)?cy&|)FmUmMkaL7>-DQPTVj>Xk@1v=ao(3g50 zZsunvig7rClR9HB-G&CD5Gajkv#kp3s6Rp zv#ki`W@}~}8z_89@~yZI1|19CFf-mh-tmv0zP2`BmUog;1Xa6wt5P;W9a7&ye?84H z*|06~rgpHpHk?&`Il4^)+E6~`KIq%}b$ng2!{o#5TYYj)Zg8kIwe|JB?q2%E>slYs zJWe%;g#>6H5`HB~{E~k}e^}raba1hh$lRPV{fk;lIBGjUROQr^U9yt_-6^qbcp63w z2RfeUC>t0fY8aP?j5nJ;MzNtV>yAS&p_KGZ$a!U(Q{iO!efr#8g9mMEZ6s*jL8rabK_Nb06+GjYXdON^xsYAXprg$IV- zBoZ#EM08Xk_l+6{;2SfF>P66AT>RKCmC+K#Bw%Nw*cOac8DI`6B$fe3yzMS*K`1AA zf^0X)7kI+idTKP1%j6BawbkSqH+3<6#AA!(EPE$Z85lq9W6oFoX{7dJySRbib>c?^ zwc4_3Fpn5v^9D@?8ahCmT4qnAzQ4XX7+z^q&le3kQyEp1+3XWJRUXq`BmsNROSJE% z?(X&Q-?;1_3xW0>MlnGDw|DqA0-LBIc7gmi)*6Gl2dp9i0pT-#Ltm&cQ}b##pi-|p zxrkfn+aT_c{(VNi318_09SCTaEfsj4njkfxoEsxGUWE?=P^JBky3>+?g*meW^F0(X z5>fmv5V8n{aEchHXc7`rMra}sgzcHk9f~*wi_g|AZI>DM8Lp=+SOY@;!yhE=vni#u z`;_Ln{cSLSH$S|aq;NHXt=^=fvHJyjUah@%-JE`K|GWD|U0DT{R>fwyRP!)a7IcQY zTRN?*D#yB9m^M-VV@a1BK``*0P2HT4dr;l+6WY#R1#;g&*;MFWVXN`t^7sI85EJbKd{HJuFHAXVMzxGfSu>FA>^OJb!%3FEBhD%#8$Fmoy z!`-dgyvM5-i^J{x(mO@y4ql%5HqDd{+Onxh>g*{-^byR{b9uH~`c+DaLWUg(5=u@r zGT8>87@Bso#&J-=!FX##=?wDSmwrSP(&>fu-IWayW*vY2BKeqGsGhcWrw_8K}I%)3B>;X7b;PeFXwh9KY zs=#q)$KS1&);2Igh^)^4`)0)V-6{0i^ezlg=;!AhUEa=xRuy|1R#)LY3l&-+f-Djz z*67#-6hh*+B=;!V&}IalR8w46Xz!*cfxpn1FtxSOC@G4l!J+9FWpBxskq^vYbNkyZ z1|C1AqQ%{{+Q!y`#Y z%*$JC_dI32a0>Xi7Y>gVMIG=juIEkgwKX&`g$lQ{Jqs~l$6ENtp3zGz^KRSi{d%Fq zNPAh!w}3t{?0Q2&7ew|`@+C*h%$!pJ^qWxH!_A6WR<~S{h@0eQYi3B3)ZQ)MBtVt@ z;H@ohykswyN;o5EVDM@Lo6{DBjB+&Vi~zn)V$`%Lzh;&DuzfpF@4>}>b0qko^Y)lo zgfeENNKl#r4_t3VG%SpmmA?;zy%)w%Dc;V%G#?AF%w@0Ra9*JrTPU>&BY~y>7xlIs zqAJ!j$R+7()X(_%AolV!jB=Lq`IIsHkQA$^O=2ePRJ0NYPUFLtI_IkxQCDaS+JTBd zLf!=xBDYs$@Zb;qbDYT;E2w%kHhlEGg@4Zs&3?;m!y!-QbIO)L@5eB~QDPgkww?L9 zhrIqh-KCd3SAh75b;#V^1>gIguwhW}>+Z8vZyR2RLOx0avdK<`T7|Ai{ zW=0fen;%Y_%I6e=$7z>qfR}+e15|ji<>7*0#hW3sLceLJkIG107AkR2)|Zbl+Fodm z1r}11Ka;`~v?*?NmH?`1g%Cs@Zs6*2yTwoz!@c-kCwAu1{9)#FddcQL!_$_w9qCIC z)?28?%mRtc&>VxDSn1V(s>x0Q8hA6FYNyd|br#`eb}UrC&ab~DfLCuzqCAOxtfi>) z{a-NW>j+!!=~)b5cbBa;^us&_f0sSZwO#}yrbK6;_s)s`b~?>7p71|ZT_9{g7x!$b z_GQHDim?nIM@NOI6mhK8i>L5Qg~#KVN}BC8t#L{PQlBjec2HW125gz{ZX|M(914E z1Bks=deQGzIPz=cP&;gt#fo^!K+ciN7Jd>k$t1X_TdQkB#8LNdkSs==4^LB%POYg~ z_F?-j6fBW2`{@_!oDI-)rEZtrVOsV`KP~u&UTb&>2;|)W15$cM5t-$%xuRb!(ZUFo zUWoEuh^-LDx=O6i&2 zQGCgKM^EV?2TZlR@X!EKdrI$~LW?a-uONL9tB)Is!+zyMu{OUWJrlbNyS~vW)1?3g z1=L?C8*|S(8%%%Gd?n|qJX_FwWdZVU?rC~@88e87c)e5}2vPuJJ<883G+$wWs4U^5 zom&~l`36cbnlF8_x8Q)8(Zv3;cZr031!tlojE;atfYNYka~JAu!Mq?x$k#5UWt~J~GRE)XQt1Vyi?B zhBBk=vUPp?K@%0wgREzOV4{&*6^3Ae~LndYvkwed8@i# zD({W`fB~A^y#=Xz+-$}&2Ism9px-JCsehb1cxm{UoPSYx$nDi`mxd}C<9fqBpv>ljiWg zZdKL+dRNsjqpUjeR=(Ry+GD4y{$}If-5k+xDDRkd6AhC0t~-Id8K<-85Jp+^IggBA z0L;^;&YG?7hT+~#)zze96Lh7#PHsc@%~RqgE#=jR$o)ieUY4tCY}VEtz4UBsOzxN4 z!(vNfVstjNTdEy&Hx{?+x~mJztI!|QyWC|;9%S?EQO$lE@kVG07aA9d{aCnCjMP7iwVuUXUw@#XeZFcG*L>X z{<7df+1hZaheEs0EnBz2i4=g1a0_<{y`ovFz90mIzB8;zdX zW@*5I?$LeW3$-m%)HdtN0y&6>y^2!+il65wX`Oe1w8I&4Bvj)S@Nj9pYS{K8bryye z9<<6bopU(NyX=OsR?fVO0!YhK8q4Zc01!q?Qc63~q-N!`2o}jRiKbSD7kO6wcvC>!|sK1*3Tbf^pxCy}U8QpFFdI5d=UTe+KD%$b=GvS2^7RJLe~ z>=T7k;b03>72K&}+*1tW$b?jLsX&(rB+H->%%cI2auex3h<4xLdS7pWtc zP@*0o$@VSsRj+u+rKIl^QUa`CWZ^Z0Bqt!uLJ$ZjYZ=RXl?)JwFmtbn|M^5kVly1Z8s^+KTWC2;0d5#`>Gnd$g7yoq!2m3~j6O;Q! zFm$1f9KS=SZ*@n@*?t!pm>gf+8GF=H+G}Q^+kjx_a62B%6iUlw``;+d2y+)Esuu-G zs)>1A9b<7#_m8t=QPz%6SiixMQg*SuX0O0!uS2oo937QlAYr_YPuk@O5IA{tEu)Sa zpeFr@}=hL?za~Py+uYPcV>G>P%(xx5pn&L(vF{A^~28graJh9Wxm+SRjMs? zokWS|2qFY`$xcyF77VEfalF#jnPgV58zX36RK_vxo5$f4^%4?GIKfhD-(xIh;{r;% z1d=TW+-d{qqvlba42lqG$lhB4(}&zn799~aN2)Jq`!bfwOgc;qRtF>?L%$O<`GU@PWX^fs8;KzMu#XnlyqP9HkkAzARzY4EdrSbIcyNqVpNS=)s~V122$<0EMa9 z9HJX#51fI;gv%YdW{ezyEYVKY?y<*sad|BPSX4y`SnPQtd(2N;9q|1vESbGjj(?a{ z%Ox5zJFgSeHm|T3niWpf(riWMfF+hPXKB(jkJKAwCdOXk_C8KE81e)n?PAzo0jVZM zr5J?-iP2mBj+-+sSspiefH;gFITa3D(mgD|iL*!_Y?jK(BcI3gLTOaZFq@}Sm>uc| z#8;A?qEu7_S}l84RK}r?SQbao#4h}vCP746VV-roDccsy+?;Ucok;G_ z+DfL?#AQdPL0t6QGAWJhgtq1twLxiY-n|23yCjy)b~FRwmDY&Pek%6Ge32KFIUyas z4AhbW0Ett7G%#tL8H9yB?nd4A9zojjPb#|aTf>Ry8@wf2WBA+n1V-QSr0XW=V)8OZErvL z1$)FL-{9fiTOQQBp&u0HT52ERhG*ul178q(pCvB%!OK#Ayppb~JSa4NfqEDNV7qE# zr(Ch|j-a!i^)o=M4TevJ2Ho1^x_X(jyNx@>YY%7!@}9{mDZ)R4C$6Y9S5048IG-sl zEC@c}e0^~fa}mGKice#Khf*C$**gQ0cz^Woa)GgP&p388PO(G%^JWqEIA%Z|;A*`W zQU}ZZT{9>Ok1Bzlx}{kr-Id(|Sh|l3v$3E|aIta%1WpNrneu;8eLv73&AD@6HL|c- z6-Y$&*OXt-yq=T_?wqW*<@&~x7Il+Ul3k*;v8qXe)x(mSnyv~3iyw3k%B5%9&A7yJ zD6O351Gih_%=^HXuk(AItgrD3Vo{|CK_9}lROxcD7=|3M{c(4c(?B%<8xrK9Tyd|v zy3CZ*wN^-(MmlodV;KeLLBZH-j}x?G;Zz*yS{uG7B-7X(jy;$&b&*l}bNhTK_y z%RVs_jdRq}d{RJQKI}iOlnpU|UewIHPOD7QYR475!}SMGux6(MsOZPP2)Zq{#tep1 z4~weM;}(AN9ZIY;XFAg$sHagdcY#pMD_gCLIUjNAZFyyFhnN6d`j)}Eiub9kP7#za z*TLKnm1dwR>nA}`2hEa4V5xIytsoc+jH^qS^<|GbL)34SRmz1NmEgd)p^L};X8%y- zqdV;lp^Pv@p%hMoQ=HLZ`D9c8}jOq%T{0-MB`w{V-9?`fE1ERrRt zXFj+#WysPlF&z&01Ov8H&Kc9C4-b?KEwne zW!i-9@kDYp;bXlf~Wx8y4B!|$4}b0(U86SZxF znzG_yos{MsrkD6>+8Nx_yy=xoNOB^XR7G#T+U_h(t<{)h$b;cyql&`QdgvQC=Vl-u z>oup|`Lh9?kdvogV1jE`A8Mjj@uE;?HX4G+7vQ5Y08*K`5?1*S(a$!Zq=7X*ONn-h zn1<)s+5?f6b$(Ujx0z|%Pm$JQSUlJaau03;1^#Z}@y-aC=c=20Tr2jS3%vbBv>j@v zG%6=mUFLO<<&AC@-q>s1REAS})W>Y25f6SVTxq5=j*j}bn8iVrg`!WSqr0k~sVyS= zhyWGh9dVaugpn*56<>-|M>JPX81681yi1nL<{7x$q#56Rw&%$ko163w$4QSb0pBpE zr`#yrCpldPUv1}XFL-DC09}KuR$cZgCFdU-sx?eIC&Q`dduYK^RH;&ZX8Ueyo;A&X zQw-Ro(|)^Z5%h`q^NKmVAbk2l0=i=ijsddw6E#U+Z6ark^3d)4AK<%*b&T?>zMKcg z6BNxa>h2b+ot7Y($X>kiNZk{vlu8DD@DgWL;n1%=bqg$_9QY;aV}`yBqWfwRXLj>F zdKu>DqyV#^ZKc^??RWBT2YU0Xv@fv0VO0KU?PNb2#9DGa#uTumL}d;_kZjz9m+NB?gc`M>kg|J1Ua z_RF5^zq8b9q(DGIsR{eUxZe%LC|C@$>QRVlw{4S`Mf&Y_+CQs$t zXy;T_wy zZQItwwrz9r<$2z!^PcmauB)oMx~qTft2cVDz1E_nwDyjf1*xV!Q5BLBt*zRz5+7h8$NTQj z?3FGPs>+@+jjWnQXJh#$;RQt7v?BGgvf9%PdP>Zkk~)wHSW{9?LRSU@UJeOc9RD2fbm95R+Az z=l^uX|7+%-O;Gj!>z6`t+}s!aJV62m{pnr&h}JU<^Dwm*BN6uigF^pokXAAU>?9CK zgCE%1meGz(YQHk}Z-9oSI*K`}uU;7F01);;z^aspt%P5jy&sw;D>j;Fy+$Qo0`Px2 z8_a?h)mv4%S7z1Av0q@Ga&S3La;ddn5MS#A^9vtxA*3d;O7dQu9xq-u&beMYrmqJ# zxd7lg6a>800BQIx|4CPiK{UOtGe10H`{)g%eLSt%5Jr1RKv&xCa1oxe+nAe@3R-qGQCOu3Y)y)_zJ$?5)u*NR7(I46G;SIMY#y3nR#UP0*dbFg+A~;PLCuHC1jDH}SLbAcJrbMY|Z%lH_+- z{^YrdJGNa5X@2+^(|AHY+dm*QpFPIVRiChbn9CZwAE>5K|Fpm9F`d~dN27>$IlD6O zHIt)-VR?9ygNuEV`j=~i!&gf3s-#Ny;~gahC}L<>0sF94M2c8wuIVq;f|*QYi3k4% z+-MylOJdHovPN&)C;~%$MM+b8R%j-ftbW*_V=dDc2UKafJep-*-xb?x5PmVyvC78dAi_B+VUT={8?rdDa8;=k z?XF>U0Zzx+jL-0pg+oea_(JbTW|3uKuMv>Tm3c!WMiOesOI4vh^J)X<<_&0z4MV5lf zdrzeSqNI{4R6rDr_0N<3T5kwlBLvivA4eUIi#U3MDgC}#UX@LlwJ|yO?-|OEk{%S> zT+gIQ(%v?JY^K3!Z|2(=c2Y)-LX?PR6V@ml{)D2cE8FYa!v@vGf~&!!tO@>MNku8% z7$6dPL1Za$EdDz3%f6PrM&viecIyEMFu{I`pZYG+?!mBl8=V173zLh=z0=`)2k2%0 z=&$EJ6yWaHBtv1t0W9|7R?FXtMUsm7JY#W67zl$Mm!VGQ)`es>Bl~k@B@5Gbx?Q(g zaGs%fHfXn4{H+Ht!0xwXfwSlFW`#;&YJmH8o>!^9mDdoEe<_1qlfVFqJOp5Hd*57; z^B`h*Ijr;;7|edInxh;)MIA&3+#Q&MSSoHB>C%9W3Ooityx{d zu5s5B%c$$2G~96NgmD^<8R3ll`^Q(+buN#pxcBO}Uy0_r)%j!_NuoJjG|Pw1QTL5> zEZWe%6qgJs%EJN&*C}5Wbgp2@jR2Sl&A}~r^zJh1>0GpfuJKvD>ur*Blu0p}56Vr8 zL3E%chx^Q6EjiykKSIS$Xub0f=6Ae0<;-go>MvHO;E{?lySwQ?kKBTDsEWkozZcR5 z!zhcq>|m|R?dUX9--T-uO*vv&tHE%q(RjyM9 z1A?_xfk zGMDo1;%`!b@RDr8SYSPP_~2zj*^u%sJvRP5C=N^#{@Vho=bI#YN(NZmV~%IBD_G4| zNf6p2Z=#TS-z0BVHH%c%mSt{++$Zhlj2pONm#k?xmYdVTezsILM*d{n!w+`Zq0G}vGo~_S#--NJN;y((c`R|&%C(6dB zhw)(ldr~fbgAb#yC3^0-j0@us`^J4-&lO!hvgk}g!h}Z!8-ofSI<9GE#4S_eMAs|U zYhoRBdH?cwE&hrNyaDb3J1BaP< z%fStmI<3x&2{=m8SH@Ar9LY@#A`JqStfown8ivmQl&m(W6r@$sQYaGgOJA=vb%xd+ zq_<=Lebe$|Kze^=+>Sk*%6l|af`!z=b@+fWzBgzYv=p( zNF8XUZrMLGO)XVkIZg3J8>@&hQxLt8YIAxQ{#fZpf;}*|eDZ{f7Lt^LZsWcL)R&f4 zLvAi5=cVj)B13sA-mczgC8G6my`Y{fHL!l(f1|`uz2cEMJA(DaC-uDHSTjbq0}yf0 zz>3$Z#LaK5+40GyTYf88ec3+0N)1;HMQVmAwwnw#nG*R_?Rr*&d7&DUrR^Y7*kRU| z@{R{f4(O7tBv;_&k3~RUvG23t`zJh`kLlq%qK2EP6f6!#`_tBc+6b4Q(_;yUZWU2GHKgAW9)--ODoUzQ0fx}mx5 z-+=C3=V(yAe`1ODFgMe-;E}wo?SAM9E$JcclGiuT-+jn`JQ$WF+p!lb_j{%4(o9Xc zFx$V8lRSyPA}XI(S{kiq>h))?)19=xU&@CT3-k)neM(=U|CrY-3%+(wE8I{78 zqXR)kif7e1l1<;m>T(vCyf&p~b6~2Zh@v5`EFn7pCpuvv<);p8$08 z76;wm6Sre5kPZC8QM(L?y%?q0h?KFUWtWIO`VJ&_21&=>^b&G{=xm(v`0g7w$gM#D6W!{k-M{Pa0D1j~pL(BSL;VqxB# zi`_0ZS-bBe%Z8+)c&v^TF)I7I0QX7^OV8Rgn-UOXc>$7wy0Yx51;uwG7H5#NGo%Q0 zeG1z1VF|@=QhisG39Ah~phuA=H0dblgro3HF~(ZjmZ$tmeE5k9%N#aNIQ9pT^Z>#m zAm%uA6_W_%_sB0Ifsv#hYQR0le}L8h>!1E_Cw64;F5do=$ycP-AyYE~nE$7oi&oN= z-(W=Lvy%oP!qGo4CWyMfQ-Id0fW9hTC<04(xi78No2MheBb$OpZlhe%m$ZC)}axg{zv$~2Qu`4*Rk3Vj`i zGMSofNT!bB)e!T{B$6$}vIedsD3xucS<_i`Y{ekc7}9(1%kDepe=1mgcP{!Xuug&n zZ(Fd^jSp**GajZG27LQ7)Am~Au-8N*ew8 zXYOH#oR>W={>jd)VQH#h(9evI&BWGyLr{ls?U*xNE&WaKgmxGr=0!O#oxZ;O47aKZ z?m(3S1w<*MWL*VIczerT=)f3!X0ugqTLK-z)iq8ffU;iW;ZAwEQlr=xG_6FPRC;FaR7u5Pklq6NbIQ{w+T_~kI6IBrx}KJsWah}08+1AY zz|pYrt}s83-|jmIv5iyC-NCL@rXzqY$^lU#{+%o=YXuN8-Vx}o9vnyk`)aH-YQNhK zGHoREdBM$CZ0|6`dka#}Q!i6Z*5iBrN_=ZKoTdITyfJ{_*03?O~X11>hll)Tqi z9a%54{e-{KGs2uIKVynerv({6-hkI-)30Igl%NWoU;%I5= zpfns3BI`XtZ`tNQ14?~UpePhOJ|3b-ZD&Q9Uy|=v@cTG=pP}WYid9M!K7wkoRKV#Sx^lQ=qErs393Q*_OwGi zldY65k2sg%r3x{DC+*WnUBANs1qeAX^PanBh&oy)R6F4CQ?jab!Dn&VcXyl>q zxH_^&I1-vNb;#+R zlyh>Ca7Iil5m}J9`}aBL#1((iZHL+NxEa65oG@Me0819<59tX5*K%Rrhaybve<=q3 z_hbLR-9GdjOh`Qt5D@K;>u<%Dd zY1C``9z`qNQgJdy!yFw=kWnEFk^l9I3UCBV1co)@20}w9dSwA)0ah_oNeCj9 zl*E^+)|5O(^{z3{>;ENOJjo`mkKeqIVK}UW14}zH+f%;@0bQYqQR@-V86IA z8=;NlAt^cT$ZE+R2FJcE1-+2R-k!q62Gq8&Y|aH4YLmEe8#@U$7@Y1ejx-Wq+?GuZ zd=DL9*)Hvvhz@#UVb0(kfhk-!xT%3Au#-Wu70F>+T zJ!035r{-lhifhsj7cZ&6^vhn6^~?XD2%`OME8mCPRGJG=Lu;KYDZtaOv>1>Di&e$O z&WI2_g;t{G%%Rj!s)r3UELdIU`uB&gNsZ2HE2B=AsU`m*>JlqtS%{kYCoLsmQ7tNd z2w9jJa#&_m-c&T~Pa!Sb$C+IY1}MZ2WAjajwsH$Eop~~CQZ*Q`x=ve8AG|QQ&*UAV zVXiZrbqwASepIrOT1xx>%Vg@X6FPS5d^OuW>1vjvOM^Yjp3W}UiZ z#~AU0=E?u6Q{e?Nw?*U0b6PvudX{cg{vNbNx{6kN*Pe$whk8{chCoqJC8J1foxi*?3x4QAip;s0O&e=LODa9eYJ%5rt#_=9^j=0wsHEcR^IYxe|sfgt_HBi-nf!lDfEH3BebU$oy=?L~Veu zamU|<9yD3WEu&B`ZlW{i{GA8tto_Ty52(S*n^#66RXRQK zdV+m9ypvY>dy~&y+a8%wk)x}VioNHdql$S9VVM%VO9Fx^ zNN(RsV%cv~oa_uWnkV-WUR;uom%Z{!%i2_Pn!KSO)@6#TShnO3mo6hZ* zZVwjkgTc`pP(RIm&0vg@K{q{<(V{db09leusYwVP|1Jv}|0Fy+f2PzirFRV;Zda1s z@tQ___RI;zf-Dvp3?)_RAu0|{en+rNZ(qKX2GpgM{PsB2Xt*qJr%UR z6xgIqm0!&ynqJ{8np@g{InAdE59+NJPNl*hyrtz04-ozGx_Fnyhy*dI3`)xqfQc)& z_t?tnn^sSZoi?SW{5ZIjT;ZXM)U-sX{#|quPa-k`Stg}raZ$8YchZK;IC`+XYTFFF z^qUQiKxEAZ<(=}pU8ohl!k<}qS8w*Y#z3bHDyZTI#@JO4iXFjL9<-iGVBnq&_{z66 zXn|4qi`RA)XZ++Wd1Kw*tr*;|0MSpQV0|I7RBi*3UYWhr4MUXPfqsF(+MDZ*l|A^f;Xv zHSWqzu>&`*eb%4+9dEc~FUo^6%GWSSm(BH^c%Q&Mu?Y+%gA8w}lpb;CfS#(`(92Jn z&&r?ayq(%xan$W(=*LktU5n7lS707MlDqN(K19?B1lqQBD2@jKS?tugpc#QtvetzDG38`rRgduQu1&A?g6|P6! zURl1O`u{AA=sVx$-mjb;o3sA|fBJ4802qc+ysfFcN>jg~5&a1s0N93_zh)-!nw|9S z^)^53-0f_<5|sTVpn5|gvq$>TCTs6qebay2LH*|r(KWV1PTwOu4EU~laDNj(`kYt0Zi@Nqq4B4YgrPWCTk z=QHgL{ESO41P;5X&?-KTiORkLl00F<$idAcGp3!(s)nNKkLv5z;t-+wMHt8d(6)rA zw)Pkfyz$B2k}FN7cA259v#hX>C>?uFM?S%L^*U@xEBHoN^}@!ly)!4z>V5^BXMFue zr@vXza8J#2IF4B%>ugJ^Y2`K+RqLyp4Gq|aXyg!;9`2zSd$J9Ftqt}0?fDJ)ZU^%b z9U1G`zql7bn*eC};9dXD_6#oAxp;_kIU1Nzqd>I-{}t!BGE{5mZs6LYq6Je{1^7Z2kb-{k#4$u` zmB^;T??Z)4*rBQ@yNVhg-AAj!xuUT>L6Ud{7ITVat^m~8SdD4a27i|igsZ4>FsE^p zasDCBp(B#-lnkusQjzt{Q(3uSJSK%QR{aqK?BqrSi_4Fkn4gv~hkP;-%A#%o2dU1^ zkK;i60rwFdS>M}2{b_nGgOpkjmW+L z90mDjy;}=P5}JAY!qR*fKZrNSp~!=V-MSL4#nZy-BGFDK2Xo68S3!%7mGoyY(C%-K zn|&RN(#gYzk44HF^ibDUW2D95Lx{Ska7UZ5;}}5bHeO-QNIG-BkX)09p#bk03`BLi zj^WUD&~ce%Y{*f~)9oWLr-giHjWd8{6)As@Tey1r+|N8QSL@goE`(&Sm|p;06p{ju z=+H5KxG0svA{qsM#Y_@*$UU?+dN&y#1*z~$!hAqazRcBHwQlPGW;aL~#I_$&uW>LN z4-e33yXw3VydC<#w`NJDE6s?x!wFe1{k=LdS zG^!bPt8Z*}4&m5PV8+OZY3fH%?&rGo(va7rxCPjlke@6DWw4X3qkj~@)A~8XC$_K$ zJJMW9f!*i8#f9(_&l_dSn!dK!!4%8uL`NUV=5U{}&?8=mClfMfiu~~@=??0Hga9?? zLlrpC-}eJW7IuJ>+F<&2aQA|H?jYleCa)oeXwq)Ug1|QG3NvcW|xn4%<0@xWhvFVIw?< zGvdYffq1Y$So0$|Mu$fpnx@~wg8&kD>XtHIw{=;iqJl(v>*oj|-A-<0BJ4NxxKPD7(SXh+k-wD^ zG4TB0uXg*Vzp>?{QM+jH=^}*_86*p8!l*RxBRX2x(066X*d&}XIyEIwMugaPJV}BW z5-=g2<7p1>5@XH5;sx799X*tyl04l^oysWS9LK0qv=h@E)>25ZP!Rl;wyt3)#4SRi zCYk(ex0lYZ2(3^id02m2kOM;SVc~MZ718qJ%-y3<;`8){geZ-gh)bvgs6O?;XR-{y z@dWe;LYv)m)u}&y!QKC5n$XW_&Bs#aYh}^~CxEAUGHKtCR;%=ntt z3p`qJJj2*)_x|*eq8oq$tojjy!~oQ^)xVWSJUh3!%_{93+Fz7aN;r2{4MPVl)^kedmtJ_+6>&PGK- zmc_ECK*8}Px-rv#GaVPnI!U(4vNJ6yOs%sXGLhpm1Fwq|mb$+T>pSQ61Ubg4f6>47 z)%h`Pl*&zfYm_Sy3j($@bu~T&qW@};JzqZwUf|cC= zeD<)*4^>`~n5TViWP~9_jQJw#Mvc_12uVAPB1R=?_wj{XKg6cuKnoOwSw>(E8#JQP zB!IiX(mIPt>KM+djTB3|rglnNViPEGePLeN-8b1)hJjajfB~Qi*BKF8ZkLecDOg`zTc=(! z)e}U%Jj)dyECDpmHqp(xGH#y54&`9YT^Q#?k!A=xH?r#fs)ePV%$x)t(Bx!w!^V^` z0S#8xKEHcAI(Td_&eRh4hMGUjhj2rAE+zFRVwd5Zb`V}ryU1MPgo)khW$u6}#D+Tr z4>&On3YHum4qLzw@EDf`Pd0%~UXaNytHw6^-fY?$0svljShN>6k&f4%CnuAJv^(l~ zj>_*he+33oFhdj6!Bo#gOkKg(3arKlA>{K(=`g-J!z;D+ewT_Y<(^k_Lms{d%m!m$u-dL3KG60WoPuR*Tt)#b7%lA-3;Sx#=CSMH? zlaO;kTTz&hC@NJ!T0O*f=ju~$wKf&03Tz5_{ShbR`Ipdo_GZieY#qNT_fQr_w74X` zeARL+f>}F67tMN$G>JMKjj{%eUS?pC?<(Voy#?l1>ES+B}Bbe%IGpjh%ID;b< zHmEq&4IVZ<6BQy*fdxTErgzIY&xW08=a!26cs2W39!=b&ctIrc5t2O};A=9WTf`3Lq#Xah%10EWD04&@Cc{ zB&tmEd6$PqG1)Sb_>PrP3ByATiK6MmU9<=je@D=(3&gCP@bUD(P~M_l`DHSiXy=46K*wT zMl>5`6Kko|>)9{{Vt;PoC(D{pl<%`Tc4&SV_@unM&25lX^E|SMqxk~rl4I-;!Oba+ z3aO`2@l|b*FA_$4*`I7xMKf=yjT)(2!m2B7!y8rVqixmK= z25Q6HR6$qWz!MX((*?N=Zvdsj$`z2%K?x(M$+|=Qg$-YqNXb4ae4~UL`Zmt4tf8r)mg#fQF+7OKx=NNS>>qEapm5UTYcC~ovTv4V0@%1q)I;-WH&!*cb~ zNS&&VEu0iRy5;rGy3DHs*0iwEIoTPN(u;iRNGIj{-j4ngdWHgO!+_v@V(?+8@FLeJ z{IJ<&;9t&6^I|#yk;zMI1fnvk!^Qi|qB6?EM`K)$!D+FPhRoM(6Y!0ts&`loxF5sG ziW!qghwW5aVyceGZR@nE%Rq$>Vs~si#~$<3+;nAKdBXSiqC;l&MA z(owE+4)L|huc&DY|ijhM*u$R54w*M&CC6d(*!OB zy=&DWpNd!9k3`~Y_2H9Y49q&@_Ju3^`zPniMWcVyVyvZl#4=phXif8wGP^m@0- z!g@}tZXAvf&;Y|_-k3AkQ!(kfti(^L@j?4Us@}o2>O<n)KM;$G{}3a_HDOXm@%K{UW70Y(W~k>?;8TBt!SyOFw2mId>EB> zBuC4RGhcR|2F(Hq3Yg?tf+5m3ckM|D>;*hrnwAJ)6%2F?!WX9`WqEw!m#bZK;2#;u z#}&y%#nVmbs=PsbI3Z+nX z+eG59AHm$LR7IVK8%82JqLU|S&MzvhIn-3xSE|alR~35e>peAPm7R^jd{}6X6S-6B zP60gHU#vo;pLz)Jd6#(zZfrK>-5T~K)!V;1i@{t?mML=D`9sk36G?;lcDF276C?w# zke==Rm&vU~5>}l=EiHN697ZwACk&IS=j<(EiKWCsi_S{zp{Gk^`&kmQ!pS1W6}Hju z?F2^!>6_JR)5Xz<-x104^TY*$icgPVM*y(DVv7_v>Y_GYae0MScq!K>Y0HS46?x>sVUH#6%`avAC6gu{}je zqY7WU;|+epWRR0R8&B!GV_}5}NMn7qOd(5;o?(i*?bK_BpnURjV(}iqpp5gr7ul!1 zM#Yi0_Q@zLY`CUkv|3DB^UVv3 zmPzaQe{D>tf5yJ(g2Sv$_k%4(w2&Box;%I(!^ghic&{;C6~y9R^*BV|LDqBn&o%jW zga_0upg<2{_4@0lU}DauZv(Wg{&CV686o{4wUqsOwa+9iOdM~6w(4b6U9z@6Vpm;W zB#jwo3u0Sq*|Q3n)G(p(a5dw%e+tStEZ8~MRghdyxhB(Hn5t;(6Pkrap69sM&gP_^ z@F;%1-T6_Kh*ojRcewn!u(9Y--qjz7_?4NO$dKD*f^FK8LS6P z%LY-Py(&qn6@xX;VS-7X9@o^;fHw9(fYQJVPn*Dw#hCh){FvT@_(Bo4OPNGdHZ(fA z*3*UcR8xp*)8D@szxbtl32JtDo<3|ma>Yw24%t$pla4&5$N;s4s}>%3gCqNFkvYI5 z_-{cFnSL)QYmyD#$eB^<6xo(>E5E+DquZmOga7CcMVNYo2i+8b;M{Wy@9z~g{+zXI za|?H7OpEWiWB;bEB$DOGaZ?f2!aTYiy+5J0DCl8~dvKpI`gf_-k%ige^QrWIkcDZZ zRf=t<{3=@h4WK4u?;6(qgNXVwOf z_CzeX3WQa!SfKkra}$o=NPglq>1goGdu2?L;4O6@gms_3<>^mh(LtZO4AH#CHRqVR z4&b^L?_UFPmc}mvQ11c1pCI3+2J||uH*&%-@z>N57?$c;#Aa?tik zdvIhm01)U4r3#0Is)vc4Bh{*dXSHaTwNxzVQpV8Hm>8*>(N&Mp4-f>|~8H?p*xm2 z;Egs|)~tLgdc=@idDWX0K2n2QdjQu43p+$Br9Wequ}ZvfS1@okM653qoX~G#XWVJE zz3el!F8U> z3@V%I@t8@`I4%}U=2U#TL@F0Esav{rA9Ltb{GEbn%#?V@R6Ou_5*IVkph--WM9Ne= z78bp;31!yhh69}TxSAmQ2O@i&*fJ=I28-NpHV6G^F=M#%QAo^ZBm<1#YSmS4KyWjq zP4_22IATD~__4HZ0nIJs7a8mzz~%)>j%V#zr-kQA7t**y(M?WZWEv+U#$1N7oF~T4 z<{0!7ryA!oS|XPv9U_`GLp?3Dsc=}v_`?`E95xF81YDi%bUsJ~1L0D>n2J<+bta;s2lY(R4frjL--tPelG0||+s;9f0qx^G>xuizozb=o&_TC>bkFmjq@pT}Lpjoug!J==U3Agwyo@bIf; ztH9aP`$RV-dydv`mw&8&oz2(%~$@@f_jz^q!E(b66*_<}G zXraN`&HDt0mM2xCftsV|!Tjglmdh-Roj1M7X&%}+w_*2f^ED6p4a+$X8{TGumdmcl zt7RUJ^K5Y|{C;-u(;iEJN7O*EK;BxejMldXLD?nKm}zWgU(90PyjGyC20eCD;~E$D zrzW>I4tMEQC=p`pjv-23nc#tofx-;{C1A+kkX-f}QqEAo6nY%|QBdKx?i|wCRd5Vn z+z~9=^Y1?hl)8v(`iu5l;&ba$Ibgwj5NDT=A&B~+LeF$*(>){rh?!B~`@Z36ab-C- z*B^kec>|1zf-s9TM2;^iInAXI<}|$##s&C?j0-qqFRi=bk8JYOZ}YA_-zCh5B0lcl z=p8InPS=`c7^y^4z9@2Jsm78$MPq1Ai!-N%Js~j=a?mB{Jnnfk-7S#W!`Y@(*Fx%2 zOqx<EyeSw9Tu4+<&0PtJ;h)Uz)@n zh!>+o%{3BC&CMG9x!sH$n zbC|^tmH_e znGDF)e^z)^`h2<)5N9U{rIB%+_UNvhMwI-EI$E@YHEHOZ7Rvk z+&Trm<}s_Gf`a#cFwrC;xYd&fuIS(e%xh=x)zYmIf?-)eKeP)PHv{R}a&T820=X^P-%CVFeK7b^ac{ur*Ri_8y-lP+v?@H>zpT={K z#dLxnQ#x7w+8?*uSqxAt?SVgnN)S+~rNcP{*|uJc8)WUZ*P++elUC)8ZMg)8J24O6 zA5AyLq`ofLR^A3-M^8Rs3Ed)g9lA(#bgtO1CP;5Da%2^P+=DIkhON=JG9c6gpK_00 z1~|De1QCGKdONF&&IQi)MzN~KushV=HQR>w`U7-*Ahr{%6c=;l?dZr&rlB$|8cO3-kwx6;6s_KFE&w{Zc-|Jf<$G5 zU*ru$2K@nDaRH$aG#Ne6bMA>Sa*peb3Al6-(?8jb=ERqr(6u|E8{&RhaJazNIOKun9w)Yf@=griCs28{9Fyo^Yw_Py+ z_0k9;u4RoemT#gX9(ltsO)msB-t!mp%HSA`CRk}hU=_xOZHUrkJs6c7uLBO_JL$`H zLI;TF$7s}Zs0(~_@FIdi(g42tDgfnXFzuz^1GRw(fQlN-SKfF=WwK!S+;ly3xhSg@ zzVYT^v~bGDh^6@y%*Qz0CD_yoVYT`7A^at_LgkKIBRHT7Bj<~U`hX)-QXAw0M*+u7 zQHuRE6oC|PK6@AbGUU1IkH3mxFKgd&J7i({M@}1S)jNr`0o^uuCJ>LW8o+SvZ#aCq zhU%Z;>m-y#_IoS(@@&-c5^I^aZKs>$=-9`O}rZLb9U3wpypW-eE zBjD$`O?R?bwr6{4lUXm;48Y>vD7m0LDjB28sjrLTi@i3Yo)T4$Z{mQH>mN?G@iUs= zW>3^+`K5#xQ^=@fsQ(Q#GKTgtunByz1@rm={!vmh$S@9uXHZL1jJRac3ltrJW%3W7 zXh{|gp}05INr9<*OIM#FgcG?rR1xR(_rbVG`H~3 z2wlr=LM41=%tn=KF+8CW-Wa{VD5p!fVj;Y6jc+S?SpqN@ug-Ph0tk4YkUCSn@GJKi zyPmY2XcyOqJ0YEKk$_u|f4bv&@D987yq0J{+Gz=h&HkRq^PC7VV+yZuA9;?Rb}=uL-0kxZOhZJH$$qz_ zq)$ng+=%}CPI(88cKN2>$D#pWKmNr@Scq?+2GnO^_76UZF91bRk9`}yfDoH!@Rx@B zFSj#xgCd-jCB2vuR}BmYbA2N0%~7R1{e9ECVSR9(Zz8Er8gfwrs#0*~7nJ9Up1lSA zmkh-prZbYbO5+D^t)kp|4i}qWW;XCxtGsfp=&M~3t6d?hT{%EF#1$SN;pe_~&`_?p zzW%-X1%h)9uK=gTc6Q$S1{JL%<#ABnJt;m=`HAYY()V3Pj?jV(u1~ax?OLMHUMS^P zeOPnu<=~#aL!r{qw%Me}h*8XE8d~XS$C5%JLDyD8T@`2L)mn&`^= zBCJlpAZ7foH)1q$7_ehVoGCPT4MOHTC6fUsCfE#7^eXhVI>{)Y-Z3_;6SqLhD{#ei zxRPcZ8S9nw6a?1}M?IiMmjfNBYv7UgEmW9z$LS=k=n)TZ;)73|&o6eXEUv5f1Kho_ zdYqGsaRA%$Rio_siHrJsVzHaz`IzJ3>P={L%_Ht;6|-uR*`dkd>UHRmMJ@~MN7k9r zIKTPx#z{E+5u`~1ONucMq&`W?*-{3Inn0y%5Z4zbh$q?`Oa$`KOVH%aJCwUfd^L$aFqkSB?t|;JLs!2tAwum7Brr41} zkkbfq07=*@7Bj$M2r-Bx&SW>F){b-o%FZZ-8huG3ErM3(=(rRzAx_8Xn~gTO7A@4w9qi z%g}+Vs`PQz51Fiv;1prm$C-5rX54{Wj$9Ne5^0BEmy`Qo)?Ns!EMGlZm0lC(1Uf(} z)pug#)Vi7{KMut6#pzSBxc5nH$%?Wl0}ej9$xlm`(XV5Ld$8m6!AXXlC{d3?(T0_( zf!|aN)>RsTksA>$V_xjBcz|FWu@^_L#gw0a#k@pw>Z?{U3%1~x)Q^h%1Eeq(#(#tJ z-Md>bMtbgMItun$FkxAS0xueZ{)qt=>?I*T{f@-Ft$p`1$BYcbt;6UEHumN51<1$; z1QT(WJPq7(7*F*c+}J+HO!qTy4bKSqPoDbU;D%s)>~_NLkDsu$ziW<^-2VIrc5FMz z&zr}O+JyHFpNiU~f727-J;M(JS`|eU3-hyN_ZYFcU+uiib*kd!s-?WX8E4vA8Rf+o zB?ZO`Hb#%sTeak>uw>A>6vA7B1GEcy!S647;uOX(i{TYVQ#%g{8WtQ=j`x@Ge@Yv< zsEm$P8XwMc0h2xp3tb2klQuIHDVN8KWl6bb_fL;uZ+M5$2max>{rp%?0# zuhTHY$h$ns7(dF`?uYzNe~DmbL&hvfE--#XDk|hN=&)EiEnpk?;)N9t1zdf|4> zZQHhO+tzN|wr$(CZQGdcp6;Hx{Xg%0bI!TRPHHEW`cTQr`mI`z*d+jK)H{WascRZ} zMClv2a=44UsKGHtywjRstdj?Ko^gCeNuYJ$MC9_0d@l=z2=i4%@s@C&%3;V7IZ9yB z6#H82d9xClyyh~|_dAw%GcGT4oxUY|UNJR7=hd*vl?*IzqiJ3{Yz<~nYST(s!(z3) zZTjV%PHxEYn#LtgXngPc93mVqL!}g{J zkEVyWCZym!0W7;5z)h&YDhBudkk(fxfti?8auS_b2cPaVXsURv;ALOuZ2UYRA( z{oW!6kBs~-WZM?~`*`IYC}S_?9^F2ql^fM*G2+0C$OD1(=O-kP-T=Vpj#Bv<*>ryT z2ilB297j7d9|eR-?}H^vk`uvZ5h1P(d1^w6U9{%Il?&0)FW?@^KA3=Y;cgw^Y{0!4 zOZH<1VX1ly6e9(VFMRUDA$tP54@6rc8ic_fsC6Lig+w3JJ2K&gPG^A9TNEEq<9)-h zBlw+0IUxEYFFYa?tp}XAy^$LU>n39P#L;05f65N#{+q%92Lf7z zO*asvmISC@|LAYg{^=zlp`=Mz$p}@@1+ke@9j>ZRi3A1}@md$I2C1hz29_FOa^TYL ze^cCy`#Y7fSmRR};`8vEU&;FUK6D#=W9B%GU2%DE1TL}8#_&w?o$v9@?{mHF`MqE4 z`2p!|#2irS#}{E7&-5aXW@kj8a7^x~qAhn|jRACwbdNsZ5Sw{*&kkV_9VCzkd3beW z4H2G|dZOPphS&)95RZcy;R1^@Tkx2YXq;!**6tC(sF4GVKH$to^ zXfM06BSUAFl3Yel3vv~Hu0vU#7;LUFPOP&zNd>dMr}z;-)LB`itIOIA|Kpkfe9 zN%J_RB~Iex;|WsQqg{v9ClD+96IN`d+U|q|=yk66AUY>^U=>Bb_<# zRgFfZ%3-ZJ54OpUx5t*Ybr#56kFLMIp8>eFqq3SBYCR&*jF`c&9_G3g?b-(BH_5m8F zm!WFbA$^0wUf$*~q}i`7tiMal=tW&$#Fgv1%j|x1uqAw=w_hOz11EXmF*ao=Qrk~F zEKK5No9%Lc^0j0TD7Z~5a!T-6I<;`090uIuL`3-VmKT6353#O>G?yA;(BUl{)q^9^ zuLR+%JLnhmZ6gR|vQMxRttE}*f&rt!v%}o{D+dn?0WA=iJ>aaS4aj2>IQ9?O@~&2v zgV7*<|BiDCMcOuO!LIVP{Q|Fo%fki^drNg6!9kep!&OE{Wx-jKq%zB{3xerG8jBF$ z)`mW5Y+93R)!(5lj{SSsTr3RL%cI?1j!3ZgRc_5!Z%QowoxmwTW%&(Rcmb>!`+qUS zgVc|GC}Hj-k$eO-GuwP}$H^1!PCpthQk}7}WMV0s?l_+|oqV>oS{g~4Xmwo_Z1<)0 zQAsNfe?nK(o#Bhql)g9!l7PjbdSsG)Q&M6^BIcJU zeFveiye%mgZoyW>%f^T@u>g>@Fj?BX)Bi1sPb6Hld3?thp%3DLl6&{1dQ9$yQdnel zL|t6oFTOz?=@UAs+B=IBxw!=j4uGkS;#aEylKvb9%Q55ry`V#U5m$NODw=Gt> zA(XG;JFSuj`T_V$h7Pb-C|JMal;ac=|BOo#;PU73;@WZw6+{cwVsy9 zFKc&C^`0Of8Ii8k7l+C&6CDOK3buq;O74rl5_Bo)qX`gF`l~*!cnfoxid=}tE0?+? zh_k;^EFvxHe*;gwd~zV8c)SR%n>s#p`2WMmdOg-V4}Kk_!`l*<%a*V&Y~knpa1!+W zbN|Z@vd~*_^c(v0hzL+MGS5lwPc-2#*8L8j%PK(OuTl7#uq=L6{iW`%@%_cuy|( z$M}IsbY>yuZQc8CMj@qKsRA#Xrk8#vLU}6#~F?1i~$Yd--ONAfp@H$lnWG@P^ zrN^V~Qm*D-*m?@ArPdkzf;bBQ2>hp^{&(B_pVEl`G*q1_TNorrARu+5^n71bjC4bK zRHAgpd}0Vdsiutz-Z(m6FVFMUU`!U<<;g)(~gRO(lV@5Hf)zle0CbC-pt zx+O3X`l#&u?e<%q%dcJdU;pks06-5={zzh|b7DpS;~tRigg9bQnK<$j6XvOLRWjS0 z_#za{6Uu6nuApCWz9%I^F{-^4!|?r{*K2fCnW>2NAYQsr#t6^KMnqB5p7P_*N~{Mt zaSYd_RgF?!3A=i zX_-oUI`;$1RXcAnv2Jfq;lIh?z~#p^Qo0XR3=O2N!*!>bbYQJnm8{I0J6kRn!8KCj z@uYI4;PJ4LEj8H+7~?fBLtWRLJI!5lD}BrXB@-#H8*#U_b|TOROG~|(~KRf zplv4j^9u1gcNtuHr6ukGHBIHR>uLLeiqB=XO}q_mIU^c??$OXp^lAm$-JE=3U_E92 zoO~otZv7=`YwCO53x0LyxhXvT2im&3yY#|~{ehr+2UM26q$YHB7VJPiTw`mV=_3We zbfe%hMAXbp%B87k$p{mD;}?eaQ(kl^gxWlEAz|3rtR?KroS-V&yqK zi2}E0$7!fk$ZxUVh^hKYIpXwC@zc&5&5_i&>#g)ESOH>4MHDjp04Z{X_eDBHcU7-| zg#M@$X#JrMNc=v@FVNi!rUx`{c;2va_?E&rBm)Ld)6Q zQqVTGYdMB(IWtZ+d$@D6TX}wfNu0S^t-FPh4c)j-gYh4o+zYw$oohau2^aQVCLBff zQ3aoa8H9!TGR2pU1U_!_$Y`;5Y{a&`{1Vaza({D;Y4>#JOt?z#0#{>dRWQ+j<31vJoe*hO9Z5@!BXMKM zgPY}a1nF;v;u{$Bp?9?WS64VDNV%C|B-jlqdJ^(H5sLFHKan2pUqCC8AZXU4M4=0# zjmVM&gT(=5lYeOgd_dR$o+yx9Negw$`q=w)x3lYozgD2AmnwSWpO6TmZz6hPpTuve zsog~}DofA@eZads(cO;NSfkb(b$SX9+$Ooktiko8cSzAPFgjHhD(5u$Vl6DI5!5(C zhIJR!2-l885Y)>$me%i!{~}A%5dMwc<&LXs_+Z`ih4==}7D$c;jCquoJn|QTa4359 z9FvF+MMuVwVSUwF$Lo!@AtXuXXbHEv+h@kBxp!Y8AB9(-2nff%GqpvxFB0z!@;}li z9Z^J?4S6UVZbq7TsyE(@0A+?u)TlMWVLDyYIwt=KUU_T%!%5SbDzY=;?hIRjPti&Q z2g#OFBbVAPsM{uur!L-!2K`AvA+sVKQW(whk6POn z`AtNPiof3Q%_nLc%@8``)BYDj2YyLEDD<66_z06WSS)RjVUf^(6!ZY*w=xE%OsL?h zkq#2fM=wefn7Hb0m}YY^wW(fJi*TU7fsnyb+MyTFmK=8vAQBS0=%1h{RvuFh+y?J_ zD$!;{Q}w$}?xj>n=4P2k<~cdU%}Xj9_!n^JUxL-6c-cpIjK_SmBkz6b02b+b5twsp zS`Tz1>SQXL6k5+7a7q0#w@t7{_qA=pUuxmoTDG6*itYWO6H|4@<6HXwkdPmdAAMu& zjuJykR;CvXRiig-SWLpFlQm>i zTHcakm=#|@@D_E2kL?xdg-BwdUk*6W>Hfe5s`m^|;v~sr$S0O|800?f^vh26~muLFmc zpCR6=Q$E-3?q05X6RlQTD5(qkU(Q^!Y4#C_N%pj_&=}gyG&>^Lgtt5OrOeMGep=ih zrUM`;8Dah`LljAwrHqmUpg{^;$ZU2yIaJAI5If}p@FjMu<)s$6E!NTw7KgXr>eoEG zf1Mo8dviISvjJYuXS3U%d*KW@kO6F-{fH0;`M!)G|7KR^$=)%c|E@yPd+vk&8=E(H zKiEGM$oQl1i}NsDpA)Fxc<)D@0T}+_ar`3qgje}@O>zYa{dXP`{(;|v|Ktp%SA-BI zkqS~pjo-B>nu>!1g@eGpXXePHDoe|>%-XXitx}6u{Twj7sX4z3QFi9IJTa?FkUhTQ zYoFEO1FkZQ(&pKqHU!S!gYwO<8xif>fa5=jr_uN_%C8$F)2TJK3)F9!%y`c5g4DsG2n%jVlEl%r{I$Mp0RX3x8dC$~x|sgEZw+Kh$D4V+Su z0kPbUKiJ8Mseo$!Qv}^AfQeXn*(*&cPV(x<$_8jNkdo-EN91lsU4$EInnt@*N}UiS zc5k7~)dtaJ$xs_=zg1Nh4se`df1(i65M7O>XjB6ZJBl_BTUx9siwr|Z=o4fR;EmMH zAxpNt5vx!`&D7PVV{S*%;rfLX(dEgiWVklk0n!V(C7EJFn~9JNMoPETk#)MBwMF2K zEe8-?At~`+@jNE7b|lW6vz85RAuS#yG6=jWUaco!dox$DFd#<7SExwLwIMofKQKYOyNpzerm`T_dx{{#Gz9ph&^cETO^)k|4oK^ob`m?Ww#ZfQJyi+VSVN4 z`aJ1y4bp3dzsqw>+#y`Z&lY$96>8~5f(;njUo&1A=H($ebO!?zc$8R} zi^U;pb930CHOi=?{@Ah#1czK9;dXO$a%FN$(@&#e^s^}rjtooFjtenc%Q`)xMcFBg zL807RCjK@n1CQxEFO?_4@+~ftZxv$6=H;%vp2JF1Ea1gfAw{!dm(lTHwi%Z-KLjA` zlE&_i-h8qJj5;#8@fwkph7@lW8)K>jXi%(7&W>fs-1vly1~y60Qq8BzN^pkHk^Q!U zP^a5Tv{6Elsc@5cZ(WB zZJdg0Dl5OVS z@J;)Kq>>dPEO}eVbQjLD^OSB{28E$Y%kq$oUh>T2aE*Oj!`blUELkzviUTOg*~T)G zube`FrKHlAjwK88eH4 z<1+F{GI?7x*ikV&hSCt=4uF(H5YkHC${VaF*Tl%QY)lHw*hP9WA;HXQuDof}Z=%)u zcPb2Gsim8BclC0EG5#GYqy8As4wTHe|;K7@KE8W%xn6**l#*zZg6~Q3 zsvcw`AtiADJ;rt_n$9#3`Qa^d854;Kh6^)+`KNwRNk}sD&zwYiiBNWz@=*NMBQs?3 z)uR&@+~Yoe2w&wa9l~d|mo59}SURXExH(=R8}0GP;P75!6%YZH>SM|Y%b>_ab=;ef z`p9oDe=u(PuJg`Atr)#gsXbMy>{?Nk7?_f!j@3QcEn}W)2q+{=j~(NNfmS=;mN0SEn-D#w;$}0JCSR@}p-QJ^vspB*3&D`L+AI-Q zVZ)`IT-1bN)H<0ODblT8H_(SrG`_yv8Xuzfpes5;msQD&6FfD1LzTgeVr0KCF>U=s8vsSIOWzu zC{`ZqbYf66i&%tdNG-Bm#MSXFw#k${5MrDXS8UuX=N8%Mh|qfO%c>Q0iC))JAKoHn zS6xF*s)Uhn%5odQG3T1#9+TeEjBODt)%-3~AD|sB>MH1pHl$jSItAud!zf!W73-v( zq}B*10aR5vL028lC7K0uRBCgH5Xg`{7$DdF{GAfsQ{ssIBzdL3r8z@B%m|a^YFXB7 zV~S5!@Ojs9Crd52BW&EKUoIN#eYuFGkA{>#$W_VATtgncnL4U+Jf>Ht55hqiL4ed4 z!xij{u09OQ%Yzj_z4rtKz=Z>l$gl7?BQXsv01bSIoZtxFs(di8R=)3e9bx4IdJLz7 z%l;5OFB8EIFjWj->G=?u7Aw4vtzH>!I9EY~=Y{l$&CiMgFEv0&h3>#V9MV9@AV%1< zVHizln*vbwLF$X()q*9Reh8p3Lp(u6I4tl{a!y0xjKFkD?^G})Xap%o+*mFm2&dfy z!0F2|*LzJvYHIiB0Gn`RP47x7At_;{f^9iPzl$oLS*BO5RoO5rOX<9>jDo~uCho{g z=a$YeBjgT(RUsXymfn~*jv4(tmnEUae5YaJib>K56R8)Wg(P>4%$BHvd&#hsSo7|Q zISY;)gi5kwi1vxIZ2DU9@e}y-0+#I@V0foUHQ#et;%?tc_?(6G^FW?xb5g8;-XWNN ze%eU*=uAuKjw$j9r>Lik@&u#_ujNDCQl#Wh8O*{R`-xT|n^xhjGVs>p(?VT?Fl^$L zBe>-o))V{X(+Nlm%Yv;Khfxwa?vg3!Zm**Omn)sRtJ6|mMZjGMkwcL847#BfptkvN zsj;hzWYGM4Pj?Sg>8y1G15uu(_i^FTUk*i00&6#}jncsMFPwcR*yHHt^_ zPcC{*2LOU*DqHZkp82R#MIpKkkpKgUw>b>~rNd~dIt>BgC8B!6k&s9pTg3MoqgRD6 z5}=2-B~;xqpJ+9|7U&i=jMoMU@Hsy~Ow%Pn4aTGgGF{kxWd++cF8O(kzQAWX(adfC zxQt+{QU~Ez;E_nx3D#>#olIN;(W09Yo&(G9TPDECJkMRn^WiR z%)Fo$if>705ZO@&&p)2lf@9}+TM%=D^j7p9Id%8qbdSLeF^M;P zA+>8+#t+#~qRR}O4=8V>7V-UI;uCy9Hzl@Q}+0rmrT#jBbrq@ zpOmXL0L!cpw>YV8_`<6ps53bBN3J8Nv*rU`heP1O%{{ub7MP!Bc}S$v^a(67#P^{? zTwglUlNh3mbwyz7WQ}*v^lM6~0eLskE)WRyazw@$o>ukO1%zB5em0=%2Yx?Lr#c8A z^=(uXvfLnGyN6HQNxYhr8G1;z`_QxuVNP1m9|B8zJ*Poe_m}AKiS5uMa3k!z-eMM> z;n`x!HVID(OV=p8){TDR_8)`&#=>>TtDMT`Q_I51h4&(Xh8Cp zwR&Yz(3wR|`7&^ia!UBv52>(`@)6H&Mr?-C%?Yr_J=d^1{tC6r>Hm?+*4rL` zW&QkZjD1eA%SpFA_FbZvcikHMaO>t=w81^+usyB;w8lOo+8*0B#6DAPbDnRHeYDu+ z)V4YKFSE~y{aj%4m?mB42wgJaGv{Hj=NAX96wJHjen8X)y_{;hV_rMlbM0-;XIDup zW*))r=I#sHE!#@6#%WhO5A5~dJ-aj<_S7z?{Ke_27M1*sL!JaD7PQ7~X(3JseZ>m; zl?niNe#q9fP99m8+_4?ZLtf23tpc|*!AS=U%$Rz7o0m+2TbJ0^Y4AI?**C0b18NRH zwA)|D$q&TgUnrRKfbg5cCo_)qiQ4$2>%2u$J3i>yTb#~XEgySmMRLe0|&6tyBIO;;I(#5CQ3XRpm#HwVbE?B3@jGgxxnwBbwx3e_CvtC( zHO+?JtLi5A746PNKslUGoniRlt*JuR&13>!cp(DCV7!Fvt4elY^X$N;IHv&y=QhAK z$8GJ;q{ZN;1@~V40v4bPmU&NH1G%Al(hjYK9>3AYh~Q66z&-K6_&~IW2-_ZP^CQ1z z2!$-T?E(-UD-a~~!yDKEA$s3@R1Ol_f*zwz_#M4qeICSj>W~;DLITh_+~?;pd3~xp zXZ=+>B%<1m6(5t88tXNemC8JY-37qQ6@o8xs%I(iWitx;MNLVpc}8G8-9sa^q-J+K z7)L=WQy`>&q0uIh&5!QyQd!=(?S+@!&m)eft&Z~r=aEr1k0N{zyy6F5F9S5zIm7=> zud12Fd1nOjomc1jPBCK{4!(!|%8kySjQLUWBOh!IhE!Vc^Flh`;g_LV3;+_xMV~)% zR|(`jf&oNBdcdjpd$n+9u>FxStm$Ptw5I;k>kF#)|^#F7f*tH=uKpqYaLglas6gDW5vAFA+oCiwB*S`ut#W zUU<59JI#Q35ls0(C_Q))PrzD74aXuLLN~Q^RqeyN>+3O@I{bQd*tOcBIi#x+mL-e$ z1c#JfFN*$G$`=3MW6xyqL+uB4e&)X(AGz1L4MGAB zs|ts)r^13h1WuO76_;S4e=>E+#aJs{_Cpve8&^U~u5U zW5a^h;lW9|g36o&6;+A(c@L{wECm>;teMf+u%oiz#O_mg?%K8ezO*Ng|LS?xzkJ!L z9|KgK|0=|+w$AL^ul{qN`~Vny%;Wpko4S61>Yc-8;b*P5IHKN`7}$cTFNS?Ml@3W4^qP@c*qz7>{|O)2WYmQ2@E>yDq3d!0xR*f7WeV zNDGzSa+>fUx!m>&HB<{BSwKk}IBjVVp>48UcY^(qc`%y5Lv_^AdW^z5;=>Urr|jhg z+;KBMC;xj1VjTA|XJ5qeMR444rw2zNos`h>Wv|6N-&@WP#JjV9cmB`U8CFnWFCBR( z1aX#=+5|;`8E}b(8|U+wBKr)DO^{={v@FX#S|i^ql-dYE;aigZL_En##cs*Gl##{+ zn#-4Gq;9d9kjMlZETwfucRtFZp8>)f+*PR44fYZ$@nSo(Kx;$Kt|nQ|#^O$yJ!@IX z=aSY|o(o+hdu|g@Hz;_sFOtXN(pRDI)_Er= ztWL6)m%G9sYd2Hdo-(pg`#fkd4#mwZc3xZg3s{k$qI1aH>_iz-duUv9qp%xZsPbZ^ zy+7{MZA}0Cry#AP4y#SkAwJBEtI92C;lr|ES91edwDG;fPNex&Q@juSw6SWiJ8lw;xvC6 z4+vNuroN1^fHha%u5T8pp|iU%`wgSLP(_G19UTn13AU0dRB|AutNMUK@BP>mr14ONS>NcTKX~maGDy}_c|fQ8 z`^XOb{-7zy<1sJDh?m~**$>$ad}l7p7hq`lwKB5tzVO#?Bx1ziFv)3fiG^vmfpuAt zrBMWXRoY^9vsYW5MLFm5%GS7r1zWmQmM*_u)??U}jE3d1B1=H$(R(Arpbj>tUGX1a zcbuIK;|Za818KbJF{TXXMz6u1>zP&ydgnG4CXv0hTaE@jNci*KwxJb-_|)M%1dyn) zEVfJxtcr?D8q-sXBXxDSH|5biV;{7wwC5?7;AqxPgFhv86sB~pb_R`zh}cZ(S1C+i z+)S-3G<+JF-J8(twY$_D7VzivMZZO}Sy*f)chSOajHDAJb{G9hS3R2l&l7aux9Okq zYlJJAq`6@kfS{d5ODWq0OjmGTHUOzES$xFXo-XN~%zC7LL4F?6*2Z>#{H_)w`w{5F zSeasICaVNW3MIWu(%FehIw@Obg!<(X;-z3+Xi<*knW&4OTI@Gt>^ob0)kd|vYGPt+ z^E0W}&mNFJ%oE(EI69%ret-?|xTMxWdM0m_ES*pNUhdAC^>IAfx=o0LUr*jcI2*xu^$&fB_LtqP3A;Rdq~) zQTuWP%!wHErr7Q`35SB;EHzpo|ER5{5p6bG9Um2e*H%z=y`@BRPlT&aw3d|ZN@=$l zL-Kh+m8 z^a~t5i0~$hY2d6aV7uj*4fahG7e(ch1lsmFJ^Di(=xn5GgLJE?b8dDAv6X8FroXku0m!5+KxgAGAL`L2D?|tWh7Hzn* zLL^?r090Md?mjY4EPc(^-}; ztd62aHM}H#z2Dby8crY@O}{GuY=^9!G2db`EVkW4@E%xA9u))_G^+Uit~ysUxI@s} zz)WH=%t-rnKD|e?I<1gJyE_C`yZB2Sw*B}kAGjLqSvMARGW@7-r(vdYH6|kCGGRMN zE#&46R!2;E8InYH#===}OCtgsWOhQk&*0D&FrzxCcr#l;l9W3?MVJKIwj;{}f+uY5d0MQs<8?{=LqkBq$_6Sx%vA#$HxPvr5pH$Vt_@e} z#Cfoq>j&sRpUcqA5KKs!<&+hq_?J)(NJ;&G-&S;6`aQ!QcPIag@9|%i^%MUU_~Y>V zzX^1JIYBI1SRkNl3?Lw)ABSKo2L^jH*L0d9Vtjy|qUt)f|NY%AVj67C1d5}_vGrE~ z_=tj~p%9XolB#wEm1rcaPvELqIxVblm1r@!-Sg(yScFyb201x|jvUz5MrAH18_V~y zJNM;!3(Lo9{4c40E#F2<8|Qp)ce~)k+^FWi`Ofp^&;9n!d(I!K=l8z3kZVgl@LHnc zOceoCv5j6Svr6pa8@yv@9iV4m>+(%rIkTv?eS>G&JD+sy3flIUfUj3H4}QC13%nEw ziH=h(_Iao0_D$b#fqV;>DP@T*0|>u#{f0%W9~BOXyLaL=i|yF0Yua@B%?lrWze9QY z?MqayzMJ6`7w-3fkU+`0?MqkeoQ&z^9WEe>AZ20~FD_MLRn)kp$EKoOA~wB!#5fV+ zt|pDLm~96ao;>WRO0;h~9xsD4I36ZxAQ+zB`Pnydf@vgeUHXaI;pJ`UJSnHT0#N4JV^EZL?>WZF!; z99-%Uididp&@N;l^lmx22e6oI8zc$xL(%}AixQJ*QsJo7^4jnZKevpcuF3V=&JxctstLd`s>yq3 zQBk`Zj=TWZsyp^>e2B$c^0{!7U1im~Ma3MJEIyxa z0e^9RGEPuOZHu708=+!1@(QrkDfgV%r2^h!pW`lJV@tVL%O2+CVX+mV){`e33)Y;q zM%j|bPN_Or7AHrR&9L|l)4Gt3UR4?kV|Ui(%?@wN-nc1|3nNBVrpOVBwk3>9MlESn zt56m~_vlGmx8T25AKgZ+Dz*x<+MwiRoPn@$As0%&po6l@l6@>EatM%=49zJe4kf=J z&i+Moqt<6hdk8DxClYMkC=3d=nqB3)^KHUua1|Z7qJ4_`3psJ%U4-hDEEzwZppcOH zo7Z>iB1t&?($0~S2Z2+^0}d?7#9CX&fY)6zS>(`;1%?$Trh4(Paf8 zD!+MR+FRI%(cT)lrbI7;_^PX(Pf7EN@`UQaU1A)AIIuSJCyaBngPUo71O@$+yw`SN zdRCVkR+0>k*xe@Tbn{0ZDux$p%qJny( z+%$}lEbGt9D}A6#l}1+#!g(FqEOU6}9y76?R)!BGUo=K9y%N(EqtYxd?X& zdSfi2MNXK?Ov)hQJQ*`-_A^vRFFU@;!BK^a+Xyt+vk;n;V|Q_4X2=zSz(XM z6{`J)F`N8cG_QLt;=0D;QrhUjrlXiy`q26HDN{z3-3dqp=Ri*Z#|8R><18q!sThK( zPaEn!Rgf)hSw>K}0qo}se!4!e!tcvu;9s*SNZ(=;k=g zC4N6ssQ9arRFWMLJ0^>JqhS?aCNmj!!N|hYVS)|Gab`8u8G$&C?OZT{MWI8$#UM=5 z3C!VH32+Kd+G@8th}V~7b4sDE3#+10J_1=IjzuF z(6Uc+Ve+H~Wb`&@WQ8l@lGW-&v#Dt&Un!DQ3ZU=nVaOpz&n7rLgu5)+IDtywsec`K zj^5^){QLwC0NO~gmwVu7TQR-kT#9Ri)Zd|xSMClvFT#K`nTxFELSK=PXdppNdj?ZIeO1FFLGzOnJL#gOyH$ zMDeRqz$nxOAnh`TNGaN}098n2iu4D1S4Lpe?2CRtKh&jv0Gtbc;T{|F8Q?s_1@*u_ zHTV_LVGcZ=Akpfmi^4`e9P$_b2zaC=MWw%^NF`2Vexlxy7jn|=QLbWeq~3G)mM;jR zC-rIKnE&!6mLT}wS@JkNPE%{ts(lI7xRs&vTw;yNzSdKC2Z~b&T-2-HQgBY+bMl! zQ7N2j)ZMgR*dH{y{ueJl8pStHkqYkVa?8^2?Wh+2%NHxAF9zZjbz9ufDDo+#f0PG6u$ZxXzLe`C(a3;=`^{lu^Sl<<=4jA7~S~c!q|t@;&?6lSfB2Hp0 zPX3uFy;RI57^jpc7K<+=Z(R4tGqSgU_;r6DVV;P`fJvte??3T-z`~z>hKTZ|0hRcq zbIo@}*_kQe{1Q8p%;wn9j%kUQPw)CZU4-uP-IX#b>Q3HiL>o$|;wCbcEeI|b_6e!y zVOeGNrPqqaWp5@h*3&e3CG5TSC$FU0`vu{>>6Eh(VWwYF3YEz*040J!n%L>#e~|y# zcK*-#`hQQ}%|na1D?c;2^bgybJ{{PP6TO|akrM_K`ad*miAMARxb(sXCMm$J^@<=G zU-K_1T#asZy=u4GHMKWT*=pjUv>-cl*lG+NoiDEJ+}&IhiX^!dKbU_oeWKqcLW7VG zg(Eytw&=DP#)O?7_H%sa&gL1v1p56U9Wm)(Auwch+Y^S1LW%7&Dl$7J4QnLHg4M!& zdmE<}HQ&60jLJw-Jqh|QVE}+|551}xci!;s?wxe+Tr}80#W<`p_prW$UZ_SQze6ya zW5JqGzp$k;78`Bmih zi#rOPSC`nouf}tT8aEh1S=@CG*NsKH6Qz>TVvUF!8HEB1pW1nj+xr3dTQUxMlkD47 zzfE)y&=7_5RrzHf2%BZi@9*UHQvLqMZPROj^6aT!ZE+V~3fMNa^)7gfpjLVme@SfP z*D6qaH#+M&kggtNLWSwqgQ2RB+q3M<7gcWKlBGDs9SXHI%a>p7(=)27@R?;OD6uY@ zda_#SD97sCU(*z2!3+XWcIhErlFQR)z;e~5Y@=d~8@-@@XC~p%{=w z#f}2MB`$P*#;A2ht96*@wQm$TK_m_|2~$88ArmKq7awi@C>FODp8OzX0UwOToT zW%y4`{a11PR8HxS@acco)LwU7-zy9dQ2kGOAxXMZKMrPk!wwTP;Qx{gHN0GP)v*8d zp3KTJ`^~PEuui+Cm#|2Q2vIGEu*zjwFOd>L*NVaPkk?NxOv+}=q(CU4nJNJbDgE|? zmL6zJXi)?%72HJI;Tym8eDTzc%Oe5&M!UZ6zMsv;ZffDOxa*$w=>Pom-*{TTyd4Jof_&`x1z<3o3b*=&V5PaS=wkFQZrX?Y z55tx1{j%boUuoyMtt=VI;@DhCDr54aXmZ$4--P0fY#B;g*lTkHtB1r*!8$=1MO1rU~WSIVr5^?mYl|*PNV~8Q@O-dg5pr6ZZ>=%`ef0vz1fB-j|iG zyMe(9;pUP_Vl~6*Y}g21?^&=-iW7gF^I+QRNs!u_yf)9)!t<&_)}c1%y9N%jsQ5RU z@b5rkEkp6*7B;DE_4&R@99vukZM(`{unuuNHrp|XP}o`-k-vh}gK0-DqZZ~o!!$;E zses0bh*4CM%)^_Xbl-NO&xfKgwsN4XGL?m^Zd8m$`t0z8GJ50}%D4GRVRbf+6z(&j-K@>(5(m*$?lL*4G(dn#*xbj)%4LeITB*U)VMN$NO?aOEE}i7q zCuamaQ~Z*3k-2TfVqU4}Y0PAsDk--m8PuT_4!dP68S3LBSFhoEp;u2r>MQkN7G#mU z8Od|bh1K1DLg-(If4qoAE|(?8UGAtE0`Q;Zbd(00iVOr%P9_To3x5yYl?z%F^&iyV^|!R z{umXgfy{y=HZst_s&OCz5oU@sn zlf6f!&uDVfy=QNoOxT{1SJ4C7k6p^ky(1#EHxa9u*IXRGGnz{R)`zFf!{fL6L@S>R@j2e*%$!Hbp=sN&yOVem z-;@M&G(I>nw9UdS+}`WtO2(AVmUg0(f)6iFY3Ngg5n1a`O4f1P_>{ABI(2x=J<|Rv z4vta;b)@_6`Lkdy#Pb5#CBPhh{xlQI0)9A`IdhW%`A0?Hd9_1cNg>Vp)PC8GADIm4 z?k2;j0}D52duZ>!C#n0>x1IwcOUOizU(%rHe^50Y!9I8pgaB+rDf3e9?vAZpFT2@hG`-0_tO#MEioIiZnk8L-Osz13Q_3>R zX$t&4Bq({!t<};)_EQ zyWSCSTZ>{_NO%sgkp}2dcb8H?oSSuW9Ry)iuEk_mI1M@(B0IPK3+{!(>ej&pE*o` z`(}f7s5_rvRo#RjC%l<6Ruj?@Z%x6;@+j0>DDF?__CU)x$zY6VV;bMWMYPh7Z0Sa@ zWx*mp)i@FE9SGJ;HCRb*xM(^FHl|OSW%uZ4aoGW>5N69!zX8#E7OHtu8*%Hj+PDLWE${BB%xCua22yz0@r}M@ z8{&IrVXm>fypslhXl*^=U5@~B;M-hM@!O+6nXgxwtGThZXKg6g)~Rl!PBl54qO9}B zx-p*Z2xot!qA?_#9E+uNI`vrlOB*rSVnYtDUf6Le2|gLKp+;M9djE@ zEdeg`3zzRNUZ!pK7roMom#d$f;AE+JnE&WcE}ff2S}I4pYkSaVihW6cc_1ZPNW9s& zOJWd=EG;a4OEknx_Vh@Q`&{Gdk3{j>Iuy|O;Kt>CIf8t=G8mUaNZT)nNYK1$SVhH4 ztq6yn_E4T|5B(pq-Z4DVw#x!d$F^VBQES(|w$}Dgh&tPEl4IMZF4jMCyZf4*jIBfO48ickv%yxfxnb11Av7Sq za#@0^Y0yymfuE^(k|I?<6d}r25B*1~sr`qK9u0|nRu}FIO!BMagCw)>@-W^$H)eKG z?d^Q!zO|*`q5>aw4TA7>&LA}faF5jjp#E|dm#f$&3#*);*1bGr*Oyq_36&9v7=P@Xo?}C=8Rja1<<#l_GD(^@ruCw1#DgT{6mC%vx&s8UttW+8g z^&|0=zl|vWn=Jo-X*8%@-S5qBARwXOKq6_De}`bffm36A7=Rd+ zHHEeB$O4&B2==ha%_kT?h*iOhaF5B7p`bCOtb`%Ncb)cZ6U8rad2EzEBD;Tz75je| zsyMC64y;IBPF-Z<%i(dEJbyW=%_IU@yjctg(Za)k;^}l!8pw+A!h?ffEK&X8z_Jgt zmJjVyn8t)pLMR_NI<}Dw9_~C)F$J`vc&L{0zi99kx`_62lz1jNY4Z6>l*Vr*f(>Xa!^J3&QD^pPV=r$o(`<);X&(3=E zKs#(dlU7mU;5FStf;&8fdZwUrhu&`OLRtX87bSuA=EU`8$%LadyI-B!KL90VW1;DG z;8$~0UVo8sXZo5cjf5? z#;}C8%@a~O)dfq-vC*5hS`F-cdSjbQD1>d#9kv+Q7^nDC!5xmyk)q^*QB6^djbH+6 zd%F{ofaGK9f^PMzHUA_KW4xd#^m6#SpCy=844 zSN^{qjdpPp!Tx;}xDU#6hxpGqY2HcHHX;Z}GSz>-{a;`q0JnQ_p#WM|hN??$f*#FW zY~HA-#PRXLMZYZBQK=!JhNUotK{HW|Sy+*Y5&0z)XO|0cyv$7tDczfS?dHr2?NBp{ zDWpZQ@V_MQenpv-+U=IC%<=wOG`gvUP3&gOWB3*V|bvO7Pt-DOU zOZdKEDGJJqtqrIKzX9%#lwZjG{^T7Mcs@r_6XYr0-5TKDgVg#`%H$SL-CTPScK`0j z@_Ir=GC16+^?b4+%!!-TICjvyaPa!8M%ay6Tsz0*`PYO0O#o@jG{NEKJcQaWvm33L z@rAM0FV12sZWig7=;?vT>xoML)q>C;yBLs9baPz``L2WT&Ifphf_vkZ5z5cKJ-3K{ z6T%q<++1(*zab$p-Cny8`uFyK8Y2$`{mi)jR&d=AfQ60r(=Fn=D!6!=1U9K{I5dAa zb=2au;bOS9GL&*%puw3|cR*OmTRSZAfVC*QgjKF)Kb;N240`dqaSTZYM0$kW&?G`> zweRL694yfapD)1J)Ab5PSiZ2_)?kPc{qV&cuH1DCg^vd)x7Ow_oKQD!Iz&5uXe-=8 zPs*vJFT}^kVs!L^XwhKhS^c83@|H z;xtl=EpK)n)G%jpu}-kK!mh3!Yo4V3l9nQS)1h6J-VB&T&|y<&RZBx7M-6=Y#bDWA z?;8U7BVE;K8u^TQ%ff7EWj}P8G{C`?`~_`e7^HzYMMV3|`P4+ZqbssLkK=AHEt6)j z-ot7k;A;V~pfbfzGu%X@P?cu^ngPx4#wHt7b&lS%jK7e7{A3_NMVB)hx^O8Y_+aHs znZWqkzqbq^=wffj!W(grWb;Z+6Y&2{s5qg(m$=-t0M}C&**g{?C#*Fbtw9kwV{1OE zOw@}*S3NQXOVWH)qG>E#RSb=W0b8$&`*FkH$;m{55-QdQBaqz8mJ~uG+++el=k@r_ zM#9cYlHSSM(n_*hWn>58w-?Z@b^#cW_X1h>QH&12qqBxy)vpgh3NwdeYY3?mnQSh3 z1;%pm>1?l7l|_JG2>=@{V9RYO9l|k%b=H3-sbG4`^KN-$s-@Mtl#i3I5xA?r9()X? z(lJ<`efHl=HxtmxO~K#fBbBm3@B!;)kHM*C+9B*JEYB8IwKK63u2(CidjHCC=I%Zj z`G^6U8!AF6TA@+7YbxqIRtIMq=N!UcSJ4R{lJHAQk(x4YE67Y{{~mL%&XrkWr~vwL?S>ig1pJSF!{csr7}OxU&HhYkM-nK=wCRrZu=4tHx|ocgRP19ov%z zJt&{aHvGv+^DIKxXyvXfVB43Hp7uVyWFxtEf~&e#O}^?g)Y$C~4P6f(u;C}0CC}9N zs>}}g7+(Q)mH-AcRirkszpk8IEk3^d7DoZ)?%p4KhJ}Y3V{m=}xrfW;`Gbxj_@aQ3 z+KPk5l=O2x*T0PG)bCL=AyA1{UQDvX!CdR#ndYaznNy(EjixF z2Zk~5+hX{Yc$Oct4Iq>(OCV($ceDa#ifq`eW|QudN%YMl33e%ya<)@Q^;k}5i;nf% zw~C@%dr0){BQpmTRPGyYLAu7!TcCfX=#_YeEKdaXSkLYmkD&LvZlL=N=!>pf&#n#H zl`K=)qcv8D1R^9DKA@-rN=g3Sjg`?L63aQoE2Qlr%SH9`uTS=)5Ds~^2896%`SH<) zXN-ln6o86wmC4?h7GdnL$z}jE2J{zlqQQl_%aCF4VJ?fQLyR5XA+grLXd#YRiM31d zi?3#K{sJNE7tFr46mUk}L%T`bG@W_*bzItOPD#`WTQ%`rOxl}f$Mj~Lj3%BxWpUwA z&DDBEaeWd$*xpw6le0U0;>`6SGYVXe1`kV{=aF9#ORM|DIJvrpbLs&3{b>d3T}Wk< z=;rifw}O>SM{c%^2ES0u#hPVG*M{lte~E=qz(75IG=?w^?MP5qhAa#9i&at_UibYW zaCD=<4CQTQ0`6wB;hTAC;pvQvZFgsJfMFy9LteBw6n~L7_B9M(q?M6l%axX>{Ot1b zdV}a;F-du*hb^&_{?QM3lonq=S=6Ip^8!zfd~Pbhk zE@8|l+VT)@=f07GPV8i2O|9J=7Lql_^s!5L7#>ioBN|SGS@fHbNACNCz7-0bMWP>j ziA56iYeK1r>7f^6SLfOD#n#`5v4QEJnmOj8j5Dm^V zAxIS8V#3a-TU7!2Q3~0VNO&5KtJuw@paf-%?JF>%hTPaWiRGq?B8#yUQ*WM`Y_3V# z>g8XY=%12vLFAQXtS-NS4yh6WvP7mpoMaRFG-!64T$1RXyklB&E@XxqY#KQ9%-VcM9IQSF}}8JG(KQjnI)+2)V+W< zbz~cs*Ifu#vRrSgX}d%;@A+LoSVy)P&ISV}>@0A>#I-%%P^&NOTx@Qg$8NYE5suPo zhm9!k)zJXMt{AajY1&gn{Py>s9=K)`(+;lRt&URzCmUSYslXbJfxixc4NHcYc}vEG z|BXp`00qsY8FQfgHF2(Lf1lkZ_`bPQnh}o1xM{~N8e!n zc~o=6?mt5hh3UL&K}@e}Cau1DD8jP27{=P>21ud=cMvv@5!8w99Zz%f5#YRp z*#PRuH#@{h3f(>d2;uFrIGJFSM7f|YAQ0a^)+j0*0hJW|Xe!1R2x~&l>By2Y3kHh` z0Kl0JMcpT6_sTYkKmbxW&*A@aLPa&Ah-7N7PNG}gvCX^17z?&n*|=u2KVnAr#Pa&l z6?o$rdstm}us`X+@=tAoMtxhhQg5$K>3v@8lH!1T z52Z?QP`lZH@qjObj5xQ2Cr#PdRPA0<1C(*#%Ew5kfU!FySy~@ogby%TMAIEhpYo;5 zooT}D@r0Y8(OYo11{7p1$f}|IwK>m*Skc6{WN(*>vx}Dn>Nj z7K|0+rDQ@lp9aNoC>)Y*ku`>V&fUgt%7}2wLDiYguh>%_*e;o3s?*sdCEz4B4OpAQ z{ceRgq-eP5eh349u+Ez%1YV5PKYpbG=Qp5MZ-?uWxo$&ZQJXMd^v%u1zjuXOv8VWF zr?A~gC^wf>NKqjR2>k|MAS(Rr6L_3HHvoQcY+ed!XC5%$bb-g7g}~}0GRf^E$m7!f zgADQ2YpNOQxaCBzLM4XAdvV(y3s9|LHCBilDmpJ13U}b7p<_LLiFc34na9Y3_sv^T z4N9jA(NS%-$&9a6X+b!5sn?C$(|ZNczqBF<=uO}YiumEWg1D)((2JlCl^YmzlzOr4 zr$%6V$NL12@9X|`WU$cdUK_;J*9-hQ0=w^64RC3jaG7>WvA^g2D&PaU>*1lU;t~C8 zzdq25z!w3ryIT$l!PQCnv`JK%NxbFoj#{ZU6SGvGsaCGbUSF!KJ7V#kL8%t&dsjEY z=>vgM?NxyHdmin>OJ0dSk#?lFo^GJ`g(ESoZ%Ww`Qp9H|fZ)CCKc^JjW~Xk-Dz%-f zQVP~(c)(`5WeQak5GaYgK#ALYC@(i&~5^w!bY_fh6aL09lxK1HV6Uogpv(?*p~?rnxPCor2;RM z4cl~o>rf)-n%@Ngsi(}Uoa+8gnh@Q^cX$nLA7qn6bYixe5bigbhVxcnJfNc?pkvWd z+zJzi)9w4BJzDQ~;d(K}^T8Ksu*K^SepiK8*~DhtGj^xF@^8A?xJgw<%$MrWX6f9; zz4$xV%WwzKHc1GLb-0<&v-}zlC|9l${iCCcF;kuIZd0}q>B&B8{XH#-<4M_j`IRgF zXBc;6EGz)bDo^rQxtIr*2eE_EF}q#i$&9++B+j$S)|ulsZ%mCsVdY%oF$*%(uruN; z{)Bf5VOu{f4MQv4U;OnSMO~X&0|`ll$wo{ZQ4BSJtg0pbC7ar_lqGZ9CFv}s$X%S zu8C?|c{9^`w{T(Da6Lx8l4ch*ph#{#T5!x-C-L9bd!Yh2Z6k(D5u1-TT5Dk(kV;q|dcazmC62jgTk!g93Q(B_XkL50trJ z&A6=wU^#lgC%%U?##1hT4d|5YpA>q6Z-qD&)}r6CeNI$lKO{^0*od1jG`kcS$P_n3F&k zu%Sz>4j(|=o-Lsv!FEi>q+&3zoBh5^zHQu{RU7&@-^ME`yuw)`b?v46GjONw!nGs> ziufC99veZFi=JUbR-2z88;YqDq84fabewph8v&&@Ld{W6N0PxKsv|s+V z=oAs1v!`R&2ie+VfQhGc*J+rX@@V$GT9}1;#W@0*_b!#JMjb#4>k*1o+|Ib=7u%7V z1vTWTTr-`no+K_oKYS~J^D^r*Ujv9O^BTkC_ zkAm!6w^dxGkp2#xzdyznfI0f2*hB+X)W2l9Hpm&BM<~V=$RQ z7G>+u^A__n{fs+wj~9`&e78Q9zoxA9k7GqjwP6Mc$qgH)M=C4NFq&*% z%BZ=kZ+M*BxUs@;Cq6v<@e?K-ul+8cb%y$n(opihApLbMg%;T<0Ou9V;ugfX96g}P z>uWZx+(ferf79>8Yiz`6iM2#PwP%#`T)pd6pp1ozxF7)IO(#%m9(Bac+ zjgo=Wf&KYqTmR5r&r*R5ag<`T6Z472gOs~AG5PL3`LW7pKLx0)A$b*5+VZ)e`7}H7 zT(p?WLrHiul@0HF05)RD7K73!KI948^w)LfAq+ceFTl`0fZrFf~2^ObHdH-#ib-g zxwq4Z(5B(vhfquiE)?@WAtsi8clSgL@$w$V46_45Cy|?_$jeccE|aRD#Edbd%@Df& z?1l1!!j1UKU`5Ed4hV}VLefxCQ;dEez+MK?u}h{SHAz_$RB+fRB%uwAW|9#^Fb*dz zxCgt|9#HN9*ZBV06V7Byd;?7aGuGU@NdjbgQ!RD;dFdx6mk$VdZ=7}#;+8>h#N0^0 z7Ob55PbeEd+vB&4Glmb6($)e;7H>fX+S&59@~A*2!hv;521sO zHJds?3|OFI(c@n~SL9AkNw6LCMkrJy8TpcbLNhH`-1x1?$P7cR;pqT;S}7#17Y7_ZM|lUFu88FN~ae1ggX*k)Cu41QHVB) zv+3*w0Q@uq^7vxQ))ji2^?QJUu%4)%p|&eT7>Ci);V%UEWY>}O4Hqba{Z>&GR-`1d z9AWwZ2IJmb?@JeUTt-{(?_)5MC21T5E8k>WkuOFt_T1b^G(pD_+6}2nWeqP%%={se zdvT*dDpO*wpM_14;FU=cUQ&q>TZCE5gO0mq01QR|WUHJ->Z4gy4@~>C_$+_s>ui%X z>)lFqreC~^^jICXC3zOmL5xdtSOS}5XWM>E32W4R-ttaK5)lYCGvJ*w=47-!y9>1P zyr^eKxmv0m7&$Bm6cJQq)n)ij=y)dnbxhpEmD%r^5~Wp&_Zpx@KO#DCrR#)Zt5%~P z0q3UI((INtU)t~GCZ>rs=pB!YrMO{C6pc}t_|0}E1i+!#{)hH7?bb&Y%)g1>LrT8J zWlxqk!c>S*o`>Hx)a)(rAxA{yS%q6aO5Yd1d(|E5iq6zUfvg#!P?$ z-c5Xa&&I9cu2706=+iG-CDOhlq*+$IBXnOg^r%yDK69GM(+n z_bBsfe7uy1ND#CYf+>hl_bM!?4&L5yeK0tR=9`D4wN}0+v4$^&hgtueX(u^MmxTuw z5TK45>YDxs&q;u#a$>|RVADs2&c(n`KtokI!{~2k#r#3Mab|})*#*^3WDT`^WqtEH z0*VC|EPo0>G@jWiQ` zo4@}}SVYGXNhcrRVTz;dQP4q8LM%gGXaeLr+pq}^c4Xn#RT@z$Fb?xR%yol+&e_op zo?dF>fAfoOA4aw{rWb0__JM(T&~!5YURM{0wXCE=8y}kSel1c?m*ZeuOsD?5Nof$k zmyLe-z8H?N%SwM~s>9z3sJE}Ma`-ePkJqvSS7oD}jfldZdM_h{x>qWW$+ap~gago6 zWBR3y7GTvx`C>Mtxv(%@?*iPn1_OVu_NXoo`zy)F(Sx~2QNmTv%JRM}E$o0H0BES< z9Uc=)Bn>ONqW&PjRiN8qiHNT*W6;7qwIM2QIa~!@Vd;d?9yHE76@uG)OoP(jvaR$_ zQC_hMaM-%tEQR_1HC^`ZyR#>Rx&<^`k=(jC%si|@B3yqq>b;DE7;rz))$`-ogzlkF zf-LfmP4{VA1)%$5A9>2{C#}><$%BC9!L$!l=BO=QZ!yYd4>6>a`9h)L<8aZnxg)nn$|y9 zZHcf#00aFO-YuT;cwu1w;#*<&PVwXm!61-8F(O^A$N(R-qFy6?@E=R#QP~JTUbdoE!IP!JFQ$T1PsV zF6>K9jfpnZ;3d#GTx}-|Qks+O(e_#V9gHN2EiK{J1(wc#8{1n(vpD?bNEKygI9+Y@ zJmid)vQUK817Ffy4)+;0+;Gz00WqybR1rSQ<<8n{q&6y}b1@rgUAud-MfcU{^S{5_OKFSzAYY=-cCeY^^hn!LHaM-pQkx9w1~e{a+n5de zV|c0{uu56Z(P5k^rP*t3nb{l?HdH2?61&vRomrualbN*3a$5KwaaoGE^&8e?5)u&= zrqi(#7M_cTKEb=m<+)qbi_gS^)iM31!+?W=tn@6Ipy5B{2^(v-QX|-1NTA7?>*5jS zR1eC=EK~f5suU*4($W@;2Q-CuP6)VfqlDA|jJk?QtxDFw9#>|vX7t8&*+<*V?7~g> zY?gwlZ4W^`p4Snp3&S>PFkEwyWhe(JK%{14@>=M{4B)g+wknGbMiG2H?&#qOuxUkn%J`Xl>;aa|@w)|_szBZO3V_TjF4kuw7CREJ z;CKRlOOuS;mH{?pb}OcUj6nYkUdJVcdA2xRO4bvNGLCA#Y{4E*&GG`P_7O_?%=hCK zDSAYRh2pc1eiU?6^41}@&JxtAoTXdZ06n2ZRaqD)N3OttM&U%*z`TB5DB$G?4{Xa{9Xv)*lz+VIdn!|9 z$Cr)3*Ihvk>@|lX2QdG?u4MPmFsFZei+y1$OgBg*aWb%+In99>8c*klQpZ9ao5fOU z1y{YS+KNwU@AIZ14cQo8=r>?DmpP{cMwS?q^1^G<<;ri8N1*HRo`Ba&^&OMJdOsDt)RZNW5HY5kIBBW z;}mLUUU$X`URfD7v8^~$TgwGElWF86sahx%;;1sz{`3+Q z>dLd~D`U}YjTM9j?3D z*r_c*uz(wqZejXdS{*q?%6#D${>D13wf#oLP1@m&(r*)fl<648Z;C8rhL}^__bk%u zN)y$swlp!;0+lur^7VFOn9N}~gm-eS#M`uJjHEQggX5@dR_3fy&e*fn++WK&uxnea zBUEH}FJ=dRVsZWAyJ6pSrQjlTa z7e@#`XSX|yP7n*xC*CQRpfqIDN#Yx2#_JA^aK3|<4^;h*P5Ap2&SX<75@cvv_6oYD z>gHH9A^lH;#~-D59r+o%d$bqFFx#w6!HfuGQrx0S<^ai2eOktVqtVTO$4CD+y8q9u zvL;Wb_g}XPl^&E3_`kepI|xVMzj=`j!0w9+A^T&(LKQ~_)^5X*G8K2?-yDGTrlrAA zQi_vGRzwV$`Vg$9nAEndCEhm3`AWZik)g0MgL^9Mqh_`6z(q>`*0uDNbPMy`^bMq~ z%`PZMFndhDR^D)U%KFw&X($y$bMP#^aaiE=(q$KlfM_f= zDZ|#0IwZUfv&a@x9@8-?7Fbiw#5$RWW)>4`T9h{PQ~Wbrx}ZAQj;9W6R2;X4Fs%%=a*VvTnN5n z7=5`<)6cri9Ss(@Zg=FTu?fthrqbFM7t(%r6fLi;`Qk#Asm~wkU7#)P(&IdM$@wvUwOMJJkcY_fnA*ia*SENiLMZ$XQ-!?g6AF1r!;*%7 zxR5pa`4jqG<0?S~^wTqA=@sy?_c7`sU&coJq+J=5tHbhJ>T~+}o<)*kx&nf%z#D>L zJcCv$Vcq7~nUc{513=7Zhr?T6yOiW3w^4@(XB&+VAQ9UXk;dw} z7+4&`ynzk%A~2!}I@5i@1`5QijQ{rdhYS5T2=V@Z|L89+$=CBpARxbpfh1JOSYOQ- zybwUcGk%;ek$V_J-32!R4P2}4q67***=94m+WG`@_Ed(kpJqA|nKO*}DEi-6QYHWC zz+k~OLK8a;7Lmz}k4Huo1!J(Erb*`f%BO=0c8Jry8MwcYrV;B*)H0BZW-cstAb4UVsYkRLKgzFJ|mWy`e{%RTOU z4zS7*wdLnPc;AiRT<~wz0X=hMOyvk$?AO(#zIa8nKljS-xzyPP!a(f9Aw5TtoW)#xacDB$59M%W-$jNoSTjId6?0J{;U5`Mb)5Ru_%V1blaW;sEvmDP zdf|AYh?tm=-N5!7$0|l#8FKsTOod=Fw;fAT^f*W|H*aEmg32>jXaUTA#@@)9!}p2) z>TEEAi=cor-muj$sEn}CRj60+6oKKO7H~V^n`zBs)s64W`RArO3qsSISkU*TNA!S} zV9pk(P#qoZx&2vm9&zoQlo4EKm zr9y2B9Ip^Hi_0wj$@js9`Q0ZGh53LdrEAOJ%C zABOx@&iF=cR3a^>i)Y;6UM=8cbIsi!FEvHo*-^8=4NwzSLfHWcW0ccu)il{dFp1)^ zLp-lD)7fc)Pbyo8!2mJGeQ+Ln9(wEOY_Tc;D>{PQOzf$g2XFc}Z)**CPJRF^Hgrqy zE{wRa1oo|W2?sw9VI)*Zf5bF;5?LlQuaB!Ydx{H=h)zj=f{A*Cc>`%wlzfL~%tn2} zpvEts89A+um8=8O210tRZ7LRm8N+3)IC)p6s2#M1oh*+s<8L~K-mdJvGD@i;pY6xt zXsWCU)#MM5XzS{pk;6eVCSS|XdeQO^N3f&Bc4kgSV01VOt$j(k3tU@NS$_W5#UDqjII*z(_aQ{Hx1Oo>$q3(rf@wjz36}&o=5O8Am0)ts+KT>jn=m8-)nNs@D~2nuHX zJYlSOm5gK8?DvFdY}Ywz`B`rg_X%dMr1jv`!2Jkj`Q3>6%He#N93j%>&^uu{@Pe1! z(=<$XCxPyOHtsxG;u}BUi5O{HSfEMP-N!U;I={Y=fuOhCX;rlPPosu=L-VgVRQ8LK zH|$fG>!7_~P$t+|RdwGqxr(8yGNBa|X5)w4S~j}v711PHbT1Ojt!{{;_w216q%s@X z;`r+o-&-Xz_&vq!UcwQH)cSdgVm({j+3y_Q)#F@cfWHOS{T=fF0LKxsjoCu?$Jw;r z?`$0-j?B1Gw^2}*xRO1RFGaaF7axQ;#9*X`T#jmXl@*U8O%>K6F!mHrmCbPcYipjZ zUDbL-sH#&;LK^E%NX?w^4wXHTC`FZ_)fKHx0S6ftxXr6aaBEpq;r1N_=7~@`XLF|h z)GI-8J9~t_BGuf0fIUL~SS_TS=^gyb(u^IjY*B*#Q~Zd;zK1?(ch-43knI_U z1}L-XPQ;ydo!O1qG|xX-pPM~kI>c~X_tl~a@+#0~`n4+qYAeiZyX?;_bFCcC(v-uh z^8F~8xtaU%YHrTliSukWcDRv`Ytl(l%EmRN#$Az`&z0&~7v|^3m0V-~l$iGv=TzSf zS(A#{#M(Ywbp}Arx+T(KCT8zZa0*B(z)AhEZL{auP6%?g)aU$5BtVEO%^fnYKws%c zX?$Z^($ftGfZ^=CNs8BS5ZL$JFv%YXB($l_ULwIy`{aXtNC8I=;92!`jnrha9FBdD0`D-iemPDq ztPEUjpcyM>Ex0c6cfR=>7ZR%w*R@0TJbQRKh5B=niB11Nj1|4b;*SFX(hyj5u97P` zd+iaEuQ<&UdO6@$gggyn&vhu=>KUWpIGZVwcy2x5T|Gn&2y3`Nft;+|gYuB>`` zS&OwC6e=GS08$gWnqlgnpjGpwAR2T{qdIpm0E<}eyI?lSY@Wk$Us7S@BVH(#Hw9B) z#s1&`9mm9Em7;%yEB{OiGZITKGq_Yg!=OU6IO_)L#ul35iPFfkUSqBw{%Ub#lAL-P zx#|MZKs=0pEGzyJ?_{`99(U9=&mFk%tSokh@sFY#qEy(ki|u?=&WXCy&M=C4g};3T zyiJiYJN7X@B?}%BNT{tveO%r3le8CS-em_x zTdzv}D0k5N(~I89RjoZo)W5iL>L^zfF#65FY7buqDa-Fdd<*{%Vc^N!ikoe0B@cZz z$C!l*Atx#|GI0>0ZJuAIl?wtxRiPO|y19wrUy!j{kjJVU=SY= zkqApwwdoMx+T#*cSahscD+K{@b9^HZ#=lkd$WJ@pwjO=v6Fx08e)bBczeR>qR}V_` ztELuPJoTCN%J@<=zq`{e`-g%w?HM};{n-tRv8nn3n|bQd?$0HGFpMt6!rS;)nyT}= z%!AG{we{=F%Ir#ZroDyvTfpXbfU|?Vv%^K70A$$|;M-Z%fwxN#?dowLlb_Q=Fe>+$ zq#;G8)(|~ZqBnmkKI4wG@Z*^AdHz1b zzfDD?lhY{LsRKJ5bKIi?KdqFDt2GkP?>l(neI4RTd)^u#?hX-s0&s@j=q-L+nUT*$ zLqLrP#31d(Wtq~7ZHiaLQ)J2lo!=8wf3;@!`LEjO^MqAS=3U~umlPbr!}?LmE;E)a zUp~(Z$Z%xZYTizCH5@Lq`-2cR@JQRHbLb=uG?Ul)Ev^G~4cK12uXL7ue_pYr`2YU> zMAG$%V)Hz9|1k`o1;{mo@{go^gP(Zf0eyl9+Q;CmG^n%b{)700t01EQ8^fx~V^k3F z_g6?%fimPC=^`t+zOcBmmV%Z7itw`~Ext7moouX!yMGv39K|YQHjjF=6@8A`;ab*~ z(~CC1+~BO*k5E>#hS#;1y{hRobB>RbM|fw>7snibA>&}w55On(@^)g}kly+|HD5Ee zP)^-2y{b<}D`ue{5jf`kWD4f31v(2+vPDbxOWoapo^nWA-8tNusxTF5HUg5OmPa;EO^zy_Jat|6Vg_(zz`& zJ}i^1&5$5Kk07p7Yg|)(D#*!_DU8{7s{Gv!s^=;M6*&H#K5XgrJJ5x}<8z?m`x;Jo zoVMd-Zw%%@4f-$61IALyV!DF z+~)$PKc6y3u4dhYS1pg_`K&_Oy?(Y=@#@&Kp{PUB>8^%tu0|=YCNcMhh4%&OFG@`B ztv?_gn*h2|>Nxup@cFWchTjo)!|_F=Vz=$vMnM6|buF>CO$7(Hw275}=OZ0?G)#VV zuW0pT#jj4}M=|sYdIUG!8CCKGPa73G66Uh2sT1z&097DbHkhk!x{HgkNN#CERVE5o zCPRi-_AeR%>*WO-qt*sf);&Y`Qk*e1&sxTalz=h)pW<7!&<6Poq$?PWB$lsr-o;!9 zrN9c};q6r9IsarDp&?hIVM>@nZPH~*v}F=y*FUBbc*pLmKWk>~h>2=;Da2Rba-?vX z{7;piF0V60M=>32pkEF#K)zO!4Pxpcv1%NH1#E zx&usCdC5~+ZB==Ba>7Wip7gtdu}+vC_!ubEyJZ{qZ4Tjb1u;1D5%}X}>|a=>^5L*~ zvps`xOiRbdF|pjOTG&vBzoO*}Dv6)Uw5C-38o13&7M3t=}Vkf7MbK zS6e$V1ABFZL$Nf0A|%#ds@H(OeMkAN=Ih0N0}Q%msSM5iH25c1@c$+RP=68s-|TOT zel0>K*njFK@AU{y;Qx6%Hb-N~5<2oD z0Gc$SG|-oDJ}2tW$0rf7rIm+YB2o3e*AG4ER%3jtgbu7>Vy-4g{U))bwVX|n$S!Xp zw|cKnrA{Y#O|v& z_&DfRYxNWm8ing$wj+t_U$sMptG{%uf*_=F;{q|%9ovl-_TtT_fG^ibTXx0^_?pJ6 zOFlpxM%5p33TvS$vy$!5nsV+UQ>V3{OVfzA+S-c}hvg%%gSD2WZeyu+6@?A{nla^n zxEl9Tpk1_!x$Sf8!*OR|0`8dMjVZQFL2~<8K)G2Krr)=+Sz4g;uN)Ma)LeL>)uD_L z$gQMna%M(8M6cSDX0BQ&y6WTrvYo0_9rS~D^Dko0Jo(UV!v%l2?x8^RHs9wtssZ;_ zUtLO}&c{h?h|S=TN|V%2!^L4{pC|ITc1%jr)tEK8Ju30-DvJ+v6{?u*km@pP;VT^Q3b*dYTh1(j* zIE_CZYNT}y2j%I%1acUtk%;mSMZY#YG>POyR#i!Rs^6mDY0A_|sdeTTNDgM6!_oTh z06QaFj(-N~G&J zC2$xL$gO9;)DL;$W{k!UMD|2lgMW%DaAu+nE^%8F!yZye&Yo0V=q#O<_P2$xUIAEZ zj1{u}$}05M`@jbE>6lr*C92+x1(Nu493q?ms71|9%Apn4a^wyGcqQF6l*G zYbb4P=iLPRF!^J@)mtGsP2d`4ep7T-?`lB$$81d}8mnK`)sv0e_P*9SoD*07Bt$nDyJaSfPImB6nLFhE3IZ8>O`VYJ79 z+(LngZrf2^OMdZ8ymYrLikJF5r`v{-a zm{isCfRtlsD{bZ{+nd3ItBsi~N|t`l=1-=3t9&3_*t(JlaEB%*b4d~K(%O+|yU;FA z1Ma_h4A~_>;&T?LuApWWQGgm)Zow5~$6)YzGPi4>3(K#Rg4N<-Le|8(Z=AK)7yNNZ z*udRCFRvZgWI9r#%~t(Yuek`;M$*pPY}41^aS*CK*mh7nm=-Ir%_Wu}lFaiacE5L3 zG0%J`7>pGOD7l1(kzBJQOWkWmL^MCA_Y(lH??t-+-$J^S8XFo^CIh3h>v>qs#7;8B ziv{WSL-Fx4;%|!YRVcjbNLQa`V$^;nG2WoH*h<|?Uaeh6y>~C~(D|cgpJUjnJ2&zH;=`LuaJb0u4J`>68zN1ov`Lwrebd}W?0f0)dCA2HFq2`62I{AUz*#XwAn4K%w!kdZ`9A*FrV5W%>C5$=*Q z9pRn*HcH(BY521wQ$C5l$)en_jeUO2R8{%_fU0D#DUjrxEv;$pi2Y!8V@|{J4rFqN zzl(CSc1QWyKS@1%Sm%}?Sr9jH)B*}{a47+v+beQzr2BUkFME9>?EQGx^RmPhX=pmi z3I8`Z;;qH`x6`x!%Hz@@(bk{+od@B}QLC>{&ofhTVWmzfFD2JrC$GwzjsYwyA)NE^ z0F|SP?8lM#xv*Nzr{&b2y z6RtMJLwlDuWN-^t%lK3EW{a*%i*`h*O{?OoJCT>|mPMNQ(#i#O3|Nl~D$Q`+Eg`v9 za&Ok`p7_xUzLGF)pE07Ssy->|zwn&c13tX>U%{E;txBS2xJ841;-IoFNY4**P!KW} z1ylYYwGF*8%$FL#fD1I&B$v>n90GNkCWRJaNwb}E(xVJzL?a``?h_2cNyns79%Hqp zg1!0DT0*|+7n`2Ui||O1i`Yn;IwB^pPBOukWeH4i)I6lpN<=gp3KtSBNB2@x0VItn z6_p!Bf16^?BBXF(-4|_i`r1*)+x8ZJc56zREc&bznL;5-qYkDEaX=eEsC6$oU^=&i zf)A7~K-pkP=QhGw97cXY&V0jG3!m;OYsC)#`E)z}7!S8zdB{hr3m|~o6J~q*hN(Be z&-NsUS#6j9Ix%MMXcxsAGAMlb0En(UB_DtR^-NXpiLLS0&*j`quo3s|Kba-U{`Dr@3EKyU*YA%eJMD*DawBaOTJ$YVW%Fz;0?n6Q1Z4 z=+&Fxi6v!L#eONtnZN~VP)!rXP>nNNKsSs@kU%E<4TDej-om4;fA}trziDotPV;h^ zZM`jcrX44rt+SD((YXwCWr-17f2yOIG+R>WuZ#YDbtfMk&CxbdopFqoQgSiaK5ENm zcVcp>OjMD%jQkZg)UPJrLQW_)RM0!~RAWX4t58kGO9vPmwUyaXG#yqo@hR$5z|d3; zo8eJoU~o6vc$TrL8z=+%{XSYLS-^0MPsp@ z_=Zo_zT#|!`G->9g7Z(@2(lGp{Uz1Gl3$3j%diV6U4TP49nxAl+GfKwHs|G8PBHQy!P;cp8Jtn6w1k$vv8xKn0GG_O1SbME zqw~?}I;Q+e$mD3j`?1zY8&>!<)KI#x?y@su$r(C}p6dqM+h%jzEOI6j@P5`IC&`kt z6m=8qlFIU$R-Wt7Xc*t#Ln@r4&WwRBW_lcmW!6aST@M;}#X9Mmy;AtxT``);%tU4- zX;1e`L!C%#jmnhECb-7$0MY8;0uy?A(HT9psszQrs>9&+7TeV^@od5?@3PYIq-KVR z<$P)HEr_){-^LU6xh8Dlrs@3d2S(?tiiIOrVT*Rx%rF2-sud=UydpudSyqP%mW*&- zu7934$&Lv-$RN(^wZT( zRqx%qyZ2hF*LElforuIQ^#X+N29^(%ddgXeM-UNgwt7x>kX~s`;v6Dk%U$;Ex7Xl^ zd|03`Br@-;DYOccgEOej??YzDnzZW4J>Re!_h+Z1UxgMWe{hofVVGWWjW~fWil`Gb zWE5>URS8A>@_S6~N$>5>IRmMb0Jy&-V;3?hjzh4Kf~s;XpQy#s)&WM6zSVIp{&K!8 zR1p`!E_3MSwNz3$&|@LgPh&(sVdW<^X1f}UBIoEIYY5z2-q`0+V_DSxesDBw9K+%7;%<(|~sitgFUv%;+IiFs1L?rp+E~IMu^5aBgHXsf7@la9ebDdW z5|pMnmM-0lr(IOr8J_31PIik_^oHl2HfJcgwfRdNml<{X?JfF1_j92xkIOd4%eH!1 z!sfpLzHcHjBe&P%z0X^ISFc^ipVvNj^xOsgQWAi=+O&D%d2IO;ypJO5df%{4VlhxOd;}_PA3zCVwk5~6CX!Emj=$s>pAFW5WZ8ew zYlSgOxFzn(tphEK@gFj>9MPtI(q%d9%ZZrST$_?Tq<-SBn|zy`kRs2Cv*%i{WJ=bt zti~b0yFuiE76@!uJs@^`eC`s+2c?QVb_eX*7g7>LDYSc|H>rH<%CiY&$BU*0OCE71 z5oo~i0bd%+{g7fZWbd4tz;C7wAZh@3HzZGG{&1uE$Yh#hN9@EZzZA=LEBtkQ_WT-9 z63W!yyt?o!bXIWf9PiE#+E9tg^WfVqiDtd2m$!kqP?O%EMzba6_pYAa1aziB)k#2$ z!{z6oXyWGSO(*o*hapmdIExnRc2mTH2lANF;LmecgjYU8BuxiS_10ZNS6=9dt}m9L z>8Fl2KmT4c!$g)xP$6w#W>56+xvK}ilCQtbU*)a< zCg!Db^;e|^4i?_i{JYt?8A)?kixj|V5aT3Q0Zh1lk6f`}>_gao^pk)YUPPAr>`5t@ zykJq(qOdT3(TIO=bR$^`6&^xZBzRL&VXJHcOV&Pa9ll!{o2;BfqgfrpDi~D&1xw9{ zOJF)%BNys%WZ}i~ZtiFLxAWrzIWH1|{2s#*zSt1w5kDrp1p)_%VRWng>L&!)gNfTm1tN^yXa<7 zG%R*cs8iWEvUtL(R`fEtj?ss{O|j>#l1Focx@SST*6CAAIWcQ`H@U_4Ll%3zWN2Wo z8Kg(;TRuUNrQT)@;Es9-RZasa#Z)Prq&*S**_%!}zXh$V-!r08E}$in6W(1)LE!qR zCu@TZnCQuNkPWU45!8>7B#6s;r%kwn*=SqchZT!?n-y7@1=`9*x4JV?q|Lrod3uHB zRPmsWRo&vG3gGh)#v>kJXu2KYQnFNysJS9c&M4@^{Qy^Yx23D6X!HR%rlaB?le>KD zXFJUsyu5VZs~=M@@j!%tnXKwH!zwgA$7mJ^?pM+&f#d|k-Qbdu>C#H}vdI%n(l_VV z)G17BQ7)7TWI~-?=Z(I#);QKKj__xqlbGI380+Kig!U9f>6HO(iE&_eJEysYDH?I2=ONgYX z+Q8ghf+{kt5e-ExF4~$J%HdB|1pJj}*l^Ik1jkP!olm|_OSsY}@ar8TXfw1#$OWab zz0@Ld3|Pv9kd`WDRWqtE^7U-gRDoQbJE^f@YEVd*2^LR?;SD7~GdCYGJl!_wt+&0* z?FE)JnM?R|x&ft2>$HA{+;6-#IRkn(!e3B9sXp}j7JYf`dsROqQFX}kJ+sbz_-99j z54Wzd7q61MBd7U&fC|X-b9=pAkZVb6RBqs#7QPwy7GL^M(Oy$qLw?`h5PdO#PKJ>d zZ!jO_(|(tk@X_FK^`t&v9-c- z>YL>#Nd0^Cf0I7_xHw2Fz6fPqTlJ3|ut9r^ z{Vvqu9ZgM=l=mi;Tsd*|eC(1q_jr-s>NDyeK6-2?vc7-j`SVV5|3tXaPjtNVQ=C@G z-fSazk3cW6A&^kWszUt+(-Q4>SymfdxrfKf?;{y_o$g#p<8m9-?Qdnb10yz+nJCyJ z=e(O6HM?+RfJ9BP^XH<->1IDrcH4e_cCbXs`?u=EF5gk^BT@COZcPh}D=U;JEEq}W zEc)*So281Eh~B^DwEHiEK$e?Th~-u!gTk89P;2Pmqmaukt6Dg{e{RrL&u5AB5EsVy z5vX?0(9YV+qY(Lnj-uTrCwY@-v41L&@^waG*!4H>6!cB4kY;BPii zqNsZ}O!?Et|F(MEmJ2w+#EFY8RE!)1+fc3sKX{24<$#t`fhNr!x-GUC%`~?qRMCMP zRFas?Pv(k=1T;P_td*t&5LIE|qTF9rg&J_e|K-~jd_g#K<$qhEw7VMn&X8`qow6Ja zy_M3v4-h3ZA1NKPR6-o>X$?{aGabdA=^QZ7>(ciA7DL-CWBN!d!X#Fp>UV~rs?~uH zmqt`R`P%7BGpUu1AI||q9hIihO*$JcT2IO@toc?hpT&Fo#*JBN4i^PX{t(CgbY-g_S=JuGhF4Uk zXT3#TPevBmk(I1(5GOuiq$hz=s z&_y3Tj&(BvR=fqy}BPV0TdhE#*H&)Ply?0xO^RT?9 zdCN1>;&=_k-b1SLZV}7{*!g^hE@XEAmLP~POKczhg9_FqVdz`1E?K-dHmh<+y#8+91Hds0 zQ`vsbx0Dls`%B^QS;&;3 zd8MD0u`baHK&`%2dsb%ENl7+_`W{CoCBDv%yT`c5_A1htWY`SH$UJD1ZPGRO+}Tug zEEr{{3bWdkh0q&~_mEXd?8eaL1%Tg4?+xiefewF;(I20gepk9aQ46$>A;b;J1?E0c z%bC$gXEWsa;p|wW2eGKhoeqUNHU|_>tQybkm2ha(WcX@DS%>l#N-FD|1%~96wHyxl zl$f2!ynYCaS~|Q?jlt%XPqK6)ijoh$bfFKh^n?wd^?{p4M7@DR*$S{a0U#V^B8YH% zt*t0iy)M_UKr8}KIJy7xa7xdKq1>N!zZ3I z>sU7*Pu)&_`k0tnC5B8I55VO6K-X|=XHu7Z_0wWH8ON^TkR{abGzmI^`_hA8=rZuU zcfCAnxi^pETDT^OA*RM>B)b=uoO!^tEt*>p^3CQuxnGtw2B|av)nNcd$cKh87?$i$ zum_+*zf+EUPCe8-uz3Q)h2)LPepP6n;2yTX9HN+6M%E)QHWXeK0`!~{NQ6r!28`9E zk>^D{l(xp`A8-O>1n=n|OfjKiNKRK-=|Uut?bK$gP}7}~;yt*dUZE}zxudxWkK3)` z@5{T_UZA)#M=Q!RFajC>RJ**T&#A0XS4o7m`J%oC2TmR3+IwT--XKt*Lkxybrz5E z9r5hJ>-(Z^vHXMnS->!ML)(YUn;w~_Eu#mvE2O{IO1eKnRZ35aY)RNA@sfn1b#blp zpSHM7eeBl2EXTRW_dksn&>EpaB_r8dW;3@vkt^^o9p%?KC;(Csw|Pg0OFO+{hq1)e zCY!7?W_+NHU;p+SirYEbc2Pa&!8I+N(~x&0&cWvefelov9YpEn;*XjWJgMx_rVQ?Jgz^pv;f8P61FEmwaHbV!oKI&5xahXah*7u1lZCB6w9b-s z1EvY7XW3r4)_`KFOV4l@p5r_1D{qu~4QT=$1@Xx;nHO7Oi5lIByAFF{YWmYN`Fb>> z#t53MeYv24RGonmv^P89A)W0pzH+%fD5k>3=sWYual zl%)X_5%tW;l943*UF9R1LnF3%qf_*gRh8y7#j}V5bNaoZ?u6;WybQV_(n5iJ?3D;} z=Cn*Ac+y6B48spJ-YC7_j%>Fo88XdOi&4k>HUPy%T#D|S`uZl9Xr%q?q$}JmbTj1V z=&UEoGaMUVK1KoDa#jqfib{;$@M}u?UqH-qz8<5wO4I6L{3p`Nrd>JXK^c*iX79O_ zm}eWNlgqh<Bo$N z9c9(f(LKU+hP)lslF?H_Ms}U9c{wcLGyF2+vJ2Mk7u${z=U(`K5XJx9L;wGWkbigc z-TH)CvJR3|9+Fqy43kKMUjhW}_CNAdH}TVScS^EEZ{JTD&!)I2 z!?W#9o>H!K?|Lbxq3kl4Svby}c%{Vk1uoQgF29pSP4?fE&Qy8b1D6_Qw$cs& z&q~yjG4l+vYP5=wX40Dn65PuXQ`<#;^WK{MXCQHw)Qu8d{jOW^`6U#S2U&S>`LsY* z+Em)WIuZj!+_KJ2Zp&F<`Z~46YXrV;1h3PiF2P!nv*=J2PmF1&e^FuiKGE65{s=`S zPh3b0>J~On-ae39afpP*yU+~OxY`B{{XwE7#w{V$}6Pb34#rad=z-1fptY~JHUN*j~BFNk?9b%R`lZq~(lv617 zI%lxQaeblU%$D{xaKnnvvz`!k83u9q;WfG=ccKwIjVr!6&sE{hP5|U*N`DC8g6B&7 zy~WBILbv`x>#Gv+70=Kg4DJYfB8@)09ysAx^auGcUwtF|z~E0O_^+iqXtR$fbbtJW zI%l+N@CEgYBXFUbE%V7_hyIG!kv@B$fYII@zovJ7*K|c1T>GMmPyDwAi*e;|+{tsh z>n|fzZynOs9}?%3p$ExMHh<6o7X2!3X+(izwC{Mg*`iGQeo$7ONDJjrZY?DZ4)v0` zADJ@12;Dy6D_^Ec{Pp#(y38|9A19>=UDTz~D@BcYt)}%YH2L zC*-5#L8$P+mky$SjCFUCO84aKbg$d&I=|QN+0XaK59lCV2Fjv6Bd`D*<$WP4w2p*H z2uE|dVQ@rbP64whW?1D)i-_6iNydmw+SuG8YI1@6x~pqv;p>Zz!_%pWkON2FKA6ic z_VOQOCiBL=M9+$=uwY8R{Q*)k!v}JNcoyUIU0d8C1Dss1OwXO#yXCdMG1zr*&ldxn zl(1I3^+@=cGP6jOR*@3K(~GDN7YBLF1j2d8w0M8M$G zY;aOvgiUr!2{o=!9)PyOS`}efn)SvEhq^-1I_8Kq_@Y@9A*XK4y_#w?qs3x{`GylU zKW(AeOBmBKGr@3!zJsk0!O2-OLlTjl4^8GQa}*uXjW6F!WLzNsRA;~~qBBDYPyBi^ zL66_B1tlU}rizb~41la_lp@DNr@&U{**AJH+4wVmX&}kr8jEeZEy=J8|}4AHqQy9Q!Kcl#-KE&(3V^#_Q!)d3w77{cz(>-&;&Wwk03X zYNEAYNbOjCgk#^mV5wCG=!e~_f&I;}Fj-U(zE|CNP-v+T9KhFln|;s7q*g-=>I~bv z%x*BY!pEygJ>FM%pbq}LZM?Nc>?jNc^N9S6&xhZ6P2Lg8In8;8xMz9)Y#xtCOx6#8 z_sZ@xjNeK`-BQ}BE0SW-yH8Nv6u*XvKY%H4eA6#?wy=o$OsCx{{N#EZrWf!3SIP6= z8TbYIMAQ0`aQW|gmd@CzEs=^P1J`b8OQ3JzpN2PXuQsPSXABXJP) zfg=6<`Wxz%o#)cFzGff59pkP!U;bP0Nacmsv> zk69gDM#28Ix`9Ij2T_rTfQ_c0u)qdsP%I!olOL)*xE@0t9D?tkcapymc!T}N6tD-K zg7}Z=*NhPY`S0NRv6u@TzCOVN5?BNe@ijO|BxGRK0xu$cYm(%OQ%@TCA580coDa+|PTqsoakuMV-q&ns|3VWyBJ|~a zR#&{w`JJ1;0DXTUSQ5)HQZkiDBdBeCvd7pD(P>JP1{A=WVKH#`=`$QAH6yJ18b=E? z^U6(d>H)fKFT2DXVfx4MT$0~8#}6+Vuk>t*p*TCOd=R8&ToLqT)>)rAw(a3pSzyOF z@Ca!IO2p3XR*&*Hi-sJtbaH~HROX&t73JC_H!D3ayO}grn#y=mj&^z@`n%;Z;M(@D zGwlAnvvd2~v{oT0XK3Q9suQsq;OC$r^(nLA)q^%A2dNKWq~gdlU^>$8h#&p`8hc=w*64I~7YIn4|$I^Z>; zU;#9kgT^8p1O$0}bB--?FOjv(?AA3#AQ$ZY6yzK(r2hhl$C%xOB)a60uA{LbDEwaa zq6?=Vb33U$gBZ+FRHFvm2E!GwNCH4*D~&M}WF_sgJ)v}^xVY{yFEVps-1QDbok2re z&bHxkhFEkJC#6W}Ug6hq)}!}f$%Ip;^Z=~cOG+r)Oh+wF^JHs6NHCaY8EnBojS+e= z9Vw(EkbtIGlwx}3*yJD=uy$6tM;a%DgRs=nX8yD))x#Dh4^QCi6=W;x*7&#i(a>qU zTLB7X%`OS67-2zn{r7O4Q;6G|Y{kf<2N8183k)L2r`vyTmH#^(@&4DN{&Ayjloooe zubG+;`A?Z0A#k1p8wDsd#|PEk)Bx=T2Ep{tbZ7Vutq6wnkJ%P4@~ZOHToD4;6iGl1 zoY^Ob1172P;}J?NRPL^u3p?`@3-8pKX0EK`awIw zT6uPc(s@!QsyeFo^$6fx1v_Q~6BP4j;(5q26IB5hikR*D^UnICVgvyl!ypB>a={hY z>lo_{O`SFSM9^E>s%lLP^XEUZAF%9|yOiDKMHpZn96HO7W&La(n>c#1McjY3C^A!{ z$i~;f(D+y>!V%DCwlTcr)GQkQ)Mm`arQwNitUIZn{5vc=y2%1duFaAc$0&GhV=3Og z%5Vy>L#e>{7tlV-0b8mXGXIFqFB-XoUvK*P5Nw!6v;UGmFu&&qxqrVIQR~{oiCK`! z!$zaJD_SkNK$6hb!uDF;zx^%OOXk>txf>c_=sTKM`AxrD`{U7L(dw#-`uP2kVJQJ; zr)}U%W(%LoF5MHWp$tvNb9xMn$K8Znx=KQ^BIJjjkN9kcqqbO2$?KS$<>*hrUzDcD z{H?RawJQ%X2A)$x4;524T}#C)6mLBdoel(YRD+xAhOimrwcO7E zUHZ4&WtW$nAp=!fD74=;(s_Tiz{ERF+F{%Oj*DQvq0IRn0qs2@+qdY9Hw;$64?k;a z44->Wvi>>m#L<(O7J-Uj_kvY|M-(hmcQ;|8MvaUvJ4{gyVflAX5)%gyI*i7%fL5ygd zM)?Qn<4@FYNFUDaU%!IZV8P~+;!~jsy)e`vpx)RZOnGA64srMWVQ#?#9C$n#0NWy~ zH*+}KlBGTHWm_W8A}vTB9SQk-W{4xk<&nlDgR*3zDWugNAku8viYHzcjl&2l zo*#hb4nXLhJ)%4)zhP4vD_j>h00h0!OU7-U-IjrJ0>`sa}AUpJDl> zYU3hmS(OJ6$jaYw-ywV1w+Cs~KU^c{|3rD@C*5Zv`>O-xDY#V%SID4TfVLgd>D%Ly zwYzM+2)b!5@kEgVul$GDw=5={geSUN=9&@_Pf;chphcdLC9M83K5sCl`q{I_u$b}h z;jwf~OJ08ubmSvG@hOI>ix3EElax!xsRPI7zY$h=K2`Y@)#;*{5;7`wpVaaOM&L9S zd|>~R%>FmD{BLv<_$U4doRhPI{<|b&hQ2L7eJ$>cus~2}Fv5&VP;4N*A|GOV-vv1e z#J^3l)W`j;zd{rz;(v06e_~-f>@9gb#D9X>ODzF41kykK@{=4zdw$`cy^w)}y-5E? zO;{k3IxlLwL<49N6q4LmsH|-?k^A#Se31kSOn$+~PIZQY0w#$fLAPtP6HxwVOnxQ_ zziPiImpV{^EbX8az}Zb`41j~P#tLrW>~>-|u~(|%1lgpRu84^)xPEJWnh6W6M5w49 zx=14ocQwj#Wv;kw#oi|wy8GY%u>7_vl{|TJR@JALy>t+4X^ZN>nuANpEfb!ZenLR1F)b^bcrWWNW6GpF*Q8~ zJ(IFRIRl4@a9{%Hw6=&dQy!aF2k%T{Ribg-HcIL~O)+|tt$)PBW!Kh!z_5BL53y3vNLi=%XLTAxtb z`OOxVw110d1(0hWb7yt-%?J()ic2lB-7o(lTUmBbd!rdy8qx9<)}M+B*VX&s=4qiY zg+q5DTS8T)$yjTZVC*|W^D`HRz$w=NTgTg`jeu#H{r4H~m_zMsvo90jFCo_;neMJ` z4S}M4e7by$8wXrvJV@C7()}E;9a8qZr8?K-F1EkV27p%$4o}^xz{owy1Ftgj10rbLqY^*iLtmo5bRiu^k>Io=d;H`{p2`jq(8#W8-N z=?ERH1q1AV%ABv~n{_dea)2>3-Yp_w;?e_aC!$an$iw|7Tc*XOMy7Su4%G3qOgU2} zhE@IbhnHMmkP>Wg#*h%fUEdJ$HE(kxj#){8UN$diPqu53A9(J(ThZIotcc^chEI(? zdTY)?!kKer4m!<+Mx5qzNt$V8x?Ih)xQR)51b}2)SxE#(R<*LIi0eH_-dvM9r^X8E zEQI9N}6b{w7;*@i7xfrj(~;i zSTn zym)lXrkoCN5?~LFy3pr#AQ~auVS4amney{Fm zDh>N(xEYJV+mQaHy%n>P$q5lgh2c#sp~T>Ub^zOv_BT`8q6T7xgAK-0FSyR|Lg2;B zMr@7WrW|TG4`n&aY0fUwHl_c_{R1fLDpyfwF%lIh?R=un7N1E0;K}oe&m2N^)6Zj7 z3mr&#q*!srPR3pN-wUe<^?Rb^i}r- z`{N(m(VtYrt95m&qp7iCH`KvP(zdjRSa23SMa=I+d=>E?W{a26ub>V7ApIMA^CO4?Y#I0N5*bpgZ<)8n-uesy@>|-^Htsy>nh-;) z%Vkrd7^H_(A~QI|;dsL7(LBI>Bm7KHiC$9({>B&<+LSaFnk6DKc|4UN%E|@Vh~$`qw_z8h{Pkde9A{jgC&YMH=pFC*pndBW zS|F-_m$WBQ`iA&3s3U4TEcV(>I3K0XKY?*ylW0ucj58r7=ceUkoE6Z7I2T1KKz%iI zKQ!Rt64Xxn1-$oO5sZ!DiB{hSn~f^C3w-76V-5@_j{0RH@s89GBfMYhjVz4ucg${I zyr-TfQwxlzqGi8lF@Zbn=q1@O)~RVt$sDrY3X*FnJMxBiD~Kw;9T`@359^k~kpgqT z_m=FZx>897iBKZq;dcPRxrahEJihLDN>{YOXnEJEGBkl41b0cKstdjRy!;QyhQ(kZ zHAmYXxSS?Py6ki=DSOU?VS3m~ zdvxom<&BlCc@K-U?ky3r3Z}Ol89lrO7gm-Mc!aH!LWhdJ=Ztty&ANH+(j}=^CZ0}J z9%|4d9_~a1R-Ay`9DDqrj5SDx?QOiQvG4afMm1QYLc><)emM&P<3AfTxz}9M4f@V) zf3!_x-qnl^#a1-#uSCGL_Z|VCo@6w`BwGd}D!#92ZThbueHTmj=l!XX;LPOHlu$S; zcMU_t8|p0GI9~poE9xfgJS;1FT_fV5G<51+wwHTW)n5RB)Wdo}d>^~ipDug-NXU7u z&PZI|@G0AHz(O4N4(M2?1&$5Xf3ya+k$frXopx5VE~XB2BTN>|2MaHycv*#$+fobL z+Xft%xryppfY0ND?9_u!tbv3Ok#%$@?AF6wRg#~WPsN98=Ij^@M(C`WF9~AS#)gM# z7MZpgC@ov?Rm7t!8?G3DYc=%+J?q>8Byj-b6aCHL@1dA)%J#$kIE~Ni&pSC58s(-M zmKmyaX@AYqEMcXTEk*7C|D>b;&FcO)9sMUIM0y`#!2QZWFaGbb&_C%QkZzS16L@(9 z4F}*aV?3hrQYpP`YYc&-!H%zthP5V(h+AkSrl~suW4_Bp`+{xdOk%apKHiBEBdkaT3Vl z>~YA|jMMhjaDZh*h^BFaPWPOHN!f-+%z#c)7r)U=uOH>TY@%KXYwkE9spSrQO9-b5 z4jOgT-6C1+Wfs-hnVb$T?cdudejvZ3l)a_FZRzb=sOJk-{G!dqM;NQsWwYPo(U(fz zoXViVhonH-o@AIbmF~kG|ww5-TAHh#!q|qUXnUe_C*WN-0Hw%z> zLItNMELiJSli^V`5Wc<{y65ey2SF=A*dzNz)1_{cbwOyWhJ}wA8-hzXq%?Usi;~Bz z`2*5rGcC3C=MYhfgSqvHH-aq3NK(o^Ts<4Y=juDD!pMB^EY9pcK`Wo^NPLV9V=SAv zzQYxL=VO&SLTYA8NCy7w^A8XCV-UawZVL6OYUq;VF#S+g(+kw?rgt3RD3>S0M?@~E zp#k16OG@AfcI9e5RH>DZo&9g06mjnx#0Q4I1euGhmfWH&ABh)k4K2&LJy~(BX@;!tN5Q|AJ1>Nt-i5C<{oeltm#XF`# zK*;s5>20CwQ*{U{_W-?2!_D%y%WtlKt0r6gnP0oPNmkviQE#pfpHNq8ZjMjJX|MZv$rh*FFAklDj5 zag3rP5Ems7tBunZ=j?j!QTYNUj!R%re3DgiKP0SR0z`k}r4h`#J&pN_s|qcKGTr$a z79lwdHz-p1@x?G1o&FSfUYbnkM>+T^-WLpGJ!GjjY3+a%)QRVi5}sA(hb_ndj5xpG zZhb-}2od8F*c_*52_E(k)atGP>@mYIUy-K7XsI6W*T|RGM3tQxMtZXE%JfmGO1>XD zRwP__9y+(%V5Yx<|Km~r^_&0osM~)wcshLqdoW1**1A?L{Z>6+VhwqEJxU4NraDPG<+{4o)@PgRB|XX0zkcle>-J#wBv^sdJy|~2 ze!pG;zn*vQ4(*Tu(NUJR?C$leKG_BJ#}Ob*OJx8~aVBY9%)irxjLPoMKPsT%m<8j-ksBSSPMKmbajmdj=N7}Z5`wTd0S;}636sl%M;E1 zJ&ISVuWO8TnTZFk*U3}qpnL`BeAM8%`|hYt$Ao;>2=i<^lp_sj`%X;zHLuwq{oxId z2jBH+dNr9=UtfGl@3@jrpiKW>^8KQD_Id%lkx3J%F%1@_FYAh$Sbl1g4%lwdI;jE1t!8g;QYpp4oiffgwlMv_GaSX@rQme|( zLYzA4Y5@!gl#B2nl@C?P2#j$sChh@Zw?At0V%k}O7qM~8J=MLYk|5+3~dQ^smq zp$z?`<=ceP{rxTt`(_U$i7t;YT5Ks6i=h#_KsbW>A9XGkAM6rn_0u0R(|UP;s~ua) zpIQ1P-s(jyje`mjhwDD4iXM`?u++)c(JQ9qB~=7h*=CEz!L8+DlD* zudj*P>MP~%aO5#Z?GQZurme03V3?of`0=z=iFN zXS&hkJ9zV8C}_fCohUWn5MN=A)W_gyd1l^4gZu-DhF6F)>SYbVWmMGekj_p~01x)s zEIXZf^CL8D-O-KuH%tXOEPaNpA|*pJJ{|Ma;>Hi}T697AZVam*&EXL00N$~Ui~2YU z^(TJt>n&X<+kW1R@Afiu*cefn3-QT1e*TdU5WDB^EYqNZr~Wd>%w*Ca5nPuD`hsM9 z-gnn10h_!2<{84_q>j@uTnug3Qx1Eh#C3U9J~JPL?dAM@h3S&rYCntJ$9sOa9W3gk zMtMo6{Oq9}zii@Q9H7Zh18DG>SI>;gYZ}Ly>BfI|Z?30D{Y9ER%B{@Pno8>gN~wXd zZt2aBMsU|gj|rJb&y%`i-ZteMWYT1I$w*k2fUrgN%YHA5sz5`SPq_FV=q^-yUoxR- z5@6ukuDcDzI>C7qbr+^srfRuhL*kvy&kuQX0R0Ay@BTq9?+41pIy%(0Wb zjx)uDb}^4mj)H~4LY^=@y;KNjfBKiOx2NYgJsaKW9Ou<^PQl(GCi$SxI*1Z>L_4bk zKcuCyxJTPF&EecHdk1neu0-0q4LY4r1P4XSEZi|NR)zsQR6s0mY*FaT_Yo{w*o;x^ zD;89MlQ8A+oNkhIh&uSFQ+0tnpu3fG+lE^HD8`(BUGLXYeMW^T#AOtX>oOAes$D|3 zDv?Q(dyRCGHEKp1QdTN$LyTfmg<6y0 zLdQB2dMD&8K0K}aYN1J+$_Bdh6c{g zoxC>={sR6%MhMIROt>%XTSM^R$my8EzdHL(AQWB5=@)rRSK2%dNz&)E)$0E05=gsf z8cI%p;Va!Q@~)ne3sc*rg;-@g=k}QTNnhv}S^zj8#(g&r+$n^9rH1;e@$e=Y3K$b! zeZ_|Qn*y-7X&CK-KD>8&X@`0&4u{u+e)WR-n{Pb-nj}!TI_xc8P^c_-&Kepju(K z-47tC1F&jFkTt3BSzd<0si}yl$y7W3<$96)Ow4?vWEc{+=PC5&e(5j1mPQQBGB(xo zI8!we^d>hf7qY#n!NYwZ_&{5UD{3k~%^sic(F-oZwCyT-eZdvKzh_s_EGos}$4rdu znpvw)QZI|l*3-k6Bx5w0UoAAOlVQJ$@dju&M`+civp^D3j_ns&3qVOD>GG?hQit(n41xjH-4|Av8Qs zWn;xshs!$3&)YbSRQ0LIjJNU4cXEDqc~7G-vXItd85VA@czuAjx>EH%o8A? zW+*p(oYS9)V?2a-3MSz6D`4E(>OQ^mAJnUp;e=3Q9UEv%htkoTi%qw z-8x=JVL)fWuWk-Q|FclUIP*k7ekX@=?v!U$fz_$^Wrd@BOr3g!!;Z%;ga6FkyMQx= z#hmzRhChBAI>xH6Ec}lR~wF2j><(h)kG4OrJB@v*GV;asU zYZ3NF<)p}}$mh}~Wa7{U)XXA4_VQM6B-wZdO6pye@5hOqvRh23w*nSs&GOK(WA9QnY2s=&LsQ=n}4kTlropbTbYUNxa zrpTGW{vc0ogr7%aa5>k*YOmc|Z1HrM+IX)H;M6KT`KuZvewFQSZ-PvGF_8n!?`pL5 z&8a3w6-S5b@aHJy-NwiOCOEa~I~MT!&ryx;kOBHhobI zVNp%`{EFPk8ADjDXM$DB42eoo3o%?Sko|tsWReV`tsKz_;`@ttv0$;1E!+&z`X|sK z4Lx)@h);wIkqk$J(htBqeyuJHfstR?&x|>@dwr5UNO&A=8nKH8wSrG(RY5 zyJKc*fNiygtbi4?1pyAfBhbORf(2ESm$?`gf%-SiaQ zHXA)TrM+^xGjcZTMq}dq_!w*uw6Zryhb&Jfk`N~~%z*f!2MOq+Bxmv9yc83W<2F9P ztZ9$gH&dPp1wz29Fl5EuuDiXT!N65*w!d=;%}+J*`XBO>RlVkyt6cix@JF+=k&qo%d>YDaWiv8p$dQ~ZULTBW7^XLvHAvA9llfu z38iF(o7_qIaPF4JN}YhzW$=zD?ez0h8YPY)~ib?*3YJ<7d)!kR?^{M*6jdEWOdQohc{x|{O8 zm!x+Ni++H~zbKHl6{S_-3qBkbpp$}G&(NE*iXVjSufRfaCa%miF+7DAsH*8w*FA$*EMM?8wlT(`0&9Y za*l=17pQWRJy1SBjy$)d@}81JTg{59cwZG^G4s%!!kxd55kl4X6UT2-#`9iba<}V| za{(NRy^huo&>weQBqzcmOMk2*$hewWQ4jzGMXnR9^f)TSmR5f@weTX(Z6~(CVjF^` zn2K&&umacMuJxtit;u)V7N9ob`hjZMZSy&&t{aL}WF>@)`Zu*mZw!(mk8pe(=^p?f z0ieBH^P8ulA)#iT=*yq;w@r8ua_8jqSx`!4j~z4?Jl4PCv0Lg?=YCzN)M{E8~rS z1Lxqbd|v28$e+!5(xu#_IC3c!unz$&`tz^!&Y<;~gJXhzrZ(2Zst95;JUDjMr6#C8oGmpdR1koGe5e8-W?ms?E@| zEygpr5rP>!Pj!&}65WM$HNkYkYG_ETpIR&GCH%x%gx_zwW8hi9;(q5qwO*KIX+p!ai1MKIVUC zORNhAZz1Dec2Xx}Z1?NF(Cz?yJr~*oss-Br8uQ8oPxJ&&;I8lI&bJtE6Wq;bOpvG8 z>;le5;4NoMrGC#TtL&@#r@c83(O(duP3am)n$m?KlWL|_Oj|=`ozRMC=o7SYXu94% zTRL?r_S?~wFCa6Z=~E<0Fg%-<5;hqc@?oYcZ@f4w8uIpggKGeb2l&xq7;o{o)}*lM1;s(C6!Uqp zES^#nK->$PX}Bm26eSGQJ=_i14@?hF_bpTAvxikNcWLZMWEz)KMPu;Er3!T@NmNM2 zx^9o5j)PXL{oL!-fT9am?-gs=mLJ^z7+NR&&SEz}c?m##^Q#(0a{c=`7Q{|vl6WgW ztV)HTue9QhWCAaB{0s`y=9;+mD))-4&&0A7VH5_rJb91o4?Xj5tUUdB6c_ksM^r2~50T?Cp(Skc^Cfpum8w+zsg=}P_a%l`$@kwqDL#<4y1N@^h@TGcBsT^8RAxSjqJAN7 zxgS=v*dwuCUl6uBaP20KTJOK+qcQ8iLSb=JjrhK@b$eI9ftvY&N+oZfMbV+^Grh3y zzv99fN8IHBg1NCb6T3i~2ne}H|8saU!-bp$yu*H}2*OqM7;FnJp(M_#NP$e0X!@i0 zpsSOk69jNYXp4VvPY~p$3$kQ=J0#*bc&9Zsz*h;-@%-?>*m){e(Bu0c4XiMW2vF<9+&H1VU zn!bk`X5T0s1x&BDUajB)TNvQ*2EUtVom#D9yoH11;T+5D9WUVBOxxpVfW%}O;` z8)GD11d8uDiqG+lsqwv;m6*6;rKwPS@s!3Cx)a#Ep}xt(M)5ajnO0~44(xi(|5_QB zG|Dsi7_ln${ds|rqMAhs6n$L1xO+ZXGjTcPP~0OM2pwr)M9 zT!agBk%mz-E(tqG@uu8>Cynq%W%-GaM9*MBI{tWO+&oh1m)5`A2j?C!xG0{x{;tZ` z3hGBHbOHq+OZ5fwvNfH3E9Rg;BLaz+q*8kl{K@vfm|WR&SNaldxP$r6hlHfS$EMnP zl67!1NAxpCgf%qOX^@6% zqZoN)3K>5C5#62y1T4<7fuW{<7Ea|6nv;XnB>lP3 zf6~Rt@bH#dz7Tp$mU-%OzS`s3BejYNWEkl01_v_EC*P55Xw6D?fUMf1Utup!2%#W! z<{kC;l7-D|8`i3T8=4(tMo)k4mf-Y=o~IR_Qot#C7)(J1l%#(mO(55z2PSdL@CtXZ zFD*aKYkiz@0f|?SA&_AD5W}W5d@1+m=)^3Id(qA^yWliGk0NC^j|CDi^nZBZ^g5Igw7F0|grng_A{Y_* zVZ(#U@^z)q;~7i?%(FxrTc&OpcWXKO42;OX`LJ@n6o$DsAVTj=42{yWJYS~I+D^8+ zSonQ>z94cVH*^>sL>N##ocAs9*Kvx zkK9(z9&3-d+BpXG%?`qKsbGheNO%KOgc)AT<>J0APaOb3Vv0R2{=wI5pmsHmX36i1 zm~6s>NpJy@>J<1QXCts7%Z;(6q?XBvT5{gOa|g3<(|p6i5;^BkouuKyxtrdJZ*Mko ztc_UdSr?Dpc;s;u7=^auv%pWXcK39$Hd%-~{E;TLb)?F4+6||)lbW@&c0_;}TUgLm zgFW(NEYav?%RRj51+V6|JGf@m>vx`2`IiY^l2xrtD#BjzTWHL_(>vVux_c z+OZ1~t776?*#7`+|69iYcY)_nl>k@b{!8mp_zySozXcALGW!S&4)~uut}gEXU~F$Y zFZCG`u8>fEVEx#Hg7s~!hvI*riT)5mi=!C&Q%;!1S6aIs;FG#9T1J>DR47Mj+7+qD z3qTjT=h_rCg)~2z^IB>(r8HG_sU-fqbfs-&;q}rE^Y;+2*zI(_Y^`wL_nYl_-~gFM z1i)k;`0}Y-Br;7##P9+{66!9D9u~6>k+G zrTZ8jJ||KGC$KNnUQg88nzFRiZVx2EYaDw;+?!Cf-2qXcHA6KHR**66^yoDsxM-bo zzzw(L*K1=+-Ft%>cNo_lOJQ5?TGZRoZXBm~GLYM0Zk)X_ZV$1Z?!#S^58gc2K9GFA zi1&>;(sM0npKOTkqtHjMwWziuTbt_xF=DTGrKki0T`<=N3NbXVfvDvk`oFJFvhjWJ z?DqvORRFi#pL~V839_XpOi!<9Gd`MqW|_00A#ZoIFFYl?ZdtX+JtO?xHM_(7^je0seC___HSl<1;1Fb_v_q{P(MBC`nWd z{cG932&#-oV%5?ngE*LkV+8{AmnLIp6c>`@{_gtbw`qR)DysQ1&ybC@OI;zataP*m zy}&SEDD<`{EE0jWoNDit7*v;dHxIkWrI#;TAZB7{>CI0B zaK0SliPZ#pDkD^|j`~BFy&1Pl9^3Z&KGO4G}7Mi3VNi;pC zSraR#x8U~;e-cHO-oH1ci+`BX0n|rmW2GxV@LTMarJ2V01O91ykKDbD`vIR@1blR=p%E zRwek5+gXif+yMzK_d|uw?$US&+A2hCm^6#e5v6=rwqE+&YtKyy;I=ZXGm%LJxlNxs zLG6iuJB;nCHw1H%MFGe7O2A7F8L4Bf?7pmLy7&hu}T%10wOM@RTAbR zQx}O;95gTY69>r>uuG-nd~srQ+e}LwSnnc|am-o2MCd3IB1F@aU#&m2-w(A^A7$LD zvVw)+TJuy7qT`aLL6&CUdPg7@iRgE2PNfOoAM8v&E&&rV2%w|+1~kjUTI^BU(m1M1 z`7v|pM!mMYf@b-E?nuKfT*$Djvi*y>G4}n{Zdhu}Rd!t(UQCQ=P``3Rlq{4UcYBci z3m&R@Qxe<1P>ya3<2YWLECf+x{r5o$Z_B??zTQ224|vMEBB{vFSI+4}qrAOFc^>G) z)Y~?_aS`3GUiO7e2mpks!M>g#H4e6xE}~V3&LSTer(727GC@@$minf4;5|uKoo^7o zkc0zGiN&nmEf?7fo_cv}+6{3SGTpgmUW&|~H0EQ)$(U)Isi%G_JTOMMv z$;gR^2Pq!3W+88z2!iM+A2tUwTMT!9shBWoqV6AgNy^6fYd~aauR8jczgYqosE^Ve zI-KLvZcF8KvA{t#xTc{=5Ge&260{wH@Tak%WXAWJ=4V(y9xHZI_@N+j8YyJ~2^Y+s z)K5(1(c=KU#xLY^x%vGWa8x3iU3N&N1UMvm&ZI%kz|H7~U{$a!{3^2j3Y&;PDwKq# zCbw7C>2rxTJb+CI~VTXT!<~yq!^H6&?gToQW5U*(tfj zR~8Cc%7%uwc%CLt7ZctMIw+;cVWq39yv&!0dw|3_4W=51&y+ z1Tn|TVw0cvYysU%eOb&kWM}9Dj9_=t5{JuTNEaXa!)#H%+ny)&@va3m5A00YEgYWe zUMN<8JRm-H^4+?OIj}B#RB?!dNjIov2rQ5V6@=YhOcmLpJe9Iug-JwONjREX7Igvn zSVO}WeiKf#R3bDx;hMU_FnClcNb2z49E?zQONyr`>_-hF5(gvKx1c3EhG2~w{%q@q{c1FaK8J}XzJqE@MstW*_FqAnMSB3>>x%qmq8$tqPF{Hwwm+MqH= z2}oD26UZ#+mog;TxNn1tG|egrm+7wxpQ4R@*<(n?V`xsx>z4}(3A0cQ8a6aXt|~V_ zNXC>PJgG2@xYd|X^AIT72g?j)JV9&zqdX_O>VSoy#sn`mUoVaBgPa@mMtD?C%{2%P z1s-1C(+87KfZxz>;rpXFSKBqR_ z$C8p{w5uuIN4Xg#sJUboxz3 z>{g-LQYFNqVokD-lVE0eIi@!I?$iqrL0v5jL#P&ZRp{i<6n-!$7+6+pMB=!p3TUaJ zUB$7W;Ah^WEO!j)Uwen4+1SY3!q3ZT44m1nG)Wk40R6s1w;IP8$7#@VUXBj6Ql2yf z?MaS{#It8B3a<(KRo*JAJ@f4zK;2Dmt2`vbvnq&F4y_F7uj%|lidLYE&OOAg?9c#J zx{rYn8z$1XCjDBK>D-R)kP+VYFMwKQQ-HL8;x?SRo9bHG0a~%PU23z+5E!GJBYn%9 zk6zvZW4g)M!CTN15${iG&CmF9zO`}4+SBJ+*0w!v)oy`5;%L;0wHwFfbECzSNO-Fl z`b+o+?a^T0xP{6l1HOJXfpQ&43hLWPK_MB^Wb|u`{Q}`}zEn7e?zJIIH=t-=@Oj-X zA_VQOmfAIfx7rBgERrPu2!u#~Nh35~Yxew{ zZH|)W3uZfh8kqyBars1k37~F3^Ical)hccN6fexHxadK70r{8z!qz5UP5aA&BB`v1 zgkk&SnUIvDJjN_oZU4?D^6fp_Vn5i~M3^jHld>lQGqQoZR8Lc{wpGaVmmGqU7ZcUf z<5rohIo-URxKEW5M5aiXWbus+a*B&wZlpEdUPeuBB}Z3ZL46#)*c%ltg{=onB${IBG1l&<%~6y z2=qUVG~gLSk5ssK1wd-pZ57bC`($qNear{Wb8{W%gq5*OmGryG#pL%9I+PjNAs$9< zLgq>KxTVr?Q|*RSTJ`>*Ws}!my;QBVSZ@xkCi}$?WJ{FDx#QM2_FLne%3+Q&oe~|S zIl;o<{e;^#p8W?V3?sp_u!@rJfj4JKy_Nf^lmw|ob+8obeSmM0490#Y8WQoe2PSPv zegvE^;7s3glp5;hw68S9?XQ6)#MZE{|O9kx{R?K-7$k{ki5*n}u%MM{#~iH`S9 za-J5l+c-anW=)5ZEG;>}=31XlYV~GLQqEe>dr}fi74TRkr z0T%CYds3f`6Tshh*-GoGV-x*^Yg#wxRsUP1NH4qMezM&+Q*Zh58fs!UH%5$uG0}{w z*q_e8l(wz9t|OPg2=X_<&bzqw`JGj8L!}fi#&TeaTsbc-dh_v}DN@E!)uLgyQa$T@ zzv#Mv35^HL3L;VYQ$9^WVEIk>sm-{u$BJLeB~o=`C7^ZuU|wy_`Q8eKq9={9oFzG# zrXUTcPoP^3Hx3dNYQZrEks@3or+)M5X^aLL))A67sTqVq<>1PMzSCHwKHM<~ZL|H< z1tZY)VqWVv8ndLMXS48Ai|TrS3ru=&pnsx#Wi@@h>WM{NMeP9T)?NQ+avmHG6OT}z zNx62;J>a@P`AbpbxlKzhyTPb}s^xH%h?5Orkf3SudR}sZBl;pyh$#4dQ>hKs!?AVE zTguM`XQ!`ZQb*Qacx&Z-vV!BW!GYw%#`f z=R(%Q1mGXx(?p*8^@6Qx|90?nx*uxSu0HCNAWN$m$mGo?HR+ZxHeKjFbi|YX(BxA$ z3)sP9NOuEd=dZof_sZ)8VJ+>$4Q4t>Q9a>6nId!3+%4cU&3T2K7mDTpbhRSg1)HRtT8vATCF`! z{ZlqHW-s0^Qu;8Lej`AH;KO%wlue=!TR4A99-eUCd|+N{%D{KD;&06MIN$ym>vxQG$mM7R?1g5;kGtfoSvY@_?0JVlTrE zJZ0-*`OU&wh=+>e`#ZmRC^JMB&%D3D1%Aa^d@~KJtb!W72rcVMEWkcyN^SyeLc?sr zr`aPO_IK5STnBKq5^UZAx$g* z%m4mNeH+&A{RN5$eXYZwjNf%w$Hr?tT7qg%Oc0N(?&!J&3Oqd##sjq{mjTc%ePAo7 zv74}{X%7-mEp__;!iX--oQY&CoR0{6%eok4{(!Qikt)Ea2wzdu5fJ)k0}?1%Txywu zk9!&R)P#pG95G~r*-H%yqS*~dcQ~0(+v2z3tkY=#{G=?+cxpa~nZvIYuzwA>RJ3FF zsX2gH_FpOD-2Kq(D2lknuSi1u=K?*5_p&7ojsMpFd+pz13Do>c>4Kf$9i;V@qR?AP zX_eg;rh6YTy=^}?`1@`-fcK}`nhI%Y<%xla55pm`@&XNmDP1+r_bTXHkg%Pf@3BWS z+(s5-cP-(7>BV^MufTp?N=afCVuP1I4w;Bs>OQiYor28XNv8n#aB)kg&~!(XMa%p| zk2EP(xR9Y+{Lk#ubaA79!WdF4k;QM+u^gE)>S2qdL%EdBU$>|r0Mr!TTNQ?YwuPcA zApK1*7m~jtg$j{bZ}3S5kxk?-+m6FT4@`o{r`+V9X7HkjHvx3rVz!4t3u z1C=;xd~K7L_^AUqHeHNrf5PY{MDxckgBHN4SeKGaxPknGr!Iotpg!~k5NR8q{zwGi zu?K@`uER#e=Mv@k0cL-Oy6hpqoj4Dq3$f}0*G=F%^wq<3n(HrTE?tfikmAU*@o?&T;0e&Q(rU`@A7th1h=ULab# zl#K%=_MBZU15B+D*yl{E5;s~%d#cj@+;bR8`*`XjFC^m1fP`Ja{yR5|LboAduM{Ff zTW@i97m_T5=SMk%YLFl0UA>W95NnuEEx#YiTfK8o(Vqc&>%y^$4x*?)cPv#U z`Kt9G^hLET0K8RMNxI7_>~AQtXW9>Bth4A#Whf~-$;ip;;{zq z5{?EKL%CdiJ$$68Pg}j}=Q6_8?8s|`V_e*6A^c(kxE%&u@OG^5cJpGku_QBbBz8q{ z|E_i72@SA6_qW|Yc(8qUM)BE&-?xOlI^BRoy|Y~aKsKyFuIK#NGsNc~)gr)F$3X(L z(>GmlBS#r>>FrS5OwjNGA8--+l85~ZOM2aa)^paRxGaOR>|r&MY=hVbEDCAR(eNM! z>^UZTkr`9A$x>)WmC+a+dY=ALykK{_0$ zF9$3Dw=9!>?6<71bh06G_7Lk#m=uQi$GNa(27L6rAUvS3ch%3x6T%Qzjgx+K4i^pXqt320(XciBfk|K}*iNGoVF`ORHxzec0Y*}u)uEg>;a?xFW zr$hlpQ4Cn$3Qm;ovbYk5smz>HBbD&QFSPcN33)t3cZ4uIB4eX4s#BRTk3+QhAE)Bn z7jj@}65dm&9OA%Cd}{1(ckC%kn4FNzyXFo9M_votp222B5Dl~;uJ*p{Mo%Ds zUl8iwJ%3WkojH5f=mIKw2nu??IdzBydnz;o&dEXaI(z0QY0|7nzXO0t?1Lh9MUOQ#WpM%> za5V&^YM*#z@Lem${T;1(U=v#IxjuSzh|DS>Hos6Ywp*l*8?9;k*z~aER~!7tScu0V zK_($)p9M$s=b&xqpU9uYK?{;$ir5qJ;X4__4AlD%;RBOkPlCXSjy&}koI72BrH=?- z4|UE)(SaV(uOBAwRRb&vqu>P4{Jb+73B@t&F}FW|_lR1<$15wIYFpSRlH%s&)M(oQrJ#Fy^3Cg)cWQ;mt^i~mW^SL(f~ahNMKr3!-VYlbCn!^r{auvze_ zHKPjA7b}R^-*D-&_s&U|E#~I|gben?fjQBGXGPUwu`4j!D{1e*o#ciLuEPN@=7MTT ztY}f5zV4F4^H&1a-h$?jmRu zt?uEq%C1{6Z_2hApp?nPv0eOS5S>KnCs$u*b&It1n10rJo%)Wfk6F97k!& z-h0x+kF8{aj{k{#SxEa5NFBtjBm1<&WBww@fjgivQ@TWTFN- zRVl0Zdm>aWmJixJB~3ID^~nKOEfIQ{(YW)71bT!SbT=CGE zIcP$+B=VV*`yVQL=H6;H9@eBRhm^fY@^KNe!cY}*7P%;eY@7m~r0~-oDLzc4yF5_7 z3?G&fOTHwd9UMGYc#a{Q1m#66>)#MsN$2`yrS9Zq@#ec5`w+=E0bXH88v#ZdV(OMM zc$rAB^hKNefKYRM9I)HJC^7>QB2I8A620iHUW~o$h#;+DKz1D1`qcDlv!|@uVM$;8 zHoD)axArc54|5<>jR%5QZcc;$Y#X`uo%-mfS%{`1h^7;u!N&p}?US_ed#Y!+U|2?Z zll3)Ws*B47DGFB^RkSg|Awb>yH8sX^oSh3`)k0g2*Iu8Tj=D7q`4>VK2n$T#Cj8#&(Ca`3Dwpn*&!Ayca**l`665x%c2vI`0Mb zQT41VXPf)a5Bre&OFm|ILMBYZ?M375O#j*~`Lv%h6LgoMqa*eEDerGimBW7Vw-47k z(aa{&i=Tg1Y>-g@3j_Xt0759B|4Q-haA?hd|0~5ioK%&D`Ip_eiEVZ>rRD*2N}YVoCD=YH}XVM7WNBnb=NmbG!^SInRcGIs!MZjiGKK+L?(l8zW9tr z#;`<&;Ecv*e%*$4vsFf1kN+_98~jV}FIR&mhKRNZ&A=HaFW+SK_4eCwMw`>g^b_Qd z$B7Qm$K4Dg!tNQP=Isz6Lhu~`t{bg4-{3%C<$ zuw}r!-wouIUhW%zz$<4cg4U;ZpX~eA5vhi8_BXyq+K>xIP5moT1VNp9)KHAZXW)>W z#%JWv1+CBYJ|?XXc5hAp>0k}^J3K~@!JQU{UA=q6(2e?M_7FSBCpdCeg;uq}gi4y-u}? z$0MJXX81K07L6e+dF)cXG?jQ`!Txp&Q<^>Bno+>20`W7zFNad89_uhH*{Lc+%u>-& z=s?T@9XxraNI5P$Fu?5yov8IfnWjHJ0_FP&fH!LbJ7MPSZqHXXeXq{v3l1IPeUt`u zX1uhJYsY0b2W=Kcl-zPOR(`b2F#qtAL41%B-8DQC`s1r=h5bpKX0;~z_8Ui1{z;j3 zqtMaA37!z(X~n?j<~@pc#G5?2=1iM2+Zwd3>2bduF`h|$K(v|c0Oth!DKX65D#WFN zV}tF@L4QDqAhD`rVyh>a)}($kBMx?piG8!r3q41z$(_Q0-B+{%gkjPd)1&y)H+ax) zSa>kdUR{jdH6y*Dan9FyE(p1$E*jN37h!zWiJk^fC6Q9UeA+2}-!w_p_%gs2!ZLMG z8K#tNXeB;?CMCtAd2U38q$7|xij<`>JXs1g{FiYwjunITTOK(gwEbm3eA^<&7;l&e z+lXVLKiL5R8XGk=!2usreR3jozm5vu6NU;S#ia19WEcliz^&N3>3t;nD=WzbBL&MU zQ-K12t!LttVEv^eZ5rPe)v#g40ke2Qp!cNetbwUi!it_ z70gd{RYz%_LTg3xAkCqQ-Fx?U!=TtRfFr6Fo{j@3H#%E+E3;)_+6nS&E;}SpZZ;$Q z?>nUPAO_(O7fh)q^YVfw02e+&-3>RY1*!Fnl7MI}93p;0;`$F3w*!G|UWzEO2ow*% zzrgpw6)q+o*vII=g*D^6(u!HA;Y~~J(i^PCK+GDs4@7XKj#KqdtBW`Ay$kD>z=H6_ z2*&KE?j?jwH3Jb+kr`$)VX*8Y2iPGo;rlN{$jmoI3PG$v=#_I|#UqJ5Zu7KK3On}A z+fPg9Zaiu~m<|Ehy+W)!fxz2RAc1|r`Z@97+1So)z zk@^eBI>wwLg8ew~s&eLGl?YUf!C;+K($;~($=&L!D)uBc5h8UIevIL8prIC~eT?sj zIyioT`{v%A1VcUy@Li>==Bv!g=I@>rjfZP3R?Y6)6Qo?rF2C_-a&5}G`D+b;g_{R9{stINc15qzcogIl4G2D9W5Pfe={}cF7%k=(2tI@A?j2eY{)SA1&N`KEyDBjC zL3?3XgKG-Vc@vC$dPdWtTbumH)+rbjR|n>tM)c!f315GDMNoFT>Ti$rH*Mhc>s+V& z5>L5H8?e(FZ{Rt{Qg*yjb%%Dba(E4_=@+&`^PELaJ0WG93oVGzSu=A$p?G889Nh4~ zkUz9pmFR6ELQ5`VvO}%+Rt=pvB=A8#m|+zH}l$Hp=v42zf7 zh2mn*AXhhF&rb`vFq@mF5}6dBxT{wFE9kl_@9)NuK_#f)?G^q%9mW4N z6l(eib>LwC?JdR&Sw5it=kx^QR~^`Y+YBfJ$u_Bf7O*a$|M9N;e@S6;B8tC-TB_lFV?l+KnXFH{&VgbV_%CM<(j2E8}C zFs%DqYm&<`Q&7)WZLc3Z7D=!au~&%5v?^Rxn^Jr`>&?`xJMC`z`}OS#>JO+ac8WY* zUg!kbJjU-%!`*U-l}FdDUU*P;Y__Xy+_cvM1cD9b`XupqAv3 zqZT!i-A9N~U+kI<6B@x=o-(t?WNg(EBwQQ$5YuE+5z6-Cll-7JAKaLtg%}?qBu_F9 z0j`-RG?UUdno=CxkuGbLcOBW&n^*c<+U(Ha#mA>FYE#80W4d_L3i22w4BRHOeujEq zc^I>)^rAew1EdvP1lJ&RN^=kbGEcQ@M+nzIdSqs!=Siz4#nrY%hnwt*KEFPrmSk8f z_WhW!G*)E--qv3CEo$(4+8bOY%jQqVM-G3Fvt$ZrFai@`-Bg_ux1mW`Nh_bqR8>W< zXnEAoF!_*E@FtcF-u)H*^3?NI%Yv+qQ1XK6x@&u?dPM2NN9-s+twrM_;)6U^337_-4OPj6QU;qy|4o1yU zpgCD&%%9Ft^Tg!A8Zz5j^^dS+J(yX##D>pN;?H&|fS4&O0OXkh&ieKCYL@(Z6j-xg zc2T}n2X;Jaff!0305y9Ate*vtIU|_b?#O!1`}Meb{V7}OkpM;K?)=uS!r+>JQoNbh zbX>g=_QlH#obIE(p4|<6FUTF0fu8m|UN5~VpWMA2?N_(Y*(icN{(oCS%4$B9!9R+) zKUKj$3jBcmez`jqM4#xNrCZpcyC2q1MFAg;ymvs;Jx!Ch2z*WHLJc*Y_7a#aYnVJa z)viKaT+Hf{d}&y~d{}Xq5^9P*;Cb5luf9wHvg{)wX_PW`9NKB0`=&VCpkP^I97-ZP z>3&(C4@v!(`nvet2r@|;oBQ-H*;{>%UowN>G#Le9>#cFJ1Ql1XLnKTj8o08;k@hw@ zhVoYR7NglUm>h~{#;lnb;wil=s8Vna+*#6t?!U2OiVO18AL&Gecd^CE0T842tMmt{ zmaWPKlMxQ0*0_!Td1`GGp>Rb9gdE@RkkTe`|WTds+PBkIgsD zu`h4f;(z5`W2s4Ipzg_A+fibx^sq!cJL@H$@A!DDMu|?I!WR|XZEKUr9@5zlGduO2 zV+&yh>e*d6Q$4k!-VMuh0TxXI&7Xd8(hd}&up2hPFFTcpD78g6u<8)t@vri(E(n>9 zE3_a!FG6;d2Y}E`D}72ZWIPw@+mcrBk)VW%X?m_NNRTWtP-dTI_``dIn&C8B^ouap zxzJz<9#oG8i0+!NVvPz%Fv@pMy#z-GS{miK+u4ZeF*6l@3M;^u0AgV7modADOu{h= z(}Q5@#Y$byEeO%VW@E7QMXNufl}-0&KpR}bcpgUsGv^%^bhSCDl;R?ikyYebG`s?k zF~YFo*8iBs&q3ZKMQ*H5TKeZdJ5zeOyP5Y!^2zW_HccK(rc}b1HZN)*sH2^hXbhBk zhP}#iRmMqc_-wps0zO?eT};JkH6C^^0&2P@{5d8ZlSPeu zd7|7?q1Ty=g*`u%^X=5+ql1kC%1)7o|9EUN4h)2rP+;VKcQPFd|AUD)qCOzT7E5cg zMfs1jrZlDH?)y|=*zzeA0=c-Zj z&)q3_NskC%0q_dd#kjegH-oy;6Hl3?OfObrK&|O)HAO1W0~B?R#oMBdnAJwU+y%w3 zkY$d~P6Zg$21_QqG;${dHG~1bUJjS9J=Yjd>(!G_6^U9--BAcuj#M?uC5q@cvw`6syAYMhwc3D?MeF)KZS>1yhph6W5#|n$#H00O92m@3~S{w6n%Z^c5yigs~Nc zFeGV*SWNSYGoi|GPjkMXoW-*+Dh@PQ&~f3yoreF=kxkL6s-+_YjgpDKYqW%?}fIc))B#!Da*{rUCtnu#-XNzPm9oLFH@DUdal>vP4Cz?jHlhnSRE^8DYE?!s!b45z+&YXmhE^Qg^b%>E?-t1Fu=We7!pHt&Dh-Z#5 zfzD^D;RvuFfo%(gz*$>FI?fkU#3;pCZHf@fHXu+nE~iyDdjk z5e<7Wz9RGckG1=>(5d0zuL!D#<8s`^hKrs+2-|Mev|HZGoe>OH0|UosZlSHo?OkdL zBwYpcNwHq*8$E8JBN_yji`DRWBk=rHb`iHoo;4RAZC16(5?pl~qyv zgP+|t+4HpIf5`c<=*o8VX{ z+yn^GxbmS;VAr z98_?NmOH)o!YP`&V!?NXGTy47++ZKHa!rRPQU_}0wGJhDo=GFhIAPUe)-_S$p+k~- z<+Pq7U{Mlv;j}I$-#l~D31pYGanc>ww%w0P`NSzE?*nq3870}Y*Q5^jkh*1iziKv@ zdE%0#&?>rz8fggV-sPk%eXw7cmWhbGUSJSVGiLpuTD7@|=oF!-lCY-6$R?t^|=WV=9uW#`IuFNOpO@P@`Sv0*e&h-QsWb+bl)@NE#E2CIxN$pws z3!0pI;gx=I-ZJ$VYCy}*t1?2Ln3uLaE6CuEs#*Z=()i7`-X5j9ErG~C$t1x~kk7#P zhXH`-y&v!xe()Hua7t3;N%dmT>5HW+$lo2r;S>0A>jW46OKfn+3_7?FPRxYo1BL47D6@#Ev@^4Tq|=JU zOh~&SqoLqX0ZIp9A8Wf`ojL(wQ%=WRoeuYw5I-NFb8bxL6TzqTrY`%Zsy}r(k~_rtZdq*|#@WF?f^x z9=Z;wj5E8HWihD3KHTZJwWX1Vo;4`;ee*$&3_5Yx2<+|Lic6pkpDKLxfw>26A37*r zZ&3%dG(a1Er=D|6ltwq~6j`lKYKXdIcC9FH2`k@B?%@c5-Wb%8g; zyh3U{;&B0{^Hvykl`BFLQAPZJmVdm)DmCTK!Con{qbfd;2+C9y>8#M=CrN|GtS`6>_ zxgY3DQ?MZW;BwFK<2P2d?EPu`W-d|i0&k3P5SfX2$=dkLCY+5Gc5?C(jJ)-C(?2DY<(xB%GBHAhk$FflIEp3(hM*91@Q*j@QpQ$Ou?}CJtJ^i`JmoAo@U3q zxplI%UWvu`s}rQrZdh1kf?odvO>&L{x6oIOWHI=<7+<)O2Ru!EW`8A) zNtyy!5>rcLN9&OU1K`A6(F4PyElgY*4+kc53hd7ue|U^Zs`ajtj@#Q>dF1Z2_X?}M zB2dYy^m@C?Y@+lPUs#M;=w{B{Eg5P31t%*zszY986?Wfs!tG+|INt%+}8o4cbdu5A1BPDHnm_OHc+x&bSsuB`uY@H(nXY3H<<LlWY+!9lRI>@VfG-%OVj0D4{+nBK#uFlQE+p98 zK-Q!O7+ zeiu|Jk5*806diK7Fa_Jv+o3Ew&aT%1G*+c_ijQ9{Wa138JupKBk1^h^dh8AG|)LswyG^&Z?8V^G=!IN)H= z435Od26lF`5bn>9gjA2Or8NRa4&G)vATvV84o(7ii(n;p{V`sYM9uq1T<(81_CK|~ zUla|dLUdU4R9SeGBx6g-ccS&^T%t!?>U5wVY+W08=RLxQ4=_%S ziV}F!2crL-H1c$su(MN=HX~3x$LtLeYh8CO^W5>Jk2GgAxdYR&%~1;fd}N zjqMUj?GhmHj4*iKo?@5r^d5YLsg7sQV`p}<01BX2MH6a0zK_}GvPH@P?4cHKqL59< z#FIhEk!{91WT3@08?_!_c(4y{iQ!DClSIU%WQpSkG}#F?UONaDqdE|eX;b-;ia=qx zkc`Yq%?RAzai=W~F zN;&cep-HK+;6?>xDR0Ov$YcHday?5$H_h6Onh>s~Nuj3B2PH_&CvV_`lkmE1C!ulO zn$Jpm)w9SCFgNsij$!sGsQ`r_*%j)^zPazn%ue|Je)yvBHK{C#6Zn6~ddr|VyJibC zKya7A-66QUySuwfa1A!W-DPlh3-0dj?rwqLPT=ys->v(db8gktQ@d(vp83BM=*?ooIGCG>2Yn+8{_$)OXRI7T0a+Z-4)=$-P)834v%GE1 zOqa9~a;o|=TDj`CxbJ_;R=&Y(E5Zhv1hJ$PoPvuwPu}ZaC-p8&LrgfZ4n!_kVAHn> zG_T#mAvc{zNCK)B9KSoPBk83{4z`rGnmOFNCR@IBZaxg4)=u_RyVSK69#1znwK&6!~Lo7{27Wcv`SDm zZG-W35{A&_4hGh^;B#W6qA_|GRx%JuV|0!VXAGHPU+Sp8j-_ zn(Nr`gN&)a>@d{rJ*v2U)Y9I40C*p_>X}v+*F#&$}=p1!t7l zNliHq#SP7^C3LNW_LZNn+=1K)-)(ko-P(4=w}x` zDhMSI90A1e3Jnc*ixzou5yAQbf zGMZa2;d6GaH=PkBrCP0BTjuM6EmQ4fQzazF?yXIQ8-5s#r#oZvZvm!7r>;dykFn?K zOrQ4?5$bET0R5ulwFhJ&(1V^BMNp#7*+?u1OZ1X-(3{0q23N)M@p)~w&h5SYvd_v) zWn!0Zwp(t6hhOz{s$CWy&X4nfdr%s|dhyvin?;zS`C2-p@xQZe;kK#nY&5wNj9Z^< z7_C{YnrSTER3mI=vR|9i`ElIFL4DzC@!lQA5pG4j%gpl0?noJtUbU{71#v8P%UH(A zhy)%uTLW*o*JAR)$+mcjHhH>Ef((d_4ugn%s&qOwS`zQh&L+{_hd(fYjPWNoF_zDNi5w&Cq@7z0(6dN_I^)`IA)v$?BH8&q5>si!oXV{3OuYg(Tksb> z9^5=y^;pYlvqxUUss%tAnIsL!%vEtP{kep2VVQ_q9W3<*t|Utd4fEe9`oERH0Q{6! z@MnBQL;d5s!vaF_<%0qJ@uY(WPO8Z|{M(U1VwOhyk!+cVfhKd-;I^j3^)pUsfmVE% z1oFPiDP<_RKIU9r{T*EJCs8kA8RP(%(~%^J#b)V6UMw}0iMb^o@9pF|@csP_(g>h9 z*Af-_lDa%xfG}5TrqmYz%PX2t9xje9OPPW*q5lV@-#Dyf{llLuSOrK#&^%m((KXLi zD9_N|1%v0WW}?V%I$~8I6&&!0b8gzcm6yK{rN7I=Uj$X5pqg``1)@_eKNdg4?O(&z zI8cJq{Nax;o(q=kJ*UcuEuc;*g3UR4ov5>7BE@l%K(c;Q=*fESA@-E*;gf)&<3IkF zp3v*Xmt5|Qlu0Y&fD`!ri7(=7r$82Yj&qhx0x}fofzBGt3xpx{jWSPr7M>_Of;hg% zndk)TgnpsCyy3RXppbXC92deNI$_AzEsf48QooYaVWwleHFW6)c%AJ#x_Syp z$-c7>LvvPh7D1aus^GTqcw|y;H48?pT`b(NQVua%NSTtDNTVeD6x+pz`SUjO5N`_*#z`Gyp zCpnV-4G{zE@j}3alLmeMFZEc+aWx<$qm}LWH<_ES#iw%d98q-DsK6|8syF=vFQ)-KmNxApq@K00BAjj92pqtjj;&$kay=y)W-y0vB6S; zX{1;s|308(CB$|hoL~=bPJ&IHygGSqVZ@5(VM(^4^QxjNY*&GK9j+76pi{e8>82}d zxU#nP`1<#c;I5|E)Pz~R{h#zz_J`LO3T{@H9UiqG4Oat1DH~FWql+qvWK*BrbnLYkP*55^lM(xZSx=Z)5=|Es zRQBGobe)N!8^pyw=HVFbgG$VOHbWadm4`Tetoni6gm09(Zz)j@uQM^1S0f(UnU>dR zk(aek0+Ar?x1WQ^+8up+k}Q67#=?A6Q5D{E>3PKzL47*3g%u%AQGluN-cS`Smy-xHMDwddJQBfom=ZxtK2xz75)-W$tnhSu#Xvv*-_DsrHvCQ1E zMp;VvzJ7s_U1RxelxwWQIDIh^ZD_lS7hQ(J{29GZNlz-yP*}O8m7AQ+P`v%3k||Jv zmVUnkjRCDAZs&5@E%^bAB2gK(U|M(n-OQ1e~&^wFDbSBE-Np-jR~i`xlj(S@}G*msMKVad*h zM&jtmgqmvXliSL}bGRt9s?vG`Odue;YB7^}jRc z{W%4(N@!s-C#~cOoA*@~egdw#qLZOfIm)pl`^PMPB)c7nNmG^dE9?CGPaGG&LLv7aVp^9|wIY0x8+$@DMfO`CDfG01*_SR;u5AVV0=H$Xs!l3*B&nJ{{m(DW zoj^qG%oBSGf)X{$vA&)}k-+u8EipvMJEcU*16muql9|JMd?`I7n8R-~&!0}7(}z(l zUUjV9iPzqObx(mo?!zdZgN?3%rtIF&>%8pA^n&jrs3(0nT5oBz`y1CA+3fjcl>#~q zp4!ihF4vsNuWs>w3@KdIPzAj4OoO^7@4+geJiaw5*`#BBk3ZY4UWVkyB~$bvTh?HqY1aP$jvlf-X(LVz z#LyE6PUoMVF-7!}W*{{;0&INUCmXc8bFF13@I8T^Xb0;E?dgCqoq@7ry8q%pSdFTQ z^rxN@BBZkD+lz)Hu$EuYP_upg3JZ$0Fi!0o4lm5DFv z;S2bo7b_atUnQWdX<*2NWsj)S)s&F-1Fsx*H+yf0@nKfejk-HZXYUcd&+!f|WJ_CE zXmd}0B#e8Sds!asRVmGt)|DEpTyq?Bax*wG2uH@j(HzMSEcTs+&wXDf5xUVOz?06W z?yH5=NZ^`~|4qKip%l*H-}-=7zECfJ^yL&M6QV7LwN9YI$-tfp;cGBosV&O27U>Qu zegFQ?Rzh-ne@MQRls@txzS3Y+ie+(d!ci8!d?opICu6=X72Me0!F+pro*n8vrgaFh zEdvC_skkMSF>gkOHI=+fQ;+*t7{!uw$Iw{4veLRCbRbWlN)07=R@P{v5QinQ~o`4Y7 zmcRF>tt5Chz6e1!-_zzA4nJHW#f0Q@OD)_n|G^! zNTy@Re!g0{2nxR*LmaG%s*w;D(;uNm?JApHVcQSvw?LLi?XxNDMd9QL-0=TgKj{>% zwdt939mybnBh=aE++kYGN6>K!a^){(g*ziWWVhF62#U=bkS zx45}GE16D`iE$F?oW~N9*xI)- zj1**%uB7+B%uT;HoT|;FYpAiMHJNSIe$$ZN6khCUY{@{_kRkhSL~vSbpma> z>qL&%-?Y9VqvXooa?Pps0?Besd4r^`b2RM%toCAZs^Kx*)GwL!VzZ?_*Uk;Ru{T+D zWlzo<`VZQ(ZzF_V-93%VQpq*WtrsU!03LJr(u-p5qSwd2Vp0DshQ55k{@?e_tKX>R zOrO9=9SX=P0{WAHbInT#;%z1%18Tl{pbrDyEgF_|ZcIv<2l`0D7Qj|hP@QC~?3Tr0 zxK(QC$wpBis@kRSyoI*_6@W%vnYo$bUw2t-!FTaZ)|9pz*XJjBb9>j=(TIk1+-wGVBIa7`^_GS*rht|BJAmOJkv^4Wsjl_(K zJo2@=3q|m0QVO)H2#7fkr3)Gu9M}@G+po7~#N>B4hFNB? zes>yT|A-Ff|5RL+X(u>IEN+!x{F-ZT)jwDUpPsD2cn~(vW@WLkVZAsv*wUADE}fNP z3$SRLKdNhV9VD11-QEky(~i&7p|&FDQ^V6WCNsg+93fj9r`}!F=;yK1ZxSPi`-;sw zUPEtQktU0QF)TWU4#e3m;VZaeAQhV+Vp$MJTu(Q)f<-V;`&D|_6R(ree+f`xvzhnm zEhd(byG{5bv=eruW=g6C%ZM{K7UHhnD2jq}hH=_dIisH@(ANiG*CdZ0qCl?ik5RCV z?l4*Db>94&;5N*dInv)P(#4)RGX= z-v*7Uvn&Bn5FEznba3p1u<*RBd=>D~jxhMbZ{Y|fn**q>Xds1MX}IXp$?~6J4JcE~ z87p@H9hEyJp2@wnw)YTmx>c{dCs=~^p~RJ^ zUrbN$q)OKXfu{#$A3LVOfqptubhu3x*4o|nPnv4w+Z^L?u|(+>E4Am=dwt^crr(|* z-l_NHA%T4~kia3`zTIZnxW*rz9i_)s{E)z2LmbavhDw*zsr~J6Tm7~Oy;b;Da7W52 zaXs{q@Fu4jEmb?dNoE_UfAt}W!df$)lzRC|ypTQ2fp1wzQy->wtoR!TvlRP82)q@A z`?N~qO3bbTGsAz1^hyd`Md}r8%*wcD1X?&m)I#prfNgp`45TFCYfn%hPX{yvA$0v* z)$7R~9Bk)R}-Dade1ceA5!j8dD!sWIjLm_A7_w07>BsC`6OMz&T;kVZL7U5LNmzOP&OS3%4Td0|#6{eSno=3!! zef*(qKU%g)aIgeKFYIr?^g;(1`}!I4;*V=$74t%mc*dG$BK>;uf*2?Af;czw1VT;C zzR9`rKKVWok8(|!%{)t8T)*}eQ6N#t0Eb{%(8Qw_MyxrIaqLwLPdX6OW+@B=#gc~e zLU^|Vd9?x_330v1bM8>n!}o8b_2HFdR@^NxmjmIgI>IY@*H@+-A=U${*H?OL3y$3y z;+S&!snXL6+gG@c;EF#f5_!j#G%w6@XlQ&Ec-e6Wck&AVwjTwf5H#{y zsA{*So=rIf+-u0nCH*{i2#Ox0A{aW;)FD3g)%zt_SU7g?y zXp5}lSiYsQ3MlpUvdEI!!}{NtwX?1&r(uM#LSKvCfsp+oP!v)33as`m%^gz(1F1>{ z$A(dkESU$Uje}m;1QqT5wwxZ|pXlym=b(75(Ryp4A)dgpog;HzbM`^QK6k(mw0HN3 zN5g-5ZvQ=6|1VC#L?B0e@@Yzqg#;qOr632zF~guWqbI?@K>r8xEzg2+21op-L-F-R z<5SJ-%NILPEi4r_ND2QF?fv<=gF%9Ce2C5I$>4S1NT{EFg_A0jk8NuGB0)j{uIh5IV4Sm1^sv@HQF?{Yw53EE zuV$A5t-BR<9^G%}pd9FfjljCcxWen200@E1U`Y(Vv(K^Zfym)2cIm6% zH@Aq=zcb*vk^_Om6b-^em{L~kT-2bPt6dZ%SrbtMg#A%H{NA{vI~s1z{GD~UY)qvL zdDU2Uu?A1Q*l(T^#NVu}Qy7$5@}11}C1qcWm)a0BaMO7-eyw4A-&&z2PKPXJ7n^J; zN?YQgPncl^6V&z+?SyraomuIf|B)v~SDc|yK^SYxpQ#00G7*C)w%3O6H<(0_ywf7a zK>VF2l(U63C(`MaJI>U&B9yyjud}KB_=cu}jLTnaL199kvbr^vHs2s6x7-I|_r_MJ zZ8=B)rc*ptTm8*`up!Qk?-ERoWn* zE;q8qz+^5DYlcm!D2|@e6IFYLN{z?b=$e$jmBO1DKA7#mOkm#uz9067i&q{353 z@sJI?vBkp*>KNneFVu)~q{VPeiuJC&Rpbyc3vLk0yi4vPFX2Xatm_Xw+p}?$D?0rU zXODqw#kJ!&H5$?Y*Jd(zlZ`H8=C9VRW^_Jz(m|J(M&cc(>PSe`Fg}H~T9?UKOlBh8 zJ!0l;>v%h0{Iy~1I61eJo}Ng?@uOTCs`~&aJ!~+)yNLBEC_&;|L?U{YMum`kELDK2 zwBpufuPEjDFE*(B!crgB_d~JUoct4 zeqf!}k1|wnIGNUuoxj=+t=Ab7t*T?vQerkcXVhiCkyI|Y<*?of;P?pSN&|x zK5eKlZ?XPqtF2YF_JhTgez@zYS&i?0n`Z9SxCGAxm&W8-J$cv!pESoTBOv#11dnQ^ z(zsw-JTKgk=}61AGUkSbR-S=|j=4m`1ZMRFp!)MkN28Zk@VAb}T5umNr@I}taADtW#oud^x!Qmq&hDHiOE^8OhQG9_EirwOdaB7+93;S|%Bo*zF7 zn`Be{V&E_tWL-PU@Lp;G#JemXQpg}Av2<}t7U)r!pd1^pcCgl}Z|ih&Sq&Uefo@ng%$givbjty2cU%x;(IvX9q!05IKzFwGhb=AFi~)A1 zLnnNax}5mx?+F9PAjONCj)eT$$3^1@$pX}@p>iItUzb{~Ds9bZ^JHz99usy2#|Q#S z)5q=IqOrZ3cYG&?+s!OPJ;pcBb8h*)5d0# zn~`&IT^=x9t)e3Y0X$LGlHZO9CAM+2SWL>^N6XxV_ux(rUsp@0w?3KP%eoE?jl4-aQmWM z2=|{L|DZSJ`?Egj4sQ#*zmJ~AkC9AdDmM^i_ePK$XsS8#{bo`qD1=P9`(m-|c_5-L zUTcY6ADc@J@f%wDZvz>%1?eM(=fke6bYYJv`mT(*za@-Ij1fa~g2Jeot3pnY0!_09 zx2_=N4nBqc2hhxQ8byj3ri3`Ofv;<&9#nIs$0245_GxuY;>2 z3Q~#F&r;VG^8JKCX?a@JKJOCy#vIwN%r0Lo-7vrPQ&)^~|Lj1`w2M7jJWvB1w`d*0 z>Xt*KYPg{gd6X1&5d`Am(~Gr!6Qd|OLe4Y!y90w+2>c$s;yZPyv3le_rM&fnBDy^} zL`|MIu4+i-%Dsu#sbW(g;`oYi{rI~2doT+8pm0;ThN0@mzSA#-(J;8*LojORrwWz@>y!%7N|+(bRb!RQsSEo0~HDBjk6Lf2i? zps+%`P~c3z?l}czU>K<<1m%`+(h6Pv?_ub|JLDQ?5j~jifFCQA5jFZx%*02=p;|w) zOh~FxA{~=UdlyZ(s{xZO3YCKlvoUyITO$R|F?pR5TL_!OGMp+};G4twTlCKd3HFT7 zQDb-QnZvxP?+u(_9wVq~WBr_SJW*c-piMgqo=E)cbOX5ZKY_D%u6d z)3P%W;Ld|f<&7U*RYq&PHy6x`38>*Vaj2BIY&-Q5N&5TFWiELjhbm27f1VPJwTxcn z?EqOIbur*5B&q*H z;2O!Mtg~DZj6mb#h10f%(=Lmo%h@>Ww2XqgC}LnCA;Pb==cq|8^##C()M`P6xYCZ- zTX#=ry~F*wUQ~u10PP*S*ilRl9y?2Khf8*0>$fs`pj(DV#WS){ujbY1Fp0sY!F^}J zIh#AJoS7nXP&sR6N^}?!>4tLc&4Mx{wtU?2xt-+caS;f@(EWZ&Jch&)$%D6M`e9vZKARgTB>ry7b9ou&&}HeSmnY-+-cn6codnf`gh)rdG*nI1AUlPc}g#n zPrt72@FnNsgt=YYG(W*O2!hEc9{`}#$SApj9x?N%M`ipdN0Cy9R zVAYVB;ez#NQs?jdfO+|LKG3I6Eg7}pp3fbdl_K8G=-7HOUU>^J? z4MBElG+~zOsc1y8;nm=E*s|!GuVDHv>E5Wi*KfL+3Z}TsLl^J?5@2XQ5+hH?!Bt)V zHzY}X12@mf-O~QwNW=JBMoUMYa!IR*$O6MyaaR;&yaR97YjN*ABepHs@d+qY_~^EV ziGu#W@PPbxLt$7{OBdJU|#to)6BAM(Ndiv@v+kpsTjvE7-gkxLqt0AGxrfDCu+qK`4n4 zD5@fIwh-GUq`H5XY5DF6ok`zeP4@Iz%=x&d4UI7U=T?melg>;oT%};n{yh ze~7M`&+qHGtY4irD&zTea#)b%E}P+@d&$=_7cH~FO`<5D(*jW;EyMXZs?lek?9raT zZ;-0kZBX8ojU!#H{KfEv5ag;065%3eeFWlrLZY`J{XPrWwLEmAf1JhVJ_X zecu*)8J9XR6i<%lqq}=?vGLKMzFqLI&n;39OoY?9gdsv!oGIH{{|ckYPvbcAFmuLn zft0^=&{`qd33SlfNVQQl_!c+-C$Rw_Gnl2>Me!ltEu^n2OZ}s4~(4h?QVTCl3Lf%`>6MJ)g zm8Z$~+8Lw-KbB`M#@j5Je|^<0OL2DFhq2Aoc}tD_E!XT~|gl~8=|vw*d9)ac{R#G2_BlwNCN1<|#wB{1hS%C(n!X8)F! za$18-VUrCJHy*8vs*S2Y&|;{Qt2xky%hgBLwP&%?cP?@p&2^Ku*T4C-M1V$oh(CNc z*JMzU#ts(l=vK5^QM?Pk(w&_E1eaK93`hnoH|$Mvx_XB%JAHSwqU?T`DLEM|m!Vaa z2EhfZ)Uqg;iUXKF2VljPD|BM)p-aMJG`*F06KP9mwb|8`PQ<>V+O=uTjnY^&Ao?Wlfdg8;E2gRt@$P$pCH{J`*mucYA{D*Z8GPP=Hs0DuR=xm(xq>@arO!F^ zaBBUnPOcwa$5{WI(WFnGd*t){B>RNA{@Zo>XZtjt%d+KxfBko<*+Z8Ajr7kl;GE&N z0sNOQ+nAv1TqIJ^)eQl1G64h}NWhgBsu^sLz!U!8bJ_J%-va9s6!QR)Oh6NYu2u*L zKs3+1NX=}b6wLqcTgP4qJ`ePN|6pPea0-q1&)MDt3ZhZ|{9z<0-WZG!q%_8Z*le=^ zhyzFZ=Udl$=HI`*e)8YoK=DirbRZ(nf0)1;lbU$y7$5R1#!}ca2pGDuG*&--h)H0L zWaH*;luhQ9;8M9KgMf(_M}SLJ%TYfcX3Ie~a->xAF6+keI>NTV4p4MS za-D*~dqguw_NgyaHuZK-lXF7PQ4{fHyrGs^EvVXWSl9wZ4aCc0`24xUDD zt{HwBQH!Iivu#qIjJ7jroe^T9F2}PowM`YSr8EW!CX$XALjba zungr9UHPfF+q@$25P+Gk{1ZS*Q)-!(xN47Q;wopVGTLS|^Mgzy7waUqE0B8PSaqJP z#J4#}YLqpp6tYgu%O+gOGrH%PPJGiZHz`v~Kkw>CCwUoQ-V7@3hlnwp2%&tF>V|m} z(apNT%wj~fR^^#*{-_9>E0V{7@uEa#*UvyNbSKH#`X6c(0NHs{cKhPY;@iUrfT}q> zE>&JV^8gHIB6Cn&6q5=XA#nT&YJ03Dgb;RnRH=qRIJp*#av$o&jIXyi3Y-l`^+`=a z5XvmdSWjr1Ip{mHrZW63&w5_caZY)qnYFm+5T;k@V{9)Y-l0sH8XJ96t=eAcsoGyI zeV0U~2LPgPY{7a;UV-Egl2mes5E?_2rU*rAb6l0nv>MNXV!F>{)@ zmmVw}SiWw}=yvaK2Y%w*_a*{7EQTm&|Pa zEwPfD(!~(_0`CmvJ|O-1YeYW!zNiDEP4rLpsJY>5+OIxe;=6)ecddN0x>^|S=Cc2! zcNV{@gOFeTzLn3diFPxQl1I*+N#5)fbK`^)3WeeK0dSC7Vu=ZR(8CGy%%3R8v5;oQ zJhrUNh^i#7D3E3!0wy-Fi)%A@0fAQ$5GNSWvy4b2Bq3L|RX`V~M{JUr8cqCa_-iQr z{OaEn+5qQ-p84NGWYFm)+PSS+C?gKU;nT!zIf+6IBn^&5T;D^JDxkCCuAyL&*^KLV zY%+EUU$4VO2a(xa_TL;lPiiZ z3gU{v+Zzbq2m~UxtK20>CT$r$k*l ztFiLNgwDP~bg?2lWjZpvf8n+Mka@r#GrL1fNLn{H*fX4wOcc5=85N!T1M=LC&e=Ki zCG92-7rZ^+?j{4@UK~9M78_qj8N@+|@IV1vnrLPAr)sqP)ZObuFO*xtGeMg#kxHxN z{JOj-b&WDrdNAXh!LhiA@Mq;#e-CZpjp0SKB`lVD_Z!}K{z7u+`Yef-$7yMn9QEBFJ|B=qxbU!6 zBRJUoEiqsOj0kr|BDmjT3<3jE%;bh<8UL!`Tc4J=b(k@21HOA#7#LJkb#!iZ1(@WX?9^i}(8$&V2I^H5abz^1`hzxuTGJ4seIO>$s zadaJwl}pWIMSU{XlU+T7I*kV9OE`BE5_xq7`6F+k3f==TnYlFTU1LW)4EX?9mOBlf zKk6CK>jqjLO%Ob@>i27sA`0NY{yv~TSIlVf1i;s%s?B!7-QiScXu2I^!St#};^ilJ+uyYsV(eK;pFChvT= zSO7x%EQ0=JBV#hML5aeM0;Tab)%!2O{`13E4MtMnJfU84ykT}B)spC?sxDzd0@%%8 zyPT9R2d%AWGh%rU{AKN zH#osuyW+bMnmsBhY9lK+*KbA=0I4BAw1@^PL(WFbwu?zCcIt4H<$IyZ>J3{~L$@Ggz1YDGMNiHXA_GI{-U@`hM%0= zxgIr1k*m_0Jqj^wQbW;2TQIYn2^w-@4c8|Qi;=bvn7bp_Th_$bzWw4WXeM9ew$@OV z^T_+aTby-k;wG$?h$+dVnC%iXy?FV0yxsi{w!=XfL@Ww#3c(FI4tK{?ZJa1BJb2H9 z1Anfp#yoCSD#g@QU=jm#k{#qLC^PYQ(iKmrEhHV((ulX%P_7+$s*a!-iU5}t*;r$7 zKWlX^)1k?@*Hf;Db6&amJ{^z3uD5)8pz|;q?n1g-z53l!Z}?VV5`XPaf6DwQ+Zu9~ z#L>*Hz!Zjb(qn992*^WgQ#y=V8%}k$(P#5hyF`?gLhS2ji|HE>tr&8Bo;IY(_{>d} z5l}9!9A{82lZtU9zwe(dXo&}C(sOEVP;ZWUbc5ahuo?LXHqB|vUY0jqFLgev6}Y!( zrE`r(RI&S=QUsvRZfjR9Rm%DSD`TR_>fx3?)rkz1DS8#Zd3Hm5Nu?S%6z3vV=Jx5) zS@h?+Yrbslm3~f>n$+`|3|3-}ua0OnE#-pHjPBRGD{@^K*!%^rw&_JN>RuKPaz4u#c&~+G6^!KnkeDoncw^Vv^b$%`rO)-Js;hdpSS^|qwuuR~jejl7B!{_o4E~wo zyyyjYIvSb0Q9Sx3$MmbA5uD*1)fJ%Xz}D&sioz!n`I;c+ieI1?`Ru zuATpHPx;D_YS@G8vzGS$R|buZ!zHo2mJ-5DEC7%nuGP6H$Ts<;!1c z(C%Nd&z`X$1S|+&hX)Rrr48?+wwU(uS+;suIHMf+JorQ71c>osh^#*Io8d+9`VGkZCX4{QQ{wKdn8|VWk3-UL zaTJcre4O#V>k*glS9-1t+B5H?iVASmN0_z< z`Cx*RD5974{-$Seo_A=$o@o`URKTiW9f)TgR7#(&Z;{(|My}e2yH-w{R_D`2aKY%3 z<2#Os2DLxKb*V6=QH4D3RYLCB84K8brK&!{4YnPW%jyG0>bZ`I+TB&-ZQu9wMFQ+m zrYTnJ5o8Cq%uV{C`8qOxq4mQd|2j}NY0>eic6Ta>T!tB0Zt<+jH9X$C-&-xbf(q`G z3*MrazaTM`1YNG#$@`Q*v#7sBAs44to~H7}d(st)t9bDZsaX(ziVpE3Zf!Y|7H+cT4wCAS;ll(_H6|e% z_&!{@!&S6AT3F$D@-oVIi^F@^hMVnEaJrjdkuzaCIdITQj<=Xsi4a#41u??$WE^=S0UT`akW zLg$w$1Ri6I{AKrZ95m!6-x7c=FAIw)a7J5`UoCCd=Q3tlq^3bBZPZpGf8DPwbkrq^ zO2Q9ym5B(HJQSDjL*wT@D~(GDFhFT#y*~iXQOFLY6XMPIK);}>=y3YTCjKfO8oxVe z(sIu27R>G`ul@EtXv>w~+6T`M7M$Rqs7GM8m0{A0emRjK+>x$TSN6#y4!mpsf*|Yy zU<=62ahx!^GRh4J(^O~y_D6b8{(%Ejg^Q(~rSpkIOW z+U{G}P_{$7Z)+098>7w9V^Su!d$2QlK|;Ioa1Ih<=*p2rg-qEFa0<3ZF6Xh>2Z%?^ zJ#`&kEJIin-y*R8kh7WdNNYZXQ3^ii%fV0b8PVkJnY0Hc$~<7zY2IaP@sV_xUq9J3 zJu%VvK$GeK$D~^3C+*y^;T(j!16F_&sf{$&Ef|R&V@CvxlzbKiOF1*7#$;P-&?wQq zIwjc)yKC~bS{pEXg^nK+I5@bOOr{uIw`RrWpqupY8M%4W%{Bj}&C{Ql(Lv0nIBZ1c zIzN?L0GShdNY=hyJ1k zt~`k~O1-!|5>+I)(tdRNRtE=*}hpa-o_=NM5TKkar8 zkxFZ%gBn-m4MR-nMznQX;2)rh_WmaAqvNl!HsZlQo%rgf%rE5g<~XO(P0opG`G>i3 z^+RWt{XYrCc|tuN#ySMBrdT{Ye2AAiy{aSnP|-#84*h-5>9r&OxPwR?@oSTQn%C%%kSKJJr5++~nE0+3nr+W7oz2%M}U-ysn(hByW202rD za8jV{29mDB1Nh2sOy8u*ZIX$wik~rnxK8X&Og(xFR1@r_SH{UXxga&Kc!>c=iNk4_ zg6tv}DHK}4QN=vbG#OwUYQwfsx+$j@+ZyJwadMCyh~a>@Pcc%4Z`D`4h${F-U_3|l zRGp_}&R|z>y7Z0CSMzZCwi8aUQgGY zyN^mqxxV!m@zZDL*W*HXn#qQiARgpOTdyxsjiaBtySg9A!~@LwGQD2u#g)1rLiak$t)0QdMARkZqrite zPkWHFr54#r*IV(r_?g0GMLsDX?x;v~=7foBEp6DGwAEmLR$k(KSvqcSi}H+P>D^J) zn>)R}i-on7NeIjv71vfrmW__A+h4YB@@?G`vW5A3OII&`EiM0Ie(?~Jh|=Ng{62Hu z@Giy&knF6j)>tj2a%pWV1gmtB{l0t@En!Org!=-cSUC-z&pgZSJ6$^wP5T9s!6&J&~EZA6YXlc1Q z41RrW)N1ayiP~neAEVQZHhs{&qT0Tc8Jur56QRp0_-Ym3n>Jt4@721fdwAj6TLggw zhNSGw@yesrQ)hKP;OCTh%1UEf?mPJVFfZue*h)997qOQ1j5MIcL33=>kr;MIBk zd%TA>+yLlE{3)pV-M&d898ntADK8Bd((6~)TM>L$j2w?CTkewkhlQZC`02^1P(|~o z6lp}qa^oKf`syv4<|;>;JdSNQJ1H&{otAvT7J0qmMkj3}5tqn`wxWW9DqCx;hNGB) z*PTioNxlk0*g0+?81;?WEer&^n?{f4fUTwIcVZxeV}!?elTp3H{-C{n;^Za9%#B!U zm_)Fs9%Jo;P}WV#P`NLk+YRP8>W?v4RV0jW&zwn(*^wr)&$g1&9xtjt(f0uw6Vq zMACpyYnnD&A}n2Gc!k{qVK>L6ZwEcq6Y}NdYnR<4S=L8ZKLV|_ZznNHDa}Pl;sraN zUU@ZBHL>(!sJfR&hyt4juW)1kpnHg3&j&PWq;S}zbGYE8r$l5Bofc~A@?USkerq*( zSDQKrBlGz+R-7q+h3;iQ9#sujgGbqLcu)vzNb)KLh2g7_ycl+3D|FdvPMLN3HAvvbVIC-uzBOUtAU=u=(;8v75pB-K3+G7Hi?xu%s# z2wRV(oCql_r3y~ry4iMVZPUjE{cI#)ei*t5~F|G}>PK|)JZsYzGZZ;2- z8SsEog>fDhQ>+K@s3B#3 z@q)fH|8l?YDy;ea3p%*BfqmKCm`O!RP8$pngdZDZ}?%YN7}F-;V_(Rbw%Rb zWYzye);k4R)@@D0-K8$uwr$(CZQI&iwz11*SC?(u>aup(c6aq(&v{P7`QrWWW~_^K zv0}|R#>mW>i5vQ(6w}gPNY(>{0fw|F?>{|aTF;_pI(1Oqn%7j)8?2A*L;@*)PohIZ zPz3NX)<^h7cjBlk_blz~hR~AnS^?1-FBp!I?gUmQc1D(vQwli@OS~gT#6woXGX136e+J9to$=@ z>(EB=6r00ywcod|rq#ICJ)IZwrc3qsb}~L8hzp{O>C1M^cQ0JSN7RDc!mx*iP%SzS zRWqE&}_s_r=GHy+gsrbPXndUZYm^n^h%{8bJjLsn|Y zgzR8C>4kx$IN~Se8J`Z_!ohe7&qr%qRyY_ace7?I{5QKI^V1 zhGOo^%+D}FF#{b|0E}GjUneZzHB?*Rp)0g z4fsliKkvCf3Y+eVs(Rxxr+yi9#|!s`8>Ghz_n&adFQ43dKMyMxG&5N

3FOo>wIs zN1Xp*((x7*B}c546l0}Ek%t5LqnT+xoux)%H`Ay|n~+`UA6VKyn5ZdBlWd!$@bziR zHd0qXv8GT*U+TVXoAS(mQsRiLa>pk6jC_TC?Tny^Unn7JBfpV^+N z=oA-JR%}D_f@~b+?+Kx*ZzCixZE`QJBIVflE8-LvrPuCrzUKD{|8)t_c8~t`?OaI> zZPC`M{hr+c+3tQQ8+eMb-o}d+k$ya(dx5fOoseMY-MGzf(Q#?tPe-QxhuIsH8d3NyS5`=lSPh8p-_jKRv{;hRUY*R$( z*0VCb894xp-j(P#7nT7)mnh(?$9Pb5Z%W>%9wKB#Zjbn=c_co|zCsrPu;sNuM;U&i zRy&o7ej>#(kCDtGPku@4S_MTi%uxG;`W@VKHBRUQ%K8+8#pHwRBD{# zk$JNEX&6)rc_dsTYo1}*-vNz?Z7gihgYuJ)2Jb!{R zj*u+IZ*Gm-0u5Zi#P{9W)2agZVtIcBQdEwOxoB^;n=9a|8{>BcL}%KE2H{EAZjiT@ z`6EiUmYBTMN<8!sog zi>$8SreyIm;!e$yL+DkNcgDJBY3{cUuhpGDSH@0JUvKkzrP>dBUJ_& z72CZaOd1@4pY0T)pcJfRhG&FfKIk>@pcG)ZqUT{bgk~E~D=P7goSzb$oC4<-YYWPa zlw_1@b$xvSee_V1qo+3IrBuzF zr$=v+>?_psjdBCnb4D9vdPQy(sprB|ys$PMX1E#v-e{ZlLBpR~bI&L53+2ECq@Sjj zm;tsFwjMa>FJk8dorcNC(`>Iu^NbIV6euCD&>p?Qg{Tgpp1s0+5-`SkULF)R7Wd5e zS@1uJ{^0xr2>lQ1`G3*Xf9O#_%nzE*Pu6o6=F1n2FJHdcI57b$4JeEO3@wn0NSM^* z4q|G^$TUsqa|_aXL#EARlbr4BF&kSAYv|r{xBWt*0S{p~m*5qPU>XORDiM3bF8HG^ zOE^~emflZa1)d!*xJcp0HxmT>X0CtF@H}sJ-`^ZDeDQPP3C9Rvz;V8`vbNCc*Grs;uHhV( zZDsz9aAJo2=7!5WM86M)e3I^=D-Y&;%toa3!UZy)!nPbH#WREV;-ldxOpYPcDgexv zU2Wn8^w(D8I;V~ZU~2TyaG?x;gVZ136RNAE1jwGB>R771C7DGla&JR0136dY=W_hf z)j1bO%xT@|*<5qjDS;#xzViAp;?09RRXM*-D%4!vO#4z_*SlIIa~E>vUd_|u3UO*j z>cFe<_2J8Jbm5D+3B;%tLNDb~8vyAPln7O`x%3J!8|S^LXp4ELt5X&-HcM^_es|X6 zP1t&(t;{51Ow>jlKOzM+Xni{_y*uje-y|04PNEteMZNS5FNX@T4M#I_wj=ixV&xLq zLjP0=zvOs_+hBQ!<$LA^^O*RO)-qY@C~XEZxkv+GI~&=QY`hfL9Ng^B8vwB4TsmtF zIWa%2a_&9!OBb)sZ}Q=WR1>&rw2STO4pU~^RLr~y6+)-jzEdn!GdXJaVv%~spk@0n zR9Pvtb}0^|*HLu`cNnx44zNB3_=|-muW>q7)UgOmA4k>s`B=SQRE5~POc1}PvC5Fv;G|*{_Pwx z3G|GI>>v39+7s{$IO8mEmG73K;H2 z{<=7P?RQnw2;=s2N*wyKmNvfBD5v_P=#=s|A!BIxFwxst?`-B z48jll9z}!U3s*q4=G#3()}Yf4kn`&&1R*<#;%F^Iu6RKlGX_%FM??AQ#Owcoy8qwv>whAXf0Z8>x6tR{ zi2uOe_oCtKke`}QLNK6-F(L_&cpC!}*zLuG&`RVBPW1IZOek|~R=@S<9V#X2e=TE# zm0;k3yn4K_zz<{r_*Nxf_F&k5-zyjhuK4vo@7*gRx-9;bw>%&K=Sc|J09txpcv=`A z@f12_KFMv0x@SW$@htK5?V2ufZnqR14Y+vtYvQuedej7yXm-i-V@Z;QrySkFlMKoQ z!h@xWP@+0IKMnh4Na3iN_#cpNhJ<#*R%+-cS`*6R_ahn0k=Fgr`Q8Jc1RHkVuk-)H z8bkLckc6VSY=)(%w7CS*07hZ_peR8V#x&7tF4z^Yl@fb+6{GW%vQUa}ue$gNd$Eb5 z2>d9cDc-XBf(#36pzEL30(1vFB(rc&!9nlh=jTvQFn_~?{tOtgI}5+aL4}6_7W*P^ zoN%x)J{-W!271kQ3vR9@T^OUUqvS?#hbC3WkcwVR(1ld3Cw(7X0W-845!DosZWH$x zwj<599GzvzxtR>|(+793Q8wktPxd=PDU3SX!FF1U!xO;<@<4w8jev~n>XTerm`T`N zp}hzReoRj6mMrQN?UxPRrS7&VFZcD1x7{EPA+>#`CqJJS)9l7_Fvm6OX+{k?!*hm)$H6G^`wcc>bPc1vQ8Xr!Ocf3%UvtJ{70!;-eKnN!gKd>w#!N`^SrX0P6 z;w@VuMrR>1=W2dN{I8VfIJHi*YH^=(QU12xrJBcNb+w$82+PIBNUc4_?u5SwqU+KMk%C~C6?alKSW!I&#(qxu0v$HYLVlP}RxKKiJt>@Gx4S*uj$ zy@}wW;bF1RI2f#y2gFab@LO+BbQj$o#h+N$Mjf-w)pcl>i;f(^;wRC?6<_KZt+(lxEGP>)=9wI;_)$cD{%eaf_y|m^t?|qR2AF!b@6Fw{eWnodX|ewRKJR$*G0GW16k`y3%8&#MV7ddA61T zlR&n4R|LJ{F#=Eowoy9qQST{(JmU;mW&AI?$R0Q|ochxxR#2_!qDIrSM+V*MLn~m; z8moR9tBz{9IhMhUueRV1Kghf_pkjr@m#L{jj_5aAW=C)}V4$T{6OGVZwQnE(86w3G z)eA)PuuBhhLyz}?ulK+Y$1}NPI0U>&W~7z5B(V)dkvb^_cKD#X7rJr((4$LE-K2J> zA|;x$ft!p4VLKzp8Vn*n3YuP%i2^8kG(`?wN252TKTtN(UEZ_b9LeW3<}1a z)(&k>PV5S~z0g5_{w$@#u~V$2Dz>-;-}dk2Vzob@JBq^L=28B_qiQ^mPPttKW_3UZ(hzolfD> z5Q+LZ7YseUzVq>l+X!79EpiQTWSj56Z5~Rshch03*eG64=Iq3D4JlyEtoiZ3VvU{& zWoV(GYfxdT3tdy7^C!np+g$B7`iASh)G9+iyUKo}2_V||`|Y36;D06T|BeQ&$Hy4D zkpHQTWeV!Iho34TawuT!Iyfm%_=X1-SU^Vz-J1A8PYC{RS>!>7Ir*>rGGG(26#PG8 zP1c}QTcyv+>jh*n#KQ*8#evfSA+LFeTa&P0Z2m30gfD-NG(MT?3sj(q8_a)SUiXFp z4dlmxK>>W~FA4w3qp_0Yre}%j$=MYC!WG?E>TxgJz!jx+2bbv6L?1SB4YHw1;cmBy z(OX%5Y)9H!z8ZY|v;F9?-qDx$m~-jm8xKen`my8322Uc-O`S6saQOZzD7bU|y#4z0 z{`U`2ABjIozaaM`b?{0tI0XjQd7$&(p?(Ss;2@xlEtuJI=;IbTFpkcIAEn>vriA;| z0b5DphCQ$xl*ybIN35If&nP;0Z~7PA9?fY%zn(PH_{5ZeoP{}+0n1LLpyZAUa0x;-`_&KVv)k$ik?Ndy%7W)@Ga z(GmcS!ci9{LZN;#N+j3qXw?7~H+&%^)10=OnxN?sUk#Zdgnlbe7T#^Nsa)DOAYCDg za}qH-Atll^t0cy?5_hpf&0N3#iFJna9_E=oiFP)$mg}xPc$j+<^-U!B@L|%zpm4v; z;SLQJaU>nDruxqPUFVJ@Z8kYN9KM}Xc>(@ zp@88;6DiNK-HWV3{7dn-cr=Dof5t=Wc829h*jyyovtTQ6n;*=Z$Ap7!q+}76m&rW~PkK;ZwPS!2bumpM z3Bj^qTP+boq9MN2e75;gm{2Py6L>R7_pPz% z&G%k2ohQ_`%AhcB>WU(M85x#{i~boZ2L4zszF#pD)ME3aM+gy`iNoDQdKj>QyWMYq z8^6aG4e+Rcu9L~-ggaAAI_qL9E<}?;ztAr+E-=>#@lj(w@efh#yLii=2G_{BZj<^>Ci=1#&RAbq&Zo$gdV?UT}{rO{8PyUEnNwY6?Ti(ubO zXv9T?18}~(w+0J<2J9@{`v48qMWamf@=QUQbcebK-354*<&jq-Ozil>C&2KBTPffkm?P=(;O{w^ZaOgbs3;zmeja+yl z;$IU8iEdxzAp)SYHj?R9&MNW> zKj$nTEi0r7p!ES*n)5GbMU0b31sl~v+7yB2>*GeKO5U#mldQMS z6VG&7iPN3EEgbm?l>lSBfaFW?{*w*oPHW1yf=$MbxT6YdgshMvmIHtFnRD^`$;=6Z zncO>nDjD}ZZkuq##{7Yw~|ZKqL(jFcs-)t?7Bh;W7BxrJ1pNA zI!qAp*tq71xw41CHCwwV#fyvQr|+%1aIP~X7JF(@;;*v@(*fQa^eu)|XkT**>e29Q zVpGuk>Iqp~d%^Xn+caStr<3T&aK6l`gLzAN~4gUpybdyQDVJAmYlCIL# z>1+N`fEB>_0s&Zqfj)g<1YLAQ#7ir6z@#MOwu}Xg-Sd$zinNj|rqp<-u0RY^x;H~h zoz?clGjtA}o#_Q{ITX+99oPRkB2B|9z3CP=Ve(AOl3-)V%fuvA%{Wb`w)n={AYNg3 z+DimZ2pKIUve--u3*}G4Eq1Dd==kvymj41A5hMuTz6&5@d@b~L#++XMol7ci@`^rTyl#V=I@%d*;hq?h1U%@7Bp8)`!L{(Z+pgj=4AuJl*fPD#F_d0#cKV2HJr zk|OThRB6DY1k(ue*)6gQ%>kZ1_?E@JO&T}6svjfyLme=lr%B`Z9yUsF@0>xPuqL>; z1}v?nWa}8T@# z*s14T)cG;hTrgW3)(B?@nwow1!PEE)-NTVVWy3CdefGh&sBB`^PoT@_18fsD?hn?b zQ4t9)@c!+*VIC8&GR@e*T{RJk@Ezr!Cg6W1`2RKm>#duNgzf*mO3_UWa{~XLhQLIF z12y0iHhhKyIxT?f0;)Cr)c(0Av4yOiOb;1S(A2DeLPstvTh)9xk1A@Nc$27{Xd9C~ zO`)Uh$<~P{JL&Qt4iJ5SrT9Z3WH#RMyF)_%Tu+jn?S*9%5*&LlmeNeEhbM;)7r`$LU z-E&lb^USW5b!M*zp4o+6t@pU2jwS|S^Xvo zQCBdEk&UJ=U6$F7`xqmkdz#1RY+O3Ub;pNq%T)BK0q4G1-#roxhp5k~&psMdT^mKx zXidWO$FnTt4&kz-=d5E;K4lo=ZLu0SL}tPx2=KDnDYpo}D9kPFcUcZ!O&9suN!T4_ zSCelUTE^EKA%sWJ{HgZ+_|tCT`B29KF??|J3%pbkl~W5%ltu*eSoMYCK8kAYgNxK& z@D1ho0fXv;zam#98h;*>NgO6hx#r-Z46>7UF8O&T9g{J&k@3&tXfkq&!kfjxI?az| zT993%P~ORoqdat%8~*aqjM$>qJ-WAjZb66vb*<^zut{$V4leH{g(HY~@c3=Z3?Wx2 z$cCSeY;i5G#)mbiwTJ_Mzx1Tk@453%l6U;t1PEra%L?Yt+ehFHTtD)fO~#j{l!oiJ z(6dr~NGDp`rriS4HA&~Xt91BvgyyJAoREI}Dz~V$TnSEQN0R0B`f>O$q(Rw18j#gn zO8LsSF@{&D_Te9WdM~-GQQl8dhz*r>5w;kW_G-^tpSmtn!&qyfibn%=2xj}U z5}<(o7??CN%Gs(v6kdt}y9r92o=MgO@e-`Uiz)EBO&61@BzeuxO%H&+m2vy1UGx9W zN>$8N90&1g4T-F183pK!F2n|J&{e`kk&wEVpO^CymdfT>Ks3+)WvxLzq7{L=nFog` zrZS#it-bjpZY1b0dQJg+g8eOyLp;Ut1~{u#-RG4Fhl8n%CFCWyQ5y3P_XOH9wvnY371~9f_Og-NcqS&TU~rZ zlCduLa(qyNFcgjdy!eOCEBRO0SVG$+c1~`Sx7q3&ONqXAHwO^qR2iG~@~SIn z-R3(lBs>C34EU;en+U?DB%(|t`0=dB!MC)mKi{kTod_STKs7WEG%x~sY zZT3`;@NF|9FJ(~X{Vy3wkZ}Fk(}T#f;g;Xu>z%*vvm4jFziGc`-^9mu&&AxL;P3CX z0YWQ6WnjWXGvHx-Q9%4Vbj7%3u@Lkh*{OT|siB`I;BdxpxkXN20PBQvuY3j4*lP4Q z3jU${!LC0znR7yu^n^*b)P!{X`!D)1d3`DVZ$2%*-$u|x#J1B{r-!|Sm zt_U5kPcfYFFOjJ^O>$dv0Q#F@w8bBA<{&1z`DB+AMpRrl0J*N0F$ zN~^0@TR5UU71gJjI4CVWYjb8Wi_#V@6vzsYMSd!f1-=x?k0ev$R@T zpOEa20t6FCqG1MDU17pbK&fyFVhVrNH0_)&u|tGo#gRSZV?6(oKHFfXbD{n;uE`jy zhgla{rPL*;i^Im6&4o1v^Y$|xh~tdF%(+$~X#=Ej^Jz!akL0X75Ca<+NksF)g5>ZL zC2ZE8mdkB0Y=Rp2W7q1XJ=BPrJEU<6loJeeHtgrer1o$VsN4ku5E%|l!t%NZ_X}>) zo2c~DwUO^>K-TUxgsx!xM(tI-vFj2EA5+Uj?}~z);DFS+Ne|xHbwK5o2MyCR0lzu9 zN(kJPS!4Xhgo72uhc`H}^Z`)K85W4~2zroJHiWot={63~(B+csJ;Q!{7ou&$d!|+s)=YU>xK6?+UIRrKt>M@-O)39%*#jiC;!!>kNzEncAxRed=ZojK zqh=|Q3%R%hb0{Vgz9d`_tH8X%wRtM+#!JcciSLfJQlR2y?N3Lj&v2n>nK=x@=h%6& z%naf%?Y}`K7I=M!rB21&4>%`7l!2d_OZ|VoP9r*^0IV6q3!eAP*ssa~TTRymI!{c( zFReKd`>_Gfnz&rZumI(bx^KTTpIzRxay#NY-M>6V&xcJ8UlR(m2?#e`gG?14h6Y`t zdKf8$H;(}@Gjwg;uO4lRg;A8xH1fL{HWe$NC{wjTu0;Qiqr1roZ~Y}US6*;)LBR8;ws*P1pz8y4whj|xLr34V(+m+0dWq#*)YN((;BCg_o>X5%(7azz|J?9SM3I-hzyWB-?WorDr zi{7{7S+`!6YJm0Q5H3gb^N1{QH5I?;BOWe_AqN1>@x!ydCu~v&#cN7vkoKT=%5(9Q zJ8CVXc#kQlQLpqxOG9@@o|RR#42_Hcy%1+^k}gY}xx^UP4Ql&pO=UF^n(Z)4yH2`& zHJcF8Xph~g!j^duvX!Zaq;0tq5rktB`J_K=MDuBoxB4ASd0+G8ig(6vZYZG_1Xun@UVaEBLdr=&Fd{;L1RCL} zBYzS~O?($JNiCdZHfD-|3ar*E>csf(YY|rme&Itgxea3`nX97qp`9gv9Infp(Y&81 z{G6&GaZc|*Qo67#jdo}=oHTGwbY-$`RdjdWh!DoNc7j2X1hW@>PPRJjuNTEGo0gon zPXA0||La2jXQvM!QiQ7pgfd8$SN*WPuSi3-D@)L>5ZUw1^$Ma^-R={=p^XAo`#a^Ko zg6~YbVlXm9rhwf*{!kR5ILSZOA(STa18qd8>^L;@N|BO4lSskK2rMjfbMp+-c8aW@ z)|3)#6ejBl(4Vjk{hO3IKka^YAb(X?@Q9sjOgUu_t6{UryCL6-(uWezL+i8!Q#?AyAn3rdsnJC4G_g^7nzcN=YwO4} zxqHQu3}{lS(^`Cux)@@f%~6R^BZqm$h2ikfoVEt>QMv0CII?`ofB6Nk4myy|=9O9k|~IYddPK=B7??oT&)a&LfCT zgpWSrc-Fc!g#axZ`(<)Sta%?Ui@RjQLN}3 zV>Na?vCZP48fU@4fWcRmRcsDZY0oHbOW7U!t4H;bQ7TpSF-f~OZT=x?Mped(s+3j2 z*c~Ul%e`Zv%Vj%&wjf$mlKVxo!1M0GLFf8 zYa8)_?THjXt)cZssHHm@5}9{ru{Rp#2IR@!qHuT+vnA)&RV&^!Su8ena3g*-y_M3OVfN0Y-+wL z#<*-#>NmVlsZ)yObtbyhS4|r#TRA0k^JXr=6N90;j-U@7$UwL53d>InAB<^c2UH*) zr7Kk|imA0|i4A61$i7t8`NyNXR>h!W$2*|ELMmxpfj&7U)4i*i}ZHe;Y>iLLJ6 zEfW8a%aKM412tG@9v*&4nK5Tez}*<%&{UUGOIwsUBF>+ z>cgJb!JHoUfEEquWL!`%$uDo7FN)B_kR$hq*MO#ax>Ta2R6MnqFYW&A=mn6S&M=5F z@35f%LmD@!84;a{YdwQki&swhdRoi%=9)gODK$EMGDvMUa)AsCG^Q&FNO4@mZ9z9w zh_~T2NDIZ$;N`HuY_GG23(~E>xSg=|#X*OLCT1L4nXNYUK*jj6!Q0T1jG7I%bbn=K zqt2nU+S;Ml+9`UVm3SNPe9m3Y4#N{aB;`&v=%=1ii-zGUQ^sMnte&oJ?#r5y!)zdx z?Kt!qTZSN?U@eQiDJGN#SnsQNCZ@mQev`q(d=A~Zo({O44Ded*j(9k|D)tl#@R*-v zmJIM=Kj?&OCspG7SOvu)Ly8;MWww|V_jamSR{NrXVLR?iv(#8tG4vkxT3`8SL4%eD9n|Dsa{G|3wd2zsKkt zXg2jn^Wbp_tJZL`V<&w}A%)m_r^UxfPss8VWa~XE=Fi<2Zo2J zy5FeP$gUz=O;9?W>XwxSA}*KWQKt6A>2qDjgeev(Mj&DJpb+Zn!c?2n7OK(I(8FS@!^2;ux`yvlv|_A>_aQui!xdOLkH}9J*EOlk<$WB7X-r7! z8^x@oLDpz(D~yR2?!K`nC?kseAt{DCfX_bT8}$F0i~kFowCc~ptp1nRt^EV80r4MR zw|_q3&EpeHAp{DI6LJB+U%{XPwUm&k0B9n}f$4<$e)!6;v=aUz%6h_z$buQDU^G&) zKZRYtlLQiPn3aAT&24v7`I{z8Dp|bOC^W(C=YURnC$lQ&>FAl4>0|kHI6BL;^HtuI z77`%=o?lW?Dj^;PoKkX`JzYFPWP~NmJ6XInd7qa)+}$S7r=MoJx2`oE-)hUy7%+wz zpajt{6mqGXb`2xtN^=3OYsw4^78aMtU$HG3>6#^OBDP2cIbfA4A8aHTje^U4Y~?1zd56D7<^a z#j=|viMkRfZs zbc22GJ<$PFK8IS1i{o&>2GCV1r178R@fVu51P|N5j=Wsb$kGUfAwL0sNSY5qYa%cN z)$HdL#lN#yBFhx?AhwRsaQH*~D|tXtW2pGA`e7CZV(2Rb`#%npotg~2Q2#E0a1giPh~l45 z(E`;LcRx4GH6+j_k&*z|`v(RtU5En-s5c7^51bvuga>Tjz!o{iI}~JBEcyh|L}Eul z!)c(kAeERd<5J5pP)-Ie^vKH}HCV*tKNqKWA4_9SPbJ&PY*Wqpq`lo zvfM6_I}bb1>=^2X&(m0l1P_0bcLbuNH`b?Ym;(~U#Jv_$ZBH)+)?M?@(s=V32;(=Y z5>iPg<}x`L$h2y&aGqLvvt|( zxS{P@MrIRp4Pu+0wDC}FMZ1<-s$_TXjWrV-Jjcg1h!xBK^^$^jTiFS1v#Q?^OwP_7 zLRE8*@H5>c&m#Kajc1%c?+UX* zWeT4#))D3u_CUh#cxE+Es@N@P^BYc=iNqyf65$V8!&w@v5(;MEIbh{JraRYMCB`9CM|07FaDpeI=aZEh*%ECv z?xpmCo_1-aT2m#5sm*~kxt3cfbs2=BnXrAt6~WP5KK5^fe+tb1{{R2C!2Gi_j-R%p z0H0g3mB$Wg4jkg2N394>NE=Z9v1QwI-Kt*y+=oTb!0$4!G(h2MBuF4*93N8ak1N6l zh<{C6zazk2LI1m?a-{V8ua)aOB`i7EzfWxiAjnbwU5-a11%v;)`w%*U13o)QL~R)VzEj4r*<)*4_A2ZY2*H-{+VIDl1eNjk_HtGIZ7Z##FqWu(rwxyQarAy$=yag=dD2D&O zEadX)?vT9xVae$-YmLh;J|xczg-}-#lVO!H8t45yCY7youISPV-D)(Rx0{*f*21{P zp0#snY&QOCSwUj{R&=(HTTHOgo@TbVKhGS=KBwT{=HLyN2yic6XCS!O{l@j)&GX%n z;I{qow9^Y<1TlVLJev`#V}5_&6Sdp@%0IS32$~**_1tUqe~1K6xcgvHtlJ11y@klX zO1dStI;X+$!;l|@!clU&urmLxIC5>lKUwfglC(pI;Fl?j0OdYU_)lNCM)!4?kl106b;vEdKI&D9QNgfTdPd1;-k^>H?ZYK_|Y*E}8a8ATF$ zQu$;Fg>V=T2=$gC2HXT~yU77NDqs?$l`TsZWn64LPcVb8vjzS31hWtKidJ>~!w+vY zk2`b{ZoKv|++R&e5I@uV7o}<<&K{&2mj@E_K@80q+2}nT`9T+zl0$W6EUUkVW3Vrl z-TCGL%Qz?jj$O?vTzE|*%+&`g5+S^-TJGy&K8xt~d z*h{l$`Q48j`Qr7OX>> zCT44_^O|mP^-rVs-=KLh2Ux7%;FTQYL*~|ioAOjN@=NgtW|f#gO;N4oPim=EPfgW5 z?zW6_!DQj$G6 z3(loT_cZC3tF2q}P+$gPWY;@lUQSvMdZPnEJ!N zj;$g-pcZ%R`ZH(slZ@a*exu5Sk?8`##oeN~Dd-$C9$2xp8`1cTSRLJ==*o&wRabwE z+FcDn9=(zLmxEba!$BkS-x|X^H@rIln7cBOVY~_r7PwTyky7q_Jfw(YDFoE53oU|j zfZDL0x8AUQj^x(@5J6W}5u%&=Z-VPeZz7_^D-HihBz6DL1RO=e@Gmqgr8<8-;3G!K zeEsljl#yP=DIai1CTt~B3bScHf|S(#CzyVhN6+5y42qMHCdV?dq0$DRi3$P$M4F{e z3qvr4rmo%ca+NPwJ&Cua(X}@mT7jt7efWG3po7PzphO4-@DY zvGX}Hw2dsC6Ah8U7Hm3}h-o~4Oi{WiiRdYH{)B${BpXbQN(Rl$34Pj06GvFyv_sZ} zi96@77Uv-rgm;wbv(DlZP|bUF+dso3mKE-UQ9=&z)8cj!^*(yB$itIK5O2L5BWFHY zCAVnbJCZKEcr)%(<$OH&FflAT*|-p_lD$gO{h)s5hZ|R-K@%}@A*g3S-vw8nJ_3WK zy9cS;G3Wfdbb?MZsS>1*!5tP+SH}B-*T#lkleXL%j@49AjS-#%4t9tNifW*^kpYwG zcqIFM_5LJI^q=Ms!YeBI0(>LIFA{@=#i2FBsvQjE+u_~<$QGBI>){p}9``?#ihrCb z(cwVIWfU2EXR&Ul!jHlON}Iq5^UrM6?Brb7R)h}tj_2~qmL(pm5M%n#KiPSI^ro=MfSoAq+7Q8mcJrFHXUs2=^Eq|9Y!m< z3)hTt*VEPHF50s(C8o`Bnr1%owtHdj9Y|OD)Fzmq?~VQ(3O=I%z)>bbwJiK*q_$M@ zAn)OTT%7A;e99!482yqjb5xtpQ5+5`vXp%6;%&N}JlILd53#5o5~ixdi`6D&YO=})o_$pmS`&kq zB)T}o@)Mr~vW>X;6X?V1BVv>EYoE%Aeony~Jxw z=mDJA0Bco0-f7B3-ufGRR701wWDbR~1XiYQ1 zNcIt4gLyEhVc-s6=IX)BFauInAo}1vXDg~b#*o}ELBYb~846?PBAS7oSO<*Uyv9~r z&aU3s-)E^*0lWI3v(Xz>A>88_JAat(A9k?Li0LpNa6@6J>5JZAC5Ew~40=!k7Zlr% zt|vqK?yR_wPW|Lg&uGjMj7aWcF!m8uXtv{o2mIf2C|<7|53XQS-TN1#x`a;&$9{sl z+92KWv5*cwaI+FP)6^TnV98|1Oh#4Hjo0Bhv z4KFZGceL0BWXse|8LErAvXg{!NW#~_i;UoBU1+()###N^47Y9ORQ}pN(1dJDAUB9y zJ^TW&!IJdp+DP0Yv0)dy5PzqBmyYf6MFVy_%Lrx$Ofjxw>m^-IKONLOQ%};FJ9!cc z8W`;r03KZgxQhl(Jsg8T9!KwvB1zny=D)E4ZZ{)QgPpe%L%Dr-A7LtD z&o$yT2I}LVeguFB2WiGW0viC|UEI{zibAHL0p0|;)QH}>xzG?@vQ)BQhRtZD;(SSq zSHdn~pXsJyDeK9wVe1R9X;mxd2WJ+j)v*;*CTWhf`-qP3k|+w_BR=55oW2Z~x`VW9 z7s483y}msp=Vpl0gg-Q*jjk!5QYP}>b=`q^1$DxWx=oB5a2EnPy5KNk_>^f&cv$!f z0Z>w+2!shIjr`z0_dGoJN`@hMl!O_^Xq^3)33K@C`?}H6P%+V3$E~TIg}w^)n#1-J zsbo$vQIB1YJbP;RzT~mhYwfNr2MxBN7jxTx(?>F1Pj>XrPj{ocgy75#S@YVGVgckf z3G0Se78%J*Xj3-dns3b5hUYP5?N`)79ZJo8z)Oc6(1*Fy z;@iU()~X{5!0lY{7zZxfvfdclVmzw^az2o12^&x_%9htGY4?&C5u(!l` z2@$K}{8JP4F$%+EKmR|n-U2wTUD+CrnVBJGW@d<)nVC6e#xXO-FlJ_one8}cW@cu_ zn1AQo^WXPgeO0O&%~nf$R8mW$)veXIadlcA(c{1EI0(I$Nce^)dQ>V=ytUx6jg)q6m>&)MQ{ zBD_p|5=>hAqsbHbZ0tiL*JCjQ^G7#5l(A6dx`++WKnm`vssZ{%QpG?9w~H9Tmo|;f ze$ah}=T7=GjZAOo`%FKb%t|-7Hb%Fwh)2mzbw`ZYS0Z6_XDp3t zbb(oChtA3rutRseU(p9MGs2snv>$J`Q8;o9)>0G$+P{}3wXy)jvO*8)XT^-!5v7cn zrl|xJ^DS;@H1v(&YHKmPnpAE!A{Cr>CJAJbuvaJ7$`2r&N54~bJQTz;=gg+-6!>Y{DY(Y_aK4&zmGY5f2sErVv&BK{d1<_ zo&h06{5l?#`U*63qCiMC;6VhosDYDwvRuw8V*5>SP|cA*=Mx3GuZL=jUUOu% z!SH-V?w5)8IBrwXprW_pg?>i^1mwt}$B4gP#IXpRE5f(*VGCwHb~FoIHXg1%Dx7{1 zXRgXeTXs9^tBFkH!bq4!nN~XyC+Wq3D4?0x$&6&q)Nl~`Erx;1Ks`j=LNe7x)tjyX zz*y4A=1f6YXI=xgG15z<=Lz7fI^DGyx4WfcVasr?o#dS#{X8S@)J!}ORx}u6|KPio zBvQL8SI({f-tJW1rJ3*!RuLJOjm(}(#b_K6{P~j8|IAEF6TvLc?k$=k zH(6KXS!@s*RBExT?)B;$!4XNV%xh9wPzFvyAoyn!fBoVNuwvoi~cXUzu zy}GGvDLW~SHJ~Ta8jo_^MiTUEokfMSN}34kLpDb}3=oTKx`PM^00?yU zwZ4{6Az4f8Zoh!)ar_e6+wTd1I=FTPgb|x{{>qV}uxv81A(^TIwR;;o>M_<95?&%D=ldNbv!DQRrDb{DK@sCJt@YeMb z64g;|l3sJTosh5NWMt<ibDb$2Vfszd&8C!z+0lrqf;tnDG zB!0r})Y_oVPI+U5T%3A_q^!G)-%?UGnBvI5e-D31V=s-{%-wZ5VQpAOI{NdRV);h5 zPiI25I~JN7pI)e@tv|z|U=yRz5L4l>BRk1_gT3P5&s~y_@3)^hNQ@urJh2NMJ3v3Y zi`_~Vn`kdm2T=bV;if&~$i&GKjoL{6J6l(r(2>W}AmkbBFMjYpI`jW1md6JH7}!6G z6@KDmy!3h7h#u_k9bPWrZ%{%Z0Ja~f@j^LxC(1Buc60mXc(l3S`@0|H7V`>7wMiKw z;W&>NC%*23<7|MAfZdCxkq7Sm<|rS|{U5R)6f-1~f}*6=D&*gjQdPX-s!Vt5 zvev5|&J&1?vLLcH2o`E45rMHNab10^kT@NSPP5*$u!Kxs$@*Oy4J&5wFlJC}_gXM! z5Nk9%n<@BfRmi$B->B%o6yQ&Wxjh%A)Wnsgxn5|@$CfU(@O@H68LeUcyujJSfS>rR z7yD6%nFkl|EGI{DdWG>t1N27js^_w1k!!{DbFKI+LXTK5BXQveejf7(I!Kkhp@m=| zBbBe4!4oZ?{UvD!CqyM`CHfVbbfGhbOlfWeJFZm-Izysxm=BH3-t;&?Yfl_;Ih2^?&RS|1)y`dw6;NVnHam6m$>>yr0>4Wi3GCCx#jZ zoJJ)7B=CpiA!`Mz;n2q4L zeKhg-*_an=4i*cIB!>OfJze#)QKg~{uDj}Hg z3^3=B24jX!fPq@IRPRFCmGc4;c~D!5jHCzI%w5Ke<%yedd=962bPk-+qC@vc%$epkRVeC|OfeVNp7pzBxvcE~gf)p_R zN`A;rJ7h`)(Q-aC|*Nt(VZ};Q8PrV+i=F9|M z($$!|rXE{?WILP*|FORZoWo>l5+)=cxq~)*i?| zi+EXkB+S+)F>wURUt(ec;4>>eJP;xd7737TK!KL(!xSum8v1C83d1NtgglH!BBe+X z{=3mM^+^(v0xeS(IccHMY~EVOy{ZjD*K=8nw#x$?9Amw@eZEQO%x1l*eckrcLZh>+ z*UT8z_~HDE(Et+q{SSNh(**zH#;w3(Hr~dc-ZEf>B|-BaD#@Q%%EX<-ON*U%2r+=J zV^XZQMDb138Lb$B^8(J}RsHAK=wB7nZ~D*KxtoinlQJK4M>-H+K6VNSr;s}DNb_DR z6yL`sY>oy6R=)l8KHg-1Mk0I*_B$zvxXWRAVJSpw)d>!0uo?`S>;Tn(PlSc=^k59OFkBgw%289W!BbmnROGIwFmcGFe#4 zt5q=p4%o%;YYypyr2B({2!MouzJf-AN24;~8gmonV%KS5eTTo>t__3;;gH`&eTv0m z5;~w5nUKN-dbP^(-SB(?dVAIl5XY2Q@~Q1w;VLY8gbuKF@{LO9U15i%y#nadq7!T- z2VY)d{3r}eKG7K+8Hw2w?8J>W>MnMD{&<<9ic^)=)PxG7^eN57RzJ1d=SDd>+C}$D zZ9<(|YuCZ&hqfk)VrI?^-kEDzcUL-~oLO6KK|@}3V_9dd-TiZu+WA*nh7HxJ8G_SN z8+*&ZK1rMMpFAdePOEZd5r8gPW1d41oS}%r&Ns*; zc~HEpXnTXk+A7Ax(^|KuP-^VEgYgz`Ipg1Mq}h#@48PMNoBHbC1|oIsqBJ2lLvLNa zA4OQ03{-)MQy9&u?rBlkPu~+=+4I0x^j?8nYc` zj`@WTC@vozv9rPvY6G+jVoe4X%p!a9Ik+}sYoU+!qh+k_=+M<>$I$4?&h}#6mSR(* z=t=E|cNc;3U(@R(RSw8->2cHmBZuIg2J%FWVZ&&v>Go62^^b+^%YRKvmUtPdwB&(u z_rtu}Fi|S6Q~xP!SsfyF66Yw|NhV+#>zCx2{Gm#bToD53#Q<;yrk_d?pRZ{ZY<7zS z7~0>wq_PkT>1RH6g(-iiNDk|6nvk6IQ`h&%di}y}Krrj%jz%c* zt5s`=&taWK+#V^=azi(T^h!>9Yk_yLC!!BavKy;FmoSd?Z&hZ!*3BOznM5>5#|Jhl z$<9eg7v~d4U;;E&ZL`Xs@gqDz4@8>Edl4OBb$>!CR#0WjM~S_aTRLtdj0>&S;=s7_ zxV9?+rF!;~(wf6Dlc;6y?(G6@$MP3?Nt5Ch#{?v=(>)QsS4LE^*7;Tqh;y-~u_5PG z^nPCP4{UEvPbh^qP#TfX(IHJcSb~(R_JC5hk|MD!DhDuzzDi+_Q@cXz)wnd39+&7f ztSUS@gyaZF@un3c(r*5c$6LR4(QCrHj~`~YmBH3=Dia7c(G}rirNugluxA&(lDCwS z%;GZ~Kq*#tRu3VcTLrQUb6es^GC-5ZlfJ1PcXf@w<+)hbqeO;5KF53?8Zv*WBYuqu zL;eO6lmLLg-#THg^Yik5pBYW|tKO*~^4tr)oz0~ZJn2b&J~nHME8AnazM92(4$M(* z7#^*E(;dIMxubQg%OSuhAs>#{8|siEbe<;$S#)9Z@XXFRyajcfp7%Q`iZ)adyr0Gt zT$kQjR&dzz_pbuALtEka-2~^>Zs-o(1cq$@1_AIO#=auaKP!B5dHw;kdF==W z3Lm*ThI@o&i!FsUj;fR|cfXx}SMgX+sqASU?Fxtvw8}$r0UL;>_pd;8s&9C-h4{{f z>i3N2HKw2dTK*C06owN7A`h0wq?po9jq#aC&H#>-cr?#c_}0*^1OP>LdO2 z&J$oKo}v)~uESs$lHuCQ{YW>|3odbA1|e_JytKg-F;B6-ALi6Xnn?I#FAm_h!a z%_vn*;cRc zcYJ~niMsat8DB$fRDy`=3&rr>!7Ry&*p3sudFYJ}ZjQ^l=C$|IOLPA;Yii36Bhx<;cySZqw66ND@1}V%X5ZyDEp@G72SyTN0(_zdx_-;sT4XVs(Hjk^ppgfUI z48iRg@<>e_&YcKC-Icboy8tVh7N-Fv$E<%~_zsI_#OGS*&omkU%~ z(si@iUcfykk|0v}?P5Av1f;Y@8M&moG)f(mk6~`2L`!~fC9`!iTt$JceiM+MqVllg z^U*+@4B4iirZARnXh1n*H2g|?W@m^vCUHz)DBp&q8k2cAJSbL>_hfM_Aae`oPq_yv zQc9SyoP)vjt%a)dB!HfJyW`#un4HC;6R?dxK;AsKEt>;_r4|r+?uU1$3f&!(!xZ^V zws(ZLx6-J&xLeUjkjEUv_J9Ff9SuLzGQ>2f#1!cN)a{Ip&r(>9R7a5et7nMvGoq(G zfg6nAIdrlte&KZ(#@d@PYUOz5>{NSMNwWH*rscnjBLTg$A8XK|GXT;Ex zM0wPi@-wbv6Wui&Dqf+)I2B3!$BuDd{1%k+{tGB8y0Z-77fZgN@q4GY;J^aFQBe{H4@IPF4 zPb_OU&1~t2LIsrZND3s!zl3KUX2X=sHKu37H(49Pvx|7V7!+o`WK_b~Ad8D*wWEVC zA4jDVl{BV`&M6J1Fs6tAgPuz$uDtmK5_6}~))STitS7WFn2>Ozh*Pm6hTCv1$wB$= zxgTWX8)Aup5FiF|@a66K5WK$lcddjaZim~sf;Aq5?BKrbA!dr+(q@h!Grg~Ag&kpa zh?t`5VMB3ohw*fwV_CQ&mAL10dm@DA@XrLKjOrdi7yNAGN6R9BBB4tHnh5aqo&8*i zHT=7t$?9Iu_p1$!?ap5% zZeB49GlGhgzH)x`!C(icT=BmmOwQ?U?&S6^-4nbwMcg_>$^GF1_Hp zj1dA4YQR=G8y=Fg!*5h_o3t?msgWjJ!WEWq_vGApYX73E)e9u|swPDy*xDn z94VylGk)ShnDPOe{lb3tfsj07k8-~`O-CgVCJ-F=@h#>KB~PU&rlurpedMEJJE4{h zvhkMbfl(OnDp#3MOzMWz8&)$M+eg8RnsFxiy^hpKGoj{ZqSl!v>}CFV^F%1d7Ntck z3^`R+gD=ZUbyug@ce_$=>t#Rsor6krl7eM|-Mc(L=gT?+5AOBWGF}g_#aL zRVpaiKggYBj5X8@V%!A=wkm6pHw^@h)C6M71qLPnm5#_8E#Ex3Y4m*#-vqI13DAS4 z!Y+?g(~EYYpEwk0csBRsy@jA+<#Ry9M4w1D{hwbcFlbQEoDnFy)ya4qQJy2GRfj-m zc9>OrOsrU-$^r+cbs6$-3oXGEayhakZ9Gt`4wX+1SlW_Dt#}oRCmq5C233wwvZcK} zaruVm(pB<)Dig8~gOd~Wqz2#5t6)0bN}XPc*QS`=c9 zx#|^lzLTrrdkDdP|4F(XnPs2h1Uj6}phW@*Y&VS|=l-dsB-HLhAi;|(f>6jA_q^#F4tJ~ScQZ28^;6?P;b`{UsTPaV@n%zgh z)?Pek14bP|Wn$MY-NhAkyOlhdvR35Aj`lJ5x8)GOUF`8CF(DuG;Rugt8aISS2^RJa zi!>KlYsc2sdna#mgv+wTY~iYrRaTQ=evEb@*2g)S$2?4+U$mVYyv?)^vVnqr7f5rz zb$U0mcDId^0jCnHdw$o#w!<6v{k0dMEvx;AA$GRl`;^dzZOJ4ubjp= zzlEEZ&T{U%Nf8mQ#b*sZ&ouOr+_$bJ7(eBZ{bDf$WxDoK>~sBHhRjCFQW%b_E4&^` zR4zG)wG55k{U(9G>12LOacNze@X4S7?dUP=rrR-F+A|Gnk|gZauYV-S7%(>Kn(88| zSL6heZ{OC$azs$Bi7tD~Km*r?@11~?ETn;anY-UUbS}IL5r<`DPU!X3eXemfX5mn+ zu})N7y<8(z+t=~?L0$;a#E(Shy+aV(!Dc0H`+{R1-tlc-O9H?T%$qFLQFkT&h03k= zu!e8M=G-%%TP*IqLlj@^bbUycOSSUGo7L*;zagVUzH~@D(5i|xo{c)BogbeWq z!7;e{ncS$7bGSL2Zyfu&Pj|L}jeH!YS8d5fx7gyKEVtPcK%jgF%92A+a`aDBLnL7j z#x;}KAC#2=17VuWiPK_tO*<6l&b{g#MNeXQAC&PztF7LY0A{OW_FB7mb?6I7Xq^_B zyV|R7d71)j$OR<`2$6kY3x@I9p_{JzKa5|og*$-*w*tYQg4Jzc(3e8m!*Pelc=hcc z!1I!GS4E{jK&ixB5SaZ`_Awo77!LT^vW zyDI#`WHpIWHJqKVg&Z1A8$^w#6<-$AsSo|RBp0EG{)9GawrP99h-z6VCD-39rV)S@ zosji|M%n)OP*zS5o8j%YUt+np0ryL1^Ghk-vP_+v1gO9f=4Cul8T-KxE?jrWpAoO_ zgwaJoBsRBz33x$tc3Hj1m9zHF3ovi-e(M_P#-A=gojf`XQy{#@{|t#rQsVITEA?=R z*~4cqDV2Fa4CUx-&x|X24 zW6=xq7XXmH;F6t-7pDx9{TdJ`q>TqXFAJ|)&^5LRQkivh;#G4nC2u=+s1_jD3r9 zaU!`9&h&h~j)TVLXF?;~{EC>rzAtlb@!}WEBY?~o-)H4+;ZDp<^_*gcrBBXilkDc8 zY1w`3@Nt|BVrAf7;fz(;ii?KXHF4C?L}y(pR9rGY=GSJ&zBe*$|rn4ixdPOhib2S<2`2 zIdC)}2nN>w$UrqiPqTkAWdD`<`i@nZ|Je>N1dtgG>>rs=KSLp`9uS1T#jZ2YBV?a| zRuS-OhJqKMqk*P@{vJvu3teDH^BWI?uIMw)Yt&*S3AA>QKWw;hG7lLk4)%{!S;Ypd z#zw3P^@{UyJ-zvohmdNscIjN#s)eoxgS(<%`Bm#LWRSr_J@Y{4hfL>(jm>7p!!g2- z_ZwP&@wJ5MOW`jPdZmNRIAAQX?OWZ#VoW^52W=@*jx|y51<}BV zIeAh5LG*D7J4&>5ZFm`F(6)yt5)^5zY7KwN$#n0vX<^*NkJv=aRayGvA4K=k4q5}L z8(5cY1HIuZ;c z8Za2z22tTSvF9J@MnAS6TFCRN{Y4D&T&nIcbrGj-oXMsR?nC+{hiMij3)vI%RhhAR zf9Ij_e`kM+ovr;dH8k6wxQc!~kNx(;hf_&G$yW24&YFcUNr2wy9#p`4CKP8n(R6bj zfTuv&`z#QYc&rs)2Dy-=*p7Y)+hoFtbSb&Sbjoa+{#;|;L=iS?;x)G7%}_~6M@o>W z_yn(snzZb-sZ_P4r@Oa}YSy$;kt3sMc{db$a-cVl_epaL^Au|4 zle;ByrZzPp0v3qcAv0ba7~`RqM{AP^2pnI^-y%KJxMHqmDON!L-p@hsaND|wp=(l> z5C#dk5C9f^(H4ydE6^|3g!02t=C?(QoY_5}?=5=$_lR)Zfbukd{-L93M>f!4S<<*cS) zpC*cwnsJ)Fq0hNkpITPfFRLc!3_#If8Gl`{8?-5`59*4|O?3%CohPhWlr1=da%_;o zr^~M;xMMJ(l%k=Y5jqoQB_ea9f{D<>go(&n4vt#q#sd5ZGERq@vs&VN_`z#X@IZV9 zhO~{HDqgEI(S2NNp}6CgkR)`>mb9t*%X=9k%HNOlK)O~%voJu z;h8DcoeB{@MLi-;70bhrW#So$s>}7>5n>{u4s0n-!3mKwAB3G^M!Rg5s#ezH;^B8i38zGY93;v0MgWym}jV2vI zmPvoYF5R`Q`nOi&uA7L!5f_kyB)ns`OKz#3{*IO4)_$arF?8uf!*nQEn&c?C zGS{=boR0KIs}9hAe{A8!ySNcza$Y!MHJMxO3nL)mV`HcZJKxVE2 zSSD2s?VnvajQ0*ct!9#4qY35WZsq2z5BWwEHN59Zzu2_y7O$6+gx?J zS;@Y0>%syj7K*HjIcLb~f`oSwa(1CUFs4Di$?2!KjqB>Jl|2Zmvv@jD8FTQ>rT+G4 zDZIgdoMCsd(5Dy=9gc?`9hJ7crf8)*6qW>T2>)(j|J^`7UCsZ8t@`iQ@?MVEc=)+H zg<%3g96-MUnW*^5no(BRL14iD7B|bcf}?`{C+ph}{uSguS;Ry3S!i&ZzXBb%6yL!9 z@n!dmE>83Pw37A#^9Vq&fkc%cXw6>O$l0I>^`C-)ahGU+sXSAFS1#<>DI5&AK;B0R zB7naWMtOs(l1vo35(>yqS8Oxn@-;|&S7A+QBDqwu{_TP*=2R36LVVwQ5$^2uBxMY< zJK}bW$!ur8!@8%3w^@rXq*S7LJJ#ULrhP2PVwkPikdAV6B1$v+A#9afUhpWyJ(|B`U_8QPhXXKI=ch7cl|rL zp*EwL8u&|sUdw|=LyKo5I_6;52cF;B=I~LI>LWI5T2l+M40ZLcIhAKQ&i<^xF8Ze?FgzC;`Os z5XPBs+D=ckmR!!7NkGyZH5X~}1+z-D6`|H6tDS+UFlNO?K^ES){id_)JQPZk?PUqmcsRsn0**)>dc|MumftchRXeV zYbuT~Z!{M{kV$W&Os1d%3_mBt`T<~ks#oDgX8^omHTv z-KYK|#aB-O(;qKk#fGWxLdkv$Uv`Yu$@=?SERm)cBRQBF5*vgug;sRx2m#%7ts>xc zL1EAYl8cDitHEFyV&UV&J&~5?lhabq0-FeWM<9h{OtR9T=tR^3w`#Hp6T-6zzG2-G z!6{Zq2eHVAhEKusbjEcMY;|G{_2k3HH*eONh`Qf?e$sE2i*Jf+nkS+w#Z7hB#7zlt zBzqzCYp}EApe1E8)IsS4i~#iWBRWYIZ0_nfKchj%01KsO`&y|wwa{$uS}F{KKtsgx z0&g%9d>den;YiBvQ1Z9EN)6U<^JOKFc3tIU=L}&Yvr_uW$v8Zs&+{r7Mg{bQIM0<> zBDWzYh_C-~Ao=gahxp&Rhx(}!&aanQ%>jq7w2%mYzoxl!Mpu7o$Rp4wkx&@G!eH{< z$l19_)mGu}-Nq(*esBDS4oM6Q6q*!FZn`dIqTi7DQqz5?s4)gji^&88?&IqOI|kPV z@!|G_(K&BQCudrD#OSE4qvI1369SWCXKU}T_vgI+s3|IvM$!>FJT80Qc}Ch1fw63n zxaKa}?NvtVL1jj<#Eqj_ne%{#EXZEV`3G4I$)}Bm^flKI6ZQ94v(UotD?)biA`!$o zXz$fmgx=Zkg_An8DdgHSw>Z%ydlwq`1mGlwntZ~QAI=tz^B_U zt_n6@iNs3RxPioaje6H~Yh`eNpvA9s@ap_gycuiF^BuObr7TQzIi!GW-3Vp8EA5`c znTNH==2@p;va?~IMae~BoM)0q&L@f)oPo(3^P>tWyH35sD!04mlkCU4p z$Y*%??XCbQ56dl-&s%XxwL?s9UXYODjv;MFe2wNqrC;R2+I0?YpPOpPdgqQi)Oh0q zpEY|?dwV|WS>eAr5~gJ5HIF|FJ3ODKv=P3bU2xz(=kRGn@@rKSOp@_rdFT%(eVr5D zK?oeWiOd_^G5Q9TGOpnMea-TDgLfWbj({hQV^7-I1udy9Oi2LX_{%tptDOtGgjNE| zEJl-}zzZZd0XL|+lFL^FyLdjTPz^bX?pXNxOC^)DzvedU&WmZJQLyO`{D94&ANa(0 ztvjd;UF5@`c2h!L$+f+Eh-eC`R%v(dAkGrkQ1SUX6F z4zwxZBWs3GV>tg$aeRqVj(VdUSM5xkU$|JD>}BaDOiPa{a25XB(>$xi@a zl}b1(u^>?XcAAaRu4oGO-}&)InJCNB`VBJ>^_w_2 zJPfihvS~885|l`w6cczq*O+ckw<*O0NkH3tn~trm65NG)ksW=Y7L(eFbz@^yr;LGtfo+9?(#B;buMPIlkwq6{>tUw%`-|W0O4#HtXv3~oO^0+rRXnzWX}!kU zQb9%UE&yfS_?_!_*70xczXsuXi2{F|XVdaEf+*CqH<^vP&EmNdDJW*c;QY#FEX9s6 ztq#O$k(amj$^r`s^ekAD=GFXqd%-jut)l3)R-4PPr<5h3u;N);=Y;JI=(wSddPX4x( zf>%7Br`UZ59%1D}L8~Ws;ddd2Y#6;1Y#wmL(v(pw5aK8PaL4vO;gn z&j|>#8BLe3zAK_d_x?&>g~it7mN57*P>pbkr*36SKcc5|_&n}!FY!)#{sNQj$+tx`5X1GLA1P^L5ciq2UR_1s5hlwlv|s&$^$sd<6RW!aN>Bv zyg_o=rS5pO=;BpL$a#=6-x#9Pm{nfx!|SJIlgQz$Dw4hUKOu1F)UF4z-;SN;Q-c7X z1fH^y-&z^MzwJoqPOtB(2w@D8R3UGlTolNRLzCM`5#GZsWWF7XaZCrLJe8GW1a_2b zVbz+N7A-GXc$q}6goD|Xn&q_4qrr*NyK^rV?05nV&F3r6@GaJKtaDKgd8Ykuj~XfC znbC|F#30pwp9dqjpnJPyi_CgbO4o)FcL*TiYon*&q@sv$iH7YN@?xz_PC+^ivt!;bw8U>o1?iYrpWCyu z(Vd-Hn=&_KU5RvnVNjD~%`*04h=yCsiT2N+gK4IM>3ka85e#h1L4Iz;e76IH0PsTO zk|iKhyy{nAcvHi1E5w~Ar>pgI?`o%vsG+ z18qUevWTr|TqQ8R7q7$Myu}KLylly~ci8b-voxemp=RTxoJnL*AFusp=qu^Gv2@7p zoeYED(ufSKMOQ0QYizPi^<=J|Sj}odaEC6Axj;hn=|ZlGsK8Cm;BH?<)5N%ly}$Cq zU1x-q$B}Y!9V5Id8@OhmyinI?Cha_H^)gN>k ztn)XN=TM5}jZDnPdDzw$I~VwET7F%KMW~2HsA|S&+G%S2lZPPKop6BklWmhm z+nfEKOIzx2>l6ADxi_$ux|ROWF92V68>Vk8rHUq+<*Y!aHGuR_C>rBV22NsH5j&KA zzDeoTT}=z!#11xA$ycpeT}%Vw1p8eKO8tR~@2J=-5$ZKGUrlTPbl9r4!jOR;L6%$X z9#z=xJKm5JqRGmuNGyLEFSxHncz}X4Tg=|78H0!>i2;P$a+Dk0I*=45k)UrdBH!-< zeh1{#N=27%q@sOO;}DXt??8A3{n+c9yq*l4mFT?aW<-$f#fcm?z|*wj9{Pn_+6~_L z%=x=A^AHBgZ<1mXKv)Q@n5(5Z;#C;0or_rQCcM6se+Bt4+CtAPSNzbBFWn8w1p64* z-b1**^Xj2MX{PRbQp@0=cKtwl-|5S_{uKygRL|KL%7m_c2y@{CwMDN-2N zI0^>G>eA&yfP)uP=fKQ-7Z;i`GnE1z-XPU--b8q64Rs%p;uW*V#`j|V_y1@wY zYVq-!W9R7!Yc%7m8;p@-dB?IDCWeqIW_4fS+SC$wR6v3MWaqx2&2vXj@!+ z@eqB&SxrygUSQ_+C4+3H#I#wNNx?=6tDYWEmiv{K!iwh2=Vb%4i=@U0;m05|U zB148luSz$A}OPh#;?*7%npd^GzgWbndxn;jc#< zi9MO4fGc)yuY&d!Rn<(!MtN?9H9D^Mp>qEZ*b%pYiH@1R+*?YYv3T=QH0{!}^Y10_ zt%~=0_RzdJH<~q^_{Oo}g%oNcD-pOwbpef%GDj+GZO7Zz`_??Iq$-U4?{nO{+8DWg zz?OxC>EyYiBQQ-&Np|~+fP|}%IxR|_lvhD%0M?6iYjjz!wk^t>x!8ua-D3X+E+Cm& zSx9Jsl5^0ji779*scomXo%R~!x0)Ii-8HR6>ubVT9~x}?*r-XiKP|4Y)5+~)pO>yn z#gsgcItLv-tEqb4clXE$UR8qhAImsrk)q=SVKp6?;$Sk zC5`KODiQdchpOhZL`yKU>{R`x?@dHMqgbxw4O{HHm8?;Wl_qE9rACFU7rmIA6AOIlS3}3#1=8=Bpm!5Ps6#_An_}V@yAr?=_E3IKA=c&65oNDZ985 zwJ;tVT&a_1Xo$u9o!z}SzQlvc*BQCD>-n8I-MdZ(Yw7u*Owjds>UIt)dpZ}x;qxM6Kqe=TWGo>ZnkClWPy{c8 zQwr!Z6VK4#GkQ8vog#{sPrkhqw-22Hk#-^SJb&~DW{+@9|AM#!#Jg4m{cwrML6D#4>^OJyZIT68AOkd6R65zgcQ3Ydl zF3PB^!iAT|;l^+m&b0WzPMjmJP5j`A<=Nwxe7`zRNO41fNl~#js4y^<*ulHt>9y-^ zQO6k$T`uuZLHeN*|GxRfB13(l)h}eg0V>E`v!XkC%OzIB4UrwfnnATcnpD0BSMXjX ztL#wpJ+hrf&|Om zQ=KPp7L4%H>{m&+bZ(Ak>aj4oN1h<;gP1%4u)g^N5Ki&sGzUGe0It>Z4sdtB1N3ku zobe^lZp!q!G`NZOMy%F2oX(3|PiBjoAw@a@i+5sGtcVaYij`RjvY@7uiGPEi>ft7j zZV{Wp?)LWm2_IN;0^o15S4TW3S%1FmgJk}RFDXJ_Zm(+!%P)RD64VAe0 zP}9KCAv)##m{ak??F33*n#tMAWa=}X?{3wT$^XnZnL#ydQpPwI3pse#6VNV~tmS78T?( zb045S@0zR9V126oP=Ag}*UVrN0P--ww}{*|zhI@Hr}#qbBe7exy}TcG?Thz(?d2-= zTOfLS_!R69vj)`l>ek5rsXDCn z#C}WyAVIqETf1X)$PX#wO2GKfm-cJo3rswMBtAZB2oL$lt4OZvl+x`VYB~curAd-S zNuM$IM)}I(L-3_BN6T(Rl=+~<2E&)O{l^1%&k=P=g#8BFbT11`SQXY^`2hn_q;^Yp z4;m$j+W;n%l?G=(_OURx{@3->&LA%BEJ32CS189&<&r)3vqRYwx3g%%MSM^7mu9`g zqmG6#Eno__!t8UlFl|;WYFenYz~r8Dwd^2YT3!)OSce>9hd28E>kC2FJ3@?lDO<@i zE+JBn!3;7@t*b_?gXVsGGJqOTNe2v+VAXBXg_Y37K{c`3hoiC^(Gh4p=$ByXGQVMp z2Ua4*#em4P>bH^u1o86dak`yJWQkUy^hm5=@x)jy17_+B^s4C59rG9cGs2mtJ z&TaFh#Zi<=w9|dV%AueyDnX@9yT_QHw<3G?Yd6vb-a%DX}0z?}baFpGka8WM5 zq?`*rOca(R^2vo;phuzML7lPasTeHYPu}Qd&=ut{?6za1lf(d2eM9#eZ%lDsaTsm# zk|Ll?cM^3O!|n?vC~ulme%;P{hbGBxM5Xb-?>D@q${u24YFM<&;w7=h!jHl;OyVb% zlg>{{xlU*MQthvI0YK8~Kb8#AOV#&T@uoxQl@qpE*R2Ftb1aZKb)7)WWzaB3yy#NY z*(7JWyOUE-az@;?~rn?^LK5`V19Z{~C$(a!W3+VR1M11>m!b{7%MhnW>2j3SPl z_+oQf5k`g;3`dexjEO0K?}{4%g82?6k1F-8uN8kpzW z^HM**mD*3V2++Iw=IL!RmerJijB)3)ocim!_)~O!eKaJjs>M^Ndmww2nP(*&DGbt5 zaA3S|QR4A@0Z5BqQ20J5-<+>fEz<70l3TfT#3q1U&(!-y6C|Qc$9OHrI+OXs6+%^^ zik1X#CF<)EE1jhX=u)>k6r2s5JCR=`*bFVJwP;iiyy*&(Ep974!LuI3c&OX;#IHpI z_b^{Y?fNG@AMmo@5-jXT>6M9^rSg2#U;1&VssV%*fZNy^N{(61n@tg=d+URAq~&lH znhjshY0;6mmNrd>TH_!(tkPU@LE0ltHI~u+89MT$kq)zOpM=k?e{83-p{BTm7+bZi zJ8$jqF z6>3GjyV#VALyc53>UzI@5Ly#GFW~q#gRY&hS%6+IP%rWCp(pJVCwpsreWbc60gA;#6N+0JRd+=7N!FG}Da9)gCW}L9NvBA&NHtg%$;+X8&RADOR{p%n3$B#_=Fd=z%I`)` zKM>0Ngp}ybFjr)b<5Tp)6zRg0XeLcsJ4($+*7wOnoEZC?HMMYmYQ&zDWxN#ekVho9 z1HL)|)(8+izqwvgr%efqB8`YRiZ(iC?a^{#m&JdwJwk7V?I)1ckoix$6H#t4juvS7 zBMD@oU76f?U~Yl6Ds3nLYW*osS(`v)TVdS_-Lm5;>*HHFI9U|TROmyURL*C$s11O4 z`xbi@TK@LKAlM-{jfKF?;;&J+88muOfXClP7KEAZlIpY?ggk|OZ%AWzVMyEVy?{&r-dI=ii3WXT zh68F#QKu7AO&CrxA^#3Jcy5Ek93pD0&4W|&m6AaXNwsQ@aEdU5$A%B1yLTO^1%%tSjbkkfLNv~{NtGP#5d!AJtry{FMeMdJXOv5D2 z#+@{B^k*F)shTQ-Ncd1#f0PV(K=;JfrkG}vFD_|uIyqxqv!J@k`%^|Uhk8yls^;a= z4yK|Wwra!BqeBomEV$tPqa_U~*g*mBTmaR2m6wwFpcXkV7BTN)qH(|Nh7K~ume{Qe znIz3@WW_GMm|49BKQBfN^qM?=i`_WKTT8!89C`{6fwC)=F%5@_(O*VF_Yfs}Kv_5E zlvH8z3kCCerS$h7r@|2_3S7)x78QvV!kuj9a`e^)b*;im4XD-K#sjFUs4~acH|GP4 z9Mssp+FI|hl1*&zr5%1hN#ol5j)Q%2WUm#}6v{oYi|VJ9fdeGIc3)F_epAZtaSJv2 z)S(_)6I7|ny25aHac&z<;{fMh$Ua@Y2ksqr#g>swll1oGN^*8kNHBlwNsW)=cXhGf zApTl*|2vwWaP|Kk=e{sNh!x3~FWJ&SW)(ONfH#cK+!mcMofGNcqn=B{F#wK zVE%G^Ks=Gul>etIk0fNGY-So!i#n}X)j}t>_M*B8y*ehIuo8HAi)xiXyB1K|_d(0| z+1j>3nf>Fk^Vg=VIR&fd%?F?1O}q0Y_hI&1wliQm*m~Zs&3W& z(_Oo2@2;-y>L0!KdY-qWz|}33x?z;~r1I9?r(L^KFHZ}PRdvmL8cWG4w4|b{!G7P| zK~9<+eST$kh7v8d>;yj{M-ZRyS1Mfhk0E1VWl2!2H2E7XDZ|2umn;lChpYf)&LTIg=Vm&qeyFdiw>qOc+hVKr zkbX4Qw)Psr2~0wCvq_~|MrgI_X)go!*VaD0f(ro?U?lC{a5@`7DZyrdoM}0`(^v+y z&~GA6Ho8p$8y^nG%QNW5@y{#L>hrJh*T6sJ`0@EVgh%OW%E~GjVro0e+l;*QFz25k z-l}dx);_9)ovUdKbhbL0ycIgztFesR(=PtK9^I~$bGq4+-0>E}4t)X}f4eOwYPQ&H zASxKSOSaqKmoG0QFXPyTAK$x6oS) z_n;Lg>Z4zr(p}y|>d4}^2V^U#Ot6lRpQnf&tH=KBn!AK@cS|~5>B^Z5ATHc3OrL(!PQA_Y>ps%T#Bzmiahnj#`ztl2aT@?byR^kH)5OE+m z2n3eqATxA*QgNmg2^!QwZ0iDCK!LYL}Vo~O{&(# zHp^awTuR0Af<^ao%9VoWL^8Dk@8U#_tD$VJ;(0U}wvFZ6W!|#5CEn6MQ)cOq$aHZ; zYG}ztCGa?1NM%I%Tj(@rKk{|GZ_Me6QIWkOs2<*#k%p?5Y5Pxcqaf<;_i= z?T5kz@Vhf5Z*9JLMT%>}=a&oeM5jW#Sz>?ejKrE*v~=F}NMmZ~Q-ferC*?8><$sV@Ov_5QVJS6F z471pgg@h{LuJ_W4E|3G7R|7b}8y$kLNxP8(S*G!$VH4#}*}|QFs9MJuOSkz~mCWO3 zxFdLu%;Q3QZ%I&y6LuMrcZ@GD9oo6g8IPs-f7yxqwo{k(^OOb29$t8)Qjta9uMX9& zbRij7W<;1!W1htM3)CAbrCNH)MRLQO7loLrNxg&63NRau9M=GX1DmD2HC~D46DfA{ z_?n4!nn@}e8+>$_${#|leRz3Ou?`Vq0^{tAKz^KZwQVXk&k;(y!miApyrD2L!P~f| zB-j^bTr9|y?iY)=sds^lsNdVJ9Uv~e(wi;#fGd}zCs3(YFD!0eul-)0pWSu13F_e0 zhAK;)3w0KZ1zj1^^MME0(`4D2+DO_D&ynOna(L z60A*h^;LRRL!%nU(5;s(b6nl0HE4s@v##9d_U~QzE=vIZJwxMGQ>w;DGa|G{-(kx} zea_&e{3zG9^Y9SV3MII6)=K$NFz8kn@uL&a9^DvQOn(-$U(V@MokLrkJzPE4`3b#w z#Ebe1O0Pn&DgIS6#`BQi@qVe~O)sYy%3M_L#UB%jTkXi3k{G-zYOe}ATgJb#U1 zk53`-1*8Do5kJ}vU<;y?Qx5(dMn}V$$PoVOZkivzhl`4>^qo5qGPES->4ph6a|%Gw z&Dn94#R%-q+g>~6d?ifyWso;ZyyGGEbMen_;)ze?VzZzP{Cld~BJQSW`AXXs8K;0f z=CW!~2c*7o4R3_FBDM)@07}!=UKM=h9n`z50qiXRuH#bgRVht!E0R?iXpsHm3)0xJ zC4I%KXuo^y)bI7T)m#{5o#YpA04VKhGB&@smsHfly?8l%FXa9j_K7v+P#0G?yRF>q zc|WjQ7|tQXyC%d6Jt)n!KSmu`370c{;(~kmZOYc?!^DYC`%fVPPtk4CB zYU0fNE1ZE!Sw!aC+|ST*a^Vc#}bwm08pg7-MTpwhX%0v<3lY3dZPjFWjL3LzV zvY1jy^UWeoo%UCLA3S_-dzYgxAO%gWz`%w z1jE8!M^~=TlQtg0pB+c;CY+-v8#(n14wY8&!WGH-6m>7^5ZT_mkm10rv9i)9eJTzl zEyM6!eCc(eo~Ap9Dqw;Cx{5oTD5BF*m+r}U!GiUy$Shr^wogI#7=djTyz%={hmIbv zHfH*++!hCJ`3p}|gM`yxsiG3FZ%pv3M&Wkf!gYnRPA}u^A7>Z#QC#@^`W(9OXv^6A zq4Kg`f^kOS$|0pg)LK~7~ zRYpN_;LyJ!Z&WY?7|3@TIWRn-2>+tRZr+{sJ+U-02c;ui4svO+$U{azIfjfY)~!j$ z@RZPgCySA27v~mpMj6s{QA)s-XVXJQBZCzI%kd_jO#w|~!P||uHYd^V=0DxPv6-rh z_tsVXu%bQ!kXK(wgmw?WRUDvSYO`$X0Yv1J#g#aXX$dj1rNb|&Ssbv3EO4>jhasi- zJ0{cDPPm_?4wo5VTd8vZLak#jf063^Zx259sVA2~ZAS&?&QqTKv?`GC3vVTERf{F? zpwm>&LIzvgoW8)HvvD*x{RPPD0rQG4S)@H!s>Z0E9nIJkS@|B?k`dv^+B7w4_b$cf zc8y>9qtBz+tQCq|!yjI}$@OgVt>IchOU!T`uQT&E_JWUC8}|!fCL-X&YBq&Rrl-;u z3@+)S@&PK~m)mW-N>adAg4OFxmEJ8dQ?1hGsymY-K{27A8?T&Qddm5>!nLh1ypN-1 z{_iR=i$*f!dlo&&ukA~_8@t<6Gxwml7&pmJ=q;*0&dn$AXZ}E;UZUB;8E0A>$O4%p z92$*JXj|@f(?3aoFU;jnxUdHrVKA+Akfp56bj}JGshtPBI1<%pvSL zqo@a1chsIxa+t#R35fG_mhq|KD%i-%LjFIw*hTTGzYeFE;WHA(5=DZ@i}C4^4aUz1 z<8jQe%h=M@suQ5XXwVj=Eti%j=eF@q^v|gZW5U!Q@l(q8WhmccQ4Do zMdgy1;Yb;(K8$-9KaA`3gKw3G9O>K_a`vRya2mOTRde6B|TqnGK0dFM*RP&!;>p(`2&xH|2;U9**C7)b{-txZd*-5nW zvRcJ3(wybDtVgI3lXI8rM*i!;xewZQw7vavcl{QRTV?P2ns5PJierD5un1xAscl!~UOFDIU)i5LFtN&LI$bQ;w| zqzon08Whvl=eDl=0;Bpzzw?>x;dKP0_Y;u}`RO*+s~4|)C+hvgxk_)`yli}UxxK*= z=O{o1dbgZs>6Dsp1PY*ZT`M0Td~X5mtZ9UNtyO#-KlvH<$BHSvYAkD>o*=&K;J#?9 zr~l%a^(cOGOaGP5?I9h@y}5m0lH5OhDE2<;G8c;pwHv07S5F8+#A%W-P;O zm&x$N;(?`Q1)GHwwtiie_)8C;=V^9szcB4>#CDC5b@1}pDz1V6!^1FBw9h}egHz(v zm;^Z<77!kQi&+e+-DOH#HByQzBy58;=S79LEoX8#(H7|XCs%A?BeCuuh}}lQEU)lo zr|Gz5)hW5$&{4W5d#s1@DCrr%npYV6KVhr!qP}cRyHY}Wb#Q7CWa1GNh+E`WkONuI zS|C0QVECGI>N`!i$uC%vV2p;)dQ}wqa8;%1l=EPCkZDY$z-ARJ16ii7sN$PLwF%96 z->#74>&Avej~TYE==2wF>tp;A<1U#~lB5X(Fvo=y3SoaF4lf1q1attRLi*Eeqe>t_ zQLheYt3ce82gUqL)+}%HoEvJEl$}d|c8i@i=*yU5?S$&`lMz1eZ!VDI3#P$8Rl3C|9`@;HrQ_3_ACz-r<^a1M z8=J6Dc5P=4Q8tTk`x8K%XLJelMENysjXH}+^`DUV2Wsqwm=9vS9KtGv47=|gh%^x% zv6u?BBDJ`3&||<1dN>4MrhIMAFx(>sA!{rE0%Y{bm*lEopH?Zpb+jQyztSO|zf*f# zsar_2W`8?7UF=>`br;n8yeuzHbW(!pM{#!WeZ%fsD~dhc>IuN^75$M}rN9Ft=k8Qo zJl)n&{2e~MFVvDwNF?~30`tabkbdH1_9RsYT%p2u-kRl|1Si?c{$?Ob2zka}Dwxx) z*ahlD%uzu3_%(t2{xo}Suizfax8@n9^Cr=>?H4PsUZ$ukOL;=EgyB$(&4!^EP#Q!l zrSi~5Hf&ykK_Nx{3bHOh&mh#=}&0{XaoS6^CBmoY`X;=~W^DNEHBePf`{ksydvWS-hH13|_IU z;^8c7xi7MdRVI-coYBtlz01)*Pv6aZI`u6I ziE1c^XCrCv=Bw-=$8<*%XE&Sf0oo(Ic5{BZ^`=7E)pP{~FpC&g~G$s1dn%0frLVw84T z)Mb08CKN{>j%z-TYqS=_Pmmiobi!93K#h#csU7np0WXVLOOu>7h*CD}1;rRN=$G^l zW7J8M$TH!?S=+jxqwslh(TQLb?CanVl#1GR-@DVFuaj&~cOjlSykxWchB0{}q0X)@ ziHy?}?L2aETb8kc$W4h@j&yGFqq3*ROQdU%vM|&>h(=i*qbN&d?yO~T8vt;z_{Eb%f`Qm(7X~q->ccS%WSudyDWq<7ymRneF@6c;;UHYdxRb9#j z7o6-P{Paw}In)pT4nl9uF+Ugn0BjV28#)pLhvzyIHCRZ^`Zw}ro}CO@7?@Ijhv(Ix zTAYS(9?hg(gob1~X0>hrmCF-zn7PRQDInpXK2brlQZLX1eLuYK7gjubkM;z@9XvG0 z-T_ID`0-?P?^e=VW1%9{)-F?9w(tRWA4D+~LYdq@s4-(%7e5q%*Ro0nNT;>7N-Hh6 z8?bRv3)cyO)Z$_(ftGOzg=ugZ&!)S)pW|`E_@knM2`*s#gYqV|-C@Hf@Z>ZZxy}%_=d0n{v6{&Jmv;uBxjiV*(R-k0^TV@@^pwlO8k8>8ICOi)(uS7Lcqyaf^bw4)# z3QJ@bH3XI4>``iug*Dq--1k!@zWcTE&6h3-SCIKjXyDB#XL#fq;1z}NpR-q#YYz8~ zk8jnj7j$<~he3h-XVtAgKJYjt>4|Smc3yvJ5)TL~41|K>O zm-7m2iE6(=2nWzV%Q{hR(^l_6vwnuRq(4R2Jkw-PhuOSP)@5z?-36@}iO2b&oHy`l|z)^*f{bB*d%zr#G@)# zmu%tpD5cWiq|z8O;j8EbEBykmOp8dMqdU1I4uvH^^Z^CG>s4YOip#B5{IJc8_Vk@g zb(`twCNJ@l_hU_RZ>+|)ZiPD#kN;$^+V2xenzt)xsm;)=O)ZzrlO10iWmqM)Kw@=ZVtp@ryHtbH%Z|QT-DA?G^tNhTk(k_rK0tl?nbDHc zMNaf{4KPI+YWCARs@gi)o5&~rqM9JB%9c(?Z<_nW8w)eG7u0Xrd_v#al>YBrJ>`0) zSzIf85-`s~r-@M~Fq@iAfE>*>0qWUD?qQ;FW{Laxm|>h;a)`Aa#V=wS);D|Lxa9ew zXPEr&Okq^(WdxGq?FML+b<1E%Ug7rIe26yJI1y1HV{xu(Tl%=h?*;^JDN zWM^Eyp!yujG&z!g@ZFo&g(Ev)d&MZ(31DJ`n=>lZ!Pd3iT)IS3Zh&1wcd;I7RHBrf z34`w^$MNWZ`N$ZTAwtM!=DX&2fT0c{Nj?sr+x|i%EOWrMzhwOm^7TzWV%^^0U+>>Kj}k~Y@qcCNgJfMvO*?kig;p0f=N2~af?a(Zn4v>I1w}6g07uX* z(>onwEw|~pT+PVW2_NqmP0gHyJLF>RQWf8umY*nACj~5K_rtf?HDr^OicXpNrHI$e zB8uRxCFlun2nT{vS?+A|ithiu>k5Gzru%c4jdRid{!IVy)=eb|5Gh(4^Thfc>=XyK z)>121F1-AEdkj!~&hL!hA3W}>05Rpq3W{^`u?C;h$s?4y^6W~;$n8Oq)<*^r9ErSq zR))ui5Km91nBTN9zvEK**^=)KR2g^{;PxS0BZqKtBB}ykr+-&r0FLn=)rhPxb z#x$6vb>FGm;I<{n^E%{ThLFzMRyc6;59ciBF~&F)iU{Ui%}6pjWX5Jo0fD_ee5g=# z>aOU((E}mmYagp7=qMAX-^s(9aT&Lmd9G+~B6{wpK6_Ta)IfBz)p2; zMhbXqpOUhuIeCj2)A=-60534;PVR-TQ{!daYUq!IGBDHTExW!3PgLe8bDYZrHL7V}8t@Aw#!i!+c$zKr z>uZf(^HXqQ8~|wGbv*#&sw$9NcV1;LmQnoedd`0m`1a2 z4gP{yb`;Pe3xtFgH}{P#3S%Xsry7|GCeA9UZbblV!lSX_(6FCC`7jo3fmmnG#5ER{ zUsa-q8ff@va21{atV|R4ti8P?fjX`EW6W`Oz(L#Kjy)AAG!D}uTG$|_q_NHqmmPsB zH{7DM=GGdURm}}7Asin3~43}+~ zG0h{*u27}83o|4RR5G3ay`{wH!J@AfOs6^3CU;FKt;?qe$W$)KnErvdZ&=86QagB> z`z0Llju-5vk@^dxD1ubh86G8LpUt9Ui#gd&>aciK8CpsC_xRsN(AS?{oxY9fv>F7t zz?qmj zI9l>#z(1ABAfEZU4BKy6H@RjSIsZA#bQ^(ED}!O1qcDdsn!KD{W;@|6LB8wC8~4z58Y__**Rt|##))EEHg#O zt?C}VslYulcC*!7<|MSMlFm>|j>ofDA5iSB#z;{nDXj?lq6x1sNzT?x=pdo`S8e7h zw;$0nHf9C71-erw9wAmssVO05W!^LGD##^TSA=lh%cbgmiKMgR1b(LT(AN9U8(ae1}+V#3=)-hJaFPak}{o>jOhx$-^1c`njqaXv-!i;=l_hN#;KEkkRJSH5L@ zd0R{8IxEEVQ8H5EvwClg(6CmAJw70MoFVbil2U3X~ zQmImjag=#MN42PvhQggCZbao_L#hTK_optb&9b38rn&Wv2j}9x1N+n|ml{U~t@vi{ zi(OlgeQuR2wVq6S?z+#3PVML9b+gV1ZCh6KEFlloRgk{VDWh;^Zu>55%>4JC;%oZr zstkCh2(%ZSz$p6(4R#Ol@hp2sQ1qRwck=R_%Z51fE#y6#L!p5SerI3SJsmXQRA|Py zb0}rj;Ti8$V6A@n#r&0=e~0{&)NcZLj@mcz6RFd0tY%hb(pP**r~qOif~_`e!|rjr&c;bUun8z&X7Dl{b?ON-a`mrxhvoIV;!60 zAALGLZcbsVSD+)w-jsS?eFUJa$ruv z%zV3G)@M-ahLJ(zRDN1=pYe$(9;0{&t+^=&i$QvNvm>%I=KO?1E{Qh;cKq`|i{hIJ zwrmq%i5%cl=s&G3|F41N|8H$+t~H?c{vU=}F^x_1Kb|tfo9pd=>?LbFJnw&5&ZDZ? zipKADr(5up*=Q8*6k$j{s^;_+?CJl_bV!7c`QN341E|sd$79w~kih{2 zK^r&4(1w~j6$J7!qfim z#=;A^%E13Gw0U3FSNiom2j&}#rA%NoI8;X`U3VI{!@|G)TWXKnA$GAjs3piN5L zZxq6m+CnZ=z<){bDqmRN#KD&@cJ?cC?s0fn22I6e7#uX|mL%196qZ=+HS4Oyr6up&_pa%VD|>ft{GhMn z>EVN{{j_7Rqcn?b6Z1WG1HpS$vUJl4EHwg{WPryrdEDipUYxLwS1$P;vfK087Z=1q4S%fKt6#zf4kzs==?(NaAiJ|NekgW( zSsUN>rjhZ+tpREsJ2XHCtm-p1feqsDewER32zI_S3}PLev9UKAQN5Q5%0;C(_fKa! zBmgL6l|4GSy(SqoK=KOG3d; z!c4(VVLsxh94;4yqp(TpM;>!;RhOHce=X}ow%Fm^%bke*O}w>lvf6b2+*3N~GeDPy zro!IIMBQaEP3|1~l(wKz5)~JM%#HJ5l79lM?3b=GB-YHSs0EUT+QJf7rk(&Zq#@HI zK9}ra1Qpf}TJ|pq6E~Os_ZQ$|s+OecrZEedA{6hUSKVV|&uh=AA*EmnuE__HBNAG#M54VQZ9GyL=l>k9faFTxOt z8c+59xmPNCX2oA8$%ayLnluoM^iQXgs#||Udwx^nO4uprAr_}L7J`-kgnr>>s7M>8 zIesXLr#2!Ip29ueT*vM!=jyHz1IK2rswu0nDi4p0O9@D+EYB}1&g*JDl?Q~KO@wG$ z4%d#QL(YtNasHg}zGv!pW}J<}e?UZxWSv&-a$F&ffCwPm_4>n7p;M8o%JupazL!hG zy~^MUc@f&~OdTXLs7;$p#8yh39}D03A{@>zAmJ#TBXr796;d5;?4hwy1PAqNE?QWM z1ns&Ja_X*GBS^ICGJs%^(*&TYp3V^{X%=7jLN!@nO#}mstDwt*z~t`3A)#SMsXnnp zgaogmTGU!OcRODNmG-h(DI^#(qp$$ABiTUNGFQo744@>O`M2IRwY{=~LpSvjGsC?%rBn#9}S$ z5&a95DV!?D*Vpynp8#hC(cD*c;BYNfxl}|*f^KXQkwqD_@XC!a^>t;nzUZfkO zRzw4z_dT_CJN??5#?PiNfD!5ChHn5xKQa!ap?R};QhSjX;KS0OH*J+CLtYj}x>grV zewif!wy@3)M3rKSvnK%Nvdh&JU|Y`ucM$+Xjp3&?WmK8 zj^l{HW}CcpCA$0I{I&8x3Q)OXOPDHwZ+Q}(slN%JyvWATjou5!TcvcR4K#qdM*3Qk ztil8*WB7Y!$M`uFrr*6gXazZTuvCvi>DOOKe|jEus)M3&`wW1k(wp>)ul_?h7$jg7 z`{UGGB(PUheWNAw6yBlo&J7L^J;Ad4;q(tA!IH;>rtm6G);(e6GQ-vnmwQoTBJSCz zjeEVd@YTTZLd)NyPj$-Q>AANMdFv%a&<**GNteF~eK(GZ*V7EWZo={V!r2+kJ=#YV zJOc0@U&BX-Knejw-rUo7M3wP&KgxZxy09*YaEABNU#P<-dvozNJEQ&yo*3)QEc0$# zq;azn42(KWX74DxQuGBZQT%g>z#hV?z)oVg z2peckznAk4ndOIvyjUG)8P&_m>is#ey#|u9v)@U~4W{^-l)L7rDW&B@vY}$0=HS3` zaCl=zqGb+H4MS)8%Vp_=&o#xvOu@OdjvD1@7r1@oF$<3(?{tMSqP@A@a_Z%cy8E-e z$d!0BOnuK2CEVlK% z!@oYj(BpI^Xz1DM`Ve^+PBur=?C{blf46@5d)Kf@Kp!cap>MG2q27-NNym=C{`Q&z@OVQ-L>#;j`#ZqMe{uO??Miiy zckyeAM1BQ>IRE%C{VGAZI_owxjGI>}hMd+t+b4%~rtHu!xbU|r>@yI|EMeX4V+JvB zNlM;IBCIw5ctb&Aj2#a@tr4XvRU|XIO~fGfJgkCvt!ZbS5>tT)+?!4E^QP!~D2fw+ zU{q*Q!+|S28>yBEZ0(DD4Oyj(!(MS8*QE~ErRDoEuUIi{aIYA!+F93d zgPoxuk9bau{rbmPI*kSPf7-7B!SB-2_#de2o>d%!z`gYjPngxs2|wHIN2#Cn9C_K{ zw$NhO*WOn_#{;o?E6OKB-?fgloz)Fn(1unq+3ZD3ITU(rZ4{Q&J@+iC2tE)? z;7qm32Ce-*F4_h zW7{BUu`5iyg0ry`)gO1m2p|)4JU&Vm`+&HPwm2q>G;b zdpKGQa?g$LY(5kw0V_B_%;9`ah!Yq%xZ$8L40N8g28Xfn!^^-!C6iFQx01?tvr(Sm zbhtrlS+{34XDyXLPYv!4H%wseu0S}7iNi&&U^B8-2=}Y@N7Z5POvtTE+!Mrl-(iNA zcuOYqG88Y@iw-ATiRUPzzB#zE$6Tpx`%>U+y(8C_=H?#f5ML$0!Wr|tFAn9F1%L19 z7l*uszajpk>~!W7kCmint)Njo>enpe)&YNg&A|hGNg?W)G1*%4#@=+4I|ze^dImE= zwS@lY8z(KxjDSoCJ17?YThlYW&EB@5r1qNcaTYYDba0k>U)g4lzI2=#%PGlG9b;Nx z=3W=inX+NNgC(VpCMY~6~#Q)8L_ zop6;)aUEu=;t&uJ=~hh-IG7vq)K8h=o=*ff)hNDT{+m$U&d{Ow1Ez@#trF4xA++4r zca5jGg*)06VYJsdOvn4+Cj_MqI3d%IOcN$c+zajJ7ATh@S{E~-m`QFc7{uUb%KG1b z>gMxs^>q#aOsAK8)iQwT{X~fsvZ_xvJ$(5e5&rW^*)4*kY$npHPI zA&0)_)8oXpgI^0`pHo~0k>8rr$;2WC+ZY1qXLkcRmX1lvagcmMER2A_kA^hfVlENI zgS_i-*_aG4;tlq{yasY&1#*h}@9Yz8g)zqHN?E-CZK_P)k);M)0eWXJI<`o4(}F1x z>x;`q4-iN_=e^Ejy(0(?ng}hESKm72li@bgEE&du-Fn*{2qMt5lI5A2tgCoA9+HTh z#1Sfhr5%`Litdn9=Wh^Hoc*4tVl!SnMbg4U2x&J5QC@sQh&1D9y5b6X+4LxOC`5w69BVQHN;JH}rz zmmw!;6@TN@J@wk3Hcpwmm^vc5ix_8cAAO7fmb@)hi1Fcaj#^oAWT2z6thb7*;f$cc zmCNHl_$Ey?9(QrPe<0ul#3HmGn&UdnQe7B(MtT{-{b3ee9T_5V4r;Z6spPG`gMV3W z7NI)Cf|^;OYi0#r%KR{6OH_5%M>|EJ-Q>GXbZk;~ zGnLXXUZ7F5g*l!lYm%*bj5-8x!tfS7(8;}X5@zYeMa+aS5y6ssBQ8+%7WQND9J*00 zlIbTckK@bZN%pTo0WuWC6nMjX>U~oHMh>!Y6Y>9sv(W69SN}ye*dELnJ8vI#FC+S7 zal!<2Yi;c%w;xnljjRYOdii9^uOzk`9UlczGi&BY$Tq}dC@>r&uQc1dHKORJg$kT3n2PD&GQN~HR0F`Fp0f9c zJP?FY77e~Ct|!Kf<=mJOiJ`1ei>AColGpbM9x#UGIlpZctoZ1xF$T`Pp-R|>ukP4c z#+VU_IXJP=B!4NiTo#|T4KlrGIFFP_>?Q*0m=NCyb>v?(k#M&F$s_b|Z^A<9 z%0Q?!4^j~S^?Fa4{`d4X>l9z)vBtLizb@I#dpcF*gNq8EkBTY2#DDA!R?t|Y6xnfH z1%*rP8$Iz^iHs4~J$JE7a!|H73jREkV<*Ak%|eGLFC}31_(dJoq}Qz>iQ28@v>f95 zmYXC>wY;|G;@<6Bn8X$Xxa()Hf<5^HNewL9{p2A9(oXfRY;3^sx7@mt6+oIa2v0-m z7O~G*4p|AKX!gROP&F)QT8J#`HkYXN7@lWuxD!>oA;aiP%Gj#h zP2sf+k!W3OtLXPEKG>3EvNo29Ux`*`P!VqEV+MvLom715W=`h;)a#F&Lhh38y58;? za$YJ-ITw?LdP_8>uuQ=WMwS6PmwDP))}HZ$2cFsBUP5~5l+2an_q+tst_YWpP& zO3h4g4M>t2!EG|+fG8YH0zVv_(m_r1tSRD4k_~-ujp>ovS@QL!FEK=l{EW^Jk;MzD z_~U3>7Rm@jLcuNp#dsC{-0EOThc*sbT!TeCtt;-Y4dzVbtt?;(N;yX==q((bxl6!)>Id>#V)3>b@VW~p zrcmsqm)UQu8K{zFXP~i|Dq4<-P={t)(#gEvUN3v(-^d#Zx|}L{Jy@(}(^@fmrai33 zICA(Bph^emcGE0r=W<+@^q=^{!S={oZdM>;rWU=TV7DjM<*6yQQV6R7-S;)&37x|> zJm%eU7W4s)(XkiCIXE+9rRr|^uz`B5kbSVd>yX1mS#Pr5Kx-n^ss?eBtZ2WOjPraT z!*}f?Ww>~7Ge5N(5Tiq-WPEMcV;&1 zw2A1S2}mY23zQ72ESi`+-9F<7>$*9GbpNNQ|kg@oja5i&-umK$^z=f+OCR+Nb7#& z(r-iQ7ciXKP-Z{A_Mfk$wGjQPR3LAD1R0)Qq@9xb9VaYj<@KI&urwUq5Sse&L$PdHs;gRSB(%!orjWuW`PTU-Z8Y>+n=&7sx35x25-JVY366v?B%iU4$v+7B@v^v zaYrZN`8q*MJbOQJn|pK)ex}95parvgrJC^9&SO4FERNX`_ee$kmVuMftAQoq0d1_% z^RtJemcF+0kjb6UwOwFulST?Fglx4F$c)cnb4i70{K-(rtS5irzoC%U_rD&44jx#AHc^4ZXO*3S? zma_6;(p)uOi7f*K&HJ9oGPBz}HJt9*eJucCVK}}+bYEauQ744pI9|+aP@?BCjE>k6 zJ|THPRy$4VDxak;Kz<3ZI?L?Zx~#S;54kZAw3&_C+P$dfuyav7f&5olEWv(>oj|V$ z_5dq@rqI)QYwDOY6SxF%P@xF-GFmUl;g!|wF|_1;Clh;lmv<(^rpi@uN}r-_Fg9gv zJqJGahBNbKFTiE&%PckBTKqu=Y++|glAbbdL|0JvQDZ}Y!pH~`=;97usP1~q!bG7_&oa6mGnC<(3a(5zvK z1tp12o;*RU2Q(jDTHyTl&B-xyJ<3OxA%J@AGQrk6!7AcYv2j?bVE+EeHNCoNY zdwJnOYa?4AlNgYcwJ8+A72=*J4ZC^nxpVNkv03-Azuw3 zm6U+>M^=y5A5Z1Q3p6&;M~o)X>=NK#-HOCKF_8osS8CIBsc8~E6{dFivQQja_76boH5b%F=`qF zRw$Ge#$CulAunt%Km6KS78?lK8oss{%LcVt#|+7a94m5m0c1}cvp^;3@09TU-KJ6& z19_0_8VgugA*BX$=F7syu3O0Dg)3+<$w{`08RE^YlV}>!4lFUgesTRH;OX4A6G?Qz z`MmdR*wswQnuWj$4#SiuX9vM<=)7`bis4=0>V^E0hi4ZOERs?!Ol!0{XuH!I-(|nA z6~I;egGP3HL*`#Rjb5-q*mNUuv2CA-Df?&M`R=}4G&Rm~&buCm-Q zN;G)x%D!nlU8{nRfKx}anSQyZ@DV+8iQ_l?Nq(SDiZuK5H2f6Y#AuA!I$%A9Wpkg1 z5e~4O9Mt2&PH2VQmOmnULb+VcVXKpW71_|%imGbH%_v0un33cN9UPRaM9p#zdwevk zPoA|ink({i130$yo8;pDW8+$e<(m-h~lLGXQ zko+RY_jB9ZkVBfyQ9m|4Uw?a&_SnWSQozUe@5TFy7j$6o-qo%YRs(yC56^MlIRs3Q^@CUIfr9^!w1yAxUM@umFCwWI$N8UZJ8BWMYj8Qgl2WDpuP?(pttia zuHDmn0dHr$9){O^R*c8nKFlg-JMH(G+mYi_c$v!a*hNo1(d5%eKT7WemV4D$ezs zh;|(~4kOy|jdFnZYqkr}v@uFDLR#%_KE*{SG{H8taA=oS64un!9j&FqzJ#Q~_BPZD zuU+M0sMTxW4&GgK3ekKre*VGoPM=PrAGHb$H`E+(Y_;e0LS$rnKp6#>BK3c`I>+v? zf`4ttZ0t0)ZQHhO+sTd^+qP}nw$s>Z8mmp7p8q-P(OUBb_Pm(+?OFGA-DVDY@Vyvy zt-d$Uc^U|fZQ&@>@bYL5i;Ao)vnFEgna;dCP}w0Ph6796Dde9L+=u!!LY+!IlEtGR z35(Ed>OEMdt;jVzVqvqF`S#p53pH~GrCfAS(o+P^p!Lc%cD0#{s=)d*X8qk3IpS*7 zfm9HOO2ep4Z<@cP-w-bPfYZe%LZnV#j*Wj#?^c9*oWf-bH zioWy3BRIJYwEW*s37}}1rIdOZ6BB9-#B)DlQ|3ppfS&T1gIdCG~u_Eg>slJLJ3c&+vMQIbH5LMg?0oRgno`NQxO}9-g6x|a0HMJno zjcl73RA0ToyAlz#ta{g#P^nFOBGRW(qB&1)QDjE8&!3y)(ly3QOW@tbwp)_~ZlbVz zB2c)f`B=cX2sBTkrgCg+7fo64(kf6@V-;$%3C*pAo|dAb{x}k<+IKM|3^jR}5D9Bv zkPsv!Q`Zssw^d-Fm=Ub@BDBI`hPwz2mdE&DJkcr_-xfBc(@(e2a6z!l<*foWrp0(3 zD*9{I+-#0$4;S1TNaPCp)}uyV;%MX6SK)6nwGwgn0ej?V`<}gq;l@L7lSu|#FCn%Y z5f$>ZS}wA=05VP*jYG?)Pn|yw46#wIF#mew%l&8yzN6Mqb(0e|>E&rQYGEnD)6;fg zW4gg2q|2)jrK=R`Nr>Ou;gY7?(q4uV^DV==W&R0|=Fk;vn#ZLqF{Q09(Y3S7jrDIy z!vEd*@h>uEM?h%m2r&^K*E|l-G-eSG$v!9(*HMClks>KNQueAq8w?`-KHvg>GO2+} zyuNJJ33}~Nf{IhX;$3Iqi@YHqv00L1!Y_hlJX=7YclN_>FiwFJJa2$bVW-Fq#bV*7 zE0nSJ&>}}Nx0#Nd&01I!)C7~(RS{k%q<}STEReKq(r3kF(V6qB?V{g`R&E+}@eYFh z$g>@09zW5yv)<(xbn)qC+iDD@(R~!95g#x2428FARJ?QNP$j+tuM}nH@QUD0J_RJ& z<$1%w*}c8vGoj-Er2vcAZZrxkwy?i3D1=cwM}Q@1!nr*AsGN4qc6UBZv(@bw-GmVxSMpE zqR3X7lP9eaqC0BOajgE#f_~vzK2x#L3&5{2I_iPm9(F$`;Lz?ZqovC=N<{Ft%~GKX zHrPufz82b-Mve5grP2ha;+o&O*n)KvFY-3d_r*t;x915#Q8({pg5~Ulw=k3yq)ARG z?$503UIg){RKg>bNi@IRHSieXHF!_0KbnL6{h*c=2>jbe`2GBo%U_dySuld_7XU$q zCZ8~K@C77!<0g4SCB2c8J-9`g5ui%;sjvvfN}n!%aZdJtP4>V|_CQVcB#~r>kId7( z-^*`^33&RAlM-aByJ-CpQEGb#-qg4-kcX&DdWx@C-~EcpU++IWJE(u{{N*J`&Ji-K zYg&5kHMt6R^c?!MSHr*Jg$LuN2uM4K$u1J$qBv%&CiG`F$-=qqnO`RJkAJ2*!X7#A z294x};zKVY{dQ#!U>KYC`9n9RXq`8=o}vI=bXeJ%)0!&Oou^&R%i?x#0j^zp?n&16 zjGM)^&t)`Ec5nmP##t|!zF#u}f=d=)ZQbA%-nY^jN&H#lDiNf0z6~C(tw=4FMTt>PWv@~>N8-}g4ZO&>S9PdkMDM)rR?3Y0tF3tbfyGws1-Q0CdM+>11*UU1}%XIg-LZj}le55=)=0^le6Yzg04$p6RW zu9zPGgg7%z>?D)0*bgRPnR(kjtmnv#+IW;LI?PpzLA3WX~~|-KuTFr%W~fkkdumabip>O8XUR*N**qg zsMlyl5zuCa);YO~*{w6UkIqF2@lulSDK1ua& z%zO3t!1_@mQ1FWi0hDkC=3i8}I|+mI8m$&VS-U&gr3L1)73MVn>_qf&`>%4Rm+~aq z0+2G7Rcee&+#naP7ZtcdIxj${Nh1=C76f+O=LOV4>FE1t)v0rot(Jx#^C)1ADdDe< zFGD_`l5Zs2YFDfT>YefAu6tR7q6Tm9?~=&{tt_Vig?WWRm+2|>d8GC47_lnz%h*pO zsTF%7=l#V#PpIq_>DH+}>WY30+z-a^iL`j6NQ8_#M89?Oz?AWh{De8FszJXpQ*o1O zu0_6F9zrB#wW>gd4wr{#Gc6;!DjwIpkD+82yUYs$dL$i|rq~fmJ@j$U5`|6m)I4UH zTndr^-4(^-QwQ(eZ<2AM75NE^^o%}g221peg5b2eh8dA$gUTKAr2U1^tG>tW*NX#Q zt^8aZ$Wnp#I^FE!DC3@gV`6e*hVLal4ejJz5uJDC*6|beCMU5E`RMv7FO-=Jbay$1 zQ;#mw_Vz}pN6YZABz{Fs`CA@$;89gNcz^i=TYX6imstCjiGPRZIWHp|1q<0cwkf)M zbGPB4r=h_z=84b=jzuWopxqBx&Kmj0Ikr`MF%I(lDs(@Oztz&lqf4C3w-OjvIC%5z zs&pt?mdjdEW17Wg(4QXKT0LEq1Z*Ue9-}c0LCx$5y_NXE311;+#qKxbO+CTG72n_h zW|=R>jhlP>_~LzU*XGJAkI8Eg{u@L%Z35j7S^R0djLJAM^SI3dX+^)CfJw$23&SHU?eTG-a2>Qso(| zWyEQUR{L$zJI2P?TFHHC_77p9AOH1=@(@Mmk!zKECtiLc-xv}f_BYwNBRFUypt=X! zSA@7+yG?6A;t%2&J&mC`Jc|F`??mC;HX$+HaHiKdx$h$q;uhd5d2{ zhV~a|Zf&`0=_LNo6Tl%S%UOvq;TROT@Fy?AF<_Da^Mn7szjEbkec67+h5(u%(Q%;1 z-$yE-%?c?PS)X@`oqvBiX;|hr0MeU$Utq*Jke^@HU%m|YS6_Pl{I(T+b#M5w>GK({ z;?;_qL^^Kx=cDaN>d#LB>h4Ewga#R%&ALKWB^qK*8}LA4tmzy_xdP@fF&buyMK)8G z+&C#4#GxEkl;nTCVvL*?OERX2Z!xSBjx(hXv2a}*6(!ZNG+k>MioP+BfLp#Z1wM<- zd4sYn(s%dAP-Vq=Y=p<8#tZ}Iw52Nv(!aY}3XV9m=c1+<{th)L3dP#Ao0TiKR42SA z-@QrWm(aDg=B;XGc+2$TIhyl|k0m(W&dG7QXZvOq;3d=fA`5H;W!oTJYUyt@>YDGW z_ZS=`j0ZluRG#S||IEm91E_q{)Eh3^G}}B!%HXZ=^CC=xOQ`&oq!P3M|g3lF$G*w}#ietLlX%C;+%YM`*mkN~){4i{w@`v<0LE1m5sT-anZ6Y> z#S}XHr2IBb>bR%Msz(du+6y?F1{9}@BW0|OOT#v4!p`*;i&YO5+{)_Of&q9R2wY7L zKL-E0e!F2)o87(coP>sAc&GgFq?+J*LDQ;KeG}c_9{uP)YwNVz>huuxDJI%t-iG*X zjdY5{^m}K_jT!ElF#y^Z2HBAy@f~YX>&T?ynw)L8BZek++PcpflAgV;q&e1cv2#q_ zEy$T&-$GB+H`ew7^azT_>Zg<-r$Z4|?CqlMn8JH>ML zm=F$rxy*dJTAgX2K9h^qX4mU*ZKcPxEy-ly&}6Uj>cYdJJP^J(maCL|8}iU?YTN=( zg*k4Rmun$;0?r0^X5_xsSE19z2sw!qbZl_F)yqug;PUWQGA-c(owX5)euBWfrr>&v z;QUj1_}{O3JZ**_FZassbZCcMB>(s>yN`xx9Q)yMoFfy3A-(A2BH9ryli^s1RysJ_ z2$$5b(lhQ(Bfyy?NiAw-I`v2JZ&PRQih$zWG)?TR1x2dS4%jtXoiy5r=aiV_eXFe=fkV# zi1E{7OJ2qdi;dtJkJi)r+%#q~(+E>e1O}WW6!mHNBEV)fXPQM|)AhQSI85NtE7E6< zFK=vKbV~@ujzf48wh-1r#b`%aF|Nwuij8Dm>$`eig-t6W|2>P6KB0P0tFQfn7b7^Y zimxhiX$&7h#aNaI8LXS5@H|{jq&7nN0T0)d5zgqENQbFyP*N`@iD-TdRlse5qQtkU zvUyZ=3jkkGnImuHoe$DJAgd|X(kk>$_BGGaD)+C}2NW5$H zm44^P?ar9&Z$`%D;@8g^uT%pjgYtgt{_>aKf5Y#fUsE`LOnb@JyMlJc2c;}aM~8k& zKH?zI(o;`-Pv9IVk#n!>Xj{K)w2_t8kL?F-;s@}}m@w}7?JgB?x^=Rx2?f5QDQxkH0&J?Rg&ha~8I-8g%2d>K?UjD{tf1`xCZy zs{-*N1acYYojf65Ff@0Bb_-ug_vuzi8eQjMw)_+)?AAOi+RhucfP^nVIoU2Mxdj z|GN8h8;7N0S}DX{A+E!ZTSauek*e_swH1ssR*E0lB0PAOG;Nh!o_s%m(5^2u7K;OO z;ew-{W)9|yOH|vZDoTt~R=dO&b;T=JDJ>4Q!7pvuV9wLQH*Z;F4ibrrY#C+FWB^>` z3a9aw&vD1C-9k)xc8)THUwAAQV+=SSk9 z%R)_7_Pc*@GoIXDD zu~mWuC2 z8H;yrBhZlwZ5;$P&npCPatGM1P;p63v!AJFHEs<2sZbsX9~^^G;x$Jy$5YD3E6M*= zSX99*;Ta<_UI~kzsX=@ZpnSZvt&zu2S8yZY{HNm+Hmic2D!b}{8k2}$D?-f-CAJ)D zgXIIUnUf~_Vp3+3+3n*Dc;Fu|T2)}jvAi~A$~Mw2h{)J-{zMgb5_P>%n|cbL6? z&hj47M~>%~x-UpPTbez})g>-hpgU^q#{2UGzbM!v+NMjDuIM(J+^yZJj4%RZhjPoh zena>rTJ{{&smJiA2=s`+dm@+A&LIj9U0zA}NcP52{u1dSHXiZqfX)=>SyZE|_@~|- zH+>eZgayGeK!+wce`p0fYp3m-*7FVL+LA2Lqq&aSkzK^|G(Fxbx?c(;hYtzNlL#10 z?~^*Yn0Gu8+(3`?j6&~c*WPtmCH%#=^Z)u!BisKWOa6bHhOQfR^#lY2L?%6h6+2<1|E8Mnd#acR?8$NW%NpIq#6gWt~+p z%dpGm=B}>nw!Z!{P8n#69*TC9dW3DLjuxvvCXT3}%Z}1?_Z(Jj z#lQZmj6zwePYA|#9FeBQHe)b(n2TCxia6LEp!q?Z(05~gQB>i4l=4A@i~ehQU*g^~ zfSNOUX?=^qjj5}NZx75^33q(?wDYjR^H0w6jT(6Ln5tKR*|NGnjAwvI`KNsO2&y&$%NyPL|tLqDu z-q;Dx)XtC2sQf>qQ0F+M%>S@G7dY7Mu>WCu4uuTI`7uF2_DF2J?Y2Q>Jvc&vv zOh~-}^6(GqI9a19oEx*{VsE${)Tp5T1taY`qbg@YaFL}kF4WPw== zFDjd*ppuc3ET|yw{AS}W3wk6wUBYauG-WaWb;z*;|i#K7yNZ6aNK`~3H!WoF zjnMLDHMZa+pH=g-q;NQ%wJQ%>>D|%$-g|ZKURTVcX|QYaThTdRGS{25-B483&BUtN z-*1#OufuvoReG8%kRLs7dYJ=Zx%o69vE^T6OtQ;p)HTICvv|aJm0xRiIVwh4z1@0>rg1vxiJmY9KMRZE_a$*~@w8rH>Np~cKXn?BC&Sebl7Xr z*B;tUiyYbVYw?bX;6dd=lZ_3#5y;5=cf(mZmcv$;x%IY2#csn}3|_A+Mz$KP9667< z|CHtIlktu5%8@3Bh^vRsHIc#MU^C+Hx&V<*B;atT8$%5;OAZ^XysjJ#za!e2#K$+b zdlr!_flERRDmkko%A^x6wx$&(*G0xC$y;f7ocq;rhprr(&1^CZ{_PY^1aJ!LkR128 z{%!Kz$9-l{Px#*^Oa7|V>S&B6zrq}YL7|kW5z8-c9FN}d3VXAUv*>Ufv(JX;F{|4? zHJ*xrf9hx8^HmV_XTc(vC2MwRnrpGdwb)77hW*Pw@8;?P~{1ZZmR zOd!vgIix4H4yC<{ku_=U1Y!}r`7dAW`lYRNq9;{$=+SyH$ACQaOy6f0GqWZumeGGj zBtD!$;!g`Y4crb=<+8BgER!fqSBt78d4$BFVJwt}yoQ~>p2o_x0!Z%VF?w^cSk`XA zsU+Mf(wVxk#AB8oMnz0A-Z*>Dhvaxm4NMFEzV7uAUBk!H0{T9cN9vvYG3#2tB_a~S zj1VY`*fWeJpMAabo&E^)Qt%;VY2^mIJ11d>Jt%b|m$D%S`x(&MbHGDE4!+dW+7W7{JG<|H;H`XHW`bWR#E zU&rs_u=h#F&U zSjt<#5B}qZt8%5A{7%(I;S*+ipnPYMI#e%EY>S%GS}bD`bG|k+?UUCL+ zOB#)m_nZgD7MD5qpK}mG>HNK+V=z9`Qd}8(2D;wB~*+%K!-8xhjCW> zrlrn7Jfd9E5scOF$4)%gV2iFAEy%s880yXJbhQI7>-`=%Ki)OY#Q`)jQ40$d0H?ZAD9uqa^XI!8W zwnX_Mh+_YN=cC&Yk*80&sHM{08qv37ilOc*TRvT-C#Eh^zA1Ai)3FlmRGD7!YW})H z{IlGk5u+LHozc(X+1^_Ke)8py`Nyt}T1Q~F%WmQm_`&6*X+gw0pO z@T$L`>??!1x95aZ2>Wy8_p6p@Z+6+GE|Cj+4$X;Iap(dY$>Ruu+HoVGtKjje$iJXX zXgwF+?@A-6QD$cDcaaz|s#}Mh1J`({$?<5wk7O?{pMQ1k`{}P#U^xN72jz^$Bc25j zj=&qLxL^2D42HO0djzSy=5KK5F{+JezdYz>nwbaf?*S2nn``1#HNGGOI-V)S0#Nll zahsR?Gpe^t-U>#IK3LhSA8jWDTs_L1GafQVfHI4Lt>s9jAD0V`V_o?s{!2D_(chsB zqO`e@C)ro?-z>Wrtl(_RF9ephdh}0@1VDKG2}8GZ1DAGUCZ{u z@qKO*HR#uCMK2$%Y}^@_%Zms~xt$n0Yt+f#uHV$l`1I=^zV}j^6qbHv74;|NNcs{nr1;Ni%-ga4 z8xu12vOIKMA@iq4$HA|ELk~2JDe3>7N&cTH=-&|JkIQt*{|{K62Twncp+rv~$l^v# z$FAZ1)~@JFr1{^7bV3_EyfPFB$QVj`Vj~B7x={@;X8R^Lwlma!kmg8_a={{E5RhBt z^dJ;?%=83RMw0X|94IJY)ECw(WwrC`BQJACjv^Vk2?UhHcz&oal)fMc0wy$&j82&L z+hjtFoW;;g=%~r6jr|U1i^yu%vPKPEQs`Kh+v2wWqK~bwuWz-!UF)i^{Z7>VSMS+} z#UveFqyMJ>Qsyo19{24%@2yYY{4MY&wmi}*%qP@3jFV~g7B(Dsk-LHO^;gve`%J4d zS~x;dw10fx_RA}KE*Rg@=|&@M?dGS~KM-0_m|>&5>AI{fbXwR!b+EI3Y4LdDi;Sk1 zOI~I$z_)vXNKP|j%1zhEp-%7e?8?;0=_Wa{;oJVH-`8+wCC{{Q0UOqvmynAmC%VjM zmc*1>Eel#UPQD@rp24-)cN3^{1{yEe0*+6;j`B63n;gqdoiMVnF z37_GDo6W&ldWi~OD z5?ffd)mnXtTQ&Q{)Ki17N^84byQsp7u@c=X|I|smw8hZB*SmTa#FU9pttnX+2$$w) z&|U^5XnR0CmL&FV=RhdD{U1&JiE>l0rcyt?-c{GUioZtV@y*U${coq^&tUQW^mdDz zOZHaV>`NQqyXoOPFFf&@PurmFU-~=m_}^z#^ymiM`Y zpOEtuM%WNVZT0Nz9m_(OIE^K!F=>$dG6u=JalJf5VlhXS}}m8M5UQb_DT4koTj30Mj=v zD;l8$VvRkS>=p>Wjqvsav^(i-Vf=zKv6J(;Xzaw_O^^ilE|XS20Xtq8BcHalU0Ypu zFlctTY1*1~M{t^bT78mxJNj9epd~XvSalZ$;ZtE;egEEjATD<$v+R0)#y1@tR5u4% z89U50St@lAw4Vo=J+;Ov&z=0&C%9>_@mn7p@Tum*aXh`&ZAD(|p#olOtn*NxAb5$N z{q4{;w*&@u0@vY1hKEq926QafAv46}G)F8H_;G|`f$Pc9z@-1;xG~iQ+L2-R#s0Nr zkCTDjl!`U25darNjw&%T&OlQ_oaxne;)dbCb4Uf!ZW4l!CD$xF4MkKCtuuL4g1I#Y zu-*Lp&EjV3nUva7;ipL0coW*-Kwry>>CQDN8Gtog*jXfC@@rw^-lrbu7DBg?Txmly zsVQC2!t_vy=+a}0Kb>yUXV9_RhyKXP&~R%O;Sf#D6_{nCzBxAU8_;NHENtk<+pMhv zg;X6*(h*C2q$<@y2(2%mWxZf^!(h1tHc9M)eOr^Cm)IF}WTwStZS~c}Q(c}QOKP=v zbWaf5w=aA^e$4zJQ_<1X)Ho~penCOGBgENlRS3FVi)JQFfjybWKZtoJY%zmb)|a-gqtoa#`d> z0kB}N{M}h5ql%z0l$D>(i@k37UhC=lY>^bxQ_y%Vnbz;z-NCs89*ZPAhE{x0De74> zvDQ*K0ql8m1?7+zI#cCKDOosWJsaUgQ0G=89`9Mn1f+sR!YxoNhq=*ZOKb<9?pU)N zZEsAMO(Qz93X=a`2-myLrSGBvQ#U;3?c;4TOrN;9z5*AYESXe8rV5~g|;mA zAx}X<<)u0m)s(gvlk%OhH zibX4J>)GzbBQDC@8$m*SvP*G&$yneNv&I8dR5!CCL)5NPCA!aINInYYKFSK)IhBt$ z^ZS4qOR?R!IrH)S%MO`A61`&l)%pFwSiA2LAN{Gz+PqL?Dh|60Ta9g%)!U9{nIucv_v* zQ9I!>bvd2^(FsJGeX>IZlX#jCp7AIq7q#v|NnuH?@S|MQo0W!6NE+oi!Y%U=|Mmt0 zAO6CFoyNu{EUiTNnF*HJKHXee2TVI_O7m1{w5vPv!JyaLFT_m!{GIPrH|O7Oot;7b zRSB?UVy@P)hXjF3A+qD%nWIAz0*dA^)6z{9IIqfrtyx&%o@rTS4khmYqO~Z&WM|Z*75a@n-beg43?v7MTlwBQq-O!p+;@D}`lNrd5 zZV(<>g-aGcJG`l!sEt_-k-=5N*KX&dykgTKJ?SnxVjE#4<1Z z?U((4&~aE6iVA~)5sVsSP)oS4PBl5P8T8KQrBg3-?W2ug4HfP2?kV#UPf5>!QnSv~ z97fW#&qe6B6|;o9y|<0ohDEOI#ks|(mWu~CdT2N1)ua=cVE3?;Cz$Xa)FfB%Bv-P7 z;23FBt8_r%dzIp^TY4hcWve^ypPvk7TH6dGHk}P=mkzCa3fX5Y=Gh>$F_p}}8o;Ao zcsCjz#^@X{yA8av?(f;4Rj3iv=&ng9Pwqz(Hqk(zP{GyEwYay8p1o+9-Ws^)*vBKCVpb;=y*it$Yb9(B=hHm3t8Ag}p@N5*O?uyegA>ze%w` ztQAD_Qqe*DO?=QGE1uILi$LCoURn z=Dj}LiGc5kUJTIs>$ATcL5jU`b=Zm=8ZeXZyr_gr5#~w`7YRc`eiK+ib}k{fq;vS3 z?nMF^2#yZs%lq6pg{fNb8iIdz{}iUu&m+-UDId^ZII?|v(Eh_M!AQJ?S(XDmMIWR< z#Z}lRm}kAL=rA+_Y4YaKFBYKsVSb8}iezRhqnX2fQO<5dYRwRlVkJQk{-dSSe|4~} z4$=N}G0?}iWHv_B(m8#7od)}t>wT_DlKCwVk=ruM=*-lGncb~gB=AhGPN#s&B0ZGy z`~a$C{>Vc5akg+A`7$-~5ojP?Y3W(+pd^DyPnW52y8lsXgn%uJT5Y5XgC#hD1p6aA z*Drhy*1`x40kMebQjuL@G)V;-NR>_vS~T6bW+3{tKYkz};Q$lj7dwZfi2VjiKe+)c zt5=++6ETWRe2woN89$o4d?BFuZNyiGr^qU@keG$smF~3w z|A%ns(zk+a-c8KrjBtY3(`jC%eZ3r*HwaHHdzP#CP5SAK`IP`HF} z9!pdfXp6AGOhq0)UcX+73I#$OvkZek}n{wcuyF_Dw}g-5j&TOW-^35k*gFi49WI% z`!PhW%TI3bH3!0Tm1Ga139U3~MK0i0hxi$i3@{2PEgU|#d%bHmGyPbPo&DP!ZG!;r zH9X3Su>Dybts7h2P9iQ+oiq8tm#0vS2*P*!dvDR0cvj_*)Od=vW8l+pTDR9X3bP=%Lc^H0jAtbOd|p@+)v8>TQb68qjqS1DL>*E ze>$JH*OX|W++B&vOY+jK_L$#8x#-C?nKiMiVZRP5vAVG(Do**Yj zR3B5b;QGbj*s*i9j!6jY+@d;Gh<$|3V{M#RyH~)K(_nbExqA)W1ck1rTYM(3_+{m! zJhm-~FIz+T85V|9^In;IurG@rJsBruAq0u&>+<$Qc)6-@>RZkw+Qenimc`CluHaB? zW2DR=-Q2Ysg^*BU?AG@2GqfheTU64tQprOTn>Te0NM}}UiJJpz3`>kb3;*n>_L{A7 z4+;Qwj)z#eW$R33TGdI#b!-K`w<8`xXwwyK2_&n$3?~OQGJ#YxIG0+lPO5l(1yrvq zcrmEdI@e?=!%PwQ5(4)QJ$;^%)z22|{o3Geq79Jy=(>kusFC^bWn zik)TCYl{dH@KC@~wP$774IP(@;IDbiLN2vidAT6F%m=?h=s=BPfuVlPj2$s+y&%?bdN(hFSmpMj3YLtJD%C6rMD zkH)I=aHUdJdXwCyAl6sJSTQ_P_xU{C0dA+fb{4JffwB$HA}u-H$0mL2MvZqhuLNmt z&gkQG+pYVZElPM65`)-;clppu12>k8YyMc1!M*c6M)gKrldCeYD3EpV4ESS59rFBrYiy+!mP?`EGQe5;wfdSP>Ow@XDc z;6IG%^$hhU?=O|}Wl|00y;xfF6{W&cnk}AL(g!4^s@{*&tF2+phZ%L+>RmjuP!TQ| zU-RZ7e#0ln0m}<=Zvn)G>s&YQ@~H6wZ_-J3T^2)`^PDk}b5^nNFq|6fvx;?In5A_C z^KGKk!;H(n0O$+W;8JbF7so}KCv6klisIQV&9D9eCMNCN*r2+C5tuu0&D_0P*vo<= zSU_7cnS_>jO_doTnK3ZJsv`&WYLDb+bC-A@+}Ms2E{dlh>dUC}%h^GI+E0JjUy|cH z@q>T0aLX^^I04E9F}{9vLB>&jp5SwT$iru{EWF-)k?50b2tCHMFxYyHfbC;_k|+*8KusULq>IRX zWya^3;a4BnC@$ngzG^yV-_aex;-tt{(Nh$Un??Dw@MDyJ#Z3&edi$>Zr^tjrjZ4YuzA4@cM-I?*gX=SkSj;=ou6 zRr8Ht2@_Ju9kcD(ZKDawjsL^Nt&<(!ZmGEuaVB*k5EkiIRlkpkHSZ(UoINw{rQ zztlW475ixNuclw5u`!n5iJKzwHzniQcpP;|4vHN*z zyXF_CeU8(yy~X4CWEs~@4;?k+eS##MMMq)Z32P;W&1xl}IPpqi-qt3=DczR|u^#@! zEM|WEDlUpzl)RWldrVV5otqz!m}|c~D2l;CcJVp?;s~7Mwd13m^2)Ol#|cNu;{mxl zTc*(wWgJ}c8~}I>SPrxiJI_eH;P^2AslihD=`wWD%UiqkG7Q zdm<6u-;+M^n?4DHw*-x3ZiTa6h{vH5{^loqq8|&wY6F60J2P(F+vm-8<=Gq3?XH+R zliWVjr%Ds4pH<2|JKn?2DY=0(c)I?}7G12O8OeR83y-meR_rz5M`!C9JYV~|K5$! zxa>j|v_3*+fU(am{m_(H>a+kUHzQiuGU$NWXaJnCQNOZ)k&qyUsmrK{LQd0igC}F8g>9~dJZ^us5`CMAaR3OzMsl!9e+Ep8M2e~> z;S*k;a(_?{cR7uGssi|(gx@!&UE?480vnyoid^YfAaSKlK^)0FoT=m?CQ>)7QEKtJ z<^E7>O<4r?VHC9Qd(-BaewRP>Xn8cqng+Hjqx^O3GVBf{O}NXEC+VGRN8rviiS{rv zxsNsY+N@R*V#yFKwGibmPexB~Dm`1Fb&o=0B<0r!8=y>^)$}BA>R6+K&b^bcy|517 z9Hx%>Q$~G*u1^wgPsuG@5~C2RmND`7g7oc&kFk^jKmf66 zVi2Aw{_!mCaSohdq(<%?S36O=e~y8u8UF}*FTQc+5$2JuTUn_XHKRI#C-}enMB1S; zZ^+ObOzSPyV|o97}aQ$Dp4M)XZo zckV*_KG?qzB5XKS2=*W#9K_waV}ZFN|Bd_ZiWH;;xldG%JA$sj+fwpBAoOR@m3f3k z@NdR8hxH2*pEUYIhRYSi3V9`91ng^bu(S7_XGwgN<8@8w3hbn36=F zwmnOFUY$6P59IvtYtO|!mv@seSJi}y4dETQdFxhYEJmhDWc13uOcJ}Sra(^}Q^$9! z#{mK#>g*2_j)Yh{2z-W^OCs@_h-)T(N@Wa56)Jy%phL*&S4SPNK&7u+944`sRW;NlnJPoDMI17r$47uNw-0Y@&(5T`2qB9yB z7fkFsRXfhkH=ruJoy>-fW6IG^uY--_AwKYrVuUOZhSQI~ScR<9_#s!*E}?y9wMKt7 z@vh$xfK($`z$`>RrZZ^?XYpqPexsCY$G#G^j!IRY({rtFBW z1Z|x)m;110U|`^^lD18Jf-oF3tL2e8o1N#H(mZ)s7001`<%uS6oom;wjYDvXo1oHfP=7k>xF2l5vnBVPao)Dh)K+~%zRH<{f$RQMA&_rLR zMrSz#D8Cp7Z#Dk-s4=uw54Px^s0v{$D6p%U@`m71g5F57Eza>?_t2_HG&$R*B=*#z zdt_83WvVA2nYsN zI|P)36Z%^`Qq5#3(%BEnaBLXR%Msgtk^GJYrtDi)GuD5MZqfzrvXlku)zWMHO!B?paXI*ql|oNk0xA zyHaMmLB7a50{-CDNnBMsV)4|xEXUjv1bh;y&&;Qo$2oioU;}+r4-jAI3`!|)cwcz# zTFMrTUn&H63@xGs_aeHSUox_A+`Gcx{vvajy-HBtF9$UJu0J5pR7uLv+!HZiu2Wco z_+->lDI+4dBhDM_Uv$=~`_$#+a(UvyP5ibcyThZm37kXmOvraME!#hWd|^Ju=y_np zLZg#TmPJ*n#THRlg>qCjwrIx|%dU!T70MXXW3*_>msYQ`y$5GWb5>}(WoOBIR?2iq z&WacJ1-`3c@YiB4wy(Tdm0&GL+rakL*j!+=rWmMBpVwE#?fz>BPewZ+h{#fIp<<D*}lzWp>3#L}UQ>paBsse6V z3Ro2b?MLO;4+swyx|7zFbLdQPG$GzRC_Gy-q^rAEdMLv{LpwIN(5(-<3FY zNztXU19sFE>j2iCy^3|=kuiX^3Th8?ME%`)YU(+DA=*hfOb?I>S()OZ4OAM-RQsci zK24cpF{ESi?8VJ&mr!AVCMcYBKtPUdIM&Z~p=mu2s;3rj5| z<1JSfc?RRY{-x1($op3i1Ojp~zSoBv|Ch;co%xI}^EcgC=3O(4`(HOFo`tF*zzK4c zYnknZMAr|4OHI6MR{DDr^y_yiVcITr1B?nvBmZz}ReVthuOx{I0^CvsB0K+(vMR?V zktep*3QJ@6`DL3YF#U=?$*a{DV{gHt=v77JsQ?+`CldWqgEHGy#tVrj+b;25k@l*+ zfh~hoBFqF@SEqosVQXIL;Tpe00DozcqIdi@+xR3f@}g%l(p!3p2yGLIy-9H?PC2>z zt@v-(;@)CyMUQUf2#2#l3vO2V5$6?cule;wSu54%cpRK#aMB@5dEl zLC$c5tjU}dzUw<}17FpE%Y#xA$d7r|MFGux`RbR#0ca%<{J8gssFiWG8ihqG>)W0(XO$_B%`x z&MDsYeR@qF&D6dm{noL`WN66u-LS_fW`1ZvT``zDh~@I>9QNDI5h&N=xso zfN#ckJl>P!`bBpR0M&h*X?1|P^*IyxM3OmA@aMOwmYEW~X}YQAnXr!)2cI_&wXmet3x>cQdVHb4X#X5`4i;WE=T|1?f6!Ue7$O}4kvh5*#`^Nb zA2V&4cc6jQI%c&3C1uRvq&l^Uz9ZJK0Su*t9k<_RrdS9I5aQKR4iKqrTS$a>uA}nR z#Pc0a!lj;GD%mcJfZ@#cARXULFMPzM#Ykrt)ie{0Wua>~h%bbzP6(ETZZ&2j(Xl{S zgiMmfQZX4Ych=U<_t9k_2|qERY0wiHrqi_!HD`@&u`6`O z-fXk+jmH-UhL!SeJ<w>lYS;6WVxvr;a{bWlGn{!bbb8nZ@0}jcdDjcG)Q;)M z`s8+V)hMc;U>!fpOz)st7xj7ue&YIikCy$DJ{)fIcQ@)8vcCu}FZ{bTqE)1v$?ll= zTG6qp6-%}LjBHjqMQ+(K5LAvMo5$U&hktAr=5>k#ovIC;*Vkr0t8|h!-O5dttN4^T zXokY}{QlHz*oo@r(H+G#hZO01tI`GhzQ#0+QXukncIi&-!G7vuw0Ts_Vjj#b?^=zU zuuU^vNzcx%7kQ)kSMYT$mt+*cJ3m*u;nK7L;ApAud@PoSFWO=GCPQ!N0* zbf35Y8h!5~_5X{kcaE;)-L^nGwr$&X(y?uJY+Ie(LC4mPZQHhO+fF*{Brm^n?tACn z^Zwam*BCYSt{S!0T&uost~uLwQ_6VC?A`O+goer^Tbx1=%~o|H_0nL5v&6}6_;B-0 ztPnF^d^Z;7M9PW>8(`h!!76hX8@B3Lx)wCfNC>Vk&Wz%J*~~krXR_;wRDUoH|DEIj z+&}yX$siiP{xbIYBAEP#0`kB4B{2V}=8I(N)20DqIhphUx{7GZ$R7&2dOFDwN$sII zC3Yy7=Masd#K3|A;doTS2dh=W-5qyeh5*G~LWj7#Qa)10v$34-EfkK0$2o*mJrKE~2LOb_tCUUh=sX2Tf~!WSwJ^ZC4jVtG4zU+4LNm z{bC~BFy;$@tlqqwxq)(1pD}4C3vy_t(c~&M*p63s=zWsFo(91w*mj!cqp<|H$Rz87 zX>>?E9{jDGyQn1YCJp&aem)#kLTGLbNp2HF7;O^ZW7}0qDFTa@aKbbO26nx$V2PnB zpgo!mTP7E*|6n7jFG=JQDlY=wF0oPL>6AM}S?HEuwEZSLdQWA^9itk270FevM_kA+ zEHtRu@~-UsWF&YB-|aj_J$~`0n1;5rbsV^rnspMYtDJG4BsM$?igUG+DmtTvtDg84 zl7R_8N;!RL-_1%#j5mH%kiYMht-K%D?W>Z<`PXSv-|C-=`|$N+J7@cgJ-6DHDIC{rsC z+9ukAy#?l+(L z`*zsXC*I(3AU(oU!24Y@tu!8+R0^W&6vE$s@udD0t6yX&lrOT=|00WjMR9W(57YO5 zv!Mie6{W+$zkNG{1_A`&iGT!Tyy(E6xbS#@R(i^y2rauXjzMH<|Fv+2&<#3{hQtEP z-gR>qSlvGR4S{vcAMa2Zx6_Yv>=xekVQ`8d6R*Y}SAQ(Dzrj=Eb*y1@O^wOObo{j!Byx^0aR(TY7O27id=qX1 zO*IH6)WT(sbZF_~N5e^&^oEc+sMrIH61-Drl@p^-za3n?38(d%aEzsoLUrd?Sf;iG z{rM_KyBIT2Km4C0pjo*~9`5J`Cwb%gQ!{@Q<))P7){)P%2w!k|U`~%If@4ckspyCw zi&+io9@P6b)Na}x zb^l870^UAa;Nde2balE8Qky?~!s<_aq~%4o{?oWz6zBPlZG=pSkUuJJ?&3DJR5I*j z7#;Gexg!MqM}zvFJh?E-%D0Tc{hmXML}m$c#DyhqqOZMSTf=OcoBu&Uh5e#VNX9x=?e%I;En!Q-imCf)%~Qa}y;CE#BgKCnKd&B17{S+@)#2p1MU4o1|IP29^wY@R6vAxxwr zyM;4-II&uP?$F)k3sDoM8^o9Un8bdRnVrUb2UZ!t9$m=#L9FHzw}CUP66uW={jfDg z%)x7K%R?+;^#dRmLEe%x+O()s6*+dXDxsT?=tVkU` zYssuIDb=kU)8Hpw!u9qUf70MC(4s{?p(wV%Ou9RE_6ZB)VvbK4g@9QM@KtI0G$;{q z$-XJ@Lj9G$f~BnU9npfHez)OIGR#ZSWTLW%a*wu1@0X$ShCuJe5T@$%a4BgEDYR)H zs%;H)Cz@Rz$ENrab8ON6E~C9Y0VpuIBIYs63N;@t?VV4t7i@jAbu? zzCgV1_$x2+Y*nae`Bb$2Gv$LQMRrkj2y2%3hQGr8cR~1iS4D_8g_PSSc9Y5dj$xV6 z7Q8U>I2eG2_AKc0Lo8j=mvo9QW+osBzuQcW+b5xu5pcYUxo!hh(zJ}EWCIvzebP2c zSk_rm8t~Hl#yZ`HtaCA43-R;#59t|O50u`tvf3Jv(tD6PXXA8BTz`b9oP!Qae}hz2 zrihSIQMO!hraG^diLPO$K!pWIqSNm2zXE57@}Effiqprh5v~7QjI9Rkg^0h(H1Agx zW(Nur!Q=l^gzqso15~ScrbAxH0YRzD0$@IUKGU8DD1I@#Qf+5-u12ISnk)19@n7$ zDzsnOIg=ba3@H!VSfsKRAoXoMO`()6dj58r*RGW9a$(Ks>_kWbPVmIzN?#B4i-Jix z)sTANcYMp&eeKkBpP8^M^z#Md+x!dzGhj0mGquf*3!TE2Ob{pGAIvTBZH2z7Tb9^Vby9$h%J#6F_%CasZpAayw+1M z8S_!F>u=}ZJq=9up-4EOtBtePy7jgm2EXp8su6o)=s~LW9R`LK&8XVcs1 zanS5$mkz%O{i%@|VLV%J7K@we*X(l{Q9iRoPxWsRZ+^ip&og%aMljf(V~mG#wooTO z)4i#NDJ#`tMD7zCFzgtZc*ngz3e13AQ82QL=>uQ6WL#KqR)qN_20D~_uog-CTWUEJ z?9|Xj>n4NTK~*t{1wAXVhi~EJlxV78*r!cJg_Rk8aP~kbW&$?y-3j(xcxf$XyW+Px zD|dF|pm0p?yo}9~`O3jy!YchkY=_$HdSy}?SqO9Z(7upX)<1c1Ddwke6KeqL&7++I z%?Abom$C7~p~-$>J3oLmCJ@n|MMHVw< zFYBCuHw9|UePQW8u|0`Q4GSR@pv^e9;>j3|m^=`(l`0nw5VZ9*=;Pd!r=e==nDz8> z*t=iVaxCVs&e{`;^6U&Z00&TV)PxyvN;xmrDwufHX8^Mv5s|6&dokWVs%HRCqCqk2 zr<$QuS`B^0;J#qRzJBHWr8OBU-1&Gz=+>7Tt1XP7K-bZ*iKm{S<&3aoihjbe5#n2) zTX#pSn*(!&l-tj~swjJodnzchuoi-f*+3G8iP^oS;F3b9){9#`hj;*S2(NKUhRtms zd=NP6MM*i|H~lh$XarC~eaIh%o`BswT74wZt9}Xh#uyfmAZ3~-0n#0~-)@s~1E8U8 zl3YV?*gzbCyza>40n9zt+)LMw$@rJ%R}I?PW6Tp2290TuJxhmP4XYK-E(eF`KSq3f zrl%Lm8Rc`!g`#{lY<7TSTdU*1X4~Vtpsd*7%Ra6`BSiz*qK5(`pA}R=qPUbz<0okJGRX-jBjlS!y7xe%w}~A5_M4yaqD&J;lp%jgfen5f=iiX#jIWF zIN~RHMbc>-waTumqbVz5yH*@`A|{l$aV;X~+}Z`l)OnQb)7lObWdx?C@qTar)N3Gd zc0LG<&EA6jL=Z-d`H0lLZ()-B3*nJ1Zr{rT&v$94aO(jG;tSl}2$+wD2Sx3T{D$-Q zusLWv$t}RG7V?LCuk46GOzpALGHx>uP1wTD<|cJPYu#p)vF1kkl-sfSCfAw|zDLF| z{wmSsgsf62q@HJaa@9WutQuV$#E%|->47v7IRA zPaK1of51Nr_}jA|$z4g*=QEsI^Tk*ULj3*So5q+j4k^2wn8nzkT87vbQmZm;NHc!_ za8N3sf5w(17x&ih!?+_E^jT~hp5zV1uw+1$?fWAD5Q7tVWFn!MRb4DwN zLk*I=T*IPsI-t>%Qc9^_W=>V1y}lixxat;nC|CcbQ&<%n&-iuh zk?9SHb8<2RCT6itqfTolTL>KE^USLrW%hVHk~`#bjk2B!oQ=Tn_wApF|50y6JkWc) ze|BctwRwlgFa)#f+5$`vQjdMun4IB_xajzKtrllN?RR|kDBh?(dF5p6Xg3jQqxU+p zvLthw;A@Xi;fW`HXR(unx%^IXDTZ+=!f^?Z2z~4+=;Z$XHZxQgh`0yA8?iD#Bs3#g>J%F z=NrY+Z8C9Pn!vWld@M%RIr)r)4fqYvS&*|<4^dXh6tlCY8Gl)*(ohM}5tl#nw$?bu zuK&%Vc(;Kv8m`-eaUa>ZMfh9g57P?<+o!x9nXKul`uJDE7cK%*CV8}Ihz-jcL(6=qoJOI>~d@J(Zu_( z%a5jSQ@jqoyB)7D$J_tB5d5e0`meSQM6Ox)fJ)$hCh&VAu?7BT0-qlXGT?u{Iw-=! zgZyjgOn8N5z~-y}yM_YB*+ZZMXDWF}0ZHoHs%VFzpQ*A$N4Ct_59F|gv)P;?t zC7~s?C<{u|9hK?2+6Z{&^|fo=`AlxF3s?rewd$^Gsjo+(pHYIYuSDFDNm=bM{_eg# z6XYJ(oyS?%zS|yGM=?*`ABcwV;pRl9`{|ewN2$O&&~KfTu23P%sq#=uDSjTHq z&JT+ENvSJxAsE#&pt?= zU~8j?7D!?;;7e4)ApFE^n;{T~do5+raw~sZE`b$`vfFOS(s%DRDMh;C^fp?o_AP(F z#cp6$ig91)0~t_jFaa&P4LM^aq%zF zrX%9lJF!df{?lZ{C{1r@gZB;R9xIgl;ba=kIih0Es)& zB5@K#xQxzSrD%z_4gh3%zem#cq^vTl7P{f{VqlLgL=p}X)1i=k`Sg}^WIAl`I?AhC z6vict9YmLf&*Z{@DbEHgpoq(*ghsf-hazq@DZocC7Kn2(#rU%rPoc6&Pr$Qqg|>=M zfU?LA&_{qhhj&oTydq{oRxk`O;bE^y@VSa6C89Bc>0UDUXhB8sy`=G?YVo3^ zKO$?XPGtUuK5jF2{YB00G%rOBlh63!y(HJYRc2bRE-hzl?NB~ILg|JLGm91;@qGrZ zgB%NQw+MGPkK~0;&+k?q^b^eW5^y^7_+wHpnESDghD${GzzWyPRnc_OHk53_8oARLTG}b`k~bJ7&Nw|f zL^~~XVC&YwDq_*TA81-hhmUmGAAu5KzxRlSy$0DZ!h~J!)18XF70k}7Qz09GC%fAe zUCr~TN`cUi{cg%N<1cuHQ{led{ST!5zgUEN4fS7`Cf5v5f)9fJmwoX1(U*DUmkJ9u z(D4!u1L*h$3lEfkM85d9e#kHdq$w-{3f-HWjLUnW(APU}EYL^`g$w9s2agPh5?iBI z2Z0c`7nk*y7}*>xgxvx+k-Z1J$Uy`;s==VavI+zki0J7^XIeu-Vara-{<`H=oVQipuwGQAsSdp{bV@+gLi~)25QDAX?P{7NFCJGrr0i$FzvC?Hab)^N=T^38=NPG@%lQ#i&ss2zYTlw~4 zx;uvLx+Tpo)>h$#T;5g6*gD1uPqtDRZ^bu5WFrt^wl)(#QFnPQvu29V*B^TC7t(_D zGA~ibEx^EvO~64!S)FnY3CJv0i5Ti2nNK7^XVHd`7&wYf@2NJqF27<~Yt-fRE5MFB z{nVwOzR9^rc9kP8-X{hio?42wN56CDO-wx@`>ctIPJEz>jV_x(vQ;z*3`i|CkIBQDZe?50z(V(Y{c_ zr6bwRs@!DIJyb8Zr6q8R%&oSkQ9AND`|4@pb)V0DH{BE@2loK?ImkMgPo>mv^MUIv zoM|t#Si)Own%Ib%^(h-0REMev9`qP(@QN08T98pfM;h2OFHgnypGTaau@gmd>o5W; zdU}!vhsjOmnaHUuQNzuY++L;*?m2bk4PnWbA1)=PRR*9_RY^SOeE_`2)PJ5ZAHevW2;B7aAY=iyW#ZHqYyCBUW^ED;T2? zxb+XC@vN9Aqol%OStM2blI3@>qGNcbNm$H@KyJzN^#)S|;?r34v2}(%$W;Q8GP(G* z9KXaYaAOhCH2Wr3s8;Mq(&S!b>7=gwephImTi;C@ zKiT$!rdp-_Yn3S*45;*nixT*Kksk@b!trlM?j%(^*>xp!Kl?Qfj;JP$MVxi`wub)!1*)b3vL6thR zl@}%&vRGxt!)_}mdRRu@yiPYVJvc>1(DX+4a^2-^+k2q}nRUG5DgyCx#9(*P@~KuP zT^A$}p?sDZ3{;9wqCc+~61X)Lj}iZaRR5{Dy7zCUjn)TJV%K4ql?yonLIvHzsKHpT z8~Y5n^SBag8&(ZBg>TZ^_r*`dbE+{exoBbCABMU=TsS@AxyfPsP22n_*M`2 zf;s`v6I#M;V4Ss-_Mixs-4b2tS>f$m`1Zr(HN#cvPhqw#uqy@j*E@EgwM*KMf@5c8 z;z$^m8HCvK^CuiT8jfUR((jHozk}~(?cY-c^lCZd?qJvvVG~-!knRBdFpe~EjLn**-QXAKB{yI>mxGpfY&fk{ zIF=6{X`kr-5Ym{Gk-;yLeNRd)6lw41AxTx2dX)!8%6kpFa4}@ZXKtobz+$H={jyJh z8spsVZ6c^LvV-M{P5L2UOTn;AQry*ca5Tia4O>3v6v#aoThi`t5nfgZOK`3=La_*t zvep=6v0HA=C>|H7YL^GYP(y$>ndouKU}lbn0TbGzB9Mjj8O-^bt+u^(_dGIKu2BRy%v)sw@SKpv=CLAnc zf%hBm@VUh4C8_dNs|1gpKN{Nu7MlRL?{Q$6Pn1=$aaZ3%yQ)82z*ReJ0T4wvupuS_ zD3^P=_@MzC>@b?<-f44{_;(jrE>PS|PF;-@8I-~iX@6j*se1$$=st$KI#1lv`2(2}X*;d^IY(oVaKnePvvMeQxpcVk?cJCq`VTyM*fAJJsskLyy@% ztn7coO*7zgC->neLqv~3U;&uW7PRE~pUWwspk3eT;k72aL4AI&T%S~R3jf{3J3;c8 zO*pe75th|~8^w&&e_&AT8hi#EED)V6-{1u&rhDsPTN)|tp1?Z6=chaBiK^&+S7?vs zsSFC&Ky)fx$05^`Mvv>|=}RCJJ9hF{vJe&Ui3yL+i00>!3#L%mC5>WfvuD~RpHwNp z%U$Ad(Y21W?J22_B&F=7F}1^05?O2^C=i4<1fHw(cKJk@FpT<`)cSa!C+82u?G% zm0@FB9cH!rq_vr?&RS9BdXfAZxjfNBDy7}LhUjmEk|P zw;QXgKHYDV8xB>E*`F_OLf`O*Ho3DzcmhhHX`OD8zOvbY~KJE57D%V{rU*{IvWu?BV6M z{P%^aPC#N(&0-9|LXyXiB!Q-2n8NlxKh@Tc|L5tfBc+gU6LJucg)rLvZFY9|c9*!N zm9{KdKuC{vHBW;KmIr1x%}-V;)cP#4>~yLlC_167l&|CD3 zfC1J=j{z6`bV@hPTCSvNYTA$aLd`Wm1O)9f}MATa& zWva%m{GKXQvsbzCl*MgmFegtMn&RnDPgdP}89sOI3T@U3s~%!k%9FG82(y1M&7?eS z-_C8w$16|Cao%d47F7lB>s*@iV6xEPYQ%_UqM2n*Gakk|;+FFe&50D3>F&+`5Ky zbLk1Mn1P~7y*uAB^_Bu&TBn=(c*m4IJcUOs*5+Og?d~?GJzsOP>y4qOD_R&)?!5{l z4E6nZYE^-?gjW1ONk!}6>PgSuvh}o)lZw{M$}|A4r+%io8UAod5Xhryu(d*AT`)| z1IfOmXZmD}*LjeHg{x?JkN6De-TR0ran~F#iZt8vNbe71mmdG$@Kbn2bXE*Tf zxLW|CglWNTUF3ZW@qV+XW?vPC0@fKhh$6YwdQ<4qQrl=}ykkWW9p@qn%ovBTu?+e# zf%3Ux!SO?yXeoIB{KJ}y*t5W2g(AA-lD~w2*jsj=kd=a!ju@da&)`lWmY4Dt%-(FX zzrSI=qEVC+V0O>NNn|7@a`$QH&bw7q5e``XEm3x*WMLj8jVl;C(VW#E_A@5vZ$3%s zLtWFdAAuV^lbDCa-E0`U$(uEv0{gM5hbviO`oc?*b%6o1TCwfwqH=E#4Gm1@_~BD@ z2p-F@fVZJ&TQTVZy)OFZZiQMLO+I3p&TSu`*}&~VdrkM~2X^_zrfGrRc;`ei zn09sKI$^Z=-|;4njN7VpOn9L+0x<|l$x}*lx9egi7~)|4jiiH1%Rdp*2|U%jFft(Q2DRHktrkOHeTx$ar3%|3dF7L7P?$>} z?zCeLQvG$#LI5+J(2X+(uoBhO3|yF-uHlDX9CDJUtTBgk2l(rl6|#H=*8S3fCvWT7iw8C5&eUUU*`~5awWzws6HZ zHN6zW2;mG0(D-w+I2Ra%y|4QxSstISt)UzWFun6*)onr58}mR`?~`3=a(>+!!m5P< z&iuYD+Gu*l6S|&CYO~DAh9-LX(STMzkJIS$=2JlH+~;!b^LF#ucvDFt_-63KJ0|8~ zc&khF`b|5qv!XYPPl?b_GG{HbV9h|q?#W`rDAEk#`qlOKt$_;74JqLD2DS82wqOn@ z&7&Fi!eOjO4q{!Bb77u`EKiD)s5A_hNT*Mr)+DP|#VLX<;=jglaMT(mbx;5>X-3yL zapEsAjjlV)yEur%JyAxdV3($u0UU!K&U?p0hULf*J({ZudqA?ENUjVETzaqo6(7^o z?Cg^9wwJqTLN*mNLixw4dXtifE$nk!ag^Q9Hm@R;6N+``vdbx+|wBAIt%iVo^-2cOPn0{ z`GWKnSBlZt1MCfRcKugfD2PVsg6rv10s%1vLSDN4ZRUz-R4vdvQ7>%|@J|i?PjUV4 zUf3-wN{!@eL!A;Fh#bs}1#}dIBnKp^YCC)pFg`ee^8zrOKU%D)DGPg0^r zjdgCMoyN0}ZsIsMtmgTEM$)V1*W5$qzX5wz4%HscLlO%Zxx1L~JW6*k?+^m~`~)#1 z$mG%DR{c)c2&%AK@!REy|A!a>7QJ4BUdenb)NK=oWxVpUY#m zI4SB>lwS$h)mb}I)M}=0heYXeVZ3*sQ74Ouy$g_J{tr}uQ7u(c3zooTgki{&z#aV` zU!YGrf9#M#299V#ZA0uV2U87Jhz?= z_Agcu_286)&DE8|tNbAd z=#sedsM5FBn7oqN%8@`L#0oXmh@A2GyyYQ&FD$8b;&o5h%14WuCCviFy%h0yo^TL@ ze6cT*TptQOSgrXCwt%FvLQNld`PwV^BgARUgNx#V?Hism4ZmP03d7hWF~M>PvFshf z%T~e@gHcyMtl-~LZF2bo*yus8#e&ycY%;guzA?~VHIqIn$M`9D{nvS|aG1GA9NZx5$rw2enmXDp@lfj_7svc79;m#yti5J%%itCF@#O4v_zAD z^-FI6q(}-cxt!>JuVb?~4)<8XnqYG=#|`Lrb^=8p6Cee-lu)_3RS8lFQt4whQt1^H zrhoGBvZ~@?ajW@R3e<)-9nxbI0|h)5b!rY{dkhX^_-hj@#r-`wxtL#u!sZ@&8Q;^p}a{HoLlUheMt4%ZM)kl+=Oz!a zn``y3n{(_|5qQ~#T&|g6!uzHUX>v0;n3bGW){s&LQ+4#I1Pk z4Rsubd7^k8P6M0@e71$SL7|!gm-SthH4YiUFyNpD%>n27)F;3Uh@r}w{OARvi^yh+A6n9iD=3t1JS9m7Fr@}46U8{YtZ?6mMe47N>qXb#52xre)xORsPJnuVjkpj*7Pt(I zqI70ILr7rvFfZ#aL@9g3yrH^bWTo{{EOSR;7A;=FgG8-ogM6j%3ydG%} zViTh6S`I*AOTIGUvoPNTj}Elpk`u$DKX+r7G%RzGXIieeh`F=hLWz~to8~-IkwAo< zNEkf-Yq97*aP{9n7VldG69+sOrq%&`Z7`yLUQ&z(n|i5z|IV3s&L+jYoF$hAn}Gg{%2 zC;eBB7|o?=-P4B8G|$t<_VtE~2jJtao#;!=yUQs zSlZ&ZC@bQ4=r}XE=LLyCt7mHv$7LH1C;Pe}N;G0=aPDA#NK8RPBIedyb^`;4_4V5L zrv3Wd!mgA=VHm#c48o>;at$MZnGC716IBnz<$HqCLKU8lg>}o1dhdk~?x+cCu6*kb z%|_QN3n+bj)=Bw?rfI1%mNeD1ky4DX^3ehbUOscwxxF`nc|ka|6(Uy;J5w*#Fj)a>Fg5&9m z!~$KlZtjVw#a(g{-C=CmuBf79sf?|15lL#r4BtP%nqH{}nFfMnVU)Yf5F?s~I~ol( ztG6C@X@|TwZ-<9pv4`3dq!>NpC>85AJ^*+d9-!$IOjM0x2x|Zs)`mE)5$?s~7X41k z9m9puG1%`=zsbj4x})Dwyz?`I)V27RNH1pt?ei}?!2wBF#+|(sN~47)A5f&q&uOb9 z7SdcZ@2;T^IYI(DszOyHO@U`$C6-K-w}HFJ430nFLXQIj!J~h&`gmW@UPWGtpFq6+`C7o~52D1845Nu?^;?r58xF->sU{ z#*dst;mNFTyX55?>SK0R(vw*lk?1nB&vr9;gK`lg59Z^fVR&FjKbS4#eBt0zF4CY* z_R@j#x~g_~i!yD*c9KbUCo%{PPQQY%<;RjGw45RG1clE5$2CuuL%-~mtg%id8boC& z7kgRu^HS`i;!<&}pyjB;+ux9^l-sRz=+S2}OwT38XlACIVVMY7LeQL|lxe#yD%;s0 zMspJd3<=rRBE`mz82;QQhYzzjEFy@cqmC8x`CDu=8<$Pta}ZjsaXr+9dXMJ?ddlDWHx#I;5S#}C^?arFIZn)$nw01^im z4vc!++znToXocpff~a&paiYX7RmraXtl!-J$8fBEzGIwp>M<#9fS0Bcf4CV5D`4Ps zL9?yeDES-S20|YjEKw>ja-;RTywy8LqV&o%V@GcXyE&9}r7wB}?jbQxN#Zy^xVf2o zL(c3uKq_hL90*g_*b;tT;<%>UaD3i!+R%JcK7Yj162)xX2Uk5J8FrJa_pD3;tHc0{ zf-GKwHK;{mZq!KKtE22MpND#BjMz5FkB=O)ms8S5!a`J}^~=NyCtfQ=QJhQ=P5xQMc3?RVM^hYpJEF1$z{t5Dyb_SU@cH|-x#T@_O=y5KWSr5{ zDG(dMgvZp}K4WoROE{ZZt>y-=Q@x^yzho%pL1gwjAKZ@a+7qvc5l2^#H@gaF#o)0e zs6~&&@&5297Og4!cJ4uvnQeS+syA!Xh-Dl~EHLi% z;LNM!vf>Tq9_9}Zmhv7lUfK8!FRTLrl~|2nR4W;@qjzF3Yfy0LidTW^5w3sr2P7fE z{?i>d6P?vGvYlgC=1qh?HRfds9d8)0b3hjBc1}Ly!7y+ogdh)#VDDF;rVC}6Qz9>c z7`db|eh`@*75hDY7>W}QYW}Hv&20|{&$9YH=_bL{i?t|2W4%;a9#Z$W(R&v7`QLzl zjWGSIQhkjtNhAJu4QtNKX0ieM*D@=FC9X~j#J6vp2*8~r`2Xq4`Fr>Y0GaCg|1fC$ zILWo+=|S>y)agRkD0^`$pTqJ{Q3Am+X38p;VdK6EHsdc2B=`WWQa1ZGM66nD`QjBNeG_gpvMgM{(Ux(8!3~ zBfKhqPP`V{@9(EZh%r_I+_r`fD_hRI6!nA{!&5MKr5!1bE2|maT%rk1-1Hvw(-YAP zuKd+kADl;a>WN57!eHDlDJgZtu;dvxR#;lF;m}Z>(J`i^J7-iyCh|LCEgz6sb}KiX z>pf-ZpVFFeJ7yaUS1>j=55PDMKGNq@%kDBxb657TJHGL51%Y@2$lB^ur`sE3Gu3ol z)|9uED=w9*&l^Su9N^k-a!gDRlxrCc;%MO3*(IDVoym$=*B__sV^eUdSa9T(^DE`> z$}u5qcN7sDEn2`$>;_pkhXxoWosO0S+(7iRZU{ph$FJ9uxM>PA+W<%4E=__5hkobL zboRW*W0R5GS6yKPe3_5}?FWeiEI7~7y>u+xl3Sh970bu)((n}EGz->6&8C-eyRnyRu6 z3mYFCEyCR~#aS;3ybc`a#YpA~3QT6m)w@8c=n9Zjl7JKfa-J@kY0nOj#`HABHukY; zZ#c}j2E{$}Id$D=;+C#BvKX{|nygtBfEhBFjS0GX{d>DJofW4(#|3iAuu=pRuxi*5((k$b_x=<%@bj-RG-o3s|r z$>U=avo{0=wA_sh7BnYXU>(|?ec@_8?afl^5XW4A(lL)RKo&XFvPF*Y1A#18;ToT- z#3_B9^M?EIIi=k+<#5_5!m|RUen?&VnL|Grgya^@+CbUFM%Z2kcdD~%xke+wQ?kjl zvS!t2)5d+%T4A!$is)XnAJT%x)(py?jp==xZf;rHVzzQFm87I3e+#T@uYZ48rFoZ^F185aZgA0Fw(`^*5 z-_%y&jNkM1&smLO{c_teb1CSz2+&5-sL0{~it|_^#D1Bg%9T!IX zE)m@D!-jbg0_T#?3~28Lj9A-dQc z5{FgLHNJr=-=zk4uz&W---Zjy+(H_Z8EJKk!4wBZtgPFrrH<9?i9&NB58?5%&T)|J z)ii6rM=EkJASe_A4FA6#WxQBV_kltM-t^9tM<-Reg|IPH%F)avAseEsrd=dS+ z$Y6m4_6Y>>H`%}EvHt_`|Gz4bqek5fiTDro;ZWiaPZY|xZ}DtEC3PGupkV?p65!v& zhe~ZYU$w>5&#qB&57yT2(BRO*k)f!RBrr%`<>Vw%mQu_}%%MG|N0d0&M?1j*rVI&ga#coilcoX*HmSCT&IAcja-1xy6XYWW!!acA^^`Iww!{IQ zE5kx>+PTmP41eo?!<@D*A~9h9+2yo1;n)4M`|Y!3E5wP2(zoQcnmpl>-U!O&%Cs(G z6ajT>ta~$n`V*Gr6_ON8hW)E`Vq+_2()11- zEirejWGO;j*V5n8;XpM&(`*(;9RNsE3dIzs{IMF)MF-RkX~BmTUbZu)j zROtwe`*HnVSj;5ZcnX7bbZdy!c%0fLl7-^6uwpFR{JoOcMxVR{Mh(C{-q+)GV$vdi z?9DmIs3MDWY!0D6Q4Ys-hyuR0D7|+l=`tP&G&*>m!Q_6-SLZi?ita zF$3=cmBY2x9|&`;?ZCn)D$l(nl>Q&4t|>UPVBH228xz~MlZkEH=ETMy+qP}nwrx$! ziT?P`Irrh-s;|0sckjpUy}MQ~sA#!M7?g>q3_?-q&Eu_=OH5Lhq!L|S#Z~dH6CM8B zQBddoV(G{cc1Yr+Mv;F~jkY2_gy3!rYT<`d;t9^pt|!tlHcG2w$Vh&eWaA_cSib#; z0_3wHSur!y=lcQ~l1U@nrcVAoiyAM})w(o|?@X9W#|wM8iCMabyaM{)hZ2IRjkHtM zIJy6dwHnHkr%0AJac0UslImX87TIuNPq1Pdz!(2*@P1P`JskFYzf>qNDSFrEM?$yP zg;72}elVwSK2_5;Zm6+3& zXVX`uU#eF&J>DnIznVC$nVz=J7!+Du#hOwTQyGzG^CXQao<G(-@raIIi0S9i&iWvV9JS_W4CBGuV_iX>{;oM=KvvyTT&#=qr+U0Ek_2 zF->wVpe@)y-`Lay)=@^wQ19=Kdg+Rqh_NF=M>bTE*=QgEojk;^=1L+j@GwYE1Jo0kZT^X4l=4#R}VPTRJ^_xVCP95Ib zr6$6NRa5j8O0ZQiUc0o2RrsG$0Q%~i@Pz}u_~plrR8vh+)H8`{y(^S{yv0wp7T&je0eGQ4&p5Xy7 zZ|+gr;|;B}lkA!7DW?Oxg5T`-Cq%a6po}(nW@&#mxOwRv7;u@kE@tp<0YTLMp78Tu zuYB>JgYo(~+$1dG$@RHulq0U#mo0A(=YRB%arPguyEm&(oI3AIJup{Sv;gsTVihBK zLn0@P4Q%YO!j0Ql2}Znt9ca!8fCD(-*AFPu^$`R2K|PkM5O1@2BI=>fJAZhjzh@>6iSi|Q z$;A~wCiF%x3A?LfF*u9X9<98(8g4p~yncB;mdC>;Ll;l3rqh$wX03PPWqM3#AMa*co zyQR246BePw-GP(uk&jPp!YZ_Z4X}=`wc(4i7E9y#z7B+2f8oA(vPs zr5)(pb;4V?{I$dG1>Z|KK%lT^;f9m&oP6<6jc@aq+~WIVmn(dXpZgaaNSyRy>XFyC zds;_05N8Gi@go^h0AOE;05gIOQV;6()2^NvrGU}!eZtB2TF+}k-H&noMJ^(a+(jF` zr)QF#9H{!bN^L5mb^8x2en0(0ep0<;7cAkov3J0~+x2@>w9 zdPP_AQDqb*+YqQ4CNncLGn&~)o+zF~RGUWQ1aG)Q8K{QlU`fv|*|lSGi%aJ}@r@S7 zG1_~h7^ZU8UNe$wOFBeM`3v>AgnMMllzz7$?f`cbj5~vQ(=1g(H$;5a`CBY`*kvZV za^Y&SBQsl80pzaoW*?HO!*OB_!5GmYz!it%HRAhWHtm)UvUrhm^IqtmiV2&z=Tm;Z z_UO()1d4Zf8uRz@sR{{P9X-o!uyx##XX^zkV*1>BG1Q6|QF}91QuH5CmRk$+bd)Gs zwb)_LYdk4AnK4@h1g*{r6}HnHe7D_^-9OPu0oFd4=c4*Dj6OkXl=pdX3d;f^ec4BsSXuH+BE2*GG>966@Do6YTSh-%AuQ(`~b18JrC9~@9kR4&jWurebTccR-L(bzwbKB;a>cN>BfvC46R z69o;SM-c`+e#<}^fwiGQ4Y=$26O!>D_=wg~fNJz>wxcrmyIoxRgSZUhodu56WAsY0 zQk;<-NO`~t`ZSzVp?hyCUJ)zP#chUgQ>o_!0n~Bnwe1RfXPCeP72$r>g9r;(!RqB! ztO$$HejSqEBXg>uh5EHD|JC%{$7T9Tr>+)sd4>_x$}VRcSw}Cgml}a}~Rip3L zPfDv43vy?um@o+`D~R4<+CZ$|0aLUDAxevc6b#pX25?*PAD}0bHml}@EU1Yv{UMUv z1q|)a@JbJPf-N$Z?n++ErIyfg8h#!po*q>F@Q}=Vr%sk~@W9rQFsmSTK)YvJ8#V&_ zB)bDcRR<=@t^c;Zt%uICgvB+(RG%E|UVbQ~rn7{mokHe#VnUA&gY9bXS>Clk(J+t1 zJtC_O7}25e*yI-B~z=yUuNb4~lf0_*-chp# zITdM;6zuHs@y{FNicl5q#lr$4q!isX`AOyg370+G)$$izabtal1)9||ME$6N25>?X zEQM|kDiT&fgyI`e+$~8eVReNS)B8hZJiX#BvI9^SFJISRCENt7(&Ust<2o}MaEj;Z zK?V%{5#)~w?>XL&=8(AV+UFHK1`8&MQ?e9JKwpQlZ53qz2TxokI_WWU_i1%-{uvvp zp@mSlicp6$aJMFPfsONu7?V@70Ju#ouwPt%kzn)rmJ9_fZQRjx9RP$JZvR*U^D;U{ zaSr|}>G=5VV;#82v86L|wmK2pBW?CKU4ec%%iv*TdUnlg+8s}o!l}mKU!*Z8zRQ){ z<0{g|v^>IG8|&(IFvi%i4|b^5aGPdA=`(<5!~ zvU`4nU%J5OXxnQ0buK;)T2khWzQ4L0b7oJP5rTX_VjV9yZ-YZR+7WSKt_qeFBC zVLy@{y0)=~WoDG{t+Kob=u@svK3$8+i~|fh{uBSfU&d1-#ekX_f07zcJq^KZ(JQPLL zu1%%!uR7RkrYgL-s+~JHs1q2slU7;ty0pBY^A}CON*9MwAhBgLerEf?0Jh0N-GM#) zms%)mB$!$lOQk2ia6|i0W-31hvy&v%8MUssySZOFn*(RM7~jFkDSpYDndb_%rEn#P zx(5-x3ELRK?*N}|04(Y#9<1lro2G6JA`AnKSbUzGs}vC1_0LlO7};TE$#H1PpSgZ4 z>#RB0IySpxHf8jFrt ze`R<&x}u=%a-R<`Lk+Ku?enL3e_TQqyfQt`@W4g=*l(=%2V`V7{m^JYOY?Qg_d!;N z6_G8wE}(~NkIHrfj~>GQK)Lm=x|2T}>@xhdi-3Di z0DYi;Hs*ccwGG94&)<&!GSuTCz!!+rj|L>-i-Fqb_JNMwAG%}V2P?Z%;tGlz&;fFF zM=A7&0m*!V0SXZz+ERba1+Jq{4{pzvy=FhW);Y^Xe1@ zb1z-8HBG~?&RLz9(z)LSP;zQ0IHCq&8;c?U7SlZyRZBs(q&aef<>~^z*=%q4K?f3Z z6X&I%^8#k-g!MZWq@IjA>o`{^OWxm@&^uMe2fBmjx%SZ-G7qbwfr~ZX| zp-)9m7dEwzqX|NgJ!yUvQjY*UruMI?M=2aeD^f3U^WVIzpfBY>Ov(@2{82LZOBH`d zUlI6Ude{HKIsYH?`47C0Nc%Z}hiJ|baERN=b#Ji^#H2RmIoq-msD|+0=sJD6p zPnz~?YHKS`+Io6BB1)pKyO*`+kJFFA8RdH~ucRwGD{6JS?|nu3YHPs97bJc#ke{f| zf*9T;5K((eJA{C8kHkn3RlL$eYML*}ktJ0=k&z}uRi{Co~uK3BQFxv zjRSOH-xI(x=icx1`6TGY8V*Z|z^i!zfv8#0#1&i_rpz3@i;(sT4b^;h3tQ0{4vYD7 zL^WAaNqR(*X0n_r?Ujl*>)aotF8_2vH9Fx{6Uuh~=**~lnzDG1fm}=hXk_ZD?ZnoSLQ1vEST}W@Vb9 zvqg28+v8Ur=|jgOQ(WZhvLUFeRs5?kl1s&d0LsQvvP#D8>8 zxf5Ev50@|2O1G_7X{h9T3SN6Rhm<-eK|}*Nc6yc>TJKo;V(Z7@3E1b-Ko_!GSky+lm$u*^h+(-n(^*P#lZ11&M&dCp19Lpr| z+DFrq+q9Uf?$v(Uoy_^oDNElp=v;$_5icWLGqG;N>!DhGh~Vtn=vv^LXuptG0AFB!nsoD!_CvQha| z(|0GzCppr=Dz>aStC?;yE}KjV$lu#RXRE{}^Y*RMsQTxg+(KZWm4yTUo9+_YtJ^HM zHUGli*?X!ypv=xyE2g}Bg-6!h&_;#x2X?Z%tF5&oDco-3JeS(r1NBX-Iiu5w2rK|M zaJgxxuGS>pSK91zdh_D7jM1{(ww&MhU^~MjW6sd$8s~+lZB{&~uBKw>R%0vB>?Xf% zWl2@NLN6TG)Ff5niG$JB-VKU9RT;xWy{#h%-!CPg%tE!4jAoV%kF&+;WP0gTeQmC~ zLLdXQ%4iIHv&-}X z#B-;78$w9n0&8!AAJBB65z*c$WOo7mHqQY|YCH79;)#c@N=)V|kKoMGS{bln-r}hq zRtsgpW)Vo<1K~FL(p>XUqbBy`v$CW6$HwlZl9+Uc#gV74sAi^TQD$fH@%OU6Jtti> zq84|HF0yEk@{YAyv&`C7hHsgeM&)X%P+q~Y=vO{e3fB@E5ly({Uvz~mO0F0o8KZ5> zu_)bUhExW6?cmPJzwD$U=Pdxq>fR#Pk_&sH-;&*^K9$=O}RN_mU zJnmiUF-y(U_NlM*TmnUb5JT{PVd*}ZrUngVLs#F0R(J?}< z;$xN4)(5NCLuS9-NK7UNM=>}udnS)=SyESW1r^&gH8}AkM)LhoIMx7S=FK5)T}YtC zR2A{kp`grL!_Fd6du{rx8q=B%ifE^BTOXxD&O_@kfVHjVsz&NFEeQ;q)!lOTmTD{K zsLiYTR=p?f%L@cKYhG-xlr9rjxJH0-0r@wKVAk^S4YW%PF8d%g47lwK%3%&#kd{wM z@!KbQ_gi8z+(H)nr8VHr+wzS>vn>66R;xqY4wizfMEY@Xs60Q==e$q{LFZGX80lY* ziJ4AA+|#+y?!hQZ$!foKb747A)X8RKhD^IwW|4idR%l-nM9c7@23DWk=TrIRy;mTv z*t8kbI0-{oTYb&rKgV3D@!+#$`aAX-wV;%l89#R9GjZlox zfSc?!Eal&8TP#>U!cODqwF}sptEC?xqw|K~FeYCn<6AQ8rjm>Z0-%^wx+x)Tl(V*> zvsbc7_y{pUOtFAGn&IVZYSA%MHI66r- z=;&jj%CKQ(@kEL+e3Dv1_WL;u=9{dK@EYQgbKvo?Wt#!H>PLpkCvB5HxU-#a$P2}e zIFZuO#xsZQn-fSfAk0;Ef}YFPb(@h*G-A?;NKbqYT5Skst@0;Z3)5T6PtGC%za`zV zYf&y5pBSnpPMPgkyyC8U-9^30k|TmKg6m>U{cz>iH3g=w@7?G#MkzA`u^AS((8^yH zGMAQYnRozK!wJUJLODViA8%s(Dr?hfSoCEGHP*Avi6u%Y^J>8=~d( zEC`lR6oVcd)%7Jjg|r2u+VrHtc24auqMG&Jx;m_NSZm=NATQ_D?`FhpF=Z_~xF^#N zVX$26ZVcz^K=FIADu%7+CS&t?bauG7ol1;rq}?cCucjGoyu#TE+L8q7pgUxL##l+Yu) z4mbeEs(Ll}zoWSV`se>TW##r_#%+c1cs)DRpAQ7vTTyulZuwzyTHDraS)h8X zk}wEqyllqPxPi5bmtF~XmK{TN506J(f{lFu(7X}Z2 zk@6FW7(4!99%9yDo3U4N*X3C@4XC;E){Gql4Be`&i(MHf8WSOoIGOoH4c?k)FjFZo z+2M;(!K(u8UvJS{GxSRC-d_l4d4l*Y@=eUDD9>ElxL59cdg{k3?0jQXf&9IMD#4)`?CQ7{^&|K$weC>aRU|Mb zf5Q4FKu{v+u%*7bmqwJY7d zK$CNX1zReE=a+3MVeJJT{u zvA-nbea>oOYnSlzG$J~F*(V`p{0!-vG%%SLc#cJB(JiVs z&z|0J`mzWie+dVDtj$aMQ&ak?Lk_8GToO{Fm=0z2u=Z+z{sjq08i=RbY~DwllQ;cJ z9sHi$Q%}!>z?z1Ok^$_9WsW1u!r=G?X_h( zlP+@Qm@W20sgH}b#QrnP#9}i4LpdrhwT=N5RG>uvL`_?b-SAK_ZLbEDNdEN^rb^8kfm#%F7T_gZ6OzF9XeDz#etOb;0kF^o8 zza7UZH@m9WjLpZpb_$H*v3tZs_Fg*;+{Ea)N?+ps2Np~Q1Hk{;~if-(5# z!Cq8E)NIvK0G*gI$EYM|INLzr&=erp$(Ly?i>3g4;ozV&9l-5B`00VxSRmjl4d#FT zs3hIxSFHEG)QZ9c4EDPN`yM`3IAHbpN_+m`<_8#geR1-Ck3D`E_3P7 zmD9m_%MO>6;)$^9QfhtY$H=b(Y&ERR_de%una4N4J8BPq_3M#)=l9WWwL(F*Hzqv` z=2jgeb4e3~eb5ta@eG9SK;A6-?H3tx<0;qc!gE;mcahIy;uwL@K~T~Sxla}RP$vp6 zYtmo0&gAEkSt1(Ym9s3}NdTnlc9DXRUB?nNTDkvFMwJEgNdhWg3owL)heBS zBVvKitGbJ;kMtS4&)A#e}?`M<#PjY|u(2C^p{}VBL zBzB74+*I&xlwK9(FS7L4bA0xdf`mFSJcqvn+>JoPV zSlXZl;Z0pLx_;6Gp!IfuI_-;(k9kwth8Q_S4QL}8pPL4;3j#v3Tuh*>8m3{dIF79( z>7r>1nRa7?cFiO1=ed|}YE1YtnFvfG?)5ZxW3b#wMqh|Vc~OTvEb*}7Vg@~QWux6q zt_iBgEFry7rhLQi{f2=;!@z=ICISq&qkAPRfzTlS-x2qBTuig`%(;SC3eCenDLN$L z(J+)F0g_QhGC=5mCUen4?vK;4`l4Ivj6W}dC+x$2=fdwV!|!p!?{#2|_((8DRSKE> zprCx$P>%+=nB>=(bEUBqnudYkbONH$0>aS(;?aGQ@c)_Pr-nT`XJ-ClT=k21GC0br zL!KfHzh4W#R~iOt3S(h1r|sK@?oS5{bT@hkOasDyWT0eEleG_}7vooJ;*QBvy8P(r zBOzjAo@1P?76$0ZmanBiFRMsRH_KxgI3b&JEQ+**dQ-wz?Ljml_D?L+VA``PQ2zp9 zN1eZ}7}=yRNQ{#b$q1N~>_^c;9jC)S3)kc@=Xf0#gP9#Q!(YFc+iZc z2m-)$(2ixKAQx`f_bQU^!dRiFmz3e*Soa9>K)#mGdxq~AR?``ghLqouP?Id=)3$^r zUkK}4$WSSo`QHbzd~m8O z>IW2!f*6`9tnDSW!-cI6G4Miq`Ctr^o0ODN19!ty8;K^B#f!+JBaEnj0#mJS$6&p( z7L=iT7Z0Z)ZDh_@U@|PJ;n-!g&<_eesZC7LkmDlCKcvFPOjW;BRlj6K%*lS2w*ge^ z^ZN~xyx0NcPo+j9SRC?qZfI_$HzA$Y4IkXaprCNNFH;7!6g2l2NI#Czm1(Qy?Cqf{|$M2B!D zmZpL8{YfDIS4dlSZp!)6v0{Nz8S<=!{8R95< zks7)%BU^~gJnMEML}N?yYD04Tp%t?3BdkLge5uSk*t=W9^l1t>cDa93ll@nnkd5*&Da`mc9$(l>+bA`>z0Yi07*`uX6OMrDflJ)3_ z%u!`tBPN<@Jq;%;uP($v>CCxIoXcnbu68nE_Re;TLZU39n4A!jJn+wr4yqf^Gp;ma zkdA5OHUThZ1dUj-8!VGXT2?u?9%_z7Q+N2c$*ba`a(DSte+8~{b_qEz>uRYuy?}jN zao$eIw}vW30ga+xdf@VI_<+sjCm1=eB@-)CwPbOJUsJ8x*2OG{FUB?&Fe?2X_98zDFwK{y*R zIG!NvjZo~3NbF8T#4*!`YJp8wL=lSW(I(z6!ZyQ$iNEkjqi0N4iUD#Qv>Y|9lG+Nx z`L2yZm*_%O;#AnDMW{LHjEU<;OD;*)$TB~N&tf%?CF>ziL2~FIxOYHY3$GJI^IDkr zN6p8`9K^{2FzI2IN6Gf%$nG-91F);*kv)Fe1vwpWjD!`e2|=@&L%s-&71g3z5ms-3 zNs;r;kFTz!6*|i{{sC+TdsCH+U?r$H7^U0rLgV7-Z9a^mwi3?Loo{A16cB=WquE~3 zKSs7K%W!pwC44GWd$`NqF49_N4!JN znAm1=mj=+!+~b#kV4rdHpagc<<|5zAtMbdKG3@9;UfeWJhymn%Fvx9uFJzdjc)OBh zYkt_Nx62x2(+uiCvbsN$m)aNkPUwRA)281z!( zaoH1b$!$(>lqczoYTY!LX(L+IR6WybHe*n(LTff5v_+kJ^It(@#j>0q;KlNas5@1u zI#pHJ&eK(Mk^xy{syV4Pv<%ccP|xybX<07Fvvw{KnGe!ZJ0&O=CoJpj_>1k}Z;lX7 zo;>h}*i6|bBy-FOr&>&5NKDrlv#dYTEufhcxNvt8s8DNXY1l@#47t#4A{AI@{pxFh z{Lo8avS}^}?giWlLNUuf1=?lq(;;&5f-`Sr{-#=YmI3Dd)o;)yZ^NwNRrm>iun|Q}w?a`EY&i&A>m*1ewacnmUjoOo-zo#;}p$R>)wm1eI7qB5J`g!a;n` zk@AaL#8zoiRcNXx>qT%U(ayg>FdEilGHk9AVh9eB$*XAkWSJnoz((@l8Rp^4qKJMXaQVgLA!h0< zQ|VVl?utF3gj!r~{FJ6}`Wue(Em5gGa<4@Vj8`yFt03a2pM@kvu@tpwi^?L9*KQcN)>E49eva^5xI* zr}Odaxp{8&L9X7>;QJpzV=$1TA&{dYvRFrx7|b_yCbGzk5m1Lyg~a@%6vKAVj=U_m zg5h(5;h*C!AjCI<;cG9*hnu;A*~KD09fWn+xp?i2GPg{ zuSWpEZ7if=4N;M+IGB5O=zG`kD+K6!MwG+LgmYZv!z|>(nqR>l6$w-Vbn3xLFl(E} z53oA{f}1=A`9RS2#@%F(x94&))y(=K{?R5p7-T&0~pxLiQDvmyJT za1SW)S6(CNJ~$enFH^#vJ<%^{N@;0#m{Pz*;5ivq8SxAgNDNEOQIjBxKR@JW^tSs3 zjMJW6`nx`U%DX-q?aQAol$&4iBaT+nFC@qBM1;3FA41Wm5q9K^7G)!o8g((*t%R3u zFbd7;(ArRI%gZr(* z4l?d~RXOqgh$>h7plTEdH+bw>RWR9-}iuCsRTfHH& zy>^q1PSFR>|4Mdf%ikhCgAJF|06+pV%}(jfY0c~G)!3%;eH99xmhlq#x<{M?0Ll3H z)qMG~z{1aVcV;4e>4XsapOS+ohJRQP9*0vKg3CkK*EG8hQKoF4hqH(EJiwTNYfHr3 zRaLH3_{&+8VK^Q2cT`~>_d8-2NbU5A?R9i|7Gsdl`GwnaZ&QA}m)Q+y=)W7K8Nw~Y z-sQUgu`9{TSp{!zi%fe*r#H}S-7EIeV<1NT!ZprinK@h#XUbY5-6)$I5-B?*@D`DX zRiFp-?qiVXOTqgk=I){w)5GKvR-x1=!bts-r5OE;o;$y;7^`+|>v;K7R-|b11U97w z%g>`ia#+IUHmue-%b^z#KMtNiAc_6&2v9VR8X_rAbgulm&x`vc;PVBRhYPU zVxHh-$S#S=`+}U{cOKa(#pz^8hm_@oUZ84zkH6}Vi9|S-665g{X~S?;SkmGIMdFkp zRV#Q^X_Eqd93+|Ih7%QK%BhCkE4OFJf|7R>B6k#}gD5kHnZuK$ z*z#z!ak9EGfcaT-RQcgDYrn*5(BP%NFNy`PFm}IrSYm}Dr@JK_4OvRl(S}>YVT5@q zwqOQ@1iI0iXEB-^BlSk&hVYvgR$wN&3|IZyo8VaNFl8&n!?Mf@vrByGK(1?PovA30 zuEiv%VNFkps*vo@$HmK~7pqJ|`K9_4F{`W5s^?dq0D3ypd!mbHS>rp0O>xgNRL1UF zl|oH?KU=tQ|B}mYm%NrQK%CS$Lqdzl8F0-LJ-}cHz(Et}53V++R*M_*+C(7@q#t%B zN|Z_+*>1FcY#Dh-wblf#Hiub>B`3XJZWs4B%<2ClDQ304Gc-2L9@!htd0TWL5&~p9 z=M0sz06Zll@nlB34D2LHG6jO#p#(lSncJ{fL9R)~6kJl)p)^)02P&^vTG@2v^a7L( zh^_LhHP-mxE%UJyUamcP)mO+Ht1LJswDRxk@%M0AjY0jV5B zvsMR^AFh?4jhrL<(E9bp@N8u1120<6JL!2dKxdJAYu(LS)a+Zy=|8jt0tq9dxQ7vcY;}x5EA?PM^xgyI{?0{fKPSD(GOk;_--GF4U*7||>^|&eko0|D$@&Wm{P2tcOZu9HP{9i;*E0DL6As7j@%Mlq zuxHha@q_rGw+kiy{cr0bawaBb`ovxXVeT*r3z88E&Y>e0-oiz_<&&xgZ4$(q$tKt$ z1Q(KC_`^Io7t$wK_To>=kt}9Ieeg1Az}lekIgjP&rS>@_v^3`+j}>h;;|_dQiH>1c zGXhV^^_{;9E?(jd(+rLRb~rzIs)CCZFzLTN+PfsL{wjfWDyT=Z{}nDODIr zInp$cw%mSXlgc~+GB1WMK!ir$I;jL=o8$OVhJx&pDdn?&-h!@H(G6dcV!11tfR`gp zUdL)ZuGO0#qsEWk*37F|2{7EzqV$V+FIhe`=-aI8X5yu+;?e&X9`H^j`>9t*K&#%X5tFs z>6vjJc4e)nVh#N>#7sO_@M4=*XFJT=pa;uU82O96|61vk(z>qDo?J>R z{UfV;V+n&)>H^{N*1V6!RsSKrrO~&-CC)8_C7ugCj!pVJDC1HLNDTZt<)s)453!dq zO*WUV--gYW2Alb(fap7t;FuLwuGwHDFgX%UQ553 zYt}tt)?LT#2H)URergu`-u0`+?DP@HvuBdsleKJ?XFP$vqZ79};bnh)!J&`uTK_HNRJ(52-1S&jw959xh~O4fx|44uk@@B>$m{L@j2-_*(7Fce z#O*hJY8ijqRGO4cmGimkl89R3+^-zf3Ty`En^{oz&O&X7DCN-m%^U5!8a;=XeAWZ6 zEt)s{6qn)vINT2;2=xBM8Cu>4Ej1*0+X!>!=GS2Yx1m}YS{Ze>VSQJjWnH05ajZyT z@Kk|#R3Us=Fc6B?^32@cvNm5_-7#xh#BjTVeSO9ceFsnY-Mvw3eOp)U*(={$bhT+w zmnDZ@3GKPGwYbAAVr{0H)U{J1t_AJ0=xY^+JVLs35>Yk$-FdmF&-|rD5g0)jo%dN3ecPfF{x4EU?soU(#xu|RU;Fm3)^-cRao-LN-lpa} zHjG|L@w?D`zczx;7W@T%oAb`!=q(-UnF#jX?~0kOpZiAum<{(w{e=Brmv5D zew{y@_(r)i$li!yH7fXg+z@Q0H-^J;SFiZ-UrVw7E|Y(T zbc_GY!~QS1?LQ01j41vp_a_L54qKX_8zW{K@&FMTKv!)=;B z3Ii3jd>&zuj>9gsRZ3fXfGNY%DpcP5kj@Sq42&d__NFG1Rvn^*PEMz?Ise~4S0GVj z*C9is-^!c7OOGe3l=-5nfZt=!`^~oV?04?xF5mqv6d_5W`B&NOgR>Q8+PFyoAI4gGJDF|lUZXNDTUjmD{St>*@a)Dx9YfISmafgzUaZUd3k86UHtS9&ze_x z@MW+<4%M?Lo}RI?M?ZF@nB8#?%FJp1McHpLK~DD}x>}vUMx3 z%CGIyB2(>Z=Bwx1sWPtH3{DD`OJHD?1!eZ;4A80(p({)G_=xcG(^}X@24g2ND zA-Ud%f3&j3NpKStRXuoMLYc8q#KrRs2HNV1*WEh=7yS*N(A7P0o3e}xd-)seqfZ(K zura%7X}sZCLnxEaS$y>patV^{lRw4%#Rfl+-Z3%6`EY6WhOsb(YkH%evv*bi5_k4d ziq|_cWPj3j_Mwgx0<`1l(>rx%%30yTixKPY)g1HS!oq_W>mj8`TKEZ@kXmhp2IlH(ttGw>r?AqGGf(^#?Ez*yNDvGfi za9Dn9)5^p5_C4{SLq@DIo;*Ltvb3|uw=}D~fD{{#9-%;WiQy26!+v<{VP;&lkQnOK zGmozDIC{s#yu5 zBm_H`H#PY%pgD{~7zrZM3p7po?3W+Mf=c(aq29K?`e)Ushf_VMj`0yQv!>$(kT+p+??U^3-%ab|Lqizb zWTdW4i(OB(v2<(=`YB=` zh&Bn;Q@4!72k-XT8V?KlcSnPtjWj~#W#~#Zw^q_{oKl*SNCB+pO026e0)b}Moypu1 z%V}dqKWzTI1(tFe(@!CYn$0s7P?Pm5sxDwohfJw$Pr2G1pyL#Uf7w}K<{M0x+Zym&#I|M0j52?fhOuel(IOxJkOEuy*Vg`z{ zjMqOwfO?aFaye*>+ux`u^~m+wd0uA}RWTCahQjyFIH~K7z-m}95s2+2_@^Q$!UkXn zTFz+2&%O&r*`h%cFC?!`t9Lsk=%HoQk{?Y*QY|hd&^_wPTAMZ^+X0jduw4a?p^RIh| zA_}z>;d6*1Wq-$SOv5yUKKU1{=B#I|N5};9A}${3Af&sp zd3Br*tFPz*_XR!3y_r0*G^@MQJ0YZnUuUIr#o(QS!Mi4RLcMGI@1kd?!DF6HEl2Vu z*7KM@20sl2DS^mKCEq3wW_M(6Q(+@aga!YOu|A?4-_&o-0}`)7L2(OuvaYPbnr+~% zeR2~C#%@F(AJ~^AT91Drf(ZX<&y@u6(-x8*5nKczeY0+tUxNpb9!Plr3;T=xt>EfO%WpQHFtVTU{sHnk9S}HacL&GvpyK%v z4PcN3Il%nrp6z1;YJAuP0Z98kQw`XF?wbBjeG>Om+a%bJbiXX_2%)Dg%x?3XsOC+pj&B&WB*{$-EQ9Tb%jI!wkzfV$~jj6R^w)h6HF z_Tuv921}AXd$C4Ep_(*08I3!7lDLQy?=lAJ-qW$ONZzSb00So2jpAZ|*;&)fnleq%pvyeW2RFd- z(8I29WsW_GK{cb6R<4J50uB(Y%vXC3?+xX@I@43i$e@u4r-DNHn|dXaF3;7q=(oU% zoY^3A>hvPJ25*mR?lHw_?~+q!tv>KN-Q9To3PxhuP4ruEy;Fs{4W(okK(;@9PVgff z4t#;3Bl(&zX6)nO7j*r{GHz_?cbNVgd>!}lQ<-ZGn8siN7yF4Coh^VFrImuONNzSp zo$3q)tzPVf+^SG8yUfK)9Sc4tY)HtFtfZY2AL1B%z2SW$(0PzTStlhKpz6iqRM9O8Iij+cwS>+!6wwrQy*xN z!^9toZ%+9aVYtbC`JzducHN@Zt-}>Fr|`~khq%JoJ=ErMkq*y_zlkoGzT%US?dLBj z%!lGyN1CMbne@`~MH$gbT!4c@X}25Wm0+0195+c^l1S8Hmt-)PqLUZ)csi=1f?~mW zOe4**VeJUlE>Udxx7wt^BA=bAZ3mw0#P=|8ILo$P zCyOrG7tCkRwAYTq3x&t6@roZ0oY7d?7pGSbNW&aq4$y*{o7>i|d;n?2TQZySYCeS2 zXO(M`yTxVf69SS0h$Etc!1b^GBJKLP>5A&yq5?5?rS0J`Y4m zADU*ZbAosudUg+w>43S;fSwjmC-mE-w~FAB@KF0z!M-O`2P%eqm5M@PxI-Ukmu~R( z?EwAS%hN!#rv+#`Gpz%?A9~t+GL~0VTmC;cYdSmQRK`7o4V?7rnW=M+H+bp$GOZBU zNTLU0LE))^g3?5)7ua8o`(9BFF5@YlI7TH4VZQ^!*i$1nD*^cJM@daiLRHNlpm)i= ziVrSnvPqnDns`y5ZukdV@y%Sr)>yoFhSEeqt__K2q=v#oT7*$%!Ov9q?Sm~^sWk2T z46-#I3y(PDM)2dckVzM182Jgw`p43YnY8V0k%xR|Ni8M5B9y@)&7`+){;&pocMU1m z4zdXZQ3R8eT{uQMX_;m2GAP7oSfHYs zo?C_L`kq87M8(6n_LWCsa2b`QpDMH!o&?2kzRQq1)qqVYx!=5%>~@QZ zcW*tGSP&52%;+sC1)I7G0TK5t_uEmQqRUVj-rj3an@uTi_m^txL`Lc3NQpkV z2YP0~uwO4=+@FCHXF%z>Ud1_yW(FSUWj@fjx_zFn7FCA1JpvhViQ6QmHC~_Q>T_8+ zsVTspGKfNnKjUhi!3|$S`^lI}#;V=ylwpNF8W^}T^@|zF(c(5GHX2&ipQtPS({#!C za`I9O-SQz8-B>h9UdWRFP!){`l~A|I!{&=O=)&MgIMw>v;YhL26cK2yaz>qX*xNrZ zQVd{g+!la`|21tg@wNXj_6dpH0r$xN$x{VL&SvjdO~x(g&DZ=S;5H0}{S09K8kF|Q zWMiDISr@jxrp!G98WZrd0pCS1yZRSN#d)0u#cI^p2UCTlc0cFeJTL|#Izmr;^I4B> ziJw2}SH-VNh|j`-n|vBt`qGg4;^z8U1(gyscgp+|s*z!tJ^L`WjUiJ>811YyrW*5r zH3l5>%1_a*b%a#U)@nUF>W}kW`ty zxu(vtDUF)B&g1u4Llzl=jlR4_`ine3;S44W>ps-wqu<1ldGvXp9*%?wDKIDZBpek1{y)i9XKt_@LO3GT`YU=6M_unE5A7o?9W(qWHxtakDc%G{L z6?40smqppKZQz9ShY}vCU;}+hw+bH@G?19n>I(Yo+#i zB@H%^7eWsf1YE(-x)R{psfKMMPo*CG=Gowl4~Q0&>;QhKvk;pooXyTL5H~9476V!{ zD(6dPv#rQe1gu!0nOFz)-id&gDgj{)_q#bZnGa1!#vmI?Yhl8mq$YR!VS=&RD7u=A zBKr;6#5G9?uIIu1pr-g%RSL&Ki)}?)FKGK?I;;5FB0BG}dMC0|L*)z|@Li+Q+zJba zvHCZZ5BW3JD395hdc~kecssZ?=N#=e9ITqdE}E2m?6Gsr4ONp1;;;ORvs2w8Xpi!@ z?)wI}3{bDLyv;nZrSnEm7>}iU@B@{$rgfz%02e7fcCOMVqzhESB2UD6KN9mA?U_Km zqC|Fcb~I(+sRcbRf^|?$mvsqqm zwqzmdQSB{cXYR&640gl38SY~MW#0Gea)DP=YYl0?y4gl=f7A5JE}%^75Bu4=WxW4Q z3FNxvnf{e4O+?v5zee$Ng(A8kGvo-7at?iF^MO>oPSJLtiX)(38Z?A-v>P!lar=A! z0%1xWj^ye~Y~k-;jn$VaNm)jAO~{yedo1%8NPRz+IV0oZX{g!{zy7vNP_;y_)H?Vo z*0ldRYQ-X95$)C2$Pz+P4c}nK`5t+ouwCU7rf#i(h`_qmnZ|n3;YyE;@s%D;Imy1A zP#~#aA*)>QXEY!oR8~kBAOg~9CD1X|yN|J}IiVxf4*AL*+wGR-O$ z{Y^VP<%Rmox~;1aUxhKjuy8xIA!9GS$Ulz#3Ocz|Kmic4&UPDQS+4FIJJEb{W_LB( z1aVOOZN>Df{<_P)>b?7->(spX1{wX6%PDnb%=}8@Y=5km z;&oSGxcpP*FYtJm-BEXI@ke?|vB$!A)6lxxKDH$qc5|UeZiWcIOyNSDLFGJ3>wIZZ z+6=Kn20XwSsd0gtGGZ=B@n*dgrIB)qINab;vGJrzhazY-F>82@?N2g67;D0Yf22f> zWJgwQA9a}P0;|hR_FPP40(1xo0olqXjzQ@+oUB|)CY)9I6**A2d-;8Y%ydQlWD)Fz z4<`=;WuzI`j6X}X>$Ap0g&;1+OT(R!K~h$i5Rj(VrFW(n>({*UdwS;+5oFf{6OV+J1?BlC<5M3w;>Pk)Tq75{p4dMEb|Bb%MvF@AyqxcHuh{;`ty z&+YDi-}|A^md&%tf4|P4ZP|ph5%oZG4%8F=tT215rfyVSy9N#PEP&O;dgU4r>myKBu*LJsqi7DsXAN>(c(QuU$hQ7|U>^{l zm9gN}u}FYF0`F+Cg*mQ`cA{v~ea>c~zB3Nfuh~1^n~l${y+7VCg_(Nb_CX{y^p8K_ zm`uCKZCPR~=@o8W6oxxR*i8Ve;TLVFLn509`vP4v^Uz&>W4gKoCGq>_x*^KWbm36H z%F@i$42JayGMw~6`fpCw2GotqwUw1!HM!hRvcQbmd_@BQeHmDA9u#hB(Pqb`!2&f3 zI1eR)#>MQWqK)o9JdED5bp)!3GiIyPXz))A={YLXOPEbe`dOn_QECBVQ3Gyhsm)IP zEv)eVW|*;!l|Oj|ZQ<=RK$5uwf)9q7WsJj~LfM|DZD~duZXW9&d)&%%F@otJ zNi^=bOBrnHqYLrslg&$5TNLz7bqrs(J-AjthDjsD;E!m5Cik`VOPkv1V~hzR#a z(}lKi&7~CtPjUhVcVBJ2cs7+`f*ia3>%J-n`Fm%Eq;ov)QWLkx_9u62s`<=(enAf3 z(Ab4-v{a0RSyfs3G4#?vR@eKTS0-TS_xXuc8fU|kS`oDfXF&3tn3V)DQ0xeq$5zt& z?C7X1(hUYF_7*URW2i{CYB0tXjDV%FiD(Rqw%Zd4P2v~K`Qs%Gs3htYE~TD{LU`~s z9Advbux6q_-KnjhRu9I}Io{XhVV}~yyH4x{Vdiej-iKH81>cC8yX7eKn7bEYkDrk` zT>Vpkv6w%L1z+ILFr9kM%o6i8jL9_YdS=v`tq`PuL>nMzKUH#;;-Ewgok%A3ISv6~ z?o4+vI{I6;1QB8f9eJdxZ1zg$5yV`2`FJb}B-LOh`zs|`VQqtwM!A>-y4_z3^ZZRX zw(NR|zXc_D>&XtMNJ_5_gFfJvBLMazE+_2SZVxL2PpZg`;IQMWR6e}jEZZ6^kXq{;?h+?D^5Q}p#d zbU%PkWJRcHZe3A{N?8I;8U1&)@M}&HEq(!qG~aBH4ZZ}lOnwy>uXCD*)84=Rz+>h+INl_+z&EcP{RK2%K0pO*=me$(V3B&D+4C!vMHYQh& zT-4k{t1BUuXJ5m9GwvL+|=Z8h%Zb5!?YLwd+@pQ z<uF2J+Xg=;3g z+`;Mg&`Hm+T^3GiL0`28#NXtg7fNfKks`*>;_Af@w>SC5e!zceuI;P+5Z#%#2D7#7iJty6Xm=6K?vj-IoNGO2% z#T35i8n~mqVocVP7)A4>yQHZanN4WwU30oJOpN6UYMHZ))wb!O*55Lw7tKULwN@TG zunlgmr!HzdRT7q#C(o}P1>*q8aW{N8iR`YS@@)QIUg=ppHzyxC&s`_Kc-~(YxOTwX zaQ@nhzyNGUEZ*C3MED1VauIi#ro5150IkZC@OFlW$H14fBM`$qfYDKEZ^J8IW^dyw zbS9wTU&fN#-|s0iTUWcLj*aX-3HtaHPObYHyC(adSG#78!8wTGOHKXu`)kdvfDkNtJ@t16K->~-b)o`$1SA}NwCYHO@b2XCrT zmRh5XIinbZeW9f6xIC>+3}7SX@mPj|YR{vU8LLOXn80Rs_N19z266R+VzZ6098Gt= z0>_AzLyQV^qa(1eGu!R| zgB!P?r>sJH_eV6tNQ#qk{0>Ljb{@Luz3J+wZ)LmOsdna5qa08bV9<<8o|L-199&+6 zWi{5av)6M-(mz%aQ}h!)u^8h}mZq)Q1}>{VZ~+ZhkuJSULo*O!NyivK`Xf1UkNJw6 zqa-V$yge&X)Ahh6btNKL%wX>Ol*C30HLhW7k}NRX)PYJ>h< z-y$rByU%V_Inv~P!fpEK+%B39l5j8>WDQ@2~$`s-$?GA>f;!%j}9$EJo7GS>_La9-SjJ{!*d`G z+d8X<5LH*}R=DV)NeX7F-pswkHr(9hMt%5srPdP};~4Er;t1F|j1r(d>3=$)^hHK% zJhe|7rcO1FZcGP!Pa4v}=^f$0Ve=9np}%8Q zeHS%P8nMQK=O!~nbz|L8>wu0vVoP@xTPU9mohee6V?w^uyRKk6zLF1vZ`Dw|&JvsT=nHRxWv<|C#%&~1vb=I(^v7E!)n2}TB zu^CJ(8b7N5fNs+S#V_hr?_G5HnNzST4*0pO{7hRkD`EGB=ANrINa+IAk{5f~Y}H3N zb~-fmP+7S9Q;!})Xe)99hUzzOo@aD7e9Qx0+t;i*T?zjWPb+RsPt=$QObAAdy%5F~ zJ>~YweCULtW3p)D?%2{74xazz!uu2CD6aVy0LI|rO)|h(o)@th3n4w4X z*J~4X2OfDPRG*q4@{K*{vIxy|f=>;f;oE6t&lxyfIMS9-pU=1{y7y7K8DRhP_Q8i}EIo16PYtMPC zh?34(#kS>W+G9`dS4!_AgO~dnzc=cLhWf#AAt^)T_FImuhK73Lobt-7CaH)gUu<2# zNDj@LTV&If+k6h``T}DU;Ww-p!8gt;&K&7ndd_*T$G@UmO|E*UyzpC+ zco4S;YnIkVez8x>CzGO0*kVn3J0bhmts#VGP9~#pdi4RJv)E;oo9#w48|`>-U38>= zZFSkmtAh>*?zOwvDcf2=Z!WCvvtOiuyX}Li%vf&H8{01L1(osbOS$b%t-6&UPj4MG z8O)>_1EGj%+gMc8@`a7?Q(@|fQ9;LTw?vFD;eRGqwzR}V?fo$UrseJ1qotF6)d5uP z<4b(4&wF*PkDu4zxr%f}KH-@O$%F;46_p{pTR`jn#!utEu&3hH*l3CV7eE2CV7^Y|FrZ{q?FhwiS`Cux8*tvt@wk7N`v=^{Wmk^{)=0@~aAQu5TTM zi;c8&?52yjB%+TK`b79A3HT4y|927)-ZH@I^rb`Eg#9v10RaKAc4TrecXRV*N*zLj z0iwjBLILv{QHcTnS4=K@rDiLukJ9spZQlb!KfSkKCmEsw{@kM3nr>hcU=1$X#jb&TC)g%d|u$Ep`2reOp=+LL2!>;Eei^ z#@AsJ-6~@V{@CE4!Eknk!Qa$S>4zx>M3Xk9sdkdJKrYuEn9X$0$=Ui;TRNg;w5Ty# zR#)}cSaD95H8)=U$+=+jvrK_zax$|X39JWA8d%Mq_t!cCwd2q z=(T;oNQJS%iq=Axb+~1ckB>;J{HtLP1|9zkn2XTmqeXf*bxSW(tE{D%>hf{&@OkyxNgq zBbaKc$3p=k)3U#C->tNMw}&##QZl|vuBBj<~{z5wQY+PLG*7Unf>7e zuwL8^I)Yt&>;Nx&oSBV#*QrJ<-KxB;rP6u7JZdVhnfmseGaSfvE0LW zgI3RZU`Npb4Jzf zMi(X7Ls9hB!p;uUtwBurhmjw@Q|w<`z}VqzSE#GvK)N73_noV*LPCbUsL)@O(fediWH0Dhe8_X8&#^{%^g=;& z&Pw4@wVz(1;E>t+8+*q-8H$SeIfJ`WKOHTlUx_=Yt@B-pIFy*1;*D=XJK6zbfKljC zy7n?|nK>a&eAP~bOD4(q9!s1Zd(&4?MsvC*M0ac{i?QO=HA&F*I3^w24#9WGIT)Wr)x zP-YbC*tNm|R#XqE!l4~AI_fug>2ZoZ@sHTY>B{8q*QY~PaC!)CTV|D61NhZRwm_PT ze-V(cY%0+%6Y>Q}tkZYz^&9+l>e(kzQow$8z!4-|Z!e2>cNY{lEsLVR<;y)-6$p|4 zaE&WRyaaQCS9po+Zj;ERODW)(TY>CUdU$~n>vrl2mKv@@0jQ0O@Jg2z@(<=PguL6E zW#dT)d3$d4{u8|Z2~k-8KSupuy<{tFC#od)zoojTM$GP@|FZtjT3|2h1O)-f0|zQU za!~?>m-vwZES&!~|431@w_Q-i_~Ee5VSsC2z2Vj~jU$C;ZcT7KDIT9ky$DPD@x#~G zfbH6WxE<%(5|I-_=mq?pEAR*Ai~{^0Vz|A;`_y~>lg*^0o8I0Z;J4rOCbb6;=^T8l zc2mPH+OGCvqdFNmzh#W-e@Vtz5rR|_C3SIK(Et%RE4me@ujE`f*pxYcYTI&X(}&AB zgI7(h(sUA`!C7-Cd0A$@iPzG~Eq#g5@(D}TzWRkIbg!|c(9`{Q(IpbDsWJ6qyi>IS zs8t(`IPWV%sl$;c?c|=NCVSu8sDz2J=_a^7_))IzR&ZrM~3ay>&P^KxT427_1a^JtcGzjL(uqMguHr zY!#IRxsZ8MAVL|A2;kRhoyuiU$i~R8^|XO377F-#PKsm(b$WX3w!!!X1{};v1bFxI zO<}8#?MGdHnBW_SR?+^=w&4FybIapT;(k_9O>?f-P8`6tR3UAWpy=Xgk^JL z!=XQjs1Em`N(p-%cWu3cV%E#n%(f7$+%vG$Rj>S!5DeIz!~h1ygl&=~ERq#fNRH;6 zrx421bhe_>;fwlMv4pXyZW+JO@u&uzw^sJ&ZHHSA+HFJMlZY1M7RgSBwSqH-(y+AcJRfQzfqq<0rqIj_9VmHP+~NpnEwE zGa#9F5as7oQin;iEQ6z)yAJPQ&de$Xp!26p$2I4-w>~&XSHe@SIk6iMC>mX2M@$?a zkWT$q1kKODUnJG6-$*kH@xMb2nvKRbG6zly@8hsbA`_s$S=7B_*mm3@3Lp?M>wZE= zM$r#y>H9n;Xj3}{ZyFa2Iw6qBd8E2c8w2p~A2KVl_A>?dGDkG@$y%K68ponfoCJPb zfrMBl`?D?8CEAPu>4c04@}DoSxFd3v_LTCZ7sV3+QM#5UYU2DEc0ffDQiy*$%l!v0 z{~J{Q2>-ow1WpIP?BsBeK|q-QlOeTEZJe`9`m5-Br7K?A*I1keG^DYj}0!Wf}DS0RQl z1$i)rD2MZ+0kpf#W5cv$>!_A(Fa0xeIRMQ>Craczvf02pKmujIZ1h#d0U>&Y-8+bH zy0xo;2@h)Jet6_%p!&S)NtRPRTMtr}+BS3|&q9pN&wh>_-_RAV82nWlYvMh%9N)!K zL4(0zBV_j7GgW8K&Jblv9J6#}mNQCZULr{_$e1~L8W3N`$&FBG=@3Hl(NJ<*QJ5Z@ z{c}uH9*r7l@Hr}XNJ6n}bRTnJZz^`7!h-8r(+1#pRkt7M5D6|xg%9bq}}kXhr(8j%0D?WzQTL%G4{5{kSwU5!(r?MbkL7MDz5_#vbkwHM1Lm2zg+ zR(6!#lyo)JWoaMyi>l|_!$jeXRHS4B)v#g51YlvzDq86;#2=9)7)G_rJ1ob2IEiu+ zVWKx{v%@pnBDC?&a8gNAUM@C0De}nYQOtREu%zHgFO3pB+1|)B#N7@po){4X?{jts z6t+k`_PDw3jR(9Df?Um#=C)Pjxlj{#antH9ZyVSC0|w3}g`CAhnQy-UYNa0e(kmxfCC#Sbw|=)a*N3)`XWyF-Wn{w}Ts zVv>o%M?g$@>gYhMc_ynadweckr#V1He!E77!0x6XI!~2Vl_|nGYc&p1NH9B%G{e4* zv4zN{%UBnsE{x6S?!4%M&1U#?`LD_t`G0oCe{a9`*;9zLFEZZ)`*%HTH=7r}^;!mM z1Qhn4-6_NG1)Ks_b8&G44E_o00%CG24$!(Nk^9ip60!=b=as?Gbdn|sAfeDZB^v=q zjE!cVBf-7LX(O zHM_7mp%1!nb@;nFKTi-S*p$5sr|Q-N8Y1)dZn$?r{1KvT6`*Oabxa4SZ^*nCmze@Udpmb{{hwH zo`B(@O*^pq&d~>ekL6(JS?;%XFda83lP1kQIa zo`353`q8tkM4mKAQq1+FXwKtT{1rh6m}s4uo(jpG#?A;0H%(u-acr%ZpG5n)N2pzW z%v(Z`k*5Yq*WCqXYhbz`7MD(`SMY6@pu!ArF-X104F3W&HGS9`-3-6z z`_m_Sfp&=&VnogAM`hyzFquci=KsyU)dx+wO|l=?142|13-{}fwkDO@vk%to&udZ! zUgm>~g=HU#7w~_W{$Gpxwf6Yn|BZ@g!_ld}uP4%9Pw4&~l!23SP$U5Me`k3yY6eP+ z!e3~((6kAFRCyPDrEbuRp*T;%MQ@ywZV)96hTgX5L~kilRM^-G&XpLDJot*Ixn`ul ze_i7GU5d?Q`}!Vd`SL9o0fl~mdBr}_<7#u<&i0WJ5ia{}j&_70B4-kc23(M(hZv>M zLbP;P6WVHMSULk(<8>?Pr|-lqoL9bEs`@a@zPFmU`T^C?q+!EN10*@5C!NBgxXKkP zTfuy*bt9XlN3Y00Ta%v8T4oiR*YSRwmG3J*v1+E$KTBhkkC#fUu#=-)s^*=VHPB-& zT=9DB$xWYOh$-0NSpFp@!iso{$K4zLTRx+BB;8Q@%sURaV_l-kRp%J{L)ZewKO*~+ z=LOUPaZ?!O`}ShX-2QXT5=?Kab2wYM(*DPGxh(Cpuf`kyq7Unh1)>0GZLa4k^=pqj znnvng6ak@b=eEx8Z*?P+hXGp9ijCHk;NN(%Vq7Ul5puohI0hK}#cmRv6s%6pAm>)x)5kY zeOg~30BnCfLOVVjGL!Gwf4ZCP{$mB`2$Ip3A|U}jm?Et|wU-e}%Smdg78H8neQ0cE zY@uRG4K)DVJcAXA&-qhuA08lJtZn7%kk?k}vk5ElHn_$2aXYFMl}|iEW=zMj3V8IF zy<#@ImeZ<$Mxc{?Mie?u9o!PLHF}*Fuyg-KC{}TW2r-ym<2G#QwTS%*dc$`}<1Ua{ z$HPMGdoEv|k5INjsCB1}f(*R%7;X7RV=|WKId;%WKnM|d=ipgv*)Ce%nksOJ;Zil5 zVjA`0{X;o>r&yvXE*RG*D0~RP77$uwJa4t$gZsWxo0v2l{Q<6inT6Y9w5sT3}F$ zY;Qhz!fMjEJ*l#Fhs zm{6S}MDi?HV?^2hO-n!ma;7feM4P0N^(aINgbqVIVd>s*PKs1qTYaIgj%J@ek-)Y2Q)l!65$CFNz9euK4$H(;$Ph+5QWPCy+o;$uGsmt|&he z@UK29H1N|D6{?j83M~u_=AZB2DO1G(ab>ymkXoEgOz?5R82&a1>n0KdgJik*0X7nc z&TC?zTKpK!OAWS+HJ#e7_0!7CNzCOf`kQ9AInDyA4ZhqO2tTxJaXc}U(9)Uho(HIB z=hF^^<+%;Yze`>JR!ZU1FzfmQ z`_IJ5#_KlWoBeVKk<8yCkuh~>0RnK!o%~4Ef?T{UYN68w<&_LJ6kMsHUe%wW`RvU4 z_z$mcIb(R@*7S!9v$V~mlXh%kp%}es*#pI;DMv zb`k)8vLtcc)hOUk)g)XP8?bU_84x>>CSEQn(^U^;LDc z|2c$id0aJ2xKysdN%yxLVLlEbLTCf1l0IPSAQd|xhp-w{&lsWlY0+9|AmBCA+&-d1RTe8j4%%taI;VAA zM4TetgqWq@oc|1?;!i9WkaJXWV(={fTf|A$1& zzqc^@G%CC%`{|Ex7GAlA-2u(uzEE4?b~+3~zwBs3ZJP{$G}7{h_Pd!(G{T2)C2<5P zT)$(&346$vEkBpq&M0pLSUuGO{RyK~Pl{k3ShEJoxpu{u+!fs4rtr7`VssecoDp}m z>G!W;SG#uYK2i?5$P8w$q~C1npP3+o13OdzxRE-aLcUNxhO4&0AmJf)^$n(y>?EY6 zL)oowmn6slbmxcEeEkdk6}75a?v3wlkjLKE?N)d9O_sZfkDS>VV{>3xBgfcvBV31q z38{kwi4p@Kf6^#)2=jN13C9#^@CqfCN;;D-%uCRH57<^A@389x72^ra%&uWLzcL~F zgyPF?%7#u;R)D;wm7j!hqQze{1>?lf!)^os5po0fjTryLH7)r`!|IvRR72mt!qI=w z^51DrSUL=aBsmDkg*@=LAqgkIlhQkB9A4+Aur$~pVM7oYE*2uWG&8I;7z`sbxfm;J zdITFf*0X;(x{tj<;nKzLkLrjS7LZcdEAy8{LVoM&|42qccZv@aJROdCgKK=lJf`Qvm;Y~ zdGrHHXtzZDhq54H;^-h0U{FH9LH0yS{~^K69d5$0BtEQ{B6lZoxwW{MXvg_p@w29X zhwbGLUVI#0*Qmn5iy@ciZ^NL@t**FSrv>qC$6I8GyD-KdtSQL1 zB0;$>YAixlw+jxmM?^S)_vq-qtHU!BFRoBRHzi50RDn;BL4fnNuF}%cDWQYh$pdWB zhm7&swzk<_ErWMkE{!`fhfl;_i`xr_ieqSlcRlZ3k~l7d_tY)` zRoqTzo48ZA{VkFHdu%3_9@}55+g$y3OD>LE2kgJY!vKyK1Fmd9%m?V30QO%6>5tCL zofBl$SFum(6rp3l*g-DT-$@0-PjSM(siOdhJwWkeIlmy=%RPq8(cku$c)Xjz^34IG zvBFVhh=<4kO&a<&qO(j&SVk=pm0$UZL4gEbP7$b^*kC<>8-6!=h4`C@ z)%uL!sfzJC9pX3|8uco;BNd3f3X@CufJQ?N+BjA_rIbn23jPwD^CES)iX=o11E;C4 z>Whs<1+PSQBy?ItJsdEDVk3jW>{kNCv91{|vR9SlTh{`>{E{7q&w#oXdN`C%cUYz0>5A{vl)vO2W_3TWe2X>QXlR~`Sn z7OHzza*|hTlOmT*rZ(+fes6Z_2h_dNv`yCHUFlFy%Fjrj$Mpj>Qe($(xU@okN&A)Y zeHJ^1=vERy+5@BJLcBP?56_me^d@F5V#MCjUPX3}octQ3V#Iy=)obumXjN6y)N}D1 zwlR(MctuldnMtpqqok+a=j?7mm$Ifm@LsH6dJ!+0@fnKO4~f)G%|Sy!?5Jz~e92O? z)~?J!zuQa3RKZ}XVqzp?`0<<3FndQ$$&bgx;w5)LO3Y-cX01b6O?A#ro7dTS@zP6; zqf@vfb4ioV<+cV>GN!z{ue0A)!3qvClXt+_h*=j4yl*HO3LrO&cLf9f zc4!I^n`sFJH_S{J=x-O^i(H*Io@f*fByKKWf+v9_HD_4^Wo;yDW4W|pm>iM}+;M zVjR!zi}32VEFk@Rj-^eB_&x9BdSZlTBN%A#j1e*J_(JHh|D zyntbh7#JNEV&(7aNmqPB`xO`}{?^e^KDx6UEV!-W#4g^gaBBz6PltHyJ3J0QR#k zId6`edsb;V?5{wt_9WzdsOSypsxmR36Z}Eg&_Ud_YKLv2f<}eI3@Lk=GdD~l zxi6*N8sSf-6N42DzL!0Yjm3MiUE!2G$Ob~48wvx8TfMm$ zWSmYz;l_BS1}QrZiwXMkj_Y2`*jNaedEiQjE7}KoumI$=6W)@5p7{bTr(dld7E1_a zSb^FbEjBjh8Qj!%Ut>y0|B%iNU!k(N5sNJHKaqaCtwMmvQO__ly#1}G>ZjarX?-c+ zU}?RICHLbdfd(lG^S9-D4+sWGLs1qA9fW_nxQU!fnK6%_28yzJLlv#TxvYMIwCJ>J zZlW|m0Lp;Zfpb;JJ zOKfeYPIPV#0}GI>c3ES|yt3u-R}kE`rzUA8GD(GS>zk&Pu9F*=HRCX30y088o zf^Kg}FoqF5u}p{e*AN20W}mPP9vr-JZ2Sw}i;&~J#g)&S>)JiKOXE;m@+NAu*9uAC zz>Avj6zj(tRUc92HSBMhx{R5CQRS<$l{DI3a$B?yN3`=~>N4vOz#xZq<4Wiv28@JX z3aECDYj=x`mfB{g85Q-i-?yx0!By?Go4!jCcp^qA11oCoMtlIkNzBSWy3D*^TN_=5 ziHI9{k)Dq+>Frr+*EOk?3{23-=MoZXa+FCyszIG`|04^s@(_=txnqS?@;Nh_Jo!7# z?|C$1hMFT)aXs)cLgVQkWvQlVBA6HVTnI+@`ND(?dX*~niflbmmlpyk%-w%BYB8Q=7evlA_EiArTX0ZngVEN-GtJv z1GwsKozVK|K03PMqTpCzk&X7o9NN$OfaQ~6ml;_MSh;F1WSZBni^kgg_}Xf)6vX^k zrzFP!qkVr1Ql8?#t;2Nw!9yATHFXuAsUjtHXwtRc3%BI}kfUnKZUw+MjCb~=?R=qu zBgBsB6iAD-2T*^jd7)DdD-9;Tb3U*D3X<%RVIImRN9MDehn{iD()0)Hw@R6ti375? zZ|2Y)m=E?=Ld{?eorNvm^<=>0KY5u}S1%(%`BmG!x(E-Ia?&u&J`^asMI2dEvcD6M zxpE;##!=w`&Pmk_su)G|g$fvjIlfoOEt!{e=o_GJSQ>3`oN2sqEIe$-yaYKp6W9yK zU5T&zEBtWmBV&`3kSvCo0Eq~t<;OO%DWg+xPwB`ge-T%df@-cl8wjv{L&+vyeLi)R zr|2#Ud^kX*OE!!`I1s0{mySPlziT6;bX~Mg%6DJ}EXADDdEIg+F{yE-pI6#M<+o&< zTewG+J7~0~pL^u|VrMjF6;+7hQpRzZ)bvjU0=!;*ebgt)yMO|mxUA}58heICw zjq$?SZ2PM2H6hR_@RVlxgpcfZl`L|$fS|Tj+7mlqA>y}1d1#=LYv0780?au7(F|MM zG;aI|BHCF_sE(Xp#Jn^}y6RZAVG?WEM{QjJkP_d|2K%e<$e`x0=RbOA(xzT>f+L>< zovxQ=L!~8bj!j-yK^~yTnT8;)=PzX!v{HVPaqfHmr)0hOtAlA&7Fm9Kh}akVz0yZr z{kt7lCVAyNJaHuX-5Cq$S8cv^tXz!^;wWOgmYB5|P_TG#J7yEtF|J%)a<1yDuvGm7 zNcY1W6wC$beL?HlECR^xzG8P(w|(t-$gIxh@&9Yy32mxxEAxyYlaSPPc_eyRl!`sRKbuLt92I{axT>_(#e-T zPfo>DuXaOAxT9Q;n9?a(FU;XM_G74ZSNyQQ!%UQZmo(PWDqm4})LXm9yYo^5m^}&W zbI>zl$p?H*JsJ?HS|@yJKEwOjXTdF;ee6S?`I{3sa>xIn!CbsZgidhm_#nxzU}!8- zZi!R!Fzz&@6;a6x9(t@7mSyZ3u)L_@Muqc{eKvi|Pyhw5#YX>y>STOR*_t zLJX~{vj%SO7Y2p;tf=3V_sY<2qtb^oh#JV&OI9CNnLGf9lK)ETWcWq{F4L?>Sp8{9 zA-?$095whebV_-Ib0(2FvC+xRHKf!CZd)}ojdFy`xe9+>{iYIG*gg2 z2<=^(FA}W;-xe}Ga)0j>9+_IM&yvPZ)WDtndoK_kWuD~*{il_a)bMQ7!R9ctg23J% z$PyKB>@ez!SbmE*^Nwx+^mJKFte|tG`x6g`T#;klC|RE4ZWdIEbuJklxzq;%^}||d zN)5}Ax&~93zUD?-N6p>kl3c|_rOo+__LlaVlLh!f$q%?O^iD%PjeCll;ZSUR11s6T zeOhoV*A(gxAs@szw-4Wc^_7Z~L{ZWyOWAcYRW24ky(=^0Vqr}JLhN0wWLA4Y6)Po> z^i8NDe&cipr7X`W?W@?nzl}id81JkjT)BS3>ueprm7*m>ZO?+bf*;7|Y-iwsbC{r| zQBpjd&!v&!hB4rKT#as&5$KR=b36L3NDy1_q}?S{@9UUve3y5MJ=S3=`!S@I<8J3S zI#(A6de}_98$+4{Xwi5*ypuILpK#YP@GTmD5^z>HbJaVhMq!YSb*r+!H=XYKnsLYR zSr|qGu6GfL6!FlA*6sM07|omuIx`jW04D)pFFcJ32NfiMGf}Iw8;behi2vgckV=?4 z{6Jhj95cO-p|=)hfLZR%l~Kxlt(YEbDr$!biJAYe_F#<_fcn_(j|1(GAIDEpxR;m6 zGFvPhWhF!$f}aBE#s(Kb)?{d{guNqkz2d8ND<|tJ}KE$ zIhLG(Z(S4J14;sE+{MOrqJxX3m`=72{tTKZH3nFW)3jsQ%A>#ij*|k3;+}s`z(kR5 z#!_zltbwh8AMVZm;U~iWz0*G_J0h{L96V=WB@-rwhI`-`4S?4y8ylnc-6^HtBAa8Z zo##EIP(n~u#Y985H87zkJxI~I-tGUf^$yOFhh5k1#I`-LHL*3Z?POxx>`ZLiwr$%s zCbl~E$@9E*>iy1Fr@E{A54d;l-`#8Pb*)*vMn<+>Gt9AjQNS;s(4r~D6W65W6KCvK zZa{05`UqOB=1ZZ@GPrlQE8Li471NT!22`#UJrCl-7&uYq@;w22@)FA z-FcKINng4UmHaw4V3x>2bP(wbeik8R^FSNWMqjDa1&-M|bGvhJ?J}#Bs=R}zT zy*{sOX=M|Http9eX~(NN@1X#eTwGcwDIxvPDHl}e8{YL>SWj2h#zt1;;eF^q6#?v1 zkQ$PapBlH`f|XrjQlnE2UxP@W@U4mP5w^p9`8x5N$R1_bH@Y zY@(xu$6!k}Y}K#_rLfEyu1n@O4y)Ak`y8%lPu@!CsY@fq2j-65Y)1Q6Rp95&?%UHd z#dv%N0pPi_6XHIbvRw6eb0EQF|can0FLr)rV}FmsD>` z9}!B?PNp`81Sj%}Tf9;Gn5Yfwjq}FtNBWqK0n7}>2yw&H2592poZ^W}cD<;yIm9mV z;)1xQ`4StnebJ=9hMX9R_S=GF%JbtILn&sLRK!*moY$pF=BHS5Y>R@N@#u?q%ptZU zkLPq7qpymQzY%ARQT#&}LCbO{EU!9WxgKC2F^C2^^V~A}eqi%wAT0+it$0i~%v>%J z08K?y`=i=`ti(xRzzMGsp9cHdpLa9ti(i~We_Hy=ev3$q>84B#eEreYd|B!ru`tJq zD!(keZ~sVa`9<5~&KSjiLlw0qcH|4Jq+6FZa=R>X=%2maV$G|5rcQ5-`W1mRKbX&u zz9^c|vHgoCV`lV zD#^_&t8_YRVHkEbh2=7V)K zw!zfIyBrgD5iNk$o;m|H2%)IbI$`Y;rzbZnm9I(?ZMmBgA~$WzS20>*i1Qm}=UH79 zAh-RQHqy~oo@59a2}Kj^Vt*au@%A|q@uF-cxYx^G;L4zDQDC~xRoWc*1K7*6f^s*v zL>r#w@QZy|Q%-+QUF0vg5(OM*!MZ`Wa?=5`o#>qsWqz`6 zZw=|Z1gf_Evf9>52mC}j(=V!Sx5i77JoYUQc4QpoE{MQ9a#56BCSe5!zl@~TbvDIX z8gFa`D_x!BsP53%6wCsl`hSIaO-^ zMlcO!L)FOgqUMj~5obzkW{P|8rc>Numj@Q&{jA0KSC>L5q)MTj8gZh;P(<6BTD^z5 zK5mjuNomxf4il1_zNc>+Eq!9jFo?fgWjHEzss>}vxo2tqkho@P4v1$avk}u0%j!F{ zXl5$$?^@XFms#v2=6SOul)o0gOO@YCxE#a!D@dHr`@}CbHf8RcI!sv);QEH7Ug-Y~ zo&xskaDM8IRJDvcjZQh0#mblbs+Lih20kxOxNz1dqFJ?%B|-2lAK{b4P3Gk^Rh0vZkmp{&0CBZ7B4$}ocWc5SVr2AqQMM~I_9-Vi+3>uHbkvx z;}4e_qv{W~bRut-MZ3+Y+Zv9|<{NB-iEEOdPwYu4abPuf0ppUs;aYjsTw_pWY$Ht} z?<<_G3Dc!afzfb|@Uo!@Y-%C~@f8uv=Og^Ep;^t;XN7sca1-Y;3-JpwgtPb};36(# zwXH>UtV_BQfHaxC-7yS_TZI$C25YF8^IIsVgfXTvVEM@2-R6QJMIdkH0sbYf{TwAC zkB2H?cJ&C?019wp4*>l2hdxcM>=}PLydGmf3SWkjJ9*;)uap_1%(*wT9b4298*6Ze z1>KuDVNC(9Eri#{;vh)7-v-s7Pt?tFt}c@Qhi-s=fQI z0os<-uJ$Wz*b_K|U|5l|ok-zMmpDS?D`G8H#hT?kz&?@&p9`a>mzZJy8zSJvdrgK{ zuMar>6TG?!O;9x`WxBB(3+|IyVYG-n-LgUemo7J8*%jTZwd?F?A*v?{Ej-E(oc5#S z{9W7!ND2`u@4G`>kc1pqIj$I}L4SUHo=?lJTd`JN)ZA$yRSq zq1Q9H`B=m~(|SxnSBicS<+FNYj(#54J1*ZujYthZn(-Fz{#TD|4eEWx#c!H=4$sSPNF~0e|8gKvLsmNpGA;R)-mw`H!kZ5f`KEoe&u7e^pB* zo%9#oosR2Zql|~YK!-{>iA6fQ_6S$c*vp=XlZ;S6+k5;WXuSx&0I&U&HtV1==}$(u ziV61C;z-mNj^@@*Y^(Q=j#r+_ive(s?5fj34ekZ=n>5r9%Y7lEq^#jce=rHFX0>#qMibYEcd%rL|W4az~ zM`zn8ECy~6NKER6ncdfbp}wEE1n~=#>jGL#M42nY1h@XBE^dcTU?v&_G%*uAS5wR2gV-L6kTQWmEWt}qS_sX<<4 z>b}UQmcND#{Q=1h#lSyEi6#`Gekm-lUMiT6!+%$osL`1n5zg4vw9!G^fLBzQPC4(L zzQp}SkY5G0WFuk7Z7Em3sZva9Ny^A5%QCKLmixSTgnwaZFUFvq*|3#GRR!#_dKFD) zceq-q##Ye>Eyv>uQCpI=sWO<9dylIYSXAPQBrIvrDDRs$r%T)|k(Dbdo0riPnv@4l zs}?niB`H>0ZxO4$eABYQlOu* zhQ0$cclMqY(h|>g7DP7!pDZ8QCbRgE9vs_16;l_jrIek_1Gxz{v&0irFp^3wxmD>4 z7gP{bE40d0BP`muBh<=@{@qw>RU{83TrF7JavP{FrDwomoWf7ZGUv063hOPXtr%CF^;|VwaHSFS@(OB5 zh^LKrJ?4ZXB2LK+<8(_&yf?M#0O>Fac4}oAfapHJk_=B}<)z;5Ntu$M!HuT7wW9su z!?<8y0`j9lfG=j>iJX0ztH{M|G-6*^}zY)$zw8|I) z#Po%HUFhd~&reVvFxqvt1l7y&EF#*e^_ob%_Si;&zY!;`c83ybJ)^hO5%hP!xya^Z zB+78;DGh63H~~4!FI5g%JLM*$?c|y%`16~i=dnauLvt!pmwjqk+$(=`8JOI=5S2s7 zf+02qjR=Kbl5K@OLp5wH>_?7CiyC>S&YQD){-r7hlI7tDor6Ge9Uo|xAx_N>B|b39 z!&1J>g8Y$IB$kdw$SD}gMt$pU-^QiyQ;!ciDg?*pVh2P^j`TZd1>H$k=j`*=YNt4z zO8lO9@Pad4oOS%z_UCq%-rQtY;E%NvjckRWaHB1%=qS@5{;R%W5xfqP(2*ji{xk>r z@H2DWrcq)`?%@p4QpK(R{i@itR}91ps;2$x{#NY`|Dl7_t8KkwfZnCAr#*S*r%0sp^)C0s_rBNa()N42dtC^s zf?+hjc0kJnO|nzdvd=w&sDz~WsCk4lgd!5fF(;tO+cR76z=I5AXArwv4Q|KtLxSqS zQ}5g*^!2w2pFi?8Ouz?4l+sTA>DdM}+fE)$6B^_ z(MQ&0M0s$D>StiZ*yvV7BdtvSDXZ+=rsT>xL=^URSMzR~Gi``VEsm(V6ZoN;U2R=$ z79zkJ^ZRG45<*}ds{;@+LfHOuBSzp*tR;8Pf*T$`04tIs0<{S=(s=t}&BFu-(H11C zM1EZ$2j#8Z&M?}e86Rs>L?Sg9ZRMO?Q<+ca6HMfVtxo3Aq8+#iToYe0ZR0tQzM|v*L-cku#PLfjQoKuo;o+D6NKNk<7B$l4n0ZQ>IvR9ZdMsc* znq-6+H)vBG3w>;zhe*t?7ml3!)F?;i=85J;EdidGM0)T-R9HI8h{3WeZQ8mFtmA}d zJ8f)jY)x*5Y5kNyY{x3&blkOQ(}B_C$gTtnnYyQ|pQ|MSQEIMTcBO`StZ)d5h-TW= zWv&|M$_k%%vn$K?JKDCHt5#P%0uRtqAa|L+#Z4aRx=;YU@alZP%)G~{I6m?;M`*kv zxI%SbWLm;6rR(oj!?@uwg6KFl+bCZ4#!oZ6c?D&jFSC)dBfE$G$O8lSE_lr(m&(KA zQrysBrsE47U@dm*GWc~H@-l5|b(SEfA?D$8{Tvv$kp9+jB8Wvl$!$)sj1EXKcW2Hb zo8Iko|8)c2hrM_%0lEECqzmQ?h>7~QCC>f>1ZIUt;>Tww)eU8O5x?Kc7jpLzX<;ML zz|K1yOfThP;U`$kmq5E7K`}id)YiWJd;lUA--X2A)AYuLI57d|!i9xsc+WZJNIe(h zg>(0>Y3BHChl5#t0+0J{(i{N4b7V1qb9laoVE;sU1f2UKb(CR*Y}#Jt$iV%WbDV2N zar7mcgux!&Day!_cP#ZcBXEhzNw-w~s_+S!ezP&b@)`Gj495jipLT>gl*Ph3iy=c|^BhWKLPcr9Yjv?0_EnFYM2{6RP4a7JqELWq#`Nl;;%*A>8hC-!caH&)p^!`HB>P0CHGcjD`iDc<9XObPk>p#DJ_Tx;-3tVu4E|% zhD>3470IebzxU8`kn*CRxgsYvB~j?nP+6!1&bAZ5 zGp}Y}ISLMNO$?4!j^ke-|JjK9-_4}|b7#rndn4{%X#v9r_QwxDqLjEeDy(D{T!@r4 ze`*QIgn^?iHnj(x6uzOKD@ zyclOXKSzyzB-0jwuyNtU#&yK7RQM79*8D%RtcFXcFgC^!`AE zB%e9P9w$=+r zWO2MEQN2pi6zs(aS(dSR?h8$h%`Kxv-BS7X)Q!oIT_C*j{}`^Ca<1R=kHAtk_o&|T zK>Q^fu)S`W(nu7OEvvFh(paIiStviYlu`?v$}Y8~+;8;JeS9C+LF3J0I2!*4D2G(< zpD*BhxZH5VLFq6^>Mg#f;hdtR#9=~kY5O^8F`1CVk=*K!lUBC6!4V&gHVn*dSg3P* z7@uzN=Ea$CRPT&VWS|t&<@b?EaY|aUQB`OcZ;6+)cmXe5Ra|hVOO6jsr=FBs4Rz%p zce`NPGr0RzPFG#EFQ)X&wiQGP(35^!mnIFiV$z0#Kx^}#u+oNQSW-!omomV&^pe^P5f8+4{mcF_UrWGouc!82XdS#P%(6F?oGpQkpFI`+WCzwvc)7g4 zwX0$5W(h?<{@coaKd!33oDXu;;rIOHd`?Xr#!(z=WO3Xwu0$b1u~Tmmz($ND?+H8X zL9BCNwnMX`i+{2gw;+aV-Xls z?_=7?B?kVwo->p>CK~xWQ9CyQ)=;uzs*%p3cY&naItnx@dC1maWDmCAV#BmUS^7cN z%Z9pQG2fxEP?%_`_K!6Lpc=0d$hW2p+x&cC)swVe3mgQE^AMDtjdz`ty#%gTiPu!> z$dA!#cd^DUSP!8FgInKwJxXkEb#39E?@We=ww&H>!K(ne)6Qh-Tuf)C8`m^HJW5PW zVT-4phqmnwISmD>cGL^xrMxlP)x0v~+IQB$pVU9YZ?(Cy+u6kcOV6oi6{12D%-D%G ze~HxcsK6IQsJA!7zbFWA1~TD~VRK{iHiQ^2vmi1_Zt} zdq}vSp?9pA5qIc|k?h9`^YzOklT@Uc?dYMSIjog^S_pW39gb6RjKsK~k#MZ<5Ll(p z6~R4Ye@H*8iNgs22w%MdY!|UL(toT@`qdD=`g=UkBemv)KlsFS1^~vo@X5hjwJPuG zeQEprkE~>M;sZmoL(MDh4w#o;%1legB!z#zJ2=Z85CGz>%j+x4>k}m|l#Z8da(|*` z)JeH!_4|_bCaoaCJPYU2LMmOY#k~{J&K}NlCcELt;-U}`9!nxittQ2)6Ya%%f#%p0 zYjxv*a6LT2y6G2V*{V5O^~)h67U`cYg4;h*VSc)S4U^?~O=-*>Dnky~UN7)7EurS$ zA>=pSS()tFPYBohT!z`dm)jGf%Q@-@Fj})UvESB8{yo31SRBi8^0A%sCJ6_3#q|Kw zE$yr}=Qjxfunw{|R;~;Ln$>H|bDd=t`)yZloQ=aT{Qht_8KEWOt!+*p&B+SV$LyAH zZmyL!)^K+w3Kg!GlE`&cXdEA*39#b1`52%h&8#1z7GiUIFb2Dv>6W-=w zFuSeRhJiXZ3u<${7%vR}PA9x`G8^WVm)lqf&r{R^g+{=aAir04?NtEF^K*XmudbSy z84|S~{nLZm&S}A0uC%uCY=&={Frr8~qoWq&ZZ4F9%3KVkyf6z4{|@}$H7IZ;FX+As zUpL0^6XeccIyes;O7R{!JcU0fEJR^mRh2ukexR=iNr-QhZO9$()H*RPJ5Y%(I5>eB z=iva5<=>_ettRAnI~$Dr&%4QpuW2krLn#p9z1klLITWtyTnb%rLM9)_5^`s_0%?ho z*4ForX?%iER8;-;=S**PypRmETN0u$9FoD_j}&w+Q(4Ew?7lfzl{y?aSjGJyL`c)+{zPOs-&w*5*+ygu^_{AL?bG4iPfv?T`vU zcW4dF=QE`tar1w?9mEIJt`eyN=4{;_2d?besZc{u?w{ z7F0u~n17i3sWY%+Wijbe_EV)8!a&UwS=biY-Q&Jd?-Zgh;?tk}g(Me+oduL5&JC{Z z+Qig$*r78r>h~Wlu|JP27)NT4y?ajWm2WbHKjB~^g~ceJ~K86EhP^fN=V8E zm}2!f`;b^Z5f#_MQ^a%^+<~G1-2u)#%Ufi*gQIc+;VmZrkbouYufD%34K?}r_LQf( z7X@xcC|8o_<`K%vjIM}SE3bxBPA_8>5 z=FB0|T6N9>T0Xa(UCtK9LmkM^d7XDCDwm%~pQ?!6ByN8M8WSAn5K(OaOi1fIe5`u4 zPO}i>O>ZISTgkcUI~&yDttqYQTc+x@YYQpbhL)FKm44Gs$PALF;fu8>1U zioO~OH-Phhd>u(jJJQup>B8ycTk*zNpZP7eGuw zCm~afpgV-squzJ1&k#(ENV7V^7n+&G&fEbVX&o|!3?7kBffg2NeOPM$w?H=04cLvK)ch5Z0b z!Z1nDcmMA51Coao9;prKCoSHqJSdc3Q^wJp7nzT4nbn{YOjDJLt!7LBPZHs3hKvkN z&Sb-)>EpaLw#(Vk^~T)%*J}LVGy6Ww`)_;B|9ZdvW6z@VkaYL_Hu!+T{vSTBtQB4aK$4oa`Zsj< zQw}v1xp*^UIXsqTI1t%ziN+=tl)7mZ*;0tgolg>RY}SBcTBh|r=b6a+mK?n!xjh-F zdWmT1kUwDzVpMA|B-6<>r>n6kN~>wA>`3i(LScsCHS}s!h@iuu{&-_v3k?L=h~a|2Q8HF zoF-GIp(boBCeR~Xk{IKgOo7?si%%n)RA>EKG{A7uO4fl|ri*fNfMaok;*lD`;d<}@ z9OJV4kO+c>>v{Yp38NP*H{d0Hvf=GLw#3;sQ3)q&!&PhhR(S1Ut*e4NJ)4B79cEoN zb&h?&WML74LxR|wzcfLLw6-pS_ih2MP{1wQuscBwDnF>vp9tgqH-@3YL{w%gn}kkU zXQTz^-m2sT@EL9z;DGgFo2tN|9u1ejXI^YT`-opu&a4(W6fy2p+25 z-4NcQ(>{uu%sQt?DzsF_t#MoYS;IbAu8JJPns%;D;ANtwRgOL4MZ^C)(V!lpsr)>D z(8Ur@oOR!*b#8vuo^FbYUY)IriVhaYEBYHkTu&54lqM1u@Ru+6Hq=xcej&_Wmm}sk z^ImFPXXq#y#J)EV{AuReO|c!K(>(mVqz2j8O8mV8Wk(VSYa zo?aPrfEFzdU_!4_g46H-q-0r! zecyF|wlsl0;Ji%rTF{2ZeJ%^mHqv*`xK+gaYa%ui=Pp~TpfT+=71y?Po4qvua(bE8 znL#-_6qvsz4ar8GQs!$3qm3OsIb38smiG;B=%QH$l(h`hZMlztoYbg{`BsDZq#%~P za4#npwzrdZjS2m|uuwa{WEYzW&^V5pzZ8VsEbhtW=(|Du@Y8lrS=QrTz|M2=?^bG@ z=*T`*uH8BPH&9qYhCtF-?gFtn{MHC{BebAV!05rjqZ@FoKBzy?bBFskOZh(XemOUm zQ$vIZkR>MExHt=dePA5hbJK)0huWo16-J$7Z`O>l8DBDyM4ex}Gyfu1^m~*i_kF~u zUSg_cPv9KLMFJR6rMzS94vyL*kk&AzXGoIGFddD!)(4;Ww4q2`!5BhZDe2^(9b+Yy za!Jkb<+sUS+*LC!xZuvD5XJ)nvAOc*2}`sAM4C~ge{meCBykPm7n@8ow5a1{+v@6D zQdMv2{CQwMA_kzJSlTn}yJ9^?4nZqhZ3E9!vhjedbJyETf2?!^w!Ufb zUcw!fiu#5}@CHQ9+PTD7=6FK9ZhuK#hA8ZFpy=f9p5q&}(fR3GIgx5lN#?L~3AaLt zA3vb}qq+QVgZjT)S4!m}{K0>@H=CqDP}A@B)(M(oVD^m|Ve0x$C^bqV_WK_!&RC(1 z{Uypbs)RA+Hkp_Opb6uwvf}aSYf;ECVKfJey5R>ADF#}HV-y$`g#0rf!U_aV4ac7VVaspl81XAf46 zKoWguxykmMbe& zjv+{eQc9FZGH*<7k|ZUoIMlWL`CyG&zY#)ivQU@@14eHMR<@yFCoIU&vR#9*PPr_= zbXJBEV9I3=@H6FcNIjxDa5bc!QZ>&;n-KL%LrY1RkThZ4k%TkPi{MpZDD@mF5r$Si z%@=Ld$fKcLo=0WYD4+d>MPGsedgK_S#$A`KV`v|wp;t7UBxjvZzSp4jLoK3-4OaK? zcfrrPWPY;24lThIPIucZ0uj0Lq6gR|t~oipKs9z)z*Ppgb5{iWLI?!V5@ah^p#Y7? zqPo}4QIW+)@dTHWbsMIjk+U>u9o^DRxTH;vu(nq_t1+rtwRAvkBWsmX#U(4jbyhBj z->^hi4%^VOC5x5k5@qid{Y*uIvh<-Zm#`ece7Ba6LZ<9im*^qn9umNrf9Egk1yjuq zh_U?*AhpqQ7l@HTW>zYVP{7U!6}ZYI4zDKg-B~cEWqK;4oJO zp{9I0po$L_A%L}5QH?QSitG5@8Us6vk3`#aJ-K`p^}kM@6`O9bSB3gZb;Tg`2^yuw;e(w;bV8fNpcoo82_e)Tu&25QD}zttu3aB78Fpu_Ir#jFMA3`F3^z6zswuFXlvS{7KJq`)(}$R%wkg4j&cL&tD?N z*vB|D(N~&|Nu=_1AAlF}P~i4uys$$@rO1&hQ3j+$Za>FnIMXAaR$ngryp04CECeJp z$UPjT9#mg}O9~$b)tX@~Ov1Y^L7|ePP#<|02p}Wt-A(2cX|=$09fjQcRR=u)WXH}3 zUhg(zHfN3bpGoN3W0X}pCV3N{C4iT9i=HNhcVp!WEyBJ*k7ErC8_Lrjip&VorFh4I ztEhXKSj!a?Bamyz7#^9rbCG!&Y`r>m7>RK}CKvb6~n%m(xg_j?j%K@}7z4T6dyq<<*Vl5`^hUN!-gVB}*(m>TYeHK;NO(fqsq~2Ib|T6*%U--aIdz(yVdN9NhJU6NOGT41XxAss zCO9BCS#;7=*o8gQ$%fYHjPAHo53Q#E>=s708%urWj9SEwJ z@aQrGk#8%PTa8+}gq0f)m{X7CETc`S?EFdneBw;r=_)xd!R_!S3d445GO$-~4L)<& z|8;5Q4RR;8Yt5EVoYURMi1-cF&yIi)$FSRD!>|jR*=iLZI?s1wOTjOH6XMw# zFJs+8fj*B|y;2i%2KtW;@gq{TZ)l(t&t0~ZcAb=OLk2m}0w#A^B9zhMJpWU$L@;-7J4fuhr0tlpbF3Wxt+2EWFc#4*}iV)VjCNrhZ z;zv5Y&eZ*5>ZESTk+$CH($b%m)sp0A(b&VjZoSZsdk*6554jmI7Y}X@IJ0BF9$vii z`D_&h9K@o#P9>HDamsjHiDvt+o+n~0)p}7ihHUx)r|`h_qiX^Z(f-ph;4OLkSH`0s$W>yfMTHmw?WE;2eG zYayZXZTX-3u_3lz8Tk+QiN{csjJ*u?ASQAO0!~e4D(~SwZl_v*o%tO_@Lq?E`e=F$cETwHU}C z*|E%zel6%TpURQ7{dgm(HC<0KuCZwC9Va;e5Q^T~1LvS!H|)d6&&VL1a*MXx9_s1$ z3R*SGfTwZ55z=UB`+UOVkXdGg#9q<}&LWbA$^4LcH?+l+WeuRl;sk5?G*;@~QG**-eGV;gZ5WyNJ{u+UzslOlmz7D?S6{-HPk z@$tDQU>m7{3Bh>ZpYWAwr#(qQ<3b8j5=+(oa@GxO0k$;3!0L$Bd%ZzzB(KB71GnfX z5@hSH;8c~zq3#Rt)3}t|825q!4-k&x28wp^5VM6*?)bq#J;^pS_VlDm+%osHA0>ujP#1CLj zski3o;s8l+rJ=TpiJ~4A-auPrY0Qhd9&4+BIX?7w$VqeTkfl}1 zbOOt`&9w&KER2?!$)&wD>Bb#$e>(>1*(*?7I}Qx5Mb1w2u=1TcAk|ZN&s%cR0Zs=F zirYTXYa5#^dRIvBu{*MciD1qP8cEu`B#}ZJ_*_0#5J7!lpE1HKM-l@YCr~lU9@?*> zwv`IoW86d%{HJcbLE})f5-0;u%cpF%O^_jCPTTHdC7@UkC2!u#BqF`vl0_(t3s}qVh zMAn^9mJv-~R^;g-M^u-m{Hjdjc29g(wVl6;AwbPFll7h?|TE7ox9 zq~+&kPVOsYb;D3Kz*tmqfw3*xIYJVPE%1DdFeB_yM+LAqbZ(b2s|T6i!~ z1FK|7iaU+wKQd^s5OxP-hPAn%U9<%dS^gk?l@s-=n9d^`&BxFWm?xUc0-`$8C(f*< z4cXK^c(Avd-x_~w*<)r)^-12pj4(HuK(8FsR%yL5QwzyUr7>y{NQY7xlA8Ko;-lo! zCB`E-QA1%QMo85gv2o1`C)5T|7%_T%@TZ3(ALhjX9YSI4DRX{4DKJORvGR&7ID|_v zPsZI+h!1bDBJ~vfI!9jn3rNE^n5jQ{a+LaD7uRIRt4k)m8qKGUEo;jR zhE_EiRy8+TGMhcSI~DFz@Y5f>sIyLn{4STc79G?q@jh0_&1lrZ%L8&5Ykt~H8P##Z z!>QBEL5)VZ-7{fi#oW<%PVke+H6u96FbimkKIVY|HE7!$frxv#7S^TiC~Y4aIu5Y( zQAuXI<&%W{uDet709u|kS}rX8D66 zgu4^&=3d0d=`0Po4lV_@#aN__GPJ10Ail*YyfHiSNSxC6ZPCnQMl_^yCE^g7pg}Y- z>&uW%3UuYsa;}iJ3;AOv)xH?tnD~AR_=jQ>B9UYcK~W_jz^Pueeh`G*X-CdM%*PnI zqki>YN0@T3NrhqBW=@EdnS$VsFiuubik{SX3r4ZKyrilz1~hHA2{Rz)I53JeE;;&7 z$cig&G%F@BvObn+jEPn4j*w+26QGIhNB{56v|(2+-wC@N+2{I4yAu5^U-VyfjSF?f zeoZG#A0pdV0Lak-9#sjEA$3=n%RT1V$%u7Wj{WpAwcR_x1Spj z*s9hhD#WPbOy18iREQ$w%V(I+<;5YHGDgfhG75qeN|inEJEnu3-D}y=B~yvRaPM+L zd9F)d$+`H65pTLer&}Zv80M^;-AiX@9ZH7S>6zOY07Vwf-8%;LW}04RD$Bv(C=*a% zH;Z*NdNF~A56F0PMRg#wx?uMc<0a+U432AJuaY+n1=#aXe<3t0V=WsH1Yn6%ANPV6 zN%b)0c{_L4R;S1P%62_(o^1XkdY)mOH#4L702Cc&CgB|Gmxva_$8ybm4NGB(o7De! zf7t>9+z|aWJP)CNvsgql(ii{y6aoi!+<9ex$Dm$#;UWw$aRa@e9QbB}4=QgPsUC3H z@dKg;4GP8@FoO_aKun5_KuR1>7zN3fZ{)OEv%RFk(~I=z5p%wh|MiFx{YKd92O*}j z1EA>(A_2#vHVNxZ^5QmPfI@Dm8!@`NpagZCJz1t3w#`;ku5D`!QEXrTm6`rmBl^E( zCc`5Au+wjG=@KmEuO{lZxFq}i;adg?3-~X3b6~leu0j`u)*rN=<}%;5;8t=#nsly^ z24#RYjRB3@n0w&@DokLaFlrbI;3qKdZq^}56sy6K=4EC(&G%$<_5S*?Pw96Yim5mf zjFF*c?O}gAJK$s8vFd=*?mk%S+^3of-?}W!T=`HK)--y@=wK4jq5M(qAFTvnNlg2* znk^KeQ_jiEO+Bx2!pom&I9imAWQ^&Gy`B;#K3w`NfhaDe!i+d(n58F8f9De{WMx}Z zarfz#erXh}wUVi%7?r)aC&GLYanEv1TTk~S)4#+lU(}^OoK7Fn2M|Y@uin$7IBDmw zYu=SEpO=J_tv4y~5-*lObNB`~d)%RZu5tDJOWefEqDqkTdDJ@qb2{6ri_vZL-t3hn zin#Hn>wB8(EstRbwS3#eVFqWHs%+(t5O|ysjaR;9Stt=cx47+aa{Sd=rr^I}!1XdA z{W$`ri@-$%Bxy=Gx+O!=#OywCt`%Ll^55mo{8{S*BB5+0B|DXWT_*_W`0J<8a&n)1 z#Wqhnw+&9``KwR(-WingHF$jmwM(X6&b-!Tv)HM@SR-!G4`~ZP6o4Qa*$1KAG@>~AiPF{V3Rh^N`q3jAm{Cen-t?KnTg8#e}`uOTL%Ac zfgnIbL;v{C3`Hhk-8wx5r5aKcVDnuQ$iB_=gpDC8=qlbKm8*Jr>L@}sRU{we0fC!=QEySmzDroB$)Hki z${V60#gCGdcn8-5sbvhfgFs~tX)WDL4Ze@l!&dp#-yokxcL7DH7GM?pw^4S&rE>Nz z5G}PrQ6|hFmx2SIFpv<0#td|P1ZA1R4c{OmVos}Ep+3T}gq*LgpP+>yBQdVa1RH+T zdEgIMWVwu*z>SS>x$x9Fdekgt6jgNnp(iXIUO6!6kMe68iO3`CK=S$eQT--Llt^{O zfkJA%v$2AK`wGj+1psZTikS>^3zCgnB`I11CgZ%n;?5<5cR+Pz?-BQ@aGi)7Eu34# zvTq&s^yX~yYc#hfFMT_I%13{8VvP{SRE{hUQT(g2W#ep7XY_MNwSpmmjpLrKj!v~d zt|twBaqASjuQa-%wb(#y#AwnYb>$XZ-lfn%a)c$3HPMAv93VOXN5EK@D>CwPVDbge zG?j;_l;%TlmU%%n3s|_Z-q6-yuW;S!n_UZ=*nKB7AA_Yg{s8qWZE!llxlduL=0hDiT$)bMTF zN(a?l6KtH?J!{9cV0+X0?q8BcYzR~`o_H*Cl{toB;`IdNJpO@J?}MIrIoi{BSr(bo z6z>&_Cmf&91+A3eT1ZlJWRhrR&uCOaYdD`tk#H_-BjVyUcH73CAr;fj!7TP1;S3A- zedF^%5+HmW<(_*=H=4~kp>4~{NbV=LZCq0R1BzkKe+%`$)%ZQJ1T|Y)1*iS0W|sOus8hThsh{fB1m%LtuqRueHn%ouH`xSI%91s*UTZd3!WY37ukm4>#Lo; zEppFP5^Nml0MQ)Jnmz2}kMS--iu7258JLH*7!kk(0l}`cnJ1Qr|mkR;;DVD>yY`JXI#3>6Y@^q0>B z2tEJ+az4j}8f`0c4XjdbiR@C~DD#ASS;`s7K{|2BkgD_EEX~r8W^Jj#U^-7~GAS%L z4D`!ZYL^y_*S_qa8EFt1ZDQHE{gt~aU5 zRUho&31w_+#bp!FCM}{%&Y!US@i)jW3>gKOB)^!RvuNBcs~_;krw520YaY&L=jZYT zlwIz@uQ9~N^pCkAju!qha2nm`r2#vkIHegk_XV}`33$Y^pb2sH& z6yEcWL~? z?{AX)E}B{J{~|yWZct%TkXd+;TPhnMZT|=F7`}*0dG$>RZ3p=uj(ctm2XxEzFflI3 z|FA(holyNj{wI|JVU#%Re?~9OJ>lc(-+QrlWGRg19N+Nw0S>qn{uN@_6u=}gK0pwC z1%n@Kl;NDpe~+guW|l4j#Y$!&v<@^~5JHF;nQa;I+X}|OQ_BM zk9dJ>!v{0i*;i{ne({{Axdr}q>-Z@>i z(JgO;UF|bPgk9Y;JC8@7I3pyL#E!v~o2GZn(39r7-w;yTVvNRn>X1A@<2`X`M&mtm zh*9G`eyB#{J$=Yd<2`xkTH`%?h)Cl-ZYW3NJ#9$e`(B;sV@e#5LIsGzT!?H!RfRlf z>mn(}9TpVj%pWwgY$VAGajOs$ZFo^OqNY7}qxOMCYJ5mCsX{$mIw2_k9rU0jvz2&Z ztG-V!<%OQU=gC+3TRRERI=s{TYbMkjrq%hOw7{rqy^O3tte`mNM!#T$yQx|#%)2@2 zcZ!!WgQ8!Xwp3U+K1-k9gQdI<0&Va*@mweUSZbb2MU|E?*mb#v%(r1`!RR0V#;<6a zy6W&q1myCq{m}+oJ^}~Mlo%wi&JLA=x{)I_bh-wGgka1na|KTTQ*N&2c{sJ$ujreV z=qvqwXEPD)R4ZXC+J>Wz)o$tkhpTstudG|Phm&+{+qR8PI<{@w_KwYtZQHhOJL%X? zhi{&9&i|f!-*2_nZ|(JARqdK%)*Mv>4kyV9i~Uk9UnL+d)~}4#DL^0Wtob89GRX}5obiZGYrz)|XYW#5ZLJMM*Y!+lPgl39udZ_t!e`*Z zDfZI{Q?FczD0rnBN!KaR#6=vmv@ytw{q%gPl(|>CStd@ZcQhcLaiZ*qRw2B7u~qm} zxLsoq&$6i(Jt^w(;r zbdFk(riYW5!j); z^gw?$^unxta60Tmz}-(8TIH19++S-8ej3y|mLN`>`P8M&Lt7wZC6#p5Se8z)=oNDc zu}p!V3eamu5y+J!Qyiw3;jSz?XiGo9&FJ|`9RvDkNU@1F4L!Omsl3bOC%9B=6{3_cq;TxR1sbC)FRuvw%anT z#dB@j8Q|tdp=u1-4y1b3P9*b}#~P3qkbq-Zlni&#jI!)($&GM@wohZ7NFx0?6BT1?|DUE8h6Dj#RT;*U-4-u9L{&e45WPx)e=6W}y> z2fMCBhPnFNMP-e7jFlo+;KGTkghf;#k-#3g<*?Zw-(7VCzsigm zlRjn!wGu^!j?$zSiG%6n_7J#ej>g?2RNTkAt*cv^^hf2VwAi{~SzeB|UQSh(kQXN5 zbAhdl-2{Zj56wyQCS9N4f=@6)1$IO#>CELXc7Uyp|pfi9mb@6JSsG+0(?dRacgF_e-)L}^_%k0#WT zs3c<`AW_bQHA7H2v_>SEsw;#YIjzdm5{S2@cFbVS_>AL#f#8+y=$qv#>IE`g12pOg z;H&SpXyG_n?*c4&LY}b-V7bv$;=N(DA?2vuTfDB({m-!GHuRa zce;q4&D&p_Rd_$zZ7Ia{dtOkUB)p-N;lrXPAWrMMukTe$K)|9bw34}5OE&O?Cu_{I z1$l{SG+ORNugWq%{;h1z;*O_WTEzY>@OEN&qd3SUvOtSJlxK2m;TZv~#9DC42{ws| zku}N_Z_1icETBrld~ShF+=!?z)klM}HP70hn1R*OzX&|SGyg7S6Qh~~y(POKHr?-? zVN2nu@O9aUuugrBz$MwRZ7S%)YZs33c zwCYo{U<2HAb4i1me#g=6Q1f}#vSUrK(22sR4dDDDcprfdGr!FyZ?YdI)e}thQscZo zCp#i%KM^QmkRkxmCL5&+#vBO)&{&9t86Jy{wC+uz$uVn4Z~no)j?c6XLZJy_9s;xO z7tKx?vmr>a4m1m{GplPiVpEg4K_=U%<<~p|b_vJck}CD1Vfc6mvje%~Aa5<_O#u^0{@ zsQh^m0LYj^=P-f5wj4H*ZG`Z9%7t0yIn^P@ruQdqRHRs!$jx#T@oEWo70*0O;h9qu zz;XQL6`OX*>&Z}WmdBO`IN^+zr(ysE1c;eqw@fQBikNsyn_-NN?CrxsAGcNBH;`7a z8gfks+|{O9jgCD^*!zqJpCfsR;G-S4ZiYXx&eFH$cFj!RGBK?)Q>?QHcrAlnez0Tf z0_EfC?FoCqo6mIQ`uP1c&)-uZ64bg9_5wC%XwTgWb)las-FJ%sL^t<~P9P+eloAIf z+2vfM86RnJ)($I)8F%$(u<1&{TWLe+?oGXxu6h=6Saq%mFROVr{3LDGJ)FXplT}!( zzB7$hs~Gi+;5ZN5FYTJoVB4NfJZS7*4|H*r0iOqEidFMCsMEv!vg*{hMtjq}oN5=l z=$OxFB6{i?&Tz>AtiEd6EptSs)jXPZiCw%cZC5_eL2Y;z`HbA&<57s557$4JTtWw>kBOLnPl!dxYbXEop{7icjdBW0)KY0;EIZ5b8!QF^HUnc7P+ zX9-kZufen;(=2(#ix)lxvsk=YiE5r+kIR+S(?!|E5!6w%M2_IwQHoVUaaUN^K z_9XP8FxO{D=7ZU?_k;^5_=I+ttG4o7c%zIN7gs|oF-zhASi6GS4`=0ij8ZL*%TCE_ zd`kc15hq_a8=C79Y5-8|^i*^dk^o#ZNa%cbnnsspwZ(ZzW79>d?{>l8my^xEUDYGh zBD&530S7kf+tX_TLzzS4BA$CavtebF17zsRMq>HvGTSW|-|G#un(KA6s$gtvQk%iF zl{;BS`g(cmDmr~i4fr_OL{|smHag||&P-?e@k&X$(5@^)B;Y6d2kkU8tO|q}mvn~* z0||qM_HUa*v>AI)aLrlET`u0429F=tyP}ZS*y_D9{@Ku11RV-LgfQ7>qo36?nP(%P zH6E}yBZ1KgYF}o1CA>*zzaH|i7@24p@1wLbd)qz`DgOLUYx$r^g;#bG0!evhSS)!j z4Vsb`J#yUg=>jP852!5h4+u-So`Gq_Fz}5uCahR5=s9+6pg78KYn;j-*cmh5&rg&U z3s?Cl3W!edE?ArbE<7tBd`uxMzShjHN~|Y3-GMmjGLm`(4(4>S>hq_zmp#4p5|Df_ zJSFU^(ms`diIMj(WEv1b-g}Acqf{G8f6+yY^4^kx+yb%}GD_Rz&Vh6ZJyK=;ujNCp z=fVi=Gga{{X9RF`K^HGb_rMn`e`G;llQ~b(1_;Dgz@X6IxvYia5%wPZ-{v4OF+V80{`b$c$gx`rzY7D0 zpegncaJ&E|8@YdkB(Ij6HYl0NQA%*&MJ|U$O$e!XC;U?5U|G}rF^vRUbaUm~i!Q}q zt{843_1NO$&HE8<>|Q9|NwYT$qClji6P&L3j=7F`kK0;ye0{mV<%b1ve~gaULxF)E zIG`#-kVHWlI2&9+v0#a4RQ|G-Otpx2)a#21fB?WU4Y2B5d2AYuc9&gYzJ*-b>*79ZcVH(f4t-xjwtqrY(G!(6PuQ)afJ8wJEenAOo@Zel&aP?`iIPpBaM-6?15e?KGPgkCiW>Vt#!yeipWmt8KZny9YU< zn*o>K-Tl!We=6UTiNyKL6%y>by2j#uvP=qY)XQ8U*Y+IYL~o%Lpv^*)`jqO&RR`nQ@5m+UW2QqruBM0rs0<%W>4%A|~=jNZURmeq#nP9Qur`%Kx8_sXee zuz7~EA|jx+2Qd+(^eiV|WpIJ6zlOPH2LU~-5(+ZfP1a6k6FcxR6NSyqxbD%akv!x= zrcNb_ez0gBKLEXgfo}E9a>J@lpNt-!h5H;xHg=Ed~D9?v9sOa!!g!@?>B>}P` z+aw9eijE$tR7FYf`D2(wD?4Ks17pvMWr6O47r8Zwqmi2+T#G^vfcy{x*n}(rf;Fo+ zdpsuNQyl*gT%9LMRr4wXSi93@nayCC_==!x@^Q3MaiwhHQYBk(NLLN)crYF&=dfvp zvVFL-WD+~{JQV3)3XJ8u#Q|E-!*Zj{<1fDm$+OkeFisjH1mpY79ssb$DBg#?8nEuA zI=vQdqOf3+Z~yu?{5N+0XT|?s@c)VGMcrhHp6^OnZAvXNH4oq)3AGlT4tdpIKL;65 z<3gJeTkdiZ*vP?|EEIh-EJ%6bwva@VH8RtF-_)=&5zP$}?8Y*0uc;EyvS&?gZ+U%7 zsAadY@B@6_fc0^3RFv&G2D>b@2R^7J)uC<)2ZaO=+vdl+2(4GR35MFjSn^``2eFD` zH6a4ylId+5B)(O*!<`ux7R9Z!9_fiz$CM;eFDdDEp5zdUJSgBnm#;U03gVY8{;&OtBy`hR~tuyAM!mC8uUrPcvV zIG`@qUn*E`*3SC(BJIPkH6a&*0s4ny($8gThqV-Juwcr zx&ZwO!Z8IC3DcNF`#UR_P6G*}j;r`nNd8YXK+K}&7VYqkEMVpfv4}cM1KI z9(Ez2SR)e>ifIC|Wef<5*^(8wfK*t-Us$wYkgO){RDeR}d37rR!<#`adP8Z)>q=_V zA<`?a2w2#mf_bK>a-JG6Ss~fwesvk5fk~(?xYYHnCvv=b?{l2#^%nXCV2300M=)t5 zrV|?Q{RulBW)z0NO#pub*vkba3A$t!#5}T63P2N#g&oo|Bi$q_KyL=62r7(X)W%XU zCd2)GqlQHj2fxRdBlcoXLGVdlfL|TdGocn=_X$8x+)n4{!8VbSs2JX^JDAD@eMn8A!jip;d){I__kMj;L35+V6mBKO&q->Mr@)g$yOZ5a*-xTz-T5UYFuHtd)j%fMOiZ4ho! zyIM~|NsF@!ouGIx;)fbf%UkMrt%HRP`5qHg>;m23>~rY2X}tahMp>fBWMGVy)mg0@ zAX9_=cYgo}z;S+jo$8W0Yh^5=O=4qIPW7a!RlFYsx>>&>fMzalSM)-`9< zfUa$M0Xn;{0rvbJ*#>hx*V1`4aLwSXUc)7ywnR7+ptxW?b0(_UfTgKO-4jKkAr@I` zN^13*S4@)CH5NvaANlf8%a*^(g?lh?53So937a7)Bt89LViCkJ_n1a=iHAOvbqAu8 zjpDZ2%-uhs+c)7JoY6&XpxkHWo;9xBhvTl=m*F01u=MUC#+SFtxvhEwPf;O?Gu=vp z#D*~m=x~?odwcOC@+|&aFPPI3oRRVBVt zVDsxE&dP(BI{8FfWL~}HCTXUp1lNH}>Ca*ez~3ArCHQzd$~pCCM!{>qOh($~t(k$i zi)Yh7;)WkpaeuA1xHEQ=8d4dHeK@~_o9i8Mp4`@rk*HXZ(*YDv-3$&87Ic+2yh_^7`#64u}K*80pDOT!i5c(k#R)&KV8jyZ> zK+`HP2nr=mLE7OOd!Eci6HHEuQAUqb@XaqbmhniRM2xrA&1VJk4m_Swl9tPdK!hRk zRv)kyhM9()Gz@;9?gi}DaskIj>!6igqPxw*h2Nw1xDR~CEDK71{%WQZ99>7*<&cKF zY>%%x@2ZT}-3@{k)=>q$4orkuaA8=m0j6@@sKS0K&yghEcg#O;&^c?M{kl#=@HAn1 z@Fx@ekM)Brcj7%*8?=3%pqlt(X=qmF&I-xd%iyHrC{R}k_9L52aEFcHA>HsKHZiE% zYx&a?pLXg|WY+2p>P2Iymw>D$7U_oKzH@m7Og4wM`;FvhgO(wM0$Z>>0O0gk#pIbE4Ya_z@-iHtSra>cx(>_bxNwVfT9_# zz$4x|Pk4J*3|T?+k5K|=PF{%zG6o|qKnUWaHExea*h@>6*d4cuuIRS$&kbvnw4o4|Ew}IMeX+z~ zYX)^SzAbXcp&lW3_z|t2Y&2PTleNJY0s0q;;x z`%1=CXC$e;TQ5=2#u19Fbfw|(k6^-kGOcOiXT#(VC=$GB1-^M?fYRhH9bHGH_4|is zXhpG){rkozXo6lYNR=Ay;J6?ySt824WhL`Cd#!^n!q5-yE}hz*85+>!T*d%We$CPD zTpvyQ%>~}BDsQJ;`r0ElXZ95^WGxzOYS69>9BR;>^N7EPcNzgW)Vv|rFtxnDi=eD= zGeWngl*f8G8efx703R`$FQn|+?;&}hGtn;){WC1A*s z*Gv@nVpJARkc*MmNXan^X~XuNEpWo1A!~;5Vq?{LLr;WvImszunrwrwfn$COnmN%! z45sFf`Z0be4&tEX1DL+PHy{v@|6P9njloc$f&QUQ{IfC(S13vGe{U^fQwX>@IRJJG zjHrCJQVq$qt^`rnHnrAP3w6pHB#=-p8LZ0PpzB`g=}StMYB5U9Uw)tfFpD39KlalI zf?Fjjng}s%&_;efZyp?YWfAay3qjWgdjg|hkq>+yk<}^=BICkThzKszejnhfmog{5 z`kr(y_1|=@4yLovas=OzavTl9a09v%c^|L?pZ#wAi0rtD9nWCTNveP;~_37)T^sf^lY=eAJ9W+%b?hmXNs7#e(t#%;EaLPAe+K85t#z|%G?EoQ%F2g5v zNO>5PSc971Jr+#94$qvsUB)G&0aE8`)XTs-C}Y2lzHXX5{yh6o4^+bnqCt7QBxb}| zYL%|7_WIzkSFcj;*GF4euiwZUs&}q7MT2PAQZ8+>G4O2*xU9Evo>b>`2;Cj+VZKGo(2 z+(YVrnM7>mX%#l{snIvr-LzkW-glkeg(Uq*j%UN zp8qAC`adD{exYMatR!ToxB9U(JDD5fg?UrE)Y zi8kW%@;-VWjZfGTk@H-66jyob^LXp7GBig%^Lm^IGcvhVV7mlFecfLqPs%IR-A{I@ zlW{p)_j`wPx!Tc_@puC!7#*0jAP!n(oo0<-EOucLICl5)I&^*_XPc(Baf*xqVjFW| zj4=5k`7RL$oaNfEymk(=BV^7DB*r~I-?jF#lgnLBIX`k#IEFz3aMnN6>N)-51U@0Y~BDlmzy zLJC2U>r5$I>|v9&ctwey!F&iLl6o1ds2oAHh9Jid3Q5$W_OTJAz<A}%}c%hMqG(`|jUq7mmy!P#~X;+3Cd zDXY-PxcnXT0X3q$6X#8CSGO`_&?jn00?UQw^8@^!H2UB0{r_)e`8SY@gTcOw*Dydp z5-D?<*jOnDwcJ!GaYooM097e_R25V{J6$=BH0RH>APGniaAuY?W5mB9cFe$75+J{G zekpLXWJwPc@;E<(2K2!3=(ObxQWwTC<6sB`6DpdVf>%^JifL40rPcxPt!?Yh@ zC$AMmpTZ{Tm^DGsrL9cP8P{0h9G&QbQ!V zXN_CM6!5@a>Vzb*0$f(Iq}SVx4#&vT*s_dEPBR~Rl#)-+Ft&!#i09M$qEr}lz>`hm zp0SGe@bw|ZQu-s|5PK7o$Zkw@Ldu>|heKRC24IOsB9B)> z6`ETr>Crvbv;TNWYD^UAq`Yt=bdK5?Gp*Vi7zY`;NVj{683@0aRWVJa8i`8ORG3V= zSRLr1^SzowhoBjUnJSZL5j2pGRs-6wKM;cAWXxJnxQekz)+~^ck?4sf=2oqQ}M^(lV)D-@>2hfA9j8b%674?I^dSS#An~{g1 z4OK6h=;SttIgQ#%IhV;;uc1t3;ZnQ_vo@FOo{?DHMN}^*BR(VUzW|` zgg0SIFb#f;`hUdb0adns50-*;BLoD?-y_IZ0~8-#SPQW#^U8?9k!|CqziyD6?|(ok zO08whXHtdt)jn`(A{)HIUM2?r#3f~-1^n7JXzP6B{xgxOlWfWWVOC;F5HE5olo)P+ zZ8;!QH~dW57ONP-PDSh!Sl_Hq`ce=ZDlK&^W0y-$zg1MoD=;K4M2y@SI^IE<-JjOR8f51pmMb_TlI*OL$xi3`OWPQj=Yi=!I7J8CycTa zM7a}%*9q0(jymE28^{Zv#bFfGqW|6ZlKGI}jq~$p;}&Wsn8ur<`j+qI_w|sBFCVDRs3^VzAEJ3JpHJ5#Ki}LpG58R*y)27Q{J}O5zDwiMjoS*${ z@Z)-f{AU;^!JT7$Uu=)SNl+`Z`6FR^r!;8iR#0yau{h+4f-tV&8*Vpgbin(sezng> z!Q)ZEF*z@o((b-Z5MXEb>%?4O7dlHnUm=Lv819zjVe-X~f06M1cfS5_h5jFi(vl&{ z0|B3GCnIq$?-gf6Sk{ z)Q`abzH4nny$bSglOd34>OXfq+^7pdkp6kYq#SF$8Wa!^8~V4tbBaR`Hbx4=UvMPA zn-}UK7GJ~K)P*g12F#iOWHKRdsx)<;rcII`)Lt_YC^Rq_9rs!msT6yMgR7c&Mb+}M zZl#K5BO0X)SP}J-A8Bs$dPh~)yU^vrQzuJ9L&gPpA%7yTcgx@I`|RhtthA4Z9RhFA zUHW&DC|Pf^Ab4-K$)hpEZ+#x)R~IyZT&O~{neiQ*hOesV%=k$LkvGiD*u%!Ww^;hM zq*2a$HtUSky^p{5uXFo9bq zI=79c4;4uUTLbhmJSmHg1xrfC+4%AAvg*LAIk@RhI%vVr-vt&VDJ|~q$MYQ>k+Gro zBqYU4E`$jZY2lYjTuxrTgf7$#(-;!2yIWCWBNu}&#f_wbT{c@$APfYcNB&C;a0dR{!ikZe z$y!qgg#0$IZV_ofa(q+g;duD>JxsnL{8&v$-x&*ev^(>NI>*dNdOGc{jk+uo<*G~M z?=x*%na3Ss@(B?~K?;A>*_Cc#|ECedI<4g}ze^2}p}KZ!N*Q-NS-JJWw92BFwHWKx zA{#F)0ecRbQocl%{e!nCfCNW4@WiB95$F1)O%(QoE@PQn&8b<^oxH1Ug3x#Hi#CHzgn zO+wmCHeV=5RA4PlBgxLCL}}A$s_WuhExy7pRobI7U9Z&JinD`|fWh$@=f)H%WfN4e zijB0AA^Z@O;)-uEZ4`0|m&2TUUY*)jS02LAV zF0io;3k@u(n7TF`K+ejI$Y#kNv~Sry9^AE|s1SjxxjPUa_SFyut2J7u&HRI5N60TG zca?N2cew!<;Pj+ZTvv-${{w1UOLqbS^B2Ssws&X>)^}`zvKRFMyBjUEo`oB)_g@2; zvZXHt(w~Jy(NQ6CH)e1ezb5w?DwD~;M21SRKO=gmPF@Lq04nyci*}P1qiVvT!SZrV z<~Hb`W1FLfw%fbz>?epJyQas^B>Zz3RaMbu{O7H3_s^4kpT?N5M*b8lSb{h%3C|;s zS#Q=Nzrk8%Ugvm9>UewCDsu#Q$lX{9!^_YdZ7z$~`95LYT~()x7wZ_SdQVBvq}{Va zz%*+#sg4zC0KOI2RPcGoS#c)F!UQ&qItaG7T1l39Mk{nRimpnduEG`iVhWP$oY~C^ zMShD;c%Y3J_gX9Dm6IAqrjiXjCbg{3s~cl{_1vVH1!=#NBugB7rTE^`pX*>%71n9j z&{VJI^mz}EPFl}1qhv$g(xpzA+59>*Q*%1ZII%l82DniGgcoxaP~o9}j-|wgOlpi}xwWX9pUZoJJxsN?wHbuq&L3!;;q11_loFqJH z24BMY{YiWurB)0%&(e#ONl~hF*+UuZ5^chSx-ev5FGZ8}DI%|JL(~G=0&NF5*Ju3jO~naJ=lF)}X^O{gGC6pU;TU8>Qpg9S z0110VW0X<}dthVsfzFdhr&g9X$f|Pau{YtQ)|pUoP6}Nv@G|Xyn1b?hF4l7=b&W}_ z1{-&T4NRw1xv(y0w_eu)Cfvu(uyvE_)k4;dtWv3RV7+076?+CR5A@lZg1rc{(sf;4 z{U1D`Kj?2hJ!rsM#qc;yN?t~rEN+d^08D`^b9dC1T427|a;Gd$C&<`EcAa4kT`G5{ zv^^n4PxW78Qk;~5-~{x7Eh}f|C@Hl|Yg4(Ewq4<>PgaFF12za}EK)x+99kDos-iB< z9OT>z76;Bsz_c8l zdBT79W|(V*y?H~~7Q%Au@5Oku0uVab)(A_t+Lf@vxhiP{c|2c`f4H`jI>6C;`|XZ_ zgV zQ|4ECzz;a71DyQ=Ry8UwMYZat=lz{3tM4l>VaTqU+^^WyYIYT$H6_oO0g#7!>Sc!g z*C~3C*}HOgn+LVpn%DCZ4wa$WXM$$adq1%rPGnXI_p8FHBXu#Zh&q}aYyR>xheqGg zmhSZ8iSX@+Pxqc%@966?+-HA+RNRs;@6(;#Z7BbtS=@5tjV1m==2^9>(p}5!Dal(N za0i57%y;L-)xZxoI$Y8d3veYs@20J(?@?9sVlzRv>@+@%bMt?`t;|$~(%yMp74*-d zlL$Ri3DQ)xDTl!;Y(?edsi9QWHyXixXpsL%*{ktH@UL)njS$te&iQd;!S}xIl;HL| zpX>v*M`ch1%#FD-qg?3UT#f$?!Tui*M1chQ2c4l)3X64pA5ZgqpJtMz7+g^kq{w(u zLjh#uaF`H%$Z^n%X*1k;D*RPD1St*u z^!qF=H{w7PKsu8=uN%4UO5?b@=H~|hul7p@F>edIN47x%ZwIF$+-4iNi&uwUVYum8jg!p{ zE=}fX*KG`buNR>;<$o+mHr2PufT}5RbE(8lO1m^U3^TarhpS&ZD&4b!h#fGR%2hf+>Iwiy z-*PdDZRjsF)ZzK%F-Y|)?!`N<;|oEmMzzp%WY^-uuM zpF$Mn>m}nS&S(3C=*85PhutKFJTx86;B(|e+C>-Q&z@}|m>t0H+9$&#r_PZuucuhN z-p4K~VGe9>P04c^KPb5!A6ZGgb!*7}u|Hq&G%WTB@=qP-e|z@-IsM;0FF?x>^!xU? z8b}I(1QT8gVJS6qi?cU%{(rFhW)(5if6QMsoKwJ1{{%|W>nz`XibFLMH=rsEOnKJU z;2Z%*CJF<)fG7@e!>}Hdp*7tWg1dDx^^_i*pjYiyQznWj1mY$y%1l+LjfiON{yvLq zhs(=!`a3G*fO&@pux)cSM8NJ;CM+52O?L#98p3-9C3Ob|YJO z?jzKH_!zu%Xj0Tf$#2H~0f^OQ-`u9=)wT@Dd*D+jjy-@0F`ayC6rOuhVw$!qhWom* zBASLGAV;vab03Y-bIk95^_5W`-tt(c8ND)uXcn@dM9Nsql+19~;N2O?x^O6Gp42Hr z@WsbG%ns>b&>`4Vx|i-ngsOnzbGMGv>Qfo?Fa26wQ9zGnFA@*I0SucT>~+g{`S}?r zoH6FGzjN4drad7AwtUZY`8ygAQY~j`ivTZ@$vmkyYG$sZ{kAEiI`j9?DBHMM3DNnf zJtdc)L{=|@B%863LBhTXOVR}E!TcejDC{IC6R%A6Y@(4)WgTDDZk?tHUxZgWK8yST zO61P3guho7u+*J*16cAMy0*`}oUzwEu9dE3^{4N1gzrJ@iqKf^LQXd~2)>}MIzy=! znUvqFr<++XP0%B3OPBgv6$$QX7V*305UQb7cDWDt)Z{n#5Vf&FjeG*v3quCrUmbn_ z^740RApRe6!$1CK-b`UyOn>0)7_SZ@?6A?jWz#2da!=~2; z*OCSKfg%HAgcuW&jFC=DBWnKwYf-J(2dP@Ow6U&OW*b~TE7`i#?O65sSg+B0_SIQj zzx$Z+ndnS|1%3Hwz3zUyy8HV2hv<^<=kd$?&V;!95aH#N;$gu%U4lPhedb=_y&oXg zQ>b@{l*M~Wevm^>@UAC6UUYeGq(#O%nKX5YOm7?hTSPv@89!#pzAcT9IJCMZawOCvrqw%UEbX} z?SDZSR^BW)^ma?SQiE{Z!@YIS4o#&0YHeU^Wvnz;=?)EUq-k%c*=>iNNf!4Pf4VcH zg_m#>cLmPzuCWz|vk;FNv&Kj4ZIr7s*H>_?PyN|Kk*VU1i!k(VHDMoaPk-&WWv^yK zk8Or~x{M%i7FR2Vs`}80-;M!@6O&(?7I4gOWrdL^sJ8@GtI`+%H28!_9-3N8VrbW;iPoVcD) zL#mK&xgAi{o7?qpA|j?<`&3$!dL~t>6Q!s=?$Og7Gy{j{Vc3h>^>qY%>A>zG`9Q>E z8ukq2K^Z0vPSQ#i>bbz0^@aUonzW^j(?D7Us+d><+!nTRBS8g#U2}vvwE+i+97U&| zC7d25mq5i!S%Ee?f_y(-w}2n}kM`icmuG=lPrQZ|O$eB}jS_&F97n&`Jo<5nFl`0& z+N8Cq$jpQe2SQ2KDPE^{PGF5cEIiWb`JNfx%%oMMkbg=#TfI{ut(^gV$6S*D-mYTu zy6?SQ5jR#B$_x##hNpujC2SKVeOcgJyHm_FJa8M z&{#(i${#yKo~GsSGPdvw<7nW;&$=nuWcY??bwH zcn^=R&c-w6Als)q@XpyO?v`88ERNe?ZoA?QXWJ4l<6&motq<1AhL*}u@TCz**eR9O&*wwcw^)f~t>N(Iy3 zGa?SIl0E`&uL>`7QzhrS^buOf2d){E4N>Z zh7L(j?s=q4m$(pOW*nCM*T$cc)i!Vt8c8K>|m5Dt+rC= z!@}eLk@@8+1$uF6Od; z@+P{0Y2@l|Ld45Q)SDTTonOr6+OdQIwG7##ewm2H9(xNl8GBtg(pYjIYbj?O>GsY8 z1w8p;j4J{4Z;W}OBy%@qyP{>CoZjoLkE}Qm;vEbDJi56rh7to9{p5xc=ly;&4w`K? z1$p*Y0xI)Ez&s$C#xQxW8VZ&_w`mh-pk{um(DD!hZ*MBmI!F8xd0LcNm5m|**j>77 zc}^+1C_2gQUa=7`ck+$54p4VEwKCS>cwZ)T3Ui^WzmrOF8zsw}dG?vP2t*H5bSj!N zV!60BUegt8n4wF#8oU1~-VVDEad2Y|*XX#k!qo7vL?nPV9IMi=p10N%V6muq_YNbM z3JO^_j!@!ff*VEneG(xcZtJ!Pz=)Rv;qt~B)Pn1_xFacuK-#BJUh%mLB=IIs7g5V! zRN0bwI|a)A8U8WZw^E?uE6?<$F(0DZres`+ zg&SrN2pLtW?u7*Z0jD1g?nK{9+fRH9Jh{*E17#nLJT=gH6!f0ASNymNdJ@-aPe8vX zppXXCLr4`2h{C;^=Ow5>Pm`%ngDJeTIlyA_SpMfYxV|n(r<4UZU0{@yp99#qsAW-7 zBiQ2uyRAzuftUx+UsC%9KqRZ^UIEUqVeW2hg93s=w*rKBUmBSM{mU+UL?1Szx)fO> zJZyTOJ{@Ld35;J)8oOWL>jH~?25hIkeZkZ&)dC^=DtLU3MV9XXw~%br4-I{|8!)cM zoRFa~deFP6+CbNno+MLwh{}=#)j5&gZngOymD;!N)}zBG;4sVlgi1KiK&Vh1liK}p-*^O+a z!F))Uz^nN7;lBdA0JFI%XS-x8{cgIT*I0qioe|Z!8+$yl`(DN55Q*f^G3#<_2@|t_ zlIz%RBk@;fsD)7aPEdzZ=p+fX66KpIRo3!@8bma95y3JTB@=vryvSNFLdn(kf^^~J z+~KB=ni}C~8PtA06Lvuid!p{lh$qBZ1(%IgIcHcLOw^UG0$N-0->Z?ReTKWe-z{SX z1RMsl@wO(pid;MD#RwSH6Sm+yOzN@8ZvH@%RoA>{6H)D!ikjac%9K+gyi~^z<<@nT z{^|l*K%VtrJS3%+4>(5kWH{LYHhcI95r>T<9t|aI9z(HWx+0M~t4;g01-$EZeR_*V zX)g39xx{}41Yov5IqZgTgBLve46>uJ+d%+-54_9cOa9UfLSVMrF);j`8g7S03SC-+ zHtVQAMrASztf*n0oEZ}8fw=ua)a#QLRd9^C$pvHIP*Et!|NsDqvhV@&v zztC+xj)0A*hFMw3y+p%35;QZfdZrS@WsP!?e2w*H3?QF+;*dx%WblHSpFpvPoSjeZ zxen}}1m~_ozMF;RCk6E4!@e7v=C?v!ninxSVIDLdV*c|F!Wz0|089LOEsW#Fl~eCK zZ_*rNbsoy<8&VJu!@i}@n7>z5w3iL}6^uX6%&-pJrdt+%iw}}UYXdEZknWGhFK?SVJr5yLTaxQC-GGnsNz6a{PLeG6wzsvc6 zlmJFacepIaT$b6v!6E6)S|YB~1<3-8syavO;hL_?M((LQdGCC@lLWnAa-)r(QK(mP z(4uIsr!hfBd2r@=P@~Jpi z`&jLR{ID)4a~%zJHm+6yF=#d@{;&DA)NxMXhStP0cjl@i<{L;8tc>`;#x?78Ssz^Z z9N=hYYL^f%{kze4Ne41>W&dI2+0_4!t8)y_q>Z|DY}+;_wx48T+qUiGiEZ1qlZhtA z#I~)8tuycU{Wzyi)vB)QuKsm*b@#4&uYE0fqkm{CFRI!-7h#jPwRsvh3N>P8^av+& zFei4oVmiKRWp;`pF00kK-tj?sXjwA%UPYkZvE1+Q!>x3GHfaP1jsu|^VDMqOIWT%zd2KK_#7>;^L>(M7>x>dEOq@ zA=GZ%^OvSYSkp@MKg#8FGP#azyzm6)hmv(KU=b?FPIx5K%2`trr(pW>Q{6#rPUOI0 zPfEREIaj{=Aa0o%^&Dl1tj7W`5ba6!2@O3emi5IN{^*s&^z5rp(;AMRnH+uVvzUpj zXP$e*CAztZstNHE*oCkPoN7GOJ=M7_@m&~@Uzka@1tjfCFf(x55ZS*_zG}7ttCq#p zY>KPgLW0;}SH8vmpc{;Y$s`MJ`f>q@{=(8x@(BdSsylhpt$qOxY>R8CCRx3wDA$z$V@XI{e{3 zs^r<4OtL-ka6IxyoADD=^7q_<_vu5SY^Nk`BUJy1DtonsB?9+X{?AJpsz$dcrAjN? zxI>4xgb+mYI8$w%zWy=elX1VWr$GqhfaRqK=(T9EqnnZcamKh=1>ULi`il?q@&%>pq&!D zy!DXaRR}Saa~3waC0!oJ7FJQ-IVo+6!9jnbv{~aSnb^!)TIrmuDYdBjOT6a8x~fyO zX49ssQ&?4+lY`!DpO0iL7v=Cr#`RW9#I*>HPs;j>{(Ap2aL>Ny-@XC3YeYOBL}x$N zDU?nDKEe0`!)R|;)6-UqwNyo?nzomBB!O2cZK5b~zDj<8N`8V$K3%!s9)ceh_haMG zOxmf4V5dj z&!f@tM+EtP;9fT>FEygZ=SXBe3AT`w{TBqm!UB@yPtFoDd}UO>S(V zX9JS>KO5Rr`S&!_=kx`_TYd!-b5$v1 zyd-hKVe^?x;={E)({lPgWz4tRm3IZ~2l zB)OtsEcay+xB^T+A$%d@_9DR6$rD-d0Si5`5|NAq%T59hAn`aA97&Ip^M4ZtwIrCR zSK0u!U>N@*bOz@sYxcd8yY{nG`9_|%_lO~G$j>Ih%7j~G!gTv)sOJ=nXq4c`RCfHg z_J0Qx3nUQWD?f=Mmno`7pRNySZHs#NZ;ClmIS{8 z70)+RXLgEHKd9`Et`{gjz@=~FPf5PTB70gUMdYp1>QL1=NuBq zK9@zR*UFY`7p&W?ID(fyqtu&$a@A`h@cQQ;PEm{0Q=fSjNxb?3Kif=N_e0CMsVi%E z#AEMtqe)&UwW0N=R(DaYwJz%z+$Yi>!^y2f?r?uB6>WPUsnGjMBzN0#i^DJFwM#WB zxvg8pTK{3GYEboEEHmnyHG+9`bQ}6DLGlKj)dwtz{J_rX`%MP-fTY|9H(9X(O&G#eG0?MK_Dv&?ahc<+r!-BGC4Si;A?@cUEA?^vK%_ zttnQvmG+>pL(U%Mrn}-{C|cuXdUC2HaMNdvi@xE2e&X5EaU zgiAAfg!N4@_?fH7sd0kTN%*91vrQM4=@TVTma1XwFwYqm%Pi{#oMpdxXpm*&3S<03 z1jPsQLR2X1c7!qauZ@DF^w_-UltVRe$ESxp;nfAB-h72=vVY~=I!-W6J_oiXnGLo? zSD;OaX}v&GAy@8fM)8MfS;Vz5>Ux$0|&FcTL>xB&} ztG6{|2qOQX<~_9tB{t#C72T_iq zD0Nmy-*|pF$y^K7+EMCXgzqqkfwAC&`okoORwgX+%yJIfIjYfLYoL9S7;y5lGv*A3 zqjGoD1PCFtty~WwB9=o8cqlVqsob%Y-FCNoc6$R6{Ek-SYARyB+wFC4Qy+hL}t%yj9P#JhrutC43(C zfmdc5W^Y+=-E7-hpd#T=qVSl64C(5q+KUQ!B9J2ER+;6hfQb&ae>+XcXd=JiNP%R| zS8qJuaaA`J;z(!A2JXYGjXcj|tiC50b0IL^;Ok{mk$_{6auT^ShzNB}hs8sBw)!u1 z`(hZZlkDt<-btmfi>cS_{qFd+x!&G-X-d7GIC@L2YT3_%=^t=5dh7}pZ3-HKYIOek zX22Q)4xk+e@!r}y< zg{{3qr4c9`uk#Ls;HNO$}t zfivAg$v128!@0VGTkZM8`IH6x2fJ2X9-!Z-tks{huIbRx;x_^Bg|7=S&aan-H;Bg) zpkt-KRr%G#%f-<30#6Le&;97p zIIup5e-X1tx4KL-_jkTRmi#ND@Tc06a=^rR?1k1L-6_bW8z3Aa zOicH;M*4!A$nkg1Ld5+NZ>0IiMUBP2zAf?bg@0ieV@h#J&(#$F)pF9-RTY(GWh0|D zXS{J)w}OI!b?D8n6oQ%FDixXMM+?3Pw1_&EWh!ig6BciUX){IHYm1YG$70sU|2Vbr zS-jTg904Yy66pe-ge1f8FF9I=ZGCC8@V`3GV)=VaDcD$PSP>33#} zI)6{-J<%W4lT1fd955ftlpQ!8A-ZfTctW#U7i)^m1vtu;UGO-{Rhk2{Vi(0=Mybv@ zR2BeoDGhdI+gR3ZXfHUAZLh2D#K0Z-=h{Wp*}*o3?P`puQ&Al`;a|uu4kt_Du1Qpd=qg%pS7iC zy;r;nayo&`0x4A{NkfqdM4mDvd{t3 zY>t7?Yqb9t{uf%0*V?GT?fuT(j@kXrz0k`G@#lxa$BgSew$L~IH$DfjJn-9a*XnAu ziE@3oz+I`PxJ>{r{<`iea#Y*K6P*4Qgn;eUbAIbo5Y2CJ>=1tyZ(sc5uJ+ z9-Em)w}r8ZTzLbpKTz)W2SGfz^Euyo*Foab%ekbDLW+p~{j%l#lh@SL+~2>eW$L~I z%Ph<~#few<#kZC&-QZ_9R|O_NTe^zNt9d%dIHMA{hTB>s_677{j)o30Sm}ApRc4N- z=IYivw{9)!zb(*>Yw3gAXcJXaB4e?1c%d4Yi z?LXa@BzVcE8ei;Fu4*)W7zD!uiL7fhW7FT_u+RpJG<{s96gJ8%SN=qxi0&1|9ZO!E zOKQaYJrWh@ic@;fR#61Q5RKopKDW$`{hI zE-ubtwF*o}wg7D}&f=J;Oil;hYtk%TG+G$fi7XQE?XJb)_tzf z(<3k zf#wKTj!B+SI==bF%DAX;(AjhC95?pHw(v1%$cu}@Tdwh#r#h#g;= zjk%u1UtHGFC}GE4ySyJeKGK8|U6{fCOF2JJ45W-at~L})&jw+k0N2}!#5~z3=!E)Z zi2(Sr-ymZkKVc1s*05n&d-izNiGJc6u+!31_iz?@;aatC>bj~P*qpsyUZ};ADzX^H z{Uw>?=glC8y=FN!1ZC3z)2t31r=Q{U0HEcz=@1#z(D|UiPQ41#r%*>2$je84@{M-< z1r2v`@ZA8DbFv<`pDLYA zmHx05Qg~-E^Rjt~Uc+6np_xRf5W^*x0Y?YWnoZ;Lj@O{0+sFC=(0;Hgr50)GpOJOm z_O)0!x=g5&#Us>+&b|YkH*>x`G)BVNGyolCo7l*Yf{ji%UUjj9M4a&rEHUeDua)Nw8p(r(7W;&?K;7R(iJk2+`>(yi_*z&wmd zhuHbJ@UUea1&gRl(Th_N0itID)u$l6skv!SgmyXoFS@^>o5!3(+Dj>+U^6$M;)U9L ze4Qo`>HQF}=+I18kseP&PF+~BEoN9>B+opD?`@;JrHom|6D7-@%qc7_rdq|dLy8Ya z#+=DqWZe-PLugZt|DH6(-{HIae|dic)Y14hmtMK%dYc4tUSjF8**s?;|e7qhDaaHB$pBfbQUK|s}Qwo9;z_at~qb8K5 zR#RLAB()N)34abaNKOjXM!_j=gYH4Pf1z@)ga-3va3lwDe9fvV-NqX&7`dI$)GLX? zO`INXEm9&r2ZWKz!7!a}N}E_wvU&)^x<7UOI+kQ&20`J;rgUmk)rp)JrB830>X7Kh zU0q>cn=6{wM6&LrN?KoNxhkb9<amvIG3^2IglpflS)^Xv!uwHm))yZu%=}L{)!iT*OZj#OY~RbO;^9e~lX))T`V> z@fO^nl=Vnzx0s2)=BynZ(8)fLy(i4}S`FaeAfs=GK6igGCVwaUO@#bjNY>aRMi5_< zM6>Cp?saw1lTh%HX^$;Kw@V$vJu=b0JRM1$$t1)5pP%KU_k+NKpt{*M06 zPygOH{N8iO{vdow=m%`@Q{&-7ft~dM+NYG+(GDt+va0{E2@?lIV%$(YHx5&1rQF(3 zO#U8}pz5}dd?Ziy=~yw{cmFRZ>-M zDec-c$9o{y1A1hC2z>RX0}bzxUnMB^;v=|21Q;r#DFte4CwRrw^eQ(@y*_?Z(HjqQ zv%5sbB3uHg-XX@#MOR6ZQx)%Fqry@f1IEKt&`Zp5IEvR`{xOB7XskyRRixxNIE?<) zrI~%!a-5M%-mjxn-yw2rYo492i--BQGjg2Cz(B-ophSc)6b9>Q{kvI zmHNmmGuN&x?{dZFxGa^%y-{}NHEr*WNp0CST>Mz8O;dcW5ZC8WSc3xZHBQd>e%ETn z1%<=YDeE`2;Ngwj41mO$o>z^IC8!*Dp=+akeOYGKG+uX>QPX!rdr#KrVt3YzX@29d zKI4Qb(2ZO9-e#@Lya`HWqL2#PVNI4X9wyVwIasVG`HFSz*Cj!6uEf-E$S#`okJ4jS z>pIi)?3@Eb#{4OcR_;$xWhx=F`Dq9Nx z!e?t!`kpRT$<}ID7aIQh1wU*c(o?4i|3(|be)`LFMdzn|gg2!7Z^C%{NOvQeS!+2v zg4`^bEE! z#%}axp0;M_&Y7!|&T3p)7xC~v&e+NBzz$U#w=dMM5S`PnvjT)^`l1Tfb^JvWe%UZf zc5LmC>%^dNWJiSKC93Ro47)1W`aTETm{_;Ph&_(w8$9XsMx6uh;0No|d+F9QeY>#A zsgUPcn5vU}x9OO>=ILAOQ^>EBaNbSE>)(ucQX0^%%xh;7>>7 zHL##?K2hF2@WdM5*>W=GF@Hp2^m~@tIyZ+3snETaaY<`yab^^`mBdsrD(Xc+ zhi?H(h9swicD9f9+$f_{PIu%M@RsC+h}B6m%}OC4YFtKIcp1f+oVO@+?7|Z=FvdAF)>GC2o zt8X=0kBaJscw^1hCZj?2mS$IcEvP_t3@wn!yq9CEX!5Q@BC>?GA0#1Hz^@-l9-5&A zIFnMC-S>+8c1k)Gi|tibgWY9s;+Jpu*=hC}oOX8)hzIDQxSOFP#;(h$M%VJty2nAo z6Y6zaitSZ}M*VQQWZ8#s^v;zO5oC5c5L-WL%Wj|U^A8xL)%m7lpsEDpoCx!^1#&WQ zo=fS1MD3cDt<^Z{D%dazfa1tCirPv<)$@b9AqUJLji^5`iK#j?+{5O(T+N7^Dc3jW z(BwLPkH{{aFlp`M=E0`X?1aqwMl~b&VZ83WDn&6bQ$yX>>JjqU?O85r=wV7ZhFPqgYmOuMkoN6x8W&8ewi3QALUqp7HT0t0%1LvKYytW$KWB0fZ@)CrVz!zIPEPbN&rE-J@e$dH4xYuh4}0ibA~;LIEu-=+ z8Nkv19Q*s4QD&z#3zS#im}h!dr-lzcH6=abDTQKWUlEXyS}KT^*C=nedSc8>>SQVt z(C~!*npX}V-Msunb>jo%ae1acfYjT1Asz(v9D9R(VC@4H z5Tgp1!y}!@r>(INPjr9a2eEH_|g5?urR=mi+tU0MO-@ zZRrMt?QrBn;G{SY8`mcMu^{t6_<&!6izI4xv_^p0Zs-xsE){-@F6@;Z3l`UAOq3d$ zZ7qaN2yvJ#wOsjX0S=lpi@ZFHCsG_0i)YGi&6@x1gKCaTmYgOLB||(0R&h`SJn1ia z0NIa_*~9q5Lzx6S01J;JyeTvaV0(-C80N5^k%8GC2oq}4Dol*QQM+`8`1n@v!khx* zcM_J&N)UAl!bx7ls0gQH#Gr3uh%Xy*#VJTdFGvO7+>#PkW**n~Rb5AL#KUv$dUUEb zJcpQ6?7dI*jI>{c4Yh^X8dTj+1g-I_$YK@;8t6s)@hB{K8Nj+4`fwefLPG~isJ!|6^fNg-W2b@fqfgP8l9mD;K zib7-70rMyc^e&c@`za0c^CnXneSit&M#!TOH`JjP(c`ooLG1qI2fIwhnN{!Si`5sP zHdiXi{C85M3+R>!`EotG+ax3)2tXa;vqiESiyu;0hw|4Wv@c~zP6Z_*RDvYI*ag`H zXy`SB38$f0CPT9Z0V%5PC;)F-HJ^&Jm7CF|kCP@n zsUkVKP&b4FH)*0AR91u?4|1Fbds=-D%fcwINU()070^u)M$feML z+P~cjjwTs|1^J`mPbi34Gz9I#`|V95S1eQC);#)Nm_{pN%W20T4bw_Y(o$l1l#zJ* zNHefgP2@2~Ky*SOu6SLjMhN%us;JZx@7>{@DRuO2*MCryi4k9E+FKk(XU2FY) zddyjt1p%s@

VVXKj}t+NxRwKOdq7Mb053Krhz7kqCVQG6cm$Ut#W;Q!q@JX@7DF zfDy{TY{^h|2EcYiSnQFJ@eWD~vwRgZ#1nkm9toMV<=+rPtv}lQxy>#BB9Y^~L-6wx zW2TnwTj`E0C}luH(I7JrR8k)RlVK!Mz8?o@EZ70nOG*cjG?{4KP^hOc$L&0=N>LR` zFZ_Fp%nNV;LBazmIQ@h@R_QXpobQ8PfspY0B-ZqieFn|`F;0B}ePY!=fC;PXzVO_A zh}z3#D7vMtA$gB4#)x$VLrg^>hg4&@(rTY`VmLpumf&Fn8P1<5(n%Had~C?XpA zIVG^tem@#ST|Be);6h@AbW|oHJWoI*(W)kuW{z=)41%{2g{T}j6oVmo;)K$!NwJcO zNdZRv5d#FNdf=wI2xB!cY`g|at{e+*jhj1DF677-jlI!ocG7%+lT$|0Fm5CoxnbBy$Kb*R;8p^``|cylyDF;}Fpyt_0B98udE)SN1c9AzMD zh$I?L8tfeg^&w}-rXa}S9-1=`J}K!;8rT?Fh6;Rn?uF^^Q7BfKz)9uHcvonSEy%@8 z@BqA0;kB1XNq-CO*R9}xv$z2}b1;c7gvqdXvebt%D0ihmaA83a08urG!(ihfZZzz1 z&07D!+R0yVg9gXx?X~`rS}f%uQw{Pko7n|@_|Qx5Yz;>BjHpsoe#?H+BQ>SC{5^~z zoxQl{p-E2%K=(V!^%TtEa(xKbC6!ct%AsyK#kH`n2NKD+H0^kmY{Cf-DG-}W#jym0E>*uiI?-RlBL_y|0XW|ni+d%q;ZP0*NjXo^NJePq$-+a@~JwfON z3Fe`%J}6d+z)2-p1>uCDhoP7P1?QkImY}b3kOFF{+2l10Sno33JN-R@I+q_W7PcTg za0METLYUyy(bp9h!S#MSpy#ikzN4<3gNUjo#0=fQe-}zYg{Ov?9RjrhTJ>fTYT8S4 z`rJHbY=i#D{!h>w%F--ibJlH6v;rC2wV8B~mVCXbv?1FU>xN?4}#QxD;(x!{m z=&(Sf;+aZ{bu;dG^S6OOLpLS{4|A07Mv-UQQ$ffrYD4EaojyE>hvd~2iU&rt&Q>!x z5-$3*SwS+c22?J=CHb^%i8a3fukCQol3>jFelSdQ0l=`6Usg>bBZj1cI!6svce*v2 z6e87(9PRup$=>Xrp~m2`)yu5|ROaU(D#t)UccZVgQdbYHTByQjf9)~=n@HpnJn z)D6D~EVM$HlulR-nAGcoabv=*R&31HY(4$(j(n%DkK^}FMXNSRCHZ-CVh6R(DtBfkrs&v4$X()x#u|`VEiXRA~ zPt-yx>4vb_Oh#r>y{UOap~+1d8j3nLAZco}=+xYx-PAxuUe-#RF%VIS#-c{ZRi;8w z)@q;84J&)0%q%A+n~bt>crel;Z>*hF)BfzdBnN&;yv20>#g zldI5t9%!!SCSbO<6iKVtvuhV(S*yiF9ct+0)b1>U$2OG3ANi}pS+}j9pNm)r`DWkv zl&~y>S+2uK;zc2^12a8q54sFtcMxq}Guv?5^&6Z1Qxiad-gW{{*omWQB}+4Yb>XMo z!AF5wS;-?e?J19sJo66F=PyQ&ZIqJnW%z(22|VHl;J9m++kJO+gR=A4U72#E+6{V_ zqPqt@EcMiySC4n#}4wQ-4H;la&GK%lwhwpi>_}sMSe_5VjXCBkJF< z(?ehS_Jme7eXb=K&w_SEf(r%SwN>G)O3nfXdtpAV-TfXPAR_Q-NW5aP>5=^6p{mHAk!_gCJ^ZwS-|A*K?IW%6VJhhX# z+VKl ztVW_cgb`fkk_yK(ZzI8!44nqo1#@S7HPmD8vkgH2I~KX=%v1I|@;E%xH|u_g+=BeZ zpZX$FeQGZZAr1ifMvB>c24D)oEWAD&vHT`hzZGqW69T~p^4tv*iyu7|8Ncruf6?ep z69-?sAn%#`tJ3li(}ny&c=(6EibtYDzuT-o{ADoi@;in4aX&--%#j7bNZ{17Mp>Ws zMNrf*Poa~XLmSb0^eNSu?8V3sxzk&5F-l=sJ| z;Z1%4gntwo&=U^ozTEDME;0LkF0>ss>R#!(Yb|rYAh$H`>Aa8EcpCP3V46LZ?W?A? zA>A>gNk5J8+{~ich2(L}rd;P&%xRc9VOh;AV3*Xpo z*QwA>FxXe%L}4{48%`bzYjAbm3=)P z$bu9uiyS~U8Fc|;h}{iDR-@U5qReUD!e^|Q+s1mGAzO@Jnrx-%B1OpPPkv6Ww&cR` zueaojH7?zcxnGq_Y_l4;jn z%D?{Wdsz_^-6Ial1LbLlVPh~-7oLI$?{X2|ib!SKb{JoQBQ%9QuSk!SJ>C;yz} z4HwxTn4--Pn(TP|Bg9Md$UB5g`LsD$U}b0%t0om(Mo5dT$@g_OLx~Qm6#`s~9m;Im zKW^_kEEy(i=IIuhjGI{%iYIs%CYnTGXq(xzz`s; zee{oif9j-1)vmKVM8PWpnz`wp$pX;UEj`S7e8|T(CmXJ71*bL%4@&j&gq;PFHW1_O zhy{zKnJM0|bdSoa=Vld)_2p{kn+Vc9dPNTD=DKmd22_IN3G5F8D~xbKNN{3A)q1fQ zy1|{lu%cQo3^UAyZs;^rQ^+wrfFOSLs0pkjn;!eAjvjxE#b(2>e9~v6ob``mtpBT~ z66&{twx8AUPeel5#{L?6z)uHTlUo!qiB7Kx2;-ZM^fI1ulbrQ2R;)_IBBY;2B{-iO zuneQLq(mGx4LK^iP>%$4DS~+nI!?W@+a<031K=#ih*|fNSKyBK%9!B-=c>`O$Bie` zB7O{ttn?(+P0e5M9=eov=`%$A{rT#PeB^?19o&mai zGpqSWsXzn`F`)ga!7g+CV#SShyd&PvAjm!q*TSDWp(e^8RT*=IX!zlX1%J^OlEYwe zpq9NE!m}8AyYQ|3@wRmd$oe*DB8l;V2x{o4Sw2Izu{*F$y>s`;Scxp z5B1EG;AcVzI1$0-C&JHDfgp;fC)%8~>L^@r6fOT{cY5hbgaAB!^xXb;D>?YCRzBzq5M^c~;)HPM{x689_nJ{^ z2bC_kS`4GuV5Oj3gxG#o)ET*}`uDKgn~*CaaTi71Aqs%i=?WiaAxt)-S6FtG1i4Gp zjT59yBK7iEx*oVo9`52MO;=EyyxhZS`7}5Z5<+Amq1u-80%MuNjn_eiSpci$EuUo@ ziUa|(i@b`ADNL_y9zGiN=e4yG1Rm8;NjrcA^rk}kq|@|}SUmB&lLOF1_TbPY2+8P@ zTq_7sst6bQrU-Kmssln$=7@;?=tAWkKQSvL8~Rmjbr1Y?p@K^%5hh+@WZ5-&5?3~U zlI639Ux}gs@>`HoPOx3?*hOONp2q@HFmr~Z>&QiiS#`T%lKKSg-GoxNXGI_>MA%BKY?NwL1HqB!GTy^oBeev9N@jRwQJ2zXT`V5`dW@`T` z77J1Y8WKoWxS#+bnjifq6MER25MkkBfx>k|1+BCA5;n6zmjeA?wyOH=HvCSGN*l3r z(q%g3g*4WkxG9hnNL_-sXq!Ffal#_5 zDIY<2Qw|(;C1j{pY+s$!sXDPkWghK$fk{r63=-I*R8~Nm7(`Wp&T}ew!Az;4NP{-l zjW#HCV3i@B8~0!*9N&dHa++;BZYep31Hh{_WiLsZOOO?pDUejOp{OKW$X_o<%gfrL zKqj(>8AGQOVND8u3j1seIJx8r{0f&5aq1Yl3YYQPE8u3`@0@s5R?H!A+IUqqZus=A z{03GyHhe#SUv)G?`N5yQsjKd}Ge4@Un&2ZW^N|#I48VLMRLpi2s9%S{uI86L_YwF( zdk5q;DDFT`JZv?n3!!tKTJ%$0I;_luvONf(tUM18ee!om3K9Pp{!)AU>Dc4n$QzGS ziDO1kXb?8P;i0?$}jhl+N{SQa5Ukj-bC%TaxTcHu)RuPiEijBHNH78p0 z9VT|%DEtT>yhlzbM@Bn$zkv7&H|sOBH~-Ri2ASl;m>fxOMDSIA2c2sue!`vCuioQQ zc%-A>7-@|6^!ew4j%~@42suLDk-Y;EDdHw-_^CBOD`h`pX83B0ld%YeNmoSeTE^uwTOK;l<1c8rNBSi;ohz>9#HsHgueyeMXMOR_dv$MJ|8`D zxbqaq;e$pYx`hDyOklNJBE2<1WQwe0_+k^B1_PJ-X4#l)9(R!rPj`^MKX!rCB~C;G z)ag9j1TYX>##oPAv^XI4kio$#)&&GsWMNUu5GvGEF*kUpMr(*t)6Vd)vX$)o;5bLJ{C@t|y^4J^r_^YP#9= zRR}XVOah*N0dN_c7a)>VDk+QYl6lsZyoayAO z%p(LTkt^sc5@35o^5uv0sQ|^XAH^X!WhvkW4loa-%t}cAnImnJhU%2#Pb{~zP$NW2 z4-au2GH>lf@

b3fdgn*AI`&=XfdichXQ$9KIi>3>c*~If|1PUkaqP=_MVfHF2nF zGEX9KP8qJs3+rZkN}J#2QT7Q8dAfVRQc~qZ*tstMO-A#Vi4%OKWKU#FpN7R((b|;u zGj;XBg6Z&iVMkg9=s^3aNB@W|dCJtGdnS6v=rD6?6kT;;zA^HnJZ9&3uD2EiUIjkfdMwsz>fEkKEL}-m=E=R zbvOFx=lFZj2DjMON%+%A;B&et$2Y46CMA^qIxy)E_1>(YEGd0{fT-V%aMlb>Ou`iA zdZuT1PxWFo;_`7)zGr+7lhRbckOGD3Q-82LpxTbvPAI-lFah|!gPnPpeiZm;L)^%H=(!mk>y~uJ$Zz|2U$L`7A4d#r1Vkm=z!Y@H$51_9 zc^W|N93Hi~c|^w25D2v?x(Vm*tiq0~tDLOwL$#rUO-`CdKtM2*1*vC24Z~I0g)pI8 z#91&5V1 zW!wxoeHaEEj7r5E8RF49o^ch9Y4n+zq{bG}M*WYH$g~oTO5u%;wiP4h04ORp2s+h46LUGH$f6IvN*mW$ zeXV8}a{pAz8f+k!DjRc%;biZ3$FXw-gc=E1AkyCr=&J1Okbrhz9(9uKqDl4H-mY}E zYYH*3s(y4!CS~MPM7&lW{RlHQ@C1U zk}8HA@)jw-GW0Q~mWaiLO23$SS6;*CyyApn-03izISy=Tn5P3#M~EI|Px|-|`0+#A zh%p+ara*KV?-lKd{;Fx!wIbqvknr~?_)>>XukwQl=~NSjIg|FKEXibOy7A!d?MoiLUaA%ewr?@1c|KI^5fT;n< z$3Ku~mPYy>EDr+^+7n|5YxH{SVJZa08&)|q6a*AP3^gRgm>Gov1N3M56ab5}%g95t z(GawBm6=Ynf!=U*ps1jzDB|VvNGtD3fPS~RKybq6Z5n9q-t^1mdiwnN=>2*!*T?^4 zldnA|bBo}`ZzVaVk+g2gzAgTD-r&xd8B<4Q^zY2kr~7uG;tS{Db%F=UHDEiq8~GLb z5PU2=A)bU-R50)Z{f=p@JYj|;U$i&C7xop-8O0gK8Rw4Ua3{fmWCze2?2F6;_6mAO zen>qABoPwz5Bz|8MZ7Z@`zHA<`bFZz0u$arf}0>KiA8~*1U?>mD6*O08v#Wd&=P%FjWARXqj*oi3(}Ca2ZgfigbaLjf9P~nWP+GE_y6{43H1b zLCS>Y0e7Yvg3m<06|pAq9|Ic$NHS7@XhCQpzyi)mNF{NUk+r}gg^Ln!NNl!$k!*vr5I%lVd_UpU!s&%W+#@tX3jdgW`i#fQp;kBT(kbglq z(*h4!$Bq){N$@vqh3fuF^n5aJTx+KvbfV=SRwg#byduJLUufdOq=$Vpu0|>I8tlfW zq}4{ePT*n195TFz2-Be8PiZwuXjNa!PP0x7a`;cf(SwtIr-XIBevP?ae~_fMR3W@~ z_h)WPf}EHenb&Zl9Mxna^3dzMu;XfQ zbX3r9B_dPz&iKNsR@S7cozgDumwo&KSK)dJz+x*;vy0|lNxz$%Cqj}%{Bu3LRHV|l za1h?PFOHOyM~cF@u;NgD&=XYNN^sjqEMa@4d&`KLhIp&dl*5%0X?az_Y<(%p^#SqW z&SGuJ%gwn$(v&=$m$e9~6t_>=f47&>VU(o(0+*ux6?obqJaiqc^iHmN9q#S|rUu!ni>6ZdW&z|l1eDSfHck#kTK0X^5QF;3 z9Q_r7KQ;|(s8rdfD@@%#nH564Tcue9*{q8WjXmKmQU>|>PLUa7mYm&f=>XDVKYkj< zjTG}^x=bi9xlNUd%pK3VD?utN{M1hpFW%JsLTY zbfJgw7KN8dzKX~(Sw5;sUeEJp%xXz1u6gu~G4|6f2U7megmr zQj4Ifhg+K)(v8=4A>=m!Q-MfU{+WO+m$f@L1OfdgXAL4F0j`f4Ct;|i{Ov$_`QKpw z6bC6e-fbbtKHG^#c?qyk7gd^Q-1S>8Fo6R$ljrUJ{d@Vp%EQ|wm7e28)NRV zKTF3oLeJ5$DV9B$CQX&`h)$jyHIkV+*>CZn$Q*!Q;FEV8?*(r8TZxnXr+cn(1hPtY zscw9Dd4r7A>Tt5R%=x*~dJL4;*MmO$&^>lr&X<(~+&^SFWwredmNu0*U?GR#_|>M` zSKrmCV4Hv_D-Wos!&bkV?94dBW_^J^TT7hHCS$|f-@q~dFyu=s$-4mE>R&el>9Mb9 z@coExf`#Fu{NwCZeThocBnwB(CG6jM-31gSi$|0IEw)?xlKXjE9ML0~ps>5B3d5ZE z!^nV+X)DDm=XYx=m&DB9>QffJ?7KkLt)(Ezj_J)1XP~jseZ&G>wh%EQ{rv)ZaNf%K z(isond$N4~R^Oo3>7a{JZZAxeq~dvQQF2uxKICDs^1V_aXMRXdE4!IUQavXiR0=ib zUc!oR>LK*vR+xYzjm5jyrmf~^iG6V)k;|OHXQb6h&Wy2QB?hv9f3u)|O&!$YYIxL) z9`>q22x$JN{ud({VggcLuM6fQZ(w1kCI-wLGptvzO4h)5?W6dpIGO(E>(0w01^?eH zf9+Poz`W+8kpXrj@C-q4fwNi!aX&vNSuV2{Srql!+dIl^@e(e{Df?`G_3C4$B8ito z&-qE18G{fm*0$)y+g?Y#9fythd`Li>o9k5xq)Q0uQW8xjWgr=kmM?_-;$ym=G^dBh9wId6VI zEzpMj35>6xv3+#|@14&_!sNQ?ndI8$Qk1}NNN{t#F1( zMLsxXj_Wo5US8$m{=~W?JoHf&{8t795R(dte1X5>-oYPgj5!p1jC_^Pv5s)$^|NvO zo8RHc`>z6UY9F~L#VY}s2UCWzf~5@tw(Qq(Y(vds%_Gg{dCYoBY)c*FdNq!8-4v5- zkqD6Vko1uHk@%5z3W)xXt*^A#{9?=}WVZ7_}8v0B~pw z3dVtPIakHE`DHzL!akzI*@@?f4-G+C!UUdl>dg+;jt2!#(D&b6X(gm1x<~f2;3l!b+n$7Km-saMI z=1c#{im`$+5rQSa|7{D3lcLS=cfm={s}Cm~fohkM2_5X~-H^1w1(TXlqMKcpC0R3j z!x%hue}QRkb$0w_E(dPbZdHLud5be$65UfMqcREY1dSQ#;w)IoEpIgxL+T2AHA#8=Wv>|O{z;Uxja#;|7F4}zRbKb zUU;F7X#u10WcJ(oo7JGCh8KHiXTIJ%&bi7XuF)%sx5Fbh0T%OLC>7U(S`NpR3}yL1 zwq*?I=?fShs%rana;s*Z zYTjSpjEo9Jrj!~2VX5;8yCe0x&vVR}YqNrlISe+#YUTbKlz&2k=6rcd>+Pu4po$(t zl=pJQ8};&Q)et6BAN_wPditap>Ju_U9U71DT-a>9IiXF zSo1b9$8GBE?*VUM9w5t~=j*6Jt5Sho3|q;q{hOi5>#HU73=)AL3@|_|JW{9Qu zWaTkmwVeINRPuRftwKkmjRuvJI}Cb*9o$uhmzU~@?3i=%tL7+>JUh(Y2v``WeI(PB+|>3t@Zj5# zCy*@4WgoacN*d;*K)!)x`2?i%PEF~`oJY)=#wa!nudkkk`ADpt&ssi$1H8cMYNRuK z^3!=|r~Zha%UCoFmjYKQR#!8^e2mu4>nvS^vV8FB?Bp_ZhQ{fdnf&2hTR1a}&Fn}X zb*8M-YxQH48swTAt|%VCzEmsT3g@PL16KfZmkBW5`H7n}Q{?Nb?V+83s8+eTRSfU= zMmZh__Q5IN@D;g|UOo*owQ{snYJe9}U5=Zv96MU7WVNh5uy}0;SU`49Hr#1dPv>7I zoz|0S=yuPI)4kKWC)bRcO4Urzt(es#-SFlW;(4KSq|1VqI+h3{-9+&DgYAgq=ujOd zvXW~I+_Y_Tt7IW_dd@t0*jhG;IVbgp?F|%x+j?0dV)<>Dq$HY3wP|>Dmqfpc3#?k1 zVv0t70};LwFs00*h<<*NW~J;A?*VSJRFA+;+06+eqt{GwKL2kb<|HQGeTdQ3WPV|r zhbF7mgP~irfSbu!7iO>8mE+s*e4p%vzpRp-MUN$6k938Pcgl|?5sy_xk7lLkM`4cy zg^y8fQ575iuneg&e5qax-)9A3nAt*LlanR6 z_hz_CNZlOeK23Ni=gWJaJWjwEYsEVsp`pA4B9ynf6jEO{pgS-$0e%PX{wa-IHyZM3 zPGefVJYWXgK(?i1U_7DP)x=+#G=Nj}O*3_VasMLeCP!2ZRNdNL->a!=^T*#zv%Snu z?r)fjaO@w_2qS65b9Z^bL1xfl=kuqNuf?xUEKB+lCivh z;rIS}AEXqUt=sjxw$&j~+gckg>Gd!uf19oY+XLSy`?iaLQbxE-ehu9O5)N^;ylLGS`4rU*P>h4eQOb)W;_^nz6l%Wx9 zi)8x^vByPnn;LL!1nRr{O?#T5&Tq&WIP0&oXWc~b%mvad4`Lw;>RyKG9!KDw>Zb?z z#-POhVhFhQ0wr`Xzdd+|#i=#Rtf~V8l5aDNLCekg<2@(UlRt$uG`VhHfxipE@Mu7> zQ!53yuq_3k?6A;y z?cusrmcUpMYA|kCFnI0px^=GnwVnxX)u9GfU5KqOb3!o?kT^N5ufaRV4iJO+Hh`CqV|RF?AHe?8Op7dho6$i;UtJQ$(R5e z%)13GyHz&JwS@b|8nv2aWI9y}t0gl$Q_8Da!X}#Ln*UVQck48!eB7q*Z6)nLx`pm3 zJ&s#Ne>%snZ&AH|_2U8`pDH|EBE(lRU2FJyTE+|3sF_W=hl4+Um6tY$1=3lFJ;b(E zxrHe&_Sa>-)}GRYw+{kZb=H=|o)Le4@ES@33LhO7q1)8%yQM6<6Zp&3?XJ+kRtnVZ z>Jpp?9#S1UJM{v2->`hi8l)DNpaz<|zT2GH{%y^#S{Lb|32z4S_2w+Bw}tQ*3Gh

C24FMJE0n)xO0HUEE!oI+XL8}|Z8L3QVUHS)UM5P}m>{+yO1i>In$bmofaH9z8n4o6GxycB&Ui6Rg~YbF23k9>iB z)WWF!g5D#83Bx^y?F-5Nwl+hPzWMWOua6VX0UmeJ7A6KZ9vhS1L!tEP&aP6Ve}EV> zA@_mcqi+fs-q8HSANt}E{EACycHs6|abotJ$qSDB#E{ViMbhUA29Gp=AwgvgUTS0? zx&upQW-TnUn4iI)pNrf98RYBzPJ!@C3r45T#+HS4t$6ksF*qVF(R2v@lC#8!Bs^~U zHxR=~lH9P}g{9;eMiEb&VSO^1QiQp!gq1CuyE>AZX?gZO)e^VdN@K&`P{SqgNGI#a zYpC@UHIlYPlr1Txi3hEu7hrpz2|;Y(|~_9|Q0*vSe!5%|M1wm|$Gp$K)T6sU5LO!HU=TzJ5P~H8qQ~ z#bKWd!l!y(#Zz3+AA~ijJPX~$a`pNgV9?Jj*A-|(@~DYNs+J)_!n|ajZH8%?S=BcV zW&7w_+jh+hyamv{2Z(S}k?EB+xQUug3sPFJR^8gU}9<-(n_xAR@+s5NwhF9s*Z=5r{AkbL75a@1mp5WSmMl#t9=8 zj35?3V}xE}Y>l4IgzWZl$EAIN)a}ol&pC;#_YDa8ipM;)T^T7`nPM+Hb)(jDtiU1vB5E87g1O4FtUsxUqK_4RcrDGJNN! zjJ}EYFFTWgx+H@;Z%SkJw;7YCY3)z-lt_co54Q;V;l?o1RppL#FWcnIm8=E{<*ZK@ z0(KEXeB{?P!4q+pirs2OLhQoAuPkvYoptep&C!y8ZZFNB(pQ%DdU%_pw46)^%U?QtVDMrjNss4Vfnm{N_*J^g4s;-$ zQ(bBzAX#^cHMq_d<+)?+ggU@GI0}?R4zsnBOtMyt-FZH&p8HS*3?U4YQ6ijk?EBL~ zH^Il>8yGQ}k7Lmk#lqA%H>2UqXS8b(W5s5TWXvq+FbS5i)ZvIYmX(DiD^^NrMK9Td z2{<9ik|g0-J4QwMyp-8sx+M;dU2f{HPQ#T6{> zGM(e1_(_Lji{s$waM9KD6~beEy=7I z3Uo^e#r+Lig0C*dI^xRB$4#__2%&KVKWqkAj4jYcC-vP%rJDlH$gGpEGXv@Ch9#|g zmzcY9ptICd!gR$RRGxt~vu?A~u|o<%iu(AySvLMOTSi6;K(NJGEocJrPH()X*$j1W zFc*bW`n>M8W}=$`@Y4zhFobzs5tPD5Xi|I;gwBpL2l%S{XGEdA52rvO5u{sjqmjGj z)jJBM$L6FCVD`H~Hb1I2n_KO;9AXFKK@Y^5cq|GUqBjM8DJoEa7Yi3+;g~c1c>NlH z84&{no#ew$7i|r<2Z5-!ja4SSl*>oK`~c`H;S(6TE(npQG*gkR?~AOdHKaG*P%)+) z)2wVL9v*Wi%jO0dfR1A;o{`Ut*<(maxX16XCzk+fj0QQgUw1C`gpZWwbJII>bMhvq z=8})h+wYEv^`t4N!jU&@R(9f=Odt!9z1?O=cLuv*x{&}92SOY#lv61&!Rw(N-=h`{ z0h)gy{z3WwV)_4~e2e!pwIwLxKY%~{OIWb#i{K&|ECpl@5i8~TCjdGHSptEi1xXNZ z83g;$3K{-gLQ4iyFaFX{I07598caVP<-%g8ILgxn^h77}(sAR0#@a00^963>+J1aJfcu zrBIV}RIBlBxkreqgdYS7KP)Vz7B}~$jGNsZ`h?b9&yQ_ZVszI4r9HwK(X|m;(ImMs z_BWvenvsJkQ}c{Pr55Nlkt6D6*8X5k)x;KM2VOvIDIGYeB}yG9S+|y3j8?DssFw}| zLo;_gc9)JSo>tv@i!%*6;HdcbmPP`F{&L|GKJwpslIi66!_+kQ%7BNo@lp^^MvPp% zt{87Wi;3q{4D}Bpl_w^O+FOo&-opY(%c#inA71#HCMkB$E33U=SC0f}BQ0Jq4RbO0 zKosC_0fnsmD8Wi`Ee;_S|F?wC>n6I)6`((W!XEqX-R*3*n}CS?4V&|Hy9BC}Cw}jk z>k07tVqC8K*T{5-$gmZ~>E4qC$3q&3p87im$djm+V`d*t&sPc+opz#4><*Me^g4X^ zkY&vJ_fJRymWHd@)i5yMsj_Z|_szOHx(gLPK#zyE{rNX6htV=H?IC2P5Uxk}OV}7) zmjRS}CqPeJNJi!x@)2(|OwAF+KW_40Bl=%AVP{Q!O8q(vR)6)5;qXZ&Br2cDwO4fQjnlb0K+MJWkEfJXtMX!jkos)yvPh zm*Q*QT`0@n__6UhkK~fseDp^L6!ym{E6b|a{h__ri#5>FR_sIL@oSNTL1%g z`t6@!Gt24=#!`s!UD-bi)EK%3sWNgkRxyp0Wf~Q$`DfTmmesYaHY*wt?U$~uG7XaC zZuM^5iV z9&%3b4n&+Jm+3!g;|IgJ465c5gD#kVhajSsDo@ag$h@?4GRPbgOffauS|n@XaNt+B zw{#?7$d1;NN$PG%{XNwp7j^VpN-jvkAuPgk7{7p^l_EsVqC9{NYKviZ#3PId;nQ|l zo4!JzKk%A}))^1gPxBNX&x@t?aW>SbGT{ zrJAgC{qb0D7@6%jz_z91(Qsz_*ojn42GCbHN%Z;45_g1Z)zp|raLd^7HoAcef=5vv-=J2Yiy#&D(fA|!=MUvNW(lqVkW*L71sIvJqF|wP$hFn2%;F9!+c&@{V#5FI5Rs=y`8%{4t>WgJcRP7+MmF*EZmt0*|D=+1+u%1e?MqW;le({5JUAS#oGpR0~|>|;Mxllm%EwlGXYnj;H2cLk%iy)QuZP<#4MNO1Lwfrz{GeX#=AUI>uY zwR%0i#p}r2C%6UQf!honQp@wu5sZcQl6|ZNaGU;Dy!YRUjizR#15Egl%>#GvOPf6YPzm1Yq|+zY0WY2w9Rz;666qs1{~c0Dk{W?SKn2TROxY+%oo z=da{uNp?jTCpTl1MDEPBxp!4ck;=vCi+A}^3S1)%;#Z{SfKVCO{mktW@gUC`_1@8> zy17}ia3MjABYmwC#I2H#*bzUb(WU1&#rAZT0x^>PB=*RaZg{06UB(LguuZ%%ujV|qTt z-f_flC@(2It%ym#N0zaLJ)`mfTW!zq3-tsC!J77o8f6}hnJmX)~k>ASfkosJ; zgobhKK8q4>^Z$d!|HaM!MdOw%I}#00n17&`Ep}6VFG06d94KWsFM#~FS>k*F8&}W+ zgE9s)(LR)haCfF$5u;Qfu(@xh-3Vzkc#>PQhNh25a2Fg)0RI`}opNCRM__z;NOdC% zU~7t_W^)VZ`xo^3n;S1@tL9L6h*j%lULYR5s?LV9gH+FIyylvXzhIRwrMz`93=C~k zVR2{27Wo~yr4Ws1c@=B)f`Re=NOpk8dQzI-r=S(H42B6#-Af_ayg))ze=gpMa2!m% zG(^n_0nQu~+q|}xmGE?i0k~opbe1; zEo|j|{1ZY&JtRWg0tH%G$+X@aV*%zGl?dpQQs`4cdp#z*^=s$>T5 z8!$bW!?{hHtR6ATTCyKSwOU61F0eBW|8@toHtVZCXKd$AdBqEF*xyscaxge=ii|Mw zfk_!Y#z;PL_d3_b7suYe7Ut{Bm-qkCrTw#HP|E$hI$t9OUpprIw=dc>BsZw4 z|C|}k^OFFImW79V{Dx=xRU*KBoU`HKnHn;h%StFjYhc^{A>Ikv|aCE z7lU{3e)*BD=AsW`zc9div5Lc*FtmIO4SIUlHDm%9>;5|@%b>Egm*b`RB(&$=XWiS8 z16ZT=!6#HHd(f<1GZw3=J%1=WNehvEurN~Nu)ZrfJp=NDig93~h5SRe-VzZBH(P(? zF?WJ!S~IuS-($Qk0JQ&?H85Md|K}n)QTV zlR1=2(%hD^E6cvzGE2x2R!HGqMC{F#3bc6UQwy3Tc?2&V!L-x)^nQQz@$Y&Wa(K&ACr3}_U+{JEK(nClY6rllroI*3{ zM`jX-?PGTXu^f^~WcCbWRnVg_C0$D)EgkIrDKg>?DARWe(Pz}WWn&WKkVu`t$Wa1D zBTFa)BTuL}JL)Zy!I-(1>{KG;VX^Y-Fh}1ZXdkx6U8ZmvKf!uFC2?wKDpIyUhooBI zpU0x4WL^~`XWf36TX7kP@?uVYWFN~$am5JqCgjz@9<{-gbV&50h`E{3GS)&v?BOU^ z`h?1R5tHnK-YZ0KF{V6R=pO}B8Q7!%LDap}p8>I=;uLsf&X6R)Z~ws4zijQxgc~9M z7oLbG^^96BMwy}hZK+ngjK1(GpRVXMKrT{ddLwHCC#NVCEv0z@^uNu6qLPJ0lz$)^ z^~=PO2l23G1yJZt)gev2;Z8t%Yfnh=f`B`E4x(Z=y<$Y~3ejc>j&W(}#RO=I&fwNdRr_)8!(T&+F z#%6XLhY5FMIDJLHuUP2bTp}ydDk7)=(ZH@4h=)%dl?g@^EiJN zdUZfP(Ve*W-Miw_r+_Q)41JRJ8g($^P0I14Vo%LHF)Viufo!D}rj*2hTE`Q&TBX)y zrIT&5uvx5LrBsQcF|^lO-36nfo4ZfUXgIM4VW4G6p+9aJOWtd0b1b6*GltF{Y;4EN zNGHnC#APoMzmsa+32q%?h?R{HU|33%6(&LC;clX2jBDF^xDcc;7LkMmM6(<(%|=)t zuyC8+2oK|DTe=tb4|GIBC_B^c{thN1V7QLQZHqe8AIV7^vu}{8T0jq@!uE_o#aE3Q zNz(K!HTq(=Dyv(WqR&?aHqjY4rMQZnY>ogR>9{%e>4na;WwMJYNVk%db9Wwz%i;ro z-g{mE*`B2IxMZsIpMgHq+ymGXOWue9TKxpJ&Gck74n6v{QSToxi^S67%^m_!`wtb4`yq`le&L= zaKWAzFbVn39NK5UcO4(#({B?ad%(ueUZK*XMS{KyGf`XCzeAMi9jlUe#zdnpjhN`! zms>o>8LWgP3wGGQ0=QMyqA6|sX<5vp2C-abhpx1B5wr zC@3b4rPW!p>-y$}k(Jil%7v{>OS_g7(U#j~mKZr0_3N9BZ#UO8@XDoYnQLp32IzB0 z>GyJv5cD}VME^b&Wp%B}4&9w{LR9!-XYn^R>a+K=?FRUq&_`E)+nka2mv)kh<8MH> z>CaD-zmmy+lOH&arlWMgA-_8+L7I$>6j6@%pxH_nIY4F`8y2GD*h&|i<3o-s7*mhA z7?DR!|Adi2#`UOA05Zt4V+r_t-GLbM(kIkW<$6=;M z(PSw~BWpi0QKHIXC|C_Y#8y5WLdB(+IjC3v5hbRs0_&I*c{o<8sSuU#A!ybmp!eMm z4xP@qAPtjB^L$8BlQaVBa0azaJbletQxa?3JS~GhkGr=53kZtA5;}q>h`|-eb zeD+AbdAOD{bgK7HnM3Jvxi#A;#C`)HssP`gJ{qSw}_m^{Z zeWX>DYtBiFcvU(>tmMhB#3)>fl-L@VG0*G}w3vxYOnc_t{EC`F>an^{6?Dn-u#cKi zc|VuDa)3q=#oL%bd1V&cGXCQjdXoF(ECl~2!|Pl6fC{Dsv#@0B-#+&1#E zo<9EcdMgZ0X0v1=M*Mg(F;VlV@bzHe!c-kw?wIi@UkRD%BWs2tTIvBe)pygI9lGJ!szM(okIWfS#|kL6(XhU}u#5?#1s5!=`dpqjELMC0{hqG4O9v;Vhk;c#c^ApQR8tY74#uMtY|XexK-IeV zbb#NEAcgQRg{%THRu+7yaP8fTk75}J(N3OhnpEw-3%@osXShCUf^n~;HV%j>#ydVi zzN<1)v#4s@bx%(trv3Ky`PFxae-3&fP|mDtGus@Wxq!0ER#h#&a3mc;tm8)f&<^a>H3qjz&Ik6h5JcM*euu00ZukvnnZyI*JDgQKB`$QWpJeHE-Lc~`9z=Vh54bbBpbL_ zI0$a*t&~Wo)|w^PBrOrD0ivY{GRw6ez-7NLv(B) zjoBEcSV-o2mH90lUy|C{yx#SE!A@Rr7MS(X{I}LI=Tte|8D*`i?PT*$N4

YHaM? zSe-Gd{x0LqKQsERDN0crwhdG6KPQ1n38;tNPZg5`ST6h*%=iI^N% zY!%R1sRnp$nJoOO*>4=Y`wAIFq{HWO^DoFRao|#t7b|*60=q0 zKH=)eVHUPH3ARS*&H$gaLcp|9*mrqQuPJYXSR@G#}kccFPN~;g`6+^ z%utonX2eO?9jn)B7hIR|WZbbb#DbyMshDCDdw=sDLvS)pUNs!P%>5qPM+GHgkI8dm zpye1wL@vEC=COhBgi1s%gEf*vhMO|o2c{;2wQsFh$eE(M5VhC>HUubSkbt+C@gTE zDBN@Zm9U3xC7G>(wqYCe6052gZ*xbsoOF#=MA}?75Ia4;+^S|cb+reTi8Pn$6!}LM z8Sm`u9u#hh)@yCKu^ITN{bz2~I;LV zL`T^%S%U6RLE!7tM?N8j-Xq!slT;h@VXTyZ+hd3j;2ie5n{sF2F^1q9l-Hf(aa7kO zS4_)d&Bcj%m7T?vwdVTj{Mv%Ag&E8G5v!%m8apyUIZ}v^kT)>nkhRIh+4YIl*^S-f zzy})|Vt7OQMyz(dMDMxi&a%}{%Ucy9mFL6A_^=y^SvyQRW^XVC^WtD^v3nC$>V_q` zM!cQJA*l*I5_=CWunu5&6%2V`pxvw}M0&LHW9gGU8uHN&Q@FqR1q`I@dP6JTo&~C~ z^Lj$=Ssf)owhFjtQc^W6s|3;N0+CNCf8-oYDf;3t@w~BQSacoSWv-n!G6Be z-0u!hYiZOQqi3{o2RfXL;46 z?NEL4IO^2P)%!G{F2AUAaDAikot9rVvN)x1PlW9bX4=;;$MR1C{sGa5#%(+-V#ypf zg)D_G7U0M*OCfVKh&e5Dsh8^wV+CYkrLnESL`2G{Q?_INd@$)jCv)roRc4g|TkD?m znGVHQ!vuJS7ua#RD(8s$JKe2@4*&%09o4g{skb+gp+CIkw_WNwE{z1@#JROWsHJ3y zwmsdI7v<*4R-#TwaTL6$MZ}gh;XlDTWAADN*uv=7yaz8u4QJi5lv@ z{gETcxRt=nA)}_aluu=AQD0UT)4UQxZ9ja7x$ne^1@>V_knaYk{tl(B-tDckSB>Rv#z|#j~%qVwz z&gVFtrQKR9_~Au$^8MAo@ntn=qW!t{haO|cFHJ|phx9@@7>paa1_$pg{l;!jieD-* z0ad^zwoTM9c+DMjYNeJJS9S=mejzYgT`|R|9B1JVaLID%`(caXRWp%yg9--Suc~VJ z`%%BICSBxz>9j%yB9TBrC{BQl#4^i2X=9&eTgh*3?SK8Sah5IReTla(<(KVbGSz51 zzqfu@EIA3je=r&XrkgRKB{DJ`#aI>J?~5Q7BdH&|MQ{F@ zwgti-#1f}|5p};zO1|{;nyYuu$KtBj=T+}lLeo8t^oOtapPd_tkm2+2lDOH;SGLOq z31cF^9j-b#a;P{TeB10s4_ReGBlV!L)KTmN`TVyu%0-{zWE19k_2dKja+tkESZPJz zWNMjaBMIkoIl5BaJNhIzYaDy{U9TJSD`ODVMo>}gcM zeWoUK3G@u7qZVu`gjMkg-Q+XPr!pkAY8-Ukf;uoHF!7Y-Nk-l9?SKW=i|=`$B~R7^mj|KHt_L%ILFMTj@@XWF$&X@2J8T6YnLj?txDx z#@Nu_#}c3s;-Cvj9j?topQP?)adOlUDG%6I1aNh^<_YNha$0P{HzMkNIpx;Qtv*8A z^gB2ebuPYoWSZ*x z23gx!Vsd|H*lg}%VC!nqOVOB|2~XVRnB z>vo6XKTtz`ws02t;2o7M%NKR^r<}Ptgn1H7o`zhP6B%T^GI^g;td#qP*Ls3+l;!N` zKEUx*{u%HEK1ks!b@x@AEg{jY5PZ)P*==S4WTHOPSYV_R}ct}~4F+p3URQS&IdH5ePHT57(lSZB}GjT%eQ z_J?s;Tww52ya37P>*MZPD@m1YjVH=af{>)_auj!@yab`MaLyiF5OX{*`3&hD(xI5( z9Ck<`-w||Mo0$Deo$!)wFA`BH|5r@)#^}7p{D8D;%-$d?E(l?W6CRtZb%^msfdtD9 z8=`eOH54jJ!hIn%k|K)A{SPM=1KxboZc3fOlEajDT7n`u|BF$<+w+Z4jCx2kj3bw( zUT;NZhM}zKR$q(8a$TimO~)a&3H5?r&-6MFsQfG|Dltkm74dwze!gF@`$6_NCVgQF z^pk2|9U22ue`BmKC%tn+!~Bh?ze%`{(RPU?QfVfl?J%L_f~{s+D-Az%jYlZjRWfsM z3Wy~9O-%hL;R*=&q8^}GaYpV`c?X2@Nw_%>OqmY*I}j!+gj*Z2-$rp!vW+n=Q;Gqj zc6tp9P-TWTTAdD!FuF>z5{e?*_eHB9=169xg9%qXBP@o3J?VV6TI*LW6IrMvFkDUU z<#_(``EF}N)$~)k3i_TA%6_-#H3Ki_a9v=DQ6cDZc^~F zM~Zm=8T4L)bBK088H-#i43K zmV3c_wjJ^uN3vZsb!P&)Oy8?2clf>sO)wyk?>vtYS%<4JlwLSI8u+|75Jd!ZqLV2Y z`3xLgr$*%5FX6IGf{NeaY7=86FmiPf9DfxtuKZWHLB7u#eA%=Yqg)US@-o85dT#GVJ_?vB)7z7pnI6FsPD zG;2F4m`BNV>J(^X02~Ngn^f&XWT*!i8SHRSR4Bw?Ac&R5XF4ynM8{epT6mV9q#X%o z?Slr5r6)U7r8TMcs*m8oLiCw?0PR8u^lV&&g`1YtOE5I71_@qdNurEngpjfT+1xl- znX)At_DesD&{|u|i%mWPxz!T%4PooC`9n!)$%~=GW5XnrfvKO-g1TCBryZ_oizS@{ z+U!hv2Js0a!eYCOAPlAv_0LDF#_w z92FUEqGQwD{SRp9!n7t8v$~{ks3>roIckm-IlE20fp(D<053h=m?p9xFI&W!rJf7T z7VfX(07pqOB`DYg4h<9vYqARkga$+JlYGmYkt7qNwi<(y&5HsSUC~H?vZZ?={SSEda2s@+*dk@??$9u{kSnnO9wxy296Whh|oU=*G#V&w8d$`;0SIF{(Ft~aKlP9g8IffX3{D#ed! z6h?H%xD{ikUqJ+CA4i0>hsvLB!bR-b8NRG8<=YXhN0oV(252L~>@wb6_I-!!Hc=K! zli+ZM>s7m9BTc&;b1+dX~;nnn@E6=*BRW6TUW;9b~Wx)FPFLptk# zlW=T{KlZV>;Nz?wN2pA8cY0D$PzSdlOh zf)}HA{447@;*fyzT#WMwvdqGUQABR4yNrNB!eQaj`o%gES7HN~^)xCy&So@9>Ct?k z=u;Y?YrZgfpk8OCIfj3mrb3j#bSojgLE z6i|0S6<@`?gsp+oNppfDze@2W4~e0HWwlHh&gL_#qOKanlMZZC-wu=Dyo}8+UDCjE zvMg}!Z%u?$b5Z}$Q%||`h&^OZm(zKaBPgN-;t9s7Nye!~@&j{m;nv2`@p0i+CT{&O=wr7cR{LYQ3mL4Pz6 z4Yt>D4roO}@oPt2`eo<=6tpbCHxOj96{kTG1G5F0Zr)?fJ?3WDfTybMa`+hJ{_-!r z(r_y0C4plflxXcbYnLjiV-G(x+asEX{vU@anMo}Kn zJ!^?Nv3I|Fqqb~+v+ZYTg@bZn$R7M~mwmfZ^vr`gaDIXHEF>_ydlcbjf@J0+7J z;w(uOP&04ERp9*q7c$CQ?;XAgG7vyB>j5oNnSuQ%DXM{5L3RerFGwBri zyzjn+A}3A#4!h?|){0Fd;*8{375-p(#K`(Bz=orUEOV*&lqVWfF$#y}%IkL$_1k~rO&)usY6C{^yW7vQmeCcs+`J4VZeE=l5I-q13QwXp?|o1%3H{y+_x>^46wL=9Vu zJ`KeYK~^Iu`dorb^qj-koEw-4-<(+eyW?ZFOw3 z9{*L?|Z&`&v(ziJ!^ zJ3VrFL*pbg;2xspS9!9lWxnf?wl74F6EYrQV1w~k?k8A^z*>y(LeKywtpQe7h0$42 zx;ImabW`Etmrs9NP&w$VS<-o~08m+g&ne-kNZ{fEd1G7ZjRH$7%w<@Coqqc;53^ir zZhngry5mceu>o6Z@w5F{@h_}NVWIOnGFHjc{T1U07hVHl80Vcj%0y#rQqMGby;xpr z1`2k@b94s2S4~(PAIMKMt>3>~$BQa5@Wt!r5*@tpu+odW9-`+m&k`-lRQf5pUpIp< zl1P;QA+_0vPz7pp`og(V){2+2@Mk&wMw_OyA!QA9jd0%nbugEkM>Q78VH8@f>!m@2 zzqq#UKh(asY=+>BO!NY%{He5~hWtPT6e)()RBFEzSe1!=0q7HWn%ECPXcwWg+PD|l z!LjKv*g(!Bkpx=uOs}4+rHb*5_RmR3r7Ln<-e`q{{Q}w`aKYs{arFDP9HVv6&}?L7 zmUAx(9#*{2s^App8^r3IV=PvLm21b$7GoFbF!hn=GHHSe6qyEm<{dQ{IC4q@XOa1% zB?;M$Xw1+$L=$k>dO{Ao!nk`v976%tzsmXrQCeQpDa2FLS`!L(DYoy&9NXMB`(L>R ziHYztR)9F8ouyQHBzUVhe2^Jl*G)}m_CmYE` zy%%JqwTfDzR*YXur^4Yz`C}d!4EJ87C#0n#yr!mj0jS)wDd~P&DMqwsjyj$-zg&I% ztK<1E_5Q#4`3l1uxI&Be0kQ=&%I8=C37iYe4hjrR4-RB}fJ_ZkGY`(ltSO@&!xaBu&a#?`om!j1Gpn$9ZamdL21re2c@y9@i${C zyxlCOu?Mj1!+Zg>roji|c*%q$W9j?lW9i4~iEIOG48{{7Szf)SXM(rQq1mB=Ay_eG zX{K|O15Sgq^l8_;Z*qo>%?evu4MR*qRfj}n#f*tSouwQ59Z`=Eo3jt!$GEL|9x(+XqW(bRQC~Jz9jb>4sv^j3KAn-H5 zBw(r`eVh&}&Rq3rGvV)ah)>%p;fa>#6mR0O{2IXolHREn1c70M{$D=lmeAu zlWD%a*jBCcYfl!N!9%OZ@-}w;B0lcE8rMe|m>i>}z+uSADBZF&qSJMVD>;D(2oEJ< z@i6vL-5nN7EckfXZm>S*n?hP^lM`*p{QZreeDfme^dj=4OzC`v;~v;OvY8&( z`DFst(%CvSMazT0W4amZ%F#DP$)moh;K~uAhor3>VI|9OQZZ_n@`6aHfg8@OqfEa6 zJkvAHAQFx-t;yrl)%5Vv%q(dNiR&IVbbWbel?>X#sFb-MOV$-flJ!Jc-&yq>z)$QTd zp7c?C*3_{MtAEk!CM)s3XWN`X-GvvHl!pum=V8BR$dt;>?#)6WF8U4M=4_mMNg4)5 zT!jAC8k{f}rR+qMX9Aa}nv|!`Z=^zQv~SP9q=TH$0f(+sfQ*>JO#yw|>pt`W*)+lUkK9u6w85=3P8l^IPj|h8-BbO5J^%BfJ!s>+O z)}Vxju%G5xKG+j3K=D?o&j@>y#S65%72HGam9V=4eFJ#c?0bU(D&7Y4Il>7$1!tX^ zA`$ny_I~(DA zDsw|9X;eDHlrzeir%1`I?APm)U|_#Z?h3NNcO*e}U_mzh9_|p{-XU@oIe$Z~wn}Y# zfIcM!Vo>RA)x*pzDs}y{GVZfG&m*<{6C-4t81RXfqJX3*jv6gFnswx^IiJj zBTy&;pYtkKBAVu(@oT=q=%>fL`Ki^=T;mf@Dxx*|ph_wZrC>ae37UcL<02&nUA&A@p)x+=xfx`T` zE^I=Q9QjiZJPiRVK!hR!?RtEgYoz)aXuzTVnQn0&V0eW7&(J4`kPPbIhYXuPWbi&W zzxp(*m3}_-ZIc29WF*gz*Rp|x!U6H0>*deiL zQ6L#r`t0g(AjaBkdU|HS_r|6)6eZ8=Gl@XTKS~=-*2$dqa8Y_V^J&}}9^nd%5&Y`x zXH5&T9}x;4-<8%|d})BANn{+qW4F7veBXTY-aP)?l>uKE0=v9uVu`cS4;gBn>AR_0 z>wpn+F^vehYWWcbyLNEuCX)zBn4uAK!Dnc%{i8**a=u8S)9@~aXdJN)SmO0!_2TJK zy97?;w0(dJb5UlATTW(}-as_wsMHDYee3(!bU}tu*I+;Oz80}(4_pB@rsEyeTLa^x zCoDgszF$Z+>LC4q{xRo>z3xfAKf$>TNI(HD4FjTYRy=K~o`}-O4#(_C=HINt?YDi_ z$1{%zS9kH@~SU+YV!Ia`xGmZOpOt)T+ok?r=3)FQ}|i3vR8 z5W|a8vqp?_*$a{oYWb5uFl->?8=#SwIBq^0AmZDWno6A~$EcS`#lh%M8s2r+YPnc% z-}YW8P|wJtnVU;(8dh0w{}5Q#d5Ld9ZMW_eIl!31oXwj` z2O^aps7Zoinl<|)t4!1p&@b=wBV#eE>k+L1i$HMrV``pr^l1bJAF=&rw5vz#ZX%pSC3U zY7~CPRr>iiwgsNg@|x-)Uy;6k+fj?y)eY2(kb{b+e*8n^!7A1sF3_4Ti!bWXNl0&& zwo-qHScr^iRxUjhH}?2 zk~KLAB_esAKg_=;U1rC`WY3+h9tN9i%VWHRS>WVEks$w*@5WTu`YLo@S8UC2J zTDs3rfBe?3rRE8!Za?G0NYfh`dNGR}9V-{~Npa$nkHLv9BmI;I9*$rp)L6>L3>a+X zYpu@l22wU(p}dIDbdCF*!Q)71xbg277OssYzUC0V9qAWX?g(GfyrNBYM}lPz+B8tq zgLx#cQu_s2ePt#oYB_c$By~up36F3_`<@+WiDB0Z>l?cWJ1CIUyN_s9Dfz~Jm6>9d zNz)RFeLn|9GYPa$YK>UlABJ>y`W33zN>S^yGvGfsA{1vK217pPiGCXCs3TI3{mLFU zBXe3z(+)+X6CD3M8;m&&Tbwk|odjIyw!o)JLbQs9J((qO%pJJ)njmaH7$6*MUiQ-c2;7eclquddELhR|&p9 zUW+lzfn+u&R^_y}2U>%rodFbHArQ~hN+#IK^OBT)YbQEQhZCXIc+_nvHxr=`^rU=y zn%`|CHPGK!>}|Df%Y#LS)2sQ~v1F1u?R}~gtpB z5C7HfT)k()KLk#QSy>|d>-1iv*nTC!@p_Rwdk@Hd7>H6Wi}`jn3GX-$&^mN{%)rhA|tw1H*@a?C0p-qz2+nVsvRCd$s= zS?sQbE~P&*PPr<#FH&||W<`}@ka<`mJ$Lt&y|96$E0cQm4D_TiBpB3b5j#tli)9?ft94i> zZ8`&sD{{#%5j3wu%#IEyD__+{5(^v)u{R3-!V=k?Y!hvq)it_GGf0)FBboA&q27-* zQ)pxxkt-1n{dt%YxDo0iDeR~VKD_sKOyp`d(ye$sS#NZ~<2)S@4szzpEyO&!%1*gb zbVHUpIg;k>s`?H5hY$VtZ48F@dANgwLI8tScMaSm4NVpOm9&ZU_VCQ-{6`axP@YO?HBT0o~5EoJdZE`@IGzI0^eWX0liobq3Gh3(BM}6#Y0?0CW5i>r~UeB zY$rGhmbX!VaKJOKZYCcoDigut*5wb>K-)HT@#;rB3Oix{WME^ zDBR;^`3=Ex%rvmAUMZhIm$ZqtzY4*qXD#XI4JJoY)%e z@Tfs`k|Q?nUQ4NV(p2&1YjVuT-&kXx9X6i_VYPUuVs$Yq8DTh+2&D!CH@z~rFh2zW zu!Z@zTk8yeS`u@c7KLBQS5zAj*bfxZ;16i*hoyT+(-8NR>#RU;j5t5Hc=g;3DH@h; z=qBY`|8g8uIVTzH*j8{uQ6IN5YKEyXiZbej?XJU{?Ld9`9_ij5kg$YCRbO7|jsR3j z*8>k!8gz}g8v>}n2)VB%@`^k=kykQm*&$FYeI7S|6zqeTOO^76VIfR$z8HlV9)=wR zBIl5)0{3SR*?FLIG_SN9tP?><=nE|brC|)d`Iosm6W&puU@=IavF)#pwBynNo95f4 zvdibp@8kM@^>w@VfcpJD<>S}CYO!(%3lC@(--oMJ;l4}7*|=aR_s z6xgZXsCac4G1It5{U`qZzrgwA*Z&Q_yU*}bt8ugw{3Q995FjT^00k&;3mh6$zek7w z`Zmi?*Anr8r3H@i&q4+&G%xwj$Jy{ed#1=ZAQeIgG|<_R04`_?jS>ZzZtIGvi97NU z(IiWNJs_z{`&FXJS~s(Kb5Ja(RGTrs6srPfle+B8q+QRGhskT;4RJ(yH^S*wk-vxo zg`?YbSUZ4Z-{`Qyx7-Pc zEOLS|iy+0QDRP3CDSK`p$q%|l9l2g5au}K@thARiQ*LI0gt(c~ev7av`YZvv>xP%m zsWLlZ(_h{YMsR~OI?YF@h=cQ6jSx(HWwg+03u#PsT4~260;bcT3X7X2b=HZ-Qro+m z`i3E+Cc&UY>xr@T`KKET9L0{WX%5XI>0%ypPaxRFWHU>`<`6#MNV9og3!!hrvQx21 zYRo)Q;dcg$HTul%>=f!HG!I6h6(VSzN%S0F>x?78AB-Im4@VEnmwgda5#;Fe$4f;| z+?e{`R)TGr$E+uQl`t6uMaMtiN&6R`^@%JPL!*H78%c5KkKRp&Soo_RV}DZSL<5KD zK{ekpS)@6pK-OG9-+6;QaGS1cwRNMdg52_OV)eTV2$)A!GAQhF&eokzLxGhxcG4*Quc;7lm#I4gQBD|@KUJLoZc;1Qta+#PD?h z{6eNiGbeQrZ5sL}nbI;{5Q+vSwYL}|)0XrWf*Be@dU2pxJt)&A5Fb7w4g(~Y`YC?2 z4IH0iWsAlrkXvGgwkZX-L2!hkm8&MRVciV0ZvxNr4qWt^pOSw3rYB>wcS-oVlkdR< z)F=%QKetf6xl;gitkhcy9cu-SEoLr5mz*!8Ba|iPu{V{NbV$>Ua#dDD*})5LSY;5_ zncVoif`^UrKo&qaUUd(i%!m7_4|Pwm8h4Xf@~?@V{B>GA$cU%U-I^3Id@3Dq4)Ih5 z%$#{Iyi`Va8O>zHTWMaN{O;^n{9!Z;{8cI-b}EE+eFvpz2!Q2wRbJdW%PmL>P+@tN z{rP|*!lt{uyJP zI)@WcO)M_MhD49i!q@Y-2^b8R>xq^fY>HBZ&nA@GQ!CI0Mmr^{o8xkE>_H;Er z!{ij2G%pQ1R@eOt-fT9$_$Ymx@d?fEJXx^A!)GYCJQt0Py*2yxbnM)0cRrtU%z%Fl z0eY#1G{G*N{?11Q{xJ-Q_iF1Lh!D8LCb{n4GTaO)Lb%p+nv8gJ((n7@B1rZH)vI8? z24l}&*z4AV&o`p;I>#`3umh`e1NO`I!I1Bru=is=?9GTcu^$+T#w{7iI4dbQtpBZC zx$kX#=Pf#kaBPCtovNP#Gl}MHy2RhsPbKZkd+58LS|42$@SYCdU$?Ie;3xIXp$?nJ zB(sF)LRW!D9ilL#W3I{i9G<}>qkWku-kFDSw|z_ocbN$yV6|#f%2i;oDx&9U6Av_G89V=D;8_-N8+i%lBdQch77YNml4kfu<4KUHKUcR)X!(;Db7RwjB>{4i zR~8K$GH1W%m9fSqn>8%SE*%!s>1xzzIx1^Avx_yMPrVehjP&JoxHsVC^Ql?Zu!H3u z73w#6&+o-e{Bs5gD`WZ#U4AE9xEE2#hdk{E??G1m zU*}A_3m|*4Kf46nNu=%{2WxWAskY=&p33??Shj+J_{TwOGBssxHYFwbv-`tJ6OnbJz_upo!TG|7W@{l2`X89* zmH@$uT~~z{1YyS>a7(Og&0FYFvj)?S$MT!VLSVm;A2%JDz7yJXZ-hW?F@iZ(yLwwn zkz1TpD*l4+Y1U+v^kGnyG7o>pX+C+sAG&PmRC)K%_cdSd?!ivE_PlZJH#$HSnue3o z&;qU&Td<{uUOy|B zBXBp1zB>%3Bz;FqHQ}aQX1{~pFhY@D>i`CNNBs_?dqyKQVd2wI?U$`j-+Tb;rUzg- zh@js;)w-nsd>K~8de?~ml6Bk-za~F4L#58LheIDg=lBj_W);(T$L~?S)dM&V-e|lN z<3M2>Qo~j}L;fw_SAK4c?vX+K3mAC81E!bD0DuDwYTtVh?XrKXfA&aLAZLr?32~K7Xm9G`F$I#LngPp(C|k z?ZZDWtDDf2y49-D>*kMP?zxI;x`N}f+rV+aSY;Qoh5lX%TB^Qc)dnRnSkH%|0=bI| z!~0rt`^PdFGS%aI&+j{b&rrH!^!QNUGM?=>+*;J4Qke+Jily>EiJD|clIA2ZVX5w} z1pkA|zC^KOcZ^sDrWA%IU+-q62x#3gUXg8PpUJb)3oYLD9^OztD8TAI$2NYvXog5d zQDpfCiAivIOyg)Id-e;IB`!aNYxQACRJH%Bp=A^JDXl~~+wu5t412}jNBjyRs433E zho1QNL;#Ze(^qL`+3IU2+v1vnpKO%%^mRy>sKtoR0I1in~F<7x%X*!^occQFis0^IjVT z$)S^?abs@%`^lO^ZQ@f3wQ?2k5gH6)5`6T@oY-zeo2YVv-;rT8Ofi2*mnYZMEXhqoRFz9wxspoX<@x6JJvebBZiS!_cQRea03 z9kq+)tpSBm5NY=}e2uqR1f%VTSj$}Hz`h{Nc{d0lAgfQSVy=zmo35y?to6n(oWBu_ zDCFRN=Wj!W?O9Us(t~%bJhHKefxP$uUT;aDI`Vc{P0WL?KJ75f#0|VA%<4|=xWs)< z)$qKR5QDCq?q`FtJVJ|>-o>!X#tI^9%}ej?g=v)?oKa=3=MBj=B@KkJqlS!Y_NuOW z#_3Osjf{^N^MfayJjdC^8ze{B7EMvvMt>%>`f^b1E{y2zskj6u_skpnE&dw;PfKmO z-Bgt;pA8_2FYxjF*DY|dje!C38hsMPia2d-+-vZH6&?Pgd|I6K0}LB;@=NfL+IOlD zmF%+JLn2LZ9B>>9XGr263|eBT3M0Hh*|B*rC0?5?OZ4(z*+ZJ{hS#ac-!dAPr{2cT z{lItN(hwHCA##!Ws^DFTx6$a#+{`GVn!Bnl7r+f7NfGjf4^`v z4I`U`eN6y7wnk`uW$AxxmA*`+s9~rPG!wY6(Ct8%x9U-|lf=tB8#d;@y0AXOZeQhI zZ%d_VF~wQWpNj)tA77HW?W#U{CYYis9VmKX_&TG!vBv(n>?*qaW@^XY;RFuzH!S3t z2L+y_eMpRQ2;m_b;UP+smwb62#llfdG+5-0*f6p8jg4*s!-_=~P88QIrc$+^FaE|mRvv@+s zbv-qY<^vb%p0`jUb{?hp-BLlK+48ulozNPsm`_Q^s^NdxCP(}g@JCM7M!W;Q}3s>sj;4zSAh zz|m!K#3;e*4+&^!%*x8oMI_^{WAz%iMV%Pret`eO_Wo~9_y0eqx0Ewd8Y2FC5?`Gk z66X25Vnc`irye9(3t9w6Fa+E`XLLii$#xL>8KEukLhG}P-CrdPT$%8gul1rebCscB zFm#3p+pTFbNSUOI^0pBfnC$!+1V;lkbB{}?Il?pVCF(funCb&@JQrasjwX*W27Bybddm`52a_MaeK=?u^- z>Xe%5h4`9}DP=e>e(A=D|y zW)l|`bLo$@Tf54uahv9kY#8a9nYX)!0wcl2QG>9ey=xVP-MoSfm)ou)PsibtFEiLv z#DbDHYlJ#H{4A*vzR@xqrrB>%O?Od8J0MxKa1Oui@S^T_n3+9&}-yuf; zjq}H?lqrROCYSrF1uU(7v-%oF#@kqM&m~<3$9{=udxj?%@=iXMq zWX56WwQ65U@H6|^A?;?_VBT`y4}Z483~FtF7P2GxDb=0If%e$OkIhn#O7*yD#RU6T zE>nY+W)?>I@?)FI64gX~>yuoaD3!oEdwijAYiV`cpNQ27#k^O|IgKg-$RnF!ApJUh z-jZZqG`EPCTr2Yd>vpm;6f}H%Ecr6tbf>E2cW1N+Gl@ACQ?L;BaA#elqJL`)qq0#S zKrqYEjNWmnCJFF9a)Ur5-H~#^IY4eY* z{AC5*etIgUUxs3x6}|W3mAotgf86>?YR*?tC|Y+?GzKzdGyEwwBwaeoq((95WL_7~ zD!8p!=Pyu#W4{OnlUJ)c5=V_uc=g2feSnW`zyS>0heQ6;o4@Kbu@yMHQ*{O@`Bybdh9=hKOlcF5T^x>sa>w1`trDD-t9 zxkjY-2Ux;LWI^bdKS)M)^%Eb2K|6B~SW{nS(@;mO>~KsNgS9-cgf4Nn%YG8CQ)m)O zlk84Ir6%N8;CMrU&J1LLVJ2S+yuV>4q3&plWXZx^ngPC+LtvAC$Gzg1Td-0NRP2IU z-XBN{6a6s+gJu@u4i@1Qy}QqeI@=f%ut?__EZiEktb1x4ENlSl14XhZlHfyz>)u6* z1>Pk~>UFOMHQie7qBAaTUg*67Z!)S4$)+_@@5*~Kzd*I?op~P*)c5E^u@`$>Y4yP) z^B9thO)+PPY>1$>I5=!_%vx}p@M6qPs*B1cVxHLDyY8i4&$NlOn4PvTD4c8X3M(%X zTMUwSfEv6+P<`E6gQAXV&dg?JPk3iZ@CzPY>*Z)F3;!!Jf+krTvNy}&X81nyUKaeITssS)yl8x><$6XQ(E&709cV>k*vzO z{fMg%4fBRyDWlq4J@#mK$n4b1qV_aOrXFzb>;L&}oT1N8cu_y}om^?YF~*|np{FVG zP*5oO^v}Q2kbf{b?lX^Ci2A<=dM<1p3H;9)S0qq@9u6@mu$vGagcbsU1Y&-JB>)=O zVQFIhz235pa47pC#>6Q0V(Z+&WJd=GGx?zxu-*`;mahxQnBKlp%9viv&;j4^yPe}P zl!=7@r$!T-8Z9b)RN)wI>|$y*Y)J=R`j2O^itOwp{MMd?6(}(naO~M zr??hyx5BSs0595jn9In8%ROG=KH#*?rI;s|$H5%0PrY=dAht9MyIP6z^rMz3$>YQt z_{%9M0o+=E)*f|SM{9@-5P?^TC^iGnW6C#d9WJr#Hs5^v$MtN^M3`mM?Kqo-eaMeT zyu^sL>lh`Rh^i#m;e>IXJv*Cq9(pJ5QsD;R-DUOi$K}V)63yUTZ-~UIJ#f8rXP};= zjWco?6G@JQj(yQwQIC*WDHh==UDzhk%zK&5?IQF#(}f5gfX6uy{THkkF+PJX6PpRQ z3487m>$Hs! zJYErQ!*Vq_n*-L>{z3Op5BNmX;w!N~Vg)kR34m!310f(0;J}coh2{t(C7=xTO5xgs z`)%?1%jCt%XdtfVj}QwV?U1co| zEFPeLq96lYQ6s19cOO8@UFoa|xxRf?bKKou=+Yh~(orMwv=yhaUww*Vak)w5n&5x)p<+}J0is^=n34WrS{f6x!*58yISD&^))OLf-FqHO*t9zm*~(; zYbj75vbK$2trNGDM}b!LjT*QZsGkbPLF59- zPuKiz!)|C9HSjP)=qgcj#j0Ep$jGww+TfDfAS3wd<=Mt0w_c+W-AtZvT`O`kD2}LM z8Mzx+AGzufPGAx=Mg`rVnq!bb1bZApxU~kmeuQpttz>Mey2>+G_5KPZj;>kLyKa;`H_Hr4iG?3MAiKWVWVEG<`0-)Hg&Hws8kj| zAPthpcb~dV6}W`dAOjp8z7L@>v6p9xT|qBSr5g>$10Q8HuX7P@J?Y@7u$GZN`@*p3 zFC&*>Ab5i`1|jHC)JuDJLdJJSU;+@C-o=~l6Xu4#NQMK$L*8A@UT|;^jQaPC`VNc^ zsQw}w?(OVje4y0b-P}=sfb%>jc!55!RMsRZ4f^>h=^UerZK)l3=H%G1C)KPF8+kJ*fF<65#k6%PJ0-U?b1<1A3b_X9=oEn_iRv)%F2OC%>liYI{>x^MgL3e!kbS?-4EI;wswpKjmh%)YTSu_0X97 zyBM`yt{Sq|x_-Q}WS>neo+*^=ysqI{M+BTcG=8aEaOKAPMo1>Y&gXm0cf$A7>0tQ& z^z;R+l$7H%V&IT7d5jT;HC3Es`tTN9*zB6omE_0?}HM*X_{bjFzvQ<2Jf1$HG+{O447p<;R&XEZ+GwwieQGIVCPuYF zk9_tvj+wp*kz_TbPG+o~a={sEt;FZ5aIrylNMn!BPoIsQzRmLNf`*GA!o4;T`}nhs z_O{r|iDzqJ+cV%G2BHFpum<>aI_K>ibCwq4I0hblef_I2VLz$BynoqADjn zgrl7?4+!R4a(`{_T-^e6Pt#njy4FEs-nVL8rmA>{DoqE8Q50V< z!WSW3!kcc+vow;iH?y4X1!F1CAi%nmIpPii;gyLb0iHBa4OvW^#W#VEeQkdfWdWd9 zqGuJ__te>NPbyWtcd5;w=R3Gao*zysq=wC_nFr z>zKSGFgQgPge8QUO1(@bN3)zro1-t?@PMN>Ba_?PA+cbFEZB!$7(z+mOE}~!GWo+u z)M#CG{>P`_|G|tw=tmH7P}$*MKE8nXLzsE^%|DPWGNwOD^SCGjn6Q;w?AfcwT3)KZ zNdn)#W3C1=^^)GtPK>O;zc4DriW7;4pQ1_ti#dUX-vfpz1NxxvKecK5!~x=vJ&y8{ zj-`dFz0PuGv0jKm10mvF{$6<~bQsvbK(qsQO5v7Y`SC2|Fh2@!ln;#z7zK_watOE$ zRI5u?10ZNj)?~8_kApHn`Pj{0IGzbT7c%~Hl>Tot|1S{WPE3A*{j?0oBZCqtxyV3) zjErKy3X`8IG7Bjh+)dm+0V)#~9N&ko9A+%!ln>ebU_U?u%0eTsw*`XRYm^&cHfmZ^ zxs%hIt~Z`NzLwXMmp8!odxAg+FD&WAc(V9LHowr=U^^t^Y?1*^7qukNRID8W+vNTM zt#BW7#l1C_>$AeSf9*HJKFRP%tIQgW+`R^%1wUUF;@mWsdxxUsL~-(}9m-Bi6g@G$+zY&^mD zTT2O92-(l+F56k5J2KxO6AprpH#bXvJsjaFOwYR*iY} zkLs!3t&B#{((nUUlmx73R+}jHf#`wDs_OLx`wVuKtNkko4&|+3jv9V3H&%GC+0Nl> zeG3vW++|VDmnPynfKBsOtkJsQ4J2vKMygh(xd42NM56li56i1*ejPi+ZV6)j#%>xy z&%mjjvhtC6y{&>KJJWXKTWouZm(F1Rfz`~5OBd~?x;yTHklu~Hf~3TAS0!Mo8T9Xf zRu^T|Dn%U|=edJ%r#C0{zhWtwjGNgd8=kIJzkQ2E=zJyJSQQ(+*Ka)%gHC(D-#M}cVwJNC-<+~*hZ%`M+F0jt^B=sxZLoo?MZI1) zArZ!8f21_;=)^p0&S1<%DH&L|46m4VOd_VvIN9C{SxzZs40VHQd>xq@Gwxq-mv~}5 z@OY#s}S{TEvkUb_=2JypwyO$6Tv zG2OyM_gMr(1yL~{U_9~*75ThoPt3=k*q46a2U!j!E&83RD(rD~4LrEJhC_fzB+v`j zUu4htG8>`o4qPasG0^o2pBTsL60ZpTUpG5l?v?+^x%~?VfPeAef03ejg(L0%r3rf+ z!~a38uu)$^_<57pXGG+ubhc& zj-CME3qpS62wnD2tB#9xu@g<4sshq3Gc}V!iEMU@NYMQckfWZwnS z3o!bDh^k50$wuGam)uDosMo5bcp){X!(}V)8AMiL%RV{9&gjwNLtWW&jbaRz%->AtoyU5_vJHTpqiq<};)gTtBytK);7Lzzfs4?Kp=tjSRIdG!Pt_QG`fDUX1}W(Q~Uhll!(M0 zO%?hK>%4pxSy$&xbod>HZo2WPyO)*CndcfC@1zvz0GWqg>l}SEw{bH;({ivOG4lTAm=cjYG9rk zhnYfX{K!)LFBe--JRWuXTg|8nV!M6#p6N!B9J}DlM__s4Hk4Y7W}mfm+{5FcfRTL= zW;phnPk~=Fv*Q4B$_md!77auX!h1W?S*0(JiXZzq>~%YNaZp}KBI+ytQiZyiYNF;I zacl0^gXkLxRLQ%Yn$u%N`RL_CX<*S*{W8V#bW7nAt4V0?5>~lLq&a)(1bE@%{R8>P z*_TM9929hIv1+O@9F)HMg^0+bFXVsYCD_cY6_dzsi3XYs$uroxMF`2KMc7}hNKYrh zglTw-Qm1-g-$J1hzwU8N4?4$Y#B&CG|2u=Sd5P|Tt?eAAQiqvG>O=7>MNo6o&NrRL z=0W|XpNI0xCm7Xx43@3uORHdQ*T{TuYmSXdv4zK|s>D`e|M~l`#qPg<<^TQXn~&(7 zE`F*q^}+x1j{H;>z=B9<8L>d|_X6-B78?N=khCe$^?x`tjRs>o?7zX$nl-XC^l7`` zBKoH}5N!*NKI8L$W{mi`yiY#ErUDjpcLE^$N2`DYntcF=0XaAmeY!I2i;ijoaTY7> zt!3#VH#@&E1!KFxlpcIK-+m&eh02Ph?(7=v4dJ2P^*>)GxCmyEvqo~zT^-szUgkFL zcrvdU_4E*fjUGrsv27-kdPH z>4FpzAcR${rK^Njx+iQ>PMO6aM6#%nK}Jc$Iw&7bht0jB9ojXhZBuN`qQhU8+Gaoc zQov4G!B*~CN_Qz1ClH@+T}xLVMP{{7_BqFkCbrUX36LIFRcuesx2vDvJ) zlGohuVYwTF<4zj={&)l0+b^8>?fJ`Zph#A-AI+eN+umRjH_79K4k6^;2|;&m(KgLA zJ=J?pt40&Lh@SYHf+WMbhQ64-q`NZW8mq$dtzmt6F?@{S9P^B8Ur#iNQU=pst-wUG_?bw^-yrj}J~BeJc@RWxc+vPp*I zMjC5bazIVm%;g^qoj$^WRd3Vls+0u9IpvjDgjWyJ*>!qfqWnbsyxKxAp{vvUB_$t* zSwR_7L7{rYwCPLo@7vZVk{u>1U6L6oN)(;X0U#NeB2-k|`1_=!eOpzY67Ysqs637A}iT3Y|7C)Vd)@OL7 zV}n#7I4MBi>;*pgs~a3X@J$TLp~b%#RTo-)#(MfJ;ZIG^6U*6{_1mB6*v0|z zzU%fACnMLJlefRyH%BBNFBdytZu>rj8*Dm~iq!GgW(VJSiAKm2fX*CKaeY&G+xO&- z+}`{H>M=YUo0D1xj4p!XRZbdM`&J^&7$>cH2giauznm8jARK+8vRGVy0RSu`iMtR=WvmMuhUKsvICsiFJv3+gaS1AB~0?iK8iwlAn%~5 z@S%=Fcx!WH+wP0V)HX1L+v(kz?xgsIUYsxL(6)cRARx>+^IiX{ha zmDB3${aOEH@nqa-GIb3n(mLK2;gT!N%%36J7&4~@&VHsUpr(S0=Hw4KXpb6-G(>^R z!(!paFH_`$_tz4nZFeCMsanRz%|Y`JsOG1UR8wfST}|wgNRql4a-AQ3Qq%n5_K6=| zbShb%ZqkqET2e_0I)ELlaudD?Z-a035*JNSUu$#d?}_c&?aC?VvyIKIZS+ZZ3-0eO zbW3u(7H6k^0MU@6vlNu3WhUV3w9HD*PO;49`Srd-dsCLp$N+Z8iC8ES0j>JN)I7Tk zo)WYThik{2u8gcI)CxgNL4|2$Nmf-=8?A{??~9C$MXE4W1;)oiWd6&jFRyJ&8A@4) z0fQdN`Oz(BDO4x@-1dwvGAWE1d%e!$)?z~~(9CI>zz(^TwgIkemjE6fA_yU>gG9~n zD=}t@1l;Ooq6?xmcd0LBwyR+&;W=Kh(p&V~bGJl(+5_gDRRtt7R?ox<@q2IhNmeVY z@$kqrZPt=m;Uv2e__e!2fG_j6!U5HL^ulPX!@G3J_uBM|gQaebG(4K$fCvrgy&*%2 zDS)+hAoRebT>CY#lHQ5i!YKC}+vX)z^s(TYhf!mA-fr3EyTZWC9pdpj2tQybTYsRA z>$A*D2}9u~kAyrd=4(zgD%U$PaQ0T(FMm(R&Gs1+`>PHRn^)DL62e`Qi6~D{Jf5-s zsaB(tAe^FPCO>N|ossp)p&U*B@-g54k3M<>a7iRXOn=A->UAQilr{%P^Roj-<4I?z zbN!$U2CoVR1ax8U*r;E3i78Lm)9<$|>wZZ|J-nOaDTnndHpR+*!VKi254UHZTUy@? z?cG%=xw5!zo0gxgpr2oTYBHF~J=jbN%mzF1j5v^Ff8;y`MDxyU=l%$3sZ|f%VuEb- z1lqaJiB1&i1;?H>Ho8e83NG`vvezY7UVlHAC4qOT=d68TB0AbI?URi-_x<-oZ zMI+|tBCzGx#!ql(uFN86Iyc$!ObIJn$^l6T6&C<(P#4CTGpIy;0rOcRtGXDNAcyLpPFPH-sSnOX)bb_=6bn4PNjvR}YNzP?UeiAO5+( zODYd;BY{kCSlMt?iw_1%mu3U4Tu<>rB<`U{v_jX$u3aN{QkdwF_Kls26`y{$AaD4Y@} zfSlxOp;%8ti5#w4->bgQ&Twf~(XrYBpPvL5f4zdFR}vQyz~w#W@uNZ1c|v4I$Uoca zCB{iA4(3a^Pc&e;%cqLIcLai*MU)HGcB4bLgzb4OsUwf2QxU?oz+j~238Q^VZUjcN zxM%!<`pKD_51=~Lo>Nu0+E3CgC+Z6e2i#EDIauiprcY^TkTPP7wq@3U@kX%my1AM}m4->c1Z@r#&Qggl4pZXn z8gYhBR3aAEx$I)NC!Pzz6rJ?92YhBU@Fyp5C#?FvN#0F+Zhztky__TYrALN)uS?{) z&L0!FodOn0Fil>ABYLF`UFLxT_ZY90?S3_3y^4Ts*-zomlLE&%E{Fjkp384N?QfPy z=^rJjge3qBpriK*&1-ABUs>2k+{c!}=WgNW{ha4%AODj4Cj;X)n%_1SuuR|{?3Kv* zsgLosAS>XY!2aG8={+>ytU&NJBck)<*x@xS0uUh6hr05Urv;0Q3++PXvTpwrI#lRx7Oa})V8i- z_4U~r4pp=|1~|sIE6$XvQSzkekw<$$TItq~l$pPgfuYfxTCl>5IO*|}papvD_(Zev z&{>=ET_!r88!<$i^~l5s-J)4eN|4UWNX+9H`4va?R-QJI51@*Ru@6wM$tO@)r@?| z;FMl|S4i{!NEglN&vMEkEEjH1%&_Z+{?$p_jc}Mb2`*Vgz4wzwnR#IOqBKbajA9CN zqK)OxmqJB)0e!kcNut%HZP6zF2_OMio1Ac59CSY0yt%^sdk*@dEZU-Ogmwcz>7dOH z-~Jl58^m6LFs+&`KYv@&#`~$xwtPA<@;UBtO&blCg1pFy6)7Ip79e;<#VlnU16JG- z?2Btu$Kqr@92GCUiapJJ`Lj3Sz4R)oXXAm+wlR+Dd|ZugiW4cJh;pMmv> zDp*@oMAIp*Np8>PaR}$U@R;GOI zkPuIoLiwXDGS!?)pW(7wp`Ae*I#?SW!{>y~B8NgzDdslU*TJ8njb7XW z$$};mRlrbPkJ?tl9$od*$BPbDRT?_s0hrzXfKKfy4*StPzm=;apfNQH)!D`JUAjSK z<$!cBCRg2Zmf}!UWcg@{5j?`y`B3DJqq6jDm4qN5$m#5k797ceSdO=BMAP?)9Pt-Gh#36R^f5pi_} z4*Rz2?)6-_J0}dkg_avcy?OP;_@fF%Fr_@Z2om3j2$Ii_ofZy9tT8zY5joui4of}5 zMnS!l$3&0|j%u;; zsk&@=_{;OzX!lwE6zLJxqw>_4jWX6#Km?cA1;cZ^2+T)D34;*8n@b1FAHjCi_J5=; zy$FA^R0sT)tQ*X#Q3L);9JLt6ANJa=wMwYc z4DhK~sy36i2AwxQ6DJ81co?;apU)eeSZS%3A^<#}$#gY*Z0sR^qub+0&AuA7nm+Vf zt+~Ci*?DKld@^CS;7Z`B!rHhSmxM|<+tIX{9u1U@{|r}F@IrCIV~xfAgF4b~!4SX1 zfVGH0|GcK+Edsf?{K3H6ITLxJ7omxd^|<0!QK%jPn%Z5tf_{AXw;P*O?3_i4>i~UA zO$jhrZ@zTW$e|;ys)@dZkEpOVrG*$WV0$(voXK@V=SFr^-pt!H+s9)3sZdmCt*HWW zX7E+ZYVr3~r=ydQsKtg-oZQ^c%C6x`I!vOYanlr~+e@uoWnOT?D#@u!o7B1$!*LfG z-%AgJ&Zvv<>WGa{!h0*qT8IJBEOHO}1}GDS?${K1f%h|EMG$&3m(q-GbLl=KEbDPY z(1$zWz2S(ch4LGOg!;CB5ct-!_si&Sc+EBo-WK-`O!U%@Oh4DJYvq2H=+zzY9jx_h zntp4%Y>@1h0V?7Y! zA)jO8dJ(R*@%#|4|3E^x|Eg+h61LC!ke4KT0s%OWlw4cf-uF*np;$ zchuFLdc5>N{t*C$%r$z*a!!qXv*=viaV%(?(`9g&j>K2~o^W^QNCLbAapA4~(~&@a z)!U}M6S7BFL9$>+gZoedm&c+RAZ}_v9fL2^xG7gN{pP}18?xRG8!j{eQ@Qf6fYmd&RCL{qQBv`S-M?cSw|^b5I^&`DypfjjQ^?J8?kZ#~|s9_J=Fe&AC#J3zNMJD2HfPK@H96;u(R{JQQ zfRWl4Nx+I`a>y_#?l;-@#N(f+9YQ3rc05AYkbe0~Dbofxjq3P<|B zF@??&fbr@27<7woltKDDB6#out;fVTluI-6Aoh>Ta|9hD`5hWbxVi8wJm%iwXwf|7 z?(28Op%@m)ARQx=DHWxkzv*$P=LLd`Y#$1{dF++Eq1=>3@c}E|Yt7HB+zzTZr8-ZB z?$dXF?%$&*!B}&FxxfOlpFsay+F!W^M0S=EyaBDh`($(y*{5>{~u6 z-P)e{jYp0VuVb*E5bx=eJZ+4&Q6J;fZ;|f4^6x1W3M} z9JiW1tRye%%4s+K z6WXp_ddx2m6nIqfH@%@NPc@X;@}nXc{t^kJ)uo3#O=(HPGQSK_&MTy1co5S-<3jiv zT=#1C19ICO4X`Kyg)NMl#4R zl4EZcA4@x(pup{wo`JjH+;aO1D&QEE3rMQY2}1d^aBR z+cyKCvjQ7E@V7N61~B24j~LLSWo?IVi4$;iw{Ci+XyS`jR-0h7Xn#JnzQ)a9KOo?= zRm@VIvcky&Z=Vq>*HslI_dFboP#L`)Q{4`nIhP%809XB@{r|%@A*}NulF8 z=8W{4uq+K_?17tBr!+kWlcCl`a>S0z#hKhF>c)3uKRvy9gp8>(-l`)m9c}Ou4e2{H zU1UU_FsuZ|orzZgN(>DC*n8n!JoxV@&kC=$XehOAO_3sd48KdK?;{~glaK-s>*Zos zC>zx1ULpfdD=ut^Pw#;JjO<f!R@cjZpKqO4GigTy4npqsRV ztE?AdD0&mr=DmPmhBQ+j{#gPcUTvXi`|bQ+rzyN#Jbom+HO|Mqq)fLxxk*$XL`wA? zp!dzeQZ`a=%SZAQU8niiFvyW3=4!vbRc0sqP*f%Me;Z5tpzr*TDIhBR`do_g*fv(PV4l>&DBIFI0 zaqtJvLeJzaU-S4G3}IbUQ(WTqE?#5zmK!n8VaEfPeq8;|fvdVNe@+f(f@3rsfWb(x zeWuM-a(Y(Tf`z>-SG?CDO|eC=V0=8|re>r8!lnSIDoo06(@McgB`pJk1vq#r)cD=U ztAOs>2oq?3N-IO>j7`!hnb1{2y2@Mm-T~A%ftAZ`=dw&Lfuco*3u$rb5Znr*X9X{kUT6^V!J{MA}d9bx!6z5^|fU|1TK(Lqn zA*2czL@zk74LvYH*lI}{-I(wI8D(b>28KdNu>ASHc=qW>RaX;!~tSrT~Qdaf-Yx-o-nJt z-=c2709@Hud=pcjPwk~ezz_`k*}I&Q>Fk_e&Qh{?Y_lJj!i9BngJWTc02#-_53^7} zW~UQ_fs{{XJ;dtnL-9TW`#SXD_K6ZpC|(6t(F=D^MK-*Zpu8zLQI1ciZPbr4;G7%k z^jn_9p0o;{)?n#XUSSD>H1d|$w~CNgqq4dFUWtQQ6E?m$3y<)81{o{fD7#XEww*Xs zGfU`3RZ&$lTkJMSV)goWT1nKUVp2Br`US6@ZwjPeOplu8_a^XNKJeWxpg1@M6#z+T zHUK(KK-I3&PMOX)24VO0*=QkGTxAD%+M)c~`&BuM= zooxwRG`@{u#M_JX`-_#2M#~@{*yKjbFKd#Evf(={D>I>@{5PJ~M#FIAOwc;m6?>BO z)^GAVlm0ldhQV7xBRKKY-O{2WxQMd^LZqkJC$5gEn^h}$|nQQlV^MEJK@F}krTw5>F{_( zq64ic7i6n|Mvxh4^ZQ~D5NZK*#Pr896-pIf%p|lt!J+?~x84k~p9^BnIH309?xZ&4 zS_ZCje4PMe?U5?FL#Qb09YNrN>BJl$A@t^Z0yMatB>znsgcvFeMJn8dJh{Mf9UaMv z0W}gOd_`ieH&OuphYuguiG?|zk=1z3m};VQ1DEAu-Qq8l&YIA;@26;VPb9%bNhHn# za*l!Z=wliz8zD@{u`n|K761ejq{>YE%zkxtp94NZ!(tY2qU3|4e~~Kv%^j+;$foq5 zw{4F$a{C9~&$g?fRD?aKL*J7|={+BxZQ(0rTmV|!lgg(+^L|J&=m*?Cao+!p!~ZuA z?^pA555<=Z&&y5nw;pXTy03Nfh6;4zJ$ zJMpN@vu}-rQ0bG*0AXq*@bGs|nHchf3#{}=tk_8`v;(QF@(K#F(Bf`n=?nDdc23d? zBWRg8u{h2QnD~v=yGl+>30tDA+M>>ias<*=_Ld9`3?i^{)VRy4&Zg{@2$gAqg(wY% z!MbueGgdeZj3R^_?sDG|WT|UZ7pUjbozO63${<=_ys~b_ z5;wEo8L{M62B@HfNudHxI=tiCXFD?(SDGJFAZrTS`cAm7Q!d>jDBxL-jwd z_ekht1y^N)Z1+Y(F<}`@+9`J8Tj>cN&@RoIW zG^p7houJzdT0n^4D=9@lVc3a>ghprd>z38ZL|EgZpRD6xYL|2^rOl%`3xdw7U3x50 zdtn}Xd&Mg@SOgThqR!HJfiN}R0nzP*dxC*C(ZQO|B0)q5u(;ll%*|U~ahDgMr-{zW z1Lt010MUnU+O<*NN2DTc#a(Qg@voC{{)B*0wWOEqD0G()_bdJ$ za3}qOnqQ@Scx9MPU=aJKH5OlH5ckthymi1gmn>jr|MxBQA+8|5WyPSDgr^70gAQ*w@kFYI%0sI5^LeiNREaE%*e4&5M68mGzkzr{2Xx z_)u;`XKRsRfx2}io29_m9IZaNQWieTbo%j*76th>j2?9uN(IUaFihg_9GEoF!pz$(49LRrpY%3%x1(MSC1XrGL zRBs*B)NhL%B2;&E4Uu4{#&>CsHvm#@4Mn1MGMWrU`p=F--w>SpC_ca^m}O8kD_m-m zmcW+>$c=wSJAs83C97=2BPN~Ax$Tg+9f2Y3I$uz1;+v6NNSTKg-81zIl5$I%2~FSI z&tRz&MK!MT!cK6-v&PQ>Ck@qiC@byL$76x_L0M{PZTUN#A{}H#G(2$L%m4;agT09a z#jexYF5^Wt-9taYrp--Du`4|xp~O6*@yG9GFF@@8zt6{}_|lNY03Gp2a#@ZRL8G>f zrt%D5kpPN}?NhstbaSibTj zk*9GM^oXQ>J|@y)C5GkP5gqOGe=>glH(vbTNTMK>pz{2syhaBB9C!mI0QTR5AOjUQ zdEkIZcYKg-S&$T{|HD>MN(XiMm!eY96Uh*X@$FkAKad2MLI9u(X zQuE7yxznRbN4G|2P)D# z3fH@@W5eKOi?Hx^#?7I)n@zJmIxqoIpER%So>4uqBLT1&7C;E9Eez@fRNUMiyP{WO zB*&Qb@`j!Hr%S~2G>77LyD{ca72%m%sWNgbs)VI^JGV}`_QpCS$yMpCqw8E2VKgsD zVa?GR`?tXBlYPfJA{D`UIWe)hy(PCT`J`}x@v$ag_#r1f8eUSP(msFCXi(t6^tCoM zAceI@#sn~>PO4c)O_ZoIvY_xWhIAdc7b(g9R9%SiJcy#EsIv~^^sdO_^tFc5FP%Fy zaEqkY^lAvIRXd?&D!dJFN7Uah`M%Hs>n~ey4s9bwUrJ$9G+?G8hZ>D(eI9aZWg|no ztiGHnYc(*l9#8BUb<5S$5Y;EyS-5_*?xh8XqYpq|m!+;v811s%Z%`v8jI-H&6|Yyd zjngY#rch0qF`~^SHj~=_^9NT*jZ<#th&wYu+R z3ow8hmSlNr7XQKQ($3aQ2@ZRX&~dSNn+25tCuWR;kFb_FIo)$s{3U%Q-8T83Xn*7I zg0q5CS#n44^Qx=kMpJZtWtTd*N}tP0?6Krp%Bzaqty9?BPgbt1v@NkO*Vx-@=}YB9 z4nj*ZWx$TA&;`#okJg6kT@ppr#FUvDgPj5>2DG?op%rnHEt_HzWj$M{f{pF_=U|ew z$T46ffu*Xg8JdzNyIx#KXRwsbT9p!T;dmyKs5g(HX=X$D%it8KwTB=y!CfIjXt-?h zZrDwl;ls4Fe8VcVPz}Ijl0Pvo?q$I!t!$dqEiYD;><}obi=1U%n!AZ= zAXgh+8XBh2c#hEwXx;lMXE7H=Sta!41u2gZZRbd*Z}K%hrYLi>edQ4do5a>&Ar+h` zV(nswn~&U%c^T#@zahio?iU!jNid@%c{a`GoJH1iR)yDahilKnn?YL=V&(ITY4O$K zyw01mC~W+JNq639Snw#vCtctyWd;vI;dg>`odrZ^aFz6>bNlA8o5(yvq& z9MB3tp{(P(UIilfD-r+nfx^8~iSbK{DcB?Ah2Jn4S8d21*tnauNy__cz&6Adf{p|3qdQ9c(%Jo^+K zjZ;Q;7USaN7qkhFSdFW*9d7mzjQfzJW{PQ7Jln z6j7oS;S5%3@Vn69Q*t?uH!rH)WX*ysj82IHU5r=uu~zBE6--n-6P025MH6F>#=AwG z%qRwT6*z82?TL13S{aH~r}SnthZw*`C{CZufT=2j=}|1Q-IaD;q$hxI?dd*rD4s_j z#$J^?_o!XnXnlUqOTp7A1_$PQxzvWmN(m*U(ymc<&%tBM#+oZC%!Qh_`>1&cO=P|@ z-jx|a2|EooYfWfV&SXs;Rw2F+EwFZ@pCg_zvx;qv(R>4i2<9s;)DpsyFRij%Ogd{o zq(H^TJsTf8HnWzzkPK){llBa#zNrwo*ltoRXW6zew}?rE-IBz%Y%|%g5)DBE&Pm~) zB1S;?92I^H%S&kOQRnhzPT;|ZdHqpiwa2ZZu)S-RpMUZ2Pmwih8M#dIPfMHwj5X5sQs_eRZWgl0DMp7e4 zI-*($gQV9qB^#H0B_X_K^3vDMGdY=|QImL@~ z`09E}0@_kzJ~|cXCATaXekigW=bV~VdgW+Tw{ec{>yO+#jFhf}hhLRHbGBBsp6@tgt~s;Bn_3>6|BVwZ=pr|<-sgCxURT)F>65W90O3HpzkYQn7qf6Vzk$;pZlnZ7K%~s z9l(0c;s0{iQ+w^He}S_0m1zGU>#1C@y{Aew80!kP9y(Mm5URSzdW{x!EWf9GfAAX4 zW<1X}HPaILV#`!463e!q3~R^lCLc}SYO%H(aFRjU-cKxO(B<&D+hmA3R^21M#}(~4 z$pW(XL7X6W{~9ClKCVgbD#`ASh>e>5F<$fzB+hoZ*XJ|n;i0>MX^MAj`9qCPsPTKF z*x_7yxGPQ|R)VT9n8NN2X=@Rva|;jEFc!J$jWBKR&M?gpXM3)rlh(?;@wqrQq2WbM zmdmo0swm77zJZwmPtzpmIZYh1U|iee8V6wLyJI(aarU}rkQ?se7P8-k6n&_#XKIK` zeGrw@K5sydR) zY;~AQ9>7o-C#eX{IjA-+lFVYf9Z25ba}&w21`mp2S&?v{*R0uK%$_VmXryp7OaN|~ zzCT!tivhVPym${cUwOTZ6FNWVrzU=gU82Y_w$i!6mE6|TYV6Ft!f7G)h1^(Rv^R>V z`V>o@mbtPBoG5zsRweQ<>k!FccR7zfs`TG@L@9a~vuw5san#+HGX_+vu2jr;WU@)w z7rlCh-lv$$0E1OC;$XD5ap5+Ur2y=Wd!$D)?GEt>c1AMV-q$_L?V1RtS6$*nY?=jq zP2@v(!DBKzZymEu8~Ht>i{Z6K3`NLPjgHo zWXrfcpSwzO7nTu%eaDf(7%+a05=yfa*coAH>Z;9GYq18cvElJ=>veS242eq%nY_rC)&#gFediM z_WberV0^-JDhlWu_XG6dw4FTZKD>#pS%BZdnU6(pf8X!pbE2C4ixaR8%hec`*~hOm zAA#}_+%53)7OUe)_y)`S(dYotzjpT4ou1$C9TMGc2%0~1{+Tkmo*-|3 zzO1VQ>-7864jg|7zdwp$up9tYd-F9#dVe?f>%)CX!M`TCS-|p)E7aa$OwrwFZYSyqwry@y0wAJ9 zmD8R;qC&%+d;;vEqsBh(su3=S@CVRRQu0J4?%H?9uZbM&IUnr9jNZz~52&U`U+MZC zHi07*N=Y@M8d}~L+#QOoK&2P%6g*{=DOX4Sq4dh(?{HEdB&R}q=9Pd3BHsy^2r25S zJNXdJaPKf}huq+3z`B|LO(%JULHmo-bmNN0PLcNYDg#U#^La*sk94Po+ZZS>k2y2y z2WQu*&1-&eKp-0H<#(8H{Z+ugX!b!$2La4VIljK%qW%hLgy$+_BkyOk!<=n!n{8_9 z?icQkOu0r?9GOgy-6js)KEr>_P!g2fI!UI^(r0wk!aaQ)>?UVcrtmLyOSHIXzr!t=*Z*ry|N zM@}3?`|sE}A=7`}M;oX;A!9%Lo>H&!VtPNi==$^|5%f{4A*1ihZGe zGl;istRGDWUPIuIdHhx2iO-5^+aW2SJ?UL&cZ7kCLNgZm$3(vpAgca?coJc9YPLTW zT2KJKuK1?BRO=!`ZnYI5L0>vuh_l>DuhtdA4ZajJ_7d%tJ@KJ&@H=Yct^R;n3%=YI#Q!AUa9g7B{^eXDhEJ!z8ejpy2CZfSkUoIOq6^aP~1c}F!0!&C6S+7pJy-EdexyB_a+jC31rz`mt?-TWPX0SHHK9PueMs9fs3U+b z-gY27*fc}M^G zi30WAG!3hxjk+Tfp}V7F&`1k?coZm=xzpRL^t7(qI{!&US(r?Gz0?L&8xjq-K zW$Kv#$kAO;4yzt9u3%lbx+)n~&Roi6j^wF!?W3KTsAR9Uh#!z88YwEdW@t|WKW~6f zg%qZ}-h0ey5%`s3&^jhXNtW|Jo93OOYK|w9(qwk-ECw0^Gm!Y>7Vp2Q6DQ+Oh*S0T zWuvL%99I&e%iav|$o0vP0#e*?v6bBCyp_t6QbnlI z)Pi#Mj+uZ$6^%X1-PXUi>B%KL*Bq*O`GnIqOSU;4g*PsayFIy0LcKMb#CCju{V>( zy^)(M&v7(d>_tN~IZYyAkd1z>NPSdN*q_)cbs83?VB^E*WO``qesEg7Rmiw#-H*VXPx=wSbeb z0~W5I?=SP$`J4%>z*=*+x)5Ni3H(;S-}X-nPlBTE)cqHP*xNY#MaMzA&NT}xS9+ME zUzNN^Z=#P64H4=db$cAD!I+eEsO14|%-S=^hCiZ)!%=>Jy?>iC^UsJ`vQP=5fIU)* zNl6;R86O4xfSGC%swZFq8t8P;(41k( z9tS2-_#e{9zd6N!1x!&&6iBosR=f+KwZd`R9Wxz>z7kQU%J!bG&}O(9Ve|~7NeGPo zc;V3_6?7Uls7AI5lspR9f-Oi*SbOi zy_str=Q}>6(K)_#ZYe?T2_p06=#x(HCU0Gc{G!0>fFjxF!KKAvC7d9X*JyJ^IBiT8 z<&kPU2jIqT>B#{-Jeu(9C!>~P-lhh&IwwJ5V4Dr9`^MZ-^vfBw>aqTGCH4>z=#h(i z7~cfgX&p|?UJaFWC3X=hJ6zI7VhleOo)Z^##WrsA<~0Tt8Bp=5W}D7ZS~P7`e`{1@ z3~BV@8Q+pwUWKgUv7!h?T6qFBXesg9G8lMb*N*5>@Vh#qUxD23LwwI1qU(w4UYHbt zhdV)|&hIJ@y_}gTu}06&wR`!rMVc#ui~0DE8^?d6xc@u<2j$D)28UHXFZOF!aR7Kz z1w{z#Zvlk?)~rB70|5s-@IZ@h=u*UghP|1Rh=G1_skg!YA&3$H=MH#?+U)8;NkRY9 z&1Zk15bz~rr12#r_X+Y9S#J5V>c|*?rU2NhF3O^heQqp3*;MF>qu*uK^epU<_$w(tQ%q!@egouA86#44wGS74lQNNKk7rY-Ny9o$1;bXzo` zHX9mhzEJU!?z6^0=pAAZc{7S4p*(9cmFECZAzfWZyqrgxslr&%uIiC=~iwFroM*bYqru&=WtJ5MA=Il zh;iRQ^-CF~Yy`p7|65UPP^EI%xyAfkNW+v7CFU4s+qlFo$vjhk%~xB%PPmuH9`4K< zC#O`Bp(vJ=8*IC9P@M6DiSdi&j>GRE519Cuc>BUz#G~KRus!z-$`oT=O3169l5=oPX@>T~H&zIr~ z)jZ0p!?afyzYGI_6GrRe_4SD;z|AXQe+4k>6Mh}!b z2LT!L>abAY%?~#LJ|%!sqpA|Bwmc66buR=(PrW;>)leHTuTgHW3&XuH4pVtPPyn_w zxXLI5!rMZBy;&}Xu0x=>v-&a5{Z*LZhI1@RL*VnCFrEMt{~UaAWLHL@HNvfa3FxPa z1TI&z<3qj7HCWz@Bimwl)?euurXqkQ`S3@n@?8Z=S>c1ga+z*h=s&s(|DB-!?+4)L zu4rMw*UWZ;1PU)<5CEB<_$b-}V-SCWg8wt|+faar5McjS{a2tZLBRj-`0x} z0^0v+rDJCQ@&Ntk(g$ASE0ka94mXIvn|wqDU~C3GCO}sc?Mqx3Qj~I7gg96&1h>Wn zdYIU%CZCkcmUa&!EV_o591I>Yg*_wRt52<_b=|kdp6Klt)W&jnZKuZD@9m}KB2Zf>8d>XAKU4jqoHEXp~V5b#*5dEwqs8X6Kb?JW~y&>D6;K zfM2)D-#&AWkk)-V!a!-TY4MwY(lcWd&Dn4mij)xgdi>Olv~(gIm55U%njb9=4!IDN zfBZ+Lbj2vX|0Aqc*8FmNriw{;=A3z4x+&-;6hKX%196@bJMNJ)0ov(-VlO)IOhjEX zoRw`~_AHja7G;bpJ#u_KGF#B-(gNqSQ>FmZm~F*2pBuja3lq zRFm=`)&hPpz)P_$n?E_(@&}^}>z`-(MmW<8v4`sF3g_sPa?-I>bR4Y|jo6VB-r)Ho z3;+Wr2L74?@YiK4U`lJGX#@tR{!u&xXmd!{Qv1 z7avObi7SMHB=|ufQ2rf^c3LpnG2bGufds$5}qgL`jYcj55RJ12|lFY~t zGTK#_yT+pWm?r81FnxJKS?qoi7%7xgCuB=&zrXjS8mCW5s?Y`1I)xyWc1`xy|B3$2T z49ljbvLEBv(8GIxOineFo}`STw^;W62$T)*zc9rraRb*qC2<6qMw(C_HC6Z|H|JM5 zk~I~zWytX~LYnWncgMtVn%BX(<#KSD7JD8?_YWxtYUINZUJMJahtZ34N4*DjoRfnE zcbKZ0AH~O|W>I$9q0zk70`glJ59+Ak%nL^8C#1x(8lB=8b?!1w{vgXVMQ`d#b=tWn zl$>kTxK^+5Xq?+sb@K^hGxC->H~t}_#7Vmi{d-rAjnFd{<0P~hI^MdX>k`k9U@x}= z!w`PND1J`aJT#gAkmwjIDdt%8hocXKg&nB@*)A|x$Rt=MEggaK8c<>?CTdonT8>o@Sdb~*?ECtOR)kbV#1qT1E7g#%BSE!)Cx znMd@TAMipk>^gS1V_%l-iFl_FxH^7WHAM_F^HwoTMmK;~UjA)P(nsfVikMa1NINbZ zzCCUBdCdotJ*NC1tMG7ROpbO@Fkg(gbg^%%$iwb7X?SiRo5lT(%B2pf!0iIJ?GfwE zqO|se7_UGW{^EQ7V3E2f4t~mUD!1Doxi^+PFuo2{)kH(wA~iM`^I*GL*n!Y!5z0@e9T>?vCBzvy)&6)--L8{zGlQSRbMo!7gDwV4eVF#BrBV#`sg-U(8y?1x_9vox7ZF zoNZB+tf8STKzD9VX;~GnJc(&lFAoF{J*%EqLjYkj$midmeND3-L-~%Oz>)@0yx;py z?g0VrdQTLltM-4$ddKKYqOEJRV<#Qkwr$(CZ97krj?I4Jj&0kv?R0EAeRIxx?|8rQ z-7)G%RgL;td+oW`oNG?Dr%rDblSliq!ue@2Wgxk`W;v}Nz}_A*wp^ra^ZJ~m** zBrRr9Sl`y6+msOw5(_$EA+)e|O*ZUk4%nGH>hGCiP!=!}(nAY*b+fTtJ;s%EVrBL5 z*C&J4X`6Dgh00mwBb5df;-e+iUsW8mvcJN3PshlNLO@wIg2G9R1iG22e8uMe^6)ZG z5Zggxu|E{+0nlqv7zt#1q_oxN}6<`dp*8+j1_LD1{$ zLlW8G@y|KiI(kL2pK{3P_;kInU^t;!N>NhXuHWEj0kq%UGWqP&!Xe#sebsRyh)+2% zi?GOBaCb>_1E+fZu;e|(ug%f@zhkLbbTwa^yax_svJQl!B7!1Y!_Gl5-p7(H(Ia)1 z`;aBX2a#2>ipQ-TLh~ZHhH;S>TIod&8>Ibc_CW}uw0cPhPvYT^UW6LJlSuacIHETS zKcF_R0B+FP=p`J4yFAQ_xcI~83ih6j8Pydr+hzOfM-!Ky154SDk4^ODHh&O_^R@pt z;<(7Iga=-N7-aqG9FO!9S7%lvgw`aQ^_2L5s%QU30OO>8;vF8qyqOCSELh{YggU{N z@BWn__ZiviSbTZRW>6T9?tn{n4y_FkuBfnF2DC;}_hvbJv|oC&YGrA=BE(Sq6WXnk z$7JCwk;A#2?9E5^y4Tv}cNy0o{2Ctpx`yr^uws5BAO;!3=%@E<1p1suI6;~&$|f(W zT|%2;JYo$~A7HcQYi~&N_G6GtL$g*R4}S(>>h~?)W4t7g95E~2J*cE?TC|h4ofxdA z12#ozK1pe=Op{-ZuvZ!Wk&y#Q&Y+9Zi`l@~N(wBveT5VaVua?gk6a|nKsj^+T{1Zv zCRW#qtL%KaUGU2wFVryoh~NlDhg=V~A0plvuF|)!cDsj|ExnOckhry*c|M@*|GAmA zvE8Jhb{?1CkYYN)<)ePAqkmv2(%sa%1MoyYsN$O9O%^v(p;gaHg+{A|M)@{3Z&7l1 z&Iv!3kj00ZzGg-{z_|Ctx^IO7PSvWRwC|O!Xt$jYXbZohhM0G%h2b!>&^worgHn4U z4cw{!4F&8f<7VY)c(lhgIgR}}kEu9!22`Vtc9{FG{W))~IEMjLTk7`_zJn=t7zO(} zh_y++U}@y1o_KeLO#(y4lAhuSYsA<5dZUDP-UUh zO2__xniT(A+Wzl?t<$^XhzHCw1G50c#D?)ScynBllhahe%I+5MdMNMi>A$m4 zZqW61+GDx{q7`#3Vl{O2Q^!dX+H;>lpglAGYH!~F;`x^I+4c4ZCX&mo5;PLv09n9? z4ICo=cJ9FM0o>9zDY~T9DEJ1>a*@ZtAIBSkF?Mkx7ECzi0x1@K3ibx zyd~2@k7rhTUg!v4@`z3C?{DEI)dJa=N7iM1#%ikLqSOHYbJChqazJa=}+%t^ZecdpK9in;#BP1SVdZmR|pe zmUr|b5o=df5!gH5bI6F95w*^0U6e(-~EFNJsl_r(=Vwr4o(=Cu2?1;J^ zJ*+FlC{R{xsCjIn$(E79{;d#kHXq(3H~$oj>!QIzXJeq-&>}%&aay3jvZ@p2voSd+ zfwi9b?qSu@Xr&@cp5I1KuH2yNP6}CKf5(!z-RMxWaEZ)1U0d`9~_A=S^Jc zW18Y{#hAFBbNAt?B7bNms%RJ`?$VA*kVY~|RzZ;aohod)T~cH}J(phbWWU6E zDXun^gy}M$_n=Z$xpwU0CuU-ZY1yK7-{j}BGWMpe&|qdlH~~oaog~Yy32D!5d&`2G z__$4P)EkE*yqS-*pIMb)S+7A=Q5L-Q`@~ zM^=-VrhcZ#nKN?mj!S*PlK!GKFeztP>|Z+3eB~k2#qLC>@k^6cFpPr)83`bU!VhE^0oecD zQ+u=LM>MCMlPmal`al?!z(p>*sXdwd$2qILI&`?id~Q`mZGKR66frl_KkD^@h~y%n z;N}C*nz*q_L8vS~kIZe)X2gldNssh95CHY!rqg+H6NTaKSmlWvXB9cz^LJ*7$+%DL zZ!Ioe0XiV>Odtw@WHRuA>TInp2~bKETKWJ>t?;bkr-(-BM^*j!f`&bUZ4gH|T0-dW zcQ(K?@`EQpznK77#q0-enbTW2syvwk%6LA*mHMgDzpiCuUubd26P~Kz0a$JjtL#BJ zcNC}uTfL}Nk=?InaQ($OV9p=K4WD{`_ndSyMHLdSgJYP~W{@NpLWT?X8z82ic)(-*1%lPyQQG|KDBhd-Rr& z`2XJTTD(vQ>_MUbb5_ar(!m7cIg`5sMi6J&tWi*Oq^LvES`ojDkEb2a(_P13$Idg{&vlyHU;VN_X2ZU- zP+PxFNR+)f{ao(rVgKjaKc{El%HzBEnQ9k^dO&~ zZR-71Dh_R}R0sW2p@X5}(Jl;~dTLI@uFBG_T8BgAT2p^koa$1d&Gie}l8?2b*sLwLzM`{3|zenahi`-$(HaANv)?v8jMT^+{z zA2Nl!PZ)zclKJk*A38z3kzd^O|L7z)F2O`}-6A*U46cz>@3%40id3k`zFm#^?zX`I z=;ZJdYm=fVfBqGsE2iN2cI+*qAmqs$QuK7PqMH@GxJGaT#439i`c^iK z2y!hSBW=e7f`rIqB+60OrKB17Jf1BP5-mQty?3-jjl6yOZ<(6Z1FVdG8p}J9ZR4n_ zmZ?@Plu0!<9jM84ks~9OjhXSy#W~3<(y%Yp$d!k6q7aY~XzN4sE-<0W(BY}0R^@qV zAvAfEQKp<)$t+N|G27w*`hZZR2h8N`*-FTe7{BUTo)yd`tYc~FZkMNF$dT2qb2Lne zoBRdyWKq}ALgc4wt*4nSQ=0uQ-!ip`p0}mwlGfvu)@Qk5cF)MnLdd%k)7Ay~z4heb zsYePH8yYzYRPqO8NcwktN@&@UiPf9eVYq~8&84I-8x^X}o+vVeWeGf!DeJ?~~ z!$&I+t&eUc3!rf?FBuWdx)RBvJ0?KZw)IdHcp$`Q{x<9|*CB99$Id}njuatayqNR( zMV&cRRZkHC^0I9UPAJ-K@GbTpWO?n}8FV_9?BMz;#V;F{t(2|aH^?s%UtahV3Z7=d zWd#WKz$>yF>U+U_1n0au7Ry3#R6>e^f2#@BV9`RAm{P+=?tT09t!)!V52?z~hyW1; z{uNOck>^RX!_`~9q4clrlexLkEfnfga9GO+1seAN=;Jnd2daw_)%sUYrhe&OxYu?k zxS#XDw-Nd}zJRYzslH9A zkAUx&*^B7l5oJLsvJ`WQkQ;gB;*sh1GV6%Y}ZHe)C+sKaxSp`Y#1w|d`Td>_P zGIq0aV~_0+?&*5&XqFb(3NP)ItlOv%8@Grcryx##kz7XzW%h#T2ede#hi4uT7lTAs zP|Byi8duC=7?9?ZgUCwSV{1`Rg@a-q9ZG;F>?dX63mIVHgLfR|>IWm`yMadH8wVo< z4CtDB!4m*)N@GTDIIweil^Gm>dwAWM(ahM>2l#pk{5fHY6JB_tES->=WbhNE{S7G* z3%7q2$x>81Nyl%XZo(_iV6ZKWyqxXZFlZwi{s;)gFi>A2enzcYv(f`DzgWc9rq zzO~*jY~c>iV+lM%??l|u+}dt(lKIAn8@c-n1O z$m@45QA4JgpMBFpLW_3s}?2Yrl%sU)LQ1@qC@U4n&puA#xQ(cN3bkdv0zbEHO ziQOBzoGJined%TiFRqXvzj4OEmMQ<2k>WEE`RaCOZHmzh8yb;wYa;zydi}tYa`mpR z-dm1db;!W_bevPKNeAQpB@M!QGGXbBsXUHN2}H_ghxC{CQw=HhlT>*bgs)vKz*$PM zl|yg{`3~M?9=02q5_9uPs4umFrDKS9kfH)C<8WZWMV1%xMM*0V_ys65f*>M5<-qtc z`v;-&C>rV)&1D^teRlPbF6I53Ugp(l;Z}HQC!*IL0lc8omxtCLne)8N}M1^Z)<#u-RaKW#wCQ;cis+UM9fnk2qawaolga$`m|3l zQPWRJA75jBHQ}E>^pub!_ppdIB@D@#WOm$m^PXS@W|RR^58nZo)P^)M*Uqg$>uxK# zcM4}~!v0cC9b@cJ)yEK1`f5I9kSF0RSbGeu>6T2kA=J%fb}G+qY86@n@J9hoZ5zqqV41S+hgAsNNsv zu}pQy+_c7?NJjp<U0ZE`$@A8@iyRah4~BCJ!e zk7f^{e8=tw0f0O`d(6sGLvU^-!}P8wROVbA@a{ho^bVB$tQ zl8@H97f`OX0O99w-pN&V91m;Ar%yFOC8ew9K1K^~;^)qtn|>td>K--lm1ot@pkSQC z<2T{6_n!d2)%myz%RG*iI3(pyWGD|lii?+@bL&QoGeZF(XOj3v05u~)_?}8r&oe}Q z;#$i*2Fy2S*i}lF&gTBW>RDg}2w;eNR~7t4TDKPpV%OUD+AuNtg>9+lboE_v!?y?G zsX3_BF}($zW)lIwtHwW z-)Yr)&Q{+AA$!VAuTi>(d4EgoR=6e_f+6XRtxk0fJA7e%WW?1=dU|`JzP@L$1?x_F z@eeF)h3Pqvx)BuoSa5ne)zVn1ud?!%#JF&vmP6 zYBH$|gBUICV031#adi7NORv=dzZ$p$9GEIBCe=Hw~0{<}ho4hhIbeAXl3GP3B znFJ(Dm|E>KBVrP)qs_qb^WTiPozkM8)SRCjyFKt+GS!7C1iQ8PL2hCK^87KL1T47{ zhbuXMX=nFUPLGy}E;-=^S7FwZgSpt@1UNH*E^MMfumkAb)>-Br z(L)83>`3Z9VO^2c|15Z^u$g4tuo(XdR1EmV|BfHYDFjNKx z56R=agA+VAhRcl5t2b>fCOD|$+~dPHN@M#YCn8_2&-&V|3z$+7tn07_E-<8heka81 z38j1LyADSX=nYw=H&p1A8gw3j?#kWk||Tz5#~4X zoo8&Xe|G;oj#UgnXHzhUj6om^V!KA(8Nvfc&3=Wh)~55j5$SmWh&CyL33ELNAg!<+ zy+LG-_L!O=4FkK65xUnA3R2H-OCKGB$9e{8o$z47Mkypu)L29wtHc^aJx2%fA9@1~ zc0e?Roq?jX25^6zf&bwDW8(3j!}`cK_4k+U|2)zEhuTB4=09Qk<_u@P={@@Yns~It z;1YQMzv>s^Ao6%n@c(?*GEP7N|35rpbI`>>73_~6{P;jYZUPdZkR=ZV5TO_Y=G&Ib z+YfaF{YxXMtGQ7^BJzhTnI6RJH*LtQ39>QDG0YJ+gSF0OYi$oVp;M<5l-+Ed%qAJX zX^orFY(3xj*9LYNDh!XzKCeh}z4$qO``N#QZ7OrlhaOHF`ljaJCr5Dbh`h%Z{8`k5w_|eXTLBxFh6ab_Kix6ZS zb5luttN~`^k+pkl*d=kX}kSpv@o5d-L*fPi_hxuyNBcIkB)}p z@Vh-g?rndSb5@?b1#%Wci2}A+^jCtjnf2H5;O)>7Hqb5*Cw$~cThHes95P;(MR$qC z@86aJ_n)6mdHd@nn@8rfHzY*AO`4;JQau+f8=A}JJ zqZ22L^qiKZn5z3yO;|o?mJ3zk)$`-I3>S{t02|2#%I;P2J1u*gBEUKUN~9dGRK0&o zuAnyCZ_ILWY;L>Vs8MnGlkMu^nd-~n4p=O?5iG~ztICA8$eBztvqlH23$aQwDw3g% z3s3u4*R3FK`09fedz ze`7{AmWp9P(Vjgf132h*gf&q7HoW6XJwUUhnJkiJ`F*s{3Yvjqtu(NegzoEnhrJ9D=3J{F#?fNXRt2~L-z(oKw|EP^IWheY-B zv&-Ej1S&ov87&TtjBmR!26={eu~ioy>JF_2LE_+B*flAJ4p0iX9b+J>hz^3G>Js|> z7r-uIPA{0wU_Si!12-b(cz2#`O&+GMzQHwqf29moS((Du|1*F*zrlxOKP6E2-;9Fd zSPXrLDmJAC@kqPxD2Og=2}fQ-o22S9Yl%}4cLqvqLfPN^RO?sGVUawWGd7$%JN!?V z7wqGbQ5Fm7H+w5` zRVrhsjBozZR!3`LIKyMvPeFv6hRn`17AX~_T#4S87jV9mt1@^@o;8dKzG(Q-J?d}I zRFFZv7yXhH!gxTAY?yeX-buD+>!scM2A`+%Q0>mgt9z)#M08$0OxLd58wQWA7P45% z!aYY0zk81IRAAW8c_RT-0;KrCwR@EDvA^X7V`FSe~)6-ug{vpxtn63z*+-4U}(JZHsqFP!b zTatZX*1+7z!_kB2PHWC+|pxr?{8SQ>nF=e z(YqtcoUAj(Go5G}<~1mG#|K#{d;IJt3mrE4`CJ&tOVK=WER$r`hoLKT6vtA*f?Ps5 zkj|i3d2Vu&6(o+P6%WwgvZ|=t@yJGMp0oaS^9O`Y+%=$3=K^p%cF>kMV)Ig&Ju1jy z0j_e9(^{IC){ZceYV3noU`aLAd2L+t=IzAo|qt?zfp07>lX(PPr^- z(NARJk#(OILKfpEv9h)Lx-boBGv%dQYky@uKhyYrxf%X8Pzn46Y7d&nEX*#*^0HJb zSZtkAv`s^S@<1tUX0&tXW;@oLtsx$L2XLD%b8v7t(b^JL=kfOBs1 zUAYhlyO8CBk9%9qInOn8bTY{g4)I}k9Qws(proR&a;ws_wB^&5%4cy7Gg9gD0m>K& z(hajPOUlG7tk%r*DO*x0!Y>O2bwXAB$sig^UQ+ZLW9^DRiN>G6Jj>-M1*ftzAl0A! zU8ssLE>M&_o;;HFE#-MKYoUv@po9kXJkCI~8&9T!Sz1^3GT#_HzK#@mg6}D_`(Ir< zZVd+hys~+Ii^r=9TB>RjF^Eb^0a~?jB3zSA%fCLO%nyh~E(R3NrlKbDKs3QRN1Pz4 ztLA1k6z4U3WaKg@OtJ-GZR>}j@m43;8Db_faL7eT&zY++syoJJ^+n~XoQuiJ(#}Hg zWoLakRF_Q!qR#C(9-2v_*^)8lJ9^ErL+Q0$1zk?XX4o)?$mORzR2#Nt0h2zH|1{#} zOpHcHDbRZRm^L}`|D?j-arkWa3RlNOXAmKx4jKn@!T(&gEi!L*Zsh>0j@L_+pXubv z;+@Da!FM2G+?fR790`^0Q#i)|_ZF*D+Y$3oETaq8k%i7HeRfT*$_t*=suVDIh$q?&QBo@)AaW@O z!f*_&H?n8Ph(iJwKHB>|d>2uPOR^D`z!@bw>{NzePtMSiXEa9!;ZWVXB@m_UEPMn< z`#hvKeFb9T5jZ~b%dAf{B@RjTO&}w|D&rIk!Ev~0&`^w##|i&csCZ6KYdTj)tRx&y z2va{}5Z*6zI}LzqBV*8e0k$gVb_C586^zgfx*xHTGz z82}GQhc#$!+PWzcLcOdJV?s>oa4-l&rn8QUT}nGz#5$@q;PiP(S|*C%Hxp*8Q3+;7$p;{($7{{>%A&g-RXkuE`GJ>8 zRK+O)7`ey#F9|u$6wudtPJS%ojqE=O(_s&))PEq5YI&edNbZ%z^BN6d!J^6uKx;aW zVMUe7?nH+Usz*yQiSu(WKqQSYv)pmn6V1#H-_nV%-&-(uWSQ{F6eU-grdoS_U+Afe z{=+=;010It(!Nr>r|^M@W%4XCkycuWtib1lbnP|0eepPa&+O>&p~p_T z9tcNvHuYpSSGVr6NO(PSVHXH)X>S0HvFSLcTOEO}3{QZvd{^IWd&0!edFRSvsh;3E z6cnxO=^cr@R}WL`<)l!K0=+-I8R(>~cTP#zMw;Yl%m(GC`uB*NRzjG#vxvZzrYEXu zp5UYbiZ}Qx@AP8#Bz>01?ZF_2VTl}oMrDcnMj+^&6ZLj$(~pV+k50t@faU+ixBn+t z{txHXLb(g(^gkfj<#4gY9QwzPpQu1#SauzNtrwm;b^v9^nlxo{v9u3yW8Q}Drro*& z92!~K5`{cby|k<+rOod*^~r*VC+YO{Q_N5^VzxMa81zzG_DSR^k zet-rKy^LJyAPo`|8-du1rXHs)&bD+oqB7>UDLm!)EzBnEA4>I2O5f*+o4_!rTWM^9 zq&*OHB;a03*twHV2w%SPREgP3qn%(oc>Er#2!Hi{L;XicnEOjs)Qs&a)jcNL;>3qI z0?AVJtXIK+3R8|woKU4bB9`P0yvO(P^EX6`S^O zaO$AAO6XK6H&#o5g?-*Wn+cuHS*414gjJH?;fY#Fo-$D;qdIuFH8qO6_&9b5MKtUn z&0`YpiZfZ7729?KwW=-F0wOQ_x2boBlo$`s8xA53bhAoa@ObNCn!YwmD^v_1Ad`0K zR;|Al38$KT=XEHT)65*r4LwE0ZJuK*EiE|$!P|g`!{o_QC9qL4_nuT}nDqpKx>RP;60xY>z|umguWH4L`ab1G0v!z>nFT(&{|7tj zi7S86A296=gWj3P`;QdSvKVnoOFRQ^+&t6M392nF$+_vx7JIH=bzPqT$;ptF&gc*H zt(-)zb?ZWZ4)h(15+i4pB>Msw)=L;Wb-szS^JYl=fr<7M|F?he;Slzi5p+|sTh{{u zV+u-*=Ka+ab_-7R))bT^J0Qc_!j`w&Q>?bcc(z+*m6=Q%^wP3Z7xg~z3k=Y3IGjZ~F@ zxpUNvlQIfnaOil`Wyl4FbI!_cWMnp%t@@~Dj3@S3whgesucHFQYm}B%#^;wm~Uo)w7eoL8-w_HQ*F#f)v?UL3dkLKvo2ukgt%wWsla2NWY|V^qOWjZY>O_U;VxyJqY7U4E z47d@J7`rY`^szlmsgA@Ol`AaW9MibbvzUsbe9jM(a$X)DoTx);K1B|3MExqpqs+5I zI6L>kJwd%C;*|&ZXyO_y=X79~c8y@F`|5zE%3S9#%}gTTZGXK!SY@DD6=pjq8n;$g z0Sn&x9@Hq^Hw>v_5I*386mG&`H(n>VY zV|q(8SCOC=OlCGA;xMB-&Z#fMfXmjUu`xP(ouS?Kfa#s45w}`HWJGgdi6n=Kv2DZqv3lQ(~~+(J>%{jvyzW`8McQx{B%!oL{M<=808u zyfFuqb_Kao9hkuU1DhB(_$%s$vhJNSE=)jmjbN7(D9{JI0RR2OrjV!7hxYrnL{TYD zF87S2JX6M&n%xa@Pp5j34?+Lvkl*s_5x8!obb?;EQ;nctbjVto=&e4(R()Poojv$i zqcm{lOjp#o%cTZsE1x3kexOB4>Btq{gqZ|r39JWf_KJ5I- zw)_|czNc-d%M*zp-}slwlk=?u*YesSX$#|O`Mu+@!E=YQ19zy=TFi%XP6H(rjo;Iz??8Pf#C4&v@6IswkOU3r(TUua)(Y+pg9GT3@W zDA+{=+J_!7RPRh@1Bb?HN^d?oPL#szQ-vciNk0$4XHRQBQ`In^ND5P?m2eqUK@%pF zNSly6VHTw>j~yZv-4~L7?En3O{)Cwa)T?rjHHuEio)UND9A6akgFLvHqtf6o7UY_$05YjkOm|2Yf?OC2%B(~l5>f=wt=-)x~&Tcmi;`h#V2CEbg;CvHe zbe!CKcT^4?B9Q?1BRwm_@k*fbf>w{2!pV}riZ=oy5D)Qo7(7@FqqK<89gs?o`IGYS z)R~S_1q)umkQbHbjcMbwMzDM*Bml{4)#es@XLWEC-I`e1S@ts|fw62Gczyii(7Ox* zh@LMyjr588AFT5K!4Q<6sQ*8(g2+mr@;};qidCj#fp2YoJ}j^ol|2EVq3eh%j`l@S z*EK!d#_bHvX1{>a>tO}7wRVP1A+y)ZDqOUe>mSnR743}evb}U*I7Gii@)AaaMe-Mx z3Pf9hC?GL-h5JAc;bQGzu9heK0Y^N+`kmMPI_+sOowa5d@NZ8b@YqRTSZ1go?Q#Iz z0C_6Q$^d=JHlMp@*Bt_IPxDrrYoshL$UxOvYVt$YxKJEG9mCl_TBLN8LEZfxcR za8;m*)?8kw;3@)Uo8vP4YPeghU-MZiG~0s@TVQAxP^WY(bFE8VQ2*t z`e4$um|dzkaVI?~RS}*b>99}-Jfr~~;K8k{X6=kS;ns{+&=Sq&p3xf&WCN zM)yf-L#M2-&1F;(u2&{q=l+u>MoQGzT~zm}R_C7N zcm+rI@Q>_R%XR5O9`;CuCF3(G{fsz(ES~RbQ|El&Pg2fo^iX4-uiA0a1|_xnK^*;g z7@s@pv7cgUn?`SF5mB;q?+0IAU5~LLTzgpZ1|H68sXmEP>v|@R#um6ZVT!-lQNfOc-IjC7YV|PV<75 z7Lp0ZO-?deAtmpt(Ub``{(den;`&wMAC`|v>zh)Oi6^4sD)|=W)zHltrpk3$la(yh zs|5h_KOX~wz5D_DDe7Lj(`@X&6wJOnIDZop?;e&G)g}e@2}-LZS%`m-i4h}(dZ7Kr zb*<6C6zezS7X(z;X~ekb(Br=zoiX5#H0g4)f{2g*9pC@o!t#Gt zpHamCxZ!V3!2uGO7l*+GRAQxv2Li%*@c>O~vWlq6XkQMzPR?|9^|=^>n5!yAXhG%d z65izUQE82RAzPOZ7?A7f7CiL3{WqLud&>y8oN290?T?b!Gug|*>dCOq7|(oH+&kWz zu16csihn>Rck|++v5s3OkPjoA*=qlndj7ONCC~$ zx%52VW3$Q4cCYz5eKCHSgnE1|a3ZF!5G^cDTJIn8(jzjIm~?O=EfQ>u+o zVyx}F4c1mu+oDXbq#S1wd~+IUf17HK=Bjy^i^B}do>sT!lL5{0sQB9KXdOAnW_`zL z=xysqdsX1ys?CqHw_+YMTw0BIqXBW3$ZYx!Od+AxKZwy+^PS3hse_Bqfiu>y;M7lI-hd0x4(EGtAT_!`B;>Dl~q?qVF-%mJ& zfq*lt`hTXsu%_G6s+_NP3l2Ex7P=#^qvnS_72h({mE zBx>>&vptByNSIP@2EHYiDFVP$MJ12Plj_NA#X2ZR21p6}1bz|&y&&qC! zZ6W%fJUh2D1x|$Iw=y|{{J|ak8C=*ETv4cuj2KbgvG$?~jyNYt<^`P{@r1#-Z#e55 zGi`d~iB$@x|1m|}`b3~}6Y&uo!0{RIfR(3Uf~U&H%Pd&Y$K%0Tp$*Uz4cSLR-H9~v zdchu$DmXiLi&LRIOQpp1zHNo|c3PzCTwt_0LgRDncMa~5e#UXw^8mBTDXoXKs=|s? z#R{U3$PMZ%ZIi)y2l>wq{J%f?|GflU(OEe_wD6&kC zeP-=3ZpIxO7YqK)0H8#x|H( zU+)0HVRciRT_(tqyW#+Tt5~m> z|1jb;R!7`Rw$misS}%EHVzc2d;BO3+`h+FcXtXB>(qN2aluW-ZwBwr)*)Ibpygm7~ z&1VP*0F_SlUIYK5IB^Y&@^1!tj^W{X zKQ0e()Njh8SA_a>x+>`!QjSdjg}B9iW;(NaK;C^*X^My=ty3qtqCq3KB6uO6CG`xg zatTx7F^DBqGcCkiL`~`@!VDVgZ@wRGg6px!-ivMz`;l<~w08FgqNxPOaH_DP1^g26 zz0m!FNU?YFApv_%-gVNkfMKSMYJO$r;sfqsa!-*VxUlDcZT%--+?{$DG`9WlSOz&J z-6Jn*?QYuSETR3GH_PK8@`h>dq_uXY<4yK5w9N3?`5H*g$6nca;Zb``QF?Cq;VdK9 z{lAH%Isf$m%1sgselaMaiwf&t!0}^RYcno-B=le&k3P*ArjwW~7Q1v9we8_kwFTi1 z3M9)41GwlEJmN0+_=({{^aj1};IEJdAmD%O0S6myWGU26-dU=}ti#Gu{iELMq!mJP ziV}*ipC#DBDq1mO@!%j(-I4GQH!%m>RqOQ01e%Pj$0eD{C@g-ikWNyKrTUNYmzsmR zHG^&wfRJSd;A3ut5fWpYE_`upt7!JS^@|Pi*H@SB*m%@b@43`YFa5qU7SHddW z;BRx?KFr_o>v^rV-lWl^Nr@WHprIm2L_*eZhb&6Dp(3*JgFiQjf9mA!h0TN7p9))_ z>if*iM;EYGr1}lsf*AJH{1NQQ_IKRwOnr>Y{2i6?j@#j-9X<)9(We zq<-sIF{}6Y7CihQ{~Yewz^tqqj~sioYV^Bn=*|2^;Mj~1=5^?wn)HCX{>O*ZVzJM$ z=`e)(@+uDFx#~pOZ8o6ax)Epo;zH3K?3;F5;o*m1*s_PaYB>-9e|z5b;X~MSWqS4l zX`pfzTD3q9|Lb;qeBGM}^cm?F#OMHne$Use>$N&`Mk%j3L?naygZG!95JFh9 zkb=Hk98{;3(EO{2a%)pJ@Kznp47hK6CP zkjEDHIKvRWM#MtRkQJEo^^Hph&5lo=wc_EIo4W$} zvn&k_dW5F(dC`pt|5tlZ3ncm^W%s;@^@Nji4O4m7eq#3gB^1|A&9$;&8IFYO!=ObMTUY!xr~L1Vpf61e!O>$01!D#nM&0_XE5$$% z8wl|vLSD)D4?_aKv%F*o$`Ua5beB6-%}n9nE1g%B+{;6*?5C`K{s%fYtuXB7EQT-D z(XP$UnK#TnPO$ROtU=jtjA3C% zXXNV|`(IZ9qEl-a!PFZWl>H|)H1#q+aP-kCRv|w1%~KG2DJp;>+;=mG^jfo)I0#%< z5iTV5%+jT<Ebe5vZ4AB$8=hYpp-#Bhm>+1YyrO}*}kgQ?W+aZYZ@Z#OZm^K_NAc0MgrM<2@M z^r4C3$EBrFtOnfKt!rtkkbL*p^!o|TKVSgGxv969AL_&0UZN8C&10p)RUtiNFTC4@ zpqT=*){8=xW000Yf}3lZJX_M-T1R~Uf=MiC^s~x-9$91m>lK7pR{1bGq0rIrQf}nK2+KR9lNIkr+-|%gh*^LwhxS z{IBqlakmyxbf}y;awxO`aEPRnI@&HrmyS+NO@>3sZo&j+8YmVQb1zm+_dCk`8l-@Y zXL~D)zqXv2NciIlEzYO*pV_^ zr_VA;?FC3L+(70hLKBPgFu&V0cQOqTNV(&kXQ6}8=Y>pUQ4X(gcR@CPP$aNpEX8ge z&j!;e{Imsco?Mb8#~rrpj$v>n!R>zPie_=iF3KW<0n*)xgQUhiXPTfnAM{kB>Lr|w z!5-h_5eHhl@)uqAZbr2UT`fZB?I!)x2QamS%>Ys&LPx{ZW+^>&nay*h|IOCq6xx2N z#k!vAAy^gt&Tg;6a+Z9jdA6cyo7lfQbmdH(s~TKVmHEy&x=d9i0OmbBS0JCUMCnU1 z5_S!gK7MS?<(E-u?`Nnqn(kKxJ4cr+HZ{L>(0mXOhHkkGP!Hlu)fZ~LqS26q$$M9u zvH~b#&2iJIau(dfaQ-TF%3LadcOhQ^d>)Xf_4(=e`K0CV&~nN$^%#oN_Da*3$=6k? zTAP}SipnYqE6g~$?zJ$xJt&ra)X!t<<=M9}f=)MDm;;hJO)a+Ft|nDZ(2ykrXtMHP zGQ6~zsQJz0Cnm>>nkU6y8Xp~Enu5w%!~uni>uL*n%7^G$%%_n0$Ved~r(#qY*=nD~ zawhhd+cJQHWBB0-u{rwLGad~vUh&1LKD~~Ty1c$~>sxk?*`f98N4KK*E1rQ3Va@n; z2|??`>}uYoIj=K`>x|+Id7FBm9NO0EMaAu0r0l#2t|{@joMhxeJhhu|bi*a4g*2;d_Bl}`u`X}oP;I8NwPKWm3fif@_zFq?c)rY&9 zn&~u4&2T{rzoYv|PCfpW7xixh6_173WHDMioZ{JWndf{d>&+SDOK_r<+#}UR1p0_T zVY1^#qauv>KBQey1I#?keKwti3cyayi*K6(whjJkPI9iex_iU&VBDoEZUIWd#ogoW z0j_hcNKoeZ!M{`kREJGsE=@Y;`Bc!x`9PVwZ?hKVkCL^;BQK7Iu|PS9nGyGnYJ^K~ zxcyJAUnkyLLjT+%iFq;^E#DgEyf{u3BfVzCSYrcxc#$|VKngD&ir2X?51|r)2*jjeHOl7mO>M4dH>OiGTjOQw_R8LTRs9zx z?Oq?MFapnoj@TUUN+?_A|A(uujEZY%)(-CO?(XgoT!RJI;O;JA(7{~>cMI+g!6mo^ zC%6T-;P6e(dDngKch{O9y>?gc-fMPO?W(7{o(lGX5jcPX*IJN+`@%X}Pa0^URB2$C z6s2_9O9@?%-OW!*VK~s3lr%?pP{K2zXqOIJWM>0Zn;7X02un(qF8d`Z*S}W{uEc&+ zx4VLMS+hB74mL4gz=@NnnEpW+)?DC?-NJc8u`DE$aLkS-Ic4;5fE_nYmpF~W9K~#w zzG0jbk(gl6_d|4e-5$9swBy3qsX=)AmMF7mM2RJFe?uGvKd`RL8k*?`b+|kF2ttZ( z)vqOUvhnRkZjrVPI2Hp~uUl5)Fu+RUcRD0X5Ox*G1>E@{Us9KwJo@)x@P&~4U^ z^FN7yv;}YWhnh) z`i){=e9-j;l~ALi3JkRTZ0BX_@()k2vsx>6T;N+H1-uO7)p}!}J_U=bU zN#b{1g@aD6xG5fyx1X3_MQV53XpVE;Sw+J)mL;dSuyzi&{rfqRi`f}zlYDSQB@#_( z85|AdQlv*lAja30ske&+VI2|hs{N2qeTikO@OI5)5Cja_7(Px>*q^Z{3eZNK0}y@n zWK`SP3aQ+xQk~bj?$ikleBmkAw87PXlcSi$UdWA zK!%&>!co77Jc9j%*@?dolN6CkcMj_tp?tLpm%WGqb=fv&j;x;qkv+y;q{fBNzO|bM ziR&M9LU>sh%dF;(P`VOz?EoEr?n+bm_jmM=bA;?WblMQ*vb4l(at!7xYeenil>&xT zzkWpx?#ybzQ}nWJfAOg*W5%G8G+x2+=W*2VsWVCDOYu@1T_pnYFKx%-z|<@=?oEs9 zm7WT%X7$NxS!Xekb@`|?mYpp( zDP$zXe;UCfy`C<1qk*>ss8uE{QqM{#B12zFW}`G52{&(;sXWBvA}|yL%_5}&mjUR zfNE_!Z7f;*fZ-2zNzxEQz6?=!xM|k;MymBK@Zzd92%)yk?(#AXn4Y`mGQF#di!Fa_ zn}6#0)&G2h_yIBNt1R|M67X3?WpgEzS~ha9$k@`-YSSn0uzNy?>;2{a(D;MVkRcxa zRv+(eWa(ze8GO49HtuAgHLNri#$*DX6JRHli0uFQEgww2?w;DDrFDV2Rk9Oh}r zT|;6bMDrMpbvp*mZidZdA*4A1iGDJ=5g)54G}ML99p91KWh#U)-AIR@g54fXou5sL zj#^0}-86yjL2fxB&c%;?VMz6-hn7e!EM|o$A^citX*_RoOe&##CeT4DHgu>?9q@@d zCUUEI(nFEUJ=27apkn0bnCEAb!>_+J`c=&g4ak~zZ)=ka=5%MP^~^Rhd1_YGGTDTR zk2Alf_Z|9S-i&1=dtd;Qw5Wz7KY~hcau*)p`QR23q4Anx%QGX&OKE$!l4(2*s9bXl zGcjh`9aLa;Do7QbxzK5NWQVl5w6@A99JeirX9!#4mckLIBTs{VIwsnF1e~E z8!LEfGEFFoWCajX!yR}87prM=66D%?>jrjw?NwXV$BIo>h0`o>*22;p?+HNQ8;AdrXPqf^QxijSIaZDp8|+IEaGFw&bcf0M!yD~F z@!{hzjX(W16#)oLy{BUCAH~EeI&jF@(+fhD&Gn+=5{)XCgnz~f51+2?-Vz4eOAmeHiO3^Vd2LGeLr*sG0VyA%LUI#Z;77w>l< z5Zj?c5F3Ah@iKpj+Qg6R$^SjBnz)wRO=ab?P|DKjyJgoD{%M*pe^g@IOPbJ&i`e_%hOQ{Ej>&<_>Z8=BVtV(= zuhDbNR8$%+ujNyGY&r0g9&|8Sz*prpjnkZdfzl=1iU^IxBh_<#`W$`#FeZl@Z&rfJ zs+%+~WaL{EK2$zjx6=dRmdfP@CA!)iN-s_1KO`v4O)%k-eQ$uiDb>2zf`X&fTrG*Z zvb3IfW@mq`eMuAJ`G2=fniz}o;>*0FqD4k=wpIi@UgY{ekM$*ud0ph!-$ z^7rw_v?CO(k5GW-*jt=}RGJ096kot6oUD-su_HSI5zj$MD}rXYZ5b}lc{zDSQ?x&j zQ*z{-R@Ceu0ji+iuIzs-*fmlx5Vvsff6~|6-2dgQAq_ zZ^MMEVSw@qG5=9st9hVSxG-IyaW!3_^erQ#SOYX+(yk6L6om7N8<25DM5hwTju8pL z!7o|TAwJ1B!RjuBt{t$+(KTxRpC5sdDfPOMLejGf7~|B$%1!9den%$-5_n%op8t7| zReU2z*&K;_*6|_AA}!bn}$KjqYy) z$owSMzxwih{IK3JQ>TNNi~*aXS4KdDgUKb=pI82yOAspo9>&n~Eq=F@SLJwH6jR)H z+j(5eSC9DOtg$!O^+Ad*nK!}H!P-4z;*`sd>5YFT?n&&uIv4ZTRE`2GyhLLU%ysfFjtuBoiPUVz=H z=yy+gwy}A$jx+i(YLDk}+(o7-X{XLa$CO<23 z{komFWw`Ad+c#r5L!^$_>gYC&MJicjBLKwidG9-NTWJonOIOxTm&v0gm*2OqVO~H{e5%*Zz7H|689WQ5Jv_*=S8+A=R1Q7PlU8c2 z%H0dt0{VROEAl+rB9EH2cz9s5Gg(P@Z4qM|CY1ixoZZVa^qFZLw2r)^%%7pl9Xz<$0*fA@l&>USO(jQ6i_g6}B+%5LZ5g z>lBF!we$$9hB$p9|BRZ3=Ww@h81DASP=<@s}6ob*XOW zewhTJ-0;8IeukfSUiA1)iyVr*3)~xAr2>}3?x_Q53Ein8rWJ3Bx$zLMuzSe6%Qlwd zt+@SXQNY_=*Q20fAFTm5!na`*lIgZf3!eb#%9s_Jy&{tf1e|8S%UZT0*87CFzUb48qCmdMV1TYZ3c2;>y8yVv~ zze%>drgDEzb9kTknP>Xbj#fNH=Dz#=ax)Bm8?fH>o3C){9e2C6t$dq@Z}rL-V;Fue zK?cSV`mzJ+Aiguf-rUM#OifC8&Kp3`?v2ElGCb+`8DpF(2PdC^Q;XS*tK={dZE4Id zlDpBoO2gNJaMBXRCXO3?wy$jVjxZJL_9L?;wH~;>=$ss86IJBaOhO_#x2Mz&3I|oH z;3_+QOjwajAq6%X5rrcf8cmEjCuTg)X`IZT`%I=q3=21aDL3bk^jg{%(B=N46dv5+;Sj zN+^n+R}-t#PVG6=i)SMSax^ZCpC0}mRixYxd@|RX!;Q~Gx-qSmczwTq@M2hcN}0-7 zJzWM+&0?*S?I$M1dDG)>;3xs4_BIHc%3!IG=dO=GHuCt(&Dg|`(bcb=qm0JoX3LWd zpUI?{z}G2V{M=#b*D|8q0=x)gIb) zUI$*Q!!s2$;8bF~nS} z$=^mz7DH!TPz05R_=e+P;Wf2)a%42C!ZXTNS;|O#q3+r^FATHyJYkA#D7b~R$#(~L zi6ce?B3)UnhV{qNRJSL`ak<`UAnWV;A{TKK$y$n!wT7)x!@~ePZ#G2q>RzA7P*cl` z`j&%$HLq|hLyu#x0E}#q{rMOAK?wDY?m8`&to}rL6BG>!_aN?ncBghuLBq_!HT`IGo&faW zwM$CY8!;Jle}Mp?Ub`gSffpnXDs<6gB-HT$qqzQKBs-nZH=)#sLDZ&VDm9Nnkdm%~%hrnh(K*0Zeg>K}@Q2 z5=w>oRmGQ|s^+6#DDed;`OD{B^$O+{^h)M!*URP!JJiaWuRD}8@0!HRV%?LKvj$&X zkjD@M97+MWIeH~FyB6Nid5h(SWs^8Z%v`gsRemlvF3V4^Ge(EM z!bZe0`$IrLv;`YK1L0#aaG;c!>%JZj=kAYBM!^i|v@Jono3Dc8LaKqtz?t7H^?Jdq zH}`#H@-qFU;2@kqk9)}OCH{6gJam5^_hAzzyHGm@ty9oxWH7W}z%`1vQF&_lW6yMr z#yT-&iZ5HEzAdqp+||CyT;pF`T$Nefh34wh*Bus! zTPp&qDy*b)Z5dSV*s1#PaDcHb!McBbzMfNS9znU^%t#IqF^%Rl#H)uKcvO8Su( zX%d>esOW-Q=g_x69t}^BSiU@?H80av-|;L*jYGH>ZR$a7RNYeD@}gh;qMx(7`CF0l z`xmr9hlH_asuLQpoS4HE610J*{!N}yAd_jf zJvrNS)Zmunuo>~&;4C>AV8GZ0#RI`Ch5ig-O`lkp>_?-nd)+AgoA+)HQZ?3Xc5?cp zL~z$CXkqh(8A`rn${}h~>zY4OIB!G%am~!uRPT*_OyvtswodF?ds)`JGxWdtshpF>Sf$Y8lDbb^;oPomfhc|1 zW>MUZOHzH0#ad zbbhv1^-<*(&mWbKjd{Tvyit!- z?RR;TH7lHdXrmHx41Q`EH`sG|l2sF!l^$7^9w~F4UCW3mZSvBY3mEo{CJ9MyFV8mA z5d9YYgRZUIb6r|9%(Uza1!IbF_mmqUrV4GlhV<+d*Ee8_eZl4my+z>H z?7_cfx%dTtE}`v98M)A2KMY&T`(z)E0jR4mDAsJJcVH!~HguUOgrQq&)4oc^FESq6 zop~JAHBIwPPlfzyENAEyel8-Q#;qF(6tl9S^ZEY$i>E&_j{i#S@IW?zQnct+r_Z>O zpvY@i1)rndggAu98!ev0q|SgQ54=q@-u7#94kngz#;$zEIUaa_JtFCyXYp~AQk~#= zc*z7d<9ltfC$KP3Q;U8NcjKA2|D5^#_j~ zN`))BXOPq%_vVtA49pLbljUnGdT(pS)c@Cdq5l%oFdb11c9j>K{>LdvR-cDFF5`rM2(A( z^~QS9dv?Qkt~L_DvW>jWb~;x{1e;Z#ray*s(F;$JMd}LTej1zV9^tURn;?H17|; z`|)uuFQ}M|teO0HB#&c(H|^)xB_cI#ssmnqWE8OguOcD8&B}{BP#&`Q=izr&1kfR!VNqlCvyq4Ebb=6j_;wEsjD04JJ-P&C)^lrJlugsx5sMrs4TLnWsB)arvTez<{4 z*>F!X#!og3-r$@IEhQ6(PA!4o)V;obkJ~QY7F}R1x;@Yn3BMavay6$FsFE)o)qnA@ z*Uw`K7v)Idf~1+T#*O-hO_~|PnHe3%jrK_+?x#jjRyk&t4Mk-aypRotV+j#@DUF;u z$~igNIknh%iJq{W2fzHCcPODVkkBPJZCls2f5}F`C_nW9qgwrKZPa>m?CPt6+NqKK zka0yhzzogY|4Dtgj7?Y6FoKQiCux(v9&-30JUAG*)_HG+tHPG7asy7Md$ByC@) zmb=xI4wbRqS@Slh&q#+)7E#_FabBs~s#|xc%-Ae#%1|lw(BK>bvv^He#h8w0HRfr( zzX?c+!7)DS2-cwPX2#6%FqT}3WF>DHTOg8W0n)`E>|{2O(_>#!oZS?@9%QL@s|q>g zf+yzM$-$*1!1Y^QUL0Zm#^j?^#ky1Hi|Ci(`P@)AiDhXrh49%tauVl{iRd0J4le-u z6{|AjrQg6`OVnlM+lYsRCfq}jB^wF75BQ)B81_Y zfVz>eX$w{o!ynvVnn%9=`e}8e@#l6$#OvDjr&({ zKpzeU8w5cbo)Cop1ofbVSRq(Y{%>4(5H~&<6&{9KqQS?&A3nQPgxXUbdbO4Pkyw~7 zL?8!zGHSqy%sRVi2_#&l=K3HD+$XqJk}3}V5n)H*Q)mH>R3uy2;>AndbP0C-ChILL zs9UGpld_qARtQ;h*S$DNJOvT>Gct}13En5gLrfv-o}gu9D?t1tTwW!V8ctTB*d|3z z%=`O>nm>82f?HY4dy$;X6T(3FA~W=S>7Qw(#d`q3DX$CXLa#PZabt|c&RSuQU;s*l+4c4Oh5;}~@)cOY$sY~*u60?y?*~7+1e&H|a z^Gstg$=peBeiHY|xgkQVdOBV%xJaxAYA$MFCxt)@xZL)&eAdZpL10JZ@rVKG16xNp z`D(UO=|>^0{=3RFyVW)M;g{cJ9ul&(%monP9^#;&)NW1IzjqC3K8 zAXgx4H4XVDfsxD!WtwbNJi>qy@kPpmvWe{10Rsf5kx(4Teo>R4uT%0HkfM~$&J40f z!#H!iH9Lf?dq&(9KB_OUe^IXg{lfnXatVW%Lh~CQ@Lz1E8+;Z84h(_~p#LpC!T=&o z6vA!IB_yNx7cL@XJc;)M!>ENYf1w)^(AFdlCJ3EU1hrK$iYN6yP>{DRG%duxppRHR zS|ijypbz-yk}n(>M~eriF(5&Maa{%x902FPs85N;Kj*dIN;fhGpXV7_n*Ih86nk}FhdjU0yt~N+bDKv zCD|=>fI?$i@k4m{#W99^$PNMnJ7`QNFy|9?Fv-;0+0}Lj!3X%J##jpHQEHZ|4QWF; zG2c?hvs-MZ>NKM}&s#bQb)}PqDwK^pc)HK7ef;7;<9d-QZWQjU@yhJn$EkNw-`e-) z;?DH(7{|xWnJ4B__ZjA?0q`|vN(=Ml0Q6cv34(G3e994eo|8XMKto>W!6S<9iHm`+ zA%V18q$U3J#6VS+fPof#f7ryuuF7LOw^bm>rZz&Op%}R!rq$Jf`J8JH@A!W8J7%!S270X6_1VIq9)-UBnw`0{DAWw8u+k; zA3hrHUGB&1`?^UXSbMZ-VV7X7a$}wO+l6$Qqb~Sd0R%<-3}nN3bM%^z4(dCWh=p(( z20qoKw&TrU-p%C!Amo;YOV~W2BdBP_XQQN5>|GM`#+KEg0#P3a*s*;t@s(#z4vX@h zGCWBJzcq2{LA{do{_-0!DHK=Hk)Ts6sR&HwmuAu=vjWrJAdA^RJ`S?_&*{6PQNwy| zzr!;yNK>0aMN0)o3*pQAhrGZl>@e+_e}OZE+CET$e@s9INI_y|v@*{L>KKMy3Avu! z=OY({9T8@3p=WbW^-E)C-I_YhD!w5z2$vD~2N4_Xa2WX|I`G zXFx3cfpMhwG5BN{hHX@sH7RU**Eiw@Tu63Okxkx@LV7Jg^^~*FFO-RVHz{VE^Qj^a z?kNg1R2ebL$%vtp9}SL1urE4j86}BZgD1cLQ_Jta%JP4WcB@qatt|9E8r7jk_oW$Z zzoq_Dq$pup5DtWUA%ND()r>X`f$9QAi*=m?%{IZM`<#*g1(T6tc@RLqq6A1=FYTZo zVg8YE5r?_?tC=&fAPjW~(pKbH9yuszp}$bHJUreQG^!xDc|`JUUIMK6LC}BImlPD3 zz=P3>N=;@7`LE=6D?~~7f1BR4WYdt)e+RXmCGp@u{xkk}6Sj9Kc#^$;)SncTPfLc> zdhZQg1NE_f|Bs7A>coK6$G&b52osk zKtuq2#RYFQ{T!mrf%xaWP|A+Ic!D!k6wpCehku`%8;=A~qP^xuq)qTnVy9g>uIEY2 zR4rjcqNJxESCkfv)PP_^QpQui%iUA#X}#LqRKJv}CD9pmmQdO{L)0yBO>D`+X z1maxq98-E-3>Urb#d?Qen{XGDby3j{F9%tag%eAeeoQy`N(Q@_Xv2qW(aiJ zUcy+9bU^`NQQS&72dpllCF! zCTXySb`hV!o6<|E4aV>PO7a(Q{ry8*SDDQ^J+?D7LkC8&oYHspq3rSYt1J>mAepZV1d-$+kC(JXG55YKQ+E(VQ~{B9b(uS%(x{}N zVt7f2J!2<6hsey1>$QA95u0_EADAa(ZRFHlXJ(mqll*e|j*vNWrLZDNZszVg@&xY* zhz`y@qkY*E`8A(c)1Wm!XPc(YESk;}RinL{xGmA1!V$0EkbP0E))HPqLjre+5;V|R z>5VddnqCqcdJ-d99uuMtDwIUx<*$`WusX?gn9#Py1sX>!fdQ+>CZpi z;hQWkF;Yzw6E>6V3LmenRs6CzGy4G%!1QbELNNe;6joLtjFN~(pgKF&m6rcpo>iPW z`IE<|APagj$=8U94?SaEF;oc$!;ajAaBu?iUn$mO{rFoPcADH$Xp^{EJ^|8+Pxu%8 zXa&mpY zEqHaAw4WIes_QfGo-|0$+r8JGcnk<*I_Noj9KKKxHT3S~D?(0O8s!R5EKc1kkPfXe z^(WYFa;2}P6~eTY(3PStUO5_hpf?6g7Wobp}- z%RZ)?mLmTufSoWxY%l!jH_F?JFA=I}U^JTAa2|U!O-(>l^aC=;x-sVaF6Ubxn5|?- zpfI~1luoOFv?P)ufFW(fJ%EdVErlta7)(VP$P`z=tZb@f_o!ycttw`5Vj0v@^nlD^_Rt~^&yDP1@?G^ zytu?Z*xB`p;J-7CEt-o0;E6BXqFa#8(ySO)x)W%}Q*^Z$0O^uR$- zD+xL4J2dLwX5IE8-7V&a54-FjiWORl)`UYeS}f>)ghZ6^m|*^GPIn+i|0m9A-K6D$ zMri~O_2J_PQ3pFZPM}06Jp43XRzgthqaZvWM??QJ76whT=e0EmLc~^b42nY6UnvIYdaw|o zyznRB)_U|L#|0R|^1OV|BLcU5`>vQ7`Y%f~4;qdTHxVU_FFlxyAFBa?iN3B(BM7pB+bBT3j1J-Vs-JymuYmB08uq~( zqr~4Edd|fW(wg2%hd6E%}?UU7sHzYc;lM%lI7B)O4Ys=8*x%nK5cd zZE0s?j9Ar~DPuHj*A5)J<@g(Z#K+kkjEt*IO@?eUF=*pPv%N8qo#{L-klnO`&Vz$$ z#9Xr>yR=X2R;Hx)@Wb!xagmU?O3XVwk#OGDI~Zw7%$3_1q+GYJorJW?W)Y+MOVVm? zZ^{zC(*bmh{NnWG+5|PEDK!A_aT*)1ni!L|xIko?*({f$e5C*Yoz_pmi!oJuN3V~P zGO-ABI#KpnMy%C73d~!+8KPne@gF@mjea?NlgW~WAgN$Vu0B5AZ@B1IZKBQ{&T|=| zv9=ZG?kftFC6`0>-xZziEmWIcTZaH#4cKpQ4Fh2hc8P0h)cJS`SxJd z#BPsNx+-+5EJ-@UfItrrk`R@h?Gxk}?cnJ1wcS!W@68w_Fo#!4+v7J4*%-BnJW9x< zjIgRiAF{hX3B#5+q$Wev1ms-mUDe)-ck9SO3m0yytWUWwCfa+KI z3vB$4&Vjw|{{kC-wQ)g=83`7Ayk)NS_dCyLc!a;#(#OKQ;6TPPWa1FWkWr3ZkEyqHO_)~?4+bG_GKQw*c0mp^xrr=`wF+0(#Q`J&)DpCtT^oNVMXJv}`mJ%;ZFa77^& z!xd+Kk$Jm=NKdj=E31Fj;3la~M4RcR>fm<`|GZJUdE@6&i;mBsL+*H1L!JAKa6&h5<}o!d=AxW`KPe>BL`FFrO-gS7IQISZ@zd7GsSm?? z7>Nnmekm8$v@XKiW)$&DxE7HBS#l_588I|+54VW|(=)kSvuf`lL;Wq^^U zI0`M-My)A;S|8AG9TgzG1TNI2R>?>FTp<;?M9G$=9KB?df!@ropvHZNk~@v*S&wIS zRu!-kzEB%KZ@>}?(3$*0EIE;md9)`6L71ATfou{VMUBeHUsM>Qi=$FhxWin*c)(t{ zl^tcpURwz4h=O6R&`Cm^-suXqz?t6m!C~8y#=*P9G_ekU5g@}6)PkRXzj=Vi>J!=1 z)~hKZs<{~8BYkK+a;}ryPevwA4_ZQzY|{H?X67#@+u3dn$XJQHyE?CK7t~ndUevRw ztkCD(OjmtU2GJFdo4iK|FWK7y9SCxFonLJ*_hB4SLcMv6b1G92-F}+JI_F>K{Wi$@ z%2Gt3C8BFy3)e&KweZ+wr5lhj{fkD!^6+le^Wi7K^c}H}{@eQb!?IexjCL%mYG@6X z0E5eg5YBfVk5$tRW2Y~2k(`o z#}LCcNEOY9QCxC#SZFX-J!KKaWrfF~GT#r6s4MIEHts7A7B@`H6;bZiVKywR`G|Q< z_;>0*!Om!IWj>@k9J!O-a6d@f!*Ypw=0ru;m9Pqjq=%KlRaAoX{-TW+%1u$B+N6+Idhk%mVL39*evSvN&d^{7#AU zmP1xw7n~hTWfH)EBYcfyD&nVOvmi!@a(|Qlg~u@gPkvacaaPUwUM_)2OD7SHIOF>@;GlPSJb%7C`CM~if1pgS0^A7!0pksA&TT%Ogp`%}oR%vHuWv72l zQ95NS6WDNQ6Qs=Y64X{UNmC*WzF?x;w9NEvu<* zkxQyt2L95UBl*03fn+fYD_OCf4q)|?+VL%m==kNxS#I%T2V2j#b8?KCtm|NwiV72w zVqn8u4$dKxn0T5p9m)?Lz5hkD)c~jz7#!W7a?ReLqA5CmaiXW#y4d^#G!sLoPh|BU zabGRA8b|I91)g}(5mK!POD+^nG~D627|qjx0P9E(7LcW(Hs$8YeKiHTAUQ1yt{~=TgBwp&_%z ztFYU9G)F^P9CakzY>55XANaKZT#UVMaE>RVE<25VH=}v>kpUPHgQ9Gh(@utl*7jEv z5>2>2vOpVuA0m74Nak2ao`b_#9CD1g;A6l>zT4s=<_^m@*F#V4S zIU%CXktb{K;u5zT;(w*5KF$lG!89enqMJ?aC&yPt_sTh!bRTYPzvyc?3V3i7*cd2;?^!mhU|@8e&KqI{ZLV z3#5gYOY&cP@xLtzCfNV@1mI@^8StY=!7qJ4z-3NY zrfNOsNAH6m``hsb#uk5o!-xO6F@J|>^73EbgD$TTH2^Jk@@8}%r^~4FziBBov6-Sa z7E99XmNd@>s~gY-7?cA&sblx$QwTV?6zml~OQ`3wvOKHjGg}~(*fVg{6B72LT8Ldv z{29h#>pg}*-Tq!g(?B&wd)9~Mb$-1GczoM`yHdBhDlfJt7tm`vo}whn0rrLwrvaj77|~o@phL>3v^F7*hN zXqEJ0eCsauh!FIojo8%IEN$0(yw~!ia%5136PUy_ntLWDfmRtjK{VP?uIHRXIzpSR`ho}al90A_)+ zsb;aqGh>TOEegkFWa61JahC3~KVgb?qXM)Vg4A-kOcH{OmrIrOK6QzmF1T;`OPpuOm1*7*ThbNLUjzo7xt;(dx8Tve55Fhf5~Q7qjoSl-^=WiNe0 zE;NyNx524gO%Xgmp_3w2PMT7?G)}XhTXbz45)IwTyquM;md07>yth~-A3aN}2!J?9 zWk@3vF%e~UP5BL9tY`;nwAtvgu+p*x6_ifD)Q|6eCMGKroSaoGN&C^4;#i!bqs0e6 z2P*|w80?2#(iDv?w~+3*m2Sjvx2P_r;@-^ zWMNo77m&2bNt0;<`{a50#d%^`lFfncR#;c0Gz!{QKPhW1Oc@h(Cjg;#U>;LqnYIAV z1YvEKXF;^E;GTJeE)>O1Gn6aN8?KN24P;(~K+C`?zc zi&}r{6}P0d5-nSCCrkI5DrbcagG_6c$2c$FDx0pz%BK|PI0wMg$ z(6ux4V*430AdpPq0V81I3JXDONRe19`O@u0qMz-G2^e(37fE3-jkK$$K5e%T{6fu= zJhzu@4015e9z@pHO2bF41RZb<3=b33!JjLTstzGUg=Ql6Bw1Ai=%ROJx!Nj^&1H?D z!#h?hcGgNsdwlZ~AoRuCcNYY(W|`mXTwXR@EJPJM>X8Oj(iCZ@#Qd^#vp25TBa=*S zOsvWsD-XT(wxLqC>)b?INZ?Gu-b~S?IfVUOkK3KOpg>Kuu(@hD{$8GQnMPNnlk%qV zFxo~qouF7s+2`z6*B6ath^V-qfX|&67~;ySqI_B|DXk`dFyBa6^-2sd*1yB$W2G)O zU@6z(`B)ub5VJ20AaO|0l1BW^Ek%u}o0-r8U|_wKD7glT|CaVd#N^TJ(&=fyBoX$R zOeq`m^UCL5Stn0~lrFRmS)x!qn9s&kdBzvHw&fJkHyqQ#WXDFKGY^RxCHWbrkbu{$ z6q{Dm!nMT5E^TfRvO5P@Q`Y^Kxle}cDJ>-BSODMA(JJ6=4P)P>AP_Tp(Y&YC9}~f6 z@TrZFOr?Az1?qM;j8Vgsy9FGIA-GC*QP?y+k@U!54mD(CYR&n~#t7qA-GvcYie0`| z+z!<`^R!rT=g?F-HQIw$F9Dj6-+rt6nmp$!Nm~S~zvpe}#!@o0?o(qIg?<);0rl z{}iMnrT2JvR{8(q>K%Y1-MV=1*tTuk$;7s8+fF*RC$^1=jfrhdY}=XW=Dg?BcW-@F z-BtTpdp~P;bwAa6ZT!}Mh99x++RK)KXs0wd7erqMOe5Ge2LL750)b*9!h7Bau^j2& zLLg!rT%%e}RSUA`2tH_pNAidVwobo@)SLJMJ(}6~8S5X%xjWV5jC=@Mm33nXUI^Sl zJoymP@@JY6bg$m)vla#l@c;#hfXt2vPx8-$m2ES-i%Mk{+SOjK=g<%Rzv4Ga7dM_x zVbd@K*Z(YFl0!oMFBte<=}<%af5U_&nm5@un-Ue&@BLEYw0uQM;xq|Na8$r$M@mo- zQDHgN2l`6i?{pLtWo8Zx{zbG*9TSLA#3G8iKPDPoC-6@~9 zr|y|I=KJI883Uk}Swqc~g0hfh$}Po-zV&E7>Q0`d%HDdjVCJq%ynBZY*tRpDYoC$+ zm8t13ZU;}ke9!Sc?FR+-VHAKiD1qD1N`YdJlcmFf2S-ey0Bt!ra!Hdj`5$Ri4ifJWLx(nC;G?MBOt+S!7 zjX>O#cheE;b+1`g_&J3zgxT&x@I=USaHk~&MTvIDiO!g>%$l2N^}*PM%r*YuF@!Fj zjdg1capPTH975sLRYD#xSs?5ElgxpyUBPu;tz?W*%0gE61FuzqF)2-an(dEf^nr;9 zmOP~$)lgUAOZBUUL$=9dQlr3~zT;h6nqy3Js%kr#Kf^Wht-rLdPzFV-YSq`UcklsR zygX$otzJ&t_cSM4xP^KRPFa@7Vl^EMt*m-Pm^^a4F!Kb&x|p89+gLDXP)Y*=?el-j z-~Znu@;#OPzcJ=N4-d}oYx3u`U~F(Yz_|*l2HKY%h%MAPEeZPewJ;GxwKiROEK)K~ z9|}S2^1NANpLK&&8oc_YN|l{mvw-$nivh1+>@apzj#XLSTE?fs<-_!E~33^MZ<2ZIkj_b zPqD4-w;CPFBex~Of)ZP%<#}50p`XNs{gA_&DWftN^Az{2?6Qsg?DUX@uw{NVfskto z2}rQhnwF2Ho3mWh<>*w{Pt!`|0NypiT@I~RluJu#QZb5Pr5eplRg1<&atTRuO~Q7S zkz2v-lbAQL;4{qU;VM&e*)Ygh0WbE8Ot#Ywkek2Q78LgoUfgx6O%j%v<@h&qnmQJ4 zxkOy-Jl*q}CU->Xein{}F{wxnL)bnT@^dg(=h6y7>p)S8@Uq4(EU&#T021rr*ZEhY z@ZoTeky90Aba#}hB`p+Y_QxP2d4g3M|!%7Py8t zGjriN(*+Kbi^-Z?6*RXD0CMtOYKRpofQ;NCPV{_b3O)#} zHt}rX_-+FBCYh}E@ucIO7lU8?rY`ljLL51k^+Ks?iMJqOFGN1|qeLzt zy`}g5iG4Qpv^`Su;l95=oZLPbhGe0+%)&KRxsR5ciq&B;1!R38s|5%yz38mK*YM_G zEl&!3su|xz{o&0|=lNPU@^*$r^&9kF2rHp%0`}7_ThBKlqYrHz;xcHsYnmx-z(^IP z6XTalH>Vd*vQk;%d1woxfT>7!*N0~BVsfqA4Ml3|FCYk4=TXIQg*U*6R8QJqp@SS4 zd062od^fOy0Bm)NVha>(JO8$$97`z=l97an7-=`_p{_nEB!5ZvH_MW;Z~K{jr{qfk z2e@)nN3Pi+lo@f_kTPwa>2!z5g7o|8*|l!)0~#{fmvoZCt79$@#+VE2e%_x#d)B23$P@Qu#iCNmMXYlT$ZiW-N>+qm0ef>|?7I8gn-3U>uJ>59b z36?gL$W#kQdXyBFHtfib@00fJh}YexR!u@^mb{@mMeXSM{;P6B>*3mX#Ua6DZW3gQ zPt@Nb03_7^#)I#pV@Urp`u`#N|BMs^DZfbdzQ+%A_%xwvOu{rIY*?h$AE2-a2 zSE~0X|Biqrz|)izkqFcBkzv8pepK=yw~Bc){P?#Q$aS&Av_Rhf{G@&RL2hLnByOuVPe{8(+IkKqS(>6Q15p}bE+%xVsw5@gzrM^$FsC7q|7qF#GlJoh83-ud1qg^R zO(2U9?>~sSTMyb-b!q9xbI-H+gsBC|*rcFlh$)IYTmwxA(eGc9t3qUCpkTs5TAGXg z%`n<&%hnowp>j^Y8>omQ_U0>_*Q}q#s?>FCY?>LC)Wz;UX8b(1WGP_JR{THx4>sGc zx4yTR-ngHqU-lIUxuir%4{3v(ZE(YuBmh}it#CG0wKcdL?G+s@Ee-AUE&k0}A^GI! zHdP$z@Ksfn2K=^9c|q*j(9{ z$O`QeghSzM@!wqKKY?&|f?{3}Y)EqE%hiCpB=Pzh=&nj`Y7!HhgG1d3dp&HjiTHquO( ztEJA+8YsHYJ&dI+OdP?SWrf_h3J%~+R$Ir_ZUwcQ_z=t$IfE~D6bl>=((>~ce< z(CKoN+ko&r8K`x4$_m&vB$C7pO{Q$Dv%tJ=O1pG>ai^}Q`#aCV;B^Fx$b(`)-j*~8 zwb&R+XE<(gajRO`+O2}}(Q=u5mUIoVV}H}>@(d~bJi#NBD|Y!{3xMzj#`m*O{$VfY zwjhX9zJFzNo>dh4sOC0ct`1w5ZFRpH((k7+j}@d?)v(DZ>`X zls|^WL84Ug@{@CQ>3?aMTAG8OYKDKb&xF<|zju(9vD1@7OX6FXaPf=|JUN1AQk~0m zdL6i@l$~_2uC=yhxIsc~u^tGs^oz|W2G2Rn|D}=!!M~+&Zz&)GNed7LKjcT$9R$T3 zK)~RMKE^y;zk{Mn6^2;CC2tkU`Gm!GwVKIiG@hmH%RdT}rl0AC;ZgX#nxZ;oQokUm z!K+46KQ7T|*_;RQ=e;_#joKFcaoOeU_Xf32W|4}vdJPB+U1Ymn?>tCbF8D8kXm@T& zMR&|i`wlGLwb?%N#sYx8lx=%nX89k9*hP}jbu2vtaKCcM%B+j6sb6Gz`@7}+>QxR*S~`%nDLH;Pec1bQM|$R#(uU4OH{T2T3DF;JH{d0 z*(21=;yGcHhjm3(?xN3u4B2%G^J`@FjL*yJ=2pBO;3r2Bf{g*7nJ3}|j;OMy(o3AY z=DRp&)<dho2Fl{$4QMkmkIK|}j*5yFA)Klzt8B`{iVSnk)g=k?QRzStgHE5+!Ey{{ z$<;dMrWjg=C8uLKh*SGOC{K;VPmrJR5*iTxW&z&VSIRUKtErRa;xqRaNQ_;GriQB%nDsK4%Yyomr}=fXcdh$<{0y zL)YFv$G`zjE);+`brk_^dkcaO3xp?PxgRL5RwncpY$IX21Qu>j{mmCkMy3YfFyrq$ zxz{tljLz)JJR3k|IlJQ?IadiY22@rBzGAl|3wr0b3v_^>a%5Jw7A;lSNpg#Gcm2KB zw`O?Liwl8Qq`RUc;rk_9g)MY@!UY1{#tgRS7qbDqEK&3aoOO{$oX*L~y{?}1;h*qt zS|kW&>^)({})W#Zg75zC|i9`HN}PS)O@3Tw4Rg}jFF*Y8k=6Yo3COWbon zqyZH1tuj`OiprnA@KcU#{)di4^nyjh3gD_x&-5GqIMsOVH!$OaJAaSXhRg5F?z0M4 zO{D=Ypr9*Tu_8=Q8vTsQLa>R$Mh5N9Y6*X{50EPBJGvk?P)CJ8LT__J`ov8~u%b24 z=e!@l1h+Fw&Eo}2XX|kPu1b|OECt<%p0UrJxi#WsS~M>%LG$n`x$qpf=z-aLCm4W< zjZ)r;wu!*#w3k$HcWlxB4FZpfa?i&*HO>IsF?49wiT#+{ro0;hlBYpKp6z3pyIJ^g z>PK;8XzP18YONzF^HHcM+SVB{kQnyYT$ELgA7_}H89BSVa>VJ-pIO#Xm?!~h)Eekf zz3PFUrw;g3)fY!%-#&1%Pt!nWh8VOZXv}2rkAM=%a03fbk`p^kc)*ZdX`G`rDKIvG zmDW}Hu%BV4*}I`Kl;7N&K+@Yv5B=Q9Jse2=JM*E6C^ZTO^AaPSIwEyxy^vp!&Vt(2 zK$Xu~Q5MGPpS-Nb0vHY;d;ca{uCJ_e9q}Yavw)HdJ zEE4KWgt~I% zs?{y4<4GC=>nh(oE`H7DBQ~mXCrFazzk@v5cCbF!I#bD|sVU42)Jad1_dQ*JcmnYk zvF_DA+^+O)JjlB;?2W--Vh8~Co;-fZ#+8SV@0cIP$& zUn#L+5bPjZ!&qyqiq#&LR9mlE1~K_4vF<8s+^o#Ya)wPS1^b3e&V#g!Z&^=G^*W<& z15}?h5M+lImkl+4r<&LSHv$>}kYehdG%uuzbM?tatD#ih9XmEO68kzmgJIyah@NI%i`KH>L21q-a+}TyD6j+_*kV9 z9i;cDFu91NC4_!({{3-v^xSif@0Gd>;P%9gmoqFuH?_%?DT&g$o!21%{!`xv)?$CR zcLKEMIe6^F?=ZLfe#R?|POO8#xUHeuv8d~P{yPUJMLgxlpgRJfrD)O!#UZ~{us7IL zc*WLWuiak;oFIB!G!#}@T+=6fKvYbKSC7oRD`t1SWa^?y#i3>~_5>Vdl^J#sI(!>$ zcEvs#hnR?w5`KHgw^*|PiPhOrvk-W<<|!$sItVb8Wm1RDi$3e~LjF`s9#RBp0c!t2 zOFnW-OT@d@rfBO(PhGw#l|1C+rHzDuc#c6*n2z+NeS!DAvOTS%QI`wnDh^tBzNzVO zSChpFH8;j+R=Os_O<|D}p3E>k%ruWUHeI+JEvw%()`5D9P*{UTO%i#71= zGb5vxXC|);_S-YEE>=Esk4XlD!&R@n)a+Qs9j3rZNMds*aq2>%qVG#(h^ZDw0b6uV zf>Z8SWJG*~CW(I-8Sisvpj|MWHsw*YvL3engwH;ex3UD3N2?R#-BrfRM;0wIHlCKj z!D9E_B)(?ru^(~@LnkIK9&mCV4D{H=aBx&nn5z*lS>jojMZ6M3x)ykRmWHKvWWF!K zzrxc`y~0=BX;f<8D8`GXCe(Jp^JZzXPnI#2>n`-wqEe|2M@^>qV<-7xrS-UbhjXf% zsKNW-qeB2RoNp5;(;9VJDO27K1-&Q~aL{NUPx>rB#x3{Rid`J1F%SG<=af)Jea);)%J3CtbFpfl2 z`89~V0Tu)YJFgR%#0a!vNxL>vHN!KpaP<07Tn zF|gFWgE3f6WlQXivo+>&_Y=5Aa?r!QHGyCZt=Q z1l=nk1eO?&$GzZu{UM*N5#1(2Hw88BG2RYH+Cxk@<6Kfk2bK`e=hGfcV^u2m2)R+Y zhcW@Y*^kHw?u%AY!g^({xp!JA_Y^ZU^yj{o4;c!;BX4H0G2YZN#y)d1vEqmm?u`Vv zg~ZPAE}-Ugw5dH5N1O}Z0e}A}h%Zf_btN;bybbv>Cu@kJtaG*)TX|q>(2f7H>4kT5 zV~SCnAkdnts=~G(JGN?8=>-3wn8+}>`@qXMX zcd&@*lQ7B%R785wam#-qBEGP2&{uga7zEDSnKrv41tVWNNAu^>AGyk*lf^Sf$=6n$ zrzmjM>cB< zJ-ZtO3g~I4uQNNvi0!xsifoM&d~h(h`3C=K)|q`EaDVfsJ2*p zEK6R7iUGsl+qDqpRa|x-sjMf%1%BBl$)RUV_&bHfErrr^G)|AS;(GPE0rI|pb)N2{}= zr?R~S*=_;Bw^6mSKEJX)weDlGN_m>UyRb#T8d9Ny5?x!V8v#4=^TMUSf`UU2-_Z6C zQm~8vih2bk@9Iy~^3R}Ifj3WpT2_UvrB!vDyb5^h!03A!6TX98+vPpR zz6A5T_u`@+B34-5{?WNEqk@cyyvn8oMTy~F3GeuyN2{A`(!?Y5`ud}Q?V?MSYxBft zb6Udz_Fpiz(GFKHBx2%oW3$XOkpmO0+gvxk9DGYSA{Hj9cy}O3<^@4~+9ID&3G{p` z;&F2?0qr@{5on>pyXbaouT&}GF-58^dFCPgVwCIU#=I%n`C8Emv)Mxe^V= zCFEKYvrsN3+&E&CbK_J1+mcH4D1st(M2RRYWvB#_-J+&cW=^!DqRSNg$rYaTx)L{{ zZdH2oumbHaT$M|TpbsQ^%AXAVaq5Zzm3;@+Ox0njv(^u_LN@eE^E|J7;!h++_@H#0 zQk-(AAU3i$G(3xLW3^$qf0_$GJ8^#3Qh?ibL%0W;Yj_uzm#1(5_O>x@E*ea8OLzqr z?D8v9CL1THVtb1F42**@>4C2Mvp^?Vf0d^6RHwVR&rH6;utJ*rTn(aMXne7u5KOuY$^SCiNmn05@~l!3*xXb9i^{Nho)fduN=i2m z0fhlVs~5aS+Z8kiAhfMce=$-p0vt{Ni6_OiBiSDaCY3Js=cN-q$r0Ubb}8USji%^4 zpLr!1-e5r5ARKQ)5dHz4=}dg~hN1(6R$pXW=-R^y(l6e9!M!8ceV*U7F)0-&;}XbN zMy4n!b?*T<3VeGGFE<7po4QVU)TD`D%dl%&ziV|O4gsVQ5YE@65%^YnM5V5Zw^`2p z7ofhzVYFS&MkBH zPCS~GF}#^jjX$RF1`m5v`wR0y&2Vt2J-P`koLeXP#5xtqtv$MsoM6N8&Z0HDfT#M; ziwFAec3&|k+C!er38A%+fi)}-8UEhitS>-iIis7R-dTRfBJyKNTsh-XVeVCgcOUX& zU|e~c;Npxd%A*Il{WU4b8xZLixa3zD>hYZhDND;V}Mu|_ncxE^N7mAVvj$G+Q$d=;`NYK!5)`t9+6y_+qjYfs zpRkaXeWZx@QO8jV5lL1N@LnYlWBi}*nYMUFLPzCT&_Kjouy#)%MquXZq091lKzPTfhCfg5Nx+tP#b#(+1D#55Bd-si=lk z64M2XW(jDgBO5z3FsnZ)n>IP@-q@=!!9A3vk=klEogY1 z5kXBJ=-pp5pxgEHA$_sY^cd;bV{otQnmVZ1k%dbUjrYY_xwcT1Nf|?^MZip<3>YNK z1zm4j7MI;P1>CjwSwObSUGxJY>JC~B)@aE`%}T;3Ri+PLgUhuBa7%ZY>z0SfH^WTa zf*R3A+w%RM4kF2x+L9xBZyg?${pMt;3rRoDArnOa4t?=g5BIU08<^DYJ*PBPqMPckczEH-yGgj2Qh9=t(0r>`wlPRyo8&in_BM&}(A2 zWA~zPuIul`WXO|hB^-od8ckON@uv_|8(4+irK4URA$90ap-0xqKtVlLdUh|P2a_Jd z^kOvqugJHt6gtUV-RSVRHNqAWZNvU(L(d*e^d)f*HNI?W{uo<3eIRHR;%ggIeM zE75uuThV(GF^>c@Ji{dUhTt|7l{tIeNVDWNk{V;*NZuV7O&?1@LPO(~(yEG5Uy1Q* zW@N#oX()oNTV#zr*YdWJwH)5>cFlcV>F~&lvP!LilvB6gU=pfy=jL#D564P@Q62AW zda|l9Ct}A@QZ&mjzr}L~%y|ixq9h7tRyPQhB>BXK_D0el+2qh?0@Ngl z23?|{_K{|t^@dCU;m!GB(y8mj1G$>TxaW5Xv;r_Lp9*SCXq+}yP)k9wEV<|QS-$L7bC)j1OaHwo6EW$|Bx~O?Ba!bfCMf+f5 zL!KTn;sg!Hf7{DW(y3OEWFdnph(gffLDXx#oDoBvrvlIcCNa_sbih4C?TU_a;44P( z;awk=ATj<>2l&uG+y$EYf}C*6Kla&qb`76>7LDeWrpTfZB4LKqe{7PQ)3~z_l(C}e zI2;A}k@+K1@FPNW3u5mfMg;1It-O;yqSh$}dZJhLh?5Q(FkQQ4(tg5p|K(kZ;I!)Y z%*q_%buIA(q}iig&)M@`V-Qk_hmDp5Rj4fD%KnWIF9t{psZpz7N2MR0%n0WcfY~z0 z#V}0>vx|wBt4+pMmDmk}Ok;o+D%&OYZgI~r8`CHyo}@6X`-)c_8o`)1TD4c6K#%{z z+%vN`sfmIf^u&nBm~4|8s#Y{l4=0i~^AD3iuj)bqcsF}O&a7EO9fZEEe*x9S&7sCh z5&OId5)df=#cXM(OZ~%I+*kZ@*~_`#F;Y?`q*|SBwZkpRyfE0Sv?qL_N@B*S>_EX;%DUS+FZwHY6rL7X9-!4lhGMiFX?@%Zx(oAy)Ty=lPsTrRE80v~pmUpj++8rqqjds4U7vI7EO9aXwpNlw8!HNoeM9ukcEDWqhH7L93jodKQnmk3ma> z5fR)*lXX*K@J!_QuYEHfvj-G)7omoZV_cd(;AhqxuKX0LY0mm zBA2{3#>8BPw98_PBTCg1_|^tjZPOEWE0KD-fYOPGd&8w=q9*qJWreuGGNn-liJt~> z1zUHvnN%b5w%ovc>5@v^k(syeNqicZDN`!5{qa`gWy!VD8if5A4I;Hms<=^)tE9HH z{z>!l5pL1D67i88;a{wu8TZd9w7Prqg9{AgbzD19J}IxvoOj1<+vb=Jx<9AIs@P7` z0p06(^L8%2uN*IT_~W$p|CDlS8A~EGuiKDP5+-1foG3~rxRFj^%T=dKUY!N#Q(!H- zb)kUufq?a^q0;5e|Bk3WQBS6q=9C{>Dvn){=_Ndi1+`3a44W;x&qqG?xZ&sY=vElo z&HgGq?VAV=f$b?E=U%SEp-L=+hYTHM+UYCEDiDy=65oR7qdj z>W&46x>r=?@Q_WyA6(p=mOgpCuc(fur)}&dy|lCe9vdUtKB0pfl1#j+jgA`tBEv^S zeS*b2=0Gc0zYxu|yyseHV>l5+K6Emb`gNn0c5u-)7V4;R;|ZIyu)ydMZQ&|k9;#M9j?Pq zvQuBgSQpeAt^2la&>x^D>AeZyIW_y>&Q5r!?T5%brE}t5DYQi0xYU@vn-5j2(bG;r z;05ERtDQ}$8|cVA3ES^hu9(N_5VZ{j$^2!}Y@F2w(_$+C0$m&2w#Q&_6&k-GL4D^g zo$|KJd{z8;XHS#*JgfNXheL`u&wYo)@=403w*?v6nP+{Vv__<3^^*@E&U*;jq?4Q9 zISWt}s7~LASC%LKlxIZW7!4wY7KZ&%pi2zlD_P;oFsj?B-S`zRm=$Z|#U$>RdN)>c)=0eqtJ({p%7XMNj#4Ss z^5GAm`WM>k&?SF<#xMxrFV%feS6=tdUjBfK6CEon{>Jy(RVav~rGKu%1G+xl-0D+9 zdg*koVycsO{u5ka?k^eay8(LjS7l%L4jn4E0rdOTS9)LT7XaoRuTuGmMrr2@T>TRl z1W_rOP-I6zXy-?0U+aRR$UN3Sz6;z>3S5Rv`!=h%-2$8e8C*C3p&g5|Wm%$lXRK=n z><47*#SFtu;}G6dGYioK`o)yf_6PgZ=59{1QZ*KKtr7A{B!O^x&H>|rb=1dGMi#eX6kmaK8Ijkx~x$aZ!AZS|M774_4qZ8X(c)58e z&Gfk*xGM&F6dV-6^bxIrgg7+y1J_VAZcv62c-iTkWb9vnFsgAh@=YQ{P3pvHFvVxHN(Oe5PY!_a+O>vp?jyFE$D< z9Cu)RZ(|J-R}Npz(ovf<)av*e5w_1L&Zlw2OZ5C9s(Z_E1jB9W zz-t+<90(;*Ia}_Dv6L9i_HJ?TD{e8)vDd(NcE)KK%`V26sFToi>@D!DYB2c5B4rB} zPtVvtJyduIWS){n(tqW3%?e{R?6_kn(mUS(d}vG%549OJ)7cKzuK zOhqF9=nI%d`;8k7){#nDW3n74Zs+bRSN1v`=(| zN_1pGbhJQp#7cD3NOa^vbTlJ4c`P_-;7?%eJ+!$;aJ-jUFzF;BiW#Cz5}#m;{!9?X zM}uI09KUB+#pOGij--TRv@8oJ2NU)TP$oiA6(A-BFrLr|mh^@Pch2F!)1kl-Vm{+0XZx{76e*HV*O;GOR7I3Vsf%?IvxyC{yFaOcz$D$G$SzDITXli( zeD9BcWJJZRSVr7pa;>cF>V842bpZozDE2a1EmLk=3KeMDSr!BiaY0EaM<5FT9lR8( z4>mdB`lAg@fnUEos~z2K2svV+9G>L1Y#DV__k8?Vjq2pruo0HiX=50ir(v51HJc=_ zd4AG7)HWJL>~|6j=8nHW@qS|E-k`yMW8L3kg2Nz_+@-~*r9!N-GI{qIdBSv|P+*?% zBX}aO)kkh-h>wDdm=6krl|Fv~Es+e66wA-{GIUq`4(@zD_%XdL8<;%KGjT-bT(m>M zam4DptP9i&lliu+(Zc2&kDM-+$zBk0tUe|d+^&JxixpwCD#aChx`ezm-jdmiS-52i zWV&LB9Jj%ylsnGL?vaEbRbZ5c2oTvLViyoIuYD=?a`e`yKjK2PhZNQTgEL9A_|Tl5 zwvLsL_i+M!bV>n$f+=;}N${+iIl*kRuXlmwQeV{X3x%j#L|egol_P!uf9>iToh-C<-kgm; zApU^^zhH}#b)OKFBM$}`GV+^aldU-d;prDAHYB-^Fkmv%W)1I^ajf*kT_$CkmYS7A z&(U(=ajJ@*QdDc68^vku5Xw(0sDwdKg>-jnBhc{Ca|=ZPQf!k>mg&^|aVYC-8(|*? z;5hp5u01fWwQ4dx>0#X)dqj0KHTKqpEQc+n!a{&B zo+^yZs@TL0a|ZLYEZxR>*>DxU==a%_i6!~O9&OB18zxFrFxLG?85iD+BZNC5!iTI@ zWe3g3$Rh?orrlK$&b_-BAET6B`iOpG_5^U_;~`KFQs?qUjXfk4IM$L@`h`kj*MSQ1 zeeLS5g98Rs2EC8f&H%e3=7f;^T{pM}rQq|O?Qwul8iV2tTqbRLY3J$Wpb79FUU5JM zW^aQrs%PQ;4htX67;FaZ%fcZidDXCN_(8p*8QLLWEZ^j9$&BQl>Gk_~qAfuvX(Op# zaC}QB+}N+-7E!&v-&+^^#c&2ylL}k)y$f-KicEjGpD%egreNHG=1%QXFY?GHDA9ib z&6u=Pqc8ZFptFuOT6f(?fQ9s0BR%j^cz zWWoV0%)&bV+Onk@N~d!ScN(iLUO63mJ3LrhI-L5HrO1D*4Z>065H_*JG(g zfC0C&TH%D~-U9xnvDy!NH`upnZHnfv8M$2!?v_E+n^A_=3Gfy_y&lKV6)-+_@~wc@ zPUJ0=mCq*|kY7W7pta_;teYF`#c^f_#7&BEN{*FQ9w4$KZOgUzc1}kJ%0f_0E7bchrPm_j9+DRN*O%k# zyG%BOmeDPDOQd06XUVl-{eTTtw9`!v+j`w>(i}E+?V5qNTTarGvy<&1OJba^9kz!v zR~bh(S%!1=pZu$2^d_A~pvOD70LiH|dIy^I@pzPFRSq0@P7%I?#COQyqY`*}e1&dv zA|BCsCbgk|-gf)AvZRarq%pgl^qL?NM{wZwFcmY^B)hAj;G3f?o77|%stedAaKve= zk(;J*q~w;V*35InRy4_7yV)VUJvkm6={AjHi-MO2+ZakSO6P|*RM?Ps02gP75Gf1w z_O;!kQP-5;>N7Ts80UuV*#*S1E8OfLS4f4I99}|r!pDsGLlKAP#r$RgoEqxx;VVJG zWkXfELg_2``Z9?A#9ih=LKDU-HejuDMj7{DtlxX;n?t*n$&vD85_kM} z@Rs#YHvam}3&b_nKZ|M~Mu8@Bv^U(>Yy%@~(M&e9Bw5*!riIc?3S*e-l|2EoUPuoq ziQ%tMr?z%Aqj~np_5o_2dXIxPTi2-l}LmSn=`4N%Ojt^FGnXYwl=5{R%$!bgE}fde@hQ(g{lV z&Y8~~J^~=L;};Cip+oEPVpI;?CuD>PfrTDRoL)(gdbIu30HnS?aJz-}$Z!|JNE7_f5d7d2{Ak@Q{6rzUNhh7Wwh3Om z!6v);E05yor-5=ZYCuH+C_n(;_@4P?eWzT&-kD;!EXBd|vN<7hU=-TY59{iN@p?l& zxMWOdT_JXG00?eu$*>A`uHr-QUj4TcPMeFPWmi zMP&O%n1Va(91x6C!%+KR;y_1hRLp+r2>VePY+}#2%@PpxQuNj~@zHIeI(Y0%^%~}V z!tXy5gWq0Q_*Hw6{veLm=^_;<+!C~m&=t95M0cq$1HgJFsXTz0k)vpBD*ejh)%=W= zg80UdGZms!CXk~mj-6iNK?~EtQ&yTE_0ZVZw?WM1KvYeVI;?%z3bOmz^MpfyJ0GzB zVubVR*j_oxY*9IYAVVmXyqCK8=QE6TOc%5222-u534Z8>vdyZA2I~&7t+9vD=nh!c z+i=ml0b+|EYv&esXbPfl?SI@)+iAeb+m|rGa?j&4-Y;jjGP;kU6J}t(gG}Et5F@dCoHl^+DGa9 z8!(IXuEzP}fT4>Ly1;k1lZdG}!4ZE1Gja7&Ze$IKNMrV#BL;MM@WjI>< zVjA|v^_BL!;*@(2dfLtKVVBc=DAs2)09P*CRxaDjX%Eb!Pu-Oy;hW4>-KGyrUnzNrH zol!xpN`1ZLUastKpmYM4NrZUj>)gh7pv$g$yje?{KjD`@p(O3ntQ=Mak8#EY@5m3N z#v>$F8*Z6?P8M>CjC1XsT-r%cbU)YA_t+Mv$46k~C+sDxStc2U$04il_0lU6`}TRh z_ySvg%Ts>KskjfXxG$%;51?1vC(IvS&mZp0ANI)~e#;+*%pdL*jC~f0jrYzE!=kue zsePpX@Jk47=_pmswx-63Q0Kh!Dzg6_|#zM3& zRP^OY4Qyz+*79c2Y;;?4tle)~3_j&%1nQMl@B|}uU;Hu-)GUAQA#$Hj#_WonzgrvJ zBKmau6etp~LP3P#y_!&3;YD$V@KDT~#>R%|$I>^E;U`B%?>&0TBL?r}n~rrL_lp={h)|Qz6bxQ0;&mM9`bJOTQDvWCG+HZ7s4- z7R5M&O#a3<+L|NJRaj-R5g^SK!#BKBh~j4F!ZlYX*glt<+PC4?%i}Ve3n0rMFcxG5 zTAcV{xr}{c3hp{)0m(xHUGDo34J1c$nvNj&`W00iD8D4Q0JvZD6H^%GmYu=uaXOHm8V#A>Ekt}cr;NsBY+*B}O*@l01E9K* zVNc0pU)x3IPbsFLz_2l^C`(A6x40m|y=^LXwr`)JKwfu!2@)XjxvHdg+#*RZ+1DW7 zDt54-&8~Ns0f?)V5<2_P;_YWcglN5 z2Ut)Qe1c>hHy;tXbRfkv0-h!l7^}(@K9#hDFe%AK8}4>6$@+u%pkTJ4$Adf5lK1!W z`u4y0PLyW8{-;^^|KcbAk(}R}xgyPWu7DwG-SrPs%ws~-D z{|B^~ry_+x{%f6@!({r8;o%A9palKbDtXLc{a;()WlRQ;|G_VnFPPr;-vBZu%RhRW za~qhltvczjacKYE@|MUE_}?wv6ht5Y`FsrlFaOWF_JFzk-$$`*h2{Ov8eY$#^&c&i z-+`p~pLG_N(H`V~bdM|1jfLKKf)opov_yMw{IrBH9&A9Y(uBM)GZO#6Ky*Y@`HNzKRlQFt6a({;1+Rzy&wg7@NFHOu77@YYMps}~@x z{$)ZMw*kO-18f4NDnRl|p$V$_D-J1NFr9lz{f$+)rDcJpAflbbnF`tYQ)wim`{)Zl zCn7J`#NLupep;fNfVbbfCL_Ix#8u#l?@+;A+u4h^M(d+lG^|`gk3uz5Ng^ZFS-Tnj zhit&F)x}f?@nDY5Wa=#m(v@AA96W>7xJYBWw-vwzVG%=ttJ&f0nfMU%2$!IE%sAYg z8@AS-t~+f~D2kM^|1n3c3Ka5PF)gE`7DRBC-P2Vt4>!=`2qiq$?@m7&&+ zj&TIta?)R*a}<|!^bN_hldY^Vfqc^#qJ* z51!T&sp9fS8^AKD>NFKjk08egNYXxGWyRPHl!5V-ChKH#Cf&wrs_Fh5z;o>W65Kc2oV_#(lytS;hIa18InzIAT49P7!b9hB17eTZ z2l?IAWYJWJ#)Wgag)4+b>DT~Xx9bKmRlCrjyL{&?JF!;!D7XUxc}c6{$~)qLNOCXN zRg#_qa(F!0!&zO>jj&w3)r}<%*&z%C*l7~%(_fg7O{~Ci4w+cfbhHL3L)7K5#>(DC zI$Ol!SWQhmG|h`p4`Jyw&&XP{7UDVIX#b(Kl@?0m9g&fTE5&l$kxN>~<5D}b>&xnm zj9G3G+;?!_FomsRmQ?kIBmM(I)ey6d-$^U_aQiGFf(~+IMgNa}mvQ{Pyz{TF- zG-8^Vi=73(-(d5V5GFUpqDdD!ITdo7XhTRWq@6luQhy_s?OeyPHoKY+dJxYKaNGe-+}oDX^_N7vuhE-WpO(obo>lvj zgzjSM_P*M<$r@HYYCclXr4)0yC_5Jm@Kik_+8Sh zEC%2tMPw_;ODLTU{}#gshf$^@9i}8yk2AFsfm=IQ`CWz&`v-05!YDuTCB?J|cd++P z4p-B@vrP7+@P<1NcW=E8v2CDx@bZ@oirrKlR`UQ9;0;Wgm2yd*rLoJ{s>_mj-t&RP zL5A~SQo>P9JLgM1$Y!-Odh-p@Y!ZQ%b_C$^fFh3<+LQ=ct($7*;&=(Zld9I+44ohH z!02I9Etm*q1r1aB(-dsXJW$fY{C3CM6+Z*Fiz{&BaRo;Cay62b z>+$ecW172a%Z~1X>-Qmt8rSiFsvRH!M^|I_2x6isNdI}|ZPM;; z)hL%>IC4v}I>WEIsx6cGON_N72GT;5b3W&kKe|m%= z;oB!GFy;z0>uwAod9q#O9I_6V{@MMx()EFz``Q2a8cCOX{hm=adMgj{l1g?Yy_gQ- zUaom?xOE`@LD*wtgNEyUzC^^$z0ETR)&7eH^}~sF=S$`OU$OkZLHhs3^1pLZUbtUe zspsgxJPD4k7glZ*puWU+DG0eP=P+P}#CHY2lnb93*YvK(GIMoXx)VHav5q4xKbvVu z=-v--oRQ@u`O4PWVb4KCkp;ze1HrNp`fTm>Z;yD=l;KVY2tv9q{Xm z^mpL`hB(3y>~wTjej&EAbw{K_`i(5%OocO47TuxQ%nRi?%a{mjEs+}mp(~gzD8}oO1GokO=Ake)Z&Em}5u3_d!(( z4QuB3Y*nt!-fUIXr?{-y2M(PA+9oA7iyyeF21(ln`a;M-SEwYjB_~LzOL1upt}arzvv4<9Bb)%)>{QRcvSl8-9Rj1Pbnbqedv_pG(2WB8`fcz}urTO(Cs!LRC^8hiE{az0-?|sNo zJZgugf`+MwWS5_TDfwP4Ait;-A~Z1v6I`D@yq>P;^zdNG#XIB?l(9mow zjkGXz&h|82F3hWjE)J8sQ#L@%0hck%5w`Za;Qy{M={S#XW?le z1rNQPn!IwlKL8cr?fPSsi(snVMw{K7G>oaDPEuW-Y*Y!4lisK7MApXvPMo}tE5?L} zdmyYAO8eZ6>J0e%$61NImT5d?WAF)AgW-BzZ$oR?QTT$crpEhQ;}`KHu%v%J{j^507J>P z+j=Z8t+cO+!W(ohUoJJI@hkh@Cn5~T`UvWA;&fy)s}RugYw?5S>}G|XGmtUHg>69#Fh;)vx$A?s1N^FySk7kTYu;Q>qC zM(0%R!UZ5_R^5nhW7kB%%3IQ~tQOf!(A1K|x~ZadPCum0Sbe!SSyNv1kYZe&yKH|v zNhnj({A5Yr=a}cu!wXxs(6>D5M!RlT{|ZH@iReh~a#W`aqsL9BEbq}VZ`k8G0vP;HHW^H0+LhT}^Yq~G{UvhI9Uy}sJQCSoaWe;WVH6o1qx zJvxY+F4@rl(b){2A<`q_d{VOOw`7;bE?s7@bF@6t9RorBey3;QRsNl~XVMkv@=O87 z6Dwf!n}>kpSlzeu+AyCi;RhwIsnb2aon8DJhZ6oQcTX->@_iQtH?ZSEmse~@zw}rl z%4y1zztVd?kzBbH(-znVjlW*-9sQCza~VSy(m7&AZrKE~N^eSU*?K4y_Pu3GZhQq| z1P4Vk<@T|lc_ndsrErNzW8#vADPdm7=br)SqRuY`IO0zk#ZU^QiFQ*8Bo8Vp_>hHp<~&=@K~y>UzQNS}h0yc^ zqw|p_7b+T|f}|+kb1m6J&~EQryP=3Xptl`fUkS}+LHLvRO>o4%Yr^{zZtKP1HWu(X zt{|={AbjM8bjqp+#g|~wm*ffMLqFqhaSQL^h70e3xZM^G+Dmb<9S_<|4(=xj^*0Cp znH1-ml4WNm_ydb&r`EJ@d(^)-3G|c7RLK8nSBH(E?&tj$0&(iqpPa$^fl|?v#L_|U zXBhADp>1>Ct6rf44XI^d_Q=;wq?U3+v>0}*Vd06rGuA!O9^Zdf%1<%_a=lRg-d5LU z^~_TCdqxJLtA&32i_Q4I8U6p}`qm;+>^2C5f0_mHh9Jz+U!a-~xFbV}0klEm#cqW= z;IIPy7vz=;RxN9U`x1x3OVwmR2RgjLVx<23jsuv|gz->a!uXh(G~F^~!N!J=*2G6< zha+qA#|48Nz=TmkgU-|Ro$QKF7?XQFLO^a-)3H7mIkk>z*3?O;&?FWL-_w6;QP;7q zY+kBZ>~2}^n1y}+K*G~mPes@wCnZ}SP1ODh&tE`nlw zKNMiroLX9CS=BWyMWJlIQpS9*)`&0xtpnry*S1heVW_Ic1Xs-A$6-NaM%4Y1m#17@ zEuEmvka%(~Pq!@4iY=8f-y?g@GEIWdR`roYu=(hkgO26lnr5&{x2f!BdwF1jFI{Cn zDfV0Wgyl6XED~c$KPA-(LLp0Yo8d-jVL5!dn@C`5a@g0Oy4z9eR=n9Bqcqy-mf14anijl(#Zzk0hpc}C&xZTcfE`1wW| z%6Zl}^oC23GwoZrtpQyX-Dwhfn7;%ROsM#zm~0e^V@D!RufloQ<&xusrd3*)haG@m zb@FrA;xJq?y!TXC9qt=C~8N)*gu=V3OVQeEfIFSSK`OsSgX)}4qVUF z?EZuouZD-Wc6tvxi~-V9B~M(=#sOt#CX~o zk!2(vL@B(cv!f`;zGEN}Hy5+NBNu=Hivb-@!qj5w;@%?GssdM;#sQ1_j^dRW)GV++ ztfs;!(NNWZ)@B2o>1yG9-ocvl{h?ZV!g84@qHPcvGbK|RrXsrezwd|7=8B^iP43{-B!!U^RUyvW?waz*JAf=?+qe92GW^!~4cJy0@jkUOPINrCAi9dYoq6R{?VA9|x1F2RSY zgId4HtdNt?-ojvj{NG&tvSMCAhIcd}`;C{eE2+mTm@cJMR=1jQ(nzyr$^{l2<7*(i z*Dj;iRWY+RED|5Qu92fpd07EkOq2H%mE1AXVY{3tUC14|ZPkj++!b_Jrn6&_!g3l% z{s9t#V47opQfLGy?@KB1K$lb?Lm{q!!s7n0cBOW}DRqNg2R!iY22wX+Xje64 za2|A*DNpqSb(#8C4_z8_ytvo#sE3a@^;?CZBG=qTLFVwJN0C1BZnU47HKX$cSO^_+~8(1rcDbTx{?+@$a@f+SV=9GTF?_<^QaC?VS zDyU2@RU71p%9OM5Bik}OJxbJMs{&OI=7Q!W8_pE_sXAGPtuz?|FJl|yaj;3KJ=8UL zb9Q^(n0(jKFYCf>@@S^N$eiUgAEY|j)m^_;iw~ySgz25(-lYLV_*udkgBjwP)neUd z68p5B$iG)#F7#y%SbVc*A?q;IU_W>vu6N~ADmk>*AxH3eoYiS?qo~r{xuRRB>nGx} zE&n(QkC~>kx-!49xgdA0am!j)*c`hsMp#XNwc&X?IbvNA2YW@y$ToXFpkJJ-t#MSe zAwAJ15$1uLvVsF}ot)?o7W+)aN5jI>jG#@Ftw_NAfBeb(C2J z;!KEHY~p?YRu~YQw|GY0tlRm)(s_~QZr99-)clu*_7Bjm3b&%{mE#fC6-oWO6~!b% zuT7&Pyt6QX5?YBXBEQC14Rd+YW6Onpg7rc1bDe{;ezL9tE|h!fzIDf3K?5cIQc2df zp3I?%8CR-6+p$Hd>X66e?`u%dDD1H^D>0Ul125TD{$3w&E*#$!a^{Nq{RD;+%R%Lt)qIHvSNK=fi z`Dhel5;Ey6EtRg@>R~x8(s9N;m-rnIwl=n`qFwZuOip~NQT=|%G%3sEedl(6mN6ZY zP89XwPP=_@`C5}MJN=fhqt%kK7(yo)rvqi3(wt>ajyJuQ>3MseFy3w)p{{T$q1Qz} z0C%%N|8hw7+M(4z#qe+UJS(&sWhG0=3s5NI-G;T1omGK+aQYm$QjaiCYDYAR)jy*< z%mkYJ;^%$EIYw)1Za$8rY%`c18xti5w!v{pHC3BnwcaxFr-h>$j@eS@wQB`beH^5^ zyGk}csIHq#-iL!UkoKFpIBa*$w)GQZ0IyCgHZ19n1Om9zVuUrM+fLZMJU&2QeU=+z zpEzCDggKk~1bWUtU?FWe^hle6?EFCt=6TxN;1v&gc^T7FL{6mK^*AG=n74cCw|gAu zW0z@kI-dL8;I2-Tj5+Ids%wVhh`JQK_401J+NX8g3$VRUXz+9KGrd6vVYNy8fC})l z-uX0P?U#I4JjM5KYN>=ITROFgQ@IxBMCGqb@t0dqj-na7t`MT_SY%b4sTX2FNk0X@ zVDn0LW}J+nuFN@->q0k*=%N=*&w$vYt?SPzh{ZXAvwH2a3F|?% zwgY1L)G)^;z5%-vrRanaJHz%Z0QP$D)Ty?!D-XZS1r2LfsGQ2s;V{Ec34x9j1L|D9 zuF`IU6tprHw2j~Y70KjnQUHdQA zq0Vn{iq@Qj)w5iV{pP{4>o=w$wL-Mln;`}r*a7YMZHXqWdHxH4lyvjOs(e7-twszk0JXnKE2)oTI(g%D7ZNYa{S?&(e#)Sg$zax z+>M&rt`T+Y?fKtsG+Ow|`UJY7dJ{z9>aYNW*ffkz-N3f_2`+to@sNu_A5R3h16Q_DJT8MRG0iOzfXJ3aWmgcwEyLOY z|Ha-)_bn)4GLk37dpIy8u3qEu4*3&CVfx!OP0}Yq#tlcDz@JXt=as`@P@p7V4gO8L zkMhoVb2&R`55S}>`=D`NoXh}S)cw{IpRX#rW;fmOjayMuzUw1>HHx8*Qz(xL+j8XN zeBod_ATf_Njp(li;2YvqUss+@*>%nE6~emREe$+u1oi>`>$d-HL-LC|;qdqim(p~j z-5xVZ-bZG-074tNVedQ3XwHQ>ufMSE;T?b4pYPkEO3y_9aQ*$a%K!fXg~(y7ED#Wo z$*+sx>$3XEXm94?0_-~FnEdywufq2`=zk`K@T7=3|?;x8^fHlQ#O zJU)z_cMA6RFA6`5Th9=$N_#8O5?2Ya_YgC3&`G&a0mS(qAE#>(>$x`#~OW_Oq z4{AGM;DV?}n>h$f$*&@}ZwF?hg^(tKy_fn7!kYd(^E;860Gbk zkNq!DRqZ)11HyYua-QcxcRyN?sBJ2ZvObwo?{ zh=4S6kxpe|y~UnqOcv16Y7dt59| ziaOv~z1;hCunfa;QA0zj++w(@>l8X1Cg3&N?CgVwNMDq{ zjVt^n%Gw$!vOZw7EHQg6v1ZepI>x+lS&zSP#}_9B#0z6FU;h;oYegeOAX*d*!G%I^ z^zB~9wj%2}NG)5@0b&daUxLWlb72$hqzbmM?u>`ZA@AvR*?98-lRK9qS~$J)ERf}q z8Xu_i9`8OF$8S^Y4L8iBl~c-n%6iV(T9gMGIXNk9UINfWMO&rg6>?iMQp?g?y2lu) zn`Dw$C6H)rsWD$RCe)uQhcwie}-5M)mMV!UcZ$r|<`z{S~|g2F8+TrbIZ@ZQ3> z3h6&8YBOUq<5uy+IodB?%sEknG^>T9e#Sn9W-wI#u1?Qkq^mj`PQZemOBL>M$TRhL zvSJe=Fbe1tsNI1zH~U*lV=2;zo_Y7HvFbv1>pFFIdM<0%$jgiq zu+&2$3LwHNTsQI%RbWI#tP}qkROghpl}BE;W&k+zjUWXtjw*DjuWp;f>ZgXO$>qTO zv1CKs&rCZ=oYBI@&Re+_%4RO{Q7>E)kvq=Fc+%2~M*XNC@HpW^j5qOGmB8x3W6jcA z(d9u`E}<`e`B>Dk2fog6S6gj+!lk~M09$LcT0*j*zO{VsC^}DtMxjSAQNg{8`>J&y z><3t1bTXwpru$wS&f5TJBV|N_uyyZALiz)Btn{Kzd(nC=HtJ6P>Ab-$t=?bH{ej-O zKZBC9f3WF#;dv0r`&WYx^kPUL)>Gj2Du6NXJK{B6sQhA^Dd`4~vias0+65y^lV_(u zP!@s14|*rxVB|UXjfKt{h)fR2v8~HY9t8lsa!U{}=0`qJ!FA#G z;n=`oL_c)G2X=`~p`7DSi0U?fH@EVitMh`_-PjM^WpR}Hdn;&JRHfC7v&$yXgPHj4 z;}^P>JK?Gh8}0`Jh1$* zVjXRXrMyrQX9Hofqc?XbW-zJB=k_5BT&kQ(Q35o|;P%syKaH_MB%$HjTK-;qV;`c* zd4-9Yar9#FI-;I_$3d#uSLH+D|Ku&v-<#)QP%#Zv*uz?6hWx$Z@J)6vkp?^E*2b|~ zQNYQf^yiVjpKZX};4cnVb-QR)P6U2{$V*|{aR$CvUi(7rYVC8~{@YyVc#7#o0redq zKVs1A20r)Ob)U}YyM-q7juEj>4uT#zx$gPZhP!JBvZZvR)>s_5!5CS9@^@|A)M)sL zLY8?;m{Y_YknOqpKc_2k4BVa>nP+L?nF0@a3e}^iIWSvk1|yS$4HG4X^^@}eZ0=OU zXME{cCr&co_iA|j3h17Ybpel21y8P=yerf;;7}bBu#s*P*-UH#QbDHLV=81^Jq@Ej zqB?-SN);q`4flBh))4asvyZR$SSIAU*?T5Lr}ytW(m&-7j$yceWe>(P8MWZAeAEX5R78aPcY;Bv=Eem= z;zH&8ySUgiV!Zgz8juhQrw1q^%9k2ag2Kvi+E+f~fCSoG!^;CEl+lFI`3XWH_QbXV z%aGC8VFJGQz>p9dM_|;Ci_uM}SYlIaOvV7OQ=!x9pZ`en}pV zvclO|+i&;O_8f1df83m(yMw5V1~cJ84ZbjuwB@)=Qy9WENSj8G@?lFMY>AV`DFrUrXyCYJ+S>QtI@ z@m&m$IbvD&>E<#bdA+$FBeuoJOICP`A{Pve0+;Iu_#;3;^1uYo#uFs@ z6^1+mWeWA{_AgtnG&3}A^G=&lDr6Bg{n>&wpkW<{9*9=&keUM@pO$1ZCXVQ zh-N{|+7VveRFx2nnaxFfMF*}w>U^>r&FrMz-VSmn_~TpBQ?xUwhcA(r$-x!{-&80V z_9G)I+lB$iX|dLq9T9kI%R!kGB?%nNK{^!Xj=aI#q3xf=apudxOrfc1AKwSe@(Gb? zyQ=0cN%CWZv7v8c5Qx8~ZuRe1?>Gn-r~EfaQI!MAlEbakt@g$zyeCv=t$Lr|z7(^| zH%%}45(qd(p@rYw59T!guGWZtv|-}cGZf8Je_R2Q6ISMMItDEpca0M51EKYZ;e)th z5w&@(u!OL>^L`cp>3oaq3PK1C`EI`chXc$+tx46$r8QAX9i@t-DW(M?eLdF_r)F;j zs9lam)@cMXtOJkphsnB2a{k%huR-!$oY`MR=m&2Mt4q;xD3jbO1Q=-yMPie+!S|b& zpNA58E!aBE*5;e*Wte@d8GMqnAifA0QoCeO0PufK%YWVVe^1Llj@-}daIpJzTF}18 zw55SJLQwcXxHNb)AVMuS0iar0&t_2(nLo)i$)1D5rt-XXa}CoCi+!iEm~k#}LK(7n zXMgj)#@i+%*$4fvQedF?bwANgobs{xK8@WwmDFf<`|)%ZH|r#@Hdg@L!yf?)VZ(A@ z;D_&qQ{7(PxnatVgbGf#fSj5#vZYiVrJ0g>VjYjNRs3jy2b!2l0)UJKaUk6{wfGwi z5Q*DEg}h>g#P&WEfbi!JY8BI_i9MhQwOxfU+U!vz+C(B%!isE9FM-;(R=6^)B5T%@ zjItm)>auOJbeGYqn{qNYq<++S%rOo?n?GLVdQlXTL6u&eJ2kHNl9UuOsh=xp@(|kb z?bty=O!NHV(@5&-4$w47(q8Q%yD3Q3(aRVc9EQ2f$EA5a8+>kXOvvydj(=Xs+`%)yV-&X+EoxnH29A;$E{WJOknY zfj}tQ5hkdoSfd_`(LUpplHA^Yl04~Pah4s{`9m>zQM?JEL%=DEs%J<1*ax|8%>=|` z$u@2f64Tc7m>oi*9*S3^e}*EZ``H0K#lp~q%(j9RgN`B`!Yxf`Y!B-(lC&Frgaf3; zv4|zZFCTXisW*6Hs2a6n*PP&-(Ehu&m?+&1bMF!TXT*Qp1MD z5%q7yV{(|Htzn!$B~KshLghw?W=(JPG$McBeA#L1rbjZ>__upo>lC=gsfvk2 z^LSkbAin!YbxDxsyp~E)V|O_3xAVXI=*t%`jyvI4HOk|q##I-0D5X5PvLkn*8dud( zmM2K?bUK>Ct0JJs^poFz=9j7B(Zf|*;z|5Fh5!Oo$^pBwBq(Ex zwT}bj7aEe}zCXH%MGdMq+7}-O>?xN+mWO?S z_@NuSMR48s5jXJV;PCfyn#^2oZUaD4nY2Vu7=7;disP8OZm-t{@sNH(zUrzpmta8% zEGHu%aQ{XuEMW_E@$gDemf4;5a|7Tjy0kC5-K*K)Nn0e*TN(c_NHZ44IaG%fm~k0n z_NG(RZDodmUrx^NzORxWOQ^D$R<({^8evh=+K|R!UjNG>f0>6_VHk6o0RiTwm(X z`s_aE784{Y_&oRn{b33ZgnZTcUF~s`yxv#5J=~DW&$JS!k{!?mBN{_4$r!`r8Ip65 zAa21p`I=zwe{B0-3;ti*BGIMGJby_**`fTieeLMQNDSPr!)5_!ya7~~P(I}ey_2S5 zV8{Y&(P2?95+PG*;&A(9L;~TYp^@`6*S#9O6LnX*Ik&p;@)+&xmX>>nTEvS)SSH&gMsx{BPSPgCO?9 z+fnrp2KW0aZ90JOgK>Zkd5NSVxZ6miqT5UhE@wYVqE#-cymPObDlKqE+=S?q+O) z`~$S5%nUs&ccC_o6>hMb7k2f9(JSU3f5;aN#&)M@)_&LhR~b0m_~{F)fH}9K%1~OOhpx~n%0!lykXWm zd=6hz2d-wbHYMh$>`owd&Y>C373eo=3~{qSbIPL=?%EUWCQL9lbjG1!|5+;wA<26b zQ>#4yHyFkI9e!oa?$xcia9svue`Dt>*>#j>Lg-z*hQMp4!6k}#{+qW^1c=IZDyUdZ zcCHu$;0Qz=V!F@`u@Fq$6iurgV-hIrS|Rk#-@xaPtH$XqX4ocjaK~PT|I*{(Fccc8 zy}={sO);ZItc&3B3p-wsBIq3vCYXUcL`G~Z=h5%WizSsh=Zds&?FuI*7;(@Ykrna4 z3BnC^&FZ7&(I1jVs6F-2qxw1O09)#sg|f>D5Pzndzvzr@4R(_V1zxwr3hrPI{LtuF zauZGE+3N4~5lGiGa|#Y~6sC{;NQl!J@%Atn$vse|ar$}!Trev4*gT}GY{8eXxFpG(4n+t3I} z3|fxEcQ)-Lz~S-%^QHnd3hxV5zU98!(V8g}#<4DZfmo~*yNNE7LGE8_1*8(xYQ%?HHiA2gVJ`!Dl+c8 zCA4FVWkv2tC|4-(zttl2HHP?_Y^N1;m`IM2pRmW&{fx7_3lk0|ir{$JPA}QXOwzBD z_|f)W-UvaHB=&Am!@tVyQW&2+nvR9JM z=wXh^83GHDEi6fM*~NMzZ?ugHL(60r$wN?^$kB2bnB=I=m*bJ?n7?`TNT>z8n%Mg< zJRKssHY#GJ!JW(|4ipxX*%)Cu0CJQ%;VcaFNc!`Eb{8Av1y|#hw3@rhzrV>y9Opa~>2AlLkab#$Ld zQ5lN|tOAZee2W-mu$u2R4dx#D3@0bSXi#PY$|JsvRAJ~6$ISE72u4u0iAcgv(E1EY z*a{s!KYlhYasT>~3-(eBO7CkBJHQtClwZbx@R<+aQ)IV`?DD}x1SINqDvzP*|D!1*<2gFhGJ8@kArn_3&G9`=JkD6)as<^2 zUiQ?Hu8^`WBS#F4Kjh(_ZS4eSnW=BB4ZH+g@g5)51^H?2t}XXC?3C#dt+m?+P1FT_ z@Sa4!lsK!zgiDqY;QYiL9d`opAzZs>;DY*D^81W^~TR$=kIz&aon_XPZL2fz8cD?#9%qzY(qM*L`jfZO-ZRWTRdOf~Sm z#oR$PgoQ0Xy`12ga(NdqmNT}+R*nU|-z#?L1$|GyH-yv;z@Do`yQXP_mSRY{rsV&l zZS>O|-Ix`QJ>O*+a&ocmki=ZnWC^$BR%Va`VV&JaeGpMT4G!A6t?`U60~gyJlqc>u z0YJEPj!k3EDj3!&;bdJ7#O!N2jaqwimx|A7@>g<P&}rW$-LFkE|_G*NUtX6WOfR6fM`BxDLA0SLbB( zwNU$eb1iyaV%pz1Z_XJ(HN|o#e%bqMLK01vKZgR~5WA>Y8QY(48sP43BBS+F z+JBDk*P33{9^TNu`20{em>JmK`Dx~|6{7dd>-o1k@~6cPF9Z5c=k2pTW=|X-o?kG2 z1TVeom`LxL*pI9WKv&3lxFI;UpL=*mN#mCh0qCDG{p_uHZm;@F0@wSicWhGU_Bkc~ z#g*cNpce&@;76q(nYa1`(?wrurZ`oeihBQ>qv~g+*^gszz>{BnC#M zrRuOO`CNolADzUL=5T_V6HW}0g*Em?DSt8sxQPa6uq64BIA@xkIq;ZQe#`x%`1Drk zpujkTn3Yv6k+<3kwaza>7cerLp1n;i5 z?iAslW!)^y6FP7*V#88&$cW-L;7aLc!t*8s0hVK6B8PP>@akQ|x~74rHemTew)93y zr`R*oHsoA31Adqst;ERloFh+{%cS%r&sR`@W#*N$7>I-Q4425X;I(>Y8tFsURKA54 zTor~*vb{FZP760*a$wbW$AQmT~fy67AI1~KD6(uZd zm`fNtf(zcC7mT;$T!nuvQR!3P=ZBfC8F&%i!#HWqGS;XRYBJfbM2FMlUXcw0_>R#s z6l6DPLWOOCLS_xsvX-qZ&z@}QUV99Sc zdyPy;;USmr5ocCzd3fYt8%Ee@y7>Mm9mApAEtARC5s1rXT{MNuK9_>47vh%E0=;C8 zKq;1niE_ksMNIdusU-=&RXMLdkT?fgdSw_Lm3pNh#9(n9jNPbKXxF4yNN_=EFSjD0 zvA|BFS2V+i_a1=v=kX9=&DT@*tKq(;^_jHVt=o~9e$X!!Co2B~xeMQt8 z<`64$frSjM{>(a2IH*Zs5O{4CBZ978u2OhTeLL3Yo^Uc*kG()8|HjTE^o^ZIEce5- zjJ4!v9v&JrUX~&oj~u7fb0qe?dHp0x#cu~?iPl*Q)eg17^%J)kfPk!f_^knk)ZJP( z7;wxjdhyOgQDo^FeY1F8DJFJq6PKWrDc@MGqU={}r-rWX`*EgF@mrf!SGc_K`D|f? zAGgJc*c>#scYI6|i^Pj2i}l$&iEGFF!fWvqqE%RK-#R?oB6Q#i9kjY7w85RYqS$L`*ni6WlR`=0*lA^&U z?TGD=&ITh2Uwg)tj3LIYMPaS_m80-HF-m?Bn!>&*Y^jCkjFU9hY6{yCjng;4QiX3A z!KQJk+sPX_9GCv0Pwp^0`*^m8e>W>+??Xu!uI+6!eC_W%K*k-He|{h;HXTYTi;%2j zu7D-|b!v)UEz{-N`B6Bqos z;(q5i7wf=xQhP+;-3qUJ%<2j%Ey}eqrp1|Ja*ldlIV3$;bL%?Pafw2aT=FU|&Nc}f zoG9N$^7tNCndquRbORP*|LnC2sviBTkduCYC@i9X}p* zrkm)y6|aB0xMGeOmRm|o&qMH#~$oH4N z*%p2^y?)l-03lR2^@VWs_u;uf!iSM2JGS#ebTgNM9PLZRL6js3e4h-8Ijx3#k1OXg zC)*+?;P(=nU5%ajZ%$pOUel1WTG6MGp~FsP65j=AvP2;+7llUgwO#>26FPFWR`R{= zL86&nQpa@84!c{{s_kv$aHpfhcsm9H{RD&t@*P$MirDCix%AQP-89Y;1E2HiAK3^J zAdcfbrQG&uORr;Ns3Zv&-)yHNJl0q6#+9@i0j6wk2b^-G6KTD-2$3GCH6JE2K$?ce z=v-_x7_Ex_>(j_3?`O5MN!Tm|Rda!C(|^Sw`_UWgAlH6sBoq z7~lrzCmz6YXKS=pa&_D9tzc_XDoRr2K~2=27k+ZL?woGe{u!j`?vtZQ^urbVr?}q( zV7FDr4-@Q`V*C%(-{~ZxT#ugkPxn19WD-8>z$66_G(Iy6p_~Q*svKvBi|hM-PbjNW z_&_DU4pTgW&?tN1=-!dc5#$lSpcjZBJ`HO8KH;cmd9F0RyT^MHUuSV8MpXrIMi!{D z4zui7l9*WL!_EZR5rnfLpTwEf#XE<8;7oRf|N-?NNZYj^}D6)k&cCa!{8u)ygj6 z?=TTeHy=R?suSZFFXp+Is(me;QSMPw;@*KH0HQrl1hoq++$me?wKoEPoa;SUMfr?d zROXZquYgUszD<;V6t6=m`Ku3rOJG2k>ua-muPoB!yPwam5`_%q=l;Q`gejgayUx1o z?Y%s$mRN?+y&}TsqzEjgb}W@&)4pjuhQ~Ly^t0URyI;f*vT5?Qh7`W6W7~~%!s2{T zK7NF~ar=Zx)ELYJ%ohpNdZC3sh6hX&r(Cr;IYuQJOLE1PuxP3=N(-?7L;R!m_gTY} z0+T#TmRA{5x$kL(hB`=mge|Wd`pJ)Z;l;Tmyj&eetGxV2(<-)$PE~!@lT2I9RP}6C z=A;2VG5D{Dyr&{QNQQ(Igh}F3O1VL;a$)z0h3mRcuRh`y~Y>dEdc|UdioC zT5Z482t!>B`G$hWtswFN0?|TmhD6#GHwkD2T9pp-96C#w?$BDo5Veh1L<2hl^3}{M zlx$Yir~N@j ze>E(&p2*E&Y{GCFc$W+b96CBv_A7ocNe?6j%`?rHnp3Pg$Wt8~f+c4hYooQ3td)~t z!G{$$H0!iAo6!;h*zw04Zl>ARUs&WhdQI?zrSQJiotcf*t~Lzs7)(9e*TqkIIw0(? z{Oy?ZdN85SCjv(GZQu;V6lb5U&z3&cJ`PN^;l6ZT2(e3@V7)Yj*!9!P$8B)MXoxtGU=yt%Q<>O z#Nje|zyqwvV+P@^%X=tkU%4ekW@nnKE7qOT5=2%Emu%}Yfb=5_&bWT`f5h#u@ahU? zSHn68ZGo$>(%ZwgT2-xbT-{iBsl^y2{E^%&MxOil99Qu&dIo83Z&tQvBN$d(QK5Di z$Fo$-b?roe9}-*Xb>q~hJp-&`W11b~RG{m>VC+M8Mcd|`1}Hv4S+HJYs5fM|OK32t zlrZZ^F^!8GpxMV!*{Ede)wDos8&K`WX*P;&`+~+f_==i#P|RovHMDQC8aC^;B+9Oh zxd}(mq9}_u(jjdAL5yLewpL`Em(kDo zq}Q;lwJlM3c3=$H)+4#k>g_}8GwqKvi`8@R-G-rt7wtzrgN9V@Qa6*0CI)*6IidT) z5Z+4>%0V6+^b%>ljXy%|xblbEMby8Eb#C$%txU46`fbe^F{zE{GVo1>d~!(8Bl*j|_Q5+mU#m zeHN&ufp&x`mKcn=#!FFct=!{|bK{%MU59UcnW8<1zR4)QTP*-+m6eZH7OJlY5xu`1 z(YQP9b-=+TztxGo@Ga9kIdAawIXa_Pk8iF_-P!SNkKJ+J{8c*Silu(qfyn`694p)c z{>t(Ja(a6wwtwC})xNyh9SV09y5RUd)OjB{1sMjOVOx|?BK&NapfkYQMc}gl-r|gA zxPjh9uv`lq>G$V1Y7%0#4d2r4^KX@m67@V&*Lh2PCx%`3{eNWrV{~QFwgn0&72CFL zR#+9=wry3sV_Q46ZC6sUZQB)A%$KTrzkAO2-jCVZYHiQ8_K(@;nCN}^|Jy?kMYkWAveL4@zZohzhQ;2dZVjRSHqx_pi6<`m_g<5fSiI- zl*8_5{>8P=GKaq|9Vz(}>+ZZ>xw?JWfrnqS=skE(PSOBh{&F<`xk34Fjt2FU+;92l zXG%o?|I70?Y2%=Nf&Tk6dXuvb9M?fn0cerUQ~y*@xWbSQqb`Kidhv0_NvJv^{B0ZX zPM;@w9VqyVh+?wXKO!J5Fr#fH>nP2`?7{c_`5o*Rr`?hxEHJPQwIHH`g>hFa=nTb9 zq*bzX!ez1fbp)*)ERibZ=SOBCnk|F*$Ykx6TuIjt{O)TsoV@Nf`j>8(ANMfHdICm5&Ia_|*}nMoYjb^3!cTk) zD_w4s{gDF^IjI4;dNAE*rtQ=kz2_9#mV(F>rd&-%T*7GA9p0Z&)=(%q`opLt0QRKC zevY0gir}(7XTJa%54L4mp){`=8DK2>-B30!IwZ5d{tZP7ATLenRDbJcb$%_v5JN4W*3I{jR)F`g(G4P9eM<599#?f;JR-+^oYWK+L`{r5Qk z9nu!q1*S|egujoxvcK14KKc3|K$1ulY@k6R4>gcU9|Z(ZsiH z1YIeh_5c{jy|kp>npQq&k=z~5KF@_~dYay=EYp^+KV7Q?%1~bpugMR^(jxdA3eoJ? zY!2mmrj~FLFlrjll(Le#=y$t^mJ50x8(UA4kiL&UeN39NRZ97b*-yThK(RN*EV z;|nT7Xcva$=ow)m6;md41j&J4N_k+5JsS3>#2!Ixb%_yF=g{U* zt}e;E*C>(U@*7%WnMLZ>@W75?x>hOY1iMk|RC4Q>&8Rs9o_LkIM_^2e{$C*eAI3gq z*Z;ZK{RhMhLllxP{{l|S^bS)K82G<1WaUo{`Q;xxec&KV2R<#zLIZ+ zjX9`OP=X6-${sgji252m$FihV+R6(^)~C(kf=6IWN!-Q|e$>DG*-NgO2Xz^_qp;6; zST#S}X}p}pYvD$d;S=J?g{QDnMF^S`NfbWZeJz4gKR>2a{mchktp3%`kk$S!!snrW zJt7i^Goh(eE1WfdBQ!_2y1%(pIjnrGsCAJH7_=d}Mw7eqnJycpREl*A#xVzgVNLmf zyYKV~&kcRS^m){9tFzldNI&nSU}?0C&~W6Bct{|N$F+LLU1e&_=IjkIBo-q8w3AY92ta|4LPtidr1@9j&gWJ!y} zS%NOvN`#)CpQtvLLw-(aR&or0VU+;M>VKGqTqS0lmRNFTJTUOTKexQaqQFDJ{CzwJFJ7Mk)?qO71EFEJRJ(n-#a zU4OD7&vUxFK0vOJe_%Sms{d@o%=14(VcZi7J3yu*QNv01(xI5^l_|u4ukf){V#s{S zJS3}bQn2Klvp0`3hezN_;5=dBdj{aq7mJnM9{g$#II`>WTB}C(W;MUwnJP-TfanWt zLhJu%>FVHxzZ-`yOLNn=x^B;e7rD+xLa(B~mz`WdWGZ|(igevV67Cdb7cFjW3RtHa zVq)}=IbVoSO4Rw|Dza8g?^H+QlYUSDMili#SmMd$QXQ4-`H<{D#s=@mHV9xKXCY-b z&Kwkq!j@*SD-fL%{O$u!kVC7gb6IQ;X&ldsJGwL5p>(#QuoycrIbB~ci!ed;v#5ju z#r$J=Tkyx3M8Da|XfIJq*il<8r{?8dt_XbAH6l0B#|am6Roib%(iGx^=-!=uAB z@peBh3H)3yiHMJVMbD)nY!a<+*`WpUh3|uDTcaDep;S^zOm4G>W;3s73{&N2HmJIM zz_%0xO(OY(iD%#kF^u{8CULBjddQe|QBQa_e$i=E=3{fH@OS?+ANn`8pNOjc|3m2C zQP-obn$`loGcv0IEM4%_G2S=%G|5JcsiOCItQH65NTutO*~5ix%t zM-`cSpn8yt6>x)&b~S!mf#Yuc1~-sYC2O(IQzQ$^{w>c&aP;eI9{o{m-w6?Q$ybDj z35gg2`d#dp*e$1{iwCa6hGi;#uvjk>K?0&tDbBh5Q9jj=MnQ}g{SmW#-yayrYfL=iIaCn6}Xd}6u{yiWmX&Ipg-VtUrDqE(j)xEBjw}ya2-zES0oy35=nEk8hqMA2_M$FbQSY zYbxIBmO?oSjx`bT<&Wrhl(6MYi7lx4YjICnnwx88;mimqA{^{HbGTZy!7diqeUb<{ z-=WGD9EdKKt~4{r*>D9y^$>D~z;Frsv5w<-uqgz+?qvy-5cKKCbJy5H-GUZPQoSlNQtlLVbG) zUa01(e0e+C?g@r3A8N7)e?>Q2DP>QH+4EPrd<8pSx=MCl9{usN7p{ccp5skC5YdWm z$N743^@6M5bw5HE%Xf$1c@OhT3ymP_0P}a&W_jh0Fnz^Ond~7L9|VUk@{UhyBd+$%cj+f-Pr1o(J%lo$51MZ4+@LY(`YF_kBm94AZLjN%~MlVMb1LZh+tG z?61NkmK*N+HSLh33xgSXyD4Q|OvUOg4d_LZxZDO>kJYBk;0vs~70BdWD zMK}vN+zO{Y58gB!4zF>WY!?rqP}BD2$%jee6@(>kJ?wO|NC579_3tj%wjCLPtl#7m z;(Kh2k+wnX~{1Mkcam^_FPkLe1#Qt)(@* z%M$C7&}vk&rL09|pWYEb>|Txppjf2av{0!RHAs|!;40i|p`=(a-{(QK1MN(~9vKdq+B!wcWi2eXSV5K!uwX7{oRVAZl}`I*CW82UnD#2=G)!a@%2kk#`bS5p7NN(;&{B%Wmt2l1)6 z%M{5yu~doa!8(MnV1>TWu%}eddxa%DfKZ|B9uzv@(kOpLtVr5221MULi6jdeCEif| zR(X$Vy-^<;JFliv?s1zVE1S6P#5-R;EX>bnb&Er2TMb8)3es`SkIY{?5INU8)ubtN zT-S+m#JiT({6H&hk9N}aF-_3ROtJn*@(}cNh%|m*ICfQ~Ig;p=ceRBPgjCDjs&R@z zuT?P1^*fWBskM+-0Vv5A6#ET8%(quAT^8*xurzw+_jd<922?f$w74G}4LSBfG-9bI zYK%G%j%h9AdV*R9(jEYwurHkMDzW>H(f6b%;gTXg3)@y&RXO|~7244FWtC9gO?0-e zMnJ6W1RzqG;^s4HTI3DEEScW3h)Hk{Y_Nn?Yc-|vAItL?0-z9Ib5Jo66pk$I=YK1A z7$c-?ZrUrgSM4h#E?x92K!e68Q>_v($GFYn6+LJVR;|;jbUuVOtxs9tbq?YfX@kyb z04JvY@RNU{vPuFfl)S=8Cjskp+QPzaU^!`l(_+-iBRFo3pQ-2U&KPg&P(5$ly+pqUN&!z>^5qp>3Il&_J5}-^iLDz zzdI6G{oDZ{@@H}eH3*0(5VRIWD20z12H=gq;PC+{_$!J*982*YEwLG+8%Z!V-UC~! zN2IUp(?~!9?Wx#^Xa6gUAz91Zfy|)$rbG?~1FkR^XMjhq-3saiQnjYT@Kqw6(sK3N zTA`$5MiO{}@7DNl3yK>05s~hk%}$R?uPvXSE6xD#Ck&9}TMI}YhgAfk$x>{96WM2R z-ce1QcdZbnHpLKi6L(cTedj+{2QuCe`mV-IT`4_%XRKbrAzY@988WX7lQ8RBBX!-t zl_k^15csX}UoBU*{x47%AJkJVtk(qI>i1n#eGd=^Yj&U3$VnT3gdOpl=P8^=7GF|g zLDlO|uXyt>&owxYT(3Ih?&1T0?y-A7>7#t!D;&w4^w$sl`Zv0?H!YBN?dhLXyRTyI zZ$ua$XM?z}YTE!Rg%4o(!!5(q^q(ac|+XGnDssP(h>STI^?49s&F>hs|%q`)j8N z&tOse36Nweg!2V6eFt3tJ0>#tPt8u@I(d`u!=2q%{Dq4BFfLqJ?V&}q;xvW3?>!#w z;`G?JZqV(P2rr#KH*fs7mQ@M$O^?e&dri2lf&$`k$w({4CAAvY!q8QbM+c3Jg4P!n zF$<*E48Wt0#e_t!3*{KsBpp;kU&k%v0#W8OjG%aK9V!Mq^}r$l*vlsfcsF)T?LneK z3ns(p6m^(vRTNfHaZ2>-CM%KJw{o0lBx}AkYymcuklINiLK$+d)XSDGX+lL$<6jNm z-KS;hDNgUs?QidwNhcSPPP#NyYGkdZ@e}h1-mB^eGp_OVzdeLlXPMq}gp9SoPtBiY zSR*~%64u>$%E8D0m}dzhpM{|U?|QN|l3cac8Tl&2)Z8(|FpejJs`8GKCB{8fU47S9cY=uZZ$^xhV)*1oSJK zE)hmJ6`s65_WF~XMeIfXA)$^GBh&%Z;TO5C1*z-f>kdu>aKc2}>2?{H@grC7CvcnX zec~Qe@2iL_lBcck0!ETsg1leuERwf~7Rlo8jH_c0qSyNJz{Etj&bcQjr>*d(h3hN| zP@pBo75Pc_a=DZaB5{Ax(})r3!-UlXD!%NEgOT&Im+|2qEC;VlAEnmnCLeo z)GZnTvcsGT1yZuO9CL06;tomMbvo%YtFYloT*3KXx23kC8#&*8OReT9!nLbdVc^^L zK1b5lwN$pJdgtws@X2U+po~aW)=C#_;s8I7lBrXP$3N_PJ*|~OiEdDnDbV97ogv|QHUIWgso?A+q%|>G1`2nBnx{S7 zZt)oNxitiI=Bz3|f6byGglYJ^`m8RNzj1{~cDOb9%c3ux?@~GL8P#n& z%uTZ+5;wTV7kT)U#Dk4XyGE>+_Eyg)5D<8+nwxFS)nqXiAUbttYZl?UZV3y-j{*jB zMMqL&f!25tqN##}@EtiTmxBZ~t9WOp&F%h$1HPmpY@aL1)`=9RZkTdMT^wRB;IuJv z@KR_TlhTM1GK6R>>&Xd&*Q5wZtlPnxdai8Z<$I)$=C<|@_uKPvWc1qEAIE&!^3v^U z4*uP3Nd`ME-WrS}l7{`TNbZ2n6Pu{wuCk)0^0M}n$gQXGs2|yBw8&sBjwEbYT76>b zKs!B8HL8vUnf_l5@Oj3Y(XR~c06W#8!(5D2pj?RfS0w_TX|!pIghg4YuKtGc`CS-_ z@qM3*pbJSddV z4~h?ZU6I!l52vx4v{5vmLrQsVSbG_UU$;KszvT6MTB_wevqprh}8k1;eYOvrLD z{B}XW3IdMl^)FSu81LG60TNU8=?1^Y8%)hGGjLh~bm>R+G7XlKjh68k2W$6cG&ot9 zpE6G$xUf+s)p(9tQg-VGT6Mm`=eV-^nh5!3nl2ta%YFLfQiMMUE845H@KfhGS4nR2 zMD0x+`6xJw_l2HyiV;&1QJ6=~#58Ic%|9)9BQ(eaw^E}a5d*hKI`nB1r_7%-UUgcvBy`OJwKe0Q?6?x1{k? zyEQ1!Hdx%?N7iw`E5J2FUbJiNicP+2AEb?K-x^fV`QTIhdej$8r~@UGx|RMUyL03s zKSC2%vd@k_rCIpt^_s}^IPCgm$EPRwAi#b&k;;!)Z=CK z`|v#pL~n}=f@}!w`_2zn24KcZ6>wG86@Sg~Isylf4w!e*^};m=)40N(-Yn{uH60+~ zUTRl29gEwi|3gH_e6YI zR&a3P7VYt*By4s{Mt9s?hp@2WW5ceT@32)2{=s2cyE;CaJ+P@>b>L^}N%lmXxWhcB zTBL2|N#CI+PcPrK!2~A3?9>YWWv<^Q0U6AC=nDXcWdx|7@k>sC$_y2u%-*Tcb9s-i z#-A+z7F|V?|DrWVbRX-tZ?_3)x0FNM3XGarkkL-)RjuqEs~)Nx>twzS(oyZfkoZCB z7Da8wPHl$2_!ae~d}3@RXSo=!#SQ%t1CfZRptA=qom|xJjr_2Yi!wTe|5#_dj@*v> zYZ4%^eQX}SRze}?`vD$)SV^7JuLzkr7WFLgF&QW+p7n?fOH-rjgy3K)xi1roiiLu`%0IlHYsV>%<-OA2Cs zd2C-R{gx}PHe{`~+Sp|7DthHUyp;_M7JR;&tCjC*q6R5kbNghzUF`he*dOIfzHmU5 zaH;$Nj!s?0Z~A7*G8=56f;5cc8fV9CcA$q@`*L>+(vB1x1TycSF?R^1wz&EgZxV#f zba3V?ur>skFL9kKL(`*GguXE9#7=!+p-j7@s+O;;8SIrz%U7%uZY+hCt&o15R8PP} z%`2m8chG+{fZKe`;s#``evNHtX#-q=&v|^2zlmx(Y<+X|#?4`I^~IOJmSHiM-)k7( zm@o^C6M#?__@(m<(sV0O-%{#sMjOZkJANS6q}E&QjrF2ySMiVKpc1>@`XnhVmKIA7;4$77N*N76?$>2`8ai7E9zbh_XJ)mWyQ5 zC`p!?^41+58^E(9F0mxCnAHPq-KQl7`e449IfmR^soQ{2WTPv2JU_X6&w52EjsMAj z8#Et<)NTRRwR{$->TmLolmo;c-1BZRnBedX2>g@!h-xuD4sMAZxo> ze20~-WZfF*as~+nw)p)2P{^VgRsCK-qkra`1Qq9IbaN7PR%tP8bK*TpA=4;YpL#j~jmZe*w!7HobH?Iv{|QK3@^XJO(-X{J zk1iPHcTZo+*}L28e21;c*?X|Rd0BgXN9_L4^1!~NiA{C8tMy1R`gyJ*-Zsc@sFf&d zw&qE=+51PXPS?@-l z<2dS(sXUQbSwEF4>4DpkJViO5d><5)33-52>b3{bFo+y;q&C7-=ysg67R;cEJ26w# z!Ubwt9?eR?^cBJ?xvejl-TsoeQ|Chm%%P|=Oj>LU&GY`HKEpYm&Ettj4 zfa#SHtGVa7q9KRoz5)Dl7tQi>YHSWj`aYM2r7*v~?33AR4SxKyq4@9njQEMCW?Zl`luBh<9Ok`>ppg|-KI3OWT0y-m% zI6=k_`U`Ei1t)QOTq=$vsJ3@nC%HT&+X;P;LPfJ`ndqv~gKAZTmSX&BkCkeIo4A1b+*M#Xl5yqg{S&w1`_{LS^53lRBH`^fb9udlHK zc=Xw(0`@}CMkE^hb;IV0zl3nTQ!qfgLYAi7!@f?YZZ}BXAlAw=UtBT zo^vsyV-{XQ=M5C#=B>%{zU6jB7pcQtF7|H50f3mcEc#sYb{HztsHaQd$v=+2+e+KC z%EP^U8V{J!@(THK0BThc&iI-s;qrw&MW!|9I2KT05sw2A0(2HVzgHB?V(sA0=j)Ow z@}q-?N2IUDjLmOY`Oud`o$@ZA&{N-^27c$jg$q18oPUURi^OaUUpZcxHn0Mp-!|}3 z-!e@XT&?vtL~}cE%m$xRu!iap2&R7%O`C_9$i{!Ji@&~N0!-}tPnlma+=;1vFt*y9 zWNjU5GhNdsQc>zKkP9KFLHDdik<{W;^qYr!Oh4h~(&)jvbDcCXrj(ZRJ|U2|sEJVY}5eFXA+ zYq;zTF;AU46vymiRSQtLj77||>&eBz!GT6FlH7V!<|x*d0a-*G$j|a5{)J$cqR*oX z5iI8_Pt{wBXobSPP*Ckp(0RY`V`z|ad!2%XiSaHFMydiPc zR45b-*7>I)f4PVFt${Q&^p!iisZdXT@i!0*JcM9~a|x+N0b{b$jG_Eas&G++m7xxe ztlCEI1`@XZ;f9T}8gO0OGU%02{AEUGi|-0BP%M}TX}s!%l!QmI_Wo(}8FxK-dcWmb zl@Tsc*#YM4=M^4_7`#FSl1vrib6shbDYs&nOrNjsa#*WaX`+gi87BTzh0=MqMJe=t z-bkq$r{HRWaYh|gvNn5qd?m|zvvzd(lb7WrU@|41OQ4!&;b(DawYIl3D8C$_yQ%@c z#P!{WBSt>;cUjUg${zO#V(<*hjrC)c;@}jvGoYTE@A~&58D3xdVkOke#B3@y1_C+Z z*qy{n7*i!A85!;B*IP3q%TX0cb`KRp$c|(rKT5$+FuLbNag4xwjO|Eelb7)E+_*DM?ChyA@2&A%-(B=s z?+%P8w$#wuza`PJi?j>r-N0yZ-?darj{{UhAZ*n|C?XRz#si3JN|lc1ZgE7%F&AJr zf~eYq2MAoKrY00ift&eZZAOIEuxR?S6L|{XlmjDDry>Q%`0UIa>4G%AwVo%*wVTk~ zqC6jy2i+T};LZ0&aO0#LBO}l?CNi4lT&Pih3&n`2!!~6Z+=azdpZN)^EBip*7z+s?*p-Eg>N{ zDkGIU`o-9XwlTUJS85o;#k}^wc4>HTkm@tDQenN^{iCj=@An^OFC-Qr`y&H*XN?eu zs)$Bxbo6uf%q-_!0TtAT>}0i?0!uwn4d4YoXXS6J?US+jy{{VowjEd#W4k+YlJ(Wn{;VE{#051~##2 z$_5SGe`vkuNnY`h-x<>%jc7RpA}9wTmLsbgBRFTwdfQ}0uy#Kn;sSWC8q;pBVkNHU z3CMP6!j$zceXx%dsZ4MVgc$+FIE_(Kr6J7*>}xvmy7RIoe7Yk1-_X{hu45ECN|BB8 ziwLJ!Mt|_jK&;)w&ca_d2Nb9rZ*cBWw{@<;#wx{g$qk639?BFQgy0N-*N*GhRIWsP zbuk$me#5*w!f(xBxGi(D^`Xy3}5nfMF$v_L%MxsT<8 zX*YiNU<`n)(M~E5;G^Zej%`|3)wH(-StV?FCHReqJ>MT7T?#*8EZzZU5jvOa6DDAj zyJKaaCH{g8cA>wJ{Ze+p%}^6L2Rju%8CvR#r}Y~r7MG$bx2U=i*S1LU(J`}fi1`I% z^#TZQypBBPfb0Xm)?-$IW>!)6)h96qJ9`FIJnGnGhN=qkiDrCvZ_p`Oo6cC3rR~L1 zHB*h~!oUjP3S(u5tQl^K2K8Z_>VY4UEz$q2&f@x+HZsNZS%{P2(Rr8^M>+;dX1D(< zn=Ic=;_fwiltA=~`M2GbsO#1liEEQf2*7aBou{l>O-|qVAL9F7_RcO3+L|Svt7lPc zAD9NFz&jwSXHX`NLNjBnTT3(7vk4w*pRpJp?P_?)c>r#pH{CJjl zIkp32h<{t%9xtdkA=KoXbT0P6*r1ye*h;&{2=fo}NMt)_@c_>#A?i{58jESpCw3fG z^b_|p@dX~s4}xl_%$4>nKgd_c?&n0>ou7_Z0oyr=$Ho5Fs-hRC@}dH&%9KJ0&H35y z&q9blO^F1?oXh%|4pqXhGik3aVutD)Yt=+aL_i_p(gSTpU@`s#5K2N z2*hp3EFTKO{te!j$}M@M>{^}_Qd3t-fh1)L->)T7-rS#!xkM^SjJZN8>Bd1L9O@)K z^THzb(4`t*(C~>`bd;8;J3(1MAmy`VD+>@#(GV4(jMEYo@n!ClW#=%f`P_XdP<>LT z|2ZoBA9eb_%qf>f3fA7I%8MBcsBp*i=_)|X3*X|`g@X0ve-x?e=hS`w25?UZndX{r^cN_w4ven+#1mwv&USd*~e9aIGkZ0@hY=?X02PJo{G`er z3Q?!TLw(}y|M*I!h%rgKw~skn-pwAbx?x{Tq*vVeSykCbNktGbyC$_>K9mG}uQTnt0$q!^a9fdK(QRFaOt3-$Nr-vY>gjHNs$QiT_5SOm&pRzgSjN<=? zhW|#?Uum;1niPKYr+O6iQ}p}|=x7Dc06ci4#s+B1e>#7@7t=G$R}o{i(O`O)#$uO%DMPTpEx$ve0o2LAh6Wwb7ViUcQJS*i>zJq*(eDt{&SCImZjMB5eI#+M~ zn7r`)csjuI)7&S;jt$uHMrV<+I2VGV_n~Z*u=-F`7K z(I2ok5<1p`v#cAUv6<+pq?p!I77pf`9@EZ*#!fi=0crIx>tXP0!YsfF-1w9c{Tmh zl6^iMjT!!+@D^P?zq1=FuuTS2@bZQj86Ci*@4&aA^;2TaI#b7vM;$h4$~M`4eMlQ~ zjMXJ#W0Xu>p%CZEXihnQB-*eR!#e#&dB_!Ls_5ut92|#S2TC^+_M@$`P<_$cUYo6W z+hV5Txw&=E;@#(^R{reM@Kct7b}K$vTXl#5ZcBv8Ojf7Y6iuCZ2zj)>$!(#Z!xnHh z;EIN0B0Ihp5)MHx=8#D8=YyN~eTHxs;kKV=yrnsao=rv* zY6FTb-Missmndlw=2!eR!Kcz4`x(hv)SdT^E&FJ}Z@aiscxm3sgC+m@A$?D6cO;v# zTUWYz8C~fl$-sUz)zu|L4EtlLml+^z?-3<{1mDLU!^qQdg4ql69GD#_P=A})9L@O* zsytF-5c>l;y)v%~vY;eOq&&m{A)dg6d}RGeV$1?zNMJ!ai+lwPj9Jfc-aXoJ4?*eE z_%n`wlRDyi6o~6Nx}36fRrD^rDH==^$M?eH`7VWmzXD*9k?Y{~J1!xdtY1!;F{XQQ7;7=+TIjJziw%P4P6% z5)3A{E7SRY{Oi_378|c1z~=?g0Odgz-e9($xENjrFO?;}R|T>qskNy ziAcvDh~~9z=i;L(p?8!}!+QI*sorSVPU_-k#ZB=b@j1L?2|!~m1w1W7k@vozO@9X| z$3Jat@3Jk}a6dVX$}HIdjF5BP2{{ZBiEtqLz6<(b4osfCh$tA|xI{FA^}cU#ubF6G zbUg0k%#Cmv6SKB-9JQ*?t^a#s3hn8AI=_CS1JamOZ$vPOPIFuz2ILDs+W?y<`Nqjy zehW2PR$KCq89+J6g{DF7pmKd-w@t(g8I+0-7@2(b=R>>==6GKevD zjV`GB_+N?pKq&%p*c@ipsQr5ur8R=vgIRHASn=WgX8;X%z)DXMQPC?&rP24INJOd*4tWqdSgtw({M`?H{hO2fnylTEYh!i3;o$wB$+9{H!g^1A>@H8iF`v}^VR5;)1ji5t1p|qe1mzz$)d4U z9&{_7n|f2s68OOdPTSKdleUxTb&-AKG0Fe1Ir08Bb@^GSt7qwZP#n;V)5LNB11uf&Ybj1QH0(EH27k4eVOJRElkRk#2DceP|pIr zj9i{pu=>oE-^q!RP27w~`3G%#02Myds2CEerYMh&7DGT1MhfTc+Uh4M+Q<=4_mn$lz`&nisG zA~SH`u>0{%w%i&$fj=G^o^GnQlbhQel-yN579OhF{@bBacI52)M5 z)!)f2XOsVvGLbzS?6UlHz zNg*g6t}kl&NDco`@W(GgXI`5rgnIsmFoAv)09Wk2vDnnH?rqN_Nz$VW*gKe*oy<&j zNoZ=Y_`2A)^&Z#BXzrD;Ra~e2(jjiOldLp{ifr!J)Io$nr3uo4P~YF?;J(6FOhIa> zz%Kh=8DU)8lIs?~vK~hUWmmNHPParA*86gY4Kz$pFdxo&b`j;BWu4e#f~o|~YL*6g z03i;E@fpV1b7l}*%qf~^*C3{Q`Jg%MgqG{q`a7}R!Kc07E5ewy5wAh?uH%CDnTZb_ zzwbM0(k};dM`ku;*Njn-7@dj;Vys);EBDZg2Z*#dmPjU< z#Gqf-3tYz9T4vIAA|mwUY|mk8H2yN52V5~eGF~)+dbNX@$Jv6p{ebF%}4kYRn#d42Qh@2nll{Q>tS zK`k@)=VVeQ14k^V zA}fW_RBh-KS0_wqZ8p0oZmIvvdU3Y(G#F`Hxb1b4e1|kh)Qa)`_fPW=U-L`0r=zXz zcgWf>bH^xS!ypru7;?LqUAA?l_9AY9AzRT4YDY~+x`Nh7CYN>PbS8|p8VSWAbsjZ{ zkO1MeoGh&2B-<7R;P)%*9sm#fnd=FIM(RDp> zdLt}8`m;tJ)1+oz!gW|1PEVt-;N>SbOowpFqO>YHXKnBiP3crV0KkFFTUlq82yeER z$qTEE=YG)%R@E=xp56h%9=47z&c&;F1P$-bd7j5@OsZ#7(6G&h0X_AdrKdNbxAwYFT7+kS1xdXQdj}5b8hZdUw%x6=Gs#SOZPhI=-c5Z%@<%I z_SVR2DFeAX(y3-$Cx<-h!ZB>!jD-7pf{-q@=;wg=3apwFiJtdzQ7g{IK|e64!MU|El+|SH+K8SIzZs^Ga85JM7c@o3=GZJ z<;gWfyadII4(pO(!L-RcW+fMn454KFLKVDDg9+dd)`8kk)e^&qgVmTJT25;okgO&d zufs!e@=y~loh(82oZLgr0a^%7cH?4MCX2oRMPRJc@BdYM`p@kBe@}5~*q6NB&-1t( zI1p6qb6&?b;6MTU+~6?)32Sz9-;hH-qUX3>m2jnCLcbw3f>Q};2u)`YldCA9k4#HN zd-g31Y7VPax3Hd-`_NYh#BGDRQ}Wk?eNlCIhaA(pa|rCTL!wRk zc?z}fo=*t!NP)r;Kzv__Blc(3D+gPGAkt!axq6RemC}-xUCDE}Aa?WpSOa`@o(N}; zwebj#GVn(kOUIUf*q;?l4C@a999tt-dVwqQ`M&O<88_ZP2qxx-m|HIcEhTWr6D<{h z@;XVt4~#>)z!_wMdXCZTJP{Oy6D{T1DfDCos@@b9dOF&$+z) zc>u461Nz62{0sU>zz!N7bU?D2qyxGt_WNdGi5I1%1Tj2`jU@4~i7~Qc5O^GN0C;^G z_?Ivv^I1t;%8j}XDr5&N1;;Jf!peleugAf(5Cui5gpKs}gtulM4WkvC)DYcVY=)8& zUpP1b-4ER#AMbYzg8FaIydZd2ii~Rcs>0fgUo4P}AqZ0n$$*8*2>?dP`-A_7s&4>} zq+z#?Z95y=w(V@JjcvO(wr$&XHnyE?jE$Ym#{1`e@2&s)Zq?LO*Gx}US55bW^PJ}# zBcaawdRXYghkOFU#6wiXu=^WoqNGD2|Sgm-k-NrOwljyxJz=tp(z_{@jlw$WdV&4k5zB+ z5i)h=0=dLH2Z>Vj3l(h+JXHxYZmi{g-b+Fj*cF~*&cOm(C0x`Q?C2HH^a3es!M+fm zn(u2lhR2!|tz)O|GJ8_IFIn+{TARws+MvvVJhlZ# z?9{tBYjk;cp7^n-?@biJei?(L{w0JYPV=~dH zF1um-ytp%VnYs*~EB{=76(`}zxk6G3R@SY!UVYRLXT?b<_MAH)2`_i)79)4%Q!;w_ z7RCMC6KVF`6SR!#)}G-w9%{j-azm&=Ze2@pLVHdY3WD zMngXtZ{ziwM#!fim|Ny)rM`P;lilK(p3};9n(%jQUt$JxGONK2-Se6uFi{k{!#V?| z6+%FL=SXgW$it8g{iLs~?S@-;hYx{dAUTt!%vd^nO_HO8s!SoB4n|vhWa}getD**S zszOl-Q;`qgap0cxX`h9h_7_etp%9Q6Q#b{sq~&uaSu&6pzy?F=*i=%+;|nY5p|H)yfXT9yZCe zgg>(2CAVjyBTI;)UV_Okj#QOM2W!BCrS}DbbSeR1Lr*P2teNm3Lkx!r+yN3`Tf_at zDry98LqaxNhS&((`jn4`-|2`lz1muKF?5`u?6||+vyY#^cyDl97z^RQ z72xmegZTG}yoo1&3%|qPiT{Cq>Oel^`w$^oIDUpKBFK?8pN6AWOQ)sepw2&iwI*yJ zts(FeA70`+s;z0a#8cd=ZFbJrPyk1UkgN_E_;#v=Vf-FpCl0$XavW_#cgycPaAG_q zBapB^hOBR5QGw@fX*pN_RjgFfeJ#abUP5G2nLgkP_u}Wb` z`D8TrLStUs z|G@lS(af-ru>W)p{sftwV`+0ExZHsLKZtGUj@4xCR&DcAD9yBkL+BWOs_} z2p>W$o4WXMaKRG1WMIKYnk)hNu^6iVp;#XOZw!IGf~D6@?FAZ^wQFj@i)a2w2)orX zo|+yBT&zLy1hr=WK#|$&0mjZ(kD69(uw@`^Sdh@y)b2X)V_* zji!e>5upk3^=*}NN&(rRg3{mmc~xKkIC@fcr`!7+;E~Ikwb>~YNU1BS4GE#!`%Yjp zl(}?~0yDGucIb6DnSYcKQe?hJNf|jZ7Wo9bgh%ohRY|(O#xg!Zuk0A@0Vo<3G78N6 zAXUq*M-?9M?6mdcukjju&8tLJT~44YJ)(Oy6H|#C;h;#$6aZhIN_hcNFz8l!hmciajkH?3d-zxWyNr z)LS&WqBs11iiLmw*Uxe>;s31j(|ig`i(Iwou6+tZM0|eB^k0G#X>rCbH`Iww7 zgDWY`&hQIeutYXFOd%SJ(IgZ?FhtXL2o6Pa14Bbrwi$UXy|a~0l|MDq>Us?+l@N$U z`QJ5aetxsnw+>lhv`$d9IcGcrW_%pByVpfeR=uyfj<C@PNKPH!2;Q%edFy!+Xt)#Kf&0P zSxuN~n|Em+n7vzISJUTg0@k_p`vi&n5Z(~Lu+l@K(zl+$x@YeoGAzQ8Lm|0`(*1L* zF?UUfR$(2}M}H%5!Z-$x!b3cLzG*i{J)Ijv2^l-W2Fc;-jqEx@X)-^t1m!WeC-$o& z)S5dY2HD`+0;V@#p;lSibNdky>@jXAvHqGvwWZ%^NN$e^yT4r`Tp6wd<^j5nNs!k= zT{vwS{j&<5ZQnWJt3NFkdL{bi`M2o|3%>u+zVY`J?t_N(mF}wn6P#LkhSWVuqx2Q; zvjh8xREf*HVF4v>>yzAM-2MKY55jjKP-*`Q+qTsJ3~--oKTw7;(!z9dVQH(svamI` zxv{;pwpE{BTT)Wd(RErKT1r6}yMh%w@oRaeWu`(f4yd$ASv*eaE~r2>0HmF{s259r zGt!_uKKZr499=3f#B34Ph|M)gwJmXQwsN$zArJE!*(@q&joMV}T2%xjY`KYb7Wvs#9?$IOdo71*db{aECu;CqB zE;=y`4f`EGNf^@{acs@8k7&*N9=Do?^j!zv8_eQG!)EmkO6u9Gk57qV&0s-c$&DIx z0B>8?A7-s{eH|bfo;*!@dFw~q@Yj=;j40~IVxXWELm9u5?M>NJKg`KL z)Zt?~xfV-Lo6*3uGA^3Ob+1L>eND!RI8DVasTlW!J9q6iZ8T-I=K&$8RPvfI0u)^e z(%9{(2s*|}Oi4r|1WH}`1!4=pCc+&FG7&HxO2kPPU~8~CxbtFvrAPvKiwAOssws-U z52srQP;blN7f;1?eQ5rORpAxB=x*l5n6zrCC5pUp2QtbC@T=k^p3)Df`JYuB!RR0yEU>M-s5W?-zLzS5qFJZM0|yH zi~E!WGkd$*^bq3XSO@bnf*ipx1Ln`b7U?FEN4jHP`0U=jtNtivrM0>n`iIUC_KgJU zw?J^px0rCI13FUvICKeL%vRN%I>LkI(}x5nX=!UkSHwbZm$W~;GDcjm5p@xc*m5^) z?P9_2o`HJIu3Sy6zlnhDmI#lYh@^4_>GmHje!8AJIo;QU{3^j7^Na&N)ql=yhyR+~ z=x#C%h!DgnXZKPzH9CYp@xArHF!e=|b`MGI1cQ@y$C;dD;y=EV@9T0e1Up1MA${}* zUq?Pk`6>-HXenyrRKZwh5awu+oxKVeaIG@?lqu;D=T9YV9ytLB69RsqbxTr+bxG5D zjQgPcW8a&J0>~uZ*nD0^JcWn;+%l08?0;{ME4^U>2uD~*V+Wm4V|P7K1^4QdoWLq# z0L(qMkC*H}e*L?0P~TXdGW)p@dd(b@`#lhPEpEUSu~?D<5&|LuGDJ9o2%x+W;2ayX zA**LXdaktTA9nzpw;a5fg3A2x;X`Io?9!L(+r}6B zXI|)ySORz=CCE6dDSWWq7&1&_((8ze>K#`kF!xc!4xp-kWcN0PB5xzBZ`B#Ja7vnV zmCM}N(D~V!_4G{S*V)ECU3czCU!vK`5Tfw((llR@x^h+8`Ro>pwe?|)`J=j}siUE{ zqs1t}K7lWn8=0%5@sqFO8@{6rht7Aurk*#=?*}|YK6X%3pIub^A)NIY6^~WUlojh6 zj~gewtj@+bJo(a+ob#ZdvbN4r=L5A)!N}+-&>2VliwZAbATiz1gj|SkAumNBeR%tb zYF4HqJ0)24x^N@-y+)vlW+!Zu*sZq6Oh)(I_ZgZ&zc^SE^8GxKZoUvh^j%4aW!!Gn zvkTZ1LwfgSXyFT0En3=UUUH6|+`)E)D&fuR(m(aB@VnZVk92X7TN)-4AdxaWwYqcQ zkqboo-5|oK9pc@?x5l|CH9pS9lAAb+of;@u#?>lEqBnP~dr;I!`Lz@q4%*Six{|l} z&9Z`!7sVJ_#&@|wjzBKc^@_frTSZ@Zqzpi#InB{2CQ^-r`ip?e$3ttTnzX@D(X%}z zL^!VgbI@Zhl#{bS{PfJ=2+qr7LQEtoZSC!QI}*AX8iTzM*o$q&=z zFHKz5(ygLf@h>%^Yn4%kGtW9+ReXv#pIKiWjE<=-9gYz!zwEc7Wc?c)exxseN|z!#+zC&8&`SD>>liu0_lgBWoZx72 z<4f0aF-l6>pK8WRa{$x;KMgv*TKHZ>Fx!G`{vGUCwmuf<4vTQXn=I`}xjiZ7Gg1KZfT@E7omwv41GK^@D`Z^gZ< z=`aJ2=K^^~rPdlzwU|(yCG!HAp`@81iI@#Za3qR`qLTkkR1U$V76j4M2aZ^ehu06! zFb8cBnCFTxcXe;28EzZh0Bmsz9byni%^|sz5v1xsWh1i+XpbdOnjl$ zSYycZ46WxxDK8=nkm?Uzsjd^5;)WZ>vFNohiaT4i$6G&Wudn4Tj_Hp#%uX56B1@0%L^L;BcF zq(m2FwuH|?ksq*{xSt)YTL!c;L>o*gv#ZAHk*l)|T~WP!#1%9WjF_~bjZ9mt=ngG3 zvP@keY&oq;TvrSIJ1~RirR!{7^(H_88%>teX1#<=?FpK z1_IO9ONBZ6t7P(Rz983zsJ0`G4E$CGZ8%`XGM_AnO?jxuCh3r;M= zfMM;Z57o5nD4^hh4@KcdH!_EmJ{&&d{U+3UxT1MWlrI?-E24Tx zqIzzWE(2K=8>vnJAkwB}KA>u)8+Ll^BA-DnC~{L~*OXd7T*Of&2dHCXCk z)Mn^A@S`by%3eX3jI<_#mUlwav&(OTjZE3CB$dl5trvB;cN45Sp!#O5b%S>l2d6CI zz(QMH&N8EZ6jp7hSEhq)sdlwg+CdGfj7gX%zS%jUd;rC=o)j_FqvX%V(S1wWB_06{ zuDTVZdY`Wm&hzmSrvc2Q5)i>JP{gX_e&YUu5N3|)`@7-vY8|5LO=^?C(wOfsO z$#N9;)Q3iauGdM%vXbgmHGN0wuZA0x<<%STi#m#U!Cy%otz!Zt7+Lv2%UIGCPtIVn z1#Rf2fPyVHP7&P8-1&hl%hPDka@^nzn5kk!6Zih=qb@^EHw>3*7wCdoYm_tS0wXH+ zE1E3?&V^1Hr}1vkbfrip<{{SFPy)~!;>d@;iMoPClg}=!AJ|Laelj-SR61TVJIfr( zIk;tWwmjyLKeum1U_`Y3Sw~=sHJ|rG%3y z_;hMoQqNzJRvp9;UR|9&?vm`uBd_;y5y-YvKErgy$-F=X!)Tvc~3wqtDE5UlD1$@YK?1L^yJStZz*z%99-n z;tT(lDy5QDhWLz6$J^N*sj`^OYG{=YrR5yhun?zLN7YemYUxzhm`!XKh}!x1${Vqy z*F!u(4<%?pVRA#2?rN2fI5A_7ilJ{Y0h5lT)qUvsVVR~87#$dNW+wDsY@*ca42sex z^1)aPVk_h;XL%a65nLNm=Fh^KJEPAI>|4C`ONW=ttmSEP*)%NH*dh-uJc5akB9B+f zDF#bZquDG{8b|D$1EgIyWeiIx8Nzg;6Sx&Lac%J|@ZaQW5DFPH&9GymJ@%+>fNM4N zDq671(@G7g$jXqi1o=goQfpcPwtNH|L;4x{uPp{eKkYo#&%ai};$Uxt$Xa@EYW9fd zZo#EiPE}fv)jv3e+w?gxLV5%-mczH?&uBgik?pW2D?Pg=%p6R76;7c-{9!h_&z1MY zz;ExLXVaG1YenU6LBW2xmRM-D3h16ixhAJlsao?vl`yqztQ4zGXi~dT2)ScK%DG|O zt4CJXpt3q2kMKQIxejR-lWlYO?td#I^LOXnKSs)X;yMk_ON#ZJy@>#_VxKxr!~cAK zIPY38pM!%}6^DwqQ&Qw=KuWR0LhY9qo5+@+dV`bfFRV5Lcy;4y*;G`v(C zp6QH@x^?O?%vpa0{f(hi)u97|JN*PMXEK1`{b7==UU3_~a>bicr(LsE8=Pl-16GNM zefE^W;cw19=BA^xmY!PAmv9poenVjeV?lw+Nw}!0T3EtJC8wV-qB`KnV5V55>`*AG znJB7RqB53Pto+D=sgqF-%(WEHjj+Q-460Kfr~7JB-1X!>ZO4Z?Tsc1fB222IO1PdL z`jg7;d;dvD{Za$2NyU5P?z~HAMPRAm_~lMWziRc8e`6Zg0MG7xMOk-L9;!bJxc%~= z;*)lhKh))#QlNHZB8eZXYNZ&DkG#9X<2OmRy8{cl9mtFJXE9(NI7Q0N+warhRZhPW zW*!cZIuS%LQp3{6xZ#yVXgi}zll{lU-8TJt3d1p{v zRKF3iVmiF9b7Oy6xgLInsm-3cbjFrSufB}iU7woaAjz;#ue7uj7M_h7r4|eO7@YF3HY(SKW zj2R1R{T?v#r{cXV;V-jTE^#3hUmS7OfsvbEcBg-EE)2ha;fzqpF<4w)#lii9>#mcK z0I{bpI@S0A-$U}81h3I{NtguxFM*U}kr3#=m_hs+l_n{yz#Xv>dG;G6?7bJ-uPE}V zG@10eMCKhzqgYC#Ubz9^yzsprAToW2** z+@C55l`F+k0jY$8YE7Fly23G;6&t8s9mouPiC>q2F_Seygx}}+_IP#L4Gh-W!Li2S zxh&ziB(Z-m#}6bVPyEb15c`5GQ@B+Q z?}`sS#Vv$(xZ^%^>cANGZRC)@$p4_i|G{tn&j|T*4$$@^o%tN>Uu|ItqPy3TPcG~? zs5I22Po_a^2On-5kvPdN7^*k@t9`&ThCbc$$Oo0x!#=q2K@b?`p#rHG9M+yf=4t-!W71I zz#T)SLO+BmB67u_qA|!3uAniyewJ3J?O;Y#9>~~QcunZBq z6ctDLn(107--VJ?^5m>q7mhf}vdM15YE-(sckGsc?U)&Qh0g5fQdm!xZknEuy4Yx3 zai2r9(W|UKIv^$w@z$Md1YBcE^jAlgygatnM$n2mi?FeU)>rq)Z!J7bMHbk@xB+s> z<_cf2H@Ta;Cs7(Wn>Q>|jT-lg>>UfMEq)<^UvwHR6Y^x#)XkoBRya>Kqn5IZHC?f7 zDz-wiv`(N*WKi_aHRa>(TlJ`7ESniDS(SMNR6nR2EFW%cA;>Hsu-Ogc+mVN@c{p0+ zDPKQ7VV|F$7iRKI{N9m=p|urb(F6E#L!&3%REpRjl1!4{9MzYChaFFGU|nX|Ebz{C z*0JacH(WyZlB#7?rj_9_BGQo0DU+w<;=j}>_topWnM%f|nlFVk&qp^ZTtJF-9;qg- zSkQ&FaE~%=~IlqrKty6OkBHLj(_(YFf@!W*n|J|k%| zN25Xp;V3hchbcLUT%qnxNlozaG6qD#%^aXZTIt1zsw)R7&oPnZ3<_hM!%eJ(i+-(- zsDf)fWDdwuIf0{1H% z#f(3)KK{jEUY@>@c@NphG@seH#+h04`x{rgycJ*BSA*CYgBjBdM|JPzr5$U%0%mkZ zh56@qVL30?bG8H3-+rq2Jy{!p(DoFwwC+t+qJ$Eqdn@Mbf@?2ftS^Ay3wR^c<#GNi zL1W%0zbZgnbQ?FbBFb3noL;WSGspkU)o7HzJUb?^j-owIV zeurt2ftuluhwc=6b`mYVj2|ukyD%E#wOPtPeck1Tnve;dCHIaw?0tEFe)}ZOpHdd9 z&a+0ZdRjgafwT#bfiZB$`R?*x@(OXUJWKn>xj=1wXkR}mMop>ViVf7v; zeI+(}q_-f0xS4y*rm7HbCpqJ(;+8XknAwTe&mml@>m5CpzWA5V^-pk{_+)d}ApCd0 z{U_{=FPo;^eGccGKSL?azhk(x3@j4lG?XwXh_w1Lgiq>@y1oaxI>uH~aSkWKTq>5O z1ofP3W-(&qmklijyGR-<;Zn6DJKhe;ph5W+UktAY;yoQKeZi-bAP2z{w6xCDoxfmP ze<)KD{6#c=<@(HS9dEik@_78p8vW)Exr@1ZH7e3g88+$csgLAf>IBo|M{RLS`RnCI z1IxigFp3V)nrkR8Dm0N9RVTc*#96Q3j8JYLh<(fVZo6b`*TAvHsorYftWLD?nod-m zc5i6#V5;A+n`iEl#8Y&w&pk-HNImizK$O$%V#oJ(4z|=o|D%9vm0p2*8A09o!>s6H znz5RUaOZsYw1mr=XK%<0(ZZv(n$O=%%u{9h&rdYKz@48ebMZ#z3isw`;uc-eag?vM zf|z-~0}NuAO)S;f?UoyAwaz&shZx6B>m~=C<6&pj)R;U6j`|Cd8h7C2`1@jd``FN; z$b}Woi~@yoXRm)4i<1kpN~q&Fs=e5tB^XXP@{zxc9V$>wSl1P6XWw?#P{VCnPX>Fw z{Hc+-lf*Npda&(jTz8e#u%$YZ5@$D`ih$})30l!vYpHY9Z5PEXABdEhUar*KDNC;RyBOLgsQxK*?W8Hy1 zIG539=buz1@0S#P7p^cWa5ch2EviF z{cLVTU&hzlA}38Y*rW5oRvhY0EP6fl!mUeV+AI7SOKl$Hg{UciL5eQ;vLYuiQ2yEo z(h>QUW8Lxzd<0ZIB@ulLuzqN0&)z=JPjCk{O*y51dxD)&eNZ1F?q&_WJ+YBCnfS81^toGGRMLYmO;lWdw9{k|7P}*b_ZrqGPO&}$YNYy zOS95-uUpU>+W`DaZK583-)BFAn8!V~nro<-C3)zxXzM0}z zjAy@SOHIC|2qB7vC;TZ`Ks>X|#L3^A;kIw2p5OnxTO)_^EqlxQ*j~Ka~fWgW=sHmN>8wqh?dNQrp?;7&3y{vF-HPu-QjZ z?m-Zu$Z;tdiP;6^@3Jlwt}JtnbRm6A77vM3{^TUe^s%ZxPAEzM_FIh(Y!3#9({s$n zRxdBnt+?T7R6-}dU&Y-h_>IGfqqH6ozAoFzG4Ua_@&DG6_L^bjlc#IlnG&qnxSK71kOIUYsV4y|18gs}ooPnWE zzg)NR36BsPr2a`M0QM3`XBZAQUs<6|cXnRFQR{>7ejLVtS36n}h&#eam{D*E<|dPSv3 zn5~ClMSOxsRA{_bZm>FB=2rl{HJECb99UubX8pJn`D-8Gy3D|iHF)9X`o2f2#ZK!T z-+RmPFya_9r_tC5GY>v?8o*?DS_fmdNFU6JcRmJ9AL`zyOJ{H*Otq^U_-US_FKrWe z{KyBHzKfW~{v+ZM;0kz~hyH)WRpN zCakFjhg}IlU5Oz$=P9KF%IV_HPe|0c#*oO8Px9GNum-0sW1F|(nqwh!$W8-@<2_UW z?~TZMRM->b3TeH(YvNHm;;2I6X;Th#tdhTDrH<*Ok1s#|U7`BtiGcr1tGdwtopAqs zY~BDGv;XBk?e-(Yfd1R;ER|k6sX&2%#3838V0|)YznFaLC|$$h!luoz@InD48oHY3 zniwCH5%SU_@Gztra^fW@j1^e(X!)YSqKKBD1qPLbi6pp$=KE9PFe@7xx-EZvY+k{A zZzX8&NJhJuNUu^uW#?y4gd&uaGtNF@KTscw7M~} z;)^BE(daqO9}}>o3)f72`DcJ7hRZYiwm+;K+hW~6RzJ_8jBDLF;8xGHx$O;wy~pXxIYf+41O= zmZfeHL0Th8xn|N0#pRWWXpx7HMk+CMiHe)0&4!^ajTi~aNS2wT5!UT-59$xh!F~jr z(+pn;_)!@YGutH1)bZqIr=?$txr+e>j^*wl28HohCI;LI-UzFgL}Jl+(Uf5a2cAfD{4W|A@zkBL-kR#Sw4EQ zV^9(bkuH|{x@&CPrf8ciE&*ceO5C=&%)@JDaG7?G8Z>bXq^0OGNrMfIB|HN>8A~l* zl=Gr3G?b-?Xa-%h(9k3q=u5=L-7il>wij{^I zWZSXB7=QI;`S}^dyIv1cnzN2M7&3B^IO;+pX~_zRmV%2}Aw+n~0eJoeuv%jSVX{Y9 zn1Z;`!0-O1p7a1*dnkhLHWo^s0nsXCcMv{=FOM;Lsp7lo7X~_~6;GW|aaIN*=9cez z^|W80Y?%oI!V7U9^7+2(>UUG^`fZw)>4FLNg@M^0#D4F33LhN>6YNJM%7c4C$h&_M z?Mr)#6!BH*VZ(csuvd$C%EF`vEs&~czDt9q~`!?RSr@aAkmh!s=3+?tiCJ7Ke_aFl|I2$rZN z*#WU~P=J<~;zpyoITA?0j=@p#h#13eZQRJhaCVkI!B5ckYef4D&N$L5afm}IZ+hQR z4s~47mh%f>@~qg~1#x!67N|6%muPG*({J!~GJ(2zdN_J$FN2CNy)BYS&Q+6^edSj6 zJE3gTtpc2T?XbwZ1ZOsC6)oNjgoE?j!|rMcOUKU@o*)!@3x>6pfJtXw(&rUU7JB!#K0w$332j=8E6lcrsQ@mAeAMi; zTaph*`iWxehcGLk)@I%iA%=xXPqX8H0rrwB3Tk-DRLJ3nAT@K&Pd&utBj)>2@wKe*Hb1>RUs6%lxn)% z^hW1stu1CMM^Gx2n*Cl#{@GOG*kcz_STlgIqC2QXSxi#SzzGS$WH=vT_}1=*2FMk| zlX8QfR#s5TuzP67!d*G|JeJ%&{bia;5Kq{uHN2&DRJTSKnpg~YP?TTF8qP<7oVlaJ z4R*UOaRKG0cHD=k6!UczA%fDD-vu6GYb!&+_QyGK6Vj!U0P-fX^cbh?isVJJdnr(v zP)|(1<`}LFs(G@K|B5i|n!qhDd*;F38oK}l0?B(sAF*%9Z-eblG|Z3>|4>X9XTjO4UgO5Z8weT`+iPc!?%Vz|DfUSCl) zR>~NrV9Ic zP?OWV54LX{tcD6|t&FtUI+Kdm!P%GtCkUDc4Lfwu2fZssV$KynBuyU8?RaxBbul%| z#%BP$ejL$#C*sAT)%s@Wv?YqkMQ_{3;lpp=I^`Q4Np zcJiG$HEvYumDP{(k?gVhAxT(VEFp9vmv^$#Yha^V^F_uNrT5EPmu-nZd854ZMNwc1 z-~GC8;ICv6Enk}0kiU!0AI0D7zeeAiDn%v`%Op&o7A#pJDQr^uelSU@J^=Nd-r*xQ zN)fea0u-&VJ^R#w9MU6JUx+_9wyK&2T9+DrYJE2w6j!i9L|4FPXol_jYr7JhvtA8- zCmdbmYoDpvY&WpzRrL4OrbWFPPS6V#f^{;xcC=`d@r{8q?B)qjKl+Qkd~VHJonlxn z62Jtn>tmbd)uMZe%&e0$9S2bNnEd=c?Hu7DVy9LfB(X6>^H@Vie9gGWp5KD`3r~kH zs43F}>A38TuN$qdWyJ?4Pg&4Nmo}`ExuUG&k=%ZuCdj6n+l=4BSb}Y%Qjph3O&zJl z^<#$om!>{P^&Oj0tpZ*3USh4XRhm;s2UM-~2FGvNt0BV)s8sE)d>V?Ue_MoqZV=Q@ zoEGG>eECm@@XxK2J3#G>oYor!MUwWNiH{)-AsPy`Et&%l=YLj2UpDd>kv?Gudi1pT zgwO5Jc$`m@IU!{MsI=J-u1_tqZQFe{JH5na3K`uq-U543F|iAX!?f!VE=+(Qj-k1L zU06=80E7BSAC4V%h$5ghS&AfibdM}di9oRbZ=caGSpyqPuBsmJuRboARq3=;5dh888nk9e_L|wU zbvCab72BED$v%siZD+c*c~Vxy(1e|AdpiEpO+0h81sgx8=@fNNV1_{}n|i-Gy{6Ed z2QTZkZblfm*-kLN_;uY96NX0?mP&{%bFa+}!BkpgzEn((S|1#=uJSofv_V@R7%;(OdP>(Vcd3lV^1LDzj2-#Wzj5C$?0n+YGGa zEVios&dHFVvf@ZfS_q9;p=dEo*Va}s%?Y~Xa>Qyp|B3Ey=%}Ev=@6Q5Zf>w3uYkrz zK+g79ZDHc>9ud@{Gd)awfm(JJonkldM%OE$(NNM!8jAf#xdV{mSnsnwJj%|{fad{! zW%LXVV-AlEJt{DAQDb+^z-;LusCGc1JF^UWXoa#;Ym5eq^G4zm{oLz_UY*q0X0Hkp z-eR#vikisJ&`|5im*LXb1ntIYJqq27TKMlnZNU+xbh(k?mj8;jB{0hmo2>!p&KzsL%q( z`r5sO!b&eTvh)4#q(%~&IH%niU)d5#iFS9lqb^{xw1#?X5oq9);Jv zzrd>)^lA1kF4PECXO_)JY-j6lwhcXKxS1!&QjbxH1#Mkwa4EYcFrlq~G`K|2bVk>| zAO@*xM*>%7OrgDIN7T`L2FtTXC#Mrrw!r6z~t5l zo#uC7L6=cexj4?cK{XZw&7&qZycF45EGhLw#eg&V`SyrE+%Q3Np&ydQS+9Ebj82o$ z^v|wamdmXMH6jJkULU%j8?1jo5^BU-o$4!|8K+8rXd%(9Z74{_()&#`IG{LHpZRFa ze|y5RdLbN$lr>X0OZkk*(1h-|_{mpPZ8Y5Ac?$hRYgylgN_HW-gAVpLMr6)sX`T@H zQ-Q$Iy&Q};$dAY@tfaB!i6h_P0u`!h%uB~b+$VnjSN1B@(;i_W#nLa()^Lxg^AQ-~ z(P%v4hs+6N;<0EvlBUtp3DOba3Z~g(5D7A*NC`Bgi{caF=HY7L1XTN|6==5+OTsZL zrsLrK{qI-bVhC(I9-pk&0>WC)p{O)`0ierf_Sre&cRB+8?N3h}KztMQ{3JU79I^vz zY(R(BDQA}K2h)Jf*g5L9a4*W9RQ%|{ca1gVhqjvs5Mw@TLUBDgi{<(Q$3;S^OlyLe zHTDD{tUZQL`DkC3ku>9O!h=y07Nw!5@$_EwB&Fe}Q=D(Hw`{1}61`}9770Q`i@+b_ zk1Ce!AC-X`)qyRcgoICVTu7@fPo{jNVeR|;%@2CPD_jMa>ZP8v?4Wb-WeKHV98c7~ z)mJA6Q%h8$U-3fcMKevU?hHT{J? z0v?0m1#hSh3#WIzAuo$9FRD)}TG$M>nZGK}dOdyLj37&wHTS$WecUR%p76R++7F1#Izj&SAkZOC4OYU>k-GLm zRWbEV2lm}y?P-MC1T%)V!CnJK?HXF6jy;ZigM7h!xV@TdeU3%-5qtUrTL{;v+oLxI z1zCH3qR4~O16ZNg*jE;7bo_meRrN7@(818S_Sk}BHzY>TgUGOaxR0>!2p+Iqxc*vC zFk&h=hUr+W2oy%~dviSQS85BR_1PdEMpBocpx=nkZ^SwGqKcruA^ikyO6t*l%s`~> zhzN$^VvP|;uSEztOt`Vi93Q;6uM`i$eLfK_bncDZ5h8CQsSP=5FI=f*F9OLlNAvEG zwp+N=*X#0k4n_(22tNM)Gc5j3nehMX z3)KHrqoIq-ma+NVA3_ID3(!XUl#%P^gG;NZC&Xy`EsPcd`tN30<+9yKeOmPavm7Aj zf-iycp`l~N;Poy%0xOP$;uzjMkch_2*#PSwybC$`UEv6X0OvaPGGsYuD?x?YK;o7f z$?;f{7MfA7Ty$tSfVj_LH&0n&UO-ibalg=!QD948>)Rh+0XE?6{pU4Et>NfE3Ib8z zqW0kpg3UdB?({$(m`zl*!05&@eFQ*(HA(R1i?w94sWh!MSrvw$|T+`D=Rg|+UW@WgPWn7ks!b+fPXQIu_D>F_hxL~(5y5gJzHnj-*jphxDYIFUD3xruMu{cs!~R80S=&VBMaQq0KkO3k9# z3CRne&2xMNgQ%Hh+8DexhON3MTGKY$%nqFbUuiEfhRSJNbb18Kpyd{YsPz`lrmo1; z@YWxQhV56WB`y+67?h+p3g zYoGO3Df#ow(ffgHcj@O^?6g)7r=r zp3IZjBz#<>^CZLIEdK-pPOUG$lzS0}Y+0n@TZL#>eme+ z6z_kRcX*`Zl)EOwDp2b1B85Y@(+`$$uv4`Aut^><-T<(il6T?!MZJ%$Snc5VaKguS zOZ<&AzJyAtD2unCJVHGlcM?v#=qgn3C^$?iI0z?LCiMX+u*pPXIpO9T1G7U=L`|fO zVPY?w5OVlUt3fl;G?hcZcL7q?5Q`NF(KOMVr5jv3<~KvkUYCih8I-GfOLsKY!x);Y z4I;my4$`XAAp6I^Eylmk@F$u|`+v~Xf5w-9Us+IEejbztsAbxpJg64n_PeRQ%kpaS z)&{H|fy~Y&oWsR4sp~?lyPpOxDO)J|Lkfap=tQ1i^v>VMLOwrN6LamXjSvAIW&TJp z(x>FxU-!&YCM({Lumm^Ao4-5o+D}mzmhS%|Odm(JU=I~g<;q_A9@Pbe>40qQQ%&3l zq1yGJ%rl?-WL9BqnAn|3qbq>21jS2BE`U-a`sd<6KRcu7|6|1d&t(g6ojav5%A%8xyk&7W17Lv6mbV8ZG@7r5eZNy4&W*+ZvkUuH=kj9BvtFFMcei1^x;8saMKpsK3fWeWNu<9&Pl+Xn}c zw-tpG_9E|k41dCh#pAb5Lal+LPG`)-9COw{;SbnaX!`|0OY9}Se8kDMsGmE@c_A2TSuh5VGN7g$= zcM`PUqRGUzHL-2mwr$(^#kOtRw#^AAb|$uyoA>?Sb8d*Q7 zwVi{P!^zCe&8-jc{rZ(3*@8RjpcxiQ&1L4uD!x=08`WNFC^_UB%D9XqSTLxtiN1K( zu4GSBX6%jcUXz<#yNp=p*_CVICaFlArpa1jA|q^c@OWx5t9nG9fOPw2fgrK zQqZpdp54Y842anTr;1Fy8@4z2X9TQo7`uqe_-o$)HT{J;+tF#kNjnqb_B~!LS7VLg zx_@k9fa4+21d9Vq8(eiilfnSimj1>tgNm}y`-Jozt0TZDEp=aHGHnUVG9U9KmIkE` zCl-0p_=Uy6nwC;E->1u*mm|?i_{zlJcj;+^?)f`l^RTsw>k-}^KGZlY*Nx$Lqo}6l z^HWKA2rsm(f=+AM_Oa8{J-mA2O3x)*z|4f(0&rG8KAMDG2_LwVI zZqqd03PK6$wamAgmReHX5x6JWk+gaLc&moR=Lr*wA8*`^^{qmFX$wW4Q-qsm$TyO- z`U*lcfJf|%sLlYM{Z**NzFxR?JHf7?|056rR_c$5We$gTHSu zEn=9HOuG0y+wEb*_MtS)YE(_2?=SF+>3}wovpm^z4S4e zZM~W@NrZnNS{C3bx^<$Je%p`dkHehANl_H5XrkLbr%nDm0_Fsa&fk-EQ%Q&hOQmqt zO;YsC#l<{;k%m-7$VB{~7digZW!nB9|FHk(6_ebR&#nC2;Yu@%g~d$!?E(vt2Dqev zZ(DDmeEOgE^Vi#S@MM}mFS|7Wz@&&09*pn-1W0y#+6V?EEp_k6$XPef&B@4le|*1# z?x(Jn+lSBfxMdMRn-k(0ya&1mM*l^_B$E@_?3tKhoFW}M63dj3y5bj+7%;C@s>t&v zFEpMVR+U@9vuUROmYsurBB42M?FnyacEh9Kd*&Ztgin~1&6lcR zMN!MoWsEoDGD1%(w6m4LHBYGB zS|@ef_DJ4hGuR$CQdtnn?u>-BV-bR&7%$z=pETB#g;PkzQN|!KsA!-e(9-J81B&s_ zC#53QgW#V_yvboDkXl^;x7@iQ$jTU@S^5OT?9+_-mh00nlI2Gncef{X8&PYHYX(87 z0!hTQW%h zMkxx<9WZ`k?fmy6rn4w&QDEfBf2FDaov)uMjs3qSTZg-cRp4i(jsI+!KZ4VJLYy@J z3NDDWDMcPEK#9tKZ7X^})~tYm16Bfe@lxx0#NTX;0s*Li!MH+h{;rnmgQZy#}hYzCWncxHr25pZCSjhbzJf6pqA zrw&X>`4$nnuoiD1bM>%P)XuhVvHuwNwF2wP&g&oZ2NQxc|5{H!7Ml$p3>d zyw2GcoOqY}ItadNvdn`k=(ND3>3~!<=Z+v^vAqSus#d5CA&hj!mFt4*6}?a;MS?}w z5f2qlfV55=b4;}~L^f_&VmxIsoxCWDXy!bi8ukq;4tI9fgme$(*9Y+i5w=MPcBD{9U-w`% za3pG|N@?4LsF5hxWZyXBOVoA_?B>)=_X!izw0a9Ivrw9dbzGH;C5;eJ1x19U%t>>q!+$}~B2q#aO^>vqhqL_pof#Lsr!n$mB6 zFs^+(gJIIZo4H?~2KT~)&(SYo5yu<%8$@1NzRgk3GNcDk0|j%AmwBMomT26(YQ`Bk z!Gix7ITVPq$!=3)RPTqM_k)e_HFAN#twA#-RBN` zJ>vI;Jw&`<@t2&Tb3Z`;Q<(pEko_+*{lB__@^yBj|4~NwjaQqc|0ww!@Y50!C@|9c zD}Ut7G$qIafL0d?GDFj*tSdVYmGdHmJk`H%S18unR#CPIf%`p?$>J zb{^~3Yj)`nm~MI3ee`1=rp)3Xyc_6o+z;^AVJR>l00zjRhnqdnPt+W{f`nH|91v)m z^m$>ZxK9R5vE#?U73YGSA0S}-4+hxyiS!869>YONH|2q2JRZ366dp>k{6_;zFA0I> zJn1n6Q!V@nJ$aMcO^u#%yZY~a8A0))Q7Hq>g!ibJ}rwb*s@-G?!t4rAi2wFng$jD~kMI<@+n8X4;wTwU`k zT0{-3kmXdap^?flqi~&pX@zZw_?bK^CpA>izpL4tmpOA+x=eGj& z0C5D7kAWJJq^lZ>eG8kzQyzO|QK<`+?!^N~C*)w|iegIK)goOjh=o!QY8qiDSCp)` z+1YaZC`Z{;GdEXklSaxv7zm+O*h9X&UMf)##D)>LKkQI4Y*6#*t!=_7x;BJqq*$hx zxc%TJ>ukO2Y%;nuPRmG3EbWX|T|obDz;Ub*v0L-vdH36J44KQ{Pg0F6!maim;p>*C zTmqT$mgMmoE-LdUq;!F$xYpSIheH$xhV9_=C91w>%qTgZs@FY6?8rby0LfyPWM&rO zAH--fNa}2-gW}?WL*dzIOx|N3qOrtyc}D@ac~K!O?G^GQn8n*vsBE5C3EkfY!%AFvB+KX*@Yc7z6i_RxU{+(5x;*%*DDg2Q>C3#` z6JQ5+i?fqwo`NoyL}+lD93GRdT0!0$KP6qrcJ1jX=JeP=hzyHIS2L3 z_!fB73rCHB+{ukgVp|xI%}INu-OkM8)~IR;4diLW)c;nxF-IqFL=Ars1#my031+O8 zm30zTktRp>?bcE81-GgE$U-BV7K-R z;v@Vw@?hUew{)!)9thV-<;wq5Z|!~Zi9$K(yHq-K@@bY|*O?M;6wyxn(<~e(H>-?N zP$f3T8YP0yD663*3hiME0Q*E23Jr3_P%3#E3HgyOI)#IAW~F(CXx4)bz1!OoEVb^f zPg?G$klShV!eU)?2y$q234-#D59LwV2=vqH56ovrw8NgH4GzQRHF&W zC2m5ke>O!Klyd4tU8llf8TX@qWfdjZntdj-Q!%MBN}@tdV^{r^3)_xd#4qx3M~&hP zr+u-v+B2rHp6_BY1Eix)k#Ux`i(GK|gx-X*1!#tlr71!AED@#?<0u1+ieFkWqloOC z*{gb6jzelOF}$gl%o3_6$4Ka{axBkT=&*~7V66$ML2C9emA_rO{^%jaJVe(#q2qRK zNULVaJ7wqpwOKKmNhsgOT=af$$Odg3dV6YroD})z+MRlT1`wHYBRNtFc%_WGG)`$& zPNXLPn*(=HnKkWViPbKlS7YpSMx>cmCbgDLa~cm;Pz|eozBg;op0%uH#?Z~$7!nVI zH`xj|rb+CjjC=eq9{oB}#2foTkBY^{!UwKyxrlLS-)m5A;?>S1njZT)=+>ihX!g4c zU!0o-KP)p64sf7e`M9m^cx!g5-@`a;?`IDPJn`#UL(3^{{PE_{NrRHM>;>f_kCUm8miOwtEA!>IzK zKh#*u`cI#aLL{07)Nd4J)AiA96-{KUIn|Y!d@CZI+HqKbZ?A>Le{RR(gQksmi+PH5*5_Cr=~Wo{F+wt?>E{sKa>k3lGG>ad4{g<1TY)zNREDKHq= zatEHkW4ERka5`?YINYLUXv3HZA22sd*N!s+8Hk1xWl)P>(H_ihQ&@XN4_VF;n=HQ~ zMhWGx*?K(#_2sw*Y~DdNZ?qAos!<*!!llX%x^mQBycTagZT#@P^6hYU)ZTVA`ly&F zWC8g7$qRlhQww#i#$iGFXbW|bIpLSbp;uCS>I5f*l13Qo@zRvUC7Zp8f-=1Jup4?x zf761GTNgH8f&-2RN7{#lv@Pizhtk>n&%DKe;cq?bf*zL+$Fjkk@lvGZD>tLC z5QyJ_Zp;~7KQoPil0{cg1x zq0YXGzf~=-E0Obz#2Ry`pLM|UN+J7*XzkpQ=SNOHdeT_~|I$p*4(+d8RVIFzfHq4+ zI+Kj>f{!KbwQ$bn!|x&q@l9rsb3C({=ZS#j4)X9AJ3K1thK#6UoA%DQ_Kv7N)dCFY zk+RP^E=_2S1M`$|S!J!j&0i7aX4UP+4kwAzeR1wc&nAb`byf2{aBYF=IHl}S{~(<^(&cG zu5j{nQZg%~WXz%Db4=u-HC+x31+6PRFjy^`BUfRb>R4Lt1Ns+qI+Fsm>>!?+wxyus zOhBm^=8`~CEG8Prm$idT!T=d8bZeH8z#GgRt#25SVkLovSV z8Yiz|SYk$+eq^&Jr|q|s`#{>0nZ_V-g;|h}4EIyy(I-E$C?}p+^&9NG(m3A!A>iZr z;boC*>FHMyUNn>l_28w5p7>L9h60*>!R0;F?2&xkX*6kyo@Aw|HGs8A7cSg~_6*Uf z4>}(X7A-bR0X{k}x5lx~u-Bo{tlkW9%Jcn<%fMfcHGL!B#w4^!JRMhZ^~)1kpZJmi z!dky<_8IDj>ebV&x_dvrsJ2^f8MNzyuwG;l1dY7g0>LUCBM#p6zx?+oGnqj?Yv(*OhnGBfdBH0 z|LaTT|2w<;e{CRi%$hj`e^=zSc(+PQ!%@wM=E8|BXj2<~t{79d=DT#+SEWCTp7 zPj7qAb?opyb3bQ$yXAd;|D4T{-O56AGScD5vY{A?#9s&ykPW2|hBamY;17`b5{t0b zeP~AH2SGDb8OT}F;tYFH^7bPG!NAnfXqX=C&eUv{Tt%9&6})q4Muqa^+H5JC!S2NcJjw zjpB%6T;aAAm(^q^S**k_CYcr~GB)8^hc+W+oV348d3!GZYI{qpQ|&NXY$jY=kMM)J zb8lS_{x<8uvz1sd1vATyunsp;NLru}H_p2lE zv(+5Id#K(R%lDae)awiPohf|+k^`F~aoQ`4(Arcs#l%F_A~_LPm1Qj|HOA6lojHFD zrC^3ILdx6i!yE0n-2N~|N1i8Ib%_KQf$c>Wp1lr|+9~pCaW^`G=~s*&SWCsE#w8V4 z=Pl2~bcqBAwbpe4bQ>dJ?x7W^}w6GP{=T62{Lg|_fHKmJ9)v( zD+0td*kSiP8_u9Mh<#sd)A9HqHIcKtMkLGl_+37M(Fu4W`yLFcT_A{HW#wntf%S`+>56-IqfVz>s5~D?O#ZgM~t)DuK zu+z>oPxRJkXC9$OC7*0a$tDnKwjX|KNW+w`*qqZXM~QS7$e8$ehM8rd$L_I;oarI~ zi$Zl*N$Yv}dv!hw%BT42(b+yc9DoSH&7X2j5dQNprrbPIX`wY@IN9kYR--&ML6)EH%a zURN9oaasrEgRu7~reXejVaUCJZ&<#9H#oZj*=IfTj;U`)0!DF7k zkS&T^a4zUzv$&1L1KE%-#?B84#0S3UgURfLC-oH#u(=O2ekXbOf_jBmDv=It*h@Z* z5d50JfB^0P#rc#vAHCB*%Z&{@D+|*776% zQ-G5O0sS9$Sm+-{wG9rQL-Xgte}?cNaHYWiGlU=~3I#zj_|X)`|5_mE{bAv>e^@xV zv;{3}tTcHJF61<+J#Y$uv^X#j*ZH03Gq_Cokhs95*mdcGFQc57O5^IA$V=dif z2_Dwf?sW3lWXrc0JL$r34i4+TL4w^p2Wy_T8gLs$3^S$^&pCH|=i}b@uYYj<(&-5M zKA5l=LgAEl!H5XN0Z1hzLl$G@N@IiJze|c8Np)fOoe)uAb7~y`GIll4W309vjT~eB zshC7q+MW1m+f;3ng^wNImu(Yu1YwxJmP%QAutI|qdB-$$}Y04e)rftc#|%f&N~Nkzb>xJp@y(ar{&Lvhj<98lU=Hb?VAHBAEZSv6nuo6Jt@ zo`el5j=#`_X%+qLZJ{&`Fj+HgG=nPmP_X|+kTsPmVuZM>x6{!lE}F9ZQZN~BvSz9g zY${e!4K{!W_-Sv2^Ygjfm052iX9;-)V>=>*!D0ep6)c64$&|In*&ZQQj!f?RZpt4c zKM&Z4IJS^3UHTXv3P5s(28I^22Wp&6N#LB07JfJC2 zTjacZlfdsc=t(DWJh~^<`w)Uxq1CUn(3;T}zgJ`v4Rg$F@1!Mm$$2WchWO9w_}>x! zzbL@}V87*rF$@g;KO@ZxYRRu3mO29@ZD12w8c>obYyY3{g`Jk1bP_w9#d&n-0f5UzNX$r3xhA{ssIc(`?D&>Kfk`>95-x4FQo5k-&Wc{-7AX81<0=hKdL>H#0Zi z@iaTE+u{33M1pO?BJ5} z0+1#BJEaqLEg#dV^c2k>mL96_$){EaWW_S&{V2g-P7Gl**wn zoUS54>5!`+!>Z)*EWQhR?I=nioVo0yKo&@kXvmEu9N;5g^H!2{&MR2n&>q&EZbpdc zED8hdTTGFK16K{f!+V&~3^*aeN;j4~O_)Y#X5$CH*S7uF`-j=b;V!t3 z!YZrgiNJkjDaqzHzO;5zE5iltJBbj;Dp4>8T0x*yFm@$jEi=!{TnghK03q*X+;BgK zkJj)kztjQSCza`2gjBh+vbq4(c{vKHJ;L8R(fk1;Spa5_ z+Zn8nmr>XOh)Hh_VYirvJ!#M{NVIRzDZfZnfQ;ww+`cidg_1Ll^eCEu6QU^-Zu?!E zg$`l%Rx8dgNP3ST1`p}CYutiM&DJ)*D=(4z$-*&C_BssS<_T;zn1SH#nPdqDkMAVk z+WR;59HoMTz3wIq39R9<(_`q=`ro&B|3S|GLdE~yZT$x?ZRTto?!fGsd& z*$tC22Zx#Z&H$FS3lE0?5Vdo0vNU!6pLoq0HEm^7b;K_(`yo0Q8c3L_5f8x-Q^%Nm zLKrGY3k4CRqJ{Kf!zZAiDVo{o!gMLs$vXrR`@49R*TE?IbZ9FXQoiD5Dg@67=|p`4 zs#joi(o8;P7{~#t7)q;??4`2zht;i}v!S`U-gn@BJ0ENz1b~-t`Dg(mv;#*l=C<;@ zIwXJW_?#lxryx3hQWP-Qm@vgKR0SIaoE#Mk`^k@i0av0^BfSj5wLxUFq@ifbfdD>~z6OtrSkms(%=jk95tsH*H5pq7WQ1Sm(C_jL5j0g zIL7!aO!Q+)s4#_yUr`!LlrT(Pyyu5Q zr3(;iIjxbU0z!Z+w zJTscM0-$=%mwoo zv0=1PbW}aI3?|pc;R*-g0NU^`b%1LBd~IRO{D)Rs#{2^| z8ktJ9g}XU3RsknmJbhpZP7#)&Ib2%nqo*6!Iwv2`Y~*FkckdY5c?IM3M`mWzny#tv z0St86==6oD=Yq*SCfY@*&$8bakR3$lGB=~8yflRGxz!8JXQhM2Y)zTOl$=au=`Id8 z2|#|U0lw1J;V$Lr_7+xsPi}Pmm`8;ccpsVM{pBZkp?>P_@$f?@T$+>BxQ7Jef_OZE zj_~i-U2|kpa-(tW<&-kba`1uC;8LIe1jyMBYD)mC))OW1V z*F`rMc)@^Za*{$%)Gd!M-YWFDE%Ax@$xx~bd&+oB;d1#^;nsOH!z@F9g11yU-CKVG)5j+LW=l>kw&3r^9rvVFeK2vw^e**K$(fa1`Kcu{Gaa#7=rS=$G4&*n& z`p$rI707cJ1L@TW6# zcl?ppxSDOxo)Xm`{wufOe($q%heca`uGLDTJ*i1;%~~X(DIzt159UoghX`;re?X<> zNv7@Z0s>s3kj?!XTwLe2m`b|8=i#(Ia7S`Q+1xo~I0OIK*3N!{-l;h<5hrb_zzA`H zJaWNQiX5nzNZ0vDHMIX*zLgdvimj2>YJvw?-uuEBb0 zunx`TUKcW^_N+%oB+=|Ib>VDx?kDC0l=E8M!i6qisd>axX_o(Dws-Frv^%*BFM&qo z6nj5}a;!d^#a{&jk<)->03RrWAB+7P&gYrwEEzqwU$CH%-b|8^Q6i8?P=o>upb~+Y0){_FkR;Q? z6Tu@Z8&t0PRxL{XXvu4pwD}|=)#WV;mqyy!HQuT}jv0#^Y9$6=&R5>|X2@g{pe-%m z=2_2IUDw;rbG+}I*YNVbPxHh;44Rr3{%jv$`hF8>0R54Lqg5ZzOaj1A+Rojci? zCf8&dJKl@APn5c+_J}(!L$<-v5xtLR8a+VumKP#=^cl|q$*(^R!Si9q<8J5RmiR-Q z=L_W&4fB$769&Wobw^qC-i3+D-C4KP1+ zwkCR(jYLTrK?Qfkp39jGf>Cko1YrwDg~%o^z4bri$!E};o)5ra^fGs_CTp8L#8CDe zrSYS23tul3L34g7_yZhcM2BL87?(MAb8;WWWYntB?~y%1om^!^C|tffRmjnxL1V}T zP`fLk(o=~lspQBar#G&i+f##cuYxt?e6HCYOZnK>@#B$4t5Rv;!mmZNyV5C&)Cwy{ z!H3YHExldy8qE)gpb4msLD6&Mlh++f6U%MR-Fe#&(Pz0m`lSl`vOVnsc~_2(-xBcL zlrStM(-bA^){v2fu`#lGv439KJ`U^-xNqGUW^Z?8bbiZ%zom7I5Z0k;bvXZ9*&Yp{ z^P{~~aV!zmA+0qlu2mdqf%g#5DT_Q;y~PV_7xYm_W9QlhvW#h;pzA&H4-*N|-Mm25 z9(ler%F?xIi^6krjMCk-!f0C`xvXp#Msp{ErLu#neT3*1bf51UP~OwKuDAmQuzCD~ z)XJDUfl6lS%*x`OffV+})VQ*RCnA}30Kcu9J3SG4>v&+JABpZLvM4d*frM|t>#7}dChsa{0a58ty zjJ9SM>7Hsbbt&p!=12)I{7mwcdWu@Eu9l`K{V@i8P30BW@XDc?5OFlZQjEl|JqZ+4 zLbPe=;%SjC>8lKuk=IYgo9wBlod@nh3l-)~`zi|LyB0DTcu+jHppe(~BkPTKCM_5y z@Dw64PgW*Ri=e#j<1rcfSvHz4Q^yKHd=Zq!q}Oxl{;1funEX~Ex_ti>;{KMatKM;9 zS3>US7$oqU;BZo99ii{f31TBGkCRLay4~4VjXg2;MImJ%$DqOr|x$wm0QAJ z)jd|USfzGRj&~j$VNiB5;_lbJ=>k8*H~MdkC(FrS>$upcf7FHB`KZO33Yt zN^r(^Q{F#igMBgo6|$ZrnUcfyz3PvIkBf*@KC>yI7H5en0sbm#ZowHpO{6llz5mWR z+!7DPG^{x?fFVqO1hXL#GsyQrJ z^`0QSiyoK}bpPExEuG*X&A`-7syv%Jac-AwA4&oelRNHc>Y8P-MTSQbA+|G>$ipfQ z^xIQP8;N!$1F%hAURd79;v5-R*#VF~hTwx6f1@a_0mh4<$X%{KWFS9K+{rstCxSktgY&uh4Og5z~Gs~_wHf$u93O$?~`@?Zr z=uQ7xKp|QpwR!UulYUN^%bH(61Km+{gAoOu3JkTrmx>DF)oUG68M*I8f>|8kwcHJY zn%1PFZCkmkqjjOlLbjUf}vtZL*Z>jRcHE_*nB_EWOdQQh|xsYs;U?;WI6KkC3Cz%`8Gue~YT%Tn?=>;jA7^~`2{?>mcl&@Ty_{p}j(d9nUBs^jra@m*!| z0K{8E{vohMhlZBnoxzb~4?N*qp=)V=dkd@LeU#4oQhLjsnn*R(&7w$xa-#w&zw$_L zRmU=_E8)`*BF*PM@);5g;_zKyciVc1Oq8A8WoxJ9cRAx)gU119TGYdTL>ZXK6EmU3 zC2zhZq4tnn{dK*1P6@_6+!AL{3!JkQfNjU^6fY-T@wut!nsqDRP9;&l6pd56{ldP# ztKA7OJ&6n}UY#fcqN%;5Zg{$=Ar0h@_264|k=$Q%)G(_cgGi~jw1*pOmWwHh`iUHx zK8=HSe<%ylibOulD{|{gNB(N4Z=SlcqeMb38N=^1$}h%(ZZ%0(4ZZEb25BuR0rchL z&Fta5%-a4mVN@p_F|ct~z90=&D(kX~RVHpgfMivDI`{C|-)l7;CM)bbb})n*SYq3yByMx{TSeISU-8f@l@yqF^qF^I{mPC#UpT<;ket8l&wfj)+1^ z%k}JT5Z@+2Q4iY0QmR3EYVtxG0G=k8gbZQ(9F;8*eE>E$rNGpB11= zX3&58XlPrD4GeVutV4G7NtHjL_m2K-TtiHX-yQR_v8wSL6$o|8D{H5g00(w-2yCqt zloi7k<&nojk^@fQe`U zQGUd5$vNS9dE?WNPcn_+*KEY%3RucEg80cnXBdyG2IcjGZ+cPo21{stxq~T{Uew?U zXD;QFf;vRkH#~$8H>MbOz}=O)1-bqgIfS>OZIPpn^u&H_b>Y9kq@T#K1tmrCls;00 z^NjN&VrdY0qB+pq!o3y_<)I3Vd_yd{`CVNP)gbo0IqcX`P&j6Xba48CDQq8qaf;Ju zNQ83>kmSudclNrH8%;ECmIn2UrCnbS1jC>Hc7GnLc~PLBV173aaMKN5cf$k-TxIb9 ziLKlRmgS*%N&-Gb7-R7O)>9dwlWmT0gVQ6APbY)7C=_n4+9#C7MMbdYz|4UR@(NF3 zy+yf&)Q7j+SG>g(a;q+hpAbR1Z71`Sl8z!Do?|%`nIATE6hY%(9ZuS(RQ(V@=|k-M zb0e2s9-$wh&zy4&SQuV9{WEMoY|q*)#Zwxw6Jf4$>n-~cP{vnPpvxmpcnB){p~zDg zfiWDF=^@!&9Z6udkG4J4tf82_bjiVJ{835uRUEN8e2#rZ5R8bYW(- zZ@WFu9ZY101lWt|u8Qc3d{(*bCF4URs0?(wVZVL$i?@OX(0?E;?Gsd5ED!UK^dXLp zq#{_fYxq=0cCp+-(JeS|)+&sYWpa>x)kJP#)1i*3I<}F?8FYlE?Hyt31*Tc8MGL~^ zhM2S5W}N;(&98u_5(?TGV6H5*h!kzGij-AkohC!iC!jQzg_&}5{Jpz99h<_Sevyeh zHiReDO?9;lK#f^ZLJk*@B5F%)PpK}pkfP#JC6!eMr#OQY_L?pRJa2}J@zO_80Iofu8dwuW}MWJ#VU(lO{OtB{v^?n zVl(C4LK6`Vhy=GE;wDMJETg33DI6dOFKu48fpgyj=wg51=L%x(K#*_Y2$OGxKE{jA zJfC;I$dif>;d#IaNu^^Dz84-^N#!yLD&M6m6CX&EXDk8Cn%9-Uy9lMa+Aq77e{8f^HeVoKN=zZH8-|z0qRmmi;eB88{ z9MMK`e;L!3>*X$O%rMB@SWYW|x$M4%N6Fd)h-$6OE}=ZKc{XZgPZHX|cx(fLRHpjK z@cLv`M%Ozlpksb`51dx`WJ_|Fk|)9Dt9>F1or=^%Av39(6q2n_KqrJHcWC63(MqDE zj{SLxYKh{!)RkJ|&c(>@kj3~H;0ty}3So}w2_o{0-*@{)p^ne1sEe~{Wb7WtXB!59 zwV()^-gG6y`)0xREbz|qB)iu?AxeYiYj1kaG6eB>47%WYr!)EzN##(`5LEpl zFOaf4lJ)aq$v$LhQ_U7d23^ zU%9Jxkf7SavUto=lU#gs0lr%lDi0!lkIhfJK_gdWg}MthRQX>!!^|5n3Bvyh&YPBmwCC?#~PfIQqsg6KHM^@%x#YlH~*`pU7sJRIs<>6@q z|KVJ>`EPakNc6^9Q%z&vtDdv8DK4n)bUir>bKUay`yFE6Oy3>BL^c-mXiHJj}Me>Jdr& z4A!Y?5pxH`(~+BU6eH2Er}bfREM_Z@hbPCK){+DbnRYUcxS9m?lK7tfVy{I#dxZD6pHrBoxC_$4n`KJ>v8f4lkdNBuG$z!dEyU=4wZxoX zg6PwnqVb22bxwCfCH>-k#-@_BaNcuQD8AVTD`8cgJ#UKcoO2~wloT~JhH5Sl z%SZS-gWeW+?x&?7|Fz&Z_s|iyI1?WVh~)`XP+PUm8J@TBrGb(O^m7=P{+s$hE5Q2^ z>FnwD00~W45nFa4T6D-*_rqm^vXw}<@040M^2 z(Pm|?;#$i}Q;%{|f<&^%l4C8%LqO=AoCnG)_Xzwonx;&dYMEMrzGRm@gvQSf3N$d# zl$zvuG}3_NcCleqxUjp8mqN=WrqLP$(D?DJ3(dDbv@vs+x7Q+GbDF8&O zIOq*4(n%ZQH0PUrNNiMdQT`r)`3%vfwpep8sSbYl0eu|loBh0#w}TPy2TKf+$v648 zVn#0RZ(9<}9`5?u#e-6?n7m2Z*owny>k!L#+)lvdQk*p@3NgHjhzd` z`DehRGmnmb!h(cRJW_Ny$G=$}ucuBsOZh(eI*F+?RK8IYQZz{XLq4biTS7T1t zcgwr`pa5*E-ceuMq3+Wsee*kAPTjlL{LRw_U*NKH#;4aHFY#hua6r2G{!+YEx|fJ2 zT#&JuZsGhu8-DdyLBW^t`^`K~)WZb9B3?f+zT+uP1c|yG?*muZtprLsa#Ma1uADXR zgfmCYCGZ?TXK{o5pA6$EOW;AzK}$zoh}rK0jcF_O{X1t`Ku|8Wg=Jpz`2kNz{*H*j zxE28s^}wZrV(GwH`IvOUqBpD>;QC>z4oE&CO~>c$gZ#UBK#e2K!s|wrQ0qpWl#PIq zHMHlMi`7j=^yKWQEF3IIx=O@8fKRnLe1E zaEaxn^0^K`Os#_5)Ma=!HTB^dtTuq-dfVOIv>Jx}->~Gz(Ys}A<1)GP;Ka@;6Mt-i z6hqLl%c*J%8c;PNbRi5~VyL$=lenw#Mw^4MkEbZ=mmb|Zw}FfHII?)C;4O?g0TdSe5Xmi24= zWM?>y&f_t}=7qHmv>!sb3v8eVV1esigH|2&(ypyeK({N{AWj1vY_jAW%-*|5?mC=-KH;U}Zy6Po!+^@9=MF1*6aCy?Ab zEolfQI-e#A0V*ES!~j7X20kINHfj*GFd?ZtWg+d?&4+IO()~=wS;zrTKel3e!8D73 zP{T1xk`S1eOAo9vb}~s?gA#H>dnPrYGuW}jr!_*(NsLJzrU}}Pl0AS7GpF#;0}1Dk zvEvsP?6P|k)Xy0B3D$LjDxI$EXLI|o?2zsraT-_VlwZ=FrfYJTv^x&=ou*8eR2Tt= z-1gk-k+kSGmpeVQw}YWl=mo*+sRO=0o0*+MHfELIqaXgpCX2$uk$=pn{o znr-Pc#M=-!0n@|5MXd=LNo^8vqw%G)Mni()611ov3cLSHz;5pP)g>GyAqX5mvYMMx_C_PM@N1^s;R6e>a zO3A_O4;Y?FT7wJ_o*Mdk*Tp<}OTVDHjKyw_x5AvX+m4&F;guUV^kP$6Qh+%60 z3p*m2>>&rL!znvZp1rOsgih`1R1=!f1&Qg%;aiaEM4k+|b&DXk>g|+DuHlgxx~B(k z86vwu&WlvJ4nZ`n*vOfpIAha|Pyze2V9!!fW^+_RdnNL=XWhQv$ZmqG zEAO~FaBm9uYTvT8C*`^yT+T;!?T&e1xa@A-U~a|zu93aY9r51t?EH?nvfuSpD$Jz7 zT$73PMo&%Bk%J(hD^|nBI?Xc#lPrfmyJ4Us<22oI`;**VRCNv<@H}}EyfMqLn3*MP@KB#r0 z!p02o12*oY+F$`fD9_dwBc)Qxez0M;&t#?m|5nTG_ZNv5usjKL{a)3M(t{{YDE*W~ zcYG#ua@`=>JKY8=jwE#MxNLwn@6elRTcib*xd3h9eu7DEe_P z%V`NePjpDn0Zx@EzUac)pKAlf7EPQGn|fB7hGL!^57}fVJGKLvc93O9`LL4r1jY(@ ztPWBmXg&BRdEx243~VpeDvf%fp=M>1E}C40dLYR$y%Szwd1*{l8(siHkgD#ERDxCQ z0#)#P@bGEi6(}U34_iT7-t2jm(t`7oP*k!4DK9q^t`!yhh9+_0T67L0XIxm3PElP; z9|d-QcTVfb<1VwwP%G5OMtt~<;_ZjVebY@J)sH{YH$iqRVJ5LPM0=IuC^!DE_#+@Y zrq_7Q^f+KksBAP(Mjil~RDp?NEb1yMl7e@kfiaYADRxjtEDb(bnh~Y$ikQUMXp(Sd zh4vpBVsKUW0RD}K zI~kO@y?-up6^EZ4_^Xgv8@v7eU?M^>BckRc7n>_BBHb+cj1hoj_Hxi}2B?LoQ|h@4 z(VG?~w7)E8k|($>dtpZmr30g=^CX2~>@N^g^CM`pQBiU6t_FokvSXv?pAJmVcQDF- zMUX}rXPI}J&{nku1m0U0mVtNn|Cm{Hipw!gZdG& z#PAY?f+|HpY*5GQWQv3P?zk&@~(Wrx@HS*wu1HR%B*d(uq!xTk*I>BJ%RF(3*Y zWRiI)vKv$VcqqeU_rGtjmrdATk)Mq36k%!&VLK(34{6%OVCeT`o}z5!NIGSdRv^=vG z;o*-p^Rq9*Xr;S`hRo&r4%~*W2su$6=_| zU|^TJ{}!PeiG~^BSm36nIQvojmp!`^fuDG!fBOseiw)$?v56-ZmF90*vKn>Wk#xcq z0;v-h*-im1QbjeI9ub$nJTp$ z&M`xKl8;yiBCWjGnJa;pcmf-@%4lqERI_A0+LVTTCYXukkeanF+aIlFD@F=XkK6Rl zp0PD?>lEa+VM(@Nmqlyx9O;nH?7TXgbc77a$0lC}yH{ z*Cq&|<=&OX5n`GGZi_6ZgP$(iB#n1Ms-ZAuc?8m3Kv-dzG&LxxcdmHr00^H#jHx>w9LpGMqEHvi)PyD-Y>y|O4vH{BuQz~C=t-58=c7Y{ z9}H|B6#2CnF3zb@MJX~5mbTDoE!H@ei5v_`TN{$SO;g^fu6+X685n4cOBVnF6R)3& zyfvh(lySOIv<^)D@lScJ4Z^LuuU|1m=*BRV5w}Em)BNyMGg+Y*OOFyjHHufTzciSY zEFT#}M|SSypX}nw&Ia$NCjwZ>lrje;9&l<(LG43;8Iflg^a@Irgpq)mqi%-wWk_zN%T;5=Ziqwg05ZG zo>SlpJA}tW^${jY@j&n8w!Qv}d%SE|+Zwp?{8V z8;@OK9cLeuFcGSC9Y{Uk@gk2U3VL0{F|cl6e$HS;XPx5fewLKIn3Fx0gwVgmCNF~< zZ56f^qOLp*Y#0Zyw+&iWci^mUw{@6lz-!46I8_#dQsji#|7~>s$*A}dj`e2j5W!dE z_b&O5z`xx{;OY{H>4{jA8X<8VpJbRH2KbIsSG;^~X&<^x=_6Xw5$hUBV$@v#`h8Gk zmcb7ga;SDs>W=k8`UBxNa!(k?P~&a4$F!;%rpYLGKnNCq6MRH5b$s%x7cCMonuek^ zeGLR>7p7B)#ukX7|3;bLI)Ks;Z##G=D!kLR?LoGUBg;X6eJ}Ll?&*M0aj5!rM6f95 zPqxs|O(}ku5{+FWHc2D?4B|0RLSU{o8v+#{hYAx zD#f1qs2V_gSx~Y=(taKxcn{F|53lWAdSNVgw~m5 z6BPq+iWa5Twuy4aV)O`}PTC&{rJQmvFgZ6}6&?F|<$ zFT80cDwaJuR`!D<2y~xzlq|3lxzskby(cKl?uh%-SUm16AySu!lp8 zEUww%dxiWB!NWu%lugur9!wnU(^ipbc+CzN_^?SN{w$8KCY+r{Tbp;c4g*f4Nl;sY zbB@DC_HF`oTN^<lS% zIXLRoj^az{#TajKO_pF$2niY?ipvMUk{pzXHXuSq#AfRg_4&~Oqu4jrMJ0nH7|FxI zs$xSEhcy($LGks~BW88>ozm53Sy`F<$EL0SJ^5$uP0XNJEw4B4UyJ=m!4u!arh7%Z zJwo862UZ*w7B8^`sBXskGQ`z}P;-2O27!NYki(x6d~O1N9fW=nwsOdgy}d<1%VHj$ zzXU-@7=b^|nz!VQFYBUr#-*?9jW6$_clqU~B?pv@=?$va&5UptM<|DYuAK)3`9`Aw zRC1rFrlislO2e4USpJISzox)hs!27tgF9y*OzaLk_wEPb2=tiJzv($&CyU zwR09RG`296axrx>G_o=MUkLYN)eXBJ#suGm?j~4L8?bP@h*XwN#J~}70#fk2dgPDm zU4b+&_cVm-pyji*2_Ye?sVMlLe=$KSQ&0^!RK&LH7hAomU7{pH+5TfRCwD<2>)K zG9<1oHhAxuTRNsZtOF9YP|OKsjAK~Yj1%*Kk-U=K+Eg*yC$AXwE4)+jvjRP&V$FIm zYu zV;6#D&@|B2EZw-J-EeCKtai%6tq$J8-6E^eCg(i+HxWFJKGIoe2$rf|LcDBPL zk|d`*=P=j%1fF0`;>uN0s@?whVX$AlHF*JYyM9o(S-rfyUQ~o6a>~in5zIl@-f3d-saeVn zf~H>}IS)nwyMSKC*c~ot+kg@&)6?6!llz>TxA$qToi%~4t_M_qjaxBP?K#rS-*b-< z$F6hPXmT{+?41DQLFi6wVVqB569<}L;B1U07Ge`7suFu@EEq$!(4j-y`lnb;9$N7m z_C3CA)WQdtZ4C#ZfMFyWOS87!RnLJVxQ&I-oVvCXIf}?!!ygHl`KZ_d`d>_n$6Wn& zQsJ^_?JODW3{Jkv9IxfbX*}fDjF&kJyACVt5$U5^b_IYDCxu^B!Y~`b!ZH&NRYfi* zPV&pnZK>v~dU{)UEbd#(0%z;GavI~~ceHSHLx0pk9QX!Wut=QQhgoUO6QQ@J9es0m z5|f%BJDCRHZw$lLpD$!RbDQYn^zVWd(+sh22VfJsb=O|>?qGBYonGTj!VZDPhcsY!!cr2!y0Q((eW7yq2jj+&XB+H{>u01Y|5(k?zR7}9F%Jy zzV}iZXSS&<&n2cplvSpUG;Sg0t4V1JF4roI=wkwmbkq6#Kz6y|+?OQb0@Uj0>xGKo z_hpO9#@64+uhp9Hxx|OND~?=O?mbCAv@-s&ek%8<$7d8p$@W2ss4QbOH!)A^uS^~^ zbkIFbE_TZ1;$WY=8A-HW8m~L>$@2g*_TbH~3JM?iMY0o9Ddm^!^8I5SYKya4tThWi zH*o=t&M+?MTn^}lomEDtzZLID*egAtbyRJJ*wHuUYK$NmDzJvqK}3c)8K@c64h^a6 z)X)sd)BOv3)%Nns3W6t8ZLCIW_aozT2G!74ZUZ6_?-C*}+VXde(erjy(DU}3J7O=o zd9X4GXRrpQxmAYi>;i&#^!*$k;9~A^3|;`PKfc;hvfV*~sRu7vP{>b&AgU0Z?l~ zz-u`tPz~gHtJ5KT%t#mKW)*@Tn*v8lZCtx*B(GJ-hv|tfJlHiiph}EwA^s<*hFgBR{ZKj}}=EZE~ z^sAp>pJL*Yr{RKfy=xWGptZih7($L8Zefxmh-jcyF#^r#1wx60J$Y>r8&LHafT7Ox z5xDeuw}TWt^C!db1w(0t@o@woA@6Y5mxkpgMDZtZKMcYSB!NL~AnDaf*;+J1yLW8j()}q` z?8FzF%fyR3Q3Pj9U{Sf+`4HegEA;;?!~bW2MuGZ~C-@Kf*f-~Fyyu6a>;{wO011zk zrr5=WpGHJP0s&}Nw^hMaL-Vuyy=u~^N!4ss;49Q5C5SP*o<@XDZAnUPFyD7~(BK`i zG_^H(94hr4v-2_kYr4$Vb(nBB&WYC@wV#9-T8$;?-RZi0#&f#;B>Qlt$L}4q7bf?d z@irn>YEypUd;#uywiv%eH;rMM}W9S=P}_-IhEv`G=CVl@y0O)MbkgDb)rtd+!?R-_p7JazZidBnSNoknib2D7@% zO@%olR+E`VhM^lP(ZFYdw8*^V&gE(+wVr+xs||qpvNKjg#Mtc7xaHatZpm@3f1zR{ z@uYzEYq@8aCl>rjh@vj|fPGG(^ZjBv( zhMu0DX^@-s)U^kHwezy#U@f1#p^OSUeIzgdI~iWLKRHwaf*?qiUcYYlI2C<74W|W8 z^$u{z2BePG+Y*Wo`u9azWJ(Yp7Va24VDzSuBYFF&T%6nazTy+kG@bD8_|jiJ+}~%9 zctT`M_K2fJ{jbO4woCogpokBc*M5bSUO?zhr*BZ~$rY&wWzomNuNs6XH4@T8&3`fD zYNL;Cx4-x5oPGhcC(sii`|l_9W)gSagbo12_D&T-LcY%01Q44}{rW-LnmKS9_aTyc z?!}_+=XuOlb%E|CVv?Ih`rlG-GJ-LhOPEs&O;K|UQWw~|`Tw%W>VGmJ+0@fLf>mv| zu=@gY?6H-s-N*#=b19~$k2m}D>szN42ezG{!IgAQVCCxLif}jc>!&gdNk!Igd}IIz z_?5G6=1XG|(Q$NuxX%|l3e!jIA;%E-KGPkHHk^JQ(%c!hL!7LwnB?IbnG;z%cIKoZ z#!VhcKAAJwtN$`|qR(ofbClaFKfaJW&7{pwD(AFQ(2Fq(zoCoJFzKps>}kzprcNob zABbH{xHwi4WvE0JrI;^A?gNPaixlvfHrv=k^3bVZM54M&9Ki6%%HK~iMd6ziUOZt# zBNpc#Z8=}Hn7Ec^Fe6e($=OQ6dESgU@Q~v0k$w-f$5S%UptB>^&-xr7Vk=}Q~;LqXlND-9B=n>VoLDN+@|CeFB zR0Hpk0I*|zs{$e5JV*A%MfeULq|f|uPE>>3b)--oqq#jbNGta>yyoA=8=OlTN z!^xJxz21Z~kO01+L^?zmkzin+IQ?@W35q23u7=+F=H^r_4z{f8jnN!wfz3anz+~~K zQ6XTj@_>elVpB`_*bR;_b_ru0@FNZz7#nE1OK8y!j2g=wv~x)V4%AXGt~ZjdHcXhn z_J3_CSC`RYC`6{4ws4Oy<83}%2oNG&5q4begYWC!pa8x6wm$_EBc_FGoIoFql)UkW zO}&F{W-?UEYp7i%Oh`R2E*|Avo$&J=hjl!g89+)S$AaIPnv7_mS@>JN36;?-YkYGx z_v?$ih$FJx6JA~g_>R=J`n_li? z^QzB_*PTw+-ru8Yo+a zRKG^yvwhV1#aU22UqXz{0hzq5m*PXjm{@dZ6K&9S!Bm3wrr9riIxf!K>qpJd&}^J^qfwB+675q!Ef5R3%E;-Gy_eixqQX zF7@JRv)nQ37%^H(^o_pH4Js6klGFqx^sJ#-n%7e@6|R_lv% zG+NtXM2A)!ELtxD%%5W6uUOE+=Qof(`Yv_1_bKah zp3**LaP{fJjQ@(fwU{U~>Y(85pZKGQE|!^2?1OVv_?L~l7c8%ELkX3Y>F2ICe@gt@ zGgnA=a(NJsNu!WQtX@3q6g(yu2+6#UpdgssM|X1J)LIa1(J!V3xG-?D)D=N4TdtMVepD)gW}NMZ;y5?R)gLFTbrvKuO`oD&GdtM6FH^y(fSLlb z&3XWb;Eq!<5#3!&R9upJtqFL?4+G0^#ijmwB< z!Bws{4HPb#vAsgHgq}>NFfzI!$ZAT4&!U(b%jnI#dX-bA349pvFpPYB?}&x-iw(^s zlvHRzNXxg<#P^m77gZziVMM`o3>5XzLcMShVH&46VS$G#9M4j8X)1mc(|{Z)up^~fF6s^!Sq1On67+YC{D8KR+!bmRmQs|V z%*{s}KBu5)F)sna+Q~_@rO28aw+;@x3UR35F5$YFEf>mmSeY|?8Qj!OjcD8UvFV`0 zlP?Na5B*%>t+=(wtpK1oH)%wMJrZP2Qg1SmF08f@6pJqe0=K70WpU)$)Em*==92WU z2?}urh1VKavh-|GQVy%OuaGflS(#u3dot^&6Wlo&&)Sdrm`WV$#Br#PBQL4*qa{P% z#Pi}qd=m7y-R)t+&@bqi!^Dqg%0BH?j46qeeT-{PZP9pd3l4ylU&eL=-pTEo%MoKu zFdXs{r2=B@yIMvO!dm6CDjH7PXyI73J_q|YtM4%j1F`|PldSM8~ROB|@(fE+tp zSrT$G6kBVvk8h~+$ZirEf5N2ci`ENDCXGAd7x9EF)6}?+B*@2}StwdQq3-#Ig?^|& zK%%D+YB7!^#|IE!kbgnfJ0m1(-2o(TN`JTF1+nIgEa)HROO!j9sB^lO*5DPvh`B#$ zv@z2*Jko9bqk4Twzn;x`*JixOD&#|`r<8$%`?iWhy;bWO^p5TfQuP2!q-g?Y6#w0u z#ke6O-;v>@XaRewo+;j{guEs1e2Q?tkaHXT_muYy<^y0%s3++@fO%bP`>3`ag5yrP z759zshvFNQ55WV=)eCCre&dm`OFV8Ck?9-yX}h#`!QOEnM{yIvRsj6+v(6C1E<1#K zTa;n5V(>SrM*#h)wr1?%=q(>X{fg-*^1ECKUi}D_(;&Y|66q7cU%YyOid}0qy|9F! znvsf`PESDfBT|rXl-QoK;%e-jmI*Ed4?>DtS(8dJF4#BeTl6v19rh8-9n;ho7#RG? zI=P-z>z=YOB(zzvTt_nVeDP|Wu3sMo#4RJ0N$Z56f%zY1hq^huk|6Yk1;cCBPq=rx z7L$^uL+MPg7x>dXq^4&uI!~-rpTy0_nYu$2)+|7-xD8=GvsdW&oL9DtM0q5{8OUk{ zaP?xb3!dYfoMjF<#6jxU2Hj_?)|-7HU&+@-QD_QlSBsXHZR6v*+4+*?{4txO}lI?V>DdtsENL#`~TMIr~5WxD`USgBn`^;Mxx#k~3T z7Zd=0yIL_NYISHG;#&HDu|9O zGRj^%Cl2szdz_)1mBZMrig4!80De?E@NABF#j_T}+0ov>_|3u%Hm)Ari>mpITu_r4 zSc0cb&I}*Jr$Ahe9o?&=q1XXWA^Y39p=*H4*>kPo6X>6BWT)4-J@DF~5Dxm5fLY}u z*(5Fes_g(Jahwg7g;n*w-BQ+!v>Q=w#+Qpe$Cl`8U`J_G3s+qasd`Y4V41YK{sC#0_F-$hPCJD4UzDQbd6vYV z;B)u+hpiBY*{kgm=Tw+potP<9s3)Z)sG$J~^;aX|h356-D@Gmm$+}SshvR)+Dy*oU zrWr5c4FRv#QpiItm*t!bcwPX?+!OTO z8WikBBIa_f)iV^%E7N{r^hOw2_T4zK)`Mf1auRjfX4M1`I+>zKO{VHpBW-}(5NABu zHQ|v*wIvM=(R=A3QmF+~Q*yH`C;z^|9%g2FDf=G-jn!IJ+T1VEm>t^YLfYJs(NssQ z9f{F1`b|H7GtZQ-^QNmk#Ln@W!~Wh|@8MC3s3+yEW6yNfP0;zUvg9cDFRdrcvLZH> zR{T=WscIPZxTK!;rQ^LB$4tO4TCUOzy3_Rf6fc#8cAWBHQGU~Kkp91kPKsm{QPjbd z%u^_KG@(Ni!H`Z~XSzy~6Txxn5g8tQ)52!yQXCO$E@|l^&ynq^62ew8`R}Gzhq_+1 zs`Qjie-%B~u(%{6-iw*k4Zr+Z#ih627Z!75T2u_6D;VCAc|E4hZU+Dx(H#H~km*oU z-E^X+&{`ZgY4p@3G=9OSe%!prRsvH(UUAsKQ~x?fW{!vus+=gvk@sc#@iE%ec>Qcr z@_XD-`iSw#?D(jnb~)*yf$Hldl;XBuxCzCI8C`)Xw+4yUcNlnNuvy+*-`=4XhQ^lG zs2aeKMv@x6Uf@Otz)pbk7TgJW{)m`G_d<8a1CYsdo35S_tK>suP{@7 zxslp3^~BH9%W8w<_Itq_YrKLCt&M1p7=Lp_n!wU&^-L+ZYEXCivySKGGXmU3Z{*RM za#C|m*f1x?sI91Vs{Vi|L8?@d<+TC*Zpf4#3Q=XiDc7#b=dK4}7kuj7o%*kp* z$&~5qO7jevDwpEMr@+i&+b5~52Th;yuaTz|Fgq*QNrYM_ZY4{*Hlguhxl6%At%c2X zIr?o{uNl``B2v11Qn|&%n^t14vV&wfKKyY_qBqVKuy|5)ErJtq;922GAxMfrCmyS! z(0EIf);GV!;tl{9-j)1#`m+b=eAi~(v|AGJ@y9rH- zqi)@9G71A&qH*$J#gdn!es>5DmPwFVWStiB8NqIm)YSLyM3rJ^jX@OWlFRi|@2O81 zw2~P00OPOCI^ic8CV_`}8;K5>?Q&Y|{S--vlrWWkVN1AwEHv!VijcTIWETsDj~U_% zJg~V3y#VP_M6Q^J2 znn91wAv$y~3*uLeQ|LKzel?QF%vn-bqw+ndFasK0r2mfxoGG7I^YDs+b~TX=LJ^}ehEJ2o`U#?-D{rYHIS!=A%PK_VisZU($65y@Fd)n zwJ>keGUV{T%a}uCyG>NPO;S4y>7$ z#ob#aZ_z4{j*->bVj19hO3;vFssNS7pcii-M7lyZ_RKEk_b->3{Q=hfd{TrW*P)7` zcJ2~yRXKj<7f1)xe-H*n1o$Y;t)@*Eyy1Qa`Aye5hI{VmFP?)5@!P>%bkaH_=>06} z;SxQ6kQlypP$paq%*|NhLQvQOey?W8lc$ zY!zoytrq+yb4`y^h;+|2bNg^P;yYpTl{X?DO1NbcvWppF9C7OZoGmM+laU8B^6F|B z`f2o=F^^tr2%X=Dnl{`XS^$DO2|lsE1~mBBUh#Rf3z96;l5RfhYkdL~w7i6$de?Lly&jCnugAoJ0k z;7M`O+wUD_+yu+s2`M<6!#lr3dw%K<_>`#>1N;bj({HhW&!3- zr2s>(8temt?zQZS4-vM)-=^3mF_79FJrrQ~)d*0)`S! z7}Cb_ca7H}@yFlfdV1S?y32i9_Ur59gc69s|3dPhBNjJ(NpX2DX2o^zHWcev`H{QU z7&wYM^|k~M+p0L{Am*{PSXZ@|8hF@rr1_-=e&Xz`PI}|8V}GKd(~RT(AQKZhRXNl$ z6IHF<*Mdv&fP1Z~#aoDm`-`nYS7;{t=r%;%H{~zKT7?|#8kbYEJ<|%x&7RSwek(6{ zxz=Db*jy=XEuL~%cS62Zfrpb)d>ZFbVB2Hk(&0@s)6lyJ%5R4GIQ7R zT98=lxE^@NM}ysxPeM-nw$nAFAE?^IQHj9J1$?WX(J}Y0)#Xf;w8hW?0Tfp76;oUA z2whK?Vo5PC=B9~zGn2YNO}j?~`gm_CL0YWou`gFyy$;2eau{T$o+FgyQK7E|+M!Cc z1sOEJguDtnQ4x18kALZeiJX}>znX)S{!aWrnbV9r?iCb(&?P@U7@HH}I%{Yz$H|g| zgWP$)H(35UDZm9;>vZpT0@+hJOCo$90f}dF!Lo~nkJ)yfxm-qSXe)X)7ZSOwVlCvo zB`dV#Of}lXBdvnMq_Z4&g`!;K8>OR3zlZ-!lTTO4sz|B*zR0Ze+aMH$~}D0R;H@m(_A_BTfESa zXMCnAEDy{ZOOKsiJs-gw=H9xBKGq)(*IrtpZ50K_@#jBqJ@D?=o#C-hT$ry-J`>Xb zo!A+@u(en6tk>pHoxDA{YtAQ35VpmU^aiDJ-KBhEhbBRQd)-U0~7Kh*T5<= z;v|y+x_egG=)qoy#V}+~Wx5J|DI-#cUMU!i8R~3=Aj*CNwx_F_QTrCL3d3=LWArXu zSHvg|n@G{(GpteT$lC#KIASEjAOUGTPt*-0>>l9|;vRQMfVL=_wJG-=RK$nZCf>w{mDU7&3kIyPuM6cPQbW8txXNCZ*CIrX zoGd&gpm6Y@R5~~w0`&n7(gO&WA%20Wj(WL#p5+gbw{-V~k11Ik2&bpF(g28Q`v$^I z=~S(SvqfQ=0;i2F|8YxW?LWM? z(J3qdy94d;R!RIXC{Ly7?$vP4ZnC@|W;J;y-65@TH_f57a5uvtI`IzDy`14!YP>zH zr`qUGTjAbcCYJ+moNmGa;?9~qNT$vJLaUdA{)5JL-Q`KylS{4zHoA?;N%QKa4D1b; zsrr33a6Z$G5|&&x81TkFvyD-Jg7ng0p(lWCf^Tfbwye3_NtK~_LrEwHI?31Xz1gA^ z+y=RoEj*eIbvFx!-K&4t@>HyoI+ZPZ82C;zn|a)#O*+oz3NfYgjp~mYBUFFn0^umX){_Vz?&Kc#^8v6%r>eCFen&8GE!|03Tk#PXc zmZMx%50WpSnfFvQ;e&R#ExSw;lhUXQ+LA`14m52_XG_w*CY-50kYU8`Q%3R-&lun# zpU*TKllHXPzJhJ<&jdHGkZoP=IVhzy9#bZxu0ghTHYYC$o-2|&>CE>5aR~)odsvR#>!|>%BC1SwnCrl_Z3se#-!#&+S2*|!qv`VF)9J06 zmejd6oT)n8mbx5vF}R+jw(VIk|6rpiSD$gV=Wkm=-O)A^hW7ZxzlcQyp=^4hElnD^ zGB8Ea(@a9TKtr}^UZD-G`^`yolC@m7;v*5K@I*rl`p9 zM^@2Jzi=X-^$1ifj;E5_bDi0fGNO$!EQ|@~Vm!N}Pa>6LRY!E2HQ7~6ao5hStc7D_ zayAbV0p#*Je(I!J3%^}gG?xI=(~i}%T09;kzi-<+4wj_rCov~%5Fd;moyl_G%`|NI zHC*P8v1M?|PW;rEsa{-pLpHeiLxW=#cH8>rq>~<&z4)!=GlydiA>NxFUGS*P$=dl8 z6?o*QkyD(xLmgyCMwmuvmhTvrU9wSS1o{+QKV;nmMojU0dVje_lZ*kJR+!?z@uM{7 z9i^^opkAxuXZ9YrYsRO0BV+Z63lBWkii4~6J7V>X<@Y{*cN9H;-ITZ8;f9=rsc5|( zMlSErz+H+}hVGnsAf};*T18A{d@x@%ouynOjO{cgUh^!zP6`;PQt_%6!o8-aMj04R zjoE_5>G6^;knCgfj~D^^T)MPIl;LIce!ZHD@-f7eaG5Q$!j!Z=%u+XvxHo-Fj=MfN z-~LYCs9S4-{kTeyEnVwPHGiAD=D6G*y(I8SNtzqsxVw!Ml~Y-Rz6ryY&>E(5;5C(& z8<3cx9~b+_8-nnVwuk&B51+60h6eNbQ857Yy{5mNI|ac?7>@xscE|nB>tEpgCUeJZ zUv{*WUEB$ge*8vvb$`){GnagJ1WySqXBAJ$+h6g^!=Kq-@Na#XLU68i{-+?H>>vFn z11R^PHT^G6`o#z8xqV^tcOKN1_yHu8erMEZn9=V_Rsh_Us9*f*#4(S|F^^*+9!x1O z^-?SbHda70aZMnAD*FYt;KtiQ)P9i1XL#X61Of6blPT0ufY~aP$#!?`z83?BfeZpl z?gz1UwMRdQvUQbDFCHVR04K4iDq*6D`+}*B3hf%9dswnbipkyd99FnAT}U`Dk1E0; zja%5nb8f11o0E@f05t&jJ38NbUymo}oJ0drzAz0>f@>dulI;+Lmp!x3+n!6Q5y^4F zIB_E;>-m)vBL2l7JKGcPp%S^t%m(kZ{Y&LM9dpIhit9qmH=|=3CGK+C5tD&@b;T`r zo$T1R>>Dw6fpM(oTUp{qZqzbeJ+g0w@d23tTxb&PALGQ#i~+-sRDvSn)nq)olyce& z?$c7fzL5){G`U_Kag0QXjtK{|%*a!MtqGDpq)R!;$|7Nfgwc_gD$v3RR!+yFrYeY* z=bQ+~!YUyw)Uk|YN13`uS?o&~$uRGPM1#Cm-Bq+2P&M18jXsy6eQ%p`zeaN}y*++@ zAE!Rvqo|fUW8?bzG#-`yX6hWL z@wJt>YDu=zK!l^xZ2N5$7YO=1|>S zZw#Td6M^jtQ~b2sN_a+`>Ngbd73s|)v}Bo!smm-pxm?c29?#Efar!{HhD!kTd-;V8fq}gdMih)jFrZQZ$JF_Wo!L<^qf6TV0|~ND}V$ro0Z{H zkr!iQ;xwSReR@>WT+fbYtKD` z);vcf{;T~Q$Ho2%gldl2VFy~^A1E^0ee~4v60Q=iHRGTj4(Np9xBKej!%jfgT(J%~ zlx@~noQb|;jwo18Y(@)~8^46axG)DTAOw}WS=z{=FPwuB*AVvNxWRS!s#9@+nXV5W;!joWj`?uHDh zk!_5D?-;F|SjipfU7^U~3Bz*c#zIf6E+O{xZ95mQZ|^%Pq~ zTrMgk^S6v&q8%kw4S)|yLnzHGgdz=R4amLuUEpnto*!;aQ%%lXDrNDA#Nwhc{ag0B z;Gv-;mDD~VbI$(TmbUh;Ia&a5ghS6F$Y#tOfL+mHDEGe%<^Ma}Kl=F}KgYEH|1ET~ zl{*(l00iMbH&S1bna-h~USA+-h^aUjX}ttLXEu6%9vDE3vYz##AR1p|OcF)OmTmLX zK*dr^hul#3FMcY17a>heZAHtz*0rrHcJdra3ffDBThPZ^sFpy)uOIVY4|8@ZDlyO` zuII@d=j)t|8-Z>=&|4r!C_)WT$*Rz{&voBAqVJzc|vYnOm|~ z?LO8gV^|jw%d)QL{iBtR(zEP08v0L$%J60vCOZHOAJmRFnqBJ1X7y->I011nP+4vG z#Dhan?z2zz2~PJ(4@EmQMGL6T4 z#Y_|Z9-j9VBfuw&>ou2acie=MfhkaAS&LD7P`Ua$-h2Y=&YX4HVQ;t9#-eosF+G(q zN0(Kq>Ig^E?Hir0P1o-SdoZE-`d>Ga{~6`~Hx^nA8*ua?{~HPv`f|ECKS-c1pfn)^ z8l3+CfWXt}I&lWT{u3yzFw2~A5dZ6UZi%TC0{K6!d%?EZZL(&bQW zFv;hdxx*{1XzEGakpH&5BlJYBqstK@@^z>Wp^gbI_8AlpI*)G_B z>o)qm-kVxOhny|%m6Oi6gbC*wtm58=rFw~6o8^_J2xdVok7^1lX?a2bAFm5mhQ;gL z{PR|Kb!GJqybuLI(axR2dujKxg(mMebjTXrpL>pUKWqxsPO^(0dVOx!tz1>Qx_9Vw z?p;-B;^!AtF%*^=;$Pd|VXvh@Zsp|UE|(FIo650enCWcx{mDA9lw;#Mn=8iS!^B1A#_N^K568cwaQ+DNuEatkXZMcVwm zwT|-`EDX0I&8|4=4PI$y3r~mW)y7guSM4ubFp|n<3Nc;be;_6&2eNh9P_m#c5$9vb zWjqf{Bx`<`hrS%Rs4`gPDq<76c^N?;5IgO^bAZ_BG(cL=(f@mA(aWA{(m0gmW);*5 zRU948hT(Vgvq*4tR!TBCGkf+?ZC5uERO87kT( zqU|qb+t7kGzN3z_bJxhsZ@l=~vIPX;pgjP8D_{b!xbEg&H|MOCHnOv?g%p+_BG9-} zk@uQ`3~Nn=_xRR|G%ZP)CS?ONp`d%rV%whBwr$(#*qqqx*yhBxZQI72JkNW+dcX7kr>eVa-`#gtS9Ps*?Y-8z*0o}h z4-;7zva{tY!xI*GvVGe$D`})k7f+za&TienG&^lZf%z=8UhkE<&0E#^**>Mk2}+0w z!AcrCt8{?nwH?kW0M6}gkYyMD57%;M@=L5xem=34SPiu!vp0>VHXj}QBsQi&pViTE zc_SeZ^{kjJ1}hP1Bd?#Bsvy|Rf$JrEv}h(=&Ft=)ojQ2ZE*_L^gpJhoj>6REQ5=7U>}MKWBUmgi2b^xFuLfZ+oy2hFN}Q``bp+NB#g z&1nB=PGbYJPzd1OV<3HI2tsC=nY=ZTT$Mr|wS2Fp3+j ziq~B}U96CEw9%F4sN9+~T#syX&`RW1z@t-ZyGu;> ziX@Q~w#4?Z@F2zlhg*;18kC1_4)L~iQeZ3J?##J55Mjm#fk)zr`l*I^+OrAk5?KSo5X*tCtpcJ#lnr?eJC>sMW^Rg0IZ z=i6&u-4(pYUofSa#KuepEYr51HsB(H1*}uz08*_spLJDTUc9Y9bT>^H3Q6Y|#$)3EmisC;$n?Gtn9fCxW& zNl?1LS&t-He%EQKT}Ztix-4DSY3h5b6ysiC21hRWl-A-5)+WAFFZ1rQnwVV4SU}G= zC2Pgqo2G^W>VDDt#I_G$I5Mh^?p_%WoteuW>3MBXrhjAe4lZy33q z#*xXv@0~gzRfL5!7o0S{??-b*fPHwWFe^Py#pZS)D_%U?Nkm0td&RG&m?RF_T9&n(iIXh1b0IMDTa2X07XI?sAkGvXsYGnn&JMQ=r^XjxNxX`MOr zJV0LCs{}6*w`WOWwRQnKAi+(pCdQ&Idcxad8p@Zj6oOeG?ZiI0B~8>NLK&jrwsh*k zXoO95PQ5TO_C!dV6%8vzp_JV?*^}+EBQQ4gfO5(>epn#`?9lH!dQNj64tgYlI?wCf zJzVIjGg0LH4%0=o1%vujPcZFJ9cX1XRQ4Ryr&8Xb8BeTfgOEcA0G73>O2$D9kBpdw z2_dL&Uh2Lu1-MBh#w!+NGb5yFob1pMF&=>*aef&T#>rFKVT`UZ5UJWM!D2IdF5bxJ zM$wSVGeFV%TWLfYi*Z;+Gpt^woV;J7oHcn3=S6uFO;VvFSlSCYU5q-8M|5aL5UjiuV?yBQBR+bQeJq+CSV~g+T2pa#5R_QRkno`Vf|_&)K=?9H7*#j~i>T0z`ANJq zK{l07eOX2bYOk-1(3)0D#x%Mhx@;a!PHxNV@lmm?U zG7ui|CnDl40DDV`I$1Obi(1kyXZ82}<#u*a1A|cg0R`)2REa=Al+C5X21%&P4qJUg^PhIGh!IJXJnhb=zz~^DcuD zPu&F?SH6&&{Eg+^-@4M`WQ#b^s>40(4!<-AA>F9Yfo_P*D}^KkEF{&SrL8X=Tj5?S zm%JQG0R6NEfr=k%-r1U;HmZAgs;@dx-2}*NaZ@VWV1g<{7#;Be2re+6lLk9(q$J5PY^NZ2v|xq#`c_{zg79Z zK-01E-}w^;E5eNwg>OLT#}#L_~2}bKDv)M>L)%jKfo(aOCe9*oGs$U=}!! z^ZFwZ^g0F*hkSbt`+kgR+9Dhm_x(6v-j9vxH#}OzZ%(vO#w5H>#JM)=pwv)?wT71d z1pp$I4k`0t2JD38dI_d_74mn0XY)IQ1)vWBMdDc3!TF&9(L`thx;`6i zueWA^h;p=va$u76-^K=1+6G+RPxRa*n0Aeh;P8s78r?q0C?UkBaoqRfyp&oLNVoxc zI7-dmy%kWNwU3OqF85h}pWz%^L#e}d1dh^3nK9TLtl0hS8{HPBmYGeAvbZcyM1XF* zU=$R4Q%?|Cf?wU?!KyKiZ}C&8Q8KbWccqGD`$Zv~cmrF%n>h1QP~_S-<-ElBHl zW?Z8(QvjB>Al^|yR^&2qYR?FjfsOv^XajWe_H?D^ zdoCy9i&B|^SV;<}6;1_<%+&2NYJmb$yUKp!x%Y&?EEdlWZ!$Ix!fc#mbliY;+_cBb zKr#kgihRd2wA@I^)lG}18-To$KaJi-7EYXXck;+|ykeTKk8m?dm}N-6t}?{h%~4dM z(N^!@r$6n>4}p<*j*RrIF$M^UJo)J}h79;2)EkPa*M`Om#pcF&$_x9{dS?{Kuav5D zPl|^q+VcbFBTE&9-`8%;O$#@8LQM8@nRoZ*6rZ2fMNlW{6uxfaY+q}bUXh>aT2?lh zWKXfzEgfc`O0^p)r?jMPu-7-vU$h<9T8%V`#kdKbfOSh<$pU1X0}vCW=^Spip=D$8 zlLaHggRk|;qC9P@n98SL&LY0wYm5nl@eZ(gU<2=I$mX2vXJAtW=#9`j1JLfNhQ^od zzki@kyl=|db&GZVBL6@(ctxu!(sM$1g}2S!+I98alKbk@R;&3?!+aU;1)rrTXimDo z3ol(`gy!GRw%^WG08ow#c5nZ}RB9b=n}C95HlSx)sU%;yM<4mBy0FAB&q>5IrhD&X z{-v~j4?ZT&EVVXD#B=_g(o~zv4!WM zrBcEb>G9#+37NC@b812bT_j+M1iJ;IB|B#h`IwjOgnzk^phkjUkTXla{u6RRdJXL! zIt4Yy5FWH9Ue1p;9qS?;oNpq+25z4<9k1Q}+?E!S-J-QcWvq|Sh6KUFT9}pq(F*)I zj}4oD*RI;q2$0yi_Ds?jV_v#bEg(B(!>bA!z8@vSYAnAPhWc z$#nvY^7N>D1*bxpihdm%{K2X-9kgk#W=1&LDh-QyeHx5YDvVPa3~VY4Y#K~tDokV= z%);`3k(?m$RxH>Ei3>VW(DEIW>QF=uY(Ym}&AUA!{mPLDgnl zYa^01mOH1r)oaHdf2g&JuOcbIso)e&x_vd6!3u*=%!ydfYEXS5!5)JFi~joNN7)mR zuQdq*AKlSD6`C&7a_YpE@zNiED&dtM3MOeK$bi!5rHg|E2>vB0r)(A+HVhj&)4iN( z$^B@iD7lvmHq>;^?`$^(iWhk$CGJiHbS1bovDk1&KF+s%bbCHQamB$3xjHe(HaKLK zYiX2fxr%Q*rsisf&`(y4^oGDk(#&$5z`I5(huS9Qyb#^tq`NVy{5Qk)r-2BB90Q>f zXaErs&h4kueXK&>UJ{D5n*X6n9~TCcb{uR11q%k%S!(77-O6P*lhhuOcze8?gHr(7 ze(ydjaOfC}+CDF#K-F_Y{Y3MV#4?tkMR6QTMB80QIW6G3+fDkL#{_89`bN}xPtd5T zxAjGj8u@Ib(a(zM6}y-{^4dspsq@{DbAWT&tG$C_a2X!9{f#poDB?n-H8U*#L4&1Y zZxa@Wq(Vdf=73qC*A7R>1Ws0`y<55LzWiWy=*)Uxa$z~hg@Jm5*!DDsriB>FqRU`e zfy8r~WQL#M2%Zgl#ow+B}Z(_cW9ZjsqUsz~ZB zBCB$SS8_E)v~u29nhS7aEBf0*!1&D_sq~&%(xYUBKFyIs7gPR{_B(c$|9ZZ(^0x3# z${@}$XXIAJuJ`16)Mn&jzJ9EzVZapxv0*sT?$>e(E1n%QUJvf8bN0GR=%=AX^sqH3 zk{p+??^xJ-43bAAl1B!T#}`-S>y@3B$j(G~-zIo?O+i@rAulRT^np^S^ixgs!>;1h ztK#*o;+0?h;Ys5G@l$Ki+pLDMQo*PRp6zhRdR%TfH3z|9+$}`7KOGAY-^3D@N;Ax) z5pPpRxRn^}n1DC;tYPB#-u&Z*cav3F&thHv2(GzQX>}RjdIxm%m$)>s0c&(XUt-QT z2?Qa>U%~NEB-r$yUhh3|Z#voJOVoG2e`486MUTvLqO19fpvwhHFd8ZLrbY?*m!itf zxl11%;m-AAlF0^-d=DN0@C@$9N-m(L!eK~%7Bajj;U@6!(T~tDifPR9g^`kx(~x0= zq*47=ucHnp1q}YhG9+M7S4;wvkmNDN#orR0Er_^zrsfwb>Lu2?e$MFKvvc72jCoQd4~FKwMi?wezbP>Q@K`2XyYoZGcI^=q zr=S*1G`5Ip8DFr3RTvBN4TZ7PEL9j*YfR|YM|2*plT>@~AF}=(KV~JM5atUbzNLTX zj$X=K2$X{Gns_;9#aXdpBlVaFmC(JT_jXR#W%kJ)a0&V6F9^?Iv@n&{T(ROgXSF$J z#eYnHB&mL3IAjHoGkHz`P|5y=8{ImbZ<-a&+bhcA_5l?KpMRGTEB)N5qLs4I{iNJ3 z<}jWYo{IuV)TFnzT#09AxxbmQ$`%D$rJX%zB7KBLr}8LKd2lNZx9ffyWiY<66$qBQ zz&ntrVDx`pGJ3_IIX)bLzfODL`o9{rNZT^$bV8BoxTFDG^Ol`?C1v!Cnm6E6PH?Q_ za8NiDBgvI9<8|Ln*I<})DjYrfC0gNG8Dh0PG%>HHpNzO0A~EM^88U38mVv=x!X44l zjNkQCSxhmeV4rd$UrhP>>tpoK477cP=?-sHHhT)(osSLP0h^ zQJ&xVN~dJ2*Hhh8=>J^1YeS3t;&VeVv!m>`L^Z-A(wu zlc&%l5z_+X?8-S%2cAAq?P#ictX z@|dLamQ~C}VZSUAM^Jz5E}643+eS0BF@7`<{})~CIQOoxxb4Ij<&gjP<9+s#!12*g zYfVWx;E+plQ=5RG{-^ZJ&GCyydI(_=36Nq0dSuQ+UWoodPiIzV?WrMG;^DR}o^24x zQDqfiqM>FwE0_n2a^L|ovzAv zubV1$X+tXf;em`|HYf=nZ?*be)>$*P$I^Uh8c<2`z#FJUj?6Xi`ond=%CtHxr~SJm z=DJSzxf7j=*Gs|K;+Rg!{r%UEO=sp9isdS+TC3V4iX2_6q3pb|Et!H#R({6(L zr8>N}AhtVaLfor^er=KqsB+w3P-iAv{LU^BNs5&;0&Z2;nQtgw6fBkPiZO~W#2VNa zpK9$>){jz_)&;PUy%z`r@|j$-c`Bo_L;YgOm`@ZQaCaHO>&l-a(KZ)P+#;)H$D#l_ z1A1i_o6RQI=%VE#S`-4JcJW|pQzU|;_l_Nht*x1m!GF@REJ)<20}lv?1Bb@le(;96 zLVeH27ColDeK{JEVm3%}KFXPY`P`GN9xSS?r*T5W;vc;B)o~gebWlK;S;vx8^_C|t z%!J{bDJQ>ho?v6O`-Td1$XAoq9B2dFphH(*2-Ek=Hwy&`Y=Re_>tnAjf*W-`LQ$VY zxqTb)!W?=20T{6;qL;`qq8Is)`|Oh2G~)Gk069IKHNBxQ_Po^PIhgX%BKv93knR_R zYz%@d5$9F+@))mUmel0I&$hp$SP(6pFh58`0LiyWe!FFBk`S}h+ z*#pRD&(3~MoyFi^&1?=h+F=7&sUASZ$TztiP9a}7tyw1gU{k0c`Rgl>Hh^`qzkj-Q z{z{@ug+zM{{3+oOBeYAPonG{LvrV*%lsUry#iJJOpJ{eIyK>4fqCJax~dPxoyB@AoEWdQ+q!sIhNjHOqa z#h`ELSbD!k1oxN|ZYNb-ZE>wz7F@G;G)ovm%zCm4D?VMAi_yILphwSinKf*UZO3pT zSoO0ChGAWl@;BnlOYBkQPMFsQk!Cyv!Ek;Elw(^xGO%%DI?3!Ee2g`1L>_E{n;(I+ z6%U!4$N?%R*(3joF}-p|;k>=s=_B2*&z&6=hPplFOmt7x;|LJY z;b2A#Ex$LuGwlbO;|e>V44*HZnue&t*t+tJcbc?m0qk*@ZWfyx@5I z7~w*^+2eQy^jp?#AVWU;J64Tb-0OaWX{5xw-RI3k0cUE)bGNpX;ywMtSa|eHIz+px zA@#c|&V6zerubIZxR~!bv%?~w3(ezeVAwnEByG1$%_npqkM_w8fEoyD>^E+|BjD%OU>smjObm;VSZ~cH;eAX!xDGGax@Qz$ z=3kG+`M9*(#tQeKZ`Dop%EkgvPmP#|j2Q74kP3{|5Ljukq zkpOy@q|L{|7@7Tpmldb=lGnx#^POJ6H@^O;{t}P)eL=iq&9$Q1qDuu)t+CO5CzPQS zI#Vsp_iVI__=;fyX#l8;^uu$QHN`oEU51|sY))Z+;Cw~-+H(*Z z;qc|~@^f~6fWpz|+NCo?QsDOHCav?en(f&PO^ext{~Sa0lejNtjKNwTs7wt>vrd|o z*V}VvFEUlSMiHir<#vx1H@o4RcntgqM1oIfL7yuc#cC7VAESI(Gfq+2NmrT!7#Lk+ zZ7%cw&ZQshTt)wred!goAQx4ns6cy~q@oM{arRfAY*qgjRXN|#k&C$n{|Vhx=}l|C zOX9CV`a=A>ZEi$H0_fj=>C^vp@&5}4mW<%5z`_2(fqDaeRgixXp=M%*_4tdV0|A<% zkimqXBGZBcnPM%40|WSvz+*;6!ZuqGD%8PjgNuY961sR8?>6Tu z^(Hq&YtYxA-eCOQ%s&Z(cF zF2Z%#?tJ+*e7lLkMlVEP%vo2G1q0SKd)2%x$?Sb6_l^RfMEIO6-0?&BV#t$B2#I+= z#tmy!4uq6Ajn8Dbc~uN9I0gY+sFI$TG4l_8PMB*ZFIWavi%r{ltz2-CI|*^a4gnFa}U};%7nm|c2=m`>6VK%dPa_4`apQGM+mPg z@JI;vZqpFO4i*!MV=YdF_f9{j4CNOCvDB0j^xGHsE_06lrlxk6a|_ga5&>4C%;jhy1sUptU2~NQ}uDVawAx{`up|t@p{OZh4k%-*IZX7spv=KtAhzbH z@My+jwq-%S6E<6o-q{RA&MWUNAB|Qr!4Yt{f}Sk)%8Q$%1f(l7_Yj|$V{TZesL+q1 zW65&s>PzL#qqL76l+g=vM|gX2-X5J5LFhdc7FRK^PqB*tB(CJWMK761q^`W^#qbT2W1`A z5b3OApu4FW~C5&g$@!@`W%dGGmOvNz)TIYex zVh`7gA-?jD>B$P`kDs^us{!&0`V0M20CY*rnz$YX*bNTDJjhP;R|b8aXgf_R>?R*}DQd)^a#y3F4;pGD zN3@`v;Bw`@ZG}!TP#Gcy+&tNFD7^{?QdR1Q?^o~MeweErog6+dNJnSS^KkW}fC7;l z^#PlYtW_!_;`&JbIFP(h1jdB>iuDcnQ%W}-B$1V|ua>Fi^saz-pCG9pl-}*iOYp*! zaS;^AY3n<2%KQ8s^5-t~{Z>{gt67uWTNIx=(E4$oQaZUz#4{U}Lx?|RP%y31C0^qj zrv&G8f+f{x6h1mr7{=9qvTVQ)3B;cb#ve|O-= z?Axr;+VyMgPNU+o;1~iNgqbsBC7j%h7;M50-YWW_>-6OH`Cr5!$qwODEJp;|6)SbH zQ}Z4>FG#$|=!54q8IoZq1d&1`9Wsr6=L!_YH3{{8SCo%ICpawnIA&b~z`A>Mer+(c z5zd_PRJ;zEPsV*+Es7bN_|qHZl0YWeCyMeIGv;hs!R#;_&XLu7WMcRZz`o9!$Ze`0 z@xqyzK+Osad{*N1B6bJ0Qqj4KxeA5vU*P_)pZ$*u{-dY$j|WyK{jf`f`1UOm;oCQg z1P2hzWLavklyYt^L_n6R#=l4YY{-rb&KadU>6zIIwcZ-?NZiUi z6`^TYsx?9Y&yVaTX7e?qERUF_C8gYx?#{~0p$TnhjTgqwDXz+EKbjNX@Q5v%XO6=z z`(=#GMqOVU7_%Z~?57N1@K8ppBzZRL)#}}kkCQhR_p?Qr=VKo+n}l!(wVGp2lc!VU z^-4A{B9CGyJNz={4kd?=JQCz>J!{a>-c+oIn8uL+_=|^-hCIAzQRe=tZ<)tjtHSG& z7QzYXoXFAbpW3zH4q{TcCkwMx#Nbm@?lIuR2D^wnTWP44E6 zUh!5|=NF}7E=U*%!JK7KIM zxY0*|_2}4q0mRG-f_u$3W;F9 zLAR@mXuhe7aJ!}hWfmLKuQH2Hk3iul-R13|HyF5w509S>OFnN9Wio0zvF1*mo&y9B zQvbQeb}@-6)apI(8sB#hj6l533y4+rj!HmQWyLJIFgF!!#$l1Nt6@HI9(BCmNe;=2 znzN@!Vhm+(mCPx5R{o_CIJGPW+gb9At5>p22X)n-Lgi^M*`p@--VkJEDwP^hEi(P5 z2khkIPATEYeQMgEC~1kQE1n@|fE1vV>6eBP!qyMUR_~z_EXAV`VPJ`|P> z09XC{7U($9Dvg6rwrO7r-cukbRqT=gQn_PzVCdy55-w))TEbnrQdspPCKOQEm{`wg z4W5ka~-OiwJtQ z^X!y*<*@CSf%`@LmtbpL$ z{NEHAVcx`yO(;I(jB*YD+g*ZJx?A+jcJE2U0|)eUtve1cGaM%~9Ph~_C~eVbu}9>y z@x7L4+-By@;H2b%MilU*#bh-L{J5VPF5_yKxefMuNE+!PAI@1J>l+jEE4!N4zf|{2 zl=U%O=S1UpyOt0iH!~{_X_BF}Oqi{0fR#4&%cp(#t8JvA%ID{RgV6#noX+)_ma79d zO*G04ywD@^n_U0UeHgiT9iZ81NcpoXMhu0JH<-gKzgkw>{`2Kju)=GEm&eL*UZfP8-3 zP)WaZJnhv$G4I3q7Y_>&4htV0(KYnIUYJhN)<-I;5#Hlr`*pN8E~+kHaQ;b6{O|qi zzbV*%cD7~;U7UjN{{|v$a4b8KU&qW&$P~g4e2j#eY_ODW5pMA2X&an=(0`+r<_ke; zs{ePXDMjt`p9to{XE0OvD^F_%lR`-T<3C?)HfF*GWU2f|Y8IU>$htXL|GQBShz%n& z{fwp+Y$Ccj<@}H7ZQ(fQI2bUn6X$^wSV?ZtHr-sOYEMU!qd#1gNZNRNOU= zE&vcxId`|g{#0yp%d^uON|4y1W3#lJf~D-hkb{nfV6|JHcIAf)(j|r6UlW$c=?O8) zci6UvQwwy~R?{i#DS|?B8eI9a-ksDPGFD4P34iLe>d$S)5;|fLTo&T~goGQ<#Rk)! zCjXZGpiOB5b{R|UP(%>xEzjsL?)wV4F93>^11cU4U=P$5n^j9o3w5ZZas<-KD!7v^ zJ^Q(Q1r6xd*OHa!Rg%P_>hHi#AA+x>mNe=t;T4j$BK>aCLa1j3io9vp$vBy;FRs?= zL?_!OLyoAL4_DC?+>#m+&n#M`62lqkGJ}n}@NPCxie*>kuea5qT{FDAj16z1kpYnt zo#1Ol=S)OV=YEa7x5{ILnbtrW)j0;&KUh>9Vd_q1>gz3jV&qEgp2-OtZ-VaBRn;vf zvcC%6;Fk)#Y*YdTUNoTZN)28>F3&R+p)6~M5fYw9Z|93-HVPF2fjFUX#j)c^&o6(a zMuTy=tvk%IS=NWiN$Ovdk$d3qcL9BD*rPe=zS-MS6*$HBQMLT{5dxwMHlD1tG#+J6 z5`o+knPd<(S%WELe|nwZm!>=vq_su5!VNm?cDY2Q*Slp@rvF2G(&F`?y_ zvbKuFYC~rVzv_}aeco(vf}~j?QB{9qp`@70oY)C)>7-desE?EQgy{m3L7e<*wkkl@ zdAhm!nY&?ESZ|2n_#NlfPW(f#7;kvSGuYV=sW+M^lCeXq-Tz0h6($Hsc%_xIoJhF! zR#CO+*5z)sc;obgk##LK2Y>*PCM{3am~!laq!P^g^@#0V7{r%l8jF~F9=VnPZ|K7L z@4V?zpD@2|R>)Mt9=}At5;56Pk9n*S8Il|??oP8r2KDsVb2vI(B^tRF19|Lu5SnKg z9DaPqk7XQaMu%Weg~F(rSWt`8cS=E~A?N-bMHi*}3q%gGP^}i80sw{+*UkRDYrcu& zBW02|>CZM45TpFhurMGpX)|#gUp|X7G960X**qOf5>l%O`+DJupLaCEWxdgX-nAQp;6IfvL9X*d4-H-FsA{40G_D}Ip`?gXmC5)R&0 zW$qI)pXT9i*I!%&f&io0o>XUs7*~dfWKvgQa@e%)c+^(WDLF?Y<29VUfsw&!7vClG z>bpSn6^);$ccu~Zcs|N?1uLupkq|uxwXGBVm!Y<`II$q(z5nW6{}*Wf&jWhNYy>mR z*We8FFBqcuAHmyZ3u8zzkpBQzyfrZt=)W-c7#@Oxi}>xE5KT&60yR#`wh$*qN?afl zI>23f!R0fyu_J9}?$7cRCP~M3e0Oq(p8yltZ<-b4I%}E{QFMrGQMuXWf;5A}Xbeg7 zHj;@yELwS##+0mtvH|ePgJJ5j6l6bUS)J|LifvY%w{ki5yW|RP#M5UWMp6*p=6sGe z+$KBCx{kC@fCPL_-@8qr+iDT@p6s7PumRa~*BSVzJ07e)67cjTCw6TLei#d9kuaVT z0`#SbTeH`7q0C#0*IM}RY`#N*TLV`X&psG`_5>((9d^PX0e<|-!+^pNe$~Dl_ZlQX zdMJNij@5G&q*^)nXK!Fvzh3E1HE&7(7R9k0+T)hRb2jAJ^%M4|Kp%hGwGU#)Gl2Zz zB!~CeKmX?OIQ-`J3Dg_MB9O89U|{hCRXE8LM@wT7$njm;?{hN-P>H1^+M!WsauvAUh@?1QxOPoFN>wn@^hD1>=~3{(%^I}x2mnM|z~@F@2_B1B1#i&>P7@JGCoN_J(ZqLV?8f|n z1zK`zY7TvPXE8$suUIUW(2C*j)Vl;cE7lMoA4OLs`UlzsckLC!j8&L0tBcTDaX3)$ z@TFOn-0yj-xh8AmRz^+y@@M3NB<2v7XaW2j%4^A~$M#UzU}LMUj>lHWsDQZE5`I+a zs_@lbVn4%t=Gsms+2)A!xHvYy1^Rl0 zhMioTaTww=TNNbJqu)3|=b+VEU3Ju6CYyqM3UtNx!y`;WH-^_cmpy9AE zso6h@ybp26prh!4kTouwhyhB*)4aGGrPmmP3!_2OKfc?>siIAMGUx!Q51T7h0e&r5 z*VuhKeujx$TNuzQgnAUeHV|J!u~*laD{!y11lk!*5#=axq`$?=_^D?H{AikXpVsWh7!D}K>AJ?Y~F#@;-x>?*>cgcZG zb#b%aP82_Cyd{Zx+Q3)LBRt!hz^%}?!JKV6ZFgiAQBo>%aCSpkY z)As%}w?FGk%J2LpRsoQYdgQ6ovQB3A z8m~cFuBc>zvrx<%agBp)-69=uSCEA7AC*v{ z8GB9V#3gMogaE7=10!)uUPau0r4(Oh^Z2O*&t4%ieu@>X29;qUQyvsrDw4tiSg@T= zfhx>#?QL9&OR6HL9|1Ti``E@4W;y!7`Z6B=`X3-0!Ie_|5rZJ(wLr?wA4Zp{fL{`= z7J0h-BD39m+6j{IU+iF2n>yuMB~zuUk;Z`NJJGGE7J!yefJRwH@Bw31VMkL{RSZq> zWo`H1Y-83z-dr!?Sov&FF*R!BvvO6wlrnukG&>cW3vPc_TSYP}*JY0v*b)h+1e3v= z`p`?Ph9t7#h$f!gc2ErcPD6~|fSL3C4jo>oLo?mkK37jwGX(>x6b6`9^0sk$#nJ%v zLi+r%BcRjJ(q+`4JK1~~jzu)Yka)aHVhCM%66UTmnOav$C#$SU8q3631{A@i-*@>_aVw)Wo>Ofl&1(%rmt)pg zUSzd3h*L`;4qVVR;h)T9{MK)#vze_LyO_-z!#_b?Z*<<$93Gf;X)p;Dj z3%CS2C8nV?_V^YN>9iGUCE)8I3=8O7ChBHYYUSv^jDwa_9wKsJ**lF}oFR*bedYhE z0MH4+Wx}psI)ox+ndtluZ1ySb`!(2@Jx>($ZEDgouP!&xTDMg|*wmX<*%2{V#L*v9 z>1lAqw}bdzwF|MDUg~1>2q(KK#L|~-xC7uxMfk|;b;e&ZMHI1kc{x*BvDfpH%2wA6 zITqYx*&b!^0wd>+$(L<4tE>t0u#~zZfHj`Rvj*iEUR-Fr9OEk2cydO>oK|YoF9cA5 z2H8LWN;3S;3z=AVnesWET5|6EeL1^NK4;zlqzx)Y>yfe)A;Vb@L#r7tIY7k=Rh)=# zQWxKJX~$Yhc!_?49HV+1#;ElDDS8||!bRVvrX$O~9q1N7gn@MbCjntl3` zuguMcFQdSpJ)KS!6WlFc{Z3cJCvCE_c6$Qw?cTID{V8gdjkYsG)NIA6XDxoQ!oGZ* zLZx!?f%`<7Z_d@tFKK7BK1`4MZ8`hy8InkDN=!6^X()?~3m3>|KL+O7FGpSO2-1F+ zO42_absVm9(+niDdK>JP4_ku=FwZ4>l$%LW4--3TF<54dFIx-ES{Q#{>a*&+WA`}y z)J0njDDO;56yQaorOzBTs-!xAo9uGsD#BRnD0%Z2L7}v@nmw){mce$5Q{%`^dQ8Rf z7_$vC)uj2j!f2xHUOb_!9yRADpFUP7b-o{#v~$_Qr7qVSdI=HwQUd+}M72$DlP^e{ z&Nb|erM=FoWR^8UDR7g2bV&Ur$Dr~qcv}9^~27mqBs>kVRV= z&5O5jPRxDScNoFxT2g8QxN@faV$_C>oVuEdoA?hVu8_I(B<$JPsfkXp_9^2{g~DSI zlgPABGl23t^iBj{)d?x7l(S*hlC=&$Hc<R2c^hzQ`p_N(#{HhZly?JD%6tVV_te7jcLFm$z*S8R1Zd4SNbLR}{r(|4Lrjody{$BmToHctv)EBT!*C&C2&1$i z+Yph=5m1BSQ$Ppo$C32mWA?y!5S)L1?WrEtQ(3;a8Z(wnV1_s zh)q^D#I1Ur#V)wFXqoES`{r?!Yqq7VsC&zxx)??DcbhVCKBZJG<<2ruCUa-8yatDN zy(ZN4+(BK5YU?~~83xp*zXeaLCUo>l81@nQFT$S-@<+Ued*8YZ8I*etGAv+ilf0i^ zua`nuzc}`gB;f1$|M$pu*gV4gzdph=Lj!v8=lw=y)_`XA@w>Je_6!U_vwF@AL9}?T zVfGd76Jzz3?xVMQZV8byHnVyz43UHG)MD{|Wx5srd!k0UE3NexW`~cbKfXznHpg-d z7mz-CR~VS=EgFO_Xbj-d^pR3Y|`OwB( zJycuAw>3;KpmJ0zN%1ArS#=1pG~a&W1K2A$k1n8!u^Osr+q7wk_7)--zy{Gaw(-f{ z!5V7QCANcG$`ab7L|r=DyM9bo#(BlGB#SziqEP5q|c>^!P#Aw&oSB` z7O}<^sf=fiXyagur(|k)A*89s9ZKGW5Aj9P8nQRSWp}Wcv{7<0YDq&0@x^b30-)Zd z)C>)GT|>FHe(B7KbjA~i^)&^IMg(v)@kdDxOmo-*<+&woaR#^(!J-Syv4!q}dRIJP zULHgcj8SF!XB78eabYADSKAHX+IN?IV9+8O!-MQ?pm!*|yzHWpKOdd!aqvTKHJqai zrvwzz2_-g3TcSD?oUvaM9tJ3@0Rt$`99fG#sAoV$u7htjUcWAt<-9elIb{)*ryy(a z+cJ~y-hFik7ntLXx?(+JzC+AIslhP2Um0DC*~5cQj?t(Ob+;wu&G+kD8vS%(ww4Z= zfeJj}dwl^NH@Y%i2BZqSV0njSdVkC5gS;{E1~a=+(C!|(#^v*udgk~O43OLXn*4o* zyyrE(TfM{Ru;V~Ei1FpPagv$ImbdL#dhxQE@pPF$60?1VoTuGPxIL^|hj>L^YVsRa zNf!b9{-6$mt+>$ZOrDUV1BZ^E#W(qxJyJ}P&pbPkK{-QXvew##K{nDBT_Iu=cI=1^ zVmBK53B+0aImO9TGhts0lv9w~7s_`o{5|b26_G$n>YO)Iq$|`T{LFm$o1mc^5wzl< z?4Ok!(<%0vq@03O;HKPhlKz|sg^K!BprxoKoL^UTkcyPk@XgZl30Ee+=MFTy!Szfc z6rfKPl(j?D`Vz1w6Lmx@L+Y`!soc@3BFNz7jxdtCVqCB%5A;L-|STeFq}WH?8SsO?95DQxO)h zIUC-#d-b7HAbg1W1F>KFBal%44XXdo$h?`p3ZD8u2}UECF^S zirUv8DkyCfRo#5WMyeTM>0YUCwUvUEEMSE2^rTP&e_E~SYFf^VR^S7C(nx4;>7&(q)4tyqxgd)mYXrV?M1Lg+1GSTH3_8=AeL0DNrdd#-jv(qAMdX6@v zB^o8_{kvZ0T`U_JW7v@4WcxliZ5XRbbRm*TEe9ozNmp$xr4umXY>EjKKv^2=@vkxG z{$wG+2}O#OW;W-H$u`gYZEI*jWSPly+t9c2ZY$D8hc&ictha4cX#5$Ojn=Vip?)+S zf+^3^R;F+|26q(IWb@`C7bR&7wqy}sT^%-a^0+3J1$FtmHg7sRPg19lIGluQ8^cZ8 z17Tek=P_w@98MvQ!k^BL0R7SPLWLBTzPf>igJbG7C&7tHwIfh11E{tWgJTX|*pcF6 z6V}d)_3s1bIs%yC8$c9UGh7uf9^hd_9c~WObAnd-u*59ekQv+#agg4UM^G)l5!!7! z<`NLPGBoBqVQ^Y@HqqGG$wiXGN;P@U30!Ym3S7-Wrf03CaGR+LKyfTAF8~!gj1Eqf zjAs6`LB1wH1>9;Htzxeh1e7ck_%pN`b)PWs6npO4Z~np_71eqddMG)-hNn`>Sw6zI zavR5c%pT}^j@myh(>$g^Eu_V$9%5NFXqLPbEIM;hR>@Pj7LywxlvheV4QGZBThgDL zH#fimkyBX}79=eIfGD{1+)rL~(K!tX!_8n>m=6+YX1isOOls0tU@=(R@z4AGJniSn z!xd3t?a;7cH>sh_9>-Unktfj)>BNeio^1oiM@9P|X8E8Dym>OOu3{wTDo zywh*_W~Y?_I7j|AF;5Y}{1CA#O&j#*6Z@DzFm-y#f)ySS+u{%A+2;4ihx2Zfvkd{E z^8h!<&1HOSKwdMx^USg^)3%y@;Nt(o);j=M)^lyc)10<#+qUg#+qOM@rfuW2ZA{y? zt!Yo&_O$+<`+nZdu1n;ldNPX*SZ#dgQRj85e%A!GH;`<*4#1vvH1oW zv72Dz8|1WsWxPtiKsujhA-NBT0M6ap6D!Ygneh9gQ-l$r0OHZ;3jp-}FqMip!z1rC zND10CeVbf6#kiNh0TLj-Ca{X@H2i`me-_gm1qkm!ar)JtrkAOggx4Yd>FzSTpWTr_ z;RAKqwqz%9G4pS)TB!g{h)breyB(cX)jgU#}aq=bd{ zmGjVwbhU-6sLE1P(WHaoM%LUt3SmzfF6=*d`%kP-U|kWMVyNG5dkM|>>MN_X8Kvu2 zR#KikShV zM(TZdqKevXdKaD{cCE{B8da&f7KRyc%+=N{$5tUCD>W_{j~$_{<<`d(9?m~)4(zXa z*SAxKWZfrkTDscCtJs(6R@l6L>yiA(|MXN!(l1g>9uy8AA~m|iE!ttL`D%V{AIy6i zKq6XaYg68u#}ktJRgg4*CD6IQu0;jFB5ux>H`b6Jx^LNzcTBlgH*bPed9f4F z+atZrqkizfhNk+I*5Ke|iYItu8$r$KsXL=I3Srqc)bjqlERqAq)dGB@)Znh$wpaz* z;q@Ljleoxb*|9~R5ydSApk|**ozFq@TGC13CX| zTz#TLZ{1g@b6Cc^xli{iKZ4Ps%L??4`ki{3!ErXQxcvRU$imk^);siUlDdn=&)q27wTM}D49TK(2?DBRKq_!ei$f1+s^ zK22;)jAkIbD(Y*9h)+1{{_^EK=kb(JIc)9;3WY#LLXjz;#8n`B@i{6^yT34ZCLf6j z_G$0f9KY;#^hM{#Md2#~zE2zpBNB9X$UY+k#>4FW^{dtqs`i7^HGIf1f1Zz{=vMy6j00c6bo z&YbTm8}{=|Sp53!#jyTvX1D+oIIJ0jGNZ6|`NW|Cr2K+JnD(QrjP!*>3)11T1q~6A z0Nm7;-yoh;1Hc9mlPVnQ_nq6f%uLk7$DEw*FMPu)gar0Lr{Q6oi$9vf{DkDTo-LQ# z{-L38MS5jLYbI0+CzNo5V1Rj4xJdc3!S=gK+B2U`wIJ0Dik*`nowV~|!`XYYKBSIW z?O$MniDEPE1=RAT!7lwZ({75;xD??qIypi1l-4djp(D2C*y`5z2h@aDN>2EpjM3i$ zY{nQqLYv!4NSwhL8_X(OW$Br_v+1yi!^4zm!~gsm^H(Lo=zJTIE&zCxD4ox$cH1fA zhKr^_!wfO5j98C`Xz(ag#PbNC{B-ap-~glRvG|oz94~7dMb05*+^^(;F)!AE-cS0| zi|&j#HnUa~F&30>2oQkrbx2U_a+UvUYbWfkkW=q`4a zI3k=GPbEa#sMe}pUP&BD0ay1u*cmSW1aWR-Qeci#@{*zI51UdY)3l7;15DMd{nr!S zYrJ3=v~K)w|{5r?PtWE>N0oh#%ar@Z6 z495R^ko@2G-EN@+D14to2N?MKA8vVEf~*H5`umStW%eYqNo$}_+N3J)X+thMiB^42q_EPS?MQ&ZULR<--d0a9 zT#Eo4B;RG>m@iy)hVbHCFi+r4Rg>KCxOvPY%|Ush@pJOn&pdcpy0&+HTV1~kK9mrF zj8>AEOYEW!bgMPGqwXJ>r|B=)=I35N)Fps<*FIxFRL=%H?eeSJ0A zGO5pI1V!dIl-es%$4)4=bqTpnhiX}T@rnk7%6m+fr0udFx4xI-!VG#)#&X4{aCl~nXaJ~_IN>`P zX$m1^n>g|O5z0exl^nyJ(}in`Mt6MIp&;(qt%vYAsx*b9MYXWsOGyDG zaG2lJmek;7t=o!sWk3Bi74@r&io(YlGEYEhh1ux*s~mTx2)zbIAL8U=&KJwhqO`b2lKsf`KVxS+yBF#9GaNptgjW|?#@7*)cyuHcU`zeuDV7X@CDS|o+r zuT;(TS$h>}xsAa^h8{jo3Gl;TwAcD`@ru=bmtiKWgv{#p`Fa&cx0nQ*h1V#vBytKS zuiYfRn7)~_b>`*Gsxa4|IHFLx21+z>z7r&(bdx$j-B%sHE)+oa`(CH!5nus$?;B~d z4#y`;e843Gn()Z5q0cwZ7&q41Eza?d+%1Or2{%+14_iw%sqwZ2UU%Wl(#2W`0W~kyqj1E~I*pZ4vQA6>o!4@eCtxgHrK~Vt4FwdkqMy3EjF5xFUb=q3r`# zP;r-pW_E_u(2BPocgoCbMLyBDC^Ha!s10!`tuS&!v|@?_-q!9z=#t4P%k}7;U`h)J zhn->ZpNg;NWE5(2gu&!RczW622>#XK|D99+uL?&X41hNJP)9)xH2MRIv1Bj^{Y-e2^@uq+%1TjW2nUvEL=K+?liDd_>88Vw&s}J z&gGK4$N`_=WlrO$u*a3hsyI=BaULk}SbbzuGXKD9;kG=Mk``2a9~IN3mvwpb@_p-h ziu?Vx+wTqe2Utt*wG3RfNURAI=Dy(wz(@;01OdpUZmb!MJ&xZpQT6FF;`?3 zi%m~C#y5eAj?Ve4;LNO-iOWITIL~=M-;wRjID8XsdM!ksr-ZlI_h%QJPXz^Fkf}Cr zTx1k2W%&v#M~hW#rGQEo0fx&@HwoY9C|lEfB(q90+ew-nnXX2o#iO_3q~pcX1<&Ta@o$#6d)n4|?c=z?13`oU1DU?c)_tKj9!;RISO( zFFa)R3E@w>I{k#WwqDNS_Gf*ad@O-@zI?>>j1*Z@Vg11oSTgVjz(0vt_MjfOF~^V` zL`?aIp4CgM5>vaKszjrUO=vFvqHWpMLhAwnU#X1Vz&>>N>Zurx3FDd1e7jeJ)W`lT z;Bo<7Vvt!{=fdh46Y&BdNH=e+pD@NhJKvr)lqo*kAyXZmIqidQa1`ZWZ(mmH*1W_a z+C<4y3c|50XyX5v7ITWXwZ}DjO+OUzVQ_yfq@f9TOkd&T=xRj0tFd-L z3n~MaQ(+_68tS1>Cq z>y2vNs2s~fN$&Wtg(qk#eBsX`Vh0RYp+p^Jeg;!dn+6?do8X_%4ZBQdD>XWk3lu1b zp_u_ZM~-~@u?yH?(RVDBt^o$c4&fM_U{Uc!0cAqs zrcf2uun^}*(onpLe~ck$B3ew+gYq9znX$xy+U_#6p|T|r{5(P>HoirW?PHpt3~*wF ztumY-fE*X9wB-C72-IdCjGrPgSEOH=iey|lsnjBJ;R)2I+e}dyVHu-vDrw_z8foKW zhrWbWq;Xb!&Fm`UXWWD!$dLnz3u2a5AY}LfL4KMPMf!HyyIfS!?y=YrgD2Q}4a_;l ztXGEm7(LAnBKpWF8Ul7j;KUs0wXpdqQGJ6Uz@0WuOD-NweeRpDmjA>K7JYDO0vWch zbV8ILxipKhOoHh^GTEpUTYS8l=@o;iZ&5Iu7UVn{e0;Ax5_;XWB{!j`H&G|O7zEDv zfGBC82QnhQ%z<+&BC}A=z(AplFNHVanqTnww&-a5o_y57h2hmJBa^SGl)yv9@WhM* zz#;K4D=BQ(^nkEbphs+CBG7_1G!zQIsejy9>9s#QPKtZ=DZ%~YMWiZ}-$mu_mN zw!&~h8&?>+rcOwZ$DS$2PNjT?68R_wI6`tOxX?Lfe%mD(Ugnx|M&o9S!Q`3>?vG+f z!8`m22f_K6ol|>V@04Q6nz2e6%qVdWsIY{>pqflsG4{Pu z@3vY#{dm%NPT?_ zNGba;k}0ei<1YX=Ma60IXxm})NAko1&Nz0P>-e;!IGc&QL~VQ=+yHaok3aXRcCoCkZ7ZVq&MlzP@jG`C!3*;5UB zT+g(_ZmYlQaQ6DfMqH|0f446k($RbJ^Yi4yipX*AcjhbHu~@GG z6|C6#Zm$`fLm$NM*&Tt1wiEcIMC^xvh`H16IUwJka~N#k*#C01Byz&vCm8F@svm^< zB!2N-deG$cwGManns(hMDPxyL;2?zl6Kb!qF%zD+{tD>C{Q&7M-dWz5iHyCCrCgs0 z{V3hxe7YCBjs+ZSLUa0|=Ik99v^;BI@=Z=Vv|g`0;X{6)J$9WAm0bI*zka&g^w>qb z+Il_*v|NSSzaqb~1chQof$2MfAw16qv+l!m6UdZt?+VK3h1g1en}>z{1U0R=k_ z8%V=j*MQ5z^{1^TAM}sh9l{Mqq>sHJ!CiyYl*5fD!6#5oM(10Uif2WduCpNx%hcc5 zg%V#lN1}5v`+B35sbGQ<_o!9XT6FVg;4gGW?)@fj-IVSN$xzq(4;!vXEElL@h{c)bQ#GVx_%Mv9BsGNz3*#vc)}W`clk|H000 zjCLKkr;?>V{gp!hj8uvME-86OHCn`ii!>*hN|9hRKnjN3G|j^Dh$TITiG?-$)_W?2 z*b=;8itre}Kx*QI;S%AD7(;=kn*%Z3CdaXidLaUEyd6m$X* z2%?TnL<~{h?-9Z(uS11c+Q)cR6%-K1Fcd}xEroUD{dUa8vkVc&;?mUdQV#qL6%~S< zGL=LZ{1*L?Jit))iqpHasX|f^nS@^$ok|%Q1C8oXfU7 z?bz+C^tGc!ZwD! zU@z()*4XVnD*hsZ$kxuZkiGnd%%k>#S$L&l`0^=3^>tyvLMKM*8q-3yXxF35m5-U= zSsK&%he~lU2>53Nu)(idwl8jglDsQ%-l#y%K)KW3=K<6TAc?N@vb5#5TJ>cDh_|C( zl?<99-tT(5ybcVfZM|-eSw(uZsF1q6L&)3|>VI5zMa&75L^U2Fquu!L<~a4GAuh!G z=4E$@UIN%z4t$XSXN$6n)2fwTJ9P1$82Ph`)9DS>jvH(Tx>GDww_fjjHb#Ba2OqCf1PH z-@2G^Uw2Q-c&ZP*CBtLEO}J$*(Lku~dA)*Zu~=gw$ilLmd!WRZ%!*Z!gsXDLq?P$G z5JOU2Ax+@SjfMl_AxCnN)c4D$>Y#{QW4_0WSo(N!VJ)dt z4Dp+Yih}2#8|pgX;#$K8lYlu7=CiFmyzoZvgI93I-lxgA%R#U>_2wvtn9Dtu$nDG{ z?pW6o2QPGRa_Y3QIRp^rLV#pvJ0GUOX5Ix^S;%iYN>^%RmMfJ2zMkVO-X9%E(b>EE zkfOzd@FM+DgtyX%OGY>~ayG~2eJkLBc6EAV*!tNdZiUPtahY;wfB(#Ag@`Byc`xkJ zMYjH@Mf~i>X+DHW=Q9s6X6U}x)b;`jQG8@sE;7EFRCcv~+N6Fv))!^?@>WIT`QE>j zV9Q$m_i=$wg4_f^Hp^aVYbl#-)oMZkkIx+RfLmfuN|#2{B`U029euEz>YI2O%@|U| zV-$p_GL0Bw>*dcs^Xnt-&vR~ZU#9$xiM$H6%wM{~LY{aDYB^K93a#li#JPh9J;}o1 zjP#Z+8I>KLT@uP8V89JfL?k_3=@iEAdb-j;@{#I+_YU-cW(L=}+_@#r%Hc#>PI1x~ zxg0FxXuXtqGjkaw6%NTGs7u740BFLRE!MK$ZKhFAZmrl^((xJkSmIZx&CJ5lz?}iz zgy8HQ>m>`7dETOk!!pf+2(9u`)oW&SfXD$6Je%xLaLul2%t>*MsJZti_H@&8H=b>` zq&NAPGRX^&eL^dGlW(u*o-vLYVfa`^ELw+3E-!r&d}Kq;I8{{TTljrJKcS1QnA;1xBLXh~?qO-1*CK^_Z;ZS@ z)`Sw{>%@g*9gPrsUcfRMV&mTSYxP&)9hE;I2opKo$M=16)v%HSFMN4uNJ5fYo_y>l z`Np#IhPn+ettD|9TbP7>I@>HA2CJ5=CE<8F&>nkT)<)&$tJk|SydStA#qe}nsf8+{ zg-#=!k^B z%N5*fNQOo0Gd<_D(`^f6cc`FTk;X0pPo(~32yZcl;TSSrr13Lb-MFUXHJlj`MyT=! zqcS>kz33OU%7p=Ir*N7jOXDidl89w1C%sZP?vSwj8XV^x1OS*|>5e2AmY)E}=I!F5XySf@2Q?9W5QGjw($g2uKPrG|| zOQuYkH+aEogQr!H8_eIfyvPX#VRqHsfIfJq^G|LFa9b@M3xgEr!I1GWAW2%cHnoXm z%qlg&2Bp_(Y%+#oUn;?Ng;3V5nnJ{wut+)`%4H@!nH-E&^yl{8!B&JOBes+ndI4VJ zW@&h7*teJ>2OMiD7~DxN&CmIPd(DF0 zufI9!4x+fuW7oEc*Nt7VNO*=y#CcJq^QgOq3k zEpY3V$`VtuASopUccUgjYUhzcNQ#9tJKrfa5bMtJ|CT^qR64n$ybwjq`lg-1vE?{V zeb~MeAEc$eWTCJ($A$~Ri-cHMa`hqT;Pt*Lz+p1`bxKqCBR#Ah5T-fiIc#(A{eY29 zs|Ht}b#uP;dOu?#8vlK%Y0ABvQ+*oFQg4S6MB2UaV)J67Y6I7!+5N#?MUPWdhCbIf zoTI`uwiL*2ZFYd;Q{8bGwI+qCsqRsB+jO zD>>g+gq?zyB%$Yve(})e@?7+YpPj=wAbUq~^JeBaE#WxLoaWR5 zb`Njd!mlibiSsS+U@yAqt|j1)p53b};;9o2Xf^0A{UL}Z0_8#Z>O(=vN-7rD|Q*U>{CJpmt_U{La{E~mDSWinJKxltVy zFW(dLO7c@b9qQp(!2U|Fx0XU4W-wUFn&^sTKJ(I|Z`asr)7IPoeMqHVD*twKDHPS9gcf@h?P|k@!!<&9Is*1| z0!OW7-^e9cMgD!{k8dv;d>^?NmtsuCCd&cyy<7T4;aSTKLx5 zRr;lC_DT5I+O$RdT(9%{Tl@D6o#vNhh%0?h!Y@#z2VfE>YFAb~{G39>1}GqXS$8%F z{Lg}BpnxkTe&mb%C2Gk)w%dyibBW$b4U2<`#e<0iXe1ARZ*aWO*dNZC%KbJnJ;LXH z+CKDEyikP21_(l&dF@{XKqJ8GQbNG_7egd6e+2AVgY8aUeY55tLVLpU)iqY#);}Tk z4LB~e;PzFH0gUNBT6wc1h*BiT-zwYfuo`rI zx(EafzkKVF15IM-GOymDzA8n{B}f64%I7DUwWn~mEDvs45Ye&D-`{+@`axEWMo^ed z;)B?G2u1veprRNYO8@Qpm2}B>P6qNg2T1ZH0tp+NO1BOvJ^(J%IvUX2vwPP<(UUwi zzzHBnU6YnzTQSaG|DS*t)7N5}k4N!4G9lb~G)Cni6(49S$!dgkW@@n`i!FyIyUct2 zL+kE+7p)V3-DQ08W|Fd*7I}RCUK)PsjFPSjKMofwM2vxvPEud+)+x4TXwB^ZZ*%1} zg(!B9!SOg7-E+#=WAoVZ0X;fQAQy0p(#r!ZRDCd|xfaU71d#->;<3Z|!zOe6oiFmd zcT$mbah*Xh{2b`s6UmRVyle-AdCx1!&zg`9i4+c~Vs6O%HaIgCHNKq?6FG^`D*R-J z`L6E_6kl~A2)tPU8ZyAj+NX;D6a1_Dwq(6+$VwpUh5?y8W|zdrPU|KLuy&bCQVOWI zu`{n{neU>IB?heAZ!M!%YGC=MNgdOt6Nbt-sDJDx)U^J07^_Ig)+F*d3s+c?WzsM+ zQDDv65ESQn_J)*=#iiR;P8A=LBUt$njhf5OuF`KSFT{L%pw`LYkT-lj8zTe_;K4&x z@@^>_&|!hz_lS>?B?!FP9nS6O&hOJJ)K{R3Lgt*$C0cP8~26>F}?G$=Nyh&bbdvESd;O-p{(YFIl&8A z{}L)9rasU>QFfjN{tY3Ur37`w(kKU2W4=M-_vnedom4WtRq#5!kMCpe{ z@+2L$Tk>0>1M#Bw#V7#$rQlCEt>mD7hzFWOh{{DJ(Y1**9%Ql&N zDOjpu#2SG*N2U82e!J>1eZ2nh&Je~bz7p+wBwg)`GL zbm&a>Gs?7W;2!%atrc}$tozqogc-)cH@=27_M-DIB8jC-vaIw?O6VD-EyIwsrkBF3 zV6-XbboO>ex)F7aC5Ku;u^+xNV zVg0$yE)H$67*5vAjlWz<-wa@=bxhXuk`8YjR|vgt{yf}<0OmQH+pCALcW*pQK# z@mx7IMO8LQ1}ftZ5`D6wE;#ueGCI;fy!fCGJef=SR8M(hx4OYI-C-nXSnyNdjr{t&;Uq%+RoO9cLI@Vi%S3TYVFeHDgHc#FAQ z9ZGv#hzC3eI<-sUAE}-c=3P<7UtFL&m{Z7HyP^P^M9}o$-|kS7Dy}-Dw?2>pFn^1) z*6YTPOyS_|*1Q`@1$l9*oNGcs#6&8yGys2_NuR)8ML-*&PMRWY42*_^+!FZ9CMucM zxU(U%PAVN~+OFlUQ;=Oa{hYYr9W6@^T9HspTaUAA+=MT84in;w!(qjzwGBeg_?@~7 zy7L53R^{4bDGLMj+5)Z@#b9#7qh??|c{+=tSVa9*ttsh5}t&Q5G zeHCwnu9)Y;q2-1=e}j*fG@8$JYN5m5d{F^#T!VL@pc2H^ngN)c>~T8s}qMs6nZK5}dzaW`;vTE5i(YC~lR^E07r#d2bP`@8dI& z%cX9fNz$<#lV{_L)lwGnleKF*LOkO98mZv??HtSY*a0uJ3T=g_b$)Tt6|Idjx8wzY zb@z}M?qKtcihv;aoE~Bb7Z5prN_TZry-XO4FRy?2t%a|(n{h99_laWVGEwR53Bwa_ zE5s}2QmAv6TJyvcXNjxp>v_!O+D)p$opJPD5o-3(Z5SqEWgquQwg4i=F5lj2CBaDz z1}4(EtPw(XEM^?uQV(5aKdiKXxwHm=u0RDE9 z*iocvKL+B<)#i9mUo+Mj`y%BlX`PKXvfxk0gYMBCQA#)}TTnFiV;3iLo^UmnbYzqbMEG9&|UQq`we=ChNNAT7g2g)80X2Sqs4Qhw3Cnnz!cMnyWe^}M4+I(4qhS-U#w zoNoXhy>e41C)RaB8{W5|=k=Xb?{>po#U) z8g_8~otKCx8+H#K-6?{=zr2$WXht2VzkFTXPj+U`a+X~917gAE(*^2x(;A&)588We z@LsBC7eL`6{oA1ZBs5y+~BAH|h{C}#YsG`J(!+M~!Eea>dAm8z?w#0-JzgOUif z$e*k$>NuzQwd2VMzocz`s`D609S3~B>&VV_C^)2-LF~{uZp^e`#~wp%iH`d8Pgy2* zj~7Dm@W0yyW*%GVVw4lNDNY!_A=u9G@hWD-_s|{ z|3%V-jVq;*xGt1*4^JPV;pM94Z<1U;rlsb{NOwh}HnmTG0$7J`=^9)VYYg&6dk<;w zEJ#Fq7LD+|fcBgR?`W-$&)?R;NmRE9s>DaDV+b`aB9uwaQmQ@+7#vHkk!g((SycFy z3oga$V;IQ-#XlD3W+~AK)#Wb|AIUM{ydJHA>x2n*VBa|DYK5TMAqfq0V8@{~4KgfU z&MO_IQQ9Io2mmp)isQ--Bu}%C@3Z1!VzW_87|%O4PY+QOZ;FL?W=qB(6Auyl}PVG zRmT!aLi-v0Opd(5Dc_Hp5H3hR-^ch zZ$-#J#fe;SNhI60lHz%tl_x9>f06k4XoJe{lWcy@A(*IHlSn9%v4Z|YY`GpD{RK>ALLMng!1B4~-45-;U{MlMCi zIZtnO7JRAyF)$jMDq9!kX0`X37U9wuhJcFjv=<=v-H(b-qa)ODo0*ga(S! zVFUa&TAiMoF53S_pq1N~U{k#J`zf=h2VigupuF>%y^9#=bgplToh?am*=~6T&*n`| z+y-NwxIqzT%{B&UNnBLJp(|#lp0pUw>?wGKodDV_Gwrw?SR+x{wh;Nr4GqgrSbX?L zw(|q_FH-J*U$y@im#RefBBx`Y7%=zA0vqs&O=bCqz@&MC76cCzf&UZpyrdap3;+3G z|10qN5FH=*9*KwC{GAho6%2goFNf0%J02V~!t!TQ?B~!O_9s!{6+F;r7oMVd>4ilI z68!Iz=H4TCJ}?BH&$@Q_8E%(8>(BxLUgshZ0CQD%NrAmGcvs+mpEkEwpt*s<|0CG- zVrTn_wRQkHK~WF@ku*S%fH{&p#Le=z@FAf8N(;fU-hzYk{w>If%vuKaZ*G_dUM1we z$>c>+Iq-jz$TlF9AP9eD=Oh`fcRrh7{hucMmg2zzwr}vl0-07xU;lmGqYGZ{e+okV z0LcOQH@CtcMDjnmOlzbNAm9vttL9!O{RZ}LGG>bu;XldT9a69V2;lFLZb3l*?aXF* zov;a@{oq$7;MF`gMl+!Q>j4P(KToVaGeQ4N3SlDzLH^VE#H{i*CZASo_E5lY?jZQU z+^MhlK))#-$Y#b9wC~{miq~AAu|WTmEcvJa0sE(5*o}Rk#%J&4K>Yi!&9Gr20R^Uc z;G1cl(40WwWj-6OkWjCa_9@c>%p(B7N_2#T1jY#?d<8myk|6@t6z7$&K0XNpB+nU` z7)apKber6SI)9T3LpwpwLjCzce6nU_qgmU~;wEA8D(QPG?6ob7iXTofL`m?P5p~U5 zV%F0$!1m@nwS4)?u(kaD=l1;TkBXKe;_y)ds#KH;D|85W@gy-O)g<=PR5?QlJ`3?w z)LvDmG~qI*9M(HPmF+cf-L<9N>gO~3+J>%S*jAI?nt>@6Rkoj=c1b^awK;R4hV8QV zuhq>OoY?0O6KJmLu`IXEnn3|M8ov#D7BqF+X;=C!fZbB7j!R{T7I|vwwv7FXYB!)-2ULAp*t(F=f?=H&UixNJ#x)2m_!AiF5tN*=g%%hO$1Vc1` zD&xLKzCp815qb;u^Sq9|da+b{yoo7ZK$(jY%hl9TToT4Ai^3_|=%jnSEOpt5pr41q zIMb+f?O-G2e4!Fj^l~tW26N zm($IRZRRE5Ow&Hv=%F$#M<$pD+NMGC`O$`4kxe2?{NvFWy3r zyvY$hHEN$AxQRfuq|QUHl@lazeqMZ&n5?cyXkNQXKxKuJY%JROLGIvtKzt{N=_;6VcOfgA1WkJcej(>xZV3<8$D&<^| z&Mwc36+=r~(yLhR>K@PsbKG5AOcCymc(nOQtINMa{MBavXYhW`j{j=2zr)%O9=dh^ zQ(;|y1@?9#;Qt-dP{1Bu)~`TE2r_tpo&)YD)BWzoy}jq0sjb%(Ga=)5noH3m}3H*#U4uAXyNac&-809 z*eF#!uBXMjtr%cy7E*316^$W_WK8-Edc2wznv+@GCwz4P0_|0kxN-d~2Hc8YKO5)m z2{hkd4*X57-#gNe&(gXkACUo6U*X#a9XpMELTn&{}Gw=YmZ;*`X18^4< z%;AyJa!xMLz{J!~%LPv$@rR&!-i|&B$2QE$g$V{I~+^BI`2n3a6Vv?CuL zke~Nv?v&V^v9a{c3OfKSlHuEFLgu1&bC>-Ofz6Lc6S|pL7TInsZ0vWDJ+mj$@Z?SD z%6a2z|KbR(I+unQ6lSUhg;T}eyH{#s?`^aM0S_3hZn_+i=XeDZPyb;e1@{!|38SDm zw4a8cx2`XsUOh6NsKl^8t~@E_JUf7?SDu^+pMHZ_vJ`i$7oq{ew6Pj@IY=*g8e9xL zHu+L4aA6T~t|1h%funzB_GCSsr0RTG;%wpMvkA1duq!)Q*wPvZDOi*9pZN$*pD*)9 zD4#8)j`0U>@7@q1ToXRvPdFSZgti>G`$}%%w7QTmPK90|O%&s(p!ndF`o07+?c4sC zgOBU`63%pN`-2PsIsiu|yldudJ0sW;p@yRwpRQlxv^jvm;TWXjs5GR}LTWh`s_Kp6 zDYp_CP#TM8#7u{rK3JM=3^3LW7C^Vy4$VL6CQ3gdL>t||Au8l4hfOC{B$6GxOHPyc zB^X)+_1+RWZXlZcfK)HIJ*tP6-l@p5nSXa`+M%Xfx}BLscrr0|B$oL>|HHi+AnSp}f6bRqql??#O>{l0qB}yytJhUOaB8zIZTwiU{+*C;Y1>{;LfBqb8vLfgb+V z6U|Vacyb`%*MFDgBUri~f^R(d&CGHLLI27KVyGT> zf9k7BSRk`1-6sjBF)vOt(;nUm_`h!(%V354S5l^|8HxUr@7bRccOD0J2iYTn%oFi{f}D#N5;ixiiC-xF1Q6q>=49NUGQ7 zdcIVZ2XE8VSY*}irnA9z3PTKn$4D-}T5QvNvC+Au^XS)I-*ESG*_mKVI*Ld&^!Ofo zoAZ%*$@hM1066Dm1c3BVFoPZqlEX7X0+d2z(SOAP7k29iV<8)!4<2@=d-m6JRhBl&e;44ghG|JY!xxX@thKpxPz9xuOThc{Thq6Jy^ zmj!vH>B`gYJ{Yd9cyuToc7RkWdHTTc)9JqiSBjMt_C@W+9k|?nM4uO&xe5hg1N@F% z7$L0C`w;8Tft)!*de{2G{^TP5go*Rc64u>k@b#MhH9Gx8hkWIeWbWll(+*+4ljLh+ z!dVV_BH)|syIR?ugYYZwJ89U*0HACbVDpqiTX3EMF4c&;yi;iNc~5mi6FU zF~&)8D<(G?M5T9{XHhn1o6U#XIglDB@W@@$pP`sZFj42TN|082tUVent+zLDjdDSM zyGCZUHd7ACwfSS~<5%#@GO3SDnLNO>YhL&kW?GhIh3|D!RFy2toO|y4O^!l8e--o-?SYcMVYZoqo&walOU?3Kor_U zlPDJH@Sz2}*&XTv4s+}0c3jU|JRS#Hq0{`)XeJhmGFU`(ZW}LlBn5)FiqoRU*Ulx& z%&>+7ueO_}ff0%&7HcgNK!&z^g++HVOFrd#l{j(cOPy{wyc z1L9(K=V27`bP}6_+nmYp0)}%!i9ChPBu#aUsVe1Xlnj7OfQ?t}#W%RNv)P3rPUY#L zD1mavs=$r`^iTA>1f2=_u{AlDd-h)ae35FJ)5MBMjk!DXkp}bW9ou{G`CR-Svx_#w zW3N^#130^UIhz(4z|+tD45RPe$Mo2F-vJl=Z*`$C-d$l2rj-iEqALzjV}CzFdn9I=jP7hG8V=1W7%V-RP37Y9S3zY$c#zlc3m} zg=6yB0%{GDkQIe*P`afZDKL0}KAp~}aXei!aSg&hvr`icK&ODa)W8~MA}NE0%WB=^ zl>&e%y3lG!xA1z*ICcE|aZX1S{NOgD?ZN z+3AQ5W77VG{`I^zNC(xY&zfi^XQI$pLPJ;0F%RYIm3L!j3>E{F^-WA~#B&RC2X8t- zQRER%$7C&>Yp4P}rg4O5*~ZtzAnsUg8enKgE;^{Hw@Lzi%qn82LM-6u?I z&17K!c)ol=u2zZX>1Q+he!96fBVcW3NqU^5`xNA6p%jQGbJTimd}QsPvGvFq8AZyF5ln)RX&HPry`PxW$P_CGkfQ4d^DG-kTwjg z*K?cbW`>BNm8N0TjYQ)rPBv_&TaPdfy|#R;*QNA?sJ1$#W^6{T66BSmaSY6PDwSC} z=gi49rbQE~96HG)mT4ugDrp#HJf3D#FH-~uwWahECnVvk{%!8#$RF$FdJ?XKIrXGC=Zl58RrONzdV{$eUR zqI2VBdOM_dFX|e+La*>@cn`SyfHuxdARAAvdG?Fpddv5l_4HO79M8up7yi#u@AIrP zwu`0v@dO?jAJUK}4PW*R?OU0P@v8Vt)!D=W;v*GzIb<4&^Tq@p%lvW4Kd%;Pg^q{a zY~)#g`bsQmcfj!(Qeo^;I&U`{;=19kZ7g$^n&KF9BTB}xM^+}boVky~0m7|+S4o4V zD$TNf-Uk_y;wxbHwM7QDE0}WDo9ainh#9*Oy^6%CZy1+Umtm`rJB$Q+%VV0`^!wd| zT<6Q{5iM7!bLc@qtT6PEo)f0e0(jm!)_-@(=!yA@jV-W+e-K$3VI9u>`~i$u?(eXN zF1bfG3Q_y_4yEJ8%+t7N72uA(T6x_a6cNe(ROrJX!j+4=Ee57$QJ{-kkC4e|G_}IZ z6v6qkC5f&#*O+sX>yqxQts&MKG@e6|EBxx7F^ATd z2S+gN&WYM3ui9wi3DtlS==H-9<)p-dckMlJX|>2E3!*Ow*0n43bHJc}*At4w!{e$p zE6L9(&zX?5y6m!x?a?=-Sjf8c<)#8qJr5+vXVr{gDmK$eOLoC%lzEvZ`M*VWBUztT zc61z1A9ynLzCV0g^eDCU;Vt1)Lg2ncShIXkyMCXQQS==dA{@b>M$1_CghAB_SVdBkB699D#eZxWfup!Hlkb~I3h#O^_w^<0M+VDT#< z^|{Ay89xhgfDky5=0*5*4L{xu{K(HZHOJ@ssW7BCk~hNRt|tFZ!}B-ESs>h-hYT^&FzacAnDIeyHT$4q7V z-&^MM!Zz>h9_=29dTQqI_8zJAo@YHJJ!f=x{lU>wH6N;iF(|#kQ(8?H|3AlQd&}~- zUJ0Ao_!b2mRp|Gu3O}=VOF7uZ7)W_z$R`pluuQICa43&oFi z&ZsLD2t*_{u2TcFZ{ao-!o+R7ehe*bcRm}g;Z7cI^F|W(yp_Eez4piM4(ZxETq`+W znbVpc30&zM&t1`V+v-s@U+5zN zL-l61>xm#>x3)Q+Z?d`^wxR5i9B)p^A7mbOi2OREuua)U5}Rd7dL*iV&r4gg=f)wP zGkdLq!(Rn3+9w^;1rr$Ex^r=d>~Zd&nz}J3ZC<-5#8rD@1%NNeZEDncOwX+g_^Y4V zm>;NLStBoa0NZNJ=%1NNLpP_~RXasUbCD@tLEC5$T_QQ8biO55R%=v4CK0!o0TSQ;RGHw<<@g{-mrc8{dnA3 zQ~lw5aBv30C(P@hrVP<@j!q5os`-aHzC3F>1p66K>M<>fWuCG08!zgECDR(Wxp=LQ zHq4zLJr?295*q>;~y+E&e8&*AtTa;FroHxHI(`TNY0 zC`ALZ2mBu8Y))K4uAQ6Ng*`z}WePG~D7Z>^JJ}K-HN@)SFWL|Bes78&jIsYlEXNSC zZI9U4Tk7^?3RP~pu-8bhJlXZYev0>Iw-b@|hXJmw4puQOOHqj}OQ}%2)mqOdbU#6C zPYVX`{4>8o< zDC`W_6jCSxOs~xgXnwo79alcg(!cSxT8ycN^3+c)R8KZy4IcK^Ig%}d(3oWn6n@00!3~?rX(11z3R~? zyCI%BQERu2yI_1^{VboJ%ypBe5MGZaB;w#-nH%J=$c;!iAUo)pLBb^H zjjNd(nftxJIXeH~LaCn>HZMw4#}?$NMK;f9>nI9Kmd|Uy#VvBbO87>_&R|POn7@md zyY$Q7xLo30D&

0C7gGQKWO_*x!Q;eERHPrcRg7mbL_x_R;F$GqSl&j_l01CZsWA@<>>*FLb zcn$z0`6zwz67Bd%t$zHDs0BkF3^e7e*sY}LD#0~rwJc$QAor;`T`ojCska%F9pt96J9KJU$(?1Ow zWX~%_k?M;Xs18Dog*Rail&i14ZJ8)AG^z(EKcGfr=vWjOO-quGtA3RblO_{Kobqn0 zHHa^6=b)G?2`%k6$zW|-+@GwNnEm#Z$TZfKQ|K>dR&Z@0_O)k7(v69=OTwR6$kDmL z?prKYTUA^|PcnlTclmGvuZ-}QA1Jn-NJ_UhwOv`M;QzvgmE)~ zpY}v{_q1M#A?+P!JTsSTs``0}w%Y_SOb2awk7bm)Kv^8;AKNS%g1qH&wXF0^Z=aLR zjEPOnU4!YJTd6{5qd==lW2Zq!Y`DG4;=m8X4U~shV(KlLM-3;iR@VCwe)g>!#kdx& zp{(#QEms`Y1fg4YjNW+5Gpq_Fve=HuEOVF&rEo`E8X0p%3i8NsK!||M&$DL z=R332A>6cBPhs0%U1ICNwl|c$jg+;jjbE-Ph19%cs}lMa6R}Xffrqyrh{kzk%FD-G zOPkiveLdBXM2jtZE(wvip@#?f9=FkWWfpA?v8F8-?l|I+MzZ3i4uAh4Dkwb$ShcKa z#)w3mM`<}@ERCC>kU0ybo1&1b-KdDt<-rJ=QJk*O()7P74r&8WY`lSlR2DXK4IiNC z?d!^Vgy}jQj>hE;$_&a&U*hOfRvrjb)*eEGccC8IeOmVxt-$m+`bmJvT84}O9Q_0r zt2U_FPvF~nyXod#ac;TOB~Rb(nrl@&@@SYvgB#$x95-UCHLoWJg?KB}3;?i{AvQXu4Tw>ngg0R2?+B#W}iMClDhzgHEIE*r#cDi)LD%1kVCJpI+5% zb9#V5;G7_V1GokXw?pqrBK8S{f!)hS4)8IC?jvp`dh_n>Ch~g2 zG|FABcIsrTM%fjb<{*ZdxG(R13N$uMErlPwD#@erc4;gyS0yG&-6pR#6n|CYqvcDb zmAwruGtGQPA~?`@IC!cEQq`*b4xIdQtA+)@hLi?->r}j$QyOt!-?Vm(E^_h$okdV= zR~q%>+(Fk;SYeJR2{%%OVDT{lX+0UVxm4z#LbT~*Po1+-2k+8$`y77IRwf#df&J-- zm7GU%L>vYaTsi=|u9UBHB0;s}4h=<&?Nu+EypHIGwVlxIR#rUhU=)JDA3iunsoW4> zRMOA7?bdLoRHMB!N=bxA^8V59*&<{}i21s_VJo{-r~Q4JB;s`=T6a=_=;bK-Uw354 zZ*|F2IMsg=)RJ1U)~NU3gS$~?q+^^Eh>5PorM7$OJ;LJIPC!t=oH(vWkg{O&W1fJ<7k+nRxaze- zJhVg%D?94L3wF7NO(JAGnG zm-@sI=&0-tOXrY?3%pRjG)2588q;zuX29B`A`5o_b#q3Z^+$$ji46Qn$qBTNjBwvmr>uc- ziRt7r>)71}bCWCuknW`&75>qYjD7hdpSK7&qp})+Z_=eB9KuDz6BFfMiZS+&-O(K8 zIQ792GF_sktI!%>o`+pOQ4yc)_Q}G)NwYl*TYT`K9zD6CEkhC*;=Pe!4KpyTex_dx zH<;_&+7=&ej@;tW4+*b87wV7SUTccV@5xH;UyHwUmms(Sm`MVxll8rDcqv(wHoH6f zI&*PMiVAXKWr|{#_I+fhC2FY9=v>7U?xHa@j}IRR!(L-WKPXFjay``nBgRIop;Q^h zM)xEX;&*9Od32*UNVh!wJTdV46&gsA-?2tWEI=^fI5_dxeQ4?(Y`%8T@?)7M_+9n- zzB;9Z6QU2`!bTR$;=-%J1HzoMYY!FHePjuD*s+qSauxw`;~* zf?VM)L-~dc5m!{~m_60@;$rU3X=-W9qY(5y6q_FJFAs2B9eBX7{aZ`%{}w|3nKJ%U zBK>gauuQd^1l9(m#(QG%V4a7d3-jVh!V$|eVj%+q8!}E7T4_eltQr+d!Uj9v(gTPv zkK$;}E2>~dh}#b02+>Q;)gf;4JX9@9|CoFI>GqpAo%&V_g8vKE6 zy6Toa2u`IPb{eHVEJ_@G^RpylD_v|8^bE-N)Hw%N9;|-Wdg2-Rdx(8Xg5NCi$R7XT7|ti%tsFfKe1wLb_zH z@o41MMMxDulb{%z?cYw8g^5_aO7n!DnGd9GJ-lAPAwnb_VJMkH3N3MJLe6~RRUX2Im1|;G0ca>r%PGWs$Ib~Y7x_KS4hpk9gJc3(gEtwdW#-M%YaQ zJ!Au;hQudttFzK2iNZd(r#3&QWrrp45bYEY zcZjyhm`%wjXbv-h0av|c-7U1wrX7ZL9~32$_GUrWe2F+uef#E*Al4H>-SQ40yBUqu z9d73b+u|7;Vgi0>6_;?2wOtPZXBESs?WO$fQUSLp1dQWCauLl@rT~YhOsTF^TLi^b zt)b}m3|&%>NP1sU!9glOg0szx5EcS3C|pyoLv3Wm>#Zm-B%BfB(Gts6-VXOSa9|61 z0CnOH7NbKABZ@-~fb|Q#^}=){7{Y2{noLaFrKY=8qPrumx;Cu3L+0Hv9=>tJFOtjC zfO=(I&@fdeP!mj;P|608TEH%*qio6@))|!=?uVopJcg8~^R8s}>L}G5%&3<(G~yMu z_bYqbCS9Lxg3ZtToLKjnmdbvC`e>Y!b)iQ2_?HXbKly^{d-g{G{ol=&Ld}i~3N8>( zkMO@SK1j$=WB?s#AJqZuudbG%wT+JT6+y``SO}=hAO~m=Com9{aZ{9{i>mm>0k0L>N4Vvm_-od;P@kBNa}CIc8~DonC=L9i>I^H;&!IFq;J z_KdtBCjX{02=WeuH#jtdDkM9`z#i@LVkT`{@|t$R+Vz9;dga4n8p2~nz6_u+SiLxw zM~!*BpD;6gU!(`qUc+719%;cN*zBjutEPDit z@LmA>qi;;hPlNMX2B9P2t#)^JOjmc}u4d$~^$mdAPH&2;+lbV)m;4?zIuenP|NL+5 zs7D|y`%X0PIYr46TMnQh%D`O+%ia)Wt!=qyfWu_{Bmhg^)T^can!O{N(0#-s6t;QC zPJJ*N?t_8gLk|1Otw(Hm3UO<9cTLx4$p6W~_Xpo-pp#@cjqn9T{4!W9*W z=8L-p|GI%ux-&$J_KGg7VBb8H*{*r1d9?trdXS@M)^gerysAX%Tp@h*SZ$qfB7f}V zH=3E%+Uqc$pOtBd=+)cp)_QY(s^euFRj-~zLK@>3hAfkWCBc)@YAc`I*rFF|gB~H1 zKF)#U3)%n&6#LxLfSsX`M8)Y5LX}6~J{#e}|Y*tCpxg$x7i zX;B8n)@hEddCr{Ecm2%k!&HHv$yN4y$1#xxmwIu9Xx~NdX^cXQFH&ho!bmMGES)DG zRko;meJ>ZqED@WpD$GY~;6bghM1TbSw55w^w4^A7URkUcr7N-nS;e_~;ox}oavf+( zv3scO>@ZWc$eCCogp)W~uuwugGw%gp_e(=LCWgF&(H22erdm8W!IMp?wG1AT57|b! zyP-S}tQno|dK`7QtW+S18e2&&G0v{4S&p+}ktrq3+uz#Ws6qPX*PNzsI?fkn7_7Z_ zaXMeJ!&{%l1ifjh-*4{5#zV~=rb0mocivzL&xBQ}Ttl7e63g<~2~Fov3m*f7u)BZk z^{p;SeTz{;zb#m(st}a(it8pP#fAkOP=ia>zr6OF(c_%3nDTMf2Uf%j9yMzljF(ml zD4fkqh+Y*zdD-zIDJAa6;x6Dp2a)5}c z*&+nLp=76@LsA&V<#Br)5p|Okbb-;(3J~pIAFB$jy!OV&SelH}h9SCE3Sh-TG%EsP znhWPEI<3k?;OpPk-sS~0IIvNh6J``$j!%_D?$fFz+SQE_mG24aTND8qL5%DL;tR`+ zHM&b}toRsu2oLQQ{L@F?eE114rh+Oy^4E z#g<*7<}HeB&YtJ+-Dav)62}yd4vw;pZ(XH?SxV$3I9&pQJ^^jlOk_?zmQ9VxyKR+h zc(Kn5rofZ+(&7Lj#$dK6MMoP~hoFbrDDVKZmsuAP{%le7R!}$EI#fHgZEJ@H5ceKF zsD`;i_60bPS6^Kp75~~kdYE!3# zTmNsV0ItvqBH0p>>}!R zh=X?x8+m7wLD)Op5Hf5Y?0yqF`v+v6e+RHP(o)?OM#6%#ppPG_BdcH!3|L?(UOx#Q zRWg}2YYTkW7}zL%&=mXN=RiYa z;R5!U6sYj-p7%sa(O^+vj16v&%Qw)lw@kX98v+1^@y+Rhap4a!bvWq#UOr5Kme2+% z7WApv>isw-{mw#VIy|OrMsrk&`QpK(=B5N$=V2J8Slc|H4VbRr&|}tu?X_nO@UM{2 zWmyApEOUzi1uc~PQ>2&bP$EbpCAMWL8NeiWz9Mh5-tHAlN1ycv=vVpQZm!vEtojq# za|4)9+%2VRf#;fluYo6|j+4?Xm@Y)bk!Rq_V-awH-dN;8LLPMWYgC^=k-kzGg25p) zXfj?U3Ns|sgd$0dkqSw6hVz6d#*Mm`1=r{%xyq$lo0@7%ozs&-u#kD=s25(9V1ON) zShyxR7g;GuTM?X?^0NCqRO#Cw6uW=ju%>jwnL{b1cqR(2mjQ=uR9{*XrDaF&XGv4A zaaUfw@+^WJF&jD9G@cyH;R4F%70WDJv|er(hTJGSybj}*$jQVwYCN~_g5$<3Qs|Ix zVIbob%Q_t~-4|V({AtsS^m$NVvwDXm^CWtB4XY3G06rgi&oZEUt zW@rv^&x4wsU4Ej=Mwx1kaxfUgNq)2djG{`m539}r>B|HYGWY?XuBRTAD6YcK)$X&b zBhC+VXgRcJkw;tP##)B&&zVir@G;1x4C!O2{YyaLh6jrxQd82U-mz-Q0r!XYN{3D~ zY;^w{cS=O7q+M}c$?5)SA7Gc)fZr3mcbS2$BzH6O1%$RAl7=m%b6JoRjauy0je`zn~^N7O=C@*?!D=lfYji z(9__dw;HFtHV2KfYR+WH;*}ED!D~fl|MjJ>tasPO;;V~i9xMW#xQM2o-PJ4AA#AY| z=craWQzo4|2vZV=i$I>B#Z7I=N;6wROqG`1*~EhvUfddRc>HoeJ6`iQ`#oFChn1_% zXJlHVFUmE3jWy2q8PGo5V8@vsmIl|AE0Z>^qpsljm{2mvlHibRn5o1gRsYU%2CtdFzh4dw_c-E z`euH*;L2$W5;+$OCUHPoxzs^F_!CbTwC$(A^$vF{K~~&Q3&0nM=!P%5tmW(h;W?zt zy|=oc3*HX%f#Kr?VVVAvXW$M^uO-<=!x|s#jew~pR~mBxXyRue%Z|>x za--6<&S8uX0^Pefnk~Q7t_I*botCD2lcW*eZCwux*`un9dUlE|uhFX`)Rj z6K};d4-`YKD}d0%gZ9yXBJ&T_qklq?9@Qttp$!OQE|?2~Ne{O}pVu6B@E~`vuusS2 zN8Qj*0&pFMtH7T;Kkm3Hu6X)1& zhuMsR3cSv+!vQVO18>CEkZ7nLXZpjFWA*%SZPxVjnSiBs!dt=zjI-X~P`k!qyn;a# z5{78~d10ssl7*PB6dS;JdPe2{v4)Fer%fy8IKKd&0O zz^dqHuamBP5@tLeVK8A_tGa)nwm{J$#O!|THWTs3M2Mkn2w|T>&^||>!j5D9*~B#q z;F&Am6u%0>+|Exu8%_a_!gj^vIT$X~s7b_fme3*cgEKBvjK8_ zDYGmD-AyRCULV>cSP%!TWCPdahHx|}qa9%sCEu+=8|>(Ek9;#UqPdDq`=x@AiZGIb?($K)p7&a5<1sX?PoC`jhK~Bhv5gL_?;V<#>st zmptwosQnnTz#!|r@Hb(KlG*|yY<#;O7>d~c&4XL`lShobO8l%G?O zP}&NjmTLAaBC}dUg8q#umcjujm=WR036STAhzT+z^z@S6^aU z0n#0)u_NXYtzE0~Liu|2qXg)_#E{Y+rF!TAC<#)l2Ai;l%WW^0Tr7JE3*k>B+%Nn` zBp#Z|2z2d{XyCN!w)FhaKeASwfMCB!7xb!v+*m)M*N2%ar&m3NZv4#%5?Ze3@(-lN zOy}2p5&OxD&?YZueF4_fiMdz9@Xf3#?!rSLy1dfhB9_Z7#K_m4Ld0^z?C`~1zs(0x zXDMZKVwg1`Kp@;Rq*$%Hq}f+-4&tzgK!NYI@#-5tc_9~ECK7P~O8*QD0J8v|f%r!& z3cdr|En&FX--$G6_n#Q+!CVfAhrJ(Yj}Uc02tSG6fh2m2HAN>?N$#0d5=-+_MMAy^ zP;e4NW$w0wSBw2*eAV-tK!#QML*_=1^>z(P<_?ln?jPyIaBVgIL>NJR1HQzyWx9*? z7gvZ@Iu3(2fjNCSk|@~!01g(>4E8@5sEcpV{5^(#7eL~1A~;=}Pz^Arg`jSuRUB=T zsl4ZA;(Q$y0r^Yslu_A!R^;oKHgv_=6@!&{0Q$*~IXbN-?*Z%-wwJ=-e1d_uD z$c1T70s}fGpGF63PY_f9MpPH!k`IQ+GIWX;$FB=@YTLs0X1!=71kC+KyPv{6esj4| z$2o?M*n2Dz%#5W_gkPiUWzVdeAhvPdG0B2VEUWqZLSd+00@FYOJ_!qvHQ^PYXct1d zgKtZWuQ;>52js%Ql@iV~nK!*+_MiomKVF|F|{z+8}jF);*pgf85=!{=+%CHCQ>rW@Of+8ys`f`|xz%kNaf%vAeUu58DrxXEjXUXS33 zz8|$J(&)PcIl`NW!4za?^1x>D;F5ZmA-kb9-EXe;@;;CYylu8T=Zp`j~D4H>@y_^t*jUg zgPr`Uw@X<#D*=7=lDAqXOM-8(Nw&i^XML=INfD3VuwB^S;{n3GsN?ojWcq1A;Xym> zfoti*@iMAs60jAtB3aJ=h+FP63)Ysrd~V%8$Pk{=yfU``ZXY+MbHd1k{V7S0dm*() zmG7PPHxsbp#nO|qpEpmQxY;q0XAV8=Y>N}=R=XxoXQlhx(#O^>G-T7J$N=!Zs9nVCk3fEQ4dk*TQU^{La&|+#9IT*H@@U

=q$S+K{g@LN-SSZlAiDB1)-sofdif1uh?svW7^_MEn;dbKpYLrA+A5PSbksF?{Nb!cBP&i7W7`%OiTo7f z`Qj#eH_UL+1NJtAPITcIZ$tHaA!Ycp-> zvlRe3TK)EExF;xo@x84_v=DfW1DTX4+EkG2%iM9jZH1m?$_K?HoxkU7TSUqM-oD|$pTFtUN2m469c^XZDc@M&9{%s*!bhE1%^3X_ zX&pUU)~{CoGLVa}Y+8z7jJ60}PzOLx$-*7s1}rAt42#RSsoS}LkCvKP`~{G%1+shC zogp~(ExX>u7*Q&QbAXJKkblu%O=?sMX)Q*z(U28ogz?ndpTVji@XGwGQs78QFgqlc ztUh8mpnojaTZ)vINj-z2ct#8|^OD{CpUT-XZx0Z+MU?M-6iOP7;vh&O;;>=2=j_P6>+?wKK*YVVCT+YlFDUer0)*03?Q4P?{%w8!%O! zc3?GWvw3Cg^dqq0B#MM^`2YaP2yF7KC8I(H;=?S6COti?=NgU(oi$v26;uE3LB=(V zMyPnRUTl)#l{_en1U${{yC?_q!>tgv?61gs9+6nX`4zE6m?_}ae(5swYTt775M4!S znPC^Jd_`>tN~tk$vsy;oSpOB{pO0bIs8Wqs4Q%lY7#U5wyWQ9GN_v1&O0>NfaVw$V z>(>aFcrrx{r4xJ|NvVF|=K%Z=-8z|vpNTj`B3586^I&^LX8yMRP=tHnBuLd;5ob7d zOeEolF~v|mv^QzMh4+lxCF6ZutODBtLJLZMp^U%J&0s|8+pBp{RyNZ;4OoBkS(B{I z6*2}_kB#4=rN_*RyVU~{9BKoUE!m<V0XESLMx#*rR> z;@MG@fT@4JX=Ml6&JSs4#v$1ij-*u5NHCjDfGq~9tO`gbcsu7O7@^O0w55EZd2DfD(GLvB00Av1MH#<>7g;9%1flW?e{;SG~< zdTADdO~vB_+g=B|#J@@sYncE82z6SQ)kuF%I?HuB=4JdG7^sxz8wZYXWsc53rdvFM z)kj#mqO)Z5Fk)_Y`(=7e{6^$2h98Uj<88~99*_%lsJ(0gnGzm5dkf0=v9#pdEz(upR+$ zAj*31t7(LE1OD4ME|&|NT`ovzdQS>{1?ht(R9e+~iu#UU4txYQgHB1ai-ZR3odg8?pn*DY? zmT23d%#AWNecktLKN;POa9d3aHPG3UUMz`R1Y4eHcUIdp>=@wNyh4k zKh3N{xrkZquG6=lJ-gb4d(PQ_Ug~So{X+Z)=J=oa+J8^pT0B>P7eW6;-}{-Koecd= z_LBntH#P5y4hkP&jVgrn$u5=o=*3?D(4tl;E?1LZDkfS6#g0W&#II@F(=xG9!qs*j z(kW;0N!LU3Oa%41OGrMcpRkC@G=k63(7?0R!otJzboR7l57b;&AI7y_w_6hEx$aaK zv*x)zvd7S8l{-|d2u6dp)5S=UAn|SSicD>gz;(T*8&VIL?s|ZFZ)Vz{2P}=&Ke6|9 zSV~Bp5ceg9JGB3}|8p6Cm|wP{Hg-*y5brrjG}@pfi|TOji`d){E?Asw;7q@i`d()< z%F?VjAB2F=0bk3THN63vxwWc+ahKP;uHOfQ8PHoi9 zKL1DLG9v*Xj+a@^G68XzW@zLU61;y;!1-2D3aS))PF(dtnm>WqF?B*^FiNBj4g3^j zS;-ii5b51|WrQ3P$%~;&wya%87JDSA=zu zb88GUTb9BSHhCI9;9o8l{|S-*j-x0^oDsqA;1T(s;MvyUf^9*PC$j|kS8PE<=yYj* zzjg;ttzG@5hW@?aflB?f#D_@ztI37XQqBYw_V0xqoFAf+-wV&cQXOMis8avgCp8X& z96VKP{d-4dVJM@2FFdZ_-J1Qz<=O+M8iKLXr^>H$L8KPm0pI@Xd#WlqcmQx9pd-Z8 z_|yNTqS^=YLc@vJo0uAtiJF=jy4rku3*up?TI>O113EQiRPfZ$KJ6?lEV$$1F#UPO zg~u7m#=!z(e<=|Wp+Xw{`t=;f&X<~>n0{IRhf>g|d#nA4qg9bsm_)5a_od8fZ!p?U zx6Nr!ymJ0_-7F28a>Jxdy6w2j_loO?=P1YdlB=cr?I`jGu*+T_cOyyuPeO5YyuX@! zle_U=S%5wdd^t)q+1-u+YT3vpi8IE)A+&FkuqQJ!WP&Z-qN9`cyXwU5+)romJg2xJm1h zk#L3RISa%RwxJya5fVd9rNlp%%1N9FmUa@Zw1AO^ruR z2>w3Sbrm@AXEzYc`unEjWA0+(2qBbRxM=9UedD%qx)#c&71nB2aymU312RS(FZ;kVQ22Qro@*(6%-%yI8F97t4u*$1*bm zBS0C9dq4DcH!rkxevm05vP9dS1vS5+?v6-|wF&FeZ*1;fd=BeJ;|o0zY9*zBtH|9U zX)5JGR6~GcjIsJyI0t<1B=xS~1k z92O~-TxPsL=Ef~ndJSjzu_C&S?A9{9ML>b&gsfO3GuY;?4cc|EU<4|fs6g)9U|m4= zmF~6L&fe!QI%KmL=a1XJ2@)3yr6)v?B_V|7V&(bhj=nZ`!KTaHJ65m7O7xkvl%qk( z4X*q)VNwkWj>jm(s`Z5KWfpUHT4s{i`w$%&YL~&RQd= zKDgH}#hDM1Bnj$09_P0z!{sYy66rt2a_S8uS8#9dqb~ImtdKKM!c(}L&A_5Nn)u6U z#vc_Z#>bm3h2+o6W~NxDYjrHh$NRDj6JzMEE} z+%*@)(78pBm;NStZjup0ov|d2-sx&Bt36}X#1G>@5jfP$?v{0!ZjSD5>Z<~#gVfzm z*Y+9*J5!%;_1qtDOxtnJa2U zj9O!?AnF~^*_Mb5L@tKF#n=M$UN=?pYQhiE8D~=2L#qFPpCfkFDFK`&aBMK611NkU z4icHa6~j%61I`^^!FAko_g@N`en-7_sg0#!99ZrK|0#eKwgHIshvB>S26`lt_a?~W&yvs{g``^a#9zbe>V1rLa6p%1sazHs z=E-!UH+G+UG(Fc6zk_+)40EZ;Gvy{B(LAuUApw&F2qu4uYQDni&2Eve8iQXjNrxcM z;yjUQr45_&HV}%34MKsx0ez$rlKoYGg__Uksc?!S5=)29-I)h~U)Wpn)k`KMm_ zPx}7vbEXCNCt3ObC=U!9krn;BbTNr1?a=&276rqk-eBN!rN)c^L#18^@?rt1)onF! z)sXxyDNLj^i8{ph>5CfSW!8uunu;|u0z^g1;gut6#||lMHqH;vBOAYZ1oDOicDRJa zrj({`G6kfwy%R-1ntu%-eyf|DkMmBAPiCLz0zTepe|c&;!flI#1h$qN<)gNsg#A>a z+Cm?*t)~k=kO%QxzvyLFZ=VK)KJ_~`!Gd@Eh#@2!E-EvUJAYEGs%)*@6+tHtNh9=F zFJn3u8=K(bg~~Z}gDbUh{$bv3YIo=skjI~^gKjOgx@n?#mZQ_)Gs(E%#4<9q@1nxVS`V`|E6*nR3t8HP75%%PM%hd<*K8tA z)6eL(UGPSY*CfqO1PXMlqywMjp7#>0jXg|s1fDOkR=!-*syM8_(b~{^29brzau#Z6__8e$QT0+Ux+mq*Gr2d zeocd}(no4j{w&&8`YhXb^eo($%BX5%&8W2aS7}CZ0HLpcL~Z68=Ma1Ga|z?8Wb>^4 zmY?h++*qkW7-YvG#Jgn2-VU}`3@t-;mf_4TsQkyI!`?xn$#BjCKZ|URQ#Lk)_JO5! zV7^U#ap})_>)K&ieuuMMaXL#>ae>;-p~~E$b%?R*fo#^-mHs?v$;E4j z5*k-Q*@Qgt7C*+6)5~6p0gS2RK2iQxP|j%lWqxW7pUmXc8Sm)ZMCx6U#>#%jssn25 zGip1S7f+_KCw>$8J;EH6SWD4x`2rR$PtG6O!-~4?oFf##`UPg}sIgKmu7w#3w;13w zP?0gn&?Qc^FNB|Xj* zqnqfqOyCs`zx!m*JM2VchOg{3na|~v6yH1SOOOCrb}mDRGVh+GhHR$#p12t9r!2Op zICih8DV7Ex8I2)f!w3{Sl@U}@s?Jaur30NBx&)aT`cjEH+=e?ZF$?4^Gv(X630*@~ z*Ho4qR51u|ig|Lc6b9 zN-ee7CMk*4HM|sPrwAo#t^slytE(@J!QpYn_ZI;GV)l{woYWJKO2oDOm()Q47>->( zXO$Gr(hc_xD>3Cwh>za<>Z2~bo1}3_&?d_zq6GQSa+>*MrxNWT`+*rGO*0j|C-a5 z(q;I!jGMXK^@fB%81xKM6qa1n6vg0`;XB+Qiwr#kIlm@HIv#7JHbo9!S`OC&8;(c|w@Kaxg&LMxO{!Tvfl%iT* zErc)*m%~(IdpcA=GYNh@&UsR>#;$25Yc{Q2FvtNKL-g5bS%uW?R9TRya({0%R*reb z&aJKrQK4r@7|%f6I$BnU(t!js?bPrafza7V|$k$LbN+@hxvmZGKw z=`ySt#3Pb`!NHin2*PuAuceX!ouWyd!idom#-ssOU~>aypfF%IGZ-sNVNojHktrV# zlr>BBeQ`_+3Socj`-)eJEgTyG1`CpV_zN@B;{RoDfNgNIwIVO}3yTb|R#ZV2$B1iw~{f+(! z!S9X=;aisQ|HtoxU>rPn;sXJ3O8yV(_6-sv{q_*-19E_$JniX&LD$gter zxVpMJp}pp}v1Whr)O7jsdDO-6gN4|?NBRAr>+RC%vh6760C1U^`@`pXo<)>35m4HY z@9aMr4B&Tih>sm~sGpZEf_g3si|nw#B_FU5fb@)tO&jv^#0%xst+?|bRM-yZZ;zuq z;a! z>?#Z{NY@WfMVjto)!a>2^4A=Z^LPD9m98J3g6~+LiuTDzqjk|gDaFwzzf+f~);eP2 z*Y3AA#igI^nC!zGkzb6g;%A`8EwV2}L)I!*J7Onnah~tPzulx3OLZA4YSlcF;~w&D z1f)druG3Eip-CY=c|4<(m-sY>cn7+EH<zF-u9d=gZbgji4bVuDyRSsQw7a#daGu6sK=klWNUuX%QTc?rjpPw4^n6zX5-l z*D)gN;z~^J9oTsm^&1SzDvKW)nKqEmqs!4{ZJtil=->m~dc&;MRN3ZnWyd(i7B(>? z7Ux^tP`X$zP$R7_KD|oXIHSNXrs90tRKp8-H9fR!q+QnBF>^EAGtzR1;&elf8yfzI zv7qZ?tlRw2!nu^!rCb|i`*=-P2?9`A$x`r`78%k=&xEuM^>@T#`Yjv#SNgDekBYN|FMJNV@-gdl$|wOMAK zNFY1m-=e!&#NTL9*NB1@TScGuS3KP_+Ro3I4Bxm@;1X-2-h%`u-?{g#niaqe{O|m1 z<#FpB`Kr6uTns0+!pyUplgD!-f)wMa^g~6t!G#=C)r)n2teUFk8IH54GsIXM<%3!o zJ~r{$!;8MEh6w#CuAi2SXC}9U=O9s``IKB2n@5uj?Hm;vyh-nHvR2I{dK~R5u9ZzJ zw{1LwujgEiZ9#uGZqY{5?i|2RvASl=HosHSS-OmdwwPi%iL%)y!FcvpPMnu?5>n14 z%Kc@yxWlvwl5R#6;XqPsYGXs#ttITAqXr{}DZEc7n?^f0|T(Rzw5x}=;ijxdG*7{^dr2Qpn z>R^zTZ;%D{iy!j#FGQ}V(37izo$w|_nLOeC+&45j%awG;`{#1oamepFQj!mL@wv6t z&EYdiCTWhaK0XZ~4Sr=OG1Tf#ShAzMI80{uq1I%6Ob( zC|48oRX)EsipGvP700pTHj2_PGj#H;KuJ``j#P8BjjpG>o79ryVY23~3~5~Jf_M&U z3Y^(t2B`rNYQzBC{iWbbH4+rd&JV*7Hys6~0{TCVY_R+vT_ioh4S?#6bS7-s$BJQb ze1cJa50d*()JU<1H3z0cCJmt^6vt|QTz+ookP_ALB#^RIwk{R{OOy5Su0*_QCs#J~ z%QX(G4AaSr_-=#5L?lDs1ohi-1^V3K>m2) zut15H2ed2UtURlk9>IcBo=o)I&KL2`Q+vE`yKp-rT@AE5Lu{ADdbShG1q%aCUn0Q* zCBVTqMZ5_M*nXT>7CiKdigvgBesu%+$p;Ug;JJVs*4KdVq5;HPgJ8lVe+a>0MBGm( zz{DOPJ&y6A7($SuWz0MOk0RD5dcZ;fZ>d+l0MnVi&ujsGvpU2h=`dV!7H)SwB9}eg&Cz~MXwAF1>^X6XNohuJud(ZKf zGY~)y|DAU0nUTNxwyE@^9;pidjnHRw8{POU`^mQ@&S1y>YF;fTh+62M-Umr&hVSO< zb*n@q0d<2Ou)KEzD=U~VFDMlIE$nz|?!f`G@Wvht7#9Q~l?A43#5Csi=G->Ey%hO3 z=JHGl@1h(37=@#$14q6g%WBcVY z-i%-uvp;NIFtJfAno+th8}A>a%k3NI93urCroV_WoJWTpzmHyLp*o}%6h7!e`~%3u zJ*}#IhW#7b+l5I^9dietcx*WA{b!o>p{VD|LeGbc*&}!Q>H1PASAGIXdBo%Wsx`v) zhLr&IT#}-iX~%D7Y!H!pn%~JHKjoV*XNgxQS5VZjW3Eletr6Xs6;VRYxLef^lmcLq zvHz{#VDc9ASu40~H__h+hp%#j_6P7db5CsL9UjECm`1=X;?TTlw`6j67wO0FGNT?Z zCViQP5S#D%upy=qSFRgHgk@>=i|VCmzGl8vKo!4HQfG&MekEOIfPDJ*c6E0yH^u{x zMA!f}p1^(M8$+>xr&c??tF>OjFSlb%I(NQtg!rn}`tbMSx`g)>_pUAb;(W~Z>gNUN|;e5pH85OGM zek!v@e$}d<1#p}>va}W6HZy?BP*b~S(wk!P!72%;Umq=UWB_b2r_s1C?uayx`D#LJ zX#t_svl7(iud+)OCO+HEjG28Ibw?UXliOg@?@VDVd;R%VdH4x8QA0j~kgk}c`03#| zJTTAAs$PW)mtY*!+R>C_F1|>-6VHPF#F{nHX$1P`;hcoxURE(5Tr~ifE5(wj+j@O< z4$qP}Ts*6DTz1js(m;NGVh&vYgab$4uCK{+xnGm+ZJE|#QgN=Quo^f8Z_#op{;c?j za+8c(#&l}j{dU|RSVp=Z}(o?@l+ z>*>(&W7+LQf;!PTEB!8qOB-b_TXq;h+gDZeo)c`4YukBU^#hJ~sIw?} zFU@?i+-R{&_#A<(u(k~yryY&3EL&;_#tXUJYT-d7yU~g=19;pCG0BBiI_1Hp6-;a) ziHj1Vz6iC8GWC;6R>pgTRB?mXZq8Ex#ZKaOAwfcjUJEhhm@5Egl*nH4O}bk>nB>qp zg^*-ygkezFH$2pKkJ+Z+dQsRaemngCijw_r{ze^6c#Nv>g$LgObNEE zRyAZaBbZ)9WmAFkS4zT`aiZNY1-=(X78&JVN`I^@!m%y*D2p{b`$f8DE14^aPUzy$ z_Zf^Q*gJ*?*-5}la(VZG^nF@-j-+Ceqf;QzvlyE2x{QHMv5C#BN$;@1Z!u}DjJ{A0 zoul$9PbC__QrW2ny}owRlAUxwmP3=ppGJ%K$v%;t;3G>MnL^&^pxH7fUjOF|+ww5| z50*|)N0tYZnB*RLY{0LWZMJ7;8YncL{Ia4Jgl05O`~pCd<IDtHSD=k>m8CP z))7P~+X%)2S|{qo*rTHjMSQvbZ?1W#!ULnWkq@|NPfU*+Uo)tW{ArV2^QOKrjT4xs zxW2J4xo)1H&OYi*QP)glt)x&bmTIq81c>bqlU*I+hAc#qug=TN`6fULD@g5<1VgZ<9{o!Il_MBY89AslsI5dv$^5Il4WOEdvd-k7qLR!F;aogmtduT>Dr+{t_My6prveiD+eSULkb!!XP_|&n~Oh zge-s>_9JXn=%9^pYX+58;kye0OiJ0V=pijZxlFjvdHJI2bDOUbN(L^}Rp~=z4~an? z_0HtbQALW36ym<#AFDKN!XX;d%4nDDpE_>Xqon$m@|t2rtFUb;wvNhXqz_0}J9QfUHLpFY1;2nowY_}}@9P$yWN8GSXu%G=4n*V0*@KJz&fZ-kH z<4l^NEPy~1(woIUwR~aGkAi>bbaB-SntyES47WQD!8IlCRNp`iRIaQirx zpPoJZ#$KEFijo0Vw`w|rwR%{fs{u*s$@kl{XCV6o+Sv19$wO)BN#Lp45SX4*yCPO{ z{=G3|Gy3^!f5JBo08j;4DV< z?!LjOo{NB_4!%=NUU+_!j46M!U=0e@7fKqJzDg-=1Bzl0&#aKFGjs#SR2dY+2C<)#!mym zHZHb+#%-xvScH)SV;Pi~K>={sl+}kuThcLj1i}*L36F#lJaUDm7VsY7hau;ilnmt< zv7$6%LvO@@*-HdHR*y&_WxeH?kVzRr$~vf{a=8C2{z?ivAK^RPNBLy=oe1xRRbz;z z$UxSb1dgfu%Fr^+Wzo(t^AXNtI(O$r@M7!dVcWV5gm9jQ{69ga02rR!t1q_l4v^l- z8o%x0j7x-b$a6dRUSW?Wj2-i^Q1nUjSv(@n_#pe3G8C^!rub59zBrqXlr&I`=3B7N z2ud1keX#GLU%z(;nJ>K_=8d(YYfI7-b;Ie~5Of{zWpohKQLVYvuP#@BJ+vUH(&mNY zfO+Ei(o`BtlVsZR0ch_0jgqMD_yx`-r*0E5!Fz|^UQ(*mJN-6;mnNzkst8Ig7i z)7J^Qo|e(;{N`Oytso?B+6Q9tVz$vuI1Hquf0B=w-q0c*)ZXv~LcMTOow;)d(Am<< z7yW9R-33DCNhCOD+J>p3n$|E5O>|?T?U@ z8z7^=D?$}|MQY=tAJ)cAZz{C<(eKs`5q3BIu;|2^rQ~du=Hr>gEnn03Xx!qv0=tSe5 zqL@22GC~qn%toTRqzbAH1>G`U`^@VS&GQeLGTu}&e5gVS0`~J>h)$AC)$u#x<%sL^ z$01MaKR{b$V?RP90P zV(V6sSmcjA@d)#1<8*hn*vksM289qNSpQI=*IgCo-*_a6@X;>Q&76FTJGXKz^7O?_ z%*mJebBgq}hRtYY4j}!y7{$I5=>y9SwxZ;%iZst?qWpN1#Pa9mKF6MPq7z1hwurH8 zd5mB;0NRQu%>g70OYVO>*}MO_-ZLkWN}m%-AKlXy;>%^w>$caP$fhtSdfgU4qRG5i zzVJqCIZo*Mnaoi9pfxXtC{gob9`s56tcLmxZk-8t0=?JCu5c0`p}`wokjY{RYhonT zJP+bKRwsSVVS>>>reNr4;XHyefM#mo64wFP=y1h`;AerpEFwImmW!qT{7wncQ{t{vf7Z|&mleicR7dAxW1|)_b2ED5y1r$$0gnE{H&c+*1 zJ7gTvp~ruE%krc*c6n@VJ&kDuI*geZygC#cfY-z!@Ot!t&sfa)3iFQmP3GU9Z-QwEFW_*{;K$0< zk*0WX**`1K2YTcB>Qj4ir8mUsgnt770&w2JG}jF^q0ch(rgGjXl=j3cL7N!$Er>I= zh@1j|Igg6v=MFf*Jr8aW-b;zp+h^@3rO}1@7&Ulw$59w|iVPDmYaLP=4eF?DMsbvy zs)5;Q!3v&jS(Az=w1z8ef;jXlEhJPRDbF}38Du)NiIqZmg(n4ts=!sU0#yLva??~W zOO``p$erj&c)uvAv0DE@q4~rZM{Pm6MPjc>#y8W@FP^u}IBC|r;{GCwmc){r1jQr6 z`Bfl60)hM)7vR_lVUK1OwkH}c(QcN3WW^&y3r>OND>8l{%crvqM6!j)(6_tiWg1p0zX!2ap+7UGE(wXWd|ViUOzO0Oc=oKO6q??#askj zoIw6gClveKmejlxb%VcXD+u7d@Vp;t&hCRV_01tTkvx}`s>wT|Ofd#L3RR=`vlrbG z5S1*Z4*Culm#VlTE-jfZ69sI35jJlBBPR2vIN>%k%@ci;MBgm6DAi=*W9>nC=}XuG zMG}Jm=UL8t(T(y~0xCL zQ$YS^ZYj6fD2`q}U#JoSXTCeW&}kuyS?ICX62n>1W3FenJ<-=7|o1Y}{k zy)0f7u;MbKgXI8;&lXgI>@|aw4h0CSf5TPc*m0&nqd`^P!&cr~DlJoazhvDM zD$iC1QdH^;+M@m3_+ixP_cxs)Y=0>ot%J&u&ThUo-8aV+5mw54>KX;_4y=9>G<`#< zJyG(6{sX`rzQX#){DZOk792nSmWcfXG^hPj;QvH`+t=^R+6`X(m+&kCAY4p1wZ`fB z&3*Qf6>YG(^_ZWPzBB(>Z5xWWB;!?tvA3?1aB6Gn70HpAZfk5%9D9gCKV0K=Gwd6$ zF-dLDXILq}N81wVN5A1&@WsLBdHBYCMb@jxs|~>3wq-gaIHECO0B3sEJo{%P63Wzg zioQTowAmmitTV!tsZVp8LWH|rIL+2ikFh`l#KV@T5lGb&+x)yPWt;!AnP2-TH8JG= z*ZS!4UJc%8I?+eGQT4Y10s&JKQ# zlR(@4SI|a3+y-xHDRl9`a!ox(-H}S`(yDnb{N<2^?}A^%Zi_gt6JrNmeOVqIG#;@E zDG-?|m5G#Rfks0+9%;yiX$Q%YAP&nuuj+v0k}jbuE6^@;8wZoAhI zIId|tQG?(=`2JkcOFmnRLc*|1{bd6=4o=L8W%JuZcC}(=%m9q75o(;1aFy9_+o-R; z!PiKCoIPXRP_8t=8a#HsY|-q>Evs^-V0k06*&th9Uk9DMrRfaD9^q>4PZ>bb_Th}d zUSPW}R~FIsbpCJygr}OmsRM4|bP%mbx!|o1LfBwr;<5Mt}A$ zw~2pCL!E{h)2OKc=HOesBmk|Wx$1Sbtm`I&U_t2pD-xF|&rZ2E)oGt>s$+19V!)pL z{^a6vF882gMU($HrMkWUO1g5niF_}0pCX}_tL883r;312{v$?0M~H@jy1vkhY_X6W z9bRl5$v=Bmq*cu0E0Ao8!{`Mm?F(LwsiLN-kM*k!7WUWR>*|K|m71Qt*v#$3Ac{PoTPrf%9 zS*)?#9yg*Wb6MkkQo3!(eTTBe<{IsC^|CF&3h)NvVl-Nwb@Q4v;1LvvDN1k0UD0pp z;)=s(UzgRtgIUS%)BujbtaRl<4{rIz+HsvGJ#)$DtkA}??Cbg!J(I7r5~v{hgj5hr zM9Ej0@jF$b0C|jkl2mRb37^n{s;04`u7x7r+D|6z<6FIvJl`5{4*0hIC$$3m@`jDL zYVgqtL@SO{aD)5$)$e_w)FgC-mLH`vB|LLMcftZOLGt_p8I{7>;74ojNeQ8l|9Y@#>(F|Nveiuc;hkAMHcP5p02_y6{K+s3lV zmi`+o74x@>Mh^`L$bc!Nwa4%F*5N??@a><*R(zL+S(r2XYXkzV-bbaT(@qwFRxsCbZz)l zw=C;i)o$3=uuC-Uz5pC|;D*SE;jg|q`fmIF?fQT1cE28fp8^>n$4ywJ%>$A^nwTST zxzTGfa^#9r(6r(zsDB&bC@z{$?+GMl(>d5}nYkvDnmVwY7ze5|Cp!qYJ>@|`X<$8OhmJ|Kp#gi`yYesiMOU{y}GoJ|J;dbwXolkW-Kd4sSFVpJ- z!x?R+mjhGUE;g#_{n3o(bO1P~o{Kizr?q6NFXgK|u*f`*8VW~z*l@wtBwMUtRozn) za2#7wdu6UoY2d-uG>%2LHLsQp)O2#E$qThLb1Utak!vyGf3b93G#Zhh<TAY-}C`sbv1zT*z?(0>rd#Lv^{+|199MRX^QmR?glhua-W;smOo^T zMOC7=X*EnmZEHNxH_PkPRXHpf%>%mAJZGS!3!RkQUx>JHx&vYQa1anR+`7I9@^Pxq;JlXpL|r?z zdn%9MIOvY7({wIhB2<1e1(mY@MW}*`n!3>pW*6|JDhWfnh(oB>fAVk-SD&?Ms`G7OUArPces-I&2# zD4E}uO;6J8(GT!!XOwb#EBCNnZ=UN4O0awkTM|$#nAt3jN``_xkh*;zvYESiR%e6k z2`u2`qxv(yoJWnHBcM`~b)iqtNQbFhE5DuXe}G`kyjlEQ84!PQgJ%a@X;ZU-do5Al zAsz5#BVRB}XaK;6xm%eLPF72+U%j$@G8oMB9dr>;cs$=xs=>J+3obwV87LXdLM;*UY*Aqmla+!^ zw&?3jt>%dJ;9)FkCw-KSObyal^H0pK@v;*i~l^ZEQq+tVs8^^?a@&3lHmDFgL{K6smnR8ltigw$J1a&GJuwg+!?#TPz zggSE+skxR`mSc#zxHjz?i=D~Ezj8!5lLQ-poua#A>_DJMLr>HO^d}2TbGs(xK3Usbx z$`NI|80cUlBlKos{LMVj?bB^r$cO=kq8Z4l6*g&(OI0T*i4Vy=-wKV&#C2P}28a{Q z?7c9pDeWr{+PD)Q3uVi=l^9Y~-Zn4UrZ#Ku=gicaJx*#@Go^ylH9{PNb^!Q`GOXsPx>Khb$!uOsV9OK2^(=x?BFex5I7v)ckdHn6}etgsbu*?tG`|+*~Ad0t^?g2&i6@cXwo; zKXlWwak$y-Ht>*Au_D9VW}bj~rotK&Too%;k%5w2b$_&0H3q{dnYec9h+--sKkan) z_s2F}8QTKJwTjwj7hv9hS}WZNfuvmVUom@QdZ!)3yqQWVBrq>F={RaexO|f{Lo5ch zd+1)JAiF`n-an8pgVMcauUHVxL3Jo{)iRdv~6daF83lUU|Q{Qz%IMP!rV5qP{az7H$QT(k(@|b|C029=bEk&Y(+s zh)=hAARv&C+6#9_0rAQjN*rbjbhktdNnOVMNL5&$39F!_sc56t|D_($YiRq~sm%tl z!*^-CsA@$)c;fnbej31P_3}Oft<16MyVV1mS;~vnjVhqDN4TGl)(v};c83eI6TGW- ztJ~d;dP7e6W-Frt-gdYB*NR*VAFdO!15eF~cXOWOH8kjD{|lyD$|X2^oge^qmim^) zzF*!Bv2<*-dldhI;xCLq|BK-uBq*|E<;kg{&Gkb zc2AA5(yj7mYs{{)6XcGZ`#$nkS`=_$D?H5gFtG!HD3t(U|AePXZR0DD>Bh#|4K#OFppOd;pcWW>+W|t@coHy^{9JB1Nnr_6jvR$}GW821f%a)(32)T6o7bvfw$KZz z)LTb2Jh~)T1t7I3ih9LrxUdHju)xaGgG;VAb!O>oR%Y8=An}wHugHajX3)lbF(5UO z(!}eQmjh_>%feZxx2}j=PDe3VvxDK-2dywd`{jiU{$ zi0`nuvWr?hr}jcjf;41N{GVx@*gH1#mf>-2R{-reHo6yC92^>)j&|-fl(xEFaHWdx zczS%`l&g*UomCYzwWX1_D=p2{_QaeyAT-YD0LBZr*RQPbi@obCn9_}>koe1q(#J5d zG-o{KnDHFW)JZhr*!KM!TH%@=OwrDRF@qX7@?gm}KE0SMCm z)x?uy4mN{_c%?~nxXC3+6LafWeGh4;r;$|WY~?;7NaF-j;KT*Veu-1M}ulxqqiDzx}e z!ai2`bR2&|sx1NdCX<<@w6@Rz5Wo*H=`K7WQZP`@4;$t#-Dqrgz|W5m?-iJCWEYkN zKd6Exc$!8uK|RPQ8H;(qLiohfEr_w0wOjLS4VF9B9I{{IWD6SOLDig(kms}EoO**D z+6RU>^6V`kmKD0hq0!@Gj(_=v@h?v-D5kOo&eybHF z*EF^dtz#+E={7>fJEyHC0MgL~Kb)l?Kw8C(HDp)4%twuMcSg-3H&d1ucC+whX;ppZ zt2pGKGoND`FZ5joSs zT1_#!eIt0c_>7Dd9Fb)vYXf=jbEWe;`~>qgvQsb&=Ge?*PajX#OdE-ODEZTcW* ztubOUJGbHLv2u~g0eC<+TwJ`<3dEZ%A@apaJ6fYK?h86`g({SdG9wicbJ>&biL)Ll z>53>`nF%ktA@q@9u*EKz*EBQXa@N2TU04dqG6ga&B0XvlH7eY%r7~yf7}ywmm^z_; zWWU2%h;`G}e-Up2Z&&8elV*Dav^fU|=2}Xd|2l{Loxbml0$A9Z9giXZb}RB-CUuJ# z(2z?&xSQ{XjWw%8H=Au8uT{;k^<8HOrrm#BP#I6EBvRA?4qeDJQ>HD{(?LIcM2A>L zzx5Y(CqU~*q~ewigS4StJu~iy@q(H(e=V$C0zhbF)v5{q$PI~Uk<$jLc>vvaxW@ET z7nSGULeU*a0kRLBdVw@H4~PdwP#(&RwNT&*abPWlJDbC_oCYQq){o#~>(y+cq;VS> z9>51clDR?UC^-|+2`Ffjf0zF&Djy>U=q^swlw;#MK};ord@DqKkgZcepVN__<4Gmz zM8t3eYDCLm^K1}1V=O{1L_=&Hmw>@vbLGfLA2cg-0dkY1^;(lQpijJ^jX=7=7%#)> zgM}Kh=~f1jkh8u)N?eF=LhJYpM;|gxoglJPYOE2}Qa95wdm*P@2BJ`7evfKH@wjT4VJrhN&Rzf@l}jR~URgD{5rlo3Rj-?2ZtjN!_*B(MYW z5dIOv0TB1mFAPHs&jkM5YDpdjc@@5AW?#oA-JBuO4k=Y&Px2pZfgfxQ)yKW7ofBIf z%=9E&VFk02!?e=AzH?QRAuYG6lPZaRV3R7@Ru$jL5mc{WxIR8LS@D3_ayhV`U8M}T z2jihYrzEahU2G_H7;ILY<{8mPZ48nuphyvn1JczpNbLhrDquBa4Xld3B_<#}rTEdp zziva=fP71Sc)#>MoyMoAS1!Ba@n<34)4> z?9F5?p0kv#u!>B?7Di*^GKc2SO7F~6prz7-)jNY+##4{JG?kmXAuOaZpda(qU}W z#jdH=jpeyf?%Pp>Ae=%qi}@=I`4CQ=00N|v2_oj@?NZxC$KG*4uXzxN3sQB~(X4bk z)QE=cU0H00wRQR8!YAwqiLVJ_2bep$LA)S$mYvXfMeo)3j0NhpW!92OFd7|3(g|m@%oA08HnH z_T(;hSlfAWeJF_J4=eSZ(2q$XIV+muAo|%ruWw?sI)nD_p~ENGKu)<6U~atvTOI9{KNgYGi$w}rkP6{B^F#IBOb`Qn^U`AeS8rd?k9P} z3rPj00r^xP!XlG9a&nn(5h?gw03LG`K%t^kCO5G;BZ@gAd4b$P{3lNi4ZcFwujIGm z<4B+$x%rcFX!JK@6AXGuvpdT%ly6B%qxqzS5zUAZO;nhFholX!;vyJ^q!@;Z{Un2g zdw@r?ZF&9=NrCW!694)D%91%L?4q9!U|kid;t!j~Zj^I{UvRD8E9s{+ppKATt@}Rm z+KdcBSN=$LARqi$(fzOHnRYQz6n|=dWysE?zfJQ=i*s}84)J<7?D~Rl3h7S#1S~iN zJva|N-+Oe#pmHUQ$?qbo9Q&2X_m~xg!P8DV$9dUbZ#nf%NLN2m_crzz@Paaw7;Xxt5JC<{(}fEkm#t z{(<;MM1B>Rl&h#Pf4CjF>`%wgkTIQeWE(?}akh3-3F(85DE_HL{VFAXlh&H!s$1cO@96Sy~-N1CVg09#7?!Gr!w2 zx78TPEFbPl3x^zer6c$;iUx0{$?`z1=8X&#c_7af6NvpqM7~(j7xGoEzhKqX@(0!z zgssvA!X3_$LP{C;7?O^I$^JQ`8Z%4Z22Fr5X3^ccCd1VwyOE}`D5N=MqzrI8gc-9s z4*xc1&=Lr(GA}We2I%1b@lb^4C$Weg+s`qpPck8?Z5PYH5K61@1Y_8#9gDEsFZs<2 zvpevG?5MDNz`Z%~6am;t&sXR$P(!rFL0w*{syH6=SA$946kM_;@ zpJL_z)*=6Iv2x^4uc{CfsO@%`4EBF?p#!GKKEa^=qdL(rhfSVtT#v60(A3pL)x_GS z41o(tYq#1i z3$)liNw$3TPMEO(2eoSux}0RY_BOo;oc?{7ncV&N`Of=G^_3%r>so^(%vb#t?uFcs zKmrM|{S?-LKtpIh4?92^&>63K2IWP9VZu;#-(#` zBn}<_X6m54XaPw7I935H-r zE1P9wTLH|P-3mVJ9ypnjxw7>_BRa7*6y;z>FJgzIuark6Eh;?=Aa|V6J!xbyWs+f^ zJ>!H^Rk!2kmt=YY!`VoclW4Nrma~L-f-5fr6Ty6Hb}&TAxbms$PJ-@DtggY8UY4uU zo?2M&a5;l+5v9$_YLUMbc}Wp+>X%FW3Df*Fv6h&YY?m0e$iE56dqz`C&Ofg`Ag;Xm zN#|H97#Xm28VY`xleuNHRiU(mdKExdeht``e9i}vWF@h%MT?cPv2#0a%9baswwvQ$ z*e`7Ma&p=KRsu&zHxM9CSgQTH2El(Qsrem0nnG@w8mF#7RxRU2gQQ?Ph3Y-q?CyRu z#6B}}dWA5@5n^w<>?&JgtJ&q2%sHt}KIOI_qv5pLf%5hNKpN&s)>dC;AvV~!xkCf(u; zlE(gOml3adP4PYp6Po4Sa)maWdJRkGzyGHW8TbbuU_r2Cfqw7sL(g}ntEV`oOfxjw zXz7fMMWa8yTcHr{_9F7vMqlPP^&^ja`ej1eJU#ufcI0GE?#v#TKi1)6DMDc$zPR!) z`F%d#ZA$Dx@1*r#8&I1mJGSSm(T7N9#b>-J^OAU6X=nVs;CHMbml2Pj&>g4jlVB@K zPq;yzfHtLXiunhW!qXI;FtU~>K6y1FG8$w|W49<86!0RaJtq`PAW9SO!N{F4?WCc3 zNG?gPQ)&l^JwrSZrNM9%2B~(ceNv$!>~yDf^tWFw*o8BRs~KLv*6@&8TK*##G`nQi zy+T~Cx8#G7@`^_6{KPm;N^D8}6p}eP!OkPO08f_@;fwAA=kt<;WS$Ij{CTlka{l4Y98HmLDv$SOK|Ij~lH1hGoc*?xJ)* z{8e6< zM?M5kum((pTn7rArbvp`#fsB7WnsTSkLinK#1}+EJPFBYg#X|e-cv?#d)OuTNimwr zx@#sYgDc;dpBd+gJ!OLKoBcr^$+h(XpjUM31sNUD@6$P6@cOWPTad;o=c0|)2yD<< zmK0WNc2geaJES0n7YQGK1$(q-1OfiGyO!SlyFj=_H75!47|zi;Ux)NX zGcQQ~ZYlO}nc%VI%MD!6{ag`=Kbs=u%sX@_TLGETj?|B-5D#vqtge&yqnWphW&Qy& zu6xiI*B4052WY5W9c1s!07ef;Ms1IyOz#)#AE>wT-edG4?~m`G{s(-a>$j;tZ;HG_ zo>D+aWy&&`xr4C7fg#iVm~wFl zOd$Vl8gP++gZ{TUHwvWz4F8`O99#1R1%5^=qtZ4I2-5poK#RUV zk30tWzx`~xKsZ4W{^PgB(CqgM6bMKe?!P;5|K*uE0EL;Z^ui0CZl8q!1+Z6L)IuBk z*H((Sq=JlWKCeHAE?Wsf=!7{1V_(5Q`>KRAij7YRkBLKnp0UL>#Va`1t7Vz;=XDUV zp3r99bx|%lJ7ogP@Np~I_vS74L9nOe?R0n65NK;y2EuvTGGWLM#`9RzIbk{|=S@D# zcY#i(iA+;=A*HlJswqc=9T55fN7{0+Xu13pt7%&bNt>#9e2S|Mp<#hl*(|4PRkpU2 zg%D3jK+tO`rMq>q!rLqFyMmnr8)hBh3wQT9O)(@?+A@Q!P>VwWIQqNX(Nega&Bofe zvCwi-W`TXbo{fhDKVP&%K>K3hwlQUuk=Nomj!!@V2o@HKk=__trn`uGbgqU=RAzBq8Gl*V0bqzqOonHJ%d{Mh|E`5zZMV1W>c%RZ;2&<|>=^qUkjx=?ssY9Q z{d?0G4J;I2x0y7iOy}?s#98$;HO&;JSKb)&ftW-8y|N^ri{+AjY~jscbcacGOdosI zyi_~JIH~X+zqWvmVDE)a!kHp^N~tt+MkFyW=yCzABr^?G6Htst7y}yF6*-hBL?%KTlX+$R zgt+8|G=L-`1R(EI4<9sQJxv{2Wj~y<{{o4%cERj=eZj|kV*(|f&buCY=Hg1A>7#ly zGEO)m&KP`Q0fsUuN%nmalze@3fLs$l()X|2Ju|4%zd5cm*Yf_> z-FLQor!8^oe&CyEZK!`|k@y)&)_~vIyeKv$|1QLhAHc7o)Tyh4<2~IDP8A4V`c*Ga zV?|p}+m4)Oq?^ zgyeG8b1K4>dkJHF8U}9MS20A8<40p?3BpZO+l?uD*C1OKt`M#2#g~54LtFMFsX&my zmtcSEBW>>r`1-H;>Hqy%|Np45`kzp7T7UoshVUQLTObb$Ud37lpikCy8S~1tQg&4F)3+xEao&@D5|CG(6}+ztAqY z)1uY&*ecsMAUFVa2NF##qXWT^9j#%}hiY~W!VtAb5>-*QHMiaHqo%lRtLsku1hXw3 z=$u;n>bOTH*-?}wgJgV((aCmREDlFAiqgWE4k`^p5R3KH=-X%e*E4)o$!%*%K?Z!O z-rJM%gq&SoQTevO1ZO8Fp6yUBhEgL;H@l|6Fk7Rz_NM@U&)cP3n!Gfahs@}=_UPyv zmxkQvo4b%j?{HQ%K34R!yK3|HXy4x1uKbaiE1JAq7YQ|CKShJf*r zH$IQRD9kbu*6)bpM^gp{l{*uGA#$2;GV~3v4-Si>Sk4|c^o@>kd`459bXRBa8&`5x z06J%#!U(|Nuk48J?S@)6Lfz>B1TN9exT=H7oyGUgV6?;W0f)nP3iWgO!If7y{LOHe zTK6UdaCbOocY5@P*UA@-fKf2hwZaJG9WnaLnh`Ij!x-?6wbDqqeLbwEj&np`yi-GZ zbpG7{!7+Dy#VeQNQwaQ@QA%~cNr=<^#jOWdTquBFm^J^@h^yXbP_&`T9Uo?&34`I@ zMzo%9q%EWCoeT`~M`cvq`ku_z41b;LY@PaNEc|-+ZsE=N1^ zZnH3@>nA1dH{$nt6ky*Y8+}*h&OSuCrB`VD@}_$G;QoWZH!ROdac=PpK9oN|5lPrp z#Su`yatag5M;ubkPCsLtZJUkDxk6~D+h2rPjWpPP?a}wV|V>7s1$1-(OT-_i!Wj zGFRSx=906OhjWRvAlYIC;|G zi#`t_-ZPsrz{8; zR8@ad@0j1XF5W9jj@C02RDUP#9_P*ZeJD>sF(E@1aegmKXvSmpPgp@!G(r@v)QYe^ z&~->+Hv$WIymS42m=apDRJa4pad8&hd~*ojfc=73XZN>|Q!eH2AL7}FmU^(lW-GZn zoM;S(j246eHQ8XdG>N+1#ihlT2~>dI<;GEQUbO_q3XkU8lzs6c+Rgzv_l$x@DD9K7 ziQYEG4_DonB$k;dLKjbKt%tkpdy1%-m#{3AUkIg?UITz>5T1S?Dw&V-4%RCDZ^C>7&%mWNZgB|fwa109cSzCd z9D{VWq8rO5;Y%%9Jn_LOgt@5$l3@O`63-wn0_Jtt8!r7BjJpdJ&b}8-Z`FYz^bPVF z37I%nWC4;f0SGr#s}%!T&NF8ta&XMz|0C-ggENV`bu+PT+qP}nwmot3W@0;eW7~Ez zv2EKEJDE&!bIy0F&b@W3c2#%vpZ#O6)z!To-CRJc(dQRYO@sNqzH+}O-qryQe487n zM9Wu}7dt3o>Mu9JlUwwHeqG`3k#o3(S5Xb^n=Rq+&xFOzrSn&aO_N>D9V48ZFFW@9 zQszGKGd1P&rkECZ`$FbVHl3{1({xkvQp+E`F;r+6U6u)1M)WsY;FOTjDjMCz*$GO( z#D&zqlyp}FXQ%B|sy#;t^&tiDbO>U@E9-9GGKBcNqD$k@Z=oewHH4T8bbuiLPVZJa z)$n;mX}ZAA%HHCRTW&DmGlA(CuymSJ!<(#f4<#AB3)3|pS}9UlJ3Of#fXNI@y6xmy zm*yYbeiLBb<&lEQS3{MQlDWvN3iemAioPrw>+lQ-|! z^^)hRWmj;^M(fPnPvY_+$@<6gk$+vd!!5D6lA?}G(*^|AKm0ziHv2=G1Egfu;H6A# zKW`3>jb%X36!d+z4}k-Wgl!G(-2zxfA_$r~(g!6V-c}0{^IX`JqrA3OzP;Z)yl}hW zPu(Q%HAhS`O%Eetr272n9bakZ=Ouwu2v^)9!DTp2sRSAJ>|P+(i^g17NCW=RY{~kw zM}kXxBx^lG`Y^x!wg2a=+A zI-tGgfW|(q3hhkz9xwj7NcS`&NI6zDa~sp{u@z_B zWTHjk!U+9A&_)=OQ&Z5-VB#wjab^UZirntfE5yAqNp4|WP^Tn`{C;ApKp-54MTc44#ec(#Bg=lN(*G@%QI8s>{ z%L3L#0_+4oQWNiA_UDPV^`qV8?U`)xuUSxqOFVrBBJ1eEv$GT7%JN>PnW6^MvwS$6 zC_CtPah~aK*5&7zPm#rnaTWrznC`a3u;TFvw&GoZiTu3D6Lc=G@s7V6@{o0PGxbO# zx<_stZ=XWWeDrnLC-6?z0(qwKDifKg<^h`;bB7P@+B@Xg2r}kLubZS-r}ny9i)Ynr zgjX`_iP5GXV!7L|7)7Otf*xW zMoykSs`@w5(i2#9m>6>FC;z;(>-p~>@~)6)0j*jmU@~3ZpE(&x78w?$jiL@?pH76<;2E}~bAKGt7 z|64dg1-GCO!4Y~Y_7SsN8=}!fi&Pho$2P-&8CAMZOuF_lF_SXO>TpjKy!S^xPj};(9o6)fR*khU;^-xk`djVaL-^^U6@yA6xV3e>t0=)@W2oh-_#Ha zj@*tfrE=@cH24(V*^3T5N@pMD$&HOFLQ5=e7%h=)eIaxFQq2FN6UjZi*#O5 zZDwnAsWQ~{PhT#J>AVcJjsdqxH|At+9}$dT)J*J!p9SGH`+++gTLiH+MIw{u@sJF7 z6oJ&-JQvI}*k->;agwn=jRAHKVY)?16=*-rRSxlCVrF$^ft_%g|4^UMlr|9+X^oy^}!x-B7~Jf@xz!zIZ{6bUL@ zp35+}h%F=YSN)y4&>jOkI7hG`c0m%0mJ2+JouG)b&Ued}PCDHkZ~+A7gO7YMeg&tS zze&Wgs>K7UgzC4M04hd!?kp2FgaYOxKPP@G>UeoEMKOekG7nyXB*1w4ZI$Lb+`e`{wglK z{DGt%QS^dv4qxZ+SvZq^(T2OoADQ(XP8q53`rp@bKXA&a6f>J`aZRt|u{Eo(D3 zl?(bUp6~v=oQ1I|LMV_!oe;`q4)M?=_{cb@mvcgdvzwOVAxJ;cM%^F5uuMjLqB9WO z4ve{m+8dSf=>R}Lq10DgxI%Ht6#qY|DZ)mLyU~PAx)N z*;H8KdB7ZVM&DHEDzHYjd=le);(0gD-qDixM-opX>Dt021D41OX^#Zl%SuMp7EX8`N)uZBv*JrS0-d%rU!g`U)u#R zWy?%t&H^TM`z8vVIMC9|Uf#>>~;{M=3W*;*M%l!Gbv3K>Nr7W5C9hoE(0(R?~6#Tp*-E?(P@td4s_8= z%P6c8oTUy~WD5mNJRLY3+N);WL=Jx#o7)vdDMTA#I>u#; zai~e$z{B@Gb9KFym%-@f+7>KTf>MF{RwBZ0*Dj{&5{C`#!awklwN@ge`HTl1-F9VVrZ83$?b{0!T|IUfCC85I9oro`f|(-!JH8!7VKy?45U=9 zDPPgH;BN*PZdtZq^OQd~kbLJL!O~%Z(~*!4$-U6w_F1yQ;t&12Kyz-%Tl>`3AZH7) zatDb&K$XIc4nl9~y73K1{_J6NqZy8d?-Ae!dEUu*AsL0d+#%im_{0$$LfRwyKn4hY zf0e0iL46JVyCpISnhk|Cim-oG#fLD8RDIP=>Hon(#9+!&SxD$`z%7qKRY1w>oP9RH zzCTLuf*Xf*YTU|=r^U42LY8uc+8dxI}3O84!A5%uFWdd|1dr#$K$=?bUu|O7^NXFL8 z9Uz#fb1u90{iwa?{8aWvn$;5Qhf86)9i^l*G?(p{Q}Fi6u%R_{blpoWl?8Nfup`y6 zF!cXXK=1;yqF(4%U-10#k$Imai>vf&x6i3x)<@>oZvRaMW{Ax#6mCnB;^RE}+^tUl z5#`^B5xRW@Tn-VETqGyeY50;?fyUs!&<&SVaIZu=8SblK0Lsh!1jw#OjAAONZV0E~ zjAjmm9TL9Agqi!X073{2`Y?dvE8PbIx(X6rEC!NlYXH(tS?{X#BiL(#ty6*pfL!XO zAN7^bEk8ws_9K`_Srb+DHJx5Xg^@DkmgXbbR!bzdH}9UX8c3y@q9hILvx^nN`RJxh z8)xoV`43Le>{7S|FbIIZbSC6ROxi3*<@2i@z@qhwlx{k~wCFo3dI5;RWS%hcNJGm; zDw2frgSe@`!a8Ia_2c*4acoffL8uL-fGJlG2(ljDaevBPtRaQcUbguov^Ik=tA&C| zKU^N*QETku4F#dUqE(^FAZkwj*j``C_Va@ z80scI*dMJ3UOuKAk+LvKQ|#wa#W@Upv2yl;jx#QOdE7yg$SpvSxPV3N2u8SIEXenO zxg zWND_kQDP8j^KSq<#m*4;7k_!)kgAMgt}MEo`xI|1i(L!NYH^V!j5^Mm3TIYd4*ZF% zXRs4=r*`UA30McMocV_#lms}4$D&leCsD}f)PUB{NU zXJ!MnjyQq&AGDBFX?jiu#Ha=H};5jz%P53Qh8tns` zv)F@+l!c!9oRc#6fjE*inTL#A@t6v5s}xn348#lLJ9EhB&J;{(8kZf)m?Tblc6e|G z{Y2;lI(V2GL;`45Q44D9wsNAn#ezN9upr*hOfT4T7g!v{ngHx32HxJRvDey-qdl6C zTUmX$qskV4mAW8gtreB6B$^-B=l$Q z3Q{od1Yep;Zoe-0DF)U9g#*)Nq1=gQQ!f`t8D0^X^9V?`j3*aa%-Pz_-;jUZywA~A zTHydW?9X#meXgm@aJZXS@%>*0FLSRpWCrMM^G^WOTkj}LratZ<4zyd0^su@hFr#2u zznsfcf`#3#xd;FS;wL2DoqyB{(E#EHw8z2jzWfI^ zJ%)n5%>mFnSXo?B3=!*e8dvBA`Jp0l!L+DxaHG;aQkB4`LgcJrc6~BS<1TJouE>mb z4RpZ1jVWd>8NOF7^>DNVk0FWW;gcz&AT{ls3JVeu8r&^bA+8aL_LEct`WI>1{_e2r zhyK!jz&tU%Y3LZHB8vn*U1`*^X{I%5lXw#~nw@xwHdR{W6Z-<=#t@8pF0~>G6-+$0 zjy(qBJQ?p0<38r8+H8wyQ+smnOSIx$;4L7EB=fErIs!|g%}WM+*hyTwTZC>0gKwzJ zwCq(koANnw5u=G)42Nn))pe*twt!nHgK|bIY8V3b)m&I`cz+%Bs92eLsw8)|Px6jI zZADd@42^t@CDCw1#@U#1@0L?UCyX=pmTxL4--e(&;*df(6OIA(DBT%_0;L0V^V@DK z2p`U|Fng3(YPbe+tlofo9%}$$Bl>gP=E-&@@q_jyYxKpD>w!BnMkXmd} z*x6(u%@hE8#z_7&XAbUy9MD^<+JdB7QAzag`i#bX9Wkjv0H11efVwSIo7kvfiaGMy zHPMqDFa22zawEk8-R@PX2_BMjfxE8l^r0%>mqjS2B5aE(f<}D+qrdNqWp3y&fgv}7 z%rPFdO)g6c)i^Wcyn3xjlFD*9QU}2l3z6#Cjr18F#WyWN0Lb2O4&WXne&(>rdh!BA z(UvQZ*&X#ce|Bjj%X3As8Z=zb*kMA-#suRLYSeQ~LaWFTvT5`~PP|d$e>F3-vY4^T zG?jI^-OTqwSs0Q_ATbQZ?*yGjtxLA~hl?4pE86r5R*eA&@wl_hj?6(8n1ETRS6<;J zgwZ+lDjwVDS{QQ82O!3mW@Z#hLCmBiDsr*N;dcDQTv5%C8KHQ+K|pKche-eZDz#_G(`M{f4L)Xz)mEcj-$Vf70`Ltvg9el4xtFd^$yA4g zBD&(oM}=aW@)oGAN>pZKDTL;$q+nds!j5^l3X|GN&|^FGzm3)>PDidekUtFO|^Ku{++^h6eteMxL zf*lVW-=lK|6`nlVR*;~D#@8VFeYr6VBNhD;K9-Uk)lP7&$Y5nLQlk>$Y7KO2E%|y8 zmf+~x%^$YAdwkt;N#M@@QjT^4Q-7!uHLQ0Gm-56Av^U8#%<|h2-RQ(f@GHF=H3Sz!A7wb<) zBRM>x02(R1wa0RwKR=$U?n>T#B0bJA{lO@fOUq4^s8eBD$usrVwTcXU_mWI>6f)p! zWGUp8PekHaiC|M8_LM748XBSFG}1=ijw{Qb>Y^-LG|d0`5ikat@>5;_Ih9?X2jMYJ z?e3Z3Vg|FFhLTDDL4}WhP|HZ{G-BaOfS3|*^ilUZ=Ee|%a&TDtwDF!A?cm8+2CcDm5OElZSl}=0CTBV(g}%2>u{U5**6K;!ti_!5;%VPx)V`pWpei{`nZ((B$tsL>ewuo-V{53A#GGY`ttR z*>0yZmqK5?-cZIrmgy;-1*6g0^9xu-ne)O@M^<%;odtuVee~y?Wd@Jw!(1es^rrRz zS@dbQSJ3^gkXju5E%2I;?U^Hc-mC3c9aJM=}{VTpk`-^>u!NQI!AL+HI zpN4$;woA$Uszlfk=&ZT&wDAtGc3ji}IVXva+aAeu4aW~@?Gw!Ew8<@Gzm%VH*N2k~ zKIu;}zBvpR#82m-ns!5*5{6 z;TPv!)GuR2gS%?WVEGF7P-Bzghg|#ncsbeaabIC6984zriZOHH=xW6c={iqDtI$H; z9GDVR0&sWnUVpUy%4#2GHms<|e8$j%?TpX~Q{&y4EnB5}dSCdu;+MGw23VaISW|~Z ze|4Ph|10p#I=E_eKEF_x{fBS`xN-5DSh}F`Uq&<%WQmA-wDeKrljTiwPrWs`9@4v3 zjUiMa)^m`m=HrL9Em%DmwO5%3n@Y6rZ0#^;vCDejeqt~S%Y6esyC)VA>a>3pkvZF0 zOfU2zR0}CP-yx54w6_~b1bjgA>{qgu|6((qJ`W%lQEK;9#) zG#M->p3uFQea_n@@ZmHMz$>(I82v@~45A$)F};dS zw*H%0F`ZA+*dvh)&E+VX+LgcYsZgs@#vjTbjzH{j)PJ@1|90U2)7l8%t;?O~VV{oiP4VCW_V*zEH^l=MIENsB3>en) z{&vfM`pV@>_HN=Q77vOcmK2AEA!&@j?!#?N@25|PCvGnmyHAQyv^=1P-U!l!_)}Y> zp}y3rX zNo?EC>twJF0N02AE=)0uK$?Yz2N=H|!7x{==!?|brX~)Tt&~;d(l>WYk_*^g@G73) za!wjlD6(#jeu+ygRnBZ!K=(ew#yu@C#l0yovk781OpSZ!lSA8PkBq|#8h71;ailFk zv*4B}Nj~zdWT&cPpAJs8?#z<4345GSr}dnduXHx!oxWG5$RS;9vT%%h1f0k^<g=3bVaJkhhWXVDou5-m965pHWl!4Xn8#G zQ6>dfVAsp^G2u3ZcgvMM1wNR}9MJC@yTbD_`y^ z8Z5X^9@wY)@Xy(FO5>sTVs)IG!j9LkP7j1zFdh^Lc8Zsj*{BQR4e?P*OiSx z21ecfF>f~v1CEmbqXh&D7O1w{V>rh)BvWgVaq861)o2YQu$wrM0k9sB2R^}sXG=va zpf!|Urv5p9v~HQPVIE@QmDljJwa{QjpPai@kUU3yp|1@EFyY+)DEtyUS@cpdcF1Q% zK?);Vy8=XAm&`LUvmt-TL}lYl?ZQ9MI~Oe`rdG3yWj@Terjx z8t+SpgH0T7vA@|w;0O+i_n^PU{_37Rqk4D(_o1!d+)lser^_HagdbGVpSQM|k7Yl3 z>>3ydQHylBwn1!|@fW`gji`1Pca92poV*YzuO0oB57cD+rRavr%Xjc}s>-yv zd1LoPw3FN7CBNsj*QJHvj#0!0_V1Gjndx5OP3uEIU`qV%3LIMr0aX$>l^MSY%1_Z-<(w~W#Sgpii6 z(FWowYgD2U4`#*=TKw)}?~)n#$0JwgpVUJZQv3kaX*OphNf|*~w-_pT}+L^2r@RKq6lXpLn!vdAXP6hF3uI70m*_zka`8sjsT-GE3CJs~P#5 zdiF2nUEemQj5y!pd%pV0VVED?6spNSD}$bvJ0e;D4DNIHMH7#p6K4*6K8 zgc?OkLO|Mnj%H@4sE;SHI>D9VqieH*3MwFuAZm&t(m3LHFGb)Ht*)30B^p>gF9%BD zpQx>DyPTA-^PrJx?at?~n90Z9ncV8^T{Cas-WFEn+!AJ2ydE-*TMG`{S+#aYa?m(8 zjJ+=U+X1q4_V5T1+bfNP95;^Y$?=XVkiQU&>rfQV{@$ok?93)M zw}6V>M>CUX4v9z4+C4BJh26^XO&pL8EbGPzO^kRmiwzK+jf57`IJd&>Atv1_ofpAh zIFo3YR3M$5<#-RR-m09J^HM7^TA0M`h4)ykbB<{EsSc452H4+~RQ+g?tedL}$&f-D zX6Fx*;&_h@wDWt$Cux1J2`fA2__pW&rx@4;7VoM()0IBXkRpU?6BE*fPzh+x^7ISd z{arm&36K>%!C3S*xLM+plA*?_FdiFpu9&U0 zc}vyN??qBD?NGweRXl$7i}MMFOINKxj*<(<`pJ3&{rGgguXsq{$_?nRr=i<+)nS&| zfp=x%J4`HgJd_Pb(`7=YsnM1ua$?;H+i-uLT05c~)Z=eq2FrkqWlJA8e1*R!SB&-H zMz-V3o|URTx!|*hXSHagr8}W!g%E`iAi#y3)Xdo@X=tNNf*u7vq;#`Ebv~|`yJVS| zL>JxdE%$@Zo5!YnJ_5*Zs$I3^fYb>lGe?K$(c4r;esGP~8J6!*y6Cb|OzF)iowEk5cy?3dNBdY#s{!)CWVW<&DUNt6wI8>`)go%33oJ10&x zRgM#<5X3n@s@SMr%=DibMjrl@sDx?$Y;O`w`R>!V#AUCJLV%*b=%+n#Df|d77^uO7DYcs2B5DS1E0fm#+1R|6buwGr?<$is-i#MJEMxIFGBcyL9ufGNG4guIk|{NE#vl2-QB)mvz! z5) z1+_A@nqtwyPZ1J##OT#kcF{7eE2f~lJ$1b1AE8&&%yu8Kj@7?gIy-7|nfwgg6y$2^ z-snZAQMxmPv<>`!S<3@|3!tTaB7LVoMOg@9d;#hL@CNwH+Tisn?CFfm4D51^jMHeG zNjVQ-K|IjM$xvRRj;_XDdq&>3L#B!KC;cJv!bbl6aTaLfx}D;o4vYnirlcz2g4Arl z4fkTT=0M)XZgsw4af21zv1|?XpeEyU9b)#KZAWdUXFMJW4;z2sME~=i+Q}XNe5Jpf zwFErJLjGk<{(!zLp~~DyzvuN!vpULm{6maIq@K9cm*z3l;cTk0fl|BlKZ|kDnsaN$ z|2rCMXrxKU@*~^jm0)Gx>9#x&08G0I?fH<2!4>YWBpo#@|4ZdCeDs4`3!!g#qAQx; zH(y~RSjCswhiT-JiEcM|U>jSH6JWy?H9_lfnBHjm*D+?(ru|V;OqZ+8fd9f0!DlT8 z%Y>hC#6HA@m-I5@n+}Kiiprn@1dByXd%=vGm3nzO|e=9-Q~ zuiPLpY*2jQSekC1u}4U`8|BM@c6ngL^5>v&)LC9A3g#cSl6B^Kycvw%9^iNeeBkhw zQ|s}Ndq7Zr%j1et=4qI(tB{1IdW_ zQ9FCVlchHTO1rsOl5!bHAIqA#3xH7$xw5xp^)VVA3^O3?FT#A`C5(m}q3n1$6sN#@sDeK^ zR5(C?Xux#q>TWu=5f!dM=XhpU4r_05D=pt4vLasw9fA z*kMa}3(-Pwl&M#+@?~kS@eD)wGJ9Is^4F~;vIuHm&r*$J`C*mEykkFKN*U+r&#%jo-aHcA>aRu32LxgR6&sx36rdwVwK!SO4!^x>%}Ap{(YY zQ_{vu}P$N)#i07ta5g{yFM_A9w#gK(RUCh!;{4L))f2SDm#Cy07FFs>*U(WT7P z!x`xjfQG5rokbzQw$}ihda!tL&Cn(3IRppNX*u%w5mU^_7>X^qSy(J{(5IUOF4HX@ z`Q|eG+`+swm&>uzz4$0dYrJVrkKmq)m{uka!NRuZBN7fB_5i^uy;tQsr64(4Qh#*0 z#(Sfw-`$`R1|0T07#J?wxv!#kJ?9THo0mxoW>9Fk!5R#R{z(EO40nXxPDS;3n#jSo zuN_vp1I3&}-4NC?{bj1=CrkEqt+}HmI0VW@HwP!}+}DXh8(yoDy*lES!;(Sv&ppkU zlJDDVP8ETwM0e)LbT-7yXz(1*)b~8?`cS$1m)VJB93Xr3fMurz*fbz`X}Ky8{6p|- zlfIwpjR$RVCaP%;{fOWK!>kBN;BFHn)1rn~s^fsu8RAFU^{8Dl+V7%&UOBk~d1pKyp@u_vkF=kXfRW8d#vK*( z53z=Q1HkaFmlT&*;f$!k3Hl(`cxj&G%cywvl0v1X*I!O9nLxX zP3ufMzO<7~O661sScc~1Mbb@NUHipzdA*QI2LPT;qyyUWnq#TYCZ=uuwv^zeSo2bA zaHf4%#d>S_Ov6=~<|f>drEkVZ_BZg>Wu4lSpt-4(|8T{22Ktx3bFMdGvE|hhERWtC$u%z=}JZ_ll?VD4R=RvT+tDXg?x8s zh9?=xSTO57fU!>qT6+*;kjVRnt8a&&7|e^D-G7y3C&C ztv>Pto*h9hjac!6FSq_{>p-)bo~*G@Dgbs1V%8FqQe%L}rdW)#*>Y>JoRQm4q1B&C zY+)K*(Hwv@UqxS%)n7u!yuj^v(L?>Nf1s^N4O*zatpD6*tAWKYV)zLMfxbcm@iiB~ zNeH=_y!xK$meXomTQtQF~(hJ6bw5Ir3fg2>3`w+p-o# z3*<@ij{KCwKz?vVkFBGZni zjx8)N#isfIKRs*F7Wu=SZWJeo=!a)bR$D^nOyv&?=2wBZuzcAy?G;)(nB^`2)rI$3 zopnpNEuW9~I+Qgw{ZF)Wxm^BlJ3mox;74S|lgL58PBuQC_P2Et zK@0V`pP!@XTZn~g41kWdxj$yVI1JxXBa=>G8DEp~H^QGZo}=K`0GX3PjZv4zJu&^X z+BwZwKrNodg46p>)j7A4{8LPImkuiH=KRwc#GVqGvl7WXZ16%l0ITOBZXI8OnNABGV4SSEj!< zaN8B^6Mn=YsS5LPM>KZu%ZlkxTELp=t}L5?&{ZEh^&r(SnY^fZEB0;)SRCu;`8S#( zal#AErW3_A!OLabfDzKU!54D8QMziU6(Bb}-+vR3fK2EA5Z`8{b_;Iq&ggB6zo+g0 zE%f`}0q_4a`~5GWA5ImJEvj3np!USk{-9JD;r2a?q_uryE0uMecKx*Qw zA|4C!h)}Y5TdZiohn=b{48AokY*iXqxVnCRsxV3=dZw5X2OgbEOT&CE*l`25#KD|c zd^af4JeP3JdNs(rbGhdGNN&#CwAJ5>lY6W{ByEO)LAo@Gv{m{^!Ae0*@KQ7kJDe7c zY-_vT>l>H|uwUZ_cBoXbc9Yim0N8(5f2wwOKmD+71#OGAtamzN|7 z5Z^~>1(QZmEG#oQ$*ak^ubRwvp4>_6mzOx1CpsSTsT8PGU64^J;ggbQyyr;YQ1z_D z@ciF#D$%4ya6zARAHw-lMaUJntUI)`91jgo%CR=t2p5fw0*ojkFH0pq0q3aHA+8B? zvu_O`6Y&|tYpz%Gk;7>OLj~{EZ~084gr>>9a?~bC7HK6@swiGxyIiEqzE;Pa(HQ7w z-aqa^h?J-WOSZwI#uhUrlLd?9g;kX4m%@>;Y{t!k@QpmRG~a1d3}DQ@Ln< znyCMjdt#++qCM1p3$I2nbr2jdH6ra1y4LxC`!68ie-HQnE(U?`VrY;ENgn(@c4y#^ z3=YD!ZAd~e(EpUij{|7Y|GD*%0jv2>m7uKwgulKqy?JOL$v;8@V1fh%ED#x-0TxL3 z!H3vJWJZDW9|{lSOQ7GX~j1CF@zy*Kj$88{9R z2uQ9Fa6XHi1n3OT4+j)JE0qKWQOd`Qipmebu7`%vL(6y^^)T@G^Q}zo~*ULxESJ&%g)YYol=v;Q# zUXjwXv-8)eiD>+C`|U(YLXP~wFyZg_^3(7o`>SJ6E${1yg*3CYv%5JXt3;a~FpSFN za9jdubg3(~l1J`mkS4zA)zQqz@~Rrl^pG?zC*aqyL|5D{8Wjjk>KGoZA#`NpJ}Z$o zv~lhNj<|-?BjWg#07vw4s*2B23gx2e(YHlKl~A`QwR~Mn#XD7a_$YlJWj#~Tt1Rwu zakTAjIqr|;#{C$KsA>Rm2Y+To~KY4d)Yn-tCvVaqB9>HJRa#*)qp&5bp_Y6Ak4m8&Z*PigbixjS%gyCbWUFL6| zC2PdPlEF7=v%ASN0*$80&75-I74XlW$?4d)#McOQcNPh|dwOuQ$DIYxhW(LGzechc zV03QeK^p#GOxROhLeb}?G4Gf|ATk@1lF4!mZROq!N*EZe_ z_L1)VmyITXQFNOGcAaE9*mZ>|WaTU2+uq`{42X1xHvU2WrCRLl%PJI#Y5k9hNoaaj z`mQ?WI>o?8Lh^Ug0S^!0%A3aUJEo~+?jCRbMuO&3kNdL*orywBi)r%gE8;VNYWSwa zYk2P~^&M$oov7!7_BUnCYKfQF^S@N|+3`0((!uP)zi8meLze4+QHhsa)NdFbz{@xN z;7cs?#_P=>V0P~X;WKc&8YVnAv2sS39XE#6+fHoXSd17IL6RBJN{np{Cs|$F|7GsV zF$~&3tmgVu7Qa?GIn)3Cx{X^JL&0(|I*!xI_!5Xr#6Jlg7Cxe{qVdkh%* zrJLl@#V#BX+!%?T~djh@F7uj*3lUQaCSPa~o^6eQAYsEF=q>W`1$gASr!#1gSV z`MPzw=x6g&`;wyE4who8YnwjsT|YZf=^}$iMb|Gtx{(}!I{q}f`!;$$%2I>i^K3ZUX36D zph77CWF~1PTiBLxu!bGz*6UXQR|rWY+TpG42-PRZ<%@WbL1tLx%yl{CaJ>P0GHj@e zh_6dWm6iy@bfU7aFEc5bo225ki#jwy++O8cgUAr!&@h6816%o-K7lq~&GkiOjRh?& zc0*TAOyEI8gv+Z^XDJKz)i2hgT;3YG^Rp;GE7(uw(1-?sk{y3 z^SGaF9-af+x!Hw7*P^Pf-0i%*xW5QrMEU$*i49hN)^WZvi;7TJ2k#epW$S1fvl6(_Dh8w|Fn49vyk0O*RYe?TBsr8b=XTG4_p$QNoXZ}?gRb=g#TQ$ zGeQy|bFiS^f&_LnlZ8-kqgjWV&+usY*ovG3&CT{v7^$2LQrhLT@c5u6g2MhwWmBl*ZiSrvO}MvaQ-|7Nu78oZ*s zD)XlVNJ1&R^AMP4E(sYfuH)DPklZ!JH+;LCk5L0sY%Yz1%P$dr+U6({z%HNse3CTt zX6Wx0HdOLN53UOG9$+^8sgpaN_n?9vB|j|tOa55cm>&`{&l&8s`IpU>%n4>*(pD%g z)#4OWX8gDhP)4X!Dq-3G4t&1Ypn0=6{LCadKw5~^rVL$3q`&CgMXe(hU_u)&%gPfy znmGaIP-$+F$)7yS_GX`JyFhrswoE;Ln@%EHG+vPn$!uLYPdj{cmyN(~U8(+#vlv1P z=XksxX~E94MKBpX@?fTooacCt&d73(l`mZ~etzz{(EgcBdw`?;lY9Xo;CVr?PbvAc zItiX*L>-5AQT;o8bTI`D5NYG+Yykpv5WYo*i60Eb?XMIukM7 zx`6jczmR8SS<)$`&DTn|gm3Z}14Y$~Sdkn7V>~m*#}8vENM4Nss94ugv^VmhPDO+3 z1-GL*c*BS*`>;}0u09%91=@2evA%I4v?>rnEEdh4Q#WQM9>}XHXae#kcob;)byEdPJ{97uP zD)EX5uyjTBRruKphCt=wG3-=39AMXGKslgs}U7isQ@j4qE4rfac0QF@ViUnLXguvD zE!QVd+9V165DpwyX8sBOm(qA!#)?wdeP!;(YD^#tbA`RSyt2;DvAWJcr6s?x;hS=Y zsK(od2O5V6Kufcd*-&*WG*PPNLsVdejmzV*=b;(>SZ4*VGrQt0)oyE^5 zpm(Q|o^arl)wvF~g!MLwQHFO>6Xc5ZkQQ4o6s7^XdCNbiWl_QYFHc8)vP_rNT39Y3w3E0V+U$=Fp zRI;`OrCa;qPlM)}rERwT2c4mGC(El`(bBB%zkdyaUx5kMrRy2!HI+-wDjE{}(2rXR z*(3+p&3n5QS`nWo9@6VofoBueOQ}uY+ZqGL+pk_Y|J7TyJt%n~V2N$cNl-@}apFUm zEt}90R7VN1c#F+T{uMzAw5-R*4EOEqdU9*Dq3LkF>-HWQMCykJe%9;Nr~wQ95{fFO#*6?P1HTtkML~*VNiq*6C_*cNZ2`R(Af#oFa6$qAjc` zP#!SwI^w{@jCfjy82l$m3>?(nV*dU8XHj8ou6~~9C_W35lwOMDJLJ3z{6A#9V{oQn zw=5jnwmq@!NhUTX=EOFi*tTuknAo;$Cr_+1@7`a1d)GOC@4Bn5KdWl>>eZ`zV$C=s zWtm-rp{zW!w;;d1r@g152TSnjQe`EzQ1`y4U=NpYyfBAdg5JM{J>!Ai+}2i;#bs!X z{GSb8y~dbVJ|_1;C2D~{OWnzSjM_dDLxrtnMVh|A$ItpLo5JjUydVDGK>W)r|1M59 z@eYj%MBJ~RhiTD&#sU8Med!v+oM6so}mjg`!6E z*7FrUb*``&P~EZK%+4^GVJ>EADulllUe%dMYQjPcw6pIjygwJ8S9*bG&t*I@Hyw2~r=}*JZWCX@Z9r0-L~L zhmmJhn!@LgszI-uJ#2-uNtuv3dLsd@7jGlLu$<1&H|OXey_I(tUD{amj@S>`mDqKT z26o-=EF`qzm_%xDVB;9=j7r|F&|Ib%W0P>I5JU3({UP8)Z(rAQk2FbdQQU3T{0&LK zKgB|>l=>{S%(SxfvKs7=>eY>8?CUaXQwDkiur}ju7JX77B2JMl>_vZUwx6m@-Z{-xOD|4gmay86R-hyeu|}?(?~b zy0f~Ix2737XXk>UBOz|wieS? z>l-dctCY+-0r?kS12_RAN+`xSz<-YZVN|kz$9HBUS795tS`3;&U6)^o^ScmDgdt0> zN~BIb-cNzFd_hYf@iwyJrp#fmWmZju=sSMB$a~s+Utds8k_Sx3+{c(#g&)y}n~*$fOLq&KeW z9}Ym$tSbyfq+H))Ulr5d`FseT4#7wG$meT(fan%gFBf=pZRO}8z)@-QVce9URZ?zV z_x}xwW|u~w{Isr*1XXb}9A3n&q1Tuv#5b2s`4L#MRUd!<=>uT*0Y5_OJ+b9=BR74| z>MRHP;*mc_z0>}4^baD;;kpG8I0BWvH|uK{cO@L5H{pbDbb6k3p2LV+-d4K8j_ zVF&`(a3hur5MSqU*rc+W=e(;cG=~hf0LSc?sakoAzC~5O$rJ2>ebjpsUkT-ru_T&^ zbk-f=lCCbPq+1I2tbsf2_!(T}ddN5JnicC=QK)0N>GeV{2%-NI<~P)JZ%`D}U{>)h z7dAOgl^@^+rq3&APoQIuQoT<>N3s`Em&w0kXYLBJG^1Gh;BiinQ(WRyOm{-3HsrLVlT&)`-lH`M zzvx(_1lpt6Y+ns`_p>7LLN)DTsd+Q6=77qDnyd2Nz5WfANh$VLxj8hoe8i*OGlofx z=>c$b&Y_pDcjD(68F;R%S1!2k`iSes%}?N1bekMQ-!r~&Qg-6)*)-giV)K~x-h5r(0uoTaKb(NK=~`8FnEP;{naXF$vblp(RH9V zxGe1ppVvuL+hcY~=JwQ$e7Z;WTTK2JPZ1zL8lKk^6^}p6j*D8tjnwxuM~@VHM-n}6 zA@uG_>OAOA@gu8lL2vsDc*X3)W(9$3k%Ec~h>(}5YVg5(`MY^ebEIe1bP#5jG-GJ! z{3*w3n07aO-hBwsCydg$Lc_9B7!Kv^_*q%l&AxfQ>|5hxVjD(mxcMh!%23Cn#w?(~ z`GR3J9enwZG41D(jkkx$hP&Z?Df@~&aXv=imqQRMqUQ-sG%59dzQ_$8Fh}GD9Umxi zj>?MoX6Dbt@d|Uq?bCh`14n{yxosjS>l~DTi+<+@C86>~sb9tB_)0^G*zCB8n&J04 z1m*Jr+b%Q!#V_R}8Y1{wSa=IAMh}oCQXsZ68U1%A;Y|My(LHeYGNmsuiA~{J+gn%W z&cuNXbAV=oG*_*BRX9|PhKfRC#-H~&mk`aa?rr{d*N2k$fjm2C)b_U+`gGs)>~?Sh z9d#@t_@y5$%ql=D}XY`_U;)<&L4I4?|j;L~yx!Mr^+co85H0SbU|+)0zo zl@NPVBK0AVF_u}Dz_q}?E&A}P*AYgXhqNu9{Xyb#hit)cQ*_~hyfyuPinOhe`2u2d zB*_t_1ybtD0P#=`^)u0&u!&6g%=Pp`1)pXs6Xf^vFd|&_41J)eRh-BflSAM*h20x! z#Wx557pf~W@jngmV3kd!N+VJinh zL_dNC^ys>H;?3d2J$(wA^sl~OmKe%9I(uYp37oG4JlPB{OP{4_z>>7P&aelQdB&3S zGst=Ny-a-eKWK~&QIOLg===)R(Qtd>{7RPg7cQi*w}79k`RbjWjSCQ+vnj3Oq!Oro zihuxxBc(1w+)S1>(rMF48y42;SZITXd}JAwV(j$U6g>B0U7n51LnM8BDmg9kLE!G4 zBkSwmup29XvNPxL;bkf8gZd3BaYq;Yia6Cs49r;Umv=Qhq3^)ip6j0wfaL-1DnI&* z?Fq^;?uiBpT(g;}@p|E7?}?J{-`TORe6Q4~%WD1vQr7cF z-=m7St**0;r8@+DKxK*wP1I&^o-MG5IbS`2rQaD*Umq$nrH4p-cvpXkDb8;O5bx5fp;+M1V-|pQkdD7$lY_@NwuW$rU6VqZ|%3dEaY_Se_m+K;;rm0 zWvDONMbQyAVfm>GQRw|LD0~ z*2~~F8FGm=Euy5Iljuc^`Qo&~=hs||D&7^_(-c4a%ZGtA_%Wt%dJWI9_i_evFnbH& zP|~L(B?llco(An+7*jn_&Y**wVgvCo>ld9aY*fR|F8m`l>}1ZFC;eeVkvV+F4X>!! z4uwqSzji4mYcOgM5<&?_ExS-Rn9!C=kUQ|eoLf%va+hV}P(oLS>^zAy*ZVSfk#>HF zCk*=i#}t}zo3?%SG4qrvcVSNQA`VqRR5#Szwgm8l!;Z~EiP9!tIh*-$4$(BhF8Fya z^-(6sx9S%6A7so}k<`CypF1yP*Xg!Nvbsv7qcusTKQDZg`Al%%{xYDOW+OQBN}SgL^1s$wh%vnKqgPqz77D-!m|CQQ2^OIEJs+~YpLQ!nh^<(3jUh* z@WMVh4MsD|ttnjEtsX^{E8!#GZqEdz07k?PK_QQSz$;rmoCaG6N(_4cMLQGi-QlB@ za&{Uy0kh=3s}&zq`a0CaYQ8Smv3B2-6N;Wd#E|p5I^U!@#rs?aoc|Rf({V%h#Q@O! zkGQdEz9LZst6;LFii~KAx7Urwp^N0|TxZR8@ZAVrjRl!_9{m+>Ix-Y`T45aZ!a5e% zJFL1X*_zbubNvZ)DT!94tky<~W0z~6 zIX)tuE4C`B`k+63Fm(L%!GC%ueDm)}bgk$tQ}Drl>h#aiz&~p}xAj!n5(C&(P62z? zMTslD3_G9eRkmhqo1-Pmp?P$#c4Z&ymaUN;fi&3@X8jDXZ4CR31~Wt*kYnPYT2jtD zx}%ApT3dCqa8C;gbG*4Zb=5v#x7pLR;*YQsPiraJgXQZHn8l#bm_{~2z@>LbR9AE_ zx;%;>1FunV2oh)ybEw=)3IL06UeU(8J?7|^FE`$!+(nTj`PXN!Rce2xJ5#)Wh)3<~ zD#PZJ&!)%$bbkO5p0Ke|X{SCp;&aUhn9aje-pHwDv!kgrdt=M^+34;Gfd%ZGJcs|?jae}bsi)EkH@vc(w^jqS8_^7D# zx@D8u)t;x0=H~_g+|^!E2v!Q#mhJEytl1jjPyrQnN` zJplM|61}Wh0H7Fyzsq-}wNn4py$NSYQD|%<*yAdj9g7VNIF5ImmHcmJfP6~nP1i}3@J_ZZ9_>+J zsolH-TcaX|!3#Xwo-=UJUI?sO%=g#1Y1$<5p1MBdn|HM1;>BM~I}nX7>4l|C)tWD9 zj&p{v7durz?3d9al5YO?&=4TD|J-M{4>0d?E(XyU+x)@=*~;Vhs>aNOtP4%TqM>j2 zi^RX!c5XZ##ED$PDw%@3HM>-eHKXR}Rg03XJRaLT*_lMFMyiY=kw2H*=TFWluPD;A zV|aW?pCHp=@Y9u+88u5{!^z>Uk#28t!$(~=!<#6e-H}MEmZXczwCMZGL<&PS6m&>v zXp097#0y91!+d`lULAI< zE1l32comfyL{5&%I*Hl@@NApLMlRA@(~)3f6TVt^a{=NlS0KZPfu*dpYg7UKh&f)Y zYYsU;P9?FpJ4``k0K80U*1*1Fc6Tol%EYqjOipHJQ*tNPEK;b%={=FNHK#$@F0gA@ zyr#5G>BLITIrqs;zC(jk>Ty&ffdA~| zK{HYz%m3*CV!=}#gOEsDccEd|K>zF5*U>pL(Es1DZ*!{uH_<FI~b$?55@uZII{Khr9!w(P$M*QdV>I0>%L4cbb@ zMzUf&Hj#^=@mTR15Y7TN){~_dWkl*U(7G7;dpFmeaH1NN`+sn6J_paB07#P&^jHU3 zH`eez;cRu8qtsz^Y+7A|TZh~T=cCkgGWj^J$LxKJq9Em@C;&2;XBpBEeLZW371uB` zP2t0wnTK2aq)9Fzdq{33+0cXiEx5`pm0*m?`Umov&XwuolG}SF18M{G?CoaZ3O4!H z_1g-)*_0V+`wXHAKsF_`xr>G;HUwI3dJ4Y42Sd>QhizBJg#8=Sjq! zlzBM*a=bl}^|}bYqYAYut)ptk>Nb`2I_aTS1h!;7&G2`*0F=0h_0`g9qVSfE-HMo7 zusxQ;)2K&2x@ID6(b9h9<&Mz7M|OU4wMPKvz&fq>Hvmn5P8>O5wbx`C<~z$lZ`^0q zynXUDFaFNju1Y!HYKX8=yV$MBc=ngAjc+CfLP&V-mb#^hM41=eT3GKl{V&)6bIA5N!IMn2d;2|dMHES%t(a_VPmJ{2g=7lNb z;sZqz?0ym~FrNxx9VLcc5*Z@O+zsmgr%M0-ii7;$Rr;U9^UX$K$FYB3X%PP7Ku*>Y zMEMT+->MbMA4E3b|K(Etea3Sf_)n_x440axOG%b0e9Hruy1UDZmHLG$fZN&;%AWn- zKiUkcJ{*I9fV_aF=4)dUrIPHzz@`eX!$79yox|X#eh2U&mgZ#;f&TXcS7;0m|KD3< zc*xhF|MeENZ~HV4JP1fJUaANx8(~TUKMP<=6V@Gn3H3{#!h_Nykz4*!9F`j{h(up0 zi-!Ip z?bvYjOBbg8?uiYWZR=j-O9G{ zM{GGbe^i}lp=>mZOI)be>#i98vmAivx^dqZpXhIE!yU=>{vIh`w;=r<>W`kObD|bw za-t)0qQ(P0_st0Emm{ekIYyug8}*3D+ILYwX1{@Ml-{v3uRVmN`#i8hbpw$nS1+W# z6nl{VRln#lln=nJ-|Qy|)i`Ww`O1UIukBS;4wNYO|Ag4HA3ouuKfZ-XgL(l1OP6o1 zYHS`f;18E`i!aqTQG>a`wX!%-MnvuCGiH;@mTSZlD*jr!%=qCgF6!CkqJ@{u#jO`m zL4`@m*`R95KGcis5XnVT^*51_bU`R}C4!3P+eIy@*JkkQ@w!`q;ppCmNXSYf%z^_G`sZrstm74{wk(#OV60aAIV`=k-?^4W8!7aMg%oDx$}}h&Z+@fa0o8>hxve>&fnEJJC{& z%h*(bl_%A;TxjZrT7)drZ%MkA-UEN`yZO`fY^}@i6C<@pBH&0=FG&;5b6|0_8 zFHGxSBz?g<2T+wa+~a@lLNnvl%{oIlw!r-!?1kW*HZgAQ zP?3^oqHU+y+hP3*{FZhX8_sl()P3SSN{jNXSbrgzF0xJp1f#%H7$yY&)})B1gqPAZ zF~sEjy_!CLEOKm^uSpR(BzuGsOrw`P^snJY>f~tC(W{DashCd~;Z7%u=R`S)_t2+A zV%~riAaM5xWU;(G8)pl3Xom-RQNJ)=(VG2QQZCp0mzRcu#HwDEG(_hRMG=$y(_d2# zWu|UtRBxRDkYyD8H=OMj%6N_j^d~RNyMwj6Lxw<_E%!2?fCxjmI9(Ps#WJ4{RMQ7l z92?Q1oPgO}0DTO$6=ciQiVQxdf~+_0);HnFBsCkf}Q->0JNUB}e~{vy0ZhL*JcG0E2&oB`q#;B64|-f=$c*xIyDr zwvcT^P4L3w)X@V+UNo<(;YICwW%(gv3qf}#Q)^>@3gs+^Zj3M+G- zMEMwv3t_wWAyVPugPCMxaiK}NmDXI$SSs8wL%jORI@akEM3*mIXL5*27+;!^RO3CZ zsyIR>08U5JOcr;7lU0egb_3yYWKX<#JrUuxVDk%jzN6IPq3P+7;CX1luOif}Bg;0R z#9}4gRLbXIrE6v7>z>x(cyk8QVD1(x)1X&CTxMgTmwFvMYTvy>Qlo zEXrGzUj~`|wZ=f%b1Y+dEm_7cJ@yv&VvE}fK(1yQ@vFSLy=6${3SE=q2<%Dr8rdrT%Dkbw6(ljc{>|4#jTgSV>T@z)r6z={shMZ z=yKju-&omNSlZlk&JEGp-HUeCQs!;cvCr<=JubiETd__0|NanqoEGufBG|tR>y*Iy zT`3OJ&A-B)<}6oUi(3_=r~gn=Ir5PW>mZ&)z(IDb96=pK-Tbh~XR(O;;tc+gZ%$w1 ziRn66QAj*LgzdXt8dL}i-g4g#aaQySV8mjh&5oEtB%FB30P`B%XY}*=cRMtIDx*6;M#AW$=7Pg>Yzw;eq_t z99zLCAhHJkN)GARj{7`nCEXC zNJus1j269IDQ=!OFQq=69CFsfS8}2CZ7KwxEl18vEMi#-CggQPnkFHXN`Soj+&^ z{nH6L;0s|HQ&q{&t4}E{`n@i|yA=|-KQOdxnqSseNcP%22?c<#az-O4 z)h75ejKA`rjo~OqmQgTJU{z+C4gH3PJJo)2n4Fi6D#3$NP&VF(iqlM0Wr?9AEv zSUF;d>z|0DNCZRbc)>0z;(*pflQc;s0kvUxbd%8#gdc+P>2V4568fOl?eHVJo%j>0 z)kXn+WOz(-!Fq2<{}~AW2WJ1@1A$B2uJiK0Ff@~YVQ9Snu)Kv`>Tg$2%+y&3esVze zvWKSX3NF;`4J$YsLZOZ@1}Ne+GhE<)KmdFc2^rmZSb;=3?lm|00RkaocuBR~Gl$G` ziB(65Oh*|@i&TcQRI9Vxv(L`6kKN>Rj~N*7>_F)A;A{JaXj9;pMGVszL_?Eg!~}jnEoqI1oXC zq25)Sl{#TUzN5a1k6=QUhlo%==hU^Xw3iy~r^;-?{^_W}OdRFU5$Cjod~*#9vmwv& z3v((*dTDEMt&JFrIc1}d?{b+M0nQ`d(-)1|D>1}|-*x`L# zn{t9tN~82WO~L2~(ZUtEVrqsp?zrjT#+Zv7_)?>_`lv+YgFoT+<1WD!4?T+@TY!nKIzy%;wT|Ub1E1)7kjxgo?oDX9|SpG*uQXSM|kZVpvWo zHTgni5qu%`D<67#zcgfsqsnX0LgsPD#|m3>R?HR(yma=WcEi{A@1#8XxcfZ%B(2;_ z1nF2HRI$W3gGKwWdAyFG5XT;`arf&guGt$8eR;&iye#M7y}TIb2`Aw6WPURy{R(cJ zzZ_}YyS!y^*n1#d^GVem@X?fL&rqJ(GnbS(ZSM~~1$GQXRGrdczR-iOxzhr&$sOMd z)QiI*rm#~vO9kg> zO!L;xJNXD#Tir!jF90}9mob!6H5$c{eN;S;$g0LR` zNsHGE#7+T|pnmrgA1MD;$0=6uDzIZd?55vd%J(XMxt-*8f)rZXQl|e@rt;!m_Fm{^ z#;Q1K5S45+IRb7fdlFu^K|I_CB+Lrzw~Nv7%)JkBDWCO{75@v~Rwvr7NQrG#uxr+* z<}I6W7PfOyv_sE+gpq-3)g8L1g0itrqRK|Sek&7XI_7g$0&~H1eDvN}R@`fGd3t)# zN?D~ev+I!odmDMnJfAk!!Y)SeIhWjUHY&T^&Hj_Ioo?ZIlaq5|uzUnKc+ zPQx@}jRd5Dw`Y4R1H<9bY?-CSBp~C$D3hR0B0O~>kbp7>H3OnDsE0lF1kaKM!8VGsfbc42Tp zCHq+H2>?3C@_va17=ywD6n}B-h0(IWXyc$t`?yE_*ht_V4Z!$e5Bn(W`!L!-H({m& zoOY>hO>Qx6RqcuR<;vqSyKm2cxWIJaZ{Qk`2iOku0KNhtfx*C^KrGK6-O<~&sbqTA z`;=6AmiwkudbazRRC*TsYE;+Ecb4e2EO%PyfLgXYPjowGhteytuDQWX*{-!gZP~7+ z!426>)BP)|ZNvR+s%_JKTl5>wJ0kQOjypkgY*`(4hp|BcD#pyVl|fAO8?HN0^yl=p zok4IaLGyhPbma7xxGPET)j`RKJ;}2HpfT_Ohz?8yssK^nEZ%V6e!ua&dAvcs(R`8t zKG8l|KjA*ij=eX1^sb&d55_GCs45*LHt17EYYO@ke$iJkqWI$}|tF z`IF00mHE514rxKXBc-O$##vxXm}_3Ht0nvKPe-uptb9kZPfPu_yxT0}x*WMPfcnbt zTW`73oOny_{fvlL_<7#PSx7HmVDt^-BCsmdJc-EYo%vz_2t$}JdB*a@GyUedOs$z) zRLDO3#C-ulScoJyufqOw0mx3M!6dZ?`K7?~061CS4=q;FUVq6Qdm91$Pj~#^Jy1~w z90Cpm1_lOX`3Lm7EPThS8c(VX43!1|PXxsmD^8_NJpwq?qgj#E|LUkNG|wKbDn{-T z&iVGr-LiewA$g5u3HS++L6QN%i+k2;PJWFA>koREI;@3D&UqtC0C=nXCDQ=xkUoql~)z6R)6buGe#v3EzL%{q*-5 zUJCY^)DErJ!WK-GCYQ)@8WxLw8c*%o@SP9Mm_@9>^k z`JFmEY?H++5L&U5TmwU@XpY6bZE~XZ@G6&SIak z#e8rD`e|gD)-BxjHU2E`$I&jtj%;RB$ET=uM3)u%z?VDV&&n5HW_Jw= zqB4^EBCN@vv2>E11TmCx*l3+{xjY#mVjFjOyR(X@FA?z9TDJRds{<9i9SDpwmUPGg zf`s-gDlp-MHWV2n=x#ySH>Wl+J&$?3CWUx<%}O6;tLoUG=55LOxA4W-RWJW#c8|bh zC6Z6QD8;|Jo6tsb2dr;~L~0pKN+jyRo*DBE!^Da0~xshJ_``3kf}h^^XhU4+F)t`(v9B zKel-`iAW&`LL8>0uZAJQkVk`6jgHkubXW1Aa3pFfZY8z3vq5p{j;u6scqJ#J26TWV zDwk~j7E5v_JM`)|Cl5Y7QarDf{d>Pi1K*0Y}_&L*Ce`4nhl%OPknjc`5Ir z)YFb4gwyH{G3YZ6BIMJ51fAN`N2UND;{4b%WdIPCq!paN(tWOOwm_KVsbrX&;)lPv&F@+d{k zBQE&B5K|XySFJDMd?9n(y*wfs6EHwsbscL&85%qm3B}T4%5qY2?T_kmTj4x5VaIaZ zC{52FVoZn)v!H7|!1(5~Pbd`}88Mw6ZQ7o=%9FIp)PEXa<)m3`qWrMzD zc=$)6qo=c-bp4*8N|kQVvReQRQ;3Z!S2PPA(Fh`#H2jH9dUBk*MCYL5g;IQ7(gCaL zMe+;8Tuo_oi-07DAD{6^TlZ*@QOkAP%Z9Uag(k&+%x1}`l3@z9a8>rGvsEF0oM>_+ z)+I3Vpi4hF2dOMY^%f-Q6zl{7t7?$6BrNp|oymV$Nu`$uj{YH!LdFN^4fOWiCPPia zpfW84_WbJX$w}iSd-72S>KgM+K}Fe<-kdX7w^k|cNfc5Vd9L8AYZxZZDKmvMPNP%* zU`u1nBEJl<>{9I0*KgQbFTIk*6jCc48QhgblE+eL^6$;0vck?%7rva6RSfL5T~A= zZ6Z=02q}!irx?4HpT_6}HOJVr=O=Ibd%+R+_kF-nlVJMwbf+(%mE2{2Z1hN1NM-@<$WL zlneRu&+hB3WK(T%EP`Go&p2LcOjj_lfDj5r!fONZ#uG^}goKvG2cvqtr~3*fxkX$+ z{^|XfJ2rIFG|UKrbfyUse7C>3c!E>FCGf@d|RMf zCl=_{z6PdIiL*E>-f&54=X{!QP~ql|5<4bt#O) zwOSy}`w0p#yvpf#Te=>L+-%$|(r}w~#;@->M#8fxIVzpKns7r`ix0_e^UsXNb2V&* zs_mbl85=aKD+WJoFSJ@nx zI4c?CT`SdotJC3V%e?s`LTqJpj_exYzJJ}C72q|f0RMJYO@3TqdbcFP#_dFdgugLX zwDq-%V$I%_bqQH(klBTH7J>B|h!6DJS;>`D{ciWJe)8uX_22~xTMxsmWATa&RNA9M z&L;yrSN=Btsm#1GV{7m3%(gBVBGJXg_<@kS_XB5o)CS7rkj~bSuUBi}U)tJD=#7;- zgI?*Ly1SoaV#vw?C*pQMt6G=8neF24Z+E1<4^%v9hAW~Sol7~r7%rv$h1*W|#*o%J zu9&n;58hU{J%OMCR8|y!2a?eEn{DA9e)lUt0BQ|GJ|6p~fV<1xuc2$R&_Iu@E$m3V z!1!s!QG`U=<)ElKlGswDpbzDU7ZV%T@MEekTd%+`TQfeD_{`cfbvnxTw$#u8Di3!F zpUyShTgaJ~QE(T#&dx0ud`P_Q;bmVwj$1U}*oKY}x4#p3+u<7=U6Glc4xu5>yYh$t zH_}&&KCTIG0wVpDTRqpR_gi1gF-14>WMnV)O?ZX6HueXSYn3}vMB?Fp$-f1A$hRPB z*gj^v*a_}3N67VFffnlV@raw56_|WWSV~RRQ5_A0ZPruGiYcauikzMp#Dz5OR0W&^ z71*!~HdZ)>*D$rcbCv>j91jZ&Y*l;z22UPL;+0$24^+`@6n<_yuCzHc3Zfir--Q=J z<_@OU6ejG51Jm!Y5#)Yg{1RJIMJSE=sRMT%UJ1#%*}o`EL#IOBwda8+wlLsAq?RW$ z0^>uOYUK|m#=_FOj)?INPtU;EMF9R3%2G&y-&?6aNBileKvO#%X49 za`FNmm95!&Xgt1bl8Q&-m>J|483S-rXDa5?-}#Cb3{rW@Q`51gbXy5Tn3bp$R zwH58B1ego%J{}fwF1Tvdo-`ByJqos&zsAEBW#+W4B%aV>3VaL;5=xfS0@-miO>H#E zR^ww>{or;2(S{Lgz+pCT47CAu)iDD9poG*$l!;JuBh{UxZcCuBr^J0|Uh>f656uM3 zqbU>f`hRW-4Nfa52xkA-bq6OkIhM1zK4SeYj7`VTBBnoZNXkvjPB0k&i3ZG|5>ovX zAO4UeOU;$Eq|Oq@RgI{s_fZ^H#~A}blOxMjnAw3j9Pzmu!}P&*w*F)h;@;f^&X7-a z0z7=SpuU-{!;0~ckcxYkqb1Yip(PVML*qcym_b<)QWf%VnLGleG2&V@vN;h~+oQTt zHH+Y@#L0jx<>)~aOA9%GuC6fyBq0YL`C7&wgjX~Q+k?;y@&TY^L^qb&@Mn3!K0B0H z2`#_Y5B=X8vt2bs^#ui8@@I6VF2wrkvK(<%O29EN4g!~spgFza-fr`B* zR{()0!G7YAJ#he3jR7BCPJ(UDWtm-Q$EJwB+1b_Z6+-Blp<){#N$O);L_2ENyI-j= z-oIZ&fKa^a<~Ym6QtN(jX341)5rnNd*S8QW5PC(V#mHgp1X4{p>i z{9_m@?+dRj#eU_^%y?D$_VIJq24ZIxxhH-%^Q~`(^5xj~mH}dC9k~Z_{#6Aedf~Ox z@3d9&y;Gd&PDT*`d=}r!p7Yhf0N6G4D%{4uY>~fhDShP7eCoOT^;#@!Blinh5|fpo z667&2C#EMeJcAKo6d^yP+Ma{ffKgiqe6ruK2HnDZ2!FUxD8WP+4Hn24IrG`jMKU0X zx3;Ald`0$SC@}rX5hEdn8cOuQA;>{EVy19!Kngq`+x7(DWche0fF-YQSd5TR%+)Jf zgw{7nQeE++V+`C+Hn$HSrK6>qN$cd$DtWjMo~xdI7vAC+{$eeKM{_4OZ9Sp=wFx`+ znnPs&D6G8DCzgjQf98J}03OgjZ}Tk>rVesAA6tXGL+)T#yyK98eE9sgOgE1JNI?{W z0j7NN!}%|OeYL@Yx@_E6@pRK7ND9$LnMnxYj9+%m8ApORfJKrN{7h?Pmf45ihW`yF ziVyCd~(gn(Dbf zDG-A45^Q@?q&K{ibSyDdkb{?vS8WR#6|d+D?IjC9Bgpnx;RD9PAb+}v*f>Puz;zhm zZ{_x#J=5YwBw%2s)dF2(5He1*PR%61#aWLI*RJ9^u(rtub(`TprI({yBqO?>a6DAU z7t9*Ala}VppUqT)+$Y;2A~;hb5GSL>V9)sAGQY8hh`4QkBa4-FT%87TcwqyoWPX?Q)AQ{Dz-_Ym?-~7zHOCVR& zISlhCGRQhj-#slBgHOH!otZRNT%UCPJ3IhTF_p~M37|6K=({7Mkhcd4|4j5LX3X?G z19>wQTS$bRd{%_`1w3TwNLXehu5%qyv|4!EvJ43=dcF)bQDEEZaHIw8_p>&vfMV9+ zDPl&-?D7Qhn2I~>gmAe6Iaf0)5`=P@s@JY1Ht9j6jzD{xlmErt=i7sxb$Q&?RL=wO zabaT522p#{%&Bw+9hEW3Jr%nrniHY|Fuv7da6L=+C^Niqj2~^Pv=!RZMHhELCQ+-7 zqn0NT&(m-vDG6$d;O0k&?$N;&$YDr){?nJi#cA{*(^d+|hq%Wzj3i>%=8{F|^%MA; z?b3W;#Pspm6MTUPg)r^nI{*FW$?gP5Ltxs~bjNMnD@_Bh-DP)2gx__^g0|cBc1PFf z+raU2s{{89&)n5$!`vRZ*aPn(791qMMI{PWxTSny=z+@Jzqs}KfDs(o>BD}({eb+2 z01VXb6(xb_OVMQ|D4(ULhSSK2NM%GsB*h|G5K;Yln4@KokgFIpDaVqOs$>SpX$2Y= zsZYk$jMUF_W|3_9qZdQ7X!H7+NkPvs5DtyaFL0}QVmTG1I&=WxCzH2-#sW!?rHBpv zfh^Mr^7}f!hn-BZbEr%A$vnUqQ_Kz^?{(OZ9#|;!{bUX)HOVxL%BzgiFUh-dZ>dry zNOkDk4p=?V)yK6R+P#Hp;+_GsA7bp^H0ooqdvvc-)^V}Mg&+K!6SaFo_Z#lvAGkG% zLew2X=vz`vA__j8KVed?x=EkBlmYN@^1-jX@m^v=lK z+4{6usy6*n9Pw>cTZ~&BvGqvjPgxrxsLG1)dsC{89CR#WPbWLidU64~6f$$%873A$ z@&;IpLNWFq;d6;5#H<8VU<|y^6hM@cRA%J9t|p#yS{!5~%$PEJ)!{m(rjuCy#%FBg-uGWT!I7 zkTT(qoaT2IOBU~R5okE$)|-P#LlJpwXhejykNK!*LJh74cE{}r{3DO~8qgT0d_hxu+&sg zqsV@@`hnL%Ve7${w;|Hqs+SFKV4p<2 zT(&0*cT-3IGV#G((3YL#hFxf?mb5{uZpxIuiH!F9Fk=FULqxrUlrQ|vcL@ii;mB)eYk&GByxBmxKoy?lmZJUgd=F&fpZYi94D7k+BP# z;{`8Z6~Yy8*b7tRdt?SmsgAA2V_2a9(`E0_vk4wgg{Sq9jBup)t6*RC3-K)IUEz8C z*6iJY{TZ+c&$CGVn6Uj;kuRShC>DhDw{qGIxe}un1^o5ox{|vjj;s0C0h)2Z+*wH}O zK>4yo)0DzWq|-b_9!fAm@Gt&V%@DQ>f{2L%8*Hh4^~Z@6K65K;3qR8ixqeF!tq?PEGmtF^g=P&$fOO(Fi%YZyMq z*xIqay82#pTmKJ3;s%-IxSrI`VGWSeaJ?kWAO){RKu&E4min0cqb}3d_ z4t(?~--J;6)`rEG{zG=D!LdnmUvH>BuxtaNn{KAZ%T{b>oEz7X^_8FtFFGfy1Tl2pS?EL zavt1W;T;ZC{T&&w^=rP!Ga-Rsl23hko2|HVdtBbeQH!-`L2fPdz_Ykha9Vtn20)vt zS?B+imT|`q6WdW)9J@{Z{)cj}D5mTPotDN8p@v%Jt`AhK8vhV70(PuacLecGVg!|E z`L4w=EgUlrYYD83IC1qHHLuS+A*4OYQy8gr5)J0Qb{aFeB?12W9i9(DH&;#F9^bnX zSSCMYhV_J+S-{w#!e~07Wz{b36hM6fuGuo1m0Wdlg9%y(a>gS8X# zM13I#P(*@FHN73N>9bbEsRbG%9iN%R zvgd!o5YIJaHTJ@R!k2)SvG+GD)nDFao{M}8VsM@*JKulRUOf!`K(xd7j(9;*S!{^X z2OrGJloyAv>=@tcQiT$mF8r0uDZPe(ip>{`&L*_< zx9$qM6={rNMuneSZ$w9#SD7C?M|@i)0H*F&P~w+Q^>Lxc-Tt4xki?7EEXx^FIYL)q z@Wf9hD1hqpZY3MQ$*|!L>SCtaXYZ(BfMR)w&8u%8Nv}ZmSCxLA0EfzEAJ19~kE`(9I-zVh zD*16(tXO2-G*9Px)!H%s6i$;Ao>?|Cax`=2IRe&(6E#*FcjD%W%5U7` zg4uk&eQJ zg4Qt>s&!<=ZuI=-G01Bhe&&?;iWGeC#%L1GtmmcNHKl@_e#I=pms6JGf&uMek%1I% zH-$WL8AM!cWJ8{TY;JF5squd-On0r%<=U#sxaHUDEZuz2oY<@$;<@?E2w2$gIZ+Vb zES_WW7W+GSd_c@u4r%))N;z~3tKNX&sLP$twc za`?mU=22EIHBS8Ttd!5VXiG`o^8=p?y&6sL;{Jwaru~N8nR&hSI7EZ}9oC=gKT;f> znr6rJz&VGx-rRY;BPF}qWyAEqeN?oF-kQ27xTfwaIk9&U63eyc(qbWjOqOhDlHITpHOe!KXsR)+JWv(SS-`%t_pIit z2xA~~H;s*Nf>@Y3G_)GA_SEk8ihdd&lI7qsW66b;NGG= z+w!7P%_Uor73ettPON^x>)mr&7?RePSx#EWZp@w$I0vYP{606!= zN2}6p=grz(&*sSZAU7i2evE$Y95yyzt%o1FjSQ*>@L?;?^o|J-o2rN>df@y5iQTEi zH@Bu9P9_!)iVlvMkxjPUk0~sks~D^m)}GFL)ppAyZnWKx6ZNKhk@Kw7>?4ePyR#r# z)uK(?zmYf~n?%SUH9L`8G_7H*!Rq6tj|xy;DZ#5zv6q|qtu?`E`2J__8S2zxN)*Fs@wS=Lt-EDe zpN4VAe%rS6QDrGDd$Pn%=r+>Iy33m*zfIaaAPP5wG({${+|522m!LaKdlr|0&CJkH ze;%F7xn!o-UiV%dn~W=6u0#bk(XH;;a!9IV@>hOh2F5hMrkf^Xe}p!^b?d%HI!nyu z?;4I}J=b4Mk=vMiMus;DXBYUNoI0g%N-rXexeZTiXEkDN!;Qr%Vm-fQJM9mqwmW+@>PuDj`r%|V_A;Iih>)u zbc%VS@I>#-IzKEa?J*5+@4)?T+zj=-FKi%c%-~^;NOgD0d$ZQChI>~z_^}9204I@1 z=d?y_^nYqWSpu&0@gwD=N|Z7uu(7<{i)+Sjft#dL>J@jS&nkZmvjvqDx_A$$)Sau^ zZJm>&OSa7bgGM24eR<1G{Jd#$W)noBJasp>83al~7;)3v6(g|ElklTY@Q29MEF2Ay zKIbyKG)dZg_g+wsca(5t`hgqqGpydWhyMec{gsn_+zoWLM z!Fw1}<7jj!fdSxYr?>Q8+$L1_U&9?=*j2AfFV;Oi)EBy zublVz+in;Yp<)ZMO8VYWn5UU8ZWL^H6 zey_q?2>hsqGc|$)98=#_`;$-4YAE8lRK$@C#gI<}#`s@Ngau;DrQEFM*!Z+nbzg$H z>dCb(;253azauZzmaKHr0v@e9W%4)i^jL#VaToX_D5|^`B&>sevuWXJdqcauDCWIm z;04MmlA)BrFA0R21xopyQPsWDmj}61#a0Xj7Q$usMeyS(8XeHEuVksq%q+S$YrAQ# zZP)|~zWCV9Okd$N>!Dl0aApqH{}3fw&iWpm;NFSHz8dHNqanZ(1c(zTgl3qiE9ckY z@eF2kT0s&FakE7hDIl{s?(^nlk$U2ocO&7gind>q1G*W*ZL|<76Bud9Cvr)1m9!@8 zP)8Ebua7=5v_M=9D8C}b>q72uGYJMGTavf9RV@2eYxWoZ0Ny zwne$NMjd`k$4Z~3<`t=@Vj)*evXov8Fj+caEjB*AH1f(K0Eh?fxQ8~;R-ak&uXPsY zc7-Ugpi|~9zlM)wl@>g7%gY%rxWL=@>Fe3`WLU~l(N3W!KqGoY-j(c3K7KS?dRA;; zy>FQ)cFFgz1-P_Clcj@@z^#vf_wy4naQv;0ER~oCC;Uq}J-;nxnm!QvUKCA4^?iu& z;m5sf$4u_50P7&D7(Pg!B^K7Z7AybRR>9Z|#UG1$L%Jg^KT|C9NxV0_#1wu^>h&q0 zm8(Y>JGe(OOnmj2nH1AX7+yjz)sS=3w|+3C;pGdb|Bl8V!EDa`GgEQ~_kIVK^a(@Y z4tHpFe&GI&JD>rC(&aB1-X0M*3AHCHxhTE+rXVse40y%7#7s&WY`?a@bORqhX+Z5O z;a5WT0BQaa)8ptUxgbbqUmyR{g8alcLPVJd41VIH(%eH&_R-Uqk4D~0!ChbVx0A%1 zA#88%*Kg3xb5vni@Y~CfH-KRJ`*6eQKaXY0yQNK8GveTC?Y-+<)xr8-dTs*Z_ zk!sb=z)(sS&&7$tx5>biOBPS1g_%D(>}iaE181s#1KasalKdAH{wOvrh{>3btx@at zbOvCIe@QxMz2?Z(O=sIrXEj&Vv=gGD&8+wEKEkPodlF=#ijDh$1K9nuAROR+VJdc2 zA}~Aexky&DdRDjm)0!@ikE01?!k(xCnU9q-L8r=V`UltKWq0bwkqf93sT(*v`<3v0 z@exkFa*Zu0K=DV&m|z-NZuCXYiiptthZ5mG!K44dWsd~^7qK-ffN>naK>ia)YW8Mf zi2I*j*&0Sljei?tV@tWTOP`ik>EFn(^%{$_@o>J>x9a68qA zQxJRo!eRa+Q<%B;`?|iSfwqxN9Rsnq&Hb`x0IUo3EIiYj^3YvM+XRI{>G3tSgwCfF=;Erfk=ylg0@e}qWVgpTQ0O?E|f*g<`1$R@2S@P zi$e>JjUH))?a$ z(i42e$_AWzvLGX+&&pW@G`$1t7G9K~^k%!u!?DmKr!l{IPV=%Y5AnhS-0`N@b_Axm zi}o{-uDEG>w)+*|lAnSW@Z3NnmaM;#8XBHXG~t}snf85Y2Mj~7@MsFQqbttx?>%Uu zfSa$_tX=lD_y`SE#}b3E2&LR%y0p98)uaJ}m~VYD`LRZ=gNi&IW>C%QawZbrwyXqX zJcsjnCr{5lwm0$yDYDZIIL|r!>RP7cSIuJIcVIa=P;E)QC1oylW%g`RDG!S4wRJj^ zIWHmVoDN(C4N-!N*lA2pfF;wZgYQpaKUF|CEjV7c6k|faRLQ@N z;*(8ENhW!S?6OPqgr6IQ7kax!7)8aIvuhvqJ9HJ5%6W_7SCQr~&g3DglUVhui~TNC z3#p_iP^Z2+bNd?f>gSp#qYucfwq6(po7y$;=a8l^H3+Am@1;i53RIQ+0jr_%0|0BF ze0m73Vx;&4L5#yKcEn~!91r0dA2BCcPSUh2cH;su7{EIi*@BUhK`P}`Oq8#ilxU-r zVdHC8nitAJ+CnhGt_)%9x+Drfoceb2pPrkw7!CH8II2FLX{{HO3@IX&{fGAVJ`Oj)26-OhTGtOF$*lW@A-amQG}M zPvm@=qm*AQmYo~4L&t+RSuL!vn(ute`w*T!A)eWDS!FMInzRIqf|8laygOug+IZPG z*znkDzcT!Z-*W+KhX5qEOvfl1^#xG+t}2M8aV3)Wxr_Cja|h6Ttqe~g2~9+jBQpra z&1@7E4n$5QRal90{_#}}H5~#|>h7>HTpRz;(<7TJr*02H@T^CB{i+87MBqTQUr$8a z-w*0QjJ^KgnR@CVH0&wJ{A=P15*Os3_A%U@+#|S0=j0zf$pN6s_2smk{-9r9WlS@0rsNKj> zQj^zcMCHjg2X`O2@igXSE@2Umo=x~-q}Rjf_n2C#Kpa`dR*w|}SG$R+YP?DU#&Kzb zYskm8smn&%a}H1xYf0)#?R%bQOQgq4pxTQHk_|opg=a?GIJwVNikk%IRxqVpQ_IaC z3Mgy;8qgj?r{pwJRGUx8*5)ju?3|-u~hp~#Hz*YDgf1EOr>^l#NpNm9}LbgyV z^CbV!1ZFst0%$n6Kn92= zc2CouAOKF?Bp_d)bady$#90NI8PyQ$1{0Mj0*AakZcDA-HDOOwdzY1YU>%FWNiKx7_jnNuO)|ru1c~ z_*1kmLksjJxB540KwIZ;hV6^TCQiuJ3|%}OBzG_TRo%BuXh6mC^1^|G%>xIQOMp=P zH2{k-Vnj-7fFj~MtZWOtv3B2=pR_v`p1OT}pw@6hDGJxP7XVGq!BX5&-Hw{C4?zuM*|1Xw4z7)Yl}u0z zz=RCQ^9^n)*gvg;_mBkim>c0c4l@2^{3ENf3i#U4Ni6%jK&84#-Bz#%y7wM)Y+(ap zT}5hk?pglXj^wj~T(Ld#LY{7=`sF#-`E#V0$Iw@S&NoR)=rCvzi1&c5T>oNuaX|Hf zXu=hPeb*whtl1QUX7gxr?IYI4!jwR3`wF?_u?T`4(oY+XK^+`73N?fbC2X9h`83My z_CT-?vAkRXJREdKRWMzh$Uy-E4uZ&oaqW2wGMb0WGfY?x{KcMR{3MTcVoENDDreSF z4Avk0*7X6SiE3hYJxo5$uGIz8bpQqthk3MCg1X^O5~U-Z#a44#^@fF#js9L(hnX$e zgVX3CgOJjGXz7?FSrYiahaC2sy6ekSl{;>#vSp)v{>E;^PbV)`iS=Wbwt2537Myb< zc#3lJH^FDxM@y;sbxEYd0-64n0-Y4bj{%oOCD9ap zR5VpsH8&0Ko8Q2x#RhhI$wW7+dnlhINV*<|vAlY+k?q0yml0Q^`|QuYF6@hIZc976ZnTKNEnSMcpAwNcqvL?s%j3&doF3l}*)MZF9Cd!PmuI zbwN5`-vSh#x5OF3;}v8I+QB+Kq_78lo|GIiHurYryswx$n9p{7wVjYrzS6x_SYy{# zRd>yGz^K|ih^N}nF<=|fgjeTA_n@%A8dHfB4%Z^>#MKEP=h1#{fPa7B1D;wQu})uR z&HPc{Wy!u)>FbBO*@EqY+eX$tEgK4Z^`jABI}}{3-@|Jvq!xqbysfQ zB~xfn)e9U)0EjMtKQe4DAqeuO<8L+j$5s(tc|`c82D$k6*YVhC&KR6Enr&a!@?FrC zGLEbaA1rX&$(C{`BkIPRxb|2r|Lw{PgXenWPQ&I7a?Yga--@;759j3<1AAbu5(*T~ z?Dyl2g&`6Lk40VFyB7%1ye4Pdw$dA7QwtD^#N?M;1l-(#z8drh!S;6F)L%BDn;v4a zlYU34cL_S$@W9;Q&Tqe>6$tho^*aDz3&s=#SH^fg?BMWN#ce;$8^Gsz{wjYLh6{ul zeSw_(MgC^L@xTd({X>ZUDo;o3T40@?^j;7!j2MZN_Xr3-ecSnn!(fxR8MRg06n*6m z9Wz^q)=sn8%6FW|&3;c2KQ?DIp8Mvlt|`sRX^P2W5*201+#ZyB$MM37H+;!UOu3p1!e|A*}LEtCpp_$t)Fouay05 zB^3Xllh2z?=lAdpLG4>0x%0yQS$(3&aJHMD=7y=KDvs;*Pk4Gkb1A;02&2Z+XJ?9^ z+Q1#!8Xr^K3TQx}PpsGRsn>AoQZRF>ECz!sKaqGsG4fZv@9*F@4P2I^adceQS=(mg zi4k)-?_O6}#+YHOG*%$5e6~zbGFN|g`mV8GS~D+{IbDWGlk3t*k9z29OjMn-0PO^} z6AqfUkK~vwL%1jjwjQBL`7~>Z;WeVdf~M~dRQwrNN)f;s+6KBh4Fta|zhApNK#aq| z5{gf0&TO;mu+Hu{K0q)AO@NL`pV|^69%|AV$>D?}X-S3LrjEd)A9mr?ZM0R>9Vfz) zc@kZ8Dj_q6Wc;W3%^v6#n%o;$$m#JTDa4tpRJ^E52)wOb6pDe(UM(^tPB}>{oAn%x zKZbg>R|HVkrax@WhG1~A!9E`pcGZA_JiVdhj_c^TZkh0)u@mML`4-4kJ8*seG`bLN zuiL&b*Q#7!FH9Nwi^!&R@OI4CuFkjS*0xN;M=i}_8v>+vFD5n+Ya>W3&MJpck+)_! zqEyW?aaq{dssD16n8%88%q`rU2)hcDI&gjthXAn8d6oFshrgTrk*s);c8QarFVHw4 zOXYY#r8zT1=p<8x`znQ$NahH)M?O3%QOz9=`)$@$N&-2{PiGzsGkSHL){bQI8hNM+ zS!&i4yC3VbYyK9C$Icts1iszsoe|9KTR<`;k2KdME`d0hc#`}6?=~+6owvk#A~J#K z1!ZmYVf1Lh(Np$&!|Nj^E>FEErU5}mq6GGdp5q*P#6#Hj5IfR7{2KkRtTl4oIeC^hX8cE;)MoYdvhQ(Yvv=@!~J^q(=WfPm!tXDCZUJPG;lP^O%69UAGM*~s-_5{*wAG7k-O zilZb1&iF#Y11-n}@SE#0I2<89{gcllfn?Y{(EkptCy@F6XZ9r@5(JL;&w_SsFvrEu z3vmD%#c>b;Ym@js?=OZ2y;-0b@(K*;pCuFh8|cuV&Hq*iKrl>fLf~~c94zp&j|90{ zSe*D9G?LEe{hTE!Ok#dAbfIDa<3ae?K$LZ4^yczmN*?(C>|PC_0AYMSh#Cx_a0whK zFs>dB8CZ_Thy>8G`ovhcKdYO-tiC`5Bqa?D>6$`n?DDdkRSX(uUG1Fk=(;2C5D%rSAI_~ z>rYc#A9sgqUyydPWP+K|hYYlg&ig~)&XG;9v=LpB^#Bv%wfIe;UMoU1z41w2D=qaj zYP+P2qq8D7k)YsE#ZcuiMe%E^4wT-=7J^=j>lGMMP$s#jbc`VJ5cah=&vU~1yBW`O zgl$*(jty%0o!g49Xg}9yX4wkV6nX`t#$!I_J{Q!Ql*LIH@Mtt1b0oK{A2p5B>bm5* zdn@~?%Q|(i2hsX^@_Sz&JkOraqO}wJ{Wt(N;JeT=lyzuTCXY&L25T0J3GH}tn-QOx zDHXzS!riWMPgdP6coP|Z;F_C|K53TwgzqN#QKb1&HdlKn695N0+Ozj}{f!ab6|M4nQu{ZhCLFMw zWb{9}G2E=p@YOvrJ_m4WteHz4W{05Gl@~et8xl@wJ`S5UaeHz^L8c9wS_gP0+8Wt3 z)ex(IVs-1O66y8k<8;3eD;3MXBDCU%;JcOI_cl-sGYx-v7bD=ekE>X-LHM*6a$UJA zH3PV&@ml44T~7nkoH4c4=QvavsG_`za(QvL4tQp-1&EZh6%eJs9|a z%h#g3d{$=8p}LNi3v)qorC*Kwc#6mQt2pbmp=Aagbx6M-1`tIRD?1RN4HzCbrE<)F?b!rooROT4<4X(nw&3ErOfVuvgm1kGjbX!* z>=f!dsvIiIg6rKudn^f$hN5_<&o&-Tz|uLOI-s>?Uf_wVd1>d8@|bWFsho-~tO8D^ z@zm08>^H`8d^DTfPduSw&%#L=Ga&GOR_;-O%D&s;YZk}6C5+C4(t617+rJM_-@pe5 zn0cxUOJ~lpggOxt54MCFp0)(K@Ct@n{Ho{u8BziMa>d40v!84okQkaNGyYRPZs`VJ zkIeK8#9**^ogTBb^Thk^FLVk@BmlhOE_1BU++cT9?;+|f){&cK56TlZ*LG-ZOO#ja zmF~%R6z$c3GI}nnTx5sQJyo`69wYGsg4Sdo9(u`s= z_9+h17w7y+ckGsU56r1{On+Qk8vA<6o(#= zQszqv@OMIVq}yVW$595a$c(Whkt?$geBenLDE!C+9`lad+Mkj`d?AAddU|I5%p6M@ zr|J>1+2DDps*lh|4&~#6UzjsDtZ=E@)Fgj!Cg%z<)TuJZ*Vmx zM-$!74V|{zd+Muf`O=?MM$Qm z_g!&>$Lf~IMq06wsy|Vw;gG1HoI!~_!|@=q8S=cy%GV@{&Qhbb$uK|U&p8urkukyN zf*fz;T=*b8fuR+rOWsiO7dB7mX=n4j;QCkgH>nJnl<7SbKLFXnF>~jhY5|gOoI!Yx zDbzKIjKhBPiuAki7{5HC_i+YRl*(!tZV?$ptPIgFv{=F0<0Ee955jV#JfrXZxuFGk zF;?8wMpFH3l`b5<`vS+dTL&7`dMR=2`k{_VY#K-gN{-odteG5jX6!Hv(6vz##5G=y z+RxO5wrNUDmw;O-*`}o=+p0Xwxh1lq^8~tIZUaH|z3crcktS9u*pTig{x(s5(rm>_C`GkVP4|F&CF~9TA-WBho_uMCC|u6fEpJS`LLZ=X@E65XOzVJJB|>t^7BeaXu-rjab(cG&BRhX$?=>vtgZ zxgN)l;V-clfAhf{&`v4cLSPX~q%9B8zMNC@8g`oK zk{CZjJLrQkF%iP6@N@^R+p4f+U}w-~(m%Q7^CIlBY$-=i&4HIwv>?T2_~#WD9c!Oe zBB3YAmCcn({|?L5dE~TKm!~d4m#fLwhtp;uFug%{I4eDe!Dx)G0eQp_&dgp-8rR(v zod|1frSW{7lm%zL5bjG$Ed_uPR0ZB;u_Kzzh1T#g-er;0M^`8!A~FtI@nnvG3AeuQ zK#pVop~7NnL+jNM&Zn5`rA zIw@ECPy~t+`r!mN1dIBxb%e^#8r*f(zuel51a&wIvVFP-6dBxE;>&>jN4zI`x0qBX z?FQe_y8HvYm^kOPQl}DI_-W?Rs8;Opi||zJ(;)27{+YMp$&1Nl_Y!2NW#1UuNK>Bs z{@U~n_{qXQ1V;ES(QoEtb3DuO#yzo{bo zrscJH8Yhc~T8Oyd$q|~Bb_vb6^R~?v6a<#X;L#HTZ={Az+!_~E!V3BaDHo4~qd0tP z`XR(um*vDy<6r=#<*uLf#-Z6JwTm|-&p6d@VFCTj&sU78>1D0pCdZx)**I;F+kY-l z5UttSeQXv$+UhAlcTABxEU$Y4dr9go3ZmptZS=!EaZN^OPC}~sVRhdT?fC2iH5Z7m zkkSo--tn={zBOc%+_EK|y?0xaebhZc#P@T5Jim(_0zE*)p<(+Rra>h2RgjS>kN_M{!sA5;xXPcGiet^;ocXsI?jQTpoUlW;tiZ3C@;U!SKF2m&h zH61%-ecMu6tEYVx5%}4<7FtuBT*mk`Ts+Yx9eU? zDUS`V*1`ZLW3ilmAF*!5%rIB`yiS*E?AqSEsTAz+PO(=D@@#aj(7Q(W8KjBXb~B<9 zCqnGwg^ig$aZ~>|YtL4(Mg^as`S!CXh`C}33fcYSgZS@sH?sYFQZZCki$A|3+f^kx z>8u?Cy}7eNs7G_7TbPP0-g|Yf&R;Li>TU+M6{P?J@85mD^4d|k&)lFiLsN%|2 zC`@E>Jk`v!?`<0+B{F9T$EXq~uk%OO62nfRUgc{EG>e+mwFUS1)c~!a$dYUnvfobq zG=5A4k4=3`H=?qR@nFA1fe@#Ur6P)h#6n{PS&-+-IG6P;Ez}+j`3FiexuQM@hFEkh zEOidhA(OrwK9zfr{54w=^FY^-dq-|SfdB!T~ z^mDWXQSeH-&phyBs49_*ddwT8BQ9Sk%lJs_#jjXR1iu`CNar$e{{_O>s4x2spN4W1@gKo)LSPn%7r%K(1N#$}+I(8!=CN+L zSqP+mhS=rMtxBIfaot2fWp^+_AX6$IGH_BC8wr>Ti}EQuu4VOIZ`tkrl751je})+x z5}cfV6b<29qj{=arx>wJieuQjX(@}e@Ve9li3dVF5&02XfBl(R zTg{eIDSMnY5#N^QNQ$?t>K1RR7Q5h9l8|Ofj_WYb=1q<+OX+=oZ^=KO8@GXvFBi;K zK%-CMo;-|?y-N%iTbK|QyKu3-$XVg8N{M{<9 zp5zh-aFxL~Rl$98EY7>D zzRq*h=g-8cT=TnTd{z&lp2QtRxyOXx3D~*M{d2=HSNizakLoA(%5l*f`}evOir@9% z8@5WF*OnzER?S&dk^DK4s^ITj&rw88TtZGQQjdXwpQFkwa}^FM)NKn^u~8EM?#pt` z8_q5bx=BQ$SGK#lzL_gM*gFHNh*`P8dvlcnRKb&&Wq`|~nXeFCH5D|Qs9J6|K|-dn zp8a!!&o@k>Lk9)5*x^t<3tWmC;? z8jrbB8)S~i^jV9=)L3OZF03=aRLo-C?4(Q$IH0=Ik(m%1jmEs!0N1QXsjctF!Ta2v z_~;#x=);})OKESM&cQ(&MJbHDyD~mW5T)R-BRalC-PyNubXHAl3pFuLkJlkyi*9wX z7W;+$yq7aXep$tu{oV+_U%S71+mNG!+LVfyoj;_6V?EFg?;TNS_WB9{LPq?cX{RaV z2%g#N1z3R`UKfILtU$!CdZLiZh{$_{c<*1XuD1akui)Ja*L1HrU%QvC(cZU$tPq*@ zFw^wZ{y0ak$oGG8Js`5QUTNH0&e2kpB`7l)(LGAQ$r@aAF%;CbDEF>QOIU>z zXO2+}$Y;{Z2J;t_#FPRgz1R=bSLlC=N;p+8vx$gP$m^)*p+^wOYnq1MM>z6lcWB0H zFi#lw7xZH$lTMeE5fshx(`rg($B7diEUR_FSsqIOLo>vuvk1xtX}6W3(rMMi({=JA zIMnkaFv_uKVp|lcy~;>dkS5XX7ka?EhV_zn`^isq-7<<&tAqgxW1jYsq_n-~xC?Cc zcAVt&yFVGvKgB7pPi8msOleInM|yo!>!SP|ACeESdEcaLgT2(y&dSHiq2^VZ2M0&W zL3aEIWl4f^FSLyxOJd88;vOk~_$7)+MI$bzv!lpWud+Aa)_ipNN`s2b$7JL9jO!<$2ZQR;UG*Q*r~cI`S-jW$)ucMFjfHu4lshC1aZEIZWr$$ev9tIUVL zGipda6~jGnv(8x`cy&9PxQHUJFIG<4O~_IdEkd-D&>*$>YM`-3kl3CWI`oxzYSLb7 z3=)|T{+_Q1eDhA1EOb&oJE(S++#O}l`Mt|JTW6xy)inYj$UI7O(GU?W+wVlXmFQ*6 zRHV{ls@A(6S*@VUm1Bh+P1GMv+Qj-4*5gz&5S4x!KwB`5GWYH0z%=FTzqM*Vz#Hcg zUAEOOWN^rsUaNVRj_E>dK~iA9q5FxNl8?KlRc)W-;B9-C~) zv?shey@GT}r;Vr}3`JzSBYybp5BgEEC@6`$xlt#GVYL=YrMQ%L*Sd|Gf^2 zr!ZEBnn;u=CJ*OokhKZj<{E5aH|S3|avrMU474IX44r2a7Sx|<4?f7)5Os;K2w7n~ zjKOE5;ynD@&5XCj%DD!+9D-KAD!&TI3E$?PZj2@WaO4<|GUtxlUl`Vs8-lMh2UX=sbjsnrtUxJD~@weds@IOdMHqE~1Nw};}#?SkwLxg?z!<9|? z7QH4VWJIzdI{~D-P9!IA$7{YhK z^=a{ZrM1?S!7Pj`5AGP%R*s`uP*cMYV8BgALw`8l;s<@HM>N?m4%Sk;j2=@i&&ZE^ zUAe*E6xH+8=>rKa)Pk{SfmHR9}>^nTsJif$sigm_;Tan(Y0UWOV zjFoV88jj@JnUwG^vZeTEyT-cZ`E;FczD!qHI5SUoO*Unp3t0Yj5absRu=3>RZsRQ( z4BDE`F3@6|@5%zaO0Z5=?ye`E<>VQRXOxwjDSk6}Cl5T}xW73z*UXEIyhMjtNK!QZ zyZOT{&jt?DI=^^^zfMFWe6;Oy{_=ZsQ6N?8d)cS z`J;+r;bnVD1tTvT~7cfQ6B!>Kocdd1CWqrt$78fLb} zVrX*uiOZl?j0bCv9(|&273!j??g{Ri%rp+kNiJSakElDPc2p%0Hn^4B1`qL7S^S-+ z$#G=3+$=~@SK(&>XK9mdT%I-OnsfC?qPz%7SDh|h9lxo9 zutj^0hiM2pTB(h~C%C>1KW|rux-qy3XX$NX86x$YGVRr-2gc!N~+x9qT2+_Fz z6@M{7c5;LXK)39!gFBC1X)bPt1vkY>BKXVLjjDd}o#)rA1=-DtlrOY#7|CB2+1nSd zjBho~Drw8a#9@*m)m}@5zu9Rn>M3heg54|cHRbbTmeeIUGwKwx*DH}ob4h(wTosZl zn`al+EKh$O;(W@EDUF(&Nif-SJL|4Uj&+OBhj-WBN;StjYf~=O6_k5{h5Tmv_5m*Q zVq;xWYruN1Q~eyBOk*1ta;sw#K!9&1oH&ZXDUJ4#iO4Ut_#JVsF7#N-qi(VlpDwjq zqr%8m4HnuL&F+-QwXE?1W8CWzLavrkV>xZs-AAnPF|qvuK;_YZ&Sy@(VNZF49N#`J zXtboW3@7x~JAN*s=##1JpfO=$nrM2%%>LQ}Rx0m~oX zvEwN7_9;q@8wHKWPTy0)^Ty2)iNkL`DKKZ(!p=louJ7?%|Iy<5FC6_3yv6t*-qwGR zs)eS1cFkvO&=D99@hMF7AD|jdD}dNM^u#d=@h`6B8-Y^+vH2tjQ{OyCrEdZLxT3175tFMyL*!dM$&u-WflXvqopkM( zlZrN*O$ouRW?Pwb`otAXEJB*apK>vgvC<4zKx0v!w+~?b^iEAqQ8!Jm<~NNUZ<;+_ z)eaZJ@{Nz|F-(D~vZhKT9TH1kO_WfG}uUx2yf1w1ZD9ZKMd$p-5lF(T zQ`T5n1Nh}jPdMA6W{toxoJh2Jg0I@@x~^j~9#M)Xx@D}b4sAc+gXn6~9)S1m~ba3k-nk3Cle($ziFaM?%TpN%B{E6qr3Cz4*cd zQE!*#!%`U)GDvP)6Q1v=S?Ckx=oSRhK%l6t?!iPezps+FNrV!o%Zp?5iODw;wwR3* z(;w*ZLFlD(%^|}LYJpkAKtWcx_(@l6790XPS-58mePX&ue!}iv|Ec+TrMA_ONGiw( z`+K@m0ypwf0t;PzF}rqu8$2FBcvzB%A6`%G5oM|7Lh_xOW=L6T9$xERov)-aZecza zH+PqMID;2&)q`5j#YzJc>bWop%xG1KH>*rF2bbF`(7IX~(o~^NEAu9^FJi_}(JTdU zxNd3Dlx|g`Cv6z=-u^7hb+5o{jnKV^PBBY>`STZj5mRX5Rcd5Xe*oeREgJ(Si72B9F5Zexz+S8)g zz0Xp)dm+nu?T+f-{`4Rn9)`C1I8XuW`00Z6gHsUBUxv6VOej?bskSWwoj`%Yrk(&o zJ1Rz1v)=hmoPGB5&Jc8E2C*m3ntRGwjRMLO+SU176ia(wsA>_P>N&fYHTKijey2t~ z$q`~A8X=F&9s0vg6#7i^N`DYgmo|M|2(te~?FVUQx6RnU|5dtXqB*HKMpb!(SNd?* zXPL^&#i2hfJlBpl1ldDo{O%j{G(SjzEiM)mlOHsz0Ha$9RaP?eRqNnTS;q ze{tc$jBWddB(yiEFBi5{NH;V~gZ~pXG+;wYKpLQLOJwf8ND%nB(PlJ(iY<}WLl|P> zCVGWn9NX>#y}!CQb!Mdc#Dp3+mwTO-%LG^SWH`@#FSE*~lLY1ZpU(D^VUn#DHSUN}6c>T$X(FA=SxyUH|EdSqp zYaYZqmrLsl#F_HH#vIjs$DW<_6f$kfEAyhVekUFRhO;E?Y)^|;wY1MJl+{-(*SN6v zxBMB*aD;swcJ~OPse;jSzDqnjj1I=kx)ta{y4gYL%h|ZCJ6MT;{A~JgPh9oPimp1E zfL`pXV;=4MCTWeHWQa0??0G90nz@O%UsVOK$9m`=_7Ai7;;_XS=yG_X!fm<4+tjJD z%Su|BaHbC;UTzkiCu&@wr^#QgG-9l%3;r+FIG8ELYDA(7$$7~(JoeI7r zH^1d>M)i@_&#+a_p$*A9wgy<4U|GMKj#Ik{V3fNB^JlGMe-4ULqQ$&+VJDMbJviR z$m!9e`JuZ2*gK|(d_gU2TD1F5+5PB!E~uvVZ@r%UjGl9YY;J0Q+;BEE^xhq;C7h5z zvyGuA4-FuvterY+e<+{a{B2*LSh*=ES!_sFO0$(F^&=&K zBKZqTnk2M=!mwD5Vk;l)qzHdQaxr9kt!w~|6-z@^i#!Nj# zKN!8G$cEC3O!kVj_L|8QEG6n2Uqh293SC$K!xBMLl&he8);}KZmR1VD0)(_1W*7z1#{gJNr zp`-rv`-v>u1miPM?NxH|(BO*uEx6S;J3USru*=M+Am*nG%||No)SmhiYmkmvasn$# zJeVtP8Y5qc?I-!pzH!+R>S07wi%*>0ZCbhTt?~zDMwMUOX1*ckb4n25Y z2D-e?-%S5h`jnxL=0LC?hvUzxjyN{m-uAMK45{l)=ZCgSOg+okuUOk? z1z@m3Z+D_ri7{;Q>$WnLz=UsoxN`_6x4>WKcA#Q@n7gmFrw02vw7%5j`!DNkiVP~L zT6;diQDNX#3wGXNvmzhV_WGo(I>*HvKH#g(aAz{U4VRkok>o`vh=7uZ>0EM0YWWhH z0CIFw)xFV~5jg2e`G!0#O=fPieF>8V09k?U@tnavWotbkBX~x_w%8TmN)>G`^QJS} z8f-Y=IG?_a<)cCkBvgwmOm;Ja)~XTiLoRyT<+INn>f4xVrJ4E2W*&z$U7av_!LjPz z-_Bmm$NoG8aCWg_Y%X=h%d(daR0>=)b(J~lY|^PHa1#|aS7%CeEd-#E`bN_anO{-l zW+YX8m-sHYvrvw9bz#LDvG7obCR14D_E&NG9Fr$ue{;PYdLy!hXwC3_sZZ&qY{jdkS^z4DTQA& zgpU}RCix;e`{KjfyL<;Ge7`qn|BkYHOvcgmmNz@ay}_Mr9#B9drw=q=UdowmTD zRe6_KoO)3dT1Kw8=hP3L<2;*4&Hm!-kk`n}omX)Mbw@XGw0thEI08oJ46bd*^_qPp zt23rr@Iix@)h$@rf&^-M{b*hJlx~`E8Gwqe9cH*33n*(vzD46o*w4D`sh%~qVRdu) z$E^&POC8^pzz`Y~nu*2ocGH_E)A?9+i0?S{_f|X3Mo1BfTtbfJZC3NY)5ERagQefV z5iWL?he3RctF<(Q1_I#kjC*>B8E~9)%P6VPQn%?)wPzD?KlF?ru8!RroV zUsqO9&+#YNrB{<%A5_H1+lPx?BnHwbAUn6I!@C^r*KWe-LIOeExKpIUwRh~#xkn39lIJwpLeUHF6PLfnz6(%>4y z!m9WgFAol4^p{ksKtlmS<3+*)Cs`cvtC>zE9_fTo%6)DRLqiW1(stCB@I1JHf_tVb zPor4Onl%TDto9J4%d^4E(@juxAzg7YN1DXvh4;cgmjSl#o5}637Si}bdyfVXr`nM>_fAl|Y9y0Na1z`AI+h#=k)0|1y;j9^`d!`(ZZ zArG%$AN&ELPk~{d(7BMG)X;+SR|XiKWC7Z5QU{*|hCxK*N9Q0L-be7k(Gb?%px)C4tIfEX0bvVmNa;_uVp}=DzJi`R=^B;UAzo+;JXXiCL#x%J z12|Pk@UorESW*fjk)ukKvl1$l#IWTRlM*6{?+yvs4oI0#1UkNuyB&AFXdH=5~f?m4x_s(AnYA!Q{2_ zrr6427kd@cv*4Q_@7QH6dUy;?6yi^~11RJXi(}?f>jX_=&E?EJl9)J?NmU)V;!Qgb zvd5Hq^{Wsq8XJKJg*R>_YfFA4e3k(soZPVj1)g2ioyu^e{-)1@b!*(AMi+fZtPIB)VLWK+Non#UID-yikTfQmkD zdoz95Y4@g>J=qMSL@y~#OXDz2P{S6>9_vv8|ggKisspHF$+5j1K6*2lj5@H zpQY{q|2{}suo&T%&`a~9+}^o9^2dcBAMp?{OH69A#4nac+1cc5SVWTz@4azzgbSER z)r-k2nlIZBps;dKDk1JD@-)0&F7U9emg^z4gR(vB$%C-W__y$N+9 zm`}sUiNSN;^raDj_IPJR1Te^W5hvsSEJ7epkE@-dveeiB4*x+5Bw@?P<%T&--pumi}4`mT?l_)g=?%X^>;ZQj4u7 z)7?Uvly%v^?o(wjr~tg35Clhj-$te*sQyPguw$8GRqjnYS7g>`RqxAL-jrA&NSyyLQ(Rb<6Tq#-lhdgwT zNBEL8iB*wR@h$6@5HE>ossl>|iD&8=jA9ko^21bNZurB@1OS5}JRS*C>k+oth}iFG z*-2&8bEVN2@k+`(oFrIQf<7>%DwlT$xaw6+I7ZRzCqD_!@<~!Hl zMzJ$U_V0*;Rtijh#g*u<)a)|Ef0dtUR>@I#r-MeA@$bswsh$z;%I6`-@8{`^0u>0R zbn6HKy7|gT90I`kCIez;lnADGNqN?VFBgv#vMxv#@3`A^3ThRC2gZtjEiwmE!ZfE; zYxfSPA0aPXgH26GXv9NpFaF_{^RuuvJ#y7WC(1X9F{Z=FnmH;|aKfFX?sd;2i?Mc% zQ=s0dY}TPf!*Uyg{(9&v(_;l{;TT^_ciR1f&mKN>mjF60ybzJ$LZKZgw^NtYo(aEt!ySzMJ+qXvHaqiW%LjvjO&fohNYn=kp zY(Gy{5z-NMKXn$N(SoD)R}qc?!odRFNLdc-ZlOrMc|9C^GDTib)(H2mS{?JPdTPa& z!MO*~wM&4x-D&-kJoy8}8%>*n{DFp&)pAr$-5_$wcD&AXYxmLR$=YUn1)x|wg12Vn z;Np$zx3)8@RrCCYxf-CN-jd(QMKc@C$e--_>6v;p`6N71nwe%|Z%?!`&uib6Us0Tn z|M_FvKasVSP_~^&`ZsqJm~Ux!pe(Gyd)^!NCRX$+y~$3qU`pW-+E=QP4$qMul9 z2Kfyu80{9C*(ivm;NeyGXcD1LD-os>rZm&d&)#LS>t?%6?U(BfsUhtdfcn3hN*^k# zb;SVX6%NEEDK3=w%xydM_a9ke&%JD%5G2<^G#qTE>m>{F?U5?Vx?My`H3>gt(NXt(gr{}ptsENFk#wAAqUTlWzd8(3|k z7vAO#^~9~~=~X8__jjSLqN1kf<1L?;V}BdaH6vs+me0Gtt>o2OYXw`>M=k4)VL#pP zfvV*62I)S@wwlRWMc$-vt2*#-j3)iSteHGffjGC{NX;lZiCE|RJu@e=+(c<3HqD}* zv3o@%Yzw-SHPM_)f9tP$C*}M)L45@Tt`l-N$i$$?v8tCUsa^GpGg3vDlFxc=_wWLs zSOKRZKS-$YMPaUfK%?oM|(0O?E(J6 z?1?a&KpdjutqPWbXVuaDdRewKz2__n_ER7X^drdUSnCO=11KpBhxfSa7l6y(vnCz0 zFRyu(2j?M~DdAnePRyrlP&sw^tucy$8Bhc%nFni53u3+t& zB1y>p0uo2pA$di`=C)z7=;g|L`Q&vux&?j!UXynKjdT!eqCdkM>kQf!F!mDJ%yf!t z9XCHOZ}@TAE555)JO4RfqJ(o7N9suE5`2##%UJM%im4BJ33<{n)pW^sTTufjy={B& zbK9TtPwO({ zdVT7N)ivbw;P&wBP$$^kyZA|w-E&D(VJF#Yd8T-zpuw$tdUo`WNY70r4Ld6Ao-l?NGjOaf zfbDPK{Z!z6>&e6<_j9Jpjorg&@|2dmR#DP&DXc(x42W9$RGI5k8C~wu9OCOb$7_J5 zY)j3^p#YM8d1%gw?%Fp1;-e7gh2Z4R$(;}4nx7`Ys58#tZa=9%Ri<+#ltTrv?+Ev! zdyfmu7Sz@>!}_&&7o|z^Lre3-Qxy>`3-M|bQmCj?v~>xngPPD;Uf8m?=ZG|Jk``3= zr5-8^9{uXaHy5cBCJzHY@ymmI$6Ks2{mzVpm(M5J)Y|q-ODIwRwh;D(8MW!@dK}hn zg*I+p+NNV}_byLbI>m}t>4otz&K@qN+H3XJ_iBsT-x!ZTYo^*GF`E{>j;zD6kPaoc ztebRc)n{>SbKR;%e$U3+6qq!|G%c9kxjcCt+_to#W~aZv0&C4~Dt_r9xK<;IT+zw? zKDmt~!mOAHG+s6Y#NjbSmS{*@Rw2@4Y;ebF8Ecq5S#V8)>s066M;JDYl2~MqYqBCv zvh|IZN-=gZAw|mZP`0E6phZA*Xw-gZ^ib}m_3$Q0 zVchCSwD2wWS9R&JBdfHoeAFAHAY$0N<~}JuUO>#T9?n~Mq7x_quNjJRcdJonAGPq{ z&N=J`Ue(jzTG*vMLip`eYUFi}nxzY7S1e5gVv>y$>Dw%yYGYFVYLu%s1Mcf^Lj_*! z(7AEC!rs~e2*tjM*JZ*t{Bf!*F|y6WZaV^hP+l|3_iM6tXXBV(s9Wd?U|d;T88(=m zU|inC zw+53l4!<>~Lw?(f{yfmfr z^KI24=nZYaocZuG{#}crXhS+Z@;}Rknvq3nyOZJeE_(W`kf}!>YE`RaF`@^P|H#5+ z=tuB!F}@M_EzeJr&heWFE3@CbV&=Zqs2!#=)5IHEr{f+U1TN@)E$d#t_rnSQ$L{Lz zEYJ<`$Z{-m>|}W-%n(E*LhpwibzGn+L1f|~7$|?2gx0C1`I9ztVN7$t%oCw|gb&kY z_)AM0K610jO+1e6==3&kZ{ttNAF)ZpGuCglv zMQ7f`+_3ptu-X&3{o8mQ5$)BuZ#=udtt)M!S1n|F(rCxpR1;{^Q(B>weMp%pIEHxm>`e^B zwuGXvniUeuHG^B)U>rY$?ro+UP0e0$-M4$7j5)eebeDdJ(!LzdJH_KGW&sE9>vQ27 zy86VJq(Bq`8sJ;^IdY=t86%yjz%?fTE?ZY=Wwg6rG12#3M-y-^GWvo<`wpndh21YI zyLuJx_z}|4g7uM`OCK4c(S;7s{I#TkdrgGFT2cV3(Ww0)gV3reW&SEV_`v&?3e{AL9v&K2q!|~?)kkox{Xw|zKnu9sx@>>; zDdEiZXZQv9I}&a{_4tYO&jkqlkN@|U>;;o z`>Q(wMNY4_qFt`8X_BZ_9*NqXIf5)c{~1RO%3yJXM>bV9Iv7(A+nO2pU^~tP+;Xr) zn?DXM9|vfy#}YIR$#_pTS$?3qd6(n0C(BA?q=7P%2y_NC_bcU4fyTGRYGuD&=l^6E z&%6#m1R6mik4*xSQ>;VLNdS(;zs*#*rpw(;hmG^1THB=qe0Y9Y#maZ~)OE@fpo~Qb z<&<^Sde}bHE`B>KQvG5~3KjI6G!wA8xKzv!ExRw(ZM0X4PE5d>YR&GOyK0G=4hs(O zgoRw;K|v)Kzvt3MJ>a%V`$fEY!_eSrttha&eT?j{v>~823C=~eq)su;YkiH^n#^)c zA!xy{{a9MO4Q|k1nq}|qmcc|hcHP)H`-1Ff`J?#Wl@?y#3EX_Ctv(A_;m(R!wk-+j z_lzaU!EIe=q?PMPWh~p8y${|s2P=9^d-_AlkKZyhR@VvVdC4M8fB#8!z-;q6DB`7= zWoEBoIDBeB-cX-%K~GjKPAX4!DXvA zkz4f6q%S{^#(u=N^Xg|F_Y(6bSP!~)rrI}0N@@e{5nZkpkBxLOab!d)S7Sw=V1NI< z+xiOMzI}H8^#h%>C06VX-X)$QNgiir2Td55(nJ+Dk;ac9zXtmAi?S6n$?G3l-s8g* zN*_!rHZs5a7G>&a}vAszQ1IU~&&HWQ)E%@gBv3;Ts{xb~aH%JkP0)~+dg zpfkSQmQ1zCR*icvW8z*RtNAfOeLGWZwVdSPp4{VJpGEq6|LEU*H?ZHSQEs;_VTld6 zhp)vpTb6_HlfVVO23#l!fbsv!*=IAN=hzii}Ma8t^$mt_VQ@xzU3TT1Y0NJJR4 zPPz`7++$Jy+MWsl(SbxX0D>fTA6aTV)1O%4%d3%9QEubd&J{fO0X53FycSfk1+zEc z5A|lPu{N~q+&Y-rZopW&Q5+gBOCO5SB(ApRJ(`n`bY&azxFou1^~N9u%y+<`;!hSB zs|{6z@!83B_Q@)33>YMCLZe>=NX)Uf5iYtt;<2(vhuA_QefYzCRG4IR&U`~?Fj}$X zdt52;hg62*eJZhPd?whuI^V_MyAQ*RFUIip`GNaDqF~S!pj9f70jLjfG?hqL9%^es zEPa7#`V0*OQyWCWDg;?-nEIp*=?OBS>zrc9G1(mD<`CInn(?1K{%0pozXBu5U&g25^<_h;H8z>)@d4q49 z47$kSB)^_O->3#E(S#YfiH)q>%@21wmnlEr?$4nD!dr%GON<2*v~yn3szd;k2uZjn zV&al`ie_YJD9-U?`blF{YT8-S{&WfJ&1fhzep9=1by6X*Xi(94&H$3B5TeMVkxink z5h31M?4|?OiT*LP#iA~#uXv6C)<`c{|2tqAn8#65JA((0Nzq*@WmAw`KtF2ckP*T6Kc#C<~{ zj1>E+M1?fbTRX22k1wYn$&zM<8LkcPkR5e>c7eDnjXJ__#-Ptg66#6yN40o~#24>Q zp&mn2N-$TP8#Rcf`S>8wV(&HstK}vw2>)W{SU$1ToYZ3!9smxDzyox0R&IF#jeSRHx~DE+OSPHXz$yM|pPO~vP`XG#*RR5%7s}Dmhnv2~i_@(3 zJNs8q%d^F`+X<(^F|RWtqvI(Y$HdFopL#dzV{=94TIntsB}DNS%O8>`MM1B4|6t?4 z2mk>|M)~^a{eQvHtdvHy1`7TUPJn|d3@?cP^vrSxfJ|TNG2(bYrA#7p;7k=aH1HIc z5*CoS@Re3cL2f05)c4)FEaFd?2<;Cg*yyEVj&#BX2)g`xo0nM$5#_`hq0R%>lle)L zDVGyhg>?#j`Wcd`?Sli?@YS{B3m*lKO*FUe$1X-mX=RbTO`prpk4r%3=l#oA`?LU( zVQVTm6D@wl&yz4_T~^Z&Qg#AwOD^=;T|@wM;267JMKG2PZ)17{=6Q|PX{RO<&i!e< zFc^kpAAT)d62<6Op(C(cXI7ngE6codL@w~i1=X!2zSImiJ0G16#W)A-KjE%Wx?pB# zMqp7lnRfHdR-z~cT2T+;o()Mi@*pr1(=IGtg(8MBs2=WwYZ!=f)7|ZBfoabkiY|aH zuKI|9m)H9iA$dmr%|Jcxa<^9z)Qe6ObAsYa1OMAI0A_jm$?_T;I9M1O_~y zaDAp8nqPQ*_3Q}1WG5=rM)EY-*wEunl?o(Sjd}a>aC30U6ce?qRu2}ls}B&nYw9Rx zsEEP9qVV43yJC8)vxRJOX}91W8lC`Cr&d>uuee}(>-^^*h^GMvr6A;T`G>*W z9uNmodsF5H;oZ!B#iM_7RZ|%`@!o|EI6XLR?;P$t6mm6i;Ty6ugez0ap;o}n&t}{? zErYeGY~hyyu;AuInU8R$9`H)7Mk(AA*>?^ z3nDn5Gc!s1NJepZqGxZo-9IB!Kvt5kB(ZN>E@@I4R#dL+4UKZufs1QW8`i)}^*Co< zkQics4heDfW=x50%%=#aSabt6VpQaIY&_qSL8D%Hq~HDQRnvV0I2G6}5WotTYf~F2eu|ZbEQ#EIT>M2xtUO>NPE+}zVbP(=i_|CgK-KVY$eXKo6G2%?Naw^ zo+XqGiMdkf&OIx&@}4#bBEj%;w`54~G>*-LBcn{rKt@tfZ`!ob|q*8s(L6#~~rEx@i^!p}zN9 zmgRG=kUgNb;cAdhFqEhG8w%%3ZhP%FrAQuZygjQ(ZK?0XzsVT+1nZP zIgpT(Iwg7*wBm`LTB?m~T?K9~)CrT*16*C*oYtMqVWrQWVpp|5K*~8i?{T3d#;1+V zUZr;^Yo5^F7n)N_bW>^^-ue&g`V-TM>=8Q=N@|)YL>hASApH$sriw0=rP`7dRIxhS zdUg+%vWgZGyx0Og%`{IrjqR7ZU$W}^Sx0#Wk6M<2<;+^JkC~^hZZ2Q2m&KDZ|Ktt) z=uKfkooSaBNY1YTLOG(UWJ$`=O4UkB4?En?hLz~Sn~A~5i9wmx(kU5)#3Un5>&cGy zWwvBi8=G}a+k-g(rRL5Y%deL|rm=LTG0r9em{L5`rZZ)UcUWn6kLEoXn`m-RsR1tX zRK^&S$;0#1>HHJ5-=-6|(0qu;_)rk852nUkSHdDvq)f$oXWM>oX7?ZJazz}ja}l}} zR)*`Q($ku$^}0K4MKx7CNjPnbYu$`f7c#D3MTG6VAYxbm7`N?PA2G~M%bXm9IQCdV3dOHfM8PSoh% zDZ2LiDxR81vVwi8AO8HjWEwpXn_jozx#s5>LKAmLo{P;ZMcL?W9!E+gr!wHmzn5?= z|ISZ?sn~K2XlvL*x+XpNv<>OcxgZv=o?x6kw z3>U4A8a2IXF>TKLsx47ECThm2R0#!iyGwyX9TsBJ*?xu zciLsR&Ip+||56b890KS*VSSAM)j`zRXE1iyva{;!_bnt$yf3W1tZyJ#w;wRI>wlA` zqfm-8J;jY}3)O3^%wGz{VOdDx7*JJa<&b_~UUmz6El6iZKU~{DKAkh0O@{GL#6*&9 z_ZW=>oDoxighM6b?e{vWIuNYP%8qCJQdaAJLakpMOjavTZU2)6#b20ED+1eUYY^L< z@20|qLLqIyhRa%5t|Z`59{WdLM^z4n$3*lRnFvXl&Vv|tVV4GY3iB#0H#{g;ImaYv zZZUN*@}L(}DQ0aNsTrn)jY}KbpW@B$#sIaC`rq-v@cFgN|&3L)2^Zl*0aKF$7^*-G(K!z>Ksh&SP$@s z?uKHn$Vl80Xz)IaD;VEsM#?zYh)dI7705hnyzIy1UVXtn@IIs&>2SKEIg)Enw@c1n zNiL<8@1s`w_SK3@tugCWVM}G3EFu6zBgg<8vcU{$TX>9?Pz?aB-X#amlH@U8^Bl^B z5Wz(u#^ui7Ch1sGKW67}j4@P9AJG<9(9_Ybvh@ecvdmCLhmX@nXQ~Y6@n+n)qRVm{ zQJttGB=Af!zvzYBlWTcd`Mm1jgNtEZmp_&^N8#9Ntb6Q_3gqDHHRy8*O%s4gV>fs) zxv);Xq_$edNKcfm>`-K6t2M{|5oGL|_?nPph_dXkfBsgbqeU{=-X1-q z*W!C>TU2P06Crg9v&M#WmMuW0gze)5{o|bajaT{UFS3DkbAUIf;>|0%iBU*( z;2)Gnst~qkV}0*;aN{Vir})h9W_0(b+01ZG4ELwHuV0w%PjgSvZl2jevzHQAEzx97 z;0M|B^rnUat&~s)DxN%eE@W7fv$|}L`}wVcSO=b%6ejmV^=lCtIB+46$JzN#zW(Tp z_ycDb%)UnG?@-sq&45E$lcyUh394294!%gIX$^jZ9Q8n_j-k(cmy6)wiWlyJQrZXK zX3an!$-2dZ=MOG_UA|x7Y~KFbO1+_PcPyfDJJQqBw3*f!aEnn`h+EA;apzfdK99yY zpxK*5Jv!pQehU_v1EeQEip}GC8L>XM%ADRVK{o2jCLcBTSpbW3pe*)U9 zf&%%>&|eXaB_2~9qj#=@~pT^f1o2n90?c~kfj(Y|;a=L|Yp z_h_@B{F!LO*lNb9f;RrPQVTY2@O<|hsOjGOL5>eJHW!teftjyZ|(@idI5+2hlL*;EdzloSy<#i z$7F7BV2>08K9DFI3%S{kkVD-bv(?30s&zMZqrgS0s|1?;J%7dpeZ5qH#e*|p=%>Y8AKQ-H%ZA>onQ+wJEkfjbJqR$&Y;ON&yWwvp`S&eg|=w5zhYW)5rNH^+mM+*QG zZB*Jo&AA^ zjBE-`g~%pK7ta@LA`tB!f6=1!(-j?HIL8KNIHl?Vc_uLRHwJN^T}h@&uFL{rt50KAUK8Px(0iz#cDB}MyBy-S4hyRzb63v;W<0q zSkV1kX^b4>sx?Y{sM3V2PwDk5TSn0)l>}co(73*`4$iobyOlMK-UaHUa$R+*XlP>6 zu&ZYJXptdh$i?EHLE|#wiIZW%vl}>m!D1fLjN>Qewp2H5V;Tb@&EwL$3g=hE%F1kA zOH0~^VJ6q)32eRad1jqPNzZ^x<5rA@`trXRRrNiSfZrKqh0eegpkUwEg3vQru1KRR`tl11uibX zzX#&4Fp1hsxJy8i1{A^kcwv)9O5_uuok6Sb=q8OIa+N@EPnb$T?*awLBz{DMU&db9 zMF#{kuNtG2`j6ivE-FPUK?Z8NBJJzs!jM z0+^eCj1DB;LdFA5vGUV4AL?^_iRk>ZTr+PS7Sg}nWoP6v(EnI9Ws%$nFyjX$BS1$H zO&ERnb8fZCj22>_BEOWXb5PBTsW>D=H5#E}DCyVHs%u(<=;CqXOkZ~Hv#_>#rj0Z>QT%&Y4oh_0ySRFWQ4^ zk0|X(Sxm!+7@1k0V#30=+LaY<7)fcAhjW-4V#SO9{9f?NlAnKpP@q!rEYLU$8w*Zf z#$s1&y*h+{5Jk1a@eK|;r6g2Y($qOe#zB5xzUnsAXcR69Q|59o$z2g54`6vMwWQfw zf4s%uCgfwJ7dAN-V$*)WJU=*!LC;rPXWBx|XR4XnQ_(c3h|9~m_NN2Sb4_qBZ%@wv zKQ-8BF z;wrP;tk-aMbuwWl=FeZWdI0{P2~QhAr&A~#YQCa5LLTNQmSg#Z%xhD@S!rMjSvN%% zi_^W!8h59hK0Z&PsR{kBxEMpkqe2AoR8I69j?a@Amr8x7#qbST1;j_=*t*3X8jXSJ zeLa#5#fTcJxxEzYdqGWyUzyvp$4jn7r*kN#h;e8mnBp12u*`e#cs#oSeIfUzKYkYc zDIMa8!##%QXa35lms5w_`}~h$@;^wy{lb{sm(0`uxwOfagA0ZH56GHSC=LEar@9TL zJ=A}ZH{wi*|1a{=eK|<~9q<|e@$&7z@NFB#hWc7I6$WT%L&*sw=H9#A#!yK(C=v0&$iLR@Y_6?s(NTnr8}7lgn8`!o~O+ zv3FI-3TIlyaEnd!%}KjHklN74q^0P3d5iCcF3R@O4)Wf*DztgxC0Yr>Bf~I&Yu8Oz z*z+T%*|6P^c_Ynll+XvD{Zzi>p%ycC`EZmllVgmqZs9Xo7ZofIPiH(ila{YiuqBS# z!vf=zm5+z`m->0?1Ghd4=RWMdmRs=Kdts{Lg04-CWmLiPl=JhVeG9InBRZ`GjuO`B z1q_q2gJ=2!vJ_K0W_?2M)Oo6atxD&~G{_}L{qGH!DYuJjcBjScbuES{Va|&2fq4d^ zFWCCxL>6rV8k%$$W->;(Cpcsc7?}2&F1JwCE@~QK7|uT;_meqV@xRre8GrEfma|hN z{Cy;cc@bX$!!DmLsw`+KOHJ_o8@3!7eaxv8#+t_Kb{071eFnSgZU??P zEwUtzcT75^3Jn+DCSEqvvr|~A2eKIzETvl7hsm{`=<_}1Ct31EOaCOHP%%`rK`(*V75H4~|4+4haFzycA zXS<{F^h{}6n0T3I z3@-^5yWa|rX#v@22D@N>!A38bz++y`J7ljHLZYi@^?}JfgClmP2|-`%+5IkN8Y0|vQepI8&`0~wmCO6Asj>6o4iwyvVWaldkjcoH<%Y?xSHVQ4fryhwT{OUZeJ ze>eSTjU7Ix>VhE>2dN~(mfE=!{DO6j+jX!4EmlOx!g_|=!cDX1%x=gh`cD@e_ zcE2l#l^hl47oHdC(n52n?Dx1UH{oe^+3-*g)1uop|4R3`5&ew=c&mG(P>xjioW9*% zR^G~JB_@FQ@M+Xn^1-mFwcKE{z8NScuICSj1mX)q*DuLL9(lio2W`EQptdyx=YUr6+aM&BiXrIfxeo_ zx0^54Eut?xKIkzYS$T)ada5gZb0{(Uf#r53! z;i%WwIMgR}4Qok1CsvYeH{`3gC%ND1hjOE_OdI*wR^qYYk!+fikK9&*dj%#FGuET- zbr1vPH>-#NSfXSX!aJCY2--05KrI;O+7r=H_0GNlP6bfIP?{vi|1e{)!sy- zS}=#I9-ZUD9xVis1&tyvh_dh_w660uO#SM{P?cP@<|`g;>=eu}*E0>GNdbMMZb)tR zSNJ892!zu$CgmtR3Z_`=b*3gx&Iv;SN}(zM-bUnU82n8ZR?`Lo+N(#ucwrYfXY|QB z6UJWzSxX}=?U}Z1TV2k|AU}*coGiRQjV;@_Lt0gMEIuy5q1=o@Rd@$GZFv#Ce?zN0 z`gvbHg2U%Ix5BhK@fZf(L<2C#{sb)|z&Wkg!aRgW(4VZ;>PF!S^tI%>^(S|>ZmUcH za$tU(hE+JNUvb~~1)6Bj63zKI#U0buI>@ejm*}l_&s`AT>q!^5iE>oqbxFW&5~s|y z_LdJKJbr1D*=ufQ;B^hYWq~x>Qw_QGKaM;G*WObeoP7@Xi=DhO>FGQ0a8@7ti@+fj zLR8_|54iP<)&hhp=^QxU;Qn!8$WGYT(!O7eCN=fVuBuqXdZwt&5hl$ZY@P>tq4 zB2&L;(H$7j=^PFn=+D51+PwFdvi`rJv}TKQIOKo3mxPSS|GGWFT}o`w|IDw?X|H(x znhya5tZV(6Z&=BT*lfVYNcG?21$UM$bm4ap5K}dv&&0-@%)q?TrUp4x%*BqD2wX_Mc@4XM#!L1%s{D-`oAMf2R;fDfVd(0&1 z)^b0)jSC@9HH*^q&Hdk~taf^}P4)Is z9vL1IJ$P$uCi_G5T`5;@p>CpoG&|~O*N*LA7|#(sJYF!_s(Z(1%1v! z$7+JAc)DJBg{l~+LmfPd3T?MS7MhA;K?q7V4tJ@lLT^K*F~c(FNc$GfYhb6zbl3>!qIqe3Z`d@{x^NBKRC8+a7=n(uKIM8D zkpJP3S8{~7_p-9G)>cwcQBogIP3i!uJf0Eqy--!zU0fnl7Z#ZgC3@HwYiw42HM`3h zPllM%o>vsVA4er6Ye)_fq{!=I60od#kAo4P^tfASv9I4~7ETX3di8Ci{{#2Q){O(2 zbuJUCxzUlsg!F&55Ttc8@B}X#kAQ#X33kPevF`Z*mRQKqQ>$E*pgr(RN|bQrOfr~) zIxS&z0-0$i4IEo+{x#d!LIvY?`~)d+ibQq6XPv{k0-L^&obk-pqMnv;9;4n zc8*D;m?Ptb&X_QPvHvFuYf9RzN|p&sj#`;c!_)2gZR{o!Sqsan3P*CzG+3aLP^eJ3SzJ(yIXxz5y_m?9+uU13BMRZ|TQ=AGH)Kg?86Y9<+3D;m<3ytBS~ zi=i=hx$Jmoj@J|g*!@MH|1ZXm@q+dag2!iTDQBDnmp)X`gO|tqk6Ov;&_E{8;fJn9 zW6T-(X-HdgNJfZuJD{fPhaxJ1C1Uz81|`Fv4GF6Rr{zIQ=KkEfQx^m+0A~M)p}hvv z#;UQ#uMDU1WkR89k%Y+NnWlEurp->~9o%+t7m+%PDWoKW1sRJ#wvA%hD`6}G@AL$t z!lZ)Wu?-SdxC4CW8j4(EC7czY7OKrHNh1WPV?cy+UA&A3i&YDkB_H}IZ_w37BgupCv0TkB1jOV|^d%11Nr)yt=ej$!U5ax z7F8>-M&&3<_$Z?mJGy~Fyi-(|@alE~`NL$CPTXbS96TdarOW$w)W(7RlRsxVdOA!< zwox&vRKo7BKu)ZHOWC0bnH1hrXwNy}*m>X>4+FS(1rsKtK4ap={6o~jpGHS?i=Yle zONl?6E|A5uu+x3O_!jUL$ED?^{KT?yt<_(fCq;zwu&x)+s3*TDwXYGu$QUMK=6}|i z$eEXlIS9JVFp@#ke>Z30imdN1+Wr{yuEu$XVQo&JNdMK3rWEgeS24)Ug9rZYvis3t zcCBA%)#Bv^N4wBceT4vl-X#@7O`f;5h-%|7x9J?K^A8sth-a97XT5G>nXLC3BXQ?O z1?%XT1<75muKng!743(GzsnSj3pOFPJN@a$!1?M&-P>m69``TadzjZLuo&}Mxc zM4-hp{*W!s35eaH$&0Q&X>VztTx2|zb66yLxq+*(8*WDrv1t;NZQ3AbU8o3Qe>xLx ztm>&Oe-&&nPga)rr%A=m#V!JC3Rv7?-QzfdeyKVIFxX=0KnL=842{do0I|>cFbubg zH95+V0uG!2X{%?2--eGsakZpmwGB#cQOI@qWgv4lwaG@eeR8>-4QVZ=133g!M2u45 z0JOYAnu^`L%Fa%6g(6Fl#KNm7SyeGdKrBd{B_-umF{sPm(>^pga#u2wY>QsuCYiM= zIJ9#WxW9uUaWr9x+ZQ>f!p>*Vm^gtXm&P_Ay_ytAHgYIM)Ub}_IP@*fUX^B?g@VP7 zKejlAPUFOkb2MkjM^$sL>JfI#O_fM%+PZIhZ&@O_kSpjY88w{ow=4cC1oOrtp*R?7 z{?m(6Ye7^GPEOv&}Nv1gh;?y(`R+i2Xa8c=p^IEVuk^SMcl$?<#52?C*p^@c0 zx_m}e1n#ov2`jcK(NA49J!lhjbE{FFbF{& z5+A8=S>E{%JE|LS-n)`{Q5n=#_aYgJqNeCGr}aT?7uvgFmlN);-c2e zA{GurI|te-`+X;tto`}F6gxRrl#2-a07AFy9t<}g?dR39L!!@v6@Hu>7-u$;EWkT( zZQ`nAdr^FK4i1>E#IK(=x0hXBLT(EyKjoUVUl6#2k>d3B?1F1kcwgjuCjM+6bWW0e z6)|WQ33UfVKt8W0TwafgRpi@u6G}1fl)G*jN@iTZS*zG8F6F~8mHgsHtn2x^>NJ-vS92n4Ta1;hHnfEFV8`a4uTST6Y4paRDTW`w!W z{*6=gOki5zVtl)&z!P0q{0*hSPVB%z=kn0|N%<8G2o1RW}`Y%SdcsTLC^;msnk{6j9;PoSkj0Tp?n5M3Nk@iqnj}XfL>m8`>`~ zHIOWFpn=Z_VY=8}Az(IxqG#LhbMCSVc$gg=w6Z7N;l8XP%KS=(*q6+yhMyle$8nnT zI%^8qcA8sX+mv3Vd8bu~jIO>aupfp099?*>^;R^P*%sALdic61(grqXf1VYcH`VtR z^50d6#*eX@7X*aivlfybu1R?~o1}dA_WS;gL6}Y%XaT1O`4GR)QWV}R-okJW;8!D_ z_Wz+^9VkTdo3n&`2>OTHXo_c8E+3b80uy%-lG#p6Ly*4UN>FmS)yz{`;r0l_lS5fd zo-94RHU2YfitBaE;1zI#0tKj#!auB>hj@-$yG!7bMLBFSQef~So)7B0nY>v(o?;H4 zMAhVnt_0{hwxuXpm2nYOY%enn+nn*YVmYu;4?@r^h zWtegl>I(u)IH-q?#@SBlA@T19*CurxH8nLO)E$RAq&1el?h;^`^vPpKpmyLW8emf{ zTklkr7V;WWf)txV?Hr9%)g*>=6{oiEDUX2(e* zNqCB zeluF&UEP-pMaM4J2CD>}qr7N?>98Ik#w=`%opr`RLIKZsq9Ydh+@acqXQS;?l~8gF zk~?EdI{C3pKiH1BRQY6&2?i|^ZgyB@GKKFgdaO>G@imh6wxF%n8UZ%U_)o?{@fPIi z7U;$E;B6^!lH1iFNeAAcYtfJw1&WRqor5#3WHWlO8JG##8Modb*^(l&TfvHB=Hct} zl96tS_&_g}PDH%o;z>zMO(()dv#OKgXF7%fDNhuBL%K5s2Qgpe=SuaD01xV1-Y5uNaTrk_J|!);=!De2iv8>8sP{a;}1X! zl-3f0))Km^$v;`nz?4;E6=ix{w(d^zcV)d}Hn4`F)nv$v!o^VDx~pdcBGK*zHwgq` zH-l4}G@m9y#`$%EGecVP2nhdd0MS`JAn(TYRaDyP9#bOgM&p!j}Yu+g4XU^sM7;=y7FEiO+**GRj;eLCAE!&O) zV#AcA8JNe$3&DprcR1E!hY}B#8c7D7W4jul&g-sRvBCwZw&*G9d@M0VPe*2zR-b_< z%f+dZ+O*=b21F^g08ey}m_8xW*nE^!%5+bOG5g@K0=8d>RY z(%P!&nhf~7?vZA4iq6Em$*N@el@6@fmgSL8>2^{8FHLo^!hsefvQG_4egZ~CC50!l zh&5=IHjGXD8xFuiNU=j#UBr2585pIRJY2Qe!82Y5Is5Tz#CL<=4;`@}%ZG5+InHe7 zjoUd5V}771%CkHshUzg(G-WbH3mZ`K&(9sKP*syER#5|7|8WW#8c&MaU=_yx>+$Kkmz5eR6*LkV0L z7=CcWE>T#P(`Vn-+N>G-^Gr-|J=m2YT^g zyz=I;@~^b+X*!waD9+=-HHP_q(;2ZZlkY=Nf|yX>Bc^| zWBBNAGBmf7sqCGWRQdBey2>3&(*3uh5(M5e#P4nGyjPBhOZ&PB-j!qBioB<~JKZV| zL@R#j$A=yzRgTq*)6!8Gxhj7Gx&ke?4DJMUn|Af(3GTM*U_G{{a-jcYarF|mXS|k_ zR$jl*X})_NK?9B~Y!k0}vGmqKMqZ-6Rg6gpX?kq5sQ7iMr01#P$w+`VC-U%SWlf`M zHl6D7DpPZpS^vgjo0_Qg;!m9^_#=WsV;Iz$d0htd#w+ltXnL@7Bj8x`dhltrZEE1rn`0gZY!lEjizBc0 zI!W32VcgJIvfbT0QzN%rKSWTt)`>i;W!4U5mCBHMk)yXFqmcf3!mSBNguA^N4J$Me z8@LCgnG6oVrlF%Va`$vOYAsY)O%R@Ha~R3a$vF@fI*k08=#} z+K(bU)p6Tl%;^I<8P3v7&@PHz1if2!k@c*_)xtie!pS%UZRQX z+;j3%_&iw|KJSdMl#-*T0|6LH5V;DPU15&y(qOU2K!ZQp5L;DCwT?)#5^p8ET*bX-wr0)Z@eR%y*W5-C2g`$fDEI<{B>LI$$vs{pPuUD#;2f z(-g`KN_i?r6~R;C4$>d&sBtwSPHjktccquq0SU#8VJ8o(o8o?|Ok6fa1Uf^9mId=# z!0Jl~UlrMIh^zljt2wlF{`pi!Bt?Ed!nO>UK*~B5UsWq}ATd7+(X7H9hGWmhwMTaf zQ2h(77O(sDG3@Yn@pV^$7;Yy)&NSv2>X=D-XTSsJX4hc^VR*X$xeccnr_q&}Bsai;KR_WBa5qz0X$Zb2}ww#-vRZ zQyEh0&RMjom|IlYnw~$G@$xw84-n50TMfYBJiAW z1JWnmIeK%a1sLcM4$OTWBb`jK^PqxNW# z4+pmc_MS*)eR0RX5P~`}t=bS5J`(}YzBK#Sirh8Fdpx;M^Pmq)6K(3)R0=Jrz(W(I zo%a55@+wcbw@~+^IKhy`SwjJ?_>K()p_()Q&l!nbkEPj4JKc9j?cVqj3Y7~9?W2vzG z$z6at`dXK!oR&DS;>-mXSnx&m3_@7{?{ljyQ!m@ps6Mq%SFf&qSq` zOut%Izx7&_=$RYK@Lx$lU_qlyt5v8OJkD0gFryIjqN5I>dn!)u$?2iX)6Wpg=F&&1v`dCzhch2|anCH926)+N z=wmO)Uk`ocPm1Zl_lkFJ-K{@`f3A*axI1F+D$Ll$I{AFO;mt{&L2k z9*txbQB4KIXRTAKa#5+1&9lXeB$KEL2Fd~^r)G>Fb&xs+cD`tNSiYSM2Y8Y1aTr#wLZ-teZbHie+`ZtQ!eYS0sYN6C5Y1S> zXIlly`WFwq_BK}vEY3p;9GyqR1LvW3&|=Ax%=Ts1V}N=Q40Qp1RM~m1B;id`9V}Na z19;z_C9<1WVpv;>BLqjLBSM*s`1;>?!U%1TRYuF@hXMt_FB7{U4IavB#C!PNjpJg- zDau|-+Q>M7%ss})sFqb}Vje~;(U{5x8J5V62kFdjEjo8rUP)0ULnJ2-O5HhvA1Xv-5nT^@b)e86^q7Qeqj~&z^de0x)4YSP` z>jTr=hFDo?{nIsXk>ryGD9vY8nkWwUvY1zZe?~!{Z0M4Or1bN^ndsKPIAFc^wQ`YZcfD? z4z@19mh%31Yhb@AE5aY0n2$3k`U#el-o+XL#?GG9>qYrl7+U<5{)BTbGf?~w1A0yT zDssY~#6tc6%9?ih4;$aA@_quD&7NMQER%Hap7s5Oe9%c1+0R)AK7Ikxe*ThvTDhT&YQGqPm~I$<1*#BUZJ1pJu#Jx) zcZJBTkR)UbkG06Y4xx5EgY18{4}q&t;8?a;6u{Jn9EHPL=3A8D)aR&$F=&zZH(@*h zC-)ki>)V0?3Zi+2D9$i&EjrthYLN6^vn=whnSAkwYe!r#YxL1GPwjR~-v%*W8BX}1 zco27c6u?e`EK1GA;`I8SrC6RK*Qk!X6}KOudP5P%S&y+39*aI} zbLHoHJt-z~%rW+U8W1j?pi+vLsIECyOHwzlMtijML;HBc?EneU%f#0Pl1>JIBdX>| z4%v%P3z6hX$AN8~cy8)HswZ0UI^ihSh7``_n_y2qJhciatBnA!o3(tAAKZzK&%siV|0#}S#oy6j8BjW9PZ zFtmtzAW}P`afU8`7jG^QUzIQb9T0UH{>@G9`I`Q#HxUq*U%5BVFg*NeK{+ftA+EZn zz-Wbm+`gl)B{uMH)c;QQmvF+msSJ_!3?>GKw}P}Di%zCi ze2TTJrkfV`GDIbNMKj}Z2D!Dn#Xr0-5B+tBQZ1P=7!@r;nADEOS%f>PEZcnAncp8! zZfXx*tv9j$jo^p#@b6t`zM^cH)vK3GikvtNxqS~Pi-c?qpPy;+Vta1Qm}`7Kj2o+L zIDPxbI0`syB$~#B?_<$FfVK`$n)}3Q&4?8IBCC7ID>$N1@Jiz-NjkDksZ!$(|{% zs{V+z{f71`FH4b{A=6A%%jxFIRB!An9WTX{Z&}27h|F$|y2r6%XLQB6{qb4E_g^nc z+**XyKq~cm@`dau`ovpL1??AQ3i_SX(E7E zu!#7*@0>#Nn?m6~V1|jx4RPYwNyV}773q5z0&+}XnYX3hzJq85>AMUpPtiC)rb#?Q zKqvBnwpx6=dGJ9Bp~*O&K>CafJ-Xr|{=h*pIzJJom{QFRIeX!RCmisGAT%ruK<*Vq z_Qb@xhktSR&w3EkzA z2-b~i4ifcD67{Hjfva`AQRD2Z2@+#xnR$Xv_DWck05a}+2hZ4QXFBKcT92W689L|L zoJ;LZ?zB&MfV|Gjr)(Wlc>7?Kq1(ILLmXm6HgW(GNfc$rguDwfWt)*xS#3D68@v8n z3&w1|$bCg%#t%ogz$_`Er}ah>O!_*a_bmhOyX)zc=~l@7#O<#K@2#QNa(?jZNaISoM^!v51wx95W`oEk2%Fb zoDe~k0|Z< zuYrK8$tQZUs=d;5vB{W#c#mt@A{(Dzc8;0;LGr{jJ&uH}ZW__$%lye?%MVaze+us( zct`*`EHg|~#@tOjt0fV$3=R4!Zj5jeP^h991O3^Mn_W>F_DnJW$`(8V=6hQH2nT#L z`7}`4h$YL4)ZP+r<#q{WD(abOwMm9*^mZ%_E-;orrAOAUIwrvCm9|KtC% zQSRFhA$z{QnIoBCsUU2y7^!|8d<3n^ij3I*k7MG(561!j-z?r*KO%1=+QDtZydqx_EL0}brlZBf>&CvNKA;*HJ z?j*xurv4UyKu^UJp+ZP~21Dit@69~PWp3pGMHjat z6*?1CDpC~|8yEuW2YRbi6&4>T>Hkdszkv$cDF=JLMO$)HZ~Yi~f%>@1$X^W00UGN< za4XC)hk;s9>jmv=IC~m+%mz#v2_WmXVj!rg9o_szSKb*~)r9?Vel2o)=kbQg zttnidu6w47^^qSO0J^w?sDY}w_l`qahPHoo=2_D+gPI?UeYF;VI&w!akge^N&WMAYX*)h{kkKv^NQ+YgX1Zy%CTTDVNAB-{* zJ4ok>W{W-nDsl@D42YSO{Og(XRuOhId@qdzTVn?=B4S^c79w(Kp`UDv^`hstq zp=7%-ZQ{!#Mby}>dvUUq+UgKC+(%j)T5nr`$A=98-IQ073SpZ})~_rS4(cYxIkco9 zY68iEiPB;J*y4utomK_P8$2f9DQHUelM5xMWbSkt0s=NKC@S(#l$Z#X3*$*J3ntu3 z3jYqY^0qMB6vsk0p@6W=hk1(0QF)HzsqQ;0I`XR2tXC_llNm}0vA$p@t4oYE<27En zNX452se7PPg%x@mET|M$RZdL3sEX6Uxbi7fd5DGO=H*s<)agBLr&o~@B^=EUBX0^S z9#4aa4DKEO_=y43!c3Go!Lt6Yx2G>`WEp{RK|YooH2^jbC5q&ycCS~l1R4p6$vafI z$PM}r4caYJCLMs`ZJ=k|2Gq*a7fRJxJ{$Ewj4M5B!$2z=-vrlwtmJ5LkoA;E`OUVi z-(vbxnKrr%JdIm)_QUi5y9`rF%kzdx!PK4hj)~tb3Z5F>>ey0q`3yacb{U7c0_4Yo z!DW?*6_M6&YnbXLfwe@%Ff^IA>9jgk4C*f-^wlda%}^0%Pcxkw4!b0qb%4!Vk*aQB z4fk~UVM?6LuIiZ&{oonUynJjnG3YS~S@lY(ea6m-n>Cax8^9ICDc(iCOZn^<^wijf z-YN}Mpx!69Tx&<-OS5Z^WpbG-!xan--P^}V4!6Sq-P>PF&3WwTT3a)ZJx}%?^4K3H zg*5~X-Ma6CLUHRC)JS(_#vG^+=B^Fwx4(rvvjM!xNGsKml+ML*+ep#F+0 zp)<#O;pqxjnS1!Q+wjUzU+U^uq<4Shu9fDWH>?d3+*6?Yq}yeDrZxVM>&F8MFbEHg zkkS2yLZNfsU?dKI6MNU~7px=M%SHJN2Hxk5%|}*o4N)lD;2EZ2jWCYKXYHlQJ2;nu zIj6)clodN!JkdHi(%4~~T+(fUYJRw78gmqv(>_4Q|C%J@9Nx`m{W)LwCjq*K-OEeF z2X+2We0+V+sT7qv37-H)glsG@beJOK6|#SRL>@k9V4>v&M(ehboMX(RyVJrY)Yxgk z|2M;LJPW0Aa&#B9qpYVG&tDOVY>C2|qfG@8NjHI1#aRm}ufUV%Cq(u?A zaX7C7T3yZx>En7AEu$!0siGJ2IZ>hGC}=Xv8HYt~ z@w|d@Qm^eO*GJ?2kvym;^5ZAlSxvuFy!x-{JXMSnGcD|}c&V(GUto?=nT=7I4sd~B z0!c1^Ne3RBHzM-uhRm2mL{+eDAQa}%0$bmB5sQdT-Pv9cyMpC_mNxRn69+OkGaEEG z3@W;#ELQnY0Y9w5CMfo1BplXcB8|P`?^47j!a2U~S zcFViyt6B`f%7Yb!Zpa~w`!wb63s#bV~IR=QhdVv{z- z=Ba6iFRjN`XTw(K=$rSI#Pf;bBtf+c@!F-%f=^S~TO{3CmQM@D2B&i@>ZP!VR~eL6 z*GUM)#_bc&Eg?mvWWhGKd&LF)ty~Rl zk1y1dFLwkm-q4`sA!5}hc{6LnQ9f#xIG$QBklD5KmjYhcj^B2lfmewAbBfJ7_;4`O zTsN_~Jb(atpWJQ)t|e<8BZI&Mf{wz9Rt zf)E=2z!HKsO;sJ}ULd;KiY|e8_$-*nU^Ko8ZTQ488T@ixl6zCqm?mK5v_Og2Hg5E5y!;MiBqZqHm6 zs=@Mt9YnCLNf56e}GK^91#k(gM%C((kM5f zk;b(2TSL+jshzV3Gz#PbMhC|lL@QuD$2o+Qw}$_$G7NgX%?d`5@V7Bp!0I-pa%)~{ z#KN-6^3N{(4RVgB&hVHeyASQa$tuTb?#2$=mcu;4EUX`C$MrjU4eg^oUJ%1>HwJt| z^rw^SE$@F8$^Tvo|I=*qADAu75!4PY2lL|xa_ZIqmIM%118o`o6EZH?G|2*u&5CAL zOe4xV3K4DrCM+Hb1cW%F$V>wQOUKp06@yJKgQ9u3q}qqvXT@#JYAquLtFdT}tR%Kb zZu|P_?Zda1P=7id8)K!NbO3>n(EPgFX&U(HbjAI&m45u`b|wAe{C1nDWhYQrWmj|q z2Yx4BBn$YbFqkWH#GPx%h%T(%JxiPLNb|VOMZhoi<_2}cgT=4(`3Y3ew+aNr5K(o_E!7_lRqbOX5y~XG+4DOJltuJ> zx+If-Z4?bH;3fucR&3=vOIn%;EAthI@uKVQ_wc8gKUhOUYn$0s2@sqwzE+Tf*}y|D zn*oJ2Md}pTwL;+N2}nuqCulPuk_^eS*qSWbG_ebygD^k$s5aK3Uwpub`&cSlzj>Ey#g@FkoacNM0hAl;R*wpG3`m+qlGuY;3kn(iq zWvdBG$X23cRn`|wL~NtaLxM8ye9#3P)kyjy=>TAJD|5f83St;l@&ybf)+Bkuf=K9@ z$6W2WQE4bdnvvDk^Tpv+Q}fzVFAl;NeZwZk(Gt}?8l`OBSXNU~JMTw-G3>8Mh5)S# zXWffL)s4c5BSb3+2$wt;1|Ox_g@d(?+W4pl^xM{Q2sZKJAE`HkkAlJ~;459mw)F0F zI&}}x?H*9eUOG^ftdi=2s+uc2?R5xMjF6jE)lcgI^hOQMUG=l6vmWtWkEB^r_-jO( zaAN3zeTd3xF0}PI$kih!rTDR23qVDB$$cNTVy`Y$z`!HM59^UciAId*yMJ9wBzSsa zFq7l?W-x_wr4*yp9IB`JN_c2+5UrJ>XsCOl+2pokX~G8CS|Wv$MI0EBxDnmpsj@VS z(D(?{>F0CoqPB!{Mvo36M6r@A5N-04Vv12*rmNM7Rt4)i5x+ML1J%j-$AIemXf&uT zHECN(=qBSWyMHB7I=~NUGfpF~tztPWrgw!XqT8B1!2uoZ&feHn3tQ!nk8ZOtr;`!B zLj8Nl%aW;bsx}d^vYr||d#@%HYekd=Vc}-45i9-3`L@DD!6o#cuIx7)GjVx?Zu$ z&4S{8{S)UMW=6KtcF3%TJm%JK|3I@AhqIx(>|W{D%ULeXE`xp5)`89xH&>b~d92KV zm$yX(il^*hc6XH}+^5TT#hMcAzT}07x-2b)!u^MszybK3WftE+U66OR;p@7{W?#mA zwXR>ogg27d$Pc3{;S$)u{@hat0SDIF$dSSdl*C)ioQ-gXf|ElR`C;D;ibC{G(}mqy zkaxM^?c3NNpJP;C3BcdwJJo4=Ae(s7(}vq$lP6#C`FkKSpwRG76Em7vbRpjr^`UI{ z!l=$$2y99BEky#};XQ#ay*{&V$G`J{XpZK7SCAJGr`(Ku)PxmvvO zqCh{q3cNtHaT(&$bL6vQ1KKfer(gagK{udh$r|_d@Xda!6{7t4(p|T+tP<6d z0+0UW`7>CK$9(=>-Rdv5+mpsrH=VZvg6l-Ex|u zin1Y%7OX_&ju%|GFew-gdIll6nn=PCXO2Tf{I3rJtPXi-MfQV+u2@R;St;dTF&w7> zCbkVAaW~$~YSUux5Djy+Ny{RtMJaR17>zPUMYYUP4yHlePS|1+{I8s$9?Xz#Y2C)2 z#^1u!A|O+4NC2A;9IX6twtQoEmU}?kA!=be+NvqqL^9ec)9R*}p}`U@owntYDu=aQ zbAC1#d$CVt!;AJbk3|%Jdfw z-SmD_g03wE3Hw6DU@2#X^JWnzmhKX#t;O@=!g600Y>R;)j=g{-`@-2j^dg-b;jWPRo2LP; z-(57(cHkUq@DCHPDX8Nc9#_oFB1`jZ5(s+~5Un9)$;K@RO0=mGF3c0h+?rc5YBB9~ zvMY4(%xf@|Yn4!!ZLnH%5{10T4$gVW+`!o@$|gvi841}YQM2nh{L1xH^}Wc{uuF9L zchsUs49s_4n-=F(r#!`;TIYi9#F?$b09Q;sqk`GTUqLP7;jRfY^6drP3u3$^Qx~+^ z7NyXahL}yMtLZ)}T@32nvjpT#jSLn~9W@io%%4`2pWJgi*8yLS>fRkOkw)5AgTP)P zErrgV-UP?ALSasCWUXM-OKcIZCGbmjclsh3tC!+G658g4hoKgbZVR5PMcNWQ!J=yq zTf}oU4<&N74FAXtL^7xTvJ*5>N~XodM8wzRaHDi1|5FX8G^<7lmeN$&bKn7K+)j_#sqQ&@w*4NL-y^7& z2V+>Uk_Y4NMrY*uMs*}(rQ=u&rKV>@Yi_88{|fZmOpNvoz$eeye9;K}#3IOQ=D{E_ zbt&jsS10co`5w#a;bcTZE|g zK*0V3Ok1y{s5F26S3Vi)RIdUC{ChHVs-Zddf8~=QQ!hFA(1Fpa+D`KlXrBi%CcG^2 zll0`oihgL4;3^_A)Il;RcsL`2Ge2zSFyaoQNl9dCx&nfpKKydi8HK*+!(!L z=lvHe_^+~^>}ylW^r4e)(KPpV*s7VXcc#m17rY_n;Xs$B^c`A^Ke5U5>TX!a=N;`v zTnLSii}v0}tUg;iOQLG7*{?;%*yosGYoMHe7tbYF{fPQe_9Ut)ppwA+4@3w4LGxIy z{k=J2tM z(2Hk0=GDl}ZD!3}#d$61+ST_aNnVvs$3M?%g0@<+nG#fqRhfA|1x+PB&PjB9`ef&X zVrPLiEcwxi!JCuoo6}`$3roT%(?GoIFG{Y2iyelB9^>j$(@H~!gv17>PL-;^8??!^ z_se8xYfA#HHk*!I_jm(K_A3zjuHD0`6r+hEWEZP7jw``8M{ym8CDY>2Qpdqczf)QE zQf#yz#YL_i;Vu9q19QFRVTdOm*d@pG^54vW@(?p-;;bDny61m$_Gs6$s4iJ%+r+|@ zeIsiWMgD_`Sf0Tvd$fa?objWSoHML-ARa%<<%;yJfA*3z-d$pN>Wj=m=#jkfL8<25 zB6(Wi5w!^yIiv-^@3Sd#$&d}7d`n9};A}w#r00?Ef=xEYeUgNxMI|czc8BpCyv6_~ zp#3-TNi#S7a-kpps?Ck z3_=`}xOlK|2)Nua1>I(js0G&mSS$|nh~^OYqxlSJs*$=X5K-n3BY&0arswS3m|#kA zfFie3BA<7Ya0ptlc0{@3Dy)j=$tVNc-6%Y9Sh%Ugj5!$sIzkORlk}AQJ~kkrrL!*` zJq*Ecxn*M&o-5;=-!2!UB>?GyvUDWaJL)r2#hS|s>e7%+qV5i90F zF9Dcq#U_9Wr^dw^MRb)qLJVcO*O7Wo#mDd1RF&i}ATeaA%Xs}S@#g>8G5_xt+d9jO z3!aRym%+$IbKKNFoaz7ApEDc=Yp9Lzwlj) zTZc*|Cc(i>B{HTW1tw`L&kLcAOt%D!l&c8|tDtomSo%}J=(RO3ks#n_D>6*4JKLs8 zqf%_9+*Rq1QbH2G|L~<6aat47?pPTWom#b-dO2?G{5p8b&F%eRfZRfY6x>dn777jn z5lo4eqoh%E6V>koBZHPQa6msM<-QVv-tWOuBt#jj0yXtwyq#QE`Fj(Z&*Ma8bsiIbBI+>M~H37IN0L{`>xOuj z9`Hyy|AT)PNF;?1#9f2gxSN(=WFIOatCjtlK!GJg0tQzi!43e>)C}oi&e1riil~Zq z<$qB(Z)17E&a2m{jzCa!c>sG}$swLa6wJbOph^)`AK=g`xy5J`}8AMLMi zIpXDI5L@}Vn;$LKbtcy85s(_5taOamCxl(r+@o?Xtg84{mkL-N1NpqR{^B}xOMevf zv%&4^1JQbc@5K^Kg{aJW1H3<32R2flK`wX^K$)WIdzrbxNwEqiL9v3*({6{v1PvVp_3(l*GA*h7xVaXI`qhXR}~K;H>uo9U=bPdec$82yf+_`pa##^$Ne* zPO30dWV6YBQ@YwNSHca~#i_^Tx}RUc4c7_j9%$fr1@`)Lv)_xVZW}4b3L^&^(X*c| zqyHY!xofWngTG4nI4tKW`co)YAWyjUSKbEdhau)GDgA36#0S*udC#o*PCWvDf$&!Z z=(cCUh~Ymx&w~8K|CV<6p9TNFzaQl9Pa!gtYV+hfsVbF<3{K4nWMwe6F?4dOR?)To zPXaZu2scx1-KDyEyM_IX!>ZAGTLla{1x6d4-Y3^&5|fso#%OUUBT}0IG{i}g4>svJ z54*!BT@qmN!3+R~>QKjvPbVm0@-5!3Y#&+0g_BGrmqd-g$t8aDj3Sy+W2sG}$fD@S z5+}{hh-yOUvgAmSKPrbPWLutp@_wplpL@Z{XCa=z!TOXBPMk3z4>2gPL^Qp5j5qS@ zN$o2H#Bf6deg#-WaP6B?k8Wl)H0HHRlh#K9;xEz|x>|uAszpXJ86?&~xh<|8$t)b- znmIbg;e#kz+qP}nnPhrmO>Enk*fu71CU!Eh zCz{*yJKz1zIrpyBy;k?TyZ?Es_VZTNu6mw1{x~SH=CO6EZPIR>E&RaHC+WHY81mI! zY@R&(d*4Jvu|c^slTaE;DazTVt)*`tIQ*tft`$;7Bbf6@?-;eJh>M(fo|13if9s|H zYvuo;hQ>MxJwf;_cNT@hDZW05YBv1upR zF>vk4L&Z^W(8A7fh=Ok2j+c)jz`)24JapB4n0;$tvg@NBLZnFOs=T+gwRNtqm6bQ} zaC4{tflWUrR!hygHA&l3EZ>WFcEgS+0uY%XMy#lMfdEN4tB9X@Q$kEUCP2Kcay zkSP!=dGI^ux8>*otGFuX;G(Xn&Yzz;O_mtG_%Rxk0xF!{>spaJHB3=nK(vtpzlm$V zU9^vY-;9?m82;_@}z4NYp`_y&FpW{{Pr?m`Evam=6$Tm^Vymd;Gu`{oh5&{;Dgk!6J4 zY`GU?x~&W)#=z*bm=;+K4%Ee<`fzF0O|WY2fZ zS%6>K{L<+}FD!9{A1^dxOdkdWaG?L(NB0YN_IJCl^~OLT_-gy0L~wX6Yn?<9eys4% z$J`@(M-U+-86k#|;-o}fGK#QsW)h`=y~O1!bYdp@_8w17Jk))>miw)Po`S6?6VBi3>@zQ8ZEJ3%+5 zkAbu*bv|AZ|89f-^}4^wm;C>wz5MHse40+JFebPPFsOgO={7=jkb}00KB&z`GA#j~ zg)=SxiUc3%_c_%xJTjgVg()$sib;MTF^e?_uI=k(2N-%h06=zP*PhSG?Qql85${40{iPuQMWCzpDon&!1Oi!c+hsIQ?2 zwLLQe+|+QUhWDO*Q9os@m!cXCBFl8A_F86woujOp1w-`U*@)*nWVxDttc*#|Wk7=% z4k8wKEOCs3wa^*S8O``R0 zy1G#*#KgKSlDcX?ndc-^yM6lU=w~a7CD(g^yJ%-^5}tT};Z>2Zd6$u6R>+Mf9n)reh z&}_ePC4{hMP@{6xglZ`9y0Gvn4Hv1n^G|@V)bfQ!Z|p@405SGyb)wEeKg9G*f!K3) zX_I3bONTDEF*{|&L?^^+Cz)KBM5BWGrpW%#XD<0rE2Kkf@zJxunHk=Us+o|_u6yD( zIE1glIkenwcv)&^O3XcbuMHE9kx11lTV zI9lYCKR&^IVCL=_7i9p0y*?OoDfOW<1)ydM@Kq8e(A5)%`co(pY%zF)i5gV>mt*}7 zA9l;i^+_>W`>JMzr3DA!@p>f8Ubwz}RttbZ)J%V^lV)-%sE;ab4t<;>JSQAeNLtiQ ztdrFd#aFzEo0CDWBCF7x^VjX{2RdW4&OU9ZJHYuNKbj)l*&WdxqqEQ+U(F8pj*|QV zgZdz6y3KP5{qqg&50v|xHpBJy&h@tH5pW0Mqik;<<$_oOPny;nI$Wq>bXFeCQg5F| z{*Tg7Cy`lly|jfHpJr0nKJuIbd1x5fFA7x;_%eGa{t_gxM=JeF8^gfb04eYoU|7c# z-;yl$7h}l?ax0|yZ~|tUJKP{hoP)YN62Z8G?Y}VdpYZYDFoQZI@*OcPzYkRhc$NNHHe{tZUabpAv({@7aq%C~KVLt<6onpK&bF!&9C-j%=Sd65iI<|yrY0i&Cg8Gwo0f4%7+h}} zFoZ$C+M63%$Q12!y8x?@&@?|&k{{qV+<|L)0XcjuRUJ;R%pOnf&I)tJnYMz9Sbq|{ zQbTqb#v5N?*$qC_42_Rg{~b8yOMG7?)57>_*cC{#UOdwgUu$HHH#Nl;`0L4Ed*L0) z=;67x0I?MtZW-=}vVE>~ThHn07S_VT3nMu_$s^p*8|i|4D_Y%FlOiy<58fC%6DmzY7{1o|KXy+~ZKmvHObuRa;1*Z!1p9s>mmVjnC9}ft5tY3&@`K?JQD}u|*vT2x_AJ&QNsS zv$Rmmi3i~v{F4^Q$t)IW1;ktaKe)M$Vb&R<_pbPzI>zd10~2Wzf$Z)>9sYl6s(pV7 zWUkrijbIHf7_v)eJLs!EO-Gh%+WBs;S=#6#QI4-CJVB+V&@&n(r(P5suX_2Bl$i5ojDnYt(|D;_oFtDbg45x8!;o4nDGT zXX_mLZNeicEZ<2L0!PO?JNlJiH{4&d?M+`bt3`^Uu(}Eh0}HfoGNXu1YoeUqVgYKd z@ZYO!#2TZxUG)01=)T0QKw>GjZ*$Fo^BJtT>aG)EBAdlTvE!)TU~N$yYd8$^(@mRc z40pb*e|w@)FDv#~?_sq;d42tH5B8D27x$bLgska5Bo**Y83@TW-O_Z@7d@=!xsVwn zShk~%2^`+m@E=Ldvv$OGNtoy!LPgjR46rQM34cgFVsFvtr^GZT_!)au90==>-kn|7 zxY=dE)qQMOdW^pa=M2ALp|0Yho9|1-Daxs$6sVTET|fzco-wtlz9osq-JRQ2i=T_F zLE>4}d4}(_03<-m-F044ptq^FKa*GEOrG1>A2GFqFpNzq@~CF+p92Yk@2&qdhe|9B z7>v5e^01Pn?Qcy_oCUwVKhx^?^)y5EgR&{vX%O)N(bD)6x|QH+huY48H@k20yhh87 zx+_`miyV$dlw%k-B_cfBPad|62U*6f9J+~ITc%`31)$j(H|P#l;E>Zo9Od^#shz5Q zb3@sTZxkdsUSP;d?_%q28AlLEZ54F4PUFUkUEbZ zCLPyCg=}5r>;T38f{fQ;Eao10gics@OQd+ky6|&v)&D@FU%kMtVw<~Bydsy4Jlz@! z%L;ME0~m9DK&h;rmq$G`-{6)69s2WkpwfVMOtH_~rz564)!-g?UzURHN*Y1N zF%(UD(clP3{yY3ECQ%Nqr6mcbqQd&$Xb09!hJi3AIb>vG=n)gts+7uPlzD}{?D1`| zjb@ly5MV=e^xzStm~26$nE??>*OW>;S}!~S8NkYximwrUu(rJsG>m58P&Ak)=9_G@ zg5f1&)+lK+g;GMqCx$u)xr_SI#O4sTt2i!^cn)M=HCyHyaS0`j>3d+HSVbc4$_Ydn zPm|3fu;We86SWgIKI5KouGiv;a_VdJ5u7n3xGVTVEp1L27hJ+0IUPU1x8K4aJwTa+ z^Z_-z&D|Gpj-1Q+qmOl@?D}ac@t)8$(z+#7NVF{L?a7JBG#7j7>c4i<)p3c*vxZ5U zm+KNT7TSF`yD{22vfK1%B{gGh~U#}0Pd6dNotu|bA^`fG=%Sf_Ps@}>KoV4>oEI2H|^Bci)& zrl;SGlrp>i!AVtE68UB!E`;in&!%96;i*iiPx(?u%}`AY$X(+H|uU@u1us8jYLu}+y6ME|%G&Cf@(d^VS zZWY`=*92tKhzoU++kB~XF99bqV)cRXV5p_dr64n1{ivv;pZ2Znfs8$j zQBjL--wG*C-%i%cWDI)L=c4d*rqs&_lrxs5xw!ype5-MSbmumc_5uO0!xsf*eMt#3s; zY9aUcDb6PpOC7&Hf24^`GXy}w0C+7}Yyt*ih3?gV%-*wzETLUI(zMFBo}xO~{szl< zRZ`9{i`0q#9F+7~iu6fLL8SR-OaW5brmok~Crp61Qmn@wjiy7&7S=uu&+tEbwEtlc z-2X_b{?-;^FzU-_K(=BB|JON}Ng_9D8a)|5LTg+Ot{52HUwghP1(;u@Awtpe13CV0 z?MeS{?dgLm@_%Yi-aqj?@tfD0g(^Tj#+WU1dM`p*gkI|NkqT6j+nD#qu!b1KZ(tuZ zHws<_vz2u;q{nRj$NXv8<8gW0JrHf-F3_9?tb3KHD*|mgk1lZZOj?Ep-l#0uLOZ%7 z!}?RRm!-ZGq03jmLW{8y4Bh);ZJS|q!nk)vrq>J=S0+wL0gLji6K=X~WrY4Zw?q@P zXE#7WL?%(G+{}dQDo{76nY;)~_y&$W;Z!#HC*7?^ibv zN>IWnvZ>e=1P%y-KVF%fp4%ezg{*der?VC!ZI2S9`Zv{$%w}}Rg@uh{SsEY}Vs{HB z%=U1^#Tuok#lkcS5psKOV9NpK#Sn&3{#aqh0;`0{xa1A#8qGa(DGF(Vrlqp59FK+1 zS0V%Y**0l_*d9O=Zak}rPqug-CDt*>(ObMf&?`lkv3?_+hF(9yGIX&AVQvxjBlY(E zHYk{huSUgNhT#}Ds>Z=`!8-mr!PfOHSCTMSMR~g!PmM?)xjgKJi8f6;*!HVm8+<#vP{fAjx~drdou7_f%ry_ zdQV1dHt4bD9hmx00t@Cc3Ne~Q+Lh+&07A_wBNv5jVd{5yCZN%mxn&o>ew$mz&@W@-APJAJ6#D`)3$1?-T&_+1EB zWe7J&zi=BS{LH@dC+ib@`aI&`eV50L7*JT;L6$+X2SC#{&a!zfP{hU@RR=M*d9qB0 z-(tgy7Xgq~C0@p_HI3CTyla6v**~0fK`Y}V=o3YJovqwS4TIazatYskQBqbB$Mp6M zDGVdH#iMeKqjD^(J0%+>07m8Rd@loM^+9NIEcmtY=yTc<<)^|?tuH=bF_k4W*stJK z6VE#PqcyADw*_vRe%a(eN7?g?yAb!*^$TdEW1QM?rS z)0x9O0a*-KKtaxOcLc<|digwi6Hs=>HQl4V50Q`u z9QKqZ7L_I~wkSzJlfo6;@Bn}RPJ$l!S>zmyGR?Y5MN8&eGzp!yJhX+a5~L0Ck9aCr z*i*o?+E;f9doq+2i{%VU*+O4G&dKec5rKhl5i&wr=8e|PR~tDDAOg_za9 zhYLb!Kt4QdpyB<5jm(%-;dqb~EVqH;PdI%B^#!HKo@i#%2K*Rop|8qjg)=4s3r+)} zqU!7SZx7kd)5fpwPb{p?bD&R8TKki*8Q+{k=B-n6HfbrQC1+Zj%tSk!&$B4Qg7EkQVpe! zj*!M+AT>BZpog_wH(jU7r>U3O)sLDcy{Wxbe{K7aXlku2%WcLsKNJmN68-u;g#4xXI0E8I?0DdmBgGDUpxG5!#x7S>x7g{4J z-u8&iGo3CB>0h4V{4m5fpD-jY+GD~o$GNmt08W_KqSdGYR*?Jj$mudpeV)+d@1bza#*glD=b>GV|D3^&>D$<5A2A>_r(Bp z^WOdXCv5m%*ZBY7o>dH8q6Bo682(>T!$%+s7Vux8L5kM4I=&{xhf{-TQe6RuhMjx? zMm-{&;{-EQg(O~iVSxs$hKhdL0ai=M=8wzLWf0xvxGl>KX88v)`=-ZRHrK~g5B|hbnU9$i=hFE zZCObTXQ_tVq()V*&Pds|&3)tDz^G~+wlf+Q!>Adn(9x(Ln&lh5Iy)@nx7LX+E!sW= ztben0<5yoK4Ce>Tg|_pzXRMvJ4-y|4Y_Gb zdVQX%2gH64>aNiRCE%JBrz(X?KcavM$P2I4 z=mmCRN%dUv9riMh*1s0bUW5`R^|Wle;_PZMY=>Qrde0*q`U~kfe77)%x;jaJ-3O&? z?_63Gn7EJP3ZENj2=cv?%*fY@imBWnZGP$R7+C?RZ%~S}kUE$l)aGudFUfzo$bwIO z;<4hnYc?ZwFC@y@qs%(?OiKce)2ap8JKRzZqE;wpmL8t55r}_|Nq_XGslf{GocTl_ zi784kK;K@aLp6M*ofF@t*>d;RLF$t$KocxaRv6ZJLUqAGld}c3TUWRX;EUVgEH9Af z`Nnlt*9(+f^u|qud2|0|6usVKt%-`KR3ttKxmEOH<3rraxzV2>f2MtiVy3P%}a8j%W)7@b39?7XK7DyHGPSh0l3T7(ZtCu!Yu~Q*{d? zVC)qthyJit0WN`l(QL)T)GA5OiEpJ?iH(1()LoQabR~r!@EJqRm0u}nYpn@k2P1|8 z4{u*=+wC&ZJ;b3xdF&QQaN)GTB;u~k)(nxlA!)+f9=yYap%*^XVT*r;9=S%Yih&n? z61^O}FN3njVw9icg$&j0gBo9T*CxV>(IES~KFoI1Ciar7E8BuU;!R`NF;Q8HGWAIJ zw+{S^>zp)Bk?s&&$rtJS(SYemP6;thGExIEJOx!3TnGzp8q!l>v)n$5)Yv+`xQLkt z1yJe^E16o>5`SvkIDani7Nh=?1Y!`I0e>JkRQb+5tFbfydg;Xo@d-)zO@2N+^r;Xn zE{(QwHAan36gTG?M7$l*Bh1(e-I};e&(`d^eH0b)8B;r;jd(0p=mm%LgHc~5lEdmX zZx3e>i*iXcLr@CX1;nnI9FHX=NIMmj(h*vdZwya_5Gv&mF1XIyG$`DH*&Hk&V)ZZPcZ!Dz~|agOD!BC>?iq z&SY}1IjWU|B@0*pBw8VgYawA7(Ehsl=pFoHL%JqIBN_OQe$~Ih?Eg(HjHG#k9={g@wGjH0VJ%hY~~L_wn%2& z_Q~8{@+P#?1|#q}vXxL4=9n=&wPF8g_*@YFuTB2bT7Og7-wG!3X$s=d!oYsHK@pUI ztR}^nIlHPL?@+B8)$dU-(PAWL1@o_5y618o^y6A5!oD{YK7T6=t_@S7L%~evKc+w5 zSrOme{C-5*fz%f=XcF`gaA4t>E>1kk?!S1H0yGCIL5b6+%5D58HBZ>Sag~zcUxu5J z5=gT&4CG`_fD%WcrMwj#0Y;0dti_cerLb5LQK2hoVA+W|mGq?JvvRcWEc6Gh@(PvE zLKZKJ4DBF4FeneDFdRGoDdqLwQO;&Cs-##5)p%MNQ1Cc2Wfwv840{Lq&&<+}hAyI1 zjArQ00=k7~`cMheRuSS;$Mj;|x7Rs{f$a0v#O(xqW=u`~B zq^!E;0%g}qqma9WOqk6i3WP)cJ|IO+C-ZW~Ei~pr+`0v2*v%P&)S{w_v zHP9I9ow%Rmc?afM)$D;Hzm6Fj2CtdZ)peX_ zI8=!gk)e7D_r~qyVL>`h!e-9##Hrj)*q;f^aVeB}FT`~Lj80IR-o@`E-)tp-CAOi8 z*pmeGG;RkhL#z=DFr$#8bPXJ>JyoVOtB^({_bJ}%ZK*v4BfSR4jy0Vz-=D_ES%3uc z?jP@pIcjVt1Rl4;=sqpA+tweN4_3d(Aw6p#6}YBg5(k)nT64}+0_Vnj2+rTb^}%7;K;8= z9&r|u`=yjjGn>SJ2}I994wn}kywX7aHPl~oRDz834vN+P+fZga}ZF=L0*ti$0+^hf@i>kS;T5?)U zxK?Lu&rZ0O)Y%js$`PwMor~4nY*`|*z&q_TFRc+hFzhz!S6Z>knc}#uD{ZK@-zuzT zAED;e>ZAdeAnIa3#h7_+f$`@n$d49$_^?P}IJ41XLRjDX@>KkziQ+|r(@LnY*56M1 zpvEB;J#~D*c*k}zUYM2vZEi~|KAFxjVw}=Eea~PwB2n9VEOJr@0Qwbf4$?C&ynw{L zcJhZCFojM~mdr*z(KlzfFLQ6YO0x%jw0`-`YbSzI8$XFUamwk&Pzg#PHTF2XcEbR( z@{n6G*-=BKnf03}K7<~C)RH`g72mzhX)i1$tIcw+C5FGv-kgXN|4GHX+`|xAT<%!) zwrzt!&>kdX!ABq9a$m@3S=O|6*>=j9>2&f8^eCeJCG9b_3w+QSQqymS*o?9A?`O#s zCQ|B~+rP94Y6;y_tUbAKr;e{4p&^_6Kok#{-P_%BK2-DWDt6tEB_CkXu1K~GTK+S* zL%<3PBK?`Pe9<{!hU4Tzt4MYt`s#I-XG6Tq1<`Hf1y2f*yf{t*fSWzcOES;FT(s@`ZMd4qbPQqvG!6R7^8h6XcxBl( zO;-ZwMb4ZA#Pvf|DND;s%+2$W?+eY3Kv={`)8Iv`T67NLN)wf(gd{YN{Sr$)l{AQ& zZboML%Ge5w z{Te!~jF1@dB%ci~jdms)yt|*6RvfIR2x@L>aT1XiEf0f**@PG zB@R<}zLtDlphz^jtKlZmqkpzSWDT6f& z8`{ckrU&_ruVtW|8?$lbcmZ>;Txl%tfa$DAcI8Dwygv3cz9e3>{z#(wxkRM3%qguJ zQc)8@1^DTQ0PuT>^orLl`@$B$@N_|e@+{8OsZfRnb6M6rQH%u*d~m?Fh~iL*5#w5N z4bTp!>7x)<_@q8B9V%a@;CJU^TD$2W=XOO!Ym<`liM|g5#mq(liAPqmZC=A}B0Cr=G2=!@C2=%jT@7Dk_qP}1aXcbhP1ESBS5|9OR$^J zOR^`~MX_huHL&wL@11#P$w!VeC#*ugfwz+NMS#zXjf(Av>I^$GUsg2kU~dN~Mu1g` zt*|f!$Osb)99I+3dixl=R-mQ3Byph!1Ckfh$B2fq8KaDD51lTtB7QXs_mFE)kfGBt z5@kHj$KoD8O{Fk~lI1M3Z1lt1$)Wxcf9dF8Q-0U7Sd-->r+6>+yHLaWdXJ0#$WQh` zeG5lRDX+0DI2&Icp;B~M`8!|k3y(EXGkQuP$!%#;f!BqaC*`t|5$)opxt{4r43G@B ztSH4`g1&t>h@R&Zk=@FhHrTJA@e7$qlFNCyqk|*Kryt4&{8d5~LN!{5&p1bvUC=J9 z+P$SOW;Ln8nv|nr@$Z=h6f15Ag&dI^4yqlRnTQ|}SEo?99tPK0_3laa5)woRK9Ao% zV-i%Xwp2LNOXvsq{H+W5xoSdFi$F34ymQGUC|NRUBgnTG3qx+4Gv+-G-0)FMf;fJr ztxfXNPgLGdA+~V>9mCl3))13F4!DsD)lBxWQCN~^7u@_B=6b7&?-f_&;z=`ho(G8gF-sD&_IVlnE=88 z)`K{bngG~S&BsKyRzU#uHem3ug*e(ULl0^t6G+sy8f~^_9}p0Bt0@f8O**O+)9!Reu0ZF6u!`oCy(c8^`A_ zIB6+i^(5cANExc#;!DlDN~L~&Fl`J`{c@6~*peI^bH*<&x*8$susBo^@~wPf%>y0n z>z)T#G{)6^4fsa(2q$o^UTu|UipU=mbB^F(Wn35>5bE^EnHC?-LIRxL!{HA{KiBr} z`!aZZ?wxG~+yT6zTdx^w*|w18(DA`&o8YFH5wlfP(ns)#VjT1OxE6-)FQ&9j&>its zQ%qQ?YL=HK5fmtoYqn*2%*;OR*j1BmHaF$PG%+MwVsgQjkH5gsLoHeb`PW7l+7Rl> z?tb0sgOD5AgoC|-;5-6RJ zb5rt$K*tKL6j?g$1WkDz=Dtq7?_X_yfnkZrv%7(2S~i1C+q02QEgKZTQ)R)E2AIS7 zhfH?4N%oJr4gp!QsB;X!_uD3l>=~%g+G58cZ_~|OgI7v@FR0?2>f5H2tk9kzy?r>c z`pVj>cG#gZzxu6ll;kkl#*z~T>gYC>oWcVR0Q8P>D*@@t(RioH^XiiNcA;?dNLW&- zvh3EnhpazuCvXEJj4*f_Hdz*_V^)0!IW`c$tv^4;YyZ zxwH#Y!m>;3OKhjOe2AE&HVZ|0A1($lhjq*+eFt4j7~?ldCuYqCT2cYhR({?~I1dIh zsgUK#5TMSil>xXPmzV5!y%Ok7utVXNYpESa6G0+Bi>*6G60^Aof}^-CGE=MC!z|Un z^xY8F8-S@ywnzSc3t@GY4l@*M`@}I-ywTu_m>rO|x3cj^fn8On(ffL9!gx^b@E%e5 zB;3)2D;|b?O5VUf`F$H-Ud*35x(Z1)z9YZr0^+Z_4%a#eINKcgygWU^+|^w%4B_GF zC@w2_!hA6aPc-JDDL8*_4PFwN{3-+&9|p(5-2vWze1g_Yu9H~Oh*a59Gi|6-Dc!4l zCWr;s4h^@&e7X@C7Ejjn@86ThI@Wom_pjN($K2jah`PG&4kFfgi;xB%?}j0Bam#Ys zVmUsgdZEIMLj3ttJm0fb98KmbiR$EI0!Ho0AX=}I-9~DLQ_gmi&2z@eO+GFJei3|- z{~ovo8frou0dNETj4vjz2*3A~gTvLRjNL4;{B~2&;#tL;z^TIVGih^bz@{sAZI;}V z>uF&W?B+M#7&K}ZX%#Hvb&6+G9l*E=5y{y-h|c{r&x@@81VNNe8ug?g-6{QAkH5R) zGjmz*0V8)%e#c3tkzhGVqfqUBTBX9h)&$gJ)~(*`hM_7`uPAHSVn3%;s@|b4!JD>_ zQB_ESGwUi)i;R{U(kVNwt!HSIEYip?+958TvKp69i@Xv>(7@DZ(w)Rmbyt?`|31Yi zvpMDRu2G~?DI%Z|=(CwbM6KCt87Zec6~y#X<6PD%35lp6^AS-V)x!#Fmbp*SX$wph zPxks6Jg#8}m2g5=nOIR4@o}b~O`;o4m*_m7R0Q!5;*n^@7>KA;0CO#UEo>rT=>Z%@&7Mc( z+cUs(95ccdZ&7%pq6gdNm;RtAnWJ&-E=qJ5tUZ^I)GK;Zk`ns`7u*{FMCj(6Xybc3 zgI?>ELi|EmNqJ(brC6kAQL#6u5y9Ft|N0$0twKXPqRPEZ2Dz7k%Emg8q8s?Y68f_( zl%HZ{p|-TS(I*SdmQq3}fa%&nC(8Q`ZRi>e5q>Km5*>c4>K$PURqoG*)JyAFShObt zZLXNgz#JxlQf&ceP*#6+jFq%a%Ni?U$fu~{On=V#*9HXd_w>b2wAa8o=qqw`zj2i& zf~-V^sralKYTvd(VT^2KWi^R@m=9Sgm;RMM@tGJrVk)kX!20W~GHTF(^!vY#gZ~#f z|KE%>Z?8|D1%#<42x9Ddl;JyfK$?>quf|8q`|qouEn4#HO?>hS}%ulZ_Z zH9;zf!DxN_5_+(^R&6*;SdP(27JWA9#p4nse+sLd1Fo`=m$=p^^$Y_p<3NH+)}{S= zC^X2Q^Co-zUw^(TJl}mc2EwoeeIJ~Shbuy-C)KNI&@rzk8WHp_-4${s3aJa6@qK`b z1~Th9H2C0zMLNLp9{AeVubS0Q?Wsa;(zt|;;^l_S1!5vff&nMI=sLk)vK{qPEas}U zYAQuolH{By7An4pSa+yI{Y>3IAF)>|z?so$-!z!3{00}I1^=_ zJSh=lUsjuWDchmK>X_PIUtg<484MS{3|I%&F@w#B$lsK!X&>kgGj!#SIOcfzj`as3 zVx}dRA3S*U4kMm_$uaQp-u$itmpSv%%b)gjb%K|GP5&p^7k;9)G%m^94NnH{Po4AK zeWwsk&2A-4>g8yJf-q)!UW#LUGRr?PtK@zU2{faj<|uI&Wez;bWk|P-`&6@sRecWoIjc3V?v6dRv7E}C#!G<( z&mr*MbW_0YhpD99!W}8rAo$1}p)DVJdm*>cZi-q>Vq+2M4B_GPcHwNZPyccxWXRcXH}?DW#&SnImGSq{<64 zv;o5l9P$*LQF7=Cw^SD(6_R587;YQ3%(M6r5yGA{4+__43x5Bm^FW|-uB1$^boBW! zO^$b)6V6>X1e?lV}acf;=} zemc@TXIHi4Xb;i>o+qk*km&&8e1o;O;AFRC>lOgR24cZ^5d|a~8|~fWS;QXbYg$&4 z#i({$HF~Qln2$IM5O1DIoGgW)&o;V1iyV${!Eb&8Zdft)nn8b*H(bUM`X)3TaWo}z z2^QUK770Q}hV@RPF>FM5=?IiD9u8heLnQR0s*~gu7o{zcyspSEz6il?7}^IoP7{z) zhd#l3TxSo-;X=C!Icf<|S+l`UB*01OVrTb{q|Y)skh&*K5>P;`eyyogCsUz)>}YVAKpLF(#THPQRSFN6fI?iA10$ zvZZJ?$D3~O%3vP#*2PXp(`YvhGaG*BqhsBO{jNBDLm7gbecrkWQc6JdB{KnZ1+E4Y zo8UC)4xf54;>8<*%T&o8E`ei>lSh<5$92=uAHE|J;yYjW$6dSY+-R+0vFU16T+TBD z1^qtJbR>~u?hxmNp*31ifM+Wizk5ya-MFMi^h67(Xj zp6`J~11D81$S(!MJ}#IgnGEg{T|=x&=?A9tG?gH~fBm9yDuXl}Sr8+_&TlPcZG}{2 z44Tn|yoT3jA`qzElf_gv6c|j5hR@~K@v7T1(adk4PiguLULw^BrQnIliG7hx-nO5=jNr{#XMmD{4Xjh7)`TN*<0r7*l5@nCsfy~(; z7%jd{K>debIDe)Bt7{Q%c4+XdvKZzoA>-$4y)Ni4C%LK?xVg5II-y+b8?RFq{jjpM z6WoRyzhvt zE`JQ(NV{fU#5}fI(C1~YkkZ20bv#WmpB-|Y=$r@{oH-RKXzU28;1|8j%4 zhd^AqQ(uFzL%m`8D+b;LoAZ$@?d)vdv3A^+B`rjHUD!W2J$29v`1<@WZ=)z z(wMoC%9?FInpI@hrC{u4ov0uc`tti8#WXMZ4Ut6n4+c?HfoF&})jbMYpKG;L1uvqz ztsm#FAHUzgUA}D+a3ljP>a6l3JPGmnxIiOGKSIfIlkovdvG)#?1J`R_QUrwGq z!mt6CA%>^T=+DVA)@tK*<1Q*089>boRvSlW5}aGY3JLGqRqPLltIiM%zTCl0q9@B_ zWl=`FnEo!Ro^3B*6j-@mjb04IbH%p6LK>~Y%aLyG8$wq~@|$_!z;3#*Yv!dIZ!NYA zXBX=382-sBR{^;_S$I$+AdLWMTi^gjnVeEiUV85KoQHl)_o5oh`#?3cXHAuT=L}{~u&o`%ceL zf6e-DTY$`p_P?WYf17&rzq}<4r_tGPD*}Bmv@!o|PB~Gn<%pok$RHMngd{r1V50$4 zXebTQU>O1%NNEeOQrBeJ>egcT8w+gooAsI>95G2{ntN3A4^=Z^=Jt~80)&56+1WQQ zU)?dk559;#WM`~Raed!x8h_esfBMmW{5E~JwN_@_eh>1UH*#XT5*4Js1D+Y>F%t6vuW!N+dK= zE<9}(EB_pV-;5QHE|7h10$b+uC=0X_oEjW-*eRp%*Bc;r@`uh#x#op4j=z=+EIbnL zKt-mN>cskqza%bL`E>&;rqUHiQT(ftsV+a0+EqMS@_{jvhz~<`5WG;IWaVVx{!A0R z8I18Lx-!i5N7;R-jrR~XrCg>%ujH0U7MGps)G9??1Jy6X<{WS(AcMlr5e%@Yd_DI> zxOjvgCrZSXq%?#qr=V@bKI_E71{}XZQ&n&n8B#O0>b+jnGY${_3J+zh!jynldeIzQ z9wJ;km{dKTQRfM2DWbA2%d33Urd2u`&(2^e#t5!rG<}QWgx=g(d); z-pBCmIw=`v3PU`TF@a#8fQ5D+k2b%2<5lQXS^P!eOZIl zYed{_i>2*>)(R|RcUrCQR*u^V*M_*-*U2GQShK4RESc<6CQJ zaie=|A46&%-;OB+nD?1|L`RHoxF5H`a80qkz}A3cHhN>*;p*Zs?n>eUtrdP5H)c_E zQS4+j2#$068sFk%aA(x<=|kFDtA1Jc@0iGi9H<_&Y;0Q4@J{9EX2+Ntu-a60&c!Zc z+D<)N1=r+s)Qte%u!Pqb?7RTF#q(9P$BE@=D$5fhJT=;v%5x*CI-wWKMLdr_1`P)Y z)iCTpv^Z7eHIPRx@m9%7jMDPWFK;dkOT&1_{ySZGHo+pmM-leVqzLkf6ZpEnKX_^K z#`80j8x8olKQ|fC|F#l4;`WW`H1k5TyXvmo&HJFxf>{EVqecuvw~<)2`6Mth%5tf# zWHg^H7zA5+Kx?j}-5Czdk_j~!%MR@qFOEh$&1Xo4uT8gz$K= zNpXL=$9qb&TU~s0L=fJYGv0f#*Ej_Re?*?udC|8=)NCa+^jsAvAYo8K*QcXTO zh7#GxVVG+`C5ZAqvMwk7lB5GLs)->73lafWn{YXo=!|I$%yrPs4Q0RDpososYnQ-C zq%V)^W!la$^$=x=`%ulg)iD~DL>jpMXh|Mg=GFA zi{bN2KAW6IAoyp7o^Vtl)7=A2RA)6J!bqJ;{}|jnDg)6V8cfHu$(-V}-<7A5+uMV=jvU1m3*8L*hgo(X?QO&1#YGeJ zhvf2mmerqrvp4LewlKf0poeGTw;(2)fxzwoD9M9DWf>n4h9NlB81+K5vSBs>x=I%< z2JV2Cn8GrZ+fAuNLyAM#N0nP%*UVxd%$z9utsY(Yv?(*6B6FTN_7uU*-Q$!l;l@{D4LPSi@^0`Xv+=S$er>FAT7PYZ+CH^3rTdw;J3KkP@q*$~O zXP8}JEt;HcT-sZ`KMuD7#TMdN+3a#3ZI=riF}wtXoCJDmMDukN;&o(<4oCzn!>YF4 zqZFWP9fWoC4bdzAlcx{*l-pPY-G(h+pyGsd>@U22lm^=ou%Cb!t?Y&2Qe02^1@lx$ zoe4QfcCdT&z8Xju{3r>1JxPqRPne+BKFROR$_N4#oM;p|tZE#Jg#SaYN+rE-aY}=gJwr$%sCdN1S{e1O4@7}v=)%oLGKfAlCSNDNV#}5!Sxco`E zI>SXc$FF6jFR#ScjybPo&nq~30AbxGS46V9g2{~2u)<8MB^gKQi~uw#yT?W4SaebX z$*pOf`_Och3^jc&1?YI$nya$h!vjb9LUikEJrGeoi3$iy|By8DNna>x>7zg@e^nE=;n}oj0gU6*w=QoS zxgjoZMB~PB-4v$ZJX%AT%44om_ex4UDVJDU)~6&&?sZwg!U-xU0{LPONW6%``8)>q z8iz?se-`w5QPxUHSrpPPCMTISv9qV9g(+k0s>vk}X6qCn z&~eW6rax%Fl#*$R=P=C203st)BI(G&Hw)Y~I@SLw_&%7mPvxzWWXVZEUeql5vF$x@2tPJGeo42ze*fFm z|J&n1|5b&-{NG)@rCP$YYGvbVA$h;}3XxzYXfUHiL#=IfWkE zNiqkm-WLCr#8kmcJ14Vc3=gN|d;hdTsm;_&YjA4mDMz^7EH%|3JXPFnsM^ z2Kymjk0mcOXRVo61DvZsjn-Ar1(5Zl^F)osnXNFf4uYuI!x@qt zs1>V#PLp_G(qgaGtK$nTrmjVi33Cz3LeQ4*mS`B%*EUz*Qh_sG z?;T6@g|Abx1TZWZ>!SUGuqIm@?nRkgNx*0__h<-^d&}kAUN?2{<|+(a|0@=x=F5OI zM9nuk)-cN+y>=AL@;rQ&k%1K`TkzA1lU`bIrKybhYP%FAyFiVUoMok0#d}7Sbum5k0@!ZX%wL%U-vGnUAHWF)yslAVCZ7uf?|-E|y`)#q zyNYQkYD(Clw=5+)9#cYA%!w_L{E1h}KKK2V(L>C3UQRd2rDjpd%a9d>8d3raWN8(& zw1~6*3;S&y3Yt}ZnYq2AD0P3sxYolqD*a^|OYQ^vT0Bzk#~LxqDmP8SRvnzQ3zvTr z9DSiPhTa7Hg)s*?$irQjg!mw5-~%*;9-;DENJy9XD4)i<0m|$w7l(QLiDN*cwHunb z4VvvQUkv_#<2xSyH;8|$dgcF#@1f^98W9!{Pzh)1-X1dM--1U<#SbQe{Qn2=Q`*oz zDoacJC#FpHDVR_JK%l_=#Bvx=kz)8i;(};VQcxHmDmY~H`X!i9&B)fBXwmaVj#mX@`&udFJT)cBi>y1zYdrc9X9C5_(S{dy2@x8AQiPBxuq zUUQGO2hI7t9~p@=N&$=Q7VI;wcRV>Y_s~?ir@Ky=YcHhO{F(%%qXQ=eHTNJCUa+Ru z#F*;0MxMIFu~MEYWjeGwDdnA$qJ^GqaMcQ)#gW5K(VbaTx$2TN%Y7AFogpjbozs2# zNXHkd)roGuc-YE)GA1i9BNj~S7F6v<$q(e0b4?8rd8Qp6FafVJ$hS1?_QwsyC+-gz z2(4%6b}n_(viBA#BWi{iN!y%s+rzZ?7RjDO$piDvV=d0QhX-Mqg?P$sTSFV3p^@(Q zbed;8o<3t!ClBEFtqOcPq!08MHDY`C1djYsXXX?S9D=OWJN0~Z};XT*8nZ;d0LoN3NbCY_WpCOSrFKqekQYjp| zUM&?qq2ulmMIAbegYQL{&Ud;#Ek51i1kd9KnRGZq&yqF5VK?r#Ek&Qrzj_XKKMS^0 zjS3*~KJtF`Q0F3y7Ag}Uw!IqqiC7@Ew?%dwIZ7N6{H6Cy#*ViPFkN^v?Y&d+9~V%+ zC}MLG#}?ju=p0e#e2{(%91*sk;`ojC$xj^T#QT0q9GRzo z&~7T@*>v*K@;;&UVW5W@X#g*y1frczz5>x-Wjgnypb6zuxGJJKFUK%JQ0u15;;^@P zc04RtB$P7whrc?%}kLWCP@TR~i|9Wn{(pN3UVWra8NHJi^J zNS3iQapBIa*R@+tEgttsGt*EDr4rXEzTLv6$1r^Z3Oq8Q%PL-wVG`*$mMH8T{ zEvxyJpp1S?UvJkPcbOT4myJR%K%wm_4)hJuBy^T`;S>@XyDG{=-+Pd0ScxdSJ75V{ z{j7tb$W~a{9A1zK06S^!l{!vu@NJfB`6N(WwF0_P<*N#lPAF&)!l}>FGPFvUSQRaasGhGK-!Z>uK zREv@pu`XjBt2z~l0&6U+sOhSX7O?lEP-rUSNRIBNZH0w*O|om2!3Q-+;~ho_En@|) zD`kz;90x@G&L+#uBzeuHJIP7xDFmN_oybxQF*a&1W~F*O1lKLd6VipfN`k|K_@27~ zI19!`1yzC5#E?LvlP8&AfKv67JW5q$F8hEL==`|~vfiqbQXVJV2^y`&0&sw`gV8_D zl|aqil(^j!m>ng=o)u`w%dH(-6zF}ya;o!H;MzG*5E3D zw!5aCRRKYx@P60+QMVSwXz1CYVgO4@Sz2sjHQLKeGT|EUV9ijD2ryT~kZnqTP=taX zLs_>${IlWM_rnyZL>~9hnfRjPL(G@i_!{qrlUl}0dYK#Q0wuTLp$3*k-MN6eZYwX! zY$_yKe)P3GgEm^(9eo)pdeU&s>}Og+$zl){6LdC<`MncAh9yl@Fl;d?-6Lj1Rxn1N zO*F!71&-_N=jj~DJ%EGKxM%(p*Clhf=Xgcs3%+P~g|$8L$alkodt<-iSydLl|#^7)P zXX!311=HiSa~gw$VS=%aIA0{+l7*>j;YsCpx%m)Zusv1xJFI@8JLp|}#rOOyez7|% zFS#G|Xfil}UP7x6{`;3O{^b|cua@E5dFJ0mEPTVcmYb3GFh8>p0?NqX$ze(n(ZC@T z>lIQhBV@;}O949~d*xJavWn=C_uXQ|A^SzU4j;$osF|o;E*AThYmUz4fC;4YE38w%434U*zz@({MGFqtcBClFvIqqT_ z!Z32-7!vV5xPcYX+|m?G@;gNFSsX5oOwdTjDzDlKu^GUE15_l}o7WZFIZJg9YfNeG z8=S$U%&Di^nW0pKX5AS2WF-}@L=}-y{Ig*#+&`0#aY(%hW_6aW%dwch9eqtTOjC87 z#T0>?)wG)D_~!%bc1>2^PYNTzWxRO6U#Fx7|KUUrCOI_jNiOxK`oxl?NP>F^F1UVm zL!Oq&u%Q#Uh>XPVLhr_h#%;vO3Fn zC6W+cbkl7GJF8cT{Tw4Ccd1G2TuG{;w!Z0?@_Il-+2aG=szUf5ellR!zbavl?$>}y zS&F&yLt{I+o_-_)Tn&uEX~Jzu@X#%R$m`fqO`mK6s~{4PUBiPsy{x(nDNYXQ>Kv9) z+6W&)Q@x6Owz9zPX6y@(uFMFg6!ZSJn$@(pwg~BN>@kPE1#mq67t{|Ub`qTJuF~;7 z{tY11VdU;l+17-Vj~lEGqQ&W0ye z61o-yg=Jv?Sj=n`h?90J#SFCl&5&GQ=XKu6BNO5mIokHaW|DHRb4-+-#4c|rFSGz3 zbtMi(u4OT+YZjXyF=3KHGUcAb-pnELtXR6s>lhnk?4;pXe9b##0`pBtv_&^b($x0r znYbOBU^eU#2~T03(6mRAlc%ziQMwN>*qrT%$|NXhh_<8Tz)*>VRd3gX|5g-yq{PZeLo4gXf=+8v^A|_p(ML6g^o~+-- zdjhRwo}Y@gU%Ir0q_ZzA#rKJ!`RSI&YLkRn0F-7>(7-Qeu8a68`Mo+iFI$54kSb$s6bDbBD;}L&odbA@r$rm?TiT zIUDUL>c&K7{e1X8&JBWgJ4nV(fr=Mz6P+C+-n{cy^>Lj>hUyYKb##6W%o;{l9UG44 z8v!fp4y{u(sSpY@k$_Yq>IZ>*Y<#nsoFG70LH!lydHx=kNOkFqAT6(XvW0#x*#z@CATWwsm;QLW-P2*C?V-nmjFYJHX==Mj%GbLr~{IGkngZPJ5hS z=TuTZTj!A2XH>~KWkkfQh>AOjkTLlZi=}#J3t;s4Ti=KY+!X~Z6_JfzVQJ2BVrf$n z4P&&lenDcZ?UXfoUL;E%^z17{Tl)hn?fI>(NgbAxg5x6>s{n{ohfP5o`--&2RJA0x z_|jh~FE)sCL7m|Q+KA{TeGsO6nfbS41eq3_snKl*Q*DJlZqC(PX%}OB^U$K~3v4ig z$T;Aq#2F@yD@KIuAUNKPe#pA^otY4k3cxwGdEA&<5e9B>1{M^EwxvSrf+Gub8PX?n zAa5wNrLYzxx&R5kTc}x6jWd|aKc;L-0mI3 zUcb<4Hkw+LcH03f5E$HJ=^EIsjY`H23qHf#&qdi#xP}{cnqq@xzVm$XI#}SO*vl@Y zfs%Q{XG19-f%SB7019b(!{GD%y~1_2ag$Y(8|W_Hwj;4W3H&=`xmt8yp?Ro=gsk zD?^VNfSA;Ya<|l~y)Fs7F{ctoKIpjm%ty~088Ys4T?%H-9QkC8Va9b|&k3By8}HwI zR{Riko{PNXn;R*fXMe&Zj-69K5R;pJxf4!JGyDRafk|8k4GfFBVU-TMb(12$%WW04=}Ji5ScJx*!8 zu)An3_f>Cw(Y&h)26n%wNt%2j{bXyfaR42NPCPI`T{-?~4)3&HD7bV)scuoj4D@^Q zV|`tX>6>CS9>1^!cipKz38pfsy1>WfCC-Y`4<)jH?Cp<2P$yY{b|)Y!{l+vxBqe+M zWk@N;!1i!Kc=tr{#_SdN2^(b~``BFs5BlCboj_NP_85C9{5^4ef|4uiB#cgiwF=-W z)~5YEL)CJ!0sAzD#1MX<+|@Xd)@%8yc>k&r`_||qs;5Qda9NOiBxh-r#Qjs8tMp9= zpXKL{XPDZn4uP>$)yR@)#l?}Q67QxJJnZMXaO3pJQ)}^eFrAjZz1xbLV)oM3tSIgU zJf^cJ5`tVs&*n5z(Kgb#3;CLn_zeJ)OBv_`2j~O0_qZiV+J08#4PO?xv-6RBe?psu z&`TSti?*a^mPx>lQ^x?l>sQhvB3IYqWK)+su+E3wjVro*EE7xqVA)++X8o%@XehBXm6@Af-cO)2RM+MVK4yqv_?Y} z{CfiEdvq}HxqXZ*C=$#wlI+b+T`Oe)8hPC!|Tc9;{&ryBU5=o`WJu(=uNw6R%oAw`cq5DeXXRfNQR8W8Zw3 zYdOAS!C`9~z6a|~Svq*TT3AHb%XL>J8z60>dCe#YG*Fy0&|pC`QzDcjzTk;4wfh=w zdpv6a^v3(kt%OXDhZumfW8kJdr=V$=e2J&+E^tyZ6l=)(D*A|?64ex1RtB8FDjiVo z>Kab;kfE6}nP@g-`m`p+%eKhHizf1Paq3(6C~!r@HL@u}CiG-vjk-X3`0%qf7BN#p zz}b_Cc3-qHh#F}c-G~(AQU2C*x!3tnj`8UqEYMhWsFuJ5R zIxE)0-q5lSK>*|0qsjQ2%;r7`>PPM@OCP?A(`wk2nMugW1_yiMTJxv5nKp1kRnoUA zkeN1oLsj6~@<*x}Zt|GT##iiQGqBl4m~0kA!_t6F>3|J9!v=XR|G{_g+C7q;OtMNo zdPAECnFKfOpPv8bG&Y0@Pe(i#go6q4y zS>AZAzeEsuywJRLl@FcpGWB9k6tV^?iCV6O_Vu(J_v$Q71VvJMs-Y|wBM<}^nF+5q z3><2`DRt1_D+&bqu)@0H3mmUNT)w>pbg#)U)NX(+vqRAW)PRX6gp7taqKsML4R9Cc z*=X~7e2MMVy>w6RDH=U19WocKQW4g8ohb!3k|$T{gg4nR2z_yf_H;Re2N|s}PXsE~ z>Fa$(3S#IR)=M8^r8_?Apt7P$0WfM&u*~rV8cFI(ZCrInXwtPM(UamVzSpOyYA)SM z(jEY~(VZ6@8b|}#?qe|lmwq$%)crKH9A}>>!(6{x9rh{x_vReF(E4Pq-hYUp9cOONlHQO(=sR zVuWi^acLw+I|mc6d2_}_v;`63eVKdDrUw9eo#}s^G|TncLuMo0HN4t<6uNW_&yU9Y z{y1TIz4ATYV?Ew;JUW4Sal&-*AawCSck#UMoRPIC`8nJ6ii%?DCVMF=GR{&9`moU6 zrG|=cPKjEE^)T`5Z?tj1JFYo|;p+q@*#BU3M-YpGQB*k>NXF{Xs$x%yj!S8IN74m+ z)^dEa!fS51Tn(_05I|NZV(oH^tQ1bso)_RE!4i~A)uTPDK2pM=6{$ng7SD0~`l&-+ zkcZSYR%^#5=DrY}<_(?>ple=NfAREMx;~uxVE?MvdhPiKllI@8!~eZ)(w(-d*#37< zk@^%wgqP}1&yAW&7EFZQ^2UjB3=HL^!ygZUL}%w6F~{CBEmekAzkw zCgkSl-u&fU?uPw}E{%xZPn!?-StmK(*Pnmh-w}aeHIx~FWCc|Ms{3JS{ViW}J`Ue-&toeO|U0bU^h4Sn~7Fz3uo3? zp{(i8i#}6@C zsf1VDhfk!sG#Blfo+$8x2+F8ny9``qIn*1TreS+dxw%6Mxl1$DIcH4NKGIlc5lzqh z*|GT>YQnQyABlD3w-#Jsc3ze-EApnhWvuZa5?|uTV1mG=OO4sPT zS?q4;zm3t}eIF+I5YHEdoKV>KCSi^ES?GmI^8sxverrS6*Rd63UweY|bxV{a43+`) zwK%Yo=a+9lU!902h&W~mk|qe$kS2hKbL=?qr!B8FoMG3hp2Icnk*}5??NZSpvu$!X z%rtVUej-LFX11g1LO-7k8So|p5vx6S!VU}bowxVb& zA71RE@89bAZ_@wo>iGw#TgRz42@dtw>R|A%+A8ILG*v%jd5{2A>YwUpE7*Q!nry;<6hX;KSO=JB3J2h_v*G92J|*TeSS`ra|6SI##G6TQ_I(`SgC?#I{fcbzl7JMMJfA2a%25W7S^DFPTN z5zdZ02?Agzs;dAhG1RcWhymmZ>cKRm9oamLlY`sy*}y=gZX{8noN#0Fz>8KSOcj~I zyDl)by?3Dx0Sg}75yXR+c!x)0bbd(#&>Yn98H!sFKdQkQ6&p&GP78}?jwUU0w~!Ne3-$&I{8WVfj~?Fc*X zNNs~qpVR$XwLzgcPUBC75OkKuEan`>dKO-*HFEX?W4pe4+LgVVEf_0jySY2`#rK>& zv=G@D=1??|(6UMc<+0U#SGz$6yahT5_UvLzugI$wyb1o_yI+s^E*%}2Z>Oyb_yq?? zyeY;6^f3U0jBQI}g&?V?R?dNIW*+HI$V^4RAl2-%&vC%TrJ?gG<$5=i!~7&Ou92qO zIJ?tHEVl7{22c4*Q(*F0FdP&f)i#k!#Vog3LP&bQhA@{%CL(C%FdIC@o5I z8i~YhB8N?kr`h!F(1b6=SRSj@lO}j%zy|)vpJ+fYn<|VQE>Es!W^-J}LUrJ=)!`rS z6nd1iwA&L&Gzp{by(8J2wdIaRrMwvD4a`@v5Sf8yUEp8fy*;X?UhQ5Jnk{t>*rUw8 z0|j#>F3Ly9G`o88?pu<@(@st~oK_K5%IzAS-P)7Ul&kRwj*HwL)03~Z?)T{UbwQaX z#l?U?TEp6UWx0&~)vTI5&=ZEwwohIjH?`YrOyatAq|(EvO=Q5R%}{6G^G=7v6qq80 z6V*FbzyFyt>RW-7{oDoi?cmdR{Nq@dXYB#NR1flyA5iVAG04V4Y*F?DduO>ECV=-2 zUJcj9=?)iX{eb9Etu?EzmBX2i<6h61NDmeYhaN&b^U2#2_9@ykyd*kiVoLqmGR*#r zXMa9}Q9HGK=*Mbf4}D_vi4=LQ3$%qrB+fJ5^Q6hUamQ(&ohr-QbJSV54bSs6O}+=L z)Pf?Gy>x}}?wu@#&L(%_Gc@AUAJtNoMOhI^weQ`n{KWO*4)Fj6&hj4DI)U( zkN)mXEB9Il#w>|B(5|JpqvOVDmTe5NoD$jTsi}MU!%H*RZ@@W*qrdD0EcQWGumt8$ ze)a-M4mTYQDQHKx+GJrR%}VBOOYWd6C-{y*@Xu2y@yo_&p?ss`t+Vrhsm?LLU*$ON z8!>B9<~zF-vXoolK$}yRgRW!;+hr+FZwufd1mnQU+$@jf8+5`98Czm` z@`smlO@|9OuDK#NL*!~N+@gsOj_Vr>%EIs505@iIbezn7oQdD@5z`eW8KtvvbEUOB zi4k&!qGIX&yzE#t5RgZ{N`;$%aLV|RaBTxU;ZE*w7ed7+ADDA0Rjh-|RR- za4hr&dFX$VVv)@eu{p%zx=eP#usOz} z46n`fY>`$ciKfg2d(jF5XmXR#`D6W94r|zMOXCWyhN&@dy;Y-Q=UPkpU%H2eG0U+C zQ%{z0PqX5#392KC`?nfY>v*P};y0HgVC9Ug==f(9U8U;;Z4uIoe_n zg}4pxZABMYmP4uDnXE3zhk*4$*dpOU@!)ao?|l=0qxUu16%|YY7G|G^mFn81)!R4) zR?E^D$qo+-^Z+rtIVS)Q`aD$ndILS9W%`C`{XS)nXx}f0F+WsPk4PN*2^Dun*%IaI zr^rP<+Jzo074C0Ixbm&S6N(qK#HM<incF=l$G9E>bD&ky% zX1~V}EG2ju%LmrK>-i^6oSyzqm~BB&!bC)ycNoOwU5KjGhB~E3H7WXEU}S6k=!<*2 z-Cgq0h`*hqwFr$^`3kN_@HTTt8(r3`j_E&b&cpD)iovYs(BK-BU|3t?b|<}Lh6Exq zJ72(h-63TwoJ?%5W_;AD6N4VzvvbIWn+j5_gE4>i4iQ-aV75HWw&3QI`S>W}^(6?) z67*h?_FvdKbbvUtAsT6c(4b#`E?uS);l(o{=yW+!cWKp@DQlrAOt{+A*4pN`=_4a0wNU;mIpGK}KDQ{|J` zRRJ7s)*!Ut{Y_{>{yIfm!I&cD#85xt`b8V5gnZXIDoi3ZNxVT1rC*}&g~;MwfcRh- ztwU8L;wZz(yv@u{vY0dOu4nXj{D4}6GhlJ85vtoq^G=CFtr4UUs+m_al_a}{;%L>~ zuq(}0k>R9i%LX65I-0}WJh<~lrKlf0YXK7^mdgWq#okP28b0`vvUM-C!e3=d#8wbI zG2phg%_gTm^Z)cdcxN?(-1u=XBR{(`>+bm9@Qjc;6>Iy+U4t>$rBgOa`sLB3ewRv~3vj-wzpEmMG&z>aeTM z5qT7S{D>+(=RnP!)g>h|fvq__nrU%G^x}c3dj55`?ofEBMIx#t%S!d?CB94sCltP=N zj%t~KC7JfpTP71-_=z9OM&lEE$3fvS@j%Zw-p6GOdv`g z_16w^iNbAHFL(EIVJR2#mNOo;g-M6&4jqwNX~@lL%Y0CC$a%IE6@X`vK9WUrg*ml< z{cF4RPg21BeQyE(Rpj_T)1{?F9H;U>EW$VPyGvXYARu4n|H!^F$f97Rex$HN0bqSp z%#*)!$4K3ooQcK+e&h#)s7sik^j9<@5dYI`*> zujMM;q-JfstlI=GB_v|)YrAB%Ri(P(+uBjxQE^diy`o-a^A|woeKI{|l7LO{3ElIm z_v~}xJMG?0zvFfBKo6uLmg`i23E)pWKAHb0W$B4K?t$4G227l$&VR(R^f(a*u4KRH zf3(My_BigUQTRpf{Cw$}+;P}t0|&(IWzK)h1sKCjb2EZ|Q#`bwbzX^Z2B?R51lur; zJ11n{>#7JuMB=H8CXJr;8qaqWDT2F1_{$|*#z#h!i4e7*va-mMFX*0@0F>5vg!9wo zXiAvUlc-rHMAa37dnZ%OVq|L2NRyeesl!fwvh0MbdTfES(>lxv_30iBY6;fIVMsD- zXIKSWWvxmOg|SWU*37sN z$+3lL;;`gy;`%)$$$jb-?8fzbD;%_OS+FCkF_R1JT*U%W&=w&SbZ$6~9s_m|DhA)y zmiE1c)fJgaD=_}e;RgY~Qf?Te@E_d$JBwOwWJDWSJj)+w&5-Z?-2e|_oF6vnU@o%f znLRrrgF=5m<0XD~^LjY^K(km(b-R@s;2P#~eYj3Z`IPm>$#lE4va=?ewB_98KCD1| z?f3mNm9LODgJ~7}z8_FB@YO58g9ZIi@{!X^3vE^?61t?ZN-F(PMfbO~`5{noKw)CN zg#sx1n1K&4Qd-VyI{{Nmgtx)_pV6Pd-%~;BV=Z>FeO8&U9VGD{Gz5R(k&cU+Gb;Yz zCXzt7iG$|*(_^e!kx6)Ky4no9O#AaA#oWYzdqFK=geevl6s?vROs?Q3ECsUmIWTm` z)K+;pDYZrFF9_DMi+QbriPH8t_ z7R2hbxzr1D6loj)pp1J5KgRRJTSmK?y2`TYS61bW24ep*k?Ki?GjwdeItxfp*Bywd ze)(u24NP!9)<(7^>VD3R=2__Yq{`!e zPi^>uW43?!*TNQhM^ah5LIRmMzxu&HcPMBtD1?E zt`-)Te*#jE-#BkOi6IM33@)psA6^`>=;e5ac;mC%?_@$-znqwd_1KMNgsfn1WVoUh zI-v&9yfE`89g5-ia+u^nG5&oj zqX2NIx&!RXU#R3NI3g9%OHCd%edLO9Zwb)d4+Jf@>8FI}KNRwDRoCU&%Q&J)U|vlUn$3C4(_t^}O5H`mcIW#%*l@MbpI6^j{Svj>#`jk>4Kqk@ul()jV zmoJKc!hoxC39wV6DV_ZYZc9@CDFjNz7Y5*0z6)Zh+4rU@2rF}eWsE(!1+RWpjo6!m z&Fhx=J)$Z#OOy=pvvPh9cD+Z(=H58@?@$GLfr*qm*iQ9~a;s$C`}TK)dAcQ+SWCqV zv7W*m+*ba6SHWui;x_zPYJQ0Y3ySKQTxVYL94B*=&|#Z&@$>wa4nZt;uu6NVrXiqe zt77(Dr9Jjq{S4Sgycjb4^Ag=heW=Cs4GABUO}`UpT)pg(fwy1%)DiI)Zh>$3FFa~9 zNrAL<5UW`6Uwhrr;Re6OV}wHk8kC9=XA9?j;%a|8&0O3XdPVa&Ckfv>E1*sU>$CeuGg4{*N+Q#~&OQp`vx|N$4FJFg{Spua&-Fg^3n_# zh(TmE*I9>CMv?wtpI&I>Er<643Rn`NE!0@egY76Bc()C198229rse=*yya`)b!j&? z##N)cQ{=W&i?#epu7hMDs<(cFO5XhB;5Pzir3@k^(}BtJ3}`(x>|{??+a9!H*_T>{ zS}*~x6PZ}_!l*m0{)D&m%R$b22sRc?7|4!hhB2<&c- zG+{4C54O+lmU93?eNah>INQ%-IM2OW$ENt#3vO0%t%Qg`>k^dlY$yaaILDCilO6Fh ztpP&AI_87{B>0ZZc&&26mvg|fgTdtcLZEd~BAq3&y;^3rAABbt)#umWbv zr^Pnu#_)Hz5>KfsbB3)0*HkNM<`BjQxN)AD!b`mI4xT2GG}b6;F5co-bA@=LjtDaF zw=@@C=DAoVusS5jT~R1_j!}q6V~UPnVU2R;BWPt?|8tNe#r9E*a*R3bpwwr&E+;Vo z2)WE_KW8!Q7e>pt`~|0{NeQNF+csP81f%;Jn24 zn(`Okb*?C_+D0pIMYDk#VjtAskxG`b2lnY~)%YXG*x&@Wu1!x(*V$ylXeajpN;X5Z zTJ|f0Ta!|dd?PY^6Vtv4=A|5&*~d1|JRN_qb%K#YJOkXj!q|M$ji0FT*7c3?b_{9g z8yWzaytnAGs@j^CWUU#rj@;DFbQpV*K3Qjn&K^D>@{EYKX(Q#n89n42eLTP9Eq(Br zlf7-SoMN%Z@#meg?lG#BZcktUd{w*j5#K;uu}t8AS4cElm0Rr4A+TK2bQf=F`hccI z?ues;hBY3IKCVhrGCfmctze_TLYpe$jIX@;F|it?ogxdxtV^% zwDg`KpARVM+h?;Jn&a=8`aG05Y1#ageNHt{OA_Q6Gi3WR@*G)IdO}Y?GTTzz9_hDS z9li^;o-sQLwwByp#)#teQkH;oz{nm~sH!fB#wQuH;@7;v!lrTbwY00AoSEmP^d+J7 zTCzv89oof67`wEzD!&-`SGw&JJ+eg0>SYsMUlf^QrZ4qo^*urk&C8KAC zh1X~!R>9fM^lb|fZRb6JYTCFtaL-719{DTV)bPCOPO z&fE@9>;RhPMA>PNl5LGtC!_T!XatIFLt$hKBgGR+wc@Jj=&8M7l`BZI$$ajz?BMnU zQCTT5k!AyikK{Q`u^vHm^DBY%kviqr_3O=APNApI?IUGx+h7#1V@U5e6MqNcbxTeN z4>*MKjO_Zb-IOxPar9n`AG{FF6_@oTFopEgTR2f9kuxF9XgXKUo5oToSS#XtptdvJ zaW4xF;p3lKo!NsGL(m)Qx}1JSIB0K7l9ZDUnP6WVTLDR{pz<_e!K)1)gH{P2$#mhP zg|(L!*EA8-G%5hl%=Hte6ZZ@j($_D!Eb_Dabj)4Y#rx<~l$zRm6`-y(orT+VT#r_X z6*xD0U95U(Tjv@NDsuu?vB94hf(Mic=PSt+r&AU)P3Ak%6i{=_Vj8ec34>0V(`#Aa zlr73A!B1wSQOk43CiJu5^p8^V=9nH$i1AG=UcA1`xxX{G)djeBaC#9{tXiA2zf!LtduBxP%or9H-$wn6xtKtnMUq-$esVoqa4$Y- z#ZFMQ;MH-VEm?qY3lyUvvJSVFU6t%kgx(k`*7x$$wvxyLNFTQdtxg*wl(wLJB#UL; zCWw~U?`DR`SVC!3FCZfE8%e4m>z(?VtnhdML#e+8$DAYdhbS4G%|5T-x0O0{(8Rn zVp9vj-OKxQSddMV-O`ZYP1%r1{we(TEOE$}*L$EmOo&f-3~?XLAd880PE*oM@EHFp zPXr)FXG*Np=ggJBU@nFlosY=CXq`_mMH7y7*W6AC&lm$=2Vea3dACRrBGwc>)Qk%@ zILS~bt`(j93BiOdi9}{SUS~8uuKy50H@{QsU*p7D5ZSmw*+{ldE!R7h%>|R|q56yz z@mQvxSpuWWt54W0!MZcsVxbgZJIqGclm?i#e(4r8?^5LI03(rJH5S&=ly}#>@A<=5 zx)Iyz@RlEN8Ry8PDq*>~?+Xns?BLCc8x=@@n=NerO;gNQC6@mlC;wa+M__=?H<$y! zQ=c6mAM%l5{w(SzjvVKh9vGVM3@;l(pxg8_xt%q6sV)%J{Ti$5cc3h;EZV(+GX$Vj zzdU-$_3?)#V;%83y(U2y^SUR=7w3OaR1do(aj!z5EFkc<9GMQ6jdTqRo*86$sqX#z z**%Y`fEoTugvM(AE7bHoPF6^ZI*`OYAfFDG%YdCCk_=-B`8DN|--V;@Gw1S*P4RK&w1lu7>q(NKD`O@yP4>`yb%Ie@9FId;av#>QkyV3f=hcWa1%@eV)TPGMiEwFFe_bu{C2l{&ZY^jTVuhw`k=JHhtooGNW zt(z}fH@Sh^hHcf|adodR5k?TA-DXeT4*vY9s;YI#p#a#W9l8?Sz0c%ibncO?vty#Y+{MBAU?+CkP$7%Q3( z4VG};!vKpNP_7}vaQXiBl(Qam!-i<)0TM8;SX~Ec=MAjTs0A=h&seWPkHB)pxBFrA zn4h0=_p!j!{r1x!meZ&!P;??}3#}5)8FUvDSR|Gqhj9DmfsKicErJhIE-h7IsPlv;B7^iEO}P$7Z-5#}6JRbmOCGHLeI=H_)R?IyPdyQ=0UZ`$!^ z{+lh&2PRL{{+o9`0RMHzO}F>=$!5cr^Y!3O&;Li(KL$w}E^D~(G^cG=+qT`)*0gQg zcD2oE+qP}n)=b+rPp`GVh`qma{?(7SBB~-X-zT5Uys!IfmPCYAjRt-eLzi@FR|v>T z5VwtDAxpQv(5m4nU1R}0TeMm3q~!Yg?215&_iZ16}U)FULplwzQsDh7>#;GF(|@Q&}K zD3pX5OT33HY*SC4nzxQOtzadHry@w&9s%`+25NhzNL``McgT3ANIL~jtHC6wja@G> z-%6h7kW7^nzBIC~NI7M(u3+B4dl%R{Cs@CjV!uVPwuBW{ZCA|?oRN8F$N;-_4!82| zvTirYr(B8LQ_k-i<~MbEb7*#F?)XICA-0L8FpJvMYP$>Xd_W>r^Vg^vdT6KA_8%&^ zfpmfn+hq5b3?f%&AfJ#|7>D?A&|dp4HP_eBSp;y`{P>F=KtFy^PX6AXF1`}2yoGlh zX~Ru6nr0ybrmZGMlo%`p6+q$-Q&tp~v?$k33eEgH<9Qk8q(X{pWa*IQ3~qtR{VT~} z#Z=ABy85EFz0Sks<;&)5{Nfut%piIMw&6)-Be~8iA(kav1d2&xM`Zo>4Q{n++lfg! zi!<#e&kK)!x4hw~T-?d9TT_Qy3DHtNIVP{V(vX+LCqS%KXKfob}dlQQ@c zqWRJB>fJ)8*9z&ZA~67cOE)4GgdZN78+|yOUw0uAgB2Y zR;4_9O7#9y7_%m?G6q2G6U!7JMxVNjt$1j~wGLa_MW4HOm~67!op`gkXy#l_bhj=> zg4tx)Sj&aWLxVjI0F+2QN&7*yTG$$>{JLBB5DzcY#Ff&b&Qe>+ilT){5Ejgnmof`E zXP;KZCLL9Rm5Iw~?=4MOT7QkUi7_yBCL{T;&BkV|pj{H(#v~ z8BoY!-sWp$P{q^X+E(zkNWkm$3qC+?V{3SQKa#nKog))c z%Rodj-XFl4jSe4OdDMS|yopEHo#ggN9EtnwU;-op7L&LI#?kCIF5Xp$KrW9}l zQ^*73r20)8$c`tyuMdWY{;CZGAUJc7{90RlNYbM0X;#EzP-qLdkweI#)>P3h2BynD znzZHB%8ye+=%iBfYKe!G(@f{4kQ2JcVv0}-dHG}U0By~WB?W5Rr%UpuMlx)XY7FvU(I0?jW$@_?BmxdH4*eN)e5E*_<9lY}Du@>5Iy6VqCW7ma1BG5u5BGF1)+&c& zHl{uc@o4FNX81#2Ut*BMx22t&xM5wzY-C^@Ss^?Io4~GSMVbXO^g3B8GH9-SZdjPR zUkW5ea?L6s$&@HquAkn4v?N_amus>)m9v2WemEd6!XZ+*i-tlJpoOli&kqV(m)fu5 zruT3){}j6txr~M2^TC7pEi7eSX$nikJt_3nj!!l90ZwN%jUrXgNV~CL|Re^wq^A_UPN33ym{){o1 z%Gjl1X&^+3apNgd^5yJiCC1QEN`Vz^wr&mIWy5u&u<_*FHpDSU{_BT#CdL^mW3O3K z(R3A%aOg6oOuU#=#7$lZnCE`hKG0`bE6YEf!I;y&zbMul~eghb3GJPkY% zo)Nct(_J+v_ErwLsq!g}I%_0zMMv5SSz!iRcz$P@6H#x1Ry2=twP<_Ey8 zG*8ACQwf_@&)!5}wVmwW#s>`fD5%_nd+vgT^E+vXO!hFt8Z`gFx+jl~FJ8>GI#z=8 z@$V*|0^=4ixK6lN@mM37(q(-|TLiknxl~tf7|cdHFO)79MHfp-`0V6noUlM{3B47z z;WJ%PrWI%5jIR@5In!*J&Vsb-9T5QDS--+gTH~Fh0&R28;M)=4zfyC4wQ@?)17)<* zX{`AoaX0?vT+6|)Pt9Q8d$f=I&6z`m^1wD>FQHj$79q}NJVy4_J<7p(hXvvXI$GVM zIcZ-)5inpmbNEdE#VqoGhToEP>lI|C2zGgGN#|5L*ieGBDd8#f82{_NoF;1nElL!y z{`{(`2BXE%;OayQsA|$FXK$LTRB5xi@XT~?Qj}eJ9ErC%ewPl^JJd4}7mlCznd%GlH1f$$cj91;xvFIS6>*fY z#3rcz>gZvP#bKsQiRoo*;tvJ_F$}|Num06H@4;D#!q?VRn=BvOUoujHV`68lQaCyx zcAko@T@lsn#|h@{5y127M57kdH(q~aE9+}$TvpcK6ANrPc`rm-bb1T$w)qxKKl!SOqUz2T_4jA-^+7uBBK;&vZs5^%-OfOlTs0M7}Cm5@B({CB|75o*HjG(zN1@HSftWNqja{$E+=twt5SP2*?;E zTjb29s4BLw2-+s{pUU33c6$nRV{OehGvsD*W-u|(w;C=r(nme>O+aA3f?H71J(;~MVJkcR5gMvQA< z+8?fq>|1J-0yp3Hlj@0Guq53bK~PUAHedN-k=J@(2*A{ZYX3Ue>VuXWE2Xn|T+uP( z*>rC}?WI8+**XV_Y*r17H{Ty+4m5~x|5O02lO9yzh~Z)KV%#s~ig@E0KNTpZ?*&kT zhg9zjVIXv?Vy&%v;e-ZpU)4%)qduJ^OA{?xxTf1l$$qOFR`e3c$Xegpmw+x`*BHW| z4Ai_&-?q?cAC)w!j#vVne{kw1CP=cNClSh9BVMmnq0mPYe4*N1DW?3I0{DC9h#8X4 z`m1G@q&nEWp{TKvQxkDugQ$$v#LtSub}8*#jX+BJ@sklQAR2y-vO_Iam_O7Q2m3*r463~hvp1UrCHlz=Q1i=Sg2W#}5fg zI}m0#1zn8gK*I?(@Bt%C0B7Ht_N^9zc{#gOu)Zfu$tA3^ED`C>1Ev`UTqv3>! zoF_P@>WO{$M7ha=b|;@clklKbcg~J^HjqT##d;P@zx+gvqBqDbkPRCE$A$rksGxu0 zOT8`***;_y_*Q02kOWDAm&^243TJe{tXdaK9f(@t^HxSZFz>KRmX{(YT^c1@1eDab zhQzW2l`iQ;3Z6o>F+|9K#bw7-uV1drX3R8u2r|eSx(81D{ktZZ|JW2vI@p} z$pm1)edFcAcXsF`z$M6Sb4F?;aCGRL{pX%Ha3dsM3rpuV)8NTI0@604h}-gy>Q0*6 zx}cv=xhvuJX*OsJYpkJlw56N>9O(EL1y+NT#J&_F`BeYhvF%VXZYN|rxC$NqfNfRR z?tAXyI(5q@ldydFy+XVUX%L*?g`ij^*2T7AE_VGZ%q*^$_WmNTY@@7v-7T(|DL`S4 zC>`Nu%KaWB6i{>D2vW8mbpLj9UOc{JPoNo5+gN&jcHJBkIkx|ZiMB42rUT$4Y()RRpIE~PzbA!v0 zGRg0ACLz}B^$FVV92nz`4t|*r;7PqrX2M_5-@Uykps{z!pyJsZ<1&fz z^m#txv?z%F!Rh1&KmX?sXK#&s+(Ism{Cy!rh0j^$Th_$|%gb0=BD*K!9t)_q-^_o$ z9mk2XCVG8`j#t0xDBheJ)@r!K9&`MG8|R3BsSK*5jD7qLT^0po{aP?<(DRtiGyAz{~#kkW+75vao)&SC6Ckl|YZw4toJQ;OXxg7ny(V;KaLR`+G`d zadHLBwt01%Qt+E%6#v@7Qj~C7ctU-^4ts*8Bv#Ly`|T+Dw$ba*P46O|^GRxt=jy@D0= zrQq?0sN5N37gI#lL)|EGw z%kOe>-?Nn0kSeDM3X^|XHdS4kWXjvX2R2$uZhPw`+07%QugD+Z6e9AAO<=xOd@N&k zz@t<=UQyUkB&kk}DSF1O583(%x&yo`fzv~rH)M>bu>++H!3uh^d(TqoLuzx{-GC4uco9!Bx;-lk2t7HP7f9}J=mURu zIQz|%Gt-V#iqScsJE3~d#u;ZvbmOKIXU6NLAvr2#fW}NA*fE4Z-8$>s7yRGLy|^cA zx`w0o@tayVTtrwfni-gw=1>q+z{%#tiB5+> zCZasml?%3M5}4kByV{cc0PWCUJ-jQ;f|yr_Z}Ms}dL!P2^2XdI!p-klZ!C}(h4S4I zpd*1Tr2N;Ra!2G+9!BlEbzHkA3v@?U(ujeaTz_`t?0moa2m{PL>QZnBp;QKqb@hxQL3!*dVS3KZ_jvM<5_PQJVYT|l( z2LM{GL5tq4DAlo;VC_r_^iseE^&=&`3WI7SE6KJt<_--dI;gnPlgI2I+_*!z* zGzjEf;qa&g?il$9`kJ}G{A_M!CNqa_BzKQ%PvbtVl2BI$`gwY~;d^YfOh}0VFSHHH zgH(T$=-erUAC@@M<&-yqrsty1lhO$$unoj3KueV18Nqy{T=w2^g%V+}6;HONdcPV_ zi-oyku4PIQBJAyWjmlnhRjYKyAP-a{yNTAZNK~+hMjf!NW8GSqDCerkm#H2ou3(Al z7{y%_FPXfUr@enxj!r7MYGE}3!6vaaK*TCs$5?#~Nu25u34m^M`LlXRMl2%WEMkj6+$gR#W!XB zDl3LOsk|7kn;|dO@s%}OA6zXansEL7c zG#!0R@O$?OsGn*`v{3^m2%ScpUsEJDmLkcK&ZlT2)nuBSrq*^1F)TZ8jPillzhXt} zVnOdxoMo{V|3SB%C2Hj#sd~bwSwye-gPwm%G8Fry?)_KrpAUqjG`Xg#=K-FwTH(ds z$G0{!ZZJgkJLltTT=h$LAk+)4+jIq8FimqGjAp5XRT0y=BcR47%(Brny|gNqN;9;e zoEW_A2c~~uxU3v5w^KUSzE)ntzZSr?)+AZP4lG)a5jyaxqmA#%7gkc-oYO3fAsa+M zTR2#gL1-u~h5)JhWMuWFksjP+}yG>&}Ovzk4PGeb{H;XEPb@rrSPLXt;=W^gfh&5es&bjcjE%cwq z8S*Q=y@GQ`pmy{snh)uhRX#aTyFg;Qv{P=Ao>YxFApkC)&{T@wLT7f71Gn%2a54NB zvh5n%ajftmL9YV}=p`%IP6a4|7d+E4Pk&TMae~CN4H)P8Uh32AoZXKN%LEK3pMv{YZ^|aIZB!ca2N-^W9y8dbFevp# zC*g2iEFhgWG47w4SGt11y_lxxQ*dgf<(~c^Tdj*4XrBKE^l97eS{<)PVj=YzZBF7b zNRwD^>d@AroSv@m)()QU7t_g&Apm}~P{z4TTjsdcow%*uFGTaoqG03CEWwzjp_)LR z2%QPBq{lEk|QAAoofuOZ^D z@ol$vH>CVP*}N>{D)J4JtpKlCGEP_U5#F@w;SQY2Wdobw2A!kM{$Zmv0KZ?VJ5sg- zhIikf`#UAPMo%j{{EV;fI+TyF_~wT7e4MqLee|tNfT!$Q z=I}tS2d!UH%3PF^i{3M87g3QIXIGp)>^_#0zxoIW*%WJ~(wnX=ATc%#Nh4ks6PeqFP3GQwt1Ig6F=gT!B21T6`Pw>$ z>L3e57;N!-ev}9^f^nyxDY!?Ej80|vS(S(E*31SP zlPLk=9f3+;Q=bUaLcPN*p|wD@tefr`#vdT~G1rCW;+b)aUo7cGRpUJY^17<4=@v$> z5y^YA=VcM#?4#Pk{f3!OzE5|D)F@YYVqc}}88q+F2vzjH>nvqTNVcGE$sCP2Z?b&o zTwXwT4~^UGz{@al2g6&#YXd5H@fC6LQ?5kXBV=a8<()4(P48LYKa;@!ovHl)k6XUw z!N3}h36THFgYhmYxx2rOBX=nO$6-uTp}A2eJ=-4In@HDPWNl^4tOdk%cQEGvD^YL&q*4I9(I=@4eKnLe1@%UZX< zUg*%mTGi0>d%;M}ZA1U!&1&8h{FLKqc zb6gB?POb3SiK?4=NYDH}NL>UhzOuOYiT*4K`ec)MxVH;<1&j8jR05I?^}zPW z9<0diuK*=(aunZF(7xz~zQShD03XVLFD;TJdC=jUIO1lRhOo0;L+ zTZoltd3EBMM>i{`gj3#di$y1;gwMSuG)L~?kRomT?jzyWgmmEIpxbB-j|r>*@(a-9 zm+d~eM!triW+9a19QBZ6^hQ8ufSP5e)SfGO7Y9DapA5r)%HtAD5QgEX6j7u~S~+Rv zkvJ#!fi`K4*j6li79`k+BkRTi*aCCQR&MuH{<2}kDDewo=@za#W!G!9P%ZGK^w?3P zyTXUi#Y?EI@d*Z%cvcPj>nK>OCgecc_}|N`Dl&nGr-oyg^|*0mh4XaXXqZ895CHM~9cjB_jFN`3?m z@LGWfbagaGiADWuObc`4{-|SAKD|VQ3r!HiU9V*5#7amW7ac1oTMlnXgID>%bLDw6;uG^Lo7a!r?toSW@KSCCfzVDq+nTGTZAP^rPvKGL^jdP zwEOfq#3qTC?0`Q5169)bB@#qZ{5x33+Kt&^SOz<5VKtZCdJ5gJC18LmoD5}P^?bif zYq}rxjd8u~Mu$=2p?z>{(`gx)a+;L-0!$MZOa88rQ21|8vQmX^94=vhP*jRN! z8W5_vI}szyO$BSL*DtIOW#SfV=z2DhV4~$QPPGsjT|_WMze%B{)0+vJ21%cL6$6)D)I|asQ-2S8i^5G2mo!HcVk(B+gq`LFOo&r~9prYk+3=H>q!TWu|C$Xm)Wja zw0X=O!%(TpRyHXvkF%>vlDvF1UCT`C%^jCR11^LLjD1K=OQ`$&MaZ$4n$!OTPEJtN zYM+-cDBV+KWf#gF-y?f!m%!e0f?8*zo;m-Kp+ZrFEBM>QEec(GR=j>^YDAeCUer}% z{!61|0V!*?E%3Vqzd3g1(jD62yxohFmi$`?WAw65nHxI>syMpXiR0f+LLqF2s(y4EE!pLPycsV7kq*$K{2 zKJrg%uwL;XxWdeIn^Jaevc6Wy($&YUmKC>vFPprs9dt-_pOrVj9TuI@V2PJx!Eji)ZR1Uu4%E~S}a^|NTBJz~o8{j7RSKvtNE4gRHz4LwK_`-g} z6p?kgOSYwi7XmSwI^BcEA_dHItZ z(1XlxG{LyVRCNZD;*l2!n}9aW6rV1GmzF1ZC5>;OACZz4be5gAjjUVKrpRLQy>4-N ziz>E0{v+{^{*$#Xse!%Wk0)b@;&nsHb^Iew~`_3|+tNKmGD@;TaqB zRL~N@nxs#44e7He{6{1LXo`O`Q_(&z_**Aw6BC;57iGB6FfhZ>@mp(dqZdlXgs2Sh z8X*(douuUWq=qrA6K=mch{{6HAP8|MumdzzsbZO=@E%+bLS=f6TXRfk2aNkWjU zpJvKCQnxYk573f&WF9j(>WaNKl@@Pvp^~4!lJ-N%ft+d7OnPj0h;XaKVTX`AzAP9C zSG?U~57cfbkIN?w3O~t^@bTG=ajNj?cUa3NV$QNW?%d$B`%jgo86+tx>0cIEJ@UPQ zc(;Z>B1~30UsoFt$5J~Za57@{CnphX!m-W%N}gJYSC8%ve*;4Zx#CeJsGYQ77!X^P zs7fu|1Wi5*f-D}Q`lnQbxaGfB$mtqPkUv=syOrtc7^xQuqZB=*F~KHCP-{^0P3O+ zD)NTPHs_GtQV0rzJ?kGalS}DsCYe&VMC|f@?9^eFR_X9WJwf<1wFa@e8eZ{31m9})C{ z(LNUB1?pi|VUjxp`;BSu$1DO6^n>v_)*zT{a?ug-HPbO|wBHS-Cy!(Yw54w}{^BuFv@eL#PuAC=EdbwU z%^V4bKfoS7KN4%tL;wO!n%i^(n8Zw^AbH10=N)Y8we8^NcN*LNUNQ*0At=mk_RgTw z+k6L2I)Kff|7R%voN|&$*Exhy9b|7Gg`q<@1&n|1mX0x@&1{7&nIFbhNfkZ7f!MV{ z*r&&--}cY6`3TtX2rSY5`_T5c3FcklXXg%NO&uW)eQ(c*xdYL90sTGz)6vhn!3XBG zPH0xz7UbiaE^p^jkf0Hmhq*C(RA%k@KGTMwajq#s^0nz#$(-T~!A`GO(a00SDBE%r z8t>LSw{!9j?y5J3fY0T5pkkVQv)Vfm=~u7q3D#EpmD-iduPMj((=7;)%gLY5@N#}n zXyEF}h}RkTKYTq*HrBC3FNFDKTnFxywVfsN4fl$CkFi%Ur^JkbcJ&WB@X<&gAL!kX;*pCaF z1np9EGDl^~Sbg^@c~)rLq|wV2jh$hd*>1djhL0ABa!Oo$fsbVEf&6ETV1)x_*gkRm zK@gD^&}jEs`aI>pMWuU|^5tW1E^hkSJ@$f@bwD{f(4tMsM&?O}w5tbVUln7{Cnd>N zy6zYzL#itxUahvH;`mD^s#65r8UFK&D^wODU@wz`!v`^lX6k;GAP9?Ln- zZ${(Q-PF?obJ~Gxn^tkPQu|Po_mzS5!i{@}bNrP&P&!_8^@7my()PMta8GwcQ<2xQ z0k8}RSq=-i#DG7UA~G*JCZ6(-Sf)m=>X5=CH|h|R_=_pEvPcWF z$a3!Ysuc@`tS@dA36#cPD8Jq@$su=xI<`3`}(@NocNH`a;kt6ac zkQPl-&2isJ+Tx`sv^Z&VoS#b}s#?5@>=GZZgZ6 zS0dmG-ODX3-vKZ*yM;ua!;(v+L8SWt#%2okS7(sav#T08|HMtHhI~QhcKGO3$qbP`R2^)#SE+9p5> zv)P3Eb0sh_rZxV3Q-nLTG&5y^*tQ_4Ghp2VKIY$+{X^1qpNezhhDDfKLii{6t5Rpo zv}35EG$yN0;`lLyCo+CJ@)@5W3tgz)pu!Rq?=XxriIj!tU*q=qQ=T&Q9!Z*uPS0KQ(Dx;4iJ~{CwcgB3F55eer5IBcn{>Bi1EFUSK-90LJ zrtQODN8BN~(tODCKNC@YTpJguxlXjWj05|k=Kj6&)(j~eP|`7P4LuYzk!lhw+vs;3Ixh)9>BrsM$q$_pt|kf25Ph0snLGW?{$@e6Ia; z6}|ndL76W19MmSbH>W)KGpfHdz3^bQN=6QfBFOed-ean{el004Sbho zpdgRGO^?)ZQ}V!&F;jADIG|FlI=LWI49mF*fJN%xt$`)Kf7x}G|noK9cF<$kmxF@Vp7c}?8-dgtQ8Kj%gu z0wHE`MW7MJ-fL-r>OCu*8 zZ}CyLU~jq6CzW@>Aw-pTiC^|5-Qrgfp>EjC0p+S`OG9uS)f2UkSjipoHG&r_10DG{ zVW35#zt${SOxLZ@Dr+p*_K5ASDqyc>jhIL@Q|z6GL0H?QA3XkAd7&k;4*i6K04``v zrOst)tNBbMFOV;_%`shnap46hhbVN?&Awyoh#$4W2uM3i{h&BaJbr&>x=80vG}n_%O=OfX6!)K^L!Vpc%~-`oS~FQGsPOQxN}>hxtB&QOjsj zxqPJ`;k#PtA}JzrJ2r0IE>161_~T{v+bVRW$Kz0qi1c_3M%#uhTc{|v3d9qpk?$lW z>Wqs26}d|hn--^q%aWuq0w5u2N%|5Lv`MUklg-RFp^oCJHgMRrQ#2&pfp}O4?pxoD zRZF>}*5cR>)93+(KF}5}{XMSMHVq>Gs{9>bh=3Q-J-Nxo`9!upG=#JQtV!c>+xD2pIgr{cq zZuRAW@F+=iir@4hKL!(Aff?A*QbTFTk0xr0dB1lyh8ItfN%~T`;eGrltXdFRR;}Tf zu${vb8n_dfGW7XO`}grI!-jU7FID2wuqo_h#2qD_x{@t$bc9VSsbs++`7(!QDHoBV zZ_C#mq<`5Br(L)`=2ko8&~3zHl;>!-PdqTun8{fW{$_|>&ORrD0II!e(+Z49dV1>m zAHx(&-2QRh+^X}-#-H|Jdbs4;))#ZVcT}rn-&+4BI@@m-t?i*}?G0c@stkCj4Of^{ zJ|B$BIsR!&XF9i9q6qF5)0ydNU}*8t2^iu1qJNW=edGlg6?<XbYDm)w?NO8zl(d9G^_8b>Z2*@orrc(k+4ZwWMcg+C4 zF%JOx5l@J_5o!}JX22F2htZ_Ns}(hM&+KfJ3!vHW;_&_J>tp#^TD$C@nvgG@F|2O^ zQ4A4$&(IYs95Hdhj@2qfC%^T=_0RYCB#+5h3{NB`T6iD|FkWo{(6WvdD-NF(@<}V- zfwWXhfYuG**7{~af%FJB+`9aP!8m}`*PaBTANPbm^jOv2AYvTD{VY%*$rs0|lyGc8 zo&c`_pq|##v0zxpq|bqbn}B_uc#C8q9+U?IK7)awCyB_c>f42bD=#B+(D|=ml z?XV|vPQO4Pz!rVmM(DD43$Pv25!5MMY?lNfEkcVN6S1BF{+Hap9()nS-SpWNF{ zgI-iX$%C!=%DDKGQYk9But7nr9k&ST*8l`380MdrfL6=`l{jtQ#ymWGF}zg~D`phA zh}yJ07)l2DDs|D42zvCbeJVwdHhSNt*@$BxEmW`=GNl*XfzOb*2h^KXjG>g8-ibo? z3|cXMo<#g!GL3K9Uch0wC;dG)v(}gq`r}{3+m^5%Ogf@tA{po+#tW=sf`85tJbZgt z(}`t=vNXmjj`z$=8oQ~Ph)MU%^G&JK@6*QJM=bqAllRPGgw>PIQ-uu<$kn|`&Ijed z#*BUsbn!n)ifD2<>{EB{JZZF#*&{O^I89uQ#uhLv+C4Nv(gflrFf2-?r+iUK)o%!H zsxW3nj2WB3wrJS(PNVQBW@t|0QeyKr$_1H?S!J6+W%p+E+ztrstvy>Z=NQ@Px|avd zQ99e1F;)$jDN(P;#S3ah1v}@W$JOvacZys%GAA0EiB5g^>bs1Sta0D`NK?^cqEf0! z91Fe}2%pK%{se@Wh#xE$*&`M)6lw_t3tykk%LhGfJo?Vdc)iKi}Z9 zyJk(%Ik9$yX{rv$uhyAt&P#AE_GmIW5?fb*$1Z8mzhp+VkG;KW2zh@>lU6`Yg;cC}e|Bmn;4xSL9YHNV*0RB&1u>o9ku7WOH^n?zE ztS#=(2xY1&v{+-67%=Rlg2h0fYT@2(;vOq??~)UO;im)bB~Q$0w?d|xLV^&T(g>i> zN7Iv;H56jeVS7qFc<=n8(5;quE8OX(IjUbi3n71s|I^4De-2#&-6WUIzmR_hu*GUv~WckM>6FjqVL#xvZ;v#d3E-CJGYS>dS|(C*Cf z22tV69{+vCW4pd;;+K(c%29qa(nU9~ImaG60)R(Upb^$_E(YbIuhYDR)BNYWi;;FA z=3zb_$vTP0?DlygsOYPPKx3rioGj9XF@9rgXFVuWC}%E5v-}@Rt|GHpD>`qz8_>`2=5M{p=q!-RvzCWi#z_ zX3WraZ{24LJ;hXZJxZ?iN{|$cVxNkh$MXKh7unl|Y6h7MIl7)!HO%6vmGqoP8kmLd zIe|;IV2it{<0hfKMg|%|o(4ta697@o^W8cP#$q`fP(+`aiklna}aQ={~&8i zpQ-f0|2va9@p>PB`~GGgWXjF}7G_E<6@Wa&R+k#`n_H_ThpK?|DZAz>O{YkMSyT#9 z{z%+N7^aP;H9K3V0v_n!GT=xVInlgkEBV6o2}Zz*oGy`n^+~yxX`)0u6xKO8nZ|NA z)xy$nbW~%v10qAX5^h@vN{SDG)L+^-T*y#GTwuui1HHgkM&d&~LQi|#^*q&7!2i`Kgt zffp$*zm=`KteNj~Agb1qZId0@wpKAP=)~TJCl%v8coPDeiP54X+qLn2Va^n!i6Ftc z6ImEz7uT$!1n?Q%n_9*G#b*BEaSJ)1S&nV`lH}JdgG%sCnREZ=-ExRxL*L@xNE7I2 z^5uKcGEqW*>e9&+N-a@UT#CskR%72V?*p=~hBh8`gWlc!LQ^F1 zpGmkb?XPDsrPD-W2U2t$(l>txe-J?a=ic_e>GA)`zM}yYWDsze|0JT$G-NX5|9lj{ z=Is25p#Pm@Zw;LL1K*h?4v|90%}EPnr8lxRaB_1%Ljhq37w9E0;cc)1^)K~Vbvn%COR<}F zWA5<-RQMNkqX zWw4L89pBH-;>Iq}Yx+e>G5B_#K*5Fs{Dao71I|E4e`tywh zl4WV3Y1g%s3sNoMKP^W2|I-fSf)%cS+!a+Yg`6jgZ4fswi5^2Yo1%~_CelbPudt zn2=FQmf=p$;}$SA=X{lat!zy3iF1J-`S*x;z*6P*5^nl9XJY#J zn1#rcAujGB%1*w3i2rUq{BMZ=-|YBL@PFnW8<&3vKglNSw* zCh3_2d+%I96gs@5@nrJc^)SQE%=GE(spaa&4E0iYpz>1eRm8?Gio@HSgWc-js&UpiJ-I3^Y4qS+8(yv8+_G zo8!~YY{XC~95vGg5OOFL(==o{f5NJ2mr*)vFFY3Fz3}^|C+U$-jlC1OYxoQB&izWq z-rKH}A2@*g7yFq7v1O}g?EznYf=r?1_e z&l4?#WK5SJ|2+{J7$OMYz8#M_p;8zHiE&e=w|NQwf4;_9$~tbSDrj3AX~yJ4jZEwk z>7Zp&89_md*fB#5%g)MA+F}Bi^lz+IUYB5*Zq-*waJC_8WjS)J^ym9 zMeTRS8WVj7$q?I)CfknA@9Jizau_~WK^S8Q`Zs?(;< zUNy6l)!P&n&S*Rr{_JEcIIz>IGHWygfGCR|@`Q>7zBASsa2+b8sRB*EH|p+R1GdUj zf3J5LqnB8mx@i68Lcj+E$p47$hThl%4|CPYXn~)S({a&!; z+~wC_3mF4>U0H8|Knc}8Tvy#<%uu4Pu8^>YhZv5rd0%xMND}MNV5!mg^L+QkmJ?YY z1{GpY60jg4!WdTAJD35lwVP=GQN`B-bxR4==_FHe&KiJjY@*K=U{@#AK7W}SLG|*1 zH|I$8lBzkRtf^@W^X%C9hOc6o28^1tX9|EFg%;5(($MWNT5c_B`(okq@7v=7!uz`A zwU&EF^|f2n@ZcB=T<}llb~l-)1B3Y`*^9;r!uKvTEGrt{sNNbjkk(*l{!Sq91%8#`*vFVf0p;~e`~M<= zyM~9V=$7=1VZ5NcFnIp_B~>ax z(>XzSmc%ATC5_N-^4`AU44iE`+=7d*4o&$rx^_l0C@{7GCx;FugbMMVQ z--}b*7}4wx-)Kj7YUKb!ylCw}_&aK!J0re(P+kxUVhrYzH&EyzR{$eP7BO)5 zW}@fz1z{To3vckYuB9k#2du^%@Xz*#5>_v(Wcwt@j6S|vt&9hl({sFDi^G!5KN1Y? z8L?q3vJ?kY0(%krCPbavLh_a|@lNUw0aFY~l9p340cCX5{o6kglGy>Dz&&l}e`5P|P^2P;&{Uu1ZKlqn7# zcp&rtl$1p(R{tp}`IKO+HdK&rK7^wr$%sZk(Gtd7hb?|NBl&)&9`^rK_v2OxJO4?i=da%=~CgOv;nQ!OozR zjaVbn{94>{`^kY5bQjqiPNV(WVL9}-CnW%-nSRYfh*x|?DbO=4h|1p1*RJe=>?c2B zS^`ZOsxVo0&U#lebBdr7d6W?P&A&zJf_OKjR_wcj&MMGpbFq$>$xX z%mHe$=8f)PQ$(!AlzwEmE!$c~hdMe&81FpBUu4+j;2f_6P8XMVTIs96%rWH0*~NeW z7oJ?1U{wwNu!p$vYxRN>kk3%48XCm+#N(+3hhzOohp*ArKxk2q(-Qj30$F1mt~sZ2 zSVKzT(p!diKD`9A?XS*CSG1rh{@0Xv)ySNT)!XG@bn9qsO-jf6-+!`wXLYkx1dp~1 z^7Q~0sZh9qru#vW!g_4X0XY<{CgFhZZnMG|>M+u)cPz}6VRN~x|LoDV8SQZ}IlUna zSwHY)&gDM_h9Re*5N5sgYv7XA73xO5TSjf$P;)G4u<`XJhLJpLXu)c5;ihhf7RwHN zh{nIaM=PDWh`&w$-A2XVWPZx6+i#(IVx+`3eWeX4_R76$9&`V?oCmF17o%ePVU zbqZiThBEZvSh@?WF6s~1XIVX#QJmX!uUju^(RZ6Km6+9iU=S|EEh(e_()9q6x5;;n z3@rSkx;8>3irX=52Z+bVgwkV@#H=)2Zw;Y~vDQedd+^buJj8 zZ?rC71*4K+=F{*8qnhg*qvFu#Pxor+x~M53br5=e+G>}yMCE$;U!ms3csKj2(Ncc+ zH?W=rjov@V*1sGDiaZO>KtC>FFA$k7VS7e6rB(z@WML2i4Q4Iyw8nt+SWl*twe&}> zkfC9>kX7r<@QP&c+1~ z+E9@-7L;8$r?wJZui*x0X{`$pV{i#wqh$9hp7qtU&DyO^y8XX%J&kl*EYhnpeQ&!y zy-zuJe|f%7P7rhXp^Y;BSbG@^8Fm8h*)a{AhQjKq#thL#oOFhwL-l-zLeto~uLlf| zvK+kbqr!eHsApo=?m1!J-9!J-b4ESbBlZS!vUOMNI(px?dFc*$vUeYaA=Z_T?RDJq zKY#l?zd>?Z9DTyv5c0nPe@icTYvTJUB7gS&`f4({XLR_Y^)Dw7bSXj;)G!1HDtkHl z+^c@c4fS;>N9NIf8Ct!S;L6_L;rO47vV4Vy{c|%7-P8;FrveG6A#ki3bJXK1+WmJk zs@;7z2+0uB4o-4ud=$5?f9NC|$h9Kk8Z8UMzx~`U<|&;Z8`YuGY{xPaI;O93n0UeP zu&CmKToR9cTuD%(QWo_sVsGE=ko7E~T8m%qoh! zY)?=iEWcHhG+kNn+LRg{YOo1KihWgsWDT%*>ZciqDQykY*Ui+?&1}1|v4XDIYY|xA zCcEdjL%4(092?|Y%2arya*oD9c~IlQQ*?wK`+ygoSl=k6@4^Yzm0|#>wN+HKSVx1{ zS#R`Kb=3A&WE|`0wl=dv>TbCNF6nZ#f$NIYbkrw~rpmhyC^3|KvCf@>@D+@vwyrLy zE1NETbk@w+cga!tAvLp#i8E1}xP^b3UMP9hE!dE~Vo!8Rky#h3NXRI0SWpF_FyCrO z9#&kAdf9+5Jm=y{U&8|=kL)9vRhlD_ce|*!Dbp=7CYw2dhZ*f+UxtS+zRB#Ht-o$B zCp`E8(yKIte?pU}i`a2k4=>h%2h;;c&ZPTMyj@uu7;%{O@2sT6;K%`N4j~_}9yk-N zPL|V3vx;t^rdxv(UB@VsqcrxvV6r zbBV1j&vr|=2CKMHk8yodC)2K`%<=g1S6&+>8${IwcGnNo+2BW z33Czc&}hPKtV`dxl&7`{>vR{k=aDrtM-!BUN}!j;=4?d*8ak^G!=)m&d{W$Gi%hX| zn7}-z&GkojV=bUJUdgy)^A84ynDVRNlH#$;q2Ykxn6q3KipwY8Ol1#wF57 z!`DPdySC)o``kb)rm{YQC3evC{qj=2crl7Rww2R2KU6X?CchzsERMfEor9hURV;LL z*eE@@42RksXJE!`ZDbGP7b9dDccu*LggrT6{P^ZvmJG1uJCe=x#2{X-fcmfq(yMU%?}}#^e&5$*D7B!bu{VQ^bXBm<#G@(yuKDYHy>=Z5WZIr*<;W z6+N&upbUjB>Qs)wt4fi!ZFUG1MCxzb?1iS^Y$!}lz0}jR-=bZFBvi)k_GNi)$3Q#j z5~s9hpV(;e3CEs;ymyQ@i4PSize*$ja_$@=j*$XZBFXy3ZLiH_&m30W*xmURmmzenz7!!Zmz+Tg5;unTP5P9A5qwBYl`!`WBdqUw#ZQOtAz^R*u5dNu<%1 zWOf23)H|z_b>uZJ8g*9BBaG`9Y)^_htCM(2zRK2B-73-`^l_N0Ws|Dc%!BQg$;#t= zV}Q9+g>Z)|3@8#vd|Mc1WGL^BajIK68+4EB`fWJbE@hK!^jWY4@}t5S=#zCgs>av{ zGgfQRn*J=7sBf}uR8Dbh%wkM3i0#oBE{OrYwc3{=-pA_yF0}V$fxK@_Reqh~sa&m@ zg)`hNmbwD%o0jQ~5vYi(-Aa;v@1HUc|GskRxmF1~pC_x9Epuzwy+TnIO^0TvH+<4brc`}jm_MJ_n!e-HyK zk`2~s$%5&3mPn=+GlR|#^4Ea%~aBQU3ai~HA zw%eWgu1BrtPJh}70Ts1QEGA-Za0l8ik0I=v^Gwd_<2LGCH;W;G7-?Xh^+pC=;Xl z-ol(Bvx>+?O?inBk1@`?S(F+~O?hJ)~D{e04&HrJ`QGfci~I1&wIk;He9OSzP`GC ziR#~+ifaBK@4x!a#BAVCk}H5YL5xMY<06K*g=F>>xy;aZ?*%M6*2Sz`<&o-*Zd@)Z z{0#1{Uqcg}&EmmW{bhQT|#h zy3WG(#(wS5=ppX&Lz@5`wtrCNh4$O(eSrxHlPiba-yo4C^#h;-?(@u&u?s+ZUGu_P z>3baO9k+G{TO6WZbJ94@Orx3k)g`<;}QJJartvz_=&$RoGczYx1^OitP~ zV-G@JGwyMz+F+%B+Hz1Iv9iBFGzi0Mc5GjH;tKXJ8%>=ELI4^ZKxeA+ae9BzV0s+R zpuIM4418?6hOVsa8L4dX!yLjzn5d@*^i*1l^7dGj`z`rvmZZ*qO&5J>8k4?c_;3uC z9Y|cC*${kj1t{KMQMk_eV>GdC>(w2kB9{{`+)Hm)j@suI4L{;p+irUWsbFv``uIJY%%k@N=Y8<)+Qm=_fVvYq=7B7XB!Y;mwBi8P`w3>HH3+tN8Z_dXuWV<1L6a9XSvw268q)zyN^~$gQ z61QRjnsfeW`NCqTsb3wj(I0pwV&DxEpRCj<_Lhck$A)!+WjOhCxvZo+X*)N+LCyE$ zk^2O@H}~I=qhISsc8i-Y_b3bVc|+vz-%db0Bme}LBbJXF?=`s1U9cysK6oa6jAkmQn8@#5D8q1yslT@_`c$z#VUfQwKhLe6b zevUO}1_92u-lLIB;6~ibcUD-BH2@GgIi?H2ROX?>ux=jYx1_Lh8Xx%jJC)GbtFD>8 zft`6b(++o;S2d$A8b{#HczSoL(>Wgf5NR?4hFvZ!C1yo}Z~FTS>gUim^JuymkVeol5GZtgw&!h+>ea`Ox?UQ@0+VX)o?I!+}}1b$X=R3e;<_ z8^md^xlg}^(`{ch#=Cqfugkh-uh-`4hkx7Re&}Ptusb0J2wALXR1&*LTrLA0Nj7c* zCvBr;?c$sg9rjCZg_=OL3I_;U2yf8bDP19;73%L6-garj4gH|v9!{C%4fKWdy@Ou{ z)?Hu#5g)7jEsZ6LJPp+VdXpDY4X~}aX)=~gA3B8aiGj^7V%1ay<5MV$a{ode=)6H4 zlT@nJ0~s%ueGB66$}qvTdrZ2SVIM--l`-v?$|kFmv+6ygPMsNUP=J2VT>cxJ#fF3y zzWzIv@*+G=T)Bj1B4F=}80N?_#3m-~X-(}{HKUg3S`yp3DjZbR#&hFaMdmdF?U~YG zLWzBKPQ^b^pl(^!zO<#$?a~7yfBsZ!Q->w))@o+@YFrkr$?8_PS4+8DN!D$|CFM%q z8lqQYyO!S`dE(|LDS*>oT{3<4tIe9T#&wBJ2BGX%vGA;`0k5QC6Th|l0U(6_Q73QS zt*{CH8mH*m8Jg*TrD6fVaG5z5FLqU02?++^0kD5|Tvih3;Ny^4hOMU-_gR)Ggc35L zyFC!VMb~RPG;4EI5C8_XP2WplR%k`a8j2RX zL{kbA4Ap*ZK|0r5ioX{0E*EG7#^FesNA{_;ykrJVsz_Q7{B>&bgoIF1nv_{9r^!qT zj_%r~6T_Kspmrx92%fHEf?ciFu?I<2)RX;u8s1+Q0-q8bO zT=|@sEP2~n_rX{Dk7b;Imz&(Ex(n|Zc%;AA$^pi+*$0Svwx)e|>$&hMr`t*^6$hRK zEVMRQf_S)G8~+e6Qx5rXNlg~RcfIq#)~&0XY`D72_89F>OY5K5DUf7pb3uD&-{lQO zPD?ed$G}bAw7?smEm8Th;IfUBny446&XB5yDG?iFuai`l?WWd>A?D=&)*vcHWub0la=nLnK}lhC z3p2OM(;~@^1!sZ^bV8tgI;9wypQREMS;lBxj0fPFm^ZieT+Wz=*v4SH1=Q2{Sh|>9I zoUjoe5D;COa~}?7+CNZkl>dr|((rvrpa9+4yQZheMuQ>s;Nvn-(9jZWLMBl0Ay6S8 z(qs+t6!%0$Ou?+87_Xgbwz|4q?JGE063SIID54Py<@T`}y1F+OuQQ!no8na(Gk{q? z$K7shf{495-;O(mpGNi<*V`Pv=Y!xe;CvMwB2!VEWg}8f+YarqnO_c>?Hm0w82|#C z%#O7|d0I}~4(p>(MW?!)ECdH0ZwJ&@4%ep8a7*uf3cJ*XDNyeAl(M#mKO5Vh#W8t_ z&Rvi&&f!78-?zu5x&s8?HUw1t2H>A)+ap;)Fu6th9osi&7DA)(kdw@GNBNMc``!jI zM$5s>cqV_kI@%;hdkD1R&K{-(umN+g{nlQi0`92B*NoQ|R3y z1GM-fWaQw2q(|y}J}su~vHWk^-C@}8*)e&F!SHY3h-ybI1Yacn(xX>=^HGMW53~eU zLyW9}hUV#Y-uJ({g-3Du>my5{UnnG;Svd_Yqa3A4$@cE|YufWTcTRb%zW~O^d&v(0 zS16`I6W^&Zko@@v2&qb116xb?2zaLhIWW5;P`5ApZ4U&9yU}|G@O$d_0*Jf6Nm&R~ z?tu{rMwJl>2AIKrKkf@-o&$=##tsHd-;g*FX}z+k{lx|((S79x?x^4VAp1+mG6|{P zxQ_S>_E8OdQ2^!ps$YIO0{}wmH`7~zqJ2fQuZo~Knz!sALDe7si*Jzu!(aEBZ+xh) zw%7pDBS+-Mt-*uuS$$;s`6awtgg7JxQGv7lPU-3F4WK4Vr?t+p+IGue zZFgNKJU||bwI&)~{wDT0=i#LKO+|O7bMyvZUKf|%mEPUbes6tuemyQ8HSfyK=4R8t zP_s_}!Jg#Ks>0SmL1&VEZf9K~TH3a$COjfE?8cU(Mt^Hvp{TXtXm54adh=Zwf z!J0j`GS$IqU%*IpJn7=Tk|#Xa@=(|D%b}11gh}bwm=gbJF*PeK{#0Xtn`B4?M??U2LRP_m^F%S7nLA9@}qQgl+OOGq&g9wsI^A-7VM<9ND(Tc#XvrhqGYl#6mDN;%ayd; z?JZlx&v89-MgOc+XZrfsTbA8tX?KDC9UxFq){poq?(-?n`>@Ag$PNid5i7vg^BW}A zGqZpj)2p9G|mr?uhevTM$$Ls zL_+m5bkiDhyeCbzo|6_*tq2zSeniFeooSWbEX0y>wOhCO9m3lZF7b#D(?>OHfYs&w`!CrKwvg%jm==^zs) zqKj*C7PEuIK|BoUHXnjwRec`dtaoAt*r9i1=IA~f2C~DGm(%mywNLGUp5hd})IIF9vXMT1QrTsA7Ldp#Z0f7lWZXqJ<|f_O5sJ9(=&BQ_Quki?@YpP(EmT-wDGwFlELYIZqQs7FQ>C z7>y=fssze%E~{o+@y60ia#Tgr8y50`#aDcYn?A#BO(UKUGj>26^7uPR`OQk@_;vqt z_$s7~XgUvP(W^vSVBiCwRI^ArrY{v{L_05|awWL$zDe+^8m6f7y*M(Jy7X=C{54YY z^a}%HZ53n8^BfgabRk`-P-3*~8EuY~+@chtfTlO{U-E+?%{T79qC?#`4)TQG5wA2~ zn7hgkeQ$9wca?7hTZ@Cs{8=KwA`Dp~3qo}im-6}|bZ7rIv_yRY!RTmvV~A>8&~EeL zG`Y$Bqd^*X8N>1-3CxisF|+MTB4K;zX+vZ-q7Bx6Y> z)|JV`=9TyG?I}S3Q>Jb)re-oDh|zWnb86WX-^D-ehUAdmpFPf``ApR^8;ORg7Ue1k zJV;UY`AIfZ>Y0(DjmcFq-YUH`=>2=tr5lco7&?_|<0j3bd5<8F#1C0ONNZ9SfE8m4cLc&ZOrnJfxj4HIL(s7Wo}>8$6m_!-~4Y?^^n zpET0>rKF%WukwScRzc#&4<>Zaol#SrQza{+ZPaT3I-^u4i$z-LE{}a}f%Rs;kzHQz zf$26j#}(#6j*gkBBdaz+LyvlNNFJHC;Y}j2p@qOP3f+(ax(2@Eq>Z}&y?10p_mQp`SrJ9Tf@y- zA+?et zN@FCRQi%ty{NqhpXlg?^pX%T2hbL5n;Jvh1H%9!Oq{+P=wjaS!;+$rt(R3c$_z3QZ z3$seqK~UIG9-7OMii6u23fSo+I63N)0xi0L2nri?y~afMq9t(3HVy}C2OIliNl)A+ zJ-f5Xq#iD^)ZsesLu)#j!AoXXA^$_H!q5}KtFoRJG>4I=m^8c+=`Xw$`A2=z62)vQ z_7duEi*_dx?wjb*W?&iYV6k@reFFz)-x$(iFYm}UV?^O};k~@$`N*_@?qgEj4qFF6 zrUGuzUp%-AJO5@VNht&CvwN#fba-=Qs15+Zb!?O!`+_!-V;rwD)8`a&8h-P-X z)xdauP+e|0#&4}hQj?$s2>UZ7L(}oMjIos=QDpkj93xYOtQ*oZFR9O-R6W}R`zrO@ zQD69EA^2@E&ipLm-`|TnzyZCV2OJN8zoifSM!tRKzKA$`Td{Zeqn_(;yVkyUL0u!S zTcd9TA0*pTeH}xdR$uy4Uiil@v7&y2eIxT-|BvTi~gk^JR(<$y&0?S7$d@P)MWwTqLY43Q6SHZBdFpXMkhYN#6QfS zUZF&nglc~&e)aI6GO(MN18dMBh)i=WL+x)UFNuC$kV~jy!a(babp#=diPVOfc~7kM z_E*wEc<^09Trbd&w0~1UZPN*cO6Q)NM0lllIUolv_{(DFp_U=@9j~aD(uJ?8m)v4^ z+LzoT&T|WJgq&d3ZbSina@nT<&sd>RD6=>rj0%$H{ZFE-<}t{+Ma~S!Pk4p)*BIw| z!}xuqp1i2wjxnF%NE_? zT*g7tYnz_^DD+w7J1sKZX4oz=tryvs@}qt101gYLu*u@T?UgbDE)B5(nOll#KId6U zaxH>0x#xFD+}TKCua{eLg|7d<0aw`Z%6A}(I{d9ex#}FR!4mPEmwZAPh;2?bP|E_? zL{9g{a+&(N9XiJLe`ji+{F*d&N@eCXjrYWQgmnny9?Q?a!b0=F57h20RHkCU0)(RM z*@@*M zz+ma2vQ3pVPUzdwLCnET>(7~}#`3$Xu7Q=}2m?n}DvM}#@)*J(^B=M3E>OBZde~C# zUhx8>F0H#S73XqR6RJ)XcwuD80qZW}JP3Kdm2{f;!RFxwE?6v+@k(j21JrDE$4f9H z){V}Y*}j?8d98fU&8j2TkHux`*kuzR*6PLGOP!N({TJcJ;JERsSR0;3$SsOV-d|9q zs%{hHsx%~|nUbeX$=d*KZCA>Td+lR)p9wmKpgpDOPk!-dK_$L#R`InyKtJJu_{S$p zH%wiB-@ba|_!<+w!5H^FB3(FVL)2)?J)>O+;ON&IVr#$dI`rMX_C3RGz$+8jzaiLt zzitG9;j9BPg5rmr2rnpdJaI?@QA&Troj172L1c*m$@Vf-^1fi}^8j?QIs15pGY%AW z#6X#JuV=;PU%4GPQFV>nfU%s2Uty@qb510e*a~9&Ly_f{T*%LGl!dtmV#}=A5x+W7 zo$uz@&WIRrxrk}%&_}qik($V){Y3M!aJ_ z@N@m;zyBo|05TPU;2k;47InUy$NzO;jIJe>YAU_dj@4!t8oTi=32~p*9 zTcC;;G>9$IiA`imp?2t2K5Dlf#Y&HMhqehsOiL-10vmDqb7X^riIOAB7xw+JaJuls);8l9EPLPDeXrhlA+Eet16}VX zY4Iq-6$P~8iF~0eFzSJ}wV<9Fz+`@ey2PORajpdjXTp?zZ3iIh3Qsr;XmBaW73OLl ze|IBR{UB_&eKd|wRk%cxsS=Ge!Oy-B`vcqm@Ut*beu3_?0oMJKhDVY8_`yTAm92iE zaWpgd1I<=@obo6NS=72Z3ykcX+N?2669lJb5j|lwdnn&>gW!g9>$P44E2?$5P1+N5!Ka<^?6n^I&|6yQ`vn+kScP#29^2FrxGA8c^ zcSLo=$an`F0PuNC;SbhczgLtRdS4i7dWhf#-+`wS4*0RZ&ukM9{=NqK?t|+5b;5gv zpV;LL~bmtQny5kyO)qHzPMYEqTL3#OjGBxsI9!7f^Yr{zMs zvx(Gs$E(%GZJ}6p;$~CT#;<^!God?1BgRN_@_;tD;S?lEyv}uN+r&EDhT;v7m~|cj zwWWX4h2Q=Zwk`ITonru2`KgKh`@mC7@*An8sK;}em^8>uuVpHPoYO@fb5w^W`y{Y^ zv%DkfYo&f%2q{m9k5)qjR6>T@JukBOn>!w8BT_nt6(fIdd4Q(mv)<@!2m=L6hrKSUoPt5T!(iA2r*Ouy=MW| zYBD^0rzqV4gM2|Ied(aJ*3T@beTWV=iixe;YHZ4w?}XB)_*Bj6JL%MLRWUo%sYElG zN~06!ahrv%C55y(B%P4<3*4#;>I%^M7MRSF<6sWzK^?T zF;(`3;uN!uJB(Y`j`bS=_s7Y^Uc-R2eZ2@l=EEqmo!A2bgz z4a^e1XN<6I?>eSEL9`v)%sXF()spA^N2S+L*Ba3dJF<&b!$_Ij@jO zaeX;qk>RwDLNVthJ`jFRQ3&sF?H7T@2+)vRxl(6(YI}&fW^kC2Cc59!;$dh# z@$@vvJF!%fJ&2)drb_AQ;ZM(|3A8Hyi0#oWvLGICS5X26Q_pS~eZ9|-Hoq{{mq?L{ zTgm@vXpJ*X=%g3e7G$>ooMthIKQy&YpoV~-lQ*89 z`jAAFYZsoH8dE17jY4M>Zj5QF+<8>n9P#VIo50#CKrqEQN+j+>AQg}p(l}~TJ@p2c z!LfgK$Z}wN7J=ZTh$Cc-*Gb_A!56)qg3$tXFMO0X?RyQC3nWJXytH2sMPsF&X~gq% z?Z~p?9c@qyVE`3xMq$Qf(+sNXqtpB&FV48&WCmF9Y?!DvU_}+X<2UgE+XU=B7yfQ0 zNHbGnC#zqYc`Zf{-QGzrt=QZ_ zm&(}zl2q>;61ygT&_!<*z6pg7*r4P=6BOrw?DDg|=o)sX8)4`icfeR)nXIg@T!JIS zoL?YUJH-8qU8T$2_b-&7(k{{HSbk~rEvF>W4brxk;Km+a=M7^JRCTG4Glk!?)i=ow z&V~kywI4-j6iVgtS^ou?uIxN*K3=6U^bgCUFHH-uRI1mJ2dp8*wS<%q=#&rJ1WTL; zxYvhT%!VnPhsn<)W)VYd?Yo=@Rau?j-~b{u>cWAx5w@tC{Ssx4nj)_Wys+2`OE7bT zUH3hag3|W+I$g)8l5d15uS=rb2}`P;sKE>Z*qx9$hRD}{ruJ~^g)z5-)P>PLNW8)A z5Y_?)ncH#l$k+uHJHE`@Qzvg24|Sbt-iZtm9Qg1x$ zEx!ND^ZZW+=YJD%ngG3Jz>&LB^VR*Q7S38UoTY3Ot)KfAt%ysnoge3>yUxNL~n zIqI@c9BFryee)KeP(Hm@r}h?V@9q+RukpF1yanmjkeqC-hJNgxmjJW|DU2{8^~!y1 zIQ4`FeaMc*;;Z$@M}IJBXg~%q`8<1CdjDsv)^;s3mk9@0#hB28Sct*;WQ9dFGHWL8 zDVxcgNJ$*STl4#FMWqGP{q)l zshp-ns=mf{4L{UmhDUDdzhsi)Nbq_*&At}B<<}6cFVLEu05ccIbqDBWtVl)C7aufI z9vxA+(Qx1LvOaG;u)Fwuxi$DlbLvwyWVv(eLC$$Bac84M2TES@YuuIuVEy*hdrVR!5Cq) zVv7B&$ps*ZjYd8n=Qw&;%r_%^O0OKVx{(|f*6JQ$Yi+n|qEh;6qHh`?{wWVI<^Ta= z3BrWZx$${SIcEV(jzhfCzt9PdVAJ{uP%!`w<_>Oh)~;^U6+*3-I>?QPvGI>w>s31YJb5N&dRFGi@)_;CApAw; zQ!=F&J6MUj9G3O2=dR~(uOs2-=iEn3@xWo`EAMP85)W! zF}4Ov$;p@tfF=1w4$f6XCj&e3OHb?Mid)NFv(iXQX8L&3szSdMW^7)rC6!}BTv^J7 z&|-2}hlevZ{KSjS9`6+P*dRpmR z@|JpmRYvZEH|yzY@L`G5AbQfyYJ_<<3~l4Q=NXn~qiz+kgpEEoiaq4v0RaCe&Hpo* z|2xg&Wcy=uSU^BxylD$@G%NreXy4?O7C}oE!`_KwSm-IC!Ra3}BsmaaHLAb>FtK1_ zV31)XWQq)Fteoi|gfzfjztW?DhQXfRs>ZztR-+!JzTB$Dr$%>cZS{4tm*ItfYwe_^ z<%U*fu46BUm(b86vHN@ebLW}vpX+VU+4l4lq3;tlkXD4C(@+YR_85Sg;jJDQPpiT- zB~EomDaHWvmc9fgE9`mfz7!zVgt;%`8ejBQz zKDE!CA$x4^ov$X>7FOn-PH()W$$Nm?fe#b&;oQW*Kf+eF-D>|-g(1~>VO%^ujsDCe z6nAbYMbGxo1pRmI>I1+Rv({OZU&H0#WEQr)J^qZ%)t-Kh!NnA|_Q--3(_XLoED)Zr zJ|XL7dwY0h#$j9Jy1Xvf7SnZMVnpNi>Tr`>&+h27L~m=h1z#^DLxwY&Cgk>@7m611 zM28X6Pw)D80c(xnX!Arb+I3-0Jm@n~4|ji)p|+%5inR^y6&~R71jFFbH0{&g+J;s) zjOcnp>hMKnF!IE3T9_1FlWwJ8wN+W5YH35M zF>er~Yr->jaB^FnnPJyO%C1U8*3sWZkrmU0{Ol^KkUzNPdX}>)@P42_P>b1WT*}zM zmPLIxUgo5PNSyFuYvf6ddNCtoMCr{2j|x4iO5QH{djR0?(9rgw*miyIybmxU0_sLL zzQ7uui%wOlkb7kN9Jhly8a(YRrzR9JukR=Y+PyUxWaVVMqZ^>WpE>Rv$C~gW)Abx# z)P@ls$gC+O^LbCV1)f=7j6A=k3stGq6_OPnO0cTrMvlBKSiM!FOR`1Y+n&}Q2*Rs2 zC?i)zwg9klTLl-yep(0FT5|QUq`=&ndvf&68p&&-KA_ID?iwr-76hHvM88m~`e;g! zR9Izp7ghh8=D6s^5WZtQFyWbD7n6+6ce z9L;KY7s}EIe=s0$V6wS9MIhw6d$x(8LdD%W1koo_7*|T9-xgvoLPs4fh#6Xx#kCQG z1uzL3U&J#jrfUW1(IUq!aq;FXTkANnC0Zt|dbdqN*_L8xk&5Xk z83C?hg%lMn`6meRz%OVAAB11pZ)_ql*c8Pyb-eW^UYbWf7cuT=3nW&XwzufvN!O5W zQ#Ae?vmKf|S8`t4OY0*aBvoz ztY&-PC8UFD$#`UulxK$0+jDzMsQ7^K;l#-1xVGr08X(lh@;FZ-I=F9GmFcEV>A6`a z66DyjD@G_w4++%^A`s6PeHEL4XCGogvD(vHyhrV)sZ7B-7O+v)JMf-I0>-HNyt|0rl*BRmQ*fF4^3{LlwxmU3N19;xA-VE{n>ELTf57Uv@K%`K)?)67Se@ z4Tk+4?TiUU`*C7T$T97T@52BBxod^c3AWuqP65Y-mt&(!%1H@I^`|iFr;I+}wWBkn zO4nbl2%|?=Nh0_qq;s7~AX^IyL^Cc)9{kZ|3w zcc(%mm^2(GQLk_x`zjw(G}E0e4g06JoxO4!59A zX~2ZRS8%>aV;67nfshYV5Zk{EnfDFab;s&(-q=(3i=0w`^G-Hb^tvEw+3%55o}%Rr z3>bEY!oBwy@cSJ0NtXfrLtexEk`K5L0uYYMTDePEAUOkpz@62F39W%vG}9Cf>!sIl zIdFVYNAO-?5ywdA_f&Yy`%HKX59xtp56+-r`{lslUf0gG_+=n{6D^bn(4Y!Rft2^q z_<3yi)O^stn?|GDcmjgO%Pg*mJ~dow!nFCVHNe@q2?dJ_*4dvp|l^~_Y1KhAH0#wg&6BU zzUj~`wGR%}$}L`#pa&08F2+1)JXsd{`3_%^kl0_HWM9KQx{NNe&bH76AMTn}3d(72 zCi3yPwX7o!^4UD#Bg;CH_1P39$%_5`Ys|APmYfkc_Unz_$SX>!Dl?OD-QWM}@-(HU z*y@|Eks}>Y*Eb)Fsm8#O9yxN=#A*Ha!;5R^S8ZHlPoZzH=5gYmYkcj6Ce}???<#it zTsMo?W=v+N$>kZL;Hv(^o>`XpHZCujPsg4z=}cNt(4w@cU%jjCeh zvi9L=ZyAzcP%|53126N+G{X((Ac!tl!S!*wA$lHi^(c=_m>Q1<0Z#S;>zmM0^d&Y{ z+rp>{y_E6akBXenHkuu?qk#AKa-okwq#118efe7=14|e|eD)?5@9@<}t=nojEd+5& z!Y?}jvvHP%0#H}xo2}UFI+(+NUOIT`kU={N5L4v*qcg($ex8z zJAp&~;}MV+I1g9FYiImFETbD(B@Yn19Mq_|z^CFnhjIa6oik{vTRw;nur;LQx3GSG zu#&NUu8TJKw1s*>&g&;ZAXYEmL9QEVsyjNsBYt?)i$YnR;FvMr9P|o7C^Runx)&S; zJ~4CDKB^i=KNrm$MagU+3Ku5Dph-FiVNfVrBUI`jgjEQAJ>D-PaI$k=D0#3!8WksY z-$actxs-iXC(?j&W;QCDJPIbfLJ!oHsC|n{MUXRVlf&f$4Ez^j`RdwSX|~#MU2-|V z3|Nu`>+4Bi1;f^3#43`UKtF!*8`^3D56$aT9L)x^d&1HYnXiS;!vhyA@G&3EE)x9{ zKMa@?yD|DBSGbXmBtoM`B4K`Lh@ND;vn{`HYv5NAeo%d6{uYRAez4G{48c6Oy%V%u z5rFq1D+E72kjYzxbKx>jmx#oOZ%hrKJJ|`o-`GLlr#e^10`Q?%z`E*%Mz>2(lSQu zB$nVDJ*&^gt{(8i9vJ*8BY6h5L0|9`K4(tvt-bY57eT-Cr-DaMKt>3PzF`6c&#>@^ zNE%y|r&fETS)9-@hu=fGNW<6WZ#t9*@GD8*UQimaS}D$!{MAyVsTTuIN=vi~OqL4P zdFvKRU!VgXqb?^y%V87KjzMdQ>y1eo@*WS{h7W2tj*~eY*ACUwX4JGP-lF?F#HEjB zpE~S1(RkyTs8-#3i1&oD2oD1oDUhPOrm@zDE7Xm>17*+cvs>{IuPf=5pu-Exr*MF= z)wLE?uCE^3;QNIe8Az5_rgwi5>4~`A2Gd*4KX~5-kNDq#n!!4rxp@4eiH$45>60E+ zX;I;Wx{XhRnoesL4NocgRUqS4FR^~-NBn}E+0-=KnLm*d*d&8=^VQ-?@88RfuhAm_ICy`Q(^ zh%E2{)LOXz(E$A!=AFId=~Cn;Cnhjz^aTCz{$Q@#wEjGDbyR$n%qa0G zrF+sN6~eAYSEP&Q#CT~`J3aaAGUl18DbLxey(QA{JncBA%bEH3hm#X}d=I#I4SGbR z8OqdZL+FU|P~uN+D3T(f+=mJjDxj^8d|)mX#DP-+u zMkIYra6`qon{K|Q-*ELHP<1|PWffqUW|$)i$q(}dy9J`{;*Y3XPz~iQvE8+QzHso;R+9Y#DW}jSrFx^4doDbl_Xk0W zid*IeLiII+wyq;Um@=Fh7A^}O#R1`)1Kqq4l4czwn^9QxU(Ms+4+k9hkPiO8u7o1A z*%FEQg)JSDc%ZvQ*{dTL6r?Uwq?|4^o|^rP*?GuqwX1n+Y}a9=ruqCRxjOwMGG`Qs z{5%!@Zpuzug1OS4$(8*T&0lY_c zwHM8?4k?);#nQ5%uVwzmJiuZ46e2hD+3 z>z|9{lVrxkq8-w!*(p-v>z9wc=J{czqv^%qT+qQO&{|`gw!$V1tqW(e44qg9`+J9t zX|1^STMO(-3Q7!0@d`!+B&k_oLWAJw2rHYy4;>EOHlYmj`T-F;2^D1ILXZ;CXu{n3 z#6(ad>St@!jPY_KL=8%zZuDTuW!QyfGk?ase%SB%9K1I$ulCmWWSyiQc10_OPHEx{ z*PF&W0CUU`gIuUR|Fxw0@zIv^7Rp`rn>e&&kIKxE)4banlJ&n@(#*6F#qp?kw!b~m z1g|Lu^_PSzj|t~$F&Yo}=n3yJgvS_UuL~rvtVX}ziAB8vi*)ELlxmY>-|iU@>}uoa z)%M5Go*f}3V#WOqcMYb!Bl6nf^Pcb01^}&aV2fVIm29TE^yzzgQ$wIIv0gCs+O+GF z2yID%NAtQ=*oGMS-US2rWo>OvnfjD+r5V-YXHKdMXtzfAdFAD|pT2lJt-0#>7{pdV zpBSJEReis@LVvvz<$GSHDZGa(+GH2$S#k{^Im0`JL51H5qt6ycIF}LWluLXn$LrEd z0F~J%5 zf6jh_`Ih-Ho@DkSJ=^q&;1kUb4T3~TzNkgqc}lK+BNH&ttaA^$A zFU30ETQi3Ir1L`m&20_#n;UoKpKN!LKA?oM)hi^R!*1sOo<>J+DLK56NhVz~?DLi_d-Q|Vd2XTr5aDrqS-)DhI1pJDjPrUK)J?_WRgY!9lz zc4h53tR9YROhpSavYi+Mw+$=f&82*>tvvTr+0ucQ`C^%%4nLpRWzfbcK=6!J&rpvE*U3CEm+QNwSSqcUg&`Bvy)8PK__kf!FlTHq@ttQ(<}seSYULY@0!EoR-Vp?;fg5n7+%3z=_H683>@ht$sp*Py@(9 zUtwhi9%;C@N`fAs6%^*jdq+p#N!}D3w+Jp&h%V5Y$j3ix)2M)_SGu>FagE#bQ~87$ zH}jxJ;XsyTRp+n^W5|IFQi|qto}Q?%rlI^L(9ooN(<=?Xl0TBxItbrdgaaML8I`%T z88&u88gPnFaYjs;0V13!Y;>~&yI*WJactRN49$y)zi5%`CQL$nDC7LnxGg_=3>{G1 z&whKc1T;)RmK6ToHGH=7ciW*C%2MRY1?cQx(PbJyU)4tLFi-ai9iYL_W@L;bLofUw zyQI{}GsyfGER4+%mda?ljF5Lu{gH=^DO{8Vdl{3j?x=X40eK#cx>Sn)CnW0H2HnvU z6EoaTZ$K!iXmyyqjoxIbnx)%+Z#6R(j5&?uZ#|SAeD^y{L)gP*?mbJ>AmB6(%pA#Y zV;-Dq`Y4}p(TMAj7OYM-&8GYiBPWG+&mxiDB9i?s9ENu=ODBgEfy=t>5cGXWGxDTL z5SJJ*gLJtJr_QiC)Y;CrdR56hc~*Vmr(D)6jisF>w1qG@TJ|k1XRb(1&mHUHrf7@7 zzv17pw2wHi2jUNBxBX(x$d2V1hrq7F<3>-=!`XfTe{zc=6<2}JK{ zJ4T7t4%cC%aHRCYh=a3mr~D-wVmhEwre&O?S;*ELND76Gd*dKGr?8mE2se6D=~y%n zdW3wk$7fU=%V@wW;acf8?}5Q!(J8aPXy<`LDX7i zl-~9eDNEq!`KLm`c`szL4!mwN4ewTQa$FP5E2-?*#MPRrn+y2qh{NGSZJp?~5=&f%r^s1@!vr|T>l zdf-7!t0O1=#Cua%$L~P=XVEPAMtG_8|3WeSfI*?URYTx-x!UN2vJ|};5C9)5Dp)h# zJwV%TY5-R58-wj)bes!{Y0^BCF?31A9%zUD^!!M!^t51ml_%x4lK#5x{YV&gZN>>s zZ)($D;Nh&v3-o5+5c1Nm-;)~CaxLC3iH}f=Y#~OAl4Dz-Ry)0jJ5`asH>TnE#wyiC zc(uaXnZeT7ouNz(Uy&RFEGP<-t8TlG*tB>jKQiJk#uX^+>!I;LZh~*`ZlR6W@}6

Qv~fMwX}W;6e15qcg8omB+Y7x+s4@}=$o`j>DgFQSxTWxL ze^1qJ<3a(lHLUb;)Li(~R#RQY>y0hm6O5%(jA~TtNx8$b)$38TQW916#c9R{T?3dm zaV4>9T-jg4E_nRniaPOkc936eHM0Vg3?)c{lj8U&C}U@T5ljo=HwtK(!`kH2{+hj9 zZ{z09*jT83&Tnut{rBGOG|Bsz#auY`@rFW=G{Xj5`{ioiT|kCiKAm*j$?LwDb>Gta za!G8NUh{Up@x^{5o4seIer(9|=i%QB0JTr@{1C@I?u)T+cSbLE=%0PNfW}+UfV{?A z)WB!m6KA9v;kP;@no~cbQ-30D5Hwpnk0<|BaXxf-nl~CS9*x^bv6+5D4FQ%&bl=)_ zdQCtSn;y1a{f{AOqza8{gL+>+aswj)hEog=iRR|{?H#|YejMhPYh zV=-h;HY;O@=a{INEDTJ1ULx}acOKR_7Du2B7W`S8hta-2OgD@yYCJA4B>c%)aq56M zByR?0E+q1CYQN-n)OhcXGL=C;S)zu|=v#Douxz%B@X_1JI5q;Qsz(W%|6&GE6o8o zew*!jSJ^_E-ro{?YQ_#a{S?6ep^S(i=%Y4pf*C`;*S4YS06y4qI)Wkm7(~Tk5y)vo zG3y0)dT6@K&`DW{hpt*vQLjE7G4P-*@zWYh&+}yA-^>n`t2SD{BH@>xC@l$i*JmD+U); zQO7;n8qi*$2)nA9ZtwnP`>yLl5Dq=+%#*Uo(-t(5th#Ux6aCJWE+hR(9FlTVvmSQQ z{=?sk>X4SfdIs!G01$o6^>uxT~8Yq5f$f~?VgBSYz}W;iOGvtB=K=Z{qifY2My~eQ{GWM zN@HKFw8HT!JUxgdP+aiN>Lo~3m%o0esN+-85PW-rAZm>4uvI~R1Bsts=^_BWm?j^U zTLVL`NUi%-@LS>jsl51Z3M-DpVkEOizn(DFu!vbXWm@1(jG|Iv5=~1&Ea@d8^x5t{ zJ1Q1)lpJ6{RDR@mkbMY<@=2O-NQREl;7AtLi^g@yJaNtVwjLmPdmTGKcwlDI5`(P$&o7=NlD)p+)xy`F{H$((8MV#37r zYPJG!Y9)|ns}ANYnhzXaPtov zQb*cIc&kBx93hIM)Q5`7|1v1c&~0nFmM^+VFa0B~H@zC+t>xY&R&o+07JA}ZgR$68 z2fDkNp|aR7yK4$W?>e`L9i0S-d9Py+`Eiq#=(zyQ6afrE&Q#Q`;7lafpW$H+k&r6xa>#R2Rz zED`QQ&Xb>?W1w(rx*ykBqmbTDdR@|Pw%=;NqNY{4P+ACB>c`CY5>-ZFG`Kn^cTju= zTa=B&l^fl!;=)@ZE7k~gnSb)BZBENN4qqHp1FL=yPu_$q zC##Fh_9p`1M)Ukw?YIJ~>u<`YxZYC#`}x}Iv2SO(j;Ovpl-5K2g}NJzaWslxx+o4rP|KYgJw_6YVw26<#(~FKLZKi)ELTKJ=?jWSL{f18OUE zq}cA(60PWPE7?Xjzhr|sa7M^UMCSdvrQ zOt$3%38*lYe0bbbmlM#!;{DBZFhN2{G*0WgX)#He$8!B<&ttuVt60Bi;*#4Pv&32v zDPJW*B9|5uV75 z?|2!cBlcA7bXVt5A0v3GdL0S!%XbsCSxmUT{mVOW4piP&?KMm<7NE{94l1%L*5Ig; zi<$n{+r}ehu%A-MB8JD=9?B#@Ni~WAXh_fCkgQS_n=5;e5%A9alKW%O`!Arxb&UtL z78On%2C|6B6%hjcM{XZ}23OM>@vqFIZ+$yv!=FSD=n{RU*9z|%mhFZYt!;u{rO^*k zo+uop$2#j=6usC7IVig(F)3)mgTz12TY@9jW;bLJrvu&Tj$Q6NA?E07>Q-G=0Dp%} zS${5RfA(o%%L6*J9{8OH4@=!Vbn;$;Iva0@`jYP^7 zD5u6~rFsSIDt?wuLfO?|_lFU`1j&^vVdk}{gsSX?1}X%)nuzA5$d&71=GTrd(DGDy z5Bx^IR*Q?4F36SMN((`E26RdTg0h74_RZ!9(v=S%lX+JDi$Re?13Y$!yT?6p1-fRF zEx$6+sA4x|V@Fmd?(o(tQVWyDWyYU3NjcY~zc9QbVpjWZJLt(r>PcaNZL?^INr+Pl zLT}icIV~H`su?2d#_!mIPXwm0?Giney@^yNu7oqr|6tt4XEY}b&g@qJ7ani0!kCVn zVi&ey~L z=Yg}`)V;|ohRv|Ctn0R#69)HE@4`36^DHcx#cEk>8u^_qFW1k_7b?dCdE)P7KKb+a z9Nu8>&(~6W{HBOYsmfQ`bCApBQj^L56HKAvJrIyTuNHx_Ci&s6(j_cF43u8|=j(%8 z@`zhNtSmxr%Z4C(C$_8SBj#Fo?az$>F3{3j!ZJBOx_p|nft7O@^w(!e(HEwF&ocZ_ zJ6$<>Yj=%D4mS-Y^gHvY@8>GM59n_=&eY;w9Y>`#$45=|4jO*m2z(#k3QiNRhG4HQ zOXB|MHnIZj$(=8R2MYn1g{n?QFH1(xOVsaYg)hGa`cF0@m$l}my_EM|S7XdWVSjXL z;mykN6=jP8^r?xTsF8euARf>|ms8g98y1usl;8rx@g4$N`d0%9#Ln@2m8c#PHcs*! z$Mr0NTHkg4u5Bn{y5DQ0$hmZf1q1%BM<)Bh^ecc+{*YpcxToCk4Cw;FLunF5Kb^L7 zPynn0uyWc8JAqH07&C-ky0zlb6|hyKV*VDv_>^=P2rFI&!dsp9PId=o<4Od8L)~Vw z8TSOAx5T}p--uHa+c^+@^2YV3FyG82$3tZeZ-L&pS&4NkVtP!6zwr_PJ*NAyDq@?b zU;2ljaezAW&Cu5xC!){q!~E5l&zPQzLarm*Z1ElF$C~6f@_uh~SaIZ}H`-rfT3N$2 zih+4p@emVgCt1~QDPP=vvYqPH0UGt}IVVz`aqr|w3oqyr{49Sgj;)q&N66W8G01;S zVZu_%v*!X%vkn#D$3-~G+Ec}}9vuSSWum1~*j{;?H0m^DZB`02|5zpp?{)#*ouM@=J)DH=JlR)6fDAXtT zaA&MM2Cz9RXZ;AFXSi;_p1wi+SL(%)241%@1(Z zlg$FSeIh<{OpdSk4nFZUM7tJL>Bmkk!?QT0Vr$Cf`CSI=QrVrAC`3<`;wUMUmGug; zVYznhMxNP#mT9b7q_}1P3vvKtox*{M-Z{-}J^l0#@5NCTypj3+7ZM1$`Ovi)qw@^= zQ|Cn4k{elf!IG~!1MQ}Av?G(mG`1~&AOEK>ACf3Lhf7cn+p?yoNYouV) zSi!Eov`1e-p8E8k=zLg(5DS`nc2SGJGfokdb2tGp`Ccf&TOZ4 z{ogUJ&dWxVsgBo>NPIz#4`P-mh)Nyq{bJUa&q@g#e$BvJOUEmA&8_bkn5R%Y42pKl=(RhI(5t+7EgnOf&DKu?;p-D+8B8fN)UF*u*~om? zO;<=ZU}|x%<&*8>yli5kHG=h7tg8Dmr{#kZ?O`ojr$tonjbB9!DW+&j#c|6M6v#U4 zJz>+7b#%+-Em^Zl?0C71DMniz2eYA~R=HELh$SLB9M&bdL$J8qg@zyjS2q znHo0syo_Iw)Q{1BfW!Lc1pOv(06{aIW2W#2t4ai2<%Je6QPwC2>nPztB}t-0f{rx7 z%Oo#ql3sKqH8BW6n=tc_$}BaB33bH5NI?>L0_(a*-4n`x8|eS;lK(&c)snr=A&c?t zKPW7R?M zYgvURt_4Tn7JQXKIXfPQ5(ELs>r1_P#}WhnGc;cL%r>sCoy;R}5QHL4@JkR8?XwIS z--eSTsQ-?}2t5+?t$`Ev8#R_}!8$@+nM!0;vxOD4${KwW9SvBZK5~+!$UKEF7VQAG zFe}w32;Kf?_fe}BmuGy7#N;OP5C%k>YDV5M7kEJI$c^d z{PMEA-izyMSMlVQSMhak#h?9L$)CmvsTf2xdE*)id(NV9YRw6mIu3$3KbgG;Y-1}r zEoGCg4%Yup5D8PD_q6^ByD6_&$3p`Ul9x0`CS=eMBzqVr7Cw8L&&Yl$XtsO#K9NVtzFj`m!6;sm}Bd5 zsoaI-t124G$p&R~52{=hJix!G`-N()4OY>C$tbFlp$5XfJ*slYCFKsDR5dh@(Irm$ zo_n^MxwG*r)V4(qUQ7|~3)&irAB_<=3r`H65TKtBSxtVg^raKs?c1jnbUl`ui;dLv zB@^^DROM;S&E+_6*H=1ltOSX>p$766hbM%-!G3cD^_jk)9%xgySitj(FR3KHfBGj2 z&mO78x7Cr@*c@ed&7F}1!?&EIA}HdxDRN%o*BakrJ3ZqmpYyxn9(d}icjp8V7n^ze z2buT)7epZKa%KN#UwO}NDV`Ha-io+U#4#3Etl)-E zYwpgbMRFDa2Uy!jOkRsyXYtu1l5`x-r6>VeJ#f`h`WqM-m!EUZI`K^D#9!#lNO`Y? z!MfAH5u!=rjPuLWT4oU|;oezb5m$RrmF~!>a&esEP(Z-u5|elc54a^IIB}^+ z=?D&f&=u2bM+iz>LlxaY6%Rv`TaCPPrm)WO7dvx1>I6Bj#hyGod&{&GhJ<+1#hWb( zFpoKK?$B%J9l7E$@gt_|q17AX8d-yh-R6j->4S8*0g%TZybSlca49Pc7ni(0X;{q} z0}dE86M7A|1CnYE3+=`oBfUped-y$ypG|M6ZfOq#R9>gMaM!)~DT23H`b6O?yO+J^ zhH887y(S0Rjc!TdSsIX6a_kLxYm_P;AfAO!Q>#AQHFLZfK+(J3<+(TR`WA&Q#uS|w{Uu6 zMNPk%CGHr%4zNS zbCch2@V6mo@7}&)fabcb2{^&4n&&n(xE+nqX*oaz)$Wu|=x?mODwmP*+Vv-mr*ad% zORZJ?GR2WyOu+>QPiVwF`Z&BmqLZ{$IuvLiSIL%GdgmPqKeAIh1;JT4f&SEzQwu-8Q0cv!_ZEg;ITV z0l3h);=Se@4Y%cR{uY>GjO4^qLT7epBeTyQ#fn*HvCbVmsyjT@Mt0B;=wHwl0m#By zGej_2!>sK${)+DBg-NG}IcypJY@pS3<~FNcdmiGcjDNN1#?MRwWll-JH=}+x)ivQ` zHVSoeCSSyy`rZb623Ns|gm=!&$P+thv;)l+yX@UuxTwm^mJ6Qax*6Fy@UpKEeH;8D zKz}3OLUMsRu}4hs%E@i_BIp0@H_+L_N5fy(toMx0d0DGs+cGxo++~s4)9_lU{&eZs zz=CBi7kPA%IFqrU2h-FI*k$$UpPWqyiG{v-p43N>s^U~_^d{*5ia>G z_7%tN72n+V`7EMxx)oLNE)8cgx<&l{qq;!9)rrbFc(u2D$clapIuupI1CagT@1P$! z>_Aq`)VzN28F|3OcSobkf{szwt%Dd18QfajzgYd^cBJ>sq8Rc^{R8_tYP&W(m3Q~j zRN2a}Y?b&rRTFCCds10ruv-MXwMXmiR}>CfIB8r?+3|20Sl&rCf%|6f- z*%)cY-^md|T&KR(aMz9~oi8PXQXF3R-ry2|cVCx#bCf)o!2~bUVQUm!p7jzCaieekXzMxwl84g9ctlRL>>7VG7L*`| z?G+GexE;#|~LyenRQje8=yk)~6RbS96xGbJ9@i{Uz7%q?!E z-3%puvLD&sVl#;&0q>7NxoIdWA8a!8=Ry>GYm?^oJ&-{0A1*ct7{e||ax42R$V?XX za@ge9Ovt)7tNT5v1{mRskQj;U$q)ozr;0oAMl$R^Wvc2n?YOM)v18*wK(>5`iMF_RjgsZ?6<=zWsawJO`8^H$M%2YCbo zi=41O0(GC@IYPsQ8x83)Tz1Bn5V+CB2;F2&XuDY0NxNec&S)Blv$dJv>JJ#`3(8j> zHLbU|aWpWqh()?O!{`gKKr#m!Zg${cY98N^G&~%ESSEjmMZA+}Zb@E|yhoTay^NbG z+?qO59UWhB1VtbU4(f$IbKUF~)0)oAIseafQ@!o) zG%reMW(!p)-lMy^mU_HFvL5z0J>m?0^x~W!amW)h`#M-jkLz&Boq<+`qigB1H$s0u zUz-eo9Ylx6;0|$^sXk_~9Z8kYA!J@2B~f?v&t6mGIi;ogmEK~jI(eKYjg~K*Z0w0A zx}_8xKlqWJgWV!y^IM@zmJ%~R>vZ-fk($gkm~P)HHWvTdehq=pb_q!;JZQIiv@pmz z#usPf*F=YfJJT%DBSc6h#@O$ZP zCt8B~P>{6`-vvH%#|v-U_H?C7aWJUv+vErf3x%~i3vV*m%m^xNXm3d9p2^NVk#Ha0 zq0RcGS;I1x6J=pQBseW7Z%Bxa_K>lQ2t2%APIwUsKZ!RU)KerP^x9~n^NzC(E{ zp)GqEdV0g|Gl$j9z=$`7ETcEjE&_oY0h%&OLBsw{16|ypjceFZ7B%HIkmjd|LHsvZ zd`%Yd^IS9Yt6IUA&kEM31i>p3_?$xE&k9yn`5!f@)oZYtUIaZWS1>rZO@-=Ay7DS; z>NVO5$o9U&j^h%G^*&QoZ)}*ldP9-ql4=-LRpC9IUn!5Z`KLeC>JJps9{;%ily0T> z71x&q`|a&6144zbrOMssNSMPZI4~wtaOy>s6oYg|-ORg}ave9GBTe)tn@Z6iNbKwp z+RPk8b2Js^xRv?ymI`vq&KdQ*b56iKAwT3`dMTkG!v?{Vy^|C1mzR?C2hUE>b3C;O z{CO8%oi~JCF=X#~wtv#STCJ%*-|#(@qp$i^Q;S|9l%W_7fF(>O669!yXV<%pu4ARf=O|adYKk>9jkN0-vO&a6 zC&KHGtZRNxh$@5I&NH`Rd_-QMc;Y0-U&iSfrpf$>dLvPbdg}XXRX4uO-#|4f zpILRK^88jJAJ<$iPF}4C#HBK68?&x+ZmeW5(yTqAN7>U29co{V)h_Lafk=4hweDar zY>D5BReVurXL)0~=TJL`cU=#=ZVtYvN&M^-pK3Gv6{ZIvk*7ijS_f=wFz^@n7L^ah zH&ZmoKr<6||60V46JGpPfj)`(IAg2F(39`jm#dYnQE@E;G(AR%?9G31mcI}5(cw2; zTr955J#z0Cw#mMTth^LV0PLq33+lfRd*7U1ENSu?`mK8vI4khFkWF=ac0Sxz#0Hs$ zO)pg=JFjb5VLheIpk$Ucr8Au}2TgBfTAn+#AT!3)ug&+QOD|C+{m~@5KVF&K%;z}N z_ebTV+jl8c^IDg^ReQ|Rj6w`sk336%#W)AUFAD19IV(geWGz|D0T)eU#N-DfXo(3b zlh#WEW;P=>^~Pu8vXRaYFYkjIi$5oB&O#>Gw0}L?@SI?>I$x z?=k*(e%xbl#zu_%R%?21w0~=l7P^~@iTCLm0T|TY?_FLX13+`S8_>JKQy)clTH z?jS_u%pK7o*dP&P3-s}_pECStZwj{;OZ@Jkm~$?p+EzYGfHJF3SoCL%2VR%ix>>*H zy;yGHy#Dw@`X_b!ug!w~RPnRyfbTO*w-0OV|Dpq?=fB6=f%PvnJ@DhHjbG-7e2+jY z#~O1RzIE^A{S@!Vrp>=LHCf-Ou!gUyQQx$vV<_rf^Qo}ce+L$6c_5Y}3_g3O7(w@a zJiIAq@OLFfh-*$|F6+_UAYi&}#8P*1$ZH9}aA!gLkkoVF=$~Dm4<$z|AJaquglbuZ zx}hS!2b9q6WsqcTJaa!+Ie0b!{2+D#*-1K;TlnzRHLS%P-_M7R-FksnF00c|p;ule ziqGMs%LU)QC!oo~D7VU2jz35L4#REFGw$Y1Ig7?i{BBb1qqj0gVU$;iVMo!LJoaw# z-sXHQ8hq%>`sP3_qoj7k<+t5lIFcuANy$^C&$L*k^+1K(a>zvkM7!~UBOU5KB{SnT zwmgNJ-^#0me)l?y?Fk$k#!45CxP|+qu4Gz-bZooRJ3!7Yz7rsFXl?xm&e!ew zx{r;g3%mZKD3%wXht2MR8bb8A*}C0#eOB>UG3a=!+9=6Qav6>ZZlsTI;=z+Y+4LE? zccRmyl>F(rl)hFq3qdfp?wbAYuVRBXl9_E!Og72G0WRBEq@wlxN&@?cs$M3Ihtx`l z+8!vy9gt&q$Ut@0i;$Lsuxz?d?u9(O;)GjW@p6-RapPF1qnosT)_XmA=qzbC^G_#2 zGrEX1#^QcOEP#zy;$IiV8Hm!358TVx*9P0RgOl+QTqTNY#W5XEb$(mRB$_+X81o>~ zxZ-f^GB#Ey|4v>d46Fg1SnJ$PZ!{3)Z-ofUN!aIc4Wi!h`jsD~9pB)BKMbY&uh zSZ^rETwg52n4^Cd37W3jCj3lAal4eyRP%$0+}J+-cVq9Y(el&&zrB%>Wyo439Gz(> z3V>cNct85eAgM_D2c>zk@@F%c z)fe$*_i~+)o{RLGb6F6WKJx1d`m|w&!pe4V$s@oS-(U8YM-tNS6MjM-`gFvHbWMYV zaJw+^Gwo#Z89^95TT+RhIqQ$IXbgX77HEim}q4Gq7yNoT1Ckb=beSP`Bp>q`S#OD}vt1?LxGF=eei(D@!b~U?-&L;!Y)9-=$`u zxIpJd38mzr$;&5s?kOZc$D#x5I;aem)X%~6{*w2UlSeN3VgY6R&F>dgf;j1K^|VC zt}a|xex#BYa`C8c3^<08gU8A|!6QDqd{h#{L&2r^oRb9pS%tct(~|)VE9S*a{*iji zt!CEVDoI2N@ME;=QLQG*T5ms4>Fo}E=s7ccr>3`K(qEdcrWmq2d1>v%3S0#j-x#s` zuWY*C_JXW|)(r#I27!M1?3TVYp++mHmly`M$rQ*p=E2 zIcDKDqSk{(Cpv0BPh^3xe{s*9mybhUaJA-~sFw}$ z6VacQMBNx!`n?+7?n|ILiIe5Y4CTPG$^%uQ5@|g@?_%CEm_iRQH!#P& z`8SHTIuvH&sZI_3K1{|lX1}}!27|Ca{r`N|}G{7Dx zBLVzW`S(|kw^D_Urr2y#%TaDn;?VAmvI4 zT}nx6^f1mR%U>@cX{FLNqua9TIpK=5wL4fTpG24WFjy(C@F&q}d?C(OOq{{@Fcw3{315nK>e4TarAkkbL+q)i*1tue*hPHb#b8m<`i z($8zb91mrmroGc~Afi#xzGtp0b^GqN6>Ciiq!tbv)tPArzuR@xOGz%_T{=`5zQh-n z98KGrZI`0Q9bGo6O~xHPHmZxqnaXUJkWD#zY?lm8IV)_J;K!LNZPoZRd7#&>3Q1R1 z!@1FA7bk&AVVcd(b$FH~(s~r#Y}A(vuAPVCdQ+U{HzzVJ1_x}wKXJ3tmk-(iyUKTX zkDlzR5Q);-RX7nPuV?IJ6ZrrU#U7r1jtE%jz&RYojrq-ANF<*4XkW`i;Rc#YU;KgH z#h#daKg9sTib4U+i~sk$K{z`oWbnehlo0KlJQffz0vwAr{zPb3HF;e5b-P<1v#c&uDrNWN?RNDv z)dp;T_B`dXCOtJIbd(?cYp{6xBle@h(-s9o5o(kjn}w;QQaIusj1MU5@^<4XtgV0F z@K}aq$1BCv0|a`sP(7Z}4vsyn8JL8g&pZmo=aR@Afz+iDHLmj;H?}P*Vs5VAy4N64 zI@?FqSyEkj`JisE@bvrXQjhVFQneh?OQ^#OCdMXNJWx zbd4@emIVtYy!f3H>~t&sk+fZfsxshv57$c-*xuZ3yB9BeE>~MU9=QCzY(ndSg$Jbg z?$i8VCJ{hy(Xzf;3`X%OE2dy8_p*GdkJ&p|&NvJi%~tNEB8ICsJ_FA)SRVP)b?R=;(vt*y(PBSaZegcR_eY?rIC8y` zk5Wr@hLl%SrW&Y&&6cd#Qgm87JAm>&uTZ=6x+({_g9QzmnnB1Ls*9~G&@^|x;%T^T zr-R+=`~^$6gE1xz) zMQpZp&y~3PcGnt+V~bVIzf~qCG(u^tUn)tW5RK;|_e9uyP*Y@a^H)$EHv#8hjJM#Z zmyCRIOUSD1i$ywBjLb^f*6I!0i)n`^=#Y^`+l|o>8d+?ug@cptu!Pm)H2k5V)6B3( zNbHN6r|hoL&KlPee8{FtX4qZXZcC?(O)heG4_?_&4QObb^{jZ>jI*MVY5r4rh$o9lw)^ydpko> zu-`f>a@q`R<7f4(7rUXBu{&Vzux7U|HA|HKq10+Q(bx+Po8vD&4lJ6r!{EK1Co9X@ z%sNo5*3DZ{Ef&25bKEkVJ}Fe(Rw@u|imy`zZ|LKB?n$>ENli1w{eJ3pMs>EyU=!7o zhmnY1e^DsCg+I+Doxlf$gWmXVcU>YIg{~iiu!zCtuu$(2H99(X)@Rm0^X&Nj7J~bf z!Xz&&)a%OO3uxDo&uyeNR+9N<-yzT)jmwtiXry)7@|@ywBur2J>tdJPM3T+yG)X_&D|6*yT??;lqYdEIW}LB~$eK5lW{$)72FxJ!?-VSe2=zkD2owj^ekxDAJ2V#u9*Hu5gr zYwH8M2a0N3McxN(N1T$UJR2XW4jOiZnm}1MhFCqeE^%RB*mH;Ox1{FT>c&UFe_<)& zZ+&~eclwtTO)CKTEr_VP*6$k@c!bV2DE9t@JtniYB;khuXE zRqK^AmFqH?gXrzI7IxAb7wxrA_h40x#Z;U9rru z%gqw01B-?6?t5jzv?6Xqm=290xnZpt)n90C?pD()H`o-`!QLVGV+~XE%!bHcuc-BN zX%R{aT$-T9|5#FJZ*H7y(4Vb3&e;*) z;FdfM!Yr!omW1hCm{T^feV!?A*1Y(oM^^SP#3Fyt;M?5zc3Q0PORXHSW>Ur4jJ^hq z$!^+4kuDAIlT~OuAv+EKNY9G9WjFJ7%3^*>lI0v%Y}#1m_~oTWut(#5$QlkfR@0<8 zag+k0WpcY3YWzokG1Uibzk8>rxh1L9+=6dVOHoOfzMr9*7$c8i#Y%=tBM;mdKAqH@ppY8^3|Zj#^)yD;inTMU1;|5)`_cm?F$+UX+(`F}B4p_+=XA~yHuuP3<_DQ?{0_Do zMzrfYNs5?zz%osbv|iC;hI$wuNKKcS5td(dAtejDjDCsLzs)T)E+6R6kL2#$KQ(a; z4o7w-MAD9MR(xwzU0)xgWV|@p+%@8Vk@Xcol{7uOxVtUx?kV zEG+KM;_kk%UcJPGx%1NzY7mCYA2w`O!+Rl`BJPu?&(9Zsu z8rf`+fb9UR;I{e$@5l7fM6jgZCCqYpJ6cEfP85r?)iQjiFLEAYx2;CpG6ba{PuGsV z-1^I;5vfMtri3Bx&4<%ZEoq<1^K3H?Ie7@82_+Qtk#Nf94QNr)Ir@@qGDM>RP+vrq zRIxtwczZ+tyOMd4cC~@@Vf4geMmTpa-OQy$=`$AS_UC=;;6bbk^-QYh;fym`d5R^h zhD-5dJm&+nSROwYd~NP?ZQ79mW0G$J$NSw~;>xb&5R7kz?s54<@I=(AEmm37weKfq zwg==*-ZP!({gHAI^oV=cYPaGcAi_a*JYI19)$`}qTI6J*LuReXboli26K=4#g998h zE-te4Q|HE5>0Zm++R;EK4(=9Nvj>vkk9J=`P|fJoeU#5Z3&+1PUkM zLyBr^+=%<9e>@WKR^CnkH2rRdFeDc|g+|S&LfgC|{R`6pQ+&#$A35nmTrJ{+Q_x(S z_D!k=L?Rsy)B~8@=xyYINN3+D_%< zh(+r4?ug#)%WM6cr_1fBoemb4yvwf&4>cy2-plO)otI{p@@Mbp51Zqgj}7i@-hw@w z$iE)@&pJao0i&C7zaDE>+P}!*&A5g~q7`pCZ^6+`#fC?N<@S`$T%${il~>P)&7sXN znc@$d6Ptbyet?M1FG+ab?sboJXTI^B1}2vZ4*~<5VJok84;RLlkIS#V46&o4?2yVaZTNY=P0WR`PJZdN3GD^1GKV`0BQcI0z1&v9I-V!OB>-e^+x{a zCgkZR96Jf>CH}`e257tAmX+cU0yjJJzS@z3Xcne6Z z{kA+60pRJ|uHWLWUU6RLyj%8X`gyzMoex{uh~wEW-oK&Rd8FnYDmA`?+9Bs5(m730 zHlB;GWm}$#WkTC6liyW|WfuI5PS@Ze47S7X`fOuG@SUP!L>TiOV%p`^e(BvF4YANgaRL+qB zLYy?Sip4kKrrvJg4vh~pZcb>+F{%}J{uqG=7OB;28X{r$--OUnSJYJ8)b`LzeB zWcH_pc*LlEx)mHcS4@P-H)7y-Jw}U&#%*s8^;c)_N@}5#yk$BY+M-!BIA-p`%)w(u zx0EH13cVT8iUbx!Q!y-(6_Yo8CakV|eoB zh6;#?yj7OQy2(Uhxz(%El{o_9F%w2cx5YfHxVaG0-=i#aSzt4eak5c1Sn7w~fDN-W zsI11KiUFn0%`;(N+}2~d{(a(1ZJ9O#8c-7AX+*WWwxOg{``u&SxgzN!>JYAE8j)65 z3m68BnTq{;t~?_>=yFT9)<{kpVYsS)yru}l`GLfmazj|ZkJ!8)o759r zOdP!C83pTY_S$vR_Oy%@Wj#l!su(HBE!3h^NI>JwW&w=e$%6 z_Us-_2QNUX4ieEkbSfj&HHedn*BNZ-EqW4lcIZVE7sYR{&wmOB!Q?v?WUNP8bFq)Z zI9FCyVDaWBvOxTd4CqNz0PM~LE*8K*lnDRsScstTE3fpKz%7OEAFu|geCk8(!*tm$ z+85s+X#n|Pk#K)LpFEj8<&kx@qB+8v)<;vwG*D_BsxB5MzmcF6GX2BwWbw^IY<@zr z4psZ_Jw5G&v$}da=M{V8Y#OWNBQXUTwHphD_=v6z11DgaCrrBd1%UAv=CZ|;GlZpN z@Whhz74==-E3dMVY1Z1!i}fbonxQffA&gka3GSzmq`sP+Udu?Eqg78AyZcOENbo^A z!|iB>!&Gx3CJvSkP&0(+*44cIs@`XYHOwr>MCsyQtd42ipyTMfp+R7<^|$AoZrg|j z5sX)~d^G=+`eo~;H$eEb$my(EkKN>N>1qG#tbs0uj^>E0&Wr?6%bV2X_B@F+b#Gac zDfwl4o(=o*l2yG)G%9QT@wIKINt2usSo4v9$Qzk56w&Je?p{fjgK|~9z2J_xY2>xy z`>JY`4bIg3ykxDaSr#O=5PUil|CZP?1%t1aH1GENyUz#PHd6qn_Ppm*XXtVO zgn*5QoBR=3-1e)i1CE+IQeT^ZKM})p5({lYicvLWBmz*3qyY+50;(Yk8DD@Sg1%>U zG+^`t*0O%?)Xlgu+AycGt*oeJbuzq>P(3u}+;BNlhn$;_q5G=Xs&Ff9dy z4es-hUaA1zalj_30;wsmlPE@$>5sqno++g6$^*!XTKZ5ERl&^_jhumfM&rln{sl1U^w8D zn+JGk_u2+L1@^X#WRC|}( zZI>A1suj>5X@}ETiMWiU_5t=fvt41LZTTzoyX*PxnwRikPfnD5I%N4OBD_z589-mh zgri4f&U<6JEu1~|}Mr62}eB5jWmfo<*e0Qn)hf_f})!M0$x0kVW( zTafX)zEcR37NA`&Nupbw7)$-n{ zw^Y{m<$WQ5{V6FQ9Bl!$Rscp0#;&u?GuAwnoq{b~Fv#<@$t&xwLE(NJ%BI84ip-nf z>`mp#EqS;%*L8JdU)T#l5AX&baM-R`yE6uld`Zw_(C&M*AT@jV>Cq&Oyd4mt$Q z^!+R~P&S}AZ5>TjKQz4+^&y@L8gNSUQ>G^2*DfPBzJV^BPGu%NK;r%tB zRxVg2QxSF770jdQZ08LqF*9{T^b&xz+inN*lb~no@bGPuddKW~xB=XoXDheSC)Tk2 zmio#SZHrBGI__O(M_>{FV2#wB={CD0%0hnS+8fK7LLtko1Mq$UbqJhowMEqa7%QbM zUG6T9-?R%F$lMov2(!xqoT5A_`ZH7q-F}+G9Dj#hePe*8_S5$BX70rA(RhSAd=29C zTY}s>1nv5(!DEn{$|M4~;l6T50sKMf^TvK^Ciwei^d3=x?*|X?@fF?9gm5lz>_(nE z1nKG{x$^XP7(;m?gzTCmxjKjL`nv>m$cyB|GoV2j1BI!?;tKf=$tWZkaB1KU#GJ|*trHM!aV^(o=v z-rB+Ti32xS!UF`I@`v9bId!?4TxAP+4!G-%mEt<+#Uq@5W@cS>Qu|bWf_Gtt@ob?7 z%}@0XS$s6NWUZMT2T|?YO}eXi@QJ?YEN=FgkbOiozWW6`A>>R<_yKQ`m;O+V%;FKE zU-J|eaoITR(2#Y-XbZ%AT)ujHQE+i>j88z1_#}@DCIVJu4v<(Mlb&(Y^-TTb4xCtD zN0Ti0zU9{&FGXOrvk9 z<6>~N3FEgTSz-t5tPP`Ck|jaoc%$^>&J5{1IXUk&X=MkMXVP<6)z= zB)8J6xc~}hu5^O0ErqwftZ@k|k_StsCk{IYmaHlx9cklZBOMuX*K2ws6H6n>lRHDw zeJ`Z;8Q}sQ)X6!Dcg`>dGUfy`hba0&=4>`4(vJ7PwwWGek4fw-FzB zh@J6Aik3$lVgP;7TCx{E#Lo012h@+WajXI&vbSt0u7Y7+(c|ZDKX3616v^il@5Zw7 zZ{#WMC#R>`B63Re4^cX$zTk>pzW*ag0I@UlXrnma#e|kC&gO`+08ssDA$jLUUFx-Rwg900OUOr{U0@1A{jrU#|Yx0k`lNh>mUOdL1PB~G_R z5`J_udsf@LfpUfBk`@6&opI|x6fD3oh1t(NO2iGRJ2b=%u{&Huel|AM&m)~yo}BTw z%2rR39oZ3Ek{7dJao^-$txms3_v)Va>B&v~JnGIxU3M}7qkWG{rQ@{QX!Bs+iz#{g8N^O+0j^rIwj(MEJg(G#o{&Lx{%eGOb$;gAdWY;k z?fdDyTyKoRd6{o9kb+o>nh*2fchRDx+&=*F3RNMcpDtLDSh|!=Z6;( z!x0YqoqB=d?6f?5tLJY(=%(DACvIB;X9t$ek*}D6_#G=N{B5iB>+gRPB);+^h+bpT z)3iLYTOG4n>sEC=x>~0V8BJx6ot@#};MQ}{O2^d5HXJkOZ5g+RGinkuoZfbp^ab&;N!wM!?q>J{G&=S=S$0#TGUQL;xenw)*ga?KrO!oH1ntfRRN< zeVx_QHE(iaPph%Kx(h-GPZ~uwuA3X5W@e{eeOlnJHV)f_F&BDDHK*8dsNZf%#Ci(i z2TqlqC%dAa9XU6VS4~zbr>ATwL?Fep4b_|y*0gj-aRFb+v$(^4{6#RkXiWVCe~~`5NQv*% z)si*p-_As^kHp)+EeV9)j^T`AIWpdDY6)6=$U zDueK+lMgDv(=iyg(!FykM9y*U&c4(=XuPy4M4Gq?FDezc7<^;mP)y#X_%w}J{~%t^ z5d$<^Z7(KLtUmkDpaFaj3In5S2T`TKxUb$PyDuewt zoTYL?&BQzA_mxT_xj&@=ErLeV3;<@J7jrLy$QTTx&e?Y+qb%Zg5$4jplj*0n1EgeknRDmap3`(S z7sVC}nH(asdkUSJLebvxiMIR`va zCb}7rLbO1bxX@YFCg{t!Nb^VJpC5rsC+Lhb{D3d(e3djVQvd0ksDJY*4J}h1xtUpH z_vsfyuXsYmlRGuY7vZrA{wvB(z)pTvXp%FNkU?c!+j#fE%~-e>%|Pt#|-wXfD^++7T;TegtK zAF5wANEg#QtNe?G^-3ciJSgxdJ9zM8lCOj+o=0j+XzMfmqb$aK4ZbZ zoL|cSqJ9Z9;WerUTsj5UEniN@{y_)GZGm5z{{~dBH^a zMinI##W~U%)him2(LYcvqF}wE34Dp$?f$wL-v6beANFf}M(flvM15>uZJ}mhb6tJk zzgTvRzBKMYerenm{bDhEvYb_XV(l}|?IBqQz_!Ci`38i2{rOwKMBRS|1?D;;8`W>- z+UDI+@r5{Uxx^mAyK7G|QPD5H}Qrw^E5b1&X(M)eAx_(t^tpM*yBi~y0Nu^o?fY+9y9q+EH;s%|Dq`o#@87(mqh zgL$8E_WRfHd^^9;wlxIBHLFTXv&44_U9Yu*2VXo{jOy*XBI-1mwq!J`lA6)6PWN)* zJcGls<#(vE6f~=X*1pDl`8#;9tyxa9%Kh_e7>9nDb(&irf?sWJe>A`D*1FG_z?)_# z3y-^oB&#)T4>34KmfKz^3k0x}e}5GQ^9tKcI&>LzvEw0VRGxDaRu|#8&b{lf{e<5a z5`%w;V&4kf3kg2G*lS6xYmC^N2+yBbIob1oH?~CAh82|ez%vpw65MfzaYy+`bEMh< zpk9#T1CWL{f|-b`P`VNwzaD?EB5GH9U1Rzl5xN3pM@U`56=xq!FyIk|6ZL{POX7)n zN3M?ueUj2A)DdatcF5V5%m;GEr|(Q#fnIU`Gmi4C^jes6N91Jd5Iq&99VMs~ll)TP zr$Y(&og$nkJE2#0ofF@%0jhoe^`F-7vA0YIPq+b8P90f}xe<4$7g;0Mx53UZ-I6V_ zjw1NSRL^rT-x9q+cL3iLyCWRockF^GzFWS78VTMzFb|lwq`Uy7gWF+sU=HWPR4ZY* zR6d=Mjk3UZFS&wGVJ=B^rJizwoy!Derd6J#J_du|>9{%3FUWNzTFh{UAK^cW@dbss z!;lIHQPQ3S7jNH~irzl)>lENV6s#P5M5qaS9Th<07Fu#kodAkUEf&^S2#B0Eb@D+D z)7$&Fx_P~rxE;blcpOilh<Iws~OD4#|rcOMF0N1AwYOA2P*}9I3g`J<%dL56WMLLLRep`|Q zyAtxl489`<2f!rbjg)h^;|WUmHnzJ)(b27&=qB^#P4bq>`SciekcS|%-QM-(wO_u z-?cEgz8u|k$6Hq4n`nA^ocKUM{;#9^Tq-@Izf>)F4|qFmt+l_oxUq_(p&FNi`ft-fr}zvp-V6Cb_gkOkq)uFLdxlbe*DmFkdok*R5}EhFV*SY|Y0UV__?)^ityjGZ@i! z8h`&K%=ss@MU=C#y1vZYILIM1#Cf{SDZuNN!ckS&Ra+!W0|q~BTN11B`tL9X7&OMf z48-?j6ziBiaO`-)oFawCggc&x`RSdr8wjO;qU-in<<<$-FVni{``L-sR7cLh70?1mtg z#Y42lCAxR`!ZZ4a@aLY1Sgh2N5K=&b0XTpMV<_KQqT)h(a(f{W6-w%)9nFOM_7uI4 zwGI!nu5WPj8rG(gN&v-yu9#g*$dx1h- z->SwSI(1L!UcwEwy^6(a2x$n*3s7ug8oE6gk+2cu&Ny^M#Fj#;JB*#K=qzetTC%+# zuM`|EOHf!8_L4I!m1O8H2TAfxS;8eQT|dKyuFzR*EIu5lBXNMo$FVe^z;IrjHbXw3 z`y;QNEuH_YLemf6D{OdEX3f6@)DWU=3*;B`oM@~33M*S@QH=doDBk})2}lIItp))gC;bzK>b#6!{8 zLKF#%gp>?B_QHt!t7!DQlsF>DSGQZAkpC2c_@CMg|8Eh92D(?d=2tA}G%$#N$OdVO zI5@Wy(B)wNVOK!V81_a20a>B} z8tox|0}@L^LpRH6kX0bT{v(aH<`Gv4NT`k{4IofaGL|JDe>uwPIDC4;zG zryVv*f&1R8kd0xky-LU;T;YYzPwmk*On6KO%bl{a?er(mh*Qs_NN}-1ZG+Y#D#t{7 zU)L|+a8)bHdsCvRK&P1Y%Q))oO1K&K@7+Eixa)O}m$n#wFQ@cibdDlnmzE2i479q| z0+=Q6-o6;8aO**?O;@9n*M-6siIb*5DdB7x! zbBlt&VKth81&lh+D74*HVG_siT?Q4)Tn5K^^^@CD)G+Bf)O@1qxAK*m<8;n|V1i>1 zQ3D<1J#j^@RExU|dL*rpC^!U23R({+SFeq2j(`)|JW?L9ebaD=>$T+cwhQUb61K&gUmz|cM5ucKuKzA+j{balI(pqWQWit^{c8X z#BEt4y`k%rE_3X`7nrPn?PkW?-X~9vw@h?92l}kO7OhsB0JVc4Lc3mep99 z^7pD6ghyI8>3+XtnkrlO^5Jr!wKC%)s~JIrSN9|E z(4g8d>&jqEpQ*YR0Mb?ynX~5Pp`mtiaRnX}Xz?TxB1@KZw~?`#cHXX4!@dz*3D>kl}%P6$z}8R!by@e9^BWC>lwxjMd~T=_9M_2FhCQ7Z|MA*Sm41# zjR=cCu=cF`$Rd^NN`lpXT0 zK4}!W%12?e9adhKJKPH4M<^*0p&eNB(uwdg*8XMjMG7-k&;X1VWkfWl=uYY#_Fn_X zy}mDr_hcfkbl4>Z%Su_ryuM$1XfP`Ro#7_wu#Mz8Futl%kiz;%DTXJ<&1Up#k|l9j zVbboAHf0{YW$O;ruKo0sesTT9xO_*sc}E#5YpNuq48+^?3BC@6tA|q{ZY6NvEyjgq zcLJAR2NAH}RL4 zCOqNc0e|#;hXTTp&|w13y)iT}KR3?IsoHtTS4`VYNzzc@Y(+Q&;Hpt%28Coqq!JMG zDSm&q4K+1vz;m|<*p^+;9{6Ut(4|?aR+zf0qf;Thx~l90TCMJTcP)Bf`04*LitHPG zxS5`9>t>NyjGSC{^>;Fz_c6!WuiN?JwCm#%s1NA~|FJEKuHXS+iA~b?fgswiCE6Pt z_H+_t66lRmyxCRJ=kr&5{SgEGd`hG56Z);x^I8iwx3cHZ_X%wBYPSj*dB=#TE1cr#A7|Kb{kgTN3AajtxG+i_I297l-wt%w?R$T`h^opqsY_0 zcOxBF8K=VQchM>UNWfoI@|q$FTTy;2KbSJ$pSbwcu66uXTopl7wl>vTv7$|2Q=ou6 z49&WYKS{dr5rH7a3@sa0Uj8Jzyr?E;nit!LS8-s0MNgKZywoA=d#d_gWDOEa@S|B$ z8)pTQB~Hw|gCeC$n=vQCJ)u%8%{eRu&$_@`*gPcZ(6<{vo~&#c0scT@OU5FvBnHE1 zB{uCLr;(^xjNHFPhCauCITKjAKlDfUyUt> zxJ-LJ`P=2?#rr!)aAjRQJf%4?>**TQ8DNprxqRuJDql*T`2r3{Qa-Z_?KLV#NjABJ z7IM9{`lAY9L|H3)*is=fI&OcK+}OmzMqf`((Pnff*3NVdZ!R<`GDlveS!F30R!6kB zi48lmMS`TBfFhd|x4x>RvM6iyd*hk_XX^B>G2dTf(V&w$PTTJ^ct#v9j+3&D5@ZHA zvaD6z+6@RvODsZ5DV4WTBzf%ot4}MXNTn|FB$*|E+UCxQO@vzd#jE7-x5Q(BM2y88 z?r~n~a3e|0LpP-trb)>Zf~$fhc50+qNm8wLxtK_dbIQ={{o(1K-k7L!!A*Pn$>Sea zsd0ruJNB+`p;OZfk_*?9t-Rv-%WCJUKGR*Bk1n#H%Ox3=q9_KZUcRQ7Fpp6K{qz z)QONIa-5V)_}P@uz@8Vamc^CDIQ41(*hM|prF?PsLND>}E{~b@Pu{VwFO$g-NM4OZ zaT>PcneSJNdCz6xZ?p4_0)j0x1&+9>si8G6{3agD3#9-`k!+1C%8UGBx(T1+bC}tU zZLh}fYp>tkzw4WEyHSQUeHXV3^U@rpyz_L&L|DUbCM&+9dL6q@-kbk3ZRQ>XSe)^> z=i>a8^T+))yq_VVAukub@3<(@a9PzOMrZdTuh)+Q-Nf2koyf@X zabux5X{rLgk=zUjUmJ33eiX!Ic-fUV#%XgQ-1Qgge*O$3m%_7hgT_$d^5FSF%{A@+b?BX@VG7hsxE6YXw5Xxjloq&PI8xtCmq2LLgs#BE(AUe_T zCjz(-Mys`-pM(-zM178NXiD3^R*2sTf*5* zaQ7sI>E$`H=cZ{4mX@Lk0FtK3p=rjG`E$b~td4dFQ9OCkSE<6NoCStc?G?ymJ^!(< zl=1fIT~O7XuYll;rGr)#XTDR!8l%qi17$M&nqrbY6aVRmTe5A~#vHY~d_OuZvUCYD zuissWZ1XPXjx)igXrc~4DzhGPU*v>Xx}a{q z2TzfwhrD9nh)ErKqOKKW#kTxGV^SLPG zrHv7sc3L1-NnQv|__;^!G=>h}>k~{&_cW*h~C}!96G{P_gL%1Iz zxOoUdv-HGdN;xC|=EGZm2zp{rc&Q#Rw66?@FB0r$Dab!$q(8&~d|B*kDVbanji&j9boagW#)v920X)B=B<&5%vppU#FZ6&3i&c_ z^ySoH)WCcRdKS6C_}ka$o%HkyPw|wg8QTpfM>R@$T*tfsPcA3bYPqn-Sm6w7#UyP{ z(wR%D$5@2@0$Jh}poxYisY^nhlcP`KG$8PG{3P7{YW*30@=0LcYla4M#E22 zQ4LU{<;I-T?0ep(4+nwQj|8WeGIIGaVx%v$pu7E>fc9G|Lc2(eLAd~nMky+}Ok8H3 zl*0Ny&5`U^a5R!Kjp4~h9Yy%mGWYEhs-IN%hf${Ff-E9G%anfziv|1giHFD5ZNS~lKVjbpjoJ&!Zjm^@&j1Eh;8r|#Vb_QagX4E^n|aY z90ko{h>kpmQGmo3g$PZQlaxC#WPqn)e24N-N7pQ{*1~l(Bzo z7_gsnWTu+_SFX@Hs&pUjjU&KnJ}ao1a+)<}mEogK@T?UqBD=8YMvipZ8qgtCozxN` zPm1xNXnc&tL+(lB3HRFwEf7Z0^`@_K5RB0v$x4 z@qnA(>0p}+`013v|BXtddEqhr8F#21)X;1o#mglF_bHbtzt@j_I)9dRTTx!9eHSSOQP?8pEFr?SED1F1!L($gO76q~ zCH$n!y=v}+0qHD&j4?hstwJSt;VylUS1vCOo2JnSPE#d39Sq@#c?!G{k5_$@g0qqv zeeF?Xk2dTOPU5WPiZjr5%lOE~(xV!RBq}w}`ZYaat7ieX`rH&(R8K==nn7<4Sf)M-So@W!InC z>G}ei7HWuRD@-;XhliZtIZi=cLfT{Oo^lO~iWF!y)*cHuhwHOrfL!n38UvE$x)dTr zB;t9MaF~r@uby`FCkC)5^||P7VWnu%m9?2+9l`G zlb5LK&N2fSIM|j4r=NkAs=AAgkfEVkx@IE+URhmx!G0V-Mei(yx~zj^ii<6>-WMyj zDr1Fy4qXi=QBl}iRS@~OFcLL)!Df#oR`9lfwi5|zv2!S|Oh*>m1PkF2VUUDybjmDx+x_m=#iRdF{Pa zw8_*nt<0J(qDnR1{FI#!oTJ&>0gsN6Ye@T}(@ATpF}G?{zF!+meVkc`Db6kJ=h8z0 z&W|`bGO1b$dF?}XD~{Ok;DR2M9FX61x@Y?vud|IDYOWVh_NDCM{FKN0E9;m2B zhXzo8cE=D$`Jh~?%LGNmP>L5N35L*&CN&0`2o)nEfwe_LEOTOATOmjrr~iWMRJ!hY z9n!h(0T;)>L=ODped5q{NK~voVV^o$G<<}t!@HrkrLmy4u<(9p(ESFN2YHyN8FUSS z>qIVX92k~FPiT6L5*4O}Z*~n6C8@}u`yKzAtj;Eh=Dpo(yJ5*hTS-PE`iMnio1F(?iB zn=Gm83PY7Zita~Ti&sAJFStb)qWR(bWUA-_o0lXS(H6gx9EQy1-*aI+*0^m<+L(ZO zJ1)DqEot@`=-E?f%=t!Mt^Dm@_H$VLgY~F852s{U6Z9u;2phQbdP&3kqqu?&iRUpJ zB}u-tf^a=%Ba_HJIyUJNylCvi#9JPXAnz*Onkp#Y>5xbR|Fhpz2e! zYXTXCC%Z=dQj)SdxZN{M?@ObJtw#ok%)0vWJy9ilR>gVq%cF^Ft7qLX=%fMS0Nw}H@D>g>_Xj+n3YM$EcQRt(Ghx-%T z(C3++Q!uAaen1hE=;3r)<|a3(KXZUpkY}V0WBff_q)2%oUVGp0c4Z3i)aGOvrw^kI z^Se%2v5@u;xA+oDHYHc8+0ABs#JS~iI0Notp>$9P3nJI`6{YQ59RV-<*?=9cy<}!6 zdz9PR>~wDdy1V6(vC}s|{oypTAeOQ&ze@cWp~ad@j_7xQjhX2jKY3BJb&3A| za;T~`;#j?f>+xn@^H1mms9Z2byM=G)0t9*ry^h|)T^~D>O%ETNzM`t&&RM?{Hm(SO zyz{G{Mc9qbIxXyMWGgyj41rk)YIdi(VI3(Bi_`Y6w>8eeQ{49^wk40_xL)>ZMyN#V z+ffP#?@VERhXJO5NQrEA1M$z%68Ik(8B>KP?LzKsqmX5ktw4B% z0oze4<#bLec^n5FJlNp3;SFAJ96;?3IyNM%-Qg_?;u7Kj#S~Px&S)cHQ9x9QgQL3@se38w?PhH$^ z1=O2KyN^SFoo|xY=*n+hElD2YcjSaAyKE^>b~JmZnSNpm7&UJMvIUIz@JjO<)JL7D za~3s`59x;^H@7Ul<|yvK*XU5~szHA2f(%|1?3QNRyi8o$=6O>?!+wlZ2?l(s5)E+p zN|}s&37ua(29G;_v#Be%v6aJ>g}ojsxQbqOqt{CSv1_U#53jP(>#>4oKAOQfJjG3% zRLS$)#BE9NZ2XD+J&&50o2+LILNBY-#|AQGe7XX$tpSW75}8zyhP$7PD%tr(PCs?Z zR5jeVJ_AZhF6&Q7)sdaf^?O`ZT0Y#XRscEbiY6a9QI^=Nj-aQ0HUg3eu)1+jGG4BR z-jEyG#XXM$PvF&ArN48nQj)LHs|~7>XTNLz-bt<U|NEQQ%%F`?2KBF>>YvWq4+`_of*P5_AvSQs5F-et zd%G8!$ma^V+0qEZ1q}C}3uRwO0$pE`;s3!H_@_%VjxQPCe;4kekMN5BOSV~JxI+Fb z{;CfV8vH-Yd0Dhnm#JUAZ$C7UMeobSzVg9?2k?-yFq+sJIXkDQd28T_V|>b^Eu)pl zhihZbFjClw(rSt5psG}IV!(%nL6k2_>T!xLx25l#fk*Mb6TdLkm8VxO>k5DWzkM}MTTtr6;8S0((+oIrH2tEVM9VXu5!zuxq%K7cB?X+i$8EdGt za5&3gqJWltsKJVWIGm@B*{q zr!_A9cnClls%Zmk1B;P)HCg7rxHTBE03{32C!2>*$wU!Ro~CB|*P0xvzG#UN$Nc6; z0+z!Q>5J*cpIe*t(m!QSR9k;_Qui?|>^?zo z@l5W@$7)>T4l_?W&$HNUV-4fmy8enT4wgfTpboJ@Q z)?OnHH$TPh5q6a-XA^TFcy`KucuNY&)^_ zCM9pLCqr-?QH7hTaDQ#3j`mTS1Bf59+9&qiRZy$l>LO!?3a>ed31M_`@|5G4el}(f z=w7Q0Bf>0Y=ZsgyJen?b6QFmL1DgvVT6(JzNhy!@NX>o)SjBW}e-9*w)0_HfM(tze zom#qTXB@?m3xmrZ=WIG)!ROQC=<|Wj=PEvgEW+C6!76`~Q^gjIKll)c3Gn1|mMRWS zJZG$tqIY2JP}atFcymripwzc6MDS&(M@`sDpG7s8n5Np$-V?X}3=>OYq8p5Z4#9rS zMI=~kVG`&=xZi7d)qVDGhy9GGY13SrLXFNL{s8T*wG148;4|q|s_8E(EdilLP=4;{ zS_3&bE+{;t7XM`4ELcFr1uP^;guduL-=Nq1dWI;!xs>|ErR$Qn6Qt=Y+a|!xjfrvQ zmIOtKYn1h}WRARz=+B)gWvEm3LwLYW=f8Uo{V;HJMwZx7)!?p+qX&;s@y1?ko#N!1OeMBO6jnF3V*uIabJyP`a^y6NA{teh)sPNgeIL zb~puN{7PAWyhGazCn&L6e-BOU55FZlKsEaJ?sfj3-Ujk z1PI9X4r#vJ>MLMe5Oh31%UM+%Frw_E8*%;fHJ;-g7>k0XFkKj)%e|w=d=&7X1dII=?jHReZLH(vzlBznI z85PatN}*YcK__{*k!u-*1tCzORY=uaKh9eyu+`-^^nht66*EL9bqo_n4rz_QIuG+; z)(}reL*28@Eg!r9p%cQAM%1P-ycki(`TUis)f<$dYpSB>-_-l7pqkMXeoKBsOlimF zlQ*x8_uvq1946!MJ4yCNc&;|rTyQUUp~#^6hVvdtnQl!87gbiB4izx=!&N?JG>9!L z3C~+7%874`*}+>Cn}Fs0|B&_0(Umn#-|)n?ZQHgpv2EKYb|&V0 zxvu+O&-1?DyH>5UYIXPCr~c@w>aP0jD#mq84!i7SCn+^krB>YV`YbmNOaNMmBT6Gd zI#0zz*%CG;1;Yh%v2(8UwYwa}SiX`4_i|NcGNCUwdjW2DXFYyVvSs<7G)aEgJdut&hi8!UcM4UASkXQ9Ik+TDCg=A%BwCtHl6^wQ`nceWis$!h@ zxzM`IaZ&6tRI2np2Q>yGr~x#&#R@K}_?AMrY`VFciw}FNed>ysb03}4-KDU@i z7O7^vypI=xRAiA1$S?Q#y0bV906!@_U+E08G^MjPIbqYJk9ejrZVM@DO9aL03Px8D z8vOSw7HQb+AzfI7eS1K>C$|EP7Zw@pwiw;u*k6MS_(I>V(IJ@lC!BRcYNzmzzceWD zcb0U1wBecVf(<7JRZS;Lt0my)`1MnRFuyiWTZZcQd$QwHjdUhdo$DLjDM71@1Peyi zhZ8T+DIOFP_T*rx?x}$6?jtmgoC43ri}mW$ktmox7$I!4(%FFi)7+BD$gDC(pK?$1 z;HJus(uN-`DTOwYe_zT~LnjI1&`vn4-^)c(uT1n}I3}}lJfdXGI^Q5Vv~sSZPvLzS zNTQ4@5VsYf9(fVcvdqU*135C638XHB&md(NY7EueN5 zScu2*$${SLd+q>AgU#=wxDy1~Y0AxuOK**{I&Pph+0%_d#^J2rkj#iY!93AaqN4eSk}-FrrISCONX;m%FR&M z4`tdV2m`=J50SXvi3ffCi*4W+%zj7(JR=&1GR1Z^7t9qupp<}9;BSjRViqgoIwgK zomiSMl1&YOI_p%zyKWrc9PFVV-;DYL+L=Vbn1hfa)YeK}#-?USXZ;7&n)nfS*w~~< z-7RnYDEBg3{AiG~-M9Q6IaI4nCnz4zU7KSo{0Jm%)ul1on(ttYW^x-evXldoujNwg zNs!?465oPg$Ni?UVfC!)KB9{NJ&&BQii3u}aA81zvRmHBMRv)F$1|h5*E4Q_UkJXR zR1P2k^7e6uqZ|$-Gt3KXO|SNRw!w-!5uNrDOh$oR?=rm zCYd{7ks7U?F^Oat-ZW!!PA6Ax?%jq;Xv3zqAhhw~DO4wLPZZXb#hm{JO?_ZC^cHL2 zmab6wZMt9gLQNWS9Xcu2SRb`x=m@y#LJjK44CLG(z-%iXDeRZmj4R42(Me8Upe{w0 z{R{bz*ZChWg7E+MIxXMvoBkVW_3mP&`fr@}d4iP_>|g%_nGKQsPbZ>QU_$^t_CH}- zVEqU$E+9>9%Vk3x{gX2uP{ivYlkKrzVx$sm&y$2IrfVfDn?kv}n;HW?n~G;a=V-62 z{YUydRPd`pH=ENS{15>zn4WWgj8p#&EWG8)z%6+h7}w9^r?(j%-u0c&*ITM@9Q?AT zTYdPP@OmQhGS^dcx3wycG!Lyod|pg(PmArgeSkg-$fu=`Al_Ei_PQ%1gsFn`kr4J) z97ZWpw~iHb;{(V~%ZCVfyL@90svEL#|Fr`HLvlQT-^cki;OG(CLL3`%Phk!fUCUB> z%hg}gfkh5@o<^xJrrERiy06^(CLSvF9wn9DA-Ri;M8&ya`n&m}9Yw5W53-f1#LvOA z9kBC9+)66R60nZ;X?$=Qsnn^cAMw56Ese}48FtI8UBkg|JfY*#x!0a(V(wuhyE9ar z%Z9E8&LI3OG>2tUnF?M(&0MU&O;a4jQ8ia4;)$&Dmvm&j*Eg6_pHZk zc=?gU#>4jPoMe+XbegpNI zw4I+UPn*j&ET~1RNsU-rEte@NXmkbXC%<(>R$Fz0(AM;?9JS6hrjNP}%!n3w1p)pJ z?q(}ZjaIvX#vgwfp~PfX2)_CZcBURHB?|Dk1Pn-4)LDG4$Se0RSBS0vN$)!>zmFa- z2w?thmj9JaC%R70Wm9zvzq0GvO&*$3gaVykZJScnX%sC4dAWmH*fo%w)IT@m)Z)kD zgobzTrB}#h0Slow%ztvduANNKj{)#=R`e#fKT}VZRPG;^5p$=H($(a zJ4w>+$0OIrlPrql0%4#LFS|`Q?IMeonagGS0Y*Mmik0yRGR`)Y^H@t7#W6Lo&^~ND z0?ieXJv7l=f?dWwC_8lnA)&aK%k|?2SjiM#3XOwuahps}G0l|mL?mfKn<*fgUzYlO z9v!ec>n14|7hvgN%4QLmP+(*a_sJsn+&Mm zzA@APb6xyGe@gJ6wyiAfk#is=U^? zH-`GFle&-o(DC0ONfW~&`F$e_%V-H%z(S5H!k>+`oXNqddpxeY7W$@kOB{ipE|<}6 zyw97?vAI-Vw71M2-Ck3p4(17;3SZ4)roMv^gAs1C@nBDr>f`v-ap6N+uh>O>u^}Ia zU>M#5IQHmp@$Ue30i=H==$%&ShhHu{AKFI1pN8*Xz`NROMpMoIh=-lxueS1?=y)sE%6UFa#!4>C1fg>mEITCX{)hPzn9 z10ID^mhJ@)z*Cxk7o*awG!Yo`DkDQzURabA3D0Hx^@%a>w2@<$3pu=d%~?cXm-wqa zrOd9l%=p4X5Bx|1KP694w3LyIv0?4Pao?+*HkY5@MPsD{8lQ4OzLItftlm4Dd& ze7bCq|FA}dbbrBNc)#{0HP?Z%JuuD~gcaDUiI3SAi%v~Py(RQCK)7f^O2vZEeMy51_b<{ zHElu3AdayAVf~w-=pg^Y29BV^qWp)AUqPSx@2gj0@XJ6E|M?dfLjWZ4m*-r70+M_X z5&#p2p%8(vyLWil$CAqzbyA0lnl(jPxh>%Aa;BSrTdi|{pyv{R=oyw z0|GAlk0`4xAhI9`|FF8sK}kMeuOA08wSjO0)a6|_)X)VK(-}sUQxZ#Ph7u`5RdhW_ z3y9`cWvoTvN)hQn35caY{e8ZU_q=ZLdrC1X|TUiE>u_ zf2oKxV~i`c*+Dj9Sn`Jlx>*^vk)6Yjxrsi6c_t)d)!{+%vp!)Y*h4m#3y2YRdkPD~lc z$IUXuQ@No$AI|g>e`N{f%!DixfXCMfYi8iVOKSy9|1r7Yl=Cj)npXJy;r=}9MSsJ` zGJFl?DZw`T2G0RznOHVue4A5RikKo~G(nRFd_|rS!_I$9dc`L$r{FUa6QS}bH9&ZMlR;o4qbb5IYUN3~+#^?oDDzE+_5;LLb;TzZ$Di)* z6+s*9iRJTKhs%X=PRFEQCH5r!d|(TG4wWhd=bjR+t8cog-<`~5dCkfR1M-?TDgA;4 zvq6f5wyDVh)08%~x!dAb1PbY%WsIrE&I73GUfJT9brj+x3OoBtyRE~;y z>tvZj4N~0xa!T2L@uP&4NF!+l+IxCGN-PiORxzQh!2fo--G>3b5&mNe|3@YNFE4!i zkNuO7X2XL(_=oj!Ft;@SvP^qGktq-gU?K}NI51&_2fgj9_b&hx@t-Bu9)#U(U#gt} z15^fOCj$mcX?2OHR`>85*YInwCFB0I&}>~YxM=9{+j`N$P+Xq=9>YzMFpeqy(p*v^u?yekus&x zpxcl~mq)*2zvG;(VxhmUhg{zlwHQpGv0%AlV{9@s2IXLYH;wU{(PO`Vp?+;g07X~O zq%8lB-_xY0Q>R|Kqh0Zy*b2+<#D9h?YEGg1KZdnNNa}o(4uT`#N7R$UId-B*30bUa zH;pTnyZbQpCVX2r!q;g0#!(RR@4-l3;=85_+6Ig^Vz|XhsjNzCMAs=!?eB=w&lTJv z`Y%XLPG4H_pt%sm7)!d3IK9ap0p7*az=ALiM^$c{L8q&}K5f_d`WJ0QFPHejfZefI zNH*^n5?wf4@#D-+s^>T_-;jv&nQ+Bq7Xc@Nzv*Z`HuAX#tLXfGQ%@qjhr(KQ@t^ECf2V6WSL=A9iILg$(UF2|1;~BfOD(y# zpvIiYt#0R(jeiASrq5ECAKEz{Szo3N?nTP!!yE~DT(y+Kn|?UKN34|m-tM?+UVuf% z^3dB>`6MIY2*0(tj=ASL9&RB<)bBZ=!K2DED7d5b02{rn4Pupq#~*Hjl)TM}hX9Ld z7Y!qF+Ztt}jrh)6i>o!l0U&S(TX7!jldopZSr-z3q`XOvf5O&)oQ|1D+XUZ1$$p%U zV-RJPyHG5TAviVRJ1iPeQ%^gwJ{bQ#;}m)A#FY-R#`Ytn`$v}(a3o9(=rHwhwrhAS ztUEK*FIosMZcSzOT%mJ)^cC|G?GFFEe(!ZVJ`-`T#Q7QT)S+l!4Dj*&!zngXdXUt$ z{1e3ub?)!@?0dqWuaehn5yEl!xB><~J@-!xKd^G9?juhxymc&{;I*?7euK*>EZBsY z;Z93UOBt25#KiKnz)hR|9Q({6HYjjL=MjE`*S>T`Z)RdH@sC2eq{}I4YTzU+w+=Ha zeQz==#Wq6)q%G}G%>ho^khK2ZSYDEh-q8Ia%359`J4rL`5Wt_Qbwfh7QIZNF%^VT7I1n7ROI` z65hX=Zb1QM0aesIP5#pNWF3jxR>(x|jfRA6opXsqRRkmDU`kIev0Cm^L@B&-&)dX_ z(VRJjl{#bjnpz-q9zmc^bY4)t8c#+mY{_R7-TJH4i#@(fyZMPmSD5icfS_^E=s(ut zf3^JoJhT66EgX08;XuIuW%;q`8$kcV2BKi>|94a{j*a5Kqs(|12_XNPjbqM>#HTOI z(hUt19w#ILB8x&p0Ubm5Vc^9a&CE^7#my~@-R*$Rqxg(~7Ik~YFR)ZV`j4DcG2}VW zqAsmw^gUT+Wf5@iBp|6ccrxV({Yn@i;mXmK?0wTG^a~a)38*q4P-ueNS0GFrI-{A$ z!PDYtEz|qR!V>Vw`wl`J)j?|&IbNuXc*|@vry2%*6*?ceZF4UIR1rn|P>~O}_7oo0 z$9PiPw2TDUxzcqy@S5;X+UshuadPpy>N3}}i?wV{^K~#RmZrt0W-i+Ad0)E~M=pn` z#z>4IiBp=_PP#iF>98}T_D>u+D&BzwBX0x*lzxW6UwW66-_f*#7+@e zruW|TVw=4a!QLy?>t`8CrAaG8+kCM9W3Nz6X+Jp8QP$1E)Mdw5yoVCb&R(D`#&E{2 zC)o`M$8@v79x+8A^C~h?8^q4D!5$3_myxsfqWon&rR1NQ$t9K>tJl`vUC2~z4?9>a zy-B1C9J*#NlGL!h8T z-kDcJ$dQJJ*QsW8Z#_Z$;$Ol}Hpo;-o2|N5Pb$u+VP+A-V1q6L%gU zxxM>+fw8;iMA*{kfz0>>T;*0@?V=!vvNZ_j)Sou-8#Rnf7G4Oug zpe0uFwzvZ%ZHP%JO<&xKt_q*H5Z^@YC7kE*=NCrko=Ds~r3aaol%)~*LEC*Vy%)3J z-h`a32suSWuTW1y+NMbR?q!ebOZo7i|5sW5AMJy;K(zXgaskgM>Hq$#TM4lRSasyz zzEN2HdzeoL2gL{cC*wLT0ON=L>t*N3qN!<(Kok{q%!Luzczm1&89gX5(Fl|sMyg(x zD_g#?@xsIdIQTU_;a`783rni;a68b) zr{f4V$L+4Ctrg!p%-**hs(i@6wFy)VBBz1x&%}m=fc?oqKX39B#)LnIEIk*$Cj$bU z19cPl_Ku%J!M0uYMJ`ktHbA7*_zwcP)GwU-sr?opc%Dn5BM6?|sbWXU)%@0B9Q;OL z$ZoYU@>OrWspJYQ-ys81baz|^y!=Flv2vLqdTaJQbCe=?)Ng$iGQxK>Zz-s}Zp|e> zLt=|Z07MvolTZ~>aKO+L(cK{x;ycX_;3#zG_iW}qmHso_yE$b~aIed5o%(wmn$1ny z`_b{{3m?>HUZwx9Ai!=Cz-`a=nF^|B#;>biK&t)0600W);Jl~%{Ph{sNYmfi{S8FF zh{EnkvTal9>)ZOLjt3;HPqZ%?3>B>U0Bjt9zk8EV^t|3Li9|wXkwhIgrthJ~pTKgW z(l*eO8etzga6|N-RXtL%RqJ}CmwAz)s42NUbjE-s=s@g#Abn4)nnWuaXAC}Y45L`u ziIhq8AZC7(D|{5|W_TLdz#o8bN^lL1j@Hh>f@Xk{qBlTB$v2G7lBOHRe?{&f|9cYP z7Ji{V3{Ry;9xaVnq=iAjU#5m(uCPimp>I-_F$`SW6o`PyI}c4WAxsd=be^^3hkJG| zR8@T8=!oQ`e2eqtHHM?_|BD}Ggt(8m!ouE?qkO7sEcB4F z!SQ>c7s?y2pHUg6q~a$LJVq7#Ds0(Ht3yD#8!A>5=dCT8PO*;T0`;AJOgv|Le|-vw z{A6#;JIIKFqD!!kUw(qF>;j20iSFDGhhl&1$)_dBN3gG_o^yBZLQxg60|=*`3f--()amT1Ds)y|Jo}Y> zmd;sSv2J!xj80XEQd3hXY{^4*5A+-|(X|Aw+_X^clA#uX%lxpm(P^??Ey>cfU_bVCbwG3Fd(x2s4+qN%$8hL(x>aSFQCAsu*G2-v7*7@Q-1pVJRLHK_O< zTog(Zd8u+JwGR$*U{&`^54rtXKGhR@;nxW2tdCX&u6P@84&-n)-$}0yAFY+MG7MLQ z>`EkfgBKm6(F*OP_siFKbp815`}!5;vH5{&nfjisUp3zO4~nS)>R2#s+2WjIL1M%z z`SsEmKh>RUXnQK6&nuRchaaG$Jt295r++3x@GkP1R)o$H?WdN6Ai7(yDs{A;?e(%C z$IBo}e-w{JTl<+)OXu3QCEqnI&heQ{YPSp$txhl(9ra0C^B(p6aRU`*`t2${&73)p z87;*}%?FQv1CK-oa4-oXV1fAtV{eVCCBGka2j9y?SnB*Ew@yO|^tvB2iDFaMj@zAooXL;AGp_<#<`Vg8qW- z5j{ppvn@wnqs@wCde#oUHNroq4w+Yh8;mAtqJ9SzKIKAZag-Wle(cXkB*2wea^eDB zyf_9jr?-r)kpde*+D<1j%uG&%8XFq>7mn_OHFOG8m|SsOMWoZ`Et?BJeEbQyX*`4+ zVs1@;g^cbNfO*LWUJs`ck@Tly%(=yHoD$@sF+Z@wyXCQHK@K?4lV=$=rxJ-m3Z5x>3Il5q@ltsjfc+p=q@{R;&4pf6`PHe?tsZ%q zI4*~^q%mMGpw(4 z3y7^G;BxLw%^3}%OL}!zvHWP@FA2v7J8g~bGGkMROR}S_%$U}X=xlZ6qG(;}{FA_k z2|1?l2b-ixjC#gEgAgPnVklZmp$) zGxEORd0-@?j8bs2t{{YRE^_kaWm4&eKOYJs0584zby<}vkd)yd2G_;($CMpH^UwzH z%|tNM1oU&dO9bQVgZY~>#MU;FnhBjUs$3C_wyJ@I`VVp1v3=W+JHF%>fi0WX-xXDN zs>Vx+i6PB@!iCuhi|%p$qRY_`yu>{|rN5ntTM2xGs!POLq#D-Z#>q<*C8WDUw}@?^ z0U9T|6QZj2zo+)V!_*~??0oOC&#xlV!ghasmlNHq&Sdk|Ot-$VRBE(pkMU?QwMY(J zV`D&awNOzx!6D=r?N;Vy_fQcrTDH*cAX)Z!XEMvq$^a+0%ZPX04ys}0xi=#``+G0l zYKKClH1juK+XygDZ?uUQzp@Sa)EE|f3@Ad2wZz_&zaakWLZP2u;d29!_!TtfE zARK7x1!7RqkMi6E>DX_`;YF}Lw`=buHAr{!RfQt7*Xl+vUVWHQev`6C4zA-q7XW5F z!#w2^*%a$kT}W_;Tt@yQWnGDS?h20K(k{WVP|ZqPmx1-j3%YZ(@60;RYal6uYJKk< z-phmlL@^cCRM7=$6D8ljTs6A_N+Of|6Ce5|S4HhNqxF#T+1 z8KBUmfaYn;nBv4`cqNj-)t;(}2ly^EX$g|&cWrb(Io`6T=Gq&q!~(xY(JY62g+~Z3 zh6=wIfUq6*gginXCsCJ}J{&?qb1XCZzM!!;KcqQq(Bp?-oHdHK+K`qAu0zLP+Qe_4 zq@c*WiECvlh3Iu+eOsBA$;N&g=%qBsqE!&hWV27wBUBwFHB=0J_(}+{2C%Q<-rQhm zDC+GDKe<89L_rmS>jXb|qvl66zBy1^@3S;0qP^khX)__~?$2jX?LUoL2jDT>@sE^4 z&UzT8?uJ>PfzS(UanDt!f7@F=rOmgWL6nT*1`xM1%vD8=4vVO}Sr_TzlRDfVI^b@e zNK{e!I0aMYp+j*vYx(J802HnFtr2;+Y`JClQ#Ab8x^)K;R^=WXqaOrC)2mwq{%k;2 zW}81&Q|6(~+V10>7+>tD4YwhuBa~a5et_g(UO>oqNgFj}Fj2(rZtU|_J=aq9L_iGT zc>?#fdX#>^Pjcii_0;WE?o%KDBZ2D9DU<=k0Toe1x6kJL=JM|ufXgGQtY?~GYdxBP zvOR?^tw9^TLSwog&eKj-Ld(_=@qfRoJBLsPFa{L%@9bVzJ(p4%p!Av$0&0vA=EGX{ zamtRlaBD3rG+5etF?tuB5CSC8da>%N_A%w(MY_tOg{=48Z^J3$BZX|+3Gp~?DcjQL zCEmldrO8+yER8W$0e9(?pR}KnD4*OO9_;Y@YtWO5L!a4{pN!iY!*N+&5$x347}wU= zeVp=Lv!KsV5a6V*`B~3cbT2#m>hkYWeihLP?C|s7>`;lw)DXiGw~6!^cpRs`IkMlv zs6HAxtEmQ}f{!rQ(u_TNAgUCMujrS})~1y~OUyg*T@|2$0HkcA~ zRTz$@B2+NL1th!(1Uknrrrz!=%V!GfC{fa;a8u{}pdnn|S2J%clOq|~MY@qv&#a;e zyD+ss*5BPrn9jra0jxD#*V}8j!tyT5_w2t%P3SS%QqnMm0 zon4)iETNiNl~Y2qn0o4WTQvbMPook|gF#D?rBaoe1X!Uvgw}O8(ZR`3NGP9{3UfR! zmsm!&l@sf)VpUbOplMP!k1v~2KeC<}pYQ|sy#B5Az4+5esg6p`JgAIbjh5X!$tAih zr;2_#b@667hzGt(562UIL4G4N~n!XB8CWCP9(7)~v*L~=Qyc8N?I5`#LrSuhbiRv{VySkG(U2Ke4CtxR1$$>{VNYXt zIA_&JHIw??O_%VnjuJ!eaeCMF0re-+sCi!NL!Kzd;r9>G<*e{u@;~S_SL~~7thBZQ z69F>{#W|BZYRNc@-#kTlb>_>TEcaqR88b0C(i7ISTYh9!;c75&tl?PGOX0^gr9h~&d2}x9^fdOa;CIBFz zUS7K7s}qK|P=vYsgVG)3&R4%GiS*}?2&piP(QRx8b;**D=~RDI87?kO3B6U+7WMIQ zNnoTy<}w=E+BP!U)!#6l3(0bSdRbDK!;FKjfyQ$=t%Qf*XP(=h zpV{F3pw+C?*_JMK-C`MK;w3r8Km#Nj@<6kC6t`5KZJ-&EUOfs|PjG)*=DQ-He<~Bi z;~F~YsIPbSWj?yDc(P}JmxMi>5;e6b%+aQ{#ME)oFWmePMz&m1`4fMJgd=%;(M(#3 zfvl{|*^J|tQJ<2GjfL=afLBf`{d*EIXl0tHNRkF01h36?>nC7;_U)qoI&M^z7ST&(5eLzZYo$3&jFoFLysm*otyt4$emkLiCoehuOm&cw zh5`FZ&%kI2ar~@#nS0Wq;}$R&8XM?9Ief}ul={x;&$^b?;P>lJbQ^PL26v}bxRl;T z7frn;PI_}pNMrY%_*}$P<2pYd!Q~tEN6UTDdK?ugs)dZow5xGyTJSui_ly$j`GJOB zKsFVoEpSCWsR=;fw_~C@RFH(A!_164%O?|Um&0u@gfPWdz8C> zc(WomNU3l1^DNf=iXCDS=~s`Dw1W)>d?J8z!d-D{v^U&Yrly7&Q77-|bYfSFzvg$- zye^N?A7{V2F1J0FXq?IUgrcu1RAAwJqgB76(i{|AsJd8g~_!G+oCuJqZ0=?skGi=y|hzKQRDcNPpG(bP?Z~A zVh-+j#|=7kA|ueB-V)azn5JBHvjze%6)8ykll~*IEnjdPOyXFM34h$Pl^v=PXXibF zXZs2JZ1@*%(Ee=E>q^Y(k_Vd6ioI!KPEToZ(*-)nsQ5aFaN7W-F=clNP&oavn;Uk zuT~33J^MPjZ92?$kDXS}#N4j-Xe>e3RvRz=y0yGODq3;6>btTaX74RF_YC*QPZ3lX z+WU9Dqs!c`r&U0wTqo6?*GQwOx4i>@wUohQ@lZ*}cF^y2mIQT4G_`&+FPH#C&;XF$ zP&w?$d<>A2PuzEzZNmtzle|gGiGvUM=a{Zu2teu2T`=%JpPXG}*VhBSZie8TUPPZD z(0@zHEkz0~1FMQ0=Ava?rSOBREsYOElwzV?n-0ZhMCt)pX$MCW5z0{g>I-1rYFGEd zf&(AS`^>QpGX#qW5VIFl!m#f#wP+%xwrD{Ha6I+#c`ITpc7d=~;fGF;@7dd!r;&R? zHz0v_EM(NWQ_+bG7Lq|omAOb>sgxUP&(dKD>d)d~e|`Ph{lMP>(`k30W(w zb>@(p8}9V9xB{s8cv>^$cuY4V%2k+2niFxXs0a#gUTP_0!3uI2HDy(2@Fx&ZP zO-zA6Gc8$EuEaC`9eP&Ojiq>u;RYFdR z(vY9ypboJhS7IK6q> zv;Ptw_3@e$eGE4+qq|au;EvDY^wAcBj~#`91e*=IVql;IYdd9Uz#{}JA&3T~fx3B3 zsy4931#|WVkUo5C_ygXsAYXltOs9xSr#OgPGyyfMYu7y7Jdc?p-4hGO!U-b~K9f%d zv?$ju*q}4NC#*>C_l6jF?Pf0RsaOSQEI_pQ8pCr%^(JpFx(l9F`1XqDAVGiwuT_oB znf~-5Xx|W=0Wu~1fUD1h4Y#auuY%|*9$832%wipQYB{m0&;}&21C?6TBuvcOI)PMs z>>(6b)L%0_DiTcILNml3ZUXxK%TiAiQwqvRODUl;FiScVy1X$%TQ(mONPz03g#5zM z!MQm-wM~RLC=%D+Q(n{Yb&xtRDg(uBKQOsrJY&vn@bbNB%yA}`alN7x>*$|&c3Nf& zo|s#ueQsEjN4m~f>==bXOQ_L__(+Z+Nzdk@ei0DbJyG;Kq-cNY`QbDU24f+YBT3R| zVy`3yY|MWV8#4&ieNSf2Is$y#79-#MesUdh>m?*NQl~SPVUo;psAZO#Vb+W{m0+%R z?)%+Hj0ooBgeVx^51ZXDRN4!Dvwb!=pgJnR&hh@r;N-jfyu0q=%do zim}k3$wz)^8qvJxk%7~%NlBPr6!`R%gR$GjkUvS_kZyY~C6ixx1c?kz($V8>3fZ7!u2Rg zfkBbFY?U);jT~^6A;GRw&!Ymam80a}j>mJHSYaFl8JWS`W-4xT#Kn>9Sc6mR=Wa>O z_S+cTxaE0sCy(WGXDRNf;6z*dbv&+t!0@o==J0XdYd5tKV zksN(Hb&VtpD8Xx0@k}NYc( zRY`f9n*0>ebE3p$A^h-V+v%~UP@fD`D4O_mj}PW|;4`016c!>{NUri=TIy_D!#Q_F#z+oFa{d2lw2^2N#n3EtehW2St49)^pRMU_JP?w zpr{Z84?(jth?qGx49W;fdr~2kk-P1J1m==_oJ!j3ckClKu3KL9QpOk~cdsHW2E{JM zp%SCTfE^1SW~4W!C{|V|;}E4D9S9Tq`^XKZqXp!An}hT{Oo-rc!F90dd|T+r5AxS^ zH2+`XhL=t!x{eE;Nrhred)Rd!O699m%L|@^6muTb(hIZcH;24!rucRYhzuW^VyhdWx8;`VRYu-F<;XT{ZqZ zm5nAR87#d{ERA;i+5TDqCsxnPsxc=^w|;XB4=#*yAlvEhc6SKSxH_dnwX|abz{jA$ zoSU@1Q)v?F>jh5aY>@sl@%QFF@V$r>ak|vVZ%fgfR>+t28g6}%C?$GrIK1N0jXGjI zn%{5j$O6)JDAeY2qO#4B5wb3z-`Y2`Zh(OD)J7}}nq*|If5-eVBJPN6Sy~A_NS4wQ-8)vnW?+nn+J|K=q%-bl&(a`W>BDtq05nIQ%k!Xl zm&twnhr;DLeleQ!CfTUP&*Lx?C?u6x0#V06H%=4_)P@l!7HaWPC6H%1V{aTs%9c?q zZeCN^w5DEg1e`*?HL^B#<305a+xc>N^vnAi{WOp3$>eH3%GSkkDa^_*tW%@A@HOVg zGwazz`Pl*IFkiZk0BEb-xo++KR%+L7XyUG54&%|E*S(ugDc9-rqoa)inDhlt_M#=2 zDiu(DHWy)^Qiux@mT9^7c7F3QNMbA)b|#B@=5NRL$fR3vZ*~E7rt*H)Po%n{5POG( z$bU16_sRgBEiTCeo1-QqYg>iN9u9zxV&qm#oQxi<|DMPd4WNoDvD|gIrFV-t-i_GL zz6tjdsk&<_d2L~qtTj?^&naoiR2!ugG^u)ytlTFni%!U(6%?iYq_5Np(_{(Nl^*uK zLBp|h!%zf!pk^7)wxJDIwit!pY6aCV!ly|F zM$Lr1az+})Fp)!jV@4G{QV{%x5q;=Wjs3vSbJAseNfHQd1v{d<4WGo?HW`{A?{<`l z{|dxn)QJ{8lTD#H%D%^RFv>JE^f6#|ZF=O9947r*NXi&TWVT0;D4|PGB90ZH`m>b+-O{x<8cMk&D9n zr}6Bjtn?obvE*!Pj{K!}==uZ(fTusacgedmPsvBFS1rj5aVEsI_oc;=R*pxyR@j+_ z`<76a2@6s;qH34&)D#2ytEz*0d;QmD@gWb5=R^GT{bR0LDZr&KKGn z=7@UHsF0O_7K2HRu2iW!wO@Ne?nKE#L@yIelO|@Wo@;cq_BVdhM(U-5YU|Hi{y|yK z?8dFQ<3YT}s#S(kJ1ShJR6_zaZn%lkE&`QpZ#|LJ%8Zs+P`x}mjl(1i!$2%tg%#84 z%1Ii?~>wa}++TGE+ATC6m|4?e3p94$jI7F#Y(kHaLj&PBf5Mh2|# z!Vu}ktz+3AeJPl5dS^hyRH?S~aeGkJ_BaN4mPGPP@dYTBT3of^tMPNtNw|PjHmq@8 zA$QHK=Irr+GjVdNYP)^w*@D=yw$s4D9xrNu>iO{v;cuQ^T+J2O)!Qgod`$f)w3SAf zzGLU*>Y5~GCjtKiBDL~NSc}_EMGjKU#4$;c9{4d0)}X%| zK)3__kp`eC40NHmpZnnH1gLgQnEcvF)&X1F*9vJIZlyPiF^!XN)hh2)PXK?Ds%N<) z({iXQ9NVAr*mdaQ%$>EGtj?dmiLv8hf8kx^PEJ7za!nk!LuP;R>FUqu8hf4-SN}5- zl9=h!*;7+^)^2_-=>?VeZ7Y21G6Co~wt*NJa$Mxj8265PB@-km zk2G!ne2AX%=6@z9@Z!-mn4xF91vaso@6y`v*YvA&3zv%;CAa6kK4cE9sgzindjtaq zv4FyTaytr@T37aO@SrZJYYyRB7S`ktVU=2CmBA&k{K3#uW@BFHGQwBlTk*sIVs@*0w!&1YEDl*UvR@X0NYqUkmJ) z`5%P7s!bqb|75}-S1Q;|2;(ye!)AaRbih0JW?D_a)_k?u3OmO}V|G>++q226!QxW2 z1(1t_uhEi&!*Zi(G7GU^@r!=$S|)@5Ona@BU%3WoCw|gNFLI=Q*9^q7F{Z(wn>_6a zzIt%OY6tQR)%Dk%1TgYk)5&SHhD0|3cUPD(1H_(oRMM>r-mXtReLH(?&~Cs_0EC=Z zPdHC8v2PSgA(qG(KTiTM_j*ulhD@A(+djkHhUWftq$a!_rEpLd`5>Zq{!64p@P%Rg zUs%-tgQCLw#kTifv?3D|^jFolf8dPh1mz6ZaNoY6Q2~vOG01?(!UUK=hB9abz_j*1 zFe;qCz0($!2{7c7h5dn$MWOmeV&5P_q@lq@2T4Fkn>8Y)$+a`5Mz*x2?P@q^TOwM1 ztuGts*#Bx>-~TEqtK+Nnb@#dS-+A)id3@WRa`51Sn}WPuF%0N!yYBYh@$Y)__kQh7 z;D;>GrbjBkNVvUGCL}!GGv)UK8?=DBfJCg5-o#dB2*D#;2ZOBJ^*Zl8d{X!wn zSf4e!La?7dc6mTr2H79fvI|QWeRPaoTT{1x3_m=kau#?}E~G0MLf9Pm{H%S5y?%)` zGbB1rtm2lEfD(LY&eE_ANd{lIU7Z}$?Bl1u`VIV=%OyG|YoTroaAIu>%#*#R^eP{Q z=(Mjq5GJj8vSR|LTW;=`+DDSf18RdN^zOr=&e2)CR#d+(ee@JBR(!!}aj`B61Fw^M$P?f$#!P7x+U2(* zTh@B<>}Bbb1kDa{ydBHhfZSSqcBi&%-3+-v+cyqA7&h;5fQ!T?E~-!JtDKaatx3lVa=l1 zrE0lorF)_4b(--|c+imH6H;_HZ{;kli;0u9yPU{Hp4G5Jk=GN}bNPE3QDi)OmpmEk z$1M*L@5+Kgz_%Q+0>i8F4SZA#P+Pr%O)D`TYe&%KO4j;LRL!%EWJyv%IhOtOS?2;w zAMB*5p%l_C&@^YV%w_YhX1+6|rbBGu7RsVHjUC2!EcfAN=OA&?K6qRVtKxO3LHl3_ z63s#!V&hA4mXP;^j}WYrq+EMD=A~}-1?D19YZMH}08o@Cr;^(eYu7PMYg8-G%Rz$d z_|xHP)U-U?bosK7#wcP$Fdy19xwST<0;!$sQ{8fH-0?hi&#bqp0}$1-oObEXBNO`w zJXKuWjw|Oi1Zko@KjE(0!{@OEgSE+B(AtHn5^<&)5&KnX8*qZ9eP@AO2dxEh7aXek z)7!c`Kx5ti$JRSVR~EHfyRmKCwvCEy+cql6ifyA}n-$x(ZB%S0C*R(u{kxt0pVrpJ zyqNQDtUkuu+vrcHoKalh0-c%^+AqSQnjz*Fl^HH)w4-4;FGmLSPizqPM30rqbHAcE zT+u;JWCT6HT7KV3{7wyV`9TI^4nwqI0#aYRWPM(eQ3c(7SB(@s$=o_jRhqVs<)ifL?Q8YBKqeHF9VOe0!^7^ds#W21iv_6rFs(YBg-kr&) zD6ZwHN8JyK>0*OQ=uB3EYT}Azk!8wrj3tVXC)!@*(lE=5s=rj_9>0nrQPdX0nGhGi zGACyI03h`&_-eakAmv6PppxhdLvU)7Xr*(Nqo1T2=%rEGYLh4yWv#-C2`~wYL$=Ie zVXSISsfub!GZSsbIW#zfYZY7)lZ77nG{^T(K0F~=R#^}{vrsDNu%6-JWsxff6ykKM zw-B95=uv||o!*XFG?|Vay{}1_2FNn1b9AZ7fHf6ew&b%Bgns8uhXUr)tGJwW%bH9S zOYVqyVdM!g^<3S2p)`>-ap1089o`Nd4-dU|o6o1%r!5eT?EW8SKTO*7>YK@?bu;Nj z5x;ll$ou0Cq&Ts=<@rGH~8f!Y4u{|%*!3C!& zKv+lDtQ^IEliKN-gVDN`j=GV1_)YJsf?7o~V(((u%7eyejH2P4AO?P3c~#F5oaI$M zxksVJv_sEO+oXH%hyovKmb7Q&4B9LwtY*aYO;_!<|FmniOFnMSH(h_$ca&KF9Of3< z8);(8C2hq8sccU|M})N<)m-7ue~YmUV0m~)v%IQgJ~NUw51EKTTt+Y(sy$ZveD2;h z{H>Z#zt5c`RqVPjZa0F%nIc8@$HsAVSUc9_%IN{~y3&=4lJ!-OWUILJ$t5_LEvVC3 zhhnRTRH{-qvx*bDE-YhRg+hhFp2S{e@bZJJc;+nJR43mBM`7_|^tu2kus;+6Kwj2h z9pg@KPp;zH7EFb}JK0p6;gtR9C_yUJJNq}eOpQ6y`3oC|^?5IzNlelW)Q`P)>fjAAEYI1h^f_??GMX~_~~>`06$kjsl4NE1jEN#hEvJ;d<(l9j?G z+4vr*DhMT}s6Z0L_v&p?jjRUM>}; z|K;@tvxv^04(bGrGkNM~p4?l?AaHH8MOSWyk^*l^5}Qy8f=>^M2~V^JXv8~jNd4uN zwDKj}VlPrp_06CcmVZk4b3w|zCV{u+H9Y>b&ytyoaEq5;p_)g%T1ixsClMfTP{tx! zYZC4rXo9CANp2yfbS|+++boV*5e{d9TJa?{bSPIUxT|BU$ii`EZG~JCG`ApzRdo{tG7IPMf z;@u`|{HR)Mgpi0u2K381vDVR64OQB~m6()ZfLlNMy|TLKGd&U%;8m}gom7)KT1-}M zU1T)2ur!J!1}<#6wwq17j$P80=3X;6QS`!C`l{v`f?Dfx2KM{CPl6fnjLhcFqG6&b z)GgkJhP)~ws8o^MIX7xF>yyxHM0XXtUV~zyVTIEA#gytQffu9K#ij7Gi8%1qm3#6bFCMX>B9I9G zwxMpWA+^IRx()4Ik4^q|yttmK-5h=lVixdK9DXUTt#_^Pi6E5S`r{nm{iD1euX%A8smn9V=C}_l(l+ zWMX0;Fz7{OIl=lqJ6?|nyAW5Kl9(?jQgrqPy+_7(ks@mOaR6Ai_mMhcx-S#Hjtd3z zVlS0740O0dmr{1qqt+{7eZ3mw6Fz#fKwjnB&9BB6eDz|b^`;fhM|U04_!f~RS|nF_ zI+JXbGfr{=y&aAn15@Y-oa`K+jW%7A{Wv>dZabo}ezjyh>ky*}wPOjjefWbF64O^? zzi19G7&;{&)Zw>vMkW6edZ6lKMz~oCgS4eVqu#vV_(8t-j0kl@>1lo$Z_rIO#>|AK zHX_#9LCyHK?#RK=O$|cc=A-lqc=U!zPN4?SZUGAd!o#G2e*Zi~&!P09U-=^*goEl* z_QdlO2A{;k?eS^ycr_JSQ|clWk45hxL$(gJT_sDp&Yafw+JVw?gVLHBb{~8^B#xD& zc!%u>wLVfU3xLglipw&Rk`bn!3W}uBok$Qavx42Kh&wdHC-32$dXRp`8sgMEf;%dD z9$u3-mLU@x>;nE1jQXD;;r|*cOxcsA%p(;-xj<}Z5( zlDD-59de~fz*;p)-BPivXz>N3L})p6tn5Frce?Ga;p356+_1H+H_9jC3ceB$kU0px z-mrA|#y|LFqS2j|{xJlWyDH`5yL&GuI;EK^3=&o45)&6?dn9*$RB>W@0YL{$V|&%* zeh~SR_H4XaX$z(RipUW}%nNKrrQ8ln(N*PjED7vF&})jxo3e4vcobI_8gb<}+!o1Y9JE9WkL#}YN zgxCo&CwqDv<|NzXH5Dps_yNVQe^uCm>hp!mfEH-k8kY~A{}gD6%+x%>=-JPAlcepO zKf}-cxk1rLfTh8}^ZN6ELX0QM*5dHOO8SP5a4iIin-Lx(JNS#jm=v`>-bpcC z0OpI~E&b?(qNTJ~PBBY1R#2r`rLgo{EXJaj3{_~efd1eMIW+NLgGTP*k6&rXHcvj? zgRWm%OpcBvEI&QgUB{<(3S8+a3nVl!9b|cdgC`X{V3p2oxWkGEge%;|=Ej{g&#K=A z70n^!-*Z{-5|f|?@X>IE6$};pI|g;$V92^{-Z~mz6;%MGj1oxP0L0MdE1EkD)mpM! zDqrs*T*_s*^(3thIJ4c5Ae$9JpuPGSlY_a zvW8!@fW?NKqWy8=ii+EG*b_|ey1F=L5vYHEy^IbQ41{+ zrUfGgo$#b?7KFaJ9X$y&b9*ahc6R$FQcG^Jfd&-!gVFJFhn$V(QEmvlf*U|bm+E%RD6OQvl%Hjq zsC|LJ4sK<<)FqxSh1c`?GFj>CJ_8G5awn3H&gIG^6%JZe}8 zy{VaNHdhT;Rz|CyHZJ7SyU@{g|LguDG4aT-Y=BxgbLK!kP0`Qb5zNnc1 z*oT(<*Ot6jkA(j2G$q%x^-Kn}OQ1Nk;6H1Hi?~=SU4br3?|oSRUJu`SZSL6tp0rO- z5HiveSIu-OYiJ)}`Y$R&WR?u7FG&URT2dRZ4m}D#F)u4c1h+wp>IW8BiEv6n9x2*5 zCGj)TL;9&lKJf*Gg_O%uD|GGPrB@pWOS?0mYTUdxVeM~c0{9beY0{RZR&$K@D}g(z zBM6zR9Iugl;C5>p!fKdQv~~ml41MD;ibq0+KL_3o0k^c7a6TsiRo*Wbw*+2|4&R6l zhnKK)C5L&8j9Qqp8KshK5!){iyFtbTGC;`YhmF{m!)V@4!Q>}|-_viQXULF#X`l}H z;!Ed&Ybu9`n zST336xZa#38ptv)Gi(|)SkkIb^E+D6tE4d`bI!-P$aqo>)-=O{yL;D1B^cFi5seid zQNI%)yVQKO&}QIvh?lec@uVg*|K^;qpPMO5iQjL1noWs?Fsw!aU44n zLbn^Tjy^3J@bF!Y6bDlP#zO*C%q*sA%^eA@uhMHMk^eF>cQ~de(>)H!_Cs=RKvd}3 zKnR1W_tct%yac@VxQ+PJ)9Vb(i9333k+Cz5ALg!y9+ih$O~()fbl?hY`(!bHiKBV!5M=O8p__7E9tR)KOJV*|VP^1;V z0w@(ZP;qp-7thd7n0fc+-HT&u$mLW&#|G1@%xu7AHT0w|dmM42{DbKeA0i)V2*=A0k*q&5peiT)A*jz~>F(u9JG?H*l+vfKDWsp9dCeyn6`*9?3ExRVTZMz@4(J((}zp3+%Y~|hC?3Twv;$vGk^~}AIW3>*S5G^UdvsKdvm$>J%+rdWG>AL*cxBUk%9rWBkx7Ru7ThW6PK2b^ytg9h{hjXK zb1(wXAZO|(q9+4BcbYOrfii;UU6v{PsXVLlI2aG+d z!eJD)aNDhbSZ2V}->0NDN;G|R7aytnBtB^B1}^5AJu2ys89cu7vU-;r>x3Fl6w+Nk zb4cBM`}%C})8q?(JcZ^(pA-YoU_pNyK=rfN80O31O;qDebd{+mIS9e% z<&QVHPKOg>;yYB2=pH;by@d=;y$>7mu1>=}FT}dQPJSL9LHa+v$Ey+)C<_F#eH%So zA}){!NW1w&H{Xyp1rZiA_(h|9w9b1p{;NxD?v~mZNlcA{epFhD0xK6@>kGMZ`P%>p z!rgjSe#3OrSOpFM|z=i^tpv!$Tt|3m>UXX)R5%>kqUl>o(H4?Tp*D&oYtj<8<}w7lRr zBVf7{H^43!kn>L~b0QbaaHKp_Fyskvw9xb?w~&2xW00~u#vsq8Etkg&2nZqHI=5i! zgM|FGNw3_dO#O?ZqCvVA{0wbf>O0f}PNRL>`QXe^ej(;$pXCnq=tbB&4E+wV67G7M zV3?Omb4ghXOC~p*zPLjO47W9*E{oOEa|L#R3#-*s0*I zp!?P6Yf9-G$J3iIMW(DqfUkxlWeu*@!zS9Ifs#O7@D!0pHT7niY3NtmN*cFV{Yfj4 zL#4E{cw=A4@TtxoW_lz$Yngt?buH;3N?P{4lJ>Vr;)*wFect>%<28NG@9{YE{dGwT z)b(P5W%p8!$1uKk1odPr-3OpW2D@c#yLXY2lenQ_(n`H5lB-spqXh|2%K7nsk6{-wYNvwp>=4p}!-W$zAYIxQqE`JP>eG1+?}4%wvAtZs&}a6$xG~~+U`LT9)0GbZe*93WKd(}FB!>7 zomGPHH{_F+Ll1>UU9$#b|%ungaS3I0hQwjgTq+OAtcbE?~bEpeEqaVmp8Qryxy93f}P#?U6OKYPc&)bIw<)qoABC zxKxdgJdBJ-6%R06)Y}`^Er@a&9x3ly?g^R%Q&Zj=Yf$z=zO8yo+gWUk#t9Z{N9#`0 z@KfVs*GXDu(4N4;Zo`nmD(5TDs>NN?O+XbO8tNv_8lzkgn;b9n8PQNUgChey;f|^{ z(W<*tJat{+iP)36_zvS&xZm<3WVkZBD)gqiFBvGus03&XGMjD6=Mr?0Vr-}J3qu6= zZvKbjqSmoKc6EgSPFUU=5eMGhtOyT6Z^Lo?Z#veh7{#G0P(k&Bpj7Px7?ehSmuzBf zj+SSl?TGgcXrrUhrisoX<$=65&r`u;a$R1Xc3OrafHJ9V6SV6Wfi0UNHA)Q~mRZ}> z#8ysc4>y2+(Q|+7+Ck4|Xqz;u3(x_dcVPC@2j)rY;^uSRzsSx<%C5WJ99nMI5=10Q z1JC^cigL|$mv19~A~8JN7$U+T$dipvI1>l^k;gzXbR??M^xBGP2EK$`%Rhb%e;ZcF z#s0CI!d1VN<9_pP(_6P1y%wp0dLQen)(w`lSpqmvv*~O2$uW>J?^%kgn|^(D*4*zn zlsqFhMr`3{!NBl+p2{vyTH$5#xtN<8x@!9&qJz2b^yCms$L3^X(d32gX#W0bIIfx+qacaF*uDB==;5F8O; z5defy@F}SE=42w}?6pwv{nNrDC(B~s_9b{AAul=%nC=m23sTfr!4KZBi5d;5dE#Al z%5PTwZm6M?CN@NZnGYup^1T>VHY$MlTXO>xdUo_eSH-v*a3*l=KR=i{QWIJ7jyIb*F`>(V?@4o-UvmE zW-n?TpC}JBrZB92-h8H1sbjnSyaJQZU*bDNmP3Jdpt74!NMvyATX-kZJE!st3W3{r z|E%oK08f?E8Lkx^P4R`?Sx?L<%)&YJi?_`8;;hp30|&N)?Z@ss(^Y)fr!UYs9xZ@J zWu5yx;mgG~rn8HI4|x5Tp$e7$KO4(9b~-O8X@6A%s*;Bglf85l*seEf_a)PT>c0LM zCeqtlo$~<@+$Rh;rK3J5>sX{-drh+!p2h*NT)w!! z3+TTX>pjq`=bBHu)tbKXZ)f(BAmbdIiR?|j?a0(`2TyKxv^DOyKtc6CnD(-`TAZJy zN{WMA7{7hEuJs0Bt4zI+ec|kJxz}G@YGaUX-UkD_rG^SEAXU7#UDE28%)|g zIGX}M#}U;G?d!{YLvDe-OEVuyXm-9NKVF#%=Ezv3EG9Uxxv3fa(!G&QW0yp9Px;vu^UP?+;^IkF1CL?XwDqa(^N)S?HtaKX&4HUlsG!4EnU?S;I4Q zQ5;6?yq6-RKQ*q^873yIm^~0>%t@T1EHoJQ7y7|+H|JBVFD6%DyQ>R{fx(0|H0NOa zgw<6>m`vVU{Y@!M+M@>!-4ZGD#AH=k)={i^rlJUAamM1yH{3bUG^R8~T10TF7eoL~ z8}jTSJXF1s633@=N=uuC_t9ztT=XCfsr7VQP2keHjJB)6r!p22VTlpCrIwH?TjFz| z32y|}>v$+;N&}0$tc9ZOOVtSEmPIU?i;H4$*4UdKkmS7fNRn;1uX!np53W2`&`n$1 zmqSEk`Lpv2JkU2_=~;iq<(StTeFXt>ape(kb4bZE^9)%;&z00^nGV;jC8f5cRwZV5 z#86w&&{Z&tGpG}*zepA{m*EO0n&Dj1L8J?ganAu5WZ_KKz^YRw9 z7I64gC@umO(k_KP&aS`1@d#P&HzEDc%W$R-7r0mF0bJFVlvpcKLuC$vMG zM?7@A&P6LiOaV4kC(kKeIR6N5W;*lr=v3NpiTM_}MIG}=iW=0P5Wb*D8wS}mHX4f~ z792=Np`9+LU*2(+UU(5YO43iDi5yZp_L1GgaO*e3?T*kAjv0Wf3tA>8bS>QhDWBkz zTWI-%h0aBV$znf{tojh;S=nwlg)O%x#!{fXYjFTbnJ?JJ=4O?6LgOoli#KeB&*f3o zG%jxLf7 z{(U_4t?a?qGbZ5Z!e#yiTVhqBANjHl5y1a~%KtKfj7mlW?+SfE1P$I)0Kqll?)%l|FmZY|*|q67XNm>;I#*|67Gyl6j{IepFZt80$ZGk0b%_D!6KBBQxaj zLaXa(>eR(V1;@hBhzaABB1F(oWpv6>kM(4nWBcZqoVWy!N(O>&h%baMCZ8dbr_l#9 z(=UwNIgM+>2wl{ih`b>g*Y@t$uYZ18)i*!Sb9cUg{GDEip+Ik_$1H$nPupb@DaPU* z6X~R4$rK8bE)sm?jpG4ml9NE1!mUv6!Z&@GWE@fngT&Mcc=33NcnNq(o20LahqQ^c zpf~WK-&OYiruje(s5+`RF-Y)rVJ?^{{ z`Ytlcd?Cv1TqQ3^b3bp5-pRe#bW?tDNdHn9AOK{%zO*B zu+r901td?}CRF>_G52_xjI+#q5^+wK(JfdhMM92TE0*ACkedwLv z{dtO27%R5znDhXDjSNR!mjUvbm*oj>oqHlQmJ(T;QzV5v94j#PJ5_8OoV0K}x}Xp} zi!0zVfArT`-LUDi&?rpAxLmE#R;Ys4G*kt!GP zgM=9m%vf#Kth#IPi3Ri;?Cg1#u-&$7ni5&*v({BPG_(P=6@G}%S2>;ULPzGv9%GT-rM&i}@k~;B9e4k&({Zh1;IfD@x|>X7lzBTl zla*d`l%@x4E0}%?c)CaW+m6cH$h3Sc@}3)#$ne;{3d=1wZjC2ZS1mB8&17V%G}Qk| z*73JJ%gt4b(g3p4P>}v*bdJzf$>yk_?GL!)Xz2Pe2=sOi`fCCH5&Z*wfOfz}q6qFnqthygSheT#2@=&XAoQAm zNFzpMm=iPfqXB@(V_F}hP|TuG%m6NfweqG0FtBw2#6P*(TF2XPlnb;x`(+`yJ{<;_ zv^=T`24L$N2x}a3J+vAd#FV=?8mx#POM(huNZ(!+AYDmA3)Qk~_w0*=Q74DHeb7xj z{#OS`ZW)RRVZz=Y`WA``WYHU5wI4U22+K|0fBokZZ>miaacBWbp{<$#X`{~IbtJ@N zFW`Un@Qb0G>6##58~q^*SaNmrO@+7bIrWhW8@KD9ytb9tIYCN4BVy4r7(=f4GapJ$Pya8Qs-Mu`$x&6ktbjQ_e*orT z6{_)$#$i>xag;&?CVqOMmjl(K`M%xYgaY#Q$?c6X0+-q}td4+gs zgV0+ME5B;~TU7s#bYlH)QT>%h*H!$3ob-iEQ%V6LO(R<6p#T&o>!>UUp^bd&z);&! zX?D_*fULtJQ4x)b6U4_NqMKLkyFkg{>T<}gR`}3n&nfvh`aFqp=fJb8;uh_iI=eq# zaQ7VQx&uBw-r$9hWuzX6a)o#zK!)L>Jq8BFV4O6yWv!Wu#g(G0=*{6KaXd%kSC0cV z?1r$-Cu%IDY5+Bwb7_Vg^)~3*!vCrkwZv; zhe+Lexir-5k|bX6w8_c4r}Jg@z_Xh>BHZL$`~H$|SeovZccF#b(qPT!Fvnb&7igR; zoqzZOCV&frV*Zrf34TRxbC4djh1qZ)0YYwbC_dshmbvCvEvV3xAHlb>iv`*=0<|ou zXnpn_drLyiuHvb(Mopcr6z)96^6)Z4$Zeu4T#3rT^tqct^-2hoJyEuw%xc4WosR9i zYO+Ef-LZBIlbD7%;;D5$XQ7G$XQlAHfG!MMF93oEsjyzq-~}#~(bU*r_7?wE;q*i3 zAJ&^x$ru!t5TzPTkg#sa(vk_MZa#Kk#xSdQSu6Yt&6;JU>K7>wti);L9#|$^bEp-b z6-bGQVWbcz2obw9C>LtgmkFTyuR3V~iey4*bfa|8xLf~$v3^(6 z^Xni`6M`f6(V$NR;3*2I#)BcH9VwEL0E#0JIazr*-`}Sur>}B*1OdRBz211xvNFu1 zLo(Fn#j~iV!@3|U=^AzML<;hK1%c$?6L90@Mi#P8!}0W*HCK+r({7E%TnH_X)v#SQ zT;y55b!tdoGBnL3$K{&$qCoeY3Nsm*e>{jdSC<(V>ZZZ5mX(JTPi8JZ3#cF?K+%*k z+_>4*S=LFQ1B6*%Ohn|7#W7aux^N1t{T7*+N-dO<&a6jqku9D*%!FI!nUp{KMExMo zz=^E{l*grz-}O38Ni0?Y=Fu|-U04rdh4>Xdj(8T|C-@561?fQJVf$J1cm?(%AF`Lu zqFU~Pb8!*~OVnbpK+7<{sCUaIz*~!kjdz0TT$mp^D3&U7v>LjN=LD0V3ouO%k(|r2Qk>Nnz$~=gj%(yC zY#;j5JiRlc&hRyG;gtoh>A6{ZH2$0jm8Z_c;~>dt`sQdl?KyO=3uC|VM4dht;pVA6 z-1Lar1Gi+%SCCauP@<*n!yw599&%ttrtE9QL%sfWivZ9SY*SVwU&XpHHL9=svn0S0Hdzv%QU z^x@^ddNKdAgvWlg-~IoloeR=|Ac6iD?fhY@AyDP#rGc~n3POT3rVe1VGzte^3_!8! zggq)V8oxY?6yZY2f-+GuCwSrmv+@uKGC|6E0hst1iPajB+SQ;n1d0Ot!R#Hf?JEoi z5fTclfIp%Uyrlmih|^_oSqKA8!CZ74|MRAckMB`j&o%(4E_ldDxBm#4%*hN3!<@Z6 zv;Qwz4Xdme$%B`RN^k<WE)?!g1jL78teSqHYJ}!u^gdl5yz;m zbi(MLg;ytj!tp}_GTg0gEP`zjPLrs@=Cd1eDS`t)2~}luebi)nnCsYX6>GsVAfHKy z82k9mkahDyrgBdG(NF=y7f^osfv6xsd#2d9h$GEkQh^TU;Z+vDmwkX*=x7SWNM1ueS2hpp|~CQi-u zih7|Hb(Sro@F`(4fC3b=-E^@q{c4E>ekN@EEh|0qQADrwitY>t*{# z$zzk>LzR3e#}P?A3Pt~oar1MRQl2~<;{6%oW|901z1K5J`-Sgtv_tgOK$*%GB^VRF z1$`Ph=-B7OXq-7LCDL9+nRk6o<~)(H^B+b3KWpy)(@63k)rA-n8`DDu0{ZFo6-H*o zPn$L2W(G`YLwl;Mq66s_WFO&7&P!RWXEZl9$?@2gW@JnYT+)%}pYZX^ zC^W#K955QZ&UQR^d0ca!2%c=Z%(Tro0ABvt00B}Lh{1cDjXC`;gHgZrpzyy{p!&b1 zufGrPa3OQt_UQx^Z}O9_XKqCUK|QF&o?MOvL+;SVJ^Dioct?ln*N&8TiNv+TP+7|q zDR{C~#VSAj|2qG zrvi8!HP_n^@DiR-APF3I^GPx#?xQS$r#xmw_V&~*6FLp5W|bjhrJt1{36SU0#-@&7 zQn;PEPK8UIL-^*^M~mkpepMvuv!91sqiL5YR&O%447{8ikIeSQ{tdq`bTxz}YTeKr1uPUcUh%3*tZdvp2f^2Xhpx4oH@-GhzO-F97;${w&H zGUrdQiaz_yVby-TGJqWT4?d(x(qtAh##*U=eOaiLqUh{xJJZjgjIvDJ`eTn^%3z`P zJZQp3`*6MB)w@ZuiJQ!UM20n9Pdpt1>Q>aCF53!R#8zs1o{3#Frg@&r$I{Bt6`wx5c6nBZZm&(p&5vUQ8v-TeN-I;>P7|ph~WX04fY> zNegN-9c!i$-H#fFTsLE3vE+p?`)gw$@cu&u9WQXt=O6q4<=zutJ7$;7mEvZoE9l$-hfe9mFDyz%J?DBipJ%(w< z^>3=C!?pj6szjl~?1_cBttn%cP5Fz5SlzEI5G4Z-#q&MR4JhzRGk1RCo~q6yLz~04 zs?cR2Nt-BmSDOR`>`DDuoF~A0V0O2p(A88JN0-57+1ZD*)&l<08sZ~cam!#B#~?^w z>y<@u!zwS=aETkrY35&+wpWSu(7 zP;0AN%(~Ashwo)UwOEJQaq$Oa6jP|y>ua4OweUD@F(O;Rr2vgv?DZD$zlMxPmmvij z%|;p|8DNub+MWKkuwl_|!z#f-L`C=8+REm)Vl^~cIcm!PxJlL#N@zp!Y(Ah66e;G& znES`!nx@}PRayWyOfTG!Y+Ztz3&T#KdBYiDq_;Y}X0K3X7@sB<>Rhk?F}OZRe`Bt;t&q+yk2CMFbxH-FQ^h?C(%{N( zhhb_9_4IP-5NtG?PcBNm@&$aGQuQd=yJ9tn7j0e6GBf%?c1Y=$#BWVuPq;YLW3s7s z^9!GHZj^+Coe@MXD=x+ris}_i*pgbL`ZZYN#9BZ-v{h{+tEbA4jw`RlwpLOjnaCeH zbN(sSfHvBn7&4&+d53 zYMldp#TQo_KLfgowx24Sb3cRRT+96WmTo{^GTiS#bw8M~IVU^s5As6?<>lMHU z??%734&7+WN^;B$C(EI0Mq(n!&TbyYK3Xk}3I^Ae63gE|lwoD4J?^J_FQomG4Y#1= zzIe~{5l(I+zq4;`6_^{k<^CYgB!Vu=66TsmzfY1GREe}p<*zC(!W)^C&U1)2TP4&f z-kSrRv+E~Yd^n~gd9cO!mT0G*mq-KB6G`e!Z#Fs;@j8@~EtkJSgze>uz|A(6aY<_@ z_sYzwIupFM$9M{iw9hFcmN|%zC*Js`_|`@rU+5_bj*&;jZ(8X`P!sXUQ83&AOn51b ze&rM7zE1i{S!5BYAjdZ^he54n{cF|^2@@_~{j6TS#y*3Gt4kRR!gl+tC<=hAR5F_n zEugR%0(zWqOa2Hv4HSQ@p7iDAO{>@6LFOAIr>pRWJjc@irn>mb=c7~4X$z0Er!3pp zn6()MqEjnT3y;ZG2-Uq8y$oL@(xPbAH!pQD78sqY?HeNIFL zp8b_)pzio669mDsPQWfiscrzW9*DwT83xA(Qr+!_2R66{yo$l{-H7|teg(tOyHc_?C%R~I_#n78l~3uUy2u*+|h|41L;SZQ`tt7*urP$IU)Pn6hc`;^*NzIuPODF zs`b7}LTQx#44mrW=%)Z@m4aF>krBtG~@SPf+bJo%B|Ex^3X)qnDu$BHvp-MwuN!e+}8DT;{pu zj^hpn)NGi+*BXG=zTvGs$jb-W4a|y&|a-mXKE|c(*xVXrcGy0dm7Nbt$n6VEH^~0D_q7GLK^{k zgBGD$QyHDE${Z}7Gfqq=4=?MMKEBdu8;_{@O4uMgou7!TE2k^+gPKFR4lx~>Dl&xR z&Qn2tN<_O`j*TGE2Q04%LC&u^87K3Y1-)Jwg2?$PKv3Js@=9qxA2tMi>*@+aP|M_cM!>k3Xdxmz z$4YQVO^{1Xm=kC22*n=SrVh+zwa1@Kz5swtJy#U!>-}P5df~2uEM4iVM`#iRsLVFn zG$sDWhc7jI=v;#nn*OxxN>nl~%}kyng?5~9Io>baIK^LeR$!cx9*@|QSVJ)^_V-Zh zarvef^7?lu1i}*8av+OVba+FlH8f{&d5v~xIvvt>w;fskYzt5SRZ#tnG1SSPUezeSLw&eCDLpV5 z{a%7<;u%t@7(ECrs@Yg=6MF6yel{k)6h*%rk>>Gta~0${6Fz=La?n7$SAV3wC>`)j zkKKpj52QR(^2s3>aCzXGo=tjT;12?LoHd5pag4ZS+*CfxZE)QXlv+A8PFG^}w{Xc# zoXp=KcLq5$vc0dRm=!kOU#pXLqB+W|SlR|f4%J-iT(pCabGxVcfCGk0)NW4d8i`}R zX}(Kc5~;}yFp4^xvWG;#ciNQtJZ6l`1AAV8aM3@=x#cil3BJ4I2-Bs)#z_FuHEn$= zO4!ed+s!8F1{RfUa08+LFbBZeb$JJ)c?~dS8ubD@j}ZQdgs*Ou$QVTSsX#TN#Nv8J zdfGi@EZ@|X26hLNyupNpG2To2A~c?2)drp@{bDr6?h0OdQ1d)%%|G*ARzf!Up#5Ms zD0T69!~LjMBDRyX1c;I(tQrC+P|P%qr;)0q_mN9)MM=Te4F7QoNasY9loIvG^~FsW zsP@Hem4t5pl@jud`z^?f7kT4xoR#bR^v(LQ35Wh@{xNf3QH2B`>bSuC}{_(;8 z?bml)y4b3*hr`Bt;;Ddgawsguchv<=*%gxU@C4>`O?VTBeUPIf@Q?|Bdu9@vQ}8;n z(q#zc&>*T^Mp+qoRVL0JT_uU`&S4njt($9Ei{7a}{C)IdVGdvT%ovI^|E}?Y8U5ABOxJCQ>djSi^A(`obHW)_{X8g%5glRNm=8lOxZWBi@1LUhpRO z*SkUM7M5&TA^EC~;)@Q|e=M8-F=qa^3BNqWYYYG5FrFi(ImNM(ra78$qo(aR{okqq>yxotr_%n9&)G z)GdreW-JK;jVuNyo&S$}_7b@ZL4#5XMuVY7K|xZc6GK-fDN-Fx2jVH0&94VeR&EnH0HG?p z1rD|fgi}zpR7M_NKnp2>U`i-KUjS#*%f|*L=&ZxKWK%hwhB1WEBZF%U=@A(nVqq#x zXULb%;M1VcKzA~5V_ur&d;P;$r2gwX(h7J9woO$4fEk%CF2aD?pgDRZk_y!(A{i84 zEV4~*nh{SMqDx++B$*U%PCOA(h18%tx+J0wZH3&REjA4|+hHZgTwQE~rj6MvO_P-K zcxNL@Y?s`uHcCN!KD0d5uc0QZV_7M34Q&Y8=%|ooXb(I4x2Y@LefIYd_6oF@0keL7 zUTd))KvMFpAI1v!A`c;Bo%%vg#5JTQ&An740mw~7H9uX1A7)D^y+QXJih^{QX5jSy zA?uxkBLVuZ(T#0mlZ|bAV>{W{+StZyY-3_`W7{@2*2KofzS-w_>%QN2?|)rWJv~#^ zr+(da`uxr*v**~+u4!+j!;Q8=3s~*5?chO|BX58-s&iIwreDo-M<1IcDR*{eM?0H= zg(9(5((X7no4Z@i$=@OqI zEm9&0sI-%mQOGMcpJdJE#z(R347Tpds(y_(Y^ZnBd%4pd`KdX?Z^C!$(Z_tgK6Zku zofL=j$BiI}qW~BGU}vaNUm(TQWJrBPFr2`PK3%hl3NQd;cJHAFB{#lhGcWMdb`@vgi<|pLk}sw3DG?t=mP34KL9Ix!dNWljUj5W+c47 z4*DGzKBRV4LN;VTGl$U5L57QrFDxR?X|C|As?nTfLOf7zybwX4wn|bYu*d-y*nrM$`>X zNrJH9AfK19$)dh0+0N^b19xLaF>8J6*VN7?4(!!3yhYsll!3;J)V!6%$VuC7*0fd4 zz$ZG6P!h_9OO^()#6`47>x_al->D0oNl*?)I>0SO#70&4nLkw^EDoNp9=WMcON&(q z*XY!_xH-eKuqm7alr%@p79c@CHGe~X$E+bqo_4e|?WR|^gq5t1Oc26H>CLADN@N-v z4Xc?|?fVdM`c^?@1NU?wB2$tw)gNTtIf$IR8A@VsES-ZLj+ViN|2Kn95`w~jG}Ts9 z6L8Eg7SelhsW37)JSw|<7Ql^<4@|}9lIW*XRVECxYCj9~uGvWDX`?FCoVWq1W=skI z94<1S5Y?u>SB@%4&G%_@XGsh>9FFA@jo#8vxeXRqdDsB=f{hpysZ%j0DOIY)MUMu$t_WJ52Qv{7l_I7?1Qj<6(?y$5*%P{VNG!A_{(Cqc9Wa&1QfD_?- zt1MR@oACP~H>-pKJ&gzfVpykC3Un-llM}d8Nk26FJ@96E(%$BFg^_@$!a>*ddOJ0W z@LWV6--BJJcy*f$YJ=L@Wd5iI!*PRef2#@1o~C3w8LIX7?mATO&8twFz$2#|KS;Ik zUQSu@ydQmN+_Py_bzUg)-~8Qm0WU^aS!oPe`)|+d)fJ7U2Gk~0?>W`MFZE)eIi&&G zUGU0UQ~tur1qwbzw|HiQ`-liTG4=I;PTGX!O41M_EUr-t@Q{eN$G8XqkM0P9qDu%7 zLF&DN7SX{h?(eXj5XH6BtK!R!`efQcXUfNKE(H<3%d(mR#*U1>#~MjAfNT@m^FiC7 zS;TlK<8c_w)sh{&%>8GNpJ&crwEJkgetohY$T=Fvfi8*L=NgU>JG8#AFludDBcYD5 zBF&{PbjZISE8I}DFLhSvk$&<; z!F1WJ=Tk?{vZFyRjM_7)z=%fsgzzG6EX)J_Azv|Gu7S!ivp#C5z_%*zUq#+o!p0!k zptks-WD!X0u;F180$eoc@{EL;WNpZ8O;ilYh1*5k}`5i3wM)j-6Hcz*8ny!^Yo277GtH02~`Mz>PxgbOF(mctIuh zz5I5ns!~`qvKnc~9bJv6(VaezzWLi8@A{gCjbTlRWFoNBGkJ=1T??o)EF(JQ^MW?w z{SK8p8HmJ}tQ+r%%(z&fWiYSWSkGf!K2Fd(;mmd`5uH^mF4{4jplrhCo;Hg%8&KC{ z;v~aBG><_CX!DK|2>RPXE>1^M>}-^!6D;^h;Ex_7edX9=OYqJ&I1(f^5)0=jbv#>0 z**5`9h9~PPVIkz)!lMthWmVUjmYrgwkCbe+n;y$D7ijr--J3$AQ6q}R^BT@}Cf31j zu{q4k%+#I=hqgJhZ z#&}#A-9i(9=G3>I%9cqRAyV5F)7F4b@0(=YhpO#vzH63)SMColJgAXfPJQ(an8p)K zncq`@+=j{sK=d)hwWO8~R#xA#;GX)Q+t0wMP84zVKO7~>o1u;zipDP$fOMY%v*LfPiBkOo&?y%-Lq<`0ODY zFRbKS)ro>72fWT0vs>uJnd(`I?p~x@$_ryacG^|s;$xwi_4;l8^ee^tO}($Q&sbeu zip~ya4%Q7|5?l9M)-4Q%lDvEkm{2qXwsXNCe@NQRv!l`Pw6xi7HJ7~RyqZT9Vzb`p z+HF|(T_Oa#E-0Woe!Y|0^}faR!AgfNa_h2i>jW4MpHu`x;~51xCeROL#;)H7gjk2CZeG5fxhpmV#biAUqO=c^FP54NAp0&}a4! zn7=N}gma^Rxv4t=qhmz1dn2d@N2CaVVN^@kheqhilFJPAEGq{?0gYqdkY+QO4AT2u zeEALA%%2G{n*^n7BkD64Qd;}XN4zz&ypG$K(*n`Jb(Wt=^`^{sUsZ#uCQ*vF%^zN=ju-W-PWEg5&GmpJ& zHD&$Y5d6nx7rYySA=@>#lr!BbZFb$#N^8OAWAfzB{HsmVebvt?HUh%`GdCY zh1We{`5I0V1%S5La0i(lGV-6dA#?{`wvp)`i#>+W_0{XAZPQZqsaC;m^*a+WTDr9D zRfNjE;{!}0KDvOi>6s_|hrg##t~!F1>`5?LMVy6F4)vq6r+1hdo<6Xdmg5sSOP*_n zzdlf)IKNNn8l5)jxFiFPu1WRl9GZ1n5+S*^0CEfEoTuqv6;~u={q%4?LLyz4!@AfV z2^7`|4A$u5&)%}wR3yQGt|(&dghd^k3?7S`x^v8`6asCbA9n>sJ{%ZjCePOwb**oj z^RbPs@4g)4Le4h+AHRsTj+(XH=shu)2#u|uYwfU(7*$>3q+Y&IP)!<`X>|y6jieypCP z`*nm?{v_)P295#tvxJ+!8tv{KTI$jzdQ9DurBY~Gv|jK9ezhH=>1alYaJh~!I2^wh z#u!_Mv9Ck%i`44cswMWDUP=)%3cO@!c-oX`-wMt06Y_d{T0UgzseBh%zhLUqd;fBA zQ_j7^)Htt6Z~)4=Vk$n2S6{%rCYW|yBcD|0`XYb-&Jpt#5*daDqpte`e+}(t4#r;d1fHuVzqS}^ z?0&EIi|!H7+9(I_M*=qWy?daWXTo98gU9@JeQrmCNZX^v+xq&lN*fbNG4i7a(8hfD z!;@&@BiNvwnXcdD9ck0WXo{hMKNWCl<@|nS^PUxYD6rfq@ZtU1D;~KnGA|k_wDXzq z9TQ6De$cFY{s6nJd(fx=e-RS05RmX+U}0gu{G%$sQLUR1eXyQU|0@Xe zb^=fPuOP6?wmbw_@!*S(Kt)6BCuMjiFGtAbxB&&!;}4s{JHpK{prD;|UU)QIaRGnTnH&n0P6Bs1TaxFwt>-f8mgK3GpV^$Gq)3Bknc@UKoAX8z zv%My+D21^5EsGd0naSw}hMBeQ3OzRM7R4STmVExoX+Qv9`1o&cTMp$;7I4oPj1lMr zV@ZYkUv^6;k^waHWw{h0Rh`MXD}RIYaU+NA?<8~@MbS$r1-czYASqf1y1QVPd;8?f-&e{mp^z$G2x9z3Tad3< zZV6N;;AG{b&#Y$w8 ziW|%%!Tm#^CHR-GUwZ$4x~Z!jR4=|y6`vzyN+87l2x095 z`Qe~M?M=;0fJA8YbO4=yXnij(qGzDPtl&~=)odsl5%C>Jr52&K6<7x`F{}F+ezY?yUGMN{PID8WF@?h;#GR>%p(w4`xSj3_5 zQJg2P6BN=|?29F)(rne&5J=fSx259D5}+J1H|xpO3Sw~%sr5z1vK5mJVq`R#^*=O* z8%MnRU^WUl>z(DHW}|-{_zNe8u>VI4^Lo+$HM#{2pxsV9mIc@(>krbi9Q{)r&*c`q z!MEzH9T@M6&SAIe##Cr7g~iDgnStmG7ziomm&1wS47K|ap#L0hY9@Un#hOs|Nb&LaK)*DlhzlqT;4E&d1|O7yzEqr;Be<)$|Pd%i<2WhHM2CY z9@$*T?jDJ;mYC(I5zasS{V&M=PYzh%vuKuw!84r0eEGtG{ZEMf6`1wNiwqFAGx-Fc z(|?O)rN)y6;V{~Jh#vd+S;hBUotR)kQj)4hGVxkS71U5$^zU^&bw%XV{WkT8t65sp zBnX4GSTLb=^zGmQe#CnwKczb~pT z-j)DD?^M60O_YTRrzrt5c|9Ry{guK5{6F;LuXWK`tLjoy9HZCl|6bil`yeolcqsS& z;2&>dsy$d^?IPPXgF>qCf!f;Nd_aE`Y_6s)9k#Z)Z;RqG?I_y@(>=UwMt=l*c_fJ> zuJy*j?93Me+rH=pnb&hV-UtJwlf0+F@QkA{W2MSt|jw{ z>1m^Ie3N8eUQ*mKDOd$RmXA@)U1Un}Sn7%oyqe-1Yy#)oS0zJv72^{_C?_3*&hb-@ zZ}DO{voY}C=EUK2+7Ra*6s5W6I+X1~r)9$)R5V1+bB#VbZiE=ml-(Hf{AT7F*F7gW zANkaVHl+LGXOsbOWs1&?6_xwbR?lW2Z08#1qWa^$gnlxDzYAFAyN?p0NH-&3moqMU znCooWe0HAZlnrFQH`_^W;$!2Yh5xR5j)|MqPU!UwTvbgqI?a%hKODn6Yn?PwN&=yZe-SuD1Q`I zIg~iuT90H=Q9yWx%=fk*B{ym%QCm_Thkgf^wpOR0ec}qo{z9^JEunT;Hwx3iCktCT zd+t6-RcC-tPah4sn}(BwtU7G5G!w_w`PkvUT5{n#~;f;$^MuB1y#(nFra27aSt(r4Y~=8&92>SDTzQY=~V zT#cK$L`;}QNuNXR6+>irDT)r`FB@aLd8*AB$Fi8xDb1lI$K&ju>psJ6& z?N--A0PPD+l{-=vwX-p&_SEIL^Z{!@lxB8iT(s?D$sdrc+_kg zNe$rlAE$i}=$7GD^YOj5qirK_wC$g`_mGH@Go2_+9o$lT{rVAC)6H_pV@IZ* z5>*l@F+L%mh#LL$4dE-T-L9_!2!>X0Kyr`EnE66p4%I)~QsdSG10A_aCU$pN_wP4D z3uoE`N6Ee})}uTj5u)DG2!xi>9qY%o2o?Y~P}2vZ;rA}R3pO4$I0h}EoMRI-d4uhq zBdLk-;BW^l(qIL{xS8H@Qwcob(?e{uu;e&rqwc5w=W(gaQKCpUzfEhuevBY7!;Kss zyq@}k!XrS=aAx?`mhs?Tr}*A^cHT@$0cT+5GK+DYj> zFG-%*SK~2wpd`$gxs!|5oo;LOrB?U_$3Y-3qF1Ts_NHg}&joB(KkehEEVLM-NI5n1 zBfJwYWaOExOzt6x3tOzyS!%t^hrI_OYdG52;i8-liH+*#S|rkGs#aknPHPx4Wj~&3Jl zk6`dcbHS_YH|f80%fTdsg?y(*nR!LL zkyMVzIAvEcyEGmCB{caK%%b>HhqK?dsUzzjSv})k^%YZhwGb1AQ!fD~sZN9s&g(}^ zZ#E6e%kOfgfD{RyOX&gkuMcb=rn_|&Fzv^LcgKRT?uo)C9WRN(=`{V(gSHiq*~6us zdpCA)a%7vMVeSd}{FgdZG>aBL{TwC9MVrJ)(YM4&z+#B<--|P3?sYiJNp+8{}QpVrB)Hq zuG8noGUB0Y54BzGwWF#+{3WmU0n1_5l2LU94UHN|r;*U0*k{h3CNCM~m&vP`w=cAT zYL#C=JG#-djXL?P{<(x1E-kGLuPFa65C3~b_&+ZXX!I{~o0e({Fc4q9h{FOwCP+BI z?>SiHz+bbl2*7tc9_Z%RE^rbE$bS=?&yd(1z>t3~;o0x-ZI!(TB0ai$}2T+sLwDCAJC z-)m$I)(B^jf z+q!bCYSl_(Y0(af-Fxc6mkp2f=lQ#vU$^%~>tVLH{zK;FCZ8hUb*A9U!ORNsYdykw z%V;q2`dJTyf=g>Iz^EMqhufhXqPaCLJR_?`@0kyBoo4%mk-K&yxV?rf5xB*$O$5fr zHXH5@)y{-LcLjlia_J5VU8XQ>{v`lT;3M;7`vVB{|R| znZ-^0W&-ZF*SE_}?kwa6f_oK%6if>)ageIX;k|CC_()?0@b3=P&wc2c61(m2;PEr@ zz7K}u1fAoEJZXS+V_xO|0$^MvU@*ykL|}X{zLkUfQQ+%ioGl@DTnqqOYH-)Yr|cnL zfAj$0Jw$slz&Fva_`texu9|+i6Fvq)eC+qML-yl@Lz+bPy?gNEx^D{vY-V3k1q4+* zDjCj?8PXF0A|znO6F5k~#F z7l-KW(vDYNn=*8q5W=;Ah38l_tk&rn+exWo69WE%pBK~GB)+eXu6Ml)igIAC<7DY^ z8^oyne8RvnBqr?-Jg4WtM5~H&7KvBd(G{`ZxA-#3N>bm@_Ai$$Y2s@!xegPeVgwt{HaBdi_^toJs? zH$9>DVh^)8OEAqv+ju_C)VQ14`us8$27@(0 zKrsdpV7-(mDrvx;5GLNAiN4X1J)a(4^-Bn3Uso@h2$pwLNLiDotfMWcA>P0=PzA-C ze_mGMgghuRt4UoO&TPqz4j*LP+XYE*X3@wfeFSQ0eDE^hK8kge+Hpf>Y?@PttHt!~ z6yl$d8;WZoP^@SONxNqrp~n;LkYYcGAd}4p%=WaP@#y-(0W7G|!|ZBKSW1|9xiJx( zNMHtV785y8Z*SfgLJNFPmu(oKy-EB3uJ{W%jX=K^>)9xq`T8aYRc;aDTL*@xY!3y! z38ZQq-}t&UW!1=ScqnaBSErIZz~3s)wOV*#a2|>;vt6uB}L=Km={}WnYhXMD0PWZi({Ri`Vz)L zZOf~mUPJQu9dgVlwrj?#p#_xCzG5o%wX3CfAUF@-sldaLuGs_E|U&hg50x{VeY|g0L0l9&N!q9@uHCSAzlvJJ* zGeE8(_Z6C( zN=vh;re)?EYGkYybH~t=;7iJYDG!BIw8Y!5VaTBn@p$1*RSRAIfBg;VxyN(0SxzIt z%@I<`wh_TB;JIagY4qrV-Iy+!`(>-_Dh{;JRCeNA!-S(GPODec!#N8ATrtdFlP%zh z{{qJI`goiZm^g$hlWzJ{SJ-RWC!NXrh1fBpu1fyiVzBrGm;!IWd2)OJIh^NmVhrdS zKlZ8c()tGtrjnLy$hWfAGl!4_BDFIzbacEb_Q-t7@_j|AXSsSp&BUp-jd-<;hKu{6 zc~N8&wnd$k9X8+Z;x zBZit7aWB`UtOy$rwI=HYe4+A)N-RIy3vE#(%uZ?=zyU#dS_T~$eq-k!!Dh58x3)X8 zw6V6{caTH(mKQoVrp+uh5;vvBM;f2~`MlaA**ZE+{48VeP_ju&8GWf*Z@>tCVn|JD zkv+|>@eulyM6RZm&CcxF2T~NHfEg2t3{5nIFhEPtaa=4BKUH-DkWR}H&>480WN70s z?5d$Jt#OH0wEsHqR#lk4u)AiT&bCKD4(3{!KP%>;vhaF&%udZ;-3i|nOCv$kS8aat zm5-3uH>LkkIhT~4DENn(0+=jO8{;+3>AaT0#zN5pM6V&&Ztl%t=hmQJR3|B$AR|BR zt2Yj_cNEUJAhO^9*z*>HRVj5$RzcT8w%bz3&fAOe6o; z318^%sZXyX0Nh?pK;3DK&hUHgN~m zD9cay3bxlp=%I%GW`Iydb}b@2sr!m9*H=N>*8+2bi?Bo(#>>3dGdPhhkc=~c+T1af zK6cFjb=&XFT=XpLA)&-s}22dUW+a%v>?zPnNu0<^PZo)50h;=P%}4ag;s^*4`Xf-tkdb%P>^V(Z zr#J}d4i9vPQ_4FN69UAyX?V#tGt+f}HnEl}c5{VwaA3b1!Ea~hu|5Zzu@PPR-w61YPx+E`y; zxW$^7;sx~q!mMshbiL1drt+ugb$2vV0Luu$#sj!Bm?knUiA}Q>f7k z*O%SY@GH=Z5$5j5_2aOCdpZI%YnCGy8JrJ_nwE+z+Z0**t_Xshbxk=UyObY+RRmF= zXx+!-RHpO$MPgrLfAtI+n<{%L~ z7Osp?O;8H*akAafwICYwCqH_x|FYk9dxdagHN$yFL)l*Q2(ri?aKGZO>Kkvtuf7p| z)zg)w?UH2F9gaPzbfL8?lvy7DRF7voJ$WEB8uK-RkWYH`*nfS+>{HN=^*&Zvmjv$m zwc!gD64ZpcfrPyfy^m4v=h<+JEUuY5e@Pyx^UK)pe0zh6r3oUY(eluB+>KKSp-O^( z6)Wpa99wxTt3y?!`|2TaJYZA=GfiWC6@Wg(lw|a((hA8$yEPy?mh7PgXvMCkZU(9E z+r7%*^;RbQ7m(E{V zxbh0BS?rYZq6{xMU&ps4T{;VhSqaft1+d?t&;7BF7k%SwT_~rP-H-|KqBARZSE}+N z-Y*l%^>HQkTo7d!7m$MjoJ6XXc1ehz5UAyMNt)r$s>v5vrVGzE+VHB=<<2*-{$8Lh zl_@W>vPI6SV#w86C_z0rzEHRB$-(fU#q5PDBixgc1qpjV>) zY?YFY@+Jg934GjUNlri9ueeA0E;% zP*rzJ2JQ_TfAKQ!?wN5l_p%`RS+qe*=)1umfSk@P)zPU_NnUd5l;t8nc?t{scv6AyoKSf zn|1=msc!SnXyv#;yvDg~Uw`K}m23{~wU8F*w)jb&rNHW&U>9@?J&dIR^F0Sa4oy04 z)u~;4&Bwt0Ld?C4J&&F8xzDp}; ze6o1#bUbx_fWOnk<}AJSaYNiqH0CKC$gPWEamHNgQMJ}I5{E25JRzadh-)*FK1kp* zO}Jq^hfk#~$B9MXA9wO63N()^Ml!P!SVKHt@q~I;+3`%MPn? z2JlPU+AqO1QB2EDgPakvDqM>%DnKvKl3$E$Oed=J0KpeCvgHR{Hm=}#O-F`3VBTxF zC4yhHw-&Qf$GoY5ZARj#En_p(`qXYy*c!UGA71khyYqG4hn{9@l&%R|yk<(f$#w0P zMkk%A^H)6zo4-U2EZN<3W(_ziHn&)*l3TqzE?A)t+UzrTnlfDZ;dXsat1-xJU(1hP zT7F#&0r1;kz}mPbN5Se342}Q(s?VH|Smft;a;~Y@X&sTb;Gw~~!E?3<-#@d($gJF5 zoUl!%nM;p#-d%#mL2}S@q`fz5ty)uQAq!6Vd##yr+HnV|Y8K;o@=1=5tUg{$iD0jF zWkB40IkALfV2@z*!X@DigA|C7J%-a2Rs?xH+u+%iV8m|1u2! zC!_I?dl10;VG#1^G-HGgbd00?3jF56huQqSn7$6~Kh#9d@?!nYC*g4p_s@ui=pTIn z2{er0M+P$2(BlJGMc^2V@-UlSS1HjbV-VpQz(l~I1JIWW9l?=`giLjQmC}3L@UXpV z*b1d8@Fgm~EeQRd*ORW*v%yV_u~>BXe%b6?TN8M>1iX7d7$TcNInvjA_Lv7xigL!9 zR?x=P6n_hJC9)N(Ei;6V^{g8l!Yyv3Qsy^vPZ2PXjNb)}aw=xiJcdLE9oqcuxC)q* z<>X^^GxMR_P5#wj)85of%3O#y&v%<#mPGGATXC-4D$#2AM?V!Rl;31UX4b`RB7O2^ zhG*~VimQKch2E^*_>ICX**pW!4|yhZdF5&iS*qtYS3^!hCdJ*?@R#-9yi^$+^)?J} zVyF6(tm6^@>F>DUXp_q(Xa&{s!;ER!jb>>S(sw%+Gd7~+c+ z-iTZ6{5~VsweYpLRn^LdKic^f7s_XZZ61^iGq;BitX19R9+XfzFj+Kz62i*4F;sHc zgtCgD9Shzg%|E!Ut@ur?2OpD7S}-4XIh}5q63k5lY~Nw>_voUvXIC3qQH2DnM=l(o z)qMgKSJqYH2Y|G8Cc?onp8ny^tP``|tX-#BQ-OYWL35d;WeTy>R&#{Z8Dv2=eXZy-vaL@w5w~1SF-~R!f7mQ4 zhrGD|bkXeK@Lwp3NTpTVRp{Il;ox7JwOy*WsyYs8^^q1lIcU?X;{6elpfUZloS`jg zWRDY5ncuunk3?Rt-Kcv~m$cZq9{tlj)O0COkSgugWuTRGIyS~SEiz?)eaLQc#P*78 zxm=^sGLR=KlDi6$%2MFx^#w#H(m{tz%s!D9KvGTJq5v}F3GD{xn5BQiUHW>AV;0m^ zHZe;1vnc@V`HPCv3FeLlK9Ul9?&BsTDf>&5#8LpYPYlx+fC~gycwnntur4(8F~rDd z*&_|~R00JY`)(pdKn->pVZyakKf5?W)mhxVGBwK>I*fZ_cP8jhIM)UV>*`K)lbA|9 z;MSPn#5{wlDcE`WB97C%;2mM*;Ejj)2{YcVj$&-9PO&&-7-0nR!BOfOha%AA;Unnw zBq`NTvLPJA5-e929#6Tj?1G`;*RSpt3w@!9aT%P=!rIH6k!iW}hR3?wwHv5{vGkL3 zcCj(6+*IS3**a_%5a?K;YGxBRB;Pg+7*4unSTD%vh^6ufE5VGXnC5i5t{|g%0r9Y= z4(LVT)T;YY_G26yefERg^xrQgTxK3gNs{N1Fbg!0aBnbz(?V%_&_9un?UgPy3YjBlQH7O*+61MvlO-08CwGH zv3qm26|W2odzw*(rr)V|TQ|VTY3+z7>bwjRt?NMap@>wg4|dCEbo-Dy6Hb zbyB{U+0(5mVjn@_5@>Wpe&1q`SP$G9rN;(i1Ky9)i$InkMa={G$LL7`w`ioTieW^k zn7yc#j!t6i;YM|muw)v#-{D~yq(_e=O{6UnuApY6t6km^j1ezc;`qpSM8yQR*DVH6 zMBoxy6)9toNzlGZ(DFLbiav9T;sYJh9;<_t>4VEn2VimK8D~M7e8>FftjzJ_fY85a zhaVLNiJXnagMM=26T5kg9~`bFBSCTHB|Tg#G_=jr#Y>-%N!8 zaRY<=Z#cwvpRe}GzjuKH;~xod5*?tSfxjvtQJV#zh+M(`3*NOmSZ1XE9C&Kvs8vj` zU%rH)0ZHD7z5?UKDX^LyY#@HXA}@X}HlLO}hVZ$V%zqaHGOIG60k3i(pqpj!iRi%K z{>@SE33BTMPP;((0ytv3g)WU`}wBXrtS3le>Y;ntLa{kX*e<6?UM1XBR))!rrw1}14!Hh(CAZ@+3y@Q4Bv{T zhQ9LrcyS`#?w-{y@~+KP_>*Lom3gbhCj(v?l?Fvt22(2&7#Gfri^m41KYvEYCIj6l z)MC|?T&|`#HciIK9Ao{%E~HE-QLRTgR3R7#D16 z6!cl}spz?>1wrl)l&yVrf3L(+d=!iHX-1}XL(7lYfYKF+ zt-|p4&L$I7Oqy-K-5-Jp8|pXVj+PvD*kaqumunXm*+w|mnDa52cUJ*lyW!ORjx`dW zjKNbEfLw*DaeSzr5&2s^W}7~wRVIdKwXbpAV7N$ko)zQ{gM-I}syDQ#_^+1zXJHgKMYz?XbuTLZe0?63Q)nFhoo7?9U=+*zaT-2W{1K4@34#k zj`x2)y$^smg@OCG1}~ZU(odkWKO`w|+mq*OGfgB!!2e9Dj)rJP_z%89PjAeGKmWo4 zIM9(9=PS?<6Aq=hqk`S)e~?I2LjeQ%-ye~$VQ)eBPeq;BCENhU%fZP2tTB{P1?r{i zN82o|R~r^%@vX~8DczJ&i80Yc#PXVq3%3(D&-4>2Pwkf1!`|@#fdNIixv>F4GFk26 z1FQVF1_x6US!vEw8!MMWg1%r55J(76y6noM{dI=hDnxEzo;E+7R|ZnT>4NIer8Rz= z6Vl+7{n#7yLv@8K%7VBCXt(5rwb|)VWVXwvKD<*W!sT^g4A_w;d!2d-poz1(?3c&f z`X0Ae*3J{^Ie1?vphBH6g7ok9Zt*Dp+{3LLmPE%|)avd^j8SK^*aX;+`QcZNOU`48^1pB6Pw{GvziSD3sN)@%!{S%^!DSK#e64Mi)CX#(lsNx4?L!*11Qj$6vo(^@?oMx=I$Ci2@1Dob7Yu{wqg9 z2eUb^-O(=i9Vrc-PuX6bQNhoAg_b9SHn>QP(#|RW+n5pwE4W6$51A3QI}HX!EyS#? z>}aa0n+TjoCT68MK)?!}<)n{!k98DOQzT*ME_c7h1`qh3?z~65p?Z}A*p=$&t=Y_& z%x4lOg8Q zE9$x-cd39Da+xf(<~#I{5jbBChjjG>UFS1z&{`kLxF<j94v}s>&G2N68)VS9zRk%qTqU?>7BP4CUP-WcApY@WQ) zZUMx5Ya0Bw>d)z+T?$)qAY!cN;tF5c5PR<2v%nysFM<&)6 z4ZL^x5C9M)Bow+Q5Ld+2rbUZx9)T8MS)MZj zz^22vF}Lc7O1`x9eTCr_k^9rzo?lYHa82zd5rv6tI4lBSjpxjVE1<%k{wbF|0{|J$ z@Tj26QbOMf7Z*z}iq47wbY&X{65g~c%tn|cnB|LfUU6R!+~U+#RyXh%Y?*D-?@akZ zY8urC8keD5c{DAOVeKz%4D1&!2G3n(kkI@{0-#VcP8}2W@#~5lS zEh0+xxTWgIf>FxX_dCMlp?&|l5Y=S*d%Vfnx<4Zja6jVXEZ1%_MG$5C8I}7Tq&_kN zt2B~r#={hj=Bci2Lyb381b}DEg^|jspZHRFwVBQhY$qHW*j;%LxcT(8B@#%v*4e$@ z!vn~iE}@4IMfX4c@mjr>;?7vcM=&xc13#e=)zX;CLbI)gKq-%hAyL^rlVYvpF)SUC zSt-L49erPCnj&%(x45-A{r0V8TTX`Ss^jJZwTUg8KS>7pl=CDB4R9K4IhpxFvh$tQ zRpl}?YchZ&M%b?P<3ot_5Qo>Sgx$7~qber^p8^7}e0^b(S}mg$qU*Ql@@3TE`Lpu+ zw}$n2Q%bB)d(zQ?WS;O2EyRi|{N$y-yuqjH`qKnz5jPUa=t=RQep;?8%H$BKVQ?fi zTY4pHTxW`g1i;V8_{k0Q{wTYw|}w4w@t#ha&yAD; zEzG_yRVd~K`p}D--~a*I;7wT<^(x%Yn7RDFwxkCp`7)Y5_hCQp_3?<3LExQ=MSNwk zmd)phyybg;HvBGqhhvwr!i!JQr3)B&GvyE$@kF(={ycw?e!HJ)XjKP=jcxd7?i1OG zM1n`&Ml{BZWB|sc%nV-&eB?lD>Ro}cjq>Vm@$-^A3q zfLp~^zO-A@D9n{PG^O_z5<9C}!)M9!i5#qVdsHPOWB^sgCf=_{!37em>fe{SrJetr z3CA>Wio)1YjP^f#HQgy zIsr6FKYiLR5THe1+SZgs-l0Q;{k0{j;8uH!V|9as2AW$bKpt+Y%1A* zDxHnn?Pw+XSAV6T*S(uk8kNAf@H#Tk3aH2nA_~_QktI$Ro3T&{V@bZel;~?8pjry{ z0Tm~49fB|RW>!PikERcPv>u~~&y%o8>B8OAwvgB}>Qt0uO$j96;q?yc1ky(6oy~kG zf&i$q6LcF%C{oW}UUgesFM4^(;9F_FydHbE=Tlzu9e$j`?d#Q96S)KH35Rp+){N7w zpf)e5cVUM0HmRWENOiSJQ&K|=w9j}Xvu?(l7HV+rrb?s64{(DJ~wXiK;F_#S)*;+v2EM7 zla6iM>Dbm6vt!#%I<{@ww(a!IzxS!T531I~s#>pWV$3y%xEsGK* zdwB&dn8PO0?9CEv!L5XzS+fW0mc5dsycfqflb$?%)JrGY_n=#YU27%VlhIR^T29pN z6k@5%5-cfA?#k*ZIlV1yqD;u!Q2k-zECE;n(V8~X2Cm6dt99NG&N}Yd=H9`)1*o^| zT{77;g`NUpXP|T@Z8C?(DgdNA&x}7xhiK*T!4Bp4MZtPG6V-G1$2ZO~bMeJi4UqHY z^6_W&O?19f%bbF2q6<7_i|7)fs3Fh9DuSC0>F~-Rk2_urm$@Vj3DJVX>$ivS-lzTD zD8)m1BdzJS#a=s_v!6i)?@(?mx2+)ut?hFwYwQxZd6$@3;#?yxdqoJ|KM=&7HwrKx&2YYV{Q#ugLr8mf zih+qUcqY84cigCV8Gs0ezfr7jlrNsQ$TRb-PlnL7MsQK+X$Qy&F-$hUvc~NqrVVaE z$kwSJKg7>%7LeRvIC}KE$avXj>@ATVLds;>8S-K|jrnm_sdc7XrcRltykaIx>>@DP zX`?wvrcnoWikKSZ4)qf&8AHdJqrBUqyDVM}^j8x6EZ{;3BceZ!zao@Vkvtyj_CChcd zVnYSr&lBg>-(qK>kLwngJSK}NABZNCal|O4*=07hX4h`zU0PbX&Qq$o`>3DNY51O# zDc!ViwO6#xvI9h05mH-4U0bqSW!5X)Sl4jK%_&SghL|L9GKCdtk*VoQ+!R?laYovH zJn#l=JnLDOh{sXk3tM**P-2VnezF_=1?ibbVH{2v5ti>Y=}t%xpW-p&wrg)jMvRrH zH?qjXgmw3{cVS|v=o=Qtl>xgXZC&A*RG2a4F3$#+VE`QFTGsZv)=n=?5OU7}C0|IFxl6OVn0k*!nfs$dQuJu85(&8SgcLbZFm;*IwKP1`BpB1H`yFn*P= z9u?kWmf+1v836Olk=^sNsf?(U^74K~#V&AcOE)fo&PbfNVp7y0?XFl8nO+gez^k;m zF&*tkSOm2D&ql7McFTBGXxH=7ajIt2z5^p4MK(n6%gH_{!C#vVi%&Mi;4{iQq6GQi zp40`okI2zwbdrMxxL=^yele{HNR9gxpD#r#p_u_W6}QmG(LgM!HF|-$dZ9tRvjx3T z#G>d=BAM3fcA~9)|7;f0sLcm$X2JY5>JxWC#sKt9cIi&jmImIpytGHK>(L`q@*hYBCInFDtYt~kE-tb!9wv%_GE)T~)KBkXvN*&x6 zI4zz6SrXv23VH71zJMhF4)AmGT!jcgAazz2$<8Chn{0BScSH};4MAO*!ail+k8IW{ zKv;rFE#a0H6yx}+?#?Vy^u0+@E9MUguT;g1b5L`82S>|9y0)R5TDmbPnR7r}q!Uk= z>dzmu>OE%_semZf?i5a%FbrUjZ&dTWH1bqKF3;r1H|S!?*MeOaAiLS3^nr&I2$=(de_T%VUu$ zzJce=cp16tC9HbmvAJQsVSxg`wFD#>e?j#=sH%$2qep+vAsJ@T1x(N)4HN?WRF3K} zf6<&G#JTIbw)YAYG*%MAq|C8A7;2BP9U0Q9Z*$7I@yf4@qO+S)F{LyaeS}$L$9ysT zGn4ucrv5)j-P{%*D)1Qi2*-1fA56B)NvlFvdXPNgt$C;Z2pZC?Yn4b;7;rPL7XEC;8i z+=K_wz}}*}Wn9vQs7#t4(_OW-zZRR>q-{){U()?K&iJ!F$`G&d#QJM=$Od~##ozEv z0ViplXM;Xc)X;sl2KE_Qc-OumF!|I9mWjy(^A8q!wVQUvhTc58tQG*Ps?lY#qj)tW z6(W*JFmCq&MZRL=w1F;Eoes!^g;F|M^hfa#XQm1UxFPFfjxW`igLnaDl(u)ARl}an z5yzVAO%{=x`$>%M)St+n0=kmkSSY(e`w4rgqvo|g$~G<=y5LQ6R^=6^94U=*v3V#V z%C_3{%^M{uB@nqOypuhbFay_5eiDkPp8l z*t)f=HSQdaj!gc7xb-DhRxlKI&0c9t3=-AeNX{w)*|3US#A=!s)Er3-^&74(g&VT2 z_?o;OD|2Law38Jjd-P*cdoYff*clZAgN$9B7LrVwPLss0DuHAV>y}S>CI8g}Rz?siKt7*jcKP=-XO0jdhP7xm$*r@N#TCv$N)t} zeomQOlfn*9xoAV4C8u&G6EOOo&kSIdXn8aR3( zJb>jH`(*zBFJ_)@uoAe@KlA97b}_(@>W}&uJ7gbp0AIT%3f>$$_)NJ6f#5Osj87Oj z>ru$d&LROwa1Ht%LnV`GEvy4;xgu)F*+=m9)P}nI?;?d~tgTP%y8Gb8hvAxBpF8 z4!Tap%@pm@#}cTnJ2uD5m;Y}5B}M4=p7!}cU&xP;{}KT)$+;sG3IX9D(U)Zl(CZivmVRx1eGn>axNAOVJD_$&$&vH zM&)ARfZAzOR}6;yN}H{JGv`%JP;v25m(7X4kSx`bmAP1WIQ48Rjw7%sZ_-_)jff(j z2v9^-uOnPl1|zBzV3{+iLQ> z*UW9*16;ny^Ze?3O@DDFEv)7gSk|du8w<17j$%&D$XF+iFde0;ps*R<+sv}pxSgTE zEY%%qu?^WTQ?DnbSK`Yri0or+G6uR?vB#MMRIA6!vEVDt0t&G)RbuOVG^?6YyAl{J z1hMi2<}*SSG>pU~Of~=1C5TeUOuHrMo@Q3iazeQ#amiDTZed4WSPn7R&L^!j!AY(C z-1nYXrKn>mE{;Ns9}%^09`s6x$X-l6i&kI9Hjc1(k>oXgovBF+NKo4ml7mxXPoew+ z@HT~as3U+RPb61~<_1n!TpRFZF0N||N3njn0Fh9LK#^iC#$8^>7E5wtWj}(a`}3V4 zxbx1uXIl5O>xpPCMx9CNN+unx3ZJYOlsu`ErRYD2P|~yg|ag4#fJ)$S{|;R z9MJU2FsL`nyvC6`TD(WE(7ET5pP`cs=)7iJHer#xx@_6U zbjXD!9q3HH3S}au%(8ioeWt&(`%Inv2F<;Tpj&^-=};BGsE@8dU2&F-u|`p^M46_` zv2ZHph+!wyqUpz$#d|aB^`o8waMMS*TtwzwTEQ9eEvRHE&AVmlOQnV$TLiyvN6NPt zKtvvHufWmaH}gXo#RsvS6!< zJ7q3UDAhYM*P^LJj4ShBl3bv4Qg77DA2*Gd15ZmI6>m9#4baI)Ph+1$Lv9>f5W7Tn z8EFXT4*sbwwLm5_6>LAmsYo+#6~tE(Js_D?Ow}$@GxDIdMmxL6>@v^BOw}NIVxU=ZFkwm^eg@9sEp=IO=OsJ@HIDV?BY>UFfn;2^tvu~6RIWtsAbf4Bq{vm{8Krx6ej2CPlrg;CTT4k&L$WS# zxg{{{v8>i+lBzS!0)_r5CPha4Re?Sx27Hg6B1gNTO?6SEFhqG3P&hbIR@R1$(e`cU zoho7@g`kg1GX#v$X~K(gDu3So`}dTS7rAezrOeBj(YdON{~9Z~p*U?Gb$;LDVOwr) zw|yLXM6*_L63W3G(u`S={(?i9@qXJCdV1+dbQ1=L zy`+nHSArdXzn*CwAgh*QV{QW%s+psa)d_j}F_SBEgNgbRX4s0~XP-KPF?3yPR}$-d z47-)Sf|Oadb2PCy1PuvO`onl0?|J5e|K3UE968(pY8jQ(`>veuJOV-d`JrR~C|~d4 zua8vm>InW*;cfa6!Mu$w7t(6N5CjTHe5DyAz1Krl^Y_WYl;)@OMyDjaFh=KmP zjcJ$-7#kni@4>`s6==+Yh=T%S$Q@GTB}o{`PiWnd-5xv557d@1&eM&2esbj6)v7sb zfb|}u$oW3G+;R$?Z%`rWuuBCauDwRQNjJF(=OQ~By(E#xApJTrVdiQm;mS#GO){bN zNs$Bwf3Kmo|5mnKi9&e0B1ucWJ0D-{7=^+lV5qq|5NGi~`e*3rDQZ)?Ed`OhsUUc; z5?yxVB;&P2R{f#!MPzGfbphYj+EQ=1v$nCQlWYSSV1t73IA748NfXM{WYX+7tT-Ph zQ=&9${gGm83(|eNgieHios>h;VJptS^Wx-3NBXaGto|W8bwbQKxBz%;%*~I78=Mvl zXL-3D&`yC{Ffe5ab=w1$kIjkcdx72H$$ZE{TpKP3WlabrBK#eyOytU-Ik&iJr=e(`)a!O{7JOWDnh2q|?&f->y_2F7oTO++fFy}786Zx?~7T@be5*@B$rD<$hQo?HS1 z!c6$Xth_)3*(Naj!LKh^=zA6mJ6N=X^%{Zt`9R%5Fus9N?)X0LK%XysuDkN?Bwr!T zyZRx~YW<^}rX2(v4KC+3O}2xLD`w{B05wv~<`GP?I3v2tK2#Wz$%;C#qyd+XlG_%{ zsn_~3GhFesAQ`F*l`#1cqjtjvybEHUv@nYs*M1;BE`JDx(J@$D(L=-+d@smjSc9X$ z*o$|A=F4A*8F?TjKwou4-J@R7L`1M#@TjV~Vlwxh8wQRwR1=*+a5$#x6JxV{fZye2 z^QhyOD(AUcB{r=pM5-&-J)(yreWHmbNEX_>FWOZDfbRuo$HG7fL#`NF6A0FfAH%{C ztDM6{Y+9o09VKuE3Tvo<`m+XMv0dXy3;>U96dV`1_fW!u&-bf7g^N% zO4U69i9D$R)y|CCb*UG*BbbUu110iCt@zj^)F-6>M(oIeq-%Wl-Zj~PBYGF__xTax zcDUP~E-n>=W1T0*RzGrKm5HTeOO!bIkyj{_ov{oOC>3hXj&KVpphLdzoePr8+mF;+ zp)XWpVUVGBmp9si_niS<;-zMVk3RP~^BFlJ+PFBVlY&!w%y+p@sUqxXXRd~JW^sZJ@pn;{4_<}wu ztYJ6c>u1zXkMNc7DsYB2NTGn>Pex*|osCd1w5Pfm3P!$xe74kZUUQtDHlG_%nhOx~ ziw&E7zwVgk3|sM0%9KQ-XXrUfFe}285a#%EY`Sa8gm|= zFLoVSj-N%#M6w)M$2wyakhjz4w^YR^!UOiA8FAbEHkqA46-&Q3XCPTx*sqP9FyMX} zIUTa=Aw?@d9gp8gzI>BSsqZ>LDKY>NNf842O={kC?2W^D1{~#nDbmY|mDTt?ha{#; zGVNcA7Ry$mhP@KLZo84V%EdUeYzBT*hqFM|D^^9fJWPiw@oHPiQ5~OT#_!`gO+BWQ z(S#h7bd}&3-PQJOvt!rnzC16RSI&R`rLzAI!2CZbbk+tc0PNo+D0L^O1n_?WQo>() z&Vw*NexzWh7&<|*rRZHi!liI~LLmWqAgQU(Vh0J$3xP;t{enVr$?L)v`Qyd!444z9 zI)3D><UJOLb!%_9Kxx|8v1T!5_|QNbKf5Qv!yQ`w*pNGW zwBPkZh2OV${}@PSpq|^8C?y8~Yk=%aRg{wugXgQ0zd!dQdw5C*`<@1`8{U9gFV;&h z5l1hcy{pnVW#K)-o40I#kym9GtLzL6uUax(``~K%a32n@9nAX??s!p!4N|78(b!aR z+<|*cYrirD7|v14t>~S|Ce>JP12(FZIW7t6r7ppp{IAbUx_PVh}bgSOx&hK z=_ianqal#Kst!73X=8z!uGhH|&6WXulznjx`;xgBda8qVbV9F&U#nD|ef z9{-}~oz1?+@yR4J&4J(SDT~U)YrKtrCxX6|7jW3L7t=JCI%p;(crd1+5r$ls&n`+% zmd@CDY-jy;uN>Qr#mCA4vcr4%)2f6Dyg|lcXLC=cz)5w6h((Xi}!j2GEr z$}=Bhq{@(t>_Tlil zhgm!^_GmhdD1a6#g594tWtKFX`Iq?chQWMUoM#Lt7H~Z;ZWHE8(V>4`t2t)5lxX zOWWrKPT%7N%y`4UHG2yI0vO1F{lFr?e1ZA0+_Ag6b@Lw4S+{bupO@YZ78dIKH4lXBT z`yw%{_xc4`>EX91b&Wo4z+7SI_F1e|n~%ok^g}SIHezFz9oNQV4TCKCaKgr(N+XDJ z#uN&}J#rFu4;m6Ng{;2862kC3@(?~irb9yX048}mG!w=YLgCC`9>cnAStZH}BaiSF zhD8D6Re@a|gcL@NAc1Aw`ADR0G>%R)H>8V=-7o=!k%-P9k_=LUplkHUOeBlLGHGEe ztZd(EF7WHpnC;Y;vN0kw2Hr35XK3yqz+w6;g0K>lc|C`J@JAER8i&r6C_!iPwunU= zdg#1;SvJd-?(#W_uTl8L4OaSS2&33PLx^2h`^sB>$b|DASVc!P-Zq(;S7P# z+U5hsOX|!{a}{0eQgcos*HUSZAD*1n%3DCgJ;L#n{;V!I+j>pJRV_b!)fBnHq01vI zG6iT&F8;Q18Pf5u%R9-USu}Z*5bvwi!UFp}+JfeSeW7NRjTb(`t_MS7k+ZsFChB2p zfxd+hIh$bXCos~XR6v7#x*S05-zcm znD?W!JSr;|O2LelooS9|4STP6GJZkFt}$(ta}0R72P26?X2nWJX|?rzp&C1$kvmfp z<=t7*mj|z&vVl)8Ylelcj7ERpbWLspG~C?gIJ7qdW9`rUA@_}&K$0e`-6Ix z1phiXCxjQk*WXw8M@!1<1z)WJj4KilDc!cS3VZ4QEk8W$VJNf9q)A)i_)Rxp)X4wa zXTazxHa(>!eeizce19!_PsonLdp-BLhk>!W1)>#r-LQoq>joYCEJ8|(gQOZqEO_YD z+aGQO!Gf%46LTrt?_ISgY1bJIb*lmBURcX;ChKHqgbuTjvypr4YDm!xV@L(yPQ~KN z60L_h7n0M8HB1ldP>i#G;9ibx)jvGS(Tq<~&6Kv8Hm+z#UQB;vuRSAJ7y zn>pdY$(G$y%bO>$cdYnAH)zhElKTV0h?iani9d#UN=}f(tS)FdH#l__mu?ulvga^t zN2*~DT*QIKXeEsEN;H6?dH@BW5sY`zLod}8E`5VfZ<^Ai=AbNDxlA~r%n$L(Ani3r zd%0VNk&poF8WxEvyiE3Ni5JE4(o8UXQ&uqg%!CdW2nOcN z;EKCJKbkc<8NcDKh!6pWzerj|2Fyo^R!Vc(ONE?(C@==vWVmORsr3ht1j<&GcfZrF z%_v#NrkM*qGHXUq)=kbp%OJf43BGp2T6aYJk+YpAoec2p+ne1ow`>Q>QwRON?bv_JvzzD zVFKv)Qms^2Ex~QWBJ~KjJrO$S_LngT(r#CeHDod4L{A>{N@Tynm$*An%9$%w) zCE_|4p}M2;LdrqyY&Ob(#^V_*hyEyoE)VM|Eo{pTXZaft`6c4XgWw9*8EWjnf^{K; z7&G(UY`~3AMiW!(ZY_#}PtmqMGZo|nlWm0U$vtU=LQaET6k{vp)EvIylXR2WQ!>8k z%zWXfKA5>Xv}#SxHJZ;s;(IVvUF1eS$D{1M1_}C1lzDJZ2ooi}yzF(V1%)fZ_VIb- zMK#rxY{UhiNI0~kOs;m;oJyOpI(GDRi#RODSE&z4qD*Ecu7{~VlbQ>%px-BvjyEww z*4uqde3yz8(n_C+b>kTtojX82(>^{IH*d{WVrzd!6=-~UH|L@%RgM7|@L7$!oMOI# zV-^urpJQQRxsZq4w+0sNc_de5VMbzT(CV3_xrYN}x+lO*FkmKVCN!#%^Ox;e^nRhc z2P$^sOz+Y0hBhaa=t#`i4xNrEo5Wtod5ziL`6N+`@nGJnR~Ri6c#EMIyr*?C9O+u7 zwH7_Qd+f1|d3p2R3mx*c-DML?wXXe=h9>YqP z^POlSE(iZnlf;WNZni%Q>Qz7;R4}o;>s9}s_0)eF{r?RNj<&^Wy}sArE8j3c?8lEE z7IqA_rp_ss#&9qxJ5F2#DK#Lx=*_L?WJ&);aDQE9g$4rsN0ieH^?gg@zg{6$S!w@E zZqbg5&<6G|^@hRW+XZO~V>Xm_%9cDRIG~EQA+prH7bZVn!Q8O5j51fqA`jgEhrhYZ zh^t#}Ur{8Y0>rf>kQ5h zei2VceK@T483fBc`I8Gxd{Rb6{y3x zk-ghC%9yy}sHfPc+ERBGT(~2C4Z0lO?6?9P}tJ0*$35LzqBO;a1Ps7t1sWwO>cWWASX8BpXz(KL@DorefG`P>cY+9^jxMgf! zH6RB(=`@XSt;omE!O^(_=#=ocs7O6nuY{~|YAz7iPIXiVRXJj7ElZ30I5BsP&$Cj` z-L8VV_H~>-cskcf_8i|jO9a+kV#zGKVH9aGq`+IeN0~ zTdcUr>qo*V#nR7&TcSMo!sR1l#QI#aO>_ra_Mk1oj%cdk9<{Q`hN~gG$elR40A60k z%AtVL+Xq9cbP6{hndm-?cuk9nEim&E$0(2ol@m9Fk^h{;z1$ot&SS`i^oU|kNFzkY z5Dx5p2b0=T>;gPU75CQEWqliWzv*JfPzaFHcR$3MZIB{BEMFiX{3|{9Q~!Lly{!-25_U-Yw4aG(&5}87h>CHy(_$VX?V&T2#LM4*?4dKrR{s<{ zdqk@<{cY z5k^RXV{7oPbe>~fLg%gT>Dowh?MCEmB+|TD>Yly(#GcAXldaNpd$eLxCHx+4E$UxS z{n*w?t3CO>8EGU}&FWFqu9PyZ)OGVwSI1nEZs&OR^%LIbdXFUnD3b{!)`pyq54Oz5 zTS_)`kQ)1wbc71Enr&%Mq`qLi=rR|SXNM6>&hxaJ>394*e6*1W3p3Q&N9MHz zYE|mS7(HcaP@P%tp3z8t#W`IPdv`@O}s~Ah&cPeC^ZM7!8CJ~xAZ$; zigUv65Ds&~JIEMh7lc%Ep0!x*T1DH<)YSCUhIyf1IeAAy;L6fEcth+lad8?%qfjd7 zUVo8e4K+#M>Ydj*v-Y3%ZsFJ#k|m+`zdWs0d9!S6(9wketlu|-c18b|r{idhH?G{D z74pUk3^C6I_JV7q1Z?G=03P;k0%THT_)IrTv3R0?=bIdQfmgWs?YftBV=>z|Y~%ND zeVg_c|9Qt*os|kU#5`jsnN7FvJSb|>dLy)Kp+G&UV1>4XC>$P=IR_>!^FopQl*#F2 z{iLu@R2Y`1API&Y6+S$LtGSy1S2|pUD;)18y3brvehC{w5&{A27-PrGVDCEMCj89D zYJVV3cP-{Kex#DzFJbNs=Ytj4hovHc&8WddHyi5)5H^N)Qqd+PZ;2!uzG4kc2HA53 zj?W+62UXn)Kt&D-#0}~nT-x2;BaFF6X?FC&b;!D-L3*Y7QaDf>YOES*^rA)#Tr1D9 zNK8iWIp;HFVzrA_O7ktHNTesM-aXLV=+|ENL+ylTU#4v{i(XBWzc3dkG-BmHLs@XG zZRQ{n_5%Ki;r~SH|Bm72s6wc$|C+}Yh5WY}7yWN06$tJZ&NVlT{G?Vx#20uiQO6948; zFF?%2VZCdV0223G+Z`fzb`poM?(Cm7-g(}6^u6S-?gaqvcs-A(nv5qm3jyz_VFF}VRBgEnAJOr>8&dcD7xoQ(y0^83l+YBHM_pe+l%@nBHz;yu(f_wc32<(VsRpE?OTI>7uplr#B_tT8yuj%~Xd%@cqy%pX{EJdrj#_t-f=`yaRvFh4WU$cr(3TPY!$2uZMW zZVwtLhOvCV{^kR`!t*5aL?0}Zw8=uO%H~!2OPbR4Wsib7w;Nb43;Ld#F+G(_;0|9t zn<1f7bK7T3NAxjm9CK&rPU*B{XA`cZ%?T|0_{fXk?1Z;+)+FHLZh1%rIY_6&)UN01 z1CM6jWdp8H^cLyq#O4YDKX*meQG(!sv%{XZLh*-Pw&4Nl_KPqggAfuPEfI#l-SJfm znId9Kbo()1Q@R?8t8GNWk;2});xBh+P=e$=wVygB-;({C-FN8l-6J8k_GImpa7v6C zy?iNrx@lz}!;sbF|3*BCp9lz<_ynm4gqe_*vF73;Vm|i3Ly5%%GC{d|=2G`g?HizbQ*#A3gkPtKz^!>Z?$DM)L15|C4wM9{V(S>YVIy;J3MU~i^7Rk$Y zlarfJ5#bO?ZGPAgGko@0$J4bQ9+7Xg3%-qBM+@|%RDb>8R!&efduSJs;>!TV&Sj2f z{;czQx>$9}_L<^p2YkGq+x@WJ50B>Gm_zDo30cR!I=M0A^@C#DR_*WnaaG}=B+h~V z0EmB03_1N_=IkA$iAS)X@FU)m$qndUV9;6cj-`eXBtuvUdy3KQp=li2)gOCh zFD1@wRR_%94f~L8TSwUxydfh)OP-cjirmeOv04+PPGwc!*A@>K)GSF-v_^}+>v}-i zFYlBLcDJU#uU3>2FJdRQ8Y=B<-SArM>bn3%$8eTMh64&6#o;lbrtnu0d zT46WVrlFmTYShumpF@rVQdOB4w$aBndHL$l8+!YHik57O()@Vu0)ON-B#UI7J%72o`$}LvSQj0k59Ajq zq5f4rh%TFBM(l-S>ln6wRIMtq(YAKtIkjCL>&r6-Q%sT#V925UR%e)E1WUhiYFaBJ zfmraW^plhiN~m0@iV8;TsT`>72?8d5f{#-_MEJYi&1!INf} zqhPDSd54(7<>0vgrae4g>GH?znyE1}z{7#n?*^(|o~}{qut6@XuB%N4h#fG0gK~2o zi~D;^RZh_mk%xy2RGuPBbuiT#Ld7SvcqGhl`3d7TmH9#mnRwyR_ z&=C=N_eNaN^bs=Y=s=y`n|kHDgw*TR{NG zh3NbXb`V?qP%bBMIxl?qUsywthuVaaK)g{+i)|xWIn-pw}H3kES!z#*S#zRmVN`y<$nLiWN)jJ@%&ky*0=Dq2&3Im0Bt^n$saIuf~e2 zKe!YMT5G^QGw9GhEmRMZlNGV>E4Gjz-bWxT-a9Dze*T7-W-lO^^f5?IFG<-u4WVIfUGFw7xO-CpAo)FG>U zWj*%>fCjP~dNaPn9{2ZqGm>fWuTZ|s_>1!M=!cT&l=h2ycfsrTka)l3hl!K% zs)f3GOAhSoJV8OpJHCzcdxPmG-6RH;i1P0DU%IOeEQ|0`)7bIMeboGiAvPm=ypArTV=Y3O8GeHkY7Q_2?u* zhDX9@(P11~cl$NX&$8^#iWr@UuyP^e;o+qJCNKC!QE)3_O-xV7$zDxMjU+pJOboTI z^4RJN)E+u)H2Y%WI8J zsKh$SnPYH7%u=GtC+ze{TySFs#+q=!WtL9MQduU`Jid%G{;&sr*lQ-;7!2geNPq{)Pt5(K)}lHvsqf$Tc*|+}7OFrXdB6 zIBmd7>_6YCEGu8<7ZG7>hs+1f6|4gTY-nK|1Ep^gD^mk`04;cC5AdN5Bp0% zaaW+}i!H3>W)$agaD`)j^FNGZ?$qylvxoUc z*G%#hva9e7Z*m`F9!VbSPNC}d$`ZTCM*R=86)S$+Pv4u8q}O|a|Jb}H56LXt+Gom43M z$b{};rY%M8s^h)4A28O7qd(N)D)2|5SHv=VjLhKf$Yq319yAlsp}84-Q`d|94 zewdcAxPaPCG+!W;~mK;yvL)Q9hQE`y?2R&cL89BhGFLdK7oDQfAieqOdG#kWi3CP}|FllE6tKnAU=)$(j~MHp}%=!pt*9%zd0 zL&%wI?Bg%Ya*A#JCh_Am{Q?>r!J62cf51%_0-slKKnFKdAf%76tU2+ZqUPs0T8Nl- zXSd44nl2enyZzmkg*9CUL18X;t6u`wNh}Bb0dY|c*pz2CYUxhROK&I2PbRGf6Lzc2 zvw-u`3lwj;;K#o~0RQN;Rj(diJTEHhRG*>Ufz|?j2)7y6utTrTRn5h@^x?Ncxv^?^ zmhAPJoI|@fRvE^&)fCfx`2`98XJ>TBzdWhGq0S2{l=EFJ^pgnwsZH2ckAlFbRKa2S z7H!}Kz=+JNQpAwpYh3LHrrWz-o-EX`Q1e^Mt@;-j|DUV8-8ZVc_ET!xoV>%IBPfoc z9hc`P9ke>duq|0qx}=F92uQ{8L8Yi<3{l|a)A)=HJGgvk^`v=oTrVfi}H!?T-F; z=w5D-pwKe2hb)u0P@;oA_$G*^=#V@fEexm`MhHnRE!^y2L=%AD+BO+ z1xcoRj4p2@O%<2U&6|{jvqth{7U4x$u`~?b&VhL>yF|zn_k^XoFp})C5I#qTx!;uj zT;Hsl{$zk;+Zg$k1@el>zl(c-?a92n1J9j#iTae<{3P&Cq5tpY{C^Jinp2|DdH$_R zlS-+-a{RX5`i1guRhqpt0(lBtF*@{jkyl1h|DFjz(t7-4#EexdLueq%RXpceQU~n? zFC*eb~q_g+fA# zJAFCaa=OlT^6hq-?mXV=e7;6#2D%tR<5)5r3H*j#5}x54$L=g+87X|`Lo@)L!;T;( z2KpW1oVZq2dcj*_pd86D#i*%!#$CEb8p*ctP3NtL;GA^=&T@f7?;xgl`sSVpkyOBxXPK;IC zT1p1C`egl`d%790gK*q@e_h#aE#GOK683J67LjAOg+_=`VSA()sJ@TJJli>B|8U+5 znTOVq9*Ho>H?R}KRGX|cJDJVTXsDzh2}~|K9X&aT?`kVMRXYSt69;&SiHom?P?fPb z>4k#t!MebdF-gpeGKK`w9F6Ww9u2WPMch1g#c&ph%9TW=O?4EVZGGNT6^s>&j_~&3 z!U-7)K6GGbQ76_OcScCDu3_0vQ5lTDCwkXv6GrTlaY^-_>N zt4m@zT~GS!QHxPTdzOFs=uVtRj=7dH8HuA#i4`)5&~UQMY?)4jTr735wY!vTQsVA=0ySsAf6PisYG2n>cwihVXjuYR@1oPNLI-ZD7x`k>K91?IsY zi$}2Aqa)ZFGKIQ+xuK_bdQ&9W>VJ1;p+~VBN#`V#Kp3~6Bhn5T?eyLe7SJ}%gSR%{Lg}cvKq#}mJA#j~J=>vFd@Z?FWQR>&LOgg)m`KIq~ zp*Zl&o3>EPAH=q(G}BeP&QulLph!0Uc?i}sn@`#=%F0|e$#ia@&%0I;Z;Owo_P>Bi zqK$g_4C7z%{OY`%8@$z!R*2-|7}uGTPU(3I3fGn+l`DM^A=QN0wCfz1=s%%l*icw! zWS=F_Wce;!YJ@$~l#)Ptq-Dwrx#!PusR_+qRAE`|fV+-WyR>Co?J|KV`(JI(U9Cjf;kk6k4T^XlONP z>m_C#Uu#49eQ*emvb#?)mqO}Jsx{miD9+oWVhYrGcNzOrB7FAv(JhoZa;THmsXCS- z8a=G}czD9?Q0uiI$-)tcPN{LOh8T*wbGEz^tgV27{e+-~&_ZmYOR`bAk*oSqJtCPe zIt5`hM^*G4>SL~Gwh@>d5HXlgvz}cndv|c$9H9R@w9d(z_syKAeLbG|ywB1BQ%Od7 z19f>(K|G@`;|cnJC+Zd%xkkME-$U83dCoucoU+_H-Q*ixIXC6ww*nLk3;@utp@N)1 zeG+yI6l8>I;TJyhciEY=zssQu0Kei;t}a-x9kA(~*)6Mx zXB;rD%5R#s140cBu%|VF^Z+rt$e7WF{(u7G&nQCM`4@PPPHkj`$iU?Q644Z>Q_L6` zb~;XfOoLo!c)Z-x!gTrssu}St_YJbfQo_gcS2ABHv&4thm0|vbX{GZiG{F(veKjAI_ z_4rNMNWW z5uJq4lw>=Rl<0^9TZ)Rck-E8^&jC`c-|3r9^0vZ)+s|F!*C^K$Iegk?W~OGx6I>ow zJs`2a1UcA=e+@I{5(rPo6ABVy3L+9ClE(=XCJPP#hk>AluhCALo+hOW_1uc2u@TPz z(gexOfNdZf$Y!P@hJk25fOMNE zX8Z*ufU_YFP)=m>VEn7t&(ZzlD!hJq*5D`2&&Q@R0e7Ck0 zTvTrwLsy!n90CTsPaRo2g~?lA_kr7nDez}zWOP3ymp2UvXNr*5d}$(LtROv^0yTM( zN(gyBLePCDNQeX?HkVfrV?c)X+yvg)Ds(71lh?@!fX+X zG|l@i>AD;?r^U;SD+MEBpeb7F&VhLH6fKwSRJyX%*f54HRex54Vtp}sa<+nsp475! zaS3lH>hD=2mD7qT7HMjl=~Hh{!0g0cUBcaza&3AkB%e6-E{DC zc>k!W85p?#(ZJ_7xa%zsxNtbhmed_gtZEb$7$U~xm*7H_ufY$uiH5K00-*j-Y3eX% za@UbNXaBCj5$5;5lIv+V=*;;WX?nhzK-qK4P6*boSN=DGnmXLKKh(ipZmVs_$cI*f6yd1jJoQe>7g2R%`r53bo(Ak6a2d0;|Zylc=u;JMpQ zjx70wTHy8ikU&&FHcd}GEFNg%V$mGHW&#ht6>GoLV1ZVcgjy_Zif&~(&jztqBcmFu zW%W`^!4%fk44$nQ01)A~l2gv`WfygrKE~2Y~N2G5Ih9skV&*Q_tl< zX1fIliYh+o;%v{B{whbM@tFf-dXG@xBLEQ#gxV`3J(CVOAGMUJ@*<^21pb6UV ztjaNJ8DW(P%A$6SYBg$cvXEy1gE>!a6$05+p?cHYMA}7)6Z%AO)`5+;T<5eEy7#m} z1~<1MqwiF30l?s9#QT@Jkx1gNNpM*h76X05{=q@3^Rw1cT}VQZAY={)5dvx0Sk<9i zmUd&pvRFqYx5T*Z(g;<}HP!L)1uU{;kd6hxJy2sRIj#d zj07ZRq_Ja}N?dnhn+3GOl79(mIjVZx*#qC!xRr})^?y@>4f;<8$UYdbiMs5k}2?kA>i(afj{wGq3medAv1>5Ud=E`6V z)$3yqfCRAjqi5pb7zXZ!B3Aq@K_=1P9YAapM4qKI4pz`~u2vGIQMh=M2#sJU4X3-ts~O z`)iu@{2GcEUOs4tXY3u!7p@oRC0QuwCErPNb^>sk4O5(@$c%QNEhGE zlRsdkLOUz021zoAuhtk=Vgp8LM^ZBM?~d_=jon#4DJPiVKe@rLG`Z{$k{xa~toNpy zG(Y0~CtjFy#q!xFu2-g6{(nY#zCy#i5~vHVnZ=p>*pZLHgb#s`56ikwjA#ZCIfO;U zmdFZaE0@gBWNp&nK{+8*{;iY^#ZImP_^^PUxhL)3Gdje(EoSM1us&~1Ol3{t<-!Zz zi3x_NSHxrDCmwzYRS}lq>y^d0WHLN`H_)#uDn8>#6r)keM!F{6|!*@d3>cWm69Z8HHbBD(1CL z1^yB7<(RL5TL&Q=4Hmy6U^w6UmsFZCbC7*Bu1+R-3pbqtGj{XX9}4 zbCEl(pI6*CFiEYz-V|5{m~jqmOsS4UWoSAjEHy!^AQUQdxlhbK%|tIRPxlPcx{zGd zYk5ANNtj`_u7U~6@Kkw79-vDH?VESHc6m6r5@`IkI!I+~(d}gqSb6dmO{LUb_G4{w ztIv5}G%GYvE9}hxiMk#}VFsOqi+*bn!hNe$*94G{=B*p+&*s(n2D4dBFtbXMY8o^O8t+V1o46u|+aLqjFiz7&6mT7Marz!$NiGek6F`bU432>h)LgQCmNZ|! z2Z$GW{;Elkcy*;CXsHaB7a45n-JoKs$>;`cI1TBzfHI`=PZCKzU-wkbL+S z3OgwKV!PJ3} zz9&=?61t=2j(N==aHHsseXl2|Oh;7MEKEa}K^oW2TfB0}G)NyyemHl>K9-zayFA`ec?)cv z5=(~xSkC}}7$5GUQ@?TbrU@#-<#WDeAx2beLs|D+j!8%!**G4MODx!Gh$!Fy^9Bd5 z`6mLv;rz`M`Q7%&A9;}vz(@z_H%jzG1G<9Nm&eU*Is>$qtXe!eVw62WpkBzOCe}T% z%IppI9YC^*i%gi2FC6KrUDyu9aze7sEkxpyqI#f0xnWu}M zLY@V)-qCswNG0*ri+56yNz(kcr-jnzP$YuX!~C}`rYZerVRR&58a$o2{GGU8v;esj z29EL623&tpy*o$H&WXZ9NRRX~c9sj1wZ?QRxfv)MKU~xs71rROMC`;CZzP?jJd|Dl z87vQ#v-YkHPw`DtG&JX4;Xt2zre!FHAcQUtnw|0_i#<9go#ElQ>4nzy*_mIgeB*uY}PK3Pu zt$>)*jdapTwL(Of_~0j8*}R=1d{_TZHzz*Ej)Y3Lz;5Y-S<;36j{XhJS>Uz;#79is zH^c@4pkU!Mc&-Hl%ZY&bz}I*HWW?LS9Yp=sMrZ69D>OFEk0CwCE-m2NAst}K|C0T> z%Yg0&LOVbT!$Kh4t3n;Q(+!Jml1>|aXjn$(q+zwBQGdlh(&u1mz9c)Sh`j`e-g!ax zX!h_x_*=~6!4o$tv&BKb{%`MP(~%RonLd)xM~2I|A&BzN7#f&NYk?n?Ix6E~1aNT_ zrNWu#4?WkibxAg&aba)wubHG4@5c=6F`yAWI7f2Ps(~@4hvlkK3zMSXspWI-NE2$^ z9tA%jlQvvNdd_ND$Wqw&?BWmLm+YO>{Rig&T8p)KJYIP8s)j>4$=yuTxD%=^;|K%w zm;v8|qe|hbC-?3ycp!NgKF|ObtxcOY6iy;&I z!hWXoPVnUnKA4t+q16GB8)ya#SVh|QoMWphzHGK zi;B8k+|rutm082QW(2s3Rpo(2uBt|GMY!J6Ycfk@9tOV}&yvy46rcv56Gm4#!)vXu z-D22lb1bfmZp;lNp~ zI@sgj{5^GsM*agORmij*kGfU2wTpG><}Ut@mow`dYSysqF7CzFy2?lacB@4fD*6n$ z0x|MMV@5l){_dn)TjT;YPtS<=ZWk0;&$HSPiF?l&OqRh6%QWA za$qhI*7vdA(ro5Ztf1jKU0mUTF3Tm~c_?ag*Pzrx@BwVNuI;ySYoVDS2)h%zUO{%@ zVMjZCe3pU!Mb02(gYfO~k7nvOU)m2HO~_DFSHn% zV}s!KZSV@00iy6je>p~!0afbmv!J}ni!l?E$TH1lY+5@`>Qa^JWAG(B3xy|!^ z1ee?+vN!iXc$M;$5FXC!oHe?Yj9yd;q~nFp4shUmvfk3bb76O*Xib}O0m&lkjD(vP zfW|?P4-w4Fl%KIv)!G>?Svb9THb5N+! ze=p9iKplVjJnC%HK+EOWhKVQY#RlqB=_8oOE+g(I#!5Qz-^H7L64rfL1u+Wvo%)Vm z5ajiZ3_FjT92~3*CBainqQz!(UMohzu_U=k=HoCXsLA-Gb(J9)pP%3#1 z;{hxQLv!UF>wCM{yhK6?rIlK{GZEPDmt(rCGek$gkM5U)RV^;!)+TQ~sHOyzjL<=y*5N0b6kXRABVbuoaStYB9qzg`E0&ukB z`WmbE=((aPUsyJ{R11wiX*F0^8(pq2FD25MNz~zJ2s9O-X}jaMhc+8O-vXL0 zeEAdjdScBGZmBK|e4)I8^at?|X;zKA>AWJAYw)~byfV56b9M>p%wCtRDZb-7|F9Xh zSMWX+ET(iCAJ#8d=3Ky)BY4eYPj1!4m|bg#UFep>-tsKOrUukji>CR|aRch?1_YlfF^9TPeo*gwuup*+W4@^BwYh_I7Z|sM z@2b3gz9_Nv+k?v1vF^d2WLsulgZxeM#kHTziFk398ahu$gcOeC6i(7pyTW=yMI8w= z9&pWv%vXu$?vl1U2CtCv@zZxIugtNQ>w9J+LA!CGuDzc2H(B)(>kz)I_W=GSxk6_v zUIH5(wIx!1Q1-Y*8@PkSU36{&4U(7eHTv*4pST)n#=|wTG{;M1nO0Y*Vx4x;uB;o~ zkz<{0u{LjHc@;ME)GVU$9kn|56(y#%sEAFfHOq3co^>rZ>}7O;QLPj1<+_9Z8pU?7 zjBy+HGb~^W>*y1cBcSL5BY-_mG4sft!LragCo9CinC1Qu*|?WjMQ3!@l*)bccU+aI zMLP6Qa=0V|JuE3#mAXa6E;$F>t!<9w*8C$4M<*Ml*ccKx9O|Kjh+SF|I1Iza;5YI2 zr)ZH#kre3`U@C>0UFPubUyeBMPVF&&ybLz*xJ)?BJrXH@EQ25Jv2mz2}i`&_EyPopTtK@fYOjEzt- z4I;7lzrkS=4h4oe%>WNVb(o6|6WdI&yi9Lz{38o`SPj6Uw1RSsSR#L0NwBcE+$oau zCG~z|NjoUn23)0~2nL4OV$6JX-~;20CMwlnDXPZJetrPAby9b=llyij)?r6%NVPQr z2dEM?6XNR&pXXCF?No=5k495)sH1#`A%}aP$u>7q`&!9q&@v^b78N)k89w0 z&m8(2XP`0ua#3!f7j)pU9^&QETtkYuQ(%tr^oMuAWDT_oGhKs|bwFeddiAyPf*;?L z>_YpZRvdI1;9Y~r-~HWH-NMk@|J()P|J8LAxZ~zadvP3?J{Zc_8(IrVFok`uSb;_` z*><{BAA}qp)j}Ym}Lw-R!C1&NK0!QwkIiWYJ*Sa(h7&p z);i=?IWl)CbG8gQS4jVv6jWd~S+H&4$0kNLh!mmV6z3v<=F{+~38X>_S*hhEyJ3dZ zOuePYIg}iTSmL&2Kmflfm=`W3FZ;v|*5S@pcvupq0g^)?CA{ncZV&%*7PA(5EM6vGk__K_+p@vjs zjwD$s74CHx7tjd^0<-e6%gajK3$r|lKMsm6WaUiEco|NTJeaR>QRWcZ*9s_ZIpDpC zX`MU8_zqwS~i_oEfqKrG6pIrb-04dz>*H0Hm{*2&QUn(lh^MuLrp}cy~ zlP_2^G1Goi9k6e=>5Ng+A+;a0oZ+ecrya=DFFGv=(;{F_@D;oFyZRrzD}5M0rF29W zJCQpczCazL82#@b!Y@JUz5l$SK1LY)`Mjy9v!ylt^rvur$;QRU8@ake4)3ir34HNz z00XJDyWU(p{@XppcQ}f>6@Thy>-b`;4O4G3SDCvqDi5qysP!h7?jNm5wtJfH0kssb z;!n8U*_TI|E5je~PiWnVpFP*6@2{q|w{#oR*SkPn(Hwj09{An4&Bq8gdLKetxdHy%D@X>IQCICMRPvIvoW)w< zu#_#HsEtI62uPOy69>`(nZ#6iw5wlRm)+;g3B0N=JX?j^T(Z>Kw)HMqUl1>!AOY@W zpUfya0#z-)V3jgid zx(@ni17#I5$mZ#v{UWsA1X9uwcYSL7vV(2J|= z9iN5pv9b+)X%`)p$_!Ya&LniTPYTi7#zj*H2R zV)3;L>iz)}@xpwN8(PT474HT`%*IVXbHmRK?xm9gs);550Y6%N2Y334#J+);-ym;y0tSrRFn?e8)BGrs5PczqP-&Y!>`T%7$}0#fn)>V`lY!kdkX9_kb`5Coa%z}&ewQO%)M58qsgZZs zm;idaE3IGp#{8=*$q}0~m@OLr&{d>V(CvAbqn(@J^@3|nYWOe4$kwdYm#RUiS-KdT zs8O&FkC{bQOj(yM2J4!*7+F=|U$B=`w6C8HUe(4J#%gu0hN=-fmAbIL&z+6?*1gq# zE-`J?f#0Do5&8zX(AwF-9Z6{Qy92B^2>_PS2VhH-z5#3*@=eh0wQM1&8b+_QR=BxF z(Z|PHC@+TRpj`{xidGQ_H=_D`Z@hB`>coRmyJVHuf!5p|1&=;q$-OW)h#trDK5o3Zq61Qm*B_?M@73uez5tzh zu{8u_zv~rjyndt*fmN5Aur05>3ut$hYra!+0qX@Xc~H8r;f_u-IOAc55{pK&tASB3 z<-XZpbV^Dl=eWSAd-L{+Zq+L=AR-6Jl6H9tmoW($WBna=tTAmbHI+rIL!JJH$(xcP z$_{-cmaFcSmLM^9U^!P@<7CNkX8_Q@z|e~Kx)H=Etm?)i9HY<=@IN}gQaQg-|DODt zW#Qok%3FxoLRE?j=)>!4ucbUAV*UC}-EN>wM4@$|;3wF?NE3BrjNQ1VVz$3Z6yZoP z*mpcg%yz3QuPUSWD3#gYUn70?k`ltW+-h2wA&GXsi8J^H;mDLIW^kGpicN zFhlxp7q%X_^=*XM@KmD`J4RGDkiyv9uyJ}+$R_h|wZl*hfQK~`lSH20B;3&F?=SfQ zwp+7o=Lqrv*zDh-4!S|GdUpFdkg`l3*pyb-pp~a_hRKEuW`Yz};LeR>JCx@#n8uSq zNK3SwQ^G$uHyF7Z>K)i}WO{&T2k`i|;bdTY#jAa{3q7xCHiVr9%no12*HGIp-v;_1 z5H&O_BRYf4%E%SPC9;a##-!*PdZYacnue23c=)~ewM67T5>qnX@C*Eqzj?4Za}|7O z(J`+X`BX2SwWcRRj9{*DhyjwhcvGR~a#X4>z#aDn?ZN8$WwtvUB zS`SHbSbyV`hV~Th+ozjwobjzV*=|S(b>-y|$a-hC2KpC`sTnGm}0gX_wfsUlZ}S=4lYC zz+CgK{2QlSK%rEr!*Kyc_f~{~b6l-lgU&Sn++7VmkDrzrf2XKfP0v}v@$jjyIaL)c zV&^>TSTLC5on!=I8I6!`fHAa9KR35AL9JaPg23Fv^nIJVUwTU zOsC~uozH#sc~b``T$sE%FA)U@)Nk@5CsTbWFAdna)+5AxH`oG1(>38PE7)cRwO~Lh zTEPBV`;)+XyEKJpR+T3)x$W(gn#4y8g8LKgEDs~-xKJR1yzOu z*1JNwag6m#JHK?$_q++na1U$q55(H9RwULggCaD zAx%0BFNp(|a*zSe7iNFk&^kt@Q#(qwLN`aIv&m@@*{Y`FQ6oH@lNqW~MQ0NfMkwf| zY(ly!Ld36<)ysSf%}DNd#s)l_X5z8*GvS>@Y+X3k=8?F!>cx~g%-rz8N(eGC`@-Yo zsUxA=f-}>@-ZoU*Ke)-a+CWkPFrXD)gqAkJuvRz^byol}w%1>Ey@BS6e17PQBnq~C z(8t=by@yMQ_J1EyD{5VP$j>cWrai&wN+@tWs`+WPHkOfb=({#pZTM9QTT#PdUH_$d?^N^CtNuTQ@S@9emsJG9VX2zyLv zk*1=Y!h-@ZlpDY+jfzt%#d%JcS-ibkEze6tYVtCnul&M0NeC!ss_#J>0gbWa1ATlf zdoeS9QX3}thxqc#M(d#?!FIN|h~~_tNA_~=&F}@ds-Gs#wiDW?a@nBe5nQXG-5}zT zysDRnGT^#T=iH~kT;D&k0)eg4Y~MN~zp`0PnG^@0+L`ZZw-GSK(gjv%8So{|`^NB- zrW*MBoe*RPqzt3`mU|86KDE3asEY$jz}$OtiVB-Me)qwR$uG!|)0*L4io?&w&QnR& zI#4&SbxK(#gC~}JDyH~2vmF1bn9_!r{DAcB(2)G-FbCxG!<0^jE6|ksfp~eM={afU zt-t~9vP~Zxgo`ZE&Yh|YyQIwj6jg61=tV%norTT(k-^8Ez+vj`7}3l;3pR!?ln6t| zH?OEx9O&{#UurI~>6TD@2YS;G5C)u6G|2ge&Y+rAhghV0dK!;>etWi-#scLe7g(LDfI?ehPf3X10|tGB(mX22Vh`e>h=t-}u3 znKxPbuQ@fFmeV5!au6YnX3EBQfHlT<j!Vr(1kVLwxTMQJBwCYQt~ekt@ESas^}Dg?>QCmx3BAYSIrG3)B6Gx zysMzV)wnS=@Nn}p_r>4W^up*HlTZt+?4;ncynsugfGtR9M-YLvAfa@F0(|>I6p11C zMIq%-4bn)J{?thZml6Zdi=mo^Ia6X_F{*Pi>azv}7ckMHr`L!Y_ZAUDR3Ww6ca8+2 z6+ms!X7q8-!rCHD{wQVwxrL?FXf^->Z1x~q^c(z#MYY|f8UoG4yxfJH{Zhpw+;yD; znnlfC6rBC#Mb%$~odfMe-Cl$q{PacTS4cd2fgDJU3Ysx;yb3TPcoq+bA+;*PI|lW( z9`)R>CgaYJ_zsi3R%xEYQ=j1+n?^;xZZ8btckEk&A_i#=;rzDIN<1N5;cNw6reD_n zmNK|7>4uD~Xd-#_nBaVa_MCcALZ*E9Q>gnp-c0yv;ZQYNxVaC1+SZii7IPfSo|2Rs z|7DeR>Xxi+1{2#?K5a7A@d03WkwcQ1F{Y}WOj9i}4=5zomaLBPh$C}Q6 zCiM{{3MR^Fhh7@4zG!>*<^%5lE#-|=7(pdDP#Q-gl~LkW(T^#SkDpfXiUajxn8j6#jWTGcKEKP(`G;nJSM$s(%(d zOYbrr+sKgn=FrYj`|F4gZsXu{VC0=J`tTFX;$0#6=Tq#70(&(}kCcP-IlE+g6nJ(4 zQ&n%rIwM%Z7X^FK3xg5;Mhmd~mJ-O4h*K z_p)HQ`NMhqw>AvC?g^~U6}3Rdq}{xnKFanL+#><}gg(zC{6kkc`9J(-(8C7f8G)b<0!jd_74hwc_K zZxop@v#6z8hmq$y+;v#GfyfTp{#+vwfWbVvp02XgIl`qQTt!Wv=`;X z*WBo}OTMnxovnQif;u+>{HRr`7=3vx51`2x)Qe3CqJO9W3#lzHp_HsdC0v;EW1!U) zVMu_1B^j9>nxU!zUa8bk{+tk`VAP5ig? z;mIf(_bf18|3{r57g;x{=b_S*^hkRNO<0dT zVRMqQ8PGP!w=gnB6&+Zk0W$l57F5~1@f}nLFc}o}J$NTHm5B0P9S67!{_&ZMR{WIk zL-V5zKhw$R8tICl=0Rvy{gnRWJyTX~wf<^DxDRr`l8}xH3)xbA%}NjR09{3GOL8MJ zj;5Id&N&Txt`k9=>bj%9t(2bN%EV&{KeWTqiZYX>OPV zCrYeiu6xwK$YGC^mEO$oe;yFGumtouheiW({J(jJBM0caBs)e@!%|<^_Dp;O-2F{t`l{e$XD zpaY2m980LTm8wKt_2@)hk{{-}RK9g7;rf*>dmQI#x9yK%D@K=y04;_vpCv29?#E=# zKcAGnYb3#q@={ja4Z(Ci#A0uXY+#~Qu%1VN4OZUW*+f6sl_Ogw-!Lx^8V8tso4#QJ z6A4Q*`7nXWwzH8@#_a*|AA{PJ497ej3)VSrd$va&zmmjwXe;#-mHqV-mHkz$!}Ck| z$^sQm#|axQk`+Z{RO`0Wl-3#%Fdda-bHvi9Oxd$_B+{Tw9kwy((qSC*cu0Q55&_Uj zEz33eC&ZnxhSNBoOiaPxzvM6y7{$#%>FJO-B(Oj)#!jKH+$mfkN95_L+x@G2UJ94O zX@y*gGUUF7BwQp^#I*o!#xy?Xpq8&h#u3qk1-gK@MO5j+aN1nny4a*Y7omO-HqCzW|lldH_o0_POnb?}F**7LZGZ#Tc zgFZvPGLZI7J|YM+p)V84I$RxC(#GYFhBzCd>xY}y<_ymzF$xsK(>5_m#~RDRefG?y zde!!VUWQs#nG{(xV~u5Gp*?d+ec`GKlAWkEpj?2^SR{+BwI0I1Fw+Ix842*Ybh$zF zDHm2M4?f7d)U%?V5(5Xt0wTi^3bBTbHX%12x8@W+5|l`XVOj-!8!2YgU=K8?1B*(e z%a?2lKwiZ*9bsqGZVG;_~InP&!P7}2{y ze!8!^xb&Gf#FZKpwh`Y7fCZQ<;W)@dKKPe1;jf{aJJo|##olt7^zYx;E${yb-?(=j zW93|1bi*!}4yen!wuhJ@>dj*JVCEbPu6!;CkHN| zneK3Lxr5E`Igw{s@88iyu-hunlEpdePUjwRi*lM5*7lU1&8S5jV+k1HkBsB!?(#{| z38%#?i)4o}oqv{rH_BawG8(?D?D7d&g;+NmOJWBRO8!2Wm#Js0|Bz&h`te@@xkMenXh9kXT z*_b?0#YTzT-9BPcD?&o)B{rWeClBbaaS%2`G zQ6dNWcx?VpYZffWC~=F3od5Bmin}U$|9c%Me|*tRBQJ<=B8x^FtXSdw7%_{%*EpfM*&O*KzVifpAcI0c6 zmxay67$M-f{@!&N#@duOd;Dxjjd|Le&6zU1LKC67i zl<5u#Olro~7EKGAhn5(WAvMu_SFQO>P0IQ2nnCwZbVYy$LG|%eoO=E*+Q2v+_|`jc za|XL0TSu~o#B%!|MfI5B50pdfG^*u34@W$e(Z?1Ho1V@SbdA_ooWn}J7I&PJ0!@Y> z*bGS`la9^jf(<@(nmO($_@ng6-kqxFqXF#?uE zyc|8#;uD^U75=cMEyQSyts{P;{uT!CTTggxC%HaMcSQajPXt-ABHi{L3|6ut{843r zj?Z8tbf36gv#ZSA!Lo%Tcd(kd`J;d`VM61%-zb~?I6zMPZb4SgIWK**)9|!OQi_XyJ$%HMy1hkia_l?E`#r>6Z-M>MM14};@n1#`b9Pi&)Qc47bzU7{m~Dp? zZsre)oKh~+rs3(;`euMk!_rk{o@+vJVj?PoGdS1pz_s7l1NUm)?f5-gI0?GB4Xy#$ zaxMHmRp%_S?fik_A_rv8Oi1tY^9=J=^W2<2tp~+yrim7K#>Gm! zsfQ@VE@3$YOHb%!^V*V`OB#Xzh?SGAkp}oZxUz-UJgM^pufNZO4XRj6LzP!%T|CgA zcOCdHCjxkl>+M!r%cpp<=OT~z_I>Are<~eBGD=4=IoKVOX!lF_muBu~B$1$edmRh{;1C; zcIw98>dnc8*#V!@M$getbWmvC#(b+}c*hId-Vd$`@hg&+Un?03?{x60nJT`M1gV*$ z`!i85nJ+*}K!B{r+!8+l@bvJs+T8h_6{}(Q5vJLd=Z|d7<4dE+&F|xF^q(#?Q;l{%O z&v9m?d#ud+F?sQE_^aLM%%{Q7K3FaeYp&+%XhMqKn>-?rMR-1d%o|t-3jDlgLD4N3 zj~4BQxaVDiH{TO70>7<0B1}4hklQ}l-Pc-JE#`mTNg)&9EzNf^+T2# zyxaae4_=LKKZrBk8i}qE!L9+v!|{ql@8H3Ma_bWB?C1o$n%O&9*8=azZ0c>(!8<&5 zk*@KvLoe%?4?j9U5)jE-Y+4LyG%KaV9Q^-d~Crx*6 z;8XJj%`4u!qVKeCyg)6;of&?dz-`z^GS3~ymgLp$Q%;`M&q= z7Xi)w2P)Wy%%FSgUWYJLm*8*Te5DU1lY6VA*zJ-J?__Ae4IK4|ovJ~XYSE3E3N=7a z?6sUa`dSU^!|3qlT7`s0gsmx~M5erctnQ5vKku$$Uj?OHgROB))#+Skv9C?8u|chz zq(ZhVyHT^Txv3;(Nu(_CF=(vWwQOo>ye#su_{_HRui?_aved^&?zuPJM|7D-BqcmL z@3QW&5+hN7GMiw7LZB#yOT|e!c;x;fqGDM{IG1G!C&oQ`dAwmbHd#s;aIFp=!%}JO z9zC0$wKB@U_!(%eb~b630!onz#dur>DNebvaSn9?-6HC$ylnDt88VlUY$BRvTUNDP zDn_Z2M>@B%Ub*R5<|6HDUItyRg4w8UnNC;n(MT>}LFYA;TU*mI=dK)#x&XaQ{LFh% zPB)Lc6h>7*wJ3xnCoiRhWns{=SgQ=ZRQh&XsZmEaf2oXS(dLZ$F{(|xv#3fnm-tJI zPFbMz+@>mJohpoF0lK|a%i_3P)s?{|Q8&-FNqWJuU8b`3xTNt3%`&@FMs=N|{P9Vv zv3CoQXIpEh`dxRe)V!v>DDIWnw&|;4Q|DH~bAhpt{bY4U|BmO<{Mof;4XA8d=`Qi| z_HA5T*0H&7RDFJHE4%E#3UaJzk6R76CY%Aox zZ7b!zZY$@$e^++h0P?%;0M%WO*NWRH-j+W%?i_i*^A2 z+OulG_%$3#wRaG+ACi01B#3Ggnuh-=TN5$CFE1Ew5i-`re~u9x6LS#uiVUXZc--Jcqt5JCSSzl(Z726+2x|79Y8>7Uf~ zg?@0m-h!r+%~=GN$h)s-3P-xxtXT(iW&&}XahT(t=Zkvnxj(mRDh&L|qB0Z6Xen=t zI9i*IEBN8dtbwJLWrX3c9V01Lib-5XMV6UWjLVE0wtCMu5I5zUF3^pqI0mH`;ZkXv zBZzBk1ar?RzyWU^yD=Sl$}t|cUH`ZH5zSjRC(-Yfv>J{YP&4F z#b0vA^0GlXw4*2Z?YA~~Aw<=T{33Qdf>;MU#k_LkP@}weSLrkF_$RxhE$$uB{f^;p zeH@^Px0DUC_z)P+*rtwv^lx>#t6)vdI$8OTS&^{*23>73<)M64k)X19?r8sS5XZT8 zsqex{_V1EPfI^_nBHdz;>6<(@%tzpmCqe(nqiY&g3q7+|d_RXoQJXWN-Q6~VT+?W{ zX*|aBwS=U+$9GQ8o8q;AMMbX?M0{2MwgRXn!3VoW+#;2v^|n1ziVE#6s(|tDdgM^e zwYBdamn64>L%1(lgHKq7gjNhW>qNTsi(n+(1wG6JKx8=kYI?ZzUG~=}Zh?cR_bURV zH|DHixRT8&)T5i;!LGP<#+F;lgwJ#P)9XVkf^ckjYpZ)z^1pIs#TGozr>9eezHp{Y zd?M*#dU0iZ@mUgMZ2@65?mLvqH~dQ{$2;iLTUMXb#I1&Clh-EU>f|_^*;NidAI>_Z z?weXXKukqG#T@EGT04DLaZde~E%2BvxNkdI;c8`G<0f|7noVBpCVa@sWq!}qFELjM z%Eo(xlufLoHC+S$O}MFMFX7}(82ZN8J41v}L4oB~IemBf0 z_kf5hnqdF(zE|1mYdE)yuH<$}w&#`D3q!*-+li#a1i8(5MU#zm9A$7I+ zS%KKt?kgdjpL{saPK7`TkPAYUCnANz9sp>u0pp}z5&tczfq*v~DXp)n?2+qPJcDO8 zz|4Toj5KnRkUL1BR_z1!a_CcmtMBaJyaiGJ@Y*Q3luXM8_&G`fmZd&e3Lh|;zdKbO zF>)dEC;Xges0T6;KC;m~rE)cS;KWziTW@lzjJEN)HCB%6oCO2Nq}h)%Vbq+z(Jc2E zPPvzzB~|9!Xg>|4SdWlf8!uIcw+pdv0mnS}-q$-!ZAKB>-Xlzd7#xh`_a&T*zjdKn z)%rMo1Ca`Ck|*3jSp5WH96f_M4n!SH%JK)GTurD;jpoqWxgl@?&0u4fkD zgTxHnX6eb`|H_*8^jr@YCIfayiu~jADdjjqY)oGvb_+W*IHZ)Ef>GkbBSm`_$n zAQ6ERDWU=XNLk;i7yTjhr#N#X6tLx7CDrdQpnxXU5)a+Nnr3%OeV)&O*4Ln~kFlqM zGJ(o0_7j99tLRq1q88WMkb#fpUUcT_a#sT46A>~kczZ41%Oz#QVMf&vCeV{YTi-QT zZ?Nye*T-&^LN5a%Rgp+kC^Cbxl=X6fxwqm{g>m3((!2J>*)!73J7FYS29Tvjb-RY~ z?;)_xNPM6x&(mh|VCCI}JBl zz0q_=Y0F5@7N!*FNh|*3^yi$(`9`=RTI&2rVNC*3x)**w>M8z_pP4e)hc`qB&f|kE zc(QM3jtwQbsGXU_hY-2E1u$Y)53iya6!{HxuUXVAa8w3F2o1Q zFJG-IJN;~$FzO&K#4dindp||^0KL!XO?1^W7z5T+H<}oB47`Bk0nH4%t4~T>16@hY z%?$f|7vmlR@6h~xT?x4n9?F;Hue9&jzA-*@-6LHw)eY6h<(G#YYhC`^w%a4uwdyyQ zPf}lgoe=_-)kh3#RxdnHT3_j%F#tPtZHp&tnq@tk#u}>%?{~HG)Xv#6qitKbdJavB zbvKRq3!X}>|3%hY0L2w-Tf@Nym*DR1?(P~~f(CaB8f=0M8Z5v70fK9A4Fq=|B)Gc= zcln8Y@80+RswryMIo*8@J$tR)HfvuQO)CR^qqLoz>sad>-p)Tg8uv|S(XIuqO z`(H^-FI;s^!(UNNUtgJ1T8Obu92QC!v`BRy7WRx^)tF_qXhzq4#i}{l=`p;wbPaus z)H7{kv^$*X*11PLO>Z&y5p&}|{M;R2@LOGhn#Mh7_ZB(02&jsMz~oAtRXc=gum zfwHxi#2>5pWLO@4Z?Z?KQk5)ks0VlUE5Ce=b2gFxr_Yjfb=r2VJa3(hJ>U5GC~WvC z0CB#Znj%%tOBsZFfk^~CR28fdvA%aG3I+v!uzFwEl{EF-Y#r9Eu=$mn(6%v;2&k)W zP;Ekt*OZ-Oyed6m+l3e275`jY%Qlyi*S7kIe(;k+GWA3$b((rg1u;1f+$W@NDBpUW0>RaJ!9YT8iivwSA0B-SnW>v zK3cdFUSJ`Ay;=xAaFK9ld&l6w##CU^Qv!0-q1@Jq^<1WqQ=(Ej45cFx2F}_!S4tqc zPXt>YzgrM6#g+!b6ElCR2`05}y=8}4y2gUqZ9ZpW_%@@SQFML`=e?|09EklHD~tGX zWcu|R#Uo^QL4C**wpZbpi`N~oS0S5vNSmzZ4WRnymkaogG^PsJZnRIw-|j zcF$K-3+&^XC2gF3f^n*S1_tq58P8h$3S+r~@}7mST%cQ(x|h2_S-9!l30v`lqVS9kl%LAmqzZ!O!I^ZH>pF`A zzW|*L`?4{{*lgT1xXHR0q)z-sW1;(pBC!e?2AZQ8RP?rO=e$eM2ypvXWtH$$OF=Kx ztc@MH-1AUb;Nn3`&j$lZA@jkXqld{Mo>uml^QmPnb?4V7aSH|jgX5>F@Pb3mG65+_ zicz0t?d`R$Ceo z$>rj%x>c%2?;8qk7hu~cwl~b>nsdP`1a5-$o#eKpnc*~s!LxM2N^$bZhD65cjRl~>SlpZ8>i{sPL|YzQF<-ug&co0#wq z&Y<7j-*vICe9UWO$#D*@0rQ{8w59Eg4gXKivhb4S%qfGu;95>JbS^)Mv z;^u1`$t;FID^b`H#9DGlf;bbj9EOcb7HVKU?EGsR)hw967+AsA{dAC2aXe@vM0YAk zhB!L32x2$F1*HphU@pQ1EhMac9QF=#S1wB|(43O{018NmHsulr>a*k5>!Kh;`{|Mb z>f#~JUXw?av*p+C+Tdn}xI}~c?D=)NTnS@lT!KJ!Hr_cPI$M7AuFpK_d20-WnI&ts zgm0!R)w(EA3vKwdyUb9PxYG;P+)-6$Dz&<}xYM&uQ$f~t{F+^F2}h?-m{Es#zCkKA zx&#PEe*#Y&P z-C6knBRhyCz{m-L12D3MXakHKAv^#hdx!^sg=^O*IGM;jSb~qRwlgaUP{z4S8|=hu zQzv1MqHhlo3@$;{cZL9hoj7()06+2FOC^Yas7)hT3#juJ5E8(WEyM`03_36km0<_pblmIOb5GFv29mF2c;shZA9P#d|1TS;$ItDLu>>33xa{+h# zgGJeQrGlRvA#wnJ2M8U&-wt98@OOgX0sL(tx&VJi2p_=T9^wV?cZQ$?9&I41fJX-i z8{p9n;tY6nf{+G3vG1w|KXKYrNZg~$J42v?*TatkSFQZp&xs)rV*Xq`@0f)#amZKa zjxr`#nB9%NaCizdDp6eMXLQovI%WS+PWf+{mp_6o4L&?IeS|C-!zzY2?c;pH@3HWQGwxByg!~U+*z>kU74E^_NKpXl9G}`a0_y^_WAula; z1He!?c=Qd@o=BxIXv8mfNYpftyP;maN`wJtKSPm#Pr{Lq8$c&40O&ur=dLh4f4_n^ z;@?CHFSilFNTqFB^}NW|@W6pPTG0Nqg9wgG5<+bFHi5na z4g3394d9>X%+RR6&3lwUf8m#0DmmbfSggW89mn4)?g7a^4r!U48A1Wb4*tcah17^rj95cGT+RO0RL8=P+E5Ib zTtz5{cFV7|h_@XW#8VZJv#JSq03A@U>IO#yjcegB=uL0hXz||a`++6r1A}!;NOy0< zX+}H;>H&Stk>H}BegcN?s7j5E?gs_Qwstt3QUW<;$(4XNK^het){{jL)h5y5G?e{G zLpD=vjZ;5lpMF-ZpL|5f+Q2d>GTn174C1sfDaE?~p2elUXJbceBMh@puI{4)WG^v} zphjYBu~$6MIi9lYW=cJR-uv*+d#L zbaG2uRChH{a&iR^2!5XzYZ;L265I-}gMBdR6irJbB}Om-rq+r6X= zEG~D}=o*jRV7RRZnzL(Ebt>v*(*C@qVi)_5ty|?lh>q zC*{nkHJ-GdbZe8e%V8sUBkSi5g0Q*Igf$Grj1#pp-;SBXHn1ytr!ND;-snCMH%53t zp(C;;gu#9hMOVQ@`XJ8^MDmrV!j>3}O3PMF%yHL;>w8-rw1JMzc4mJ^(0S(o^IBv{ zTW8$m)w~>)a)PVQ zKx5Y4LtREhnTA&SWW>yaYq;M5WVrNvS6pRWFH61<`leV66RV;Gtag2|!tMg(b$+hq({G5v0h~WxJa*@m}}4$ z)?h7maNP}Am|@}ArH4}9reiJdVFZi~T|E8Kjr<>0LHut%{zE;goOn9Wu=>AqF^#B` zIm7=o@#bjJq5hiWG^`l#fAp}fm(|jvUpP+}9(=lkLjwMeEJy+@(^l6dlEHqWMRd^P z6%Pb3%ofzzVP+}W8!%*dvK(BY?4CRnZ9%IEzzeKila5qL%o}h_{*a$ZZ9l1hceVtC1J^`j+xre8{o3 zZ_y1q*!7cJy)lloWg16HN#FJk1$MRc2`An)J1Q3&&AroO9BRJ`)quZZy*5$}sh47p zX`#3dT|jEcrdq+BCr!rZ#OcX!uJ;y^r3{en`Q%0L^79oV;jmZ!YjX!>m&V8eA}2|+ zy7}cMJo9temGc&oDs|YYBtW%DOf-!>SIifEC z=Ir{#jZF@x1$~cckCk^?{60dyKa$}u`7VFV!qlBz?fpy$G(g$T@RyrPLU{lM@_@1< zzug+jP-FuSpMG5m#fO#efuBXR*7A^2lgN=sswh0Pt?p~bY5XCbu+s$FP2^uP>FgGp zai()b)=J-G;}A5=-b9m zLi!GRjZwaWcLkxE_zmn)GBDRHexT|=7G(4jNaqglCTMM<&6gDA+#hW>%@N)pXMDA; z{94Z$ih#DMfa;d+{B9KRp@r3Nf27Cndjol&(}v!sT@5Q+>4y_dgcG>AjXnuRZOtTMO(DCl$H1B|D5B6KnHd(ij_*G<`KG_BK9C9^4d)8N3bpVw7Uz4E6=+9iFzj_ zkU;k9=P9ROn0Z#`BxQu|CH!rW?vwX+f|o?!fIWLtW0jaUihXZ-s`_l<9tkGsUoRug z>1;IW8uSUd7P{c>w41TW^~iV66Njo!%&8l8`-WVIzAdYtAr{RN8R0^`g0BW_X@zfa zVsohKLnB|vIEQen=tI(`1hBT0Pvufp%s;m1LG-bx*aR;LBU+BlQ1{^Df2Nq$@ zLiJsI3){X09+s(8)Vl<|+=p#1N_+4K|H0~iBfjPZ<{8j!#x*Y}=0o@`dWV;S`^zthKQx&)yEL-cotZ7+FFx!?{!;uj3&uugvEAnU%I@#8u9Ejg)pFyT)>m z*H0HIRK7|6h)|T?lgvN^Tf?qdYIV$Z| zv0!*Wr+u9a?s*HfOVep6(Blbt-$bxuL-!CGEX$om5_P;kQ9USxyH3f>!58VUXAIgxXv zAA1I0{yaE;f64!Fldl5u{%JpvpHVbd{sZgCL7|PJo>=A880KifiDTU{(?*FQ%C=fc zg~s>ZA2Am)Hoi_j7xwryUelNeW6x_~%K}@cUVUEBx9TUc9C2}=!k*6M8Q<$tVH2&g zkh4TZSu~|bv51yj;9wa$nw%q+PhCEtTuD%>kL38!9?x2$l$XQOl)M4gFH;H_+Dtq^n9X6;r^I zS0+R^pbZ0Ve%NEq;UA5n#Fkp&yHZ9Zmk&ThrB65O(mX;4H2%=|{0wF<%F>ug8!ODBJ^d=So};Xi2)p8)nDV-4 zB1=mL&CV*RFy9NQSEc)x@D=pC1IEc>WASnunjKk;&uF4w!5tSi9RzAaOR%Y8e z#ZJZ94?SJdW-Vo2=*@dGbUOS1@idG2s6$PE+)x_;fMSrMXVfexLn;%evVGJ%b;K+q z2@p4Z9Y5^~IIpErThLfVFZ--dB4%NaG*tau}&Zn#4Q{F1wA43*4CfinV;42$9lENm`|%9NiB z7?{`HV%YcxQRZ_Tl^Sdi$3ByGjXRoJMuQF%7ucqiOMNCi2t7cWmXiYu6W3oiKc!`1 zFqzELD$lv@L*=!zUpO7InCi0Zc}BAy-{IzIUK}+!3)c5l4MTLR{jU+bR%LH}t60uZ zS8C6caxxL8fygA@WB08}m)(zPdzJ+RocGwgBz@O@x>M&kdrnH%KKH3t5o2lfdt)5J zBOt`xxunccFV>>-{cb|3Z@XGFT^-f7LtDM*UnY!jzTlRjB$7z!*?02%bhGCu7_`Pm zS68wE4XPIJ8=w%pv1pz^ZT7yG*#j%M zzpcF}V$Be~*R$rIlOoe<1UXk`uNnBs!4qr~b8#_lh{W6zl77Uu?v&<8M;sT6Q2}+4 zbPupkANVZb#=|YOcraXRcg7Wj*MGP1Qa>V_KAx-dfUT3AJM(cOjU_8NtH&? zg4G5rh<3M@P>GApA3sr0keWMdO+My!Fi?qj7ku&D9l>#DD=giX?*I6>ZwI}(=LyJU zwONgA5MDBol-&hl(@8`YIG0`!(}@-Aq`kYdzX^dY)@H$e$6vp*e3fhLw<%m z2jrK59&=4Qd{?=EKS~Y*yYj*xUu*CSy%@W=BE#d^Ha*)3k}tsToLyFNFo!gOrtdhf za0f1TwJhO_5soqC&2M1#Jp4JY3CRvu-@!6|;FcG~jPr+z`O5zBYeYL+;y#~Y&l_?1 z6_q-1bA%IBP#Jkpeu+y!%UZ;!jDcNt{_-`!#M^gSb%4^7@F)F}#ujx2|2C7ry*iTjs zYRN2>)f~Dc zT7lr&Qklad#>2PY6Lkq_%nTNpye6ogZw>fX8Z+=S$DTMvcs|us9@*Jk)r)x2YImAh zxAD}=&O#GeTcy0L^CuU&kyECn6LOc)8T*a>m`JVPXH#}wrA^6!eQTD!bvKkZtTBed zV0*bK7rA=;ydVSG1eUa|6#EX-feYTt1eIi76~p=Dv4f39K{VBhox&3 zeYA9{Bt!81;$-X;Ufmpon}mH24s9U)qXXmnC4Rl1u{n94WcW@{+~zQ&wrd&@U75}5 zH}UHa3`UPj2Cf&Yqix~PPt+u^(r{|YbF;LE=Ut>EgnTT8HBRtVL%bO`QiBMF2ffSR zp`D}ngWd9D5Gy8u(-}q{Ew!pidCmNMj<7MB`AcqZDM4CuELuW=rd+Kh&l=4R=~%%I zQmZMmcDW+KTjq~1BMkIydErRrg9*AV8fU#7COhJxyi4d>sQEW9-~9A9=|=GZKk zH$kRNd15M>@XZDf?b!4ne}{3JW5nq|5FT@Hyi;t{>ovg#o?4eJ0ca^Q1@3mpEje>` zpeClSvYkFnPd!|`p|z>a4gMiydzCQlk=J~li&Oca2N&_+4L9Hi)(T8~4r+=u9ac%W z#D;m0PR_#|Z5|Ib$SWQnC$jWIXIM1o2LkS6n2TXQQKRm`UMe^M#b@BCN}c*K>w6Bd z2;amhzB5==qm^15jay^uzfs{YT<kx@OF=O{DG~oj@e2wCRKL(DB^EOB^I1peurc{bM3C^IpQB{WddAI#>p=U%djc zf=}_0slah>Sy3BA*?IS|QGeg2Tj$l=e`(GKtcr##3^Y>3QpXT!s9maOt|gyRmClwo z$jisUU?F3~E@pTKosgxf^gUZCdC98TaaHU#3sF8%dgT%4ZZ(SS31(w)30hLYp=xCL zWvskCJeN9me|cRv_G-ergZ&$!LWaS#akW{oe)T&WLN>jUY&y55x*t_4U*Pf$4Yd^G z-Kl8;Ie;iz4M`;lQ`r7Yd)^krQJ>;!8jJNn0^ggGQ%BiDMun7IxZJ1VzmBD0 z#3a@rreQvyh}HpyFk*UHSHbb)NU0Q){AWs|bA5(*5ejAG*;B!D-)gT-!;9hmeSn=mddY%jb_n$s;qZd_WFOvOOB zN;{h;vzk{@I?dfBUt8Qin-)#Fc-0l!jKZQ`^-Bi$ic~sgFFHIn*HkU7-539Y>LY#E z7us2iH1f>5VW?IA0UxB)g4d09*(bXl+EbmKg960+0jCQua6(kP* zh!&c9jb2nuEtMmaCzek%A8L>tmm)Z^ba9bD=P#D$7~d9Uq3XB$%Kuafr8^eGF!ooj zGa1LrRu#Pb54tei;R!;|^!RRsclOYUrZMiRn|p7QGd~_Zq6YXCa@jev>=7%h9pnPJ zBhc~veS?0zyRpM}^S}+So-_(__lOL_^1E^ zl$_my@OKX1QDpjkhg~z#`T*YyH|Dt46|@xl>a;%|x6l}NJn+pO{OW@t&z$yWQ^PKg zif8oSaQYwr|C3c11qK%B6#@dnt3M#}dX4-9`v)Z^xunp)Q4$3{%^@QPL;A?Dz$W*Rtd48U-nFUgepgyhM19apEc2Was{@O zTdsAdq}W<6$hb3_qte!c?b&~)56e8ya2^xzZF{_ZfD;F>5hO%rPBhvc=Mfg<6bv?% zoK+X^>NA=oeagR7nI9Rt+z&F0vdXbV1V)?I)-v#IB{JG4B_74svEWLNZ-N~5doKp< zc)^wkUp&eu~H`YUZb0_*TKY(s8EqP!~*6a~jI| z1F>(txTr^C$ti{*lYoswxMFlLKJDmh*C00uu1_S0U;@w>9^)F4)b)Tjpn;wlC{=zT z=8-$lFZ8OifZ+pf@u>n}G0eRWLR6;_uidA&(-6o*#>vY!K;BWlQ(7HqriHJP4$%1^^58C0KTJx^1k-h@thU3SjKgrmYs1dwSyzXL=p zCm1LH)Sgq@^hl}*4?X_;xi@2eO`<5z#;>ZXZuC=f4_4IVE+=gAB}jeS4H&tl0~sta z)~y|c<}1AeNL3cuF7}tcZy7ykR1rD-sHnuroF;w8)l_ZNNPreKPPL>m_UZ$tFkPAC z6Kld_#JC6w?UgH^>qm&b8D4O6c#VJb5yfpFv0z*wkC_lvsB8NKEPWUAo@R^b4amLXW2xgMk0p}SZ$ z#mXCqR9}kv^10G2HFm4@4a%nKWc|_$Wi@?udT{`U@LVvsFiYZXszA0~762PA7hel| zNf|eL!TE2zAWkJ)+rLDm9-v{ke5CQ~igW)$uwvT)?026p2N&~~S-SCBJP^P$doN9 z*$`27OCJiEd>W;JtAqE1ya$+P@@M3&GRC4T_S&WbvII5c?d8pM`o4q%?Wm7`#CY8r zBM?ZsxBi+n9-6)W5~x{e&@IfX+$AK4Th$#XO5XCNBlBQEnm%2bK+x%3YL|b%+ox=5 z%*FvXUHPDZ4oypd)RtxSO-fjTe40^(P}%fwXD4)*;TekCR8`Cm08RrwK2TV=RmB0n z_q!gd^_cLydX0xOJFZR8U z{0Xg?gGV5+SfFhJ4xu;9Hj2t=h*(8)n(+C=evE*CcakmY^@n-jiQ=%-bC4k=lC9IAuUzjX8; zbpQWG!|xn;{hL%sz> zSq=+U-(h{%km&)}{FhQ@?(PZ?URH@j2*EDN7@}Zy4?3I%$_Qj+#J>%nWMFLlYuIrf zPZRnN56RqZ9QM5=E0ux*kH=u}gNFg+Sm06+-TB`--HUjq|LTN%;&uDSa!c!Y6#w3< zpaNO*AK^V#B6CCkc}35>x_q3MaS0l@3Jrh*CO3dX2m1+&U^VFE@Us1V?$5)3=bM)z zO7P$;Szhc0HCA%5e-9dtqWgdUHCqf_3moEa4&^UXY_9}EQ1Rt~TRK_fjCfE-a82Qn z;>dR5n4e`7B7dB$SCb63dnP+H;0MJ0x(Vu)27VM9^!8kk%?q_jOXcz{U-24M20q#SP+RRuh}|RXs?}FgZtRW=BfiB6BGbhF4mW8dxW<>6KXg z5aP$J|5|Fx*jeraOu#zwK2v_tbC0ZWFll03A+E}#x-gls3&-qr9?Q=TW}iS|B?@`1 z@jNUmg_ljmmUJthOnY`2!VDCTlhSU1HZ^^i!r(cNNtYe!89d83Fmi-U9J(dlw4UsD%F0ot9X|=(kE1KU2l9XX z_QlWrUJl>)BgsAZ%|B!?}s!Sn4c${G_SAu%D576eGHpl0z@Fofyo zp#A8mZ$ravtU8=v2|kw72$1`FhU~2}4{sBHr(Y~q99gpbRwo4^PSSor$nDo*%g!Q~ z*DsUjhbQ=t3J>kNBPAmi}|5K=Fs zs>|g4(z>S+P-p2is6x_oR|JoPcEw!YpFSwk3e*^43aOB`w~>^2`9X@JZ9r+G#-QUC zF4K_GxtqaUUEN{tw4G%u=U6*0K^j+q<6RAkqMf%$1aGZ<_y*zH-9mv}V<;0;AwfdD zH%&unAOftEDx;Vo0vyr?P|8{VgYtW>TD}mb_do*~w$oW2M^J&5s93t{I+fqIZ1l&8 zF`(l5i@1gYW|@3FIY^ZNm(_IF@jR_)FUDZT3uVQXS@l^x$G0gDynuFymy_&a)Jm41 z;mXvB>T#po<%q(OSdpG_KU!`ZLau7KgHxeTNz^oIwlwadTky*sDF3<4r*#Nb%*Q+> zJIQYJRKzLeNz_O;lfpDTZK?RYiIe$>j)T$jC*m&d%4exUT6q~eWgGlHYN@T~qH%En zy1rV_JtvUTTu)u9-odTH%=h6t!WTEL2)5HC01g)+_gprN>18H6G#GZg21Fsuwlmv$ zpqK2T?WZ!L=cZ!_JkPg{Xe#1RE=2kT&x2M2au)OyoVXb7OMkQen$$i;CRt*GVi3}O z!`}tF)K250_ol^3Bhqh^K90puF{vy4TVQJ|1ExJQru~$QA%~}_!6*uVmxGgKP7YD{ zyPu>{baBvpK{0KKM8=xWzVZi}mHe5>vmmgVO^+saaRIs|&;vBdMMx##u4%M$naevU zPzu|F** zX?&jheRxa*ur?xFd2BNRD`PjOP7?U(P1Z1Zd0xCYpHF`>Vh}q8`}~T>BLbfw7JsMs zHXcf=><)*MvwN@W2F6#itVet=DyB;ibRtI4H8sA;DruP`e%_-9w6nNVmdafBUBqJy z?Xo5g)eUdnosqmHb525T`WYWaju)}H;W~-g{wv2mU?b2>*c(fB@L{A{-jZp!jB+}| znR+>^8(mfAE>&0~Hq2oUVQ;1y+i?RUmCCP%`;5*frV>(jLzHZ;7SOW_C75qcV2?7X*P~B;KvD}?V>TP4qD~}kN`k&iq;z5!l4oE_(b72;X9y+!thStOAoF9# zS%VS8I}&Gy)mm5=<;hV(DRKQRl#%)g?l*V*W(d@m2?y2x+7$X74X5X@*1#O;aALrN zH&_zbPaxd-G8x711Nn(x=A$aZ@wjQEw=lRAX=(3X zi6)*N)`Jug?W*1GM^oKKuSe#{#M%Pj*~7w6KJ29_+W4DI*Saeg*PdW|4XcD5R3DJA zMp0!eO76`Se}IWLt@Cb={l4d-&0zvOBtbC1+n3j#L0nQHL=FjSray3M~<^|+n@1_W8S}Ks*qmDUhZ&`N4Ih3Vln@cbj z;}P6(mTe!tA=br+%|SIn$&LmdOM9Ul$#aHWwtnMO#(S1>8pJ=di_GTx+)@NgzaXez z)93L88twWw)SmU?%6XKPbD(?`c#Eq3g&2fSiIWW`HqPJodFsdd7|Jgjccu`{iQ?Ml z>R8@We-;|I81KAiDjfS&n!Oj|aadxkLl#J%^c}8csKe#$Yu~r4~ed#-Y593zJ(&%Jw89mmsM9 zLt3^v47{iT;h~PL=@J>ZCa&62%~MootWtkz${1#^mb<~E;ykxkz-}ONy=i+~)apCW z7;Y(NV^fYpUm0f4%Hf8+A|l%=eMw5bl|4MnV@gEGivNXf!}xBuwj|Jb@|ZQlpE`L6 zC`)+_Cqf4ORUmC+QwsMYCw5whThK3rWpXh#P&ULX%2QGJ*ojPpF@pvp4hV zh@Qf~c|`7A_xyN<3+S3+5AH}Sc#XvQX*`Yc07W@c5ckk3_z+8F?mhx}Rx`TNTE1#W zzJ}7{r=aUese`ZOyORih368EtJQswufJ#|*Rr?gbkMMzP$c#{we@{mI@za;+xBueB z|2z6Ml(%A>{~JT*cVGnn8(9wE;LrXY&Az!hng_g8{R9J6>p&&}ulCWQfhmX%yLFDcbAfkiMb_ostMNRGXLx6BQU`oU8VZ-+(=p5r)wS z#n^{GQ4g=e!VZa3i$4_bwX)tOt~mN7)|v@D+aUr{$#(b}w&v@Ljbw`~wi)aK@hNe1eOUA+XL19>jtTESn^ zyQqZ621@iy$8|Zz2f7tJM|F0SupTqFDN3Mj^-mP%v!6Th*04YpRTq(_w%hf^f>h3G zIEGj05W5k( zDARkfUZY;KGC&S&B3P-4z!psd{Xr10*HWYQ+ zHW)Z!O;fvl;k-Y3-@{QQI)GhVoa2*Dp^GQ#9JB2Hq4Jj}jpepr_CZSl9^aO%+$ZQ* zT^Z^3F~1aesP9op(X?VazTJuf(HSR5&~Dk=B;~Tk=?2b1ccpW1vIE^hyuvfc6_{36 zky$eXxI{N53U`~{2ifZ%z|`V#7*V99jce!Z5g1XGjM+rk&=!!zQx3o}o1wvdGyCL{ z@f33(g09v}njH+@#h|T};GL)!IMZIl7;Jrb|h+_96Z+%QoqV;P@(DY%ruu01=!vPlp3E z(417mczR=t$g*Xot#&|Tr#)W+2dkwoAp@~%>15A!w)J7|h+|h6Y)3yEfzLSQ%;mcK z7P}uD^MuYmqnQnhWl6`noj9))3Km|)Sh{ymS}q#Jr;(@T)51F8w|hqWX*IA1S5}0cAN+HZOwJZaLeXM~~-2oMKgH zdnAhb5LftPd!(7k)?(3pTpv|uoycO>QOVkNz98|@u3CS%z6>vRi>x8$NDnQr0B!`5 zP9RCRm(J8O4B<&Ij@fvH){D`R&3|P^ABhpe9+H7GV%1;YWmBa3NRx@OsS=Z5!VYSo z8li+O^q*x*Nw?sy_cxyMFxI|c(y6QOMYRsl+&YM{BE*!%x0w8t%tYH@+2y{;Iir%x zC{*xcefvmayTUIjCZFKv58YtkBuAbrD6wMZ5C`PGaK;n)x2!Pn?1bdIJmBKCF`0*hav}v5Mturv zFGsl1I`r^io-BS+R{o*iH4N5Qj7tPRyl}714gWGI^B>vRA^1j9C7^ZS6ti96N%~vY z<~c1K|F>1#`QOfII3NurI%x)_qFn{z3kC^kLXMo z`H5EfQqBVT{6bed4k@FGwa10tt6ZHo zLG-P6C{B~MNOU&XANcy^$f;{m!Tgul{=eb^QuA?&ZNomg;qxj_($2lhltZ>=?uH+{4HH1Oz zYsk5iXyEeE_Px9~p36A=;5hTQ*V&dty}7ujri3pG;l6H_#LSH!l~gbjs7U^alj^Z!pIHm{P?ba zBhK0zrcu6^60Fc*6J`_v@Gn9D0$5}P4hw@yW|7`$5J6JAGXLJ4a6S?IZUv442>&I7 z$}v=Zc_nK2K)QRnJJoH_dwj+!p{QeuK_F)Po3t0>=ns#Dj91b&Bb$bi(rFQmq$65~ zFo}pqY?riLvN4!AS&FlF9*Yvo61xve@*f4WJxe}aS%;jB9Km))Lm#scz1AFV$(^eE-7XZ9 zqh9pO_4oHlt$Y))y~v4GI*Uk^qlO*O8?tMmNUGwGsK2}T|8)~4>q}4mpP)`8&PM(( zsI%U{J^mw7wmZ1%zpFl)-wi0GUb+|y8mz8@LI(a=FASf20*3&0k`#FjhCk+I2Lkhg z8Y6;`NV|uIzohcrlh&v@qsP9guaca7JrUbD%@>d(vkT9OgqiQ1_b3(6k)ZaKK>&-x zM0d?mcnG7*DLcX(Nh+mvei0reH zGrRg(+9%0YIr_sW`CS;vnoFi!5g_%Z{!w-q=C;u+y)S(yOQ&SgyMu^u#O6n!VMq9a zkpS)PwPGra0HUKYF?-pv5cXI6mFkzZ*rCS7d8%S zl_g7r-YV;}90?Jw24zp7>mqmF<4PBMMr2ehuqbU&Hkt*!f1olMJ(ac1m`G(A9(I{- zm*MEunT?e)=WGYD&I;7;e*aJ3{oihPA)I}M_7apn!-E4~BjbWke!c8&tSF+OHbD4M zxc;rG9p3WN{QEwNmN0bwejk|4FKNSGwoG23gDWdub|ElU;E=(D)NDjxVGdzxV42Q> zCeaMZ6B;H$cMuj9T?|`Qm5~8L^kKKBk(7%=yEJSFD(9z+W!$}wGxE)Yc6f)pj+2Hy z3iB6dkT;zEyPKh4;Y8bv5UYYc1M54oi|olTW77H5H*Avxu^Wj2Z9ioZP>Mx z?64Qy@r@DM6b#rCzg$ZBGE+WU!5JaJ;O-pmYgKai-D=dJhAyY_vP8hT0I5LaGvT%4 zLOSCv{yq+xb9i0reM;wYbLB5wKBPX;(E;;tro+~QQyfAMj}-hl$WV!j2oUKq^KB*z z)1nw=KSqklNnOS5F=W1OIbNu4e=paPn=6?>OKlH<3MI54BFb8Ba_(^TBQraD)oE0! zvLF}rq@WLrZ$TlBpuO-^9JffJ&6b9BP+RTM9s!FX8GYJJb)G?X#h5*q8k^9EW#Q}& zyNTET*u5{WGkzouo?~|}1;|~A9NYc6hg9*0PpM2oxg)N|4srqCQDYC3(S&CvIY%uM zetjkxOOjh)B$8vqOHc4tlEJUfhNzT?#ghy9Y#4~Zw-nKrUyqx9V1 z5nB@4$WEz^&A(>sBjln1@oo=#|a86;@%iW4lyfQmG!=Dn_6QzdmKKg@yl^+9d#26 z)GK#~bj2ONF)x}Cv9PfyFx!dY*j@6KUT?D0yj=>kkMw)Lh+v?qaHz<&OMh=qCzF2t z-rE?X)9gyIDgsKbMoMXV7xV8W9n6B3@eEdaTMh_i=YYdV{+Lex?H{%u=5QL;PcjG9 z9_4h_DP<0t0wN&6Gw++I**OwXg^_g^ZMx2~|KU+!onAENe*ehtzS>w{G%wX1j6V^MWN-pCHa8+zxQydDh z-nNn((cmY$6We^}tWo|g_aJ%Wsc3Z~^ z>tR7p1OFLDpkeWSLdLIm&Syr%KLVxCM5x(Xg1h+Q6g`*Q$2ZG~yd5~&eM84ul1TVL z8}Hw`Q7dZ$*)hpHypuLagyB!%r#2ccZhfoos2-mXk8ZdeRkLb3#am=*ee|FBZ{6~8 zE}e6)QKj*yT7(3Ehbpz_Y*isONNti`HZv9Dl^>JKy_}kCp zo0z@(mHdqM-9*OiSe(%=jVXwHhbGot#k+%OxlJ3ulY8W$oQVzmt`F%J+%Zz0D?aU{ zw!WlL;TGuSA{OUjVLf@k!|!lacDXR)k9@zv)DMVXdn%qboBJ5 z>`9^)NErxJF9cHc)NM2>pf#|EqxWy}iulWm4K?ObIy7GPmjce36pS0fZ=vRvXxq>S z-Vsy)C$@R!AL4k$G(LG7@vJRe_`r^^G~9Z45DA&vG=>%oailv0B*a?EauR-KV|kNx z!%pa&(A+7b4EpXC^z4y<6W>ts{q&t>ej1U-4R>ZiMtZHpXt_R2v=W7^?#J(hT}9uf zY_#SW=8ejag>jFgm0-aqZ$s-4PK7Y)&j%KQfjU9#1KI=QIuf&!PcDy`!+?j{$OMb2 zn|hyQ9c2nU@&?K=b!vQDWv(kGQO~fXJCadviICRm%9awl2ZwG?lkf{d>l(73QPR1% z#YL6gc+|Jtfr25lNvqA)Pjr{mI#;p?`6J}STc58riw4*)aY8?dKX)mmL`}cbYKm6Z z2Cg7*>vqU<$4;$gwtET0ygxkcQ!R|!80$Rt)K8P&!EDtFhV6Ga z{!i%qzu~x{;DRpY-|2y}9DLb-0r&?dA2HOQnP<8p1d9yN^z*>N>^e-y=7vfj{ zGv)5-z5-ymBwh~Bt+}8)L!tdPbw68eP`;dFV8;Y^$Pwa!D_ez8!RW3ih~UKv0c2p0 zj_>b-;Lk>wvlx_&l%YAoA$H*%^r9s_`V3jJF!U1Gb+I&t?>G+Et%7e#>R$9@ZNwy= z9fe5#Kd!zhIFL5lcH)U`O*FA>+qP{_oQ`eVwmq?JPHdYK-1+NP-E-wKJT6|gaIBhm-PU?+MGZS(7XD=fT*Hm3GWpaAf(<(7N7-t2xATf*W zhTx2M3oMH;J;&&{8uWk5l`4ZCYlG1ZcHR2$;Khq zB?nwfx(rr_HugbHX-$=&NVf^dzCTOQomI(YEg&z0eFt;uOM({4MzO$=Jx^hyA8Ae! z?Ji6FydpG|WCEq3+YNF#Ad=F#zovn5mY038b(v9E8~`yRV(Kpm9H9U;0t_C7 z&$uS@q%ocob*k@7KoJlN4X`;-6rO;p zclItJn5F<6lmr_?j_Yef@FNh1++Evd`a1_dFHt$=B96VLB;i?A<9g-SaY`>*klhz7 zGZ$huLX*y=rlH!a-yoJyb3W)#t@(Ft?}o{#^#3()2lPr`j5?J!Gn3cHrZpL@A2|~T z?%r8Czc#gdujBF2IKFJbr)4`j*eW&; zDwWMr8kXEA96!U(#)z5|@2_09obMVM!@RN)+@VA*4i1VpUAM0~?A*e82a(jzt7+)wf?Sc zZh;MShV<9gY46~&Bq)&AMeq)6R+ic84R6_heP(NV*r>U-Oc}H6LFfC5RONIW*ToxM zG(utJci?f)**oLadA0A!t59LR2hRTupyv|_7@LeQw|!zhBNqxC$jMPp#omlAq4H#tZA0{aia{s&P0H-h~KHT*YU>}-hNzA>_{a>V5C z1`4qmBTdPP{Mq2iikj$(p{K=HuWyiYyyhBvXT0BtP%B z?k!TdGfLx!j+w8w+-88U#y`&2K7{liPp8ZxO{&0P7K_}Mq8!dgzz}^lQqNxZm&+`W ztW5dA9{FW>YmL!kc#9GFWqJ$HeC7_>Yrfa$#3A(}iPA)5yVO(cmYT|G(M!FK^%X>k za>9~F!eGEeO2r|mzW?iyFyiE2;t-()tKJdI7G7jl17pY0tTL&O7_ti080fLa-vj;B z9Rc=)Fl1@yj2syU^!gih&d|<>OcCnyWmvdRt>%>Iw1&h=S4k`I_nbmkFNzT6hA(NW zoFO8sX=+Ry(IahXYRnwrBds-cM)o{0E@`;1di(YuF(x!@%oWG>Y-z?d{)`r+1y2xh z{VQHegcN~gzA~E|FoKHArukLx(AX9}WCvgnVEm?e!~2(en|IJjU|g?SO6HhZRRv@Vd9wjluD5+j9Cp(UMSn z9!{*t283#CxuYv_h1{?7eeh<}bAmSBe6psNO|O?yZe0+p2%M;MC|3Jb?&9EhT6O@I z>T-6#)4MaTdMYx=8XHYz9vyq3cd9V7xqI-S3nUO9`7L^`Fv1fbZz3AePLTLE?%q4WhP^mFE2~PB;e(nJjDO4y8>>XF5xn>)Mr&ufa1QUR+GTqZDi>l%G{2; z5EpWS9IoTh$5)!~;Bei^pKUli-x6tm<8p(pNYhN1heNM`LSqgd>DJNK!Zwgw3r^E= zmxZ8+B}m|@*vRZ_!6<*vDZx?EwS?B6_hvS?eH1y0+%r=vn>FU6MP7I#nd1P|psKS($(@s3~51lO&^;5=oo4z3hx;(8Le*e>TkdlS@AN3+yE9Q1;Pa*Zd5He z=xwB#+XCeA-|kA}EIt2b0&*S7EHYQ$uY!>&4l2#cmbb^(F?i3Zc&2~DG zp2Q2nRyvVS6?0%0if;KfIHiEIKMZ??cq<$Fl7f0OqhO_XIKy|#Ku?(w9LL~+_%8U@mj zV#S(ieszsn?+lpKl=86=G2?iXYjY>BVs8CD?DA=OHRtKePfFfD=z_o?gsQB$3)a)r zIb10c9&6UzudO!SEW-5%e3pI+ZxB)oQsnC;VWKU8^MLs)Prc7GBJ1j*x8^eJpmj+} zb9k#fec0G^$D?3N6~Wfis!4X$`ly%=Xp%Y{D+?_*nRQ*%sjGeBC>VCW8mLYos#N-# zl~Z0Ro@!0gCG(Y44Hba9G=8g4WFe#MBzrE53q!?NEl4@zu<(8c-g6Q)TqgSNS2&4m zE5Sbg!nna}X7zjoVI(-B#cfMgB?IYDg^Br(4F51FJ0b3rExMQy#|uS|MyiBxGOrW{K6lpzBq?fTXC0PF|AlTqcwHKZ<0PE>LMEzhI+zrTWE1Zz|-oiy9$x=NC`u>&k zObv)U;h0vqcZ;8vAT264VDA&oDJ8MA=QKmrQbGQUea3>XdV&_&=pvF&KOe~vG`d$X zTls+gC)BCLPpEc|9o;j%e!A|(xI;Xj-*}Nfk6(p?$R)H*78&Y9PY|=@20FM2n)P6$ zSxR~<9|EwwQW}trcQ|oC?erDGwZ>T*CQ+;&ix638!m*C9LiL2Zw>HKbj1!ijBBmim zU<@V+IIxO=b*TyaIl?f1DT8-$U3yLR@a-c|2yy-#hIeq!@GVAY{ zK;Jv1yLJ|iKikAnN@Q0L!(1cFwCnhkK&Sl3mJs+pf}-rLyfWOSthLytQ4(i+Ase*E zl`D|xaA_kIV=4!X$?y#g#xT_v${c@Bel`a@(Fs<%-5PR z1AKPd?Wc#ndP2VQ)?C@;<}F=)#&3bh_fBb%5Y)B;MH`oHuAZJ9^=0#V88AzhK6XFo z<*7&~M)2VK!_0tV?H>xaXTEmR45wu^n9@{si)pZ1L5to<2c&*Z6I0gvM)P zkoUh4k;eH_RFf8K%w9kCDX2RSugqxM0KTa*uR3Q8e4`$UrZ_JK@v4?D+N9SMnxY|* z%r1|NRhOVWP);XeQR)$~n|bZNluO3J)WNp6>m5 zX{c%)GI1bgA^Sxs3jx!sY1>$fbd_4^fiE4YLbP@KIO#_dg|FBIeb`0R5`~T^0E^&I zQ-&KCamAr6VvlH7=}s%+S3g=`870r_oPN+YdNuB`M3bF&M1+=knR|*<2%kB{rky~N zfS%}t1uo~QRa0#~g(r@TYJ_6esJoN_mH$^=^B%su1AB6TEX(XVm-s}AI*)Miz!3An z4Pxq~Li#^}2OP$+!ztXa2g4CKK=mbb!YpoQU}8=+obW+)7b<({g)%q)@WpXjAsR@r zk|(89>oY1qXtK-WeKGZn;H($Dj$?dc_*WR2aZp#Cxy;{y92KJ|dQMZ#A?iv#OT(P` zSX$#jBxUUoyryHynry5Ra%AOO!;-slXkfGvS7KLcLGWmaJQ^$F(F^iq}?T zDfg|8B>Ix-OJ$jXW(CdDoJ0s-F0jBw#DGs4Wx`WHjxaYo0apVYjVl;9j zKThx+h?h6?mp51Uzp|c$9|2qPJtbaI0&MAe@{o75)N$_*d~YNo$Gl?>J*L5n-p{@d zo0e6yrnbDFR@Ud|!1w$ktTqak`N9 zz~PzuEqsP_U(-}}l8xo($C3n)^MFb5kb!n3uhceI@yoQ@`sl<~xcYu+!SW$DXjz#u zo9wXZM{2pMpsni2|Ju$vU~!m%iI&hrW5v&q{vfwgXEYj2pF2EDxJoaGCc{T>wZ z#b5uz=jb*)XIutTEiVVh(DVa`wKa% zH{t(OhF(5$V8Od)u?@0*&N=(JE;JCVX?7Z8(?kdgTt?$rj5rb-vi#9A`SZ)H@B?<% zHwo;LSxg69KkVNV)4~sM3;wCGOJ~USsH{wG+{;vfzp4mL74Q0De`Ce$Y}WW<$qJ$dWbVDMXzoepNR^?Giv;|CSC>^>ozw#)P!` ztuhnS@G(8O>Z;Yy75C>RXl1S6lJ>pUN7dljF~(~1^Fiwy7g8I6S}T43StIP_Iv=i@ zo6$U-#wL8F*PYgf$5_or@V~bmx2}VM{;mT$?2*;@d*YTraMchE<^?r)!7HDSn%}NT z)q3KZJQH2JsHvtoHo|=<%{vPY)~+H~`(&psQ+l#WEe+a!ImCqIObL81QPDI{h)4a~ z)4uXMNyl6gLo}V{Nm{cjONZ`NQ|8Fv_IY{w7G@d_n`ST^E~+eP=pS5f(3CjWy-)!@ zQ)!Rq9M%WMw|10~D<_#Pntzf5hgwfINK-mr{#FGy53P$zYdgknWcGolZhQPJ{)x~; zI1DXbz-uG_S1>C&f~l$ghh0m4c85CHDG}#9ZKSwoRClH+a-3bW`P}{kZ#uu!g(Z;E zO~2sV$$Lq@0|tHX^VB3i&dbr@jyMgBdUOpC#Qr_}RKK+Rmx!aG#P;PNx%NKnF=0lj z1hVEGwUy-IyqF^zJJxJl8w}{oYrT|`RR^=DG+`8d(p%6j!q?t3 zJJ7VBW2SH_*>0rY7m}Gy_s|hJo1KhEhpnPI*=~3|@82i6)z0;ddDpEG#Hs_Q+otQ8 z7nVIV&w`n}m&b?c&gHz9jZT*sUhC!uldY=@e6%ZD2J(NCbUN4ME+$UMAC#s4GVOc% zPmEi_AgmaDaIwQ9S;^MOFtu+RPCDF_Mr9{EDtkQ;m}ma`>vRenp@^Aq|60`H!SYdO$+JMqe(PF=2-AI*XlcF#2WID=@z&$qRv*U4zjf>BDGtoJ zBpY>3voYrp8U_7CV=^tAgUTGUhT)(_ct&U!PmxMs;`2|lMayTo=O#XZ8W|eZBa_qU z1y7yyWXcz~NAHu1AAj>!9&<=s-a`{VA7?+U>W`|JyYGeKNtsHw&VS7n#MA^336!rr zh=NEc1(%&mKR7O)USnqr6{SkDak(M%ak(KXo_PztDjTlm%^COd6-wUU4;?{H8zE4b z5saPihQ31I5Unv>7dQ(jc7Y{z7dQ#Q^7`ZF568d~6K=?bncnTt((e$D!RQW8q|yFj zlqz%g1^w?c_7w{cAX6kG^YQvf3}F&&eMAw9ei$)cIN+V;{2Q@hM|j&5%G zUlW^9`OtEWnl*c6OY>hN%+6J-%HpZ7?B^|+pSS_&$6NPTozH(7>FE(apYQd->qI^e z2cq|G4MWh-94QvX2ateTLK~MhAn6whbY;{ zrwYXb-h^h70=KlY3$Ty9#UVOJ141a&OAQi?^lIy=Cn2e?2Tp)yMQ6<{Y>mpvA>S*% zX34)!OsQ`Zu{_^})mCq;p&%i>8weXkG z9q&Ey&mJ8E5pa`ZAZu-yisnL1=MNI68j3YCiigAlwBaomz_fiVl5Jsht4mwX_Yr*l zl*e|Wb%?F#mCpiKMlwm($*pj2#t*4 z=d;z;E9lZf30y@q=ZyJ+QRp9_+q9ipXMbz-~MLkNuS%^dz zmsznAROLdsL=zW^v}U*lgH|;zaXs(SFUsu+&h4 zb3!_UO8_P;9=p%y)z&Mv+C)n`J_hHJV%_NVhhWL=|IDkcsI_XRD-fJnpW|plDA8lJ z{v?nt38-IOHej(mth9kB_A-uVMOdG5)aFRqdS|S)fh!^YTOHzVBQHyg%rgo*Wx#@S zt(hA3tdP-V5!!ohqlb&3!-S&Dg~6%tn+>Jesv0n%Kt7clQdQvd`8{G4Myedp`&ySp zlsDP{VL*tN+f<-Nv6LXC;N*n>V-`KQ$YcVG>`rp~!N%kdz&V|7;{V~DYcF8jT1dFM ziV<5Jr3%HyZcg(kr#fvhGK48*P5YCoBkTyVQOFXSUYIFt28O8TdxcM(wKA7yJ!I8w z1qz7CJLIBUpZ`P)XID0K>?>+N!-DRwH?buxA1Tl7M~~MaZI-&}tB8*>zl`XRb)zTD zPU!$sqo~HI0u%)i6BQ8>%&h6zwG5%KC3Og+f(sAKKGpoqWqD*QWi7%6tA<_FJCiM3 z57gI@5FrY+1H;4kIwNyb%n!04dH$GF1OZohN{kiNaETzcLr&NP%7AA0EK_Fylnq?b zXy)YlLDC@BJB;2FN{|QwiW(C>8PvX(Qr~R1Y@eMrj*Ndu*HT<7V>Hg)FG2t2o}s;# z;zElJba~m%d2A_8BPXzdyOb19RZ+b%X9)q`+D6h^5)@fVw|OBPu7W;ZoIGwO0igdZ z;=1%KMTbIFDxvmDVDzkIO3PSc5oTCNJxBMS7!%IzNXRvrg?C7~S$Gaz+(h zEj7YL^VTI|BYfG%;-J1jql4O*2T%^1lhVW$gI+RoBPM7_7&D{B;wyVt%V5$888hTc zf%~P#79Wtit|L^GByEbQ zszr}Ky;oP+yIV38jc0XQ%n|qX*W(+xqbW;|bJtlQ_3ep9N=`;7EW+P(r?_(=H?Ul; zv-ZdBr&5ak%;_`!={qX81%zkH{-L}^ndE4Nb;KlKyj8?Ao#d4_I;K=N60=GnF-&%mh?R8g&n9*dYbmaz-Ki2SH`2~kI`Nrhs_>_w}66cH$;fji!!SC z4H0VVo^XK2u_jKsJK>i7D3r|W4+8FydPDCFC#={^cZ}G$&a8KWXh6#=K(Rw%e7wFS z64EWsRO{}{g%oF5qthW*7vq&XK*eJjD+lL%kY zKy`_?2H(Frkg{;*w(UXu7r$_O*hg;Do0<$6TE$d}LDUC#WxD}HnRZ=1v+OzSN!Dzl4_8)_FDl|FC93}6kK7S;8P4hf%mY4D{zUo=cigncqt*AHZryJo)@n`~ zC*(WVa3LS{ZQDyjG-@A#E-cN~C9H>0f9yw7T|OI#5Yz6^UxAA7+zEE@%CF}%%W?MP zWvJv%d&-A69-Hx2RlnF9WPRnFZejkU>$8;W)0Asi)AzD>1+sfx)La#MpM+(~*m4sB zRG1fN*KNXG!;oCX{-|)w8+&En$xC;=xlfVD+agN-D|jt=m#g-g zrw~j6F}GbdxKuckch&M0@A!G`D^4iX?WBXq2IaPKEv`er^# zDDztA9rY5ru1y?&)e=ed+(ra29a<*CnBHQ(R#F0qfH~Z2C*{{HH9a=A{fSEzA2;js zVG;rf*ut!m62nK5C?nC&2 z!t=XVfP_6dg(TDjoc6FPC%VS3a+pS1ylK#R#yq#B)=qpTWfc02b=LtGtceyi#h}lQ z*4YAI_1Mfp0&@3bp`m!OtoC<#TVoN#`e^hYMYew-GM6y!P|^v%>%TX;jvS$-kDC9Q zKXVQ2rk}NnjFi@&e@0RJlkc9&=E_=Mw|T9@0z}wp`Xq)pi5pahhxxvbA!oEQr@2)2 zXHCvJF{C$iys%+JwY$=K7>->xN`H#g{KThgWAnZFQ(_R2t?a8#S`z#P`L}tdech7! zmrC?Jf9&Jf{<|9&Z3wdDva4&UMt01(QEf7Z#9xDvvMP9w8f6knN3CerId?CHbsJYw zT_B>B>o>%JfaB(J^J5f}E$y4-q;{*To367nYF}4OU1PakX8T*c*G)Z+sBuq*qcYSZ z)8Ye@K06cggrb-^PrTBGHMWs=M+Xl(TF#6)|NJJZ%-|xgc*eza{Bq&po3l%`3?{2N zOYL;gmrbHcN5aOk&bF2~?1`j^Qp!e~DNwYoqpmD%Wiciwp2m_)EK4`=)XY-7CV>_$ zTm3J2(0ZlSw&djkZ?P%GrCbRu4GTt0_>^MP=q7P>vclQB+NnePNf3LRl5on+MVaw0 z_Y-GLV%gMrYL7x4H&#ShH6`nuMV9X6i$p^)bGDRXO7krNj0foNc9MjX`K7wwN&ss} z3tQqtjC(=l@;s86SIJXIW=A3sdyZTMPBklyOvy&)V!fLB?@{%ZS7Y;{X3Gz|P9FC3 z*zRBzQp8~)+j@zTg0LoWM1;l&#(GSp^`}t(d_|T_J(vqF3GR*f5>7FCNgo$d8f%FQ zCn>Z7X))SLuQ2j8{i*V^y5uO*D!`$oW5teS3Nuwdj=mb+YO}bc;h#%IIbo zxdFJfzvrXk$31e39^^0!@Pf|08j(3qSJAT?N>DMJ@>Q$qdy+^pvmfObulxuR18q?< zzU(EQ?2Z2R)1cxPeNc%NJwzWj^zp8)w=cWh(Y@Ej@pFO7Uzz^m_0@IePC&=YY{qJ) zwS!sP(95jVZswxrGTqh{n}`yW%B~0REY)b=&{r1)nIAc{eoW?elSCu6IGpo z;iuz)I$Bl`xdRTwhJ3*dI9$NirGFv?7YO05QScNh7AA$J6X-q|dd@ub2Sqy}s(c>H zY^)?XgU1`P%y)OWJR2iuHQ=!yca(gx@o=LwcTkj^fefnV2ahW^dydEC1s3)8KMGo%v7?az)ppv`}g8&v{voF@4fUVoshlkNWw zqNioCa}Ozp`x1cFHq(mGNjks3{4Ls^5aOJ zwO!D-+L@Ve8!c889q62hxRJ$0!8>^W(R~(_ zdPPDZ?iymwAfyg7jFY6f;6;MHCY>e>u-qlOul>N?w?)>CMfu}5=5C8{YN|Ouabvq4 zA?(R#!lch1B>u-MOzs`s{{Lt{&N6!&RAJISJ9Bd9dq{|ts=xm5e-WatLV%%4N zE}TJI?n{5He3o{m+quUoWK;vx?z7P5vaz4)$U4N{t_;m3R`M+rB4eMS!S3kZgz5ppG zt$ms1k22@H=D?PNIA>f}Uv4VwJac9S#0LHmvJvzXo2_S3-!rkJ5ej^GKKbL?EHw}A z9Hp9Yz+3f)}Fo7Om#JpDCjAm z~M9sMM6NqPgUXDDWNhG*o#a@qM24rhsLXqZ5Tb?(^ngJck8P%afz%u|}C! zuwE*)!#|@ZW*9g*6kSp?agQy19MhhBQRXKkq!O`LLa&tD32{}Ueabs~$nPYm-nSJpj3V0<6*SNQp0gZD za<$Mxl@PUNV9FJ_0VYtuYYs|dFoGEQt^}LANc7`ef%bfaBEYjTVL!V&9WQt4XEu4b zY-RjU@(=uzp&-cUm0B$anA+*kOe}28S~sO#_!!hJfz(ZwqlsXejC&i=*9)miJn06L zTB8wA%OE0c2~X4rQ|56~+>-I7G%{`lGVZZ)I-1?}WF76HW-g$H)uCFkW~@PD5}4V| zZSl)=LNo1e?Cb4`aGOV=Zoht|skF^XreY1}8ODSKCx5QH3Y^ z=5&>hTyxCZ@dEmJsb`2Bt*2N+>@C}iK}6l<+r{Us%Sh|NWZjq*Ajw4;L?@GECzFCT ze$bIcJzkm~K!v0Tk?AS+5ZX;Xq}~heo3@Ne zJmF+o&gz&y>_&_K335aPEoOeZE%Bm=#4ZkfyFK!%Ya_JBrcufE^d$mYDXs#eA7Dvf z%)DG2Y9rovIifCOQGD=-s|1s9^sVmc1h(P*eri`A%catA+aUvupu1Ikd$g-ov z(BN@--;E3tJfOi6(_JursM#*9FJFB$M>g42(=N}eiI6Lt%`aDM+E`nwZ)mO9wP>w( zB&GNsceB$okCB4ED_;G+raR8Lwt4=;?fu-N?-dt@ahr*A+-aFsDyjHGG(WTE@DvBL zeGC|5cA8^W_KYx{B!L(A!z&S@GiR^2_4;*-@Xm9C@; zUCb47y2s|x52KlFTbDJ}A>nPFS0=CaWM{ugpT8SF3vQUuT)XD+eQ$%KaawTFqZvY> zoKER3W-4ZkrQ0D>qWS0PvGA_R#JGby)dYUxv5ZyS7irYoz6!m8Qf3re#2bkcv^r%r zt+QUmy$MCDOKp}rre~Qv|i;nU8 zr<3WrsbXu7+#;^F$%a^&ZndIoo$oteJ=uFgw<~Y|{&6AO-r0MGjX`XC2@Kd1#sM_; zV;PcbJ1Fc1qjMa#QNmt51A*a)-L$!mO-vYU!JofTU5i*xBE_O9l;fL!U@&?I zVuluU`v#6-!y&?9A^*HS6IjJ%GO9PU=fU{y36`IR4E-9QmyYxu-krNTX30trHL&D-qdQlS!b#1g@~o~djM#iIEH@{ zz}GQEog6=x4X*3WLIL%NbXd#;ps{Tra?Z zW(zA)yfOp6mT=!CK!L=aTVCubL@K*4{;OLT{7e`5F;mN8UB?+Zv5@s5)C5)yN@zIa z#h?~>?T0q>mP0PRdpJExoIOyvfV@HA=QN!Y=P7N^f^qveq}0p~v%z@~iItDFkQI(? zK7Ix@yHNTs{i|Np028b8p*p-@DGT{N1E1Cx7z`836cL&ZW5kLvj9o*WGRnv`{Og$c zz5Y*J{YH~nG(XH{=>)aqG5!*(a#CGzJryc`{hQ!KL3#z#9G?C2)h588VOl-wyfLVx ziyQkGvQP9C;STWQ*|_M{p54GeM6oWp1pMUb&w|RxnGpLun6X&P!$OT@lyMg#v$h&u zZqlEferM?OS#)n`4N3pX%c#FSZMPj~I(PNN>oXj8#}v&UB5-(r9VTwP+0AP;$3^0l zPv5eM1YoMC{7~6kS^|n)P9uk+`tsB6Cg^_3ieNJP6|mqv$P49Ls0?r38qhG1Mpbb^ zXp$cf3H$SIP2bd{NDQ)9iRA1j{!|_c|2Y5|%_}TMs9?jg)*6(&Ngl(+OQHC~Rb>B* z8+%~1>QYVuN$0G^uxZ?kED&{Kve0DN?BVZ^`WS&(4SP5NOBuj=JK)c7ymCX}KxR>8 zNdsu8A7>A`Hb@ZjpKU_c*->@It(#W33@`{7L8^1=ZP0ULce+d$NSg)17I01WL)oEW z-M~2z*j1JxU`F9bILmgy8W=5vd<+xEH00rfva9Vi!m457FuRZj#--xdtY;!}Q5!$f z;Au|LS~SuHE(4%~L?B-E6V(SVJNVOK=E;4Ma0?$aJD+36L>+dgUtsY<0-+Kls{_K{ z!knUd8$T?@S0*xs$ZehUwuOWsQk;~pRgR{?HU7rnhZT#ge*~M!v}pwo5ogwJpM(2% zMyvP5CtZ##bz?N94(K;oA*D9u9~IZZPDNR1USTYyGnfG8QG~Kfy`dLrm$oU_sGOmh zqIMkf0)%D2B-ZHsP~Po`q16o4~{a5+5t5jDShqf#T zX0fA%E0^J{uoFc)=o|*Mc1DT}W&-rcp3HDY77N=*T_bF3Z7t%odb0yjl;Of4<*bS1 zZqkm~r6xczq*z9IgHx7HbezNdZ-kTS-1q1fWuWB9IrSyzXUD?;OFO6J4%=O~w z`9s)r?lHYZm;7;O(Xnxmit&@htc9dr*g8qPgUE}o*fBlKSm!B5cF*8n*3X7%{oUF# z?^q;>*|2G-7g(*2`>?vNrJhjWBBA=FU@r)uc&)}96%3DBv z5XQ4CEKo&R!oqoB;6P3#<}02tK|olG0)Cs;hCv1Rw%^Z$$JSI~u^@g#tlq@cR|y@H zzmns8sT`NSm~;8~@s-`rykZ*d_bi@Ld`XpV54@jnQs!U^zq9gJFTg$|O!vgkY~Nk8 zLfTzRLJH>Jwko^yiT3NY{4@Samr58iA_TyRR}9abzVr0t-A4x8Q78mVVLA$&0q+SG z&q-|B1^0w6vHe$(r4qX~AXTRiFkh0T&+~uNGHiy_!>kUO;5VOes0_<)RP)T$f8fQc zVLf#@=9TxR&;Amdv3ie~uD#nvZkBgZ z$6ye5{8U#Vzm_)&mA~b8U|Xb)1m4M&2P8N5(xAfl9%oxA``uL*oKFtN&rufpJLZ(rEM@M_!;>lTEqpZ)>qcZ~`Nvz+MyUgCX8aax zddlU2*dpoR4WBu?CX#{WNJOTxxy(ur?D=ZgNLQ~MU0vom?$Abhyfefd;A%hnAemei zsI~%0S-{dKCTnk`Tji0=8AP>yp5!d>+XpUp1%O;lAgsO*KJrc#2KCMj}8rmegy zr^Q^!tYhX+_F9F6n(3-islKU0GK9{YzILLDsEy?n)!k;{rRG}FYC)vI+voMSBaeQW zFb1v#yUq9oJ-7WP#U#w}83JB^v>`rU?9%kS|9ANcD>Tf?#jriwlun z#)tfKU-EXK!1}8iThfO9O7V!B@15$2PAonlRQe?HXEtEr9&Pvn4~TOZwaZw-$Qu(1 zAL0|4T-d>i3X)=ukNb0d?cN4{PBBY<)^2;tpoj2T-m8cv`uM~66h-e_e!|e?! z@{d&A!prTt4t>f3gXplCm-RobXDFrz>v+&q?A4afL&BEHa#`eZ5AaPN6iMijq&R)D z9M8i)T$>VGII#1n{>pKG{vk{RVM@;R!~Z6raCrj}FqTNS&^e-_0G*xF)?r)`+4VXc zcYOa>9jbtlgmk%)Z^$!=&#_B`alk8I2Kpi0+|FrCMK8$=D5kVaI$ex1qvG!pEcGp9 z=gaFq<;XTqnIN;oL^bu7A?QI4UU<2rHjtmhj5IbFle}F3;oaj1fJ3(87M@F^UxW&; zkua6JOrt8I{OtdJeu;K8Gb%z8h?%6;DtA;IU5d5J;%kh8S}o-{a1n0CL1md%LsoJ= zD(OrrPc9Sq>{u70 zJ)_qLjpdrz$CQg4ttO_1h}*DmM_by(B`}xouj|J+mwG)#rP#?ei_)IvbTRyy6(`d6 z^)lFgT@EKTBUsy}bkXD`HOIz99uS(ctdN)Oyf^$lsF7CK9lrQ@$>~nLQ$XjmBWEAX z%)_1sxSi7rTWSUs_uT6)-qyj<$w!%m8lM8iTGm@JcEUmPI>7#11ziNHNElL;5@pCE zSCmrF7Gq_HYPo9%q(8NhJLGbq?V{31)O)0sI*uH9q|VF-xYIO71eP7 zOMy;mu4ZQC$y+fAEds8gci%&CVuP;CSsA26+PkY=X0 zpKn<(Pb_ol;rm!qN5(4qbZ*7 zYjNuTE=uDmrD}U>=cE)rx3c9zm`%bzk3S`!@hBifT`Y zh>Y<}zU;GT)@V)`hK%N%EGhYz_`D9WsY2+B1g_YWytf0URIL;QALN2>bIP;hGkM{; zBnY3NMlY;W#ie~`_CEr)`^kaOws$zshztqs|jAse9+$ zscm($t74cp5UmW*+Q_3w{4XG%3<0Xtl4Z4#lK|}u5^~^{m z;HOWacvtmzOq<^4=wYB~RFIlV6ywk}nK;6Q3RU_*KTNJx5T$m3=8b)-(D)|$i+CEZ z6vOYrphTJ@lFk@PcX}GlZQD%RB$GDYHf}W5uQYsG;be*V!OU=?hYD+XLbbVsCIGtjtZfda$!ugnZR%GmVv7^=b%mO=+nQi|( zEN;aEK)0B*9|OjjB!*qnG8!CY+N|)ImO_5RosmzIU>+;;VJc8!&V%O2x$*>JI5B1q zHJ<3u6&~s!Agc(1!va47xWe-KLULnYo>3?6T2}*BjiCF3Z*G;N%qU-i4|qbRV&-_J zeG;3L#52%x0ys02+He=DG^mLJOlV{H=~aC&TZl01GHin zb0?|6!InSYD@j2y=rPSto2Q3Tgcm)TJwyo+m9+!bQ+HRZBov^gXTWtSMBDcfOi*43 z@{|8Q6IqazIj4h}uFPlc;lW{y88{+%s;Q20YNtl#rThk>*yscOT4!0hvyN^y$^u&F zy`SP!|CYtWYXQR_i<4lp2fwtYxbw!4Z1?Lfx15n+$-=n(yy=7j@CNZ6pqazKVrkR$ zv$KpLcFH-s( zEp9poE!%JjS?5odQL^;<5`#20j;eK(?d>|rX>zlY9fEPKQx7q^prHy3us9Wn8F zw;Mk^SDvSBZJsET(eO>GGiex=TGtVx7fih>Z5oWWm&iY8NbElTsnuh7@rG!w^Yt!r z&K|1^7z4wP^s|w_mNQI$7A45(rsUBYGs%Cd3|Bouc#aX@_@<>`i3YZbvx7b!MsY6O zKDLBfQeXIN@KIIp3w6t)ym+cIRCbOmcZ$;_#_!!$*Zti<8oCky@1!2e#kHYjpCxqY zj^I~c$UT`!Svpc$W~5p=pf?A|lv3UeY#dzz(KV05Yw$UU*{%xTuej>i;vl$d zxMltsYt}@Hlz|L;AJktT6yE3B9JB=aMxSh0UU0roZBrV2YQKeQskSrB8#!i-6tNa4 z0PAKkU}ore4w?)fi~981Jn2DEd;YqfR5$RETV$SFbl%gJwteL=~AlX^p02=7L zW5Xw-$p@*)2WrNU{H}&Om*_GpTO5w|vg!lkn|Vc*x#dGGs7qJKL*Kz^;m)SvEt=j! z+(CH_$Ss{U;gk)Bknm2qZ_t(4$lTgg*fYSJ z5KgHYKhE3#YCBJ4OZ=6;qmC4+Ukv)mJ+)IfdjI9G@s2Z{zhVj9K z4Q*N1s^~n&PJjrP1tyj(6S=!_1+%$sNly;+|7AmXp!+F+Q;b zl;Tcd9yl|TL=B2tZi$xrGPwX?HP&0tg2Fth-2i1lXe^~`lp=vo>SPlkNd^VOkMs{< zN*?ww7*()9`buOnE^;1W57A@DU;!Bs{qs+t#p)fpevm8sX^lDF7u1pqEfdmlO5ljKDd)jBG88sguwSv8?ErUUhPVS*WKM}25vXO2 zPbXzGM}z7jUE1*;G=VBK-izAkEKoGk%=I4?iJZ>^}ie7IqA_CMkAm zNUQ*993fP`gqSPS_NCOdHI1Hs0kT9H#X}gPvAL9m(7s*Mw!y~EJTMIjGhd{6{E&2X z{0|Th;$dzaelMiP23J{Lrc-Th*Bjpm&)>fqYjuAEU2}$)I(81_2Ry?kUB z+StNevb*BHpFa|Iq=Fyllc1cre{c$s!_ITCszo8UO}II_<|JO!b#tTiO@e`WGVyCA zrYN%5NyH}>ePf3YA?E&$jZ!;--G!SN;mU!_0$<=2-$GaP9s6-)=&Lvrv5)n9BQ%Ns zSSIqA)mDgevfCc@^Z{`>Hy*_w33vQk{Im8W*dCozwp06XufS1vz@{wdvJi9AKK~88 z{NKv`k4N}FnqFyX18t?$T{E%(IR7slvME8^Zk`b{SdSwwudl#_QEs1+kjEud1qDMw zSc3AeAhi*8?GKmM_N*#9l7(!l@effS!|yfFJ%57UnBU%V5w3lKV_;#1?@k~5+kHBC8o+2wtaZ8gzludQ9RR~x+ zjA6A;EAt2(Sl{?ZGT{d4zl(Q1@<%ly^79kM9MVBsh;5Ef=5uak3bNTqbn=aq(2_@U ztw>_elq?Jka7{2%UVs;7ntK&~MRy%^%1X`PKnl1NP6U zpYQL@on_P{b1LYfjT?CZ_gCLpK#sv!+#fECpruV()$zX`RZ?_s86f~C*B01h#$F2? z>(U|mm+K*=O+CHuI&{`b=ZrV}xuW~`Nv&ZloG>as#-)esfd$Cc-<(SN`96Z3APXNg zH!=2k!tMlpUGix02pBQ3_-*zyKc4YN{y!Vzsg){`%;nj7|FMk!SM;BQeEk1d#{cs= zK>r0?UrFNr`c=i70wjinkrIo^1DU*s1ecOq!vhYe@`UnMS?v0Lb~7V)Gj(j}|0N6w z6%i`~X~1HxLFxzkn@C~Fzlzh?wqHM0(u|z;O+kC8yoye(1C%2iW|2Zo2pARZLDS;W zaJymgD#Ep)TTQQ}gg@~6+U+WfM>v)6<@G(zYntmhbNltn_qZkdUqngwE9_q?tu}&| z8D|1O^E{caug=*>)dJxr)?Mo9{uQmoWb%>;vq`iwPSJXzypu(CrFk*}-JL;q4!VLB zKV4zW!91}REx}r0lZckI%UkJ>Z!}GA{2r+gjk{_>9INUsv-umPy&`C4#93>9gfhK) zUJI*J2cKYi1oJIWq5U`hx$%XRK#)b3^EwznN<^!em-}FrYTYJQT%{f7&>_jW!#F|q zF7%a3Zj;+*P993?EYb1Lh#)U5zJ##a!;1xrO`M0eOD7%4y7BqXQ`mgpIM(Qmmd66m z4+b9Zc9rMe8XIkIjHos#y)I`?$pzK|E2dvgP1{Am?UIIal17dX95Nz(w&eA4v*rL@ ze@Yn6oI?n#EZS495}#8JZcP|`M7Q)Fjo==`2i^YlcR07N$3Sk9kUSsF`!w?8bc=Z% ztbcOz?Tm{Hx1o$KIwfWmytord{7B9&RZ~My4~hR?oO@r!!YU1MH9u@(ePr-8Bk}JIQ_Kg^!+!m zTway7)sIIOV&dxZjsaM8LtFrd-Zp7ZrSpBhy3nQ}4Yr(RjHkh=!X4?LfEdsZ! zV&+wu<049dz-Ia$50*!1F3=vc8ZZn;Q(9tOhX2BcMSqfgS|+V*GJ0%1&ex7yXi}!E zbY?pv87B=pdYt>qUW(1V{uFr9g^zG+=bNe?FC4iVJ(`@Z&!f;tfEy6pi^KwZE4wZ@*wTx~P$<_H^;1sbLUUmC9;wxjiwpkJn7r z8lo(+XE#SOXthflDvkDw0h&xokvAqLTe`v2n?c zKm_!(1a;GFN0qAgF9M0wuDqWB`o24r*3a^xU0X8PL&`4LPEo)_tXmZ_q-RS}cHo_P zC7*%q-cRT+cfNsoKeMwsT_RX^d63m%7D1Fvw4Mn)vX9{nlT*Y=K$^|aA<}&&>V0bB zPuDx5Bp+|R9yR<~aab7gAlCNr_pNzyeM7gZ>M?Rwipf&ij__Voe+I39@SZj&zreIn zByIPqilfH3Lm+^W){DZm;+9UzsbclheQkh`_)~BS3wD^8Jq&R|*0@UHUyJGLmHpi- zb{mRanGRBcUyLJIqf$KVy0vB)y|5@lS# zYVBIbgnd5UWlTbUCmAn**2-Q()#b{3pu4@?W%n<^JuQGZnsPc_P!YX^EvyBL88Pc4 z)k17y^zO=@Cnv&9*}Vc!s?g$IJ#}nQtl8gb@W%N<*J0Qni`yq6fo6%($;yor-63!; zRU+^eltT4TYDRI{>4af9GYPoBp!J?kMxtyY+RUlpG(tS(6Bu(qzms9p? zUp9kda224nLMtg*tjYbt^!_*P*q7rkl(r=YkcgNi|Kl*?{!? zXzU@Bug1fwW0ej_{>MuEQ4(>`k9TyJ{p5PsI}Cq8!+a}Ja$TEfoR!4!=dCZok*8C2 zEU(GVDrNq5nx&&{z*e_#_?8!K)3`dEynwffX*eFbB6O-aU6i&VSQ9Hg3tUsEzka&c z(*wYfhIbp7R)v`Txb!Tfxy_=5x?}6q>J#9TD2JKtvGvsmkj&+LaxtS0lontmOHOeL zzfjUMra+Z#&WuCWtKBvmf10wsuD>V!zH^t)-?R=3rwu;V;Z%h0?=^Ct>Q%o{1=${W zhI^y;neN?OG09CKc<`~dv032Ke@~l!ijXy_GdWm&P(W{|zRU| z>LQ8}z+nw44sUgZ{(N@voX`C(?g3!BLtJA&sC$n0+TCCTY!!bqo!Gf+Q{LqFz?8n% zW+=opNvfx;VqNWdGnsI|jT1V7GG%=k%m|$7f#dIHQRTn}=p>rO<`Aqz)SN%`fOiQ; z=4m(^JZN^Xl5=~JQcbLxo14zvsrYY-D9fwBYkjl~HiDzScs&5_IRDL+RRZ?@P9U?8 zz$GQrWO28xiU>GJnDDq;hWMpORX)c8@ESP-xq(F|u=-FaAA zetBq?qTu+*%A9l>Vy5sVRV5Wx7CJ?;Q>R#3b!xvc-F%7Xnv{fbNR3NG_E8r7!*q0p z;x}(aEDb**+s3{nsp^ay(pGxI(yHOu8~?Ho_F~j3F9&0k^)XM(Lv?lAj+ zx_Vxb$aO;Xj@qw|RMSruuwR?#Go`al|N3GR+r2Vp)=8w$AS<)aW!| z3DnM+xijJMlBS0-)MD(jfF&ulIBT-1Fhx(M*(seBzNXw{pGQj0uU#CEtCn5HHZDW1 zf2XdRS|uYXRbIblLcWY#o<60zCY@BFqt$MS*H<*iWv`mm1~~6+DbP`M8FN50o6=s? zacr7Jy4<_YE5{XTYhvRV{kJ;DU1J%~*m4G(a?rU#IT1+*UvfZhrp#JQ&7yiW=`3m76e8&0U0dU`3gg^#yx8W$B5cCslSJL$A<;f#^-OG!$}bRae>q-L>} z#d~SsKx+wp)6-lne8&>;98gvIg1aEi3{R8#-rNKK~>o0z0s7ro_#NR4YFWG(3# zubS@FTm6@+J^7TKxGH}!by<6{6)H&q+ zVdW8ZmrI#qS53rH?U`~|h)n%w-XF>Nn2t)o_mSuM3p4pxAHLn=i(Pud<6>6Z=+mgy zhHDH*4$whZtkr*6)_UOj8g9?+W-(S=eKUvtf$own@)*^Y(M;V|wDq z*`!ICr_*H7$z}`$t|vmiq$M)jlxi`{7h{Y-K46JMauEU*A%O?zN`40rvx~!f?_k?> zqZ76LHAAFOdPLYrvEb=}-aj-AXHNo6z-}n;9IVmc@JoJd`g!9G8lIW%Hv#-At6mU< z$49mvOa2j~PE*LPt$j##;4$c(urs2Emm=*Zxcm%?GeIDJd`U=1C&i939>Y&OzFo*U zBH#mfsOVRaJI_vnjxRYQq-0_qL_sZ<>PmRnN^pb&o9Pa>>6s^(EBgQ@7+#2G?cFl? zJ>JUwVRGQE)H!8@q;gfs4ePZrW0aYD5Y7l{Ky?D;yu$|N;p_AsSH?fHwER%B?p|G$ zd*CB`l50>BGoquQCEhUa=Ol>s7e;PN{W7KvYH%FhOYsG z5yPbcY{CnL;H+N_EbV{2ruVhkc~Y1<`lIp7?bn%G5_@Yx`9|ZtDW!p&f4Df~b^s2P zpV;3C@+UHKa%jf9em9Eo+ptgXEi@m>v@3jK&s;nqs#_pf;)~XM-DHhdBG@TN{Z_4a zMganYv-g!k`~gn273{BeJqthWG6(EFP#nKTx!Uj!cTueFNuwpGvzdnkOp=K1{TrSU zetw2~nv&U^-GR;v3G@M7GoG4(8-UwQzU-yYmQ+f^xM{06=M(5g7bU^oKSLfu>usAl zzCwX@`P+MoVwcR%!G?($rwh$&b&Wgq4C zu^>TNszBegT|;|!XdbK)>a|J`K`y?4$_<UOQE|qp zfW&=?^B|y0kaA8e~gZ3!t}^8bFL2AN6k?+Un9M zf^JW@(j&z3R?RM$018~OP#<0AT3E6QE9Mhgxp1(G+L}Z@YVdw`|GSB{Rg0`OPl!XB{$v-Eg9&wXeHXj0zG9My2fN>s7nL7- z*nu#VnA(UyA(|OGOi?Awffc)n6-ybzmJxeBL5ys%Ae?@PG(FCRI03Jid>xtyadW2J+4lPf z@Y^|jSG64Y22fa58{Y~St(j;_=(1txkM667Knb`UDD;f!B!}RQd*kI#3+K7+lVEGp zfG>~vWbK*Yl5sdY>%z?A2Mf6RTe5dV^Ui3~BvL`;#ihvQlR^1AxC=d+F1O7)`(9sx zrD198xFvS_nj}>&!U3!4c9pJ=SS#G55!T8Athz9%6%ejQBxzNbMnAIn;6TB1OISQL zF?8IRTJ_vAVNG{^hn3$8;FhL5Csj0|ID%&K~xPSb7sKVgWZYB`_7(n$}3rR3y&Q zg%0ANgRYLEuJp65<#k!*!;-HOlcM%0L2^cW?roMmTts zVt(%+vc-pycN?E;dj#_Xw&4g5;Me4|llAEnnxKO@0E4dPjXZkm7c`i8p8&x`e*lLD ziL5Q^t3U+r@<}D2w7(Xsn$et5K&9pO>-$X}OThRx(_Wl+LWq9T-idQhsnO`opf)}c z1Xz;s8i*;?g!%J0QLW=yjk zvVNmInmxW@YLAlLxnGgc>!q-usLo^919-NB@SO5*2G z&cGiYe}MFrjM;Np3lW$%a*xQ?&*B#J^^Au;A%rXDhn_>i78keR<`Zo=wMdtqR)E1P zVCR-2C^|lI{6a6bQ>5q;#5om}nteX8eB=Kk&p_qkl{oo2ok{b_@zrnZ!49Ei0Z39{ zGg|?e1ko4wStr}=+Q7Seake3mG3@aEEF2)TK&Meq7CbsB-c>WNd9zQqB+g>zS6UPt zE^R0JNt|C>cVmxRR+V3D(AF)MFO3_u(d`qy%Rjj_thROby?bf~rpW09?-@t=4h!w3V2bq&OT57ed(+VfLpt!^Hs9G3k?AVY0V=qi40z43 zd0zSA!Mx&+x!OJ@$@K;kt`AuU{dG#t6idVJ_P!+hGBkaM_g~W9HQY~!(i_}wyL}_R zZEW4+@RFDCsjkfO^GS1N{NdkQ^L+NLgieCs zw5_=s+PTw(g>H4rV;oG|98sbClI$VXh7nv2W-c{=r4Ef-vyWUJ11OZJ>W8bHQYQN` zRc`XAYI4SRQnwvk;_WEg>~OE=$x~&hve;);&bue~d<5DBY?}q-(C*C7)LbL@qzGQ~ zQ9Ll+m}B(SLlCUMs8tM++T??5UJEje`DZ6_uFmk82|q(R_k7^_!+lc! z#E`ex5_vbu|uk=G650uae3R;P_HOIP{3nw*`!Wa}q&7=g? zt_f*uBJ64iXgXa))mW+80QmIP`iz8{Gq}xLtr8ftdx0t#hTE^rm z59q#7;P!!!0jiZpm=Mj!H7dsyORIVbZODQxD>NE&dWY7~T1B)&#v5w~dI=+ok>OK; zy`n+9qT$wKJgJL>n!(Zg9{Gn)Sv5*}y(p2+xzgu(j-T(Rw9M!sXaMhV1ARvb2Qd4` zuC=*&I$@d*_&Y%shmlYHpCX~kndYUz(rs&hO19Tw04WEfiu5jPDg{p~D4lcTuUC|r z4m4#pCT5Lrx;8w}jprJQHR1{#aY-%^Vz*wrJrzwE^}J@yj>T5(i0UC0)2AUL=TkW~ z^Y&DwV{@SvFf3G$y|ys7kjJGlhWprHAJi^K=oFP5LRMlJc^F{GZPFZgzb`PC`EUO` z3ASLn1CY0x*Dx4W(b{d>`k`Bub!d~kC(7NVi?EYBfaybujt-Tc6KuTXdyJ(^bNdRk zgr=D)Oy_D*TXtNBJ$bGa?4)(o^(n{C(UX^}N@ra2l{x|@52Y(z^h~49MI)MFq?_Jr z;~P%c*6=ImuSQhY0wl~43^JNAa^~=dW!Fer0is^0Im=JGuD*bzZ8wXUUH(g)=x3b~ zt^k*m+W2?1s0l}>gYNaXcU$b;Q6I|AB$5dE{wjJ6%acDRj*$E=kcU$?Wlpy{XTtg0 zhk4Be9&3Jn^aQPEW?peU$qiwmX{ouGxcM^;5>JjR1`G`ZWdkSwRw_ha8P`waXdm$_ z0cYOQUE)GKlW%g46oX(pF`PZ~mD$ps+);Zr;`5kZ>zC{Z0m5vbEt?yFbu;&MBTd+% zmZW4d5g-L4px7==!_$^r352%F7D1v{WudJDhbF{uoH59``h(*J4KnFa=QwNW{YoxY zU9Qmb%+vK`-_s0N;u5zqS9H4uhCbm>0DW0dGFM5z+i{mQs?W+;eYyU~{`&FwhM$(l zoHXt)U9q$$64$R9V<=jiOCV6n1C?w3u!a2%rw!l&a*9a0Px|Qhz8j{<@CJjAA8yJp zqplfdPY)v|N@;QlbRfff+z;VSG zn%TdMA=@%^FiYz!I zK0x(c8AlcMdovcoI$52%Ns2&_DtLzdiHfd~SBaW3Fa4KIUDcMcU4QbV?ZyQ&^A{QS ze&qJY9z+%rT9(7d1;L^Fm8s@${5b<=uPL9b?boc4uGiyjfiGZtlyrY5jc5Z=AxMJ9 z9UJeBC`qu2(cv4BDS@~>WhzJLuujx-2f$!0(NHm}e33i(kQ3FNlDl|w;fqW-22*F` zE?PjlVS4`*W~UkoTd=JxX8r_T-%QIx5{Zng`S`{HtX6ABxweHe=ag=H>*T))iDMB5 zH-~AY2wRIj#!6y0TCb1RP1pes*~C^WPR(w0M9p#ZnAU-N+a}ACep3nj-S%kPLIB6a zUNhwtq|IzSY~SQSLf}Hpa&lw6M$4n4=E`p)eQnsTW=d!ub#pG8)hcEYGZ$qyeC7@5 z?C84#a$!y*3smRta@%mW7A3?dZiZ1-Qni1hD+kT=$6tmkt4mZqZd#vV9X8sVpIIJsiAl_Gvm|#$%%*B4h9%v@?Zu3q}3@_co0?2OrW#vGV0hWr_1~?qirZm z=^_@1*h)1xD4b`uFlCa?cGp}u3xlFiCzaT6rU}7WU=+zuS=)?L`3#8j#|0=`wC#sV zE#@uptf|;2VO1!x1y1}CP;_*jM{`u?2wPvk;i#?LRVezTQ@BgE!Wc68lWhO+{dV|f z)jh5nNhAvA7VeAFLlh;T?f`^g^zs`_oCuuy_;Owc9qoFrGh8vUw3T!S_o`eYb`RXP zyp=?pKwoPJ*7WeIa)YccG!YOEi_f&dEs6*W!VG5-ka75s&~K$!Se$M^A3P4DMpUlM*765 z^Kt&ZSk=2#7$6;ygg5%DMz8Ph+q=pQy(Nr%BbG4eA1|+~++DNo92^&2-2Z;U)vv<4 zE-7+SIH+`9;#~ zxHB~5IgeGGSDf`2ryc`NLLACyaD1z%xsGPC&TMUvpYl4%j`tWPEWzP<0mxJMr3~t4 zhg_0ZA9;kX9EXIlJx2)dTm}`&^)G+0DfcQkx0GNAeM?NNMLuAj;J1^bW8pm>wj0d|J1VstTn@i z6mJwpYrr2iS#5?Ya+`kvVKYt8-IV#obdW+KWc)eS@;8kcoAyuEvw%f^)tfwlTmh*} zNq=ZHp<4qt)9$p_qxyz#dWJ7xjsADuxzq_G=y@<>MrFcF*`IG+D%$7@37$FzSaJsU z-EWi2uw#OMj4FO5o{EU>8J8>Ar|S&Xm)Zr!EdVA+jiLuOJj)LHTITP-4=w3l3z6-_ zUJ>OGON!IxB(B)Yzp+XzcP2COKa*cjJX`f#Uc!hEf-!t-o7FZK!^K#Q-c51~Y`;h& z(Oa(=yXrT4sB}KM0Kz-4^7|&7H`lpS%6*3ewYk*;NDC6--JnlOrtY_LkLa<_AOYVL z0YFg%t)u2d8)d;Baf}&IIx0guw)rQ6yuE<0?xuh8!uL?pLf$80ov?qjD$D0gsd2vdd43-37_CI)bK4I;Y z>WT$5p;BVx1oT1MV||EJu4hZvjq9M!F2xm-fHHtODJ!R94Ui+6~gUWCa8&~ z#5{(92zWAz`T{OxTPF3KF_EHM(lmK#bV&XN(Yc=lBSLt$Vh25+X^v5se@xYOnNTky zuBjr6X2o+;)OZq zftoC*@Yb;=fRb>D^UI_NUwF5&PK#Os>&Sqw9*q=ebMegtzF$v)Bhd(+<$54|VFJZd zsBt8kZ65GT50O;SL-2+s3va{?*L$p+7j6kw&(jff{u2(10E=AjXbEP8`yeM^hU`UVL5l9#)OV6VMT}&sEw4RLg(# z{I6<1dLsXSdJ32_E9epY`b8z5qVULpk&-LOOAhGbg>p|^Y&zaZ`zMh05r`;3NaP1) z6)y=1j4bmDO7mxfT>=<#rJwW_MU0%q0UcfYxu~kyc2P}1OA!^L5;z{z0<0oP>s-mo zYI#{}eeIF&@=2Gj>N@lN`+hn;4bgCWKcVCEIos>W>8dMW`)#>l8mO1|mIO>-|F2Ld zSr#B%yX}S6O<<&Iz14*W!_D4-0>h2!#RI~4XLF-1B!a>IR+RqAr59_XU`H3ic;5$h z*O&gOXg}p<{NoY`?3qVD{_9dm_YM&QqSalaPyE7t21xlv5sbh|5Q*P*grb#ya45m` z1A1Bh_MimM9W$nIU(s8v&&c)Sm;nUdeiu;SD&w?6h<|J`eaBD1d{gnl2A1R03(Wg( zprqA3JECU&77R?!SrE-P;5b|DCK4oh#|P_0*83mO<_jG}cjWfb5{*vhScUHX(s^XR zTkAV`OScusD+LVh3%PHMpO0Q&3+_y0%>5{W`$5RY8&bpd$i^E~%@>v34MDOM{|Dek z`0IuR;$d&P%{?PR?`DOE_b>%(XQZX=g)87K$-l?7SFJx$Y|8}Jcc8&{eCQhWgD>C< zgykz{<4vCaDVt?bssXs{KSH$lO#~aX3ADUd5ZEI@0Rea6icV#y3B0@7 z!pe}NsYunIg*eM_&k%D&8q~Tupy>py2(b_iGPQNlgg>k_hEc>?h{X%fuU%X0fMZv> zMxq_4HlEJ4yG^1h*NpY!`Az_&CCQ?nRt*p4g8zce`MoE1Q*gDbn>|lIX4g70{ORYd zXV0uz<+AwVXz3PS$u$BFu$i4Pji4;`ttUC0T?26 zKH~sUOiTYhj0I56x^^Fmxvgjc(+nxbjFP&Ivd`oiolTW&RMVq2y{39kwXd{|vNX=6 zMn}ErWcph%-DAqqQcE*ObEOdFqTBNoi;P26-VST_$rAGAaoE{UwC!tTs%@8*Kg{*>RQWw$z`#dpEqJzic%g$eaP>;FfxCrB zqZ1qJbV~6u&M{)?Z_hHyLb>?gz`2rXQA%zK&S+sGw`0JQsauW#pdx-JYodcSI*`EP z>*^|3?5WjfP~jU7cl65Dt26lfVkrQs;$NFk0yB?mJV~N%Y^TDXGqwXYo(}`PSUY&``;`;2$g09aqk(xi@3`wMZF+Jj=XLl_&Fd^+9JU*Z$ zF;0TgOemRHtid=wtlTM;CG&Vk`f(^AWj@-yJ<7%1&d{X#*}3qfz)HXcr1q5*lzk#2 zc-i`mMptM;25XFYYY$sZB=!r(!G@1-pFV(ttl+82Z6PW{6d$s^lNT29XrhJdgTIPB z(&wb6{pg;?=5?~Tepl(IZA<10iZYgOD@yet>_PBfGah4#k zti(}QhewdG`=p~H&%9isd}|bU^XD}yD)e1(gNF$df+aBrjWLavH$`E2Sxv zUHL<5Be0SuN~R@dpm7tWl4mO5A9F)v0k|NZK9B^E8yD zyDv2~dxQYNZ9*-2OTkv`q_L5R0V_1vS;#_OO%erJz7muUEfCV8`T~Y_G6Wb`eS!K= z8JcuY3s2b-LrdAmL~9||BhP++sp;>yp`ia78_H3w)3x4N+Ac#(iPz%-Ot$?^HWUmZ z=!;RUL)uawBv-W#e#W}k9<|687$j0JrtXB_Rva?fGomJ5Q`w;Bmpc$^$PueiKX>r5 z0_NM*b*wI?DY5RK!1I@L69hQDeGpNYTu8WhtsQ?=W8SJm9LQVhPUdVtp}wn7(iw_|3$E1 zl9-CraYjwy4A|}F2mB@E`Q*R0J)W{jpT>emRzeQY6@5|J#qgni+5ZZo=R4&HpI54g;ugh ztExj$tv(1>-Wn&iqYk*S3Y1ioToj>KJO=>+2%>Q?C1&h0l)ogtG*a`E*eVWH!BQjK ztKMolbW!tD*zS8Qm+!gKU+wJ4IiyhY%j@mpy^#GrH;*D(%m|{sLVqsZgNfXX((zW@ zkcQ6M^A{SjtW+7oXO^o=`S?YDaOJZxHdyx$cQlrzD4a|qoDtxW+K?5FCb4z9H(MTf z+nXQ$?f{0TkK)cIAyelEYMtIi-A#+HuNfEk6ahl(RbLn~)D-S6f(9{E zQ0^|n8_memj=+@_ilju2ZZFLis%7QN`L?-QLGYNFx9Ovmq>LnAx)7@|UJ(T^g| zRv%hk6%oH=d_ z@Je!gUnRVOm%l{2gXFmv#w0nX%JNpBr%>*7l)Ec?$Hb(^Ve*+xWg_3z3ENdh zF;w5X(?}i-`OmGYF9f}JuF*Sfg>G2n3x|3QhusP>iI|aXiN!S+cK;Qk#C@^_GZ?!O zr!y?1SUF00Yn6m~K?+yOS|yRU%0Ns843@CHo&ZeFpW3}BWvONA^rN%|ta;9~wbRp= zma24?#}I(lrj|y}Cmsyw2lB63l;bDEbq$CPP!W83mh}$bPIk6~6tb=RV~I$l$^L>3 zMGg2ghkU#t_6P3ep}a1Jh^$_o?q(_TKMqlARAtJ+NQ?2NT{MZ$qt$D6(^L3>Mruwr zlmikx0s^@{ew6}6JdS03O(jZ$GjUD*_rlh9f$+$&7!v*vC2`qm+eJfKQU{7C=q zxc`*MCWZnLO_&N#Wv}C0wG(%bE=nZ;4F=%P!`a`)Kn&mWzDHNuT_ghQvKVdMgw{e+9Lu47f*zX^e3m;tZA%J4Y>GFrywIMpO4qG)o#3Quo|AX< za!zuZ*ue)DhTQfQTl!Lw3)>4wup!3aPU4IIG{PU$)^lDU zWZ^*x4A=(!REk#>TRpce?tG(wY~8V?0|toYRxk0JJF{S2?OXD+EKUisH9-HJtKxNP zwT~k7GQYnH>A-%3Y!gxwU@DQ!4~PWP z2Kp+J4U(=I#4z%d#kWyWF~CD{jt&(Bw^X1Q^2Q3wxpmQpM*z=G#WR+`Wy4h={kaN0EqK*5LP*gr&0h8E1RM<8fY&;pY$wbhYJOwVqJEg_8O?FK z2kSi;GJJ8R#rqd|w6k7KT}{l>#IQvX!%(JY%U}LG3Ku$g~{zo zaY6xy%S9%us^K+$xb#8)(1{uE4R|^cjqk2>d)o47&fq zK@^G6tw1UI7rzo@tosWADGY)T0^g696B47(s1_JMgxwwZZAWqi zN{=X-AFKt?Z?FO*zB7I!6FGLtk=C$nJmnj}wIL-8Mr&Tuj5p_4e2jJwlOiL{VLU6> z9kN>*Xq-nMDH_(#iPw;0c~vs!Yd3yP%^mrv($*k)6>AV$Sh@L0kuJ4jF|*PssaIIsbfYtnC&PBe&-vBXe<-QB6(-LQ;y zU?c!)$fmcXvb$0`JyUt#DE^gf|1NeRz~NE-VDYH zs#ZRK1AaA~xi@x?X`PMzFhg@_Iwdhw2Yyv@8t1D@Z4pQ?S(pCio=pu)iw)Y>r#2m+ z&HQDO3IfA~fv${aV2ULpm^}bW$AY96yvhmCt#`qa%U$(1Rhu7smNjX~B%4XI;8f%$ z8k7{##y`UG-W0z_E2k+z-|WJ;E7o#5Uc352PGnwMI1NdH$g@8KH81ST%r9H2xa!UU zF8y#v^NDoi&anoyhc4Vz!Hkfj{OSG=I};n3MlPzPhpO)x)5eCX8xl3q*0Tj^v4;UT z!#W6K6tbjhotZw;5Dyu&L;EUH2>i&Ej`N-Sx%WX?6YgrbBnh<8*A7xDakVEx=FD=N zCgu=Q4p!a_Y)rIlwYX;!w6zj5#H`l9b2pPRmJrS>L+Y0)q{`wEwLm@Gkt9o?k9G|; zqNQ_@QhxL3!e-cG=>(6Hg-NG@CGr78BsL7kOkvQGcr@5E?Au9QssCAlOa2q%5+Rx@ ztYejQB_F4m_CzP&)ugNwD-^qm(PSFeADRwD3&g2RZZtjC+Fyyanv^BEOWbIbs*v+A zsBR))S6OqqlyQ@znyfu8maM-N9?ceYfVW3E*fITm$xPW#r{Je88E^Z$9RwRti=viK zE~!hEWY#2kXPI!YsH7QWJ;`3_-9)-Ytvw{etP-2ec>5Gue`K5)q>Q2JCJINGp$Fw?QSf2;>M}wTK z2YUR6GBN<{T$_$^E0-xuCyHECf8bR49SHV+xO%7H%EGQ)yJOq7JL=d@I<{@wTtUaS zZQJY>J6W;qblh>$f8OukyWU-2)pIZp=HVPQpSs5w*QLQQqHPN`!-0E)9NS2k><+} zX7$Q&`wKfYQoIW^J%oie5+tUf&iNI6>s>LtK)IhjHed+G)o1a;7;)-1RM@9y#^ou? zs&V-XjthH=+*oi+bIY3czS9Uyft+;o!LX={1TS-d+#K$wq4RiC@_;}SNv?S#+du6~a*CL{*7m8W~tm5yufK_dY`QN@cm)t??yNpW0;&{IM%zE z@Kfk#N@PC@RAPeY6X7O@w5(9I=O~+uo5wM%pNp3s& zGE}QPQC9DDE%25(^xq3li$||dVkm~Fm@1*XnLt6uO(GpWDDw(+S6LxOjJ{cduRQjN zCheVhf?z9Icn9t&tf0NWKAGIxsN=$pG4unNkGH{_DYTI4azUcXkW8#d6T@;^^cI=H z8gv^=s;Hu)lo#l9Rsw)#{L$dNny>l8(`y*F63GecG~?L9<>;8^j1d2hP)aXVptVkl zNEdB9j~ZV27er0l;W&m1-vTJPZBu#!NWS3SjLCVanr3qG%`f}@+yI#(v9?!k!iMJz zX3v4rG!)8WFcl7G0)`n8N?nUFCTH8I0MfAn>EI?y(37rX)&Sa7+rcz(ai`gkqdi6{ zNCaJB_mOdn=)*XD4;#k&F=YBa(}w=y(CF`E*nHGRUR)&A8HQF*FEq`e083xt&YIz9 zDCaXSTc(5@e|+0C*}O?<-s4P$6%0@L9bdd{U%uahmb;Y#6q*Ii=iA%Q~C zX~$-Qv?`9EZ3bY7G-IsXFT*udb$~C{!-1})9kI|eT~+0cXjUpPlBM2T%1UaOr9p3_ zqBE|gHfF{h9Zlt>OMczEI&NA>5Np|_uzmqLKa)#knw0s;cZzsJ17v8jCKU&IUAFVY zv&x-83|*vRRxBEBVu`0zD2Eg%hcqZ|P~Lf477dvacmSw{i5x9!7n9Q8k%(Hs>AN~1 z7zeA(m~^X>+Ay~V5Jqt=<%eV2;&FPpDHMOGr5rfL^+hkPO_g_MA*)Nx#Y;X&V`yeB zlQ98j+ctJhT3l$IY>>Ak$b@(4>f7Qj90 zLrB~xtlfFlj|?<>sf`)^3nrC$bzC$ynH=Mlpa*ysD-nYkD-}RY9-?3oeSYvo~64&N0KB@_UAKRO%m3Q^dwQ z0{3A5js)tev4)b3PXd2}IVz;OMjiBkGb%L)5|Mi@zujj35mkaKVvR}QlnH9nu=5z! ztSEr^Z*+yB+{cgeSu`4xlolW?c9`e(Ph5qFXhK~$Rekh!j6Bh{{(?om$GWIgC2u8G0KN9?hS+dh z3b$S7x4j?F!S6|=7zY+Pe~lc?^uFn`xBEd5D4ue2=4W$eQs1!H_V=`p3ta7{q{KhS$MLW0WUn4845D6ve;=C zl3&rbFHa!j14!EA1)nDbuLuwy?L))M#v$S(!qX)z743JgFLIUU$YsV3K}egswW4DO z4!gDKK&Oi!St%iOpRssT)CA6ogt*CZqMrYVYI>%F4$Tj91RLG5TZIp~L0?i=>zDhB zr3T18hf?bo{cEb$Z_^5-tTHU-2p9R+yi@Un_qSKlKVEsqdQmopcSEvb7Kryu^SfI= zMyh{_o_HvISj_rtAMZAeJ1y))& z@&6-pO4ZL)Fho#4E#S0&Y6MjT?xRuJAhQR;$fynB_5>G2dZhrurfI7u9c(;G%h(`Qj zs@$~)F>%eTr??luU-+|h&IXa=0Nf(lq5n*tnK?;E(h;S*%UyH;=h~I7NMj>{W;rN|7tEQdT+3$GzeX6nGauG^- zNo2qEks2kK+XO$*0W6I#& zM5u3uSw`W>ns>JAz3NZt@HM~ETBa5>)f^%7Htj zIf$EL!@^ikNK*%3t%q$~F@;Y!oqPzhkq8{)%uBst2Q67yZ7e%kUx>Dn0&5emsCTi zOv9d=Y?lH!2b_EWY-gTT%`&HI*%NgeZj@(v(t(0^b3VRH*3ecZ#qyqslC!9f3+G^2 zSZBQx1cv918GG=A-&Qz`Yi8xKVXwN6aTJi}%2pBJo+c8nUDS2wnnn@wW>UGb_s@^Qe8W z;^P9Pgm@!IA@|pL%vz-7te{d_qQUCT7)-_(;=on7W&C~&o#Mt}T(}ij?kIs0=M$+y z^b+=A73OdYeg9HEg1(D%fpw+9P@q*A-@JkSry~DbcmKO0Aw^&bJz&4Y5Q7})!X969 zy-uWj_~{9%0$-dW9d9F5^{#(?)MxSw2q~gL*a;Ftpy4JsSgk)%>N@m{M(RUzn`y#X zLzL#uE6zJoW-S#R1q;f`2sTka>e#N>9M}#zVu}yG^Ugly&7SaE>b(W*^m4UZXZ-@+ zNX_=19Ljw8o2A_^@V@(9#iz!VxWH zi|P60@&?UkV69!umVXvbrdAdoZdh60mGDVsHG*PXl{rtkI|4^#J%fQ5%`Yr@l)2;Z ziq4>1rmQj5T(^E;IG=Y`blHs>o`7i98HkWhzA*W5%ry)s0guyNk%6ULIW%1Ar-q$= zD~Fdy|}J%4+rQRr0a zSEtg!n@!&-xu@gJEN0e zzJX(L?((MKDCTwHFc1=UoiU)>|CU&h3#n zo%Oim5%sT2o>hJJ3uQK$CA3+9G!GRI7D;`^a((6}og-$XMik5LFlS4IA`xsmz!$!ft8D7stNQ#H(;>NeP5%X+D^dm(3gI4YovPztZ&|`tB4VQ;RnEL` z?;cK(c5a%)W@;l%MqQdRqEs+QupCwdV;ThYafx)XZ0E2P9Ly=F6b#K)M)#HqbHbyw z5k#;ov|J#;2_;H!{*K1Dc4=N(t;M_{a?2f$eQF)yZ~qtT`> zM1$n_wVZUKb`{>&sVUg${K)wIhu57b4;^ApKe@1*tCGBb4EiExI8NlI{Q^W94PaC} zyzT)m9o&{*_G*WHiHuG8d#dt<%cf6QNftk7x^{`5nYhb$+kL zN^y;gtaykFx+|B6hxq*@4kVKxT6r6UGhfRvyhSn$!o`CxSNakZ=Ln#t;<-1-MnBD3 z^ubirY%7=kZ_ z(WdtYvoyuFi%b!W5or&>nrJ2hNL8pzQbZ(j0yRq;LXZm#oT7V)d8}nopb;-`YqWHB zRaM@%A1Z8yp?vp{RVILISCy{Jl&cf9t#Y*&@FYhwpTWuCrkB#Uc}C^(;O89H`EYsp zWUMSg%szCQ@5kb%3SJ67fU70qv$i~2+T#`I9(>AxlYNc+%=PZ92P3TuteO*=jmToN zQhLK-&1yF)`_LhXZAsKS4tjmjbRhrREJ$JoNz}DgX5&QBx|2)Dr1YSjW<6}{s@#qk(*6aVa- znxAkbXdN)4SoNOLBQQN%@m|v-FxnqDsPU5i-Xo(qcf`7RnyrjhU1Td-DE!mOFO|h# zc@2|aaRQqJa+}*j|2yWk*+~AXA zgPzy4wFfv1FhrWynPqs|?Qe;im#>`6i*mc>epKXc8V?nXqYVv*CaVX;?`S@w&;tel zYG)>|i4!DuQuP0%)84Q?R7(Nl8=N_8=cP0Wy+q%BV0~&G2`$vZ-o7!^nWF(jA~N(U z?2*rpnrWZF{$b%MMb;hfpsq~FQkO52YWqYF@&Ux-C)_ssGdWSXC!G}vj}s0HE3`1g z+kv9+WvvFs>DNG_YkSO5~PsmLERmCB~qTBZ4_<{u+3 zy>8O>j4X20TIJKJy$a5eq{RRA$mo2$@$g|)4yA&iEEg^DH<(%-HK}Iy$ z8MvMQU^Fcs`^W98W1~x|cHp3|d2xYQzhJ@<3d}c&o<{z@8rA*{`F+1IEoK4T+Fh1U zhZVv@u+eFxu(~*RtWKlLKeq`@W#02t-BHYVs}NmTljf}i7XGjQbP9sW+EXGogbx5| z6Z7GO(Ac;)J9BR zO0R|@bMPB;6&r#gyLzZWY{aBXGE@jVObs1|xy+qvl`3-h4hQF)-V6(o)5kWC^`9gy zjg3uWuL`(Ox@d#Ou+0Nge~JeVyBI(z$=o~`(ceU{uE{aTm=fiHl01vIt-i2T@?9Xj zaS~>0FB>kbZ>-VzjW=J9R-I|GbEcY7M#>0_V_9K_ng*Rksj`ccXpOETRF+m8(Ji?> z?M;{|0oo?0z?yhkl>j!=Q=ek@zPdM1m9lMSol^5#<#hhBiMhzRzO1J@SvkPhAqz>7 zqLDrG+}Z(#ZR>YIfU{=SR6fe9*>N>0`!J{wgPJM2a5%>@!(V)L1MR{3HQp8JDvnIA zPOHCzhXU+CfHh8wO)sy*Hwx_o!_7JZhnO)&{;w8?sOk{W8RDe!d~yiIKF7Ru62p(e z2Krn)Xqss_Z_I`F5Vt$_Xdu9ii8WUWnLZ&a8&>h~pVzUBcf4h(+J-pKK?%1g0Q$HX zQPh{ir`2qfVE(kIcc?c-xh39?QKHuj5JT8~D-cpA!k1%bH7Q+qV1-GfxKdP(@8(9N zA76gc^ZQU6A`j0C8KQYA?Y6h%A*ne(Q6w9;}e&n>9fN*#(iyv zCsxzSP_!T1(jIvve*J%_yX|p~*Y}_6;|>h1?)s4c@N1)%>v1^8?Yh7TIb&oe!d3>6 zg;wVr!n88JQ{?0hLKkypJ&EBtTep-u^p$nuc2g$XQ9U=Btx|+-5zp+Nx=={IJPdaO z(;{{c=GcQI+M_4-loUaQ12!vl>G&BpN(IrN&O>lm@^fM4Z-53jBTCqU9Ek(xIB_?$ zj5r|npAug;TKSfg3@dsv$0v|t;l6#?`Xtu*vie(Qr4Er7ti6Y_b7bQTT;#=O5kj_| zT(h{D*pUCL7w48DN}%ZK)sHpKYzB?!sK#}4rqovIo8CE{UzcJr^)|Qa@_l#I(G6$l z#Dw1qOBUz+5&*v@Rb)DB6Mlu?Oefql(CsPA@!aHSzmp?vNMY!@pj?znE2aL^d~yOGylyi`pxSE8SvuD z?UDP(Jh3Bjpul6sV<(>&kqD^tzAV9h;@)`=MDABxjw zR>mTL(5QChCaC@YoJGy;*V;qUjT_^zOk(NCC%j0FS9YrH>mRpFO+Vb);VjvtIMxBK+dEaae#~8$jsN6GBc7{Y}_HUy6bel7)8rq|!M` z*+S_u(K`#ml+OiXD0OD!;KE}~dtHF@UiLT!CFVrYwv>p}%B?-@3BE*U8<%)zPq8!kYez^p@mw2s|9;VFPgDsp;%|M4+aa^^{6pZ&kqTBvH$9snV$N?XDIa*VH|M04h`R>v?3g(i~ zdm{6maLYg5^9joSD8bt*i)uqHL(lMlzj(wdjnzAPe~DO1i7Q}v=u zbz+*{S7y(@Kh+|Z9lhtSIs*TS4Hj(jgl%6~?(QgcW>sl{s@4d3I5+HFG4Tbq-n?4_ zQXmbcB6~eKzRq97yDB~j@kPWB#r)ZMyF3qsv&HG&f*(GFj$^_cZU)R+b9 zKR0LK z3xo!YvTSVpnjd`On*i)OS{-twEWj9W+G5W?4N#j7Bz_>7tJ6QcInuIGAjjj~$)*QM zZSdD1cpXOVTUItjmthNur6T-s$*IO|sI5z1!S|ka7~sU2q;U{7s>e#OqY3xN;g#*G zfRD#GX1Ga(3La&Je|3N6W0j%N=iQmFE&t=350>8etczdj;}77Dibopgwb%<-6QwdN z?5$)f$%jdQIchEPdOaQpS+3uV@}F@@&pf%9ys1keXn*|33n-Y#-#Rm(kXaLiUaRi zI;x&fLoWkQoWzSzz6A|}joFV$+Ep~$JxC3GQHXZ^h5+R#U@`Z#j`cc@8&0qWWkSYo zH1KMdWjeORVi{%cL3w_G!Z5vR(ap}s%fYHMSl`F;l{Qcx_%pkcH~R?lnNii z`UxnBajf3+^aKa)^BFe#o!C-Wm>98I_!}=nHjn=mnASK2GzN{DlJsLgX&bo-zDV*? z!TY}uO|uC4zqIbEeKL@a=LS-cW$)@HZNjad{X3;+%dX(8&v%K%-BMOFB>Gu5kmyN1wJkRI zV(@GIs$-LR<%coT%x-d-oU|e00$J(=9_hQ`G7$UXtPnH47n8P$N_&op@DPUVxv&GO z6CEc-mF%Ao+Zd=9Q?AE9lVcS`+6NiVk%-G9X97yk@ghk2{d>p~DfaoeWXLp3(hmTq zyt}c;KlD)@F;t1h*W{~xbc>G*gK#crK^Qrb&ipAi? zY$4vw?ft1$mFPyqx3($;TtWY@LzOAsN-(unYAA$m^ z2tNv-R0qxo^9U=zv3cB%Q|zZGI(pK8i!?GR7#2I+&!6CsVgrOm74U;s2{m>e3j*L) z7s?X4@yiQN;-*_m5*Nl4-9cRSY~*MO(& z!_s6$0eNu*`{o=st`nZh+=9pYNv+9POhG&&3`3!#6NhOOXvy!D? z!MjavM%gPa`Ny<$BRdtc&+qY4y|Np<%-Lj%KH!l;mPt{a(;c`0ZLzDlTVBkJ=N09r>Lx)f8tu(j2)i@n90G4h#L_R}1f2o#!3A0bRM~c#8;N zyIp55I@hglt}mG&w+DK~Xfb8P);pK5cPP@jSx(n%k>1uGzFx7*!^&!GoZwl-dL3$I zg#8oMkb3$^L1!cyt50-KUctsFJ%mfPUb=bsx5Jr$g3fT)HwSkY7|)5ZY{XC1rM&bp z+zd~>@eRyJZ9ZmXI4FH<_RNpFyud(!`=qbn_u>L}GHW1kK7w5U-Z4Fu2y~fpOmAI> zT@cUxTx{)!WN2GJBlB9ETz_zZ^_cKbp9s2PoYT{}5V@n+ zFQd94O1Nj56wtu1bA@C%Bw$*T38v>W?he|vaQ<6`$;7IXtF!+?S)2Q_rsjgq&Qwds ztxT;H8(IfMfF~vH;>Vr~BY^Pqu4~dZM&ah$Z}|GF`xbZKmWH73t-~98E5O8pnUZ4X=zDJOr#;U(b~Ko7V?kfkR}U z?!lk}FmhZ-aX8;@Ck{zu1jv{iwkUwZ?#h<2;{bDxZugOl+_Ke{FrejgN+IqoCyTMr zNy#-l)^aIJccfTt`UJtw5D(EVyeGqpV#pR1=`KofHYdwSX#ZQLZ`g5{mdA_Y0qng!`j#}-hc}mx&kED#Zm-SAL0~b{;Ktzks|IJ?E<6? zyq#mDo9A0k(IiwcHvq9sk_#+xtM*M`SE(9%9lHu@wuMp9YO7^P2S?!Bz`cTh(fa12 zby{OYDsESs{zlo`Mi*Hk^i)rf#SQ5|N_*KcNS?8^`ut)Y|KftPm9wVO(LlMqvJwhL zlTE$?=~>Kpn#7>~g}lHK^qqCB;8eO|h9HTcFo9?J99O9KXag!|cTQr_B#)S9jr>}2=z^E?1X7a&Mh*=uvFIKY2^>rVNs>knY zMgn#3YgfKB%(~Gr`Gm=8dMgcW0pYsW*eOP(0-I+<*0$v>ZVlzS7_aD6AlfV8TlZ+J z1A4I-O(!jVz6qy=pN)kCrP0HT)NWGpJlNQb5U8`WAIrc7I-j)y-~~n zLhm5p8)g77;5)H8(Il`;ny)uuu!i^dBm|)!SmyOY!R%r z0U+i~PC5$)UXQiTQs(poGIn}voc+3H;9Cn;I*;iDfU>ZjVfWTy_(CPvN3#INru_Bq zpM5N*$Ws&NwP7SGXCI>)@CEUo0s1)z%3eaM*1^rU$b~%nPXrD_PK&q1urahuL8eX~ zr+Z-)j!t3w)E>Jd&8o3a_H#YP$%TQ^NYDHP---DKgBp&j7yQXtmMHwAk%{hip*f)8 zg(49`0BHTrj16U85Vf$D9kzvFUR62@s4iU4eBfKf*vA6tz3EAuyO#i`@0E6} zd|+nx3q-Ddz$7uF=kX72mO*d_rh)~2R)HWMNEa4WeQ9}K<7DE|lUAde-6iP-B;zTZ z06A!e7K{m=geulT{x31c*i{ddL`s$M$P9j3M}k*UxIFUWU$Lt|;g?Q{1$~}vfhU!? z&>$2>jrfp)-aqI{^@R>^OYw;ew!%-%>=tV*xPL-X%m$8}4mMg=oiqn|8fF9K!kOl{ z74LHb3%O&rTwmkvVjW)nW(Z`m#!-O@2qRK`!6P%PPk7+q<%bSW<&2LG=~uWGhvWPi zomhOwCH~lL9S_`5d(jS1I68TON{yQvr0*SJ27VxhEQ#Njyc)5fs=ZKsrU99&-=iXq zH|PQ3{nd9*&(^8^v&>Zwe4jxw{a{D$^#4%djxxS0?2%Nyn|{LK`-mOo-_2qIK07Ds zZ<*CTss6$5xg&Df1`{PRIem{SOb$m8dB6Fbq%g>rLJv`6psI1Je7F6KW$zD-&>Mb@ zbD&_Xa8vK7(qE?2{ENO492JG0YIw2(w*#oZBX}Q}*BqNJWlBFKsBg>mQ(D|EKXnG+*~GCP{T_qx!*+xA3kT zxSX7T+fP$GJ1^iQWB+;Ts^^o!z*N73b+xE(fsCSm6KSga+%fU*$P^x8<;@Sh&bUM_ z`1jBDM8@F^K5K45Tn*4EQm8sHcq3;M>wAw)^IFpGrFu3I`!D$K)*FtdfOn#W3?`(Fl|b$neeX^Hl_zRJ2n>!Yð8w5i9gT%7frE9Bn?-* z_x#_&%j54~MiE!+wZ!rcuNJp9vcUJsA}C#OhAjP#JF754y2~1PleKukN-41O!|&@g7R0wYEVbfSIWj(*k*;h%WrDyw2<1HThX<*&T7+$;+>R4b&DPd#&z zn(t&?C2TakOAkj@^`$78$)=|mOhzl-7gm`0x$2`OGUuq@0%lll(Uj2E&pg{-dgq-^ zw)VxWe>6YKHIx^mu*3IW?*Af`o$orj{W*tj?i5H@oOnz5m;@`=#T3Vi+wdT&s1;t zVBFf+%NE2Y2RIa(X?fsdvzwDx)XJna*+Qeu6MVOK)w!h;RLL3^D8c3Uw<&(i91c>N zD6VSrqNzcxemeU^HU{g-CqPysRO#Yc1mVGj|Gw=S@L;hZXSb~8TBWKcHZ|YY{(%10 zZod-zi3ZLVBA7vZcjnVB9q;!I(YqcTj!s}dLhP=x0ASV=wX+v1v9yU+UNo;E7%8Hi z^+0roH{|HX1J_}CL#7ilD~Bmi)S0es&d`<_nUs=PK5&y4xYEo7J4(chWO9KDo0ls5 zBuVsRRn0DH(dBd+nqi@)YCMO^J%W>a%4uPfkf_dH>@?Ge_I&;4>T8T~z=QRTu4=h+ zQ>TOa27niKXVvzxCdy5BLk5+nqmb^gRzyF(OdMMFYe`Lnly<#GWDBFZzQiKN=%A-r zB?l|3*`<8;FR9Ul@4bDy}H2Xk` z3-?4;$et6`376uyy$YYDDy7&zfV&6nf3rdkK6mhm7L@x}h)Prr zi=>RQVGUCbTJjoKP$-YTqwGq%KLwb2&AaP~bKN0!MV`#j+%#o-GrhV*-H;EcgE`>s z+JI@IbSwqxB6L^`w1U?WIce_pKy!0#jp&M0zTb|hd!1?CxW?pxDegqY7EC;Fymr0E zkrX@5aTT8a@JY)H!5cgE&X7uC^#Ox6=aOVjdZz+1e%Vj3V<@3K-=|n~p#c0Zjm90w zPRHtlZmMVIr2Vnfduu60a}4nQx`o#bqC4uRzT$XdEeScrEA#$4?^3v*MnDT|9W0+z zf@1kxQ6a)2>5@EfY$-@VgDJj!#C;2fJYx4H6Bh&px}t4xG3r zo}b!uvvS_N;%PbDjofLu3*b!hpcBVSF#lUT(feCxsMMg`3^w*|z7irn(897162x7+&_?W2EN!AT3KEfHtWdm6_WpET4NCTlg!&$k3fn?%1royj$6$o3*%KeLfYh7^7>4xnU?ZjW0=weozC zGP?KUNg2(}oJsE*%BfE@d9sI6av}7A;g!RR9^Ab@C~U20@o3HwDU38<_@7VAjDy`o z^8`A`dfqb8Rz;g5OV+=j<_j@ZKBz|@t*mCDoRSc?Gkn@X(o0$`KUE)4wJlHOY^Mqz zKZj0GL&qm$D#R$k1^9`Bw7><$DUN$IK#yRY-YmQXzPj(CWtigIY*jT(r$=1e#TjRF zDnGNy-Sp?d_n9R5GWHbeM>A=)1#>Af=*~a|9=h^zTv^Ts@AeN?#lobs+(e9ix-{S$ z+_5|S+M-VAr&Z{pknxMW7q)6Nm#GB1UmJ@R7Z2){mm!@N3!pOpJWfe~qNmP?-(6d% zc9SbAj0F5C-iS~rk}Y)&8!64p>~^+sV}fx*b_8ne%|tTYSm^^@{!;Go#droB0{x91@5CVXui$i!|7BGXgSWo zc%??7Jg@}E1^|f|Uo6QwG>kM?{KtQifOc`WZ|qTEwe+j<7{A+O<=WOB$t1a+RApls zvUR$ftUpNgUvDKmlxf~E_sc0q^W+Kic^>t-_m<272}}gX=V~{}r}i?%0MVvvC1TNe zo(Cr2wlA#rHGWGPzM~dau{+kspcB4#!K-~n3=%5aHvjvz0H1|G3H~gq zh2?N^Mv7S?CUe%)fq1gHK2d%DKj_A=1iKZ8a&d;+I`8LkRA{2Y$>fSIVNkEnZ+>=m zH*|l&SF&5*Ua|T#eBze;x`u12;ceN+60mhOB*^SW)N2(9*ECEEuSbyFX%KytjuwkB z{?%GjAs`);74y5tw6_I+UrTTw5xeXnI$72=nT9%qw|$ z#IWCd=c%d?WOPQOcgZg0!_0#}30(=n<#AOecN&N@$l$3e!(94=oY=VLd$$F#TSO4& z472USw!aYY@RoRd#RbWY)7Fnw_1rbzFLCXP4)9SW6GU(&KP1)8k#j{OnM{mcq}b6K z9Co9dEuTzu?C=J2zY#r`+(z3Q9^=EF=QwrTMM%_DTd3{%b+wP*m7czEFedeA${%=p zkK6xi^N_XHdWxXd$Q*4InfnaP}DCf{xshja4{t^NA+>a&YULB#RiG6%NQt;Atvi z>@O0SBYalpn%9o=3dVcT$AeAyMDQLxV&YVP`=+!LQuwyDz%O3+3md&6ZbeCU!dUC3 z)jz)4z*G{@FVnh9;`^C%CiQj90mev**tg{}e@-qy58!lZ zNLN`?RPD&FdKpw3710?t262P$`#3E;W~5g_$B{wSQkL*)OFpCL_iv#uVbYS6L;yIW zxsGMN==2D5=2$KwPL^3fkI>|J)%h<0{ErMV&KLQ-;RfctIyGTJ866$9XVO2+4Cm#x z*lVtrX-n6oErf5R`Wv$C1h!Oz0Na(nAO)U)kGf%Wl@!7^wAT#z4Ash?NUn{GESf@M zTxz^7$ta*$80!`H?mGEi1Y+HZ%r`;Ni|E-f>U?38Su4nbUws9$`V`;;rdhFDZ3*$~PYx}Qi zuUZ53-iTgGw#VRW4k;f_WhKIg9&b7}9?z}|EE&G?%A_aQC;d(GSYSZwemTcKBusw_>pR^F22rV>- zn!rl(L_29F7YrZf&Pul73%XxOdt%o5v_Rxr*SNRX(KT-RC1NHwMwF}Q*U&4P0mT4i=jVPXvvIpLVP(Z*NvJs95oW!Q*-fmDd{jwha5JkFOJkcUy4WAcKi5((yu?>&2O9e&- z_g)Yd0Ha5T24zxFOlzgPMhs`(rGsLqG?u3ZV#j-bjbV+&2=kN8hYt&_Q`=X6ZKK&= z>A^`fudEg$&b`sn@i&>$)j>1}7e3G!N)}I8&tNpPPuDtzV>i*NfH5Qy~E*Ts^8QgyiJ#ckZD2hoJ>E#YGXdxLV z>?T*c$z2zY1(&S}qx`KI^!vvAC&QGo*Xd*2$kqQTWdsSjD?KR%UlpXS%5 zKT3!6a0SPf8-V}B!7MXI{tTv*8%*d)6?oYh$+hc^_FdCXJv14~{dClT3izMF`2T(= z|L-UCABpe$MkLLDR#gh&(i;z0aMO21$cWMlj9JhCwu=9Q7ERyss3a9cC5aH72N$Ii zRGL`OfvTwxW*HeW)222roJ!iN1=inIllW7bu(Jj{O5t5krG>Mrq)sFmUo65K6{F z2MKt(GoO6woJGy2J2X6w4#f*QE;_J)LFW2o@TF}@;k z5aaXk9Vbt6vGDti!as?8x90;yk3P)slo(hES$r&Wp%7Q5dESuTgGpuvmDE}Qa05htxvK1~32Y0NUO`}~=I#T)QI#AfB~naH`W+~?TgY5R3$OQA7TqXVC5Wu> z07UyU0J}K-!?2ipm%L#WbSp9X;M{{JKG$-a%FFrgoN3$Mh1en?f`LWROcTKV z>aFrp*pNhQx(mO_<&;aQ;J{srx7dNS+Ul)1eEx{VEM0xL5@i8>ILDnKZ<>U+>IMe< zVIoex2+LkN2Y5z#K5w*Zj2_QoY?QQjrHqj|gh?!WXkc?v99+_b9&{&&jb?mtR>#ol z96v4WEELonN2|=t?7gY;y=RY{whM5jW3!IMT!Uqxom~lw_ULsW*p>#d5zLfc0aeD< zCPb;;Jp>WPb~f{ImAh>v?)Ges!rBw>Q+_?|zIZGI(hYijU;mei?Ef&qArhX|hhrAL z@GtoxsZONWM>JM3jm@mFaGu2J6~n}^;6wb6o1*4ZZ2g?CU`1dv`!WTz6$XH%Rhr#K z)rwqK6F1D&3VQL9UzwGoJ3;Y@X@Lcu8|rV4EE=rD7QG#fG(kM6leRkW!Vyt?bk*i!iS)wthN2Giu7tE> zdI?YUkRSpT2(ks|I#EGV$TL8_r@LdYKmRx7o$VHdRU&w0noC59p-S#GdXw%X*hPP- z_akcbe6-{SJDf|}&Ig97Z0^DUi!-&adr^=@a8f>ZBJyrkp0J|3J?Ng5QDKQh${_yV z|5W3DYwrK8#{YomXH@1J9={l|L{Q(_+luh4z#;!rqwRTMbi!cB|9PR?^I`QC>f5&( z4HnJY%UfS&$NFRRtcaWLR`k zqKJ`n*k%}E;V=PNj>epqj*3i8?G64(6|0cHwk(zGV~~WoWtc&T4vdE@1+vyn+-esxI^S-E)?>EoPRr#86AO zW-2~N8T(NfVNt9mF3a#9qq^x?b!H65Ru6}RMnf?B+0~6va)yhVrHl&uP_B${Dc~Xoo13* zWYv#NS}~!YoHY_~HwbVe03{ese$yVf+Sb%$gMl`pid$eSD!p|&L1D(%a!G5=lwZ4N>UXuFi77Rw9D%zIXv;J}F|_LNdWoG_K=_b8A>V;G_Z* z3E`R~x%d+i`EU{o5n-Wzgt$1wv>Ktrj)@qyv#tit++XeG;ese}fXde-Rt3k@8h>MG zix8cmG$`#))0)ILVKOTTiIv*IAjeqB^{|@4eX#&semDh&b1DLyZaqwoB*BG(Yzt;H z;<|zGbgT~1K-79(IkDQ1iqgLvN|IEj9PU!AR z>NS&iD=}WH{r+?>0B_SM_;?i$nX_1$C*W&+))DLK_nNIq4h<-l!6Yd8+{#!1!qPHT zlXiOv_k3U{aYp-H6rIF}upMegIWSI%4L0VvI!6(X&6?c(+m0G?x8U1}jszW8M8zB( z7D{i?`!VeSH=3uFezwM2Gu~Sr^q<7$3F=00N~|xk0igTn01;FXTyM?(In4kZzWSVu zI;Pq&W(ILF_@jsrcGq4hmu)G3n>>ygN9@kr4tGTdrfnkusP2A+VSSLa=z3hcd#yEl z5?Y1eOs zPW4<(t=#nk0Ogm!69WqOJA3045gkL+_At)ke4ud+3s+c?`H~%dSSovYR~iOpds4EH zl}+=D#W{7t=ARs6ijEm2jp^w1hK;UB-KM6CPvfhNe%RiyJCF;mj7$*@UXR0 zM0hBkt33aJoBGTkD=r`m>$FGXEg9Ay+fyso$Q*F5eV))qW99Z0+Cr{*rU3F1Chg3g zPxAMOp!OC*>mmtT8rgLTQw>+rtuY#)7Jj*87v+i#?z3iIn#$7?T-?L8O!T);6)<%(asyNhLJ6&^3SWuKPh(WdcI|*2Vuy`yQD3(Pd`bLG z6TBsQX%*78*#o`QMd00G@E#G7zshz@i`PU|D-R@V)N-cmEZ(TDl+#p}nm1!4BD+G> zkZOy&c;s&cDcFdpB7O;5P+!ykdS2QDE#<9|~H zlAr&@hyM#L|A0uQ)eHXWH?+)x{L^HdB}x9{7dKeaAK#}xP4OSU@_>l{`@sMG=|7-T z{WqXn1op>|4eWojlb=3x@JY@MoQQw~RS#WFrIfFXk&KqRX0L*b#yJxxsWg?i#&|O^ zaRM=gR2HdLDRE1=F~dXB4!V?-$@LR~r6EFE+UnNx(qv`NzZNKl1z@w_^S*7qx!Kv| zp;baIs46Z_OGBjruWy%jZ?hO-BxX*xO_=@M| ztiBGO^&pwy^DM7tVsMaW6rKyRb8jE+?LL!h-5+1RQ=_*zKcsi#Vc_4cf8sqD;0F3H znDMa_jqUu%9%{P5#r+Mr>jHMm?bA-(_vG~c z>o1)*`8NK^b4?`RDG#s8^#=Lnb3W>;8!&oq;;k8w1wB!gd-&V&Jqdr}xd6QjDgEoa z^(W}X5{Gx*9ssqyO}!PXiN%?jQghLmDG6dcvzWO0nz$+VSS=VZq-I%&nAx^Mf`foaZ#UeOq6I+mr#9910_Kf%3xL8uFYp5 zJt!HVW~JO~IraFId{QYYvyRd@VoL1oMQk;~0Y884r!1^DA($z~kLAJO%z@OvS#CH- zzlOseIGays#;8X{O-T!Yp_$2AlGig5I&ie49~-SFl~-%K(_M@ti6TZ3r=)zM8Xw*V znxuv01Z%TW<4j{2p+<-xV>A}gkHQS0h(~W)v!*f{G@~8z-c(OgLsV z=TWS!(UB^)!WG7e2nGgzAY)$A=LOv{MclA~^hg(~9D>@{FM8OgLFMF{P&DVoSO#K& zl5s0P4(vjO(n1UFrz2)q{FWkCr%D`ui2(ZZ0xU9{XjQ0zOn*oI(ljeGBT9+XaBi@E z9))8?TkeT65qbnj&%vO|G-a5vc%Xu5!9o{@jCm`QR$scx7xB=FQ=rvwiw-6axUV%eCX9wXtoYW3$Rifu)H#!iK6u z0u?Y$<`=;*d(c5XLvCaKjXJz%)|CGt>(1N%WM}SLj z_d%73z;C7v8!D6-uCf^5QZV8oa-zWW-u-U(N~=}oGAv{p8w+SvG)4<+35YE;fIH`8+&I-p$dWI!V;G?x&Y&RoghOtDFZAb^ z+ew=m)3o@nrIzZza7LAsRX{N($rqs}N@>X#$-27iq|&shW*D)zSM!92<Shi$gOZ$oE9)1QOgLC&U1vRXv0#&eATdsi)?ERQ zhh#}IJFN-q`QTb|0SD+kyG{;lW&_SeaUaTugyDChu(NC0C}NipQc3zeEqAGLA0^Ec z9ReRDDc;mH19uuJG19)TVsF)JlKrYn%hq8Ix779|e`4#*1Y*>Kt6nbX;besXs#@YJ$Ixfrn@QMt{pb~6Fm(}s) z#{S`f(ZD&TeQb$LC#RW8L&fpj)L}nAc!;$VTYK6$O7#QKR<+iY^fVK-5Yx`RQ3D!+ zMHESs)b@qhLkzg{R4b|&r|%BM)z^Oq%QLP&zw~|9LsfNLN8I z9A8y#pA@eKND1z`wRiZ>&|*peZ@G0eh`bifOrvF9YjdWg=@hF25wWFoE86PCP_O(Z zUH*-6U_g(vLz<)~CUmEWMxF&wMv~3e-<(&@M&q*~UPCFt(l*%9nR&n#A=iu%@_TN_$nr_b<+IJ|4OE7wmKMs`>xI)<%nES|`{O^#s znmG6I2Dus|`+ib7s#IYD20bsq8}_cN+>oHbw^3!NaZnr4fFd&@7l2WWS!lYFm4D}5 zD9_n9)N3UtTI60-<^^X&s^F?Y_1a89|E}|^##w0GBV8Y7G(;2OeHo+nAznDMdoQF} zR5tZ5vL!DnX!M`P{Oiu=IWtHgSW|}AlfE}C&2fsOQ?dUefA1XTFY|?eRa#VIag$LURs>bc>TN83FTf2=RlcGx3(yu^H z`Cbn9ZqSR5Oc})8>62=hqH2)!=cx_S;~C`_Z$DZ>WM;uTMVqc;R)>ckq}u}!-;@Uz zbQn)Jg~ywS{9Lw`+mQi58n)=xeQ$Q00c{}o0~zC=fbI|f%zoZom`_NPmNzDzzg0Va zt}$2JD$nnpfQ03*A(4Z#W)Zh*J7(;sn}-jv1{A+j2G7u_jkg7GPhqQmlLce0{8T&n zi*BM)kf>5xu0r`|t_JRkO<%@SmddNWS0L}M-YW;Tvdkf5Y1)m^U;wH_i4yp&%8T}C zqED24gF+t+)q&mNl-^|fJ~N0&vXb^ljOSy^LS20~fZlkT;UPFK=zD55gjr0}Y_7#Z zhxDYGsjLfU%gG?xgEtuw_kF)sy(%wsg5kbM^)}AoY5kU)al6XU8vD`>9PZWp4J(JW z-1s$Bl@jBWUZXqg61h{Pn``F0kgb=*My34L?wMnInRwuxqQl^R@vkJ88sPMzd!c5z zAdMSqz-66$@mfjHd^YlefNERWq1nhtn|Wq`E0ATm8}Qlc_Yv|^g*M0Uxuu+}vR%A9 z%|>$^+T`|!<#SQR=K6vfsWL=yiDlyYOBoxumQsL?%3{vnRKn4eT9raPYLKMD8(H?q z9r@~2G$Y&U_ETjky$8)Z(00O1lhc3jvgd9QU>b5^^aDgEcUWeoM(2?E0Y>1hOm+CL zMxcoud!_(b=j8k|xWJp6HgHCQ;=a}pZp1__F_`?%s z`)@{F!0@k^7-VS=44j5`|F1-#y+cUm{#x=OhVovIpL4+t>0)-AdaSxY$h*?L>yrLm zfFX~JUawr)MVW0c$@ujQ6v8cspU2H-%;Shr@gIn6|#;=%FvsqCsJgg{!8{5HI z$O%W2ih1%kpS5V6LOjlIw5f$)# z%(>#w1;4diz}NOYGGKPr^_>q88WpO7W{3%La5@g>_Os_eiu29Nj?>L8_tyug>Pi|^j#}xyIp3wiP|NNJJ{Vy?Vnm#~G`Bxa!3<CyU~3t&x`G*^h|}c$lOv5mR55xiR@^t=_Xk#`~Nm$i)iF zhM@VA-NtRwZL;O5o)-Y%2C5G6P&2G16cQ3bQ$ih^c2FIFHLEy~70}`n;8W<~5HQwX zeGPVq(rX~n?Azh9HB}Xuvun0mt~6Q|78!k{cFZEcg$bx?8g~7KgUDrgyr^F=2)y6| zz?8Rg>BfY@-MK#6^W%BaPVq`3EI??raOZ!Oqa%6Ko6eN$TxyisI@+%s(3@*A&CF+Y z7F{H+W4K)h*-=o`lzt40Q6#hlONe5C#q9G*Osaj-g1Lyik{ zKUZx$gEpxe^UL9stb0Hv(_^@r>_9fX3w0Tf*;R)fZ`6SzZ3u)gy5*q#r z){xsk{mMwff^LjC)TNorG?)C@Rk%x*y;ADO?~U z+w-L0zYt94N7d|4LfDMsevVH02WlX;8LQC)c1f)0cq%U(+OWib+&^L${+|u^-@Wv| z8;&jj(GCjo#}68`Bp)>-^rY`yhSwBxfk+GqO2PLnRB!mf@B{z5q@2PG0{K_IcH<;~ z1A_ZUgfOjD1rvq%@q-2J-|4D<6}ixpy0SPB0R^fTIyeic-j|nf&~1JEzj_2=vGQZ6 z$xFm@C&GGQp}+;Oe$~h+n7EiEL5__(8z&-Ic$rywSyU{Vmo+xjEUYSP7%+?T-oqc` zFX&9Aw7`IHLQ{EePiA;bxn(*`u|K7&eZF-8z971_o^xT}WF`4TC<1um-B^+WeI(lH z0a>{@v4h^!*Sfd5H?jvi5M0Y#Y+TJ|^VP#YlH3~m`%t~TLT|i>>x$eiL||TFrZNw9 z*zqz=V(^P@klVZKvTj$WTwUoyl44!iLsVj2m9h(rIn>2uc8{2{9u9#|{mo9DD|6W+ zFMpn=F_r5nxNK1M1+YgnQueWbtz22)00^{PCT(Bgp>T@Pc}kM%)dvzdOu|dG=_O@O zioYAv+WD;b6gH%yaoI}$im3~A%#0LMXVcOcK6?yLteO^I%2%4EJYrhgc4cIv!Y&Ruk}mJcOcXndYuJf2QP&25;~@o@<1d^sDqpi1F)0c zz(G~+x$oMeg=DHO^ozMsak{KhFGU)u3t?FFvF_EkVfa0yU2isTndxw!#7fR!WXG(G zuBy!uPQxWVc)Fr_x)BX#2tGx9XIK=~XCtC}xI#(CDi2hPkMzV>2pTN(cy5h%*coow zI?h0gn@z9Ol+DK~Zenc_CP}_n6>y%dDA)8*yVIQR51EFP@JPEBDB*$LmBUGys)m<2 zszd0J#z@$?9se@8U={~kAU=_j2VGjVuPa@aa5Ria)~LTQo^GXe*hURsweWo1Mo0qc zEV^Dd%b0$)VQ;lqFHm+dSE-x~5G)(zpP?-fhOVp`eh})ENlw)igIGR41sJT^(r4v! z`X|<*m7->VEREQulS{GXF)!VQ?1E5l3Jo<5p*hWE)I`{?@Cs<^5I0@Mh#pZqs&5u! zT%xk%k0X_sl^LVoL%{OvO3eseN;k>~q4|lm7w#sxY7HQ4%HB}5r}S4Tc|zOV*!~$< zYcAep^Um8P_Ac8cXwTF41q21TxhA=g-DHL`HYfLADZb+YWN&zWe<7+Ud3N_@U7Mid z?-HQ`dWI-JM=fNkhqqiGT|9Gl6*50Sz9e@UQI}NjQ8pEC*sngIRlqJc`9IK_>!dB1 zu9{f}SQ%`An^;4y6(=S%nCsX&XF(rXW7FxDpf+p8)(ZMnrINBk0U{|#Uiwfi%H*<$ zr}*rm86ITJMM~{vODy9IT5&BE_lKz#ih(7isOF7QZfxg9d&{xf*@_KO^K+Wb-63i5 z&%mrU)8r$C8xfBz)-`2WRG33$t7Se@^y6B`2E`F!t6cSQ1yK`@aIs22h>dp-`8{yIGna&zNakYTJ;|4VwR^L#rAVTIs^S^uVACD>peQMNp zv>|>j+0{U8769y+rac$uZdwtbk_(A5*l|CYO0>yQp`}rje23!kKG6;1^LCAk6*bMp zHwxO0S$+ZqJm=XM_;P9g;R067%$#0&Vu227vaVBsdI4AOmoC@xkyR+$ZDl z4As65gh7%gqI<)V`k{_+#q!8!`uzAIfsz=w;5i_>Ricmzf)rEH*h(_DSkZc#Xs-2W z!3wgOko@fr);!2QYr8xT9n(OVsNis`s`B zX@#q1k^pAO2F4z4H4}hH#|Q0-&f4@R-K~qAGGSKir42Ummdl{6H}^GaJ*7-Q`WD)c zCa@vXJ|^Qub^r_AXFOAEV77slzq!;D z?vQP1)hQm7k~gmT7S`t5sZS&CW&5e{sg6?NfWVQo4}VKTFTP_3~VtiY`hK;5aoF}?IB5&`hPOPO-Idmy-fz9j>tt91DJ z2`=frg%h<&ScD+r=fAjbTL6O&{2$!E>VEN~hyT8v5-07*5U>N> zxWA~~XOBaKeiHbD#v;;T(fFz2)Coh!%25+XgJP^e;u+)787Ipb3!~XNE%en?e<&Eb*!vfTGus5#j$AZ#x@^qx}{=@G3pn0%3O82Tzxb- zd|gbAb$MR8{lGQ^K(5}1oZbwb+HC;(h9pjJgy1jxlVIAW-$L*;b zbns942ta}HXDbT8uZw1ItK^0Xyi+~A%B>nyM}@O9NA;t_2Ozu03hOB$=c&QnX|Uaf zH@}Pea1$kd&Ra6{B!1)d-rWt)tr~$}5(QA6`_kXm;Z_a8FN*@)Wn=NmodW>o+gl4m zfZ~&^lBTcw?XK&s(;ME?8wYspfiHJkkS_9`-#1uKkm5r9Ie5~Qp%UEdcoawi9jFTU z6cngY@rj`nMuF7V!~AOdF$40}!$|P0*Wxvbu7)LwShbSX&3mGRnDC%+A3MvRri8Zh zNXZ>D#i&=Iph%EZ1mq*?lz`CwNZh*J?;$^Ihf(Sy`~CoG%*f}ynS9hJa6<0}0vztc zs|K=HrXCi)k4m3DEiVv(jgq6vKCL#+*SEGeIG3%Bg$|cymu#Kwvo_xAKi&8wSKBu~ z8iI-6wPI9zp_~eM+GM~gW(?WTNCZT#n{}n&LJ6q!!Qy;~5XTC44*_6yR0y*iY9agG z#edw*O!=>f+uF(@`LwjPOK=SpN|(~B<<@7bU zX-1WL4e}kTm{#qFDgZpjbQ+R-Qe%lCGNTFBZ#2a@CfJ=#82FDv!K#xPh?Nnc5c=MS zQesnw`PXbiQkhjn8;kkfrZ@A`)KI0%8V4P7to6;<+81Syr7dk*fn^xm>;d_dQK9A1 zK)?0Ml>yH85vh36`8(QwqrnKmf$r)(Sjljd!}=CbTpDh2MguNHdtod0Z*2Mb7)6*1 zB$cBLHA_vIvTczSkD5eeS>jjSu}{Q>44mVw z9?{B01UT3dTqxhp!Fo7s419Xz^tt~Cyz0{r4y1b4QWSf*W1Cs01$l62j`oK1od3N! zuNB@?ri)gDs{y3b$K-S3|0L^GEF!{+AOTr5J9bvfiOZ4eiK~ufRBdf18pSm~q|BIE zj5hDdUGXW^BRlC$pU2v~>UgNamT}zN z76^e8>QrzWq?^a8vwxVTvPcax;EvH1vS-EvowxX0Nhd?P!tQ)uEU&MCMk;cj-sM1O zX6L41u!&l*&V>Hj&-IorG9Ss1;FIuH3g#Zli4fB7dq3nEk892y9X@aZ-Vj)h0^?zV zTE!RkMgzbTPzk3M<9!3H$T@PMK~mtHS`TXLUjI?l+FLtz#3bi1Cwutzm|5!2*E$SQ z3Y@~UL}(P9-JEL9aAaW3y6G8JYOVB~LxTEoodiCdot#XtYjVmUDemmCHSqLhOVOKB zr1yx13TOoOfe++dp(&DQyBfRJL8VitKS`-=g#xZ4#GkpFesSqA9sX{g%VnXE+iRjc zHI@?8v+9Fkc8!0tJ|MS|>lP;YrNT+y^;dHm)dpVp{Ij1C1v?C)BC~q*ZtSquOi5C5 z-Oob`XV8ljPOrbeYOjKHGekrZCuU*%K49^R1!=oH#IJd+|HotMau6TO&qAhQtVelv zbu@s&A|<;^&Q!Ehp=*!j`4yz>maGV;ZK4O9Qjhv z%$CA<{HZRGon?Qb{GMf5V3*Ed)KQ;O%g0hHMpaEfXB20tto#*9(KL3Zx**jefce;* z>aE#WU7O-6&VpUL5N}nx5sh1T-XBNGLs(^Uz*vy6Rv^G3bwU_%9&0V&F4>~soYoG2 zxnQY*_$a3@Z0Ebw&yll#E!On`$F3cU_a=tLq%*2IM|Oqo7%O;1ZrHbCGMtVl*jZVe zK;)uf+SpM(xYI@(1m@6jzI-X$bv!mZ8kKiCc$czi+;n?IAWh)XSXSQ5&-{r6H4@Wq zFy2?D*XmK6aYR}Ob*C@RhQ%WsnBfd)zEfPWL3gdNg;1c89;5kC%(j7r0J$-BneSw8 z9eg3VcB`WW6BuC=YciUE!H*gmd_dJGu`W#$DdlXG$F*A=!CIuCVoOv!*+@H!ti=E1o`ewN1im5*`gen@pIlu-8_^kKtzTu5sC_BLfHkLYD+Pj1` zNDrbGWC`x?$IOhXB!1&chso7-PG82GhkGl;$>nA{jZ0w-x}rbwic3t#+&DiYEjjb( z=sRrtr`=HKld!-dMI+GOjV-|bHPO!FvrOu$ol9k3O+2T8+D&t0{^N7JZ9eqfxvWfh zT^zhO98!jXCug*-lU32pA*fIEG_l;;F2(8N4(|$!fQqjt$MFSc7uPc!I?bBG)B|`l zD&<2wN^NA*l(6Job~>dZCNX(~0sXm8PXN!iK7fK6Glwj<=BD_w!w8V<@dP0;BfPiwBz39$1?-V>&%h?pFfPF#&v(k*^XRG*q)<;4U7TAj{u`nA=hWZ5du1KL zv&Kin<9;U7jSCB0Az%O#*}4TUJm{io(Q8n&+Xl}Y`zMsqjAmbwpi!#CHd?)t1qYN4 z;0Mz`n2WtZXpz#YD2=Gl&p3hDoYl$3it;t6Ou3nq%+uiwOg$n_8m7TnftCfB+=_0| z^CDs`ag=X0FhjQKT#LDwdYvutn9cZPOl$(d+r$xyHc)?pycGZl9Eb-V*qT*x$zy-U zyNi8Tq5J>5Go$onr0j)fR=OvU5tYJA?F7Q&px)_k)m=;G-eN{HGE|CA1vTUvn@3f| z7fai~eUMPM1(w#;9p|whYn2YwoqVu#XpCe@;Y<-a7X{QQOb~d!Fe^?K;!jbe)8tBZ zMYS!cKe&~%Mc4yMW$%#0tFj-bNy^(xc~FPu)jXJP;q@r+CU@1X93oD=YU9tp#eI&W zZrr&i>3PzUc)OlRf{*Y_66f7rYClG>qn{FOGqZW;;lKZfVKFi#8+ia-!que zv?|>Pf`=Kp0A{#!0yZjI{e_l#f+j?$?GL(Zi~(M0zwgPb>fd(#CluyS43?ce^#+bh z)|d3|wfd8^IocLAk#f_#vF%nLbr2f>Dl|?d4w~XNu516`8$E-3`8x&lB02PHswFfR zm*5j-JTqcNZ2jdVPAed_^^3>p4aNxo&isrca72xDL#<9<;#8H75f_x_i0*}yVla!f zWEvq+Tmul$lC=8RG(A(vA;^=6m+Na-s6>Av*_F7&dKLn%h4^z9WCBGPr+SOv|qQOaUb(w=M&OETV z)lKxitoOEl4-6a%a8aNMGQLw=8&c0$^HMC8%K%IhdrC)2q02c~!}F=dr&_WfuX=-D z9+sJCzlEY{k4P?B8q_Ba|G^6s*bEeR<4hdklqHGE7RD3uYln`w)n*!bn~hW!Lg{mj zTC84JoaVE>k|Jsr#7z?jgnd7Rr9Qktgy#lFd4M zyc+_;MeS#^=JT&S*TZoe3&7a2BA-NNH5H&R$iccYeA+jH(_A{Q{)B-gW9YEX|NQYY zk)mBX;Gi@ym12FTTY1S|1-Z$(B&C1R&-~1OX=QA=@USY6VMt$r)XjO;gdi!T8%2Dd zozNC6?7*#cP^moD;(8Q%nfUiE+zW8yh*SNT#Bt(p{VY$&w*#Yf8XakL$=_N#Oqqar zKDJorLqyMHhjw}Tm}hi*Znj&|)|pGF(yX?3r#6C(`HS!2>hwBv@xM zJ74bJ!HkAFTe-v4;dvUDg~DvR-!7~UPl>Db57w}-N2p8==xM3Eka2Nq4}uQh#okyD zFO)v7SU#^DKCj?TAEa6b8pTOAb4q|tft@W9(JI{{Pk60cuk%EFT~%+f}Arxm0+i5cNo&EN6v(8ANiAjp~%}hXo!>Sl# z%idA{U(8hB#Npqv1IZOmKTmPtm^ji+oto?T2eaTX@p*()wg)nXJOj0oK)OtVT+I)q zoDpbb`4o+|=UA&1QR5tgsZV9QrY6j!le0`%(XAYy*+R%vBMy<4(D^mPd37<@1f1kV;A(_QoLfQiqibDD1Le;yt%j9WZG!0=h*3#IInsn7eB$_ReZT3dU z5d*8t_VsygBxvE(q}qSXO3<>;WTfnQzFKc z&~97$-NIA&##e9nTP>PZ+Bm1>H+e!L=jfJiTGG+0bK5@{IDYHypgLOAxNInGYHxaB zaxYCg0dX?GMW1S}CWo`$dLfrxUT|Qo+a_O>`gavKc>{Iec(npI7_gp+q7Jd5B39-Kr7+XlAaJjhRKbX1+C>s1q?R5kv#;NpNbkB!z83i9ECCyd$8(&eb>+fFFatN-oCb{J3l zRnbr{bSV#OOHQRS+3qj^i*l z?*st`59iKpNOz?N#NL_@c-9>zh4sQL1_T1++qMbZQlBX;u?aqE8Hz5HC7|%=VE}Vo zT#Vyi6h<=Qtmz>w#4)9v!aj%Vuvp07Q~#2SZd_(YyGiP!oIT>nf;BV&pj140XPfq1 zp^8E`KR!_IaM8e{!Om$<29L4%BR?Njf(9&wY;J(gAxqkHG9oR>s`_5enIx&C7>0k! zV>G;T4N|j6+f7r>Fjl8N|M*9o`R5JAeY-MyzDswq5+I;ZKYsrF`QsncWz(4xg(5iY ze=<7y9CPB|8J)x=DPk{~78=TBKWoxsW6cg~JPR0GMO zY=dhvL6E5Ec49q7U@D5_7&d)ME!uIrtthODg3)2Sq}C;dbQA;MwK5b*3PAJNbVpA` zzC}5EZ*r-kX5SsgYZ#TMpNW-#p+sb-d9W_o5W+XtPr^}&CYviEk!jXMcPPA@e|)C$ z<4dct>YLD5zk|7rr%H{yC)zZ6Fp}0#UL>8M?eo;iu)HSm+hDR%kYu1PgxfLcGMo*A z*=biGSkw}lV>v9fkO>BF#az_)`T*iV<*?Ku8R=tgo zH;X9rWhbVWmSN8oj#pLD2SC@lDrWmuXhfQ8m#JPYd`=684~H01@|af{k;{IVMCBnZ5A}^m7JT_)|#npR)u+aG|7M_)4yYg)>lRW1%Rg1(j_AiPEk|GJ;2yLla7M+NL}c^cnEf4sq}QtAfH#Y?t_)A2P!ZgM1cWWx^{f8?78o z;D^XNAeUamEXjHRhLhLCcJg|hdjTI*68L%bpZfaD|36;eg~|UpgHaS%@nE2K|Hzge*aQ`f0G7?@t;ePyhBTfRErNBtKjs*uzvd!bb0~Dx1>M9(0eR;*6ljWPxv9KiT z%Hja?*V0G(BN@Onvg=1%VSz-NMRH3_3IDq1!Ub*8m}OZ>HPDKxSBtCsex*(ZVEol~ z>j9P)X`4?qm7KMC7O9n-YgS@w(_2p+WNM8h1~v$rZ<|jaS8ocP*Q<3?K+A;sJoh_y zJf1*0fca|@Se?gRDjwHsrIs(|m*-aakLSQZ8}}O;e9z}cLThgkeb1$!uKuU}BOSBn z&h-}ySiqhD^QW@)XU4jlT;=Cx7~pyj|3(JOdwYO)tO8%W^tah_45+&rkN5dd*EX5= zJJqKY&!<)&!1P(B|u9nI;_l;P0Di?>$2f6XkpU{u~G-ML~6`QYVV~ zi+8C~mhR`NBaeA5lbP?URrssZ0x6s3P6h=R&-9DqyZ1ZY8YXzoj}4himEVN?ZPo@L z!}8#uv6|{vkXoatu3$I4=BC8(FglG&I^i+KiZRZZkp%7b0lTAj$!9yn%#Irhr`4sn=CWswCzs^Mm?r+X*|&8_sza3fxdzt0;Q`lhWz9QJJ53jW z)3CzKY1uXrj>yOWzO!+1Oz+bWQ|tt&WzUdeAq$BsOrEGSB(Zgyk7;saIoTC$^p&ry zPoc}h3=^*i#bK4k42|4tyCoI;qzur4ZZ|t0VKU zpO(aB07qUg300LNvg;QeIB2S)c&-{u3=nK{e@c%`B}!>dAT9{)3`gXiu?h!}s}{m0 z*hG-+rLd$@H}Cv(Gm0EIDU~9j*0XC%9uo8LvgKJ7#fx-=^vk4;mzFTelVg;6+$GXD5 zf0Y+r2T$fHGFi-MG>=%twKW7#A}?x9N`Z*3YK&hNf2n- zKO-zI5g|w@|0`Z{w1B+?C54BP2sVaR5LH!9{gR|E8XZ%izv0<-Ry?KVgooFW+GvAH z>DN8Xp+o4FB9e2X!)l|2eg~;3Es@?Z}LJ)^=!X3PMtMtgeq%KxSm4w&Tztu41k8`NSxK z+=()Zf^IhGL@Y6UVR_t5Jv)BkYA`I7?FaM7%+ty~lNCDsD7CNP$yu5>_>m@Go2&YX zLCvhe;-xD`YThW@rOW_h{BIm;3SAo(MpL7>n#l8Mgto@~v(P#K&Lp;BP>itUC{h_ zpZZ-)TRJCVp*T1IvVbN52`_6cmLz5+S(+$L{k)~}pN>}05@Xz_?Ip(=G0Vk%G)e;N zSwtafky=Bo>4Aw|G@r@b)&0zB|{-v>-dP&<-#xx zA%#hxxgZpu>3F*Wk3C_+&%s#mPH302W_P6gqGfKW1kbC5emt%7fj*ASPza_P z8T2N^5Gw;-Y)wQB_PBFl+^9Vo>hZdD67B__Hsn zQ&b2ww8|5}&1A{7J9nCIcVtJMK$?To#P}EgYmEW;;Rjk9f?Ze%w@y=>a*sFxH=e^Mzo6=f)+&TCL3*>v?-fnA@?<20Mn! z^Gh=9_qGffxR^Ea^ND?Aob-K6_*D#bDL=YG-VAX7n`CjjtZb~b;W?zqi-8#Xs`eE< z8k86`gSx`DY)%_Jv}jTTB2H*G7F+bo_(c=0mJoGCsEzF>eqg_C9VYm2{Bj}ZL`N_$ zvs1!v5=X~k3kE-A^&C#e@?9G{j+?341iVhHO?4te5#rt?E+i<20^l~Oq{%;fwr5~T zifBgwcILqvYOnU{t}_&NZV!dydaI(F8!q5Sz!#KKyD|&)5 z(>N&lvKy(+i8zvNhcBF>EgSy%>TdX`)h^I+tUJS$Xsaa8t*A=xNZ!4R4hiYr`jz6 zZHHK~3QJ57N$c=Tw*oBkPGy^D)Y=z^dJ%ijRbXF3oERMiDMb^e%G&yg=;EFpEGgqj zM``Hs%O0?553YJojN5IoC*d*Sv7RCXZVz`RIrN7waVBcl8*sU;DIGfO#Z75Rs zTixEKY|%v`Y~kBWFNsX@EUQ z+ZsrRNx#0Q@A$COzk>Q*Mi;v)#wSpravwH*Ya?a zW95-qIY$=!U;73VRJQL{T|7f_5KSvRGE^im!VoJY#`=+Hx~;d# zPe=4kY{}ZBf**IP+{j;@x%6YuYt-*1vh;z>CMMgwvY=Z;Hkxpsd!A$OswR67q_!P@ zh`D^geaK{c24I1w3@%;(jMsw28TobKti|Qrg_J(y$w6{HBC=s3m_}FICu=omSBmKt zVjaG;-UwX+gE3)}3MkSu?_?8rX%fjU-SvmL|4f6e~XmuKp?F-G8pC>(6v!Jnkoxn;3 zn3m5jhdWyb#69NCIszSmE@fYM9;mP?Ik{i)p5eRPqXkz7n|p3eF6`DiXjlTzkkp4| zH+9oo_pxu5ojA(?g6kPy1l7agcv+wG($xXAK=H1rntLF^QPbkY#KeONV_+#@gXLx3 zy6jJaun)3ZT2gVIg)+>|DE?O9((9`IJF&y8iCrnUu!UOJ$eb6LL_x!iH56r|DQCM2 zFp>&k?UqGdh*h!{7oWBxRqM&PySd__fp!${A{HD0ZJ1_2;veVZQ zQ@~(ck?FvMOzyaovSZs?zUP?j9ESDEX~~Tzf{2w*Q>m#1MVyg(z{+f*eT#R-AFUUR zMs6Q)jUbJ-lbF?OVs&u-h`nVVA7$YkKWl;<%o0OIz~lIW$<(j;-59iGxg$B!U#p>m z^z{cCz0yShlKHHHZq)fxCL-1WLdvZuy3NgmsQo=f*zdY_(#j7~vnUzFJ@&=`s`}x( z@HXBdmUl4c-MR3osp`Lg#EV_9#Mh~sen1S%8xjQ|puS7wRL8*TgJbUrd;gZ^_Jx++ z3p;9u^BJMDs@$8b#nM0%@B=>|fQ6Zaub-O~vy}mOrpwG#jH18g=?y0*s=&q!yV}N+ z%7{uVx~BI`r@dip{rdYMml*a$*P5AI*)m>p;?0i=1oH zJIDq84%}q^9H+8l5_BwkVGuCKgAzUWM_7lXswBGl?mSL2tIv&WYBk;TA)pr~Qp>6w-Fih>5DDox3L{t3``OjjWe4m5T84tO8aKAdXQkTd@$R7 zZm%odCTQ8_IAxWzl&z4BMNlp7S}oj+fM5;U0vBdq9}@aiq{USpwI}VgHrCYtD4lDf zi_@V}s3D|Cr458lI@L2Kw<}j9G`4z@s$5Kal;$)-FHla-VC+g4Q`M?rzh?*8avBB* zhYVog8dKL-Fh5vhhA0RG?m{PCJVkc3G8J~D0(1(J%%^$eA@_YO3T+;&aGdy>a;Cf z0KLW&DF|Xc(+;O4SSdD{yxC}je^!Gdq%-0Q9VKtd?lUQ3aRO~~NJ9XXu=?w~PNxg_ zAGPbh%=!N^?f=kgxj{N0w11g)F)LQYzqTP771FrdZ*^=A$^EZk=oRtrV8D39? zbMI!7-Qni>sOpOMhvm*Ap|tKo*ci}gGtxF;rLt{>Zv#{%BPdAa+5aQ!8-p{6x^6RZ zo@ip*p4hf++cuv}Y}>Z2iET}6Ol(hdGw-ds?^oZg>K~`NPjyvSuXFa^XYaMPSah$}q)Cx80WomyXjlS|_q1f%X?L&wJt;FMg-tFhQFz&iva?(2Og7|e!H(S0&D;|l zOS@7wGtCF$Qx0CKhcJZ## zz9?b#n-7{9Pc5h)7=0XlfN^hQTE8`_x#g~e2ALOf-bXz%<0lVUo9-8J@1R^%F z8dt=`Dg%3i4rry0kfZ8333J})G${SA2m45d`(e*93eO?)Npz!FVnKfs zm_v-WVT|L6J!bCO1Uf`g?~zlSQ40fZkJ!IJLPm^6+$Kfs;w*IEMtA*HO&L^S3pVh;|^dQ%M^?V<@Rv`H2iUC;`6>m+0HNN6^T}n zs8uxe$CXGj*d=y6vDyEMp`cp`nO>2Eb3EbjJ(j++zpr0-YS|C^n6~=%Q@6Fo>@n+# zd;8GmsW6uReLxpP{Kkz$C+*g8tYWmC5G4GVugO>oD5^IUg{UXvZ8-WKp`#Zs-3Nu% zsF$XPS{#$0(@mP%5D|e`T~`!SvB*}sSApowcroBN$!_?qcKmR{$y0Vn{ov*-!BK0x zCx5RJw7W4Qo#wcFRY|Znmr3>)kH#!D+ArcdXG%|UM|xgAszhu#(`i$15viT}z5>m9 zmW7O2AV76f^hs{HKEatMS>jGZ@K?xSa7IXW`ELFJL<4*bIrG@ABUP;pE+W}k3`8}K zv@AJ8GzH|mD6s+9&G0O>hA2*7m6>UHT8+N~u3U$1KRi`%zNW?h`rJ zfSiK#*28%eITqB(32jmn(W|FbK~0`8^#~ByvhP;apejDu=d2oe6l#59Nkt zn-ttoZ!a$mt~=gL3i$~XvNfuSuc!bWnAHXoxLa05Ay2446HsA_s$qpzslGInqC%In zD zMT_j5y;@~n_Hj4f!pr;KoBN6z=!D#_1npeL6Lj7j(M-}}dhaLZ1Ww>WbTrsFCvLO! z@WMo)9~f`RHm0_)G?5!9IdRyc%fsXvj8fR@Jddw^E{*|JcCcE|zgwUka(7y{|4}-+ z|DB-2ZFIzqAx8izw`k_OwLp@GIUYXAhY4Oy9=!bTUGXsY-@BWGt7$<`z%i!1F>@l| z%n_+l>U7EK0Ljz zR<|^@4~eKJTqUEj?HjI>o9He*U8)dbQX|R5EEmH`*k9C*6N_^AG_X`js;>nSpqYt{S{Pcg5 zAJkNh1kl9+2|$3f4jr0;kqw|vblkVTZfwQFxdkx%jkdST9w&3mz*}G-u8*9Hd6~G_ z)8-0*CqLKtlX@|-@>)+v&^Xc_OE%y1IrU}*;O0ZR-yPhYcz;dKi@YU<7>B=F6v`(O zm|c)JFRIFIBq-YMgEJLrK`KE=jJ1)_w~6K{&eWE?VcWCijsS@)f{p$!JCRiV$@1CPLgx^{<)Z*aXWP~Rvy!F}v~PHYVm zl0pR8t4}UINRErj3xi=zk~OV+-%Y8u@u%F1WU?i!pl%^jL49U7htbS$#!Y+e%;skC9=waAb*qZj&2VAE zR_1VCIB2u#2|@dr=e2|!w}tAha{|z2$*sw7HZ=5bia`{jhpnkQq^GvljRE%4qpbZK z`z*F`a`UDju6zBTGKjW>1XlveG(df5au6=P69!ghA@QRstR_^An(8cKc-yMe!BoR7|H(oW! z%ryxLtMYK4XmCN8z!}Hc9rNV4z?@sbe)$I?91NlA84Vwey+Tl!G$%4}ffc~#$j=t- zB|Kw1ZUjbZU*o^+<#P1)>a>8>IzEr)uk||a73k988t4y%mELUsbJ_lDQvbKHy_XOoU)2EMpp{zZ@wsWoyugsql^R2Ki2l?dl6Cd~OISH=UE+9!^U`|mMID?%}q z4g~x^dyUE^+vfN6il$V@C`!zKZ$MA2?%;$1Mk~rn4+x-U*(w_wH#M*QeTNWjf>4I_ zQuLw%2)zpoy|U@9(K(VxQvYoP7rh~b_PP-dw_gzIQ-)dXVxOGMoO*rn{RB}M62pf4 zLswL1knb;wB2AtmTlFJK6b}oX22LCPgj-5vu5jP2O{w4c(r4Q_<}lFW_ve^` zV3L#=Ysl?r!37Rv;bkkha0yW|$Jr*EsKZnyK2dwfmD5!Sl+qK}>}((AIm#{cDL$gG zpsnTL&mrFNbU3ewX*%AGj$7}aG%>}G)1ReRE-$>L7UJU%ZweKU{nc5v3K@I1CP6fj~hx=KD#puHJOTBM37}1meC9~HmcRA zlg9Hd=B;`gG_zA4bMn~XBWGbMtQM^Ysgw14rD`R3hHx|Gl)jy#B&uNirIjIq9ZYsJ zzLz0kzg~;=Ht4#Cdki_49h{$b_kk;P1beq-6ha#m+ zxcn|%bh!8e!*KQHU$(nl@efS*=(gFJnH!9x`WPW5NN{Uhf}EbpdMNejYR-`}O0tHU zO1h&zoMa7Mi*xk_YJorCSX&K`sXcYJH+Oz}#aQhI(pUK1p~iKSlx99?!;zDjn2)X0 zx2ZdidX!V$aI@n2qFvSG<>CtI5Yff|y~Y-3bm5g_qVa3zH108mrDep9j7epyQ6Xp&qbS83Yl{ z89||c+P=_1cpG`ueTg|A=H|s&Xj**E(5h|VC=k-eMFBt zO*)&&xbdu0*x`S!V5F!wT~H5L+-9(Aax5!{hGtm5d)n<-3|h#OYSTsJciP3fgFhYQ zgzM-Pe)e0Gj05{waTU8Q_+-1(of*Zrt0P-c#h!Oe*>>!1Mwn>_EV}W{y&2b7aF%1#ged8k|vV<_eBq>VtCx0(i-D4IC z#hlD4h~7oLfD48G^uK)s$LpEbMumBf7VJ}Qv!cv_UKGD zSprYi*pQ`g6r0dNM_d_N7|rJTg(f*23d@ZqPfok67RL&V(#P z-yhCLTh24QNBlFLGrt}Ol|hyw6@$om4tMt(pycg9C{omKk#BaRamx5S&v)73DlOyblsw|Bfma>!Ovg=x*QW&U zhc2&Q6+V#vCQr`swH$~4ZioNcx%mPE`(q){^(znkNj(hQ2lA}=fBwFfMr0!!d{<@s zK3^>C82{a`cMz$gttY{U1Gb$9Q4-he-Q5rl(fQ)%{?M|@WKXF)*(JZehOf2p>doY{3*A5eGm%qUR z`%wQJQmxy^W;!O4PAwdnnQCFfhCX5E$5S<{o7mN3aL+tX?P?I>D)o!V4;t4C9D9gA zvjoQvpi-KNg@7)`pmOiS;8~ZZ$wJxblkxyj!>G&Er==E8l^f_A;8yrurQ8XznJtY3 zeeKX;6cdU}8eMCes?+%j1}SaegFx$T1AOe?1%FoZ@>o!7LkXG5`y7{Hov_D1aw^Z^ zJ>v8Qijn4ZUr4H-i}(}PPJp+6iVU(qKFt~0HYK{Pp}~xVde%&=7f)N?xS2i;f(HV# zUHTnNipU}ZD>upNRj36wbv$lt1^}6|?bB`rgv<55i~~TYT1K0`-oZji_(B?!oEF>+ ze~-qCtTFF&8{E(+tA?y|(0hRaz><``Z$zH%BhQn~Ovh}-I$0GM-b2#6F43KgOx+@P zqz4+c2qYGMCvnI&4>}@bxO=GkOb!q?4k~04%%v(t(>J|~jIzz~&wk`KE@B$vUu8lW zAJbTY$eS>StcK1#9!hg5XVSSMwYoDdiL9_~w5|5C7Av<3aBg>GeJ`O*x-If8r|8SG|E{vRx%H1mm-TN7&5dBkAbps>sY%@CH9Wz`&rjque^LisjMr_#e4MMHkE91@%K z{1DX*ad~cv;H$BGB=6hXFt0;~U2}$wm)UWS6UC961*FW3%Q=`p4%LRTh*JAsbnOF} zw7|R5v*>vtW~tzMwxh((mTG{yG#qiFE1F6#aN@E*=Y_ zJ6NO=V~jcE2pcNFp}U$t)^Kth@}F}G%F7@MgnfQ z6l=9h*=m-NgtL6dP|0=s@;o=78yKPxHQidQBl5_L@ivqio$7|IbFj$${_Lb=*lJrZ zEm=2wRk%4Nl4rWC(y~{g*cqCp8Ep+z^)TQ>+hT0yl>3n0AffZ8_7Fe-e5jd@;fuv zOIY~7M+}FC4j^e2xoaSyU_+7M8NxLNoXiJ1b8V4mr(Za!%isa823}cL3-<-H77o~S zJ4|ZZCi81$UuVIiE?m z@QJ2&w2YKi3uiYP3FyGOyRx_>uNQ2k14^MO_`5m2a(gkqw)D z8vBdW$WVVa3=dX3WTi4KU+}V6p~BM5c1T;hei|7cDiK zx;&eGFBfSjTXAh=?K^DS!dQxO4hV_G?qk6)uwVGSu)Mvrur;=p*a>d7I$ns~wos@{^EXgX?c2r0lf7&gh`tIY~@>*CkP&{ zyZs@k+#<|ea&9#c&0bQN(7wU9(7=r@CzL{;1}3D7V4BV2>A%3D%j3c{xM;e`{Q5gs zdiHE;?W06Ss|UV$vdY0~TPFeiTUGy-O{7xEJym-Mc!O&!C>7)K8QdI2k zo?0g76k-B`GsotYn8%k-%5Fu1s+J0W2HHrW&L^uUzL@=EV}MTa;jSOQIlry6y0o>n z@mFhUpxy#!shZEDKaUUIvq^KfQI4kSJPqA$6jOb@n^uqHX4WR1lRj=qhSTy7W^(;QA#aeKDe=_e{$e`#GP4(9&! z=q!|OSl^fZQSZqDPuN`i?w^~K0vKE4@i}?Ge%@!iZnV(WRa~x@Vhd@X->GrApQJkb zlxfq)AXvCfn5yJqwqjOAi)e4%9s4V^R|oa<-O|3l6De^ma6^|^5T0C0$p0YW3@xS- zx~HYA$^1UPXj?9zAG$)?_BqRQCq+mYn61i={X499&eB0iirKnT<5tCWM{AmyYmAXEOGIwn^UC-UJY4-~c_EOTOfQU^$XZXkl zK+rlRzDP$(6t^M3#eQ^o(=O)NNHUkwU(pY9o#BZt{o5jP$Fj+0uA<3wq7pzps};sQ zt?wOvUyQQC<^Gq^hFjX*3zrxL*v5+J?qK!`F~aT>g}h^Y9~oUt8b?3_<)sjle~6x{rB)Z}dVW{po#e7;{@am%C5GBl^-W zgsLN&9@40;;Et9vn70A>%Fq_y%DnD1^`;aBNCI7;d(;5TLs+7B#yWQaH*ZXa4os&h zu^<^&A~?(y75VM@on$XhUHZ_h7z6TOi-i3eP3Q4wT#Q zW_G@J_m;gB6pO<@{Z?WQHpeedAQ`tk`(vPPC8*|$CK@it319Yc*_!{-cIec#`jzDa z-3Iszr+28AS~d%Z0R=yRBuBOHged-5MK$8M#p;aMKN$>Rx+ia(RsMQXzoS*$sZzhO zmZa1!Sn4^T>IE2y?|xmvQ~>pYAG;*dHG~ZL_1KDqPAf^5u#Tfnd7 zbR5S&VQGmj1T9L{8YIb9cT{ulGFt>jvMO$EFn=q{Ox}6D(O_A9{!;(xtjWtNsXN~r z>)t{A9s2UgT=fkmlt(;}DpY4V&c?7q1$H3gm%05usT~lJ{`_ zrssU|_(EaT+@T8+*lWM?;0m{TO^4B|Xn9JKD+Uw4dnwzpsXBK9ATmS!$$~2sHlr8w zG*X09wP=BHXfYFKMKTczCt(WVGy9I(;3Yfj3CK}vtj7aBl`)p48P>HH@1yncHE0T~I8Jq2nD)YOq z9X@$>lHiztZ2G9dY_TiM@yCFrq9D1>0&?xXJ_YzCmsv-N)Zz}k_(Xvz8EbrDN{|b8 zCQOY6uOSUf;h^zAT=Zun8-}JxRc;krmaR~y#h|gRgPL5as|*Nh_u`lXesnJM3Q+tjxxH ziW8cAM)5%|B;$JUy0DmU=|D^)=`Lp1@4^zSY_a$8o@*O4Xg zE6SHljo%}}NmUr&MoaBf=J}qQ*UJqD{G~RftgVXrxha4wfqz&CY73?nWaXg%dM^!Y zL_|j?Wl1MteYQ-}9t5=up|7V^#`OVpxyjLCE@)2sgFdUn_3(=~bt*Oz9Ycd>-t>@_ zDlH*wVQ}8oe&v^S$KB(6$$8X0bLsa66>2h-)n4S83 zZUqwEHf+uSbq&ya*k|4HyL@Vv2PBCrJ38fgEjAr! znE6Sqp$w}exSfhAGm>yxu8`OMHsgGyoK*6Z&dbm5e8gebPkQ&fH17qK2Cc zdZ4}r`t6a3hE$<*h{S1dl%rgFK1@7uzRpx9k9N{;sic^hzWQ0g%u8zxxc^7?_K%K7 z(K`|2Mk0N04XHNAnZM@Fdq8JFOg0 zmouQA7?ofn6o?KKdPJGq&;GU{9}PPk4QgU#$=4jv#9cBVsm_O;dEK=bHPFaw1h0H$ zQW${FS(ptxjcH3f& zjfztLIZ5ILYSAl7yNpiM3`*S zr+G(32!YjdlgzH>?PjZgaNq51`QdwNwuroWzl~XttA-c?Yzz6BfnDRWHfIqFAXu!m zL=(pvKUvRn-v|-E;^_)u`Zev1!{-bCp-nGDlRE$##;}W~Pej=*Q2mVku!qw$w<_3g z5H1W2c3G88>$#C0gRm?_fj|>Eazb;goeu<`;i6`<1=kSwf$e#o9{Szj1WNiF@W{z@_61p^MlJ*ICgWg z!sZMK901T1G|WnC9&1kvzIZH<+@^V=MTP1M7W0VWZr(4c-$=X|0gruEPF7#2Pwfxi zs`uw_pLd>~k?vLUwopb%+7y5IsVEDvS8AXH_LMhB1Z+v0UZoM^kT$@(&$dPQ(m*6t z-cmv}_Ob@P>Cl8-POVfM0e`kU^T=xtRfjC-K6lM`OT~C>W?bwb5ZR&q=FdnT8~O!m zya?66>LchH1kiEe2A$FCCF_4M*bvR^O|%>3S!~C?0iP1nMSX+)cfk8UH~If%iq@}* z4A2m)zj!{Vg=f$OUuI+%BsCuWOXJ7@lK6Y-U>EWr{J*E03-j|B;2L)a{IB?x=LEt-YW@h3}OFpXBc(uX; zIjYq-83SQag`|iGy{ZyKK3`t+FBbY??dt5j4+I7tFk~U<2e4<#!73C~qBzO~ce7~^ z^JxpS$Jf|e3XsdbX=4}+=TSpW@g*{N2AsBnK2^FJ%X*_P2D@qqUFGXiWR_qefo4t< zD3a(%3J^K}UR+2syMEWMlm4au*FFgn6Y1j_-0Dco6@=b>5RoSB1Ev!9JQ|!cJj3_u z-OO$U@2;yv^KK%%%cr(mmK+ZA4kULy6ziyRz7QXEgk%z{oDX}|K^ngM!cwLvp^FO0 z_*)Tx@tDR@2`+&fbFvxjU1FSYnBOv@4`E)OfqR7}Foy`y0v+wIieLpu7 zHuIUwJp0`%r6f)FTDslDBW05?FTLkuybV83gWqEVs^}ma@xg^Io_XM0a{p-IYxY_6r%1vz-Rs`h4VzY7245|8{GkRbnG{Wq&OX+Y3LqrqUj}NHCV^{#Ix&P4k4;=7cK2ORQ z$Me5v{11ZcVX5Q55DdQXX4*pejpJ*=5>zT77YA)Bp*jx~uv+y*2}SVxM+&%6f+|9L z7e^rJ?uvh_2y4nb1j)fvk zlboKDep&l^cA5TubGPmeGLQyK67h`_Zd@=@K$JRy2u^%9)LN;7z8FVwW3ZaiZeI@C zT^JLFsG6_~n4ej%j_=%pc5l*Nttk^`YpdGoMDENoFwH1Dp;^Ob*M6sDDri zJLwBe3mVFd2L_wBrkTYtHd>e-y|zpO6pTkKyaO^thM9Oq3{z&wd3QO36)NF36ICt7 zYff-F%ubggTG~!mGMU)YhcUV@!Tn&H}qF61dk0fwSM-BLP*Y8F1xgGjM7>v%^D z5_OntcvO!Cu?Qa)gG|#}CaRM)$<;1&pr2Ht_PQe=v(z0CnyF6}Y73AMg)o(GzGI>7 zTPV$iz@R0j0e0gfEO`(N#YE8?T`H+_iZe7Z;4_y^9C^sYtKhf*R+D##2cuO#Ci{kqBFo$ocAo>Yy zfOq9b#M4EgR-ZTvnLD&^Xs`SC72zogmjy+MzAB!Tf%FgGpvr3*{6tJ5 zMK8p^TMP-n9oBviVZQAG*sHjH@2DNKKmtXmm(v1|wb{#&6uu}rS@UdEO^mUM8No}J zMAW;vO8Vx)5p&OEHHdNyN1z{bQ8S@WNmNRr@FJ(?;Xw%l%<4@IwlvVzx}=WxjNMIN%^vg?9%s*y?xU==jeJ`Q z_?QfrdTVz9YI}7nhsAQ-=)MI9^dw#NIPI~$H8*{Tjcx;8xFNP_1~cVU_FoY;qI=x; z2(Duh$UhHy`S+oOK7gWppQ>8?sJDeeZ<#XhPeNI0*95^H3VlGXxu8$#{;wQg#I^BT zZCoqrTM;z>(7KQx`N0tYKOtQm6%$}I?ff)j;1BA&?!X3JCD9tpvWubaE#(LdL1&bAM|@mppCHzXIQ1sUd6b@zI=<{4TYOk~Cpd61p$w$R;~=o*J^OO`qp%R)r(-eWPBRM&y1BC4r(jf(W*j&;oL(nX-AKVG9oAA1ow4^6z^6@eSEv z^9*ReW3vPVnQkSn1ubDN3Xw$X5(yK`kxVgF$z}|Q68R2-|LD=Via3cc@9Ry&kHLZh z?KANIEwvd0kOlxNw`Zn~cNW_$x|YNy{cz;;>S+sJjezkMgt%7Ag+IeC&38!PCIfH} zM8Uk8-w~*#zpn^MtIA5E}4K(f$!7okm)jc-iVCp3R%j0tni(zh% z0UM^!6%bfNHcd!w`jB?$9@U=}Qq|dTm`J)fXOTe^pniSpDKAmz=_8|de-e1*J;5kk zwv!mjo`CqV8O#ZP+DMom3z)Hfaa33%ZCiMI$iN0YN2TD&)`O&weaAMed5hz>37dI# z)nBP~e-XAprTeMSyCOSUUCk{sYQ6tWsGR-#ytk3@$irwJWjfjw{A<&WAsup@eI@e@ zCYrJ3D6K~1=ZHz}#Pf}w`r}s~*;u(6q zY1lJO;TU?*c8{V_ukz%+s9oyixbr!c_20>?r+?PY(igHHvZ(bvYFt>qAM-4{_)Xh!sPALIphO7i&|-Uq(!+^X(Y_EEanZus5vv>^dUYWu|ah%k>)NV9ma(L2XAo zQ?nYNR~UWbMxC3pg*mBo;l!3f8|xHjepkGqCsM?uHcC@qq%r}72;rzy4Ea)D4*3EB zzsnE$l&R(R{en+b*Cg00pUHUg_6*Y<#QqSByaqk1JohOr7K`Tiy^AP3jky~5)%P+f z22TEkM=OM;Tr8U>Uz=pTejp=2qd-z954Tff49Sb|4hrg(EaDon2oUVf0@o%Xsb{2r z;v)C`+t!Bx-5-)u=S_<{w+R_}V&?>050OG(Bz{6qZkcBbZmDbvbgO8Je0Hf+wK8m; zuNHDsZj@@3Jacm(TwO;=9fkuv zp=i>WZ98S7eo$aY)wOIDM=FRtdu(#vt1>yJ3TN?(1436cZQUG;mk0fvI zM}`)1Q4v3jA}rtXh2E**XY;ZS2)^f~VbT?<#dehIp1_C~9t8Ux)v3-i>_-QhFx z;8~G>xF!W==%=5Kh6b1nWxd7C&xo#e>OW$B>Rs)|7*}6hY~XURaclsW83nFse+3az zqGKAKJv`%FRk-k!VNvwQGfp5ucb><4trXOv?Qn>@KET1XJT|$0*J3J)Ee4TPYHjDe zY4-3%UR3cZ7+efkdLh{33mA`h++09m^D^qoOOn_McAL-2i94H5*XTjf`cd-aw5v!3 z(Ya|aHDPUAr!kw&3RVgP4Y5yxUCIz>8%}B;9_L^U*J>T+i$QjZaVyrkO`tYGXT!aL z(!lt`fCU$~{F}W#8pZRm*iBM>dzWtH?_X}NFA-DjH5i1(!qyS)EdR}foz<2&^L17H zrKR0Ue5P7dBNQ0Y?Ba%&>C8)!f+t0~wL*F9KWZ^Q^hV#x%9?qB#X|ZVCZv9kns&}h zpmCKi?Z`?|b?K^YpjEt90~QMuw;@pjMvy^h&1~eeWF7Q7(s?%Nmpb?6nbsfZPSq8c ze@8H$S}%+Bieg-D^x2lP&N_`(-n^DwMPw>$1b_CL>*a}}e0vdF7uMDIFlO#>gcE>p zDxy75i%|*L!HHi5;_rq|@HYC+dpQ@Ib&piJk9s-(vg&TvV%^rM%doG}oO+@<^Pc~k z)6m`S@pZ2?!=7GqianM3&#PhNXVh3vhUxxo+DYNt1I&RF?D6en08dvW)`90V5fBn0 zrwfZRr=bB0)oXm>2UzGmhi8)oWW^Crgt!b;QK{4i1LleFL;~yx>gHxrlFVf z?ee_`c}1CEq#>WP-aS9a(mloKN*uAP)Ncv`RCa>R_ zt-WpGoG%FIgDdw4B29l6$%~=Lsly+M=2(@b zO-pp=F)lPT794@~r}c=Al$St){$9||G1;PYSJe`T-JSa6tRM0o))^tz-q6(hsKus! zTJHX9pQj>c#ELOLI4`zNB(}sqg7; z;)H#w`IjPIb|05~@{3#PPD&asfjH3zi0S!rz=hnTlu;)KhiuxotD{H8PC99GOY=Cs zv`Hy2xSLP*RG#@SL8-)r3^6yBd4Ezy>!fnkt{QYw^R%o}(Vd>811BJ3X?9^}>zOl^n64&njfNjW8d0=6(b6W@LPnW7% zXLzYk7FznU3jz)w%n0YAT;UGx>2wL4$K+3-m$SjXCd!A|PkAjTRvJ(b7=*4#HAk$8 zYk+v%vIDo?$d9cZ=+A1@8J_6vqrh(jQIuOJqjc3ZD}~osWM2q69GLSDF3l9D^qkCE16mCD zGFln{XHZHzMt(u&T=X-UQNt6mPExvo%6lx!(!d(i#ul=UN}8!VYAw4Y1G433pB1ye zV#hk)H4q&{QL$fK@j>DS7Q=OIsBxkCJCPbgKALv$S_Z?+IFY?$6HW zw^&n-&rsQk$@p!(1o1S9yfcQqCBv3)?w6kDf1bFOFT#htPSQlrgx0T%SBC!<*=E(V zd#Yz75v4F?363+TjO89F*ah4GIaSgWKULAo_8P#Ccn4})G@JNG@G9x^wMcs=#|`?v z8f(?Bq|(+PL|RW3=wv%<%=cJj9Oew^*&7wORM&SP=nV2Hssty=J>aCL+$4_3f8z)6 z2fwjg@9&74RqScKHC>Zfcbd(ys`q$k5amg-bx|cPSZ>HX8FTZ7ee;3_y3cnq%Ej({ z^FlR0=A}A$QkB--&%#0;8o_eOkmNIzOLj<7=d8dTvIX#cvPq>$MZVDQ!KNtbOM5V& z>54pE(>_j^(a;N4r?#TDH3*-1b1v%$!@Q?`v2U)&pE{&hBxQyt24utgR5DtV) zcyU}|vjMCTJTu{FV+{vs+KmK`m{IpgGN}nq-01yma(ho)_Y8P-TzB~2Gk6ofG$tx` z$3yvkDtUp@SYn?YQ!UOfSQN<{Lis{wyPyx;b8g7a5K1(L&3k46OCo(#^m|{AIeq+d zMn;YyWELVhr1kqDykHpSCp@C*cQfw+-O{ytvVQ@70=4@i_pWR&?4yF)X?FnXhM4v^ zryEhb_~c7Gd*k8&bLp=xQ}gq=Mx#R;GwJw`JS+<%Vf%2OG&FBt70OW1|BZtGUwjPp zrBL=SKyIa-N49}R`Zx5^e_Q5&1p$G@1_7Z-ozwc_1pD`L!lrg@@&JI%8n8OJhe)5m zCxcjH;XTPb%mR#P58OE6N?QQ9sL&`;942AJ@bJ_oBt7TjwH?DZYun1`-&)mbwwW5u zs_V}Z*2wVR3+K9xnxkiyS}ZR*AK&Y)))rmV(j2=OvJtd9QfT(cOsvW`SBI%B9}KI(qf^| zcn*RIym?kej}RJrq1RbXhxB+RYhTlLksxBd9Lk|mX;!X8(eQ!BumndVsp;=D_&Q_j z`iCRS8hdB7-u~qjX}dUpoO>h2obeI%8@fMS$9)Ua-iFs^85w>+0HC zb7y99L5OG&*7BRVq&t!~2}+`3dexc@vz;ut`A&wV!-E2+1%|A?sVfsp4bRv`VtI*o z0JC$>=a+^oN0w8WSJwMd(2MyW$J|Ypn2O3wCnu6>2hn*n-7u z7&!!;g14_Xct7JW3kf!bDAIUISn;$(^*l+qiAsigo@rzbld$A?yxt}%2L;q|I(`mX zY>UP?DMu`zXp1lSA-VQw<{XsdZV0bFVnWvb-9;K0_bg=SrNDDMs-k<=<`7f_&x>!h zCGyQ)niO($Cw)a6t3Y$@nf{O`xFAhp*vlAISW{Aj1}%sKk4S+n`y1#3vrM~AaXM&= z{?2frYdY%rsWds8vu2q$H?G*OOsPw!mMqs#t6Vtu=-!OpxgA7#Yno&=dt&_N&g_MQ z*`y+%1W!hc_C=Ma1EJVZxE;Z77-qxvSZd2WV)f!{t9a;O@7;1%6R%e|*<1&@63et0 zL8a63*gu zzkM0tk9Ytt;Qul&c>WzJ2)fapZ~1@M@ja-YhfRXcH$S=%9TCJJzdvsj4o!IuJcd zH`Z@rL-se^5IyQQP;c=S9TEIDe{ydk0(zivGYbRDo@vx7pK7eU-++q z@=G@iXkcw3eLbpO5uas0JV#8bgHa|j1TM5^G(&vSNN)AfY8^FNcBr?w3yca~pr^Ej zphKy%M0T-tM0pL(MzY7}=|$UA<0(*8Jw*6|9g)?0BLt&t0&JyrcY zMGHSB;}t(OFmww53`J@6QJMw^ip&gKcmW$sIE5;dR7s1rPA zjU`v>G3?aJJX0GPS@BT<7cN3AKYNEh|kO6b17hLElNWX&o;VnbT%e;Iza37N0HTFN#REv+rSyla`Az#H(0IQ5iNzR zCg_>@np*7AY zl@v}0oWvOpE-5g%#3=Q4W~w-v*F=kNik4*pOHqni-9G^YwhoT&y9NTg4-et@%|0Mh zF93I>5p<3ZAM#)=U9B=5-cz6F!M0K|T_9S7kEI;*oN=gu=$Ef4*rOc+tI zVkjKefvCF5R90+4x@brgXwAX_s_uY#$@^V!o_!{j9bk%@!#jHP#FXoP{>wmbb+c>@ zs@a^t>>(;|mfaiE`49X)W^Qa46L}1o5vC|f+N*{`cWs);Q(n-8v(&3xP-u>}To=bKEy&fWNS(DJ^RuH0tOaGjCWuge)MO%5yW4A8e3c_FNp(@IUO zTZwdswh;~#O$D3}eO1vVdo#)#(8v)N$m}>y+Ebi*{H*xJ@pJoqK*M3{ktDQ*z%!_D zC*gKGOCRs{a&j)pfsQ>iDCk&0=Jd zZEW+ht>N3-OPefMK-lMis8>gFkg&jEnx!W(3sKWM;dVr&$Q~&hy$nWZY5R2w0?%ji zq()hD;lSy>a-vLGJa6(rJ~?Ezt}t&>D)oQ3ddKLqw)r?{@>P;z3oT{&wo^<2-bwDY9^bJ z*f_Q_khDtHDFN(9;=r9LY1GW_{#2=vl)B~0_=YraNQT6l-f?G&7fn|)xavm;cZ$h7 zmD_ei6`rI;;?ioBX9Vk_ZB%3h-LiTn^)b&EWwPh_WW!~0bRC&NL)5&v&YufN1%v`yM=snv&<=5Qc4M_CjiPF@*Ko;rTRKV5b$Xn? zUjcgWViBdbV$fckC{?=+iwv`aF#*S!d1td60jz8hkZFZaUb&`c2*-oo%JzBdG@fB0 zH{fwQF1%{S(7iwg*2(p|y+Q2Fb7=Wd zYP(S>{KUd{@`#SlLy_#87m;qnLTOqpU9lobpKa7jq*5Fn9#w**_X^7D$Zy^y?+rM6 zSR6okDa4ODtR2=%`bu`m!d4(}vjgc?#-HNopP%otO3arI-r=~=EP>!Z?| zikaPG=?ISYuQH+sOV?bvMFKzA- z?xxn56Gj-~pH}HlNR6meUfiNvod1->|9tTOyCgDq>R2GZ*#)-WB|-BoSv>d&kNuxv zCVYzg_BVwfN=eHKMF91av_)pGIf`XX({9l=W55z)@KNNaWCA~ZN_=AIy#Q&45R(g+ zmMdG_C!U^{s(^4I-*@O!_Nr(=N=VPQowVEGhUww?6TaRwfIB#An6dgOwqG)O242Pv zyR~T4hUU_|&T^;Pk0n@4^{Of|(;y;8bFL*ZG@l7bg_L&{C8F;_=i71CZVZb!B`Y!# z%Wh3?oWqf=%;+%{ORe?_SH`oA1_vusChY0Vh|~+f|6{CS{fhAP${m|77wOa$%4Jt- zIZ?(J>@|%K5nU?PrB_*AJ1I9%7@&Wa8pB~yLos|OR2HpCg#B6#81HEhgHM#!_P5B7 z3=adotr)A%5;hMn#%(@&5}T=~^f_4x>_CzJGH(gdw9f03kMG9I+5B{IU02EbP(xW_ z4Oo?z1o!+_e0yUAS;S54sI&js$+Q6}>)u_g0HH+{Vi?#Z8+3vh- z%&5T}YY4&e*^wOVde%9~JKAS{+tln@x>y31W4ODKx*r+6dp6WcI;E6w7zder;9?U- za7-lV5OwU#fFgKNK5ROXq;SWDWs|#eF zyvLRiz$STxYL>Xt$55G*KrPi3RWpIsCe6%k5F!HO9oI=N zMkUALSg0`a3I3ll`k%$}zssm4_Xx5D^uMSmyQ!%UB;V55>u;MKI(EDidOaS5l-eO= z6dQhQ*@J z5lq(6r)9CdHlCD;%@4-nQdpX6%xwlC+S#nZ`Kj6T%3L<_{BP-?o+<3w8F}`mHb5&f z6JAg87Rz+3X1bfS4M~#WHmquEFN?a%oLnNW-Az-gnuK*G8wITEm}mT%l0kF z%do>{cKw{R8}Mx6DLQR*6&Vxbh>RVoO|(^QSPaUE$F)iD59*lNN{GPyM9Axn3_Nd( zAsm!z=?n=w(s!nDxS=D}q2rcRbm=Q2R?Atv#PMA6rK8A+PnaivJmkOy^a8h8%5D_2 z$j%uRf7j(FVV>SW5*ezFVKfCTIvb9dt^m0-*jbH>Nx==p%G8wR^RuX&9Ajb2iyg+x zS7)W8+(@Ts$5gVLUM3>^CF=KLisp)k5Rdbn*|baAC3=+YF)meWp`R#!=qKi#ES-w? zVoGBrgW_0V?Zc4rn<((`pa73gVBADr+c~wsN{Y=i!RDDtaE^QBUA+P1seK8I#c13? zSPDYvw@OQihWIIM26Tp^TRAOZKdanb@cWzJ!;|ue#AvE>P2a+?naRgm`D3`&Me z-Mq@`-NMIgp>_3gI1L!!S+FFFk)gA!MYRPJx3V18|44ztk+d(4Ld}KI3Oy(EddBBk z3Bfg0;H*ko^lwZhUjg@+P1XORLK*bU(!^AZGV^hZr|{D&;^+(#UJ6FwIP`W<95cr9 za2~l%)GhpsQR<=IyB~s(#!l!p2{&w(kZsBui<#|->c++F7xCX7*5grJB; z+IBjzsc$RNE%54CSIXFYlzLNb$Gv3|d1q|N%rs{5l)9U(AMK$@ncJh8kec3~&t9Ab zX|R$6yT7)NJ9(d^DWg{fL5=~213uxtuC7XI%68Fp54*#gZ&SQ!p-G}Cb#_XZmwO3Q z*d8pUeS$h}6)Wp7#)V9SqatdbucCulXrc*pzx6M6UB|pcbCyo=xbtDJ{-C$9WH! zEcH<>N(YeK6>-i7mKzwMWx@6aOT+m1E_IV^+&E^QP=_Hs)Gf#et~h>;^l^;f#Psc| zDz2rX^MPiadtLhqq0C5~TMjaOlqF0<@~-Ciz@{wumrm-B3c7X@tZiFS$WK?A2V@!W zM#0~E2y8#$Ws#F7o`WP$5EG?HCKdmz8^vgpzCRqWk^ z#IDR1-@p~x4@;*K3o)SZ79m2}r^V_v{d!A{qUB-6{yW9w|86S(r_ucX_L)<5b=bdt z?ohr-&!Xd0UVOtC=kMfEzLeRvZ(Q5+3xyu2YNMnjg6hKxy=vda?+=l;7>;WFBRNo0 zkJ2s(i5WYT6}3oZczhDl7XHcfQVQyw!pGs1eV)!mQcM>&?I+(`0q(Re?QgKEFrU^A zPuHBg%!cXg&3`rDap7P+GF}C_82HUfOl7%{^#3>~X5U~xHs z5O{N`gG9q(7srCx`zX609LH)rtd^D#f9j5dEm* zTTAV=YK;$+llQ39Iu+HH&4d^Fj6?}QgJV9qB)6cdw&cm&Lm8!&)$&7j|JVeX2239; zjEi79%c=SmSFFC#**CZ>61DliT62~qC#j!%N&k+6=v^c*0b($SO=6nuux@F{H@u#JD4!I8 z%}n)a`k1zZL3v%q)~V@dtBJ=OgWz(e+{>lKatE0nWx`AYC5hD2ON04@CKo*Tj@ol& zfAG{)w0o@_6A#JD)O#t1F(_YeimV}(wTD3Z&3U|*7+t2DgHC8QObQH-(-U8%ozU!d zIs5fD4OS`LUrnnin@36jEs&HzUuCOB<9I3yXI9l45IqKMtGpkW)yoP`(!&*t)az&m zhB9L-shkk4RE@V$`5K#X;a*jSs1(8LUmbf`dq;Bdm%D=ng)O=6aj~V7{Lbg;)v&`@ z*(35FF4-q4^tj=jb7LrTdmp?eMT$6IcE3JF3ZLjZ3C%Y$CjRd5&YiXcH>P_2a)Wh^ zLjH}n!a0Q0;W0jjxCHC3_)#F1OyBiDs4>XTO&gJQPChArfpQ98k3I^cu$_awE=KMW zYERh|GGiBtn`?tVd&JvM&=fv!Gi-juw<}YxVo1OTXAtN?D)rY-shT8|LK6QAi~ANl z?pi4w?#);MdPfdB6d9lZjPGU*Y`>b(B9o%u@ggNF(}Q6TfCCB-|1`7;G3S%T<9`uI z$1^S;29P$1-du@wZ$k0$ALSgc2|d~FE`*FEj2HrL2x=;J5|@>JKht6EOw*6J#r^v` zH@*GK(QA9UB?ftP_G%Av>yb&%a)dkhntvHA8mEyl+Bx=}#xO-3c<>{vDLY%J)6t9c zEDTu;UXq;5>xN^^hg3gS+OaCi)!pJL^KKvVsJRErd58_{2tqJ~gpt3oq~W+d5{k&ySzwZ5G-E9vQLGH<%v`sYAocE)5IY{RhDR z3eWlfmIM6%tOx%Wt^Q}FU@6%)#rQt-RDUbEI8q$`ViTv>)1oK>y`T}jzzmn5t6E8j zG)BKIKqoi~n9e?KKlaT)5FPlSP+Djb9mNeeLg2>GFV$Pe^{b^w`r(6}i`s|vnLDHP zQ!OM(bVY}4cCT)mrQ%+SkL=xI-W2~L`irh4_yxiAZzQunzW)((E-Jy^IGJ63hb;=+ zws%Jg_1$$)@@Pf|8epzp9AC>nKT=IQxp{W)WEJMWLy5Y(L=jdDtY19F*$cv|SUz*8 z;mKsLD(B%d;VMu0709qxrPuw5j3+u=kQWHn8baMH3j1zn9Cuzm`Zs^-C=e z+u8|L%*X2t_K%R|yAda*23jEh#E;6J`3dZ|NvWvb#Ai_fHhn5Er|GsOW4uKFA$>4L z_o?v8UC*8WTZX#~TXf$)1mXy;u;nC);nCUsg#fGwNlAdf}~ zgixW*KA2X00)|Fo(I*?w-`J&E(^iOVNoFn9%#JBYK6$j6?=LErQ4h;r=apJ9`qD*Uv_w4DuUndo^VS7=Xvg0;ey_|F~8+27ZluhAduY zGPzw&Y_u_HzLeQKapzjB{m_&0yFSAKkR8G*@j_(TevP-|*@Nhd58wm%e7iM2qg~_dxb@)s z@&SN=Psn$$70Bn5o)Q2!Kh!gbhOae00s%%W)Cd$R2n+x+_s0x^Je)ZqIlvjj89Wh` z6Tk`94g_fjZ3nc2d4PC;dH`HO?;-C2=Wr_!wN$xux%jz!J>bs(Fc2_607w8p2t){A z;%nf$>-*Dp6NDI008#)j0x<&YgX{wie4%~Ad?kEGeJOm)d@X!odYpUUdZc?UeR)8V zd^LP|01W_7z#+&X=po<{aso;jfDVEViVgsxgFOPyVbwv^Vbmej5mpf8;aBjHKq^5N zKr8_Q{CN5#xxbJ=aY57}%t4qz`+UXy=pCj9lu&|XP+5Cf@(cx6h4Dx&nG{08IHlF~ z`_NHT(`u&!i%6|m4Z6ZyQ_bVTn$zr~q1n>xv!Kb+?Mwdil@E=Paoq~NonhY*2CSxM zh?TyN4Et$nxR7>CgVH}Me3XWnPs+IYJ$lU-9M{2tXJ&!03Thrd zmoKEowYJsOYtApKrp2VG2xVp!0UG8V?leB!eo>~n6j|Y{uxR8qtZN=3(r){5)?iHbQX!1i>)F3Oo$;5-rj<)xk_0*5eJ?X z9E^eJ?Hw#RHt;X<*S}>*tNh*}{ZHZZPm(*ThP#e&PqV=w#-+uP`IVXNSvtQ1xrZP3~Rekm`CYwk#XFYA$w<<>Fwxt zpu(SFMW!w;ahI6WZl>cw*ECl2sJz>XcoSGMQCZ)l!N>U)u*}o)@&M-y5jERcvb4W7 z@*;FV6O(q?QC$>sb3$7K>LwQVI$`2lBMDD&YgJALfvE9*`9{w$$R}nqd$QL?9^R%4 zX-ZM=v=}H^Gc?lbs!NF?b`s9^%RFPi<olF9>!4xkY1UP{=TYYK8(Us=zIVms431^$&{JIf};zlp)@&&H3mc_g?R z>*|u|D3(BaGtcLFw341#+#kImt|wmUvGbLJf@gvgtO*M~{FR-D;wnzeflKf*&|oYJ zwAeHzXpesn5V{DFbTv#<#8RClp{uy2QP=3PTh=SAgCg2o{sF;>J^GwpfBb2C9e?>0FPM?9*d3eH)Ls#2EGta0W|C77v-#~)`w;EwVEf4`8 zPe1#)sv&fyN&v*xM(X7^9EPCIMME;G*J`grhhI`rZzyao3LBCG*`f@KDLonWeB5Xj zOJ&hIY{7F0SVtiGZYmm@bU>w_1srQCs~c?7wD`Bu%?T!X9L^-B*Hx3QEZ~L*8nyA> zW*Ah}hSeHWCHMZjr>o`VVp`RW+P`Zl;j;m0Oq-IvI0UMT4^s8Z&`dJR2l$D0KzoE5 z8ju}0<-sekRE2A=Rz41cOb?c6SrN%3x?%cha)m%^g^dm{8}%x&Hur^LiIlbf3y}CM z;wsLx;v*`%?2MloDxM0Iv}CWxuwzLqPZd1dV5QOCMwNpfM?D~`Etcu4tjI4SoZ0Xf zS1Mv;#sh9D;gG&ixz&PMWv;NV>a_*%uukc{7gn=^h-RFvi%%4LoHY<8l7H40NlW_yX&qK`|po-l1{i<|z z8NdS50~NB%e`;iIu0e}sS{(?B;ggzWZ^cPNYQto(X(~|(>1rj`&9|9rbr&TB z?34SgcZ)-%pm-ngVkxrwLvK_`Z4G3jveIijBr=Il<`+o7gKt(c%HBcQ^G5RF0`-uT zOV*u_ix% zq!YzR3VB&6V?QtNH!7(q`%ElyL`T~t$#{#@`6ne2&91T^$Eb}#i?O^@)Qo>1KXAg* z>cF>1dbAH)a)=l{Ptz(OH=M6LOaUi+5j?+%9mxvCO7W5_o9=7coA)t1zzC1 zKLQ7AR^?6!amO4h-_XWZBWR8K)0L0{Uz~}Lhh;{1=SCShggZ65+vE>UK94@%NL z(CM82JS}_0u$bp!F9bz6^D>Hk6IksbL60rK+8m&po+;XsAqe~{#`gA+jp6U``St5& z{DTYG$#n6eFiZN%L68M zI0_pbi}7Me?)>6(^*6_e2&jr?0ar|I4AIoyo*JGIM^Lt zL1OIqBBDuRuk<4PR$^=9f+JjV&Vz#3L6OcRKi-RX8ikpa!FcIwvcKulq~j zq!x-Tv=E+4s0w-tAaw90#1<|f?4ze4N{6e%MuHg7*JR3tsD-RWuSKZ+Wlyj};8075 z0KkP|hqG^G??Vy`FMxAsWFJDp8K?_p*Ui)qjq=@BcoFRh1MdA6vPS;K=u87AdvHRT z#8VKNP`NQj5wIW_ID5$2I{p<>3Rfn3jzZcb)vN<9&wqFl?Lam{*K9$pk=Std5VSX; z%^@~OE?Eam9T0bXgt&hUq2I(3{z8ud`87Oj=J3G11L5TbtyTk~J2<=ye-pf78Ok=O z75qo6(<=075C|3gj4AsLeqHuFft1rm8;sZI4T+Gb&~q%!mKzOq(ZkCd91EaUH-OO8v=D-mRWsg0dRZ?} z+?@6atFGaX%ccGhtq?5Els3mhfnsn%*+!(?29h~u1tO>yKKv}cwn`^$MEDWo%;5XU zCl0q1h$;AFX2JPUVsQb9V7WM;lg?{Dla+p@+FlTm*IM)P>?z{Fx<{JqR4p87>)c)jQ1%#Tgc~e7M4j7R6mDXM1i8A1gDWp89cNHa7@A(Ewn`o4Bje9-qCb^2k6qW6}-1afW zqRnOx5`2P~jF==U_f)BH$gy|2M6b2|ikAT6&lbr~%V<9J!>_`BVjQ1HX`y;3Xkdy-?$~b0JD(% z6IXfv@JDXMM&EfQKLz%7!1d$~Vv+@0k4ygH_uUMSy!%Oh=HUV>C+L)J5%-_>jX%Xn zzU&8$n9}VBHniSZB<%wqFevUG3Jy%K5!tdbuI`@ya~-ex-1h&PUpqWaH{%1^M$&wvd$4N}iN;sdxT6U(LHW;kvl%Ny>=4UT%y+sH>r!?`69!3h_s2| z6*dBT;XBj7-_NjA&)}mxY4)I(+#+t-&5<5MjzbO67fzW|jFp6(tt#u=Ht#tLguo)_ zw7Al7q-hSRAcGf?miJrk5Ym_F8a^%2{gr@pu(LZL?WaTBAA$R9BVG>}9Xa9C z3-6+BHm7v1d`I!Ewc5a<_%8@EpzvYE-^q`7E_RD}1lfB^k}s;M&qc?N@)~=Htqp@> z;RPPwl=;M!m^5TPu5AjS&{^^v0im0CRJ{O@Y0%BmNBVG?>c!3p!6F%NuJj*p`bNvx zM|t$i@8i%iin=yg;xDhW_6spplWH<&YFs!}d5-ywnp9OnQMC9Qnk0qpKf}VMaYY

jx;~hY;YeD%!!0 zw#-{mjTa@w&xr=UaCczl@#C=CznB`Hw*nr!2Z{PczSu=Rk4G`+nobGT&!j})-FXZ` zC!ehA+dkdx@OJjcY+TDL>ZoPT%&m9388Gp7tYmDAqcHPq6+i2K3L3a#Fc|vx$Uc1> zzgw`X$L!DbFy;6uZR1Up77egV$f4cZ1zKaXci$7W=nqep60eZzrSzCiXLp{T+txfa zky*RWoDkLR!$fQ7y?A99&w&WI-HMadp2p>?YccH#$$t7uzb?wno=^H==0!4F8OP_u zq(o&{`Qg3c@_-yF$DayU1ZW zQ>kkl7}7K@J@l+t*E213BHH+(M`^319y_T8 zuBnKF?Cg@YzI`(aZA4w7_4=7NwX@FCqWV+hn)2sexDt=FO3e$6#TrC=VuV+ zD3v9zaY!EibZB1s1tgK-?ynFRSZ$c%3Pdc#O~lQzA7vVGQ~jyZx>iT#j{b>zB|Syx zC8v9&${pt6h%%qBkIcid6=p5e#@o78<;D5Xi&f6sGu^vD$<_)~nIcKLOQh^)Pr})x z#7Gz;kxbgn9Fwt4qLMhEEE@YEdtll!_E6{;U>Uei<}e}OI!NP01s}AYnkOH}4x|6K zseQ5P4+eZt2h-Jf)o%dT^7Cqq=aEN-*1J#D%oBfcmX)E}<+*m1rKqD!>!!|l|3aV` zH!jT&D%z+yF()SA0Yb}TLg7bZ2=(eK9=y-sh{g8Fl23Nyw-U)_c)sqHy~uT2wQ8~I zgVN$}vW6DEoy9k-PUM0DR=B2UWw@^KB68Vy=O+X;^xxRFJVF(I7z}kcI;sASdbi&= zNB@cp-ao<$F+y}_-(xG)M!9rYzJ+Jm)k4(h;=v`m5jq6wwmixeemIFP52K{RpQpS$ zwH1Bk&fJ!8HmWl%cqobFR5GI5(QRC*zN>vlR%uyO7OV`Zdxw|PLG7xvKxyx3tZEvl zMz_{Wmy2`6opFIIn_L@7mT`NvSh_brb$yO^&MZwK? z#>FV_ZY{E%yL3wQI)I}x_M zkjT~(1g(*1UZXUaB)#g8y$6*$snzgu|&c6>Zs1ZRG61x-n%)U-k%lJxeg3!E)V zy@$aM^{H+8izVzZU{S*zQz`p*G+WlRLC66KdZRK4ibjztXRTCZShbp#H~mXucgiYv zgeQI06QyRh>|wJ-{LPpmJP^qQ)+fl!wKl^sX7YUq6LKHyNz5}`cm+J?X?(CqI8&Iy zWMhQ=m*$g>txZuHTQ-ukg?QVN@4^ytZDbR2yDpMD!_K?Y8o1(e%gzsx8{IaNHUxmK z+oNkXB<9alj1yjemTCSVLT_adMlRL)U$sHB#O?@122t(%ZvaN#Y`caVJ1d8i#JnCd z$DnMJ%)}2gL`eo(zl2&Qq!wCJ(uVN#KdeEInkyFC7Q`I$Z-W_AEO*i$>ejQ}N;R%1 zr&F;XY6}?KtI%@9x1&tF3lB`od@O+4`C`Z#CTRt6@bGO|IVG_THjZy)|CXPpy+SyA zCVTZ>Uz9(0olj%9o#L%U7OU3GPDVz{&ye-le^(UuzP7cLpCzgQcdJfw+fV6uRAln> zo5sxa+)cz#z^0?s+`+2!A6z(nbYb`ouS2-3_%HdXLADDEj*4e8REB_Oi_buGv^TDr zbC%O$;8od4BY%y|f-M`589~()-ya+XfOpoot0cP5{m_;kkFn3~&=%bdN^;>rqU6VU zRF}5S6W!Lz_`R)#wI)w({)6kZfUX9_`Zd8}L&Qs3T&?)2xzgf85 zK{^SWL2@jk=KIAW9I58}J`GUQT0eh~)o#`>fK&7RZ1ImuBT~9rGpFW{mo6((O=-za zs90-{nCvRSrv>M(4xiWxk4$b@cN3HHt7`o7Y9ITGkz36VLP?KiHjg$RbAMYcIcH$S z$c^Y#;&TQi#XWBo(K3#ArBCYmb9WWdG)@=Tywl6p@gQ7R4~{pS5FPMdlnt|eH;Px1 zPg{cSR&?6T((cHn+ioAyyROB)AqwmMqD!^r*L^jgQc1C#im2CY@4Clq?ijgt#u z0ZH2ej(cGI#wd<^Yy8G4j(ckS#vo3cw8Z6U6t}X(3Ja4Y(vrP~0idHWT?=5LaU=ekcs z#r;0+t*z#}O22@X@L$zgz0ZC9GSC|}_#hu{$rG}7{Qpd?35fs4Z2EtXttEBLmmt2q z1}|Vz1oSD(_#S2#sd+jlilKeU zbgzE6ra-Xz*${RR;s*|!qKpL8LQCWxLRe=6GXD-tRIFe9Ku)%Go!^wfVfI%qZ{me{ zQ0Og3tt)*1Sl}q;%3Eze;5P9_KO$N!I+gSMdQg85#~S=bk#r3rH zGWlVv+v$AB?d$kH6AS}{i!g}6%*u_ zjd`Pa&W8vYTonr(q|>^gFsPRnz}m*$~p~{bhw0;9wCEQGt!$`T$>v&&PyxoYz=E?JFBYeX1Ckp z47MH*H~rSZNaFY#{~Aj%{)&EFp0U&{@^(_w@|=hKYA&XML$^q4#x#XyH5$;yi4IOU z!$OtYRfXI_f;|DkIc4-Kbh()4s>Qg0dk{Cq&*Q>E5@~_m)TWl8yF03}_vYHpg>WV8 z{s`E_k2<{x7AJ_Cu`tqbVpm08Oc>=M=@Rnx_UJ71kyEU}KSi07Qqug_&@P2LD8L<<%s?r)5B+7|hfG=-eUcb<5m&Z}+I8JHhE&G5+ugSOG1wA&crk?n5!GV1hIg?@Z3 z0?Y-Ju%hozVyyv3VpyJ+V~zBA5!rzfSp|I6;y*1L`gl-ASd?5yuskF@(KPFOyL-18 z*jk$~_`!hJ;98JsBNzlVXvB*ZKdtlQAyO{Zd?g^ zV6=EQmp2t|I3I3#3W?O+l)p-|VV~={8nfGMSRxa7Nvw>0MIlI1FbOYaM*%Hnl_7Zk z-)<6Q&^vqo-oyt&+!e{LxnhhMSX^RksMrr)+hMFs6^E&4GW+a2eMz`IMm9tewx`A6 z;Cx`-Kq5i)YzWkYz9OQERgvZe=Ojp8NUsJv$6`1$F_GqU+!=VIN^g%o0jBKbO(+N* z{etC*-t5@dgPQSh!tWO0q(j=u0|^#(V)L`_s-^#)W|`36~M8mK@@7cg+Rwdt$?-hqV* z{WO$dtnU7RQrXsSEtCAK#Voy?warFiD3T=jCEH>=$Eh9cq-^?PIU#FSxRtLwBoLUP zYLm=>czXDFNd%H z&I0{x}rv-Ib$3&exBcOa$j+dT~Pnz zC3PmSEwv^_(Addn)s!hDpl{=s3e9c8s}r2j>xCB!tB=5#sR$`w@e`hyrAtCb1+$o? zBF4y13CD6!`N3F6c*9Ps^ClcEREnB%-vHE{uue}JrJE%Nqjk}}hrjwO8CHODMvo9r zKU19!Go9N!mpWdqIh0u3UZlgP&zTSKgg(g+J+g`!*q^TjFPSrj;=FqX5KM2Z}t&k~`T?Q?_7 zaV9%BQg*EwwS)1OBf4W+ab|$jLU4*I=EdNDC<3YmD98)CU71Xhl$xbxM6F2E(e_36 zCd|pQta|SzQSA9u>Vg=G^8@0l1{vkD%D5-x4kW&Sw~Z?!^QYF*=UtpiHL?k1j_sWG zuMC-T=Kg>i^UbiomHejymSiF8SMDl-(cuWkv>TSD?{kr&Ra zS($)9M7EN?hoU|-g9%_jesevvLwu)&@hmXoSRxWBQIL@-g7z(!tHvB|IdXrkzmziq zB5hnZwf#I(+wc~ApGb5B`HCDjK&y}7N^tAvwbEAO8WA$K2yYa}+V753NaT{b@r!-; z)6nU}qtuRNGgYs0C71MQ!b9an1G1|djm}1WtYPGm!QJBW4`-8l2O=U432aY zuMHP&RLu8CR(gQr^dsKl*OG8|l;;O5tr^RENPiJR+4T#`g4>2RF8XaqI>U> z`oT(K=Py|&BJx+^IrYy%N^Rh=mVFmk2EmvA2Xff7($LN&zxgO+{|5QH$TEuoe7y64C$hE38XX>DPcI^(M`$xrLQdD zBhjY}bopVo=-(=MCj4QaF*sbLa^pbehJvbgh^Z2uFr+6jB{I@r7YCW03<<8E5mX!i z)6!7H9<*eb!s$h#8xc{m)3nl4!SC@MG+WB7!B3ik>`(-g9uVHW&Jw*SD#9eJ7ne^Z z-Y6N{_?E&&$HFvKle+ibpy-WNM(4(H8`FOZbUgP&YOokRP+xQx^fFFv(|#&aWO*qU zEqn00C8r)UvL`oE&&?b%G|AMg8CIyy6iT`UKTW(_u+{02-E%p>8b~yA=(xrGCd%&r zI)K0>^7wm;96h%sG!|p8_`)zn&ey9nHvSxRtMo4Wg0&^@6U}CEm48cBOkPtfBAh}A z{A*f~sJ4oIz-HZbi@jd@?em3vA^)B3JEJTA0)`|1Ir2zuSId>eSLbQ&BmWW+XpFCt zX^E8I1my1CIW&55jb_G!y*@ttX+n1O`$ z0NtA7Myv;_?Ly1MH@p8}|JnZoBEu)|8@J zPrco2Jr4EwJ&1|=kWxMbw2p(hauGje^wqrTy-V!^=|9%1b)Rbu6e4fLEc?{ zKZo?_j^nk>kI6XMKf8B;y=i3`u{CO$AI9T-gooVkSh87J_}kD0R9c>ToZRG6t2x!` z`Zf)0hya`!$S=pq9^7hD{uPlzb!M}SVZh|bEbg!(4q&i)GFLOdQ@Xi-e*|;y>Y}I5 zHA`XoHqmqg9+gi`Md^5ZkLbe(VAI5ji(yIYMI=)2` zup)m=Ri=12tYnSv>lZMj&M0;qJsmXMEZE}S9z0s4MgB6PFY;GVV@%*9Xx&x5a=%gL z2J!|p^i45NuJ-lJ$3>sOiB?9H+&={Uqvh$4SWlV(N+4$@t>l~R-&1q)y!8Hw?&W~l zsaHYnZ}09z-|Ubbw->)>!S9$Kz7!03sIz0d7Hf(`k2*rk7437H36#3~zqw3~)Zr*$H9NH#@CBu#quv@r6T11LFHf>>^ zCG4rw8J^K3!2DyveOODlX3rVb|>y>DkH@pVE3C`lc zPn(8N8)=+P^su8`O*YYZzYp`7g(TV2#7j@!1Zv|4Kr93Nh8et^z*m=T<9dd zCk&oRPEiPDlZBzA{+E$I$2qMy_LYM*%>PJpVNMOrQIc?sh`rs9!Fq=7;_4W2WYHWZ zz2Fai9XQcEEJ%u}sE(U3%w!S>@tTd`$5FN@&$v#Y*WYEs=t@(?7luiYm`ULLNhgEd z2b61cr%?~OPKMj`GJ{}{sVm9&@lT>mw;1xH#)lY#9alnhx57soChxsZmfms0yT^d8 zw^+w>)hZw}qte>0#npP!u5O~srL@)5IzZgzBg_Yj@Y1_Ya*ocyZ&=UOSvZ%Z_ZOV6 z7lsx+*z=+>4z6(%RGk(|z0~D)-b@H{1FE7mt=)lpfy;jZ;~;e^TQz#@RL6#3=TW_y zn?`$v`C?M>POIl7+lQ@CX{x&7PXy%8h_875P=b*)Ky zi3AsLXDHW{tamSWeu@?s@Xef&>>q(kzZ2!H8jye$-7;L;uwHm?>(4B=gwWLt1XNP( z?At#WGP9m5n_>j~a!b&_`UDXvRB55P=E$H-)7AJUg=NZzj+5=sC!T?}blim7M4q6ig32b8Api}Wiui>QK8P3mZWHu|^nGqoO7{;jqf9eYhC^cIeR$S@fX;Ltg{A>M&UHc6mVy8Pu%ibP|8xe)IT3? zczv#bFcj9&zgq^NE6$4X#Cm+Rns9g47j&BHZIMS{ooNSiH}~CNONMqy82Bu3Th`D! zf_$0VT&vMCVmI-kN0z&BlzfKUsu>gdTs9i%Kw|2o$Ma6=%Dgkq5L5B208WyKof2Dh zP1t@IY-ilv@$5M-qX!SH@S0`&}atX3mIN+7c+U{?GcZMZI5tW zASnLkY35uhGqtaUEFvX79fD#vmnf3B_=%GHBLh{W7PYVZkSQBh&k(mEMY$ThB}!&8 zd^#ZriNqes#VU#cW(>0X&G_Bcse8J`Y1uu;*#sL}?H${QRs#;!00$R79&)zR;jrAy zw=#mgm^=oH!D?L6HVwtRX@bjkbHVkJokc^~%nawT1l(Szsa?$2o=#^X+=UX?8JRqj zT^^99nl{06TXlRtm+4@_b3-1X#O6wRQWV_T^^dkku?IMO2lGW z#2&GJOEV}szlLL*1zS9cYJ8$b2s#n1c(5}G()$ffLlsxIKQwl`c}_`s)9Gu{Aiu&_ zEfLP%n2TL_U2X4Ps8cb=1;#Xfc#zZYdqJ+w7}unW2v9D|0*FZk&yg*4z>R-H>2>m(^fR`uw-0msXwe5+l$`fUFP{J%qpLuV4E=GWe8s( z!7-x)l5qMCJt z^Nv|G+Kx&O;Qw-f5|!n@Knl=|xry7hEBAu2#5t=mf)d5^hd}9{08jBNgU~L4EO^+{ zv7P$cdMZB<3=CHwLQ~MCqL}K_oh|$#4Hs5n$)w8b zTF!OoYN7b${WmRBArnLVE)b42hVb{5Ircb47#3?GmO4CX|EJ^bh2riQKeZk&by2%? z>nlKq+~n^CG?t2Nxzvg%(P8mDui`hOtDq>IiM9Ywp21mH(u3$6BVKMBXP7xDrH3Ka zFNcf?%qUYB98Y8-N@wYnYQCOepc!i^3Up740pV>4-@m3pY~n|9;0=U5kpH%1ad6o* zd*z_AJ*{-mnLds-BJ(zcItaU_pFzqA`xrnUNeD=eE}4`98mk8h%*|(82`blaPhH!f zrb!YLm%ZavduL&DIDd~JXT91@(|grw^iagJujq+F+`dZLk2e&B$DE{xfT%}T94cLg zBiLDe%Arv-_1O%%Dl>{Nv_4oqb%;i@irwLCV6<2SPkULGs{BUc1V;%bUQUW~4h9e} zRuGRJn9WUI(60UG)`QZd@tb@$df_f9nWB{a9uv=0#s88TUz^SUQW$>=13pS9MfpT2 z|0yilqW9;vNO2U@G(aZ|J7m{vCqLL=P$GGmKGb86w0!+}X`W++VrR%;v_t|j=aE!; z;O0;%Mv)FTOagiPDCC|%bxV&Q%NGC*yX_<8OvjREPISl}N<6AQ>|siqbT}rMTJ`rs zv@DhdQ83~SO72sYlaD_fQ6Vx)J~E03q^nWbsf<#rEZSdF&EV1{1!Y11iuD@3xcGdq zb?p9)J8jgrVpw3wfKE;5bsYGltQen+^w5F7RREiE`?qr*lVS(xtc-mf@hpG>GQWzN zB*KlqRRr6j7Id~^RD2FU+Ez(4_2Rd482K$s>jfe8>%b>z zqvF)~(U+>Cr;hxsV%QcDptBwJS;UG}ptB#>zlmqbA@lv4>y(5}-S|DB2r$IU7{8Rj zp+w0W5PItO7&^)oYw;7gkXZmVejs6*QQb|?l~+XTlsLs59BckxJ~0w;WYG!;*o=kA z(R`@0KYx_CK)tQwy)y!L!e^CazmUWYx|1IJDTEryvj>Q;QzTu=ny9>xY*8pEZREznLDA6zz=eST}uxg9`q+|uu=^mIbUUd_y zBtNu)c-H|TVr^^D;#&BM^i_fqwr%$IdWt9+BlBa(`8)hkGIS8GZidV12vQxK7x!2P z^L4EofN-4fhU^^j0NZ(+reqOcHxIiXavv^Mmq%r+TPMUCw;clC*lErddY2ZdFx%GX z%s$$NEDXarVsGQIZ^{FN5$t*KKnNYCOhexRgG=wBxoczJA)Ab&jwVM@8-4y|I#e1> zo}xA)P#EMX34aM^I?SRnLZCGIMAEI+_bk=VRPH|CPcDjQ)KebxR`9r}qNw^H^Svqe zduatZ9K*X4yAIu5-BRrKPB6-3a|4of?7Rr^5S{;;D}DIF9})yO6y|~Pr`%~t%CbsDus4&vt4DSgBntQ9hM1Gd}gf+$X-JMJv9Pepll4+eeJCV8G1b^S0M#0qDT)T`IP)oiAgqr zn?eH+XkSpA-Vcci6RKS*eyiJ ze~p$i3xRxPSGcSOCTCG-94X+QO)p5dxcHJFEulq0;nf&+?Q_BDE}^ou8MH<*3RVQX zX;IidYI_6FL#9pDLEd$)TQVD0&S%KNYgPbne0q)!bz-3Xp}A`r ziHIq0WUFK`$5~XO2hl>xpTFJ-;%yyue%jbNo@@q1^Tl;ok%)NCVEi2mP*RT3b>KG~ z6|ds{(GnlC>B#R$P114kdtX7+a|pOZ6#GO6l9c{VBf2#GQ0Hoe*u{~wi)r=^K^z!x zu;8;@-TI?UMfi0Yqf%3QqPcD{TT@3%X+@SrX~UTzQOjSsC>x;)iTw1+;eM-i>u|` z6cK@{tBN?Z?0#v2x-8sdMajT-Si6lITUCABZL-_S0kMY_x6h5P*3z)5FbhB!%7EEv z<+#L}gAW40s{ps-Y-ArfdBdw{`=X|uH@`pJ9@J4m0y7qIchw(8j_7I%Nj9{Jx@kXD#!Sc~k&~gD7J`__N^%OhV810K zF)E?oTAB>kp)I;~IQ!J0Q2^{>6(=jb>l)q{Y@citM;X#cD2|{dr|KBW($91-G@}C9 z3>VFWJcxGgVu+G;Tq7N~f^7j^B4I>I|Mj3PSJ9{6Ib!{-IfY2HekFSHIy&IBOsrc$ zsSBHCMZcNuctap`L!gFXm)YXH3fgQlZ~DQsY_#hk%@g&P9kBJX;r)bM0*qPRC<&_C zJi)98O6TlLw8~3*&^jF7B$QnAsbLPi?u6u>S=8O9u#OQQhY@dhlfNWt)s3zx@#0kD z(I)Q2o5}`LF(2w4^41Z@fEo7Lu0&p{kg6w~)&bUVnM8Y>M{}IadZW$X_5GKO3H!RS zPtq?B0oZH~4%Z+C1rnj7ePg!ILN{65pfKXw=F%uuwGv77%mf!x4y%~symfB0 zFkBl2NxNuSM>x;51zJSP9ks`#rd^Rl8{465gsQIvaM#okbM~YehiVSadGtl7BX(3Z zdnPe?7GVfnnk{rgR2@g6D}oinK$gMZ?|(#Fso|XJ04Sp?ehpYqG|hS?79{zi#xd$~ zvBA0r5_Z`sb*ue_O&UXI;46;oSw#|xwQ`fAPfF$}y$e&!NEeII61I(hKcQ%WF&b0^P% zlVMn31~>I8?DdX%IA1ayoi!y-Ws5Nn4{?%vc#*FRO9zZpA3J@<_Bdg{u)&&o_98uw z{{hd$2#YQ6GUhO*`ui1|V}Q;zt4TBga@QYH0x)q%VeZku5JUPYD%(*>>;=~oYuV3A z?f|K!*Mpj{ zzz~6`JRWdmYhz=91?UZ)JtBFrRqcCAj%!R$pM2V5r72dQM;Zc0$Hb1Lt7`F6gLR2p zBP;5$GTd|wJeg?VVO%O_JoXquww*kT_*QBY4ufq;&^E{!w$ImJsEfRAS8Y7eaBZX* zPqct_3ngkauHag?YW2%mAF%OU)3sQ%?45u}CcBk&oQ%zyJ$c|oB>^Mm^ zoQVbhdu7k#leV?PCA$lc9~`ahpdkjVrmX#K8}V&6?*LC&_AD%hECmPsH@2arE_H%C zQyslBKzs3qN_R;?P-Do26KnYR8I78YuMgfbFX?A=Dj*&TJ81mNDL(zda|2^R2Qe)s z6Sn^D!6s!=^7ncU(S6PI1as$A0L6xJu=M9lm99jT=2&Va;+kw%OT0J5So}n zI8C1MNKC{G?+sKAe85+b zK%TUFSV+JzpD~XhsX(y4Gb$s=9b|p5G5po=GXUTDz>74V@I8FQ745rGHItKpFuj@> z6B+H$o3nv5jPxgbZxG^@PzV4tvE2VhFJuq`q7t`j?j0IJXqh}FUy}bN#`r^+^pdFL zJzU|ySB!uRcDP*v^RFaazkr||Wb_@F1QQY?Q<^>^_d)2e`oP^?cq1=YF7fbd?*1GY zxdtV%WEcFDaQd|r(EV57^C&q=7ZK@PExyH7^w=8TJU55<6C-iqG+{q={j|d`4oP@> zguWQ8P0OkOsB-=PYx{q5(wZXSN#a1ER0Q9o5Ar%-lBR!fnuF9%Cvg#R(&GGcMeSRD zR`dJ$KOd|6#}oHR96l5j?B8K3C0Yas!aqYFFN>$QZ`oQruq20Qa?GS_0dA-!8!AXn zkpIlAlw?W+`_Ifj_-k(8{jhSlB;X@l+_>pH*rY5-YF2>FKQQBKyXrh=qxplj;rBEe zwKa-$Xgd@}4@!>etUT0@9Aazsrjf?9jp{Z^TB>I*U-_Y{cbIf!&2uQ zWfY7h3ri`}ev4vCiiYKyLGQ$ThIRD38(>)qZkwgTx>SoZBk_sm@rfm^q@PoH(vJ6%tu1#WyF}5r`0uXK6qVXiE*%T1kmP}J>y1Ct0bPn3; zY5e?5#qE!qjc*Mi#b{a%#E~R1qehvb72v&`P91ldqE84!8VkmhY&_5)yMMY%0y>B> zguQP$_@nduRC*C*V4Bx*S$l$BROBzJ^asKeY;YkMbvTo)M0KA%KUqDik%$C;0WV%g z4Hfx_3d6YlN5W+n!=Zy@?*#(}|96C_^=2^e^&Np6VuF>m6!yRB1nmqi4^4Q00I~>K zVK64b3N(4=V`frn){B)WlW5ltUXJcP3WwnRF_rx#H!F(lrowm&;o#v1a>+Had2O$~ zonuffe9la$!@ZuF!WGgJuhIMZrO09ouEtpqz!G5ayt9Xo=C0H!Hh4ZYzrR&3| zuE<)n{na`eLecW~l{z1Av3i4W+-O_5!FS;rlnV01CS;< zg%-yAcoC6OkB!u9+*TpJ;CPqe;-41tjSB-TBXJ!yhnZ{p4lcT5~Waryok5LA#(jq6e7~QHfe=VFfS&UvAXYzhP1aU+#nG<3E zK+yvU`$_!Ar2hTSVldR_F2LgF@rDRfw@hns$#RedUbcpv3ap(0)U{|Ytd49n=@#(R zqtPuH3w=L8WR2Oml7Cc0P|6&T>JIk8$c1(JR%TEmY1q^MN9;JfDx3PJX;ifdwiY6d zQQEY+s4?^g;tY9qgZuPjPR1s9s}2VADvhgAq#p~kpNLFr_>)7}B_OR4l2a30$=@3o z^5k^1eTE2+d|J$WJ+!;=f;HWf@Ydhx2p_(R*;2r|}tskB{Dk@}uxdh}??AFZ^2=Pp6or#ObH{#2_}gFI)T% zM_LT*pVtH*zfLs@3IRe_aU2ImbkK757npy=e}1CnUk!`6(itqxEpFX-R+80(Z>(Tm zljH+=`~T=HK!s#S@hK3!LH~8iUV&gY*A|{Q_>W zrG;+>vkF|&jWa7|(smCQZWBK*Bry2DsLq4ceVF(Cb{BNgjW|4J(sVIAN)np_E*U^c z>pOX5OE+d8*-^AKpG$P4TqOnTl~L9d5ZA(mZ=gk@(q;dN>v_MQK0YEr`M@RXqcC*# zEdRx?=TNwN&@-c^_rb~~>v9oiuK*Q32RXUvb>Z^X*|3Pv#ru3>2Gpkxy~B`+kOz-S zB)y9rS;D{~T_#5^9eZc#MkK9?xd6CafubYA9dvTfBq>@htHbt&=Wks3XcUb`p9BHx3ww4=t#H;|&vD5*57 z`^)2>=l~O&LuR&m>sZiCy1`jv9FtFbgw;C=yQF-SDzBu$D?SOPAoo-M3)pzrnjDm?cU8L?f>g?!h z->~l`D_7+a>nE*&VepWiVf>+$_bHNl2`^NPvvyD6IYrkoet6I*5T`@=YN)kJFoS_a zRo4;jW-^JyMl}#h#xqxgZ3uYFFIc%v6eH7g5jknU7EUNH)R4onn+%E7vpltkw8~e` zyUIOD93lT=z`GPv=|)0V$v3}*cTmcwHK@uzvTdF$l@wK=Wc!G_td$%jOxM7n2vSwU z$Vj^ttl6(P2rPKL687#g-_a=A#Q~Fia2|6#gD>9ZirhxVR~UB~KLd#8q~)At@1D5B zdEmn~SMG_>oVr8PpSgnH%3h6O4PSUa~7wWv;MU|%m8zy;aMc9!hvU?@Ry`lEavp9m8wAn(1s~U=sn-H{$NvZ(9-4#kD6_f9y`KE%V0D{EmhTKTGcv9T|yAm5lJB493}ATV3nsQw76P280d=gAp)2|I3-@y5z3>ViSya| zL#BkkmA9kC#lDT@lXil!F7#UsX@?z|L?bCyV93E4=<^bdOG59Wb^7A%tmh>4tLwPm zB13KyvMp+mk@kVQz70$E1H^_03D)`t14cN+6)_I1u)n4OXwPTyBj_`GNI!_?7X$)r zKgd~0={24EDFL8&q!UdiU~$ADFY6eHIR4ls;91N3z%DPN%OByBZugxx{PxKq67XBX z7gFsd_+ESc=8tx6|7r|BJ%PyY?)&Mv>4aB#KI3JFY zx_x5_1h#=iv4?>60NFVw#A2zH5RA%3yKYPf(_D&V*?ItE`<5vE0Dy2lFf&X{r#&L= zo~9+}1T16knqci>;C$;5dGswEHiRBSN90tZj;~_-o(4bf4Is&qo}6UHqYd zK%&ovD3^G_r|1a=Q3}coBTPJ$|AhAPuU+oNx_2fmy4aP#a-6z@-JR7Wd4*HC&90n4F0uW|KQ*M zz}WxAzotnR>V9zee{$U?)aFC^Z)wWvr0iIDyrgasZt$jVF6!w2WWu^&g2|8H`E3OJ zzd#vRoeB2ulq8*BoTStlY8vXOYG|L3vFip{;0p5;bm;Lwmeq^!1*5wtBGvu5B^6wx z>;|OLyT)<3$?Sm30UPkBZI(W4aCDBbY(_Y0``z;JZ1|8h_7>piy`LN`0jy@0Y2L@~uCYqpp5O!(oJvdYt9D4y#`o4SyFNGc)9;!XYpl^ZH zVnNhQW%K6zY-Bk`o{Db&F{Vu9DQc$7gq!qaKKdX`w01y95}Gx-ZrAvAC>Uoiysafh zlKsq_n=I~9yTU|>`$u94Zp=nX0vIB>%tq=x!A{%|%f`6gV;D5ye7L+d4Yib}=lq(K zC+wz{EP86p_VM4q;owxhjqQ(-8UtH`rJ%fe#U)Hw%Ne4H`2jE2nF6u!WQn*$9y!1D z`gUHHV*>!M_~N4;)NOOY5S#(AH=i+7pvZWg0v~qKFewc+G7~oI1rQlaJglBz0&-oe z$MRzp+b>VU^nM+Ku(%4zS}FsP%vPg9K3RDtn)Jm8d+FZQPT|?)K<(wl&M~i82;|Zz zqyXnZa=4uO9FF=9JzUg7e7{@|w8(+H0PXFj`fb3m0@k=@#94 z0GLV8K3HE`{~W=_bj2GbkK7m%cD<;xpv(OUV^t3=6Puv4NXv-~X)d-Y#O^u=9?rv^ z`LbNf0nyGtkF>!c*hA>AL>B+|-*mBC>I9t{)e@DZ?yz=J>N=Sm=s)da9pzJ*ss6tE zoyY=CXQDH@Ge2d~ueNp2uXcQL&Ol2G9Bj!$;Gty#HJvP-C_qu$_>^yu`4Y3qITk!v z=75wpOrGA3^plolBP|v~nzW*x;ljk4DV+htL00>!#GKxdkd#lEx0cKd$gRsSB>{(R z=ZrH+e}xs@=d>3CrbJDK6NVa!35eqZ9u5E}qq_l0SrbDQ;*6}(p;I@3%l0~g4Hyyq zP&`bBXBkocmDuLuy`_TYk=g@E&yvZ_L_t+!n0Ty-{60dP&xkHIDL>HtKk3l7cDbP+ zI1y|!Q>4@>oX6>kp9y{yKO-+;`zku1Hj$pccXf#ToVRvBG(O8?>)IS?$_*xdkU0Uw zMdM|8JxZwSo@U2YyDyr`@=_w(nkYC|d)&&sasDij3R5bMuaXhDtV#4Mutp36T?v&5 zZs}KoA{F*wjMz25d+yI;>iOGn7`tSK(DK+=K%h6-5Z%EZ(4CErL!yO017d%w~(;rVX)eOb#_`VpaOk?AQj`OcPDzv8 zE9hAvk(am~2Ga1xRw$p;yF30G%W?{8o7k)9+4Uz7zgFG3F5KAz$5yMASgh4svy~|# zsGGcrBv2(KaKWM+Iy%itsre_Mu9hxBlxYpEIpTrRT0AFZaqb#yy%J4RJmcJTG3Ll7 zVBYsNPY3a6ieBatXls#$tU^85#g+#K9>uSonNQi+YQ z86+~7_^d&y!?`e6oB0yKYEzP{>n_aYkji-kTf#HX>es7DqUs%K_{w8_~^P zIhK*XO~9*6LJZyFBpXa8B}E~(I_h!vDYDZJqbIK>7E&m0j~}~sE=iQEo8`12^R{C zh_WfL7(7hp<1l00HW4aD%HvnZ8c5r8ZiyP-ay}ry&B6-G4}|!a!hTeg_dmz}{s-{? z?>&PnUdUtj`$nM+ndA@(iJP=7#EF@-X#)xM&2Gje2dp&G#H(bBTt|)$V8kC#B@t7B z3sdxl?B2~5nxv48S-Tcu@t^YxSUo>YNmwq?)l}>teRAsjX z45v>Q?&%zyGg1hNcxwL45UzXta4ggn`iDUoZnP)bI!GnF7)yn5FxxSP8GMSFC7&(a zM(9a<4ls)uQ2{2j5t@P;bjF#!BlB`9Y+#N}TvTKoBb6_$g}Lssb#@jstiC>T=OG0c zs9WD5o``Gwshz2@x@$;gH}UL6PHqvE3aylUNhX(^x%*HPm=R4CFe4VhF7 zB7$Eb?IH8*iax5-0Z0wbau$Utrh^v!#Y?SSf+u{!+suA%od@Waee*&qBJwZ4F)UQ7 zMW`#4rj8}IXdF$EvI(c>eSYh&5ebSFD_nz~>O!lMWF$oK6^r;uNCT60+3v7aFazp$ zWRY9=`7O{BhBN7l$tnDE>%k&-=`LTkAMfgersh5EGgU{;z%oSk zu^wirge{EU(33Y%ljWgN1u-w*Y#F!sBCVpu11B$$D`QoH?bP1>^)&jQ1@M2DMx!zT z2$x=xfIPJ^;4>yFA&j{B8uXpw#+1sHND;c_iM-_~W`FPQ0>7Jx%_hGlUI=E4yj)$q zPTsJ3_0R0CEb}>w@itY?L&vuy+Fsan1o74baTlL0r|IycpExxi65N1j)|TVaQR$f{ zSA40|Q3`5FKA#B(m7K|2L=_9R;zPF3`Ad-Ut|Nyz;O}*6)4Mvoxl`AOkj3Iw8s$bD zvd|=Ir@0k21;mjcCGw&VfrwwECE!{9*1LWEqbC-y1{-9cl`bKanZ~R+=`-r#wKVE7L7zF=pJpZ|n#yo_zd^zA+K?CZ z*iC{msvJi(itR0i+B7f7v!?(e526Ur;NacGw>} zAW1PHgir$hZzud16!jS0N(R}$uL5DT1Yy)X#v3v0D;nHs3Jaf8!!=b6D zEFfv4L1DsCpL%U}1HvHydY>FZ~r~=AyOgPiZ+FoNru{eh(`K^)taTi zuG5j}Na87gt;lAFkXvH6Q{`2uN04IyNo(tPv<+BUtbzW>!$F_BlSq(Uf2i2cf`V@3 z%~~j(i=to+ozPBHdN-W@Q>iNlEbCU}4UMq^PRL$V3C5RL$9R?|MX3{$s6{I< zyVEgl+8cM>4!t9ohFax#D9 zPV2P*dxRre?fxHq_g`|c2JK@ICmL>OQB;?|iO2%>hAF7FD(93h-EN-c(AY#+K=ei$ ziDQOZg>q$ot>zO>hKkT~X=q@1LS(N*vkPSkv}vTd!{#+cjnjERgw`KIfqudi4L8=y zNRxdN%FIzHzln=yZ8@t1;ZMW9VxK^~a!)aUxAaVJATxKFXlcPUIqqlNWN$>W&r9~# zocK>cKNGT1p+4C=o?IROP%)3%Hf6i3OXQkCe?p`i%>5TNxNEzv7|buyDJ5axB&Az$ zOX57ON||&24=ukPAz{h93TjFh1ho>^35Imx$ZgHqs9kBrL;)eJA&Xi%vY6+8f)p3( z-&p*P(0|JQ|2sY5?7?HkbyvV9MM!YNCg~4yK>!j|wbcIEVg&_&?b;`W84kdzK$)ADsYtc| z$o3N`rpRqvHE{glzZW1&?rvpe*Oh}FdP=ze7N<>~v_H zb>;ZT>GgcH_3IPln#eai2h?c6e25Hs4~5+D87;|#1>2|@WnclTlmVj;tx_U+6A*OP zOa#rI_e>L{89p+8fT**umC+e> zoG*Caz&_B8afXj5DYsZwf>?PbkPhkyPnet5X_SVqD1=IBkd}oH-TdRc3Q*z94QOqp zY&V_Fdtfvo$~ISLQXCI2NHAE__;a14;MQMX$LWCXkZ^P*D7U0){NNg4P1dpMD6O`k z5;^39xAW++(w;J7UrWtuM}q^sW`BsX$9%^l^r6)QZ$|UST00__=#+|}O1p>23bW9d zuXL`I!bGp+Xj^`QN{zD+2QYq!FIVul4v@0&P#bA|Zrov_A~A;$scb#_Yx0PO(&)^x zEf`!+j6U*=BZLlr1LaZQ4ow3e_E}Nv=jPImAK8`8B#+^mB8SnfV*dXl}skOekxFF=h8f|1lf{qXf* zDL=s1gEwzSKjk1|*9w12pCWXKt8>7Ot8)ku2Z6?ojUn#N{iP$pYi!qcvts{;^5#>swW#}nm$Oa4*Lr10oSne|_R-gK-e*K)cfVOv0 zMd~?JG8yx$!u{EveE^1)xvta)yk1(0%iI}hbc3yh>oi-wT7ACCq@zl!y7eO_J-v+8 z-35=Cc>W{R*oX&2SkBHgj+*sG!t52Ei`|7HEp<$lYaa~yFT>+KwHklDL&@{_GY|s! zP8I!MI0u7hTZq?Dm}7))V|AS}mq1%*&r4xK<2>UE5JZN++^1iMJ!gx3IwXA*;@_n7Noj!!5tkVOaL zQM{sjY}C(#iOGP(lae##qTY zM+(u*(|46(HDNWKJ=#S&AQqc^I${=oqpb|dPXTgkS(pEzTIg9QX? z5pBviR`v26SVvT1rDKd!RVy)R9ei&3>u2UEbCaes=*AT1EE~0q+ho}M6HVC~``HNW zr6ChjVenXQk3GUJ)?dipduviid?DbI=^bX3ooD+EGj$wiyNSApfyacm^F82yPIOgn zkHnh&;RCFrAWoF=Uf-c*Z^7}@m#ZWuCR{&30kWz9d%Kldw9S`xr{=njKf)z>kL!~56ta%}PzVqzDp6s{eYJNbdPJago5=*WE=TJp}Rs^}N@4 z@xuwl^1yYix;SQ9JGX^?)_RAw%pF8x{qwvX#`hc*?$)S9svZ-&EaP^y&iRzi{B-I2 z_F{qB4X=GNK%`x7UxEp(-5A}(KsoIsWr4nlA>X2LfX*7Y<5@%3uotTb2hzj=kdqtv zX`{4absC)x2|v5gP3zUJ4l9)rW;o5OnH5L$BH4=QW2eHopL81rQc`;q-Mw8t>)h2+ zEB)!G?aIl_a<($76R+KK#*qSZMujxon4lxVRj!CcAveSZHj7>{Lf^&&719&c#w{(3 z?Id%aql8j@-nf7N2Z!9g8Tm;Nu9I&4*&pj>JAY6;$diO1Jf40W$2 zWbrF2E}}UqWi=~sL_0I4uZwbQwo*x@gTV(ecgVhC?(n!W7TZP`&&Y(bR`4P~9R*84 zK#XDR&em?h>XmInWzfX;7jx^Ybmb_Er#^IeTJy1_4z;RwlI3j2-cl+x04`Ibt^~QZ zSU^^T0*dFAzACVLU)O0J{!qy&*p^@!rjlr2tM*0X{fO@O7F8sb`0c>TNJ%e5zZL>2 zN3Yx62xt}dYmm{lHf#_dTtACH$0#ZJsD8){j~iZ?#gT{2>02HB%oFzCpSa?-*tf>Q zd`K}JMsim%gTdnZ^(xi0faxgYNoc&NQsLA-#`KUE*msuIzx!o2L%a%yM#71<7lXzi zKiixdMmwBS4OB?1>t6pwbU=uxjqGzKWis z5O$%pLoQLhiJ9#`QDTh7m+Akh`TiSs-`NT9w;|?#XD3iV{}LR22a@Om|C@8*yjlx2 zeP`N&ph*l(z?lD(b(=~Hn9hO!EeTi2kNhV+0iZfN{0XabOp9h`J^aPo1{(J84= z8|5A_vvd$D$Ffbxw8FoTuW2r7t}Ib5D`1@{ZIiLd!bEi0yoFn%VhCtnV4#__Z!=!* zUG*wq8JXUXylNVGYVM=WD05xAb?=znRb5~W>rA@VW25Qw>aLClxD)=~XA_cdI1}DB z{L40a;jeQ!U#}DAYDlKFG;TA&*zaV=e9ZdoLdyNlrsbw383?h!(tJ?i5(PGe;1=GI z1jh}^Y9tuEz6=LHPB>g{)i=%O;6?0AZ$tQ#4Ot(6OI(ZMDP33uYW@X&C0@1J|9^HNSl!)jJskFYZU9q^*n#ZHs1>i5NR&Qw{6VmQIa3&;dznF*Jv=*J8r`$Rs) zxqBPjzTjLXb97W*ai?fsH3#@`S$-MxAGlmH9l%lr03#ziWDe15xA3oXFTJ80kQE?{ z+cA9@4R2w&n@UW+LHO*bNk4&^^Z?$ot^RAu4?DxjBj9&oae>sk6abQEp!6#giqI+4 z4kOP59*Oc~=U_jzbkS_d7Q-h_3?b2n-|^PTII?jqay|%O32wA`yU-pAcyiJ2OXLm^ zjO%DVz*3ZVEM@CC&P;2d$FBhIEL(ysB7ymSDhXF3v`l_eSkUo4u;hyai5uYG)BwZV zx8*?gW~?@}G|#YQMsk3N^0gsfRdf`WoYACQ_U*qNqW@m$|I4xLAF${@g}=Z0#*y`J z9MSuxO?^9eH)W2%oBS77YGMlB9r!<+n&~-st^XR*FTij9*RJP!hNB%+^4m`V07B=AWv%#g52+h;rwfDH{TXVf9=Plyp+DX3}_scKgcu)uyMOjK0y z?)dqdKdD$kF|d@fN{ek)UIOLH7AABuYxO6*WGB3V-)FGMB+|&idub)o@;$8O2t8%x z^t@i@A8Vp`i<$tUXEQn&y^KM928US>AY6Bmwui(Z%txb z*gALO{t)2@_>j%> zyApb4NKj8y7ed%`Du+2UAKc4XCQg`JWuI8G*T66E1=h6#toH19fpl-AJXbsR{XOOfv-(%D}~k);P@hM*?wEec;{ zmSvq@3>O0`I8)HzyQPz3ILl^p+ZP`2qO!;RbmfKp-4bkCf;vw#)c-_K6BDN9^QX;=t`Yb801< z!q@~PhP7fe%v1hycq(=JJArXnVCZ&!#9l7UV|&_e^hmcL-IV;e{Z9 zIC+sXHRG4fX-d3EN+~UNia1aE}!RGd+&O4u)gqib(bkR$GWFg^3 z9@zja{|%kDGN4Betv!3Y2*$T?JKYZxy*+>11l$P5cXBzdDM{RR(?Fxk{_-k(@y;dJ zEXcs*mq>Y*LuIZ8F-tZBV$HI=1U-NzM&iWu(X2vKw(+bb4oEdutA`%M9lT5V+7hVB zfEmDn)m>$usl^Md-7R8!u|FW7dk($rq7AdnR2l6E3TO`fe{{WbbS2%hK0dKCv2EM7 zIk9aUljOv)9x}SURcm4i2XVvPnd#~<3wY#dGUG>!9+b#H@ zbHs3g*0&anYj@YoL^iWKF0Wb}kgfnIFi%tm>Swnvz5VMT8v(pvnz860T|pjR_b{QB zQ-sRlDb~JbwT~!!2Ty3BdW}tBvoHetYo?m&R-4Rhm=lRc!|=3m#F`xsSXbqC+03@( z-}9AiS*ut(@(+x0)8)ux0Tw|z2JRbq!h_)nEW#WJh@%oOp|2jQZ{8|Sjt<4AGz1T9 zn|+2=*Sq=nVDR_~1u~=Ahdt*H>Y7VG1(+bR#02U`X}aR8E(%`MSr#Ua+$oh^d5WMm z#V6~^Pk@?DBNL99Nk6Q3kL`65L01QE9x0qQ~Zn=U4 zo7}=4J_|W@wn_G1;h)rF+(unTw`6GBQ}v?rhdJUfbI77&)R2rI;ua{CLKa%dc66R< zRE(;BdYH(iJE_Pnl$@MK6AbuTN<#TVK;H!=R1pc{mfM3AqCPNq)<2}?ys#>OdrG(iF}oC;NQ(MIVd34QVnnYn^au$7fTnRFZ8M^3d!mgssSB@= zW|&MXV|#voC~6{mJK{G1zM{JY^WYWkM zRN=4LZg1eF0lRE#H&y;mSUx^=!I+6RA}IP&18}=WiCG$&xO$$%@Jd%WA;SSQ1A9Wg zUhQx&c_}-WQ~|9EAs&cPdY*E>?jZ6xn3|d4W;E!AFkzM^8X;{^-8YGz0`Bc$|wHfb1mA???ebCXnvt|;Ir7hsSE za4E$%R+0UY=v>*ZPd;;h6ZPj47mkO6X#g8!_<@*E3Nf%IJHt1y*Vp~|!3}QyN{B(k z3l0%k#|0rjuow6XCQ4-o6Qh(UUQ#$LPpDj|{{scqm;u588HCS|ol;W)lroR{f|1;x zna&&zfK=d(eGd&fA}&=yCX}M3ZkDQ+Zl)Hgk+P=-u_9sQR2NxCfRy>QwFP>GyUtn- zJ1*J(Jsj?_3ywn9UN+b$f{6|}lzgC03@a6S5XDH7PE}lU0eX-e zdQc6SfJ)eNR{R-r97V&m$!3b3PPa2Q@&X10@FOCV_LT&KIqsMS1GBW15Ei%3x`KDL zhR;`${JCt=L-mZ8<`;g-ORS8qWN}a7w{7L}>oTGt$|@^gN!bVd>WXVgIOyHw%(!oC zmBG3}4`0JoHTTYp-K5L77r%u@c3Yai)&-ua+4kn`L1@1n$b3W2?;I+FW+W*OpRM7f z0hEQ$Zk{*^i;I0jPEZviLr0bt7rSs5Fp^h6b}HXl8g~@j%v;1dDqAhYWto!Mm+UM8PAp5#3?P7ywiSZXkNBQ}GCka`DpRGld8(n)CNJL8K!h~< z=gE;pA}!rm#T4wWCQru>S4S7ZskhU_evLc~gp%n)C_u?{|HBid_-`#di0w&iI=(PE zQq4!e-@y4lZtDLCoWCg1y~^Q!Y>!BPy23ONz}-MlOdydsH!3h%oExh(ml(tZ6#Va) z)^`fLIe0|=PtSR;bQDzU(`!~k1~ThGeGabzL23QL1mXqx&(GoUPTK*Wp|=kxP&tzY z6X@^Ehtay4Nt+1*zWjN1tE@1H9SHc}iCP`9pd7)$dHzn%UI0Z4g~1L`}(zhZ6( z9yloC-vP<9)H$x7K|<&kAP71pK5)mK4+Y5d%m)ditAc_B@{+O!|1-|ZRMZUf6Ift` z1on3${xc2^*tbmz4UB!}gK7mQV`T>W_tv>7S;?XQ6|+a7Jiz}Iu_mGPK>roJrlC6j z>oN6luyTPQ{C#4uGYGLRpYtpJ1uke?waf7`!2Y?v$o@U%9H=i}Fuwwo51{dZLR^UW zfOQvK3-mv}^D;KFHlv)YZpNk7-VtQH%!@GrR8B$ooycl?NS!b+c6-h4)R&4rUEzRT^^ya`0(~je7k`LrZ`5X-Dh#nk-MN9Sq^&>wi_0m<_oWyVdil zCD&&5E(4(RBzUoG^WD;-q1VD+z zTxG2A1V2GNYxS$2EtKKeT$$+N=w3YOD)dP!_V6n#RfOCf7yV$35v=W)cC1$s8>^W& z>PxabFTECL>H4VK%--=?$)2K4k)6kq^DExM6XS#Yd8|>UzCrBb!HecZygpluvSKdt z*Nm?7;xTIymov0XI9z8%Lz4O`Qh=lk-oh1bO;s2TO_<&&=kfbP!-gt#E~N_Iyb^Wp zbk5J2b6@i9hB5m&#l{cA>0Hc}Ci?*YLuglf%OJK6av7ZkYZFjwa-{Z0t{Qr`;L z-jEn=JRxdXo-#rm_R>O^)4vCeW%{g5bodQzFkGOWXyWfHjWr;-B@ODCQ*tMe& zT)IKkHP7Cssj>8g5-g6iWg?@{q2w%``GM8b!1;_osrrgyF3=GZPuZVnN>x%ozraKsh%Ufr!58=&$bet&P7B*$9 z%t$B8SuIf~bbhxK*><&#H%+yUYZ!u*YxskGYc02_8kf!2IHGwfxrkG2k=5;vZ6E8X zOI0{r1+HX9n|2=%jtt0<-ZFn?Z;Ul5sZ|+?c3K~wn|j&fwrP3Mw%T#YQT%y)k~Qj3 zSTXh4oYaQrV6SF~N509t61K+mXJt&>xld~Y`E05tZCX$9CmYV0*Cs={iI>6(&sy=% z-F2;)YE1ByVs;0w2`+*q*$}IWsUX^7$`XE$*o9;T`;V!nY6<|g>@>E(Svz-NzU+;P zB&#H1&R8t0tAzuply`yGGVYwqLQ2QAlqpMVcl_iI!;5Y^P#}H4a`#k37w#)od&vRL zPCm)9z8ArcmvANQ3zrb#9OK*q2$d`7VsQ&~zj*kz%ornT%#U{{DZj72X&{&A+z$%` zWTWmOi!G%UQxpK~kfoMI63}uxIOviN%)Fuw3?%KPjaZ~PE1LG%fzV>LC^~;fDbkyPK4DLt!m}P$e|PEO*o4P!I&empA|k~!5xxws zcgwB5LrQ7&mjL^+{g zaB-*_!BB*X1;M4K_a7)nH#A452;Kqu0iRfA?jzwo^;?3%pFoQn>YU+iM`XY!6aC#3KANB9g@=(_X+>jm_Xx*>BPA;!}1uIp#RwL|LLH~ z{Pj-~cv(F_5dVq}K|{CipB{ie^xtF(e4zg$Efzq>0aqOD!@Rpwzf)mvrG2Lja{_uIDOm~eou^$*85rJn|&p|*%Gi^Yy=NVXUm?U^s zx)x(>ClNvXB`fQ;kHxgdbk=Px;O&{!-^_NDIqDD#UU%<^++P3a2;H9Hev?+Y3un|u za~ELF9@)&LV6@uDLs;wVGSwFvAP$kV~fM$i%nQast+c+cR(MFqG z`&B#Qn5&5Xvh~f5Ny=|jJ;K5~3a!T|>2m`&+AKVxKBYg;7qc=Eauy!ScE|~s0u}&m zDSdcHsGgyj@SJcPrtMz%uDTzBsYH>SsBPiLC{+2AOoIARQ~HCO9z5`;%C$3If2s~q zJ>k^sTciNoqqt`DiZ2t~t9(Ap#7=WUKKQi9VyI3IaF|$$`{HPa`JJ4Ic(r?LdL1DT z5jrS4<1jNur>>ow?isJ}QN%I7ALIeAt8-eVa4CSTk;aET^HVizsoRbPx?x$yk!V4N zXA|{k=Gk;E(MP)Ko>0*2k*wcXlXSoG(+&klj*JN_(v@FlC!;?TY5cihrxW<2%IavU zjei&=vK2Yy6zCBhkP!EE@FlrVgBg!BGc)g>p)v2jkQk4kR~V15TMuOwE))Soq9m@$ z)8q2qkbcZUq+#x&B<_Tnppi^+vs~%Ix=jc3RPIl!m)(n$HF8|}P%m=ho>j&18X&rr z07B{B7k3ok7I6-e^Uiw+ zacf!ekk&#~Y^U3A*Cn6JvxRImEp6tPVbvuGp+&MOu1sa}g7cb0DSN7{z!oN`fwlg= zOrC87QT;?NzMFmc7OX1|EuuV*Oh<8GWg?l>@)P5&YU)XU) z@8FM5;~u0Kvz@1FJW5Ns6T004WoSrXvpM2d>R!n@wvR8s zv|KmE4i(HMl>;!kkijr>FtWc`XTv$x}E`8o>lhK;n7%8E@-nrEUpXIx>6rD061oGm?(o>nyV;~SkB z794*)@e+ZB%4_(ynL<9*@p<(A&I@N48X{oP6?Tv_S0=UUL{1~!oC2n`wa|D5 z4{hr{=FI`Mt)-xe5{D9dAtbGxwQ4A_E-m@>r*PmK8mWBW4zjXKMRrkGU7FxT#lJxK zv=B$MT;G$qF3Yy;*^JDWu76)1jQGR(S^O6&;}a<3GmlgpMY;!C>*ZPl)7H)Gq{vm4 zuljKcggb~~X;R=v)>F&ip^|;mL@uGqiOhN0d`bZ%=V9nW2}6UcrN{z@ zAyFP0wko`29$-P8Yy2R67)?_$#XssCeHHdiCY2%9dc=}Lesk+)ty}8=L*+tg1%<(H zGXf3U;HgGQgDb^8yHVR*zlikI5IL`)xabF<_)JQ?IF}l?7TQQJ42mW&+ZgGg9@Vli zmy7;^cOrt~LLBRXHozNE49V2Y|AJ_J`k|`OnKXr_O02B z()l5ydm-%iy4E=UbOg4bWs%1{BE<9`z7!3@jp+kRQm!!UzX{y1(8az3pDkO@Q$StLBup%CVb8YLZsOFJjM*n77~7U;=B zAH1%fct;n$uZVdPi`bH~l^6jK$L444tt?yFnlf?HhIQTNUa27E*qL+G{CV%dDtels zIdA(-vW9Xq4HKckotu6b8|?0hB@7eRlMcy{acZc&BDq;m?@W{#h}0pO4Wvo|nvWih zmx#P59nZNGW?@5p8g^JAxtyD#;(38cG{LF1xy**bP@Zkz_6Qlo@Qx^y3GnDiy{n16 z#qpxFT7ICVatG&4saDqR9KHwu%7xxvh-ZWaySAv;$a5SL2Tg|~2o?C8<@Or`hz%O} zHY0Fk=o{K&G3FEX<_q=64g8LP#aK_K@#g4sZ2@9>y5Y67RBdAiSNg(RL=kTGE*-`7 z=5nPOyEV$(LS%#Z`&PU5HS<%e#)qccc;2?*WbXFi zti)>F6?fT{Sn>-hGDR0n#fBTjC6f~5Vgkj(G2-p zTAVA)gTbIKdFC-radz^HiK9JY7bmKQ=p)8*SOJm2xD_ubhGXA}1iVE?j)#kvvX4Y{>gQE^p=pi~v4UcRo9C{kQ%3 zkG(>I{_@u}{WdB?PX7elcE13X-T#7s4xkB!S6%F*Tg^q1LTfC*&m#6TirUT9#Q5*adZErL}PAlSz#N%$a~ z`-UHDk@QKpC8_PZn#KQp`YC(eUf|8=iJ%WDbs&7lot7TVh)QWt%7jS-YET3x!f+9I zCB_tb3w^bh6CE856N=$6NcF22TjD;nT6@FoX%S(HCSya{t@DSQ+n`%(*rHdxx!uHG zht0}TmqQ}4Kg=M@hIHsHKvNX@RxMO1^#Y6ME_4`kLI>ii$`T@pE!*oe<3{Z><3>O! zRO|x2B%C7(2c9L^4_lBAkirokngst*~->>fpj6go7_Bf8Z@R@AvDU z2bDKn-?hn@2DTrf2-AIte5O2ZuK2UF!362s&)aL$1;Lvj6et0KmkovY+@jI;Pq(39 z1?;dqNNTg8?!@wdG0!|h99EVFSaarLqA+gkujrE|!Xu;*aS6a%F+2tqx%d4pRCQ{4N3RUlvof)pz?zV-`i}kJW362i*3&=BozVJSkU*LoBmFdK1P!_5JCWC<9}p-L9f-&u=N=;6 zQFyHEh;esFJ1o`sxT9IQz6N74?2u^bk?4ghbU3ZvcZl4n>5@tt#(DGX3Bdjy3*tK_ ze2LFl3_5QDs0(A;zu{lpNA!Es8k)zu(cI$9>jWdtPVP+5L&bkRR2c(nV!(En^)Tqrc~l96Fe&40Ozx95Pt zMJ^|nf(huePR$dhO4g=q+$IBSY;NNMmI_`>KRM;*e(!zy-fQsw6!quJN^B)h?6M)fAIdOf)~pjtXzrdW8jc?Y zU_x>HSx){pZgM%gI0j89G5o}u6g-Cl+%@*9n`uVWfVlpNM99G@79Uyx-<=87 zUR7*8)wB_xtC73AdfGJHJ1b8Pd2lX6MF9!f%Z1OlT905(gM{3&nnDGo15btF>eWrp z-fa9G0qQt|h}+V`6v!)|i*g3it>nc3AQf@oN*tMv_?(K-?67fMCg%C$T~+_xW!LvUbeyhFyL3Z`3qKy4DC zNu@92H5W~1Zax{7RqEw0F7y1nw30s@Z`*fvzXR=v9n<5h@CZc0ors5k!x4FL8LM=Z z(8MJhwtr^(vT%5)zmr!%<7xya?NWoopvyEl%7mmGAMy`uIcsUX5YT8b=2W=&X>42F zSEW14d2qS84o$?t90|vZqylPu0Pet)!4CRHDhbY1^s!fyoXdMZone**<1pkp{PeIP zCj8i>aoA+MTT4gS<3?dAaUH9Q*Wdf(DQ@?;l1Iu0>^qrD>CxOcWZ1bb_%r8idHWFf zy@on2(J7M!?y+)>~jvd&GmJ!SniN_ z+3ujb;;aNuK@!CGYTRbTZsR=T>o4Di25{bf2jjgVlAP=#ypaU^gq^{zcN8HX_w?{l zA-4uU8?Qw7VoBwTjv65N0Ek2gMJU6T`@-79er(KX=kJq(0R~4Q8TKOs6mLGV7wcw{ zHyo&1H6bsZ$D~&3MCge(8s)l4nrgFbx^vQ1`n~R5ePp;CCFhfF*8ZG+tGv0IKYoOs zibyr%epap4>5)r5qXm)-3`UqSMnq0(4Yybn2T%PR^j74Kyf0Xv0z784EL_&*%SV_+ zz13WQrC&x^XiyE{O^c1qtlabXsv-;Fbk-P_%x=$3W72WtIQ!f7&y_Bou)0nMc_(en z0VPnJ>8zqY1gz;d26ers$tc&IuY12nO&V+6PuI=^DO-&tP#in_xbqqlWV zaU}{qa@@?47CB6+K7c#Hey)tKp3Un8O3`J5=EtNNwM`;gd}G>TCP{7c90)ju~pBd&OL*}UHK{s;{fMbvC&mOOjFBy<&m{S`maUsMoe7N z@6mNgI4p3MoVVU~Sg!0X?O#!B6!FZV8w2;rgGHCu1ZlI=tg3WSaIi<#f>x@4;Wu^m zux@M35M`Xw6|-c7`GA7-kC0z%As89vWe#~^<~LNn$U<#F*|n1JajcGs(diJ-tZ+!? zObUHBsz&^T%_!MIDnS)Twf#B8Ugr@+GjaL+T>Zo!W^*HRSEEr}DCN7xiNh{9?_d9V zzW=eN|JTlb5(kidW}E(l)OcDVQi1;?mR?fQ{fpT=VizpOeD02uLIH)w2#A45BG6Dk zhaf%}cu@yaGZQi~Gjk(1TcFb@9wm@5n-vN$tomQ&84P+swd|Ee}W-{YD;knh4>RD|D%&`2ZI6U_%C1u#nfof!)-60lr1W7UD0&ynDnQ2BR zibnK(6GTV2$#*~fWY#pF#1Fi8J8m{IAUjVDQmI00pN&haJhDiWcmNc$f!AX~6|eBu zuy)gK#!nO)Sn%?bvpOjOo*9dPWwD3A+TUoWwV?!08lGgozSCNs*t6a|(s&$|EBYFV zis_CqQH8nfDJT1NIhx8xMaUGGOSi`?T43?ROzq$q$|XOP$M(q6oJNYJUzu=obC9b=k+Zt={B- z(Ps0t4<6rvq`cHFU+KLYvy@k5HBi5^{Wv1+s-zve!*1(O>%tb!z6y2jZ=D#dV%#oN zc)cGhEjd>HbO>CVuUg)a0LN7KnFI|9ikJrs*-pdZay`??+=O?be0WH!DP*J(n&~7# zDds?9=s}?~D;Wtv9!UBD*f}dz#amL%h}20;tXEgBZmX_Pvm3Bv{_n~BZNU?Uk6;CO z`TX+SlIn#Wk;T_1&%qnJXLOe;WAYWW4_Ii})~7r=ki`2Q{dI}Pnes!s*~4E2OzKLfV^e+5Sae$;V80jgCu?NNl$J|J6c z`>TzAE)`JGXvHFFAYBM?V|%*;j+tA+<>-{47>vKWUgkX!=u1~q{zwV^!arPJ)(XNaA; z`%RQ)?!rTd#Ai%LK>7$bVDue~K*og-mVQAy^azcX#{OF)3P6(WV&i!Y>+YLZ{t0aH zQ?Rw{uOo4T*1#TXLGC4U@6UQ`AnKS$GG%2FR0#hBX(Qo-sN#i`%E~XmLDxqrp|kar z%HeNeu`it>9)T@E+{t*815WXQb~Ut3{0kS*gLhK2p+};b{Hv$}F_RMM`2yW595nDz zr9XLzuOkBx1_7^Q9>4j%V~=7@U8tT^YnXTZo_|JXeuDZ-39IAskuZgo6gti%gx^qR7=BB*E!msXYl> z0IC-ceUTF!X81$`MGOhcfsxS-Umr~>^DEq6)%%Zv{a@AF@_0Wv`^;RT`DDgZ1UiwU zlmCUp;ebBJAi4lpvUs6AwD7Z*dLklI@XX!{n~I)_njB68gKo&Wm`d^kr@3)*T3iB# z?`V=`-6-Qarzs?X!epcGPuFj=halV zDRK+PA3|mNbj^em{Mau+Z2T*a*E*X&dFFUC^+XtU9D)JE>O}{IbB+dQu`6-h4I45d zSB##p0*XQHu1MGe-k%iLk89x76sQomp^`N z5g!!e=hSgO@x#RY2F`nvAchda{e1EYmHM@r(Us38@vy1yCr#A!C*+T45$o)jN~iTQ z8)0O^ZfYfccPTw@w{P384tBKq!OKeqUSSDrm&EczX*&hhF*7*HiT%xM{ zFL8DWir85~slGXhZULkSv1Th6*1wofiYK^-wVmfYCOzZCA8C^4=!_s7x3H^N1{-Kp z)RR`VTJ)=X*d>m*F2FE-UF%|`*Wi!jjh1+$T(a!c>OZ?33~G$k)3ZE|V6EM9I!Yt* z16cpv^GH##vq2R`^5bYfNc3c9R+m55X)cxb8Iz6-rz#kNqdyXtD<^!(w`GuRbo-9U zkXZrf>)o?J7ZZB{dsE)Kz@`h#3s_J0GWWFS5GRyinPOyym~Ms+M^dxI-qVX^slrdkPWh!hS-DE zSAGOniGUCBV~YWsOEm9IJh+rgNi`wg!n(@qnfc0Xb~7j>j*5Nd7EB;6iVM)NYeTFt znGyz}#8XvvaRlSUPL&TY^=vzKEBs)?YA#X|6^t!h>8E$O;IM{A!1j`c>(YYjDuU3T z2fb!2ZaascHphx=D}Od6p@1*mSMBcafo2=b_Oh~D#6J5I`*JV9!kW#0U~Mgk}Yk`_%6D1Kj=niZWj^vBsF;Y>{U7u^+JFb$C+g-C$%M z7Qs0OGBbtd$T&uv01B>15yd-go#?u_ap2wI@ z)Xi6;Foqm7Njgw%(^H(lxKA1Vj4I>Kj!|TO}K*E!2d=Pbb2vc#p&56|idha_Q-|4;`ch10qnJ45->K-@D z4Ac#1EJWSc(cf!NvbQ9?KILePtp2EcZ~kE#o?O(sVREmekas4K&J~}V)a)>VFgvB5 zdIPv9TWC~c+9egH-~}*%sqoVZGyIrewB9t;8TCa8-;(P6CU9({FZb#vN;c(ML)rPw z;NimCgdAtt60JEZOCEVkJ#0G6tz>ue3K(tZscvQn$J{^s2Mt2ZfLdvK(uV>2W4C5Cd%rx zkid|Z7wJfAH|FbD6j%-Q|J!URCRV;eXQbBeW^zyu+HKbXDZReI^U7?AIN|pepyrgZDgPefdh#ogi@Zp@J#*fh>4^jIim$F;F})) zmHIq)BC4~4=fjT$3+h=IwF#+j_)TDv12f>MY1ZF{UDR`~@$ zlvp3iy(f$zZ96R*zm7{Ar%aShr^d_;#yevllcg=-CyvoZ0m9^`zI7)MUfXE=L>pyV z4X6QXy_aNMa?Vq?EV)`|MoyB^1$3f-jmM+N(!M0DZ22u?9Qh4U|yq_>PSLa9yom&Q~*mzgp>S@Jq z&hh3^(8CapKSL8vaK*u~a2VsK8&QV=43t&JJk30#EhEanUlH)-kE&BPp3M;?6Z6VIMFb=DULA?J|wejz#-s` z4t8#X;GTibJCbEKD&rPU^f=$SR&a6V1KP7p;(e?8Csiie`myF#ySY5eMj!m^@pzph zzx0zEfF0jvdZzPkBTXH@o@(&iTvA%AHnwam%Pehijg$a{TSf)6l+vX{8iI9+Ma(2h zlvljZj=EgXro%UpDOxxnQY5;CT9?Zg+blTJ&GhN&WwUGH2VBDHqj&&G42sJkozEnD zAdi_%fytlzqu9spZ2SmO`s{4{Y|vkgCxef^0DC8hH>ZXjz^xG$^^vQeS!J(`AN^f+9}cEbpt>(xROMRgfLF*XLJzHra*a-%3$S$E;}_ z&k*Z*ScgQCe)sO=OqR%J$&V}px4hVJZlKdkdC{)iwA5=wH3guc;sw(#14@EL2NaO%;vO+ld@IN=2 zPaTTFb*lL}dG5U}N;vFiaI^W{O!AzOJ=`;B+y5+hk>A=3VBoaQ+xwrqu`P>f#1v9_ zL-bek}I&Fyg3GucWEmm6g!Yy5jA9RO5mHK0B~S zdT|nk`8iIiv#5BIGjp>j9iL(LI2H~gPl1iB`dh)9(}&e!Q$n%@`A(|na`-zcpCOsk zVxdArr9#A|K1LT*1h&5vp#s?5LA20g-c&o^#TTnL83L`Vp|ItzGYNz_rPmFZe0{ij zJ294}T-keJh@5m1O4eb@o1m&dKs|R{NGvs~CR0GC{>HaKsPd8%kPMM9YDCy2Z4nuf zE^w``Xrd{&AsE<6jXII62XKpQxB|qum3|@P8+mQ?xs0A;OLMf|Ak&~L#Ox98*zmKE z{KfC#v-1FS4>;Yvg0tZd#rgyP8U9S6G8<6#Wo&)&!O8=*c#VF`6MCt9K;zBpNPUxG z&8w77xJvM|S`}KY5&abI(D@3@Rv*JH#bbnW989O)!zSWO*ri7gLY&M=M}JG67T3Q1 z+G~V4A^&ELxdq$b3 z7J@i;UW2;0)=Lar7a0VvN!*iL@Ju}&!9AYE;#+95qC6w}GCwu}z>zh}PZQ5}6yFxX zwbOdhT=kYC($}PmYnt6;M=Z&1la;Day>3JB2siU+eOqPo4;amG?vGx8MD;Ts@-3#= zumL{~J6D5E*)pDUj6{SU@kh|GaU|eWaa{Nb;`W$~|A!?OgqimVi}lQ&e&muFLhFtF z<}V*#?bBrHC$}Hf03GBSonON%fSh1Rq@pZ?T`D!n81y{{)wP~Hx;ILQtL^e~--yFK zLe-(3qUbkF@1fpJ92P>^xu~M*BVoy@FKY40mFDP?_VNBcD6lYR-{N|TPe^?!->GeD zHPlp+t3K=6k6#>-i;qj&-7=?uYsFnLiMWy%%kP#Y=?C%6^o8z=)g@j{WH3{oEtMs=mF2I}z5D~z?lp;$E^t5&$z364Cbo31n!I;qzA4J4SgOzsnD8=?4p%2N?0vE$wZDPvpZj3|8k+J*C+I09Zki{k_3F0Hmx@d zeZ{zwhGo>I(xlSXG{zD<(61>DEP;m*ZJ_}h#nkTg*nk8)`R;NbUcD7$HjE)HBVZrN zz)!rJj+cKI;kH-8tXY`tXg*V&Y^(NENtR!{@FDXpCuHrKjB9Z#s~qMJO&c?lQY&W6 zY0hNN*T2t@Oc=A?adwEKwaQTpVs?Rr{2pehy1H*fbyClkSi3=kPlMVNyr&K6$8Fid zmmlFH04M?MU$)p+*6cp7Bbuk-rcT9g@HmT4lm%lDFdaamt!aCkE_kfVz!>m?p^Rw= z>$Dr|WRNQUF#-JcCv>J+Nxk+Krh0$Z%f}Vo9B4p*BF6#Gt9$o>j5qeyyKo za_KR^U3ScVF=^6fDT*N;-T&yh?{X*7ovL%}`KAU4)t-f5k^A-ZC^KBg775WY;UQzK zjNHYbx}s(H?D?pt-jQ|Kx4b9ipr7XfSOQWQInrR~$S;)>-sC6& zsjz0oS{uXYBGTIU=RfxUp%?i_1wp^ye`)w%>}P*n{ItI9*V@mV1XvIt@jC%Nuuhl^ z7l>fP3kCeJ#2-WWJLIXj`FGN1sNe{kLnJ2y`fq|j0Eu?EQ2^}!hM2{wrwWToXdhQ{ zVbjs#kQpWMHkUFi%C~3DEjn+4$lLx9 zoK#A3N0I|zj{b2Ji@W->p_#lh;DT5ALAplUZ`eg-6_F;EUZc>`Mu@@LS|Z=XZrt2d z98uJ`cqmVQJm7K8y@iDH#;j30j;24>!1XE-RwP@1S%e1bq z%M=V|6?+^Ty|EjU5*?YJBu7?hmXSD&0)~-^$OwrMd#ycUPwS(D1vAa9Q`-^0 zwjjg^DbjP;E|gsNn1d86g=u7rO9DG|LntDu5D#7Q^LuDm`6DNnZ-^*G3cDglt&*EP zbORvmV`3MizzVP5-xv-Px_giC=T~o0W1Z3Z-5FnZgwIo?pIWO4{FK9yg=%$OS#8Gf zaEO+PSFF(k054Kl)5&)t>a~I_5U1+e$HU^4&us{b(Y<|Pkfb!lYL%vIwvJckzjMkkAO7egy>ywDFJtj z8{9)8jbeBag#R|_Cus}>jPXMU-a-s?`mzmqI{^Ww{HBP%m?u%>vioenagCo5Kv~)% zc%|~;x*uA;os?3tOZ&h&C{yGV`4UBeoBhVFj?mxOgIVanuYG8cdX&vdj)T!^9+50v zn^n*if||3}zHA~zmSm7uX~Ohfuw$bOyYcS0{AW+%zjgQjQG9DnGYpQeU%r@918*5& z{tJ>dF#2Tht2QUY0%%$psxG^I#F`gzPM}7`_XQyhUhG-*?;@EWW^B_N~mO?R` zJ(Eaa*^uuN`Q-63xcFCM8Ov^c9*g?OGhXgyIGn8Z9*s%$Ri-L4JQq1?DLOExJmr%_ zTImvs2{a;5;-vH*S8ZtN%zsWCo7RxPFl;|e8 z>9^}I**FJje>R)I{pnFik;zt$)D~E8Ps(F02H~t(=ekG;i}<`R(YZ^=RKP4OOvhvj zj~f{2+b?Ce5RDjF?m#Hyj=-U@hO%r>G@czO}s};JZ8IV83S}QSaW(?vyX6 z1)N6WOAo+slV;@rUpob@__Nyx>_a!tcSL z=B1I)DYifpE{-y>Ojc^Di`Z8%504i~TV?+0QgpzfAsOC+DJwgrfy&P$t0P#Yw>sns zyZWBXR2KRK#|;IAjl#_lei%|brMq@)em|XVh$-vFwIto? z?pUF>hiD-kEbi!9RE$DhR~aZ|Epe2G*M?O~l&UQ31Ph5o<#wZ`wQQID!g^sdm-59T znIvlp9@dFPrYB2h)JDGQ1$;dCg-~Ubu0Mc3RANWysv%R(9nr36U3QkNr*v6I=3xUq zb0;vkBSJb%Xy2rpk4p=H$`7EjuEIO?%*L0lXIUELkUYu;=_y0Pq#OgPmJ;4v^VktM zC6aB+*x^+7<}>IE(j&;+ek{NSlZh$k$pLzx%%*_b_*MM>vB}?@`i*x_w9lx@fh(2FT*}V#M*IeoXl6YRS|PO{9%!(wIBDQ`i4?R zd)L|L-BtaezxG{sU)9~$Z>=?Ls=g7=j{@v8$xU`=;OIZb=8A!>*h&y%Gh`w{<-=J%t+%tP+^oa||wC;W6)yDP6Y5)T^XBBVbc$3OPw%t-@#TCEz4$ z244m@pERqD2{D(h6Bo$ z#zGxw$e+7e#piT%>EW`k70V{EHO8NNs<>Y~vk44(SIELCoHs}U+^mvxAl%@*%;KB| z@}P$-mhj~57mD?GlBplhojWfVid54W)@sK<^seKBcnC+(GF>m`NxNl?YfcxA;y25s zd#7o^dOeiUy}}qI56dVg_;Snc#0cE#h=yXu z-2_Gm@He_{q*-WAeAeW~)@(1+J$-p`RNe;zX7MkW3A9pc63<4CwSdIfO{hdzZ!8M1 zuwWXAR^60)NXk|7k#am0tXrY33K>Y*h+jW!A~~^b5Rk~6WbhJ&`s+}LBH@dI`Y9Ih z2qVNq&V4Id-3d?61Y0^=tE>MJ7tj2u7q-fEn)kf};KscMkedUIRJcc>9o-t zh*mO`cU3I;TK=sr>jLdP9&18%o#_W%7XjO{-wC^xI%owN4Kmb^vtNHD5{DqUFO2z} z>_=NRRSKxE8>2P^Ncpgd&N-kD&c!#;kSxmE7HC|xId6{mp|UzVV9HFY_2}Q8TwS*$ zsu;1^;RuyV&YoUGVTOcktZuG4y4@5Je@nkqahlXk-IH}fuL7+D4#)D@T>6~1D&SS4 z7s^ncpdv2VGe{?WUCg49;{MDqkwTb2QU6NGM>6v;Q&}I3lHF;BQ^{@K3^vt6L=mLy z#l&_Oa2+BYQ`6=n{VH`-*(JhZ9ssLAoyy7W925xWnDq8ktSRmD!OTG8Xv!^VTiqPwI9uvAF67F4QgtyxZ63hFnr_Bvh zPS^dbYn&Zxw~P>5w$e$=89Pob$VhF>dCBl$N1q~wY=x+7@NuQk=|p-bv* zFNzV8tOFoJ&Gdn0C7qi>fnjFI2>!JY$YZ9dCSXFvls&78I6Ns^^^fTH^@4- z=d)9cuuJUVndbmoP=T%?>_Un}GnaTXwg>5-7bBM(lZZlTWb>V|dT1SeQt6PcyWE2= zsQB)sL-x=cO@mO4n+NXFUChOk%Gj_RP=kI9%UGb3r!R!+Us!j`o_FfgZ0CrXe@<#S zl6&B(&$?(@JfRKasMDJAMj3{v6Bk?T0#E3XcaEpj;!YudUx8vMRG7N`Qvxd(i+Uo` zUtS%TI9wF<72DNRWqK6U6W@2YE+QOkQTAkf#s`Ue4uCby~~k~xPq%sCHcFz#fb^Xe?h_Qk$97i>mkkO#m&YM9gLQSM^HvYPCOh( zJ)ESS4W^y;rk(ZPIR8UJPU9o{6QWE#K)nWxWStN2$O?ak?)Od3jrl2IDHzBePTIe1 z_9-z{pH@d$>X_Lr;tOT*mqDf*wRTruWwbPVr_Zk~?{rlLDQ5_5v4C(!wdbi%Ja)=R8?D;C1kSxc)j>wfXK{qyqP?_;HJjC%Jx%?Q3O2c~3LK~2;T`O0 zZ6`9s^L$X>WRRv?d=0e>!_2-CQzmxjB9Rzd0jO$Qcuq9~5i_9k-_XTZX*KO0`Iv0k zX&NPFc54k1vQLSCF>yHWf3tst4 zG83=GhwOBIvkAU=?m&YR#;JkGN0W$MDbpI=Z~vuf~|j^a^Qx7e9d_s}cMAX$VSiK%g@ z?==x&mDGH?@)9^cM1vz!lR!83cM**<&e~y&k(E{RP^}5R zcSEm7rbmKRGi}o=memjtE90xCDCJ)R`V0$eYs8vrZrx8Q(d|fg^Sop;fuRV+g`-qD zBuBXf##U2p4S=2rjN=;@tSazvD)pFla?C)S+NmMck6gZ3oPu25OD5A<2Y1N_RdFr1 zNOy?fK`&Ex98Gb7YgdutS*HThz|f2M1Mq1)se)*E&X-n5nJ0nqu0JoC;5J)9R-4N0$Kj+fvH;K${9@^+h3t`EY;MCHbKS`BLUhwDO6a&t?e@Telp+jXXb7+B0*+vt zt(sK8NdV#+&7hE57(+WFPC2(7erUO}BXFnI z^pSPP+B^bGO8vls0F*9sL+C+B{eEUJn09ZB5rphMJ20YsQUVWnx`Ud4xO(GVG~bGY z+yHgok^>hogI(DUv+f+q(D{iXIep>{JI9Pf1?5K>gGfUMzecFM+#FbD-OG^lS3*7d zO8HtsCbkayg0GXDgC}7?* z>ftBb3G2ozqmA?xU;cyO@Q=Y$j@W4bxCJzopY_0`bq?dE*;Ar5iFf8`sus1$#J>oq z!5?r1mzvRaJC2DTvC@xF=9al;KpIJgmQnZpG18E4qfoExA#MyiLaXS?t|{Um7|S-H zd62qZrb^YOq^5;Y^GMec^)}XETCNC-QD#dp_ z!1})h;s4)JX(gfpW%zHLSUi)U5ca>w;>#ieNwEK#a+VQPfczh+9MBSxRqp%oPZln% z)(Vt7P4PGM&ooCBj{=6j+ z4t8>S(>-i@+Sc46iflYc*^70~GP*fpE*%?99zEZVOvZR1>MLp3#iF!xgMoznmFa1l z`ziOv_vt$zK3Ger0#{hX_uSQQt2ywtx2kTnBsBBJBU4^4gQgyAp#Z*=@3Ysmqq^eYH?kZnB^r*lg?iyo z#4^#X!)<1;UtNstrxqO|TV|qVy&- zj|ZM#_fb4Yy&9cPDojV8P)fRJ4&T3&^NF6%A=H=49(U@AO+YH?5v z>rIJnw+7Xe-dF57cF!f0F?Tjsth_SpGXect4j$Z)=A>exH9+PnyS#)ZH@w9i3ZJ0) zeQwdQALpzCAHmp0AT|3Sk+#Kmu@o%?V{)KDtD=z`XZvmWiPrGr zIyEV=!c6?TO91X~-zOd%zr=^#?*AL)9v<;!T%3hQ?)zNph@$5wyXuUTKL9)M1OR@1X>>cx^rmoX z-6z$*6Lf&E6MhV?Kj;tYI`^~9;J@QUAEPV#J8{CQ`L-R<@0 zO*mGdU*)QR9&`l11BODl6$bA$r9BX4kN%BL9_4mZW~#Q}M=jS zj8B<$|Lx}@jrrKM8L-e){VD$!f0oZVMPbW}(PKTB|4|zylQLb#?Ao2$MkF+~R z7~-@(04OsMQe-*UF8^VupnyIGm)KQQsu|SSikO1UN*S4IecgvufqvZD?Qs}VReZF#%@W?yMyX4;@dVs|0Xh$OmG%1Udo?hi7$cqe`Y>*R^; zjAJ`8d!c0qMA%#C5k{LCdI%MHw0Mt+27W?<+;pQH<&~!o^$bSON&F=p$<$g6{V@(^ z!Ip4LH7mGEQ6JN{K+M0lp>yhk-DM*`g`~LN{TuorKkIS#>*y8gf zg1|TAnz?Ty&YrcMNWO}fVzSyqr(Yt06=(As^xWXK8}&*2im;Zs>$Jth?NF;|>d|(h zxtz;w@8}dpD`(O1p@~22PWgC$mOhj+$Ly5|C9A!rX)`TjJY5t$H}_l#rv z&hD*`x)Q``1)m6C7g5QkUoLBe?En^;X!&6M%Y$jY4DX(RHtv)wJ1p`HLIse uYk7Czy7f*IZB3Vkz%(@I@fiAD_>iCmQ_ zTBST}An`Vj58X4iNE-bVdIpBVR%@L~`Jl4#7u#rq%)dF(Fs{o2FZ{7xd;5jw@F>jN zIe3~w6K8G)GDS<@;wU~}b;zsKYv7hIUr_C_&7rd#!w7qi#=(VdQ{oOet!`dF=ieCT zcIgA1qPJyNf6krb{f!qytE~)$xLY6HwoP)s8@JoH=WBR>L866cuz8obD28JsSSTXr z+iqW#RW9*S|91GIgS+TSUedUk z@go;pp#DK4gZ04)izPi!n5UO2jj{MX?ZwNgn|Sggs>*M8=xf|yY!~ar+i16vT}r{8IlN`IMG7emYZhb29%LLU$WE+?x!ckI~~#{`&*KFvIi0Isc*Kb>qSENc6Me z3!MVU=m(TKS$U3Q(ETJ!6P!2za=$PR+$!wd*E+vs>3%Xe^yDHY0J}+ObJ5(s!kV^S z?{peQH*oVTGQ0Uv&uFZn$L%`z?AS#RNF0bSpCkfFD7smYnMBO|9k^_R*~R4+Xk*2r zg;ufPwHj@@awO8NwjV=7^`;P-r<@tBxAm0)`pPR^#T8Aq7V2hxU<;8n26aq^Vn0XV zc)J2LdQoU>VOntMf#0di{zlqioN5mE*Bqeq63wkPq^jQCwfws=u3_*cIcVPPWXzqs z`}bI>p*p@juTrB}mOE`Q$tO!39BeE0b~QC+wTxRDbGiogo%IDJRiFe28#WUeeC4UM z=tUJVQj&ETO~=s^Hkww0?j=g{L7fT(Ko1ABXe$f6HAT@zf9d;)CReUe|1fq=eg^Fs zr6^&Wx#O2>VWI11D;~#UvhLtT5)tl#`Ut(GU)}$Aa~tY=hkw;1&f}{Trd|#~Q3V=I zDj|hlj>2cS1&n5J_qsdLyQ4mDyl5KF!9C zV+BnEk798D1<};rbE;B$re-Lrw=Fe-?ag!PC(r%cI$`%W|5es(?(nWBQ|m!q6kUjw zb97_6K9IBfzG{e_!{a?E;dft0cXjpP0I}Lt15kxrCA@MSI;{H&k^_8&hZ_G9u^IdJ zUjrZ@4M`a3lbFY;%Nn^5?(L5`ZXoo&fZS4Y2{q*c!p){aA~LSeKHy zILQRI4W~h8@x;WU^9ab!A!=s^Ivx{8L;h_HxS9=#flAY+QC7eg1=WXJrc4~3*+9k( zf#NZm>MjhozmBAmwD7h%1KI<16-tvJqR=ipv!@DU-rGEg!`DToLV)6QaU^Zh13+b+Zn!M zaKVI^B@WrK=?%2nm|+Y@#`$soZsN{Ni!foxO^ju_jxI{hpG?Bga$9NCr)%3|VD%c# zFktMMHx)GN1|w$J+Jty>YYF2|)iQt3HV|ZK00onF3{Ouy!7?!3!DHA%^$;96vA z)b*c~vDDBkI60dJ0*guAnccsaeo-*zcb1uO1t(A$%3O^oYzmILd`mc)ybQ7SaSMst zO%lCA9<+W37S*Tl%?~sN%fPpcUjYj74IkklNn5ljboePVYIj!j!fHMA#$r0}Qox-L z0VhCOOcC1+^3VNL2jQwnnqdZ3sL6Zy$A}5lGK?-jYug;WG*PST+k15b%2ORI8&B|2 zPn;Z=t=Qy0na%{?P+yebqp93I^$T!ix91q4PFv!wtlzxcEgF8ZVI*33Xz}O7*c9XP zvXj^{fcLq5j*oaF*OXU3$)c7kSRh0k0b`Uber&_`i79-6#;Wm?qgh_-9Ia12ZxUAn*#L%{3~IWHUJJNC+xu3&fJm9QX{%U4IFw z=JHe62B}g5AFaf^Fna!s);d6M7KDfolJN*SD(fqiD(KE9-uTPE5= ztY(<%Njoe|odK(_n_nr?tfE<|mhkJHQ|!PAtsb00O`|Y%b_usR(Q1uLRNTu08Y!(`P$_GcP&@il zfnk2ptfNDho%DWg2?%mzHl8O1OX|Z@R*AV(K`Uyvq?T2)t{zt7dk(8dpyIFR1{(;n zCs@|u$^e0k&CfTAM*dRz60t4QyJPwEdK5ezJ`;~&G-;$>RT+Mc`6|igrlvV!8gXzQ zTvLOttY2JNxx_9d9x7A{FEP&72pU z7<{-uxFr!1ux_Y?=p(-78e^T(Y?xa4{%uh}&reHaLell0vRiKrAu=~VRfCVz){L4a zJ_neyf`!NGgSpY=;L0&8K=UvdfmE`IR99CE8wT=Y4?}=-zU3#jb=bV?E1aku{l!X+ zb~}}=_;erwtwuE!pdO$}gjOZUkQR1y7w$MO+ReicjEM41es$xK|B%xyt5hsgEP0eB zZHFzni{MF15Y(H+R&aZcV4<0Fq9m5%HX*<$X_+vB$|!FP%!dj($R-f9$6aWpRHzI+ z894Cd?}^XNo_&J2HJ54;rDo8Wh3ddFa+MR-6wn1yt!hTaxX;(&H zSzVzBoN}TYN+bjOW8%QBxwy2n9OFco@Lq%et0_HCd7f{geyv>k8cJKUv#us%x~K&K zCmd?%xDLm;B!Nn%fKD%mC5Z6Pz~Ai8#EN`s%X0@>QeV9Gw>*I0EyNKpq+C1qs4zwQ zKxy`wJAdK5sJ+4q0~UJ1Im6s1F&)Qt`nR7c@Y0*Yy@_kiL94LFny|{6WYEELFxCG` zQmIa0O)TZ`zOajg9BEkY zkTt=!V~i8L(FDj%sbM-AC|6;LGKCQhtTqL9%8Fvu|%j&6*yg* zt|F=EvgEUE8lIa}oEMhO>bg0K?^RWmJ5W_q<6Te+lXlVN&su!GmO&WS;N(215xLy) zze)Pojq0insv%2Xt>02jLl36|Q9gcYn3r%iLMf({wa}v53d@uDFE%$flytRMwRCi# zYv`vK*C0!z=j9WSTTSTnKTG2uo~-IgpydgJhvZW!Xg`l?Gt4l#B%W-ic9JYum}oi))LT*&!rty!K_Xkti$FVy_rW z;{+Az1wwBosxsZb9(tGo*GNw@`( zdRILq$EM=TxE^+uv?)Xt>s31$*3F%!;IftLIf+(%TjMjxu&SC$gSyn!1(IXJV(yvb zP@R7saQLXb12>(4x#2BYxdo+jU)qhmNHeA7L|59*za!t&9&+<7m=9b$0WrLA-J1)4 z6fs`i5A6oNIcv=*Q}$DEzXu586YWgrhEo~x(+DxL*vc{3OK)MvgwAb z!9%YMwMQbOPA5)zWheS3oE?+k^kq)HzJs3A%}OEC=aBr<0*>z=Kx^8Q#lJ+ptZV8c zdPEra^#dtwwl#0r@@yUEf5!B<6K++_tsLFO(Bt~u%a4n2`oSgHreA{ceo5@KaLmAw zz#)*vc!9A2GlcDA>aXCzCGJA=8o2gY-IX`@!2C z?Z#>e6@LK*&8>A1_CQK(N0R1F7{&W*C64_S#@=9|FS1V);(0SD{tYL+G2RBmULhJE z%9%BzVe^CSf;&^scJLmbZze`yV`i2QZW<($5&8t$&~j)_PwDbUM=W+n<_5D69E9GL z?S8))P*YZ9h2%6<6=Nxox!nL)(gRJt{~x~DeAep4#%#mA(7RpwxEGB@361-A1o`@u z)#4wHj^ry>Zr(BkAAx~0iuE+(#?EhPiK`d7opQ>7Q&Zf?oB*NZn_1Gh}*T7mW}5$%uLgx8JAp zGuCwgk@h1A(vkG3q)pc+EpqoW+yOefZ1Fy+oT20ZCzP5ql6@#7|$|CNWQtQKLN|Y@RZ=D4&6#mmy z*zl}`S#XE$pn^@xS(Y?vfGc%McmD%4T1N>P6VH-YRU8}#odz0@fKKCsSdgz3?)FUs z1z}mf&)5ZfdrrH?gMPPFUG0?8VN1uX|CD%eAxcncf1#WJ17HKff{AS6Sc4_)gWYfP zak|o+XZ8QQP4XWB9`@fEO9Ule4CH_Xf?4KG>7Ns9q?akCh*h-D=cZi1SQKPJt~S`qSL7wx3!9XYUVh=eZ<96I3=s zu+_vj@mUC+0rmPgidz!P6e8fes|b`xQvjISq;N-g=c%y(yaz^p_on=mZ!qA6%I%i9 ztpbfA*Y0;F6#olvH%K83r1gnhshg9U&L4IBm_ongPZaSts?7r(ign37{FyB2K4thW z+wl`B{G*9qcBp#&u(9*~Eta@OH4tfMY{jfe_zub~!-!lBE^I(qxg20BMO0vpW2Je| z0B61;mp7lsFAiq)y!@yv^OL8!;{`Q95*$B|v|js+vQfP}on6$bv37lTpWRLA9Y}>z zQ>zeU3kyr39as|q!Hd0*u%P;QGC^sX(lxK*9Ls2aMO?*Dezy>gy(H06_Mfe7$!302 z58l7iHH3g)>>hbV+FC#tkNT-1J<6sFs4`(o6AhDoG^e3$P^AcmP~#nug%^#?shbiA zUp(7A_m23R-vqvt@7sEZM^7Maoo5uEKemRRX8GTRy@M6!OnePCzw`oH^`!pck9q>Q z!2~8DCF|qda=~}MUHAFc*z|r4sd6o#?;P-97z+!)T5Fd)mn;CKb0HG|!wt#!CnSpU z2WLzXy_p)fIx!@_al`=B_O+^+nKqG!ftIjd;5OlZPfi}^a+8Yw&2Ll6)gJP#48uc=~#E346^ui|= zVETPDA@R&Dnoa=o{4yU`4khI8KR3(Frmst7Ur$w(L!lr#!9GZK?7tW9J$|N zEO14KO?32lzAb_Ps+S`C1{(99``>U4spE0^9sXWw9oe%AIN|KxyOvdZYAJzu9$Lb z0k#z3>2Iix7XcCcGcL0Lh1Q>tf+pv3L^Gmv6pblCUQ}jfX;{PzviMiR*Y8$s@;bVv zEWiY-1{R==lkYN`@)SU7y45mrSe8xymE*{!EBd9th3x|WOo#0;yAa+ig{2EWaq6h6 z%rv9}yy}9aA5k?h<7M=p?>WfbpU&5_NPV6~#m^*HYMFQiK6oYKm>=%y^F@a2pbA*3 z^1^ELuyDrKB(Mo$X6wi_vAOpXh(cI%KG4g@S|6vQ?hg9L zA5wr@W5-4}#iaY^vAG2+a0M|do2TEE#U%W|UPoV$;3l6)W(TJ6hMNRc^cB@9tJ}30 zJ3Y0!zt3Ju=LEcrax>P}qJ@IO*eI)d4XMh-oOh@{^YJLWr7@?P0%SD}^39uM{eyT) zP*jsWC%-aFVS!zH<#En!{;NiyOAhK=mll*lW z-D=>Es+GHo^Ou>ql27>l*I8Tw$OpXT7e2%&<8F|F@aLQCX!=zq zto2KX^UlTJBVnd~_{$KhcZ@aJn6r1P5d(OS znlemic7Pyr5HpU0wB!+3p;eb;A3PKVVbcb{tO^($@nol41<7rY+iD14-Df{VE7d`D zsj|#?4ehQvX|8h(**d)I5Z17p|8=}p(6~HeF;wMEe&&!bWDXu(?4aB{jAyvYe9xfm z9J|SMneNvstQxO7UJFv#uNx&1iw6|ToL$6wJEPdNAHMo=-8#DEzst4-Fhl5hLqGGH z51xGjpv|$+)qj6Zp*elDGlRv6jFb&ZDoWj zh>xmB$&p` zgMK~#v~5B=euo`s!e}o^`j7_{-cq!_$!A7-b|5GvsQMpZBb4sMnor8NGsfFObeW5I zbb~z*l4t6qX0^A+T{jT4K?mLBEOc01Q)i~xqTUQDFuUqa{xpqhcTu9=u}hFDS~I84 z$4ro?5A{r)m1~CAd5o8&F9rIh$Zn_!v;eT5jNp`X!>_6D(#c0;k5jXhJo)Ti>q^QL z#w3($CVk=7_nF|lercjs_UHA(Uj6IwLB-FJr1ZAyo{@t zysz16zeEzWe}@DoOUUlx(@Y{0|B%O!cN7KNE^4P@J(-ja!?leem<*80JbNQbwmoL^ zJEZ2Brv>I);?BtuZ2==#uH?tAH*ooZ`b|gP0<~`1- zFN%2(Gi}F|`cRwhQ|CKy5FFbbYnw4tS@Hr@n$$Cm?4kS;4+ZI15o%79U%>%!d!OY~ z_KWQa8G4J0{K&6RKqw*ggBmJmcwpqF&rD}Qb}dLs2*&nnuM)rG8q`=I&5tk z0XTv8O70TH%@Mx{Pa^O zTdfJfDw(y;t7+q~@7x4wLPf?;#9PM(mB5Fr9kzG#vBKTiVT-vEE8)}FO_8QA4$PV} zeR7EinTpI7;#^v>Q{Vcg2vz1{`1UZfR0tU`h$10xz@0^5F6_2Z;~<{IvNnruUh^4m zm_1*Bd&Jg3Diuq|?l_Y)Ml8`wA4;}vfVYB=;2Wr-j50|EBzLb3P3HnO3_-|F3C}C* zz5}iJHWsy_cGS5kLg+QcV=4dGSc~X}%SbMYji6L$@<9#%+8g8E1+_81Q93C{Y|NRfOe0rG=ch9YK{?b+ zZC>Z0{D(rOmcPFabIKOnhdOHYr*jk9L-L+k6}$j&Tf~v&e>p;IC2~U|V`vE1+oGxQ zZj2^}xfs^cl`5s#Zr^ zE0{j&2Gu#RDRwGyfF9$DEaOUq6!;zIYH{1{y*{UD@HqjULtolenEr;(UZdtrn?t&b z9;OXzyuW0S+ZAaE1ZUg5{OC%X|GjTI)@%n#v>px{dYMbz{&HtI+Z7IJ1}HJiJ%y+ z0ay5YXAYB&SEeW{XRLxUUr;Is2NhW$Y^tIQ_hLU4DOj8N8u0Tia5ek>3RJsGD=fT6 zuSTSboWlFFt+E4C+3DqV8v0im| zMSYW>sjM@MUHwCDCj<>)afh#Q$e5xn{p2*P@8KF)tr&Pm>g-VmMrm~o)47L-GO*h8 zm2g93p4Y(xNf0cFtb7@|7um=AEaH zkM6O`H$>w-XaBv!O-W&rgiDH+#z#G2ug;>f+yZsVU0P#`0NilOtk=qe*+lq;9U`Oy z!Tz)1|L>lK`QID9sMuzP`g_Aue;bCPuo9&G6$FL;54$Q1ZmSMROiTU^t;}7RTM7tR zd^NNA1GbW8Buk|kxh!=t=3GhZ*FMsWb?H{DbRbPtC>5eZLNt+!WrZ{97&5ET-M3(q zMwk!X3cq(3k*YR+1vHQj{g<|v4=o}+x2ARJeL%1E!LL$lLKJ42>ibK5GC?=pdyfuQ zs0xQoqB49QWm({*$ck}k=NJ1n$(DnE=+gVI=zh78baGQEfg<>27>}R=-Za|+Xl~s9 zS;PNcU2AUvH0pl?yj4k{Y5zSPMI@j3FGFCfQOv;DZ_`Q^u(Um7I?^;m5De0^>pgDb zR`DX}G7xZ-|K!K51L;5@|0`E^Cxnh0^#5eWihoGo^EY#QU)N{t+2|~L5sMn7J<9D&Q zywn1mTI}ZxDB8aw)SqC2D@kAFHoZ)LWsx-Y)N@`K?fi@JK0DESfx|ZB&qVJMC;BVL zt&rPP;%M6syY$&Ce_Im$Mo5wO8qkxa^4nb1)M)IPr6eL{i^ceOG_AwW`o&dB-X>hl zLB>W!?8c?lXV5K~=*rle8!*Um@`KpxW z(Kc&~(~enh^-4%u0CAmD`z|BJsKQMJw?q8|0LYnLg417<8;)WMb)X2l2OlfPT#gx(7%I9KlG%h7w%1~=B zbx%z85v4+v0otOrop<88H`v4>Dqn_d<%c)Zb8}b@L4pPuO{=*}1_k_`7NRTKnJ%Jb z#w`H-wDK?f{=2QZ1oXg1Ml@5kKUiUQ$P9FRX@FE>@*d3)piulvC4wz}ve_me6&R>mG z<>Y&YQ`kj`6~JV<>ULYND=k=9(gFPj-@zv2JsLlHEw<;S(c{+2_v?#tE2Xc%B;r?JKl{Y?g_j*yVeMSE6$>X zl*plxcnS@D(P<9T>fa9A9wNXzB--xuioWMI44n1pUoa2@SE`{K$wWWAQnTpAY&+nF zfi6_u7F!A@@=)e9UasPoO#>k~iK1!4AxJSdHU>VqaIRCXjvDuNNHIFmB9+ZX zC9Vs`_pb(oH93!uzSNox@e(Gfg>Q6O<51NJN~SdZOzRL^td^W_c9Yt(7i#@$GaKYR zWCY({$L$RVx?@S7$|{NtnS$Ik*yaeaz&P`AE)U3UD~nxypE^49HN5VAnGj)rhOKAd zsCi4>9CczS8{G1=AoAPQ{Ze5gbjv9Bwgp5eThBo1daKj+L&~#=g{&qSvRv@vUDw%R zMN2-(R3cpVjDStFx1+JsB&<$A+fpvrhnBO{O)i2S{fqf&o>5JAv01F_d5+5nu!LJ^ zMraX-3&$e}OqTCs&Xms5EfgJ~c0y9rOj88|XF!UZ%~-|aMBttoZnFBvo3e$ zQ{$Mh}#Cr}O7nQ6`5W{7Y_Z_JLG{W@pJpPBUZnj71NE7+Hq2~Az}v50B#5VPgp07^ z%*-Ql@|}j-#3k)9K|0%3kC3zv@RCySqPT$qbNGh9jEEs}vC;94Uv8QL!TX+fc*|nRMQei?4IHJk*%kEUwR+RyR?xQ8M(}1e!lsC0XuPYkg`K) za#(%7^2<@Dkq@l4DNp1HLlwVdA=i6wGlXg@b83${iBe4n&n|wtIhkpxt>^RIjQ{L6B3bzi*XE?Fd z`f`xH>BePqH#X-yQr|W%0rEK|YjL5)sao4%D;p>(UHpXkZKv`+e?TBCYe0F4mj|l8 zYllh>L_H=C2i+|#@A1D$F&Rf!IWC5doM#pXzB@FBs0F(l2Iz6xb^5#;e%gn0^@_MD zHs3Qhzez>s98+P!FpoCkh*&q>9z!JL!ou2Pm)da)J*LWlEX8A;17>WsrkRE)UguY_ z^jV|qnpiqRy>l`jIq`!%879}3ZHXFm6O4FIbPOCH3#jtB)p*mL(FVA<1Lkg3cvC0SXs5^fjyhNqi^u{Ak%!E!@le6M|&^JRZ^&KgNUVY-0*TpRCGi; zLcfk3xWl4D{OP(k?0Jp-EAt7;02DxIrsUsKzZR`rXFG$51#H35aylL)rA%cD*{3@a z+MF*$hZsLErf#b#Gh6%1SRc?XNJ(b>vaU8)NwDqtaMp#uRp%D zb}h*T(y6#=@Le$MYrNxKC4T6*VVO`?8@rlJ*t^*XO6pkBizw8JG)HGT3Z`CzEbiSc z2L!VS**N}n0C27`XJT8X;WMK)4hLFcB*^bZZ>w6STKTMMgW_cYzkLiJz70*52pZ^# zJiQhh#btZ6;4d~dQqu!VlX!0AHRqK3r;WXwmA6OWCKue(%UGRA9$@00mAvQ*V$YCa zRE36a3U@^qZZQdj=~L)=Q%)&r(k12?hpNxPRIgno0dA!w0|!s7^94ARJ;CQ-rG=HIO*Io}?)rCBWQF=Tejzq@V37c>5l zx-s;>Ii>%D1^?JznP++6Z->txffKv~_6B(Z8mYTv&qnaj+8I#jzkKM-A6Y1oUXfeQ zA0WfC1@J%<-Y(KwupsV3@oPZUgf zswUyG*|SWF2P}qrS{ni#u{OLxDVIg=kgBqQ%)EZ;G5fQ*=Hh{ce*gLytMBxs0jD;w z^E^(BG)}Y!=J%~iG~8tzhzunS9B$#(BD@_U0Hv@-EZsiUN4gE;Dux;4C1-;@qh+?x z32?g%xj-qKG`)0o?Q>Po6XXR!@+!^lPK(R5*7K8NkF=R@e0>Qobfed<_iOReUQ-6> zoe`pZ(GAZK41=6AzQ610y@LY}+ajt6ug?)RDE$-Nbwsz4rkvKQeK(EB%jix9`CKRbEcH?BJ;|Qn^wN?_W zjC0^51My;LS`;wGf(tYSNE#g4=zwx}^WKV-VkQOBqcrZKs$+MxqMZXDo(@UvWbn7O zY@>1o61wCMOZ#}-W@zGuXNVshoG5qwR~p6ag83 z6CU}2aD5ChW6@D%kHTv}xZD+__?4xid!k#v?gCQgs1+WyDY9q%6oV#DD{2-6fPMnj z+k*Qk|6n~n>5joce8dT}rO?e=)F5EOV0@=~iouP#6(FLSsuNVxd}C8?c2f8Gc}|@^ zl-Bm@j0LWO*$y7<>$%z8+hmvI>-oiBA6R5ZESAnMVr;nGnd`cY1M#uiZIITxm7i2H zf=lHVzUYQO1{2Yq`^24e{Yq^Hpb5UrWlKGx#ijfbV>J1rmWThU<#IrTu<|1eSAjT< zwN$vk#%MW@v>ZQfGOfHtuiKw>em%I^0S$4g`W8x!z$9aG@Y~R>DTO-UtsLsaN!3r; ziyCWYs)u?EmoH{02)kG(8`QO@eu9l21(;^(D;%k-)%eYa6zl-5o+`#&0JK;L?Rm@B zrx=4?FXCPetCey`Y=Mz3Ip%3axaexNeAOdY3!SeM&~M|~uWo-h;JIPuswv{6Y3A;n zPo{SKq#Er>U(6G$qyPvaf2+1Y$bbW@<#Lt&RlF7pof;6M)t1JC>#rihewkEwm6a=Z z%+!ybM2HWLE#|D9r8*0Mj;)e-&*qlfcYKTTIf;ZUQhCTq&^4VRWywvJj!5+pMqmD`YAQHGnvb^E%qKQMv5tP)kd&Nnp>NIl3 zKC33~*)AH5X>=y>4m(FT&TXHbosv&V8(5k`on$=Fv-{8 zb>f^`JN({Qu7}1hZ7wwUjg?8BwWcqP(Vv|dIzhV{bmi!gexa!Ek$H;i1V~$F1tvJ#bPZ8F&!OEXHblim(f0ODOdPVvwXpnHSC@`W|-5SXBuuEt5(F7Far0jApy?bkRfL&@Mos=fi@xt^=+<*1#_m)m%W0 z?pG9hro+Wah#?WOGYX>xJQe;@GKm)n8>1sb$NFl2K+$nJ2Z&#y=0bvrFiBe>a+C1* zBG)pah*mcHkRfTteR0|j#!Zs#z~;v5mb_v1V{IBSxr+A(hj?|jOdWGWk&7SxN#RQU z$(Jg^k}8-;?3_*oE%Ls!c_&BW5%Wn zP>`GZho_cDRrM4rJ3EBSn8dm{SHMzQoYb5yY|=*{$)S(o#1{A{yJME(BT5)C^+C1Q za?xu#{sWcJW2&X-$ZN}Ws^ua&ORwu&bF?Rh=oeQ~zn`vcqye%DY7_k`U5YwKjbZMf zKG#yAqDP2gDD143lDQM+2b+Lqk~NVF(O*u?2F{z2Br7ax~Lh~n({PewEZXEF(#B40n21w zuWYlWr`;L$j|y(}yl4+JIFWs(k|Lol9|?Y}S_ppfh`BWu_bV!4u|cb>9?J)%eao^U zCILmN;Lv(HIrg<-+$woC-8^G9Y6#_87HI<UVB7l#Y~SOWFoEQpS@hfDzd6&KQ-x8e`zYjIDY)9Y}r3A z&1d4ykFzSh4R)}*HEp0%ZyZxiL>zaTa(eRInCsZRD(d?DrVpX_AT%_jaKDGzur4oE zi0JsNEO96KSGsS4N?=Hc(4%@85KO*4TV1{b>l>m~gKISRJ)XPGlxx-*F(T|Fp|sXk zy&n=-Xbe+fHLQDKxB8wbqj{#%lbk;hYON!R$8h%sn5VSmt>#$7|};TsVM zs>WhU(PVR(B0ASlCOT;gXvPl2E=6xz;Pp^bnWM&i3R|0X(2y~>TXlE>*Tqfs(n|i` z$5@c_BdzjBc}BF4YFUz|dob>%%jF6L?=rt_Z4Jj!%Moj`S@(yZm1eZ+gw7Vdht*bG zTcwU84vs3A2dv8q^M+}LYa*xa4rZL9N;HP$L@PPaX27Djg19gN5Fhmo?lgqBENRWM zC6Bns2&RnZOYD{BOX9oxK+QQ-kPN4Dz9y(W>iZ)5`cjQAk{vaUYOkQUr49ax7TUzGmTL9Fm1xD>=3K>EfnJjF!7M2q^n)$w}n6ku|h)Z^N7C z))7iY1b_ysRmvhER)-{QJ8lk$%Hc z3#!1LlV0pfdrQO;S4Tqe=ue2)PB&^~fnQ5TnNB<9JYpedJT{lClMq401t)p7D9s>l zvrMAqRrv~%#B#0TC4tT(L#w6xH*be}&~2 zG$4DY?MJ;w09XNXkDwHMN$;I^dZLkU#sHsukDJN|*TfzzNg<>f9rWp9DR4B&J`*&y zdk7Eg=WZk@j6p zLa}nXRlPp$owp-Zx>4p7VWA6Qea@N=UUiol=UW0Y7y)&e_sFE*6gE%ZAdm2NKmRQ} z{x4koe^<*UUpaE)|Av=$VNj%R{|#3P2E;+bt0Z4MKdHu^kKQQBPT8XPLM8g;by>VZC3~U#K`F4RkBD`hv!d z1!t?Kim$f~)uuDswuRDrz-0$(N&r~UMMyQ|EaK?~UN|p!fCp3hxc^Bm3`1vj68m)_gyr z@|p0$Uq=Rk-XR7rKcklKUge?t*>Wt}ArghKTKzp5aRme`spbsh-MlS!VRPi%yNVkv z`9!T0sB9_634EB730~}ZKPni=ezQo#kF-!BH^HWvNHR>ulvrA)0$`DE>`F(|kmpbkwkz1}6a9h*Q z?F(UQnzZ1QjNTt^@2e)=A$B(hKXlHW<2#Mx%9EXcW7E`2wt^q^UJ5{4RZ3oQE!Cx4p4>0H{EC=Uk4i66obGG}G7y zyoomGE1sUOvQJEGu5$E%HLl{Mu63N3-YJ&Ar=Q%tMt*gv?IXHE2yLA0il(=rtLN|4 z{HUHiCz(iJ{NXx`87{`9%i-ZQ;?_>LCpjTM9~q*F38fYkp>8c!&^091;-4sbWtvwv zcU*qI8mVdJ130VAO*#(`RHUC9}&JQq=qgzzdm=d$$LSq$5LKDqjo=;H}u2Hlk z!+;9Tb@na(?Kpj$zYLTgkounU!#ahi&Un(pH!tZ7%stXKW(JRvG?s&1Lw*_S@zw9l zJJ#oKBQJhU<_~42E3_37VOyy+tKFK39k=4Tkq}Yu4M6@egT%~M>Vy^?pxgstW&AwS zg_a@;H=mh0|8tZ=FKZ%UtW76YHP>=CFJQkPt+a06tzeC(vT2!PWu2f*poEWpv{XSB zuknwO&z59h%Ez+H&s9(8!~72X(BM7h!R}-46%*?DV`(m?iaD!(4^cZ7gcpiQSl77W zgjmtd832&U>82PSi1$0hWPw7_7Q^~8?aY1rBJr}W0tEH>^UAH76KiZ9Rm>3L2@!iO zM8j{E?N|8YjEIVpspvdP@PNBZe?MQ3dlAk#hPrBz(T(x7F=Q_7UJsEY?Kj|Zd1#h+ z1@w%;+LmVEo@UK-ZLT3*4*Acekpjt5Hl2w`3xKj1k?B1O9;n~CE-X+Jly}7Uut~K5 zwI&oED|ygGvCwrHOHAtO4S-~#ebD-Sra>B_CR1E{o1dHVEK)o{U2}s;*HG!Y+@`^_ zATJf{XYFh}7-4TSau`d&BCfz+mPF;GMoik0^MwXzSea^Vs+W%a{rSNM=+Ps61tA)A zN&wmHFgK}9)dN{RcZ;TVPmAQ+_o?f*iJM9WS-l!k)AA5kWRs=ZgoOZvmd7d#OY5ub z?3T31%OAs5dFF4*c+Sa&R&+vuDp_91DA4NA$2W{T&UO_Q8WRhnScf1!Pf6k7qiFEP z)ErzkuN*f*{65=;zpWA52uZ4c^t5}zc>#f+jhW(ZX_8*bk2F8sN16w*U7q2 zOI(UP0_DJI47>;`<$af$WrUjThfw)as_453*;2=rPaL1$d$+Ab`8g!>T&{mx%>`hO z4f!pWu_`xIWY@r;)2ut>+?=T?H&>GqHWx{V&nVSI!OwFyi6Pp=pe89Fud5@3vREKj zX=qb0M&7J7q-s59l4of%#34h{`pdqIf>~#`c42(Oq{v69)jIy7^*knnr!ySWY263c zS=-;ry&8wZRc7~XJMvfW`i+u90|g*!@g`fd5tP^YB|^Io#^IcdIGLSq8VL?(qchu` zLt^n%IWda>GYVZKQr=AAHfztip>twnXi+F=ru1K7TXkmY*y8_4d0J z0(kk6&Tu1&8V`6|19E;|( zeJRU}3!cuR`6o1l<69Gvd)iwSozBAXmoL5_d45+K%)g%P#bY|PO7S|a=Sjb*l(s6y zI7=j2#4Z>11#~Wl4E~;)i|ZgGf=%q@Q4+8;KF##NmyW_@=VFaLl#=K?4mUZyH1cJS z=;}O<(Ep0djqA5&-}y;?M&J$*A0l{@9%8(f(jiE))ge$=bw^`_#z+^3ZrIuEw3|Q1 zyAV_EcXeuyQgeRG^JouJV|SMZ4oP5q=I2_?VquMiDFG?>B~&HzdPH7WBNrK8o!bkV zy(F34e!Q&A+m~02`Pk*PSl}=RKJ>8lu8I=k>kTY{)y?%06&(J~2q*yT{geRuOCkk3 z_vNgpVYb4J%#Di|eJ|z9$}%JnP^&(!lHBkM1WbSrKn;oi42Y-UQsAAGW%RD8#!9J} zWpb}2$dI2?n431;2RFQC?Vc&rM=I&-$SEYuJbZcp8)nT>9;htu-G|$df2a&?u z4opnX{rJ&cS(iN67MTptD2h@WpazPOzU~ z8ScaQ3F&bScTfT&sP{r>>wi&KmhN_nb74FN`=6njH8^RzJ=YgjRaksMGF7)~3%ynN z`B|KAicyyMOd6;2D%MD8Ld9Us=&bEoQ}6OYW$vTpxu>*BJoD|pn$j+vPx7dJ_o7}% zBBGRyQt6CERfYle(YP!|+{%A<{#g7yI#qhWG#PD-kC(f4pwX6-E}m*-%I4w8;k!><OKjGrg)!PmF9L?<*nd|cDL|A z%`5M$u(>F7FPPXEi8s^?g`}A6UqfKAoVCo_oZ;?TM{*{uW zH9blQcBtMD zHz%4EuE*bmD;Qe8^bgegVCzE$3TAJb<;$Iz3g$u-gC_T}8DhoLL$`WnU*OHq5V$A1 zF#^7sCs)emTg#{&Dcntf5^n{buEX^dtxR^CJr5}X3RVuOafY)Bqf*~8$GQ2u@7!6t zaAHOQgr6g{A@aUo*eL84~=7~}tVjHmF`dQKk>?f%YXM|>;qKXoJ3l(c*I{C&%h zaaVOx)p+eWF^7ZJ^GoNU`Pt#p?{FDX>@e?qip|0_?W}{Mf_l*_Ey;2th|&Jd#ZmXL zfBe7?z*u*ozoquPpW~x!tGec3!XdUg792x*9Bc*BqJj{;2bKOlU50m_MN~X`Yy4oe ztGoY5S{0VQUv~A@VX62tG4Gz*ZUhO2by`8;4^n&(Q_2M81U-74W9-EFGCm$a#?j6 zhI7m*i5I9VDeRUsjmAW4a9sk=ZOjXDp+)l(sq}(fR%R~d*z?m}_z;nlU<%^R*nllH zXCzZBqiZilIR1OG&`cd9lk;UCMf7c<ux4To`L%}rx-2yq`2&jjkdKcYck&@4dq)n+h8 z^fb{8#52}E5~Ww5v38n1jk-s6a38*QHdV|vdY70Ft_UXPS9-0<5Vcm*4l;U?wTAs^ z($DH0=Zdku;- zBA~rGB{6hK!8&C7KdAZK0RAA)tm8vn^4D3YI9o(`of6NUu)tk4r<|?iL5rQz7iksH zekb@Ml&_udb;mOmi0&>20Oby*A&^XwRel|I zCDc!;Kb6~D)6pIqsf}YrP^Op#^cxetD9k-FHgoK$P(QwXcR!fLO&B_In z&L)Lr96;3s#N3wgfI5=`>ndtqIM>9KnkCH^Z;&$=;w%m;i65`|x5DP+vWU48rw)9U znhfk&0O`x98oq7jl;VR}>2VT=#1PawvS5RRaEcTk_Bh};;HWP7B6^!#k-CT5Ed{GM z-BoASeQ5{MNrPEy)P%LU&$9JKL6ZBt_DmYsz?X}ZlBHh~L}=+I(ieRzAg*dJ=)yI> z!L2AosPNR$_|*X9a}|2nHG)D< z^45e*FO%X00O!nU!>fhcmB6sBDiK?y3)&dzn zNfCc|hSt4GY#QwTt!xecHJdZB24mek$=R?re+R`ANIe?iAb?k!z^emNMEA3kP70zi z{_`^>qzH^o97bIq)*RerkOuxY7qe*3RxK!;F{{I6?A(RKE(vIx!f59=t?EH z$wpUHFpZu}){so$OdYc;ad|{MNx<&@v^>dvu1K&}K)3Y5{rRV--@0bTOl#KWgeevUD{z_f)0cj9;ZAwX~CJpz@Lo*VfTdVrAXNmI0cTO#Vscqa$(OFlIlI50;+6 zE4ZcwQ1TW+qa8Ljfx`e~1PpZ=bqdar5|?yor%y;b&7&C@CVH2XOasE0-}fl z{D%HTI`~5l8V(q9#|_tnYzo~D3H{GAiQiV)tATcY1SWtT^;8CWeMD-@D-N(#c@IT% z`d}zDb9%9G)r&YfVk{Tt73G`GE3Gq?Xpb#co;{xi)~K!Yo6j5nKgZAQBSB>6xVJ$j zSefl@S9h=cOP+kMPY+w4Akz_9?aPrCj4nObz5Q^68&hJ;47)IN9~3!y&1Uq?RYBR^o;L%m0?3vrU60+}Tm~xIcaF|mLH|c&y9mNB%y6qvr=VKGu zI~dF=hZ`KsD#sfL%-;?-7?{5uZ=(EM(c6o*L4$4^Ly&8tSd%+HLa>>f#!}&>Ek&tH zhra)0&4&byPhv+sCbk!io=O(g{ykwHMM*&+USkAUoaIG9tpKO?aH7v2+O2rVgDc2j z8>(&;G(7&&Yqsa-D3267k&)@bp3lwGr0dPP02VA7lTB8a@Z&|D7);CxZ-kn&tmzXvU0p%=s+W(wj%|naHZ@DYAgp*9ksiZLXmuz4Z+v^VO*Ho2i3Wpc$1{V&$ zFCe4L%a<1!6`DXy98;n58sfki1^PEShXbNjTap_Um)Gi%M-?njn9lDWFuEW}MVSia zD4(FtOsML{f7!+5^eqV%=*)jtn1F&*^lW%`lcXd~ZyJ;skT*yXR*m4m@8hA76(xO0 zV*4J)Yf0qHhLr2QB5TH#Y8;>Jo33?29-DJ%-*&^Q+(&y*u$KO0WgqV7R5)oGDhc34 z1~1zWpyN%HEj1DuN)P`X!xhQ08X0Ifp>m~Zo8H3q3XEa~whTqg+K3ldboTPBIqZ() zIVsbG@oj@jy?zN|%yeUC#Iz#M^G<*b;7RgOdL1z2J|U!p zs>@$RT{l*#&KF)BgsLb+BdXwan*hKGOSt^TXjbi{2NVP>2TU{t#>^R&y&~k4opw0T zvv$M$rnsKa(Sw_dhmL4EyMZ*Fy-PF=pIc1N#zso^-<4>1xSTmBOON}gF#BjAn0p79 z`UVGp@?s>tYG_UG6@FLBUeScg={-n@wSmEFz5LcY&-q*f+;R>xy%+0~6M#nA=h&Tn zn8!XiZ`y8-n9E`-V6qk6;h-544CB=OL=6iei}TAAuI41}owR}!3P!yP%aB9ig(AhA z9xO}q_*1W~gi27?K4Pq)9Qe};W2tl^n{XZTt=}X6#o8P8+2~@vlZpcBW@32dR2op5 zsYzR!R!LHf_a_b+GO5$!96)l`)YOTz#gXMzfBneZ{G!hIq@CNt+{`HbFo&RcSr|lS zIdCKU9@MGOFN$4|pMBaKtKc3|`571IY25!^$OsWf!s7(&`Wrj+co-EM}z}PO!DUq#_T%@vkupHiy!;sJ+GfIVa#U!u>R>oKxbb zn(}+-X;4}5&33t<5r9&IW6Gnpj(DVtxS*?BPsG)LvWEH3#%g(R{M3(v2@iBmCb~%( z9aUQzOozFeA1Kulqi(Vo{!PZdV;J2|3ai_wF52~QM>Uh!Q?d!3E+zx<9dMr0X>2l> z)&3Q>1ZNr6@coO)!4-s{99kYbc=e!4Z+FYvWNX1Ja*nEQqW~edd5njV?2;>| zZ0fBw;~A2nCg4~I0USU!z$Jah(1~+Udzqj=gF#Usq%BGBLV#GEWzU=IDtFnYc}-+c zD=>gi>QMSF zbN57Ul2F~QcQ+lpMGj+t>iwb9`&zL1`EQB)-?i%tUh#|Ye@h<}$iF3Qgm(lK(EpUD zTSPGAp`rh=hD+;dWdkc=5vKqV5m$ht6QcZtKTepi#{)!{6ty)cv2U^+NBj` z5X-oA_;@8HjsV(1&bniz+zaFT<_5}`kMHE@gxX@wB`(v-@sv*`8fQ%L6h?Ugg)ClQ zJ5KrU&AHY&Q*4fZ&3gdpop~H`70iuw*+oh?{E(DiQUHVMPSswqvFza+D>><6tOP~EY?iX z1%4^t!poT{J;~19aQ5ohd)NLJF%u&wvzCJfMwBo&Z`<<&Irjjf1%FXnXzGw~)g1j- z&%h_dEjMiF(ADBjbXb-U4jmdVN=9Sk^*Yaw7bi~a9@&wQ^2t7pP%L2vak<&8Qh?P4 zw#$xr&PD8vX<5>Xt(KF;X3%M7s5+rLlF(0?wKB>N$3eEXc}=RZ(9ztL@qNcWnV~qT z7MY<@Wui0{2e|+V-D_i~b+)>6n8RESUq}g%ix-bLi=j9GlhQDGgHCJnsk2hsA}kh4 zLP|UMHBA)6wUQT6e=v&x%r?m%jBZsY$ZKW+=xoG}Z(cMU+Pzbt&)@>_(;S5T8)!8V zv79P><m4US_XMjH|rzGbu3B_XZ=a5fW(BNW8l{zfcz}@Da@>kd^ zz;7--8!!W84XS4galx#MPt4a-;3ALM*iD>09xZUNQAIP2^t?;`>GEh6-;%j)6Z78v zm>}^Yr^cmo^^|jr!uu>a%CJkqWLkRZsm^XHYF>a(<4xM+amz>57s3^w#TY|CdQ|dQ!qDBQjf^O&paq9us!Vzg%xat$>^r#Kikv5C!h+R}< ze;`gk@-MEl!&cj(i6i*}tg28>p=pkO8~h|w4|w$>+xYm{8tv_R5nGqEGVIT*Cl_s( zlrzkiTuH);O~emH&YF$IQYcNZmDE^~JWy;k;>O?>a;z8IrW$cZ zwUnJ&3obBuifw>8WIE^(%e-kEp=bAoa`pl{c<~e7Rb(W-Uj=*0{_tRh@wrB>+7Y?z znvcW588u6__YU9E*hz9r=!KwY=RjK&^OFp~5%H-HJ+iJ%Qb)k>@x4kW4qHJr?w1KV zwaw>+ljM>@lXFmIa4RXRlUNBar;ZxN8^^qYtuaViHAsef1OvoYX!SADdEAu4`7gxA zJWNKr^}ZD%mu>VY!EV{`gh;aN&R>#J7L5gLYig-4ZnG4lg+ovHr}!S3-Rz!3{8|)% zlOA_JjQjd=H7n!-t^5|cK@(NqPVud_P|``TE7J(il;Ur&0My9E1Y&L7ASRF{na+sjP{O*5_Ozi!wTk!;>fq?!?QNSv0~RUzV&h)hTmY8 zJyX&z5a6&YQ=P6SXdP#i z4}0lHr-l*+@$M*82%>hxJ(1aiNhE(GHw2U9kYHTrta-=_88#2;RI-%DCs5A|^$Fn> z*+zsCG_42GQ5hu7x`ekg{jC}EL!l6Bvk)N)0U|Y@zt15%EhF9$b4jF7_(78Zn2N`c zem;Kws2=-SJ@nIVx+Nwu9*Tz7raEMFu0)~;oNm{(Ze(hP{H6hQRSix4)d z4!NBUIYlcTMS5*asH>0b@?e`U0M;;`(SdN3bbgDvMjv?nYVvT<_lKI!dnom0W$)L)nN8HBjtZjkrnb%e}+NQ3^1)P3;2=lK243_$8ubkJC zt{QIPkW@+9jhUB5)N^xdfb-TU_UIbxeP%!F)`@d^cKH03!j6^@Gj+dncEp<9s|(hx z;x1C;rqI_X>q@{j`U~mFJF3NJ+`_Y&>PAN`&AQ0~LHP=D8B7{>UWSj1>bK@H%PgYN zD}!Cs4*yU0qB0LsMC8=g$Uor`UDMt*@BC? zk~V1{9NWAZOvrwwYRXnvw&1+~tM~NpZUG7e~O@Ra)fP6%p$U!heKT*`YsxQ=M#s)kQnha{Z z>Cxnd=On-5p$p*S?=5^cn5omT_!(2oqMc4(b`#4E*9(k)WeiwF`s{9VIs6|5oM5TPp7jNaS-t5 zbU<>VuHUah`6<1UP$?C@KF>d=t&|}6_noG2uslawD~aJD9vhKB7ZL^U~PBSOt$|}mo$@w(puSe>T?C?zJL}Ny~C__@JA5Dm6r5Kyc!{(>3_~>8FWwL2Lw(}@T#%9dkP8e} zjH><{1?I6E?6KP{Bs%x7vzk)_`Uigc4gD{v;op!$@2yVKTA=dm6(`mYTG|SHv=c1h zuuPzr-7V)oF#rF@UH^Z8(xwtLL_CQ9K%~Ueptrsw{IlBz>q2Bb_<~3&F#YFWf)vP0 z1e8m`Km`1U{GdYIZbcdIO_E+ha}J#)3`7+M?LP#^PnA&XV8b+tX%C z3S2zqZrIyX*XLgPUDjR3(0j0&py#7INcb_JMcM88tM=}ucE5=_K`3(9X)b^?5uE#i zIP+E%AgE#2hOYX?4Nthks7mn0IP=!z_D%>*ShQ31tOuPX{wxS9&|sLV{=$iec}Gd+ zWf};dRfHVkhD%LcC=^3OK_KSM zr$)0B_3U&&-5<-bmc3Yk4YNQIl^xc*Uc7!YOtzhN1_HK0(}J`C`cE326Z6l?o0kOr z#3ghXYU17$l?M2UNp`K$Qrfe#j0UZA+_Ln@uh9Mwu)BIunZXb|wddO0rri!0!*x)0 zeJko0tZY)$;Jy!S*;YOK;!&t_{;S^Cx3*2Koay|ie(|1;8W>EO(XE!dwxkFshU zgEnmsNFnA&AQUvI2+h05xo~Z0QG^1=4B0!YndDE|+lDvZu~7zwZwqMKb#V&p`z97$ zOTX`WCEwc-rD~#8x4MySV?qN+F%&BvYY;BEb%kt~R5w_+F;qy{^w~wAy_kr#@y!-I zcYW<>Up+t&)80fjI=K|5I=`w^)xMBUlxHhsTUbAqO7WC!Ds#1&UZ`Meb#Z^MeAFCR z!oDa=s7XZX3Oq)DZ^rvw|m+f!X`XWbAD@3-Ge zqUzR2sqD>yIu4i)C2A{cYnXHX$)gQSnDQ-|Ln2 z_wh;7!lTT-AxdYj#A@d$*2Yl~+PQfyk96zsQ;ToZKHulgL`1=eqBV#|fqC{OHi>Ku z+_fZXCHV6?3MkS^PTvIiqZ0{pc`wi^@QJK$Gjgz4$Bv+phP>}18}TOK+X zqEZy&Ft1Grj}2j9GkNlw8{iC6+WL!G^q5}-bnsgKtTh%8L1A2jP!p%Lg1IBibxoeJ z?c}2f)9QhvldGGCua-+*4mr-B^B~ipjRii>Z;J5nQ`9(#*rGuY?NA@npaMPWXUqT; zNBFNDM_6jL>5B4)kXFOP;sTwNY=*vQp9ucv z)(9pN7MEPHL0d2r?I>Df?a(sgdqZHvSobBfqdMxtc}3(OM6s)E;$R{tTF*{t80X<; zc-Ij(sI8=uc=eMgi9=_EbryGX-W+2Tc36YO>KJx_E+ZrbK z`v?&;#OeI@8Acl%8jR)|5_Y%Q-Y4Dy(%s~gf};Iz_~M!)W-Y=9qr#6usn7~LIL0xw z-0EM*DK3i9zdD4d+qjxW>TzG8V@_Ds$@VbTM?+hc3&&1iQY!PyWPjob$6%Szm)u;S zsr>i}7_pKcypkzHVx@;JS~Vmr759WIwc7juHs znpCY{JKz%`uqsuYSv|49A|HJOEz_w|ST*a+qBEr{{2BmX{(snd$LLDiG+4M}+qP{d z9ox2TJ3F@RbZpzUZQJgolXP!S1GL1++gz{kGpW-#ouH5QFXoG!}l}$ zqP&~Fr7~LD`Jij!-Fv9o?nu8xd}o>P5n;xO7K03|1%m$~JJxxp<0ssQVy=tFO)OD_ z5+t1i9N%I^c2mxwyp#%V$DdH2Ctc!K7l^!+#3Y8cr27?bEi-cS2btmKkmSTrrh#-i zeH8yXL}2LSVWg&{ph>!@ln8pMWqXB7UqXi&XQ$)7$g&Y*TWpx%>SR+YVnn|r2&Qw? zBajVb6w;YA%duiOa$}Wh?Sm3%(=%GMH?W=nPEcd@reFaa+Q1{{C$Y_jufjC#%KIqv z5#6&23<#rTIWEuPvz3TxM)YxM(o_SUAyAiZ5#c#IC{MvDQu^;&_M}T_6tU=G6iSn> z(SLOr^SvwXt))KG)45tGb+XzwHm&)H5)Gxn4Y>^c5p>>X!-^+@6l%F%c*Lo7bl?=t~050F|u%UaH?=RKTb#S z%9)RTd%)gi2?%8nl6U2(94F*h-d=^q}YAQg$6or!DQ6yn5-$Lpx3$;ElkO z4(8?*M(iR&**}W=TrbaWljpq&S{VAMy9E!2Qv@WAbQFCOb4#eYHOwx({RR5nFLLT2 zcItV5t#8E|4$W3>t$@VyAcm1SUGzkK6W*3G6QgRaJaoDkNQJWY_2pAjpHD^#K(_qF zRp95$ECjv0i}l+lLrIQS3L`pNK~^dZ88x9e5(!UlR=6T9GBL+GAQhid?i`F}F6offCjpZ5aB^u@okMmPkbA5-N->q;| zDjC-rdDp(YcgIO{ZDR|!x3Mntp_d8e;HA5+cV%~Q&Mm?Vz;>ZfZ;kAf z&GSHeAsRn>j8{Y0Txcrh!{hUbpZDghE-~%cN7$yr#(9!Y2EHQdFOpmfH8I+)!VIm0q{uh$&YuT9#9khg$ zSBX=z6M|Hajfjp&x+*K(e?sbbTV$^{U~fcUoK9zewbHG+yhXp$$P+3Q+3&~ko5H|8R^T-^xIsVF z!P)BL#23o67p(OnOZ`qek(#HFicRl#mdETgbqGt7D!7=EN*DA<>pl*5HC;49J$3C7 z=0cc(?W7bB@Sl(aKw_d`5ej%bQIWLTxp|pJN0Orp#tXvG-586<5!24C!G?Q9CnZS6 z0l0?wZ=fi0gU)u3AIcXQgDUXi89`%G15smDjO#L^3KazdKh@CA1WAC?`3cRa87`begl00`|Ku#5Mc6L)yDBkqbJ z6#GMhI?=|V0x9(P%^*i50Nph*1fK2%4j%&-@A%L z)t-1fR&r`58%-qP3;xrRj5}EnsiIBYcu@QZPbfUsP#LWBqE&YLxavIhVz7dE(rSv9 z*&A2TV&d(bCFx+z4;^br#^6w>Ejc&aaGBbmF2R0I1}NH8vRNRUSVQMMpQFG-L(dY; zYQjvic>|L~Jir|^^P7$Jlvls*j=%mJNautch64qFCzeS3srsa@Rfg z;T{L>BVhL^{X?5}NE=t#FHH3n#Q4&7=@75&4j&>=ph3K6HuekX@J!e(gb%9KCk0<$UY-JDJpEzI9z&1YJ|Y`yTW~rmD7b-CHEuCq@2=k9WA|kz0CqmOJr|3FiVa zH#DS#an(Fsyy61y)Of;iD&D~Z?Q6R1=mgZvNjlFkHW;X&J+Dv40XPIXR)W(DcaaPh zA0VC(Y_5L7xfU0CmyIvJQjbWCT8xpFEOxk5(MCC8zFm33b8v z+GV~eE>`06E{zpfNZ0s798jkgL?~ZuqRSY3(Q<(Ndm7t!NyiHn1KucS zf~QcZ0Y)4paqayNA#xTMg7vEhg42{{VUR0`)2qQHp(0+CR-8@;r00_V`BHdeyS~J< z#!os8vCH~Mb_2ZTf&_LW8e7rt$Pk+CAf5{uzDc2gjQYOfMo{UQmTG%ion`6cE&yZg zh@8D;C8^F|8Wso!CDk#5h|oZcNBbHAmH_Ds0sDm0BK`1}9sE3sA-g8nG;unICC~nUhr;SL}J1;_p?@Twqjs@#cBj<*+GE)+>(5mbh*Bnlf$oSawKex1HiFE0hdV) z{XHA)Rp{{_Nrb+E?iIiJrDKAD5uhf%ADtWPLqQk_dx_rhzK~fEcW8hA{wP}D*N}RQ zGm;j#UB}*TWV4I2PUh3H#szqRm6YWV$OxPWe}O8WVV2BKrc2g3#QUIGpOv=xzb`*w zXP%iWoSmK(>+dn&=*#s+D$cNlI@R>yT(J!0A@hG^<>TTg#j*T3q%`g(twoxkTEMR= z(#MgqGBz5hwnG|#X|&VG$14%xTbH>m+K?s(v44Yg_1Xj(;|6ul`3+FAyBJIswy0Hh z9jz4_4*LcBt8jpqWYk3f+KhDD{+EdVpjmEzr;!EEhC@w-PGiLu7o%^esMX%sYxa2-Sy%CX{2~Ku zj(qdpE?2qr4vVfU8$4y$&%ai@SoG_7v`4owaWJCI^n18M6g0&&lS$2gDc!olXG#S- z3RK3t8avoj`C{CReJ;ZtUj&0v)Bp?y@w6vbbUg;2zfQVZu~<3#Z1@e?n!eP|W{ckz zOZYzF{~53SXI%Eb!?t1z43*CscA$k4ewH8Ep-&bMM**KsRyq?Lm$E6ehzX9&c4O=z5KPFah^<7Rxc66H z22{RW$?dsI$yXV{#<_0Bnp~fo8D?DZc4g=3j`Q!El+VxG^VuIYeQ<`5t4GH{jG_02 zpdjf9X_rVo!8p+Tv3ulZXj4=Y?uTla0K+NqI)8a$jUgotemG<7Jjv2%R))!D8fd#P zsE63VS+p(Hc+>E!3$?B@Zl01+tQm6|@y2$ zH!}R5QJtClNA3A&!IV~}xsSNUSy6npF{cfmjJ6qD=ut;kXiv5`Qcag7wvjln0Gx=$ z!RG1D1HMp$Oa&}2mXc|8DITtynhUNdT3@WuxoYnY>xn-JOqA=)WzwLBC3M@5oU-;g z+^esq#DEbhCv}VxCXCAS zQatTDJ|Yg2!MFiL>of>qd9||u$`kwxDsAO-zUrmdS*>uB_cGqj8Bu{r>UAXt(p8lnFy^XV0UWB? zgT83``_^a(cQJwXs=T0dIabZ?Q7-M11SxG4l)&q{1;xY=K}5@ThHUDZ0R1CAYLsQ# z?VY7m+9>-A^l~zf7`DL!=}ojro~@|$*zg1+A&%>Ge4bqSDMw=lup|{ppc*5PtR@I* zXhzBT29EK{C>0w+?T`>WGy#uKj+ptL|W>HZgx>Pf%`(2XIP6U z+3baK$>b8LGR zqgxcCp|=l>1Ucw>?RBk#HR;;qG}%khBbM}S1+{mHf`JZvX;G;009k3)U=A*r*$!eQ(|(K?-3oUD!z!Un#I{pemV9 zFxLVGl`L2a0+Kt}{pX%Bq9tfAV2#bZ^r}6!H{!G6uq&tnv}`CNSueG@(YoIt-xY(v zPm@1BvGej}e6aake4`QJ8N(@tUgMF!kf?Y=Le0|$<%d2ossM)^n6EirzX{fc^Z4gU z6)}gPcLW0|;|%aI#>ri&SI5$zECVIv7dRvLdL^jRm43l}1=a7I(Mmc+tVqSwF6csI zNR{{v_o2Y%R@Gd@35x^KFA7L}*XaJPxc#Evd9+>hAUI@&UR)#+K>I-W2ebZ#J>PNa z`tQuNf;2h^DD)3V$bWlJVJ(4EiNDn|ZNx67K-z+5EPmCd@jM)-^1CHLizpSM`T)otGdGy+W4+x6Vz-P_sqe&hY} zx6&S@4F}MgkBp#oPZhm67c8$0u%v!86A~GVWK2gJWwH#yN?;ao90;WvnqZ7UQKJlh zv=kbWgh^z#38EHi%m=30$wp=}?x!V~xj$i1%N{Ye z(GG*!BpKJgM%$|yM`9P#0S%=4$wRrdIB?+{EG(!|d>ZQ(GKW5_UP&6Li^jQK+%bur4$pYg`)N#44~#q9fZG zeYt=KE$VsBAU;QiN%fLeY6{9b9oI~hxnU$;1EA5LZj=|b zA_Fe+w?BDUN$dmwimy+ki}*mASPyNVqBDM1%&C59m@~D=UwGyqvl_$K3)7U#T)N2hO{5ImXCzxO_F__>$9GJJb zB1e6gV{%)H&tQc=xM!}7i&t`_Bk+A0~ z*0@_oR@4dM6qZRWCq7@adj{n#)>hh{qNKP3`fUu~0(QTqDFfK)14oB8!Ab`3g|ws` zuv=|jGN_F{jyLdzlqg5GBd2MfA-?G~*=YILC`D4pI5)UB5 z)2+w=v+}9-;|q2V6g7cDoQ-yh7fzfb#_sU`n11`3G7Y%Gn&3o3kPF~Lzm7#vAMw6Z zJO-5}ai2n9I~oIN%lY)l>d=CDA3p1J6Vl=x&qnHevu@%xr<}Lg_rblRYE3!PbRlpX z4M}!J$H0m**H=@5v>CQ1t`^ZVIxXHb5;rcuDwvgiHTSAau~X`Zix(y14ZMeUxn4J$ zvl`KbLMW;DL~(owuQN8sE#6EP%F5k9dGV4-LV%GQ{^Qx{G)lw7jcxwNnEJ}%7;coj z+(YoC*5OK7L46VNs7|)Og>LtH2>;(79QKkBpvmIP^y~?lPgGs6Fqb#Qy)jf%7*y2& zbcTnV9FjkN7PcQFX{W}4E z&{fME)Q^IsUsZZDrji{28M^}&y#qV4!)&t(SvgCeDiBz0cw{MF4If zR~99CJSpX@n}OR;jU@{yBWPV-#XBMZZ`(C_Tb^l?w~nkbhRW*7+8S1Dbi%H3f~#LE zVtjLwe}Sw7qg|9Tw;|37gE}brJMzS|pO7%j=fuPILKQJi$0K>9ai^BlmuHNnb$yE+ zfkHPfCK*@MGU>k?HgyFd0tAha``kb!EGP@i9LaXZdeE@TWC{chm`2=X(LZ+wZ|5~F6L-xuqi zs=(CYpW|kX-AV0Ms+clDhezINiT|6`t~+p(d&=z*ZEHtxsuII7%wN4eC=`vL9_RXi z7+1_hWZHO^mp^kP2SC#PNB8k{ZQ_q!=U6Eh=Q}5E)V34cb+el^_6&UGc1QF@U^btQ zT+DVuxE9EA9U7O|`;}VGEsj^4gK=DfXTT!msIbXvW7!Bv*l5Sy6v0ot24lL^%vs&sPGpNm00v%(t24}-m4>QigXzc z+e(kpT zEGX_`IsU7_KcwS7%;JA9N|9d0rO)511}Vw@?Kt=u!9(^hy7B)z4m6=WRF=^`<(XOR znIDl24Elfq>MYYGnGuIPWFcU)3`_NX9 z-qRrw2R?f8;lXh*CLggHU&UUOo)45hx`4joU3%c?@LeTzPs!e;ozGy9=W7YZ?f{CO z56-URVQ1hU$@(*IJ0Jb7kDpy13jJ}xLj*Lh5fA{PkEN)*{Zpei3~0aeky_sL5dEl> zUdO{$Xus2b=#QPC(mM#E*Er~0wL1z9CH$%Z4zS(4khWgbz1r8Tz~0V#g!)_aq1ey! z{@%N)K!A50(EIjl9K`ItCl~|b(wm14hn0~Qb#W-s0A_Zs;sj;r>&Ur zz6D@wDW(k_CJJoc!@O;WoK+&!Y5(ym$fj0b8=iBM=@~LD8jO;VwhEfM83S-MSDRa zJwk$bWJIo)bd~rRdRwBNa-gE0uP*aR6*zTLet5&oUF_&tt_id zY&W|~`!$8e8c&e?I&viW46UuLis1$BP7haBe704@rbNk24ximYs%Pm|WL!4dct$`m z5%wz1;+3HrKkLTWNKRw=Ze%3WgMbTAf~v@@8?#t%*4dQlEA3D|AaO>C(=&x@Ad z!>73BTu&-V4bFyG3#B+maK!xR^zPsU5@qx9!)()wZnNgVG_AM5C7+_^i*`J14QGrtmrenv5nltO)#s5|18<%PZC=xoH^*P-+j^e#wx0ks-7^ zXA_;{6Rlz4!$k3(yX?Mf1+ z&f-I>8QHW_vIKVGw~;an-jkR$xD*&XYHWvu$m=#`Q5G!~)`EydPpTIY>SM(oDR`HP zwcu%JE>$PP#z-|AqABbv*|zr0>ef}ua{3!3R3#NBWgv6-6y77%dI&!1op2a5B~|Jo zV!1DMJ=JWc%C(!cs0=`QB00Kw2a=((P%#y&>TuXSQc6ieT%AqT)%;RR)Odd{Y%n0i`} zg2R`u+-*~9D_UEHJLyEKVmqL$4Xw4DHSXlm)#mD2n`$kzNou&sU;Fp!W<7%*eYAer z9B~TzA~I0P<%q*&yw)I@Y&=9iA!u&#!n#poF6=_|qM=Plyl%mqQc57IQhD+Or}+_O zlMgLKd5q0nNid*m&rGQL0Nh82vtB2^PHNJEt2ATPnjr7;mlT>yY-F{Cl0}HrsS+8% z=sh{k{v?nk+41qQG#a;jaY|EUt!>bIy&S=Qm*kF=9FmzT+)M<@ISAyiadUxA?Z|2% z@(iXW(f*aAS%_oR6zl8bXxZ$RK~Z9TS$sz2!N~{t{vJS(!~KlJN4fl3F4Noe+T?FY zGls3zc28q*b9>KyAUb_tP(%+o5{hY_F)9DD#r5^25tCvKwi8|XOv})K$nyf8&q-UT zSxZBi$&sD$otYAQzP(`df`wJkPU=lnOw`~C{Ux!rTMG=ap(=bYTq|4HvyS%mIIKT< zoSFAZOesLwhgu_{TG{UgyLh$%X>5vK_*PEmyJPj?2W}+=k>wuu@cdK=);vp58^wG9 z)1PxgQ^}dleh;DJ@X4K4|NeTtTM5h+h!95~dhA84L#Gy_v+;Cv!)o0x>*^pu>MN## zk{K&kgOyNblUm>r^8=Zqnle@h36l6v#vzBanqI`<^PSTFIo zJ4p&wNf#WQVUmtC*_$nV#$>62;tWX#l4HyAHzzXJv3r!|AhibNzEW|oN)P!tjp+E$ zz_S5?$Kr4E{6FA;;5%ca%gF6~K6SiLn|h3dmJJKh_sQcKBJ^e^pC2aMBrq<;rz;LW z7{Z517)_MQz=AE$UvE4LDCh-5Td|pS*^cGf2cxn7#`2AwY4|snf$WWB96Ssct{+s4 z%ICEmj#KPj93?UMJ&a@U_?<-D#ODFo98v)GjU;*UPVs!?xB8Q%e<=Nt-Oni5Q#=9_ z4oe`Q<%1PpH!h=mRS6{~^>>A7B&0Z_ zX&Kb2G}Z9RgbZX;~I|loU!E0BcnYK%_H8>62A<|79`WtAEAB#N%~G0 z7Cfl*WNe@Qu}L=zlxmItBu6EJ`O!WTs@CC8{FZwQH*{CGQ0`I$^OJljD9X4)X1~0w zXMUQ!v(c{WoTVY^<{f>lt(3HL!5_fb)IOs2i5yFtzgt@~*jkX1%6$c;xt$s1Co0(!=}!!7UIVH~xo|Qtt|8>j{Dj@FQ>z5XOgc{N(R-i0(w; ze|=tWPEY^{;-`B2ZZGw{x8`T~Mz{QZe*Dqaj;Ob$%dLBYB~OP)ywL=Ept%S0fIHF5 zKnjx~?!EmCpQncarzj2Bp;&RZD-V-lx#-^q)5IgZ9mw|khpmXSd(>Tk6ei@M#pvQz zXPFSX+diH@;Q}Gd+D=(h_BHFZMiGJTRBany&@+m6H_rO~(EQcNj;qn3Z4de3@57vk z?$t4Emsi;yf%hZfgAohjNIgi(n*Ah@X<)d(^CX3HVtFikxIa-r?j5tIZ=D-=oQWq| zqxVl>+eo2-r3B1u^VFsQ{hB*r&;iKiKksuEC{h z#QSoPCN|q+Kmh)am0(F?a_ zVmVnA|Ce}()_I{VX=w+4^nF6Q5n-29m+RuIiXhmy7smLl0ZS|ZMcx?apWtf~riT1} z5*X~1DXJg)CNMkHK>G(pEm*h4KPizBC-<_EJkXY5AwmZvmj>vRU*FMue)c~@^IJ?y z_*1^dukIu2W0?ka01-6rH$=HYbvp~i71j|{w!Bxw7QBIuy8Ym(3^Os~Mu68EcUINJ z%uHI}$)5UsvS<7X;IyxmeNawVdWX`sXGp^R$OcsGC|GI|?8ML3Xp_V}Dr~z}b>_P2 zQ@TRXAz^JlvUHqob_`2!vI6FyA^68Bn+dDGdul0uhvEmRCl?%Y^h-hrH4N#iY-qT3 zEhOllKk@vH-K;O3x~)PqIHuMw4qt|)eYu|q{E&*R>yDO}_fMAV@8>2I5Z{*<7?y3U*EbmjA5 zF=~#fq!#Q3=&N3klnIb72=DZVp9igDk#~yP<4dt{h5G$LqtVB(MVW2V1b^rk5HTc9 zq5f6pC!s}b?7?Nav@;cP&D!^fty@<^N1! z!uJXD&vo)Ym&X6SPPRH0;~xV5Yu#t{4?O?Ehn$4Fh2Q{~YaVw%-aIqTdWA zt__$DGRiNwy%EW1TjET1-=KKbks3=VqI985_Uo~SWwm%Rb%x-NBEJh4?1cq{9KV~m z3m5rR+mZgYU>UK(&CbOe{@Ls6>YLrK&o`Vt*$>WWMIYi!dC|uV3w{KW$YID-2cAIF zEvQ5E8!^Bg*iA%x6=qGi9(}M+`cp4#KH>niu!;OUij^u8Yz>P{wPlHo!>A|yyR%aZ z5e61Lq-xZr>tdZ}0F9~LI;^uT$^z}>R-$U-lxq70r)&GqnA(oBpk=zolJ#t&9WFDZ=iUNC@RPvEe69v#{K`k;4H8pjc ztVM_8sY%(bqE|`&Y1spL?+40tSD+AZsTEe-p4)8Pgs#dr`%;HteoBm~AV~ z0SX0qr$0Ryrj3#))DbF417SZP7>*bAn}{Q63%zF{wvb)zB?Q~wYYe!*>J6ZL{50eT z^M-Qlk~clOIt?ds&csySw!PJ&(WPRVod{;{*}!G|*x-xsT5UXm@9iI0RVBHNlmVbo z@KYteeD_MsI@HuGx68356D3^T_zkWS`>*S9KMB`+Oruh;uRg-&))kSY_x9efyWqF9 zcA&BF`OLMuSDku_E_M^MB=GjA@t5d)Nj7@aG(zs48_j*)5>=H^x1N*N|NN?kz3x4o zCG_51NNrTH^jLSFE2ukjw#u0{ivh^x#EanFr5EF3O}D;s#Ed0j$!uqc;50clV>`~S51`~gxvz6TUoF2MMT;|sUPXB9REP8&VBL{Q4r zFu+PQcnNju!i6Xr-2=6Y?TJt5N#+|)q%}h%1awuThd2WR%DeKE5pGO3mX5Fv99i=nF4O|?6~PA`cA(Dr197Tvbo3gN#Jga)Yn|nS2$TB>;qn*b zLHL2PXM12Y{hs#B{QSP}>mTI#52*Y%^86E>xnZ2x!un2zU4#A$Zc?$ysng(-cp(7A zDz@?{f{1=Cd|oZtO~eNGU@&U72p3pA1tS_nZ0Kx1_a2`+oFFFkoEcQV!T>|TBWDl* ze}M_kPTQ-}g>*A#mPY4U-bNl4_uj9M4=@9q7+xn0<%1sN6kwUESz{0c85LPx*86b7 zn%hXLIKx7`;MgO{hv9o4bt(Gn69oVO(n|+4@s{xjZ4IV#%h)0s1acLP*XapoR6a3x zk$4^zC{0qCi;6w%RZz!w;_hOiPreH!Y==vppNFE$xonz9F{-crHuq7Yo)5+(zRTcM zD=WwDZTai6=GhorI0JT=U9?x4ft#1xi6@XZ>Uz$3xE=@0O27%bf*p1RM+$(=7J@UH z-BN4bqv@I@l@g;Q*-#Rz$_cwTcIX{2(ZSAiGoG>Oe*VolU+Wqk*^$P$-`ROkbA@x!j!24GdV3KvYO}!u=aA%3L-!8 zw2>-_{UN+1h`S@gBzL4#NzwtsKhM;Eo}B-Frv6DAdaJMVaD2bQ&Y{v0EZ5@c4cjqRk`sj^$!KYSmi zd;2e<5N2K@SyIcU&N>0f&s#V>*a42LhgVbGr@7w0vv)sdxL$4uzSSKZ|5gRXyVyt| z9rc91^gu^26l!ip^!}oB`*1_*z$~L4KrwOfW{!!UIIKIGj5s4oe^n$`S7iatqtRG} zMxVFPL{2{+Ep=P%0q&0aT|giN^aE@JR*w}9f1Y!UE5EJw*1dVF&y9!Lvf*4q?BKTL z<{j@1)6mp~*Yt7j+SEOq9ZMapu>STTf~Vs&w~uQ=*SuK(klm_l$YfYt16U-#=uT4YR%^RObS6Kqlw@Kq)kTLHy@q?)F^;zathSy!EsgH1 z%y^3p!N8E-zSFFNEwu;8gFc;QrEWVDHWPL;$(br9YP#z)^B618{PTBI_)wI2man*A z)JAaNk=Qc4N(u{o^%Hl;Uvr&_T&$0W2GEZhsa4y+D$rzhlzhatvFV#Re1=iX|5*y+J?P5E(ESl@&-ywb!0OTG0j+ zH6X4|Q5^0PCVf!Vw#L$Vdqfte74#x;CzeXRlNsIOcWmHdpLF2U++#}g;@~JS#$+@- z7)WCK&rS?;Jt-;QyHG)H2?;Mn_FrQ-)NBtOHWMn~wMWScMHYK;g9%eA=%x`ruO+ib zF3NxQvQ55@E}Q3H^8r;3t^-GPa~5CUO9CVwToK8OhD{aYufUjBPMIyr4W|-QJ7rQ3 zf{ZUrC<=p7^|=@pl1(U}94y> zRq8JPsb1M_>bNmC#+rZBxZL3VLPKE90)c47TT6GZ^X~`(Ke63HTKkfh0!#I)+yX$w zc`$-brG7S>e+45Wcs5z{K9Of?VR*~ZRj=U;uOA%m0ejS}OB@#EC)9c{+9Uf?n68c&r zT>84K-;})oK!3Uqjn6*~qyBJ*#ls&pd+(4ETzOr0_{gh6vadx`WBlNSW7ijt$8kVz z)|v4F2nf?=cBHXuJc#=ENDSVzZ3EG7*_Ycg9fkHGL(w1Jo_b~D>4~*pyOX_YHY5oB z^NjD;ou>c7+?*GNq-K#^&Qt<_=RiQ<6MuShx(`GHfM35O>`95RKeWTgZ&M8>Z_ z;HkPX9Y;~i&^*Su$i@feAv^H_^CC-X@Y6`>oN#~ zVA!|^`A`^$17#LU=;3E0->X3o+cH#L{N0A4#driC%EpGSCS+#@8T!7(Z(N|at3YI6 zUsm-D@a(YO^jQX8e>SJ;M0J-SlIuk;gd3`#LxCD)Bdcnr$q%Nk)7r*U1M2di8rM%DK-bBuc(Z>nA2NH#O zN+K%-9Tc>U>Km>pqBn`|tR7ZpveW2AzJV(TFu4;hRDy5)%|v%asNlPoFet6Kc5^S| z^UJYOH}>%uP9S%krmTz&$!z{4#M0rj_vOp~g}rhS&b3o}rDBXTG4}FcVAWn-oU6mRP6517QXarl|GBPsf26)!4 zM4A{%9MEghfABBorRV7i^Phqw=BD1UKrv7d-AITkHcyBv1Z;7`3@r}l;Ln8uf+L&u zjxsqu{J?OKCF-7&e~y*;kz;gj-o%j_0X374g$#2E7i-MQB{*_F?VmDZ&rudegqh*% z^)r}!AlFL0rbK^Rewh8DcA;qVG9^?a48mOui7>`r`9291oq~&XBXm`5Cl{K%{RWzFc`5VeD6~Pa&+TeN3@qs zl42A$6d>&$p-T{q-S^jGS-=dO>5RW$Tcdz$N0L+BmgkM9=!{ASr!vvhBJ*bi0LmtN ze8kE%ED_RD*-3Gr*YpG8E9I^v`nyS|^9pZ*2@6D}`Q9*Fa-AH82weg&Knl}XCqXQX zo>(_FKyA}d$C5H0G&LwxFM%GFF+mJ%*hL;i<{^x}&fBV@Q5Kr2XfbJoHIpI|8|^Go zLz^0fb_-O%%DJ+@k)P8Wu8|-rd+=zsYrxtQl&u^rFwuViAxj3 zTc1goYuKC6-K5Pqj2!?#G2ep|888HnbTRsUxOO*y>P@)pQ01-$-V}^X?VgLMfBS?z z_mK5-#Y2JPIa?H_fm5#vZH-xYfK;bp3K^(d^YAp8ALOy}7`T0L)jlBV)BU>tOtPa0ZnBv8s#zIjg#fB-iYKze$X|O|d`EqO9M|br%K7Q!Vlu+exo_G_< z`Uff%5!kE;9dUDj4E7{Dvm$TpChYTQzen>%(%EBZ```2B-a?nhx~B)y=dneP5@o5Y*M_NJLb+1HMB4IGtffdTeqFKsZho1KwAk`J-#o{Q}) z;`xgrn19L~Qt>Swbo+~*64|O7Y0@^B3HDi5IvrQUDke|?2};_|w$i4}>ITcnFfOLN zh_EFn`4)Cc&r!?~&8{AmAF{cNm<%YQtv|CaE&g7zDsmt=U+0#Lh}D)ZuP#ZvSoN+h z=J0pCv)WT^^Il|r6p6ZZ+$@P6l*4pehgz{0kLs~&x+q80u8336OGnv8f>E)+^(!&n z;DuYctdHdYx@y~t%bUj+MKDUU$6>B_(0lC$l>lM2%gU$LRkAGa=J#JqJP^;W&f%jZ zH`rJv)yTw8PCAPQ3v9(bCH351zS9{s_>b--aU=z2Wk^hEO1wn8KKrGr_qNOs@nZGH@O zwKLaje#XSg%E{8*-Dirlrg!ZFBm#7vPRG7nwyU$Rj+aTSs$p~nG5h=5#r0k_!6qI5(Fc+|NWViL50Zb-JfB16V~tlBF6j zjdJt4VCXp)BjRJgE5b}qZQnZ4)08ovMwB8g-LF*a~E7G*KsS zVPAq^)qA}Q<|I6!_#Sx9)QoF7%+2ruh=$#WVAUCJs4*r@lvsU*j*|q0L-fz~6wX)x zUvvn34yK@9*dTZG@}NVk!8Btm&d5D?eJa|oQ>dan?i}s}N(@E3B6q*<#SA>rI$^e; zV8R+h%+P;KW9VxrJ2fbZyG=?ZU;bts+3o>1{1s~WObR|Ng^Y_*43@6LN-Ka)OOmf8 zwATv((uvV+13%r@I%or0HHK_G5Zj6XWYr;X7{j*iAx`r~Y~B&ujncFUk2;7i)Pjei zcSY;r3G?U?FSR0tYI@|hy-Aen@Cu*VlbA$G=*3gQoI7&pp1ng30>K(l2nFxccU;5A z|5oF6#GBuwlp`kX#w>A{(3BKX8CXX)LOysK13g>FVFaCVuZN6vu#7IY+O z!^Rs2HFgy|e%J+wdD;B>9`Ga*;Q|nGcw!jGK7WY*baDhOMuje1doo}GVh>E3m=4As zba_BjliyMn^Y-uq7C4k*$qK7;BR1!9e4v(;7R&6{>p!46gEy`p9AIJ#R1W<%U9J-} z@%q!{kcSQLmWO2lU`robd@ZX0ljQ~e#BM9V9%QiSA)yxnE`Am+*?8sdvSt6(w6#uMQ%S0%$>U@@)_7ey zH4spd?=M@U5kTo6;Q7AU`p=?^S?TZJm4T=AzXN}7hbH2KOA~dx`h}0ada5S0x&1a{%Vsw?958)lBSKjajW z!wgKk|A(x1fR3~4`iC3aw$<2fY};vU+tZ-2HL=mynW(XC+l|pC4VwO^ckkzY-uL^~ znl)?Bb#R@v=KS{AXLBXq%-YM~Mz2RT!jdS1yGNB=R*9l&BB1EF`-w_w4d9R~_9M3g z5`+({|6E*|*?rqex?Is4_3aHhi`5+@CfdRC?K6*7z~a2S@l+%SC1=_Rw2v6CB&qlzez2 zIz)U-K%1ai3-{V<=QMF>2@r!cW$ZtYX0YwZI|0Ig%93e1)j7i2O$E=#{G`f~skGG( z3@n{s1*o*63k78xZ1~PIhgw!VigpX5WoB`Nv~Qt(3EnhtkT^D_gfwo66_5{pi-aPN zpfQ}xDRQGAN)Y9`oH+qvw%I+@<(RI%#Sei(UZe-+XdVyoWtltV0hd8*9MIfX4(AlJ zRViPmhoaTK`dQO6ZUf!!}ItlX6_V zn<`(^c{+Zx(D-e~p2G*tcn~l6LvE(MTSD2-A$rI?c0ZVrt>BwqmEN@rq2I6dJ$-x$ zUg4xWBr56=l0^r7OF$&&zT-FHdJ;_WQ>t}RhrE7-B0DVzx;Ygg`A#VA8#+Hps=@Mw zPWVz88E=d-kw%4OuemobE4EwIfscjN57Hamh34EsKQo;U?5HW$tUa#8el=i)Q3sY` zQ2B*3(T-k`?+^OB#COrk5f(+f3eiSdgU#b%0{p5+5VF0sWk8~Fb@p$wH;x)3$ z2!`Z4 z@UQgnKTfdSrd-(RPfP!CTm3KG{x@L%0I~NgfZQ6a?e7LB*5UX^{GlN55>SDxAaE+WSuQ*{ z;3qFZc%VT(H!{EgY;yqD;!dq)j=@%C(ZVP-V;zOuhG8!WwwXVJ!o%7Lx3s|nZgmL8 z+~~aG(H8Z=9W-6j68?_4om54HHzV_6v7UZnabI)p5dlE3eO{wZ#@Cvf!d4&k>=iZU z{)~#vZndi&y*!Mf3rmZv$boOLIkoPVO`+ozFoC6hr3>(i^UrgsnGgD%>Th2QFW*f! z)S6=@JDP=MMTd_V!QJwUdg5Uv%u!1c|E@BlN*hY;XdX>-DfWQhE!~uV#ZRxQ=#CNj z7BiU5BaK8wD1k>us(w(%DEfP9V0I^?kjr8hMl2&1@-e44k~|BVgxEqudZ{Jdn8uFl zVeT`89|-Uf4+0o}o%TA_!ALzM{q}VTDJd4R(1?2ZM~3N7z#4LHO!7t@W7oEc(v zM^2ZGL6S{wS5hNdMUO1mgS@C0ACr9wQi2Q~_E5kGvV^rDJ9{MZR@pKrkD^9WkL9s) zIc5pPR3Ut>NpD=49wwJt;@Z^Fec%+LSzg6j(lv19J#$ai-Hsm-Mj(@X)`zIp9J>_d zHvBc7+p|%|aIb6`irXJ2cQT*S?Sg-6=e~X?I8Hbxzdu(d&uUc=s$8ajruKgC)UD-@ zUq%K{Ktv{qEL}L`c(=GPRY0ae6lm+QV22i$jX9aVKEtvxtW>%F263R2Qbi5(1F2NEVSu?h&TY9EqAOxwh?jv4M*})NUf)Olho!}*yvA~qU+)` z;17-Zt$kCw9BAtze9B3o0WoM7#2L>jaBy>&jfN_qe6UK;r}Ja(c1dKg?{-rOUT$5P zSD!>c*!wZhKj`H3s-5>_``1$)SrGJWM1sL=C>El!nQWg1-w?)uPu2ke0LGqulD*$m zUyO6wCDu)7!c|+0ai7?n_yij|Y1cgu~imI ztB;4;ve>nRFD!&amY0euAFFZBlCz=!O9K(vqO_*M1Z|!eS0?^|Qi9wntDlr>F~Inc z09#`+HZs6?3O>C3rrt>kpt#4^l!UOPNYEc|qb+ZfbRjho6|2fw@)5H_LtF_#P(Y|c zxe7yqn=^@XOs^{H7IyiTgZiqu-8{L{OMPjDw0Mi>8Y@HS znw5(>JFSQQ=+_`B&MLaY7MTpJHs|Gv9&jyoKJSHleteoKp;R$DdS0i>#DhVF{UQ)!&V{ ztkG5G<2JGQL4PTu0~CLiZ~uru?49B}wVYB!e4RM66kTeEwJ)M8C4YRnvInJ;iIM++ zBW7$KSq&{72sU~{PZQkMhm*#vfM+1yX#EFP^9X9^i4)L zvLCD6N-X8C$9vbu&l_G|NzJ@L#{P@ne@A7-dj2w&8`#(AJMi{Qdt&yb)a3I95*mF*!iyPUG{dn+$Hf zSC{`IGZ)~h{c{=Q&o~$^|LAl`?6YRS+;?%cj3&YiA(m1G5Ja+b#+ukJWS%_fGQ2qT znb;(G`$6hLs=c8lt8OC!jGnD(<)1ExK#-(G!0B6<3w%}+>z9Fhc$^ck0}?5-$%+yy zbKv%)_oww5{2{L|@O^_UhJ|Q%D)nS5w|w1YjH3XhoZdRV&pYfPBBO316aq1FgXVm@ zG*3LJ`-#BbjWB^QSRt`%vF5j^xmMBa$DMgaMpyi~efdPO6gYVK>DY#=YYdixbJz+( zA*Z1|cjGS6Icyo1MWS*>tudvrJh$qkvaMfAC_e*W+Av1uuH(>rF3J7ARm|!plt1lAAEF8Cz{9J7E{U|H zz}+o0Xkg+A8WIpSi60MC-v>^B@K3**oV#Qbu;lU?SV#(i4@BQYLjg6JkVQa3|B-7z z4r+MVFwpjY$Y>j~D;U&2q!P|}BnFt-cO(C;i^G~Bhz4YPFN^@vyys?w{-;kPWbyv-5&B_Hh12nP*A|m+CTrC>BM+;B8Tr-e5(Y zFr+DrR{4uk!CBe3XZ5hj#kkyf$JczWZK^&v(a%70!bE|f8Sr9gWl1al(}HaV+Qi2mCV}?V;~eGn=@JKPM5i$p0qFdB;(7!KY$0a z)Eio4nzRqPWm>BD;~?xHDKY>lIa%ZYoUJF%PVU+TfgaWG6GzptgSU&$1^v$ZqOr1; zqh+{dqJtfMwHQ4zz2E!CmRatU&q*p)V9FARF*fJsOw7D^NvjpH)M#11=CwGfxlDn+ z7VJG&(Uj|<*k&Ym0$owuIXK!_*e2_~&j`k6Fe_)U9uW#NVc#OSFe zrkv_+GE#1+BIrkdhDdAKOnHr0FLUGF%cBF{^{r0@xO^bna>wC;) zJJ*W~UqF`~31zI?biXL;eS#WFttU_1+F>MnN*4ZoFzq>Ee@lK}Y_2dgGdt8Q%8_Ew zT|UHfLd&s_KDEN}I`52W ziFmzcj$5PL!y|R7OS;PDF6Bc+?`k()>IX6FZ8ZUgjrq{7k#~0x0IylG5HaK=G_2-X zWt!a|xPu$K6+N>L$#oApvee_*vXe5aa;q4{lB)#fc!k(|xXEgXiwfNxAH{<3!s;XH zsfH!Oa@ij+<=M*+I?aR>GlJ@VU>si_1B!^`vd-TL*cg9jBqu8r8Ki!~5y)^#Jl7pm zITR_5e>o&2-PgI>0o(|_kpsd;e8k_dXKr#{Vcv#ueFeKqHcMuuGuOjtEdqx4OJDG} zG){Y{_W0!eOP_-~WWO5~_I{KJR^BDLgYKsJ6@O*wH!l~~P_-g5xwvZ+adY#D;6z<*Fvn;{4b*wRA+2jI25tx%_M za`FeG!?S&2vq?-0d#7fGf`)=&L5a&Huu3*P*Fhjk|LdklefzpceSe?5 zT)KQ%Rhk(e7t}WcK!k|Tjm)c@4&cEA?#*nsj>YgqjPz|Jl=qfb*jFKY zz*-gqWZ9KXhGnRJkrc%-MwK#je#|EIO@+ppV`^%T`d3aZV3c#SO(%dQ^IKBN5jHUO z^q2{h^S;Sb>nxpLy{yWOl|rDWFrHz*aJ=H*NaTDiuq@u(A?x`i$$gpKMVucTZPzASo2 zNq)92a>WYTfX))Rp5?Z~fV65o8iT3<0bui&uydovKAM`!WF$=_y!V3FI#Y%gqeCof zUkE#?;ZCo;<$6CSFfe8dVOLMnII_HMQe9$tj9KM^+DZlz6X7;&oZA-@YakcMqM*$^ z_e6zAW08iNnO56sU9}F*e#8+GWzFDBR-s9qNQFcV0;YXQf@S^r@UvHN9no5Es~B;& z8r-+X*J*RXMnoSI1(&4cFNBKyhtj{!I@bMO+?j4M=UQ#rXz*hqiZJ>JsJe=6*(k3$ z){Am;B?mvlB=PP5h(JzWf}Oq^=3^X%<|@t#))yP(uqTdRBnifV}Sy9!>Y1 ziqfTT1bh~iy6p3z#73YV^(UM+H|LQuM+p8orVX3=H9H#~tnw^WY7)di)L8Wi!QPrG zWDdKChk;QDCuSurWV1gvz-6YX=^84OrEk!@(f@{!WuA)5CuWJZauIboXb7!X7%qSHoy$=J7Unq0 zs{?G}l99K6?YX!pkQ)+#3y2cLQm0?rAbHVT=qE2Mj|uG0ooN!;mM5jeOGf_?(`L{S z!ic1Ky%dCf0NNkj4w^eced`j=lLZ`lQp13&{;-%O&U8$BZIO3Q=!&Ea{z{rsJ2wl( zTTSaF_w?67AN?*RVdjv>^wE3B=S08CBo)w;EfBGUsUkyeNm3NwpBnNCvB;NA(ifhz zBec$uR(Hn@>rKt{LFe^=)I3HNKeidBV}EN;lrlpzpAL7`7%z_cly>ozHk&DK9T!#4 zz?aL=hI1i`u5Yw1ZUc962~8q0zY0N&4R9EcN*|uS@50tdGiRz+!8oPfAx(lj%>w9P zsfg=gGceze_VDIm_!N9yPZ-{znfi163kvxcq3-lw?u{58gS-_BXu^8SB#4}(xVM%C zmv|&=+aHO3REftlMUBVPgC^XbG6~AP8jLaGJYivn>fH^oa$awzD zHSsIU4jWLKLpSG3f4~$Ugi=wGK{lV3_xGWYC~PgZi4{uX04VE zD7CE(G80=wTyVKEei#g}ABVdU$VV=2a>zpJD>;yalR}mHS^W(W4ofWGNX#yv%#cOr z)TPuUOC?2ZQ-^SK4#&Z~yYiO#3k!(%B*P8InDfZ4eK-|uyv0FCv0b?qFXwezN>$4} zl!14jQ2Lr@10fJ8U@OVT8#oLo*yaL6P;@G`%Q;y|6qh)&A8Bi?Yqg1sh**qAXa(w= zojCezrEWABs3uV?bV&b({V25F%AIjT12uwY*05l59LN1Z;))F%gQegy@1j=4VfhQ* znD@|0<0>g3>yEvBCGkaZjhU<*_Yg}+n>j%JIzP>|r^j~V+ZyY1?M5Xa_~mLwyvh%| zSy|zJ8qUC5vQy$C<(auxJX}-KZJL)%t8~!@)wh8u?d1lNeFzd#u3MunvwiqH%t;b0 zQ?^@6Aq4qlLRj zhJ;Gi0BKUrqb9qwncEb=)q`(e4n{eW1}b`*F5DW!oI;14&5))~D0Q(x9*#JsgS4z2*W%UqhI@#CiWOxfq!1#Eqdh0{j;8tiKGN zdo!{!F$lXIk-{MWyK@_D~@E^xT=jl6CX$s z%t@bw=aFT0&$z}C;0}w*T_(q9cA(BvB_4~ABLyozWrFH2C?}h}Mu?)KfEGrcT6+g4 zePiIA5XwZk;!}~3QgnqshlJb@caKxsAH56;Z=c>xm^aXC813~=pD6*8rTV(m+FiyV zTV;QTaj@w6h;#b^1}Sua&xLk}kH~MXl`b}43)W7c2iE>0i?Nk2RxGa}TK z<=V*ubA3R3{2Fa7Bn_utTEUi>G74!XhSEsAc{VXTL{Cwtp}@r$x=sft%x_UDwhv+1(+U)u+Q&<$+Idnj;_a92=*CE>U?2+TYL6n+I= z8r4{vvmGY@n&F;;^0SVI-3Ym1>KuwsY}MUrZg}&W8eB>AL@M5l$bWE;%YPM`T{b1U zw=jOFg&&XdW)n4vSbiH6sP(DZIP>+@Sgkz`?l=MZ?M2N)fGuh)9KOvD%|@ z3>O8f&z8#jORNq!neEhuL{w{hxTuaCy}lU*kUB|$=`fkvZLHqmv@MaAy{O?ql^VTu zUV#!_ofkP`-6tT~Z>6iR~ zuEP~RyNMo|I#nUQa3YOc1MFy!&B?*F9l!9e35n(RzJ27oXVh%CyTljo5)&BK`P{gr zKB_MVq;cbY6I{xA(rmHn)~?A!)3)DE%XxBW0UR47bMDlj2V~g8ISoL^J21Dq(|KQo z!~i};SsvlIe{DA|=e=edw0AiTJt+9>4wRw+9=Q3qu&hj|#isp)_nrndxPpd?Yf}{( zw8zT_^cl+em*st@TC~S&7WB6UwengzJ~IyNU*}N+Km(3jY#)X`x*D@*PJENNQaxVo z1yBo`^Wzorhqm9^9uJpi66ISt1!Vo_hcg4`jDCfHet?lZSqz1KNSBfg9B-0UV8%r}4lby(fwcOPmN*Rh~!m;Z$FDG*gGn6<_$z zmP5V~Y_-*^d|^c#b?0ouWc|fb;qW4Tz@i%dpNdD5MwFXTT|ToBbede?ZTlA(Sj($yU)NYO9dfa zcc=s0lNd!lgl{$PjI`U+e%qEu^Vq{Od?Ej3vE#U#0(-@K^C=h(=q4qUl$kZF9?lBhS87Bp3KT>o&cr-fr@xGKZX+ik_SF}#ej9pO23(IQjNfOX6}PVH z*<2N>RFOEcp`jEizrn$2C8bVKLr0e7Y?D%(44wlG3h z#FV=hoX{+AouX@ItFG?ev%p7-53l*bGsdz4&f>YnDnO@-yV8k2(RqI+HrS#3^D zp4oO@4(wVBa^r~rH>gk-yp-+SHXz`N>Ia-GfYsRtJ;4l()EZuBDA&5}p7=vd-GRFg z2HXvY-wjpX8)^jp$3XR1I*a#tsp`Z@@~EJd06G_la6nn}CEwgXoAY@7fGJBtiQlINc#l`f)sh0~lDmZHUMgm?_ex^#FE{9ID*}lu0Uwj^ zR*oo$_s8S#EYbYh?{+QM;y#ph1)P}|rNK^V-YHyd4z~bkdqm8~GwyZR#_$k<6h%s|XiPRml}R{MZ38AQYxHpYc!9v%`2LuvTKe0tWr zY#sSiSCCHvV5DSiTCNFUp9qywE-`3*o7gKfZN~AP-6{On)yiC-bp4ga##C2OSNE&0 zjh2WDG^0L1Ph`gN3q1gmkwo|78_mz_fwt&7>)q=iUlkJ^UYydFx2HEU9b#F$VGUb^ zT{I>Iz~Pz7_k40y81?aH6FEoHLbGtoD^05AzjhCGtvvp=DX6 z!Zu${rOc0twU@LoH0SgMC!V}jT6-+B*7OFnR}Rx%?J8aUa;f!m%NMfjB+8?PNpsTF zP2wibOVU?E+VW+#gA~7abU7K6@?31c<&y;Ww5-xkA7C)9KEbK)a7o$~x3`#88n?Bg zouiHdKD8RGz7uyN@Kp90{M@iV3_TX1qqsPMUkz+%%UC0)KR{2vJ+-px(5Ky#&Wtq} zirK*5c^v0@9`+!{CTr4QXj0MZU~W&!4PGCdBG&(0r;9S=*UJ8}bjWS(p_vnjwB#tn zAvDig^Q7UcZQn{V^XP=!L_E#$N=-(-C}a000MLenRroeLY(j2hwVProeX;+@7p~>| zsxx~@B^N!K0a=cx?IZ07S9+c=l9aQlg9u_$b_nD`&jaIcEB{a-1=D7Y=j^RI#l~kG z+@hj%Le%{ac~VC;t~PG&vj0qSnC@Joj*F~op=${Qy`y2Hj9$DzdPPL0<7T0 z1n{3J(hn8brCQC(!qmpX%?c&!6^m{Z@bqbA#D6o9s)Qg#%;7hky4i}p5c zT_f{5itowi6;@ajq1v=>wb4f;0DzJiL@fWzURrfJP%2ZtjrO=$RD*`{@PMtU##j6c zFgd=ed^eZ0IPRe&<0#`RGS=-!q)Win;<)Nx-;2M$K#N znO%}r_Qg5}O?xeaKk5Q!={ezSFrwFu$+Mp4nra9hjqMllui6y5B$^HmyxLaDv1qh% zk%L!#*Nsu;j0Nxcks$UnTpYoS?c@&LjFk43k}|J z3PY;ERhozfoxFakJE@L&AYe5~*q$MvsD}D=Q@ytKyI7Wbt@Y&OH_>3{W5v+%UoiKf zm)r*QS?_BC#z*$@l9wpTPG6`7C}4HTil3A4^sqmQX9i4|4FtWU8>D~sF< zNW+>tvUXR|IATgFdIpRW+Ooj=AUd2GP>Ym*+A|$u$cFv2R6V-F9s&rbN+5`Sjj0~x z-ECnSHIJa|Ny72f(LU@i90iO<2Vfb`ga}ecTNp;gOy3MH`zEzKc_Twy0vZ91c z;Wtoqa4k9&UYVPQ0C$-d?)7VSI!{alUvjG3Df#&WT>>B={QZ{`FWbLe@@H;X!ZG76 zyp0HPL}ivFE7d{$wv^s_^Z%g8jdpmGH2xdWD8Q;a>Sg+YLyqTQsOjtPf1=U<)u4hE zsQ+8H+630E;?!S9d6K{JRv4A7{5NEN?^0a&o* zA_x|!md$|&tY_ti2PqsPvOvNAX@&rGAOWip=)w`mp4Ue{h|%owf&>xiP@qWvw8()6 zB7oDjsw(^^nJXrE3?u3iKhQ7*tmK162!a58OCdvmB>eMG5^Q;)|3CHM{&WY`Um^-Z z!2cl+xeb_s7i2ub03_qK$VmMbXs%rpa6B`;K6dcj36~KCVPmdE%(KcX+=o-X1Djic zc$E)7vx!HVY?=8Mw24g~a<-bR9|KNWza@Ok?}p?H_GfSyk&{wpaMMrXj8;55?O#*& z*w2Y~%F#ItiAOM8bdjU`JV97%(^g3_GTUzFpXo-fy_sk~3gASZbnQ8OR=5usL$Neb z(D~`!5(sNP{OFtbc)@l^AyK=FIB}x>p($Eiw$!-QO=^h5_cDQc;dvOz^WAi#5}vn+P+lL+vJ%&4zITk|NjgYu@8D^sn@3fL? zt@-=C-;4DI+B%AsvPR19-q`AVF09%+ceMoe^BxK2!84dQ8M4GAhucE&f(t6XdmxlG z2L^Agnit>&!am~=48{f*w^{D;Ai`QL-JxEoSF}8?_flB!*l|z2)4KPQyuu}OmEj<^ z1-J*x#{o1t?J~^U8izP6SJ$X%0CGf8Cio5~C?xNrd`S_V(@tL1jnOTRrUM>&kluE1 zmXfsAuBPIfzt9UGx~+SC{UQ#&gV?q3G`+=s6T{{ty}@L57@-Krymj@nGj@80ShM^6 z8|#>MKW$Ah*CeBx(x~8r7-g>Zhst;0$f27`)pLixkg^oQk4 znR2PRM$UXxFm;SeC+D0`4MXQPJ*l$F&z1>wFv1U#x}~(!972L9FG}>##yVW%e8^~+ zpI=b^SZMyB&KZnkiT@%Q4xE?yA8g`c)BT2qUV0CPsZ0(T62xB;9|GwN0`<>R1}oN3 zWH78*VSuWrNMyhQ3?wWNY9JmnB>bP|dS>D@;G8B8II&_HKPC{lmkbB=9E{fu^>-d7 zDITkTr_`+^n}YrO)E5`Y4)QO_&_Gu4Z-**9^o75Z&UBKb1KFg&lP0nWQvyHo(BXrg zqE(~hkYGrCAUsjQXD8zQK z#lbd+Q5R){Q53~g4A31{>a=7pmxE%tRUz4H;j9EoMz{%R)Jgs zr;dp~APX>6s9=;+np2P)p*qlYh=hsBj<(ELTWV2c89?X1Wa^ukT_Pm!n?ZM!%SO*b zE_RBPhv$G^q-84{4xcf}NSZ^@#8aXlYkLF%Vkv z2Fz1(RGn3(nad7senzbuJMFt97iycB#l?Ct{vNSA&`VLse)a`PGiyi;83scuPPdSq>0r*D*fZRE>e+1g!A|l%MtMmfzzHH2*wy zlZr(>%GhGKi&mvR6wkn{L8|>iW7J?rE9HMbOdC}jqNdr*J*1h25?2@f`6?x4So-LO z+)&lCTXSQ3*J@!u24&%tBTPFsOYhLV@{$Kog~wf7c~&arloqE1>npa>p1G20m!{(i zu5}8PmTn=*74`kZ!_wb{5fh_WrURH~d}onCTO;{Knk;;qB>a8o(Umpm(~VyMUSKD_ z71%T?u@A-HOrc`G=n76v{;38*HJo7HGI_+h>3f*N^)>uHT&@}*Th3Y*apzsBT)-e; zz+2`gs&qjV`+^z9m!OTilx0^%RoLLRS;-3*bUG1=iJLcU2N@*r=c3N)>#@xv_Ai+* zVbnM%#hUAbwG|}vH>EW1`i-0Zbco4~U+@?SF!TG1F>C^JQoqr2^2-@IvpJ&e+pf^w z{&D{P9~i*>ds1WllXv|OLfX=-Vl~0O4LB%3v1&ZBKM;Wi_R7K_gW?nM0{#VlHXDo; zWkQdz$J z#W#@F3;ppgyjjfY-a{k&Sx9V-S^Q^Ufgg`DNNb6u>|ZGVUS?tbm&9;~A^&%(r&M&? z|8xjL>i_p$M&2;C|1uD2AG*hXpOOX8js9Kx3apn$JvhF23koR14XD&l$g!#Mn^B_94e_!$`m1p|z zOSHDRc%H#dxmysxq)+H1z}f~L6ksj9AR>tB77zO`%XFnxXB7Z11P&6&=7mlIl8b_| z{M%w&xwmTc{9UHU96ow)H>-zF=4evMG{NFz=s0w}2geZmamHmhR?l%U|e=PT`m zg5JF6CV^U5CGzgToP&8W@{9vxtzKRcid|Nq4?`nlp$?*UA1M}mN#i%Y4|1i}#XhJF z@Q|n2e7a~rX51;7Vm@QN$}IpJ^UHLE zDUc9$)LJrnQ6QAFbm9;-e_He*wc9Jzf&VlJMf z*Ke4Lb1AnOEoa1wKtjIkaYGg&?GMrXdN^b(Qm&XgDb*|sopyljRC}poW0SP^R!hww z`5IX9fsjwfsk(8}A3}(AA9z}ImU5d(mkgPk1~Jlvzc}%9-aVyA%JryqQMT+-KaMGHUd_^6 zZrSCxj4AV?chUfk07@OK6VgH2dif##`vfGdH8HV2(_?=sSbb0%=e(~SQ=g`vCQJ)y zZX?G#6Vj`nbZ;R-vKrd@R#4r3t|7hlHNhu}?QvAGLVGN;4mX+>Hi~rjBfM$t2j9cn zp_3GAY(OWu?!A%W&e4{08V%c*xys;r;bQ2B<9F%QpVJ*EL8Z*L>)><%iD=Bb=3kX-M}0lY~LvdqXe0k%dN;bF>og&a{l5 z2zzH-jt<**^V@c*)KL+-Z*?w!JsQt*alMUapyHtRJ&}%Ebd5Xt{Nl^fUYvBnAT_|Q z9Ic)Q$5;32CPml51I~Wxpsvz~x(4IA1%Zj_04N!@C?I`$IKnvduKmx`>O@EZ92 z+k5kE54hUV{D_Ql>dc;jM>yh%SCsA~laGnCRE6EDMNW*RS?*o*DR1Pi?^j7Is-cu1 ztvW(?cC7>80@1PaMV_ofvrg8G^`?h5CK+V`Kazj)AHI}Ler23gSAG;`J@7-iX&J4U9 zWDxaftj8o0*?s?rzVx%i`>|_Gyq6Ctfb^-PW!~ZB;G>4ae9PLEO#53kp{KPJ08I~4 zJhTVkWA99;MJ221%u~t1Xcnldz*g63mP`KqmwbMp*0kL6F70Jbt-z=g%!mEAk;$Jd zk0O0eucZFBGWu^(#(;{dJ-kk{gKD=SOCed_9)b4|`7wZj zCH(k+TupFXiyHcyvH?9)jA+u!FUn>qJ&Zx5NKqs?rJCVn+MCwq@}u3xjSDC4u4mRm zw4as9@JW*URU-1aP0X5z2zJ?DCv!5Nx}T4N~Hz?TT;H>$9bR>)z=G>`>cEAkAq%ghB;;O=th5vqA<>>hWMOAC$W7XHs+CzVG8<6< zKy3oF#N&#nytL!)@MLesY0GZgwK_$UNnXl{saljr^TY8dPB3jIY#E571d%ZR*tbL( zeK%l|W-%U3P-2;LvQm?rS*1!+l(9Z(y5JU(tKML8^HowG;tTu(82#T>cJOO`6X$*&gbkK3pm@&w6pnyWf# zMzW8X!G*F^`-DgtxNN81pJAkOf@Cw;8Ba|0^|nS&TN6)ACAQ(|85Hy0Upl&jOT?>H zo_s1i(Hip@BAtPknLUe^%dYkxi7w3CZK}&ZA8HzG)@p1DFv2IM3L`E^h)F0*hZ{CPma4tmKk7a=-8*-RFkeD&CN2}wELHP$0 zjN^9}h3?{c4xaZ4LDDY}`p>Ad0FRLPDJqXV(&FEOusg+1&;!|pb5xrGX>CDhpr#zX zzBFen$t+)Ffhe?5RFmZG0etbWH;u@oN7#TK3pa2IFw&w}xoG-(z1kZ!74)qff#L5M zRv+@b^`$juFtnaYXSC7Jg>kqro#LSxbqs#&2X`l4BaxDkgjIg3KyfwAnC^BCX!J(zzaUm!^{ zI5ce>j}`r2Iogq!)$LzFdvy{$4&om{Tpa|zLwN2v7kd{)7d|FV9!SO`I(3ZUDW z_^^zUnjJ<_v6}8Wdu#Y2=IpAe2!~B5i!}L$W4Is`e8#XxI!~1O^VYm#{R6=$Lh1+= z%AuYoG)p}|NDTwl3i?>u5t6_tOEfRw%!MJTcl>;rTF=Slo>sz1-vF6MBH2-)%2#lY|uOBO#kpkmpC*9fNak@ z3LDd6N}#9<$cQwQ2o&AkZS+(QZ~1xVlVN<74d?N-sSVFeHxk1OE&qr#B;O%RjTTRD z4_qN|EqQ;7`2F8-tbhJ}@Rn))6V~`&HS<3V+;$or_TTMtOU1g2_>ay99afU%34FVH z1sXFU3jnzO=333w*l=19MHfy9>+m#kqflx~hlIcc?N&znAc)JVp+{mbLH2D(Xy%X4_v(RH#C4lwrsQWR%haA$8Lgfpw z&c_n<=K)z=J_%jHq^sIYa)*55$j?&SY`772l&}Yy35!Hex)m#7eoepY;+5@Zg5xav z_XiUQ*H!TKevu{C7mS$BubKg#%U3LquOm1X(co3G@6mHgx+CW?-S|#SV62r|rBFA<;f;5V&s^uCNWB@V^TBdx*!^!ZYjMs_CUQ&aW{Wb41H606nZV1XiD^L{gR z3jnI^T{%v7yyl$fdho`pdh#Zyx{K`hzaXz3wxpwAyJ7%8F>!K*;hrE7i|zP+jeU;2 zp*U!dj(&kyQqet8eTg;T?-DxpD2pU0H*K6^`3`0IJm$8EY-<{79$e#9pJ__Zw9vou z4}bcfZ2>;q|L-=cJ5n>N0$+Qu@Gbq1N!s$J|E$n`kDfn`~0=j<3 z;WLiD5L+_s_(K0}RiC+nu;VO)cIEOAX@PH%$(Rk5F>89tMd01ADu9`k-Z|AmY3?y) zh7r!NrD>=YJ3GgCaS>(2OVN0d$)MW(^8V|fAyfX8HG#ObH(~*hVs6U`;wQoI^kOpl zByy6nJsSDwE~qLCM>)yz5{d7g6lZV7w!`X#Ae}O|2X`xp{T)9%SvbBQrI~P~DMBE= z=_!slKCa*~Qn}n{JwWZG%*;Mq+2!l7<23qP0_}`+azER;P~;Eq_jO3g?^rlwqbuEF z6y<6>^mfx@859)~T{2CNi7_AyE#d97*IYsFB)$v>gjR5Dq3vby9?|jEEkq>DTU|!n zaUl89;UbUkSP1SWGq>Oj^9mU+s<(ep0uYK!J?{%)!)Kom(z7kl?TngEJ4{4Bz zrIM1+vJ;H1K!9r<>SfcUcAUu>K;$JF9o(^%`5{sfYH6VM5t!;{&Ya7`-lAS#eR&>D z7r3{1PTm|tL;*-c8PR+SKbF^9S;{iCeGZD+&@xXD+nyh56IW_ra#G8GKHcY-edNT( ze9Y?p`r#REAF>dQVSGWKT%GBSpl}s0mb8<NL0Gy(KZ$BDxOyn-#077ynX&&IVESrq|O3Hi3yf?~EGD;NtSEMPSYjk>jICS)4tvyRQt3ZS&uUT6d3g|*f z-&YNdw9{Rrb(1)1n64IKa!IUb=g;x}-+ll8g^vr^Svd~s9SHh?=kP!3Fm%|qe>HIs z3HZwYF&`))Q~wnMOiIsg8!+Cz3!(ob24DkSUJ$W?u!rcVfH#ZtdM3e!+zB~ZNtTV| z1ZvSwgZMeCa*6@a6YQRR$=2V7aFb|s{FKe+T~ez=8D_=#J3NVsEsWL(G3!YhVZE?h z(^0%Un+-M^s_eZzbU^7Go7ca;TI;1-jI1Ti(D_~e{{4IV`or(H^@zGWxO%-j-0CP= z!u<}@Se{t?Ie@MNC{I3?dcd!^@hN5f;jtav?#!X)FkeH&VdwQhs@=eov(JGy1FbG zFC_PC7KQ3Ll0`RdPiFIICBowldwI&e-zg8P0Kl%&bzhuZ@*zi-&~I>gG0(Sxe59Du ztCbGZBcm;93)n1mK|oRlkEF3a>k@lY#-3pF|G0X`AWg!iOSo;@J>Ao`ZQHhOPV;Wt zwrv~Jp0;h<_V)8^eBZmfKPsvs>Zs6Ctb|~f9=w}tjgmJ9hd@Q?K^muJ@J4>Z2 z37->5vnm;r0cSW@e<0>WtSYUK3^$}g0OS(tV_YMj2bQ!OEhqF1=RQ^^RVZwNB;1#Yg8`m|K;l|kXgZJ zq)lEcS@1oj3IE&Q68j_A6%>;gF_-Fc-zjGsw$;ps*rQ|Ty%L+*buut1^{1pj$e2N@ zH8b`M$IDa3UXvYBBY<}7Nc<2;P!l0@ikrL|5K!!aw&K=Z+%Ful56op&lLtd0{r=?t z2spJpQ(kH+AiC6GzJEbJldfUEw9O&!axmg3Jt|J_WiP=FMHwX;_9uvC=-Dus(?eU1 zl`{|7)Ut|O6|BqeFynh&f&~4oSL84lYPCR7%Q39Qu8!io??xYEyb}7$Y7vs$yR4P3 zdp3qMZQNHcx$6Q%zZtv{wY_5)rx@2Az^a#c~ zk0o=aRm6+DWVETplYph9m|-bnojT!*Ax}Ba%UgUGrOcjVDiwY{0$1XZEW55E2YBWD z=)%k$I8L{&PAGi2-Nj5k=e&^)z`g&+O4R+C-H^KPLr-sq@Is{?1jUPUjIJx>um}i(Rg< z-iTL%3{|p|;^l@(E7B37_T?s^7_jQNwkDy`7iTz`PI*hCr@EI_2z!9$#NHz$`si|P zZCaZ4!##b2qetKG@j-SR3#&aCy^Vo}7>i{an(?mOaTiy4)Z)EX>)hh~LbXLPe7z&P z7LtL5!zSkQ!fTr8b;CLwb7=X#N2ODRF%sIULg}>geA>R`DF3FQkWBu01gN#J-^UL= zLX_mReC!VxS<}vZws|DvSIk`~&rH(O-RG|;{jhLqRrcczoHyQ!7ldsAlbRC_Tv3lww0^rJC- z$I(ni?MXgD3vR>s8I4|I5tJ-?8OyWt8C*v|@7Gj$Xp4_*F}=)Cy!+t&$GE7eENSBX^fK=GHgbg&#uz0K?)nd*-~o3r+8r z8kjDo4^rWNi6-)xvr+tvF?vAp({-PIr4g4qg-H`DpmHsv*BE))_$1S;@qC-Q(>FKY zw!DTmhbf6qS&b*jB3`2MY(^Q4JQsH!WoXi=VE%rrpwvQ2`6L%30})lJ92E=YyR!Mcj#fWguCGxo8l>FgP8w zR2v3t+>F~TGLbSQtAcEq+>R@xZBRIy^f5o^Y#-zJKJJqba4Y3l&oMYcm<+cp1KO-8 z4XRbe$;rSp2*g+gOpH5xB#=y7IXyTRE3Hp+lU}&GSmPVy<*i2`5j>;OW zFER~zvmwS5kc~4Vx&%%%_VWA@$m;J2{u#xqCka2oBY2_8Y8aM7bzpidm zqyXO-jnt?I`Uhu#(F>~kqu{M9JHu*9Y_tEXBuy=M%rD^b6=E#1v*d1+ZuK%bdyed; zlzw3^eB&!__c)%H^Er2Z|Jq_~CiIDnvxOvD(HSl!rs|Ht`fobl`eCx%s_vVlX6L2y=YN7~`-NIeSBbuj z;@wU%cGIu- z!fWo6!y9zZcZ?<7OEKNNK9EAZo3`1VLVWF@iK0$IS6Rg=6O`vr1iD5 z1m?z-X-mp&Ny>?+e?d}$U}ysVf$DJ91)UdTpG(zfvb^-*Q8PcOJ9guIJ-jK^Xo|jE zsisDf@t9ljr5XxD=%9}+vZ5~MGAskv1eVFxp10LZbJ*{A{yq(umL>Z-tjL$h1?bDg zL^^=C0!zS3xa!9~a!pG&sM?Pox<^bSsfc!c%x!2+${c6T;s5 zc_-N0adfS3!rhCv`FTh1)A_oP=89SfQkPR}si)@xN0QI5g&@rBUf(-l=7<>fMMm(V zc8(W4hlQ^~i-T9%#CK@5)Vmb^1%Q=7b%iH5pPW%|bI0T?gP6}`PC5q{YoS_m!o0qN z0dhqsbN1dW6G7sWQNHCa8g6S@_`zZ13!jrv$OFr#@1G$y!h}$NXzmJ0s9?az9mYBA zvP*ggx&8uMkTnqFkjkjUPH}Lj@twDhYGiye@o<<@&$DFmunCt(gxjtA0lKBcrS=5s z;M{42Oz;9%y90+re~H#{?AJV3QLskDK~Xv0V8lR=9lGrC z2))4FHMrfuOL?B{=A8p;cY6Mgso(hRH8BWBYtLob)nAzP$d)@whv4`;tv1oJ&(!+4 zhaYpU-Frwqg6jm>Gu?VeLLaOTKI518UuQ5XK%^g-5$&%;dj%EU1JZioa{mUlyn5MA zyMqVp|2a@e>1>~~>Aa!jVH>#AwQR0v6gL-HMl{oSW!79E94WP>{{B|@j+cFg7UByE zwbi;_B|tY~J1H|A^8}XTRJAUtC4YU#t023P*)io|2+{w8F>djEJeA)>|F4?+1>l0G z&p}#xXIENn`)W!t3HbBRD68e1iCLpWKV*(Lh)D!$hPe;8H!jhakuPsr&WN+PVh@e* zfMipmkS!c`eaKd8BN7CJ(R6t|STKj=2@`8s4aM7U0lgVLVw>^f?3}{3bh$&WfP*7M zzw`Fs;JGwgl~GLw^TU;lu6PtZ+{*b{V+vrZSN7gsizIRs8uhXJy zn=5jg)*z~;;IrooP6~9R*B%bwkg%%U?erciC)e{a2RBe9&`7&VHvEr}87|of;ppEV z2H|0{C+IlRB!DBOsZL060}QnZt6C(XwF#}23BFY^OlA?zq*CA}ihzWbVwy zsHg*N(}ThTFdG9M03Nv()4!W37edIzOA@4m^fLl#{?u)uvtRyPx1o4t71p6=198>>F)0Rb1Zf4V}2=g~~Le2+-54B>3Rm(CiuGB?@Pr(;1P|FqM#``i4!j zh`t-^pez!HF}$uWm_c>g*x({Rf0rBA-FKSC*5t-lOPwcDE7ZDDo#bCbYT33MAVa=x@1W$ktH!qRnU&B#h_>5Qux|y2q*mHpFo^xT1pszS z!Wf2DAGHG&v*(64;Dc2R)w(Y#9==(|0f^8gRG1dE+WJHIE-`^)19Q-LG*2pj+a3aO zzV#CEG@a~W-TTWbDc;e3}7gG(3j`PY8YJ>A-CpX)>R{rUYy2y|Ru z@jE6Grzh4}NF3~57;6L(hwe0NR~#Oe3x>j~t;$ewKpv;&1Pd@c#Nn%UugaRV;2<07 zf6kS1BaO$-va#S0X*kpL8i@(_1K_?W=d`vr#?!BM%~CMHsaC;h2K-jkVL}JgKB?$5Gge&uvkIZ$Bm0A7d?!dk9Lf{$bZh*nPQ$ zF8e_B%A{f?5ZU_s7Av?9HUdQ%GG#-P!*_vAlMHA;s=3ZP=a$QWckp$@xP%yB@6A<9 zOgpiuXfos9+ER=1(5PWzyqRlVa5%ePLQ~Q`z+YAP=>Z%wb0|}2RQt+(A2XC+Ajlju z4d0Y&4ohseooU5kuHT%M7m5$Ls48p>Kt{^oJH{+_5K^!a+!_}hmjCi@n5UhHXYM-0 zaxqOjaMy{4t+jU+sAr~h)A58TQ^y2IO0t`J21v)l$dwX!U8#+!6e&Xx!H$kh{Y;`6Gz#7dJ;7_F!8FkYp>(aHT^#1y{w{f@s?1ggs@^ryYnstzUTE zG`iT5y+?>TKFX&d81VA+dL_NC(p1h?eK-U7#)D*4Q1xnI&9irH=)Z`R9q@x|UI?%| z_8WO`yn5JGo8+tKl2Wpt*OzwP9PEN^7V6=xD*<+}y?*1~kt(@YxB0N-LvTK&`+0hW z`a-V!lag;m;tqI>}z)q1i=Wm|3WDV+A|yfdGqB9!|N}3pv*uuX6LAXcxS|qAXdn4ALQNqb%*r z{{RqZfYKxwBffWXh(dcLYr#Uwt^OJAliIOP1~qrau8zR?K?_(&RSnosEr|m+hZkBy zU}KSefnO#;B%tR1n504Gyg*_S*hB3Zihkkb2GZLl4}oGA@VqQxXB^Rh~gWR9sL5LpCAc z!9jLq93K1yk%%nGK+2rV_$hpmOK&+2O6qj6KkHVeEe4yWiowSsrgwu=zFGdrh=dc1^KM1Z+ z6&t`IW%0RjOHm$lO~aBaEh`aBraY=JE)6quVB>`zz{I;;^h`+6xIaPsByjF!rS=KacKnsQ&kY6ihPAk67ip=L^ z&1VWx;eY&QZAUTrO=5XUyK3MNqm-LfhXkOYbz|kct|0ftk{W4|V~JlerXXlVDpG1+ zuMbp_#PU>YKIiq?Suz%P#I1FjaMamPGsmKCiBCN+{Pic0ssLbj z`j;PSxcAF3+`stql@aN^Z%EIzOfC?Y~E))6YQ0AT$DWKS}EdQ|ixw zLjo6B*vJtY!!d^1$s7~K5cve$ZOm^`2{(c?A{t{DgDeMa{3~=;q)Ax5QM=>=x3wfg$rpa9_WED?kWU1EJ=scyv)5#C;`3Bx$+aRXNYGjPL7fo`= z4~uLwNn|Tc9mlBAXtT;6R#5o-RZgqfHq6jIC#^EU`*>hVrF1Js<}J*?5nGJrNPPg1 zBt{8=y!_=bq*vaE=D7OWvw* zI7aIWTd4`56n#{JfE^!t1>q}#jSL5X!g#FDb)!zfeMVrukH?Nn<1g{2pCZYG!Yp(f zQEKMZ{Z`5i+T-cp1k|NPL$)79q@bDd{ORXH$ynHK#3(A zRjyn~h>qd@SaLB0!wd}B9@BB) z45!h+atSJr>IIfP^FN6e2acSp=&jU`d~+XJu0IpEW~I zzzy|E6Z*a=`+D)Xkn75I0!qGwSCw_diRuA43eHw<3%s6e?dRklzukRmJnfupgVw$w zPx#k%@gNjW!c*s5ybh(VX%m6rFLLfa41TFzH_v?LY5PTEYcr-MX0Xp*>Nhe81C5|eO z4fG{O|K_)vMy+z{{O*q9h|3~RXTBt*2lbWigp)kxX1cb%z*}wP_)9uqG@&5VB?c$f z>7VJM*JZ$-qFuBZ!`#+*6HrhUrBszTv{P+$N$q5A?9@c(B=hHLPmf03k-r5eg;NW+ zz9;7VAT^dbqmMX0m9k1RDe=?(KfC7t*vbEEOs5}5duXX%1{5ZmLQrv$Dy27R6$M2q zzl!}V7GT0tqZ_4K(pVeV1$x4k?}SL>UV!*u7_CFqpd`>pQLe+35w(%7w2thEgo*h-K-lBIGOQqYPkW#jh}K5FrJ4#as?BthZw#S#lr zuFGP#Nm=kZbQ6@y_jE4CxcuYYlz5VFJ9`O&O2Qy?y`>-9^UOBX3rCz@5AC04yHZN6 zGNho!!ku4kw|r|N?~FM2m59h4elR0TU_QMLOuaUwzyXM7oT7^{Ri&Su-0-O75Q-?d zx(8o@XXDvSthSBZIIKm$sd9#jW|{wC`3x`ke%S6d*1+b34rv};uvNj*+CSnx9b7s; znyQeE5Q?@FfIX6&FF5s!f&1a66Jyingm{;cgF)z~MV)AJ2>H-S5ODLi8IB5QfO`SI zR5Ho7mjHBEG#D4f8u=@S1f~(?n!sa6xdUUKdVK|LiqOTobJpyP?Xi{X;HPL5hV^MD zpJAHBh%H!TFQnb*3_PyX?19rgwNrrpZn9-QQ)aM8pxu6ZTWcx zEZ){A^p=1R9GHjzIegO@cr;O@U}zx_qXTi`@NnJb`aKKDb$_1skr$bM;#sim$Du*@ zPx+|r?o1Q|Ftfb`pY85T-ed3Q&eF?NeL_SaIHMK<_?bTvH*p7saiK&6S2qsqL1J*B zGeKTVSD{G#P`NyDj8w)l!@}5rSS%d!xt!~>kX{=v&m|KDGuw3+Wsl8!nOLH(;^V^1 znnptpmqzm`CzkbxPyES$r%epC#Zs>xmZKEHu;Lux{Fi7K6DCBNthym9<$r^JQ$y=d z--hVzc*eK8`baO|b##}|9_pBD%OAIkh1J&MZ;xFu)Tyq803G5_%OuAEte`CuZ8T@* zl|Fn+s<%(jO3tL_7`){ggK_EY~4n-5;MiWiHPu0`OJevR_7I7DYksnHbt`T;rBDh)0`nW`m?%`)8m++ zU6yddi*{UGGP0G9uxvvBtW;Z#2iN_{t{%!TQ=XE0Jr|f7Lpg1ovP-OOpl$SIPE%)4 zji=kjP9I4D zmOP}rD$(W^Ts#%Q4;_{-3)=(I^e`>DRs(Q>YNWiv*5HH;3lZdiU?S%?Z%)NfNA-?*ZjL(U8g`(tSsRHXU82(ecdJ&^x)DlwVhTkzrBu0@oG|bO$A5 zc1nIsuPP=D75@&@;z#R<-II?GY~C;=9vc+JtuvMtV=8?_8Cdm#ik>hP3*EpX zpvvOch^-c543Gr@ti+f^m4phd)EagQa44`vwPAB@x(p+ZsI5SZd!7$1L$-rsR@X>K zl_+Nf;g^DI;=F{uLMNn*q0jkUrS32F(muKF?Fju%sCi-(?g?x~_E6H>vy`6#n1{Y& z84t=Tg-{{|I6>!D``6cx zFWN$;0*fJIrWz!$L#4`d^PslM`BGqkg8ete3qPdy`rrF?!g3bVD=H9B7*pykIyMWy zK;^KRFI0vpadc8hVgBDgI$393+uJC)Ni z1;O49BK4k!JNGh=Jj;b_I`xSY+nbL5-_HG?{J8teE{edhKlO1in$3fP^b+lW1}J6UVu7)2xIZqkX=zj+ z!{Dw(j=GP{yzs71!VPZXc07>2xNyE9<96q0PgEtZK74iC$`ZtF z8=u;Ad*XgkBDX!DokMqUkRMwFl*(m0&EOMX=OqenoF9yH&`gFU7UH2M+a$xcaOPKD zB5V;I9=i4POA<}F+$1aMksjP5^k|J)n>=EgtToGRjV^P(ctQeZ1)n)NXtV8iaPeJg zZ+axf>|5?>Wt7zy7Sw9Wtu1SxYiyo9mzJBB|7i0(Ceubw%9t;{3LpNs0nVo4*6bxL z#G-&lH%|CVhf3|YQ!^RL>r6YKq;(!s)%4q?wwL-Oa=4#rp0sqtbo6GRMuGj^$ls!QYmkWT zy^|`%u=BPWq^+mHt-p*7D-Dpg^N_jR^LETvBE0L}y>iuYn%{UMPd0PlIEn=93MAy?7=- z0rGqI(V%`(2qLA9CKy~}rNoSyMYU1)GR(4WF6AZg>xAj&n~WB#ccqM~;h%xUTyOyt zY{MRHlm=*fWPt$>2TQvv{>?pg`fFmPv}?4bjech3b|vV)ZTQ9NVWrGdacnQ5-*$WJ z-U{|OxT~KZjdE4PeK0UAFDmaf_w>~AYoT5ObPuC5>4H&M2P$c#c6toa)X)cZf5jwN zH<`{TX+#TH5PyjvJk9)dG>_@vpC8O0Tbt$1t6Mcj0#ycFnv-^N78))-k3Wd{;A__< zhRn&yGd-gFuwe~@>@YKNhOiHYa$G!QWUgDe>>X-k8O0WAO_D=&a zY_ZUXu^Pu^C@#aITrQGmxLp;^*(>5se-`I^S$=Q9J=H6cgg*^^ zM}{D`mF8k{(cB12Zr31NHgr@dmOSX}CeTyYH)^pX9CjKZ8>Ea(y zzu8Ou@%`0N4Hs(E-Em1m=t^@6g*A;<%8mv*o~e|uM5z`V-i~6IJ{FQ+dGSExYp+_Z zqmmDKA}>Us_M+CIoX;JkBh8K85$Q(UiZ#zImfv~m8GME8R2!$;r2Vp4JL4ntG-C(g z-Y%Je^zIb8p0`Kuh_)MXWxnIBxWRvD>vxBFjfks-X_x-DDfGXe6LN!Xm;d+q>jBCK zH2`N~nkO5np1;F5nSL%>M>hZXl-q5q@CNjQRRFuI63572K=~hG&}q6lA;};j)raer z^|3w02NHi2n8L33i95{ozIZwrK%*!R29QXo`of6hnXadJ0{KwVAA9c`yd@O9g?3a$KP-55LP9m(}*`i9vep_t?mt}e#KR+v{GVo zD-yWbx!!tJX}Qa&Bx|s0dI}Q#YQHq?=(EuyS%|-oJsn0|syfXKjvFM$Pm$G5%sQc} zploFkHBv;^kQE0OQnx1$KwTv07R=UJ@aLpMZ0H$?5JZA+;(*{yRm{9g()CrRJ3mZ6 zAZ)xgb6qF6M@YZfxwl{Zr7Sy^@}p^GJK6=+PANF{z7^d!BvZ@t6{_Zt#s?0Dk0V`dc$vlLjYq07mK7`E)WyHltg~)U^ z3?R0b{-7DzsF=V%o_8de%VCZ_7rgW_IC<30n@mh8!Dc%MkK3bzjd54m*{i#>W@>+k zJFmw{*UKk*S-ZUfFucB{GQUBmpT6r|J(-h|;bxy$n%~{wC=04kq`tJss&-D;Ie+NY z#JE_#Sb2`q&8o4i)7@evYwNYKA(V~oY2D|WY<^hbb{Xl=>5Vdc$!*eBQ}6okq==~S z_BSg0!T{f({Ie&H+ETu?t*o&0WJiY8t4j^Ot*P^V23g4jfanm=)VNz%f(jCmL$K8WK zx0D_iKr5U@jQV=+MRip9=gc_f6^&w#Q#}|dM*VDYxw`WSS3;u9&R8Wx zWgM7yL#MWpkt2MvQf~5N2Gcg%P)X#3@YyQBKoadEp1b8JBhIu-kkck<=#QdW%Bz52 z6O323Ja$u>lO?olelbg6$}FCi+>naoy~70ff#A7No?f+~+o^@jL^Fh0`6jSpjPNJ& zTBy>@brFkg)#N1S7<1Bc%AEM?7MnJq>_bcO(=3U-wico6Q_JeRMMUf5thjQwE$0b< zO2EHI2<;>*gt!}V*;+fMG1e^a-^uUP)?70J2E-!!g(82I2U+;Ijw{08;vGc`Ag>=S z_Cg(%J0?@SQq7sgi;lUF2J`90;;ht++zk75>G|hy%8oP;HL=a^Y+$^}nd zs%!2?y>J_wdA6L0>hPt)W?)=>nMMo1@9bzNBHaRiM`IHV(|O~EHa6#Y%1+Uy+@4}V zT@m@2L_fE`admv(d5i%5$MnXW{wm=zbH#`|O3n+HwAbHlA-?~7QFIAe5ninwq(oQ4 zgv|&_nb)1Bxr9W?ZKMs+{v6`{W>*$oWu!9HaEGQUmLV^a*H~W`Ymq-ID{L% zcT=_+oAW94)o9!1d82XivlFX?^`k4sTLUSC{#)f~z(iD0Ixd4e5ttV<+xnyEstgv* zkr4JPlrGPFN3jEHkn!Pl8s|PY4>hfra3kdracuDZ+GKSkt(5SCGipz?mN2XKZ{JXJ zOa>&sHi90=z86@{ocw+494F)qAghdhnl&Tw;RvLV^A(vxlRumM5v03E;Y|6#@|dt# zbS|rc9TRs_s9V#JA-#k>6>IX8E4NrFH-9-*a+7pEW(K@J2r`;l0z7YhW-TUBU+~q1 zRJL$>K(w|*Nu=Ky36k1iRW(eB8u=Rs>ANd#)N`;$Hq6_1uxorLZ+$llu>5Oj69}^# zp7F|Vu=PFfu{-XSPo#+)Y5EO$1Y%%LQCVF46g-X2I%Aes+Eb6^Azn6J^CdCfP(hf8 z-tfiLb4cM|rzhGzi@T&HKkYJl{5c)3Wb&uVqWog@`-!kFp<_4o%zH~sO4}CBL(WWd zB5il`XA_FwdLegi;C-j*QmTwQ@?6)G7h+hW$ zbS|QQjwTY!jRx(7s0zi9bqzF)^f|?t~;#*q+@1C$yr{0fT$q|5Z#IE z-#XtA-Myl1pMTLBLB%^j!$_da|N8n?Rnsxd;LVB7Ta@Mk)qi-5AT9M;7rau66oPhQ zdxg!F;Ql?6hK-50$7}9DTsU8mlC|0Qw*Xofxr}*(;`l;@;v4?AZ3vN@h8YU#k+_2! zs)wJ82k4*#RG&UDD{K8-%z``ZOu>kYHvrBuvIOR5r_Eb#(=NfhK<*3yw?WCqJaFTY zuw1^SRP+@qxLXxulPjf}%I6`FPy7Rd?xdG>w;$^q>w#y%HSYgLzcVqAUrI{C&ohMb z=Jh~rd@Ba(*>jqZy_b-HT{1+p6ctq;xy#uL^I&=i)tMZsVD4j}8 z9pft}06_|70Q5Bg#*0cPap1<%4k7Wit-DO>}#L<~buzgBJE|D`l+Z>Xp0sAe5B z?CG#14^xhd?)8@PW5MOJPqjD2c5H%o!i9Vuu&2|D+UxDw@t`8i1{u3hj^8%Ix+#kh zl?+6;!Zq$Evm~p}(meg5?25yA)0+? zLd0F){FE8MSo{arEK+uLsFpc2eQPLb40m5<+Ae3irW?!jxHrS!yvKhKm}K=6O#*Bl z82~wK81R~j?d2Pw7w^PvN67p1VDSv!wut#RB+zS=c5{R>D6)+Gxxp7k+MQq2sjv() zZ%&?3b6#9~vv{PuTy_{%DQ2d{cdZAOiL35fCAa*cJ*;hT6AIinD1aac#DoSfpk7~0 z6|a~pMiF-Gu?szQ#%&&nY-jNWc}@yh63|JR;3o3oKU1P0wM6a`lg;;qbuE{Y+ffya zCWAJonj-XO0!DO+MO^e(SUA|Fov!GwgwWhlG)<8;QBgM196ZuoG}4?b(i|<)T(!z{ ztKg4HoJ6azgnJBC5r71JuGD3Wm`N{RbePQJo;>52H!SLzEIjItNZ40r;S1h532-r( z);%(&c;GG>ITSNE2mmY@&Lg@muW~xFRH$*D^2|aeJ8!h$_EJ6Y8y9^Iqp|i!$ z8ZgRKq!5OmdVmrGT#SHaV*q#Zq4FnsvqZ=DhisP_y-UX0t-mMdt)W{gXl6P}@ z5BQ4vs&G108JqXt9#~}J;RBKnksq|a$*qH5#R;-67kVDf8w$Tq`jEQDXXVF3SuDZ6 zW~}qRHQyVT{09Vk&*+{uP0r4$e9C<5DL3#NUH0AVv-g9F-`Ryt4ux9-K#4`U86o%Z4E>zP9YnA*dn@g3b$r}3TKW2f;Q-*cz&o!&#F z`TFPbHMloT^VPpsOY=3n2aW+SzeU9OHo2w8_%^#GB>Y?!r++&E1bEK^bH*#|1*Eu@ zX(qg)1@g(Lw)dNabRQxnCo zrgg_yDM>9!Iw4(1IrE0z!IP`iP%G-iW`g18nHJkr6(lQ{YN0X~^U0DX%lr65uT$Rl zPRcK43Y@$vt135IxTT|jSt(Bnbj)O4!U_W%z|86TZg8`Ybv$$Qv zhaQ$BdC0qXHV|P<4hc2<8tA0fi1nu>s&wJW;QZ--EF9Db#r~D!kjL- z8w{`x5`juUC2IQ_A2z|Y-XJP-(TFMiDVRYk)LW9NKq*79Ddz*oI+IkCpdX)qcgIr> z!EewSw6FC;Xn3{R>Z{bR6e#@i+TAf}Mu+FdxtCCq%nc$o=%5;2Q53~j(^Bemr}h-- zR%7%qbXFtPg;Zy0IyUxc$$=kQFFADKg(8{9LB~3cL2%TaQNWyupEya2jG>L_(VmAq z6j50%p!QR}0b2&Bu{E_Gi#8BfRSOy*A?_$ZJ`?VP2!C zJYLo}ekfx#MP77x7Zm+gF00Wex(08K0KK-`u)bk-G;|LVyAojLu%i@WR-K~nMqn); zx;b_mZDnJCt0r=t^w=FRg)_>@V&7mC%9g;U!JOc1(x9~#rd{;&FnW0fvU(;RunVi@ zZA)X$xd1>L1v1CHGC|H2(zKat;*IYz%9c(JSVFZ$y`bAKng{!u~$#4LCc|~UM2?S#bHf4EnY>;c(HN3&R zy*d(8PY?4qs2U3(>Pj9{tmA>_HEHQPN z^Z}@swpaDh&)YI4u?TWeKt%df(X@LEGrQ3_`ODNhDA9sB*yqnOlpsEetNZyHa_P!cmJ zXmP-Bc%g|`zk*Q&U3fZ=Xz4-GEQV|(HB*%uQ0yKH#Vi9U>Dq`;314m+X*lzsO>P;Q zOUZb(*i3snp3~8lgA880H+T$;d?EbY%y%l&rc4 zbAW7PJObG5F)PnV6n{?{F{-27i_peSrO@;=7PXB0AoK^4#de|2I_w;AYZ}R+QZv#+3;PTS#7S(H4*?u zC#P+B<6CBaC@H#l`#ZISsx!Izal0IOeMVC|v|BH<2qRoVTbues%82%j$I%J+1B-mJ zZ_}T@Z|ro++wRP4!M`5*is`K}4gMnPHWPyLPfsJR?D(=8+BEkFNcdpXZR+qu)9VF{ zRf(dy_|`Csgl0%wHmad6oR{VVTT26kaV#%DY=ihWHFB4cQ145#0TH+wDWk$&fnc<< z%Z)AjMTiQCZWR9}a`9d#414o6b=v?eCh~((R_uY=r#f2lxDqpBR3%h36(S4$q)Q){ zEP%{gM0BjimYwp*h#PSgY#r|;5r?RV7zIa%TV$$s#BR|CD}q9le-Ko~_mY4Du?$(k z6XeNy7D}N_kt0-Xv=?E@5v={tayZ+9;JzJ2sz+W4)9xYTgUE-JsCfQ4ZLSd8d2O~3 zd?q<&1Wq#AV$%}rQINVx8|Gshal*59+k?7Gk#Qpd!aVFg$Ap;K31N%3YO^86jG;-V zN{3*%&g^v*?blb@QuI5TbWIAcSI+WcCRKof_* zMLx9!IhNjO-EmIkP1E>8wTYUjXz=Ce;>l3v;L`y(jSi!)ao>>i@Ttsr?$}O3Xqi=3 z#farpOK16j)A&U-c4_0VC7q|b)>iRkcIXz@_-NCX)&qo}Z_*JV4JY8Ftux9@pv8X! zA|w1GJ+_$#v9cX+)fxNF>~{lsR98C;3p&S`-Lj?%{tw!<{()BP+vZY?nw5qRWGy2| z7}_shd|z2vIewvWF$~t#ez~9mG+nXSH=yE_Hbqbh4#^#(tV@q^wvkAt22?fO$MTfd zZ_lx5tNE{i=PIJg*aLTF|X`GZ6zf3Y1P4Vkxq()!M_aCY#0&ez zYYNid4UtUs?T1m{7pG5bAB}nE4(PO|vo|oE7ovUiA#T+{B$! zZ5z$L6kUBWo_W(j_+VePaZH&M8t9;%7UPC!dUww2Mh9RISa1+@=64h0UV?p#+sQS# zXIQXvKiVdgQa8*J0I!wfSzI%Xr|bxKt#{~TML%K95KQ|&U1gm+RdRog*?f@aJ(?hV zJRb~>xDL{`q?+vWgIAY2@AM*S#}@kI#^A0MuUbq4U9RF;v>M+t;n2z2S)`g4@`H)&%19;ECEc84!8Vw<;`98cZm5j^IHyEmgfBH1r- zH-4{h{b;UTPZ?(~dv|r_d4HG)zW$jWPl>IO@Bkd{m%@yV$YUnOLc)%Xc)^T`FGC$A z$JX%h9PQh3IHM&fKKIyQA4VjJi(QsZW7G}Ul?a0$?)&^48b7bG7koXz`R=-#O%5IL z#v8(3oYRX@=FVMrw9ANnH(r-MeaH9N+n(HX=6_VWyAUs)+3#dMbt`v~hA0)@Zk{GULephYuT z%5?^3{6 z4$hr;;*Pqp(3%=%meVQRf=9dgwX5EJv0m6_)0&W$q@390^mV!fFBfIXC2C%|3h;F^ zWb6+4Cg(=|W1oin^E}yA)hsdrmEG!d|BI`0fRVK8)^^WK+qP|6)3!Bj+je)gZQHhO z+qP{@yZ?UQllG9|Dj~T8G9{V#ZY%vYRmmBXj&{F-h05qdOx<#3|XQ4 zZaHojE|es0vHz$IH|`jLr3;SW$mxv_WR?bt^XS7Hh5FISry}7U>Dd0Z177iV0|4j$ zOJ%M@sheI<%zuM`Q%Cow*J@&)P);o?Zq@@LoT;xkK#y1WZ_G0+f~Vj|3e6p~uOJgS zl?dPxrrIz<@^Q~UE$HJ&?q&*NEuFJ0hFBwcXMO$|UUdgNF>)te-sde7qS#cJ1! zCuAF9m%&qzbQ^k4gV9@qcRP0a8r04OYIr+-+C})rqwn(re~T{Tcg*x(|7U)1KnPyb zj_9);`0rl4(c@_UXC#cTg`RAX+hYR9-D3Xti5{JR8kFD3(>i0v)dJ5J?*b({n9o?L z?^fV|;BElh4P4#_p8s>P$~)61NdRE5J1{1$@+;~K9{r0V`wMcn`}Q}%JLDJDT{P4q zK;T2YgiMSgMnR$kG#4`v5`2+Zsa`Hfjz3pD0P-8_mS9$5L@;o3pQA1VZZ0fW30kHo!h!2CACr|V8ea=fNC|JehJ9Dy$js(oUPx?Q$9iQ^|Z@lJ7U^% z@An2F;*5(JHh=&@Q3O9zyE`GbvtVSgeLfLWrsPyl3@K88v@kf8dOaBtV`RaU#2zci z5B1&MeIPLBES>Q_-820X`xY)w8g}U10%j^K+(tqa&KUGxoZpB%2xw1tM6*hSn9~u$ zihxtqxCq;K*mIfkW?clgHHT9m1e;RKu?I4wP}47RVWv`3PR_JY-0SG5kqtQu6crL` zIzTawL!L?fQ$w;#7^HE9YQyHIpEBhtNgxl@?U;ie${zPVWq4B-~Dot z=w9vT3F45jF|Iy$Y80&DE8Jg_fqKl;D?pfy@Xs6RWcN2HnbE1oKShlqT;(EY%4n2p zCV4Jv!ZzOq!O#oUf!bcrUfdchc z+Vf^=%FD2bxLjqRtK0NnsfIT?hP9+)S7xFZ0#&jz>u4ZB2pJ|HT$EGBL2erq$pH`T zO(Y%~muqU_Svd7~S$&oVV;gRD2`Y2tlP4z+tyL@N1A}l; zxsPFA>($INlbnRy4o;q>(QzCxJ{(NiYg3H)pTzt)($EUlDtc;b^>xL;m8a%&M{v@k za_8*XaC^9vy4L!&VSl_UIt6s1WB|;f0h7=yYx^0jBgJMNy3#XB`BZ`WOC#5%)+LKf z7YF7A?CXm4?7Cu030 zNHL*UlOx7hDIt?C^sUXynDt6vEEy8laB8?M`WlNw&`HzN5o1^CQNsb9H~>y0jq}Rq zVIn^Ni0R#835&trI9;n!&hNs9q&vEr%-cYsz9*H~`_#Ok4nKoKO85PjZ@bjgS(woR z*2OtWoB8YMtbV9{;cZ8z!DaRmj!pN7mS}O<5;9Y5#eqDrq%;J8*Q}NBY8tCf>bwmz zv#ojv!uL;j?ix&EEE@$+8-PS-!UL>iIemgjHB)jpjNGr;5b87Q=v`>m?$EM06NAk{ zQt#GOS;%9csc(s@lphJUNZRuU6KNLRf||DySP@L)sM2zYjG`07o3g7@qb;L}Do0S; zl2ZC2Dw0iKdbt-x{39gkRv25lE7+FVo>$B80J|lUn^c;4`I_-wKA>effTROL>aKvZ z5I3|uw<-P1j8eY@r|3+D3xizoQlVXXG2h*7>hyU+P4QA*O6XX?(jd@}M-N5yCQHj9 zw^9-ns$Q!w_0(+m@j^3M@gmFu0=1k;e!?2{Y2;km!#$7TjnhYlR{YrD!oCc_R7wyS0Y%%No_EpB-qCt#-NCU zaF!xxmm3AeLcIWyiztwZOj#--#SJyhy2uph9wkSi=2cZu1URK(bj5vpOx&7R!1o>$ zad&r5*_3xm5N({xWtPmXWY*a8Qdn_GJxM|5DQY6G$Vx8RY_=ze7*%vxQDgPVD)(rY zt6%EXsy*OdmK#RNlNG;i12#NlpWSZT^hKE}K21?h(W9~Hu6v8w&8#cNc1ijJe_arF zWhr>~+!&683xKLHD0NlZ!eWV#VbZ>U6>Ukw{GabC&3S`YtW-1Y*6%Qq>uSM=#`qDZGvm*4g7VI0=D9=t1HlQ$;+X1`p{4@jX6$(N zoTS{6b``QV@Bm3NK-q!My{xZ_(@u53XTavK@<-CaWMaMS}qcq(N;{lV(9 zUz+^^f|lEHOY3OrX<0x@rPvB>eH*Ec#2;<60YLT>9nGePn)|sqnrQvJzsjgVe{N(= zawhK7X2O&#w1!FaO1+&PEtmB1Wnao=k|UTliz2s{;}r5^b&C9QTnQ<_fRZX#yR86? zbTr+2?;XQ zl!I_r{7ASE`n~kexpl~(l)7D?KwzHt+tbJT>BOo^NnCBhD0t_5wDJ>;gvUK)7kwa; zQQ(h)vHy&)(A3Yy1Krw2Rl5Q~=wwE#foee# z{YgZ+GeOBmU1iUxAnoQ3a&Nd+dKabO*T$QK{&6%&9?^;Py{g z&_j|iK8+6K9F^%#V!#9Cl+Jwuzk4~{sg-LxoLOAH5a&i$Z3IfQv#AiyBYdYX<~F#U z1&S7;kXB;meMlc~o-v+i|IaApWWe5zEKB6R^~U~w3l{&B@8o4>`V#08YjUlc~wn}pC(H+y1jCy-|%2h9BNBdEP)sksWsf(Sk*%{C0=vu_zIJB9C z&H~_j^+?VZANR?^l9zRbeDB{^_wwCI7Sh;U>n_gpkyakRtbX`nyW)2=(Ew~=&!_af za7OS7;@(!Wd4*Q+W@BtBN>YP2MjPQQMluu7=Z$8?bAEPxBkEL{JKlTCx%!6RVePtU zFJj(R${m57{OS-3j)U0}X1!strt|l?S#eHbbsTUruY2plPdE{}o_@0aM%+T|4yh!g z)x`=2yp2t`aaTrwkKV5^U<0mb4NUZxuNa*-qc7inas_e*XW3p?!T8eY%lb6^xv*G+ zU+ICh+YADdRsNNS-be3DH0qqTuL?2Azr=o{R?%~;dJV@&=Wv0UJ#6(3fjtOv<(AMs zw($s`{wjRs70}*2pz>~X1xTUwHsXETcYKO}M)pq|Xms^a@^P2lod>i<+6ZE26BL%b zxvP6-4K;*2<2_iI;Qfp?sP}b{6VMLr@3y$bW=>0XgL(HBu!;nJz4*T2a=u1sSK_nd` zV8)GMCdM!Y3!^*Ok(ureTuhmTzBQU)JpJ{TInTMp$JeiaPOtuZC-WZ@+dq4pWolEa zWbAMRM?f@wuMWOEA2)v13;93=)|}FV^*C4i{x|A&qT=IiIyxIDsWz@x8u{Ty6^=>G z@P;^?0$F;(c+?=BJT--hVVb_}8Z@v3qdbTkUZd80*qlejZUlqWyvq^z&spa_ixz?F zj#AVG!XAfUca(K--JvKly_<1QIuW?=LunBn3IG*j?91Ox1Jt)(JzD9+y{zodI^}_N zd4FEN&U4=%Iti~b>46j>lzFag@wDZ#k)IQzqMTQw8*Z;kuDtoCb#(y~4%!-Me6 zKbkB?{Q(Ag2$HCvG0ih@;{veF)X@5?0vNL2Fjd*nKAw$YCdWuh#)C#mGTXtbdgbl? zEd3M?KzkY+eB(+o8;FTW~rw(8!EnJr!=)wy0;vKJ1B!tiC&sj4`13vh!!Vs{V}ZG%6+id z>UPYhl=m+p<|hQqA+U|27o0nUtPAp(31+(d^eN!Z`3mJV;e7jVQVR*D!nWOhiUDo8 z4kn%+Mz%Y-gVq=}Zj574FtcpDSAq`7T`jCzYQ>Pr#T?y(SD;&)j5J>ktXXncLL}KU zgcNBPh19+^VVL}5+&SGpV*5u@w%=Zi@&DD}?;o)>izs7Q{`~hu6f_yr4fOklJ@90C z5;h8eiiRSR3A#6LTn&LxAwL4cS(#El1;wKr6fV04u@Ds)Owt<33wZdrb=#`X#6zCT z2Lz5y##7P8LtYb8-$Lpd73X~R!s!C+LxS1Y z+fWz|1y74zqAw*NvUMe8V&`Jd!o|jvjg~dRgN2)tiiE@#r^6~28DqQt(}pdS)-WEUf||>$DuI)76Sn4?Y9QiB2L zsuwk14X2=((ykBD-qy&|&hDD#8Q@_*{*hjPsA4i+8^u_v*iDmA{GNbG6LCGG4W|W` zr*7w^8*zH;a}uL@25P19jf@h{=2K-u2w6Ty7Q=Txk3X-~nkI)!8Rp}N>Jl6#^Nw3S z#*CELvg{!ak&Wjncblf)TKEjdDB1xWr5&!f^`y!?^4Nk{w~OZKwUR+5NvP7PWRcM# zTFZ7IoM_^q6X#j^8d1Go;%8Owb(U?&YmTakZAP>$LG8H0TvtN36jyafjV<2zgoduVM@AQ~Oc_~8?JuhEAlxT_HIa*qY;p@F!T7N7_(W<}7Mo-lmYdh)L()T5Y_nnaQLB+aO5`5ym1yjsR~-6ahQMY=d5Zeg z5zml7zdt#zpyrN}G?%qnovvzzknGjnZ+Xh~QIz@wNy;O%LTj!p>yB^vAK9tYKDF|_ zTQL#OC`meb=&2pCi(FnaasfQ`L$&I84qJ<~PX5T|@=F^&{%|CF%J}-w79N&4sxwii zn13p;z)~?c?>rby#xQ8VvQ4XfeRp%GOywnKw^qJCqnIjAClteD+G5*D$lN6DRek3D7JUCf)8ebUBf52N8)wm3ojE|U zT}{i#moa14cIJunW4kqSYTeY$x6P8eCKSJV8sQb^kuLi(AdFz-5m#{Qm_w2t?!D$m zhyk=as7X2hzOoSIm@I(z4l|D1zwx@@&q^ZmLMz1ODpn4Zt4^XaLk~yBF|Np7qr)c9 z?SkS{XG%!mFAY;yzaq2{Lxafl5E;uH{5gGlD6lyVcJ|fA$f2rZb|GiR&GnR%r)i>8 z7-&Ox(yf^%rH11Yo_VwaMs@k1SAhkQb{(B45FKZ0qPZ8S0Z_o6MhK7Z(0neM>C_Oc z&F%=PdWpwSbsun>niUf59^`B5@Ms@QM=-;NzDt@dT75wSVYZ1p8~wpG$6Sq+8v^c9 z%k+LO9o)0fhYxHOnRt4X2;#iNHR~KInHr}<6O+YH@v6>=tUj<7KH%gK{@j)cn{%p( zge&p3fIlf@5u>oo8=}*22Orh5I|i9|I%&CpL8!Vx3%& zA6J+PX=&(@MjLyKQmY47<5aMcCB`H1Hq;5)9r~`hLLr>YQl=*#lAAB-B9j|||I@ML zKOyq}rlh@lLM~jsgJ;`!Nr_N$EF?Wka?TDuGC<1~Ng4T*)x)J(JShp(&#G`i!iSkg znp~Cgw*&=EK?8^x4FJ+Q-m1N3GcsM;_tl@bCw%|;M9O3U%-A-i7i<$vM^w|1mWfL?7@v6sE>v5LD^w&|khmJQ$4$Sqld@#=s?nvTc_m{e4Ixo*t zI6w~QFaC?~OdK7^&hU>e9`pftZCG7&uwO#B3bv};8sx~&QwXJUufVp*)wUqjlvro_fgo@4Ke6JJ5j@$ANm*OWtS_=9QWU~|I>^pk;yvQ2FX{^ncR6R{M zYGrS8>|$@@4lBc%G<)Ls|43$DECU#|-#bCk6>HRb*;&9qOf|-+mE+J4SD~TTID+>~ zZ`Lt!aj#XZOTxgHN>rf)2PGEDRRcj$rgP*6>s2fFfMqD@m8E<-DpQ>*@4XcrxuPy@ zK{hG(OqF!YM|Pa*WQ4gw;(-zcA1e&!>>n92(P`}=NtHAD%ur0|g*%)0P5??BzHAiR z;k0BRHl?oV=^uUZiF-)oV(NuQDUy`zR5?Zr!ua?KGJ25g5A68`9T>aIWKQFk^_LMo z%Cki;jGKR7lJ@m$1Xklho^X5lC;27B_n}DGN8DhhJ>T0b!@Ba<3>Nzj@=4~Vt2kx z3gv9j=oVv;2$O-FujfEw z;J2fKsv&TAaC7fJl9p}|gls9YQT?}@g-I`=tqFD^e5fGYXL{}<%w0jXcnV_ z5sMOTHL6~?b0&@h!KCWv&vv(9Vtdgg$bde^Ta$&bZr#LZD*b_Np`sX0=6sYJdh4X= z{uchc;Ms`$aE_tK#E}$WR#fS`#$3Oyp3U-@VtKT;pMB5?SS9fEkvr1Qa5{oJs8(Rb z^1?owJ`#n)Rhq+bp8(WatDL|_E4^CvC2s#QX9{Yq9tLmDh6e+sAy_2Tq&YX><>?%E z4uJtg2qOGUD9Ph01GjeUq(1w!aBrpOD%aUD4z$Mc^J*f^aZ`x#KLcNJ8VO)ZmG7GoQ+EM`nj|4*)-2VLiWRth2#w1cWcX z@@F!{z3mm^Pk-23+$Ke8j})IysR4Kc_9m<9 z*bY%yp4E1oW56x99<)mhutv?IHVV19m8Kh2&{FmR=hE(Kgl3)m12vd`ouq0NhPYh-%{rU|A|YbT;Zwmn$I zZc4wgOZ2UStL?R0<@T{zm<@7RRz2ivhQAxtU>t`&LCF7E{r-1s{@+_ITg)d?sc+Nq zp1}Vv0z^;7GtMqe0bdLb%Q{dMzG~PQ>ZTjt3?>)9FnRJNcvT|3O#%iX>hsMf?XHR z0quZs#4usPp0x`cgoL5Vktdf^ zGe@gZEB)IAAY-6939qZ#4{r#*Dom%roynya1TL768?8qp*sWEuf9@Ux1C6zM2WLNk zu@?+$S7cffUVpJ?3C0rQ(6T+Sqi1Sls%Oi@%+$b?F;Eks$I%ZNq=-Ic#2&rt83ct+ zYuFO9s~;qZPHWT>v1=bhgq1O8t zZPWT7kKaX?@|La~40u_|-(X6AbLO@#g=FQj&Ro8nx};g0yGoV(#k(O*{=h9h{Dtlj ztdmk$Kmehao^p&TDY-?~?~`a;I#cx*@C9izqy%}Kv!FOX_iwN4g~ohg5YJW>5b@w6 zK{ngW5zJd_H55}``8xu?s2ZCB4&>AGA3wJ&uCvEV>zgO@vIaS5aQh0SeCbYk06BQ0 z@S)aY{c5 zALCA&8#seZGKa3xvp5tJQ}Ry;zP^v17C}*B;Z8hudi-)%<%HwLjl2%M=XQ|d79ZA> z$^IEI9!rpHQs~4nyMla>s@h+Q*e#N3(do-eVQBtf&H$R|J(OI74P}GG7&bN>k zybB^;>@9f!`d9+bGXBD3SpZA*J7kl~Q0uZBBhT8)SQMmu3LG+K*+7D zQEJ#ZySQs>Ynw$M`5QR%_tu==i5eAk0{2EFH^pG-5XSlMTYG+a%@D#oOO_fhfU;kB zS4bHoEl^Jq9e0T)jJQa`o0!*yaA;#RY9P%G!E5$0z*Z;{$4#*)mdzQQH__nxSpBZz-)$sYg>AORz+cP68?_8F$oYk>H2`T5t6FL-`HK^58-2vMi@ z0VOE1zCs2$oB0KWf`#B;>W(mA<;3B|?pYyTgT^*oOO*r-+NgZ@B;xx*FYj&hWVq3I zFZYVnVwCJ$jY9j2pEkgB5G<-R*#f{7WZDXOlTzra7tt+~Eb^Q`0194NT{19h zg#69ZN)rc|9p{+v(AN?ye&0n!?y)R`7E%EjAeVXoC8xiy{MFIR^xeN@a!v+fBnY@e zD}$0Hlb=phR4SwGX+mHy7aca&9D?{l990G(8X<$Hu*W6GDV0sS!ARilg2&$*?tfL%WVOvr|@?P^7sFf=YNiVVLkg>&!=IIMub*&`>7A$%~6&%T6Ju zh$K;r8fa7<6|^E9RcW)8d>v^`3;~pRiO~}Gl=?+0XYNHx+?&h`L`V88R}?68IGcn8 zx6{Z>B=s?2I3uwYtdTZi$e*}SHI5Th$+|XOqhKnNKlDps|K01Z~$Ok->Ahr|z=gzO(iNTH1rN2BzFh)xF4D6cDO?NjrtRl}~VRUw-tr zL*p@e_Bq@I0dUNbUlTsdW=tyXpn}%T%C(@$_nd0iX}5X^dJiNca{S{e^(>*u`fzX1 zZ2DwxQ8RXy$~W92_1Ikg(sz*X{k2N!?7yMy2mAL21$j}re~DUq_1cnX70x8@@dbA2 zKL_n<*wqkP?%@j?hK+{Rg!iMw`eP9$6Ubucgb)I1y9p%*iG|;BNoOey)EKrWhuMY1 z4(JW-yVeHS$qTP@vIPI4^e1~RPszK7WuYo@^ufgEP$cO%_)&`7L=??(TY@G>8dhuk z`2$>Rp|da|`UbCso1Q}wg=dBh>;5Zoa%K9Kj5OoDR%c%3Al`FJ*Gld!4)aY{p|MNk z^7aV;lJ`E^Qd(cwR$1ocYuWy7+IFrgZfmWtEiG!=TJ-&(x}q+vv~pftZd@F*mb5Bw zUs6__Cp=zUW}&SR<|?mhou5A#vYS`d*G-EPd}e^!)4 z;)+dz5#8Q2I}cGw8!Z#Vq0E+7d~vzBxX=%f(XLoOF)J4xJn$Brq;}a9@Vb~=YC9vT zGAExl8KS8Yaj7paOma%R`2eA9E-r0tLV(7^vM48T9t4c5THi{x?Cw`ap$mOye`&iIJP;JY<(9%Yq&OP1eP`ic3*2ZH)JeP7VI zkjWq{;#An%XjA-;xxbNrYo>%s7h ztZlJWSzh+sbVvhgI;tbg8VGUM{txH2C7E;r_;>puU}HNtWGKyP$39&Zz1e=lYjifg z)vKt8AKaceslQeb9SvdRs@nT`eTwUeJbdmpa01~E_kXJWpoK@>i(}#wZea%nl%OYX z{J@M3XXGPkIe?L)pndtJ6eQUTB;8YJ>+@s>H`le!-&^FC1{WQAl*bm$x;M5(5tnzh z3A46>MbPi@r;UI&u@^ZNWyhQAkQ2((urq)iIQ;PrX5!3UQ`idpG)}zX;4xdXB8=&f>TDguuMg~{CrTeA$*+8F69i0`Cj}? zh*BNnK+;SWvHF+Ob;=D!00;D(0e&W(!vH=H>Ku__8Ttb*m|5%uy1%2$o64&?X%v6I zrq~zZ{V;we*eMlbB=hMzKc{z$jL*Z5RB6yoG6DDS2}1t`kkdoPMd&$7u{WddR;!CK z?ndSq>NiHD_KNQ8autx&qQRaFBa_;&l?S0!eu&TETOBx!oVjgfZAAVq~n=OeWXo-s$3n@(z62Sh|xT*6PYRreFthYwsZ0E%69s- zqQYxK%;MD4bL{!<`iABa(&svhquyY9Xf9|o^JjnmK+f)cVL*pZ3;|^C#z?#<{mGTQsFaNN<{wxY@Wot3Wk$`faDmD1ZLLi z;PDG96O*ZIXR&43w6>{{15?CAAuC1rMZd`yoeqEv9~ZP53{m^52N-#{{DB9VoLLM~ zc6OdKaInSr25Xq3;c-dkr~c>GpKQ|Y@@bLjnR-k*ES}&bX$zj{C3Z0#J}arSHAtu` z<^D#@78R&v)B>kVlE^XJWo;Rh&3GC)Q=mW*In$qkM-9l@453%$wd5JOiYe?uyU4sV zBr^c=f;{v{#*8cQw>V8ZM3fL%O2qLJRqAMY3RXFWtOsv{h#>{>!$u{jIWq!Gt(;v! z<~9Tv7At0LHno_|Qq%(_DCD0xHK53TIq(vayD;$}tMOre%Rd91a%ymqS$+D38j!b` zb8k?m1O$;8p^;&+3uup@M2c(;At8_b`#=CWwglvw34D5**n!)#*GS|YnUrXrsCl*| z3q9q4f4TvK_BMh`$txQ8b}{5!QIWHL1s=~UJdXtQ4O27)l+Sg~yZHBDxfWkz z^m2W~E1?Y8^q#xuJ@B}G@OFGEs#kwa7xZ~b02aFsGm90;^&nCF9o79luv@gM@%jos zy5*amxi|()c5b6nJ~aJ5;!-{kU$FqNeL-|AX@=|Y`GXT;Sq5LQrzWL-^=ksQU7)2l z@EKo(?At{!Q@^GB?%l!?zzd`yG*}PU?8Rz1L}Czi3hvyc-K(vP=sA7wEZ^k1thO7p zzvc(d0fjf5DO>0M{5$ZjUMJukpDq)IBvstBNTh=6LCLJ?^EW&$*JOOX-S&X9c-a}K z{7sHqZ%B-%B3RUQN{V^bO*x#)TE3M=n+waAaa96N`9DdQ87=qrfgdDQEc{!JGt zO_r~K$}aFv2D$3h{)Wt~5-Z$>N!h&1^Mb@TEQh9Dp%3L5olK!GfK$`A*} zLKFDf*+e5h!a3r5dr4}OX&#_S#MDBfj7D%i2}9#RFYAfK{GLz40?|s%KJlekjmgRa zsgcIN=}^4E=zPt7;!o>u`E<0kwm4Ght)(D$M>0{`t z{|uf)UN;}I`7Al&pz#_jv3aVQu&71)h3(CK)V!x>r%-YU>`KmjOL9 zR&`E2^#p4X8uMX_1t-AcAdPYG060456x#xKb%5XamVJ57^0OYHe)U)_{R2i{x9{E4 zqQ09kdIy9k3jd2H0wWr#@@SvHU~E`$I3YUN%jU(j%nbR(6fHPi_@2-L*C=heDPvMJ zf+Br!BV)`u#ux!+}ApZPcz>O?ViCQ54|Ray*sENh;OhAcfIn z?VtDGB#t8P3M1H$VeU_d%1LhtrL%eyU7KFK$C}LahA(`r40CU1%g^znrRYZE4=ac1 zW7YJJfYwWcib?O@ji=cq<8E_akMBQRojT1swRW+2)bAPyv^_7OqJk6q7e; zVei)vLjw;eAf?Fv>PG{BGKK8;@5jS^k`LFLwQy(#8`e2IU98@FnM`w}{{{fwf$2l^ zp()!?7KqfqKzJAiVijVB`Z*u+WW~}1+PoQ{N6?itw^Y`Nj*b*D7@Sje#wFReP6k%$bioAV zfwG`TgHElN7Qy(57{z;-o*T%#Yf<6#Hp<&Gr^HZAdK_71 zbg~YBL5fn>C^SmUnTk9RH55VDQhjtHM(xrOJtYzCL3BvUpJl}-d^1EEc4gH#|HgK0vJ~gV>&e` zYCzs3`w-?|hr+mV9y*V_PvX04EtAcMGK@;`u-d7IyEZG^@x4O6=9=Y|59m&`Qirm} z2at)<0>^&*rckNh{F7QGne{x2fSE746%2#GWJ=^Q?-#Nu15pC3UIMvbGPD7N3p=b6 zJi}*K25Xb-&s5ZeWSC*~+m;jASE_}XWj~7o zD-ttw#7>_$3BWH&uiyr8CWvLTh&Y%6Z0nc}^+o}#HD5l&E%eTX+y+ulfER>UWRZ(} z{|(CO0_qAXAy>&qWRk!5ByAodZvbS|FOHbO=18jb5Gzn_kYH#1A7;=Dr9LWV*qd{} zjB~{Ha|;C8`EMN}c33VUi&}9+F>T)&@_+$bx-6oqzk*e%F&5N`pCCq9PQ$^KkQlWy zA6-<0%NQ$UH~2Gyvw@^>ENNnB#ogux!~!1v$9VswyZ(0!}6% z#Qb*>VI_~XAm9NsK*|CHV)!WV)f0~+>xnfctPh})>GoO`S{sBtf$#!ToTy*||5FKqRgaDK{H zHX8*#>~0FkHOh~yi~JeJYMJjOXRMkjzdoc@p{Clr+hb`|MT#L|5;F}3)(2x&WKxwJ zt1zGHeAHu96VxV&BcH{y5FJ1#!U!HX9g{s0dSe<69cg@i9Z3vM7e|h1`JuzWoU}ZL z)(KaY5};5{8!cjBZcxtQp!_9|?9>M7Mo;aNC}9kMa1>N{SW_jwMl@SD#FEY3e1Ts2 z3|r8{u9spIW#~bWAf}op&*-Oww*ZZtb-+CYrV^_$za*-m69eRRyO$Yr4UZJwoY9Lk zkhA$aYUcYOatACdgGeE#9R@~NfT)qdM@XuezMXn!Ns<>*1dk|zkb-~fTK<1*`f z$U$EOX!GQn5?Gj5z^Vl@%H@vV_C+7yFjd>ps6kzkQKc#LRf9DNulTD~mv77UE09+P zFW;gAHk2H=K+-3Z!_n_;4yMmH`U-h&h*1(sl5!FY*g35E&-&W1253&oFelR~ zxWB*yu{}yu%(aO_&W1510)UUb_3icb>u%sMt~Jwks+jWbOYj4Y-9*F`GmHGF_|L+y zsRJXv)K=#cBo)MrCpAVJ4^Gq-%kvqgv9;=~?f9XR9t)WLPc$7Z4a@?Hou&2cgUE-U zW^G%gkqxJ;c!pE%^3SQY(pmkd%qKN9ff!vVF6ukkpN#d?S@ zXMFdAl==7F$90TjDf(`M3Px)1B|IK(zJqU7WTCZf)mO0cg1+APe6;K-1HY1ZO=0Tf>K3TN*#PNFo%O2#x128W_* zcp;4+z~RS;r&#urbAW$B#eYWhzsLF?GVk%&9Ig^P5YRdC|KRg=<2fOkagFFh;1T}0 zd~w~)aPch#qkjE7d2or~x}@iUW@=hqJ0jVOjma|pS9W$VV)#Kr{E^u@&0vQ2h3kZMn-wIO|K zV4wghQcGd-I!}tgY+plQTp&h})JBZFUX)%J<*#8hJI{j_n5(xwhbAnEo7W_{!=zk$ z^7j@@;9Ub0;$29PYJh~!kOj2lU{rpw3;Nt)4lVj?`5c;T{85(Ny5}^}xGsxrYjc0% z!fP!3d%(VVtUa6xOqK9x@2m zxp)S-o|uU?da7PWC0GAQbs%=&?=UrMv<`Kio#2hzmhNlID*z5So3PqUVm<$Sce z*;IvSMpMai8H(`q+9sq0VS#3&aW=9d5_e+Fmfpkw7YY^Cidrw#xMg2ncjRL#N$(7g z$_V`x!*dP{6i_UP30<1E<)|UDF2F+aD7z48m;4mjUKMf@GBr!!^@MkP@!n-snnOXv zQ)F=I86sK{WgfcQN3Y4fesHQ4F&k*$yGy7_Tg)Z=R}QU{r)G3bat`x!Eskcg;}?|| z-!j(zyr@Nxrw`aku~WfeAU5a^A~R-@X?R)!qnbX03V`P+&5B2XwgI0cUaWDpZ7$R_ zi{l#KU-=N3tUT@s~ z%-akRSlJNjV9+*g^w8X0{}$l>f4Tjyn#g}t`dT^Oq5%#BM2e8?tM-kLiLS>W0C;@U zV7$pzCoM_i`F;X}6O(7~&S1MO5no})N znrdk9X}JF&CMZf=gKBtM{v`TX3|{IoF|JdS7BI*n+kW&s_L%md-M$$9`q&W!s(daE zYL6)&$h1}ZKK_-4IkYv(itbgE2ZZEG3qo5%uNywLFcV?Q=qe31Gt1zX_wBn1h~#5t z4Jcw}4K)|O$e_I5L@;kBMw%#d7wd7zj2*a{b&<7c&)j%*s0JqA_@5N-W|MoUC27YV z1ygz#>?y-E?;AoF7XYNw@P=>N9)8c({HWq{7wk`W6&y5!Nw-xP#E3+#0{|ok*C8Jq zX-(R2nqsHZ-9Qp?v^v2!uf2#j^L53S^M~y62kq78T6g_idK{z;N~C>J#~yGwOLDr% z7c_1p3dg9+s2o;IWqQxinO&nRccS=shkvE&(rnugG;-%}v#yKsFd-NCg;}Xg-8E6E zCZ6OuPCazYmNL;O>PH5n0E9R*DoSkbO}|RD)oaC)^-0kAm{>jSacaTXt`gjs_|YCS{V)<-K~C7>C;;cj1jE z#;%dAu#?Cz@#Jn5QGVwGT(|0eAygHklq~U0*-c)Uyf3c15gNdH0FvS@4EQKZ#yH=k ziKvQZpAty|p3tIjSvFBu&FHYLkRlH?6>B_YFv^%L*GDf{?bX(mZed>_h8jFbf4@ym z+}@nXa2UHrwe!X#>rxUKKrKo5hZ1q?ed4sjGqgZ#{1BdE3)mpweil#+l81SbiQNB1I%>#uxjikXb zqraWfyKsNGC1;O~(>uJA(z|&7Z%fI39CPw*7qf?xW^amLQR?XSzN=T!@~&;t?Sy6M z6Hbj|#Q17?%FL~@<)2&MDDijxK|CkV5DQLFzO~PQXvcR9JPK8Aka%s1UGvk1-h$0B zmNE0#K2 z;rd=@FEMV>-@l3pQK^wc?~wX7DZ3seP;-l2!Z%EC7oe+^XE$Zu!3(ed_?9yI04R7? zX!-d&fH%;6-tyH(Ycgp{z53%`Fh~#KeA#g4?yj zkr+drGhiaO%WiA?2bHyav>V1gM>1KPKde75gKVV-rrKRTMlrq(rzES1vSQeKb3-3G zE~^8!Wd;ql33cFOfi!LfPxD%on@r8Xg~SB1C6ZFFV4a?JZn5sFW-pFfKip9JGY^)q zzmj7VWYpb}!aGp@UYV&T2S8_pm$0He_L%TD5i6PU-X%Zba;Jnh zrmyn^_MDEe zwZ?^o3JGKBZ|j$Zgcg5b55?bI=KL@#G9_(|@o!?+Tw_c2c8}Kk&LsgvCvbbT++T;k zJ^(llTptPnnYRG)Pcd?ee%$RCvCFUKHk9H4#%(74D&bV|LP){|IcY|=jyHWg$#aCZlSnvu4LdQCdRL#o5N+1igO2cb`^h-I9fqGC86w>LL|OVXzoma=>FxXuN97*g;Sish zW~$cNNkyBT?78e6@5*(#GlUAgHbDo+c?%=GHjqo{8r8}jwV-fIus`g&4M%{9sBVP_ ztX+YB3CQJ^4j|n&AV82+2nU{zDuSpuPS(*K$FR&A2P@y=#)}QsV|`zH%MSlPVAGEGBjj}O2ZA8MS#FS(T2*P$~WG5?=VO52;Oh!F>MtM zOk?m|%Ux0D|HUqxu0zZux&>VCjk{J8uQeevb+Jsi!eB|}Sz!=<@n>MjPW>%$eIV7h z{GGeOe9B-Phmzzo>HPyJNlG6HV#^cbl~|qQbMGYTH^ zoo^fY>Hm@Sj?s~IUDt4I+qP}n9UC2V>~z#Y#kMN8*|C$3JGRrYZR1N{_dTBLd7p2L z`msmtQ|HfFXU(;0%{`|N$AN!@oYblpkCDog@ZYlh-vZ=+%5rEJ7?6L;YP$o}-2ar- z&7g>+4$%LU&XC__dul$*aVlVsJQO|f&JTeDf<10iyqVD-Apn%(f42W0Jq6lNW3^TD zTOGPjV~;xoU|jsC3f@#00wfSlMgR#At$E~v?~d!cple0xdB)>LN&g@!_iZ^`#?nqE zaUteR6_?(vf@54Xqp~QuF&wF07m~kol_sd3plDNhMHvp1%KKMOOi7m27P!%4q+edr zfx)oN&JqIq%!LQw;;AF_wiy6;Wc4F|^ua;i_J%R)b0Iw)x~bEL*&##VRU8H+BvVo| znB#cG1Hb8}1Tds90DH|^8A?oWD3c^bC6(B(FI`o>!dKBzON%o`;Y`5q$)OChbono{ z>^o{?8=+iB;u)e%135d;1}i)i!)k0i#KjH#g*e!*Z!M`@NE7+E%SB{qVT!4gxNCa0 z@`m%a=E~~fR^-gfOePWcoSOhTRS7N%{=#}?$%K&B^KI4%?Pg=ysV)j#ZiFo58nbSd za;X$51vkp8Bc%g#h{cehGl^$r!|FpYpFxj>Au<+I} zDNmWqa8o4^R2das5zq0Ktgj?dZ=$1KlZK4Hk{`q*mmPs4A6Bp6EIbmweWhX$s-szI zDiak;bBa3MYhc}x8u)a-#)INqBN4c&RhRQtJ>nsRRlj~JmyPOElbX-8H2lsdaE75- z9mKX*Jr<5f?BWz%vRe#TrgADpYrDs7Px!V!^NKf1A+a3|5+UWu$II$EJjRE}A~$d4 z|IsD=%)*z!X|Yyl{fLF`%b;g#&YoE|cUt)rL}_KMC)-b|l3&l=Rl2px*A*y4D64}O zX7gAbbFuZp_aoRD#fdO<_qI6G*N^!4Q}&Y2mqDIc>DWVFN%{&<7?ZPUmtV9~BKCy4 z@49B|sCeul(+`Mo|LzSMirKHZgQH73tg>3Y#OLep&e>72gXLScua(_&ko3wNRR}sy(Uo9hia8jIa8!o{bV`QI( zCr&M_CM&kHzh&DwpcN|C6!|69>MMm}v%b`NOqzG%){Q3sdLkI>d)u8;#sxUm1#W5| zL-M!X9E(*O@jR^9IW0QKbYHJnn>WA;3;R{jRK=E>VU?m!g~jm#N3h0D_hhxsFz*#P zgqn)a*kTl=4Q9_O<{Gvz3T3ODXL)k^JXdc;*6T)27Qp8B$LLJQJ?|!;cIF@?3Q*k! z2{&>+J|a*7p--mQB)?-xn&jKB79HAequZ(1I@QwCy&`WL+lj5$6YJdj3wb>k%R*=U zPx?BFwsv3-mDU0iv_(eAyw~QQ*g?OsqiUO)4w zz9L;GJ8`qd&!Q-yBpYWZ!W@0{ibpsoKT%)euAoe@r?GBI7OZ@%0z+9W#k4Hx!2|z#FyE-Wj>;4(a=rhgR>GOv+$pC>L4_ z#m*L*O$ok$<|)Fvi)9V&6x3*IRNCA_yMF8`AM~u9DNMP`jMCtHJ8X(Iuv+ zwS~>)Ng6-4tIAw>jfHy+&5OvWcgn#xIQUCWvrE4CuE9kR{JH>I@av!D=siNs zfa2q%$aYLEcPGuV4@fL0V$q|3an*D3`v5DP4Mb6HOWMypDjDd zPQNI-T8q8k44;U4Wxuoxpd-B-XI-x1!%;8eBW__l_1{#@WO?zqGngK*xz#FaWze2t z4MQJp)|Apjwmno4q3HcV&26#fz8-E~2bA#RKlg7u7hjT~VxX(y`a_UFV4r45e~K4R zF0}{w7=CzRE?k}`a;^^uT$u0-AlZJ$UZvzK-QO~HPHh&1$xd)TDX8sotpbf+0+gAKn}qFf@~&TosyloTKf;alOuP~Y<-tCx0mQ=M8@XPA3lch8GIZ(ZD4fR77l zVsM^-Xe5o3`qc7zoL9lNaHR7zbdj#)&UR#%<`cmu$J6>#9$C6C`Z`=CJcy5 zjvcEWBb90YnsR}sMl5f_4dv-*3osAOOK!;?7ceIHlk~R~zXNa&QOR!0AIiU)uk!tX z5Z=Ai@B&*BuaFv|nGqJ@iB^ z^xd7Yqr$$WOAUq_j^IF2Tcp7?;Aox77&2o;=Lh^*Z%x&3HFMxdi=y5yxRT(=&W@F) z?mJpKDg~n~&aJ2e#!zfumH%0N%c{M6l!)F-TT*DSZe2CPq8zrQ)3vjh_4QyQsA!tn zsS%D(&B7zjkdpu?%`Zg_SR{nYmMdu3-8-|#uQhP7W<6 zUuOUI?e^8`&jjoRGZ!QNM1FGeA3P$PCZ{HKjylvVD>8%6)1lAV(j4 zb=zZNjF(!w4Yau=bAVtY%7@!E+oU{u7-wKRZ+cp#>E+kgahoY`fy9SKdft)3%7iRd zD(4yWs_b%9)LBO~R>X_#h!9u;Ef#%pKe_HP9P<>wGrg##wun?;A&${|#i4pu^hj-9*7GNJP1VzP;gai7Ix9$=f0v{oJs)DU=%q_rF1J6d)?W8KYi~L* zuS^=XK-BA-yd=VR{c2|R^P^H}V7cH8&o{f#7R}Du;tdY$lqHo35-Gy|y z+gByPr|CRr6f6(vW<8p+qXiPq0)dycG#uEYNZ&;f9{}Z28k88$L%qQO^O!YHGq{av z^9vYR-MvNLV6ZN_wSfq(7RKpHHU=V~M;h+g9%~EHBs`e%xoq?O%a!KU-a5k}BCu_~ zMg1}W7#igspj9QyjIrbbhqTzTAIP8bx-AFz;7@7pci8WU#qUx-dl2boKK zvXfY-Y^-ZNa8P>v@3!2!wBIVF!kvgiTcLR?nw+$E#l_0?&T}HaJU*%r3-9mzjQtb@ zyt-9uE});M|3%{ z#Aq7+25y!SZ0DqtY+La&eY7dwuDi=WrWNs&tdqYbM9|BgTswH?fPl0%WOcFjfv;-b zG~E%RDv0k*XY(*?z#ixJo*;Br65%T>X(n(u%aa9H9@HuO{Gme6LPMpy$tWI?k`hFs^c4KcoeT7iF<4S!!G*`dw;GL}0l+V%< zaud```@jC!PJ5+3c+~={0PCTmZTPz%D8xa=0#SV(x8wr&`C~LUk1(sBBb%Hp&Qw!z zAr09vfwnjo<`%PdNY{^^j%$bJnv^x$>=Krrp>yKiHbmy}?cuTcFaK!7|BJH!8;Som z((i9dH+P>c^UP;Y%=|f$vT|Uy2S)0kAp-r>1+W3^|AFFUwg37=2c_ZxgTz{s<^4hI zA{-$cAV^8knG#yV@`bMIbeN+nz8O?4l6KsP%T#@8OoDntp?O1@EWpB*dcrgb)YoT! zT1D4y`1(A-ZDLh3ZlY*r@*EVz+z3r<282cuX$tC_T!!j5Hh5z%ql!~2)ho*bY&Jqo zel@1hb!<#lZS_i-zOXKaCs2(vB$KaxD_s>$Z{|Zdfv!c~#WyS!p29w=Rxe9I`p!LH z);Vu?{5*m>k} zCJ3=K7{Q>JAIK7qB-`Pd@Uz`_F1 zt^(O>Gg$jgoEY%E5f!!{U%q;Zd4zfskXc@!ZV=JssASYVQT8E=y)obh)?NhF%AsZ+ z&}LE@6!8DRlBbMN5#77NC;D5){qGR@_uSy$2~o4a3GNQWKR)BK%If+NpHs`v|4o=J zKZ!}7gHlxBpGO)P;8Y7gCh+Bv1|CqVC+Z8KQcH8(gb_hQgVCeQ+N0h{V-mHl&}^!s zZ9aUie(V3*dMV~0m;Y%pF7L60{|GY3Zih*WfnLANZZUE3FrN0OMp)PflqFgp!<3ap zKZ-O)UblS>R=!4-QFTRtC@{QXfvSNEcR(FO3+D&e7XO+7y6>jP_4EiKlp7$HH2c9U zw?*A%ttujB`TerqVFQ4`_UYPtR4N&QTk*5U=rRg}PVM$oF)zhv#Y9i#Gr>}8F};w% z)D3e&kzXW3io5s3Z;QHCP`qjwIuFn2UD|^6!XlkzNQqbRgZ3#a$$fF69AT`&8N;Lv z&T;odfr%KyBKYF2S;KbEM-2dG3@3x_x#;cTztp-l)*%>})Lc4x`FDrPD|}!5VsV{YdoQ{x6~uvR~RF=F#+XPWB{1ZM(829eE58o zX#a!C8pJ4grY%|e@*Sxo|LL^g&o8A+a&Ira-)wzBu2ViN1;bpcXdgctNNrusyi2a} zs6zbvbWzuteA;h1@KDwoTo2<`qM4M`KZu-r<%Iq$?l#*jW9o^~WucvZ!EKD0n1Vt7 zz{4&+e2JNj^{-}$DaZgU5Um6|1{h8;9#W5jSnp~cE_Rc%2&Y<=RurnWap7*Y~%`X zNm8U>S%HWz)_4&sLk7@EHw<_f)tGa2j9Zgt){MeSAWjK$qvO(N_Fn!Y1M_dJ{@-j^ zf@5$``5DC#KvYOtYM?_R9}0l?zbR1tKPeEu-N{rTk&02-9_)uQHnt-(wu||CFTSg29bH2OXRl+rL|Ix%~RLR+}pLtsyvVY3eF48SB;Yz zJFZb_y@rE5tg? zgyrL7UTxPig0f?5njEwtNW9rym_1G`+3pkw3t2;GG{z9K+x1^FD2}1tG+@EUxJ$Tm zHK|IZjCZ(4&8EtNP^!AW=|f?Evclkk(YK^|toPa_(7mahE7K{D)6U4Js%fkkUB80; zO_~1_M}K)C^576~|CmMpVE9a#zg}^p!I1tgjVYtQ8+-;EI3!S|nne&8L(dQ0JXglz z4hHehl}#GlH2+<`I?-Z+{*z~aF`C(|ePXpZJP8thV&VwGvRoTv~|UxC9hI=t~o@22!Zwm5e^## zN3gIPaF6EnoSHsg7Tjn)ICZyuyV`K*25*V9Mxzr_+CukE7sJabvV3mMw3ojTNe`V7 zNZ5zBaw}Yp-WlbQa{>sMr=ltLjoYzRrXaRkI+3np+!dSatT71fZ`fqh%4HqF?<8e} z3^dao`WxeJql$A9^U>W7&I1wThO1pdD!w@j+$dSEm05Bo3M|)E+3d@Yv+l0B)3HH^ z&|{9X$Tv|wrhQdLhMA{LOFwbeU4t9VIT6oOM=$BhAyvXZp#r$$q}-U6@9mtrbEzCg zbvyUthtRk5ogmnl!g1kvh%%0%%n#s-AEJcJt*1?V2{zWVC-6e3_Z69UenyJF?`&^l z(XAVMfBiM%J(+*;w7?pq$*v&Y5ub#JTlM^HscLcW`$`Vc6?!`*q0Z#qgkhv!{x(ju zROq$wA;m~dV+uf%BfRpl<39bwm(oN2_bRbY;p|OhrhNLMk?sMS#qpU$p=jImr- zX9R~xgRxfm{_yX#QQKO0CGF4CmxWjX;S;QX-y zq{=hjJ7#XjNaWXb@O(mPC5{A*v|n&5e@^&HN`$`$SZ)!F=;?-ijj_do%Y6+`y~bu6 z*+^A;?-6(sK7=p@jTG^skk$6OWr`-_FzodbTIK_E?&%g@nqqs({YV0=T3TDoOPjXD z;BzF#>)8oEts_AVYV$kh)h`J`(FVmx(dE{PU1Z!ce1TFE3*2k_%FmJbf*zdnYXbGe zxVTU_MMNOwv`_K76S}FR1LT$x;!ZKFJ0HjS8Ohq?ryz!-h4@aj0@JbBVg5dKAY%xW zo@5UIn!m7F-AdKG!#Aeiabz(*?9XAMOC5aXt^b~7^e#SbZcE_j2WdKfPms)j=aCqA=j-wiZy)u#GJSa&D;$d}KuVo>oqz3!>-3L1D;~!1% zf8YQAomoDMF?faX&C$ET&wTO<1&oiU`O8b>!*2%Ya-W0!V?>|P$U*4yxqSx#jK}(< zEuw@%K>&Z2ARvNr0A*Zg2>~APGA1Z!3S|_O0*_p_LvD`zv!@d#k6MqpXlPe|h`$N6 zR~7ofH7zgmHeYxj)Cv1$XM$<}iX_s;`I3kK(9Rm)OHr(vItUK$3D1sY&N#;UJuMx9 z>HwE2LX}?*$K|kv^Dp&PGCg7L1X?UDeji+G%h;Dq~NZh)M|Ml?d|2`Iq(TbbPFzDBsVt6&hhoBCNJ{0R71n zE9Y}~xDU4Zy>xWVl8X^&H<*=PBP^9RoXv!P?z^-doeJj~o>#(95s@)f@-uogRZ)>- zZ3-x1w+e)wLDx_Lc<$kq&5!JbX^bS&olLn8ghU;m+0^8rJgHSIB>yGJ9;dKy8q{7+ z4U&(o%H%DEL=PeKiI&361qSHRCk0C{t!z|s3&6v3Vbz*#g-4fo3{HK7T3f9wTGEK$ zn}8yiepII!zsi~6)!4(_Wi$mJsNCqyC@qCOGR`2>6ng-eXwm^fD=n`&Uo8s{WWEK} zv>e!at#vH=zBT5aV)o{0l)s1yWr26=M$vCLqSD~$ag4J8tt$4 zjtrKdGIhVe&lDj9oxNHNw@BxBa9gVM)Gb>I9*$n*4oRgqa340wITy-~PxwG(yhX%( zsr?c%!E_A(?IMvsP@aYNIfUNU3mQBW_Upk)5h1&mJ3tFJrHGyORL|s(5dHDHGjc2q zr#x{2J)tZ9JcywL;_i(>ky%h2CuV1+Du_Gd`?@xcC|UvQfL;q>_r=$!b=kEdXP57z z@e^V||gOUFXcjcQ9;^86wb{N37FsNrlr%$wGS0Ndc{tRCu zAnPP8F7PtpGkRGH`3Re-1<;KD6X9ot*`&Uogtb_Dpz_^k5AxZ;A_Lc0__3Rx1n^Q} z{<&duniL)$=s(k9&bT*3|IgO95$F^}g9G#{gu()F{ioQAR#}zn6G0JNm6|P{Ua&rS zgOY53+<|<~zb(>6IflgCXg->$?vJJ@JG6j}*bXG#6oz>w-?r7Ic~npNGnSdj>*98B z^Y8}#i)dk%g~h@c;RY*0o2Ay$WVoNL&topx*ecYxp4LY?oaV>(H6|@NkBO`cFC}py zK$u%+Y$m#+2xqA3aOt2jZmZ=HJey&o)=YR z(IG>-wl<;z5uFy_im<9De+X|LQd~(Mz)z$_cu?s|z7}@QAH?)S?(U+UHJzfftBJcwH1}%8=%>EYYEcihWr;CrdS0nit5l`)9%SWfM8Dct?NK3}pO5jU zjBie`?5-crjCIrk#x;1AbtZV(HTx2Nf;gqTR=q>&DYa((ONYNM+AOOJ6y!4=?VJtP za!Ujhk$WHU-yr#)NA&sN|E;6`X+16E@TfumNo->8oDUD5&v6_4KRiXp8d??b>^PC; z{uO%I|L;AFS4*^iecl-p3Mk^njGcIi1`9-C=SKnJmvIXNu+gu`yB>FAS+}!_ zu5-BA(<5dS)o|T7uYBsVTwy#Ab zx7umv@3TY+(xoOxG3q~f7E9RcX{Wu)0v;$V!$n<3i_tT_$-Ln+WZT2{p-7n*v09#{ ztkxCVweg0GKjPj48mW?&OfC=TL%tp|r8X^GXkvKkm)nig*NFT1DX5RHtBQYVc-=2BgWCE;=FkRt4>1EVHK;(91Fp-sp zX^T;UaQ#jy-|wp5%NP=m6;9#9=d4}K|05)yEvh*WXq>BqA>N=Nit}8!A2o26HD+~; zi)MFY2E)?}sM5lv+Cg6RB9hZz(H5AV@R@Eg#%S4Ls~lYQb3hEIc?;w_%-KCTif@D{ zE$Nj*Ze+>?Hvq-kv#M6!$Wk~9q%3*ZlJ5$>UV%R)txzSo1U(owyyS8(z<%@NPKdj0 zLA_s~*mQ?DwEW8J3L@P1;~LQ$|ESNin~fL-y7yfRkSZ4n?h*PHEHtzDck1CHAOatW@pH7D?T+|W; z0~Joz$=)^y4BtiZoK=pQhSuo&8G?$af>;ps{avCvq}DW{Ad^Yuk%v>*zyG(AM_O8c z(&A7&{pCV}3@Q=L^jLxXhQr^N;oormzk%EgTFIRO3iJ29fN$g>0kWKvQULwR_@IFm z>VinX#bHuDh)I^|oMy}sQamt!#=Y+UL;3xid1++R?}$HlE+PM0LiBMH0v&_U zkN}EbRz*;QT$)bYJ+Q+Ny;+L4Nct3u!X z-b>4O?#dM{ot=>HXD3d(9t7A=a=N=dK>XY{H8#IN8ft8E29`_B#v8DTq~St&8M-u6 z7Utx$F8Vr7%wnJ$jpoJ<0+V>KuH*2D=mA2O7|S6F_GG#DhJ}Ci&?vVwaBPu30wIU*|2C z6Rzuq0)6#DJZLyBC~?@dN1~@JzJH5kz=F;w6UDspOOZA_2|)Hd?{SzJOnVGm=iF|T5m%`#K zZ_^-;$AdSmj(GwBOi3XW<7YJ7d;lU$RyFi<)szCK_sfNKy#D@Rb!R0olz>0Nz`DFIo6t zbZpIY?Ggrp00Qh15R;CSQ|}0?3ju)}IC&P}Q_**WOs=W@E$Hq-i5x z7QrQyDpETj7kMx0-rk7(vLtQPw7R&WsmGY|mUFS*nx|&Ei^vLjDXX?|q_LPn@znKT^3aLT7%6Q@nbcZ?;^NsY5 z{M~FIdcLWYrUPc)++%QU9*dx|?CKNu;M~mMWu~RDK7e?Rs7kp3AaOurqu6tm3+Y0+ z*ujTfen)vrM~1Kx+@Pp$9yzsRTO-)jdpvVKC4~A2?C>nHeUuK|pqz9c&3J&R1i;N$ z`#OZ7c~|sYpi6aZY$6Ah#ycc#xdjT-czx)EAP+l+A`oN!CXU}y`V?~(c`ik8AHL?o z6`~pqgxVl1bbD_B^yI+IaCP-eKK#B(6zu|Dpkw*Ur98loeSvooBJU&Reb&K6g0`Xr zF#Sfp46O71y-7~$n@Su%aDaIJ;36A$e`zlK1>J-wd5abFrKRT!1nmTO&?8L=A`#1@ zc3i^!bIY&CfL}bMeTi(|D%zJtC^@?&W{-6}U!af7g(zFP0c7B%X2kbql0Oq0sKH(nlN*>w-jkCXz?Y9-n+I>N z;(-9O%UHyB*u?Ilvd*sBuFh<$`(n!uF7qN)Xlwp^2@}?e_NE@+ChamA3kpn~0StRv zT`j;~@qLyaAcKpOv%$fBwYA9ZrfGk$n5+0&aZ&nS(%KYc2SjFp%B!GJugboH9P^8< zZ2yBKe-*^9D4RBG_!w@^bK8yDA}e~eaGxfAw3G5b>2eaL21|PNR_&QAAU6Ee_AwVN9Nefku-Lf zQ(ZKE-E;KwGqxIVm^a?Tp5OnjwwG_gy$Jj65J{0==Vr+Ep1k4++T`gn`;Jt>jTt=z zGQ4r}6<|)co*wc26q5Ls^^v1EV@SbWl$UczRTIqUCp+dvn0rm+twcL=)K&JoWld*} z)%26e*Yv5{q2nbl$V=NmM@OUL{Obri<(m0vZH`7LZ-ujPD(L{m+eWFKKH3Vrb_st+ zt0&JJ=#uKEj~$`h$w_Fzhx~LsS?S1wC`EI@7{Kh;bUi~>ti?K>jV6_`HDbjD_rm-a z2FBTnH1#2(Ai*U7N4O&E<rY3`k7@g zl6WR~KdR&dk0}kzaUPWV@ex-=vcyJc)rFO=KkcriG{@2KxJ&##rKS^mvCZT&K!RH* zzNCf=*?tEMVJbhjyR#7P-$h_6s7E0vTmT7^tnG_;^!)mR76bP4WXr=CibUt&;J&;1 z@#Cd~I6Y=%vY$h(_fAezO`t*0lV>xFBWub{r%KwtIKGt*2~A~Ot+?C_5&E=`w^u_i zqMCnSzcR`t)Jf_QmZTbNAOUW8eu+RJDL)Ih9C0=q_-^%dSp~`H;bLU1)xo@Q1OrfL z79n(YYlzfh|RQHw67PhAyhMsK^%8z6q}M82 zF+U=kA7^SuuZ7Zov+p4x$wC2)@&-9_ zf=Lu-7&5auKo}JSn{k1yELO6S^dXZFrrM?9mozUnh zC*ZxZ2LkLbnGxP8*<$VK>n5E~XW@U1I|?S&lW_0sqdoaOBnZ?Dh1^2^S`S8P!#)Zd zf4zUy_4>nCjxeFtc69^xC?x~1*PoG-6>AuXaP6%X6WP;}f5(qg+x?YwM8&1wu; zecn{`sJ$w&MST*V6+=t?EMow zS)Krol8#^FIJE{7d^$U?AX3{~wHk0Q>roU}2~LER_w8vThemd7mnk*y=7|-bA^rs7 zW}y_m!2}2Mz&o-)@ZJKbi@zOcg{^d?)C0JVgr1XR-Digs`4m5$w$XZ0jkuneU$qLu zsJeS=#AKnETeyfqd;1mM@dfR!br5bTy3H>)6osfGYepQ?Nq_4UEGQI{J*Su9fct!^ zO3-{OC)?{0+2BaldQy0174&B_LFv?#Hl9K+W)rEoHVxqy_*M?^gTpOfQs0^LP*YS! zSJcR-O0#C`wk%iW98g#L=H?GazQZ}sp`%|~EKU^+o>XSO$#+b~2|2vHG{bm`^;>OT zA#vi;Bd&Zzfm42b;_yHjo^76h6FY_Vhpu_K=O+5gy!Vorf8mHaJcMT}0w$Y!`OmU) z^~N&mIk7}VD1bF!o-EJ0b}WTO13~^LIOM+-9WpFiPID83)eiJEADo9 z8I#uCwK-K~gK|nxu4#E=x#+#(E?H-Yu0uKEX)rz!9um9Q9P=@H_-{~8n{sp4;4&__Gt6AVJ_X$34C_Q|(Ly$j ziokV6*C3_{5)40BbO&EGv~aU z?N+irwD~o_qquImjb^4uTN)b5yE=j8%+Pq znh9K*a#tbfR{Pfgm?B)t0+B0zf#`rqJUw!h2Kglb{^0P)Y3{}M<(m@FnMwj#$Pg~H zo?0KQOo?75#cvZ0SFagafy;|Osm{}{1Bizf#^~L{rv>JjD6o#I|I84|p2blAh?5{S znvrZg$1;D2$WYlnFwSx;%X6eRnNO`NHTSQpVyS^^$R1j0h^C5f+Qh=fv`ytn8v}bn zwe-;g_>4}3Al_9~DQ37Wa5r!}*sQd+x?1SXKJQnZ+1;IuXXtGO!*Lt2aoXpO%6$W2 z#?R9+y_j4@V|8B+A8wLy&-uQcKg?AD-?c2K-Byxb#E=%bnr*8=m5=xVaTi6kC>IF@ z1{vOVkh1ep{9QJiX?!XZxgMS-uJcT;qmOD5!1U+ZfIX%7S;oM)@2u2Nx5Z0N8I-`mH`#M8WV4+Z|F)O9al3 zetNz1emimW<~iAe*AGQueKz4D>t8zGP@k~I=dgLSDIM}dm217wlC2VfBo}pr-3B02`8kH4@Q3I zVc5EGJkf%LKmb5!n<`MK=F{c)a2-Aj?3?C1`)-YZRM)$ynYtz+IUpVEjgjlw8oE<9 zv20FK7!f3V9RXxf1>}1MU@p7{OJ=D`mbR~p7KzrmKnIDI%`1rrT!3_h1arw!?T$xq zOSw^ZR)XW`lWmLhpmqfdcSgesUHZ6NkZu#WxFPR*4CP>or;F+Y`<1|zw5s2HOT6@& z6)*Gzaq82=R8&z5Zc<3G*2ZcYTLHEpXjJIS&)b8bTj0?zw9I$vs z#8|p|i5SgBUZD(5&QRbuLv9)cJdwa_@;r7nBq>>83FKSd05Xf51G~^@+Vq*OHGoZ& z69cbedg&uEyK84#mF-W2OGH*{$o#@0_1XE$tsR~W7f%*PHWN>gSvREk!qtMGdNv|N zkylfT(jctd*!_u~qTYdB;RawP(d48#ma855KJRQTPv5a(eX7%+4iS9dl|Ll=OrN#9 zKbqhuI76>v z51cpLtUf`u{LS4gl@n}>jK~3KIS5MdVLMCjg@6AA!1UvWBhVM7>H+Z?^gXl?@y9!7?x5-K zDgcahB|&lnVeJQm#yL)Ic_Qz~rIDvP8~#kJz6_8j9rI_AMyH?_yDuyVvzb5qF6e_P z_S$tGph+_k{*1e2&Ow^=oEpVCUPSH7HXm7J4om|U0=ypU+^4y@f@}8Al6@XV^L~HI zh~p09FH0H4MMGA&Ev;+V_aoOPZ%m4#Cs9A+i{(>ZD)w5pX?1n5VW6|~h9yS>i@E|a z;-Xzw2Evh_s* zu5ma^_ufw*-pO6_DqVTLqXG3JN&!g=WcNf1UvzSM2EH*)7F=Fl3xG>yO;@6&bfNcz z9s<6$F$Z)4BqSc$lddUsC78i`jZkrn@}h6)64JY;Ilre=n+m%+f=GE`#Z6A@k@_RE z17?S=UNPvO5wNF?Pk93bV~u^21UY^g{z&loj&{)<@L)9bim*Rv{4S1UrE)L!d72mM z8Tt83m^--v^4kloM0>PbFN7~tGOpj%Yo@P%4N9>MiLk$OCrF;x5%fwt{n|D^Uk!b) zL3WXmsD?nxw9_-;n_>G4^bs-(G4Xk(3V_7_Du0LCo_vRZ345maRyqhQ@ZD@G2k{3i z)*ajX8zi+MrE5xh!#J)d7vCT^?#^$o34ajw4QVpFmSsLW>Jq@fF=h{Q+qnByf^SE6 zS@!in5%Z z#IjWRvZRtJhHN>PC$REe7LA+lx+CnCkVX=&0+CHuwm;w}iK)eWv4QFCH*%>%zaUP%bL2`D02>`lPN;o z$MvbmfiZG}k@VLtXq5iW2XXxh=&f+yOb)?#GA`>q~$qyig~s zlDXf?ZvS&&n9S0$YSyw&IHKN*-Es}ua}fMY1?+3I^qxr_PMS~R&Q8QJLMxaY*hn2WD_*NuQbHQ&J^ z4<_0%E(`b;eFD6(gC<&v#;7Cg7qqqOMR26bRRZzUJY$2DVuK+R_1y8Sx2k!V-|eV~ z6)#!Ntq_>Dnm(i4%BTQpD}rpwHR(hTVh+dd3;{*3E99OHnKSl2N|O-rpt0g#@MQ{5 zf6RXo!&(`KHkg;ZBIViEUo0nkP3qNIH^oJgL0CMn^SkU@dFLI*zRm}NH~nd?)d??j z*5X|JZQ!!m#ZM1DR?cY9QVgzT?i@}QZ9M3RFwe=g*T>S)jRFM-M>gC7pQ@{^| z+tvr!m>K$A`a?<=#LF#70{ZYdaXxL{P-kACz>89?f@JEP@50zlPuJ43|x?ogulS_L{aey>& z$d5m;&<`rpD7;vLuo2dX|L)(oydX{chTm+A`gdf4LEf zrITaYt|Y7c_Y)sKCP6GveKYImiL@z{pXFw0a@Ak|+X~0urmg2D&LJwp4XI1Rf=Xax z6Y>Yc5R&&o>u^N^4CHV|y*L0~l$j3BD+J%=?mV)`KwNT={qjpK z()MbZiU7Fn#!Ex$mDP^}WwnQ3pAuTEd%}_o%MX2n`UBl-a}UDL8rI9c%3q{!X(E)Ke<2w&;`3IEX9*$uEXd)1>ONIxRtrmn?un`jwp@#~nlqzLwQ6~&} z$TNf_uSjk3!u8kYkbUaLY;zpxrNwGx>7wi!xZ^7IrBdU0l-5jD>Nl2w>0b=2K`AvL z{}h%VnQO>8XlZ<}(3K%pjy$w)Og-0<*Z1*kN5YXt(QdPBs^E>T?jmqb1N{C5cqauu zg%a$K+FGJav*x z$M0&7I@3Up=vE7^>3b1B#mmm`Cv*JyH#Ogv=-&!KI%*_ArLHK`PXmI$~j=Aq}_&0p@%VfZ?(GzU7{Nasa%wdr=)<88*_?i!-0THNB^X*SiuO zT-C)^FZO5vaCJ(>W7sv;OraDP-X^% zW*%JHneuy6U2KgGx;+Xd)%77W6Z9Eh-xO7ApF8?ERU zz{1-A8Ml)lT#=x+B7E1A{i0`loaIsD5v>nH`Qjjxej4FB{vrBt27f8^tvr|cvit9> zx%uzB4j%-BTJrx+@?7t|rVT%7t@Lm}&>32y=ITrCBnSwbztmQQH(C)`M7Ym6a)7J* z$owZu^%*!$M#c@a@8pGUwjP1H_+-LCe`+X0jim@2|)HKd!zxx{{{fdXfoF zY}>Xyv2EM7b0*Hjwr!gePHfxu#QgF+_pbZi`>oZhSD(}USJ$rUU)A2VcL}LyUgIqT z77*WlvUH^0<}8fpG((7)_E2i<-HJEy;X6U{zUFAu_5o50b!N#VrOMFsL}PNZzS9vY zUW=J@Tb*z)`{c1MEsOz0s&2?+URhr@x!-76_mWoP5Y_uh)g6iQfW`&! zkQN4AOre$ho6+A@ANSXQfn4_5We#64b)Ck?8hAuo?O01XwD z3(OXLUttdfwCphV=U@S)o`xu|0#Yvw-6o;M-lG{uh7I@04U6ni*Yl>k{#6FA@7oN; zH(GO7J=#B6_+hPM5gj)`$?lF&fy?_{{&fa}jGywJH1Oww?77xBr56 zZx6z*Q*DDFA%s5>1HB~%`ib$87>@^SaA}89u1(T-4aKZi0aS*x7U1L@sO4PPt_P( z{g2?h{dIl{kp)jgz&DBC4VhapT^7HcLH+vD-c<&sI-j_*AE);)K>bGa{m6E~k$N6> zMc>2*q>1$oOMB36c)CBb`#*nse#Y^fp@9I6C}$%t?FLN%e^f{+X_6NRax!PA zii>Ic9tIZdbem_3`xJN%JY)~ET!@0vXzWPcXw=lsPn$%5=kCZ0%bT0ay^Qv+Bc)iQ zcouuGc_)=QM$LVmVH37Dcp2=?36`{@3bQ81RMLx8RrSegI*AK3l$6#gx=oABtpN+# zSnIN4@iJLR%C1Fp8*^o;83%Gvgx#reu}xL=Ww(#C)3z5X%CHTtZQ_pC-2LqaLsGKE zrO2@X9qlhbR@vd}ZL@P$C3KA5CQg4mkc6~p^AisX0A?jWQ@Sg~H+ii7dY?n+G;I`K zU6gam@$sW+i*zS}{x^erD&`+L3DZNy3GAvNvrunz7m9rj&VQz7ty3ErZ>KtW6FqwD zM!dBiNE7v>*rysR}1;n%&9TvvUI6F=J5}Q1}i0I z%~AYIRezZ+BddXk5!gW};VY$%3|21D*amW@7C{}`0`(azY6`qGsT8rw{;Ke%ISFJr zrWkiQam6MS4kohi>~5FD#FL6+1H5M^%{tu3RnjxBnAGHhzJZymmN#JJqngi_8(Tct zs|U(JG<}ra6NhN$4;eRnaqSDzNsy(qc2c{MUmI0UHB~TdmhIeNZ)w$FAl}Esx%~3) zSubrrjps+fJEgMHdvQ2s&4e^-oZf{?77(o4e4wrd;Gyh=Lx1*2JkflR=ixa*X+LK* z%l2*EATXbNT6BPZQ0?e@$O_@GODzYrC~e9DJ%;I_E@i7n&OF7rjub_1otQHGU{!H- zGW-|aM%?@sZ`&13qYMrEj>}|=?pJlazUE@V){=9y7ca2&LBKt@b`0{rCwPzYtEA+j zA{bDiSOIg1!mK2A`P8jM_DaK~Z;+o!4!FkilEp(>`2mJ^0)%y-heomFOZ1KgcE8Yo z^~52EPj*c$DU_g22Kn@$L1?kcAe&&Io!|<;Ad$tiq#<1TpTvmvB`xc)$8@}(-@rBo@2jLN``}kEtVAMAA7Br=QQG+udPwz`gX5op z>!bzSAh8wknyMY%afXgj>FcSt&?;@!*3NH*MHXshy~lBLrY$l0;svobwdGKh#Qa>+k4 zMSV~SF6a_*a1Xx|33_pNW?7tA%aT>N=QbwGxim6V4HNh(;~ZJz7&66xM#T)aM3@IQ zl2%SBMjD5N^jf~RnzLjS3Cg@I)#%S23zk@nQIalrjVle$ZP5)MzxHxMGa2=!Fu0~| z+$sGNWWq(uH_gx0iZ$XE99}KWvPuuzM=CpI@UNJ8S=>j<*gFKi;}5cO{i0~D zD`RRlB>Ak(V#`85VE66fvzN=SN_F^=kmm@TyBJ)f&|ELId_$o;4B9=mm#Rf(Ym#NZ z&*ZwK`O5mbZQL{_U7+`VnDl9;%N$?+uyTHiM4=^lMMHz-mUQ_EoN>uiqL){$ssGbw zaa~BDZ+`6?YKmBg3tL4gM3!TJ(y=-vuA@csj4NxNVX6OEpox(c_8}~acy3Ob#(`(O^?O!h0<(m~GBH-XLOrb7+^A!) zAS!I{y+gEUoUGGtAbC!b;7I&mU5xgV^}sBJlSr0y>%H%nf~^G2!7>hQ0@fS!`O9#0 zG+xM}y(-~?w;Bj{w1<7P#qNl~!j#;whSVPgLH3c}gYV>mWXKu^DB{~(fGeUfynaf4 zzzL!hH*~0?*9p-$p@OE(uc@O9uAp06X&iCqtm(lLXNk!hTDkphtl>PBeSM<2AIrwDJ*GoGI#rhgbk z3tnG9*wPhWfKgA%(|M>pqTZHJ4`;Y2J*+{}7E6$`8hYqcdv1Q~Hh!2X4TOS*B5#1R zxT79OGn|1LaG-xAz;EUXOSB2+6^xy++5X$*;xuJ4 zR{c5quHL9chY#Q=;nEIh_;A2@N+SA^ulNC(6Sj>ZRu|t;zSGgd82g<#U!OOCD^J*h z&|9pw+-8uGShHy;J56Ex+hKf2fkr>powF$ez*Ke}$QRHA*}8$j|1095zzZx2vX)Zi ziSx$gc`>PqPDSP$6cMtS0vLN~^53w`A7%+2GGz41?+mdC#knf-En%JGj z{6G`{q-HfjTFirx6DRYIrd>a36?KTDJL(W~ zY@O-7W)4Q<$ad$Dd)eRIAOwS$2TF1$QfuJrhM#`pAwIZ(n79%OX3@Dd`7 z>FxXWrqbc*4r;K5r~Za< zM0#vi=qYgGjPu%mmC7-jt)QrK%Ay@?ouxU;?b=T1w4E%M(K%`~?Nm8Q^=Dm`9D}uW z3YY7;cri87^14hkLUc?d%r80OVwIa@q!$+1gJ6M?5i6=8*Y#B>yVc=_zi^^n(jDyn zvL%p?gjYE_C-H>*f$XdGv1x^6srpuge8U%F_@rTy7)XQHdj=aU^AKhM94dW5H(3(| zuenqgsT*j1n{Y!ev(-7u5R|HmH-k8UHth^2EqCjm83^CWipEcwO-up7;(X6+ zmm`f#(>IvfRF$EE4Hv>caf?=xL$bN^SMFsm!^os-phu>Gj879yu6ZLyZJAx-8=QXG zV}7r+H!fTEUv{#|vT{2P3?=-E zIraz*OJV!FmveoCpnxB!;tvv^{g;OM0p_1R@V}A%fBS$3#f^{OS04aCeEY`yUmhDN z&-m0RDP28WXe|PS)H$H=|4e|jL0Mtwzn;4ROW8((BS@Lx;DJlwujhsYW_|f@pfDlw zP4Z;~2eCG;LxU@&+vMROGL)*49*YV|!4RtwcnD5bC$+A6%;*t66-agd9C-If*e*ca zvlXcTXfIIl~uAbU?Bf3 zcSbPji!^;%cQz}y90S_>C&<92$8q3H;)R;lC^%0LG1RBeHl`vE_-^AsnHe@d>dU3V z8~#1~_Lfu(S8~eD=U_mPP{rixnsJo?sWD^Seb2_GJzah)=DYa0L{{$5c-9gti_}%K z6R$v`RVz(?+!gu^vto#0en2x6Oa-jUM+f<(;luzgw#sv&q^U(w7heCmWbsE~oFSJg zWr&S*OJpxzsM7Q~(4|yBzsRnc#q_~*TfQ*k?!N!?##loE^_>A7R$i;*B{)$fiKSlq z53*{7$W+>bReN;f=xph4uzoHr*SI_a#>xeGgGCGz{%QKazoalG-*NM#oIaMJ+2;LC zie`@UPVjSCqVB`YE)<*i1AIlBvtL7eAg;{6ZIS&vtUE!yRv?bTwLG-e25V&mZ)kgul+cqpXV{Zs)*o!zw~5;i4iYFBC;hJtBmcqE|G(7I@{^hg9~AB% z&|%i}v15FJ?njCM5F0OL%O0BoxMqi}h^h-o`JFRSsw<+oSrw!J<%+zjcsY$+mXeI+ zYjei1Ki7hCD2s{Nk5{TM5>$)8#^?I`8|7X|{ELxr|JeE1igitwx#{P}-@h0?#amPS zLuf2X9oYh{TeHPLIG9_1fx|^$+c71|S~t{d+cu7;&pFX{SgursTdpPx0}cC|sn@F7 zyXMxJ=`waSRbN7o6NR`Z_ONH?> zG$9^!^Cl8DPt^v)szu06el2tX5cb}QBkJ$)kLkRz{T`VzB3Guv*FMy@vCGbUG2-5S z(`JRa#a|vr{p?p!5DT>fv5s$k(?}UkoKK@cGx+C!L6+ir zx$vk8*=Wx+dTrs{UDibvg(bzYJVQ#9Z6_HyNEvynH?epwd}s0|a%ASh^E^P$#AjIw z!$opi&FAGmY~(4`DUBq?uA@|{Kbdex-jfaQQ74Vr$Q4VPt>g-nfNQmwdfDWwn%r@2 zXP=>0)XyXM6y&lA7dM2c@Fx|-SKkm$8d~C7Ul2vs}BI9$9 z@pPWh3%S=IFzR8h4Ng|a8#l4u=Ke6A=6&h&_Vj#@^RomHe$B`e(2?qFw%Ol>v|qQq z(;W(lkB$XRBSq6O8cU<@h6TKJMUe`*37K`G;9E_&k?zKr88+z$;9uNow6$`iw!Bgs zUbi$RJM!&H5>r*OCqgwgK5&Ma8s$ev1}aeWw#(_c?6uFRbvMAsTfwT!Xb-JZ7n~iI z;z>$(!W<}v&zDmoibb$S7d7{qSeL5wq)jm{VN5E#X#aM6Z2^WozLVEvf)4Og zlquGthmg|6?SxzohoNTnB9cZ*)aOctv-GWrQPeMTZsT3X_M)d1BZbaBEL&CFF-5Y| zVT7)AZj9NtmT9HC2m2jsr%KB%+V+*ue+}1eT#V@3Y6puYjfpst zDy&Y{Ep^<$T<$m;!+XOQ~OADy?zH^;(N8zxq6fg8Z0%me4neo6|pjo60cZphq& zeTOArl9Y?y;kl*2H1Rk9{KeYAbTV18Jb;tg(-hM=mAgdZxuPx(yIsu@-H`azdA5J_ zyV$aP%ppXgV-s$D{6)Jpi_SIloRf{BCN>WK{a-(h|KRR_kA(kl5$gLbB_eRVVDQ`2 zuQHiUXUzv+0R9b@G80XWo?<|c0+!;($%Bxx!Oa5zMk%j-#rq)gWkkS{(mccYd`{F@rpq^v-ezC7z?7W3r*lcuY$ z``HF|4PuY0;bCh$BnZ&}f1BFCWM42~P$f-v4dd zcx%~+Ro%X_g(#BA^>9Q3niPxm_?;J6I~N9d!X>1)Hc<4Y{)%eI5ie3(N-2O;%$a9m z2@NZ}sT*7!>BlYfmYtPptuR_Wnrb`Z1p`IActs4g78?UK@c3s){Q7irDw;EJp-g?> zKUfN8Y5kMv{ zjQDT$2qTA)9^uR*9YJQp(Yl1rt(m4r!`Xvqmy}XWqD)X{%9vM7+0&0DEq4%9@sG52 zhE_%yqr6v-twLcVEy>A15_k4o$+PB>pd)a-p$D{LU0(l9snra29f+p3K0+_{M)+0QxZLXqZKmrk*;~6)I)_=5JY>1_!L=ipumdHt3|6x zOS_^`15NvEru^>paH6|&JYz-n{pR)EeY5lV<8O|Awf$jmO3%hCDv?R~!_S!Wyc*Br zGrimV7>hdu&wcP02(#`q@vVHIhzAho@8(VX64R?Z;EMPSYli`F^?a;<@|ybcH4pFG zLi~t!0PF|{0SSGAu9^BM0HH|yqE8fkJ&63mPaJ(m0DVHAU)N}TK*W!TYqmZPz!ov? zjpZ~bK^WgI@%}K9ge7+8w9ZpsR17`Fegg*A6ETLq$2AiN^H>I8gW51aSdW0Sf>&fHTq|(YSc2XsK}N zAL=8Ld7^paO-leA00t4An3b4Th!f9{NeHD6Ij9IQhL}u}>0b+KNY+Ok#DX*_#EG+m z8{~`>0GI#}1)+m|&qK?bC~2}bl&KQ)tzc?`$orm$LKJ`~dMv~VL?8-56qg_*MOq}T z6xN8T1=;Zlg5g~a=3J*2YF#eS-6iopvDUj#dRm|tZpGb!4x&*U^a^qXSR=&))IA$3 zs2tfSw5hNb5u??_B1MdZ$#`3^>qg7rpbDs0xFg~-Eh-1Y6q#!cT^(#|nL){JOi2#= z&ejw9;v9BSrAFU?Qt?n@l#c4++OSgL*nLRGhV>0AVRTUJ?my%#g;`P0HP9o2Z_>=$ zQ7(2k$qH~2WzPb1b;YD>Is=MD7#qz5Ya=v;GJbO;!am5yj#b@_-BOrX^nAsA2L;B3 zAwdeZjSRPI)JDW3eaNtL^+rcqgvC3aDmmARFd~U6ZYtCQX)vKF*na?@DCYAR&X{M8 z#3Nl7lA*u@-zK+THgWYiyO#mYtYX|ULrJk{ji#RcH|Y-J0J7hV5MN#b^s`2lp(U<$ z4t-v?Ls4Mz$**V3)YLz)&%cd>3ZN0jU9@4|(OqQ>ir#*NKmJ`BRG6>^| zAr9#)nlm;7&FXY*9ml@~`_^mZ9XyDo2EUj!1?}Y-RD#pBrTX($P_0@n*TY0f@M5vp z7`?gD)CGp~^dsf*aaGM|0KygS+%n=y!^H$W!1RJ9toTx)OeYrA{56I(uz!WF5g-0! z*Jt(L+YwdvW=&>H)2KJ3pSNP!zM28OOANQK^}sg)(lcuHjmRBq(nQ%y*$mE+L2%yI zn%Ln)`v;XlA*ZTa(2Q*6g}rp&dMMgB3$xCeh4-p(x3uH&pwU&x-g@HA5FVwQqu2Z* zuj*AaM&IquzN}GZV8;q6t003{9UnHxa!fpO!;EVd;UW8DZZs^h0}k$kiGNU|t(sI; z*$fQ<9HFF5O8cv!FlVkX3sL3%k;oo5oS>rje^u@vob@Me&|5f)5hJFIPcIB8U{wzV~z7@49p1>>$^ zdWv;0Ok0#&(w=|AXca=GYcHpU+aNpSDM=ClM8OMD+-f(fomNrH=xKQRm_v$eR9(IJ zxzazT#H?eBpck=*9_OUv!F<7K9f&81c<`$P152lCE<$3xIc$H>wnAvVn|gm}&Y90{ z!gZg8f|0Uxs*sSuE@S)V4osYyzro)?j~tYHid0hGn^Hfa&L>K8Y6(o87rif1BW$V$ zxYL}cgHab_atx%ytt0op&W<0M6`TiKa~0zML2=j7dXkYOMhh)WO^>BiZkS+LxWFV( z$Q4o!iNAJUMRX8#6BRuOP_?OmZNA%x-3C7w_k0S7De|G2U`Ouw%A@-o7fR-vnt|`` zb@jcQrb)YS4A#Z0J}-F1Ch_nF^$>jp2-DD#+o?DGXRO{W!+14CPV4!w6nxK04}_pI zmL(=dK1P1Hd?)w`k@bBZX~@oijp1u23Zqx=ct$dj$Ry)*NyP+n^(YJTvMDR=qxoj4 z*h_=0r1IP6V>Ii+CmwtDulG&tWnMfAn#*BG-sVlzAh{uHEkhV;Qm&m+(_2LPt+=8oK76H3|Rf@%IH~pIWuJP zBf%|lCZaAiQf(T_&+?(UCCvx&iW+83*bjxMVmTNX%+S%mQ@}iAAhbCL=D9WOiLVqp zFe64RH zL_Dv~m)=&Ts8AOCpu5=JfifzfH@h35xnELo{t_2`y!q{cR3F}%^K z{r(jeU0 zBPS>hNs-E2S%ejSmG+X|wB39ej&mog`jQ=Jt|x$-->y#uLr^I|3ja&CuE%c6! z11TS59u`s1v>05L*o-to+>*;GIF%Q^5NbPsFgo^!1aYOPMrbX*;rG60?yjGDrE|cD z>2o8mUvdnC^QHiGLY-g-xQ)^|JJJiW+5dI&`ra1>_~C_;Zkx|>c8Bg&A;&StgLol^2;pNy?=D*-_Zk0G2oXxV&>A{&H8b-Kakru zj6bqgyCfzbhM*wkj1?Fx`nYY3;B^Ysj$xQ1motFz8fPnT%KNkI`<-PLe`s({;EY%f zg?;~Gk;X5Vmw;x`rDm%~K`ck4a;XrWW#9IcDaW4bjT&3w<>=b)qMhY)DaT`ssx11z zej!6qv8Y8bGl~S$KB<27AL9(JdO!PDl~g%CGuvRSPY-E>vdV#Zb|dj2ZHs>u6o+Od zzR0Iq*d%h2teIp7nvPESs{sj#U%U%PYFfIPd_aD3-%B9J$?&v&6k4qnumFt{#PL7Eb;ln3DNHG zuCQA&HMc?v!(a6Z{e+o)@cK~1{5Yj`J3`g)SD3d;io{MKJ@7X!UR!c{O5tHgG;aW0 zrv&r^lC5Z&HG(5SDS%hb36v_nBg3OPAs~EX;FWV^O%Ui4d;8-QiP1Su|64SX-f!q# zau5mGpmOpAqC#X58UM_(YkTZxvP4W9HBmoNzo;z_qTpG|XShP?urmOr75=R@!ZRL` zu=5W1d7O?p*_+bSM43nA{)G`-f4Z~gAylL;2aJOot6vmO%}x|;3-)n81VpSgkHTeu z$RvFQS0E6|FMuB_-5HvSv|+~+9!sj+R+h3QT|rI{vL;>D0i}u&VfM?8LPF&ztlYY& z-0<^gNV%;JBTAW6zfs?y;Zf@Wk$d>!J}cneDEOUcy&GG~*!b^^pGY_=ZOw7gwsB2z z0_UVxY#tGtbJy7+ztVaF z>Q&$tY13!us9&lDFk}9MWZ@Wxes!y?_A_-YXPk#7#(fbGXAn`9aZ6ir7$@(+fu|-( zg_71RU4*M9MTG%{KTdq0Y%P@YyXZ-VN}TJ@7=7$@m~8A6@|`@|m;{A1w8n#k=0}kc z)`5v9vc`k7CckoBq+2~N4DFPp=0}wg)*jI08LPIF)bv?rka=c+)nu%k!2^=dow3N6 zw-ld=CO@sTyDDHHPRt2AH!Gr?M)im!`OC`U9+Itu^ zZX#n)*Av+=Q?`(Ln0TDRey^@b8PAq>LMVx$gj-ICjT9!S+GVK`pIO(0W^A__w5X%_ zmr6kcfTmD1^He4B0*YSXH_KkR|32_Z8XtK$T8);^vfhYm<(E?Z5A=HE2pN}^aeQWJ zlZ^O5(<1bYg*#e$oF9-*<(iqJ{p9uv8;iS9g)s8 zKS6dwwm=g)cty>4MsOtgbq9uN#{1Ac{YCYDk1(mid(x{-D0X6hpXBnGrrTYBbt)Og zH5Zlc^}wpINY`91FCG5(2#YpBGlZ^iX`mJ_>r^|av`6!}7(dqwT(S1Dl?gdi;W-JM z&OAJyFuW^2j8tVS8Pe1T;;@!<5lmqH$zZ|vI|wwWPUu8|5F|Y#-Ha7|*q2V3(l(xKfB(hGqwHEUKxkF*%u>nl2RsG-k2Ft_%-HvB_!} zqYu4t5M8`s?1!bhaSjdd*c|CXYOT3|EQ%9ZbN_ED_SkE9Vm5!Usd|OHoTD9F{ zXnu;I`!I_56p84G`-|b=NA*0hJ*#cAf6FhQFx5CLf3$zt$q)#j5yj5D%jkWA$LaFf zEIBbRd#B&&#(3)n7H#W)fk0pmOsi@I&D`#iPh}Y#?Vi9M zWWA#a*DTaqm~wJSJS66y$Z9GWeTH`9)}@V#B^L^EX`UveW2kscKSm}J(kgG~EQym_oaNl(?A6rP+TF{3?s){fbNlvetk{84nYaXAD*AVsZA0l}eHp=ofz^ z$TBTo6LTG3QK7i(OB#&}AIv2$v`HUOiXP0h9|B-tkCKwzD(AtEO1nca^WS01f}D$F zEnGJ_EgK3S1j~Fg2K<`myEP92+>Y3i*ws%1Vn@L*@_oBhgkacO-&u9A9J+;3`s#T| zs}zB5MRM9juoXqH(uJ_L=>0C2VwWLolNA)U!Siqz=%V)Wsi^tyD@q(eCsuX>!5vH( z6oDQig~2w9ZOQl8t2@VF4*7r0B&|~xH=)GsjyO*p^e8N|lwM2kRlf)F>QF^IyJ}Nh zizty&XG%4tUTBTF9U8%K>Pfp&VRc-~dp!)%hbW_vMp|$=Erv2* zmrP0Jm3=tPIAqBUb(%>a48$Y<`Jmt_-t45Zd`VO(Wf3LcuqgvQh<8YPqW7(r#`e@B zI)84E-bd*du$Em*&T5X3Uyn>s=llpWe=(_2deovgxXATxkm4v%IngVb?wZsb;REKr z>J?o&>O6Ykt8l$IbtO3B+#R@y>hJ2NR582`C@8l3FU=mE5iHgV>Tu&H0^f}>bZqz% zZ*!)eQ?8#jYz8S|x278jJrdQy+^Nb&XOhzX9ctV72=G}Q@!1bO zPMtaglzm!pCv(?&l*=K=>OW69{Rmh1=ryMKSE@e15R61-4~*sTtIQ75*T z2VU3a%%<^j)4Zyj5Ch^N6%gu(rO(dP#i1#o77-AE8sy5$eXJ%9xx|SDiJW5{Yb0Lg zECRWKl^+--ce_Kn#U@lYwx~HH*kzn@p(X~o#7%<*&W}}f3l#(3&LRT2<*bAc*B+yI z(wzT2%}&+j+Si;6MevdI-zJ7D_kD%zt3FLHUp~8jkZ7`ZpThyFzAgQF`$O}L!*XpN z22+h;%_)h5@zu}iTP53c+<3ACAT#xoOd+>K?32N)0Fh(%NCoRIcGFA&%*)7ro8 z3UFZuaYR=ko|90zCfc1|^6nMwAiA>P`X=t+i9a(SZW52Vbw6fu6cxo>OIYUvcIqVe zwUXK9h18V;Bv&Q2G=@ z&Qy9z@Mu@Who7*sPy@&O2);odp|Ga$ZqAaAKS?72pq3by8pr87`1{#l^&xTrTY{H* zhIWW<4)K)dPyoHhc>ii&LMw)}?NxhUCu%}d8B8F(J;<2&KgT5r0oKGoyVoFks*(85 zWpPz~dwj&ujfPpnM@(NGyHAojdHuxI5|#dsKSu90QWuYhBw$^yYV- zKmhrC=&U^KYm(qOiClU=;=tRINoz2XnkGv00cQ0>y!x|e@dzn8vKUw`7X7BTu$GW>TJI9!WTC|(#r zY<0(R*Rm{p=Wb^C;w|*?ZW9s7K$Jb)CBY;4#TzD(>V5U771tu(gk zKun1)!S)7bP`^P7DC(7hArt%-Co;R34giY|qFZ6%;dh>Vs=T-o==S-3{S6KVLQSSp zYuI2y2){*EMP}WQQA=qgcOtfGu^qOJu7&{2v4!dA zmW=iks4{FAV7643^x#(d)fv@shDSc`5{@*R4^cqe62tF-%wOKHF(3Mx3G>jDyU4;c z6Y7HY>s!8tloajbwTZdb`Cm>=i(+J2Ju~wH$xg&zXLs_& z204@?zw;~dBHBnY3px?MQ_?GpTQWcuW$4dQ3(vfS1IPUWvk1L$p)#3#!Z4+ntVVMi z##)6QrAyA+os#OUXTX*s)(V}Bxp%o$U3wJleZ(#~4aN3chEmO;I0`g|oKzNQF^+X2 zZnZh(TYXq(aOssE^O<6-IzOQP;TivJ?Jw}G{U0yZe{g`79}ydm`0e|*V(Jt@18VFP zhE{HvlphUDkU%XMounhr&z{UN$WVDO)Yu3>7{nxqVmxM`1lZt@0L;Poa@oBbhoA)9 zNup5UeDagoLEA0{*#^6nS=x#AHZ#qVGQwX`?A9fk>24FruDVNA9WHI>@lCR;MTd)@ zfBm-Fe*W~hO~^Uw;=g+3Px*Z1oqB!ux!fp`1F=wh#|PT{fr0o~45aOh4YYmY`Obj- zoF@Ip}dgPW3VPJI2WY;bWnHvS(zt=P38vCy}v0 z@+DB}ku1$uv%N;*Ox5x|Mdqyq@-y%=>C#&1EeBN~ZntFRNt0cV<2uAY$GaA}2l4jv zaa;CHDIDkwv3dJDw8gZ?z0gBF{859_qgIIOGQYm?EVhrzxEl9d!4Bb4F(j{^Pa_ng z6fB7Z48L6PKx!;83C2k^TPwBdmaqH=n%6vWlSVi(7Cre~xrju{aCOlvbwa*QJ8$(5 zRf#&4A?l*0aRpU;)ha94rgeoTXftf|vasS6PEQ|EZ(8I0lYXQWFS*;akfGu6>{Vs1oo&w;hkayQv68Kw;ggK zfG9*p`-MM#mqC}%q{6hoEIy3b8}vch!t2n(6~a)uU6@$EmlEZxP#_Y}hj!?D%7olX zdkL^0YQF7zo(3agi@RNRVD}OF=fy`c0E!!aC=k!ftII|a6+sk!RmeR~nvQnCZ7>_l zeT@qh<^_^f@xqgP*X@j1!wg25S1KXM+@OdwYH&hpdtm%NEt9>1I=PUA@jmx03_^xm zJ$P|ir9FRm*={ zD>#g>TIUQl4QUf%Lq!f+EP5me;N!uAr?0v$2+jn4gl+fbtVhVSX$!|Gbjs#c*}An)^@=>{T$BkN}xG zf5=qw_~r@gq?gFXvDZ$|x~)o#)PDZFgx4Tn-zT_%6uJonn^iG`k0!;`n#Op9wA(l( znDKpq`en#hdb$Yh!~pg?oSCxAN7FSS$LetEevRVu4gT{Cqq zSB~h2TAQgMM_AO03(5?#o;iamKm$A+jO<%TTkp=|DI@sOgTpso+SJ`)Jj_|zt+~V0 zjJ`BWm%*O7^er$;+e0g(cGyoQu&8ziG8TEg*QI{u7P*z=(4dwd{!E%Uw1B{_U+gA0 z;KUkA*IaO2nzQDzdkoba5)m%GPz&Jk)JCgrvvBY0u+l}@jD6WyPj+q?Yy+awy=&5X zXrKCJ?7}E*jB+LJp!r)JZf+5uW{$ylm^*?hKh%n7#ZD{ccrbd^i)?{r2)P;k$O9g- z9EHpWqteLeWFx6KBWE&gQeM`xe}}cyo!nJqXm#Sx=X(2YVt*wnY(F=xvA=k~G%?q5 zjvhlrV>Y%_d4Z`N`W6TBiw=0oeK-g0FJs^`)49Q)^7MC|oFr`B7379~Wd*&q{xLLF z&6?q~wBm|mUW#9W8v!hUgnMDex5Pw-)YM_k=s-?=!F0EM@VV?89d7MjziQf(%7mLV zIqa~~j-f5NoDjNR)(%}_)1(J2Yi8Zy%T~S;_&B5ULh=qa#AbcL?CxU? zTEul^dCKicS6 z=a?|Tkoz4%C$tRd$<#n)-Qv^2QgzDGau|HfJnF#jU!kBli2Dy}3XqLrR1Ak=Ru>CefzIn?wh0~}eM1mJ}XtBTG;xjUlR>|+okwcgoMOm@W z9~8F>>00iZMH}Rxwg49VM$TxCyk?qJZkYH)L7{VN5A)$FYA=C8)@HAXB;{Jx==e=$_R z6;2fM@vTdCvCdW{Z5J>}-Hr&%?es#wg%Rh>?DRP3gA zh|M^+j_$VGHFkfQ7}9dvOf|)hD`?BOR-H1dw3JmVFJ+X6hAC>R)|55>!lQ6Z4{prv z)t9=FRo?iqS8-9U-0HWf!gKvi`5wk%Si!e=wT%Ai5ZH~oznI4mIF0(dw7kD(E({ZLOycjK@unUIk5z44|V;X$o6`2~X zCau2TxxF-rNa3&ul}>ZUWOChrd34oi}({J zIzf>$_0Y#lyQi7s`z@EClKN|gtQ5HR`wfxjOUUt+j!*SzV!j*@Nt56CE+hGXAtP;6 zE-~0h)piSrm*-(%Q2{JuN3B{YY(lrZh)f+>6(*0gu=$}R@hfR&2dR6Y%)i?}XL-F# z`@y!e)V?E;o3)**)!wF-g_sa`T56vi+%k_a&mmC$q;8wN+asa`l-{)Gjp{c!Y>?tu z_^%HYfL4Ky<#c-2hv8~38{ylr$uX}&HbKl30TK;xWqBD_#;8A*!DS=9$3)rawZW|U zL#9N$rj@eG_6BnIF3)G>Z|>!=#X130?0vLL@{~8t_?a<}nlVoE_hI2QPTA&LGD0Ck zgf-14#8-}@dx1zHDC3dtoOj>37yrhOZ?CdhD_FV2Rirh|$xTZ5O+&u^MHUgmw$yT?lpbTL#!v`>@@O6}H)Wdo< zkiTH)nGfh+UPgakhU%u*7^B}pSdOphCu+PvgA+7ZqX##M&HEiu`TUJG7;frYOpuHA zDF33jg|EXUP!foPi>Jut-VTuzD%8tFX;(o5?a?KD=bCkgY<(@))I5EE*;d>)^LN=s z*`GF#6_&>5i#1n0;Q!fe&=@J-r=^G~*riRb)pA1z|1whR>n}fqr7ZqCQ+BFBf66F= z*F@!tTVRzHR>vD8m7K9`-791>WaP3d5y<=7!FnMKHbVk7nKQR7KQ}yujLI-OEs*aBl4-VYN6emc%Z zq{8fYKt4n8=S}9_>D+?)#S1a{HU%kp@+vH!<|%BTmL~veXmUh+Gn~tG{26;s*Lk37 z!X!Plb)uUtiQUVjN$JS~;5(>pXo{*`;?#*l4=AW^OB8;ySUV7$-e`t< z3ao1CM?i+vOCp~rdRK(>TTZV0O6^6fRN<;f?kTj!EXlB8QsFkbz^o08f=jqaK;!x z4nmpY$9W`Ne^TVptIQ4iSA*@Long*VO=lS!@Z@Wwe49(&aKlL z_+NY`fO>W_iem@!8FY}yg9x#$m%9E6>(ucLu7GM@oyj3`r6TFx;w71ar0jY}KpBEnM$aXiv!!?gx_J$!aGIO%t;YTuYJ?mo}cIYAR2b+#BK%Tot zmScKuxATmnDj3V?Ps#*&DjngQ-LaM*n-TX16s{H$QVROgsN#iQI9>+R1{%xIv3V3g zZJV+m`&phC#@V(xKBpivt#U%{qYR49uv>o)(epIw7RgH8z*Q@A7}28>PGMhc7M}wT z=#QHpjjY@3kuArQ6xjxHw_GOd3EvLuG|rAU&KkDU7)?w(7DG$;lJ22Vm`p>6{nJN4SM_2XbefVHhEwC&}wRNzjunXeDd$ zkcmWk#1!e|szliv-r^s2_aKsY$6EqVK`%#N_3djsbTv+H3ztvz2bur8$0D^!vuT0v z(4<<@JNE2wFv$T0`opaCpU}m6(kh4QBjw-7fesT_NAjwBMir`D@VKdDOi2^gk;2(k_kTlY`7;06Y3caAder7tVc^<>^m zcz^Zv$8X^{AXn3C)oDv|Y)=I^4i$uQNK`aq5aU{08W{QI_D>U4F`GpS@o4zGZ+4v< zeOq|5Z08jEX64jp)pqP1T~gEAY3nSUDpGM$R6DL1iB(eHws0m^;pK(Y<)t<0?AleA z(_6q#cqQrdwLcr1y~CMaTFs)Z{wm`^t#H#g;m1L8c1I9t`2KK5ZfgUOE{m_tr0h<5uszO;xP3 z_ZryeElf(sL)6m4^H@>E4e2WDhPDm=gc_J`6{d(*C|!U&*af>FR5$0-&35{WY~&%pEIw>gao_X`O47SfL-A zB3yaT-gz5g+sVi6dg?xl1!aTgq;a&J`q(wcPIH4mrqNw9-t<_!&>>v&$`ee@W8UEQ zJ++8(Y$+bijK!@8K$SLh>N{)`yhV95>rbJK_c&lzF*`c8SOqMiQ?s3n8v_`XvVE$e<`0^-JnalYf3n#%fdmty=a7hC$na-+PWR^!eXH@7pfN`dhIjynF|0}beT#{d5W?Rr_K zH41t%7J!;guE2o@y(&kTu*H9nqY&J{c9v~*kSe6>Q8U<77-9&t1*Zh0k^ zbVY}zaZ4nnbTlijvRy<7UuSmUPN){f>5SWTM^D+vwotMMGBw5S`0vTxfE=rh{p^*V z*ZXJHH|JZ4QRu&=u|`;M$tjt-EwyZ2vEG$J2;5A4MPpPyNne!djskM*v=8tz{i@0bD!Ap18Bk6Ac%+BwItIX<1tQzYc{@HVtS-^70BV%9=f$db+ zQV<#+GoY8)c7@k~YV<``!hMBN5TT}3x$pK~tr)LLW7Q<3OX?f? z)^W`SiZdXlqueJv15Ja=neQ%GDk{HR$9m9zJ0+NM9j-Rf_g9r*=MgvkX+is;kY8gI zZCpJ;9&Nslyq{X(mSJ3doWrnr0M4{EieGY>rbAqKXOENbH(d)5Yj`gFvGWs03(mWe z@`&XKO=ajMI$X(CDh0?6dLfiXW*Nmt^RUQn$+pg@Y94GKG)B(-q10XB=}K$fZx&fdv9Tc z$vWnQ2imiIIRY zH7L4$YTU!)%3WtkWpNa-fMqrgZT@XI&S1IY!N{58fwCgG1yIYb&ZmqiSuU7tLKw*x z&n8!7+p=uLT_#J|=A|&~5W5y(%HJ}g(t9h#Z46hIzI*5nODHAMKm0iZ8`6kkFpbTY z{}d8LCMXULk2%D_zUY{?G2TQsE*{G0_c3);d1Vc2SCL z=K6|*1DnV6hX@H)_T^6}A5&A`4;xP?TL_pLL<}6pitpZPD2lJkVw`+d?3OP*%Vy@x zenPk9yx_mbW*rw$Nxpj(cNbBJCnkr{-4pu3Z>BNtng{AdXqXnDW@{y?|0MI^(zKurybMp4s> zk#AOatqJC<+YYavoz7HWw+F@2c9n(diuGl3Ol7)|pZHB>Dr|l|H2;ezee^3=) zJX}3|2b#`YDwW&H&e(cRyr8szEc{dd8KnwB_`@P1er(hCG0<5|w6dVs zz(APT1CTmWvdSwKHb1yt07)n)^A2xcH0xNu;&KM#*K?!iRoZzFV_82Uoe{ryF8Fy?GtM|M8r0y7T9*sh6Y2@(6q1Vn!74u(a zl%C%iNUUUpq^v_@zu(Qcq0Cf%UN2X=to#M4P&`-;J9;-42l>8FYQ*)2Ov+!lx3*^R zp607bqy5L|jvOg}lzpV>_}n~6F?j_JKGRV2`qZvjj@Q9C2S=Vcw(^mmdzP-)a{Qj2 zvMaY)KfEm!0UrFxLI;IH%qOHH`LEzNYl5BxWQLw51kJCRrEy>1Rp2kXMEue{Jd7nh zIbfP|!0JMuDS(O3Sv;J@S)ee*mXDXvev;=Vf7TkE!U<4NOgJ#_qUniO2@q+&Cp}@x z5LKS5p?tey2e(Wdc0hLi)dZoTJSNWWd%=gYnTaM=!Yhrvy?6RTK`|QzRG_*RBvOf z=xE(L=mKXKwpStzQRZoyY*=i7Ryv!nVX&mOKbeAhL<_PHuB2_7+*cJDYgMY~=a~4p zm+IA97wa|Kt9sv?i~S^>^Hs)q?>`U{_iiRIV0AIkP&I#rN^g#b^8sdn1I*u5Y}59` zPM4Of8_B0iPlEi}l8R2X@sQ^76sEXkFX;)^# z6&v@>wp*bqR_c$+cZO%U$+)$MG&b8VTUvXp)Ie8!`F(y7H!( z{0{^QHdb+{R0+z*M6!7d1uJrLPnNuEfvk#Amrg2Dc~l#_C7w!1=*NK3=CJPJF5Ctkbo*sN3hqOPz+ZveuAZNafuUpC$^( zYNU-JVCIh38_M{~?506PB`aXNner_y;UswsaW-wZ9bu;}@&m*U4Isp#p}e{NtjC7& zu)9L7z`j(M$-%m|AU(wVZfuGa#x*#yi%Xte?NXS>g`aG2nt>q1!+!V=s=jmMXl}l)e~c5J7viO&VpCjJlAWW(?WNd)=B9ykt8m1s zxO6psi9ui!?~nlb0r0EK-c;1LXUsEjhyUbFb5PA4G&JnqJ50x>73$V7U^wCe`J2$` zl>e;QRh+pu)J#gdI*1#4Hcj1Bs+V>EAz#;to<`95#pd zd5*z#Xo$acrRY0nU|aRzZL#(38!B6|E9;+(MqT!%G7TOuJdZ6idyx5VrjGp4Sr(or zov7|fy0Ynk##Pv%NII&ULi2-+H+AX9Kudj@am45oGb6zkO*j zR9j~`qoSNDXaP2IFR_iEIZ{I1Y5@j+S_JT4{fiuQrm8BcnOfXz$m99tVr^972?BV0 zhgR|tJpX>%DJF`Y{T|JhmirAZW%FlSk}Y_Ge%nT6_dPIy#^7M{L8(v*&HncdnVpf^ zV#55d-8m@RzHSbmlSg&E;CyfKI>@ZJ{MvxZQPOrdr!(Im0am|WILm&FWkce5)YZ{Y zKsncC02>lF*y5c>*x4q4OXUh^nskkZ~LjKf2=IZ zjQ!xC08Yi z#(@=_N2T0OX2;GBCSTKd2sLR_y{86#4g5(^XTADCWS@pEUMN;8b0stMixSBB9eTg6 zu1!1x>DbtJ*PMDr1iCMF%7>q{M?gfaf(T16vyMT>X3li(!yvL&Aq0%RfxmU42v;Gz z7{_(jjLSHoh+E{}{|?EGDCvS%I6jY&k^y3HU5i2|hB37>I~n)8)88ye`3$Unn}$q9 ztSjXS#MTguru6KgN2yk)AyaZ&##)bHYsBjkVR9Ize9+b>Hub4n-sI73AkxoAEC0g5 z3?uHu(7z8ch-)B|hOcXWV zgRSu{V%zN7yD_R3Bqk)`9Q{lL80-Lk?RE%EGDuT=2u z0v|3ySNo5UZfVZZyt#u6XBzs_OFgi?Mj2(Ax}FImkPTI zrwRvA=^fHhsf4g%iaRomrZh+2heMmzsOh~d7Xrg!=53DQ?HD=&)&7OBC3Zmc>)}-R zexg~mOiElEPWc5n#p<8#vv*;Zl>SQjR*n$}6UnIS;Yyb{Z z6+g6>WY3#-^AnPVx{qb>)?zZNR{XGBB*BDnz3=3{L{PC0V0{lx$MNw(Fw&P~i0d6+ z)@1vtgV~)_0oDTB_Am9?AsYiD5Z_@wXWX8|i;po*mj0;Qp9wdIy-By%wtV}&*t?T2 z5Qe?_I*>Qtp3{)`V5mQMqqdB+pXq%|!&5pf=tkGu_xN=;fG9tOh~?&6E-Z2bEiG)z zTCvaVm5#v?`2B+0KkPzwIG%H3(3@+UC8h;W<}bH@<+n>02;5a*y|o39=GBEZi5VER z-@+C69B#-`a0iq+KxuG?9_b%w&+L8>n zH%lvYn?X+R15vv_LP#M1>-HT1&GRGp?J)WKjqxi#pFr}SlC$lxj@*3=_Km4)L;_&& znw;w;jOm~R%r%^g{StDK4rUrQ#(s%ldT0R)^=~#jk$-=Yt8NwR%O!l3gy`%&5^?Lc{vntJUw3wwPO z`?ux%zxS2@XUAE_@~F&E8VM)(<;|N`qV(P_Y%yTI3)Np!>y|C>p|I0_SML+I-~rC= zjNs4RA~XkzV4M$Xm3eb%yhTHzrkUdEE>RBIImk1R&IOk&LD6EapPIOJ6t><-GU`l> zpJdR!`P!}oMAJS)JdX3 z!baRf6+Y<&Jm*CH)?BzUMaVcpX)m2&(e7C|DrS+Z-OhMhZceECGDO65YEfP+gVE`e zPX!bENXTmvNqfJi5=GkuZVvl&0+Rycp9cVPSr$W`8@e{VeVoXmHBmVuHl912RTT0*d`ZsUD z&?0h)-~J?m(EFr?9K+`8HHaT_8zN0 z5^gG=wmNOjZpQi@-EIKgJ&)Z>HH;6I5bzY>HDVabCgQR4B38BjI!ijW?z~A*aQN_C z(ev6p@3Dm_;`hAybXe9_1o-kyIr2 zpwKTCr>#@djB4CCZ_bFzW0tfgd8);IEVVQ=mrmwPEYp3giITDY35(PRwb0KK&c7u~4QjnqF1J*TW-vzqEPSVeSmv2SyBrpHk3g`&|<18%)!PE|!Q(Iu<$L({l- zwkzp;U;6Qmn0^V+U8z|Ov5+vzUY&%cf z{Z7SC?(5_C5@BMP6D|X}Qa9nb>Xl>>@@*IqOH)SW8QPQNxtZ)`D8`|{I5h?80vmJ9sS4qDq7vtoq)-?0se)OlTJ5F_so{Jp*@+>iabxwIzof$}Xp<5ue&&&kg{+j8{${FRW_C`~ z)|UR2*HBpQdLyY5-q5)2SkBNWq&695a>Z2iSMh6?B%_gumn12FMOLbXhKq@wMU!E< zXG$vGDoswbq!pn^6>E^KjF#4rjL$*>f$$N{4?PV)P2Gg;IGO&ivL2uR)F#15Monmh z`jOEu9Q*n$e%TmR8a;uHhZaMrS_46~I%7+^R56VljJ9=iS7*I#Y+Y&Tn317K$GnlY zIm6Gq-->GF==lcx^`)4$QyMtvtBe&PSdYzIw_`G3-8W;@8#&8Ad<c*rzp8mn*ehbD|vVQt|TpNwa|U9K5!oXBk=7Q|%1(5zD4UiG6|ts(UD z0J^*HFWty-y-#iJl8oV5yITBAq~DjK`a^8}OxoV!mO8`>MprP7OI#|FIQYQh+UbT& zDwef}v=fXrzzsof(P#kE+Cyv4)T@!)71=+S;H4CLN1s}dfQt5{$2%!55kbwnlOjM?&u+K1^&q-A5;QaANmyQl>aKM(c+qmEsg>D5o z)rO#!q#QgLG4abPLElxlbva+aA2#s&&YH)yJCZlzA+$`NW76I6fjV3k%P-Iz6{si4 zza|L7>F3u!Tdkp&$rPXh1yO}wEs;^!4T*aWB#3#EDi$~+tOTVs_Szf?BALX8+Nv+c z3&Zz&pIfIlJt@bZRD=fhY(Ba9%VLCmfgQd3b}H#z z6Xd55KTj5SPJ0grqjN)z3`IeW3VEb!#ErTJ2{Imb_wD&Bue9$_pwi`R z&M)8haM0SrhuGi%>fFkL5eiUMA)QQb55@f_QjtSgBPCye6YdG|Kdnvi!g#5P?4x*J zZ}Lwp&1ilA5f+Y6f<_BVWJ?U1U~^QTYt?({lE`XQ0)M~>M_$e`gSg~XqvLY4>;9y> ztkQzCXSe5O=2)AB4i7QGMTI4ra`DFYUb6*A-h_vTP}MSlcxS1)u;Ml)p{NWNlBq!m zv&jH6$Z^(edM5b;NQ7HO6aI-~HklP<`#Os0OdN~)Z@!Ra#^o5)85GhP_XZS}NXx}b zG3&6F@W8(C9+oZ4%+VJs`F6sd(TRK%gj#W3$yXoliFkIdyrAS`@0CKlYA--3aw_O6 z2q&BhRTZRGuFM}+G|gGY-b?V?5f&J5q26Qy*@P)MMQkewWD&u2wq(rV!00@OeS*20 z6ZLHm8WA0~iHIX>2)auk%M8Mx5`pd*F5wM+x_y5SL$2gYBIs*7_&_NF)sUnwk;k<+ z{JOX=R6*I-+{XttSl~~y&^e`7Sn^R0iUq2$j-dU79CT8VY*r?V-Zrj2R|ae-^5qyTqLTjCMvNLJ0c;cgTSX;R;)W z6AqnZxq|OJb5T)(6SmGBdT5KBBZBGqWcU@ORr!G!ie|sv2Jq?72COU|K&(K)8m>&-0tg9iH|^bcV=Rt_VmJ2lG^3G$H|I&00aucV&o zbb7O?bJ;+B&v-CSG$QCQJ*n682M1-i<1)TP@SQ;LwYSGLH!{@uZScT;|C>-ScgbYA zK}%wzHXT8JNavW&hpJgn+dWgCA(r+p0!;gLB!;{b$S<`(GE|YJGVVmu;Khs z4;9`!^N5YYo|QzCBEY=jkCq^{+G1$Jciy9`%)vj=%qHg-j4*>{ZB^`HyR)x%Pyh&7 z{}udtZQ9)Xx(#`#__^EZxpy#ZG$NKTIwKt)d;M(r zCvcq+4@dExJq%L5-REEIVuC%OJ}IYK738J%3SD0j<)IiuA6Z~Gl3eFZ(PA@IVH*YvP2c>lZ-fV$<2-$6bViCE<=0E2PX zg5=F~sDKCFwtZdt=ao%`bXvUyFQYc?aXIf`RQ~}ZSM;?P^mQG2ntq<}|K|gWaKZ;u zsI8%UHAFDvs=l^RgQ9pN9A2P6QVUD|TlVP^q04@*jIX}Obt7^sKw2US)01}HczggH zVm@O0Tla|-;*I6vKQ!YX9%6wRXXh#kVyh$LvDyVWkXfxK}|&dQ3y`7BsD%M)?hre>Z&PiG_r!eHE!=pggYDO)5mY8ME;Qo z=hGl-DAVYP_}7M8?Z;^4Wk_h&uH`*=9j3ME)@dP79#bi;d#G zsRfWIS~cnUDG}gb`46q6k1{pR{jT8liRx;fxM=ViyKlzv|icAHV5KGGs^38f3rmtom6ot^E7% z5Tl)oc6%Mf?bi_Dc3MNx;mu&-c51__m>zf^cbWDdgvQ~*?G%Pa!!^=mhLP3!!)$V7 z4Qso=mnVJ?3dt0&2w8Jd%NjzHGSgZJ(h?16Zf2Pp1EoPk7_2e9sh1c1OL|BV+7 z$BV{}%h-7m0f2SMzu~gHE0EyQssTzcF9&q*3CUxhoV5dobK4OiSd~vP zuVnX${|M2&I+TdTJXu2+a-SeCD#kvFXO?nDTq+QD=DjX*VDp!|2gR862{>6h^+^w)u3 z#zRmNpP>5hRiZm3{{}_dDfy==q02eFd%_-gs0dK00J8gxf5P!y>Od9vLjKEmVCp}bUyuDY3K5d5h-X_CFi!(4ulCK^2`ulq zcH4~ws)Q#FgLCUlJZXEc>O~CA@-jkhf-kD-yOxf8ICF>E>qa`y=mUu{=$poj8O8%q z;E4RIH#7fm;7tv?c-78}p?c4mjB$53oz%6TcKvwJ+5o~D*A*;g_Me;tm7NfWv+WVgv< zeU?|7zQ5o#Myl0HE6LrB;82oC1Q zESR%J=!D8%Yatx@5Z7CmPZH`4=9=+IzY9hrwY*c$az6~lAMpiR)zH4oN&0K&;t6~) zFiqOz-6Pn=$Wz?So-Wyh^Eh>#fe@|?nz3m`AdD`!V!pJiBRX1X3Tq^;24IB`9E$)_II zO9BKx3(Qu~-Yc{aujW_tjV&bbGzs$7ZE4$6##aO=!Bzw%N8Q841yxJ-rlNIm3-eA6 z#SyEq3K1gBHzBqwV@92p#aYEGC1SASE(1*|%h;7GcJJ%VmT4SS+EfyY99J*>k2V0(3NJ(X-4*l zP1?y*r54qQvE&)}Wu;(XL=T_la2hPUR%aad@YM$@Z8bMymt*~wFq@#wH4rG-V0Pkj z5+cE%s$A)u5bHh3Kuz6Jf+Ysar)72Z2?y4!n31Mill9CUwV_%6$tP=gLI=L_=^u3; zsmE+ep(_*jHvi#4*(Z;;VQT~mkdwK5I!>7bT5M`oL@q0Roe_7O8g>b{^Cn|GI({;H zjM&Dy;_dMapOE2!iw8kZ+?OZ`jc;<`{t{3F!;)wOt6?ysY3Qoiz5v#1H3!^T4%W*iS|rej(6ltfk2#wfDtO)PIuF!|8Z#q~SILjH{SxMBSkjh%u>FM!<-shrZ5 zXg;?M^QlS`CS77wxSFUmv~gA55?>W#$!} z^PXBzm0Tm9q(|P6*S8y>_6ba@7q7n&ZLPfqSHxt9Z+}!?PK^ggs!S%s7pV#Nk(qQ-b zOW^wix{2TH;(k_sMuX?Q>Yn1PESHWAbdye|XJh-cBy>IcI|z6${HVETZh*nQO(o7w zQVQj!2?~`luZirG>7*k5Z<_B*nJeDP_R9z}EN5%jH~_}5P_a8;THX44mE7B_w1*jU zkhfy_0P{9HYACGMubb{byr5vJbg*PQ*f?B>dZTx!B-&4ulJs5nD)9up@Q*F=^e%{MRLFVGbJXk{mi{E@R7cI>S1(6^I4d zC^B~5Tlm_aBiFf!jyF)~IhM0u9#Gwi8cJc<_=kxSyhL~HLG6oAA*8Htm)O_T@~!q* z)DS;Acm3|7^$MAdD_@<_@fs27UlanBdKa&U21MMMdHMbTRDJM=$jmAV7>Wh*vnqoT zjs!FJ!6Pe_oo_C3Tc47N)It$TovGK)k3l;tK2oHob zx>_oeE4NjE>}&^Of|cC&E~$dM6E98 z1Z#qARm&l~+Y8=%3J57{@tm96v%eN`A2YBx-!t*yNC?{ZXk@&wOdO-piq5wL#hvMd z-XI=-pYX=FQ0kN(XN&G!IwVc! zasw``rDGGmsV#ZjDoYpXd-w@+=kTXk4#W@)i0}+(zW4Km%}H9(L=`4nod9GRbN(PL!E|x4f+o@Ba@tapGfXSL-gl01rd>foBbY?- z5l-IC>#4iEUFVZ$K@2xLRR>nVZ-sXORBy#A!nb!1DGWyVGD^56UVB7TY%!X2Kd;o}z@@$q`ZZ%y1e-ypXE zV07kIE?EuKhrND;wHhzl#Wp<>7Di+?wv7 zBY5&>hdBNCV&GdJ)03my1H7fKKO@!o~t z_dQo9XYbLPB(r=X!%Tm_^h%hMMV|TRyHoQ{s!6|F$0`LK$9L?h1UR$a<1fc%NXR-7 zetJvJ#=GjjdmTOF*1E{R705kYVP1cJ^M#Z%9MFEmT%%f@i0fU&J!?KUfpSe2ud47`5f6Xp~jsaptd_TXWW=eB(Zv_-f0{Uln2BkXUTnTDcOlc7;7tESA1r zeV2P4s9MS$->k|!0s z?5L-c&b3@Zied!a&wTx_q#VL@6D0!bTpieb=057PA{GE?La)fW3E^bSDraD?nFb21Ly)uE%<*Y;DQxEfa`o{RSX(%NTM^xYNP4X{QK13OnKO}lPa|yq! z1V0E#eh>mYr1d{xb!VQ3-x9A{jxgBn+;C0jpwq|&`3&EZc`}ZXn1{-URts-iJ;s_r zSEl0JK@ zyheH4^+RC^QFAm4FKtYx=cw?~@7sRoj}ad(VJo(Sq5HP?f8UwX)*g!rZi?RK+wv}6 zoc^w8<1B(;JMwD3C?ZOlJnF3$VC|VcGn3c0XI@BuCUi|Gv)sNG{vL?l@D`I`qwmIU zq_~Y&*hr-FP(^w0H3w?pXEMCi?4dI&?kk`U z55VPFpB+VzdcUR4nqB~w(x_}brerb8wW86pCb?OEB7g3IC2ofOg8HUF zuX;{XF;PRO$8HX{dwH6q zySRAf1Kn2Z;n6#HwYdeN6;C|eZ1WT3w{xqum9sdXJdiTAH_%UgFR9wzuHgZzxw&(C zg4Vju1_olR99u^-)IAnNuV$WG$jT3yGGEqr?jV>!_Jt8eul^2F z0zbv^hhrsNu9dXc4fMqZt+w8N-@EzAFIt!`*dgh!M^$B6byZ7Ezw?!M@{~pkv1Rag zW_J|BD*M_LIIrg3$49R+N^2|UL;D@s0g+2eN(#G9a=c+p7r({%9Nn)##~-~pLsja< zZRp<6f|b6zLSWnhpdL5e$hxgtIJITCwKVa|>K(e>MJ$yVW&nmKzZ=|fZCKqC#9K{- z3rUNS`fl%IO@7T^E!3!Nj!>;FJiDY@FRs!x8!oG<=?c|a^RLIB{@7Z?vDigc*}|mZ z(-km!bjp~&G1fx6!4nJIqf7n?Al3E)34t{LB&6oAjStaYl z;pMCt6u*s=Y#6??wsYaA7;-HgYOBN>&&M0j>@YIVJpEkqBS-%F=~$;zq5c-Ra@zhr z`mF7fXp?hCne!KQe|K@L1(U?>I?R(n-cQ|4YTZTFzmZdnjotEtV+Yz9#QeCpCZ4M zw7@@)y_!7!xg2+D&Xqa+JB3Fa(ty=dmuDl`*7{o9A-e32>WtF61{(S;mKW`Y=i@ZZ zuBcaT*nSc~gn-WkG@Xa9$aseLt%ohDa`p5*wfYHl;C$5~l*|b|s>v=B3EX#Y{P4bLX|fLfiNL1^!Pekbuc{W5mA;z|oA~7ttGKg@-O`kBm*(hT)sj zLtg?;OwXL8W~V=E2btkZ)XX)i9ahyW?DC2aLbg1yWtna76C@^aJo#$zl_c(U!|vr} z!8;?aB}5ck2%^xSz>xF8PQe_l3&%NiZNlUY(>T!8n0!pO{+(U$6@!7NF=201K!)Kq z|BWRDFnx;l=c~rKmlvUnDx+0&yYXD=Fa&%Fa3PPsYU|!_A~T7tIdK|U`J(|?@BSag z-a0CdCVCqT?(RB3aDux#1eYMeAq01K8g~m4f)m_=yEAwQ7DCVfgFAye?7Y9-J>Q=1 z>^Zyt%=DSAzJ05ns@qjl{XF-!bbU=M{fDD^{Y1Up<09iVUDKY+{S#f9-#;zSU4To{ zq&tVIUs0(G1)9Xo^!0>%!=HWQ8q&faEcT1Um1)tM$@G225&KHmpyPGOI}J6yHGs^c zZ!GDHc$)$nPG9LHNF8`ObeK1)Au?WyyU?4FfG{#jCPXRCK1*?|>|3J!i>|ZmGQjnn zgDrjX2Lp*}UlR!X<65SG&H&!i_40cGrbrm2r9Pr6YC+|nst5HIZUu7%ZF%@1s>|>C z0$I&7Q=s(8l9O|KSlPWo%&P|pn2Kj^^R!C3<@}iciD~)f8tMAcxF8j=i-uYPWcPb; zb~UKYw;;Vm&_oZy9w(>+b^bR^e9z7-#NSsP{jb`{r#YixUB)iqBGf#s{A5l*L{ujc z;^a(w`RoLJLE~%d#xnGqFPvOFtT|xMUm1~?Rhyu$ao-siY*^D^EjYFSGB&L%JSz7L zQ||Fy@)M*G23NzH#r5t3bj4fo7V|~`dF&hk>xdC_hW7AQ+y(J57PEF_b~z2i z9)5p|JODzwM^SWmjIeyAD!ehnpf0X2ctAZHACRtH)7*=6z&6#12IDzxM_RQR@D2BG zws_>5g`R(d;!f>heXaHv8R{=ar><%1JmEFnM|LG;&V#_QR+E~Q$w@?&ZefGC#e~isn~Y?97AIt zFTQqJg?o0>wV_OFK#|17L4)h>H6wlRpD!N)T&dgR?U7srz3Jl!ld@ZjsOLBTjQos> z4|<$T#uJyQE?S*TloFTDey(gMIhmlX_!IPJ^I4xc@SW@lpasPl2Ioxdj}KVxVxP)7 zDKIHw%{k)-aKG&`?~T^63v z0Nhc^!m4Oj5{1vC{oj(SRdhc#pgtA{@5c#P$Im8;v4gk*+_91KlqywS{)P7##EW#_6s+w?l)I0z=MZXA=A6Q_R+W z*qHVXg%odQcnZvj`L3f!p%c?K1hdwvi`ROSzUG{hi}QN^dKx}qBpKpYI#?cFGv@mH zjDK{@m;Zz)VvCW(l7V_j@dw$OW|?LVbCH zzx@uqNwKEyygR*;3^Nc-rEY>7ALZFjf9Ah;{1_3B-R5O1GnhXVwFI|3ZFw57fcfi< zj&TJz*B*#u5O_lTQRNsYu44!CK6&781_gz%e@mA<#{9=iUgFoc^LJtJY2ra#c%w1< z&`rGWQGDh4S>F^#q#pBk)U@i zC8;oZ|M)kchO(FlDAoAz*)a5a23+kb)rg)bn*;e`=|+CRBm#5kR14%K?8ip-+iAli zPG|Jr@XgSfB~L%^$t~DBM>Ue{Z#(IU^=5HR(Svx#dmO1jSpj))3v?q#;IsGgy&B1N zJ3q(^=NEh6ey%eWH`K{rCPeWtNGD|x5sd=>v;yt6_`0M8Tu{2;h}?Pxzv?PoDEak0 zMFF~N)~PEQXf=ggnF|S@so~i1WY5OMdJx2&R;hxWMZ0hb7~igq&UJqrIqP#u_AJ19 zr18i>yr;IJ%k*KW*JW8Wx?RqX3b_<*J81U>oTW;xdY8%q);VZXpV%HQV!v#QKU~b> zP1JFMKuWcIVGQSr`KhKkaw}sBNuA<8TW5wx^cZB>SCF1KbU1NdsA}-2lk! ziujiwY-VyD%|ZJr&o8p02T7&X4QXltT2XbL!)(2~GIuj3s%kD!2HtmppFYv0l&;zKl4Qyv;PvR<{YKHV z8x=GK*{#$?i@nbIR!aK0_L0S{1yEUodl2XRBy*v7S4G=vmXxNev#6Y~V!_uPZt?74 zZ~mzV5Ajf4m+>#>M)a%z0-(!%O_ZzI+4f0vAU8Hk>C!8YWTuv6(rX3z+Y$D4cJZ3| z8K4%#d!A%35Q>G4>=%Uz%ME$;uXgvg@mqa>1^#ED3_rO2`*EC-#;pwp0$_tXPm*!K z5*bj9Qsf2r`im1K%la)j07TwPDG5L|dKCZVTd317&?NNZAjk>&aRTH4Y4{?!J9q8w z`z~~p@nt_`)cWOPIL9vp-v#5f+NbEA#_k_S>Zb5Jl64Ivl1uzIGtb{F_bv~`a&(zQ zi`T+}dXK1m!_FD3 zOzOobf#d$?<+*uZk@UWq7Yo1SCu&CN&JRb{)JAVG9O9qi9ZKgdCEs_k-c6zh>hJJY z2fojOvi^%(-4UC15?Nfx_^=l`YBt~(l6>R1l)OMcx{0@G*0xzQ1I#^(oMm$D;~;Z> z+agu4Isca0l(BNHc!yNN=oXB>=Mn$(c5;ubCTXqV=hBISAJE#(>jcWdLs-lAZ4GSd zY+4~Wl1z1FgJj;fFEx*~i=sb2e%d-+beq*~P>O14S`ipx=A;1F_N|HZW6a&PVH zB9?^NlgVfAC;5%c3!tre^j6F(x2KlNPV8zLu?S^ot>7B3$?|@N5@Nbf{|ZM2lWm6) zgY2?w%){4uxCZEq&V$m2w^BRfY>hk8QlhQfwEDZ>;&ybti z##!5j6EFQ4`n%A656yC4A?C@;WWMdL;i$uD65Log8R7$*hH>85uLG|U>w7wvkJwI~ zP5NVY)9=DvFuB)3`ZhxJtHR@_jtBvwV<%zD2cLZ?79WX!E}CocsBLL%qN)6fCrvTA zBRWeqI6dyM0|Gvl3?6Tse*1mRLj>X3IbpZQ!`8$z9HNu+P44?Zx^8em?KO?y?#HLH zR=czX_VJ?eD}7)%{rQ@|H<9O&PJH9yxGQT}lZy9ilC~8oTI*mEH+IpfhvZ%LWdU~3 z>i}8#>mXzP%f9AMaU&Uyz@n7=3$V4fGf3|Kdq<~f0D#r^RoIt}MDW<`D8Yg^kjE6W zeVc)=LNe<-_&9yC3DcAPw)FB_j?)((>i~fShuDGKp5lZhuqg>oz-B)6Ao4Qr`{}xO8Br=T56)NpAX_AM;t;w1ItfFZ@_1nRBW1b8grbsS6Q>M4f@z6 zS#|5v<_tbr{js&I^#}-@2V}f{MlbWspE#RlbUF2~OZXQeJnY;r*KXux&S7e}G-Caz z4>j_vaX;HNY+dz3HW*ta{#e!VpY8NX$lvPd)bMPj|@gzkP2pd0Rx^|Ma%FTXiE3@Dx1?-H~p51LS* zKDtk=dhRu_?Bg!6d)o%3ny$vjfESxO84%*lauK!K7Je$5BIN&9>qg&c`<9^H36X@f zA$GkY9^)!V@(O4+<9m{K>#I2KTjNv~djN7Cps7H@sVL_bzJ^%YiS2>m-@#qd>3;kdhbsbcr4kM#J9GcwF^qoc$V!a)3ecVGQ1&n z{zt(r-&Q2eFYMR~I69-qe@(o;wW7F(w<;t!D-}h)+zq<8?{#oHE#G+T!YzI5*Z`hz z?SK~NT?vAA<#v^Vpcy~AF^A$da6gt9lOH4{OOVL$`8nSTkktXAAnvB`gf7xXKZ=)iBB(Dh<=`F>4Hr^lSSo*q~Vj`m`@>R~#0I(^V!r`Kcga0TIbsqAA6ju@M7zyYq$hUw? zE=S^u!3opIm?q2i?MtRQhy*}{w?^;weHy!vHzX3mwX;XhLeOBj$D$7{{^=vFckL%N zsy=&OpJly{Ilkzak}|jYXQ?!;f4y-dJ;qAxyD$}aITm+>EWF`;Nmh?w>V{y7&ztOk z>C~u+s5(!b$PI0Nab%69G$Wnc6?NID#~0*Qnn7e=@VqF|x6!#F^a`}rmYfbe1!ZP4 z2mZh~PCGZSwlh&&!ce$X`qkI&)ZSV9!8tg}X!C)bScd>XrMt2~`102lFoJy%#P;{E z-}uLg527Zq|Mo5)eS4WN|4>aeIm}%eub-td5^RX*1vr@V{OP%b=NjC@+~XB=!TFf@ zO(Y|TVo{v^Nus*M90dxH`;0c>M~7x7`}Yy|MQAAVzjxHPOIYH)i5|a0s#<)-ho@RZ zw3X=lU={#v3|RF0@#KDv&2s0aOPId$VT2+MLwVv@ehxDgMu#%hHBQ%e^4TmFfU%M^ zOLqRFQZ;tVgyh}u^}Jy>HU0>Bhe2R$Rp%=6$s#*^|FBBiW(erMo>$ds)5S&MHfn^+)cnAtsWqggQEl6O6o-X~fmS7diAYr?ETfN;G7y70L&#v3!a;ZW;1dWKC6 z9Kml07&&x_No14XGFSH-(CCW(RK_?F``+O(I+r=3;jN=ftX^ySArVWU<4?n3+_+}4 z->`k;FXZVdkx5*~o7nCprbaQ+-KjQGbc~Gw13PBcLGRKR_wNr>uYFYOrUT0>rdFIh z?X1WA*a6=O0P6U{o_IDUxN)`-ZvIJEH-6@mHXC+#_<^Sy16xkyNTSG_q7I_cC2t!g zE40B2Us)P94 z(_C0X9-W8lLKG}WT^AkZn#*T#(#%|t{pq;ab=y2z?@ z6svB&1%bI}CQMB}>K?l4>8WML#g~jpf77aG$jZ(#Rzeaf|poQ^MYlM>%D!CxeE;5{K$O>*w7y zsEx>9YQdIU{gW@yI+8W_gHxR$=3lHt9vb4&W7e)9Jv({4J>n)eGfk_y!i~rqur5ZG zvc6P{aD%0$TbNxf?r&qfue)&)c3<@zQsoqK0J;kiG{gS(4P$=6Mb*GlDM#Wb$zY|D z7xWX&4OWj&U7XEzAuONpkF?^x za8Gd?{vlmpA8W9lY~&crRSdD}b%KNe!Zf|8p+8yU#6@ksu*Bm|(D5soC;W|MwETNL zG%3hF1!rejW>Dp=ckW&UcXs4@J;!O#e8~Il=5eHa1bv?L_3Fer%PM33hK|bZ*$r1X zTZA&~xu6`SHWO=iF&EtxgBG%5<<0$lY=?xgP~x72^#^>H^GBF)tv@%RE#s$uK;X3D zR`c|y3$7FW%umEzZ8mgfdWNRx9?c~qX{0nnpVZ#71Q`*RwastyM@RX*2f>z65r;Bd zOa@5$c=mfjiz3)f4DUY&+2Xo64ZUb(=q>86C{v|iG=Keav=oPQ|07s0QC+A;Txyh+ zzV^2=&DWp0zeLS6yBlj7KSge{h5@CDkw5wk8c{WK6X}EmXY8xvU~EHKhjs~6rAg$P zf`=yLNN_b57(bCHy(|nh(W>DiXKjA1A;ob6@m=%;ikD(c)TP7!@E6GwGOP}?ZF_cv z^qc&MR~l=Ym(iS+xiJ$C@#hIxSdq*!B#B+yPbP$D&;@3MxNFSQ$1DYGh&mURZz zhp>GNUzO>qUQQx@lVzZrfg~Tfk&Pt#hmyN@20Mwz#o}-Ho1s!Cp)hQiv>3?>sWIBz z6S}C6W;J?Tyb1>I1iF+NVCVHdNjSp(t8%q$7V{763514sO>A^KHu42eH8Y9G(gZCPj33tK}0Uq$@wP)3)Kr#F*9$X5tzuNpLkZ`k3oxlk_<_>gQxPx%o}kmCBp z(tneq^1)9cEyKf1iOvWVqH7}Ag3V`MDThW8XvDt3En^VkL}=$ndxq}9J0UHjHY3*Oz9NkYYVvAQ%#asi7qUzBNQs8F!mBN!o`(Wu2q$oc@O*a65eRB1_G^ge zA&ic2euXSjqPQr_2+fGLzYSoOIW+J^L_dH#K{|Ys4dp38;UlwUn;|Tul9~&>*|_JL zDUoBwnb9d^kaCi82oVVp_!L;EMhddU7*Xu_F_Z6G^~`#eNuM(45G+Lm^I&(O@Hw>g z%{8180&s}3+>7;Fl;X-I`UYhgo)GxX;)gPZ-~_*n-5ex%HENME?L9^^LtJ<-_2EV; zI0Xh(GnOsWjCbLGwP3`F4t%4AvW&q;Vhg4SmBO0wEL4-q33&+l+l_V(gO)!b-<=gW%*v>fYOECZY>X?s|Byx5m&ILa{5Kp+R0CD*c zw#xjZCXhlA*%yiIuLyC8koF7xN*NOP#tAW7jQk?mr5j=R4AgJ-?I+UVe+^CqYjEsz zgynE|kc^1$F%BOQtP@bcT*roi#rZotg?c7Jqp%MFhAtoS#VyM=j_o*LLxaP>POe z>dlGN6K?aUa0*aZDV5V3a`PtpkzpSx8%p$tX-1(?TxyPpd9tqXX5PhV<@4=0SpY*~| zUuYJvw2XlafxxwgZ}(merQuT6ryKCKs+)b+?GkYd3tQ5_y+icKbY;8=h~Px(UO1d2 zYJ7{$RwCur_IkR5?dFs;jcKY`LZZo8%2PrC(K2B#Nl|(Y>Po{U&g?=(ZWaRHgq)+q z`!g{nN`meuw|tg`JFnE#4Tbx+X}T0PxQ_2y*@4+KnLfi83D(jIO}+|RXU})y`~VhW zW2FD2EZzjLL7~b2vazU_;t{gyV{v^HYXHhIBJp8EpQi3kO|z#T8R-6(WO+}GhWH!t znSo~TCkVlk5Zn|7X)^G|o-V?g8+D(2(_xrmRCWTw_&fS%cr7H=AUqQ#Cw@FLG|)lr z{>bmre%sAa9R4CWMUt}gOWR+pkD#;ILA@lFB(N_~Hqd?1h-?PRUz3hGLjvviCNRGI z?)tPax_|{8n(WHaw`Dw9`^YK#iGc+am?FLGKYMkHhU**-MbiEl{ONB9YLpRecWc3& z74B(>>rGdUHM-f)L4()wyMyJ6DWhZ)|1V6Zr=j0&cY}Mbm-5p zyl57U1CUZ3#ERQ4#GEP47p%mbca3{`!UO~&L1kRDr{9y0|GG3S6dQ7L9;reac7FEal-V(9AUE`VJV6oib4@V zf;_mF7R0)#r?c7D5r8ztA8PF9x;k0rXM>&1*t|=@1@0c0ZPMwAzp268^lCRtMBh=)Br5FDzfou?B;qMdO#(OITiAB0$fHP+A1PyJquywO^0 zmfsLaz=5%zBVcu{?7P?+nlS_JX}k9hf`8&bg7aLD`vPdY5x`~`QI~$={vT{jVeD!p|XN*vn0#3jiT6_k$B<-kb;P*Wj? zcLr;WyA{oz5JJbJbnTAd_P19I_u!QHh23e=Yz-V4Uz7-(q`&EN#w0+S#9x9-#nYSN zrzB(4bP&A5f!@anmzhH&O_HXtq=vai0?&l)Z}J4fPZklzXLyY&W~P zJEabtu@qs8gWM?JUJ?KyRNKkVub1a&Gj}y&C+dP9&c)kV`M`R1M|Ff+OiQ zjEk1#r(Z@Om|n*}6Yl)Q{iE?IM?~T7tpdXI32X|cvn|sx#%DKAO7O}f8pw_Kj^H`M zEHr@WIf63^TBi2pks(Tyw%aZ3Z$U_~9a`enBg+fE8>LP32sd!~DqV}PEivjX)U*gh~nl6)@hB{CMLY4o4~7L|}Nr)A0ey3UmiYU%#a2GY$36}%u*)qOUgJ+b$ z#)3kG+oSjoEm6&D9xkV!4GkAhg-`Ie9w9&m(qR}du$w`7{MSm>R?>0|^*w0Y^mpr{ zv4*2b;He4s(U$Y~A0_5}IdB&*dCS+G%V)b5bvb~Ks%Ij3bRv*)<|mHptB`}kye>w` z-t9Zvrl}!+uPC+Fz=rEPKh7~pA`Tqya}fhsN9E~%^lP|Xab2UG;P&(pKV4YR?g%Xz15qu;PZnfYF!R zhI;Bc(#b8!?^uEh!!q+L^I7>Vl4GU4RH!-;EliiM*2S-*`lB8w#YTh6plvD|&gd2# z=c2{GqY5@rlls42;qI|hULAY`z;|7t$1m{IHg@gno|Kyq9~$a|9L_o5q5W6q-M)Zx zE=lBOCFp@?DukLdxb=xsy^Y{{3gVd=B1rJk8q$UzEY%+r0V0NCgS(K7c|0!I&IfaZ zyi%U_%Urv|x0whe(<&w@O?xBNAP1zo9-TV4~ipRRoGGfxdsBEz9215NEgzo!rkK#h%Q_1PG6=oxT?<-?u<4NCeDHV){u>I zK2scg025gkbnn#bX8N-bO0aHBMo20?O~|Ev)%HVz>1>ogB6ep$IT5?RLO~Ip zRO!xF>)>-taM~+zEmOfyeytP@x_dJCQ?rpu-F_o9GuDlk2p|(C8Uls!{-xTD`U5J~ zs0l7yiv4YgToE8JnUC?ED<`;Hyi*caPRM%9wZa`IOsoW^VIfSV@<-t|V-&AIfsaM; z@OEzHmtB1yi97k@N1-*@J1)4){rsbka2j-C^8Hv5;wmC>2c#dUPh< zwZ)d<_ZHl7#y(Ky2FpIV6|M28M~matO2$HGH6o5&$^u`Kpc63+uV4>QS)V=heC|5*W4In9B#{(U^aHtm-ppQDBfoM0d zJL>>62s?W&qgDqazC>QdiL?41ECBuPMOu~itq!rleTfTU!h87|@^}Whs{gO=8N-wjlOJq^V2!T{S9k3$ z@)f#qSN_6l+RuyHr`?~1TEMXJ;+1R4w(%okof!tJ{(q?YbDMyB;PU~d^2txZ_QFBg zP&K(8&%PI{O{z0;(!|Z0E0Y?4a0CFsc(Oyur2PlDo1(lzRLX&;;khq&4sQTkHT$>n zZiJ??VI_LXu@If0SE%&wx>~l>*wk--?x?3-_|CBnaVu`4m^=BhZI%3mPN{|~JCpXi zQ|3Zt{QHo7Ujh`|C8WBuopH`>wC*&9I6ttRzMgAfU1OXRxNLh8&Z+=0rb?ayvkY?a zv9)ymOtUDl67rq|vnF!41kb7aXhr8u%t1w&xIL-+jzual!~`btSL#JL_(5^|j742B zr%VqFvjlQmye$7e3jM!Y0{(YXYUvWQD*vBO{lDIC_*{&M*CU(OYU(NXJc^Ccg!_PX zaHrBPv;2kj1Scvz@`f(FqksH7*6v`7!A|=++uddWa4(!2wj~}j{6(^gNw7+}0aH*0nhDr8z^Fn{$ z-33yFE-@s2GwqncZ&(*;=cYTgs0x=!jnz|6Pggc9x8_}hR6>XdEyskCw4WVqiRr=T zS7<~#@Ox(-x4gcwo6_vpljLhIn;S*VvWAO}5EawdD7ski#2H$BgMFZvsa9opg@{HB zjG%^v`M0EQN|dz)3%!)$M{L~`3R#I!V%?M_tL`5A^;>m>b~S>TZL>T<9e+*qX`ork z&ET$@u~&q%ve_Um7r6dVf;N~ef>WiFhcf&Ce3&q!9n#;^&8WKM8UYc~N_2kz$aL6U z_khK!yYnY`@nv4-DL=t=nVf{9ZbXC5zYDj|&s)gJ2$x!tB{$t|hXcI4gfCWvR)%$rZI_kzRvkBWyJ2(x4NL|c+f zz|eHz9hnVjpIVGd@m;IQmmT>RBHCby;py_b@$kVNESV=k`n4PTz->~pp8JOJGq<(xX|II^tV86rzZB<*2p{z}^QUd{xZ5Gc>KByE%ADpWAK7-SoqF5y`Oa zLq04q^T?h{xO~*;>(jt|AMjQ#CX}*Ge#kQHQcsAQA~zX)-o7D5kTi<2m+TrSUDj;eGAikf*dLto9(p^MAQUO)7 zBIK^VFb{1s8s$V$NP=#&lSP3XmU9boSFs_4J*tQ@3X43Hls=HVZxW?1k$_chu9hvmihr|N(H43L(H>aE(SD_HKorl*AuFLRMk ztAkKE7AyVsr**H3SH4mQT;O3vAkXtQt=r$(M*z}?Uz^WaH8=5%*`5ThFAS;&*M@gW z07VCjB0h0Oz!ZTDzyXL);^2AM%&p%~6s8VxiuhA=Na6~Dn8MvwAK*)*i~OyNsNPHS3}i`6gtYqu68BUPUY%UTIaD*4 z0S@H}^8-EbyoZiRSt>QXvAB=!`5tN=!xPue68E$Qjcr+lC}TZGN#D&C{mP zVKlr%GR#f&VW1kV4ef3)VgbE98CKmh##PRZw_BCLU!%74@$#kbBvH+%r;j?+D)x-t zn~G<4e}fXkyi+9tGF@{*3SbU0snI@x;dD4)nm+Aj{cLFZ(@wfi(qLZ1Kqd$_%wD%a zzaqm-I{_3E)@p7nK#c12`9omz_9EbLd;&LdM}= zm|=EbNgyony|EXS`H#*t-D$4ZVn$SEJuq}f86pZKBbM`~a`f>eV7~|Y zH&MPwgcVq?dU?gI&CjfU0M==Z#m!&)*cT)0338zzO$+MJmT^@Qok9aQbPfpCL!$D^MBFDhypziY~S z9x&iTJfG~}t9oY)7Dd(1RKkFvQ%QhkCf}~#57NHImH09OOgO;&IiCIk!)*R?C;q*J zRJ%X*kIt>}SbB}Ug@v?0a!8bsFc2xC4!ihh<-JB0tH$2rE8{&fcs>BhzDHcS-Ct6= zN1-jtl>*{cvKKD522!>SDsSDgiR}*Jjkpn)_y6PZalVc;Pi`w|%EJa%0*3yRenwwD z)DW%?VCdHE2Qk3vS66wkhL;4$_(p{kFv8H;%T;?mNrFCUyj|63?=(m)-Nee(Z=;Aq z%iMp;UX`|^f*f|-s6je=T6RNr!#GKpD`!lq9~ zsC_Z6*S*TZkg9@>w~sS2J9#1rwiISpp@K>Lm&$OGvS`L&lVj*psH0 zB|20!)-|%QgQKz60JHitl77I9psVjiJt0Y2&#Shh4baiX`u@gwG45FxAoN-1^Zlh4 zRC0gA!xLI6;yYY1c3tle5VkyinyS9Mej8E#Kia4+bF;7Cv4hjE-hUR<5&g;~()iRN zawkW%DwO?zkoeXMc(Mlf+`5@QQ>$h{4Zjz(;jKwu7v5Yhc>R8M_1r$c+p=Ao&WQ+? zpb>7&=N|uZZCv|Sl1aF^y!?5a=|hLTu*QVW0ZcgTHZ_pJGS$fHZbN#Z#c>J>XqCU+^H4D5y{IF-Z!_|g2t;}I6A)bJ z8wh%`8JRvycmhE$8EO0G#07 zP`^Ng3&4!@QIhPwFX8D&SAEjP(;ibwUqEK->-&k=Gq(AFuwi$TjG~j@Wh#wj4qcO< z8uA!qC{0Zy@S5hz^<~DY-|ihAKK15NN-}DWa+`kBa7uO zo7a>@_=!P+7oQlr&u__Cv|voG?qV*WQ-KQ~?d4n{lunN4<=bKbg(fOWD33HoleawQ z^a1j7eGf+O^aL-=E^EzZE8AL`B2Yy+2x0$uV6M_vn&^c2(ehYwP`*d9iC84gMH?TA zA+s!0_*Gl(IwTqmp>k8fcqVP*sYpTd7M28oBP)jTgs(E%Srppja{m6xjNvR8fLJB_ zVLY>dg30^vTk?6XIj$J;dp_*;=JDN1WCPY=CXT0e3R8SXcuh)`_^F|rUyJSHH05SX zawI+%va%yjjmL&%jYSv<;l`O}uN2@)4aceS9%dZN%t;w`PzX>jBbsWN`GsXeK9f2r zXbV2HNLQba#6Qq;$wn@aSD}|LV-;M@@n4@aiIYr4dPf4_S7)3II}m zkUw7^!3TI~j5$KQ;@rrUI^??uT1+arU9KiA?l1f)aJVK5Ixp(I#m<4`#NcPMyvbkU511#(%&e4Z{bjlY(`^wCs#!0ic!5I8}|j^N++JR zWWIirjn=~Ppm_E1Eu%i}ub8&Nt?fdaTKz~$^)#HONBdi`A4h&KmF3@?$Sy;KFoHX{ ztH)M+=?r$Xl8eMkZpoR*OXTT!sm5@hAq_(eZVCwoe1cTA_-7$XL75{FS2@nQ6UP3@>(AL=cUb(i+=Q!mt%Wu_M zYXT8r|Lke9B$J^Zoc#HlssBqw_ySk<6y7#h zCPj%D-E}XqGV2lVBpVE31bAECG0uW?-sV;H#aln!O@|0*I2NK4F@Iq9{lMCoqcd4u zdvBaK?mREw9o%zgF(zud1GIT<1-%P)qZBmZ+ zTkE^)-mr+3~J%FOufHX8{bxGUnv|6tBzl_QS;=! zWJ#f z)hYMYNcEJmm}N{^9=wh1RJMQ_G+t$O@<@f)&#)=!yV11g0^!jN8PN>sGB|OUs_LM< ztS5It#Q{@!q_rcb^-4ST!=hZv`%X6CRMBLi@Qe)GgkoSzGz~)AF*aY=67Sw6(TJjR z7`LuF{8U&T?wzK=sA<_Bhk1GYmWf>TXYFBVx52i?K}<#Uy{bUYY}s5#BunD=VxE-s zbDh4kk3IMrrBI^5`1(m%ULg@5x$z;q5rI^edC+xGy1=}VUs8kJlo><_!5ll zq>9EI(>##RvReczoKPSBR6h_klu@Y=|AV1>!$&`1xF)+Lkj`maHO!&zh@^>>;QgL{ zUF-bTb_gZx1&``3Lc($OQ?&Ktn{H#&y6}8vxP-1mzH)w=E5SRAiJ3T1!oLeGxQ{=bS zYQHC;AaGtu-$wWuywOfY8(G7b{pGVj80BdMcXS}(WjmWyRaj9N5>h;-Y;Ez68d854 zTb@GCNxiA6%(a$FwK^)VG2$pVv#Kq!GRgih$xfe-SLKt+w;ksjJ7oDb@d+Yq+0Eg6 zk&&fVR)gy`u6Khy)z( z<3+7@&VD9kA*CoCssdNX_0sC34z@|EcLMSQ?_hH4z}3-YUZ2s5uwzebj@i|OEc$pjHkr%PU8ix z$O2NZ#cjGVT5^NDei#r^)F$J8Cut0{cGZ9F$NQO6<8y>aPN(}wQ<27*6=M=*Sg?rO zxwn^fm}IO}7hh!)cj&0tM4$Whdwg?p&&Hn&j;EGbZO=SEZhlR8lUwGHZ^**e?)v}x{QTr89$$SDNlp& zdE8Jtg{QA6(d=vPyJPA_st#ISc~xBV^KoR&eF@+&$Ly5P;^Blf?XYIfC>0KljG6xA}1@qDe;L!}Z zA>p1yyF~Gym33h2h}~e6k)*lWCb?QoJzf;>x5ILjS&{|~VuLYA#%yxz zRN=ozQZ0U8SBjL%QIxXo zqWXpf)O!m}!Z8m~FAl*VOrOJG4}}2#GXeNt;pKmiL1I;=6aKIavYbenTy1b9nbXQV zXqnGuSlF2x)3BL9A;nuPAf4_miH2xpYX|-AtGVhJ65)qygt*xC+65%yyda|znqc~6HTYBZKBOXRxW0gDHORTP*88vMe8<|gl7=$4$I z`x)z~Ve(I!CnaJQ>UpE;#+asJy*1LvC;V5_gD7p9wFTPnJoj+l@zAoK7_nSp+Mw`;qX%N|oI}i`HOwBAjvkEY8XZ4YA|9&DzKS zfzvIx!NsKnCu6V`dE^?~xTb0s`GfmJ(9$9#)cTubt-n??U##YKMa9s)|#rpka9+8P&Z!qLI@z#Xs5K z&yCSTx3TtmD;Hcd(-KPLhg%t^!v*cJEPnK1TLH1 zH)MMl4-%;?;Q;;*VRuApZq*~^B~_^W?-YCF)Y(jbbsJCE8I9| z(XS8dC9GsCh|U)D48s-n2YWdGJvRSW$He{L`)u3(dkQUh#Q&a~OwH z^s0%%a5YE@K1HgZHGUtb=I;|90mIJMa%P+zjamMki}pR&(-|4wGyFZ<+{at2?lYg~ z2bN&=Lqz-x(FDFB6|%0rhYcD55EasK5m*HF4PnCQ}Ba;h{IC8}Yo8o$$LFc-fp7oIeVhr|RE zo@4oVjHORmv4ALZPY%frT#Z)zBCHy7n{c!OSuJ0yyK?Y;h&>d1qZewky5vF0W8QOh zCb1hNy`^%klC~NnV@6RT{jR04{fn=`t#H+6Lkw6wqzqYRnJFR>R+^1lmtig|5Z%ny z%3J+BqC%RIZ;14H%(gPkvhHWuo(n>qu8TQ}zY*ao!NWZ*><0X+!NZL_LqUmF)+^Y# zUU-#-J8TkJ;NeZH`XVjtpqoZ1w+jaH37%(|JDp&KwT$#5gy zCQwj9n;?ywwV1vp50`>l;8>Iv#8Gr3V7sWc*{tJnjEk=tu1<7op%Qfyd&)Clez0$< z!3U>g(NO|U%+^&{v6T~*YB(1i%%tW;$^$&)p?kC63g^>F>NL5FfqrkNSGFnMS?l0G z27H!Vcp4l=@_I6y`Rd$OuVi)Q+K@gNddX~TV%MwE<-uCSo@q^Lq_IGjLXU>d4kXQ* zH)^raWnk#59GKNcs(|L{MRW}bPwyFwRtME5GA2vy;|eA@<7OZMm)BAvve>$v-+||b z7xkZU2JORePo_}VzO7dXU1}=Im>@xx~d9<%x0$4Hnj;k7DumS&*ER6(r#Q~w&Uw``#H&de)YGK z`$+Vs2VYBj$n`5qFF5d}gj<{nn*(#FU}l^II-*;5++b!VxMH!$eB$(bW{9?E_aTN& zWp>2;yz`=yxXAa(JL!s$-cjWyxPro7I@CQHQw)V5<;-btH}U^i<_3pE3l$8NfQW04 z_6GD*Zoj`N49DFVhYpj3si>jzwDa_f21Pe=jb!B^e`oQFycdm#&f7g^@dF0vkAGFy zmJQKUdz$me)b6)8(do61V8H1aux9a#`(xqT;0VJlFjJt(sQyqf=x85=V!spS-j2KU zY=toFzZ{S50+n>o2Z_6YEVVv;`w>9Qv|hyJq+ifRDtD=zQ0Oqs#`?Hcf~-0L;vTGX z2qiO0@98oqRMgo{e&tv%T?X>Kds+_EZKO7|=xOzhesqc@a-_CQJN#*%o(&BwV+KGa z30cE>T68U!l@^N6^@khU_C;E8>|1{P74O^$oUWU@dZ~3iEg|9=6X#pM&;=m2!&aS8 zGo8{+tDZ%BOtyiF0(a@^P!F7xLVuHIF#m)VgakXF{kT*EL^Y9#CIrz>0&L+z9DNjm_jDR)x-^fjFrvP5yxgDFMZ#xg zspJmC3VqVLI%dX`^RWvR)E2j!i6fJ%q)EXK4h(X4CDn&X4$&5)=^tygBe%!g4eJ|u81zu@6{ew3)ao?eBbKl`e&X2RwAuhd> zn!h)*bM5QO&i+Mv%faGA!FnZF!L-knv0GqGz&UVD#`{)+M&&A(FGH=YQ_7EMC?AvF z6DDpGCanI`4G7)Qg6uJGu&QeQ-`CnuiE5j&Mdg51qIAe&B)bVw+Ll}*oVAqNWI~lg z+R`V(SzgqoUEfMOU-WZ2_Q3|pX0Uli zyUKJvks**W4 zOJQg%XuybqI&y{lb)(PIu*!S~_s)pta+8v*Au#|A)_XV9K|wrS_{$bup zQZ2Bp8p1IPreRKoYdZRmbi5OB7Zg%~k|d2=rkdF7FTS3r|Ed%o>{c`D>?t>m@8T~a0O&8naUpT)Kuj3W*a;F zS>%uv)VzH3emA9LYW<5(<_52H4>M^@x%2ETpKsS(EqN6rKX6Q0fcc1~aWfjFp#eU` z|F~kyZ$}ld{fe*tLW7H>jO?v1w4b4Dv7y$ua!oKYWeyFLr_b&EM#tj2EE|P!gVg^- z(FKwmdzYu{CBsC+pr4Fru0|)0j9^NtOZm+|`8~yxxz{AR4)(%0Wo4$)(SK>@$Nes~ zS`?{F{oM~yxM?t#yI|?tR=AGSRiW`zD##G`VdNqW|9B5NCSqCBvGj02*jJk?ge`^K zSS>YLQybWq4#F6e@Pn5h_v+tnyD?+O4XOHMV&i{hS|ATk`H&WgXvFfGx zvhJ|HTuU@yV66^>wATLzF1H%vaRUB>p<7=BX>Y&%2RZZhPC5*K0|S%%o~D?^LXr0B zgNF=+=y27+hcykeD~0{G3w_&9qE{&X15i$N_Bx0|t}HuZVWtxiCm1L%WChp@K-wun zcF&gd6dq2!K4Lv)=>B*=K4t`qt8a~jYdajeg#LCV+7{43%pqbEvSMbD=ak~dHWbkm zXlPCrB?EUXU|NzyGXa;AHVl2)5X61NEwTeNn=kbZy(ISpNh7nxl}YFGS>)tAbcO6*tD%=CSz z2IJsrc_~q#U@fb3k*q_&+C?Lo0!!pnDvA7Nj``;y`syDLP4(geXa7{BRCo?{homu3 z)M7-V&ng-lkNL_l22wpMr(i#Vb?A}Jn^OQ2R0Nq;@LaNd56NlWc7nD5J=Elo$lHJI z{%Jh<84CYP9k&O7GUIG$`>$zx1hYmREEG9LZiEvqzI56WfcG${QPhexh8w?4mh(WE z9rQ?;TQv6?d(;Y>9{<1wEl?!#v~;HD5csPNUM026#6`bvIPeC?FkOC*m$aLe%`d;% zB#KYu25`A!J%l||4>wFCf6tN|CuIfi3sNZXYx#0K$-rxO}YvJk3pAO#Hs3P)A| zJYC26mCX-AG41B&7NeRaBgSSZLjKuA!4d#pWsQVgR?Bc|uL67!7{a+7Os$YW!_Pm!=^@IZ)Sq_ey~ zE#sY*5dV1(Lax0&mYMJ?dG_-z94gN1Mm1O9sBfRpXDX_89|kG+U`XEchONjs|ot{N)q<(->i+xX6x;^&P)7T(W?JqbFl!92go14J&6>cgWYkM;P|xXzQkpVf~7cB$zIsRLr^ToP!3!sTLcF!@r{@ z3}U15?KX!irK80b>w^5>T2+6bNc#w^d}Nu5CHCLJSr7KQs;d_3Sd$`_X*w4Ds9QP9 z6Gnx-Fw&C?DwZV*mn%Ar;OlLmDNwx&ZwVU~4G^G6dCzzs+Avpz==Q_Ob#p8&&#Hf! z7r0)9yc|po3hw1?uN-O<}LYor$%4cLhkNfbt@;1@g=P4A3thrWqhVh zh)SbXRABJZZ=hprfaY7x&yJI@kwz2166z71#}npi89SKB0}?^FvU7P$4>Lo%kX~Ve zsKhS-N=axX^q2o_v@gqc^B_F4!)GSRSWSd_c;dr|*B8wD`AvsC<54M8oE>>B z9}}N6WH$rxg4LlKT?xQ-H7iA)#L1ER*gsO=#!oD|WC$(H{}S-zbh=!R2X(Y%l`RSZ znkSmfkC-6c zzt6|$Q@^nL6&awlqrYJa0oK#AgO*^IcXR;Oa~H$c=x>BTg*v3`0>b%%jd}W!)E$)% zH;4h+ecr?lY8Q@Dm{Vfrl+vffn&(HL=u$8v>XI*UL&O@TzCKg+e1t(&&xr@g>+1N zB1lPYHLS}J6%OaXngoYoGdu*_vV#iHc8usnLY~MeV23p!Jp{{p95El1U zIlO3?vc2(@6$kHcWyas7r;D|yly3j-+iga#rP<#tQYk` zy~)F#%B&w!OXr}g1CmgHE8jnnF4pli!pfVQ4v;y%;2ahh#zS5vXC{WjCxwL7?tvsh zEPlmUf^pVgx#L#+Q|L6(oIR)Fq6`Jb#QT^Bg*d8X7p;i|L>q29c8N3mLrKa;d~X?k z74CluftM>n;av|L2q8N_@{u1L7=%K7#9k<~;q4X>R^jjX7i_CQbrkP!F{YDxq89;~ zLg<8vHv`s~hFEjAkd*yjA}-0q!mPx%g`~R}CHplpeNnqgXpC~K0p~D`%&2;fJbKo! zxw~$;jz8evUC`|LBa4H^!)~phcBP>%68!~&mRQ<0*wW4M7r@Zi8o=ulZgSc8}M!3%_Gevn3xirgfee!fu z!3B8%Pe&8Ab8SdUP1_xv6*rO<+Z@%nCWl!;S4Ul!Y>6(O#!2s=V%%Rfzv)7IWV^Ot zEb@L}5OX1fUqQZJvT--n5V&IK{h)^BvKwqg)N(zHSaMI2DP#w9tpct_G+>#NWrPLT zh*3IamhLg!+!)`)y)eid_r+{jo6Ms-KJ`RWC;Km7_|7{Xfdn&gW3|V569_n`ZzP6J z2mCUJ{BZyI4MW>y_*1(KmbIJWf=nquraVB9DI`U>W;%(PaVFPyRtS(H1pa1WIzze< z$TajZCb@Z|VR5{v#H{po(qRHZhV%9;XH&@vRFb^e&ux`!S!_yth*QVtfPDzHQ&n|C zG;|Sl7P~!djsgmpPRvAO$7sGgH@>-HH1^>ALonFT9$I)-xE~b>hXtzCQ5|wOv;w*+ zrdaz0b*#O&f&36_dxpsoyFYyL?T{Z;f-LCM0zm<5-zJUA|F%ds!5T}|3^_!RQ=$D* zq{d~^q#KZ_$9FXXU`D1`pjFKhXV!%r5!XYa)MLWTMRyef)FWNz=io+RY+aVSi8#mG zt{ZhgGiwH~lOnn17rnx#Ez%wo9YDUTKRe{DFpg^zR8Qk7xIERbQdR6mrB{{us}gr&?-7=2{7SY+0=GO z*pf`MIsALsicD4hVo-RJB@C7D2v|6#l0Z54gitU88T*> zJMQq3SQlrg3|nw4o7Bc&TWwn}pW05W1hfH1iwz-gJ)&fT`@T?HB~3;kfL1v@DZt zSw7D4nA;EO1Z8LNZf=k(hTwRk;H8-ct;#{xWRJ?pw%N|FF$kTpp}_iw=pu7m{)N+u zB69}*`;!d$84@paFU*C;;M|gjP_c9Ml4j#QuAy^?wV|Sx zZeWik_DBKt9!4cNQHdqW`wO#|0K@xT#D7Or{g2u6zwIggA~dv0iM~yNP~!OK79du?)+77ib4)bcn+if2Lg_0peV5>hjhthI`>3VjqM+ajt#$H|eVBteOsQGwZ;a*EJ4u)SL`&7uU|Q(_DX;MTBhbYl)G}DUm1=Pc{^PxBFAK9^Dd? zps}c1wC)v#vT7qb`NSW@lEm7TJzu-bg0ODO0W4Hps6WSFXLsv>|C_oXBn%%wu>i9Y zdVU2nsIvf@K8XBPX6gsFOb6*KGLc0hI6HjEoT2j+PIjSk>pC4NYPfifpw6el6j@$R zf0OJp-LRTkGl%UT_p^Xxm2RsS$D{+c4Vf^x(43)nwcmE?ANzh z+3|L^%cP|qu}DYlITRskxbJw;Zlp#lK;ok5h&i=p1emY7sYV@8T`FO|lg32TAI9KXn|R!G>H^t28uAGyy3mQjxLPUF``b z-@#4u9b!<;!8A;Ta6t8?@yWVVXr)b15N2tZrRTi_9y$D$gHLcZZXSsRcI2xvP#wt? zck1X69lhF-sfaUEHx#WXS`A*C&RCZUmm;>NX~TjJorEIvt{|bAa)U@$mHMrYWHr2T zG&;~M?T1wiVQOJ;GQKsJ-|($-tiHor;=VdQqtjdFK5FcCJpW^dh`0t_CfyB37D3C^ zFbSmb)Tcgj;=KN+uLgCBl#p{KupNUl)6j6~BAN^6mk59P(1{Y9h!}>tzQTS{ZS-iF zNJ(HcL}@))4%#YRn99e-ADaHG6`Q5<)y@VBgD3>=9By7#Yz!)w8lWS!ohlX;)KvU) zqapcSE`uZhMKqbb+VMvbAl>az{(`8*Y`0rf|JT7~EpJQo`;V-Q0(1u`(QEA{-JC?RQF$LrLx}4+CBjQXq zc6%YrHdWwM}K{u}x@*Q0I3c;&J&4 z+=q0DEBKWXV4*;}IU!a9>l8m+Ds;_jf!ycO-g!d~XtcXa`5g7F#}?aGvFH4y6~~X) zOw*5Yua#i^hWFAB^%B+WEoT3k=7YFNjiQ9nvE##n4&RNW$phmPI5p(y~G?_hw>&X>>bJ|X2ew_OH$W`o(&b($$M~j>6VS!pk?8*>i{hjG(UgC z5dh$$hW+7aXida`G_+>qDreBOsY=W#xXvMEBULuww91;?S>+8@ zccrLkgz41{)YXB8`z!R)_7Mt9DYL&!7ev9-2TiDPx05th810e2Lk*X<9zY{=sK8GC zz}6QqL&*L+Bp_ebi(BuN|D8+MR1oU4{juBf-Kmq-$`l=83g~GQjdsP8{dZ4wI@%-U! zsWN~RwgMh88McyMudxeq-hXn=p>LJLp^(g6Ql-k{thMK|uRg4NGRi<hL<%8R?g?%iV$`Yuq0*RMakmWHMPf z2%Y+gxfK1R7Yl$k!t{4cr(;+aO^G~be$4g*K5E3^XT#NHDTvzp%P!&m;md5@P}hrR zeO5fMke1U_wBvMgPQ_GLSN|Ey_wO)I!2EY=)uH8I{vG+=q>iSx>K8Byb)rD`387@j zUxP&w0#@WBY-N1g9vAUbqtpi+~luOo$~J=L0D#G!9aIC2fZSmjQ|< z16saBt0A!)=`$l8lA874)?K9+vJy1a9~HLH3G^5y-gt^bOcp9J+6{n9HhZW!p@UAj zQY=Ror*!;v(~h8ly_3B8K^8;uT>CsRIgNwsJSZikS^H zM0^xp=X$F1q3z6Il)x=6`)7lL^^A`8oYiLXgMpf3o>j-gHqb#&XN9!85*Z8Tm`~Ct zgl+o~_a|&zDi9rj`!i5U>QUH|`q5Q{e=<-S2^P_y^oSpoh!K5X>jUMYcJWRO82^}m z^5geXr$oINygf4MkWik_>A8@U!qTD$l`rZ(j-8kyBSn)-5#342%bA$xFC>a510~L; zeD!;)$|K(3G0|Bv<-l>FWaz20UjI^jAu&c8Ew&Oo<)H3yJsInas)YI!{u?y!**{S@ z^d)8C1sP$q&XJbi%B(;V3S{72K&UdnJ)-c!U96EXQl%FYZ+h8by}2N>Uwf)Mw6hme zS~TUr@n&QMo3EhEC;XASu@DkwA|$T%VwqF@mtO0INs5FrqG5zecayv)V`{Ss9PbSD zztM!uI6mgy@(Lsezp$lV;wQX-^Hlg=HqlKwcksxQfj{ z81$zFs@XDS|2*+N{;)tVYJej`$erg&aWXSCam&I2q7Ee7tHHiBmj@&Nr38e#g2k4+ z&|hT0{yw@|7_~s|5btQ9uLq(12;AU1$yQ971H0a^_YKYOAt{{!CkFy$$M3WwQ!d-v zGnjjrZX8!+QAkX;->OhrfsL+hLCt#%%K;s$ZW{wO8xw_XahDWcjC1NJz6glpRK8&3 zCg%K4d$7_o9MUbm#k%C)uw$9-^DFlcF66%h<6PACbxo4u*iuGDct+9GlY9kGX4*ol zgNuKY0-2r?NaO$I+Y?npxvjd@{ zUJhWbL)_*g+{#@geT0jAkWZsv-UWllVAQ$tB!xY?nYW6E$Ts=eS4NqS#Pens zUaqU9fnNVuarm>@1iM~jmT_FQW%g!o-GT5}9(b$KGwh-CP)7t#?6rSWKX!oKE_^6_ z4&aR^PFVppEMz8QF#jZQoB$&eundP2He={56Yn<2ZL5@T)X$*0muhv)#C5nV9XiOX zYz=3KJc`dwQhd)eLmFL#-dPqApXEkw5$6C*!n8aLP){*5OA{vFgiD(2w&J}7^KWQY z?6}~akLs$s!c!BBzSog(U=8qLTrFZ=9&35(*ld}7NXxCDZJM5&-vw4Hby}0nDJ~74 z7-NBT8a^g!zKP>#YAPIL=e*!bO)iy`*(A+x2-3?hE}zi0Q<3zYqkY8$o>1j=_n0vi z7AhSc1@}7+uJNUj&v3+xMg^!nV2>IK4-^MbZ?HN1s@WKB3ZP(Pbphrk=KkFxbnd?X z3%~!VFNfgy-CYpppmHwIaOc=+2L1bKR_fbQ9F2(6Z!NZ)deEfo0KV`OH!mfG@+KYV%QpRNi6FQo_u?lyz#D^6foi)hr48> zIDx-`q{daIESn9u7qp;eYo~Jfri+q_W9Vw`)+dmE#G07E?&I$PZ`C0C%1-Qs<{Mxu zyhgmZNmS^fUg)ymoG{B6qE{U>s$aS&*WVS{FH5z-zo=G-C<;vdhyMdbDkz_O<-8NqILxm_7z-)kbBEz$g|*h8UDaFu;qn(7zzp89i&rTHRE*Q~n3{R>^9c)5MzN*Rb$`2oxKs z@V$a0MvNAuOnP;?;y(7Ap4i;%`FwwL2UEZC1;2b*>}`fPV_FO5h$c_yYYRC4XCiD* zfo9Q*z%O%p33NeaZOa&>vpd~o<=pCaf0|Jcc zF;lYOvUw}{7)1`(=f`A_YYjKtE**pos4zU&mP-@*fa6PjbCT;Rc;hn{Bd5BU;WHlA zCTp3^y3Gn}I;tOuI<8cfGhUvjbicqIdq|_h0KCX;PjaEGlUSR7kh{*kwV!agp~(oZ zSx~v|?S{c)#Bd;fdVFLaHOBP_YIpB=22=*1dZe!zw`ytWQ-V&N%1eKpE5{C`3t#u@ z;B%jX1DR)t3-&eG#yDreH^UZ;7o~y9;3P(hZ#)VpSB^i^emz^}BEys|M z4dBC>u)^E4Nk-CGt#hT07|lhc7{_kKgQq?nmNt2A2M%7 z1@x`h*C`IAKTSqw93E4#mkpcth!w#<3gsby8e2WH*IeBAR`eqX&^-0BXUFDRj0hi) z%-KaRZm`mKVa80?D6#gU^-TkLGcbGm_%~usxT6`7^8C;e?6cNbN@oeS40wxzTr+*6B3kAAlAgL&7}hB6DRk*It$dn? zd|jL0jR?#)G~AJg7i|mGTaGH5yA=e)F#8H%UPtx_I_+SwZJtLmI>_;rBx#7!#6XPv zB1>8rh%|_q1`L~re{PKmxI+YNuQW5sn+Fo^Z17l8LFTPOwgIXbgI{vaMUvo#o#c__rMU)5C zP79FR>8H8m7l&K>>4nL??3iPe8E|ef1QEq1g~eShf5OEBvouZxg_C!by&AC( z`$<8*)JXh$K7lM$-GZ6v4GE6Sfr*{wY!vuk8_56c694-MhNfStfrkPEb3+6JGfuk= zC1p%w;^n~vnmV~ynpqkfx|k}NI=Y%VyO^4Y7~0qv85&zN7~2>+J2$IqE8~yg@OQPY zPUKI}&{{`?#G@E#B1a*MWABdV%G` z^~4pSh{L}!RS*tANXI))m$J!FGJRJbaK`;Tgfd`*Z>fR%d&IB^e@F+sws&qve0Qk+ zJH}%h%0N~6pE_g-SbeqB?!rkQ)qy{SFJ9=|;d@z%KD3ec)-SQ4VhGz9;ZojfO@}zf za`~o{K)Pd_Q%^V$&H#hv{21tW3NI(+WuP?zJscs{3NtO%G!!YGSP5Fo z0?kcV3-+CvjCy{B87m{fOg2kYleb>CQo?Ua;4jaC28u8JF@FK0uf0t>^jw-ocay90 z-btEgo?+YvWTXsSZ8DW!pGX6-T@$PX|0Kb5PQ?PCX3WjaMf62P2&I&mwcCmDL>Iam zU1T~Aq|bmFG#q6hja51`Hr^U@azGP4od@Y|bJ?tPYiJZ-***y~aB$A^^pf&ICGZD8 znq14nHbKzIoqK?;>OJ>FU70f=Zn!jsSon|5{V#~OPPmYI!~UIV>_#VxZmMIxMjK@g z9X1+evIa&9264M^U{HDgxLz<304VkF7^=xR-4Sg!cA-1kSpt@`6kSY?#I~&<*wg52?EQtwI#dk z2({K<|FxSRnzZo;TppAH0yPx9Ji+ZhkPfmIdk~OoAwa8+;2}3mS~R}^gjRH`F*k}} zt5)N<=4&(vOuM#6$Tf5bSTd&QXn!NnG#f&L-wK2F&<}ys%7$z0 z8SnE2)#qH{oGyGp6a-MS)w|w2-El-GFZk2<(f>@qM0H29UJ82qrfiW-Kj{g6_Qy%} zLpE3LQu6pk<}uf=b=%O%v!lKYQwLQfvu!Nia`;X2tPV~iue=r>I2vihSfAB8PE#+e zx?)T*y1b$G7y(XS*u4O50!Bdd6$kUt_{4uPv5Y&W1U*C75IikzCQcLCjPP|-a)|nm zlmP{N+o5+$txS|wyN93WQmvt#&Pjhv5_msPB7Y@?;;+{^=_NE&M=H_Z}{wr$!!0%&xv+nlxKB?=qZG!YSoIRy${Q9T`*S)$=wpejH)zKbV9J z6QyD;oP>-~Iz^L9v)%{7@_0U52lx6x#H$>WGGLCk4)2$>JIYl|*?!+M3J8kc|0n5MmZZc_K^fq~S*rsTf$xogQI}U; z8H4Ibn!7^Yva;(l3ZA@#va7E93ODBYU`^O({-yy>3(%b_NAmod*b&aQ>$ao(m))_%x9gHGj}+5&H^DhK7NDGVJkz)nWA7g zD57D|MG~S+KiGgA>Cg;OTxNralaUa-ZLt8wFtK_B`^ZZhSYF@kEu$^l!80cNH=fvp zbkY-X0=M0J4@~}Wv}R)bA!o^(nktG`N%XjaH0wW7%Z?9A zsL(JvFxi0&1*+oR>QLw8ZwtVRkW8D#1I?7VZJdC+KR}|2+lwIHPG-}Yrk^KBPr*tB zUDx>~{1`m*XmTOfdlg%fJa^wnTJu{=q+WBu80bSscemG4-|kf4`3BO7bH#No#DXjY z8&|@rSK@#og%R)WCsZEtp)^R>U38^;me;bG}OPjZ=&{AWk9j~6w_oqDy7EY-&h@D<@ItY|VvzvigoQa9wj=Q$UG zR*2_0w|`u%POu*}iSCD}DBAx0EY48xG3Xr?@h=E&IV;9(@$)}6@Be+Q{{JNl+;%oa zb78>1u5iG>sFEDPvC~GbzjC`u)T6xrE4NGb3yO~fC)#?_1*Af*QT!5K3mwK*ve28o zku=IhFfe5CPO?eAsBUZ&RH$gy(r#9}RV+}TFV6=6U}4b7ReTmJYwT-kdj^sCp07+A zJ2DV{B{6PibhuCd%{k^jp6=k>Zin1ad{ZBW^ON+Li^KJq3uD}!5CC=t*Lo-hruk|_ z*WNAtdE<(%b-qLBewLN%Cf~;(ioCU*1`bN-NSJnx=Z*M4_-q6r2y}*CkG?R3dF1qP z6Ym!z3Hii*_sP3{^RJb2bjnO3a=IgB_=Kc1H_3i9#DIEbKK_8AJm%t_83V{|9UPEQ zrg3H57|cgawCxROw*k-k44UqUQEnvgOg{aj#2F7 zq2Uv@I5{=zObXTCW?J!_>m-WHx!8CnW^p{_tabv_v#%03<6ExxHR1xM;bAg#gi1%X4!J<}oF`G?)yppO0MYDbQJGRjc(~`U z9GeLz3IF8A;;Q65yI)IBcIw9cSUAcmNGd5RJt}W3ztL)Mej7^)d;Xm+-I$wDsAD#d zJqiax3M-0`;ns6VY6w875QQw_MkpYwABUaA4R&a<5d!N;Ua62m4l&SQ=kFpxm2#}8 zoh-VUC`EMw8o6cfTMi_p5>!8n%LYwSrAyH5GFFphF&9*0BHHCFNKwzm7pBEWcBsf% zw+%qzBpbuMHej$ON}%Ad@%6iAe$*uy4hp$q#>opK8OUhiD?Br`bC++RaoU$Yb&Fa> zB98K9abS7kbpaDMz6+zHEz2ZxivJ-~ly)gIL*jg*j&pCr12koIxZdUv6}d`%bw-&m z{;Voa_rabPr@N?g1iza^MEWUi)lqp2&hI7X62s7%-GS_UoZC#0@2IHH{Omw=`e?wDDZx&e4%;cr&W7vL`WbNB6avc;%!S{ z-hODbKGh4kU&&5Ux%!QDXORK_S;aH;3&NJ_jkcU$7+&_9UVkKaM|)u0K0n}k-x9!E z7q#d7N;+(sdo-AQ3k3iU=8@5jp;>Jw&u8=0(gf zya-m1OHakH5#0uvp2Cs|;)JI25U8?!--{6>_I=72NA}4!pD_+@CWnZ#p} zRK5NB)VGr-lu5Q8Pf_;$Nsje}^{$Tk4p0rfYfcnuYpO|x8To9;YD*H8tQIfUbLw$w z$!d(m$;6H#`Yb2nrdS>P`3-gr(CQSe9E_-7dy(9-EBaaT)~0vA;u+~j?<-mj?M@$> zMKP!BL*X}ue{bIy)VdK)p(AN)McMU-327E9k*NF8H0lf30?!WAr}G6Y76q}?%lB>*C4-{pXE^zM_Gu!afPoFxsBmZe zzt;-$%eFeoPpJ5W{<2)bwuh_ShCJ(ZG1_i(s$OF;27Od`i8)XyQwc)}h!MH$5V`JY zh92LxzxXQg{W>+iLAZUb_P{L6I6?O94%j18j+UmPDM628@lp;_i~K_&?29U2fZ$yh zqIJhXR`-Qy<=O>8&F92;fd8P~!59~|4A*#ty=x1?*q(OT{z)x>O)n%{64Q6=kpIO9 z?u;PO7uj@YE$EP%(VMRmdzlTXF$t03=#*UJP?Dn}Ep8-0u0q^P8mvre!iewE*%!Yn z&2mHPRd$MsH^t#kjMl^PePB%JHS^jM%~?4~yIrJz*&W*>0Vny+2|OjpU~-7M*~H=7 zd!%<7U~i;{bNtG~%E=Ow%(sVBmmQ%K}HfT~qw=$S~ZCANRz zy;eT8A$7=(8fvwX?r2P4g&v^L02`rP)$?EOeQ>QBQ@1xC1nBr3r{*u z1QfX=cuJNk8^T~T2KrL85nV=7H02Yy@(uO-nP#?yink{FYFJ4mWs)oNLIl3Smbu;1 zueEnPOhp{~GhiNuZU*rX&QhF~x+u)qsM?{|V+_TZjB>aO5r`c|z)Vt+>9K*BTOPrV z$bKuW72a6^@1=}`FPoukZa$mVbkzMZ3!f@KN>tRqmunW!Xo|d08qZ(j8j@*V%_FY=soOXpx(rt=$ySgV ziW_#-74~;OYr4RJl(K;2cj^|Q7e}|E-p}Bsx&+v(d|-^a?LG+Ry1*uFxp=oNiHl4| zAExDIz|9doq#P!BhFmw9;+{*Nii;gAJ37F`&S$bI$!j0j zMi2e1ws^AZ?^3Mu9qIk}0?C{Vli8$ zdegT0W)lU4;XP53VL0G!(`p4JeD`!9MB9|Zn=CX~Q4dc&VGoJO7;Z6VbQ!x8{}baqrTJa8((4M~Ja~}BkT`sx zTv55LVEeUVlCNY|q-3^UGO0w}__-SD_WyA8mQitZ&DwBq_rcxWU4y#^cXx;2Fc93G z!Civ8yA#~qEw}{NeB94{-tRf<{M$3#Ro!cK)vnrQSE>l3_d-&ZEjM0gU;Jf>_FB7w zpK01DV^KgkYlq{;xBKcdn?&*<`Z(X}r>rBPJY4)b!x563Y>|3obdDbLWi*OlQvtG@(_8os<{ zNfat|HY|8V5MdPF)pYaFE;87kYL=^o6-dM#4tDqd6&h`#zAVC*XcsF6x)f`%eGl3_ z_!dlG@to4MM`s@I)m&9feZtPEXxQ)REuX{nWw(xt$Z;PoBCm)gP^Ldg{E|$P=z2$J zr7*0_kfoMdsxrj0(+%aOI2;*W!L3Zxjium`6;LI6`76O@SJ)C$CB>?M(47U|$66^? zjIoH{SR4i8A{a@^nEC1#kaez~KAWXLDx9j_E%|tx_bnQbh|@=dt`@xP6nWkFHi%&` zOh$MGm!>s>)UVqb6<-@sbIrNi@`SvN@e$zjlyFVmcujAN_)RU@c(6=7%sEG4kBa|? z<2B3)1caNlTlT;xCsd8;Gx4`~LuR%jKe{=d80%G*iTKG`PKgg3A<{XqJ794@#qEA$ z6u1VQCEV8>;=c(0sQiD`^?yfFl^I^;Q{5Yi%n`nP8N&Zh{~3rxf)^TuqQZ{_?9zf$ z(_6&-V49f7#5F)?gOib#9<&04GRkn$4T_VXDmT|tBrff-2pA& zbE)Fv>)KaUp&?P2aJokOKm#79%rNSf@k8Xc4O&pg!5@P*g7`a+adY{;*A1&1 z6V^-TLiN7auIpW{HiG2wE|=#3m@ubYY~sPN&Q@X+nx|TP;)F01?;90?gCQGV`Aebd z9b<0fj+^iw&fG+k1ExG=pa=o^y`O65X?jGUf>MCM?hy5L5w^MKSj@gEaA6)^=%lao z+8RYD2!klncQ`7>lM5wx$d$+!Irni7;i+K5Pv)|_a)*AdpoZ=#1K$wTx_xazfB8#{ zv%73Z*THu^>T0LM!B=SyXxO_sySlY$u)RgLW4nsq%BGBoEFN0ke&d}}S|vZT8mxK@4S4Q9#0_jyRB(-EC54=*`;tq6rk2&~1bVcSweI`gh9 zkC5}@DU!JJ_L5fiV9OsO4@BE*GYu5qE4T6uC;*r7wEo2tJb;%$gS~-IO>!f-m?(fY+jrM=& zxZ6bFOfI$~B9EzrB9X9xm`3f!lJ%;q@};G>scK6GIRbsv^=Eg46lw@@!fhkO6;7DI z!ra1o6H}GNZRKL`Y;P`wfm%s{;m{Df`hlL(Lb_qdemop4M0K8SZaw*;VH=$3BMV}j zTUuoHz$pz$BPmIsL<5(u&cb!iz@TrOsA*Qccr_MH zmhqVqGIC;tLr?W0B@FJkKu`%&{7iG!K!FJp%wrR@i6mYfIqg&64`RBfgvu^YwK(WP zR`w zzVZWU|ccDE#;Sw6s>{Kf-n?3Q=5}bbiwucE9o*kXeKh|*c z5Jcx6f(F&2afAP+(RZcxPixq<6eh4gS39{GN=6V+U9JErmVct$p%s^JR2SJJJ=G&y z)Faad)Vx9IByn<2)z6k8C+zn<0|kIvniiOsTPOk z;%uQbCIqFz6LKc7Yk+ZIJTAM(QMUU%zzgW zg9Wp-cD6PwcnH=I--p|TZDmRjY%Jd9yr3V`aRoLVqyLnjL|o2A*HJk+$Vk=--u`la zbh}@4U*vW@ubdF3RMuw8MRV1|Tr#^0jqXHW?$;5XLbeDepklQRpFdBWvy=3@5#<0U zirwL}k0*v^Z>hI)G<4MzQP3#IOam@QJrr#bD@a10S9fd(IL2oAp5q&p1(OYH#_I1? zlrkH+>QA$qhw7hO_+!u9B?R1nQCvyH%z7rV(sQLwc+jS^XN<*L_-f$Z*rY;zelswp zl^FLrOkBT_&4ta)*_JZLZ(O-*Segnp9b>|4-s{-IsV=&vM zCT#k+DE6^--cYuilmJu+=(}69;o3U&U(n3fYG-M4M_X46n|2TQCW6oQ#~UA1!t>hQ zbwkaYr8#HFXa{eU)x^uT2=K7XnT5$h_p3G6TKA-y&q}aK7Fci4qSc_{)=tJ#a?t$q zM%JHBA|Ws1qb=&IiDAcz`|-?%n{l58k#V`CoAGfDAP6-`)2$lx$C}^e8Cvl&{Ui^UzZ`yCZpD;LV0l^_^mr122kdpsBkgfwjxl& zZf@1e0YTxiD1#UJ@(}A%p~(Gu`HDGHyc!4}yCuUD*L#R9^^+I7MNi=45Bc@WGLgtu z^ufJ2q|L54TgY@yu2ToaFsYCV})Ak7j$liL=whLqQH&CF03j?By0E-XMloH zm5fltcSNgvTkz5gCbff8X1CE}ES{?CBfO(!)}KaPnY&$paj*8+;hNItLk?ezBv%B% z6){JHP%ynXk&STD5;fQsQ|8NIp#l5sl3&g1!*(q(s}A@Ln_L!c3YTWGJkJZ#ZZXvz zi0d?BrsXI%??9V_PSFrO_)yGXz&Yo~1DYq=>;O)kiJoyXdext7|CQLPah^y#`0^uL zIg!Bd$u+Awrkk(O({QbRXT$;X9!JLOQdH47;0=R6aM|E_rK*ZGd}mEeHMG+dP3TVG zV@gPP?L%~ao4J>=8?wVR1aS>ZQ>L5xCdM+-@rNRb9|EVf!R;#<3)*6L?O@!e&3dNu zx?&~u;)YtmY_E@!Rq5*ow1eqUOXWlEbMxq3q`T6Hr`WPI@$g{vhQ2Q>?UUsk;B-Zs zP%9@IICG<)UC?H;Hi6RXRh=I#`xzBR{J5PR$M7C)I=2QfjsN@t2p!}G{N)+b8ILRI zj8WIB8v+`qq*sa?Qi8xoAj`GP7TH9-U}_6`6SkdEy`W-?O->DHFy-?^Et_oN@qI?> ztO+s^viChP@uztgW%FGSf5S(cMpaJTx@T%@8WHC*qf}yVVO(o7AIw8C8L>`N99bNJ zInm#DXCFGjr0yzW_3PV#OwZpR=Pbjl4*l+K-+`@AC_5xX^0Z50&6dd8InE7PHGTNU zXb}&R|4RMCl&C+uY5VY5j~mD^+~v1y!fL?6Sxgpayz~u`+W>cEi|6=JbfFS&`T-5B#Q_$XO(2CI~@>Lo?j>N4!#J*KRjo;{l9VI{i zHOmFxga9!3`7aIiCge2r$$eq#ml3PKfb|uO#zRlp$>=o$!lq6-kE>pOYg5Xw)sAR5 z9s{Bh>Te_VEo|9a;C+h_l$dJ3mhzy$;Lv)u_31Xcox4a$?` zB?kUm-;k||;EMm*?xc~}Rp03THaf0C1WzH73Q>?0k;8OvR`zXX}-8 zU#(R|07Dbk46q7p7A6iAh491Cp$QzmNpq7eWv9$`=gYR{f9Ea!eO|q6tLp;r8GhW_ z^tpV@cD?kf{CMb5`V0G6-%v9^q_+bU**E?HBPbe#)<;Ma?kOLPD}ock5-Ff7kldek z@N5D}R`N7KNkoKwBMv#~nMmB-N27Fe&4Ds4e#6lHYoNvRi3?@==9-WwCvFd_bJS(~ zaOCRdl+vJU0>4K5#_x?Jl6X%YttP@#KOlMb`;fe?OWjWZR3qItD22&#jW`V;U93l( zcBFNUr_w0wN&@FyL4FzUs%q#kX4f>1@`0Y@6?aT$89>`cTzC^FmzNQn8CGlC#~>&9 zTSn#YZxEFFASuqnfB-7pBEwnN@*E|6%1!hcBc=g(iz-^M2!loGlSp%YS@RKdRZY3# z?C0Rh-yCyWGmPeAWD6GAM}6nO)p8Z zYz#opBEr6$t3kiSC=7g*7uC;DtwC1&)^6ZiL1k-QY#l4jhXkW?7Sx15xFIBqF8d+6l08w@kFI(`Qc{Wsr z!dV1@HD-jDJL0UleQ5s_;Lwgk`OWES1(f72Ggn)Zfx!i#V*Ieg>=17M>t!)iL^k>=aI?WLBdP@hI6V?tbclnl&lfqEsk0Vtgpv3gkp#9%Q9SX^n}IZ=G@j*0Ee^ z3aw=gmf9zMt$#dqMv@uL7tG|DtF8iTi)?s@0H_r!-ycRO-*CR}?v{e%?ZYJ&MXR~I z?uYFW+M}1;K%ZR$OMku^pxvge`*UJ z_mt}IcnS&r_Ea1k=Jh$4j?qN2-^bIipN04tDJo~k4?Xtw^m97UU>Lf8Se&?C!ZoW9 z+Kh9>3*3r6?a$cMpLj}ME{st^4_@6GOn{&B@U85#9dDnNU3+GSUCPmR#i>Gaa`D4(U>xH)#(nBo2jFCA?l@SMXcI*9xOoy(t%Yq`k*q-GghvL9iD`^6qV_Y~ z7Ww)`{6(h}T)e`fTI4}~DER2hvaHW#?KVtnu zQu8q@A~Yy&qQK zA+k3jCyO6)`Tdjag&e>sdWkf{=lE}HnD@}V5AYA&kNK5Ug6>G=MnVar^vX< zE%FNH{hx?szfrSF(pm8m^R*$@AEydAYZG_XO*~`b4IV>8gGOd{*sf3MkBgK1tHWkm zfIr5$LR*`YcQdG}xx>#UGhMPNq?iu*LINZ8>e%!^5od&Z-j5}EL0kp7XS?dH&8fQ? zq}7k%XYZU&**D;1zoEsWO^h=GyzHHpSf+Gn!iz%7E+gT83J{Pv?;kpp<$ zCnfmw6M=YV`YX0u?>N5fS&Jq`4a6%hR;yxymCxY7*TX$-6~WD4kzu3H zEsVbuMxKHBH?DM;+%w@lFP<3!IaEsTQUabsjE2sthOL2GF1yW+rE#SdT3_Sz!GIH@ zS`M?@zY2gxFdrwTkv{S_#aI`47Yzm-_yyF+J(Kd!ygHunQ>NS!E~;9hhd2++1uxuH ze}fdBYHAQ1+)wlp6=IBikqkX#RNbW@Y@T$4uJzPydp54b%JPxLY22O&?(0VT`^$Z2#Lz{a+8_fBP28pT0%Qju3$}6eQngXt1SchQUy!jumK0rsE!P=$@CZa(j;1_PlDqL6`l?*vF%SP|F} zz##em;dr6}Mg;aRa|A80uK)Sh-V7`c`akq|mtlBypY+84L(ekG2LJ)p`Jq92??|)% zAp{*)z#RR{4Xh2&1PLkl&s(>50q(((nLeLEQkQpq{+XG`LIfEbV-kXlHprn{c6tG{ zkdXgeZmGs+*8u-_2}{I&1o0t*QDWqR|5-YaBm2Rk{llrf zT~Ds=bKF-U;L8_^qs7Njz#qU~kHZicg(8X! zoLuQUZrtxvt5Es9Q7HVDQ|Ef!&lc{$kB>h{y*v)$s8S_xU_V98hB&eNr6CJvf?Lu1 zwIGen`f;#-2cv2Zqyc`b!OkOopnLF$xi|;)>}33^0)Dq&twXXEoJMtP?W*Q40w9h% z9pR{+B>7V_relur8`7R2-bf70T`h4fyS;)LvVT`8Q$09>Z$WZ?jP@_JQ^Tdgj)}B5 ziM^GjcdalxZPqB_khnN3DHGh_xLQvsFsM?EfvPxNX(}*PV*p>vnaYYZ1RCs99bWLj zzizFC0u9{$q)UB{^6zxLN*Np|yGk_=!_H;AVT4OcGlvCvHJv|{6tN?y4K7u1CS~F3 z_bU!nO`&R6FPsWQADMPCz zRUH-)8;aHu-!08O86S+fFeShmL}zBm+rT`~fWclkn}`vTV-&a?_bgtMnVqs(Ergj7 z97k~yM5vety1MP^COEw2ZHiNiwt+79CBLWrV?rPhq>od^0VWFAgYWJCNa5lM9g}-L9{c(ImjnsH}vc zNq<7pNCn53#I}3>s3Tvv5j*Aa(j#g01GdKa*FTR!h#P#P8TQZT4(=m4gNH}9s_JUq zN(9ACp<#PP(W!(r?`pzFeB{(eX{KRLzctR>qL|2wrWHt}Nn)|1Ym{cr1T9M1Bkr=L zU>Gs=ZhI%a=g$fK5L1>8D}7B1;PU0E9&vfN3;T$?J|)UXOyJAV_nnMTBb+sh;j&Dlud;#XjJRAu#7zVYFo|_5U7sEhy?UO_?{S&sGu74c)!*+KdWX&dSK6c# z0*^_?QFNpzPyl{Ti!Tz|Et;Bza@pG2qNwh>p2duB-S1DNhf2F!><#y}7W0~C&XM&D z)9`I~fstIxgHMrV(pWr8-8pbSIjQ$$nx zPkim*tDZ*s4f>7(3lMCH+0{ks!#<&hwO@qI;}L?TD*813MpcW7cVio)?1$biEks2Q z0E1|DQVdi(uu}`al%Z9SCzxCVn%C#N@$>apa*`eE*9q=&OuM&KpVUD9-o7>>%2HJ z1mcOiUnDmYMIGgv1ZcTTBpm}ut*;G(`!8(t?WR6(HwCw3I^$27@@PVB=xx+rA*upA zzo;)UVh`b+IM3E@6|l9@uz~@`gg>;0ro(-U^dWKUX~lmbqxXBc8Nqfno}tvwN)aVf zt61+eA_u26uYIFD!yqHpadO5aP+_r%N9||fjk*4#JCQw zbife5Kx&d${ANtaB{xg4=NKc}S+61pG-c@Atj;8N2h+MHoz`>ws@OMU46i`~@@;rw5QW8f z9v*9A&^5oU@axp}PVCz0@7TLDRZDO!$o|VPOv_W0X|G}lpz)=);f`__{I#KY zK%DGD6_~e54yV86|-Xj?0=+(hn%^d9*v%y?vY$RZ8JnV)~cG*_W3tXT=YCL2}etlwp z0{0C->5cQS7_l94PE_}g5t^8qntBzO?!29PjWu}RMqmbC`Ozae8`Z+8eEUH2a$Oq> zlqv|{AvAf)E_h^`?c|S?x=D+D6h|BK`7S^d5=iYPfIDQGiXjCDMD;u6c|Hcx4P!R3n6Q5-wU;Ra=}UFZdN zlvO6}uP!A2#%w8ih;WayRlGH3<&#e3XITH^&W1=)!M^F1MM~C=Xe-H>Nz9<6k#5a%<5@Jlsr+Qbe*Uam4W_Tom{@v~SzwB1NC6&ggPn0)J%!OijeCf~)t zyO`>1k}bHz+-ek|OzL+e1qg|C0VDv*oIfGxD#5k6{gGt+&B_bWdp3)nhl;e(P^H3c z3CFMQ{MQ^B6ICv36<=_wgw;th@FmgP^tfs`mGpEatn3Wo9Es7X2XQ9rB>iCM9pElB zQGISrY#n1tOzUMjH~%4RXm#EQ>9|vU*7|{F<$R8jn(b;4iy4uqiIj#FU~5S+n08O1 zw5|}y^Gmf^gz6@JYoy?!>g3L3OqN^HM!VZ*GjZizBe@xxdVcMXZ)1&0qPL~U z5qp-Pf^R^B$J#FSsR?smy&*&A>kKu$`tClN<|=fZLuYbcE;F3xTUMZX5D`j{OQRG! zYE}BpZ>5P~`47OlJd%e1m{e_TAdA=+cmfwwfQk%~`xd5exkguZt2|_((Uke4rtFyg zFx9ty?9g&7bsXEnQ;v<0E+o?Id_DX{OLeJ?PQe@wnA%raU#+3}0bSJ4wCX>L$a1G? zs~Gt8D0!1%+ce_9WxgIaQ|S}d0s>;ige1g&_7!=cxgs(86OWAv=!srdkA<**g-5d& z4TinVeoZiS^3)wjydyC1Fq0OQfEysBDeQ00774pCRk^eLTD*h~A%+hDKK z4Ljji?D!s*Yq>Wa;7A3uf6%t7MCBJ9cuW znIkCL$}wzAgOvI*rR9~x*>Vps?d_a`=yKD!Kg?Jzos-hi>Z7#Ldv)zI`_|V>d_!Fq z|JBe|x5gWx*sPJ_G~TTA4h`KA&dl{Yb^2wtOaj6KMVLV*(Dq#Ic(9mli!<>oW$uS` zJ=3Fnl>6=F!F|x3#y+!l8fX|NwmX&E&@j=-gf1F`xs7I->Tw*MKxRzN;sg4!UsV(z z;9DQ7xGmg0j!rWB5VosZdAEOpPneeCm9p@is>V~?m+=kS29AvbZq;D2?95e92F_S6YV%&;{|*U8fM`pWL{mX{Ei?<1ry8ntnM zWEFt!qi)hcX9vfNs{vJ~=;An=wriBe(*}M~GJAL^UL@Z@^P%yO@b{N z&eGu7wd?njm>agdflz`)W3{++pxg326YBf11o6*3U?3-Fge_G7mDS#oOrgijx`y}s zG492s7m>~*xXsjTrsTdb;Uw5AezVFps-Jirp~rj@7k&`@uJbVmMz}|A*pe$+f_$am zoIyvx+GQ-%!lYQP)Vly)s*^4%X?Tw*<138G-A3;REwB3vPo&tL`;Ngc{8Gmn5Z`_) zN{2Ez&_@ghOL_sn&tYM1LU5%*EoQbOZLv~Kg1rmDs#9E2?Mpo{KjK|?Zp1Grpqn?( zaagFxnrpy+qXss1t{%!Oij0NrCIhNcCvf+xmBrsGBP%!@9q>@B=)C|l9#78D(MUwR zciM%<8N@IrEmkKNPe#FY%z;yR6ENF$QJ zseqm&JsOk4G;vch;n@9P^gk3;T_TX~9!ixp>W;;zUDnSo%lRs{Np2`lt`L0WdF1$(QY3o)`(UD+M!MdLQZ=3)~&(Em6bot3LEGi*ouUbN6X{iiBg_Rq5SJYZ_}$ z-^ae%gK{YuLz{G9)=qHe`RJ!qrAS)IQ8&Q-}f)gA4 zRa83?-EIXONYy!~nUb0k%)TcS#jNCQ0yRkK(MDw<>(K)%Atg>7X#f4i^G9}x!%H!* ziB`K#o`Lj_;)npBX80yi<-B)sc$JIT8aSGLU^tgN9hyCzyM&{xIW|Hm;9e%3@WhT; zE_E$6QoTMP01A>pi!zB-mE4C+lC0iCwd!*~b)-$EnO5Mb;Y9(9e#b6Lu;vV3K*5Do~m_8D!%S2Kw(wQZ-2X0WhYi=hn^9s+$TtM%N z)?$I>q&t3bE3Nv)J_80bF9+nEtLy%DRcP9S0A~-&-4wB~ zrnNsa(NK0#E8^cr_?*6)Q@oG=v?Bfgjtu^PeVl(?ob`m5oE+#cUpz2C2^5^9AYo%( za*!c7COXjCJ5V%VR00YqF*P~)kH0KjAoQB5RWX+GDwSLV$7W2XJ?jnxnuv<9gi0Ao zFAO*tiWjB@0nl~$>n~%1=~V^8nYtly>iLexoUX@?-@d<{A0Og+E%D#Uty`G9y*uprK+6045DTsF;V15qh@95-ot2O-{GItj+mN;bzQ$e57@+8)Rnp zJ&Bx0bS1BJ^N!Zi`pRq{6$7;W{NXPhZ6`FM1nr;jWK%! z*H|c%l!QDqUpqfIL)Mz3a#9gx+qd4P)0$&FwZwgK*hEJ@&jzr6+WK9S#~olJm|>j6 zMtSeEK*YIXe->dRXx|@QoaF_cVmbn#kzNIsD{x_RbWA&?YVffH>vCnSxXn>&_R))b zC)p(6lCI^6MTM+V1g=0}RZXZSggt82We5txOY-9{m`P?!2EsuoMyz?qafzeKrqR03 z5Bm3MRE?h-|MCoL#+t{3^~{`fpf9gl=JmITE7=g>N--LcOm?4%iG`kw&18j29V7r2 zQCRG2U#_BW9;?)9?x`1Ho+quuEkla(UZd^zFe&qmdRLj3S(oG(c9GstAxn3L1fe_b z#}D%JRPKZic6ch8fML<`(A@|L#cKtd!pF*G3G3{MExnNYlHJfM%J_y(Q+6AzTHnd~ zMo^*~?%GB;dj7%9Qg>3^_wrpqPWHOy7oT|=U6RD-`sSx*Ud(q5Y`1#I(UM`UmbgjZ5^rHCT;CGkZn zV|`i;iG4BQtabru%uGxHGLvDm(T)G2l8nyRG@V-cJQO?EkTte0jJ(psX9=oY(kQ*g ztHwcUA245(J`Hwj-s_8;HVp>6MT+1yJQ|%^N89aZk&PE7ujkd)moI;m zgLD_vksu7^6y=Ck`|qIj4zb5 zZ!A!i5|!Thy~*S*G%5zte&WkEQo?lDbD^KauS!6dP08r<6}agY)wLBjvjVWJrb1!{ zZhiqMPoqZB;1F!xN#4i?rWZv3HfK0XjXO;L!ZC)L@6n5cBT>-7Z@ z54wmQ5=LtOyMT(seC|1+_2D=0Ds_tDTAJxWEF??5ghf6e4n+gBBvF5wOkycJ(<6_2 zwFI1PG+7q)Z^K>=y`s0)VC`0UhZkIYr_Z_WgzGm5ZSPcj05h4565(U=((NJb4Sc6} zX!txGokM5T*K|IRad|jXO!e97Y|>6!oc6)-tCr+NZZaQ{YUl|{f-Mi(fL0tGH612( z)^3p@P86#&?)Y(AmPunw2<3h)27aCAq?@liT`t4HU;B_7#<9Q9|Acb>YrX!jU0X`S zRbTi#(W@c;Yu8XkVG%(ycVN(t;sj zuc&%0OB&5k`gUkJbLYiKB{I}9{C9oSe_@ChoC*AtB`uTD1C_3Fs388cjVAxNQua$Px_p;{>}&3};Q6lW*?zJVQdR@iH=HI! zJ)%9Wg^fv@Z83?EY6^u7SNLpJE+!YuB!limQ_V@mD zckpPA{v9@dX$<&_v$|y+qcmbB9DL^)7ZHPHWrzk~E6p%yGr1P{qIqy1ar08M_fT6W&jh5~RJ(2zQD zwu8Yh>42{r8Y+gfFcSwRlY-eBBVq`7W9)Gn z2ZG&hNy-(BrfRL{OIftAf0li@2X+A-z-Mo?sLVd|=)Me$(-PC=uo;|&zL6Oy>WBIz z>!A(hhT3hD{5oW~pai|`zY*+SJ>+&>=##DqT;yF>kc3B55m*V{7d_8RaxLhML5GLsSfNeHqfmNGP|FQIXj_QriI2Hu+ENsv$7a z+ont*LhCBHDOfjTwj%W5AyXZuFV`p z5;NzQl>PYnIT2@U8+I~vrWwuSk{aD}3w-y6I{p5#(6+c0v5epILpX0@cSYCyjUl^I zIPvu~%@70pJWpmc+j+U&L&oh<1b*1<0=T%gtpbD{=f4v>CxCi%h~@Zq)GqJ_yD6Id z5Qi|Kf5^q!MwI>6Pyh~vVmOic6W8SnQ-DYm;lEq#3Gea){?JgvqNM7BPFf}G`A4uh z`_F*=^F!dfX{X#i?@0G;c(<3{2!Hzd_CNc=f3^Sr?h6B{JtElPpy~lI1K{>&mIC$T z=mlX6teB22JWqp>QAga=goHxJ-fHgJ2#KPbae1hE^)lg%C>)r?EQEtj{08w>7~NoL zUxZ?!>e`m>*q#3FZprofKC*UZmk6%sguM3}Q6W?HE1dO<~RPg{7-B?1~r zdU-uH(_WWUz2nNP@CkOyZU|mYWpm-aBpf}C;hC8LVB83z(PE>FHlBeha!0s)8BtP} z?CBY(-%427%57(_2PQSYj8Yd<8IvXk zs*U{FXwH_T@cv8|v8DM+Y0V}cWnZ{lpNf&hRl+NL4P$G;XNmOrGgS8dqMgza4uk3v zDb1HZx=@%aIaBB4hd*&Q_DH`JGKLUU!Xs;3g%FlAd6;@XkdQK;H+6&L@9WZZ*kV+T z1&>t!5)VutU(XoBSD-Q-U}FcTSlGeau9U}ye3F<&zsw&@z1&vL@>+;}x6hJ);4qJl ztr0qx$S9)?bA&ack03sejLEr64CG9?8sHd;+tP>Z1G=S;vXtgc+9F;A4!*>(N+_Yf zKO5eQ`4KN`r8s3fXS{>|^*w=jFC}R6X1=%nnPHB6caFm8DDv~j*jjvYU^3WHGSDwU z1)6e#ZLGcr<%;?r5&Bn9zI-|O1Rnc;Vc~y|Oesbt@z9@HuSCdya^f_gIPK5z=yq_J za6o;>c@}`6L4BxsU?}VG6H}ou2rN*LH0?pSsZ5%XB=q63WGRETiGC!V3ohqbP?}L% z;2PpdCH$;WH8-HV-lfJq+voDW&e8w<;|8Obt2tGZcd!;%C*zB*yaEP z^i{Iu{a_X=$68rrG0TyFqPL!G!q8-$BhYzuGT4F6dhoLsveqsLTIYUVkHt9#l{5AK+!&w#n%WbwwkpVuE8CZ1zmWqhDK8DlnGlvTp*3ZG1o^MrRIU0E)-h>O1sk(n2xyO<5TB zaRnL0f_X-h)kUIVOtYlX>^>S+0W~?#tVgS%+#UQXe14#-9%+XLTjiV<#8~>FIY+gz zUw=D7X*6shX##SDE9`H4tkP)zUw^Z-1)S{R8W}%nrzcC zfuHgM1N&bo`M>hhvNZx`0S5U`f}zEKm%s!Z`Jbi2FxE~e(w8sqRG{5^Btp>jS8#OD zlnyvX$|fuv&=;_X`?qJ(!m@#%S4*0N8jOqqUt09b5F{KajQBDnhG?*caRWbTQ0Ata zN9vcZqOe6B8~Y1Vx;3$~R2|49)cMFYEZwFD2SINeU7y?>`xO7H$@7FDv*j7C=l88n zuiMS#*DX0BpZ!1>FtvtTN<{PCP-CI2##xqrOO)$XAWg1cI*Q@$KoGiO`Wb8#&}TGM zLm&$}i`R7MkW1i4jqDR1%q!nt-?dn6PqlvTr`t@fgFRNbxOEnkxYZlTZpvL5eE1vt zk7s@jJ=)z4jq~JdLB-uycZOb+^Cwb&iro>etgtXj0rLTLfyv$6-3H5xmGIo_-BRBt zTw;GF;4Zet+gzxuz(|N^PQyz|u%Ye^KEB2-FY+BYvA^xWXCxI4Sep-xqI)Ct@rFVC zF%j0|bl&6AJMa6Hi3A+^`gATK)H(R8n_(C8!7DSEJ% zr9%$s=?6siyHH$-o%%pI79#5?V_*Qf&9zTI(3o-}e)b#%+06=4Bo{A*NNKPJw`TPo zjd7*=?x%lGo$AWUg5LJYR()YbaZ_EDt<>+M;TJ{){iQ+#_K^*{ttym2y4_hZSqU@lasDfRiz%(92>Mn{Bl~vKC@Qp(5bm};A!Sr zHbXzoSQ4`2F1;lJ&bZB04Jb}@e)zXPFPAWYQx|6h63C=1)afaoUM%;&ASVFlsJrx1SShA@PvKTTnZf8&(LjXy~4AmK(^u~I?=?P3n5VSg&wz= zmy;LLTXvWl8j5tN;-^nC>(paJZE1fBcpZi z4?ey^oV8K1o#@w1_ii}G#``}VC~M`;iZ_#XO@2?XIm?SIs@5&HncAuk0Fge7(y*_R z1tb%ed}7a*08rycH{K~ut(E2Fk;Z>Hd`l~UzrvlD6t){%=$NJ=%Y#8SAF5x!t?KMS zYP$Rwv6R04i>_AEMZafLxo0DfO;7t|*Um&c=E3>f=K{#Bp~@<&R((4ylf^LtXV43$ zxGbI(RfMzNt+VX?E?Fw23xrhH(3N6A|3f!P=Quf5VysoS-ViqCZV|*J`o}CWEK$lBOPcLR?kSjJBZ6#Q5 z&Eg)N=syN@6Hy*U8<&86#Yn}bkWf* zsEv}@sK39F?Xx(5fUCX)Mq22lt1^OmKp;yIZ8z6iwt(n1&u)ghL{u>N!a%Y+!i!*^ za@kTv@~{$Rv3IUkc_`F24lnL0(OzhKgk5kZP`hm+weApc@^?wEDydNl zdPU+Yyv8K`P{5$Zm>uTh(1d7vhWBafhy7;9FMq^}Z8iFV+M~$_BALapU{o#CpjhRL zN;TT=qF`~`$^pyY{FipRfVRrEF;6GoW99`7W_2Dtm**8(!NOw|-CzNt)!Eh5Iqh9z zG6Gy@`j7QRK$_uL(~rN}0~E3LQ)Yc9ENB{yKbsNxzje6cn3hHmr7h61JFrQA>_q9UB~C99$c+xK(o|`;Ms8Drtee>? zBM&E&SOHeIhYbBH5hW^>ip1U+?(}R&SE4}V=-36)1fKl#sFD7-19S~a(^i{TaRUgf zPFUTW3RbGbqWr;rLP*x+{(*VQ$!PUzx@)A1jLW@s$xfo+JTX`GdnulJV8yIq#X$qv zCvvYWU%}nr6Iq%Twld$`J3MLa0P(6!!z>V@Pd=;SPE3wk^; ziP7=CDKan+DC@vhIz2wo?#=Wju_SVQKJE&(0+y{P?5p2tpsYzGb!P>?ndu>jvFT7xPYDo&FdX6QH9igs7ahfr7%HKoQqatXdc&flcdzf5z_ z$S2Ho4$2rjw>RtO=+_r6EhQ=-E?Lw|o61ABfXfIDk5Fbxstr>v3uLS8Oo-SZBNz)q{UVX7K+Y|m#)nGbjhCuDA3%nW<3bJYt;fM-Pg)cyWP7vAxE~8GOP|Qq3 z&Q73%4On~E<#sz~1QjKvuV3j*(=ZZK2ETXp8VU6V4DKD6R-8!-xr9<`x?fIEev zvY@H2RK@s?r7R^c;m-%O#J}3Gcs;4NBsd0su<1s7%n!;7Ya*B<<3N`K#=uVW#tK6LOYj*tI4e#m^&(baZ>+3vfctH&aT-O4(=|&3GNQT3GVJ5+?~N8 z0|a+>cXx;2?(R0YJHg@S{oeE6bI-X|PmR<(RWmia_pa{MYpq60V2hw~l=0U(XYo6C zC*HJAxwq+0k|TE=V(+gaVLB0Li_BUVZwwVkb5Ai zj!?K6;$93&uZz%p1VU=e4+fNS1c2B!u*RsFs|!V^^@I`&MDcdRPp^bTjx=C{RKm%r z05g@t1Y-N&B^H>~GiZ4N?i%v>0eQT~&a)oS~ z4FpZ?K|K}mEmzEYfiR_e`tAtQEl3hdBHt@rfE9$jVhF)CcCaq%CD%B^=(POuyrUQi zW(t(#;!qS+TQZ23Jn$Bcyn}ApxuR-L14c7a6>bc8qS_htBK`d;dr`$B{ZopA0_iy} zYO<%aBrxGrd|!75rDplgHyzczf6cOA78&Ia?PrR!E>L_;Hr0jU&m9$4{2?Wbg=)Ml z22?(#P(C(Af5Tc0)F0sKNQzN)VOcmax>lh|U`HFAms*I7#_f&%aMn=%jSe4 zJ3~gT-}Md8$5QLH58CdGhYifDsxsr212ribjaHWX&2l4^zV@5-^&JSBdcf9u2v51j zwMXWKrAOOygsyqakK-l2E8wj?6;m6|^`DtZ+pt~RP)>Mb(v;We+gxLVAaM`Sk6nzI zKuHyl<*APpWoSXC$B}bWyK6ry)NqCBLr;bWjdx#_Q&5vVf)Xfyp;;@g8MJ&a2YS=e z-cHrkN?7+Bt`nUK0*(o)(CWTLqgenk9)3@@B!&f%eUqugzRwb$6W3w`C+DpPw&W@oNM58juAT9^b0>8uMH67R57qCFmDh1;7pBtcEoT2eLp!NV zJ7h=+Y}RJgL^i9{!hX-zR;UTPfBIJh@Za^@|5NXU3XwZO{af!@EP)||A^lVDX$k%z zT==A&1twoF!4f4iw}T-h+yCZ+OZM?XMgk@(PyIuoeJ|2$1Lzg{lID?0v%rSdtJ{}^ zT7FBzXvOiq9y!`P6P?MPB zw+6T9KsC4l>8u&;$TTP(@q!q6I7Sv;PL_@ZfWbSh;u;(Ke3oOjS zY84pGgnuW;dCH~5ndKamTw3r@l61~>nAA^VJlel-@*s<8+-$1UXJSYFREf&*t$zP->SPUr+UF~!&sXngM*{3yd+{fa76 z&920j2av~`ViKf$ITVfpPbD3M0N!k^&BA%)^e&fw;DD|z-&u+W&hc|?bH_(=??iY* zYEzAfCNOR3Su3UvlJl_2c#^k>Md(nsc{r(F^*&VLr>-1ao3m&;3W$el`2M`O*#(`| zv0KQsqSjHbY9*26Cog^xk*%(g4{j3F7sjTVkeoXAhI)||N79h|UX~*!4P+(!x=o8M zSznVpvj4Tfnxy#!squKf0G>ybn$|bbaR&06ri>=E^&irb(*y)X1*zb&gUJhp`L1ti z1BKjvZ9-Ie!|+HhpJ?zvC2f=VjkZ{zQv21!zmf8PR{sAY9R7M$5gb0!6r;C~?o8Tf>yB>zDS_@kJ9AAPDHs6L0C|ARyQ=gzHTFkI;Wd~EQ% zs{isyUd+V*M`rOapwO%%4oeP)_D}p(63X|;e14buKfmHLf5)QuH~jki`2K(SYo)TB z0yqn*&;I8O9_l>rqn_U|?1LP-9QxlxCr+UH{?K#G)3+C3=KmNArzCm>d#Bz7^u*`% zxF3z*O^kcZNp1bm-}*9N8!v*6PmoSP0e$fxhBNA$->Yajs>6aj-pbPFxn@BF9VwO8 z)B3F%-w!8pFQn&IbuS1wn<9}+>*+UNMSNdy`Pa4@pG_An5fAz%*O1N2c|-w+wU}1* zAc90|?UK2`21aC4CAWhJrA>JJ1@9f=21u>c84c<3nn5}yZaRTmOUU}cZf^7!<&v5B5dE>z=j2I@Xhnl{~ zcS`SBVwQFaYNC0>Ng5%Tn*8_=)%1V-=Q0qG|Bi6@XEk(hJW1C-F-6T!8+Oqz$@fff zgvs67;H1gZ3b1HEAt4zwCSBEeT29kicqVisQM@!&$qK>z=Paz1arY>aDeM11b4r%@4+(I-fe7U7ydVikZH|1P8K{8+?aS$1BM?RKDVZ z@&F$ZBMH8gIO4GyqHVr+iqIjF8t-*1jDZJlI z(txC$0Tk2u?yS}plJ04ElbmToZr#$Atv+g-ORMuLkhy&dxkfe>%d%ZxZ}^Xft{NW| z)a_f0KM^`&69;rCZB!9MFC%^yl62{|C zXduHd+$TH`NaB3BfIZhC)N8rcMTS`*Cd7=cs2hy$zG(e^Qi8TBgosX%{zPSlBSAn$ z#wBh@Z=&5zXsdym7E+SSz=3Ip$D!1ubC{eAELux?_#P2Fd`n=u$~F7QyU`eo>|!yc z*J93oSi|2yR|G4L z8_^S)QL;Rfr~WI<|GxwOf5PlZ-qWlRp9fSqM6wb%yFPGAeoYW{uq!e?r68yvH|9rS zpa|WJuR|go5Lxbb0HzW(mRdBvmnb$ae&v#+2P{!ha0xmLk#Df@IL|uQouG-w!R_$v zi>vv0@KC$g7yb$jrCuv2F?L&v)}t18CPnE;_C2G>{nSM8O(lZ(3kg;^8Y~<{wVff8 z@qXelt_Yyyix9ft_a zppA~t-_9Hdhj{zzL%OBAxDmFBEQ(a*BSo!;sSlv>neHh5aZeu%im57phtR@+gkQV| zlU$D7pVttX_vjf4DBY7OkRoJ;oEPIuwGoI8cti1y0$;KwRa3TR=z#6*rMcJ9e}d8l za;S3o5CSw8Q3INb9o`QrIiirDdo1&g8WbH@HJx}Oz`vW>x`&&7(^wSW8^yzV^h{P! zS7aLDhH9}eCZTHLcFp<^!u{V1^uIHnuOqb@$>|?pLBQvO_|B(;bbEmkw#jAH!u z^AGAFXZz-a=w)2It?MDr!rExpW)bJRH6Vv%;yVz(BhP&wL51AJe;g1qq4`9&s|s)JFP< za}?jx40sA_AWAhC#0Vs1h zD5^9|)H+T(JlB&sw4C^!>2u&Dk)EX&?c&VKL}Fud`&`CI(vn4UNRjcNqP3-|!+`S~ zpWv+}|8q%(`%gk8>^1jKZylzJy|KL>W03jxrXu&Cp+LivIQjwWGLWYS-p^)i{?kQL8``Eq}$8p~>bpO_!_PIZ8qu z?J$`$*9f86bUilaR1BQH3Dan z!QqfIPI(nbxe=_ zv@2rqYCb4UL2J#hi3o^Mv*%GAH9Way+gP3CH}g=vxcjI`4@e&hmgLjE-KZKJ;iI0l zXr9^c!+e{X$?s{FEwfC#@qYK=V{4GnIEErBruwNBcEboJO69||KWCXe=*`nd17zQH zHaJ2jX6+@pBMYUAxuVUb$$Je4e0vM(9D;q7l@tzJteN(a|LZLI@|Sa(ssN24#XDfy z7_!x83+`pfLM>YpMEh3I^4_*Q!oM!;I|+_RSsg)4njy>Y|4NqtBVh+U{N*ug{Y~&S zx_5*6*aE>WjLgw4i2&xRZTTJ=3Yb2S{ws`6jfgMd6>9YO>=#dx-mqKoHbcj`y#P7Q zN9QXyG-fBn8#Qt~<Ad8!heB?GB&$#A|4SWGrt6_5+(B_Y0o2-xP}bT;aV@##}B8bjLsu~djKCVX-LDf z@;S0nUc)u8*c|=&;(opGv>JIurZ<3rYTq3=qd0F#{OsFU8=M;A!9(q?#&{P%k;|# z-nP^Cgc3_zW?4V1@S?Q&4cupVhY_#L3PMfxFU8s~(CTksrwHft>p9#KEH#@(X=iJ1 z)R-&?1xZ52>?H>Pvl>?z&8HKMnpFi8am6U^noYw|tEis+Um~O^l^(Q zHx#_j2?!#|Lhzr|V*_?r;^b~|Ec}1EoeMt>&Fbr;M}7vz|<|S!p3!O0|}L(v&?8=aW$BsY2mq~0eejtYdO!z$5*?uvT2c4y2%6L5f-rX~Pm;*q>DJ;qQTa~$yjp8^1n@81UD;+ zWkT3E>K2nHZCUn{Td(hrGzS{gBj`Ak#xquSOtN;;yBVY-_$*JN(76vJu4@m-WE>AJ zvuT})=^f0p<+A;d;)lQCj`3^DkFy@q9u%xY z=$;M%%-KF?Y(bUabpYaT0f+m6{gu3rH^$f#t{3epJ=xTcDv(}zbeK$;{tnvC)Y=#` z=%Jlyj6ianMI#PB3q^!^00lb#oE5eO;rph85zCM9XmL{QZ)joS*cN=0>IwjP1To=I zZrI}FOxWBJIy9Tx(on@|f-nmiul6O&TGx*@=6g&I35fSAX&{ike@eVKlGB&j#BhA8 zveEM0z?nL0np7K6$d>e`_Q?>)S2Yt;NY$-*jJ9Vv#prpi@Ywp*FxGSnERT4mm*e*S z#mI+)G;R7j#Yy@s#L~Sh02TlC8GW^5GwQjBBR)Ok++CxAlfdRALe#iHNLK_1sOI_p z5_dT$2UZ!p0;IfP_ETK<&I11KqCATREH=0eo_;8|R z*PC_>!|mVNM{g-!{q7mwi7aDhYkw7OBSX2-K01d@VJ6bp#kHmw?MuxJ8`a@6U@uLW z-(ROpDK(qKnEfUVR&KkK2nZ7z7kgYc?~AvOoK(-P+JVwWvD6QTe3m@LZ&Wn@ipwZw z?_iY<4g_twTR`9fYk$bXYoDRGjiX5HYS+k}Mp1G}2JFbiMp;y>O*!2Mk3iS6mxF!&O*&Q#(9+dG`9uPWn)$0@D;==TiYGFm4L=9px;&>hiBBjs1}x zGZvb+&Mh_Z&VT?Otd4XW?@pVC!2%~#GxORUTyvTOGg+$DGP|MMNGw;f@rlg~WxtLe zGycf>Ghcar>R=-UFT)m_kC}O*&)_beZ;3WQ{HaaBW{C(c(6xwFc4nU)p*vZNnems3 z0eVd9X39-6dS*-e9MGoD+L6{DB?wu;Qf#CnASNgbop z@ShYl(^}8rSA;4`>x!d1vnFaWk!4aUWLxvVa!;_bwwTshxKURj@yH_6QjWOK=T~Fi zrNYUcUl%t@Mb8Z9S@el_p=n_wlhP7!0HJP1X@wF*J=M$=b(Ypz1rH|kEJQUf!ic1c zR&)Ad*H@+^i+tF{z*#EO|B_D_iJAD2x3Q3(Ba)@a{t!~cLfIxGzk@FAL)+!?&Wg(7Z~Kf8EWz+ z(2}(BbdqeP&-oU8Zn0y9^xT_ zW^XULHZP)TvEc6iT0F_iQuS%AOojEcpGwUe?xSL7>@f^}%h$f_@vC=YcWYQR=oJ24 zZJ-N;Z&D*s4OCr5@>2800G`v*J|>0P?{2BR!o0`#PVNi_K}FgMU~TodVZA1YU4mZV z-_sv%fwaGTiOU{|y)_1+uk->^Evt;R1<7Rb??3;hg9&&8AHrFl;y%xI(k7SGx zDqnX!vWy+o(E4hF2P%ppmuf9z#+JjkE2_GGAreiQKd}CJYHe?A1GJE;7X7la){xW6 zQWzd8ghk{ZtOn}lR0Ux&VrBXMEWd)~lXh1Mqzcr*DjQb$^rwXv&7}aH?Zw>qBJVRa5{a+Oleqs{u4;fSMS z4v=H8=td}lxLM**fnl_KMhT?i<$0nJN&`|R;0}rrssoxJH4v0tB*tzlhfv^;P-QjQ zehJw1X$TabP|s}&t5B&O6SaiUkuvJTKOM+GHL<`FRgnEv5WM}Ym};mMEd^eOpF@aI zIj3OkkF+b+vcg}9D|+n}m5qfXOCijxm9++{;a(mEgNBG8V1sd5PC^r-XA+9a1KzN( zc5w-s3MPvh)?7azfD&S}M$fxbBb;j4UY0{jn?}><82R}1(F;RQ_ZT&4Bd63TC?&M! zRTNg5T`=fauQ(O}Zj!RMTUKJ)JCThIYU%oIkwt1&dWGFl8KwdvRGSC=P#dTCfpNdX zj82 zZ;4TrMNUF_n1{dB^-WWPeN0zj>62C_Pqn268p5&({z@d(FIM8a=)8;p1P>S1YP2_& z?v71Gu3^6)rt)BfU<4yxjm@c7xtzbNU3@l)B7;hefVr9lVFQ@Nr3(dN6g!A&>}*2} z7F3l!XC-_HPXf~H{ux9O)j8zHl)cK(2g5i-?4#8KT0V}fWo5?RWQSoZkI>fi z`uwn?bs0R|2sjnM%L#k!)y?ImhFlL@V;vo9ppI)}X~tONF*LA6!@3aXdxjd@G*yPb zZ>ga1K(%Q-u1_G?k4s`LTjcFnt@B_N&3c+ zw)r9Vkg|&UVOFQzXOXHIZU(xwdUR68G8oR^n5dCx()*Xaq?mgO_pDUanOo~8*1Epi zz{UDfl1GHGI+~a0nAc*eJ#XRc^U7NtIUSjSEK33};cUYJ8yD2SyYszs9-t$0qB~Y* z5g*Sd+z#(Ewx6Bc^SMyZ;D(h0#Mh^GT9`B6pFKkn7$mRDIcLTs9Cs_fuc1sH8Cx7Q z^i)g0@plj5m6E}{+2rrH^zL`tIF1uw1BtKVWnHBVwxcD>qO??w%G)CHZVk{CXLf=P z0)sfKY^`l9ZHq^(N4IzV$uO73lv9JLI~6AJ3`g?CfqBVz`J)viX^=R#j)fR-SLj2} z4y16;Q494my}+RV)cR@yQkoy%1w1PFj|+rx2v zfSo!&Ip)Gv?M2;(4@$!01)V>Qz%Qd_;2t*L>NR=!>1YT}x1CcAN8if2J+#o?18%2M zSDcd_H&gKs>U@>EdGuuHaF65UhJSksG+RgC9R9MT7Q?mTh!V;)Y{RdowqP155!ZPQ zWjwh+zGh+?O`-=a1#Y~{A%VTmn`TvEr@|B`&8h07^2y(!d--iLYdOe#0O!=+yv2xH z`!x+rfu}Me02m71-%4M5z(18Uy#dDEV8*BujO#cEN@23ZA7m9jqHdS+r>DSU_D&@v zESmw+QAr~woxIlSi}X+IDUML|_a_NL!YDH2T!CalTjoS?%3FMEcM4f_PZo6KkvsFd zS@gihS%Wt~c+JfmU#vnV(25P!DR}2rh1jXSBEE9&9J1n-@D(F41=~DeAS*NMvAnJ< zSy^qsphhI?J|`2?RsqXKM5Ofg^iF{YTIc<=>#CN>M?q?BfxfA{RK0!tp!{p{bI>ZP zbQBskj(i{~Atv6~Xq^Z@T_N(Cv)M#HNbusc>_MgoAFoyomY z1axtL4mZ-@VvVe>1>Wxuy{&|HaySdrjmI9xJ0#pD%9B36^dwVCv%seuan z0+R0T6&ARM^v(VDh??NpC9A-8BsIOE_URqGJnANLf$WKXh1IrdiP)X#z@VqL;@YDD zIt54wU==8d_Nc=H;V~7L%J<#;Y*ChF;&Nb1>Th=$9p-7&+Pk%Ec$*iM^p$pMzqY#m zZeMIgIyuoL8ZfS=ZNfUFBw_mcSh^e=@JgV$?|5GBd>$YiU2YL_u1~FMgg0rZCDam~2F{8V4B*cyNXH8gD^REwr)WHg`Il!|>YbB=0PTp&KVG zAvZ=DNmN>Gm^faj^h*gRJ{-$N6SKHLd{X(^)y$)X1v4vQ8)0w>20d=|0yC-S9ilTDd_FLAJDk30z z?reko<5yk-Oi_=13c`aR+qw+qZ5 zZreMUe*R1P75yt=mI&CCG0TQTPdFFad&C?FD+8h$2=aC({MxJIj@=Py4KgMt_u~(7 z$K|*8>)K7;)_cW%MX(QeuzG!~WK1E{Z-@&*Nc|_yh;qr#P$ZPJ9& zuj+GsWVw}KZ6$7+)8XXi==L1`A-m&7dxZh@D-8Y>@wiUJFS9`p>esI*a5q9yvMc~> z*^!c}27y03A7uSC^K}t2bENsPvj&|5=iA0TkVO{U=#=qB@Gnn-qX}oJ?jZ6J$=@>o zQ^B-iqKalp-}f#A$|B80k2oaHkX6dwD54J*_sW<(bnD$)6RNd^zz515 zrX14ewVZ6iHxV9mz>~A~mB%TLfaP0Jsiv!}4_x5;^1VY38~bzcDK}Af*#+L9M{%i~ zAE&ges-J!PdQ&;cv3QtLA8nk?M)p_FxL@FXieO=oxV<3zeb6JLBm5X#-S+obb5G$w z6}|x*#?e=_%tcglEjv_^N;Kzf7B*mrZY?hyTPUHPQKJ-BAcfqblI(1Hp~NI0g}&)4 zU5B4ars7!*BvUbk`#H173>K#M7lh#=_*~fF81(SrlHeXE0hy}IVu%=I))X!~2>=!<6Z{HjEf z?Wd)ht60&VnONt9dxKK1Ei~?Qa7Qth{&q1cxGukIg4>U@or5WD7c_dj560GXzvA_ z!Ozvw$qrS;tWCu-Y<9{u{Urn~qbP00*yK$ZI6D^^A9^~W_^Jp85}wht78Sp0GBOg1 z;0PLq1qgqcy**)@9**$g9gmEX>9s8NiN4({qPK5axEYmt_sw8W2KF%P$`5Wq=Q}{r zcdI#2Rs}7%BLhIKS5S8{x#KaLZ7Z0gW-8R_a=A7|kRHBAUj;=^cr;EHkeWo8!|r%I zs7;0s&_)Z};F*ee$Li))K?DIwqi@^P)$2&b#ovZ9JOf32&IhE!Jzc=vqKpx_1_- zXH9T1=HGaAY;E>r26<2`ZeTVk9pO6iohud2VyL1|*#@BMZXX0BKHhWav$K7F+yep{BQ54n;+WO&QrTir81u0jik~H=p!&@; z?DHpZ#wq$g#6(>F)@nhKAAq5@_yKF%lI=8TjERutYkeiv5_-|RqvF6RfP}Jr34&|; z!2bBcv|E3L;vFTOZeCi!9Rl?Tp0pkA&bmQ&h1L6|IuGS%Ja(<-QhBKCs+`Fu*c14_ftV6BWS*GYm*HBYn6vTX9Ws*3$~8v*(0cB;gY82V3z8CghNEKqbX#;%b* z)I=;^XGYO<`2^O2%jNyc8vc_q69XLjP6<~_fbc3MylytLC#76=8AG`hIPyrOy8D7@ zFHmsy7tQiI##BYX`9Bxi>C5((@(omtpqWp4LLEjwNs6rSsXT}+s zUMPY{Z`o%Ah}cCcTPTT{16yWGCfycKPF+>NHDgQvNr6;L!td#?Zat1}itXgf?}L#jr%@aq`N zLhOZ0j)6Z*m=F3Y_g;m)TI*h=%i<#U6FIW?m_(F&Y1ZloyqK7GA>06sY4MK|0Pz;k zm*4nHqti^;_jkwI>)I=92$n_dz&RrU-N`rtJ#HR24OlBATs;~flv}#eid{1ML;#(c zU{(s8XNlZ_;1W-w>g04@e_Nif84U|K1L} zi}9eXN#NhNb8JL>kL@T`Obrnu#0>MK#;T{V?sE}N60N9rEzSrmv<$@p*Oe11WUPe> z>Zd+D56bZSQF+p?-;`c(t{SCJv9;z%omlAqS9P4tk;^;sV$R|Wn^vKkjQewD`c*@w zAoj%$Z(;#eZu~`{Y{|5c=`s6jpYGHL#6SAG|L!jR@1|z6oHe}#_`iHiR2XnqsQ=LA zrdCV8kfDD00!fvuln73iT)o6Yne1kcg$9g{wFN;H#0J_J<9m=}TlZl!;K*p|gB!7X zHo+9&vat2KO5(%nNX<3K;f49l#uX{8qLQwE1L1(%Dz7UaZ5GIVjcbr=E|6P|L6_)x zno7sYuO*M)QFzGox@7#&&=MT~csdXmW5r zd%^l|`~qPOTC6fwlh|&=0VDN1+s|B3TgtENPrmb^oZj1S+dmua4}iP-Ku@3-bUV*q zS3s<`q~a!aG=&OC)5_l6WSzFKHp8D{MFb)@>+hyt&!QqQ+fEH0+qAijw`~pp=%XkJ z>z5cpS7*hI9BQlJtwi%Xv|#^E@M9HEE*5R6gdZ|! zF1EPm{TO15ML|K8Xr~TbVrgrPVQxaV18{`!fvOibovK_;UT*H=NfE#vt6_A%7Yov~ zy4AkYn(ZF&XEc4(dU3q9*)6@}Q%OEkLK%+Wn-G!RVC2~7#a)_mmsdsSBS9rR_I(R> zOO7S1fg=KiNGf`1tA2Nck(s{mP4X+#%vrpP5Tpm96B&w%D{B6aFMr~fQxTw&Cr9FQ z0Fu?tv3sb5WF)YAD3QS8JhFKJJ!_VvX0596fvOI4a`6}`$nnp}l8escq{FTxmupMy zha~AQRhPA8jkhlSTNR47{4w7H$`g{HJfA%QEQTKECV=-Y3eo4NGH2xyqD2} zQ%pPR5_G08GQk(CyRPgNd}b;%u?`bxF&JABa*tq;fhKWkYWwi`5&cPYcm>?4B)XQk zT1W8d)WXg+-9A&Bw&8E1_n|)8RqMJTB>Y$(xAs{7xuHYT? z*st)$zLze#mjtH4@II}~)aBtAu6FHe_T$azrm~wrDWT|;zB~9EuRW#9SWz))%qR`6$-|M`)HG&HCsO6{4hrwWW54#A3wjE4I=QOso?cL8oU zBSYJWcabaJkg}{+>r4`{sFxGq*&7R+So3SHNY20g0M&4kvDl7?mU?w9k$?;#tMk#F z)W}7=@nMyDDNw^=FV&q18cL&!Zc%PMtOS|9!TR=^0Bam1J^}8Ny(qXKH zs;By?I{-*7C;56tF?E^zo1>^AK?YD_W>i_)7!?@{Va=skUur0kNmFjpn5CBw#lEBo zW4#>zb?3rjxuY`5YP4xB?CQHwdLqtWY`T2T9*ZUz+#;oHzLotqeA%$Bb&WYO~~qA`w}HN3K=hHPzei&rl4$>?f{P z1OUoJ8%z+;%*N=exQ<1Z?ePcQrOsA!Zkmkh;~E^K{h3dxi$>gL;j`P3^6pJ|bLvj` zeWR~pr#@*HE2LtF_5AmSw6r^cWS@l;Uk*1(&N)z}9V?Wom!|FQOurXqb-)a~C-Lj` zFwI>UcU|>%-P1xTf+pQVou&_nDC`et-3Ewdk~ph9@gUDo%i{Wu6euUbGtp7G;jVLF z2wz!+q-U{~7*?FTT)EJRp>%p^^Yx+;f4`!4g>Igaek%TAFmPK1%!a{6!$NCY?JzN6 z(M@lfx;=TSs>&h`DKLmi?qqZlvM2D*@RGU0Y5@McHiCPJ$I6PyKE@sW)rs%~<|F>hDX@a^4)2LJRsdJ|i!>V!BBM8qR^ioolpk^6{vPjkgyV z9s85Bvt2W89uq|EJQ?7LC8Zp zaODgYH+ssik-1h|oGG2pVS$Y&Hp*+fJngQ3-L86??3rPBkpY3)9OjtKBF?!kQ)|Y3 z*o7SGKeyd2T2|gonX%+Q{A=IATQ+d%KRv<_f;&CQGi;(M*|b92NC4E*`&Ga{-B`BY z<(;p=ojbwrgL@^-x#Gg8a2&7zVtNy51mzZx*k0_%Opdqw@gFPpC69e!c#QAUQ^Q^k zRpzOL3}4Gspp>RDh!(z;Se>39UmhJ3pB-7A+M?X+{#<^WxGypO#ui_TxHK8}(`-z( zN|E;x{{FH!7h};v7if1i;hF+eQchEy&8$D0bk#!3y8W@JY65kBa@5)h{4`vzV}{#$ zW{kVy3*=0F3%=EQIuGOxHYKMQ5akF}E#OEvoWYPY+lunH{i;^wKqEbk(CtgiX@mml zlaZK{eVYz{;KfZ$opRY{CQsUmm&Tv~#mC;22Wxns5!oG=M%S&rnNL ztFlZGFjfh3xLfvG+a7R5#L!L}3=9mOZf3U1nZOzU|BxUg+12>0MkSdqpn_n$zt3!5oQIyv^h% z{nJXOE3*yUKcNUW(L#uANV0og=(a2HxJEJIT z%tD%^Q6UtmG*H@B(sr-gHXymrBWfuTw`HR{1a_+8x{AfxJ;ff_(hx@oO_(1y+LZ8i zqN1tgp9A8Y?2dt>zrpvL5|IbRp?3tFQRGH(n^%+II&le@Om* zCuca}yLC@sw=r6q&G zpml4PDOHkR=m?nrY+W%Io(eLj(f7cZ;I`_8dYNDe zP@i7=&Vz)2(vH>fkV zn`uTFfZg&AH$=M|jG;^}c$8Al;UUMW8KwGI!6Tm*88W}=m$}cE=I%}NLBAHGAOi)u zc}H!I>^4D=KUZ2{hzm~bzcoMX2&{^c9ndFDbejXW-|Zk!#tlmOzvH-H5#Zh~JKj0> zmO8kHSgao+7N%x8n-p;+k#({CHiZ@dU{kPZ#zN^r|Jv-l={YrlF1K{~^*yFH-67k! z6EG!>1Np-{26(9GH@c!^VE%yuaafW^YEnY*_C_8*)+OcL{C)}M_YD1Wh5b8pWN(f= zY}*lVabd2d_cG0-$>BK?fLIc2ZKfUWayQ&iTt&&So!_uMq-97XghaG|VchVrJN{c? zS>G|Fo#^77lp)t;RD}4=bkzhRH+uF$+E9iO!fs?WQ*rPjKp^O6E!%cd#Ri-YpRJp) zWqWyKW=x2c$i5Gkie*N}(4Kz$=&7;?zm^BkM?M?m{IoG=fykP|-!#7|RYcUh6(75r znbd~uawx?w)YkQz!yPZ#Fs^+nCo8rcDZ9C_Qz0{~owo7UoU&JFe9~$@!G3|=sMJP6 zd>ajw3&t-FG8YUN@7|=g&@_2P{XkW!AS6rr78^;IoyimOG9JgHOT~f;fh*=?+WjP8 z{BnaMd^)k%la!6pQNZG3e2WvF4i$}@owHZZfm$0)e5-fk_`#8kW*Cz(mj0kwh9STA zz@muUb5=$H1GkjUmGxar9Ab1-VHE}AD~}V2#pI;+<2NrbjZ3DD zVWZX*pv+_Q9k=crXQcJ*nd&n6&thxuOp1?t@R>Cyo5p=n`#$?xBx+C>oHq|c0Xw#Ms_oe9Hx+;I zhWEG)qhvmGarIP&MbKTtv+rk8vtFCm6XX3I|H<7;0|-B2hFCF%(~iGtuY5y`Gq=fRVldXZcqe9R4&9&@QVN$DU7X0mU&z= zlR@aM8HIVsF6h(-vkeFqrTkN@xHY(6@s(ZYWM?NLaaC(@G759~o^O4E*4kOY8YyGU z>dW6Fo^{_aJ&6LF1t!j#Pj2;%%=!L)VO$w}ZX?8+h0H$dmWDG*Y}m`29_UnEQBj|5 zMfpxgG@B~KtyNm$#3G~pZWk0Q{-X9Jp;Uho<)UHU4A0N`k%FTrh5**0GOCJ_tKy(l7@-cA zY=?6tvE;HJ2fBVs)NU9fb@0Jmkw20iVx&{UZJ3g_@2n{B(siFHEvp#K50{SestxOC zrQLOTCmCeUb?isr>;qu7JGmE`f<07g3 zR#$r~k3`UUuXC0+e8q zIO(-(fS%MW+~e|EVn;E{qzhaajveBkC_DY>(9RwfxS(ka`i889Bc2?H21wMbtt^Um-&VFSRgAg5(^5kbJ}`b zZfL0^h0uVW+>s^UZ>cC@B@X0T0cw~u_k9eYv_Hj;k_2MaXjVxO(e*5)|7^uft5H|k zZ|woD)9P!D0lhRaP63xP({ueUgfd7FZ4kdNSW$799eaZy{Qlx&?s0-@*FOl5eE8N$@+UNfizeI=!;zqwa4uLcZ6ryn7O4R!K+CphJ_bW1 zenA+1>AcG~yj*O(sqe-zaWkk*H?ro;zQ{Jl703;F7<#hKyw%mh&lsGKupkWWV@Qrj z#}i(w(o6FP=*6eaPB+PiTX(;~{Ywsud9vQr@5iD|Z5Rfn*0`4FEWygT-FSume0Az;bZChTH=KWYgvj2hz zM}B}qfy4fD(;wI`V#&ynSn|M&#Y!C-d*d8LZ;2H_nr}>L3;|GOee{SLO#zR72~L4V z$CAgAj~cA#EO_WwcO6H>I*DM#Zq%y@x|+BU!w*$a~0Br9=;b@&x9Gy z>%{9ugngw$HIBD=n=2%z(sFdue0l!YNt|T)3#mEtWRDFNySEvyoA8@C?}9suRroY^ zdt~#3=GmL>K648`Z+zfs@e=^SVSY&9ZgdfUVQUWIU&B_mEl|Hxtj;1`Vd>r$bYU<- z&m|ZBpCR|6`Lw$oz~90PJUUq}x&A{x6Nt=pPOx%+MyL)~5=@rTkY(YVPoe2?_SW|J z%!EubRJtS(*;5aB_jh>|)<3_%kp9Fk0%vkO%{0vlT4o6OPrhTAsTxDp{P$7G-Ac%a|-_ z7A}h7CtXo=5Wb-3Ir>enQc>?Q3<3u#cJ`cuSukE>5uOS-!LUS;zrr5+`F{DoAnLI{ z1#A#SA_#?Il$#KMfH|R(IlOyu+1Qd}ZMhGy|Nd(KcLcq_zHxUH-v#9VtJP^y_6H02 zKO*teIv9$B|M{!!W}zR`@%^YbTB=hl8(AvxDlbwhj};F}%R)5R(*JEra8nF}@_$U| zmN~cRn7@S@RPd=g0bqQo+b9&U06R@Qar8e>DYjCW(z1cKCRGu3S{XoM69z3?gSbVZ zCbdN^!;q;U{f_hDbEqB7=e4ix)Ev|6MVHf?L?(ex^4Jl+#3U@m__z$0$G_=(Y%^?6 z-^w4a_b0`l;A#F_u@LrBb<{(ADw|29bm9sTo#?^X(?&EBRUUg<5-bun00-$C0|bi7 z!McbI-oGn1paZj<9#pn{bOnT!!ESvm5g1S!X0r!j7?;dNGBLR~V%V1!^qD?6j%;ly z28RYi7!O%j!6sB$mfTcWkZ@E?*8LSQGwbGI(sx|_YmK5{hNjqYP<5tl(YVF}Q3skq$#ny@oQc{gD)PnAVj>%`l+~=0A5R6H zrQcss&CAO#DXoRn6#W5vJUfUX4ys$i=zNH}PE~s6t4(8WPz;i$2!RwMXcf^_h{`<^ zTzd6c3@?}-D_&diAI-{%`9{nUV)`>}{4fx1HEsusgY4pz-8B7y0KF4=d6i~_<&2haxlD@Yo2Pav|L)^#mElwQu~CiXL9rRI#zUq2|`ACLNANhZyD}zxP?V< zXsp(WF}V6N%{ryN0&M;i+J{p}65c+uTx)jkZe(pzd#0R0yEbTu{d&V7Aj18I3_(rr zQGi>rm~G=HK!uo}d372YK@>s_n|a_f;kNq4BKii@$0#Ch4p>47j$lD`L#)&p_ktP2 zfveAMWuGxNVW;F$1l;oS7z9qh5M<3MpRu9f3tW~ZCR%+DAFwStNPx6bEtT^?O>l(h zn2DP&pO5a#dIWb}OY)<*kH*nd6{=2%k`8MuK6oloA`dHyfkETX!u&28%j?Xv%G7-Z@} zduL&1q!?7YYp?sipbs=9DTg)=~ zS?~o~b=FrwP8J092!pmrNQt+z!=VSJCVM;ILsZ1P2C$ea6@+<(zW4oNN;5={n#-6C zF=l2FZ`j|JY}^r?hjc-zJ6jKPPgn9LBAsS#hBiItD_WQ*t&?mNKu{7bR7?;*UO0)o z+-VENKL1XhDAIojd2ik$VBQc1&OM^)f7bRAn#Q~4e3Wr=Y>@tas`8saw*l%{=_%)6 zjdoN78Q}Dh0S+ukG|rd{!JaZ^+C`cV6hN|kD_Y`umj+Q=%Isn)S{^m%}CBQ_mOAQ#NVO-D)2o&eCbpQn9w_`aaA2is2gsZ(e?o~VZR zyRy&q0X3uXC1!Vx_@Cvz-oufWH%+#_1plsgGoLj$yQ7afJqekl?y<}PcL`Yk?b`d! zMADwCUjJs&*oEf#&nENVJ2&*tK1Gc8$;hSEqhghxsRKPq*Y#LEgEgJc9DfS8Tj2hb zY$JhLyHFQNo*^(`E|`L8OSG0q#=<@81{edRxTXX?TXERm`{K$ZMKg9%!TOY9d)9D% z)lpVrrfn@d{yjWw4tWLGWpdg5swi_MuF^NsnJ`~hnEECI4a><`wm|!cWZ=Y} zf>9J2)YJj+a8t|9mI^Lj51a(hXEkf~1~)^ZGkKfsZOW~x9O`gH`lsSrfSAqDYJ89@z@SmyLbko*Eb_Y%Q)oPYvqtR!eZ zO1fsPFB{MLmPGX#=IQ7qP&6kV(qD9fj)uQIjq)O}ADO>pFv(lR=vEee#AT%(t zlW>QQ;9JkAshFE6u;+F6_;??Fg)UoZQ6fOI@A)#|CiP?sx2)lj&m>#jL|rZxW|9dg zpVDB##@$4?hS>w6@GO{3jyo>26N+0_mi>AlX{lmK9Np)%Qn{@P?YMbrozQI*M~G8; zR$VB6K!&&5I_H56ZsCg_~EttkTQcFm7xBnt7Se#n)$fTLtc8J0YX%th-)MMxLuSIud*Vp$YLDTn5 zdQ1mpU>rL@QY|aw!u?AT7`xsIc4+V4n2uDnJ&Q)2o5Tqwb4A>;+o)c+BngZWi0`{s zPTR$I>{Pg1PjXj{rxl1nBT z#)<@i1fZpj!Dg|ib!27@kR}N}kSpXk>kj8&%ZP251~oDI2jqum-(1>HS8SP{Hr?3E znCnpCZc%YYT%DV-dt$ET^s`F${Y$S!qs+hBk6APz_C?j@j5r|2$hB5I2H@^8lOZp0 zunCvXX}s*VB(GslcrQaB98J1oXhZAza1$aPM+@T+-X-8?O%yF@6BMN{n)fRA&AZ-m zL36Um@M1!f*)SW)qM@w!TuS2MgU5m zsqD=VDxR@Er&={`h&l^)0qUK@v<{<98R;87?1hx;X#es@ZgL zx@b8AJ2*_UIx7oM@E{Jk#0qyxYv>rYU$XA^*#^3SF){L=GUBajfE~bD@p3JCxc|`?-v*Wk{A?{e6m*^H+u>a{vKC|vAjdv%qyyVx0dfmymp zGsUJHb0k-0PZu%HL5LZ&tMAG$ENpWV?Qp5uPoFQ$uh7f<`HXJ!r0vrhhzV;O zk*T9^*sR;eXnzPSaZbSuKPJfm9PFue4w;5z-Pe03bzXqc90*u=&g!s54#Rz;G5Mk7 zSQ)Ju{&8Z(>c{g_y4jRG_9=OICP6~Ozu%P0jrF3zO1S2z)|=pU=N95SCX$J3bcdV4 zl)Jetxn=?qRGB%_S)O9exWZl27=DK9BA^N6N`^9BJ)Y<-ql$+%?nl}Gbw6iG zcXn98igtLthku*&LrLKZ7N==0VPX8)J?aO801R`XthA>T7h#KXG)jt%l7rz;5n&r( zB|RTt6O+d2Yf8cYaM4tY)B?&A?2sLyR2!KQk9q*;HO*N>Ei(CVTUdV1;AG&{{zDo^ zF|tRU<_sDiGM_pftW{>Oi`YYzpC(q%PNPrtux9yfp4_v1dK#GVEq8C^Y-zH!sj;$^ za?-_dpv0n;lgzSOV9bSVEOM9f?!fgpP#dRH-kRk&`RnZs%5^BZU^dX`Yw|L@L-bPb z<#r5kwnw=6`WTM8T*39~Js~;58%G}#K12*h#mfmue5UfKTYzo%x^F$4+PpX z@Oa+Y$aG&#HJx0iCU2ZtoQ<32^Ho+P>;X-*I9Q*lxFPYHKprfG;|2_M?T;k*1b+Jm5#yTs5A)(}^B@6-N)#ryE&Lf0yaC4ct~0cQzSX{Vp^<%Y!o=KpeuU z1t2tuHlZ2T;1ALsfP`zmdc}P+%w>1K8NszV`!fCg9prYM8oSw<5Ds{ZZrAM4HNgN+ zM);aWTi?#A!H(JfyS0RY>mqTn0IQJT&2&y@zfpZOQrT&o){Jk{truJ9HBr2z=s>)H zfNN$wMX(joMn?D*(+dNrzhqaJKQ5D#+#3v)=0hhF=E+8BY52g3i{4>Uc|*>teqR#` zPtQ8L#0_zdOS6=lkYYJa_9{${FxmiKQ1KY+-UEVk;PC^;sH6vn=Irks#Za{^>`=D_ zAs&QGn&@rLW#F}&{_%}AS3Ek?hfUa;50aYQ{;ch?1?eOv61h~Ct8&L`g=UuDcHbW_ zIjJPO97!-8-S9?WEB!aw`Cgc24x5|6l^H9kJR8b0A}b>UzgZ1BIn|4nwh{_ZjWz1_ z0dT;+-eSrWi8Ho7>J}B%&=Ye=d!bw)=*bS~g$}jm{P_bu_S+AYE8!Vh`iW9D+f9uz z)nave)a6c|5#hWxszt!K^I~(QwyED;%(2b9^Md_P)G!-C;wrY&^rlt%@&hj>-D=Y{ z#O;not!^J7q$V|cfzq{tf+rTB+WB04ch$z>1O&4o4qJ$%;!E-vJXQ#&7nQ4dlG;NPsUYs z<6MnfuIKgO1T+I;Y95SiS!&~sj0O7ko@=u%&mXLRn;|CO{!W*@w431oHYcv>et}8B zCsvWm0sBYI9seNxNESq&8~XzNe$Unn!YiOHOLaco`0;9H?e*WF;3rx?Bl1TMOE<|;aiu2A0VX3jb={Jxg3nPTO`_cFiOS)94uY3%1S@^pUS<_ z+_j>;$gVLSXwb`-^5w=Eg?U|*AMRL<^9PDN%C0dDG=x{~?qtSZx5ndX#Vgg4<{JeU zcX#!kl4(;@EPu*RFbYJlF%by4L2;&xSsq|(sCIBv|EWv{KWk2s zVHgiVU7(+so?&?a2q@MwecO;|*L^MRJw2Oh zJnH_}bo_5?@&9i+N}hki_1fWpfVhi-fSCTT=N*EmgPFN0xtO_yv72qG(+ECgYK$Ka z6kv+i>)VOqm`PCTfDC6ik|9n$qHw#eYr|dxZYqnrfA% zDox#{zGee;omkocSQLy{jScH79lNu+0qct9_2p)3Wx)5_*4DMqf0ucDaB`wENU=6UhU~KfDeSW%i|wB=r}ye!?CA#*w;bshw?}CBo0-9}ZXSm=wfkd>-?e}{ zw{q5sZlJqxTq%1pA%=Yp$(?x-|1?;fDsYZ)33VrA6&4(`;XAdavL8CTLgq98H|2<% zTcfgy9;OV-ZBA(R%Z{>?_CsuXN+A#LzqTP0ChZWuRNq&pO)cAR==nE-1#S1MbE)<2&rCXc3vI!{qCmx|w?5o&s(%o{R^at|&Ka1C}sC_5AQ za`#4#aF(%Ht=`n59<8BVqeE71Q_ej1BXR$PL-$nfJVb3nab-nv-A-!itp@wBcS;Y` zv>Qz8RGxH1xLrvSeKxb7IdTR{Dh z9q7<=gw(L{2(Z!qv0{R>_fr5I zi50dB=Q$1!X=eaoALS4KY5%};T1c$Tb5Mw)^|OAEbh1PP6Jb%(hzA#O;y1X z795T+#BYYC_=r3vT#f=5K$~+kM+Xo30|y>V;ohyeEca|93#vW5n3b)C8?hcLt4ZB7 zYEN5x&{0+Pay^qhA^Pn|9C;XAK;5cZQ+^ER0h?*O z7BPN=22?@!F!4;uhz|JNyGqb0IZ73(V$mRxJ?lF~6@3UQP1FNs+1hX`+>VAX-X^x_*3ipE4w&gb_qRf{dI;Y z$Y1!eNZK&jiNtzPL)_PHkw&^tLEbb8FATJpVZFbZX$n{jAk&|r;urFqB~|<5uWs#PX;*3K!KaxDAkinRGBM%x|i!EF7{=bu#mE! zi9;EiiJMaoz;GSNhkOci#a#Xy^*IO(nyBeUDNJ5T2Hwi0j0YzcGJrfe?k8b*9O>A3 znqgs?b?Q#6i*LpH0oVFV@B`kRxCe0*0Rp4B1*!^J9cFnkv-BW)uqLIC4xaL{4auN= z?!sdOtOq@E1YRdSvUU6%KNoroWCovCDe0z!GWjqB0Ex6>Co2eNDWf03j8R0CHh+OV zFussR0kl~)##Ho%GX!f(jI=xt|C{^x?r$>|mnpxIoz2?DO5>sOAe?E=D5|BzJR6|u zTzG%zNl!=ulS7hPlR(E!7hevpHZsH5XjPu~-V3{(;DnN_}> z{9(vEEB50Eh8)JS#KdlTHZh|8Eu>sd^_|>KC3ru+veOoM2q{7LwSu*3t=JZ+9S7W5*1o6)( z*c<0&#U9@?7s1zbRIZzz9YaqN4{pug&ue{xukf$=9l_x*2VcdJryFp@ufjc)ccpX%!ZC{?w!Hpe#^(lM6KPJEf&^_%Q+-uD)PDDy$+)s;POgXp(z$sk{Kmpa zFvC<`>*9U`rstljNTzV(OT{Nh>(a)TQI_y|frY~gwPIMZH|Fy63q#J2l_%1|)re5h zn7s6oI;91OL--cV`GDwFRzes~r!Hno)ry_?;5^CY?wn@sF3toFZyouJsT-c0s_tWx zSVjd5I#*>zjnjevqGWu)0_&W(_<_??%L}@r7>4@qv#ABSgylF%vrRq@6|DOr(iIvVUI_96ILt5-xudp1Kf2;O1J6J(iAYe0rUX-rpI&QVQXAoS5u2 z#a!fi&L;fB%F|`^3jQV9Xtr}+1>e|?wA=b5l)&Tz&u-lMH30d>cVbh3a;q`n- zKt4ZrBlJ2VFE6%lEO*p{c0~$#^|jSmCyMG}+zF{fx7R*k5^;}2LtH_MU(z(tq_L)~ zY#ocjMMu5%qV(IM7vZht_YC*66M?o30nalS9hs4SJZY3Wy>!%?dLsj-7l>xZ%AXIdm3(duv`kV5 z!)5({`{pA7GI72rnYRZ;(nASJGV&u(wE1B2tPPcJz-jbjF>ITJw!bTF!z;#W7be*# z^i4(#p`qy0h|2;--r`>J;_7bJTy@Sfs9s)arzRTGF5{{5Da5fN6Wb+PM+a&&x_jCq zt9J>O{R?{%SvTfSc%&~ap@B(7!{1f~{ zWP`casDwnU?nA9mk=O+Oj#=l1vG*@|T=|L^J6U6wTx`!+RY|8Q=R%_n1j~h#)YA~U zt8oWJd;t{!JK0G;O;92fB@*9hL*vdOhoYJRk7*o6_LFcb(g(+MpR0_#8bnpyCfnq& zB^*k>OJRwmdua{?U#=#;+(c84p}|fwhZ_b+X#DoMChTbLe)g+TEi}Df+8N5JJ?}D9 znW;od_;$K3){WsV-lPj-7KuLsXk-3J){_r8_>xKe4fNe>I7wUtI`U9ISRPKC3oSwb znwzcQ0R=@7!z+L7ufNzLp!qTpvFjO+pBn&De-_U(J1&^IaOxa)`HBJ!;I^7ZYO9-n zsw?1;f@hFxRZ4>6OYis0Sd*WyW#doW2xA+V0BzndO?QzkmU{tJ*rz_VbjSx+Yllmt z>q+uvo+GA7ZG04~Ld?i#X8Db2>UN@l+$-Pu+B9xvF!E~A$FJC<8R0O>^w%%=U(Yja zbGM%Kdd8perOd^m3I&%lfN7}5vPc5>d7$aS$edSKZ((iGTK8Pn-f(cOU$fE zOiO;Vcvze|Jtuy01bvb98dICgbux71{v(NUU@bsj$~=!{bYn53CtMOE6FH~@IHo4W zL`gFf(r#`0Y3Lw;!5hYXdyv3E6DX2T7=X9y}c>z$Q0nNnR1FMr~LW$O`IRT7!nL4hx;1 zsuL8t9S7M2j;>Uet+okoA0&_mxTF(=@cxZt_Rvoas*6)f>VJ4EV$KVi4a#UvDW|58 z(Az}`0?*wefDkJcl(?3VXynm~*OYvq!ea))46}JFi z7;kiVTW*`vFfszO{rU0n>n;F@E|Q}HAwG@|y;qEBhxtf~>xF8qpjqLcslPO3|Xa7Qe69zd_5 zcv(c4=9@OM80grq;lk>GB*&(#s7&Nv_5kPcqcp0{7aMww}{mhVMB z&J<}ztmN+zN`4+aSr-!(psU8OPKHNLoz^&|9!@5iqgMEPM3D5bt!bSnzXozdQzHAw zPYK_wpM|72@I*dwBx{4LLUfYXK0Ns_|NOKAjrk_LKITJ3vBreEOwEPnq#M@CHcEX) zFq$p#QfikZy`MrO2tK2embw73Vbk7Hi>GjUrL;jJ&!ZYZ+1oc}ja6yarP=E22v0)2I6UJs&CxDnRscDov>(OvdD)yNb$95Wv-`c8tBX z@)T=>A=G3HwoIDXhBW8uEcL}SP_FjVOs3m$JD~NHY{L}LNd(|%CgF0rNLrp=f1W_vg;5&B?*8wRFF1tYA4dpSb58y})z7xvBk%&U!g zUjd!4S7;Jh{pDbIgUFnZM=aSkxAReGclA|l7gU0(Y-^CwqEttI2;qbV`Ch|Ysnk%V z)KH4`{u2cruzEDcxrPA;{}!4cu-%;05X(9??(IOhrX!A|Mybx8EEM1jUv{jaGRHOg zaOH=V#78suSJ9gH{JK#jGfX5B=FeL)g9t{D4SUzu`r*8#Q@SPY2Sh<}#0j1ovc^F* z@GmZ8LQ<3o{WuzV7gcz`74Pay1Iv+i8pDjBKM(*8VA45eY&IE3R~5E_?XwD)cYgF9 z)|8L85ZE(qZy<(V*7h7pob$VdA63br+MJ1x|4e8F^d0)Lx9(7QYOAmg0nOk#!Frmg z=Y(t*w>|7nKmRtTVon%Xq_vl>forGGQK5_FG#bJ>L#6C^}K&Ds0y?%q~{KjfU z5(XE4wF@Oc+qy36irb;^-ALTkl9K=(CmO>C7?81a{-7AgQv!Cv9*pZM<-cQ;C%5D) zdqMY)|J!js_~%%1OL$LB)H^U9>dv1gRcla8?YGzg_9A=0T6ESGC5m00j~+9k>#QPMCj443JoS56hDa&5~`tYV_dnpM|s6a>TA?8Av& zK=y##z5p3lB_tLX;a%so48#UG;kSVRaS-iE6^9SdUv8@)9MHr@&W_Vb_Nz`efwS#c z!;BgX2#GMRncMulKx(Q5h!KW9(;fqujU8KU9oGr5TzS1%f8|px?2?#JemAhJB84Cm zH`JuEMC1Gn=#3?`9a}dz_Hxx7JvZFJ#S=$=UT{WL%XO$lDMy2V7O8wH<;iCN{JT0- z`c`w2`@qI^%;L5EM5mSkmES81ifF=TCKSLD9&B-&h)5`A6gYI%|5KiZ)+>lKyJ#)F zG>8?RXXCOVvKdXzKd9XZcZ|IgS`YYZWYkR@I)AG??bq`rV{7Ec<0fI)Dx|QQP&6!A zGjz4%GAuMnH#Bbq#Ug9^ei;nltn?=Y-`KwUfxG*Fm+LR?+=WfAEk|J_y+rdCEtkR0<=u=GH28ilT>0Yy zPsF|{8x!e%eNL*R9i_lkky4*X*g)cjyo4jT@Jku!_AcfoVA%2>L7q536SsV~ybNDw z?0Xab^Muy|7*PO^nZLoP z8vGbsB(W|-etTEay8|J;($N516Tf@ngj{J9&a&|7sQh=SDX+8 zGe(TT_s7BYO}i5;3OAQD0_y^RtW|J4hq0bYrZ?`G{$&U}=+bc3Rq#KAw{YT3w5wJ0 z<|)CeJ`JgeM1_39j)-m&x6NO<|EV=PFNqicZ1&f+3&lBYL&rBM%HgJ0_25vQ?m!Rr~M=ec#(rC+zhq@?nAl-| zkl`ymm}3zkDNpZkOubLy7 zejO+7Da`ux&@T^j23<5LQgB`&K1K(nYQZHM;|*Wjs7oO3?H}FQ>)eH3?kA^D`t`8c z|GTi5hH+7+D$-x<#zLdyH_S26TRGl;L7%}84#@@3Pas*Xw_Y>Z8p$qg$^0OgR&^Q3 za|&)2__mF1jaBwWa4>5k*^~^MRM-+P9SKIw}UVN(zL@6>MpsEdb zIHJFh0wGw@a}Y5};IzqDJ@!HQT-z$5;XhF)E2h%%KisCojtgu--2n=XX;e)dkln&| z-~{dVh2kx$lWpmLFY_MZm;tF;7cvzOIu|X1j*OehIpI9@sh}+;IDe=FRFfhahWHB{ ze{NDLgl_&Hn}_e2&i^yg`QLq56#8$rGIAG;0>qF1LVE3Fbyg@45C^o>{IBnAv)(2! zVHkH&pBNij`*+0wn3`U~2Y=Dwqs3E#r%f^t zCu`Hpjjs=FP;^?SNk>d##0?~bhdZP2Py<#TAYrSl3gafvfNgPEw2F#nL;KqSWk@L{ zZ{$G_sZ#|y3A4;4)7TjhR&kW@cRC~$vcE__8gp`4EUk?70!*%|`hUTf4X7fbn2o>IN6)AxTvWZC zM6$={~provYwFIe5yb`{Ahyp z9tWM9Ge(bUrRH~E#x*GLw2o7xS`7Ax4g z_1SS?yll*gE9!kG5D8=4OLb-*SdC%zxY6X`cK*kD;ET@8Bgtnaj=W{LyGgB9mx0xQ z8jIs2t*=GLcFD$?OR|JGrZGUnwir)F7&a7u;cCJx&sSIIS$u+<4BdG zRvUO^!f$Kogm0-8MKkKi6qrdb;c4$T1cRg~!~M!!tqBYLIUy(BQ($GOC;||Gq%)43 zX_9;Tt*qJLyL5-JW$mmyb7DOvD1k+OuG5u*-{M=ENO;T;ca2R7ZIDy7j*bGiO))ph4b0KoR24{by zC-JmF$rrmK#YOEhX->8Hrz6H4-zqBWf!y5=^MY27EQU1ZK1!ZTLNrqV3k`eY5>$ia zGaD;>qYy`2hD$P%WI7l00g^j z(&szPTV!hS?*p@d<_7KU46^r>_I%F*!mQEM3MOa@%5f!Qb2uhGl_CGSX?p;_1O91@ zfBeyDdf5s$Y78+Nlde&txDnz=@*z$7bJ+KzI3E?x1+BY@HANQ%TJcEv-?q1XMq=is^Y{K}*C!p<*|+YGESBL&bnr3QV0mK9)f0Hb?52Sv_u*CLWC- zGORT&Nowp~xc>>1Y)$xRdTNZ9iV8kQD#oZcGSQ8Zye4kGLQv{C#AHT4N6L&UVTC`w zs55@_`$pnCo}g4aL_(TnHnjq|gEuM8eKp!p@V=dj_XxDK+@f>|RRW+EZLEV(%8$f> z+!!n2N$TV`5qUP4G<1wNF)UBK?{T}=%^Uoh7-DT`U+)sAiSBv zR!^Q!ukhl)dtqJAlKIkEev`kQa|K6kvG_woAS=5nl8tG_W+&=3ak<*%>wh#E|65)D z-wOx-Q-aQ7G3X&6{wE~L#rOn;SoxL_wY=rhM}s2%7p^I0`aOTklTK56-@rrwE~uL5 ze;CQ>ZC7xwt5SYRt*g|-YDeR$M%Q_f&|rjPLYgk7BkFy-S5NdNaQ=H^dqsQgbEoCz zUHEeq>&)-r0iR@)Y<6>|r|LOJi$^;VWNx#RYm*NB|YvQIptW6)e#MQ4C z8P~>WEZ&cZgF+u#N1MYH{GyQt;J`7igl|E_$s83MwGFOgDCmJFqs#dRKcb83jA}1C z$|jfQy|C7NA=78&*p*VnTuU!ndUC8#F_w3Lpxe;8hi^UOT48bjJ#hZ288VK4D7yW*yVCvp1e_B3_805w7DX^2avBm z57X`@iPtB-WIM)kPZ<_a&Nm*{6<>EWL~ERO2^G!t71k23SPW&%CFMkfAXFRFLebPE8y{;`S)-s}iyBRjzX$}+y-$@L(`^Z&93?%D5sh&(RP0KLm=`=n8UODGJ9@kspHnop& z9Xs0`Zq1Xw&2^pvexOJ;stBr%o8RF${X~4p`lQYRH1x7>wOI-m27!G18b1Ldg((+W zrahL#5%bAAaG?}2`4tTZ64d_FAHk`w?+^$??#?4}Tz45reTH+?1WRc%k|v^;G!jOs z7^X5LnH6?KcF6hsQKa7)bhI{<_?y4(HA4dwi|2ssYOdoN5L-?VRd&NUEPLvUm#-2_ z9KuqsIm(DKs4i#J7E8TO5#b(QU@s1Vl1-Z{<)R8Uy!4agV;K^uEt8HhZ4e$4W)>ju`VvpfoiENbw=qBN*woHmUXl=u53%`g?vPoXH=a|oQK=tQ+2fTN@qm-c8jwLHQWnRB6d%4V@HZ=uZ& z_BkZ!5kkaaGTH&t*RKQEhtAd1zbaf^;#ep1eBY}}crgD@KKTETnEoeejI9!!z`*<$ zKIh9YrTUt&ivjEu7nIQd*r!+E1tTJJ{ZpolK|IFq4x}{0i8u3GgCSzKZoeFlDAr%s z1q1ZuZuj*1IS@sC>k)@tIv~T~?1|(Sd=zIUT)z2hCm%O=XIvoS{4pdUna=59WHGuQ zO7Y)>x;F0Ncuf`zs*T179E|n|Fww7OV@tUe6kkbZGXPym1)pA+{+2Cn%z$0j;@@0c zg@HqZvJ0LOWDNO<8Dx&MYTZ_e_-8_P(-a_pP;!MIn0&_9U!8`_(49$APe@FM)=kZ2B zsRzV12EZjwKqG}mg}ICH^&ZueI}5V^i0JqnSN21p7 zX+`T)98U@ha*F&Q0oyhICCrD`xaSlierrAMoNerJ-(8e8H}i z+B$8B05MSN*DP}$Nq7hD3mwr)FOTQ6X%0bJTD}F*Qq-`@ z!8wuAbN5~kV*)E{I@Nfe`mxplfvqdv2;1(h3aa1|O2IUJau|x^R(+*M<=}}_A?L<2 zr^Ec^!t`unoWtU!sh6F`rqpYidCMGjA*cnj(lP@%!j=a7j}V$ULb;XfJ0D3k)tCnv!XU>`M0^kafm@GibHL^k&PUUL!z zy%K%#W6MR^f~S$%7f#obCI*tntAxaw+ei^vNbQj_qI@1OM-)2-2f@%(Q(}#JRwcTo zxZ_|?9Q25OgncvMYNXS379@TKk?z;ce(~A|wL6)G!S-Vgix56T{zNZ-ChUBY7wc^uIl*fkV29*)2d0?rpGVG@D?`%8nYEj7M9(4 z3*FCiKUJaRd#*M!sSuq9-h)JFiCT@ zPHv($rk&@Jrco7@2xtf=k{@u$r`5I{4# ziCOP6dfAO0UaNatnoKP$dM;I9SAvorZFci^>zPl}NgZNF?f=jLmFx7NLyiCTRdb^?T^@a+|)oL0Nb>2=g=^xdHg zRyq~?5e(9}eVdMC9iA#6++^)`R$yYdTu!Dp#&Pb$f! zD@34^Z(Sqt#t-U3T8N)gVfOY?xha!d(Cm2Q&bK8os7|_qX#vxiRAx5Rrd2ZH!evAl z$ZTd0jQ~EpmK0HxiP;)|4oVQ8Daxv0Oi(^RN7#jD3d+}JW_>9=Opvj%A}n2d#je~5}Bl5NHiz!_*b;npxa zsu<%?^Zd!%4fRwP=c1;@l=tf*vD0pgvOC_J`)3Ax{3Wf5RAb{{2dgX5PaJt>Je88B zM+7KRHHNGy^CYlT%NmM>z?35nHLWJ=`a5dTWIxcvm;NhYwg^|yx|1bOB>wP5!mU2` zv}-y!aK_R+yu_!1xnpCpm5q(mFil>3d36K$tHyl&McF?ke|3eXyqfAGgJxx-sd_Rz z72Mr>pD+`*&7JH*sU2}c+O%7u^oL@5J1!u;Afq~#EmAk-(4l)gOwFg2ziv*Qt)ndT ze!{|xmd$2em&4zQGmcqg-t^pN^1-4Ag_3p8&5d|v6?d!7+h^X)iH*!qZN(A1O969s zAP*)v%$TH$9`l9`bcx-7-{RdI_uuyw1D6Sh?UVL{nuok=@xsCBO?NA&KIoN|5kOdP zNHP*U7&wmJ)G`di-Vuq_76vP=j!=F<30NLJ z;bPYKRk?wp^OtyXuwO6!_SJ7FM%7Z@{dZ=Edn46oOh{%)6D%avjFdjYva!ZY zUo_2h(g>(f7#Y(sW^I`MFaX=;W*lO=cMJ+Fb?~YDwig&VGvU=rS$N=qqEwJUQ83!> z3yFwnEBY(-5aHO|!;s9mFcNd_Lh4{Ff zNvpY{=jv$8*ndSa4j`T?dY&dM9ID^{BJ~y(c^6e8N-{HqxHad^`2a?`7f6p61T|kA z+s*}cC%t{PFv6V?BE!PjoTD@0cQEK1Ey3{K1CqeN&>Nb6AxK;V6#-^cMcsg^33o1r z=YIZruqa8y(4hk#E#P**^$^)7MnIZz{x=r|!U4E2W&>A9zypm!xNtU#;{o{tfdR(d zQS%#AUAW=`EyEG}2Z+y(YIsvFO7}rVs4Ghxi`vhwFnVSKL1el(_o|3@ar`yMVnhYm z>G%qe^o~e|7o=rrnF-lbxJ^+vTH4?aO!f*h_tfZZYzJ7RvwbZBM1F3}j18GHky(dvd%AfC( z#^pGiyEYqC9e~#sqK?p)>R8#pJ(9!Gu*~zeWRVj1!T(qi99jU@J zi!U^-dvx`LFvl1AxGpEtut@)oP>nvd1`~cc(i34I}-D1HcFf`$D@#+>i5K+Lk7~;Vqdg zHrkbfG$VqN?ogn#sbE9tY61gxmsv=zxbFDZZEs2tfY%uyy%3HVtUa=^sApd6c7Uw1 zsVL1S1hXLHq*mZ!y(g&p7jv=~@!aDL*FyC+dRO3pubHW0Xnu8a{a&rH&f~;!gW^vr zeG0`91^@WW^}<$-T;&Jx#7DS8XPk5jAiBdt_G#_b!JFjW@jnUhe{Z z{~0g(eR-kMM*MjR+nCu2ph5mSK?-mZJpIoaRO#dh=>N6GPrDI_I|~pHuw9xUH-Rbu z+7C_qmFwF;{m(NVzKhT=pgv>?JoB%CxJLA~a3CaYGbC94DG;5FDJD6pRN=AiI{UWCu^lLF^$jJw|rB!Ns+ zJc}xYsE7(?RyiGQu!5aF`=SYe;}Yy0xP=}1#2zNdK2s?EjxJ0=6)YRaxO37E9Fi5t zfQ31vFYA0Pqi{9{LMbn_3xfn=7Wjk_SBUrgFs$h~;!v&$BvJ28rd#d|!Xa!k4o22N zdWpwWg@aQ((P)xsrtJj-ErwCAbLLdSvPaWw&Ev?2&+(&Ozcwl(zJM{n6Xm@ri68}Y zu}Fa2=@=U0HLAFS@-+ter$7qflGo5uk4Y%i$yeHgDTSa#$B1376HZ*_P?o}efvYuD zz|>gASdqe?LEJ;Q-R}$s=De}I5yU=Sy8T<1+lqb3UxctbCdiEH6Q-4L&Kzk`*q|#8 z4a2lHV)IBX1l@YsL;S=5#+6Gx61vL{d+N1R=Nr8?Er;-@?L)uaac}|Wg7)w>Z)QP; zgCfJpICQxebiQMjfNHbFpI9y(z1bek`y-hEy^*-8cV~Fe_Eatnx8pJQPal5A6NuZM z1xg1LPS-8^=oj6NBMwHk$gOv0$R^D~;#;>U_3fri$Xe&9uD26_!!4PC{*=mRlIAw) z>cf%2t?u0Rn=hy@)}estwqrOX()3_vrVv%#rlBTgxhNCH$JEfkyh|{$xca~Vd!Uoo z`@Rf7Ufi9Ct({K`nTk*k8(|H>K*bQhw(_4!^*7~$ZLi>Uo{>DBIFm@A>m{hWad6;R z@Bpi&mZ-OecZncC4KrVRCvx_c@ddHApu=NppRcOexVN&C1fb7V*Vt>p3^FSou8UNy zWuT^x8G9X^NeA_EPDcVZFn$=~MH%Mu($^gP^`bhx3B#5WxvpDd-KASc-ck#(|O?;q$lIbmR1TUV zvn<8x20+e=%s-aLb@WwID5>iM*aM~P9(iDyiqZa!( z7niWVU*w;R?cOY~uR`NL4*oo;$JTy%ePNm0J&p{wz{DJbB)EzjV0!BoPg%$RCQRIE zCJjjzew+MV%x*JiK4lw$U?EOBaLZN4KXI#MfK6qXy*DV$A}?H78Ko1qDTVJ@EPqTm zV4(xhAyzzZFYi&vl&YkN{TNLmi6n|mCndJuPsr?k3m-Ks>r>qlT}qg;VgDu%u6-Y4 z{?Ob^;_%u>ZEHJa)c+l;qCc|Q;cDAJ=1}i_DGA0u&H+OF{6s@EcUXF~yD%r)Xv}*4 zz=f2s5)n0Ci+w1ZX>d{ZPX~Z&ky&A)wpj-l&=`)cx375AY>OXJoidpyCXv3^RBL{Y z7BC=pKK7AG(4e9!`;3&oLT?4*_H~<=S4L#cIKdZ_zEh_xT#p_4{>Dirh1Mw?c>*hT zvKi<&3EXs2uMhozz^@>*TwPvSTZIrce)UjOeFm$kz2plDONw9}hlC5&0=&c~QcVFg zumj^&M$UfJvUkc3>I+-Vvb5?2p zGfp!&X8R|XBQsY!7Vh%WRaX+9)5H7V6Y~hoqKSv2;i>3J1?Ho_Bfnl^Xi{%Lfv?b* z1%nVDh7njxzJO*tiS3gG2P}kqXf`Xml)%~>ShVa`pVdlw6b-v z?=|I0fMwBUDf5VAne%4J=reu5s+G}7Re~eqw{Hi+&(;Hwjd@WsY4BQS5!IHSnQR*s zHb5SK3!1)bq3Nn9D3!SL5P$)P6YtgV ztLhK=;d6Tx-8@>DPUUFUuOZ>jf$=5+Fy`=l;qxu*#=1u0-AO8&Qu5#tg!EUA$f#>7 z0C#@hFRhWlaB|BeG(WI5d+T!E0AlVvd$U@~43n*XF?uX^BM}|m+rur zxit+<Nsk3{uk;; zDj1X^ySlq9TtlG9SYv!&X{?%fAb%m(Vhqz!n#`NBajc3hj>ob{7F`{}o9&yDSTRFE zf8xGM*a~HEknCf-t}{vs&22Qo%VByF7m@V>5^7v6NZf%>+ZlS-Mh{@WV_5GfMR$bn zE|yMOC%tN6+ureDId4mZch~%?M0^J%_G0Tbx~{IW)sJc^flq|@!s^f-w2uVAx`jMw zVA)cSr*NflhE>rlixLYX=1^-u|Ds(^RKZPPAr+3uH!ikP$pJoxPKc6L1tTrEX!4R- zyT6p1W=J0whOR|wZ2&N^42D53YDZ@71g0TH%?zefqeHx0_BO-AEAmq$34T)#$BKSq zM|?&Og`(SB!44c{&O;5qPlv0l2DOW=!M4?w?Yz3=cf7*fYRLD*eIDPR(gH@Xv6DxNDsdnHVInYWHl#< zdIxt%sxo_RF;31xxW8h3}2rS^a*r*G?GyNXjPyTafQ5^#YEb^-* z^zAn_0Y)5C`3!*K6Y(|61U3R*iOFlb%a)LvqVwPvackzGazl*PI~OW3g&#;>;*MaZ z0>=UC8=mnSqs&yp(88rcFm)v!v^jL#z)^9nypko(7he*od zBnm&sy{z=#{PGRBgv?haq9gW-nf=viE9eAeNR@(-AWNnYCBLGjLXT7FYV@|F)YrJ0B9xu2S; zQH1-yB2WR$=*FvcgAmmJtn^!+OG&YtmXT}BBg1w9Qp0_hJ6}!z;Yh!4R z5q%kl_7AbtyCq*qS&5d1RCxB0{)hak-9Nb3f@mDjhJURA%jIKp~FZ?TS(lfcf-WFvnr z@^&Wkz|DASgz(=Kt4xY(;i43?1#$m_tN*Cv3; z+QewX%?F9R((6#o%tqd-tf*lE(N`vO7H{H4O3purIxq&(3Wi%wj3WZp3NRwCj7L;1 z@SiQmvZ;T)DS!VyIS}HCveWAC8&2*qClq~UH>YtOb}vSG!0{Omh`#(@rhv?!8@4si zq=t-4TlXSlG3W6#8<$hHkWP#~J2nESA{d``=BA5Xr%+~i(K4su(S#*nCs3-E8%-Xt zYz=ow?H@zgq>@6TKcZ!rPAa>wi8!9wO~z9v)IMl#dF=2&EZD=zkBmJalZa9{<7ggk z@&Je8$#N*=+DtmLq)~J3CVR8=t&CT|&LSj-#ce6RVb`Uaoa_znN%2D;aLNN>6QbOb z`_YUy9dqJ^u-r@-DE2($Pwi~%0tSM&GsMlGo+wUcspzSA9Rzvi3^A#W)!1)H8QX(5 zKaZPLqLsB&bW(K_HGeU^av^UDP(^AcE*1t=loRZ>x0%^s)3X&t*Zqzg9mJ^9rn9Sr z!{hd_>p+r_kgQs5&y%)W|2+<%{F~8*BhiS?`iTC+%-^0`KweB6VLjblL2eFCZvHfN zDo2Xu!D!>(xZU|!NkbX?y#ZvTLN=Tl#TAl=?SHC~huoY6b-tMz_%Ccg{9f_nJZRLh z`n}N3byyWs8&&Aws|(06rh}_wMZF^6XI}XJ!`_et}F_z5gj-D~SY1B8-G48t2@vcuzik}iWW@Ea&c%vjdLO8*Cex$}qkW)Wa zg;OxxaM}T1f{7|s$kqn*d&hSMk+&H4YGrd25mn|eIUeY(u7&|L=40JWF6zpyo??W| z-t>6o$J+*8cP+|N6CDi`6|Y$CjLziW+B6=-@jnw zaQ4-^Z7wekHV#&hlc|kS1}l7wD>plcEYr~{uCRZEo2L})vj_mv+!rPZdzFf@7huJ~ z$x)~N34avX3#iIj4li+5D%<{QkC-JUxeRJ^@$?b!ecxHef^om6>5(pfW}*Wof(#aM zX6Y?lHXHi%BWM};YKZ#K19wusJN0PesBF3GDrtYRC&4j}BD{}CoimHHS8@m|o+>*B z7b{-%iEwxLPW1pbIr`E;8k({J@X8?=me{t^UzN1LFsGFmD_}I$X!UZqT9ckdA!CV# zn8dS&^17E;#%UUq(ma*S#Ev7q0BR+qWNw6dV+(YET9vTngIqdKkyk)~gEoUPPg^w| zr=topy`1GqUF*1_V;DN@mLe=AcdMg>gY-pk5ccXHgAf2YDl)@6yWZ2*y|NwiyL8CX z_(^(Qdazl2ZGdnNNO~8OUo)DQ!M_2U-y@dv6w~VVdl2zDx?*xl*Ocm`+bPK$ZTPY( zTICFsB!;8Nw;epLIt$Bso9O}t_4!nJZPQg^e}*H?XR_+iq|%>xQ1FBLCwxwT4;J0P}I+l z&f7L^D{YzoB2nyzN3664J``;pBbrwFSTn0g;3eSKV5@tPbzV1lD^xa6Yru@86vI_z z%DI+0=a2`)^_0A%b!;0S%2iH=pI_9N_jdKEvmdR-z>Sn!ltstb92B2 zZ-2gh*V#4DoQIAT=G&$Wgc!-KrC=Ng>M3xwzV+d*;d~$1V)UDtM$GHFbcY9f}|f z{k~X4xYL~s5{q;5@eieg`(EJQzse7N}5ol;*q%mgVdpm{)j{A3PAlRE(@3gC!Fufz=ACu)EXPg)I;UKn+z6mkfe9#Gzsi~Zlqrci=hE9+4&~e{+xg^&NuClwGjY5&6&)wd z=UYwNDUDwtmtS+oF zf-aj_znWMsnfT@I-Y~8bn^H!j+@nQ1blH&9Y=&t38yffcH&x{b$=u^i#45k_vQ$A0 z%4#oXpD!)Vq-{WHGOcVQ+$zL0j_p z%P%WK-h@1wpkWP^q9=`p&*6o3w0(uyz9*m@A#6F)3Ydd!@(EjeAb`)FC#mhXe~ zc5ZfbVTO8PmP7;$ON?D61fUD0qbpFABi+bGP<4VxHql;|wgm7{=0u1(W93XiCOw!n zO0IVJ<>QB%frfSY(@Ac`vgtdz?vZ5?IqMee1D`9jKu`Pe27=W4W=JH3ON4aQv`#=( zN#je#X(NWN0%dL=z5xW$hs03nv z#aElRsY@!ST>{`ZmCqBUom-jRL_pPfl4Kh)5P~%n)9S|hI4n_)dbOpe@VBl}%|?&1 z6OJu2k9u^T{Z^Uz#A;vS!4qNO8oVCDOM zW*^1EeWX3#x}L}ZUm(7n8)DWvzKM`-u)UbqfodU@o+Is80%F1XdUJ^(N$@vm8K}1G zUtLL1S035|=sYMc)Dj{}n?rEJ8xq=KDV0OcwbPXu-fz)OSfIOty`g8{ z2 zM3#l^z@2ASa3m2}1ty+uJDKfFEX-=7QXt=euAV>lXwo_?9o9(h;1I z;Td7zX|iY>2kW5CmEmTYQCGLu!pL{#J&%&a=NXu&7^m3E;8qd*7p`+P`|XJ5x}aa6MQ~p$?igAs~Z5#0h#j&69xgAW#5VwyALh4Q0~Q#0lMHTl*a-mb98b=;H%D z)qv+dn;#;^L77sFp$L2cIOf4xJ;)F2+Tow$jUPmZ!LNFVzL2g5UaNl4`Cv$6PJ*Bw zFD!!r3$G|FH^LM`82oQiBms1gS6JvhXm|uDP~r9k%6|}}dE~uuD3dc*N|6{P9xZ@p zp9?uENgNL&!Yi8`<(_Q;88ci=kBY~%<~)>E3{j8T=WwMEbC*cvUSk1rExfl~1Fjz; zAtJ^E>K29aC~p{Qn^s}qY?z9i`)Y!`C*m;7kB+FszpYX}e={sNEW!*(|5!-3#{s66 z3xI)ZQ?Mtu2ONI!21?e!Yvm9L+Vg21UNaqFGBP8gPztCGVCR05 zuWtr6iCh<)Tn9e8-ED+Tog?n$t)nu<%~tG#T^&NAVVEZR?bXW?eufwob{geCuv7%i zpabu$rq_+bhpj`Z*H}{UE)J36bb+ANkfjzaOj8OPLZa*(PX#0ze@9Fq3dsy@B2Mgv zu1X}0CQ7IbqHRKR9AV9*F9!(IHi4MNFlW-7c00`1y5GO>f*6=3P0o#YN-?RfKROg( zBA6Ql=a4z-S2r>07*mNwr-U68?;jX(*%H4~Y;V6lUD#Q1?)1Qo?UR0O4^K^w)U6V+1~N|A{G?@4 zSB>f~>lSAn(;n03wIh?0XxA7)CgjRiunqgiei|}+sp(|c09ZgI$t)04=7m!yN>6@b zuZr15RZNCL+z^uJ41SYz(!(E5jQ8k+)QA*pj&})NEdDRx!-DhdA-~dHn}>RNy)7X( z+aTMC!Qs?SooPOocp61CrL9`Rg%!8O`L;Sx>?Nl3%wHI*wJ}+aAnQ;qRUj_zTY+Uq7MDK z;p%MCRgO^Z26x>k!+_Ho?oTUKh)%Y^x3Cj9tGV9cQyD%i_ zVSPGEwR$5eBgy$apd?bFb2YF)V_hXZxDh2*cyRK{TBwW0e-Cs8GBND<&4F@)Yh4kkrGyLB zlLzM~7hp%{XE%IJ{W`yfkee4eIN=4uWR=yV{Ilof!#AKsa@&%9=Pds=phVJ-%$A!e z^m&Y89fEgU^57BsQMq>vOs?3I!8b`hM(7MMO`8P4~#SMMFM$c zcj_svzZeUDnpgmP`v{H{lBaNFJ0wK-3zO6*0l-BJ6x{94fR?kM92%lffftg@Fa!$= zqHxGM81VzsMV{n7MFY44X51i813X6re!V;&06a@W531DKzX={IR^ z(A@~@IR=1)@C;iHX)Y7^XI+;s%74T)o!nq};gCF=#9(x3gqACBGUO1x8l~>g z1n_{f8r65m^1x$2vpa}8vUgALfFMA&J91+}c#Cm&pnK@=XZVRPi1IUBH`4cz|E2`> zkZ?{XKw;-6^Y$il96mQ}_0W+{)QglfoMsaGrt>z!8JzP)Le+~dThXnAhn5z8Hw^R8 z)2{JJREr5YoO2lOCH1LmM;i6kh<3W)MhYcxZ8px+~FSo5H$CY-s!~mk;2b zAluM>O_i?BdZ0U(eQsuJ{MCSl(j<|JRNRW>C!OhmyK*`H!%T(O%ulJ9P zDc)|p*+d~B_DU%s`6@XZ(IdEkJ6hrVPl9D~_zGu6!u2GRnAV2^09}Wm!&M35LEH$u{LjRaR z&jV)-;WsZHONxniVHy872M%I`5nu)iK}fXtn?C&!ex-x`m6(iDz2*Z^83D)WW?%zQVwE^&I#!7ti@W23#yX~XlK}Xyf_pA8cmyS?8y%C+++~yhj3%n zjH*Q=y8xd2uO}mQ5v0aIJl$f64mj;rOMD_ZZUJ%a9M5_7$ZSOAMMwGg$gerQuD z``D!>320>BowiE7Vj^*0QUkG;@t$8-l3Xh!nOb)HBQ&M+Qb-IDW{NKX``|oj^Y*Ct zHD6uG+61!;Qf-FG6@$kTUcq1U>v(A*c~J{##Y_1T>(%4?J21LXYXH>_XWQb0>=a4p zMUB?ep$<#k;xS2ghZ}VJ^wuWfRb4vOtm~i@IIMFrP>9re=IBi&x11#Vxxp@bx~#^T z8WhCPmH8=ar#5+EuSCA=p!;SSVJ~iQDwNptYZyO*J%hQ6p!O{dT2<} zL?@#wyEQovAr4FTSHNvB1)^OTiGoOT&{lvxIHccbdA8_+!g@rp{LS>$2V}DG>Gbc% zH{50WbQfojep9~OdSQHVX$A$a&H_rfU^%KV$$e(8L593Appo{W31|jQa{;x4(LMx6 zGTE6EeY1`ZzxfFcKgm@BGveWi2{ID#d3g*`MQ(VANtN;BC4g~!HZZ9zH{|6BU%OH+aMiJPz4FHavNl58gp8MnM!y60ud@>oFJ*=x&f|@GMTSsD zGxkriG^6V30W@y}gfv;_VmRa2>LEYuQRqej*b^7JgtG%ZZ{S>N#z7iyW&=9Lfj#9f z9&hqpD$V_wM*vH=Z0&H%qvuTq;P8gp5iA?(T=0O$jl~v1M*C{3D1}4xbO__VHo#m~?;e${vN|}h!AY<-KND9~ z%G^m}=yyo{y1^f$7Q@g?7*EnR8NDWwDy~lq%|`nmu}>Xy0W!CtxqDA9@`{oeF2xg} z>hVkT^G!LqV=)bLnmXr{Nn5SbYE_j(;qzU&fZ(c`e2w=wi^b4gQb@5M$ax-46i+NN zc*oErFF=9m%rc(nsCKf2O76`K_@PPUj$=J!$~u(i$ezut+zKF zbQ{&1hz&Y@yub2TJN%b6zfnhbZxgygbB<}^^J0(7`qmP?r6_mc9)%-sIpUM*kxh;% zPznWw0Pjp@<26hY0x(PE+2n1!8R5k9hTQmgFo3yOnSI}hJ)-A*;d9Bc_EB!RUjK-A zBg0iP-L>5=@aAp9%D>R>TsKG3;b~n(wNNP8Q)9L{zJV1Uxh1A-!_hhS$$TKk1|aWv zWaDRBKZ*Sc$FSe77`SibuO4H(8VA-_2m>ms^o*Wjeb12r9cw<;8&NX%i| zcP1tCG#ukNoBLW)bh)0UCKepyyVL6SbP2E2!X5^VyfFt0!m!M&HV&b&#AgBxi_>Io z?d%p??-N;tmI%Cb#3>^*In5Ke#H@D(0)PXATOsH-sM|a6JKYaOFaS#bL~>sCI}GiS zl1VH8V(8?INhSas{1Nw2|NEa`A|Rc#kM!q+i2C&IfbJcXE*X-D3Nax_pc;ua6=@CK zKOaCNnjrkmu5#-YSI?`wxG(V(f<8VF<_nV zRM-h0jCr^JQA^~79-&e&-#h!!dur0mXOT5?zhfrhn!L#OL`xj#QL}a6J%c96n;jj~ zLc*Mdn|Bb6+8&#=bcb0id|l2(fKhr8je8lWCg>xn8EOE6!aAZ;{bpkpt*q@m8$xP& zVKIqR%(O2lJbgPfe5WASp5J64;#mXwyl_Y) z8x^M70+ycd9(EqmXO6XwM%X~o;wB*!Y!4{_c+!K%{_)NiI`7VpCOYrDpEaVQ6Km!YnI9O}?)+i;J% z8|#V0N#0M~ttzgi4>-`(8>xVZc3e>7q8c3M8u##a!^0}6kPwB8Et!zSQV;+kc4jBx z_02F-HKkg-31jXRWnv!1+?vko+87#lTJN-z4{kfb6(i-WX%du4>sgmb2W0e5hL&5j;nCNyj4CiXdV?0QQO=x16oZ4){=zSH(&v}DkyvPm0GzC8$gpf zvq4NWbNI#D0*KjPBM2hj98xOd7b&%c)hp(Q(X`$&g7>bvngIC{wk3hyhW2e4+ZVUT z?5?~CZEG&uzuP2uQ`&+xw3-_TxDn^d?*W_H%v&h=0_U}=GfI4=v9PK;P`as0FX{;d z|JMMf>kB)#{2i*kx#iXSicmYhJ3`y+@0Ih4*e(8-5>UG!FzDM7=luH>+F?~-d zUG^_T-r{$ZZwu&Kp$C%Tio2ga=36O)b6)L%WGAd(8B$P*FY$X_Nqni<<}dbi`!$Fo zAMzarzaznu`zty5ZIZZRNune-GgTNYi#Sqk_(`*FKA za{`$Z@q4Z771o8M0fr&jtNbGweC^>i_P!$XE?lb4{VWm9!mu%SVxb6zs0sLjnOa1(`UG$HO9i!k~e3xf}_$o@#-d$9R zOaFYWJ6Llcn(5WQ!X*Mgl`m{im!QTKxd>A)a#|ILL02zH`xO36+l7dMT`vVrc>;3E z9{@g=w!g~Z!_P1M^d4MS=4w2#62DzH)_&PO5END-Tsh|yRwCXG`l74z_S5Lqp-etF z@qXvpFi4SMo(ha`>mNXL6ZVqvccXk9Dg|9DFXw?0tUl&KJrD}LAff`0a${%kfSWIl zpuoYoYuTEnkb9mh548#LO9l?&N}A~5c|xidpPnUVVeqHGC3FQ8xlExTUTjKghUK8US%52H}Vh5U{s6M&BzQ(u~9~* zlt*dRzB5#t9qwbcnq@EuK|oJ z(c?wr*h5osWZ~EmI?p@etK-6YeF&b{o`_W$e-VFxVU;QdH++z#mnw#Wf1p8~iw}#9 z42zK?En4<>_)zOrrwzdPQ!vllwTlYEYh1bR8~8IYFJ9D33xaxFJ{@5D17EMq1)aGG zc9*6NZTWs$*5$*# z=gZe;#2v;h{NimOz>l)fx;l)p?3Aug1Lc{-n2XY!Oqp{PnGL|bCi2$;7v7p^VcC^?I?V&!jp31&^IRsj9LSEA7| z5kNGbkL7bbZ$q#O_&_-;K5?HC8z)!{tVb$zd!(hn-5J-Z-AukE!=VR~<8*ZWD;mMc{62$>$yui89tigqWD z_?j{Y&*U|}6pOl=td6<)b8fD_0%E5K_>+(2);PrB7jB4e+{7(-!?6`1LBDM7FV}um z7((VeUln8pfEHhJ4my2NEcUhaSA5hcNPthw6$t;y8@xK#CNw@H{#Fw5uX7|9PFTYvu&j%?ST~D zo`51=u{N7(64AuHVmRjT#jqC-7xY9MI>H)+pwFX_yu*8mK$b-YRWg*_01a1ewGe+-1%S_T}e@28sL{u2kEEm6CzmJ0-3 z`mYWkafeptsC(L~-HE>_U*+3EY%zm0418R2DX+~=t}c%;{4t|q_zGJ|AdV3C7!>3d z{VNQMJ59m}^1L=s(v95U-j3kamWTF>h z{K9E!&<>JEltdCv)Hg&~K@qV{>%^I~N}P-nr??R;bD)@u!WBvm%FJb=o49cHgdcoc z407b+3?*P+)lckExHYoJ^JfRH<_F5$ix+@$Gzt*psn|?f(fdX4I+E)yx&=i!f{brS z1bRf$TX zSvt7O>wt7LTa>)v^1t0#|I51kU+N}p$64F%ACc|fCR#(j_CLIgfdDiu0`z}Co4BPj z@wGI^3{W!wS#WR!i4J{9O`(RUkjR3D&N2)z7_M?);maaVuqYNDi!;l>q7CD#iKAc% zVsq7^IpN1<%STpcXXhU?C}|6jBy_@c%hT!a-mjaUIr7h^y*vOg*6f(C74*Swfn3-%y7I1X+O#PxK$M>{$IM+j6yb%Xc#DJsX6oxneFMV6!; zLD;=Glws(7ijbB1>648F%yTgtX+lbBORkniQ&cA#seimY5>hl(3Mh1%x0 z4vQ_cn?n%>jI1mej(*#*m2pJ(t4?XDRE69`ufj!j#P_c%a>MTVtk) z-`EWx0~3nu0(U~uTAN#6OF#ZJyEA`DPI6MJ>Z9An=JQq#qS`!0U5t!DiP8qRrq3`i5z7m-074*N;84HvVeO1pj&?%fHkM@l6w$kO2%ruYHw()<5Sx6L}^xT2X3A$$$A_{AZ&Env% z=(ePoW36YW2Fr~#Xodh_6`Ng?ShEKx!rh+V4#TzhZlbem^vx)DXLK8{w_EZ?C4Vn+ zM$~UZq(#g0MayqD|CSuPHj7BXm$C8!!WgwObJG)olQ){>{^?wMtX%&(@n{g|r)rw5 zQl(pS7d*tQUbut5G>L$Z-M0j--FF5JaTVs{N{YTo479y51ed`mm?{lmPt#zDPYMKnX(Y z1vq&1_nw6{=BPYH}rAns5&M#!qu+cUuo@ zRzV4~sa6ft^wb(bJ5=M}3X6;L@x|m45!CGx5}A(%?fyR3bc19R#>w(RUrf*MGv!h+ zaGXz4cml>j9RB$g{zpJmwfhbmka<*=OMG7y!z27k6!eQL@E2D^eYkPl`9J{jnDhq* z;~}yrqmiJqp0S|wF~#2qQ_dwj&X4SIPwQhx-=jX#_{%Tcf<2_|^PT((3tlTo)TjV3 z@t`0CA{vOBUyxBultxI%S=p1ajx|U>Bsd~{Ip@DnFF}a0mIk3yOVAIr0nVu;a4F1f z$tu>)>EOG?0EtLF%WKcU#Ou;CE97;mhn9Ox=h;)0NCSklefqmB+hbJo z(A_gA_5%sOBAOcL<_KAHYjIg?@du9CO_ETVopO|wON^99G;zr^PSN!lr=29@c7#xG zMF<{l{Lo_6bdEBn1PV1zz+VoMLj`T^O&N3RBKOB!`U55go?vK6m3XMEs+Db8RaCl) zf~PHDM4o*O2*<*kG@5R@p|;d}w@lu*f<_-C1Lr?pAvo@PO%!hUUdzQDwy4US^078w z`4aLJP7|rZYcZI{cNDMf&}-rM#_)qvUVFsV314pc;p^-f^G!3;a{Q4{YBeX|c5{DbejDTn)Cxtk+mFHe^B zDdxF6Oog?bedZR*mfI!Ob7gN5G_w9S9IaFQ>13E4FLJLrG8tac(VCQko#hwG;yn;{aU2`NIOwAnIaz=8i4SNVO- zia>+_1eDE`R<{XClIFCEKnq|nwKH~g9npsM(=$)`0c1>Uo3VJa*(Y+$b2M9LDNx8? zz*@;R1gDl#46u_k4}!EM$+9qIK(i*TgG|v>HOe@*l$UgBV0zSCHFYK`(Son3-`_Aq z79PT;sB3Ea?dx>9{haOcawZ+`POz2Q0s68#{`h7;3I4eD-tq3;U;+$3m!*NOmMP$W zFvR!;G-V#V;p4y$#*Q8z!QljR)Y7@D6C*BxK1k$me8JF@Xnjsj>8JxKx>X8`l}en?mMlE^=# zhl)4~DaNR|iD{UVPB@Cm1$aNSMTTh3p(4V;D9XhL`JW$RZ{lcCP~r1p$>CJ!$Y{!? z74ahax-M3ZbjHtjlV+iw*gky;GHrg8vEvWWcHn5O>tTEywvvlPw-W-6- z&dkZf)op8LRhO_?bQjZ9(n*TFa>VaLLviO(B*=;-OWQn$P6R@AH+#O2#rtBsCHA=8 zafN)HkT!bQ+z-(@iqbI6A+H`E!r(u?Kdsf5E6uWU3$V)k#g-T`qU(8aF`#9V&zrQ! zI@xEVSukWGg{D%)5j;8d7SZS`ImKUR%$UN zYhp>E3!ni}ZT*PTw|U9rKO392Bc~!|T(HIlZb&CO0=r!nCV>*#;q-UQ{Qq(Fj?t9` z-PUky+qOEkZFg+j9VaJFI%c2PwrxA#BP(iaOWk@yv(!e=vNnmYLVRHz4nPi(e zeR!@`Nh|dPg4?u95u-MrrUviMF=ou2e67g`P>L?zA2;lngv;?vc-n? zmyf0C%vXRqh=!I^s0v7kCmX}HOq5Ce_yR36LT}9nvT;W)nk<*ml0noZ7G{pQ5+_~j zGLM#qhY)dSHV)5i@c5ik6O8$T}d zkGjXa$!@5l{U%2fQ`M$kVf6Dhf0Q(=R(VvQRw*LR(E4uw4Rv)K^m?z!X{>g^rj)DN zXy85*VAL0U>vzeo@2I!tt+8y#Qn31SkbhE%bPW5Q@2-N5KrJbXERJ4%4?*mrLR>jc zGRCbP;C%6?39gJsPT0}1*pG?kkJQ8ywmbYoXSQeaOo~u_dGN+qg8IS=Zc-cyyuM^L zDh3?3eD}l%nj3rG$c4hEO>-!7>0#(>j6i49{tDmiAE>^i^IAO@xxJhlPvTp!St||1glws==aX$gr^Wu`x1>_#qnG z2B{)YM>pRKS3sI%7hygp`992%uG8OT8p{n|R#YychHYho8$}4ru#jfd7oADlRf+N9XbEu6-~Ran4U8cIv3%fTRT6HEB2qm?M-*_^&2?SLr4VA_h$ z{6raMo<5x~7)K$}6|<>oS03a=V+(9#lzGkX2S)4Elp&HfC9v-i^%r19l0W;PK1H@F z{n?}nstGULi+^y-0rAR+@SzM=MQdKe1ZZG1u3&o4{B|y_^~WOa(%Mkkvc~)hDP?dK zT|Ht-Ywx^=P9qlyaU|aD6P2323KfP^@tjre^i48oNUemtR&YMcpjy4m^Q|iZKOhB1MG{ljZ zL17CNTa~yC5a0JMxSD`|!|N&-K+Qh!j6}DgefQIK4cF1fcPmz@dzJ2V#*OO+WG$Re z3g5UqR$s8s?rZKy;t-|u*fh@Syvl(0t0N1mm7teag5_IAv#BA&UEWUv$Jp<8_G(h%Ya?Xb!QB(FON=fY@CUsQ^8z zOlsMIC2rj^{*4IGd1Y8AR5mH#Hr4v%i?bkCYOmQsJR99kv~2xyKij0iFvKT+( z4xtPR)o)r$;8`K?@5!w^t1#e>m55^pKs|*2CL$W}3Rl5$AR2g)O0W_{T9VFMMX4H> zbd=F-J>dS89=I_*a?=am1qZuoF;D?C6>vcAne29U(|sqpyR zacvQcCVH0peJN8iVATmOf$V6k_{jl@FTMy^N6(5a)ikS1ASfvae1gk}LBIT-0qV)N z(UCFchn(dX1>3Wfej;2o9cXg_xyV!7ifzS%a_LOs8_{C|Fe*R{?9<9~BA|CAQ$1r4 zll(Gk#|f}3$!)y7GsySlQ|U&h*Upe|BG%y%$jG7RIb=jgRBry=*X4|N8N<^mc5Pjv zV&URx5DQ*uNjntCM1~UceaGT5MIofU9Y(&|^iZo0i@AwtOq=?9>;s)o*66(M3k7@t zWKmVez>>Z(;LIP!7<&qN->TY)U6VVVJ~afIM@)IkcHV!3eQK)N!Z7v_86)@Qg2#5v z#tT}Pp?GtQb7{1hYa=?^s+`4M@*TOLXa@Vl-Q$#;r`VQYo+I`IZ|=v6+i85b8Y*&$ zAParT<^lJELs*0+$TErqzbLywz|>P}pU^hM-xM`8z{EmlF&BT{8ZV>J)GG2!s@^IF z_jc`~kOH8gK{JSDXgGs%P%73{mC5#;AFvleRJf1sr1lS~#cYS{x z`Zc0F4Y08xu=Ra5&W^hHLAd9{hjYPqLkB1J2#YB6pc@4ryIfUP@!HxeHW@VIsbf?Xyw}RMoUm3#+l4J>lGT+ z0h}>JtO-YkviQ%KjAd3$h+WroCQTh0gw|$JV87ie9n$daAE%YL^0|7!p`%{oFOkjE zLA@}>^=;iqzX=F@fXh7|dG$P#=Rc#{ZetU4Gnw~2vgjmJNb@ihO%|#z%vm>JWn?mWkc$0>}=oMME80m;9u2~#XG-8oe zi$V%iR^W(7EHuiw*fV;#;)k7np)G;pQ)x7&>{%+0EC?9VV%e**=8TnT;Y~3#yXYzE zeQ(M-7!_+?Y;u>th<`PiSg`lCpH`qYs=}NW+!pk)HExKjU8%Ehtc8CtN~Kj(1Z*)) zm5`9y*MjNc7PoOD2}51tzhUsf!w+R`#_DO)M57xp@1uRkiPqKH(2`wRyJ6r+ zi8j_0ge-~;TGPU7qqq7dE&{8#YlNyRLJugaN!3+S=^h*EEE}aw|5&-k{iS^Sri~Tl zqK|Ox>5ccT_+LHN`s2=h^ z-F~WP*ri~Q{`C>UEll-<_+qs`(SW}*C`f_Z(6A^iTP@&J|6!`PGG?jVh#(*`B*2>^ zOfsO69XKQq1%eR^;Hkcd`KiP@CTBrp76>5;5d<+Ri=xb2NeU}N0hFV1wsRGM?*xD~@NQ7kHZBCdaPKM+e zzTF&jzg_aZz09;dAt@2~ZySR&SQin-w2Jj*AyI5I@%aY=4C$V~)8RR_5Iz}&gY!Ap z1ihVi)=vbg-xua%Kdt=^hdy74feyVVeR-{&BvPpz36wv_EH{K-EMR??Gyr5o`mKDc ziG%&vyL}BI>={Uc`?_M>2#$UejW9EQ?ZZd4!vi@JWjwROg87ZcQ>BmiJT;I{^*WLN zuay*}LGvCa;GGt-W_%o!uXt}Jf1ji7@dxfE$`8j#Z9u}RE^Ft7Fxn<+dr$H0xjy98 zs!HV3!5TDL60-So#`C>LnxEN-(6tgIG2b<;;P>lqe2POo<~MA{FM1Y+qfW|wz}t&> z1VdVGJ1N3g8VD+KG!JB&90_+a@6UDQH}cA1LYX zDo4`&Melb?L51a(U#)z4f;oqbv|KrhOokPj@X2p}?adOFP(|Ne-*lTm@|?JnmfA$V zzPyecUDZ#@(3iOlhWfxh@eN1?fB0QA- z1zrQjAn7JTCq>=-{+@RD;ch&{o{ENjA}b&ru!dk&6eoit9m24IX|;TRn|HQHE6xs923elf_Z0>Sd5|Z!kxvjh^I%EMkfG+6s{Mrzkn;3#jJqh^S?=O& z7W6M0&h+%5@I9D9?cgYBlkJkT`cq1k;aX^`N5>f*s^p%WwlQTX3pu_2_%R7!7};k4 z{%);A*>rC^C&L2hn?pjS;%op!`F(4LKOnVbZz4o((kz@rF+F$IpfA z<+83?x7m}7c-*{LF z(~!;!_BjRSrZ_wjlHZv+78w=;5f@j>?vS0js=DS`0OPK%<>w=Bz;4bKRy~XXTrECv zkkc_Tm%d8WNEmHKJ>C5_VccJUn^He}(Zt!tQ73Hoa6{F@QHP+5ubXaEkeqzl?wr0r9dXLE5$fm)fDFKDJlZp zfm)z!sCuDlU2F3%3pCkXEEDsnRauyMliIPJal*}b911a?nfmw<+xtE@bWNVmW&$p* zC!<)|hGX#*i+O_3!*Vw?1tSigY@^tjnxj@1GXAj}Xcs8*W3%}jFSNxM09}@75glW0 zC>>KjCnj)?4AWt~bk|}%f6fH2Vh&BGTospzbP|^q?BH{n37~9C2-VrtC_nVjDmk>s zsz*Oy!(P#fv`nL3tqRF_K!jE_a&uXwO38SHg23n?GDVhBUaB)xWnvSfk=Fa(c>X(o~wxsFB2Ae9t~EC4Bgn@Ktm>l-?%KDO+W{w8JD z+R27=SrhDn1A4xnJ8j9pY2t3cKjMI18Ur;G#GTKbI>22$kv*uHiYKa=y2jifV#*%@ zJot=PG8~e#GZ3EFum6U`i>g({4|qn4a&L0+ZpcrlN!aVFBiNJzobP}aJ}`AtIN?ob zE^*ypy7-yPTxXc9J2-ayARegJ>&4Gp;Z_gJ#+)`t9WphaL}(JG=ud zC4UB!HXjcg@9>-*Jrro&ED`!zI7t4G<^n_TSIi-L-tn|dWP97c;gMnNIH)=2r>Aqi z1i`zlmk>+clg82sQK9FfE)cO4>{tgjf@VKi1{qW*CoA9H07>bX3g7UT+}wKJR9qRJ z>zOir3t(?KM~-Gz%4#AClZj(-o-u>l?o9D`Qnd-vQOc}GH-Za_p!J9Zvs%Ji7>Cd+ z&(G&t@GdfaL@mrWtQ8o~Y$xzCmULsxuL+N>R7~)1_qv6@oMe0y(`uST_Bync>F?4j zKp8vxQaVI00EDmcYp1;&jN_pkOEq4nhXGQ{Hoo|tt|sgjM_H(T;WC_6WZYoqnyLOy zJxEs??nk44H~*X z!6AYaXx>J0j$M%Go&qOOg%~UDq;0Lo{xp8lGQxMUL2vQv_WqZLGe5-y@4RieyjCdq zqU0Hycx$7=feouD?s{eQ=;)cU$gx+F z=|zGaUVIbDZN5&Nx;B@;js_htBHMo!m36R`4W<2o?tU31AMuqZz26O)OTMte@Q&53 zo~4HQP!ASdrjWe8{bZmf)XaILw1)^HI5tU>kx- z!mJ{iv~e#Uo%k6dIH@RHhZ1qMmKV0CY_^g;5;)D!l;(_Y>q|hcN7aY7J{)jC+`uxkcw$DJoE2henSR)fC;s3#Bf=&rK*Ye1;`? z4uFhaw2z*zz$sW}8)A(9-GP(!#M0|7GC;kFCn>xqqjL+eW1N2PkrC_BKD z9OF2zmnkRbM8r7q(ujmTo7NiNyhC<6%=sra>V&8pX=9?L2-=xPAd>#rSx;tWm(Ckp zbH2m34mH;>_BVqqnytMUot(T`7ooIq8eq?JYQXla-*zqDaoUNhYDk;Ep-(J^8{dE_ zl4P;AtD-d;X`7?@4{|7nq9nGiV^UCEq>P1Xk=f)ICWBb!QB4#wUZZBTRwwCoJ&VB7 z+G6ELj3+f}zxQ!re2L>Ye%zAE?|QBr9T4`reLA6pO095b1i51P>h`t4O@kwm0yO(x zpjWHmqvg?<20fzl|1$l(2~Ctbfy=pRG4o%-el%vv>&|U0#iqu@J7}*BIHJbB6iL(u z-c7cj_q{t5VqXl>e&ph+HR2aBP8S+5)Y36UXMiLLbv2ZZKJ-z>+H z_}?hsDAhlgXuZ!nNA)nJKkI9e?E;Z7{}Mz2vWI$5N}eHJ9>)LVr9UfxWl(lm6VDy| z4z520+ZFs4iH0bVrmE%nGwGA{=ms-vp#rr-^H-5wl!%h4_Tj^MK$*x9R~@SuCxJuE zk5CCe8ZHpIFc~;54mG)EP(b^#*M+@-UoibSW8og5m>SZ2;dPJ0x)FCOsyp+5i*h5q zbg9Wo3G+{UE*;YKQIjFC5^hT$7)-{W>znlWKDC|N<5f1jN^H4TrU-)Dhi}GOY;55V zJZ*(cY<3EeDX-WS+Nx)dQEAVS;>|YyUXtqRB;r2&z zplfkn7z}MQ#y&)H1sFSoz!zvOlbSg}R<^OSb+mck3SH zs9E_k1W?2T0pSJ#0kL)jj=h0l{;Mk@0Eus5kb&3Guuv^;-Qc%S|6zDsf7q`e{xdAg zAsYM-n-i~|1AK=A0eQy(ilDL)0&CY{ae)ESj8H(QU_LkiGxL9HkX2gnp6WxGpVS?z z9X_sswZi>iff5yVXuHFu%4p)ZA))bu6cmUNo((=S8k`=$4i8eRMH|2Ef0YdLH!Z!Mr};dkm^V=T`8GD$#%4JUIGp6MoZe6KAofBK zM&$b)2#9j3i-6Ko4T;-Pg=QKUA#5ekjA%vEbhbezW)rNzaz?Dibjl*kK~V`gdznP^ zJxt1cO%71vb0T*@b7~-LS*g<;WLT#&p^}WOMFzS|uX(!iS@r~TdE6c{p+2-`sO9V| z2q4*9GWJ{6ZNDi0B$6FeMrs98LlR)tp((B2utR_RBAkJF zZVs@-wP0tjxoK*8`Lkqae_M6)gRNC(2Iw)qnM#D<#bGXt`{+K}UTtNw!F^f%)y&ZX z3a~bwi?^W3;>my59$$&%+2O`minKt;w@ZIzT+0~-4hL(CST3t;S8DjX{ooTNhFf>Q zpJ##GR^&4ar|~qnk48+*mihcUrvBCiVe+D|^zTl8w7n;!Q}4kJxs$lCk2Orcm9>g{ z4_q@~{)KvD(;xw1atpoA4~`4rKPj1;D}dE;)Hsy{&-}X22&|=nPp63ZBZRR5#&rHd zx6uwUW1iXaQt($2@H>`CjTC*_5JgHn*b0@o7g;3(dtH;fSsgOo6haot^5p~b!)RMf zo?NTn?YAhVqQPhE|@cwraDORL8rT*`f`}XW_? z;$!sG$LSH3$}^O0!m)_p5)%*5T>y=P_o>4kq!d&*hYFPx_U8CfW?C;I_mqthWw-;o zdvr?vQqI3KFjfnlNMekesQ(%tMVf#Mk&-g5WZG&7m2QVXT>hs2$P=e)@bvtJ*3r*0 zRnrLJh30ZJMptr;LAL#d<{v;TYtzPZ+I3TidIy00Tohus z0}1@y-5kdt5hmDrUgthrbunbj;MVLWVh9Zru*EC@i zf)$3Q-p*IPMyPrG*|mv8gx)F zQ#AVCGq48f$#FPJ_PduON(zmlK4?QpI zr{$Qs6wRqw!r0@7t{b|W;q=7#Y8Z3G>M&vl7}XSElu4aoRDG*rc9Q&9mYUcIawT6C zQihyzT-B9dL=MB4YSVxySkPp&gZJ<9-9t&$ko0j$Z6jNMy_DFe{z&@E9#L>Mw={8Z zxxf8c!#ON5j-G24JVl05qcq%9lYzfX5MyD|URkLtZs`7bx@G-YTRtJeVI$9ynXL^q z-&Fd;`LF7S?1uPW#&}u0^F1e*TN78cV=>q|Yi>LrQGcT-p0CIXV2Jp7!ClH&5X&f@RY&5g|!#ki8R!#$Jr`Q3dz z!6dnIMhqgHZS6l9n5iNuLYxQjKsJ<`Wgw0f2iELdIW416Dp52i?MGdpMGEi@w4j&# zi3#CaME2;#uzJjuq=Y7G+gu*jSbFIrnBcr|Z^(Z57V!{}-%gOoMRD*m9SwHOzerGg zDJvUm1Me{JB4SiWo#iBRr#)7e_r|Em{_VM;ZxV|Cb>fcyMwU7?UwLVCPhK8P2FA5P zo2xv*X@$9IiXabdW^W?gVyg)9$I!^43r-;|Akb33`+I18@mNRM^>wnNL+(D`;+YLO zZYC!S26r%k{lU(GxiA7s5IRp!V_53WO4--0mB&>gl}!_*=;GHye7?_+a=rm#V~EM` zzRKhTr?I~yAkU)=1k z-*a0;=Tf|A+xfyRf!FG25cW-Wt8m8|X&Bb2a)NcZK7wNK9!2MwArv(&6-luSmJf3<5M)`@?QqC@Vps&dmC2P7OuEH|CwhsDrX61;cnoUi;-oCO z-D?8T7aF>`EoXHDmPX68EH+qec~PvS;C&{5c|?Ito0gt0Ov!>QlQmM>)|y<`z`5KY zcX>>gs`Yl_ZYfX5{4RIb6NtB}P7X_+{V(%13$Jm`BHH4E;*`ly=rJFLu&s@slndoF z5;#QS%HUap)TT7!9zAP0lvf#SSn5p9oTcPQX0CH}y$ zvK&yfvfL#X=YeT}@>_vOh^Mc^JrV|xAz(K<(ei5~sz@%ht&A5$m)?=6B|pSld|W3r zx!1p;28jM{j?n_Ax#h;HvAD{E&;SJogMw5BHEt*f_lL}M9Km{}gyM`98MQ#E{jg7l zgXtRhi`tXrzcm=mRri-M(k6_Q7oW{9H35m64yZvm9L~s{Doeyk&Jt}-fg}Z>bLKua zA|LJq=gYF`K+gNhL1k$0az#|a%?UMN3gf(bB_R8X%@><>AR6aq%^_TV5Y<)~j9C0eZ1OXM zdzj7hkCM~_Y*H6e@e`qQm)*@Gq@sw;7X+R)jJnMZvj~v3TjZ=dA2+7qq`ne0fHzD5nebXy63Zjs(SNz#HuaRLDV~W z?UKSkS1p(L&hjLGL?Vw;CuQTWK2!U(bigj4zqJM6Ev{#= zdV)Qk9u&eycCUKx0;_JH?VxGS+XZ6(yNrbs*EK||urDmG_(2`Yi?(@kAj4!{jQCvy z8r*>qXn~oyKiW+MH|2+!xC*|=uH~@6l0a#6aGguKu{>L}br{Q7d}%;t*j3cH^9lUF z>7w}Wr2OCMN|_M}E&q~1ioyPKoG=A^pK-*O`0|qOxTN|_D74NHhCk4RkG=5Bqevbk zZFEK}-EWK#2R#U2H$a|lKGU}K*xWg;gC#79O|gEn*MQo}6|0 zExOo`Cbo35mL(jLN3V-u7rrx@NWWo~*2i!@X_r*?qR_`|;8E1NVb3LAJA_XpL3P$W z&}C25YLf*qWwnfm+g;t!%_xC!vD%@>DS*0y)W`kkfbruTIv@3QS^)LI?srrOg1%vx zdk>o7AHxWvV7*PWie25fFJCQyzp^<_ivJW4m6A2{D3+Y}H{0me$kRbaHalhqo+t2{zl|-!pUY{my&}ZxV@t3qa(vPl%n}zX%+4t)(z976 z>@>cOzIwlJ)9CTNtZ)iojGMz+`S~PQ64&s^KJ4Twjq?Mp`rPHKJNOb?*bRZhwgr-= zQ;PB1-!>#%gX?U<5IzrfZ^b#Av|UzM*i4Cuec%C4q#>F>xzrkA4nq8PyKGRD+8mB=s$Al*vTTQdn3#Zl6 zrBeC~>Ym(bIH3X~9yo5L;)x~>WAN{9X2%PU&Aq!t1V+02k`~zX=kA5bOR}zzjc6_7 zJ!8|esngp`2j~3&yVhj>^rVX99NGl{y3Jue*ySLNR0Gu8Q*K+e5+>0N@Xby z6K%?Uj>&DdDZ}W5w|9w7Iq1p85;_^{8sRs@`rJNanL=@`gjPpWI!7Y4j!OKjEM@T& zGq}SftojA8toL(rGi2!d*2=S=Hw`iZQn?7tn}0w5r+ob1HT=I<_rF4MH(1lP$=5BI zUPxeGEEYFVnT-(v2ngpx2I89WA_AJ!#~jc_Q2gwb*3CvmIY~k&DNr1Hr6VMaFhI2_ zIS?bY*SUQ9gmpD;>K#n(i3Rf-y6r0PbKV(vKA6~j?dpAXjlkzNSgK4$O6K45=-`z z+#AS(>r2_CS#@spe`}!!T0U!8owQ@KR;KW8si>ammoTX}k%Es<;w+chZn!j?Q{HbV zlz9-e6%@e<7={+k+Cj3iN8iaqO$1tvOT5&RvZI<3V&J@{^lZ~BSRmMC8SAxH+nwZ> zKihxTCv{aRV5VX%Lad|A3Q-au^u7G znB{g+$EIw_3D-qD5EG@i;pOJ~i&;!*RL1=3NJ0m}p#coB(CehQpbZfKEpDwK_mImHuqF)vr(# zktEI*A?oku1Yu^8A#2#}UmVyk)rAh%`bwXb8&tp{6?0b3q*+wzFs!%eq-9{*X6}!G zRxgTUXw|l|IHN0TU@u9PJ4IHWrUjN*Bj1M_`~Wv*ULkXkN*9MzOrFS7OIe-eiO0`Y%A6dB~-RjfWu!Sk1|9Y`7um_Wgy0I*eClf)3PMF7)g zVZsoFw;n4niWcz_Ry7$Pfa0=O5QPs76SOe%s!QQclrq442LC{QN3hVrnM+e#i~CT? z+DyU3kJ#vQE8eSnw0XMZvAnwMs_6Oj|G*D;dlnhuOI7(DzDiV2MCbXXIUue`9oU0W zP1U^q8R@_vUK>Hl1861?!NcWgCL^wZt4G^cgH*!s9Y$enRW#qIi_%{99IE)*kMe9c z>ly2MXHj!jOkgZC!7Pubv^fE%UE{7E^P|ABl5a<;l3o)|j4woPVG?6xnFpB%$DG)^ z!ryS+^CtB-C~(xi!K(az^6WlM{noa!+~Vawz_qL|#a*oC4^VYgLRI&by7lZ9?h507nR~ow~925G=Wv3f08L&^^GU$~ z^_Qt8rtkD@1Hc&5Ud)pz=Hp%nx}(gj4Ycb$W;A*SQw}4oXhP1`$l;fsr4rK`^aVR& zjcf&h)oEX?J7%P?dxM%;6WknLjYbUFmK&JqkkHxXJ7zGQhxUdO|MI@JoWo6w#tR4I zcN?7E2ww!7&|j?roZ_`Ljj+#Q=Q3*IV3YBAv-wq`SS9vm z!K;f)G_zOX>&6y2U7xNY6igW)`~X>L-9(n|nD&_^4$p*_v4$*Ci#kQzqjE)=y5pms z+_iZ}Jb=+>5p6*J{ld;#A?v}6k_^<<@}WsvuHYysA+kb+MtPG@%||5^=2A6quXta_$7~6+MB7B@|fQvFOTYvAOuR3#a0B#S}w)J z(S!>2P<9Gv@nSMXlf**M9-G$l9dWmrFxCqk)823fz@trn`6Hijg`4v~K2X|B{_wN* zl?FH)b^l0G7VZ9l?;(9A^)Nb`&2ozS^dh>aGUyj&9E4)dPc^1!63Nik&uC+AXOc9g zrbnLF*ho`xB7olksoEv&4fM4yXyW-{U)SD@Q=B(H$!lvDy6eSdl!3U5DSSN|Il{#Ug4pI~vd>_xZ!WsggO2XveR z=lpjLPW1m>!l}}-HbNi4{9NMINL#CU2dTVmL{+e{GH!dqJ z%kOTJ%W;$AZRHM`AI7L)|90Cum1sSnHA4~r?Zq~lLApt>J5|-$rZ7Cgb8^wf9j&?F zAEViRL)WP@Y`gBsXmB)qb_0ccxD%6ks*t%o#Tx5$+H#%N;#~{lxHq--vd;H>jD)=T z(__rD1zPSVTdx!Q>Tdgk`?No+^RF8afrt%Sc7`Dp^@(TIGv6W7@|`N@=Z*qon4#5zVi zbrVMeF&G!^7D~GY74e>}L5@QV4erll1@w1At(g0l_u^`aR~xGs)T@{i6*jzxFl&Ww zJR~XTj4*jDmPV@(i$Vc3ETmGgsdipKZ~KDSPAT5KZ42w`9G!7suA^qmr~4F+@)XF} z7;=bK?v`VDyY;=5UVVc`l+_^?>}LH5PVvmS^+tm`5W>flUE1V!tr6xd0{Nb`OZNhi ztvkU=81YK=M@$bj)rEyLd8@v?&H}b^vDYZJDtRCCydS6u`gN#0+o7C|+2}H0!284G z*z#4R*n`q-Rx~O){?Cj~<)WShdPhsMe)a{?Ae9R^1bKfyUn*X*lj$l&=ODCU!6NlO zha@=_KC2p!`{Si0XhWFm1hPV}ZES{f)@)^JN}hzs^G2{_10Hk1D=BOhur#|+=N^K7 z_e>_}Hzm6sr|GU0e5x2){SqpG5r_GeiSs<9{y1qGNxD=XzJiAcsb$s(nh(Vi$}mOB z0{_@i)3m>wMm_#)f}3#<;iAQwrq_hZ10FYrx-R`UDo4^=U8bGiY9>WB_7wy1cCJ!hdua zr3$9dNC^0`T8uP4ZIPKdys_R^koV(k>Fa;1HZ_qg#~7f4^6rqjcPfRag+SZVxcHV| z@mFJbNu>B=d~Lg(JD>Czix(uk=2tfIWjfZa;;K&2|*60icEhihX^GQq1M=g zFMZdS+zSDA0uDCDCb+PJKe8m`w>TB^AsILn7{(-X6o54f3HG-L7R9{7n{rJ*D^t|! zzm-4P^pQjr(gVTR)WPbR@*kJOo_Ql8c$QR&y&FmXRrZ zW0hm)9HxAxb+5JV@tS9=6ygKQZ>cL1+Ra{p>?RcTOT!FvFO7QU$Pqmx$uc_g3l!Ks zg%DDvx+Oid8~Wan$Vm|G(F!VU&nd}mE@Q>w(%eXCUJDNJHd2)3{hSOCY*H6jNz|F(*fh{kFJGZ_q(GZ`E@ia>=nLNCBZ zh3LR>MnNL=_7T0`5NA{~JSCKfZSlq7bR1$n-%K>-?hJi2=I#oOH6Gm*Jz~~NO4aC> zxUksuA$%rY^D2x8^#qNCop9-|4U2cGWW;)+SepC)|5Wo>sjI93Vtl{RZBCqb#}9I9 zJaI0O91m#trSg(Bqsh6T1*qvqPA)^$Mmyjs4^m249G__SbNb&^7;wZu{q$*27p)oo z=nKY*)`MJq)W+mR@bU>Dd1u~_7POZCF|ey4=-aiCs@qeUq<%GlHEnG_;>L?4T%z|g zRJ~wu&mNxJ*t{JFn2gZK4%bD;Yi&glnrb=7++tPY3~=meSxYH_jX3XXPW*JPd!Ln2 z6-O<1`qiu%#g;D=<(l6DyZB_n()6P+)0=k=3S-V0E1_O9F|h)Ltf+Zzob&aMBJfl3 zz5t$OJ!EZX1l4U1XKFK;|`v zv5M6oJ*w+|L($eyvO@h&j(IxtNleCw1uxvP3Ii!(H5uZZN{rD%R{X_C-GVzol-kj< z)}8y>&%*Xh0N6snifY0TGpu(52n}oi>$#jA{25K~tsx~Wg2ybFL+n}SF2z`{tm~|b zV3=)Oe!%89h3d{9NNx`VuauJQrq~ZNRnI^zzkq3(5jvbwaXpY59lg+L5=pLkYiW~4 zWIqZH$=X$!W;S#Jo1xO@Cf6_)XY63)bWyz%A0Z5w$e2kz)p}Wf9pMXKXE@ zAjHMT`IS~5jEqs`3OG(RHM^4Hplnn<4@We+gl!IS$MDe{#{n?bv;%Gi5KV0t6eSYBR`g3(pN;}laR(&=_I|*^0do(DU|NVqvFsor1it8M%Z?QzG#Jvqp(*;1 z?mIFFNG&1InHif9XyihL3SP*R2hCFf(NCf63tj&oA=q|JKF4gx?5f* z2ALa`B;ENLS%L#(Au$z|;G*)9rax9GOFfLHh4kZBqK~-aFUZX~(#CZ~uyJnE+Vsu? zy42wGOBz9O25a2&1J-8REV>+)ak1P=e%+pvc;@_J>mpo!RMW5@X{OQGdU5Z%Tx zH1;0is&L;t7DSVkBiW@~*gA;i(L) zK1}BB%q!+We528xuxDJFsbchuv3WcdU`V80EamW>FmP^t+Z6KL(~})g@Hc~IpWI7} zh75bDptHe|1W0}n9;;)KnlM44UHB0Pda(u)j+K+*2$Z01bUDmeoylQp~$lstzMAz(QnThJqgMV}}5 zCE3Vq)uTUt-q|$>KzB=NBhG2%bV9=l9GgFitkxnBY?oEzFsHVE{AKK)OfKe9ac~>f zO~Nh=K7ky-m2#vO)^9I67FrKIn^Y52?B+Ez&vEoQ?t3xI<5Ftsb{{;84n5Oa4*3Rk znbufyMm0B@D&5EKOf*kbzryTfxWe4Ph^f6zWA4dp&}WDl4RTLcIIm40vyxCM zgcT{ul-a#J=q~#bXNkzX5S^?e7TZ3D+#C{F1nP|TkQ%e?0S2n=1BS{Oyh)BUu}G{{ zW@n$^HrN<8Kt}3w@Ef?j%-&q&nLbH|r9|4HM zlkj|N*cpUZ@IKB>+a!8W&uXaQRX=s*F7lwzz_Ukj@vkys!dW7WA4c65JpsD*8)Ubw z3yP5{fB($~gfRRS;N$z8nwLA@ld+;mS2DAz6 zG$fx7?Q(*f5Pf_DdzyYk5q+jbsq^Q?xcemlU4zIo5XeuVUJigSe}M6(CxGeqUWOk` za$Cp#tyXf?U|E3Q9=qjArc}*Vk{#WdTI;nRUXw8|&OUE!s}j8k3x?X{OC1XdEpY0c zM;Jko_J^Dr=b5ZtfBK+4mO3HA^c%vtT6{32(W)FTF89w#!;8`UvKhy4T9Uqf*0`rsHV_j14NJoq zW45+7)SlKHzGJ1q3HL4J@&B>)mO*tbUAJ&>hv017-Q5Z9?iSqLogE1764Z&zrb=ivsjp54PMbV1uO4R8|_8;MV6eU!PYds`Bv===uQ$)fcY z*2-m5?>f3=eyIt{yDljBIv8@3khbQ9)BW+SgMs}&i1`qE`Tu|WnxCoRlM(*e-Os*B zcSL-^$^aTf@|%#mc|j4f9Q>a>ok1!8n-wVjk&qGeC595InZ}bN0vz@)6xE+$tU!3b z2^C=4%x(I0npi-NzX|cc@Ig_(38{eJ;q7%?#?uaH^(qz|@jUd{_1E2M$F>P?TL7u( zAsV`r0cb25`%Z9tG**O zJm8P;E5tHwq@-&?bENWSd4ZZ6ibN(WUb*MVO?SmA(zh8qvS;{j(g?2(1UA4|#Dh3? zz^Ps7KzhX%s&}l1u}-QD6>98bYceMxW3t@_;-+lEtkJ~0n$pw$xi~vlDN@W7#!9k) zq#M_$D|u6)Jlh|pr(P1FLstp-J@TZ}?MCL#g0tPMP^?9My`G9tfSSOJYYbN0MdakloZIi5H~ybDp#6E5<68f#cWniZ11yc;F_1Nmnj!Tf}j0Z zy{R?eZqCZkR3oQ21J%!-A<_RpNUV46fap z7S}?mpBNYK1Q*0`bEJ^`I>>QjrZQ0jb6fBP@fRe2-{s3kB16Fc1(IYTZtBbr zWCVi(D2WI`%tQj%pc*0pSkTu#aD--PH_GOJKTPDqUyJ`DNE|eaOhE`z3gaPfo>GA< z__y~b0?|0#NADoWks14=7o3+6R2qbg(7eYE&kGL8{{g3F5-#{I=zou$y&jbH;QyK^ z+K^NKV|sd1YW;f{wV>zdu0j9wiIoA=`wISFyEY8y4@jT zYrh7a_KixLjZ6FHwT%n==2!n^yU010%U_^W5^_=3-{*(1ukDYIe2*C}-A7K-or#Zv zKF=JV>W?9ix`A_7vG)6WOLK3D?0?ROy0;DNh7bZ&dHHQt!I` zR84q_S^PBLw_0im{^aj8GxBdsp!y&6DHvV*SSjqY4i?+{h9s_&9i}Lfcj8;wGM>99 zX5zCt%H*#&Yq3S^F^>Hlxqw#AN_QReFoE}{>Idh_6bAg+E zBdrk%w1Fv-*z**5TcpozTAM#*=ZRgT=X4V!L31k6Y3nT!$)nq>(P^>Cc=u7-NNqS;ezOO<~zLiZp68lSVH_afuqR-%(zT||xB7LrP|6m9x4a;Te< zqe(afKJy&iz72?EX)CuT8&+FP1=XjCq5|_NRytS0Djyt>Z_{@VEqjQ*p2Y^6+IBWj zwTWCSi_CQ7yGu`O*=d=Z%sxcilh)qZFLFXq13DR>xu#ix-Yybcy|AiyVHxner`wVf?oMJn57juH7PajxtaQ zYh7*BUuzqse(yO+JgKu9zp+!oHR?V|+jnS@;Mabf$+u@eQEGEYw4m}X$XxA0>j2Y< zyIB$KtEQpLtbqr02~t2Lvm6|-VqS{5P&xsRt?l=DII6~i6*cy=iX-6qd!2gF(V_8PA|t50~-TuVj-6m zOV{j{c)v-h2K0#cUSJ8Qo*$x{Y-%FW!OZrXT?cFJ28y0D$li>8l^_D|mKG>kpp{@s z#5&OLEvM7B&sJxW=q{EIC%lCju@su`B%PXZw}`;WN-kH56Fl=04|?KJ^8j>85U0ku zf_fDXetco?i$`Z84Nn`YT<6sJam#)>6SkNCEDM>6ByB74E0ujQj9>r+l|9diwXq71 z1@%49HGo6U%vG{^EK&pxr~pjDzxi&&l1r*9sLl6cR(=fsN|wn%$7wLR!svthw8cD? z2Db73O?A=;$(CF7KISZHcuzpJn1xce5^l7{2vf%#=K7#dq$AnErd}y+Nd3p zo5Y)luomwed~*H6yMl?E+IEv#woo1eRs@Gm{-{=u7FxoX*?F9LHWe`WU?Gds5xu%j zwWH456$cp$lfIPFu#;c;Eedee=@wGSA}&pq26|ULX5=*Y9bfF=umrM} z2~uf|3JbA+b8Snmnm_i=w9mnFYlDlmy2H%ulHqeZA6DhP$V+!y57dR5m-Iq5n&Xb(I?4NXgcadz)gQor5v{h0Jq}zNXPdz8^aI8@zr<&CaAtC!B5i`FyUc zq6SMR3RahS?QVLJXRd9bUAgm}9nNjBmq#d`io-@71@jPwZ4P6hoehe`FefCLghi6B z-fEiy60aQIdc{C0(<^d;K{M&5gaBp>41dk+%j{ic#WUA$S+ghF-U`J>;(xFUywn<= zvHZ*Co3(9TV%T9$;$pb2;w6|)T~od$+_U~f&Ha_{p9%d$usICF>n|ng6+3@o)-ik( zF-?RuKZU&!b8-jY`7($zfd*ZD?O#0aZsveC-|AgutH9JZr`zF)?bWRp63jO&hOw5LJjml4z7aVtlt+C(Y0v8|2#03wh}S6XoLu(Lj4I;me?0X0p}eA*a#UP6@0 zO^n|zO=Zz;V7{)HG@sU3VM0P(JuwvK;D^l*@Bmie{7C)^Z0LHSV__@9;V)?vU0R6_ z;wNKGjzmQ_g2q;fHzOMu-Tq%s)TJ?+botjxqzy^>#(LuO%r#d#25Yn+MLJ##X|r3a z`=Jh1PHC4s-u8jE-*9eR&bVtx#h}8ZVpXR}bY*N_I^HxSV4OiEm$CCYy;LIOK8?}V(Z&^j17PT}<9r5S<5R%D?lAETA>O7XohsLo z@oaG5!~~s27^NbyzCoK&~G0(I+_PiuVrGfS;feR)p;=Rp%((9 z{EIK_foCr08?i4Ssd2#W#37Stdm9k zh7Q78VaIc$#pAcpZs#l~KW=LQP>FImxY7hGwt9~EOZRO8B1$wTXDv~THb6emQytv1 z6MXoK4!)GI01q6->@O~l z0-a*ldG-H5>sJ^s-8X4}Qwvq$k#_!Ns~3y+r6vBAhx$ux##O5UeeQ2WRP7fDFu2M&o zu3p6*$Lc{NhHlJNr!FvVDh!|vgd^tBL+?gthySrv%wUa6I|A;R_JktJ8BNA;YCXf$ zTzl+21;w7f?2vgfiIDMWS6ST+nvavP&=hXjncs-XOZ(3>47-T@7?>;h!tN^1qmrx~ zOV!TI(vU72%}(EuT#uYC0DXYOG+EA5`K>` zFl4=vm&F*#_*|BzC_5Y~Yef#Qqik}-mrx`L9r zV_&Dru^^rO)Y--^eqRyDJ;$knJ;oTHvW2Vk9F~oc{PYPlp_9DNG7Ym^--<1!uU?sg zZ+}`d(#K#%4iO%-m=>7?+|BefBfCI@`gh z2vw~iY%)yR1iM@zX%onQYYe-1qJPX7Yvawx?vvf8OHpADImy}!8&%eLk2(q~&i z$ENsc{81*qLS?DvsTYFXNPxCC(KufGz&lJXrk?w3OM;5bEH>yzP^vP4;Z`0M+) z=0+?OjDkNf1$mWsYQBex{+J7K*#S_rm5)fJ6qA@DqxYzS%t(rdl?X9D5{|jwM?0n- zM}MG=eI13pV5~KZL0(BCrE|@Ikf5Oe(OkDsikNTe&i86TQ5@X+Sr8${X$sxZlG_tp zo&F*2#&0(&p3Ebr$mOuY@J7#FW#eWH%Iwt;8$$;8Qlmt8v-#%kq)o)g(oMh0ur&lq zWz@E)zF{VfT*H}q;gw7p?qzBfYMwzq7wy-p4DxHPouW`xFk2!vNG_Qs#BP59%iz%D z3n{#y&=E@GCLVjp@2II!peuJtvo))VwqUf`o6tF~oPWC4(pwBYuoKx{-JeY{?UVO- z%oPWAj&6{1>gK*Yr;kCmi>Hslv=bIBFN7x96@3LY6ByejI}I&QJ`b>;X>byoKw=+= z7K87rk_@S!7OAqB&uP&Uqu^TNr#C=qD$#&U?8uk(g|b5k6k5gbE(P-soe$Eeq*at8lh71bT?URj}Vf<W2Z?vFoj}z(>Yi6LAWD>BK9iN2Ke}*%PXD;OtG{cp^CH!g=F%C(aqhrUG0w zg2vC905E>;oAHxBxR2XuEyCuB#C4H@I%?1*Sk29vKIh3~&aniwHOv=bHq#;6QV% z4fAHODr|WT_S2T-z$}~P|r2&MP@J;v{cC7v*vnHya4*~?mQ$vWXo=gglvb9*Q z=n>E~?z_rp8#NZogYR$Zzn(v&$CCCaOh>?kFR_=(Fzn+X-ntBHxpsxkDC=E}Aci{` z%e2LCbQ2ylARNsxbQGSC&U6SCdijk_+$X2HZB_y&UYkl=5nRC|IX3%&Z25cczNTOV zN(YGBM!F7GoaLL2K29D&q{D2V)ZO?u2ET1a>$Sm{9B)RcOqi7J85$@I7oZ7*2=84h za@(YUb6p{(vh`}ZLF)Rd)OPIRez_ajo^tY9i`#>^Fsnt$OCc1Y5y&8M7l99kOV&yt z(xIPs)d`dt6!o6KSGSA+wiHcD(mPP^zG_eHMZKVB{LU7G6UZbfXQwE?TWC;C0J9}| zM-(EU3^PY``(vSqlGg`aW&XZC_Z!@t+I`A|%Zq?TLHJsE~npeDYvGYIO8>SuyQo1fgLA-9$V9^9l<)@ZfjTPO*848;!;8CbyLJ%Aegr>5opwl)zRo2i*F|-! z&H+c&%lZEVGz-*LB0*0Gb>DZ4`M!bu4Oadq`uE=v?V6ezYU__M;~3N@5b1A1E+Ff_ zdI6bg+79zyKEHD+*d(M;3bf=Fu>V*Pki&yVAtuLY#K1s?QzPHRx3FsBJ{$R}xf8w> zv=F!rc0CabZk|akPchffxSzVh)0H{?{`+WH@so{bT<>2#?`tBTwy6?sKe#P=r1l(r zsupLhai0&mlj;XuHAi{ksM58Uk(lF z6f+<5mD_MIsf4!%b16Xq!k1VryC4o7sBB^;MuaV5HGSobAx1X8|y z8HSKTLRvOiAt;4ar<3)1QMT0U1ci#P3Qtg8J;rTh38CO(djsl^)74ZR%e!g@sn z*$cs>hCXAsB^YjA}$Qsp^ULjg$=0PS~&U%AL@gc!$531D^*QNCXl zF^CmJ<+-C%-xZ)c(NoB)X%d>^MADubIz2y`GZ7G*F%k9DbOT9#2j9`ip%YLzY%?)_ z?v&!aq4Ebvkw?qER&oW{;@;CZT_n{%!Z{EPc|;XlU4CXRpQI0VGQSHr^v>bY>vs$6 z5Oz*`l6!~i-8%0BwO;{zuCXP$``&;>8l)C&IRezDUr4UA2P?(6LP;4Zq*+c4Zr=V0 zsQ-(^k8F((DE=4ia!CEm?DPRrD(L?QriW_m>;iP2d+24 zD27jMdmN3pJ_?@+*du&XmVzeftnJxiYPUiS#eay#2@FB$_4!bgHU8JW)y0 z<$XbHo`|mA<7?N{03B!}^8F!~Dz5%Es3l-apD1jUcE?j^}(bHugc`in>tQw!yc>$f%4 z#+|F~_~Xax)BChAcF-X-D2ANS234tRE;eabr5&gk6@H}~1Jqv~mg&xZMksBKA2F9t z{8UhCay%|Kb1W*9i9Aywaz=h4XJn0Cp|mcZR};E_RJ21p$_@w&ZuxQ>DSw1`GXBNI zoVaw$2}%^^8@R@Nnc9)*S$T_utF)fO%SQ#UYeXGkKSkP#t2r0Mz&{y7{{h;6LvUST`_J45 z-SY?zGUA272K87#qJfxgc?my?SM_bsg)s!`+fPiFFFkZLU|$8f+YG|$9g^067fXIc)sp4z3wxWn(DCC?F+>` z)FX<7$B1uxS~y-i=mavd`YKUeXK08{+s76kl#n1vt2~GvvN~nz{0A7mMY zf_*TkXAWabh(4E9lpBU3UgWADpx{*G9P-jz-WjoxO&&h+U=n|n;7+Khj!Rxgy$M}_ zGsub{f(C2Vsb$JKt)2$yqiN0JU&d&0FiJle;nThBK5Eb!CPqRB94<&Hr=!*Gn@PbW z^}W@jr)AruXQ)>aR0XyjzlpzG>FL1}F*f@LeOg z1kEot&Cd?Et-7?gwFqo2WtrUhij%)90&}yM)1d!b^&Z&PQ~$(qb1EutQYz|VRO;Ff zz8Yx8gR?_Nj~Y4Ut?%CnQy={Qp@IMN%KsY-ZrD5EiXUM3yK#bUk1+B6Ccz+rN=O7y zn?Ye5R$%`y#6vKiwJ#sV2*_w4k^mGM(3eUSDA1n_e(dJmF$N6yf4Jf<#B}w0ke@dt zWAg`BO!hBVmwAhaeT?|&(;n_$YMB7k3du_ftki`2N1oN;L#fQ%6ONo2E}p)hl+po> zG?HHUdj&5u`4@0l9aGlZ1Vjb%5N!;-Q<=!crY1u=EVUn4je#ZP!t)Wb<&~8(jarpz zHa5DtkJdIe%05$>AkgU0ktW|3*HNa&uhvJQ$BIXR`x8+Vh!j%i46Ug2RNv`<%w^!< zI)egzZ>9TsQQuJzQu{^AMhstu~iR~g!8B0L=N zp(P@_=1DP?KX+F;`4v$8#N(pK@(hGZG+^~g8)cKrGK(jZ^B>&jp`2!W=19z!`&zTd zMf+9lIsx5nd9vJpe2;0(NpA3$T1VclIu%w8Zg=w@2j4sKw&i99JMOM986)#pb1yKr zn-VQzapDY+$q=`6xz#{@>A>YHQNXc6N!$VcZ1h}5Z5s3)iGcx*Y{jEO(jBNawA>d) z!AmN@&(%VbB-Mn6ii-<~2udIUrsSf_&kmXLW1uVPzU5w#0s|ojbOQPesZp~dumt5ZydS(e3 zK^96NAZD4`ltrZtxfQH5PJlEJ&EGbk*M|y(9D7xOa@HiBkCMD1N|$sqYP01E-|oV3 zcG8|tj#jKL4kgK?O;nxTtMy#U3WFQW8@x)BSFGODSTU-ugV9D;qJU`xQub(1>plpk z!KC1WqQ)~Zvl#8&LMh(_{C*JY>MMl&j$cd*IYg59R*dl4X6#ZiOo0Ak=l0U`)2;r( zQ@^LxQMD(GLUdNDFGeW-8Beql708NNtiSsH>j6&_u{NsSh#^(_qlS-`Oxah0bXF|%9x1Ji42d-zl^vqh0yKyI zjkQQ%Ip6zdoabgQn*Razqu575yxKkrx$1k3gW<~9ICCtBZnmanr}+FnPLfaf@l#oqO&8b#!<1Gv)4!6une0vW zh4OBgOO>N6?jW~-0z;7;8%LU#9?DZxcI&hxzH`?V`POX%yO~A32HmukK|mxTbWTD| z%V|*qI<%f8EdnM=oJ3)}mWnndaajp>N}Mh66Q%$Xa7e`;j`bNK7+76@rn+-gXBnmz ztJ^xI;s6_UmIS;B>Hi_I(|Wy#;Xx>yU|gUXmd`TB%Y+Fp+wb%8@Wq3VXMMxA<@co) zJL>KaS2o*n)M_`IC%X5ceR-r05(z#StdrHmj~ur&CdF!_J+!Amcy;GUi!(1f4-swgc=!+=!yK&&1)nilNU=8pIIu@hSY!ij^RwReU87K?x znB>vQHv)1{hf&?>vV?Ve_FCI6elS`&J34yBJ7{`kJL7VU!?NmTI&uBDOq*++aP zSLLi&>a1AltXRG~lA>L@!}nB0Sr;sF?*qFq0S`fNs=b83dd`39K_B>it zU8&kGJ1o1puO_O?Uwf-fLCsawK%rzd9I4=`e=gba@FZuXpgs+Wl&@h;S}-jUkysuOrDmWgU|l5Y#=*M%$!wxqZV-x4%-#CqGh z?pwv29er3o>Z_t$Xo00k@?4NWo^5mB!VT%xaU13zzNs-QStFuYELc>)&ex8xD@fBG zGkgq(Ul_ZM?I`Hvt%G*914BNX1~*!3ubp6WQAZ(BA3lrBm;0$T>O*{yM+Up@r61S< zh4f20rLCa`D$%BQUx&Gs64AxiSKQh58$KKXb1qYW%=VWqf1x^gS5PH_jF4xpQ4kK} z&a?tk&Gs$BlP`kcR^W8=_N~0G z=v_1+g`Q{lqg6W~Qj=x^6U~D0rvL%6;Xto(GzdnH-{Ao2XW$%qCdhAsR=9#V2mhnK^YJC&yWq*Nk(3*9S!Yx+ zIH^LpXd|)86mkyk&+d63`3o^fnho;*}7mD<&W#|62|GQ}o+i(5B?q$Y0`8Tyv!ph9W547mUUeoAQ=K*iP!O*{@wJ1hA^clOCgp)*34-wRYaj@$w_gV7`wx)4e3ohFr2Oule%*#Ca1aQchnB}ys zrYD_23NaQ_u}7^bF=tg>SFvcXGno%^AeggFRIqHaF~VLlOPK_kq)vYP$mUtobB;~E zX*iAg2IXY3djx?ud&8)8mLYKb@Yp-<+;@i`gSd01?Zl|Ao-daP=Lvro&rIPLWwDdx zG~W?TAx&j@tMiY{K1$t~lQb_gm2g<`M_|vCXlItkr&eoHXl5xHrzDZiTQFOjudlfK ztWds7wHER-)gI`v@+82mUr4%B*S?@2SM4Tc??+``!RU@()3G8F46{pFH;rs*OcgP} zh!MF)-R+)8_3KOJ@^>g>Cfq@NEvie2^ySc?<=Du<&K5QPZ*+sgdQ`lQo!miqgWp@! zUH25OnI9tik4awz7zqdGcUi1nZ@HZv2O>hqG){Pole2)=%b_*}*hR*myC{*_*>4PW zhL^+KFOXI_)F-5s-7*;w!=*>|g^kDnAu_|I21RPbq-(NRlNRa`YB_k|dP-$Y(q_w` z{Vj&-@6;2zPF-5^$Io>&0lR1^@_)qd4D6@w-=^=C&e9sqB&NR7n>1zbS_8nxaObjXlSo{N3Iz< znPQ9!uU&Dl={N6q*EiesFHLSXMGHcK5=b0s7IA@fvdb2{nF^tM+(R@Cfr6=%{9W0m zJh4ZP*p9Z{04j7<4owdwlDYcUOMI~vzZPbpfDqj zIYj|A1w#@B1%3$n%Ku(UtJJKdJ*eXN^G)6KDJ!@lBkH|ON4H!fu6)`4K{IL9b2X%_ zfpRTe&|E$v-3lg4SNk%DS7|;PN@IauPIA&RCQV$R@K;6F70>IHQr+fP)NyYw#Ds%y z4CCQ6+FmNpBBh%$vd>B8ZtVNCZ31xXheG;0a3W)r73O1} zz<(1vXM7<)g9HaqpqsbbP-wvac@%Bs5pBZ9N-qA_NRTw-R$gq_XrkU0oJ`Wu7?~rE62L?>6 zc(NZnOtVZoB|g-@qnYbDULgK88JjsC|7Q-GHjW}JNS(jwrjDZ6r;z`+ww|2?Y3U!@ zwQ?x`mNso}eUvt(gMUT=>Uy9rVZO)MlgXOYAVJ%rK@9cQA(b>B$&-SN@!DZc&sP~S zMuKZy$&molE%~TT>73GuUlSXoy{W7cAHqRZRpM(s&c7Tg+#8t={*eEkvLp?tv8mM{ zY?!vo_IrLfh8G1%nbT&Cf-$stjAGqdG%wcEX2mC!ct3qTe-sO zDJ=w>-B+L3$=mk5b)~lTd6WmtNjk@F#f^VhS2r0P#@Wk)M&CIjfnxss-MBF&D>*4b zIe}+B5fGrzdMe!xH0FKCpijk|)kGa)A7v|cUp5HTcfEbPj?1&@a-{<7C} zB&VBI&iKJ7^=dV=%qO|rgt%=w$~Pi&l6_sNfW;qARai~VYEEXvbMH;C=l$@1L(*0Z zon^`(Kp~kMt67Wq_mhOS@tTH*iF3N46@Hs+H`9uEGT;K~WmKW6g;z1=d?m{X?xS=F=WRlwDf`07i0t~f zm6Z(rjD;3Jtm?525KsK<73rCDzbtN+>499;;X$2bi?$h4z2o6zf}N*2Xm>1|I}H!F z;B%I}{!a)r&Im1Eb_b|?9EH;8raI+Rj`axF6;3UTbWD}FECdB+xKm44RTM>L)^M-l z)5C)wrGYi|rI4|-w>USgWf!TkHyzq6fC`p1cYZQp&u>gzegn>R9gKVW4RRhn+T2wRIR--ccB`*J>1soHtF zLVulkf>)n->N9L9?ZZ6f#`xne)CO{KbQJE{y@iDr>>vZRJsP1Y92kxWqg?j=WOf~tnol5SRNw_5^ zp|%;|l0=N6kRdvk=HQ_^rp4wz=XMY;G>B?PfEJ-e(LiX+uIlOjN%G_49| z7153hKA7?yhy!;O1#nVI}!Sa;L2xa)p^6n3U)u~w!)q}S3~bJ@4QHB zAT9IGmT$_DCz&s!K*x_o`HMRjHU~fzy_{_I`OJAD-v0H0o~mxkw1y@Y18#bibk4G$ zt{PIe)8j2n&}_Lkk%fMl<_p;x#W2o?03|8G;VHsCqSHc%*gw66GK3v*8;}Hy|Kc5p;6o}mT5MGeXCWEQ)oiap6}MxYp%=@BQWbT zxN)biuB4Eg=o((NYI*c0oj{8c7{w@D_W5({^d4_tRYUCVdP zYloOW2UUT;ts1W1IXAr$=3ZgF-9;(S_wz$;=G&1X7C zXp7$slWrNtImJAxG*8^qr1*&w=fIq!5r#X?I=;hl{u4ee)ojEJE7<5geDgrxuW;5? zR2e6$Yxh!5)t+})FN|WWT%Skp-@ypDMsY-7;#aW?T(YyxRo5rcI@$Qs4^Onu3uXjT zD$9{)2erG;laAItzl7P|DJWY$C9oDAuHhg4b^~+)@LE-u(7e5mn zrn4OhZe?r`b$9tgFb6>4WwzLxeBxqXUvQM|&j}&H<3Y5s5)<7N>Fq0l0+9$%B}E+p zqIz@Ppn>%`YLKFj32M7mcFpvC9A6)#6Sd*_FRO2rPF+N;hSolKBpI3Tn&Vq)G*?%U z={7Cuu$xDEc?6TlQhr5NyH@O=P7Qrtm_vA|2wi``wRVVmb(8L`mmJOaUZL3ookp2n zUfWs?zwOrr5&WR~N&YmvkTjjJ05T-ereIaPCE!nh^PkI@UVA-B=m-)DxHb)w%TDb> zA2=thp;0T2lHe>)dXD&&Ze_N%{w-(kNXfC=?^0!zW~dG@my?A_L?4g%p>w7?P&ESi!8VJc16DWKB=*<0pcwpX4$z!OiDeJ_wNaZ z_n1g0X9byF*7hlHua1O8jGLBnvf6r5Y5G0lbwy|=XwuMTC-!1`l$A*N-DgxPSbXa2 zB?(;*>U#Mb1wzD*X&ND}?g`DA)FVdp8`pS$@%$gu{x_bRLzx-d!657Zp5cUHIU~T~ z{+caxHbeO!At-Wu&=ouC^rEK`=4%*Vj({q=*kQm z7E~_HhztD7cyVdhPny3rayB;o)UdOu1H<-RAeG9|3d5?n%XLVcG;G?KO5O-x=-%yy zeG>M&(tgd!`6Jm;->yWB5s2twahb_?l+o(K^dY_mz0JalY`b2SA94V#QM1}Zu`%@_!Ri~30?#IH5-)H1t6;|Z7LEwu>i7@@VeO?3qydqrVg_QUUAeZgB z@Bys%loCQZ_y_vB*wFo9OF7bifj)gEQ^vK#r)WjeWDiR?O_;GXA;;x z4r#@OrA^6W@~YBbLyY9=N}3ptJdx?g&_MswRu5`#SZzMuc#Rdln1_eopGH9zY095f zMoqL89Z>9rc`+OBNu|d=P|%pa(I55Zhk^XfoA z6B{L{t3QvZWFE?#r%B7y1eq(q*Fq_L7(9*QkmTrdHQl?U+LSUy1{<_s5AG}F9e``7 z^|R=1uW>?lO0mJ!F<0`5qQXAbXNpS;{|v zb}A7Wh3$+U0Ltm#5ixtMSu=F2gxDO6i+@H>yuxd3SBskade|~s8x*_U##CPs<1yBOtYxPG8&lZsljK0^hI|D;Tduif=gP{;A@|DjYW z``|L{Enh5l(`3X`Wkq!1SbhFOjej^07=Zz;PVOWs&h(;&XcT4Cpl+nwreD{hR5Mbu zEL;TI=jh+YoJK|b&fu?H8)SybRFh|nc}DA;fVh(s*b#wbJ2$|0_E}VAu~NIuuiyeJ zwslAOY+C(Sc@sf{;07GAsK{lu1QCt|tEM6_Pk>sqyZg0V#4$ZizPy1ZgZ5+?_-A-( zX{2bI8*?jYNXT=gnI*kw6MJ}R!l!vRHRghL+5C8HEsNU-hnxQCcbRul;e`^m?{AgF zjwOD@nakh4;;O633!G<%E9^^Ss8Nl4J%|xsX)#?v?n4rXHFn^t7uAqZ)qs~Y(}5X3 z;YgM?bDuB6Xgj7VgkQdKX9{P}14|2hc*M3o+b8^Fu56Nu`K)MGd}g?k$}6(6!rQmP zCf4<|*c>J?iu)9?CI;ghgdyPwbEWxl86}_0HN*nh$F0l3=QJYm*~h%Ff42WLQ6wkGoyD za(QBCPki2#rS)dOef>ws_#cRVY}~~kuIYaYcmIMHG@-(H5BZb!1JTSr*jZrznlCI= z#b9uMTMxI>BK1KiDxBn?FD3lM%@x{=9^n74g&c9 zOmYE0fWrK@HC>2m68vA&R)i`6>R)qGjT0I2U$Y_yyYYX_Zegey=zm)o#Sm;^|24VQ zpoadZ70-xM7y;?;;x+@PvKl{xM1(=WMr=NyHiEC3X6m18Q^HYGViG2)5J?$X>5!lw z7}OM2tDn(u_=3{lW@fBACiD`lsMuCg;XV(p>aE*li=e6t4W2Q?;b;ord;FH}h z&^jR5GLNIGcpklS@5Hi#16T2Ev<1rsDJ41}zMx7xFuVq9EjPNA1dNu8`kGPB`AL`0e%+1pj z!@ZfOV|VJ+>RqKdjda?oEKsd9Er04&_+_*v3;DblBfu?MJ{aHhmjP|n$S2Mr;JOxN z4KaU~-05}0EQON%Xe;_!r&a(oO7b3QK;w#ce|QT`Zlk7b;gMygx^rz$pDPv)lZH_# z3}{>Bg&I>9e_}}H@km(7WMDR|NCl&pNy&t7FxtG$M(c%o4UJ9WYlV0@3`0wlscbr@ z=AkbA_NJmsleDmoiK9Pf^E4MX@N2~|G|GkX`oR%TcfD>G@I3uH7^h~)8SI$>obn+#z(31?MLOpOB>qt8O;fbET@YHQf@ z5Z7V;Qmq2SO0<#_SsX4ZJ~5#UuEy-R_7^Hl;i8L_OfRSj6=o}zZQEw{=UCKKiS4@; z#YQ5Ba!ggjx&q4($#pkj_7Bb>q!UO(%hBL;tb_;y#xFHk2!)Len=aH9v4#2JEqD&t zTA2#uxDt&|V&CIo8CXurfFUIfO0n$t%zkmSy?oAn7&27a-`9{BFK-QDjWmWzcGARI z=E%nbSWhtOjEda#gm9SIC#C>#D5be4(CtCdyOg63=}^{qc9b~85(4{bdOh)k;YD@U znJkmhLN~@>SYymud$fr9qXnGt&L$a=kCG-V0TO&14-qK4aWF7nfUDdbG?F=E<_r|Y zdA|z6WglXThZDV-(`3%pDRC07CUpVvs7&_4r2`Ly0fn@P8)Q2CRl_d;(0O<>0DWs5 ze3&L4M^dt(z}2W3i!e9ghG&62VjOjy1oXx-+!yYYO0d?rAmu^Yw*tyyij?ZV2-gyG zdSdKQg!k%}BRZ7_M!`QJRAwf6{kCRY0)8U;aV8`NF^_KY;x@ov=JJlrAipe)rnU39 z`xW9M6e@W|{+wcRBJ~lEjcA)OStV+nl`zQ?>hxyB8X0v&@=Ce&)FethCCdc$G(7Yq zvVi8EMD%o+ZBO^Ubq@t^57-x&W6YQ@{Q&JkP0?O@q3kOHobr##h%}!UJQ$>M(n1#8 z88RY-Lo}fwl(p@HhPkY?T&5RnAo-_#>95s+|RxBz4!N4^&h*>+11r`o_#jfv(|d5rKz&} zmOSldcWD+Q?C5N^VvTr0y3wieq6RHq04FJ=Mx6u!(f))%8thF{>L<4RA0+&$ z{QESZeTL>+&di6LWbMrc@B4;w^Rt=8^vcL*DilBB*81ZB-<;ym2j(xp#hD zWcR3yQHqC?pA*|U8(Tnhu9z4AG>ntx0w=Uh(@ZBg(sM40JfZXT6cqnj30};bPIB`{ ze~g*iTi|}w`D*3BVCjhGZ;$rgu-dT9m%?h>`~9D9{=3~*LQbk5hIS53$8irDDicJ6%mw+w9?k~ z_8HdolQ7nW^2)%hu-4+FP@TH`FF&Ut5%!N}r!gt8UUjkR`9@(iXogU|dP3pf5G*xR}h>Tn^~s_31fVTZ}W ze{`5!=r#YzTnEfNMBP+$tq$_*o<~T`s8MPlbgQ_NEJg#ZLJ%)*+Wek^+J^N^TSDj! zrZbf~WYV=Sm`|(sZk_$y|x~Y1O+Vb>6Iv=b;d7NPt7V`4umbKdGVL_KE;3_ z@0}-K6lx%B?J8_h7?$+t$DQ~4N*ne!dPqYo(F%0TehyH z;A+j_%9V2R^{^zp2aC$QhDTFbJu8n;(7W^p-ViuXIRCXIo9c`o=A83ybRh<~Do-e^ z_c?0Z42i?<4)D5*rtAy2HzlR%_Eh^-y&5tjN2ewi3Tci6hxZmPi*4%boZ5q(*;^`$ z;R(Q_)Kgp46(mTD0f%m1q ziH$A>Sl3I0lSDeA&FmVa^a`(w>Bq3eTUiXKGZ~LUcd6Bl&b<%#S*i*le!ACa<|q5U zMo%BT$?uYLyaJuX8GHKv~xl6QdtHc1EloOlQ3-q2foJ9h`5@eIJ`>SY9ZL5WX ztrmRNvgUAbE9O6an0?+%zshixR*jja{&Gs}bj*`u_An7kRioBt{l4xadmO19g-t^q zi=2~&CTThgeypN|?C`DKY(`=(g5n*}UlbQhiKVSXPBn*XvyS5V55YO@jv3`$t$Uo4 zbZdOu^h{R;+}pr3$9|IN?kBSBZ8b8WF;c^xg2aJpgS5m+$~N=rC8}VhOl||$ND7$y z2vPE~JD0$<$iYAe&W88MEsE;sn^qLepCL&tHKEZ&p;2Oi{uAfOfo&t3A+mz){G%b0 z@tC1cqOmZJBI|iJDVLypW`fw2_WG}42ck8iKId=}+1g7E5Ao8wLnM!NZbmWSS#_v} z(wfCMvEXn1*zFDCPKulM@Io`EOY*F=K{jv2?ISqW_tu1zXY#LAMi-b_7DI=Nm}3%l z8!O*=d&0&pNheq`YT|9c&6i*JGcUmtuAIk9uq|%^U+3!O#zCwjnumj?9B6|5J)}XG zvFfil11mf3$rMM0J%4EM$4Z5Pb&nGPCB50Z(L7kgM^DBYJg=nQhD9fuEze`x`@j<}o~D~W3?{$bsNefV{pgxp*s2k)52GWZ(1DoXWrNyy509tuq2Rk) z7NClVqP>(?9Z!-O2g$Jz6+5M-l$)xY{_ulZU_V*UvBxzO2pTdw3_G9!zM9qB!vi-m zYHND5Om(Prfn=$Qc(|AHJ^{UKO`}=eW-)7Lem+sk`yr zIu=;Z!2DBSaRthsiyJqfd2*gMZ-)%8kL>I=VR|NyXo!71&N$M0%*qRf)ho^6J3ZZF zwaQ^<6J`ffOu)lLdcfAmw=3uS_W)|Yk%jK2!tZ>VoAwb0J^bt_lXw$C=DD7ko%W~` z=_OdbUy_qVArry9|>B$2MMiKl%}}EN1j>9Fh&{du-_ejYCzqgF@<}wd_)v z|5|0PHy7xBona8daZ%jOE;KwJzmd({2c7VD4`&fg-bH8$2O%ia($;9HnZMR??}Ex( zt~u8NTi@I%BTM#gB=6yzU!Go_blrMGy?Sqz7e8HRO0HXmRfCKP@q&}w@~W+qZ%CXX4Yne*t%j><-2NajbDNe8@T zbNoodHLJx}Y0;guae3tAwSHW3eALL_&aiEVi@Ot4&+9Ss(vs>0UXJ=?)O@!2q*Aeg7r zzZk12%BVQDzMn}{8VjR8f8=S*c)ASsT))*Ge5C)nFliLm79MIBus%wCm!o%!zQ<{< zK}=U4?Yy7P%7*~eG)MDT5Ko=%p7sw>SR=ZNP%JaZ2D{Hfp(Pm=- z7mDKw2GcC!s?Z~_d6>zP1tN+S!cS;>WRF$Zj2CW<{Gmft z35oo#n^VYnhW4M1rM|& z88Z%0{Qqfg!w~cR2f#QvWyJo^)z&}Q#Q$lYRUkC}r+J>uNe=!GfZ^-c*tPu$D3^o4 z8c_*BH3gg)ph|pR7+{jhnjHoUsz8Ew@2nk@(rDp2IJs_*1{R7`DXlsbdA|xR*4@TR z%6!saaRYnuXLVwcIFCONkMQL05G?I`xiUJCDEG^|pU#oBZ*NOW_FoofC@RXx*=JiH zj1Cf!lN@JpbNloe*4O$81(Gfb08+b)tBNPYy<pZON95T^9`?rT|-Lib? z6o8ET=vR^>IoxyCDa)QdD38DcsUx?Ffh!gxyKuP@Tk8CEN~0^pV_y`u6x+N&&u^^> z7ryjON&^(%27m-5iI>dJF%~-kX6b-lr$hd&z?HCHyjA-XdbZoVc6$=-a~G#ZJTQq4 z5mB~OpSX5Ryw;JgQPPq16J@p*pNhn}sid#vQ0hSzF77t#*xyBi_!TWdO{m+{FxPlm zHXn)&5msUkrpUOaOLEl;$3|M%*E=7fZmRcYGeW@wZlHIFW(+`iXBl2Y$b{rEV2sF9 zvsXV$VDJ(y*&)azU6Nrc(eHA+G7@W%=b%O65)Oe%>xsF?UhWF`tUKP#a}Z+U$HCom zChc?=xr4P%r)0LyWDJ4At+9M0?gHPZ*3e%F{72jMe*@r?)bd|oZUO0YTqgR$=Z1e? z{sj%%mZ8D~UFuMwfZk)FV46#ym?|Lt!Qh-vA%@*RLd~cGz!4dU&x%B5#u7u05iefo zVWiO>>ijMKsMu;9?&hlnX0r?e0(rnfs)1ibewOs?rCK^Doq;3BJzmy?xwB_qP<#VZg^E#S)@10KMXA*^ zs=yJ6FO~a=bB}OoDJs<4-AyC@h^IFCkVI-~l(q7-o?6pmBzM;iA93=XYEAdy03(SM zUnRHTnZx0Tc2o8y9^~lZ_B((S73hs(22JQjEoBY-z>zc#%e7FT5i8z^Xi_tZOJa)J z@pG*U5(Y*9S!LmdtXSG8Cf<>e`!dJ7p%?cuW0(95&7O>&w`MlWZDO+pH;f^1FiWp% za(*Hi;7~bnZRf736g@Tg%eodajuJiE_=66c&(hXijW3saNiZEK{{oIBD| zH&!xJ?HS(>--v)IO|3Buuc~vaP;^Sc z_vB#YUZrwp9@i-5m+t%-R4Nxto26vyKq;?#Pjo}#RAEH@z!LsCh~ADXz>$L#TK5V1 z^IUx;8U`hvw|KQ$WAuF>|Mlovx7JR=GHd{4 zWv;AFdNcr1T=CpPy?JFB_OL5=KSC3^z4W53ILnMS3SzDrcE!aga=MUlPj79^cA534 zpBrXOrm$W(e0Mr_JJoC(LxRz0+9grZ7+v*K2qb^hm6x}TEMoFK;-Hq8UxuqwIlk)* zsBz6y<=HfVkJlE#gptyt8R(Hz|G6A0pZC+}f-*PJlTDuU^v1xq9q~+zze*SqoLnSwh!c(3xw(&lVIU*4hmy1kvrWb|!_nO4TLU6J7ZDPn%Pe zM0I!wT5^8~yC@M24=aMrz8R+f#U*qA>MrX+*)PY194~(JTWMP;T2Jr@U4T6Qg&^cN zJb>OiwC{v9)0>e7)yW|2?Dz_*kHCyFKoKM{PhI%wOZX!K-v7~N^-&C^ZnNbo?(5#o z+F&v7Ww^=<^mzI4HQ;UNc}Bi^T%-VT(IG3ntCEzDxi9{h(?%1sd?=*1`U24asKLkA zs&qyxJ?B+#ku|^X5hlWEwqPNsd|lQ>;dXjCVY}qi@e^;G6l_F)3Cj;B7Ph6|?y_Mo zc>g0FufxNq?lzDolZGW~;w?J8J7{R@hD&=*jLRzpw}W>cnc{IRLhX$8EgUw_rztHh zG-LOkXI6re1N#Ys?ILzmzH6%3cO0qi5p%QbWD4w7hWD|(CFj!Y0nTWtE)$~|B zcaZeSE`awZA`+>r=y#M>!@kjO zd}Pa0IN&oEJK{C&0mrCp&?~!~KI99@SwD(gK1s`zM`=T82Sg^(=R5!d``@?!f9HjN zb|k@o;x6q^Qe@NTj)W$G1qm0#{8{xD$n$TFd6I^;D#oYL)~2Z=OJIZueU8SZ3LNZJ zfI3)`?zAxa^p~%dt7e}5NfW_#oK52{yl)V1RcvP#UAGau`O6A?XCSwtj}kW9vl}~b zIB?fM?aA{MXU|FJNl%c_$J-V5m%})0ZlitU=-ZA)t)BYRlzop`S42KeW1wtb+>?Ko#f;qu-qnWYzR065WmcY8FgKSWZkml}@0ubZ8vbQ^f>9wAzy|A1*& zJn1ptEy6s-Y<&X8RS4P0qY|&5ZVrcb@WH5f`m!<2F~i~4AD#BUo#-#pgloun@g#5> zNq&@IOm~(XC(Q<@8-&HaxEwTf)|bjMp)lv`;tp^iN)TpQ|| zQeDh{&?^KdJOyBk>3Fhn?(Rj2uMiEQ;cvwBJ;qAi=SfQ9{nDD;UH{Bqw#>%)v=6*J zxRvGcsT<)Bg`uuG#&2AyiC;0vx1MP86p#1aJH4p03HBbNlDNUDrAm+}BFeX~qVM^H zEfncXW0epxpQ+@>B#9O{rv|j@(YqnUN2drcDJY=t^{^)nG;A|USNCpb28uS_XPS_M*18mEXkVUrV<_=bQ75wx1C^# zc4crDmaCzY_|FVEuLscJc4=|3-j+CFocKz4p;zSlny_dJS2Q-{#N2H~aQblIj9~L^ zpslY&*qkI?Bq@v{w}V%E)4>}Dj_DY5IR>Flw(6GR_5#`VN`UOPeMtxde(;DI@rhS@ z@in&JxjffyTwuwfMo?8!(7E-SNQKhV1zTzLf#kHS;~0m`OO=w9x{7gQO#!0zV;XZh zQI^rijML#mQ#P=Gb+jX;3vVf7#ngVndMQ!KhU!Yv%?#lO?>mDbfvWj{45m+kJW)Nz zBPI$Y?^c|XE)Xqy#t%e#zrO#HqG>4y7H8I8H@Mm%_t?2%qiK5BBVvN=?R`yvC)<9m z-_~;2JNxc&t^ZP{kYod=ljO32uZV3+-b{?8GT3N=w4|~xUoM0{RCFCytgjpL^|dEP`M$+}>P^ajI%m`cmMAf! zyGSJV`m^%jZpuAdL+f|V0}6!=ZAsUYap=6MS6DRm+@Kn zcB|#%A;1(Ay$aZ(qx6H%;=$efWHV?fGc$RgRqf@!f=~44>2<|Ga!x;hL*TW%W2&qP z^NZUs#f8eDez#)80g(3U`sze91wBQ^w2I4C=WBQ7*iesH8M=K0Cl{AsVQh5gQM5** zn!N=6(eF%ogiCE59;fq(KYfNknqon1eZfV6&;m+kdNH>dn2>}{V*l_OT*dRWL9XkR z+1%?^l2OQQ?w`b%Cky`-Ytt~uB|>i;1`P3J=*wg5D#UwmjVY0}J+^yk!#W|1C0rxI zPH3ZFIWP)u7aRf`&uQJO{k9tJw!**%+Mqe5!sP_px>tw(h!XXXkFs8&#dU-nI;BZ`~JMY^i#od!P%Cm_y&q!$+Zy z2LwalZtEeK9=#z}Vvxy#GFvL}v|53pFYd&F-EuJBU&_^)7$EqaGnjAJ-vekukIK3W zEvDom`bRj27d_NXv)N_yJ9Gl{X+Y2%VzmkxRQjZ^Kh^}xO)Ov?E26+_g26m~)r{I(2skJMGg{3j*_bY&s!;v(5NY5zE!%3Y z?`kacwa|a-vSpCJ3)mTS#A~A5%^0l}m%p>+NJ5&>Ezj3Uvcf)qjoe`1t6rV0QDq)W z6T@l>CZnXHUsKuA+S2G|>Xa}MJ?l!Vw7uhd;Aon+vHaneE_@GrPz6hlZq5&g7RS)- z4HLHLM^yd=>)yhFuSgEGMuzPDTz^aGCuYhSdp(m07SA%vF3R*V1vXa~F=7bRR1oP$ zU4Gy#^zZt=4>|-W*98{o5k&_l;6dHbs<-oNg`3%Bz+``aZ$QA~&h9PJ-uMlVYwCk# z?H>V&+{nb+>T;3+xG%b;9-!s2p-x9#Uu1{Cn&(8#n|^BjRyhqcY_^JlQ7>WqEmLuy zgIa%0b7r%E&hzc+R-58{`_?%A_H3@$2a2DiwIXsT^F|B`IZrZ6M=rGvj|l;Imo*6X?Fma=n#Y6FBm;Zk=m+|bg~Z$CjO;kp3^^@+iQ_D zS)ybuaHb11Y_l+NR}huKx;CPamPxc&e(BbS%}ts^R3;7pzgx@S-3`be*p^V@mPMq1 zAA?!JdOZwWbx#nZ+_j~dnB&CuW zTgvp(YM6~tppgF242NB7)4Nk)|8?XYND<>_bX64M)8mAJMh0T(`F^bdw2#sPlz^8U$8}KSt#uhi( z8$TNyY#G#vaiP+;#Xx<8bnBj0xt*xUzNzf`PF%)6V`+t&57V|&Yml9$E&jR#uI}q^stC2=$_#Nb zpTw!5P_~X9t$sXQURUO|(4*x8q{e@EvFln0df`ur@GLX;aZm}!9)Q8E6e^g0_78(9 z7m8!|M)FB1tmT>pZTi(KTt`g8DOrBR?1bV0iF%ZK@S#&EoQhLJ3wLPoCbP8>xwJRB z>f=ZWdxth!zMxp}i|<*7-WSH>+e4RZQNfm^MklMW79X~#Kz$|_*p9Lag9qG!c>%%faXOWi#VK9V=^2(*=W)B>$d%$N~oNI2NBF)tU z8SxL1DHB*Xt|D=TNv9>$tMwqCB-cdxJGHQQ(6Hb^f&YR>!38$($vS zp2%dhJBl;bB55R+O{tx}GO;=Lr%=A%WP`D*(jHHGZg5<8aSVukhFXAQi()UcS8+V& z+ku}2|LlCA4M|F;usCManOKBns46Rf2%Agm+)+UE<@-h4M`-tvD>)sD<4v8$HT&(g zPb0F6n*Hzm8EPIAgaKewA8cKRgS(G{FKwiyVlgluWw1NDk6@hyk^Sfg(Q3$;HcjuZ50z-3Q1HhA zx`wK3!@CmqanIDk>d>VSKAu7elGkPT;<6@nMoCQq-pwCPjkRjaO=0HY)fVHslZ82) z5SDx0*GVLZZft&8JrO(dxA>NW4fsMVtY`dM?V&ASj4149O zZlJu!|M+cz`I26 z?|1d}OsE!063^0#A5!kY`T8sad6Pj_v>~r>Kn>ghIZTiLq(rrfLoHFTJYFTyc z$~{>J`MIUBTV+n};k1+b`@P;R$gy85Z9lF?B3CWVE4B8vv@PZV`Ha7TXNtVAk@fNF zb4^~CC>hG?r>?=(r8y*6F{UU6fS)GLnu*8l6;~D7Q;v*{R0zOKqolh%6C}jqE_UA$ z*@Cu;5CuVMcuM?}u^C(5@ElAj8kgaH*VZvF<16MEc>mDG29dv7a#Oi_la)dFDOmjU_ z_w%CDc#^uBfa!o^RX05z@n^(FgXCdPnOT*^rp3g`ttz!ZOj`ZF&QrGZ%R8!#notdw zrfpApD4s0${dYKbi`LNI;4=<+&7%}_gSj@eNO2MCb;26(@KpFfI7+m%5lR1%+)4SW zc-AbKf{a!)4ixi9cKf=hMG#GsphfsG&WQgcq@%?4zKR)hOvQ-O3gk?f@TQ@Dex)XM z1m!M^(^Crxh*~bnHYNmx*|;rf2ULVLrGR@;0e5n#&w(Y*@042MI#%e^3WLD@b1W%! zwP0w$RM0^SQ)dw~ zPvvj&or>hI+Q$NtA0o$yKgjOt7NQ1vZ%Whq(tijzxyBrs>>GRXDUaSF{C*ggsPnT7+!`uZk-y% z#9fJNQnA^#s1(zXdqM1zHZ{_%@n9C|t(2grrahb%U0bJM=a|3|@F0t+4%*6cS<;hB z-ap9%;k@QR8uP{P7>_Bp+CD1wyywYy6g?4nDo`VOcziOA(kR0s9KnUfr>8+a;L%FA z@@kTH=DFdD)Fce85{@b#>2@kXAvW%^kzjvk`K&Yt<|kD^x_reO(RHVoUKYayI-G_A z*9P=-u@A#9CDO5rS;2^H+e%wG_#;`&WAX!X*>=UiC-9t0J74DbcNJ}PRt!_oT>9Y# zN{A7)Ua|+8oh1q);F_k`qZT^nqU{)z zuBbJ7fYVd)N##=PtaAQ?T#Oj$>kf&>D^U~X#EHFxY;66+jO&RfHiqiPsi}dudorpV zi}D?cvEP_`kB6}n%~MNypSLE2s(eia#amf)`1QP6Yw;CxEe=Uso zl(pASvFXAqxl#4x!odc@t*$YNb&baxdue?lscO)wsxj_Nm%2NX&PBzG>wW~AC;#g` z%eY9EKLVoOkM9)Kq`z?{Eahz2mEa7%#yEm`H_4`R2U3px_K5Xb{Y*~V;E6q!0e9ZS z$0Ew0(X8p|h{SfNlOu^2lzpb3#JPD-_bUma>lTgH`2%$&#!k(CwPfbop&;!<7t{1r zovDBG^t{{^Xh44(-RRKddrv%fzFzHn$6Id7j*ezld%m57C$2U@O49H;EEk&AqCHsQ z$;h+GZMX=&de1A%^HqzVq^rkY1|~+w99+Y{Fe4K^@X>ZgUr9X*XDxOIJZJxYt0YjU z&c7okZ1^Skk+KyB7j6uH!%H9LSNwR>Zy&m)?uQ#>aL7z}b%>Iav;G(Vvwnw<{ktVT z;ozKP>@9DE;PxZ+b%eVSlX+`&yIjD91OmVpUh4BbG&X=NKZ{E!`Nrn|GR6Gy zas8M;qrYwoz6!~w)?oBfm*o%ZUS;`7@g5*!-_!E(KJ{0ndVKHH?adA=6_DSR%vm03MZ|-Y@rp=JP79ygZDKviJbYC2y9!cOOLf0EIQD62YD#Nqm z>XnglcG8ANqK*J%#8nIZz?ZWpNQNiN)l2Et@I)gjLAs@#n}$6{r(KL|WPZAf!J753 z1Sc+uJrN(p=y*1v0HRU24mriF9l|Wog^8M9AogDIW8G`ygXSg2;BQUKH_%=+x2^M^ zznolMJsyvgw(md&uRm)|vX=yFNso*sJKjyaKG{w0xwcFOP5LoJYXRf90U3JH@hrG~ zxkl&UzWS-q#{$Dswv9ihf9FFJa!GYLNNm4Yl)2a6A)1$WE!9kOx<@W%xh98Ykfj#c z)6y6K`Y0T-ec>-TAxhcGSkz_jzDCbn*>i^cT?fcxXXF975~hIoqo)89?ygc!#G`fq z=~tSTgr2e!DY3M%gV6vZ?NfITf_>RwGeD1V6+M#>4fe95UL)t*;@`9TdYrhl37=@u5SXG9}w2pM72>g^MOLDyGt$Ch9zT$BTVmu zX|L{!yTBtEz3>hectN+I9MKio{!@X4x^uuX&uae3qt{tKCXpa)6zR+;F6~@b@CEx&Y0kMGa|aa#leEH|aYjLB zW_`0)E~+b*+;kFSA^WQbXf2j|qa>|a5`I81eMP!=TCAk}gbN)h!FjGA9(qN(xd$s1 zYu9eQRcA!aUL2&FKg$4!U>Eo&x~Ft6_^V_1H`8xe^|GGy@Ky&cGnF4rI9lYaO7#hgFbP8Pvbf*P5dH!pKO5fE-dR>RbjKi6etNc z432t2)Gv5rvyM_G_L+2@8aT)@4XxMg<}t(3$Q+JR#`(_p;gmcQdxOzD;yCdI2VqHR z^4eEY=cdbqB%V&Rq~?e+R^LytoFst94|rc;9IZ&N8@7gF?{P()#;CPdJesW&%{M2) zJ!5WB~6lKZN*liMu64Im(DOlaen_5!ynLJ>umr|WMe&f8lql0Dg3G#ZxqQPpsW1bxC36RN^obl6w@_s-#a@K%)2FW`pS+woGUl8q9Wl9{?;8$OrrhX0dkI|j2~yV2oO7WXTCG`r`G-ryKEiw>yB>@{)&R}#6M za1Fp~o}#^90fbB>P3eMF!%QXZMjJi#vOSj8E#s;*(P{r3oN~h&pA`9)=71s%P{CBa z__R=+a5A7vEe6#+u8z|^7s(CBbF4)=jxVH$qH(PAA`SFk7@V}t(|7HyT z1xN!<5WQDF)msxFe@+A}kRlTfKFH6E7!`=Q0(hT12BZOS0m6uMRX=x5nc0|shL@w} z<(-e_nPY2ya_>?ta~qFIHrVJ7{JI5-Rj|tAI~c01No9;FUW$_V4^*USlS0(?>q=#4 zODoCMrAsKbjxB>DN{>{MLU@W_Zl9(;-nJe-K2nJ~-#UrD;QZEvd_FavY?LWXZv{Gy zxH(OwzelRK@SA8)ihD0!w{g;E&>z}sQ4oA1cHGF|a*jS$IAoO5Z@$)BfYfQyc__05 ze^WdcWxyQcuafwF@xeK#OgDX7QYpMkmIK90_$${igakFUc0AyB&y{lA}beM zu_gQj-_@!be>-O>Bk-!|rQB3_LL85NlMrUA1luxV7+=Z4`bLT!H?Q?T;SUX*O5+$w z<9k!7J^lJ95(zjG?dCQ5`QxdR+a7M*hVI&S8HYQt&ryQAq-E-mVe7)HsoK7Moh{A6 zk$v($+?F+98 z4EzQG8A1A{wPS!-bvBoQOR+03QYF}Pjz$-|ip728`SE}N@sWM9-w}Sww(KP=-h%F> zKG3)FXd+GR>W7uj-{4%$!RGt$+;d5HfCt#FW8^me+Axr(P=yJ5hdcTKetzY>ea*i8=l^r6~36TOO zXXS>zIgw3vK$|a{4^1u?w)&5+WB;X}|6an%(IfHS;_v&~r0Wj74E)8oBrC>~sVU*R zqQeODhXiAv)yt}B8o}lQ3s)g_)985rzTG)kc?YpY0o1vhgvaj{VM+~{9r8)>`7GWj zhj{h!%-q-F(X6G#CJrxw+q{_Dj@i}5Yv8^f2%{Y2WH#wYx_PE(0-Su$BSG*#33M5i zwtQo_zH=fUX4af>Rlu;ET*gmdvH0sljcJOh@s*cd)Y<>R2nWUNCY%Q44}d4MT6Uyx zWy%&=l=JQ9;i}=w-QMi4(2OByo_Gqs5ArxO`fN3>oF1&FS=#ghv)qoycAXExN>z_h zv#FCiJ6s1fhHxzOm#zbT(q~s{Un@YcPj+BFk}o8EA=g=;=zGwqx;**yyZ9-2m!4a4 z?xE(dTg@9Ae$*)GW{}2Y_I|ZQAWJw5x>70-BCbBTy)@KEW;I0K#6bMwX zaNj_bCzFQZ?C;I5G`#y$({#XSktc-6VMAlJcld0CSZjt#+QCiYZvj|Q>`~VV)x?<> zo?nU&k=tU#xd|hE@9=OG9@YoYx|mE#GA|svqUm=YeettxfV_Y=LDGb@=cdR*DoV05 zE1kh7Z;vY^JD;b&B`9au%3EwBY`cr-$bwir-7U|JcgwS;75+g}vOKXP)e^_M=+AH* z@LZvq2s;AOSH!?yC9e#|*X}=JyCutCAy)U(rACBEsXhXYo-86A{gmIKH1`ZyD7FH~ zK7#XJ`S_n=fInk0OSDDQLb?L%qBFymyN{$x+Sv;vss7@+cgbM2PdN^&$Z(h?spxtC zB(PGEotLv*y2IrKM>5R+9e`Wr58hl@-?NVIS0Sqz84t*%58{m#^)?4+g0B4Asr5#> z3<^u~!SiLt%X43qJ-}>Pi4PF#0oU2&2V2a0@NT2cz!^%-xOsA~Mp)=Z>fi?4NUKH< zkqDD-CJ0MXoyk87Id<9>1M3#?V@-%hA*5t?3H3ee9HI==0)TAf6eA}sevB`uace8> zDV>ea8l2M|=8kt0a`)|*6pTz&?`eEw3pjbpc*L3`85I%NAw-LC<@5l-C= ztWrO;Pv^a+OZQ7;h*(!iZY@I`web2`&ZJNgfO9@jt|B^34v{P?=`PK<`cbFraxBCI+aFpJ83;&Csk&7~tgp z?fU=zy%#wIBb+ww&s;GhILMG5kq0y-CIAB}e}{qtR+qXqGA7zoGV4d(n03XOd`K&Fe#Ff}BjiN0(H{xB9ke*x zgb`Q7Iv@ytnuu{}h}p8tdy(vc#e(7Nd5DIfi1Or^z~X~l8_Ijbs!5?aiG!Ct(XN*& zvxRpoX@VzSvDXYFK@3@Hg|NI!RP|Nc>zB=HeQ=o3dC9UaL z9KMj@c6f%JMkB6hzCc+yI8TkIp+SCD zr7&O?z9i=-wa0el&j)|4lI{odx)mEvxSrQqwwukDu7?-DiwEC}PX717UnpSQqL>Ku zO{b$N$K35eI@M^b%E;wfDcU$@6NUY>lFo%t&fCjQBM?BE$Bh({#NiL8CA7(+VU)|DC)Bl8Sy?np}>Z8dCzOky*QC)_+OAvddjP~0^b^_V)*B{I;&4L0#G zYsKNzj!s3poUq)6&kC1ib-%R*Dx5XXNx9A)eqhda0CA|!gb@tE!Aj-wJ|iG4^uW^RCStEgXps6?SD)MDbEij%}zTP5FksH%RNH^EdSOib8B_9V_@t!OJG*_v;EdXR7Bl~8B8OQp(=HH7T)#;R2b|UW%ILWYFgA2rAUBMITpJ4q5sJrJ}PVyw;{pA^G zCjQ8A#K&`ROZHi@kB-+ZsqQmKaPb%_vD(-axh_j9DX{~&(m3W#7_6m6F5l&&`kYMV zdld!*1S%4W{<>nCBu9Gx3Pvq+$%s@1?j1__&vIp^kYp5Z`_TMd;a1nNH%25YPBbdy zAn1hxkAOC^i=aA+b66ZJKY2Szv8RB%Yx^2kB+Mm!0tohXl>_7T5 zJ}2vBI!{TU?4l@)Z9I->(JoG~&nE#oKF4DVmZHr}g8)?NEcm5K4UAYoN8LKFF6(N+ z(j?EfSh@wVS)!HT4i4k!CVv)#rAS?2*!}(md@lIs1L@+GL5DrK*QH<8z~%I}mc+1G zBZ;UDgVygP=8Y}*n#1_-IloWWf$b>^wsxlb$lEgha14zMe|wQ!|LUCdfjdhqKky@v zJ~LM_{#I`Z9^Q3hr>&_dutdzyi1US98X2wv+0zcjz-4qyxR9=CQk|Y(GH)NZh@3MC z`h0Vw^vq87ZiPC|SmS?A8)FdSalrybUts3C{eH#q1Ua6xh{qHDu2do215(J6vS|-F z=PrDz1R3+&L{)^f|2ncQiVR4zxr=BcLP-BbSGa;&r;zlG8-k)7`MIlSC%YD0(nWz4 zRn@FMYgY0;_SC{cH`gI^y!h*def`%8hf-Ro7=I#E??N1@x9pUmGU&*_GnJjd(t82@_+ zFBI^@p9wWpY==$C!pQ9+kw~Nh!6gp zgId+j-l;mmwxhP-S)&HK6@{yhWEM+Rue}qx4d@uT5t?AoAm>~x1V#yeQR!D2 zq{ELa9$VMBfh;NllR(}duklMsDXsMjd=Q|$$)Ihog)8e9ai$o{wgKY&oB)3HgwZ>6x z5yPHr_KrxQ1VQoo=x&*S?B82j5YlDlC9Q{=>gpqG*P-qAut0Zt0t{tp)#t5WRmn1% z=|kS@giBF_%@L}5)uCdY?E`g=xeBtouHiAo6N5OVVd5=h6{D`f_)KI}Qc#CF+GWJc1h zO>NDQ(i7LnvA~l}E6R>!fN02BGzwzouoddDj&;8-3?uaiu+Y`LImiTC*FA?*iy10B z)w6E0Nn1MNr79D5_>n_XPm@4Y-YwshN&8LB*)6K8^h1v>>=B3#FW@!xq(#(v$vjs~>+x0bT)Lvc z_Z8MShN-+S0aLPL&$T6|%ra{)%8L;LKKiqhM4g!gvI8O+ttb`*{VT8Xclkxw<;&Vz z28$s@zkXaDM)+l{eJ1lnXY(4dZGD8YH&?xDQ>(FM>~$Y~m4!$l}yqk#+VP-*kEPQ6V;sG z;~?&NcU#d#*7?4)2*+SF6wbDkZkfKJLuK5=M-scv@x2C;UKjJXAlL6s~BW zH691@C4`6zGO{#4pgHcTa=_5>o+IWg$RFSPJYbM@1I@EiwATUr=LxrvyFZe*H@u-F z6i_H=Cv2HWjwz})+t2EWFm#AnS^seE`M`g9em@yb`>6jE(m`8xHB-R5s%VeOY^m_mLEPW?RUG;_iO?|OF?*`S<3{7+z2R#A{r$Kr7K+7?mt*5 zA~A&a!pHH~8@lg1b^t^%yX#G(cKZJ`I2A}>J^n5TjNh25% z9_k?PHQV|3c?|iqdIR2yDFypVf)V3|0)3~-I6nUf=R;5Wei`}i157I+8#-U~#RG+o zH07MEA@?!hi{LHn&kyx{J)U03#4njwj!r`AhnuudTPH&W^tz$(3)gESS-4~~ zqaVH7cW51U9=Tk(II&HbKJA3D_|yn0OrqV(p@e9VuQwO;{R_udMLh5YU!S$$-uxyB zY8F85x2*E9r~U@dASs`uPI2nL#xl>tE#!PvN#&#p?s)UU?na)`NUSQ@NHUK8gvIkW zMvU;sCPjuGg1#c`g7SrQCxVkxgWQ(6PArbIl@#)h=_OESmTK$fE01)&0wv!dc4#w2 z-Z1BiE@4)Oj+N?~R^_eqI?}YSKy1??XIueL-GB04m>OK6k7FTkfq%xvylSgE5CMcR z78rm};(FD#F(iN36OWPgVY&{9@A}2wdP3;alX~%`>6x*BJ$-^~}laLP*|qSeq<7B;6RYMogw3+c)l!|y$nnVopRNhMUu+)LNzKa zg-DDfI;q8~wY9a*Cf8b5#R}KkbA4Z52L9uzahU)LnRG_hrzxM~4X3H=_o>#W&YNB3 z>2Ia|yg|wnR{;CIn#&LS>R&_sm^iZ^78^A*>u`S~{`^ig)Sa=u$I(N|J8|@>Ac#2+ zkGb*4VC2ki!fXu+P)vTgOI5fV#r51JMvK`_Mw%G&JjD4K>k%(*KT2oah{5U*jEPe* z8fCu+ZiTEJTerg&pe+lXj4Hu_qkX^C(kUiCum?Bhi~s;^pj^+ibTu|+*=tFw2{4!* z^Qg+OuToRdiHt8zx<6`x`dN@i31nnlY^5;K34cwgi&ARI>7+1mwBg1jLtt7lQ?q2-hx(0OH8;axA< zDw*cbyW*W8N=c&>=8`zPlVpa5iy|H>K{~GvrGN^~XGK3{wxoUV)pHwR>hV;EUhcUv zZ84kVtpah`_8g>il$h~a5=2pr(XGKW4ji>`Xh9t4dWzjZ%nE4#@)YCtAk;S;!6}IO zzATyKl?_Q!Jq$F4EH-YM2{P?Gq0|qDAgHdO+DS{cOJzu|anO{Nlvi#rOq34EepjA_ zAi!gcBBuS4VP_7=cF&GZnxi_0)_765A%k4Xz2jn%sg4e}mWFJ~;v`V)HQM6dnihOU@SL6qbh_ycg?2PlVo*_^vZ0gHgBbkxCa5f4w#-;q z@NbFdVzCZ4i>TiPt27-ijHP^qer=l6L<5#>QtmE<`(m+A%<6S|aH^74V0wHeRS(@Q zB11bt;aFR~tq-kKBx2MHjI$NJnm>rHdX(f5s)y66o6l4OZ;UiUIJwm+P0Y?}L#%H# zL5_Hi6m`1G1&vmyB~q<&`r-FhqIGz(ro!2GUH84E&=1O1U#1Q1>Sfj~J$#g?0qw?lgUZ6Va(*9J(W2)N(&q(;#t+|)4`aXh-Cov2u=RBY^Y-*SKCONc<|v>;+L zF5}E4K8fm!*M}^pK5=%1-I;2NkH{HA35*c69eedS!qCHLS}ApopIz=Ze_a_D#R>P!n}k>; z@K)v`TtD=l=&gQsTveylhXo92s5ZL$E)D-$TE1%lS)^1mw==k-ZM4&5yV~$AF(p$U ze5tw^Bz$?GHY1bm>L`kJFALo&3{kg_Bn0&3KtKzeft3|}R0nm0t;QQ*N6znSE#k8t za159OyaR319C39h^=)EW&3|8BuGwfzNsYSKeq<}ozK%*#nfFp|N&uiSKlLLtj|+cX z&XqrL`7W8L&l{|nf4!Na%lzbmi^_`HsJbosw zmoHF#X#%x&AXxt^zNYQ5)W>@JNH&DzT)cm&NmWv>C<^~>)Cj;Ew+s$k&$t=AoV(p} z?qC#%$b2-gz=~fom_%B$Lcrgbt91mmqL8zfTiq~U8LYiRqjgi-L60D)49HP4#;J6e z=pH1r6_86{IMnCnOA(BT{H}lob0z+vsMFV@`)b3)7AA|WBBc~ysWOI6YY|n3F(pqs zWKio|1Bn)b9FkuwU&g_n`z9uAUN1~VSr2-FjnOfep1nh^4aVCu!u>Jkbqsu(Dj zvYysNUq2c~n9_mpzd7$dthNq!n|-tg2Y$wzfkb0QML{A1h>jhlqRJ9P<)wO-ZQ_3W-Xoi{CzcY=PiY5SO4|I>AE-<7ta?f7A zbHE4ag95@JpHC6^0tj#EViVg58E=%k{RE{rv;S&5&=IOSVFcU4-V*!fpEO}z?}=u& z*5G{(%6lA{8Fhr@+)e~S5snKUr*Zz2Zd|tjdIgoiuZ{8A1&aB|OjfiWRqu9r(;v;b zq{?&J54R&nTJkD8L*krL6rWDY%xk9>^Ev>aH(>%@i6xu!a%_*)+jwgNuLdTaF}q_o zqCwgq-%j-K@;BYnzu)j}hGL%jc8GZ0vRw98+<)6LT6MD!6vz*yiX|vcbB;5cEl`n+ zoh@??lPl>*D<+Mokfrh_4Aw!JZ$wm-5+TeDIUyC;WxGPsLz8SA$-zXt6LjdfHqHiM zvPwm4|D5%*pQiN%g2=1RcSm?XL-?Mm%%hYPcoaH(@;Jp6;iN)#J6LT80_Q4~l99I# ze0(Ffx!;03l2kdB&F1rrr+zlqPr4rhZ{Oq<^pb*EhVMe}iG5=3&{Em6&d(M>O5pkY zBM-Q6hV-+0wv`M!@0ROP={u{ymM=gO>=uM`FQtf#J;oG-qR-nQICnm6_iw_#xpE$2zkjDppUg%Nspf1>?(>rc*Sw$qi*QNk?Z-Klei2Px8A5DQzQ8 z!$ebrqyNC36evm%>5UOulRA*^CraFpW*RnX6rn0pR8611LQ<^BeEPHXFFg3hPe5S} z5?w;e7v)jPSh1(CFF$V386&ioD{~N;&a5VN)CePh%B+5xR1AdX@9*_|rr9iFU8Ueu5JN{0tQ0bxyHI}?-d)?!>x~Y5l z_!=l68mB=@RXKO)0FeTfuFXQxGshuBy|~5DbFk3PCuMAJT(W1K%)2>ru?!pnUn@-0s;UFpeCTg^P0_fGz_s!46X`4$bxep`(ZB2lyi;bv_;U zc6evdbI35PkZQ`M!%x7SJLT%(3^1ib-K-4JwA5D0(`+%7m-kRU#*ScBs0}jw=?g*6 z6t)hP8ZoDVG_YB*PMAHSqp&+-Z+nW1aC#xaYVk|;C0l2;ZN-2hJk|QBC@!^5knb=U zx&-h1H(F!L8e@B)l?N6$89>v;I0s4@%9Ry*)lxAtqC&q6#?_5hAoN_uU6z5m^5^7! z*^fWpM%s$kCgnu;qb9*c!L>v8P9xYROtLUVw+fyWE-e~g53X#1^=tHRitcF)pHiZy z_3gU~E?@|&41h?Kd)mh;hMc}YGGO=O>hejY;>H~Y{`|b^HGU1`4B$?@NlEI=-KFU5 zCyru?ry@vt;D=sMFG+ryQeB8F3n;0$$c$DH^1hY_Hf|qL;(JKEE_mO5-_)PMfWuM& zv7jl={Iug(S;Zzn9`lcp%&%o)HT(VNQ4Qfm*jAoKHl+NKC%u8&1ohDby-`_zH+F*> zhqsR(AkuxdeAR(_0DTM3j_;;9uhqG0;4>=6_Ml^{$>;&s8`Ky+)Jv?)%mQDLm23B2V_IFM$j^~XUC5Ia1} zVt0_{EcObeVagzf6^C1{SUgQ+IVF}K>&6OO8u&9dVXL<>|7N3qdaz$}c4^m9UEYot z<;XhD2NztH01&p$?ovqn*bz3B@87~tI!WRg)n~!xm}!{FW^l4d)D|Mge)L+mvsu0r zi;mR&ElMn=StYrK7OX8KTVa9An=7ybU=Zz8d!waC0qKcO$f4>Ib7qamn9CQ-g4D^` zjQ^Wkbb1KNUB@!!dv?aN4~BKZ&4X3QM+=4*+f8|34Pa5!cuAHh+^lBeh#tqf*s$D4 z9NTmljFu}VtYb2cacRk1G|VLaIxGCK-)**J+6d2JFy{0YYz!))Yzf=C8b```Q7 zYxx8XF+c`+ZKf!GBs!g1U8(tOPkuaywn!pbdTT9A07p1R_*BWYWFN5>c^i_vGj>No zt5!ZCOYE0*!vH>J{mS6v_{*u4FN{(**_sIN7T#{XUKVBU&TocrW%rGThM)NvmcwDh zs(RJh@j~r^idBcLTO&r4h+SEOof{niO;Y!yYXFMbRp%H{6=|br4Qu3c`y2KakUcU& zMyWhzyffw1eGapxuWDrMslEE7hVaHg;YS(VM&9=2p=E*CNi$0n*-1x`&q=l>UoGwI z=S5i3Oy(KDn{9N<)=4t!bdM{UNwM^2{?Bi(5HGiz7JQqgED!?I(ok$?$o^{sao&|f zSHSn+mdGm%=Zn;7=zy`I8(StE{O%f5qu83=BGe~-)9d?vxkI?9okHIf8tAlAeNP=K z${cZ5Am=_*{}bH?xxI+ua{|*OwU-tDnR@aJdySQJ^c+3pCdh~PLPJ+kTcy>5(&g%` zMuC8NlNx8WolSXJWwYCkOjLGqW8>uGE1<1Ot8!JTeNEB2y3BI+;_Ms;xVXn%S>@ql zod||W>gD63{PueI_*!&cah`!!p=~u8;`2-@Q#=I8^r&WoiBoX27;_kKL9R&Hx0L=a zf_{5Ak?&h!c{aoz<<^n9IGR)BUJK>*8=gBQOU#v% z%J<$12``P{{PxbV2S9wtUk)yr#EK8vHqu_zLusmmq_!j^*B8fbN+OoBY`!d7c$>+R zG5;Y~#5U5J1Ous-F5b$P17k^i4~Vn?^Mqlia-0|YBHHM}-tBOQ4m9)i2}-kgnR3G=*^z#lj-A^nBKzGR3kBIH(xsFYGbidk;+ z{sS0DNab{!$SmOzLp-((Hb9o_&Oz~`0jm;5%U_>@){?-b&ez4}8&3xa8$c~zEOrv! zG}jT*bS>m0&WFtsF&$g0{)e4bquH(i&7N8DQs!??@=9(|J2T74vbJAMOLR*a+*(fQ z0$l3W1m#ahmXoI&Kwb3v{6{EuJP>)Ijp4hS_Jn7Ncgt89}emJEG=}h@) zYD%yl+!pYaVp$D_O(jH>_<+)o#(Q0@(643ODXU7XXK>{%;PVEyet==oT1ZpH{N`4c zZxmU|yx0}wuQSg>tmfmddmJ)NfPv7;@#0_~8Mx4$p!n*D@3#*k%+EAan zSt6>vI7N_8+A85TTd6fe+8{3ERfh2f^$WV;uNDl6y>uSG?zl+M65#tK3_US2l<5q^ z%v}N|0k1hGez7SfBj5druF2s0;~aClvTNv^GY-0tYr&2eHEeI{_e&XY4}t(M(7tv& zVjB|1p$EfWTVcspBtF3@29P$CK@Zq5C(Ho;An*(EuPIqgcNFq9NeiJ7u|pfeasaN> z@~~&y^C_={eL^}HJ)n&6JbeCjDmX?tRWxlLrlDIhaqb>3JJgU%`;<2nloC$Cz{@Xz zAoq?qy}kVdIepo`F7q=r!lFp#0&Nso`UY}FN5cC9hk-Vn<>bxTpQr1G`2<`I@$UwM zj-8)6I>4`~NC#+orM=T3mX6++@_2B#u5IWM&)%O@4H}avvT-%hVL6EVodL$X+aAgaqW1HP(7h70e;~4xcKPd z-T*_6-;v}VSd#~^MdO5Fdg*D2x@~-qxO_!a6aK;l!K+g~cr>WqusPVh09EcW?cUlV zXZU;{B0Z{o4RE8EP|$vyCIfv*qg+#1HP`G7OU@o3@jYEV7UgJjCRo zQ@SVrA)=fJfNk6U`Be(v5GX!v`$qKxW%KY? zO1w}hTcW_*5L$6GHW$TXtCO4sr-F@!S6o^`b$wi`9|I2wL@xfZCtst5fPQ1t20gsGRWXjF_E}*U33>N0wDIsP;~@ zI28B0P{qLrTvB?(U$m2wWt2ZJDLVD8ZiQOiaBR_`*gHo`{RWa)AxcP_AW{vo0o`Hd zN`00bZc6D1PS*X-@UDEoMF_ zb3ssuYVOO{t_8W6NcrCfF)OXF#3Sse;P8_YHD+YY`g(3s5tO9X`-W{X)m<@}13EF8 znGCvyG9Pjsz?WnA1H7U%R|dOLG+=CH&S)`3eYPwG5F;j9?^^1p59`NH?*{OGG~RIiH55)Z zw)oZN_`#Jo_A(ztbKD87I^_jf32=n-Nd2AZ6}_)-HucKjso$I(S`V=;YyU07^#rMs za8iKZ1W-`>gyOVi$C>SMzeyeQ&^Xi;%VO@+S$my!^xKZ3Dcf|)v!vFgjQe~&>Nv}& z_zv7BH|F)`^Pj%>Ki%(t&xHQziz(A8G^s$p^h-Cvfjh}?m_U1h@9020VJZZmTNxH7 zI3C&ax8}MstY;9Mf8YAgo&R%@W)4?I8gTf3)+2>L9)tTj*T@beenH0icbkpq00#qL z`M>gG%Bx@EqiFn7940(7mKh=JGbumnljO9+Da$3Oq(nuef+Jq9WMsc)-D6Rd0o|bC zMSK2;I|WF~iDBfG3UPqfQxloi_iREM!Eh3(UTb9&<&i6hcx{SD!&H&3mBiwuIhDJNp-`ShMPH;LCHTi+ zf{s)=cu2&WQv+!7w!yTF{vA2YK7|)^ayaR3U0;_NL!Y5sxYcxz z?;KA)$is`5m6xAh!nz@VV5dow8ZkbJfnMnl$4akRQR85}PKXd`gDPtr2|_YyCjjCi8ir0!~$RC%|p%&%Tc}dNmsDz z%ttzNNW2mfxQ$nWY|_kP=uDrO<8`y*JW%Vr_gU}e*|G#JzP<&?s*9e_TjHxY9ccM0 z;&ddON~$x}{B!=v&i|J)|0}T)lA2=gznCdah`?Jt;(sM4BoLjA4-t^8{^qo92DA6HMFAoIQk!SNI)f2Xh=*^WV?Eri`Hro4{v+^zjvfzV4) z<_v;q2_x?0akD`c>csIBSR+NcnZf)$zbVAC`O%^#M|ITUtX{zN3%d8SJIif@mOEtO?@W9Wm<+%8;_W)iZ8 zcp-(#cb43Fs4bAzR9RrrMy+F=O12EaagQn2MjVUtc#4ac z){+%W=!&}#8X^E=z9O~YGOX99ylU-@8+mx{@*qWaVnUqNak)U7oRRiXZ);oLZ{!ez_SfMQHDH z`ke1j=LCRn zRjsX_viV9Zi0%>JKeY{&+P%WTK{l_N2+ffp?uDN10w#+lsnh ziC<*BEs%gxT8+56xCUnvX`0IiXH9*!W2$J4L>N$Lu`m&_S8QucidFz8+N0Z|-J?$g z{ftBi{hUhgrHCFk(Bb#fCM}p`2TGel5&1?P=|1LRf#T<<$PZ*Ez7L9>Yc6sF7|Kqq zbkacZi=i4nUZ%c%LQsK0b2gId3{j9A*1&FqabO&k%sGDQM7}a!Q)N*f{^}H#J$zJb z3r8S^wR)-z95$dga9UJ8E!2uvut)iJE&OQx4Q{L?BdDdV() zuW#48>H8NWFP}37&1cfS<%2~qDuA;;Wesp*Hy_>oH%s>uGReh6ew+?ln;CT_4(ktV ztFqC=HW<71B1#hM;DRZ=VmfEE=~rbXVbLYCdJV9pJD=h#Fd0LaQkc3n4csQQI8K;l zxcdcBR-17kxtf+3X_LgQ?C*Vn3L9)wMR&yNQJaa5NG_db8Q+g_7hwzCrervqiwB@; zs_qzYZPl=LuS|oBnw7P|+{ECV365UNlKcC6Kg$OVdPX#23cn>e`}&?Aaff2b(s@hf zldbe*lHYkv{`#ambY~`|Vuyjd<7p1Zq0}ug+smFX%ZLHPw zZ+oO}NaOf!_zB!cAQ1H}ur31#9{of36a?qU>Imkji^Cp4FsVu4KmBe@owLNJ*`X*y zc%R@PA9V)dw;KwDNH!$(HNNxF6PFL--Ra?gJHLlf#TKB%_QAkLP&Gc8?sL+2;dRC^ zDyKfibrnA$uZfp(v}f_)hd}_0dU-srU15=kb^xOp@G>n)1q?LwaS~>EDLtDAP*KUQHpLuFyV&`sml8t zP_+r)LfOfQ zxEhNMN6M_*2xq6ag*J%Z|K+{+pXcL$L$ZIo_nI9*u?fF}BmZ+rhk#uGhVaiNXMx_i z+!s1)0tfEyA`<}PYPb=bBcqAzK>zhODPO8c+!s&b1{8SPkAev_NaR5T{#fKg1hSxE za{)wV^pPlQbLEA>ciUOQOh*Kyv%GSv_sz zkwJ1D{gV(cHEYS(E%Hy;tmej#)~M*MLhPaBEAZtyFVEl9$}SRNu+xc(Dl2r!3V?j( zQNL_-I$$Iqq_`wj4I^^~WbKzC9%)C*2b7w(@U~CFejGW*svBH_O9KBelH&okSll&@ zO=9fy#U9uSt5UajHWff&b6OC`Rh=|=mdpum|6rbNC=p9egAF@O@aCyj-cOumLy_S>YSYFm4Z5c0r9Zc({9CmGL#0MINtQU-f>FzB z{jKZzt;X%wc1GHTt*g0V0pMcYuIt*X+s*gdXPb38hZoVHlE`B=%8S1Rk?=kj`ia%U z@0bAXmKcB-v)ckRBQUySk{9eNKqZm8`z+x5V@MIp%sqI(mN<*;jwc_+MQS87Al{+j zS8toS|LH(MXnI$OS;RHHPMOL#cwk9EXnf~Dk#nXK3Ka2PjkL-{zx^Fn?GC8L&_8^s4u5XL-M%s6hwbvSV_9!b z_0AY|OZ5hhKBao+j3TDK$Bo9OzH1D~i@d80OoL0%w^fN`uyJE38-WhY_{pGtxg}~Z zN;Fns4dq#RRcT>ISZKNYw3`SUBPJcumt9iI@*51ilh`Nz8GfmdpJ8hJBfQ+x-CTto z4Fu4q22$mzsvZpwYV|%ZTy=t>uz+?-nz~w9US$)D_((I$(5haLmarrtvD9{0I=202 z63?8QK%Lw=P}t8Ydhg3FbLQ0_F)LtFikzi!{7S3!^u;zmYss@({B?))>&Cg==5@R4=Qt!5K}R&`3RU#^qXMl%-V*kGu*9>x!lWsgY>uShuxjz=mUjdTyaM z4|a~&Jf`--O^+OmZi8$L0aA?x*YR9JSyW~GZo^=3Keu{uT~Q?$_mbr?8$K^!MKL|F z8$VbRXC1OYng2JqvGaGwl7RWCQ;LcNl-=Co0&t$6D(A8-9Q9erZ;T8{Zi|3f`avFV z%cx6kUW*7j&qt^rEvrRHBWp{sz6+#oNCJJWlzIrg2_V>xiGbBxF}NzfO@7G-`47o{aj@!Pa^?U&d7fhrn^vUM*G#!?kkA`b&mJ zk`vLTP&!P&S`Qj8GS%xd&|Fjds&LIHaWWZxW^%!l#PyAe%mS)L+E?aed9Cg0=NO0C zNsaYQw^9$)p*6DK2&@J2QcNp{TwDs znjVjcHf#Q3R~K)3beYd(YudJrRQRx9-ecZloVYsH{9&I> zi44p!xSK)xI!NHj{-(`3eV-)0o@A^CtOd-4L|(YzB3nn;%6?o#Hxw*Lg58bX;VEv) zT?Yo5G+KzUtR(Z^W@aBBh_W|HCiZ7V^hUP@>r2)LOlAbWwee_i*de%rXT0nZTp% zXr=JSPC=;$v(64k>H$n-4eu5OGf(hnINjrJTU_(UeHdPU872Vm1yA@p0 zxEY*cnO(yb>RhoOXGjzru^qQ0YPHP9)?iDYQ570utYz9XR0cKCL#A8hUvhRMtPn4s z=mU0|4ht_gY~Jw)Tz(>WZ!V?q7-HZQiAi#3apql4_&>sk$g){gajX$PvXrPCYEGcu zupMm&qwiB)Vh;yoGt<#&CiUzkyg<`Q`Zeft^nS?zD#6an%@Rm8cHSu8ms2%SpR!Ay zvXk*=O9>0Ym(B9Fp48bCypERIBNT{@Kmj<;YMg$kn-svKq-TR&9%X`b4q|TuqwSE? z`5-@No%ZJGN2mvA>u(WJmnXSz>;``obDkI-2C1b)yVB-urhr|#xOaw&$$!KB#e2CM z*v51N9wFx70eUFDi#=GrZy&uEp`xcTcwK2ZXqN{+flZWepjQyaISa~X>lf$x|A|O^_s$cthu7%HS&=kWjA&o9XKs?suZ}v61_n?GG?0`EwYrOD1B*opCWr@H)G;P zm$|oA?wpYd1|Kw~gSTNpKibiCZ;r0*MM$8n3DB~l=;J^^3(74Xi zGd5IRA61cgMH7f9^ZDbQPBMrwk@?;C=jsHzt6n=7Z`Jrk8|_?+uB5GJ2)D!X(XcEd zce`eB3y?wq2zksNseLPQoz31L$f<_8_4uu0j=NP9njL zIa1^3;zowwY_L}n46WA6pasW8m~MNYFJf7?EZu%aq3EtnIA5K%nomt+z1;pe=lnKZ zs}d5~PQYcxKe0nF42Fns#C60?;6TUJQjA&Dh#rEOMTOA7c0>o|C2U;=`a-Nm$MyXO)%_0wQSAGay6}^Y>S8+mBWK^ zAxan-@|mf1#eSC^lV6i6EGxV@ym4Aq8}*?cWqWWr?_ZCuCri2};ho%A3?HZ5De=&W~l`BP=L=E#0LS&)S{*#KP5h1;B4CmvNEMDg%Uk*c6Hqbd49` z(A)ifK>OfbILn(@SX40PJ*6!Ts#POY$_*2b|8;1v;M3?e>by1g`5pp;vY5|4AQ%4} zKQ^}oGt5e}BshR=R%K+6%>V`2FIyuuxD)b00k@!QP&MIn)TIEg4+5;|%T`*8L_a1)=!dB_;;&So`eN5} ze@W$A+A#ugA{Y_=$q5>OtUN9=k`G0mbV^ZB!{M7Y_m8+0G;mBLC3OEFGyxGLw_)ai zk=uOT5)?t$GW1eN-yff{i1!-It|Sh5ka35#=J#6}x9+~5Zx2ugVB-)6%-BXq%EUv= z_y!a`Hh$b8h)tDsj>(4gRS7E+)^^6TM?(_$;~667&)Tf@)R?Y-Z4dYisjaeyZ@ePg zs$#4BA(zQ+H|mU7ca4SQ*kb2nBm1@`)bmbT3ux4fsJ#ym^2vYRdOK)=#5`oaTJwD4 zWUbnE)4E%^XHJ&Oak>sJLD7X7Fdn%&>A|0<%hfM#at~Lyd5JP<6{MEylC(YEK@fCe zP)ulfeo8^)K?}^xKG3I$4z;csXCp zC#b4U>8Ff+k79H_qFx3U?r3&xHp5{b#stvsglzv8E&4s$~YEI2L-6qUq2`nprWT z&zUtqIy98U;{@>}=Tz{-}qTCLF?nVn-c&w(j&RU&=pxgx2p%r z%!fy;93OX7DxG?$k<2ta8^<&~n?&UF8z8B%BNW?GDy?x~Avt2-Ku+giBcXA$k`yPq zbGr+_Ev>Qp92le;BPgY@|4bHB0+&#^pcSE6%_MWGCyubem0ll^sr^bj!kBL4z9f@BQ}!bC*TOx z@ifaon@s9X1YNcwSylI~!E;RL#I{{G{%E6}Eo5^6TP(K0NWm1d#Y#h~(-c^dzDl%g z(Yi#l$o{L-rWEd2a`hqKa52IqTD)*XaXG12lhLj;MOS6fvwKNtwr(=rj429H3a8u9 zVrorq^IzY;R$IPIHG)S+(#`lsFvCudpx2nKLPvqvsA?t4zuL)T)twKYC;z-hkU5U9 z7Ru||p-kWKg?pvSkxWj~aXM{GIFUO2fvY6L;L<*z=$}O498Zne7e&!zP;1dVG=raq z)6c9E40)ysaxJ}+oe8ObBKSQ4hh*cXJz@d5@ zdMBu)S@=d&VrqJ`_n@J)+lSBqphJ!{Zyc6Pkx*q0`C9px6Xy2-heYN%l@5^D?ezob`b)T#+WXk=gS-r1gwD??rIk za@RulR-v@J`klYE2-zq-+oEKkbZM8ZliVTL#e^Th>5%XW$?6SqRT-izTiSA-c%>F< zZWW3Zmu6|tIN)+z?MQI{8l(6%iZD4_!tHu`U)_M+J!|$Zw~ll^t@cxLXGr^>ef^)^ z`#(!}omX4<=-0k}{tg5YL&gL$rBlHIb4U1)0MRPjZaAW7A0bFY=M;tU#msRLsgnuc$mmsw>>@T=quLz5_G&y?mX7|C| zt~b28?XIeKr@MXM@O$9BD-CfX4L4x+n?jK3#BR!7w-onS$pw~?dgidEj5SFe)GcYH z02}l6gqo&axeco1OG%_9hWLod?r35HQNwVFt$S*DH5uzuzf#n%Ty1!8Eun{sy#l5F?qS6qH2eJ7h5I7MSX^t4DDA%WL}Hn zd{y|*kRP35xrgP4%SFkVP1@+^#bcZtNi&U@?|$d0{e1o8MYtHvxMEv%B7xePUusge zN)}f&G>=86D{z>vLgc6EuKPmk4OV@o63)EoHci?rn#iNc?W5>4yBybpFXE{v_k*Xj#CFTj*DFu{QA^JYp>$mMXh4G|JJIX+ z1+q@NKHk__I|R($8v#o}B$C_@G8RgM+nv~!=kNYmoLH;eLiSeoSb5LPeZygy1{bk1 z@9YgS9f59oKR<&NNF_WugC%eV8h%_X-y++RM2pVZ?tF(Ev1 zQu{Lrm^m*jT)mXhqfFFKb^!*Sx1R82-Y=5#s^C1bc?mkVqVL9)PYb=j_nTu)zQg!n zdE0&CTN|erO3Mk*MEw1i4cynAf-BI2P^WN-YTL#vbf)*`VzaRbYedC%7sEapJHkpn zOhjrAD$e>m$m1%(TQb4hukNa_@`}Jw0pL;ND|X*K%^O}k!LX{%A8vr!-(GQa@r`q) z>zxhzymm0us7H(Zg)Hpp?io_=1Jt4KQ2It^GE1-DtbY&V{Lv(B;4UdD?sQ*EMA-5v zGPyu6R_V0K-9no+8*%J!KROcprw;sQxBt&N5X36Lzxf5>B)^9G|AKHptO!P=e}j6Y z(J!5=PdX+9fEffwHm8PP6_L`9M5J73G@}^+&J3-J&|{f!gq`73V6?K#utyxqQ^&(J#%OUDk-ZoizS#qntcY>gNDeGn1(elt%*w?@D* z3V4NK1#HzOl&viJ)Wx+`HjbbUMhc_BUAM7BHZmPa12SW{;L9}d%%ujeA%jOfteSo= zv9|7rj}8#O;y#Du*^1SL@xU=oPuK0*6{VOZ8g8T$f8?K5U$eGIa-hLQS1lNo@Zp4Y zOhZP{P1>e7HcmTPK9tQij+?He8Xl=P=uk4UH`4RgZjX=gp%WG!R=o?}tP&GCN!d%6 z(9jkN0G_EPsBk^qv38INAX}nnL5`5XEeyDSz!F~$!M!jCxt41<_#gpr@FErz(5Ga) zik1>-Z1jF#!A(!Y=N>%UuS`>9ZYm`uiL>@O7>s^zVSF{EM0{%2Q+9+8w+q>E)9~4>MuL((K5?SORU4u7X+fQ3`k#n_c*gw)3G2LdN;BS=j+3jIGF#JWiGVm{?SUr;b zqYK1P;|KiPobbP5|No=DhDmFO+^?*F_S*LqKhQE7Fkx2U&p80vxe6;0UJ!ZQR9yJK2<9*poW#>wUYll5S=LDlcZU7 zH@>&&?AJSLE-pS+tOO4if}6<>u|mqRs{=!46d-!=kDI-*x=d5jptr zs>AbBDMZ&+Ud-`FvBb#S_JjE`jJ9%HOAA|In;i{(NNDgSj-tLSb}nzZf-!eI;4xrBId`3Ng{g&kA%buST9u+uFxKVv_DV^2?+uO zVaUkR+8vWKvnPWmz3=_^Ub;13eOue!r7Kx?Q*02beAPOXb2{3q6t!8a#YddVFy*(; z1`-gn?+*_X8E;o_PrtW5abKpl{(LLlog|tN>40AhAFJsPJ7tKf$HNA&4ERuplnKyi z+TbI*jNbJUafCxp{4T)8Hf^cE;9a4qWrr9IGkG$^=L?J*yAoU7R2gu=Pud(deF>(? zd)QTeZoJ^z?>E2Y)a0YDqlZ{W2fZc5?~bkwpQ*Ka&h+o0i_bEBf+c(|)$of7wSQ3v zutztBZic#2gWXOzb_WGGWzlRS6K*fx0OR|G<6A!0hvp%Tkz?JqI_bnvf}7!itVs5QZo1_8MnWk{k&$*#Dl?HZY5 zwnpsbbY(V7)Eeq6mLD2c3id1Z7lufXe4&O)N0YH_4i0TR4<02W zG1XXB)^=C2NW1{i_acdr?|;aeNmEQ7B72KrI@Tep zz?9z~%&9Y|Nk6!b2#SsA{ECdx%+6bJ-7Qv1jk(-ll<#xRg z5*oJi>_YZtxmc`+?^$$2e{4wMg4@ZDRoj>hJ#Pi?nRleWns=1IfZO+L)*HTd6dA^M zP#FeZDcA?Zc_<7EduR+32YR50v+$JeJMt9lV|*0vPIJXVA4zKF_RVwUA|9F72N4$- zGv+28dfN4En3nI;_uU7Edx)iqI|vQyGiEsqzEoBY#;Zg2ZF9hK=?zG8$w>{2HJ~)b zrVXyDJG8|1?5pC|?(+2|M_G8ne$L#skJvwd;-CPqwg_~xgw2cLclBO}pq>u={sCKk;@6gkD4jLTmUJJo5EYO^G@NF}V5)3l5>cR-|p zk>!o+=k?wPFp#mQ!n4kE?Z?FVbW@m{C8c+$y~2$s-okZoqgA*j*BAKxQe`5HiUMvE zy^aSs$Y9E4CS9L0i6f=QdXl*}Od0E3CAsUsHJkObvi~%x?l#p_R`#XdW{klxV%|bv zE~G};X3rHcp5sG}s8kz{7E2)(4k52oOJros5x(&q(kb;>&e)gd+b8Myn?jX5$2uQi~`0?~N+$QlwzkRB?6aph^akZdN$d zGnU|r`B1{cPaSnD(aX3pd*@My@sPVd6=R7cyTKipmqfLh4Wb;fXdy24^M%IH5!` z?MkoC0@GJ5fBJ4k`}!Dx|AgxBFb=zEJjqQUoK)Pfa-@qgXx_tBSxWzP@Bw*gV-jH* zl6JQgZl(fj>W=cHUR*kV)x6SEA28H=_9Il-Hh=2sB9hXwQ_=kNY&aRZl#jW3 z5p#1EE_OOwVCnQRFd8%S>Juypnt2FUagLtqWl#VEQ_)ck`xJZ3L&T+Hzb!1%CJmwV zu1uX3;B=G{o4N{5L7QS3Px$iwHG^gcIm~^TC=MXtIGC)PA4%Df&?39y09y+PkpXEB zittaMWSS7ijo*mi7%~PrQ|`MnXAXJumirA@@WTBlYHlxK-g1QML@>b81}T z=16rge%?dl6rftNpI{j8t7Q}FZm+C?<%?p!tB@U0+EC6FJJQTJtiNSB}1=R z$}8g1H>UEGWU@5v)XFR5>_k;dqIMq1D=2lI-xBptlrN`f!-ap(JH!CW0OjAk_H&74hgKBP1z+Mu?@5EGpWfNtJ}QNGaZW6cvD z-7);gfK$z(qGbl;qpmIXu(njX4ykHq*CN?~Rhse_){$0V!M>bva%j`_4wVJg*0?t8 z^}It@=#%xcf2ga#Z%Yss3}QLs*3jk_hMrW$vV!|pw!7`2laNkoqlW$ry1r-=b)f#S zQgQ}kY8H8Vo)_+TNr0VkBrfR4{4p{ykz}I~E&sR&JMz3@kP;u5U!Y#xanD~)^B?B& z-_^@s`IJI3@b>kWPzoFTU*R!<3k-P6gbo7+@V}C(u?a-`@Bco|}S6xFvWF(GrOGKP&(H7ynggQJ32P>HT_#NZPCpA{&79|ElJ- zt&s&$KFA_gO_(i>thBDvunNRZLkr>c5KJmm!lATY`7&TuE6!zBb7C&1sTx&$Zor?5 zBF*Ond=Lnsy{B^5-KW^iAD;%N^}jXWmd8@J9u8h2sBS3$29!`)6W&}{hY{fdIL7whOd#k5AR(_2y5J#=WIP2DmBvOaj+zm*Nq(C)8 zx+Kl`Lnl^In0;n#+vwe2*U@GZ1R&zuGA|!+lcCiXwp|$BTbD5P<{O&9)S{g8O|Z0| z&OuE($qwi#L(|NTp=1ib84EwDzgd;q|dd~lADHw>Uj!UHb}c&iQhP}LZvu=KOK95c{Ut`B>BCM(D(z6Oka zCw`D!g~Iyr8yzlhKe?W5TxZyI4)vAVu8_IPNTi>?>7R1>|Bvoh`STwV{WtE?I%rYT zUvZcH`h@P^6AwF6m$b`WL{-p_z=#j@7V$mA0wDbVUiY6j{Do}CoBfA%njs7&A7EUr zPXL9tzbc#p>iO6580d4QA+;l6Z!m~y0HLGin8;8v<{TVV-eIH&^5&c35H~HBv5rO3 zXVNecC@`pBsDFSn=pat@+(ru-heOFxlbRRXDlK=>ep8#3PHvOk;f}Lc%=HQ>C%U(tuz+qas)#Y<&XEueN z;%^(6OGQI=zk>-quc2)>#$+45pU+w{nP9=w;ae7%7#K#~#bUkD@uQnLowBzSX&+A& z9AeM?Rv|c;1YHJRY^lo;g+dcvMX-?^pFo4Y;9~RynBA}@7=Gfr8&HG}g#s9sit1xu zVM3;z8vabS-l_K_M*VY7Xa9Z4Dg_XVWLJ5 z$HT3Ihv@!9ulz>2UB>s~UZ@qz&e#%>)+qJM-2*WMazqh6gmRR%rOk#?n%5S~g#VjH z{`t>;#m@IDVf>8=9MnIvp8p2Lg_gY_Z9@!94)C>mOKCe?EDQX@m1e1`ZN{gY!-*MhV`Ty?#v7$Ok3s=I&mP{X=#lv}Qj(kHJac<*ii{(=o&hu%9>h0KHh zfYVIP{YBc2Q>Xcdd6TXp{Fvt+p6wg0DBvPHiLedz=5>o~8$RNUf{`XOWn7UmMOB!n zHtLMPf6p{e*|>QlRvrDk*-EWc{GqdwFhCHd!k91sBoSu64AMHsOxa^;Oprj0YZBzj zFv0Nd`%~ySOK{pRUmgRAJmTsV9(3CE6s}Z^)DLGqm)!v^Osk~4R|T@whPRxhL&#Y6-ts1!Af z{nn*HAL*7zs!1Q=3j|~;_1X( z265$l^27iN)I(P9*Cdr9C9V>sK35rqGnP@`aJZPqG8Nb)T{UZznzLD`Q{%b@D%QIo zJj#&?$pOk@F_FXq{USV*KD|vi{jFY%O=J{&aw&C*_5!@(*NWwP3ai^Z1*$Kc{skl8 z3!|8z|2zBqhHb-M`pP`7;NQN9rq$~(V5ceIA|a$%5pu(&sh=Vu0<`V1zgqW0k6yUm z^JJy4Y2O{!0Rp>89>fYW|&-O{OwGKgK~O*En1KHB;Y56~HG34u(3>Q7=8;;PZH z+mGRnbvW`cPU%f2;ujp;j|n9hyp=P?_kbZ%C9(_4f;s#{CLSuNJeR3%d1j=%G8wOA z@Q~AJcL!_7CcMmStG_l$fWX9JJ}7G#)>k3cUPE7%)vTcoQc$7=;rFZVT;x$mh25gM znRT?X1I@|X6_8mg4!WRNuw^K*{BC9tRaH}Ynjo%jA`)V;1Bt`rfT0e<6X%Dk3ama` zX59A+I-P_ESh>|~2Sd-4;VB~p2zs8Y2D>%D8s;9AF03z`n*CihL3`B|=|w1(-Dx*+ zWmJ^KktON^vpZ~-~g_`cI>CqPks4b1E&GeC0zW};%@$T4~M z?$ZV}wEi+xaX;mtas{iyNGCGiAxy8AV9(F+DwWrGMLSkaRcjD6cZ5g6(8Q^RGyl zk=Y1eOu&pxTg8ayd&FMO)5}m7zd$aIQy|i>28;ynVOj<>drL$|Y8w zGOHZB2uIbL76-+AP<)Ac;O~d>HWDhiZNrBED&YkXqXZIV8ea%CoDYop^?{)qlxpyW z)JIU3OX=dK4v$O|-$wB>53p$B04T#vxJreb4&c1Zbv(v0<|T7*a14NlwV*OZBFW#BVkqM zju|8BR>_uew(q2d)~dgW7AL&O6i`Qc-6qTT#u)xx=|FkV6Kxaam|enhyHNcN?$6p2 z24KNYSnv%<^in)~)~~Plk=g8ozxGcc>81M0+O?t_XCK8LPpmOPW!n4{T?lDA2>9r- zf(uN<+ga#@{3*qTc*GL_sAaT_YQ2<4>W5~b*J%KskV@)D?bnc(e@sLFI|2MpTCr=3 zUT5_ED>p3td!utFhYLEb4uKLCa4kP?fZ(@Pt=*x7F~PVYq9C{lrX+B)OeS+IBau2; zEb#NWVgib^L|ROI4?s$cSTuXH%0K4uWQQ+C5b8RroA-2;>wUHPdwLtc2k1v3Mo^$A zu6$iFQMHlgFo)X853BSU%GNPVc~+U9`t$KdJvD}rf0$9HL6cv@q+|(<0HAI=5E$Cs z*Nx-3zCIT?&@?KWsi)k6H-k2KD=2p|wTq3s4P(t4+?i=jphTY9+ZA2N;VI`$S|y#j z1y2@jS}o6_IorOa(4wuL?DfL`^%a%CToY&SLbH&Lk zi+=#nRwmToFOk_fh-HuBp_nv^gb#b`in)rDh(Pj4_*nkDr zt`?^b6U9P{_iH+Wv_mbe>_5KgPBkV z;(Ku+WRSfCa`C&MZ($3dgL{yWTBX^jjI)yYgI4`Mu!QbZQFCnMSBJ!WgHG728!^9c)7+60|yoGqMg7Yr(6tEy!ec&!( z7$IFm0}&yk!$FFWqD4u=t9l?pIvNXMOQH~~5KoO3tVBA(wapRA)N2Sfmg9x>UO_h! z_h*0hixeimfd8de|03`UfdAuqC=&P=y-M-4o=EGI1``3Cr3y(sQ^l;4(GEZe(i4Di z#Hz5?QYSk0NY+}-6N9nBCPyPNi>Z@oVGq!{43?}Z!2Y3Ql5gx!kVkNSV<&Om@F`j5a zqX~|KrE3bnrL@LxY(AdQy>nQ>3rhkq;sH~*dzQy&Cfh?ID@qqJWb0s)KnCtbAkmz2 z+;IT8;AWvspe?XPB-^l^NVo)FK9S%8qN(SB-tjy+&r#}9EFgNAi0d@ulmhRp&cy~} zMX3K5DF#^P3R_)}Hw-JPku+Y;irz*ZNKq`r;fyRGk%Q4eAJ>}F&^XIbc)(4qgSUEf z%BqEap?lomA|Zt%F>ZJ_y*FF&IcT0i0`tt-#K>Z*pWXMoPX&FtHJf5dt;lmDhr1W~ zEDEsW1;hP`T0A_eAfjV7Sz?E-_gdS@rrUq|>YEn*bKg7n9H-sZIcBI@Xn%M@brICG zAQ%*&L)Im6f_!HJl!I?@+(We;|5o|&?rxF!iMi$n76k&$T#igLWe|~{+e<$aZV#%8 zVv&5lLeeE#^qKxZrK0Pi+S6k{XYd2O>LCuue6&HOcj8wGm0pJ-_h0M^NG%DpPnFZDjE>f!%H`UPG7l9Oyi z){)v>FB|OWJRDwR>)5G=-cB-#jF$77iz>scLFs7r(r_$A93*tVwwiKUeGt`vHKAeK zub3kZm0`7j3iX8p%C@d%74ZmQ_ep%<3>g0-&GUe=jc5i?p@IQN3%2ocY71(iSPF!l z?fg}w3g#Tk6QY_Yo7X4%-zM!R59yTIUvti@9DC zn~R>^xd0T!VVEYkAVqks0_rJ`${KKd-6TiU!U#Sd!$SPptgkQHv+_)i7$96$ouk#3f)%Eqot~ z5L}<}s4tTzSFX{Hqhf|B1&QzXZ92`jo?9s`yTEP!`B`|m9}g4 zlX#3remmphj(anX*YJQpPn^f1`w4Haz?*}gAPdA@)zAlO zvciUN{07nM8n`=FIpq%R0i|aD`F;3gHL$=14tX_g+7HN?Qn!NMbPdD%OnObj7utK? z0{U@NLI__FlGBMTou_~d^qJUcP5xxjZz%o|TC9;@an4Uv^9^>mahtK~PXOqv{(`aH ze&nNiVz}s|=}$knup}bg%i`pI(f0FPa(?0gSORBoVJ@*_7nJLKp>x|YW>KJWs1(Qe z4_^_~lK{WSa=q8QF^tcD&Xxb!(_cG%=c`Ql4@;r^vYlwDFQ9dfnyy z`9{CpBd5prI-vZG>P8)WYRHVdJMtW7C9u}i>H6I0B~+sZ9UQ>U;Ye)vb2kgl#HeAV zTa%_G{7}u&vdW`lGIae?V^;v~s?IniNLw4Xd?074J>z#2Sm=^vmv({_YFihIbS!u9WRx~H4;c1 zWI^0|{dxrNiZ!605B#xcJird8u4p(Q$fGY*n2o1Gpc*!;>Eu109&O%;?cJYSy2*h6 z!%i0u_AE0nTA{c}HHzwH36yxNuJjyts}&E%GssXkD%6*-mVltEWoe~63B3~KfVR3u zUqQR)=I(IZDyx#--l9gYx!mjOJE&Ck_Xhp#@_drPg&x3=xT?_BOT;meJ_kaf8KTo> zc~X{a-0S;!Fq!h3K9fuCa)EVyDF|+@#*#(1e;{VioHRClw6$Y3hp)KKqHiw(e*F$v zpn)x7bj7m~sBt8ynj$X-vCXpsGy5mK#HBrziMp@W|%Ho15j3_&uUpRmwEjkF8E9#KfdL}j$ zCax{G@Cj-j3oT2(X1aHy8j(EH6kaUJ+?NJRru;R^6`N^B9>{R0GZ)RZf zypjh9Tri`k*+H3u-?7tHxp#jS-N@z?2!>BkrVfe3ynXbSFHfc1Obf^Mo@mjMPItZ9 z$(XM?&PQ0Cz7LBAabXeLQ|wVgMrl9#t!$HNP*4c>!$ZF6nJ~A0R}*!5*Aw-gL4Ocu zLbV@oYSv__ zE%5b}QJnzk$dF=eZ&UHK{mcs6uV}C38Ta1fB)?(ziR#0j4Ey8DN$Z)p%8`LdRxfi1 z4k@-tyzsU0vC*0c0(;3?ahAsBIVut$e~UTaVE^851V?;eUZ)t#z>I z6o&i;1tk}MuS(89))mH-)VCITFx=VO&|vk=5SZPxqgnBO_;+}D&MDPn%-Pv*ohsP1 zN*4$@iP1R>nDLx;iQ+DKb?b#96$v$Ha@Z=^VHf=sc^_?X3#}$$_v$1J8XP`APo8Xp z$Lj+WsC#T@bnJ5yUU&UnVB-ljJU$X#|<95Bkst>*cZ(7*4MdiJp^Uu%89TCDDr) z&GoD-Y^A5H5kfUP*@9;2h8PC`uB~j-HLIAR=hc-x!#_m2vAjkq8l@pDc{sCRnHpKj zC(}HuH>7cW{%(CYt|zmx#j`=McG<1t;ty33mi4{pu;YT|`m0#KVt_OAa@JD#FF3w~ zHfk_bf+i|M1Heuk@g{H*+*tYescDX7&8tewB9A*=bV~**yIp-x@S{oqK{^%};ip>I zLB%&W57&v;j8*CC|vPdwZ z)iAfJ8L&(V;F%dwy5AuImK_OGf$n6>2l3I3c9?^Ikj3u< zG2qS)i*MWeh9iIf<@>-UO-^$pE@_{dh^r(-8mT}^S_!EaIuP~D4}3&4j6>*5mFJ!fh* zO|D&ZrfMCq+Wn8fu90pcQ=vY^aSX&9L|eBKHTN13ulQH&!v9HePUtF&lY9P52=-tr4Qy zxlDWBz*#%@Zc|fqK(_A4`M`;fb$1E)eyVd4_7$p2>|rEb`5tYRLsTuaW}lZJOE>n5 zccR*~u?}!m`0-(SUt13jEsPq!aB0deh6&gY(oL>VlW=lFKhc?3>iUR=PqcwL)OHa| zx4t-WtZd z87k)bn#$+K7F9EL>DcjqRlHzM8{D~>q#l~P7)Kh&{9=YHT!}t>NqwC%rk7aC8Fou$ z{3UlDtaVMv4+x)^)tGu!*jtswrn)6?GpY=D(^?|l!P@&kz0!?B(i#3H&Io2tX4QgFqJ5fBZ9$yNxDi|B0B*lInAFkc41aad^#;)&q~pG)#}ka}0rO-` zj~D-;3C8U?;ZNphA0qxbEag`w`Drr4Tve<-(P?dd3b0IZ?AgAt0A(|VJq^aq!a%o< zX{8r5OZY=^XQN>*94?3Lj#4t`YrFBvKJ*UW9KsfS*-1%6zB{gTW`Oz#&(17mN{6sd zY8s}<9RJqJpCZXrkt?kTzhT`*H09mEp^fAYf5k95i}u^V8sEs z68q~XIc(;u6Hr9Qgph|4hfGk?dBeai=iUvEBY;CDN1p_ZzCNqFj|Px3`+`;Ve6Df= zYgxrzpfr~W)}IN5g4H}_UV`$NRT2WnhO3n>3GIny%0eF9@zCnGgH$4y%(=RTQ@Cy; zjhQ~jF$=#{+-wg!2T#Kwi^6c$l@jbgYVZS7OuL#m^#>_Pl|_X3&e0>p$Ux3dC(Une zppr;IMe1shM-9ho_@VE@lykbH%;}eYu@9*RxF8d~GnG!i!p&Dc| zqIvy<-;@!Cz51i&@S;$1c!`*G`d|=2A%(|YD?ce0Kcl7oq?~=mRDR@G{RslPk&jjU zl(G(QNIH|3sLZoXe)W^G|0O1TIytm)fD6EUq@=w!+2xR;jVnm+rc*>|MRi`2Ih3_J zVqrNiij#cjohTP!D-v(<7P;JE`mmAiWn)Rh?+p$ZlOeC%I5ZM(UJ!Q4}pykQg&61s5|Oq7J|e za!ygM6Tpl@BgL;ufefcWPHGB$CBLuTWYFr=9h4j$xOWvJb-H^Yb<>Xu9)P?D?2-|o z)y71)utF>LN%|$Q_{cepH>c4bty_|{lY7&5w#^S7!$2#MMI#{=$n;sr>4l6{SazqA z5P!d;RXyM+L}gm(&Yt*VCJ7@mT5WSOgt;E0=!VbaMD4}kj31Jl!CVGzektr`^;1qn z2yqw;eW)P@wJt&KNLiw;uh^-MwY~%D&f2ji`$mRCJbbhs`8U)Q$NuL(t*rm9HvUt< z{5}6=m7A(UOpE6sWCpPRd*3 zq_}{b7Sq5|lj>?}FXag&*9oIUOa;gD4L&5@HJXS3=@h7%hri0cJ7Y8B_Y=N;4^V5| zZ%d9=yMdpG)O4i|{e!9-EPJUzl|k!pR}8JbDBoJ{JgzkLew@;VM^FYeYOD0h@;k9360FlvBGpu9!Qw{+cfD#%pU6l zsf{G^%7S(P<2hti{2AF5N&SQ$85dV}8}dT8EKCQ|`$1QDH^g$Nc9jZ4mc@_m!B>O2 z;8l25<=z}|n(%aHJRV}nbtk>z{ry;23&j!E#uQm9vZITB_oBxF!I?FNjlU~v7jCfB z$PbDuBnJu(2sQDCAO7H9;*@MHLgMJt^q%2@1Tp1{|tSb(8vCDw+vqQhn~V` zYQ~u)Dmb($BFaRaDb!O7m7oEJzNpE6FxxeY0VM$RGgw02AZJg>*a z4I9;(i(GmKR`?!-jn4dpT7yV8pMQ!~hiIHMISJK2?{CxVAE5_wCf5Augxr8J#+1gluEn~7Rz1+EMXpH0qa@#cXZ}3+#zC@~}xd6OD zuI4z*t=j46*X;$iM+_-LB*{>#FbF+U{PwwvyBeZ#C2~ZaOGjjGUlW=(C5(6U0W+b& zd6mR=PV3O$;3}H4CYcBAQCF8F8Hj*8%Zm~9(x5BcJH9fQb**xenBTq9mfn&oy|DP~ zichr_Ff4*HhGi#lx~gZ-AaGgJ##ZVF-_c3tgW@o zG5CC(KQm!h36N-+3`y7qx#gL76Ns82-%pUNDpoPWbsEE>a!)cP4EGGLXEFiIlYoyL z2~DJ&S8a?^%z9u#zg%TAtw{x_f2UlD2k_xXJv^~o3e80|s$pl58^m;wQ?PUwK9jot zNOWN_uvaYl^Ma!@h?xy~ms4c&klTM3ena@efL`b*<1#ee>YVA_C;Fs+ZdpM{_U2a6Jb&QYY>%* zd^lVGg{AlJ|6&Ot3``0HAMt^{MK2621PJRtpZ`Da^QS&l~xO*z&4dbQ%$^9DWN6}Ybxr7oejV8I?#t@VhIZalFFnRtD zK}8d!9DDetMCqyX@;166i7@-^Jh|P(6}VvMWIOxAw#h;@0n{9K@Q+-4Ho;PS{8mY{ zHpF3y1zXNn*p*QB4C|Y+RXftV8hW@Lt7@pH>v$I%*wJ6U$z5_DP5m?58}%0yZzQ zIwbu^fBrpuVNse9k2UqIqLFTdb?_1i35YfK6j*mBdU?|%zO06FewGI!+03TzFsvRoNoLkjmqdBZ^FFZsmet*YyQJ%2HmS@wpuR#8e8!pGI~Z@N z$#F%@9B>x4v7WHilg5JgIlo2$h@Y^05z~?PP(#`%@NKjq1+@&PSw~G_p2AyHtwZy~ zvAAY5ma(Okad4btP{R1EAFm9@>bj`eyY?fVZYqYL$1WjK+;Pwo)&`0Vx+>~UDT>@G{RS=^7$WvgTELMju#20Y@rJy^acX?`*TZmIHAaQkbjU@unzH{vN$^PG_rs`vbh!9deyO1Yq029IE*{%!n$zH4=efBkJQUs+TzZT6UfDy^iFk_d36 zwN#CQ0h~fxO#}m` z2yEH)m?xHgJP|%!;%+r{F_dcpJ?Q|zs!!XS_1!1*rmq-z*jvjw+z@;gs)dtuVn3$t zaeSiybx-HM(-kOCUFm^vpg#gFPMxJ$kBTY&RAz)Rl4>n!^lWXgdFhlrOk{kvTD>i2fI-M)o^}}`?e(A*# z=$kG>x=rmJXMb&Mdx8R$5z?YLi}fS_pk6Vci1Dv9#FJ;;7{Y}`mat40i!{8m(uXll z-ER}IWJ~K=aJIf4Y5rc=<--dAgBcv(kKGcnTw&7*p;ui#t+0IHE3p!*1Cfw}&d@@b zCy0wCF=GUOEh^q5r;PQ2dc>@UmdKMZZk2TBxkIJ&!^;qSCo@V=l^v(nXK(}392l+{ zpr5J!UE}~Am5?_i)>2*v7r`f>sj15&{(iuJ3Uvi_)`~tzLn!Ey1}0+FT!H8awM}qc zih7C@!AwFdnnG;qbHQynM`?Rh;}d!Nh`BPFcSJ^vG;JY)DnygGB<|^NLxOKaW##{f z?J1+i-1CpO!GFW+KTPj0zSlerdyv02#8%psJd!Bj+!&!11=z%Za# zR0)Mjf^s0FjOmq`#tf5VV8s;KkgmjlJII->?jYFf4Mhjl zxE8C(Kr@NsegWE_^1Z_;G^erncvI#z*O1J0ea?3{AvKN0FcykM9s^YR-X{zsvf7dJZqZs6~>=g@?8EkjZ>6BQ6?S)o$-r|mAATS34q?o$gq!bsDAEB^^s zE#@etC*G{-%iOj2?~MIW9GvTPV749wds~EVwLNZ65-#2;6Enf=8<6}n@@1_AU5sy` zze+v!PW{j=W@Ncrx%MK=A6;IOuQ8^NotaN{{D_>L$oSckr1JQ(c#|?lv!%_4$Qxc5 z@z79-HRqE-CyR?NN1ZG+WSi;%VlR#*?1h+S2-6AA#!}?RxNl(a5yy*_PV4oBPE26? z@@4sKv?Wfk-3DzII|EXM;9wffg3CU!wdWC5Dn?!~0RLNa{d{L&-*e*`o=8FY(m+~k z&z}f7ToIYQE-#p=S0kX`i+{8!&p~@}NWV*c>*bYDU(I%TVN}3V*YVm0lqk*%+mi%T zhvx?kGvyCbs{{lA(uOmN%R2b5Xc-U zJ|Th|-)@T=;k7yS{&q_Zz*3u~aDcDhF31K1m+}j{-RqLPsw1MdM!^voeFq5Yib8KP z1()~4!B$yR_dncWRVJ+xI-%M=m_q&;ui}gaFJvdEU zA?xnbyJZC%VCSIg7M1e}Q@rIZ9YBJ)1&-Vk-$3HuRla4mi#i+z{B$nM&f$K@_swBa z7=?3&Lvjs2F{4;_YqqhqwjnLD(c6hpT7yIG0v5M8l+q=EgMJF06-n(UVB;4#cF{Kh zy-dk;Eez-cd=veVFUiv<*5?|v_^V?kC^ykP>^tUcFj=Q&vkQa&w@X}I_*q|gDHF(q zSN#!_9W-qVlkiqt;Su(HIbOjo?Ymcs(7-4?iFAw;?U-B4f1*%B9|5Ax5SuExM!zw85VBY@EV}3RJI?%rh@S4GT|E0EQ zH(#b-UwptHG_3&mi&iM~v%{qsxFaE_x+1{=WbMDoq#Rl4@0R(njj)z4R?vywy@AL| zAqq$WB2o)fV$}mcmisarwi}yPU2w^HLjm}|;)J5vp)Cn^{b{T7%#UAUjy*4~`Mdpq z{f}>oK{`lRsC$jUQb1($x1yX?DhzMttgqdJ??|znPs7S|q@w?bgb;sm{(%Aopfg6m z*{`c6vgp)AVMN?`9Twu2gUc4oll*QCkFJsm%_pDmPu23YQL$2$njx-Y9K}@76}dIMjO-lRHK1ww3miH(6<9_lNAK1n%1=kI(X|U?D0 zuiyett5V^GOOxJUhXicPBMTyYG*|O-HaF*k-jbhK)*;p0l@Fywk&vjuU~IK4!|HOl zjb7pn@sEQP?gqF=e@G9kS|cOPq;DOXxtaC6xS6?K9S@#j`m^%n^}%jD?Foi+;e~4a zD*6@&OarE2l+K5w>5`rWY)aa%B%h%necU~Cs$fCez=3%xPYf8r4sFM#!gj?}s|~7_ zd@6~8wY8yY(A6^jj{l-&dAP-BLo1K=y8T1%f?)xPIUgyLx8!hmFQ5K?x-c)nyxCkS z;WAvu)BEDb3l+vQ*zAEM*^d+xH}WxRt)jw8&jLxhH(W^x@2%DrA)(#GBqO++3GIJ|z zGPV5ZNmtNg`VFh056CqR@-`zCze0xzv~;LG%I^-71LBMHdMR>`1D>nIn%Qet_y$vy zqQJ4^e?fR!!oXL7p%0VRD~^Sp_N$T$F75Mx5Cxz%s)j9& z(p3^5RyYa6HXxH57#z(r3`6DyAxDttKol61Q-Nr^K+uj_aU1XIN2F;U?L=CWsp@imVo_}i8` z1O8F#Odx!`g&-2;OLO^7ZXf`@A7Y=FQ(k&JF);41btCZYS%gmwg$Kdn=Uua5#&>dL z>@{OryE8W)TT?by?^#9}$l$r5_KMYYP~9CD-`dlXk_yarHd*Xa!)b_zI5r4$O*DuJ zE$isTR0bK|rmm(+^wh>h9Ca>aHL-D8Yf#~-u0j7C*2wSsko}3f7v_N5FQzQZi%!oK zuuld{*`MjEPF?419mG4H2CT=cR?O5?02Zy&QGD2d2Pwls(#?Nf6|P>IMz*=hIjIvz zUt@QI>1}niGY}R`+YgUA{s!{kBx3iQ4RxP#)6WQ%t!q+wSlOHZ48mJl;R0`YK=wY$ zS)SP|qiQb%mBX0>BLJ{tOxtqlGSV&c-OmYyBcZuz!c1wL+w=$%;rYx!P#@h_-I$N$ zbjC~dq-mMXr5Nw?{Zo>8FC4d~?qD{XTc~uTzyED}x+C0`#_srDt#@hJf?~|N53<`9 zyUjx@%}4lExg)=ADS>zZ6%26SnT&-~T+Tya++if5~|v#me89S z>1@zl!)2a6$1bFIo4|-)(HiWEs#ymW1fHZ?A0Qi9LKH@eN=D;UtqmTq+piUyqoS;+ zhqw6BS&h!JP6fER9$aXa__8{^o|Re$?bj*0;4OMnKh)jsZjszno>7dgD^t#5PPMcS zWmy(kbCx8-{$se{$Ui3QX=R^VV$7 zAhX^h=MWl>Fg=1s#1&DivgG;!hLIfCqN`KlFG(fc)5DWcz9_Rkp6d`gMmYho@YI=s zwRaEzWF!E7oA3mHmol#bIqA0n&m-R@;BmM|r{!n9Ukf8Uc*bkfa~B*(&$U zZs8l%X2&&B_`ofF4ec?XKpCnig0seGgnh#y!YHPq>R4=BZWQHsNwkgn+@1tA#LMM4 z*EX;~mvfD&M;3ljM(kF2rSjX5oU(0zN(U^XYyu#fiq|Si$K1G4aEN=C;pzdy{9HFI zOcbr+A;N>Fx=4)o%5)S$bwoYjgSu>yoc?JhM7fYbuHjhkiR*tTuk$;7s8+fM!yPi#+Y+qP}nwv%t(z0X;DpYN_Oo~viA>R+$w z>MGwzYPii)b?tV;hstYuaIR7LzmX;cWj z>a!>*_HDq})3%bLm4^fH-gN2B@2Xj+zy52p@jspJ|KASZ0!^4|pn-tQaDafg{_ka5 zYWhAg>c1I_7b?z84S1j;S}TWE<|^DmFrZ|S4^3)I(lafg!^ zJH?EH&dbw1=Ei4d$ERn9m+xh#BnNo*Hy~6o%p>I1zEfhDpJf|&@I&|(aIkHr+oNE+ z&B$<*dMEhj5OMtVx95)`l+CRo6bu1IaO|Hh2{S$leF3!o15e>fU(I0&;a4gFjQ$a) zVyK*6O7NW$u)dO^g~HXZsHmAx#n2h_J-X>JE=EBHusc)a?0!|Kg)}&0@!=t)<~^$b zURu@H&Q@O`-RyWL>R@eM0EW4*(r~KqD@!-YK?O^8@*O+0v;|99dKEN#^1Lz%$JJ7_ zz!YCzx}W%Et9esQ`KCf6&X|#(sBmpZS5I42TaT8oSE{{eVuo2(%<#OZQdp}_!(yv- zSv1OWOOpp1;pzFW^_A_V?e(1nA6Em189@sjpPJzDYLjzl_6)Kp0BBr9@$i#nPRgut ziqSP|G^)2QD>SMk7Aa{4H8BsRYyUi|62Fy|xw4!nctAE@xH}K59!HR2G`8&zL&0t0C=1%#>! z=FN2e;LbT4M;@;aU|nIT(Mg6gK@=v2cb=T?reSgQ&faa)+~j&5Ud~&&#a)J~v=S#* z(?->z(c|zjTycvRovNG%Afc!}7`YCsi8O%_5^U%7p(q8V!9H?OoRY6(maMU`(lZkd)sL+v<+ zYl1bpI&nhS?0)gLC{9;)9mAFzQ6}#qV8Gxrsuy-eH1cG|>R7 zElQucR4(dyGi_msTcj6E3!9Tiv#d-B(_*MI$*3^7b*{L?_}YJy%kn(s7)-FgDj9ZZ zBK?pbbvceXKB`3)4l&ZZ2nO7w40#TK@tX~pnZi0JX8I%K&Q+zXSQL@3jyx2&``RC{JfL? zJT9pT6lY^@I|e~xWnY@dB0(TBl92RVWgG6N0mVdRCeK6V!UibcTJX&f45!nAimJv! zxnx8oGgE#DvamA?*rx@bVXw%l1usmY=dsxNp zYEW=eve8+-i+68c!-B71)4(+z6vtVX(!}gba#~L;o=~PO5q_=2)jd7*r)p_O;wfA% zJj@_^K1yvW1rlPcc<-Wk{eZ(8Yondq+iv|u!L`%ZZWMTXuG`J4|nodmIy)@ z{ss2>>cl1!;pLc1w>i&T89b=!XBSihEN=G724NwRe|(I?mh|9*_PPH~8#OW9>>7MDa4f_GlsfSk zH?6P@1huKm6WJFoHAUDC+ z{gBUUHVq=$+X~_1^xk3m+ai@P_wq`>d`zpvyr4LxJ#-@zGh4*9r#sMa`*zku$o6?O z+hBb6CG?+Lfic7bZPJMX|7gDgIs-kB3gei7g&Xhf;L|~=gmKsH-qrX5f@e2gVFj&I z?^voIG+hR_guR6{(6|x3Ao{H6ctcd_(>>SDE##5z`IM%`mZ49NY&f8{dtOCwko;6(77zWMDe6de78pgteEBxA(gm`CaSYs zhiqD*TC%Qg>OIh@Ys}=^GAXl0Uoetf4ha1mVyj;A$Nr0nUD6j71GX-%eS%ZJx{qwa zUH-Tl3X~d0oFAtDtSHLj^?cdLU{}Q4CAi+6DEX*`erZr7IQRyo8VM@&|MM7|aA36l z0_8=}Lcc1s<(?n{7+{JYM;?aC5k+op^S3du80-`!85pgOrq4qsoZanaq!|KC%Bb5%?qXR zwULo*%EPUx6vp?eSU=EaZQdVNC(5~NhGDjw99(Owi>Pz0ENI}`ENZE(}suL;+Js?X1 z+7RuZyV$mV18;h;DRID15V)uakvMsAnj$+iZa9O~xVSNxT-a1xrLFtQ7Bmv$VE$8e zlR9pbm{O*9({B#4;r%&v9lI;P)#df&?WPtxyU4ii$TcBkuY z*%y#Jsy;gBJ_R@@ZL3%aeoOluG6210ufW-0f-{!SFPy;OH%+g+K zZ5{bs8{4w{*RR!W^mDi-NOg}YD?2;PTL{KE&I>gq{$&+(mc)2Xg)BiM6TTFdr?fWp z*k@dOUysGi-^RmJIPU^NFyhW1KjHc84chHIs@ zJ!0_sIW8zMu)(@r4|cdZc{G8EjasIChB%A(O|@lb$%rJ3Dmdv~d74oMs8ye&tn}#| z4YgpQ+7!0%ta9#olqFwf=5V>^xGcL^eyE_++kPW$MgkfPH_pM}_fJQ|xm2ONM3P1F)y z7#|_Qi91$~*Dn|)#Hrj!ewxd6lH0~!%Wg|HDaofkQi zUefo9;dL32oq|G@SuT)APx9$-j2(L5=LhGi!-pPEyyqn5Rc_}!U+8^TNCqpeOo5&( znQzp~HriNaW2h{Sl3Bs}k$}xiJi}Tg^I+rb8`e1WDXZN#4p_^9A{66zbl9X~q8zc> zU6;h!;tsDblj4XQrC5hvn62h&WSJ@Szfz6!DnxL>`H6L&3a($e?jBn%+|_~UuG-~= z$%pBlU?|-6i5icPq%+Q@He@vDObKA}A%$D(CnZY|fVznGfU+|7%fCu!Yf6uP1_B{ZOL*Zz3V3jZs$7lVDd`sUAI`+<(Kgd%An z4MRB#$Fzu|2<0;LaSiYt_$ym*EJ5>7qzFGwJ9bg>s!P+kJ{CL>2-k~U-$#~1Q%C;lS7j0wT*HO<~xE-mKk0x4{dBWakc!A8mjj!fx-WOvn zNq2bpkD6!l&6#l@>A#~0Q#z!?K2p<=j-!yGR6l#zGZ1tyK8Odf?Z7>_N_(4|B>bP! zZ9ww_A}B|e>`CIVI@UFNf2Q`nxeOp_;wW*i@aQ}S?=T9<1Z_;$UAlOO{*Ui*9w&tI z7Ry)Aa)**pK9-Ko8;olbC*v{hdgA#`-1uL0t$(!6_JgzIJY}*N9Q~lhZLh0xu)_0sw`y!7P0?M)iflH4(Xd z3`$KN>dF!I{5dbE2Ed3v^kEv|P0Nbfp3M(t2RFf+s(w9y-6S7c=MF{ALZa`GH8`2{ z#O<+yVa_dPC9jmB@(PESFJX(qOrXnbvEiw?G1Rp0o%F}2sl6Mo{0xNcOYBf9Y0<}KP>>>n`kA@oAKf!bC1_ohMvh4<*fop5Ga55fb68e?;V$?oEL5$5(=A;M zFkOb)1;Zrqu)v@ug8y6ymxcRZG6uoG6#T)I6xsyxK$&>5VPhvyWhO1@FWooloab6@ z3p<(Vb#-SmaDnLgpj%ojMMvzP?LH9j)W`%U4hpqCbO3i~Ax*1tpIV?8-;Es?ETUm^ zpdF+kDr8-X@5PPv#zxw&=_&3Ks|PWr%pHl15p0kZHi9vAP+B}n0i6tL=oQJm{7LWq zu&Bn@!eyhxoS}4vt|)~ytG}0+w9lF~MCqaBOYJ!Y{SS%j@rhC%_&N5zTo$=33+ytT zH-vrZ3Y@w97X*D;3cVeaE58IBys9d_EuoKpa8-`L43Z*t~ksjkQsLvsYdBb_x9)KiaY z^AEUz2O7rht>$ppU>j0;2TYnXJ?M4}#=>y*yHArIMC7%k9-#;2*J7?-U^`Q{v|c;8 z9iqc*{7g6}#aICbELrKF!8dEuvO7%~$CYPjGOBc~VimR30`};-HLEQRU(gc=H37!) zxMny_lW#opz)vwUAag?fP?rP^_!qur-|a_o87nfnQqgVT4X$I4wr{+;QSS}zTDx;u z^kH!Q1qkgi4)h=MQn1Aq+}*u(5PoE{d>1KxVKxbq7_k6ovPt8)XS)?)mR3JW_p6X_NMfqkQ z-1g*u$3Gp@Ug^R!_CP(hGaFI2voaO_X}x(k53&K{OJf<(J9Lv-te9vB8-+0r7oPft zokoLG0#zb%v|xTt+#lGcTU&|Vtl(?)X}?)`v1)7eEB8HHov#Em)$WS_a*yaaPuG9I zF=AGNuNk8;&FE7+-;CwN6SZ70Ta9{Zd4smz4v)PoJwt9{5m~TNp6-Q#>HV0$j zKfzjDKNovvZe1|)1B5>Ndl~Nnd%v&bN2S?sN$g=SE0%VDpQO{)Bax5Sz;y<1K=?gBo-0e`USfLVLwQ&XMEBgQ zW|Z4_H=Gn=1Acuo{sPhjKzHz&N~w^FNf0GmwQ4isgrekb<$_B6Q|EN66l^9Cp*739FK}A$x%e|_UT2v}W`Y6p}k!6UDjql)eED4&bjl0^~JL(;07jOOwfATsi?USUn3_ZPk|c z=yF@>r(u6w%^MMs{9890@bdPFmKLqrd;e5q^ z9>D_DNTeS3otkngQKZzf;!F*^g^{_#yLV6b<*rB5Hequ2D)TiL;nQG8-|$Pc%()Tse3{qgHEm7G*iZ+!1tTpO?G*-8o=1Sh&ake_v>7QB2E ztmC&edBneBba~X5RytVP=N0DX-f@)WT76l?Spid)_i0LQlMAa8b(qiuRAT#9$_|uw zcx5lUXc{@aiLWds%)gbI@2KJSu;EN%G6be?znBJNewA;sMRPsDS@Q#=Uif=8pIA9? z%wEda@eNE&k1{69bksby^S???r@^u*$2Cst(P#z}oa6>bg|lr-u!fArtwbT(B!s6} z)utL2;R>e}t%E-#=02gr2VJAiU&~{diLsiV0fLc{vGIo)T>XSb{(lcfAw|;&We#Zv z{1uO>8ravnLj689`hfu5v0pZlu$6niVTf-}Z!t&|&J zU7baz80Mvv#EkjKnB`!di`a?g{3zc~8mmrKMt2j0Z{XM}POH=&mub9dDVV}XvrGPP zqZA+(t9c~rK;Zv!c$O1aex~Ii{%no7KY6R#AnY+@-ea2M@$~iuUirmB-p}` zI;E$R^wj)!H2430HUEbO`>&^!oSdc}_`hFSopXrs|5HN1qe6oq{O5y;#2rz!e~cJE zR4NiYDqgAr7Y_>j)Ctk8# zo-dnwetdtR4dER6xk8+avt|8d*_zY}rj6$EdCEH>cw33Jz}Gz|11sg7E;w-?6eRgp5_o@zpzCX|WP!t&oXVEo{F#2l zz(lb&Gx?)DF=w(y(|K|&TPJym04(h-hBoUK0|;Xv&UDb$*%T?R_o;{@-_l*kuCc!j zl+kaP8ZQvvH(2{46qj06uebXUN=380n4C%S*IU2>LpsGI&1R;x!yX;|@K#iaaGDlt z&<$K6KJ`{sh;f>#HNpla)G5fQG}=ow*Eql^*DW6=+eE1!7Fgelh8ch82>!b+cmjjov zq5QH9);o?xlQ~$|MS!#7hFm~k=!Pr#D`LBx20=(;sKG2}8zCi+OB20B#$tXhEe^-k zUcMQs1YrM^W(f~o4^~HBuaCUFu)xsO0N^ouq!1+M994+$H(GS?G!d`xhKM2Vh(t=P z9(pIh^qf}H84-W%5MhQGrxjf41${n%>S_2{}+pR{QnXz{$Cu!2}C!e|AQ$JWa=(5JYnh-CoepJ?f)1yt8OU& zQ~U8Z2$GD!0JnJxwPBG#kBW;b38H~gnF&S`A-=O+#+W*-$z9p!dfvf)0{8SZ`Xvy%rX>yOMozeL1^6yMFtAna}_d@zjl_N^q7E%gz540j!%kVTh%T zMqEgGIsos*LROXuFoFwnqK(>sn{%@tf~{^a_YTaXwBgC1_XViXz84pr#Z+PHX{>fX zR~E7kBi%m~V=AydURibYyHvXES%+R(sWfxWtPBdj$!0`lHlHalnWCz#t0IuK4Ju|j z!NGJe+pHURw>Hylignx0Nx3Y~wKWausoWn8Xk&8)@z`tt`lmEEAHeFy&PgH=RS4wd zg<3Z`@p%s_(dqV3t@)gMD_O;QPE&;8yI?FsuOrUk>>vkr=q?Ly3lr`<-IHq`5}0hf z{nIH??NU;t26D-|>Y(4TrjLEAd$JEgSiJqFb?dySC*LBhvig%dfs{~mHudMS8ERm* ze&vmlBciAQ6szQuRmo1!^z6jlG>m`h*O_>;|`kRK6yz*=O+Y_Cm+@!fCJByN?+CzuQ1w61`QM=0rTLJ(g?aqFP74I|V310fPlAz(WXJ~#_&*2Hy3oF8 zOJ6@-oV>gpnNm>Du!n)8=|PUD%zuf2p(O-CL&&Qy|KH=s^TBbtl8ztut%V3Y& zu1Knzo7>sz*?I_W_7|bOfEiSdlS@?QjMtcxefMoRYQ9VXfPo}>);ict8?&Z}PfCEg za1f@<>0n>{WOypNFPc)!yC~L&dNXZRuGpDzD(*}mgP8S|A~92FRq?$IdvXd%1mS3p zPQ?79RQcF*?(ja5PXqz9<pv@hUSq5i?6PwKP$m(Uwvy*)^V3!`NeF&HHm)e3^ufgjc zFXu1qgqLl$g8Hbm{qe&Uh`)C;F(yv^xJ0ybJTa!vAxdb&B4x!9%dlCSLN=)eRy{o`81VqBMs~!7K9+We*wYW@I2? z)G;^9aXdYNSGbhqu9Q?~2g(waN4v7Hkbx*lMG04j0yc|3cu6-HZOk+V7sni~r#VLf?lp)-US~ z>3%HpaH{heH?(s|LLzP6fkmeKDBg~sZH3Yyxy9?~>K;KogSTyMWQw4sQ|9_*v_s{P z|9(T?tH$FuNUX%w}ikcARZC>J~viBnOH=?c2p!Vw0aW<8sa>a zZ3!HQMkL-)&vyG+(@}rIT*zFF3olBQ4J?1oa4{|x3MYlhON|R>74NccFfJB`1p`7y zV@Mc2a1&lmrx}bikb(}HT@DJ;r0xn;`_@#L3J+GiKOhvEj~H#6C2yQSGaqBJcNrZP zFri_a)?(ck_(p3DhW-SixPn$~6D}IKw;Bu{U`Yo|Tlusc+#C#Amm;aC_-4iUx9u>n zS1bV9XtHJ%=g;qqsa{i#UHD2pr6%X^P&HaiTTQjDR&Oo4CL(49MQbq?E*#iklL}CH zdZAaW3{uP4`Fj^mGNPWmV(Tm4$r@vM0HZE*Pp9)hf*JaejJE(mf5mZ?7I%;NfHPeA z)2c&zuB**MM98YP+_1nKSRu^`Bb7mu=~3uCIh?~$J$j(HD{6;S-mktC87~n~$tZSa zivarD_UhvAc(|dvr2xr4H&;R};L3{XJspCH?-F_C zjYJ2a7w6vy7o#f4Gv%7(A5AT1gAW7nT-G!X-!T8V`-UBkU2`fRJ6y3J< z743!k>6UXcc2l(j_u6-(%7q0{07`-v4xvO3mew;Pw1IOep;)!0;o#>Cx;U22L;Gtuf9#vU(lbTB70Hl6 zE_)LUu_6uk(BEaHr4unkAzriFg%ly4i+iRDsWgcB^#R8cC zXLVXD96Ydt3rSrq-nBbR0VMMxaVE*RTf{~8(#T4$GC=btIyRIeogzy8ZY4vDWCuGC z{PMd}+n(V{W93U{OdDA@8qBced5m0@2wFdF7Ic|fE{19-&Mii&6Wbjb8ITEj%1aTS z5b$JcEET&Mp`u;7f z!(x=vH!9Y5@91a82`C@)73#SQbaPPQgXBFR13JpguaoQ@YvU4TbuqSGeW#EKEsl<+ zpI@Uan8TG;Q}#98GR2Yx_qn9sD~7@yIt>O-Nz*o-;Ft*_EsLFYsDkAf5tcKG>A_n_ z;S5>z1q;r}C&RA0QnoQ{T*$zos}{RlRgCH%i;E44!?<;_0^}}H48kH0;cLkM?A1kV z;b5#++$V@hPc9W)jvJ9;f!06W;rqfY-COEW4HEyZQ3v@w?9meMi=A-yWWAT`5xUNZ6%gj`3a&FrTGCTG0H1=l1juPF9L zO0#65?z=rMmc+mB1p;X?x+Ct-&iy5PtfOL%Rlg9YfORjmco@nTb&p?eNuKPaRw<1k zkx-^X$n6LN2|tnmZ&OhCM_pTGm6i=co1T^R@C8imp9ht*T+YAzi`|flc(XzIa_g&e zk`kiyMSh*YI{TVAhxX=c)w8gS_#E3|ZT6uSXeYf{!cD7xS#P5 zy6=u+lBZJX@Efg&`W{Y;xZaXvWfMJf1uH8W!jL&=8%U9RT~(fC>_KAox0A#QfM8$1V9Bm>1aMw)8Tg%{(4ULVN3hcauWl(oF`}GP z0l;%DNueN5;WUjpM@S^6_iN&&09jbFP=x}21Z)q3VDY}IE;}~Ds{?*BX=a=d>prab zGv0SP=@&+WEHh-Zum1NS+8T+(?4RJvnmpoG@>xZKwfTPb5>iZ|gIt!kav*5&2hhED z--}Q3jGa}K=pzT1;o>E7JkMXdn?{JYUj^9C<`2t3$-HP4=wuGj*W%mYXb0}iL&cwrAyhgeU+#M#iEu(e_D$i9 zQRQ3bdn74md^$ zA%-SffLcGX_y_&WMDBJKLM3tESY?4NmD)XGLe{CMI5Q*jXvNAp^q6@_1gZ9T$yYK@ z)(I`L32jQ0#mu@4sWpW{pREa4o9C6n08`a-of`kp>0hY&AV{4dXW2fu9f0V**dJkx z{+({RFQup2jL3qeNtQ^ir{ovbI+4nQO%SsB$7ram-5-v{!e0SF+9@e^*4*0FWk<|j z18h0e)tD`5#$_CMl_(aO+5*4$MR5BHW7ry%ia9dE2@9f+ih&X^Tbx}o^UyQ2s(dAnRkH@=sX8+L(P#?zj;*$6m#y} z+|_c`<(Qhn@9{-4O|27~sC&gSts{5``Tt;ac__ER5xv27IG}nk9UM5`fQNESElM}B z3Mu|-x3OMmYLLCNYgMm;i?+nvVQ<2=90a~I{A6pOBh=s8E}$c{;sW3mb1b06Mo7py zN%9$6HptpFbA*9&r+Vqx&;L3#8Dvw*EK$SKCT~>8bg+i)^>lm>quXqNx63P2vx<~` z$M1^HO&shLms2K-(wG_H-E;GW6B=H`nD9XQt=&H*CkL-P+))*iN5AKnf}`D3M!%4f zxz4=~qf!yDWkV}xmkbE~ovNX{hPYb)8Og}rsI9`Qp<`lcsyIK5s+v?Qmaqo5y*RzH42yF$DS?rE`(CnP>u((8(ELRD;D{kU z+pe2;(O_-9P|IurKO3}Bv{zqm!b?fl?`zM_e)m$W)v@RoA)M1vN0A2Zm775m@O8Ds ziZ~5anzvs$GiRU39r7{yvxYl77XMjyauhO}L37g@ESG1ICjkHBrDoxfW}v5~rVnrW zp?@mgR6bn<-vO{~La2G$ni&I+l^*mm{`}O&A1QHD(81XiVXTUOFzo8(!CA=t@NJRA zcSr2Crowe0%L>`rs9A^RjgBdbjzuFm?$Y6 zVTV4s?M$ci_nda~y+?IE`J1&;M2466!pC+}A~hYFNB|f`cLY8zb3tfvPB@I5T=1eR zPnwd8e+y`@!YGjV9Z|?=t)&f$lA;Px;5HkVa@!1*LkfTT*6ff^&Qtx{Sk5V#Ky~(n zTcPIf1P9rK#_6ZJulLKur~s{&+iafZ8D7=x#R{2)16)3z`JN^)9;fKRdlgz@8*0RI zqbP}7DLx)rW^DIXd7`(bdRuVmbKCD(^I**bW+$x~@3l+QhmmH$QOgj7G7v zq_idFuHLcrOPOm7d%is5wZpB-{+LdGN}_~sQ-#qA2ecGdOBEF7iuLYjl#^MB zdJQB=^scgwXA$isUkiL%<}wk1d29JDKX(YcgbTnYr`g=hXzTOWTbQRJ4o-}Nlg$pa z!vv2MWwWfiEswmt>0_`2^HU{|FynOVFHj-gCSc_7I!MaA5(mtLKAM(mi&2s&h`y4T ze`+-1r4qv(0nPLi6@31{Q(r}TdBAT2DUGMAns~H-`Vv`s-)5oTDQ{$_E3Bi*Q1#~| zI~ciz%Vga{&MMHBn*LG-DAqk|o( zDS@!mh^w_q+OJ?hYxMG)+Yx7N0Zw3y*AA#F72zffD5foiQVw1Wb=*)gH+Nj1wE;n9 z>4cMV zQvlxC?h~srb>I0Zb{2)YreQP7XXx{2LcKB={<-XgzXUXTBJs6$A;6KxMScV`h8Ixe zbG_H@ioPFjqPK1=I2!{lxJx?)x60d?}GBHmxtP2?n)x@{kzl=xIUm5 zs*ziae9EbWsb-|Z=zEcwIhl8=2E8lK(yLWq9t~tV6E_9ppG7_aKJ?y zdA^g~6drND5Z$2Srgx7%z~q1`ms^0=Z;U2Z-H;9}$umTsEuskSqyv7h0mb*zP|7~3 zK3Kb?&#T6p2cV@I6$MzV)ag%6&M~GZA8d1PsGS?@(Jswpgi$@|<7DZaKX=^ZUOw*m zuPl@FYztZ8%INvHt`K-=_^fV7gC4aZwgbNOq)R~K_CXD|gH#0(g|s2h=pIkzNE`nU6uCWORY~+@XXiqA zCol}Taq^34%ptdjW-D!SBhHE|S7QNPxAv{>xjRI%LS(Oo<}PT6wHJ}PwqcbHCP*7# zV4qs`nwLn5&4B;a7NhWP-+#8ecQt|?Kne;!tn|3eg-t)BdYg0`*F-X2QNX8n|3^Mg-bl zIQU3qSgWjovL07ww2te=t#PPhl=>3b!{496t zOz64`y!t9~n;*8jNHDN!R0=U+#QJ8*oRxQIsF)4{?x4<4QL?YwqTD|Oe`&4hCU;pip2!d;PSv@MFsGI)w7D-P$tl_W)dhO=<$ctyLe@s zP)YIrYTwqjzIUusu4ndl!2M^!^Ni~syK`As=`MZIJt45LB2VfgH+IHM{vx|$n1eZ< z!u=-{KL4|?TOMalinNC3H}>(cGP@n=Ec54e^y<#7&&$FnNza^6LBtn9<5Q79Pfhfu z{|+F4KXmmVfVw9>Q)6dpYAOlb9szGP zA{;sv{jebZ$*emw22!UhYIO6+F{nMUsTp9t;t=={k=Z?IIKudG_00EF@|@7%_M(bm2i_T+osKhXD2VCjS>~&#lJyUCT6~Y19KEmU8nkR4c*IZGLvj z(%duZx%8sffoM1aBY;X*!Du*sN9_KvqW{fFT2~Dg0zaYr{=3;h4emwHce@I@+?2av zP{b@a2L(V6eQ|J4fTMGjhV4l_eIH8aVDV?hKT0fC6#UML=$b(>d)`KMJOZ3~yHlb+ zk6((}2oG08o*6?pF$kzk)?*G7l;Mk;m6WuYqg_o-Q*F1we#t;6uT1N)y08piiHc_> zDh2xAP`P8nV|47=^vIG9UI)V%qt$*XZWZ-43dwj};`G(o#5DMjoFDiU_`4}AqDwLY z#E%@wTPyhTcFpA#e#tJ8KBXF`mTMj6Hd5GJ6bxPxMsu8UT^bIl0PGX4U(2 z8jx996;spB$!A`6hesc0OmkkNmRK`^e)+ zvGYWCShHSuN09fBj}iq2QQu^Dg~ER2^~Yt#Wjxfo9}0Fh+145n3IEhkU^eixuzLl+ z8mh2KDS_rA0D5V~xB%lem=Ia|443X+7~S1p>Rip;xKSpF_*p^92bJmEr9!^DZfpAU z1RSwJs%Z1m`d2SucX)Ph${i699hFFb`F^iH-yj+M*7ADbVSY>6st-l1_O|sP_{~0 zg4cHO^0s;NRhO^WP^SWA@{2pk4W(tv%$$7={L%y=*?MhE5S)79d%lIGK5kTT5Di9) z#}{f#vK?FXDZr|{k|AJdnvvQo2?GstqlC~RwOxAI5NPg#*&fmGyZ`zq)<{2DP^_Ue z2txxi*>yHyJ%MzKgz9Kh?-N$~nUL%`M;I(8@184^>6shtlWPjifXIhm2v$(ADXCH3 z8ZC}NY{_;u4b-7Sy%Mxh>bCi$tkVgfiPzu@6FGM{574$yQ6}Gh z83la%2d)csniWzvLj7dok1LaE5kQ!Y;GGa+#3#6+2i@cgt+r9+fQBImxr$^W_-N7@ zyC!0h5o^%!m}~|uv@pV_VnClNRp@N66Lr%ly>;>%(4`GuOk<4g)S4&IE`{tY(}{hH zW@Ogai5|cJcdDf&$~5lym_nbgaOklYk0#ojMSXur&LS+2nd}sy$p9VpU|OzRZH~Qp z^yfNyv<$u-cWj(eDKzUOvvGsaJHA2;kV>Su)6O?{2&Nbe$a{^LB{GOGO%1?O1NotIpe9i z%bty$%P?Wz;tjNNP>v3NhRNsqK(1b2pO72Xf;kJsgULv?VJDVrP*-R|!!Y=qb#}G? z0f$Pgc^yo`&Vzg%lwT*DG9b9vJOoPchmoxbsMW^vVZ|(=Ox;z_bVC^K^)kbslLAMy zEp=iiTAZ_S#TViTk}@$GH$fv;L)=r9{|}IcF}}nuTqAW`dc^tDuV^S*8e61HGJO0V zjPlINpcguf1pL_Pm(@=@>ulYjKRF8z1nZah0!l)EY#8r5=r?}bD2nH%r~0KE)G849 z0mE$$hS#ndXKU#$a)vZCTT&JFES=^EQ^S~CTU4ae^M@bSpcl)u>a%61Z^ybM7MMG_ zE<*S@6JWLFIb!0gf(5BO1*F0Bq-{0O2&w%5;Co7Hl;CGWhcy%tWD9X z6Q*2hqIx}ynR6m78#cvB+JbNohZf11fcZZRo5m-y&}NUC?>nq8&;yfq{0#^!W8cnv zitl;9#W9G|JNSMvI^W2+^=|GJ5m%G;)7HKMqB;$T8G;L9G5lm^5aMBz0pjz@n`LOy zD{J|S2ByJT*M{xKMTWFJ;^p_+<4G$NKXz^)RC)-&l+gmM*a>Chg|tC$oosTofJ4$s z4H3B|+8C|N`xXRXyPi!~jpZ}MJk9QLlUGF#i6bLa2X~IXv9D=)M@vaC{p)#0-uXu@ zfvp>-@Sc3p3GT{uoslJ@U#gThB|P_p6-hfGgEzrR8&+MPh>%BLi_}Q7h;JZQ2}tvH zo})CdYXA|vy^RN*A;yaf?T3jy<%~Bym^?O510xGxqYirRo4N?2r4)rpXB=%0XqGIxhJ2c zb2*a?QR zVu|~>kM+jLS~u~?H>+P_LK!W$m=IE}*oBSXBB)*9N5u1ju&W&RYElnn%O?Uta;;K~ zPj<)zR~&RFq_iKB*k5pll$%-WH%9cYSO=?{fn;AHmN%+(#=o*F!1110g!|f2$ZuiW zdy_*jVDZRmTAQ>x z5LGbvTG(Ngj8S7GT>}4%07WCO9kpV_De2#_sCuJZTyJ-dt`Sk0M=Wxte}u{7+ohvs z4)yAg92}smb%+b3vdkjs9v4VJ;#k^R3VT|4jVD+=!X>01nO%W*tX6css=sO=?-!$aw`3N9Ja|K@OJ6W(3{r2-MpG@|qNUcJl)+qN2DbCDZ6Q+7xhWA&8<=S>Y(Ke((e`MI)cXKZvrQCNe96=!pEKH)u3_(_ z+gO#wo0VClE}6I0I}Fsq=jB*a?#P99$OWBO*rl9MgKsYh8;HltqO_;jfiAbqwk&e& zP33(i>E5$>OwFL6#;Bxxf)PGIM5pSpBPgdA!)TYam1#RKSWo3mX~T*&s{H%y zMcScrRX#eP>ye}gD^+PQ9*o&$<+X*pLQO~$%?AVc-EjP>0k}+~vBCttpEd%`J#X>< zl%Z{99{8e9$>*qMl~9cNq^+S_;o!HBOTUsRekB5tbO_OgY=fp+NP8(6GWoR;5oVoe zRPe;L)*Z%I?)L^*RPcl$*CjKGqTTrGL!FYXELA~5hf4Zr1+|o9D^_4l{yl7T9Tc)` z-4PnJai6I96Jdic%@4UdhRl*+Yk`Vo5pp|;42M67&&1!Z^GBz8b61Mc&MtG(qt(&S z(`CTFjblmqa?z=Wgdyvc&&q#^WOUai_-w*G9V!T(u}7;p;~?EIBb{hhmZUVE^aTY; zOg*yxoH@>u#6Dbi%Waxx=IkIQb~rm#KSb6x&pi$_8c!Ludrcj<$PKUXo~~yhwlf}h zxj3~si-dXwsD=1>8A0b)yuTl~p!JA%wM7GmPp*nVdAw=F0y+}w9p1#J#h>=SF$GWl z;PIVhLcd)bUcd00G0QdI+xLC9&kQdLCpAaO+VaTFPUkz>Qe2v0WLH*~L~AnyH*ugZ zO_!?M0-ZMqRq2Yu_~NZJDdIwPb5-691v`N# zIX4Y!)8#m|Qo^c`+qT(_FGisc0)9ooqosOr4y*BCLpDm8rPMvN=)E1k{F%xYlPv+` zig3hfe(IzP_R(x#oGzb?*IRauUDa4CBW9a(QtkqBANRKo47tM98>4X76~3!RrNtH1 zNfoxCZ4G3V2aZ6-a>QW147HP5WiXUg-++4#zQvK1kTnnI5CCm5J%?a;C&L?s)7f@NCe zMmyzk;qjJ4zpciemOkIY+S<(5)c3Mld)w>luFvQ09y`>RoX2Xpj#v3WBtziiLd4?7 zm2vPYpzG;wBkrjS<~@z_S|pBQsj~5@PS0m0q-`EEv$QPDAl;{fs}Xu^mQdQTY9!#e zF5mG+f1{C}sn94Cd>_<1)Dv;p(XJq{y+s4#hIbwOp!Kb!T#*vJ6WX@+DdubYJh>ps z?NrKEYS{f16&le!(U}fmv>ZND|YWf*eayZGS8{r}!Q-S@%}7yUq!2;M^QX zLJxX;a~;&-8c~&M#=qU-RLiw1%}<(%CqNlvwJ^;Q?kCk$ zqhZx`q0zYXWMl2r1qESHx>dtNA=ZT}bFt8qWW(ajxs5?k?pOt|p% zT6+pewA>Uv7!uj-$iAbgMlpyl(umXec z0=wTEe9-d?AC|`Q+j{2>iT^N!8TqAm4lCKqp`xDx_Qbxx9w4^imhbP5mY2EZEPjIV zHIwoMXFj3lIS_$oIfyg?5KH(&9C(i%KMQt%E72AaQO^TmQJnRFg5%4aKszJV%+}u! z`5DPo-jTJ8Ip+6*i(L1GB#*yjIcE020Xdii9b?R`zPmZ=o( z$+=9aHsAJL`n|q8V*PQ+rP7ZPP*j^A`sDhUM-MDRHDjp=%4o(=Z?ugNMe}9A&pTw z&todn{IT~pDkP7V?j&&~UPOR`9~D)=qs{|WaaDxV2uQxz2#2z-h=(xkl>hJpNcu!A zF+Z|y1+>}j_1RZQc7b>6WdZ2xJVI26S{L$&eS-D68I3N|Q$0m1D>z~Txr!{>kSTLc zpN;f4LOo-f7kZ6*z5%hAxsS}Z>EO9)61o~tOG^na{N(#Zz+iY^F0+f&1<5a1+)@Lv zg~v*(m~Rg4JOde-rwhw!KXBVc`yyUlra_J>ATYbTXh$^ltc4po1kN&!JBoceuXr@; zMvdyg*)s4o=iZ3yjQ=j_YpajZaj+)SeXi8a@sA9C6JUjr&+Tq?VIOD1Ihq{M5gA*( z2sd82X5?!??m-9B7iJOu;78&IVn0vmXBi&QT7P71TScoL+CLoO{04F;b=aYP!_spN zbYE$hXzjX~x_%V*)ym(ICj8>WE!#mv+!D4pb)U!f&2I>mJ9XsU%*N5|b_2#@eFBBz>^S|Ilt{%4&pb z3^IiYy4U_0PRe_bvS%J-V)%+vKiwlLU|qjFIPZn32PPAR8_d~<3zoPq*!wTouqWw{ zJR#`r3_dYXpH`~FN=jT_!1D+plHdG~GxmQ2#s7cQxP{A*b{iDsKdEDbbBDPpG!T$4 zRuB-DR8U`Tyi}nvKB#1S9yDOKHk^;@Qu^O%)9p4G5@cdoRA_K=SuiExxLd_-N`&wT ze^AiS^h{GzRx}Gr{`GgiX0?k#WnmqR@>dYhVB}(JBAr#&j*Y9D_6@g=jS;&_w=M0C zD?7c)FW^Z|o3k@pV$spa&1Boj-QVpUo}GXdLBBUO(srS&$AzXxAR+KQDAnzCOqj^) z^e^BsK6mp`g2{KXCpQmeF#0wzQs5CbR0G}f_X3k{{}mSD-jkB->!;=CCi;g}Y0zGm2~E8f(doTAvIKQQO{$IsCko1(Q-cWwyDlxeft zD@j%o7?Q(d>q;>zW5B#B2pVC0h@2S0=(N+TYgMj#2}9top97NB?4QZ!tPOTeRh3l& zj3aYNjp=^THs5V>@MD?V{nL&Ok`)B%?s@Rzpt#uS(61%UvZVv$Ha8M)I-i$NMQ2gdDqxtj@g{0v_dJ%TEy5;ohW^Ss-oW4>1AW%7qBqE3X1eZ2m1Y+$9ZMF|y@c0*`>t->Qkq9dSPARf{EjLw7xkf4N4 zvcPRlhZ$4LEJk;Ya5L?jTs85VjBh#QgB+(^8(D`P3P^HKs|2fFV9BWQuKkHsXlu;v z&ORpYU;2X<0go>!E3jm-Ws=rV9*E_3w0xvTu_kwsz8RM{{wYJj)NRV2p?5M?^o+f| z7hT-o5^-)}o4|ly7U4-p%+q1|b0>L-R(!{}Ac-*MygWrzbtk%3uUwPz1rpJTu5<;f z$neQe1IUVD#+rC;JAS_F==PQ4;H5>2u{~g9_Q`u;VXKemb>C1}38#&!8scgb>Lq9z z{MWn16;JyxnxWy(90nWq^W9{7jw!8)Y&CcZ&tfy|^|Zn5wx;{f*qWunx^I0G3RP~$ zSA6@sTL&4^!r8e4r7;m5);Zf>&>G*DVOi?NId9K>oNo`em<7TK)W}{VR;}GXrm90=o^4M#Gh?9UVZYgzk#H|o*!6G+TC5GhO z95DCnVy0?B=yY3*Oow(2iIDPW2j;xeP2#azmOfq58v-$SO?^qit3UiVp#oPmjD^>u zD*fD4D_zu_*gV_CNW1oImLRwnnn{PF=2+r5o(%Cl+FolOj;FBt$o}!R`q&CjVHKFA z0V{z?ALJB1c-di@GuC5HpFmeMN=hT{G;)1p2Vn1;gbqA~2 ze~KN4?PF^9Up<(CUZS~_HE1-kk~XAd?;PvGj?vde9qjL5Th-y&kBkkjnc@aTA5OpF z@t{r43`x^r{%!XuZ_YD{hw}~TWxV2lp{AZ^;Tk@DcMD{Rcr_L(Cl3*GsNxED*-`R-YSA%~9<>bM)lVSD%fZ(qP6j7PMe*p1~QoFh;cZRD(xQs1y4 zG$a+hZ4?)`5}$*~`VsG4sBot~j_MSYlay6-AXh2vhjo3eels9jDwr&t&c){}JdDw~ zlp-ENqT|F|?0BGHU?0`hM0QWIgN#snxEfWe?q{}L%5P#g7%m%Hl%o#O9LRwXRaC7a z*x>$!ZP|J3e&1HV*PlQEBzi4)>5G)3@XjGum%D1=@#RI}O@olZ)V$n2-9 zw1{u_-Oo^+t~1uJbj?m0Z%a-+Oe71<>15swnG>>hdJTH>*a0An(nP7&H99T1l5LWh z*W+l6n+*k*qorqt*{>|yAGDDpUM9MN&v(^Lb+Z70o~#;4T_m=Vr?)WABL-coLuTz8?)AB3rw&n+InRa!?v(M!SJD=^qn&cZJwEJb}>t8%4 zQ)o&82T=aSva_>q@g1SMG){x;ql8I&nPXp0=WctTJMMgl7Nq~Wt zYW6`Lx_)&+7?+0qTlG=OjIu2T5>4~wn)XB{T@!zbbfucQwt!52lj&t3JW~rc-TF@G zX^Q(`x=AHWZ&>FN24YT1U@hak)jdCPbnn9->HaZyY+)c)ETq|>O^ZPW%Lg)=OsxcK zd=5Qb&RnmScd33qv!Js=$BUWNB zhdtK%mMEzZp_B3Bv*HsCDuPA%Av+K6M5KKDLsNxB`7wJNuR>zS1G?vU7x-Z%F7aOw zj9k*ICTrk^3rtC+^McwE^qFoB6O(J#&ja4b3aIj5dZi5+KqxEzS{RP3qI<<3d1w(( zmivy(%A(%0oGLWyo`(f+V-gQmV0xm7RwQ~tk+)50jHShmpar6*s6X6UAasOtOOjpi zs{5IgMv)@~L~@f0b{Lb%O>ex)Mo7H5@-#%b3Iu`iS-JjU3nE;er{)2{92`x(t5Pl4 z5e1dKa3?HKrcrZag8J{Ri8t1!;3 zEcINmJh%-8iAO6Sa%(f-iVhSPsNgCuU^rLus8+%ZMpo@B8T+6 zQcJ-6VXQK9|7oLv-?X~m=$mN`3VcaojWE1x@E^L-wbg%SI9tuq&aL<^(Sw^@G>b#E zcPj`SmKkP+k^BJa0(*uDKr8VGDe^0*Jkr{KdARk5u7@v@5Ev#ySNWnge!`!9e# zv&Jx6pnrQ3D|$cK+e6L!X`K}zri(w2ZC^6KFAHhXybaaK+ z^=5u{qGlFq_z8}#9av&SV>0CqMVkeLw&I{`UK@g=WbdHUyZsoJZUrpzT9_1XJdbTr zej?6#;)x{NQ3W0%+k(!5u5&|_(K`5JszM|7D+Wv#QiJ$E zH0gZ^`w@%?`F&cK{yYs|px88#(0#zOhK!Sd<%Pg>#ro~aYB4OuEB!E4yk6Uvc6Dma0X z@4)SdSuSI-TzQO0J(FQ2{>*!dn`}Ag9n;6@n7l}xbCy;}7p;l4q9eVFWheL!>|xsi z#;I=!Bb~7LwgE_cl2;x-c2R-D$2`^zXlIk%FI8r>3!5LK4)zT7e4FkMUKfRr7){QE z5JxC zp-wC1A(CAkhH85aLLK2jp1%buVr}G0OZ-Na3>ZOI#X^L{@|*aHN;j*iw})RO^Rtbg zWJ^yrPnkujV47_?v&;+#o9sOjuSih;QqW$MMAG#36qirAtS_0k~lyt4-1}U578+w$`wuFlWu_W zb6PWeMh#CS`&3QOIYC|fRL(eH5Z);~{SeEQU1?DzI?XJffZYlRO(CWl$h{*!nm8D~ zaMW0^Rbu&3tXsyczG0_rjQA#+2)SlY`XN{T`RE$_DQ$Gf;da<2l*46yr~yZ2U@g)8 zV-#*A>Qme35R>m7m6(Zu?VbyktYtku72I7Kh4~0g?=#g}9XKzZ--DHFaf@$HNRZg`&K5H1NZXZ0RW~<9xb8)jjDOst3Ls zh6+BjbP?S$t_C|~TP8zNGY3>!?q8a#D@nh!CwV_fngkf(P)JTQz9SU@tfkhk$sur9 z<+#rr;Qc_!YDUKOTXno~eb3>Az50#+87mbr>&`OZ{%YQRuXaPFF}DYMA~656^mN>O zg~Ua134Zq79BYN8fw0aiWLGMbEiL^RJ-z>?PeI&g3Ap;xDt~kdQ?%K2k{E z<)VZM8sji@_E_03wnLFa&D*9`yq;9O>VaO>4m3kvB?yWZ^0g}I-i}<4UP$%5T@L#n zG}MIt0z~XjV>eh@%~%<~nAq3!E$h1!>!tw9rFeaYrwe3HXT=5k`%7-Ni1UJUeKjgI zTDCm*m01pR#rLzd$09wvr8`dg^n}CtJ+Z@)U+BoOnz}9PrtR9ex}!OcyRj1SEI@90 zai}u`bTTfwhUF02EJ~_ASSpV;>uwoV@aUYw<>~2zLrew5U~UkAfE9~x^_YD1Vpd!i zmznxePP{uUa%T|df-~cLfbp*`X)KBqm%6cQM+Ke6I6I3?QvCq~VL7V^8W7R25dm4zkR1t3yOdq5o9=mbw7ZF6c)U))nPNc#6W1)6= zXzDca5}&WMFP`*^XY(?i74m-c5!KWShc!E7p$jZsKUlUl9VIb&j4eG5#eng-8>k`IZUZte94id!xUWTUmANvFK~+cC>KhKzp(ElGw9TYN_)-OElaIjKoALo%+ zzmR4dL~WJg6Nmh>O{^bR<-gT%`Nrg>ptMmey-Qm%fXW;D05q{@5QMp*V(NL5p8>os z|FMZ|AvxlUyVry0_vVfPB3SK62>BSsw~i-y)O#(Ph5LS{U4#yrVZYkHH*I+bZwdP4 zj(EQrQRwtBg)qxbPbkO)Yz#buZNIPXalCj>wfOJMLLb}Cc#vmuOKv!7@yNx?d#FER z;{q&>mDL%40vE23*{79E_v{LO_9)e095BD?u%UC(X2xi6s+ozV9`#ej6X+9v_+Pf9 zIdzV`cq}9xOtrq|UQV!IDT-F$bZ73wc$*>s1bdSzz_?qyE{hvX@PDKz)tT_ixg@Fd zvh@92^-9Qz`yIJ=(xdVl$Q1wmqVOL^!T+5J690E7_`l?j|BMUa^)rmj|GDD!|7kn{ zsg6CkWT{%Yq=Z1#D?3zSbbtzG^RKz}b*2bd?TT6JmTI*^_A&~V0AgH{FTq~)=6*Xp zOf(;m9f^ORc1T3Mp{(2K4A?M4&6$_m9c(8#&L`U)9~UpNxggJX+)?D52oR1dy?`Q@ zUuUc4KM1s&kIZlyRRw%yDNhz52Ylx0#mYBoU|aet8MJ{~^t>F~yvH>ZJ2O}uwN|d! z>M}7K$JHwZSFK&{1x7q>F)JAmVTZOlt!c8skKnzhHE+>9Wz9QGOs{zw$U^Ok<(|?3VrQJ95&OepoSBx z84)I<4nn54Uh4gr0=0i^t}-N1k#rriF}rkVvc-_fcYG#;9oHn+l8Vv;>jEElv%C1( zSv$wwc*2q{F6>6UG%um3p)pq=^0UjOgaZMRD?X5!zwL07`BIPDfFKr|Q#`oKC3q^) zEW`O}sLT)kGcFfw;CK`jVMrM;U;YUsF$*d%(UEIl^FscfHW7Z}7AQtG5mn_}4HoaA z^qiU{2t*9gx*z{hmVB#zNw?}g>64!d(*MYQ2xViK!Z8M|?n8X+poS965^E#cAmsPE zst4F2sb1L3grW`WL%9}bx)Ea?ezeO45rVP)gH4A9zYIM(ATYp8*=$^AC}W-rd5D)r zl1cnsxHy+sT+qo8+4F#(H6rtOPjL((Ro2bcFo7ADL+~YUwT!2grZ=vlYgN9h3wFWK{+Pgy=v0iIut=#ex8YF-%%&5_p-; z7|S)4Op7ByOOW(uCi#U*9YAb^6!Pac!)||^Y`-BZOd?Dqsdk^Q)rGb$9phPpCL~E& znWk-1Q?qZiZ!KJt`chowm#C^&`R5#0k>yVqP>GtTH=VU zMxYdpO;P090Yp^X~QR-f|>5DZ6Jx3{DQA`5!|0&m?re zG>$Jvbl+iOy@L{Soi=)kBY{|g2Qj(M8%(Y5GO^qYe#s%{nVWYu^4rEmVX!@wpe#lAnkaD8Q`|+M;Rj12YKvxmKkIWyPWV>Mx0qe zc@v9~!?I4}g|(m>QWA+-w5}5p^_(JP1SGT3aJ1FYjSAr*2X_FKW;YJ5P!kmo z>sG7291jm+4QE#y8*dZpZNf~l(JjpNF4krAn0(-<4Kv@aC`OA|#(^eNwWE&4BWf2} zsfvZ!(X>NVTa)ifgyIQBpm^ZSd$XdAje)oXS9w?MkCvv9XL@)JJ2;INaqR-zh!K_` z_jA#%J5WBN2A-qgC}HO$cAbKJ4-NRL7{JADo6X8}dMFMVFZ8|b4oef*IK3XI&8AS~0C$z0kU}k(Sm(iXSV6+ZWQ}=51+VDXqaj z&pV{)px#r`W?>;jReZx%L`uj(G>0wm6VI7`iHLt8|JuWucU%&uQmunAAtQd8j|1 z(o8%~H(g9N`f6^t20J&BhWV{o@*v4NdC8E7!J3MP!bjI6V!uGr!jeuN7I9(o$0EDJ z?4nU_3GO$SBfzEoPggijWwhvktDddSKwXn824D5i?DA8-=I;yf!Z1T|A03FOlTx6r zv?m;M#@gx)eA_x|bHYaIZB7itT|plWA?g_3fPUF`8eAD9P5jZGE zOzOk%NPW)vpcaD;WCL-3k^2=xS5Q_IVw`oT*Ndj9LLrVaN+NS0sa1i(z~w)K ztoZ3=$!uuNNw;nkBCR<*?Hwr5L7s>hFvsx6kgB!a-76}wXAueSDW znTLn39#j8&DW#XWh(dJ!n(qU4>isqAqrLIUXRH#Q++KbE^Mf{+Bbe`??im2JBz(Ev zUTKs#Z51P$2{nl7-pa9AQuq(CY<9~(_6WLWuA@@VLN?gPq~*>YtaCuJyQIl|78)wR zG5&J_g`GZ0tAI`?(}h1``&oUW-LB>=P)fuxeGjKKD=n_tBxYH$(QF(zUcG@rdVC75 z&dKE-g2Kd3h`+9f7t=i0LbE~uc3Bh78N9bK?>(rbVZDEmf_P0TL^!Iq8K)Xu%#QpDB z&N|m-BY>~?F*$sh0-eF@qm^&Q+B&8dD6f1B=xVf-Y|qXQ1LI)7tF~j@FWxkT7+P@d z{b4_owl?i8;rDAGl|A>&CH^1Vo;41Khr+3DbW+KA{9fadwc=}_NZ5vLP(HueZ3viN zJ*u&(MKK?7?|MdGoI2@tERnS=Sh%OKyI!M68%EG~T3^3n;OlAwqNL)pfBjyW|MNeY z7Gwch2)^mO-GTwaI^V8GP%${bUM_+X601=;u52&2B8Erj@(?s=7jr`1Gl}otQmH4V zW9R^rTC%*X(^m%s3LOO2NZKT}1IFoKg8gn}&Yu7@2$S*;-bo#8iUQ@{4}`B_A`7#&*)G06lbc$aTE+~Us0+;M}i zCjWqI2rfeO_K<{N=V|EIzZDV*)Fdj;k! zo>CJmpZ=gyaCC}FM9boz8gt}@A0SEsMt=(d7f*4#)C+g!ym6myi%!n8gdr_3sEvWj zvXmNvi4Q>bcf{=Z(~Ydo-mw;L&+x?5_|b<$pH&;D((`P8n)^#@2qQVhI5<70cuK4Z z3f!)rwfn}dm>Y|S%3cPE6LDecb0u>I!Fp)I@3Lq=CFNFs4`wSRayYe#Tg=kWoG3F% zKi)j1QQE|Zh0~XI9;Q^F)hb1r)IDRP=@p0j#0LTSGAK1hdF7%|Ju_}2P$bi~UrssX zUWY8%7hpp;`(XE`&s4{ulg!IeCBAK)O0y)k`vOej$pbpbbcXr=lBz4h8BlkWT z)~$iI2rb0M!`khykP|Nd);WazVr&*%=1x@jU-2aJ&Bj9o^S9TC`0)-ZW^;@Gn2KSM zjOwO^l$3S;f~acnZi&RXCtQ8QztMh8{OLW;R7z&pjIu1ybFp;l z=oBsdThe1hXBDSq-L$<^;Hml2SaoL_Pf=)HiqE3-jO^NGJ?$AV_DwQdtbwJc`Xq^KY z<9=b1=c75|;2mS^OUqqiL9=_(RNO0jrpu7_WSRXzg8702K()^A2qKc=MF!y?LR`Eo zS~)xgy=V(;!58rPX1$4>RQ+Qzcnu@n2z$m&X>B7wIVMw<|L#yh9q4@4d*4u(uKl&! zGuiD9uPI-j59O|!-7qgKfLdjZFq8=djl>GZSt!#um(3EMXaN^5#msVVwugUEDP;Nr z_=FML@eg(I8SiDk3rzFCrLYBn3DVyC@-8kyOvZz0-zOF_(ZmN|Iao!9e<)})R(H|N zR}t<_iG6JGsObK@nk28c^l;z2cPSCH$$9C){F(dGCwIOiN@dBPWi5B*dUXdB1v*Kn-%5*8;?>}Qz57Z1<78KGUU&n^K$mP8(p1|<4fDMO( zn;u!a?Wp<7=zN<3oX&vq?#8>FAF`hd?`YIp$sUM%JxqAli}yHa#>kIJkn|2OP%BuV zl{zCiJjM;H1S}7pdWZQkYR~~MQhx{^frI|d3eZS6ydvvusp;$REgu!GnN~j_m{>FC z_vhJ&4y|eAoS@U#Jg*u~De2yQ`YPLl!Z;wQ%41(*;Pf%<_`l3Uy$mAaw3=O zbmljg*g5Wu{CtSf%o*Rv662^T8H6D`#I4lfw3z^#fd`)$GX32`iS}!GMC6X~dp$P0 zCeke=W8j%|JRe$ecBL%to*M*>PP!=MH$B_nUIweQDHZT09Zg zJt3K((V;>me3D3nI9MhHvMDTX?rp$i9CT?B;Eovd}eZ zjdz%D>lXvN4qO+|@8(V1CVoxZ^hKTvH*#ut|I)eBuh5mAX+YGw>^ECU5f+?hyFUHC zlvkbRDbEaE@2@+;tu^FZ@g=$PJ5B4mEVrf}Ve2v%#XrACoO zW!YbO@^#7k1<~!ISXby8^LCD8S19gt#rq`fvi!o{LsIR!ozfqJcrL-O7(lAh6|7u<{;X%#tz9;YIgyWJ;a~YKsysdH z^QVGmzVyaUmEv86-vx0W0z#SB(*HIw6P?CFiJX{l#=0E}D)fQ8@l1*l zJW@4LGZmLRBAU?n1_%bdAe}K;UmBeCLfEy*@@ z)e8LqwkTt6m8j?D&#-FD9Bc7?7YfIn62kXjU2Z~$T#s-9iTq0l8!g?F>VA_GshbLS>^1_6o8N1+ zwf)2yNmvcTe@X_0Gn^h>nI9eWTHJJG>OR}mZy;t$=!q_0mHjq|jatIQ-5^q+~q%HY$8=>fy)X@dyapq24*Y?-wgkEX0d)4K`Hpi3$ z*{*J10qj@AT^9|4atmt4=D!$NQ&=L6{4z{&f_;{ z`E3pJOMfmm9~(Drnw|*?E2P@a3_bLLvnRa5CvOY-0)Ylx#Tw?sFBVy__X!boXzV+u zNM^KeBS$m51S!pwRqgBB|RZ$m*r zij@5e!a;tjfclz)Khi^;Y6Wtr_g1ce_x=KFTn$Xt24N@wA#VYR+5{!O#}<-A0vUA% zA!6$P_=_BOfB+J+;E#JFvIiWlAnGp>%;yRY+D4W4(kWWtBMsfgfPB4eFGvq-?&fqUk(xaMSZ^gUzW=ZXBKwCfGFzltr zm} zPL9t6MTlfcSq7zI1Z>{C^0i8f$5^#_T~4U$ZyzF1-uFa{+#Ga^FR)FLAPsLLuS>Mx z1yC`9{ZKJN048V{OeQE@1@L%k2tDwhtZVn)mX5_+_7Z;+v=9%nc;@YTW}K!k&Dvsw zkQmJB{j%cBq&`u9Kf=^i0kKJ}5UVP1)tf-xH-ALi1t4}HV?PVOul%enML=+(mgerh zNE!~Nr3f5zl|JB{O{1+hig#0_RZBTmRL}E1yFgu6V;6u#@copGvH)1bT@e|cur24z zgz@+bB;<}GP_-}o=qh|b0wX0N+hF_6=hl)CtCyUJw*=aUQ*aXn!I0x?Ac=MoZLhp5`QljH<66nw)(EaQR-EQLK(0AD?RlvqenA+v+-@ zEi;9db()n7GiOKa*R1l=!dpU3V>`7qr}+%XQgH0pZUS7VY;Bq+$tMYpw?f@ z*#j?Y26E3LOGwysUoEn}9#<^@ij?}tFG5%Em{s z+Nu>-F&}=N8hqxtf5tDhH~%W z2Wn^*U>_Cc?k>|Hi=OC7*ihVtGA-7GoM+KMk|cpgElPvd)Pn4o?7>acYHlppaH~^2 z-T`V%pN;6oplnK%K*Ywjnz_vk5BQixFBj=&x;y@UZO*$?@6>z^HU3mj>Tpg`B%b^{ z=Bhlkw+#}GCYOl7gBsfB_YuQS?_}C^O3;mko3{G2}j96sbkR@4|Y*%!}wh-qw`RKnIlYS;?!p0H8bp}|} z8s^&g#}Q@DO4+$Oh*!!P01@1lbDnbzGk>2452Zd`07FV-^B?1%PwgJU&*~lmmcGqr zE+#b!4ag!)yw3Y-Be~jAb4!v1`bpF*ob;_`0~+aY+A_vsr*4Arda@|P=p1Q#!Yo}e zrjRvMid#dDBW;Y>R{ARS6l?|&8kIIBvBERYP#xi}kPQlK2CH2XTM?d}_4|FLm zZ**V9pr@dpaBd^zHFpRKzyy}$n*I-0?;PAo7j+9~VrSw^Y}>YN+qU(a*tTukwr$(a z#Q5^O_mB5}w`zA)S9SO4)74c6YwxwzBMAe{!Mr(~4g!J~#&8|tmp!ZGnKL4{9f@Q* zBM^>lCTrKR1qkmcRpcNv=N3n9g_mLr>l>`_LPT#l#Z4KJ*;C%1w~eV$;1PiNK*BUw zSx*1m*>Wc&hust$%YkWY$4-!%_m<MHsb~M3UM*#7z##a1}Ggj!IY&x2peByXTjN`M~&=m zv$nAG!i<2NPEvw{4(!P8lMX;Z$4a4wmG~-QU{wNu;;}ALi87(%pCPPxLct-ue6$PPNQvU8| ziFfrW#I^>%irBA5D>ZD3GtwPw{^29MzGBh&@=FGVuixfqgjwqYbQ`b>x4W7%@Az1Q z#K$J^p$vAhGlC0}+xKiL%4#JHN5~9g)#izZdQ72wfcPDCNB-wv=r!yvf4P_GU~Dd8 z_nLpuH4N3;2GzS`3}ji9UhEC(m9KcvrV+keokH~w>1+61tU97~sTb6_I(-ke{fv?Y z9bnCluV|;>33{GRfgBJBE%b&OJP|`hoj7fe&|{sXWvjlw0$FRPkp2Ro@6ap>r#K`FvK_IzV_sU5-XQ(D-7}ZunGVGN;V*GvQ%lgz| zfZOnv$b3a%@#U+|95JMsVA>>*5Yy-Xb;ygo%<$VVOX8YCC<$N^Ov$#l(!0KnabB<} z_=%aKoQpt^GvLdAqzd+h1N{N{@@TZV#LIvhDh^UvNL4$jU-er+--iz@`JbBpwg!s& zBtSb!kUrxL=J>$?syOvqkP56V*!Qr|CiM_gF@9q;C(|p{awBoR9=nWM$`p1%ldYui zOj3f$E0vj!-U9H2q#H72WdmIj>kM`5zxQe#Y;6Vewr<#(!&1y6lhS}AE7+?6{U&wh zowSiq^WlhsI>Akui|q1BziLPmm)ed4f`t>|{=KkuLthqN9?PV8BrlJt`jx2}6Ktk&G%? zC#22`fdxTcFhm3j?C>K~lpMo1c-kTCwuJdf$q_(3PqS@%(nCagfIv9Nx=7SIL3*B& zyB(h4n~y8fh4Z{2>;rLrATn9$)-NzDl0xuy5!F(!hP={;<$%L2$wsi$GqO{uR&Q6= zj83-EV{Ahw*$h})*G;wl4sY_2uK!fE@)D^3WF>Ag;neT$nl$eO84G5_xgf=4+a+Aj z1O=3;^-W4FE2e7;Xw*kL9!Wxvs@Tg`{K8wAHP(=0I@C<3S()>2DT&nkD z9hI5AHMNXKmlxI*zq%(}r=TnDa)j|w2tL(pOR|lNa_+Fs4`7sj9y9e6Zk}a3V&$YH zdaT@(r5jU?Snyh^4Sd(9!HdZ9q25y&Kmu@ol(d0!Ppcrf3O6;E^RK~#nEz->d+Kli z*|h-=Ecm}=ziwyyklAIwQ(g)98g_aF>{MI&dc78O0Zff2W3*;SYsSv!SzhW>VDB^om*1t8(k$V%-w`CjcKJ z4Z|lOimRF6DZ=J;BR(owVCC(loCKFf$iy7vB&MV}1 z{=UJL?hxBGv(IKg{2VmVL|KPojj*=+c#ZPf)yYVPc+_Xm4n+=tj z1nEwOIu>Axf*ls>#*(F&LCdOV7Vc2gHEbC8R*F8QaFbe4%@Qf^aP1m)k@}zjezJ1F z=mA147ZdW3C>Sbga`HH<*A-^=QewWm;Jl;c%=bW*Am<(0q+4n8%Jp{;JsWFChO+06 zK=diL^|b)$h0B6a=oG>MH@E+X!r|6v4=>*h;Ku+60`mVZuK53#1+@sb5K^Io{#Sv~ zGT4U61q}b+X>LDuVc&~M-v_{3_y@H1H3=~?@SdGVPuQTu)3#wxma^U z18VO+KN!K=>0X2W6>9IlF^sr%4Gv^X30srX7oYCH>PDCSE)0Y(WLwHmH}~;S4dQxs z?bC2}niHftX!o>G3ssVmUTSN8QzAJ!FM%G_L54<}h-?<)NiROSm5>QtirOMYl~&bI z00fl$NKtPeY53eXB)6Ce!cJi^SS@q{r{IgN%Hoj zwTT1-6vmi>C&7T0A~((jnKJA~i3mtahw|1Mx*IVw(Mi_ZCyEP;%bFM*4Mz@%1&!y& zCmUsF9ptbFc0qYxLyRLK8Vdv^S5qSY6w)Gho7nns+{>&RKs_CrLtA%j3~J{8M$wGM z<^QIU&;6Y0URsz)Fg`=T%Ra7hHS1>jF@m9bNSMLp@CA|Jvh!lrgP|KW%5%Iwv&@!o8-p^q>u=jv#SU!ct zrjFIne54M=tbK@3@g2VmA_5F)`Jx{~Jx~q1)pwD_!V20}1dnwF=M7oiVWniO4Zvf# zm&uYt4=4xk49mo*4+JLKJiz!=9amI7*!#$2Q4EsT)j7k8T$5EB7W)**q{teap{1fyGY*q^5t1;w-(dX zW8LgY?1?Ev9vj-^NQ=DueYn~vE$`uJUrLnDk1p`_#6)N&qIocz7c2k*-Wn4&TyRF> z508Mhf(Or5gMO%;eF|6rQ)w1E@Ep0_MdN8>ttSaiPIrBbsEQ#j_^8IrI;)yX%>Bc$ zzA_Yo6qdT}mXG^KF}YW1jD1er+QXQ99!q8b6%kFD0fU2UgRz4 zOdk^VPj5VPDTt&v)N#XwOa@O2=vekopo23^Am0-_n{5^taRFB{XBJD&>a}f!8WP0s z&eA3{?Rd;^S)0-D+vb)c%w^BTU`b#ZZ25UBk7#hA{5*f5(+t5Jd^!xvSFpe|Huod-pu*56oGm#(26FihR^x$>Fl=nYZ{Oc2T9GUO# zF3c8u9D&3AfCrqjl4H&;z=j1E`0x3D>@oN_iNL+occWOZMEzh?^4aZ1p5FLfDzl6F z7!AC|enfwh?OY;Rjz=dkoX@aj5#UNX<{|&>?laINkp%Av+I+?NdsEF^FNMs{33DGI zp0ZCsFL$sY#az@WUqHn@s`C7aLa5z75~Z3Xl<+Tl8U}byR(Pr~n?6}2P?n&a1?3`; zC@Po^D-I>jd;#d~jA3k*DKzps=h*LAal-`WM6F@`(#VuPLS8oaij z4Ev#XG0G=PJ~;~2s~xT9C2K>DRF(bpi#Fe^>~6k+0jq#$U~l6#1W`$>EDJN*c&~vKXIm%G5jEEcSDn{&s~niH63d4 z#X%k%x}3%FM6T57uK?ZMx6mM@{_X<3)Q z-=hH;GE2NX4JQ*@(B)eLcUNPhvAO(vHd$h>B*6(?oGXw53HR7WrVS!d(x7WGhB$B?s!6AjbIcA-tQw;XF?GbzEgAt4-ML{frz%n46x9m;!QA_TGTOwn%Y*2t zPAR0_a~ibO!gn=~t%+ za!a^@Do~fy~X#S%rlJ@8k^B-07o}+z<*ZnM6ne)$?OwHU~F<`{YrLEx9 zM-p+`Y-oBW&EjaPVWD2VHQI1amT3f`q@ZqjvN<1Zn6oiY@t^I|4`G^k9%q1|6Su0_ zdjeR~+M@3|QpUl-$(HPrG}#1J3dC_EzNw`lXXe@9f_%y7P5@?kHrj9@M;u;Mo){h6 z$;NKHX)ALB<|Lf^2O81C0mxc9(qWSuO!SjIR$i(j>aP2B(R^nWjzgyZ>(_TN5~Hj# zk|Iw*2Pt?Az8_@C{85?LN{Xh8Urc#=+VB3Iq&6f3{gahIk2eYOXn)qXV_F1-k^Ua5 z6Hh1V6vUfn2zDdFQ^H(96NXG_v(Kn2ApX|}Nl7;oE{Cb8Iv)Je2*8EJd=B~87#;K$ z^SD&KM;gv`q~A%8sw#@6ivx3R^EC1deE7_jp?a_>s)A|`*z?BGsXNt2*_A2638)JJ zdjP&eEHjvFVCYD2Gtc1G9aFFF8_TRYJHNG_G*0nweFEQbCuA2i0r75`qe}mu&~)Xn*xHE+{Xp>4_5rJ~5R^Mf7e z8MErT37h;-4-oNKb=2E8Jyi_NEf?ryV}DJY&RE^^+y`~J+aDQ#+um!BNJewCKlgGdFx^YE?REIUn66dMt&(2J(#
J6-zHC( zwut^j(kUKpF0G5M=V^`;N{)kfB9_gyJj7<1hdUexd72wYp0k}eorszf6sv%+NeKid zQU(yIU{B0M^Yfq)v8hr!PE%GmB~J-9%R(-3aIly~%?@rgvdROnT*`U#5bx>iUv>O8 z$U)m&{WfmkH~qT(uEDMmIgJOXdn%^S++MeQ)$o|FMAIRFGk4Ll^1LRE_2hiH-3Vt- z42!1=D5)9wOe?c#b4d@4U>NTqjTZ%p2LX5D5h|$zuF}Jp(j#S%)}XTR1mYB}?Z_dY zF!r?DB5LFIl>~H;FK5t`J;-(mcc1W}`|gWi1vaSXAk%ZT(X*+4oS=MUOdY{es}q;< zpm(%`5Qq1FxO#FH*}dhjk(%yTp)qCapNNd|w@tmgOS13AIvH-|4k_|PF^WVJh=5%Q zZmBiuhq1#i@Yx;<1w~uRw&BT?{xnxD-G`Or*n*CS=)orOsLTk}ujzOCC6}1_yqLA& zo4Uy7^aHzGtfiNd*_e`iDwJC;EKO0!Fr}E7;EO1BP<%PtRn$8!x2qD0jxw+trB|*m zL(H#??b@{)?7U&Pj6)!jFLl zcyJn`4&*C**j7;Q4l!9?gl+OsVv!a;hly>gJu-s%e1%q$ngnxwyF{>B>C?YpCOBWR znx@N3%%e*n(^rPXx)B=k1nEQx(sANkj3l4= z%@#{Rrv@KM`=99B&8i1Hw|0W}RKT8#6S479CG}bOqDu!~lIr!(rqCd3Sfp1n%c%Ar z>eRcDB#-L;$&s|!G-0^j&H0^p2QfE1d{XDo^WeF-q>$b=+Jprx*qx(^)~0!`O~edC zW$v15@BrDwE9-R> zzVok=+!xF%&29`zQIsnHLjD>p^ z9D=P<0q-i9>v;)KlFcl0to56z5)1yzYiG*ygHV&?*^a_xJnzNH#A2?!-e5f9!e`zP z8RMxtHuM?W#u}h0wu{(`Es^UY93CEnFz55rDz~j+?4Os27r$lcL($kvU<}&h@m;aw z8SaX$GVu+!f47j>Vaa5Q{$VE|fzK3@6kWSBLc*;*s^SOe;8gU^sDbG&xvDfO?4qO4 z-iob3f6Q@!;d*0EUm*tu_~%Dz1|_IMvSkr=ioksMHuUnbtNP*L6!gxpt=*%zGemnE zaxaCC=Y#q7Xzb!mQrs4p`t6iLe+*hru~@$MXRgzd#w)#)i7skdz6&or{THjOmha>| zh4YpPfby#(h@~fy=6Et-dIh_&!w-@h15e?k&P_@#$r#E6wE~&q90Md4f|g<$Q5mg6 z=gT;^m@#Is=bn0N(FluPi$aQh`gBs$RKy%`5Qy3-PMfx@U)G?z2g#ePvcBxYkM5`M ztgDmB_Dt*C9tKsY7ppuGn_vcJz^-WeW+2TffPPxPIoL`s*wqdc>*8dVa!pmpcZfDJ zk#_-=$r-QhcY;sxs7wDJH^MBAQgHhuJx0NiHTX(H#7#}>u~y54u>V436LcC+jvV9) zXz%MFD`r0Px?UbA=-&(NMv)Y@-5mG!;{S<{!Z6=!hb($2loU*kNX0F#3_o5s1q8e>cZ$6p^ zxeW?5m80Tv$0EsDIfpmOzs;M`&A8@R;ki6U=M4o_wFOzKpjT`C=2Tbah`BZt{!b77 z<`tLbAR@EA0zK4$uKqYQ;EjMjLi3rlw@Y)f^|^SF*=)K%2~?W zt-vuUz15~APQCtiV3E~%QIqH}HCdWUtk`qy$6p`zsHeZj=k27dJuYusIr-ovMIc5T zYp-Z{%GjXQ#0YfLhI|mj^1`?^F1{9LEmrq6E4rLa?8`?kGp~ zsM0>J`VULi#&&>SGkzLn0)IW@IJ67+z=`eAVtnlg8sjq~10WpUq)30IFpPRRj($Vk z1N=S);>W%pZ0z8C!$yl`{>93gnLmyX1zELYF(r-Q{E!Xm$ z!1PEj(NCgncnc%6?!c zjK!q^F~Gk^xz`qGi0>NF@B+Q=gwA4>rWx<}a>4VGrt2pkWfrFQhcE*V%r@+WJtv|i zPE+-#t;rh{+I0NyeX2vS1*v`H@*}$Kxe=f~KC7*J*;;Qdx)^JIY1J1hi1 z_g;%RfV+7kL=YTdar6tp)vIn8f)3YsjR*Jg-#)?iH#CCY(Vi8rx%+HDNSFIygqRQH zt^jXmOvuQ~>Waq`94_w3%4g*XP!DfkjoiI`ES&pzNXZ>+5AVc4C`6hdBMR5Z$cPSi z?JPM$h`alAz=@A}?@#XRcn=J(JB$dxjrFjnGVtu-9=77! z+kozlpNO|_!s|K<78&DpM?~j)IYd<%;_bZ@kmG)v&aYtBULqz3J!!{+6y>nR9D-R@ zG@MMJWGC@Vjec7+X+9r0*zpesn2>OH>0s-;sgx>N{A{bSTO*gbY~;~g(k=o}v#;_? zZZeb!^1v>_HimUL|89HOmC0hzOKV=VQcDiWcMh7!HdO|=uic`mbccLbH|dmgwq6xj zCkOX*F##(-i$N2rk+eO(E^4l@@5XY@=IIZG-$8LeG zjjR1pU_pFP3HKv2`gfkFGzZsjnWu#USxl-v!{NUM@toThatQFG@uVYRyX1KUgfjQJ6dGqpSB1bFW#fRkbj`A>urg>=5iS2eEjb(BL+C9CkU9ZgY~rp)%!F zu1$5^uOYAAmg;zWto}L%Xq$(7!SXC`*=kET>3I=mUCysybSiLgAg0vfB_d$zj;O++ z=LUpKaSNaIGLsY<@T`+RIBr~o^}-;DZZ0Oc&eM<5v3x*pfTA4>6V;_|V@0u$c`bl= zZZ-~uaZnx2KwG@9_znzUVfsd3fknGupRmp*vu=25BAvOrjXU@P_JV~8nSVzi1aYRN z$EHELoD&o?HL^GAu5>67NY025*AHNbBLqgGSZIc*!dcx?!1Wk`PYVeTXyW?Du(4#PJ%T@4VYb(Rl{gLtbQ|wOQ#Ck^+C*OJ%6&B{m{PjQW!PZ)%tWOLPFs&1 zrFD=h=G9CFAggXQCWh}IslaTOwD!bjtGXm5{W~}HZ7|An71@7W2ES098gw`+TpTCe z3ZqP&?xM(V!)|&!s>cLA$q(fnD8-pwQTm(V=UtsN18fU=?~~=Jh+b$0m(>B3Q+y}r zY2_C|{7bJjj{t-r+`r^-%~dynONM-yW0k+5Et!}W&@cL>UGT>W4XH+GuPP|x_bX>OD?&{wZexUkAZiY*sft;5R|XGxdOod&LSUQN0VdNTxW!Zc4fCDBX3twwdZouyhu5XY8;x2}rP$ zz^&SkY!dk1cV#U)9>cL3w#T!w(cRY_&&o;xglB&f;o(V=DhekLK(JjS4_ zb`*i-lA7X8H?-Sc14eNYPphaRS~Gbq#fFsLx3tigH7@(ousr8rh7y@TmZL#?83Hq7Zl3i zntnQ0Z0eJp;u2RNDoAI(4xy=P*|n(YYG%V0!%MQ<-Y8Gz04i~{x2n+<0(B&-lvoiB zF#kcEo+JA}+77K|m@?37YJ!g}53W`Mk%d=~J{)7-A=MA#ybyD(ZPJqvNGUQ%jC7sh z4@#zu0JgE;n3gRqRtIKc64U^Rn|r=?Rl01*l|Ph7fv8ah-b~Rda-a2k6IsS{)IiY> zC&hQ8#`S20dm`58TFju&*Qz@g`(Wh~&|*QGH1q0u7BVvZ^k^=O+UzbWl&I(ws4A}aiP)kbE~6qw0LmcJ;u)&KdlkjobRiuT-#5_vL^y?nPj+X6?ySqIKi-0+zdThgA?%80LC0(s`}eS z;yq#%&)o-T6fH@K8!jrgD=KZ}R3-ZtSKalS*D1I~>;Xzoa<}Ioif?-gp2u*`P2%DT zxz({;P*VrvyUJV_#}AU=C0B7*X{NrCP7br{d_E%o4(7ui3SF1sEWs+_*iG@K73l}m z$C;KknADpmZoI^|Rg$h@0n@n|V>7!;9gOS^MmsMuk_!S{11a(mz+X~vZ=sAIfYJf7 z#Kr^jBCRHwRt9`%24e44KhDFc{Ke>FMNXdt>$Ag*0yN-TPb*bVdnVrDy8PTPqC+>; zw~OLY-Mmp+xc4sB4KtikGp$L{xKjIC9Zs-~#~gb_AQ+p`BV=H5z;S`RG7ib#788(A zQ5CwNJt1`m`X2Q`UeNd;;`sG0+ln7DKLB$a<+aMh))WvO%Kj9Qv*U3%E z3+lb!=SjmIDl_KgfbFTB7C#w3rFp)S^jYy}DrA@D9;KK!8Li2ww&ZIrSD4nYsl z_Hx=_Mf&E;0bVk$4x1sqB3=lgqB@x=#no;LGab`|FotwTUa%KdOp2muEh}p4o5#=chr`M_m7-Gf?|Mojfg=Q{f z5Qxv?n2=8v4isV#fS`ci1JuBtH6I%X3B`QiJn|#DSi79;A+*1O&M5Q~kAqU&kv7B@ zBF%vYS;r5fkP*y%6e~x&m?cNW_%=zemJRa99_i*`3mQ=@bm0S1WlMOM z8Q&vg{I!R6Rle6E2KrzSjA=2uWDE&G_YZwUIl^You=h|vVeTkGu?{E-r92xq*nE=;f$;zMT|9uO9e|zuC9X$tVC>S{TO!%53`0xyahX;s!U*hrZh=N*!>*Pwjza%t0Ctxy9aQ{W z?zAj_mA~}(JG2N49h(vQ8I2PT)r~LVLBEaN1)u%G1wEJUXh&);Pz4UKeN$iHSzlX& z^Y!?&1cD=5EeMN+rka(U<;jE5tCN>%7Bnjj6GV4Mm)pMT+_*C%Jb%3Q6WoG|5dWY9 zxywGDg-aesQGhne5&uW_f#Hx`^jRvFmQxC~40n&D6w_8Hge^%v`!CyK3r-R>_&)#C zbp`O4^&L;jehrRZ6deHQJn^C(W{SZN`Fq=Ia3n^uowTe7Exf&2R**TV4d=@Yu-=-7H>j$J&@`#|fy4|pwgv4j6ABOWim-YvMVOJ^ z%paPttF;gR9p-b2Fz((C%sgBEg8b-&L?~!ga#6!J_il3LOcZ1!kIX^P00?TP1!B)o zh;&FCbYat7DdRtY7%xk3s8m5|NIm*luk3URV#ppKz2AfMI0w1ZhW6h9OZ%;{wRk$< z1X9GiY^UJicY-mw)#7aG%N%sHqP}ZLmg+edlvhT5r$Ei~_+kCmV6T2~KDRNG;;HXu z7g}WO+?eDuZ{ehCu*2|w@PRRqu#h6TvMvnZjbK|b6wTWJi*vxGQm#`vgTR-~hxPls z)IK&psusv`jI=xj1rWpKa^Lwr*TQI@Yw;5#5K6Oji)TZU0K2CxwwrE``JaDk;`xgp z{ZZK%Lf8%Dp+UM=@u=2eGl3h#f8PuWgjUIi^F6C)CN$pq0o-_>DXm7i*?zY#e#635 z5K49^RpJ1^{*~zsknPTB+}XBk-#vElRZsAf=~Se?U;l; zpa%)}Ct5RH#vvaW5gTTCcyd**X0?;e@(WAVrYs!IgF?*Dp#Ytej!q*uj6@0kE^zpv zv+h%+{=gr*SN>q-7=RBDVcigjtu>3i+hR4d{~iEn2!4w69pS;uOND|XbVuc`94ZTQ zQsMRsYJPs2UY=qrrIIZXb58}|*?RL{D*9UTT8gxrirP9?*e+AB01;+nN zV*@Gjf6b9^7m>Ggl}eKomgH9iaaD$IV-^T6g&bd{op82S&3I87o>}U@B=h#`5pyDx z3a9|A9d!sPhen3p!(aS4VrM_HrWC(jpc~n~J{UO~_H?T$bi*_wN~aFcbh$0q;uY?( zXVmsroh#LDghg$b@M1m4&a)PTXk?s_Z%S&_R;Vst#$Y|{uwet3|L#^+uvmoCU@R<%G)fuQ8Qq#e#Uw=2 z{GB3IzKDo;AFiWg)Rb8}zB@_Z?GGVq!A2gpJgC11l~lKNDQZf#j(;w&MZgW}a1X8` ztCTKm-HzMfm@$suxZ2)#p-Oc^i&1r=ZBvmRv1DqS&DgGR&FR+T)~+BWV!CS=Xyphv zvt*gs5d<2WLF@U4S3n)dEQɒ@(!(g)s&GGRPKK<%I`X0YNSXM3j8MB)^%^Stm= zN<=WSu$mG%l>aki7X%pZwZ%TPGK8X5`J=^2Cc(SSMrJJGHx);(%+>Uw^yVTePo=l! zo9HHtL=58_eXg<`L|S+!)PYMuK~4j_%Cj!i`z!h(zdj6vGZ#=k7TzO+Pdp6+tmaKz z0#22WqM))uVq3$vljV>l*okE~w0P6@B+BWJ@|{6}eK5)El_2r@MQbfBno!x(nU(g& z1~aIL6q34!Zv=6&;w;h-Uzqg^W~3j)(GH!z)MfD#sM0wa4>`< z4j9OIABn6)?$35EL6~W|MP1=Yh;8_FoV;jXp>=U4Am@^3^%%gjzbo-!xw5r{-4=%1 z1Q-SV?W&Q+b)$aY-HxS2Yw`nXym|x?PmO??>al-h^xbt2hO7&x&D0DaKdArBA7Qe3 zI-*^^#h_82b31GEXQ^4A4~;jI_ag~Gsy(ZU8(yb4%lo?Yvw?409zMXX+hY^)y!SPv zSI}4*)#727U5Or83x}Do4@(H{qh&LIRiD!B3j6*|^aClKqcBqBm$LwrN5A!T3@Pi7 zlS`#5F*EU!rp>yAC0gft6Gm;Gw$U;tx!zCFnss46dCzDaj zSxLZ^*zUq{26?wy>U2gFsvs$%37lWI4WfyWe`V>Srle6()4=5{aLf(|68&Vt?jN)f zmm9y^`0Ys=xugFw{Z0Vz*4V07G%bR30(IK=xeT9sYE-T5p>(x^W(fr$%+jGN-)f>H zWj8Eq+f+s<>qf}Sr$|P_zDgDi;5UiiURK3tSQP;`<@RQ=q{nVNxLQa^2;GJnp(1q~z=A@{-ii8^gukUD@676;yV)UQ zS-#b?QB|-d(}4_-^mM}?IufpZ*>6TLuRglc%R)tOxeiUo%^6$MnNtaD;d$0NO&4jw z?jIU^!xNTJDrDm;fILbp?G!5C9vopVJTFW2M6&NQSJpmHHT+!O;;FsOKSB=nIRu;`X!xqd^rROZ>r(*JMT{|8vuM_XfQY7zcSJ6b8(sbaK( zmecF9A8i+Kpe#A55?nI3Q)TwS-Jy&vdQoR6f38qgc%?7!oLAt2Dcj0`HuB$y`4K_U z&Y13bEUP!=EYxO~?Y=Bib8u2K(K|;gvB&jpL+G0HOSv9)**;>waKt5UgO4kR5T{zT zXAoUsECe`~+o`7yog-y>ay7j~0;fl(;h$zsE@eqY)O3M8@ca|_8*Y>NO;KazuCvR& zz{c((r|ttZ+eHR;Fh*>U$Df*yD>b#Y1hvCej_tYO@) zWf!0c)%vxyHFh0xnuMCG)f?(ow>wawYzJi?Cn#FI&9in8ikNd{g8lJ10Eqx9dXa4G z@=|pLbV@1c11ca2-H8XPC9XRpDD|y9=w)4oc<~>9u)5#}%W>r!`JX##p&INWOj7%P zq?JmW*2DSbre%w)YQAN+-c*}$wbXaLB}0G?!88%0u!m^tjvo|er~y!_LJr;jqw*&d ze1DG2K0Guw_q_os@p38Mut!e{#t3yp57mGOV@P}CmT5MSQl-(o0NS%5B2x&q0&XQY3+Q^79P0OM6{C0%xAP_T9c+jCGy!ZfqKsc zc!Na=oF!Ksa-&}f77#AZ5Mr+SxgFr65g)Q^=4Vd*?XuSHXCT*8M`oqU? zyr!Px@Dv)7I%fG1t+r`q?OFoR zXKUv!jK+Q`zuWvxZE3xJax#@asjwP9oGNW}$C3d2yEc?`Gm(Y{x1_w0Uk`9r=0Zb2 zc~f1MaKkVi$Y)?yU2%VODS{OYl3iG$J!Ka^{4{GtGL(# z)!fPo!|A?*BbKM+IBR@vS1eqX$)y%{O+ql__i{KWdi|0Lq6r=poJ3`}$Q(3z0KvjZ zah|gE^-;SuF^R54hb+MCmp=fap{YR$E*m~SDW^XUckuw8?y9L~#wFW%&7iw08 zjp|}tnsl`3ZEt;DpRno|&}YwLD%nz7N}$B(UR!S1Lmx?d!V#=1H=z=XC;j)~d>aS* zFA14X>wqrX(OKZ);aL9rsP};S^&Ih?A~xTlcizuTxhnQPs}H>Y_WA$2SO35F`3+R> z(&M!Y*^|G3fM`Ke?!@6pQgZpZNL$Ewu#SI${&%_M{Ee~z82-Q0Kkciu<{&^o0T3yD zUGV5BGm?x@DbcfB=q(TUME)R%|J@_t+{=oG0R%)w4+KP*Vqgq`mvS@91rDhCG4E0O z_mA&{bgE0nPR|c*oasJx78q5&H(y*!(}Kc=1PnBbg}8+|o|Hwx<8*WL*qiI*SlJl? zZy`J%IhAGfx66yG)3l8(@6+tm>$jmyl<$Mj9-3>~k} zZC^;Jc{HQa5g}Ptzvr)FWl?}FCA;xwnfB!&5^Ykkbh=}bPS9T*f+_Jel|AAl7Yj-i z+@d>kvLvS@W+P=>s^15ouAR(3qDFCBBHpla@n^;s#o|mV=|)-HQ9P7#Zt)ovCW=H; zuokAA9%w6^XYcoW?AbI^*hwdx7MyrBcTkiwPV=zd!c2MxVDmZTGFgDkX8D7JOy(T9 zW%sxQxxFFEl?l&)@hDl{ei=+Qu2*GU%|V*Y^Fx!(;(JFwYd1#-H)xjy=kA?()ZJklA7w)8) zlri)y?_U53{O-#8AlY;DV4t9#^81xcl!FN7ul8Z|iD#+8HsvSn;=3ckQB>LP_!^{P zZ=v*xh?flSDCV8{mkq*pE0!-=0tUSkVW%!;w$bFO{v0fm2^j$6TlMe`73lTNM3-;w z-4tV+$bnW-kIVs{B%ov1O`NavF00-m189dtd9&)yHR=X&@LBSKDCx#%UaQ}LXdUBz z$WSzL3^BY?&rUeaa>GAvy%sUXt_&v>La3?Fels-KCr|SiWbn#LvpS5;ad40{ky`Ug zO}U*#g`K&D)fqr@VN1Pdeq&I1&NbSupoaBuAiH|o+&IBbXB5c&EWY^oEy+e_D~cvd zDnr#hZ9G2q*ukbx9h*OBd-tuDj`uf9YVn~rF9k>)8!ng(47$Mtc|>S4DU4H#n~9Vf z@@k~lTc2ZUr4pn>S)AYNf$`6Mt;5`ug6LWsTnUVpn;0m5GFpnv@uq5(#^no1KHFo>)N(a!Sjw zj8r?9nzOSDvumSAN0*F_G*NMAV8NX}F}4;`WY}+DeF=QS!A_9R&p7kW0lW*ddXM4& zTrVSHeMrEFzd?adu3Z-5RXmP#b#v$ry5>;?xQQ(dr(vDEkvCKIJakYv$=p!qtoR`HZ?f!=6{aO2k_E<0Mi z$lHOmNT2$rs2+V{VHrw;JYbb`pwnuWn8HY8%N!tV)N8(t=^kF?q4+dwKJz%h0!#~U zP2bNzUgV-nj}mFe5Lem11m52rc_g~S4|^CnE|_Eif!d8S7uENrOLUF{IXr|kp~S`b zxYRay8iVPZBk{SY{ghbyuA+BwO`VHYUtEXvi9#aleGP0-vP!t9Qnk?CjA1b zQ7C`JlRpC??2+`Ftr(Jvkbk;U=YAkAj9&(~#&Xd&dIr1xqN{eyqPdc@2y;$LIhg?Z zHa*f zvoZeuhczWI=eQMrLET2U@Y0#M|1JsmjkRz_;faYE4=x$9*2-rsS&p_RdW3i3&4+OI z0Yv7!J%!Leb$A!axh2_h));WTbmq{mSMuY21m&juKq+0jNC}&J%rxI#mwYBHG-#Y`PZ2u6xjm!# ztTC&(>?3yi{l(z4U^?cufOCt;`BOaoy>LzBx%@*Z>v`c*x*#NOsboL#hq~t}dops! zS#x*5$=4@W{*gE>_rOHSSMe|dIW=>XU%>bfa0^aUyNG@fq=%B96EGO0u($%d}tBV zw?Vpm;q-uNWW~`%c2+)6eONN4+Q9bTD}QxO19th!KghE53QAObbx!ym^mgN;kNKT_ z1FqFxYdCwx?9QB-whSp3shtL>tp=#AQ&FSUwUm9ePTXE}iY+w8W+A^ue+KP|wBsohv*}V{Cz~B*rHZNe^I3I%}H7LJzPM%*rrA-SftyIrXHCkh*zjqOKhj^z*2= zqxjF^gJrz#CX#*iI^6(yTC!UO3tErb$w|dVyh!86)&UcuR4F^jHBPz1th5N~fm zX2r3aU9q8bC@kK7G^Bz=76WxOJ{P+Un-|TkMsd6YIDLkTTV^l)bsDxc2CZHPcVF%6 znh|Vq>oR?Cee2ocD8)=nxMv_wX=oXq2R;OB-K~feaGQSycpm_l8j$E7Cb)jNg>2Lk zp9$Y!JzMe7!W=2C2!r~l*Tl|QUg<|UB@9z9lU~z3`sue{JDi_Fmrm#ODFI@F1f9H8 zHE$n$jOLoU^*=5tS??b(isy!&wQuH&Eh6LCpnY$w?kZu-A8pt6b&jO3rT`QVTEq1x|oEBha?6%6!96;IB3V7 zMAy?PV*)!y4h1uCZqUzQ&Ar745aY)|)u)nm)rsOtl;8lZ6rbo+1xP`AG0#nlE&Gz6 zx`YfMP6=P@O+Pw?&bv<@l4;tTok4+_JM z1=W#C!x!VACi&H^Nk-0Z1Y6Zq35jEzTM`?@J-5v^ng1UElt630v(yNMBDyk#_Up8d z9!4}Wmvv<-?YAa#b${hpbAm%Q&eoM_GEvRREKEHa(>Xfrlt}$!p=`Racv8ADa;Nhh zx5v8cbWub?@gI>rAi5=_CG5k>H;;f;W_zD+gJG4D$kv@k*eqhqTC~8^x>izGdw);V0F2$!Dpg(D1ZhBsQ&!~zTt#SqzD~zY2Bh#_@NXb_ZX{d+dzMy3HS>z{qY*tmH zj!>Xrb1=39xt++O454FrcKL%U7JMrV2O|yUzKq%xYbh)}=9AAcM-fVDx9Uq_9_`7C zMvn6w;jyVRut>ViA9Wlx3($jjgVzPT$M0yScNNi!6`I&fX zrG8TRdK!=RS&K(f&jU8x3@k*Z>IPAUN^;Y`e7n}la@r7G(^AmbB` zP~#EP`-3f>Gx7*G&l-~WBXBTT@3kePOzL!(WC_i|Xn!o8QJ!ZpDwCo-*{g7P(ouwO zod^3H4!sD3c|?)_Nnpa!D@uqrYVTQOwt4tVmY5N3a)Mr!ru~q~b}J zA6}H(1CZVEpX(ts{j5*~k++0Vvx69>hQpW(CH%ZEtkaLM!D|M-PAl#uFYUzfRtUx2 z@-Tj_@PD}&P~Ko)f5=F_Da`y_B*CKwjCBJrh3cbAHp64e%K^(cuM@y5(|s!C65e%*D1jZKJ1R=8`88G0M~0lHU~FNAUIa$Qb;|_py2FfnI*7D9y2l_H|ywZ>iqNb}Xm;b~0<{t4Nb>ivEM zCLt;s357(T^+2RE8r+B!aJ_h$<2^ZN4}Yxpo6l6!^Z8U)BX5N0M@x}GOm|?c&>-e0HPZ9D0C#uJe1Ch+ z{2sd3#P=hszt8G-nEhxeFNWdEM$3C2#+Brhngd^kF#DM7B)^V#*%l^9`T9t*U3(KP z9Z3a?%gXEt(iTs&C8*H8a3Yqj-Ac12Cdj#ZkdAG)7RFl(6?JxNMKf2b?EA)SCmWU1 zM5?7CvtNvJ7E^=S+vowbi>Q={Rew~r8WnRPm2(k|W}U`!F-_!hn#L}w^znnJf569k;>RjBX|mp1!ZoGC-X5h zl_!x0YsX>j1dJD8+<@_^Jk8`pHBOw0lOCZ*F~VKwr?g-G&j*Yi!w7*8yca$xCU<6& z!*PJB?0g6dHnx~_fXdNkt$#gq5KI%UQK39*TdGrwUjFo*q(4EbeUR+7?WVHHr5y5= zk24P{AjP>DW|ONK`(ob@?$YNMR^7F+7>ZFwQ#OzLSb&6K8BX1&~kC(-!1x z3$i3ZC0nU5L8UlI$vx=Q)Iqv=&@=R`wS8{x_A+ee`*OB-nA=s^u7A?cfZ*r!oQ2?2 z$XWizfKiWAm*{2dMt^>+DCjGy6I9h!^EegMEW>ftyJ?I<+tb#P9s3<9yKjhP(vAG&_4IuxR4{}^CBrKl(ln-)zF68$B z`5L`$K^mY}U~6{Lgd53KJwy|4q@q3KUh@DQvk%irTgi!T@_(S&O-azyd(upB%4PW$ zs^DAA%j-#0^k@2u<@y|Ct0-=^1MZEqB8tTdTtW2%bnLLsC8@hT15p9}mEJUuc?+9O zf%0iMT5NWEf~GA?kf&`AO<$uvK-J4#HSII@&%#k0z~hv>K(42Ed- zZknUeYS&z>AAi4!XUw!vt3xzzhHWp+-%Se?`1KPLR3~?|n|GW? z#kj+rFx+jV@!g0t+i4_>cp>kgCA^FJ`F{Ea57Bj;pqqI&-HHBgelYDH$zS;ePGfwj7Xz@(!VTmw^-sXfeD?=NJCWMTeKssL?z;v z_z_sqqlmuyGw_zszv)9uk>GiXf%ZQsEz<6RdW^t%V_l~V(c&8|zrBklxK2q>{Vs&0 zWv)hK4S!3v(g}O0X^pG-0Xh|y++fMkwNwPzGDNM?FG~s&B(~X*1?_h&w+Y^P32H}L zQQcOPxwA9R&fRpHLe)>uckcqf@7k}y1Pbkj|qO%YzOjLWQ8{5t{ZKj+q z6sxfRyth11g%?L>AMjA7ts?7A1TgfpCFtM91Kvmld&I%N!Ut+3LtJK}} zWd=`1tm5Q?2+sk41WASes{*5x|G4-k=VMl`w&oVtbg(5?I! z-OfL!JNPvk5L@^l?SBFXcPNmSQcOeO;*S*<-LH(JgmMh+Ry_2eGLwFy%%Q#D zXhK1aq%6hSa;&Yu+G^UTtffbkv*}TA_z`6T`Y=5PF8@@C(SGI2^tf^{J)vAhPbypK zDP=c3qdZB^DlgK{l$Yo^<#qan@)kX>yiYGEAJMOsL-ZS)i(a&ir+?qtJoJ)nCcSK% zO|RJI(W|zF^aoor{mIrzui4hp>$U*>#TKSFY@6v#+gIr=+m-Z=?Rt9Gb_@L-CBS>O zA^O1f5dG7(pZ;xojy|-#LjSS7MgO(EOCQ@lp#RxEqEBoG>44o%2kjaivg=ItGL$Qo ztk}n(YLQs>CXIsJkAJ0)=)VYE*U>bLZo_ClH01*`g1>{5$y|dBrl1yn6?NAh1mj1r zRt&zTMy}Kd^7@eehYG5Q+u3c_QVu>1HQOf`Y1{?*AE1Ne1pjn8ggVRtu1!EleHCBl zDnrat^YbF*ZY0;p9C)#^4H@aZD76}uJCH}-iwvb%xeYpXFMky%Te%Y{^)_+{Uy;&p zBNbe{jcpidGzKBxZoVZ*)^R*C;^YZ~=@XEDSUB!V%LAN6qsIor(&t zUFCvfTQHdR35`OQ;yi>zSV};%`Gg#3Kc*>%=u9fcP-QH<$6&l&Wfl3&i8vq3@L=o5 zqaiXnk%s)u0{{Ri3IG5%0001KZv$CduC3-tKSzK70A^{XcI3jNumpJ(yr}oym9dKU1wbnGopLiK~4%Gs`5sBJ!x{)+X)AxTfpD_YK)<=cE8e+oN-vQ4p# zA3ZAaJdn)&W<9quRavb0HrX!u-gA3H`coW;UJ3665(nuCjD{GF<(4vqz_;DfR6EqT z<&-=Ump3`1eWn7&y_ZE<32e`$ydEWwx!fGWc~`z@+j#rnx*U7=1vT+=cMD#jRxD>f z6{=AXQtv7RR`FThs;K_cf4B?$ev}RzbNm_}XqZ!HsgtU@)R>s@WP4W%Es!+JPSY?M z1+V58<*d!4eX(6sS2#fv-y>-vfy>N|ns|tL6FKBfEMSozESc!VR|1#6tiS>n_DVVD zII?6pRPaDHG#GT*#3L*UmNX*@B;6Whl0a@Mklwmt2*TVTE)opGU7}|UmqV03WSdHl>>}C%?>5;@E4;zF zW&_W_w-{1^8VWgr?}+apNt}4v6Jf}!Bx9t$R66|%9j|ese}0MWKue|nhQ81`F0C)#N1=lAw~8YeK~(=haYr;<(U3tzhfd!Q z97HIJ)A)8tk*^ z{|Hb^2MAW{!UWm_001cp002-+0|W{H2nYxOvn%^d3XdXC|4vK*09>n=!TSOilc2I8 zlY&GNf6TnxOp<{F0&;{&!l^;SAt4D82nmt^f=U>Z0V27~fuQRRp6h+Apdy}eJs}zr zM2U)u$F91bEAFm)xvQ?buIGAi{8jzlyqSavp!@x+yLo-Q?&|95>guZMdc#ZmAKwW8 z6SUI}{vU&X6T?RFZ|U(JJ-(;MUN=7W(PKXqe-Ge802T$K2dAQ-hoquLIo-|95=IO= zU$_hr>EHM`Vf7RP?*q9Fa}M ze;n$TOT{tNcr3k*qv?#J+<3}Optlq0aWp-SiNQx68SW1s`i+nMex=(S7sbZQT zj(3UaZkQzsTw(^bD0GP;N{iiMrZ|Bfv*=MmkJl5G$#!mL8|m7^~<}M~`}X+d%CasYeqn?irM8Hbjd{w7Ovp&9luAs||6c z8_p4HTw*PyXVK$qm-v}WoZ}MbQt3Q5{7e)OUgsO)0ykVL))3?i>2VQ_wvHa_f2qO6 zgv%u~(53YDGDBP*!{>-AsPJ<)TqmxicUMvStEuTVhPc)xuA@<|cZnO^utD5N^*0f) zUr^hdWB4Sof%PE4wra> z(kBhE(+$sw0z>q;VK=2ux%dn>yhy};+9jTGiCxs|S$aIDf%Tt8aGxiTyWPOW3x;^n zC0=sF+tll2m-rR!p9!9Dff;` z{FbWUrNm%nm}uW6z_dimU|5e_WD#q>w19=12{5;!_+~>Mm&@Zqh}C7?*TY>TyYv(pZ;_ zqcq+n6A&MnNW8uV2`-adGT9{uxM9Ea8Zw1orV2$&%%!=c&n*YaLDVvxpl48Supx)I z**FV8CT?jt)Gdce5>j%wTaJ(;sq-jmpXHLHagn5-3fWYhL%Cd>f4>|dw#$Q5@lil(Zd8%73l~r!JOa@%?G?%P)%NiMU$>lD&!YxM)r>#*E?IQs((;PpA_nQDrw3LC z))3IKEiH9pD=JEh*Hj0aS`iuSp4x9m-~8gT;uFd%OA0DW%FC)2lvOO4H?MqtCA~B2 zf`L^IsR~~#oWOvUFo-Q|Y-ni>G_)=Z)U^c#$)h9be?5^}vct8kEUj2Qo?3FG(BWHF zskY+C+D@Rh5|=M}+X@E!WSrm7qT-na3rZ`iW|db|mKDq`u3A`7T2f@K=*-fB6R_cc zxdkUx%_}V^DXT0#sj{jR*`RV3gLtZ*UtCx|zo@FBWJxiD#G>Mg%K7Dsi;Joj%tIK( z^BGK1f81p)VRd6ebz3v8OzYS{Yh!(F_1FTH5X&O3&a4gA)huYLL53%0sQ+<2e>6|T zm6WNuR~44bn?Ie~(BJ-LXTN|qz5o1YWfx%IS*xQxOf$CL?0Tv&=)sJ3my*aqNumLHQ zf7NG@g41vcf$G-2Ah3_Sx~>*Eu%NAVC2pMPBNsYNCX)~hw6qpfSEFDjk_-tp)XZx{ zfn8FAZLDQN>J-$iXl$-+U0IJiEfC5pvC|yD#x1=WvHY~tQS=e;mWJ9{jV-NZfqFVc z5+Zg>X>AL&7~ZF9%EA0hq;S;-TB}zEf143VN<@#s#=1Hj$vOmlQD02)Q~%i7mXe0k zkV-X0jSa|NagA-Qs#sXpsOCN3^qN(LjrC19+tX_6YFpRNZLA48VOkVhvUvVFn9e@t5z zT+!HCYmaQSScmkF+(f-kdSWS>R9zwNTBQEalPxN@1X0!@na0v?9$OgR(%2!s1(&TZ zSkYWPLy-s9%TaP3@@k+p%kxq*g&QYgk9k|1*q>Oois z8v>`*1iFS=OY(?!^xV}k_5OQ=21|trZo3jpT zhT|m}=s@U0ijm?;i4J&hM-Q3N%&cSA!>nak2 zQIBb);i{Ig)PzI?9vaoFa2qO$pfQ4qz9CZ3gfe8SA&xaTrJL7jTK1Kq%%>_1n1S!e?fyBP!wnlps>%1 zDp5*^q%H)y2qhz7i`#}yhJLmYN!*s^YKPYkl24*8NO&|gBExHSwe_{FhFoKa_z+gb zNa(dqNGc1f@%>?S2m}&|QTxm;Ejk>UoPm>@uSA%rp)HVxg3Q5U9w{8HrFitv@$c*J z)i`{IBA~jh4#no&f4~~6c4lKUNkn4i5=F=OrVv&kh_MZA^>j+&8LAF_wS1^Cpx8l7 zB0=_+ww`c196O`7g(lTfusTp%N1Ls`D;3iMAp{BoO@V4W^c+mxe@)h;${JBB@RcowT#IDu zrG~b(kUe}z1cvwNFxg_^;^t;DOJi_@;W8)^$DdZ)XsyMJ;POVYK=l@T4AeJj%9k&1 zA&oj#O@P$1wt6~+tTHMKHEq>YjzciSLxqh^YiG~~ku-M%)@TFJ>(n5IxEyC>us3j~ z&RIueH3ZMJe^6&IKkElh4wv>Y>tV>V5QPx&)*NmC8}9!(8Vp7r*~knAIRu#-SUVFJ zt7WA#o|;W!NT@AB#o1oiaajkS-O+H0P%_+&O~K}Xy$T}Oj8JAof%1dl#yD`1@NEs& zE<3T`UDjXy;VC+xDMAd9tSfJ*!!2Vr^%7jf=Cu8Se-49s?TzM$GbKO}amCq|JWNu0 z>Wtp3? zSy4KYHW!2XH|hng)SIE^#rjwpkz&cL?gi6#5X2d6wRJU?AsNxMqA6Hyh%*hb7ePl- zC1ftQSlX(MctoviY-&|5hF%uZ+NPjw6P%5bytXCOp*TdO)U1%^(QCG-!FXycQ*f^0 ze}%XvyQkFxv1o~g1w-K*(-bZz)Ym$een~@3a1B)j*7T`z(@aBZcid?~WK8D_LbR(_ z2CG+D>>BcU42gEKI^?ItinYaV=q#R)b+{cGMVB#ZTZ-$ONR0MoijWbqgxns)(#cWY z+NKqm9OP+0!mJEWYJ4=~;xO361jDwyf2_tmW{4gX;PnTc+YFZdP}Sy#W0Ta&XtJqn zG=)^^#D*YpM(;I9Y;By4qeN^sR8;y^CHidKq(BH3yg1A`7J+1bUwRh82=I zU^CppV6(+#3CO++T7rcUJ~El5oDU=j`@BiS#$`C2!UzW?I}Phs99&Or7X=zxe|y!f zIJhnb+aWf!%h%dg3l8mGttxDnT@9`^87|l`vg7c6+%XZZQG%7`>`!}V6{4|b@lM)` zeqYh#ap~s;ngjL0R^_xD5{*;C4aogKCQ?L?jKj1yR$HrJ$e&pY(PzZWD9jrHKzWE7 zutX-l@}&$4d43f^Z3m+x;s&1?e;|dYzXpapN7;3UXhO$OKSHv$r6}x__bN$MVL6To z!@1=8MLlVyUBEF<%ON)SViVq(*4o%j@ zKg@k+d%F(yuSB~-;A_T(M;lM0P5N=8EeV5}@Lzz5nES+W5vUzag(W`9*no?b7JKfzCYP-#>CEwAoLk={CM!`cS%$4@FEs3>g zXk*U>xTjl8b}PHmWUsN`f0(R;-C?p{v0t0~c77K@llwD@Xc2nLbe~CIl1h3VR%~+@Qd&J#S zdU;{Ryv&x?01~ocMP_YFWGK7#Goe{F1&A-9-vtL!l3qo#aJ zcAD%tqV2Op+igU=E&@1^DA-L9_VD*ixm`YvFXRqWJ|UmPw{nLe*O+pr>@nq2@@Z2( zBX^ndS^1nPpU0+rI6Gj--KKnjk2K|r1nVVi%6>~5?G}2xOGJM`zHIVrdiyI{h+oql z8!d0goqiBue+!ydw9zd^1Z_=n;V6=}k?X}%)6yC)9Th1C(7HlPyL|Z!vb9?xK&19# zj?!LCKtW`AQ}Z)T`HFm%9@-nl6YtUFlKF`mzT4zp~8p>tcEvbr- z9b*@>@Sq$$+LXWH@0#*;?!rCm>~V5JkxR|m=MJtWe_u;m<5_y#$L==SJ^0|0iFR+$ z<4q(W&*8Z!><($cnDQ-NX7UI5y~KTQo09ec`yDO*Z`lDP|0+r|gGAH(%(Kri<)`xB3`9Q;4bNBx0a}MBCWP(h6i5YJlu!@BB>5vmCMMMCLSPx-W_Czmi{@JcAE5 z<$p+44dIz4Kb_X&8xmqG_)1fLOAPcKE#&udFYaktL%Ogtc^e()t<-uS2_1j4^4jLs zhTOJ@N`&85Np6ZCITIx}U&K$sGngauf7BC{v2s72##CIySDCzlq{lqlGnlFTf;E|^ zsSBS>E1-$C1YLcPYO!8dA#}8P^D~inBN858lhRmGx_ZK_Ag+}m&o;GGEzQ(?M6iKG zD@}zin*17mt*H&78bj5L8E|h0$0LW}ouV zg;n^B(DsKYR3?L2RoE=f{%FG9IFPJ_#=W)O#VFIZEAU>6YXGk z81iSPHc6sR{{(x|(2g~=f8(@#lkH?ZhBn#Mrf5@5Z5nO1nPtx8*lI@8Kl8XF>(HZn#v z;WT_3JTvUee^Ee62nb!8RMw$^+8PSErkyf7`ZcV~noWs;d^p1_`OTYm5{|< zeR+`@g;-YP-H0O!SXa@j^^L28MEBW0#@B6qu@DS2+g<_}8ZfmMN1#k7s%=(Bom*MW zm5qwae{el4#v2g2jJCX+q4*TDM)eBqj`XZUMqj#+pY2$p@mc+a5BA?ZwjSr-ywO)Wy%pAgA^Z%Qj=D3oW@#fcuAhskL*U{enKZMxv0jlWF1)fRAW<6-^(0R{ zR#>zhGvHd`%pSRWHlav2(g($Fx@MM?l~l~4e?wKL$PWro$vvVw=Wv8Zy0ctSU%Ofx zLs1d#+Ln+X3n@%gAz5JKnH&y*jG$utj0H1i7SFFLUsybUW@-5%by4;IDQu8Jk`akQ zV%ta@Vu`d7nC3ybhI=NbKL(24iDopj zf3B^iH&qY%rYpwz{tLU3)?jkDeo;YLCDs=pQx51;YadpeK3UV9PuUQ9X`m%E zLhcXxMpXq|6$MXgTj5-1Yc*74aNMCIe=rYL1tZor!RFfqWkn&zwE_#G26PYXFCvq~ zA@%D=X&Btvyta5v6S8tY4Je!+u`yQ$aOXME#z~O`vpylg6{Fw=Cmc;J)BNJX;*y29 zjuo>OR2C_tJubA8p*`)T^C8NWZ(?Ksj5;AGC@=ane`HWzBH>t3jBF9c&qd#Ze^G=b z#irY{QDQSO%t;lCD+&w9Jd2H}qc)?CXu*k~sRauQN=lU;Wqq(&Rj~j$vw}X$37_G3 zBG6INs{#*()<)%;LcJ<#*i_VDDGCC!!go{*-t6dcf5ZA=eHEsq2F2d7#dGIXE=Fr= zKJI+Pcyx4oGaM^aXb1LD%zA61e`8;zpl%;-R1gm4PtPhmlsKf=9w#8UB{Z#Yd|@=j z7Bv_Dp%(6GQh?=TsbrX8^ zR9?b^Y9>;6FxpADBPl#Tct%@*0s>8?GC2q+O`H^nmI$m8`gl00f1t%Of4N6QWvGKc z&Wv>Ou5D=z)>9mzrEON$)(7X-*45FMU}Jumfo6qOMbuZQyUU7NvMM|^jge7{j)%z# z&QlTHD^$#v6@jK(5o>*{@56+__YG#PUKwaEXtktZA#S|+#V0N(nUCj(t9%}PV_sls zoz~IsKb%q@>4EXka1OG=e^fMCP17Miug!@{Oq%4RipU5t6?}jkj4-2l5Xo{rWB^!n zfi!&KaJ(QR^4LgqPNfey-wKD-q-9D_7j(9jZCj`Acq(l&IA zwy?AB;5!1xObdise@9%4sGmo{<){g^Alw?~$~)xiJo*Qzm@@<9%2;d7u0;K!M|H5z z2R>@VgB%)%|9zF={d91SL(8R^*w(PBfm9*7F(lUrj$a<#GjkA9hnBXH9}tf|yhF#K zg{#&fL~fK1Wc%$&Om2OY6~$RJ>F^}&=f_{tlzS}dPmOp(f3NJH8wu6B5e>%!bIQRZ z_e-}Q{BtBQl#OC%0cU5kpTRLOkm3jcFkQ#itMp$$`Bb`-^*H^W!gf!yXPojb{Cn0O|2dT=Pz0J9 zZ?|23fxW28FR_=w_O?pjvElz#&4*K1l=|_X zJ?uRMpGJW>SX_=Zzh{3y%-#nOw}1w*SaXg)*DJdq(cb|{o#6KGfaJygE*Q`WUaU&# zgjD|kNb7`HzkCt~cETV#P49*be@@ciZW!_uWOl+(fA%&Q_7oh|2|5B9zL?4*x?p4{ zjG}B-e<#HFvmb-eo#3Z5yAveU=X63YmB)0#Sie{6gmG9Hza3Bs&NqC%gyX0fvjJjo zQtecjOu-Fa9d$o;_LV2q^y#or) z?}izRkYb*8NX++G^+i5UHx&EayV}8%Z(`~*d;DH~1H^W~O#e1G0cU|6mar8U;V$Tg zXK;C*foE0vE-PWBPzZ0ZQ`s_Hpk*vbDG2sQe0D$%KED71?u9sb2nNE_Fcf=c!5$a~ zfA7Ju@Fh%!uV5B@4HfVosD^J~6?_XV@Ex2D-@`?)7p{PPa4qbITj2oQ#W-{`2|Y}M zXE1#k(^r^|qZ^1x41CC9VIPZUk|i*cd08q;VMCaYjbsB^F3VsO*kD%7h9S-)St&b; zm9gP$0UN=Vpa@#Yvf0mA4*L^g+yow+f2W)M8Mly&m4V6r!v2bN$bjqv_BX|@=di!C z4}qy1Z3(V&AK_*bD)%w6wNyFUK$^<^19Ljg?^>`rw0(T{@!y{fB$u2 zQf%r>1Q%P3kMF=JSKR<&qI9$S{0Fz_H|$$$qr!F^180p4jGVX;Vt2r-#YrVyFuTX% zR;yx;s#rYt8O8M7P@03h=PJwDmD{rw2{{s(Vv-#;~byESshlFNKx@e~t?+6z0BT-y?cN;(;I&k;n-k6z+R%(%@7jw%Et^ z+tZ*$q@VdZCh9ao6?3g=ls;qaqq6K>Tjpl(%IQ&47>kntHIZ@76b9p(11rWU%)}{_ z;1p)#6z1X-%3vHTSLmC>Q*exhzCwBc2tqp6Cb)$$_TR=1*yv8cKGe&Fe~5UuLHUD9 zu+oUS;?T+n$S#{Oi4;I<_dnL|@3#8STjby3KyWM0UX1;gL_l#zK;br<pK-C$Zuk zv<*%?7)%w!poj>slE+yk4;OUaB0$+6TJsNv7eus{M@36=EgD)VVJi*>vkK9w>jy0z zk4-|$;I8P@uIx+gGon)4f8a58|EV@YF*cnR^aZdw0+kpCl^E_;6dKPx+_XWhz&12Y zb_Xm(Xzp#WXv_mx9z+mHO5vZ+&imLFY%*$#!;NuDQk@6G*@eh~>msnuj^IAr=fi2x zSRQ8&I@TUkBes925AIh*fYEH+$5Xth-FcnW4vy=9ll($%gT)&le+L1n^KUyW5wOTU zmHhsgQ}HBJXQN!w2`6{LDcx}D);_!GMwHGsp*r;oR3vXk%s1lR*@VL2_6S6diasJu+Olm> zy{H|QVT+;Z+o6U-f4-ibk&e~DMLu0%|K*Fay^@Lq#HV+|iZF=y2oN#Z8I0K6821w|2m5@`Dn zJ+=?^PJ<894j|?JYsF6e_B?}Ss^N)zR03{wpX*4={TRHWuoprxg}roNwp9fYDa7f&NLfNg{3G@ ztQD9B@rcZeIJ1|K1z(0Cs0U}G4m_T{h79)`T!A-WCVLa+qTYKldk2=X-@-EXF0RBL zT#4T+iqAq89uK*SRvIkBl_W(&!;z*diqlbA?@+wnf1!B2L-Cas#T9i2gV>8`96;2x z5cNVN{SLPRr(j{5h`;R1^+?1~5&TXB%c$vwRa4V9L6b!|V%MOxK~OEM^Ga$^pGR_F zs8_Rhfy3@z&F(!oyfKKi8xTTcXmdnk`*h;^d2lXVg7{qmm#NgE`D{o++&@5?{|#yW zcii$Hf5Hg%5gwkOpi2EuC`3BUWB-OK_8C;O&ymhwAjf|R=c1ajj(r1{V0tyC*Rk&% zjvor~e5j&k6=d1DJUGSX-$E!?wAK)#8Buq^FtsMvA$5oL-UVt+3PA2d4FDHqZUm>R zWq}Cp2Nqr`;(i%wiXg_JsdT`Rz08rlO?}P{f6k-G-f{e>D4lWj`gV?q(m8=XYOy;{ zw52|sT1_2r#cQkv4z|oGs~8UIfs*e$aY?vp#)&ZJ--kF6|3!paPa@ zb#RF7<)hRQNVzOU>~VZ_l<9RrYhQqaqfC#}*KKy^COQ}}9qzY|pSDhDqU<;_r;uDI zf2b)j)^tL1_y|%*C?A1pf*+3J*%4z5@1?-;B}$Rjlpe11IEJHVJE+gGiA!S4-2tob z0YkKh?6@-@bjGvhG9hA|l!$RO;Bhkva1PJ40iTXtNQk7`^v+mc8%$q+!!5RL+81)3 zg6y}tIAl-dV{Edgss-RGm*wy{j_aLHe>&1wJ`RbF<0e0i00*G@v?qHvDa&gXC#~&* zvwD0oR6LuC>DfE)1W$U#vh1CAL*5Sf+2S2=&f+dOH|e}>aDMtWxFG35%w$mJBFtp( zq})2pZG-h)aB&Yw=u0+$X9rx0-7X`8T6+f9>GBOQ7?UeDA`QDBUK!FqZwI{tf3Eat zyS9itouR;yO6m;>N&Wy;EM2dXtE4(ZNCOzE%9eC9R9ge3o6pz=S5qUZ>4t0aNEb-&Nj0Qbw}oqE2mE3q|oa%2$x|MR+f9w`Th})15K7+~)UX-Gnw!!U-s6>a?E?>+RYs}4g zx>vsqF7$W69fZuCKM^uraMwn-NMYgCe=;<@E_>x_@^r-I?w=ADB!wDSZGFDZ%H4Ah zGH(am+o|M0X#F<9Kt>Z8AV%>Np`>$&N?FvZGQYo z&_Riu=e7>N{UXnko@aVZulr6oCEIIG$dC2K?uOwc#Y5w4LFitSWb@XLY(A^U>p?AH zGwAIw#Ov|JB6iF2JgV|8e{gv{w}K0^d7gD9Yd>(8B{jXC{=}yFzbAu<$ArgOk#wBD z9FUzpA>Z`jOs~p|^~P?*$%*eQI{ExQ(;JI3@_I-q?t+fIxORBhnwDNehi^?@yf^+f zxFTY5UJtGEjrj@x`{^ZYQ5bvUL)d!bHbEyM6t^ri?Q0NW+{51Zf0|{ux#QIlH*O=u zdUb`yW|Tp`7;6f5{xnl)=Vz0SpR_*l<2QOc9}2$gQ& zTPYPI#B!DHk;5sKH_Hx6wUJtZN;~yglFe%)Q0l4j1So|TxWV%}rZ=(+DP`+tsq@OJnjWMSOOsbf>$BQpD#~mA>vdg;K;v!M98*(G0)C zv8RIM(;$vdhh$y=8N3Lxc`+>Ivtc=(0}Z?kT6j5}#ZQDAc_rM;7r-t2B)E$&h8}(j zyvk37Kk)#3f6Qy(Gk!W8;Hy}o7|GJaD3&3z*cjnwD?|=kD{|R|_-;LKVps7q*!8@b zZNhffLs~9>Sj*#& zXa!=Ve>O{u(&maRZ7H_<1-APow(G!mo%~VlasHV0H1E`&<=eDZc$fAj@7CVs+qLib z4xRHSbdC4u@%$-0oj>01N3 z%ep(ZCp9;0k8CfRZ@6F0U%UXZZzg~o;5)RZ25$$zviAZe%J`kZAQ{O#9I7xZ9A}IS z(PJMy%218kHvI493iT__p6`3mE%1AOW%%3r;buA#?e%!L<~3y)??rWZWaXLZ;=O6q z0i*vqoz_9?UZncm=qt&R;yZ?_x5s|*)>gaf!*hAntLB`$d)qk)zFITTiHDpSpvrcX zgB*M=!Z7!kj(+F?wfiCSVi+8nnR-jeMyiX>pcKSq(u|Jbn2(OqAa68O+54Pv>kZbu z?RV`0DK~jf$a@51)d05M8DsrL^KmsxIKrcT&>kV&`3@YnLyLxSRb)`eMK0PVho*kj zQ1{Pp<;dOYp0e9h+o*l}(6)6OfHtuC-RqKL*zp}o9V7L^141~WMBu-9|7?Wq#^*hxD~@jQfwFY+UHC{4{Po3WB_Z!aY2n@o4Hcz-+MdeedP zA~@QMVsDJc3~P1H0=3*5k{^2@P93{{u#5AbE#7;T&u_=J`?_fSth7h(D%IIWKDD~QQ&GD=O*5^Kp|EwDnW~u2tKjDXFxW14U*H>J!gX6sdf=Cdmx0}4kyV# z!^`@}HS^TZuzRs$_XIexa|!J;y$UMJ3kQm|+(f)b8YM}K-hrU}`qxN@&sWqjI>5U7 z$y#@%!>2M34GF{LTU?6<8giSCJbo1-JiMZl>T9<_vF)c^Llf9jR2gz9#CYSB+>ICR z8)S+Ya*l}trb2prrjr^P+aOr`@|!$3Q0Pg2JcMmIdw+M$!4i=Fj*ph3A1M}6!oi!F zg+8nW*1$K)FGCYKR&c`Ensq6H3;kms;rN~P=!3e_8FXQgW1`2+2?gCFR z;z`ka4?K0DV#w6&9eF=&MGud(D2cg9gMCb)4 zm0NMFqjq*!e?MEL^;`S*$5(9JE_DxJ>r)7|iftZ9Qjc@f^89O^Xs`NMz zR|Hf0-OSzsK3Bgo$t;B9J!TKiykLB-&07zuLBwx#|>@S{L2 zA$xSiPgQgyal}oxa2QwP6z-k|XSaP_WYW5oPY-pg{ZY)swoif|i$+1N3DJYm3FD~d zr>hmvjR8C^_$Y#Lcj08~w~m|na%VMCahY;)Q~9yaJ}qxT9l^TUFhqZGJM7zxch00B zRrC@i3zV1o2@r;AwPFv4CLx%~ILxFEqER-Im&`!gYg<4P&(YneMVyok1fl!K<8R;9 z^t7^{7SX!Be)s3$I?Q}azS=S82#4D1z*s|0ZUJCM9zi5qN>G&i`{z=d4q0O=(WG=gFOgp+aCA-gI3aQJne}O04n{Kk=9HPPd6DC*5$Ig}k+(z1K?IPSYtVxzclkPBrha8WS*~yYD>Zv|;UAsMW~sVT5#H+%{v~ zMt7|+H?S+pCynQSGETSc)y+>tF3jvbXQ?T@>0kBVKfOX#iFN zzQvSwn6E|z(5iX#JA+9aFy1Hk9MhJU&NImLAK+1uOJC0YTqi}e)cUc^-COb@34y#W zj3Qr9JaUep8AUe%(|SfT1noVk$NbN;H6iMTWM*dgi^pMGf1?Oea0Y}!OD>}qnkzw{ zwX&Dfu$c+AdoV&1ctd>?tQYV*0G+txb}!8qFx4Q^>LIPgb2Sf;o6Kwf+$b9B3|B2s zyG_Vrq6-z>g?gg571Lznh9PL%2&`X!?w103#2AU>P}Vc4$R4$Gx2GXtl51oebBkJm9)=9{kb$% zJUUpj6|MbQ!xYaecYOe@*bOpn2U&g^a!`d4tmJ5xd5W1JQbW@_(Dty65l}lm_5@Za zXnjZqI2VZLyCC)qUc|IP(}Tu0RjvLK_6Uv<6>6QW=_Fy6YVEDbFi6XF3VR-`QX8X` zwKjFFX*4KUYVuZv0KtqZw3dd7A(Li-jWN~;&y}t{ou^te(=5QA&r_hA?lR0pu?YhS!GJ7_b%Q>^TPJhvTu z)2zF4aj)z|^5pfsZr00K?J_rZsEvKjFi4|V|*_2)1+o&=)>E`Ida1{6_UD;y>UlzOhY{yGdE zJ1=0!k&0xEQIMvCVY5!;_@82MbJ+AXeqR8(QEeeS{dU|R$J|Sh8W0FmKMY3sPO)FW zLn`B1#zpD$C!uR0)PDt;ZsB?08GlFPA*at1V68XcGf$db!mGz~>prCD#| zSSz0CzT1r2J@gszr2z*lxekPwp*jhAvSk7+)NAE(1;v|pqclGx@K#~6&rYq`MuT>e zBlKdm{;WDsM8>Jm%YYJEjjTp*(qmPDPzjqh8*)uW*W%3jkIJ60#AsyeO&5&9@`m3} z$es)~+#7zpkNh1;l+3R)f@f~SHaLJZx=TX%EpCqr+ohLt)vIF?q5?y*=}v?>cGd~_ z65fIdOJ?cuw4dXPW9HdSrL=4X6^}Q~sk=2sqI7#zwiD1V^G?A~3cRH`9v-29C+Z{QN4jCPIzd_rK}qfI(bENqnf| zJ6z@eo)ckDUw0B^xKxvs1= z4;7j=5J$nu9=CS)U)8pZ3c6s|kq-P&*ErU98W{VTT&22|UnHS);S8kJ$SRl-b(z%; zY2ahb35(0_mcWy-PIkCv`do84ncTr!kly#2NBb^KDv35kHCkUC+js7*Z-V;azPq7! z#b4pGZ_f;W;Xx~!3nO=RBH05vvEmPTLie+F20q{?@kp+tdR+?xuEoK&eRVMqOlJvYeH90`&KseFaC;k zSGBN>%qC0FwZud73^{0H>eDIA@zD3ys~Lr_it6^tzmh~I=?ZjzV9LgB_hY}}OeO(5^|LpwZ%%#)c*`qf z_g)!WKzixUHWlha{w7xe`jJHzrJ#ETmCmFs@>R5d>QN2x&9Q&fQSbggMIKhsnB&F9 zP?4z%5&p(N<>-{5DGviA*JsI1xlNaoZ@>Vn9eus+Gz%Ki6Zl{z?R;E@f@rC)b7VPSsxHBD ze=CuGWTtLe2AA%{>fyXceM@OYsDQY-uT!F7B5cr4pzIga4zaYHzmhWM?O1T~FGVTf zm}FpzwO``Pih6~~kUlgp!)8GG{;c44O-|$sjC#qe=tls9HbE>%l{I>nTXXg{qej=M z-D5(CIe`Yppq-=RIxhHv0{zhX^iV7yRzJ~xqlzrTZ!5cNAyBc;cpE$F7@j2Yj-+yE zg3SaOFheWw)rAphyZ$KQItV*k2eqqd zHuA$`92(%z)0&oJ&?%=o16zpf{lSLx^xaqsGDkMerr0{L4ncO_s5n741s6hnJvAJL zgHNCN$m-iXx?DXmAvVT2V#&N!gYD4DkXFmHOh zlF5<4n7voG=(Z}S&>j3P~^B1$uJfE?1okcFE$^ zj<>l(Edx^3Vt&WOnscMMwmFjo*KF|wEtH^p@x~`-y29>M^ za5j)z>NzXa?F96jTl^IZ82ZJ1m(%y_>mblztt`|=6_}kCP5#6&YRm0J2y{K)he&ff zVYrdrZAd&ciUP^NG|t!!mL|wbR{#mM>0jMtsLqUhngg<$fbc9IS`zD@lW7lw-M`18SaTPoTSOAtaH0kR@@{g z*J41{8NnF!eA!fExo2JjNJ#?dNZ)O^u#&O|_xNu~;}=JgOJX}qDLr2o%0hD~{ZPQ) zpn6hYFO?{6m4JE@@DdBE9Vcs&MvIaGkTuFy!3wc%3;u2kf3Zd;y^f{5lePl+C#|xn zIIlz=)ittln~V(`xA^&EZ)lT#kR?r@q%QKMjuI;jWnhN)KHHlL8U|!r-8Sui*c?Wx z!l8tl-Sr)qs_a2uLTRorJxeADzYG9^jC9sJxsGgtjEquKF06_BW0;?I8d!NfMHE^B zht&;oAIv^ojzGn~ieKALOSL&-uzjHJ=9mgf7S*Em$9CyEOUO|wbK$u}!D zAv(P0D&$5-5=gg9$tzj=M|JM3TXBB7`Y>W4*qjqtu9%vVRN_M%dEkO6^RR&x~<~sbIxNx>lm6#s5^h=|! z=T5cTJyKF_cDM{Jg6+vj=a){&%LO&B}w=rNWa5V(UWer4Lf!`7w=)H@Iz8Y_0#FD=+bI#BG~RX1c+Kwtj*-i3Jx<<5PoYc8lYJbm$v zWUKhrWLRP5(ND-!GubAGYfM%U#S@Lk>Q&7di{^B-RE;Ssp#K!dj|=XyfwVtF5l<$e z%Fb?u zL9lQ?huF*+!(so_?oGTlqq7aL2|N zz3e9jWs7fLku^RA)puqC+zCQtU9=N*pjL54kJk#D-Ild`Dzam+F@^JL)rp_t^tzlU z4)irN_yTW~U4DT5l;z}SP*n>FwMH-l!a(b(E=JvQ?ut%8t6#3_15CuS2+x}XNeys-#W^{mmK00j>G%=LjU7is+&VpR4 zhO0bhDSw(9!m#PlxPHyq{@RY(pPJ#yPROH`TtPeAVwxJx{fpk*lQS7nIB01XFn#R;H2RwC&Ky!V zit;b_R-FUjOFDWhM*8@9a6!)@VvEQGqOo4%Yy_mV*#qehAES9tH z9Kg~qZ4b+vf0W7K1MA=K%<2lEYQjvudlx?`O_Brzq_%d&>fh+LgTZV>ke=v>>)Rk> zce#vxe*E@K$+UO}ovY{CiS{HIyLC;t{GOm$_4Y9or1KT-0#|%UL+~X1x5{LG7Cf@b zMA3yU4o7%S?`f%#Pxfznh_s*yR|liX^XjC*^I;CWSvms#cdI98>E^&Go|I$%TBUc6 z(Qj7(+!@+M7=oj?!%^%Yp2LM3HvSauj{b;$O-mi#xtqMG2H7jTDH(Baw^~!DnDU^1RtpVDUHylqiX`eFfk4*CHC$M1qd$Erp z=p}op&rgofTJE)v$6iU*%FkqPStAgDUR%#p|Mf85y@wzy;uY%#jn9=Q9jvRj4$ z)J+4xiy?0TGtHTc#}2~V&pE|Qn8x@(Do#@ z1^FF2)YN16I*=5J0AW)2D2ll<2(9Wx*0rKVs9h%fe`IfM=$#TjJh)Qdyy^LS@Dv}x z?4@|2nI6P))wUrITU+lRM|P0Akjfuu)rxFGNI&hl*Ib0DFIQpBr;aLCT|CXZ;P=;{ z>`OOH(u+?i>Hw8gaeU`?^IjQ%3?X$%eWVQs z-uwCdUcnYD`?%vWTX&WBROv>Ug6`7fzPOkS-IZURwVaa7#_*n^>ck}J6r!4IM~#}= z4VPQFEjNRvF*?L=wVACjfQFOJiPEVdp>%Q2rkQxJ^500A{}*F=Vvi`G+7o2&QPrM8 zCxGD1Rxj|2(*nmfx6uoV^xSs9=@UmQxy4@nsTc9{83EfPoCgjjDjDr1g1rU3bcc?k zyI%rAAfolViQ|JK_ zxYsq7c|nfYC(F}!w@?RMxilyK#<*WsM|nK~Q?S7ZDA5CSzkO1?IcJ++f_)uKUm8Kx z%Jt4c&#>wgQZK2M#2$|L$(4*i7})3C{N zq;OLGW^y>tl^6M-3U7tGQTv7uKE>-Z`!wWU`tmE^B`byfLVBL;v77dSWO*}fwdD
dE!MwdGiu>gm zMchc3kfqyQ=8F#ag2au*Ot&;#y!MA&2`YK77$l?TRv4!=gt(31jae1Srp17YbjD2%m?@3FQO0bSOk@IWjzgVG=5XlRn)_ zBmEF=3_S5ba063%w_U7?Lh+o1V9!mOyp4hwT66H6Jwtpl;muz(zig0E1kMd}vBX3a z5H=p*{;p>QJW;Uq-#z5gwpkgj%+LRm6(fPZEK-5nEw~~$DSE>w=(RNJ!#8BEmOqGD zjogX3U%2`4U%GEX(0*}!y_3I?KJ=j_i|$Q)^S>AzKfmG?e3;371c)uX%6&|W-AFlo zI(vJz$X+`|<}Z#O)K#=e9CtX#9L zMm*5JE=aO=c#%+}9sxCXgR$Af|YWBy?-u3?9I1!l>mLh7kUf0vq5ZRFiheElC`1!TZdKgLYx$a0$eov{cFa*Z(3D{~vqy-z`v8L5_Fyz&~}*Vh||`f0)q{YO{e;dPTTE5^@qT zQshLrh+5Rlu_S?E{=--NU`>f+{V9G1NzwO!;Q?qsxM`_6f5kA*dEPT8U);W7YvN+l ztsS!4??vwENhC*hiZ_$c)kBzYd(@L*Op`MetUy{zwgd`l;D9>5Rc-f}>(_Q09|VDt+plqc>4ep#zhMK-u9J_BKUT+?`==rZ2Ifqu;nHlU*JaRBKy)S`bEx=o+oI759$+Mb;0LVL{%g43nG zBnJV8dQNYyLBE9HdaB5_8E$eUUwWXumH~oM;Sc#!+w;5coF6T?-!XnE&>y}18cCL9 zUoFsd`zTP~A$S^YDBrewVm_V0Qm8s3F)|;*$86Fc$`D$tw@fHEGC0-ff1qgG`ubTB zq{0hRfx3nA_Zn&uYf=>rB+Dp*2}o^2b-^YQ8OI5J3z?`Z(xL=UvKmK|2&8EWMgXup za*V4EA8TrVwb z_JaQF@w4Jya>OX}-l!)N*kMH=Y%;0m;>5FPu8L8oP8;(b>tWBXBd}zd-GH%tb?cFW zDSpyq>Db;VmDl9)?Fy?eVaEBr2QsAtD~HXE90k*V(>l>&C#hG`;a~Q%-k;A;B+54w zCH~GJS6UI^mL*+mHk&ru>>Xwy2eZYnUCQYWZ2lxnd5Swib8DX&tTk$UYU<#XN_-6~V+N^8KQMI)bx8FiaX?QwC!dmPlwWmn zic~Sa?9>=9*_9cMTB2PviZs5T<0JT3o;!_!3fe;is9=m^=2AnZaN)(a^{ zeV|+#fs_O{&Z#S_F+hf;#G;;U!#@wa>eeXzxZ-{aifKvn2xr!^%q39@l>;dfHAy%A8AI;n0@%sdHt`5%y$Plnoz$j0 z%90~WD)qfJR_L4Q0u2x)CJ5#AF%uCc;z~0|hAr7;vg-?-{ZXWY31)_5{q7Icj1LGq zkuPP5pBl`f$N7tIvm@yHPZ3K#Oy z8uHQ}$S9U651>q&49t(~Yn;#hYYh+VFd)Xmj?Rzl{hR&7&>MUd(~H`dD74N|aH zNn*+UAGeT=;z$k5=s$-Dz1Kw~W?vs{*ilZHtTqq}eSVvy_2JZGV*3d_>a7yE!+eNs zVHOIrB%eS1dA^k?>(1a{g%Vfbqo68oHh%5nODB{zvj9wiVkE^~I{WxCtXQtYs$eT< z@rFk9Im}p@epGXbt}k=*^^IAKwYmyJIY&87GfT}Hs3vH0xKR^#EJq!X{SLQ)!|X<9 zoAhv;Seo*lv93ORbK(j4^B@iArIzS zmxhGQ4WQOSOf5y2Q~#pM>C+hKVWuoY>tq7YKlQ3l7;{*$pAP&?yO;jKmZS-nGY-%* z&L=ixk{Q(5-D$fCI<@=g_ga3>Q3`zj9Enp^O93-bHrZR1)XB1i)0eJXh^G?q7#oF| zz$m4ecY(*wj!8__Ia@y3K(Yo>+dK&cOk(zq(x82xs@Dd?g=rEE6)rs>ni+5B zZ-7xlpmSa%|8(;pTD=mO`Smf++fFcB4MX+Fc&l8fzf)RX>qH7$`mD^_Li93@X)bA8 zM>!HO$JR(N$~MPdEcI(IVqK_Kf2q6J;w2kN7TCS7(2jkc{MYbFTdQ((PReT3m2kUN zRZRxjm?Yo^)8$!E6!?0h6}YDltaWOn)Bq}veyApbKHjTY=;EZ>EG9A!t)G~rQWda` z%F{ErHK~XQTg`OQ^`gx>UJ_>7A&1;%qU21e1$}81A*HD8(!|4d$^o~vri1bs z%$UM?nZ@N#DMgzW`ot-AFXtm8-K^{t1yfII+~sYpRBlb+#d-N>n=5e=u+%W2>}i!Q zYYCWd5D@7uk+IkEX65TXnU%1P2f#5S^->K;HR}0u6c?Vuajr}E&+fiybMeotli~;> z%x5nj$nM%wkm{LRdBl9c94;#F6~0c0eQ0J6+RlE>?_c2?5>5=tzC8ip6+~?ZR#NxZ zgrGXQsDXI1I2VerP`s#5G9Hte#uo-%tsz$md24P)E%GSCaHg2dpvVmDq*J zPYThdfZ$%Sr~JAaV3^C0es@cL@<~L5Ok7@w&UcE2bABYEzbhl7k9bJ*hzjA6?c$cw zRy9`z(=KBja>ouScBs_kES^S=7CLbIR^ix+aW&@nLiBNm90B8w9x5QuL&5*YLPQ_% zrJ$xiz?!@SQ!vRIi~ydJ4d549VUM}fSw|Nb3exCkoO)U%DM-a(mCdEF$`++(UuK=w z*I^))Jw893tJRf5Tk&~*;6{A{MO*zsSV6OxGGvfIt->%hz!I@Dg0urLM)UBt#CZhB_4 z+a59UPIcnrAMc5h1a#)S#KeB>AKbiy?++Z@k-vU0aD4toIy01h7(s^xgzNU@?7H3@ z53&nH$~k->zXF2iG7Gs)DP}3W5K3;br*>4XbO%aqIkR(#a}W}?dm8ML=d$AQduo#R zpnDudjfg1Bc6g%N600(~xBlH3D&6T0)iAhGVdVEjZ{u6x0Q7q#XrRYgo7X}_+`zB( zB}43~d?Kh45Et%-Js~_dyWV+U#%2SO%=Mx{*i=_{jwrD+N z%Rq0e{`!lV2Zfl6B_#jeXRr%0p<%i40m7rd5n#CK_<)+~am6qOUW;ccEFQmR(Z@0z@N7haLMKhjGM4OZbt zWwInnGKJl=TPFCYvK&@h4ch>P^Hx z^-tJK5I|o`Qy$yoHc;=w7QG%zzt&wbTi}Q$jcdbzA_Q6opJ4W*yU65mX&;kaPJ=w) zn5nE@c8JCI{o_}0cl+8qF7g*_P^}4EjVR76!t;WvuWKVXc&vtTV`tF=+it4hR;Tcu-;WH{<4XcM`^thjaHEeTjSe;=6INb21j{LP z8TY&&vBu{rsO5MH;ZoFCHTM#)42SLkBJU+Hz_d#FZ1nw)N?L&5wtM-d!b5I4lDC9R zfUkJ9R%+wKB}Acd_NjPCiju~L!J|%L6RCvBxG2Vy_~{hydj6~UpoKnEF;ponn^?~< zmch@CHbCfmNZWU=@^}dG0qNj=s*Lu>3z4Lz;3cbk`$!t(r8F#@OIQh@KS{z|+P27i zFWI=2>XLFu+dM}dSc-QlSh&*S_p^bmLN#;i~- zce6mJxw?6FZ`>yGuAZoyS0}ZB`APwl!+Hb!{nPW!NbO%kKkMB#@-x>pLh|k6UN2hk zoHloSLIu?!o!XV@12;WoyOge9#5i0>UkXP74)l8IsU2SVk zkvop{W0=CdiB5Qc?DO7_cYDzbPZHn#v7LLsANa7X@K>X}Fw+@w97*ogZq%P&+20-i zgD?NTDVy-WhxHXtz5H^bU%w>PfBoY4l|u4~iBC)5!W~G#d7Q6r-d;g32UC~T$-?H1~qG33R*Xz#KKFW zG&-04097>B*DRiws+yaduRgkuyHm!9)2Q!kzjLfgvL_T--a0j~~q@M9uhsqs|olt2^G2n*gZ?fFa<|M|{rrINii~ zkNc@e!Z+lr5P8E_^b~`=ET3}3neI!{}i`sbBncy7gng*$s~=K z&!y9JXmtv0RI12ACiG9`zK}ej`_-q3KPlr(xklrp zgq`ouSo$5SN&F3M!k2_`#9T2mnzwsf8Kqd`tYl(=Q7LOz1u!AwT&$W!!mRS9l7vZ@ zs4P`6Tw@^stFbTL)Gvqkqp=nx_1BbiRE||E7C4H_pDPy21;Rxs(hJ#D!&wxiQ}S0! zG#1=_n*LTz$e%BQ=6Mz%{~QYA)_By+xO5|D6y5*vOS@7L;4^9+s?ku%P+gsCTdZj= zyHpGWwhL)=0hs)9?P%mI1c%ZZtl?Y?x?|V26V9`wB&}*ybxyNL$$_ni_^@bcSDVo? zD1T1->D7CSZeLKwyP5A&)ICy7si2Z}F3Pz&9{D(!(#j>)9 zYkSc+@UQM^?WS3NMF;1Cc@aiBfwEe)ueEC_I^S3N*W%?RQ((z)8Rdu=L#>UJ_ zZCia=8KAqmy}k(cvi$yUWvO;UTbqp`4bn1$@lxf6m+WQmzsDvR)0RpjOwAsmRw}h6 zTqgxC_9_>y)`sEe^{QL&&~yghQWh8LITY)r^MqXb@n!5aRM_E;eAfA51!(mpum&D< z8_SjgBAILW(gMPHC9ay?1kNQmExA}HuOokaYRY96DAF=q z6tEY!B<>l|MS}>)6&AvQU@U(PBub{%9m#VRW z8zjhIGuNf2)a~Ly|07mxN3<~$Zug$frOv$ zmPPxHzf_k23I#)cSqt7%$Fc;GwGv9;0=T{!gvIWpL$CuI#`dcJgG%ty7v~Rc(3`-DrqFtFH9$j! zEk9KhSqhTFYNOTuNrT*}Y3aPu3$)G;Vel&F@_VUs5eO@7A1=i~mN ztgcM)A|Rnua=_))-wvf4eScF?9>C3asUXj*?g~yp{p|OlGecl@$^$EbvbbCQ)izBj zOaIZ)+rjeR+J=U*Y07@%Cd(X6V_Uc@$S|d8Xu|+Aczfp+^1AA!zitej=FtNRy?f?e zLNi9?)A+q&@G(}s$~n#ku1cGsW}&BT-*Rh@Jp%Yjg!q4E{?W3j0t;P&y8=AYEXp(B zL11s=PcsRuCSu=BhT%(-r$mX5H{8ccnDPNeHXzZDoXrpWPIQL!Nx2I;WKx=0oHsTkOCCwLDtf zsd@ur40_6~pAe$crku}(WSHrT`!0=_mC;U4f-gTRXl1y@njaWPd&vs4;Y zrUyfab^v23R0l6@YRrxu2By87l{47d&5$1vinsSGm8pL(BdfDeJ(t1CWKuM?dmvf9 ziwLph99}0F_z;up!iEMXz)iBTD+3h=aIf4>oIuQ9b`M zqM#sBh;Lt5W4w$CC!l?!3~#Zxyt}coT~+N~bfmibh5?DLV!tDUkP{(fW8JEdGcJM! zCG{U5Zt%7Gp=ybj4G8fMNd~{7B%sk}_GA8?j0d|PQ6r0ZY=wC6eNYe_Uj8383Ti?e z{b93BjIe==R3jCt+(jId6f2DU(ZO>7vo`!Q&@XSC0K@3t_<$|g5(p2dkgTZ#@a9&b z66qq>QmNB9fyAdS841%lOL1?q(M)#cKQ}I>U?&;Olf?{|qW&qWkG`Oc0LD{s@(O6h5p`U7;kjmMYc{jgu&R~a+A_=P@hP9Lke+y(} zrJ^uM*Q;IbcOX`Ry<)-;oP@(^@2sEx(vV@qOV!dTxIVe`PZDWWfZ7T zL1md7)Syw8doi4N@*c$Vw_nHn75(CI+TW-@m!*NZre>F1Vwf_i%!Ho6W&5Kd0+7M| z!h zu9T~Anm7QbF8eV%CQ0G+8S(Ea;s|bIw*t#@rXlD;@MKU9r6109D$~)l*h;HCcmijC z*pwbp+!?y^W%f4P=AX2y#)M28YrJeFkZFH`zxD@29@q6tF37cSKf!NIVT-#9cru9`mupiuLtWH`Uo9|*gQ%7v28{)W)J5q*Z4TZKE)x31>N^}!!n z!x)q_qKps?fdk>6?CGhox3{$$cZoNSD2gq#&^Rlrz2Hz>#qH@(L`&FfZJL(7gB$=l z?p}BveMNfmbv4ffbWF=a7QBl+Rus2YW7N*T&O$<$K+`YFtK~g2&}YZI3-MtoTx8XA z_4JHI>WXkxVkB;fqnh8oJQwXV~RZ~OHgLFggVeH@_^bGjm8B$ zzgVg^3v%rfK;9zVH4ZZ>&9j;_dzoYp5pO_;>pd)1 z2z?T4$pTRB5TW%ik%5toQ11S?J-@CY%|52qy|DofU9(KC#+qpiGlmJ+w~(SQit*d> zx0W&;LrzQfGwllQkEy@~9k7ar*rT_-t={5R8iVJ3Cx#ET+vLH%6Rc#9gyp_kXdT(| zhhb&gz8?OBSH^Cwa4PCAjje-ig{;dZKxBLk#t~aaIRvA~XQ%2D&B2ekAwA z=&{2DI$bVOUx;- zG={+p2^6L?u1Ll9;l_YTMoDVJW`v)Ea9N%)dnlTMh17YvUI`3s!gCy4rR%3ClevJnI_u)8Qi1PPmNIIgTOC_PwE6Xq2I1hGoVhiO|G@V$EhTA~X(& znOGZQdqy^6J+8Qas(=%(A1K!&T z#$Sul18?%(hr==HA)>=IJ7FPTdX-G&BNjuysV-7DQ&jvP7Zv|P^#({Va8|~jK<>O$ zuanu0^SiV=NkzOxX{=Ww5)MLyR|Xo4ML(5ts|BoZmi1Zf{69jk#9%r536)EiF6UFh z;+=UFUD~GA0Y?BCXWrO7{z;;oWghdY$)vX*DSMbSG?rDPu%|9(wWdpgTUzOvcNU z9Wb6OIUL=zY4*drbNX|U+|rg+Bq(L_LoSxd>F;MI_*(;5<29bIM5k8IdbK6kF*8`% zm~BKYRbO*)S3U?^Mi6{6zN~FDlsfWVWUPf)gj1K=mSn zLG}YUw~Ut$q+f4;9ybz9FJrsA?RIYMx4Rm%nH(I*vdCP+YD}pyE?Z)+@>=IJx2EF? zyJ-L^HM%Tz9;z$mz@h+es8e&fHRQ0M!AUkzwzb(%wdi)_;NNHG9i=fd|3lV0MrYPE zZNsr`+qP|66Wg}WBokW`V`6hMv2EM7&B>d&uIIg<`+L71=UR34uHAdDwX2V=?y5d2 zx>2d}DoHtRJ%?$+$D?()Vdqo*k9%e_Bd={q_=_>ANOZW##K>v_v6F}?{O7mm;rq>x ziwesFq&y8(AEr)pE9Dq^dl7>V({LW5?zj`QljJ|3VXuH@Y=}YltTc0YbP+@zL76DY z3~No+3%m&rfr2(1+?$-S6&q5~gYdYEzDfsJBU(-P9m~77?@t@r#U|qtfsPFL2d>7e zO1~&6Nid^BDf`w%VvQq`60w<&GY|v4<*8uIf=L~#doCKyOVg@yv)A|nLrrQ|*?!*X zMXrM{!iNF4`d*|;5ndbHo(hmLXLj=(kk3MgsK1FXGuvImh?*@Ysv6?lp)3nrWQMji z>Yn+EHLYGwtBkoKq|vOc=D&_9U8N|A;krl;@-U18`OfBz9}5o3w3gE0?6{-sY;in2 z3fzQ9kDMEI?o4x-EZdCDL+|u@xcyH3$pAXPa3Tm0MpiGAi6wn7fP^1KmF#5Anl+f9 zeLyedYP2lW15jE178iRGy?ErP6Yt1Xyl9?R=>i94LV;`0Qy z%3O*nE5>I6&)-F8)d@rRT;G$K?HJ_u3Y`G5TKM-zp7u+QykLF7{{%(z(K50+j2p?U zAp(6rh$HwQ3OUBY@FFZJaAP+cHRM>S_#~;1_dAE2nH@Eh6~%O>h$DVEs}$)ljk6!g zpL3H@u&8pUs`ojG)K~*ZB@X^-mpN%@IcX?PbCwG_Er&!d$;eU^}y?fKU?12Adb5 z*E14!;8&RxETO4n{$#!FDf-jA-90vd8(|UQPpdFX@1H=dQ&9V61o`QKAX}oQw*sS7 z*KTvvLspB#G|=NC*;PYeGKfl6h!fl*F8uSNcL-mU&i0qSAND=OG@KX~;Pp|r?fkB% z#M;eq&NTILv%ByuvM}|Cb5=tj9&;iu57Q@*rMtJF4QO3v&)I=nw_`rX;{xG;UkFNH zq&189KTjd4b$Dh@i!%ty-SS4gEyWSDJM0O;Y3gC?Rp!G^GF7?mUU)o z@Pdu=i@?i)_y%t~KhFLDUm4_|cMYCYUv{^laR!8#ZX~GZkNS06 z|2QkgXAkb-OglJQpecwE&EH5^{!G_6AlndL;RT;78RgcIfHCvSx*}2-Aa7DGYi)_t zNYtoDzKUpa7i@$oAf^`JyeIL>CA(v9C!LfUDT=?eb#%imw<% zKZ$`lZPPmKERU&OeWIdim1%pN@}K`AW9)jCEyb|(ORm7VU#&+O4EI)Z;#0S&UM$+* zR>Kea)VSyQk5rD*eu)h8OMz1oEz2d9r0JSE*Nv$16L46ijLgYt_tjppRU_97oFc*& z-;bSK0Tybl*=pw&ZM0m_YT3@f=nI(px|st{SXs6Te@(G_nZvKDZsnOLDJCqXr=#jq zvU+IGKtC5D*BMsBjpAcO8j4sTkBEdxN`7ybfPOg;g@qFGT@2|aIzA-)X}8?DcI4Oh z(glQO19_oUFk;yIT-^YGqf_6x|+R*o^(Cg|CMxI*g|trX#{*QeW!a_oj0!cooL1P2#D-~ zI(~l|mKw4N0ngVyvS+;Hwzq#<;C2i$v5RA2{MIRww|NvlE21aFXXL&+8^?p}`IKlS z84yf1C%k*BVwOeTuG?GE@;oVCGA|&;AChAa15b7g5<|NUM4pdJm^+D%(-oI`%R+BO zX!^n?*UzPo5W#bdTc_O5cz{=gj|);%2`9(=!gPVh6;oA0?H+(Sw@+3)GLs@|wQtcf zycVEKoAljRA&+S4uCTjP)WqOeh_d#G0`T?2NH}8<(Pk+qwIbJSVTCZ-1Cv}-wq9aA z^!lRD?LO(1o--pa9+Qzv{gK8%%dCf?Ip(_T%C1|1*&eLfJjVcv4RV2w;I8Vq?c4U? z`+^{6EKtL1fbb~Rg1hfez{FAwcADe{Mi6)^8>d8>^n4=2#%M~@`Kx!Y0fuh~0F)aO zZ_rQ@AL)8wxr)Y=YncDx3!gRiJ9&sOU%C`ld1@b#Qdw&9y!HyqmZfEsyyb?OaUsz9|p0z4l?aT`B8~YFdPVaz)IgOaaPXp z`)3BpCm5eMNaQ;i+6y(+ZIaS{$D@wo*>fOiu0TUoX5uXytnFE!G%_EMG2q2D9WZh<&It`aY1w`l$Tx#AN;!PbhmoG!p=q=p#JJ_yyi5!x0zBB=X5j5|T7p2W z_VDTjRS$Tk7tkpgcwb=;-u8|(&Fs(i-ohvb9tpG>BffkY%t>ThHHXkVp&N-$5m`NG zcvABVeEbasdLDn$mb4=^Bw3S7n0ve#(+Bd>it%TMXz@t)_?`hM!iU#NX?W`>`DwXn zel>wya_|1=PmqH=07-*|2zXLNq|-Wid6Wk{(J4tPRgEvg6;g;Lo&43AB^Zy)8iry^ng^qJdKcTGgNMZJCZ7PL| z_9g7v`f*9a$uoYIr)ZklL;~FzC(uIaRvBtqgXl$IFiLnFwJ<1CNwF2s>iLGZy|I5* ztO(CEX}w|y34p5Vh-UN{$~}yI;@uPMV3Fdmb6UNS3K5_~&7e9c+U}(oO#lrX8&Tzl;Va5qt6xK5FHLQ%y4LIQ`51Ug zmae*KF#s!RhPRjqRbd1aBU>@G7hBqooPPF!vNl8ZM77o3T{Mlts7P`|X)Djw4N=UE z@!P4usOD2&y&tB|12B&~ztoS@w+0PmF!kHhZ zPOaJ>X#&7$xWIQ_-Rm%EH6VwXr@~49 zn*rEjgJ=AtoQb_ltF{S?w0eC%o~oS!0FyH+6?;ce zl&0>i>gSi0f`0L5a_{Q?;Y-AwrALMZwE-{#fX_8Dv~4TyQP~pUk{lM--zrhlAL=cT z^}>qQcNq>KFI7^wWvM$nYJ7Y<6bTK+MjvC%&62EA)@imit8FaIfc;<|aWo!zG#)X` zh%`p}NGNb4&9MnG!n_4e=VBTP(HUeb9kBt*Aozxg`a7O1vqSgv%*n4}{gNCM`oDaS3eG5ztrxX;obLIB z#JW+}PBks+QSdNf6BNSlT6&E!N*!c3jcm~*mL)FB2b_=+&dR?Xsg;@7BBbz4KjW5I zj7tvXVxBYwCDbQdwwwlPJ_RUffp7;u6)#Q2hDFJ^%mZH?Wm@D9>pCO|%PRYJq?O}@ zvJS~7_3OR9wFXY(Ztdl)I`6F^Q|AZiAx}HxOt%xyx;(!};5{k-{l&tuK>1a1=Px6L z6Qfp0r^Aj`w@OC*bgIVg(tuUEB03Z97|zK;Y61vtKGKI=V654eu+Ky`BCBPPUwq#Y~tn1Q^#xv(yAPt$%s->Ej@q~6>3?>qMH zg*X@R3eO3=nVvj3`CS3=6!=A9H;-?SCpZE}RQy4Yh-{{+ekb4AU_+n6j?N467a+mn zTtXhPg5Fgqrv*g>RiuLW9|FItbr4#A6LnKljO%^3FUh`drkzG7(}a2OjFU>lp^nNx zva`)sJuYCHGk~}k;EB5IKXx#B7Z%zCAO7fe!qP(MS-e-G{GytG0h5kDsNJ$nH z3*%ad{yL|m~dUVWs$MUbRihZC;iwm16|vMe*j-l+%~FZK68c?+@EV;U#h)q zdL<+y>A7@XkEtw|%gvRSkE0_-AXHBRQ8;X7)4gz$Lhyp&2%}1Hnn7__NikHG%h6^^ zlf*n99adG0t#8wn0Jo-cyIalJ4zuPgqo1bB+@Wq08l-0nk3%xvkdDMcFa>*;2dVUzDzT!V$qqM-E{O&%N~8oiikv`8*+e64DZ-# zhIIO4VwKHd9H|23kQN$_iyNjO$fnV;oP|OYAUTgwE*Iy}`6~RY)J6umOG0Mx zwomoOZZj^Ge*w^H4u2MeGTCGl=NX~uY;&nxq$Gg?$=4v|lQZU% z^#oUhC8zJfK2#dP14J%K`8vRwg_MDtMEMW}f>ag(8V{C8WrhU37eK7hY}7=a7IguH z0N}r4a6{Mv}d7y;OvnCZ#8C4$k*4OSGVDURrx)MK108^F($hg z|Nd-X{gAkjvzf9NL9q!qHZ^y-YB}=rf6970>Jt0_*+%MwHbE4CG9rp*O+&sh!gSxO z40q+6B%X`8Yetel!W7rN1#o;rZb35di)X@=z)J1w_aE46KCPNnRk7?$*sS*o3dkuPM232}&mR;S6eNM}T zwokUW;-!#PUBQ_~#i^P9_b@!Cg^j*Yd$%LYqEdOH#`m)bwTCW>bQWq@*-KAVqNBT} zmebe3;F0e*q_`%?oIV*F1ZJ8Q57x8o{`CT6G!Q%I!Q%LmWwkalxfv% z{e^!uD;)R^WN`!*H07jPiO)&1_w84dj{mT`tHunkVgE@31_TH)l||?!F^HloQj;;0 zQMMFWu;IFlei@j|K-W2#%YYp9)M@73#!g}ipChrCO@mzMc=}Y9#R?dAyf&*)bW5lO z5)G#A>BWcUIL?8y4HAK7py^+9G?7^HlAbAhWvIbS+ewkZ=Fn>?#RmVDfQw@~5x>Eb zTBSX4kT!9p#s+9rcAjZ4D@sUL&SrGh(+BOaWn*F6FEXnM7GM9t6qtlUQAf6ewwY8L z?SrB}gbUYpDH!i^QmH!>tV^bJ-J2Ht_AJ&93Z1*l3Z1(L8`{^;cR@H-P0g!Isr43H zeKmvCw_MSLj%*^(eqL%8G3!pCI#f;rQG*CeUaKf*jic!iM*IF2uFo07Y=NCG!L)n9|%&xYE{> z-&-BHo*a5kWot^bnvf73Q}$<kz2lF_3?NbM(&!D^Vi`9E0 zLD%HBfhr;spyu}y9=~AEE=Wz8*uJpSR2rUX5eeYIzsD-ET|*haLxO2czd@<^rp?>x z2%7>tP0{@dSMS>Ap5CdvkA0BuafAe&!;MqgvkENE&G7or$(*wF>o|{nP@9f5MF=R= z1bbEY--sj0V~rx;;a-_U>VanydZ-Ydx$91EbfmV4#ILG0iCYlxiXX_q{s^n|BO_l4 z6ySl2{AxWV{YL8<>z ztg-BQ`XYgNC0(N);Kia6<^`Yj#;hnVYw1QnUm9Rgd%r#?cjWK3nYduxblMomIW#IeY}%O463Q{rhU>D&0uloI;vSnj}dq`&KXQ526zO z7kK|KLjN~-|Kr~Xtl=7g{sZIo@}=KmzxaK!K4KV%6s1^X= zLZ`a>Lr#PJRZF$4W%>L^YLcFT{|@rck!X?=hzb6$XLF?HdE?@xZi{kbrf%9nLIIc= zOl*x@T(VX56w#Q`_{}iQ@g=o1x*gtu+%VlB*(6Y*pl#=E>udXkH`{)hH5E1#&Jl0d z^kR|=JcGU~5AuQvqlv&J$vODg`&?eMGzkLyK7qH%4}FszOM=@3E{$e~{P?bKkm7tj z({7nAcIv_?;Y49oWSfSSN*6h>A_4Go=y0U;Wdqi&8_4e29%#frQX*Pbw>8YqOlpr` zS&X1Ev|-nj_@u$;w;Bqee>>U>p1W+Q_l8R6_r@8kYUP>CFqOIIuWcgp$f)&A8&Ank zO{gMpEWqt8Xxl_x-6}b4z-E5cbClhz>-tLOlI@S(rcF4vx>S*c_Ty^XfdWX2Wob)e zSAA=tMQ3(THKvTw=c1e^se4V3wxin|JI_(YJkp<{l4|ha(V$3ZZ6eX#a^>fP-`=C? z)3G{jA$l+oJ|&adK$14+ovp;_X!uoNI{qswWlkOjb7ZPllwAm(l^_S)GZvHJ6j)>& zqI8?5?@ShYLg|ktdFV)#nK8iAEkXBz>05hi^>-$astbqk!RAeWfrOdK14NwLVs>gxe`g!&Zq;1$a^o<(Cfn&^6R5B zYhD58IfV%IHTSrGU6}tE*_Rpr>${D&h!~rTOcrQRms#h;uY~IC% zbA-O-oS@Bz7%J%JE~bCLGcvk3;WoK?N@;*f(KCtqtK~~~Zhn#Rls*)st_rkS5zErt z+{WD9+2zTK!RN=0B#`Wl8BiQiUOogq@VSTulA^IkK(QVTJ%DeZPO?7taW9=HM?7(l zTkswz=r~1GMjR)ofHFYl-r2M;v8PV+&m(xexL_DHr3MflcdQ?xd9Bbyxr|7y~fL@uEmA)E8@NMtSjz1OE3VbfI{d{~nefLCT2t{sx!UVx@8F3i z_B)og=;LziEuh07Wm?hXW$_;twTw_xhTbJ`^}t7T|;Re zMi~3}0ADOAj()##tO<^O?-&k7o_ZfGSGA#aL@B_=6TOBy?aZw1J|U~L&f8k0DXaQq zW+Z`a=nQAGJ9B>qj76;HoIc1}acXfLLREgU(Q?iIZehATRzml zj67R=R1yqEoHA0uE_WZs2}zYIN=(t7&2Me{RL2f)npt1OA#Md5PXNf<;6T8F$66=h zb~7Lp@g^ZiJgM9WUi3L_4;9|xIz4piz)=IiVEB8($5bi~zU_Rz@N8q51*!dL;W`#l zLL$uA)q#wPl9vU$k{X49R};CrGDu2@nH_x_q-&`Y+CdlT8`g%E8)|#mp-Z0622GQKJH=Y#+=r|@#m^FzUMGNcDmxpM779ashDXeKmC{#@%=&3uD*~g2 z>2({kD2gH>On+_HBcBciz|wDx`o3XLIa*@6mvUbgF#;{JyMn6<|9 z-lQOw!0UAGAWb>~MG2Fe*!{&~F_y-*4sL=qv89FViYhz&k!mf;`d?*5>5F|P0ds(y zg$;tS%Plf0J}V;?sqeG$Gzdd3ri0m`dv%kI7tXSR`8NLrMwcShmAdJ=Duk4zmr^xhofzWJ%Gj~oznG#m#Y`WeCSr&TeurXG({PAB#-mt zkW;H-OIcoP27GtH! z?k*@)2Ujh%U|?7_h28lsVY7s(C-Ke1>!Zogr8BGsE(ZM}FH*^C*aP_hQ;B(fn6b~G zDD?RWDtQMcNKbv0e0}K{@kR8V8nHkTFaR>d8N1>L}F|i|p*d4@T zMTcu@b5Qdf-?vX;7lt<^ciBqU#rvCknL21yVE%$OI#fal zIF%E}%&Rw;JrWW+9y8&Pc`QAi4!2#IqwO1Hh1LEt0i42z&e!=xJ7in(P7_NU#6wO#S!7^na%; zL?|F7x-S-)iAG(8FPG^@Y90YFZBrx}o&yNj(wBqPR9b>&2aNDne0c!KgM8^If~Drc zg5jqMm7zhWPDy;ZZP3pw)F9A*b+XCLG7wUggn?O_>|R;efx&pcI&ZS+Bvpb0&=uGSg(qTUjV?U*cFOCE*3kn9AF9q?~G)qNh>stUZ z61I*RRB;uFh2_*K#gtbYxIJS9_;I;hlo#t?@Z24S75WB^Rz5 zO=wANmw_rO6Zt)M;-qTwPIPudE1HkHGWwP`0Mj~4m!WjlkTrsIIVRnmKgEYeRAQS* zwl6=&QCIBT6xHDt)>Uq+t1@*WP?Lk2TBM1}KvPTFCxR_`3TMAL#S^=yyTVn07^$=K z<(9FoMCgsVV0+`H^boaxi6g<$iSzC2sjq!g8)Gl-b9uvIY-=^+LBX)fZQ|3}yKDU_ zypkh%gs%*~L1qU(mPdopqEjEq;ypu2OlblgXX+b^3>{~-Uq@#oK1j1@>Ko{WZ2&VB z8eMZOHmF)D>q)U^Q_2D}ZHmTB^4%n9M+0=w+6@nLa{7*knOc#PDmZVA6~9Ke(KZzm z@~d1oE0GB-Ik7AD+A{KQRKX`kZhbgSMF5Nzjjc;eHJLv(Q>YE4QVCD{+u@vOpXyKI zQg}rE5ZAY!@q!L4(Iw`LqtnpDy-S!XIMIv=^$c8|lm@#8!_7u_)&m^#AK&uXoZS(I zIH1}~df;Z2{A|wDwY+cbB{)<}Fvo3%5hv67jR0gW%BEv|MFXZ3PcR=3G&*qh_J9`h zt{XUICtYCqhl-67+475dY^bXJT=KgUIQ)}}Mk|#D4fwJ!l~V*l#q+y&+_+8pVBD|r z09Km$%!#G)x}$0x{gve9S8;T@%gx#hF29D6_^%iy)2zQp#Uw0EJcfUtzU#$h!54;C z-=S3R`nk>xh3r81S!En6)_|;b$OAG&XbXP7JR&dGbhhQ1MmgYh%O^Us16@rM6e1dj z=N~iq$Ss=Jf3WBa&#BeYK7#P0EE3pEEdQ5{;b9qWL6Q7A~`C47*I*7OYKSE1{Es~8oH?i5>ll?61^5bju8 zNP=hQS0cPWwFlXt>k5i2<>vPVZ0&&XNF%70kREqawO$jrVf5I&4lu`B$q|K6Ig8D( zAU_8S{32if9*FhG9CCC&@Ik=(jQvYMwvGmsJtde!Zg_l20=jKv$zC}W=-?gNB$-mm z3#@Ba5X&@gr?In9coe37?wD0+9sg4RZIr=w_ja=gkk-E8A3Oc|*XRCU$oOxc`yYtm z(#KPW_!nZ9>(Fw65&pLJm>Jnud?8lh7de&@I)2I{A3;;=C;T+XzpxYwf?xpx#{U=P zUceB}K)@LOBGfS?f*|O>A}uU}Iq?68-H_m*|L8WJl5zq6dsh=Og5Upi;E@OX27>Un zufX%s-0QE0fk{vDWBsgUM* z;GTv&rnsMcx47Hh-VD0lVYcbdA)Fw-fnR~|h`NKI4&M+@g@F)iH21KBFManUtN>8f zhN^eA_GogVt;3N-S4Qgi{s};AQm?F7R&olviu7Gps~sTXmY6UN#~{Qqz`|a0d}JkW zvf_T@4u$q#w8k=HiN}?{b2(!ctwn6aqTp|8v{^~dUvO?w9a;5&YuX?ySs!-lv|M0~ zJ!v&CD-Ttzn=midtw}o=-^<8esRFF%bWFFPN0J@U4>@aM)ko~DbigF86Z_k^0*e+} zavy56HCk{q#qjGeRj0!H3|VUXZh@0GStD0H1NOb4avzm3wR&_{Bai1qzEE6hB^W!@3th<$v! zGW46k_|1;!x=Hi13=M5_4RJ_AP@=@t;2PXqL1axMeVZ`{_0Bzr!<2|_ABVs$=x?E6 z(2)2LE@f#(&xvu)?_pISPtO3Yuid56Cdm*q6lFfQE!QR60`VVQ>}vkNhoO8vw;^G$ ziafY6p(kosDT8cJH#w&)h1)I>zrT14ah7mDEg9)&$M)emT*$<{A}rAl)KTtqJ&Iog zjdB(R=!D4E#C3`G*+n`-b053g|G>!s5Vk0%wgrO=OAwH!=fdJ2ibMfaf-px*R(!DO zgW}E^lEJN@SV%2~n@laK!qlk3TxW61<0V8@t>Gltvi{UMUw zq+e?#syd)NvZx?9iD!VaQ6zp5Jlly3qb=ZKjyVnjg!k~K`;Ayf(txg0*qA zV45b!{j$a<>^FyI%n$nV@iT};6nfx8U5@Rm;&GCafMoW_ybNlIJ05Jl|RF?oB| zQf!ghp~#FqM{PHE$n{5uP@$YB-JM2dt*5R@$820p&z!u207r@(7k0mw294X3L50eb zcCv9EwEF6xZ>BDA@B@LGTpag#^#X_vdKQwTng@p%qp)JF;QJF8c#FX&yui&?< zER-Xp_PjYJcWX|zzwi@v_A)W5Rf0)nH(Bi3fd+jKqU{9(Z|BtxJgfQeRPolaS`SCNq`g(^?h zQ?u9_pi&mnik@bf!Z);gitd@>5h zW5JNQSWex>@O`0nWkw>OTX~)ey3v(S8)^908M!h>0bCV$Uf=Xmt=9?-U@0mkC2rd; z)P|Rb$iq1RKM{f2rq7tG-_g}#u+-gIW2G2%28nXbPp4YY^IKbfSu9HK-mYY-d#8?)SAWn`C$#sFT9`VBS%iz#5w20EJv-qfOMu=|2Ns0~&R(<6}w zPSM^Vd$RRwV9>X^@ht~H?nW1MvhXKmPsEy)Fl48tk4&;@OsJ#+>IL#3lYL#9XisP! z%47~zIE(sB_IKy2M)+otGBbUMPnG7}F1g-vu?VW|64sgnIA<5_TYxpU5m|j7cpIDK zrZ~&rj=q$#|agYZd)=F$Ju-py94nD74-)})h41VtEG*I->e+utI ztoma-Y8Q66dyAxt$&DB`-OByg;P2e;DpKA@&GxEh(ii0jmwhB)Eg>qP0a?}S!E#GtV6LCfnpd~JYsLZ8Zcn{S9D}AEO z7rKZeFdywFz-Tuh315IBGxfl|dq>wIum$#&Gct=*LMl*FgH$9)q&`(fxsG~bwI;-; zJ#_A6><;~gCePwC`3s3j7JOU<_?KEYT}X+qIrZ&{l`%!3Nk20kc@HlMc~4taC?~#! z&Y*KH=GfyaKno-MhGf9GEJ6z=N1qN{kuhcH)L6QtVv1VPAeeK5J?nBqDF1h?V#0If zPg!R2Tz{($-t9U3>l1`$*F7)_sV1CE1UJIBXoIv7>vMQlDz{+{V%}pl-3Z^X5_BR; zzkO>oo?*Xa%@IN5guI`g9mrl*Yx64=G&u}HdV&Big2;W@j zTfKQ}0M#+c@0Xmu$|>n5?g4N@nfs0^K|{rm@Fg|D@8!M>(tLDLqy+KdkYu}DM@ivt z(hP-nDg`(h46(vxUN<%Lzx{BRb-I8JRH*(47d=i<$k^$J=HtA9FuX3OT)FqUWIhUU z`5#rqe!^aic2(r(LJ^??n+ACw5t5sV1T{&WSb6CtlA%7U+ClLURWJQfrfZGWBD^GQ zMreFtJ~z|h5<-~EDI#-D6B#!93ZhJi^9fDyT7Bqg4SuMVXDA(8ic(G(h~56PkonKV zivIsj&(SokbULYsTx20&lv^K|P3c@@S-`-YsT@3HYQT)CQ9NYKg#YM#tuUvd=?-7&un?)hjv%C|NJ99SO*p({@gVR*oB1_wr;ojlK4$!0BBS#j9v z$^x?7R)EH=(+ZBTPN*ybW5{ey4j7}TPHP@?+`8;aSjF{t&Z8&gMbjo+Ga+~lRBFV0 z>HRgNO~`RxCHC|i<-PmTrW#z-9FUoJJq>G0^sG!pt=Xt&Fe8LS2E*QXLj>L}^nk1%+LO!KJ>UO4 zMn?s#vqG(qZoQYM4;7mmU`3ntjW(+RU11gG$bozX{wvH47h(fWU~BXci6V?JZ;IotR8cGx+gO) z^u@j<%ErYhe90}mu)q73nfF+ee1M*D12Vx!tC`fKK-X(wI3pkluneO6CUn8NJhC(^ zX(~WQ`vx64;|IAX4s!?QLhTy|cat?l;#_L{LxNHg?yf&x(QDH<&^5GB&bVip)d?e8 zY^FoT+dXy{7!*^=fV?dJi*Zxs=cfmLg<`7zSI7zy>VLod|IZ2;u85n#fq{T_KvRu) zATd*w@Ia7L36yy-Q?KWEpqkL|(P#h1aXckJSN_*qq#nyyo0K$q7ix%HVJX)waR zBo8$#21KM;@EtCC_3gs`mH{NJfhUZ|XVsi$v=pv{F27C2>u{9T-4wsUSloIE;~Kb9 zAB8ra0?*Ab4~MF5yARZ|&Dq3y|6tcUik(XxJHVn~m9K{>?2gg}wU_GVx524?(OI!G z-2c4a$CWAoq2{T|{8KaUWID`4E%6Z?jiHWMuDHt96+ybW2=TwebUC_kzHrD z)i1xB4QOG9g3eXUuX^U~>Xa-M_ z7i&TqZe;cqHihkxj|4zC&R4)Bb%^6NA@`B&U=9Nysc&g`sg5OZuASpv;26tzq_%*2 zt`TN=cxG#mWGVA&W^S=(+SbDT#@t~K8;HyWET;G55*W6Dodn4CC65P8vI0Y0sJz?x zd6(Tbfff1hG(-SELslD+-HeHZ$&NwFawMEFL0jAEuaQBXj{wl1E#i<(BDD2V52TwQ z-?-7{DT??$s0Lm02r7l$sngs{^8%FGjjw zBq*fGFf{Dl);0q?4+40B!RdIeC3)A1lIOkaaCd{_l|Zf5+dD_>%_JSNOP%K2`b07i1gS@e=*mfPx12qAhuS0$R;~^ClY7 z8vzFmW0};U&_Q8??lPf3X{4Ci6Qi8cwSnUJ{H{2kBpKn(>i?b8>Q=+%zS~+ohu07@ z_nZhu9FturKieShO*XN>VVA|>vAXUt{M3Ox5t|~eO}w)6G}r$2+4<&oMNjy7@FoEC z^M`)?=S&0Z3Z4D17|(;;SH#n>vMQy1iBONQm=C6~`cX z3Sjl@AQ<$VNGN&jOYL})h}iUYWA+{w`uVU!i_?iC#NS!GbEe|o7YE_X8sMXm$3=Qk$DxeB2Sn`!x3v)Dwt`KusbhD zYD46aB=EQlZcjSKrNdCdkKw>t(6b_QB&8%MNT3>}8&XUKlPXBYX-`iscO-PKy&;On zo?jlX`1lV6(!HqNr~V6HEaZ=NqS_%b+ZN5zRzX>_L;_#}@5M z(D7M*>#N%NxV>;ay5s1KR`RZ3E}pR#*%snqw=Jwx>M7LD6uo2?hDai)|HQym26B%4}8QbDOd?E3govQ46B1abM^^ zj*GE30w zJ5E^FwKhl*_0>n@$Y41&hNIe=t%ba22gk^}sD7@R1lKF%bl+YuCrKE%A15aWkWAYK z@Rw<^IniZs6kWKceEUv;`=|OvI=F@TQ0f%0pcPYll<90C;w{|NUf){c-teT@{C1VY zNwlNK7_^KLeATK&%u2zAX?GK(GJt^7_KTC}JS`V+WjT(e!SHhiK3HjPbr-+1k~*I$ zX)t_gN;Y%RS?wXtvvm=j)J(_oqQ1E|)=f_F=FC%;)kHG1U$@CL*WlFgEP}rG2|EG+ zJ&yb}VdKZTxUQMN;f^#XM91JuXU?+D??78%T#(fadi#%+G-Mda|#`}#fmn+^FmDZjbhW#DemZTpAC@Cf zm&`M9Lm6Y4CB&znH^9xb&r7NNIEm=mT^veJEC8>M9zW6C*OkdRKI`RIryg6T>Z`9g-011N%ihjw!X>i4O5_{Skprx8 z31xfAmdtS$qp_IWR*_0b$oyA%h2mBIwdu zdu%JLZv$=zdVZ){#eI^8Z6YE;8BxNW}Pf4(eak=dg`zu^CH_07SVb<5XD zCN?IvZQHhOXJY%w6Wh*26WjL0wrx8TO!Vcw_x|dBb*uJY`oih$o&%SBod53d1*f67M|tC7{25 zZS1WpZ1zn_t@yebhtwY0G)(*=De+;aywHzdhu&?c#Y1i;Ba=aUSjj`_Xkaw|V~)15 zF}IM!Sm&;Yq|28}=bNa{0f?ySkTw-#?&z{MfO#RgRxNsGm{Ixu>y;(m69vt)+n58L zH*5MP+F0I6*7Iej!;Z11*greGBAqL~}kO9!sD4d*lfg@D|mpI0%(*9UxzEdYlF9I|9YaY=1^c@(#_3SM$XMf{;twKc9x zyxZ;$_i;T|YL||VBK4#ol^%8HM*>b3i?#3$n35CD@o#o6?Llthrg+Ljwr!FN)A7ur z5R0Ao>&O0W9jE0aRMn!Gu>p?X{ppf;aeEKP9TQ*ugU-oU!T@<2JFNuCJ{dGGd&M%{ zV`bdFE~?~)`)Z>eJVwf0X4f0t33bn4gWGH^JW6A%Ab`@09MMTK4F0kuvqHrfTXFK{ zzP&Y6ABuV|T0tonc`D)%&b=MCf@l;_!_u}8iaIVrJ4&_-6|0y>KFrDEJZX6@8s%0~ z#4F+CiWm0$BR~{1BAen~W-8bXeyKzWRQFIwjnnRrXMsO5P#@}uZ<<)$(c=j-WHzFG z?M^58M>~o4t&iLbaRqb>sCwl%<3q~WHr)Y-!A@#n48M$pGHIccw73x`tHW}3>n!v1 zdu3&233WJu684aVQJ)Vmdc%5^I6IToJpqg4BXxBIpn#7nQ8xP zhdMXB0T9Hx;y4!58jZ?O+c|v=Nj(ioJZTyEbcVw%&3*O_Qn~Hj6Mu*X@1Q;VNiN*q z;Pz&}Wwyr}74oE|Xw?hpfp3DruS$b0rGw9Y3;Rej0i0fPrXjMwVa;8GI~HwhdVjR3 z4B^+Peg|}Jq4;QNBPmU(+zRf(=8L!q6vn&x1O~}DSGbO}v?lBKD;i(+w;B)$1VU0X z5Q(*dbf*YL_lh^d%b9F8{0cZk)5VT7quL{`b`xqv(~j+-juBn;<(;9r3I7J+d7TK^;aQm z832;qWeW9PIl~go!O#r92(LAhn^)OdeF-g91k(g>a>l{5eDCAOJWD%aKMaI!P!#DQ=4o?WDHlj`&x+5h)yK zODIW1$)#IGX!`li8<$Y&8q%Z^g0haE`s*Msw8g-rD$TL1u3`0%@IF{_rNkDuJuF{5 z(QdG#jAbJf=3?Y&5nyKc4$iTgl!Lr|gt~mBRl7i-o=JDo1Qy4Nzp>%{@w(zF9f;dT8)P z_K&qYhO-CR9tD10UcmlVX&L`#aBf9-aMQggwI8L2x_}bAxGX>2_e)83*ZDdPF{zNm zb=hkeiK=ssOS^++lhaj%zJUPVJE3aU1}ofew=P46H5Bn?VZmE#FvE$-?=lZ6uhoXx zvwjZaK8{#261QISJsyAa-~QIR0O1=4_Y6rXzp55xk7R-5)O+H$?;gvxIKGKBZZK>XKvvNRTIZl2d1t|J89wRs5qn~9K_reXZCpx@WY!$O} zZ(Z=RB>cp8biZekI3yTyUEa;@R2dLR$E+JgiUDI4zYrIQJyFEuS}MO^+-AAd@{us| zyyc?!Ik=8TTCVh{<=JG%hyG}zp7nj-_~(6PEXmZ;Z?Vp}{rAJDZ-Y)Z{%>x~r_#HF zggZolm%Xt)3_;BROO6o0p63UVfSQqIRcp4Op5t-kp8E&xn;mfQaFG53x&-2J&z;g^ zR%Bc;+E9AFxzDP)&&rmGGNsQqI<9j3g^@P)GY-=%zmW^$gWz+A&R&W1LRvBZ0Ol}b zt4TnhhrrR|%EN*I!Cii5reERr4ciiA$JGAKJk=<-y;8RFq}G7hv1B~jqBlc!rLZQ- z0h-MjC3kU`!1H^=GdVVf%T4az|L!jTci#yykyIR*rVKzd1zfA5Nn-iCGakrsF$KcXs8N+6z&M8su7k{htNhAS zP{w;daMGUKb;7*Hm__1wclbNV$VFn8UAKKE_gSbge7ngjmtBXEenaf@+WmUfFx}l# z59s=Qrw?%2kjnpNIe#+1&CZ)U1fgd;jLcBB`_;dg)=ha>42OqeUdmVo0g&b zjfBarhr+gj!Q6D?r>d!o$D?emcxZE`W6@x5IlQ#C$L|ZQ{)Nwi>G0R3Oxgm`BfoWL z+Ug#IM}F*QhE-G>=ij$@+<-x)u_j1$d;=4yWNWDYKCfZ42m(DUSt$ z+D!qFTMa=Ew8*;-0tss*FU#0MkY=V=eS)?e4zhZer$6R>N-;G0F5*;Y>)ZX`yU1E< z)yd7FDj1d3+@EGt7l1xHc(f`!**4SFX6XKlRpv7rM%@c>AF8YOiXF6AODGd{&pWgN zu8D;M)V5OLNA{TV1$X@M7_R|~1xqnK#gGle0h4Rm{XK>bZCkvAnM$1)V{grl?iIbz zG$x79?CiX52n4+pocCk*G9E}4Z1q*LAB|g4rfdY1r!=e94*>pjM%%!{mfm~6tQiie zL8rhWg{FyfDJQ6GUkJZKUYQ?_5bsKn1|*%rO%{b~3Tvcz#kOr#y2zs?Zs}DB{}reG*Q)#m0-^!) zzw5O{DG2fVzvHX%t!1@;zIcVBu&fv#?KAt($xK;7kvNlc1SXVLPf<47Xbrx zYQSAV9)JTvgNyvyCBa40zh#+lPXw;6GVx|*VXAeac5aW+00J2`G7<-A2%E+Tci#Ij z>HU<(>t!nF8u=_O8jxw@5x@y8{<p#I@f3ORR*FA{xg z&oTsry`)r%5MIF)5PsX};*5TcH#znHa3qU{x*_r)+`H<3Q-|B9a!sA%qv*#PKvu*? z?C-5(itRtBiw2Vw))2x@)zMkw271;c$ZO3xQK0r`rf?Kx&oo$Q()fJ_#l1dh)q^1fG6M2J+n9gD0xR(NwzLzHNw`WXa7bO0r{(AzC+xDjlY@N}; zCJt}mu6SSPVnQc%0iX28Ws_&De6#2}!nh+3_>_2!gfjLAg+AqY1~*zpj#)XSn{l@S z9Gx`edD$@(CF}*kV^GInn{EWYJ~`KD%{cGiSvQiyjNy4Ms#rE_X0M z0s||rL^YTD5}SFuPbgng*ur{taq77A89{esYh~^{c-Urv9+8IsfmKk9)oG(K0!*-V z2_BvtMhX?F0Uh|#6fT3aprzcL8lW+QLtAOheFnzVAoy?uZ27>at(ncV*HIi~% z?CIR98QjX@u`(K(p7n~Z42IAEaaf2)`a~IExfzGUdHuK%Y$gMsW0R9P zj%LZYD-t(B)9IT=nN|!WdW=?q34P}G=yI(7)<78aBu}xTD4(c6Np-%^VRc&(kwIy` z{2gg@#ogq{Nh_6GwjAYKIt6M^g~5AI;X!M;GOi#@^v=RPb6fS7Y7*`}vZaD3Y=K)=Uc} z`l>bWmQnC?b&coMieaPfh=uQ*9kNIuKg+qt!dle6+D`YGrhR?1(nI{3=9NcU|1Q@4 zC*{O6TzQ@UtHWn|=!B?6B$?pQ%=t+sQe*igHb3pNbDoVsvLf=l5b0|nz9u#XfcD)g~dU?Amx8Gg>`8Ui}1nNyqpSi*vv{KjBZSHgK3{t`%HNvjCP3U9Dt> zCy((OMvR`!>>fmI;VcutRUozA8>lqu^7jy{@}Ej+<>R{5bk<6HRV4rhqHrCBXR<}- zDGyzlS9{qE!qT$cx!!D;awHC+9g0K&5oq-Vj^Vv|O@ zpcWZdj1e7bro@R~=7nhT7J?GUEqCA^937gWN!v4Q#lW%^>NXL8rdynKk~>gFBNmGT ztXfPQvdS`I_pg2gD3o3VF_eCUaM+hBS%VmA3Tde$dAY7sL+|8@j%#6vPT$JgbqaE* zrwsIN5i{4IMq}B)g$mtWWT!IwYbgkq9Uq*iz_#_7pPp7pC6WSN*0Ycn#mU zfM-y*JxR9kTs<_v4?cEsOI#;hdyh#D72ySip_UFFF?@t8p)gCsN)oV@d1@An;^Av! zB{(e-tlqhX(kmk(hz01*Ao4m-AXu!m+MQKpfKK2seR)(vB3QMHW_>XTzp%H+gP2xU ziM1DcGsM|9h|0VPf=VR+Gk@qB2BSkdLrx~(o+Zg6CdnUwW@E&5t>{}U+8&Zzq@a_^ zTgd!g=Q$wkSYq_EEb&vdHAw0KWmo%x4E3E$xWgUOU6qWo4nF&Pb{=lF&GS@JLJxDE zQCwhGV<#NHc(J?6d2W;`Ap=RTCfE)w=*_OehPVTv_vV)W8Y2<3KaYT4I9kzbyr@TM z;?Q3vROC-8uz%dW|02Wx!iNZoTqNfgQ!OeDXoX0YR;$4cndWHqm9~3P#aD*>S9Umg z3d$S=4g>@qFHIPQ6+aam3_tDr6DtP5%LvaL{nO9K%$;+djb2`MzL8EzCU#DKu9#jr ziNmr3D>GhRR|e=_&nsQLIT5;4hjr-N+K|k+}La`4f()MIOsX-@H0UzJlpwF!U(iY?HHvznX$xh!WD8hq5-Isgf z4_~d{l6=op1$dR?3x6~Vb6}d-k`;{9oi~>QtURQ;|%rGV3 zQ!tY3>WZ#7HZ`akqA;TKui&XIKk5RiDNsp)Ic5$uHsF~Os5YwgEeb?{nPPGdP^T0j zgI|VGHPzosM{EDkdXvfc?rMo4K>bCF=sW90{NNaN)kmOgOz-X|AszvA(q~qate{XbS}zedw}uDhmb#H}QHvN>*Kx>*TdAN7O2q%UJ5VcRU3~`qtP|FMMr1lf zE{7~+jr~e$LIUf049^ZI=vhai5tSjznE3^Cr?<~gLUv7PLRNq17V_vp;hlISiNMQL zFd0bvjuWEl1(B-Id^=(^HdRqFV?P>reZno}Uh#wI10p{dRlv24#j0Mp8pcypC%2j` z5i=0PXB3uB{nV7tjZ=8DsN5#j(KbRj(8du(}R4jx;! z4Ba@qdIXzdS|xcAt4;sx4&oE#A5pw*Y;`&fmQoKPY{arilq<>>Na{Q;ql!Rpu_7To zqf+CP!YC#@aj*RB7qS8om~kH;AJ4KVu_VVdq1$*Ht1eQzOccdN?6Clx=rAb;5^{G3 z#_8I5>l$R3C0uoYr+{Gf8Hz3?XdFgMBbl*CT_K9bI$LZ!`aI<@kSL}uz#BTO=QUQxhUFjcNx{6bid^}yVtA3 z)y*?rOr7)te{(Y%Snk&EFWSXWHiXvn5w-HE>LNUJeG7#R7}y8Prci9!ZdZJBFMZy( zD13ox^!y1}Pv1ovh9~7t*E-(LM|>q61quTgCyPZU9U#A7Q|`mONe|^}H}vdC1EXAS z>d=*|EI=%nCVTG*HTL}ZQ*Xj_Iy!=QTf&N^W`5i)n1RRNXkr6IJA!-0JtOI#k=|5) z_n7g;0AxKSh8no&_qpFBivuv~9idCe^y(d>__P61ze#5v-fAF#RrjFQDtp+$Z(l8f z(q#+@T?4ZH(%72FI>G9WeKeN&mD+RkR>-S&wa;2MXREe|338d|(p_Yg>0xivck=bpzW3!$ne=2v-D_j5%D@yp zbCw5CH+I+S80|qscT{#}n8M42#aBUlSv-e?51t;zQ?0?1O4Kc-yKFiyHlpWhXHFn9 zH^$|wprdI{>WD-zh3U&GDT^LT2t(dElMAzly9As|dRQ}!jK542;<$6yn}piAE7wVg zGj6KMB*$}3>mfo|8o)T0B|9r3bXba?+Li%!VkOxbG&Sl=8)l?&;I@UEOEa{qGtN0# zzS_+H(oRWqiEhoTO@6$9y>x(Qhl;3NagI}1FpeSs@7*6a}+~32YG44NS)0t;hU)~tjm}m;&-6OuMme*aaGJo?|fx13HG2!O(I$p7>(o@#o=KQ^Yg1-Cx#w9$uEXmitw4 z-U=e_l?GD=$VSm-<+wsKrfp3hDQ#z?^Z1wA2XK$jc3ck*Ra<>#%BO<>1u#Z{!jO7D zo-3}7VXn=MPq%=_Yfgrt>*&%%g}nqhhcPP%CEi4 zLgb>Pwfk{eh2P2s#0q`d2F!@d;2U83BfKzWzL(UVu~|y;5Bl(!euR<&{3{wWs*`b) zvvRDm*4Y6wftLZR>p-GOeF;ARrrlqXzl-_V-JMPWI>J>QZwj7qMSb@Zqa|`et0c9c z5IynI8W#T17Mlz18gjjoy@ULsug&Ya?uJ>lx#H4V54md=)W2Un` zllg=s6p#G8Yq9){b*4~y7wrfxV#9I>^66~$QoN=6?1cT9&(6jgnQ`1t^JZH4^NXSn z^%cw4Dxi+UIBiY(0UqHrEE4y0)?spIqnKB-*s4MsPt2M{!QU~({P2V{i+DILg#oZu zG0;vEV~lpjn5!(v_e?f`H-8LX@i>fM`G_oxr#40@FT$C?w}n1H=!(;VJSm3>N}IAb zNXSOC@oJ8%14c+%+;|MN6+M&pcN%yP=6BuKHyalE~49XEJJn%S{ zhaks}SZ$x5x`x8JiB5D5THKL)usG=Vkxe?C&Um!3dp9JQ9PL8@WfKNgQIwd}IAE%# zohxP=K4-y%91IrFKv|p ziwpEL6QPAY$T|oBXQcOr*d@wrz`8i;6|y`QCVi`6<1TR>K2j9Er!+mWYwXDdZHxyQImTh&C2T}-g2bLx~PRI zKN?bKMbLkl5kO^^{K!+CR+)k;xn|3caG|6|m$m$A$30B>RRUcod2066bzX9p$KZ+w zhLO<8RErn*g$ydrn~-S@-Dcm#UFT}DgnEH!r9XvIzCFI2l-I7=i0DpR;p$rl>NAS`~f@)1iOUA~q@g9<)LP-xxt`WfBZ_>i-fmC(cV(t4te#~Kaj^U)1X~9TpQu$4GELs2gbk-E@0Ih8JsSQe z#6RBle;wxk^|nRJl&T58D&S_nrS1Mf_~Or^;^U>oj)Fo1SpQF##O?>0FtR^~L)yR( zT+_wI?#0K(2u5ye6y+$bIXHR))X|jZe8RD#dJfH+mc#loX#dBWXKe_SgbxrvGM5kd zt9T^3y~SfjR)%GI_WR5ED{}z)inyLcQ2^f&tZU}-O(d`aM{oYD9f%8Q&%)Q-*#f)^ zqxT0`LCi%4t6WzS$3fqlkNs!i2wlD`E6U(|#J{tewwHCdK*|C_Ln+{muMF_xyA!g9HVLhbGF^NPGc0!&_}7KvN_9w=sGm-x z2L%aO%*hH;yzhRI7M^#X;E-qVe^l>3opo0uCh>0_E3k}> z`RC(riNZb8>0s>?yj(Wzt~fSgk8Lm9vo$Q&jnl7qa^^1+hA-Kc^h3{j?vgRAHsKmS zD%EDwsud5InZ?~RXP^pC!TJ6|y1x}XLzGQ_ktoF}=3l}mqQH-!Efn*FnbLX{lMQ<5 z_;cuJw0<3qnPhCOiPl?BDDCR0d`R)j6snRPqm6d4C`p{h{n~+(?xkZ$oZLY@g-B$1 zxieE9LmNf4=g)mb_{YNjYe)ZUVJ(*;=po4snl&Ow%h$#HqGdOMA*Zda@!+P_ z_CX^6H0}P8UlaHN25c=2U?f^1v4i+VtRW=cDC9}K0!~k)R)^S?xz$>4B3lY6boK6P+j%W1&n1a1_}pq&XY zEweUP7|v)>u5XsWJ*lKJo?y&T zW9IsUt>9eQ=Zof(ZgyH|U5G7nqi;G+6}(xi0&AjZD#PTWB#ezj!5a6Kk#&96dEd8IwHSE=pg$nifT$ z8H2>~sG4Q*Z2f2Shj>ilBKp28vK*86a-A}rEuvx8rc!yvfVT!Ei?qIr{Q^(LC~E&p zn@}XVoA}`D;T*&^9;hmbtSYM8jE`mVKzMZltxTYteNHErY)J|P82(CWJKq3cCv@oqifmQ+fiiZ?Oo?U7Ah@G9}4CyjkwStD# zzE6Hzx5OC`zW80zXRXj6Wa4?6zK=IpIim+{wC!g)c4tv@X<61=tOc$V>Ai|9@=TYK zIScwt+q@r?#J(i`_cwv?xxX!0d=U+IKxIQ2cWJS^L!GeOje{pj;k(!XXCcNc+ar#! z28p}6&kCVTuA?)G#c1thSnHMhRi zKm~cbzlBUQqFzwXLZGr$cDYPRWfo7NdIq>zkgaGI!_)=2qq~15r_4n|yw2#wgxz7W zVpNL1>T!|-FYspIC<;6Q%?ixq!oxNqa_bGA6jidUKh_$*^V?A9CEs{IYUlb1Eo3CR zPo-e)fcWD!fpml1_Qb9s2p%cyVZCD7;TjCyqD_VM=!#zAcN+@p0O{jyoZ;|9a0d*9 zj)~gC92YKHMR;97RtlIzWFAY=Bz0|1q!9m7K0NL_VC8G}Vt&8@YzTrref0-n)jm*Q zKf!=XRB-CghJNHfLv~W=Y_p3F1bANzN*e|FY4hP=T>BNcAY&ZyRG-$&L=anLDJ~LW zjz?y~*;V!s4!83Uic-I##HasjyLl^c$+&|JVuq2J+wVOK0_Ye^*+mG%4a<217z zBM?sqf#34c<1Ix2FOHVz1z2oqAShG;T$w>wLW*})u^=qm!U<8wJ7_#sB?7dhz?7RN@s6W?nq+*TJw zayQ-my*&@}UR^G1;`Ql4E%WUkXK_}RnGZsPJfr&;=UFj25>li0g9+V;4 zjf)sGvPRO4sRU2BeushqUa1ELm7Xxv>w*C&i1qHp%PI|@sC~*W1pnPU;JtB}ebsD8 z>OM@=CF3H7biCHV5}abCL)%a%WKonf9ZNv%tz!l-Hf-C0<;2=P7B9=Hzlm;H*xXon z?JBwoNefYnizp?)RGE~$)c#{lyjT-n8Vgf)RPb?M~Z6NoLgl_=u#sz(AF0OCykFt^w89#->g-)aCq8t>O zKb1;=GOq#tv`oy4wtr3f%a<-3i_7>4pJ)%sUe+o9@_az2W@l_VR=E%S&uN%nmyj-8 zNDGnqy0Nwe2k4Z{v^AG9uf81FNw`N98$83huH?^u<=OrVeg8iw{v&Jhv*1v7|4Yj2 zpoKh~?n~8V>1(bf;sA^W6XBo3YAI)VBwq=-mNX+CD6F)AAuy!0_y|78G)7lmq%`PV zboeILZ+zeWEvPbU%Le*yK^65l_tUU15C(%t>w5*GPs?!SMFFI!+I{gLko{9r4Y&~p z#-enB1C`cAxx{=yqb0=UlFA8fm~QOWs}FCympAMp-mpG_glx)EWV6>FRQnf?D$(L- z&Z8Go8xyS46Blo9e=d0g&?~dBhB_mN51^zrn3R~5*kw&tE8d!4)@Bj#h)HC?Xf%pecveWkQc<-IR)xJ{AB4w@e!}ECzSPaSfhMl&Y6(qis z6$IKWhj~L?b_DG*aWT?u#}e=r=A4ec#E1C}!|&~sA|(<`?36Z(NZInYsV)54`99+F zE3jUwwoGKO;v}qCFxsbcx)eB{;^!gQ5jZf+peKWS%pSey z6mFdEE$HMZbgCBV`}nO`A%25f_;ZR`1$!UbnuN;{{2nVv@Zv&~>_YJ%W99@q@|$fU zoR}Aki`ag$W4A09AzIE{f^BB?yqY1S%eli*g(XXFpDo@S)m2e`sdlCg<;KDhJFL!U zP$JuGk=e;?>BO?z#D>E^-sXRe^nVuxlK#UK_!r-&5Hzi?3lTq!$dwm1EzgDxHjOEc z53*%&jgH{oIvGuj1XTY~6Kd%~oPtFBXZi3duzmBz009A|OS?TrCrJ$^hfKT4{Ra8< z0*qbVT>iZys?vt>RaIXSJZ>hhA(z??sz+ywl;Na;{Y6KUk3hac!7dHlQxScYAtf%8 z9{Ux$&k@TLOUux@7s=C_kMZ3D|H;o|L0QsrqXh{c-~8b(ZQrBsapXsRq#i8@C!;Nb)(E)S0FanD;yq?V_vTu zXiYC25C^COn%ip&_8_RE#3NLegbIB18b{(Nns;0xoi$ zwL$FNYG7w{T2!-$zpnOxc<#15C&TUAt?$1MzlKwUV_Gg*)##z!`Y_wqe; z$eWKqQ+TIuX74E&0GUfP^Q|s?`k(=PisKp2%cZ*4OT5v*wuJA7^?^GjTL~HGgui0UrJCpJ1l-aRydctAusp)MJ_rqD-_W?Mbm4gUbpPhnonVzMCa#^0Og8;eA&)Pw`tj}IShD^`m zL53{P=0S$c&+0*jtj}&i+f2`lkUiEnx{y6~H--rQ!#f@b{?j`>*l*o^gssol3?EQQ z%4qOnZ$Jvd`GLP$^*qLwEVLE8zapAjv&-$}n^&tA@iXKs|KJIYD0onyS@ZskuxOu~ z&j<}ZekEmhc>@63Q-_q8P%B}U!ZITBmY;tccUq|Q1*Pk;6X$a7RSbG=W#SkXXbgh~ z0x=v^%@mjp;yg(|yt**}M(#`tApBSX8A7${XO4gHp#4N`PV$whWV^Bn2LhN*thl==9uS9?o` zk6_~fF3Lq7lyWY_q`q}_eIwou;E2XELKW_t+S9xgHUc*YI?PbLUNcfX6 z#$!&ACke2n{U^PHc}*}|KK;+~$@e2I!VAwi)LYKyK>37=f$8d}sEQZ2J`h{@>L-XZ z5*KH01Zy$6KA|c@-K@($%j4@Px^|;d2}h|?Oe-qa2#b9KGgX4@A#bZs!eu~Xd7z^z z>0PLogSAj(Cw;<^MC10Lb=)$sL*7(L0et_CWb8?QJUIyngs` zw^5xYMQuG52S=Xlxh=2}1zm7h!ORS!=vJZ%pTnht zx#0xl*%t8)bXOPf(B-^rS!g?2#oE{l^G}L$t&$_*a@Jcb$fJk@Tyrbqu*p-i<#@y0cNh+jXVVMbk$dNhP77aL!owVBXsbowP}{}OfO?^ zJ+_7)=PmtjIQWk{f3g<8{_yz=BE;k5zyOAf-jPrL8$wMzcG8d3;kklH7eS+?JbJL{e4 zikQt0dG!QEZ+!&K4fP*$P0u^h?^tPhm3avBU3q?;#@dD;_D+cJLsye|Yz0QeHtMp@ zkMA;5zE<-dQ+8FS{hGLtQYn93KGG)PWErh0!wNW?{fM6W(#fpo!Vk4D_L~Nr*(_&F z_45`dtlKqYbD7m*S2okid~`SAy}1ws=$>*rPC!cVobR48JJvu;@Emkcg&i*- zCb+?OPw5?VpeDEhx~Ke(8;}$HgXE^TR~O6y?oE9MALs$Gh4!4)`wB(~{ieS|2u29& zOLkM&O9_?>>q}9aA!(}XbU*wSVP*p@W7i=&Kl##`6eZ|(&qY9w6bi@^+@IEe3(4v^ zsM&aMkbYWz69mgTy!G)teO(<*cR%+#j~l)moNeW7YW-1`J#xT?(|t+7R6&d3(4e|^ z0{+&?hU1`EoVsS0b0OJJs{4m6E~{tPp=eT2|1CSK=NQoT0`;Nni$`87{P7bmeOGbSTT27uAAZymV}5QRBJ^RW1qy!9J#PdYsI1c(L?C3f4hwP0LCCw zdI`E=CRM0>QHd9#JbtE-;{&%J@jjB{d|L2Nhhhi4u|the8dpFC3Hyxv_MG761N%Et zrcCcPRf{Zwb4B)oQ5uaqt*W*?yBLC71uFZ*q|Z%`b9ks^omkps@%2MtNZK$<75<*Q zDNTlSGlh9SUEr2Y$-MWc^CA)1GnHn(MugS^GtkVWvxRm(L2i(Vyw1FNI5(a<(!#Ql zD*j3|aQArJ4uVO!q8}(JVk`E48RKiG8dsS(*f+GtMxMN0BV?DV9}6NCSKRf z!Aq=3%(cU1!M<{`e3YPRkl|+CndCp0F+or#3);Fwi1;6^}iT``7)-BnXE7v}bJ4`CqKz3Gu z0Ny&hsKKk*#hI^>dN!kq`}E9{QZwJ|9QD-PqQl4LQh^u9*HXRUmq>kq>Zu>Z4iVM< z5Shh;S@O%yU4cR7@4>I9E^1>AF*Mp1o948CVy#(IW%QfVc*nsS3^aONRHPE z+<ov|_Kp&jFdawBUSyIW-@jN_L7t?5Y1C1T9NqGY8_$Cx28nK3x9tnP>>z_3lJn)qb~Xiz4x>P zU;6IO)cwkR^|Z{G2M(A+dWAx4UvfT$zg2IAALnM5%HO(YqG8S9?#LiAIT_OIxX+z z3P79TojR${#DCwo-XN{}w&(pVr2)idJZp4GWfJj+^A_Wj(jBjWn>=7f*-3u)LeEIr z(vEDVoqgTxhia=CT{CM#J?4A%c=L9x-Ul?3H5BL8C6s=9Hm~^7^xKjdWpjz2hwJLb z?xgv%yZ=C>PV7D~yQ*r|4|WGL4Of-(vZMBC)xMEeD(MZT#B!xkPXsBW=^bdTsnVE} zZeQ|lUARFI-n)tD;0W+6==b3EZRyX#>mJ_q!S9~leZ}|h-G##UAK4AV_aE35}TY9`_<3L{Z`*!!}V6)U&H;@-tWNmh8ec) z`b;0T?fzVY*t37rf!K3=vyJD!y^DnR(b50u_$(SG==^LNCJ1nPRt*!Jw;j#UW>uIl zA}=w0kM3`bEJSF|1zX=a?!DYe>x~OC z{3ZzMkM>6PjL)=o_qtQkM;Mq33V`*;cq7N+@MJB7y~8)-12?z_ADND~iOz<~fp!Nz zfgyn_f+0aFLL!0YA`l=7KV0OYi^WA9O(A;p{ya0A&ZXk?Sj^O%g2Qh(95H*mB zP=@dV$i6VGkiOKdh`!j*L^p>!M9;8}Se=l3gudj@$Tv+p7&mS^8aFyOCO3a}$aipi zrEg+(*gWw)xt}R+n0Iu0dmsv07o>L_`g))WMivO`+f*!pJy4_)LqDkpewTu{Nppn3 zcHb)QQ~(;<)P5|_2k0&*!I=ob5{t(JkvYwf%_v8Vq?iYSwc?56^W#xRm_{5&phu)e zsHCXP*v#O~EJqlSk<2tl?Bna=H;H|O9)qKUl7+{J(c@*L(9QVFzzT6#Q0g(w97f1S z%ti#_spI{KUxkZBiiL|sErcF>QhYS~DA^E{!vN3Lu{Za8afU|_Z#;|#Eze5+X-LON z&(I&?n75W2LxxVg_ILQLM}(ugT-D)dG`YMfebk3X(KJWqHdHlAq|a9DnQ`g_0t%N? z!xbz7%IdeHHYB;fDDKN_1O@abcUTwX1XLIAR&5Y-E6%)9Y!2Ymf*Dodo=x1S49jD8 z#sTWMHOlzDDUtW=9t^!;b``uI6SZxV`(yqwIpvT+uU`0T0)`ZcN!19dTT7T~(`S_{#1`#u`rHNR*Q&XDQ zb0r-gHRMc;8hnlzb3{T=TD5Qs@-kwz9j3a4fipzh8b>aFVban|gg>sJK$6=%luU76EZ@vfcYLgZSMesXQFI)|}Ap+?O%uV0q2sIJY+-syL3>`3Y)WaO6NA1s75v8b=2Lxcr)z*h2G zqAwl$D@~@Zdg8YHr7Elp>;qJWmEmQ9rxY6wixk)Lhe>+X8M~jeK+XHhW8@rf*|)Z6 z`SF?+v8|&7u4FtfO8Ttkwzni77xtr8BikTtPS9Cp2Cl_xSsJ?yK1G>w%VmBi z66_&4s;{%-Hkk@{cyz&SY$>vySdU?43nTpr2for~ihG8q(r>g1d*XoCRLR~F^@GC% z&$JpidpvBx0`Gfe$9laR`x%UhIgHGJn!{4UQX{yR8v(*0fn#L_aW&p`dEr6)g}UnN zeEG?VRV`S&xb*IWIe#9;)uVwhRWU#|=lzL4&v45n-bu^-GT+eY#uXl_WS)zq-=V3R z9x{hMRF za~Yy%7uoPWGk*$m=|Lnwg)}QGQ5U^VoMFmLnwmsD%JK-4a%A_bBfRj9V>k zuKuOgoY8t}!4Ceh@&uTE6F2-ipbE?VX7Umo)vLjh(zWYZ4Nyl~ba9pql~N#%Cv~WN zdnU(oIjhb|7wVXW!6THfFJsRpU0y(L{_dr# zf}QsPW@I;l4GR!9ZhU68CYmjRwou0Hr1>CI{5Z?X6Nu# zu}m+v=W#mt)_2@X89MmpcWq1BI`r%>eCM`Ru^e70Zm}A}>Au}=)fyfgFJR|-g1EM5gx;ej?B)rcA<)n4BtApe2w8u-w3w^4G+#2l^;zSPfRZ?=O$HG)^~?X zVQr&ONSxzTDX@8EyH#mCvAj^6^Hg2g-O&SyceA;dn zFTOj><5HBPd2D9PJoP>`1csWRt!Gf|+cKehf-qdMt3`btCFW*Q1$20%r_O-k1tazU z)fEF+&X}~v@42Gmi*xQ?O7Mgd6z@A#yyJHXm&`-CV)#W&&mcA8K&_0orB@FP?|MJZ zAgqMi579rm=LqAr#a2LTem-=Xn66`y=REsToEk^k$n&CG5*8BH=arXb4(f7Dg!y_D-IlaBQq^AH~7D- z`@ww;x_2L+qCkQiIZ((!A{V^ypp6YaY>*3@AZep$AP4IIEh|GvU!mud^@Ka(&B4jcL3B}0EB?bUw3Msm>qDF#r3 z+T3|zKvB26@Qv>AaAVL&|D^o8A?LoWeEbbAAisBFVh{@vF%GCp8(R`mC{)R)5ncy7 z1Dx{z9rMx3_P@ZCe`c=+lxY`!tPqU(51BU^D0&(K5i~}{h!1Fdcg8To^lgw$;aHW^ zrN8MW`fa6xA0H-`J&6t9rcB8Q)!OM+Yl}U-TD2r{jP+FnRkRO*Z5!t?hdn$emob+; zGSj7hy_YAW_D=xE(yYvhg3{Uu?}X?0uFv}G@wtI7#D%~a46(|mk=Gn@L z2{YpUj-RhrHtRQ)f|3IF&fb2t(P1yPR*4l*P^8DFMHrwJqwvuXp|HZd)Xb&b_Zl2e zTBT2p{fP`{8b|#~A5{3FQYKhiFddgndEOODcw8Wf`Q5mHGLtw_MT7j+L$gzEOMTXV zN!jl+6jC-mNk7Pw)&+4UGZ?NEM6r$rfQB*$hlAJ9(TN&N~{$JG2M8B z0J5+4fAMw{+=mP7S?8T3x>}_}&s>5d>K5jTBRQ0z_;>c?1*ZS5qm!D9G ziQ5&OGk4SS-wgk33+L5#K3u$ieE^>v3)?r(&V=w{tH3p_r@lGo`;ODABVi>i%ZkOi zR>N~2UQC@Ej8&t?p?x^VKD*)=rUfT=j9tZCfE9j-C$90N)*esrjj#3>+|y-7hEGnybS+SJIFydF+JW|z z2u}G$6(?=nYotnQ+TLZNLgus&1fpmy*jD`FyH?gFR*x7WdndFw+jd;Mp6Z*}D6;Y& z0AA`2Fl{;6n&B9=>SPQONvvE&RUkena*|wVm3WNGx`p|k9qju-Z4uT>BQbb-goSL7 zxT2$+^XF zMr;?(^ui*DwlTlN(Ap^9@33Byara=}*F~c+Bw1N7j2DazPS~x~2>RPUuR}~OwJ0L{ z-MRMo^pypUPbIy|`*%$~S;th#svl3bRU>$m^WD|<%E>>H(1e08Ag;NWV$ z-njzLegeGBD@y&8q*0Uc>^(TtK8!gf~AjL0R=^sz&Z~uVqIi zDnZ?vdw~kHJl3?M@&^y3cSK>9;MJO@Lc&-&YgZvp%NY@+tuyfcuL$n{2DJYh>i)r# zGh+qE)H9XLfn#b#0 zplI_-1kCE(qL;!vQDPayLQ{0XN5tIa+c4!e-_>ZIRap6;zmt$)#`{CPe|rDyX!j!s zH9G`p*s+$M|4(Lediv_QaQjEV`*g3ciK+GJDCCwvAc!`o72%YcTCz^%CpM#gXr7Zn03UCCvmA#u2eDLGvLE|f=$~q2KHp{gp&R5hgl7C4&c9>^;IB!Yz$@uOYggp59n{&)nU<69vU7nXorCY;b zC}c+ZhL$g|?R#mZ-#E;*u(e7l`bZ2l zkV)zCnPq&883V~j@ba`9Jo@Ad>KudWucd2~-H<2Sl7^KCb(ttfsmj^=9#~AxZU=+R zccRMR#5c&9sOU+I z0R56C3gCg#58>bdTjv=mRNryOoMr3{N?Xg%vb1@#Zs@)8*^F?}hJ*@^`uM2DX4_TMVWZztI!|8FCr)J@N;hY)4rESRAmV#=|wN>5Kvs0lzI)@V-z$;Ow zq?!qCr=%nk?Nu{5XdAaT0W$VMEQT550bbK2sk<(=!m0^~>142@i;dXEc5O2+!sx^n6c?Z3$T=>9JZ{m)*xkYum0|loOh;Lk5yHq=*nbO zIK!XA|ARPE>M&9ytIfR!J}?wlHJS;vqM0b2kiUGmFp9*4YuDg1MG>QGsATch1L54haC zRH$T;s5R|%C^P)yP3EDqmt&epG>wQUFyf-2U{va=il_d-e&GJuBgR!>eZNlcj7{GJ zr-<-XKOI4JIV1HpW1fiV@PYk(t;N|75fVl9NEZb#3sQZrPu-MJPMPm-)wZX`ov_tz zt@v`wiHX~Pmxe3zH@jDl9@YQP^rEwA>j6~vmUF`u>y%N-enwI%wP>a@wLiJBcB9hI#NEDX5Z62Xn)^H3ysj9qf0*2$yIeD8IkyvHcO_keZ#D@>qQJmfb zRbCIk?;^1}v9)s1ccJtkPNMamS>>So?3`bveX#R*Qhzl?1`2uPm7E)&gUO0JISFUV z@xyj$R$>t?{tl>A`3!yIEP-~n)jjR3np`EtLmy*FY}Phvn!nUyYE9Q!^cSR+G-)tE zWC(lqE%F?k{GRNmQDZ0NU zY9D{8ZRcD+7)lgqlvSeZaY+3E@wgPI+xK#^x}xLe=vy33zl%G{6kgZBs=O=?KmJ1q zCp(Ao#5w`tOF2tAUsi8=Px7(c0pj-g1y_z8LcrL?8^Je?c>n&Cr^q7ZC}&Yc-VC`$ zAW-9Rl$$GVb<}9xakfs#9aco27JAo! zN0a!MTK*l5=|k1Z;5Q1~{8f^;x}UIKp_zl>_P@-i8_~d+?30I{!uJPNL59|WI ze0my${=A8iBc4Y=pVp6yDmToZ8x};j3aGXN)Cl#Z9PPh^f=F27Jwss`If95Jhjk9~GQR)R{`Wz*E8ep0e|qoTic#LvEIQNPE~Y=cC@RHO!WX_YN_nc_567C@C0k zqIl1!iMe6g^iCN5)twl}ff}+j7=HeN%7KVoEr)gs#XRW0$o9XE#J)OhQ3KP{vS_#fYG@#Qp4o`4WqOmpqym2j@ROS5aPXce*S`Vkzleclc!K z#|S4wzuclzZriwA9<^$ndF-|z8gqC#)Z;@iXo_ zBXHO*FeD$Dtsjc<^ zq^t8-{@)aQ-J?d%xkkN}hRDqHn{OKShVH5?Hk{hPCQ&u<@J>sG1!?OHNo!(F!3lhT z|0?F6ad;P9Wot5Bs7JrA@yCMYgeoEI#a7{by=4vxRb{le#fZAqiy6ZiB2E0JlJtMb zS2Z-v0EX$mnoANo9Vtr$FA(W}bN%#7Q}|r_45(aUZ~bo)x*S*<8&cZKWZYg)+UgF7|S$+uqXleJ*)F} zd`#HN)x8(BIa4wmc;a!&M>h|SE`5W})6%3e2tm#CY$Bn&Ca+9MTzq+VH&opvriiD* zKz#1fqG$aaR;j+a?Vz{#Fj6s#sUKNFFRvesr$e}0qi_=iVF;4MQ1$-1pk4o*Y@bTO z@J5ToC`bI4!K#z8?je)lY+-EmAs+HHW?U+l0WoY`U2k+XyCkibogOEZl*plT^oU}) zbK)^d;=sdnw+DPdI3$`lH8??Bc~jCNfN&{yf@1K)dqQLnr?d;vuax_T1QXoqlHgT! zf5Y}+4+LxKku$U;&!>~05PNuH%gJA|wM9V;*a{Sh{@cO6xWI-APwVJck;GR1fJjr3 zRS*Z%2N2Y&A7Vs=pA-nQ%|6oJU@;^mshTVuqGh3HqLs-MC5zn}l41^odEi|F0V+o^ z7L;Eg|BUyeSscEEwsraMFIt_FsT~>1?4ShQGGx!UPS2DJyhbpQ|Jj1sLbRILT@IJR z+9;Z{?>o7`bWt*OESMtcoDh24D zkF=Z=5QlXz?zh1-$W)K=MOj?asJo-X6t!t0sGrM`3OHz8Q_MI^L9~oDQ}K#&bNVi| zdC5+pb@*G{`HIU>^fbBK5=5DNgTJV|Ly~H=u`?t3On8T|OFn~!t-VOQPoFm2G7>PB zlOZy$3{any4%r)eK$h;tGBQv5?V$P;u*7p*XERHdlJg!8M($1m{D57al=U#R_Sm*O z+?77(6s#L3C^XFQ(p^qX;YjBLMNnsmbWBrN%XOs8$&dCnUg9DsVu^p=h``aU?3b~p zJVLsbX**)-M6NY3Gv%1ZI6~3yOb|n0!;NR^^+*$@Wr@JmNH=;1 z)XBQr#DU+iHgyx5F;J@VzX$az{ypMqsH?ZL*&3@Ws4u7}Z=Ykz*}fbKw(VEZOVuBF^;^kG30d@#!~MFFA2|I6-;W^!L|DteEI^uk6Kq9EgruNa zUPJ%8hHh&g+R$if{KFHaIlR$B6Yyr$0sHGAWlp(FL`is%$}%)I|CCbyciwSw?at*^ z>l^7EVhPfRK3FTZsxv{7ZXSl=uw@(G>*TB+qMzm$b%dD&6&jq&x|W>sqh zUIdaH1(WMDAE$EHyV`s&xoouBDYm!5P#~c%6I{UwtvX^;-@2(kMMAnQj}vR3k-%@^ zp{>=33KOt!Rv(x3v%N+51^&|wxSCJ9^6O2mmf>e**1`|-FtYaZY>g}OJ@qtMW#j$wAsn=_=*b-_@-p~HNyx3BBNDFa` z@NLatVk)QS9|_>DV z6H3~CacY3?6>)aQ?U8HBk^gPn{fF+61G7CZKxm>@{___*=?v&t;;f7&IL1u|1I6Ly z0?uaZ8vSHzG(2t#|71_KddZ|#8=lMxu2o!(0?P+ z(IlzQXE<68x6$kBX-3C7=p`q%#z473C*CkW`hmx}ntxbtC#%b(wCvPu8lqv<6P{k;=}%Pfa{2greAI6?Wb_?1(p35 zlcb6E_D)aCUG>9uX9awLh_DO)kMiIxt23wi za#iEoPJVuP|AlD7ev3TPC?5UWa454puE)q@ehQi12&o;lgkD$PT4d@VaLX5TCN1kJ zXu-V^*M!!%ZSp|=hHcP2)&#OY>nZm@&g{Q#LSo)x_Xx%8=(M=xJL@FDt2^Jm-s;7n zgfNJ;TBNylEMyq?rV{oiAM3d<3Ze$@{&(OrW%#ohcRTsQcIg$Zx8mKFXfq1R5QT>p z@-`Z;U*!lp&u8ZC=vIFtz!OV)mz#&$exs>1(0=I00~4c9IGP9fSnsyg%%#M#jW zrqYq=E263wP3^hVa`czd${Zr@DdxGlhwTSQpGTp%XY+#cuoiuZV>?kkPA$6$hNnZ- zBl?U&!A7UOyfot%qEmNe>*Bo=HMq1-kmGi_dQ-I>G$0PaKk39yz=kti$h`eo6+jV4Uh4=69h0AU2d)eU```m6x%tz#=) z?i|v9gCqVOn)%48boE(CKdt|KI!n?H3`*|h#zyge?CrL93ern>!Zj!UgEJ-@-rg37 z<4990$bfxTf57@hgTNguf8Ru?z56I@v`jHYJFL>0ea;5eAa_1_a1^Wc!}yLu zmg1_8uOoMIodO5Qx&q{AS3cx{CTlA(L>EER5_6@!3Gy(xj(%}`J`gar>26D>K~bE! zo@P@*qr8!crjBct3ump0aIQ?ej+WtEK(?a&bhAB>jd-5NK_zrbr*6Hsg^H}IuM*6* z7BE*?(ts={T;D^mKt^+%p=yt_>v6_=Ymq;PZgG#Qqm52P5nE^_?Av*X>EwyuSGHIi zVoovLCVMDJJB{SaADV2lJI5T<@USq*kv_qPW!pzLPgq8)CW)1kRt6zM(kn17028gu z3i%F*exom3L-?hheY*Xn__KqL9qg&DN{=m=PvzEMMFFo2+q@{}*mIQ5Y`rr5@@pXJvw|Hl@aV&zS&J-G6A3!Gt1jFUiQKFpm z^mNR#k7@H%p`NN+JF8xC6`vgWF|gPmzR>!U^<4}$55{UI(balx?Hg5$W(&(F**aGP zVtZ4dd@W`(OLP(@BSi!mUavfGVwKkv{2gYL+7~o=g$*nSJ4_k*^22XOGpqD?B_olf1P9Z+@w& z`e#Q;u{e4qQpW)qOn8XF&;lR?#tWKEK1~UDeW!9U`pA7yji~Mgo z=l_>HLVkzKgZfuwyWSv=pZ!B1$p-g7eX>pqNOX`$ISLer)S3?wzia;0vRI0%k->bUcHCN)m6u`OY?u_I4Mk8KvGsHL6OW#SIxH?m+c*7rv0>Y{(XM-FK^Fxj|QJ`s*R0NHIckttc+n7 z5cLL-ni)Q*qdkd$#CD5 zhg7n*lV<_Fs}u7$ZagOSm8=72vW76%zXtanS}y*sXPEZa`)(r`>5F@&kh#5vYO6@= z*yYje@yL-Z20y-*lqX`t($XH;IXfk8>H#clrfav~syhvLpTlX(*evBujd;`?ecGp& z^?+@oG5u>x8eKTTzq<9}niX>e)HBAoN2Cl0#LMKccooD&i*|%mf(;+tzy+9&W#J$KCixUMF$3ao3muIo#;x%2#tl`J1#-2l zM$jCLVPpf}pK^#ntZZZRTxL|z{Pj>ozo4b%+k;n6rG_^gwxR~_c$GI5Fi1+N z4<>JUht1*~D7mp-sGqgJe@V|r(}AXC49i_+p{}cz!B-WuQjnYu`K9qnzx3Su1B7!kv*p>t$qccU-MDpG=uxYVP`zRscn@ODMf4)27ppkMkP7QrvLx zVWvo8D~y$ZWUVu$s*shTlo*2r^c&1Sg|q+Nq5s!05q|9YGr!oR=nue>6T*KP=!+0| zIFMm8H6-YqN)V$_yM(g@3i6+Ujl~lv{owzZ0}CjWkVyaBrLt*fz5M`Ok$^yoyo;_n*mK2v3XfuQNK2N`h~r`t)fH0I>|9{OdE2f&V(E z$MoR5F(u~#bwO#i!{}%Pa5A`i)7Vd<$VkB=q_HyK_>kZ=Je1-g0g!P&H!AkGDsWo# z>GkP7P}L+PNJ;&orD=H3?4^64PA2*&~vu3`1yNv|e z?vL7z&OA2<29Uu!E#I4CS;CwKx6rAr;lr7Fk z1n#0G70k!SP$(`)H12whb<^mFOb_R;z(=3MZUJt;lk8$Sow3$T6tg&}IrRP|LYOB*>Waq;1X7H-mPSmDn7pITxZLnr>n;>M` zGMXZ^IZE5DOii-tT~`Np-};w>M6L%@3YJC21`qwr-sX_Xp*s@Eu#2f#LMlrtzuz=g z*hv{GqvoZ4sni-^1nwIdDlZz#MB)p`T)x8+1z0O) z;a!@sw6Y&vi}8`?`!9NZi4Vsh5T~)V3AYXy<(#G!N_2$$1Q#Q*yudxLUY|n?bu_u? z%6sCt!f_XR8r*I)!4&D#So7K&)h`?Zfc=~R%7!fWev$q!@=oQ?WUsO9icth z<;iX9*K_KtEetprJlW_2d%L^J((Xal+V<)MpsW5p3KbpP+7%4*dy@Nn^Bd<|4ANVF zOH3^&gP_PNJsM{zASQG)IVW*Uuz~P>VH-Hv&@*8e#&^ou7SJIr(Rt;hr+X=tu<5o5 zkA=EZ(yM7*q*uD(U6P8%m7MvG6k)~}y}@amBC09PS;#e?fv2@p7v#_Ljf=zZ|T zXI)w9IV(j)Brt$tJOF7aa5iBR;8sQlG+XeSDmV>=t#vYZjdcb?Vt3!npV6rNl|l)r zD$vK2)kjiObWelTLygp(lA7dLf)hC1*oqWnHL=N8%L@AWp3!#w6`KjwiOIiNiN|$d zNaY)A6%%UUvw>z(HNsrF?N(e^?OF(FDpW=tosTu4>5d5SKc|Lr6QT3R_nTq>%KqGF zau|^q)$Pa;=w=A{Mp4mheY5EgR%-$bf-AR->>3r;-)JhT=VX=9)K5<9`Has=CvJ?@{*flu)g_>}_DnB_?Q&toCD`AZ8?6}n?crZbf?(7jRgA-jy`8Ce z-Rq4-G)^)hc%Wi7tpZY6?GX!HIlaX zD!VBlxIvG};B<+ulj3N|)kmklMgP?TQ(SD;8@bV~1;@kz+fOEnr*%3AB=pNwaPV;V zTboJS>XgkEKn2Eqk0oj@(Hc_6+bM9>2~K`BAwAHyOJf_1SmPCz!$CwXRLUYHYqe41 zcr07}UMc0Gte2YLnC)^Ev2)PA{ygd)oW##>VqDg*h-M}zbco9p zpny((wHS&qZ6eY5HMb@k2pkUb5J*vuae`6?%??AZ1aeEZI&FudeJ|?3qd~u1Iofvg zU`!8z#yvahV&5&9wRwQLP2f`0lc26Fhb6tLFi+m*`|j|Sosph(F?WXECtu%eBU`?j zYMuD|50f0m-n|M!(UaoZB4ZGSZW#VjOpdS5HxrCg8FOvB%;uJA04qDSx`Dc?q`tiv zO>+`qrB%y(@-9HdR(I8vy5O6dSU>Jbptr9al?V?;%bd2IprShdd^_7}hPjNPf}1`j zToI@v=*W@#UE5L=%dMlftWBcXy!TV`kV#Y%pP8O`t_)>MHU|!R4~td!k8~_>Ey$aO zLk5Ijbh%=gZN=aLLN z(wYZsIt`|#tEYDA9R!b4e6 z4@Yo3ls(f*2B68;33MKg~ZXUNJ+sh!VVcz}eVVNA7TS=ge$H{+hscww% z>_)Z(sXxxd-QpRq#9C)n594!r;Vvtu+6$V5jz#d&HQx%-{lNw7$?W6A<)=TJw0r`k z*S8le=>SnTM90b)kbP}%MeooRHDu=Ef!e^zbBLP}cu++|Y|fG;uj+VB@9K-yp|ihm zQM&5=YrN8Ig8lPR1Jr?ous~K1w&l#y7vJ=8UxZW_?4Rq0HZb1BJ*6&ph--oC-3WX? zLfgtdDD=UDtp5xk@V#So=k z>x1@lkG3yv>FZtRvyb)<-}u&hxsPA$e51zCckd6q5%jw)7_@`G6~Wn4^i69ArG5;5 zEn_Ya-t@<)Q}@##?3|YQP%eEV?N}RIII{UPR28vaREx71#`hzE(fvBu#f?dKbMM%t z9Y{AL@p=I9|J%1ijfF05Z7dUTyxo%R~IcIO)# zAn_9;NY_w~%VNjQR))p8oUe&2|8fOpjAKWOxY{(2KUKBcy@iasUrD>RBP_G*xemyc zsn+W5|JZOYJ9Swbzc=oi@q{X>O(;{|2cA*M-f-8WT=643^(;!4@Y6{)U|ioI0rX>P zYhO7FVH2FA7po7>iOk&zBQWu?V3#}R2L1iu$7{ZK#tvk9cxx&4*5BE?9U(J)z<@zF zWM+!N{L<|Sbb3d(!Q3O7FQ&7d^;_FG7h7G*!UKuR8y^6nYH01NYT^DtG2Zw9khs2a z0kRgl4lLirvzIh5V^*WUt6`!Af1oB;+q&G7QT8@$v)r-#@e_4hxP%o95&W&+M?BDq z_!=C3OY~Ne+EG|b^Y9FL0K@m=w_rrWIP*Xnm%xS*I@VtWBit{DO9ca?+)nbJRChvt z6F8Snsw~zOPI_UgUQ{i!1SFMB17aa~SUM_NU>G_1+O0S6K7+xJqK~tls-Q`J4fgTg zmnx^b6{-Y_u@tHdJHu`k77mViiu*k&;FrWHgH7&$$_T{A`6E?_UUSA=A%38VZKiIF zgMm-|L6>!NN@JdT8e1`03$i}JS0U7i4=q#*7yMQ?Xi1f-Ak;ZU=ScS$Sb*H)NGGw( zXtOzcR7@A>WnhO_Z2p0jf=_?gt6}fMo^37JGLKN`%dfMz9G<;=66di1cayr`0O zI#4A4@=1J0m2i3Xu2JoG3uP~xf^H@bI)}p{i;t?FQvQO(SLlHnuPc(ug+12jwzj53 z=)_Q@?vkWarvA?~gEXlkTu}E$ee4TMBbE>**{|lg?H`1e6VGkDFD@IIcZsRjY|e$_MIJ~Xv5_?v=cRCT5>uG%eIN0 ziBq!Crp1>`L85F2Kp+=Kp|nTg@~~VHmW61^BDcCSk|&)Azs3zVs}&&|Xf`Ad+C3d} z()g!ISR0!7soVXCO`~|ltMK1fof2{~QoMBl>zztTKuMpQt30 zIJO~zEMri`Uk>K*~ zEw4v;8f0qVD`)Q*Q$xU!$-X*8z_f%={DaJvIgjLLII!C3)Bdd!h0nTj9m&W0nJl-3 z=;C_g{P9lzKoQav}cxW{Ukt`}+0oV^K_E2g;Lk($0uA3|F!m zAxq0**KZ&_IZ~X3ZKfKbriM#2@jzA+;!d;yy3s$F$*Jfs(;%Z;oh#v(%bCbDGFTwW zKBOU1lM!RfB_ARhPL}N}CUj5WMqpSLBvX)66XIUC7ub-gI0Mv>w+EVUa86+-q6ggE zzRaeY2i^on9y_sq*Uv2xYZJ>_@J&R2b^||gj!1wMDfeAt?)~%KyWrXcWI05to}I@S z5@0xrRJm>N)94cY4>zQny_`5XS%@gJ!5cBImbM3O8WcbLFS^2zKbkq@n%P&HzmN$S zpx5P#=jBVJEbKS&2GAVEkV?DmRjhAft#MzYO5UWag^ST(PJT?vI*A_%8?QHx;Yly1 zcmD=z#0qp28jK)LSuCZKYS%u&Ipk7Vn$j-S=AFP);G2wU$yhg<| z$?SjYd!g~{kUfsDoGw4A@W%{@ zX<$MPqKgV%;m`Lw6^UUW1t&M%FnIoE{$3 z5DHQf!yc0P%jV@%f-O7!&izdWIT(1CZXP|%7+BCEOUlF@x*)_$O&e;fJ@uwA`ZqW)tz!M+00 zvhuggflgs2>bJ|m^&a$-+xxR$jtx%$76c$j{T4ddp^5Vl)wug!nbDh>K_&YZK?W6P zf|5o!{wSP|DNk(?MRf&8V#}LHZlWaz>cMECo6BcbnV?ilDB-}%5`Phl7EzIgI%n2>qc*j5lRv2_Ki!LP=@&QU= zXkIs*I7?erUN;pBn{|3VeC|INI)!LNJ}IZkjs~IZ_hjVdsLzzUn38O{hQ&NiIPyId zV|!{Z^6{`0s@>G+qU$svA;>Dp1L8U|q*4=bRLzo|eQg|+AlY|!(M#-SGki%{6{WDO z4C>+$Dqb^qKr&mXtg^0K^Xu1|w9s}zmQ4+Z^T1I4yPiDj*0dGm!tvNvdiaAKp}|4s zMF`cqE;Zz9grc_c?YI}ilFuby5<{K;qM&PG-xxuV2AlObBtqnBGhx<>Vy9+&4m^br zSI$wu14>O23UsaaXG5q&z7`p6 z>zJBZa;swPqzGKubHmX%eGS*TY_ph}VREwsNwSPo3bo;c4EZ?Oc4cfeS{0DL56dsG z-S_9LcpzKJe0s@haR=SnyyfdeL!@wO34UCXVt_VU2_OFSl5EitF78Bsw;C66LG2~o z07;Y}bdGiWDTuajB!0{6g7^!!bp3!#Uv1W84$tlbbVTm&elW@HkQ`PmMGo$?XrxDE z+fEU)&w9U$5aWs{Dd;f24gZk{{KU&Mo*&^{_y?Hz<}Ti$VA1l!#WBrKme9CnZETtE z%^GGz0@DM%n+?y17puR9)`9LVy{mv$t6xsb@;zU#OQph)RyrXFL93rmODdp?O{3d* znkUPWP?@6@J&|F9h=p*=^swg&mQ3K2xfcYboKRi5_b&3c{B`yocrrH~rTq1S&y)ci zh8_SWZS05#5=A*fJavk)rB+c7C^Iz7cHVnlh41UEuF#wt$q2}@&mCL{?mW-2ZVo)_@hgR__4-d+5 z;=|M)X4&Mt>?UsqV;6|*EyfLGS~H_2G=jId*P<4{f6NRp${$hS@$cvuV5_bk1PFs_ znAoRa-$uL`Fa+@)Gp7UJwrw9P50qcSmI5m1yI|i+y-Lt~KMyZbhG&1%*g|^&H1E}@ zBGFQ-X>F_*623ydBY2gerv$yoFWywRgOfiPtf=yX*+5@0_f4*uQ4x0mK1%9aHU)!b z!<6q~JF2y#-s{*hD=52jw1tg5Wi*z~CEZNM`fy*|uqdm)lSbH9*{GP&WwUAX@)MP% zswK-|ws6Z!Nk_ywix`lhiP$CmSLg%{tnJu8GA24c1vSgY%Y61LkMIHS^7KJdheqJj+x4cbyW`RJWEY3jfIQ>m0bgNeZ9|I zpDh}`tGNkiEELOGpclCeGhp|Nbli zI@a4%tl?C~&!sUHo87xrWb3%VdJ`{1GEd8GwlpbfyT{PR1;;E+%ad}9c<+C+ z^*O<72@Ri7g9bh>v>*uB9t_x?IGJtQq`nNPIB)EdC1(dN{xTI{b%s<@{gW?Lv;o)- zXh|b+`JANe?s-gYAc1^D3Qw9ae8HLn{y_X>)bPxhqF05im38zJUK@tt6gz0s zwTo06$Fv^~Qnlykm;0&63z7c=y8i>b{{x-B2d3zY;P}}01ApKr{2lb@z;WM66nrlp zy2SA9sXJddAi<0vhZRy7^JU94$e;;k9R5dd*V@3r4cLtli4Jj|8GU|j!t~N~{gdH) z;E`y`=GP^;E}P=4spF!@ENwOXx#4VRvca9NoZqklf%G4-%qwwhk_OY*|xguF59-v-DTUhZC96V+wQX2^;MtujQig2 zjN%R`ZU1tiD(wH15Vr|ft6t5KMI z46}UljT1JW9YD9>PPg#RpMai^pFJN?2^x@zJ3_i+Ttj(;2GDNQJLJ+kBkxw|_DwLY znktUdsy&m9KxW2JfB#BXcHtEfh^uvycjJoEoDUvP|NH>~()Ig$q*(l^*jBj>J$O%T_Kp?@bTcN;3cbJRzoC z2cViR$!%3KAj*O0_kezWc4nx-Wux9gwQm3$q>(3Z4VyX~ry{6c@h-?AlUp!NdBj*>aLK zf-x6+BMjWI*y{+OI(j4>dd$s)9!m-r0kk1N9w__Ttb@?yWU6l>t#jm>WhrxX%LuDBt zF!lVK4!IbAzNy@D0j;vusTi&~Ef^n^a@a77Y8q_PPbtjD7i=|-qP4Ze>5wI)0pwNL z1}CTbKPMH_FcihQyd0a9$&(VfO6sqX+-D{}N`+NtbO&JVI}yP3Y)6xolD?QkC=2_v zzEQRl#lag_F>A4(Q>s8`TGo*uURXYnBhxXI%(OTt%<4*+FN+9zk-=;Rn%4+5PV@XQ z&#@Eqs**R&!e475Ujyc8oEGn01`Pdt1ckYPZTEY8AzwQ$q1>5X3HmB6X1**c_$Gw* zYAEuCBJwu3ANVMW(a3-?rfnGV=!kmZ$!=!3t5C#TJEs1|I zPzZU6qBU~l&MXtJ{Ss=d5%^)QXDj%IDU_zSmba;&-!+GDp(TGE#gzL22oQP;x6Sow z6MBNUJ3H)73#`dyEz4C?u}6(Cey}y7 z@Eaq{A8+S7jHYsfEjQQxPP+`l%cxHX@XiRE@bE|Qz(=`)+z%80%m+;V2X(&B0cT#< zs{iArP@32I#G4t$T{qI=H9sE2Gp_lT-H`rPJM)!Shrw1xNvc=LQ(mN&VJHRWAd1|U zXwSBIkHGgH0baoy;|L$#Hcn067cAh<1J1z&_BUoT`Bs+S$SCdN9+RWVlP)v;tY7JUP+E5~4LFIeE?cNq)LEUMbD|EC^ehBL^vIg8F}J zqei-3g?h#Vdyhm_KH@djJ9^KrU>47a@n_)tVKU|Vw=vwt@obo2y1-`nA!qq@y*76N z|6DjuQTTcXgh{WLzfTuUoIRj;WKM&8O%CRb4&vL{+|j>3+W&ZS5N>{R;0@+|;Sf&2 zv*X0rZD$!<<6V3#0Is$L@xvyA;Zw3z(HqiD#Xbs|0p0>LFBDU_(>3jIHtp~W9zN95 z*_2m|S>SuD9MNe*f7Y2WohK1MCTq{ze|6Dd+tNZ!JzIWIls7Ltq0npI?l-o3105x8 zmq$G@fKa>tXLn}%_>4KM4_L@D?m&U;Lzr;S`v9@Pd>vx-q5S#%?E7=L&VBW)3*5=z zw|A0}P)tkXw@t>%pSl&RkiYg;B(p;iFTp~GVA*W&aq$J=2CeeeqGC${n_!Y_bg{)Y zfp8bDlyjFqq1UqH^FJVtEZur}ZCJn>J9ben2wLH$Yi62z7Bw?o5o&P6!qrSS7=fu* z=%2x&D#1*9yxfoczrCv9&7u4Lt8I`EH!r|ml|NV)hMS+G75hOrZ zLEh0dw~{VD-5d1B5&{e=-Cs-@{=5;;O|A=GdCt&qS?6T%1@i%QgR;zB4!5n35~Mh( z1x}o7)|yZ)%Y|o_vP^_YO^s^)xZ9VdFfJuttI%OPqk z4>4{Y3$b1?kskR^Q}cPY^Piq7BhO1?6hGO;2^2^WQ%R6@@3ORq5nD5%onhgeS2-AI zq4oux%PCe8=E_Xtauj|F!P)hEa>S#a2W9jTd|E=%P9@>(Y9D35A-pO`VNci&oaX5- zW%wQx0|X7oKgt23raT*XKVGTE$J@Kmafajq7#$w--}M_?-YpPr_sj=2D~>+FX>s51YtA4C zeBY{qFSxX^9Up>1%;;_zFkm*;FlaXR!O=T<&=j;(lNb_amsr;F%kT%uzz6QwGDooCk;{7K4+Mv~ zUQgB9_O9*)O{?A;4#DCdix*4JnDhib%O8)WT&pC2V^W0XaY9BZ!Okc`#<4)2AS3l( z>01Ry7BQI=Un>r5x1~51zHFPA8{5M1xVoRf`>Y?--)Itw zKppg2pS-PZ2ncrhc=v)5NzO|P+c2nNo7>*<;YHhA42~^?3#+$A%?T;s_tp znZ^78GLRwGA*K#B+M{XRw*3BPODQgnMQ}r1p!%v+zvQBgYU%_e^aw1Vk=GE2J+F;E zy{$s^Yfx$M-XXl^O&McszOYPU9PR5OztQUfBCpr2#c|VHBWL%{Z4B5^8y71A+dV8jYIH$)95P4M`sG#Sn+}nI$VEQU9$)-MXTX-B8G!ohTpVn*~r% zu)Zvq6oacqv@zK^x0izZbrs7LCx0`GjoGVa&KcYfX$!(v$a}dn&MaRuR6o^X+v53< zk14nAw+ruvU!O0(?SO*yMP{OUDfh6#_jrTBTZwu?fiS^a(FRQ-#TbJf9J;;%dIEve zz*EtP2z%csp{%8To-&#mwg&$;m4aFBMmm$hw= zW|nTLFW`Rn%wiI=+;7k0L4}JbBfm!wVT)&Tm!(>m4*uTDir0M@S%iu05AGDjg2!+M z!O)gbu~A-H!ItlgAuC%&SDsA&g&=JxUntn=T}9rhdWNj5>=huGO@^tOSG>kfkp}$< zOrggn-Ui9~#i)45fvs4p9RPE#C0t0j_tdt_Sm-0dLnktdNb=%-Q|+mzVp(2Stz4e` z@_ix1J&&i|1U09MQ3URmdC=;NebWLWIu6~&NVn@4J%K!s_>0X~dY`2oKxzA)<1z|E z$oyjm9-B`hSRs)qCO#5;Bld>ac8oH@3cQHXc2s>p5o?^8zb`y8<|Zss?j|jAUFg** zwn)X?pmKDlmBa{N1)KYl*%?1h8Vi@%a>W?ylflOr%SOm&(Un%ki5EB9@vWk;(asBw zNoBs(h5{1O>HfZcvNMkrz(%_!PiU)~vBtpD-fS?9L$=6x$_J(xBCCr;uBYp`H|IJN z=&@K9-&ZVkM<_Kp2|cJpuVZvg|2eKtr&jFxSB1R|3_sfIl~8`)yAFE(SvK*>5JjbGAVa2#Hb^)D_A0VSy9v&#Vq`tAv}R?dhs*uSUW!IlKBSqkyxPU!q?{K>M!B+f*Y zrFLxcv##L=>|vmL`nP(q^TBWE<6R0S#Aq}_pDI7aYx%%#A6TC0JW)Y5MN5BJlo9Uo zH9BU?iFCgs{Prj9mH`OAaeeCuy(Df{rWhd+^aFdepC~Zi1N8DW`ULDmxAG0&xZ7i1 z6l6zCjLy(=XXo3nikvW<#Mj=V@j>#VKrXpa!lFYE6rhq5c$oE6^uuwNBACrmKjbNK zhg2pG)w2Esr;0F<^rN0-$Z{qqX7vx56XlH;yHRfsq?j}DCX3p&vH%f~NYMQtFJ(qV z9@Mxe4x|X(vBFafq}zv?J1OB-+GBr6b90v$2`BFOlP`8VG7AB*w@5aA2+zpd6^Tay6O=P3GRpCCX| z@?gO*Q?6myp;BbR32_1PlTsiGSbT2I@A&hTv_(1@FqWk@D1q2mN8e=aXZN^uuZP@A zOfN;czF(oz1^N@@s>q@FutR0kRl6q>xg_*}H4QY7FkPeoqn7;tvyWq$|bkOl>L04T} zFsYHKJvS$#?gp*#V|p)S98U6xYg^6n>q{&a#lOZ9*R?I0Nw9au=~Ffrp=8-Jqxr@l z>YOF6XA5U@aus-P$(+@}jeOs9IHqo^#(a(DjGJwJUC-%Idr#J))b}$u*0ZHMFkrI& zkbITwA^5@(*21+ZB@SX8vD7wEh-%#12(6qclt_||Q($c=(T}-l6-f=x_^nRbXbMX? zd=Ho0J%Zba-}Md~*^3rNK}rJsD4Q?ViZ@>E{STVuf7|%~wtle<6IR^U83(x(hkRzN z|1`c_5ieRxpBQ}w2*TeEqL|v#5&!C-H{cXv3M6)b{P+(LL8M9zU1UKdQIvi`+n+}BmUZqF~qxKR4pzA z)*OerD?a2};_i4AhARwtF&~7fSXOPiJ zZQK9bG06jDn&@90gPo!<3XGLPn9jqRLKa8J2`2D!V7SE~j&Kv0^ndbvfr$MV{poL? zDe)_G$bF?A!l%SN<6{0Tj`?ped6DY69hwQ+r_ZKM$tM0Cf=_&6@EZ=i-TpWYzBqzzW1Sq9mj^K8&5QScxEm10txi&nLfjBc=mGk0%h; zi!kd?`h8pYwy!epcv9^CJRfh#0kQ8$K|Q#$1&W@~6fDvcxP|~~!Mxm?TF~D5PQ?A? z9z*PbAU)WShO9Wzh79lw%9gSB;rYE1)Wl!T>9#Csc}IkqBhg56;BR78?Pv3PYGvX4XkBDF{p_SDREfT5ODFJ?8oImde|L$#8Ttb(ddz)5!W>I4uRR91@&d^0*>|*fiqyK&4v& zWhPp2PDL+kZ;1K0A|P-W=DQFHi;qW;yz^a=Y?#|$E5%@8HFTNYaY%Ds4G63tlRKXi z^CnT&Gc%(mQv#urb&B7??Ij;&a)Zr4mS{Hu0oOx@a)}{346Q%~z{nVm{hG0|lvE=$ zQM4|L!#4nY#`xN`H0h(W=$P^d_qV4-q@AFbjs?VM^^*iU7m4TQs93?MuB^-07md=o zOZka*JQnVjh*BDuoEM(Yrk%B?A?pkum75QF668}S8?=R^tCtyq;VQElQek(jGcQX; zI~SO39SzN!p=T-^^egOsE6`USq>3!V5?AhBr~U!hkMwV#R^7!X_yU{Hp4}egLf1J_ zogMJxc7G$-Qo8}3qG`?D`$qGDl%nh%=!CW^#G=$YOwHeG;+N!fvy?=fJnZc;_Ub?0 zoV54(%Q6!yb1YmN1V;JCH2N6O=8d%F1xpjvnRs;-ndIVGaVMHgEBokFTw*WscrJ$K z<^nH3D~p}K?3W&bq*l5lb_;W{*U#uuF+3ZIN3)EA@3QmfmT2n*fuySF$#c$wpL+UC z*c{7r%#OX|btBEoguxLK-`Y;@2)=uPF;%=+m78@ufmKb0;jF47d6Lr$8l$L{Sl+m+xh zM_Q)5R_>x`PDY0mUt`rpa;OH?NAM$6k!`J6F&*sgOf99hHV(2vh+98rSA;s)xiMK& zL;~E1Y&{*V%;9F?&WSg=)TL?IXo6fqXD6V3`veli8?VaWBZTKikRzoEQHlQ?@&$_n ze8+zDl4#PfAPiHpW+-Tx8&<_gFyp6++7h~E54Sr&lJ2}C@nBDSl@Q%<`*_=q*IkkF@hMzQBIDN{w{)to9Kj`8<(Rp4AaI-Rv<*^(bm7T@cot4pP4z4Q7iNvrYiei-D z_R6X%rbV9jHEOeaz{W~KjM32V|HvR~q>ONbWr z^xS{XD)us?6r4KRt>E%@!FZ3x`=gL^yJr3sSNd*M8VW&yL@cY)T}&H<+z#srfF8H^ zYt$8ir$4l)n6#*PXtKGr4Ns?7uq7#Ou>4ghfL;4VIFC!VtD~$m;@)z^ZV6$I+y+DD zj^F!Tob@|~s20vhH*Z9;Jg46{t1fW_zbHO`!Q4xx;ybL=ZAG@rY-N0=OI5c^=N^z# z-J~#uE7B1;gDmKw%6iTLb@sj`pkSEi_BUO|vfi-*vE$eP++M`GLTlJL1RP0naMRk;$tC`h7L`{rEoh7;bkyOzHp# zj7EP{PZXXwF6C>F+rX@B_y(ONJF0SHOIgtdG0A0InLvD)fKu{@knFx1;Jp7To5sd? z&CxKC3ubOIOBsTvQ(O9AL;9dq#vKmB15UNzfOt&gP*|DT{RMcef-qKwikO%FwrC4K zmsSyg2@Ci>DjQQl#ohr`zHIc2&gx#K7C5L|l&CoKyUg>8Q24J$`>*Hu-#!WD%O|z0 z{Gh)EMgA*p-$*BczykqYlK=rxrr5z#V5A_}V&VW6(f?dc7;hOL0TX}MUxC9SHikrI zBEpCzL;1#r9NEf8K5eX@DrrUz+!#?M-IO7%)1z4htJt9~-yDjAmbA1scv)3drPJuT zjcFzO>9w6PnlUQFe4T`N*>$z$G{twk@tNs$>D~D}g(?S{N@0p&1pGK=nnNgkofBjCEe@=dXb_Lq* zbvrzMBHex>8%Jb4k0lgXxF&(Gc~%G^guDizc2cvwmLUqtC9zw-P7q-k<4wOLm`d{tz+NEk-`2- zFrh%R5bs0{aP!DcEpvnzJt(lGZ76Ifz=@yr&lY0G;KX@KB1O9{ajc?Fh8rN01+@eE z;-c?55_Z)S`^TfRwlST!MEgh~L8_3QZb6C@@vLJD>d zP=MVIGao>*59<(=xViDzv^<+_nfGmmilNM;GK`2x>bF8e4qNz6C0&C~^TRen@&Ft(sQ^Gc?AfR?oTq)(O`F{f|MraWf<3*UnZw{!&nq}}%hkcT) z{y91Z3odm#vO&y9_v<_Q)FmBYj?15sN)uUjh$}lbO_^qa6+9aDVZcP2c3R@i1otjB z#wC9JMYDkoDK%4%CVo(HE&?t>ff32<$V3D4lAS~*e6?c@IWy0l8<+#m&9code6dZ2 zuT9#_=0{7`d(#SQe5b$5{52*#VDXv*9r8)wM8=N9V_N~$=#O>tjRZ@C7sEPIX~FoPnS}!Zd_DX`}vN_8=fyu zic$1h!p)b^LVKNK1ocKHJMe>?`Xv))=VMWCwKwH zNHR$$w`wYz$D?2xUu*#Yohc_PXfB0m;L=Cz)ynfl07GC&Dk$E9e1}3KAi2SD} zT3N*Rq_?WNZ1)ju?Bt+{XIf~Nxf1TLNlf%zt=KSkXNFylAYMH{am$eTkfJ?Q0{_@B zD0)#{5bq0~g~}wv|icj(4#*zTK_>H~rvYy@!PU>&{K{y~wxUhGS)?jZ|cv zboOIJDoUh=t~kgTGm`VYnaFbpdW0lzZHDw&B;+^Ev!e1cOAagq2MRoT(uXnubGTqN{d~J3 zmfh9weP9k+m!<3D{j(zR*%k8xXgV^3>$jP%b)i@q#5ytn)OoVp%jG)M7KtF+xQ!Hx zJv5!f2<)~=928KHC~)mO4CoUHR>{!8nd3EDV$ zHbqGqnAzHZEcsK3fhrN7#`(SQ3@xLs^6b%vm64kbM<+tRAg&Quw4H_u?W2P_Fo&VM zo9=B+ncZxY73d=;lN##Ewx;Psi#!zwHPf)3GEr(#r!fXVHxrHVhqs<>jsR{X*xX4Sq;&dg$UUXrHA0C`lP%ePgmS5?vRK`MAEj_Iq|=vu35OWx8h z_}QeL;!ocK?i1<1ac*@$9Jwm*gjW!qTiE4%PMr8S|HZL+FwpuqcN4;x_p4m&6=1i< zHcy2*5R+2>o{2r(RczlfZ*Gbnm8h8r(7K)1T-bpr3c{0QZBJz3wU1)ht@M-4tlQZw z*3PZsE>cQc2*XhwA+fWo`_YYvsnQ`pf0_RyRpqb9b z2A2+o%4&T^&=>PXiGpOAd)`W?nF6DS$MYxgfx<8$IS3w+J-oi*B|6U%#}^;fS%G&b z-{a|0kq=g*52q@4@L?w(RzbJ5T-jYyWlrUf+!Vca^A7|~06LvFgWTl@g4OT2r6mKJ z526Lib6)TSI)AVGm7i7K0FD%`YXaQ#lvS*pSXJM85EC||ZNFYcOGxeMM~E|b zlA&F4$XAL&myBG;Y;;fC(Ao1n3H4~l7Lpzra0P7_B&DS#B%VdK7(u)tWs3S%&Z?s3 zuGu?l8^U7P`&~5EHPPOD{zTaGK3*=vC|@)d+%7@WLZ!pje1!@GX5g$NfPZEN*^6mM zt=Z0CZb%%E_(2=|bD*(LJpm;C-NL1I7s8-Sb%^|E^C>1{ z2?;6!8IXzkIX$k6+d)1nfN5U=`1lVecl4S%0DMiwR^QBE_!BDcX$I8|M&u!=Ee?NJ zT5A@{)y||drkpuNTt@^{8?M?ufp&zKNBC3RK5#XEsQ3+1_JHl>HKQ(Z`St)wUkJ%r zW{r`j?({?g=`E`ijoIy{*arU?U_DGD3Qu>>6_Rcbxa7KLc2T4r@Ev)({%YUmG+)dP zX_+Q8+A6FFaR*1!9(5g&Sz7XJ}LnEGSib3Db!qG-E;rCp+cFxiP z#GFP)wba$xfDT>Ta>=0-ci0leE@3WeRqbbOFfP8qC&E}IG(1OFP!P>RC40~Bu`8nE z?HKdt(jn&G3%IdW>>1Ud~29JC}CuIWA+&(E)MPK0%Ub1)m-Mr^1!0rI_eU; zIoO)5F_f(PQt|tP*I#q7NXK$65>tYk_C`2kVzV^H(odLHkI(cOc59go%dN6L)iM|q zzc*IJC}n2T+3F%VkkN`>HSsY|n?8=-yI{dKykK%Z`>_8yOnjp>bNRugiNF=*ap)a< zdc?bt(~qUr42X3K#!;$IoIr##;16x6k7?nixcgCDoi`B`#I&86;GtkJk-}`CIRRgo zjG5OG9<;Z&+uDWDszG$yX_~7R0$0BS4@$UL%N6lD}@a%g%+nU(xRcI=*uG={d2Wt z1QOO9-xnC705v#CQ8|kZ))pmrB?ToD9TksNWoJ5-xu&tbHHeF&15X#-YEhQi zMgZ6lqvBd@hNDzclhoMb$QlN0zPY0?y9-H6XMENebD$iLv^~pxBc#MNv&h;Uc@RZ^ zDQZ5-I~;!zxJ+r?Gd5gL*;CiHL4z! z9~(JMiFGj3P+~YUXh_Zfq~AtByLEg98THK<4q`{}{^tyq%K|X<jHj|Pf42VA0x2Yd0Oh3Lu-@a1)Hm#*tjr)Ezy_AdZGfkp z-#orXj56^rLZ~uLIKw1y`%D2PR2A?91E@02-#}(n1MJRZrx?8x@1s=h7A)HhZXX8I zvGB(fR;cgNuEGsRMfYYRCdnE^Q}v@QOsst@h?Do1S^i_6&lqO$v9x=FuXbcgZJ z!XTA3;|*p>=W^zgF8ZqM)guW`Z9QK{6L!HZ&~M%1&ZQ&I4T9r{+5i5wt2`&?E~y|8p*yT` z-cofp`y1>qiQYq~?Tp-ZipYFY8(xu#2t?;l79x<^XN5n3&T}frytJ78ty(W1K0*Jk zSp1IyeDP<=!%KsL!2m%)K>_`vT90b<7XMMLmKY->kblKxDT?BMMNJzdaIpWVRIoEr zCNT2fVk|;y7WH57PkUcAN+!f1(53vWx_TlnOti7VhSMzdd9c^v3$N0D zn2y7pr?Z}eZr{hiPt!^P9CZVb0Ig=cj8>*YOq-6TZ##aC{??#n5*26b_l^7OZ0m6* z8rsbu>rR+ju9e)TMfIf&CMoqGqpor|h9E{YPsMp^J3=YXM+^MGACoL|GYt0Z*D-E< z;DL3}#`709>?^CToIHzur1lWde(sab5amtcvNd`2xcGwfN=e?81!|M z6_B{T@b6do7i+!oJ?FR~n(Aq5(mS*eZ^*qbVDX$Ks+cGe1`<{D&?58UERijHC%x++ zbvrY~1WrSrr^sUX4;9rt^6g3l`3!jvgmfMYr6)WfBs!?G*VR2~`LwImaJ&541++HW z@JcnTO7zz@EtLkE{YXJRfES~Hyp*J6F4LBbZ4J2{A2JaL!dSKB-t@w?Zd%*1of#UK z4GZ+vU3M*Z8^Y)MgW3cZctq>a-_MK)_#9v7Z|hh$P_Rlz^jGVRiwp@l<8O+dRJbs% z74TXFPg#drO!)U|cGU`Pf|GiN)fFjxgs9(&savE$j^}Z!Hj05r0SrcDQ=Mw7vdvm| z1alWc?M{ubSu@HMi9Otx<%|O8t#gj){cUVn+WHXnMQgYd`gF9X?kns2WC@_UoogY} zoZUKTc7m>mK39+lASSq7?byRU*3jDAU0@*oqfuBo+zjj0hkzIDL6J}(TmuVTB3dAG@0fLH)T!b&+mcdRcewtoFmzwBM<%Vti)cr~`NPnqQ`6Y?4P0blH0 zc{kbBTHdtFvSTOA-Jcd{RNQKsk5~p8>lLWs1U50&G-Xuy8OLnlaf_oxf#uYkhj6dGk06Drrp&H#~5E^64u=&Vq zif3KQns4}?{OReVhel?dJ(m{F6XG%L&GJQZE9d+u)U z4RS>}%s4|?4h3)2HGXetTwy%W=jaMf4=GN9cf`N8>AxEOzgJWLF-_exMD`&6_?kU*mrA$I<~Cd}kGG(wnZa6V>qP-sCo~g-)5NuJmt0uTXLC2huDLNcG~+~<2H!!e(A-E9*_RuJ^kLRdh=%R z$9#-(I`B~K^VnJwO7#US4-Ci{TNURro+6wv*3ln%z;6!OtUTx3=lb$KdF&<)P$M`c zNN&^8C|=wC9^~NBSjg%_7Z0TNZucckGku$nqmZYki6FOaEXRs!`$PO-nW8m1LzsDR zV~0p$8ohSne#_&}4;^BaaR-TtxoTJJ+xvR*Oh|2V`Hs+eO3d6{xPVK0xp+|zwH4s2 zIHLXmfc`J>0{Sqv0FA-+@2d!erihP{uE=+iTOegunz{pFzY4CU{ME20#!l|oRLgZl zNt_xvT!M=LdCZXqmm&XzU-9 zwJ$zuU1(ZD*SV(}1|z}?WhJyHl&4fic;Zy)fO{|VR3aHAGbxEIq=at;rOW*L&n28i zf>unx4j@I(HA1xDU6Q{gGo?n|;_`4AEm?drH*w~q(V%~vTDs9+3PMx;$`ug#VTbJK zJ2UFir!uQu71LJrQFY*JL8jmirn8-Ehl!}7jwiSaO=UgTBAtrgnpb3dukiM)#*r=O zuTbsph%PQ=#_`qIpY96UWVyaY}tx&jLQmYP7rm;>!MLScZTx@{dai%GjhJ> ze%oJ>8UO#JQlaTw+wE)9UmGMP?k5s{O78bB5XR9a*o=Q2)@&@eGsHhm$hX=@F6C>h zc?~{=NeLD|g|G<;rDdX=y#?YwUJkJhDeB)s5duWH|E6}V!bU*-C&TV3tpC5RZ6BWj z>fgK_&EQ{@B>#-`$TN2Oe>1S3fhU9ibK7@?I>tKKFQQGnzrGJ6C02<6rp1;BZXfoa z%VZ)v!lGYq_6(MyAdiHRa(xAkm0}4A!3KDTLP7BcQ~ZX6=nX^|gk`4~R8CDTRB1eI z&g&F?ys||o@XH6XHW)!z=QSEd6ERCddfLI~P)v|0(KS1qAcpsY`#Q#$vOJ3xOL#XH zv31{J(M|UPA4bhA&R9KFbX!E~hEL8o&Xh8H$UKcTI0;tKlxeuO)SJ#r80G;KIT+wb z)Xuyh6h>q$vE1Bz)`4-`I>5-WqsxY-8XOO$-Vd(ACjP@adHx%csi_WiSq@`ojZW*C9oTb9 zG{tMm7wZ?8t$J%D&5%|jV~qcetrnm)9DIMXNj}pZJ=l11A*n7?9OemIQ4A}1NEEly zRU*|Gi7J@aO=nHZrAoG);6pI~bJhd?f!DFNEC3#$H=}g&O8?~QK{QFT!h|aP;%x8k zUC+3kB-I%i{V2j1b+G-auoEte%^;vueH^0r&=SOd{dDTvB`?ywd&7uEObsA4g>e>+ zrALc=1kzCCZ!Yw*j!=>=5%~hsE{OcCyhvITLV7f!dOn23adCvza=yrPxpKIG^>prN z8cFsBTb@Hj5w_U6M5KHTgy^@z(h5g+OIP2(79utzfXH^GOBypCQG3&yEIgrGf$R7fJ^jY#mgCXMpl6gi+{wbH>&mVHS2Z zm3*yXHDu*~27Ol;;*pjW_C6(dI-Y8K%6yCa_&7dh0II1ITTv#Toof?R8p_igt3UsdCN^nxPQAZ^|mE|GN<&BE3Z1hq#Vbe@EgMcbP#N1xIF@tL~*ppFx{M*&|# z__^}I7{lEFZr_Hmz5N}-%sCZ+P5^D$Ck-*~I)Um|`Lcq!vt)iky@+8;+Rb7LYBul4 z$Cowu|7~1Lg)bI6%ztc5EF1*dzf)~39R~mZ z%&%e4a|4LJiqV6o)Zswzr%a3RL4J|+5^?|Mver1_p7huBdjaaNKF^O)mB}{`vBblFUV}gwuo6|4B0&kcq*jKId52jl1 zsOGaq2&CVt%8g2u)>W>ij`zM=xUk1AtS{Jl?&k zfUE8hHbVMl!rJSO=bg$IpfPyG8)e?Z(SkQtdwHUk3DXx2XJM=H4N8ohoE(duoprOw$phjE8tU40wNUj%!=xb0a_*px!pS(mW4;(9dsP z+{%NWuti7HuPJSt{yQIEe3(F4+rlnfg{PLP8+-YIG^P1?F6q9N?HAo(OJ&TFO_E~4 zNqgV2{5J_WMK!V$FSP!jmdtB(rMS^cXsPWSry98}yj77)&9l=do;C2cbMsI^Crjk> z_jb$iaDERv_Qu{qfbpLLka!;AG!?UWB$NF2;_2kO)E39)({&7-AE0$HoQhY~sNxlu z3mbl~KiKp++Ei{RbIArCB!^GbIktTJLT_=GiF|ei2Q)6GcJP3=%em#_8vu2gy*9Fj zxKd_Jm=;IQ&dz$Gtl9FmdoXJACxa&x+AAjU0O;v*naI$vwJ4=ABeu*) zh#zt!S8g4OsXc>3+IrQ~F2UiRMa6|oRUWMrkKi2qd$$~<%l!S3%u|-PJQBhEjTy@b z>-KUvGf&}LidfJ8e4zhV&;G9l3oGBV27RGEW1s>7(R=|pTiB(jnh@b9r(q!gyqwX@ z&_8_|I%MiKR!B($-ICRyao6A_J%Ysj0>$U9DWMA+m$=UnMiV#FIC-Rd@@<+HG=Jj~ z%CCP@RjM-YMM1f@FITMtuX^0nqHTxLtK^cl~;M zzC0*HL?8r~k7@xg#!{VFz&FzZ(C%Z6uE5Z3R#RxMyYQ_V7CL)5R+!i=*`N#MUSrrq zU`Sh98oHLBma_;y0r&&s*>TcEgIn`(){L ztMwZ$Cs0c{Gnjv!T3=+JbOsqe@Q8Ug{4^&O`Oy)Vtu1PlPHzi87(Brt)2~6vSZ2t0 zWbs=VD@@@8?&jW|c#$Q#Kw5p^ZANU>gHkB>Msj1tj+Q>t*m*n&(JHI;irL)UVa zKPQW6TB_#A#zI`KER{SKU}jlN_M&5jBV0OgPn6%PNK`sV)}ByNQ4UUyc&1s97@bEQ z>OFD7pyMLhBi{<4CcT@+wTna5>21I3OB}6b%4@XVemZbg8F{*(m1UZ6G_Zag#E9#3 z0PJKjLk3k!;bN%n z7JpQHpV`%%$%2G{{FyTPg#f2{vz$zqY^y5RO%y)C=%npy%V2a90Xelsh zE>{Jl08fi$KEt`AW<|4-7BfmSf$faj&){_lp?}E<{IteAQF1FXnMg=Ig~cnG)ysAe z+^~d8bu4rkqeD6XaC&0X-Ze3k5axmpJ(jyu?FFnMko~&j+=IGQ}31JDVwi% z=hE+h%8xT3Zr|{IGUU4)_}k5lyW;XW9>E~!Yuq}@o`i(+Jsx)h88=%>(#x{RedG`A zwHbN`rqp6kU*+D*qpR9@NxyjhgSR(4AsAEO_*yWSNbSouz~pW)5hU=(;Mw&@+*JJJV#48b>Os_*H#x? za#abdE~)7{K(jd$tl})9J|XaiOjQM8E-Af#>K;8kZ4SyPH-FZqQ?GxDs5`ty`XOu} zt>V5vZ9MXwyqk>05%XcN$(T2Rlr_@!`&KL*z;~^V8)qVro@7)8H=CulXtSBS!DuXC z;@k8pw#wZ4m9df&gRq=k6`dy(;^-3c(36Cerj3mwfI1Amc2)mktvESS5@HY4>KaP6 z8mOTe7V%i=3K6WjlU4yehA}D1xxzF14ATiKZbV)BOUP>_k7uJm zmB*qxz^%i`c$`kixeR1~L*ukEIv?=Juj_cLSM~^y7LvE+mfaBz2jIq~nIzfdtny_Z zXNXl}zt4C<3i;r)TJE$Axse~&H1FIyRPfns`ZKCa#o-htDr7EQB!Nb^Anq{38kJ%h z7OTt|-()eL0*?zM^;Obk4iQE_gEG$3P3)Gvbu7JEDSfxpsv zEPf1;BmLo~=`Csbi;W&$ndNM zK4gX-9)kRHVsL+Un6VmC?IHK|_1snU4`6B&YSf2Oyz*j*jn`!~|Moo#QJ*~QVU2f*eE-a0vB{Ue8vl^tYUuR@f*GPIp!BL2KE42#@J_(Jeb~shoVUKG#eQR@ zi?Slozu^7dK#GHX^AuI5WP8+cb1nkGZy39HtN?2+C@yK!-qk48xaxe60P62-$*c>gTokr@m1f>gl4jU-Z&l;DCLUF?9PD z{8BP8%qTRp1Q~srh&=mn%OWp7jgWV_v;?zmlhSMEZc0f2%%Le4LkArAK?^8?Y_W5CZ@YIc|#fk@sUWa>T& zg+fMOt0@t>q13|!K!@F*)IGVY!DL{hs?19v=ev8`jl*q{mWtx1oow1=OhxHpx!AaQ zzUP>83H*T+9Yw@$CLYFbu@R@Vi$>Hc6;^j*hYI_bJm3Y6atMd$GIMd6F91?c7(EP( zJ$}RA{oEqgHkTlo^WhExM!9{4BBWa$2OGhV5((~hS0Qe6!L$8tFWnyuk!YVsZ(D%p z=qEiH^ef~irBu4D;if&-C8oUna1iMmxwTmwUK1>Fae?; z)UqRnEeFC)sgMj;f~3=%2Zjl{(5QGGibxoWP#Ve*RuR~Kl29UzXejkyYFVOTG;`lW zVQ8=!s(xBH9|e;TIj~-8Cn=UJ<=u#>r$JOZn<=d!x)*vS1659L3Sg&J)jy}i_=})= zD<&VBjI(b}Rr>n58~V-jIztC41}bsKJe*Lt{$a;Whw}fC^$x(5bzRqJl8$ZLw%M_5 z+crAM2|8AXC+OI=(XnmYwv)bjp7*=={r>+}?OnBM?{)U6I&1EA=A2{BG0N3l2XL3k zUgFNrct4O84P!97#P5A34dvOmn7jWB&B0$|7o8BTUu)Lre#{^U+20#M?^#q|8+eYf z$$cM1(V#>Pmo8B%A}-PkTp3jxS1mFQ1gOOD7DARMuUIJ`=BuX#BaG@b&@?Vr&^rm! z_WV+Xe%q2P++vDMDL?j$qv;-jokFfeyZoE@6EBi~g~sQ4HvikInZX}qgm`rG+NGE+ z`Xq{%Iw03W!Egr-Y9;?qp@VdLa@mK5;f-=ToZgKO9017@&V*ez?4}Ea`g_com>?%% zNFhGCKS`CSw?V2c$&yKF1@qP4e?sj4?X~@%5gEzGUZU4mtY!rEFQC!94a?O8`fol* zTb>u|=If5txBoyLPH(Un06BAGTX6?B7w`YjE6yol`zvT_Xh!ypAgjH9pSMp> zKBnlGqO@RhrA+zNy`k-CW=Y4;WJ&oZ>a!Q)6gbmQv>nIVRa_n}WzuoXb2W6weh-0TDT>pYt7LN?n*_hUL>eJc%d{GVlnq1Si!ngAx&nY#=Wpu9wt#vk@cc%2 z*;?K9O+;&$g+6Pj!j`)4_K&UBFdMo8EQ_9%dq-uAf@xyz=4UqH6889E=#rTcL+E=O ze4$V8uX43!g7NJ<1T^v9k}kOtxooNriFq`dyaOX#1yIPtaN5mSukWcYg?=9Uji-z# zd6xmX_nrx6OCJDY2D?O!8DY>Sjn2=Z#}4Pb{;m)_^ke}P+S0*^M32cz8HCMVdpxdM zIWoC`LXZ7Bql}>h&u_JbNTDXWp?PepSCQON<6e<(eZY6(J9{SmUDr~AVvJBy2XKCA zu^@Gug#suOBVkX4MfFasHu=aI#GV?FP181<4b?`|aesgejOU3PrLdk%>C^F?Zomy> zQLSVvDx)RXgIqVi*vlSHmv?0L5{jwR5r380I@V80bvdDu&Mm+|HglF55g(TQ@d2wZXhlSYKnhWCI(WE1{Udk_a7RQSq% zg24lo9=Nc8HC_A&KtmQ3NZ=MbKU%XK7qu1mzZ1Q)7PXF0K|uDHfh453|1)u`1{>)s z^C$q>oEpspiuUhB;^B+e{I4#z3d;ZUiMRmmFBOLajK7Li@+l3--b@s9K;&<0IQa_db1GSrV6X|;DGACc*; zv%zx&t1qhJj~n<&pKBTd-zCP+WXHD#kBY6k*T!z zpb@6D_sEfIf#(>gKVN$yDu2(040~B9#q%2b29dh_UBDL3-PHz`=3d#KS|$rznd$Y8 znD&WBGl82&nx0N$0dirRWTn|0aSqqYpXMBBVr3U8O7j&@<)}ps*2?a({IkUXTzwhC zREeU-6Jd(=N^IPx%pOwLX3WGI{&Hg7@83B>)%sB8DkPyAkV`KRN-c(RN>7x7Qgu|< znj~>l-knw9-qw&szeH7Nc;;l5w4jTs zA{G2c0BWsb#3PdJ$DsoZK%KmE#b9LYzGbBvS{`4=wP1fP0g zHY+PB_I;Hm)JP(EDlZ$~o^oP$TVtxkk1`hA1^EGz6c6wq$s^2UA=K0h ztuPVGNhsCgz##mX&X*;%b5+?G*coAOeiVp_Ty!| zyW|E1J46TlJM;&IDLt9JDK3FGHtSS-$Ww%ewe4y6J!@w81G;G)Y3+{N5ztH#*SpB| zouQzYW+l+U64-*hK~|iQWlGiA*K7JB8{h`eq%eB`bJ~SlX~MIhKjIwCbBTXI-5R(y z=hth^qZT$7R@OHf{Jud%t}(2AV~9Ch@^s1aJh!p2+gx9)jjzh&u+Bzx@bDf{e0kii zb!aB|=0^0zFheH&VloA#Ea5*ZMss8H;r4#$)p|(@&0yoP_GH^S1?kZyOL;FZDH3t< z!)MAI;8)Hy@OqItAN45xN?O)ztCDoNYJc4KZruCnD<)V=Soxj<$1;Oz?xYl>g7Jqu zd1{lvRIoaF<$J+cMQxU1HY11okMPmM^zo2}JQea9yu0aeN+~M|P}=A4ZzYOH1Ol3K zL2L;Z_Whwc)WyL`8Xs_Ux=1 zhhlt9AR18=()amyNr%{fFev7ssvch?l=N=FWqjZid(e&^)$@WiY1{HzM^9=%eS%7U9Jqt~z$f~`$nnCxkh{xXT<+IXA33CLWcJqEong?| z0ftd|pa}Em!mIkxD@fZl=ClMb^FU>5I0qIFq;-L&d~MRnSq16*E@+KDrTWpmGzSN< zgM3!MVx1)Sj+CT?>IOBNxdTg^Hs^e!m`rrQG%1y4T=D7bA1Y@?!HcMr#PeP4=~Nac zbCj8OqKytC^3bGzVCF_!v9;DgB+{JL1E??Is1L`eam%G6jWn|M5& z&oI|zWS+uKp^@l6e7?(Qkw>SQV04lC|6C1$MklD|JhG~K0c z$bzQ2_5|IUpwehr(HJ<;%!!k&jg)Rk4~>~AA%85s%1El~HZ-h=ijezj^OEkx0s!Us z@e44corxFpZq6zta{jI4^OKjE0d0Z06YPgz4jn<#p0_QpYjs4bj<=_vCC*XnmS{kmKW z(ci?7W_g_Ps(LPH*4&j>3W?n8hl=$~oj104hncc=0Lj zX*!#tDzRoWw*eHwyEYL@Y|~$SdmFM=jw3;jtZ)~c<(}=P8)KCLnL z*RFgDSIHw*?#8c(NA_HSM2HK6QNxo+i7o|hD_MdDR6~6*x>?Vopd_-g{y`Flw)pTC z&9=QF*_gXkqJyHjb8*HNRRKAvumP!Z%2+YYD)=a9>qfq)uQdHFUpJs#WSU|-e@%mx zccrh>>FYEySfRf+Qdps%?=$Bl>S@0lEsB{PmSDvo!bBHVgKVmm$-#JG=BwV|J1u_1AD+xxPalD{O~|0E`A)~ zm}cAAh`;v}ngv$4bV?ZrEVr$?C&P1q&~p??t4LbjV} z5nZ@I|CWq2ON*?u%*@VDXKx52h&yq!o0f1Gxs+dhY7q@o-ux4$I>eSpZhHEdTJ|Ty zaLJ9uSpecbC5X)(={(c}dk<6F0>`aR%X)i2OoHS@#xG5=nj%sAB?tpX#D1kSXtVU0 zE>vvW6D4~JNqzGsb1ve7*+^d;w}KhFVxv%9%vAXm(!WY8`uPk$@QO7G$H6Xj6N+$# zxiyk%*rWgp0JU5gGjn9IuJ7)(9-6rs_dcUu_$yLA8TcgjX4Dgi9UFqCV5Q+bh(fp)6B0T93G#eP4Dz>q%%GGv>7 zXk^*SKa87SV{Xt5%a^%cQ;T)ajM7pBB!4iCniCROBl?La5i`rJLaszdF9N$n=7Egc zzC+XALX+nBq6L>EXTIIxcPL~(Ta(5#vXuFW({|vbRy3sQs+Il(Jco|2Fet`45B;V1 z^Xrh$YyNLszFflF=xh|e=mFI~0s8-5)BgtO|1@rPy-XcrKz{)(7(h@76b^tP{sP8d z#P;)UR(=HR1Z)Z=eUJD?mZq6P7--o#8R$A>RJ;C>OAxe_w2O1OfCdd))5=v&KP~$_ zKcRCn`xqqRqJ?E(w&LHCqN8#6_AypC_M$)Qt#PhhM?aE7St&@>7aNELkH# zXmYJ1XR_m6c_^rI+=&I3xdhoc-7Bv4=cRnTh*^BiY$}*uyi@{yAuHLB+E~a4jW#&4A4K=bQWp{I~IlXjO#lpiBLev?`H@jD{AVnS>QyB{%LBS&&U?e15An3dE* ze!5Az?96T41BzFR zx*PgGN?h?+_Lq{xVVgTMU=#I{bH__~5(Q@VYK#4k@b11l10fl@x_^?l!GU=s51~vO zZjOfMPL=bR>kifdVk?e9Xamuld=fJ^5IGyARTU-(66+oOp%JOIMWsjFekM`S{75_P za1og(ZCEX~Zlk%&Rncogr)UjAB)N3I%$VhpmrN9@21l?{2lWi?xXsTTm5cNS(-ri3U-f$B09ER^_bz6C5wO!c+VAUlKR{W-u`_-1D?@4E! zP$t&EEyksXRn+2PxoE=IVUK`#ouxL&fZbEzfJ0-hbilTHkXF|2y@!hN044se*UNM* z3j^hxltJR)PqDM~9P{&=S9qACG&g**%PuCCN-y(d!o;l!*yTnIcaGec#-JWioZVwH zssuCTt|zKLAWkT}h13`wfPO;-0~l0Cxw?LBM;8w<#cyK$Vry7~@C%^pphzuhDrC;B6wJenuiuP{Ti=aIvK(I|XO- zaSvYdFdAi-d9EFA$b9m17TgZ@W}!A@;0z$&)c*Vf*tsJ8B-zP+1_J2>tYMc8IioqL z`|luXI#gOaK^Dj|Gg{I&Y$nwN;fZ9_u zWZGMk`OGLr8MM}AvNfcTpA?JBHs0G~myDI3pe=1R-q=5rSSsX>=XI&gODnKeN*dth zg7r2BBt2Mv?>{N*U?ePq73hDS-bI7ZMh|gowoSRyXfS@^#jvBe?B22 zLzIv(?QsU;WeM~l`BzzS`NJpB#-f$rKH_l>4g zNRBOTdYl}L9rCm{zN&pQuqHARRVU9)kD8kv*F5G=#v~7c>l{-4na>=Rl_* zfZs-TDVONm^@_)C#>x$FwKe-k5LVb8!?yEwpdq(;R#= zOshsy(uer({4dY+5h1&~U-@axD+yo6mTr`7;t!bFr?t+ZYi?S5LeT?nwdA84=$mkq zxGVI=&ib)|k*huZP@GxE73R5Fv2#`;fI*VfW$}92 zs{JCAYR{{}EVhe~UQ$$?ZALNdw_d`+l1|}rC7x|a^g}U_IHLg9sHJ-+6x(fX2Lqn$ zla4L-P3Qxa&?s{|$6DTOvaSGvyj(E6c!?5r^guk_+0KdYmk+2k#>dTg z;N<;fc=pr0eRJA2pbjdZ=v%1T(@~xq8>T7H-w!6A9PMFF)F14MnC%DLvzFb+D3?I# zTM7Nq(@)yFRJSf9cKj3VGuRf22zt?6jP-5|G!X~{3<`H? zcFz@V`Zhw>Z3fyM?-`Wg+9? zrt@ZymLKa3d|ZjN*ie0s?zOk4>Oj2p;LEkN!@GN?Z;4h97+d=mNUPU<=f$V=@h{8K zf4zqP?LGd>31R+#eG7~D&(Pl>4%_dm*eVvNFrezxb`STYN? z>ks&oG8^d7p7i_vyW<+hcYOm-8dg130HWtp3)2HHOi|-^$TJ+!7j+<1V9tRER>2GG z34sd3&Yco%bs3^=6*P>`A=ruZjRm?ySNMU&X1`$HzWECL2Oo*&ro)W}>E^oEQFw|4 z0z5tQ{uGNuuA9B2x5EBg!MvcP^<~MhXMfQ=Hkv(GlT&vbH-}4~s~(1wBS7{iKz@PQ zNSAsq5CZN46H=+RCOix?rN(XtMT4GSA|!y%ai|x&YNqwLXa=6Aa>S#x+3`;a5j2%q z>czNfqA;~p6Fk{po2l19rFQ@*d}r74lGo42whyXr3ePn0!r< z?ACPOHE8~TqEUkso9{p}kSHPqjA|Hvh?jkcdDaXlb;LSYD}uFr%F}gX&DLf8C2nql zMEp)&RBGxPqDi9v zRb(|2)xyZ{*C>P)8G%ULt|UbVkH$TqjIgSW%$UH{NZ4?zM=@O+@(=e1dz*!7^g0oJ z+Z^)G`Sh_!2*3na^;>59>Z*I=arJnr2KfK|)=l${{T*5pjQeN##m_>B7RqjS&?1Ob z7$Zzq(D6M9D9m3d4HY0<00?{K09}!>3q;>QNASA@B2TbqOwW%Nszq~%Ck!iC7kEk4 zG&>&)(hQwR1+yw9%URjdgVvRU@rjW%x0FBz7W|gRUk+JQ3`Lk#MU;#)(&;_TdD<^| zxx|Xk>Je-u)?=){OY<~Ju`xnT=ZZ3G;Fl^X%Br;G=&^m|nQ#>h0hxfGxdNt`^b?i5 z{!=^- zhB*S$V}z@Owqa5jfc%O#>MEDL&Y)dm65H>J6f|>ag|#*fy?K1qb3C!js#LYakvK&h z(PRB@jV^Oi{y?7rLHQoo_JX6i<#--U2;*mp_ZwJU`|3*lMq4uJIDK!|M>9Qn!_1k{ zm{G6HGI3^nZME-fO&mK|eZd;v-T)3XUfsVoDob@IZhpZp1MawKx{>=z7crQhB_B-o z79m(Y7G`3`ilnrzV)LBiAuxi!CWJ>i+?N~=0cjzTHBf}r29CHZa&k-{ih{yuHI&ti zO2d0JoWUg_Byys!)VRTXV^}aST4>9N0g!ThjC~P_WEzEbd=Bd+Ef&(QG4Ke{JanoP z$#R-KWEi%9930F|36si|K$ZRYoeFFn191dJY({@OO*XgBW+JOky9CU#6bf9Wo+tPX zzLe}u&J33oj3I+^PlvjNJl1|)G~?rGw~Z3D*U(9-(zMG4Um8+f2u2*F2AkRuA;%O1 z8#b0RD8XOJlt1#niXX?7SUX}0ac=eKhGV5TAksAfGCfG}!Ik06ZXwN^H&QxDQ?ms5 zW?GjW#s~7{%$n}59t)$2i+VS0M?1E=f*waP1S{yp22W4eI=u5;t>)}Ttb%^X@4SP zc1!3%YtIY~q{_f!II0f~Tci zSe<*W5~0ritm2=@ONm~dmpnGRvYw_Tsu)~^3f>cULO8F{5NM%GK8hSnCv=N(uTqNs9X1Tj%w#8*7-wEsBd|4{X=4a zFIIH12Z^nmx!~e^jJM6c)$(;)V=ko z#H$#2@@N;3GndF0j6zgo)mR4x`9`FiLLK_$uo7`fV)vm|{Xpv^i3o%&-@`75-#4*63-gQMl? zi+r(rsSQCnlSuOSf@19iWotNYY7`r?iSliQW6S{Jm7g_rK%2;LdTf7yhqov@oxK{z+u~&wh;d?}q%}OvXQ3 zbu;ua6AmQ!zfMS4JT({?_`g@zX>c^YA?7i>$<^_KCNd*~fV^}62gh|H=EDU#{X~Ta zRDJ=uXzI9s4Rq5vGqW1dsjYY}K+#+IOQcrO!wcl)MPzSE=G<^0XA3B4Ag=HnH_QdJ zms|(jm2*wX>%!?qBZnfFs)nlC+BTmW&wS1=pC;ty@R>%~8$TX|?z+6UK02>D(*z&3 z!iYcvyw7RL?rR$Cu4~p;J)kskDTRNVsE-C;~Y$IO-_L>T6lQ*>cwG#Y_*M zvwI-R5j&yEk!XoCN92U$B;_o0YgpEDl2vHfYl-$-gHs{PQE|r`LH0=R4~&3cfEOc& zgA;%=1}T6=lp;S5M^? zGBSINHC8nDl7o8WIjS1-mgu}~&J%1!H;Qu9me$N~y?hy2j_;T(er?e}iZ>gcS7{!x zq!NtLJv{Sgr9+Ci?z<%shgaov2-kVr3nTzRb^D0xH6BNMDBuq@qNTI|=5WE$K8D3# zG%n`V^Yd#PBrlmJVNf*U8tkicq_5_3iC&ifYcu;n+dlPSBW?2*MS0G8N4%cw&mxDx|+b3h8X*@@vrp=lAnV zi^5H&O0+~dGOl9%e;EkF7Y1Iu5@zB4Lx_Y#L1B1^krSw^<|`)ctpOn z=B_1 zh_<@i4{!PqrN9JtbWsPPrWj`}GT-qgVK2VRrO3-OYQ^BWc-2mE*B1G8_DHPk4XP-q?#kQhcWeeOs$w)4DT$JUo7JI8S?BmP(S>JgYzJvF@=`QSbOFA7YbK!9$ zmxQdg$QX4^WWU4#@@o3?vrM8&1AV@(GBZc3Z{cV%Kz;iB&Q_g}bNmr$>u3yA7KxF? zF-P)Vb{++;M2S?o)n&gWHMPWc?Y(X-`UrBgYBk!+ZLLHUl>qi=$`yT6Ewk&S z@ahVQuj7WjMjoZQ4Eu7E@ zh}#C%hzEE|o&EqPLe)dmgm&U=pYCQvUAAa}WW%&b8zO5JkNdWjR5-0o^pxo{qkDPh z!uO0MxzMByRuv~b~`mG%w$@4cC5~S*Gva}T__5ySM2Z3VBam7E=P*ED^j7ik-f_OBaK@xx`A)9`Lu(%=C=~1fW)QQ;d(k8 ze8F8;(qNg5Hkduaa_)c(xVYH(D;1fZpaHP}< z#mT~y3BNg^Fhf`N*%_KM0Jv*snyKmOxD^*gKE;KFxf_c=HRzkUe|sd(nb^)^P2?)Z zlMtobd$@7LHLS?X$66$I=EA^;OVx1AH!q*399V>@a;&SNJnTG!XJ)(gb+O|mIi@u+ z`kv*X$;binrdk>}1{x?&Z-?nTO^Bp!RA~!aMAhf={FNtqfx;J@fXlxSEeE|hKHzi8 zOr87i9Av&9CD3eBDEDIq%_au?xgFv^Ir6GNAm{d09FjXb@d!rgU#q@UP-|@ZUE8;RBCWOuGJ774qb_b z)nHcfxB!uL3t4$lT~veCdAY>f<o~^>kwILv5q;r?~xG80b>cKM#&F zVherURl*H-f!WlLe0uqnksT()!{a>~my$7d&5|WJzYZucH<4f|wzE)wr~pmPwJ0gE zSMagBG2N|z0Bdj=U7oyD+`>AN47w8>6iVLk;bh_nfU3%iH!?X@>jpTjTmMN!Z% zu;7dK0SBmzEfj<)$BC7p1hYX)+D|PL%KU{)jc&oj+Tk4zd0>X&XGUl>987&-fV2wW&C!jC}pjg#?-$+;8| zR_y%k_t5ROFDDM8*&NG^8sve^VCX2w)jVrXI~`z$cKm!mbRdtvo&5gXra;*m{rhGC z>N^cBEe*|}ZIK4vpD(?wN*jUk5TBB5%3U8G9}vtfkk>8^R z)E6n-v;GKNxfNT~`57E;9C$xK`5bbl50S1~0&?VS(yyn$FTZCWpD@n>1ww}7Zs z8eA{h*!a-J#UYt9N$aPf_(wj9t!Tt8erv#?mNimdL)fm=TPG!gp{92hXTQu3@WP82 zvI_&fG;#z(D6b}lsN_d(nIELF4Foa<*qz4LcDjUg7c@zaZhC8%ngtiTYfYpYtHMc- zzWCNiR!s=kbknGG7uD1k$=tJ$dm6-DUABudI#1L+1HYT#V(|i!*DBj{YY{|e8Z-gg zgA&7_<%{3v^9S&xywR{;bV&xl%=-Fjbkhv;(ws`mTna%y1MUTKK_1F7aFT3j#~`~k z8uR0mCShqXY7BMB+7{nR?!OiIhfDaPpVoM%o80EI@Bg4#oRio2D}sirj`m1--Nw_p zWJr2I$9{f>;`L!gS{=8v^YB!14^;s843~IMnm-eA>F}oyTzV`psD~dmHvMjtMt1FE zNLo!W(rM9J!j9ykEx2DBRA3-9dGi`E?SXOD$kw`4Xw~g?@6!w2u}Nf;iC?XGa@~fX ztRP%noVPTW;PX*DjVN&bkOX@`MPZVmqG3w);Ts2PnK0%g|pR*epDrn4p!b$q(d zpf?n2KL`VPn*rP^jOv{gooWEag>%I--n;(HE2M+UZ-URCsrpIoB9w`{=Xs7jLg?0x zR9d=Jl!m$+T89QXQ@5{6i+mA_i_*M{i#R3sa(cR!JFC2WC#$bimiC<)-Dio3Km1^D zooIagR-illnc|l&cssfvlMSK_48sfzv**vAcnko)y3bS-e)!?xI`IK~{Dz=AKAGc} z@(`=9#0#Htm-wd7GRdCWy2T7nIM5Dl&M4~3d_Z{fG#Ez-3|iC5`KDc1eKB@`)dTIbV5JE zjfta0{fVvEvh=?kWj3EaY765r0GWmHMo~+SEyTZs;sOS6TQMdKc5s;qs^$2M&6lGu zQzf)VZ^~+ujJ|gY0;W$lt?o-3NJ6R`osdGeCd`W21WY}O{Fi{%*A}E(Q)WY=Cs6+b zwd1|bsiUmQxF1Ac-8-)>L#XM3A{M_GkbPukap&)I_lsyq_Axo%u;%^0&!->*J? zDf~q+GFB-9$%pI8t^H4d&405E|2r%FPaZ-E3$+ONRbW&5g^dXR)8pEVWPo!63jWW% z=I?1J(O~}>UfmI+LHuXPtHdn)KUaIynYaEY%Yi?PQU;FjPnIK{e4?ZB>v4-fk|-2x zpiv?(5zyd@8WEt3{HdU;)ozzdnzM~iYeECGFUOB zFYBx;j!8B;C3MbE@mngbY~ZdH2l1>$|e?cB7DFm*$jm?<+X%}f+=3XNgcs#iA{nxn6_I&zO43N&Z zSX)Qm6%4S%3}oOrekoq>=iEBzucL=wyP>abGuvN;xs3pKEo{~~(QU!QX# zVuQ8d7~csD_2!?yhK9q%RGzK*iGyHHKUbEWixk>mmcOQFcT&D!KX+~27^^q+jl@hA zmXPh30gM@4r%3F$eSMNU(9j)BSr`7)ea@VFM-0GDS3ww6;3qxm8w^Fb^P~~0qwUx- znROmwqJ~3;zg2d{RwEy}GnZ_6T;e)r(Dbd^coF-xo(jxvlx+s*Hiv*y79~v&oesJT zX7_}G$I#X!*%0qA){!R&yD20Mes5*YQhNTlXtoPeM_>^=Jt66-rP=6tkulKIsgGHJ z76_O+I>~GBWylY~emBH~JPDKuOo6A*OyLUHti=B4JSMvmuRFA+7qD@-C(ybSFmdcl zl_g)Jfvl}QrfO%3>;+-`mP=7SLpV0JQ1JlioEGSE333i{Glgxd6EE{sk&k>a^gc>5 zDXzh)ath@pg?sZ$f@n2-kgKHj3On-SJB#8Ynry8h2?GO|-E14?VGU`-nK4#yi!iS+ z(-m^t2yRXqd*}KQ+ANlzPQm9FnFAE`|7Xcxpv(W6-z&_xt^f*8p|}B1V#ramKa>X9 z)(Ue-3+;JbeGbR3^gIEdyu6S#;l(_usk5>py&#uIEHO}&qnOqfa%ys4K%(g|QHyXX zM#x*&K71mFkQ=z>Y5pFEK8e@1+S*%2(+I4n28W-v4l~OXbBhj;)2K@@q&9Yob-71 zoz$8Q8{Z%s9|FfRJ4CVG#p+t(Z_b%jUBR5>?Ir#M!S29C-E%wdK=2Tn`kLjSX!A_IrW_)vgmg*X&|(sjSoM*4xCKTs~H#)G`_-O`A+G|xfe(PQEuql^k zx+hQu&J-L%wlx#X=nUtN*SKl#D$zKPs;jy(WM%X1R;Fa(6{^#(4+y~BX))K1)I11@ znlHB`e9Cnk=FS4RQjDF4dE~PdZ75a_lox76o4dvC#W>@nl|-!?6`#mOuk_68;U3d8 zaOC1+E|CDyoOPiA;-4Z+a9qr2_4`h6_5D3gc&;!@j0ZL0A9$=(`f~5b^#^$^B&L+* zz(nTSl3*BrrWqH4O?jG;r{sNSr~xe)1T#5`9wmyW8zVW2qrYlk^pDBQwXDRtC%w}FkpGy{zXtPV zN~HfkQ-V9wF3QXEq>B5uCx+axZ-eOYV zY16gqitlK%y?U$Tm7trULd=;kTY@EW4OUQ5TzLi;00Q()~1!6|iooa{b-NthOfUZ(T>1zQ5W^)&3G(`5N8JLKs*c zHFM3hQ%2XxD_+rD9k|3_7z0K@txwH{*%cJ2jF$5}EbZb=);fsl{2Vm?kQf8Kt^Vff7~PYOKEnfqC8 z7JX9%ljoMq5`%6`m?g0D_sCXdeO#!Rk>Usjd7+pAgcXjZ_TKxPGiVZ%F&V8?Ov>mQ z=1u(+w@O!#Kjkn{w?(TtrApb{!^xnnElH`B;SkE~#jX49JNwm{=)P~EW67S6WAX5f zo<-P%(R|<@SNXK|w*bNK5sDfVMF*3pPifX~tSnI}1ZzA05@-JNqyN)MUjMbv{=beP z95l#3(GQXV-@rQ%0fL4O==k+l4Pg79?s2sXhdE|+0R>S}QL8b|!7m>~YprLN<(VV( zC#+N=Zz{IipN)Ba0dvI2L}`A^*=J-SFAzdl=S;VB>_i!~ftyajr*$O5x1}p0Ay9sa z_8?Y=7AL2AFsx?1^+5tGxRx`cnVv7HPEj6$FD7^1mMJB_nI-GqKEQqOtS>cohkT*f zY(&eV2wR1RIX-oKQFf41zcmeP>{svA`QM2eh4-m~gcHcV#KHN+3Burw2q<*vO1fmP zM1p(q+pH(D*>ShMUy=e@Qe<(-O(UVk_BxuMxFI?+tm3A9E$*JX^e_Xq)>=mjIMwHO zq6#dXD#qxxrH@Twynsk3LPDbvQ3xynHI}Q~z@p%kFmryl>)}3FFxuVuT$cHmKH^;) zCa<+EEMu9I?2=7$8H^JsRTtH)Z>4>yDe%RBLChG0=EUdkrzWzIDb{TC3iNF1?#8I0 zLEFaE4UgGP5pK3F*%e%WLpsn!v&4tO?O&9|=2`XuCqlXK^^l@z;k6#Bz(XV&lJAfs z5_?T4e*zl2dR~8WEx!C$G1vdQ_|PE#Lj7>PI8yqSTGV+p6#${Q}dC@FR zNHM`NAjFlwTdfGw&=T8Ar)%Yk*OqKog@;GB2l15*Yo2>f`Z-c2nz7L;lbHKGoLgGQzah<5B4sgv zJK;u}Z80*YjXU9ClJ~+Y`CXPhJwx*D*p8}5sdW61k0k?hIMmfFI6ZkJUr)i=T8+fC zP(q!RaeTnPEJZR#BI-AF;Q%fJN_XL{R?&NVdj@sT$y`bf(DN@M}zVCbQ`2N&5W1KV2sH!#h z-fQhW*PJF(+8zP`eTCA5B^TC;sRAgg#HGJ@zwC`5f6bRlcj?B8Vq48;N;1sTO0Aj9 zQF`cp<8K}t&ypl;&?3Sf^UyJA;y@NEQFv}+h+)io+@J9}QPEmstRI7DloUL-k5BQw zWzL)z-?%-V8D^s6b2`4&?T7(WDE3K_5T#?HUS|Rw$K2n zfEhf?(9T%LZ*JyVvf)mJu?>RDd&(=RFE~6B)n6XOy$C>2r3^8^?WiwzRFotX;zmms`Y7aFR1$H zyFiS};N?Us4U5$$hLgz8Hn=xZBr!9owL3yN^L+kiGDMcF}qDk_`%)LWc&#YE8Wkjj!@K!tvMjzL^b{w^fN zeaF7ULc)t$gXwy5lGJ-CMfy|t3s_r(Aomn#}pI>CDHMMjY9BB6Fnx(>r&R^;*5iwpX98M>E#N43W^>1*0sTTr=wCM4nZm zWU+EeZ2ZOtWH&p}5JlFOWQ# zxbj6=Q<=xd+?(gHt+?9WQIYO5(^}DXip1U`ES%}aCa|@Skn3Ql=<(reQ)eP;0g4Mg zq^=NSd9dkBg}V$>o_m`Y*PceK3i5C!gaSL$M0D{7vM4e&6VwNtGNtw)*qiF#6ai8)Bk602QZhclqk>C77DI5?Q~DKLQm`%uz`8Q9}cDSgE(U z5DbUkR>=qG;A0BQcy=2lfYYu`?#2DMsAqNt0+03sGP^%W#_LR0ucaEQmm($b25D1C`7BZ8I^vC`?uPc7Vp`=yBaf&SiCf)Mrmtx>V)hQbV z4aLE=%MXhPmGsF;QMjxr!Wxl#dSvcnZ2c8)gH3KTDEBIUwj&6<#+?IkOz8U;SE&c= z3uCcL*ywLpiEpT^K71+nu zh`yQKkVKE}ArZQb?)A4Q0Li&0KH?o%$gUMpCnt09;#9L|Pe`%2eTbSJK&AqB3KvzS zKEVuoshPRlQVIU3wqTRz-yH- z%@p7X6wev%*2jeZ0tf(y#9~x)qF?Q+qnE_CxOVsO{+>BULOBzN>IY*mV!}c5xD-)i zJL^Uvqi{A<;7md!YZ`Wlxv0j)S1Gl8+k8pVZS6iCeW&0cSv72p}O~15Gumv**U#u^OP>h znTy!d)LJJw!M>O6YgK^=!=q?dDzL$JSP^g9}sSfS>3|eBFe`L-2StNAULX3R;L~wcF=_Ezt za^qXULVz{)8j*SXV`iMAMWdkmZ5GVnrwB+%iE-lQ-)_)Xv^o z=qG?&_cPEsUhWD@jibE0q%H^^JK6ePKWnq_bMLUf91uM=s+z@{KQ^r7IR*?f?vFPs zw@B=2#XSs`Ccui>Xt5ZmZM?m}n6h-Pp;1=Fj%?xC@xA?}K9or)nbfCC5#6Orn#tjM zQZAEG$AYwpaV^FJa(SsBwNUF7Sy58`=#+*u2>at#zmyS5(_s|N8akD%NAI{Re##qsqV$o@mzI zxtghmqi7{AZBQ*)aY|8}26`YQQO=aWx$Q4!P}X$3`Z=U4)d{l>_)gj=LR05@7BkiC zoz75Wy|qIPtadgmf_$sYP1@F*9JXSep$spoJ3z3Dy2{fOIn&Qnklc>}4^XLl!qF?KoA?SnW4* zH)j^QX)_@C<%(rAb>n3wyC+?gr_5gIt!v0zS76;NDbcB|46tccEMdSId(>yn*0`nO z8bBuLL8meh;3Z#fJhxTEy$lgms>J^@7gc3ePlb|EcfW}bW4nQuf+!$FAI#yCm=kT2 zJ_TiLOa(`R5*pLK4D~OxaqzYw{GL4Qpn2eiwA*(r(*t?$Y1?O(qRI&7O2D5o{)0xP=&hHLAo+;FtxT+ntlFCMl;pO@ zr-DtQ;=*}iT6O0X?R*+_=cM9tQJWMP={)8LAiC7TMPUc65lTg%EF*4WBASr>!rjBM z*-xVwI44M|AIk75R;)p&CJxtBa`KHn8GTfgQwdTwKt>_-S~3^Bk!@_~iMC*s6u=T9 zX_>u7M}N)@%=6DyA0&Ji;T)Yv3sX2KRxMOY_+X6%#R3~9LGr%CGuipw{C*sO=1Y^& zwI?I3XgulN@^r>LviHwJPd(Pk(*fmM{7*NH65TUk5h`93%Z7gvWITL*^Q*PZ3C=ww zS*K-8f>G~S00CVRzKea_uhMX8b%5D$=C<7wCkM1zZnh@FTNFc zOCyL$cy>Tg4x#}>6~5HHe4LvSha@m+J$nA-_ohkOQYh>2V_1ZYo$oDo_B5q-xI;Pgs21^_3zl0AMcj)Jn5mEe-q0 z{X+jFK_T^kVYcZCaVH2p;S2NdC&Yt+4tB~#+MQC{{Uu4x27MA6EjlXb_8c1c{kis@ z`U0oN0scT`EFlx9yh^jl4)~VCMM3IBiSR+^-q%+XFPTA7?G6V|L!Ur+a8c)>N11(i z9@eh4iLV9~v5n@rGo-G@*|a}0wG4gEC0860-NO6Zp_C25?P&PIWQLTMW6JwsFUE~U zL8mC~8&b!Me~$ZV^k+BuH-0&9{_35B^^hNOJcttcyZDVT89;i5fYnpHWBCES3&b^J z|FDhL2cWY#$lOS~2j^_{a530&-DTMGEgYap|3EOv6H6j5HPZ<8mm#$S53oc?jWK6| zoT4amv=T{D)#BRP6^%#s;v`+AfbG05FoS7TC4DBcd-sLg_*US_7(Ln)pO)D2R#%4Y z`a4p)#hi0L6v2lMxxIy^O{U- z!hPeprUF78b1z^^qrg&p$q@qDcl_2f+pVLAyLVXeP71r8wQ`-dGav6t^h(cs6e~EJ>D1ev!{j3amwvNFve0?B4CmV1TBl*Mc|OUH+XEiP!BcyO>J>hqlojdkp=Ro5hhy(( zV{(m_Eh)o~=`+Qh;`xnS`^e3ldb(1kj(?P_hSxnki}aCC){D6J*5J%=_i>eCpj-IH z+hIiBB4)g;o7a+cF|bx~X=y52B!_Z%px~=9uP*lij{~-mDjQF6O^EvvWt!&Tm#^ud z8~~)|6Kvh(1Y&Gyrb>%STEeFGDm=xp5gt_oD{4Vb7Nojr0fP-N6&A3IGb-Yx5z^cl zUv>nT^pFyr;&lm#&lGR*(hR!-OnX0)`$|a87%(iK8iThohr|%OeG(QdfmgeA`R&UG zgdKN`#a#y==OIB6X2mV)1oTU$SE6G(pn!%(d+7#C5bQ>czXd)KE{*kPVG01wK?vgdC&>Us#8<4O$=SwA))A4oR1iEyy>2adoayY|>kgR$QDf>{eIQ zB-1fm>(}EoI}A1^tTfRH{|rAFA8*YR&>#!ZAkUfXXV(r$Yrv|vAe1R3GM)b|J`WfM z+xXo$ABgA-%D7z{0Q@!YZ|i7G)kyaJSBoF#PJ%P4ZGr2$bVky3N?k5krtARtAlhYJ zV9wOZ8vVR%O^$INv1Ws5=uXTN`rLcCss3S=qK-s6i1wtV+~V~qu3B%^`ms79V39jG z6#2A?)0YtstK>7hTT5nb%9swdhEh9 zWuc5Th5p2LTJ9LATUZ@+Z&V#%>$C#7!X5A$bbWO=NfsU6Uh^nlTBQp=xAHwK zq>H#A5b>=Poe2MGoK7Nm7dOPV|BGu3S-C^2z?*f}YSJcX_`J4UBAoRn@ez<4%s8lW zYea!du;j(f|D?skr^Q36p}^!ijrk`Prmq2-CEO7CVo1FKdue*1C0g+JnMX$$Fylmf zU4dAR+8P;1tE%r_7a?;XKQl_0fq;VuA*_?3Y}VlhKV^TB}OgveBv;ql0ns7P00VNxHAL1*y8 zX<$OWp0xiY8eOW+OqFJM1b|}loiM|ny5o+Uzg%#2MkO!)CJ`<#&^UT&XrVHu(WH}( z>yHiLPEfo-MLfR}#RBbtPrM{;D~PToY8Q%!+?O+t%|^7eGsxXBk{V$4z4XvtMbxc+ z_fi;tV{Eh?S?9!0?6};p8zso3Oc%qvnk}wk_1ezXMxI0N_dENc<&$C+On2muEvbQB z&=EoYDgm%P5_3u*?YcVH$H>dE(~ zmKb#uH$$J47>^tk5qe{%w~h8mtPw7I*tbsdqCEatm`MbXOj{~>{&$gZNru{{5kE;Y zZ<@|Y8OLpIaR{7oTAc$gPN|FM_ELKpfc!6T$usDsMO0}O2{eF~__Q(@XscEe50WrC zr-S-rt4xE()V!TfM{|Wg37<~JY9Z4xJ;O+bBgeFfe#pfTjFSjYsF@0&-U^gGJnnRP zA*2Sf>BiunQ(%IkQZtx5vpF`P=y4NZRgEGNP1c7j3SavM69%!rM8+a3BpH2hVzLMd zuBCEE2&O=Ts4@WPlNM$+X)Fn%tytHzO_V$$PU<#i__A1(nA-kuQDrSU$~*gTJeXvo z1aZr@NGD;;4)t|*wij?4@5?UJR{Jr1)XvBx?uq?5?aXS+e81_UjgA}w`Rb!d9Wm;o z$wzz1hxFZ$4W^{UVvN?NS2v$lXXBQ%WLuiFM}PG>`sxGxR5HiFDi7A{d3{-ObEbar zAHd_t_aw!#r-nSjM%kzp+3&qa+6%?}$kInZ9{<$PG;ZZkXc+RgCKwR(Ua!1Jj8vl2X`F}T}X})BW!b5itmJmQdCI~=4X#Zh#rQpn?qNGfRB0#3B zeWm(5)Z4ivMTlth4Dm{0^NfYnzewb*Zu>%p z57*T=R}O;%c9!OOHYC%O8pP84+jvSY`>Rj7*4O7Je%U%@*z(S2Ic<%N$x4gfP;I_W zwjO1_3LRy8O~Ud&_XZ$?_Yf>n_w~CT`xlS_2oknY`82m@L=3KR5VxB5uEFt$N_Ilf zY?}MgugS@>6SuA6J;H-a!rqOIKE?dA2A*jUh3Ep!x3&2Ap{M3n;nB{?9;Pha|vq-(>s+#NN__w@{8uVIR?TrEZp_{`OdEa zl27QryhZ&***~@X+r^$D2oL-8!+aBhyQQx|Xx`chKS1rmt`+$jViy4`J6qRH5LZ&y z#IPT9z3pfnRK1^YGd+7I2d`w_B)uVM;9)HLbzrdhrGZ*vl+2}i?W@?=4-GEP&z%d=6cc0j-Mz3K!vBM z^xM}d>PCkMWNtQljGXONZ0jJdYOW(i(nzD}^HypUmZOx`I@r@8k}b6wQc(p_GF`OR zf*Lo5PPvh%bO@?H?@&YVIlp|idLShzNvgUFRY2t!`B5^!inWUQ_}M~Rd#E+UbSHGS z>YK>Tbu3;4lWt^5(ldBv+K-yLFNrHq@Izyzd8~+hapgun9BEpt@$ZqnYVylVtyc26 zz;18(+)nFXR9j+TkO;IKepMWPnP^X^;T&O$(MSPwtTm2DEY!zmrUurqlKrPfQ2t{_ zmqTR?ixJioggd3PX$0%J1K_=gsQcVsUM~iPd^pJHm28kMB$0XCPDYI10iy3#%ng>N z_Mulmf`}8cKN%A}yN(5He@HzaGDY~4xd+=eLU#j)|Ag{p9^mD1e*HI8rM$qb| z(bqX=l+(*U{N3Sr^)6hhBV7UZq^W}a2r+Vx=0GDtsKC+JHMF*4%8s_lunsB~MJU#? zfvjttH-lQJ`nyNa(HG^o3xHDCtqNavETvssN>REt(Gd%Kr~r2d59qD@77sJ9XtkJR z0X!)kX3!WAhsHpp4XAzM1g}Q<;qTs@V8NOik9SvbW8aV?+&Wgd6ADrAK7p6=ojQdo z02RZfO>RsNpOcoeeVT#1B@BVVwO4vBVF9LXOQGf&m*F+7Jq#&c6X8f!f`|W%?bZA9CwsHmn2R<49%t zna@*pGOihLoBD<0!dMeD^ALXYzq9@ciZIoL#2y`=q#UV1mjIY*Dt0W`=M9^rVfvBP z8KouFsNlNx8sakMnF`tZZN$d!sE~n+U6KZ+ptc4NcNSKJ4BNKE_9oV*)U)G8->WP* znyhtoR=dTGkER9$kFY0>U8^rsv6hU}E@R*#_4@4RjS@vOv7UuV{v>L~)G#Q3Uc+h9jel{hCup8Z`78W{CjPkFV zhta@VX66qxK_tfygLSE!xwvcn$;M>0;BAbC+7e5q9UnkqTW81PVGvg`;f$KJHkg}f zsVzgekoH&H$g0%nND*mWQzmjL1t`f3+s$Z&;wL|fIe%ZI|a zwh~!}YTE*=sI(bg+S+QjT3{}wNOVTn+Dw@V7Mv(gYav6J0}NgH$<1qVj&|q^JulIQ}xe-taI+3V7O)Q*HV3QKC2=_;Fq;0qn?9PNH2@!(f>?JE5 z$Nq%GEin;QEpI()2Ex_pLYAJurh1q9X12oBb0w&LGxiu-7p#q2K`7oHY=I^V)!(ug zN~@2?DxvzhXgftSb?JjJ!|0s?`tmynPrf(no10tWg0Qg-Y!8L`o+$HJN-jmX`y zWhlG4G1i)__L1b6;P5h_nTI?Gkm}9fXHaYYIKhPUdd{o52rVP+rbD7}Yy8Fpg1}hh z1Ol3Pw#ib#&b}X5>q37sWCQsOcR19h=Sxv}-uUt;t6695s>kcdU)5buslFt;INOWobyT(*im9+2-e=p5C0%r;HIrW#1 z7r7EESP>2N7{P9jz;6`Oz#qZaqsqjD?$N~~E%Q)M2)g~k>^+SZ)}WtlNZs@;&Higl z_69Jo+=&?zjQzo<^}It<1wZj8=U|QXqEw~)jI$RkDPk85fhHv`5TlYkY2f>f?!rqu zj|%pEM{*=3_#`FdpPopLywH|A`bz)@3vpg4Yp6mt{C!4<62A`gC5p_9FFmmDkS~rh zkVXQ(x+?oL+{yDRYcAa6zCH;2Mp1iuXxq&`aXR$H%r`;QuPI6giQ=vq4~w~Q-7~h~ zsRy5y5ZhEUlzxhCLg-6aL(Hp$4Oas@p>DNF;h$|=lqG!;Z;vB!)4f!czK+97QP_L> zd?n?Be*d4Nh29zm_2LEugSta`i@RMT5FUOb%sjcwTG@WUx$%e~ z%CRLEJfjh`+q*fzum!X3tue+6vh0VEn+pFeW4myHW961R$~QyhwxjwE<@rdY`mF^q z*Opv&L!zZE%D$2%r~(EOqYBVOKkgOf-l-n*Ly__)KF!as*PS+4<(exkPZIcP@q7nB zi`wCIwh?!FGTkB>_~NqoJ>cXDNIZE)UjN0}(J_Y8M_NsTOrMDX7igvzy)G<%foWMJ zglg_*gR39;t<2r&45ZFvfs=q9j-D@W>IfgUmdVAb-6c;-m1Ga1|h zW&%5j3T_m9ty;@KzVbij!Au<(;F1dg{CzvkU+zg^0-(Vq^|);BTkYeGqsC@8yfI-N z`({*JwWS)%57cxX8t%uA)joVXKk4Y0h0AXRe~|Oku~;jc8hn8Y>{Hze+uuGXm&+-B zFWGw&9_K(2Tn9UJ{Z0$b<~I4`l$zsC^IpDKZNYO(ZF50h(*sad7l4BYZq8*$(1_FQ zU0!I;H0E0Yp+&FM^NG*f7Bx1^9gD8ReD>x?n&m#?|5iy>C%8tE!**qeX1VCP*yYG= zbn?uH?Uj2ynw%o_2b6~2U(fDrBDL>pKSvD6U0S>1GyPy3IU#|7|Um27ty zPdmcMuys&aX0!EM%ioo=b`)9u)C*opK9rxjUY+aELb`&Xm+gHX>K#KC4hw7jHbL+9 zgkWZ!u4QRdCBna9M@rxzJ{!h(>%tZ>7q8N+-or0gpj0RAZw?k(^ z@tJ%CHWi*=Gk}5>Qe@29z+}SgPB-L=EAv2*kt$pcu}kRKHJqvKgj{2DI|%z@`3(?$ zprIMLqLYe|ePi59yBY5|GJk!Yn$iblci@V0L_$~+aS{l?=TWP{gj(a-w`Yr>FSks! zoX3IlyQXmS>O(Fp$zFLyI0vxwi>#^o9m+hm@S#Zq?j#Q;nlE*NJ8+PL3dBEq9?Ry< zq&8AyPN5O37bfX^M;THoj4LX9%A$3eR*a<@zM7+3rsHG?q7|Y8KNE>t6KP!eLuzNv-9F!L9>U5#RsX3WNB_%~ychY&Ly%o=C?R-r zjqI%YnM?g!ysFpdPoAhYP z>(RO&PeOWKF?!0!Ai%$HJDS`-ch-U(N2B5}!lT@@f$IiWqrBdrvc?Iif`}%*=n-b1 zCX64!GZ$XDL5ej_B2JMj$h3|Qqg9GT;mq{{e)-O6z;zOZiE`Lmq^>YyKaEGLr0SB79krMS&AId|oa4h6#DorQy5;;&jx$6fr5`#><4gY5dlSwkwW^0f3i|$W zOmbS~YQwTpX^d`2eYU_84lQ5-KF98#?u42IF2!XUHvXX)1iItaw^H5ID%qj?Ux$sF zQ<(b&NO0zQ0LlFiqmr;D-Akx7G)MIGB{+t1q-ec#Zo^i|Oi06+)cp>rf#xqUq7I*; zrqTY(MfUP$>0q;G!ztGX-8?r3v_Ic?kEJ4fw5DtQ;sbX--Dctf^7gQjUB=1F$b$+4 z_gieVS=a7(q+7C?baBfzc^uAvn+SHlNMXkOeLaxw2H0dacGOGD{+T-*0!m#`uJ9i4@VaU}F;K;U6a{b3m}H1X=QNoW`(O7-0E;NFeMQ+4o;3 z%m3o(|Btlh^e_Z!Fr$efT|g(?{0vALR|aO%5v=Vnq8CG|CYuoIs&4b}enPus}Xl3{K} z-*W=#$Jf24#(5quvtFN^KS6TClL+ck@0i0qkaKFbx9bDMp=A+lQ!3)I00^Q*jC z??7vLNggIw_g=<&9C&G@y$AGvbkEpBVj~N0DqJ@>AB55L5yAi&<71M4I_gV*Ds!vW4=a z0pX99e5NMnqdy^XMj`^G_@6)o6S?d0h+<1qc${pzoencSe7t-?Yr?a; zxmYZX!bw$Qv>A4m_A3nxFoT`lDFdF-`W@9h#eW3u$k#dy>HR3(?drgqAfIVjZG|0F zm|xWxwY~;S-*HjfskX7M8&NYj$7^cw8#23gQYKwh%lsgaz;@XozH1|k&?{ng>GLW3 z+cYyxsB&a_$(dBDt4j-2A>#6;HyH3e)~+)E>#&>qK62jj8yML)Y%EKhv~=ue#9Aho zjkoI1Bq!fN2Issr5hzG_%T&r@+{KTD;4s|a+va(Iz|3wO9P-m&(8gg!@oVy zVPCFaA2o&DH(d99ncdwH$v0Cn)H9OuGgtCGi|CMbS^|5^BJIM+Sy-0pvkBkoCuhl1 z>+b^t6THN31*6Z4>X{xCl@*BIsbi5MV`Chpn*E-Ntu1azf>-y+q$OAn_(?n-!(@R& ztvJTteIOcGU=e^vFi=vf&TP!W78zuY!}#Zf|Ah75Fnx{AxUV*y|8k4}2`lTw5>DWk zHo7m`f8-;Tn4$1fh*A+?0ABWLKL|fAFE=_fCMdrDF4Upcf@>JFCF;mZ)9VgdF(8T4#r5?$zMon2_4XaAcBILrZX>D2yY06!t%E?@HbMH6f1qTTruDP?s z#Zw9)Mz_=Dgqc>nQLzH$Q8w`E`JL$KJmd3uQXHh>rji+tIoFUG0YD5jsuV-YqG^cq zI%xl0ls52zSp;V>b}<%K6!;xR>!wo?p$W&)Fwt1^niuc(AT=i7fIF}sPol4M=$(|G zWI%vAI3_e91};5h)B%DRy}y5Vd(X&SERzVczrcT=g~dx3K@p}+F?44HBuc0c5c;l| zyoZyG)?dQTU%4{}2Pj`12PMyOAud}sg{>AOFrdHE=?`>R? zWGR&q%{CiLz-l2{p11HW1K+fgW=LoLMWMIuVZAV=Fyr8a1w2O!kikk=EdGS-jnY_k zOvfm#?WFU3vYfbA{d=xJ_sjJlkR3B z7E4nTXk$yn41j>fWd7O2iSrN(0bN4ztGaEbiiBs4VSLFA!132uq~0AxJzV;x4&`*K ziE`WIscYoEU!}w%{FyghR)ajb`DGVv)Q>?k+h4g`ay-G2+!48C460M)9!clj#RB?1~40a>E1hq2d%4u^-7Q60nSkgDOviU5@wH!Jl|;&G62Yh1hI|iiF}(8oig7z-!a*8cHYb>o*6;5)dre2*3}fZ{E$dpHK6&;9~{Aw zy*utaW5h+;<)@#c;y@?&pZ)NCE7+I~lTwTGZlT}z3;l!Yc#EC(e|l>I8Sjsuz!Xj_ zfU=P#(M-Ftyj_viq4fer%O!gi+?9LQ{--?W^MD(djz4?&M$h5g+yN=h9pQGY*v^@l zd`I>&!WVJ`JBXaPY3;8M8Pc55>1PdG8fPYGF3c1NRHD-G;cUxMb#hy|ahNO`N?e$| z96CSEYHYB=CfVN|hDTa-Y91;)~-wBslZ@>fm?LA?Dwo%i2`uv3<@-Ti_U0SrQrDudBZ||1AqNDx)(rX z<-OwpF)6)BNh5dQP6rfEjl#(Zr@QvORBIh?0iW0n8ctKnm%?Po4sR?}enK32H-=Fd zMUdT0f|+(Fm31rQ*Rmn8C~y$zGdulzhaXObMMxOr+C)N5?Xf8PQ7Wri6U1ccNDFXu zrMe`O3|ZgWz47Upn>|_a2x`accHKoY)B9f9TxWx@vs7RT)bH7ar@rb0$vPe{|ARAl zeoTEBX>^H0s~^)H?4fL&mHs4nn9~zK@W>1hWk&m>0<7&p&MuGJ<}UYM_Q6{)@&H}v zRJSrh;QCU3uV=NFzm@c#yg_d+rqHL8snFs0NLB=8Vybpfhe^5lmyXUA>^1^CLsm79Ow-zxH~;RQ&uR{~qev z!o8~=U^5X<OU)^PBt_9_VhPwqZ;`pJ^=*7VTjBb_%gfmFa1MPK=c6XW^_G{P8Bfx_}J?Z{{69?;uY!LUEw< z{@7~RgPGp2YV7bRGkmR&DzH0H9Qi**TKNT{j8Tc`_g){Kr045fvj)H!Z@#>QOJb8v zP+j~-6ZOv7J$3VSjsa!dWXJemHi^nyjF@pf+_vgsQoBZY33GUl_O9QQq|EdtHw-q& z>Geu9$~xqug<|AY>MJOPZ-i$!t%HnZ2meg|6_2=9OxGy#Y|`^M(gVbw*bfO<0C4RoWVe4!2IMo5r8A;dCwh-HOigC(n~vRY zXQzW~%3Tgr2h)7!avIAhn{m6EApD@A4upfGl)Um32lBb|hp|x0WB4B*@T6PawVDgD z-XSS9$nUIda(qAY^DkbgtFuoC4;PeU7J8=vp#P*Y|4mr_H@a{qVI-Ry7 z|4%A&4TlEN%$vn5g#hu-?dDA%sAo{5e=fsRrphV4QlU@S6gxi}?38UrUg+jOwls3! z|9KJ-2hSetKPsz!czE&9|E9YB#w;g(DTF~#`~d3$OXCLY#!Wj-ak|t~QGE=3G2u{5 zHSeur*>*HH3hdq~G>k!Lp`I8t^Ev-6tz;WBq%Q^RmFJg_CkXrBf5GU-d2j|h0%);p z=vOuCYR8$A?@e^AnQfj_ykx$OBw*jSsF2-bkTR4h5(mk6R}v$I`F=~1i^gqg3faC| zX9Z|dwUOh5cd6wkZ8nQL_pgzXV(LTlcO1mAKZ7(x5<)#@LKGy#0&8=f?>>JvQnrqX zm*sn{-Mdt|;6d)n;jP^2pKcNY_VEyy?UUFP?-jGQKQ;}v zm{ejd|2eF_)NZpCMv?~$$g5B*`v^SyEC<9jN;RnBx>kuxiSXuIx}$`0S6y)7nT+Pk z&ocJNP#7n>@{8z$BIWI25USdulytqqX#o0%orWAvifMmm8T;>AOs^>rA%o{7X>4`!^I{ z>1^)*^hy4oR=KAJ1VvE9e@c@$-_*mSFOq^_ime|F10Y6Nw*Nay&{hLS*@9ICmEzC8 zyZ%37NJVlRLVgLj>!cLfYH^Y(4191Pe}DkgE(#4A!P%}mxjGEHxN`JCE*;`_`3FN| zkG6ar<~&&R_8IJ8jII5D`5TtBCb#OMSJ_c^sm-ud ztM?0*k^$;$%W2(z2=Q;p+>?CPR2eZrt3(F5tjsS8yllgytU{VUb6we*Zbk)Ol3!GW z%|N=eNZ>R^(5AMi)u$wBqWz`1V4}PwPQs*kOpg};)(Nk`Il42*b;X$VSRun*Oj&kH` zhKBv;k&80kxuX=JXed#@n(f?o6ak%#)VAQlqQGQpcu;@8w);kyKr&j|AS_h1fdb>| z^r-a>t6!44^|5HN?%Uu`*x#N&2hNA`kumaelgwQu%_k=brhMO@pW%NqG#n}SyBOQz z)F1Zux5ZSn9PSi{=n!cmL@8%BvL+3z)Uz+}PvNF-81jdck3?7iQsgkV%ru#ojVq(Y zCUFa{9;ksyH+{YBzTXfcx><_&lbXM zq)RC7kUP-B9caz(D7n#?FR=B=;(LZk<3=7h7mp$QG*Y1n(`PE9WhnUd%5(I8gX91) z>+KJpA9dYiuA}+$-uTw+FGM(#eU|KzWFQ`BgP!qWo1+8(9~X9*MyYmeXV(#?kVh(m zodDRwz7TZo9jX>$de1PMsOcUqU6Qp1gt3P7-y67J<{xMpw2LM7HbX|-l(Q@j(L3G) zY0XrLT}96dL2oq;z3h4UE;UwhDGos$)|)JhtMs09587{F|IDlZhW!gLum3F(;bHy* zn#!?Zb z0~kChJ+f5+TX0Ao8J~D=J9?X3+SvMh*`Wd{F)Zf}XQSbrEI;n6$(<}4-fd);U@qTw z!GsG|&jjKFM0BN%5wI~LOL7fGm=i?!Db_p6wiz(zQkAcp{LB3#5G3J?ubh}_^nf`v zK2D93)(AF1q`2!1?;>G&nvHt>JHj=Vnq2b09hZWgHW%a3&F#jUVv}eHa%b~-*Yj?AY6 zPZ+tg2l0ATQ?Usnsa6Gexc-%v!8Pn6JLz8f{-ouk82mzlzf!BzVFlw~q#CKT8127{ za;5M9+hZ;7Q7n1yQ|Yr_ox`@W9q#Of=_qpu^=W$qMeCM89>^4qabv$con>5?x#u1n zFaFkq>?2P&EDj6@@qV;qOx*>0ME0Q)hum`tSF@)Kwlm!OtNG8<4(YN=k!#?9@^(wD^+$M7iqmcHw?P~HU_U)B`lVWt zsXYY)xvDvy*4|2H`_8z|qGx?sZOXN2WtP_azot8ZC=UZpfz&jo1VuZy2 ze)(?UJDO3i7{yI|!}p93c}zzqwU=NFu!VBX%b+9)s5-(qfugYk@v#pO4x$6n_tmHG zTcWdWu}{g|umaAMBTzQzQ>c}&c(e+;9xqe7kQmxTAHU^JP|@ zq0VdIb(T)|Jt~v&I^Yfq#KguD=lu}=%;6?wt8EbxhGf$|QBZOSj49mg27lNB432U! zXe&dg=g-!M(#~;di9r8cVc1qG%*#KldyHH2C$ke(@P>Zgn*3|_*#OsB=&V?(Uqlbl z1<}YK(Lt|VQO6xb07YK?^zSnN&#)f+nvLMTwx|B@GWH)ag#Oqo;peZp>hS9>(?2{! zJ0@FGXJ?O;7%eo|l;3Ln*eMq){Lp|T)&Hzu8G(TydD2foIez4dy zkYdU??kj(qqv+~&Dwjy#U_Slk;Re8odn8HCvT}dNhZv02v+ePio4J|;K3+b+dj!_Y zcQF>TTK4lI$A~l>iu@}uZPe_edhxFd^Y($$NVwV?n$R-ZuQVt=Rf%!{e_-c49-|F+ zoxVLL#cZIq7Ei;k;JOQ-Wx-$#N0blkqet?aUsqU+2^!tWcVuJCHA#dkt|N(eJ4+oe zgd!!$_`F35{~s$M+3Z*DOr%pPKj5>^9nkr_&gokyZNlL{meuEv zV2sG8Y^*J$q26kp4=FZ4kn0&o#9 zF-~2YQ$Q*kYMq7Q?w>UayUnHGM;K?dt!;CDb6jQSPQQikrtsHD%+CC-%vE@iL8{uJ zp9F0^q{i($s^zEj{?0O9WI=?!#R9jblwRWz?N@2!S9uWmLG3!OS%|=R3-W0~cBm5J&;T#UHMa*Y?HZb)x z<7@!Krod;J)asyTiPWP$?}}YM%&m&2I8w}WZC^u?pvmd_4*k92kqPtYYC9*Wi`qSV zyOBkoul@SDiew!JcN7ARNWHA!bGR7PBq(PZJn9Zv4pEW%WMjxX1gF8fxx^-%)fC+X z$Mza4o;S;^p-%Zuv$=%xa_i%Vjgp`%E1%EAF*c zt1E2zvpC-|tu`jL!piouP&M7jNlcQn)}r9>CzU=@;d$1GXwEeWRdLxAb}6ouob~2P z6KSXjGqJv*Cy72o1)CI%gDkuhoQ;SpOSWPYQP1#}Y7=+G-=4rKnzdb_ItYA@1wt9_?&!U4t@9Cw6QqHV3?ETV#>Z92;^yjDznZ#2 zq5ILKw)ySBa%qqpu%yfmK_@qYm{zP!|MERk=42xLfL3^e`#RYq5 z8LtSj##A@C>chtg;Ui!AC*mX1iuEUjkTmIXl=1l%l$I?86y;~nW7U&r*VzjNyyaRs z>Refd|1cz6@D>WhrZ8uo)>WM)x@;5zt1=3zGU8nK1%+*oN1=5O{G1cgjZWHaWNm?)KX+2gcrk%WxE-4Q1Z2|V*FQ+zkoZYcJhdIg~=xXNI=Cu2&B{i zdW&DK)al>;<(l>{nE5nx$^Yz2{Qm%?>31yZ{h97Yg2N=|LZaiwUwtlJXdD7)$s{+d zD8PSjnr3P&f0kfaLc6?dZ1x1bS_@4vVmmRsbo$2u8bRb_(19|gfu7E1qFj_t<^qh_ z1LCDn7+<=luDS(wqRzn-nnrr~F{{e_1_vuW31kka4M`Iq+72 z_C>EoYMK-w8X~;kLooA#v<3(K^^UjdZ2SvE=d5YD3B+ zT(GxiQLbS^RpJ;){MEsR#5V9EXZ*F|`jR#9MSvV&70gLbPDJ3$TS*qKZt528?7fz7 zU>mUA34=UKkaYF6hrMsc=@<(b^D!uV=RJlhIEw(2)8jvFklFzOU5->Aa!2U43}s@W zgBN>F*45iysi=b8_eOSwUu$!dv)o)!!+c8tCynKQJ*EqXB@s`F5 zIO<887MdPvLhK4P!?g-Y3ib5(Fi9(y7HrBZS6(_r{qS>?AcPgJc}NOUR4YcgsFG>PZ&G ze}-GKiIwdDgl}*Np9h|sVicd43rB7R8E@_=H?%@9Q1!1YxN%k4o!KO?dMNNVs&gSRsc)V&+NI?e zX-emg@R&md_e);S6j5Q{83`jPesXFskmjTmKd)P$V`o zPB@mR+B6%oG{I>w-yOeUXerF~2-IxsiG|q1Ox-lvx?B8q{+)Y{TqR#prK_VcAyc2a zKs#4PsORIJqg~%Rw}E6;R+5mmoTt}{_Y#lsQRJv_G1(f!|IG()UcNS8t|b^dW4fcppzAPez-r%Tz4nOg~aXeeOXctv(8j zXpu1ExBFx35)f>?JtTg$2#pHk`$axwr~|hu%&*K(XQLIAtP9I{Q_?eY0*?@ zPP&;;J*qRrk_FLdpTHz4NB#B!T`^s;H8zPASYEV5ZPJK8fWE`GxWexL6|#8ZAOfHs zd(-eoR(;lPt@y!q?s+B+tXbbJodghz=1kyhFj9ilT9ujfM_c9B5r!_EY6H75cvX;3 zy&nes4I7mL;nc9-ul$OPeW~4@mc}&`k(=^&`pn;v(yhLdjA#e2dW`^dv!znFGCbrO3Rb)ijEuT2+o2g+`>m zX(VLYHqbP?wvD3l@}*HyQCbe81X02{Ks%6K=KFU}q*+zN1NiLdbyJ04o>B!1&Uu6k z3C#l~jJiLe`ogl^N;svpnxzrh6FZrf3vz%{H_=YrAc}USBSkA51zm&{P)|2qjEpW_ zm?2pW4XF6$%C(mbla62kgkjU`jw-YC2XjNE2IW-JA^zz#hB2c6#%^Y*S)&gfI3OcR zk(RgYk zbY8J|I%H4bjKsoGq8C+V$^L@TdvJE_;rz>^nIF2Vo3MQ%1y45F(cNa(E(WcDz25YL zV2#pb!raTda-rCgW77Os&)_i`h>%vR)x&+q4vkw)g<#)r1Krhoj<7)`+%pt+(ilQyrb0m>-gqQBMBw zc*VZHf3{~Crm`^NdK5N~b;j)(5OLIEby!%8^w#c6p}YIrO%X@`PGSODCC8Hr+R3|~ z@{Fc^Gmjp$=GhnU#aIr2cf&?elnuBl7q1Q-vKwB$Xn&^ANXSmz<7s2YUnCRm_qbi6 z5(`HCL?RK@;N`B74rs(@`%LO456x#*Wl0^DkQ3Zua~n@qr_P$EWFR?E+l#zjJ0W7kU}sb#xkYR~ zwti$KK1CF}tS!E=FU1R^K1!`^Ku7E}Ft%Il7R|w0&_u3b`AoDg94tWIoBxksQv9@YwwAKblF>bYuosDnUY}{orGBkmb}~X2G4LE zpJRzw@-EtqVlai;OKFS`$E2f7lsYTWVx zI~`?4liMjs@keEE-gf1yZmAeN&mzbSu37hn$s~G1T>7sZFA%mgo_H&(xN~pPq57kU z%*#>vg!lsc!8N>xwMl3LUVFf9?yiT`e-r(bd)Rvki795~`KbHCw3Q9#m3+WV*7XxM z{HY29c+=QBUF?(?NH#t;1(q#Ab>RXk^1|Sw){12zP(E_0rMeLe$lP-no?-+?#&3CU zd5rgcLA1t~n8gn_;TfvPMVBCIc%J0@5jSm84Rq5#hdc0~!9_Of0n7!~a5?LkCzcf? z%i$pBKA_QSOZ2bhorGo^liM`_1AF|q4mb*cge6zV-?5{3SlbV8bwE-EiQ23KVbkHe zWEY-+YSE@7B)e5ZsTe3~f8j?(6a#5eIkWQ4I1+^WxBM15cG!7RX_q3A&~aHkVV4jw zt?2rCT#;vzIKC6YT-&F0kK#*@FP$~E>ZtA$jRoOuL`7795(`;^J4tCjojL#={{aIx)te zH$VMv?e$OQD*daM!vY=*EFt+i7(*X;vp`LbokW|G#)&X^6{bQ)I+pKc5@Ml4t!sI7 z!5FxtwYtPbxq12629yvHva-~{IhbUxc{7>X`16=? z`MlwB$@jRi_~CP{3ATEjO={nR3GT}sYUs%xD(D-fpM7{y48f}(0hV=;#Zm+Gqz=`5 zB9G48ow9=TjPmL1|8n@}<=NDx1Fb*21+6<7QU+S}C;q#CaEP(#HG6nCBXjK&Lb$e0 z`{)jPI0K`RSw%I>RU;|=wcEurOt?ec6Hd58{S(TIf98OPu5a|fqwedE0aV@B(1B>( z*T4Z~-PeeLDc#qA0T#$7O|UK?$`%g>0wI?nhlhMieycjiiR(OP@hQ1kSXKFFS%J=1 zmgZ){C0sh#w;&xZG&k)fk*cBNw_z(fw}mBEHPnFQD_vtm!*D^HokxA35sFF`54ce)}naAe9FQ&3F=G{1QKl!?v5}CV> zjE{aqBAh2R9=wi}{vZ#WgRV`1omJU|$tAYsebC3@ICM*C-YufB^e~(^k_RYr>piv%()tG_MS~BN}Vrop)ep6LOU_jAsI?pB`s(&(T zsLM?9qq+aY7WB7Nw&q*nuQFw3W3W}#@Wnbyd#u)m^0%phyDR|1=R^k*e5DVcC#$QD zbL`{;(*UBTQV9ydR{Di=>*ds#HnYPr28xs^CqFYvMz6gbF`&&N_bKyi;%!`e-AbTUDbidG7@R`;l6d8z>^NBrOO2t_%UY^~Oom3ZD*j zkV_p|+1fi?v@j-Ak?CmbZ5AtB6(vo|pjTz^fUv1ob5YRFd2dDznNPAvC)LIMe5dma z50s|FOcaNPP8lY{!PC3=;!18p6~Yc6aK8&raIJT3S_WdMC-Qz#NT7#QIs=qYSyF1C zOc?mgo~t5Y%f zaiNTlR2sD2$khqdJX#0L8P*2Rgj9j1a zUZ`aKn+}+b?9Wh2ySizsLafzGZVp8> z2{56+k1p}y>QvWy@myaUZl=aZwoXtv4QOV;J1=9ejfsfZ-0}GS=z~AY%e<@PD%V2& zLaZ|QWqPa(n7{_1IhcKAg};6jdyV^=IQjm^)&dw}U#6P4_8NJQVkR?4xGMuz(zrEBV^WU$y8-3)y*NTuklYL@dNkBYerU*ovL(@+ZNQY^OZ>Hxug0KX#bN|d zcB%6&fMIu&mu%PHS8Y&sP-tLB)K_8<;F~hg_xg;6ic~v_CNQ^j-GqG%gfl+I zKo}MRR8>!YJt7PzRDUgK_{#15mpaY@9oV^aja)4~FU={W7hSm&q))o71$KX@Yytk< z64{B8|3k04Rpq=1UnW<6c=d}M84w1Vt^&_DEBK}+eUDzADNt1e9a~7Y<6~ z{Iw;o<0Y&i0GZl@$lgtePMWV4V0qQpWa5p2-GbNwMWQ5dkR@z$uP*RKt+emkkJb`< zex%81v|rk)MC!D)UI`&hhCO9iazOPKn=8fut%$Kn>M(weL8j@Rk84fRGgwX zQ*PY#GR}JX5d7wcT`)eB;%99>qVw8cbBhko!>a~I(?SyY8mE~SA{Wz!xUD~K6zM92 z!{a_{_$6hiMI)5V=A5t^3|kqys3XIwD>88%i@{tiocoz!EO(-^$SFD=<81Qr6Of z(@|R`K{%K$t4C5CaCW_rcEGi5DXZjP++WL$yPI+q)2p+3sCkUy&ZwE%V=7Yz5#zE= z7_vm$GxfZ*80;&P`e_P7uPvKHd_u1U1o6^Q%=mpX`FT;#ZFp9E;LZbKR|GBnmXCW>Bkkdj-B^WDm#iWdykT5G^4R@E*fIfu{}8eCR%UC4&O|xH0`UbexJ^g7;K*IP0L;rbb%rt7;}XVe?ACXMJoRWRe9uh z$XMeFPLnuJADz(Y?E*i)`B9Y$-Pv?W z&P#sa!rd9VZuOOw5LK2(hls7WFW+LK*8_iZJycUJOTZsWXd+v9jvK6xD9vf-px^)*<;R z&&z~%ZsVozn}*W!-{XuMO+2p6=f6uIOvXWiuX_xQB4z-gcaeYEiT{^+k^Nso9QKoD z^3RrBbL%b{2Kc{ecQ3UV1rR8W-xT>~3>{Zva`sf^)DbE>oh( zr_FL5l*ca)i=lf60O{%NiIqldpmX}Nuv0-Zu-FGEUp9PDaVjR~;9z_Aar`zTZF zz*ip6so%8v5oROC4FG<+vsa*L=??Mgh~$GE+f zk+iTX{Mk{P;5>n38at%LKVMfeYar;KkYw^JQhg@9Qne-;Ec?)V&RYXXGb<(jsi6UjaT1alQxZ1LhK$?+eNaLL^3q zsEpDQ?Zh4i%3CP-hvM)f>Whr6+89t2}it( z&Qvo!{YyChU#$8+DEvf1gny9m^b7es1k}I6X)In_sDFnoP{?KfIs4j=;RBBH?GyX9 z%>=JRKW8$DASI(AV-h9*=;lLA22k>T0Tybny3c9gy+61X$rjToz*^@9G|WWJ;*V-b zik83${rE;Vj1}Wet|&$mLv4l7mh`b9 zTbv0%z}D9GaDMF@<+c_SAoGh#s$x#N=YKwmxv zv*bV*U!N?NYWWsYGx|tm&wOl$ol33tNDoQ_1vAXgp3wdnr!i;e5kO{|3J+k=8xa@= zrQT}+^JmAivvQIEY598WBtx%G!>35&!EiQ$$HqG&cCxz&FD*o2^$wK{>$d!o z7&!m#f;56IO+!LBq_~;_XIYHw5ht(@a^alkFK(pk#I2sOMmW1uWW?M zaSnao%(PO8T5Z~n(#aqPATw=7ZV}l|I?A`6ClW;X>}G&85p6f}$tYe4^l#s63+r8E z8|Y)xPbnKQM!Z$`hHShnX%CYf@-;PDOEd}6^93UXl-#JQIeyw9e$e#x4K^sH)eq{9 zYT)^WSKv2^jPpddo*0q=Tb*b%s#;A-vHaWvI3aJwYi`k}k?bst=c_U_%2iD{(rj#l z8*c^`y(Vk&JCP2`P3UAu#p*5tMa^W&)13{Ic{4vmQllqicL<&nLpT}eZ(Nt* z4JLSPoNprcTS5d>YB1Lk!tfHm(WO8e(#;;Fe%~#V5=G5Yt3m7l&N;C)=I4BYhV~m) z-Det$W^6o2Ipt>-pSsC8I-olxH^d%eq9C(s*Kdr|enhKk^pt*HH`ZQ(+eexOi;-C$fq z>+vI*MapF0Kdnf?^O*F zR_u~IqaFb}H{b-JPV2WsO5~>4{lyw#iPZyRIGf}S#|3~p|76P9Pr7OdSwQK9?i9t? z%p=}JkG_PzhfO$Y5V(^j%7*NZx-~N$!l#CM-?-l@9*}1c{rr{CKcss=AJ(G3G?VR>^v9 zk_Oz;@L%I54sAkMRKr!naVb6Y0w>KDqq;yje4%{&2LUy6|&Hb%-i)4myO84SLzMJJ$LvE_K+#lg@c`K z6Z1{B$#aQ{rg#T~o+#h)yYoP_YX8uG#IwC{Mf*L)9;7@;7qJK z7-Ka$Xx~XVPyyfWq{e$q&nlYYmuh#HCJ!i;(x|E3l3)TT87NTzvn`Rc4S-qBwibzBJe#37rY|B*zeQ&XEXlbk)h=| zDdN351m-mtm&W&r`(38b!+bxwL&o;03aRTbsBl}^{HfRo$%dDyPvkbwxG(!_oH;X%ludN^VR0lm(*9|o-SsWYK@&kPfdy^ov%Bn&YE%Q zqx2&bv-`qpQUt&l?KA|I=YneUEE(Kxe1eI)ZKX6#QBBUaxH15WgQ5!qfMnj9vkQZ` zp*}f>gQwsg#j} zl0+N(W;LO~AAf*Tsb*Mqtf_dRg$liOva6C9T9olv-V;9G@nEXqhW;{AWXiJl?n!!$ zafR^<0J5gWH>s?k$F?gPNP^zUR0SBkC?4q_7sTQln5`YQ8lg3UYitNWqqOAd7&KyQ>jnPqazHU{YNJo}yxm zuHYrImNNh?Q-xB0?piu`ISEd3%p3j=MHjKAG`l%hNGg0WD-F2rH=v$srsBn#D%2I{ z!Ys}BLX^)*k-<$~+!`r*KLP)PgU#yOWP66?@1zfQ`n#gRX1r=<6=PnR03;RXZXc!a zb2tVF^?mF57lSD}W%@?PUg7kJ!U-X>-!1^vGx(ptq!0!QyF04LpKJ~*1)y^SKEBl= z1%}o|OLh%0YO}dGN?lzHx~djAm5`ru_iRa)ue0`hl(>->Th=l<2rn@?O;j*2yG5Bv z$;#Evu*OdnS}-dPG%Xi8g3PrtZA(}*yr_kL`B-nKQ+EQwk``S%mo^Hw(AH$>h6d)c z?u~narE0yo+l{|rg%4w7n+V*W+;+N zN!_usTy~x*4RO%GHL9yLTbW+d(zpxj-T-2Otm*d&oaV|j-^PFA+c+~%{!YHZRB~lz zLQwLoz#CVbxFBGYSH{A=Ez%~+&TaRTl$S>|sree4<%r^WuGV;J5YA~^sLL}ssb_Ln zrM6s}ZB)5SWfO77Pu6idNR=rZ8Leu>lJJozGn z{zmnXC|(%XlLBJ?^MXo)X?f=;Y+JWZ;NiI`*R&yjoD4imG{Q?X8iD;*2^ChW>q!ww zr2A<@$iqoPbT9w;A$s1{ufO1#pI$LsxFlC!R&ZA~K_?>4+ds8Ff?%V#o%?X>M>>mU z$y)XEjWRE-q+Pv*i07|42_r9hG$SwOkC%{TT9v;0DQcKoyBZY3NzQ}HB<}75fbVfR z_3%oxd`gZW&ir@RUX~&Gu5A%J+a31%<0sbpQ>^@WDu)_T4NtlJ79~h?K*BU2M(OlZ zS7cD+CZ|?bA{{vpt^_dsSRH>vUTzzxp*wx+U>jtR54Q~Vx5WgpQOg5Wsva2_%eU&Z;IiNgUH{Z+Q0=& zbFKuXyL#832RML^Kefi3s7`6ZC&{3`K>b(na5u(9VIV(1LO~=xY=Fvk9@CzQ&N8#! z;Q=picJ=7I?oa^E*8U8NLMQs@jAocg$%c3F9l?7FkMPfM>=3vg^X$4^ zDTcTfWux{umaXPC$V0(A$e2{MZ#Ka{nQ%}941j(f5)(K;Wj%7|sk-SgUvHt+We*H0 zwsv)++lBn-C&%o_Y3KtEt?Af2J~8NpP>J?NOcJq9p0iC3VM=ilwM}WXnkA*l?TcU` zj5vLB$=NEG{mB(19O6KrKCy(sYw%QQa#JbHGjmha!zvKjO6>DUas+pYdVZ05b)x|% zon^VK^pb%a_7c9XH#9Bh1J|m`=eN4DD&G6i#LT^b++kWjVO9!szx6C)Ifbf%Grl`5 zCK*#3Ra<_`?C)8fB8iqDt17WE;AB6r@BBFrCHANG`&ZP_aH`R8BYvHjZ5zlKP*pA%(TF@B4_yg4FVN7cGX0cg4E&oFLTv|f~wyJP03zI#K^TMbHFTMOUZQqUHiyA8PmiF_yTexbZBVXiXa5Y=JyglP1Bo+NDt0%DWeW&=(l)ih$@chXd+>pc#0F;!>R;r2<6_slCW zSLl58Cm!PqFrtMt=8KX6nW+xCY=(uwqunB&OomCuWB^gfiM#@BATo~ztw5v2{Xf^@J z+N>hH5zyi#mG@B6wbMc3?e1f+HKd-{;oi`WwW~q_tz?XU;2dIFy81H`wvpHD`;qLM zuNJk&frH$0wM59)eZnqni z9McCWBF0)JIUdCZFxw&CToArWFlfiaVwFE|1G5;*aY!b93EHwdI~eVG?6480k|SyN zaAcbzKDGVPU2(x4^Dh1Udp_!$m#wZHKB{eS?T(7wvPH#rqQNq8LG|^M();$7+O-bV z^x_GI4?O$Q`|;U@MsH{vw@U49lDVUuauLfdDr51)wB*_f1F3IVzO(OOX`VX7bDe>~ zMnIlwEdx=r7VV}AKXAouBh(KD2c&4zNWwEkWD0AX*|x;c%9N3FRh%;tln@_PM9498 zQOpp@P{lvlFMZ!>ev zmPJ|#C&LY{J;nEVuBx?oc(1zdZtMukqJ>NDJTG6NXLo&cy~bn}OXjtGBdc%e)Kv zZS{`KB%}28sXmoZo2^hjFF%><0$cqK70D|~+vcJ!HcLHKSI@$mAv4fybMNRKXJL0W zqqws#^`fmJ@RCmR+}z{ToFLm`O#Ui!oc^5hrH9I5B)(<*eQU>UcEs?uCC$KOlh#a+ zgUCP?^u5JMRHm?M`!pqAR-yJKg5TYL(#iC9dDDf|^Af~jjIf2Yz4R8_L0x& z5ZX{WGMI&L(64rMtkZ2mmWUYTGA+}DMUg%RnNPv`DXcGWPzgXohulRc8z$?I-IZ9p z(@LYg77UNAjc&eIUHjW^Pb(Nlca$o_@OinmkZGz;ee%q))8rd}*MxnXGBdqlmAvks zexCL-7Zw4^^0H*GiyH94(?7f{A1VO z{w0(C-&+5_q@sVi0L=#^^aS8g{|ehZwo_YywKVsN|1PZyt;;+C z+yOTlM?!rC^T+DzD3`>k%lIu+-`duCxaJcbb@wO&e1`^e^R(*~Y*2W@;|46Vn-F#u zAE8Uyh56?ku}Kn#WvMiXgP0|i>Aa-c_8ay`l_y z1;?4SM%N%;#C6ZF>8qmbUP%!n}GwPgVsWYcTcpPddZ>swQuA15lEn~j* zSD*OBV%oa@)~Zs?h~IV6BljdkCmZi~R@#iEOTCBE^U$pba9cB2VUh16KYAK~nmv{1 z_S$oi-8gZg%ViPRcx(hF1Q1m_aEFu!L9V|3y`@^*S1^-o3o6?E%=(!;5BhBfD>o%g ztA2f%IZh?zGFyvb!0wPfp%8@*~kT{53n;>m4iFRZPvUVIS3yucz+(zO+mAq{5YjT;!9 z){qRI3Pj5F*X)J(y79w}ZjYxX()xs1MW<-*{y9be6PF<=(x~L&(*Kc7VM32Icb1-UuwieimzOB3CGW-vfNQm0@b070^a5%d6LW|FT_KxcMq zF%HDmJGtVMZ1ELc)$|_}`+D*YP^WWd$V-fmegp7NI~ zZ?v>eKdcCkT7t8g!qE`;%L+|UC{O{zVc?2`z&I}?MosWVAEG@&Z!N6b7*KH<80Wp?SFok@h^zq~s9V(lNZGDa|F7!f}LspxJXrBEcSTO*Hh;AZ^ znKT?hx3U-S$EDt=n^q9Xw=s@mQ=eIe+zD;AvPLXMw%oT1zp@5YuT>*@^XHbR^KNv?p_}`8&KtF*mohHAi*Nny-ivp~_9avJO~5v7 zv8@2(8wtW1EJp|KQz(`a$y9uy)CES>rlFVP*QSx3?o;#E{MP&F;^F!^3H($4oJ^QZ&JNUdCDg_r`rv8Oims$Yp)1YIp+=6iAec=MlTTAw zmHlmC0WfD!R4a=rtQGn#=q#Pj3ok!5gEJ$Bo5R55Y66he!zEayES!d5N`OS|Mic~N zxdLnNhJ@uMLfPbT;fj-9z{rkxf1K?2xb$dtu-tgx3|a*H-Z$0PLf{d6EerIek9gdk z^?gU)wRf{j^;zd~JF6>dqSJmUZi??@GLm0z1728 zeW_w^Se~ok=g8mRGvvpvU8#~^jS=@u<~Flw-*vJ0JkntT`au zlk3MN!&1;<3x7WF8x%aW&lM!JtQJq*O0*xs!%hf-y{X#|&9HeFZ#K#Cf}lx{5(vK!tM!wXR~#jILrW|Ktf?IfNzF-Zcd@b@{698=i=& z%3hn~THco8IK3z0A_H&FN%2$a*9z|P#LvmQVOLNcm0MUqCB(bN3o8qQgjTS(ZOGA>otVt^snin1>&Gm}*>oA`m^TSHPjfGs0)rfz6|VfTCNfx+3D ze$Vqw-Tn}R=pZfc`T7!w>~k~|xhBrS{*hH0vd zsVijsyp#CAlbWd*I&^`%9lXd{rAYT=3W(%I;bEAe@T5}9%A4aOc?!9y!SbGuzegf= zMsNn=HdyC(uu!{i9jCaYM${b;i!)pbUESVLHQ-+;u{~By1gh%&0+xPe?)9FCw`|xG z_mh@#+wcXN;BV{w-S_FtlB`H^Ie&9sjBL8sEBj;lbmT z37*joB;dh_-rG&Fr<$5kVW%Z6juch-xXgQ>8V~rj$jGf;qZZ=5)_Qn@I%b0qT>BRI zHKQ|NdC;vq?*L+RhykYJ)R~SWaZIIQ<3ie#j)UV>|d8G z&kg(eFENirA$&?k`Of%53=eBuaqhe|q`o9h8tx0qoFy;Rh$yJvz@2wXmy^5xL2m`F z!5K2Q>1B5ejlEOWip;aWc#1!45Ik6gzUi6*t1KSA`#sznJskdi3rbpc8LNF6J0rAN zGNyb;4)v?DaIX}8X=^__{O#9i@$ftN3`FpN*zX^^WNeq}S10^pVf3*0`z>jI>{dHOv}7(Hxa zyj<#8PMPWPN00EFgr;}r-+Fo=zh2S>ef(70`cf_?v4)1uBM~Tu$1Uhr_vcCp4pz7| zl*#Bx1pPHz{|C?ar&YqfA)Boau`3|eQ&%IB!$QdAZSP0s)(42)7qre*@2m(M;pk+) z;#^pIp;j^4et#m;WZV)GOyd^V(DPOU|0uXj0)mF$BjvBYRP+JMO{LeDRC(MJHc&k0 z<(r~)q!aIg7|AAOLjB><_i6UAL*PtLQK*m4P2x7PRwvCEAep}9^w$48z!oE=U74m@ z0WtX=Vf%l_0RN6dmRwF)?xY6vce!s1eB(~0b+Ib~6kR&RjW=eOA9+lWZ@l0ogty!{ z`hu?>964)3cN-a@|1_=;lW<)H$1cuBqL`p~Dj~i#b2K6sR_cPtuF<}U`uLag_W!Q) z|9=KR)#$H~f&m6r`nmkm{cFoHnMRfkCHWgOCRDONuK@PvC^o=1gLyY$>H2oics-+h z5|eHm)$(AH`A8ho&{CYmFLQizNx18oBv1EI|pYH zb?x4<&53P0nb=MywkDa__8r@{ZQHzKdy-6S+b7TSp0B?5bnSNcAHBP}Yt_20_1iP7 z?~#0;PRtaQowav>C~N#H0Nz>RTYrh^I;4X5bgiJ*1IwK~!v z>}#2|W*OIdgy!3V>D^b5TBP~nzr>=2^fmQuG*aDv%^lbhHwhxeceBZM#7Gki;m$8l z`DWJ^nlsCr08~GhF(SzmsWK@O=Hg2>QDdRD2Dbsj(CkhREb}oUAwa+P4H9 z_NBl@T(jgPep1FX*?>0xqIc}#O_2&Q2G7&b+0f%EEtK1ut{2Wq z+eVKTcU6dbl-%_i_#o4yImN+)i9LrOExfJ57A@RC2W0QuTnn*#poN^9-unGqfOOhF z+{TX8;k9;ZKS^oY%7ag)NsM===WAg5xtx#0%(3{0k7$5x8{wZM;Q!FlQB^s6X>Q_7 zk3^Qd4_HMBqr`u#GqTwfrq^Y|P}0n^dRBx^WWvRZ%JQom&+OT$cc1rIf|g{K<=4B$ z^jOD}?JFb;?2hk09x7`@M{2Nt<>mQ3#yYgP$dafQoNq?YxsSiyk0DoEwt`FY{%>IY z(n3a_u7Nl5N%1wWwzroG&3OFmD)cxIR){*&2;c?6+d;tpEJ<3Qzt6nT)nnrRPt(p_ zmnUKPEUm(-6WL{QlKTq9IpgY_mpbKBM^TsD-t%>f)npGLYaV;ucbKfsz(Jyx;J*L z37|J}fL9S$p!Bo3@l+Lv+#YSJdE_^=aJDir<1_(GnC4t|8xLbi^LN%P^w2&vF=?Qb0!MN^t1_J+S{azqNk3<%+gEcL4bw)p(q z>f;D^PPkcet@J+XWaMK-jpvJF)}k~y1fV?%d&rbxdfoPqk-;WPqSf#vNFAZpt&vy< zoKQRn7@>tzp%TRfKl{bpQZN#o#kBiCKi?IwBr3CQ1!C;36>K=BciItW$(gR?1fO$b z#UijSnenXhu%?6|oPtx5_SM-=>(X!e*^chQ$7@25j3{B#TA5ALK-N27oZx4M0eIBF zr0$FpDaz&If%^IZrUoMAO`Vs&VzS{279#ZwxwG2136a4|naZ0RO=ynjmRrI% z4zkTtjH(f+(JK$#F=$u$mS=&gi^{#N8x%4T}swtbUwL{mf-wR4xTU&NF$(#nm8WV$oVPqa{P@OMzQ%0AqCq_JcLQHG-Ws^`mzZZ<} z+omKTp0q$c)#LP#O}JY|0lOrbp*#`5q%V!aoJo$`KVKZ21Vz|Nx7GqO2F))&VpsWS zC7zu;VSPDIE{?k?M@Hji$O1b@$ZqO>F&CjQ1&(Jd#RTs_;-hOk*T@fIh?8U__L62# z9%o%Xt%<-f2-V#GvUa>+#6CU64?o+DM`eE-W7&@jUi<0SWNNCI4)EhM76#uxCneDj zCFf$Os?N-DiksPu4-dMmM_qaeMBgJ*0_mIIUDu^4r^cgDbiNK+KtHR{$`PgU0!7Y3 z8y(y$iM&c=+{(S|XYe?hlmjP>Yj$Sp5luMW#lNq2>(hlhLPU^+im_pnZw&r>h2eSM zRA^Mhu9lo%&aA4&2N2(HYP(}g5li1(<9wAUC{&8qZ3Xf)l@eb_NHSb>)Vz_L)R8$v zvzE0SL(jCa7x&aXCpdHROcdR4GOy|JzSUGccQ`dQ6*MF}S@nZFy7SR8s%8{AT%}lM zKP({~>A6WVUO+-5>~XFxr``>O+r&T{?Y7CMH|g`V@Y>Fs0Xk~Nm}A=mmgpoB2rqwy z_BCbeCc8%Dp=ob8i7I@|xDv?`f97?dxkY}viCCcMp7ga^*>n0#ikcltM zSD?vUvNNU>;82=pgt`cMCo8MzR6~PL{F5!s6Qe)xXIIGkVo{hH??xFBxzrStIm(a; zCBiLrKzp?Y)Nxq6DU34dwv`qq4G6?FO#|i4+fqLkwr#G_L1CdY!d1G2xoV=tuNm$O z{%m36!h#);u?6-nj6xP4M?47B*Wd?(qQ1K%1B09bfMJA4y7Ri8CLCgl2ySTIHK7HY zX^FX+@_AbD?}1;|D^MCmhuMVza>}V~QU`d-FCnT4lKUAks_>h1t#|-tKgViD8Va|7kUuWCiX0pqCb5 zk05|K{R)xAD(Kwmw3(>mHsPc}zdN>O-vfFmJ|k)c7Q-DL?Ko*#zDUry!fAw><2i%~ zNmJ0L&X8RU`{5&VBg%4PwgBT3HUec`q$%i|OLrfS@YV(raS9skAVB0*;ry$koXm}e zx<}hkW&y>U;@p8deQ$0Hr|P(jcxCP6(_a8ouUVkl@8oBqJj_2mXjXbwdGv7wyN88-uceu2{S`ihGD}uK3OSo`G_*52Ok*Jc*VI`D7{R(L&Oj%d7 zX`Mtfqj|xwFvMW`5n|1$hjY**>H@Be-|yE)Xoy9GXhme=9I8X9e4l%>=(c3ib_6g9 zpNb{_X1QHFjP*fyr_)5sWI%JOUVZ0Vrk=GJqE6g-L{m|dJL#*?(2e?Krwm-?^+JT( z$gA_;_ldC7{wOu?d88!YQSU_$`}2vx zFlf!AKs>E}&UD9X+F|%*qNQ1K?*yQW^Zns+=t^50I}g_Zj(PM{9(`HF*dZ%jhxaC9 zf9Pty$iy8*zY+cawPt$OHyHCbA)oXT$4JM4uK89Pletl$T_k3gp>C|&gI|Aq{zRxR zi?*p%($RJo9Dt{35c(#lx8Lg(aek4M0q`Gg&{Y+lXcr%lPy?Hl0VF0Iozj>*>6=w` z5iVDY=S+h> zl}p@Pl6fvjBz=rWAvUNBPKD$`4npsIAd}1rhBga|PVyn(2?YFn@PKrIPtvZkq4qps z&*McrRzTD?{mq0-4OluxU)PV+eeBSgz+8ZBwM-!rQZw|{0JY?t(7hx^#I**uomiqN=2OyV^y7FTJ7{}c7ZVz26tST z%suYXAN-&^eGi3*%7XpJB{^1?hLacCUkDE;pLW*mQlJ;qBfyIKW(yjEnVfaK2qzhp z4sTj%uc#e=EH{`X2fVhHc&^)T@lj2qz?}0r1F^?6uvTj?F+B~)B1hpN(nDFyEYaUS$5c;YvvzLF7VB9GD|+H_ac`7>O`Uk$y(3g$dI z3{`cuQC-{16#$t#{-uWb;DOa(^%#A{w&NPTSI}l4wR43wXXa)tPPac-LQ_Z7ly}a; zZ)U1nZxyncQnH!Um@enlmz22pQ{HBB;Zsg|AIh{*x(dk#tuLssfzqwSnuESOkXuL;ki8-{U1+dykkl} z){F(r+!qbvS6WdCQNAD{em^_7wB?a(8U+osRdgXRny%-rmQLxmwi)3!5{r z3S}eXmjSig$1BROh;gOFi~Rhyg!2LVmHe%&WVXxJ>GX=piLb6;M=RX-c-%m{_~ET_ zrJNzqtlNEvX)qXX$q+O6IKd?$+eK4-eWqYNvU0{7Fxfk(Trluz0UBei zWnmvRV#?Ma7mgQip z&;e3A44ZK@ZJ^XM{arlOHJ$pBBywOY!Rlx&NSKP%bRjjDoI)PT{F^253d|#_RqlH% zO7w-{pTV$0NeA&B_!5_gS~MWb_s{%m;luW z*(Ls|v6dFs3{c+ieY_gASl~V4flOlauG-L`Sv|Xa0sOmccvJFX*O%7j@A!~)h7Ujc z@JjUraq;#acUYr4eC>(&tdTLV_0!CVl5at^$Dzy9k>{X#=G87ZaqsaEU!#SXQe(i* zQHu!mf?jNBU2Jj9=G=HOuQu{Xivc})PmTsiO`c29<`Yl`;a3a$lEqGp!>6B>y*2cs z2d$H5@ij$0y(;i}{P>~q1+O$i;2?r9*Qm5dBo#YG&zcxCXJwV6=3HP+c%bVv-#Xtz z>)AAe*qQxB8=_$(NsJyGS?k}HHOT$Mw}sPfmLDNGAB=>XSz+D?1!MJfx4bhVtm7x z)EWd$u9`BZ1&z&xwn7j+r2ulv>*EbR-GWxzj$AU+W=b3T~p=)&Sjes;8_%f{zOR43Lp zQl?Lk%yXgY7?BgCQZlH)pFWL~lzIr*H%JrKmx3D(&v?_9o@qC;7eLSS?L?gbPX+uQ zE#l@THnHNr`pPa5>!O@iv-hC>TtAqnfeV3?3V}+5cQvR_TBk+;PEKx(m2Y!z-y8h_ z(KR(l>iwU0VBh5xGh~RPf4;6Eb+2U*?!3s794Z`Anrmm1So$s>hy!5|H*$O0@{baZ z5Q}J|*Y*2tZoYI*cK~-KVvoZ1EY%8~Re_kGXeuG~sc(dh zm^$2c2KcL!*4!85N3lS=!U-s>p1YuU>K{knWfy^eSxkQ=2HO;V9_dUzT`Y}{da?pc zwNFX&deJe^^rE$QP*#hrvhNCiaf1zjDt-jFs0rthY_l zquVW<_=WHw(vhwBYYV5?Pa0~Ma@x*pyRcW~Yddd3E$(P1LRjY9>RA5g`OMdUD@Ffz zitGPPezoHI(suq|aeU2>_7w5I;&|YnN{h8`Q7jJSTOd!JYsW=Tjnn6YN!9(yivvj3 zcyU8t!TcQ2mPi?e+JLb)L2bm;0>We@k#mwrplH+RQ$s~L>@ufF-1@OA7on~;S*)|i zwMci^*>-R-=!s-D^}C9IZ!VO)^}H_%7L}Gh_29(*FKLi+b z!v3p%P9<^^wket`BBCksD0&Guc?ST0M}$xB<0sz9Id)~F8a*@bhXw=?iStnwk{6y! z(#a9gzIXTSlO|dSF6;belVhwebh|m;lp{2|B|PNsFMN8v7Qj(_z47Hc82zJbH2bxhH+82!{FhT8_bD1)~+#W`m~u9c@a@hxWq zpKxXIzAiqK$0BJd%2kZXf(EegxJZ?+q=z6j(9#^3{4dwJAPGg2#T~XsvN6M6#32}c z(*c%nQ_5-~DF@O}pUjwe6Qf4%O&(td;_j3Y&rxc=Cn?9_;tTl2u#Pv2sSq`) z;kL-EkiIJEP}6o$NzLgnKS2c-3O|ek*FT9Jt|%=P?m^v^p@Wl~Sm!IlfXv9Gs?qiq zm6s&ZDxMx2^UiDc-6*_+I?D>ciedg6yH)>Z% zrrnI73)%VKWFd5i_LTZ%KTaA7NsSf~7Q?jWqpiCNj^?$)qkU4Z8ElR1Qolto3Q8ur zX93?lLwsbX=U8fY$Q8`V8)rgzIQ?Oc!|sxQP*4_cDc0w5lAVAm%YP0UnwgBs5lzl^ z8sVg~173wgmzO2yOu-y{9x)~+@4x!b+GzGt0*H1-vl4QF&*cHRI~h^AbkC4mzpiE9 zW%@Ipo1?a=pBa2h|2@fl`U%oLqkmTYW1|1ifCLPt1@yALK!4I0J$=oAlb=^nb*pBH z*s5g16$WHAy$J(WQOnaj87W;Ce>v3~^DIqb$h?qgD;55|(Xv(4QG$^#rRECN5WrI{ z)Un#sF|;mE+>g~3SKE+w+_+J5%OH2lSjNPYPr!?ItDga?*-(!4e3a8LxNKd55DM5X z=7PIcsU1+hwlEcAEhX7bhb<(dB(3T<1OJta*A}_qP1FE%f4*lgo~mm-rb9+j7>Pc+ z3fr#zrL0;|Gzv-_D}qVX9>`agrpS?O?zyz*ax0@`A(%E9$K~3zPk7BJR(9>Mv3l@e zZ>GQM@fMy^Yciq@i=Gf|p?FQn4jRtu*8M@t(_t6f6@|P=n2o8tWTpp=VfY3&Eczrb zJyGdQ*4PS2p*Rpu-e{w%#e2+|S8rxaIgiuVRNZb$TiU4MQKbE*c+d67{dRECLyWxD zpigqCRI#yVZIT;JT)6vrFrZe261~iW_%ML#)*eK#)6pvYAitHQQHso4yxyD=0#W=5 z)Lt?qg8dp!u(foxIIn#>)_Ncu(f9nbDQ+j_iFgg*7b!G-og)y<^0IKoWqbu+C+#e} zizAKKn5_d@`Sy)a+Jaw_t&@^p24Qa;XSWe_v!69I&4z;e{n(5RG)a_1&dX)r&wSoo~{6~|VG(*u&eluOXVIO~J$uXXJ59pt& zY7at&R*{)P3XJT*4nN)xu$I4lul0E51euaLI&b)#L5C=5gCic+9pWOHX7ySSVtX;O ze>nd2bouetkjmWsqYsF*%D#sWo{l(R(3k_@BWm?r;YW9g#kWDk@SBsaBd5z0g-4JJ z#b4#|`}*B_BsKh0I7urC(j_HVG}8Rny&dz;yr{1zjT@+>fwTO)(FZVjtHwn7!e5uy892a%4-E~cPjiY z^q{ApKHi;w*<}M~!Mu6rmYC*>>*hz&5f=}V0Zyy+_*N!Jynz<$t*!KiE5aGUUYi6- z3ASj_5Ta&Bbzbb_QRWYXwuN0_xLQ+yUl)JoVw_eZ?wk{x7IkkI5u#}EKagU4p(O=( z{z=p)YfsSl8)N@umf?dBg;(Z<%_-T}B(7m&e;py~S6F9>Xevzv=4hZg8I?I<#S>f2o%%E+lqa8>?~1 z7;8PsL_4i?1{T!D33b~2zy8!EBmcVBWfo9J(pR^7$St>|5r8h7v7m{CdujY1)$D%< z0{=H~!1yk#{*Q5HhmcEf;yXt0!~Bnd?!Pg@=O2FjR^KM>1L*&8+g;S)+Jhtg=i;`) z#?2qNARyHaskz?hk^uGu1RC~oFy)5DS=Eeuaz9}iVyRYZ&P)@MwdhW~2*i2_uR)|o zMl9>_?lY~+#k>3aMU|OxkJD^V@otr?r|k6X&8|tWyST3p@t(=)$+vgkE3RG*x5wdJ zLeD4`&i<$0gl~%)NfQD$m`C+`J+{xQ1oPnY==1OmL`7Uh$bcfIBBCNe6A%*sPzXp2 zGz5aik;LW48O80yp@k8#g0Z5p!m;AALa|~g!{I>VAg6<*!=yv-pg5xJ8v;#$7(i7Z zJJ1f9=4H7q@%b>rIYl!W5j>!94-rGJsZw`x47NL9~9w^v^ z<2U?KCHGQ<92JrzvBHF*O`_|>4pMAIu=6ll$l*c-aV~MpMWD*a<$*!O&2hvXYwSxeaa1 zimn%Oaw{yi0E_C$aCR2Y`RWJ!t)6;kXF4UK4XIs8(VOFsM(B4AE4;JkR# zICntagfX>-9$_g&Cxg1g!i{3)&qDh+%%(w{>l`0wpZ7#AB>Ibh#B6iRTTON}XkUoX;;lOMJiK2is5iXr?IqIrmN>fawR z9kk<2J=CNtCr(Jwsc?ICw3!?J;kC4TS_r(^c#aX-BS9i|F>jBheFF)$DAdb-*Ls6aV3rh5Y&)M8yBfyKFU9W z|17#&G_QTVW%cmrzR%4EIYte6<9IXeHR#mH+5I}mzGeUFxd(I?HhW33jYTHl#GR5L z-+whAv2yhY{IxC(u4jR&O_(xFGpNOzY>r5uBAwxDX%yUkkDI(hxNiGC=xS8mzXR(d_HK(NYP~VaJyVUjsS@m%i zW2SxnJTx(E3lQNjrwc}3Q%DjTQngkh>QZl>xM_4mUhs%oddbuawUkRxuvW{%#co&1 zQ~zQoke!VV!>Ks~_@zC_nX1aIhZzZFk&GUlj&#ULB{(l5?!i^f&vNMn3ctWt1~Y7N zJcb~VY4Jx%h^1lO-lJ)FlJ^M0?CnKT>QKpNQuyzOHXuhd&-wU*s9wU+z<}8VJ8}?J zCPFIDL1{djF`W93$`6L$RR*;bI#1|!S&!#Hnt6&J=135Ouo23>kl4N7Mqyloarj&ECKc$zqcs+r zWgd17WE`T0&c)A*+tqCH!6Pt>%Nahpjo;_ewg2A+< znM1~(S*2~vo3P2-N{mzI8>lcPBc>C8_``Sp*3x0B;}>)O(?2%6fq%|tx>?#N(d*Nn z-FiWd)weFmBakm{$Xf;^MO}5S4@SQ_&>Y z$Rp;(vZtR89hJ+f@qv%FFWI#2|MVH&RSqaGrb0w3RzX(8eNSJ7#?}Q~Bmu2|^k4s^ zHXDC0FzruoAwD<&p65@Nsl5D;UTiT%;}=&w(bnilt<#Wdkjf@z%MVk-N)b~nA2zoLTHj_1M9_fB0 zapPUSFkEW3S=G&~5(a)m``A3^Jn|dY0U7`}=g^OKfgjMf;BTmsg1A`=l9z!d*K&bo z!VNbpp>8bhj5BQR-%wsXl3*yn)kr5^7xb0IwZ6At;8V!`jt}A%@;nhMNP{4e`L%Fd z|2s$^@{TB?Kol~UJM5FFgs_C@h{%ZWh$y89C7w5JuIV+az~H~eXY0UJQCearVy73r zK#g5<0l~?C@Gqo+8veSFj+j6-;{VyG4WZ|Hmch^ohO0TWLg+~W5E)&Qkx1SYA`<#7 z3I)Vz6SBI01Npdq@J+7Zc=Ow_yQBYJ8Y26Dg0Dcke|U&Ju)=>QV>zB3xPKBlg6)d| zv3wKyTX&C5d?Wf54@z;mFrU@`>G<10_WXC_g%fxJB?vw|y>Eu|uIroA-+C8Gj4gT$ z*rRw?24X^O`xg>$uZM?rq0`t?SbAXg;yOn6*B-10*Kj=}KD(3k!a6qg+xbUtT(`OH zylf%0gF8~~GxD8m#&v>rLpw6<3;6c-Z@72%6A6V81vQ5fBa0IH^F&% z$$+74zXk*)SiPf0?{EU1wPLQc2A{HGC(rx!y!xK;d2V?q`1IU!*m^IM0@ib{AOpXm z0XEl6^Ir)d0><5;2x$>77$}DyEX?zoRT#Qe7I2 zTj<_md=#l;J~t#3I0O@K$Q!c%gb%@rWtpQfU?XM%Ss;KsYounZ4kc}l<{0%nYOZqd z30b}$8(J|^v}E&9O|w|sF7$?i^||t%pkp)6TwSX3oniNI1v7KS#(xt5qr@5}GgOa8 zwk&re~JGNDJ5 zcr)}{d}MNYc1&On>8*%}?UT@>I;mGUa014o9Xf%sA8y~5^p;jc|HA*gRO}Heo(ttx z0@(yW(GVNf6&=^f7YCjM>>+jJBDs1Ln%@=nKX(iDvnDg?gmwhPp{*H+35WnC`Qw`E zl!&3jFaZ&9ieAJP=|w`9fKos2oH!- z9pX&A2LzFYt4tGT(}|TP<%dxfvpCW0G*sksE9_THIFZHXD^IIH!R0wP_?2c^{#YPm_!c6HI$KOx}_R0+nI!C<{1A<_rs5g35?Zl)2qf zlN+|^)#UYhyd$OJ`~rU_U|%Y(8m*eDMOTH)pjhN~+^`)-1$%$Zyx5^i(V#u*7r{>N zlU%6ud*8kDKj=9YTBkY-$A)>6@YN;&X`-N2S{5k>PclF;k<#ajMu>n%UgvYeL)OXQ6DoMgm3X#uv^!iAPsdxn5a(G%&b z@A^F)hfkB#mEt|<`aO^3>yLO}khxIvg`-;!q)nMz44+@Tcy#k0Vnq2js>mEC$(Gx) z;;)(osg{5@fm|ZWVq_b$J+W8v0-~~=H?YK>IHm-Nk?rY`$oq1!+!A2}>-ow>vK@$c zfyCoRz7yH=__}dp@+|7zBQ)O}v(mbv1u4w~9yA3sc?Mz?c>d66ifO5~riD7CVnSGC z3Xi5mxQBTd2>w;(|A=5D|T+{S*v*6GmUcD6uh3R zgYiM*j>B=Od066B?G481587cO->bDz~4dkj19U^Zm0w$ig8#(%sEE z8VA6ZakGg}ix0K0CaVpqPcyQI+jZV)3k?9BFGPFmRNSk5V~~2dk0yhI4au zI-aU#!NoiAI$YWxEZ#wRbRUL2bcAw0bvOWO4|F673S~j-@tk=ScIn>*4O0T5 zxMMpVGm<(}LnE^L*dRQYjh`CAB~5@z7l0~du^uU4D!CN>HNA=IvPN`4k7W-vPXVuXyOov00+(6+k#){vA;n6=m07Lhn6_i)MNpNkE5F-==1{MQ{DI3&ysH-vK7hpZYAlior zleCut=TS+WI9VnrrhKGJh0B+WfBjS*B9eKQksj?C!bwVTJzdpF4xCp46r_urnAWqi z7Q31(2F}lsI^zfaGdoC;j%ze-BgP(Q8{c zI*+W~IIXr^d``g|$ zAAdQke7hL_EFZK@FAd1K%Pzi#RJfho$IcjQ%uW%#1ujOKDv-136m?Gqo0_yNoXXUf z*rKGDqTR92;D8(pB-X^pX%6K|KQ})9zWRxiF8f>@VcM*~?j(@jE`*jo@qA!v!kEud zXu0jv4bvTzmwFj|#SF3SM(tyVF?CMPrs4VAGdTD}iRwupTnqSu?733yenLgfJY_IB zP+t+>RCwU!SBiJ0iocxST5nfw970B^2KNg4`b5EWrQb3^x^^WXqNE?p-4MvQ_C>Y|yd> z%|*XuBHX^eQtsJss0w?>Bk6q<+Nu#Fq$2&21_C^YuYQrnV#j}x?8g!Ud6R+Y5GEn9 z2PP)&#NT)LI~LR(rOpM%wG#q2W%jX9wS<5o$ore80D)I>5*9X3l*ZCWH=n1n6CH6+ zgMz1plI_ZpT>5!vRG&J|kMGz@)a@na0V7@@>zHDFyT0Yd)nb$_<7bY_a1x&UGfNHu z@?kvzZw|ETHbxeucDj(dt^wId=DVXpB`{qMe`O`?#u$XhEE7mWHjSV$JxDu!LRuQY z?WD^Ch%Mtb3(I;LasY5VD=i^?rT<>2B8_>g?2j$yHVe->7}DAq_t0%G^leG&wt^-1 zBI0fzz70b2hzYAAF_{H_DMIt839}|PSp|Q=0IUnUZq@u$6qA6#d04ISl=vR{sV7Vh zr9nKvd-*TeT7*FxbsTmW&={7+%kA0YGvMR^;PftT%5b33M$81kdQpmuLxt*CiSdF) zq9W+tmu|2S{}KrOiN{7h&PCD^iD^z@wLvaBnVP+IqjmQIDt3g4`9T`*%Q-i5Q$+Sl zfbCc9x_^a3p#i5s{iWBBHFI^HG$ z!1%Ev!na(iH{w{sYhH!LOBAzlVkm7>=pK*bxFYRiD{3us9QO6#ceCwxSpxY6v-Du_ zp>gAgbP)31zRvd<%!2ey6Ny{x#%y%_UTKvnMc1%N&fhIwhie;;@B0LwH6LQSP&L`4 zZ>YZoPg#Fy#($FCZ0%jKjVTa!bt{tt1jL6*DYkrxt2XKB0Hkz86~0~$$>w_=4L1R2 z*N^cLPR&4%bnegxqlA zxR2gd7fHXPvt2~Il(I_{Z^?|K=^x#&0j20WfZ>n`cZd^=6skyeW}&3`6)gr(>iW^N+EG-xiOT#;p0Fb_W3TuECSA8VH( z0&ZLWjxJYNEX0DScV-$-0&czl!TrmvYA{;1!~;`P>A!E07hBj0_a&>B2c>x;K5Ta| zX}O0wJ$oD2@3MF*MsKFBFolFBtuH^FA~f|)9WsnV_R{)$cb_-SD1B$FN|nRVV9Vpb zm^Yd&F;dK0OGi4bxf9wAw4n}o>%1S6Kaw>BzCv`aBo6QF)>c$GIZk%~ldaAprxaiM zs=_5-MjEC8{HHEB(ykk;1mzKFu4}VRhR*fpSW^Xt?+c8A(?}AJKqZ@V1+|lNRSUDP z)vpCj?=_5%0Wa{0lbzbo_d29chBh@`Gzl!2&VX6qQVsBhbLyq&ZyNN2n zYuMV=>ZV*EKVisOIAh58>lJBm^7Cq4hp$#gdtRq2PvaqZy2P&H^rP&Q=~S(Ie3Uo+ ziYsHXx5%t#{OHWu*BS4vx#k67Ww&>Zu4gv&1f#X9boZ0w~PmxdrkX`BWn*S26ZQI$}>#bhd-tU~I-k!@;I_$vm)u^>kW*Ti?2P|ID zCQP(QKgw{Ykv7mT{-;KBcUB? z@lbnY`~wV^9?$iM1>>n#oR+7IW8NXMr2kpzZQ`%!CA4QOYbCH;()H=}41E0;Ao2gJ zO8*-p`5(QhHFAPQ1n$3HmXNR9&arRE1Nyi8#P}c316``EJ}nfWSkuQ@=MT=;Cgp|v z!bXj}U<)Ts5~r~R=P~D@pNsXxdd%{l3(d}Y+tW6A+l{JL>6E{=l?74AsA3ReRH7*O z;20&Yz;zMGcL?F(Z}tNwritVGwiDkdTlI|7_moL#!;_Q4_w;Rl16j+~gJedlD2j6VWyi{`=E+IvnjU!|t6z(ylEP zU)!j(_NB|NfOmdc$5Z5U0ebQ?nwtXiU}`#F-#&RFbh>x&m?$Y>%O*P;+hBGrt4GjS zp)Jgj){AjSK3weF5y#&ua`ot|`EA#NY1rNk`%N+6H<=Jj+F- zk4-^0@QV9}ewv55A3ss91mAty|Hx()j+-U=D#E?r-xldz9F-~jIR~Vi4Ob1Vx*|)C zRvL6-eZ3X>5yA`E)z#(E6)FH)2fP!kI=kBkEJy*gxVqk4Ik9kEUyDG^ns@`OVxE;3 z{EhydqY)6&>2`T~X?b;eK|+!QUuKO6q4NX*-hBb>#p~MomQ?h~UHLrloQZ zqJ*q^I>0o~kWB98z)}Zh8td}9htTXO2%wO0;lHndJT4F?-N^T@@J1Yo@D87({B5R5%&jWOL&1wS6m&WiS0Lu|}t8c%M)R$ss z#a(m&3Bp3sq|$O_W+7_Pj{ICKHOMH40{=HFLa+{8D^xp28r>n^88L}}5kxeFWIFyx zSP*yOV3;EvZJuvFzS*cRT|S;FC@z3rwxVJ@uL=q+u#hb2@nTd&MmKUuYH$AB`Y$K0 z(qL9Irk0nlad*)6&Xe3$5q<3w&FF5@zKpx^M~0d_TFgIc9e`|K3Qq#vfcm4kTqThl zb<#z+r;Sw1vd%Qzc7#3p#X^+E=0~WM+yK$FBrITHgY=Vrhv}Iv8Ph<0M;74mjFjl+ zQ9H9viFa16V1n$!+$;7jKj8Hv^S}U(1NP|V3I8myd)&E-iIb$u3Q~9Hf(Lg_L{v4s zD?Uvl_0BQ;PFq$tvp@-g991=jy2392+}V_UpDEnJ_a_4s<`fDodrxIGm973Ld|B10OAKF^J1Bs>jIXS*>w$5A#j!jtd3I@6#^9UCr+pa7;Iz4#@CDWc zzVQ}cxB+oGNb9Er^dt)0xWb*B8M`JbM_*5ars`It6~wxwJW zpe-t2+Ohzi2VVLQg;M}vb>7ZD@{FQcPZ#Or40I(|Z^(h*VJjfJ&Ch2h&k(K!c9jVb z(a{kkBJ|yr}AL_XcE+(v#KQ?zs(s=VfKJvSejM-Pr68vi?O)vngxH6pp-Wye@6kFx)y+T z;QJ(#T$E(Ul@tlOCGKggB%Lo}t9E&8BE(6;GmyB0n!_EwKU!cJPXQ7(PLdTuXE z2~@c>hzh__Ef5kD><$Z`uIu!i`9UUbG+3SYNexZ=J~QBghNK;m;Xo=oDo-?iFODA& z2J;+}8s0ciw_SP%mr$d{!hS`Dy*4(Ubojg13v6N}eHmbS7L#~6NG(2(rc;cgLjiT~ zW2Q;7RE>bEKEOUiTcM(!<=bio=Cb!Xnr(CRV-2V9k!n(mM0i1=*(g*;bdgp{hrX~s zEPT~Pf_CxvOk=keB@em;a1`X4m9S%KxozoRZOUm)^I)-B!)54na(X0sfu!t1Ro*ob zkTDT8!U4eJkmgE|U@*y3FUCtJYVsPJ^l9RL(h;g;ol;HDQ=5W{PCbc(UH2s1{TN_w zPr)-)_9~Og_2>hOTlh7DJKttaiin>-cr{J)RR;+fdXh6}AIni3s1g0oY7<%HmQM;d zSNx%EgDp7H8G><|IBwmyRBFmNsBdo(Fc2V8Z2>4*7Rz^Kgd2QODw~MeJ;|a*I1)9A zd4C3@Mqt7XeznG!j?aJl(FS>X|d*%_iiZ3MnITOE7 zW=Ck(a+xaqM29g|=@FBq`52GR^@}yq7+&XcW_H}tN}cxG>U(qKV8=f-k`5TKxY%9=J=$s z^Mx(=VKR{>1UugHAHn#DTlv3k$(8K8RX@Pf388(?>6aiqqw8rpOgD!~4LPQvUE${R zmt|f|_SZ&UpfIxg;7c}CN_5D=CyjSlmx~SY01t;h-9#p(J!X4u^lUFZH&+HXQK4sH zCDvoyT`;@rKz8uNQqe|saO6hf$RA@kY&SUUSV(M*3Deff10{@h?de~8?CL0LdJF(j zyz?d3*h-4;uQvzLx!YcSLyv*$`Z)cz`rRQnK-H#|sBkPDpx!tyZzdPlN&Vsv53vk@ z*~ej562c1tgE-zGS*;hP9&mB4JjRr%%Y8Hma}AZ@peIQ(mb2fH$cGvYE^gD)4Do_! zjTtu`pcV@q7`32zrsgp+orSF!E(eh91v!|U=k?;1|0$f;GM2jDRU6!~>GDI{0ZV}w z|B~p3Tm@Ev=dv?dR66_lQ!vJUKXza!uO*Y%Ag*$ek6Ap`=#RRsb<`F{=GhkRIoDIZvO4DrY3 z%G1k}^fB^%T#*D=ST`U^DJU9D5bqCL5pJ;`3817Tusm&I;zMskcZsnbSJXzNv`}j)5arC819!&kX)Pwh&KfZHh-vXWqSdyU*YX zpTr*mV*mUS1?76HZNm=SRHit=pkeVzufwL1M}=?!NsNR}ccU-pKLH88x&Mc%uK=nW zTDvXoZU-;!?oM%c_k+88fzyLiytum-cXxN!Qrz94*yH|p-aqeUzD)MYUYSgiJxONo zwb$n=u59~a<9j0j*-dkW47SQT6XX%;%PHW8yhj_DzU2q?qO`t3wB6s7!j+y8?6^p#!Zw(A)4*vo!QOZvV&B=2tS|C0(IBf3dswZ!I}$b{~b~)8Vh(N5|bB@1!&z28YHi zd3YallifQX9{mad%?TjIq^>+U(5Bk40OV$Z`N#Y-nxoD{>|8zR#w$QXsVFBr3)i-k z7z_ywfKUA==wS*S1XMC`}a` zh&(3Bn(!XeHlC;+XULL-0eSe$ZzgnLi?*E-D~cJ_#Qi-O(o=(QE!hK&QVq$k=mixj zT9`VLLY+ANs%6y9#p=4zZ(3z!x<9_qpevi8fb}3fpidAgsAfb#x$&Y{U`xnT_zIdD zaE}yZ27IQZ2?w|yAya(I08$@5Q`&?9e2ZXD;S2#rLNJinGKEY*Z%OR2fM%v~h5<9k zW-FJuYx__qaj7i*ETq zvY>m%|4pK+#9>Acgjsk|uld&9tx160F6bhE@ zYjiYN3bq#OWNBe}5!Nj3&@b0=Jk1=DVjvN}Jjo<0K3*tZ7Zrtu&ajcx;qLJJNL6R!dj@Xc%(P z$`GHn9)Pr}EADx?c%T8ub!F^r$aKrF>FM04dd${v8Fchg_LIiy>e{2ZUWIy}8J)TJ zOUoVMHSz)W7*JjoDvTP#TS`aU|Blwb0c98iqCx#p2V7W;8aj0B4f$l~rKWKUpAL32 z8iv~IytQq!-5sG9{8HM>T0A%$fH;5CP`o76OwhDwmWp;r`LULkAz{x$G91AH1{-XI zg-*3)S9e}|)xKh*jiyII7JuLVHw4o|?q>#V<;P_tgQ%Le8t7%{KD`%gImc4m2=0V zn^uQLN$|a#Uj7dN3#0OAz(8%9i-b+)yf4eN(t*5saMjS;hO;teqRAt{TIFyp&(o&M zx)j$cooAdk5r>Uh| z=ahDVMdLf%G|@z^RSLEXw(2P%$m~8abmPoza0bq@eyzHf-2> zTSaOW6`aj);((rxt41F9{twJ&?hJK zM*^w1%2c7;gwBa=`R`Xn@Ei^YYg*obCDcK?F&ql^6!bSy`I`Uq0VK2xs>VAU&((YslhOu9!`-~QrOCw#bm3WOl>WkV1 z>0(PbWhQuye$+#Oi9WB^;OZw@`Q${)_Xp9JOwaK1pemH}a>Vec28?wXTmw&6PB>~_ zh72t-Y0C?}pKZ$ocI6Bh`?(sTT&5h>W&7AD2aA(jwkUOZ8(bi(DccE? z=mGqVFX~o@^igyuDDTmUAi{3jhn~URcs3V4j?_4d1o1($^JDAq&}cvyfdno_d+{!p zR%Ym`ypUAV$y(BrLGoDO&61u&scCYM+5RpRUWfxavWqQB+r){?Al#!?&Zc z+?T4XxGcumLdf4DH-Z@jnKN9=LC~3*E2pKD*Gtn-JVDit8I#Q^NwF^i#xQA_d#8g@ zg-b|9b0bPl;leNB;$7#R7 zHHp+}mn%A`_$fmRhG1sCYzKPRF`l*2O6WjS5xlmg62q@?k{+LOW$bG4CEBZh5!&xN@I-nZ_1*?u2ZTdr@6)bZrM zIjQ+-Ld9-sDdgrM7jr%L-4Qh`X^PC}Ul8>yoaz4GZ0vJ`QSv}{8=R5C@^+Oc61xKk zhdDv`ITDA9jR&?007$u6^omjjQg+_KC8z__aN9bsVX&(9a#HNT7lYVYu71ykMkVQ* zwGRF}&WaUDsVF%&&Ct0oIfex2`ROKFOg4z9k_d~9Bes>@L%$RYJ=^fgN!uZzN{TLW zt=VHyN4<$o#==2dSW!kRObuIq&3VFzX0JV65<{HL(iazW**KWOzf--0NMM z6$%wx%v2ewXjWp`jYb|$e;v>t8HqusSb`4v>B_wbWI+enNV+-?;2_kxq;#Ms#2utl z>#5fJoM3Mlb5?6ELBA%ZR$cQ{WSd!FaVSP3H zKUv#K!nU&44pP=wBNYv@*WlW~jy^e2nb5=7#gz<^1Y)#s?7 z9GWFABplG!oq;2C8UjTYM9ziTpF=DQb1Tg764H4f&qUj)sTlra{G_+r`IP)KV|Jo; z0&VSOc~b0EnV!3-fYn*P&~Z_$-3I-{po01qc+@^67mKK-eCw!`x4HsCQxu|ph-j@s zq=)gCDJT@j!1rYq0scp~j6N<{A67w(UD<>jSTj(H4uI8Fv$CwJ6|;T;^0aSjV&7_~ zfu7&Yw9A+fw}6sgDWr}9_Q4#V^l+&XLC3VPoqmhwfRfY2zRGZOYZ$@JcMm^G23PEY71@byes^$HTcZrZAv8LTiatJinq4m9tgn@{YMF9a1D_gGb9Wv4 z%*u+_$X0Jd!}lHQCl1f1C)mPsB12sY9*{|j2f1opM{QZJ@hbL|NzH76fxKkx!4emb z2CVf^)UhLPSP42Gnm`cKpx;+amhEwA60Jd!fhr7B3n@3n$qIh9cvh=t9r@>sFecMT zp)>cUf&OGRdmAi)$eHE*W5!<#3(#8s?BocvxaWB-ewoB!W+TNqRaN!gC0qZA+y$vT zJ$M0=rJvhx*ucupwh2kZD=M*7*roIn^dt#f?zv$(L*s(>G%wL%=Vp(-EBXu2r60)oY0xx17c9-QQBP6w9pvdSPIF5 z_C5pJi;zf;!o6|qP2)|p^fPW{p;MPLDnRU8QTgnSSbtPVR9{7aCh&rl_c!RH zV%DsV|BS%=@izf4dWq=v>_`E7(rNjQs-~uy3UPk=a=fUwxRhkQkWbn=M#PGBxm};E zH;s{Cc~Hgmm+vD5C(xOt3?S|tO%pwmtXT)th1@~S6lst8Fn6w@0^bE4w3j&lJ=NVA z!YA_up&Z7uXsX5xf=l4E`frvxJa?|%0_r+ap);xn_NfywU;1HPbAKVrC2)D#{Q~l{ z4K2ZId}sb|SrRUMhcP26+v?h7q4E?zJL>7%(v3xI!{h0&WKFMaQQ0?h49uxj7V_g2S?qrYQ*mXz5ejAd zZ>jq0sBx(1J+RQ<7tVo76J-fT$Y&z`&w<8jWjsk;`D&z%)nWeQPB%3c_hw6!c@W*>(@;3}e|}{)N1bV4(QrH)Bc3 z8(YQKmhhes5HvHHGaVQS-e7-wk@7U{RSvB>Z;k-65&8sv;g&(L11uN)>Wb!ckfK`A z_R0SrIz(YJ_{4$p3%hVc>|@P7Fa1sZlO^QtHN=pX-_Ifx4ZUe?%u;fdh#wAlYCn1q zZ5@ak*Mo^v_ITl3YJz+BL_@R_691w6hR)oAK6#58{)Fyh9@n~)hxv{j4#u+|Z4aJ6+sLQt18A^FdBqo*pG z;DSdrSQ3b7u86^&*k94z7#Tlac= zn=~L>X>4Lj*v&_!({VokQ=4A}L}4nAuDg~OfTgHtmJX#c{+*e}&InTDjT>E^H4SgTYFj|VWZGZ%2Xia78w4m~_?Jv7|U{EbT%faF@XRvPVPW4YY#E%qxIw^HJ-u;3Gq4 z2+pv+*g0(wp`Ai*opF2R4dZ!meAsj$JnX(=<{A2sU>LnkKF!4g(p-Zp_|oYuh~XZZZ1>$<`VZMeG&=sd9mBnQ z?pKyM_DZ2&ZTZD*ButqOCBAU5bC?9UffD_(-K&y6C3k*M5}&4CIO#Aw&>S4D+B8$fb)?~_@vhF-0H)*^wn+2#^@omIj zemh0+e&gQJs{PW3&T25U%+6aeR=(x(NCCMv+tnw%

@9Qk$1f}YRWAG& zHXAOuVIWx_Jrll`6(q|k@~6~+o7C#R!O0@$QIQQOE@CW@TtbpuxYn)A%4Tm6nmJ@0 zU=~DnL0Zq%H}zwnG;a_yHO0i$BWp$J$Vxo+ka0Mm9qHD-B`>?c;d6-lRj}6}7ON#w z2~XGd#R7s>lDRm~y(G`ExFFvu_vA3Q?XV9z6;3^m2vkjPLdW<+P6yu3Bt9;nytkFk zzw-ZvgNiRB*OriCB0=pQda*2oQt@n1@hlK=wWQklH#O&%GCM&`XOsO(5q-n56ZNS`&29gF076}?q#p?Umt8&9sP922W70*`NoDn1=z(-joA z3ak6l|EC*a@-HLrG`sVPB?fw_guxCd4zBjhp3!Cd`icyRwAX zqztvWmEj7X^(a!v44$kJjl6um?fk7?o-uEI7-Os1P2hot{|OAK?LOKR`GN&#|4sU2mW3Eo zo5Rbe7q*{IvTz@0O$qi*35@nbOx*Ig+`Ds@+VF2ibRm^es4_NxUvmTepE0;zxNIK? zk6<^&F#W9Xp*yJwjS^6*SsC!C=Gc5m9WX{}4@JTc`>dvg;Q0jd^g-_!;e5!g0KVI7 z6~qN1W5T_|<~}-@H6WmY_2V1h52Cmp(3%jL!TqVQ+_RsOzKCBB=nXzuN2o1Bt$tw5 zT!X|q0xMVAh6K_8Rr0b=WERnS%Fz1Z8Tn9q;H#SyG!(FKA_im{zeraQV~4MsX#7K! z8R_-bmoZ?p3&}?`27xa$y#leEqN=V=UJ$eL#x)MHXv^>(ddXbfco_pwcC#W}ptg8z zuV@ya_xXwHQlUb(Y1Jn(RmKchvs&{4eN12?Lws@Jtm++WLa2I*7%{3$WSvpuLkV>e zA09}YdE|rqC*aU(kgW%A$QVT;_N{b^Yet9^jJ0y>9jumfL5+1LaP3-A-u+u*p2U9( zRvt2Kst*;u3t6q5^4YMk2o6*Ymq@9wKw9l6v&QE?qpWUV#em`F@tFe!H`*}@qNf~f zhr^E%H}bt*uznmF2^0=%j$9;(s^&+=42Fs#6C!S`utxUNq-N5j0x7r2mLaj4xwjl2z z0!C);b+Lho>19iOt&p^2`7ypF*I^X7d#5)KUit=04>E(cb)}>)MBiBja9;Ad=VKm) z>B@3vgimoINY#(BW-IpI)nFIb**Ft01o5TXvh-+tN63IlvS;RBvDXQCw7Y)Rd))}0 zZr-{@x_Y*FdX@7F;QHGQf#s^^D|C|_Qu!weHLu^=xmI80SV>`MXHba7KE4Q`)?|}O zT&#c23fiy@V@)8JOAIc{r@Vn5==yceK3~g({vCOgBnaC*zhG;3c&6!mCAQE?)svb? zq2-UN34u?(qYte=#$nu2#vEb9uabFx20n}erzAGtkf>Rgz3TXUj^xZzHZdlgH2 zf=3lDS@~`TVMiVME!PfG>#wM13?v;N7Vqk;o*=loAG6Q0fXIM0Wqxj9nRSF0w0>UX zY^o63cQSWK+6A>naaM3I>XoVuIg|v0dtrH5j5uqAI1FDNKZS^9ClrCe_T9yg-LGh# z&_X&+HXrIpf4va@Y-27bov^zYQdbEqDz;pFbwu7Vt&qT&L)1KRDZc?k2|*|DY2^gA zR)J)GvVA=h&vF7IZDXFHK(U$HOA`wsD(}u1MeW-SHWDCK+oSAL@d_Crl~rJTm%>7p zKvDkIIW_dhjl?n&C)J4=n6b_1$?f~v3P5h|qe`tZ3{6||^lO63Y22=tzFwVwI>Yz# zgw7F)<(raEjqZpcQA`#h4y&X~?fx~b9tZ-k$UD^SNd-7reP1!!-ZV}pxuR+}dPtF{(Hr%j%&BA)c1g2Ch zL`(>ytZ=)95&b%j@HToJ%CPS~zL(nFA$X?6#PX}0nn)@Ts!aiT*kcgl6e$3OW zHEQuXWc(z%V~Pi;2P02~py&heWBi5VGJ$DnBxD=n_)u zFnHc5&zapv!^O2Fw}_y;&-v*`kk&M)$- zkt$)2>8s-h1S8l)@6AYFjMA@Rk^wx>{g=#qJyjfB4NA5n5kVhJeUhO0E%)uzp&Jd% z(&4yhv$zNZ5|kEcAS8Vy4vtgmv!4W6ky&}LPvXLgU*dQEse7ey@o!$ho2Y3iU0CV< z7$@-1-yy+s;)eb9EwfADnuS|_%$4-(57QWNHv zSx!wOcYW*!5;4jVl&t{sc)(Nb)BnKua2Iy;9I&#V-O2*U9R z;!i|r!thLNRH6mjS~mQ1sWd@$mowZES|7OrHZIG=w~TBOOqGfAn*QIJ--r(5Ml^dH z;aLU=Ga^+1&BdyS4|hX^pHZhEAo-I89BHm1PTN zs30*sjpp*6Rz+a4r#WHUvXA_WKO&~PY88e_4cskGQdLK4V?{2fIinH>TCW@oeN*cU z-(OkH$BfY-SLDSWvz}g|t%KUqN^hA1gqakYe)NkF^1mSB;u{i`A$1_Na@g}9IC*bn zok@@AUCs!Xa4EO;ElGzx=YpgpJ0(4lPcHu#jOm2~+YJ4+Gof@NsPG~P73rqqOQs*R_mj+mFPXOw2J&738Q!?~ zlMYAnfSLI!)%o9m-MvWKuw` z1CsB4<8=e^68^3YTHaDHnZyE4Oo&+?;CkOSqpS_*b-RPn-`7rjj0f7)sY0ZX_a0MDEnI>Rf;K1 z1w(YeN?q$A(MSR_dBYQyS5AK=RQ^i%91GR`?!-%DcNv|VM!jc6d{YncylTE(G7yvV zqCoNKY0;iW{9V^OpP2a#_e}K=7`ygU$QG1Abve%YqWF_a)?)@hG%V52PPHW;MNGMr z1?dmvpd2OpxPdA^@2~movdASB^FwHchX(4_Pi`3}YAhOu?`MUWJ`ogK_~U6Cq++|i zi7|gMkg3bP075qpCmsGAy;zDDsFKOMaD^y{0M+^WMC7oz6iowHC-^24$D z(?@VjRM7z)DN2ffr;j?C<6xSQZ^(QtdNnaXVBnyzWWHDdiE_r5?5|5a_fL9PvE3oQ zxQMkjXwCrq7{U~Bo&Xri>)$=Y)u91?0(;kXH7o{^w_B$`{%(k|XVlUi0U%B{{>d}b z%z#&K(yP|Ws8_#H0O4!$B^d76$QzW}+%x{l>>aj=l5_KYi{V}WDRdKV_~~t;_pzS$ z{fqno+yomHn%e*m>&@>}vBY|wK^->Y5}F|)Ql%e%F>OYbbSP=~Sj0gOO)`2---M)= zylRZF&MYqp^u;Q^I}7&BM@_OBPxX9gZf=a zo43lTF9Pb_P?B1dj_qFhu5xO(1QcH_sJoI}ym)USo1#D?!oNnORYx04+EWcq7^zw~ zb#}_7HY=@ugM)E*PohYWx$#wP$NQD5|MG}{S@)j5(BM)Qw7xggUZQnrs z`c|e2GJgo}_0#E5k&~&`EpFj$vrCjeTMT zf`%qeQnJrzPnC0~DM4vc7(Q09@y(8(%*LDAPDvFFzyab7=brV*$p0MZNRROw=O#&D zh%X^@D6|`|2~mF(3j{SbR=%(zXT`<@(3r!gVdlPNdLM>nx_hK`tht^G-$v|nYxU90 z08fUx#f`edyx&ERBD6*iq=AAWmfLUc36JxNe-8;ekapXC5GEuF(~g0ZloB00^W9B( z>*9zdRe5=_C(vL zjjcE5>%IZe(!v6OFg69$;8BFB`I`Y-7w4dLg6Twx>5-?Y0Rbk(WP?b$hS)?M7CBPL)5%_N3!_4K91}Bwt6e%d?K+iM z6DwJ0K6<}O_%`i88Lb2aBHPU&;vXdQBF1Gy!p_ZiMLJ*1c5Q<=WbJIi zPs1j-B2=SnFQ=pUdtohs%fGBP&Jt=vVu9?a>`9n_QczUbr~WmPQ(n;~Iem^1IVE(d z=U+bOW@aI$c9?AaM&Hc?`gopnaqfCiE^tmk+(t>e!S%QT zI?P(HS~nx%LX+c$#3{tinn!<}E!m#cM%(L~&tYS|+M1lSDpICV(voSF;v@g>a#S%X zFoR}^E66Bxc@c40v+%p&xRjZXZmo9FJwUUf;k=&lcl2;8KkaRArG_Yv1;khjz?Zf} zJhTqlHHpKdv<_Q}Zrmh&Ck$rHRVQDg&$S4#Jn72@FAF;fD68K}~J?3_V9ey}Y+l8D)V*L4GqsvcaAh_s_Bm8kG$q;(bPiUWgB4+5nV$mc9zuM1oA3Dlni36*I|K zMisOqRvwt3VnKYTzSPfZ0fkLZm?yCc?Uq=-g>1o5{$XFLIR&=dH(1HF?=5s^$Slc3 zTfcgAeeIN6wM@4FDEzXd3pTY$#srW}vcr|&eyg|VnXKF39QWju=gVUKF}|wY$H?ZK z$0?6XKx>{%Dfd;~hAUB_-nx9s(G}^XAp`VXHME_F;35X`J&8ELY5^65;GHNxyUG$o zxi2T9Uey=l&$Y*}4s3Bn!KPalOLHY=RI3Ur8beVj`UC$vDMMvfz@)Nne!;%$_^Mwq zjD?-JYhvdry4aFXagsaVvKcYViIccX)s~y-&vpVMs`BXX@X)YYYNpQE9_L5GHA0Z8 zm$;u1DpUTZAH3ss&2-C(2fX7xtJrB3RsY4QCzwKYb=k0;mlN`yl6}lE%C$Mq^da|U z0~IP!z@)~Nrft*kfCvgscZwHJ{(wRuUMiJ5dJ9oIM1~^xE2jfN9~Y)9VxS8vju55h zK4+d9*u^k>FL7AFzm%7sZ3mB>uLFcihxyz;*{KJd7G8$mX!#?#z7dPpk`xGVidyc3 zt5Y-lo0=Drec@|VUZU-_5xdB86+6>Bn2Yw>*zSm7)C`{^u2gm=y-g~*O)9aC_CwhR z(3y*))W5kp_U$fM9Pke{<0V!SeLBXZ`)bSG|Cb8FYOhoIz-PqLyC5Vzfm$;NPisD> zX`m$4>(CDKI?YBVOQ_d2Q{un$@yDv-t9y7SMI8I&q_SrD6i)pcQ#Tk7sZBB0;ZIDj zMH;hjb{+I<5S`9N0h}qd&1Pkmq0%4vWL4JSeNRa6O{g!|vCnJ_bsDBs&YQ~~s@?S} zZLz9lAnx6|CxWdM5zW341o=)Ov3~Qxu%u)8)_TKEe;r#oplyKjKN54t_G>NsvtR6*Qs8P9;+i54k`GK9s?x97^^S^Bw_GId6lB2q znRJur`%{M2a!(yMURgJ|8pp93UIe+QBLuP=#b}>+yjI|puK%kU9jGHC)l=YW@5&#j zB%{8hXr+XDjgpm;)&lY~@75dP85+KGWgA$1=P_JBr##)UQj+_IDZ|ASO&*nfn&k6{ z*rJ1)M~246uX4Fdc=>Xc@%jiU0zoK_6~A~_9|77^W}-X9i3SE7zHl$czkd4(G>}xk#%`Q=%e`ER2Q*>;} z3X{`y_P_Hfh=Ts6{;;6Vx0j`%TNlbQM@s37V=~xrUhR@x?Gj(@l3slet9$!l2yJyU zqL=1MWMTI)SjXl&ytG6Zu)a#DL1pW7CAE;+uVwaP-vPuzeFQzuL67X998^Ae3plA{ zZK)WdSGVOg6F~Wt!yzu{0IEHM)VN5-76*)b3EOfC?H@gUp%k@j(o4<|a;Kr4nV-hg z73JCssWfRMGNFW)9nzs`uZ_N>1B=nE?~|2bS)tqHFE4whctiIq7JbCZ;L z9X`>4a*P|%fu?np&7;6y%j3-4(L2mR zxwz~CC|Pi>ozcmYn|I^K4waM7+njR`q$sb`+XjiYtkvz*Pj$86kQDoPKZb{zNE@olQRTf?hho@9!c6QQ|v2uKpFElh?G8i4}8RyS3IPV zBM=rMhBhVYdpg!kn(mop@s#NwE#&bYJd)(nGQQ zHW;;qTj!xBl)43^yc}N)sp7~yl}>261&};2#uq;`&nV`k8`5@>Ay2mxk-Jp8U-yK| zIop@XzJGK4`jP&=&Dc&7a`HxAi}J{f%&2(Dn%lvh8;lq88$p7vdU?POS0;T%F3lAq3roOXZKRdJLGQx~=3uem`smOwb+v zS3VqnL;hDIH+q7E$FG^Ih_M$5SZcnr>}L(dni=@OvAZ_Uz(~s+73zCATmOolbB|4A z$Lc05sT+Tz=H*7hxxn%&`UhJb#u;cM*S2NF=8c^uLMN*~a{*P_NK9YC`%h2ac)wf@ zm)?G9dxda+4C^f+2`0Knhux57Jpz%0mYo!^8u}AimMQlb+|ynCR_;f<_p%P(>?gSQ zu~sww5q%Qr9PFh>pERwGZ9n9+)9L7!XJ4QxiRkscHKb+wZw>l zNXSqFF6{jBLj$sP8p*RUe~;jWickbjC-@SqyJ1@hsNE|ow~Rvp zzoI%5jL--^#-qCa+TOuxpp{3gsNddV9C&y zR&;+>I^v@>>NrH4;@nez4G4dTC78fXhu@H(8`c#Xc~c|D`ilpGRMsBZHs?H`R*r+FrRtFAQM@E_Xy~{Q*WEjCLl~L z7wJ}@@HwkYiM=~AKq^oI%YVPy0M+09dolg)hPE!2ll z>@Bnh{Z9!Hq_lNCbOvkO#se!Y^yw=V0--+~327L2oPBX04o9*Jh9mWE)L*mf@J*$$ zX3MoBxw5X^THNU^p#q5(CqqB$u_G^cs1$mHu`|I~+Lf^?%LgG&!dcl0T$?@AYINb(~WI8?KqG{6_hP20%o!?$6U?K+mp05Cl{=BPctY zb_nZx2O{m>^KTKHGPk7bI0K8jUZx>^ z!#D(i5apv@ZG_G_8Yw_^27n=kIFmej(7Z`J|C&G2Psu8=(kOoX);ypCC8I#1p~U6O zs&~;qGgRAl<5k2f_mg3O_0g%)YJ(Oussm?v0P=v{VCg@Xabx+}e>nq02NTB@i9dS5f3M0xt(wAM~dwKM)hIkI-v}=^u&WJ6IWi#RolHKXB zZ;kW0q6|T!Ywo;?84zv_X-DeyCh^>a$<}VbB<|@bT#%4S#}x+gxi+QWK!jV+cAvDt zO4bV^X&yk;f&TZJhE}5nZc+fv@SSZQ_u3yLGEWj~s|NShy6R#E&Yg#&hUwti&iQ5k&CA*DCX-XI<8_mkY zYgTW3V><6{mS{wrP{1l`PKjzaDr!WPZH@(o` zMUkj0uU?U72blR46$rIrBQ+q0orXz3GO~kUKiba6pYi`femd}{->Or-n1)ywU~D$b zk_^;*|5C>$G&+yEgB}2H_=YHSk0d%~X)0n@H-tP){Z7A6gq1h+>oKd@D(LamELd;V_}VdD}MVNNiRa{^IZ z>sI#nJqo{CS1pc=0C*b4_)50!kKgmx*Vz@aJ7BS})EPSWA;ZWenRkfMwhx#4jV~%? zfsm2t_m1%XE%{W_&OqU3`aRofQ191s4?saD1Xd#t+iM3*WMea1k}qD9&1EmjS`IMm z0k1{>1^r@PI54cC8~E?ei?`6<^x|s_IGX*d;s9S#ZICy?R9O`9z+lbFpKpSuYnS0R z-4t^H{x0tt!w>myuG^NaK>%MuxSVSj_Gw-mcZG}&{&qsF65RtU+>53{kfN_Ek8is+ z2Wh;c4+1I6xk0EWbtrcPUccUU2ugd{+Vin&M ze#3VvJ{Ip&8MyQrIRpXkm>py0t~=XC<9-xBqm`~5Zq0T5;60aD=6UgMdOe!+EZ zxCEv?h|tak{3u&fO)YwZ2km=1)jnV`F0bW%W{pEhWq|e?l6R)$I^b3((+a5lO4=@Q zrP!`G2hU7II$(#wtIn6Dj9y~@A{yfDq}`h- zmt9*5T2s}r`5PXrw4VCziLY5xk@AR5|Mb>lz-jQQ$c<%a$x;+RgZAc5dEA+ z?ptZmlX6wDf1gX|)=Db(9NQ(KeISi3)c&o}v4x3lk9p7K2f_XrkB?3!fZrf`jyvOp z@#TZ<6&*GKoB$xHo&*NjkS%#b3l^0^Bx4{O49FP0uzPKXtsC?(>|LeG3Cg6*KZ02mI+#jCdpg1pxdf*D{pxDW+0XXXgQ-%)`Eux4 zj~f43g-|*5f}Zt*TH)hhW6V;6--3e(slM7slRfkDN@7fE6L6AFoky|E`9FLFJ@X$p zFUWSLKh5+wI;%3nK}BDM3aJaD4{;iiVM5c=n$mC@eoHmHF4ER*VPj$zzTM$PVKNHE z&56Oy0pku|kF()138OI?eZp8=7U@V9(MT4FNER_rB#Trei)18=c;w6M&iKp@L`z>S zBvTirr$G6((y*=fm?{(t|H+o3G}Pu+fEK9@SzWT5gKh=8qd^_SM6^~Ge z*9ZD<+!$#R`k9E|dbU579|@dsKas9&Z8}7X4C8)K5Wc?HUmvdRd`!ZAkmX$jYXY1m4(WaMVGg zpp`fwQ(dP>w$Y(fbF+z{ayHkrGy)xy@rM%byU$th=cFOV%zKgcoo_kZ8phE(En}G9 zJY0f>@6u2foAU*s`;Vpx+>6(mQQj@Y?>lKx)icRTnfJ%}NoRYW|8_f+2B5=e1}#s( zD?N}BmXqehmDkG3v_oKIp1Bd$-I7JY|9gt zZ)VPTRJ2>)T0}=^zfYYbV+ykZ*H~&CpcpX&@ zyY~WwHxU>d|3HS@RFt*J_1rB-y^y9)P&_K8K3@^{BC|ZWcWY z18T??8Tx7#75#*xXGtS!DLfWJH9o=%k*k(g68_DGYFr@5EUqQ+TZ-FWeU?@e_#MSh z7O>*V5A77yLc>~I+1Iyd2tjHg@iH#VGF&Tei;m*9X9z8}dx}{y_cjHWT<+vtY~tfQc@IY8>QoW50mzo&KMu5-63e82px6C9G}2VgQSB zlvS9AyirO*ikMY*WW`g5e%g8a5$P32EQ+i=h!@r|BJP1?>Am}5W{`IyQh`D<87WXi zIdh-fh-bx@+ifkbobPjV-dcIgLfqB?x{4b{c2yBObv<;Be?6j4mpUMO;E-=S)k>GO z6~?W8fU$?>5AHmF(LVKKX3pd^#vi;}H>zk1bLQ9RqG~#)F&hiE${L2uKQOpbrixNc zspP?3d9!1J+aoD!ejsu2{X-ZL2?U-X7kZ$~6a;R2knlr^@Puhl&KR4pWRmGKHVzZy zEKR8}(n|vRKh4}z?I87TT+{p0p5L4B!f(_gy_>!NlPUynyjwMu{FCLIJWMyH2q4h7cErKL1^C(et$ICmYNnf2otmrS zxUhT*T7|J}`8$&arI+ud){C+7?RYW5A_|NpdpJmSit80g+qEOq1 zNCX)8&p4iwJXbnD8BI>@^Q3b%!xz)lzc>nFHY(^YJ!203P*Aw^`zN>0b3`qOnUN$|8t-<@fB%DEreytYjp=~mjSxF40= zo(gbmHYbIE_?RGl<)Ft=B2b0+ApgxA&EQFklHwBkib3CP&*ri0@vP+M5D@!j`ky*W zYaHF-fkMu+y1vqvl!(aNpEgqDJN*57EG*MY|;^q)PED^yhgoQ$MOrQV5U-W=A{vGIlJl)p+{BK#(0yb|Vi#I(L zrMccbZwz0Tlt$pb`VHn&ux~OYf6IL3cGI*>_J*8csVz!UTt{e=10h|LjpR8|*Uys# z4ujD8@WXWQgThQyk2t7?a>T}iym?{NfN_Fs6GFQH+9AA=XOb@M*S}Mi5KJ4T_S9V1 zm%-UHeU{<`j6<7zap8~+HLY6iJOm^*6!sVR&)hXyVlN7B959Uw zU%q@%gZT0V`*Zt$my`@LK5FRyFjAPv`3O=r$@uW0HU0x%$@rL{>lyxA!2zxLpMB=! zd;xSh(`BGzM$k|{4be=icjXhCENeQ*HwT=eKh^J%f&spi@Ol)j3l@u zBshftp_Jkf+#QOOP#{w*cqtx=l%ko#Nt^t4Y-eU? zb#}wlO^mevwnqwfl=j>u8dM=i2_LKOD@rnj}wIizLXQK zHmDeseQ8Uyx`rry9Ieh4zoo~hyBm4<(z%#&%4V_U738G4#e!71EWWfewjA!h5sp_i zBMw#N5{!*2ml>7zr8{xz7(u`_9`)ss16WtjHuZ_2Ra*M z=|{yzIPN1W6VnEwx> zri76~FwTBT$TVEm3>T#SD(xz)~;ROaLE%!5e1ZZvywWC5sE3pFu`GZHm1SvCH+ zJKcV5;a;HDSW~f9t>oYq@7C_Bhc{x+hnbjYB=CKq<=~D~`Wt#P-gh2I)8cl8YUM@N zRG97J}f+kOYld62Z4L{pV1BrWQp3luuqR}0RQ-iqdgsQu?Iofkxj zs|@P-=}O7{cykkz(^e)XUrAIUheZSjCZg9Z|>HFm*;czuio?&uKl)wr4Xg z!E~d0h&9E2G|-zI(>kphr259bvQ$kvartL8(uN74wj9v-n%((^RJGLJG%iycOg4?R zu3QV}W_9VEV`gHa2PIjxi+TvIrw{vC)858fVYQAA4vcL8BQFI`hE$8J-d}2A8cGS> zbwQlNu5V`NYB81@3e`nQ7_NQkYa9R_y!q4IdBhZUc%Y2!vmT2rO(Yyw2OPvYy(C;zCWm zY2O(*0?LkXB39w zyez)~%0B!p%2Hcso#>&))ShlN0AUWW@F)n+ePTd9stFeSZZVqG$^^f@KUfD*{sg$2 z2J(Ni)n@DfXpx{r+1)z4qUX!3qe$7t)QR3T@%N_uzTl$;5;bj!Xx8{O0|=qb4NMW~ zn)_%z_TSm3sooQ$LP^ShO53?N~8B`4_{5i&kKA#J*XJj1B8I^sE$^sKIZK95% z{cZd`$vIT-ZovppNdwfx&0Lf%1tA){sj3wvHiVOLfX5lAQ*%28prV{lj?UDmzo`SI zw?W02j0@KFvj57~#3T(ZLpJwp3``72Hl+Ah=SU4l!kP>z!DagU+h`ii;3o|vpUU)M zNJ{G0R$EOO4^XR&nN{lnY$IY;T?urxADL#UgQ&FF-I^Y@)y7&dc=G)UwapF^Wo0Kc zuC$>Y{Ed~>`RS!*31~fZsvOwkg9ZG_$^O1{a;VOo!?!)P{?;Av^@*wMbTz{TeBy60 zmbB(nj3d=*f^f#X?%yqMCKAH&1t_JCB=E4RtZtZmxI4wr#-30hgYWp}f=vK~x(LsZ zD>zPVg!iRE?X`g%{)YxX4*wdS)By0Q5`AYqgZCjta&iFOZ?E;{@I^MeoBj)3=Rqc! z{;d=aU+@#`Sua}~$H7CrhNvfCRH_KCIH+a~twC);9u3Kz_GfEdh0_N$g$G5p6lFmh z{T<2dYn`F&$#6ud<~gG5))1{b-Fz8iO(kW}Rbv=r--nX*OCj4k)F)o5i%l1Sl%M8g zO`|(%tttDx&O`Ye3QIZM;_gHsY(@XmRYEfFN^@Gnd2&ZOxmnfi`fcaIy{Z3E{|It0 z)4Orp#c$0ug$N;|5-S_Wr#r(9B1Mk2iUngsA2M0RMvBkHpLG6Y)4&O1#fuQ00fcp1 zIdKjQ7Xw3onM^QET^RjE4%*lVH8`E6aR_2ao8B+HG#SEAM=VQ|gg;4XnQ)H8-KJKw z^tGu!jTxhJ7m^gVLK`feBqZ5D$y&NB7{)3R9n&w zRjdn*4u{JzA&yneJ5U5*Yp7n9ylf^z3Zr50MVb$)n#FoaHq%hi&YSj{#v zL2SCO4dV0@mk$kD3bPdA znU#WEVwL0=rEwLii5~|KYduc@6kSJE^S{D>*7U@%fK_$Q>du3|g6P?xs+FkjDqu)3 zcxpf(>AIm4Wbjnn4dG+d0zoiQVP;F6tn^l7-(9QZ1oy4=&)f+@-VjwD9R9t~NYIO( z{gZ4@3QN6ExY_Lb1$SWa(|E~PMzUWlkYMp(D$X|OVtB$&m+!%~TEn#(OH#LARY9`c zZvsMiAQ7VbTIqewcKuWV0HS+~vc#Kkq4afvS#B4#8;!ZC_Te@1w7HM-p72{(x2?qN z`4&Md>$AHPDMsr;=~ycaPnn)34!XHI3pG;wC+x4^CYY2x)JAd8gI=R9^+jdgpuNg2 z+GUTR|HW?8a@5`&$*%4Fs1-m80FN4yP|3wIk^|K(Gzit`&D6GU5U5}$I!I< zYC}sJ_Yz``c_Y|5nRB&D@m0(-i3j{~G$N;gN;H9wjXP<%4Tv(}&L~yi+fG_zAm3e*N(UB6C>kRd{-3ndD z4Ns$l!Bnd^nl_VgP_>{rd2qZ#|7qHny^R8d)2%ILj->2kKw4Ah?4dp29tSBJN>beN zHzEbnxU)76^z5*WHEq0FuF}LlsGCcm5h|HVa^T5KlsUIs96tgtZW&m)xx}c9@Cv$m zRZ~Uisog!$>K$n;;~ae`Z%8?_G8Tb0#Tfy@>4SY$2TC0b06uV@J{95mQGx&{`U?i0 zJuTK%j3yaCxPH-ZH4%r#U@n^`RrFCcfutOywWAwz(FyQzKBqdu4s5^I%2cf=C%LLU z<#tA7=TNGFaPVEa3Q(N}YL%kw@pQuzz1?ofLyYdk4>q7 zq_~c?i}$m1fgGvl0?lwX`RfTC%TxHzb!8-HTG#=7EgKGADyR||1OG@t6PM@cp$!F6 z)^Uv+t!Y{E100Qr>Y9Q zHz$>`!miIPkg5aMlq=b@jUYAdtu|Uv$ruP%C~E(OisG`99W5G&0sciub$3A8KwCI# z7gYbt(kvSrx#tK{Sr62~9P|%u%U3@E)CrvB<&#mJ1!(FmjTI?ZgZ};pXy>ffE#?NF zdpNd#w)F(O&g6I)wH3VMpEB|~aI>3#P(bZe;E!|cOU0hMc^v!~QM z8XFon(p*7NY`u(uI{aS6JPSd0F4iqlCHik?RK{!&5a=o}nfyM=^?CNNeGOE%gM|4@ zzQFw4Uei=)xLG!0YR32cGxu9~oGZPnjWI1FWgl^(rvu2OKoGer?Kw}7D$CaB1*v79 z5DCf(46;4J@?@U7-jW`E()tU4zGblH#ub9s27)v zMH)S8$mqPAThzk<5Y zq2)A@_&0i90Y0&JU^u0WEsvY~8NnSdD*%9?HLX7y=189|BeF9eAcxy9&)qiy$|=Fq zkrux+G^Uu_pAYwhX>KD9W=d?Uz8CPF%yg;Tm9{MX#;i9in2Qi3&rJM22~gg)ZVbUz z)OVvWbYW_xYcJHk=1G^PT6{TFYv0{zhnc`on15`}9KjMd{FUHv0C!=x;NVYIw zNmX!Yq-O;`rAz*%%vkj%1>n!PWCyimCmim(Q>{1M{iZSY7VTs$5pn;od=5)J*XBA4EtPK#~RB^VY!udK+s^zD5RRyzVp*d;p>3DP8!tTrH7QYN4|rhly$>cc}*tN_$U3E!60!<;L&d%b>@O z7ZHJP&Nhg=;X~4{P(+InZj}dpihdlS2IWq~U}(xztvj#d(ba!d8H(n;64t6K>1o|i z1Ba}TYSy&rNT4I-w=W-)<=EUKqxFg^jJhXA5~QhLY+)^G_eIjL(JRc&gr-Bp!}D% z0o2@H7r5DDt7kTU8gtEpF`!OocYWU?_P+(_pqa zB|Xx_kaM8knm&xEWIXFD`>0EnWaZgUIkbM+4>GM4^fe^eoT6c&0=b{|HT64t4)7Li_pT88*L@ zM#(|DH?xedj*;hC|H@;+QOnIqHBFsJHJd;_Mt0lThu^k?eDzUnd8**CiIa_@_{W-9 zv3A#|2b$&G&HQ^PZR(+k3}Q5{)!eR$1pal@QC`z^qGqG8NTrOjbE5oNm_IRa%3FkL z7H%`A@j?1%y0}OuL@`lzI`$jLq@tE`;QHY=gD7hqb$%+gBAmbZ>po)v%eK7a=CuE? zIUSf69!kmq78X=G%7QPDI|eeOOp8%1P}e;`lmp#*YBmOPmW;BnqR}(eVYZCN)jO?X zcT>@DwTJoF$IU6Nk0zoZRBuCJqjH0f_Usu5(gA2ovOA^z10PveO+{~~ z+F@59Oz*=Ho=!YCV+$A*)~Vd^4!yyCz;c~N$>^HD{Cn+bG#_iYsoXwGu;o1i^YpT* zcD9uBlZ6Of!Xuu}#vtgemjD>I(ZY)~c^D0}gSQlSiN4L8z74qU z>_$yT8sOKh`@Z%Kgu(sj{^SOC*N+b5sUzvx4Rpo%jl8TWKPm__g+7REj1x7^gA_4g zg6i7M7S6QM3!8JSEcf(y)c{ca7r|9lS9mz1TAxETxqaRw1F{bKkF1~K1=U|y+X?#$ zj;HJ}3R;)QD8$jypY5#3-yVZM#_@-s@7^qhfqsPTWrKRC8GImpUZJz4N7urIm=?WQ z6&}w)6+HDqk+-7tUUd2iG`-Z-u*sO-z^%@HGX!YnYSgW)tCtOZ`d;m(as~ZmcP}UIbVrlNLQS+JPyb8{>OVvi zMnUrs)fnu(7MojVV@UG^f(J{mq0nbdzS_~J3vDw|7mv#ry->D&!QTbLVtg6llAwgTC1craYcFnrA_`M|s6j_EL<$ zxbPj_5a_SZwV@Fx%?t!;1;;>PaP}SawveuQWEpa)7uUg1!7DBnpwV^_y{p+C4#Y-({`J8wS z<}V6YYp$9SR%+`^A6_@f;bRDltg8y=w%!9}FT*~~qpEs0&VG2p&-Fh; zZ-<~adCO(xK1ewwB9?v`qYnaJaQwT77;i&`6RRtz=5vE{Jp8d$tsl@gTf+{`B~$df4*S>8 zXHZIXxnKCMg5ynjWcWVHL1|BUtF*$)_HRi@eFJUTq6cUp5yB%$lr%x3rUf{|E@D`&WD5k zHK49~6C(>y*^5{BG0<2VS3_w-BXV>Oyx~4~`|(wOy1fO9R7C-T)|sEMGs-G|JF8%B z5C$%U%FEr?*dPY#OPg!yBRS~A$9saug5^y#MOjZKp{z!}6lag!cSiNmhdZrzK*Dp- zmE7N6iV&60nWFKen^Vd=qb>GWYkC?b$`rodE_B1o+fkgRVg#ekm#laOf|383qEv|$ zNc`aGt`wysjs0CAFok1wj>E#t)Ob>@#^B-1xd@vtEoH!|?2{i*qXTeoc@1%80`{NM zFpe0LpbNHP672G7h%+QpHJ;{glP&4MMjWlmpK1sdhs@4wW5s2{&bgX9} z>AXg*;;?D%(av?jezI$2ZEwu9q%>#dp$dHLZLJ*L?f75^v?`xgQj)~7LkXBzjrkVSf%bhPDHrt(9@EV>gqMLGf@}Msa<~b zDxbCV??6qS32p8N_LG%rdgg0|EuMRZU{)=u)jn^yV|RpOS>DJ2mXztLttV8|cLpN) zk3&J@Y=l3pnW6FG{Ic{zmOeup>48WdY?PfX)mIsAa0jZQ^HL?Jy-%NUe$00RAy+CU zqw;*Z|71%#TQxG0PVe=0pun?;!VG~v>_H`)Ot+-xKP3dw-hG(*r|i{vb816Mtp0U} zld=(UV~KO=OinF=QfFxbD1SD*kx^@$?mK)RYFZWS<@nz_{|oyVnl%p{217{f)wRwM zP*W?H#|u}wJ@NwVSD-6L&4;@{ccapq67DCsQK=3GN*Ds| z*25~D_wqs?sx~{)gO4y&E8^a#Z(lioG8@^x zBKcgwGBShR9JzA172w*yGV;8me<@(w)QF;=W8vJ)C|M;tFU14y4SHa?6FUDVJn5h= zjLHsTWre|h-siji5a{lNcgPlRcaPKcDQIBj&X37(Icf~+-1BG1%*LCt%wrxJWOfgS z%#3#Jv;+FbpnVoTAorW|9s`yfuru$g#fiS(9IPofTiondfQl4R z*3=%KID6Z_y{0oaVFx;7HFSMtP{aNP2FGu6AU$gkEVURvp_0Ys|BJQu5?zu7(=0g^ zBAT^-ZIFl^0Z~Pv#lKxX*_{o@7NQ2h68hvfFy~oa0A)lP+T%AFLDg2m%TsDs?%Za- zCE2PfFT?^EBgHV$Hgwq0Ut~0?Q`0Z^t_If(-uu zvHa}%l(Zd_?Qx%dZ0O;695rP0-(A0YgbhKOqfN*bZQV%F&q~zzQR|NyS0%GjLiT{W zM`8V9*UBAyHO#n~Jc=n+7F4vx!GSh*u3=9li?F)MHJ5l1!-m!^1XOBGohudAQ@h&q zhtzdGH}6e1o9H~v)@9S}U(_KKFj-|oSJN@{W98rN3qREWwW^M;RG#GA@5tc2F|N^A zP;R;|NO;Bbbnu$qnelQbg|}8qxeGP!LXKxUv9z(j3o8qzxLCbyCqi9 z8U2F85d7#b;%mZTtzKz|uT66gt7`!HG8#;!p88i>z8CVQesJ)mzI$=5A_uN}f5nbV9R9|3lDAY*aIvO)SJ+T)OtOAmb{%}kBPiRf(f*9@Wk@!oa{RJ%3WfIRG(htvks#l9Yt zuv3%3VS_9NUpfM~259cG9JU_TG`6EoNfy6iBZ$#$bYhTd7p#8^s|PY{WVnHD{-z`= ziun^;O$=dNhI7BhXbt0`ohV6iZCpi!2A!~h!4SrtPW6sOH6MB6ES!XJJ;8vG@;64m zsVjo5==e~KHyPH}-_9;M4>pZaXE|!M!I9@VUG-WH)?kZAF`E;thAy#MYW%lq{FGWa z2X`B1lgB=d#!~zptkW^NH+Rq8_YWfBW_Wvu5u6R`Kc*U_DY~9yOOvZ+fp;Bh@H z?Pc|DdNeS)Rc;GPYz+o%W)s)$2p!&!Pt|OVipE?9=#D zr-KMWtbFX!MlW8W$Nmu=hHS@2{fsza@S)WPjRwgbn?? zFb}FPNwa0D07X`tA5GM&Mc&tkO^35{A-x(c;A;u~#%vBhx2Q0R5)Y}X3a+CysKt#r zf>h2$Z77v!u%5vsva5q={MTGjHtq#h^_LyOk<-jNhCHmMf4*eURR_fyX%q)^akW$ zAjRl~nf21M9Se4RS{_D+ER%UykyBG7BRKUz2<4Jw9!09@BuXk#yU#7&Xiw9Q;)FCS zDV^w}|KIvM`lFyZZiIUjr8bX=q)vwo+b8b^&MP~Q9->HD*|CqV6-cEAG?tX}NM}fH zyP5*6zxmZb(4bk|5b(XD4c)qD*zcKiE8o!@Ez0pf&4u_s9Eko zkaqaZ0B`{W*BsP@@w&Br-*5RrG=sn3)^fLh^B{-ksz3^!Jq70Kd~BIMCI_!*cZE`O zbslEsrvy=DBWr8wpR4oYM6b`;?dpuCau7ok*=TX+j6!fjTN3U^GPqX1!x~>&cMAh%jfi2V-4|Rq}eh_feAOb^rKi1uzmej#0g9-Y%wD8>)IKmUi zBCl5rePTqyS*v1=9Ti^I3XNP${{8S0cu;HW%Hx^5=K=}V_NQBYz^?cQWHKtRU`dB1 zsnPSgXs+{L8A#W|O%Hs3{82Nwc-Cw7*@u+Pv2D%|+}O3i=caGP=hU_C7!juUTT#;% z$lPK3)!lByKA4GSl#GZWN2z|F1(KqTDVE0{JBlqxjd<6V*&Nc>k@VEXikAPW z_L-}&uw%Joo;0?V){1X_^9;2mKhlv^L%PCv8O}p3=&AAD>%i0pB<}L!Mw6DLCcHhu? z@guSOJrkDofnC+5qKMF{yynnR6)Y8naJl2Z$H08bIyP?Xd7J)9&T`jJOSjZy?WPZ4YRi@$=7 ze>4a%GH3R!4XDgcm&zmVWvLaAPG7TjRv5NGxd!|1sSl*u;IM4zy)xyIWs*VNa&5FvZ`r zk(DVaKgq^|I^9C{$IyNncMi3;?B3uwYSs;BsJ)9AR1jq>vk9TT2{;|8xQ64OX@%O1IOO+?%H8S_2(w(;Mp@keOR18jdOYQtJt&ylLF{Uson zVtzq{W>uh3x8NFs?=fk(`CaMwEv)IWvW$1CyEdqv2G&Y7l4O&l15YbxC%;r3!r?96 zwa)kf@O>bB9SOe1a{SsH5GY;(slFsxv7He~LDSR1 zf(oud{k*Fyaz8;O{V`Sxt zUtY@!RQ=m&JURT>8SA2ULk*>fy{#nOegCb1&w6I#MoZsos|a1-fmUC$l3vC#YYKW{ z6GH(t(An|v{MexY*xye8lQwtuL>Z4+lMJkTBb&V+Dsm4o_MAc%Y-JNi_fkG072 z)Xz5|S${M-xxqa}RywoDxM5fkV7zay2zs&-P2v^&Sze4zMJ_r6g*K$F?mjEe2U+?IEdDMOWp|#bgE-;Nt*W_>hQVKB z+R;)n___7e|Bbo|jeBYvD6Sk~g5>$X`RxXLdklPA7UXOZgHNXCcQK%0>QCr)VPPi( z;2aESnoFd;w=vKtDpNZLQsjK}42<;m-!7l40sP;nV_ymW^&SDg%tqHjRP^*)ROZxS z1DUzHKuUV2_7G)>6H&JIcTx8DM{{v#mvMVv5WM?L$7MuJ6U*jk{l12haSdXn z_Vs?<dahvHbdg8yl}e z-7BDOdA)qk2LYcmAl!xGXCZ-{!Bvt7iHg@Qj(1Zmxm0i<}&}7#>a+% zH!8X8NKF;k4}4-LigEHEw+KGN%sHoTxHzS4%OIcDyS;udZI0G^nDxeWfQ(YAN{%BDN}QsEQ~$rzb4ifaeFps%jbL7JqmJxU{dG}&8~ zk1sYj!L+qQMH%X;B!xOJxpr#Inxl9zS;y*%gx`afl)F?q;ojY&Q=ymTI_`R8sipc z8QF9d{3;uMC7WYjW0{*Z&B`+@hfIAq;Oon1199Le_a%$4^~*F5onrMl4>o7ZEt4jl z^86h>atZy9+(%!{6!5t=UM`ecsKcRSnDD6bzX(;XYj3;;O^ zmUOHiZi8Y;4pbJ38jZllvW+H~;=}YF;`7LP=w#;h5}!4?AU9ePhFwpF(!+0$voBWpaEs!zB<3hG?v)?`q92^`shJ`NY~l<4I_`JQk%ri8V-r`widkTfm0@*wH%NCwZB zCmm^hAeQ@i3dQo}XkP)dMcKH4NN>NfT<0Uo3L~JjZuxSVHQkQTd-9BjfKOfcXi-RET1+irJOxNsk* zfafFjUk72v&sMM}y$#u2t*P6`MZTnAj9v;EXFf9^87s%Rj9TwIC<|`64%(7i zOVB?YA(X!Q0RulS?h{7n*uO$aU+Cv2VSn`II|F?=@8sP0{O8%A?G8)G#$5ivN-3&t?=(NkaPTx|Yo79FcSsyxxiJM=+QWP?RJt7V4KDC4Ml0Tz zeU2p~A71gt-xZoKMRT};=r509UimZlx*Q54m(@4WF?R;DP|11Us1Obkyn|@oY8>oh z+|rx5<`hAd{xF)n@DfoSu#1BHXxLV-M3&pb4|N}!GIny0U~CuwSnhu^YtndsWQi7g zmY?TN3q#xXXiv|*5iBotf_u&iGmxp*CG)^qu-ycIhcms=mu>r+0ccoyfjiCm-d<1@ z`hISKp6kUoMga5ym%Z%1qy}8|d)fFkyn&6ojbton<^<$g zeJ(6-t?Q7WV0NJ|14yP~9~_HbG}ux|%B$Dzz_TKjz1PmC@*MWhT!F-^H)MRden%}s z+)f_B%6#bL6ib|JUxsd?$ zgt2r_hdek&=GEHYeqyOIHi&9r5}Cc@1)0J%hOCRdMF);tLO(pQSLMwI=gEv}G*yqq zfj>qycDL(=yP(z>wv>gd{fnU5s1}x}cI6u?^TW<=%0$?FB$}h#(~sRL1lGIb;K*BC z^(ZQn9t%@A?Gdy}k47lJVDJxWBi$m^=mtj$r!3U`{FISx@f(S0*%YiTkYO^5rjoh^AHXyH$Zp*ZUi7=qh*M*BJ_$hWSc|7hs4 zB<>*?urpn9czLUIU|9@NbYYF!;IkWdqU%TvF))r*B{)`2uJB!puF5H_B1Fzd%P4-U z3a`)+*Ma$O!H;d}$&tQZILE3=XM!=lU=`|dxNnLQI?|w0vbleZG*&2_lg-plU6}s6wez8+67l?l!zng41q?vz&%to zk(b0$M#^S-WA)6%(9yg(Q+ntPDpUv|b0kB4jT1*a4dhm12VB+dWfjWRAV}T}L!WaW zvl{U8qrOT9C;DL*PEa#`X_u1@^#ZwP5KtZnzR}~Ru#IZHprHu|jRUnI);qMgO<4}e zCl4lk0CKO94l-taS z)-*mA^M-zNadu;DA8@j}#FpMMa_bbex%qA=U?~t~q<-xfQj%#BEqB+t*|MYu&(+)g z+mq=mY5hzk?-Z^+~X&qTkUiGERdFXSGzp7s8=#d5)4Yz{n|- zicc8eU##|8G8M4iXk)Tu2M1N)RI3^yFQfncs2^?Tpoura>QuT^lO@BA^gYPGb|iK* zsvDRn`fl#U78=f@(aT}yr*TFkoV(et3*byFyZg3dN3N=4fmPlL^O{saC|$8Gio@Tw z{BD082sH#kHI{^`IlThBVlDDA?$^+TaWmho|BreW;^6nml~YY$<};*VvW>zz1@nk^ zJFdhX3_~Y|&RcG`=T|V$MEcal#ftXUgRk;AK_{eqn7#d7@Xjj2_G#)y+#R)aW~4RE z=z<{4sFJD(9Yf)9BUGsPJf;?qe?MvbY{e2va0mX}NLRX53o2zSyuaIUI}BCcj}wrn zfbrj|@yE_oz}#4*ja0HMi?y16i&^M72V2X->dcD*UNPH6MICl3ttoK}S^`7rs?2{} z=wV{g9+j56&5M@}NC%Wk>N_8s9SnYeN!%&cy|6VMIrjeXRKPE5p6E@5N8n5hzJ{`} ztqSlF=+fl4F#aC_pSUm*H+!bx;M>D?{5KgMjlvG?CRj)*a`|7vZKZ?vL$?5J{K$I?|jBT@W?((ZXYkkaoukU9(Qs zX33&8QMGtUK#Kr_8jz^sOzn^j!I{2}|1|ku2)Gx@mgnA1K?a23jWCDj?dAFwaPAQ7 zDDqjJHbOcLOQGPo1C4aU=oBq)gnaQ9*rhJdc(I|IjgXGb&|h~vda4+*HH2(3tKs1l z5q@o{ce7!6D^07H^rxmVNW%OcsZv{FRbJ#=LuEyo-#G+`rBen)dm_p6qF&K&V63-4NVwqB^J33jLQZ zg|~$!I*LrT&)F{4U*Za{s9#n@-aQZsffF~7jPM9=03gH3hiWyhkwA@$5Mo$j3@uv* zk_~W`l1l@Xk`nBFF?T^0jEWVG>N6zhDyY4ISn{x88qy9pLaHN@G&lmQN4=dVURVOy z3GAaJ_GfS?b@>WS2-!C^*ksBAo3Q$!IH8SBB0_?VLB};`m$s^^KV*2-{11=O=orTZ zRUYOX$#JB!2NS*M%7my;u1gDsK+7{R>t5drdb^|xrqWD(kS+WEzQ$U+J9Mjw&cpoE z5MCHXSJHLvw#<3^-%jhlh&IpBc$lXQhoV^8*px&=3Rdj!M#ETw0qdXTrGQ!^4W?Lf z39E42QI}MKK3&H-lCB9(yw7?9_hGLIh(go7@RxTe4BV^LaDyFN)hG7}`j|I}hBEEl z6QJT1RuQypFXqvV_SdUBO?v=yJb@R>!P9I$hxMYY#;&e3W-anj6}xe!onfWVz1o`% zyV+{EJVI|s94nH#?#9Jz47TU)-0xn4L&1OCr{O}V?5ZxAGPYQG&?68RHG01r?Q+{< zM;gD?sya2@Q~o`EKbu}Gg@y86ed*Z|l`HQA1k#o@(9xP@cok@ps}JQ2bPb{#`w_O7 zC8qt+G$;ZhuvKeW+_)8A5RkU|JPOB_*JwPs;5+tDKi&rBc?pM-vvhai%_7w$RRoPa zsls*&%PeIzO%8JW=LWp)JFt->!kA6)zGG47P7i1UIjsKI^Vg08ZWi<|ckbIZJ5v8g zs(4cG!1-xLmvURf`%1SMzB}sBMA=Y^6WZzB>})rvcDqC6J??JXID(2w+vw9&z2L4R zi>mnyK>Qep5snSq3w|U{ChCvS_OOZnI0#ku=?u~LHe2S zfjFt19Y5$uxz54yRCWk6QI^F+C#qs0W6xo7siASFM01=BznQG{5@mr$Fc{f-Qj}%w z#ol4RY8bt;9F+FqzQ3@Ile? Ps`vq46~^QYxvgQE|}o9Sa)mhd4!0rrgzh{DYD9 zqY~F>FS>pW!x%>T=bf3s9nqN&#RxzSjlLI+@XmCrIyQe8eBJdenxsG!cE7${-SamD zys{WKk}tDLvL^2I|iC^&GpC005h?#e8!{8 z9YMeJx@$P)ck#8O8P%XyhD5hZF=FfVfB|^Zk{zgJ(Kd|_Cm6dW&TBY?Z3Tkzkl@NA zBYd1lxX8GTQN=H17%AM=d6TYD5|+jBaDgaX|1~@>>tFQk-Sj>*^^`W8e23P=&QmQk zEtY1HJ+)x(YN$R7st=buVf=do$xdDLwp8mkq`eWM95LS9D zsyz+}>@EShZY*@ z|0lR8AeKt1f(D1Dc*Ybmn;dEO9fOjpT&L;@SdIGy(-^rM?1eo!)s~uG)bU%r$b#xW zCN%aTqLpq)<)yM`iMR@mnf=qAXh)GJ!=&l{jr*#hQ0BrmmFI4;9h~S|4}B8f)o?m^ zE@LHpdK_Ynyp-_IKzg2z87s@3Aw?rcce?crdOEfhLLa_?Pn%$A&@)1lw3;&J{01j%=!I5p!i#Z^k%^dP$ixWY(k0YzIoJo1O~tfUK2EZ7wtz3s zrg5SVjWAt^{p2SSKSb@b?xXf2zOM{=`5q22aP@3t02AhNfC3dVTN2wo!y|590PfD3pg|7KI!hc|#UdTn|qT z7HeW~w4=WtE)>N(QG)XN!&u6-SW^kIUu|nu1)xPo0OYVvutye&8{Ig9B~n(S+uwRW z%t7OM4o_)kv?iN@EZHXDi&NCC1g}#MQMt)jX=RE>VvTP>&nkFfqL$hrh>>#fKmCXW zX$y==(j-!&s98IWd%3=XX|LX3-wAVe-S7fu1^Q08i-|54Y*JIOAr2QruOBvw{Ce*-vRKQA+|g< z%s4LKktGwx$1y^PMNgmtr<2AqOFHvJ7b41L)rOX1&WJMD-j^C(E>FN3bwcfDT4E@= zs12i%6S&Tm*UR@3>UH9*vCM{w=f)^$na0|N5)PN!IXx3lug?i6=sx0$>+*D=`g5-= zm_erfkuKEnf`H1mj?vMT{1{KMh|QP3etPK>$2dTj zEznoX;dR;-0nZZ6Ox3uo83#@o2ZeXgXgwMlLve1bxK){nb(u@BvBp9^C!pJ%5% zs98gpdXGJ&_0t5>jE30wW}odstZTEoN31}qkyQ5&800e0tP*t?bsCJmh?9wL1Tte3 zqAX{VyC3arfdeS4EV@Ojdu$)(4zyoaqBY-Bz!$$lK644n6v`Ql@WcroZvukOJ_4W= z=WXbHHnM`0>Cg^4Y?9SIwN+P8O@xM0C8}2g1d_r7Z>rq*1+(8{gAwu>(#X51Gk2g( zYp{+WcO)B?GLoBko`H*!F$68;M1%k++ZTgpiE!zz$mm(m%Iso2hkF0{ZWoLhCMqqpo$EDx1=t)+t9VIuzJPvg*xu|+|UPgd;-nMZJMfA z!0gmS6Bfka7hEjqcMz~YW5OoSGTf=zH^`PNmjrM0PVu*)zKjt&=ka=Wr+sX&(hA7( zV7{(iFlu^I6CuvUqzyyWI;XJD{B0^ndUy@**!4RP!!TMszju3j5jyUxsBG@QdKHlR zMkHI&vtd}uWJq=mklKeg_nsQnpCF*>#t zMfqnm5j@*x{n^8=Yyp@(yCt`)ufGzYWoI)@js1G-piDQ^r(t_E~vnkKc-Xn5}ttN^C}LNZULzLB?1b@@TYyx4GBFTcmMfxEh@hU&8oH}SGz40 z;AbB*br!FzF@jx>w!eQKz}fKOdJ@vWZAMY-DJT*zp3HuMCDsJIo+Sj%o`Be}YkLK# z#S?Jh3ZuGn(a{TsVS{P`mB%x~_Ev!R%8vD+pos<->ZGYQoMo%51){vN@#A5EgybJj z>f1j+ykWyRB^5cv2cX|-;aC~z+3!Xq1wF*0K|<;iU&Dp^o-#_otsC^^{j=7ZsIOUe z?+@s>TT2T0>xMvr-I{RT|E}wQ@f!$y2nv@2>At%hl(&7OeTJ>dxb+xfPQv1$9K0sn zF9%2G-XCo%;Dy1ak5lJA906%&Lz<3~G^hSGN)thOFSM~d747o2Q+vZ`e0QXx{qacv ze0el3rzq5cZgoem#9e9s3(P2cS>oO0&+Z{q+Fp$dpsjGAnw|JoF^E6!%t>cQSF6I& zR*f?bfYxho#n~Hcd6>}K53rn##Ah1+Ln)Q#?OU2TLe5#}`or!84 zz>jS57!?rq*i@q#e?eXbOHF%Ybt#^MJn3dVn{#?P(KdBu|BBdjd(nl_yQy=6DxvPII|8JNcK?}}@%%lyU_ zE0lc=_et@>fZ0I$E1e^SP>0#L$%TDCoY(BxNEmDpTqn-Rmdzq5^GhYI>8f-23o1_i z;e>sf84A<;s{_P9;@x#{S9MnK+t-oHh6jI<@hq6v!mC2&h20?8aPt5v&etiZ@kcD9 zvz&-^tG*kw2)0Q>829AXo{0YD)qp&QpljNbG{bMVc)aaL4$^cQ#`2U5~D(iG+|k zpCe#Ah(~}q?5)dp(AZKVqoaP(ro+Rl#0wNhALcYIhE6{&$zHW5FgFB6XEu z_Z;Htu1_AJsOCBE-xR4+f`-vg*$bL0x_(xS<*jG=aYzRqUvGB!Yyk-PEtPYei1 z5c1&Za{N`0kLxyds{u22hf3vM@zFE%3Xd?q#_JzGH2Wdkk|p~5GWvcSHm2!`z%P$V zzkBrs0XM1s8R$VyGZAJO76r~kTXKHOA!5j8CL#>`-qfnWv0%u17(bQd6|f;R)BzV& zG1%HQmzlE+t@kjFUNX+8_Xf_hC#z%mBNT`AcRT(D&&_qO9OWCCxUkngs`Jbl5gtc9 zf(-}RyRf%QqUgvePfu#s70rXs&)x^Z3ctBAl4Fedb^=KyCP*IvYFscr4qEGf-a3c$|pX zc{RB5nowJ`a%d)$@~7bX0c7V`lWrCURZQlN87*IbME$0~F7hbsNfQHBnMIH@ed~hd z8GfTj!ZXOwys4oO30b~Ewj23ykn357?}2l)!JL6_^rO6y(DMq6HRPqL#%)~6`!iQk z1nUiS;-{lOi*DyaNkg@fw7Dmq8p)qtZp@Co!MtUAfyna}mi5+z*|D1Jj{Y#h+MHbS z)$Zo@odg7bdHwP;>}GKLjB!E38~F7X(s=J8AGfewalx(L3}RQih@9=@vJe~R`@VJI z@2(W|!J=G$)&iC>!A^H)lzqry7T)c-Ib9ts%Hkouxlcc1S%i2alBY5?E`!Xo2Dp?z z`{HbTEJR$Aw|Nc}*KQ~t0i72fLeJ*l}rxvm_u@+m2V=0DmPhBwUi^TlZ982o>93v$z_C{7Gll(6bE>!d* z&TxKtcz(Pam_Ggf3sY+vvpU?0zdUcp>hY<=&w7m+6$NYE&0mwDme3m33_QH7__~;i z@oK-R^j%MQbuaX&a%7sl(7^wf#2_dB0w_lqyX>z;+fgN!b0Tk(v{_nFVD^MPf5#?p z=ITN0vYAxGsrpir9*3tn&v_QcawECuZJ)$09^q16rt_cV#K^=1P?r+6>i^oke51+^4UP)pfj zC>-sn6L0+lH#qiT2>S5Gh)S{Y!3K-fR}x|;Y}CX(g%AyCC@E< zY4%$LO@?qjCDz^w4sy<=@;cW0+XA6rutq^UV)ULI{)c0ehg!p_c4Muqo}~Ep4_)S- zHN_54S1)fmyg(bl;m_=7Qs4miB)EtSf8l8b_>J#05&YOh#@L!MpJ3SIsDupP@}&z) zs%p-Ufwih$^TlgQamE>*!~fvBA%QB6j;9z-e!uUvx&>P`tPSQWNzmv|SUqBiC>Q48 zWl44-+4sP+u-Ei_9)>bAOnF<)G#LAUi8b(A@in&O`5oNVLpzM2l zSN4=jEE#i`IMt(p}HKc9S3Nw*5rp5_+L27uH8 zbqzuJSrGiJ22S{J%B9H>xa@2>#v=U0cIN-_^%Y=MC*S)kBHbVzm+n#w?!72>E4FJO zx^`jjiXvjKiK{ZUs2Hmt*ee!xu7a+vE!NtK-THga%!dnu-~XTI@f&x}IdkUBnfT1i zdyWc)>+Ea*(ZQmo(kh8vt`U;T%HBp~j>!$h;RB(s_^a#IYZ4RIz^E#8w!F^ObFvYr zt_kDRB4mHd)?3E=!gi-IOy#Azr5mU9V6``D-FPDmuQiy*JeHsl1xWhBQ`ST*j`rYn zj^96dcVEc)F}=+^Z*I;y%sffauVKgIjrw9{UXJ~aJT>&e_lFJ!wJ=2^QUx5CKRxUl zrnN;Qe)Xc{5?t?h36Wct}%9g!Yt)QExADB4n2)2!ty~}&&pQF%Ilso0)3WP%7l^&#oD@Bv| z?^W3u1C!Rou55OPGq0z80C;AU24F?z1xF8FB^Z zwOTMo!g)AyRm4Inn1)-FoP7Yqc~~g!Kj*mF;At3c!>|s?u2}t|!u32v>&{#CrLb|5 zGu-wG+*UpSa=oU|#y3>CvcVoD4mH#aW?OkzY4RCwq(ULcDwmd)@Ka;b*kLT`EHzVZzWK6rP<0YT^Lgg;Z+}}>=wuMZ)5&n!KWiorr8C^dP|{PU z9?dpJ0cZ?mGf7%n0yJ&i5KjIf$d&8|%LM5P!Wys>D zAF!tyjBQdY$qo~$a|RVlsI1Y7#4APn(MNA<@!zYkltsvoKr(D|CXB8C{?+K7#*&PB zbxO5#Wwq`BfK}jtwVppR4L=BJd2L{+^SIehIH(N~@56gIVr+Cs;I?pL4FAuFtbOX$ z%ImD~54>g{)Dva&xY>C=rb8-Q+QGPa9W;3yHe>Q6GozO+<+-qBAs5m1;%tmDV$igB zdaXGO`zyLmJ`xS;$Ga|)rOa1X=Cq&h{PX=hY}yI5zEbmU3{aFT&O{Ic%6K4J5OEkS z(;KW5o@k7e71ZK-`?tAngCI3D4U`1!8Hz!%6a#ysM(e`gawnS_Y&gp1IwMuUa77M) z2VO${pd*m~-$*E$t@2|zNASf@@=sqoQbf_Q2`emj{)thXwiX)#V2zOR0`QxPp1<+m zJ=iT5ytj&|jra$=YnswLjd?Wyu@W8mY&b!6|%U6>L8wd<(dAv>xEF49?{BuU^bJu_#os zEk9oBO^0_K(?+x8lQ@^Bs!Y0P9A#K&Y~hIwBn#Wk<06s%F&(+)BXX^6)3p%@2f3i6 zEyGTuE$*v-Xp5QF8ZVUS@iSHM;6-e4wLj~P>5p?udUti>u#1gEXhDcyk{XGNfW z^7+Wb9EBFPK+aVfM`ZSKS${O{1BRP$OnLq~wFN#vxQCQy0WSP0KKIP@Ctuv?6n^Iq zHO$+FzN6$nOX?~)P{!Lfb){&aHt^zD$u#$Oa9TcDA^ca#JeByknTFoZ1Y_>;lNgHr zQasXrDfZQ*x&BW%O}{~jLokgzPc7cV#~*^D?2dL|FMKeiau0KDj4Acg0n|JmlUPFI z7(edz1y_t3<*@H(aZHnU$d*~(vqlK)b6PK6OEZ(ttL~t=lyh5N&@Ue2wDv52w3!3P z)NuQ3>t6kG5!y>JVC0q5`Xp!YWP1W}2t+diyA=p8ZF|m^U`T%A`7qaXW&s>yVS%lH z2a`4S4uH+_+MKAwn7R<~5X~`!ad@d|c6vy>!7lb4!Z; zK+3qb=(g$%ww7X9%lEe zVa;aypuHNRZknH)!tl(*$&{Uq#7z^OWmc_w=$JJei9+Se>-kx4XmmItt_;Yi zc{uv6tDhZ#i~?CgHD?!rEum(7y1FvV0tOAiwCf|OdObj?IXW&z@u9p-c(pcC`5mjF zH*fC7i-V8R>>dOw$`=y)Rqa@v%i3l@vUr7zEPsaN6giG`(pI z(-Uf$)}yvUJM>SarTLhp_@~dS4d-ajOe?rN1J$d8#w#OYr~Io^gf-Le2Q7O z1FCOCJv4c;jf!n*7h}O>OifqKzG|^hnzqa`UT9?lh3j-{tuJ@Y^+mU*Q~+%(Xys+4 zL1Q}t{e+~CvQg5{kwFfe$m+8Q<7udS`SX!YP3;KyqhcXzZOZh#_35c29-)Cd(ED-) zn>xZ(XjQq`DaNiMWHa~9n8rVL!Oyo1Yz5Y)mPJi-X7VSxgTOfPbx*6>!(x{p&|h++ z4&6(+Dzp5n7|g_A(*Cw%D2OM*XXFn5sfWUl(*aqrYWsT`vVvpi7{aEc=@oj=`D+&# zrKMEOj9#3pl{hiwIF_f#!c*}`U;euTQ(f0aa2Y>84Q@gwbc<1Y14%}!!RUl)npjpd z7jS$oL)8rp&rjqsoLO4+nx^dO8AX|4b{l%%gd%&f?8xi5W|$qT;1=m;;L~8p$l?9j zyf(nDlDsQfLB^2*&^@vI#E|qWuOon&t?dPX`2X#hUpP>qQR2gJ&+Ca1VLq^o= z@k_tnO-A+}&Jy+jQVc%dfiuQ;2hU7QeX?cGtQLGC$sF}U^72Qg=y-NCHq`QhK79kH z1rlK&LJX@^$UMXQj*uEF&IYV8#eA^_`vv*vYGW1}PMLNh8U>lM?2^f0HfQHakC`RSC|Q9NHI@W|ymT0f3rnZJ8t#uN1C z7gQjB6dXIlg}bq?yAjFuh8mqf>YthS?jORP{s#uxKvw5Cg9Gcn!r7@be@$||b9?D} z^c#8u8$#V*W@@`ac+8U^FQ(n-Y{}Es3e?S64QAY5FK}rA>HY9JgeqopwfOn!Ri(+W zQ6(0#*BM#Xx?#J{DN_2t-%ee#Bg9c4-<@!kN32#q8GxBC67Rgcb@kI{Y_~ghtmtB- zPbqTh_T<6KJ~_Z{*lc4i(EVkmIzF;vJ-rVcE^h+vJp@33wQ1yA3cJ3N}j!|1qmi(tTAR^ zy94mE@m~R$u<3jwtro_>XE+?{(*0$o#)av4Tv%JR+E1VrvSaYiRT?@@t2Ds8`KQoo4Cs*ZBF`JpWXsALHEg=i)#u_p428f zUBM>gEVN3bm(R;A%I80+rq!uu#5QRV$C+&U*eScF8#McfzU-=NDD|U9JwcBwJiaU~ z1z8=&#k)|NJv`1P&OEcI+cTlrJ`6PZ!|Ygn&RGi4DH!bH!BcT_#!COCy*Np(14en} z_^WX_#=Lt==y;(3q;P@%FdzO4`S$#0{MK{VL zwQtMPigxv?g2(5H7(!GLGdqE`s1PfS+}^ z6t3;N->2HTS|Y$XR)1OBk)28^4q!!35cz@XG_XjobrQCBLO;ms`q#dkobX)!92~__ zEx)ulUGprN9O%JwH;+U`gr5` zaatF>ptrnfSw2kRd8Mt)3f3#&CSi@*Es-9fP5Na(Xn`TKXOmkZMV}%XDJV>Gq&Wp(!7R%6xi)6Q+BSQ+C$ve51_0*?eaw# zPtcRoR-@v;nZC2G8W^HGegw|UKUju-Wg`UBE(OkXZiLFrj>rU|qO#&Qc%S|Jm?+Ed zfG6p{nN_u9-NvG2bp76bwq{!mbbc^;tg++(_fK=Cl59Zc>_K}(kvrEE=VElf&m!BS z8GoEBQ|p0_uC7cqINIC;<43YRacoirx%YE2H$`p`>ye_^mlky`JyqBD^hYGQT}&FlpEcU$_<%QCyl*m$ljK4 zn5^4EHWCY&u+J@#J(H~C-nmme^~Im?7x_A})!h<#_UvJ51qF{q`gis@({vN`{)T_@ zK4QrO&I5}Yvcyg}qjR{4bPELIIs0vF_h;z$85r4eJ!>8cy@2EKE(0c7b`x#-@I=v> z@($(S^0pQ4GkD&i??hnoLJ%PDkQKW+#TBu5PGQ6mK~}X&M<$}ST0d-Srxoj@0?U01 zR^pFxvM<0&1Kx=;o;|4eLbMl$^2}MVzZd0DZjDJ?p2t>x(9t77O$lD;v31tT${0l+*NA zLkY$yV6sCB!4qiXr>!p2nTicZ=9KeJ#Yl$3OY|W`VDuSM>ytJNJgG3&#-&eQ(C0 zuh;sO5$N7p9moaNwr}_704mRd3M1slx zj#Zu-*!0g`R-tgrRR8h{=Y+{j_AKr_vayZ(j3GswPeVqzUWFHS!}eXaWaAjYIdqAe z6FV>*h^~NM3brGB=-&PTXpWz8`Pq&&6ih{J-HcemEhLNB+aX%ynQ688_DCBSASUXT zpDse12`1jh$l7DdVy-H%s9KBnpX~$JYGzn|x4gcO!t@M1@vXlbJ_0*M3u`AVUcQiW zzBPf<%hpaEAeeY}zX&mMX4zM9{LbAi@ey3{%^)3tE!%@%M49Ru=8+YN|D#3=_GcvE z921seX(BOuYksZY7Ti1rz!E3;uwx+slr*LH<#3fk}h zi++E&M9E)fYRAz!dd$lID~Fa7bL<@LE3|ief|d;)SB9nA6HSmPn@MHP$ihs6?8p<1 z=&N4yy!UoO3#MRq+qR@y{C8&Rp^1XDV3M{zZ*b28Ic+HBhXv-QZW8IzX@WGi2Xbu` zPgVd{diM90%9u-VQ{gBTq|FjRDftYB`jHnS?3Y>h?hv7P{=P5zMCy}&V zw@?-V8m|5Jyak;MVe|@Uf!xpvg^J*|0Ffnj*R$rA)0}D0*M51IF|!=-F3;5en$m>2 z$j{Yk6!KvaW=)`h4s^hDwfX@O@RUvN%%-E-O_Y6p2d}vFNR&mq*VGa}f4vW1z5Gn~ zvoA~Tgw>1yM2S0B)~*FNpfIA%C5u#fp{s&tg^J|#nugPhTrhio0*`F=$1ls!BkhtM z%RLC~bqSjmy>*J&2QFF+Pv+Kj5BL^8Yxifq`hR4*p<(I0^3as@jY%_8b0e&aa&xck& z{|wnZFn!CuV(uYGGkQf?vR_SO969}!?q5}X(SS?vE&1fCzqg=Q{f>}fSC? z)bUVT-qPA3oB&ypH@D@c%BCW9AgTAs=;E@2Xzoj>D1RCj*CQ5()YKdJ5J55*PI4Fh*SelW~+UO39r_!h)PL7aT< z(XB6uLkfrnz9uZTqOk#UeO!{4Xy!-^%QgK4LH@|-ATG=Ha!TxX$e}%^EUVq1GP11n zLrs|b7p!!=`&Qt8>E#VALvZ7Z#*Kf2Mx6K7cf$AX3LHDg{x)lrppS5Kw`boTngd(6 z3jIwVt`(NBN)$bOE;NyQ%>a3is73j;kxHvzS4B&f+Xa&eX)%1{%t@&jIXkfU$$KCR zrZA1f)*|lLD5M5Lc1)rARUoqTUM&^`1$IcM;QXxwWcYb3%G@#(HOFb1@p`^LivEwl z7w<9SwwI=oIco%If}OD+`)(cMBdW-l2r&!Ri!!U%m|>8U$BccSsBsbmUFTuX`fh`c zz>;Om)B20Dq6AF1&o+y)VrR?9(th_Jzx!-7X7IK67Qniw`^!wdmCNbV|Io&8l}Du& zjij&zy|CMp$K#y6LUMt=Hk_N@H1A{kXjm~0R+NRFIV^-a`vr-JTx9eI9XmYjVT%yH z{^bKAj6EhKXLwnvn1y4E6BoZfWX*uKm=*RxZ&`fHt0m$inNd$=;6G^j&$DhgiFoH) zp4{P{_ zOV$d`js*Pzn&h$CQ%q`C`{Q><9N=<{lxl$5SNwkPDSlt&rTlLAOf7%`mrnT9`~yR7 zF=l7kFa!KJt&hmXRGd`Jj3cr5d;NQ!zvTV_R}KTGyqxxqa?qwcZ z(FSzEHsuGmCeBg7NpdrAJAKU?rvW-lNxcvGHrFr^C`Gk=>iE5b;7U(6v0&)~5a^Vk zBVGMNH!c8u2hhutQ?1U9timcS?hxlhdkEA>_#8YOS9eotM_M6c7O@WDfhqV5mv-jx z4O;`~+#lPuaGhh)*d5WGb9u9t!|>{D@j^W5;h2vM0-{{%4gYjR6n2jJxQU-UsqpF) z-w$>e3psE)d1Y@jOkqgdiiBPpoe?TFZZ(D@apl&(e~4Z=YtSnB4E5j`g=<9s!pf1u zoPB7t;3@i}5B6AcS9*?h;lx0Acpe_k0T_IDSA+siPvGGHqa z#h$bF?KojY4(c0<`s97Xt(9e2X9odIMhbSyIp%;Yh?29S8980h{4(vor+>(HNO8CXTuVJlILFL^h`2^hNo{1Z)9%yCF&bHK= z6Ma$JhhsOv#oytdyjMN_uOl;Gq6uOipOq&g+b^7cHVo#BfipuJ-Ct(vKbIW|#^c9k zg(|^;^?mNVeu8$=(3fdfURP+>{fygEdc$3Re8`ONu=+yCZmyF`!|mQJM}g{D9u`we zydDo-ig~Tu9YagZxA=>dTELt<8-_#)giuEpCjwEaS@;oPh4M2>_Ut;2Uekg1d z1&tS(R$)bcT7hDE@cHF@dWcTXznV%K@793!Vpx6F$G@cZE!APBD|ovBlurxhQUe=) zuGOlQ=)RxoaQZsz`^sof@q3{f^xx_`5eBCb`!pJ}B++*n(rQOK>N}4fl@G(lG~xO+ zWm%dMyw1{G=)I(c(x{ZxxaLZH8)n3wn08L=U`s$pR6DGkL>yOQ@E`KXX(c+J-h0XnDIL(PXjLF0 z`aO7qq+19NFGedI)6&BiM{``%gvHD>`c{B`n7ncB%pu{CyksW>F zW-J2T8?;I#$-O?@i6w?V$o?xqnikCsnW;4!tUXC*+tggx^mAZN?ub5T`61JV2l%? zQQTOoU5evIFCS!->4{G~e3Ht0j`*=o%+m@X&q?I#-S_v?e7zL5$8@Co!^4~97Ds~aOwTJvAoj~~e998qWc6?sn`AWVa1q5fRQ+@L87HGaTdSBkyS1!=WH)q`= zkqYHjl-6AL+LCkM)a0%H4OLP!xw<;9pjrLzt7W*;s%awxrlwYPd`RyAvV{!A)7^a6 zSX;25CIJFM!KE5Kir;7gOHryM*(q&`bPX|N&#G%ZIek=vPruWS<}3cmJFH*t%QfNU zC9n;gDP`gJighqEq+(`}Kk+90SB44cWt>>iK`e3PENpff&Q$`R5rH$MWpLv$rnw`H za#%9&AK2j{v+@QBq^b|-|Ao-=BPneIOGqD%7mXw=l zBRY&!@JR8AZ2-1gs7l^uoqDaSrZRic8XZ==8OOs^m%-w8&9=chG=PGoe(L{LFfGw| z+A;G$Jox184AY{YM!gedrMYfCObMtgP&FLt61xRw5&b_1DrA8%V+DCCM`12b(CnDj zrNXx|<;7+Z2yzT_@~ki3Ve=C@qaAe5F7LPcS7HZ);UYYLP3~3%+U4?0JWwGbgp0B_ zNK^B(hbD;2`gvQWJW8QKQC=HJO-!~dBP-(tQpn93g~3k(t&ha!m}8oZvMwJ6$=0qc zQ<2JmJzJ#q;Koe>`*Zw9`7U0iK!wVaz9<(0YuUC`vkxg2g1)p-Rt0DjDRH@x zpPv;9H%uL+cKa)>n1w%|7w6kLWF1h8TsR@lWwDR_et3x`KYl9}!`!+b)b2qR;&8)dq^V zG4_XPv1@%n@82xOpEv9CxX>APAWg)FiM&s`&_rl`AlC`Uf$_K%@Th|skqo%*3BxS9?iVU^V0(g)&Hj0q+L1#DW_VBOe32EjKo<1qKB88uX$JZ%cRdm_ zy~@I?JnFUy2I=|ZVK(}rkxkvZUm2$S?ln~G(H^CJ1xHS|8wsXtFv;K54F~9$6bB`H zbQ%9;{!!Svu|xSoIfp@I=mCmBrlDr2gb*p?=ebwzh>>XyMtK!zGMY1`S9J#VFjKA1 z4uolZInVa>HLU-GVqd{0e@0eFD&g_vs47l=>i6rjC?>$aAeTG0*|-vNE9N%E(^9N< zq}IJLSM59<5v&Z;th^)oeZ0hD$bzb15g;D6Vhy?p=a_s(C}@R%hBwwA-8V0r{TJ?m z>Mkzftwb4pOxgE8fQz7nExeb$&ac(?Ul{uZoJv;IJyQrRii-*6q+6on&s4(9eFPJ6 zfHW4`t{0@Lk!FVMOADXxjhe1@5j7$3kH>2+W5Ypk2_og?+ROM z#kw~B;yI&-aNABeCXug8`@Gf3H)1dL;ySG7Ms+YpGd(+(Y|369z)OR??}8|QBX%vd zt|Qx2;1(qt)0yR!d>vqp%CLF4bzaYfKsjz~?jww%858j-Mylt&9U4bzeh6NDzD1y_=Hz#3B=G!V*Zb3LR^t3@eRxXqVk<1=NIg$tV-U{jf$%|<{#LXi|NagD1D zAwh5y*-3U6;NqP2xA4Tw-$8dA706#1vA$4xva=s^x(Ns?$=wz{Lq)F! zYNA8rZmiVzG-6A8;&_MS zPGA)sv6Nqk6*E6`9{j1;qxOG@;zjhkCyi-3(j1XS(X2pe6?nXn(>kzId+8qj1Mx zrL8gniU{Y{R5-G4#CW=FABwGvY@&&IaOl4^H9a-hQQ1Cuc*d{}Qac_lKjX#kcVX|p zI@h);o70iwDnbuo;Vb;k3hUzYqy(qqU7A8?#mh^rg0MyYyK?#QF<6haZe7BJygRO? z;GP$En|o)W!Qb#t_JN;}8;Y0!?r~wa_Nzr6j*I;pIJ%-Is$;g3S5_oaDxo)J`@SN< z(-3!W6MnbCfdmUTJkjREH3wsol53i7B{+v>S8-z-HmPk1O|k0r*a4ql$nI#7JTq*x z7vz9T7_*Ec(IEmzQE&w2F@rZi>C%)i0Y?=8QA@Lq$H!u;_%FIw9^kv(gghj$F=Wj; zdRJ!g=dtl6n+I)v?@W9xSebkN$^}j= z0C$y~Z~Uk!6Vt$l+8CL4*;F{=QhG4Y6CsMl^Sv$-xx3Im*~rL5qYs==SyV9{?btD+LZG!4z)iYNzcu5LAxI_s{A!( z+EUQRwnw%O>Jy=#+ac8N+`goK6V|D-oxm6ICigJ*?M2ZY=<{CQ2OsMqmuI5Q4W-J? z{?6$+Ov>kU6@=6}g707AF<4q@DNWLpeMa7{8O<)`$vvj_Q^jGuj~dnRdMIYC2&}-e zSzn|Pxi9;SkThKK)$Y@u)`vFqpjp0uAAz@4i=WUq@F${|6owAGUSf(56cnRO*AK1FT{Qf9)|U|*K9ISP@j8pDhv zdC309HKv1jKmN%xx?NtG+8&O?YZR`D^>uXdH>C^$omltl8V?TV>1EV!)F_(VFm&X1 zD3uTEWVy1=O%dRi%Zg}@>Hht3opM~3x!kHIpA4o=hIuPuF_zbm>!);#2*lqJF**oS z#7@73sZJE|GT@aWc}Fe1-oT(4`m`A8$zLZuF9<2OM&pQO{5!3O$Tzg=9i%v2qB3V@ zWn@u_U4N$y=GdNkcie)_u|TZFm%Z#CfBaj9UKIzF6Jdc?C7-{23N5&Zp(GzIRl1|2 zH(^CC0FLe7ZivpIH`c^?dcXkvE!bX7@GHN)ib6IEMKf!*@-*feqK|6d@%A9J^7Id5 zqV7jQUohU)f&KljouSByL=Q4jd;=YufQj;F%5$yF7h>WlKUY3)a#I?2Pccwo;L**Y z{5iH4*)WKn!kFT$DTF=XEg8FX+K$(#YCi zPn;estvj$CNsX)e@nk{*u+ZvXsB+we2#|y<5)X{${u{000{f44-$(b;i!f3;NCMN` zi5^d>ijeLsyS>_#ZH~mFA<}|*^uW+N?Ij4h_rPNt{&Ul0nD(kS{T#^VCTqQvpU-1m zn(HU1k-`VrnqDI*VL`FdsQK#k?bp$E1bVcWbQl#tOdbrmaRhQA=UYziP>|A@we0aL zf`t4L7t|tG(}U_y)82z@A3C(RBsCJryQpRV69f{*W5RIT{5$OVeN#wCg9Oc)FJxd6m|uuvX{QmGxqg#EP20t$9T+M#(C)?i?`^77> zb6X$!0yCGbfBBQ9PF)cRTNBJw4sIqabR81P(^F8CKI;6WfC6FqrB{?RU%)l|=N3HevpW z8c$AdaI*f%ZrDqkR4f0s_UdM$=MxU1-q+Q0FEioH8;VwD=L^4jaS;y=Zw&&tYj>3f$AP8n|}`B+SI_sfi@v_P;t$wqZG!@vxP5MPVQG z)t!f;%&Hd!z3Yq17%JO8c+*riXR6j-FXf4-9X9smiWc9oN}?x>!!X_cELGF`d8wk6 z>LvE-{R?xq4hdvqV=hZjJSTWekCa z30N7aEPn@VIib+Vlzkh5VM7GJTTM$g$FxU}Pvn*E8dgSy$(XrX;6+dVT?}5njf*qJ znQE(VEOL~_o6W`l;V-q9p6>K}6`1e3dwJ!i{=}?jVv_-khn>jgz{TVt1&m@k`1|x-!-nnY*U=mC_ zAzR;)O{s?Yn!4$j=H2RcuquThl25mf;i5~Sw_-7s+zna6uh=6{yNxe?2`$72ypeDD zkNDBGiOHR%J%tAmX*(aQ&EZ%(replby|%GAk$N%JTbx0!Y~X9ah7PJ|&gSAxlz6EB zd<+IqY?l{UGpBZZ-oNt*#m~yYNRYewvj}kw#*ZmcbJG~8`EG0(0%)PnJZ2(K&D<9y?rfd} zs(f7Y=yZh`q(De2iN!F)7L6G@qYS~`EP%N(UY1BG{tl~p%qk^V7=^5!)I`?Q4!l8W zrA2IxU5qF1#xr+aZ+OE+63~2kp8anr(R;D*8NQb6>&s|?8QW*l=8rGvoX4<(yhj_p zg3CwZ8dvtb7~d+~OR@%hT!Pv7EtW+2yXV1bp$<|k;h}viV{@L#iR-%%;SKffZV=^G zDY$-|ddSz6)lSiv@oYGsTCCaXh!7yi47YESj>?$re^p|vRw=MWwLQ8r2bPh7b0^a1 zaQ7Md{%Gt@d|Jp2h}%o_Uaa<2BS$uBGa?QbB;fOnm?B@?QpMwzrZRe7ZboswsO0Nl zV^g*(9=Yd<-tCVCsX=J+N$>Jbm0O^z#DHZy#wKRvV{AN$iDkWp57fP-6Nsz^%}RRL z;tk49+~Z{n<17tXt2fF(RP~J?c@%pe=3M^#zWWo2+>FIP#r}tKIdLWbyVY>ZSXhFP zq79`rV9tBa$h&gW6JHTNK{4_8Y5LgmT`Z)G3jB*yjEanNz*SI*1!-o#j}os1r# zam@x1Up3r5CE*QNu05{csL8%ZFU})Sb_aZuG?G;4{S%v)hDj~sW0VbB-a{=$V8Z=R zPkrEhb>V$2Bzk13<@6juGodlU(kHJ#kK}Gv3!jPEaJkt`<~a#LUk(||q?n&30q0Np zY~1V6F&g?j_~9h=uz4A#zcYqG}K$`!>`@%Gr|aR~HDm}C1(P4#jS^hME@RdqS9tV+>JOD}KOw*Uru z4Tl{fagOy6oIoqpWR>lZsOft(U|7vBXOD4-BxRh_;aj(V)8KLGXcG>Ibbpzt7sCZ7z={UU zD%>7-yWK5K*kBrfgfwcxIvCf7HQInvK3F&qB{(fV8T<1LQhfEh_B8&^4b;>Si<$hj zl30`I5p;K8sD*el%s+NFen%f}iVx)Vtr8-n{8Q%eP4$67?$7dP1t{TaC^_Go=*kU&n=0{pUtSQ zZ^3F$Q}$UEr`!qHS|4xGe=q+vIHbLhS~M}VvIw2QWo4)^#HKD0WhG6pV=0NSArUNF z^09RgoVy4%gk0Sp-r_GWqh@GDbpWS-WRx3t2pb?v3{LqA@ycqUaJ-%x$(Z$VEJDu> z`c7?`n~!dOCg}KNvx?Fk zQTkN*b8*TMU2PWZxB)66T$Ec}8`C4$bSwZ*Q>mgpCj@=*bQ4o{X{9zwV56=tz=F2= zl&;JekGK7-SkOXzdXYe8F$@OHJzIt#Z6Q+JW}UZ0MA%^%-<)bNQT5fUKl+wByTFK{a6nb~yXfal+5bx;kLlPrRcmGh~Y#aB@v$$EWzaY=dnt z<7-%U#ff(XeQX*=pV1nvGk4whH{+Jgf^9ou8kF0S{#elOHbo>BI^bTecOZTnNNZqJ z$f74dCsNz=>S0Jb{!zKqv%DC6l-?)tfCEy`EqYZ#Zo#Tg!P7yKKES%M(M>3^3cFWuY5_hyA|x5F%v1z@2`;JI2e`UU#?l>X1TYoN~F_?`-;4^q*eDd?s_DL z?l|P0?;E7zSectk9E?c}CpZ2ly)S`%jzU*?&e|5Lq6|CDL|ZrhiM#L_Q7Xa<;dAph zByzdsp`bG6z8ZYc2Qx+;99!0v+FBJR;VFfZ{rnwx+z%lwVI!m+#~H@$aSkEn4AD5a zxyK3&!IRi0$j8`)aa>vz4p$O__7RxppY~jk2b1_i+Uk!XSJm@Wy-a}Cx2)s#wRYrou3++uzA#6gHR>eu*H(C34Q;<>ypN=}sX-3cQ zN2CBt-E~0t@)WfXr?2+>)JV#N5d)rf68+pkL|>J4oS+UDHcR%$eD!LG_&IlhsVN)! zo4+-C=nt$fH|m$=Sk1zQ>j-?=&=t}4qU@{#rppc^b!7^3v?F!~oOu?K7U>;I76cKQ zR=6}=Vdc)V{%}oKaz;aM%@uqnosqYheolA6uhr+#uTJc=x1$-0*kWqL zRwAahV5_4=bwlP0g3S|voouW3;(5?NqL{yXO$q$ z&BUObV}h?w61}$CxRd7SF+%jq)7+>`L7z5NZNpLynwqeOd$DmNrb}0BZ!=W#7>sfy z_(_zPW<-tyhEtF--^4d;-XP<4l(FFNjT&L-53QRbFp4^**amnduYz!WJF z5RFb$ZxxD}uw$NzfPp^z!ISvC)^_}Ev{%KRIfniG8M^yv?6Ns+cErZY209hEw`3aw^og`Z{y$#+gW&X*!_17?^&H%>QXRF6zt^Y^dqv{nLhd#M9VunHmFtLv_LG&+ z`<(OZ+ZMJw1}qvOe>~kbW!Ae(s^d_yPW!dKoGDT>#^55FvAL-;!ZWBXI8Y`<<5GTq)DwHF1dOy$sTbBDn}M!6-ntAIprJU>fjAc*pW8JUH*jMx z$H+}g2$btIXS3sRiD-@Ovr)gt@k?3GJX|8aszp?h3?YXF^yGQxE){N~5f6UomzS|Z zAR&UP)(rb)P>&HK%04M1$rDmuJa6I(T*42pOGD}jZUuK?&N6avm`$7cfs;0kbi$l{ z+NAt($GPr;CoUJa_p#Z?I%a&d+K1uyf6wa1lq(nuxP@pDJ!n3{3vFMqSa1etY8T~o~aTxpgugq z>WUyJh56LidU2t9etmP9f^cVHXv@n(yQ_jHH_#ce!dfBt0KJpvny(oA(Sz`)=b^p) zxshE&OkON;K627d$yPuRxf7=3H^PU`4VUufe|kq$0)Uw%8@&TD!l8g4K>3l2z}|*IrdnJ>5gd9x&`DcJK&hFVN$_@f;%BW(8oA$c=q^ z7j{5jDhh&6yELKPT>Z9os?->sS{;|4^5DbMB`%A@3J@(AqS+K9$ z5fvec&TP~f(787thonQX3G&b0FalPg6oOTF0UbWWgA=6xqpik)T>knIn@3>#cp56m zU$_$iD^x^Tus6Fko}wit;i#r#kSL3bFhK67aNJW+D_`V)oD_+5WRy>NXCEFS=z;69 zXYqL&8{S3vqiy0@=J=gGgaCYaCKl%|doOVMx<3^vB0Wf(nit!zVX^bNap$ zMxNJ#mz6@dhBnt4vRb=v%S@+}^Oi6C^bI59Gny)^)(XHmR1HQqmN3oJfDKuQjW_Y^ zuGuf_Ps}w(FxP}hb4O0RP^1KKRTh!84=IfFw&O;S+5-YNW*nioPiRNK`L8eOL&Y|sv?UpQkz?9 zLqyq4D*I=mTxQEG)&?50uvW^JNJPVbrr_f*znM1@x4>1y~>*x-Ikh*t(Pzc>|nM?)9sG3Qi8MLjEI9 zJ|GV5{5s)p=+hD=kXM&AS%Ti`HnJ}jT&ZEpRvc5*o84u`0hRtwKb?bn$=4d^vUTd= zUD&BWGq$`9g@GHADPmESW+HL%kPleXwX7Y%}mE~vQP>4ub zog)V!>zMp`b|oP&r31{DaZ{8b{ZgQc z@J79N22`jC5tlG%jKh^=q&jYV~M#pVczi0{y=m97khirqPM;n%k&^!-pRDb!6%So{s`1$ zZo`7RC_4Auc>i`5oayop4)vcs7tT{r;o>VE`KXe6AWa})e+ungvM7?xNElRp!~Dp> zTi~h8cOfY!(NFA5 zGmLHSuPD;ATfa+{Fm|6|kM4^ml>B9;_Kopo^LoWp5&AaR{pJt!W^b$p@~}Bj$D5+G~v9e^xDv%{;;w&)vgVgCmF}Fu(v)v zyVpl;@|Qs65%3#8>J}y93vCG{n#^3+tE`i&{MG`*1>-x9^@#|a+dNu$c*K$ zcLtXlZyAV-g`R~9-^YN0pl-1zHHghXNIgo!bbrl06FP$IjDPawKzg$G{M&X;^CK#o zu($!hGG9HbF;VazHL;6nHl7t0U}7tQbcL|lwXmkpP0RPfZ(Y35uJ!25I+C7OQcIcA z_p76XJk!J2GYptqTEu|%^5G8#xV)|X<{haGSqtG9@}Mc0Dwu%w@?^_naA-*NC#Sky zqtv}wSf}JIed0`_#|xzXT79Wf8ym{{$j(4y}=B5+85qP1tSF-ZTFcLbXj zD%U#Z|F;H89Kd%)1F2CP(m4-P9ml=bx_X*0P9ITtHlj0>mLH@r&B2C7EGWgrjSV`j7;$GicXfA6fIY!2?;PRFi3|x$RAfX5rK$$uh^v$LF_EpmnL{kLfV-vZ>KjM!{$0x40+P9-z|7r z1{*QUT53np4}Uksdi(Yu^DM;GkjF)IltnX*$egv|;`V$>vUDFNtt;ug0uhE#J%i4n zrLdjb-D*GZ4&Sp2!n6iXp3Vg~g()Bp)lL)HcMWVlfutX%EcUc57 zd9fC~fbU8;j4dbkp2$nk`ftP`BC1@qTE&h|@y2`ElGI(!p9a<21#{Ph_2khWd0sGW zTnpG_+Eho7m!R_Z@#-jvn)U2Wz; zx@~H~_J2fcvQa%|Ox<5*>SOlu1~w^Dx-nDs{4P#zh;i(x#9C%xqNJ9-IyRd0`rHr| zJ*}^H740``fclT#Zawf!gg@!Q z)n#vQ_kf(hhgnnw&XQ7h=@Yn$HEn>|LiLJ7At!JHOw+R(MEi)cIjzwR7vAW~4499< ztD|^(L$5#XtUs>zGkUkZb$MspP=OUb#w`m{#+Wh3MtH2spOdsE3{@v1QFd}^uoXL{ zi>&>EvQvebh%!nv#a6z?$5xXncrugI0ppA$v^Z4ro(aZSAM_+n&2)d6sVhwd=a5B) zrfkx=Xg`jT>fc+f$%BTu*pD^RQIpYatpq&~S?(-pwV@4vzA2uJ%q)N^bsYpjenE6? z@pH>&`1zQV?&pwn1GR&Ador^FEVav3De9n!5I=8hjDGypo&Q{tfXP!7b*0=(dvO0w zWeTwUM)zAMr&c8 z5BN4AF=Kkw>+OKy6b{46!>N@GZ;F#1Xufe~pA&A>I+wwSqqqP);{i}ws z>;X6|;YLXL2aS2&P*-NhUi=1sIQI|MF(UZ>abb^_7*8p12YLGH-;6V*Z$NUhW@ijN zcq9PMCcz4yRg(ijY1jT40j65|0V{^n0sPdBLBMVuXrJ&Y91l0Jp7v zm{X3%2~i%|uI8-spZE_RDV&&Q9J~{`0!DDKEWZI@?F3V7O@x<7>w@0qn5eyg%I`Z->w3q-btvsow4XM3)*3QHksyJt9!;Aw_4<(B!Uvsbf!h^%SGImS>8b#O=A5j6hvP3WU$*!KKFzh*_W&(w zo~AUEz$||clHfyE+d$1t*t=1aE4Nj9dl28~FG+ouptXi@b$QnNmlZvwhlc?n&b+Z) z!FNup|DBCa^M;bWB`?3mMAiQj$mPGVHeOIreS%+&;I0zcIk|eBIE+oJOOxdi?(P6ws(|eC&tWYBb;gvUCCK;tQD%g>iS%}U< zwS-s?PRxdMXVg?ps%64z;tFEn`Y}!%w=(;7En8M06{mt}YeXlV?W_mTMr4SMz(H4h~c6Ifowmjb^+OnaKk1K2O6xSgnf73vVsgGf; zD_HDhpGZF}8i*uUW~|{f?8u3p&FX>q^f7j7q-$|9z6);j0GgMVHBUr4Tu%x(QfgtXlVZpW$9D;SyXKf zqcp?!L?@}Cd$Hkq3_w|{3?l^W>7}IfvK%vWWE9-!8a{$#!_2`3iH6(FUV!V8D*rv$ zqV4XA*QlVA{UPcuse))7FO>A?Sr?vBs%p$|D0to zcL8P!+1yEbtj=qgG`AI!bin4eY*0VNG?{A$o(n`JgvM_yHFbi4U}}+J6wKnCO6G^q z+vpX95L7b}1Q_+mS~?he`uFqotXNIJHn^@arbE33 zpxSLew5`BYR13_jE$jHEBne&`RlCMQlvz}$?9HCvMqCd9Qn7B+FgG8tqB6w7y%&ve zofAUDj!dl6XNlm5y&$mqO=H1&q(|HHHL}6iCa+dm;Q-UW{J}5gW+5=j!H`WD4Fqn0 z3GTh>#{ zd$z=wcEQD{JP%awE~M}VdbII1Ww#nBYT2CL?YIyoU4{;+E~#~Q0I!1q%VW!8>Q^*m zZZ$AWxNcg?A9Q_?j=+>fUMMkJAz_;S8z#y)Kr3N$ly7On7AC-5jYpO#@4nsFbzwgC z-duKeH>|+k6RDiTc1+p3ZrDEaviPxR{Iscp7a$@__H9i?1C~wbUrt~a59XI?Is$vv zH6CBXRQ9~z&cE7V$1o75`LbK@SWNVREcqs2pMQ2JsjL+tclu`uS+Ig3OM9&VrWRJX zT8(11)W>zW?85!F6FpEg&3&!ejb0V)SpGzJC9fHIT=LpFP8(tC-f({TD3nSkM9YR zwWUCH;V-o09$L~wlBNAe$l{)}@-sfkL(i3^gk5xb(8&(2-;)o^_w#N_$ z2UH#VJ_&=+&AJR#i^<5ONM+eSsajA-UVkjvvMTy^W#|E^;!;gnB`54W$&H>zoVxCW zPIv}xnR7XEyFj0GN+)G8UvYx59j|C=fbeE5bzf|6$U-0Dup$a; z=OOFJMka7|6?#w}@?K%0wgT4}Gq!6HqNefGN6#l?$j^xsWe2_jYnJgJz@^ovz(wiv zF*ft@xn>Z!0b^g@B_F6sOunp^xz^K|Nb~olsQR*46~@m{EtOSa6!m}~ibOYa>y`}* zSiovkC97#`D|(I7H)BCIT32yavD6Fc_UbD|UWnXD(OM2RN?J`x`<6l>dA!_eC`f_9 zQGhK8VID>TG(@TlFUS3kl{-+_C~X@U-*>OT3da_$e$2*7x+ zBQbS|(^bb2I=4&bYj4RxI%(U@$x+2VqtAChcwT4mO>S?T*~nqka}lK;IZ6laU>}6@+6(j&E_T zjJ4?R_|kF)6bxbx0XicHvJ5d@h( zHsXwh5&^MKByjr#K@h$OXLtPPr-9f=tedL)8J|2sF_l$RwpH(s)79bjpJ6HaJHu{f z8G58CLZs46>}!eXIzj}$vBM`ZTwWe^*>lP>ZC-@0FMiTH!v&_&%OnCq4FT`vf}U*TLqKH65t?KxZ;+U9uovxT#Tk$KN^v;wL92R&W(xV^@!Z5+7q3$m;)%vg>vtFhN{Fr|QQ#V%I z3jWJq2H*Hh?Kp-k+~XXgSlE}6-&cIW?_*E+Q68KS*7g6|x(=u+kER`^BfWPny*B~n zUM>hKRunb%LQ#pm#vW1e)7TY7S;dCEVecI^u`8&tVMC)vV*zU{i2pPDzFb)RJ%^K% z`_9bH&d&CEc2?qcmaJX!ah*g+DKNGg2WSn&LPi!00V zJVtDgtJ;Ity6~`bPA$}WFaE^iy+e0+;hrovv@%W2z;6)bdC!~hN&(Cy1I|rV=Yl4` zUJ~^GL!@Ru;DREnP*y<|kaO#iT)Bc~!0yb=vNfr~QnaOXM)Uun+5}dWBh|)v9GJg8 z=SnD9)*=~CbgS;_$&?;75ItYq-ZX#3Y4Psr7+gh69E-q1P4*(L`sr48H+aL155bK^ zkJ5hw2WjTxSCz~tuH|;!%>TEsX1?RJN|v`7nfE4zArydG_gr9|#C7nwoA~q9_#TZ& zqrZTmn+{F~M*(Lt!LzctmU|%(efU=>eZjUKEAx`wc=ocMbEl$}W}uQ{1@pdpvlIF1 zPLh3`wqm?kUW>W;sKZ&@&0uG?WH)9|(j`tkYj-|GtocF2<=SkRznY_HOjrh@aoD`I z7&W->&YXl%bv%S^hDGS^z4G7M%oJUXKn{e`XWMLX zwfM7zqBdP@s|nl@g{Sg~KHqp#Hf=izC(yrAb^Ld2=3FGeA@e~wM5aW= z=(1mQv9cwm$9fms=c5q1wiOFG>7-%869G}!8C$JKd?~N8%l}}*TP_j-$6bwGZ{}Ks zyE2XS$RY429oUMhAbOxu6)(2gPe!1d{1BaA?4QqLBqrxQ$fQW5uER_-Jka!i;m=DbIx^lz!Z;S~!CX(YexE0=r(7*` z_=lh&&A+voTN*Mq7sPZ`Tm&5Q$Rb?h5PiG-`q$~jlzlLpxL8#;mgtwR!;L19I{G$E z+75v~L%ew7bF>LF4+U!0e}+m%D?LO`soH0nNgJYJl3fThTte%LW*FW{VruYKM~`ivh7sy!!%F9F0f12Lm0cwQ z(+vO#W;KejlbrkE7TQrIOW%qdboH4cUv>>vXpTCo*dp4EYyax%FViTT zBNf&W!^4*Kljw)G0_K?BF66$Ey6)dMJIxV%3Lg^P=#K$Dgp2iHhO>~FjFwH;SB$w$ zIJfoaQR4RELWU1hsPL3jeqCKBW^9AGl^DjX8*pkItl|o*h}~~*rVqVSuVAHhk&u^Y zgIuN^+X=JVLV0mDku}AKwaHTXuuED<&D%8{niC|Z{HGLuijiCk&lBgBty6v2l?Y!S z7B>g4mQ%346k}>0g$b(ebRQNRrHfiGSTk> z=3~J7rU+x*7a;15M7kJ#`arm%eV`Yo^TiAC2uUkE=-=8t+KF51=aZu=Dp2`BkSp$9 z&gDSv1m7SYx6Nq}gl4p-!{2fksy$S${I@pKE!T&wI;BqI#V@$ItnO!Qo~C1!A5cky z|E|r#jg)N>3>I@#(XiA)INlj&X*Kuhy?#f>RrP z$C|jrit};_dAk9rDn6sO&?l8{2KF7q+AhR9^UrR|5|=MTQ_wfN(2wTySh+S#<}KLb zcyXn=hs|}^QD;1GPYk-uVXQ7j;z#;5LlHynRcCM?kG@0<6AGrkq&TfZFiK;ggsG2Y zZSqPK7R=EEa7SVo>hRG!5zX_cVNDATepXq+I$vG(Z7DhlMSOFvUdMxPn!$u~;3rLm zVw}r47j9s4@N7xw4s%DgwTarufXbS>thJvbo5)nwTC2ZTC*zrr$N}QIEE%>F%@YLK zK7w2qoW3TL*prYxY^X05J@1j~Pp*ePGyc^GulHbB;|edkyYa~Yc*kP2pV+C9 zG=noCVBeb=Zh`_N@XOW<2PnsvH^y0U+E~u(Kvq~ znkZw_onPK}@kb#a(3LzjEHn!2!|5Iwgu{ajo2eZnsL{><=vr4P3k*=Xb1{ILKUYGG zHx^#*f*5C2H8H030Witljkp+UsETi`-+h3>guW8(v_4*9;&>2EN~Pj1{hQ2z(|kc8 zqCq}2lIVN9fFmRx#gQ?|Z{DVT_nBxnBea_sX8oWU(R;FcpL}sLb|Bh~PXHV2dVKU^ zZG&+$aHAy=)MdHBuoJA7Cy_0h)hnGl$|3kxBy8 zNluyd$7&g)`gGC8clDs!6uX-Da*fk8Os33iGf>w~VOVEVNo5AJI<1HN=lB!X@wW#` zOaK$&=Jp_hI6`Lt{#CjUw?%|+V&#-P>gFI$RSBH(lZJ#9p;&QE94ZOm-Ry2?w1@O9 zB@Ewsj+MR*JrZrf^1^UePs&7QeKV#N!d_HeoYE(alVz<1qU`-a89#J&`^1mep!h`a zh|2dEuc6mrMeC8^%_~W5&0%N_DO9ZL9?q)2%1MY}8LPv2F zCpoKk`$|cW+8kAe@%%^Q2aRn{88ADeFmXkDcfU+aaGpi4jU8COcG`kgT#c7kGjn7X< zFJ$P2@q+ubzvRnKsG}NIWm@X{zbxBx(43OSil_Q5Z_C=G?o}WG%bZj;KTLN4!DZL@hf%02iTeVs~M2&uy7|AJQH^l%*j@lZtC^xBX;2 zg3H7~Mzo;4)?;5b>^Fyc>@^;V<+KeJt@fimvMXqt3bg-w>PzXFnpM?wp?>`jKSS;~ z)Kr|SE3)#2m_tafZ>$T7w^gL^B`rp0v zWfPOt5|roAktnOR_SpM%t2y5{{|2%SmR_tZ7E4ePsTX@$Hy?FAhJi%f27leKTf$FraMdAeid|%q1=x zyb`J^Zp2c*ySwm;bFSj4^(Xj16D=I$I1A>gXew8FW4{;+w@p~^`)!<|txEmrrvr#b z6i1Pv?ffXKQ~--Rs&aIr+nZCfrp%9pHxI+|u8|?{>>=J5U~vr`Jy~21EJ(69@AXj7V_9zvYC;r-AFN_9VGnw%^|BSAZmdA9 zG}0$d_2%RYY(-(Tjh6Q@1X~4JIkImlKogb}qMr}}YtRtBSW)1|GUHXQ?9~ZOA~PCB z`|@(9w|PCLE)?}d*`jOxe2O#JGG(k1G1w$%W*<1ttv~(P{AiUeJJv@X!DSrEZs~Fq z_CD)fbFNRkAk!xG4A5aAeUUnh1U;DAzkrVKZBZy(TJsOLDmOR-9@;yLXPUG7Rzj2J zB%lg^-|}NS2O&2Fz)LDF=yB3-t!KkYDRNdE<6Q1n(dzI7PnzsdoHM%MBBG5Euz>dT z(I`Mm1IrRv@p4^TmX)Sw$jX~yC?lQ!oKfor>6?v;wiFnGAIX($eG@l1StkJb=hj+= zq)%GMp2~U@s#Tr_^m~V8rQ5!-wn@>}TJ2s|Z$a#AGa&wh`b!7$3b=BA)= z1y?$lT^e9)#LD{O@{kye?tQ#DkbECQkvIxoHuRTUrXl9Cfa{evp{wFxg2fj8l0g^m zH+E&4Z>o(I5SDFlWc)K!?HPKAxI{W;!-WMg#WW8+j*{V}yYCt|h{EO>h7B=x!Nfr> zUC~Gx&S~|7u5UGhrlU-dMMP+%NkR}}fp`|~h-XS=k3O*ImxNKMO*7O+d`?F<)SqR(#GrQfcwK9j zSB7m3_l`Nk(3k!}iY4Bc;5r^Wi4`yT*??j8^zwNrsw(!6<4TzU9mz@jE+ z)b6$Em}39L>QP)StdBwD4YtM1v~9FImuwGa!z;O_im2rIklp_fyHH!$g&SzN038`ymtx~bUUlX}SXg}t5h zSw%CPEK#`0z2@mHc7uVYt$3lq?j*~lD>`A6zCRYTF_lpC_``(GAST3{IDC9dsZtks zrp~pD`5WB47{C6xTf(sr2!V4o5{fyN#<|?t_NHJcY3*b{*_w2@hEjFo04~z%*;f{h zq*A8*&3mp*@1L8vO^3W=_!E!cGx~7`%o$gdZd`|-x~_TB7KSc{e~P`QPk))V?3~I+ ziuQE`{76eN#6LUC(2meuK@zYilYX`zjvFRZ6Xba>u<~FJ0R56a*7*I%?mh_}CQ;?h2K$8L;pJz>c=%`)#zW%rtGjnK?I_2AtBGLxC;#(jVMp`< zF}?^aS?N<$W_v3f#uL}3bkqJJC@TjeLUV!3WSY!X=4ukg86=AxZw1rrnd#5k_CT<1 z<|w5D7j)=Q_m(@+AJ$maTrx&xYpe^nQx7bw3F*`E>zS`^C|Dorii?Y=B@#VPf#e?* z?#S}LppL{8yYe-{CK4(jJqN z$S`?#^%Cy>9qq{(PURerr!5ZMskReJIJe)QT^R`Y?4ioj9WXTgbwn;1X>8n?YhSFbNmVup$)U7jkMWN|?0gyqK>EGa zX`7PYjgcMPyXN|SZGoiGq5?eOP=#@Fdj>hVmt}hZj#8h?IX&^o$1SMQJN${BWA>M< zQ0~oI$irrUpam+pOWaXsx`-%O@{kqIR03w$08{B4VG4|)Y8W;Y8qxZO3RYyHM zx15qy^(@pUT7?(1!ki<&x;`Z^yacMO$`8 zhi)uzVg*%I97hWkB_Rqse+Le*i8E(>QC(Uv5@tz!^`L{P#91;Z(1$Im9bw0Q8-c@1 zVzNEAv(tGnP@>}&LUHNl8b$F?!5hqM&OKyYi^;`Tr|5e43;a!-TQ*p8rX20s@-qWe z#>KvKN1Hleq0y+OD~H=jWfa{}da!~f*2o3{Tr*e9siZKLj*UYxCW+nmWL=3V&cfN5 z9UopB=_LjP>amGaFoaWaY-bNNPH?0|g~kEwR}Wx4O6|l&7Zz{&$ph;=ZHy*RN%L=Q z=0=soRPX|ai92qp!Z`hkyr>S{p~jy$krsDq>XtJ7`&u|=GplXOz9-9uowRS_Vka0j z2ctwMO%b#R*KR8@p)~?o-!wUo+uLK$jb?(L((#B&t!uBTi9RK_!a0$c<123vjwQnb7>!Y))ovq+hRWPW{QXPtJuIHsViQ#*Imn3MwEQ z_J3Ci^D6C&Mohy4$#M_lRX1kJeo8r4oy#K-OzT@!pvEd($s6_V1%$#seXw>ES6IvX zOG3@Q>S1YM?xn{coZ(DMFZ}ygEJkQEx0*|jdP5{81@LaHEKuK^eM7cQ&a~lEM!_Jg zqer3HdI>Fk_cw`2)7Oo7vMJBT>A$3n9LR>9aq?z!C)w*V#eEDc9Nx6_F=|zeCw-~f zfHy_+^ihFb7Iml#M;e0ZM(m}5xe_UIPMNV+gD}Y7|A^>QlBNG`<>CYIC}*s2MW!1- zMv(j}rmWlaS{j%}J8ka10PWfhrfVovMt7Zt$%Pjph277aiQ`uN4fKs?jKXp9OdJ8w z68m=2(n+77z`qzHJ~ z6cN}n6%1P&_GK^%2QK>pE-1($+q#0w9&+B)IjWt+1DketOUfv z#c3HBvDkrdOllVP?|4Xkdqb8Q>JrToJ7K}VzxB_+HNZ(Se3{2UMFV_03<99iI&G!STwMW9}wFcHa ztH^sVF~w!6gIU1@%%@zrd&9a=B2x-U7kj~6#FJ#;VKK8=FyGLvUY@MvyWGHk4EQ$m z43;|PSfq+hh%E(Y8v4rGj4g?UdX$}tmo)F%KM6Ncio&+$oWBY$98gI^wbj<_Zf|5= zy06K3^_7xheZ-RkM4-5Dt-}BVvX;=@r2oB{&(A%D@p?cV@%Zs;Ly0M=6hO~@VKydw z^T$=Gco}Y@z!q*9jxt&aPSv=X#5ClFNgzwl!4O7;?5y?IQNqB+VyF;%!Gn&RKD!L> z*c8riK^Eekn4pQr!D7Muf6#O2u??}O(fa^Yn-c37AIT+`QP!lA-ty0S24?KRexyXZ zk4qaa>uvMii=$A%=8Brz!-W146D(uI8q7!PVBVdwhPqF(sCiUmHbo-Mx~gTwN>;jHgJ~cS z8;!SJo!9}!Dz~eN3%WIfGvz$PF3;`>?mS6f=Mk-jk3}z714nEusIz9FrVMK~t0!O& zM8Bvoe(Gk7oQ*IrbQkF7;!Py7ZR%uY1hemrQJc^}$7=5_>j~Kfs7F_U{$#F(-kUis z!ehrI|3s(Qm!BcO8LE&V(C06&rq3DANb9Q1Mx{5$`rm~@3M}Qsh$FkzoC$SyWQrWz zR~+~RfEz)MSS@GVIF8C^@kklS+AKySL)8N+MDXiib?#}5)v-UgH`^mIh4wZ@#~1`p zAxzT9m1j(>Vaenl2Css{5`DopB(8Ew3qmr1f#}Y}l&(L0M?;t<7mGfzJBJ>v#>Bx* zE@y4bc-mxXw``P)>80^sN* zwUUZ*tJ#>clG+GH<`c~LxzP>)GN;X>3CPdTh#j4a7i#I#h_@xG86bYF*fQo{b)G)(62T%_iFu0G}>VANcq-?{>7x`M<^I01x+gM z?X+G>^kp*v=q#HDgf;S!%Nd=cDXY`_*Al@{MNA-<)#0;VKe`0Xgwmp{lW%Iu3Z926 zZf*o~Q#6lCQ&&~$M_NQpG4JWx-wM$xi(@2+`q^_ z*Ty?GYE+6%pr3b?OFwhV^3+qQgS~ZFN_|5q-<%|25*0sOi^XSGw*TkRND%S`qxsf)J!fC(|3-dW!AKUjh?qKiyl^{X-UF_+)A=@S@v zdoqaUL8L^FSF_QD2)*G|)3&t&aA4LohGQiz|rk#+w%!4a+%xGx`+S>eR z?V;fkz1bL~m1{NETaOK<$3m$l%ilTLKEux8Zw!v2;m!k#PlX)RRt57HcZZBvadZQ8 zZwecWM~U{0B+@!_A`RI6nb>oEQMlUhDOd8Y(o#5LK$A)>0Qk14kM}z0ym!R!+t0^f zNFRwt5Z9z3O(puF`nIM_z2DS`uiQ9O(~|~EJ3*CvIF`8hyx2lwLV~h7%r!d_$rz1o z4Viu`baXn8zEf*g#{g7d3kCyg!$|8TC-2I;e@fjH-mJY-(%NRmcq&02 zlrCi4_gO4^r051ob0ww%zZhT11?M3DSnX!>rJwPm$XAoJ4u^AQfr#oMf&&2qPOpIj=wOp41}-gHRE;SyTj|N^PtAQF`V`%9E_mWJ z2BL!$wxk+8FQhCTsqPN1Q@isJX@~G%?z}{`{s(*b3Y;&N1`#08nFR&K*m2tEQKK&$ z2ki#X1`4!6D>!P@2J9w~0}Td!m4 zn6T81P==Q`bR(2$vnhx`Vk$PYd$1{6F;-=yPaji?PY+|?$008I_k2#^&lL8+I#DVu z&YfckWoK^isb z6s?EVB2EZ3|JG*K`8$XuS>WjhpQX6RBiaowbf1=E5&Z;x7nfb~{^ZLm_M%(Hd)FLn9X`P9u?i<$>g7t&h!>&HQJj*99rO;$`$`Fik%b`7Xf0}=5GmVUc*|6102R3nav;?>8bA7JU z7En~+Pwe-%t$4}E46{^e#g;9@bN*bd2U}3+6YF3WpNC5YMZTP3XH<`ITQ9?SzE`kiYxCOIS%a!r0M7I(?_SzmTo8iHKug6-H2AkIuTNI7Z zs(RkMfKMr)858y%dx(nA8M-()CHn?5oYnfV2M6mI2$xNy)4~kHRdjkw zLyb6mtnlY8O8x97w8X-QVAj?InWo0>#%&zQEo#@!?gD7)L62qv zZDUn1%Uq-EAhkpB8uX;8(P+qUrGmW<#L3&rJRQvIc>a_NS~lQoQJFQO3H)nT=3Y!N z+d0iAfTimqD2WQS|JUfZVp!=Cy0Ca`5YRptSsUyESjA>MAVjpxVGG=&UpL6eRl2dx zTM)cRzi-&m;=~;o#0m!S6cj()12W=c!Z?|0Cr9Ubux~GnETY|>rQyK~$`pn4O?K6C zJN;{d@^17FW+z(0Zz;r_3(~&u*n%RSC}c52D9^Jmx&md8s@R8%KGLmDn%$u&81%;! zSttARLL*pVV;t4-`lXnmyq*JPWpd1-UAT-j!84oHf(%+KiQ{eN5cCF5RibPjAjY&x zhjJC*=!8u*xZ+^$hfe^CRQ}mKB=Wu%$SX=cOF&YckO`KzyD`X#tIOVFxJHi5#REy*Y80Q@7+vk^lqzXRqJzqFk^kwUh%8=D*4FeLxf`$9$y+=>lhdD)@r@n9Fy4W-82(=?$-gvB;{o*u+ zh2JqNh#uT*AB1;8K8Y802z@ul0*xsg;8p{U1QYnrhxK-dcIUt6Y{$LVv1zqVf}YZ9-tIXbWH)abdILk6?nC`LWm- zfLsZ)wD#)v!Wn4Cc$~3_dcQo4;fy292(tB}0WRi=L3>?`^ji>4kMfGbuU>&0o>n*E zWZo?=I#b#w-yd`^|BK5Fm)em;>hy8gMUTq7#H=dn;Buq74v#w7TXIwgFJaejhi1?l z&!2vn5p0U9bLE{?J8_07;juCIpi=W6G*{l2RWdt=L55V?b7{b^JkaW(fFQxWGfJup zN5&@uHoFhfR;Bu)*AwfEEjK1*p&L!XZa7h39rH+LZL z+srBiZ7a|Q3$+jVPcFW|0ec#gN9KAga2aeywLkoINW}nf5!OZQ$z4BV8S&WMh7D?j zU+Nn{-Q7>x1z@#~V=T3(T7!sAW=tnz}{T zXPus#n{wPBM`}DAWV-_i{(uhYRGaNQf}K_+>dTU~K7#03(qhrr0T&TVue7O+T=N2v zJ{!LaM=)gM!0odVs3(@7=%zx^1N0;&juz_g26uStgBL^yPsUcXz$P!RjzqvyQQd11 zW5ZD!Cd*x2wn7jA^TgwsmZlPY-cF?xi!;Za{cmyDwvrYZi4O{k!89C94FxT_nn_GR z9y+n*Eo0nxA+xt9uUBC{(sQc0wOeb>%ufQd9Qr4M(|`vuDz> zt^m0~c->Ca#o^&NoDr~x{g|@`nmgWz3;E5!Z1yVW6E5_l9KTO@N#E8 zR^jr2D${Rbcau-3$P<`V959yFt5$4cD=Y)3o!G_e*i)B9$=?o!dP$o|lEv-^LZ$CY zS*2;Z92sTA2(!u4OXXTVxx!lSLn8;OHAJ}J+=nv?F8t1FoEWHLrEi97RKW%Wi)uB$ zcLOulHUze~+<<=rHp+}SPj%C0OZNlYPehHf8c*>hRKGl+wAASmRF{(Dju75+!BMzeH^=qo8{wesxdjW zL5?+g#z1m`W$R&Gbw>>WSH+X7K4JWe&*ZD^p@lfKrF7!-l`t|H7B?^gqV{zWKAEz{ zD^&86N@avF+uL0#3JqzCm-03BN#UTDGQRG*&}&-*UfG72TYTK?k}!;XZDfgaopPX$8OTp&u#j7 za<9{^67mA{gX42PZW-a9>9YK5`VhGSMTiPYRLHZ+U-4yK&lJ<(M{2 zP{$TnXNji(L9=CAB)W2w5L|?OYP@)}cL0n`4{D2h*T1r4hJufwF07y!+Ykp3IN_ktS3(fC@sg4x5 zlJi0c#^=h^&7;iO>%Ca9a9mo-9_*_ES98r>SneKG2rqt1_JhWhxi|(3tT;iCaxE|3 z&$RL2UmKYLttMEdZwo&#A}Bg~tquTkuree{xy0(BP0+VTxxo#GJ8)KN0hs?6~H zq+)Dq-lhEXlbcY?z%3z!vi! zvgUSgL|K)Bby$PH;WQlL6@LNVand}LC0~e+=HRECfOmi0MmsHroy3my#upn5b#wwJ zo4-EZcOA^~7=ONkc|rq1rNmoj&ST?3rFeb$*zGQs!ZTn2@w_A;FjTU@>Cyh?)lyvD zJA|fBZh()3%u{Ui$8GarImMOs!nv_0(2b7R)GXOAUS=r$$HAOumEgs13~rwGEBc-( z#+UYjs&R>-61Z&orRZjywyeX`Y*WyxQGhrWMYauP=KW#-oZRkU#EwP7P4uuz+kE<6 zk8!4$EPBa(}(}eF<;IskmsPJnCxDGVvn{>bWA0!UtoC zyHKS2;82c5$u3$0M0XJ)&q;L07bmrgA=(di5clizf7j60p)6G9%(MXxYKdu{dzb(^a7O*bQTAV zsJqpeSZy_ufRn^qzoo9;?16E%OU-5El^2{Lf^{7t!;zMjrZu_(Tg*aJ1PV5JWEw_T zH(QqaLT$(bH#-_>4C#5gM|24~W;V8*;-GJA9##$aHKa!X9xWP-y$7k~)avV&5$MW) z!3E=lq8eJEsF!#~wrC*0N*@-~HDi-9Wq?)WA5+=BPs#2L)sl|7eP2i zaLqqF+Y=r}a3(LdX&Xipiqe_as^L2mIKWZ-i5_+~GE4(e($1*|pis#MD?la&q4LC; z^A2@a)~`my3Y%b4Z}6$E{8TmZyHSez3tv2@_FpmAED@@4%VdnZYg~ zM`N$Scj+4XM0(obEg)WtlH!C?R(9eH3YJ;{sG`|NY$*c`-dD=QG?+P-`gtk!GOQ@q(EkIzA zipN+8l2q`(L!T{bq4MLXhL0nP?yPJaUe3+zrukjRf;(_mgQ~nVUFE z%R6FVPHpLeV1WEFwXM>N3(9R=8KU1-YSFbUBFY?3j%*cQnhxS(!A_v{O1s6e$leDjSdDmh@tmKX~Bjz=xRch`o-@BnJy9?F@JG7Zh)vfzG%z zz#*r?16$QcpeJCFUj1g1=~aMTpcnh%?#wU_DB}>OjAX>7zM*>7LS7)PfZvhq3h*0% zT@2Y+ZRBfd&;{L&6BIv&cGu>>5QG3iQWI+ynu=+QKE_ToUK?W!*Kw;k{HM>88IaAy zhBbJpr^^9(l1PVJFp0r(n1oPBK*HFgHYjiDpT414Ifb z#`D4*szGcXK7~b^Y_5`P&BFhQw&Y-g*9UU?vDW_p-R-hfwg3lBtp|amX>h6A0eI{~ z^fs}~-0gCiE^RRtmR&LI5~cAL3rUB5 zHoIkML(-$2xuEWS6HRD(n}!ikoCD76;ez~GRwt#40m(R%@Lxw?7XLHe)v`YzS9IuG zq#tZGSZ;cqi`nZb=A|Pxg&hUk?K&+pv}upiOp-PuRP}f_Xbe#WadmUxysTwOH>EDS z*B;1yV#qqMX$QR_a2o@)D7@%GmD#OWW&S4X5ql^X#X`Wk)rzKB*##VNG7P#Ieau}AdXa?SunRa}hN)g31gNlge^DabfrI=cIb zk6|n=BEpR&P7gF^lhW~#B5v8;A<#_gll;+PDBcm^bLz^WINt~}W%0;>$R?cz1}1BU z8ZgfhI91kvh#+PPLgYcXxl`s(--wo`v0XeSJZVvleUxd81gGdQZ`AW?uy1WNwm62L zvDDBb-z6aIJf5@g^Dc8;(#9t0aP=Ia&cM*bkW}=(S9Vz`-1cR*hmb1fHz41h2%8Fpmk8 z2o8-wqn>J@A#i4wCu*7SpbKv0@QHs9yNLDk5^N#G;ld9geWVwPwPwG)vbJVUC2|1$ z{x6#i+ri52u(DX0IV4ms?}0wreuo>}sLtse)@gn#-AT^wmLZ zuTOA>KA1qnoj}Dhsh}MtXuTba)pc0KH>HCVd+$_% z)+k=3AuwePoMV9aCy)e5-Fc*V##dC;1l1OgoV3?yn4CBUh=hPR`M9;GyvLwQcdzPsB};H zf+pzZKcQQRn`a$En88#&BS9@&BRHW9McAI7w6^h7`1?6X*w^HBN=TA(9>qhy004;pJTQ@7P zV)px%EytJZ2Bj>ShLQnc8CqhB3 zZ$e+%B}zow7okG1E*Bt$74MG3>(eb=jhMk(xGCihX_D-i)&jlK+`Fd5GfX5-#AU!=vaw_b1lesB~m$pMdt@ zY3`;IAeAzE3@!V^l!hf8=+#OX$_6%P^S^>VbaAvZTWX~8nInB@s>Yc((-lZ0ITQE#L1A|DL7xQA5r@Fo^lCf^E>`d|d@eug(G+ET#hQ_mZ4W1b>$ zhgfRq`Y4g&k8AxJE~Ph_XoaY}nd{|zC=cGWTvQu^$}old3c@~&k{L_}+gh{n$e$p^ zlMNeIu~V?`m9#VdyfJ+} zD9Z3B8tB9C;hIokR~`YjYewz!#xP9Q4_@(US~$C6ri$fcuZugwmZf~N21+8R_P|5&*J26W8|r?_ih7StI5+!<+= z0jOUoxyS7J;VeE3Zw+-HBirNfFEb8bGcm#R|aj?qk3K*8loL^tUwyxAQ$EC;zD)0NoG zq^}HT59`OYVq;fUzDa*43ITjMt5NzB+t%#!3OVdKD0hD2`4D~zHTzj8VA2NAQY^2@ z6qOBE=F9ugQo0E^fhI{5Xcuha%J3*%yqvH?j@?V^zNSB2j8n2X@fc^sRx||#9uIzt zWG`k0(MCKV*uI(FrazFz3QxLQ?m$aU^9g6IrpdPOSi84{Bc#zeq5v{WMHH0BxM0>Xv=s6#eVZuCGGgCvLKxj&P2YgSYi|cpld~N6$;|5a8XGS^>bGw zEb&3>JjMVhj?6u8^D6r>7llgl9;T=+Pl>pO8#+`eL52q{ zH($9z2%tqaawCMMZv6+RjwJWg=V8y>;bhNXb#dFL^q0AK(unSdabrOYEMqHr`SCz} zoT`Xpsq5wD?{>lT5oO=|3qy_82sADwGu6>MC;bTrsR4&L%CDE7PTw%JvOtse_BOn`0v=(S)aVVtU#D z)>av9%WFTnYq`fU_)Y|vd;}l*8pjzNS#d&)J*R!EU2LL+;L8{}#BtW7A=>|oj}MD( zQrS`Fc*6-+B}A|ZL4Ze}@kV!~AX&CN0t38bVuTdPn;M2-(FYM~R|EPucEyd(|FF{G zIJvvrCW2yY6|7{39Wc0ff|$qHD$c(8vIfhweE74tvb5=4B~pHxpG0mPG3~>C)NL2^ z#yy|r-`Y%e2FdE|H4=#thq{& zASbYZQ@27?az!s>J~)~nOPi29&yE5Vr=!P+O>trhthL`QfEE0LhtQQJ=zBaAr^S34 zTc+`%tXR>JXl*ueJ}xWiV|-HH+D*{D6n~;_39}<8y@WU0SDE4>uj!P_{ZT6IXq#wi zOw&!XIj|}P{~DYE+Q%Gnua^~0-4EI*qnh)fcQz_0=a_I~-T%P&O^S?lb-B9&291P4 z#T`SVC7i*R9h>iK$p*)QflAm|Jhc1^>@XQ4qiBcbOF4rpd*B$Ykb-nowX#Dox|CyT z7Mpj?3fPRoi+M0?P}3N1ZHk^{OI^?wQQPH@MK00xq~Gr?f-Pfr^51#(1xH@M;jcDM zyD*y^LUqLgF7M{F-mH}?9x(}huh8YWJtS%6=i#(Z4pd3wxg4xmS490u6mb_p;eS}x z14`QEbPTm)a~4)MuGtbOWPGx^({x*cV{$T4uIHsATVW!4&{+&`Oz?Zj>nwZQj=~v^_TWWBG-Mg(i`v$#6Ucw2vxJJ<}i(oqg(b-BV0+R>Je&2^eBfs(AZJB9(H-FY+ zKL$|xICtH+GH=v!J&YnQSmVA|G3c_S-w`S2xG)q)(3IqdSc66^9OnntMyR-WkaV_I zB<|M#lembcG&lfkoAgLXcIi|4D?>O``w zh-BtX-4Fs}Y@x@x>{WVjh8gV&H+zBjJLX-nf}vi}#?iu_IW@uU^Xd;_|+!WwSU0+6{ZlK$e;nI4P}R}p)V8d)(dCb z64v)e+V=5y&+R4-vrCd&4wnb^_U&Ta|v(| ze*^}7(uDg2ax%L|<0-zmZpLih5%gJZ@|8rPXdX$^FrOPG9K`ei!Sg67w9XkFRxNw}%ir$@IsbY02PSzpgdHgu&I|urTfwWJ%aWW;d zYm4&&D!#G%T+<;~>wL4TIW=EQ#v2AxED#nk8cths0$xW9P5*Z?JRQ!x(5>dd#;ShQ zT5dkfWts&};?ad9tCk_qR%`69k!;utH#b(&DaMi+tAVuO8Bw_1Q?_8*nl}O?#yP}2 zUq(?ZbeA0AD@DD&>S?$6Vatht~f&ClYV&gn6@|?lC@`_CiAViM6JJtN8T(VkWK(z5IJyTCI({sQb z&%Xj`a&)YR6i2#uLgHdfJDbpP#SwVNZCNd3cyN_8nwbiXR=&iX+b1TL^_k#r$%Y)n zGM-8?+LIVO2Mg(OSXPLKK*^06^Sy;k~4wv{XlCnYAv+k;B zFR9R?bhP6XoqDY8o+w1P_rn}dyE_aHGq?o{Hn=kcC&4|qySuv++-;Bqf(LgAE}QTF?(Y5f z*4^*cPFGKN)x3SC`kdFF=cp>d!K1+Z^$DwvRQ}h^e|#bTxq@UgC0P}eW!Y8#h~dE? z{W(Jaix}R&i-9DS6=Y>JG}%D1S(=89%N%&Va$DlMz8RW`IPZ%jMP`x}vcBZ?-Kc{` z1TdVPk4HY;eXm&uz46B=SKem4TXQ{a%Q2(!9EchePvbKfg=tf6F!EMSA78NJ>tna; zSCUF`qppG4d1s%s{4AzuK5LKre74H^E;4!GXf5G!s@@H89~cs_iCqFDCdmE1%wxsV zyiz5l4lvtg=&}i$vm-9p8{RbM8!*Cd8<@HoRrenbqLJuLqE2h_9#&UiS?B0vfhVS} ztvd%CHTf5F>RvM@lo@wS#LykMAk`kdB`l>ZTJq3{OfgbP})-2d` zWb4<6(w7+{CwOAf`s;@%ZttBJmc>66?#HC4CD}8kR>(@`Hx6n5mdp_jByKPY0!l&z z9fL`*%Z}|8H=i?fZWkY2pwJ11xE<%fZbkd9x16-tEh$d-g6%=+K%ri0Ns;K9_Ica> zRqrOPi?&8!pwdFH2GA;N+by$(bb?{R;nK%D`#q3Lt(2E2m?r3yK8We+bh&V}$w;~lx3L<52?TjDm4 z1857KR&HP3S^@chm+LIjadWnXRGR8}a7%oN#Qdqr8r#3e@Bo1g<`N)i2Xt^2`>XW=kfjiKLw5j*$E%j0VX zly8;5K$fB-d#|ku^K?k7{4Bl3MaDFSf|oiucraK zLgz15m0EobG(Aa5-wvBTV-CoK^#mVvHXD|-IdwSU9yN)pDj_1#9p$m_zJY<^dkX{e zm!14=PXEAx5F%ijJ7IDjD#&wmiZ`FAmE2ME~3!Nl$#V z3B>vn_#Ym``t#ubvHyRQ{@V?>CI8>vg8rZX2H<940d`?CvompZ4OXdB?w3IGT`EKe zp|XNaWY8MXGL2bHt!UrkeS{GAJA`Qom=Wqj@X#C8qqmX!T;lQheoXv}fgv z0)}589NU;^Q=m^i*R>-0llz-*9`#sAdtl`^fzl*imFpVy>`bhw?`Xlr5yAD#x87V3 z$95b66$m4`U`yPxK)o#f99UMci%zg2MX6rHUw)^pDB@OScvP_}iN}X;MPRuGNUmEq zN5MiWp3*eAZ{(e_=HR#E_<%G^An>@PYW&>;Z>)2y1R|5x&#U*;9`l8n;LYoFmN{P3 z$vAtM3Y%V-fIIQ!8^p}V4g?4`Tz(`uH#w&0<#A~n@e}M{8nfOX%lPtV!oaY9Ys`N? zr40W(r9f*_7ZVrHzvnbs`$&h>lFTo}@KXr*%Ort(aLwn-@Bw{W3ZCQdQ*~Aw;vf8##KCk?9_7gB0n+s|BzkNn z>Y_xO480)p(fM{eFm{XoBr%j>dY5eF$B?cB*tEwU26NWST$Nk-6&kBuE1^_}y$qxH zQ9V?cJmHBaT(}}E;XMHWn-X%L-2r`Ydk@7;f>opQm7~Q$dqj)B%y1IFYt(6G+8>S{7!otX>hMqm4t!l(;iG3yVC5GS}uUqlD8N zByjWi`nfvkY2a;+WNF}qp^Qu^zuzT`#io%hWtrIjoDY75sB_El3}apW%>SkEy^XbY zlTmI--LQQZG<%(GFH9Y>4d!Y1sH8mW(z0DhriceDq)D_GBs$RuDRb3I-|b1M(_LoC z()@Y${yF}fbpS2GLnCCIW}-29&RR)moHye%ymG&laE43{aBOy-H9=&dNT`-bBf=^O zt@+`nT?ltXy$x^OJj#r`d3s;y;Qbahz{R!7{A)jVZP+h8=Q}G)EdI zuL?z>l)X9yUtF9ewqH-Qxo6kZ#B# zpPWqo;+f*SRs$-@!d`WL*UT=bNo_FA3MS)hVRgm9*i~RrDZeI~t5P6Hh-;#qB#rC~ zhYTqk=EvjA8|+2{3ofYnGS-63r}oHruJ-6{CMcXof<( zL)wMgahTLMU|%XYo?9NNzUO+CVa|Sk%k_)8>~6Nn9_x$>Z`# z@45%Bbo(O$TaP>LqDHr*5M=qD({>sJ!J{I6W6T5gOVW0VHok_&^D~?uv-TReXWC$U zOG~5qvF1e~&}0#3+FJbqbJU`!S=e^Yttmmvg7k9QLYRvZxxcGt{urUu$geE=PQ11> zXZ{oeu)U+B(ZOntOFeFQw@O$Oq1=5@ZA^{?tY-M**8(OV4A$70F=tpy6K6XCw%w>D zrWN`&wea`uUy>fQS(Lchw13%iqHgeYpYBq=tu~6(RtvFZbNC?K2(eb%S7p4gfbM!6@w_U

kWlPapOqe8l*(2y-fRs zaW(%;!gxUpKJ3B9sEgSk{T+CH**Tp|C_tr80vb4kdcMcCfal_=w z5KRdN7!;k7Ddwghtu!roT|L8Q34Qz`)2*PS`)j~dJZPG;PXiamN7l~5k4 z)NgF5A{jSyZz#3FKVzr^sO24~RM8H41P*l$8u17maeDozmagiqyt}RO*z%o4y}zVs z`(u`nR>dzmoePD=Qz(o~KP1^F;n{EtHhp#hF--jgoFEDqzWNKqbN1=CM457$1|K;V zd_qV6X8WDpY2+J)sPxE^Q>uLMrIIcfsbs~I$wL-&z*jOt{rnFfF2vJ^bN za;y>eNw-%{7H-z8()fCIz@7m|qoWO3E19Fo+?ZR3B}e|?^vKLr|$CdNNy z6+Hi(RY-wdAl4SvW+o7@HrVCQ-T%(ol2TB8G~GT8e|}AigL#V&|BLpLPJ|}x3qzQ) zg+n}YLYgzVKa3FCD2R(Ajl}YF03IJ7dtF6UO-)i}u*D#W`OdHA=(PQ0`O}KS%4%~* zmA!k*YV}I!>P=?G=lEB$H}>K_Blj<-uRmXpGGFi4@1Jhv)?sUkM{#2&a5bkJW;!y~b1}&7K7@DeVDHLeU10^cvP8;HiHY1*f z5J2IJ!+{B@ble~lp$uy#s2~uUTFgCZoIc7EY0U>EEp7v90k2Ybxmk_%Mj&iZs$!es zLEs)ItBFvKH6xU(cn3(xQDxj7dq7p31guD9v(TG_AVZ6Qe#OMaFdDH6X64{=vvNxm zPF{{G6BKSJ|{Wm_{R_EZmwEDpYJy{2NG;%Ex6m1wqhAx5m{7vF6r5 z)nHDTnje~D~ zT?8K#Lm<&#kdUKN^dg6$W%5+T2!JL^RtApog^e0y+XHhR!xCC(o-|d_AQYB08!y!b z&>|tiQfoZ2n@bSJKOC=2cpzQMwxv?MmEQZ5;62=5e$__r^7U=gi{Owt@22YeE?HZG z)attqMCGLiDU*_E>+%?(pSj~dc2=eHM ze@9y#jrk5UcF$^ZkyU~CZKB_}ChkeHCReJ|EN#s~0H{ksk5vd@o3eMhYll(#XP7eZ z1TPi|AJC;ll zA#edge0sCTFm2|i=c^kzA2+**z{6UaL@W84QKu4QXAmjX z+c>L%@C`fOfQ$Y1UEG1XLwHw|kVAk%Pet;iaJAAJQ~3a_kZO20HwlK^QgTY{OBo|~ zj-cj^bQua?g^^9EwW6lk2z6p&_UQm-h>dMiD?uE-09Lg1>ELWvT*P3$^H}3I&bg*^ zgUrW>FssbU#Po10CUOR0GfvhuB1IQ(4F=*H`j`yn69)l&NQ(+&?-N(Gbz)?k?5ohN z!<|{D{9aM_6$IiUe#L-CHe|MH;+`JM>NVWPKJpX621kGyUgyWgZT5f{3E$Z|`?Z0M4pPRx=dNQx$f^yYSSMPF@~L#ljkfTn%l9K@Qc0G3CCHwXnZ8rYHciR;9qJ^aA0*bs=xu{v@6 z9?A;zPICZzg&pt}ho(7LdIcR|6#D|*xkSexgite}JGbZ*1QY64?45?eu{wMW4|OP} zPrKm|9fXKOC5sVLvn{-058f4b0jIe{2OwbwCdGTHypl&Pga)qZpsqlkgM%VP8G2s4;A@yerF;lOs1G8fOR525wzCZ1~ z$?JE}4B)zn(Ac#gl(YDZt84HY0lETQH}eWTcm$qtJ%2e6EoM(!vwWaD#U>K<7&+n| z#1Ps~+Jc)}!ekzl+cINHe(fn>bQ;>Qj&xYdZ%KU)EG-JeaB`>9nJch|?C-;1su{q` zwxM8UeJ7xHw5azKD%~#c+v%_T#pE=qU_s&x?4ywi6Wy(DWg6s=zU(M!5Ag9iiRIa+tuq0uqDnHRUJ{& zVs%71WDQL6q`*8$Oy>C^58lpOdf$ga?CP1iC?H)EFOkcrYX}O#@qN3We_U^qkkqPyB zH4O{@;BN;r=Ckwtmy$dm{$Z-vymiaxY14NN)RlWHlNePcGE4e~oQdoMiot9Jr4^io zFB+Z=s!ZumeY&@;J~W2(v}!J2t=Hp88V0ALe$3@n4)pM@r02c{f}r!o-M2W6@6!H6 zsa_n0X^lZ(2RY}1tAT>|E#>i1c=zGk3}LSOMbg2FST~!%MYF?u)5TmdIt)&Ubw+2h zo_dTCnFBMsAKJ1}B95;Tii0S!Drc&ksm@Ou-|V~>Cw_%6;VbhiDzIH1jO;{Jv{oR4 zT~&+un?&Z`E<_s*nry6wNzx}1H(2Dv%;8NmtMN_>Ba{#ByQI{bmQ{>1s0yJ$^eR&5 zU6^u~1mm{6X7A_Kp&INE3y_WN7_ngaT6OD;!;WHAb+fbLSQ>(%{h+ibQGx6Bx4t-C5SibtY+lKMQhP>He`(0IDA`L;wh<0|lz zcL=&p{-jIGTdw=g&CFq<=cf-!q63-hVsOqi{1Fz}d=}?G%pjGcLSd`(dx;S#!XU{!^7tD;>ztGu1nZB9H?-CtlW*{?KPKJOHYPXL zC_J_|I?8)X>*+7^7l|^RaQ;!88++^ZxEn+3^@JN=Yw_3{DQofg8#8P1xEl>?@q`=U zIX|`&_o8RO@|nV;ul48mCn<9;Gj$1fmK5-Q(#PL)&at62k`Isc*M;eo@4d&KsIh8< z*u;x_`ZGhJ=Ox!I_0zDQL&WKY--v(vIFiEt-qC@?jVN1zuhX$_FK@q9xGqsyc9Y#x zS2LoeRgIk?Tmz^M3=~-ps&w~gqkI)(YZWWk)(hn(_$NfYWbbW-XtoT zn>DJ5bE?mS#PUO`u~|Ms`j|PYhFzw;NqMv@3$TH6wGWMin{}d|bES`z#FbU^N2;Ys zcQgV4!LqC6IGT^G#VI(tF&M~CSf_QbuI|b;&Z649Uqz`a1ad9 z&xqM*=Dosnm1gK|@vcoCaMFs>lqq6T));(OPE-(4S58zDfn1(i7{OScS{`9to|+f& zp**!9f=O#aXAq#pqCRM<#iBN-uEnA=h@xds7@<|JTNR!vKdE-V8~nUTaS0MUgDBvH_oNaZk&UEjS`zxuCQG=$6JJoPRz=2~3BEj|Nj zUJu&Uhl+i=o>Ygu1<1A$pk==Z!u@d2T#-k@BGIBA>p&ds5TjQg4ZvH0G#GNirQ7dS3kL_&j>Bat=6VL5q9 zPbfMOj%w#o60_Xo#g=r6*@OEx22m9v==&FyGBqXmD-ylzZ8_TE=t zscwjzZM~f){Jwe5>2Uo-DlFRb4P<1T;*RHVquj^3*4Dbvf!E^s+`3Bgjcg6Q6}sZz z(MkLBWYyoh(|z0MOzh{LXwd2!f9>!i<>{?oIAW*xx4^ZUx}`^9kzZGvChJxrTNx3) z-3T7@EpKA(c|=Yl&3f3Q;krq)&wp87GRZ<%6!y863ZcRIo_1!2O#e4zfPa$WqN z%XJM6psI`Yr$5!Y64(<2hFCe8OM}f^Je?q#Ca$)BV*^?0>ngaCSTDuyCz67a(t`o& zjbakG0)Ux;`MWRT0na$H&-?jTE& z)g$Kqn#jimr3wdD2CGrObL+?o1V#tNdAmGmrTZ=0R#9L-#aOH0>MGoMxEHC55vAxxo;&5oPi9f~ly6rO{n}FJ$s|iPfTt~`i3RWRZVR6{$o)1O z&7?#3)P63YibgT$g&g!idHmCit`j|D(B+Q(+JVH_oIiFig4O5;R}Du^EE#$M&I(?! zpgQ>a0YbJvhX@7)WW zKtx2Oa-t#Qxu{~sE|h? zm8kj~y4j)fz7dwXvN-LJ73LiKIb#=t7-slICMSY9Km3yp2ID%*lir+?p70{Mh6Hx9 zs5jND@|DXpPwg~Cal|AcL6N8@OxgOF)?`v8X=*QCN#CF<(~K70r;)}?Y1!PnzRJ=U zak9m)Z>qta8=ucW96CSu|Io}DIwtoC`SAV-eO9b)n0%;*a_a_;*d4!RR7l5uanw;?n=@!%(OqtZT<&G{(mp`|IoYpZ=Av3 z9>6zoMH{M$74nKM4?8X?F|}tVSyZeFFB6;WA~yf_;)|YCd$2L+1ITzZ;f}QWr@7}Q z+?^_$B_P*0vTNiOY+?EG1Df^pa&<-c1&NE-BOuKNVw!xw2<0voEZzfN0a1bMsRU`J zK<8rGVt6PQ6a~rzC58$?-$Lo21W-OGhDQ7W7^VzH(_dR_6Ya!{Mf=yfU0Qas_Ji{O7y~33yhgA=a`$5K@{jv1L3wYwP2}0Qr~CJS_X!*@@Gs zCfpT|O z(-9s~FO;3k+I4HQI|q?8#ZQn}rnB<@x=(J5aY+j|I{~6@XkmGKkV*4<1$gw&k6$O3 z#T<}nh(pDoFi`Je$zn{P6p$h?==Y*m| zlRY)YL+n@Q?Y4m-N{!0fySx1#CPmQBNJ*8lLgqD*4AIq%H^q}$MAhtZ496{|H7*+_ zlr&P(+@P&a_3*^3%oTH=lj<9P5w5Do@C=fObx}^rFrLxihj!s=lHMp!fd%p`tA!y( zm=)^cd2^mSbNLKP7)Fb(Ok;DA8(`7?MO$B80>d5fLj1cL)6poMs?#~w86DYy5x{0c zHs4n2KC`iv9Vkl(!bms#={~cSzaW_r`3aFsytJ?v&#=V$Q391|HsbyAjQnJCUf}Xc zA?TCZ_UgP5X4TOkX*8`Y^@k#7B>0r2x(el2Wa?fJyT#ZT91(dM4lyG!(5g45O!GhY-8hxSH0tHy(kKek5viz3e3_e`R|hGhQ+(Qejo0Cw12I zXe13$MsEp-vyk(b)`hy;*JJ(Bx{ZGf-Xj0sTBl)c>F{rwr*8Nsufr2<={Ny=02@lM zhE)aVW%S|%P#_OUN6{rJ0u?qBClO?8KBX~E`ZcIaaehfbjMzBAp&!9X4jF9E6I591dERI??df}8PlTo3Kcw~qLt2Tpy3ZH-C7nCL!T|a zKfR~x8zx=!^+>(oj4TgD6eHil-aJ@{qO>D~dqH4LHOkke3A7TzFxf*%KBu!I=s)L5 zrKO$tzG|jXH|TA!xx+@uv*?6J>~4o}oQm1U`*mt#`7>&jSJe~xyk3`dnU*X2rcYB& zjehC)Eh}C7WG{G1hhsX4jKGB9%}*FRyh}=wcWT43vWrc#RLMk}TXe2Tl}~w-{>BAs z?Z%s^0_&T}7vOTkIv=-+rP`=(n(V2E2+1p(FTFlN#5v&cx=X~r^yjFKuX6EEHn97T z`Xlqt`lI|Wf|my5&&ABd&dyUE{D*-Bb}$2*D>_*JT`IW#ogNgWJi3y|;0#+*Vj;oC z$I{bb0tAqLZhjvk-NeWHK}Q!~34MV5`EAw1od(po{39bvi{;egRptqzbqn_y<};qm z^s&xdWu>O!LJ`x8RlB#hS8<1><>%*Z-?xt+w~s@&kB7gA-unZK?iq|@ zdi-4ZjfW4J2w=UtDK%!9-aY~xubzWwEQy)6M%k9;zEX-(9h*nW zx-_4aX5@EaAkfFiql^tSpT{=LlOYbjgKF{xA5;i@7B@!q%(er&rTq@Z2qs5aT{Kf7 z{AtmvER>Oq(m&+XCS;I?(L%E6#ArPI(EsD&a>8SfWPD$ONRMO4jd!GR#I;O)nGh$0 zr}V0ny}5mL&W?%^j$H~FWLQ)dHnPC&pc_dQH@lA$o!&TpRJJ<6wa_h*OmyAo2Ok9&~n{;avR=y0LDW?YJVgLAb zKT4N%QMBFVGQJaf6137M#DgAp1MK0U6Rc7>@L=imlCPV8q!Rir?@^O1@~hw{dcWzb zf5x`UpebmV56+t_{K#PVzA`kKnBLOHxyp2w1&5l%=i?pSaMh^Okr?azbH2<;p(h>H zk0dHttfKj*_y_6KNfGv-YE+P9c1}T!?eYXkLj*7Ofttm!S>CZ)4}SXG)mXs|$i{SUk&ogN zg#`G)%1kPmS9mXQ>@hpl9()==HysGR43!KHwXK}JjiFGbvCBZTGO-B1@P~kI9$)i|6C8dgh@@g zrHFAM(J_-Y7Mvq>*^@wQ_-jbasadGTeo?U`*MFRrA1t;(cj(PrMm8c)_^PqM`%z0w zE|mNC<-B=4eEC&Y<}G~I?^5TQ;|h?j7s~hFW$n{YfjpntA66fojzVP%cgDBoGLgsF z${N?B;J7Y)GP8at3N@eHs3?|KEo@pDVN_FO`G9T;H%B`Q`r7I$O1PQwO{eVLX~Mm% z3=K2GzoDLu*%jkwkx)+?U)m|D4acje&kbo*DLmpRt-|7$ReA;O$Rsm@b~v2dDJ@Vv zLyah#V}+=6f}3xHtvyjhV?!Vv>a-AqvR}cUm>IVsPBV2}WKm}m8M`by(QCk;@TZpD z2>rZ60|~jie!3K#bl7@dm%4%M`_ycX4N~@Wo9-l{ekXuR&>PC!5ep;xvXs`|{X&r{ zo2b~Ut!+UrjMU~x7>;O9zQ)r6f8MaS8q77C@ufGenbcA~oN#SWZL%v>%v&wBIT8US z9Sz@%bvT~Ih2C5q3LCARmHjMi3C53R?!&ml&4k@)yQuIR{_)H*zL4r_|S;GaGjF{ zHx{E9)Aic`b_|)sINL6s?5(h53nx8(DE7(WWjwSUncc(rm*6wLS&b|2)V4wN)y+?j zY7b90w}VffdDqEp^HDnE_*qDX;*Rou(TzfBX98Q!m!fx9+`$PMA+< zPoz(8o|y0W&y~)Z&(Ai*B`{6{v55lT90!QjVk(})dce?azyKu?EH=a?(RXuUQJs*) z;4!~p-|^nDpRaH5C(mcBw@*Evy8jj2N_wbjZ2qLT>^caMN(GmUnc8D4W=W zK_(D0tN+bX0Z)peyy3)y-|#$oDNRZ;l0-%Sz8o1QKKSVqr^q5p2lBvPfKJ0X47nmt zQm_UZ)Mw)8+2f6_scJ66483_3fh%QH$%a24?hla^v_wcez1kja7LxASE_U~1x& z!;=<9`;KoADS;aH?jrLrl{+}g$*En`OZu+X5t{+AlxJDENLs+JY|4kkT$j+Xx|)Qi&RAmLCMoV&XO(7m=OXTA!jL< zhR(bT;=?+mZH$|t17slEQ$pzSs)DAdAbfeQNLfnzB7hYndwPgi9xBqwdkAG7Or98k zD$fsqllML^E9fm!FcLb>vJ3-|0l=WhAj_bO2|b>hT6;wZyLv31n^Y`;yF19lx57+6ksT6dZ}h-SoLR5OlWmfD=s!9zBGFd zji5QwMXM?_1oLR^q?_xKuSus9!zPngC~z7Yg30tNit4{sH7kBeb|D*OXGlTpGl@UL zGDaR9G-1R>IaLV$Bx{CT*7Jb>v@I3E#Ll7goUbeLEnbYB^}5L}l3#d|MJMbV_9-*_s0H>6P}F zdrd8}rEBulNCOTg5Mb33w?=AM1leTZd`%efF*g+boor~Z6}P->6}=qcw>DKFB_+zg z(GdcF-CxFQy)i@`k%>0YMEF#)wJE`np7@Dg!-TP&Td;RqqCALZ206Vi|9K0xj9GNG_}%g&>q-M@L2IFjgMeG;;73 zrz?P2LgSljn_Lm-(PEB5p5e(`HW{p^h4KJ@a$^_|cjbCJB_1ECPP;Ln%oe(t zrE{9v@Ktxzde0;@<+7BYo@F#Gbobr0>hVxqYu4L5YBllLB?c^qC~}rC6@N5@it5fI zxFRs?SNH!;&Sk4S#Jv&OsgLKe+4uJmD=EUy2yXsX@8YaAT^q)COn;W}cEzfoj43VG z8t6&tpfg2q992gl$E)Y})gI}&oQz*9=>zvbz>1YKH=W}Z;=(vSkQ6UMPpl$cvftd| zq%|c%E;6})u+VCP2reSKu6~T>ZKrYrzza!*yymT*%)&ci2V%YN}HD~ zE>C{V>esHxJZ6|g$z)pliA-l0n7M3`d_oN$XjT-xxblBR{yt9iv~c;xA4rmJLVoN6&1+bQ))s$!Xbr%2~u z1GVZ<$|~w$Il$E|LJ+6|7*=v1TUfgqak9EOB(xVl-3^mXWKT!I#^tL`FKXJ-9e%li}b%53wiX$t2FJX7U<0dI}@S-drx z#G^)YK+|Pv;M$Y-W2#NQ!Gb{b4r{xSZux;rgTCU@$HgXp_3|0ogr;x_hrY{8?f@-i z-Vb=`AKK?Ea8iqg@M=6JTJnREFbC7kJ-2IoJzRCe&H?lGDtjK=_Oa0cY1)CrMPRbU z33ML7m#J<31v(G1&TiyxO!2|JqYypHRT^{nCIzC<$Mhg%$`$w?f5MCnut%BpOnQ(n zLx^vz6%HLT2xkm?3_G2QM@F8$5SI(-UC`y9gN+^{NSP3;Fha$H{+I!uF@u)(CXZQO z7__elctkdou?m@42(mynl-h^M3k<4$4`Iz?MmCh&X96tYwx`da2USx-!e%4^p~!|x z`?7!~?Dp`P-$4#^5V$;jBr+-pb6$VYSFHB98T%jyDu{4i2J))pJ^{cTr#)%L@ULG; z{%msMav!Oy^Zg!M>p2~+SQF_u&Vt zE6b~+Q(O+~9)HS(zE5XzcA?Vpkmqb_pP#Acbg%`UIut7jq8%L_<8473zE{5&VHX)v z-GFOtZucG(>hFst$b^8?5$`-$=uY?f!p$Yil7zFEvlgi36WHV$2;VJ+7a7UQrem=s z7z_|M3%ze^^1KX6=bHekedq0uC`r#o+r?wye@yxiS?A)vH#6o*@SHAF{ne^1sx%d% zd}SF%JZL!ap)%bPoAH+&Yh_fKA1-^Um&GZ1D?IIfc|_V)f;vxEOSYt8N!>`jj*(Iv ziCQgIeT!nn#{syN%@nmWB$6R>$}_>uNj_UOQ!h%hc(nx{oN_~sHp`-mqCopArO3UN z9l|NzdR-%4Po0l~j`eNY$6edTxJX!tU#e=~um!-oa6OyDEBp%edE&UF7W3o3^!9mT zxb*gY8XWn!waqRgwzJJHWpr_2Hd5m2^VV}=IEGqWZTDmIgm~42xE*|$jw*>U&+q|W ziBQSg5>I9Jz4>4BO!`Y~`Yo!>Y|Y4g4k6GV^%sa^Lm$5_wZ%g#1iI3pwgRFf2Thqq z!w0`J*Ln{ql`&nts7v^ry|_zSoV=JzokM#>vYpFr~clsL0%r*KZp1AH>%{1HvBwDw%0MvB*xw6 z&JyLygze#FuWs|ts&1*nKnZ^@Mxwi`e|JhH>Sy%=tEJTdcPQ~Ly%&|`Gx9TxUs(N|T5nA1)3DD^mG7o^vdqHOITt6-nH^WC z6F4q=Dd$wv$t>6!gvr9SeZjb-0<*yh7=pEP!OAp-Y?Us?O{nz3wL`&87@1prWf+;e zfEKtT1KqRU1GZ}crw~*U(tS6R0=Nu=@5j9agi{dX3RFu$r)JbW0jFxzJ;B<^;1i6@ zvpxijW2a33;v2SW!T0SmXJ38+X8g0*&;+ntOlJHI*@DQzTCv>T%OWz?26mDyZs-i; z-k5;`s)fX$`;^ixeu%>IL4 z5go7e7qA{Yy@P$_muNXwP|Hlh;?^Gk!WCAfk}e{Wk^dl#d#jG#ihBsedbVZx3sWB; zLp^`b*JS=-UGz4yEjw+vuj54oab3d>^b z2}5a`(&Xc+a*UN(zicZT0*7zL?U0R77WdHX4!COXFqK zkazDaz^NLXF$f4!OayhgTXpbCXlqsGVqG6cx|O+WS)@LWHQN9h2EJM>wneUCzvoI& zDZb%{c1!|3av8S1d0zQ3Z{hXGPTf&V^fOs|6V1?@1WmCm{~+TjF;8y0Nw(fbH$;~n zHLnZZ6LhMoGA5Qw{&}kQPPMx`U@+Nii7Y)aM2m{+ zvxqk`O-A!VfMLzyQyL{YT64uPW)p5S59F&m4!~KDEf%zxS zL5q+$!F(S_{I<42V&F|i?$4jbRd!30fyQx&C4_|-=|KY{xAw;Xp7Q2v0 ziJ^^i1W?LUSLkYFCQ{D0+;@76xP99|8O7u$8fzlQmtLqu1g+#}5hH!>T5C1ZRXh(sS;5KggES z$G>`nzcg-KVuS2KtiNgHtXtwXg&}U1**u2E?5_r11WkV&w@2SBw?}^+cPWMZVsHoE z_I{Co5_@{V=I-ba72SSlq}e+qT}c$%5o0n>k#9;OGO^|T==!ap&b_0*1SUp*Nf;{A z$+>Kb?ey>8%WOFy;G+0Mlbn}B)Ry2x>@Y-BDl`w!KL_oRc75FRK08k3S>6^I6L{+r zU0)0vB1cBsxwn7x{vy9Xy0jq3>#HppjgeboDx|!x=U-dHD7@Qn2L71GDf~Zq!vBF@ z@^7B-|8{`(aVyGiII%)3b~M^pz@Saoxwul zI>-=f{RYzlX1`211|e3uZ1Bs5*tyxOy3(TJ9EJuH6;HDI#{GColhfCk2aT`l1l8mK z@5v@Mjg=xxItul#s$y*sKN(z9gp-aIbJ{CgWOO)XQYtlVW<&788u(hOvp7ArF{ z7p8HPDJ#b|a>@inC@q~9rBbu9pt8axrLuBEP?@xBiX0beRF>2*ExAPzRD|Dwc|M=- z@AbT%*Yo@RQ*Q)sZq9w)*SW6i9B%y=U3Fh6e}`LswsK^hdnW$fH*%frX49X}(W;g- z{c-mP>t@TE74UlHNUd;IGcMt;2Lu<_7`gCAt8 zRhn~KGeW%_E)M2yD%`r0vSa++d*@eMFd6N-=7NiYgJG zYd|kp+4}pv{3E68^IdCLKhHcVw;C1iXKwoR!0X;AebbWzz0}$7hp%iDH<(k=x#gec zA%im*_S@-SUL7yztxp}JUr4APAeEi`Qmk@(9}H7$Ya>q-v5(mt|3R`M0Nt{o>BE8$ zFPm>W7RT&tS|VL)Uo!LO_phzqwr@~P|LHM*)6)F%0G|zus?UzUo;%z9*u22Oxi6PC zwf*rc zX8y zh{FTR2cA1E`|(L`dK{6q>)L};0T)gC zeg0f}>c{W*uG=&-{>QUFO?&M0$(=dV#?Efb2OU1BtbNoYOVR)9N)y@|_2=LpRo4iI zpS|zq&p|jtyI$1WXBT^WmtvVGFkw&3>ms%Uzxy8f<$RxHLzB*`BEL^4-Q23R(mnTB z68nokGb+8y+`FUulds}vY`FB*?a<>W_2e{H#9`h3&19QB4!n(FTbZ$DDETV!&_4&W z?{ScR=if10(ko);)FtJqRT;-SE0%Q5xc2!hb3+(zvhd_{|B=Q z-m#Gc_o*SD{FhVR4*Nm~_Mgq)Xu|mO^i=b^FT9>VH=h>J^!MBr%bh-HPiG$VCW>uA)BI-`lzUI7OpCUecb5}BUm%dy0<@adu4|CriQr*&uwF4cA(F1F> zgKDAEq!m_Z^SfqkDKnCvniFoyM!(ZY&8#D>ez4`vKAR((GpH)mPy&x6Mx+!uayQ)h zVqa>H+fZZoE#f;Vqy0@*&A{x^-W>&(AMD&By!q-G`^ZLbyOU|>pU*-G29YiOmzdbB^z=;uTNUF-14lyxyc;AAfnr z3a3)Xz{L-a9sXuxlDEF1!ez@(?ZJlwEBk&Vc%P57IED`vz07ThTYzu=dSM4`mHSo6 zmG*5*e`gIJ3tRTruRh;q`EM9loSYEd{Pc4FZSs+e{Xt!+OK=oUp~|7Xe&MOs<3la? z*5$5t5Uee0{IvhIm&e?$kM8x=vxeGU90)RvcqW>~WDNZLwEp$-MMGbY{=+)BoEF*! zjafnAbIUZ3uDkY8*~-#qn2nxq%G;g}wJe+SK7w3TGs`lue{|dJ8&}1b`mas7ZR4LA zQ08nNxAZAJsp#2`JNk^_sZ*{+xO%QTv9@xN&$6k4PkXZ2_T{Cke^np6@nHP{g^hK9 zt-Zucl-?Rsy~ORdxr4*qZv@w?)i1o}4a{4*!Y@*Se%kTG!DLCq9h-3Xfb6rkcJFQ7 z#r&=$_u$;lSw0sc5-xg#j9$KZ$#>P&h{g^~b;!Nt@3Jqv>?%7NoBPx&0(VH)5OnzK z>)(Dp(s-)kfnD#jcTaL2UuJatb8GjFg8Z$AKNVbe|M^75+s)XM=T~*Cy}JL;C$XCH zJs+2v?|--pvnKmqScd%Y=dJtpU~#$I>b(vv8A<(v8uJ)=*0I?3{KGx9uw zN&h4AVmYO1|8oDQUi4T{^Za6Z3bo1@E=7RHYI-`YkV)0ki5ww zQc~tZ)Q{_Rsa)&#lM^RzomBV!{9k_E+CL&*ofJgueu^HrMn(;9 zYFX4Dew&KjK37e)7WFyj-0#}ieSCTc3iT@UF!_$fmG|ycD?3vuZ@wf$~!c1{b$Bf=V0;EnLWE_>0m zHcS7UQt$lp#_1R>5`Ps{V4C}&pS_lE($!FJhjEP)>7ui z(hddV_0}I2ZEtW}?Q@~ktUGw>W#OF^s?GT8H9lK;r~Qp`7>zOxt69WP9`DQUsV>W5h zK5%9JrwjhS-zv=@)&KW9j~$8GXS_2S@jrJiBOZx47DpzMeby4!dM!UeqIOO@9UeL{ zGC2LfwB0Mex_@{7^o*JxKX`q2=+5_(_n6GK`fg|Cj@-N7c3i*l`s;JO z*rVievYZw;$dd(PF6h|NL6p1CzeL~2w;K%mq8q;`Hh)v8_IGMCrE_9 z+cm6uc_0@ZaiU9H(e+Jbmyks2l)H3Ha6^3@^em!Iziz-+{HHHj>}TQnc~ntctPwSb z(T6W{vi!YYUAs4=_vL*0(j_A6Z#Q&o%vJD4x4&*({586hzP*z^@}a+i-!V~t&P_Zv z=I69FBk+iqqF?@vE~lU$#ma8P??{s0P>aY>rNQs?`t+C!WjAxi8Z47WR0C&3Hy0*8 z@vfjhQ+Ct(w`Y#e^Vl)aZ|}N#%yWe{V2e;>MHHSOMCv(r2Q;Evf3jA*JM1J-M833V ztkm6H`EG-Nv7)Vie-sAgi1l}nS9M; zW&u4nHnPm2!DVo9JpT&bsq4m}N1_|`s?vK+`T|*LdRNIIXSG$7>$}xs1g|m9T4nI1 zp_>PWHJ4QXyziB~?p@cO+d)5iOVP8%Z*OWCYtAMlWGbKWBB9%nCC5jby^WNJFW47q_L^0_{yFI+(cZVHSbFJAVElu zUDSqNzW9~?C6bfQylAC=+GOih!aC34EFR)Gidt#)TGFrG@yKFg&L1n^U2DR%+UuiI z8BSdv`QyTenVOCD0E@S}kT>fWS{^F4|Fl#!EJYT;KQ>nYV$rx10;FJXn~<=wg- zzg8!`*ZUlOky8AZj_bYR@vnogkeUyguT6tzCX;^Ey+ST9MfFep+YRxQqAkKYm|r_tw?P$$6;0-rvsJ zCh?+$oiNbzHo0Yjt{$q?yE4lbep|+@{;lcNFC<;B^o}>F=__SuOVy15pDop+4Gf!f zRKiem-bO*!Q1ygON4)foY{LJb(8=D_8BO$E;TKOH(Y%jM8GE%?aPcojAT^A5Wc2u4 zc6C#&SY}~f>F@07(94rdaIeVD$J*d8#sB*2r|&C=yyzqE?=er|ipM`jrF4mJJ*oYe zcOqE)b=`uzE^+d}m9)92jK5gki3RarKTL@mvK^ep)ZBWxpkWz(z4w|*?Z!^;q2_@4 zs%RX&>FSd#?-wVJ9E-<9R<6H(WuW1+4Z~A8H{IH1`BxtQxBZXSJ9MU83fvKn7zf4An>;d1L!_~`v#qA{y5YW{*-KDEQrj=Q59PvA7q$jJ#j0 z?O}Og63L_O16FFk@;~yN(Gk9*1*dR#O5=)FAs;KPgaeTqI@Qf7DS2-sX#1^$SHIE7 zpEg!M|3>>q+8cY~+Rm+I2X(J{>v~u?FJUe=en9%+x+qEcP5L(QtyACama7B)l|xVN zs1KfOdjDx)mbKMENSME@-hGQKk6=z+M$a0k9?W22pK(v@TkIyjBy@6QmNzb5UVGku zK|p==xs9C{e?RhDt@P8d=I`_DK=s7O6N$p7528#v{V!H}Uz&DX61a2)>3y|#>>v7) z5ATy-mNv{?&bjVQn!hr3?BX+@sFaK5^5yjT%k950x01%@f?clY93Es-Koov*~hF96N5*CrF8)!WvSMl@f06e^;! zGwxRR`4#c|$wxl%d@z11ZVhCd9Q=WlQK>k|`FvM)&Q|fxg?JW!LNvUoB!2pJvL|-c z%DtWOi%V{tAH6}ARIk1|AYQh6K|{RVzEPjKo(_?i#L;{8_gH&XzfKhXR4FdbbG^C_16X^^|wRx$7x<1;8GR1!2GqFFmn z)Ym6){)lj7dSB_KFS;=kOXX~|=~Uk_&FjPuk9?#{um^3vKK^FYKA`x8ugRt!W+H!X8o{E4$ocC4-Jaj-8(&JEOsr{ew6RHasO`4Et5T)6~$vX z2iH8loQ~5;VvGM_>#-RMx?}NY_s^6O^x@VXrDSwe0_B{}zNaRi(+}~*obzhmqT!Lv zwF9_-we~+B{S?1*SoRZMw;75^u~=Wb)|ng=bzF@TBE+ZYBjHqPL~|~)-&45 zU2ve^lm2z+^OwPAJq*G~s)~H%pG}hOkqW5{GS$BaHfLf2X7j z6)+cTM+>SYio><^V&+sY#6i(QReAHj}hfI%^~IN4i3{xJH0VlI+-#zQu0ugnL@fp zb#=XF_o4Ivhdq_CgqTbZh)lI>4VGy=i07*0g!F!1Fo$i!Dq^NKHJ{B{gIlI~obXRQ zCSDq&B73XHclRWDkHiu4gV?Kqxw^}x;68(@T!|k_BIjgt_f;n%LGa*OvN6K+jKO+ zFZvdNDSsepHqnNtX3ONS0x+KGY%8G$ab-dwGV<^reGUB))rD5foeU>1jUPEWS9^-$ z6nVR30;P@YsSbDJbH=0;l}5HkHRTfXJ4^v{hIXCmEZpaF&KU^NIqz26l&Yiv=YayZ z$)t*Yi9Oyj5_L##N~-FY^uDgU*!Q}Y{pfx_nwQG089Z_ijvCOIBVGUI&DW1VjiWV_ z8wqfz$Q1w9;52jrGj$~{up}qpXJm63L zDe?@jroUs%M~8&i>7}yKX$3Kw&t{ zhbx(LMc6xV>PuB8rQWZq+gIvuMx(W1JrhVn&t!5U)^Hjg=*lF+^(pE*UqcMW&(@v6 z)DJ!?nm}>$s3rpgQQOGBVBO0}b0YLeT(?7pzwqMs)SJq6n&@$KIeT!nbRJxu0bWqD zov2M`!OKfnTCWYMh&&vlH)RyEj}3c~Bl}{0*R?jUO85s;GEn3~-i;NzOfavGOAizi zX3%m52=g>4$`u%NlgbKrtaeNFpd)rA?8rSD0-;%D%5oP9nyY%p&M5MzGY5F+&}56& z;8c1zc24=Y?x#7fq4@Ovd(6t%Nb7PUHu8Zas_1Z%%;MDqGIlyrB1>C}K1&~&upV(P z7R8f)#Lg^lCvB2#Q2i-AhC`=SC6P_Ux(s;TF*I}4I}dA4p_^$u(QbeGU^nDWw=Cr* z%cSEQ;o{eI+c^p(R+W{`w8ZdOhF{PJr?852zL-Z$bM5?|T!i>49LaxIWX<0&nz2$S zzWssWQXq;iAFuoy6;C9XOJ$~UxU-VH!L!neHLx0AgM)M7Y?1MU=~iMf0S@n-jjzF7 zc9StbyP4@bLL&5A@a9a+kLqah57>3(<9c48?E zv?~EWiS{@lc?*83YY5)n&mTK8&&`sAE`$vAQT>_DoWd<+F4SUF_b)NMAb+j2>^=h` zLB>$Xo3Uo)2vDOwqa5==nm6CyW?^w>-m!AjiWKaM8T zcmw)jN~MkxdEzB=~CcZwh3@K^} z_D*p8#Z%=a6BV!z3_pWJJ1LDXY^aJ?x}q_UTyffU34sCFUx)JeE`oRY*G1 zk^6}0+fjC{gr3Kq#wlc*Nr+uBPFNRm1Xi5%!7aZbUeb-Fk%O_5HGszudvO@vo|c+0 ze+zy#Pm7}}bQfwpcjG&Hx7AGaZL8yi;TwCm2~zrA*Sz^R`afYQnv_0wd+cU%2zHSM z-5R_$VJT7$FkIFeygH!|^2!mR7qDZ7pFt^lR*^9oQkM~r8^HAq)_Mms^g2T=;6m0# zwAijns6*~0%ttN{I0faonR^zw*}(a3v&U{Dhhp8yJG!1yMSR36JWcbe2FF71?Y&=W`uhf5d(~$XsHh!K zgrT~ej&8N}f^GH!K5_Zyb^C@l$;7cxwgMdg8|&!9823CkYZRakCkC@8&&>i=+<^CYw34}mO5eQlw5>hCnp@*f_Gtt;7w`O2{*Ukr$drB(O3z-TNvu^O2Snd zDLsUtul-M~UnKmv1#cf2f}g@`G_Ta9GwmTye5e~$3okJj!akc#wNu?sXIgPfnBQX_ zXTS$`Lm7j;smmQG5xiw1@)EEGHds6twhmw$fVq?g$qlEyERPMR%{IlyV8o752AKCO1h5&xn0d`(y1`MNiK|A%PvgNHI#COkzRCH#Jz6k&k6GeETFYh%*c z`-ih+n^hrFMIrmpa7+2}z5+ptFUP=n2(8V@oM(3p*T$4n)R34j;pe+bHfE|dvOv`o z=}gs>bmk0hF>{4>hHBF#=8}##zd;d-VRmi7n+y=}vzWX8o7zMFP3;W-q2Lt>7l5m5 zM=sR=H?>!LF>f{nr@6H@pC%SEA2u~7vs;@}y-!BE6Ne>d6w`u($M;!aKom}^iHsZ{dfV%=I28iyTe%*)Ub;T;g%jy zwaBXSjW`|(IBrp7#PK*4nK3ecIf2U9`Jd!&`8T<%n6ieW_xfk>4(<}6s94HO-Db)h z-g)jc$u9$~V@V#A8N6bQC9XLg%`0`nQIouBn)+i}`QaKvH0$Jg<@>QSt3^(g+y%6b$o zI*g^rWilsUp#t!)P3rx%UVVd^V$IsL@S&_^>V$xuD#m8EKSD1A79Hlq^p{Qu~+6 zWX}-ZWR!>G46A^v-o$syLe*VtOq1^Ei9*(AGXw#^eTD9bCJi<;2msLUm|T#5BfIi7Ynv9|Gp1LhJ!gN9HuhLRD@b@&?^x8+wAj2Kh~N zcXtVEGD*Gtt46qwlH4&RI$!SB&**{pArz`=R6g2Qev~|jzey*L1-_-b2&?#qbh4uq z=g1s4sZ8yqNA6(E)ZJeySL%=Rm3yMJ>!P%w8T#AqZzOHfK~Z9)+y~c7F$Gx36n~{m zkE9jLkGiUbSVc_{N!5{PDKBzGb9D!W6UhB~K)`I&Q{w0h%o3EKOQFV?;1Ayl8hI6k z87RUE1p+6vZt(*Xu5&NtihNC;e35H>1Ctpz5GcSjEua) z3YF#IDog{DGlCHg9=>Hp-Zi>voZmKN*%8p2uwv}+*J&f%*RI6@7rXQqN-Rc5h5eIV zC$+J%>NUgPm!DCXj}xaz)p3Oe}h*F*~YL3y5GRr>Tp39GtkE@vR zQ0Nn_A8|0|^!m4Mm_8A;?hWxhwJuZXg(-saR-O9UcA$W5PO2Q;uTQv%FDHsWV;*rR zQToc8Ww?E}`p(y;EP9LT@Wj{V^4qjEZ}@+YI!;*U@n5>C%D(h48;CXBPR^jJ|luS@G`_>5>MY1nxagp`0x%IK(^|9*#*SU6XXuLC%{=*P*Zzamp(fD6|Y(Mgq z&%7|5KgwG*8oQda!%x2_3KK_G#^1Dv!URTP zmPKL2MVV-COUaRZwePGHdZ=rfVgbGemE$HG>T%ax$5`Y1nP-U0gmt7GxfK6;2%)<+ z519aPUPKr@i!?X;+y&9EJZ^bY=J zlaTms8rk3VG^g^he1oP^IfJ^IV$aLOm`DO#&j|reRc6GWbT201g$l|{CaZmD7G~Xt zp;g>s>Kt*FNE%9Uan13T*4|}$%X&hD;Xn|moV(F`0Z{rrF)gG_<%UY-JTf^*xPUX5 z&`+7g45443+$J%*(9OQTYa0%^$YLqy8v+lz$aGhj1Ts&#l#7LY%;|~CAc{{%_XLtd z#Gib6bfQ>Z7ZxQbQZB(51ZKb*AtD{T8BHGddAk~Os|h104e(<9_2w{=&wV+RJud!1 zzwIWP>|50DJb()lt>ca*MZn*wQqClYH>Ag6DwqxHn3;*p(~0ap%4ARGLshbT=x0x| zhFWZgDWsm4t2P$fHJo5F?J$xP&ohvRTyxHw@LSOp`oO>1m)o$G7Zx+yRV$kPuR3{ENEMNhTrM`S6l8M8s8;*|M zPH>?YNfg`qR&E>P%Kk7c9k@)Dd=J%fOs zV2agc;}1Wch~nU;d-lb;(q4?cah4ZU>&!dW7s&%F>$^y@QDwiFT!*xGQdV*rVl+jK zcG5L!39hav;g__;1I2dQB$b7vp4t~{uQQ7ft|a?Na?^acIsu^Y>wrXNav8m42*WLG zIG|Y^A0x40T>2v4DHE@+ZqUUa=fR06?&zja@o*_yyD$+$#8gEQ=$Wd59Ma#C)+AHE zhpzf6as=<5H)}6BY$6C`LagSa(yOCtk8nLXh%tnCv;==9(^{RPZ+8XJox~^Q<93g_ zmL-U5+XuMP(4J;w%|Khy)rqnZ<>ryzSO?l!kSyKS)k8;1{U^l4l|pqdRpzcrlYMMY z1XZvL8}nJw+cShDWpPem<`hq-eyMhpC-xHjtZ?{IpN^{FeiMDR(rm0D zj@A`A>BnF7LjVet(*&7FVMb0$V!|)-Sk~YDGduJrg-J*MN^xzF4wjvyr?MsQC+0@B zYO`c4K2c=aG5&*kktU)iS?9F1}^TFB#abcY>DQpektCtI0vX_pOW1 zc9ZSFz7rHx>fRtpEx~f~K=Z^5-94p_eup@^LW^r`faiA#gmj>O+C!9?;+C$Z+QmJG zxA3ePVu=%nCO{T5+CN6f(?<*X5>6BEGVL)%;_or#zmv4<$PwaH5-*;_Qt0iwnz~kKKa*63 zfFD;76KJ)NkEs$4GgTIthtmpa^zEZw+(n*(oY(D0_K1R_$)pj3nW^}k9NFsbqncwS zdg26+)t`p$8w+)I8}yg<7KJlq_2)@kBA ze61t|Z$6+KpZK1tPHvm2i#B+I>h;@G2?iNw&fna{aT1@U=fffWde*0ohxMuH1#Xk* zyGQTH_G%hY0N11?=+&6X#EsxJs*iS(tc7(V0^XJPP-!LzK<3|Y|WTc-}e&-L>kb=KOK zWgtnd!O4uJ#8kQvfXfS_#|-z>obf$oK|8M$FP)%054R(4b4`|Xe@5#*&#c7PDR%o! z!~NK`Lx2wPGo}CacC|JJL{4&IWDPM#Bvq7;o~^>A-M7}~<_bf{lU}@ggCahpPSO?R zMr)|xcOP~;Nlzio$JdFfL&q)Q%%i2#<2sxHkzHN0?PO6pgvLV733kMEw*W_`_wuF{#=)8zcku|wFZOIKLHJwtXw4U056Zd zg}^&T23nQy57M5cupbC7mXN@3)8#8S2n5J@C)}6xo^qwY7P;AutHFi&SrdcQ!tb^2 zW%Sb)*`754o-r;mYj{8^hUAK-xQI`VTab5*&ZS=KP_M%%x@nqy6nk1S#ss&Kd78Kc zU&lE$WED@h3AKp?lv$=8vyp@lh1X>5uZj?UA=`%jOT*C`R%V&>O-H|EP*fS zgRq{5ocGT{)w1IIP1LSULuNX=GW-h6G-4n&Mw8YUP3~9#CGvSo@agev&pPj{99ci~ z9wcF-j+ptRnv}knT6Hkb8n=5O@Fgu0OVi{lCu?GWXjnkF2_EQkL{El*{}7DC$vwe{ zTaSZ=30}Trw;zi5xosxxH24~x&JV`tQEzh1alc@#HEoUFvZ`0@$Yd3nrBIVR6e(^! z^E8(+N6538f5&zqGka{2=5BY*O0dK@L4MQuJDF*oVfdLehH?=(kaKtr`DbX0G=onO z-WOBX(9)a=?{3qfe%f|8orc>9zS}q$1l5Y)Tx0=YKjDM@%q2URr&;;n!9+uOO6u98 zLA7M2p_UR)r+MKPY25n?j^FAlsKaNAWFX6@;liQ0VN#)({z1|>ffRLC)^M`@rQYSE zX%Y{79Vu=Ig(J$d&q)fXHo7~xbj>+%ju>qU&ny|foC5d9Ce`FBZL|>;a)nymff0B% z`52V$JdN8ioh+wf!kpUJRH<-ao+ZjgrXdA@F)JhF2Q;n6Sd%0n@X4%Tm!!A5)zI@; zZ;m<*6n|9kkB?6wZ)&i%2ZU+-W-FEK@;gL!rlr7()WLpctVP0_H}Ko~5+#MaJu;Qi z=z-Y6Sqm?F%P19PVSM6pwS((vNx_h1ZSFt%G#PZ~5Q|R8l4B6+v~CCTp02;1Ho#UZ z8&J8>uF?2;Q|K*kaoTB+Wb}>Dms?l1{G_^)dy3C5q|VS0DE6=+BU0&wR4ZO%JW#oZ z@Q8Dl>d0eaP?Db_Pm|_xYRa++GUAo-w%Q@WK$AjLQy~i0#5BsSJZtnlk$73O2QiiS^|OOtPf$b|5}rk>Q9x1`6|qg}+bE=nM;8GY z|A7mvze}CUQ=V>{$-A4d4^e@v$2&Lq)_Dhmm0JP6PgW%T1acIznpDJ`k5NQ6$kszz zGxFKyl8xfi^kD2V&1vOP&7F>L)c6VTF4;@brwJz`=V|VOrzG4BjhT&rUwN(2v*S=L zX}R#WR8_Y$4ycMp9karjIreYzylMoI80Z)L7lm* zjMJ1JbCHRl*+(aiX2^JxMgsb#)#Oz0>9xoT?J?CR87sGVyjV6(v!HKVt!TI3QSw|} zR)P|VX|%KQtR?QooAGWaLk0Z^4pLeYG6q{#Op?R4A zEJFY+o^kzCC4tzL8Wk{<_=>o*iUMf1q5yh1RjOotXyn=Wj)YpZD7FYG7#qFeNZkk- zejTdlV~D?Zk2rNg7XcTbGz(P|4GPx`;?E%XSqdAwz)ORooQ1CyUW=Bx8H(uJiTFdQ zb~1J$+e%-M5xx>IWg}n;C!6FbPGvoCB`x@>&N5`Xywx@t7`wghK3up5edh)Gdqy}v z1aOXC!a7HvFBH)4g1?_#QOT(jT_s5QVR&oac?kSLX$VB#s=$}XlQFG z+)x4!@K+gOe&#F>e2pRuZ|QfR>ge~q5Pa#^f}Ewq72GNGP1rdaR2IiEG7WbZjPjE` z;m8R{7Zw!HsIO(&YA4sT{6ery5A`faZIF_xCako+J(;O&8M+ddHa#a&?WFos;|`jQ zjXBnwI>JP8+ssIUG4>B3_Bq+=wKqB@5bIXkv-~{jD`cuR05~5m$nO|=mMgR|{Nk7c zMX7Qrv=%4pHuqdi&5J8F?1N z(KU@x=r&CfIdo$c##GNs_#LUv5;{e8`#=j2T|j+_IBMssyvV#pZ)=i(vzy`VGQOlI zDb+&C55Z5RRTxWWc&lxiA2RiJ^4TcVIVbsv2{wI5n(b2y^3|fwEL@@MEE8 zV<0GNU;2Q9uXsp(kF0>6%spgJV}ljGhCtrYKb821`VaCwwGeWXq03(ZG@0ubE!J9<>o#YckcBI__CwPs# zpLUo$8AQo;@K%1r zE`}5HtQ-g;>k+jcan?5Uxz{9d`rv2|v|{KNJXWp|I3c+$&B+bfB&UI-0*NAsfu?1D zp7$$xsxBXTFwZia)?jStKV!FPZu=fzuto_`1pI7Tv5}s91o%+wGQbCK zD;QA`qsg*Gnn7@=!xQ|%tT_UF5Z-p+u;FuB0DY$3rJ>vv-rz+6d@wxFD>3H!f`nPf zGvzixGvd^m+2fNfbclpv!#@PjD(1;XITyhef* zrxptRWa1raGqRTIp(3{rkDqTGw?YBQg+xAMOl0Vur4b7ZF{Z{tA}{JGO_j^W3)xfo z_|dDfIk1%qYFVL15_lF!AR!w_;P|Q$CxGeo#vrG)*q-C8yKe;PClKYb5Mz|V?$Zny zK^h;1H=~_4f^=z1^I7y=>KnvHYogK|98TO$!D)RcGiWy;m7B8-sjT0Gou)ZwL}_{f z^ZW6C<9r|Dyrt(&DnS-s$lhrL3f*Y{iiRxpi`xJcl0vtsq^0Op7!&$-fap~{U8!y| zdI%KBx|NlZx(TH-2#eX$W}tr2uihA1jOP@ERBIlaJ6Zn&)JV!$1$<>?UqzYA$dkT7 z0qXBq17LVyO5)B6Lk%>9cM~w$HVf~-Ji{qqI%7CgF*2We6KEfC2@&v_RRSIR$e!HP zGFf;*!*uny?SBKUd-&f#1DE(upqu9!lFgOJ0k#cW)FJW=`VrHS>efUc!$U?v@;1ekp)>K2k{x=Lsto5 zkQyQ;k!R>^RSTj0VyJUaD(2F7%5{S20G7eY`t8P`-iDyUb%CH7vzL&io(~j57Y^`1 zCm7)&3*ez(Ot-NES*T_7Z4v z5StL77m6%W9qV^^umvkFnb18|`e`iSb?XbDc&!K4djaq3CzQ zhRuW%Tc^^T>Q_f+sRd5(D7whaioR!bs#d79hiQ+62L5x_^F;qt{tqzqV>w32b*5TF z*X;?^M1oi}F&7whA@T+goAi)ssyjfLPI~}|aywhi%Mcx?K*PnHB3qq}a&SEZHpxoZ z7CzK#m}&Y4knH!B4w}orxY7Z({Q*;Q%31tm$zA4BU_ytH@f;9ksQd;!2sHu}BVngj zK$pWP$GDMv;AF78sdOiIg7}CjBc0K2gFn`^S#$B2#6!3MPV3myGKwo)_Dx2;A*@B<10 znD!K*m*eCT#6wgC<;f<*Xh5(`;-EZx7Nq;&v87226}4) zB^EM8l6)2$Z-Be()LQ+;kmu7L;!01;f(697&XsfZ1en#wpqF3fEFa~x0fKWU}@ z)~awR)1ToK8=2raHi)CkMqn2fxS2^dk3N&_$q0r58fh$`-k@@5*Np{q85B^)5(5)N z6*ZK_^XYgBU$#M0(l<>2`*0uGipK_;f?mq19RoB?END$g1z;#Jt6~KNGZq}@Qs99=zYRRlA2MB^X9V`80PKfd z*-$#H$=hHnngh{R99uH3JqQ%VtzY48Q&ScS{DJ{w@OHL=ZsYC&2L1g4oU@6R%rm0a z%qU^R*w7?v&pP5RsHpE_JyJzd7!&zlR7Hzbk;g~I^NoZs1BP8L84KtfR-11f=T>Db z$yrq6|; zJ9#$BV3h=b8N0sG-)iadtg4M_QplS*n!0`mH3t|j8enGZI^SLhut_^@n zZi%a+h9=oeVaX zf|sC%!k4A(2iE6HVEpN{B;SzA+J&Il`9Uz7Tn+o~HDEd`*uVy1FieTy{$(iy02O~D zsCb4zfa`a|Nh)eQgBhM;Mfg%9=`IA#4u28{G&`%%ZMr02)VU)u5Kt@35Ln;@C}{aE zf{~ITuuc3sv;qz=6xs!)xE6{jEe~+yM3w=zCHEn?vwFIa!`<67#ZXKYA%0P08?d%5 z5y=vw7uSj(fDP3IH;SylegU!#uy?dPek&iB4|7LYBiUx;yP4=W8M$^8;4|^d6u>=U z4eR*$T+Pi!IfR#W3YK$?Y?~Di%XdcENIX7%ZR2iUw6H?es>5KU zjI9h3bQv(hAQBA9@dwZ}7Y*9{EczaEm3Co|0VvWy(?H|Hzy+`L4bGaN zzIF7mEJ))FJp|7stQ8nDmSNr#Yt%Fi`Ru7`9tYwx2S(fvA#1WZ&^=&)^(5J0fQbcT z6FvYFaMN%D8B7M4s5X*8FtlcC*Eh1EcYu$D8P1=LT+rc|-W2yQ+5QMi{w`x}96hE# zv7fKsVFcxKhM#XV*2YD!)E66L{H7B|wyzqAVNnMx~h`$PqvrH%rYK zMEN$2{vHOu2usi_FjMG%!qR&Q+91&Xf~DHSz_{skumr~s94CAgOh5)2!z?r`ql}8i zvwT9>G@%3ty4E1=hw*##ZX*{t16<^R(RcL@ z!%u_#_8+ScL@&s67<@emo|rgbCltXpLlF!!6hU4w@NnD`v^Qp&WCJu1!i_4Oe~)RR zU0`&ZstVbbYStq|5!84CGN)yM8}fEQjcw^lqX&+jmd(?M!Qn$0J!~EeI7gNNml0E# zdiD(RL*XL+S57TkxRMst{JsSlMdf}|7U6PqBZ(>%~;(F;%lkm9pxSCuO@ z70M~v9jg0Mf;%f!(haPL2^W%{N1d#@1;D)-u7m!fG88U|i^0oBQLBUF_-nYP{C~dD zKdZ*W{#BEGaBU<0743!$D$T+>GE<3$E56PcaQZh`MrC_~{UrGW%hg#VdvUTv2Frjf zwfPes99<2G4=$I%;hP6p0}9PqXHgc3BtnO!>sW-;^9@XkbXGoY_!cegXFeQvqfV!o z>dv+cYxo6JC)XBXHf3nMS~jv8sRp48ZMqtZ?jX&4$}C<8Gnq9{SnCPXA)6@?UaBShOqJ}y z?$vN0n@O1s(%d4q$*}PkQt1>&zl|Vf9yGI;(Q{Myev1N3f6PcXEAqYdtL zgi+3nLZKcqmPMEeG1dKxB!q&7@D_z=sbV$e2ZINdX)KCP1~WC!h$5%JgJLXY0f*+} zfp;_;c?UWOngvf2JH5Em=s&_hc7Oq|Ury+_2_^_C8|CPl3OjR&Hn=A$)lEtQ3QhdB z!`SSVfR^E97@NHZAg3jp4DJY%1G!!nW^hM%jp%0?f(c%!(NA3iUu*I=<})zLDcW!& zuLIPW3%c4MHNY-`)sK@VuY=PD=SgFii_$kmIG1Nmta|SZV z;PGVutNYg)F@xw52?i$>XuP$Cm}Te;po?4;KXvG{4V?irn>nN7%@+Xd**YxLLm*;% z$(BIw%^@HL&e06r7}6u^Ys6A}MAaglX9Q_n2;Pa8VnB|(3Ecufjw*KpYv%9320=C$ z%xcbVyrnC_KbxWqu|_yPb3wvjXQ z8(=9nDinGREOAD{q5A_aiefkK2zicfy#WXw7|@wJ09bKNDb|2hjWYm?eu>fJtA*d| zzx@Evcblf@|KaJnndh0kJDLCsj$ICOa3P`WI7smZ zn+U_bBQhWi6cDikb76ha6a-8*q;K5}*sM-HVj{SuX}2LRxFIN-+5@TtDE?E_638?V zIlB5|xG-|*ue}n9e}gGpB7_e42ILXn0Yh2{Ridqh;nEf-H=#wj>)}28g~hh-<+_ zn6lvxGlQ_-i~(M_5OA9L0SjeMJQ~eOh%GP@ae(d;Qlo-(O4c~4E%KDh zoOZ4fh=FRcE;o^^?48Y=pAx~aEtMCEgetPPVvZcSLXh$71G0y%0aTA-8i2f`8_*{SG!ESQO2g*j1#^IVScJ%V3#578@w zy&dYed>YZ&e7;@e=Fu>wgzYx6AdHU*0g;L2i)6yFop1>T#bxo!NH-WKrRs>SEj|LC zUhtNJZ40|2$BqI8R{}isC@W+!2@rxb$DW-X$O_SY7v4(b2=@l71kiFWM_C~^fu~V+ zJeI+N6T-nE{PVG$@CJ5*di~h3O9Y@t&#|2V{ir~O9}`=bKsE??VE+>3$iODIyBwpa z@DBi7Ldg+wFM|MZ$`Zh+&|?_&8Q`_-4hw+iCxc@K@pg6D(NNZ|%%g337h!kCIU@^a zI%E}7pI+U1SPf+w)OZlW9^n3oWD+q05%uV0TNj7 zAR&kP5s->6z+epmq~c-7KRaA@w-tag)1nLmbB~}^!cB`adB;2qSS|91n5#lLnjqT& z5mTe-1C-wNhoM~N1O)K;z&t|=^UgID&2#|!0RkgIK)(QlgXHCtPe(X+1YJYFMX+g; zVgaQ$iX(gn+a*UFLHL_-e9i!Y?>JxqY@r~`_*m&CN&sJzXuOdMaMp?n8D{80!;yIg zxgAf#pi3a?;Q|au5X@*V7N>Z$cn9VfLKr#KTrw#S~GEYPxxZ*?{?6F3HU zj;Pxsf9hzO0)QnSiJxZy&YUleX$LIOnKgvzt^@44a};1lRA451n%YXxHb9T4h8rfJ z=&H-m!KZfO0FupaYw;hvL7NF;BK%A#Sq2-t)F2CZT8nm!-!eVoCP0*iyLo^*7%88C zKP~e=0=Ehfxb$-lu^`)ao7A_m8+=UPf-X}A?*9>eivc|nKFl2CqN2@8EgZ+UsQu4m zr-IGko5v=*9BiTrj!ZT=s2&p~lfLVJ1Z>NIZIOy&pkr=FgGi7+a;Oobs{vRZ`TQeX z8v`~`H;<8yC{P9AIC`N3gUcWlM^MK{IJFZLQ9O>tL8Kr^GT}Lz(*!9{P^P--XjYsE zvjp_1)v?oY_9*KNtmbVx<)RO!MegHR+|DW~Sk+nWcv`do$P|ESj-o^!V5~)sDO|vv zxncE=8G}(9z!;PtN8*uSSU)`u#95Cp?CK*^qR~(V+)TS;pTh#Miy-Vh3d4T^J_c2O z%*UAiKa>O_WG+&=$dR~0z-==p?H`+x_NDdQaYFX-v^Zr6+`K2D71c(P7knv!34#*h_KSe=;0k(VO*}XrW8d;9;JI9X)a^Z9#%*dD~)~Ty& z5RuVJ?GsZL!cijUj}SZRO?*bQNa2xs$^P|_D3B!v_%1Tu2SE!$T_&#@Pu^nsk9 z)*QQYcK_ffC@II7Xy!P3i(Rwzu6(-|k$nP=rCkw8KT1HL4z4m_9ppp+PM5!t3TAbi zV`j!`I4uUO`t2jqhC2alddl`_00X>ZiAdkA#sc-592$%gscs6!0iPodtm-`ADE57W3yohygHs^v*9_E_6($B0+0hh4Huh=j}s8gfDa#)d4vbx@MY+O zGb6`P9Z4S;jZ4Q+9WV=ml!hv5G>#9Zm#5Nn8ro$Fz!5^v5r1g6+%!haQd>OKuofa2 zE-^Yz5GuF=A<>Ad!ZFn{Nv&#`rC5jVK_cIVcDPIk4Klk4WYRmwh)6=)lG5w30kcTc z{fL4Ic8INTtmo_*m~CEUz7&Y=;tWNWbk4W$?!TI+f&UF_oW$C7FB9`S1&H}QV6k$y z15p4Z!J31-Fx79t4^dOHyRC$b0~~uN9Wie1Uc;_0!A}sm%mCI2ZJ+rCY1&f*Y4H_k zXBuSRelz-LwTA^ms4z&B1Nr43J^7BvqX^u0-A|STZS15w$n5U%71A9G6O0upr2H>` z_nZDv{;uZG=G)&hPP!pUgpIDSY2u1&Gsq|(LyR`IH%~p(#re%Guh=13fhT%!VZ`*C zdXu)*GBbwaJQD@oVNYOK)2Ui-@jOTPPJ7t>6H95ooKKv;s)K10S$U98hafn-^(2k5 z_aBN-o&&13bQYuyI1_?Cu~@8BkDdX^t7UEDz=d%Q0nhjhv;_hl2x`-YVgteT9|Y&s zrA17boy<5ye(xJ)Kn)&h41g$tGctIHGa3b)rQll@Ajxdw49RKZ92vjC8(y+MLu3Yo zjtIz-p2{i(E9D@pBDToW7|N&M!z}FNPo3(PXWeQEd6~ZlFN8j|!@*%e7(LmGU|7k4f1{KAW^(r+f`Xdb+zfeiJHKW} z(`&V7%*%xxUCALl{BzT)E#b*q8JxjEcNQ_q>o(+;B+lUb`7EL{xjBQwr?&7BBcW$c z2W%~8+Fcv1P-TQX>sf?J1x8povvx4}Qf@Os@@LjLbYIUHbLhs;*fLsN*q{#gZ(p)# zL;F&~88Nf4lxTs~D#x}Y%Fj)gHk{xkRWiIhg~~yB2Y!&rGa($52C(Hr!iY1fk$t~KL)?cDU@M?58sa*OiiV)q zxfu}|Gu|AO#-q#5qccXIu3Nu zBqz}8{EUQ(91D~%qsR|Xkx1XVEh7foa2}`T-=x>Jsa~bP+~Mgzp@3sn7nZ?U_-i}2 z<!^rVY8Eo8ZMjJ$YSzHf_*3|1VPE(&$aersp%4B+0Ug^_}Ew zzJFr4ng-o@wWogU_!<)!<&18=C~jnm1k;WewBXhFb#M0dR62Rm&TmUV9;a{40xE}r z%3%%??VM6gp>*wOl1*O{ab+AnbN<$hs6r*E3&$F6Kiw2I5E^UON&dwGnh>67*R|v0 z2CIwCs=fzHhz~f-mLmI9q4p~8HpTQLWe^{;V-Sqc8A3vqFFP94`y8*A8f`bSNzU>+ zZk9v_{s+Sw#OtJClZt85@o~%(8Kv0=g5i*UYL-m1=6~q{@}SH^LJ$ z{LahVXsgi>R>G|dTBQ5N%oy7gY+F<^rBOL_^Os%DFYnz_+L6b77KEm3<6&{p_yFx4 zZ)1x@nPwsT8$06#bF*I6*d6o?tW= zl!NnNBoHrX+qP{2Xd|Z=vkqgdo@^yv-)sKPnr)%ifNGR33-A0 z%dz(D=QWfDhj4Z~QG2f$vav5c6ULXPG~>;pl)fIpQl0GE^O)PywmWp67iHL%rt7|L zRuk=j@E|uvm-uYbSR zgp|v^;TO!iH@{mHerYS6xXhw6#e6s^&Y0h_*V9I{YqR&M?#1*}Tmz3aIfEl_v~N4hx3Kwg+;2nhfA}Q&x6E2$|Ee0Luz&lP z98R7kOlsmm5_o7N`tp@}Af2Qhsfh(1IZ5zi4`Fh0_7;ih6PcU2JJGX!0U_9koHK&) z;4x{w%VKL&lG-nqt)w>vD62F5b2f^bn`G#rdLaC3grFuQ!{X`sW`nK=WBn69#Ccr3 z9|f#0ytHyC8F;9WHYczH=jd>LGxcdVt$({_75rG;;TuD|wTqp%aOAVkFUly`#$@^y zLb`!%rSO=L9_Quja|mO0-&mV8{6?0yb%HTj;c|a(P<*^+SuZXc(zd;rzN$iQFKvaD z;n*Rpzn~eug$PTjiK!`L(kG<`Ncx-r#j&k=mi+4;DTqbEZcZN8qmBsRjhMEjp8?+ll-OJ@bKHHS9-RCX@Iq2Vyvn->$sb792o%ue>2IMdonTkzzqC zM=0ve%$S7L6TQot!v=Qn3LkSR%p4pL2T5g7X{BHQ^=y|ysE zDV;20D_Q$ph_|v&FCNU%{FF&`M(8~3eY6bUG8>0ac&V06TLK}lf@964G}(7XYhD8! zM2E*lAB?IQZ=6+Z2`Lk3fX|-yD3CWi0+UPj1HaMo^JOl8u4q- zK`Q*aC8Di!F`1~^HzkX6PRw3z8xmop{`(*kRIu!w8f_@BeZ9?9y(%^Y`Zbz;o~*$E zmqmLH#Z=Yrapc__;$#f=ZTfgVuvsyS_&q*U9LMh4R1S74Gl??lh=gcJ`}%xi|Mpk? zHalOSPM5CrKcTY7=>w;kB|SW%DH`vj9tpRSYMS_?CSZ`rZ*U_ox}NJIh40?%(Ig;- zGnU|;6rXXl(CH1!IMs#WQP#fErXy&Px`)50ZP*5##wTsK!F7EUQ;Ax~JtoJP-@aGj zt`h?JBMw{$gcmQRA*hw44xf~9Zc7M(Rz!RH!Btp5zsYM)5{+OEJo0D{MySRQVxqD? zhk#y>aVqi86kpyTulUX4l)_+J)eo)oU5!QdfN2MUWX=ru3c1dqY4CoxEbaTl!! z=Cu4va%sL2Rhyk8X#=&Med3|DB1hhzwgvF`;bGa)!Tyd}#3se~wPd14Vz3EpVB~Yo zRXHX2BwAr+qUtsu9v?gnJcH3v7~`l$-Z`AGZuM))9%}6GvM*$pkgya#IK?Pqy)fKS zdL|rlCovxC8kqNyT?x&1ks4}%?;(@PW(ALxAW&P$%>WH|;0=)HE>anF5mGbci95v# zc=g?|i+zV+vW~;+CBz34fv5Jj>7Qy3S)2N6xj1}g^}J16PxU+=l|zOGU>#h z?CX?kZHLRL;JGu*aEa-&NIH5x0(jw@9sLCJR9`Rd9*i|3W(F1$eXtw#m?aIvp_`ts z_BeBo6|o(poY%HDgJM&hlEXQL!_5vgYn$CU@;rM5OkrrHaF>c+MDpmk`7I>hz7cHi z9ui~Qmf^w&*177FbFSRs-OE$n$Oaw*u-*q_YNqO0`I5LiwyM%z%&w=MQNw5UZ@1bG zHyJPb^m-Y?RxZFN&D%DUE^3!a)#l`6_WN(xU+XD5ErI4VxDmjs(G8Ap)JJEg)q>Mt zG^T^mc*fv{DX&KEz1edPV4VEpx4$tt((msPo#Hyl-FPT%413bex6;hv+%=aKlVT-g zkhqtvOgNOjjF_uWb%N^mLo=e~M-9L1n!tV^hBq5`*mQt--1|vv{R(48a2}fcXw%sk zfmD)zl3#>+vnV2iLlO6yVT^>?2AejA20@#)Fg0QTD~B+8J)E(6rG56S3G5_ZDV0bg zSe?GQ^mEqQm~^+#MG@Bp94p_L9$oJn;bBB-H~3~mnh{jWB9bP}Zp1`GywhJrE8!aqGY9()f*GON zzwq*sO$+y8m;EW)-Dng?o=zNYVv<*;M>|J`}n)T+(XkHU6G z7uZ_g89~Phi-NrwsR?xkPuBKo6Vm-Y2yo?)(I~f^o-|^jfi6lCCm+8Ve2=8Vfqc1H z9|JrBqCsCZSz()pb;6mknjD#=CE2vwFHko=>Fcw`u9l?m{~^ zDqvK3oZ_&f?(U`z7Nu)~bxDatpN72wVL5h=7G4!;cN&%$IeHqF5*c=q{E!BEhKQ$c zvT2d0cl_H*%peaGWOuVZbZM*Uk_(>6`AfFfhViMWUHu6_GBYSZ`Y0+buyfBH~yf z+B{KgE-kzuQofRgEKZ}!O!`SHERTIbD=dT6qt|wBQ947sPJ^;;NjpitNMpf8?4h5K z!?vGv#^B#|-;+)V`R= zQbt&DWYSq6q=*!Dm{!-m#p|TDQ;QBQJRwqznZyg!=35b2&jgE&tY;+MrWvIV&xs5^ z4J(NZW@<2R$so{@>S!Nx6PHhF+qT%yByswF2aO_a)8?JS&d_Tcw}AG@VeioTa1jNL zL`0m1eU0>F20df(A5ohIW!d6+nsl88!$ZtI*>tl-h7okvlmgbB7XB(Sinw&% zqX&vCJq^oHjETFC7UZ5+ z=)h+;$nS$>-P{$KP?-VGq>h2kHHR4if-JB%syWcM(W@NoMSNxRF@lVDYLO)x~J) zRiHLFm}}&61svpCYy}0?#C4*<#i2!Jqc<*Ra7l`)GE!G6`4lv#fty6>d|qlN>h(orI`RK*Do>zJCJ8v=fD}66k+on z&O_dTMIbV}LT2AQ$HxhnT3#tp1 z92f;>T%xvuKn+=tc;Oto zRS9;K#C3o+Ku48=S|KmCAfP%CJ|PNRs4_2MYt@Y^0+fdb%A0zzB?PWYOF^cA z5(Y&_%5h2BastO(Q}TeWbvPD}^kF9&p$3GIGQx<0=EiU7fMa;k2qkcg&uGdJG=zU( z7l>pis&iD)Ec+D0F7uCF@qlR zO355usnG_?)QLoSsSO6utJPqUn&A3~n z0?q^^!&AX6Sc;A`P%3t)1dg39`UiGoPz?LBU{AtGN%|nkYEQxmRME`7SPB{wEPluW zs<17}2ZBvTZXJzZ&7v%z%Te(|L2$;<2x75AS4n24n&0|&AsYiR?fOtgSI&JHxUsIn zi~KNVvHuBo!Q_ifz3?|5Gc8lgf3k*Pt>0E|bx!e6s9F4~=7ZC7oi(xu&0W zKeF#>a<+G4J%w9c7v0(rzkI-R`TNDYUwxVdy<<;6e3U6vT!_-y_zeywm1A{OxI?7roVCP%q!zp%r?PQrQ9_g1$)-s z_3-oUu8~7frPaa@S6FCroo_$a3~|eyY*sM%EEP6ZE%O<@t;ZU{_1z13fAn$k5I%my zOMpf;FRA!h$$B-y>&dfF9$woZs`^^@LsSfGY%B47I>3f}9(PC`h|BbSYUn+NA{b5v{Ugja5Z9_g=preDr)~?ACOH0Q`{*asDgqYiX&l zFUkS$Bv-~pZx z-5!Ps>Fs;>@K67#a@!u8?%#iG6%n-RohHOn%eYq;s!~>PL0{hT_w{zFkn4F*0oONu zx1FzKjy9Aqd6wN(nBnbt_@LJ_ma5f#+vH9n%iSI41chMY)3P~*Gx2i|7+qU=yr&f~ z0hx%8;+f-F|MmE~eT%DDo>fq)iC@Jy2d>{LIAtc9dF!!e-*2|vlP+J-DI*s>-?eiM z7wv1a+2u5Y!ou;TuG2o(o(d(7 z$+ud9zP-5J^6*CF$KUx5Mu|cq?OH6ng{zYxPn2_Y;jeDb_gjQ6I^2$Ql(AqKl4h(3+ZWxzVtR zRSM4gicOXP)z(QqZNd1TBB%G52%lQ3*CX+=0PwB$K0;iL>v}r-_c8p3s=d!_!B@XI z8_~|A`Dp`^M|S?yN$Y@D2o^3mFI~xX-1<{+i0g;<4Zy#jfnm+}|LXO={#HLZA3m-h zFDlP=&?eb?W3=Z%68GM>7enfw#$F~;+6rzm{2R2dv;CZ%))+W^c+uW<^vO_O)w%U2 zs}?I0JVK9(!^bM|POqBNi)SzVmus{Cs@RQVggsc+%WP6qcy?Sun{Nz~fHnExKLc`33#mWF4>IF8il-FRtCMH^1M_pm-}3jGcXwci0-oN2%=ltHmV9DOeb24t{$(ZGlv~8_W>y)*-SdI4 z`>1fluA_k`=O|6YVDWXO?c2KBA71f)g0+pHId6;jF>z_j{$=03z}yY}U7+r7`u*yl zE$m~p2p*HHbjRiu>(FPNUdfzC!}4gqwB+|JU5+$-1KFOP+{2y!si{7*kCcB#KOro6 z*UxttpYe*fnjgJ_WY}zV*6>PFM{%-W4WA)_>#=`Qpa$V#5 z`+HKu39PAs4b0!XDc3k_D|mZO!s)MX_^~uq+k0Fsddwx|zw88$7%o$6)2iYV*-Dg2 zxjG)rk=WcJ*-LjiT(9-c%^2psmWxvwmv7FmKC>RZrEocJM_@?&qxj~Hz3>mcDv@8I zu_jT)0@kt<7Yrw*MO^eCi;Ax`o~+HFAf1g-vAfTuqTmA}aryR~gl1d6%=zAc6nFZ= zBFKwdk2^?7HA9Z33S3)haJ4UP{+SvK&$QB5ycq(e1;k!r21eX}i=Hzap3f@7?p0@x zSpC*sI_J>D`HUbFoX33rL>#ehm+|~vf+!Qx^WOREdQ5R)9O>S+D+y=AMLPROJSH9t zYL*3=%YBg|jH*FKMZOH+-ddmRqrNw@vlahQg59sb!hSPW;?=*-Vkh=5kqic|nZ>NL zRG~4p2x2nQq+}$vIdW*;znRWi!|IK5qL*g5eCV~b3_<(o>KN|e680LF`=-g6S6?eS zJZT9CzDzN@75C)niG}uqzEyVPB(*O^&HfSIgNn~Y3I5BaGZ572oKx?%WG%jIIlYE^ z+ZB}ueel}8P+n&FZy=_?atrEsTSPj(IBSlpc>HSOE#J@fIY>T^a}Sbh*qi2Gc3*gz z7smUjl8IHO{(`13uess(<(l2$3;h;JmQ0~xP`Vo>YE4vO3S8;sR^9Bt{(odvZt(f7 z&AO{_v+_l8mt89#o#JOFcnx97W9OX+@tu_zdQ(rb=9Tpufww4Pl|d(T5`Ukf{#0HV zfW|FFLA49d4bR3uv^Fd6d1TOP`!=k_#R#q=Ia(;Hk)LEFdk_(M?Po(s;(Igpub66! z#r>z96Z`(*)U|dZWfV`^-lN+-fF*uN!bMY~@BLQAugq4ecYQV_bYE=Ee!F5{=&b$x zz5VI5oZYKj?oKEQopOuScX>IQa^I3C6L!vL6Iee87buuxhTw%SnJjRY>`OIUA?JET z-o7h%fX{uGsq4!&_7@4c=`6dP#M;tgFr}3(T*>YZVv=OEZ)MS3i1dL-Em9<{TG|QocdxJZ znX2}>0=b2DTIRh4e@fwLmJ@fsc)bBU>S*>*?Zn%sL0<DN_P{zD8S`2X-cUT7&3 zYrS*r+nAG%jh()*TLERfn0SkKb;SJh^LTc3o_TqTgpv`LPrtR;pN(GCPtSjM^^3Y1J>O%(95P*)__xx>rl z=5YHRF1t)&$K7{*tv#--jvoSg*}q#!*VpU^?$cT4KjG|K64ZR~K|TLMyBhs)Wp#+^ z6}theVQH0v?f~p+zwDy~X5m0HZ@rWX3@+#chMNXY8bqu3!$uDKaJu1%vZ?e5_)bx-o17DOtJyZ zpvU1Aq3nj}(+mzuwnBBZ_j~!7Aa@R8?{e)z7Vvq$WG*f%NR0keD)#`qT z^}CW(MW}GB*3bRIqxR|?y^6B z*RCbJ@A!H6Y-EfQgp~prk+{NEK}F58j`|QS1F)Pv|qIcb`r@l-)Z) zpIOB)wa`T0B)&gsj`UY~%U&uQ?`v)qe@-vvLe8eyd{fe=t1qgxwcdVP)MqtmVG5Mi z);(d08m``Emi==|M~-FIkC29(XU|y}^LF#ue%Sx~hW>rq;`CZqmhYU-DJ#a4#p7#H zWX1b_)-&)=-==ij0`B-8#uC|tw8wThl2Xq%cZ!_hs_Zyd;#{NhiLLZy<7x_H(|8G7 zroFy7kmowjFJ?+g)2sbm;Gzo(R`IWtit#<^-CvgcXe<&WU|w;RfyubQYjotA;b{En zzzsg6(8F)6%w88(lC^btH!kyEgrDO3lC}H&>RgQJqx&H`(DYFQd=qxVANya?ne0w2 z>?h!Q_nE%)z9nt3_JcoM(0cWqz0to@%%=-Ng&VWy*K$0sK=A!+*KA79Cz{kbqHxm;(VYfi~KID9JDE8ZQ+yh)d9t-@;-d@6~im&k9w{c zGmO>pPZ`+1-x#*j-wZgOQc||_pM5$Xp4`)6Rr>9$MBaZLmS;Xtc?Ma{`llY~eDu*E z=-t__t~tHr@jMKPe(5{Q#hCBbIR*$Wu|(bPW<5;n^DLAO8{?=*W=UI zwd%;K<-b59ch*LJsd~Fm&n&7e@2R}d+HNO=vZpmt)qan*bQck(RILwX-^(jdQH0%o zygAw-|MF|Ly-w(Br~W{{iAaKLR7<1%NF(0vYV!0_)|C*ymR%OrLxuk&e-JOLx&9!A zaaTQgpn0nIsnEM!C3T8d`}&lV8=8FsHu1EvIz;XoZZ3)Dwh>@KYPEl1=Rg**}Z0c~R*#8i<}DuzY%II*ahHog}YEWSF+vN!3m7 zHCbPbOYRGsvX<3lou|HsMk3B^Br(mGeu=0Xzw+5jd^=$L@mM3vb1}bP;eRV%etzUn zOR4>yk2Y6*WdADQddU-IPA{i4_~h#AxaFiC#hSg`0%1}4kHfp7IiKIIq}`q|faOo(Z-j(PlfGYlpOY6l{_5P%sj9w~EdfIE zVeSPN`o&%4ta-r`&yu7Gamh`B1FK}30mX0auc7hbz z^%wv5gYLY2qQ${R@a5$c=v^O`REvL{uhD+ykG^`bMz5WK8iJ3C2~PJ`COf_1r7zs2 zM=&V788eRi%G1Wh{#Afi_JWc@`2~>#`S))gZbCjQJQ+~`5gVjj_ub2DEJpZ#5)%7l zzBp#bAxD^6z}1-M^{%A$q_o5biRNoT7Go9;k3^NAX3DbX0SLSHd-?C3rwy_l;iafa zi^-UEk+4B_${y?61ktiAzFR>(SkaU{nv`kS!R0FQ*!b(tMX|$s>y|%>aZ{^qluzWZ zsLgx{GG)`dFQ4b>)a$ACLr7^3PyfWpvyM_fU%TpFk`4FoD5$%7WyVYIAo%^EsF%r? zxTiV9jnrpztc}s+D9VUexGamN%0%8Zt5FJBeNU*9Dha-nhKfi@fgg+@&@1xYhudKH60geDj1X=BqzGXSwpA^r@Hd8kqJbEhv22 z)N7Q$&Av%(cx}ht%w-fr=-WLxeH$3WJAaEB1@`rejS$<2d1zq%RutySgd)&&rGt$C(qZX zcRP&xC=@J;KlM-c<61s;e5wnP*+pCII`4Gz_Ph7bDT|ZP0$pjdOg!Bkav=3*6iJqj z&*GmP+TvrgaPyxjXAC71ji`_Hs$Tez%lPM`!nZf}LgvM)Ar)H2`MdS5RLmf4GtDB$ zyK#1bnacjf8H-!{8l4C$(Ze(&&r`)}O64_vn zzX~zFdv8WKg00`nchw1{@^b85aZ$W}dhe(hM$~L3^fHakAR2r#!rp_{Vux8&t2IYT-lrU z*+d!eR5>6C=7+{s=8B3NTIzJNP zTSUuK6!j-Qs{Z!AY(C018`FQSUkk{N9#8o~1#+?Sqw2c2mL!K468DxQ z5#)Z$&rZo6qqw&8>&f1>*~Z!6H@P~uBSs)`m%lxV`K|PDYDtOr)*o`r!bsvXD)rSD zIOA5)Z6oTV4%uxNvHGo9=%>q~k~JA?F3QM-6xdJ3!?tss5L08C&VU4g=XGpq>))h1 zTK@<-dM@&u#nnae^wduDXR2&lJU7`E5(r237FaQ8niY0@D?Fd(QFRXC6mZ{ZW~Np` zPwlpe@1MMs9rKbokppq`u0W%##Om$Xor5;{;j-o7&UKH2A!w(u&j`v#-nQ zVjQV=9|~H17v#;ob&)IW;eqw2#Qgf2e07&#=*`?l8x(E-4*THJP+7qK9iI<9{UhHx zezL808adPxLsk3}HN2Kw9+*Ny@ezkDZk;D4+=MllB;DACP3s?V&eZzR*?6ZH*Vgpl zz4#61R&=75Ib(H2jdRByw>RGuxEOnFC3pBE;;Zm8+(!?UFZB^iY{T>l@8-RhEq@vA zx4s(BR~;y9hn_-NKl4(UOlrKnEA=ahZ!-dMmCupf8tkdMk@^uao6H z?H>oM^lBffBtL!|y~aXg?qg=JV@=lsu@|x$7aF@7A$l(5!d5;bU5#0m)JYBI9o<%$ zn;O$$(=3N$4&4c{&;4e~-15(8P-?#<3BEWr61L@_yXxBOaF#0Q(EnZckM7~p^4}&0 zx6k*NxT|CNy?;L-oqs!h(xtmp{hzGG$J&9@pGo{bZ*80znOQiX@%PNW`*@hjMj8wW zXz1!yp1G3zHvX3JtRqK=YJb0G!_edvu8+e9`W@%?+uYLLLncn|U04+qtq`-?T9?!K z<&=HuoR`@`joRZ9=LZgwBJ2j$hbP~Lcrd@K+RDhk`C*i9Pv}9ZS$~+U;i9 z{XXuBRuwLa8|L!PN^DCFf|UQu4P$UBFJS|c-IC|@BY!dY$5o+yadw{c0T;^Y?xLCH z$(L%+#3$a7yw8e=_(2LPvdLHB2;LW05_jC;lYi-&;jM2dFB9nkN6rtnU{VSN)%sN~ zJXuJo&19*(*LpG#npnngskS4&VCp@UgpAtf^i5qDS+PR>dK+p4eRKt*S}FnyU*T>E z(lX2PVEgCL=qw;&M^xC&UbVQ7vY-t*yB$auasXfb^;D;7VT`@7}>t zhjc>teDQP-?YZ9V7T@Lc=5Z5t}Xfd{X`Q9Fw%uUbzJer%d z63|i(b)K{Rp580J#IK0y?r9;mdDcWQ{m6pze`oLc9IPhHepQqb#>OH4Y@A&)yh2To zdXl|lQha`oFHqHXDK-D}QYwmhSjuyR>XfB03w`p@G-1#_R@K2I^@EE~B>Wre!5nM9 zq345f9*-x>S?ohG%UL~Vm^2U`TZ1xmhk@=+AK?4OSH@i~Z5<>XZl^a=%Tbe3Tcg=j zE5yH=^VW#N@iFDYpovF?C};lns3&Mi?!;E(*(p`_n-?Edxo6ly9(AZV4F1hy4wI8E zzOBYL!!^Nlt^}R8?Qq@3&hnOpepFsyXal}XEbqJP5d9X#W{Yx~LeZek9oz|*Te0}D z`Zn&q%`TbkFp8Ps+<6CdXc=_}O=ob~cQ=M^_Q93c1g|jg>RxE+i}|?i<#(<(|1JI^ ziiUV?gf8%9cCg&6ZFFOY)$_*xjbo&cRPGKI!U>pX*qyIFt4Y=9zjae-!S`zWh2`U+ zOy;%bVsi`&DcigJ#FnJeD|V*AcT9HvYi~fUJY>z3`AU0FN!QK4!F9x^E$2r7|LP%m zQ?IcrWyC`^B<@Sa@2K$!fUwtKJz;u9Z@8dXb&-BX28w@Tn-+(6J zHywTkNvn95Zf?C#spk=V7|+`m_0;8+@D6M61BfQHOYR?`w_Pj&&+Is;6yM@Laf*Ha z-bnLRsj5+QtNbG(|BFS}x(}u)5#EZQ26rlLsS}y0zugMG43%Af7uun{4&cxo7qJuepGi_;{w!%^~^>1Fr_Md+ofo6&ylv9wJ8ff%Q zqb%anIraL9_Pq`hjJ~}2@y-br`#*7-VZU_PS@KnWEry>irKvkz=Rwk`$r<#(G#2`; z-2Ly;$j6Zh$b?4Fh)`cx|Kcu;tA&j4JW9hh^yTKivn^)r8ZKfP?PQa~XUwP0?V6Pj6-ZBc zI>58aqy}s=oLQF~xTg}l=1GF~9rZ5vHj6VX*oBz8-<^5f!B0jZH#Ix6?$ID%SE@hN zxHKUo3jcV@DNED^U6zzxo3@5n6VtnHg?@S^v`ShxA@)K7`3_1)v8?C!?|tB3k}Lf4 z=<{j*%GR=2^1w6Zm6#Rbi`Ip$;aOyKPr2R>tZpek<*D zJDh!N5yL)EKIHdu;nkn7CvzCynq?~oQ8#QhY>_H_aVNq)%lL0A!FyhEXEnum>GL5s zRD1Qkb8BDIA7%(&revVo9bL4ArqV*O50;{0z{jXW0`<|;6?c+MYAGI(wsq$h1`_F} zcR#`zObqYdsD0mp`pbT(CBS}=qqbNLU&>#(LyH*umt^p!-$}>i6-k#Xsqslcmf(rR zh0xsvu_wE~q_uO0U2S~r#8#?Lpg+F99h}EFp8D*?{YmbB1Ur?2)En%Tg!=GoGx53m zwgDv`UZ2uG$@X)W+ukfhpg898Ta0%V<-4Nuq*n;YtE^7+DsJt@_Us?U1PuQEOEZJL z^?S@({Ux=!>hSg6k};`2V>q()gBQnYi$7OEKHFq%LIKmS;I;l5OP=REzH-zOn-4s% z+`87h^RNDCalLo_UT7D0>It^Ciu;e_1Q_^#B+$jhB+v~yztuXm z@tI>(Z(ad>_3@^uuM2m>iJtqK$w?W{uYV3an+k(?9$q||?EW!6lCFnFZB?t}E3QLe z!e!=Zy>0(Hx4)YWVi#ybYY_;wE1%ORAKs7Y#N4}l$3*UQ)Y5EsoR~UGKjx=j@Za4g z+KejC1H52W&7ZD%%BFLToSJ9U>UoDcy#vB~^U{jT^+4kn2-zj?d<4^lv_zqgvP0F0`wzA#1M2MfB$)*8jA zpxk%_GK&9-u#_%^4&0mgm%lzG`#$cczxpg&St?zCPd|YiEJtHp+17v&qaS+))D+bd`g=DjS$c%1K7O7;}}VX`do zXpdoe`sXQdUG&UV0~GI%quIF&S`T%NHP;*S*Y1>~HGKVVbol)XJ-NR?dG)~WeRg!v zc|Gy@>-c<*|9zH{iz)ho>V=mCg*cePcluJLgW~8^4DSZQCa#0ZT=EIkgh&YR&RI(V z-qr`sQH;Qb2X|D*OBBV}z~hcN;UV5m&j2Nu5e1I~!Oq+5uDOM{oY04DB;r0i(2%j> zW@)R7sHR_Ah}%PEj{)S(GKx|cn1lBVct?+6>(Rits$qb_j~9tENZyA=iR;_0ht0{y zz0s76t@q#c8J>R`6};Yk7G`Yzr?^wVn@Z1EdW1({g2+++{WL9zGxH ze|~smsOP=>&o_m9?p2- zq0)R|xnejCHps+!kj29K^fgqH6V{%~CL?gislPPWIWxp$X66HaA^1C z>vx>uxVGDv9`?>WrSN=j8b{Yd3kya?3Kx`o+<2;=cgld;KR3$j6UYv4Ng6LNy8Viq zz1`UB)nsd>a_~rWI=}R(?A52DIH(FavrYRr?MS*jU(= z$=+`D8|?oG?Z3+KH?&`1o3RKvhQBpatJ5~=yRjcCek4_K4x%^HCbIsi@b)RVeR%Qj zf%mc<0Ofq2SvkC8O0M^|tN4LzX@3qV=^eWd_n)DEnfejykCLtmx4#hhwN2iZz04nt z;QUGm{^0m}ySV%Eek!<4bU1)TRh=*r%GBm>)IZ%Dqp)qHuB?q&S{p9I9`~Pp#HvSQ zrJwcmA1Nw^@lrh`!60~vr~A=HRxkhcA?|-qVXZP~O+ntzb#=II(r0IDE&fQb_MH_g zE}>ocCl1qH!b&i0QWg9n z4FPI)9Q@Y6Z#?{ZRF^f8w%mOOlE6b(tfnjGK3`}?iBGK}iJ070Y1eK_FypOP4)?T5 zT^oY;RoE&ZO}X9eNQlQ>-i`W4y?ZM9{Ofz7tTMaXnGlP`rmd|dvCFsjLr zzkeG}jx^t)x5L9s*E?vK4eu=a3SEk8PLeIz*M8>o@{HmplfAex$(HsjJ0C~l9eASq zgns*5II~@_%kJud8G+j3;LYy3){J|$;Wrk?Uho@-`@rEh9`|&^FUyI|@NT`<9>#>d z-++Pt8XVI2b*J^j!JIQ6Y{YIzzw-P3WFB>M;(krgOTQc51B?yG*qbv3H#lLv1^BQ2 zArud9x1Zw{A-!Q2&r|PP5QDSTv6s8(f;WC%z+Fr4n*xlvnn&=jOWuQae^;_`ULe3v zhFsc>|1RXeG6X*k@KrbdDCGYpN#+Nist>c6Xfu|0A4tkK?*jkw zko<@HhTvb0{LYa4?}y@dB7bH`{#l{;Gm*bvX#GR+_e1{u!$a}!jtRlP{~g@_9g=@_ zDE?27za%98tWf+V$bVBv{$Zi`Z$kcrko<@Hgy2s={$s;J=6@*uV=I8af%6CY;|Sg8 zkGHMh=bwiCO@Hhi>*s%+^XG@)57GxBih;VqxBvO|xeN8VGFX4w8@bInl%4G>OvcnS zb*}F^z@i{V=CTbkP<^ML8|K%a_5A^NBQ;6k{ z{3W6I_1g|8A<7^5Z@S#bZwM?}^SVi^Ri&4&1lAhav)7jegf_IG<;$ahU2s30nE3%Ry*zjgCr=n)a@x z)W$Ecq}M})mI3;m_#XREwynh3OCQ;a`&n$24%3gS29c7jA`{}1;s#aP#=*)%d>P-0 z1u6KJWJn~>eM2`18QAV7ydLj}O3#~bRTLP_6{pXlzjClm%=7Q)-KM&q!BF0Yvkr4& z_0zhj6CLI%#`95`+UxI9jN!bHjZJq#4e5Ca+sx}dzeC~szxm$Ya@VtLcsnN+*@@|e z!@>LI6W+r3=Mw+;#W^)dmrB1baXQYEfi5Hcx-3T#paaOpjcO<#Bwlwh^m|8s!S>CO zNr`yB_hJ88Wqj|uZ{qpCODMvZ+dDfCx9AVth39-^!BF_i41e7UztHy=q~&6B!To(h zKyKBk!2AX<2^bz`N051+mP(>4!T6Z8UJYHzvWanJPPqK!E)N) z00TZB8uGm6aNIAw-g%nqBh0DN;{|;GGKmAfuzv^zim!SC=M!H7ar*s3|L#1?|H<>q zH`44B-Ev02<1N)c9wi*4)xaF*WiB0_5^HxlL?|}T?D1X<*-OtxP zq44qjQYFm5>l@q+ywyKLsTxZMjn2vX+t>6|_>x}N3%I1WBmq}z0LsF<(Hd>Z>yy{( zM@RK!QyNFu)S~JtGWO+)i*wljy^iyLpk_`8eCZ2D8 z?60t`Z2ofS19J86FY=#j-r`8Xu?Vd zA5`dH83LX{<}z_|fH9f^jHx1HDrZ!YunLk}_WBd3Vgi*X$U!1oNbqVPL5`RphbNeU zM4pgfRv^I)F#*j%g42sLCi<`|iP3-E5`$0U_|w>?;z~tP0?#+I=dt{GVpDMyf36>E z!J>F1i7~#(7?U_-d=m{U1A>k(#cN^4#q(Y#j-O`E{QLcv8QS)V8QLa@77Q2CL`|I( z7?B=9O9%J8tSGCaB5B?c>G2Pl*fHLdv}-1M#1mLK=r$Xur(0E5lHU1qf4U{6nue(K zXmuOP+(YB4LNjhYg26KHC7X&`xiNTY?14^o=pT#LBL7-%B=prs)pwv_;btcnX_s+) zoheg$r-C}1k6)seGSy61h(uaS{t1l05n(AA$^MG6l*F;W%$AblxTCZ9r&vqLHzwL) zefef%K2{$b`17p5pM3&%|8uo970-*#G%5OqkvOt`wwXHD z^h+UlqXrG(&}#hZm3~fYPYSk5y^4R{`CF(mO~p8Gc9(*G8DCP<|0<-;J0AJ;*oXLL z9Nc)YO;`Lk0l8sJy6tPs>zF>r`x|8d4gH2c>8d9E%_IF4cIk}da?5e5(Wn1BB&agb z{UVj*fH(TbfiIK$AN{d$jmTCHx&ME^%P*esC42NugZ(uCE9z(Z{5}1Z_G;Q!M26;` zi|5gg`VIf#tT2X0iH4t5$hJ;dN>0OZbf>U?bv68F)^GlrN%WrG#m_+^9;;tt;3xhP z@F@Kv1K)1oP?o0~;HQ5DtzTpU;(qnf4(V>+Kuj{)XC3*VJrx&BLYFfhAUP z{hoch!CA$Z>o*nSa{WBMTt;n+b_&$^LAG4)vo~GGR_hbKj^W3}&cV0iPLtB(&O4Yq zKAEUaZFQvmIL|>#u&((v=nWt`?lPtkcd{E%a9MukQl{;y2zt*xW5t#x_Ox8g4l34i zwn3L4Das7a&n#`-#4J3uOcxBmNT-_S({S~D`v7!=g@ts)pXV#TuL(2j&0XyEnS9a2 zzm(s^?!&@o$y;g5)Luh1T-ncXJ?>xa_BdRFETxZ9nS5t=J;xUG<`zPyjcdfu#+r<5 z4J;BPT>>#wPj(wM+wqn0MFf{Au3NYFCVTnA;)c}Vjpc#+5N{}?FyT8W)*Xwm&{J9$ zu|@W0(TVa!_C|+mX0v|&S24z(FwO3=vF(_@`j^-3T4sWog)=|9zhp5^=jZN3Rr{`h)`Dl>d@3-PT`3HB|tDxd?^h?E}| zhN}u^uBRmm=*n@w#iMvneb@@|xKts|>gTTzb{{L3aP?v6e^DQ*p8l`u!){(5UTB8; z@KuQVVC;X=cbq}xj2n5X^x7S4&(n=(D*g$M1uV0n6#5@sfNf@u+3zg1IvSgoY<1c> zORLDj_cnPlaCv7=z*%BQ&I4WjM3b?#>6bH<5SK(#|7WWzuAv)FEG1+3MXj_UC)cAl z(NGI_XFYW#Ip@esK4vO1#%g%UNx8E>jtE>f@MHKn6=LYY_LMZF^W$o^O1!HPgf9CQ z_xbr~&(|i}1?|({&&Qoan(ZaaQ@DYIHbg-w7;CIS`=M4S{%ugCKiih{eZFy}N{wE9 zdtZ7dzfz^kf2B$d+ek_`X;jmdDh0RuuT=5xjQaL}v@^=G_#yhHLqS>lnW#^bt(--Z zHVTdTy)PcNJqgEV&nW(_Nxpmt_xpPv#clIhxT(iMyP8t=JQ{j;l1Kk(owz&c6XkDq zC*h~c{JWF%L$R#&%JV>h_KW+A@@RKdeh#XG>-4sAP(n8um6Usq5)ZBpf-uXc!I42` zz;16O-?z>4!~5%BY~#?>b$C0vclKX@7oVJGRCyK@mKeq9` zxFrdB_;`dy@A`sC*01!7yMT!s5D{8|D`VnP_V~q>ZuvXKXGV5Y+*=q@@nw%we%x2d#J;`-$@D+| zP5Uvh+j@En^+!0rJ@;`k5?z^@6ibMf(-V8kzDVwf2;S( zh;Djc5L)kpr6hLLt4wTr3+a7*K=1Eb{Ccl074#m#^xiP9o8BMHL%m0Fy*2iNTAnda zw)~ymf0?@Jee1lCdS6j$RFsb^sNAbxBz-QK*R9+wq|ZC-vo`l~JtF>AkAm)cteqQD zj};|k-jA;^vF~iA{2TtFKmT6X?9aaiB|`okeV+8acW$@*`^Q|=_Y{ADoxR{7FQDr; z|DE1nD&6${qadW-M~X@8rCjf$n@H~kFZlJ|yUDM2W3iz3D5m$af^K?00g&@L6uuM^ z)X>iN;OmXb{k~*QvEWOl7xOjl_RS=+jE!N;-A)3O3)6-eOMxD<;idi^Tr|;{a$R?* zUw7y?rwF-Hwm8Wk)m2o@Uq2;RJc`1`laPH@@jNHHxH~XkD$|(VQ35~Gu@Y^T==dPo zs{D`M#}B9m(#J}4d@wDYPnFYmlmGuSKOF-V=_rYgmgpW5-BY5?vi-sNS7UwbZ|(n& z`D^}v+}>UieVRn~mgqhb9V5|wCE6mE zX6EYSn(##Re{o+!@b{f?5qt!$1q|bx3L1MVMQI9NjxhD-f(ynrW)AUTudN?_lp=+| zJ>yItI;OcvFL(rEi>>^A{#@_1Hz>mQGw2fC;`Bb#ARcfID^B;_g@4R1PCtTwSW0l| zs5pHup=114jWK;cXVYyd&o0At8V@dMCHwU0JL!a)7QG4*Ip3qhHtpZ>b0gE?w zC)s<%l``0Xe)&%R-iRy3WISy9&fq^hl|x^K;Q!iqSncPpH6V8g{uhjgMSlLr4Jfx8 zziM1P3T3p&hnq9llnMMDh2L<6U%X4HB+^6}-kL=7r20qq_3_W&y+(zUe*2wxAqE|; z(d%D`HcoTg2alp|r$=;xu*!JU;LOHpvbFqk7uhrE1BGLPZ+c+mrH}K!&UqEcbFx>V zbI^1AuPRuI6O|xp<$Zl92%kACnpVrE!7p8zwM)ER0_C>f>zBJ5p9lEg2mbr>7(cE%wp0Z8@ywU^0e-uunBR*|VtyJ?3gque%+u;9XWwKG z{Ehj))Kly+;ehLLD7A`Q)fH9eZx5wXxnL8s_;2>ZDRQY{(GD*U>AMp@4=%|&w|tRE2s3un;#1X(Rtte-qP^H zzn(^|jMlZz_?6a1oOl(czB0+8;HB2a{VjO(=$O7|a$lunS3&P7B#8C{vh;MH!(oafIHYF;t%4)YwnYtH7)myxIr%_H_T= z7gV!*RQig`zCYN!{KzSNsma`kFjzSfR0(-%ocq!KLs|M|JpIYZEL}vnbn^W}{R{uV zDbYVvz76ImEB_-u;}q{6JYKcl@Ep5m^`zw<@Kq!jh|E_^6 zBaRx2*2A@^5^*g8_jrV=UlUo)npEwY6pNP!s9KiC_*#!m^EkAxc-^?conZ2x_py00 zEuZ$L9hPZrP#tHV-hO95->kHs=SN+-F0i6W?P=YG7#54 zJgiHBOlh(;1agG(+g1ZMoEgZEId`dAZgZx#t=$au8S`U&$oxp%RcMg4)xipV{+!)R|hggP|;^&GjL$iu3LnoD6hUQjUhEA)t44vt*3@vD} z44vPmy1%sU@%fTHt1tv|xZgkulvs^ZJ4e-bphJjN-R?wMlQ_^+ihtlJq^a~W_=}C# zN|W}`2gIyC_F;9P^rF}Lvx&leY=SHiEb?XNtBV>?@hNOSdxq8lFUGj0R$(I#Fb{O- zpmn2Dpd0A^zlzevgyg6yU#yH+z~vDcrXv^fjo^E;y!1y zzHn?`#Rw+-_Vf6~;hxr_x^q43LYhR?b&HvH$NWqe!A)xBU%j?@^jmw;_qY4wZaD5I z1LG6g|Kp>$r(idw;kQ{0p;XQd*oWhH%quY*z2S5MS0*2?WgBgN)lZoTS_TkH{j07s zXt4loH?vP#7t&P%et0#9?+Aw9;_&qVvqMf};jeNw9&GAKdh?A$@XNM4LKoI_T^xtv z`EMe7G%pGMPUV=D|El~qoB!tU-#q?HdoUZX%g4VT}ebZYpZ08$)!!Y)Si}@P>wXh23ErA#C@8X!JguOrLuZe4L zaLW!U#_7Y~3Y_V9HUby^G82*vy$$1x{inu0Rbv~eVT}gX7@Ka5y|jEAt$(X5-z978 zgl!y$r7(_{!Z=O2=Hwt6Jx798`fa!N zW%-3?KgLnEodM5I45Do7$i}A=*~cC3G2!N@D--Z)BQwT{%WC>yraH6{tPrjM8Z#s@ z%jHb4VsU^-ARbLf)vtOJW9Q?7-K3!N8oXzqtONcW@9uXM-6}h&S#N%WXY;sJJQ8au z>B+X^<4J30Z+@jz{Vd#JlbN6zO@5q;WH$VfjWx!X9fykK`Srcx8dhaM3a>Uv>3tRC znc?T@%Xnr053gu_fCsw5%!K^1art(aBLPDE@ekr355XS~*x(p%R6_BnS20Jm;^t?`yo}U5F4WZuY{(Wb7kyo(O4%=Sa&ZBDa-3{H_ zYs#lWd!;62d>_SiT|5qSadb3-UBw8YhftY**=pjWd-JfqnUi-p@XARTYdc&s=+?!z9q^j8dQ0((R1rh=4=no3%kjqxrXpwwmxMLMRkPuO#^ z@G{+3MXy=Sj(7d4TexPAO=r8viu506>`p<{OZuNkk&EcuRa2kgl!SG-Xrv$^j zIefm4?VcO;B{tVc?+CU}NbWxzT0Laf4Apgws;$hYU~*CsJL#TP!9Vt3w6CIbsC_*& z9_xy&2v5-QE&bHSwtUn_!5pDw?Vt78Cs|EgVPZzt9j9EY#dx6ruTGw!wRGR_tlwwH z&LFE2KYL1B(aJE2f?%6211s5WV&~LQV*dZf+?&ToQ6+)nJvj$R=x_%?2^taaM3y5Q zG6NaXqZ5fj1jQhVM!Z;*89>-Tpl2XWI}#NX&yB|_t`}Y*#Dj1p!HbBiqNoH!=y6s- zkPtaC->Q1AkC_avyWijU^ZEUe%qUcEY4jiP_9jU(~_?d0!-ltt(aG6<7f z?-DL^*4&LVyvTx?|9C1*FsN!Ky152?gdslFn}+zv6{xO8ZksH6*aB1Zb=m1xiwV^S@yI~_8LY?M_MS;SSL(IP#&c&@dq{6GqWwAqA(=km zeg=jjXN??7lb?PvO$hzQoQ~@9x%9K4KrL<$9C(v0E8}h`0D!sN@KY+Oblj2D!V1I( z#b9kFWD+DjZUb373U}JDW^I~4Xb@^SrPo7kh31giLVA_jll7DLd%C3u;U}mE@13&f zgYsw)bnkG4DXJ3netj)1y>3XHb#+?fTtIa1eaM0SWoIlLf{65hKu)2(pF$@NtK|H( zJN=9r-l>;RdlaRx|Gd@euTySqgJW$l{paOKNq$8Dmoot4J*_KupEevi9GY*&XG-5h zS%1N-v!k<$n00(4%Kv{b{UOR9#{Ays{JS5A{QV-){zFaqDbYc6l(dZ2#|bpkBKxfd zsz%Ew;DL3}-#RmBK`N@Xj_b_)6a&_MJDBEgEoyb5iY-x7R~_*|8Al=!f1R?~I~+l| zqb5xk$V<2qFMFL4A{KA@p!Vi#v>6PGmXE{u-c;fe;e^N0z9fAmeeIYK4cOdW+TO>2 zpLh*&+)Fv|8m{(L6SpBp-DB-qWCGEH+Sb?7m@i=Tk+5*M-cOi&(LbK9Wzr8TZ2lEC z{~e6^pB{JqE8@*RbYcW_B6QMRFEIa<^#o>pC#t_0nDuMs#}D#7UJ-BJA)mGy^1Wo_ zySE}9KH>N=-{Q8Y_z2PRq2N;}yMV*~YQlh2I;95RXX1_TB_+U8t5DFP_5T&8+o^2v z4R_XLC(u}0mi3I9HN;9cG%GFM!VI>WW8{8(r$Qoj6KxRQw`jM7Dys11{Ry;WpQe`y z{N-HsQt<@Y78P8Dd*S*-_8Fzs#R=xtsRr;XK4DZe!02+mIUYk%RG2mhXU!#a6O~8R zl~e+cROJbx6z{%Rh} z)4eu{%8`ENEo`64aMqyRTIh{7!_H#2w)!CBFMhzCL&XQVHBsbZreoUe2Pt66r3iX< z&C{SAy@{!Nj%4GhqYoGi05+xHTdAo|&Ke1u8ppSX%Xljk>hadJG@fTX<~bX!``GyC zgBZAwQRbATh-(vZtyKz9vp(SwK(^%5#_NG;s3NM=q6u{rElw25g~Ax=xFd8Wbg6|gTzzNEuEh` zfRNj8j3zE%@N<~^rwHR~?qYmb0K$@Cda2byk?=>B z{d9~Rk6vmIVZP_0_D}u9In=F=%+_*&<;dLuLm4Z#CU~zh#6AOciOvMrQ{NlSJaBq1 z?o5HO-3>|E0<}VxjGw^gTJCDe&iP+U@E9aEGhe(pKVdG=%O zc4XWPNhyQniri^&l}BAW5bw9EMs?QjtH}>PE-yTg(JAeF?bn|fR=>no&@(@Ay2~_N z$@1tEp63fjagnbWeu2UJX3|OYpf)t0vEu7VAN)c_Phh4eFUFZ(l)$;(dZEbR{~t_a zCL_VY|4b+TBzFft%xI4h#NnH~MT&E7byYcQX0qL{wH#K$vsCkC;pg@5t1%vs>X?CI zg-8a>O&+&simA|2-yBxo8mw;tt1omlvDNr>Df_y-ob%=Qb&zg{#YQ@_97lRvLNtF5 z1J>THyjn@~C` zF;QH0z76Fw{%lfmS9a&CGcvWxbAe+DXx=+X{__u{pef$J09@-31Y<7fth1Ebrj_`OuXlu(||lY%aqZKf@aDjIQxws4+C#5-&nr z=c}W57~5-+|6%9WW4qfhd^85+m9@qJ~qYb>RoQIEh+?ORA=`c7_V-_&YRIU49u zZv$Mr)U8g;&LZC{KD&;quyuee(>~r7`N)0LAOD$mWyWb5InmH zgMp7hs_q~ep0+u!v%r;7)(q?{u(Q&B?S`Lm6~Eh*z^>X2=Sl(C$Yp%%?WV1MCXE?> z(dl?OKFL%O~1gqkRi#)F!x#@kT}Z1W8+ge8-M z``_PicF@qjcC0x&+C%rJ36?7;!jL1I(I{b+Yn}H}NiM4syaRFnr}C(=P}|MIg&E0^Sa0LGorUl>tl#Bj>jW=FXt}wH2xkpRF0z=fs9*lBzXSdiCAZ4tfp}X@ z(>d8r$OHQq@ydg*ruLA9Er2FfS7Ye!ZSD27qRwOb^er2x-ze}I%pfN}EilEtQ8!C&fD$1cD#B~8?Z3G zFZlG1VL5!?^Xc82Ow+q-Nayz5w#MfH{^uN+?`^U3eecul=ezAG{aqoSZ)qR}M#K4* z^!cu2^LQn)&9tJtN*xKvUHX?2GQ$TuL6yT{&W zOsB5JvY-wCOx|F|g8q7TXU+-kg|`#ntr7Lt#0MOCBV^S5-D%-L?fK1RNd(&R!6v`i zz-irkjEMS%VnvJ2ZqHZ_M1Nqcykjt}yaf@9ypdfWqv`~Fxhal5qj4c~Oj5oimr4V* zug+x1aRtBqLk??xmK5|g16nm}!=l}o4_=};`=LRvYe78WHUVKi$pwdzuw)6@iT{TZ}m>Z^vK;@zPwfIzZ;n{ z3cvk}=I`5?E@!8q3jn%*?5M5H>P&Y?(P>Xso%B8Hct6(hDVUbOH#$D!drQYZ>eIgC$NKS( zVK7(I!tJh>jSKg%`Uyg`%mfEM3K>bH7 z`irE9Q(z}QG{141TOaZEdfL7P^j0EP$GL) z$7SRdWYCFMy($BH*t;%od`5{_uxWFw{|I6bo$~JyrSM~477E5mAFf68SK;ptW;uYq z1aQ$hQ3yx=reH_NfL$Nsk4mcGQO`knRUsa12cMeTQ^?(>6gQgw;_AF(J$ZY)mwV9f zvnpj`e0nvbQ2E>=y1sTn?G@acpLeW0zpq1Fwg>xuO<(u)?vHqEha-FSb0U=X!M>RT z$xIKqs2te-{F1xFnfDo<@`!_a{B+-u-^lVIPSSrD{O&Wo! zfIkpsT%CcQsy%tF^$xq>5%Yh?E&>u=k%^c^G67d-;3O5oIFsBkPPbB!;Zf#gqK^wcN`Qr}-qAq_J;D8vP`&=(|9!nsX92lP zbY-xkn@7#UnMMp#ZbflN=*d%&2y5?5%n5Vl9Sp7F++f{yw{>pV1mevpvXEmlfj`aA z=Llyse7{El{<64^d}f{I{L0+Qiq+^=wh}dqD+)iYcYPw%yS8(Apdaw1!G?U)c7gd@ z8(AXyOJS9a{F~^%Dgphg$a{CoKRL^f_nwm$ENUu0-a%aUEAUDqhl~D;Awz?!5yx9z z_9@hG(#BQwa(8i6)Ot8E<+hY6C7Tr2YQ^j%bRiXZ>SY0d>DhJVXl7MU?yy&Gg z6)J6V^X|g)e~~rk$li1I2gfvx>@a6{aLniB$8Qu@ZX9V7AE~o9`aZ#nTBU4Fg=?ee ze+PJVxvVi+^e=>-l^?$lnB*E@8EPEE>SvdFg6Hn*Gg`>q6}|}dh3w2JBOV=o9!H9; zsbkAW4i&r?q9TJE)}by6_t33T!g^{V;x8H#I5zFM-y z71-sQkh?3&UgD)*qk%>I7HV>D+?NcsU8Ro9!Ww6>8aq*qqW?bHWj5QO@iTo51kS+R zR05!_ysQQg>h3k{0=*nqf6>1empzOTVF=S0$OAmUX{>i`1VWL(bUH@<6*tHEz`)f$ zZ8>Dzx-bp%387sAyN@4vF{73vFBJXF_}ibw;~eP_{loBgiWumG-u%Ss0vn)^4JQoH z$QG(SWxY=uIRd9BOI*3h{+X}q$cx0lFHzsAJeCbD!S|KyZj#7s5-uWM|8_!%&{X4o z86N@B5(&{h!6kc&CjvXIdJ1a`+!k2P7#p@;E z_5^g&#GcWpn1$$E=dhP5u1)#v!scxk{Zo@D$`_<2LP}nkZF7jL>K$TrOYrSE=;En#gm04=xRZ8> z#w@1J>Xc8|yi?$R64QN$R3eLb?8$2a^iRSOT!A|_?16{)9(Wi@E9`++XHxVAB zJ)SI&ydVZ98;Hqn4SG;8!F}PboL_QeNV@xD9I_X=E*#j8z+1`39dm&rJ@Xo%ZpxR^ z^t87m+t-mjPWj}bMO*Qs13u=>0>)%|clh*8Wbb<9U-I+zEZP%(em(Lw;ls}NaK(mK zIp4!?;UG3%;e3ximcv-P_=vwbemn6uziT>wb33eKSJ0bn%Z+I7FGv>(r?*@Y?E4Pm zu6Jv9EZ{!->P7zp#2M}q1J~K~!QvQvUBlNIa_?(sRPOTmFvedcH33QCnesg3B~WH(17zwkxVaEw40qi2a4^dDZnxD@0732+G{v9`! zPX*tgk;A1 zEY`S{4?}H$*W?g4^v(&w;X|cy^;@rj_RxO!zH}k9m3nq9)WKyMR(rJM)!jx5wT9b+ zc=8T7m*lHgXBoQmObxFgfp$Roc!GCE?k@PXc{A`7z^xAmA5>>wxfMtp>Sqwc|D1q{ z1h(CyNd1#i=i*iy(7Ffd>H%dYWtz6K56Pf4AoK7dHB`y>b*KER4GSbDs_wqx%DVji zjrf?9=nz-dJNy4i1Q_Fg$-ifK=jZN{e|Lz1+hEN3ifboujI8zn$2;-bi<@DyfDfrK z8uVNdT0#v8dZ);~QSa2~=uQC~(AmVhx`^I7ZhxJ%K{4p4Q6@mi8$Y1wAODKpV&Lz_;B@vvf6aNk}>(ipGK_@bzC;yz7GXV z_KrZ6f&u8UAcfA5gIC-_r1pUyfD}|P<0kFIC>lrU;z&Z`oGzie_M{MAhh$0!p?Bf8 zGLF2FF8&wUwLWvwb=!x zjhF{P&%d3H#N;?$m02~XlV?W>&n)^=5xhoQ>O(5F%jZU~g!-+&TfN+^q%hggDrlup znaL#7xseLjcI_G{15|l?Ot>-2z2_lsx~{>OK`xz1@ANnTF#IUqcHb12?p=Q`UEMrx zkvCcNi`nI;9Y>yf7U|`C)Y*2_G2evJeq}RWX6^D0CV3@faOZ9Dj)2Ct+#DA!Ki@;f z2iiMlb!L|w_Bil~Xr*4Vh4$r=yDGGDAgC-l3!zeGhaCAhRx}akB5}PA6k}_%({&Al z4DwZsF|M`4BLNPtBmJKipZ?#&=sy$=U1!KS&_4o{`B4P`;N%jmd}OBJqnPsj>hm)rLO7G(r{VWt_e?zDiPcG&6$zx2@@-#~U28TeZ}Yvt zQ2R7`yT2bvv@^K$?vL#eP)AfLUL&sDjLM-a>+GAoy@At*Q%`INbCg#`D8h^-MhsDb zqxZrFi>$WxGNf|1G4M&j9x-1DeTPd&67nU~ePj&~BhPs0jhXUmR2AFrlqmWbC z#@vVp-L=w9d`Xp-ierpu0{j3Z*OS?QalW!GU-^(Z-lkAP8z|n88itQ7%e8u@eC1P? zk7bz=rjYSjJR2FfMrF=dwrgV{llEy%ihz!&ORBEFPV}RaO-%ckl~rA9r~X54JH+;Y zXTTBi%J=^i-2Pd7|qR8YjR6$su+1X&&Ms&{A5d(=yHmX`rD|8KYI6G&^dunJALwW5C1kVh?qbyUKoHyto2ptB;W z3oH>^xF&XGhMc4*{|9*D(5GyV79UUSCZ2qt#hWtBT1@B^`W#x@z&s|o)@Xm9sc$II z=QW~#DB2`)6EFCsJ3^opLG%;wA=%IruJI_owOVy6-QcaoFi?eS*j?-<05d-McZvQY z8#ne-E+T`!AsD*`OKW${#DI{Oo5fUKi~(|ub|qv$T{(Qmloa6QBfWIULN713;i&)) z1lzTF9zO*FX6uf^1O6+x)6Q@HjQQ1$K7Z`chE+&1hR!t36-8&Q-}0LJ-ur!(~p1BL~?bQsbU z@lWM{Oomc2bshM z`P5x7$VDvYyzAoG4N{lCyq#BCz;E0w{x&j{O^f$JDzf-E{7>{H9NCO4t%9eu6MDh_L zu0qQ|Xumr?ex}LIHnbm83;H7eDk1+m7WvoCxpmh2FLH+`FUqocL)gn#OR1OD*HJH5 zK%Z-Kc`wf%gz`^vUk5z0(O+>L>{ZD}_eCO$x8r!B^uXlN!g}sz4)*&TI(sVUhoG!U z>ohD9*=Et_1AuXFVC4*cSyBBjmKXJf4fMykbJR(F;Vj$tCK8fVO137tpR0M!7InW} z^K#7dqQvOuKw`W5aOC4^s_A_txwo_4*9X({ZY7vJJDFhe%vS)D`v4|dm&+m%fNvH; zH3zS=5_9WlsvGH8F!&x;U=|keb0B_~%k_uJ4IGY7ozxvZ;2C@3oU`TQN#1n%xV`3m z=#B48?9ck>{=y-=W*wr$571CQ){vbwl!^^Kj6Iv1Yn+E_YNE?Sdzo?C`@m@Lz4qh1k?I<>cY61Df}m#B@3?WJ1UdpEkgT99qv-yq}swEt$RXgsUvDy-;r4xx8*X#XHT zZ~&j0XAUU0%UE?@s{8lA8#S zDc2GplfM8!9>*E+UCQxHEyzUpPe%C9M)*&KQs=Q!O~q7d`@f*n&sb{orN%r$7m|E; zKvS*7J)_q{a`5avf<+%Axwg1x)N^&r^ICdFL+okP>~*Z!C4fw|E!b@1B}8u@WS@t# z&m-{jVf_5kB~kZ<4Bi+&ZIL~6=AG#R*K5%AB{+O#8bn{m+!97O4wP63hXtQdmRidk z?6-Iml%8Bqq6T|Um)YU{L?2MhN|(&xFIhY}V5UCC`A&cAM(X#HiPZ1KpX+khUoQsw z&`y5_h~;_3%wLJFgm;43&Sj{xyTf#;p48-Ukk_J+&b;`JJN;8*Lx_Q zh1EI)_4R^-{m^cBg|4R^99%X5<2iCU6+F+u=ex}3d-3@O^SSq$GlY=Ge4dKmN1D&O z*1_{d{F#E5!d~CxQCvH>K#%I$2@F6}XfV?xyHd9cL3!)N6M71KPy5gQ zeq+DK(hKxx6ZJiw|H~iQ@xHU9b_MRX6*k&Kl2QnN#jTO1ZIVuv@%?Q+hM(L4$*k03#bOyC)1f0c4(y>3I78N=jhg!*>)I1qzrQg-TOZ z)@bgx+b}RB>Db$<(%DS_cR+~03e%v6v&f4G8nvuZt3Ru=?s?CH5|SZ3$h7$NG%{>K zQPBbIp$wE{_+CTtOwUh@{N1_c{KH+jc{0V@fjm1tCa)hK(a+$CX{4KpeeIpj9K?N= zMhfp*UREd8i;O*;8%GK@_E64M#Kg#=c3ClJM2zI0=n5Dpk}FEZ^r+J&3K z-V8-U2Tb`Tc0}SGS=*0yWLa}(2B4W}T5w-cnh?5$HTf&ahE~KhiK7|>t%lMfaqZ)* z@)+d^Kx6-@m?ce~h>2hQqbnAU6)^qEE>h4#3Mj3b7I79)O9e^bvirsJ>`VMupOyB37Nf31&LkNrtq#b&S5W@Ga_Q?1&CUy_!TI7W}9~sqV@L7H}gInT| zJn_TM!0}^EbI#(BE8N|gnRFb`ZtlirGfK#W^!wEt$$B6LhH!K^RDwok@$Cy?wK$c3 zxs_9Hr9~^qXHdHFOxd@^0A zMOWV}CKGwxN5$&9&bJ9pWo!5)U7w50>8c;^EYzm4+~Fp=m#OK%2DE5uPbPMunZ5u#xjUC84z2l zLk#uOe=foH9ql2eU3vqP-rm-iz|?H9H*~Ax;=*a<9R{4X(vl+z#!PwZ{z)_g!;hU~adadDO<-(NA`h*9C^8790poA7RQGI%%ZP@a!a_l|_N7gO6a;=p?I^$gZ0 z&WMGz1+Us(HsM6jebt2Sv#%J?JvI{--kV$w9eBTApt6L2=8?!WHbu_KGlGBQ;0gP6> zYT*3?k67r$zZjheuz^gvUZ)ecTlMeZ%P8vcBhmE2Kl#6)7hhe@;CHZBTzX+2{fG48 zeUj_c#=+spB!BXtZnD4bX_&R!0JbLj7<=+sEuiYp87xdz`Q4HRTdd7wK>QPbfw$h8M zsqMXSVEyp54Avj@`~!ON>5DP+;wclt39lItZn(ojFWS%3ycHH&QDf)>zcY#U{W$PG zRt&(oQHOVs@_iWx-rF(X&{%kvQ2qna@P5A7%8s3mAN$R4^|*3enxGvV!Qh_SL+8Gn zHEnEpwO-5Vgig81I;QhTHWN=1rBs{g=i!~yYvhX_SY_(^?7)u{iBBsy8y9wi78y-m z8Lo4o6EyM^20nEh7B7T(V6OTQ~N zJh*hQ)*I3KL8Zv)K>O#CFX`5q%qzZjt}BhsmN*~lXrIx~Z<4w!`dL}m9P>T;Ii5Yi ze{nt&*A;ZPp8w4FBFvv;%`j)wa}@r3$&4~j6eJ}qXwW`y@xL?CaX|1@$8o^d zVyOIsZxfiEs{J=Kik>}C3O-eL6b;~yk>d-g}?5*Ydb7#45A zYTN5))ZUxK539$t8I;}o85{q{amTM50Z4yvCWlx{C9}Wx_KJq&)K%Ono;*G1@wuYp z4``2y7D{Y4RTpKz9;7Y3h|IrH2=VgP6rza}x&c0|o86tZfO^eeNkhuY-Gv+r8Cd3S zO3~G}EczC~vbEeV&Ox;69eEFptr1P0@)hz<|2u7G-ggj)pG>CoOYAWEJWv<)8j|*@ zSCh2wDfp?ygRho&2I69+X&*;p$)vIP?j|l#lWBS zSqz9a+%bVtJ;H_Pe@9?ej5m9d1$9A&_R|Ha7*#;Kw^MZQlC?T`gAPsok75yd zfnD%ULwwRUr|4c*nH&6dTB<;HOYkwgErm6;$vX%qg(8)BlrIZ0)Ff>D#f_RYB9*if ziW_|r^vXiV#i#`a;jgw83>L#P|kpWbrb#UXdJSTxa+rfcz-Xg-jTWc|DsS zLW50br}0+2k24FI@G1c6N8{Q2)6b|U;ivXen*U65{-fgs591bh=-{ld#zWrEN5ayr zPbhLLUy;8cce24(ZQ$Td1duz`g&ErN^HUAc%R~HQeIpLr!(g8?{8F9c!*i7hLE?vAE86Jt`YRHCng4WaJru9f2YD z^QgD3B~7QlLEb`TWOktfbd9+;a^;eLtDE6VXuIxL3)OrH_WrH#DzuKAU>&38e=pS4 zy|6`Zce0D3qu$?#CF44HBaCe4@5AF>R8P4QIF1J=*!_ zv*2TH1n8+7Pomp#sRIsr5NrMcecC|V)RaQqr!xkAX7?y=*c81kXmgdPCk5)h%vrNO z244`5H(bGZ`bmZ9e6>ZpV?+KtILA&dT^qDtFU!Eul2!zF6m?c%I;qd0vIh1hcUSmz zgJ0M3;sFHPEx3Xlavi%8GrTFZ6XczDg>in0)fYr~93yyqYK6xt*fBRauuE5ZqTzw| zs_w*%TjPbmOFSY{P_m1 z$8DGc=jy?zCox+4{YnH+- zR!QorCh9KvZk?4w0ACKwmT)8z`QSaW`rtfNM0?{W*w}!5$s+FfC!EF0FF@to_i@Oa zpK8&rn8~g>ZYC~*T}DOrh5DO%)p*1k z5^o-1nkE;}G}_(x{YVVDYs2Ad_zkQ6|6~AZtq~UTH2x0YtWYYqWdb#Y2ImZe^qEX< zQo6Wpm{ath1Fzv_Z_)o4)?|1qHi8kb`XEaW(O=C{y?kFWt+W(3zv>ypPOo3?zg+l$ z0yA>0paw_A-?0H@794);6ptv;TDh2PxPpHfmmbqs=tB?h9q_-G7`POb z)u=a1$CY2#v*{bs$4F4W6>cMS0`8I?+ zW_#N33(MBiOf%n^kZ;2eJRkZ+)}HBxfupyCZf9tB`(<&_?xQiVXKXd>UW*sEZDW=Y z-ieFb5Src_7Pnz!nKQc0prQ2A>zReV>1T#+uYZacf@-&d`ZDCkFzhnFG!-&M*adjkm2-Z zaoyM?bmvp5*w|V6tV%OeSIXqw#Vho^o?F+>biyGNm&%bI2;Fgz=;bYf|C!$F;eT)M zE%5(L@66n~(#4TsPH!^zx0d{4z6G9=T$d6fugS}ghNmk(8lG3@N5iuuKN_C5=f}V^ zNZnEWT^`Nf#$M=j6lBnQEA#I+96!_xg626p&5F{L2NS0c&v`I6j_ z9~#~%ioa3kI`B1C+w_JIS3br4-Jqe+5ul;u_C}k24N=ca;bgGWqu$+w`}&Pza9_XB zqfEEc)G>YK==eIp^n1n{3(0C~vSf5T|24XQo(75zM10_VIcY*k=(#~@+#g9)sEUSi zq|T(~|JgXSewvUYskK$uva*DJ$j1^hEV^_Y)=|zWKG_wS{(vf(KcF4nY_WQY9hPaG zmLxaZeChDmfgV$sjwaKW!mj<7<&)!-w~Qwf>*BVEat#g3n@PgVfvC+km_Tg$$*f_6 zEY>thN+TOO(#)?<5O_wOZ>$wYue##Fh{f0jKs~9rG_3>VQC%8}hoj>8oW6fKf84({ z1%JAK>3^bsx99(1{}S5wZ`dF7uTazM)W1UBKiePm@9zBn+`nyOWBO-$!^!Zd=TXq0 zLiMifS&~|w1+}V04#1kuE#;Mf6G`44dc2bitG{5Ux@sjfqWs$bv%Ft@dt74m+u2Xz zYLoZd#6Ufy+#~X{CyV|InRdNfe!q(FT-oe>i~J{V;PTA_&d4hK1Oyr*u4v0QpoL|N zH^`eERX7G<)f)KJpi!JpDzAO1qFd`lIKs&ez@3;bNK|^Il z#4%0GhKYEbwXjPJ^dPU!$U!v`_lfXlRRw-Rp~M>Q-g))mblt&>(6X!}tWBHfSIM)3 z0W>jyMwlU=;Ws>3)|A_#ok}9n3w2&We-YY)otXd3e3DKCE+^^4UH^m=$n$6!`)yJ? z|Cz@A>%==sfrqRqm?6nEnTTXwrzd##S~#FH7s=y;BZt5;La(>lOeaZ7wE>AZH{R5- z_);PD1>dIi zDd<|mjaUwP)af~TB%B@7>AT_%!Zf&pf6K<0AQuA|_`30}L5mqFIL?MLJF$99WMgZR zv|6lAvQ?zeXUiwKS(NgQvv0o5(3R1au3)$n?eiP4IasEZqQ5H>fK@QbKYh-h&Q6}k z+9*PZR|0~w8gsjCU;g)f@#G!W`@-(|j^56yUrnaV#T63zvxoPESF-oz^nNTLysrwK z!_v$~Dl^H4LRpnIcm$x_RLWE^P4{P)SQ2TbiC4>Ur$fE2pJVr95A65od}v6qnzuC8{Qkh>rRfGmc_@hwC_yZ5~1nHJ(4rB12C$j_4;i`HXC|&t|kD~PNjIC2Wei? z0j#@FgXN{d7qxJwTD-4Nz1k73iFaP+9(D^XE83f#8$p+VxSCzf@M7LYc!&kchXQd< zYk_|)Yld~R)ke-MFx!1!YbADCGi#l+X09GVYi8Js`kGl|il)hJm>CMFF5Fc#MEsU zWiZxl9Be)Xhu}A(RGBnfk4qREmQX2Z1qPZ2CoEM8hD8M^ETZ38{COT#u&kVKL=DUQ zeAjMRh8l-ieKCdp@`dLTFmj=>V>K*WZ6>RX;;1BVV=vd5-)qcdotbPjlTBuFrbnn#pD} zshP<`X0nwhVLiJO7{C*Zk9ISeVJ0)pWR{s6XeM)bQf{;%u+0w+{X_Gf`ehgtM5v%v zwWmYODqae{i&GCrlJ_1>6QbxXg~(%fliG?MxF8>wG*lH7u2nCniL5^gP zeNSHw+4N#Buonnr?G9FKwej&f_M(EnxLcFhi*h5!PH~S$d6`Tx%3Zip1!YTxWhaDUxhb zB-x}$a9~AmX*+kCBtVg^lwjsdL&dZ;kfOee3rV|h8(y9HL_=?k#{~fTGwnD#KhC%`T?p^yarMM?f5|5Q$zS1-+WWZi z;_NT!g(4|C!*5&lfp4NtAoy!*EuFacBJ%U1pttuvaYfHXltVX#*%m+3g^(%q*EZ8X z%Hjr!`3-DYMq|3pmYMwlNS|#8QGy#NFDhi7)5zdpMsqEv{4lb~zmt~&9NHOYGuIy69SrymUi#2Md= z7PQSlbg?kq@a=CfNT_5Zu_IytgVcl{io>t9^Pe=3Vrq^}0j>>^O?&BllWQBFg49#L}9%ZasuPv1jfO^;h4sMIbm^d!cuNhlX8=Ul$&&;+#nX^j98QtvGAyO z0x2wlKMBa8gOjcDaE1KxzVZa=$!O1>mu~2i`q4`U;K{`7 zE8Oba*%VJ8SO!~EQAc_!Ve4A+A`%UX~#GJ7LS}%53C=D=M*0{D^_%B|^WVftYUM@8LNIP|EHUtNqzH z{V?Ywa%#41Y)+K-9)mewype;f5Qc*MWTEw#0M7EcreTCy(T_4JDX^nZ@y+md;vVdV9c+u}?IEv=?KKmB zST;`^VYD}owHMH*O$?yFBE4V{@2Er_eV@pB`=BQ3fkeid_y(jcJDpX&|Mke0~95{Yx8jmWg^TxkOaNTc>f4teheGgD%u$nK{& zF(~@RYcU6Oz8237k)p}4z2qSGJBrC6GrJ(eItrjvi!Q2khIM}bGz9vC~`V8aspeDR3i4_Ps}mkZq0VR})w(mOwh`E#zmp6IIjGJ(7F!!q9-Ou$(haSnp_W&AwT z0$xwQG|Khc?ns2<7+|153wpHBU3->DF^Vlf`b!1?>Bepbkd`Fs-LGNYkFdZNEz8&t z0!N6U`M%y&C1?%f6VMmj3c7Gr>g7GQ^51Q4b^Ns6EsOUHha!>1^$A&6$u^+pHuUCp z7y=GYieZ{5Tu0xJ-gl_atj$2wJF<;jup z)I_|^#pTHzz$S|Rp|laE-*z5dl&tB`ykrI8WR2#2TI2u_P=2P<{p&-YL+ef%`YJG7 zz}MyMeruZ;*n?d~u&f9nu2+hIzI3yZer11(wubZlm1VwfIFoKEX3$()=DEBI=Xp0Z zH}n)<1{dViyB@d<_GRJ)#C1iJ^Tl=2^-^%wEGe<5h_42>(j@wKGYEN<5UC?Yn1+Dp zp9@{d7pq+lD4X!=Pzw51An+=NB$AckgBTyAY=z)c*v#P4AzzuCTM41N3UtA-4eydU zTyQeaJHmt@m-b_m8hQksxG@a`ZTW!sBh({n{^Vt0TLWL5M9`5aCXxA5GUnVr7)InI z$5jAIAq&Y)ZM8vgdi6~cYHzVFGT*-wgwWe3v0LOU;vXXu2s3He9E3WSD$83rhMtdu zI6jxQ17Sk;8Bv)*XXayTFj+O7Z(B2b9{2LD4i_S&Q_YVlJ+&VHw3Y|ml z$Dc=k5v$9TtlbV9CSfTX&^65S zf2Xq$j{inD?zOC+zGtMdcrT&PkFy?>%Jc4kUVVzeC@K+s0z1S&9XUkN=j#BgvgS;z za${nQv}{{%*x}|KXCdI1;o!bL!3S4CgWKkPBJ@{QMfy2JIcAv0zQV)%u4dp!>jXmRs;z4~#9-zk!!_OJ}S+-30 z6kch*Z~c+I7y4Mv%NW{;ygkLl&kH8scYT6TT>c_XB|=Y93I;~$mnskAciOyc?Y;J#HMb5rkY?2vt--q{i#(Q7bnkhRI>}BS zTb9((n{Da}JU*8!{++I;#Db0F|MF3K8BFS0i?2u7=-Zf0hP;;RC#vOdW4_JibB)06 z>rtr#4YubM!OA~H%b_60$@1qN@|6~<<&8vwJ}pH3Pj_O@>_Nh3^G!KJU=I8zl_d{GldnOAs)Y9@kTFQ;j!Zji|(C_&?`0 zGI;9n84A5*@Nu@3GXOoG&miPv5OM>A!j+UIteE~8Nuueb*S0zWms=degzEf!urZVT zsbD|GSVp#^dzu~{GJd>KGI@{U%r_(h)A^Gf_pk-Xbgp10#3-ioVVZQ#bS|Y8>pNKQ zIw)vYyMXIdws7GOobyK7pl~~$v(7s^U)hUSnH-d9>^AU5CaK&@fZ_1#4NFxIgR7Ym5XRtem_8alOAWn;%h+vtTC({76k6A=od z9e>)FwimG%Z7(fLbjGywyS4~LCN>0t2efBHjH>>#2fqagoyksVpY@E#foOZX>O1FA z(ka`gSLS;o^gUDef+N-|Y{&crUW6aibA&#y+`mNO2RnRxB?w%x)IgP$iGV5_QVdji zJ`So>T2V!|qRN1=e-2ex+zKnY{FTrp??0eRN-jf}ncZX1WpWIjpuIw9LV{pyM+E~z zU9J9b5PzED;17DGpp$hB{`}p-aNZS9|=q z`2fS8JG$xklg#kv!R~SK=h+M$f4*ynKlQQr^G-DWyzpP+&qN#G&!VLN5r6hO{@>zH z{Ur>4F6sIw@aOC0fIm9B^8Xxvu6*nN0)MW`WN}ft68_9tsN>K6o$c}GY|hnP&?N?c zuI?Hae{Syl{~7*7`#YE%vrF}kbOL@jL04>FqKk7za&e9izxTXf1>D+^3kcibpI6e-ZR` z;Wfy;mC(_4F2mK59)$2ui-hoj1vrs z6KVH|^+cKyF;Ar4^P%Yr#JH!?5uMoFhWe2_iCbBJabW@{f5jGg1|x+^C6i}FP@WOd zF1R3pL?RER<7um`5T-@_wCrg>?R> zl5PAf(m$9x&-Z{NHxwx)-)hJDCBi!R*4hD6e`JTRS8g474?}!YnnDv%aPqK=30-=~j1J^uJR{KxTY*6|PbOyA2E_ZS%dy|dhTTW0qpQQtvL{Z|{1 z;pXrg9RGmOy@1~Qn%T3`6n^xy_8Q6;ruXWBF;w*P-+_v@o$SPRCYn*Co8B}W(fHUA z1?*|@t~MO{_PvJa$%Gtr`I8K#0>vaIb#1SAeJTJI{Zvw1HR0PUF=U@_3}*GPdzUDo zGRNeC^B5+--Z2)F(XO1Y!^C_p4s9XID#(GQ=Tv^PEV*edfNJXR2CDH(a=!Y` zg?_czv0n>fS6^x}u0G;85j9)NanxDf3vq%+yu>DWP6_*j9u*S2!*ttL)2<}_F{->g zDcVj#`zJ8rQ*!H=!!zJliBCBC5=p4n=hlTQ^zzqHdDJ_Ovdg$5iEbIEM%ca*OwXl! zb7FxzhUFU)`2%=afHmkS0BdTi0j#bzA(r9mqc7~P%mWMa-NM!pTNDYt(Q++gORmk` zLC&mjbm#pmfi6fY*#zXca`#PP_boPb%0%8SIp5T<k`KcANaBaUw2$-YGcc8m#_Tpx-Xe8sH zsl4bHktx{$;NFyi;9eC2ZdQ%<39!kbeQeb{Mj5wY#&fLm4_U$r=YyQ`#}RD}uxX*! z8J|$hX68q=lE%5idscbPSqTW)a~;Fh_JfqKFNW5TREHF4F0Cwah}B;bdl%ZcBSFC1 z>HgI2m$9q$vIM@^+Urv>?%$Q$BB?i$%GMyREJu!jj%tr4Q8+)z)H|}Mo%{MaX`-j( zD~Aj}x|dk>yE2owLKYSGW_y(9s_4^R<}j6&*SdVt774F6>74!79`P>F?|_qR_#u;D z)$OaxD-+PHT}hGHt@CQqgh+^y$BEUCqRK*`rC3~5x3M{iEVf9mf$X?87b|loOP2UM z@I2A_4CQIZ<>eDtc~sdT!Q*C2fAnS3U96^ku!L!2Y>tZ0q34G@NJ;cOrE%orb}9Ng zeT+OWqq7gNv_^xTvkT8-4GvPkheNiet8&vF#1@$?z49t-<^(ry*Gd}NqT@B81bSh| z==K>msvOY{{zN97eKu;D$|fRsw1uJ304D@OU>EV0-t$x zG(By(Wc!dpZshBR*fti-Nv=x{SJ!mHCI5O2`YH{tVP?Wi_>7+RD9%&(LAQECHmOv; zSKkr4bTfsJ3!$Twny#2aX?%@{DU=7+bjH+VOr>KA{nOM?_}hF;b-@(wH8qG~oEC8T zJ4Jp!E5bBRE~nPSz#RTOm9_p2rf-Hco`>@AJVe6@<~x!@KhP!O4cVFG?|UmNKZKXp zpD#z^@>&i0uMNG~X7$IQl06sBLWe^NrK97gD2IxVHJw8r5EWS#tG@}=_(u4qlJc%z zAEx8hS1#!FVJf;l{Tr{Zucd!|PuIV`M*sS@>tEmK{`oEa^PjGNexrZ>cK!2P`-lE% z2PPOV(B5YokvzBZrL)+roYLJ4f5bflh=G@o@To3++xMkeRQ)UQh%oFKelFhcR(+G* zYH<#ldsi<3~N%@dW+O^H~ZIO`6 zdsn@P~m z&;&eer#Pu36g-K|pA0Xq)Z=4A#mBQGbpxtG&bjpk>4&~I?@*IrV7t14fkk6 zfyx+f4gTo92-5dGu)u0}n)E$)msW9t`8HxY&Q)eFmhgBqdU4A_9P#Ok71l3aI>r1YdfXOTqW->aZP;NM?{ABsMOYV5yl zX^Sv?WbtgUj&87v&&vuuHrXP-Yv^9TK^#3i?i>KHEdz zbc93r$!(keT=^qgV#;6hr^;vP<==`e-!*=Di#)@r93|6{89#I+ubqzd9SucoywT0w zk4$qWHejS9D5m@GHW{OUa6*GaF_!C zCQaT+m(p1tWja?nco4Gb;&N^`vbmBGhdGcrKTq?$Li!LjQ6+u#;cZd=68D_I@m>-; z-owW_q5k8ue`Y*c#(3V_V2)?j9yT6nU7Yc}#V^^RS6ppk0UYd=4< zKic6yQ-`b>?u##;#0_z9#WC6!ABLaWc`0mP{Pz8}Nc8%flmpz;5}e@#pKY`C{G)An zl+UW0OD1%8AHAV9_`V-X2A6lrBAKu(k zsC^K_77oWA{(1jxvGMF;p3U>2V;-~jVmuVR8mhfN3C`zC<;8#p+K&fzc)LT^`B^;Q zDxU91JN~E!?ZYkY!^kpPkO01&5-sII=%01C6Vq9VfQk7rr=`(kM^oNpo|i9jW#L-0jemt|um-ZmOBmbS`9e)$8&cU|d2>*M+Pi;Ed zdJnIS!he|Hd3gK7bl58Svn4*`GV(*VZlykpG5E>88}o}NC%Uh-5c1!2d53&F$9oBM zZTt~L;4@+Ou6@R`gHJoyiCvP|P5P|8JDtp`LvH0ex4bpO>%|jW=?ge~vinDyKQ+Ux zbk9a(6t|KOXC(GTQj0RsLk_a`nfM{o{E&qovM>|~ei*0~zmVRJsbus7P2munvx$xj ze)BMav*EA~&Tvgz6uk_zP!aFYVjQkMd+5^0dm&Ua@h}ekFR??v^_NauLVaX2_j8J` zUpth>IO(-|dw7)tcy-cFd1ZIIW1_nJg1Abw!V>975lhh@ZF)+rl1L1cd z8b0CR-O9>r(^E(A5b7PjmyK={+ztM0!+ipL?XLY5zB1K?n{_{~>~R9S9Uk&s4{kT} zrc)ou6F=ifLb+W2s?*DLmA3f!BQ0*=OIK?;*T(b%qv+2fqL&eG7XWC(G|ab#=Q~39 zYL>4Y@aM|U7+@_AFXomr%Ggm{=SV3r=%BU#e?b44e=k+gf_^In_f{u_Xf5k7v@KAX??!sz*j@3kDJ+G|FZ_Vd>bvu3iY zmd@9=OY?hA`fgM;eZF-^HQU3x?jL;vC3RUP-w)HG_sfDDIL4QwSt8D7?0$&!X+xz> znZ0(?za*BEYsldS6Bi!aiTI_gBOhMMqfJ%<8pU7f{=`H_#o9Xl4h=tv9+$5`(>}W{ z@3Q7kEw8|9Ok0#cT9!ZQ5Ar9(#y2wJp|#L$_ZQl2WrBX78?7r*4UyCufzyo(w4;9X z(Vv2(Lm+(+c%Fhxs_8?6)j!4E;G%?^ITDTi9=H7ie~aI~-DuxuwvR9A`G0SpX(y;O zuee>Amf4I--WEE(bzJB(Y`;!YaI`_U*8bzPh%{mRe)xXa> z&3Cu5Lus@qLAaDpC%AIA6)Ia{3nyV~Sq;W|Q3_uX{V3{n;f{~NQ(a19p{>a!R%aH% z5zPNA)aDA_XUkWPxN`RuqN{V)>U^beO`+{8k5bZz@lagKcOIpzF5Jgh?@+vCX!*5jPtA%SY zTnoJXq!6P8e(wr;IzaDW_{(gt$rUOZyTe${=-;;R_xVaAWKT;7|7&eRXjc34F27G2 z7=6EmzI^oSt*G_6ZY?7qHI^BR`d-JrH=gFZA+N*Z^id{h-9WCrR%pmx?81#-cVC2Q z-G32}BWAbqLliHJiox<^#k_so%9B)xG>%B(*8>&_Dxo3XW_%I%-tXQYxzoV#6Q_wM zNTRUhc@awD=HU=pw66~Ee&=V;(>qs_+e?Dyn<#qD?PzD`2;X)FAj82op}gTJJX=kvW-KY!&jH5$8j1UGK*T?W&y z5jLph#G4sNL1oxS_(ARrP=uXh77vTYA4gpL5vw1aY`GszVfUM8(!Fdl-H%4u8@Ekh zGTni2KvjGnOG$pdJH_yCo_P-;tA*=^oTVB1}N@a^{89a^VK%%)fDdl3<1*^h>w z@YC6!6aM^|rWe)Tet^aD-p*gR8s!Xmr}e&@%_qTyF##EI=sU_E7(cMXdxluO z92o*2;B@yb?>W(-v0@3nGRNJXe@HTp)A>>RC&n52O&f8Nlv};MkPKWkg9Wo2a6^hP zih?CviZR1~L@<1T8(Fyp3^(mDHaxSxMzp(gT2R%FTpN0n=S&ZWFCXnBl&dscb9asA z*B03YZ*jYhK6Jv;(SO{=I$AbG@8|%ObM3>9?u+ecO|yAeGx{eV?PT%gFpc|iNat^@ zkPB~(wkI2(Lo%n9#dwo%^NMh8bgXqqVnGusTk`XWeB3St*Q3*n*&|yvCHcx={>~Ar z7vr0jO$ik#-X2PmxNc%C0K_#G4&)gcryHmMgN@9 zZ%(;IKj{lFuf-(NJv#S3I&%tW_e-iGDSM5LNOu{7X`Ku6b+Ia=81%eykFgJ7Deuil z(klh)J{0_CTf|xx^=Ura+XFqOJ8MqZV_Z}0{=SnC-4bTBzCm{F(|Fs0E2l-Pg5sfT z;_vsSK1gzuK8WQzTX!+^n{N@A#dl7O!blHEt*m1^LnGS@dTu6vVoFcz`i<3J=?lfC zv0^jyVn%;*cOWuGm8Uc08F)X6iOzWPw5~6b2p8`ox;1AH(XGYsQ+rkib`bmUMXS6C z&Rqk!f&q)Q$Is2ic^iuyn8kY96}s3Ijlb>Xlaq3cGdUB(Iefh>($1b-TZbv)n0%k! z-hYz+#ym&SUt1j`zo!iHTPapkg2IH5V7k4&lBsVLRi4z1u>6z4ACxb~0i!&2IqKP{ zOPIQ_Z3v!QFPkd0-`7TcJ~+y(i@6qz+Ba3U-s_QEzR+Z6qJJ zkyrl0tcHI3HZ3~x(-GYc<)GI1BI{4nx5$q`BjLv^@oy|;!XH*zJ${GZEvj-u!qoyo?K zg^ofX2ujqTxS#|<35X;l0~4JH0ufwL5Qr$EfHQz%G&mDU+m1#7!5zf~{am6*5`sVy zmIU0wBCAM5g&u>z2O9RwyG!*llR>}#y?5Sn&VS%!s=BLdxpnK-y;XM!InOxdkpB&% zickjVdJaDK8FgPjjCV>Jlad|v8pL=!wRGO?KYC-l1uhrM{+2HzKI!Ar8 zYuV}qs(9;0a`qg0-l@MI(!a%fO;HYErtCoZpl!H*7x52(-l41x(;><0MqQQdJL4^6{AY?s4APrWRv>Y40F4`l zyFv{2m$&rc&O%YL4@!@HjDdP~@<1#y`dNs|19#j4pm<4IYi z5B8&|DpQ)jjzni6h#LDTjDJ`7@o$R@X_fR>Lc1z???Mo4-2ouDO>++se16aQ_-J6;8U3K{Y zgpny3qg&_;l`k?b3|o|UgekwoR37s^Lh^lr^D{Dn=#xCd5u7hi4$>D?<^<>E@|1|6 z{)N-`HPN#*=ZFUtw*2vW{-T9`aCm*!@g|uRX@m^HWJqRnHw{G$vdz%7u1QvMX zo>2c6K%LwViM!eH>IXAeF90_;n>2@lYT}V1#&ym_vSJY?0*Dz;UV&P=OVC6Y{Z#zx zN5b%f+X%zA4Ag9Hvff}5UCmFG7Htr3(S+OCSaN5F)&ygMJkKtC%J4jU_YdSypFGbF zcUpQB(P`?s?XGD|wzRc*^W~Hn25b_dI(&ash8%&D_9# zevzyjSig4Gs3=7yjYRLG+c6<{k3J9QhkP_?A+EuH@C@Ya{Tyv~94>o^J1nq2*8>Q!3Eg zyt;u(yEbE7&Ybk9RCgV-k*523S2tjD`=COI6;FqrO(cA)aa8}_!;!wpecv&&K1C0y zEW%OU(8nQHOJxJCknG`TT_=^@+=_qT<EaIQ;gAN^cuv?HzT0|CyTzg8fDF<`u68I-#GY`- zPqaDV@By>!`S}EzlwNVvy56(@gs0gyZ58~c1v2Fmo?6?jN_<56@muA1>$F4qez+vc z-5GlFvQ(DSV@X7D8`pCH#PSibc#P4edGs1+2^&&I#GZi%XTs<1 zzQ2io+utzsJRZ_VA}od+{2!jzMS6Zd-r~G2NIqrhvRRi-q5hEb!)PwDS1)4lpTBao zX8W2Tc~y^x6mq^16?DE4VxMESuepQUJLmXfJU_1va~^rl^3k53_x@MPxAOe_REzQk zy(5Jc5;hww6uw$)RX_+OZxL0U)rTZVROHB3mQeTLo3$ih%7_Xl`mm^Z}eU=bQ? zL({^>P<)39PSoed*OBjm?wBr440!w(&qw}m=UeEj{V(eG{gZt4zkL3r9ZMf0!Lq2y zl}jfW$JO(1i=hhoOThCh9i;_@!YC2CPJ)0iF z0WSNNg-Xv8;akPYg{RJ{x2u==1IiH9pc6V#%)l1`J)8PS7KJ=cK!{!Xb-YRYy(PWL z(e>?B^)8F1{?JELf4J79{&1-}eOv;m1Z(;arpOYL#*yDaP3<96-&w!^@1p_r9Qpwn z`g`N-lKBHXPjE@20Vn*oZj3kl0E+R`0S>+2?!TtRd#OAouM**ypWX_qmlmL^3_7qA z-{;RhWz`}$>E>J<)f`_%NZu<%GD`AZBa+dQ_d1b`k-U948D(QTeMm+_av&#TAUT+m zkx-daB%!j~L^7J=z+*DT)dhZ9*;b%!p{r9+j2MQ=iZ}uc>0nm>60JV8Ck|Z(JABbk zXAH17tZTCSJsRbvn5LT^6qe>vrl&c^bz$c0W|YlbokMKEgq)U@-1uUCiE@_>#HVMF zr2pIyM{Dk+lh@v+j2KJ$8mTBBjSW|5bY-`Hs@Z=M*-jDR=XzI=Y4NQ_p6o1dGV)QF zzsAVx_L3rxS~UJPMxJaOn|1kN1qu%)d13{ceUkq)k&ngvcP4*DzJ$YHhPzQgM)lTp zq590h_tgQ^E=>#2s;Q}s=Phu9h4#WsY2iAl7^BP&@%2hYCoyAcjcey$E$rg9jp0Ci zY)LaJHQsqP5U4~biS4%Xx#mEQ=ZtlFH>o@;B0oJ+^3Jgc%G($xl}8QFPtP9S>~dqs z(VX=8;^^SzSqVFaRj>m91Q$(Msz5xDJ_t;n`>7u zg>E&j#rv#Ex{&IHy8)hXYxSQG^*1ht_a#fi>M#qI=2p zc7mFu;>lL@=6_?Vx^^AKWL`+~1Co6pn(KDTV?iVzy+rap8H3t2PI*Fct^HjhZNkq_ zT9qAH_HP!$#Mh$KF;LFN#b#oczh*4 zzIiD>j^W2e=SAa4wJre&kaqA?u(McA-nLU1T)z0Y89vHnOua(4k1YH=EI#h5jfRa!2rL( zRX7L8p{Q)3R++mT!1fFPqVZm&BQW7HgzG%z{xN4+zZ#xn2leL;@f^n?dB3yhLy)}V z0ES{1Qqk?C!5y83kXGEqj&yg4?#?#$_CfVn>r(r9fAa-FtzhM&gEz|pX3`l(M= z0~M-`Wa`2WYE~oVRl8P;tplk4c&k`uwRVai=%~%G@h*FswBoPafVi!M;`Y<42)D1= zn1o7FF!h_N8MmK1L!TY1!_S@x{p=S0?3)eRXRnIS^7yl_=(FEKKWoRI{ic65PJDI| ze^x}Fy&wA7o|W|3J9XM;SBuXM(Ac8=(0l*a0Rg(&lwNw)DDwe33c3g?D$ZMy zS#c<|rAN7?hoFPfs$)+16g0LP7D%>Kmbb(|lR^2z$6>dw8&;yC1sD0M7K^;nV*Fe^ zzGX@>u1bubKP#fo>ROEdEBb7U{@K;yvjeB-v+f({sBLg9TYt_d+;mVv> z)4D#4K39q^e>yyUEv{&$*Ph3_Sw>6qtHSv&Htp{ikXrW=X>%&MiB83x>{JGjzK4?0 zjR^nqBsrn6U6eZ)xh=a?UgeYnnYRZLJtrdWyu&q6DpT-19?Q?GsoqD*WE+d;;xief zz6m?Tr8Ms-I)6&R0k97DDg@9voR#m9z3O7jifmt~W&@)QYp!vjevf7^b;z+!Ikz|) zs0;9K4lK5^Qw-@_Wh?d)s2T?W^r+sUB;%&^)-j9>N$-GcN!|g-=4Z}6VGT&$Vl5rP z!=*Gx1qIr35DT8v(m-0D5^0n&3Pl>ljM*ZMa>mmlje>?pOGBMrEe&-RX=$i)u1L3n zZyzTr6V(*~AjJW#A=4b7s`g?hY;?T_phZb_Lv)Uk-71kT`v4XtfqExy{us}qU?E82 zOVT#v$UE?jIiseT?xQaq)v7T-5l;fi+?0{nc`$n7^V_vo|@$!rWv-s6W-r zRS5X}5~r{G9ccrA3>?wXIhxQpI#7>jOemZyH(+&z>j){|fx=i6kT)8Tw=1zAN?u7W zMk+u+?^WAx#FClP{I!0U@T;cd-<56S>Ede?xfj!LaD;d|5QXVajQO>vHIbhoEzb&z z}8UCC=h(A_Syh&CS z%f~h6M5vV46C3^;9RShd`&}eq?Hv|LSSo*tB&?K^oWzCK%t>5#kr9-{WoP3guDS$~ zgv!nrNvN!YNXEEQ@^QswuB7v(OHImKV%sdqT#55-_Q!TSr~0>O_0NsSxFjT8Gr*E# z{Wi0oJ<45;8=#z9CEM%Nyit7iM3ytTRnwC^r=aJbA;WQs1OkdLlt244lAeWDPC|eG z5J~9oQIUlH9v4Zd;tVHo@KzedX&ihECvos`oW#K=iX>E)B$80s1tJNRb>SqEoK&2K zvZ$`fL~@MrW)>DC1JgcC^OJ2?Cfn>qe@NbX3g~kj23)6>f6FIj->36Q*$3T$+@`OC zk=IE@AK-=e`P%x!Ymtg()B2{<6=VHmt|SziZ{%rx7a4h4-)wAKcr}dp@lfG7ZdLA(Ajg@>pM~4M(8|R0#gTH9xarlk1jXVwi;gIt}(|IEe z-*nzc!#ABb((p~^jWm4Id1I``2Z%h8JJWe1kvrqOG05y3$p>1el_R5qG9PTqk{5R@ zwcWZn{VcG~q968kucM7aUhBRdR@0ajNLf(Hq%y`tSt?57NDr*XI!$A%LA%BS`ep|% zdQ6Iv71f^>*1T%Zi5NUw+9VaBSNZgnaBuh4 z+BiU{8&h-S&Z(@i(jA_NlZW38eG$!`vzemqY*RZ7Ga~p-vZ6WAlX$O+2&yV;;w)N_ zdE|en>BH)~>%~M!-sObGg9xmdiaiWmR~vBslHvOP^007i{*>T~4Ll0%HeQ18dVKl0 z;QRfjJkiIuw?eys29sR?@;}SZ1?L6uUE_L*N5(weXU#&oUhhH-*`eGO2OW1eZ6=7lj^e1NhPJO(cV7%M!;PFZV@8+D3<8|dA z?60Z`#ku9xP?1R+wNQ-ni*jr?D`Xzk*r0Yz_=!FO>5g-QRJ@WfFK-8`IQj6W3A%OU zdx4s3`TogiZFHoM-r?#8joGQO5g`yw+pWI#aU5C4wXNpsOkM2N^f?DHFe8bl9+iTT zrCWk?B7<`*w6wb{EI6h2qg;u;HNIahfPuSr^fd;){~9_Z1;Wnn>H^^q>w^j;lG zNsW29lD-}vy566LuN&6{;r-s*gYo_ai2(i@ws76*P`s~RrLXbFjPrirf4>aZ9?~J{ zqj3Afvs8X`=sMZ=Vap+9Si;it!OjGIVPNPkb^S^M@V^^?-&@6iU+2z%cFzynPxZkM znaJ*2$qVP%%|vWN?00G6J`Fo@SM$W7|L@P1hMl-pAL7LQ(YO%z=K>x4WAe=4KddIU zA@Kizz4>w_g1-RJYP8S-9SBbZcbIF^qwyv5T^b(#f2c1kiQ}W_p!E^E6zTCFnH7(O z(&KxV#03q#B|UxviUIj`Dj$ReMH+(w^uICq@b=zjdi+;7dOT*ML64(C=dbW1v(@qSqPSy_6Hp(lnbZ_>ZF(HXBirCZQ`VA$vJn~?J^_Mg%-e|8`;8gEsg zamo9N1qHW2WDGnxNl)g8CsFX^TX^Dn406_hYX<&Au;w-l>I_$(=`cxqHL}a@Dpuw^@pDro9i<8X=QWW%ZgTbZ}d6fWr@R673BW} zo|nUVY&@XjZ?q5d+l~A<%-0(Ebj+6<`4=(&fstPT?KCbh@{=(?$H?!6yf~IZ{eb9Y zm^b1PMk|;fVdTeRKASJGT4B16`8uB6**ejE~xxvVjyt&-Sqr6GZPDCE%O_Tjw1(IM>{73e|Jzd`~D@fkVH}WKJ-frYc z-W+7)N#49(&zs==xdHD_{}003)s7%d0Cpv6dW02mI=owVW_UXQ-gK0@dvu_|u%AWO zavIz>mHfX3@Ba_*`NInYK1*fU5y@*bzgPdC;>PEcuUucw~KJrdH$BD>d z>$BRd4E%Ma!NZyTHMOeN?})qctUj($xkXJ*d0d9jt0Eix){~k8!76SnKc9F$tRBjp zZwc=TT3MnOP`>W@u$RGCQu)U3H2s$$d zXxG8xnFF1^UY&4g*SxKJjOpVVfley{WI zM|@$&HH^!B(Bi(S-gJZwTt2F3bwfv~vc1lglC~SUjIu?fPS!2x&XxH+c^gPdX@Q(g z(0z9WdU#Qb7vVHFIg~s+gRcO2DuT4Ku7|hi>#0h;Y>|M&S||X8@}N-L2-L^QR-Sl1 zO6mN3CSF&2(O*;M>;UVVe8?HrH%Z0ogwq+4cNTm$`!p-jyg+GIe3?sWR-;+KYBWE8 zz-lx}!D=+=Z_pk6n@f@`+43p6rGI8o9N!O?P#b9QrM1y$HQG3A)!J|`g3UDQBiNX) z=XubIziuj?Vsxo!GVi~CeVAksdaN^T4!O?Zsg7`LQXT0V^E2fev!yu)=@|NeQ*Kgk z+s>yp50KH-8)SWstE%H$4DOIs#eB{!zNqW@GP_PKeN!x~T3lP)H}rkwlpoE&>yXm> zQ8RJnP;xov{RJ1UvJxH8sbz0c6FId-ziU;sT>>F<;u(0 zLrAT|Q&49?^Av1yznTfnF{}E@SwsFQ+MFx@DHR>X@J^9@A=5<(i_o)QgGstu=qKs@ zE^-7AJlyZ*H{vbfJKb2Mt(WLKVB-BxH0%6PN6bHD&^G+k*AM-bgd|H6+ z=JHMa;1&43vr)b)=Hre0#h5>HR?q(cXzD-6`Yx%@h8sHY;@($b@MF&e0&46Jj26Sr z=;QXkq@Q1B3SEz)Gw>QcS^r7;I?)x_$egnVsJ6U*E>QU|Za&w2SMzW(-D9(s4(c_{4{DQ|gQoy%2t{*WCqjy(L(_A-v6FiIHGx*^ z_{Df46?SF(3sIR$=NB?PXMk2bP2}MW(~3_x!?Yq>nGPM(DYGiy$@kSO=q9G(@WT+R1XicMZ0 zz$%awIBGph7{xGIDw3|7xV=;ov(LOD+M}Y9cQRGvt|KD06u&_m);OVNa_Bydq!eiD zsc5rPnOr9oeS-w$muAyB*XFlzZQU9X?DiVgR<{K(quIOw7e5o0_C#Zm4zzskvH34C zpWj6PdvP$m*swU5UaVVeq8Avi81q*f`5zVs(~CWegXsm8H`9y7f?kOHCd~iH-T$>H zZ1)Gf4BfvIC_xD<5~dHfvrHc>taP^!D_H<(^`m;mpevAY@D%!soAtE%>caSQ%%uOi zzt?Xe=UKjjDxx8^_A36|j@M@vb~pAnT$I1I_jpY-l;6c zoe?M93A_a(c+&zz@IS)vXE(xQsThOe4)HzFolsm}U@&RCiNe~*l^rpesN+9m-ucoO^0e zEQ7i8wYd*R(;R3?Qv8&~Fdn&delTI%#Pg3|gwFpxrupXsdu-UhwHf0T`VwN_z$uFl z39XIi_4mFM$ErL*=f(B2!{Gtl|GtDK9vZ$Z30qE$#Y%5uY$=)@7F!Oy2t}?zY?=HG zO)bt4saR_Sq_ehP1_@Q@GZztx!U{I28T0;bIo*KEDPxH5>QqFSt#i%G3FXE=#A?FH z`&}&B@9qv29bphvg<;2e$&2TZ8%qkq+rN|uxiQ4P&E&uT@Kx*;-a#>=N;sautr^0{ zYzOo`jpwB4-y43QoxZ5P&h#4B4G!O}*$(UXj`WkRtE95&*%^I$x_9N5ppg(T0ah&2 z7GJiJ%B+1NTnB{3VyL~M*{>CQ#1TQ4=DuVYGi`zAwdB()Z376uzFCR+a#j?&a?6)* zv5^T$zC1jpHVY3iEa*Dkw*n7fh2#MeNVfQOir*39_ip@MpEVxS6U4L0;@2&H3-I@I z8_c!lyU*l6*(v{PAwOAUybQDAAc+n)aDr+*NNzCfMXhJ8NWtqmqh!62Y%r3lkvw7~ zPZ&vp5GQR$GSNsT8_5(S*-az?ZiS1{jeP@v56_6!d(g=;^wK3t)Ou#>sV9sI3XP=Z zUtj(57+_yO-E${~&8D%@#R?#sQ@?)$+}&Cb)??`P&%?t1j_o9xBd1>}l?}F5H`wGF zNJLaOB(|=X$~xemk&)H=l3UlC{B4o_v#{-T_;Qcb+6h?hm<{ZI1J&)C9rE|;usfOq zQknhVqF(z^QE%Q6`g?4HsJSJ-K=!XbcLVMv>c8@kip<3k0-xC6NUxc3i$ks!gHCI5 zT(L=+ixUnK#uasrbjabPOM}*fN#jfh{-Q2)x$W$kYB6KI2R1V&y3J|m`OU3 ziE0dW(!-t1g-+H znNrqTF&51V+7u6R1s1(xQ(Bd*aP_AE{?n5))4$>2KDtu>isG#wxuU*tpV|MSsRP_q zXwv9_(Rkx*jlSWKrSch3u(>y@N_Pu-V~LTy|3Vq zI-I|2E0q@K*Bh9@D$RC&XK+uMSKwbDElevVQjZ8SAj#$KgHx4hydT zZms@Yz)r4zv*bO?XbNo%>>qBGr~^Wq2KtWee!|f^jy$J{!P=QJ znVP}&2c%?GA}7@=ySK9B_zDu~f!djKDV%EqIqHrwm--!TSsZ~K&5`XrT;tLA5I&tf zum?RiQ5sncDd4(Y{qQ+*Q^dhi{v~36NM2MyR(;vk&fgxOT zWcdFrMhTdGkR4F%b;$Dzk!@LD8ch@O=94j&oY`le3IG9_^CA*e)ae~EB}bWE!t*Y9 zHzFK-^HHzF>061mXPL`!(4=I((cXBn2Y%PLgols7tMAC5tl z%A?89f^w1?I(~UHRek|7f#a5Ond46DA@!pQZq`D|XvfvQ zZ^V)ovb;`}DET{3L4?YA-WV!~I8&Kn5x&8Xt0%wY3RW9Eb@=AbZX8}iCB zetKNkJYUOA7h(7Pg`{ujl)u+T_u^^vwxEx03_&Y1QA{$Fnu#spIRsUpUVKx6MXbVr z`tB927&KT29eAvyGX0^9zrAUF04DK5@=c+BJ5n56u{!Kqn0)>|Z8O>;$m?-3XDt&j z$MUWZL^sr4f5IGdX?t3gK!*c4icG7`H#HG98Fk20F-E7@w~J_B^v|K7u_hqR{0bUd z5n{Dk+&@9XaQ~pQffn>U$iCa>E+wDX+a12V3LwpWK+GeiO#OKQMS1t0N8(&Xs8x?K5Zr($1%R}zRLfsr;? zC^w>|;>h@D@zf?M`xF+n_O4>GjhtQkZ|Z0`{(IX#sg+LuWABe#oW- zCjT=g|LwxxQ8F~mJmQ=rcDjnh65T-6mDVGz?7Naa~k`I+hVME7&B zS0l5LlkQDDLPx4ISRO^mLhXGmsn|$mbFo=>o2Hrw5}+jS@Bq8Gv(^6&T@Y@y+4)YN z8V0ccTJ!nel&K&vO;96mr5PwX59hVJaX*SuRN4kjD)c>-h|*WSJhCZY9{gv%Jmf^} z;3HOM#0HF}ULqd<3qyJ7QK082tG>9^AJy^dE|*!w_ZR#A}#uZHT-}) z9$wC+$u+&^|M5&S`G4%5pt+Ew7-if+f78>N-yG57{1)c{?CkcDQO@*3u4~xYEv!cL zHFR9<(3zc6xLsP0-nX|;gjD9pUjkbh))r?LF zw4phLIfI4vOm?2>ogGi-KhU7-Du=wF*k}c9jv+nA)Q!d>o!-~dbCJ|mr4cVrzz^LL z)A5bEyU;kd+3^4Jxy|DLKDP-&Q_lQMG^J~fK~s#7o0mNl_Ozz>2|8)qsmC{%k({7; zC?in}tq+0@9$!voOnnpU_st~&R7;rYw6!^N59pPBMDh^+IeY~ARFfNMelBxfK21OCiL)-w9=14LGPC#r~doe1RcIcP`D&`J*SDl zTzEO1NR(*RXGCy6$vouN(yD?rE<9Ax!q{W z<_b2Dr_i(StfwKOb$w>j`iNq|A;K&I|CD7zN6ttv_;d<$Ybm$Z!h__sMN3Tb{$b&X z&5*M)GL;dj88oq|KSYrgrf4qRC4fbV8fUz=s3 z1Nh(xdi_MuSQ@OY>v<5^J**&CL{24Dh=H@{Qa(K0a9xXF-}|_B>48ODJy}$tLmVfD z8K_nFWC_2*nV$Vtspt@VT1Xflg@y=?X@F_f_lLdCxO8S*kl#P_&wrYaF#+apzvf2UGBKzq~nV3tku>2$YYN8 z{5PBG%T>NiMd~DoRIc)FD((d8ua9YWOJVj1K4ZYsgM!C22p^&6-_Y~LCul|O6Sqn; z2SQKT!k6PJDY&WT_`XcV-$zF@)c`ow!6$ekxZe`bFF|CVKwwm|+n40%$X)kjB9eRI`4&lX}-pf0PJv@f*!#|@4bXChp5 zOJjXVJOBZ2`G`X7jW-2US#0& z2JyTW!}g3qGqY@SJhr132FFhu_FddL;-~Ep^Wneu`?zrN)AaXJeVF$l_ge(oZg`Di z!=uqNF-W<7%JHmEl)PU@@+IsImn3k^9v{eKzUbt5T9+5cTj(}ZQ+xdUL3UF&p^m-w zk+bQKF_He8k)K#)R{!;q;$rp zf6J$k6qsAA8VcC0rIjMlr0cdEWxi<5Wj9*l^8-h}JZPxDx&HY<)fXAH7ipEFzJxZO zFX$kLN;%~v^r>Pmg(nRbRtK=K)LHo$I?2bQT_2yE$o2{Zt~z!i#VW#!;+LU}GhKc3 zI;Zk4EQ9eAFV9r$8Sq^fo{5B)FocaS>9ts_{g8Sce4UfN&UFuaD3Sy{%#{zKk3yQp z(v$=vUhv|B#1;jcK%sp#k4Uzc`m6bJ87S>^__A>W<1sd#)Rh`fXd@bb`2mdXHlF|` z*V18ij^bljZ-70m9~wl_`ss($8AlZ0S%xwflVAKhD9>FYrp-pa$oB>|<8eVlV<14C zr1NLd?_&JQe@K@%5o^AnH*-yA*8Xob{c^gUrOAPxVj2K7EM`Z{sDGfQuTE(4M9T_y zpA~-ea!W({QjHquFm6;x!NA>&ck*odES4_q&{yYN{^n`=<`}tEq-qEPhA`%Dd-j~K zMxOiCJFH@oJA+hA;wO=c$;Fd26%%_}bEy6*dh#f08t!VP3<9)sw>LY&G5J>mCIg*0 z%Eu7H+FXR^V+Dww$6673p1~jaCw1NgK<@N4t`1o2awrz!S4VuclD}FnzPjmtz$-MW z^Jr^0=t3mN=M!rfQaEaMO-ghAhDBe=|H-8YcB|w&zhV!4-RikF3XYUn~`NEsp}LK8h!Lnz(=)3+T1}>pSzpS z>7S3z7iKz?0S!@J1t$X!)yud)(+4?E;#Q|M;t-wwrHO71H$o+42GT3T<>N2VvClAz?h7xo>+ zT~*X174@Ril7cBsbFiHVspwsz9ot~QsFa{yeG+>{X}?SxNbaJ_OL;;a@^=o|hi$6v zSCI>p*Ekf)OWp|T$@l8(U&V5NkC8zU0|FSw7f>04v?qM-l3_y~s4st5y=E0kj1hO< z!Tv-&Cn6>9KC1|}H`Pl1XY&A_QK@MG4U#!>yHpNi1)N#;1LRZ_iWT8}6(>t&_7{j^ zXBM|&`8yVioYNmLu+fnSKIxenI*{kqg zK`tV=`!W+*yN3Xmbpax~9Tr`0$7Z4pW1~nH|jN26BqAgqWfv)8Hj-51Z0veFD- zMd(&v-Pnwm04u4|Yv@vU{um-^^eN3dPgV7m`c&aLheLUmbu;Rw-KTs}g-KLoT|a#U z0Z7g(p}U{YP-Ca*H^RRX|J%Bszs-$1n0^G@6hc}#E5U~D8R@8X$e3$d{`_d z%xA3A(cT|(*3hQ`NXR~{OEV)O&{8iCUrj2;)l!)oF&7I}H`uE8CzHZ)jizv1)w-tj zQJ83dN7K24yj_F)6=!jRc!_+joj;SGKpd3+8Mdr?bPDYZ&5zK|kN~;LZ*XV0>MC;0 zJ%V5F<=0rMau2?azpAA^tjSi!h9QKS3h`hg#XZRs)a_m+qaExjqwtCz2o>JaUypP) zp8UC;!Ela~*yv&0C|74(d+(efzsMAEI)5)adOLI<-@6$W=9=87K6*LsXu3Sxr2@|# zfudv|QRd)L6$j-E6K_h$$6TuAd%65|OOSlT@~r~~3J6wEN`jfa$~D;GyElCZau8g0HmEMH`FAk6uQ zT)rga{gROPr6KQ2L*84g!S93fa`};v?}PKG*>{|l+Ha!+4SN1Ja@H4Y{ z3G*LE9Ln@W4mBEDAOrnhG$&jj!cA1L{Z_2Tgj=I<*bjPf0zeBl5qx*(#}khB+m=)eJ^wnw4s zoI;S{r0h-k={;Tc{PbxN!26=Z;50}_xvq0y5Z11cj&WTM=_pr6NZVYUARXgQfOPC= zXaMo3I_JppwVvG<~r5s}m1pvN{>0XBt>JeEfgyO5L#L`W)kZamTW zblT?p{!p}>!bjER_m7H{p6SQN_OCyTrygk0Uzuz=oosivIGqgf_vhX0E~bDm6*%$a zNRIFl`-E1C{0@e?Z^<`TYTVN9Okj9CRVPC5Wz562I zRj49I%C67v*V8p1zaLA$#N3GV?J$A_qxB0Yn%kLW0QU!qi?nf;j}TDmJ}9`ZBO^E%J? z7=JFB_8vpO8@kDm%8#gCA8GtQ-{Pi4o0r>1curYe6LA2;i#EG!X_3yW!9|VUcpA$X zz6O3vbMQh)u0hve89e_czm7Ct{s;6t_7kdW{7kfmWhFEK;wf+; zK{L@zdA(~`e&*}|jMTe^78eGe5-RC=b;u_)rfPO1hj7O)_?)a6?vTpus{qhZsoLSi zUiIi_B&oQ6$nmU=aQ2(9$o(7&%nu-WdlbnVZo|6qYm?e@Bx*s~OV(QkHKv*FZxWuR zPwvN0j6rRV0RQfdnly3gBYmZ}QR64u(E-w=3*p?O2r`9TAx?558I32x*gVMa$BC3C6dX zh@(M2yp_KjqfvX^E+XR*SxJzdSxk)sadB^idQlk-z+kMl!am5LwZUgnc6ENg9(T%wM#C=R^00h3Bs${sx|X5uV%bM|cMB6szt} zVtxpAih;CK+(?vp0qqn$Xs1wUr%1(}LWActg69bxp3_3$IgESJkKuWD7ayY}*-3b8DadZCc8pPD0u?EY=`-cHv z^g9}#9~Qpe;S67m@6+M@FCES`1n2$_=Odgqcz%vTXDJNl+w%#|+k@bIGQ#zO2IrHJ zE{_K1lK|&OO>jO5a2}__`3@7DZ{}W1VmOZwaJ~UbtYSD%i4EmXea|x-PXfGAkJ#T@ z;IDz#0=`pX8*B1QmNR@Ev5m`^uJgPMjct7IobYbDnhVXe>3e@LoO_~w*YRQD+<91> z;H|a$y>r65lHuJvz9qc(k2l~wh~S+;@Sa2P?uGDXyE8M)?+=0bNFC<6CYTT6p4`tc zA1Yvel?mp?sI$Ys+u9P|hZx?r#`OlgZH>!=;B9OCQ0H&-`?fAaNt|dcg6z63k;=1} zdF?~&YERt6T~`O+yJhvz@5&1Cu1qS=h{A8&->^M4Nb5L|yUg;lcE2|yxV?c=`S3(y zG?*=!aT{bJB7sog3iZHj7ZrOd&mG324?$Y_zQ!u3c;?1r1i8oOD$S52Wvqv*$ z+@SUNI;`npNc6=XCu=?Kiaox;*dGnL5z?;+{QhUHUw@2++!jb&27NNit6IOHw`1`Q zh#TA=oVD|5*5YZ_VuXAdA}_EMP4YEkwK-gAoDPyE~V)sx}+KM)&# zn5 zFT&gFtchybR1tu}gVjj$t1!yxtcj4S!S64wzsoHDmcu`fY;dK?{bM*V|%NDs;;G)Td9{xqbvI zbMU?Ds9)nP{&e0K(Q(haq5l8a{s^X0{p4X(f5=Wv{z?JM!A{|P`M2Av3 zl=Qo%2m=*XjytD6u-OOk%xQhZP^a=4qAmjYf*kn_--_%Y`>JDVF%QYz2^yQd7QbjI zKSKwczGV$`BY)Vb%$&#_%t&!68KZKGs&b|5!4nbqMRj;#^4?$Zd;?a5z6I26V{j2r z+Nn?pr|hTqHBNcGx_vAezGY#MGEx#kZP&^;gKd(Fu&?S~N1R2WBz!BLoqixeJR{>@ zhtd{%aa=ui7u}U^L&*7h*WZN!Q;#>1_9}2y(O=$$9F+Ztdiucxib#pp^UB0T^{Pn; zZ0&MLJ@P2Oawt<+K?Uo5?LoY7VI@MS|B)P}oZ}N657axx<&_|6%p(hsC)=vGrpNI- z`ESwX(kxqH1%7RDH8^}X2P&?ig?=(~Mm%(@I}Bec%T7d30XU{&Yg!dW@G!*_Q%Bv9 zU}>D&BAyang0t1(`-?0VL(}4cjXI^vmET(=(EU7h>Uknc1l>5hdJAHwFNh>{h+YC; z09lsGs)AGsaldW+P#;*Gc2q2+^^$iD8j7&yR-)uRjfICF z6@|`G@vOBv`=JUHjYB}A{ZWQS^Lhgse=dtiufasqs(s4O>r_e^E>vygm z$*xIzY9~@6{!?}w!bkFAtT%tMrhg;8rl5sd27}b4^Kt*QkqDHOP@r_pCPnJ0W5xWsbJP9+%87pK5$HrpJV`HK%KwxCyQD% zumsh2592PHTgRAG>X36`S@~#U3z@F$$NBjKi=pq>TJ?*c=n|0b-peozTROU!tx=bJ zCdfi9h?am4#XsqaV?IdcSl2nzC)eSvX+Zt)vxJstPWoVYa%WY)g7<(2rBc!30ageL z-E{xX-zDCA;w60>Wy7nZ>mRqQtlBaaqTsHgwXmeV51`xEVK@U-ZV_Ow)l zIRie2VhP!oX!q|GxRSm(qyy;1JqCd0?;!v!VF1nfG#Ehl?-2l6uRgQ~0kqzk?yhrP ziU3;iDG`oEfZh#P63u80b3h*8g?_;zw7kszlNZi6V&Rku7q*IhkweK{h^~BPtKXvI z7oy}**16g{g8Z^Zh2YssnSf{C3y-4V;=NFe+^2e2*bP4<6D_X9AYbaJuZMT3fSd=^ zfe2X73G4J9)ddF)(ymxY`N77XnJ)Z;)UY0X(D2t%U_P zhqn!PD*3OiCLVg-6&fAg4Ii2=(x^V*_0**YsikRe=E`pFH3fx*MR*gh)O~q$A5}e^ z?xVhipURISQ9g(@-$#+6vA=0pgvs81w10n4K8PPj4=r?fpI4#2&}cZADiND^Ck-~h zP`_?ws7uZCe8NG(r&8PmG#>i4kCCI9nNEsOz&Rrqz48Z*i(ada9Y{Pla?upE_&nYr z-B%;rw(cY@`jZvRMITldAEf=Kg?|@~PkxlKYl%bYC)4+8ez=bXpV+FY z5J6{i#S_%mgzzaSu*^P4HVuF}>O*fdqeDx+90KmsX4MrbwDTMeg9Ge3c4E7_>vzVp z8Q9O`%LB|(PnoFRK9U&FI{_^| zzYSN*<`xVnI*wQGxd|Ci;X__Ld;DJJ0Pb6zAl?NJL%Ux&pf2hT;&Q)h@s6XQSdBQs z-Mwg0%KGP%985--wm;!ar+@g>SIH(_g{zf zUVXnmZ9lJv_a7W2ufDcQmsd+xQOoI!d3t}J{7LJtw@=vq zu3R15-#6y{bAPW5-`{=x&e`8!|G<3&O_sdq=uC}$l~!0jpwmP=g0!&Ua=Ye7N@HTk zzZ7|!+KYbGq~b2LsZK6+!p83&%!;MCni#Zo2ysWQBt-rDzXE}T*HtG* zH3!s>;z%D18@~mOOtOCLZ_#a&l#vq=8t(q;bKjD7KCl?Bs~t+kA`8o6DeB^{$Qi*R zi|L3k+wc|KLMJ(d5l)M{d$}qd4|i#LQ{|WG2JNj<0%?+hj z;~9wm{0Rx{t)qLa$)7WO>EX5-c8Rtw6d_Xa&CEpqj`0Z=Q~#-wIBGIOZ|~h*q3wM! zUbMHe^*P#`a*p<9hH3A$*1_%dYSG@baiYE4bG2tVM|*w7wQTRIkoMN@6zu^f%)$1! z*w41+P<}9Y{+`<@+LW|5<6sv}gh|_ABSpXSy(Acjwb&C<bxPKM#J3-JgzIlN=x;}^`!ucA-jz92 z_x)J+46J*pMPIK7uX#HJyjm%9GIVf#(W<$He>pBcen;qhK7GHK&w^GV^GWl+!!#cb zy@A_Wh0KST{~Jx|H|26@$ zct=r_I|A_N+0$nO>NnB4{Yr58yS4J2)X!^)-jNXIYUNN$DhO=#P=_x%6ke%X|Ds`T zCNh|^p_Hf}g^fRvmQ&&5KZwHT#UAuE|FptiB2yf>?d1z&S`)ZfyYRR10=B#kRh{pJh0qDyYXz!1re z>(qS_L^iz+BAcFUBAY#W1KDhk((@Cr8sK$Tem$OFPl2DxQ%F@u!|QYK+k!9Q`HD{6 z!YC~jJ!iqAy&usTwQD?zQCFy_o&PTigr|&7MqAj9`203ibAn~S+>+xdfbk90!U}JP z@wGO_*G4_~IgW1)@Z`yjl~`Tlm(czELDc;Ty{P+-_tMPN-aG|seWjHkFmN!HnVm&t zd|BEMABQr37Z01)clKtnWbYI){X{I<&cBlIm+tg+KksuAzbcbSdIX78J^W+5kj;9( zOHY4SfBZ2-5;UF9t8K&Wk0yMsTVK7wFY<`>0omCNBuQju|qOKcl|2?||nsVY>j$ ze-=0!eBXDw2||e)#`mn(F#bzA9O22pt?LZYUQ=KWVZX9vhjxbNO65DwYVBh( zoAUJ@`yrnt9al&0qWM7or#7+4O5PeS)sIT4y|g0u>H`a#?LNXax^6b>f6q!$JAA62 zLBxMb0A7}aUyMsp5AF;Ca*Ojw9F)2~On2IojMn?(CDfhwpc)Q`2j!gdqtq|bgM#r0 z1$v%mQHTFKK1@BBNvmENK&yTM{8V0#bkExalwtANvtia-*!?K`J8eIDxy5Wx5VOs@ z63_P9k3(m>rdpfrwQBw~eDtT!b~oIZ6MqsDHpjvTP~WeMH@mPwc&Uxsv@=R=o_FpM zpNU(S$3D=`3?JDM2CyG-so_*gb?z|D^9zDQn8P)?dg6I8O>6IuX^*uINwmS=(Qm z&oBRk`P_0k+;$EZ~N!<+ilIc*YCl1 zsXM>@@ISeJufB=q`|Ex*-`o3X^S$;c%=a%R|HJuC5*x>p8?|GsQY-Vz2#$%gdGO3X z7G~y8ED4?Yrj^Fbzj|fJ%wLV5&fX;^PZ2X8S<_`K ztziIN$)!A0N_}FhY2M!w^IpD_=Y1fp=M?q+CHj2dy@cjFI#JB`ja&bDzWY>%=|&3A z_nCL78-Ky5_?XQ+-+5xbCx)5t9X#JR-bnMkJDukHK)N>HTaUnew?7eXzK@2|YfU;T zsnevRcg``}lXS|xU7zjP4?<_#_LV`oKkgPnxvxQjdHW7AWj;Z4i!V|OPJ@2oxfspoc!?$;t(OxHR zZ*9ER-l=-Mz0cygy#w`O+e-~@PvdTPt`pqt7!765N0jB3Ki*FKtxb4J1Ae>fEALVd z2I@VSY4o7G=)r{WJy`eW*%tCC_vdzWI3?sff<)Fj0f`|w@}3;exVMPR!i={sBg`nfOq1bu z9SoMw&e6ZcYs2>MKp!({#EV_>0iMy>R$x+&&vI4ulKKS*#7C~J=&~Bt`)P| zlkN#&RhjWZ zizU%w-UOR4vu!ke;InnX!!rq&G zloDrY{3=u**q5i03%%naW=D!zlZN3`Nd8PQ{At068en7`sAs?W?Juo~T<^mBU;B^h zAD6^i6#I|*Hzj#Fa(#}xLHDY4W*_QU{svGkfbNa=v#3Ki+@NFz!FA z_506G;qO1+3Az8A@gLlO_6&djc~RK=Pp2<>rb7B?*00IM_p_Ai{V2Kc`E9g67C4aq zU_C9Z66nGZ zb3bg1w}^YXsIIs{Hdd%f&}fNSl4Z7Os3;7dezc zLvdE^$i6u|>qh~PK_MBY?m~a5XkMAt0CRG>!eBCQLDY**EMdTm#YfPKb&SL5&MO^1lQ6{fMz zkZXy&J(mLHZ9Lw`?$}SSXZNJng(>uUeu_qbm&0py#~)_{CVw2Jjp5l^L>+yBi z~G|E$@6YS8&;vUw6MtF%c>rK|dJ zHe7gaLR*X!v;@(|H35m7#*F?!YCv?fpjwMbpU94^DEiR@&h+8W4LRiei&zfniq3e4 zV!OkUaha@Nh5ytZc9Br6GIQl>>o{i5MrS_e_?BAo``K{!X~FkbH>?m#>OyGb$~_eI1v)b2 z#b$I4uj=e&*6%I(d5V{6wwE&1PnGYg6qR43oyj0B{`or^D2y9rVqSmq6TLLn`!V|d zRqgxk;>w7eQ>#l+5vpi6hws6_8|rL$q6I^S>RyKG@=JpIqir^WD#d1V89cC|jmKtJ zdxvso3iWMN%f3D2KN|?%Kd66NeND?nedq>AQ>D2`3)?+PjWKnW8B}*0*3Z%H>R|v? z{&9q*d8{0_Lq-)AMWTZzniU8UGp&UNRf` zOBZVS?UZjDlAp`vrSh1(N>sYZPe&vD(b57mN{|NDXnKZxC&4HYCht&Mk;fb*qetN) z^oh_MY(Haq@2D+YB#iP*_!U||I*^;E;b2C~_KG%>Qx@0tx~3fr2#w#~c{mn%Ck*&h zsMyd!`Eu<#MXuBB-Q>Dp|Flpawcu(iAfw(1^^A53SvBM;|Ly13dnlp!{U zXE`|+b9W4H-9-BOz~8%95v#X7NB%*b^awfD=#aTSre_#?1*Z=!3=X4;a`hDQ9kC6y zn%7MxXA!OqNTVw_`g6+ z;2;Gcz*Mjs%#+T@V$z{a|6cO;$AbZDbxwMnRCa@_Yo=6|n9*mOt7ncc!`gTr-)mM` zrLv&`eB8;AUgtjK@YGs!(oecK`U64nA_w*;E~%!O&+1WFj3|-f-=q6o3ySAOx8T>h z+H(B8h4@ufV}$1yt2=pxK6iSoSwRmE=g3>t-Weoj??m$W;X@3L&N+%*v}g_blI&`B zH-66gI%2-Ny^0jrS0wBq0c?Xek%$dC^-4i+D7FQV zpeFSmezb<#K>I?4NVrr!#irgz6}vt_^T5OEmzk7%1)H4Mkfw)4`BEhJIMP-0nQH@M-EqXa zuTy#Oj5hLKsjL*~u6k?$J%1Bh`=A4(-o0et(fHgs=PM(B_EViBBVomh`0>i{9>ADM zWl!Uxu2Q2t4c5ZSM8eGZHGS%(BJ}6G${Jt@ci~aBZGoE^gqNh9a7f*GIrVIp2`^)K z5Vc4^U|dpJNg_%1rBNvG75AdIA7T?fwx{$PhQHNNcmuJ!|7s$3K0Z2p;Bp{#8?mCm z_P@KoLVOj}AC?~mGqccqhQ$Mbiio{@K>h` zb{dF-+bi(gai$HOi@Scddw+!?mb%tDmG`i{RhX+>}b}~-MM6-3H#%~VN`AwhuF!P+r z>cbOoFGJa&u;4K8M{{w1pz^TOd-gSBSRmSdQ0okt(2`L8GaP>jM8b6|?L_WX%-{ky zq4u0WqrVmR=$4$L!GFQBE-tRtl7a&}Cx2?g{d6z3iD$8Bj*XEY| zXxX1%(#Ixw9b~CkWyRyV!#KQaTQbJyv{wWQbWvm^u#q#`Y3frB(}xs__4KN%1WJ8! z69KU!Ej>pV_yIghX#F*-V=txgZ$;z;7$9~%--RAk1E)E$sc=*TlJH9@NXK6Fi_g$U zoa;SoCjWe1Y7g>Ko4YDX?8c*eTahrMhrLUnD{@U*6*|lwj2`pKgZR^%3`&DKqQl8A zXfq=ot9Wfu-=Z0EcOsOaA*E>4)!1Aajcr@p)*ARb7Jkxo}J@ zTTZ?$>~GZR<7nPq#Ps817yw?x>y@3X!a*}p0Xok?RYvmH?0SRtP2c1<9kq7y-j4UR zXjR5EBk@ae%7EulUB|?-e+GM@+IkED64;UT*xtr_V=awW8vX{zFW`i+%t2hhE9lLf zbdb#???nNQ)Vu9o?&Bth$6jo8DE4AUKkAZHG)r>|6(XOTf!;>|JsmKo6eUmr_xA&bwm>qUE%oiQayM=0)OSCyE+!Ex^&( z#QSHl@acOvGXueY(**>6JRTkXMg0F@?M>jDs*qzkmA6)57)=0&K?_Ns&lsr!ihJFw2vZf>P$-newkTpjK@P_;#fNMU!b$mu3u>@GPM0mfv zl$LcnZY)w$Te!zyZ zn#74g43m$s6^M8Z=^f_(rXye8JDT%hKXJGU|1P}kC`*Csd*hYw zLh^6)hMXySPA1nBYu1sh-D~?wbls9oi*rFVZXSN-xH9lMpLx*b>K5fC0gqXupYxP%3E0w2d{5iSD1x&RaY+iiW7Gkp@+4 zVShabHi;4hOW@bI80mr&+RfS>{6&vrL09nL&ahmF^1s|I`HBJwfnVW^N%?Cmfey$- z6Wk|5#!IhM#vck zc~r}Sq^TOm#XY%rJr=iM@d7T+=Hiwf95EV;&tUO1F7Cm_d%5_wi*-Z!JH9}9 zdt3BCmGt0JWMznvI3FR!AKuIN>NZ#+!auI}aVV7b%m(%q*2 zvM#T@2RKS8hg^0QiKzaC8tfhWiy$r7@I-yQuNEuO;rX&>70Bi(4n zgRt>cw*nCp_irYEP`P+37w>{URkvVqE)+-LL04Ln`1(q^RQxWcj^hFL!<&2f+TIS3 zlJoMSrzuc-wBF28n~*UuY$r*h8{4f5y?)dZ$W021QR!$FTaSrWAjH;4|uA#uk>lHIu%w z+B_FF1WV9p5H;9-+n{;P4j8dU7P?S93PO{?WVU|m-#=|@4d`FTR&!KrYFgLYPy)u6#Tx=U$WOsY}cz@%Lfb=B?b}UOBphVj#cR${Xh)7S*-7aV=obzYr-O zly(901MsJ63@Tfm+Sp3|$l?xK`x)^Y4gO&$^S_1;^j@U+V&gT`fnp%*EH9p1Dz9jB zNJ!z_V_4GV9qH8<;U)c%<>FPi@z0s1h^%p0+wxHD9H4PMV8!8F#G_xnD!J?{RG<4e zkYit7_YQfT-TUL5o!54dopk4+b|z8brre00i~KnYN~OMJTc@x*ly8|$q9u*xp)A+W z(v30zv8e^}794Ykp=@grDj8iiGZN(KIuX z9}xA=r+JKn$-ZVZ6U48Id^EFIf*jG0MAL15S|{RH26f}1PQs!7ghSP8nvMSG+bt{* zD~NK1qzud5am5~f8RLu8(5O4Ho-|_AkNBcN*KgfH3)vIvzz6q0y@-Ao9`yUF7=JrY zX(#FPZcpK4cN;|}z26PD30FEYJOiMWMt@;=KfV{w){v~s`4f*hh-Oo5;75sH#sPkL z*??azBI^@(6{Q(-rXsd+TtnkKR$7E^a0DCC_ zVszNiqT`Ne;O01jF%l7;4{+X6HFiv-v99lH7iE?%vS`7G zM81+Dn##19!q=Tez1Z5`v&noK8s{49*|tmngqT^F1}El;$V@?b%RH2~v^Pg& z7x$h>q07^N#s#zS-;I_(HKV_PzUuw#KFVVH&|*nr2TM_gk*E9%43fPT+%qSd z6c2j;69Z2EYOGMZnDVu^Ew4gBBdz+c;toEkJHjpcW=<4`eW(eni+We@L%H}_3O9IA zob9SBmQRV>&*&SnK(^1>jpwws&!zj=9|n7pxr^j9J6M`LbdfjklS!@kNj7$(c~3Dp zzeTxY^vZ0q`PO6*%`TpYG~4+pUg?NtOHTBdc#`$ZuQLxhEkh#L*xTO!J|FIN*?x4^FP(#{Wygusl`9$8(=YXNtRINYNlf$&2cm*sZNnVe*<|XqA|o@7rE32owXbs} zHoNr{9qbQq>Y1b;KGA`)YiCNsJp7$4e+Tp6^vA{@CPwL$P20+wGt(|LC)W>5M6v)> z^6o$27pFs2we_lhczh>&PR;bLtK)d9W=2ilp{=*|-|Wa)ugv$rYTf2f=xEnIhiMWK zp%j^_=BEUtvZoIZLHmAd2?Z~=BaCr!%}?OH>Z7XAJ?H0S{WZ2%C$^u!{Ofj3yQz5D zqWRWzTh4l^?osB(p*3p%^)yo39N@%4PdT(L(=So$sB7X!r0!?<@eZaS`rKuOMD}rv z-FCn=SkDXU&o2hnfPe;h+Q`>-^9k{NuS8OPi8*^xOyT@icv8xbb&%R8tnd7{p*c89 zR&5^w0JN}#TTsL>Xn_Vx*@o3gu41VB400>+G;?utAFS!zsZJ_*MxA6WR?QpjIX}7#F80RzWYg|X%Gzzy+6OPduaZ_Yhs4yi%^2pc{ipH4W^@gVj3uinGL$qLisBg% z-9OA)Y`Ba@=@BpesE=}8+wl^0_;P5q7-F4lg59U#(60ijmz(UEfY7` z|85QD_toJ4Um|i?c_D$H@c)zuz(35Iwb|y+Pj_hhRP8%dx%oy;3zd|RA=k`%E$iqk zQXEhF{zT&W5;`YhxJ>MOxOTer`K0`F?$c)11b~152V8e($I&z$t|mLrqvu*(aTX4NF?vv_1(@UB6Z2}1ZfJWG~9Xt}%H+~w1-=)BJDT?=7F6#hg&PVZm2YBB>@vh_dsrY^@wv`HPr7GU{`F$F`zZvVN zLH#uLiGcSV@%WXc;(9PFAKWNrjz!E`jT>x5_ki_P=~S{u zy#5m%e{U16orE_!9pd*lQuS|y*hAkF*u0vaXbR3~4%^KEKosi$k+AQ2rB@@crJ^lvLUhz0Ky6YF@(z!sI0dv z1AwpsK=kIaL@G;=WdIOX0Eh%GYe!|Gh06dStN;*#7ItKND%&E<03fUY5L>t`iOR}l z832Sei9O1pESbs{$ua;4D*(hIE=!@ZnX(K3!kXgxH@xpa?du6N<oYDHP@q`=?wsc117$53vs(K5N>&Ax znjQu@YIIGbedQA^$zi#gWHH~Pi<2%_+to#vs~r^2kax1n1B~g1VR=$3RpX6{=VCG! zoLxrB!mn-6%=%)vyz}2627N>(+?dLcBe&EXNwm0sfkRy&4OjEiIe4r>%3@53G z;<<$1giS6LFF6}%d(rqr_=lv(81kj!Ir?o~g5=LTgjLI)HKuTAu)huq%Y)S6F#+x| z-h1P(bniPTRQ}kxvvEabD;=y1uHXjd*=Ys(bQZbJBqd`BOrK5fj{#0#1N7T_DvHqNk*K+Yq%}~4%i!-^n z2N%D=#beGv@p>%o^pWo08|JT0VbK1yDPbu`FT7AaL>F3;E2%Lym}s{=xr7WZeifIo z6lE9N50a@$M}GAg`Z`GaLB*@lX|N{QeG{i(dMR%yzkt#8tZUcBWLeV=UPMyuENe%p zc6Qm!KiG19aAwi^E0xz3&S}ZExRMJkb7o(Ps0zg!;=3`Q^S^uGo|4$CO8bXW4l_1`2et%6N58CZRpF}Fg=(-S&~AiV%1zr#7d96 z%nzfBy3`md9?Xo5Q@8o@lT?2}Ia>|xVbePJm0u>#4yNV5A)LQ@r$UM8sMEG$o{UH;DxldrZhs)go6QMir5KbGgq}G&2 zsZIfYzcaJ_$0>qeA4304iD>`v0sC8SkB|(zZvZosEZ)cDRD<1e<-63zKib-l!saW; zboiVZNe*8|hU#05o&;jtPaTMA)wxUbta^3sCGw}UKd#L0`{dn~s$eRPPnQrb|4sVM zo`li0GQ9ugOF(&nTg)-3G)BU`Hme3RxyIyoo?H$GvL$ObP_N}^xy+}sJ&GUMjJ2Hd zM7($dZ__B5|LoL_Ea$Rjg&5hNZUrZHtp!`aOcyWZNE&yZvUCy9PH+sLGeCkA^Cy7^wb$mZHQU^u!9m{R+K2^92T7x;{}4CrT7vq-+EAz} z&ScWFUmSb~5TE;;**VCfRNfv}Xvyj5T+BM7i^6*2{3S|d>Meyi_FGz**JhLKU9JN> zaf$*4jc0A?tfwim8uo1A4=EV>9%acuJVU{OTswYCn>#!iYBx4udSV9V3;tka41cPS zL<`S*&7mlJ=)7o1m|Da7#+`bSZ*o!ypp$|~Do*8fCDYu0-Dt2OG#FT(B%`HAuICN@ ziRAVC;QR(3?u3Ujv#TC|kHCU$A`9>r+C7yB>z)prhYE#Ou<9kZ`G?#fj{O)00K4WY z=B$6P=WB6a>>6rbtz}_9MsfT)?(QF$=>}hnmKK<+@#t4t*8(*tv}o;|Dl90bmq-4N zj5ZHqcCmQ0?KDh#&V01-bgV7RQPW-Ds_tx)a~tNfvFDt&D0Xzx`nn5fcpdC`8aZ(APk*`G; zeU$s`zP|%yzRu>mHzQNWha-QWWDS3U50{btd3O{(>}J=NGCn*+WTj603}>=&E==sF z`Zgza>vuS@-~<1Kz9N4j^W687Sde*N#4+{axhw==#k-36V=*Ty3Wx)uXg2||bJA#k z#tn!|2EYrOc@u}b1LWnpAh--c5$PM?97oPC&YPqG`#Wr+B+&lr1Ww?Cd<87g0=W$&r zfy(4DVdF_uK;!-Cq3f6*j0)Ld)R!mmq@8 z<_7zth;_v`tn^+t5utGXcO-TO(3=>6!YT46Bh4-xH;S%#8wMKe2=rkRZJ8p6^b4NP zi)SI^$HmV`gwCZdI-^7MvH=zqi)b3;a@7klCO||Xd8ufBw z%r)xRjudnR-%R(qV`AF1JnvQH1WVTRN5ySydiE~iW3J#_LJ5W}vij3D4DUm!oSrSB zvt=!&ykXwWQgpBLu|$=a&(Br7Sx5|gBWBut%Vv@cdv>`!erhqMj*IXH)}n|#bi+0B z(#kP~&yo|cORL$!F6FUd1zQxpj&LQUjrD9-icBboNqUG)l zW~^`#mUNLNFB(c*zuJ9M%j~|{r4HZpOq*{lxe(s{Eg?Hw-8@|}1V3{P%-=C_YNxzh zQWfVt4sCg^j_g_x*)_e<9Ab;_QF`7#f7+!-@R%;eO^P>YL`QuiHKM^wDZMG_TVP|%A!CVVqoJKl<~l>TSenOG!qO%m(%0L>v7;(c-DG_rQ5 z`{B{UbU*xwz0f#50KeMWP-_Y{oP4DEzn3yIJwfrTfBamEFI)B$6!>F2Cl- zTc3D&M+cLSer6#DpBupys{C1~twi}LwT4noys!>eDucs`hY}z7Fy7EY^8ds{zj~e{ zB&=?H>l~x{;xJ#&b6C*?kfeAI#FK5QDhJVJM+g3xM}mi{o9atfeHW-XpDW&9<5@*; z^c$>3jS}GR00^K_gv7~q4EGyIWY~nSY8vSd>-i4}6IbI`NL+pGLwVg%qVOVB4T)Xf zbG`YfesWjiT;XQ5%S=8OpW?DwYL3QPihhL)=!!U|ca&d@zOyKcYl9b|a5Licr+C-U z@xy?Cd5GdzUqHEWyQIOzEOJ{<#A}*j-Jnz^70x*W7xbaP*~AqN&9)S3-tnZ0e1@eO zW(LUiEWn(O0?gNw03e@u#b6H=vo~o^N9|44W_@=GHUD^8Xnm95`i^-w-<9W&KpE-L z7v~At;61-2k(c;>I6qPj5E8TG%Tdh;K=@`e+W~sA;SJf#b6&-TWNq*Jluv$$`v*7| zXPR&see|q&8qVX*LV~OJ4iXI$xQY|oJDA!A(BPvj5jj)tM13h0?@?Si_gS+t?*RN- zl@0hH_!`2DQn`$#@2|#N3oMrSxfoZQi5Y%ZTZ0f{y_A?n(oymKAK7A!@zt+0x#ho=fc2zQZSJ2 zKSwKKii-i}&(AS1ci6@Cs^B>|q}k{{Q+goTL^77ogbRMkb+y4jUf8BHtZl5W;!&C@ zF02@^O=ntjNk;fC!N5{A3YTB`a&sv7sR_)ew3Jr2lh`t!W^YQMaA7PWK?wC5;y=h`Z zuMn|cFRAKnlBqY2ihDVtLjwnSASPflzp6(3{$*csW$r!WKNN4eue3$IwYT`Ruk<*+ zwV!w~{5w;;f?xOs!|!aNoN7Tc&Z`W!fJd%i%yVyukNhK8eJBd6l=&)grBlmg)#qf@ zELqjUV*zB_OTHP%*M)#cJKH6ZyQ}O4}s0D9^W0{|&PLU6*tJGiCpQM!cg~Fn{S}nYh%| zM@r4^l6CG1cRf*z$3pGaDZ!imyS)nDzb)DhmQzn&PIl>(E^^!y)i+}z9=k6uBYl)e zIa8OrswupyFTdWVwYO>GlIRYL`wHQ~L-^AbUOQ1!hy6A6j$KXl=Po18!4pANq>AI|7k zDq3M#Yn%@_a{jO?m49)Lwrh=z9q|04(qfNuSmE%&eO)IoH?Nra2qV&5<;B`U%bfGT z2`sM3O*-z}j;VYLhiiS*=oICYx2P`8o-@0~^?FuK<36Lm(gMDJR?ehRZ6H%BgT1@& z;fcX(_~J2~T={vjDzyT!`Q@&;84nvuoGcwMlHT-?$qZ8C=mO zd3_BUXTs$t5$@+R&1-Km@x;W|1*G+M1z~M|t0$Sd(+dAlihnmr zq53QJx=gj+H4p~!$;vhd(isL4{LvK7zpiI_hKpl8&hhH7J|3MxgqKRvHe7GlQ%j)nyY8FpFi&9cYJ%@$@GWJg~pT4(fo19bZIgPF!^>3fH=Ost*CA&zi3#|=6((QECqJy0PYsa(+)INLS=i9U~>D9 z1d|1O;L?CbGh+AxgTDvaIuLNnuW=vmXk78&ios#}(#`Z*g{oGB}?R&rr!{7;;oleslA7L^X|{gz?od&2?D3 zJB5$5&vjUtKMBp{SGFZjqB(R&9B20h%1zR4Xf5q?IeG)bvH<1vY{xXTBd4H!!2~oa z0-ns`{=+=Hc8*wQ<`gw=+p;o8hoxT&JP31$#YA zV?T{o+Q^rt`#X(>f9I4l*NNPsRz?6%c|w72Q7Us(w*dZcJuGJkmowMX9PA96S&oYwz=hAPIojUcWCs5D zzs6_k&U9}iobeAC0FNl15CPyP41mCw7EPcVQ=>!s-M!CTZ2cnAUeW9?OwNam^5xx} z7X)CZbB5|}1mM{e0m6`L4+c)W1WMc4+(#I)hU#Uo5*wV$qH~1A#4ACJdflJ3y% zd{g>?ahTzzq<90Z=-`eq5DI)>XSHs<>zB4-f|bPj3+wr)+Xy&T)Ykccb1@7JA5?dsfm9Gy6%#T5b8#my z#nUEAX1NqKxT;n?Q<^8e+m-p`AG58WlJZrlFW!=@YrBJ)8drL56um{EX zFW(@ZHv>yjSQT&9H|4@Lv?&USMw z#bCN{lhKrG01S8#4fxsi(F#oQ*VU}B7-)!1kN8gPx{$Yc6F~D(smc2h+BxzywzC-9 z*=T5|hivEZXy!}OFFRbWVnSB=3rgY2<R*L7{KS@nfd+#c{O*j z#bGh&4dB?o-Hu%*$?``7{?z)as@;)98o_sD6KdF>hC2R|ccGIdPytgShaFx1dXVJA=oGJKZR_rd)vzy_*8mPh7p$Ibc@ zH%l>Sw;ox=Ylr;E_}CDK7+)o@seRabfyOE0IiRvMx9xZi()+)^NqUpQv_4}q-A>`a zSrLEG$ql;y|8^pRKd|TULQXrh6~BIeJ(EYi{#E~J{yq$CzLg?Weub%vD2~gQLZi3> zu0eQf+=h$rc~T7IKNv7Vex}hbStT*fG0~a#Wi{cybYA4;cHh#|(m)X2hw{=32V1wI z_KC#hXhxSx0&{+UJz9-4bY^9uCGa(l;-p~|KUI?XK&CO*V+nL&ij(G-J_ihCzujlu zMKXHmupizMNuJ?*o0KY*j;*S1A)4%=t&TRTomxlQ7W~I~h0OPBNn5070hX#RriBSO#MSRB;L?HEqe$|AB3Q%8L$^{%Ri{k67Ne0;Ejq_I%$FOl~ z3-AT+;%`i>(tT}jSvO@3p^{m$P%bg-PpxSH*6bxE2iqT)+TRAn`;FveFU4_Tf7Pgt znq4Z=;6<=6W5`R%_%i|RdXkL#{}qOP91uLo&Lh(KuoO-soKeRTUE>C%3r~D+JeZh~ z#^$fgi{pWJa*g!YJZ$^h)vPm|PW!hSs|feX4j57M zS=Hy>$Ji$?q6`}!whYK?7rd2$yY+nS{$&lYm1AV$w>gK$OC7i-9ed#9EGF+D=^d!h zXf1)=9ZWKnE8Gx?8}ZL@r>KyQA)%tUJY>j&2;4a>N+I7Dzmm1 zYKu!-I7D?=HUB2J8wc&evWMySF;P^P%*WZwex-XYp{}?;b&BV5Ggm2qD(Y3Te1nt8 z?Qrd*pNrSQP|w)ySkRjVMO(Q53YgxY=T8uv`Ke>oBv}Ad9$F-uW-&RG8?&W zdUu=eNeX|dh_RRJt}$Q4k*yq#Wm_#C8d^Y`|E{ias<{C!?TQEeH?1C^5W~p2v?LLC zbig zgyP44l6}pxP`-~PqVbh^ea=l%Z#F$oxY2Q4Q zFc0^wWt#;)B{e72rEow+@4lo+7gLl(6LE{AgX2YC^fNCR+O=2$c3iUW81vGgK0BJs z8Nb5*A(Qw6&#Nu^!hXD$-#krzq6dO4=MmdJIbM|Y;eL;IMCa_WF!tw96C3YLR~+6Z zI4O0(4tN1mDt{+?N!j#h+8{)S`$_*OEg!OJTxcM*KdVXjW;Ta{g%SG@zxOz68Oc_V zoD;>1b^K*tA|l>&LW9+enXpm`R(ecUYN18_nS9ziGvy?JV@SPfN#F$lIsv)DGXH{E?~n-etik*$^YFf?W0@ySNvfk z|0Pyswx^Y}mWeSm4(Ck3tV>1@MKj9INWVxuRh#KP*H!Ug;2w)_KH^E`?Ol%_;L%t@ zV}WgU?Zjl^pLJ%8Au*06PzZn~#(#v;iG+Iqa||6#n4|QwD9rKVf3QN@kxj@jxKd*h zS@!sd`1tHLX1X<&{F1Z)!!u*0q(@EIi$YRAICsiZ-+`-f&jV|b=f zHRGE$Q547H^sUR-ui&5v#7qA4J5$(!q%)amxiVJ|{wXhUT;?2HJR=+U7eo7!RfAJL=kJP%LdxD-*l=7n`a|a z58z^9*DRsC+9N~e9nE^vK^vK1yn;KoSSJ`tMZq#r@FB3>l`iOW=to$qMl4;HVxm-c z6lvB!=qd-Y2UC}$?+(o0M;#7V-ah?UKdVxC1?qpza1!bMu)Ws0d>}LI0CmI@k{dr} zsv@Et>grl+_w}VXIUFLfxAda--IXz=&RR2#R&VY!-2kzOo}VXd$_5Uh2KVVW#e>Yd z`*gf>hAx>jia`)Ntql?9Fxw*@$}pQeH9m)jIaH2cJ4n>QS-@{Khey0;PBB3R={=FZ zL|@q8Rm8F0l`+jQD3VDRy^S)=X1<~0{YpF#U?zj}kEhkFhxU7#4QK5Nbe29`mSU=U zR42gr-6-J0pnl1EJX~Mk7I_SZYfV{7Yw+>q*MvXs0G~Img9b)Z1C`joJB9{kVFUi5 zTd@@IJM8f0r5F&lV0kdVnT;{(SJ22kBhcRJv6erzx3X#Hr9Pbx8Hd~d(8RYbD~Xml zRNzjEQRy04B~L>UpN0EjA8V}(@Z!86X3GrtqDXYJGroYj=IPrT-$Q=m@y*E07h4xN zKLp6skE(wZGMPun@`+DqNhJSYx=EhX7HXKzhxog+p8P8zauSyHi|1Fj z00DoXgUxM`L+Cn&UUTa@bCg|6;ACoKr8~VJK1&m8=q%m;q1WmhNVjl^6((Av%nUY08;E!Aw&f90uT{j#B_Us+-a&ikjZAG+wpM0ef@njB?-BvV{6u|&G0|WBp$Gn zW5P&47)AI&C@B|)_erNYh4LEX|mHA)NO$}4O?yS5YC#nCvlGj{Dv^ms4`Q+}D^`9}!D zJ$%-93;P#^dMWeXBU~CNAv_2tO!2%%^M}RM4mq^bh%G*^7vhHJvv2d9Qi%j@tC*CUqD=khxHT=iHJ39Wd$(yZs7C;+C^HA3|bwrb9dT)Wnn z9ds>!A$|7S9PcqitJ3(AN(*-eM>a7)BMVM9$Op)^%|CNE#&|gXXmhBsOlXTzO!RRu zpO4hu=i4%Au~g$#M+|;v|V)to35kVH6$#G}k+?b>tj(_P3zbdCnKk`)vNaxW+D#d}15ol*&;pSaho` zXQ%5ZT5CCS&b!{X1n%xgq8ECXy5GJv=0q3E%}}6ubx)o20y4E*2i!=PqrJre7BUW~k~Rk8;n;;GKKcc$4!6 zC_09Kr$>W{!y@0~+|jOe2DfJ2lbMV-`_y*e+$2#Bi~<2LV|AL_YcYU z?dbiuq;Nr=EI7h~7wlJD@}7*C8T__6vP!jYz$!VRJcb`>VmHwpaO&!b+z&(0Ql;a; z^t)C6N)DyCm%{_JdDKMlmHIi`V~U)tJE2e5dTh6iI2#7IJaaL@<=lV)F1-M&HuqB! zF2@B7aQQG6`DPg5azQj)s!+r3uV%PhO#ta%+?{%B5r1D}=D~XIXQvZcfFCOOAqB`0 z*8}9(>Z~SQYXZzLd>P>rsr1X28)}AHBJlF)inibi;AP1x053naw34UTw3ab&o=C+o z=WrSIHUcox*mT&3LQ+W#BgGF#tCY4Vqy8ZA=LqQW{HNICLmgXrK3@Sn(%nX0!$QLI zcQfuzS?un>t)z(hsFiLvqN+)%3;+>7dXgm_3k&B|5$yUg(F5wl zF4cuQe?!UqeyMZPJ%mKBx$)SYUH2U$fj4Vp?Aqt@+^`5aiVxVrs7)eku-9TT|!aqSnEmZDiOjUlG4EtxMKJWYohTSPkw`I zi2MTRhh5uDjPibHFq>wynA*MjpGZyYzQwGxqB6eKJ20y~u)22X#8ln|Bdr>pp^%5` zNrZ{lwWTN37$1RmyY`hhosS!Iz)9J_8tLH(J{%KA+lS{QHl!q)f*&+X_(#0H*|om0 z4#0Bq#|W0w5G+?jf#rfFa1f>wo?6LDN?Hq;8BdTGPJ4hS=;sMind0eXtiLm6O~yXd z#+rBz)5X$S0@z@RE*4#!$vyGc7#x-w*m0JP`|Liip$TX(lUjP8D3J8wTgi+2|!v|TvKe_E9QPkd589f4Pa=#EHG;?vPId6bRJaK@u87x^Ed9}=`@!}c)#hwMbR zVfg7N&8SN&@gCprNi^j@&Nf1Y!v{}&-L;DHgZc|f$-WY)3K}$8OGQ^~8cntEomq5) zM^vV_mHM#aMO2h}oAP0s*6(W%(49`~R!ItQuMPe{Apn@BU++F>4JQi>13RSABtc6B|Z#Q{BT%u1;!Q5s8$L zx0&-?vZ>_c#(cnE?*+(<aIys12 zGtqR(R+o^$&r2vsmR5p)ct+mcFeYs;W(Z4~VkQ7RzKCp^ju|qJA>XyR5|QgT_^2ur zL9U|D^JFqym7?QQQS<<*I?|2jvh{fHzKO=$F*>gq=08*GZb{xj1`h)N^)Bc~Z>tuGIf`1vLn{P7#ebSvJPZp-=X@&3^N+*Ndj=C7nPbO8QT zJ%hBeY6AL;K|Q}c(Zunk9L2)9x+Ua3Z6-~W+GD%w-hUcSE1eSehBR0c40c!~mcbgZ z4Qzg?J?G5yOx4_AH=o8nFUL7^pBHkUW!R@5`y4Srj~A{E#+9|!OurV!TBd)ulh6C^ z74%&Izk3$H>lpq{smNagH~9eNJk$0n!#7xp+ly!6mN8tZthW@8jDuF|hiji(m5TN# z30!|R$+9}p#L`+z@i?=p!6zdV)b)pwEMsl{8Hw;Rp65%(yyYQRNywFM$%jiul4Xo# zT%KJkSYp>^mx%d~(Lv5d0bG8G-B++g@!ZtHny^fz;%>!%o7uCuc$A`e(NhiHbd+y8 zD&D90O`3d@=K420rOKyN*Q4;%K|XbGO@yZu`IO?i6P}XgQ?hF~JSEAeBp2LU)0WbG z>`haQXWkM&ifAYAYRf+Yy#}>ic5NF4d?Y|Cww8;;{FV?PjPh8x1YUMTC|}Ki+uASF zuT=f1?TgH2XZJ=`hgYQ{wY~oivuAs8=k~6FdWIh<5+P zL_+~mBxh%l@(><#bNr0tCO5hznq61fwS6QKHspO1Z$CsZ0nB4fTDTzB#O*QD=*t$> zmy^f)TB1Pi&r%FU8~U__>}JvU?tigwZ5czcslE!MsDgAf$sDf1(eWqrfTeysP4y>q zwB@W<=9-D$J!e*4+)y;TtQ%bLdT%JsFIBu{W*MTG=P3lzH+M}UVi?1FoWJ5h9h_)4 zg;AZMRGEj`{i4~WCQeqSea~*zi2Ovw6hupkrz1*8d<7*o^KNBsGKoue!8xx{=AMVD zwwyX;u8*qBE^(L-Vq-9kUN{cLyPfpt6{sCSS3F0_7`t}Xp#^0`VXdH441-$}-kAzY z$KQ=YRgBwgk1wdffTVVF+$vJeG%gDrQ+#8_qA|QUhftR zqM3~A9gOGk;q{rubs>)HiKubq;kdqNiHu7_#2;?%r#^hL9K`@N`at|+LUQ;)471)E zzIu>U5;pU#$vh|APGx>v)VzLgZZ)sRsW+2M}lq@b07Q1nH_#dWNfrD<4@_r zfr8Qr_bHx0cr_w;s)u;sZ;|zzk)p4iYq(sPQhoGip&r)3;n;nyVw<}*#gx!=h9S&oHk|$R(x7aP2pXwrTzTq~r}s<#sV2YFUa}%+4(MZ=CZs z=+^}O*2WL81n$Ih_OoFChrDtCK}^eL^&G%7UaoOptm+^tJuu|hz=<=qeIExfZU^vd=2g58f!n0TbpV!92iVsDfG_9p|yilDs zBgc)gT_vQ>xKP~wfTT>i0+xaOpm6;)mMC{MQ0~Ush<|Z3{T+I#PW@eJdgYgxPKWE{ zT+C>(Q8VhRPm*+JPKzlQL(%jw^HmmuZ5mEtH}S(nol+u~A}4ZF)@CHcKn`XMMRAdM z;jc^|Kcz0MLLHimHZSvkqK1s}ApA}zu+ukUC#qJk%Xu&0-UjiHiD6RI037SC89A7G zjMl?>tz>1PM9bf0nBGl2c?s~Pqrr8lj00#GFLXt!f3`!w_wGabrUZwuO~74r+y>=* zK&Ngb{HdCUn#)HVbSMXL@kLyG3l?J>K;huxu3UVsj3eM<@zYp*ts^$RX{?_zUz8nM zE!(qT0w6~K_=n!)wd%8Wr(jP$YdS%w4_!f2v-NOqAcx&nap~ z=U9OPp*fcgj@yg@bguCH^+@oeqo~%_&c4x$!`gQ6v5uszy#a($Sc zp4C)^a~7jhuoIisfQcMYtDxZh~9N=L8S4OT-V~kf&e?ZNfTuu+ws47qv>I<0GXaf2qw^ zgkO#u8*&u`)(yFe(7WW8dq_Y)hUY;EfwD8jvy@&GafjK^AzEFE&?6}(6!fCNSF4Yo zi_S0Ph(y2CoD<4CG?6K+qi1t2a%r=Ggpyhm@%GVPy5H6(vd7>hwE9cCDc-9|ZI_y% zeI8%JE&o=imB^N-cPOkY;#M2W3g?_P<<5AM*+B(0E_ zy(Jb~i-PN-Qk-pz>YJ4*R*(ZOTbE+}Jg$HIJPz2DJAGkr02_jcE8k=5>zjGtofYr( zXwl`Ky&Nvx{X^_)Bd-5C*;JHmCZg!gM!%9GREE=wq5%Hvt;}Py{WMQ04juF9RqZwm z{h$Bhp{MXrCZ7)@_wZTC6X(Y=R+?_L&3Q7(Ow3?6GYUdrNy*^>I_OIspdVb-IYS5O zvV(!#K>}W^pb_RM(frlsh9|=BkGCHF3>7HD(ji@({fBgQrsK+FUQNQ~JbaHJQacy< zb`xCMvTB0wox|PjYV+NmBwd70{1yrY7r<*v;9jx>OPlpZXI=ubzpkY&O^r(|fvH@6 z50)cO8r;RV;Uv`0-D2}~1X@U5Rj1(EV-6Ktmx_(;Xb}LLx3g(ulbqWqn1Lmb%T1+Y zQ)nd?Ty>5%A@n+~(dbzCIXsis=sI4beP`Lg^H2D}Z^tlNrC1SLG2g4Iac0rq-zI*e zH}sVJ77?pwUngSqD*UOs4>`Z}w={<+o<;FHM3}z8-)ky^&*sq7&ef?9*0ag3hrKy~JeK8M0*)~Fm z8PIE@>?hXbIYRIJE=+JL-l1=tThQMZpBM4-nplW_SloRVVgF|s`ws-H`z&Gq39xi` zzt$%9A31~LBXkX3%_nN$Yi%O_Ub5bZU;-UpKjHkBP!|k%UwRRr<=LfU$BgdV%S=ME zOLatFm|#Toi>F8%F~;>|BPPb&Oz6IheFn*8Jo77ZP$d?*{|3hP-NMNJBJ-aid$#L3 zC5~@JtDjS7JG5h~#Po#ap8+h7yM)0~BgkIbWgHDj8@4(zIzNK)=^PkQetUBa%Eu(0 zBK1K;`O1I6_?x0J{z0LQY!10kC(O7;^>_IZHbSdWCnd~P%^O1d3rleMQ`&+$*tO$y zA_lo$p@ZRuKUGiQ!6+NnOqjR4hfY0~>(m3Ke?$$G`Ex?jKsp%TjGOBNl2S`(LTQ_# z)P*TZ&>E6xel1@pU@R}UNlt!rG6Z!FZaDuZl;q)qOi4UMNie?ti|3hg9Djo;$1MbW zKA;@gOgWwf%5e=_u^L}L;V z2OkiPkvh>hcR69A`|&+OnR{RJkS>yPDBdrLUSmWmKQT$Zd|+(MXhU#EQt)C5{hePT zaoU@Qfu6XVb$UV;hB^(=U`9P@Mo3DChUha&d0bzuV-)tJcr1VpTYDeqxD?ttX5vc)y|k5Uf8RurlLtvYMXx zB73|`tJpJ{Eg?4V#h{XnPyZMCe?DTrKO#w6+(7s7vFT*Z??roDrDB8`2nObX9MVJa z?18F7Iw;=nQqUrJ8CmSsdC^>(sJnk9Tjvp7+D~C3UH2iO37fKtHM!)C*z*;fZH1|o ziC~?CWI#V)9AR z_u^D(>Y79*f={(a<*OU@%^1_s`z?&X*_~pjDixHzutvm(OH@BmO}n3r_h1c9%Qb-Z zDBytKS_(gpbJa`OhO}PBqTW>}!vZ{&5Dq-`H(D)Bnz7O(G3$xX*@FOYRgo1$%rRpa zE#fXh5O}q!ddhG#gO8&3Ni~fK118ZgF~*GxTb)FM8?@t4b0}*&toIgJhM+HEkFW2l zp0?zecv(hDj7y}>4SRX~W*WvFG>loT_i`c*Bmy46j24-%qgKxU3bqNwF^TIgqWy3^ z3@_NjzeNn&il!B`wszmt(r~03Sg5WD7*EYjp|CUTzxB_#DePG>upJv;X71q3qvMdZ zuEuSH*UQLN(z}Vw8gHF~?jJINqq$uwyK;QyyhL9qtyY}v#RIeAVpO+JrA5dKFGAx` zS_C_xNhDfP{#yl8;r#xi#po|av&*Vkg^q(&7x0pE^qZBJvgL3s0gS2E+Y|qsRF?*j zi7Ujsg1%~mjzW^r5c1zcZLjc?4Zt0?l49+LqtJ7OV$ z4RVIn^!zi&!f;(04ay%4F`I(78}fxi>j7DHhYlOvx~Ec@Mb{m;WwJ$m+eRfs<=spo zo^~$%A+Shk7nOGt3rrje8u3R_bx&S^Nx*u97ocPG0syypo{idZ5k=V!;+Q;(5={D$ z63%WsMBs~=hnXU4>%)9I{=cUOos`&CdeAR{WrzS9F4(NWuwck1>3RqL1as2*%g`cT zJh3rZdNt8#@7_o<8^K;Jcx;yZ-p&*Zo!9+eX#6X>GL-j@;E0cZMI0eM#F^C*mc$Lh zFp;}x4sSn27WxB;)Tz+bjw*j zFTbvrq-Qm}@L7v;+{kl1J_e^SAUuWA96}{#nljp%S7F$q2tG}R5&B$3aMw7wko~_O ziAJy+PwSvpL>Ys~(}K2lM`wxKYMr-OvtUf`;^sND^LE~DgTE+680pdlGc-~#QA{_f zcrdvc21jz_{O;Nv88gKL&8A_GT8P7b4u^e3)UYQ^L8t4LjN^t{dFb!_J$mR#r};`6 zL9EYnvnR=UwmJr}LbGwuY;fBtj@QY+Ly#td7pC%Jzk_##YGi6m?8>6o(?~P`i!UR31k!Vdzvio?c(KU4w+xruP zASEN>i*-t_xXeTmK(TMYL=*tp#nYr4n&jDAPVU~d< z;YIax9Fn7t*%*>TY%NTQVI{7HW`bK!$y*Bn40-VVZce3fb!-klM&f+F{O~ma*3~?NPJ{YHEjN9_(1zk+mKZQ8~7%Gb1=OLH}DzS7NvrCE7~2U z;yLsZ2ZqaB{yyY#WGx+--TgB0ZJQ8#Czl= zSpcoRO6SIkt|BJ)ezUmwH$4L0gOQVqd$8krv}>K+AkG(dh#jl$Z`Wf1*?pT$F;U76 zh`;v2g&WNA|F?X*E;q3R%`ud`;Qh5<2~Ck2kjGRN5H%UX(GsB$6r}SNEw4v z&q@;eDefX^UtI%@Lpdm3ypE37X+PTSnlOmPD~98h3`5V_j?NcaL#zFVU3!Lkrkt1l z{3Ukn=&5b)pUh(JE#&pN0khN7Yr2O*H$guE`_q2bbfOsRY2+h*y1F^q{}irQ$ZNv= z9Q_@M!@@8vUcNStOhS7r-^AXc_@QKzrD;B;{&n8B)|4Pku}+HDA0nu{n)6ppTu9(Q z)e;CWO>Xaxj#wuyeE<`B;$`Tahm-LRm=t>l1k_30`(MMTU#r;Z6k7kD!bawKYMkr1 zvy;zL*9(C)&SdeYfm!YP6w~X+NTu^sl1Qw651WSMjm-BX^3>HrN=S~ zHh*}M=Fra)s73U2V<;Tgm&S$n&{bX-^G1##O7XszobMn*vk`;##CeQXxmf7o%Jz!) z=hIl1>dvL%9>ubMQOR?#ycqV&?0m}R{&&VilXJ$}3nWJWa50Ujqb2Y)j))Tinn*T| zu>g*Whq#TsIJb$S1N2V6^x+&jXB-ZWw>uAKIkx@?@8DZucx#%EfrQHYR=C5dsg|n$-84i4k zP|xucrr?HLMZ>QS5C4#uFpnx1kw3u{-soxJeS)@`9pI*E!^L|YV4rg~DsDe9tq9S@-u2Ut4Zu% z$muyS6g4F+;&>?XTmfDQN6$MIt7Z{Rm$C@Rm5F&$HBv&v;90TZKBm%Q!e2X>f=kZp zc^>6)&YfcSm2%3>gW`o0DJCSwzZ8PK&O?@$1#kp zwe-! zd8pvDth>hC+F)AuuO}gZj1DWTe@mtiHmel+ z{x@E3UI2`H{p0*_E)>J6axcMqiD3}`oSbZGERpwW;K6Q}awUzA)hgAu8bga-1Bm+u=$jS!b#21`Ivl zsRDcGqcP@*deUoX8-YnEgO}swz~6j^=G&fU_pF5Qb59`xc|E+(hq zIfqMl`{AoF2}Y`$I#|Nc?RL2g6fXOenDl_$VUj3o-*cco-r>8M=9Nz(nujeLbEfG2 zsD`^z;Z?*i)OksUj8O5c;iT`nJH^ykq1(kQ=J}FJ`{p^cS0yq^pz@Rq3_&u}AZjp- z`CKTkzVAvC$M;{8EwpSc{fF$d=(b*DwzMep+I z;ps`F?R}C~eVwEQ2gRlv@P$SD%(;|IDvOHIp*pUyw5@pNV+|bn#G)t7m{BUz`ABUM z-T_STBEdN_;#Y_xy!=X9zSygx}xKFn){8)IoWY;hle0+M-CNOnrjBObXQueIDevTM9aG9Z9F1Q zjRjIVf%!}SN@O==3R2R8eD|@=O-6^##hA5X$S0gPmaz4}cw#`Xz~*0(aX~!BCo5-; z=^;k1Tct6(N4~z3U)RakL*;9<6e_B&BhicZTB_q^y@u?Zq7EycaP2fHR$jMUipkEC zn!FkpDL&qr9lab+TtUmReJ(BsD*cT%6Sy2AKZO&~!>#5Ss3jNUl=#PgSe5add>*zO zao6Zf)t9MMR&S+^_-DH6zXMK{3M}6!xVfw8@E4wQyQ|tQ-Nl;2GGpLp&K@XBwI6Fn z;sFpS)}Lt!0RvP!eJYIZdKw+NY*qbcK*bIiT0~}qvsrpM9JtScmy4Wze7)#$n&go= z*bK%Ni$+LCL_AJ%vP|sUbTmAQ!EzMJTntl(_fQU1eVHTYqYMmC?d)a=bmIHg0%)Cv z7rd7ZMvAg(RwZL+7(IIJRm7jKM12q{A+BTg6nCx=F0qB06|UH6?>(p}&S z6S?wmB<5N#h88o`dOv3c4A5MTNliLwKb3wQU+YEFh=H7Hn5weHro60i9ClJ>P(=d- zqn$Vd6qJT1hYW-_PA`!vCXaYmZv;#%b@p5-QE69?5vd8w*@TDln7 z+eB92*PX*tT+ag3l{67QhvQ!yh%6j_Sw~AI89P<}lxrr+USbQAJ5a{ddVU}DA@+{; z;LE#!y<2b=Z6b#A@-#jP%Y$Yw!N7Ss(P!ag$;8ZW^rCTcp742RnP`prlGkS_t2dQNDyOs5)V7l8B$9a4tv)- zqNDnTgcCju7M`KK;7~mJUWop*d#-WV@;fXJes6YI4i;yQYI#i_v^>?=@KWXE zJ9!JHz}HoiaSH~W3mY9=IGsrG%lLc#74-XA2XJP+ZT@F6P!F25J>2B`5}qEG7>Saa zyRHmZ$Ey*7uTld82d)|Q{B^$#??1 zs!svLOzK>s{y*%!d3aP+(lFkg1v(+&28a*@H8Vyft_g}JAex3W+{Vrj#jvQjzGlD? zMeG(vWNhee$o1NqnE}NG#T|EC2u(DQ04Cr@02kbb00OrOn=B!Or2DJo+eQ)IRi{pswK3YHpUc7IA0ElHm~oYZ$-l+@PMOpB_n+c^+v9#u ziu?UQ-0y$J{r({C_r|#2JK}zyjQidHADw@1o89^M&-D8b;{A+c#fJ#@16zf$6<-gC z*gSFO7W9tqxU4SE#`{sFU1swK;(W@#HePhx%Ww34c{r1=G6V|*y|$H*VFULXmHc*W$^%<5 z%#r6gjt)LQS&nYLwK3F-`&nV)fL!ENZ&K0%U{t+A8RdS{R1_JpcsyQNErWB7-(Fbz zGJ=)}xt(gZXhTKKD@Eh`UK32sd92FArqlpJpm0GgwDV9mh5iEMxHEPWcT_mT6gAj^ zR_H>aunv4`ih}#na=c0imcjo6W0(`W4)++@6|Q30=|nwc;gKA2DRbb2#v+5Yc{BLj zM9*X18Y<$JbiFkkTp!bzgfq57uHf`y?VQ)}X891_Qzg;o5%NVA(MEmOOk6F#_hPl=4(1tK}jgLO>glow2->6l+F?FQX;o73~~}l229aLgx13y2DHdebc#{lQ0+)3MKB2*pl}6|E97`B4p+p?HmH_f~MJ13d`CGc_-cZpg z*a%r>Q=%EAD8V2&{TPNxs_1EB&1%c1+`&-z?HK0Ck^&vcJU&N$v{-3{?T>Tn!k;Ym z=OiCQC}Y{AfG?Q_LnK`aoYkF2ilat;){Y@@EX`Mx$JXyGTuFg#I8D`&WnV)tf(q;1 zeXtEa5v&<_-`Xnz;EaMic^g0-L5;wb0`En18`?2Z@m>$(i+Ip?Kj7GaWc)|~38WhX zvO5i_7giAdKf4%@U$=b9en7PIWD>XI?}tIjP$efMzWb&z3AiswBpuS4cd>EE&1RBz zAD5Do5IEdXwyytxQK3a{$Yww*+!kXm)YbSJN$j56-NEGdNlgdc=%D$Vs4JmV>v#Dpo{>M3kVg9gY3HlS6o z4(L#Kb4nNJ`qQrFV?oBqX3iu_%p{A%QAdT))3UhC)SQ%9+L{-6c00QBmVR_8?O%K7 zJ=|@9Q3-+rR4WRI&#NpVb#agcTs26{pNL?kuJY359Pzi>s4=9t=Nx8v16$;Vxn6p> zh9r1yh4_>L7o`wH@}#N{5eh{u^)|`>i47BLG#WyH*-S4o=yKtLMuOW{X>_9XaFcxl zvSw$*T=M0N2Jx<6g7Myt?n4gYE7AtsD&gI99WqOjwMtu&`NAwwiCT# z1!5!u+|73;8s`Rv13=&j}(1>^x4cZSf@cb8u{o5JeV9T|n3pCwhBFrXy zgrn0O|80pD;mewBSALkv=<|amUi|_`EX$FsbP2}lUSh_#>_t`+T$_Yq+C-w&>37YD zi()v>5$~h52^O%qyxhQZlTHE+*(w(Cr}r3V&rJ{{8=WOeekqwz*yTc%ME6x+ScF!X zr;YltshIzKNeZ7MfCK>E@Yih&H@o*W1P2Ct)NG=SyLwIv{Z-I^j`N%i zOO*^OhFK-OIIE+byDvGDe;;T!gv8jx40Tx{{3rbS3En5208>D$zXkA1dy|%J!F@=B zpGO7xKVo+HQhO@xk?{BJM(C~mcXNlKcHRVOV^S>P_7)B46f)+$N47hg=e{`@u$o- zSVybv5?RAKWuZn^TP(7&PsJ!{`_O0tO_7+<)a%jX%Egcm(u}tlff_Q89MWxCrmi|<80>zANi04U)6taPIWdp32qkOZ4rudfTjy{=mBLO}G0Rtl{iq*fek}K?WNELt9*g+!iMpAd(g0 zwg5}J8JbpE`DX>&wQBy94mhh?s5x!HQ$oaki;wlZXje%5(lcfpJ{@06^qSLS7y?Fc zAw={2&W|w)XXjlkI1fiQoR$K%&u57!MSsbvQVxL28bikVm|>hLh7n;u+DpaF$+^!m z9o*Ytw(5VMs5e7K*SU)52odd{Z*{<;)GTKeCT(2Ppq5ma0nBp`f(^JkME2JI36pUQ zzpi0R>@PTHJ>)oj)EM;J41et2AW4`g@Z&8enP!(VAFiEor{D!*4u|Zp1<*wMd@ow# zP)lM2*wq=()$QEXBH64M(khOM;nFa~;L`9QmK55AZ<>8ZzQwfsu@dlK*>)aodMxN%Nj! zl8l^?J@h1Vz0>|mR`N)Pcu5`~8qf9aOSM|U)5t9MF_N!_JxQBXoyS7g)3l8Py^i8^ zuTGT<+uTECYn@!U!9BQGs)fu|yVT_krw;;)34p-QlfA#QKAD zW%Z=^_GC-`iS!h|jt_RHXcZB_)qO-PJhnvtd>^l&&@QCo2*wRcfQ-(l!7XzR~eLl#u3KG zr?(dZLBHY{aiqzZc_v*8&5tQ^cA-v`pFWBkt)`WYE~BLx<-=$-ffFH)tHvTWTA#S= zXE+-xIvWCd?0SB4!Zz_#74r1x^qjYq41&c?m=f4;74p>~t^cjO_y2x0_D}#A;;A60 zMiLQEJRz*7p9M$ZEa(}|JQ!;}kKkHrLvW#vEBqzjTRbs^81=pXMU48~*#M+Mgv+Re z4*jMqJm(0a!3ahJ_XU}jaOjlDrQBO7mNB=(Bo-4yVj#1_xH%E`D9&IFhB4F7F*Hdn zvDk!xxWeytve%2VH^G=(-#C>PzN*k7g&4%`*SOH_p58YXE81!HJX&R84`!YcF?{e{ zHi{jALF~1sV7{H|#Kyz%p5Ik?F}y;;103Nx;Zu=I8IuoMd!KJy*4H)7mDSBOvTqG5zgf5l1rYASdm zUqfWH2Fq(VUD<8l$Kynu#?)+JPMU_>uplt zeIyHSkcJcSFbiADAwy%gA{rvgEJ8)>@&YGgiHJ-zg*qIv{%#!|-9X8Z$83!xidnc;j&z6&`TxeRF+0pR8T2i z-FhwgSsf=on-pHlrzt^i;1Gsh5`dZRQU2h*k>~@)uR7)mexyepaj9z#6Q6Lc=KT&$ z7TSOfL*tyCW9iX+-#E80hCcL!D>!?Qc6F(hY39gmce<>s<(v5N?d@dhdT=ai8pmam zRcqM>?NTg_F)ncL1s{a;9VN=i5=DbbH$ZJIxR~ZCEvlW=b-(}Ze29?Fnh4WEHLrHd zYdEJ9G^aMLC(o((X>F4^()6(+L}dd`#}ynm$bE(jr6rmb4FAznFfAB9y{3TQw~a+t zwB~f|6KFyj)F@FlW6;c+r~e1@aRuMNS&VZ9(LdKISXK%a{vA18JNg$CPD>ugP^2yS zKxArhLerijlG48m_vpO-YT4tmdR;c*KfAYb#5+Sq}oYX0Td%jDZjje0r{(`We{8(^=fufq=c<-s-2gm97YKQeL zQP#70H@^v!u z3%F5S@dmPqYLEb-j?7akkpFD<$M7T1+=l!J8X(6ysz-ig?4`i*j*rIjPe$uUt<<_T zu64YH4xbw}?Y))l#bUs{Zi+;U#emuF+o?X$NL_6d#}0^>(xN(9IpUC3)@rp?Oz<{G zIG7Zyh_?Twh)yY?HP;fHV48)ja32i>bLvX}}b?cYZGfaMlq?Ng|m0KbDVy^}~sPjabLAW$1!L9Ma{Rg}hy z!vvVxZ?IoqtQ^$#Cs+*wD&P9Na9)T0j)Z)H$a5PkBsuKVMzpm@ajCHbO`o$XSu(xW=D2dhh6L~qZ`|LOPq)T2R% zQyrV_49*o2-C#~GnEc?aNWb`Uw*Ch3 zMq!+`>6nQlQ)yYoW(#`{INxX8l1azFGlx6qvSKuqfLZBA^;q-|G|i1sIza587eGE%xDk% zrjM2$2z(q70}c)ua8o853~kp!2Y9eg5&arz;t=wj!UqST&MK&ikW+I=W3XE91K z$qov=9};A(+(8RuGZzSRf!|SU2JsqwJ}C~JvS@*jZEzkoG3viW->^VuOvD9hLkiSy zKQGWI%5@;VWfKthu+bhJNLxlRM*Yd;;HV{Kdr7cn3Qa;BVB`lx`>>eCY%#}dBc#gU z0jEvsgj7*;%-Z5Z=72wn7z2J}f-&IoW*+dOh!}8C#C41ruppZ>mcC-_cT;(k!2{x< zn0OBY&?oOE!kW$v0w`rm{;ya7 zC7b6TE@1UW-6bf1{wn!5b?B6q=K0QeIv3<%C2CT&hxhSZ-{ev;6u2Xn0z0zZBSB6D zC?ZCPi*`YRN6_~98#|)iSJrv^Xm94g-nq^=$Ag4XN?U9bQs7eZTj#|6^NF-_1taHn z?GhnsN+yFt@zaKKaw{`N%(4yHI$jretHyBtwodt@V1emXg!YukCtbDK?Dj%&lgjra z;wDv~s&t#RJGhn4>Lo|6YWK2M|-)7 z#0_oe(-C1iOQ*|#3sWqs&tTFPuTIl0y^xe2GxBR?-<%43wm^6{$6LXbZ#|DkI|@cC zRZ$wqRDD5a0@Nv1_K7v$tL?^9s7wE>L2{^rV@AOsjL$9+Z;IJl{#g@DA8egea&#!2 zQxL03t6?W0Lq5GxM~fGQLSuNR{O@iQ(WSLVMgCcgZMr${hO0yXZnGNcK}tpOgAr3X z_vH@8h94P)=&TURIr2Vzl3+nf1*>8RkKs_nu){Eb*(-;Ec{0}kW_Uy75B%>6-Rl&W zIy2kpi{!DNP?!1!Js#lTtn@xUSXM!=QKJsUos*_bw9}!Id721IqjPP!{6v9x+c8`TZ1lBX(O{Z=~ z*yM-ZPr{jB9Mx5XBHXZk^UNeOT*_>i)-aHCg*Q!-{1_KRrHbgM*BOoeH}N`H&S`f7 z{YNd#@PEf{{*t>89r~Z2VgcSEvnv8{$@xS8a&=Yuv_CP9mR-=5Djoe?Nh@jN{?|Cu z_d@4Qd(Rc@d%@h0J+9X`?7?DoPhsYzUa!;eEycY19_YMT2VB9)H<_E2VlzjymqZ03 z++w?)^&j4@p{F)U|)`;CY>`#IE#?{eep1`eL$2MFqs&mXZ{&s!;f{^f98PjA0$S{k-yQ2fdcoj z!$;jk#Kje)K+8HUgr(t+k>67UgK69dE8n(ZB>Te)lIX6Zm~ZseK$?z^=!H~W#Dfo# z=QmZn{oFyK`1as{ZO6}vdf<2~cfs!TvG(uC8za*y=(G_H_5=_2$Y^K;HE`;=t z&}ONh|^2|LbrEH3EkdhJr0<#9N|^5r#;#=zGHm8Z-ugrUg54_&h62O z+Q>?wZCkHZJd%Xr*X3YQqN`Aw=>ndO{f@E)51`3>D3QNK_}M+_BM_J19)O=`(wdx( z`UXAw!A0+Et*84XdU8OjrCGd#QRlOF*X^=89RQS}-4ICP%MToc`(Guw-wM_N$~c2% z9a?^D0S2)v$kE+{i!SDZLPO(HzIG|ZlV}4(E&3(YG?Jh7NOLO1y^2cp*?9yte^h=O z#K4K9d7hr+x~nKn4$2w!8*V@iOK3OR5w8~`0hV*(O_Cp9aF+zO7(5tjmA2s7XFa5y z0ZSHM9TENj;`GI`vK48xZ(SCtjK6{fq_D&aX^*sL3Uua~M4}&O7>`Ic%|O2ZwF&l6 z?*tLfaF1X({6-~$!fQq)r|(dL_S6gtdF{+?&^`0Gu>h;BV6R-wdxf-~fSnC!2ffx$ zXwa9}zBT@wFo6l%>1qZc93F$r!vmq{664u@6~=YHQgGK!h9_V7P=*{FlfmjT z`vkmJ81*YzZae(Zi8pyJHiD8nH$9bY|{LG$*{XWKU3Ud)4?nG>yi!mSRAD2EL30v ztbUSVGDmJo5sD@1-31P*>f@X+KCBt~agK%pi?0Ihu~mN#+9E)9q_|=QKJlt2!#hii z7*?X};e#GaA0|;$yx^iS2?r-|)Xhhi8$%TN$Z;j)vU)j0bn>j4=o=)&6&?nBKBI)6;gxMHH@D`&ByyjZw(C-IAt2O z6t05JnTDYEj*!|WS{N%KZUQ23O|+s#=UlDoUg49`=(k#Q9gxnBnp958`i&v zMVK~r$^Wag$?fXG94`&)yT64wV@t00)Dks9X;rZ>Yp?y&Et zpu^m+=`YLlW*>Lqn=$ac;SAV<&zW)qpAF0_YM%R<*GXh7%{H^-;B=eTS7iSh=^{H# zyGmq#)>U?fmL;-ZM<$_cDRi9`({=Uz!qS*embqBS@jR}?L-1Lw-VGIM_+Vv4PXB4Z zZx*hX0^>P9g^N}f=THd+$T#YWGGzD6bgO%hYd~O^3xpw9qw9y-sc}Tp% zZ?>85!F2KwUVDmR+=&7&_y;@fZy80LR~G#lJ_Y_%--lLBJ%4F8{02J;Xc!x-Yk@$o z#)qX1UTg|~YRA8~S??Qmo%!iScw!X zL}0a>M8yjt2DWJ9??Du4V{uNCh1wyR^Z=x~X4eYW=5Pjvv48`eO@V%^?i))sF8ZOv zphAKs31Y+nOiGmB?4iD*@DmpzF}i7z#|D@buIeyH^vzTlQDLk5W`eTt$+TBrU27%5 z;Qjs-fQ3mEof6?M*j2za#kd&VaWU4$EXKhMeKE$i831kY?iW^n+O5>!@6)_+cU;5Y zk1zVDO}nTMnh?*<;Ef$%fVKIyl(5JOlyxZ*ZKZ)PTx8kXtkV#;26J$brqb155t z*GHJPJ_qet0g@x6DtZKZbzk>&SND_rr(kYZXWB!P$XL9Fck?+TGdl;(_*TMj72P>B zRhN1b!J+8x#8$Aoo@v4-E*;HHe?>vtktDPTf?*LH&2quJ80QroqT2qy5`n`Ql$79X za1fka%0c*~Gtl1P;1ke7z7;)J!vt;}f$MuW${^EXrqPq9F}SOMG`fGU6?x_vuAMd2 z5Z2kc(PPs#vT4#YW?GtdoYMkkT86em_(~*MkdBQ(F)jP1a_(5no&0>UTCxz&3RmG5 z%90#No7vN8{dq=8N#PF3_cRMuiXA649Vhiv%%lo?(xfg6b2tEDHeZz2DY#D(+Ln?d z-|)alr~6|nZx3BbQask8NJK=GIULdcj?;OUSbvMmUy}w$dv`FhH1v%z;1`UF9n$oI za^Grnv2K<8Ka+WE*<#=!QFmUB2bB#57M#^xTKRe5NoheFbgOL2ZSY0ANlzvN^_{x$ z-+;tQqg~lj7tYAtP`+8S)|JT>(A z@d;Ak0}4^$sI(V!lL8p`U3Dz9larZ$pq?0c)%@-h1uX-njSuKc0vjd&)x@jwc{};; z6#ZLw^EZ@|=)MGloyl;L1yHSFSO1aZL)T5HS93b(WVO7@YVDPZ6o%*S6rz2Zeob?kGV zUsFY|n@^ccld>oy2G@7=O0ld#KWtjR^d4+@w$9LGQc20bJSm1(7o0kxprNM-a*Iny zpsGleRyr1!+m(?CfSSQsY4DY#txD?wdx}a>KzXjhnhd8h(kA(ju-`EpW3_PuC^}dY zJLI0d*@>qZlT_9VhrS;IKq8;v)qSGhQte8B|se&=V%tl#FkQ>PA9*>CYWObIU zGhp>STGazvJ^BYk3=^i+S&OPX4v;ZhcBT!X~+8jY9r7)ZTC&dsg}+ZP7lDioBP{aK>ejbx7A2av|l#0V|G1fZ{X~1m>tCI z)tvptP|ALjv$2utpRvCeIormyALHyrn4O8Wy^wACkEeU2(FJyCWhI_G%I2wCh+o)H zw=W~N5t)U?@;WUcnE4qo3u~3e+&XFHMyLAx28xXs_{rT}_RY?)xclIBFL39v{`RG+ zM5l7hsdV72iLGK32D!7T1;5n?ZJ3?qTbrgGy#<{JIPx&a8`LIu47FO4 zugU6p*{Qm6vdDWE1}5lB5Y~Es*+J4TyICXofT)BRMy$Ot%og2pD5u!V1E3|vI8K3T7_WXSL3_v9;|8BB$TW=@Th^G^8hS`czK8ozgFZ^khyi=X7!l=Y86QFWUjN zmNS~SxWAQ=!}WYk5pp427U?ydUWHTR(8iF7)8h0OaB7?m!zH^V$e*a^1ILK80H!HX z@GLBI2A8?rEHj?VBujx_qD%^xnPQe1%w>Q{w~8_+NxeZ+4yVVA3yx(HX2a2lIT%bt zPkG^e5wmwp{5vLo8!@{Jw0Is5V^Enc;!&VsiE^ zKNV*WgUmyV#p>?v&5q>PuB(jIH)!u4LL+2PPuz)6$de|dI@CO@qYBMODo?*ZH}w_Q zYL6tqMi$TSxZdx??*P)sb#5#0eu5fai@U&)|Eo)B!mvkjT~m^5T_>xbkPo@@sa4-B zp;vLLlYu1oG?K!aW`0pmpQJ#mCk--vdb2xBRSYMbIt}if-V?&|b*wHThIqSE`OdC< zsom0zmMp`tsu21MpeM)hZYM2|pxwm2n*VYYyEG+btg;=*3|bdz3ofVRPqE{c)UeBCvzjT6JAt02u(kdydDOLs^KDHh+FNJ-)G@n$3amrM!G(v8`Z7}v2 z`|Cc^$}uVB${4FuHKs?!m_)HmUB0)8j>F6EQ)Osd>tguZ4hBQ@v3y%SypyCF8TjO zg+~{N=IFM_=S6s0`L(BP-5@KAXVERlgo6B^ij}4G^m*lDc*|7prOTmmrNWPcnWxdQ zg7#=p@+@wPe~xF|M=SY66|c)9syOgmpo;jiTC+E_M=LRSbRNC`jEDBl_7qF=3lwiL z1^v$4UtssvFk%-u@>`5|REM3udctp~^)shhL-;)k#^h`W==e>QrK$kdb{5vp_}pF} z5Ewq%#PHGa7>)_2@K5;fhMau*e#89_E2Yv(Rnkheb$jRnm{=vvc*qTx(ZtXFvp#XT z-n1W(>vksPM`dfH+~760V_89iSNGM`3I5j3@Yg0s82Ftzm12z7XXAaX6N6zu4CtL! zSM;DSHq#|Na2Bqg@jsvk_$Pck5{-6E&rr}x&?a_cV#cp<$L#DJKV)SPqUNB z;j?ze+oOekCHk^k*Mll;WpZ$=Rc;t<?AFwT}FaXyNbM z+v>>w3F^zaC+F6oDPUbV@u>WEel&lh+(2O%Erj;fWY-+5#>2W?c`SptymIA33JTpY zFH01}2H|J6Uawf0hn!LFNl3g{FViruKzz=vo5l5u%)II1Qw*dQlJtdXnCCSHH1A_8 z&Ix*W3^Ag99_>#kTwxThG)M8Tg;eG#QD(7GW~o``*;lE|>!Qpuqs(&gRX?waX~4Yo z)Hf6gndA9}(i-N~8l~$+OB;-q8jZZ~MBX+dugPrcR~pD3E0xsv58*?@y!}S`!^YP! z6L_sgqV5g=7EyfRd$PFq?sL}>y;C||!8foUcb4ua%eSBSfqPr=j%JtVP>FKTIkz5{ z$(=)btY#2*&PA@B$}QZ>k5a#E1IknQAJ2BDx}?C$V$A>~lk2{sV^BZ!!mmIsjQEV}fcpOr-Tymu?~LV)`CXv9K-&-+>iwUh{{MZPFIPrp$jZ0^9fROl%Gj+d z7mP^VXdE`;9uXc3lKQ=;QzV`Gd`0%mC{b-4LUqMOWfcrjYa0$KD557jm z1?L$1=i=k~{t3QD@(=BsVa8j~TYe>{Hrm1yxNm;$v~MV0z52O@<3*m}s6c?q*rxpLO^%ks2F*?Fh-D(C0&aWnY8!k|IbD1vy=3+KuZ%?KX`u16K4`Tudu#n~HKm6&Tuxz8&Dz&KN zdI9CBK{wO;+qrEp&PQingre>)OeXHvm`@>@&CgUpl=%$dpL@9ZZNRZ_bBXm?k<0q8 zJAw5%XhHo~O?&h-`3YeF?2qBwrNL|9u~$cPbwVi?RVop7G5wqR7!VLnsTEj0c)b-Q zXr&UrXI|-RW3lU@Y#3KgF7lfA-nhEnV)B{~9Z9i}L}Jik?ZIOmWbAO3tj^BP-_*n(+@2saTT ztfFO*M%6i$;vAb($;{CO*K7eJechjTHYA~|8Q79g(xMk)?>SJil3ItB-LDHwdI-^@ z>HDap@CrVVhQMh4{VXbRbp| z5VuJFxiLUo6k<9ct7QPhrwE9Ts3qIn_sdK-WR*6xocn#@1_olAtjuYWe8V8sgyjYk zmQvtS$l{GP{|s!R!BI;PQXCubzT{%uFQmH26HtYD7d}C7RgA(7jtzwW^AUx7wN_ti zf#-q%f-U#z@N`!XyNfM#Ql9f!Pp1@x(`3s0bn8AmNpybZDnbf3jv zRDW#LX5+k}t|<|VK&!!YKXr`PG}T*UI{y29B>v4$=={zRIk=LIZofxL9e$>>y~YF^ z3=R&%fuu?bei$O)?bc>MvB^Ml7S&2swMPy85De-f3@(e1I1flM6^HL`N5%UdbWKQx zYwMUEtONNqeaN}A{9FD^%fAF)nQs;Sq*k?emY4JB{8YV5`D8pz-_?0O8pdjZVJ^gB zV*J?&I`mGT(u*BDrG@&GfWHbQa;EtRjN@;()UkOkby|iC;|Mg+scrYI!W^r+J1~pO z6FhxRGh8a7=#8?nf}ewXmH-O7$j`?Y^Z*RsjZItbQamB;6?#qMIme}3PS!(l%Nr9v znsi35@=Ko07~8&@UwTAi9s#XzQ1*wAU2Ej9y)aqyx$%~kem9Q*v>aY!2cqn!$bGUJ zVzZuWEs`Id-Dz13u)BsiUH~}x#Y-TkFOqH#rBlD*5@}R@HHuqgrRP%KCEG0F*6X?+l2g4gr@+Nm)-JZ>s%x^PY$z|BhXR=U0-V`RFrX6Us56)EB3#MSHM7 z$%Pwco-OCr$-a}gYk?n9EMYSAtSTc= zI#qX0fn2u}K(UtLseis=e!gmczQ&)849~R;&qioXz_U?SpJI46&dif@;iPW6R#rAY zHTs^D({o6^=&W@sdR`1&Pys0T+KQMSk?cs#0ht z?eouo%*ejlbV0w^fA_~ms&>h$c9UNI_wC)YzN_|bG*Y!+Lb2_|z3(8#`3sC(UrYf^ z%9a%?j*qcR2VOb7#w60{R`PErW{$dcDJ3DTc^x8GNyw#qE+>{{pk966zb&lLES6SM zHQ;y49cVH3234-W%68u&tL!_K<_a#qm|$KKk``=Z4q(Muh-z0uG6r`uP%UFm%IPmk zje7)|Hw|gx=|l374~JmpKxg6R2S*}YdduNx7J=!oRi>E zTBWM>#mZVo#UnXMn3U)Yj!LL&v)Lc5Ir$8rG1QpvxWz~sQlt+tZjjX8-?Plo+^hf-IJL$i~&Z0rEG zn|*>DyzZ!6x6cMx5_IRJWA8oeH^{-V4LGPYG>qS|iO5Hu|Bw8CL4%7|)7#L53G}!e z*c;$Uhm~#McQzG{5R7S=>|2}eLdSZCnCgoFpBY5BKYVaQm!a7x0NfSk2Q=YT~)*A{a8G|wqf*P z;!KqZEpo$Xyq03Oi?gSEV55&u@M*OEt;Kw!IHP}0iTPfpf1_KhPjDkN@O=bzqwXh- zqEFY1!KbN6@ac*K?bRrS+lo5!YrU^v1adEW*=q3RB^YH3%+Dh8Q#L+5EiFXcHpxL9YdR;nnokxKb(M&){g8<)%}9&zQ@PjLJq;@AbAX5 zW>jn7qKk=UFF}kg39g`6^ugiR0aLBL&IJ%1v}gGn5k`B0zY(RhdHjt?r#-~qh?d$t z{7qhdZ{0~@m$f}0Z8rDf_)&W`c_-aLxX+Ebq&S2zu4R?pZ2Uw)%9_97_&YTu8^E|T zJF8Q}bK$tEj8n13-k$)kzNRRVLh5OYc8Cj!b&~&Gbo-zTYADJeA{d+vpJ-`mX(&Q# zUDBS99GJgFJXbAO#+@xUjLgO?K`Deg%Kr-l!=;?~KOhz}PyIh)Vfp_vvGCx7w){4V z+Ibd5AtD#5Hur3&GBD?6m(rwPtD)tIP*G~Nr}+KwpzW!aPGy4W?O$|haCz&`u>VE% z=4DWBNk1Y@X=e~=YI;v6O`g~3NIjdg4`fny8~mw$9ka*3uIn|Ws=ru?Eic^cE)&X3 zyYh{!9F2*cKH(i=BVW%%#sFg|#0F2+HvWWM!Gl)l;gw7bz_&BMR?FO;g5vPxe2~+- z(w7Fy8ij%xsbzzskzwsXj@1$^oyZrB{ej(S)NKDDqk?Sx&WQQ)#QbQyvdX32e!KSL zFBrel(a7#QxZR+q3Am9aNZzLRfl5<*^c(1)33}3Ag<8%3EbnYDb!|7)+PX&yA8N8X z&gMB=^`W>{jdI*@YY_^XBi0%=HkOuowOBNS*`9CAbX@1d|A<7DwLos>wk0M$^o3MK z&H9?GS6k)&WHNSM^jZr&xwzT1GwlXAl@jAmRUMzL+tjEJWuaOmt4&-%oOVCfGA{oo za-hD+6}-oWQi&V@c6UxC@VL*@y)ni+!x)MAE%;)@>H93JKEVQKNbz8NPj||*9bi@| z2PZFg1*g3ytED#dYED5if$>$!LKsm&*JqXv7QW$8y(c2fzeA=`o&rFhzxGD-!Pw#y zZAM3*$3V2T*LNEX^OxgTUiQC(UZS-thL9R0gbKfWX{G1A{4LI4aZXmbz3_@0Pv7PV zhCkB=fL~U5;dIS?9Y#QK6}EZyi3hk7*f{w=y>K9Sw;UT6ShMm#?xTAV9p31GfX6Y~ z<}I*z3hEILJhkvU1GR&)0RG#$~d&lg_stt6|gmzP^R>-);7UkUrQTFriyZPf2& z6sNEY{pldwA?N}Rpsp6>xzrGbi!DLAD95;Gd&zN zNJp5V4aDTa&+MV|p{nPe%J}?tUk?k2a*_|UM_fd2G~nfa67lZth@8MZ*%|Az6_Z2h1vCv?h0A`km8vb6ZyyW`L*FpvcK&#Xgb|q#wO4S6^?8ThyUJVvegKih3mrP z0cA>k3SfNL%Ddar6GN=nL(_1d{mH5aS~>UI6oiB^%l=z@JOAWcTdF<$n!vcQ!VdhV zN_1$3(&^CL4}Yrh@tyXk7u%!ad?3H%=fHW8C;O^#H(I2?;}$_Bcsbm+;}X_84qb|q zM{oCs&{<|9teZQPE))1%2&RusN3wwL4`WwHoAC{;3)bRg+5D;{<|ctC$~8S{sOdb^ z3LNS@9O^?a(D8o`XCF+XY&&Mp#q0?$v`0JN_qHrRw$g@P%J16afzhr(9)}eTZ$6Z4 z@v4+RLaJJLF6Q7l?Av+R_j#()C{=Ait#G4Z?_m0~Ay(x&)?GgFY>1+Rekh4w}&Ug#Ui*pf?7lr#hjLhZ?sGn$8Nz1ksqNL|Ic)Hp_X~cAHZ)M1@RVoU=>~n zC95yji7mqNk(UXUU!-QTj~jbvbqs+%xS2YbOumeM^;`gh?3G|)pW|{cltjCdVtW)X zX(tck;VU?+x0#hIvYP93e_@8bn=9y93V5-Uz;pq{|GPU9bx2hlHCtZ^*0Ruow7?-` z2bM`Y_lSl3)B)-&rRD8xzwNxMB+VJb;OIlHO3frO^q)_mqfhtOvbtoV&?ZU?up2!{ zx0N*ik{`WI0CATBh*lLZ_9C2EA|9}l+q4#P=a7`MDwB>D;ZItG1l>p@PoI^7Yt|OO zX3sU^?H)8z5x}S3+JGNy=-m*%#Z+P@Li)JNsG)FtLpMM};fUe?1PF4UiWd=u!ee9j z7hn~W2zEc>73d4k;lmG@BnsPJk>A_Q|A(F*PLAsSN`)_l7Cd=xZLygSB5BUrMaMYK zS}NWoV@USjc@F?Q7+Y3oHb(TDtSGn0OZSSmMAXWJ~`Mc7trpoTE5e6fNqSUx=qyPURR3(^F4~Is8uO z9Pkr^?i9Gx%aGp-BERPWo_Ef2Uu& z`Lt;1bacjq7TIBZI8AG*2G#asybPzhfMVw~1R6UGmnqxCXR$gxQ!UE!H6?;>&@z^4i1(X+PEV2( zS72Q@Pk9bM@u|Cl@b|{#IK|m{8|kWFdv~+V67CgqKEkxa8J*?*>One#q}}=qHF`TW z>PSBQ9QtYl9K!26@n5t~tLI|%v71m8wPHCxa$iOfnb&Il5Ah-Tn3&nr%{Iel6M>bS zvJ1e4`|*9RE4ZSF%i}rSvV*JiAXabDHV(Vj*ufs+LI*kbK*Tt>KNA(uEwj3amLbUA zN1=R$C||>ly&FX+YHJ1ihJ9U(^8tW#r8xD6?B`Rz9-X4FgJl>8*F+}=I3m30%+Q`Y zV$l&6uTj&~5oy?0u!90Hlh`3uJ%*<8_D^wlH)_5e5n%9AHdK%Sy}e;{ri8oxR-|o} z`B4+!v_l(HEo<^DzqezxCpU^)0Lqp8FSCYZTY^yxD{HpxDeSJJUh=<_NN@k1PA3-{ zRB!dA$OdNErIk;onW+_}R`CVoH$n4HO(qzwpa=7TqS(Dg8+T+A19@RP1O4Bpl67PO zn4`}GK#j;zE_8o8nVxK8zgc2kT}p5%eeveK0UKav$xFE$A(*-d+q%aJjpEipp815f zC%GH6ixK1`e;@MtMwvQMIU(4()>FLF-+#?}3WpKibt?8e>Wqi){lpd5kn^-Pcz01x zbY`#|46=I{LW&Y~0p1>zs83cTur3CJYGMF_z~8JCxETwJz?FNoy^CPI|AzgdecnzB z3-rf;JXB*mWWoP_mSQotD%s;@vf7*cR1)C>1KCHRtJ^<6U6nFy}oE7*3U4e{(=IZs_1j&EpDUAJ7K093k7+0#MEBZi zkTy0xfyr&P*5%jo6PEC@wpjXhNNfF)S1qLtSLS^VWx7DWctY#QbC6w=gJ)doLknCq z3X%evsQqw#DlFvt9OQI?en%Bst971P>r+vyxE>FtHFU#RfZq)Z^!p_E)bEqR9qf*+ zOqL4aUOfLt%!}2!K*M_3;e%k&r~n5kJ(HgToiY zpX$~m)E@-VCB~s>W4I^ViNV*YOJIDd4M zUzP1S&=Cc}1-r+Ki>6lDSDR+U1#wsmzwi#5?3-0&Wq)qL(NXt9Ii4R2BVc`kwY>{%CsP+$#P-WqOg$`2Lx>n2W$!FhC(p)Cx z#KtQLvhUfLuq8q>FcE8PA|@$-H1>IBTl6&asHhIX~6{#%DxDiKR1ic+uQnYGpQ zAFI|-aKZIkI@pOk`34sO+?f9no_8Fvr=l?$3v^XD-H2=w6XI zuydv~Zzi>{N$+s+jUBO2P7zGQTX+c346I@t;cE=K<*lKqNrBJMv{K(xA$ITMJ&6>w zCz9LuV<_n18la#zo{9tXvto}7gN*Rd)8laj7a&r_kV$|G8fBAL5rt#G1Q*~E=iqi(XI$bk;2)P2FWj{qW3;WroFJ<{j&ZZ z*J#fQu+oM=dv;&c>Y3+IJ((1>h2hToqI?&>lX^6XYuXzenW#D}ue7hhMw&l1{7tt6 zkR`U3sCUC+9qK{uEHeS_mJt49r9YIR!V$@UV{a}3%E67>E92-UjlhqM2 zUZD(-)fuI9g@TZL2$dEUZ7Ez%d#yjqD(|KaOD5xbLJGW{BG!Hlsw2VyQ=e6HIDx0D&AOgC3lwzY38N-1lx9MwG)6u| zIHD>vmLGtQ+|POEyp$&SBWXs{T0S*v550mSQec;WCaHmXn2jH$m$9T@!^6|d{(>a` zoO2iw*X<@mmi&pN{)2Pp?+M&e3N_LT8`;=yt{->)YAMettveS^)d9wVtSokK4H?+b zdR3H*nWGLbWHSdhz9X((f|o4XMwJ}aB9&ctvsQLTXMT0VG-EFY-0i%8XXxYkgp@(Q z2UjW3&!#UR@|fZE?dEzD|FxT=e@#7y?_TgiLSKA;BE4gHDG~5SKp&NeWR@?6>WT4R zO>bv(_y@LPzuq9K>SVnp+XRziXH&n&<5>7BjU(`LBGCY>mcbVgdS#mJSEr2rCJXwL z!P`sw@KJhqndc0ao$U;I1_I>R!`2$D;id=y{81~z0(bXirv%|*;?<}9kfz?^|ALtU&xr! zTTh9icWrg#M2x8+x*-$p?hDO}FeFgKcVa>0!0swdBQNZ01H7e23ZeDSD*wLx6iPLY zY-g}!ilEQ}Lem#;iC%Q1-8|8_z+W(w3@_2R%&9CfEi@m6bR2MkKH$H=fU8R_G52Rz zFCZD@nj?6E)}YMx4@-=27uDXbN+FnsSEJlgiOpzl>|~@Hm~1e({cC+q3FVh0TFdQ9 zfYl5mCnDMF*WyJrNQUH}Wh3v9&}qLbd0FTz14%GXOP<8)hqEILD6Dxe-cPs8qt)=- zrj`~3wzwzyx7aI)`ID;F%Ib9!-Q#?%HaU2GvE-8_i)9&zw#EJEW%p32%2_OP0GnkA z6+PU!7=xK2_(&^nOo)^#H(Ddjnc_S&Sc-Fd@TT2*gn|8d2X9!bfT4X2y=tj#)$f-Q zk?c~amOWzIJ&sW3J0S`NP!a`ZesS zn;bW1D2sjn26brPpZ@K`56)FYu2 zJZ@)rlp6401fjQW+NLxzGE4VTm=QQLuRjxqb!Vb@2TT5n4rvpA6i4dvHvB-HcPk^g z%{)>cI?m^$U^b8e%Ak6`>HQ2;x2g)ahKHfhqM#Z2OK~W`ORrc^r%+x z0Xfm8hSK!$J$cNKQ$mSov}>0}InLIpWB8Xg24Fft9N`{Lf*RF5%}w&BZ&_e{CYc@^ z3;tRhf1hGu#y(PLANz)0(=-vP1clFEpeZPI1v!S9h^oU;!SvWzbeDnyUzf_bO{`Gp zQl9e#`{OWq$?{mz03GA}TfIE4@CC<&MgibUEBPOR;Sa;QR419E9rcaa|N1>V?Ei1Y z{-=@a_$~qiu<-&$Z`vM*@{fyQ@9bsiJTUD}993u-OeZ|W^si0&Lj(U$2>f3m_>_bg z{HF+)0S+XFig4}vbON!5;$MzqO7YMDX{GL9pVt5CUs<{wm~PRZK4{xMGw=g=Kzu>V zaK~lQnm4enIxCfsDcrBa+*$TD$m52ae*@@s(iG2s6p|0Lv#b*?!)+FX-r6TZFP_7+(Z%X4yo42+5T%ZhfbgAFj>-mPVM4eGTN(f}e?>Ojw>L@hvq2Ffq}UfsyyW0G z8@qnS3o-4nWCPnU;seZRRW`>pAVs zJe^4|TPim=mNGj*47;pU>I{1y?Ssp(+?=J%+rC1-#zC!lTm*pKbQG3OFpS;3OK}ZP zi5E(6-W$&_@UFH4iiYu@|J}3om1HI~Jy7Tip)eY*Z81*kGdA;bPj2Hllr}9fNr0-= zq6-6zj{q;7@YB=A@NWmPTgOsk(G1!k)iE*ox?OBJj=LzUm3f3u%ZcGr#Qp0sz8Dkn z`e?nruH~#CMMfyjNf0emGvhrZvagTD*L04YyH*=1TDh}b;BZvS*u;|>+s>OH?<3ff zZ@2P2k6J?kBkKdmjAye}>e>?G;sba%g~tI-bY_woA4br<4gd^)*Fw})M5Z!_L~~z^ zL%*Ei?Vo#)8oUeW*AYB^L-;;jr(!R^0aOeP{lk_R z`j6}+5eow++Y%U$zCUH+WJ{D>;*d4Wnp1oSwG-2Dx4ef5>T7&7@6|q!;xW7zr~N*D zs&^Hj=mJE=>XyW$pPGRl`R^*w+kXlQIu(BohprVCxcbpFX+0bL)*a`=^yDty)*pz& zG)5;x2mBov<5@LiXIbfi7X$9pR#|cPXq7pDEZ!f$sX?d8*Va0^seLWH?r@aDAO(YY zI(xZE2TVQz#vWRQ(!2j>i+QYkA3^XSeOr8&B4CNm*kgtK0czcioy z5DabPNB zyJ>^qGLIV!DxZPc-D^!nd^jhD)z;_NK-@kfPHTrvh_#pp6!}Y3>V|$rIcgCRk-G_) z8I7!iJaU|O(qUlsh9{EtdHZpL4fD;)1itEPNbt?dhOhO<`Oa8~i9n%0q?8+TrnJnP z1QilIS+Xxgv=jNn5WQNJ{Er_eRh2;k1G7o^B#mh?Yi#LwpT8%D^FE|qXJMFpddQqD z;}QpBuBUs_!LZS_S`edU-yAP=wl|6K<=s<)Xk0*7BK&$yb3>~xf;?fMM;`;AH{c!SL79ozjn#xv3 zq*IanOX(b;d6pIFhQ!L?L zhfU(Ss!hy!%mIS<6#daT4L%QXF7JS`IWhhWYrF3F3qD&~L25E|s&zcW)v7IYlg6(`!>~=o*04(^V!!iD?5TX*% zR$_n)4^e}geug9FX*yzxlgz1K`%*l8Sb`1O4x78|os$d<@@dJJCX2GnjYC2g+i9vS z@3XrgNMuP`=+E%9T8iyG-;R6BGad#t^k<&sEqIS=60NtE(ou|Unq$+FaA5ZzVpQlS z1{J_CZi49vZ*1YrwpPmYEP>8b8QZaEdk7)t5*p6n3Ni)Mu~%_aATk zJt)XoXj06P|Ne`x9;0yG(hu=jiAL6;WQ%csGGrIkkNybtTWFl4V(Q;-)c@)rYJ(JD z0KF8ZBY`N6U0UhJkoA0Puz5BV+1a-WPT6Z zfmxNI`{Iw5>+YwsspLo8e2*nrI_Y;%n^XM)?7vgMlb9@kQmgb(Z%k-cZcJUf1_CyMa6Vrz&NC* z??WZdDZyKy4#?oPe&#jyTMn{}^0?mRp#$bHXNp9BA&tYGGqFVXThSuYc$QTl*>$RG*i#J4*FtQ|TAHeh+|Gsthb=nO zy`WWm;tS82dPFY4^e%AYvx%>c+f3n|TYHO=NUU=PCln(^-FtpFd@y5g7+r)?lY5}u z`-Tv9AIX{MRBp+c$U^czvLn$LG5vuiZ1#KYsuu<6rooA!O8&pm@^DYe(3| zqK;su3WPal|5mZ$hG48MVTAKXkOf~!#hvOReqDK}T$#~LJM|rsZqE>W2q^}#9|v;O z7|0LL2}a%+$W=Iym-iV>ihCFB#Tzk%(IR&G%4i!CjWnIFH<0xLDpVnTsb;f$`p=fnY%e|-~ftPz&IV|^^d_uqYGujpo z`O|k~Vwn3qDuV%*+V}rO&olcy)}@u>-=9UZn=rNT%a7E;_HU^L30s(f@q`y~3*Rcd zLPi_h!d1_v&;vGq&oJ$3qyZnD6^W9PdF?OM^5frP=i|D?aS!d!i0jJdexWeu7-W&K zE2u$YDi(%zhtw7pC^Vd{XCxi1>ey04q?;R zpe*^zo$b-nMc$RdcI}_L&1oizY5E`8&D{8r!(!fl+s$m8zO98C>tA`ut}PNTL-^tP zJoDlDxWPS)>r5ZLYlmGjhU)8#+|d8>%X? zF{&212b+-}LaO3MA_~U?8g9Zt4M%r5#a6|K&aPpr(J_bn5ojWDD*)#c0{BNUNA-hG z>*|8dd+=Sv=uD{l!Xf(1g3o`$XZRjw2Wz(=PU8Ev>fJFonfG)&p?c!4;-;nMN0uh0 z^=6gIJ!}^Xj5jHAM-2nI9}J?vMVhNNz6T<3Jo_=L1NP{OzC+rCPgy3}ifTe193cSO zsMVCuSAL8PRtg*7t+pDc1j#c8Sxm0v@^oZ%KSWDy5ADJM1U}l$64pe*2Xx>fRaMg4 z2&sy!#Tqr=POGoUTF}T(ll@!VcL7-5+>BspMzDMt1D5_z2^~OqCU4cbP_Ft@T7mas zR^Ycq`U;HQZHNahC4?&%-o$6sDC*(>cJWnA7gbQT`MDUWFx%ZnShgj&?Ot+hWBGf5 zd9*Ur+z(+D-G;_fq+eBpdYc?$;t$mESB)^{vxwt79y8|QIOhAX+{vSKi%rY<{YCT& zbh&Qz6iPAM`)C_;ruftyuMPC{=P*6=GvDLghP|QFlJL|W)cs#D;UIzR+nDZGFVsOc z@K>|@i5IchbO6m@kFk@1bZp>mYM?r%fjLG4KkXv1x7E}}OI3|p#{s4kJDRDLhE34Q z*#w38F|Ax>w6X|VS)G)~_jcKZES23%TwF0~Uqu+gWI!IwdAKa&4e;d;w%YG*1$~tX#a|~aH zS1;PuRa|&PTP*)WL|Gok@VYxIHN^xru2P!Cy zG;#x}RH1+@ilP-6RD?uOL{gI2UN6zQFyeyZf})IyAZ;lir9umcSQ$k}M_Jr%C`c7) zfiC&p<=iEq{=V;dKK`J&=bU@)S>E$*=Ut?7(rQthQK5AaPX3{_EbPtXrwUTt@c(N` zHxfd@RvY{+_Icaw;vPw$|NEDj=>L|N0R0d6g;gJ!a(MIrIXpL0&i>tdy&UdvXxz4Yr`qGXNHb_dnvp$L<6e$)p>C4ljS) z&6YECs4q(&Sx<`Jmc3I7K0?Z`vUJzbblDd)=URYx?$bID*BvzV#ev1$d!i49RE&Wx z)QR2U<7t3|3`<}m9(#w#o^CcXo@uy4?q{*jx7NizXT&}K{(0>4wz%i-;`%R&t9Qmd zr^G#fu{ZYlg}CQC;+`|&o_~*<&&IgtqPS|Mb!hxFw%zHtd2Nh)emAat zi}g7+{>LiRRA8V7=axhp8)g(8MazGWIzqQVPN#UMMcc7BK4)8wt&;t9DWH4)`e@XZ zc?>Xl9UNHV<<(_tnWfL6CkhIODjRspRcEH7&#p7@Gq4Mo9S5B&ff+<|TpD_9Xp;CT zg>n;g|J7T#Ws(%ELxpYVm=qkw_IM8VGLqFCnegLpB>09|Tl6f4MM5?KPa(Pd%IZ4y z&^?1Qt)mMJ%zan=E) z>hLHVu#FU0iUo~3V9xhDX#C!;@R(2o?3SbOhT7cBHuB<|-o`F2mo0PHoM9jO(@qja z@-k+O-y`>*E|K8opMXX3hWlx;V}M-QifxUR3*Jn$1@KG{7$lfZ=eCezp0z=o)q+6Tfg; zI4@zi724QPF{5z${!2fbcPF|0^=pteTVirMD_%rN;Ly@ob~37sm_`$hT;FQR1MwI0 zoX4Y}Jb%Dv;nhSp)?d&dyZFW!`ZCmfBUd@1jT0|@#}2Id;-x)f znD%ZLzR1I$bLjS1SsyZ&{9p1`eM9&R(?FC1vKpS$ruz)vQzo2+;Y4FIH9r-Z5Oi~u z%ARGH;SywkVfLIWvhE0a4Ij3r>@FeB*iHVWwIfLzADLy{pDq*jBF~y~mLb|PE&4?4 z?O0TsHU(8IXi`+so5jTRy#$)M#(wO6B33|*Y6I}46YR!Bdq3nyA-W~f8g3%1ri9m0 zY*|rxxeg%x!WiX{UI2X5k&q<8HaY?mNG-$7&DzB_z$3qGWg{$9n353|DkfZ6-@!yY zQ2(E$aQYs!3}W43UCB&^o7hE!LamHl z4)?IRO~22gB*KQ8XDM<>DrH+SBbvfXQt&lm&+x@s*?cimyx0xL|JWIG{MBkFJ|P(6 zwcVvbw<#S79MHl4Z~tLeMRi3c;mh;wyj;OAneU@FvS}Th6+7vgXvI)!Ih4`gr9l2F zpD+PM`)^4s8wLA&D~^a>i!5q0hkVgtKJ%-ajWSqcKWlmYI!50peDhVpN&#^$ctRCS zKT1V}2N=|8^w)j@*5*ofV=eiVu|(evG2YRGagg(@gM4NZKqcDONM+QyQSmTVv2sF%Jej)(YvU>ARI7m>Gl1&9XZJz;w2w@1y6DOPGG){xy) zKc?64B9jWmTja3cYarUm7_B=If(i@F#*pmnEN3@J#qY6{Jkk#Tb(1spdcrqGfk}|c zs(?w2%1w20-&1n<5%{%F-gat)-1m&U?aYY2)ouWhl4aQaLzk1|2)v&(46_IUYmD!C z`tHORprfX`J_JOb>{{P{rSd>3c~%a;+9j2Fu10RRmsIwpb5QQpcDGWOmsRHu&xyL5 z>flXJ-x^OiheoljZG^jTl{@T=x}jHR-v*DeS8m!b_iezh@~L?mJp7w2UO2vbb7jva9uTO+p$lJk>KvhcA58Wo27dRjxfE z{L~9ERawPt2;ZQZ2e#~j3|f4(!;YZC3#Ir5)9PrVcbW{0@mA;n#zL|hVnnD2FR9C)uo zG5yfpKUc)q%4p=AXNR5;LD9%iSzSg$wFXAp*_p4<1}tSm**?jrk{;C^fUUltVP6a) z_odo~4}{Ct4&d9!1#c9(UrHg$Q9=%$R!PC!Pk7m)cCt&MO%9GBf>yW{_#ZP2k$pDu zqeSIpRI7Q1`SJV7Jx~N3xx$YA8oWLreUFfWBha=cuo%^QgVBUyOR}xK(a^g4w~`fo zo_dv3GC!HeV7g7x80v2>EOiGc9EpwA{~}=(YTUuqghqR#=K#&}~4Sk5<0jrO;uDxirG!6g$V=zEdpwB@|mdVZORT zc*0B-%Zu(h(3_Z;-@LyU>xc6TjUuz$VzRW5LhjD|DpH(^PUg;_WpAN1o(`#DF4c-k z9G15~?Z9ibk-aOv#W*|qnh~`9f!$a1W0wBT?cL7e_PQ3O=-q3}6bCJEWRs9rU|&`y zi2V!w+L1?Ux;OGjDIt#@clZE~@%v&t16xV$cSkSMI$q{h7dy1111-e--jmFG_S0H2 zhQAfy#KGPc4zhTsS79Kz>I3P*#2|zE@tsFPU9g#4UPqx_-cW$Uqwx4~6%Rmrs+jqW zsVOXoLDA+CF|os~!cX#2U2lX3WVPMf)2%**A8mAYPw=G#4z`t0jjjgt_V8CwN?czm zEBaPqZJW8=xRk$)LI#dpxT*J#d(()v$=f37C8N_=BSw#DC8N7q9vvm4&!xwb8@jPl zFub67_7L?2FG{jB{yFAy%3{Wt>@8<}q!+J>l5yzF^*b!fu=xAJm3P*<&T;0(){Bo_o z)TZ-L=JrRk>Qp7VAq) z7t^S%O;;8Bu~e9(M{eqfxZ5xAz1W}Fo!LuTW;iBlzTrWhfga?PYXa|9rm|{ZuIk1< zn^JIBCp?F*cVc+!cbre`EYb`WBR02t2a#QyC8}{>5!G09>+Kpil_-Vo!-xU?xA7ga zd1&k-Y?UA1l**`tg4-pUsAS|B#yf=1*HKtV-1|`CX{Jc%{2zJgjDc?eFB$U|krip) z`^hkyDk^pFuJn~Gc(1fG>R>H$niRanAS-zB5!rfLKQUT?cx6r!V-{b~K_Pw|*x$@( ztGU`U0gLqzj$~17(d$G=LrYNgG;BVQnST-^LTZwIr+6;s(6q^Par|k+pBYYWm%l*e zQQU%x^N}&kJfr)`6`RJ0t52g&qJimgaWwTs`tX!-UMok6MLO{cA>_pt z`zJpw0{YSN?SQ&$4*OdPs$m4JScbPWnsYy6-v!-pwA)!-5ZMyv0rt`(vFe-S+Ue0T z{%OPyMXo-j&+}|{AozQeIsQ*=tjfQd$G_Dw{*^x<67v4rQuN-d)anI`QOkkGn0QLC z+37_NTDOgpR1V@U|u% z%DTghBEM=ixRS0?&~p-a^kJr7EVk5E3V*ZDPYQptCs0x#z$IU^!zYdQfah>9E#x23 z<)51o*Uv;;|9&j4@BT1|EZ^Dmsn++AB$LuFrUvPw# zJ6KHTPn-S@SA1cq$ia$HqN1XOFYoW^Ee2a$(%ohg-YT#E%J}YCc7qWoP#YuG<57&O zqc-Z}ajF}tm_5VKXdONhfr0-tR#a4Zl=p~>8TmA~XeOVbWh<>DGpdDtXVc?FA|*-( zKj9lT@`!wW!rb5RRx*QT3U}Kw#-^q-Q1hS2EDCxPeEu|FvcGkvRVO4Okilw`%De!T zPXALB$UxUUF)X;>iuRFEs?_vNaS$zBUlSY?WjoLFp{q8<#P;}g=g|&<}2bpdH ztC#acFFI+OpX4O*$vD@9mA=BP|K4qx)yPq zHsAx^iOZt#JC9$RwUleSpwV`}=CcKYjlkmZL<>* zjZ^uIoQJXGi5?c$@;b{YJtPj)VJ55uNoT-wpZABP&+5q7mt-(Y++@+3osN1d5YwD%&8GA649tLc6Z11qs>s#fc; zX(!ItBk+}y5h3@DTZZxh{I2Y_R}Tz;tm+(imz{WeTYN+Ar6aazWZa3>~ZZs*CU`u z&ZV1~9>-ra5$5sVEDT#zo3)DdBm?>SYUYRH4oCk4f1vyQDRGW~bFW7SbF%v1z^aBu zDR?uW19s*I_Pbkxfxm%k>A>={gCpOEkqgO10st?!7#G~moxgmtet zh+Pb6J_n@v1v9l(4v~(^QdZ>Xtm9Df%|D`j@T~CvHU-;9pXPJ235p`&qXu6IxPrA; zBD6YvpA6!^y|`76lB)+Pt))Ns8bG>Rq0qVBktB$RFFtb_U1oZ+4@#l&U3v3ve4vxf zqrR6ZH0S9om2#U)M4NsOp#Az*VZVOeQH-}JsA#d^9|s&I6Z-}MRcQhfbUkV=92B3o zgp1GA8$p?kWla_er}psag5+IeyyP?4$>ZdhXHy+GI??G8Wrna zoRM^n_^o~QEAsC`NhJk4cAy(G5#9TFgdeMo>aPyE1=LkvTCd}+hQAdtVM+A*Q~dfe zM5g-nzO&?cEVaBXd^U1*pN-wWwZ@nv5~~WdcDhb$Kks0nwRkE&W`^SW_Yso6MMw@= zj^dG7@zhfu#uD$$qr?G3zC6VXpZ|%tVLy-J4HR>ZKH`_z@Q|JGW%e8;gB7VfEQCp! zhynkfLmKa5GCbV$AyUK1w|egrqK|fOiOpu=+`oN_A-ejwp#x)tP5y{h+2{b(zax)OyAn*<5a~*SAK~{#{}LM+ zDLRsc{zak7Zt^BqLVQgdVtgH1Kovmsf2>83pf*&}#@kvOS}J~duh6OWfheY@q2CHM z)GqL+0I}>SJy#`_RVnm}5NBrqAE$<~K-nGGoe-=-Tp{Itp)8|Dfd#V7n-&OTFG+#j zNnGqvR?Vb8Tc$I2=TTST&})e+#DBTUhB??m-9qzFR$z7gB?djfa01-}@Ect!iQB_oY2U0DL6Z7UW%(*77!!Wr55R?@~t*Dy|WS4 zneqv`;AoBX55Ec!(6xye!-VD`j;1$Fp22c^cD3~Qq`o9HzVnbjtOy(YA`je_SHfhmB@_v;wX0~3+}E*O@G*5NoJ%-6=LP3X<_7UmqcWePcDEW;ETYe9qE*EHsY{YBUAA z2>DR6o0q{6)sVh!kXG~-Ge(JaqAFNuM&UZ9&AkTg<$n>eYiO&ro9;2Top@E+b`W3a zGKApSqr8?$qVi$%fVn=iB^o&cG*O(K{~OvjFtO}_EZqztcQWgh-5>>iYq2O9Ya$0) z$Z`iCBX)Ls3BTQqNfj)Og;UMGOXpNyh|oKkQA=(Gbud)?D?$L%z(Rgr_yZ1hK&M-C zJKA&&oe10Vb`iV0oIDkyikPl2<@wtnVECh6&1(jr+0~ibR6!Wj`BQq?ywr z|1mXH6fbcRpW9^3RAAY~1kQhzx6uVq9Qox}L;n?j?L?04A00FJ75(b^e2D}E??8** z%6pDz~iUCWo)N;T-@A#ETF0 zgQubIpbJ`utKbhbh_=bnjScX_7L_prW`d3t$5*0b934FXcr0><$M;Vo_lp{>CWU10 zQ~)My5y|;A#_gj$Z|srR-&i_gw?@KWKh$)u!x+oZ4;#6 zw}SgUy^#%Tz5ofEavKH~mGmieD#fW3nu60WqL*i$2)mOhD-UM=RoER)R<&rY9NOp2 z$ZFI&U*1A80ZTN1B)s^?0Jsm!i(O7WhD#l3a90?eL7utQ@HFG+P$VX zaQCiBI9o5Za(W7DT_*g%n|L4zsr#@|7fJyRpwguhUank0M>@e&;%J4_b-XyCasqy9 z>(4RT+kbt;3MQFlGVdG0T}cXFoxq|#9)C)|E{NrSz6bH9=rqRayrwha5ft>L;$tLU zUeo+mH`}@-IKi#d%8sq--y%Rd41c8Os-?1hvXt8(+xJB_va#SFRDaE>*U9CvKMtXW z_~#5v|7Zfjk#dKvexM~^r7qzYZ8SgtadTofDR|Fmu4*h7AfQ8#TCfwdFjEo@G_gi3 ztCeD_aE}NLgjUO=yocwjhIvW0cEw}3yv;C0Tp)hBm9J!G^Thc?j9q_`)4wPIfQ+)3 zg*)tpu7QzDLIy4v;o%Dv6Gzq(ve_1P+uAVG!i&6KLDPF5pcsb+jGN2mTPfmz={g=o z1Ss1whr1J|0@^jgH3}W$LI^;j#bX$^J!F{*ZzQ{=i`k?w>O>3pgZxlLj5wW_Jg++v%m{_)_*^DfpD7 zx%YpO-x`fcMh-X9^wXI~WO_^x6gkqW!#cskD`O$MfN z+rf^3!&`~9#F9p;PeMJ!cwqIGaLRyAfrDE`18vnGD2@3pclzJP=dEWGxyW-Ow&b@D zV~j`(3m}CaZp8GH8_AUT4(8wvy#n}vMpZG{60s#bMdwwnVC?zt*9eQOdxfybNqlQ9 z4(QE`^{hN~zv)?dUWZ?S#ofEJKiHjyKUl-ccKyLNzY^=W^uVjJ&#%21`}|#@&I$35Q~_uMz``SQ5&MOVc5gB@N&@s;5_ zD`MNd9QQoxmA~-!GV|4s%9H~Hihw!?Q#Lhkr&Fnx)n}-!y;J=XLm!~kPjJ{ot4gX?me3#4Rb4sJ zA#UHql1QOOJpRKOrN8H{F4Giv!yox@|Mt0Z3(k zV)aiYMBU1TeRK~WJ+3BG()wL7_SQ|5JLl(Vk(Iq#qay#ftj>o8dZmt5<~#4hvf7!` z0iwnGHFE9abpq{Rjs;mAhgx6E0q?yX-&D&GD@F4iP|y=UzrVmC(rM}AgbDt%5+{fp zT;)LP1=&wpGR(2QmghrXHQp`#7;ho9p6CIjY_ET#^sIcNfulodOp0% zeBiv6kiPLdDcG7|vF`zFjg0cdb7i%NXA9Kuc>!YC`{&&*XZ#^&e=AF){zLIcP}e1A z|3_L0GihvpaU6oIp^J8TaCeQe$v8Ugm7ryK>aEG7H0HZlUL zYiBH83C0m2nS#%ER%2vVj6JSWP@(l2@1Th=FHPeB*|<)Cb&57|JbG&vRB+7I3Q-{i zr=ajUIvQhdq+%8;hX+J97+NK+Hs1<#yP#FXY^^|-feO3c2`g#ft)g01utJl3ht|z! zvz;}4v`}{bp$0xmNT>WO2wbkg!3x2_s6kvBd2)PnDtc_&6?deoJ;Ph^P-0qDye;9r3wA;_Hg69FcLc)kMJh4kKaI~|cA5!)wFdjl^ zmlPUgr_Q7KX^7H__!R(?5{8dmQV^%8mQwqDQmBQNtgBoEABirOiZ6xQ!oY%AKm$K8 zU>Q)NtTfEMq<*ZxM~O?QSbYL}Rm4MAr3P{5Wpd(!u=or6WK?TO-T7pW%UrytcF9_3 zMrPr-ii{VbnHKFq(#VrM6v{vI4e#kU`koS87msyNeXakXUT(mH*blmon~fLtkKJUQ zx7sx_$cjHXgEczqIxDjHhSu{UfN-_t#4X4dx!)rm!gb>^%^5_p`LOnS5rJliJ&sym zXS~{#+Z}%a+g%zDTs@3-5fVN|iX`KtCZO^!+C-dcL{0vkI;r~7X5lH^%AcO_eRT{j z12`rs<^s|>t_DO9_V%~q!^33Wn@^EAMw|!5(~(?2PBUOJ%h< zJSu;*f4Ue;pgIXZr>%G!cV=9sRQ3d}U{xeBaLDc~N$qdM#Yh72$M<+}m-k{nHA##2 zz!G`dsy-2VL{Ix}8NW;rvp>l*&yQUs@1;02%2EQW5eB{h z^Gm#iEAkLs0nJ=Z3v!opbCNiakI%!+%1`1bjoF9i;wbY4DPwpIcj>OpVuA%Ky*_&koJW1Nc1h$fQ=OJ3qytF6uBk>$satUdobSjR~+B^#k#@&S%1 z#m@uOARn|HS87-Su>a86B#S?t1vb~xfwL_AeQRNZaHK}QFrrijHrYSS74U;z6*RHe9BRy29Axl)?NOBPfnMY&{Ro3K!4$l4oJgFX4bN!;ODIf?c9W z8~z1}iBjkve4jeLzg^iGIJkJ*K}Sk;(HR>$PR*!Z+%Iq@;jtV*unF#LpjOLJA2$!+ zx+lS9WCkm+w?BjHco&k&otAYxA_M=szKxo^Molnfd1bAoNi8-hr6H7$o9tHS<-?CL zljOh|yYx7U4sLZ5g#!&7NszkXg<1Q^ zUDXpNEu_{Up}*#aM7A%{y1q_3HvqLiLj z?J}4qdji#pp0%IzgydrYvJ|YK042X^ii`GTA?(W?rMNF2#cSD%Z%koJ25}o87S0B2 zCp%RFE|r2;wNSX`&_`qpH!+j>nWV^@XZZf5l;oner${ck37#sSh7W18(pvQ7Cww-U zV5<3PM8G;TO$pw{ife4bV$Zqrxxz&|a$l*a#2F2l!Bqok&jb`5*ZhxgF~+qK>B%X% z%YfO!^`k8+GFg;8#3jgl0<{JX={PDfF1nqI_t9w+iMB$*^njM+*a~rXK?1IBXUla( zuM#XirqYY=fW&3<)zS_2uiDQK(;iefD|Rr`{q>Wv4#8TYOYA{pK6W;Eq#@=q4qKul zs1DkKk!Njo2-CwwL;&A~PvS0bjz4{`<)+x`Up8{Cts@ng?>cA2UNoSZNy|*&npH)R zYQU-XGWyFOCMRv>cM6nSbK=d&lvaH??_q7*2Fo^M7+!OM;V&U`MPGkdiVL13pu9IsKza9i zSoFnslm1ngmZ;93E+zh--tq2H3znMvKN*I4E0_UW+Ajb960=`Y8O=x8U3DZ8?FbqfOLc*cg;`po(h&to!-QwX&V$b%22W3Zy9pf|kQTXKWogfWs5w})ohoR@GP zn%o&XEFF2&1rBY(OU&uzg^mekPgWms_-4_NwV|Sx%%Gp3+6=q@bsSO?WlW_st;e;x z`|wWR*VoFL1xDC>X&{j9wDCW!IR_D(V1X_E`QgFIX!3eFoc8(>t9~u7!lSyXEYi%z z6iyTy`32#miUSIrDJWD%O)6BzPL^wvnb6C%qa4iij6(Rexu_^z6!qbvUGQ^&D9Yrb zeeiRrD9XXV1kLaui^6g228D4g4c}jXDk1y;y>H?5ALeX|2p2T!OAVsw-A2TnLl++LoMkt_}5Qn zwPyZoFn{WS&`-!~V99b|m>)X$x8L%+%JMru0Y^mgea#X(|4HCKi3!|kQiA^dRPn23 zNg6llDw>>YG&x`V`p^Mu@#2MaH%Ok_VuYrB+>%W5=TP%!j`>s1rGz7eO*1Q&n?H58 z1(F`)Q;PJ`__pzTbsxfg3$FYF2HYVTlhBxO#YC6n~s2`)vXwq3>TNOAR|;D z@kqaoIS;=Rt+?N4;CgQw*^_tYHg|EIX04kp>SX?qsV5zB9U_EHbwFaAy!f9Ht5i~{_AaR6%l1`6;SD8O%` z03%XCt=~WaeiH@wR~m%|3h)~!z;B=czkve$78Fp5D1hT(RMq$y6ZjbukbxNdVN5`J z0h&5(t<`AYdYafQbYGCK3o( zkN}DgwE+_k1WY^-(D8sa;~tDW9s+Luc#gEJMoT1DQdb>_=(MyRbv;SnN`TK~qc?-% z6QhiuitRO9`UsPzrF>>H0f)nrc@Qc92I~aZu zlz0bH23^c+k;zF^_%nxfyZy z8nJM!FQojflX#7`55vV<(;yZWHF;XqIoxJ7)EV)V)HRxDLu>r9XcHV3hr{O$iNoP{ z4vE9zHxG%!;Z-994o}Ww9Imx_c*QRnhHiSf{TVA7S3M^_DK*C|1*Z!+%OS?paBU}B zWK7I;YRvt#Q`uv%@QSZQRti>Yy-%~}^#vVpt|K$us-N#hF_=&U{chB?9O2qk5^AhL zi-a{)N+>r}rMNarCCMEyutDs?Hx^OPO>1pJFOp)2K#bJ0F`8Rz~SIY}yi0<+$#{zm~` zGIlz%w?EVw8TcCXr&G4OvRhQwW|uVTS2@wO+SwRFl$->euZ7X00c25&Eyre?KdaCg zSTxJ#yA#)Z$pN|r_lhK_ytC*>123r093)uCbZ6{`iF%Yjq|iWuwS7JxzK~U=`Zg=; zaQbImI@6k0O@gF-G={d^dlS6rng##)X5vA{4rjwSN}UzG zXvOl@;wbGk?nK{oIb&O{axh=p`Hp_sokVh#ZzuMz-7$tXS45O*2cSP9+OhtP&NKtVKnNukdKew&$3 zuGLRw+N}d<_Nf2NbgH4HHm*gIhR}$%;RJ|<_9X~}&g@z#*uk!U(asraof)-u*XGD| z`aL<}&kb$ST;;HAubZ1HHZ@R5v1Cs(giP~yDhfN78Y zb8Ts~Ir^7ZpHlERrY!i`@;*Xt#Ae3RO>}K;U#}1IKR5{GS;R6@FqvH9h-;C`{1_Oi zO6$m&AySXYvRs>;N(j63zmBzA5dqcM?0gxyoQ+hkSWJ07@I*aeIZ^V}`eo)EZr6d@ zq3$K-UbTxS+~8K@R)2<|zm9nAN(fXisWg#8_&_*YmA}&(#><3yIL9wBW9-!+Uyf9u8ph z1Ov>Ei*SX;qKfK}WrfwI%i zzm@K%4rt$Gwqj17t^PATz3?BLL`*0Us`3qjP2PzrrSsqi_2gDuzz-ClUg|NjkKp_A zxluC?4q|I~OI?SsRB0QEF#Yj4=ya5EMpjM(^_f4}zO7Js63gmeGvo7VbG~X78G}di z<@ooH8b;4$mQ^^F$*XV|>jO`=#oR|sU%t zkFVZidG))i@YSpEE&W=_8h;ZY^`Jl7?)%tcMnev6iffNHqNxQA`kwlwR{5Rmk$zT7 z*p9?k7}_U2w=K|S_vHq6d2?7=!>)2l)}q|Pk{a=^@McNrMG5?iOh%1N27^ zGQ0%Z5_~rVcloAqM<_kiZ{Yj%1CQVY1NJy9apuYtOFVnR7sU5Ivye9MokwT`|9k}P zj&K7zou>Vr$JI}8^+Bx0Oh;P%sphld&Bp3i2e3^enbt_bPiSUiGo`Ypacb&^Y&(<= zr(_=9@A^05DkRhv=>n@1M4*t1wqd!GZnP?U*o5&Cg8ij0S)5^ zwjFk)RBJwH!$LvqnMeDi@`rJqR>ql-D6R_a^D9zps;iI^Us|;746j`6%GD#v-RR5jd^K;yA6rm+5u`zd*khMr+Is#HNix z9&5hNV~w4S#>sg~#97gUcNKUm+1}tz95sDb0(xlOwNmG?wzV;jg~M%gp2cDb_KFoJ z2rM3DwaH=iu<7~}67r#apn@7MepF{jQar=x1>Y=Yta|wFKq%`rirFR#9d0sJxLs*^c54UTjjsGX_GYzkF#~Jb4oywOa^D1LcX~*P5)WEE{KLJhA-0834L5~y{MHT9q|&~>TBHmhapf&?1g~YJ*y{I|wxho_ z19;qmm8H$YaFaUReD;HBtRIadLC6B5~- z=sH@xFWs$dmz6MCx^e_FZQX7Pt9!k(Vie)MF26q%cO}>`03ja zntd0?h58|ZeBh|RY+7`d^5Ix})KxfwT$R(dtGwar23&m(->9jh67h2KB4V6qZ?#iU zg!+_~8J%pAb9K8%-m+9zBH9zAOU1L_(EEtKZM7Lomns!MoM2tZ z?`+1B>C765m-bIM3%79*$-DfT<)TMhJPJbAe&t6zW#{pSaHgvd5&P?}z~DIg-NjwtVSDvW}twbgo}2=&&eA z5?kgV6IFN*R^BoWG*VFxYgcjqI3HKZDx7n!aq1E+AeO{N1bFudJ_}X>- z4eSxw(oDXn2#pk28r}*#MBjguzuy5STR8(?+w@y=RI6?zp7M5XGl|=D#Ww%MHqZY? z)j3?9&DHL?Xz_v73%I&3SD(+-%~*}MDB7)1T^fUbC&W_>x3XWm|y5t_++FjWBe+NoYS4!|37(rw7c4T~8sFdS(fC4iA zhjnC2JpG!LB$z;UuT~%jA8NRpo39_NP&KQ{emr``<^bazwUo zb7y}hD|z3{zf>+d-4=y;Unnb`E5&s}`V^%u-ajJT<%I`?mdjc3m1UEk>6=3DWbHHF zDrNpaWjji4OyO@kd}$tKXKu+& zsq4DP+cZbXjh$tsf$S$e%06SfG4fo|(Gsz3z>X{+T;J|5LKKngyW zh#>>Z=regrYo79_OWEeEiX=J%hwM@it(0&ZszB-}DDm~c)I5zp@e_r_VUN923{R z3}4~0JE?4)jG@s<0R?3JnG0^$22fTrb6@}qHQZ}&)}4hk;LqYUsU z(;e(@x-(u5&T7OpM;Id&}@bM+qZGp)8|7O$xD#xg~ie+SM)w@Rx=s zqM;j>tJFxP7d$Gx?NO;O*J&3<>`77zN zUbT%@a=WaJbg}BUbl3rq@r{FZ!$C)OSn{W-i~J=8s{t4tQrTA}dG9*ccP=S-M^=Z( zif^Sa4FImh^>62TI4?&WB?W8buxq8PxJn{pc^uBbX`tCV(ac&7uSW=)o06;Sq6|Q* zvi7Y`@}{Ugm0lFTWOaf!RVq*IS*TD>qlDD}X7?fuFH7oKlGH0Q4fz=84?CNi>QQz9 zhTv2}&c=$Atp5yAzxVrBwUAvk33#zx;m*-qrjlxOBO(O6Dfzbg3 z(}9STAT|Wp@OvG(`tb-11Kc*9z%(xhWIEJ)mg1qwRl;;M6gqR7>jtEQxqJeF-N!m9 z_$wi{y9apP^lDu*EY>I5xFy?Hg5(+W(#e{o2SN40I60I9{&;E zaN`|rIB%^~ww@MGSvHgL3a-YO@bGMpLg9h5UpFTa3|>frDvwNj!lP0XkH~GHQWutS zx$U*^S@_u1cpDjPltLdR*=Q9ccX)gUn>?&wfK+_HpbTZt(&$4-AjG)=LhgYDf(9MBhc zcyL!Ka5YSEHJs<$D0Qp1msHksb%L{`0A_oyWy76S^$ECqxTSQT2TK-g!rpX3dP)lI zO{749Aj(pOPDtH>C@l1DE)#?l%3#mSxJ(dIC<8+JI+qDT3S~)vRk%zLQz+{o1s~xu zK~SNrqZBOUGC@?KELjTP!eti1Dh03SG7E8)f&;nCLSUs}PcE|%St*#nlNN+Fa)Q@c zfZcGQM(^-3PeTxv)s!(xYIY>7--GHWftD!|FHHkcKmw;MghL7xBbT%bcLkwfb%GSa zoLxvJzMyDnN#~R#Ns%2bR{!pz1V?U(YjrM?y>*=_il=KOaEMab_7c}Sz8=sh(@J)dKVc_Y4CZ*~001=srN90#6ABS{fRL>9VybIyKC?1Vn z+cU)$>7cKpN%QJo!TwB|SYN{WtMa6CKzd7(N$!;&Qy_AG5$Xu<+MR% zBiEVwc_`=7Rgqa%enM7zy2FczKDm_w8%D|S^n{*O)P$IbU3oWAsY_=NmHGx2sPZ|aQiTVI zh)G=S;Oa+apfx&HkK*cvyQn&HCsohIxAEpo8^G0HaP@Ak?tpJE!0HsPUdPpIxH@{L zem`{NhvvVquXkk~1NzuR^l^_6983}kq*1VVzH|&n1wV;Ueqa*HA#^oNM6v{NYSyke zGI`;!_Q)hgozGK_coGXTr6t{nWvoN|CoRFOv%uX-OEx56T!H>q!4Mk*VniXPF09ey z572v`K5%K0(Z17pOeh1(~o4(&KHWzW>m4 zJYR2G=*xxJ>9^;$->D(*zC-W58t+H;`nf4YjSOo#zF(_#k%?-glQ`FaLT(BV?qkH0 z&2F04CC}dK8FnAgCUois=A##q#=Ptk(vs^8u44_wN`4mismQ*H3FT@ z5kIA11zAniIjgh;76=cp(6}as_1KX&;~1r2LA$NSYM?@BqDQjkk4Z6%2M9Fzyt$K@ z@yPJ7RHNPh5i=h1nLfnw9YTzm#9|EgU>TI#zN4`kK7}T>E{s)>mBfot=nkhQ%fs>p zJT#S&UnC(0--Nql+>YyysWU4dwyb}@uI;VA$NmkI{W2u`ADa9=ax)|eNkI%C&wN0N z6!^Q~1M-*;=pp!kUQ%eHxdSMb!(3nK>J%skPZ;tcLCjRB|$nsG~Wd@zm){( z0MUC_IXsgfP1?FlR|PziARQn&@7f5@BuEE{#=9!vIYy8+_&ntBNbf;R9wCca03qK3 zk2lN4>>f(kj0b+p(pf%%3#6c@#mf1i0m$ zJ}(KMujc1>JmE#9q>Uk?^){_Dea`!|tOhv#R8%UTlcoi8*md)vD7q=TLXEyebzF`d zo|lWA73Om->8fUoONDZ>uD(f>;|&xlkpX(!C|}p*BH;lYkM%qGL)xojM8bn8rzF;! z&lk;4!%0^4C`t+qaEAZkka)I6-tO}=w2lkP!H?@BMY1XegWu+Rw(?G7h z-tFqMUOUI!xsbq2cm?F*E{&+{WPa=_{#a*cMJf{3wImrm%SPDb?SQ?$$i047DC_}g zb)?XG_~YM1nPR0-PvQtSk(rkFVa%d^d`@K-3;c_Qr3Ak{u;F{n#(eqa`|>wB>AwN& zktS;?R7A3hnD>w?xg(UXl8ss?Z~y5+Ax*o<7Kf3LEu*=~7EyKAvs52M|= zX1h6}9VI^^v(Ct|GkV;zzpqDUmQ!}=aWqD%>}I}}&99MBgU~WfvBjAJ zi(|I_Mt*Iyutxywz-No|M*OM`k{q_krDsfhP$AxYSiHFnSN|=M2LMe1^$E+D@;WI5 z5YzD6JM=uP=qGHzzi9U=wY!_(-)Ps%Y^mgicubqc=*u}$Ihr^n0 z?BDrjKWjxlbEqFQ%#2*2_X9i{`U;7#0#U8|Xzp!1vh6{Wf2mA2`aFwIvYi3d*fnmFomFEft|g_RtPfdV&xja&G*e52Oz4) z2MUfw%eB2Yh4OhAR#7VyU%L$}s+uvv1P>e0OeB}6?6W0=k=FagdaRShdf+>1D~{q( zQPoRy_cm57ZvLzI0dA#<&EU2Wn~R}dzjP~mMHnH?IS=FNQ5|3{(nE0I55eJ3sEL2y zO~^VQ6LU>Kz8XnO{I3Nh5=D_r+VRD!qO{X zELddsJ!^y~Ij%j*m3}>YPf;66(NZy8*M+xWFcWR&n{1dgi8bh)<%ccL!dqA}M%JlN z$xosk*1o=rCUq6whfzLKnBA=f-FRLRwAix57i0^PLG(QlcO|M7Or>FM!{7g%%u%yH zlFGMGJ+Fdx#|xOnub4d4csNE--r%*g+FKBLS(&@*d0^)|E3V{GeKKAjRhaP*esJbw zjfAEIV(o-7;VPTk<$U8EDi02pZl~qN}dfWbr9O;yG0B=Yu`rZiH{r0 z@2axw?3vjdGT2$Mk8$3sw{(j=krh#(O!smUVj9zsNerPYB85>S%8b@`k^rL(dWnQ3 zVrMz-i*|G(ZQ?-z8lipOG+EW5;%@D;-OWaSUr^JqCA5@aDvy^q@6so?ONWk;572tM z$nzvrwP3jZYS_0jYah+N2r25uj?jm8H=zMvKWD`xUe$MR)>qZri~w;E5$W+Z0L#PSTt)~eg^NT9kTFa)FJ0rPcwoW!e$PFq>mOgV#oIAaV;aQoN<`FZX9ea)N>51M?;tl)8NCx`b55z*n3MUPIG6Q z3gDQQ6wk5S%=7iTkScI0BoTBLvu!wSqF-p0X0#%H{pZiQ)ox0^Na43&;kt^+?0tY| z{+$0RUR7$BWaxFW^=yXnWIW0s=xY?(s&J z;fhLx6cD-$DJuSF<;z#!#-ln+qngE|$}mPnSz~y@oKe9>!3ZgnjU1lSQCe0VIMkN1 zyQCCLh+)=t^D=}w5^MAXce(#2y~~xS%!xiSUW8nFAkjda$e{FlPS^)ld#}-f9u*Xf z#znH7F7IzMP!(8H;-u}bgEJyGW;~E(K}P{DJistwwj~ig(}edlw5)2|!NkDfrj$Bj-AjtFl&n`Y z+kJ$zu5?!H=7m1#(KmQ2CX=Xmj*EXVidQ#UK&kgxpyU8h+IBc8aOk)JO(+IvkA8;G z@-aY&KH)PDUG`}3~VkroTH$JG)h zduvff(yFKTxBh0as1tc)gw>+s`%{{z*(PR@mBh%7lduc7 zph|(;xz(){#ZM^~;<~qo9KPm|w*?TP_U$UOU4YsFWWT}}2b2j60mA0HO%GExPz)sa zeX9ZNqAqI}^mk?gpY}&tz3Ci-T<-_*T<`c(4ud!BaMB>PcrcYx{a{MLcP-4{S%-7Z zyRnCJGPU|z8}JMs&bJsiwvE;gr~0QcQ0?w~F^O^%4D=HaY-=$lEQNkYTce8Q+|47S zbr{S2ljtQE&oGLwpkn-JB+_i!W3{IronAl^Lisc!yPA} zb#8LBuN>v?8~V||4`GH@*N=AN=eIy^v*t8L3(WKW| zF`A{z(Me`6>Ahm~qGi_mHd8P`%yMjoy#?RZ}n8q~%;2 zuNQ5!S@fbVp%S`QIATkktB5e80CdFQie8ZKW0h&M~psna-2St+AQZV<)L&_A37*r zANt%=7JX=~=6eeDkY`8*)ujD8^e^mmKh6GRtm>~xyXrjd-~&{7btL~pq5q`* ztGb<~|C}7B|K$CTX-E3cL2>#|YB#6bS^CdT&4&K-m3H)>y#L)S`p>O@rT=WxZhj?J z{~2yq|G6}N5eBAP^`Et-{&NNDKWk0>=LHu1XOp%*i(i}km-^2$$v{lb(tkE-4~sX^ z1}yS%tp1ZGET4Op{_}90{*(9r+_Utbm2vt{YImyZS^Cds;`N_R+TlzdR&|S^u{8Cc z)X&0Jo8uj&nB&UutonlHctNu`;l$Y zf69NU|GXhy|9RY@>pz#9`p+iq%d0Ho~emZ7O_|nloE#r`Q9Dftz$Z2OB>?*`Vu<;PSFo$ps4S@&X+gx8BHch zjB`f6?x;gJqhAv><3G~g9YiC1ThsL$9r?N8Jv0vPT>I0434slNFr; z@0xkJtodeSb%eHbT{Fg{-G_l?NxCUiMmh?Wah(d4F{vVb%U}5$zy!yUw;S#>D(Nu{ zk5oP}75XQ$A$Rtd^Lvt?+F9|sFj8-_toU|nN6MR{bP{VYiUQkNdBT@xdemX=rth7B zpOZ1idk?96Tqi^?7!qtUqNaTxBD;BYVXpE|P7MRUW~9b_4cDvmI^4P}TU^+SB}W|W z4;N#fHaa|ffXCjZ{j{MuTECp~1?AKGIkq^b$s3NKAQQO&_Px|xAE|sS40BYfoUv2R zZkU^F-&?56@AL^So0bnFhUMubXa6zxTv@3rw9j`GX3S5`Xed-hi8hyi80+6r7+BoN z=1uv;y8hPv+q3rn&;I?XyJmXS2Qo2O+hpM1ZcbqA@?Z}PCxrn_$iF~)aB5rBUDSeG z4V#70yQJb{c8Z?in(E0uC@m=_4u4@fMo|)-T;7B8sAG^%k2)jMS@8f1hG&Ke?|=C0 zqy6E#AR|0Y-l^BFSR4N^Id=f=sifgBK;C+3wVV5V#wIlO+$Hz^E-JYttc1Aix_59p; zzO*Ml5@)Mc3LeE%r{?9m;j2yPFp-S8{{DuM>2LzBNpxG|Cvx$*=7mq?DU0$8U5alK z{CD?MlKJpFY{w@A{>8X7KR3J*SdJ{PVC>QESd}-aZQZ` zNHsXvJ|;YTh^*x8)TXRMjGniXuQI}2sR6sn5m(CuxpbAvF(L{K4TdM!i$BZD1aYtX$zdn_QzhvWI%S2*Aat{S{NK79t<`)dn{-J z?A(kCAO&xUk|$lANNL}sRcsI`Gf7U^o(AkkY}f3+_w-OQ*6T`tuKO?EY#zk5b$`%o z^;cM!pR2gOfxWI%3jU|HtZBx`$h)$tbg(wTeB`8D6(^jStE+QCrt5f@D0!^8P@vnmW33JbzS$~yOK4uo2G80aVdE-;lBbL#vVsJ*7mvW9Z9SMw@8b1fBKF|W zQV(HKHwsVXJx>mN;y^0)*2qMnX20E#Xw%6HQPxs39=cQxOx}KN*!?Y0d<}@*DP;a@ z@V$SG<^TZRo^G{!f4A!GPu(6ElV~Fbojme&|MpWr*G04-bgGA8b($9O9>h82v=fhA z*tw3pDbY3!Sm4M&7~|RX(<1$DGU64*<_}&VtH~G?Fj@a-cm=9=^TM-vl&75-u7%`< zS8k+WF6ZO#XW%>Wiri01J>-cZysHnOg7*9|_>Ol^uDV5(Y7=ju05GPHL=F9F%PrZ0 z`dV#%ULpxjHORQ1krQo7bygMZ$1`qqcz>)@o-Suoo)dZ2xUgJ2;IMPyTb$-DnC2E zQ~UB`Ch=oAiIg~mzWsS_qE&vZ$|3K7%-oouAqAVo=HE=QFueOPbSR-*x6-k`_N@3) z;SQ$;VG^)dFk_folG^SWt9d#3)bbES+>elnqRH+7U#{l4%#5z&`i7}W?<^B&qOVbn5Fd-i12Kk(cphp8fFH@ZfXlGV5yuqU;(e zv^3F(UhP(UVMO7ky1+quwri;rd@+Hu057>Sk%BF_!-G8@ag&e-kFuJ%Zy z>heHj7oOQT2UJ1J+0{}I?Zw-HG`jzww95{d7EB3eCWSKS3Uh>3^+O8Kyu47On`xR` zMAni*6IePDpsp2zu+`A!AEE1LP@^0XHP*i++~wY@kV>NoM&%L<3w-ByK%kSO z#y!spc8**%M@Q{kpseB>U&7$J1N=XdF*-aK@rzlz;8pfOqb9KZ<>S(FRp0-^-nYO< zRa}klCfQ^okKORlpjZ<@CDpcBwQW*SvXH>N^e%})1Rn@iMe3uXBtlgb*sSDwy%irQ zZN*ZnEv?$pRuEBe7cdF$BnSks5~9Lgf)IivAwc$j9&>l^hTx;0U*-S(io)KRId{&P zGiT16nLBgNlOSjdLAbnQyrW^HfHBw-DV-lFo$FH#W7Oa2}yL2O_llOU<3(4j3-Rs}B{ zW!H!`ihYk{UklXEN692{3}9U#q9BeSl~98a$M8KT&HKlp?_ob-tjD-@*YX|EPZLf? zzmv2Y&JR<0Z!@K&yHCZrsv`U=j=9%oZ7Q!~MF=_@z)bj;N z-xMzk>_Sz8Y6jH}EJG!hCs3}&>RVaiJe(jDPHjs9Jd6j0^FHs+%j z2E3g0FIjsW*V2l;QL^tM0eX^#^lDe22tTT!uO(JG&$q!FLg~EdDk~-|1&a}Mc1EJv zl{?T4s9HJ`X3zqtCIs3Cl+Y1$uHl+}j0W`%MelyOlyz9i4TI_lV>~G6dmk+P^`!f$ zx*CMbTCM1pB}lpJCD-uvGOpbA!QkN$#dr0Z6LAc27JuOJtfg%aH{SaT1hvu^(K+>u zh2G*wX*uvKQaTeS8>8P}I}IBA;-L9nMAH+)Lf=3to5zl2;NT~GjcyWlF??P$65rsl z(YOQ0#*5M6vz3P+g4DmxAXNgSOF!k>_+{ZcI%4(I{=!fvj;$E|mK#KY%{CzE!)Z&@g^D4|y@u}Bdz=RZ_`npWIy1}|-x&mw2LA!p7%uNvs`{?8 zjVE8Q%*nX7w+ss!=baj#nSE6o54W5kD^*x`EUqs}a@!o}@HQettnlw1A4%~Do12I-dU)i+7@V5fND zT5)ybo;Z6U_mZyVe~Z&*_}0)S;Gk&8Wq?n;q)RzW0QkmDr^-VF4;A6+{QgNNIppVR z<+dr^jf4>=BR-6Ii|Z-Y-hS`3&<%Y6DfI05&s$Ao>`zc@ z*7fSC_%~6!0H}5Y`3lER6n{E-f;NusGr#{m$d2tFyj3es_fW(I{C0r{=vTqIrP!^w z^6O>2d?ufKgz@eMy2g6PV6OqP0@u=+82`!m{+k{2l_uY{w5ltIW_h4T(@}-v?Ph7# z3^#vgd4V)|f!hNw2Zy5aHpYCA%F|-E55!;wvvoJY4a*kcEw@Yhlx$fansMY+Qj!I3 z3Sb>FoRWWr8@l#>H$fhG)muI^&$YC`4ZmJj0B_)0;uR@s@iOCs+s)1~eqMkR6)^A`m)`Rs^VJ{#8{fYI7x4)r% zgEmJSLt#e3^Vvm?Th<>MN}n%*;iG?hoVXl*sI&c^_gwIigEMEW1ZxKo(%!sh7k<<=N~+0wXLj^vRcxRAb85p7)UOG#Hh6X!3pY~y)aCa= zk~Z&+js-YRb*hEz8jKfWZt%maTLG}~ahg|)>tH}vQ+%BlJ0h;7b>X`svGaNH`m;s2 zWj`eq zTZyu;l5UM<-p9^V=-i@4NN3W?Y$v*F#bpEfXrK^T6cuHuc{skDP#fMYDPaC34SbtV z+3csY?E4}`MYxPvV~bXbC#@Ib8p-&9Q=D_@R7l6Udx=7dD5EW@>)>iNknn)Jcqkm#b<5}th$>N+ApE@Uu)3?y)<1!SFl@6f^O?P)Q-BXZ9s?qT;CuN~~5btp%HhrT8h@7G3I0CGSy11Gv0Z}?7{8U7V0 z9`@26Fc|FNBf@_Pi!p6Bk3AWQ#V>*`!=+2~D9)?{%czBcaagxM=*-t1LpN19EbG}U z`9++;K&dZMR$$SHX^rote*hilH`4*IBg-vzxvHMdK*^_fdw73sUw9VlIhc5?p21C$baOU!~FvGvYh6sMGq> zqR#bWSkz(M8{@I3kDKdwpH_-;QxuaMJa))pQ?{1unR!KgQ*az0unENL_G!!Yfw zyiQ?IS74hW)hLX=O(T{~I1eXfItm@Im|Z<8oVMjf=6sA?xchkwmGBAerOfYRW5eiU zqeMO#2|s-z(glC5tPNb|+fkBwqM+kH{dW03 zKIvbb6#niBbH8G|i3iWosk;eJ$%`US94FjuTp4Eh`dO-tzDptvAXp!p=UA~GMSqoV z3-2?2xB>M5TR^10W8&NNDO{T;QTAS{9;d^=MTuy!AmqavQ>e1={Nof~2Bt3T6>Pq~ zU@rApB?T8D$uK@#kCSXNS4>{7h&NQ}+$5|NdWRq{go??}fUGZY(ruU@{unw*d@I~- z4E+SzF z+dFAApS0upsf&6xoQ#gcq_ewJ!EXWgtO0&&)#NmcCtlGGQwB4qi_~D zqk#KdOXmm0BqDGZS#Wm~+S1s|wV~_m>c$uvb*I1C?r*RgDc3Q;3n$~KC=yi6Jc)Q| zo+4!kw3y6ThW~_p$D-rNtA0eAT2X(0q`!TWL3xwq^X@ZT=a>$>gsY?OpNvWqb#i!G zyJg?9I0Gc;-gA4C3N3w_UwgWpNh|#gr}1Q@m?9>lE}-c$QSiJ-;JTUa3lED;EO;vg zTaaR7jr|VFXn1ZL_2&v4JS#B4mlL*~cKaQz*$5s9g7mQPGt+$a;$x8{1>O`Cc(?B$E9r$l1`2T}bhIn>c5AJ7HQJ z!(`NP$wCr#I$eF#!EDiES+l!w)s7wy-9gKu;2zIIlj(rJo&(jz^5)$m7wRii5vmpsHgu z$%-q?J7Y-1+HDbDOXt%ia6+6LM?>tEtWU|tW*04MnAWQ=jYM{oCCunK9Anp>1>8QbeM-_ zaT0x9H0yYdcapH{$~Gq3ICSoqvk)0w0;kg`f$R?9pLKyGmY%ov z6?b5rq^p)>i~P6ws$*GlGM>ld`?6a{pNNQHIxFyAz#Z40h)lYQkWPCYUn;|lT+aBd z|Ka}7H*b==%1;zm{>SL6na3;6D=|SOD@)8|pEt*5G6!=#dzaC_o_l^9m9D_$nX#yk zAbepmfi1t>lseoZ$<0_TL@$>fuJjw|xmxo2*?+=hC`oed-4h3$J!=~-N45C zWHO#XRflgpN*w;atGBovNkNZA)gVzjK@Y_w26>!=b(S{KNKw>}@tdi-pown4U(%I0 zRM3XNrKg&KQG$GA!8gHJ>#}C3{LV2Z_DSGh;Q{;%U;mIS` z{drg{{1xYt9;ca0Za!kpx4ptVuZ=uH^KDaXzMWpfCD}*VX$%bz{{hZC?}(+V_Zf$M8;pS)&-i`u!I>K^M-OMw>qN9`!e2;cZpz&{AOy3Ykul#{Po-B~R)EqaB zZ>|H`ujQ1F9VsqT2dF?KqvL1$XIKMgS-?vY9yzk^)hB0kPh*)@5 zEwzpd`aV1dUYrUl04BVBF(Y%iOO(&t$$eIIIF7tyMVy@*pWV#llwEk(wDhcti`Z6C z(*bLL(`1&M*u`XKuFIqI+0#A6w2;HJn)6xM#q-&Z=*|ewXAWQdd^WEzI(`QCviXmv~a{TVyjWlL#M)n^5Zuc^cs)7vZYTU1P&6&#cH|-~g)psTBU?N!+^wg)E zK8_o2e88IM{U)rlI73+B&x`E8HZf_Fo@LTrYKof%C+uUeIglNGtr_9j(KR@LudL*5G`6N0^ANh>y0}45$Z!4xmHK0j=-9lL#*|prf;AgCZ zuw%V%+QW-Ki>+^QytOTf?rxC`(e4%~Q`!y_!|!Ffjf)>>8w}%7(A>R0BhCGS6Q?CS zgSq^#7R_DoPp}OZ9;9XG&%5Dq-&N$#X(kV0{B#?>m}*$iWKlscbrT{#^jI_#PjR<& zXOSOvA|~>^MxNliOKVDk*f%dLF_HfpHY*g%nD!E0BI^&_KLCs5#<|w-qYKy}-K(aC zJlCXQWt%YL_Zbfg{Hu@0@INC*@rMxqg%;e|GV&1sBPutg5!I|V(Zb*;*|5> z)^qlqpPizdpA$t&tZ2D;6@30@te>-?EPOZZL3G+gym&U0+-8`2gM2*OGBCpn8mTus zQfMQ0RWfblCP8ZX$0$ilAEv`QySexeT)Y{6mN#JW3s9V|ejK(Nc!zr^tBth_dkIH1 z7c`Pc>a`XgZ0Xu21QRZ3^gQGtt&sgV9X0PHA_T(lIyu7Wqcf z2~ZQ?^yb}%Kb3)uj%b*LLFS{r1a}7`O9s)u8C)>{%oPB}p0he _#8bm`!4ODs)c zqpb)!0ij@t!r!l2uQ(a;`cK$hWuKzLdMb>z-Y!z@&9v;{oesR(ZT$91Ja@K&cbdu2 z0-SE4%%D11CGAmiw(`k8DlqG@@YKZ1=gU22qE9n~IJ;LsPAGc2%52h429 zDE*G4X*5``D(4MtxLU+QQ)F!kmTPkntuf=mlacUp zBWBONgP&f)>3gYpx$8=X<-3+9VKg{<{!jAHj+8Vuz69bqS17Sjm(mqc&L|xq1;CO4 z=_CAsCD@B24E$kw7lhU`hr^Wfkykfmp+c)Hs3Z9>83o76tQ93dRQe84r|X-bm-jb2d3lUB|O{w*tA(13m31IfV`}bs(nHuWRH~I5zry=& zBKMLjkb>0Cq(B8H5?p0Z?W3neR~a6_1%qPVKCGH{pu@-cFzsRo4VRKc8ZKqdwKbGDTzv;Q=;JQg0n=a{&>?C+0_RVc3i?8EW7;lzyhK`3468w4N~fm8r?Wbxy`9pxc1qvhDXn%&S0%)!H&EJox+0(NkJyIxwoP)~ z=UDhrU%COMI<`GB*yc)eY=HpHhM@OjqwQX71pYM;8!n;DWr zZA0YN+I+Q2_E*_`nbrBJuqfPB`Z}OiZmV-Wo`WY-^mDF>t+C1;CN3Z#-totT2B6kqPDlG}Y*t0aG;-KUT2 zkZZ!SKV%2m-ll$V4c$?w4@^~ZSC`}}_KHcaLqSQuvRML{Nv_6Z$zS1Ca@Rg^2s)%n zvDZ&>P2J&yg3u(%^=L>+-R#T#pd^WrjfhCC+ojA(DHItZtH&g@4VbHB*5#*GiPM0d zAn?pJ$3q9oSPpUzPi>I&KAWYiO?Zl4FiOsCQd}b{B!5SW;u>B-j6z3D%&1<9F562+ zjqKKEGcpQNz$lSQgcLAJq>>;7j1nmaq<~Q(l>#YX6sF)Q1yGjUtYlRv`t`sh=z&#s zKyNS9`}ZozL$)aPkfJAT_N?VN$-p14s}U#)y5-z;(+(noeD)ok8MIE$JyMdyNJavK zYHyM=Yf#W+{aQOn7l@Vj=G5wFJYcoS6A3HP^@6Of_0)#%h~P?MSNM6HQB}hdC$sbU z8#4g0mLh2r-1*wXBuTAOvg+mB4L)_Bq!v_7yO_TdNAPo5);?Lk@*2f{C_-+w6t&%F z-zVi3S55th?s9kfaL@M#t%EjSD@@W-iu2XAP!I|yc|x=HxZsbcDzqml-?P>Wi*Lj2 zsdBv)^n^n7Zm(LE@2L)j6A3!?p>I;vf9`Slsc|D;5wb~n6o8%1{=MKH% z$=SAK{PrHo!?MiOSM4TaC8REkI?n8kL=@WKmF zi5GIs0*nwas$@7mI*_W4%Nn4*cmvDruxvPcc)Uorf4K@@aa z9FGK5L$B!G6;%XHe0q|oDqyN8nPMgdpPnSDD2vJR3c9F>=^95eL)|z0W-rLGr;4yv zeQE!7rYpYxqI8v=N>|zG>5B5JXsF;9&@~3S5ynOwT~D4$*MEdxl7(N&8N{zV3%~Ne zJ$}V_g){Q6Z(W3Tm#4xW^nzuIGUsG-49zqs_M)NY zX+kRxthivl8oG$AIxyf8K*l^c0NkiV(EAaAyHya$s?dJ04p+@;KwHrZ24O>zr#fFv zQAZ^4xpv?OSad!AiWc4Qq%%bkq1;#$clWQQxVtv&VCcKQDdpYARNg%7yTj_WPo@kdd`j^f)gn)$g!;M+dh#8)#qif<_c{rG&w_qWlf;47B25l(4&-Aj(p_yzB*m)=g< z`pA#IoAsNYNh{C!*OcrV?KeCvz4xc5j@{nl``vfZ#}~J>Jokp4t#t=}&{t9=r=(7C zOKMTN^tPv4FmmU`SKKMBOzS&t@|=-x{z_W@N@4rT^Q9#t05St02TAJmY=F*@D)aKF zA(eR&{#MSU3$XO`0sMf00aF^MHXO5KD!{3ZW=cKAshKjp_e)OIW>ab^rv`=z z@?K62=hU5?s@hAbTLr#PD0LI3E;~%AVor_vlv0J93Ob208B!Mi89f#47N??(azh)d z3iUqcp%k_5Db$lvy<-)9aH{F9PfbzOYDpdIgi=Wz;YR;3>ezI-wO&#~idrYBTjbV{ zB|AN*ZIRT%3@V#2NYZXem(|sZ+J-*ITR)cV@LVmYeW0ilvLUUgcji!e6-ESY-6N|Z zS*=ynpwGU;m)4-Dg?R)uRsy_55~3PA4sz(r4`fY*^kn`5R4gNnQ5S>Qx^n@&)2B^@ zzmwgH_8=1O)AG_K?ZI>;SWV#?osd8eSx7~#-1>=PuTs)#xutd7(oZbqpUT+MCOHkB zKaRp?Fzg^KkHOXxY#qQ#t$Qu+#c*V!LpXJ8j`?04zjqS>*Yk_so1@@)ifTy656B%a zt5hMq;Jsyh6bLwlb&jK6U?#0DmX^#g-l77(yQRyQjB+C97E z(5F!Fk)ryRDQdxD+4ah%L5kh*sqeeq@&%Q&W}mvr=X!mkTMiv`%l7w)H`^2yy87NA zAgc#t`vDmq4k@hyG&xYG$xb|`8mb>8xn5Z}h}dID@33x~839K5)G7=h;Cj6&2P(f3m8Sezl7Ih?3T!WbMj#y)}NZ(r<5p=M!o-2vVz}?oeE>>>4D6_5hum zwARKzrm|*2hneAgy-SoTxsY!#P!FlO~-d%J-A1f0$EBJ*7%H_5F7!bswi} z4$_uC$CT>0i!(26BaquUrLUyaEu8w@OOzVNsizG}UB{_EG*V>>r{3R9so|VTSxk_5 zBh2Z>GT(##D!`Ll$Kr=)Pf1IFjwV(83paGQPm`R`;~;PZjv-l#t0ej$j+{l!sf}}k z_C%4i*U&=?Jp1g8K6NYfkT30X$nn|#rJ#1!rlkAQP<%ZRsA!o~xe9CXl8OR?lD33Abf&czqMeWsbM8OjA2B^VuV5|p(YMak~l4jNxU<+ce zMf{*g*HX>3WXEg)>6kC8>%g1=0kb#Dpk>v0dBmJ8P)lpQqAnn3x#|%L5K^!Z$WTBu zVaiumqlZSaXzHUDjnsNa(CnNY*jZ3DO13tBriV{#;@Ad?kUubdVC-;L(OsHaZa+Co z!2u0af=RU&4}4Me9RCVPN@IUWroWPjQ= zM7%?Adk5gb$2R&^;5FzaWDCn6u_}Tx2s7&HWh`9gFtEs5EoGsO0RB_on><7|cO_sf zc!D25vQ7+x(ZZ}z>_NdJoJrZ&H&|TTS+c;aViu^xc%;3B+R%&#>d-?v`7)5z8ieNo zM$!tF*?uvY(2+V1C7`SI4iK~VakN8j>m8sE4nun>LJy`Fv;1%Z5;t0e z2IXinVI;XZf{cb4rP* zF~%eqbfB!;aDGcWEQAPb^0^>!lP%h>N+?;>QzTlfXhA#gL%*}6kGA0?is_g>+Gpw` zAx~^QAQ>N_Kxw`p>76>%9J_=VAziEaMxShl`Q`&!!hIMMaGZQdp9rs}_6E?u-0!e3 z*oOVS8I;HrA-Z7@CGr)G5(mVXyeZ&m0+tPspk-z8GEaoXDXm@7W-J5#0C!+6^`YNQ zoP6xCoI1up^$Y|mbC*xu8C`1?czB+|;FGAEO3Ou$R-@}%eDvCzy{@IrgnJWK0%l1} zauz{5BP1B_3&{}^vsH3XnAm|Tx~r`-dD=vSl+@cn2`499STQv~D2_{%Bb>bLfOt2l zYf$tuFTKcuBl%%dZZ!>9tZXeUs9S)6VECZVW)=EP^DR`;){`uPIRT>kFVMP+cz7yW zG0HN@W0f$*%$cBIF)_NT<>SOxJ5swHEL>wy{zQ0W|?{%OPIt*9v^_p7z<#(RG?Fq|uaC!sKygH_1R zelUM8K#Pe@x*9O)q)h0B8aZ==q;7_lk&;?3r#8u%U_1|VlChNtOD2-%%o<#HsKeak zHaW9ep;Wr$Uy~Q-Z_~8DV_OIbV0{B=Xm4w`xPe5l@0>s;Nj_3mgOmZ8Kx|eOz_}sq z*@c~z1?#UGx!;Feib-u$I&=w`=IM~_Q}+^_k_CGFh|=$4In+3W(~yvGow^p(H;qCm z)G?%!qH2f*YzK{lESuNTpli?3nRRV8JiE!^5we_t-v#$8>xIIM>oX$*5C;{yizOzrIcK z2koIfiPDyak*R%BFgAPXhtYT>V@(f*K-7^}x^Zirh88F`NGmddOt$UtS|zEq8%yKRxqCXYVZ3n`pqE4FTPvlS%mu;>zs)#=B|X04Ipy)Rni#TwQ70}SIr z3_2oF?2ySAF4QQ*Th@BXKnW`xN{KfIa=6-vfgzH8>M_&ru^CL|ZoEDdwD{j#1%oWK zKM~|u(w^IjC`pd@m-k6Tp(at@A?Z`4>>&9GwZf0tI@bhFjskxu&038_bP2!s)XF~u zoisUqxf!?raV(YhKhl0gn+d;J8r}a$;hZWb?SI%gHIlrJIgbC3QmL{mPB3W@9 z;?$Ifs4UE>m!IbEWSn}gmNqc{#i^y7+Rmw*YD(2`>fB9~TF;o6ck~nO`}>sywR}mmeShK{BZb=cdrs|lQ7yma)MY&=HIGv(ds6BtPBrbJ zl**~urTm?MQ*YJ~WivRnjZ@P&HEuJd?&s7`Hd5+tPQ7>*)p7@?IzFSa37o3`fXZ&T z($YS>KUCL~?+;D+i+O+OkC)T^p+8-I%Kag{2PF-rZHAV-l!Q{(`0%l39JZ*ujt@UE z(qW5(f4dX>4fEsRf72OW?F7GeK^*+@84la&;wfw!jgedE;#eNouQNfM0=2GRrJ?&q z81nj3lzabvJL&c(x6ik6z}Y4X{%&sc7AfUcm3}ViL-7KN6sUHUZzAU|bU6}c1@iIo zE>1rbqXG43IqK0{j+tr;70Q%@LZMLVX9r0Gu?HfdbKpI`pU3w#)j&F@*okK#gHr~l z(9$saAGE5ho^+#_>kt)!+9scuM)PM~;>(W1mlKCCFAg6??s@$)E54WFQgh={v+4S> z@T?i5g{~w{veF_J2qp>R;z*bnM*=P_gdcz?o^$cSxBq~>^gz^uE#`aQb+fv}*Qosm ztT?B{;hb*82@5h4VL^Ee&p!@W@JzAb$%x{ahRtZH&f0X&3Y9s>bO1+PsM!!hx$0bK{jp(_f|u|BMxWvIU-;0hcsJCvbqBWQCm) zg%xhHj1h6bxpApW8)=#!7k$a-7gydbE_HH0L6^s%8~2-o5N(csQTc*$(~9EwVVJ4+ ztx3Rp`NYem`_TtPOUWp(KBluDf|3UB~j|g5c%$YGFb#={RyGfANpzVjzLHM0`bnpb}(hi<4qY!?=fT} zdP2V?ITmjaU4Iz!AHXmbjT<=r1TB{$Y+%#Y|Mjr$7@jV_D?U!y7419txmLj?@M;Cj zpUjYwxn4@GkaeY7M9KtXZxuApj$gkdyvUceMG7?xl2Q*#DlWNRNv)GIcVdA8tEJ4`t%k=ZwPM^lGC>om)P(Px*xNDo5ba8yiKxrt=VefBD_P zwztosH51I1+mA@EK~GjV?tpCj_*1_3u-oy<5jkRQl_!2BvBs*LQ!hH6*JEEDUaFjv zV~@QWJx?;$35z)}JM@RJL!`=~V+h*6uT9E2<`3N*3MT^1l?nC%1$R~sO;TUmc&Kuy zv*W|%^DBpTe{js@ZtMNZ|3rP`;_93DrS(+~bvNy)T`ktHo$J#dxIKl^k$}LBENDwQ#6?=x`!!(^-KWN&!;6R+cT46MRPh2U2aa_U^!PwCZzrm$*>y!Yxx$Z9LDB9G>R; z6#BY=)ehc6$KaXhti}hcrHNp}wULX41X@xBnrP`e6hB(E@goKxw2}TCnl4=I2AnvP z^f>OY$$I|4JdO^@)|v4_t0U@AzEx3I7`XI;7A!<<_fRXo^6tQyHsN)?#izeM5QWy5 zK<|MT7H;1E*3O_o0)kdti8oEO#O6s&h060!kN90KQ?739&GxVcQ%a+u6 zY=9Oy=`@R0&iQ+w6X zlB^XwF#xZ|ZB<7XL5>@@7$J-8@6qERE1kC&x$WxciD=i+<1i?B7HPl$U{KYw^Tjh% z^6WzKj4%{)bk>?Go9L?Xt7HA6&zOmQ)Qn}}1`C&(e} z5VVf5HwvLQMt0d7g}|NSV3%D1-CzuJA!~}EdO&4m3fk~ijsV6ub^vDL1=RgD8DVaN z#Lo|SrpfZjor8IRNTR8(O#@cUM zkUWBa%`as~Ogl5WrlqaK-_T36guO5J((SRw8)A>6xGMfc*KWm+s~@8aPCCqx!!p9?eG`sG=ZLXbfmjy@f+fA6`g*haN+TCv)mG4+$_ktzeSt{H zK+`$>KBFJKO>a<6MDnkaj6}@k=s5YR5x=haT@cqbSeJEOQ{ef`SaAmh9n^bVd88Nm zA}S5y&(+Vv&+^qc{eJiZ4A*H{Uuh{?UJ?w$2YZmE8=a?Qb#1;jqQ9gSy6Ma1I?w`& zy4`r~8V5W16zA6pz}Ra=hvW1m7c8wA?5D)PA3cr0x+p_xom4N|gOb12hQ3GaDflFV z-PbzFknN#pKCD-$pc)FAtS^jk$KGfYZvc2Y0r)F3WD|dNX}K>@TXKt!&g@977e|3z zRD-|W4nQT>nB|k4vD%yhA^{=#BVGV%o#d7Th4w;0MYW;^E%yB>?f*qr?I$v0O;ShH zpUMEW&^Wd6>WHRJ4Oa`>EDe4E8?1D0Ft$`tUD%ZDXiI~!7Q#w)gtO#aE^^0ecAL$E z-^P|x<1==Q##2A)Msz8oSX;<19X~kDLJVFWbnFErsyZvAg?u_TYxpuc1%IgupuaZhy_qLx*Bqo^=8m? z0osN&?!YfC=#v+SbCn(LAczcvpo&_JWdIQNMTCEAGaFF=Ta4dCu_O(YDNN*t=yUAP z9SjegrM;Ba7=Jx2;fpRi&ih{^pRgFq3N|KTe~#3WDn|BBS~XHls`Tl3b}6#9v{Aya zGj?9Tbp?Ok7r>)B+JNdvw#7oqaYTbpjql%zJ1z0{gx=#xf`6PI4;k6N9K#k2P z_8VZ9eyWZbq;;RIjmXXoO}$#1mY!7whDKI}IxXGb;G*^doWCtnk`bsay$qM4oW>}u zWVMWSlK0ordN9Alnx6HTf4{xaY|m5ck-pz1#OQMg`ffReKI{1KQoP6yCfPT(iVKnc!44`+LVmBS{}kfJTm)dgHNw4n9ncTbZ1 z$A-A(C^VQpDd{cgk0cHfwJrJX@rX8hve&*fU%h*BRw(uaH)kL^eY#WCm!SqbGMovyM5$Qb5&Ys`;3 zCVAB{DIMV&8d^OuoG+PCBinweDL-zL{0(igs-(1`Yrw1u#SS_6!Mx4snoS?%Amv#r zX@h%8>fo~^e+aD7&;y6Uvk{@zQyu9Xrj7aW3B++Cd@p+1vaa{vGgvmv1F25q*Appv zBxn~`*=Ja&Pj{jfT*(t|<;;2;|ACRln=G?JSPA!&q+NyHAjcZDccB;ZL`A=C9LNCl zYZ0%m_i12r4;y1N^f`+>PoK5g$RtI zUH-NUTu-4F-bL<`spx0zQLh^G`s-c!SwXM5x+K-#=5__nBdaO6$nGku#UX``59;#E z^LrjwZkv$qrJJM4HEq7!)eoLCN=sTLZ!w?_yEjyyly49E+cF;WD!FS*E=R9!keYV? zB7127G-KmC{x;WR`30U`jjsuMj0}HUQ;Em>4qD5NI=b-PMOpO13Mulv=%f(x{_$4mhunvlE!r}#7yp)R{Od?DycPi3=9+ym=hRG7<1iFi=4U{5XIO3Y{GAT^b}ZhlFu#CCTfve>Q>~0zv;LO z-OWi$7=1C68E%uN=0i3{a3h`U zA*@=d8}|k^n0W%_T+bAmj&OTt-gJJ%dS>z?)+4@y8G|;{m*pZ}GSZ$feSJkzKdg&= zedREexbW5%7<_JvwSYgP2e8@3o14TpIBdT0$4%6P zbXwHo>YJjg&P5pEp_x{}d=hLNzMo$w&W7PfMq}v)~|$KBBdUey3w&@_{rfP+7tAzz`nG3UGqRZ>EORBupc*u{41~{ZIEA3R$ERz zUg9jPbp-}eBujCKnvZY=sQIc+^x$!|1xROp`8*aEqRjYwz4crF^BW_P=vQ$(#HQ%? z&ZR(emBvFG>AU8{ll+3dp!y^`KfZONsGnFedZW2VM0rFnb{HZ{*&f~KfXVgPHfySJ zTNI}nG5)p9dL%5e0m&K!xWPG~8`qlPN+pMWyLn`03l$)y&uO_+fBeJ1G zL@(jsl0pqbpi>rdhauNw$ic&(G9Ccce+lxtO~Dl+*VVdBYOOxvdQ^#A^M*`|GUPqQ*O<=p~07Syw+;UD67!)JaR5yJ= zu7sM8xn|!=I%CPQ7~z0z!Lr!chosJ}yk{5uT?;j50>U>0rjOxo82(b#faOzH+5iB# z)eU(c64tbz0wK7FTM?w{?n^~_o1%naJGc!tIyaEsTK)*FFFA-WCd8W(bgx$z5rLZ?qAd6tnLz%C7EoG{OYz2J`SE`o2GA?J7|SF{-!Vhs9V*7U(_?xUG} z#bnlzvNjeb^RZ6&o9^qDCU-jrFPrf+F6M8TU|6_lV2EiDG|qG$9;8Oqb;jqFs0s@f zo2St#X(v8Fb5mVqbYO{Tl{TSi$DeB1Bqn>~ua$H}x5}Iu+KCPwRiDxDzI8?@t2943 zU}|`c^~_364bK+l*;_%5QB)bV5BRgB!Xrk}m+(2=Rl-;>F0Nz~f~Kri;~YxP3z{$P zGqzSo%-{#mWxP>N`^O>@w+O4D>a67TuFL?kcZ_!`sL3=dV|!q>e}liLY%+dV!Dax} zeF!QgGeD)>d2D_i#0HP3wzB5!HBoCy>P7^Z^|LZpw0V~49yB}(gPgia>9%Bk%qV~q zeWB^bH<-H7Cr|j{N&jdfZkyMVA<${=UmTtL2RU%T_y*7Y4YZfeRWp~#uo4YlFh9-& zAkG|KZ9Vq=?E8_p)8~5?{f@b!W2e1OlP~wreza!kX`gXRRAwk1>H9(;_%k`f>r2Q4 zNZK4b3V+CG*D7>=H8u=a)|Z8J(toq@W<(ob$V~T~3|%VNIv!(Y!C` zu8y1cc@7P_W@mhLlD}-i_oB@bH{*NRY6-Aeq))WW{8DZe!n(d__J{JqQ>TB4oY{w5 zv**R8e}UsGSOIBYWCgg=+z2B_jR03#UKcjNDTAJ@%y8cmX8cNF2c(*IfX0>}&HIut z2jqBjKr+n%`BZa2B6Hw=GzTO!2kwf9IM|)+0dvOB>DmyO-jyLR{WOMvlpcQaWT*BZ z#ZA$Mn0AeX*=?3=(GUq6Vzb&iZq5e-AmEG|I_cqEQ9Qg5Q1}z&Hdy_H9|O4P{SVMn zI4jVurmSuiSA^(Z30(o1bpTxm$?67U#8Aj|!mrU7Nu^5-+X=^lrG(=e!g2l~$6%=KGx&EsUs^fnplUw(XD_J6 zfP|yMaTtEGp{#++)?bI0C>)Sulg4yBl0s;gJf8shH$h@1x#TN`CqCEb z_*u$e%Xv}2_vfzgJ!j&ZlVUo9n-#_Pa0)xzgW8^Cd?kK)HV)!2DjPLEP@C`Ka zy#S?`5|*c<__P?l7a89VUqyV=q7e5-F(VBekkK(c#k_74$KNjTk9o#||FSOdr^n%6 z^2)!7UrWI+eoIGFq>$;_*ra@I_#+s)B$(*+ghHzl(T6AcsosAsI{O@Y#dA-e;^?!_ z(_K3H1W5AG_?#N&{PnSPES{vN!Mg}HtD*j3&*J63cq5!8Z{B&fO?(wS(k zIW5Dt`xzvPZi^`PIRl4pI;)j4Vmi&}G@TASQ~1Tx)^nC=>x_}FYzx1-iWl~6@hiu| zufb=CeC2hLudd?JeOvsJEc{Z=Ao(h?@T;qMhu@aDHqOGYo6jI~ZK8!=x1TBUWzqMO zaoYgQwMWh%8Jl9^S;?7at_`35|B<~W>3whb{HiYY_Z}7ddq0NK^J&k{15dnrcd2-i zRL5X=1aQa2n13E5W)ehScoO@W7s6xQe&)}+!uQDxfo}wqUO;ttqxi0h;Zqo2UlsQg zFNs2AMlmA|Vn1;ZVeVv~7J^CYDn_TQpN5R2d{5Oy9MUcns!#G%QHU6~C**o-Ee1Mv z=X=&ts5~gqdd4E?Fh~s8yEXeg>t;1j9KITRt!I^r7ru^M z)7QpUaJ;yJP5l3(6>N^AvQ8uEzgF- zvIjj)UbQ;3--)4($&jlKH8_D{4;g*c>Nd9dD!}Gbn=uNq+q25`RxN_L)sUw-6i)Kg z%-Vw{q-R}d4_J`uIb>#k1NP>q>0i;{e56Z*bM}>Ipuzd?eoZik=l6^>ir$1SOj@w;hTb<_Qk zNQGl0W>z}9yQ6!!qqz7!mE%koa&qB3@;fg>>PzvL{lbFZeLDQ3z8L?Z@;Lk%UjzS@ z_kN}JPx(UpKbRBW{^!0r{x|RWO6~t*{C|5qzWrYVf63insr~nTasLP6+yCrW$N%|V zU#b0HjQ>zseEYu;fBg4_l9$;Vu;WfB{R^Euz7^5XH-npyCyePHt~&6KNTky7Q%E=o zwjf3TY(SI0%XPaueb4xHSNM`leAA%xk5tF~QG9p9lX?D}@%1HqH%0Lk$MF5EGrm7} zh3~oF3w(2+^ku4JRutdE;t9{6Grm{+mBE$Bi!mqb5Gm;g_ice z9&Nup+Ww6A_Rm9l?u>Hi)+i^i50V&m^6pc=ADs?=A1nU9bix04{73KpAM*c|-`QgP zf8tB|-}gF`|L!RN88g!0utoXb)%pLf@DH@&f6Z>!knZe$)ETivP>!pQDrhr~2gt$yoWJ$Pe^J9g+<*>-qDRGceY0Y!MCy9rM+)bt74(*g#RqqX`g|Z31II67(n>f*o}<<;miHJv>si7O8V7* zgd#=1EpK@bcxNSIHGc9@yTu=cYiVy$%Y#&l$!=0Ghpwt85vuO)v$$JE@z2HO4P@AWK|jzvkvYvlN@( z4is&AsF$r!AAF@~{b7KcYi;+MhobG?x&zyNBHHf1LQ$;Evi&1t*uRi>mi9jWP_(@- z?7;ROv9|ZkI&LpI0bT~|z?+QmeuLTXM_2J4*nydsFWMe1X5-`3~TF;K(lq1HPXeEqoR5J$7BsYXRSrlO4YWe6Q{BUjX>}ZLGfu z@MXl~)325)udxNQwy_PqKsD9hKr*B5?_xscV^m-%9B+0q`-V5zX$ z-=0+J^{~i`fChWty9OSN>^kIYTco5`LO-l!^)!C3KI@mgY{nYkx$)SRsJ#2zySd5? zdbqfBXF=ezl51S$=mF9bq%O`!{}3OvN1(dWyZu*P>WA68+Ra_Nwmpgv7|EeTc z;3-D5d^$)#izjHDvy)ydX$S

ig5Dmj1;PF;;B_L61P%n7pk$0vI2A4&Hed-Z3sg z;u>mU9L@Dt*;ghoesvX8wP6d>5CGVmp9Ac1kS${=!V+I99S!eag^q3`7NW>c=hgkO z>J11!2f!uGQT|WVMSYv5vERbU@T27b4g=_EOyl~r-oRg1U~fkRMx`eh&*0751cP^k ze?wkPuATvv7?TmlfZBHUk1PCDyXK(Z;9@B&6z5qGJ+OW*s~w(Q(CaypHUhfTw~WHA z>KJv`j87%&%K0|0c71=JKDfVY>HT)s(qAUzyOySf_PIlQov9s=PRvi;23gl8h4!UW zMzUHO1Xza&qqagSB|mk0Xm5IItE81KR%fh{v>A(xqZ1<$tvIOGc+1ZTg`M8=Lc3ZM z+Mn(%ADgHZuL*_Skd=VNW0SbtLFH61HU-kjvX+c3wXc?P$1ijFf685uL%z39_0P)A zyC=<6baeFZ|0L**JkS=W+^$34{kYdey6qA4lNMO(DjNe2{vpsMmthVDe>j3+C@KOI zk3}Lj44QTfZ`@-vh$T(*NKq&x&c!GcMf{j9X(K113!Y=-IHmd+I;eghGTc<2AgjI{ zNq+*b&HMB_Ci?UxL3&(~VMjGfFJMdsa`6|a;}q6k&yN`8!k|h^Il=`8Qh+3ewQ#qy zcN*quB=(G=hi_l{q{Ejm|e?x>wA)hMAe2$2YmV)^(cKc1k&QeC@rrH zT!{XRV35A^h6sSN&sYqPKK=2GBoU0HQc(j=pvZWEV48gT(wTM}d*WXu8jjfJFD3`@ zoIEJ&PvueF?LPG+2S-iEN*-8@F;m{dO+Io&wAYwO?-2?pn3=>c>F?FkQ-zVY7#+vc z+Z6h2lhKn1c>^&)t0pFjcqoUAqpxFCOPGWX?|bdYIelfGgW|IoEq_DYD}#w5T+2b@ z6V6#k0DFxk@V;wlU;@355emF3JD>{umYfpWpX_h2+gBNL{{-~tfhi;aD>9CeXBebT z@&_{{b%m32Z159CGZp=6c>RYQUjIRdcUA;xa+Uo&0hLp8zIxdE?hwc^T`R1>iu9o{ zPM_F= z_L}|n>U{k&Z@#*}X20>#ACZGgY$RbspRxWa2RhsbP;&WMiJp50=+v0j?Pv!KSTdK| z(Jl>`w+Np+hdFybW$RpuFAcJmF67erW@#V?p9bjJl(Ad#R7={tbn}gm(4!;2fiD`x ze~pGH`0i`b#|vVQePWNPvB!?TIDo^EB-fnh&|%HedB`z^c`2FV5V6<#8-{=`%K89! zm#@~h?(nJ`GOK`p-qsJinICvl*Fdh<|B1cE@Y>f5*9YDT3AqZansXOeE1eB&bc3oH(~KOF0SF?S}t}%@jo^igQ4sc`xNH2 z6JR#xj4nA_8=*+*@kI$GXD_msUMN)#BOm4>8XEa^c`U~mo1ALXhAE5eB_G`ciVGuF zaGvo@)LVuir zU9Pg}iR7o(sc09j#CUoY0rb*h6TwkXnJen6A`E7e2yQh@?=x^#LkAh!fi}ogtFyL& zPHC-ELzxvIMXfcNHK~VH9Lz9uYxF*nT3utf%2p(huXkYMPhLU(Eg&y5q55RWUIp4n z(qO*QInauPj~o$GLTg|q{K917D~DfDOTM}AI|zQ~!*8~v<>gRRAFvd_r1{s8J~lEQ zG{;VUTn3NZ`7sEOAMj(9D=-;dsEF7!`uAS>XD^#~WiqdPKJdQB5p38!D``V=>8hwk zfqGO{D;lpb*Gc~OQ{-6{xQz48I+18cl}KNM{FTYx(4oYkp#QE$CKjmsKuzVV;R5~6 zl#qc+(jT;=R>V@$ip#MVwivFZLA3((;=z>GkJQ%8X8&G$>-Nkxi$c`<3`ee(%tnp^ z4}pQQc4ad3wVUD@lqBQqpDvz(hf6ZVGw^ZAAn^>1A{TD>4E!WNZup!nYx&7JW)Lfm zyIOhQHT)Uk~SKwLJ=+%x?UZNp^_F$Y6^O#B?1e5l#-nR0syePr=-HVLlgOs3tduWGWm7*$X)v<2?aqUJ&7>a73M|dRSz|!XHJp zvS2YSj6j=J4&(5GmBWhgmmFaaBL~^8GWO~Ba0asIJ1Vek4lAQoI;j)?jigSV8xPt> zMYCrATCkqex3-IT5OLP>2DAyCsbmU52i3KZN3!}B8sHgikw{6Z4?UU>?uN3xVsRT0 z+?|$hD-aAw4A|Nzd?paSA_qM$RR*#l=K`kNH7{^UD|IWC>0}gD22MbN_hSN0>E0fhG z;OBWQeQaCWV4#HR6H`~IZT@{!!LeusUcE2_itRxdQ0uUk_C#sEbXzU1{LfamgN$Me zw?zE=>{4o_(fBf~45EWH2T(I2jblY))2-WF9B$MY4#KO(Hpw zlQ=2s8jhWO-H(>GJ~dCRB2#Js@&_6r&4(5w5(}UKP^1qI*l2AS6s3IuG(R61;oZ{ih1PjP4Gz69 zvB0zTi@>`B_%SaUP@z3ZdSOC=C-_C+NdVp^0uUtXg?5BSrC)OXDbU6sq6H@NI~kl0x*Y3HfyfZb;0)SJ-I zbGEeOAV^&WW!Pl}o)&dmf#&XaJQfSSP47?lm2pXbPq0&+*y;!ME zDm|8pP6o5xZclnNEvGeY+R8cUg;QFMMQrROKoU?f2?_zK1ibAbC;@8{0%X2**?TSt zc-sEwKmYf9zmH+|d%vyqzH7Z}y=z@%+sUQYVCb{yCyXWWK*FCADw|}mzm;tB?-YaG z%Jf~z^ft6ym+w`kx1jxfOln87{nH~N6ON31BL53Z@Z6#rA>+5*O6Fhr%cEp|$X~fi z<^X@?DVeAEt588}f%sLdWPZS3vz5$2{wfvrmryd19ihHLXQ0vio0g&BLW~H}P$Us* zffK2ohLzbVLT4w-;S4&L<Ojhp~IAX7ut_YuZ?#*O_$u)pZgkSzLFxm-kQy{}8L* z9`3OU0jp@@mOt17UwOm~)JF*Yx{322qTC&J6N86V4s*;-$Pw$Dx|9UW@Az9<4qfAdo$?A$z9Rv5K2XM z{W_{=b15~tMGT=G<`aNws@BV$pixU5$f;q$$%nj$A>+zHo@l2*U)lB8LDBNg4Web_ z1~I)Kj&*3!TD)b6>BXC!_OOVr=00WmA!YhuG5wU1*$Qnc(+^_l%-v8Lr$x!!A?R8O zJy@e1Nd~-PTE4p3%wm;7b-HnmMp9qO4E_=tGygclm_wCJs2VRuroSOF`;pk78zi}U zDKvnlL>a1SACNB#BakI_c(B}c0jx*{Rb2PE2WMQfjE>a#Yb{3!dNsrEjR*gr$G^Gkf+?Q3S4+!57y9?4YIc)Hgj z^o_EWYzh@o!aPxhgLr%c-Pe(oR5YZD&}Vj4C0R+;fa&}tl6mHKi0-Sj$Ez4>K`%mq z%15OF{C{acJELj?H(^>t)!wC#hMNHGsmaV=Fa#)}U6K;oxl#$W!Qc7G!CAh6T~gH+ zx<-9hbR8GQtedGm*nHB3ku_5_)S0EKQ9d&dor*k&9ode4eGxF2dtlyyIfPge&!7<( zf#NM@D-xVTwB>^od-N&%%ZC-vhf7di89D(FI}A)z`QQ+*^ZE$i{8x2|cI3Vt-w6ui z0k>)tyv7r=!p*|Bn`1W`-@9fA*hcfykHHLuM$!~5BRC)Ux=6tI{)lRn<}NnAlGo4v z2zCHMEgGUL+IBMZ?eBzHa+{|gB|m+g=r3nTLBq+RlTy_yBDB84QI#7A56IyMh+N&7 zmkb$tpaXFUdRx@pC)El)r(I;jp9X&ke|z0fzN#!KvtG3g3plL&ZKLy#Y9 z#2jPSzXFL27S^izSKV^8jsw)o3rYbpgpt!J(DmLU6saQW^q_Eh2|70dkTpLCM=LIt z@afeB=w&J#39j{RBUCGjs3kk(hkojyyD&L?HS(85JACb&6aTYBS7@F;Anh_|UOx`j z_Ur@l!zdRP;lov*Bx|zCLn!C&31J8+icTn3*J6?Tz8*U3)(*;{hoR)a4woD*gu27K zgW14C`W*u;X}+5Bf+ceJ^Q0F1Bz2+h=aw96?l^WAWBQ&Sj`<``9o2LKe-!NqWDmXh z018W5j%Pscm7*hN)pg?>0qK0SJnr8QVZ!*n8?&J+L71l{T92YHO^5HL#`a0dZ+HX>#@N*>zueA-e7>_l!gv zMv~e^NwL+g(9#y^<%QV3XGjzcD@K+_6zvyQqhBKzeNZoU&!?A9JO^Ks&P5Mc@mg9L zl*UNUx#(Q##UINjQlpbbqS%Z7W8vw|S^r;KR2bc<~^yPJVoUqCtMs@vS?9JB;hXAsw0A1{htR1DW{MT;mcy>pco^9H4)N0k@sfThe(<@q~k z$d5|sQ+P-cIt=F~DR`qY!PSNCX(5yrEVV*T2swk-S|JyNT(I$S2u|pm(u0>-p$rI( zt8!xqzA^whgI@%IRE>%erR%s7YL(Dk?qXxwCBb=$)UjGV8Chh^ZSfA+ek4O}(rURc z1HCdSC8zH@3G`ZrDjixa-+R!F=KNO+`TM})_Nyg_gDz?YWeP9K;U8mZBj;tRCEQ(A z1%cNIl=Lcx?@y8yecA0ObFy`IlKk}zn#-OYjwPqkxok|#z2y>z-}{zt+gzw)`(2K> zXIb+KTghU0C_R1lL*gfD9p6c`M|E5UjivazO}I-U_8d2v*(-VJ)!o7=o`10V}^k zTwvutbxF{N%dSct0b|wWaR4f%HebBn}_<^l+^25esg5D=xU+_dtDp|EX%P_+g9cjJ#dEH6h#HP4%&}GBvN~*qiQC9 z9m)GJDK6*~rD|I`=x1{ebp)@ExI|MYRd+x^ZNHu5_xFT5U<69Bd%mU6gzr|xsB0iX zxd}180!YJ^ICsbEm^cfRd|0(x)u|>`=N63S=g-n52sTT8TYdUvp<`Dj@FO&wCqFy| z=<$J8Nv{4+C&yI(f?_;!^d}oFf%mbk;6}9;LRbGzA=N6 zPb;f}1-UgYS?fTruEV0F?K`=O?rIaL6Ckzpx};WYV~cc?{z`p2PQEqpg%Gjiy6 zWRQ4c<=MD+Lp7QrxVV2N+1oZ2Rh&gJC=86bv{J##2cLP?N<`Bn@n!z-Q$o5 zkZazBVRj9PO#tRVC>#TPTzYAqACbG6godtCAMgsW38ec%xP~g~UX-Vo3f+qrYiE(K z{x*u>*6^oC=n6{S)}Ez1m~|989-Ugz*FqF{*6=4}y%}9;)>78GO%#iMC>!XDyi+#v zCuA)}x0G8cYk1pPyr21I8r?0@KzXf!W025o7=CKxoL)O))_oOkiX+hdE5AR(C%-T* z9MM|k@Bki>Ff<&NT#f#3VAR3iVHKG5f!8c4jnL~6L6U?u%(8CAR$z&{P&9KiBS~O! z0hx{b>@26&Z}rj^*+3Ulq&CLq7Vgh+*cDw&cEd6aRXa*o;hnfbxH`gkBBK=6^k)3X zLufLpGw}T%P(3$Wv$O;~kPJLR0w}~A7*mADRQit(1a7&P%9@E}5ve z7GT)mjG$bqO29Do-*hv;NpcIn_yM;-uo2#L-tz0G)24~14bvH;{Hq``T4tO2J#634ckxyE+F{gG*aLmVFX80wWH+zjb8E>kUm{cI4GK9q)E-`z zm>3bC&-!IlzFtuI`eM#H+fLV-dTA895!Twng^4P1hwXcCyhN9FtCRQnz0gC$qymS< z^Y%@*>ZEFMcD<>c3Vmnq&JEU&F}{a`Hk>zH^c7G+g=BaAH6C2!6aO zdkh0y3iL&Jfyal0%m6Ux$e)wLci_)+fIs&EYwiKU`U;$6Z(4vd2r;lLzM(s z@R4hOEMXoA7~P`tJ|L?|y7k^TU^4^Z_kez=x|LL>W{{(-?-k_wwB_hzd#9Tn_9X@x z{l>3|ZpEJGAe;XQ+dnB)2(?vZ`L*vAETd}r{zR9v14_%=4m=`xBSqP zqz~0QjC2f7m&wb+te1RWnf?9j_{eK>-DslbbzRY2WUctm(W=;a^QR$VJl@h&uU zUVeDKM0XJH@gBaxPkZHp&%j>!pQy}r#Z#6Z7qBa=PIt=h-5n_E?i+L8zz6gnoRjF*liadu2L7PC%Gq>l^385^SGjtgeD|~rygFE2Cf{A24m{A|gYeh7 zTJGC1PcCw0pgcq^>AvqZQF1DHnLvul$dN^U`NUc^f4qub3)94^id>*|oPfT;d8l;m zd5RudJwfUm5AMY2wH}}(*3%oY?m<`eqhy38RSv&FK8dMB;=Kgl`|R#}@M8OiqLE)? z)J+X&F9oz+3b5`Eo>-Q~N_iT&t3_|%DzWsIZ&05{E+TD2lti;z&`u5^z?%~U@x{fg z>u8=(Y%jXNoTFz*^Z0_8?WPzq(36RFbC+VdfF0nRo$NXmT`nulM5e?>V&(%>H#r_A zrAOPtvAGe7i|0@v6FVDo0!>0{lG`XE1`&EwF)Q)bk_XQ{D$u-6gY+Y{4i0os;1%(l zi{{DU5=EzkWT{;mvqy-UBUR0!J&>JsIY}K8i2PD|y))}O08R?b8)xej&N@r|mo+iN zg8dRxS%a%CY8Ot{7dCQkrmK@??g>(`?CQ9V6+{j4LW}GND;S&KUO>};_j((J{Z2vE zn#j3t^?Z^_<5c;*0|D*SjmETHI4+1>M{`k+?JqR4?AQ(5$~KD4v14m^oDqt3QEWgn zRVV2>tJ#K|a0X~$;J@l>Ea$j9*LkGB%0HTc{zjGd76-dthPh-X%;a|$2;^Ks?k3NO zot#Gz&FIC{_nd>CnH+$8g+#F$2(vyubh7J)sEm!SXo6;Kh1jB8UXct$}qi1 z33IROu$Sg3T7tS>_945(T2(1r%vCSLA1OSWLo4ZQxK?q_DuBpR3N3V_^Jx5Zss(O{ z78HH6N73iyDo!Bv6lXb}<4p{uOYcRA7Pizjb#I@{2 z8gWB;K926NHb&Nif)PQPvDmE`vz}IFT(1~88w4YQGUHYc-aj$Ct>sBabW;r124x27 zK-X>H9l`Pwjit!aTlxl=h?>kq^ zch_M~Ac~SpZ(&?)%H`6%T-xN)TcPweDA>&Ij@fZE6^6iO%FA8tBzMP- zBCuJa^I$)4(oXUbkDM}rVLh!rH0!J<|Lxl5#`AWwsP_-3f*x`KJ(f|Zi5`MvH`A6i zqltVmPi@8pQfKhyH?^53oN}r#&KcIm*35g>E0EvKE5my!jn{g*VDAAlns~8!Th*J% zMfB9>A@*OL7mEWfa!y=kbq0G&pSrn+%N570bGO+VyftJR=nQs^KGpJ> z-MonrUPjec^gI3 z3r+|{C`2M{i2M;n`r%}F=qVzDk3kkVSxx~r1!es`=D!ZuXR`V5a>V@m1+qr>)A#eN z#Pp8&O$>&1uDm_vx&FoIwZGzH{X&KAtt;`BZ-=s~G*6VYRb>KYaGfaW2wn%&bnH_o z;rSFwc;-105}w+<87SB;^hU^JBERK|gk z9#iDs$tbvq!e+Hd&w5vtOm8=;ZM#b**aVaYgHd$dVTM@m3|&CMltIw=%sm zQ1T9tn^k`Rgx|K%nAI;z-jJ)mMn}mhQuygWWgjQ$_2smQsiYsYBU#@EsC!s>IGPos zG)An%#!nL~G2erFN`nVP3FI&HK)WzK^5YSAE9)yNp>rAX#@A^1dTcqQTGFe?3w8p6 z1M?faRG_|CtH_@q@-Oo}Es4xMLdynNQ(?lr-AYjlFa#?)0>(`vLW8ct1_cq79jZ~_ zRJx9%Q+whKfLlPZqLBK%yI1?%VW8v-+=-|A219#bcs;Xbvyx}*wzi5Rbp*?hlsjm>yH^}}Jhud5W#CGfE$*?7i_s3h{>3Br#2+cF5* z^3#NVGvL~BV2|fc1&TNmhy+DzRSlOTu^HguCI)-qBwa}n^~?C76gy5If*g3@DT9Dg z{tc?u39w!x2H#O|KY$&(TMWLbNc24!2v5N8!FN=xaucMhS_IM+$cyPioZg^NdLxzZ zpz^SiyH#zUz+C;pXrh)U??e2D%tBkS4frDN?GeNW71dTNJ1EVM{pm5YKW9c;rr*@ znz9PLET`qTr>5d&Oj}B6m6V3DWw+K;+(NxJA5-uiXuaEd%%!yZ9oFw`2G*o2AU2Dk zJGE9{jjI6a^wlgaknCDfgU5+6X?mFpvy>OOtUT$-c``6&oL=VUJZ>vbW^$eh7&Bfk z%i=s)R-TE;d1Q>46u(f5?~|cpuBr;W<#-*=;KsBz$st#F!zUV-Sgn;$9*OQ8NxuIM zcMC&xPSN(u)&GQCNknS}BrNG~b4_;K^Fef~EoawSdG*@MC!fRL=z8M)CZ4V&{H(l& z@Y8V};AbDstm|<^U#J+~} z57vnf5rf4L%EiL?{xEnIRR9BkI;^t6Nsz;hUBowyRjqV;vVkXH57{N@`!dg)3fvY< zj#~^K41{+>&)c9BZ10QR!R3(V8IyJ%(mcUZNXs3a7J>dO&J7m8o=yW&_3?_4C8ZRH zy+f{gWOeNamYo`!g0I_CKq(HAE5mUjc&u-3HwAjZR*sVRyA2aHZTPy zyED`)ML#%i!P^HrKG=btMgQ~#g0KIaN$|C6CcdKND?jt+wZpO63+!LZdH-6RvVTn* zvwxlDePmkl{`LJWWA?9~{CDqP`>l5MvfRY}b!*G<+?tA|fZ@1p!Cpl>7;Ia&VCwCZ zy39)b0{%kMZM=u=`K`T&E&sdrFxaoi07^~))obr&yw@alu?)(zw7?b5l^)NP*vBSV zx!mzwnekkSolLfJWyNz%jOUsJ`#I;zwsLvmxhBVRU6Qinl_%|YTN3--FH-iqCoZtx zU4i>u?EKjMuJ8QC_PYS@ci$#Grq9%0MeykH+WTG3W4zy8flC{W#P>V47f>PepUuXL zX6>8^9}a#7R<|)tuKpR6Y?S2iuL-P>26pIWpK44CeyvfesuJyGB)>um{n9o!i4at! zy}Dqw#QZp9%!Q2C+>R0pT5L2sV95dW?-k58-`HgpYm|ZqZ$}ZsY&1U54$<~Lqu-xv z{t?31uoTpR4ahbB1HyeZ);>4K+(gMa(9`N$nC39U5EVrNY~x32~EYgwO=vrivfS=LIE5;Fm36D&yh*;#t-meBrwT-2fk}s zEuZu^$VL7Jq@sq?D8Yk~TOo35P^O4HjEF!)L@8-!xg$*Q(sW1h()0!-P1KTMIrK@6 z4yH?`#lcH4K20e(AaA_Hj!(1WofMB|nJC_A$Ga&0m>uu3O!?6EJQ*#Z9o|vM}yiip#R& zCSu&jDQ==2Hwoj$LtJ_Kq~danNE=&h7)k+O%G`jQVyrDU)UOjGN&0o>i+`b-zmljK zLi;ZV4E!1E%dM}q`(oblBIPv=mm`t)*H5|}(Y3K;{bgG@eV|((?q=#Ti(T zlUB9-&uEdv>~3~DU~5yfU2^pgv8q)SK=977B(zoEGEYGURh!XXmPJ-k%REF8kOF@| ztAYqO4KG#i9!Q$CpTOI=ui|ao*By!5xU}a}mGo^s3b+i zUq6rPy9?{{CDr$b?HP`~2UG70U#0s(WW|7Tr~SWh)c#*?e2Sk3Y6DtjN4aJ?%RKT! zcmp07gennw|?u=KrQN6Y~)N5MQ0SP#TyB= zSX4F&qFT&x*rD)7vf;Eln2qFBxwdGTiZb5)=EuXsv2y1(*6L-!d!4XB0cTOJePgXw z7Suyz?G#7TYTUnE;DCO~Y{fcNi&-2EUiZFl{-yzKf(1H+@%#otgLbOf z8q+%a924!KtyVM~X-d&cinJFGwt)6_%E89(MFIez)i)AFeE+&~?U&S7SUgobmU_B< zuG-R%(C^JLZjqu3?fV3%7rSnsae#>(SeHhf>w9{2;(bYPKsz1Kj#_hB^?A;j%duM7 zvgp=i{7gO{V{kqOicXk~KTCH+6+TyOO?_az0$b61l&V!m%#GLMBtby0jKIooMWaiK z)C!Oi^^b5l<9p6^JUnwvXgCJFkf^WG1?Q9b3(S{mjqlVN^9=PlB=(QhOn7^??MYoe7?8i z4SM{1f8@M%z7MFn>v~mlT`NLuaK5*}`Ti|Ps~tlxcx`$?iqi{8^?mK!#p#6rJ^;|K z)9%Tdr!V!MgRY;8jz6D7LwJ#ffXITR@ubq551wcw<&ioRvFeo`p*6E0w28$*MlXq- z!zd%d3m^mKOmg+*;}P*$#K`(6c)pi<@LxARrYA9>fgyIZ@1WHYou86bX$t9z3=x9#8n#x1KM#*v%wNP2DA@++el*0GDp=(hT(kAvAEo@ z6d5{MBG*G5uWLAOX;`$WT74r+>FGy%Mb&9eSb)wZ=9-qp1?WSh1)=Ix1B%Km5j3kk zSULwNJlClALZ=pCctvfN-hr5M0Ck30v}g$vac*KtQL8d#A9OB{1%V`Pmp(yrmCMs= zb{CK-n->yu#WH?NKugL9JW;1D?E=-gh3d;D6KgOy4~w3$*EDT{=gX27^ERBFP$TGw z8`=x!(k!4xHqC-HraVhgB$klJCsYl(tqI_)geca@@D%9Q>f$zF%(}F$Dm6KlAC(hP8c-LQLC&)gEG-0Od!#a+sqg2j4^W`Wo%l) zeJ1UTdQy$}@l{E>EYM35LO;)yykY)!IxY4#wDAt$fF-u0X$j?d>~}179+7bksg9^l zHj>!hf=O!jW|AstFGJ<5!4WQI)f@(ky4e(}vlM`MGV4pvvxu9J@GaGO*9JAaszx#juvuDR|Cp4UG-w6H_lC z@UG!pHM|zwUFs(QJNEHBK{)|`gutC91g%7!Pe|JIIuYLhsKP13j4oV)O#_Cx+pLbo z0yrqRjGF3EryNTvfrbG))FP9_avHuBAT@z0r%5*|p+tdOi_u(5Ljyhl6$dIenTKHk zEo~gs%-fbw+6%Md!ye%zbrdlpya|7t2G5QS3W6R2Qr_(e<(+=W-ay%D0mY2E+L0tU zvevi*ay9a5GlatCN2qQVr&M;_G)w$8Bys^&Uol2)Js8e}= zok6NBXdIHYJZ5w2rM&d(Set`&JDOM)N&^>Do0m~vP~F16RV65>1Um07SmUZl{FH%# z8O>X1ma8_Y8q9}wXQTO-E2+tb^K9{k76~nv(IyWhVlL|1c2ep$cf9O8{jgPO1mb%!9`Ya7ZrNTuaDI zJ%TJfT*b*lsHVge@X2FxQ9ZS%H#cyuqp126u~O|`c5o;5SsmDCjp8p@EB-<)w!R+k z(<(|K?Ig=axOWy_#no1-kBP2=PVSAerL$O4}>k14s8Ytm=IP6t+R9xH?gu5DKUSk8%mxU zb3d1)50&J(1a&lx&7>`8Y$23=@I(aXEB{*P7ZaPT8#QQad}(bW|DFuXfGvk6*;-xt zJvpM~eYa)BmJ>zXge`^^LrBC!w?JqK7hG-?TxKW!ffc5jR>mV(S*mGGJc5*{^nXRo zDN+{ENCSiP#QEq!)fE63)C`2Ln=<&U!B6ONWv$R}Vl5LmQ8*epTiL*$(9_CB{)BE; zZiURtt<2jf`Ul*>$_RxXWR0#LBs97nWR0#9rAP6SnUw`VH8;>B%Qlm|9zSoZ(YMu zPF0I~Bcs$c=pNn@cajT}|08uZtQ#GgJ#=7(vjKTx5S~c!nPSY1n;~O38 zergbx?e)jT2&nH#`&$Ln@Ls3!DB zJLfZroK4mzWP8?OS@|(H8kP(_emC~Fh|$ZI{|y3by=*1ty4=c@6VG*NJlAD<*;hE% zR4Z3*JlB+Xu8-+uYdF`HR<3FBTvx<%eOxd5Cg;kxa^=NyO^@fg>h{!o-Ow?Y9LD)V zZf?=JxGcT&9LdrLR%^k@BhhQm+2;nz-S5S`UrNZ`Z%L86)3teRl77+eE+B9Ji7jsr zcPoZEj^y(03*_S~std{Gk5Ua|;~z$=3C6^my(F)1N+Wsw%E`98{wv=i^}=6#f|B=g zvM-+Ohh*vyLSOV>@m4CX%;Y?ubl+I6qcx%3@?&VP#AhgQ-2E{537e{QdUxaf;YHrD z;a&ZhJ8|u-2%T6LdLY+jol3A7;Kg3(3@p+J)xN$K&Q(5?x>X?S4WQY9YhCf#bjw=_ z5($exOa36}Tr(~`3R8g;KncWd&vV3mk(7K3pLG!;1g{jvC^H~P(Lf(sjM*u&XSFI?aF@eY@HsA~3{bD$$M&8pNp1hR zyN0QD`H9xPpTXYHxdRqvDX*LV{A661)eQ0L8d_XEmfOSu6-j5R;B7%_;6bS8Ip06peaSpWVa&p z&qLYZUh4|^uDge0eJw1f>BvI4%9lsm#kuY08MHg48U#y{$JAzBP`+~&mFsJ>^>Im_ zgYyRiie7Kar=a|jtP9F>jx|vEjThGk$NScG65Y4rZJmB=sr@QNGw^N?X6&&Q8L*ce zmq^X^(fwq`KON`a+imr0f(P+Nlyo58%mcjX7U530=4!fds;AoG6sXFtSLlvECh{L= zI;xI&Fd(HuOiDe@q*R!>oX}I8i}-FLrNZ<;$*_30Hhj#%w@>C?nG$-5^MDE4!EgMK zAu)e38FL>ZppKbA4^iHDv8zpvUn)H&$7AnJ$?i{Nzb%pd4sMR$QJ}n=o1@FFnUgJE zAZE^%DIEdyibU*HiP#LgElN0;=*0UTyRGNZIFR}Nq2Y zhXmt6_T2oDoEVQzJPPnv#GL5H63X4b$e8O`Y<%iERM|FWWybGHhOFxnQsoZw2H$UU z9jIfzgLKGWRR()isYjh`>7YmAI_Nv-`LihoHzUKNNIS4jovyi*$&j+5)=E;fJ2xo^ z!5vUS3?9X|$67ZaVn8|w!wF12hPvtT%*e9fv#K!jQNXI8hCI4(tL!lUeG<>19L}Oq z@|B~`cNQAow7y}koEV?Vs!6HqRFxH9kJ~0C%eSocs!XJRc9Okb(0~PkRy)s+v)Gw| zMD_`jh(56DnV3j8JuzwB`AVTFzV3(2r?cXn?U)$H$d@O?WB)Lb@1aKGi5ucYp8paK zs&*pXY4!5O4LjZ8SBC#1>yP(y@%7)7vi|&3>u+3t&KzHV76Of0|E%QoSM@E7M@=-X zMt#HvgX8A%2^O*(F~6M^pSkahCvf41zqC3_vuN({+&~Mq>jd9;C++pfjPpJ}S+Jmo z_ITeU=EuW)Z{W+ZURx3j^3@S<@_3P8zC1n&PO0I;6Rdr2&|G^B^#PA6ON}BtVCi01 zy-B?BL!jw|)$oq#=4)ZP+#qhhj;}KCg4zNi^Yi25;8ih!W{_V! zAHqqRKAupkb|gN7|1=?ft+Nw_TMzEU@j?XvV ze(*#G9l!KmR@L^J`p0OkXbIjm>*e@$*gbXv5s}q0O%IoX{o!Lo;(q^{OV~MJ%v=Q- zEGlyZ3skH^DmS=TT~$_EKBp{K4nI%i5s|wpKFw((7Bh@hn*P2KVr2ktF&yq*_i312 z?CL1}6r$w%-dsJ}{sDUI!g~*+YC}~qw*T~$gUOa3NLW82x> zIBh3ohO6Le@?S{)0y9$m1sb1u_xej5DgFXy`OwTl3~gC`$W)lG>pjbdS2^r5KSX^F z8aQrkzcu040h6$UsbK}p60Jjbc099+v1T8e3gfh z5s~k2qy)avgSztl9pzfo*Gyi;p!$|GH_(erZzp;iBj06O2ilFpEtIYpaMuv%({F@) zm>JevcGS~ZmX~l$R%#h%P_-eyc2a2dc%^N*D9dGkjidaK`LrYKhsQVfI*H;~n8>-T|$L zyI5#haw=~ai5p+OvCR*!u`*e%$;eY9+q+11X@>r|K3$%k?7{89v*f@n+zy#HG$V{V zf}e#WjTwP@zWHZ%K{xp9T33qu@^z2UXjxW$2y0$C7pH+rhSs4^vr}{|)HiY+B#tv| zw`Pq(2l%4Z%tn%ddben$0C+%$zd%hR?Op+?09wdN;(fBtZq860?cOD90>qZH|MDNU&~PQzdcFlN)DOJ2s6nX;Zw6&P>98V_8H2r*YQ>K z;G1JeDSVEZAe-K{>6{^;bBM?xxg;d0*1OI`-gk(6>Ib@s>KIinuz??L*2GI76@?{w z&>d2Y2rhaHYE_cdcz?W-TE%`Hx`Uj6BuFu!;y4!%fap3X_hNv^IH1hlM2RK68tu1V zjqXJ&BvKo8!Jy~wV)6l@Rc(berb!t*Dzx?6tXHE%y7i?c=+}=E4bkZM$oKEbXFFN7 zNZ9DvYiTvX^45@^6+i4}T_skTB2zs_@HAmft4&r4**!fUSEI6&s?fNPZPLfX(I_Z4 z=R4w^o{?s0dNf(eC&dZ+=%gcN=}Va|940CMSXO9I+-mN1e3ZUaK-4HSwe#>u0EEm z7?rFFrRw##ZmFtJZ>dE+qgA1f#Z{s3@g?iLC{wy|DBQZG1omGlKgBMn{FF{(WFss( znHW)6fdXlF-EgF^7BJ~#!9s6BWlTluRJ46UI}`BkQ!`*fgD&7_m8cb6& zcPp9wO6D2MPRdBkp=f^<-eJYNQwf=plGzpK6C^eBsPOI=NK@pP>{2pcHm6EB1?%&amiKP3%~NGa$Ys54Sy2g26cZ?3uKv$) zNorkN*2Xc*_X5kLj$cH}%d9#-lC0ddabzT++%@tKDR)(z8?jM-Hq=IXz67u1&szy6 z%`34PgB5JhED)f{h@z1M{H=I#Rcl7IA@ekK(Ngd_PMVp-k|@o#rWPr6nZL%8tTaMu zbAK^A&W>DA$16#*YfG|bm!+(g-NeVsi@>Rhow&Azh+kaz>D zQ}VUM<{DP=itB1=q+lgaX$@;{;c{4-TagsatyGmQvg1@_s#8v5{iwUeDqF1P zL)RFr+@rkU;Qu7n@WTDDW3{bj~oS1zQ(uM9gU7yI`)z(mO-IvQCYGMDix>kHatzKDrTHGy5Pb*bHD@8NQR>{&? zyD6@j)iYKzE1{Bw^b)ILp=uV=y=cB@gmoX1RI)5}tibR{+**~s-V;~JVpSh8cnJ2M zE1~DCUquQs2?Z_Gz#=8BTRo67k*mQ{(}I+%lJ&Gu6N@#nNHOYST3NSwkdU~jR@PO2 zw^r88SYh5Mm8_{BsgiZO=WkNS`q+ilvBIn(L`I_^*Oxv2FR5c~{kzq%8vjOhteUio z*q2G@GqLIt(V0+9BkMD5Wql?bxcP%Eu~^^JA7Rf3FGzaa4tB;|m&*<$*%$I2*fY_w zq~(u!-9)@fYR(%<_Q#9`(A0Hp++U^bj~U%#PIcJ+n190V8OdiR@C}3LYVk?iA2az> zzZb6oYS>?MKY2lHDrPTj8*FFITG+$s8`z;DIu&I9XwB4`{BV0Z`e*(L`)A&sPTVTD z*_6wEWyy)`d$29TA>;S%_M4XUjJ(#9!*H$#I}(+Awzx=z9e zS@btCMa}=`1~s#pJ(*&tQOz9TudNW@7?0nkW?Em70M!IvKop+kug*Z`8UE@CWS--% z-azIsf0=>I^Qds1KW_*^Kso|JcUFYXc;s*(#$GMAGO%rB&zBscUg^yU8kYadFy(Db zWCEz%Iv96^nW(6KYjPgL%Hv7O1LQ?Yo;oW}UP_)VDS7IxJjE$_k~D>=k2Qp0OF zWI0D&%OSy`b?h^98@Z0HW4wsyqhX%CEN_j609)Pz=AH0o0Kj*xcF;l#^hut!Z%8|6 zIrN?t=AB?jq&Hf~gMP`=_6=(Xy@yjAduO&`woqhta;7tOmP>#{@S-4rk}k>YM`~)5 zcVOSJcXu2oPJTbfVPQnVKF!~a4*?qk8Ur?bZ)FHZJcvVeBo65vap>=iV?<9JOL|lA z%Va#Valyhd0fUd@1DcGrT&Kp!@h-v|`~0nMU|6#}05>F1Bax>rCC}!DRQ%vrBPoyI zT2Y9ZKoj1U@)9M&NhOwy!V?QsJu{BH?Krcw#yHnWVhN5*NBs9qxtNg82}O^;zcu z?X514vB-r^%h_*Bx^-9pGg3?CE1bq6r{$N!$L?7G$N@N8(zN}MoQ5t`-f>v~IyZYC z%)>$SG~W_+8abrul6e|ZW=*hsqZ^B)&^wYefcrdFAla~uHFt%MN=A;sr83cC--FF~ z9-_fKvFX0)$XBEcz*rQcvRHM(fBJ@0G}gLOuy}VQnoft`;lYE5$q}uC4#l?yJEwPQ zZMNg|4(Ll}e~p2ALaN_%32?3s{mQc6bUKL8Z@M@6y{3hK-VVk)mxXs9fq|cmBbz&p zVxBls=Ee~#FOF=5aYQYSqvz~6QkJG5pok;jG6*e6M!*({wu#W0$#S?n7RNAeYy_zfxvV45oj=kyrXAEYB$70V}5^He5j1ZJnljkm;RSNABroHj$tB!5%TaRZMRa-Y$Xf;i#POlGojt&FpNx zj!$Nk%+NVJy-o4oEou_Mcc;c(p zQ4h>EWofdgh@>+2J1B~sjk6Jd`R`gCNl zkMOP~6sE?Y-R!F5zIm66?$>lBsZ>q#{>C;Rf&@5xd1e_pO?@1oy__?@oO^1d53S+}CKl5{0P z>RLczwUFRtN8EDmkAFmhsY}%Sg&8UfrvyDO$zOO2KBLS3F>4i4m*duTn|f%IXddFN z%uRsWWS&9)g3lE6GO6cAW611iiDxgIOZkkBLCp83f(ELKCK5KTUf!xq#_WFHg&b6( zu9MKo=wYntC6I0IXUH95oMA6e>;y?8*Gq~~U9f>VU!Tj~2=gfR$$|*P+Ah!u2o$5d zV5MMgBDxqi@3=l`S+8m{tHemaS#3EmUjm6RuSf8=0pFiI5Q)V-my6Jjgy-^^4`1-K~sz@SHMtRr2(Vdm1mmufG5ubc+$zlY_{#V96R!tB<5NpN`@Sl z6aSrHv7?*%0iK1C{%`Z=5Su?xnvqr|uZW)cq0151-&hg-br^z4_7&2(#mKWqx@SHavbk?jcz4wT%bZPt zMCNSd>if|n%31jM0}5NiCs3i`(l_7-ExP1-mGA{lA7PpFcyQ%gQRxMbgr3Ak2{@b-qv?Pc32j<~Av|%$bh6orie!IR-}9EfiTRCc%zx=$=esb2-CFQ&>I^!z3P+MhKrMdvv1EN^@2is?fB1@T z+XoKPRr)yAS{2NI9w+zf@esB1QffO@Kez4L8J9}`?1~R9ir1IyU-Z)F`qCXycOpEc zGASS4l2ZM@{1J`&w`WIUQD-c!&uPB+1hxCtE0nzAY+^p*bv%ReiXC{CV(d1 zd7*rdLcU5AY==xw(zR1SYeTa*%f#p2jl~`i+WM`ikfLh-8u{TX$*xtTF+&dL(8HVb z37dh(!x#D=p!piPyhmuAXjvJRrg4k&H<*P!RO)uB#??P9o8xHr|Ca0n`5VU(-jMZR z)u@UfpNYySn*YfJJp2C4lAeB{uV@HpBjx&CzXfu!&mUiJ!{sQ~j+YnT)s6^glow>A2|b&YwiNAFRD#$V z{FV7kxy#Y_AFFHiz>`wox6p9tF0_=0PPG+1L+F88p-H;G28S8ixh&~Cpe?`exYa*v zesfc<^^zxdo6xGP$^9fzAS-I%u*C_UYMGOQN&fd=<~Z=&HoonK4Qog>)<;A*27up= z2JcZ4p`&h3#B9D{IL3CMVYZ9s%^?oI-M^Kl+7P&}w)B@9vlO)It_7aN^(iy5u7d$; zfoD)pRllpDTwnhT^rNxwiq*B|PksQZL@2hIJyB+M< z`AE*RY_w8z?!^5o(`ML>HAtvsP^SuLUYMiQ%)1>SgpjplBb+)hel(`90-&u`e_t%MXu``_chb z+fg+sn&ZSXsuZgTexh7EjYp0T;1RjN93`{^=I*5AyS6gs3%w)zYARw?uPLFJEN^@- z4UG|<31~e5?fvKg9d+fXul|S&7GuRWRrjMwpbq)j3R!;k-btZWsis1P8JA8&aAHlx zB>Ow7redP~J)x!|%l;l;Q!&B*&a9~zZ-2XMDl+ZwjG79!{XMRxBE$Ypuc;Vkf4gcb z((P|&O@+(;POGVK+TYN(H2SWoKvZ;OEBa;1%$x#6^B@o@GnW>)l~B}!{-4NWw6m?D!1CUZB|53>CL=!U<8|=NP>{V3DXq;>GEB>6b|Ol zNX%E0{6riBzVjN|M?QmUS)aN3Wzx=iHE^NL;g*7aUHiXRu*V7}bm7KhrlW)T0+cCM2>bCk5 z9;o!8nrOJ2ltth6xg%AjfWN^xp)=Bbl>k;npPS5AwjHvrvK zOVQ(3L<=5v5&o`-ZvQ-;n7@=r?JUTR%h))+J~DrE9rdl6lPft{!en$MVLp1@@I~#7 zl0UJ~iP^Yy!%hV7-RRS&urCSPWj>Uvt8F`6_p@efK-(!--;Ko4VMW_tHKXs-%&%#O zlu)|_`--)}z{!SAl&`%t7J3u*m&#Z$JrEwK>QQj3+E|2@X{YGD`tk&3HS_im6hYWHtX_|eqW7xsiP`tET1Fw zZNmB3F_#PV?S}63ZGa{I`2d06FZR$9?|>hAoNWGT)6uqfdKX)!tO#l!KY?TgmfdRsbFBgTnV+2Q-k+@WZms%@M||Y;@@}^W&i*IiO9u zTxg%a?^0NB7g?#*2F1+JD%#u(p$qr z4xH$=4C2N?clmIF;rqRBaTk8$xZu!#WIpm) z7HeG+)gEpfNsSjlPRVKOIMpF6dX+D%xMlwLYZ6 zR2@dCuWwY0O2CU_{b3@b(Q#M&2W|FTaD?{t7dMEl_ z9!&5iw*P@u;oAwSs`-7Zs`-ZNlYQx{YtgSH3SW`M)&@(jEh#T)_0`PHD6R2AWk;i* z;>u)Fvephor^V=`X!?L~@r&(*i`Tc?yYUs?G5WvZZa~Rk#){~-CE&Nc5z-*NQ2I@2 z7ftViEOIqohls&eRO1OQxB0qI)Wtp&O7Q^_`XkV81%OxpfHB{sH%Yew3$&mUg8dvc zA=nGSAqp-4_WV}(Y{6<_eZsqgbFf0J>D5dAW69J*!z_UzW&U^Y5;;7YL7 zLOQQT>&JQ{q%QDwFh5Wf=>s@$wW-;zR#kH)`#S(^iG+Hk6+14Nf9gl%N7YOXMmJ8? zx&o+|_;x_+sz7HkgI5WuMVY)$33b7~EJHfvQU+JLk^Gw6B`_LBr3~gJB2?`TH;#%T zTLeb#qg(;>pK}mPL)ECkJh_YNr5Zhn2&llrk-5}*yaLRY663+jQetvZ7Y-ZA<2Dj~ z%7xUt(S6G0u_VQKxG%~5#*QS%jEfH>FJ&OmNFMjQa7@4R#`HUPRKG`$EpN=&3NJb~ z1g+xHV=Ejpw!AT8%N;#7>RR5I@fTir{KYAQFGkp*@z3UdD^>+6O+LT<|96eoZ)X^sn1V)Ei6>q=UyCc zfBcds5fR@XzXYouNkLT17D+*7>`GEhZUA3>^g!E5TUV#OKjvcoluls{o-y52(Wf-D zuJzoWsAC20e?0S8iovWi0Wfww*1Q+;yyqr{!SlYL7!qgiNIw63I58{9{57b7jy=-r`8`w*NxtxMxCRA1K-26OB|GGYP(=`pn_FH7m68@k zp5KC+MI_-wi(|tge}a;qQ{ejpx*Y^+y&!<(KfNHsw;S!~dJt-TV8n7eNKfI=<)G*~ zi5>@mazn4?HwevOHS$2CYPj6Mtax_^q+Warl<)P z0c}Vb?2LQLrDkFN*?0n~b-^%zLH2p%pSH4fUFt>28TqH=3i6vSvK=&PDZB&Z|B;?6 z^GaM&`YbQRCHSaFYvxI55wsAz+{R)Y5SilBXwq<1@OQD1^^)j{(&EsB!|E7wGa)l| zldisVGQ1JRnENXFoWI+Zfo?I#Riu=>GUy>v;xX;H=YVY^LTVEwq+lMrTrqM^0|l!B zT;{tz5c%T;d6-vcuCMWj2jnLr)qdbw*4G5bht@yomaBj2L{xT7MLD=0cv986Eq&BY zmTh2wEm%@sOMVZk`vH#TI(@gA-#rU!6?hWwBv4V$+6ADrs?9AhZ$FFcQ5jL4m5qQZ z)>=Y-KY%x25oRK9;guccKSADQx52VmR%@{#SXdQVP`#=axxe)hlINca6vfCN=#Bc^ zR~Bc_-LqJm`%=JAZ2Fy_j+};!r~{?#4;1b0bJ2aZp7mFCYSsqrgKqANzXyl zPg~qwa`mstBF`bwatt8rI6zi6Ko-s~a9%-R!3WXbeEu9IKg;F&J-`-hB@Zebe%*bSB;z-R5ZM5ZP5G<>MFSyRt+k;LdpT7mYVtxFUgC z6;3FaF&Fd0b2|mhm(buh>HR!bp;tbsP9K4ZQ>XWrONac@Vca*Q5nsu^@6Ymu`XsSx zkFRK-Vz@dM8cU}D1D*vW1DuSoTf$Nnm&!kCvy2m!%Fhl|y(v(S(|am*j? zB#!wvyMSXxxtrNHo&5bEbjA;;lrtnc<#dvLF?{-^`)iS1_jZ&2foL^r`r%bR(SFh**U7 zn-IaHo5<|i#IHqZCV#VoX43gbn8{mkCV!qsH^B=z`B6?Tg8AP51x$X3lV@=9U7UOh zezq^e5yjF$7K0Ip^-7>vN;BgW%V*%zSMrigy+O6_G zgW%xP_WFzHVKJUHbpH1`1IFAmvLmpn#eTL!d*D$z%Ke*&K0sY7dY05mghB12wAIzX zl0@j(s@gz=$WfG7!V;KwKZ;~ZKjvu}y3*%u^WpPKf3077M_IKO=mb9smi@JI_1i?G zw4-PfHd=IFjJtgpj)r-7f}wvSc41tNcDZ0w^2UG8NIAhoNsAz2!ggT{J>8?^Uk^tZ zxiCHrN0`8;p8zXx5~zlcCS zYM6G3kUB{L4o-cx-%j)b@3t{>nodfx)`R*u!BDl0WI!QPl&R$R=640(6^3m-cE2oSz-A!}BTJk5oj9T-}B$zoB7R zjXx1-*OOR{R#=TzJW`pU7(ksgT#X67t@z4Rfla@tRXAZ^W#Ud4_l(%vKcO;i^zY*$ z1dOa!)L_I+F!?}2Tt}pYMT{78TfGrl8_6c?GDfXU2KV|@5?L349tLw^iFTl&TbPeU z`C09>N=P__&H|Beif-+nK^j3t3px9lM$it@BTwohRI@2y%xd-BVy^mtNSY3$l5yCu zRY88tJgdN*|A9q80Ap~6%3BpOOW}xbnH*l`a?n-w1LN^RUw*71&Ee2``lgb9qLS$M zt|~7%Dp#L|?m$ScjylnhNlWxfsio@-BxxP%53)iZ(&z*ezFYiS)UWmWigx>oVC9a` zL8+~PF1=@d4J}uO);FN|Ir0TS&TU@|Xm7~X=)nldOW>Yr@cHY^J8RL?IqC!!MQr*& z=&t=rHvDO>?xgcBLdHkVC|U$&tQ@cgn4KGqEOf8ySY)`yll!Bh?movNJ?HqMsW~SX zYB^@WxGUA4`7W#iPJ0bHf#w|d(@f*le`T$C)yWY6WZ+LwpryS-4wMG>Uckqe}h_ z01>zOkotvT$rjy>uLidvm~JCpv*-Zs8bE?97!lGQWI>5;G3Iu|hC;Lp>?M%m-M!eD z<^VwMw(Slgm|h;IRVF9XDgdyT^ME?;gU#d(wPZyk_;Kh!qpBTSXk-mS4;2aawj5c> z|Exd->`NeZlW+p`P4CBE2Bx2YK`NO$(UeH^y%9M5K9v_m4{P%cSJ6BpXS$%HG_Z|8 zc2zepu%>ikB&JtBQ(kneyy!T@k;TcKq*#)|gIaLFYzI9*KnnYxk$R*ZB|Ym)i3IAzE)?w$M)}8(aXSD!L3^@(tE5P~)DlzG zihS~qcCovjF2DTjdsT0uAfft;EJ4_Dl+NL|pQm&9Cx3!-_~U4?^KdTHLv;A9=m~H+t!@zTAsUSn)+C zjJki3iPjdEs@iVM5qk^l{1J^~$MP=HfZzrS?FM?s)Oz8P-0cguNWy{CZeOet zY@qNWl`P{*iZ9$XY`$>CMT)Lu;lyliyf|^qpK{^9hO6OvyXo#;Tf7!lc-@Xvd91x( z;ARm>>=!mntWdSfRf~vrR^X$v*WjdeDP8XgW7cF~`rqUG@2ZYP#@ulWjaibaMHRfi zUQbUnfF{jKUQb2O8BwO@Kzh!IFdm50r%(#-d?0;pi!ya?o1)F#8JC}nDZqx5dhi|K z;348#%L_b;)(!lUGT1Fj+K_n#c4=FJ4s)^F?^&)*FGw6kWgr!g(w8s)mABu7{a_eG`VC z>BT#T1}_=Ja!tTXG$cdaqqrqE>VByuuOPRCM6Z9FkcLFb`o`d7#mL&J=I6AD{K;x& zHyTkeT(2{khWb#)Sv{;VfPO^D00&;fY%?>+#)6fD>`{Nn-j23fu}D|}AC-ah)1mX^ ziAp<&tS2nb#MPtLr(Ma;X`^eV#Qb-2ek5#gnKD-=^3UA~>(vVD)tbCsDqb^a2LtFO zGLn+t)erm)kTI^reJS!c$0U4W{jP-UxsfwfkzN-?LrTeO_ZEtlH(kI;o>ZhGKo9BT zQI1m50$im0BV`fPv=ECbITT z@_zT{T*9Zf&0qM*R(^<)y%_FUV<33+6UTRr#Xb$zrmd?%P*_^g=jWL6)$xt#-9p`kve?* z4b)W+9s~d>5PGQ_hn}wfZ}#3jFskb68=gr*G9iHzG)gR3W0SU|S|yq)BPcm!hMW`5 z0LoP<*2lEewpv@50dF8UGs5ZV(X^Eo+n1JBZK;nJszs?SbtZvaKp_bl0$2#BaE2g+ zfVq;)yVlzKoVg@u`+VQ~egC|FKxUuoK5MVF_S$Q&^;_zRXrrt>R_MTfk1+PQ98}jm z$0$KpzIK1%M2wue0>PucM9zwpdBzpw$~j~F-sK?B|7YQEWEIaBIhPdggU_vurw4p6 z9?R+~%T4iJZG01MhM#s{5%2#$aeUzW=kV{ZT$U|`kEa?)54_O3KSragTzCz|J)JY- zE}erQM?zq$Eva_&d8u-~L#iyt`q8i!06n|BQd-l;DjMvuv0BnX@PE;IaS9;}7>}3n zat5_RU>}}HEGfOn_$^!@SI#YTuu2B7@V*fq3?d)5{+o)I%UVexuFgjqGqU@lZXm1! z2QCK-AE;RV$6TOVe8K7sT$#=4N3V~||7rX0F~ij>SXxT;2HrH|Pe*N|>xR@rs9@JU zZCzJI*A1y(@Flt~@@?FnOpF&!8p>QCOE#y8Hv`sy3rF@v>&loA#T9iR`IrZC+WK?M zmAdsW!ekP}-fjm5Pre4t#fb4Z59XA78D?orfBjhQJ}8I#rYcuAivDBybc_I0!;u_} zNGXOH>VO>Xg$o)mwF2{Ls}TtkgTRWbK zo!CG>tdW9`E^agdYOk(`uPYVS*NOh4IikPKQ7XKFXsZ6>X`;PPYBKysQ{46@bV1s& zl=W1Xz1Du#Uca~|nO@qr9R7aWs#_#&tW;Tp^PQwUmeK={_uUNoxP5FKzxWNH>qB&I z9HOtEE)D_ZyuJKh?Xl_}lruOL{xIEm7xbgOblQ_sZIB#%(b8{+_FawL2N>o&GJd%6-syYZYS^ga$LZZ_ThRDh`yVf-o#E>d2geN==yKr_1}rR{)WWsvG996 zTc&TMX=N^m$y-0bE2WIS=>JFK`^kl9RCncNED9T`On&%$V@^Lf?F@YcXxBrIr??~@`IfK<%SssnmSuy@Nte$!Vo<}jh+&;(c z08<2f`HAEb(amC8WZRvz^$@FGs5MML>8?~lXM=fzv#d+v-{eyW?>{~V*ssB|_TtcCkqUl^7i-^cr} z_&-FC2c~=uJ$^dne@qX@3&Yam*UrzN$3R+)9`(-;OOLypiS&rihv=GvfWjmxd1MDt2&j)q4YPr9msrTgY1@ zpf$pKId5e^E8XXymcA+jUg=#R6;&(xv_ZMapp6cHpM92|Yx7QCBAf|H_6}Ju2_q>( zu+D5xmoVkR8Cr1$7M=Bqeg6_+*#QTxIhRPns*sc!R-Db=G<-hAtP9VCU_2$YySSnK zR~G%Zu{Fh#Vdv}jUD~}lq^&);msxoiSTTc=^PKSHIig&ORCG?(VPOY$O86*HE&`N0 z0F>J;`TLP_yD4>tYD!_Sz6QfJpD35sNRj&be4-q+fO7MRa?}FK%_quH3n(|AC`T=z z+l_O<-}`!*rpmnF+vUkT`! z6_$LconSN{9yIqQv;{es!k~T(YGHi{^BI~+Aq#=iq&LbP#Oz0Dd^42zfepjJ)+6Jx z^$E#>8kV&jA#fPUkTu(Z;vfXtSSts`fz@4uRvwCj5cr6-@=+Xwz;V`^Ku-;>W37qe zTU3|9S?`@I71b-c6dTqa7}oKb-bpyDAPq34d_cK0@B#JM4<--jF2&jEO`DOG+lU%* zeYNmLv$`NBrtgCdz9k4I)!eR8orbqC1w2Mak$?{(-;K2qeIv3U)<)t-%42Ob;$n5# zL_~13MZOAcLK_FN@ki!HGO97dZXAuPFUjWs*S_1|KKT+Ikft_XujwmF09dI zN%QC34sDu4RO_5IE3-xQ2u`h)tw2tFjvcYkD?8PB?-<%3;Q@V5oA-`fStn~Nu_R)l zM;k4w^BojP>#KA*`&Uf`D&wld=F-a$uSD&YQ?m=*MLjIMWv@v$wFPcUEeQ)hhxJ`a z9djLpd?#cz_rNy0InBm>LOmWX@AcRB zwA?*2#^u&msJVNs_*LnOX8a)F7PoVUpnsDMdib>yC6#3ac`pWOfYv6OEj|*#&5~4mm8dvqxV?SQp>~G7!k0tv14ZFvA-urvm zzXKC8mBJ|KR0m8epdUVzv{cOA#2B37=5086wKrRGeWdu1vHT=Kf>|9iM!Yi^CBvcy z1P!xUoh<6YRZUC$+n!=ksK3EFsmK)H>FynFz3?dzZ9iWxD3wc?zN;0TJ62}9)#IQD zmC80*?G|;V(2il)m1ogz-+%~iK-=eRh$jMSZxHnyu*l7_<^~0k1#1E!h((#*&hsb` zCsN(|+(OcNgL10R2I|fYHnYFlUL~dC-a#Ws1!X`|=^_x7M&EINuWg2!3+m?&WLSie zjbz3~(-dH%oS2`7_~`5Y$VK@mZ%95nbPO6;7=Z=vh zS+DGaDKGNln7wz97ZW~4qYUi3efakqKL8%_TL(bGB+`M8!QRvV$qIM1#) z&1~nB2@`9rlzG{FP24|XvC28?BGjs(nj{@AnVCT|vtzBf*>e%X`yZpjNwjI>BldwJdVUb_(7P5Zqg zfnF;fB+5l1gNf^mjKRcpM)yGCI_1Pb;yPvjK;pWlF4m@X&zojjE_z2%zZB_)=QEON zBl#`QV6lKI?$1MffTLhxReh#EVL+<qVa}I3KbLc@3 z)_N#Bq2G=O5}I8I3=pe}z)}c?x0kg*VERE|npg`2rXK|69BYBVAdua9)|x<14c4)i zB{1i_3uN`sZ~{}p*P3AbpXCC>od=3@&r-hD>`a!I4i7vMF5o9>ZFk+&y+`adIDF zEl%!Q*5c%TnYAX0e`Dl6jO+Syie46TboDbw*F|V8J3GvEeN!^6JIPT~k(Jv(VJ|5d zt*v=-ikpE3==7?F2`neh?}GCBV>@p6qr8qV<%SnXI%7QA!HlCCoOQq)GN}hD0ZD94 zgApvF0po|y8;#Gyl&nah#c>H@H&w1A3jjo`N6^>V5%gP8+dS$iS=}k>v(Ok; z=hbt2c-?6v1X#k_ym}$ zQkUg$TQ}J^&-{7Qa$pR4aNQ@PUPAS;Rh2DK@LMz$AC zR$<*wiN#OyS?Z6BHSUFlAQ+3+3|R+WG0ZxU*^4lLea3dek}8+lr&RVt9x~+)uPP1= z^@p(1yV#d6u)eggzFa?~FIQq;F1?uj%=%-$Xn$zE&>UMY%+zv2GcV1pou%f6Juy6& zW)ap2Y{B4B8za2~<~rWm&w^De&2mCOOhGeufXrM`ea~pPnp?UPQ~oVW_plH^J5jD& zPR8!ZcBYVy(Xsh3bT=iKyFzdaP6OI}Fn1m5oE%T(c^OtJev|`6OI3xVd!uNwCHd)5xO@93WRk#S~R26NfR)0h0t^G=}Od8I9K6Z5-_UpuVN`4xP? zSEA7*{JObwa2FB%^XfUt*(p4EImucE+eqyAoQ=c-wz!SN~0v5mwI=2%_E zHWE9SV|5wZNbF#a)n#lWv4c5Qm$8k+4)~RUiFk`>Be5bTYkf@CrZUHBtE^866^EGq z4#k;SVN>2(F1nf6fjcF4it0BDt>sYs`UPyD)$#N)t~o)H)+$#bq?~$C*0KvRQ<2AK zy}{+|uOES&{TcKzjDK)g`TEiqxL-6Oo(ao&1+tL*`nPfs>puw$;wRxuD30*FN}`+G8lC&|;O*3Yws4h;_lOmh$`Stt*|sIy$y zTo3wO&MF00C2)uLfD=j-dgKe5HCd~l@nd5ODS=yr5+dWRhD<}oI2c&|M zgGy!#*cV4)bLhGF@!)Z-W}nB0m4fVQ>T|ZWIdSZqH^2&8bo;enAz{~_8?FVgUV>L<2~}k;0oZNjX8wBEY<cl;CpfY z;fFsWj`dmoVVoS%Ka5*t4ty&0QMMm0Za^1tHpNF7K*@97Ya|$B;f*<98yN(?;mS|L$GQyEmwLaJ&>9eEW8#qp@R#GUdjcSw3>+J!TBPs^4$4o7)R^DfT zQjWUTl|_B;aR(*Mm-A#;-IyVBxXQW9p=cu!P^j_{MP9Vy@KX92xnEt+pR>BCBt*6}}sU1{YkM&<1nQb@4{+!48K3ng=icr?wQKe|V zq%Y`DiW(^{6|)G{_vG+L@Uv(1^7~-&Xprrl#UYP$}i$U@HS#cS0)@TMlf^fYwNC zWn$}AXk}q*B(|=HRyMY>uyr-GMq!I=txKUb8e5~VMSU4v911k}#sHW=XTM@qV5$M& zo5sgIai=2=KN@k2&-?PwcZ7-1<6vO~(`E2qFkS9O{Me!oC?)lvqV6#sKMYolTGQSf zm*0V-9~nOwTd%}3R|POv3bJKogXBCbJb9Qg*F;%;_p_Pn6S^>)n5&uMB}V4j7+=*V z@KrnGt9^>T2>D75x2NGW=utlymanQ4_)2lMF6Df6^nc7()r#{&ldnYJt6UhxR=3LRm5C8pt#R64@=N*sYl(&z}|D={690AGcXufz;&rDF^DN_1cg zH@uOr#7t~qACRxak=V+_7Vwprg{_g;0=^Qnv6Y1_;45(ywz9DWd?k*?)+lUIUy!ew zePeMS57_FguOK15D~_+ok>2UOQYzXd*$0b5ZsCn=8aRYGcdZ)@E48KVsMj%h`l>^~ zW!srPQ_mJRN3I=Y>M+J%e7=X3`8S44_XRlJTlRuE-;bjC9&u!mZ{SVm!8-K`l9sVp z(rnhcp)Q2A3ix)?aPM;udEbr0zK_`#j1G0_%({p@58nR~zJDx#zoe$qrDi{d;@zqp zQsqpCRJp=|*h1>Zk~Vd5Waqtu+@AsS_cbsG-mkEobo_uCQIAqA^Z z@su0%A4aJtgRc{V^ty!gAu~QE;WgS+<6d}0adYsHu)bPt>ON3hZ*OdWvx@qIh*m9x zBJ;5^87^BKHm<6P$@?4^Sak@;7lQdCSd!6+{c^DC4 z);6O3RWD^Nb>PK;B68y{&7r-jY}Ux5t*SFNYUI;SRWoZ$ppB}ntTB=HsW!02B#{bM zIK$q>>Rz!Zq}CMGx%63sVpBv!pnZ6LxfSFk>B>Q{f!Hbq0ZDR1SfzBJsJoMbykM15 z>vA6PW_T)_E3!(O89|@HaQ3+I4hg}pp^un(As>y!XQ5FRAKyhT(75<6<_Eoad>45{ zznT;uHg<^pz!(wO#(!!e?Tz|cr8;*T<9WBi|90SCZ?-lohqxoy;7xBJc|q8(F2)s8 zC4FA2us&=^_Y6iGtntR(NR^V~0^Ky+QxS1wsP9KuTW_WBCCvnaks^7ij7 z<|PcwcrsX({*Zj|?LR|0aX)eu_f-3=abZsE@5I_PuJdAT8h3ZBO?>i=SewT6%Vr!O z5RPARNtE`(v;{e4d}WY))8!5viVnK=W=XqT@^{)v_{is`&vfVyMa|)H_6p^D5m#mH z4wSuKDQmt1cK`#F?~*k?$KMMh_WC~u>kkBiab3Y1R0}Rcc z*w8Gg!cAiIu`nXF|Af;oCU?Xoksl<{e^Ju-%t#u)$+0$#?=@!I#TT$(;ef{TO)!-J zuG`@k+l(DagG7Dgwm0rXB1+VaW0+#3LZt|38mAat&Uz74qe!~(JsTo~)#K(EDMzX^ zm~}f-j!Ylk&Xi*(YcS>bDQht0*uxr3IsS$N3HNtmBiXKj)k!{jdO0Ye#H5m%g?Gz2`fk02uI!0 zpHz>E{0kF@7yw@y&c6_|H@tmIswz++PQlv9ALWXEfLTp2fJxlY-SiIN^;@O%-&dly>eS!ug* z$n}a8Z$+$~s#m0XZ;7?j^olg^*JAAvdc_Fu7h>%Uy&~N=lDGAW47an|d#}5w+O5{P z^}7b$>U&~SyNK0MfH38H$vE$=9{T{}u-tYx?pZSyOF^^T&CT+9m;G#v#bjVHB{Sr9 zp7&;$Z+13`!ptUMv!}2~hoW7D6v1NKT-`MZ<-7cSBZQ~1z_8k+XsMnXQ{f?`^R6bR zYMH}xW0`}Ffn;6Dj^)6y96AO9b_F|@2gma0*a#u`KE19iACBcS6#{3RyS%=lU81_n zrQZwkbV~H^u+#TmTj-rhi;(cKRD_l)xfDKGwjZRt{*nCK&!mk;E$DW_61Tm6>4&&3 zaoZ#2x}+XutU=1G7oG2WN24axa(BVV=nT^0b;@!YMv3NGr zlLLcTzPur_FwQ@rHWi0xBi7t6X8Nxgme5((&w19zU7Sn%A;F6WE`L>`w{%Gs#9)$l{3m53%)u5IAHr17l=$kgdkG z$McQ05e}P2yN*z}u-ZJ?dxzSur+uKB5tx;nwn*%9?aa0N=1^WbW~9EAPIsYaN%HsS zcxU2O9{t-pF-F8|Q20MAN@}pOTyBc@4cz}_DeSA$;~(jL87mOMQ;dc9DtHt;)%TjL z1=n#8kviRBOrbvh&;gzFsMiTWj4fV-b?excOk_NqkM}=oyRi)RKN}!33A~;Ka!bu0>$B@_bN#>Wm34_1dcByo6x0Pk9@EiT_0U8~3MKo=}%o zHi4#jZCSpg=VGb@5$JpG_5!azns2O5!(711e3!cHT19Qs=1ge+rAsflLu@+jQoE(_ ziInzNC4JP_AHe+wOuT=~Ym4na{K1}K^Ou#_Bj1bP-_T~2fF;r3TR~swyCV^5A#ZZ| zc<;A^SYb;OU~#U_av%xH^3+DEovNp?So<2{7M2Ogn9g4|(H;Y4D77KJ4BUK7@cmQL z)O8fYGt1Hb7ZYdTZ7hmq+?~pJc)6snqi966+FDPius_qdOT0P{elwVK=PqBHP*hYHIs&+#LPE7+YQ0B-&=* z_EItp@3v76e?ipJXNmYV{!^1&+Px0#ZU<(a1+KSKfuq1zaqAm#QnC8uURm9PDITh? z-uUeR145CVdxe0CJ}KuJGY->JFKX_b8^s(^EzeO~T}5raOv0i3J)EAVmlXK>1=Y7h zcLgtaK6{vHt4M{pvq>^C~!Mqh7~){Xm)xtzz;rd1B-LE-5j z#cXw}o-M}dfhhAHS9@n_o-LyDoyQ%{-NI8#>2+FH@HnY$v8eiNH)#jI=?=6fyJ88##P<4Ju5}068IyRh44w2^mkxJhV z^nL%7-sjzyNv)vCUq^*&_r91(mi^p<~9_ z?ey8Auzp8mK8-DvqinknWm^t)Lhyc=aQJF@H51HkIsp^Kq6Q=MTZ9`2$|12L2Pqzmzd!NW zUrl`W#DC-2xqpgA@4UPH4|3%N;~^k#WDP^|n^r~pH8a0Y`!!oZ+TOYgk!M_Wy?DBS zoxYx(_Ty=cBR0NVH%$J3S?oa*9nNVE!86UFRQ7m+QQr;BPez=R#87K?tV?7kE52w4 zR=!mbb9fJK52?P*MwH53pD^}Q_39Ha9QDR->vza_yN{4lLpH+DaBmZysfK2~4a_4) zwQmXr|K3c1n(ATW^|y$byk9}cWku~W{=g4hhF1VD8voTs0q?i6xVJ;1Hn$Mu#d{gZ zirOlw(2Cp|7YA~}X#4eGRBeq+h}Y-r>4QctJI+oeK$|F0HNhN| zE<|$DupVbrR%?Nd=04I$svXtrKZ<}!SPP>2fOIvGYqz9&HYjSZp*}lEH6oy=E^YDi z3}AX4F<5sCx__=dEMhvYmLvSChHa1IM!Dpyll1Z&+9j{%fLcGxwAlinaMet#(Q!!R z&Z5KK3k&Q^8;Azz`YtvCOSjjF|Z~(Sy51OMk&aVGdmSEN6{X@ zq6ZlAXWX!XX+00fC(I@rqyi5Rb&^am1`Ue-V2<3~5zo-S|Vms1`-pPeV3;oMJgD})omuo#GqYP zFJTEgQc>+&q+#^CV-y_^6kWUy20wTRp5B5XAm5Wz-6CdljF@^nF=c&xA~8b5z;$Nie#XB8;vF4C(_Cl`YtTxdvlq=xN6)3bIcq1r9m0K~c?lL)KCxZ8T$A8P&rs z6VR%KbR>zd3^@OHsvYK(;|?S2XZ)oOqXRa z@3>WIqgEEC*K%a+u?oWeHAP?{GgZk9n+z-{SZ;egLS9q$9K%G>XACO(H`Bp(A=>mJO9xVi3B4OU?(%jFJ;Cl4MgNphxA%HVhgq0#nGe)?eM1PkP z-I8c(DSCD~vYVo(WdIqkoG>Am;#AHYTvEk~l~_Q|wq!xJGxYQW6b7R>>D}J3U`QXv zo`Ys9$UPzzU}B!6=6JBa!}}IbrH;U@x@mbpFD|W)8X@pWT2kc};wSX8hrCzFT7^T_ zE@5MI5}wMOzKB72&W$1D4KiaUS)ViL(XUOxt1yWLf#=o58)W^PDGrlwQe}pxUXQUp zt`WAOjKec=nwIVb8QOrDkIowJSXrC93wn{ZO9~%jtOCQ72A2dGY(1+)q@|@$VSy%N z^~!b7#Z)1%6h3YB%GEX@a9NZr5< z0!u&7+=VF-q=)`Md{Cp!dp<536jrq+n+E>+ zuUG?b&c8JNxI7^RWKSM`8yo&zOZQV0RJ->JvYvL3^_M!E4WpmKM&Iw=WY^rACc_jR z+}eZF?F+0<4j7SYmsDBGvbHwZSyZT%4*?o^I?vbouT*n~q*m507Xr7XAwt1?zC-e5 z*5W%P*{sEPNY15Fi|>#eWG%i!5@xN5;x@KJ(&5dQiaI1+8bq{79vcfnk|_i+-(^Nt zZY@K)Ci}!4k}7AjP^W*Y106 zzG?TU{c)1I3=7|+$a-!%Od3t6G1CO7eG@WV3UyW#+S6o|r$+UHnPpR2Rk@nRY1VKFI|o z)u6fR;_$xVOI93<{G|sk#keq3&0uAv!FAbn3Rcm{j=Qd`8)vo<&L=o|S7_O_s3=fv zW!8I~wH2zXu4hGCWMeNDf*@UjIZj&cF1ifVN@k<%gs!;u;$^?@$3Wvq-Ua%T*@rl= zO~oNFcwmZ9!1^hGdCTfYZLA+Z>B9;#ZFp%T|GuMyXP(W!Z&%{?Eu=Vx~BReAFNW@&+u_juhagNGj&YY@-c~+b-Zw+S|C1ezNo%`bnPGE9oaW zWiO$h{wMp&%Xv5N=^qmB zQC`C0Lwyfu?gGHiO%q{+q4yVMfLjpXi=@YBF}$J;CYOO-QN@S z<_DX7*TP*6)|RxNGdI-&$6EqC-PfkhnpjuP-KG>*dD7mF{z7_7$>N%X_$A5N&FW)$ z(C5_$AFUw=pD*ZhE6PYCKfk8%>Uois;Lcux|ad#Shq26eGe0_s2>s(wu z9w25FEa5+4&mikuSQ~c)73BygyQk0 z{U6%V8-cLX-3MTCj`-g=<*U2FQkT2;@rnOk8Y3)MMfsU(pBz30V{{V6Xcvr86HK=r z^!X_2L0m!=jFk&);$Q7#F=u#+VB|clxXUN&cbcQLxRq7AWTPbUQBpw4D)f@E6qHCr zBXD7Rc$$UO0$Ft4?{G5ppq{m{E}3rWE8UnFl1*krdw6GHZ#YBq6Y;uvJ3~t{Omuj(QPi{fMVvd@C#! z*{~#p-nx><1?;mBx6cyCrQX=RJz1Z1CZw!J372Wjscp0=ks)adOBL-$lB8~&L}I+L z6#uA)#b}GC^1G!KdhW+GPoaH(74CcJdKJ=Cf3uWk%Ds}RqXXHv;k90JjQ7#1fQ~8p z3sr+u*NpNwcfm6yJ=jKvj*&~cR#Jm_C7szmk(NWjNp!(cNn1~zX{wKw&z=))6t%T& zHWn9BO5K??n_oyN&gT8mMo;CeQb}LATGnPwVgM-cm?lWm3$n{=q{f<(gZ)c0lUBtLfWD9GF+zCtMPPWec&~C0XTO>^o z0y>tk%bLa4nG$R9b*7uO_&ReMYw>mFRMz6_%qgs81rc_7=SfAK!(n4`Jd^R=F+39v z6jr8r4sJtaSO0W`$sCnIcbPIILPWy85i34j;DJYk}UF(dE!IsL_HEI(#h*@ zsddJEYiQpEF?3~$WL;Trq`k_v`4DJ?A6#dgd4;Dw<3kD37K=UJ7R!D1jxO#p9)77e zs+G}6b-DxO(@{5_1#nQcJ$5bJ1(yqOdBsZz0ED}nzIwPG?&O!l5<^%=D4&&$So;_Y zy_OL8E#@ogY478nj;wZLtxj?KRI)z~Gk=U^`n-{!f~2ptbe9@|_4p)dpMgH`OmsGA zi?=CiA2$6Rb~rEkKe0>u+N9J=rOXu3KM?i3D*Mkkl*ubsE9yDlvyy-12AlVCUU+tc z@yhRrs6wD|fOgKR7>=V5sHTl@tg5!Ms`!xcB)@KbO=L5}eZb&@{aBh~bW%M8Ep1S` z6#l7c@%34-tMxMT-fK5PBUGVHabkH*OqkX#mG+&%+8vp<;W&ZKuAt~|Ax~Nu3x+si z&!5sFkDCIEhbc|*^?dQ}qMn#}{)ABh|wwSh`q)!>=(F@05A$5CaT#1({ z%=~yT`<5gF;}(`adk{;!EZ!v94=BzWAux(2_>w(fXf%+Wez}~vPf@=ZIX__SgU9T1 z*6k_i!)&3B!ynT=k5ms>NE>JREc}g+`YcqA1AP`&$DTe5!K0?nLgm=jXF+fX{Ll-u z{fgNLVB4)-46iT-{@VLl_?iQAsA~CcObb)@y1|@4QxhXGaz9}8*H3-GIrP#LCV&Lf z;T!4J<`(9-VSuJ%hVKm2Kzd4&<|;i5nyW#ZLkbY%EKmW?K?Rtn)+Z>WQW*aOWMtI2 z)OkB-WBIjzF=g><3J3Jm7lW*V{n)FhyD&HETo0bc6E1Du4k56NKoi6q)V>HF@78X2 zxbem3NP06Rjf52A%*;^j+a035G-|S)vr{N<@ejfjh|w;F*ZSm_W1Nm8>z5CO;c%OcJ+Itagj{&Y~8vs999^>8l2t&iL!FS1B$% zdn$9Y&eZ1~a1=MQQtvG4&AHzU)lT$}Rl`ci<=lfoz@dt)w6L2HPC-FeyMtYBb)M~S zOrNYk8xe;L;nWneNmNh0@x^zMi0qnmbMTrvH**WkDWRhn^*)^pkt2Bhsz(Q-K=Fvv z%0Gj-V_ZVKG2Bo+X!*r1wc-}ZHy(*vxK_;Fb|%&)UOy6RlNjucwMjzWjf0g;>+7hN zm@j0U+RtO2YQ3x`EZ92}{z$PSW849d7iMdS>fH`sfvC+ZDLw>B@nO((qB;*%-Gl^N zuZr0fE4N9NKXMpHUZ4lneQ>D{Qz!0@b;A2~ z38%I}dimu^FP{xp?*n#I--)C#dy3And*AZf1*D(Ed9^*=Z>b&)}QBp_c{9Wxu1Qu{+#pXf2=Dg%*N zjM2BZnbY^pVd#s2e*cf@OSpg4PqA;m4pt_sBzN&A2($y=y%WCsQ}4|lEZW{1`|cah zv+piGgznMx@30|Aa-5Yie@XiX#$SJE!Qs5yuE%}9j1ibk%!|J`@~?BltrsjjTh4vw zf%}YOO`I%&|5I#tOhHU{;JmXSt;lv$&&9Ld?{`NN%FGyz4_FLWjSP2?>UVZ=Xa!#y z?yGC%xE249GLM{Iap2ZCzhOy5LDI)J5uDilO+8o-VKQ#N9+Uj-4&(ZbIX1=FE~&M? z6LDz9#i2NVBQB}2&XP-;&M*i%hj3Zf97(?`Dpz*M>iY<+6UJB}csFJ=InN1eTZw>& zjkaGiortNj3dgF$^d=e0_VpQSp6x{+zyIh|?`4R{&b;*d+R(*OakcA~9#NZ~l$#yS=sjax;G>hGUHtB`~M-lm#$lVM4{~ z9wf_MVi>&%F;QxhorB)%jCA@k?@o^INI6?EJoCxeRSY}Np*{~Q!WO=dI=9f#{y_|X z?FSps-8tRS{uX_;5SRsH-~RW6@xitg!;{8`=Qdfp8Nv3^MRVj=I6_Ai(D67*xw3Y7(jcY4AeEbg zbT^FAbQq#~j06Kyr?Lh{r?$uDdjzk>LXo8yGX#P@WslAG9Z9XnVfh3mYd$K|V}sIu z-Di)3G3O_2LJmHTPVS4=E5oWVsFhxn(GNX5uEK;AGis3GD%?Z3$tJGCQr6;8gL7Gn zM-4JucN15EAuOA?3M1J$YfGg5KgCs0oZ+RPkE;L-!GNLzz2pO$pF?;*=#~$ElCG%T zCQRBLzhcAPNFMv?h}$J=qhrv1La-IxYQay;ad3gsQh(?Wo@%tS@r)U{E~g<18bn#cvA?X98_qo z(>+)L=!yzG?R5KSsvj1^mpx~1(8d3Ixbz6ov^U~)9WkN%qFuVTL3%VKTvZ;gnx$shLVKw=*7#{-FZzwZwu<^}IG z+cS0ew0Pl@3t%F%MMm>FOxT&ZJ2{HN{0x^7ZV z^FW@%Qe~~$rO%2gdTuo>{JvS;Q*736k+mDb$8sc04?UBw_Joa`%tlcSsns6g)vz2k z@?AyMO(#Wrqqxs-39r_;ibBp-!H?Oa;kF#%&1h!N=U%?-$7&~B96p(oS*@(oHqH5>CWfjm9;mxkhPPnMWx~R74w7nyC(U)caNr%UIPzZ`qZUZ&O z)is5=xwyIAFn;?RiUoXN5rt|UzM4X{s=hJC_LhXR#-~-o)|;{FYpJ_&eXog_+?%D{ zJ_&Wi8uZv>c54Y*BOmTClmEO|NV@MGGF&3eXKyTTAY<)oV3lxpe@VQ%e}XFs7O~VO zYAYPL`kz9&L(t~E!yWWeW+SqQ<)YVD#XR(VLiv6|Q+X0S^nI3xzB}GSe=y!dzdhbV zzbW2B?~nJ;FEKsz3{9oadsor^q83rz7mubgn1H6@HC^;v8;j1y;n7r>i{3<2Ny9t@ zULAl!3q&=Ykt~R{maAyJ@oiXt?qvI`+U*lu+QSnk-+rz4T1gYFOzJ6B5rm(w5?N{= zdt&5qOP;~Zc#QI@>)ii9i`2NZE`vcThS}B|ORgRppInVAz-(X?n&%$P$2Fxf9&gh; zTQv7Sv^f*hvTG%E2ArvnEMx0q?6^0E6~EKeXL{NlO@>;hy8l6W*sHDimf^!Zo22*D zBt7|XlAiqDV4@zpb1+d4$_)K-+V#VxdcRqd&QiT^97v9TO~#|=DU^(=ST2(#Df_m# zgCF_Iq)O3&Wivf$t)gWrT8fNois@8IN8*Nc&V6(d!94 zG!=R|yAXT%GWPNZv0mZ=U^H};r(_OEqBZB2LqgzY%IT;!pk4ZS`!IUi)RY#^A^bJLAj_| zMz9qYbT5N2D*E`Fgy3bAR)9so!Jtr~q0QSAY+jWvcG>K{%QT8nc`6KFA$Hk8e&0-E z3B$jg7z| zGUhNTHPFa@hT!K;_Hz^be2e|8f}gLlpX=b~^Xw<)-aLyx@x!VpSK%~ezfxA0arklW z1@h5uDa&JRFc03t4u)vk)_7tBv$rbP(RJ+T{rqSN+iu-}i{FLZ_nyy>QGi7mKbRFe z7&5No2fOUn!7Ag57uZfIr4@1gh}lTm%1sK(UzMCSl2G1CD{s2W`1pDDAS(5tu`EuE z;CPoGUmttHYGd2;`~^Rv7d&h?yMDyjn0V@L3`-(XY4bMMQM!|jX4drZG0eUV-jA0( zF_r=kwz8aB-)c!)3=ex?vq$}xqORPaZN;mcTiGM;No7tZ0%+B4qv-i`kCwvn6h-(W zcelz{qjNKO2xN0L;vaOl(S@psD+6E&|rvK_pt^;)cQGVFhs3KSc4&I-OCyb zQLBPA%x&~)?;^EDTfA9RYl>?1Rf9z}w3QB;&ZXy0_1=u6o&&bLSlO4aGCH8ZmghW7($lN}v7ejQcwdfdmebB;7nEg$d$G_f3ZGsPE z`5u@hZ|QR~YgxX?pZaM2KgjY(p5^!9bjY$u*5Y}BDP*}ME2xbl#WG61i)~!-si*Q_>~dPxa1$eZKVXbambrb0@|#o zK};|;mftlPwFI0FDFyD;#UT$5{%>9ObwzE$DX}2CmM11jD(_j_TPz8Nlex5)1{Ear z5N!~+Ly|U82>zZr(+px?RPE6`U|3^fKsSakuf`WFP_)IM2%0VU){*V~WK z&W5&f8_wYeAC7@%{Fq7XS1C2;8wnX8zA7Ttyw#jwh*U_uyeaa9C?<9FqlVju zuNn4&*|1+ z(d`3|+CwR9*0l;kG7S60A9xVnolxosoS{)xYuLtzO^Q28=&bJ5GAQT3pX@xYypP7G z&V%l2w4;HyWNkE7MAIg*&qT~fMRWL%+tFKs6DZD6M`4aSAYpQ2*mHHSqV7>t(5f)V z&pd@snFq!sA4B&hMWa&+n?}(?qg?`z$B?KcboEMnC*?72r5tsv#}=GTcTY7&Qn!L} zPr$89cnB6S#12j}PCVVq1%BmfVl3dBsc6UC!%l$8^R1(Ano2w+DB4K)0{jY00q8z} zkAqn>GmNZhea}6d7s5LKBrqr7Tkc8x7Gmg2=!JWXU*e_Kmsi0%>2rOn>FXvLx3Uhc zlekJy`{JoG6~4*%0$k!JJHsB?(M`hm>lq}ByYomGeP01#L}WUnc}*|IS6ldD$Nt6_ zOx51mlKzkbEqv~}M|a|h9;v9;A1)0?QqM{MoealRH;r!>Oyk=E)AIS>32dd+XvN`1 zy=93YQ@NW{y!UOk`<6*{s3Bpcb}bzF3LLpXsw-p9BmDd$ixx*7gI?l#+Qf~5gn;vDJI)Uteh7_3Vtc!HmJr^(QmD5+%=@y8_iF)6-u!Qyi@RmL@I zc)ifbQs9azV;pbiv-&*6q4vF>U5~um@|WZ#`b!XeIp!~U{dMjy*&zgeW#ceQGtq>> zXlS+0IJSnqZWRVNQ>otgJp!;H(NoIy3Ch_%0lcbM)O`84;f{4WwK*l0Yy2N9SkHG0 z)jypx7;V4F^y`fpb16=|a&lbxAO6biMRGrQYh%Hg!{IlPAL75r4>3O84^db}F2;oT zKa9WUzK3_n_wcJXV!nsN7xz6>Js$6S2pOB+Am2ksbH9_|duV?mHa_u+GH|tH-vme37|-G2<#&{Z?F@1YX4-n2G@G<$s*m>v0xmf1?~9W-kOk zrf={+ifyk;jd`uj_ZXCbLUsC_yhVLO|@cJrhzlv?d0l>*%Wvp7J>T_TOj&TZj zL9Idc#rMl5ucT}O?g+OTKUl?hfY~O?@?2V3J|U%P=`?9UL&<6J0yKoo}^EA zh`7vg_DcH0d8`moPpR;WI?>T91kiHy_l**sq;Y^{5JTml#Za{{y&8HA)6pu2_!!ky;H|XQ1(nd#`wb3z#!KFC+ zmL3=FheYQ=1SUOHp`{%Li=;wLJK$E`iN&QOKjZNrqW;hzPFr<$bDo%6>p*-JbuI$q zhV1Y~zLV|$Ou)NF6j!BkdOlUhf`y$Q@qA#NtQ1wx(7|5%?7CRAkI3p`$@-62U93*r zGlLeZi-oOJXtBCj*h+&ItBZxL5zu0Fv9Of^Emjw+6Mj)$tOdi>#TswbphKjAY97@z zv?kW!VQOL>q?%ZS-QPfn*I+0XH?(iC^xKenmK0wQH$}9d3y~1iEAPz5j5+eJ?DNUc zxy%)1n0G?@4n(d@)=Kkc=(#t6ku!-DN7^`oSplK?<|q`ZYf-4C#R=6XucX_I3Cnw< zvHYLyrBU1~abS`k^0@yB@(69khGegWGlC!MfB+jlU7JobJK>O(PIMQ4f-yO;4pM`Z zI;aOr39BfWPln?T=0o88qOO8`+z%AHff?+L@CCA(_JOcI ztYk(AiVWHfSO`maqgjTzSiZVRarS%EcYQB{|`q99r*IBvpH}!M|>@n7NHAr z;C0>-#Hi9A9h92RpbX*@6}@n(5d4$P>g3FV?3M~Wtx+kkd4LxYcZ)`pf!U-Pzu%I9 z&SB)Op?H02H70(j-C-kDs&idVK%ls?1lu&SejpBiI7|lX3keLeEF3vJ!1M%dnXAND zj}Brs0qZb!7L52w4Et&Vhk!y9vPslK;$OsrVth<^P=^$K3c&-JTejI57sUa}V2RfI zSM2xLk70qA;0Xkn>?7wx;^457pYg+^j-}$gAT_n6WZU04mTdd{3=^rK%hww<_JE~V zm55UdM%-*kdm0ly5X)gH8MZB;?n)K?fhwGK#?#K`qEd459+uU^#@8OAiKq}P7b{`~ zjnOZ=Rc~B5F*9^zb{yg@#S5M`@*eJuwl`Vxr<#qT{t6lkJ|^l2onfT>cRJl~1$`~H zaCQ-2ID^WhosZ7G2WEIMjAu-wl-3i*_xqz!vo0#f*Wk*@qwc_XUeHey$PO*dmk?TI zH+8f(lSh8|d+lK8F27FJq&$yaI#E;)7{~7%L5PT(@Paiv@b={V9cPPohN8c`g_3lq zWSU`D2a$+%(+8t0zDrS0(l$xAQFaLTMZC9D^9(U~=w(=VY+7R0e^8ie;}7*+#xi~3 z;p6D)pX1ekhG!Bh;A=YG0K4~qk@@4^g!+#4yf0MUWpU(Wg8i0M&)om^Bo-`j9x0@@ zXm{jDmF1;48NNs~yPN8rqbC?6IVr-oM%Hsf3H4YHz+AW+_f9~~?531d@|ROR)=MNl z7v^?gJywa&h15!w_*_V>G>OlJ)EXi2xsX~J5}ym9MRVb9oC|j=G59WCgPYHV2tB4x ziPdA}b78~q^&wdR$vz5l?f~A#h_h9|?>pJd)$F@H#{4&7yI9tOlZZZ^QpScrZA`ab zIS^yU1l(uj)J@Ak4tDZ9BDC9$RZDRfJO0t$7g$^+$caSjm@<2k-}~U7Fh4dKR~oL* zyVPMxQlI3wJ+&8oiEWLg=iu=SI--i=g>oN}(&j!058sg#g<7o?ys>%LO{Go|Dyg+q0@nGO``wqRMk9l}7BKe5%WjLqN)e%&j3H5xhxP>Nxlc_Wb9Lj}BAcB~=XCDOpZxLpF z2M4azCc>K2DCvTOYjz-^6Y4Ww%p9Y*?k?G1E1A(3eW*(F@LuO}--n7ucf;f8 z;Tk<$ctf|y?b5d6XlmOV>E$2tZm{ScU_AdBao;g@Jw8O%=N3BL>IKPvtVb%^2i&&> zj@pk%>c-9RL){+gLnb*EKAEmKI|ctY$&T9CN;JsbrwGp-6a3{A`t^|z7?FzJ^{9V4 zcEwJ8sUyZBZTsYL7$4R(7h?`Z)A32|BSdH8^6Omc@uC*>Y&Zh1I=gJFtYtSy!gFVr zjZ&LjMZKcBf8PoJUOQG(alW_g-C#(iM+rgPTomEaQlcNfRW6e z&Ay$8I^uns?I4~yOTyo#O96oQx0uLs)fk*uiYkHqEo6A>6i zT$rgx)G)Fj*6KCxT|)Z}GGeP^^|;xJHftgnn|V0H6WLuv_8uTR=9VjZ+D=CHPD!7) zMFzTKv24O9QGh@Q(Y-uh26`$8jDYxHxOu_J{ya!x1I%YK(Ywcv*DOCMX&#{9kE2vY z(zngmfbJP@4JG+lheN*C&`;xi32QLK4%hpBgAN<-dN|w|JKX3CuxRH;h+3Z3*eRga zcZfzkK9bX)9nn-HANP6Mr{u+v8Dw808l5kZ1iGFo@od4@SwvH?P$$9-#}^nMp3 zHM)k4?r--FSDym$55UF;`zjAZl-9P9VYId90^7U(F-CEqQKxU;gjh$+V#C$AzOiF~ zu!tBGQ`GonCmLp+Qdyrad6+3Isf5Qz2u+RfL}Mwx2E8nZ7-U|?5?(eMxX{EnvS0^o z8ZTaLl=3?~`Y)@pL!RY^M{(=ERe+$6fV0xo@yv zY-P}Gy%E-hhmCKYP3VWL`({%PR=43ErX%PXHr{#dA}3zI*oj}ihMm~nV|Aj_>O>cg z-NBCd8*#E{-(WOtxyXra=!4gY>M%((GV*%p+4LHZ{x*Y*!TVtGREw{<^tB|KbIZcG zEAjn&z}VXi&lZb<>;;4DGS-!Y#@Ffm_6~lYyyEoz{NWq_l0X&fGmOARPQWMN!;EkL zh33tkPX5-lXNKn%I{C<}WND+w4Tq;i0{8POGUbD|IJwhb zm_5J*gRwpc3$T7YQNPZ}Ju^fC$z>BSfgPtg3+NkBpyRo}iv(#S4yn;_n)M#uk790d zbsvfoNhlj2A{06)P{lm|%J84uMJ|i21X`A}B}q;g*}IMISCdYdggpGJsWaBM#pw)f zE7KPHiP59R*we|p#fOd_v%QV=lSKd2C&W;#$>SR3B6)vuS4>WtGyI8eao$rGd0hX?Lr+jA z?|=?O_FYKMSHyxJWHQ8|!1bd`FC8}lcS)+AN7m__kxZC^4Z>3oW`Ru&^Qis=ldCD8 z(pHrk%tz~3tuF5J=xLMLx;yIz$zSb2l%n$l+uGeCUELyu2dCo9h_)LXP`5hBtlpYO z_jgEkL~9$t$REsM4_y-fJjvNN%=0>|=h^6a)E`BEIiJ?ken!Rh^;C6c;{-al^+$M4 zJp<|>hhzW++_=1*Zl^a_&cKO=-_Fl>og}WGz|VuZ9*TcOr=)H-=q-m)3BWI72i`%K zVnoUeV8rz~?3?qM-vDbr0{2dNiNDf(mN~9|^8s)(<@BE9Cmwk*=|m?x0fNdtar+k3 zLUUd;`x?CGg4I`ee-6DLx?CrdYHP#pUdBEUyUqyEWp$_x z@soT!QP^uEu|A?$Xu%20X~L+~ZEXFsc{jZi<+m5h8VgS?$^@gg*A9~szSO%&$=dFK zAO7AH?=1M8L%&nKW$-(Xey4eFhTr-0dxZBI_&tGsr+cr0-xKL~hWAqVJ&ArhyczJj zl)h*|RPNp{U%g)rN2kIsAR+4w`y`#aIhaIk3{l@XJ4N|?VjMDtp#vXdU_}6CK$*Xs zD2z7^Vy`@ly^dK|BO`VYcw0zfQ_VJ{ZqSG);OQ6v+jt2j@8|mk-XUsOrS$Y@_Kx&_g*P$eGh_(A%W*X!!}5bRIdaL2WzBxCqpBLHAZ-^gGL@} zq#R&y&{S`#aHi=!hKWmmlh;O%WU-JN&f8)@+XXjUns>k zA=o+siT^SIq2lM6-_{vF4rlmK7=4?Q2Y|T{4Do~|0fd4Ycb?C~B&lDBUDh_p=7-*d ztM!*B_L;Udo-&N%!^-R#WAAZt-%xL9oKJJnZ)IKfs2#@N{}?a69hOW`mw)>iCtiDFj;JLfjN|Fc%&QOfhW>;>3dkfL+$o{RncA`N2l%@fc379PxKUI zUx0ZW-zFDKQZo0->Lgiz6z=Pj^+$5B(2(rxXMq|$LSRA~t*6t4C%=e3Ys?&5%sXtO zoffr?WEPeEWb89XK-1tO#%>l$ty$@*J#*11>Ke}V^C)hium7NDr;s22h`c9Wv% zc1}^;`HE4L>>;YP!zkL2K+zlvV}5J`eC3$5u`H=%_lo_2oni(fB->IL86iscNs-x1c8AW=!O|CVo+4PMP1k1D$D>XS7%1)Z5!E7 zQFND2)MaH?U0GK&qToydNeBqJAQz+Dg&syBT+EGR=0E3D^<0v{{r>-XexFCuJzZT@ zU3KczIj7Ef52MYOdr}t%`FfG@G^;{VG8*xCH?y5;hUat9(DSWm??p9X4<2lM0!l}? z+OjS5eIouJ+`rhCc;65TwI<&`8BNNkvMtml68BGlkFF;cLlbYQ0hWV<5GzlP+uP(p z!)oL~*?XDeHsRh&gsVBqz#HW2(5Pn{C(jkK(&kilx&{r|0d?$jW}T9iVHK342l2l{ z?;Cd5cKz5EZy&VIov^OA%Y)qCn53)-R8VPDaG27AkX@63a!BPg)-?L%ezS3@Q04H4 zBx){}b8(!@8Hv(#=u&N2 zRMNVBKz`OYwjLw9^JSqN(?%DXu8VF*n_Mb<8ty-(&Ku)z5SLHK)IdtDOIwKp0vd#j z?&MXn_VTx=#29$1QWsbb0Srq)J!32hEdDBSSuV!tsVd3NMSE=4X9kG=tI`Z}c0YWo zJ-mdHDm|08^e{}>74RA4AOnIw1}?pbe!bk3y@adLPDIbCqFgwunR7~1AA6+Te#{i) z6Z*3U@WFK0YE|}t2C8&FQ%TS}$zSL0+kUG*h2e9f4&c)U8c+A=_O)X@3b_vHy$&Qb zE^8i%YH8Fj{D4O=RzC`q0#w^^+;uh{@qTkFc+-agO!s4M;#2>E^{b<}~ zaNq~o<5ls;t3-b#dz>Nq0_^dc_~SLA|AUBOn}3HrPPG4v{+A<$ZTfr=*!(W!aZFh;Vg&*yogX7LqD z#Z%NsX`uZLl4}2D2~W%v0||bd&U3mh??;Pt;8|Lv`r#bzH??_cvk^}Yx(9xuHVya{ z8ConLiuv|3LqdI=|8yCS{dZae*#8Mp^mz3*iOY770h`RvG1Z(5HFwt~i|-)Slg^RL zk}0F-&2HwRw=Yo%ILnr8MDJJEZgtU`dw35MZzm?y<6C5k*Mo~kLjVhX<5a}?cRvKI zzKU9YRW~us4#PZm>Kpg9Q3xwN6_-6s9+4OVvUH2NqaT!d$Cyx$g#8T0A3Ygq20WKA z=$fqphM7%qj6i z3ZXLYk8If%{(^wPKr3VO@IO;9nuo4K-O$+S--hGnkvmR39#Ka0f&2N)8*TnC^A_au zs~?`6(C*~KXWP&V@9e+`kx2OMi>>eJ42@(0%PaT>xz*lGIXus##0lCZnBq7=y+N46 z`&f2ChD%{&7u?J4I8M-RcE@pos@WaK33{2`4Yhq`-~<&CPEafXFT)8M($7epkg=1y zdD+A;C7{G)Lp*iDPaXWcFysb-f{xP=euC`d4ulBuBd_w;iOatusce+3psn7o^UDO$hAl1+GBFTRte+v zj>vs?x-4O*<*YO7JJElb=(a_wKE-p(LYaZhn4ZRRK^nLV_vz}eb70UZ9F&Bek~b`P zG^f%89}G;m@wikFM5pw=wG2sQpXjHM2nv0wPxR|a^cVVVFJgYbIZDj$a6K@;Yj7h7 zPHLufasEG<7@W_skQZ$uUs1H|9J-{#`$y@LB5P;SCB@s`Mwb*rJAp3AX<;Ng&|jNN zZGLuy+B^tbq?2{2hfNu0pRzdDip z`-r}0l3=oWCc$L&N`lGilLV91m)&vl?`ObdeaXl_$jM)dJ31oy=V!&pe^&?c-)4~i z(G+fuhlu3@>#U2P0EvsO31UsyFfrRI>f1xgoUww2aQjaIn6l)p#s^Ppe`#+(T2qXK_2(&la7TI*fU&HrnGI*56#1 zX$n~)3CLBfjulWxs(d?xrJ}KBbRD57y^w|hJm)3#dSG_6I#O>AH>4$R4tJ*|ZVt23 z5;uqY(h@g^2htKZhxD|>&7o5o-yCAABPEF3kQj*E$erzdfGg&45V>%Q#i7&UI6#Y| zdm57W<3Qd&i~fUbrHIR~g^Pwv*cz@B;t+jCTq4>D?z)vgEfpB6Q}~kgkm$|8K_gU% zYyqKe0(Qkwj42@(zdj9z4D3t%ysTLNij)3ag;wp+F=qfXLItsR6 zIGh2XHYqEkMelwKty=dmSkH@o@32I}T^?Cs^Og(ft9J(?`jCgA;*H-EFMj4E@M4St(Wg0@qbpcy?{FXqPwsqd{aw*+9b6JgUI#7h z*1^g6J*~6v+LG5CuKwNL9QT+0f)zShM}-z+ zp|x0Oshzf-x7g$D?C~r37NAbb2gd;B4LyZ|2~GLk;dp0L%ikGTvwm*lVCYM=${ zH!k22rlq3q3)78m1adIi*JJ<`3Vx}0EdW{VawD)4#-(2=Hy+fdLILFftX7xO8jM_FIahGbu8quth3oQ1Mc+}{{Eo}sXlEw^3N-E! zUu>f_??_5Ay_KgjJZxgedmJ@Ow(gb1@p~OS&k$x~Ru0L7E|CX0Otyaun7>+anB;=J zPUYbM%QLi?@(cygCywPAI*oaTcvL0knCw5=rDgQ61f0T2G#SEpv8?xu<6b0@p=52w z{F3`!)`RXIlQjD`rb&Z_RXPU^kh9J^l>shi~hn0o9=M0NHjDh z7QbNRIi$}yf_a+dLAP(EZ0`#GWeJfzhr;XP@y7=AfIfOIk?VMMx*0^2MR!SRmn7-L zV@1hCO z(S~Wn{s_Qq)_TQNj0G)JP<0hUS!z`Dy-lSs!`x41Tl5A>E!M-Kgd8Ar7J&Lb+*VMb z_q{wCz4tyW1}j~W%q;%YT22J*et`MMS?ct6u8;C0XJKa?pO*vt+Bm{5&UAaQWTM9+ zLg1;uf!1Fd!ZRkT%^VwTiP&`&}g zt&2H?TT>C595vS0<{a803IqC;V`3ceFAN~22YvtW@-b#Cq*r%k&c>mf1bLTDWk7%J z8G}n#U=^bO&(NRHlxRG!K%<`1gYxQKk$9dW7q#ibZ(!sT`U;*HbETBSywaS4D23DY z!sw)VMb}tuSXR3^oi)0{oFY!tRO~m*Bh4vH=ppzrm{6Ea#9dj{Ga9{XCSlvc!d%jc zHt?4Z(guF$5NzPMbo3WS(*FJcd;BDOyc-|iMfU%ZWdEP6AP@>lq?>Tg*-+ z=*g44&1O#z0-8P$dMP{RPwi4F@E1?F2#ErEe=8}laecn%4l}UxrzL-bdwL-X<1Ahh zmDkZ|HM+ZN3@H9;%rdAP)Seb_C~g}?%&)gpgt-TM!U9$%L0|Gq400pBaxuJVcGaGr zu}Dk1Iv1EwgSZ^=Pt>u)WM#LcY^We>=LG@~xs3os1_%Tof?yxRBRgKHqRXHy(d&n?VS2onLHqyP1{QI4Kf7GlBACv>wIiNbe<@DKyPcd$cUkU!Km z;`aO@?!}1of@mZ*#hK4=QGfQT*?#q1L$Ls9ELa*J=6!|Z5XZ!6&ZkY zx5|ijOc;OYz5i<9CjP{4W53^eCH#)3CS$c>8(8~MB>ut?Y&ibec9G$D=o)x&7+dk* z@6SnvZZhquOdtrLT_H3o8Q&y%f6wFlJ2TFyvvXwSTg0?^$An8|+5pm{ce#`;2>!k3 z68nhhKqdKn4J>$^HcI9ehuKFmpA1qnZuW+&seA>0*t4-)jE5taf)PO$jhNFzy7jBb)URt1%87Gn+mWA^!rFX}X*c zU;ot-uWTw>a&{)zpRAQRsc>g1AI~&}1~E##mjb446f0@-nF~7RMhr01kKfpwnm#c& zULux_32`=n1EqElp7jjU)80N!m7KNJX03F0mOx?bGNjpscB@O*)2K? zzx{hW*1|mmy>PLm*GeSsCMP?AvG>uo+&V>Ej%h2({fOU*U-+e&HjXtB9;6-f_gG1g4JxPZ2I_gW zZf{{r`$M7&KRdf8(kBmD>NB3A_2=b$#2Z)QM``0Cc4h6~sMZZsPF+itYmb>F?Vgex zr$Vp0a8Q1b)YVvPc*r9hmu6m*3*=3*tYLV-m(3)8TqAqS%`WR{(f2pL6WsO)-vK&% zD&zba51-W41P9`;b6e4@vMnEZ!Si`~AfMS;q{lj&!k6=NZD6<8!fwT|K^L^4o8mFj z3sz&F)p4*nxSvDQRTQJJ8i|Lc-i6Bg4Kr4h?^$}B&UIi$UJ^q$1@)L3m|^6_ zViPk4CuTrwV#tK|SR@gj0-G>rf+&JOBC9(tF2p0@Db_0}Sc(EVg6@k3I zgI8q#djqe?(x>4S*#Nz*=fJ6N)c;ArW-VK|-Qlkj{pT1}*%hK? zg<|j=%5hG&oVH`~ra`-g6LV_=TOSXRk9L{hutpZusZ_~`)vwX`P`#ptP)pSplGmD4 zRLbMU)z`MtDfx%IzFeeKYee+*!%uJAi|4LFB=nK7KSI~t3t*r;Pr((eG|}Het6gKY zvqg{5X0*p4Q|s}uk$ft&vDN6Es2wp;x1b9}rIPw7>qWzrOjY+k3qnC?S!=?1j`x%h zx+cz1;!J~mh+SDtT=Cd1Oe&!}(SIA(Go}@L0()y6&w`EuZ1{zXv8Z=`VMCD>TN!CF zf~CdwN*ghmP$mQA8`wm`KuwT@fiG%67}$fF=>F?SQB7ly|HB?f@G+vo>l@+mMet%; z2}a{vkZH%LuwRxt0<>hy9Tog<6{!e;MFo;-aTZm{!6WGty$gCbnLJZu;cU1o#^ate z%iibUms^3!cIr=&2GJMpZYW7}KkGdrh<=331o>}FA}$bY$evP3 z*=MQ2*QLgr@T>}GpqLmHx{WKYbu2L0RjfbqJ7TI2VyHClr9?&m$9fR9)=5epd;^W4 zC(8Fjbzwkp6zka!gN9PLTXhs;S9~5)(Hil|*@xhXqd2K!Ab)6G9|+(+s%Nx2cDwRH z^0Okb^5XgGWMv0@iHkCmV}25}dM99RqV2;_nQ^Rcy+5WRPm-+r+&3C%33)xX;6|X2DB@~n zz&*!MG4tA~nC1G7LqNq;0;FX+aklL68ccVw>ev$3GF|WwJPw4$$MjzP!nF)P@dx58 zalFedEUyR5?>F01s)J+ikEw$XMq=OH5l^N*wy?ECMwVeQB6m)EaRq zzwo>Uu=mWkH#W`P&=O)z@FzZ&D&c&)3^WP*E6+)qe>pqs`x+a=AudM@^teA9`b$#)<&mEmySj^%z6l^gL)p1h zb04Xw0)46^?}AJ4q#t2ZB%wOo9oCOGWR|?o;;$BgG*rw}H-RTaV^$p6Ly;*E?k;Sd zvYMSw+h@t#d9ZZ`aqFD(q=9Eb`_k?Avfk{7P-av!xg?-)Nm!W4E(yrZ(D2_QF<+MG z|Fl`mHwk;4-Rju^`wlW~>ysQrU-ZqQT>%pbQ>x7m`WptHEupb-I>A}a zX={syMn`xa)`R-}_b~fQO9Gx9XK$zV8{Q>7(s!YaTR0f~;;IhS#p|^xh33$GI0f*4 znSVlGCD?QDx#+tdw5RQ(&Fkd>2y@UcWm=zc(Y=9pQuFIz;=KU7WawnJye; zxj?i&vRpWZbx4tk{I~m;DeV>S-oxz`=45+Ch)DK0lI-v9A*T77O`SqE3Z!gx9GNIy zHnlfVkXt`t=G_T%kRn+pjlX@wb7s+BPlka;{lH*Gg>%X8cv7CE_H=59-b_&c4y9Rt zq6L@Syz~Xh7N;r=JH)+L z{knf~l%Tc^EAmr#l(lFfLW1Fz7}5WDC-TS>qVIn>UmP`)@uMsk666Zb562OiT*=r6 z=rNW6f(T8tsduntUqehtOg`!xk1IZVMf{xaMmpzf&wJQOUn*m>v7YW=bA{PT>0FY; z{BfH7I+RLM*qbY_X6Gte1*sJyaIJdVGDP1t5-;0eIaRSMaq+#JQer0OZ}spahP~Nk zZH)P&e8ud|glQ4f}|3LZeYM z)P8tXQoEORX!pzR@csqtyfN~k508gKcSIPzoR*PgoQa^di$y8bd!9xJ$a#+;?x3tx zxhF^%0y>Rg7}p}YzUL~kV4fiC#oqT26e+$-2P293vkw#hDAON&7^Iu+lCno{5XpRFC6d-_8WeGA!^Tp-#LsUaBh7jJ zu2*23U=3X+c}vrr)^na?9k?X))8UVLZm`r~N~zaMc#2K;zM%p5!Ceg-YK|oULdePv z{1vxXv~k|ho$PMpT`OrevrU^9jTzkP#R+BJNN4e-N_Ywb(PbJlxDirkmubx4_6xgV z2DcyB4KuiHWjD;=_7S^b2Dg>$#>jJ5=Dt^Hv=x+N3Z3$VRCmy|4!Q2Q#f|5>3#86< z7m%__;<@f}(X#@ENBpdY9Exz?K&N+>D=zHgx!XMS6B#NVqf4@5&>qc_!QD=mWVMj# zGCjuofqwiA9D@1w>0e^VmAkX04#?Kiix6YW^p|3g2eF{|Vgn8S z1hY%aj#)dM)(Tmi5R|>mF#d#htyC}+>nXrII*~K>lpN-@H-X*dF|WM>b~k`|?R8;y zgPGUf&ouB;hBB|cUF>c++2kV@suibCRV@ zvXn|$aIsGcHjS1B27xa$)`m87xeF()HG&~2#-!|CMqq&hGsA0BlF){Vx)ZqZ+Z*cV!H0P46ZDpCe-snZyA z3RHH&oMw}&6y>ik7JaW|qCZ$MK6YyBmJs&<&3Pw9d^pzWm)@9R;_{D(m+He6peU`Q zaGNr{by!9mH;5;{??@RJ-`vZ_BTmPWh80X@*J7|*2s4_+*T7ke{xu@`tT zGcPtfb+Nm7u^e7((oOM?Y2jrWuVH1n^D@`a_Z;UPJ*dx1@lGj!C!N1@?#7hOf8UQ4 z%jCs&q%O9B7t7|w)}}7DoEOXE#TMPzv17mC9qY@>+{NgF59dMspDEsHxSGAwiNAAo z>gGk>{3X0tbXdw_`lYN`E-$ukSjQH9yv1I;%%{UxFEe>BGg9<&Du1Ule`iVR=KsK( z@5YPGOkHd&FV>$IyFNuP4_w8{T*k|ESo4?g<`1TL=UVGF*?R z#B@oGR`E6SgQ3+gS^ZwLH{5&@kolt9AyhpCkGgI{EHt3Jd+Zy%! z4Qge{?&5WO`moMn6;P-6S&sLJsDEwgSWTBf#N@X;+0$fjc>4X^l~@DrQwQ|UL+Bc9 zqr*3wO^KRPX!}$UrL|RTJ1Sm6(au{#Pf%QV*U98Hb>ozeNpD>G73huV&#YhF$2h-s zB22I3Jr=Mo+1>5opy?{a)~M)G=nokIjOh?++;czlQKsVKg0a zpW#Q`7+E7zDwjK=9nZF(7unhNWF~6ca`0$+G845WcEdz%3%g;Wb`HB?qIR=LH%!#N zoIQ!1ZQox|!X#~NV#KD@@a*={G;LUx`&NFieE_6812;6`e7l03Zwrmd0UC1FHoJ8X z9&eYsx@4ER&_c7RD-8)wbjsuHepKg0kB{4| zWj4`OW)}`=xg|?m0=*DX8gY^m^geO<2G|(1=f)^2*AwtnR8MpAkOBV+Bo^PhBA%)o z3aG4vH!ubs;WxAKDk@=xmk?!nvgyGA4B_Z$ zT#0{Q$!t}B?}BEu3RPO3do&XiBDQppOXhPgE z537BYI#{g_`Ht1>?1EUuG#jG-Rsy9$b@m2xj3f|O8*9*>9x4^lc-XAvi~7>)(?*tJ zq>d4#qrcdr8RmqB3IlqU|1O8d2)`N2W66na@>NFQMca8TISExR8Yg1n_^`W^V9 z_d@qO?-KpbGyPhde$js1#`{qnNZk*rGJ{n)5Vp;`J)fwwAvzVBpN+o<1>3IO4}9lP z<4O1@*mOm8OF@a2@jrIuNI{9XtTdYiODsE`9ou11#0q6{_Y6w_Jt(?LYQ`@3U8>c+ zhXi#sw&Aa_Yef;!hfEu)&_oWp{jg61<%*Tbt*0g}0li7s(Z5Xp=5@hO!CkW4TQd25Gpayya$r?x)#A@;LjEveQpg!-RE2irI^G$}yUw7BcXTRbVSZWfeKBC{Y z58;Ju$`;t(_R*#}H-i>GAPy>msN)2bM?3jrG?YpC%yN8P!~PM~SY+K42OTQAj~^d1oeENrO} z0^U&C{M02%pajN?)p$ec^LxjNWCI;RkyFZ9qmZRefHE0SO$D9nmSZ*j15$0+dsHy^ zacyk)Q32wNXGr};c~bX_KWaFA(MNR`e-wy6N+71`NmSAQs|=(<+nu(VqVfu!OKfh7 z?-lB%r)l%Eqk-!?{p&)EzuAPc2MYyNzJn+FI{2B%9)kD^FhNe}mO8ukD9oJxW+OfH z*UX=((s}@uBtQj_@ph_+hn z%109M>sSIsyS2$CPACW6qb~?X4hyi+yCTX;TEJz3vSAF0Le*iLw^SIX<@QtdDl1nZ z6Dm<)iG54wc8Y&0$y*tM(j?SNOkh|1PTFj2IhaW~6gvmED^-?#(kjA}X{?e4R-y41 zBpPu!I>JlZ3>2CWIZ+z1fnIyHWQ=*Q=sQD^+flRl;)yi;w4dEs7%VlYx0KbLF&|K# z0%TP6-^ms__KHv-HT(hLWsitNyD0k6(U3xq41Tk0#RZbBCARgAp`j-QL?tx> zJv%mFUK(_@K|ekm;({_Lvx2rQ2?Cb2RqYtm%+um z`IA_1Xj+S5|Hos7#r1|CZdHBEe`9k_Oz%T!)lw4@sMi~4RN>=J1`R({>>|V2aj2l< zD5R=}?m9!o6E~TgTPOyMuxsq1{ZQ!jX82OtTZh6`=pK)V)VTF`bK( z3TG`S!+*YmahJX1R{IhA6k@I<%-it@;k5qesS(t})5YaK5tQp%T(^mWdHC{2R*;0q zUI132&w_XC)^noocv?qNJ!$j{Dj!B&@B~8BaXbPE`NMOAvB+Z3-4>q;U4Jenm*5_m zLcV|Cw1^F?1{JCnT)g3)0Gcp{b1|F@*lrhAa7_Rm9Tc zjQCNKA2PY$dMn8bIE5yzF^T@JX~tfGYkvk~$@s{wt|l&_(hI=+L1nF!|kE>Zl$i8G%Vgp z88I+Y@#8vi`CopOA0t+o&3gg(@c~nNevCNLqHmKa#*8fnJ5FZC`35u2O=QMBkQw(t zW^4nU^`PCVISoEMGKmidfDcFPBe>1&hG>j*Rlk&is#h{oLPQ~(fL^zaP1ps@OXgq2 z=4RQqzn}xd*3fMPxxEs{IIO`kT+w0u)x!E?xt#q>1tL}v`tdZ%5upX=2>={;<)Ad+ zmCrPhhK&OlDsQCT;Dp2D419cuXqyqR*7e`p$@u#2N;1AqdOyds@kv<5`acBn7ycji z*vlUG$H$loPyZ7> zuVIh#*yBIE36GmGcheQ@v6($?evKYKhmY^Tw@s`k_&MHlB+YZqnz|LI>_wY%-V_}6aM|Eqs(-Oc|O|JrX&?fh$x-TZ&_uluTW+ zdqIoC+T`vo6;;}WlazF@sD|K%4LoHe?GtQ;c$-YhIJ3BXHf<_haJaBl?}F~=D!Yex z`0d^6!@UhT&U?}+dxrq>;Or77t3d;-e6an@jduNv+NC-ok~60TdL)(fLwdrrP$0a#bWce zL}6d{R2ko>I(88FQ{O1D$sbyqqU08B7xYjk_zgZ9``8|mApwuLYu}aRA8BgmA9>~V z1QIqlmb(?-b13`dqS`4iR85j_R&K0<57;ZiK5F3n!Zh@dmF(^~ewd`>bm*xF^@0T; zh5kt41d0A5{)TwSh9^%dFC|>&1VA3fS0UZWf@+twN>VC42P9=X=@6jfHJcVqg(~HuAa}@2L;p)js0w#W#9cWp z_+gW5_Jj>bT*KWlbXknQIF!leP`4<`T$9c21EKT|i0ZOU3NoNf5LcwnjJ)g|f1}eTrTf zDqx5<1(wN^xcs$eqms2r^j?K7!9+?0L3=@sQ@#B_0lT~sZLjFLA#B0dFODX0b(XaU(&j<|$+r#5mA zFcX(>+)~5A6{l!JvPd-SgCIv3PmU*%@bat*hC%;`c&_phu{M^-y*CPsPV27O+nhq! zX$`ptVc#Tm!2vU+Vah!q7X+l*6VNrO_N0{6GD*qZGEu$#K$A=P0sC99Rnq#iM2Ep2 z)1k~H<@O;>lE2PlZx>t7_Vt3GQQoNyp$T(=5WGtg4uArqe{`WK8ZJS+-iFY?rl#l$ ztYeZkatI7$yefE@0T0PesFdOuhN_J?U|1E6-^p74-IxjE3fR+8)hF50sb=qyYjylS zx%RLtt3pEPAbV|-=ey8vjClPRo~DiB#LT!CV}4b1Q7f7YJmG@AznKWKSIkv3(Vv#g z4QOkyqdGQ1zX#(8VWs3KAWGqiF*cspO&vEL&oh!xY2Y={egnVFqByhUy(SLL)V_V- z-!or9{{6r0`S*iE7|0{%-+VsUevnid{N#Qk9O@Wv&-~keA^CHQwDhNC$SIyP`=g6) zL@*s-r7`X&`p2P72{>R!T+9MYM{yeVPxToyF#DpiMU$LGJ0-RIWG#bZ9#u?aJoEt> zYM}8z1}qaFb$L(0cwjs~0%;e4*0`Vvs`b4kFyBwKOuX5NMnZT4M*>}T6A>@qXza zc;96L@ zm`B8yNd)rX%fR&;MSPnE&+{Cd!~LKE%wY+Hua4mk&LO!1x!J})Cd@`Q}1js{tWgi9#^)vh-5K7-sp|7rJi>i)_aDcq-;RF795>3#q3h$xO#L5dwFlwRj zQ`m^~73Y~RK6ZQ>MRkog!=^O^GuOwKAqIP13Vc4C!|B#ohn;90VoZYS_!PHSbz#8R zz_Dp>V2o0tI6l=25}G+l{{Dplbu6~@TOg6-mh!NrZWO1L+z!5^j5U{!Wi~l=1|gV& z_GB#+U%oP(6r_N+Buy$ht4qH`Vhuh^4LSlS?cj->O`9fItHf6>%cNee!BnyFgM3Sa z9m7o3zOpv5)Tzv>&_6uegyXlyuHx^rDzGl7BTboVmdS@G(+1qM+_8$7opRPza0YYB zO2Oml2>${b$IvGDVTq({LM`8&C8_qI^*}qTR2yQm*|cW_&?I4JS_bW{fQ6A zo~bW7AE78?*E+Ru4K%;Pt`;s6eSf30j-73=A)_|HqS01AN>O1`%*x8ANkVk2lZ#5A zs$~e92w$o=R_NDJh3tElNkvrBsnV;W&qXI0o^>dL zFQrLV+E6jBM}Tf?rTD^AP<&n%ibyXO(HQM_YKx-c3wRdqEYO{O&!WSI*M{m1Fm`x8 zN2h2!@<)$~{^!$?)Et(Y&`o4A?;iRHQ@4{#Zs_t0cu6+co^(luSuQ1o{*^vakZ?#Q?S36wlP5HKS%gC;)JkS$`O6fvk=KOlyDh}PcVu< zNX0Yqh0vXh5mbP1r!Nb%MJbcBY7JdSk@7?!PL-$w2#9D&UCYd5j(|Fq#eF?YSSaRp z(Vj0*@M%RjxBWi*2Z6_XQq(weDC6#Bh+E>z!stNYj4U=*y*o#baULsRErRI6K0S2AIPS^nVwF647aA(L}a_bRFG{dIM z%HhFvi(DN7>mI_v?Go4HZcmD<3tws2s~iE~k)d~Sp1n`5GlzCuU@=`Da|{!aZ;yG# z_Q--Fg=SVTd=RO4d$5hcVuV7#kT%m|z<7?$v_j6}6)hVbuV_&mAn=r^=UN;dOARbx zHbgi)^pMVIVC&&4=c7jc6HFJKTT$dBx%R4gA+d#B!^eoeZ*b4u1^@NO%?e7Ub9F<8=i^=*JHn$**~VNGhC-i*S%XorpZ z2&i_^LsV}Z#P7ewTkb@ZG9%>j5i^H>LU)+uZ_w7f%cZpG=PyS)c?HDi_ip|#r zSP1>~9mB`(1?YpQ3*R8n3mdDDT3Q?-@OjTLwt511N#csY2q zb9-QXf1S9j8U6+@WTIJ7dtShB_+24p7CUHS0Qa)m&04zP3gtUV8yO^rS;ZBIIn7Qx zL85iQS+pG^;w{mM-t27x9dCq~p8*V(uGSON>;uv^#=_%IW`X-yl_Ngf!V3N7zCe^~ znZLKz_3(44@%Q#_)+O(Xbce zXqCwW?8^4gfB1UWN96GnrY)kc2Z2-EI{>i&L!VLqSlr>`ZOL(icef?S6;5hPjyoLD zmK>LOZTz0LpMGtWUYB=PhFc#DrJs{Nkjp!hy2N=tFi)AYefY}}>+Shw zLEFxK1KL9^1KIlP=08Ft8ntV4fJ7VgAt#VT-BabF2|%9W zuaM^_#~67^MRg8ko*CC|_I@Bwo$xSmPts2lB#P&5j>D{f+?#>3_5mi0vWrN6qxu{D zJ9mWEfSdD!!TINsGb`^J$_+@QmV`FhQH4Jx>IQie_#oO-@o{L zwZeQgGLR<}3=kvzFIAmEd>xsO*6>|$O@alQEPAyI^^V8=FN0zTAA{mSK4nKb$RP*x zj{s#rn!n&tVx|=KD#N(a1E{7d8TsMuzlM+0>QuHnHP3qJodjw`9;)St^t1+Mcs*xE z#r$u8%{ay%J5^VnlVWN2>y01MJY4Ir{H!{*tb=U>4>s2c4ATVqCncw{AUkO6bx=dr zpxTRD>PH{4!x4!;X7R94ucTD!_5YxUf2H@ZfRXwfp6j*$;7n;N9p}gWOBSxz-?$W^ zrM5tae1BwsQ@Rg+3OyK2@Qd<5V+iZMN=H00wydCQt3=;br2b%MPRX#k&;^E?G_lWK zU{fdLu1j+dTPJwN!1WNmyx=Dr{508EQ%FW#vDtkIcrmg%T5syx*+g{+ZbTu`b#^~w zU7F`>SzVl`?=^l9+@HuA{7?ozgg++vhQ^|BjSF^n7(?n@qyESTY^^>R>v?l5qM zb?l)d@nOP9D(hk+$-uQz)3;OmI9{0*mYoA;RD z@EvoD?N(tt`_Q{kw84un0^R(E%m`irm%zRjqLi!_U82%Rv9ZIEes2eHg1lr- zAZgT7{b4f6iT*pIvHkN{xPtiR^JZWd`fJVH3y#KI^rLGn%J)j)7X8s`E(R&o)^P(} zcW5pBj#dV>5VqUj3QW2!I!mU{!%c>f!#!0}RGhNV?F>9c_q=87($L~6YViTwGan|& z`T4szWK2A;DMMjoeSFh+Z-UY22!^?1|| z1|G2cYNvwP%jXOw8!n6!PO^UMc_u-jH$|uyb#@neH6m9pZBoy&l#|&Y@p1xA)DTT znWGqNP5YD3?+>yL?H)TE>hMu?sSdUBDO1tWhr|Ar= zdjix7%j39W<%}M^q>0jF_0i=Eq3Igg8+cMTam_sN$`9h@$A0vAm%zJniN6huF-1 zL^C_t?BBNNH?rDWN;=ffqeJ~fTs2Q8R5j{t%4Bm4ohmkYDAem;9HTL~6UYP;HIn+x zCQ$q_!k~>BX&6?ViVZ_v(s^g}Q2e~xwL_nO6m)dg4h+_qoa6a(2T6<0%TNMp3<+`l z`6R8KMtvg9yL%f_Es*PPL1{@gTJ-1GacrMX_)fB0C^P6gNqFuV-_M!JYR<3hU);Y5 zjYAj6K7?Za}PF1C44lARG#=bW-N+U!DlSyC~Ttt1yfv) z~Hm}&wM}^l8teqTu^p^eHk+sD0I_Q1B`PKPB z`I+*im&)i%^9`9da1;H!3v=CmC7bUMWi0;=>AlvyqOTk6MEeQnq+DOumCa^=uOx4? zNnCCt`&g#4Xqy~7jHs$+)YZ`uKo)}GJ~8>s8#2obKaujkgb1%N*@7}U!+ZlUyHTQuC8 z`{P&|2%pk@JcdmPTYh~9y#d#Ms3<|cB^*f{jko^7L~LAd+``;$!41F#*S+02x#tpE zZ=ooeqGp&06&G7!sbNBm%>NJ{iM}KF%a4%`_=k-g|8TOIEGb`;z>`U^nEzp$8}0yi zqXkN-C4AhydSw=y>~}Arzo4k;Z(L%`yZcU;^+(aSlej98>S=x3J|fjo38cEvl;};0 z5)s=cVM)V94(CMoS28)rkLmPmXmjNF-M2B^yN-$$**o5RD(0V-Xl;6-OQs1y-=Y`Q z>z@86!#QkMC-b+%6A@Ho2*@hFKJ*Xv@a4qtC7A%|Dsq@zNa7ONrW_`;0&ruWjDS?* zD`YbE{Y*|lO@b4ReXuQ!AT!-mAaOMI&182uG8y~Eu)92&jD6Rzy8$v8`?|5a!BngN zBwQkj;BS zK*D$TqZsWh-~f_p@*e7pcBj=jpu2pWztjOM74bPKJ(W#eXqL(YctilHW zLu})%!MF+05yag@^v%VH4MjH$-!2p_%0gWe@JLmU89oIxm?sDXka`!oKaLxSLI;9v zV*bZ%p@*4G>iFYGvQtd$Omw@^A)y|!JHDiq4ygmG5fp|>)k7vhK3;e-XoTe z2!r~nGOn+@BE;B2RR3f*lc%0Dyp!!pfT4XjVC%5cc0k)2%F8en3yUcq`>Y*Gkj|-U zkJN@?+31(<11*4gH#>?ke6q0)UzF70v;s*1)zDM=JGrq&vX)EQ1k59dSoAE9;6Szp zJBu&&p+46eks`7+04p_s)(*_`VE`F8r4xizmMK+-D8nEvuqSHB15#+h^j?-4gcQi`iAMNy%mc5k68?{Q;1SAz zY~40{FP-BHTV-pxyZb~fgb{SC~jd>J7$U`xjkUyO=5$PT_z36EvGTaEn`^-1D+$H zxmS%BPgVY$@L zP6?yCrc-7jUbN}|f)}k$A8c2jA39?C5Yfl<{c)ImN}0qW5S}vRV?)-&z{z5wqnq>OBXrM|a~chU;cO&r}Qg=NqjtsQ%FgK0D$|y@zfUjVJPRCd9 z*~+*fk}?UNRD`<3<&s#u!dDcpunILG=H}PjrGxEvemX{{a>G z&}djy&a9xMh;jibVU}$!ZCb>s4atkqL_Dj}^G= z>|IT;x1AXc|5OQ-!?Sw^h*!ZJm$hnkwaa>Vl9n+DTeX)#v)N^djb@jV0HfM_H%ZmH zq}cm3CF;3G-cEQ6mxoi$D1$e#0(cv~N>ZHVO`b#HjlU*upXaGO^U$|cU$Tn?srzK) zc8te$ArdI@%(toT{2UM;mFZ?>PL8;IIkK|22suUqA)4u>9Dm$%-LK^N>}Z!~`9Aa7 z#|5mAl^~nl{WPb~!kk7VZBd>LrV~U6jBvA=o0z`MCclyZqo3a92CWtN0p z=f=S_XX2dGG~-?89T$|y9%{7S!jcfxcf|2?WfEW)sd&zOrLgUl}}Vg zh9qqyxg<-fdlby}DM@i#n?lRbj{u_qV0_@6J=o-AHDeFF$L6`*Wv!g3EU0dpeJK2u zf@#k%s|7zByV!XWa>n81nbcg7jAn*Om?2p0*^nl<(Vm?{JabNtzrp43dxUrx%`WFCRZ>;`rgiljgT*o*4nq)zL0xi@s<&T1K8#a91yB zV~09#MItd*(B-k~fc`@4x{zd_-ob=|tyIoafj(NG zkr-~2Jo(OeGqbWr^jkOcieFcv>4BrzsXV#HsXiIN?7xHos(9A9 zl(Lw1jNABnxyWS(2~Ac&0!d3ZME5c&cpPZ;)M&}NQxYefkqAq68lBTO3zD*vMECo+ z=q_QNSl==(y2Bla?tkQ>JEAd*z8PF}hdU76Z{?yp)#}erOBCHD>lsV{f5r)Hjq@>V za*Ec!M=Sw+y`=%<^;iOUr*%(ixxGxvIxAbZC&=vs(%6x=r$R2OvZ2T7to30R*H=|% zz8M7ypua14t^{{)14{mco?Fc(sykW2=_ebDCGkZ_`71Qz;5n3fTTn+`VD3i~)r`iutL%b-m8im1uS*pF)?`>wBl01UxU3s%@nt}!u34pet8w^g-YZ= zKYd9FtlSImK+}ex@FME~vdOvsMtCMkd~e^J_l?F0t@UbZ+&uxkbqG(ZsFKj|)IPd8 zl^Fv|s0n^7qxYhQ-M!-p7^d%J=F%@Hru$(7_bna?*^B3@g}cz}t%-RkoBq;GUT+-hfm-IKrCH;>{hd9E=Y5FcU zbj&aHBp(Rv8}=3Fpf8~t$@q`Hw7^_dTt1$4NF(@p`hV&pWV&}$pkrCrE>GlOz6fTa zE^b2R@^k`o>D1bZp#f6{*m6<~fG=#`Z&V`sYDgWNpJVd$lM$5&q0Q9c!1r|Xlr~;! z{$)gghn<*XxY1IluikG2e6FO*`OjBxj5!`HjQmPInnu)TVu!(l5v!aZu8O`R0wr)n zZhHmt1>%Hip@|G_+x0n<8C_r=24>V-+Ljs1S^up-hP+>~>&e!G?%`M`N_u}UXOpp< z6eOTQ^=8ChNU)Kh--$BLj2%+O2NpeuOBL7FOq6J2h?q#5_fWbvPj15bZ83_h_Rk}fk!^}P4$oTe zUnh7Tm&$Rjfus+$3v)!dqkYBGZ=ybbjFCWJCd8+}o6Z#W7mSs;zctf@%?a+6IB3&a zGgB1j>rU^vwTaTQgvug`_8MiMr7n~emv{AkpT+VAkH9@FWKJp%4FNsPfI?&|U!jNJ z;3m^j(LWqsaMRg=^5lGP?IVS$JTDqVQ zD8JgsvLuYYBPC5yhqBO&DQ8h$4ZqL+mi~%7+Bltzzk8VRcQ^)@W3=?=zeJe*w;>Yq z^+_k>9AIR#Dc?_{=*vpuYSdY7`CQY;sD7QmqXsF3&I`2fX6LsHxh3Crp)wvX6!YhL zM~LOo;Z8C(!$}4^bt&)MS@u^}%JdV)Wx4)6UXrsm$Sk7ou+i(EO#D>(D2@2S0C0qT z?6Shdbs1s`?tIP_M&|4HO=LV_ZyIClzblRa4SI%Zxi?v>L{}ANcu@+AttIX!WzCU? zej_OK%s(GaSYnLJY`AYT>z2GoDohiS+ zNi&pROe5oMin=cct}vAAuQxM{20!Xyal9I4#~u%_`D@@s&*j*sdsenZ!-C}iq`D%hiQ+}D|2 zLO)i6iz_Ai+x+!i%%8u4`EclWmN7amX0KM*$OD_d7rq%&=!w3;0+W0RlQ^R1^LKo# zssIBBdWs1VzNdIiL}EZ)P6k{?hc76~M)dO&_?}q&?Aw-rrN(ImB@1KMie@;7bolhE7%a6uw2eyJG11W{CcAI{!3B?+#K`K7V;87bKEbmpw$dLt!x_*PM4-j;x5F=_gpkz^w@{c}ZS4x?{M?B% zqq&`sQ!dJp$|)Bu+ie@rCvY$mF(_Fgzg)pD@$mq>K$Yv&`_mAOjih1m;TJrxP-WnIb_v47M#J-r69@&Vv{K-y<`RCM4NG~*| zhTG+xO>9{2M0hggqem%Z=p!uV)Ha}Vn~lObVtiRY#kc89(U(uV?c64#!js67JM#)V zr<>?NNA{MP?ISi}=N4>+svk9U6I6WuG>q}A?+A^P*f@`Ic z;Mz(=?=4g>yhkXE-WI{pH;UbctG+efBN1;&xPlk)9IQQWC{0|3yqQ;5HTOEI zN_?+!&LF$`5x&MixeA(Xg#Tr${n6(AMzAX%#j8wb=L|-b>EbdBWBs)%(~TB=HHxHRR&g75wEI~QB|i7RXHp_JNz}CA$*@T z=*ED-+rg8#?!XtlTLp*p^ujt`tx*;W+NwcT-5N`SyOO_#35{%xd+h>Ixws6IK{+fn z>Pox~eIDQ6W90YvMv`_1pbBFg%Nd?h$;uMh<*vrWJ`CV-!0(E zF`+9h%e(d`laY4~+2hE60^ima=3r0P*#|e!yHzzLr#43Ghx#peWN9 z@u%R@)d*3d&i02x2)LDM2$O_ThP|@v{cRSPWSa-oFs{ zOThEVTbb{A1g#MP&fSo+vCxqIJL91t>9V&pN3!mk$79JWQiO&CMgM8&ZRiyIvx2ps z6KkJL7uUqwcOc8eut+vspnAIaC!iwFQdNW&W0)r;kO;ZylB^zopiBCod+Cx4Ak*oR z3?LF*dWkOzoS+`LKpBczr>N@v7jU&87wf}*&4_h1o!IsBEjUe28|8m#X{elGTkm@^<*17_LP$IT!#V@ATq zB{dxdo+9+(L94&f1SV06FCaX>))vL{56BXVY|i$la+jR!L`MqAhUoZy_G?XqTn+8d z6AAuDT($s*=?=3^ska1d!KO@`wQAuEr`m^^$dogP;87Hi8>^=b%q~HIco`^qlDU7vR2RQMC~9bv2tb>D_;7@3T!9U_oIU| zLOExSm|$KAf{^#{XwM{tvd}JCEMc5sB5ugc6u2P{h8uE`?{%u8HdTqiT24)16Dzq7Jd&-0q<1)eQ{#W=v%q{_wgll^FT~MkJkrpl6fTzR|oSfodj_+! zIR~Dxd5`6I;R~SqcHyYa`=c3@4V&lBqjsP>c6)nltOH0IePwO+$cn zpG9yWq0)|rYyly7I8&$`xHqfG0aDK^mB4Ufj=N~K>43*Fow;cYCKngcKM>uxFbWMc zUgQynX!(tzpxlH|#GoWj;DR*jlZl~-S0sfZTG$;Iq*aa1_9F%h|1$M0mZh(@|XNgK8+*)_>&4H`!r)gAU5r?rZXICh~Mq^*6NUMdi_ zxL?lXbi*&_@wi{?2^WYp-yVT&-g-dveacNce>+O2Tj>4v=F#|}mj>4^KWWyY?@@vK zjol~2Y)H2YJc449z=ZKGe@Bm_8_(a|27hk-=2NRlaeSrsT8$AhU%`Xz@C)$7YF#PO zPk+p~`*i~O0hox(Qh;DH*;7vIPZqNh-wJWp<+58Z<5{Gug_K#O-{R(&4!8nKg^T2oj+s9W=v#x4*q5kcdn7d zdkS+AQhrai6GsL94&P04h@c`c;*=Vd?8{xZjmQ0`q`m$1Pe(f?oWe|rpTe}q;xqQK zrBn|FgCytwJ=&=wzTFC>h2byI`OVrk^#5Z=G*`EA`r*egz2o;Jie$#`NnmMcjop)2 zb3A@evdx~>_&$5sf&JTq{Yy{i-@8XT86MYmo)=q<`=wz&>kpBQAvBUqE|?b^1ugPH zOTa@ufQ0aE4f*e1hWSz_1DRsPd7Zv%P3&0ZLuT_Xq`siP;l`&D`qCTw@=_!oOv#5u zgwJ4cl}|*N{8}h})|uyuW%7~u*#v=p;QS2k)7ssqC*C9PV)0#4`AbvNOzbPP#b(-N zc%}V$CL2vKR^!1%&)!t5i zO?xT!;n|~^6!|GDf!NW{5~F9pFz}H5M3Y?5WLp}*w?SA)2M&*Mpen_RyPva(pY@il zdp$N-!!E^RGDM@((JDMY(}A`1kjot=HnXSoo1$b9YoFDY(M$C>oTEulbmM$BN0`=4 zpM9v4$*3c|F##X!9V$Nx%P&YMKQ^)apHhz>>;B_*@w0BSQX?tAjkkI7CFN$y-MR-Q zO;6{9vAsJs@6c{R>V8%pgmo8x1(si;{}$^gyd55!!}=R8YmbL0DSHxnDSieDOQq*( zr#jV)ZnsKj7vj{~qBR{mNB$Cj9-FdX*Qe^&3Wrh=othK=G%nu~P4!P4r)2uNd)B$q zcJ|2`JfemK6pDh#!FtqLc^DQcel|5t6222(SVPRZJBAU&#_ z5$LP|u#e=ySi%3+<377AE&COndWViJ^RD*;r_5R{jO;g$KX$2l{TA=>-9_x|qp`+wi{Gt7RS z{akxJ_Fj7}p6AkdbxFLs7$zL(Cc5e6 z8AYa2sm_?1x0D<&JtS@Qxft#wxoJX$BsV8~MZ!*nY3L8Tbp~3M+h8Ci1lz+Z}>kc#tfdXW996M)>^%U)t9321{GarY3uDKT{WLG+X-4Fe2Uzxa1dc?$j zXC6s^GiTu0A5t}VGiY2{P&HRTHPLYiJyLaWvFJE6^RCfq-T>fLXWovpRRo{kLZ7#% zgWit)(2GF{`V5m(jFU6Wn4GZl{s~@usFa)+U}WR;GExW}=+yTXPV>p%0uVGY;r_=x z*xE@`QVEP&n~be(h5W!{bkR|F4_$OTFcL31u)tiXqYHeDQZ3fx8^F&!bfjSw9ouHQ zMyi%8B>;$_APW1?4oLYu3=r*I00Q4Qf4q-2SoH{)U(I-&PWTt`UTHDnP>(U>eevpi z+83Y1n|Z{1L{wc*T7(CD8+k9-JG4u`iN(8CnZzp_hH-BalaT1mMDC3cKt$C3*RK0P z>B6r-6t@WUu>|{QHS80i9Bl9lv}J$>;k*Q!;0s@lQh~evQTVJBKf4x4C5%PROkVKo z^jlc>d<9v`Y?+ZK$~}^{IFt1K)KFYXG|9fvowAhA?oH4qd$|3NP_Xt#w5E}vw5EsQ zXCpq$(pLXEma31K!cWzjO#0SJ=ciDIp<90HZGPYTu{LABt`1Z0yFFnDgKKM8XjII+ zP+EM(ZX(G0y&aoOqwZWfW{RLG?sUGHVjrl=dA5=Ph^gq(b zk$rk2@;<2bvX;6VMuK_#{l0ADp8d>a_Nv&PH7Wao+(uR13xeVf1}^MXcikDE=+1{< z-knZ$8FmLx`ZMJg>=Jw-w~*}$EFfn}N}zxq_47M(ymu@Dereko@fFEGx4;JY#!`S( zs>T+KS3l(JDD7jb?vXUURci+>Zy))ew$ONu+eSWQ)INTK^y!lAajr#{UH;z@ld(mtaCk* zHA4oZHkN!qIyjQ?yzwjf|I6lD+mpKVsSBMxv)wO*4hzH|roX45_amg@?WCKn*XS+P zr|w`A8hilYaLJGN69T^`2{;pb^?=d6UMvG2#9pGN>>L{=Xm{`VKLu2m{AfR{0xy>3 zLCD?ABK5U0T{ZS|XNRkxPX=)J4Q2f?NiNF=F+YQ~b}VW|%zyZ-FBkpn2AAu)Jb4{k zAhrw&``hlrL!hOH5sj&R9<79hsFj)Wpu{J7XTIU-QZCM(2yF+9>z)<$`DZbwr6^`y79HR zLJF)<@Vj*DI)FIpkD11X??izAD+1$;6kt5piol2>0?O$ywRsDDa`Z9X9wsi~T^}U& z0IrJ=Scm&d84#VB(4T=R(x*-c6q2o49@wHRGh{e%jDG}a&_9ilIh}!|xohCxsO*mx_T%t+_ z55cv#E+dWK`7yK(xqVZ+h!6Y?ReCBUau-d|9h3*7gGtXXBWVwdO5F&h+TVb3doQKF zhjHqj?mnIzHC>FyD^b4}BWLK&s~h`2?~?@=)<;!x0O112XPe1|hDlh4MpMM9F%Jiu zs(r3t`E()hEFQPjSo#akSaC--SPHORmojI%`-TR0Q4CdBZUWpPRW-U~cO%_*eunc@ zQHUpS*OA_fyU}g76p+ESM=(L2^Kke6DqKp9#p#bjYk2aEY zQCW2iJ?T3L&4oR3<$9Ow-6E>vTpm2dnB3T<`Io4#-YqUNFIl-BZ3?^PVFm7@Mz49d z*Zi;O9clKTa0K*mO_ftdLUy^g3cN?DYR26UV&2>y&~OJ1U5mdMiz}lZ#^E~6jk<1S z*~7-1b$c$He&VO&mj*h|5uOXvN5Z;r+IPv>roLj6zFXbUU%nCiZ6`ij1i^8RORCy4 zN*$c7Jp`R2C*6z}^gczOGR`+?9IZSQno*`d*7Hb9xj3AkW$1>|?2Z%&9zJ@n7gKEJZokf5GhQ)lq(f zRVjbicP&u&X-3zQTmb3!?R6G~9h+z6>Fnn+cWUoA z<$eXmcd%)jFkP<42f#IJ9(2li1#8Th3qzQ1$F1f?*7ip!TkY@f(uQ2fi0y&z(KyH2 zX`FjN#@L9jN41vcW9jeRzet!M-L4aS-d{+kcz@w_KaQ3qe3vpEpuH2O;vv*-8S|Li z9uGG8`nZuUpAv%f)WQ5N>(_{K3mvW>Mp>(Xy^7ojUz=y;C}j^rWTs9}V4Kn0`7Cbp z;DXCe?H1~A&O0QOfqS0;-I@;Qcz>BRiNaT=f4GpkU))853!qu*pIYXVs`3gX^9Rr? ztKbjgs5`d7_$Qj#t!3c*NJ@ltm6#5s)h7hTpf5)GG*DPiAE2-vTW54wiajr^NmOV1 z1b=;^wi}Gv4k7HWEB9e#&XC*0nv=rcT~~Hab;_<@ozb_`;hlGsHGM}V@?qI`q@Gu! zcBJUvJQpZd%!L%{NDRnUA0%5gpjCUNqOC6TNrPA|P7tdWAXdOx_A)u@MR{|F`;_&I|R*v+X6C9B~(Ir@P*@#Ok45U%5|oG>huHXE97GAGSFijWdqA%!l2MBSK&Ru?_XG zx;eg3&un!#ELgp;tS2U~NTy~`YKN(7Y9^(&rKMUZb$eQB7N!0zEj61`f0>q=L#d&( zR4b&;pOu=DizyGMrd)w3V^dRlU`lCfN>5C=JvF5lru0iqxe`;Zz?AWjl9%n0uV#Z% ziT5tb7-ocE4?LV;{!{cl=UqW{O)46^N>2LKag^IcyMJ|9U6!$-)=+n^&K!(z6#H^n) z@eH8qxaerCK0&Xwrq)t@sBG8%=Z#llQ|Pc}V=oMi#ZXTSp>>LlJuq}HhOWTSU<~DA z=oSp&2|=_!hR91Q-iNXw|DG74JI@RZ(QRi8h$kEL{|rNPWBL(>Xt4WPL;S{xgeV^@ zB1GANVgXKuw*Q$}{POcU5Sk0)ZRzqi-Vbyn_^UOLub4w{%b0D^f=2tJAT=H{J-0MJ5ehR1+8aG{Z|O!5Nal(+ciLM3Kk-rkw@ zZQ>KE$GMB3YZ6Y&rE-M-%-8)L~nr!W}OnN<3#^C>m+k(zPc?SDwQpA(+j;?xi5 z-NL$$+LsxA-})xW_IaIb_e~+&KxShGnML=CsbqWbl{(cv5xrHnmteG^Cy-BTB-yKgdKKmwu@!o?BW}xHgSUg4Fr&71A zkM`}#`vC+wJO*3fWEftn^9622wD=xXW^FTbRXG?@s=d>mR=dH!#VS;7Zns(g%Zqe-sSdz z|JaAjEXAicq_GqO-^xlg<(-gUx!>5HAd94((!QVXPkX(3LwEcT zws&w7^C$gTuwQsXx6=QHO(gwaAboITO~|euI)C~~er6?zk2w?7rA2m#$B)l=p`oR`*%>j#V!2pHPA3{DmK#O-(daI%ea1 z`R)sL_^u`vF|X8jg*-c7FM@iA-@x|o%1G+OQTrKP`l%PMtxPPR-XEph(nTN6nEqtE zXKGw2&%^nKx##st#P2iKFLA$!R*NtA26cbCE6Mu}y|nYrMfG?&o7zo zZ^rx;t115~J^umDAD+k${hsnqE#Z6z=ij*0$lv=r%D-RFzn1eCFER2zryY7`TID3cp^XaTW;Ua`5QR@#z3O|-*EeS{tKMH*q>;h^Iu=Ak6&+paiaZS zbNhP!1A6<3{Lrts{e}AY_4XGf+F!-(>-pE}?JsoDe#&ydMB=h^l;&hqUk~Ni^L*R@ z6Qmt~oC;UcWYIbKkZZZR#S0+&UCv&&fU}YQWJQ;z)u$!Luf4<{1pUeS9(b9u|2iMC z=W+HY^Hb`3A-%qQ9`7O;fac*3)-Fue_rfnJ``A3lzKyf{ChEiah^?do(mbAzTrM+X zo-tpOIRDkToFDxO<+si?^4Gsa`MG+26X)NN$j|5eAI{+sL{zn*R6 zPviWSSvvf4enujH?@uWI`+EM9od3`)%J-;!CXKL}5;pJFm79SN&k)uLE(m8w+N_bo zw%le&%XCKCY`KlXx-pi>;e1NU64ouqi5#{@j^?{^>vOluExG4Vr@&&bUvyZ2|Lo0? z!xsB?ryOz0&CbZ7-cUzxr#<4dH#>1c-+9Nms11Ky_C`Q?y!wo^+ zLo0Z4(NXxwBu36za?c6t&M!J_g18ttY7-;pZMo-iTbTc>mp3~>VmOu&Ir23p{N?U~ zHsphlP_7KTB6p8mA8EU;y$9jD4x#$pnEn#zFc3{;~%T=1!?41 z9@OIOF>Hmr*?-s^Y3~Dd_4OaghKfqCW07NmEB6RG{qFIP6($r7mFH?rz}%Jfcy0xX z$7OhbFkp9slXgZ zj@WX~LuRBNMz%R)xIBKeWrW;swB(OiB>%Zmf!>0qpEr^|vohO8wn0AD<^kK<<^5IS zKJN!M(mFec1}Y|!A&kEjEF^k4$EuCc6HLU_&#`GkAR)!YdYb;{_;I50JsWBF9mUrh zIdnSl23Ch}U#(gP@w0E3%>2**+2z#QQtA`dT?2dPrf_ejw8FYPTj(0t31$jS)b|bs zGjPjr5Pl|FIB4NrgF*P2n9V`>nV2K2v)DrAIo;T=sg&do*fokW(c)5afEPl)@D_mi zZ{jX<+QX4Hi@n(eKzYn&uNVH_iW|=cyGd?{Xc@Fu2El*0S2E=7_Lzu}Y{h*-TL=nU zaig?E?JbKA(bm~+Zh}pCwOnS=3TIu4^QE`;gS7TyH;x{b)?~!j2m!wX@hh257@#PfS2qsnhO|I%5=!vVD?%*v+rK?361Quxx;=^zFL%b z;*>n{XktpPs7_4Dm`DGorv&8{kcB`(6qY*hg(CW&Pxm1CKI|m)e2Wg5lycZt-u0qN zCmUl*? z`H+?=w?sa(!2lsK1L9c_Kb9Yf+MsL>#Iq&pq-5Uaaa6CLF?W=jH`u8x=r>v&I0%T$ zSlBcpVYl1?;cr_ahip!merN7>NX>HQo)*^4$cFifd}7Nz1>qc5?!L$;`4k2YxfOLY z6z^(GIvqsEw`9riLYhc>mjcezVsDfd#-{E$-l)4 zq45PV_}5d=Df-*3l7Fie`|yc1cN>+u0F#`%n~Gh;j&SNFJ6hD2(UWf{EdkXK&uxwz zg4vbpU2-Gr9`_cl-&~m9dFWz$iyU)?=EdA0??qr2oFVr)ITq0XCf(;T0D$Yh0H1N* zB?@6?Cx!Aw{{;YDcU)L{H(3$%0z6G#4|$?`f6RXfgkDGi0n;!?CYmd2;V&fm=dCaa zflH+C=w0Da+_jRqku7c(TS$Y@c^hgF_WY@6-X1^fhDG4zfKWviOa*XB0R0hK5Z(zb zIQ_6v`Fe6TmddTiV2(4ll``r@#l3>wr!G;^j0(AK_ST}GPTJR#7rN!0Zh4PKK7gCu zD{qnHckz5%Sm(v|t21yVAOJHataD%Dwb_E(4*HXO3X5ED;?$qRLKkTZVq&N44a?rY zK{;pcNlb`40oav!tDpp=pc63{EDS3BXeWRXMrZNvEsFPVr1E1w?Xt(@-OdmKMaTt9 z4yXk?+TaZN3eL&90g+(l1=tCIVvboc`8=jvGWG^WI(fNV1+ZSXuvckZuZ5y2#?Tl) zpfnK#YEkX^E-#iB7R#$IR*+05$tTJ5_<5`}YK3NDOSIV9Cvpga{1Rbz&CrBv6>X%f@OZ=8z{B!+v`$ zZUJH4bVRX*fSoJ`%t0D3sP95{76H>2kbtDXCI2|AlY{9wx#zI61dcciAO}F+wBTZd zv@J*n(80JDAniO3#AQDznd72A4%=}nag#$@@dCDL*ju8xGyaLZ8@-qyd(A9>oz+4Ax~%UlS8FCN!`{XpC{jh z@Y}8?-y1Z$lrjDAMpzkR^Qh&6@%U11&|1$WS?$a8$LW2CLGZ?y@zE-*#x?NwJt7gX z91$dKuG|(EvaNs!oXAk+He(cM@?MDM>(NY|)^QTZMNP7|4Te@dQoE-E4cD~+t&>|q zS0kN^S^@ltVF-=O1XzjUE{EOqWz^5GdwB%CNB^zMeK2B1Gttq#Lh@XB9^@?P?2ctn!7d?h`U{GVDy|K|TXJwzk21Ic$Wn7{FJq>mGbJ_5d3g&2l-n8SMCus;(naNnA+ z{6zp0JsBp_8-~BtSUR^AlK@+9$K|3zt+-k)xe};l`SzpK< zq8uYL2oVhy(Xy2n5oX5>vn1@0Rbcn1rBJS8{+AVqp5!sl8JaiRF&e$(>U=k3yCL0X?IJIb$=XS>XUqr zIx>xT-Vd2ibRnLrKoBS5i7RQ}LwimqqQg!i!V4oa0_zAgP;L!fO-#XNz<$6Q7}yUG zpS*`rAQIU|K!NgZMuIx|1RMyW!ML1gMpu8dA?!igxt+y8RfvvBV98FL*#Sb4-O%6F z`VhwCU{7J-*l)lePDUU)*Y=twpJXV3?1h6Shh2jnL2dWf6!&V;e4a^ZJ(1F04L9OG z9=t&`pNbzstH1tZW*ArK3Kz)7FpoeTkyHW;9e0Jk9|wF%iTOl7Kg47rrcWetsCeiB zprBji&C#ccmR@wpXQH!+mWI*N2@%BY80mQa!*(t1=xIsf&;vRcJE}^4fHneFnvC zt8m^g$xYg5C^YHYO=<7tCr4);GxXoOxlc>KKTE$~E9??a+%U0WBvH4FgiL_$H@GYP zf4e{DYKwnGk9K_5r`h{XngK6OeBRyn!~b*NcX_|Iz^VIsVDEWH)?ApcJnAdc$+2u4zzztCidqEn{?JOi z>_={=ZVDwJe^d8;J!q92TSIlI4e4_`m3aeE;R~-8wU1xUX0tmk5-c~>P#4Mlu9mwB zXN*z8>%s0vbGq&o@2tD_A&Q0h3I_6P3CWDt}Z|)|OIMzMe(e z0FWwjK4f?tYb)p*%AY;jUt=j1mYyJ96`j}mH=E-X{S!4)SC#1lwHe$^{8lnINJZOx zWA#Gh*-wXGuI*QZPk^6LHcg`OeAsQ3*F%*abh!B;nglxZ1AIzCACQd}Z58hEMO$!j zFY^b5daY*#)7xLim0yo4H~OnC`vEQgC45vBoSvLx;GfYHL~2s|bK0yKq~`F#KvHw~ z5mYCt_bBkQaT7Lw)l|xUinB*?_Dwx)rj4&-_UT7rcpe2H{@`Vs;Lcxm-2_eq=d#N- z0UF@$b_+zmSeecCQqLHA1yDhxeUtX`6J7U1_wBD+`Cn|m`peqC{)zPc$oKmRhe1NX zOd#Hu9s_8{)HXC2d( zbVFZsTM|aIj=Ow?%6vlRIJ;W@i#jB3^_3D7mcNL2>`|*n9w7SX*i1fPUgzYSd@lWM zudS=!gjD?ww4ZQV=!(L4!NxvM&vcuh#Mzr^b{kLA>_$%;80zuw)8y~v>{mHEkYiw| z3drV*@bnSEp8b+#(AU6st3F^AE4ulG#DMRk0Z(`#F<|nxkuv5|dRuwfh*QRE?tZ)v z{hY?T14`uZfcBoE0i8Q#jCa8l8m~s`$U`=8_H)_Bc&A`C>wf{AOPrNle%_AviD=mX ziPh{|CHeh0T73CPR5Hmj20j4<@n=#kbEB^Cv zOph0ncp2e%>7%e|UwJgvoqd6>?+-6Z@*XmtgLe(+qY}8FHC#Kpo`K)6{!7lPg}{3_ ze5Hl}0-pnFeV^Ca8MHRTZm-;{wNgw)GlS?T65Uy$KNDhJ&}9XBGneqR0n=eyAot>4 z!%hteX7Vx4{KEkgaU+xrGueSTzHFMxzBH5b)-#LnFKiNG+LoVFAWZgLVLETd*SF*2 zy}NNHspH#l^a_bue2CY zxPLNbmvMF;XAe1H!0?-ptzG&C3 zasC~N{1u$vGFI3B;r!OIM!v@RPw4qF=g&ywmvmD8ReJspIlp%zzc5buuiwx466gQ* z{)GOKiS>Q-{36aDo5-KW`G+cW{U^>pT|xP8+ju_6bG6p%Im73YZytMLy0Bc|8oxHB z%1#ViU0pB%!@=J)@Hf8UUs7IFh9*8Kvm$sGiLjG&n;yEopY{_-*R&nq()|NLL1k!l ztM;|gopHiASB+-p4=*f7M|`SoB+M-`De3J z-dj-lqt7PGFTwJ=A)!5h!QEMSMJsN(=bNlwXaSc-@TgE zNPlf=E%K4EpTX$uaim*JQ8Ga~zoz>AZoQ~!M? zw#fZ<2$A;#KTG-vTS+KSH;%M$p_%%WY845;FsvdSZ15LD)XXEkjJeW{vpC~<1z7t* zUylbxIcC_^833)?V%!%_V_$ra$Ajfib=cS; zm^~1(Ct}7@6MIX^XP`Q&efCUpUmS+;)LO%C0T~8*?-}YtbOE{ghK=&pGx|npf8Kb< zub^*$-|6WC(Qx)cqxM=z)h)wOd?Ci!^00}03j739X2#=b_8QXpiQSU?9-+gxyq$4> zIOm>g@LkVR@>it}kuMNBzD07Fv`sF6RsTKTgfPHlJOPtimnC7+Y5tGHq?&a=beNKG z`DH*!uC6lUef|WcVKINm3u_}p{VUG*$}u+aM|Ry9g70S=TCvOd2Y|Z9js(<|niIWMxTH|z^f>Oj6^WU>f-q}>AL zu$3FB;^@dk`NEEF_s#jAQoT3oDP39k0Dk~%qIhMD5*b==TmrVsoN_jm!-Q!q5QPyKc6EkeUGrwd9w%*qvQ$!?3UC+ zvO-#h5Qvy4W`kHJ%h?dihnPhOY~@&gh-C?Z296a{t-&Ua73+R+PWvjwqSGFAY|>-t zdG=W9Ky%(+)pAPrSn3qoP3VPJa-3%W5gzm2beE;4JdXW7i`1iM1TA!vIE0>~lW-91 zsD1=Z%A-kId{;p)ZDu9tDLa9fnvB!yoC(P|z1ta)jMIyrJCkvGuY&?Vc^rXhO9DQFWg*jTuqsXZs*wIDz zt!1^H+>9w@{qfh9G$@mLzSdM-Z4bkSDW&rkZRogI97Hx_f4@!Zb$w#J$VE?7NXBXX z;|aT2b_@~u7=g%luf$5YVD)Rm#=ry*1pxHOzlZe$nn8($B4X) z>POxtcZ*fBA;Sv+Jm!HJK?c#7iP_p>8D;ug8_Muqq~Va8A5=@aV^xqz4YA&Mvz_wp zDRv6+(+^Uf%d!3QC0S`-@cGv5?Y|&{5h=x5pAR%!IL*0yLDQLBpduG>8NZc(Rs< zbV40hVVI1Xgnmnb3uN;-*HP~B6+})ui~>SkSXh@d1FPeSE+>A-cb}<1I4W&dF1xQv zyAJ|UL-}5*|AJ}$-J&wFM6B9OR`8X%88JZK)pv^K=Jvix{su3Al^V*r&fjjX{)`Mm z`_38PDs$^&Fz=8RIg+j2BuViP>5LFr8&Z{FUU67H27EOc}3?TvR99&KW$4XxHsjzMN@FTTEpIZxU{(f5OF4?fYKs_T($ z_3b0$ng-8`H$KhC0_sfbFX1cX>};B)v@`Bi9xEtDFuxW}G@rf?JxR^W*P{2KKZz6C z6Txg^3x5c|(fK;PNtd_&v;EIY@BeSU@gL~_hXJGiuKUvae}~ckBEA2w2e|*Q2EL;I z-e9;AZ9WBC@eQUW5U%biqV3D)BCe7~vWtA^I4B`01{a=Cui zzpsB0)xRpe{&C&ZU)-JkSHix?bh0mE4{yLH!->^UC)c81UX*QjBNyyHK24POld*p{ z+U}26aB9;JN+^1g4?0TBNf-yT1lz2Ro?{o1pT305|$v3V4UhsOW2peQn*JW zz*7~)+o7Ms&WP6yJ1xn>J{Q$lW-;DW-7a1m_a8N*xeB!+4!i->h9`@XTG5GmjQE!1 z>>jb@m`L1Or&&~c_7;{fU+xasJKXlBGs67IAyf>5KxJsE@Yn(PYdzx>9=8mZ$T!R5 zJqJw{4%EZn4&)Kg62M^g*0O!}9?8)t1n#A4-JW=6X_DmgBAPRgm`ACDdyX71zE`bh zKwnR8E0h%FzQU5L@Nh74C{r@OBMxx&s+9%+t;mf@vZWyUMe@GW-qijYEaJ*MI_)cN zr_;W(`vud+KOn0Mfsi+gi;k}+VxkuE6*fLZ*M3+^LX}6M z)QFbpj}3Rq4Z>2=-*y}nmJ+WF`v&&H25iKml;(@7d!Z<}8_D447Q(43i|G=0C%vNC z?_|f8ege|C?*U0&0AiZ@Eh^I?s+OU+U(j0tfxtF%XhIxcvVV#@?p9HC-Lp_s*Dj#q z#}f2MSZC$(_2$q>D1Wm0Q&9=_r&1+`BRm{r+I=oy$WLG%eNo&Ddoq)EX0#bb%qJYW z1s~DwBtyUGGi(R;r@zxuopEL}ex{k+v3R9u7)ypGULJBOPA3QV)C~pUwrxlbdKB<|UsSUG z2?>(_q8ZQOl&p0SlQUno!T0$UeNYegDsZQdQ zImL`%H_2Hhk6KmIP*z|ehOdxMdyZBwwEah?02U2q!B>Uj{9_0^#pARhYx;-KmrcpZw&axD^`%pCn?5Cm zzE@ds`)^YHyYKCquihhJDa^x-t;E<{|@; za>aONa~FPMTwfl_kLYadD|m{Z&NX<7lXs@`6d&HH^AsQ4$vnm6xy(~k!Y@gAG|uw5 z`mtMdh#{9?p<*mFh-`kRz^lIj8GLPKpBz*2iZa#SQ*otku%d@jT09xu@S&a3ggyVR z_BJZ{McBuayYat9^u>QX-=o|`RA4o*rf*Ln;`fC%ZccKeG-RY~l+sDOLpHQsx)5+ehm$1S5;d*j+27FnshqmnWUm^ zK-3#h4KH$d7qzEV8~ zGS~D>-yM3)0x^rPK#yfXEDLx|j^#isyV^=Ib$Sjo>3c{j+U-?GcSW&uK4kJ@>H+kLUgkdOThC8R?-V_%jD z&sUB39s4=njXjW$=s zkp^JJiX;k+x78};h8YyLd3Co6WA?3BJ21QY-n8lh$+<`p6H*q-sDvuE2t2HVUwXzcbW`QAB|U3 z;2fiF>RXv6zRc|5Vb6;LP$YoK^WcMxPVJ+IFqs@l$ooKGCf^8W(F(I1C_qL|yQ6}A zLmum7+W~y+b{YvS+5^C>_sH=0kixCBKg`3?{>d#lCM;b_r&E0{Igz(I3bU%Ul7(!i zP`4%eVp5(%Sil`H0R1Pt+wBh6@B|>Lhp`t`9$59NSyloUyvhA=3n9`_2jsSI0&Jsp zex-k~_jIK!ue^;3^vz*_Kz{90V*-`<9t)BJqx7q=mlpVbCHe^PqV{<@(L0Nhp1%xoTynSq5^ zuG2Gsd%DBmr;v%z4$h!*d06gqdLnQ_+kwTB=jqz0$Bdot8zjkFiAm%8uqRq@IdMxE zek%puVA|2gVMV`A=lif`*LS_|Oq{Pw!t)ixb7dz&$G%lTOmTmyJE-#THvHKwAwl{f;THAAUcpI(cHIBY6Z~JIDZ>Lw4_`Jk> zZL8CN*j#;wQeFz^s7RSuY7g`H{9&`_2J}K}b<{67eP%N{NJpPe8T+@k9UL`pqoo&%w zyNmvFX5d%u6PA$js_b5EZ*tpP@uX+9NA|9UfOj<>_rPD%8R2oul@fDhkN4}LANlM9 zPHeMqmE}s&9|1P){EWSlqZw}idg6(*$%{`T>!FchS9%7F@3jUQ-2s9pw*^XxvM{R; zT>+SPdj@#=tPuxfdvXB{W#fIPy|sO#mg9=0=%b$0-&)e_Rd@g(M3`WY>RyB5V*BFG zcvF%effd9hfmIxW$0F+ZZB~R7lh}h6QFQFC{+te?AQL?lB{f><17f=6U5^qP)jDAYUBdc*E?;Jt)fe zN@%a5vJtN&yy^p;cud!bjv?@bPHJh#HEaad*^J9KMjd#aR~-z}bb;9(7tJT#_BgI$ z&wC|+0y@hZ25Y$;{x&0L4eMC~@L}^h4$j!)F}HgiQJ>%)khfPHkhjm1+bYU=A11NT z8T}kd_fMnuwO?2qFf2lS9kvaPF~Z2I51&cO+c2MeEQO>Vg!gY$O0>crxq%m1i{2TF zPX?_19Nz0(u5SeVSR2usrki#9PXmY_tG_v+Z+43Ei}d^roPT2?e=p}hag%QUiSrlV zl;G!dYVrO!;O5YKc2#*kQesq%kgC2@U2+kLl8u9@C{QJ*G=r6a#5XbnNgw zC>HJTsG|&N3z}zX>pqsYB9c0+(cZ*<;mN;t4z9Q4XqtVZJ28v>YwyB4UuXl5)He&k zKM>cSkamQi%n4{B8_34s0+e_}Ak!f6OaO`J1WG(ZHZh^DW09d_XSX8Qdp}H5)h{K| zS~4USZ~t>bo-uvUtvvHAeBAyUc7NG1D8|Q=Xm7Il{ceTAhoC$?XQO(%J~2o=FWP2r zjlPngcU}33-Q&bX$M>R(aOFv>N4Y_i*IF;d$sgx)_2?Svxv;ff2)=CA9Z~(<%$BgL zhFggn9zxv51yQ*h?`0Dz947&0PO?jO4*?CjWZxz#f)JRC@g8Gzm+WtGOn1q?ietJ< z_Qf32U9vySG2JEm2#ysyH?pe0N#6vq=;UQC+54G%cO&F@)PZM{Ivj4392>VI)a{&vRopS&EIyLuv3j0YF-c1#@ z|B2V%$p&4U5`IkBBXRO~y$9;e}+J6cprSoD>+I>`d@o6RaKp4=;j8U*g$& zd+^GfiG}0{1C0nv1RJ-*rw}p&HIbA%guwF{?=izF5p2xom`x(sc!Xp562XRtWBnz9 zjX@kMq+01b31Y?0HxV{Gj<9dCR1`)=LT4YSNpQB4EfTVZu;_}WdKXv}O^O`LlpOUQp}g51_6A$= z5ZX0T^<0hBivD)<)LLO3bUn)VDZ}G{#;ZS-fLHN4nrD6u<&S*-Cplypxac(1*`fV( z2)cH1737QR%9GSt0BA%*XF;;kUaRROxdn2AYY;Vnyi+VHb=>a`7>kL<0>qDcto7Ka ze?TSi)W9fwRy(l)Rg0;dSIH}o)G<~s63-9mhN<2eA6vrm;4w~%S4_u!#u`k~izI%q z^CP_DI}j~k$5HE>5b*z=wv+V!41GAu#7#zj4>dU7BzXTAP20aopuV4i1m*8hkN|uO z1qrq{P>}lY=S#X?72q;`CoQ<3zXJvk)=PyhrM%a!8k7HT_=OF(e+9pwl;`XG;5WaV zA2dD&`8;V{FDXCbq3$$b>_J|f2>jo?Rmj};eG5-7w0EXv^98Mb9suHoB<~>i=R1(m z7t-|qMd0_Pt1Sma;P=H;ejmTs?k#F+@71NeN9noSdtJ{~j||Av;P<`PHUYoy^dB;N ziW-v){}uvvQD^w;fkjR50q4x4fDaIstYn6N1!BJg$Ske^=5Ylw{43HJeq@4zm$+Cphgc=&gNbwN1w&YYE$ITDeONa5ue8K{}cVE+x{QYf4a4M{U?+#6K~T> z3FOrB3be9}hM&Wt`d4!BOYUPo7K_fI9*{pG8ut?o=md4Rt|5flLW}8zHzLS$iz289 zq+fY9EKXQ(ZdlxGlsT>bl(d%RvnkGn|K0h`qT^a`{AN@BrwD!vY@%G(|5HSfK@B&cR}QY0kkC2rlGc zDFj0t6d_nkL2rd9t1CD-NtDM_!0#0Jodmzr;CBlAPKV!V@LL1F(2u~i&f!?D#IThFmI91C-7 zJ;xe37Uozp#~QgWyxz}KqC!gu#8)c_GijsfD<|T#8Wv^@EYJwxXz@y8G%U~< zSf~;GB~mdQ4GT2}7HmYJL`Y{eEZ7)WxDmw=2VfYDVZadO01YEbB}HF4S|z>9qP&a~ zpW*fVIjW;vN;=vg8NC)#XkId?Nr%&F8qmfrAqr^sTVun^%wnpz56o zAUo`lKXKdR+J;R8YRPJR8&K$|hkmxipfv506{2b>g+D-mmIfdE1;UIun-9iomgw=&O z2*6cW%Rzvvx)mG*kg8k7L4csT)fB8{Oi;ImV~h;y)^m)3r!LGf2A8@0-)s zM3b0OjJOw5YIKB&i6w}ICB9U8x_~#C%KNDfCy&4kvucw#Q<9%6;0;)Z0}X^RU&ouP z^!#S}eY#*(5}jgnbrPRqbWIYWVsw2Hqhd6iM5!3XUmwW{~WxgBIc!!ohK_Z$ij7F&n$thql~b%9YA5)-|^>f-DHoTWZ1=i0+ zb$A>vTdH=7H`E)9cjWwin5IlG!EfzxF9^#SQ)@knrPFhrCFYSWF<|Wtc{l6?fmaB= zk&)os6|oc@#tF+-pX=f;{D$#cMUnl(oryl(6)tT53m5-WaPbe*e0U`Qbh(M?8)C;=0=>#|;6S7^CZ#xUg_ps~6aUY`k z^N?YGA}wfjonYFCiZI&H>tb=O{$|7egs5bbx}lsSDkA{gK(tUs0IY#Xfo7Uy5G9lm z0A(OTC?f#GKy*+>0DOVSpo{?E0#QL30gwe^f-)Mw3M3R|bOp+X@QZTdB=|)+aSHsR zoHz}BQBIr=zbGfxhzdy&lXdA(RDRANq)Qsh7=&2FP?j+Wu~4BbV-R9dLRqFuhpRcJ zONVPXrb~zGImRHwqJ*+cmkt?{e$J3YLd@o*zg7^un?H)j4b>`1?gRlsQeP(NanbSB z{YaoKJrd`$)3wDEQcLJ3xbO%H>wj58@5@ibb$?AT(ie%~XvaUroqGf;@6<~3+PcpYfeb^tu;k6L5X zzG>@>tCQk#P=pl6UU{W<1cj1c=ip64@FYYUf*$ynPBzsQJm*j=09v!EqJeIN)|V#6wm9XcyFVq+a8W+P^@9rb$TBZRpTyihAyL5O6ayLcN<`D zoE_MF!doATa`T4kXv$VLQsI?20eBvz)L|)4)lAqPD;Lsgyh$98I-=7huRV?_Et-?w zyom5;$pxU9L~|X1Szke+M$`@y)t+?4iI*?tMkuxVi2g>@iyq3Fp!6DqGQqV|@^@x? z}5W zg@UW`ZV(TSuTIq`a0*2h$f>u9jYbzBK2dpgt%peY!xE_n*g*p)Jk8pdvoLZ=q3x&B* z29k3u6xf(rNaR9|TqqL>JrxS37OLPv&0NTWEC3a%NG&vp3+?4XS;!nvq3co$P2oaq ze2W4_k#T@Duj3SJmWlpN2s~!rd{J3j!^o_e=Y3@}FM}Sf!(Y<+P{$8Ji<1jE-s({c zcH3KFfpB4Lv@odYk(a(QHe4CWad!4@_?V#4d`29)7Mu4Cf+WO1lDuW4n)f00f508E z7SsiWmSpsRf)62xDV|)wu6O8wH0yC4gs=}^0zC%o02EB2t-`=_jbz?MJ%s-DAh;R2 zwt|Xd&#Bj$Gj@nd9d0AhQHNUr55?CK&=0WO1;OuO@iBQ`(Ak`D0 zwsAnJCqixGYUmBIyqwAOM68geOJ2^T8tPomWE!ho&NLcpUd{v>D__pk8S7upq#2u7 z&U6`DSo*w+(sv8>ki2}!ERYow~af5rU-UIgtXz3-iDC>M>e zp_0sVVZ6|Pu09a<-4mgXJaLDLND7cvtK8o?{jEP z9Z_KqSBh+JiEnmBlyZTe6QC8HgOU3qB0d67UIv|lVkwtFOTO$fXi4%iXi4HSXt%nI zF9-G{E`w^j&hPv5zJ}hel#xn$6U?*_xC_F{w{5`#LhuZ=9dD#JxM!VsuN$O|ueeRF zko!HFolbg(OS;miU2SMu;|;h=Sm%Dv9ncM$(qWG}Tck-gKGBaqP{WykSz=}xl0-Ab z^h1>jj;-R@B#!AvDN{J6S2c}e`a#Ncj)gf^!!iBxgjCR&S#H0T{J)ODOY!f+A}?Om zo%Xc-fcB(|A8@3ZuKh z8?OEwfD|~I-K6Y$!%1D=*WQ7b=+>0`^OK(IdcyMcJe_zS?Y*`MFVCy*WPRV;lrezv zn(eJeMA0zeY*S{Y_73Rj`n^K@2|VlUr74+JkHNTdYU@1AYcf zYj1m#N7m?mr-wDMJ|eE&*%mR_STkExpTP|yMUEnqwUBgP)`m!-=e#6eBo65yun9GY z+K%=_>m(Eo^2>x1u#>kzA?+Q=!dC>(ZLW5?1084|buHBf#FlH*UZo;JK+Z58&@bA8 z!%ju7DfZy-sUm&r4jk|(H2?{lYN*0eQ3zDRchC&T71)RnIF6rGjV~xQ?(n|{U+E9& zlpxlk%WpMQA@JDK$r=!S4(y0X9|KJB=R#nLZlRm{zFYgW2lbt<^rNmWd9KjPX%woP zj&rn`NQjVuea{|G*Apm#SmAUuEI5H;4k|h!Fg6M=P=mWt z^f1EFt)$$c;v;swZIY6Y(pst5Y`oc{*0s@OQC#G4k_Ojqy`#?fp0rLz-=vDY3lp^l zr_)!*5|!5&mDh6RydZU<)XHnPa-&wXc8-;g?&M2ddI3*gi1I%A2&(@KKF~WUs%zUi zbqabEm`#y8iXTKLQoeg3R4^@B1|Jx2(>u%cgp%z-b+z0(tP7+mu8zk0`6c2%x`HV% zy+At~MD2WT%97uI=_}r^I5#0QGYM2cXwLCni+J57YnSUedX(8@KJi13GT(|%_CElQ zb?Aux)$fW`qv7)jXr4-}noU-#lm#|z`OhxK{hF!zPpFSF)Kwr>|He~QR?2T~=y{eh zr^F*~qc?|B^}-AB;o(SKQbyBF{UvpxRZ?f+!(ot1wRsoxM?YJm_tSe1N%a*}i0YgQ zQr_}5Y8R|AdWht06oR+MlTVF6aA!ocUE51VgdjdG>Ha;su7V>fcZ$k=qNCGy5Afwf zM-=)BfxgwQM*6lcBWWjz)JPw?2lYCQ_w&JUb*}adA)M+R9>)>ZxH+U7Drbx3z*(b3 zD_;Rj+RS_m09QS;KL&uOo>_G4B}HXd6bALF!q@h7?=cs zNf?*{fhic627zf9m=1yI7^s0jjil(#TxE8`3Q<0^dxP(D_?kzQm0CTQr|yjvdUO>> zSLsncxO-!@9_5p}H`eG;KDv8jy&es7G^|JY@a~O9J<6wdZ#3)OVE#?PADY?A{7vO6 z-zy9rsLm1n3#=x3_ju}exIB~1UCrzP`widj;i^^<0Q z+$<1VDygDXogFtc{6S(b-|6qn5tgoi8s$?`k5a4uoLLC`gY;)hZT@o^Lf`;L^8M$~ z#2!cb`_EZ~z#lnM=s%Yw1eSB87~LKQr*ovl`6lX_I(GUd7ws&1&nfS8sS`S#@+oJ_ zF(_7drhhdVCCqsRET z{mtLvVm^e6IXz`DRY#U7+Ca{3I_wA9BPatt)S1vfeS;NV=hMeerhN>bXkPdpzLUKw z`co#8cH7Th;-{-4DNH*I zY4~>axKlUw4!`}G_gGM@y}76}Zm0KwKl{2q7v;rlxC`H&qKH^hL{B(h1`5#}2s`EH z@o|xFu&AIrGZ1G^z(hXWm!EmQL+>PKmrCl}C{hB|(>9nn>-A1nIhJv9w2EFacKboP z>8$J>eI}{38@QBE_I)y`?E90XvTs#V*>@5KItP z4@3()xwsHGjj3OBrrA3+Z0jL92VCydarr=4xh=oOm1{}IaD|hNIWsG-gG~V~l(s&!is!=3Fp?zER2_UKC3o5*->+?w{<#g!xaNWpJ zWnJVb!@kr>v@a#f!%nB!m#V&3DMRt;Ysx4?d@4;ubWk zQmwVWj|;Pf&QvTf(wWNpuL!1%A7igZ#m^#&`5R|{`~qdKz--*L+Ca$e*8c_ji2xC! z7V#}!F?=q`mxMr7hGD_)s=JaFL}p~9Snw+(b0Ys43x5A;>wR2r64cM9TJQs8MwTxe zokHLNq^S(fE|?a}%iZ!~y9c4&ouS&!SCCyli1_TzY~29O-Y#|z2+ogwPV4&@$?-4W zeo66SDPXREN%I2M9tAv_7g)4Wz@mA9LmLGgnim+fQNW;ifj=7s{FxWnvr)jFc>!`q z0e4m@;`uCV-HxhYmD^F1Smk!q6jr$%HH}qnM@?sy+fjxp4lq0m37zX&w~DT0!#vsY z6b&56OVPN2*eX+so~@FiX9KYmH5-VfXxTt4Y)a9yRhSvm^=uX9#dJN}<@A&E3zD>7 zxHH=n-F7L(PRO*C-P%*U`wk?D(Z^Bf{Bz4i-B^L|xTusw`{>W@FSmya7+kp%1^YC7 zmPw5qNe6%r?A_tB_-V#xPLMxJgRS$2(~*du#e=7J;pBalUO}+A#s)jW3h+qAAWur2 z@gzn{dC!;PtHq`h9gi}maOh~pKVx3PJ`YL9!8h0m|JZaPLhpA|^;N*zXj<86osH3| z{~;=Oh>kP9{z@6nL-lJgraU_CfO6L*%RMJ5nJ6Dw$!=i1HC|DO^(W2b8urr@{4{i# zP1xaW1sb74bnpP35q?hdzekeaMRYnzN2eud|N0kX;U;|v#un=j=hCdj>Emjx&eI-0 zMeFerr|%aLpyo#Y&_~Q>>*Sy)DB_-Iu`h{`-9GLNmHIeU?F*3{Q?)#@wxAQ!uPn3 z20p zFbh(0Zzm@Yni+nhd6)RUB0(2NL4(mDsdVv%T7;M@%xQM9ZoIXdt%mra|IfRnqo;2{ z`~Z~nReV_;>Qa<~po=e>(Wq-I6%WN}-=`w=?}z#<9%>Aff*R z5-OF9%r+X|}wtL{I~j&mY1H|{_E$7^(Q>eh!)j4~YIPn3$g z45771x0h6tI-fL-v&U#0o1Lnwpwr)WNeEtoFJM%Ft(N@=Mn;2Lietj}m+EtSa9Fy1 z7%Pvb8SgKgap`P}C)g;229r6DIVIYhW0&GCwIatEIhN`4U$jW-So28L(iuLy#|uClo@EZ;2-qLy)hCCzObapYMn9{n+3FU^#`r zUOWONwo?eKA(-#A@Mqnz;Miwv^i5gRJy98FrR%9v)W?zZ$^m*f(k97c@HZcRp_?Q7 z!!Pu4WFh=Q2S*meFZ6C?3H(CWMwY@a^lP{%k;dxC3VIv?fk~1qVMSBm7b}_uzgW?9 z_{EB9;1??TDG@VSg1UanFB&AMj-HZAva?iF7xOGWYrJpZXoZz$w8F;IS&`3kS<#;- zvZ9b@v7(r#u%d+Lue1~=59a7Qg!8|FU{{j@n*n z+?g|H=A1Kg<{ZLP21jl0yg@yfb&t$Il8yWO;%s8P(w`1??yj$c!X2BE`x%YF0wO9H zoz73_Jd}<~a6gT2zo084pgdgn!mE?(o}817Dui6tlst#UI`AxZxM44{=gMBQx^NC| z$m^r4I z0Ia-Auy~uv|Ir^$>#pO8)MG9bgSD}L+fxct^s_1YkzJVQ?HNbrSOg=kLQ3+gMYXrVTu^=NtoB(MszYqMHXxVjQE=Iv1H)ATW^2XK#2DlG5jYOXSJp_z2+snY>!A3zA#a}nEOapy)O}_ zBUBp}`S7q=Br1lVqP5(My8B`@pH+j;;8)H(LXJx!zvI0$eP4H~-y0^~%3cIDt>j< z037T0;a80y>>cw||4SIZhnx*GL7!^1D;ah3d>fIp)Uci$6+TY39{0fd7_7x}m^x?b zc?4Tx9!FHy%H&}3fWGy_2|98(la>uM#;$Gvi5^cR^b6uVua}O)gMhx8TKy@jF*Fq} z!`rBV%@@Ax=44kj!obDy4-8zq@&U)i3m+J`sQtje#nT^9z&_{SVG7^8ji#^%p08Ce zYA2C^$7~+k`2plV4Edn<`k0P84Ev6e$v9~O^WV(0uqK`ne)rhd>K{vsXvk{y5g!cD zKXN`W^$+0#u79v1u7AugLf_VUd=`Ck>}}ARtJ#Ql4|}j!|70tc*&d$+c~S!|%<)^% z`%1%3K229JvH!?Ec{XsS7Y*xy!SV&G2Oo59fH$f9O$Tw9^BRnSnL=_T*xWKfJTl!d=+td7)H)?2Q-jIU{51 zwOqCUX&gm?etw64`Yn=U|D4p{ez;&zf9bW@~=bR zxVwApoIb>@75*cLd#B`7h#Rt?4{^gx#N`@@O~lI0^{jjK`nb3#h$Vgfhiy| zOILV{17o1WGv^O_Uh=>Qj=sD94Xu}AG@$a zGvfZNq)E8q^-N2Lo@$>ucE|Q4eH@sVN*}ewyo1GOlfV9up|AXKUedi$s+@J(S){Mr zl=N3xHg8Zmxty09#e-dZ@Mk*L$(={%I*qD%t~2oyI`&z~vkyE**>9=lsnBJR-M8P| z7abhcpT3jMN<;A(1vTxAU~4`fsOZ0RCf{`+zt;W7&#rIWMS~aPr!QXw3XBcsdfcr- z%>0EHsKpu-t=eYpbs9eYH*Onb{owJWk`AN|KZdkpD2AUW>$dxCw5&cu+h6o)F)XGi z9tSxq%!&-Ucd_G)3}bd(GNNyG*#KWapuf^n%?^yXynLFql#k#)M##zXLY};c{}l5d z7ylW}e`sD}umPJNR3pAM2s&YjfW$YRH`B(~)@k8Ek0?f+L2l=UK8m?xh5niWdbJ$_ z1v(xz1 z`H|&Cdq%7)0jeA1+PF)l9cbD|lA`__G9L-vhnP4gzcC~3>NbVk#@^)8r89n zV1dekQrh>2bChl!+M6Jc$lqn$h2ferGA%a)l1;2Z6-OV&Dz3pQ9!U9y>tnrV81QDi zJ!%NE+`KS%?Zbn9bKNlh&D-Fcu(jy8o`$a<>X#b1xZ+?Q~@RNb_V%+J5nwYxx zzgRJ%7LR--(-Pm?XD@5Gvia^l?zM0>nV*v1^W{gAe$UYuUjOlHePfAC1vu5O2j3s# zgIU(i-ow-D)wMsPgthR(u*A z`pqUYPc+Cb*xiltK@M+wT%zn{DMp@9i0A2_f7neUiO#`09yiZBCnY&1bc>iKZWxyJ z1SEA%@f98on7uspVAm>vTToLPfxD+0o2phi^owwF>%+WCC$F-A)NZtxrpo{DC!_KLUb&OD zztKOw%pZrC0rl&$JhfE}f}pK41sH8qg(X zWG$e(6c)2TQ~Pose`GKQ;NGR*G6W^al@$1`GZ$6NmDm>U-0@(3riGr6bM1u$Xk_>I zn(|^P)j542!DSC2d9l(w2R2$v{n2yNaD$beeQcwCUtVt_C3%%ddbbHt>D7Qo(Ix{g zUQ0g8TA)H3uh2>OC3NbM1AH&&&vz(qI z^uxftx&1IOHg7Ntj7nmlnK00=>p%VcpK9uO16nm;49os<+-#XqF#cL^-z2&`#|VfJ6T?pu3V1j^KC9x*AfJRA z`_OnR`gVmU^vKz$pc%(ge0;feCdfJQTloC}nv~dNb8?{>r|&sFtIQEcI4sJ|@jPnM zM)nk^kC%@pIELcgBz@3~<)dy)(D+x!P5t-(F8{f)_e6jGBXhvl^Ph9y`G@(>Gu!`R z{xf6y0RF=^RX4^nx_1CtSZjA%QJcgX8?Himrmd(ov`6sR8>1p?3bhMu`-Qe>=1KQM zC9OO$&;~x#lZkg9#6>PThN@|W6*V)>jQ$Kljz`UwwtVbUXS7ymBSvZ!38^Tigw_Z= z{9r5wEE$H-%JJI#u-pRbHLB#3aj=Ruwe%BQf0ykjEpIwKuII@RdT=cNkYc$8O zlI?OCG+^OM0F`+g*$(mXbzB;Hkt`N33*1E->S8jIT${<8gn2k=I=7ZzuB3ss`ubl%w%WuQ4Wo$L{WtbLa>&2g;p78lL?lp8Z{(eLlX8mk#ucc(#{k=ke?k%*H50dM?jC zk7pm8PTAiA0d@J!n0??q%Fg21Z9MxIphqmn@LT%E_qtEk-;>T=i;Z6wUzk6E$5I1S zZv4u|?*!v_it#(m_%*cfN<%BJG!$@d&d%-A-oA_1uHWV$l_GFw-{zRn0jvj3{_0f-_4=Ara)cOsIF(ty8dj|C4X~WHVV!- zpVw6Ybrs?U){hWd)2R;LQMUVQs(B0XOlFM=^sjIKfH*uZ=jYOWD@sqt^sQz(@Vky* z>n$I4^Yc3FM3!I~3~}w*0}v+)#s~1u$3KU-C$k42?l)#*TmNaq$=Ur8cMsJw&#dPk zMV{`=p^bP#;+J+FANvb)pAxIQ6Y(~!pS>R9?>8M!_s91aC<=Xsiu%mIcMrI~hrX** zRe}=~DUQ8^A*kkmR!kUOD~8`gPH=@7v{SGdI-IH~md9eJ#6@;A1T0%EF519_T`}@q zG(uqQ9`-z5_@H8eJX)ybk|r^79%N4!@6Wie^fZfKyQ7#a6FvlpHi0Xq)5U|WVu{v9 z^4|R#^^t?x4baR!P$viZ=NV~Bf~OZ7X{V8{HPVGfx*pT;2P~t7Af3*9>ep&yemS%OFNaPj!2bPi%xMp=dkvwE*Rg}53V3Z~S+bYc zA!|>q=hsg8h$i%3+-E36h_VP~#y?KAH(1FF!uuQaeH*(I@f9b~o;7sTCTrmg%zW^+ zt|qG(A;UpD^I8rCx|-6OQQD zHQr79Zx8%XfDnD-yMypQ^xzU|!I+bPHuHM?sqJsITWx4{fUkw-TFb>nwY2oiSWMG) zw}L?*4uHz9sOSm$QKvfdB2~6B|KoZr69e1MmT|kF`YzfoD3Z+Zy8W#UQtzLSx2e^S zn-1R#Fpb}xa}l@FfG@#>M`$2RonVz%{z&*DeptUAzQ;En{!Kq?N9y@PvOgQ3KV)Yn zvm<`i0vd*&y-Q|!Y`1w14BFaVrnc5n;YzNwEfofPt|&Kt4Yup8Gtvh8tuWYig~6^X zrWwD5M)lK->h*(0OpkN;b$K#}!`uB`6srQ(^J;z=y<5M7wp-}<-Y0r=I^MnBN@uLk z$f~!NSV+6M{t5;=GCMS&rR~YNz5#c*$y^A7(@@jCjeO{=K7KwK z(-V^X{Ack+xbl1nGIKxu&8Iz&%Nx=FxgTIBd_)7lcS@1LjI;W*D%qz!46|3;j#H~8 zQrBye0(mn?r3w~lr0EBIsfF~+RAl?uQ>`aRDX*tO+)R=!t)*&>fq><^CDLkC>(gcy z`!pH8KgEWOnZAJ!eDK&Nd^X-Y3h$LBzZjX<*D^jjxPjb%fD}VmRSaQONeZixPUCH? ze}eg7yd3FCR?UeC^Nb3|8x@W>DjZ*67U(Zr;6owDiPlDuTYjr^iDv^2X zjo$|2x6x=$-^`56?0cM>eDuyEYW!t4XBhM$;2ZQ=l}1Dizd*BXWxWw`hW)mco{Y9V zg|Fz(-_9#|s$T_EGU)SZD=|oo*WLA2xrJX1cwgW)4BD66Z)W66@@;^<(jWSgJQ|gf zh|)R0Zk2f0HUlO4#kZ%lJMgrR__$s`&_EQHh-sX;;mRPZOSf|hl)%h#T&N5v3~8x{ocTAC<`1$5tXf{CLGA zLL;hjVrR6e1l1nzHv`JKZ;QGU{p3fLssvo&JNkETBR+A4Po=s$vifix|2f?^QHfN1 z+C4JfZP8}R`jHUmG)2!8t{a3+H5P6B5k_=%IiwzAIIkM;-I z0d!A)P&R<3^atet$lD*}0MMxZpj-f*V}e3Wc?d0Qo9z&ny>>b;HF~%rv?wLbfw4dZ z4Ls_XLDKq|F0N1f9!*PM>1V&{-W*??dcWM5Xc9%CieGfh-0yZbcKjYB%Qqe-$?`9^ zl4L3W7$nQPae)o4GUO{+%|aHdmRnQz8|c1OE6Af;jqMCb;W%O6gJ*H}Y_u#6zPkT^4nN^W$ ze&lFIf7RS(*7Yr+ziK|-sOlcuKrTpnc|EVl)XT;2MvGAqTD1<)&AkRU^&0hfQxx+# z*t?c=gP(uX9>c))ozyntv#0uIilIa~Z|+;DJiXqVl(&vz`is8wr6_-0gfiE>cL(bG zJU&$5UUbWL=czW|{?$8g^6i^Ju8aTt;NbRueYZt}>#;ltQwPJ;K)5UR$g=9cthu?DiNI$U<;?0Z(L0>|LzQ<`v=| zHlvjavzZC@6rGhpm!WtK{xv zQ&)HxzA^svqBeJ@N7ya_l47m79;lmbm+VyYa7pN7TLCMd!plckcTtO%y@Q`AaBq~f z(WUtQ8CdX4KnzvzsS_#~uUXG9YB&oRn0a<+& zc{FPQK!0#4zPZue?X^=IMiy;zcT1X;Ui%Kdb{-(2-}ldBTaAyOZ(yJp8UTp-2wHKB zBW5Rne{i9DBMjzCCUQ#fl?w#oGbQ^`?6}w7hLyOAHd3$4unIT6I*NL|hkp#;u;3feVUK+azHwR6HYLh?U)q%geC^Q zw-??!I9|KRF4+^jXISRZ6TD}>u@+Lfi)JNH*o(fja!g=W3Pqt>kF<)HOmZN->hJ+$B^w$;9& zs9CZ@iP2DE#tLX5%q@p!N>m>X8 z_+gK>BAs$u?VZqwSd(2U`l_f+vajLahl$>ZU%rfg85(2X0L@AHeoX^>5H>!bjoLP-3{*m}DCOD41=)2Cy|Sk)a2j9=q<Z_*$$n9kTN<+rwaeIsgsWS`>q}YK}3i8NM)-^6Up#6Ewq9w7+PB z{V)vnaJH^eTX6wqB6{A9HQNzAhfVaTHI0b0Cd5G&eZ3RD-G(EVL{U9_JvjiI5JiU& ziLt%`;O%a<1OC>~0K8wc!9<{mk#+Va{5X4`t;4sQJof#Feo5G-RvXqD+T#v<_NaHP@VI+~HV-=rjd4TIUK@=W?k323O<2cbfLM?F2=>Rl z$*Vm(6mx|`Y-=o*<8F3u5I$6=u8uW}UhTFCJlo)xV;$55@Say;q4UERCVr`AfsDp~a|mSU?!Pdc() zC!1pPFxlZ@Q>=1mMw1E%h5D|j$l^tOEU#VeY;U{{(Uz&lU$C%U+Aa}RMZ9c zl7Zy|@GQ3oT#E7t#;*9j9=YU@X3Y+@_k?fTiS5hOv)mw5(S%U+CjQjM$$gU56>TZdR? zeGxy?v}CL-hQq+XXVgE+({j&#Z~{aZpS#*xj(+ZR(vO=%}8 zI(F5Zt7R=e9SHO)^b7ZhwSfG%cIoZV2CwkQ$SG*ghIa}C%Yb9Ja*JA-ugOB6e?*Ba z^_7wqF@pEu{o1`F3qWosa`BZM^9#L5=*zz3TSA5IM#I^aC3m+rhOY92-^AltmFd#+J^W5_64VZJPwy=`(yvrhm>JN)(7#07L_ zhx2%?tX`QXvn#XZP|PNm#bzCh*OQ5T9=qCBp-vog9KUfq{u}OL=@j%lcNR`KoNB~Hn{gCX-^j10ofRhSpZtL# z4fJ&~e%7mGmxL`xN{ydKgpo=Yjug6BF&I7AptT)8H|e2oel-5gm>==yl6H}&rk6v# z1Ly@$oBPqr#($n(p8XnnnKu}{+%u3~+>LMKaE_L8EQDtXRSHWe>?`tYjAV3|9O(kg z^MeLbl_LRoB5j#Y97th~p!*`!nfXqF7&%6KpxGOpN5OGSK1BPkGw_tjI}dldkVt^p zcpH(~RL3|Y@~%?7VPrYgO_;Zu^4267k+&sU-OL%$9i4cnjV*(M>BO)EgPEnP)K21O zX~@s4l$~K@XQ|Y9%Fe`W8)e&#>>QPPMA_Mx?V#-3K5i(&Jc(B2QIPjR{O7Yy+(CgU zq{I$nGH+P3euZ#Fjyi4liDG==>K!s|u7TfeKmJE3}8xg3#40)Q+udorRRL>ws< z7viaZpCIuEbDHrvhMues=+y@ki3SoQdVo{Tx?a)z@Y~0qJ*OKo#2%Po?~c>*i-E{M z55Zup=bKv^%@0Z7eWv;?@2*dUO! z=k(tA*(U#sY*sX%mH6a!iIe%{6$#Vd4~45h4}r0LnflKTBoh9Tuf*`@P>)~ej&I^z zbKgwjnw3`**W5qBu{7dAwi$DJ#E58jC z-3&XH|B(EaPIP(6AoAN;-z2}?avxFR%1WZdY1f*R_@fs|eoH?ChbrFLC%^sgSZ|l! znr8W5v)=an`9HnhKL3L>%m0G)_QR=ZmciEBd+#AC-*_ca`A@Dfsr=Epe|Ek7*YC&w z-_wWM>i(0KlloBW&1shZvHDQoevJU#_}A{&{Pm_Z%m1kTn%*0c z&i+U8`~N2UHMh~7ip$oFp_OpvcylETJok_6*MvSR_T$ywV7<($8-3)_>AU35d#%DY z!W`oFeHx(?xdaA@pjQl6aBw|m`7ZepXMEvpXrI> zg4!tf%Jf+uW1hGOE!4ziqth@|9vz(yOot$WGyy<)<6^k>{ouaF>o4Lb)YL?8SnmLXj6aLXJY6yB2(`(KB1jFWqxG5 zDvAqV#$YDEr!bf4ezwn`Uj>6}cx!#Ex(@FMV>iehS~tZ_0wDT8Q@yo{dYbHZ64k0V z*ktzvbmvJFtZ0**es*}yQYa2}w@IPBVv>+q6UEPs?X@C3`-HuUutpJ9V+1ypPm^+r z>*+^%%fs&&L$|Bzu_Yw)bwNYijZk-ehn?gBp!c(Z+|z*vI;;}VJSI|64?n#di}xq)m-6S)IH%P9n{gpQvhF8Y#+p(qb_xM$V z5z}ZweN{#iYS`r>cf6wi=(lte?*ti&UAC2H--_8anC<1+AM)(idA1F+(P@l+-fvC0 zx>{kIF%*`1of9Qwb-slgUj>GF)ayq0w93&wwk^?S-dk9#R&t=qdPthv7HJF))z3R_ zT)j@}$@=X_lAu(1Nz!H>2OTxk>wwSADU`Yzh2WWz_CRl_H*?MqnGKiOb+&rJ!z!H~ zc3u8S{(eEmu9jOG6NITxLPxS@XOMkcY!5fik+-y?L&{`3ss~(tf+eSvTnhPHeasJh zy@;7gNa;u-M7N3mqn5ni+p!Jdy*R^8!FkfmK)dJN^-V4#;3I_f>U!eOm zN~jlBorUDV94KrCOy-(;iT&Njg0RrMvl!hU`q)0aa8O+!g-(cbzUN{4CCzIC0d%OM zjm}rtG2f+U!z}i*5rB>F&pBOIN2ek5hH*HT>A}Fh|AKXQQib#Bbp{fD}j}ro#5Em#IbP(-uez2cAh5>&n!MX!#Z;N z=0m7iP&D8Nun1;YMhz+ZFgiqv=4Y|a;3stBlAL5rB75io6|$Z2oZjU9EGl>L;N?0Z zu#`M_Gc6?}E}^C5tBXbMrA*)TOn2gx=R>s11e2|1p3IigT@_Xx#o&#oEru^iGZ^`K zTttPr#{HDoKrK1VoN;z?n>+-4Cwwc7F4)6z9z}*(%j0JISf`(z@Uv!xwZyL%I6X?E z@_-uuO|E|)=>EF)2LrvgWC3}Jdo&jNU@GA58Fkqya#N<0U@rNLWH>FA|k|l;b%FT_v$zN zl33o0JUhU%hoS_EvCQ;Ke@W+bgT%X#i9|^RU2OP@qwq&OI5W^CileEdx!z)s<|<2) zG`AUIh9z(lbPARcpH@=~yzy33UjcD~L~+0i2@ZsZ>cxPB3P9k+Y-{pGp#}~!`8Z{I^cGRVUZw?S}$X@sx>K!$-BAR~@RtyzlbY+V?Lc ztPQ`=^c#5MNkZ!Pc=kCw`>iV=dmmb0o6lL%Dx3M78Li4PpEIIWndY-KT9siwr$?)-=5t!KD&2gBwx!W?v`Tut)V=)Nu#2j0hJT7^vj;T3V)O`>TfEUP{4wNj{EMDdl2uAs#< z`%T+?L=*{9`%>{#A>x-nWfD*^VJ*=LF{ayC6XfIxZPAK6GnI>}Tr=gsl*3HrU@FH< zWn(JaOxZDIH&ZrD+00ZHrn1abCZ;mYR0gIp%#;;VRx_23sdO`yhN(0&C16T`lqDU| z$2Jgu2Yeh9fXYbeLUl|I!2*UVS@5HAnomF@FDDzQWG_xq@a)|%uimPnd3FBzG_Tr6 z!@Qb`^J?J}rhHCxx@ZdB`|63K_JY4Z;oA0N{B`|2dY$3xx8UoQPtdHL#7x1CaarGbfev}j^RP~oktB{ztAJiwl+foeV=EIVXGzg%DAuoHlPj4lj96VcRjRFI_qB!mD1@1t_kyR5&cONmB*P^O|gs zCulm;^jrrJb9&;lIX&^&oSyh>PEUL`rzbv}(-WV~>50$g^u%X#dag64rxN?v>MLpY z3;X=SC$x99a1u(sZ1r34<3z{qz;><5eB3B>`ozk86})E&Kf=S zu%q4xUJY&(*Rd)`@YnGiy07bJhhYt<_o%@JFFWRE2Nkvk#sJjd1`nIs7?`Q>eMzN7 zqDvHfJDO_VK(458{n(BlP^Z#r0?KNG1RUPOc6vd0$BzYmf`jLwru3U9FrfKae4gZ| z1wXFT0vFRX@i5nbdD7?KqK}AXW7xxcCvbiET%zf}xPhjhN1C4aGwx3WhZV+?ZHnUX z!brve)0hGQ3dYh*fpi6+zN}7l`P551>SInAo2)sPjME%+5Sprm%Vple*{8c3W$ljb1LT$l^i}66>4YNA z;NPXt!8A#`HeudGmNp$6co}(|W<6n~$-BP26+?!lhg!>Ng^GIXWeqp*p66F~>Fk(% z98=UAnenUg=r}*pnn7!k7`~crHhK%82SU{Qwz#m26k~6p42wMg)1uzD#PG;IAoevq z>RoBLqTxVLJA18B?+P(|)RF{Zof%Q@o7}ss0W1b!X4Jb}47c?Gk+!p<-Z#YXn|;7i z0NSFseg7u|_(nP5Rfi0F)cd*^*31k!nUNj!E(^>h;DjVFC+b}qxSfDgl0Zk)`&!^y z0!~W;bEDo@17!rfD+$btdjA#}Nx+#&pcwVO5)cVEmje<1sy7NN5x?3fYcqjBTcpJM zbZ|(4yXGHD^a8>cx~xDZr~q_TH_BzJ#6^p&D0pq37mW3~rmjJhy1Y&3y&Ev@@~+}f zHkWsGLnd`P81=JH`FB7q!D!Pm3(hnA=Ip{~?}1|6FFh9)u42|KhdKl~VppI*y}eQ&R{_{T|LK zqcEWgeQHgCPrJ)W{%5ij^*RtAI-IiG+bFvYMs@(!RNAayUfQ>m5?kfaF>%gDYNZ-1 zFn1eeZDOVqfhfMw0Twa6Y?r?gc#Oa;EX8G43od^QH&eMM3IL?J{7tyq$|HFIAl>C( zh1;>_;l|Zgmwz?x*9IPeM25@11~+p9_d_Dn&9D#!DxmGn-q$UG%c0%P-es168`|9L zU1~v6b2NKjvjon8wl;fTwFHKRdh(jRf3pN~p~21GS1bYeP(6Basl^tnHBs~zjV*>j z#L-o~b?^%aBIh5Lz*(Fkh+1TIB@g*E)2aU@+MVd%O_g`^ze^X5#sSua7Vop1f%AwJ z_W^(MJ}awB$?FPr5AWOW0c;dZ@Uyr+>nq?^KR{M7nB4=c0j{Yrz6>-rP=xR=kCA_Z zYA29@fL?_bFX~Rj&ppchqw*b!N3_p=WP4Y2%r$i_ zF2c!G$dz1$ieVMvPlYb;8vX>URdaH+YDunEtqs}OOg}~Lz{y!%5TA8CG4Q@_f8m>q zw7!(*e}>*ah|w}XtWUH)0K+{;o&2xhx1s%M1D$?s+TZ8)gL7ET`abbuZBoMCHW;&& zIhPe9PiOvrhWcN^Pzk?2_S?B!$WJoWEifEDmB36tf?R7AIwnvn^O0r!H(Xm3wbGWluwWD4g}(EJMVngdKlSYVx1FtqD$DHNi3 z2$vXOtrAQlpx(J;xh9uYqfpq#lF^r!PYo6(@xPMGlrK+7koy%s`^sma6t_+o_ZK%@ zQ6IB`;sg8|#2R-^D_J`rr^XY;nmQamJc-|mIxOecK9#DJLw6L4K4Y0T*tcK1H^YhT zv4~-`DFpS&Yk{Q?LBPCsTR2E?c~L_z1opk&0Hxuo1t4JHZy2C-T)qHg<re~tiS`9%);djlD5EYn8Y;>llDLrChacoT1;jKJebUO zdDmewCvX`ib6nncFzE=Khe?OayB?Fdf#I0Ub$Q<<><4X_q4nO;V%hJ^H=_&Gbb5foT7PDA?dHVYxPv;u)bTi7+D><9p z&&6pu#OwEly8^d?Sp65H-jn(H4-*vM8qkCj*qqBX1yW)ROlJkY%g7%UPIs8s6ZFM# zwO@P9eq|3&Y~@iD%l^9G4~ z6;#v*Y>En6M@<8D4n+1~qav=AMG*C~Wj4+xYp1^y!eB#w3c!aUzb!(*q26HaYDyC$ z?PzPvniSUPW9y{QUbHv4O4+0<-8&WF*l!z4{T?6N>}TuHd3n)MWz+u9K0$GBmb*Xn zu_iy;A{X__oAxWQkFN5uEk1UAxnkcUZ|acQVcEUY$9DOcB+G0ofvyKux(~|TSnOjA z7YW(^g%6J(=A*cR6=UMY0;t^Y{?N2gEb5lg+U9{$dD8*e*jJL>y$b8b6K(c@)9>CU zySMq=8}M+~#~zxXu-$%l3x>dA54n7_cs?{mVcVdf-~FDf00(jUp=o|9>|;Ni z36N(1au;Zae)gr`y;^p!^10W^?1@?;{pxzf-K-#Nqu;$w!4MuloGY_mkq>S+VCa{{odcmQtdp1Yx!VCr zJ){_CY(FgxYuD)`=>Yh2Xc`Qp*D}cYPHg`$9|IcP7}_iO)H@nNN2S1p`m(dqup;qn ze9t-(9#@||Xcdd~obOQ4Kc1H?`o?*@XiGYY3;kahcs@p6C%=%{(Q(@7HC#a6h#SN{ z?Yl$fmtx>7>k9w{YQ%eeU8~{Y-vqQz*(6v<^M8W+{IE@CJAl=iPIW>QbqtZI)sRsr z`XDVi;$yw@V;JAW2;ib+tQd1VF;Fn|v4qs+%}SG52qT`0k#79rAJ)MawQFFuzgj>> zvTr)cuFAW@&tOn?>JPv=G4DD2a4$s{5Xd4o<0kk5VuoU^6!$|v#tXRlZKET<`lgP| z=+}`013L2Dd;z1Z4eX222R%IhRIP`mqk8n}F?2S*LR_{4zesLQLwkGeZBc3gb*PYD z%qWAZjG^xRxnMb|S2%Ai`l^jazqJ3{0G;){ZyccDkM)eT-!M4u9ZemafDARqF5?3O zkKfBqJ5@ifnI~A{*5v(2{R&8TJed-2I^&2v%@Qx{HRDHT#D4~FIE}QsF`kj?pGyX< zXC9tS%P(+w;}dCORq!FQ3OXK?-bCL)7{Av0CwxDukF^^U(>DW{rdG5 zH1w{VLHqT>i+%lCjPaXt2JO!RD%P=W!1*|8=a>EIcRb=S4=Z#6FRY+>lXl0vH(j(Q ze|DGLQpwHwTNV`>Q_%?O*na|!{ly)n7QOoROdfYx9lH?Hp~Q*c8S$euKW23&LZwzQ za+-j-cR?=Q+t{Zc3U(*Z3q@cTs!~5YjFc^`mP=Lzmy=6bZEC%uJ;lxZ4oVG$)HfNm zDZbZ>0fW)e#dPt0FS?`7nD#xB#JSy|A3kuh|7Lfi!Z!7b4!ypLyV7W$a11~%{MRsq5X`^)SP*{(=DXy@iN~|5EOAzZHo}k!4L|tpT z7WNhG@Ue3*_i{yetTmZ_Xo%e>F-! zOKRyyC5h0z$;U!mM0~E1ettueAbYco`h8SpuWl#!i|qtgJ9)J*9hMM1yJJ3gCt#h2 zdd%2hLrGBW;evpZ19#XF7%HF(4+g(aVf7~oiA2Y-_%Mu<7uq9*j!1zL{mCID_I*Ay zEv4q)=cm-%@kiu;ddHbmFgzm7;*EUVS9ZyWWLXe3;sN7+*GcvQc&~ITN`$a-I%I}{ zy+K=1e?N_A+;l%^`t&O|eoc3S{oE0@ED9U*r)AMR<4FvUA~Ef{X}I}Mk=Is}&s>Zi zgUC1fFNw>77A5qV6J-zA7`%;wphJ>t4jw%R55q$mK70WWXhR?T5+2ZQGWZodWZ**= zJY?cSH#}tFLk~RQhD7iPJlOH!C_JFadGHuKePxZbHj&ya{lW_askpiE`+uz^@FV&pbhLT?ab% z(KIpqt^rB|P#S>J#qe?ilnx-gS7;T(|89V+0I~uog9n2`PzHc90F)_)=Nq6*0A&Iw zD=-U_S&+;^&-{Vg$zdNPZIHAFCSuYKNjoI7171vKLoyqZIf3tCG6#}5kaPsj$D{+2 z4oKz((7W8Vxsc3-q!>hOPJ)h3&O-!YP=PIYp@cjp2V0fcC+Uj#uHSvXEwM#XZ*_?8 zgjNaSJBl`^kYABVNQtLs;^N9~b(|y`f}E$GhN~cmpu;6~lC8N?6xd{2$J^Bud$Py$ zN+~|M28sB+Dp|eN89Y;tbOlepnO_C#ACHvmRDX|iTYHsD7H`rVZXr2?Q*mgXA(4Qv zK@xMPT+)*fMP{w1vzWV)I6!CutPJ@t^KQ5 ztS-SKkgYH`dvxWMnAXat!!MN|cPq*FIo#NW5+(F$rx+P+DpNAwbgjlPC$1`}ZS{>k1j;d$Da`wfxL_-i;2Z5o zl-SI&b>gB`X(nj^U6kSwA0or?$qsQH2uqUsG_n*R?76f93xuPLl( z`5-8v9RS**iOW2qlkCuRVbu6Uv=SC`k5~>Xb?`lO&l!IZhwlotjbWXRCe8znGiXcA-yKe$fD04@Fl)vBkU#Yn*6d zD-q`7P7iAnTP0Qr-SI+qDxo_b?2Z?@Q(1rsGFhGT93ZPst!YGb;~n@;A8QY_0DIJj zu$t9407~q0E8-gowRBnSy&W5SDMabChFv)G<_Y`G%Lv&{etYaBv^2R~^sb z2l~w~IaE^zQqWI~s~HO7lz)ki67Xf9Q2wQkCqW;b1Cx;GP3n`U;$I$53>KdyIsfVg zoqsSZksyru_qs9v7K;lI1q0^a^TNRS_eXg4s`opLsV9l=MmFQkfj|vm8RpxyL^n7K zSxZL-#xS`^4#fnIxJ*ib){ZvZ*u8^-m&LrJd|6IU^xA~i<;kLHxDlrzyl+g!e{dRZ zrHMDcrXHy`WdcTx2{_MiN}u%C9>lw#i;ejaq*EGWR1(-ZACun;6pH;N$oVYT<)b_x%rr=m8`qQ*VQ5>hXzX5x1OzTDoN0+q@oYLb{ zX7uX<9T39~+8Nrwqiuk&U7=09 z%z}3-6Dc#0W*-uqxXdwxW}Y+(W@6Ma+?;`XVFt#VdQX~Y&thBpPqdc@v)=%9)sSvd z=aCPWyd?7i0k=_3uPhZ8%pxgfo4mYl8nuawZlPk8rKXg#9%s_6IJZLc_`YuhudH{4 z=Hi}+7{O8=^&W>ZzZZ!DCYiVpTPAB2(`chct3ax*qf1h~3TrX-rTJ?(4etZ_(l8!g zZV+69GXEgbnKHi_e-dpTw%?@xdT||KvySe)RXNb`c>Zc$NSU9&7R-6x$9nu&XS1Ij zQjjS`vIKs?EbZ?Hk`BL^1k!DNLQd3 zuYfgC13F$kr9Be{A8!zN@NTA+;*NWm=v(`3IZ1^IJMPow*xqoXJjC9|a5!4k1ch~j zqNHfA(I&f$(R!L*IqwVkc{2v}ZhiJaGS(=C^{xUxpgvA}?Q|*Y*IWWIqIn%L-k5lt z$6dLIh@+<161c>C3knzeSe%=UU}35z9dh)ujkmgJVo(?rS$%3Q2FMYYy+-@|a;!a_ zTx$5VE8jOHvsy87CC*-PqtCsP$DMh&9Cz*9JSGi{DK0IWF+L8a-DWJDyLbv;Q9`}W z2HalriItr=D{4S;2=h;-dy8fDx1|)`#)wI9ev*!JU->f6A~n2$`_cDX`u3v_KWXK` z4)KMP;wD3@VbKb@5LfbH9w*^>F1oL$xJl_V(Adf*hA>X7)YWViBe-KsW?#4A3bw4G z9KdN()C=MY1_$%9t_tyW&zRon6#!sBpTEp=S#SJGJl{WgWcGi-zc!t|7QdPM&%~h6 z)W4@(y?wKeP*%)G0pamqnf_ET5DCpXF9v;5)QMR=z*IAO6uem-YVT1>)_Bp|&_OA* zuZKFa;3b+U`%E7>vN~GylpU7YXxpp<6#R(x$F4D&cLFcyAHXm_73E%xvf#B;sGc#$ zv2VxY)68~M``J4KLs*N;7EK^Js|ID)4|8d10mux6z+8)gf0p{xnsT4eHfyx6tZ~k* zK2}reV?mdnZTGPmK#_M8$ZRU$tGc+qIu?K8IN3|&#t!3nzk=f(Od0RACDxO*H7x%G zm0ybG?@TGb`4?78#|E?g(0VlWb4U}R$pux=-ePW{jD45^Z4Q=;5j4P2)Cwz-Htww3 zEK|nD!9B8>=^p=SxK*ilkS);gs@4y0r*bI)2 zI|>061+exJI;1xoDO(ru1;;2p3ef){{P0x#@I$He|3tqJ_sy62hX%!86O@nNklgps za&J5|DBgaJRAL z@Et!mD84Via@mPN^cg>E`6uMB+ep58Z-D&e8z6sez`1{2Kly8Is{9qetO|!A{GedF zX7+spg)j6EY|2}Eladt5QHG?&x%hHk0;5qH5*ZfRX9F+R=4JDDQ z4MeV6PAzi5eEg5eThD%-y!CiLdFyBrWBOxLP~KXt2+c}KOZ<+cyoKUY9T&K`Kt*Ho zaJf&N+9)Nq3_y7`$X-($0PabjL1LI+*nq-S$wrX9Nahj4zdN;DG#TU~l&_$iyKymH z4@J3Z6v|mqye66|U)|XIwf5lTucR3FrakI!dWhZb;?|iSzESIqY{s~so6-Fy!RZ3# zdSiQ54BtUIxPOF#r#6RV_Mtbj)~Bkp%{-FioD1=6bQ#GxfeSpWeZXEimBBml`f0=n zgCCgZt%Nc6F&EIGHpONgZ_etretYiWJ0Q#r)U>9lyHha zzb3hX67=Af`l_F2&}_z7T6|xewB8dY->>T6d_Rtqn3uGN)ZaY7;pn+C-j%dF!8>`Z zSE}?#EYHF`i@o^8i9tA6<>uEG!`lB2t3Mlo2X52gL z=I7Hs+~Wd$K7LnH{vz$afxDffCZW5r>c+SSQP(dX`8)1LlK2fu_h~CB!k%FZfeHbt zNAo{xr3Nf5rpxF{k)BB{sGQ(q5j5xVF|C01C_q92)+)0WLrd7fcjNR4AEB1862AkuZ8ehlhOBS|^3w}&{Sq2gcVQPwtCexSc1=YJ6*P|dpBD;dQ9c4+o*uZ(L8COa= zXL$TF65&G~pq9eAiDwNQ6nx*jEzi8G{i>GKQx>G7NEj)gR%yl5p)QbXdDjy9w~cV1 z>r!+N)H7g3`7N<;(;!cc%q10Lq>=Q3`jLSV+mT$JM_t0N-C3@P8+`7i6o(gTJ6*n2y?j0gc5p;`W70O+&=bYgf^Q;lP72t8;0gjeQoy+gzKFoNDd0Q=pN@wu);w33pdFcHm$#W!ua2)jZmzGP z9$311VT>QFaj!*Z#jiqB>(ays-d?B!q2MlzHN4E49ku4gZ%K(e&-KV8U9}ksWc)+g z5&zm5*rxNv{80?+SwVGLShIUG=!~Q>`c(=b) z9%jvxL$S2@O{To6X0yCEthn4>MfdSR(Px z(_~s$))Y7&=2`2vMWpb7Gyfm79({g+lG}FgiZH@ zGTFf&L?WyxTu#esO7E4>ai=1Ve-Dr93r^(^A#~I>>p_%L71awZ5Lla{1~PhN77%)5 zO|lVcCdV1W4lRp3{4$Gtcnecp2{ZNGdPM zvRIp>PRuw?UX;)7yI-Q6zV?Pv!%V=q;9yh7SvTpSc46wO#eISX$9ci-mpYg4)3 zxN+KuvpceIUE-cN9Qo2H$MCWbrKn>VI~?5KQICFDUOee#wzDG!E5$Ykxun?Mo=5hQ z@I!v9_1#yWY%hr}PPu;ypLX3{=wJfN3rl3{PQTTCNKo!~E$btF@{r7xc3E+OMX(@2uv6v%jC zw}7R{+=^c)0N`uw2;^#BGEy_QG8{N3h#!!BrU!%Wo#Ul_KtZH3Nq8QnQtwUY?n8LmU=28f%4~DuI!+(Vw z!{5{VUJI08ZXx@x1n>h`sQ2LlZeyFs=KFbfBP_-NyPCpeML6IPUw33qXO1io zZPU7%gg_Qd^2{it8LS!aDJ5Iw*a52#4c`s{nOl9r5m+<>=PdTAAOgwUsWI~CwLT`p z;#S^+dlY+NYLLr`qqN6SY#!rHMqPUTYV25l-+Vu_(L#~+F)b9YeGUu7IGpjz#+&}E zWHn1xC)=uV%NR|XKatrmnT@xZ=MZ=c=ZH&ObZ44@OWHzr;gW!Y5iX{v#G=kT-pOss z#}}Y2IsAg4<86oEV))g?@b74>9_aLGQUZHVrai0^9WpDSo;-0Ow*L$9{xmt%D}e0p zZA5$Yz_%fp2Ht+`o9aE0EEMr^|BLA#q^|ZhU|^ znqE5Y}eaH|Q=AoyAno?*f>3I2)+&otp#1pk!@ z&obdQg3mYMHWLmSz@gbQ2xd27*$BIlVA&=t2Vq`<<(M!B!rTONn6O-geT!hZCM*wO zb_C@igf0g7v^yZ{X>J33%)DpdV|(yYKyD^o3h28f;NXBZ1hgX{hXdO6acfaq3>nu3 z+U1szd2PV7w^qHnY2O@L?MmMK_yYMR$l(~DXYtAR-@ZhbN9dh2>`3dQ&$meZ&6Kr= z5{4TDPMC(*38Kw;25m+S+FVSu2^85!jd)RlC{hk3fFd95Pm$ccLJ9p0pv!sC=Xoh~ zxfs%mi7szV(&cLzGr8jsPMYH>GbSyxHRAnvs*9lEVoU#|v|; zDLI@(KQwYOQgS$vt~PQqQ*t<&{@KXMO3C4by4c9ErQ{H){sf`+6etnvj}V%j0wr?2 z7NI#QP$Jk;ggR27M6%yTXl@FWh}OZ0HcUVxq7wRZku`6dqWuo(%g+uPHHnv4tX(`0rY@E9MSz3|gl2Yh|&BNki=rs#^1QjE9lK}wTw@^d*+!_EQ z8*q2E#yGGLABN4mh}b$#^AsjbxWD9}UpT;hFQVx)ovPf;O$X{-;==QUK`lwhU(^ZX z+e+w@PBb=*?YH_Pn*-N^ge|f6#4r(v(dGphHb>DOwfWc~57U*FOu|8nC$onShH>S~ zBgFZAulTvf{sMVl#DI=o_s4jlM!_f?AK?vQBVLla2s@%^qt8aa)}&5B;jq>gb~3eAgVULhE$bvg5QLg7Kk@jdBYJ)>3mOjLlDb%`j1U zy%fwOX;(lm<4J$#QhGm3CRQQ7JP&TDQ*MqN@dj>bPglA6jb362&6ZFCaM|>#05uTcEGcQ_WUdsYDAUJ;VA<(WOVkd1MywtLnVKt#D;;o zYsIKRnaAU6`r@^`MIt1|3mP&fgU1UZBLV$}vE6_jwrcLQz||l;ZTW7gML+w(OcbPA z;F*2v6@>T*a@ifzkHgY5_Lm5{5kP9j&*3rDb0YA)vQ@#W$WnmxL3ctwa%FeIonWi< zTKvMU$Nm5>YCH2R`ZTFKp`Ud*`9;Pz*zTLB#vdMwE~YXZXW{iTDz_Xr>$y`m)utcA zm-dw<;|G8kXKc* zhh|?=HkiKJ&p!iRW&$mA{%6~fxZb>tBcMNCl8K%{ z>52lLRV(vXj6}H{Pq4*B%>wtRI#<@HI%WO|T#==>@qfQK{sd<&(p!A}=3AdE)=`w= zZ<7vHj2Hc^8{hJxZg`aLqI|0FH_j#Ik_o{vB^0Ix2>y~2!5DnE7GE!0Erz{_rKbo> zn}Zj_K$Kqyq;O3!5U`qY?}bEP-@OpsTYTpS@Piko=_>r)SYrO_?E?=9h(8$}d#jM- zx)%KrG|TyOZ|Z%{9T&i7K0ZGa*8T5fK*e|^L{wB>yUa2UT|l4y8x}hO#SoW~*Px2G z@eoeGU{(hD76K#(F2^tYAL`x&Jc=rN9PR4vBpnh`L1B!F3W`cx6J=`wNkMn0f^GpJ zf(wchMR6Cp5rsgoyCpScK}KN)9UYlbXP8lE&=H6znhubJMV6qlsUR>?ZCCmZ=*OPdihN3dZd0C?>SrruG)+8WKg8kxN zzBOz`;eQbfw@61OV)Ed#DVE3&05KjO7|$m}=Z`;)p9@nX*R?0-<2p}10$?N2trnfg z+$()!J^BFBLMv*=7|nG{vsa(q4D;`Q5FQ#7bq8?J^?l;|L(wPogC@Ry4q0>k3M1S3 zhbih;XddA|0vzztuTiLe7+S7-g7EhR3MUNzlVptbXeWXJIxhzrLlA?D$sxVUliMQi z`bkdTiK4IYPuBR8J^ti~KV`?C`pD^3GUg7_doq1G2BiO&m3&NMUg0=i?qtv(W{xIq z2XGw27#zz=Lq2z~ zUjinnWQA`c#_G5FM$4MF3>Iu9>=g09Oe{c^ZGT>v@ z%}QEVmpy)-b;bN2F^3geKb&D z5D%v-kcKJ$96fp?AIIst&yiw{ypt%WV0mexOuDXMqKw0i_!4Cb{q#`0JXW7K5Wdd% zctBGuGOQzZ?tNW6E(b$8Pz1LIMwXxysb*>9@VE>NJFCsjM)_{|2iOo1wjoBNHDj7++TjVfakPuX;1mqBmoeGF!lmBEO4%${ zYHwD`VS!SkRx0(uD5+xb5G(bkUPD!^lqY?HsDvN%O_FnKJ$gYDLpR|TgP$Cy_q6zi z(8|;Y?8x7YZZWXYXr;c=}BQw3)fPQ3sJ_2B09KzlYKgN6CAO@c%SG>ahi1ohi!&$o^nr97Nd57S(J7WEExio4^fPQr zqZOht=8^wkH+eC&$G+B$N@{<}dD|=6uzrAFU@Xr7^@iz9(_-WE6m4F=Y@=*OiUpq> zUiAqmI{S6N&3s?KPWfg{d90#;PE^sy)>KQxouIR%PiW&0@8I7feL@@m#zdJm{$+_W zZT#+uGHv{h_IR0&r4#LR+D+X>sIhkW)SIW;=Zd`tiqkCG#(C0K$!#F*$-T3|yPD^eQgGdv|#S?*U7%GPk+vt*BgJ0DG zjg(?fKbWiQVIjHHS*R842TO4WEJe5%jKNK0o=2&K9X#lUI?8cAA6;1+Q6Fad{V6*U zLud7b-R_|s z`2LglzLfO-O!odu|C#@yoVg2+Q$&*D*>Dt+=huDRV!_DYZR@jaWlK6${P1)_)z2Ch|AKyR8hCl!tla3 zwybFb?9VTChfesXpf}BGo3RP@%$s`Y3tP~3vL;{gq~6t z^dEi?J*6$UeE6;OGF1aH$vPn_22F<6w=w4U zw1shD6kPrX5>Ld8oq18J#B^1B4r4~!M4Ur$Ng8>G=;L`XP_KH3Jh4%1dCK_RFVu)4 zvSax#MN|%bNQc;J7IO5Iv1CG`S%AXH3gTjn`z{fqT7L)T|2RZ_p5fLtFW-q{^0|_F zQK%Q+6nq}ef5?t&H&7$EhElgca??mo;wsF&f>x6Mo%{nHZTx2Z5X4W`5kWGa_BVJ8me&dV5sJ@B){FW#Kvv(Q@@$-8oeIZ>=f9d?ybaB2C zjmP8Z6kS7sT^u+*gb9+hJG1eYd5aKRi-E?}t-!?Pcqz;2N-;DaXR%o7!%}BdcZ~@i zKQgKP3ghYP*=ktJE{a9l@ie|04|-eTG|oR@EV|!r(TX-=kka5)0!A-_lfyR@`8;*% zO2hGBibZp;tiKM=X9Qg0YPVBOjAgaQ4fDo*Y-YcsO7u+MXV#yh`7^F#uZIpGDY7O` z55how8I>l;| z)L$k_COOj|n2g!C1zH8)K&j$IHlKSpa(%st`!DX}xbJ_Czo3MH!55*b&yAW0jL9x7 z>;R^Ifw+W>?sfNkx9*L()ss@7dIYW{WcXd8+~87c!_8Mn;b>~uxWuLUw@E@+3j8oa z%7s5IBi!nCUoS~l1a@dCyN{t5gACJ(OSZv*?jlR{sPRL8HV8mECHNz=f>AC7QDbODCmB7D(e})w35at)7^I`~NE-8T>f=9hA9v!_@PM?(4P(N{c zxm1BCUF3Uno;dZDur=wQ94;trtZy@4)&ui{&2|A8^DbC@+ z6aG~jaARN`)lPKWmG|Q0MDCArd(K|rA8}b!yT~RA)@k1eECdK1UnvP*>N& zP6{7N9d5xiFL+wX;vKs@K&)Lvpc3s?sdExQbb_&J&s~Vu4DFJcaEMY~-t-bt)M zVBSUe0~EhbXsy41^fN!f@4kRwH0b~|`Hur*$3MHk;uhCt&ht9Mvn~M?&&{usg$AYR zG%y*y$r#SY0|@vQ&yqQao3u+h;7Ua#h@n52#(4ElV?QPyu@~smK96_Rfbqy?XPA8* zs{crK(r1b<-c9Pfur-3mEY1tw*tp8`6ei`1+(k4&lM%~jM<0*7zi=b2nPh>emg>P{IyvLPqa0|kN*L3P2r{=m2B@R)4jBJ1IsDTImvf6Gj@||5Uz#Kq?!vFHvZLU# zz(H}~iqfc%O%dRxutE;SerrpmLLZ9ymc?rNQ@pp+tR^qeng#?Ms~IAFiU*hE+~vDl z%H1X9)~UNB{rTvwQ&(+AcXA6yy2_nu-Oe9; zX-e^KCBwdxPTzf$4e)@XekTS;!>7PDleJX*us=afBl__zx5eq;Q^ti?vD5s3jZ79S zQjI&C(ICtOpvLWBO$`iSa<2b6=2<@@5Qc*fq++OpDGtJN_{cCNTtC9?-0Bub??ViW zi%#Mcs9Tkc@i1j=@W#7@tMS=T0tByRr5q>$hF@c)K2QP_A7G{aPy!sUXQezS0g~Th zr6JOKM6%nt&-a{L-QiOAxpNO<#6n45z%Z?6j!?aoE_IhHcVD2zilL~ywTN`Dejgxx zb5efiurOT`>upM4L$QT!1>-#aH(#N>U}#z)C3Lr<-N+pdRQ5Fy zMKK8NyaElF)L2|h`U?Fn2ikq{z3XfhNAC&508yvllhjr@qmXUdD9V>J3fZRpkrmjc z%}${L+qBKBhHctItiU#HEh`L>KEd_mZ1vrfy9YL{y4$6Xj!L@>DNqTcvW|`mi^u4m z?>p~V$+`2n2DWKC3yV>IAwie0Vm6P-uy3b@@KoI2#@v3m@9=LV!2D&z(08dsm26#Y z%HP(_OrlC&zLThuDRH?T5xtS}HeSxJRGfhCA?=LUd{RVize_+O(m$Zw^utspH?4~t zHsyvii+i%TfMgTRWN1Yj+QFQFt6=~f?0E1?YT*Fv7mkb0@*up1nj6~S$BhVvRQJ%r zSYbRgu7j?qVdFa(%w22b#h+Ii{o>E7jGXv$g^?M5t~4y;`LJq`@)v_MNEpEuSb%g5 zCAA*)G%yrK(anraK<6%{7!&bonf)Yo4?8G-IZUlft?VMdsho8T=|>>(-VlQm>_moR z>K7|xAUx`iF64cFFVa9<$o#MX{F-|e$tYxX>I#qga5*$vuBboP6tFW^-B5oAm-lLE zQq5=<)>rKVkNngSI+KgiS z*(VZyE67-X+KQA99J1Y767sN9D2BfJPHc^~K5+bTFg?<5-awD^bA9QNzT{VwF3k^AbV&vI z+5CX=nuljH>R=p}AlF18f{a9zW5R@rf;WgBm!mGvr{?di)CG<6m ze@@i+6|eoJokoH#SH|HCE=xMp&R#KxchdO%^{jbydmJ~F#-E3?0i+`CD*iycQ1^kW zfQ5f5={#=t*ez_cMHb`Rc-AdXPnu3RA;K_|kq6p%-aJ6Al_Ph-73_EKXlKZ!FOMU% z#<=+!HiVnn<5q&TfP52sr;P6pF@pa_q~0AEuLjNz7lwz&GeCQ|+&4t35ZTaQ_-7W= zZAS`7ST=QqRFT4}9>6NRWg=Jes(Qr1Di?YPMUB0evtjZ(B&vZIvVpaBLwC{GPVDP#=*wF? z+JWQqI4ft3RWuRC6{c~WM3T^F?ks){ZST*%?8(-1zHHV_$zOGn&bwb)NT`nEw=ZF~ zpoh`6;5vLAy1)Azs$oVmO-Th`8MtuS)EFDSXOnWKg{}5ZNd37hQn3ZwciLiRM7pkp zZ5`a{{A-@!U-K60awYnACidK)hqBs*E%648HZ_D z=D~rwg^p`Ud_I6t)d?1eH{7{E2pWPLxZ*r>WZ!c+nQ`B>Oqb)X z@GZes^?S`;BY`%Qh$qH|(^_&HLjwyfX-*c^M~!K`aulmv+3A!V#VfBNHOiB3P$0#w z`LED__-h&Sj&G#Q3ECjlyG?dReO_&J|DlEbeM4Y3_WJ}W%j1@W{*ftnW5&%gS-Z$C zwYjU^H6S$I7?(~VNphJ(S%6j=|r znF4Nbp{cD}Zd@OKUSV7jf37etj6bh5(h5yoTDcg!fVicS3XAVzS+ik4JF=xw55~q% zp#@JyP9-Zz{{i){F6XN%(PRf%Pho&p%y9JsUI9z5X6XHxs?c{C<Fb*f&8)$94&-Q!>8qF z(;EeWr4tRpj@Ur<6Lfr(*JsedkogUZ+F3QTfmnqwJIReTxOQ$666SZfcJdDxKfCRg z`gzmKwTZ|NVtlgo)D#tUroO>kX0^~ZvXt@7a`D&#paJ1fo;U=_RHJ@59?N3c8mO!`wn^g|T@Mok4G&XSQydK{wrtkqNLF5tD1kV;~0DmBeK= zE@#O$-wX_yM$8;$5?tr%gtqlAK*dN@b?v^zZeiwzIi;;Of7aD#aU5uV3iwxMkntq- zachz`i1+b4_7P>ul5z~MNNdfbHbUK#z3CcUwjsh4Ca6(Go4HnFRxh@aHZbK^d0>vi zA}$_7M~@>iB*wnyN3={ENX7@}lh6Ik;}v*;)FH>>L3r4V=ZDR$FzU4*7@rU#00j3f3KNfNxZFbyay_y|{zxMr|hJtHkV9K~Bu9~liCYWH6(1%5=y zl-;E^xXS#slDM?mUl^?Pjg%^0A$dagJQ&8hCzA%?f&t_oaFYj+v1bHv$wSn==gPhG zxkJzg-K7>)8aLZJVlKTP4gGmB05>L{aR(c{`V?WTZnr6-YXizzZj~zjh>YeUW+(FT zzYqoJD!lHYq!Cn&glT);wCjM8kju$&#)Wy*spR_uKU-n+FX$j<-XhUo;)Q=M;M6Q8 zktO3#7uq=j5$T(U2ZN$75IoG*6#Bq8>J>)bb?jEVmr`WF#=}h~7e51=Z-_B#8Q16G zmtlmil^)HzLM(e35nZvGp)vANXEeeJYm;E(XTS?VKB>iYg?V@)7N8D4akg?HB{dF8G*#G@ILH5VI4A z*KTGFF1y%l(2K=wDC(TONKtn%7p~&F9TenS>w%Gs3S~VL#nNa5?a_SQnmzi1%@}la z8ay=A_f7K8(lO+^j@C+=&+5Q7!!RXFpkDAFVTRA>2fQ+?i~V!Kd9=99{uITpNZJtv za>_$Nq`Z}etvelE+c1j;-eVX~YL}s;S0f;7SmRH2PkO`uWE;H`-8#^Kxea|TSBb%^ zNjkmRs})(q&<#-V3SW7(E4+ONu&8K5>5^^}l$vU0r`$l_ey3xMo_VV&_>-MmWO4K^ znMBH0VUOw-RFf#$NQW4F1_jv}BePl7mMN^1!xFa4W2HVUSIa}J)Ssnk@vu^!Od`Mm ztTf~cGGBIX@!coq?!qja`m<5=>fUuyrkE8`t4AL;$afnPBKztb+>}`o>r*iTx9t4p zxsxad^a!zYCca3HVNGx(!<*9QG1B_%_uvVAeNl3wFgZ{w3AeMU19yqZsuTvNrS=nSpB%nNav z&O1eq2wkS=6Crtu{v;$%kt!rl(X&GG6x}T(Pmv^anxb94`(cV`e$;LF<+;1~EWs#X zmh6K}fwm_;OL2qqGfbfE5o*qvB(HNLnr(2QKstmqGvyG_XtEBPC}>Dw^Z zC`=F3E9zHL(^q7}zzF<3#tr3!cUS0;7(~TNoMTg`I%s=#(Pk@X?O2C-4J$B)O5_SM z(Ck%Ii=k*UD}DI9 zj3E|sE%B~0D$k}+oVZI|+T3i{(BY*}CkCxHiis$OQ23yYbp%>~uKAj%-V0^TiYmh^ zh2Nz&I_!7kSMoSrmGRZ= za#qefuj-*vPOo-3_i)w2fGxNt8pY5)!bf&~B^4^gHzX06~p#0>o8$ zlCy!?@OQ;cwVd9Nq?Xx2G}P}c%*-Z)GJGI55QfQn;r2yqR>s6$nZm3iULV9X88Zi) z+6do`uw-p7x1l@cw~iqevb3X8LIkM8k*J?%ddkC;=(YE%c|5}P5bg|1v9{)xcGPn3K+tXjqT*W3k=kdmUd>0 zTjM5E7I6s@Vq}*K5IaFITqoJ*0wjeXUm#=&gQODV1BBeeAZY|a+K}va0pcJCS|A3R z2d%plKNKx^#c{yBRz#lV=Nm5MmpkI}@b65$70yWMH$YeZi`(nKxJ#SSi7=+R-!a_d zqBv=ix(VeBYX@?9LJg%m3O+`jgr{A|HARhm+<0(Olv!1%15FiFxls~-E;XKwKbIK~ z(=*0f=MG%P+0VG0Em5`5FEj&Q+ilD z`fvgM$l@JU7`;f=?UI?X0`(H&+CgZ0t^Q)PJlPSOj-$vy7q>vQ)$5G<4^or?%+?|X z_nYGCSXZ-B*gkzsW*kk--X&vS?cvgz&=cGOSgs+XWyUFx zt*flQN%CrDsx0enxbw-XRo12gGfM`&>sa*H)dYr#f8=K-{M|;pt^6q=Qa2dyRkME4b5q^xN^h2z%T0R&CGQ9nj$!dnf@v-+!;!BxE9yaYL;YO||6pz> ze9Y*9zgWBzg9>3@9H7+HW9Zb@v{z1dR{+gsoVo#5T`T@VogAuSArt}|1n1OuL`--M zJfU}mImE+kh*JJOIQkS_iYZOIfc7Iw6Z$&4iLJA_C>?z@E)PMFr6s=KCdBnv;@V#j z)$PD}MHaNh&mH5^yo7Q6Stegu8QItx*LyQHRo}S2jOnU+B+5iY2#GS05~t(8%amU~ zB+5jR?CMf}H?D7FzqhQp^Y4Y4JO7@)x%2Nm+}!#1ilCf-fLvT&K;_(Rq?s(KfE_`r zSPY>p1D?=`1g4Xp(9YYiY>~8qe3#+I(N;fk&dv!EHs8$>S(_%Bcp5QvMIFbXw=VL zZr&+u#Ba65t0H-{+q!StNW`1_bt2yW2!Gb~u%Y~PXX!Hr9CuSK`Jee=2T3BBr<=w-Yx ziuVy5V`|Ymz|JoBplEo>5*XRSDBep89$6S=x6x-fzYFArQFax5hSR^mxiHG^q0ex2 zEVyoAlwCrf;U%=-(m-?&-92;*(FOEb++L_;r65%tht>-+>aUqz4)5TC(Td)IyK|ZQ zQ>eF!Yuumu2L&sAw@4KYOm3R;T2?&DJ~8YY^Eku)XlMHg;{fbwI2n*;JK#HrTP*jc z4m#C@n@uS=FxW34G3Q?C(I4Jc<9?q;W(eq594J{13+@9LRB6elcnyj{%#djuxsmnq zDeH}Huz$v%%Z#^gGWV^oCrXu3|2X1Cx(P#*tt6n*MiJHGw6Y%?HGq$giO@42 z&Z3_*|NBy`h!*q_#Ebi#P^EvQR`i)WzY@-0;+3L!y4m;{y`EZa3pB%Ri0&5ndj|nU z0r&et8tVT{3{FILqDv?A0V7$w_Qme}qZ~2ial{~^ zU%Z}#ijTU~LrGI?+?OTPF9|y9iJeHAgfc$x)>9W!;q9;jRR5v zNCX(4tY20OB{mP@+FP-&uO9{)0r{bq9wK$j2CK(;)_)l8=U zL&K-<-3D}8C97vCBFp%v8*lDp-@UlQq#yjH{M%0((**Y6&oZ4qR2G$@6kg6?A9i;~ z{L>vQdUiQFe;9MFWA8b;yjLo)_xdt>g);oGZ`-+orGtB#An>oL4A|V>8FQuQ!`$n4 zFwB{r{Q70T0Z9F|+A<&0>y@eB7|R|a8q1f9sFR1jNuE+c^yw4VUWhd7H!3wZo?s3A zVI*MqMAlkJbax^4MO0$cSWO`RE9gm27f&L7_d(;)t$^x;PkH{W`L zxXt&=mUEZ;NnGv|8)9+9m@f(RlJ8z~xA=0;yHj%bt9TF9FXnUr`2$9(OglEx`o@eC zZ2m*zAJ1jaZ{+B{IbU}83b-eMZ1X0AZ%S^5PSZc4B`%ZCmF;=DM>PjR(p~R;yZhi z45HGWB!hU{o+N`{#*#@gi2G0majpoWx5YP-IYdXsrYHUezhQHgAlCv#Y!r;$@Qv;1(DSa#XxnhFt)i&e1 zNc;b;pDR!ergxbbyxGbQfMZv)JLijQ;&(iXEXL0zh9|dqu*L8lB9X<7@Jzd`$+)12 z?XoI55std^Pw;E*Lm__6q4mz|{W@agqx20j9O|L^Ov-x~x;n+eoQBIR=gEhL_Be8J z_f1WMQ9iK-{wUfA)n8%Mzrf@Zjc8MmB%i=lGyg*rO#HJw$$y5KbiNiZ<3jr?As@Ip z8kY~mjAsmFmz&U{VBiP}E}**KH~xNgk}zO}7|cdGyXbuu(9YP#ZJ)>)4Ny4mYLjGI zZai~!f|PnF{#p4pCkdxSuO=6-kuZU(tszJp1(XPK1)d%z$Xb@6tIwf zc)zS}H=dcrShOjEIV01{fg@jqvlTpoGJ*Ly#@r+Hk%lcd#S$44_anx!Pum3dfXw70 z*w6b3yTT6BUedVtN{)^V#IJ@v17s3GP66G@IX|_zy)!@cNqc90>Tm6x`Kj01JM&X& zduM*_`7Zsvd=a)e!v7YXPkP_l{(H7H*aCho=zqF%Md_;3%qy-qPQ7dG&c=V${z`tx85clQ3^ zTM}AK|9LP;#J2Y)p#}b`q;5jUX8>`f8$?mH$jcc?DL3twP9~-Eav>KWny!H9PjI$wq$z&CGa! zU6LOLqB$Pz>1=Tk#>?^ODYM?mvZxJ;y4_QJ7$ad9+e;OpQi>fi<5ud+trW*Is1R|o z4C9fZ+t6(&`)1jtZl)YMM-^>Z8PY7pI&wC?KAzkkaIxsax}i1Tizi^{tjcVtuSb;^ z+CUAQi}LCa@`iN2VB&^&d}@8c0H~t~oFp^}nJJP;3ZWHbD(ZUd4{9)xQjBEZAsjZf zS3~%lrkoq&D(*cuj-74VCO@uSEncZJI&ugKUbIbKyUH{I$=rf_n6h} z_dQ#@-yYQMWEb492vpz2K`}YZwXsATm_R#R4AUtNYr9QcycA&?VZfDp2#LAH3oSSU zgfVY%JE;s?jidFAF_$(9MuXg}wwg^7HFD+tC>J+kc=8M2#;<=b%i9Wny}%Kj zkO}>P-4~L!p4Lsw*9&@U<~J?oYcTDvNJL+bldCxCcRbpI*D3sQ!&gKujwaprfV}yy0xGiS4{{HE7hj%#BmXX7 zr3T1AcBU9l?S#Zf+T zaq+#lf}CH8OYX!QyBI#{E6l2p`we`EqOl(i{f?wS^Bl0ppKTV`T%k`Cf^}sViJ`r8 zJj`=MlYtvJup=4x4-RZk2EN9DZOOnO2eu{ypC+Kk(IT$7m;qry-vhu>d!9fXqxs#- zl9mchMueoe+l}8wU^2etc63UhdH~_jtA0$A8JWrSMOqJ70v=hN-0V^hQ@|^AS;F+f)$K&#XUHOfXRS7yqo9vR+ z9@w`AT`H>QS4qKSzi|scdvY73z|jDOKzhGs+3Y6H*VP||T1@t{zqhdS8>zXhMr4qwn7DS}sE1a9@H+GC{V>eD__ zb@gqJ{Md#%v%}(APmkzyXc&#$(2h&(=tyy`t50+~wu(k7+bDR-Iq?P_UCH7N?EL%x zZoGj>uv-D0Z(}ug&SK!ezDKYn0 z&D9h7(|54Zb+rRNQ8IWmXVTaV~}%+Gc1`q+N-|DE;m#|7tI9}6z} z|ML3i``G`+`ndjYiS_Z%ch0ju*3I}oSRd;Tox47y`R839SN`gMTptup4$Wf%(FL-K zLUl^kwGw`=3=Zk+`41K}R*&$by>S@^F)5zo2*fP&OJpruF0SEmK&;Ic|C6$?Q*Ev9 zj`@sv{15-b82h6E#y4XHF-0?H6H|2fCAieW$UB7(Hd21}yJzG0G4QErT#q6G&`6Nf{Vo@dxyoQAL*>itcN0)c53Z`+dWhZRa}s(v z;Joo=48o%KdhV?(i*fi*W>^qr&9Q66u#-0O*p99=UA-AS1 zbm*f5)%^x)L5gznW9s*@J4x$kw{iC%G58{zUs#f-l;ZmxE^V5__%@r_(tq3|ZoPH2 z%Xvg8-bk)GqIUyaxs;XHU2Ysa$V8xfh*NeC$Ui_%tafg-B9k78@t=tz$4Bko2yelQGFng$!O|DE2*qOd_5DVYxfZq)tN+s=*M;|_M zKo83~+NEIw(2}R06u8v4*;dVQF(WW_2WA!?Z`47U*dGuit6G?{$Ta1^t zq%fx>y%&~I#F^f}W@`&^IH&Rt7;8m_q$Sx2Uzt%PsPvYiOv;N=9~Q(7n6!(zCrMwD z?|b@DXougXuo$UY(KclH0}XNc9QP&0?Y>L6KnqrYHuVkT=I_{gUDA%8ho{icaPk0T;kMc+ zloE!QoID_6v?l2XHGerCDnEbA7C|r5Lt7JJTX?n0UFsq6jt#EdBOYx8E*uG^cHd(V zxHp*qlZ2J9oITpLBu(653>n=Hmkd>F-g0!TMz0xj{(R;I5xhh?`%HB`<2$p26qT2h(xw}1TNjb$j(&t9c ziu0$I1KCqPU3}tzyz3N{i^?%D6O`*bdhZcl-QFX=(jy$s-&{_Kg)A&;%F~(F>EeMZ z_4!%A@ ze`sR?Jl_AHEf!gpu&>hmtAK-?LFOT_x^T+2cTgA%G1#M>5RUG8>(KODAcyo?1*x}U z1^u4!_=S^V(SvOShcLLkKqbL;j1h##zX9c!>FJuL_ecJ>ouu>peoy0W2)+E z&$Dq)k?Y8Rb+)CndU!qqYi&7Hxzvb{guZ0NtGtHw&%8m=zn%6V6@3 zdoXvGlM>sT!=qrj$$1>@@fr;aH}XK~3owv4?I@qDp%a7TQ)gMY(-7L|v*;y}wkVJP z37x|<^=5+Oq9>6CI)XG%X@UkquJPVXJDM~{&NiazVjfpeF!9&ZZD_2?^+T{VT9vCZ z!^%}XH)4Dr75-qfbms>D;mBPb@dy)ZTjLQX-fxXZnD|p`Ji^59TH`S)wAOfxic)+e zv){-7`WosXmmaBy%juDNxR4&H2THq9!D3l=(5F831b%9NXt+*Kd@3Aw#xogqOC*h> ztRi8wnBGdqA^YY?EA`UzcSKZP_5=}?jkAELL@})K#Mfl7c2g($%U><=WQlOfp}%A3 zD9S-y{d%sn`kq)R_+L=C>7Pd{c66$}9BOYcYi0EyC5fKiJ;lsN33sG18H%jE;QIX=6OSJUD?K{V4FXL%FT&UT+8(zn|7!dn)Ra`zJv9zp5K2J^A7^~ zdY1XG_6O1o>4D}xYGJnTMi~!F^1a{lI;o-VHAj4IPHf6mSo~}sa)+%Uam}B?taE{&Xj!>xNQe zXrTMI#fNAdS;@L$QG17?%VLgK{ec?L2aJ4!z1~h|=}_|{4rAR*ud%vAP{-wFkqcv8 z;~DUL#SU#jwnux)fj&KN5((wcMnNC0Uzhd*;QSkyMF|-G`Q2PY=Q_k-n!rr`PT5(! zftr*!u|J5yW1&2rJS+To;R5&Pdkz^*A5N2wQSrHtD{*8Or`Z$k_f>>zub z@P(gO_HJA3Kk32TLrWZD=oD$Nn<#$9Zbsk}1^Dv!?8}l$njn`IF%B0}O zmbiG-rOsv1@FT;x+(1CaR4#5X?){1Nx1WAub|3tWzt%BWwF$%NZ&0+0Od07chf+Mw z0i4EngW9=2?Hi^)@L?W~{m!Lqji3QU%8P60d^$@jtXzdiNRB4x6m=CAGVawHv|P1)9@m_r(z?xs@Jgl~>(3Q}Mz% zKFFh96;El4xtK4Z?r>11VK;SGZ>8=U%=|z40KubH8Ixd!>OX_mjy#U9;ci~{Y|?8V zv)4ZPZ}CuZh34Q2mDQr!*-zstRn#wGq0TIzMH+?WiHmf!V8uNYIBD}giCHnLU6;qW z!KD~_n_XZzoz0SAU}I=tA6OCr2ibBtYUIG!>bF9ZYpKb~b5`n6;}-#%tn}!QSY^G} zavepM$1pLv{?O$yqj^azR)2{p=k4{po|On%_bt>(r6Vl9A6XpXWB$_-CJ*x0!rr+Y zgRkU#*%%`r+Gl$4PB*u#7|axCt#uP$x<#PX)E7|U&Oe|quVqGjD!Q*HD*| z=2<}rw~q)kS_{NAjetYsD&@Y_ayU8y`X4E-v8Q|VM`Pp}ZIB@f5C=^=y?wt{`oeNJ z>FxU^M$c55zVY@w>rJosI>Y|Wiax5*3!HIZ_>6{bN>iODf3MQ_jH1sHJn5BE1-ZB9 zZ&1Q*Bb27y-q3#E4NC55k8_Jx-8S=5bcqlBnJ{uy=Oi)ZQUxV$*1Nwq0}e4YX~n!~ z`44$)7D7vDRn@vVNsWs-H%_8&_3jg}eZ|%;vua#<{{DsS9yrTFTVYD>h{a;@{F<`! z^mDh&oCYT%T~Gcjek1YtK`3fOTbgQP%+|(OypGPlNAdSD148n`gUew>%w|dI#r(cM zQTueE^FD_5&zz@y*LcD#1xgvwL*8szEh&>(It_|3lIq~X7GRrQ+Vgf*67R}s9;oPt6uD+j&9a%X$XeHmPUITE|p-Q`{Ttl{0p zCS>zhc}EykhI73vR7Y%l4vREnFAx{yJIJ1bH;9ez!7bmwpO2s>pPOto;B#Cf+UB+ff!&l7X) zc`PN|=9x!5LY2o^0~>+QK~E{q)eCL*X|KO@Jp+4s!aR7;3Uj2D;1`;3s|mLeyv@Sl zHWO|q_z@FsH{mG+-)h2BOn556|7F5cO?Vo?|6sz?Ot^#K0Tb>p;pqUM{WQVSO;|UC z-Ak};CM*MCqY0K_!ZHzdE5R~NSa*b7L9p&7EDK>70Gre^OUj#+LVHHmOIQXTQe-}R zFWWQ`0aaSC!(YR@U7y^b=;LCPq!S~O!979~*pel_^cM5-DZ_{(XT_%ranju7Vvj<> zHwa^nAgl+=@7|-Zhwn;^Jc6*kEXR9~!oI#NEa$Tv?>!3h6`WvHWWYdE%B8%Xsj@y^ zP*8!FK~PW1Z>7A5h;~5%MMw!&wLg+{{jo;_|6>0yRbcSB`s(}x$id## zYPYC+>+3L|bg#PuS*8!d=OGMj(wd|{8LQ754nM&Yry3{1$oSB6WLPWO!!ZA8MW5Fl zqbeKURso?hAR(7Udx&XBKaB4}Zy2}Gcq3y?eqo^WYf_uNuol&1I=jZ)rN)?l;)3oz z3&HG{VJVkJHZ`&La0NeuK|5z)%;IwB@xzW-Y+aTGy^7o5sXIR1i%*&OGy$J7@Cn18 zuIq+R!|^E{pKiq`WctaXog%rDrxJ^L+zX#r{AGM1*{Og}Bsy(s<+kP&?U+SNJ<^)s z4>wOhJRp~E%GoYMJY3gkvn0ImBMX{M{wenXY>zJHo#b+@zYyLoWc<4srvy{O+i9aS zA@W}TiAXxLg?|1eV$R1*Am;pW__Ge3K#kL1G-CBVtonLZojM(=(J$1fgKFc#ZSYS$ zP>^x{NI|MHiA7p?yw(yY;?^@Y@@i5otIP5jNyL+j^$~-M)98}V!Ao9rmE>qs>>eQ5 z$LjV$z<`R2qL^?%{m!E<>yL}7>1(gB2_^f}n@j@0qfwO@yb!t?r)S>c(ceT>6(uE# zkX&b3j!6L>gdguig*WjfW5HcficCf&SWHx>QF}4?b}FYG+G&GLY>@S_=RXEfl#Uk{ zTtsjXQrc+jA(#I6+;kFRbq`7%&8qN^?>agVu?kN_nsdxSja4Fn>0m4sbAj-Mt0Op* zMj{J%j9Z1gr75J8b8Z!1nnE?>tx)5$S_|?%?#`-qi!TjlFWIQd235V{Z(YgWvQw44 z(_3lmtrV(C>Gald+Q#EksVcS8Tf3-APNS-{PH(MYZ#k&S(dn%}u(#5wD!tQNrR=S4 zRMjo%t=Ti+jSK>3B)u{FPCQKsoSEEuD75ZQ;O@z-`$6k00%s++#+ELtm>2BDml6vx z#XK=#L9sQY(;6B&iLD`>ts$xtgVV_@ebekXK zrhS1QM2|C~h@-zmb=YPW2n+@UWR1O_Oul+67`Y_@2*^$hyCnuc<}fS5tN;U&^LHF( zLzoR0G6KUFCAg& z00WY97>9L3ST}$H$$2@4WgraroE9KCEgY7KuuOmf$$8kuVBHbc9biCmzQSQy2+OJv ziF-e(=p`{Zce||L)gb5YFn6n>?oqV@iNOH^_n*LX70}+(j&zB~`I8v9nA4X6fHQmICRChXi!WJC zu$6;t1g_`QpJ3x)JAuD4!FCQ#A@F(=oWjAW1pbQ&PUYY<0xvPaX&emX$=9>z5y-(o z=?I!ZpmYxEhM>C$)Qy8O5HyNF861>}pgaO)a!_{!T}Ghp9F&D1+^sGFCdS^=0~97v znwmcwSWTHUu$zJ6M@t1dxsB`YVH=2_;}m!o_u4U{L6+ewM&lVfT=w*go=S62Is`v* zf!w!Q)?bLBi&Nh{a^D&`92=oHzmuDQRzsJU{LL~8a2U@UssyU>__X&{s{&^PaZQ~^ zABpdKc(ge<*PH5;z72BUZ@5R4R}CwD_sV^DV!RghEJ~z2eK(_cDm^Si!}J=j^Q8Z; za(a`$2w5C#le<+O`X=6X;-c5dk?;UokI-$X=mRn_MdF4}ka7wQp`+xOtmYWI8tmMi za$rI}@#_B`ul`Wd)gL=yBR$LS8@Q=mt)**zSa|vLUHI}_UFn7@c)30d zl*8X!lIT0Q`8Q7?y36_xq`N|W=;~%?-HvyW^h#;|076D}kkE#oDVN9M6H!PKJ`rU$7@vqTyAhv=GP??&h%&nvpNKLO@rfw2 z6ntV?FM#%9dhOGRd@R$u>{}9U8~&4#X-uD8SA^)P6S@e5-#%`$MBd`!;Qy8%i^cn{ zag$Rziaa!6Btx5lo;f20T9Epoa54#!F3C?(u5p+0zeuH9m`q0u4kEXxmZ&dPeEBwk zX(x^R587eHWVrMxF)45)P1ck?bU(CMP(k!26B-?i?Q&`K>{9oUf@1fN@aCjOK@R&dsei!og8NZoT46-X~#Tq~8%6Jwu=!ETJdcyAJ-=~t+nTVUxV zdw#y&h9L{k@+j*h&}A&-lwqakl2Vetg?MxkBF`^LMS&5Id)HYGUe1OUFa#7wv(YX2 zH42g|DDJr4{VEs-O%%oMEe}+tg$-LoH1#f$x-qgnDSn$&VP)^tPlWe=Lu;**Jy=CL ztKW+CFJOJ&RU-SuQ}$G#8dA~NOmf!FJn9n8xR7d~5U#~l@H3fsB@6mU z0m%5pSqNU91hpcR^bE;R8$u6vh1wCitt&JIp=-KAQxUqND>MzEzv&8fI5*6n(+QFe zkm;Qu-2gJF6C?v5BRfGd0g~4V(j6d|%gX~*S#h9q_xwz`xDGF3SbFvz;V1yYp|QR@9^aC4yBIu6;-G8{eG;tm8ThxfRS-i5=#V=)CrjPy z%588t4~Zc%-({k9#kt9Mm!y^4fCo|<87tDRGI~Z593|gppVMpW=OlCh#!V5VL%2TR zT~=dwR7#`GT%a>}{D3X!fT_RxOyXr%?pe62W8<^=P)Mu**X_v?($i=>M9RW+Yr+^F5xnaSnNCBGWu^2XU zJ4`9is<;&BFKwn2=#4g03iL{wDUm9ROQhzBISuoq(;Ui zQbXbrshiqpS{@p4ULn(_BMB6ZYNnRI_L1D@))6H4$(m%!efEDqVxdv2x+kk1IKq?~ zeG1h^`zPn~kN2w8+=S>+SzF-n!lDsfr)3O&sTM7FX%E`v;?XQamdf`Qo_#(JV|2-EAb?a3s{LKXFy45(J%)UO z=3NmuoTe5nr6|XD_Muk4Cq37}SRGbt{-x@MP@}&$36(w}MWK5osxPZ8Oh~7?SE8tZ z)D|yZh$^H&2W+qh*;7;uK7!(~;!W<*335Iu%f->F2c?R+=zE@vj-6522{nrjFstL2X%*~_5T}wrwMpjEowdd)i+(+q=y=BIUBm9`GL2n(vJ1JO8 zDyTC1sGX%kz&ejg&{cFI*20PSY#N@37)1_MmFSdZo`A>a;D(Md_2eqFK%%TFCV)lZ zIGF&j6RtPO4*jJB@536FRNz^*9L7=OUd~mQ6~>$+M2S|SFBVavZ>4a;uAL}Rt!TMB z|34#!V+1Gk*C^HjN)5BwU=TAiSm+NZEYBaw^e4mZq}*O1qow{CPwpW(cZ;mMtkKdu zfe3aP?;YquQMa;SsGi*Ky}COa*jAM8$wCuqGKE%;6G(?wD?PbCD!Olws5pNq5MQm9 z91V=HQ;4#|grsuQ{=i|;TSid%1r!Ma*vewV zf6BpD0NVj-7aBB+!f(p?SRG+%WU7M+kd_YA=xP!TxOOTmwnr4vR)F6l7yQ=2ze4h ztpr8>kD%5(3FfcRM$jWB)Rrf~^cC6(`lSiA=SeVog((DGYeG}3+Y!*Y?Fxwh=6}2SvJ`cL_O1>Kjmn0J*3b7M2Zur(l zz6%?_ClkR(JEsCe|M}^RK)KY-bNbbP&g_m$R?s0l_IrLpZ(=7j^BeP{!&&Jg=+It6 z&5q8Xz?Nl^qb=O-^WhbdRV@iUow^rp7C5}%T9E^W6)zl;6q#{Rm=WxjE_p`Rb;&dO z_e7cKqSq5;qEdd7C=>niLUVjRFj*ZYZoIFxb9^ZG%E)y|an|+j3kRK-#5ug*X5ls{ z5lrkSRrna@ei&s+0w}f@;3mH=ArBDOczRj#JlO)vzjelcBC?zbivBhVf9FMOXE_>9 z+H>$zkwWO~b0X^YVK_u0>UQ805p|pJiHN#S@rj7K5AlhJItAvm0f+4m(;Lf1vPenz#xx8u^ZwhN{WprfFlaUX zJ038}_}c+{+y~+WP2n5|N;^zPh0;S7_8(enA8VF#TNZ|~lW@d#R(?;`hGHt+$;M$| zD`d^)(Tk^e^vC6zQRA6S<1wv3R!2=T&MZlxopj0Vc>CLAtsom6Bhrx|+lhI~&{R$4 zHrq+|ZNo2qg@R4iJ<=94zAGn*XCZ=A8*HBXMW!9)@DB<2Jz7Xs!<~UJ8F^&9tW7|+W zlgjI~;k-0J8nW~=?LuvWj`izEMZCA)i! z4X@xxK1CAPSj(P~3^v|l&qxXzZ^5(nD(UyNPj?bc?4K0*#6m%+7=Bj@{->21Jhm)e z&DReTX!$Obv_f>nngx{$cqNYD&>p+l5$hFmib~F}XFcXp4Ug8Z21j4-)i5nmqi+@K z#s8+F-EO=KuToj?{T6BOWl*kP4C}|V+&yc?xrD}Z5{(yP<42(!nakFjaW5MLx>XwE z@L8Wb#klKb)BlS)iwxRh<~b~q12y9@SWw`Z6pI-wsLOtx2rI{-O>qR;Eb|9Ztc4Ba z$gj<_HpBY!-zp0`>o4Tz2zv8ta|nJZCK)@KUwQA;4ZapxbI8TJ;hkO0xR0PJd?eMF zy0kqeF!NgT_d<>ElfE?kO<7GvKU;jxmQ^b-8}ytbtAc6wlia>fQv3eU9_fqZ`n;t2 zAxZrebn355JUox~98S8p6obzC9hd^Wg7#eOY+MfU#((FH_a`?lhBhZ?dVJS#v+%j&S3v1bhsGC*|jBGHSoF)AvR)VoCn2O~xvM7iUNMB;ZNMm`8iM5850s%-aE> zFCf-|-CEG!;(LlP4(M(~boYbW-je1f43=^X>V!czAIB`{>?lLwH6HEp{uE=Lz2eIU z4Bnn-THL?0)jaOQWXpBhhb0!{)NcH6H=?Tw?~#L0uCGkg?}*nk|0rzp>25m>Ch}D5 zT>ms$Z%^kXzM9rsZPI$fj8ts>ZKiTT^7`AJSbv=m(s})vUpY4qS5YqDVhG|Q-uiF6 z^}bH6i=k0tVg(j-T7eSaZ(L#p&MG+n3UvJs>+h|)#QOUX*WY3Q)c=#MKfvdLx&%H4 z;6m$LPVh&P;IshyPEV}AQ-)ke>n|^onY{i^8F>UR9um2-%lb=-*UhtrQ8H42GBB!` zCAXjt5%icuM^<;lb}HIk4oRiRKEW`4PGCNs3~T7iQh9sdNLcl;ooZNeZp6?wGsi00 ze21dX&Nh~nvq%f*E<@3AW&PnU$wT>lukJCLm$b(izZdUn)@mReR@@v9yt62YaQNGH zo2CBm@wh1y;8@)wV;;~|(2q->mPVeN;p4W*@K`b*GJ(3oIECxVle)VHx{FLP^^;!| zAnY$Y?To`Yi7ZLnQ|!uP zw{-`spL>~lc+ySy@i*beFR>&uwZ=VP+LHXI<8GW7f8BKB#9{8q6jUVhhAY1L$WS-rL2;65|EeVy0k>rVDRBhMH*nr_y-w%UJB5yD{Z5?spI6OQ(#& zt1v$2J0G_v^0#`mryM*%y;AITpf9Fz^mw zP}9qVu6*JNV;nYZ?UIl6m>~lT7inhpo6sz>H%Wip<@<~$HUmqX6rZ$f{qe0(pVnzz zMy@5zJ&wnnv@WZ*+St~KRJNm8%IqnM{&_hK30-213qE50pO2vloVK`}w&q>glzkb+ zpCv04UCA*HJkXJpG`(mAM>62Q0=KB|2Fx_i+%qGw4z%}qtj!E+6Pg47;Gg$=2?W_d zW}TXG1CAC$9wz&0Bldd0fFl?VyEDC|rC|~qequOcJ59c0fy0Pt=EhIl=UN&bS=crc zxSFYfL&64@w~@V9MDE|Gfx@o~B8C5NmK(?4Zp^&VZt0X8Cr%H_&}?et#hZXeEKih2 z-ptC$=VJ$)k3Uni$d;L6umHCJ+T>9lx|RYlc)x&2!`{Z&tamMJSwTTl?;+Tq==mmk z{xjwmLwSJD2?*U9cAi+1|HESWE+3KoNx zSXodkN&$(k1fehKTytBzY9`BiSYzckUY9s_QpMNoK)LnmEX%r*qz}4av+Z2Z|8gbJ z($6KSfdP!X40I*dm%MycDG<#EHqr|nIA<^lhRL|{E`bqrW zK-3}Yvr@&tN7PX@20s>9`ifu}+pHOz3SeuQLkxZxMe~r;;#yx;bh_$uM9DzOJa>eC zxDXEELU=g|Z~JS`gSR+-6b&xa`HugKy*Ghxs>&Y6lQt~@3NMwdD2`D(PCIUm0=5d; zKngFIKv^oFBFHWRWl32SERa6nO^D(s4x^&XxH026!?<8U(6pp!%Vw#Htfef%3q?{y z+EPmMKj+-{UX~Wb-+bry|9yV{KYW^d-@Esnd+)jDp5-p*;w*tx>}U}$SzQi2L>(O) ztf|d-zjJC`mbTekbS@T3zQDid@6{YRANOcHFY3`Fr)yEi23}mGZ%}3<*vtc;y2qkD zPm?$A`2%b!?Xlajs;8p2q}^LeTiFNVimFQw7&D6<`@3xEO^C84{}*`m$)oX9n#<9# z2{6#)Uw7pi{u7s;)srpX4b7P3x_-QtX=L5`rCv;FMc)Ur2=o{f4Qq->bLQ3*r|;#2 z{&X2%zFxyF4YV>-W*v(rL3B`zJZou=QN;XLWwMxmBetcOM-Tu1p_sS2<{SA^0na+K zu}u4g;7voby46f0Z#A>IKf-+Q#t_BDs!(H|e3o%uqBt6de_c zjtfPvz47g8rv+PGjB^bJ!&X26X?2ZEL7m>!k1qB&%w*ji_qFOmVxJc$ZaN$_VyYKY zW5dX<(2FN7Dw>tM0|L^frSg2sQg)fc`%6(d*@opxuVdUzrzWKrDZY5MB(TUvX%R-m zqmr(uDd-}Y{?%6^_E&ZoO|@8uwOAfji~GDp{Nvd@L5xr3sQ$2N2YNu-ww-x)xR+Gk zULCv9R?X9|JOJM>iem)f?t;%){jdMXa%)-c>aLV~Gv?N@+$oTI^DFGKg1^YQzoA)i zYAludlTAD&6>MW-eiK7|5_}6{aa%RJ;SuwK&Se!3NMD>}P~0q|9%{{t`*?8;;zHW} z38wNcdg26}I&u;a7(iq{x$={6WaU9O?~~r9+}uib;-G`^7DU`Ph`3k65O=9Ky!-rp zBKZ^OKfl4xP-sTz<;BH>@1Jn0{I#cLXEob7;RB|Jd_rISGitV9TE)KF%63=0NSVqp zQ$bj!DUmh$J7aoJQ|3LGd45>t2O~2tzCw6QW4^G~cueJ1vSQa1Wt8Q0q^C^wgm=hv zNFbmG5$N+_Kz}HT0GdJC)))GVkcG`!8X#?~Vx3*|;zf+2T`{d458yV43f^V+fLO`cIlnKAB#`X$ z4brrw%vAoPhFMV2_Z==l)@Jv}1~Ps;58Y8XjjbBTtGGz8(XHq;PT_oapl^#Drwc1iq(trqIlCBv-fE^ zgj!2O{S`e$$%W z3wcLj-uCkP4tBuECofY6_-|14a?>HdpZuvqZP|6X*qRr%fc(5SVn237dh=FD=g9K3 zdw|-(g#C({u*aog?~pU6_wBBinC&{!0M>~IFQsIAp}=_k%}^%QUSP)S2J-m~aD9Ix zUi5pb_#3wBX-DBT>j_y(WO?#>n$&x~{6m@3qoFsqX1QYfw_^KAQbL;@5%(QM+lQg? zp9eOUykF^G!~+&14|tcFlBJ<*@b=inT?zvj0D?AHVHQj68TIpR_Kee9!A90En|olr zP(9r;D_s74AsdM4;r5iYbCF^*Gg$LJ1rYNHL{>}`GsypfOpEmBq0Q_OGkw%ITJ8?I z%14SlvKZJ_obpPCyY)?RuHA^271Rc^6m3j@yboah&NiqYChutS1_LRudglQ zq%vvKR&*j+<{uoohr`|B*o|1iQ!Vx*oRm&)j#V}2npf5P*{erg4%WP?=#yhM`n!e3 zFW?*o&o+7RfY3%b+BJ1770IW_j2s+HYsseis@=bXV+>_h}S8&RFAX zGSc&AAn#+lk!@nS|9phLhWmYwvhk#)_DyZ2eQlt9g0cy;k131w?Zkv zk#L6*bN+ILIMPDoNIwx(XszP6cG0J_ZIlh$=Da*7+JKXtqUQ~?I-8OmX7qd2N!>Ue zKHYQs=fs|(ruA1#cP;>pif<#9VQb%7je`1Xyn$L8wmEET>RS2S5-kvAZ9oK1)7r)J zyhsaA%Nr55qo2x~h5iqM(qr(gE9X;jzfg%D9~T#jV>U?WPL;l$)QVp}^m}##Ry-}5 zzfIe8Eqw=Svz~Y708`C_#w{+ii^srd+E>GzInHDwsEmIo$KyzbGVZ1_hDYtHm*Sxe zI>)pcZ+U>R$h=(_umko%DSjz#(a-zRe@ocB^sz*Q)i2tj+431#G=FSvasTRYUffA; zIn6rg{6apX*pY_*K?%;J=nk9nSt}g|^a=WE&@Pr_iF*{{cs5vpD-Mgxq_;i~Ir1si zu>OMHD?^99fGNYYSnN6Kw4k)N0aY7QIW)`+JX^FE()TWVAr&f?g`qoaekxxt)#s_> zW4XG1)<&sLPg0#l5C2eBnPUp_3j;QAUNlR2`BP|MIl6hJ^-6&Ib_lK~L>CL`^Xbue z9`-5W`H+0S49(g5v&8c_p^oQWgVOraU{EZg)!kO_i?NDL!)Y@;PD7!B0CZo*6)Yq| zr#YyZT3MsEg7s|PBrDZp;{GeMPsk`c{F_a8{J=u1SRx(#w<2^49Yb-U)P}y~$j3~* zboU#S=y=TPvLvhOC%Jfb8Km5JCPMKpUUT7Awq36g_ydX1nl8-FVEpXkX#D>4F&;y# zWBVxOt5V<6hTjN2>t9Vla%@x39&cA_#-rw3!5_3J{hmx7g;}mVgVQ3fCgXMx3y3-K zs81K8k?xol2ai&V>*6zMK8Nb_b~GOt|1s$~QzPfA*@kav%^OT)_R_K?u@OK+&RVUr zTDvmqb9VmI>RgnJ{B;msT;bfBO!H)1!6ymKp};7%YPUfe1Xa#*6(r-TcNX2Q+z!ce z6}5q^-6a`N&7s=7+x*YQD)+_l;$GU|=$@s8WUk;boS_tJEg2y7d~=QgB2m0saGfv} z3Xpn!5R_fGas_AGqT|=B_2#7wLA+sNc^QzD`AbCSYAl%P2_upXbi-sbGTb=wR+~2k zm$$}+-zSGi=d&*d{ZBN@{>~qf(q}dWX?)!~vx(T*=bz{h{r*LveV^vSB1RsEfE&>; z^$#@vlWSG3x!E@Quu6Y$LNxd$h$((zc9) z4jmPTOA?4rw!eT>-&d125VzTHm;0UBXXm7X@r=OH>|q#p*s-s-ekAMCeXM-b;j(~hGJ^ukIm6Mk&{#?Te7wMXd0{KIGA+Yoevz`mOj_oQY#;ICsgoa*|n(bo;Zcd+FIRnh{_Ub8I{c;Dx)W$vN=R$ z^aNBkhp3F6fXe0&mC+MW*&L#>gj$FZl|2qr#=DY&Ih0zVJxBR6I8<8mC=Cxrw%O+7 zqG!E6Rv-y$hR4PY^kB!?xB(tJtw6=2?=JL(TW+f4+jssNT~*-iNf&Gx9>*A4Drjyz z^roNQ1MS${o!iotEai4>*({HOK1(1n(F-i24&vFT)2zr4ezuCy#ak*`@gt|+4h;r{VA8J&<#@ESiIn2E9q3o z-HJ}fNOm!qh6&9R@bM*-Q^wzp`-~VB2wJX!LaMISK6pR01BEF1pt*b zB#%xwm-WG1q!1Z^Sme_sf(Y5l3IS6$3-V)<^fm%?As^aG0Ptd8%p@(R1RyUb(+j~m zXZoOj`6W8UE#_>H_71ol1j*_@+sIlCvP$X;4VUVKCb1&>j>h8x94Q(X3qV!|^2gH5d6 zFisC-LA$fo`b(Q=Q7mgM8axiYTbH?4f^_uWTXZ@G#=FwrOB@dYTo4akc-LGb`808N zrrk5r6|5fa@$dXM>I&fS{fz)z?0*-0;?JXn*A6?$Vt!K-@|aAu&xG1L`}fq+p&zZh zrDiQU^9aR9XnhHgS)s(mxM~Q!!=y zi!QMHjM;uIS7s*d?-~=Vrg1#Rf4nw&KQOyb;(k7wC%0xC z>3AlAB99J#jh@du z{|*eR=|S1=J?(#+7&e!W?OkqQ1o=KeXpUao``0_IPhH`B``iGK$ryFLMxHMvN#2MS z-if~igJ#j2C7k4hpBOb|QYm*v&zV}Wo?g{Ie~0drP2b8q?=6_~y~y=^njy;b&^FI$ z8mPiFO{fwxKSDnoUL$$|niFL7lLh^s&bCubQrtORef>q9uf>Tkn=1n>;H_3XDgiw+ zoF;7Tu7|<+-6|H@T_e-&z_K4#rf{J<&Dm@MmlqG3gOyK>g|EhhP2WDYf~5Ix^p`4?rs=ltN+ z=BsJ^SkW8NF1mY`#t*bBE3}3C6{1sdyDj4~w_?3)!S8!VnS135o>R%SK^|P3fcw5} z87H`9Xoqnt0DVA$zv{K@WYP`TleCmqDt$7&j@ke5>X1NxRZ8EUqpjdT6qvAtI9pbk z`BN<~Ce=7d9{_si#;Y5>sea4HF>?Q0ixl>CG`V}$x3V)X>|7~*b*#mf=9AKj^|8b{ zZ+VB^e;et;O5P`MxgEFsErQhlB6@sWaVu+%+1Ey6##>BmsmIVIzw-%G)TT$J5tu!+64tx)J&J z>RY@2MEdL7v9ub~DLVgO^sU_&AkBgNHG8aK_o5U^d!C_tQ3~Y~D{LP6X?<(=2T1qf z_b=kQp}w_yQ=}W-ZuLB$Dj(7;t{gg!m;f?D9S;^U$1ujnNaWBq)UqXO4Mo;j+ysniA zewyFELKpM^*YK+AKQy6EhLJ}qd^;poac8Q>Z zn{9oF@()Mj{C7{O^X`9E=kv2PNCd3kn2>-D3QUW|>z2uW!TSM%6;F*NCSvB+cx(Pd z=SRyMu(`KJ&nN%&P0}qB%j>&W2X}-;@qFBXaiGXd#8S zyFia;2pTDZr6%TWpmj1Eo~6y(08$I=H^`B4l~ptRaIjT8Kns=UEx6zxz21T=bz;t& zxLAibK}$_fvNJ0#c;|w5U`gW96%}tbT8qk}wJ5W=*DBUpaCb!|-V~Ba_ao-@FnL@# zp90~(LMvkc^mlL3%2>@Zq-R2~MLcGsYsXm+q*4#yB>=eYMtC1<9t8=X65AHrFh7?Y zaDGS`OFvE1V!uBNI^gK1YQAZCz74N{a-Pjbp#+e6*JbirT90a5kNuZ~R>mL0>Hl?k ze?xZoxip;Y+5Zi;%W;zJJ-Tt7yiC&Rzq3gfkHH$0eXTb4jP=S*7|IGgj_{l(fTE?u zzIft{Ti%WBB1)^nhV_nW7@BGE&^A}rr+E4d_vPd1vv0IvyJrE9&exi7{t@SF8{|1y zoIm?kp4~QTHiCq%qFs2pWYz0WX;y_C?hmIwRr)X1(tZhCz!S6Hl&aoPoK0{A|DsBd z@z~s1Z`$#Wx}q(JqOb521MaCg-o(~8apOO5Js{C4JDxQ0hqBQsrnK-F%b1~Mn{hY4 zfzIju_o19yP3uOkY(3ujd4Qy(rcMJb*^T#9x(47C)GroMtK#YXXQYbu`mK?+zU~?J zfJSONG>+!?fbP2N9`=B;wf2UoZ0f+-%nlyj=9VvNBWRR;@mjh*6h4rWd$LOAK&o@` z+89oFv|057_H>wk3beWEU&rzCDcz11wqNRJ?B47s>)3QiN?8k%E+#1ZtuR2^z@Y9I?sz3^f5`a@2!B)*{Io+-*`EOZU+H2`ob!ULKFAuu{NDQ$ zF~2XN&!mODLzCP0;@14a+hXysMXTq3L5AL9aV<7jUH#K>n0AX~P-wl${JOspxFVWn zR&f_uZJ_Rm&G^mpKwibxjBmMR=QMcCEC}Y%A|h*c4sYt8l~rThrNb5cjAPITF~F2NV8`RMm=n2KiDtGK5~)p0bN)Yu=FcQ}F=WZ#MB%$bNH7 zC)@2!R@s%T+F-lwWic%~F$eX#i|W_HcIyT8^FP6~RmRU+7-u(76UYszbP9$@%}bE? z2(gH0+X-ms(<42s*)1LH|%x2+_gE$p_J z5#;BXMYh2J3ff=*X&Q3?OW@AD&1^7nd(DSnpF2-22_q5nHUu?Xz5+_j+q$>eJpfoi z&PLD|mxy_*cyaM+F{{wMTmw>--Q8WNiPvaSYsIXU?6P|{+k5-+SdH}0-j~%?NxV~X z>Ef-ZdvY(9&`zzz8>OZk=*k8ox5A|?jZ3M2k{SMz`2X6v_{Epm^2F{x%Io4jsaT16 znSTX&(c(ijzMlLmu=rZ(l|SNQ^+~H(%oPmO(0MBKud#T8#__W&^_Y2=j^%#cr zm=soz-$OlcsDBMIDhSg1EYo_HX+6udK9mXd6Y1ZBDHB`ZUA}>3 z+7QY_u0mF^OdDCIjV#l~P$u#rGKFO-WSI(CrovDr4`%AeGGQEu3-WnbK98Kw%xzjx z&BE6(bXIGf$D7+$m{%^xN9ndLjMBng5g@U-@>b!x@i%`89}%=3A|L145~H5ss)|uJ z+c&VN5u%^6LtZHmEnrdHGjM~BqPS3m_Y+uM`3Brutc<`5gK4=)Ezrrwsxem1BMlhZ6zr>^o6Y1AJVJ-uZB;N1xf?nVga^#P3Fy_zB7VfYyslyQ zzs*?U$35GoI-7Wg=94z4ERZjPMc(pw8rt9fcHTjz(5NuP;; z@DM>50uV+Tc`%z%CcEj_nYadze@SQZd^dl|d)I z2n;X35zp2@VdBUX|A(ZiG8%ekki0xOP_NP)hyK%Ujo5FlWOu09lho|hcx}>^rVz{_ zY9@o4=?lzsLMBxC8c~*8JB9VepD(dQ*P2B{7R}O%Q4wQrft*44a1;!cjI32^`aSHt zls1Xh3mSjyfa?YCfKsx>t;6}_k#%YfKe8${Oi@4Yjx`(^){K>ECSz#)H{dyXxf!mU zm2zIJ?K5K|YKt!JNUA8{GE{KsYNq!v)1}LyOr6zCX=v^$k;{g#G#$$}DG_0-#) z2y!JtUK9qo2ST1yAM!UM)-E}DIb5R@f2R4Kz8XDp!+(&Bx$;$#hHZlE<5sMDZMd?> zbPLq=nA%sG*%>RvE@~H#qEo-|BZ)zncvq}^ZMF^f4v$X0oYf9M~j>-?DXH!D_=SnkeUCNG_FLd`ROCsZ0 z+n*dFNabeo-F6>5;*qvS$N~gcIxw5zbg%4lPW*vSz2qz!$K3X9)Rx9*Z_k=i+D|8b zxI0kVPv3la%i7X@3G+tWXDIDws6AG>o%~OkJev~;j@-jGH+fg5`5zbyKht^AZ)h8= zPzQkJYf2Hc;D8uGr|kn&09g=JM%EH6gA>r7g305{TD4>|cHNc-u(Z@-7%iNDdjs&! zQTL=CP!7lvv3O-xY0;Ual{sFqxaf>wWlo7$?8+)AI-_5iJIK+u zv?o`qvAP!JwLSnn=Ys!1#crYF)wo60T68L2dTWMOV^95=E5HM< zc5S_XFSE$spluPoN>37teh_DC@FFSc;cUh4x>;_k8QR)}5r0ppirqc2Ll6I{tDH|6 z&IXbm_nnM&4eV?dzqg7_(0%UC6&V%tC%T3tLI0Y?5y_pgL$zh-tS1S#D?dprwVF_R z`H}g9K+d4Y=tt&fTHT9uJ!o#BPO665WoZ{BKS}fFUVXG?PX2obCP8o4{7Ll>#I~=e zmU2Q7_4M>)pq10Cw^J|MwavJ5Y~FMnSBsL9q2EfOI&v>3_fR4uSa+X>?!IDn+rgYV zVlDbHp8Q|_2!kwdAbDZO)4~X+hgp0Q8pW!e3f1=8ok2a`)jHRNo~ytBBo4c)d4YIxLTdjLeq@dm4CYtz7|R`97l zn-<)lmCt8 zsPocWl=IR4L}>7M^$O+Ch7uadYm$c2$ad zoyGc$Vi=mY@8GJQNc5NxcVotVv&!HBtAAm=3_a)<@$FfR`wXCK0&FY z$yv-{+Clg%r$`Q$0 zf{d{kSAQML)P^#RNY<&j;?!L6uKs$KtG%2{ujXp2=4$8apTKf)a;^k5R|hp$M^}FX z%hgHFWl(bgU#GDtXT3pHR>o|19@pg!$HD#)5Xy!apB6YCP92NI!1F{)Vzd9%i1lZo zl6TwUiSIREY03A_DEv_@X5C$CNo4HG&N$t_V~NupIZB)^`<%k*Mm&mphyXHnEwNeU z9D}NVjB8c@m=)4L_P-TZ1o}st?-un2{iEBFS!tkuWQ^`u5Bi6xy(d`OuXAFTQBzC% zC4KPHrHQ5edb~TN@ypVF(~+yF=URV7c~efcod)Fz|D5s#lsD(X%sx=wgBORq2j%4t z>o^q3TQxQA4Jhx!1I}BZyzaZJpMdg=VdY(gU(>nRRv0SqQ)46>=T_jg$F3pCGAlQR zSa~wC@~OniKT}wFM}?Jh%a28}gdApvq3W%)Lq<(neso2cfSq)N-lO!g0c ziOK#jBK$?^Z+4AHp6+by!4(*BCtRbE6M}TgPnX=9MaOJ8te&n_gN~VUn67Wto@16A zrYE#&&M{jK(+$@!=*WvKrgx+C&T@JeIlU{UC*cDmCdrB2<-{9T_P+_!d*JuCP?jEY zmYd})zlrp38#e#(e>YnEIz>*=Y6rLCQZw4#F=$2lX}Dgq_0C1f)7x;%zoMMKfC$$77dEzH}ETh7UOuZcO#dlR?(#1)n|3G?36 zIwC3WE!>JfL0h)D(a)d zF1OT2A6D*(mDhYNdZvwjz|Q`4y@YCLuOZ)&Hd=f_(MEllsEu0UQ;`AryIzj4uWNbN ze0C`42WcHS8gRsnL6=IKY3F+{Z8br=4T3Fr^NzG2vo7?#$+NkmCT#vob_?aE+dhFw z<5mgh*#)>vEYE*o0;!t9M2L~LNq5+825E$hdyC zAq>}h=C_XP%wLP=9B6XQx)ykzxsvega3YR($^LHvzmLIt+1@F{rjA+n#m*dFUXJF) zHhdvA`}e7@kB65$dpS_g7a|HC*N%P}jUjQMfiFZ%>_2;Nm(mrquZ(3(;tLTkQ|`Eb zO6dx``8vz8Q&0}eFMAdKavQrqvyHGkE%bv%ei0Px3LcF$_Id~pin-nGNY1u`cAg2{ z+?Pi`yq09%>|$rUfXe5Z3zv{G?XlsFfq$Xy_b@+Yw$$zBQkE0{$qHH>DX}HeCa521 z_W5#o!lAsouVPO^6?vSGrJdKL=U>jB|G4wAUOX`O`MT{oPUrz23FVGXpzCI7fD@x? zmCf9To5eS&1~q>T`FC>ni6=e$oK)1OE@;-uIhH8{eG5eM-R@#ws^pF81xTWX!`d6w-I(O5x^wz@mV z&+Tol^N^|zhR)Z_l*`hWBF=d^jxCZrA5Yzvf;w4@ z#$lvs!gkFy-IzStoXcSHVi_mo#WUv|nG7tCc3DC4*O-Isuve4!5b+oDWT3s8yjy)t z!pB|eqY)o>s*fr7xI=lA(h)2>X$uq&>1mi=&eAchG^|KpWRWG2MS3HP+#XqEai~aE zN&y{?#-t);)JPehXf-rDv7XKV7bJg6SWM#z(1(n(A(Io9(*_h>GEW*mfJ$H4i4~Iu z_JLd=SO{`WRt}Gqs|c2XTz9YySPZQL+BHqUI080YuR?39LTjf&Yp+5}P@#2Dp>;%P z27=a&K{Ke(5>;q7sL(p8&^oKox~R~)BD5s-2%wTwP~BBfH>#j+QbF}lLETI>xrIUd z4MH>0o8K{jMit<#D!|)RfVZmvlU0DfRqJ*K^BjZF?q$$YRA{LxwBM`H?o^@Or9!(~ zg?0}@OCx9xFlcEiG?NPLJ{8*iDzu&|v|cK--UuyS#ap@xDnkYJpbF|CD)kWtt&a+= zuL|wqNOm2j&xdv(5dZ%Fsq+6es-#QTL;mTsXL#o&{oE;aMLIS`OKy~cmzb3u16KA0 zh@0vri;l!*|LM@ZrqAi|0z!6QLaC*lJiUik05pOSGkHLSiREBG`P(%{_(Sif4}Fy= ze^uRpJ2JF8a4M9p1zz?7nxME)U(T26*iLPl_}n`u;2$4$KRg2kWc?4VAp7xS2)3L?240p{ca`<@rDu7Z>U8ykk-ZH) z0P*5yWfZKDE|XIQybgOo3Ygc6+G9q6Ub~V&9`NG2{Wy_OaDr0kYDHcYHqyJV>D@NW zAdNnB1po>gFy8muD{S|UP)PZ9F{t}#`eK1Wx^hQT&ur!#ma7DOz1EMuZZjmUr`%eZp<>sJwgLrAZe+;xlQ-+&NEu{7L&Phc972~ z&O2)@s*c4AjF-Ky57lA<-r0?fspNzqZPhm0T-RowN zGnj3KY)4`>1?#XPQcdL*Kt6lrqE~z=5f4%`L8MJJlp=pB4Mf=N8#%}K(*5v`RRCIZ z2W6mdXP638F4~Cu!*gmBlah;y`I-{)pU1xqDt@@!a3uwZ?$AQ{ZBoU8qx! zLwYk=`rSPs9l~D8EbNQX?3gtFT&+2T&~SKReD6AZIW;2@6{q@*fpCR|o5(nzd@^)UEYg;w#T^^8ToiI3wcC7*38lXd@ zO?aD{YaI?7mey#8Y`lAJlFXivTwjJvc1X42B`)r5j75Eh{DB>y$7^@QZELYD@$=~Z z6~;Hp$Lll*6+u#|XAoL{Dw3LOn=mfa+y!7tpR$PWDGh$(PPQ`|{g4lGcm6?csf+r!&m)-hc`wpAusf`BZFmfhE_BCqmlK zlI7D6>`9hSyV#Q~pFUwvvV1zso@Dv7ojs+QH#7NEfo}IIK(IHX98^5-p|7?ck2oGe z)(KQvVe-(tibo679K*|Q_X6ym`U5T8J6bb#I&_b=?^#Cf9iM6-v9Oj!jnVrYmY(eGHl3K}6;FIt)K>NF|1pW8cw?NBV^z~;Lw}?bb(zxH(1!cPOf6*ip z*Zdf)60jt{nFei@lo6IqxPnyr6OS$ePUHL(FS=ksi&g1!9!CrD_R%;da zArWYc{B!Vy+fA}d+>3_+*3oE6_0;G?{-$-8Z8zx}?4s@x@6K2FtH&mK(0wDwB))#X zOiD14fr%k^btgO`AHtD&Dm0+-Dm>LH9tvlxy8Erbn2Euf7)wRg%Gm1ttd|?-09)Mw za_eGHsWWm|o6KE-AGPNd4(s0qnanhC;(wwJqj2cAERq9}9y4^otFbi0?L zsQLSo>*&@rc?xY9^qZ-?W(D3SZJf!zfC19!OEDb03OQolWc_o43Eh3TW}9Z~w3gq_7HnU&s89Ql|D{?p!ES zEX(vUWTKKlEtb*L6EG&Z!A7A+U`KLQ!iCz|4R zDW)qbH;eVsr7lf$fPd)|It`v`CZ93teuKj4TI>sD-ZyN2E4DUCdgB~cGLM61Bt_g~ z-=DDX0AazTVqxO8NGwRtjv^E!sgovQ?SB|SJC0G~9ph~78P{lbK>Evhx!GQaHKN%8 zY2P_)cAsQucKa^aY;+2s?ZPfC6CcNRJGx*kBByjLYWw>|v|sVB@f|OM0rMVe&0xw_ zV|`HPu8@`SxVJ?E=-%I=?hSghhVYsj_h%7RZn{Nv+( zTEJ{pO2Y*SRJ}!kCuN&0X2K(8oU#8q>Xr>4=~|onF&j;t<%Dr{m$xq$0j~? zyH#5SR;qsfXs&& z1p@vH`}MaQADx*y-&~kGQ={4ZDB595G!aYEfF|TT!|^IUk^-lpe^S~JhANOv6wWl7 zXU?5JX%60wM=y*cCe3viE&V2gbTZ-_;M$ZhY@pFP^BKo8J|PV_+|BS_qW4cvH)hRr z%*pi$CU`#q-q+ImSqqKy&nMgm@4-qH8t6S#JA2Zcxsd;UcpnGv4Xh?lPk-jQrxAWn zcs~^0Cn@=jmSOz{`h;Ha-iq)qQbiv%k1(1o0KYf9zYXDERPrNQe8L0pet=I%hhVo) z$bjGt5RE?JL5P+3gohyb!YA~B;Ji=h3&9wl@Gt}xpYR9-|MUr&5OnehW(XGegnkhG z+b8sg;G|EmK(NRsWUddbK4ByTXM6(6 zoD82Z8iG8Z@HhmceZmtE+~*UXgrMFhjDf)96P|+LE}t+Kg0((j90Z+x!qX7k2?*5G zhm(X)7%wASUms2xK4F5~p@w>yFosQ`Gzbp)gl8eR-6u?EPctC+!6(ebp7#m4 z5UlYDvmkiaC(MQbdVda5mrt0>K5(#>=0T9?6XroM&nL{s5`Dq~2%uF9A$Y#-u?QPk5aX^BWM{;}hP5;5(nN z4uTCn;VlRz`-Hb4=;{;JL$J*!`~`v}pYT@*`uK!*Sfk&C;O{=+JqVuj3GcJgKY-u? zpRj>Z@!uf$&?o#If?hu1LkK4NgnvNL#V33Ofz~Jd6M{#4!oMK!0`R`XN{k)DOXFpneEe0QE!gJWzif zjo`5mIDq;gr~>MT;3FV*2x@?~>u6+5WZzAKAO;8(f_$LRI;!r}IzrYn5cCH6tfM}9 zwvHM$y^bhk1_U1feL`>%=o5kjpwBw$;@J?q0`v(%HPB}r(X|7Dmw`T6t>;1TD9|Sa zZlKRPBItz>{1)gFg7<+w>xd65g1`>+Sw|G|0tCx}KG_FLAUFv0Sx2$2KFdj)uXBWQWc;Vn>a9g*z{2zCMWLa-F5w~j{O zAL@t!xfo_vK~N0Ti;6B#FT>hP5bQ^}Uxy9}?nIsOsadpfU>fg=`=Y)#>XYUo(1&os zl4hy0a5AQjcj={fe}kR?1v2{4p8hEw-PE8fsq7?lqf<&rC?TcM3z+48E6#04Uw%go zv$?lvNI71Y)~IOOHu3xxR7}VzX%*CD*ntijrXNUua-go(O9O8x4=Q6Py4TT(-8X2m z4uAsoI?TjxQr!qVZw9~^49_5HNE}P1IoMdPAeDT}#OB1P$1uE2Y1ybw?c%Bjr`D^NXY=2l`BOrUt%ad2zT)m#Ag1YEDq__Wh3*>U&-4ipr z`_6y>=hq&VKIu;WGtJzl9MJh6hMd9{N^_QIrPmMor!ctE=}{?J)N|5b0Rx3pYmgDmvLW9b!oE-_lI? zag-BwnaesE``A5mzwe;QGf7QzuENx8Nn3$glOqi(v9@lOb)tqUu%B&S8L< z-C_2BukJnRc=EJP574c}b?Rlq>7i(5oq7v0W{~3I@CA$9k#u zA+;{sE>K-MUZpPY1D*J<&;<8K8&P?1_uctz2fA4rC!E~dHew*2ldi16G2i!nOcd*+ zbL{aYdTf@C!lTW-rkdsr;%39!X z%imxdy4<^swc##m1K9bP6&;_5G%Xp|8G&Pyc0zsv%SuowScaNFP`muRl`vEFErYu6 z!OP6&N5O+c7=249;oz6}wUksuhWNMan?X>(mddZ0sHJi zC46R-KbtInwu`#E@5u1aZba+Z-DMH`Y*8hvVb2HI7q8&#=@tcPputYHA8s@F+`d-le2|s-~uK1$VR5acb&oYHB*Wtrd4~?^IG3sHvG; z!FlBW&cP$}Hn(v2B8`@{dy$^bgUuasO(@;|4|0Y%0%)V6mWchaFQ5myhY>qc`qpkh z>78(eJZL&3y*L{q(#YLcf_@VjZuRhX!E7erGV%3|%;!tf7V4fOfY>b>E$h|Q8Q4b- zHzb#QO^m9!f@&t;vKy-0fYnsFC!liov&ww~@V0b|s@yGDxp_3y>e=9w&dwrozY?IY zCg4{$C|}_xXS>2D{o#|Z-o_5?klKs{_Sg=Oa$mn%th1s6Qxcw<+Tq7xn2g%4#SN%ALrc5cI58W@xWqL?jjNmJ8k9%Vl7`mF3C zL*;k7>He09X!Bo+rOVo>ydPN{vq{GwE-i{x{oXhdzx+MHEtR^uh?C_3CN7*KOn5c} zCOU>=;&>Op1R9@E9=g<6w(|;2GyNf~xEI2Tn?v!qcNn}UHp`=2h4)8<=O;C$?;T$w zgKH#INq;-4On0$MGkKSRyetumlJ0Cnz6p88vGCfk2BU($xZbR=Nt>g)@wZ*V1+`;} zqI!ktT{xXqHIt$9{T*ytP5LNCo>Y4`hE1y7#p^VFRHLD^vLC6m`oieaKJSc2HtQ66 zUnxV)2Bpz#-rgUF##sN!MHV?}-i)GudW@|5Jd#jm&ArAW)PP#BZ zCGIsi?SYpsQ@3AuPcD+%ln|4a5YMw+1UKYKz=Lmr;hW}ZHA5O^5YlMf8OApj4WiS% zx`9!s(zw2_!1!~mSwKr_0^4cerg;?T+@_#Hh~`3rET&2YzKPxW&Wr9s5{GLt;f*ZE zgmmQp^}A@!^8x77pKGX3t5J|ADt&4?guQy~h6rU(*n)H6J`z;h(nzRx`C^kS|L!Nw z^)IL@mC0%ac*B_q(y^quCH(exd7~Z_Rbkd@CN02%B$x#t|{!I?WL= z-}D=|w;HmpcfR$GmH+$ZTO;>t;RLzOHQQjv)z4W(JQr-PRNi%;Rcx|l zoO3*Bc4ehJG$6&%3wrp44{ZwIP-V55;T=d#p6o@rD zgZ;P_Ik9XLIq1?$(0x-meH%cZ+qikdF)K?5TEu3qpex=}=w6eChmPHr-nep@PyXZf zqTL6kfQIejDAHP1n!(;H%d6p?myPTu6mB^a$d}U`i{Lkt6H>JDP0`;=g{Ol-=3n*_ zbK)@a?}(-eh^EO5O*dgz2!w(`kHl(R6!q_5*;6R{a81Y&C}x*#eS}uH@cs|E-ssPe zO5D_>l(>QpL|^W~30Ofo%4PC^*ugQ5N1f_z;=WP=F~g2*6*MbHwpKPPN48cqD@V3g zH!DZB(5nSIvbDBZIkNR;vwUPrK+%FWzjSw1V~`z#kUlO3_A^Tsq+R;yWS7jRm@_-e z->{8&n$TJ!wXyHR)W#xml>r}816tOd9q+@sv*UeOcXqrF>&}k%VcpsBKCC-C-iLK( z$NRAEbi9x1PLgf*k;b6^ks#p@x_Ldko0o!J%oVI+ScPuJONIgM4F7bA{bc2S3&8p9 zkQ#yW{~oISYnodv>%V~-e<~x)++D=b|MdYe^i*W%-{FNurSWyaaR2mS_et=0^*o-+ zN$2r~16NLp%{3+sZ)ukA>$QpYLYphkfLG1iT#Jo1F=xF^9J9fZXR5P{Ug?c^JX+f8 z)4!AJ%eyc4)<~m%#;XwG%n%1)8*F$T(#foE~k~tgfAo6xxz`cI(j1D)Q zXZ6;-KT>gWIrh}EAfmg#N9N(?VNf;m$Y9<^sEqkB>8l5uaXKN5lyPuXyp@8T1UK3AwQI__$LWDl7#GYs8n|XhOvpy%RDl+e5{SQi~Z!AgKTV zhK1CY?48@JSDN|+E6H1A4m7Y{IS}}P3Q7urH+WBs=Da|yqb(;42w%b8#N=75ml_Oh zsSqUEaxBGwgu1t+d~TH<$@2izx)~|+7{IS4_9lr>WpFnZKd`09LjNYh2BYi2HyLd9mK(vo*CCdp6B$wl)vX zEU`8pkw<%K^A7-3Q&Dnj^N#>ov!BL(N%!W9P=uzMm_W(P&AXv2jReI2Ab?ipKf(M7 zFal`IUj&130%4yTKwSVsxf?YFA_EQ79*E@{SznwF%ui!Iem;<&2|r`Au}Xk%(~%vF ztwF_(RFs@=D#p%Sz`OIuK@E=_DY?`HT>ahTCB&-djU%oHwLNmGWG}F~@rg$QCFM?PLH(j}lNd(F?~f`)l=3H1{17O_Qkw>8uC%q;Kt42{QrYqRy% z)ArQ*ISJOKK@ITexv{p?Ca6z-w$1$jgi|tY?ycR>N`Z&mG_W3YP$h`ZH+qkoH4P9O zY49E&t*M6CE#BipH4=$|H1Bb%rWT^plQX@?pMm~w%y%SbdyhY@*$>eXFd`fCpG=-5 zy>SUZBib0o2wQ|H!V+T0+b=^C3r)#4d!WfS*MbV0Yu0|7YihO4HBRDPQ^7zS)xf(! zv{zi%9(CUw)avOatha6M89^lG)i(Fj>q}+_b}6z6ZGKg#r3Stw!FFrlFbTKrAoW1N zl_UnvqU0)R8~C}VT^Rjedr+Ad82TjJ-4l}y=HTwUN~}naWJc4+0>5LN=2#%tz{vPm z0I*7A{B&So8YAOlfsryBJ_Z!bXw-CM7n1IGNUc@y!I5et$}@~07bJE>R2mSKMnq*A zqB0XvnT^y;C<@iTwV~T`&=OjdF-y2nhAbk8hAm=mcFwx(E`MiIx#CP%=>NRfyK+?hT~cHIvNkGdd{-Y5=Ixc)Z84CC}wsfJ-006$ur| z&wISPOsYVXi6K&Li!=?SnkJ!w+UkI#EYMVo|b@u|US{ie`L` zpJXsb0Hv1!<6+EzAr$GGVHAm+p%W>c;gZOmkojD`E!hC@9Za1t*Orve*hhMjroa za7+9%8u49OB1z?<*@1HJ_R*Hptf+BV9pY=VyeFQ422fGiUUSBbGJtswUw)J5HJbuu zSSjyzvn8kjKqpbW%f(cCi`vfEgW}zji17D7u|InIn=D@&7``_L8cY5i_*ur4Ik4M% zqOIiwd~5~Xp(t_2cistYEf?V1X9C}q{2L@c!=2YWG*Cq1nI!XxM2n>0a4(Rvx%-S= z-U;xMs<#Uj&~x7Y8CX?9;A0Yg>*#n zdj?mbC)LXU-CP{;><3wo?|vl@SNRjyu|q=pc5U@-;^TC&G33HB{9kd@vod z2C4}D9N-KIpeuNc_(1i8ycTV3-jPritR*HL*?&RtcYYo4tMKEhaQIbJ|1kJ+7HkD3 z9lZwpk=0jk0{$Fp*$(`vW`%tdl{wTR{HcawudBRJeXEYOY#087l^32l)I$8ZN_iut z@FLmalIT|AkN+x7#UF{4U*6hhQH{nQ>4Lm(`q$Jas$o_BtDpjZ!s^rFvnZtCPwVvo zs)j59YT#a_K62v%68zQ5jSMt!1|3$#P;+ynK6n*;96pafm#&3S{2}-atSO&e8&qpZ z#vdB$4Xx96So^BN+gBCdzN%kpUu*cS!b(+Lx>g6_55W&>-!(wB)@k@dtmrE31A1c3 zV~q*#L1bPHlP$!-5Tg()Q`^}2dixJPOFvvhzvb80mz38-zh$i5>#o0uYxgncIYOY1 zl>J(KR7K#U>YDhtRwqH>RaDuk^6Tq+%1f253d6^B7IAHwz&u9@^wAJLc<1q2u3%7{ zM&nw_#GKYD?lg#}+rWoNTHw}K~XTnc@_5}-< zJ8&HS4w=QfzasN}=@|@a)ctE98*SnN1Y;2^xWuOn1dU6aY$S+W;>VT^ctjxKuva+xcDr+)9N5BY^z=Z?imljhy3;OC1 z4c)-(H_n;G8K8EwErBykF~Yk{ZrKKUf;tv_NwN8vY>J?Qp7OJ~f>UzE29C?(MwUzS zo4^=L?20vMwHp2%vC4gQfPW-T-ffLgQ`*7{MNI)n{y)P9Y+ohrhn8Xcod_n9A4zzB zPFUIgv6Ls2ez~DV-TV&`bT(c_2Bn5etybUP8P^65wVsG=^IwQ*Q6H6>@Urv&Rce|V zrl01j8flH?4r3Qet+bvv1^AbLoqEr_1P(jfd13xlu@%YraSMNQ5EWu3A6;Cz_V@eCX==mG;!U5Jn$B zzex?(jRI_b%_9-z)&FYzaU;nLApM_l)eB}WHw`#%ZiiL9X{$rui4cMyGH;;`xbE(l zqdO$*BOwHM)YMy3i)%DUd|;D7m$m_X!_%V%Gr$MQ1@ZpIkgU8O^TEu0d!7&O2#HPJ zoo}#kBa5+M==FFk_j}$w&ycr5F=nvgW-hTiHk{D}wDf}Nzk;<7>XODyZJbaI3n3*=E@;wyi98*KV-Ftw&W)vC*)~^d8R`io4bqsTyIU5YnTo*Ej_N$ zWa=>EBE%2W+JQCBML#CwV+qewqBx!__zC*J?Cw`F$Zbp9SiiqW()n-Kkus#> zSC34^l01$OoF$^Jn`=v+MOcPj2itf(p&0snHK7-~JOKzpdo{zvx)Ht$jdS>L^6HRr zf}i(*&ttA;rI9X`D$I&CkYYE&eoIe>H)j z4;d%xv>?jTj{vIBi#ui^-OunZD?tiInU@h z=g*gkW9Yd84=u!6m!Nn9KCa`IjY3vkP^lq}N?{hLxdOh!=IxMw8|3MZV`#^NVDA<} zngtbZ3+mco=#zQ}7z@qC>INzaWmw_Bp$l9-wk~iIBGkOVj}W8w1%84U8Xo!cP(hF9 zmbGVU`PgjG11@7zxq=3!<4i#l(aX%hA6n|WRDEn}uq>*lLBi)9lchVT5!e`N0zT1u zKQ+dJ_5zd&+G|$`4#|B%)n=9bkt$pBEHgl_r+h5>RH$ew5zCj0fhNby=|PoIN_2Ff z%E04EvJyvnOg}104=FrQAylOm9h^(mC6f}wrCNaes52pE#8@T9(lB-iW0@E`iLq>q z{fIH@{yL12u@$tso^7zYZ@;j8HPd&TXJ?_Z18P*F*|h|<3JcSv;0^xq2D>{mOhtuP zdUzwNYQ*uaTFWS@8S63mSSY!SaT`d!#HHGj zQGA2BIS>qg@`VgiyNeNYbIxElfNE}Y$8Ajo`#fxY9xZdkg@^2NHin4JnW@(SyBCzf zb&?2-jC3MPA~iDAvHVgd_QzpTojd@tjHtTcn+D9XmoYzx;d~wFtTd3JZbU4{7@vli zA5%#YMV<{W&f^Qza6ixj?tO{Wy5Z>ZLwebB3R;>`_ zunL+hBI*=48d)cWm_l{3kP6S>VEq8}Srs7GuUZ92opL$^aDODgaBRtqyvRf=$c>`N zYQgxNQypF_Gk3@nQ7fFf&(efi8{_C^X*)5RCpWG!gTe^7k}l=FU%3U|_4Hv=bpX z-zjgy<}LwVTFuPKt<^^sUX>_J0dA)9HDXCB1h|BA1iX)NHd(@URRk6upcR~EG(1-OD-aE&SSwFO z{a3|oq=b#qAO1(=t;xgJ7vsqjZjFZr9S`qAyS51Kb^HJ2BG__u5u9-wh+t)Y>zCwj zRiyl_ij==qSC_w4t;^r4DEV6zA%Cl)DuVl)n+!3X{K8k@B}HQvOy&%HL3}RQaoRQAqwCZ6SY;GBPT5^=hqic&@;;|$*AU3&P z6~H(Gsydz!AZ*R_YD=HfzaX@=s~NsuM9GHE~0g#-5dE*0XmI<4x^N(sm?kY6~#(g}aEG}d~Y>e1x&vU6Lj(WS1n z=%nsKKQ6b=?Ks|$jHN3&Cg+2)LWyYP{vYZD*V@YX2q+wYx z3VNqzlya?HC>5<#NO7rDP&B=b6qW6p?R(i~oAu7Xj0%XdDvAq>-~b{hDB=SD=X=hb zg<(+Zec#{vem?(qKA-QMyPWep&pGEg&-0vf?>%=-v=WcmM=OoVbA64;aUCd>=txA4 z5NZggK}c^YCsg>c7YCy8wN-h-qjmgp(OcB;e)gx8>@Kvo8{m!!??u1z$?hF8K{i1W z$;ie}XM{14wQv!?Q@v(APqs9iMd(+`eY@EUlj^5ngU{Me`?{*3dUosi8?FfUmwh;y zb26FgyZLkvt}T_Xop)nVeKmi+;9>dN1rPSjlw)2$QbFySF#Hk<7~{O1%v6qP8&fo< zV=na)lOwc)+IcpW-B9(MpMv49A1%c8zYbECVt4RO*geS{E{;cpVs}5gx#8&dwPiR= zb9DaNV>q;O3r&ZoaA@PUnz>Lq+{Qzn2*g6P*TQZjx~m!KXh#+Yx$BzO2?7=(Ip!oz z@R*3jK^T#uH?Z#5bn#gO@DF#Wa%B%@l*7Hw-+8hZ{teIDu|A9r#pjno9n>%X1!kz4HG*u>Zp^ zV*l^z;4*x86u{*JivOw)c)GfWm-1cO{-5RcpWiyg!ICbNZ7H-TVoXu*d9#$!XP(d+3*ke{oL>(dge=JT6)=<$;hexsZ>A_4X5a!#Y&v z;HzCyj<~M5{-c-_wn~#2loMjE{~{)Zc-BD_EZMV949TsGm%pzH;TAu{yaHLuKs*lR zWss(GJ%Cjk4awbZNypPsdOv*bRCu-%jh%F>O7M^r)m4;k zRTG|q64qSLAB&=qwLY%8k4Jk)2mzRyR`UmEW8}vc#>L^)395xnc%EL|Zfjx7telN= zy*J_ZG<{ACf0|ymx!v9#!=7pvlL{2#s`5lW0~2;AT~uCRGiw($Co!!aq1R=nT->n4 zw0cHgU!3xnsMq}^>Ye`@arIvlUBe%oxqSM;!s?xuwBa61!v9(kqT=WOCGydKjeJ~n z_jc{6b6U@hEt`^O)9vm8IG(3*69rGqM7dne*~MUpnQ#wObdhZ@D%ydSrNjZ?e^*nd2%dL7?kbpK^OV%mi(}b z>?!35N_lFtg-uc->|zs@n3trJmEC>iXVck(=?izZw9pZ&<=R}@n;iX5vx9H8&=G$Q z8v!;dDccqJRX)dGNpO3A;BIEl{+3B)cRy%nyZxthw<$%h)z<3ypwpsM$%_E-$-X;h zb=f@ix){E=-Ck<9DB<~ElukL>8v<2o;n|NA{0#wWv;F)`u7bZQKxfMM8UAo^Y%rZU z#?M@>qln43X%Vxq z2eaitPwN}7ZqJJ!Q`xZs8%54*&5Q3Blb7GBX74f!LO>@WfW5zrXosJp#$fkriqf6u z1IDO@fP3r2;d_@BQ6f17-g54Qs0JTQW|P4#e&TjB^Z#pGgf%m<<^*EChuB+HorK9p z49UGdG0Tr>uPRlj6vStwdR4Q6-A?(EI_RoqGm5_E1ts>GIa}L<^8bmiHnOL+6*M$n z#kwyEbjRTxO8brMsbSjnu?ISK{p{TwyDGbnV^?i&^p$rt_G6CSaC?bkH_^VwvD?p{ z>DV1+H{-Ra0EHA66=JS@j=dv=_Z)`U5lFG`sNoBqXBi)u^8_Due^OyXaB7Va7B(ow zS~Ah`Ena%DDB&PcwJ@q*c%+Xe)i8DyDIx3$F7>sM`KrG-{IH|x36&Q zCfdLEk!w3xBuV=D$kUrt-|KyOO{e%e>YdfKAHTi_6^wUW6a^@8zN6j|t5Ybj=nM+l zc863rt`JLz;*)_*etd^bsO7QNKudfe#c4>E7h1g!Mb*)bO@xK3ovQf>S4|<{0kvE; zUFCw2ruKE!5_{fn>I4ROP)&XElDMbZWIl>8#OLl}-i!Qt7PR zM5SD#YSd^tAI?*K4)Wpk8SaDn>?GGGYLWgQP`7&D!M$EDQJqXhxeik0#gmhW%WmQJ zFzg^{TC96bd+cNqawumM*@kcpM5$g&a)+2{7~R^wZ8tZ|OTT)X`jM@ZyT4~GC2T9E z!K9V`Hn6|+7Gbi55~et=52`{E^cFc8Np{YU_Nf%<6XmXAap+8*>;2+Eg~B#FMOlM{ zZTrUx0aI83s=s&fS6>$YZKZTQpZ+$mzw|GJ$vO3De|7bXloZePJMd*b2k`S;pJQD+ z=N+fJbr<~I(V=4}KX)kQybkp9fG%=8Tk%7`Ab$2d4POo9-?8q8~^QF41BY^f8roD;TGMw)t_jx_aejx=eH3tJk6Ef<6>Rgs|+PWwiN zPHpy$H2Kl#)0RrLu;o}}=&aMemMV2*XvA?}OSRgVoS?SY>BUWuIeny$aFrA63Mqce4)l>D?(XCiq`<5UqL*%2W{5g{cJA*m2p zSUZj_5bO|G*a9xX7O)YvFg_r8`>uR5w|AwODPDq>f9&Ynv*gzs?j7v00umYGk3rM|on1Lp9TrY9`RVW_ZBk z?BB-ZXVDLJre%pb)7(vwrYLiyX-cuq6h?;{It-`7XxH~bJ@muKUWnY1O(-Ii5K_5( ztj>;)5w`T~O`HB#(?)weZTvb?ZTk-swlvWH;Jbw_O*;C&nKri#(^*PXsA5aGDl#O4 zDwD`%P~}Ea6O^kWP^rR}5qDFqSt?YdAC{w3?{!4fzdNIU@w;ZMINlPhh&sjRe7 z>ZDYiRK!}fDI&xiK|*c?O^Z435B8rHs2i|zKmorNAyw!s8EUCkXUS9(o6b7Ru};*k zmKxd?(Ipk}yn-&OB7!P{YI%!ajQLKffb-T_cB&&nES4_qNI@2&qDyfV`XIdFf zry@;EL#Q4or7EP#l0`kOB+~ReV$P7N%Bl z8~2H;N)R!MK|3iZ)`(KPRxL^q+CaUuS2jbYPSn+@r;{C^cJ;e2dGIX;X@pjk=4n-i z|F&?%AnjW0C(wwvv}@Eu)~fhd~dwShWGsgoY`(@A=jPP*UUU|N}N zFfA!Gm=+WpOfh!Rw4hEjO+ON~;nT>>bq9Z?N=NWvn<*16f zq--8(sLcDkUG3^))J2iVX&T+i)2bs9tqL{uWa^2njUqGjUQz~H$^+euA*z-cLswEV zhSZ~3cXl^}DC(J(mbTXL-)#?N*CA}Fy@)+L#q6QWP*H`!5W*&sk)*1HNv!535%#f~ zD%jm5dfLNAooo-8T}10p1(Q+Kv5K~naVEQHu@j|Zj;iYq_Woesvm;qQ)L~g5S;CMy zGK$VRlWpM#N-QkDg?)_1Ql=3tr5e5IPZbV37mA&8^rmmHW`$llp_dx;11$OhI}K8% zK`PZ-Dmxj-Q0mEo8It>VHUwoFf=cxR&gci669-g_1FG~Ph5C>@y>wD9oiYwcH4Zpv zkn#*tm7YxOxMY9ISwsKYeukhtLr|4|K$CvJc}LED`jC7_&J5##a#v1$h#496A(=ei zxa6Ll$0ZNe>O(C0ke&JgXY~UvIH>H`hZGn>vJD}X#*l1dNVO=XiBg6!BuBJVYm6bs zL`xO@rCQ7?W5_{^g|-=%G}@M1QW1v`J55ALC9$Kwyoiu0BN=-#>_+LJj_f+WZ048u z>B!s~r5!qpjsD8?7CZ9ji4Wz|o5(3z)Os?-I*XP5zhT;GF{_QHSIx{UQ^qth-6clT zH)xB9khF-9{gtyW@cS?SNqZc8O?&oOwAA?OmB;PRpZ~jN*&)B1@R+%|3Pm!_(+(0% zQ?DtZkiaqstHXIuU*xIb`aLQOWRdVd{QP@=^SZQ27soiqiC9>5~WU} zG+71Z2}!T+K-C9pO$K)A&_o2wWq~1u?0^}Ilkx$(qH6Wu1QX$nv}+;QAuM0L&x|_WBMmql(+$y)YN79d)voE;d6W#y_%G=6w_5CiJD0=g9MtamZj+w zlLC2q{!{VDyup3<1XBw%3mi`&PD{<3s63ODJ- zk~T~VbV@jmgq#)r2Qaoy1*~iI@j*wUamr zAdLxTCA%$jOrSI-EObnNY0L;b<6CJU71%x|xH~UsbtieQ&pG~e>F)#fXcnaSX_L+< zgtZ5#!6HqY>sZE?dz1ZtHb~p_$s^Xsq&RCz5KWU+&Wh?TN~B`ASCO1hCk}61yjM)_ zFs}z{HjetkvS86PH_Rw)*CicSrod6@O`Y`Ad;e#l{}~Gn^9<7ro9r^i+zRWYPCAK< zuSCXIA{$8+FRWdSwH!js27Hswinm>=az0W@ zC;OVx0r$Uqr3c`(s3wI`h&V}yPk({A;w!{eXBC(OTwTQs9Df~X?Lkag3BQ&07gp!8 z@4al*ZpH^+l8%~<(n(|TyNb^EMCy6{s?rw9pUTyj z?beyrLEDYeS!2*irvPUm5mrt~E%Fey?YP4knC zDrusBD&OPv=C}Yt;Rd=QUPPwalkRmA z8lepjovMupHE6}q30i~E6lgR>s~}eVAdm{jmj}dl1Y?p{ZxpFpgc+o34ALBxN6M6> ztz30j;|puJ<6wCa?87`Q+xux zx!aCT8K_feSh?+sUD}0nm#1QjKhg7ky|`~%H4EkI@#v*Ay+md`hrEYD+Kszl@~tsL zP>#qS-XYVb-=509Y+GoMN-FPjt=dbOBb6Yvn5}X(Bg)l|UbvO(>yvHDbrCgL z7#V-vB^9N5{Y7-7`X1O!x9cQNI8nqpDKJC^ssk}OUK_4YzE_RP@O&OZK3)`#0>&ioiSvwZm);P}#|ZvdyCc_tZXIeD@hu;+;< z{s0ce#(oF9XtPZKqJx8@03W5&7uYpt&ga0Sn{HYG9Bppy3;eWq@1ub6%P%W{V@sC2 z3A~Vz5eZzgdi6Vil%5_AoL#YE9Z-ArY%k#Z2Oq2ko*y$N9XMB4*A>ux{q-^6hy46G zz>E(*xBx_c`)w6)-P>>f0~r6&M~%Rdg$rK=-ZGmX1{VMPb1G1B;X;4lu9si_1~6QI z{o}yAVZ$B*R^D@u4mj1==mR9}+&K=Id)sYOf%`xHxBjv51{h%^9NqpzWo7U^mW(G2Hr_an+^mA1`YylfA!V(fi4|72*53G zzWEc-y;G;ofcA{-n8i)@X-SgoCcQMd8ZNhdC#6l zfE7RdunTw~DCimBU|HGS!1P;gc@pTcX3cBBC*#MT0)EZOnF}~>L|AHS1_R+5O*Bw; z{`@t-Q{R2J1K9HL!!;P>q8CjeV*?Tx^=FTN-TYR;YO4Lmz)lnr>(Xxt0LgoaK80@kkG2s{@Vxf`f# zYElD!K0Y0R=7fY*z^16EI$+s|5fQ+^_ul&rxbL&i%772XlZnOnCW29|EwVgbJ0zke|>a=?I@fJUtj2FhDn zt^_XB*LMg0k()aoSe2D!00RB}dji+@=rI&XJ9g|A;LjN|N`RrSz4igHJ|bc#aC4tN z!-1Er*2%!cPd_~goL{+eHSqLZcZ~sd%$oHT@Y}R$M}b@W_Pq(HShVOBz#JXD3Fy0O z)$72!^TDX^w6QFfUVP~mjb=JcD)Mtx~OOgP*hxeCot>QTPFeWcib@+=-H*q)xb-7eG%Z_ zv11oN)N1Dezf7Ha1b8nsbsF$TUfu#=)K_2C0@g<#{R9~P;)|aFA+Nmh5s-J}$nC(% zr=Bta1xJtG3Yd267zHf2{q_fe{qyJl3@DS6Hvr!i6wCtN9y>M{(0~4UF;F;f-jBdn zlO~-7>Kht50lTZJ1_BwgXMYPcJpJ_Zz?g5osRYz2RUi=i+iwLx-LhrtfqMrIj0L9N zeDgvetbczkF!_^DP5_I3{Bb+*+O}=?1DSUFb-?=v4=xAn@$qj1cVBbO0N~a8?mGxP z^2Z+sfNmz!vw-m2b8CU(xVRKx`Q3Mq1fKlqCo8ZsCguwuMHCBxj=sJffX36Ol|cXQ z-LC*rA9>_M;PIhDX8_+!p4 z`plUgz)i2e{sVArKtL#v``BZ@1E25P7YB5D{&@)q-MH~nV8~lp6DPI+QG*6813o@{_-Vj*!-f}t2j74H4A5DU5`inbb?XDn{>MM| z1A8kfLVz_RM{WbM=g$2FSgq5g0YPuPu^D)1$dIML)q)TJJhORoCh*yW3Fm?NzyH1m z7~H$}9l*@xOP@LUM64}XIp?P4yORI1AesSld!B_TM@X)3Q?%eZ3P}wbO#^)$%H0Qs2 z`0>m?hu>?MvOBx>i*uulp=%?Xd=jEYy!YAtABMm1(mP?Nwr<%!K;2TGo8{l**o@aA z`dB|*dDpCIeHTTq8h7B@8&lSgKK@J3L({t!72kSC7kx+V)YQDM9)0nZBTpUOar^w_ zg0Y{^o77M>`{{2~zbzYhbN^3%+-5%*f6aY=n4XKf`==Pu_jLD1hED$VS<6-LmYjJ# z;IVzrZ+z?Ob(Om&4m!NyeW}|&Dn`!Ly)i`CJmL4=|11Ch%K!g!`7u7lH(l_lVLWMt zQI$r47X=iGMERMHVc3^{&GN6G{JX#5(4 z3&5f89?}B?f9p2~h^*@VIxwx-Tmq!sve*Y$WvIIXNFM##hrkngP1^w97iJsUu3=hJsY0E<(l$AO0DpIQdIG3D+jfE#{& z@*wbM$+#^*-TJTl0O^{qVt^I9?)VXydHCKyp!~{^)xc}_heiXBj!%dLzMHkQ82J5( zZ`J_Kt2Xrqnp8J!1meWGQ-BvDFZ2aA9-6lfc;J~cKLcMb%DWC&c%M%xPwvM3mA((m-rtY~bnNhF0g!Xsqt^h|Pnw?tZo7H>JHQ9^6E^`Jo-4T( z_~>-?VxVi%vEe{?bnO?w^e5lz2>h`?eLrybfajWkR}Xf)3pjXphi?F5#UG=A7bhhD z1UzY-)&cPAm>&y7NA+p}`o4Zn3+(x$@-;vTAGH&hv+v!dz-If%KY&h>X+3bX@55oh zQ#+0q0q-yGG6q;aQm6#ZthqWJaExO*CqA15+zAY>C3-$0nD3jEC)thTRaN5qF zxs|s8rcpslfXwT*9RRk}oSzH)a^%EJU~kBQ3}EEUwQ8WpYuAPYYqkYwfVKB(R{*)c z-*pD~rs>uIV0+N}^MR}0z2!dO!1uxRK(}WfI0h`Wlw|`q{m^?J@Oj*?ZvX)s#k0Ve z%30R}pPk=e1vb^)b0sio#o$8V+R&6*V4Z&Kc3@vvjs@6|@j)~2dVbeoz$d3x?FK%6 z`p_U?|Ig1~58U(hKRyBm-|@~7AZpo5-GIku?0gHDuyXXH!0F^3sX$8MFLvPO;f>RQ z)oI_J0^XaJ_$TmK_RGtG*n%OS1G@)a@dL2qt2gch)PXBL1?+FnTmTf^`TH|K*7eIe z0}CDuYyghE`Lhy88u#XBK*cKyW&>ZeRICIBY~8H_YJ1r$fZvY3wGX&AcGxbU>b=hS zK+xvpdw`ISzW)}u@t*5KfNdXd3=a?w3NV9#Ao z0gJZZ{tsY$V^$R~_UA{te-Zd+zw?vszHeLZ zQ<^u6%+1TP9%{Je;Af%hJ8cj7?h5sTHzXdL5q*Ec=Y4;=x7(2Nr{8-y{q5RK#djLN zx$)HI9o4^&NE-jfBYkf2Tlw44J=d2!)^p~>0sbr6i;Eog|9|!Kzxw%K{rvmt$ML-R z<$qz)bN$Mzg8K2rFxZ3nc%SeYwlEf!zpFK4gZDz&tEOQJtXZdE%L8GRKlUX##cpia zn{{?<@WPwoFgAEcUV-<+6sCOff%8P(ic(g z+EaAl6fTs}PQ4P8E3v_fIkD+M$e*Lwz@mS&cRC}c;%34s%S4Olu&z; zE}X=LN)!wWMNkoSyyRR8KbI6pXN45!DED?pl@vn=mh&(Z!p|jyt|g=T!4%{Oql9>k zt5(Fje^J5iEicqg!JDYjQEn|S)Xv50!*OxmT3$>}#5kI+m&>mhW0gqWmXxfvN-k)F z{LaY%URAZbWcg$F@Y2zS9hP8o5Mt&zm)c21v8%9$RW4$i*xG=OHAow!he%>n0Y)dxY;>PT13R%$wr#Cx#G!e$$C;8R4k z7uXxNIERH?V(n-HrZ9Mox9BJivA|RHDJ03`EF@HpmN=tgj778h< zz@|n%;;Vz8d!EHk12$OMGZy8KWt=g5vsT7AnzACC}@OC zZ@*KeStF6=G8Nei84uy(2}p@|pPJg|NwQ~ho~Skl%blDh>&`k(d7qPGdfT!PNg}j| zE$u*=urwM~I6va+b=C9rRA5_pkH!k0)ObSeAsp1o)`)7JEmA|B6ne$=y9(=PY)~rM z@+5IBFH_n$Gq$yOgFY_vWo~+Dw}9=U`Q!GpU1)h+1=~eg<0{#119ltOZWcBbC;|jK z&lVJ-qqvr_Y*mUF;>7k!9qh>R%G^l3g}AfqDpV?N+gRSsNDQ_|;>R_xJt{<8nefUNX*b2gG029U_HtnlVb^zZ z0|5&NONh#ZRhP}7(!qjQQ?5+zPFgvJgAT`V*kJ{$oi4@AG3@hj3}k>7?eTGn;v?HA zsAmyVifryG7M=Ft&AaF%GwCOs!gzBGHg1yL9fRHJ7!G@L49r*hIUSTA>x%4ELZ`1R zhp&`hD_m6X*V(L0$EQ9b^hclYq4 z_~Y{{JmZh-|HU8wi$DJ5@kbuB%{;QG<}J)S!Wm{|WYuRLCzW@5W(BagIQQf%6o};h zsFpby8xK4p`SQ}SY^{}jeW!8~)>g#{uTlTLm6FnU*+wZn=EK7E5PutY5gCz=Pvn}4`LF~CHww6up`1FN<0OkezH`JMaOrVX%$>9NgZp?E zY4BKqg%??DmqjIcY?syD1>}O7#qD!M3$?Y}J|j>W%bQInuP&R;65tQ;y8S5{z`FR$0yk!A2G z$zipXa*(;p&1#qOFo~^=D7zz<)s`J$D@RuCaAdU(t9_9B!q$$gvfX%ziQY%jw}trTxjLU3Z0Iu)|(^C$o zvW`|=G+t`*9xv@~Ctj+RyXS7XtM2uPmw0}!g<}Bzio%?b&+M&%M zT)5`fh!rGW!_-wlTW$p)YAQev>?|O}Hmi8A>D(Bx{)l28~(kZ>PM=$L* zNNIY@aX%?lv=ny|lkCxAkd?9;q$Zt|YY%T`3v-OnQiB8&Mrg6-S*SkgxC%=F)1eOE z>Qm#-pw-}Sf|h+pc&auWf3vhvL=AiMw9!Nnd-2*hlx(raUeeIQOwbCJlc+^9YD{|~ z8ATTJVxPz|CB{C{W<3h~!~pyC*q_J0ZIX|DFroc&tSY-gVNCu=i>krmjggQxVLn(U zOoOZmMu?iQI11Z|(b&$8GbVqH3@E9Pq{d_WN!wJynmQkaf|s0%Mdi^aHTwvwfA!%X zh>pNpsLSO8lAf;m39I|bC#VR*ns1c+fXYBuI|{3Z$R}pu1%gdnMphDDMSs@+ORFf} zRYg=Sms1^;7nH5n^O6M{_|FR#?-J~i!c;DCtVbsE|M=wJ;vetge-WOQDy8%L<}$yb zm(MTW#m_5`$}bdd>xIhn3r<{=e!Ppm2hHjk9`=#bTkC~hypx`jM)d%FGAVQNVvRnz zk3G@KGcDF2&0@BUb0d~#4#g_rF>Hw*tVfj)&KAWHhufAO7n38G8BEKvTQ9dfrA{=> zbz5$^)%(3|4HINkw6#veDN@5f_52F;A}P&_mD+EAvx!-$W_S%)Za*#goS)9{IzB;=MU4{F%3VP2eX>Q6?WJrg z9rn=?AoE(QHA6>MD7Z2+bzw0C(_)q&kEtNj_6BAySm(;(U6w_ntUpDQyaTxvbeUGF zl;&sRgtg1b$5hrUoc#@kmKTGC5?ooioy}5k>&q*aix){I_b2(h+zIz_0lD>kL%t>( zfhsOBUwh8YAmwrKVUrg==4Le;cv<2nx;dCEUZ`rF{QZ;_8tOm6=s%l(Z)t(-ep(|b zw%Qaq7~fp4jSOz#vMPif#^m{GDMC9~-?D45^`Gcfmp%XFD|GGD(WiTt&K;E9{8fGm zLDj*hdsjbYw=SI&{+&De`YMz@So*1Rm#!)-2sP2;7wYj7Jbp?aUj_1aRdw!y{6d!w zite5Le7mVS`S^G3sO;LwR{(!yx6U2FpV;|gp(#qf-8{4+g% zBOi+&vS{98GiT4zw7%7O%lPr*BSQykBIiFg@3BWT_oaQbC~o$`!J4Sq@v|3d#?PF) z^pQtr57y`w%z8=_6+7>#N1ljXqG-Fmx7&IItBwUysSLuFF#`3->-w=56~&OEi1p@( z^=al}KXJPS?}Zyo5gkciZF7+dNuY>RUo?1ia?DhLaOSPXvCMl9CbMz*pk|IIjGnR6KR%z&**P57Q zq$(e~nI)aJo{0V;C-w0{RZUdQoN5YD-61FS^+Gj}sAhAjuM^cmIjNr)sv4qN%&Bf7 zs&+YP2QO3y5!E_Q^)pncoV24Cs=)?RM^1Dz5mm`4J9!}*Mnu(|=r2SxP)@4yLNuI2 z)o`k2qB=-U+SvrC++5i>Rh54$ElLJpT0~^+T9CPo-DFXP9hLBBCw2do)MF_B^pem z^d`ZmM6mEnABDp4y;5UncfDzn%9y-FZ3sz4zmOLIAzNTcRjtn!w(O_AOT6gFX$f#T zgHDt2(Do+*j3HJgid|(Cwj3dmRMh_tCXz5=hET&n1@fZjqSvm^rtUaS_*pxXlO`I2 z{fWNbawOOoVl!fqgc6xljZ7-bK`P5DsliXQ7DsO>59T5nv4jGb$~0N^-znICr&OlN ze*c|<{dY=b>hs?z*MC;3RV8c;=Uuvr9v_)QBnN26xV3zKl z7$N2umX1}$;8vGVH01XHcC!r=<0mOoY%0HTh-uZh!m)h%_Kt{Dtl1lp5rFL>g?#g3 zUkvWD35$w|*rM|)em@%zP_W5yHtpNNG5@P%bHlVV+_6rtO3ulixX9$07G*o0AYca_ z_*H(^!l&tlxME1Pzg|e_gLL#i28$XCWh&a8qxmltJ=XqTDp=e>XPq9T3XBbQRBE`Z zQr+ffNxwQ_jplfP?e`{N#d)@z_eIKg;6GMAl8lPb$q^r8I-u9j-FZRe0<2_T!%(D( zNtHDGj_trMVwtB7&p$d$VZ!HVnu3MZBWZj#5MQUci+`8qRxI9@R2eM(q9`_N5~IU6 zVLEvU`;f*;e30!Sw)+sLj(fD&3bEBxOJ!sEZH+r3_On{5VzqRXO;O>#GZIAd?+Y^J zz~Auz_IFe*e>}EEP_K+)-{}Px_FWGjnF+S3?zD9B3jeqp&9#j^q9@-zGqbY}`U(eq zG5LXTDqy&<8s8L_*E82qlI;p7+hCa^PVWv7Cc^+dq~WYVZM&LFAWY7Rk-ut1$w_kd zp*5BSH2@j;=Xh8?G#rzaNy(~5$C{tE`&5owe6bz6cjL1>=`jXj0g|%2PGO22{PrL( z4l4q_JUHmT$eNBe~;ej5n$SiJrmH5_-AQ+>YX*3Dt8W@qw(AC8&nCJ6e6S% zp2cgRDPF&i7~_2QR5ohG$h?ga+QkW}dFg7iAW9d9{)Z_txKt5xbEWe+OfDXy$SaVU zEmnWrZdJ!Po_96#O5}RXrAJ-zqM@h!?OW*?k1awW`1$iAfA^@(g} zmfFt7_O>-q2u7RI7F)@p+JFq*ed6r zv^B-1+R|%J>bIw2>aX%O+IW87&|3DOP_43^O^B_tHO()ymDCnm%budD!8VmHKSIXE zcA~bJt{kxK1E&%yr>&{hN@p|apl&#wuDjc|ueJ3&VWEQ?$3T*sAM1e-~YNI@(&+nYy1<7gD>+I#Q{)fa6^ONSlhIb4Y!sJkEm6HlbR9b%>YJJ4JC}Olg}{HyDB#Z zU>J;Q=1fr8PM(~R1_~)Nckl^!GS_%8Yh9>d#*GWmilzFZETg=zjB~KO?ZPylaF)`}oMb2wRAxn<>g zkXzOkRGa56k!JD%SL5PXGSArZM=j0zG;RkF%4#bA~0&;(w1)V|y{kO{^IDnWT8u29Q@ zvC43Y#FT@D!qVbPAoX6Jm+@9+`~RR9UI@JPE79Eg=q*^36hse99u{z zAyn~b2hZ#Ag3CqO=IRZ_Tz;j~6fhD@k%vQLEQ=1SIF%aqu*q%=4!#o!szj}T@$K#%2s^cD@=YOi#59nz6^Z)QL{vWO7|A}$@ zf73kvZx;D~@dPKR zaO<5N?-1{K?(Urx$ zH-3LpL(s56%ZB_m`25Xx-;x+N@=3o%+ZL{0dhZhd_+5)rRz8r>bH$$JuRrzBvi?sW zdS=tSN9Su}D;9kA#N&?-o_qYU&5tgedD|nWqkozG^sEsv&2!A6k3QnQXYbuHuFF`1 z;dx{B=qtudh^E}` z-fA!wi z{9oC%G3C7n-tGDJo_Ah<{h>Gdzjf%%P05dL&^}-B!e`GtZW=5dU%xqN;p*EGPp|rA z&C_c~tZRPO{JGC(5nnv}<&JNwdrWxA<+>ld8-!`kwHe$-Q+IQfrg%YT6C zN4!h6)a|aR!sWx5_{zJG~ zH^7Ro5K(bis-g&GsZKgzN$pHRA=QR}`yZNqy>380$xTdIqO=HQiB7UwEZs@6Mkd)8 z+qYEe2J9r^(ef-+UWBrYB(kLSr264vO;g{p$0^pJ_QV?BGs4Fy)`<4RT2=9Wp+l@m z0DeG$zv}kn+Ho%Ls8cR&dvaA|l?-so71o|ydpg~o=9J67J-OC(D`;@arD;#Dnw}3u zI_0{pJ-IqmmVNG&OVyrSXFjYw@06>5dvfjbiJjw=E4V$mo-^E-?2xN>8?n%~BmzVm zIs9x33P_p>b%m`7l)SxG!t;@t1}t^_kORLj=}HRw7Qu|$QeM%h)1xg(w8LcG>;<0xI{ zG$!4C)|Na;Y8N`iMfvo)jbG|I=P?=gF}>kQECQY40uA5v@_Ak7OeSd&D=<%z){9Q@ zsz`hA;o0u8__vkirJepSxXUuAtt=}VYFpf8@oOtf&a^da-DT0Vm1W$@&tlzW8PQgj z;)=n|?y>~7m8It><%jOF+}2i>_XAX8-DOd?m1XtH@r~}X3~npS(35xk;Vz4+tt`Li z?!D1n7HwNuUd(&8+D(?;two7Ynj^415urRQCT$BxgtB^iV@r$sYVvGpcT+q*STPG9 z>zS$z#ESD8G08fJB7fIpRZ5H7&pnbyc#m_Q-@EjL&n4h{3Ahg<++}H&<+*?0%=4)k zzt*pLx6h|vpRL&HE-U`WZB-oqVn@!-n0Dsh@bsE7#B2;nH-_XILyC+cHHMoy8v2b@ z8Tw698zi%#-}FE+d1ym3?u%#WlMX5jz8$gteh*60W0Kk^9WeBm9%x)!V-O-CaZly>N)9I-J*rB_<$KupS2=%rjb zq1G!?An|w-?a*RQ#`lLdIp3zKa=|I}9 z56BgzDt%J6LbMcX@Z6I=C`~ktQi**FjLI~9z%hMNfkKwdDU2u;aOtWzrPNmENyut` zK21JvkkU!k-TX4idW;ny#H?3Z4OG^E0tmpQ&JvXtIwb~>#F_e}eX_(EPCg`Y9+No7 zO=2ratY#AL)CbhC(#R60dP?jenN#9p{JhmEaSlnm*C}xgmpGkOYPLQg8v-zi3z)>& zOyYcf(tcUuOeY_bIG;(J>n5>{Bo1T}7w7}*tTeL3X`T{$NamEdhM%`NCEiC8A9hM? z=Mv}Ym1cufs1L}208HY8OyWW&@j-o3ku32}Cm)iyfJvO^Cb5}GON1)*j9Z2Art6JT zwXxpRHV}1S5JpxQg^|_tZy{tEq#F!UEF~O6sm2AmnJawigE~2;Yv@u!}YNfNi3wqds6S$xB9onjZDS+CEhzgZT-%)l_i=({dL=iij=o`# z_Jg2{P^!Lf9+@}R1;~#mi;V%5M#*XnIB5(xXOMf`O&yKW9*1i&N{5+iA?+v4x9Wva z7NaogB;lOck}0hH5FZc2r@+SGreV54T8jIqRM`oE^j~y&T_W7wg^=YpU12bFH&EbV zFiq#5LP=hdNM2>w8Rk`XpTQ*H@2AtPBkND+T1V!cE~3W9!4Ya?YijlB?fQT!eE?b7 zeA2}lYij*z_5s5fgAjJwQjcvGJGtI*GM$|)U?)=?Cv(`zgY4w%=tQRRDt2KXyHFlu zq-}9Mws!avYpFXQ2gy=*J`R$l?tEl&>?HV*VzEN7izV!04dZ2Jyv$J3)@5=RzYVWc zZ@k>fq;A8flJT)HKCQ~+F8+FN;yd}snc+vyJ%y)ER0t=dP?guxr^V1SE@H& zZef(5>KA_sIJJ)(vr`;f(b+1jMQ$y51dYv{Q!YEMVb^ll{%KsRnuCk`+37-d zx~1Mjfl}G-3AQ~0165jmdMqBq$Hkjl*(5r#K;>yY7N|Y@-hx24;@jB18AgKndg z^cWWnl`Oc6hMJw(7vt3F1%d30g`LS_`#aEWT~yo1;cgCXZnC#KZl~0T+?edn5@dHa=Ca+H{6J-!`0l~s?(vLUC)eyRv?)n9%*sw|-v9U-a)p<-Y;}O?DjXEer zj2Ng<)S+)YDq3nZMx}*0WHtn&aRWLhjgyF}WBg^82Xvz6xX5*0-D>pVIv?nY@m-@X zvU_?dD_LA9bs-L@WMT@|eXnhN6g1k1ta@_~&8B8Nh}| zata#5IQXY|^7qO|?c(Ffzjf5dy}&`erOd_u01Y4wQi(pm%)-JN7Dpx>qtGqQ*!L91 z&}2W1eVb%QQ9`JJgAhihF()C6%y2jg9pqxnp0s;O}Yq5?LC z%wT~5f^=3D$7Cz>FO150fC295j+o8VSm_y>>zTE22t)S9F?&NM=U*63^Dq&@WH@3) zQ)889OwF|Jm_rycK3Vd_4B4N5VI0i8K7`4146jU$)t!#QyjtN-wi5FuO>1jy?9JX z$yiN#k1FYEisNdlxN;5)Ea@^Tw{2u8@*cuPnlneH)hnQ~g(o;+O-*my!NW~!YG$Ju z296md$Zlr+CcQD8k0LW0Elgk2aE;RJz%@QfqHC#4eayULQWvCanS4}}*_cLthwKTi z6~Q<;<`l@p4P9(+X>nQ)W58ju9-{1VT-L)Vog?cZ&n$Ra5BfjnW+WXTCzL9^X9q&Ztm-eN|Fw-}2~SS^u7O;+7qMHZ`WV9_C~?y8~+ ztFBkkagt#V$#B?enO0kDwal;0vsz}?9>8HaO0&+^oY03bLqnT6;rv0fi?xqjzJhYM z@?D4>xVD^?t;0L~f8bB$t;^Vpm+@x85|nW)$~XyS{6;S07FNcCOPXwVE!k(givABa z7u)V?HruXhE|$w`D=4b9Wfe7Abwi6Xt-2vac_?J9RoAVkk)$agX%3-~IVj|Q>=j_| z5K6kr#U4y#N{(EvYLIK7z4BAQVDP=p(Blb}K`J)%7^OD!xG#{!nR#-YX^}0uQI0c< zV9|?Rai&G?^%?FK-L<~|P1hOvapL`mct1wm*tbZZWM#AiPB8~dv;&%qYl{p*1d)#5 zq$6a~5pJYCyPF}X*wB{-E>=Td5;Q~4h>J;P8Vl_(1e_A#Fr^%0z*$B-l88sv5G+O^ zGJ^z7VF42YLIX^m*%*-i&llb!zV3UIxBCN00j(xqRvr6H!QL(EC`$G8Hx>g$jlr~F z5QWvj2GiIu^16FV*)(c#DGWzKgf^PC25p=nd9uckJR=;p2)1L+giXBUiOD0h_Wrn$ zXBr(&LAppmI+jqF0!x8!qKDg8HsGoskJQEF+h}JwqV))h){%QUyFQBKxQ{4|CkhV} zg_%TQ5m9(pgDI4~_GKqpTI@gI1}261(FpkGipf*?m*?Z~A@612rm?47xI~h0d)9?^ z1p=?#_MP?QphBpTilIS9#2=>&(g_yf(+DD{C?~Zb&QX2%Ye7dY@l?lv z2~E(?oyyn?Jevmfjbcpk1Oj8K_TenJi?80ndMfLN(i*MC2a))X+Cl7pt2UVZuhEA2 zpn8^TqwRa^(BYR*_PCPoD1Qm<)PVH?VomW23m%rxf7}tO$x$TfB9inOl62G*u*&59 z6^ge3>SW?fp(czuDMK`rVJ{;gU(S7DEq zb8V7yZDP4LFp>t-My;LM#~})l5-boc%0JC`j%o1gZ6P1^!9XNUW zvSVDtVmVjwrD_Mh-#${7X%mx)`ccjqCK4Ct?!~i7nw6MRU%Ah?*1ULc5x4Aw-1oZ+x0 zQLrXcxaLhe?yce5u~u8P(WvalsxRVHpo6O8-gIyS%Z(rGh+@x`GlP;5+lyUKu zxroXAwHK{yW;hF~WK~qis;Ib}CVH<5&ISPjYx=(!C1Bo+0tK>n81Py|(^xf&6ikx> z?U9WxD-CuU6e;_(T-mH$zS8bs{~NT~a(^k56Nj_%H)5`-)tw-wxpDy$6@RU-v2OBh zB^MxEZtZYck;73-4h5MC1(^i}ktKPml9xrHbQ90-G1AuC2WyoR-03|koa z@CMebG3?_nMH`t8(9YY;`cYrZCg6owy#7jk^KdG9D*fZl)&ndDx8IaQw%;rVssCj3 zff-S)_-fqw2D|euc2KEPUMAl}M~;-%eEA@%729xkwo&eE6^`PnFSodx9Bemx6c?R; zMljlTna?FXuGjfrjW_>0)6@Ujm5%r{uYDKn%}s_ooN<)H8Am&uah$^$Cpw&Qipv@M zI-K!+;5FXij2{kWR5!J9zH+5-$I5t^9Xw?9*ze|l_c2V=b7bRTETk&?>BHS9qM^c@^mB@2^Vjz@SdgJ&c|%V`SQQy znZYQ%oJ$+6bi@~vPxA1u9Y-MB(EIIj%!*VFXSf{BFoHWnW;1WH%f)DAh%QnL=0b;- zd$T+nt*wmD4VYo*?hh3aGniET&DU*Cdt3hTEBL}-Brth$+aiiiKo&2vrl-n z&e3x5qUGXkl8g75T)a(k@ixoF`$djdH1?iHS`Qf7tx#}3TIL*V$d-L^oi}sbo>}Zw zk&j+cbk%Y8|4$f2Z^jt4=j2+aaI=bDZQMoqhC6aZUD6mU+zYFAH^yE)$wk>lJ95Oe zYm5UgWsK$Z{z8fjJ=&Rn!PAmS2manHg!}wXKQB`-?A{OP=eL=L(F?zo5ra|0U=(LC zN@g&s6@x?v{l1=4Fr0Y0n3UaS8b&5AOHZOh>{gG2x0#>fH|#w7@;_5K-k*QDtSOiY zIcy4MTF&K9!3<|pFq1r|V9dO)_B)EJJsr_-_+LwqA@#j?h=O2fQd7pnd{8#2*HeO`@rP$m2%V92J`|~fGyv@HH{`bwl)Ldl# zl-mkDlRx7l<4<$~8%Z_iN4QzLT^UCgCG1wSeFrh;u}U>vgrUT0vGGMlzDFq^#0 zz~mj2HBr&t42-u%dgtK3YX)YNw;7mSe`N+n>1_sPqsPGKGP@e>=nre8YzZ~EoGar&k0^QC`dip7f?{O_D%k^SB91Ffc5di<3smQ9W+mW}&7r&zA7 z=e?!v$Wg6m{I7v?=N{?Fsdl@oMMST$z!{AY`ttGyBEn3sdI{@*vk}) zeZO3ccBfd(UZz+cDD-AsJf~PNTkmChrB~sF{f6{DG^sSQ9jk;A4Oqnx=68T5T*me@KANQ$ZU%6 zH3>>vh#`C(Pp9*aixRH15<=}JL>Yv**3jc|5@I0<@pK?wJ0#Y}!2c9B%@CrNgs5p( z2*;D%7z{%QH4`F`389gN7=(nuNEn8M;pow#coP!$L)v~w+RvaJv?vNG9cu>QETtTT zl)*?DhLqt*DT-_?EFqnQbTk+qBtQJjo?R3JE>zBF#%m$SD6#CNn_u0BA%9)I^mUm$Lste zSJ7%;d;}=T8a4f4-3;Ay-89|QY3}-stHSmZVgQX}1w3!L_H;buO~pfaxUi@vofnmR z#h^WlYh{}CX_U+P{Dq&^^^G0dnWcE4L@J1V$i%5lCPJ5*UF5Mj(L^NMHo8lgwNR$#A}< z#nv1!W^D_dYQxhU5)eGt5mg zpRTZ-%1^gGy5hk}iME>jT&rb8fDYj+hpT=I0 z&6eMSohLO`-Cg-+>^>FFD$uH1k)VsVecdoR5wxdovaL+GFWPEZ+>n9N7_?Lm^BZ#c zKGnwThW&h>h)ioJ<@;1K6C0|q@2|C5j14FGX{w~ihO>O1xZTx|%EUMGGtqjfQTHcX zofrDl_WM-Ukt-gYv5ARhyO5t0-hxIQ(m;him2ZlckEq}^`QsC@OC>MQe|Qsisptpu zXPU7aMAh*`G4`I+P<@1A@2POAkY}hahBWA+!Dh1Ac31u=YLW!ThW;4?+rWkxqBlQ3 z->Mtf@Fa?#e+aveq4fFX*nI?r&p(FU5Gp%WIIg4U`4`xAlsw;RwG5Pb%_yeQjEW{6 z;}dD~@Fv@kh6N6KB)VCqM*;{2~jR~j$6)U?Q4g$L%>-}6-4N5;jOG=y&N_a*?!d~wbAcXU$ zJrsCxeijojoYxpygja4j>TRmlOsd7|coaoD535MD1&&-?3P(MM^Li#J9^q0%qjqL# zSqEeqxiSGV4QYsBNe{@Ck-Fl+b*vjw>I(3GjPqZQ8Xc@fwy9dymx-1!IRPTXMDyw< zW=}?=qR33zE;Rhf>8wmZw#a;nzHkt)W&NIx@D)Fmu4R|7cG3mXla;YN5TQQ3frVeR z;|h1pj8?Zr*;nDY;_bGN7v)?<1uxod9d$u*`eoh^X}9QQ>!@x%Ilk6W1AR{WqLx#w zqlWqv`O2*_KR=IZ?>?x`%gku7CtdIvEayZF@uhx8Tj*Bo!!SP3gxDuwe5eIupV8(*sW`jX>|lq|?ZK&O#dCJT!1A%TMmFkR(AH9%s43nNVoRdX^&^&36{NKP7URiczuKSaGeoUJcV; zhy7n%=q_F+E=>Fd7p9$I@7lS9%rz*B^_FUC%5Dv(kYZw>L0Rau3D#^Mxf>52!uOlxet8tfo^0Ox+HgHh|1U&(X{H1k<@^&IOXO)Kp-uI{t*_q(VB z^Uma;us8~$SG%&tLG+U>{r_X{Ti~Op?)`VN8?r!P76~;<#Eq^RG-}oc8x(C;cgZa7 zN)QmJATLQkfdmK$tr#Grb-H&g)jnvgy|&t0TfMiowO+MKv&nA4Bjf=g0Rki>(3`M8 z5(0VfNdCXy@67DZP97v0?*0Ei|LaFKbIzQ{@A*BybLN~GyXWj}nDYImS#5 zX)d6e0pp`4cBca)6?-L(Y`(bued@VE)(Dw6Qd|Q5dYGutD-1OqEtNa~Nk0gvP)5zh zDXbTnIO-d9v6)&HBr`5}%B>rN{QNV}@N&DD52R8fz?h%Z_F7PFwu%ebjG%@2px5(Q zMHj0%2^s5LU2GRC2oyMc)FlnuUeCA+5BWIZz68+{u1d%lz&D0dghDdzfYhv}MLsve zhi9`AK74PlJ8+wXA)Zy$Rlc9DJ9wJ}B%Wo}HF$X{fGO8B=iZc%1yCE1<+cE_JUK4x zl$O~9;LSNL163T6Ze+IrV<}q{eM5TsI|>3IJ%^2pc$}aXNT~n<7F7Qika7sFW>+@> zDU|_IoYn0>N>zXqTQ$Z!NIxS8u~(*tXsf@b$qUeih$7L3SR#>yF!JPPqR3WpmB!5f zRyu(5dla-lGQ+4K2+}VnGpf3>vmu(yrU6h8yDZ(i`P|PhfxBOYxVoX|7h{*hly&YP zzZe_(UugUe!8>$?@q5nM-{6o33nV0?*!bWyjL_qONRc>QK`;xPa#NH?_e9H0f)CaXy=x^=;0;U`8bRa!%#h zT?;5$osAvoOfa`kmtVPd3M2Xhz#1DTq1KPE7RL9!zk%Nf;$eaR)6^{vZns@LiZ2Cv z%>m$|2gtPWf661+v5VFCa-SUzEr6U|JO+uhoenJy_YfSv13-EhLgL*R59J{l)f7r$ zSXs*)61?d+6deXZaV?fIQp|$|lF_U#VKa-+B=@`tfpG`ipyN`+lAfYHz+xcNvW4L; zUcScQ6)$&SWB?!x*E29WfLH-wam710s2rpS&-c5F7LayDnU4j`=-n8Dl5@j7katCc zkAh{4Rk8LWM1p^knO}n)sqjMsM()dU^bK8v|0U8KQowjn6he6;VKLD3=`i$BHS2Zg$WgnAJB zC?KK2-4&#B?5G~=c1Yp#xQl+Q8cjN<3RpG?H6JKDSDIl+Whwf&B$xdI1s`AqR9p)6 zp(GA+`;q+}L6bd<{C{K@3j(H!5`GOOOyWQZgGkC`O6!<_p^@eQxg}Z*2%Z{L^NwgoKSb@+A3gtrNS{5SL=1{pBLgkte zBG=SVxn_pSRY@oopZZ!*xX9Q0M68oLhB5>?k2%EJfLuf5xx1%48$#u&50z(Jh&+W* zc`gr?=WG4Q(?}%Rb9@N3me(YKdNJ@rhTjhrB$AmYaSN!36c%H>hJb{K%tHZbBAJH+ z(j+oZ1t4&=u9w9%i+vV z1L4fSOX18ZDV+J86waKJ!toDI2skgQ37!^!9iaFuQxD;?gxg(hA@=G7&186 zXTV1=0`CEaZV4eTnUQCNvT;y00s7QI`B+x|aVQ_A=15>wlAua56d%txN&=4bAsnSL zj_~v0sX*g6#?eE-(dZD4<}w}4#jkurpFCqJ9$1PBVQCR#X*rb5gbs%@67L2QZw;Yu zIiqi*gS|<@;rcbAAat$8_qMQFk3lW1nz2l(_&R&EN^gjOO1Cohz6Il2xe zV^oB*oZXCyJ6PE=q~hDGPf*aa35<#umQ#yVOlEz8S3jS`sQ3oUX+|o($=d0_kG(T0 z43u*-5^+E4QV&E3jEE5|$AI15!#atEZc|yeI+g?M?1QqgtZW<-@i^-;0d0d3F`nfl zArVQ8h-9=HM#MOlBOnzIv362{isg(7p5@F%D&kq6i-3wuM#XTJvmE=qoAr_j{cdFa zYFQ4X%Al-~$`bY=A&*hFKc|t_oX1GHmF4YVMGC$}eg7+aq00~6VSnCde?DS=KBcEV zz3o52_aAg++_zRuNkFrZ|F;mzZ%ehO@p=8!r07H(W&)zKid7Kw>|q+2*wje0G8lp|0UWsB8ET>RL79 z=2A7|;#@VO{*oW2!PSiNYZ=3^n$dUKe+H{2W7S1il^B7l!~j$!!jH46MD&-{^gzpZ z$lvJuTlMSI#c_X~@87F`mdEn_zTZ{h6Ad%ODk)FzhyY4z(yPAc!N{#i07mCvpI zHQQhD3F@D!VR0;mw~zZzd+Rq(KF#^c{b#*(wuf=*{-EH_Hg@mjbK7ynfKB})&?MY< z2A)5LcQ#JaxRUjs;kmQV(Pm;j(jhc!eon2?(-7Xz`2fVl{{G^B_#SwC#5GLAz8|Ih zr;+}0(%&%YuU2ti`5&k0|NZmo_si?9^AEhCvtg3P<{DvhJ)yI?p3z&{^RT6?yDVE# zKL*Rzlkg|mHhi~bD{dD|<+rC#&}bU(Fn};%N>wW_|Jd;@X=eqy*h0%8;sJc{oosO> zSX>*S<+*yBYf+@lwcOwzp?E*NYYe~F;##4zinU^tNo?WwTg`_(p77X?sqPQ&{l%tj z-?4CzRqH95VBvf=*BZT5JS?s>L4`&OUt;dSvdvT7@7=}97Gw27DC@PjW<-h|Rx!&2 zRZjAI&CO8M8rzWK{?!ClcAvS!;_*S*0-(SkHjA@Nkyh~tlF&e9Yu#Twf;=~pbK|o*k_vC9v0`P?t#3N&SCB{)~ zUn>%Uv00`BRM&n5U$?6H0uquyT`NtAz!s4Z(AQ&Xwq{HLwvd1%;A?MyFN^!*hgFPy zGv0)J%`#09TfVsxtcDnaD`U_Yfh}kPSSh%mP~_(hMMjSzu#S2d)y)Y=7fF zvYI&A&(vF<4+KOLUEmdsD5`+alF)du0y0s|R9sFHi(zDerJp201w=LlZJ4Sw1K+X@ zmm^h1$&+G`eGn9rSd`9vMpGQQ6yOgN zPe4+7v}!fmjHGzxS#iiX$VXxvAOwwD%{bDO!2By7l@9VDw&B8n7Ov)+QI*L23@RQW z`7ob=t0*vZ72AxcB+1(nf$=Dl#5bS{n!B2Dq-hFyToM_)PEs~N3z~ZmWdlixCUUze zlC?`Nf|o`#b`{r*Bpvx*qKVl$a}c;Np`rI+JD(hnyimt1U2+aU8?bVs91Z+I9PaBVBHXbF}xG`YkNqyUR zjAY}{%*IVJDop9i$c@a%W5~$kBxT2&`m%8ovnHrjPtpMpVD_#w09n9?YXzWRPiSz5fT%0l$^i-rq5{gLtV0aYizwFmm2a!hWFq#ev}^IBM1l( z3VndptjC(*0M^)QNkk{mBbnVn{dyb2=nEmjDfcO4Lw?B&5b6`H#9VtXW`AXAS$wnrM z!TdCakmUf}fk%ZHj!z*Ra&P8YF_LG+NuHHxy1rwXB*!u`$BLI7OLslTiX+D|kz*yu zj-?-fW5ttW#WBZ93_4ci^&Bffa;$jfSV^*D8LsD8QdmdsG$k;|M4@9B04 zNx@ws3NbjyM58}pl^%mkAjo4Lq?1RP7<4775jx5QLOtd=dU=pBf(tR4RD+BZ<}r7P z4305?wv$)|)fgj(c;qIAz!(#U4%36{d}Vys$l^O`jEP4_QSr=3lEXX6F`@%wOai)y znrTK=l;p<-<`FUS7?X&up=O+-JLbMo$~XJ=Dx1JK~ruJRuN2>A}`j+)6oY-FQFz@DS}wC5y=$0(C*$g7vNEVAUD2!H{V9~-nT?_McEzHxk%MBJW!6HJI2w5UziOUW4oHLx$ZBOQN zpQIPL(aS%Shagrr3$EuC$Ia|wy}UkvZ(Yx0O9LHtu?63@?x`E*sS|j$2Y<@b_-vgCt}c7%#R&$y+yj?jg?L%+A04{BkY^DR5IVU#ZMmZ#}{C1K8cLB*Oi!CxACKe|!ApkRt z<{GgQSdYQ{3&czVa15oV)}jkxxm;#Fqtc8bT-S-*w1M(r z@FXSNF6$rY^N{+H#|cadz|9580OB#fnYd{ImeGo9Q4DAWkk>3*0iSv0yqRQ)EOnKT z6@X!{yeiuOGc+^;zUc}R0dV7!Odt_k>mbgVp&3d0&|a0SeYh?%pAQ&XH)%gfGBip5 zWN4^lAi!%CcLpnaKn#)z=#bXk#5gho;R9+PIR{b7)(_G)7n0J^S^OTE_spvE zTx%?G4g5u-wvo3Eb0r9YI}8Nu*W%0)RlTxIEJ9=EoAz>-%=q@|jNMG}U3naOThm_9 zVHA*co_TOaId_@V1V{oba%X_r=roiG&{D;uM(Q#bNoKv5D+dx|kg&Y-KuifarujTD z!69Z;0+B{Sn3W!}291$#I>IwE8nJ<324r(S zAT}-yy~lh)Y-0XJK4-3x%(9Fqjz?z}u7qhvp0~}g=(-ELN5e(x5^Pr44~tjtAHyf+ylaOl6VPTQO3-#6G+rR ztR9HD7-p^%Ya~UX$(yTC#z0IR5gduc6r7iYjPfPHA{=UQu?}^=(knJGQ$CNPMQX|! z_;nRCTW$a$M@ix(d_)-|HIXKgG!Tncb1BSxUM!XLgpg%!l90WD2qvp3J`V(nCxA|} zVge@wG%c=8LW?l15J#DZ&VwSF5Q~AAc!VXOTr2?w$h<4j{Sc^ti-cH6 zF+*MnvIiRhqzb^z(?;++AhVD_r|B|jArUc1LSR51Dgl}_GL@VFGTXxtWtbj;ucX)$ z#HRl;Fef&F36Z;I7NUlVkR#w_3XVF+MkRPB+5LHB0C7qiV?uyo2EMWgB9HTkRBRkU z{enLNk)>u*Zysrfh877}^H4$32ruc0z>1Vb`US4^s4gTj4_HHWF*nXEJr6!ngZuy| zk&=c`Ig(aLAqZRG=BN&4`73v$IuN=7F+x80DNtEPfkM+B(oK}48>Dg%Du|8TlA0KF zo#CY63%H6#Z6T3+(Ck24%)2wo&V#R%B0s=MjHDk_jieJ&2f`V6KPrP6*~)TM20~h3 z?B5Ok3q*RzteOsxW=xW1kjMk59%)3Dw8S9I$4SEya1}?x0Z@4W%?`9BVZR5k-z;N) zyriI@^Hh@S)Bpzw4FB_G{68k+zhrh@dl~*827VG1_+KF7Ka8;Q03d;!Gys&T27tq) zhAC?3*Xm4&L`=F}4Ds1mCu65h4MMmr7=b+q>C}Cs8poZJ%lYj;;^;#Q0}7onV8*S* zghm)fhk#L+hol*WP){Lra5$x4nuv`?j1Qta5gVh1bRgD9V!|jrL`q%ESWPG;7MurV zMPWLT5GP4YLT&0ez85Ja(C~-Bf5_<9RZb)%qP$qxACwwR0EmVpltNGd)bSK)nBzLA zTZ>j>(Pru_T9~b9QzI=RByq}=sS{f*qM#Q`En;e+oS)@Vw+bt^QL-nzw_6Hv9Q-s@S@<@b74bhY|AvrQ)WfdCSIvrI6_e?&wrQ~Ai$};C6-2HuV*6Q zCAN5y|L{PBJW0e1m>%-)?V*_=f+YXpo(S`KZ~Y3sQO_oWKytCUoPVzxNj|3Inqy0S zw#WR{3iB)MPh#6-jHc@%CY#P9o6I+Ge`o;VOJ)8Fd-TN7DK=*MS&H#LSV0s5k z0^MF>RGRmBJ$9IPF&C1^EM#eG1hjdOa4L+!1@5llMF$rgVHsuCHLNwzd~DzrHTGcGeeQIW*lQVUGvo%?$C{S6Li~Se_O_7Be6rp$NVm@Dw zfJW0aCPqlo6b+$I7AGK^JUe3!y0$_xkf3CJM__7(b2M3~C{S6BfNeN6ljRr`o_Fay zzCe!v#U^J=f{><(8Dg0%$r-^p8S}Wcha{5-$kbO36Ed8c$>Ky&%3=ht!U-8tgyE!Q znKT*mE}g-9^@uj|lnjZ3bUViQ2uw_*40q%>--L)GSp#AT5-Ohe?ZF9HjU?zoY2pHt zFhU*hJD?Q9e#v4(CSXV^2KbW662NQ-LZ1icUZs*APDtFq{EI+F;sauS$&^FpUPL`a_mV9W6a+y=nu8%52r?Me zBg8OGFp!baL8ky2gbbz@f|yX!0R$XW0RaxF zfdD6M&n1hqnN6RH|E*CMcy7yyk(mLD?DVsZ}^xdMo!`4%#OK!mbKAY$?-3<@a0 zq(GpcJ_vzAmBMCNgg+7opbW_+LS|Pgf2c%n2p|*9vLuu~$!1s#nIvEdPOc=B4j@wy zrU{u~iq#uTBQi-dt5BGRxfO+80VrXXA_Iy%w+h7)d3F_w!Sehn6n<%j^){XD9(ZKQ z!heb{8NN_#83Y-ME#>(@=+Gz62tu)0o)d(^GMg1p1j|H5(Q8m{EP9pa2`a%!Glft9 zmgfpRX6F{2^3+@(dyrW#u^p^so8Oa%$%T}*jxAK9Ewds#*Lcjm#zU{NWt5klw;ZB&Lw&&ySYCiYLC_JKHt2jH4=O1#;E)v{PKf@`m zAijbx3&Z!PiL*?pxS4vcE5fx<=bEl}&55+SQVdqt^eC%qPPEmvFvcHF&s)VB!F~Td zV>B9bsnvD2P4rpBLJMDP@$4UM6H9GopT$!$+-9z_iYAlATxg3ef?MG>?il33t&vc| z#$C0DMON)H4HU4LYoNd|i&$-q?S=wjHtwv&e8tMu*|>Vay-H`}TCJ`$okcukgXuQw z$to6E%!jZiE07_+VA9#lCt|BuPZf}dJz)v%s#UDPp5VT@h;@{233OyLU$$~5ZCoAe z=mK;Emb%|2R!SXN#43wE+!+r*ys15Y-wiuF`!V?5c!BTAk+ ztz5m0JIT6gfv&(Ij@!f%sVe{(^KtA6W%q(9(q=A+tzrFmndtC7GE~aCDY0TUc2lP0 zsKd(D+PGTiCd$S&LpR`mK3hOZ7O}>{lw@I|GuS{0OiLE7!XnGgM;sNgt_p3Ky?>Zh z+#BS`d>J%kWwNu9?4oSqaZ5mUVv$W^s0RCqvY0Eaj3WzIj1od)K#rK)@HSJCRrCZI zvYIYmChk@;)B)r;^zaN=~%#Y&w_oKKW^rQIx{3w3N{3xM+KS~%nKT7P+j}nK> zkCOWLqog78qbdFS(Uh<6M}ECLURqta{nYh}3Aj58bw+}V*~Bi3r!>rFK5g-w)k0>3 z&D?JBoF4(XV{B$ncDq#z#&6Yv1BlG83UAzqh9v5N!YLAk;F%V&E0jPhBaqQ&W%R}2 z4-SMDG^rCeY#>DXVECV63eb09fD(+y9~wcC!pa*sf+CFukfAO>W6OYKn1DYx@RAHm zZqUGs)J35Y>I38r)Cd#thlXq8lM4fqU=se&NRLKed80;p7w&e( zuQ~+CYZ;&fr{E992;`L|H!em15{*D%RDi;PxG|y(gbs*^*w=|07FoD(A1ja-J!l-% z;g7LDARMZg1c{6T8e;~DgnEMO{*iDfLK&nl9))NeG!#Y>V)qY)LlXockqJPfX^>E8 zAjIwu3Wp+gLh2GxgmHt0!YBgl{-JPaqAs8@2}q0|BosyyVD|@wLlUgHa0fPie{0ZC z7(;;FKNJp0nCFT&0fC8wLP7u$g6saFa0r6KEt|C(KDYi7?k9i6lng+KpGVb3Xcy|V z)Aiap;HoL$4Aa5h=735U#t4sQnNr0FaiLC}t{36H2=_&}FT#CsVGO^m?L6C@KDXyv z`a?FTCu?>-_`@fDZ2i`n(kpp?`(5?l8rA9N@}HgDGNxtTd)vPK#-Zn9K2WE3{xRY( z_gv0RyK>E@g3Xb$)#0L*I(Ni>-jftN{(-Bj;+-(9mr}E#Ov9d^=zNO)fa?`M1ROU_^eKGT$7Y*w4H$3m%qF?!~367VA zd;a#tM~Uk6Un65CPy6HHr>F1z)pM;G4=h)w&u{xtR>sobZH(Hud%HfY=wH<7%iX(n z_*bUB*Zywjt_vI26sptLTR+rrVOst05hJ^$|J3Qe^5yhQ!Sc|ghrj-cW1sJr8pX7AK@=UP!t2L7g3Rtkx&Un;Tjl2;OZg)cq|aTwnB@6 z5$88U_gx63VOH)a zxUKQDB^E}uLs)#IRZHiVYp?>9BvRVVMKE4aQH*C|yQ$*Zsu4Es3?Mp?;$@_ir}675 z$Jn^b6qw*9*)BFHz<`H*4afUo7=oC+(#>!RQb06?X;MaC_al8kov?861Nk zPGZ-{v=zIrv~i%A%#Kus*w8h80Hxi+bwa=)kygTlfCPyUvJ|rpQ3%H^Oc6HR;fe&s z!oYOgid#0YkQL(;Ne!9ZdWb;)?9nnb68li88;C2haSaff1~rjq<2oSKOfdx-N}@@! zg*uxExyRtDgOT8~Fg@6qEu7R@V!ak-3s-cw5x|B4r>p=Na7(SIflyn3%2w`-6>YE! zBa<)-caV&MyKF--WOiJ(aFqdDu%a3Qwt#Vh`B=afEVQTjv@I5*?c-z%xYqy`0mZQ# zvjhkyAZ}2Y0@VX~Te(IXnL;rmWG_kH%5~x@5LY6bf>ngz;#j~GFjfef0)+|{WD0Nt zw=r07GbdPrg;@f`2@n!UmQaep0)&X1A8K@_QUD1ZpZ zY-onZSidJ2HL(yS@25@cPzEEJV*x8b1&|RKl}3Qk1J__tY9P-_M$m$bu4DvRFyCfY zP#-{q&R!9rPJsv=fC%$p9yB;esOl9GI=?O?n7#reR175~#PtUWRZ2*>Jk*d7-!CL2 z_XP=6A&_t(00}T=NY+prz=X@aVnV$F6FLzS5(bM2HN9cN<*$heiC+N|jtwCuB=v_0 zHA+mlGNhO=r5{XK)E6eygkVCe0uvIpe3?TVZgcmvcXu-}AUN~@1_lYC;Q_+|N$s-x zORmqnUUGQm>!E%Qp32-?9_%HrCXWt!Gjn9gi39!{>buNsgD$HWzw+G~yp6SrKGws(m}iBKv&^3Yq)Kiixe$4e2blQ>O+Ppi_AvN>u?Os&Y+N#LX4yTgIylG$jG62> zK&528LCa-UI{*_68Lx)UMol-uM&oXXjmF;y8%?+YHkxcM=L?WGfI z*j_htDg0HajT3Q8~mEkhhaSV08f-fRXK)IIKZ9?4*Xx`*5U3}6S; zg%~6_$WdLSfl$QfSywfNNMKk3Gzu} zFd-5EK%l8Y#b^jTb)+)5q3($;20GL~Q4)Y2>KJY^8SJC310aMqB_a@su;Uhq3dAGq zSRw*J36d~p4H-R&00vD#-X(03x(tF7m_ninPu85&Nu454dFwa9998NS@hYf&hFV}~ zrNPmav~MB(v>|?x_VF2wLi+(ELs2pH6P}Uw8K?!c&yX#seTHuV?FT@Pq#PKH1G;B$ z7ovQq0wfnR;0q`}fPRt&N+btpAsvXvB=bK+%C|_$2dIOl6vH+!)S1d~F{pcnjv=~d z7>P|wdH|rLj%BnAXr4i4Q1c8psaHk;q9-KR87mglUwAOSB>Ob%q@m+f9*^9plU znEF>8-3DgTfvN0`uxZncv+1}SXwzS1EE%9pC)_ZbPP}0@ophsYdhp0SWb@v)<#{hm zQ?E1i4V)j@Bo6TW$c~$Pd)~YCEPh@Arp%RW%KU|C3q2S9*z_9x{fTJ@{oQWDFLVg5 z2*I^bFSw=~1eh~BTq#C}YkHi+H7CL0TA0*hzAVszlAn#iFTIyKTz5P1^X!FozS!>B zf3s68b((#4Pf57bT;&i=CcC-N8CwLmbWZLVW|++r+N+16NM5igi`#WL!DLBTBA19bCPWwg~}OEzni8T|Dj-OQfy@ zvD9upj$H{TzZXo=PIHM6Tf@5Y3QT}l1i30@U6nX68@nn~a@FDBYMr#92Doa5u8elk z=M1RIF4owYs_aaFMklC(=?cGqCkxO=Totjd3Z0m}f3!o~8{|qbUj{`vNPv0=2{6tn z9=8VsD2PQ)nX4M?D$Z`MbTF>$T(N+XLbE`w1SUW%z;xvhJwdJ<=1x$Qg9I4uAOR*g z#Y6Uh0G(oulbMB`=_))iP_DeG#tlB*6-l!NKYLApxr&#hiC zB@dxr3H|$(Fob@U+P`0=4xwMo?ccBF4xwKy>ff&x4WVBx@87SM520UW_U~7jL+Do< z`}eDjL*iHdVQF+8(t(FL95B2CgJ+>i1UQ^i?6P}G?{%6_+dXF^?B)BN=61X1{4mHI z>okMT+wr@)fkQC-pCOhx2vNZqXQ0eVPJC1#2*R;(EclLWGH|e!cmUK0F5mJ zljBDG!5D&kwB$y{5I|!x$}uiL=VCD8{70qN9O8ynl2YnrH`UoQrZy9xOaY6O8u{k3$nCA&rYrj>2H!(MUMnKRgai ze1$YFM>(br79QgW$NPuJAqvcZ#!Qss+`+|bN z2i!OX++;eK;T%x&!X)9*kEKib2IAn-wyqY3(wLl z`_+dp<9Ql&dVSb^^YBbf&Z&P}jpu6A>540sFY&C+-)4@g=Bdfxxb%Tw@Nk2?L& z|2UU|XMG->v@HtH`>4||Xj>e3=7&po;tM?YqfY<$=N)Bu7ARrYtYLT_NS%&;rl}Io z1R4IYY&Mq)z|B|9TCcC3;_Y z>m;5hQm2=GXO9oh6z%(=r5?`}snhG~m%fZ=jo$tAFO_)SNS(g=m!<`H=4jD_H$REz zj@0S3A2k04&mx_j{KBnx9!Z^!o3QjwJd@P;<+c-eE=ippx2fhvd1Jic#Nh0k&CtIMwbYGU?0b^7e(%11uP*thWJxhd2BWl8+48zqTpkHhnkSzb3fv-+Y-arEP59 z{7sqywwEn`jb=m<<;c~ty z!Dp#hum~=#5ci9PR5BIDS}L~)hFU5Gk%iAcDpY=3efS+?_AWyw|KT|p&71LUn#W)s zUkn*-;a<xFw>k9l1)gfB5Bm|S=*u}%3< z8mkY(>IWiVq5(mi*D|@ugVjevSb|krB5;<%`SC_R#9LS+#`l|bRl^+IQ9E~qD&UGK zhi_hP*Aj(J5nn}Fk>?b>M4<1_>#D}Wtifq6#-5K7eK2vro-f>MkA-0K5Z*ZA!1oKn z!~%ORhd`&85r}nQk^uo6l_L7=NFVZQKH=offQDvofl!Ocb8zKO?lMgpDnYDKCic2T ziL?v&y3gZ!yI7$R9!?j`Uax0O#XgC+N+vveY%$!Eh3CM9i7H8YUWN3WD7}-tNDrqT zTq8-(5tLpTNN=B=>y)H-+{xivpqpjs;ma~BcH1TOg7nJ9;#>i=tB_uaEIkM@k4w_C z!h}gIWzw_D>cu)RwSc%yB|T&oLRGE-ta|o4FaedMm*?a{5zNPe;>$p+U*P~en|&(7_2aGkhV3_2wcVDI0RXludurPhG!D#eHM2eiJ$ z71a`TC9>;Qz&%-ffFdw~ki@67KA226`taQnN5FThWbqvU@f{H)-=&EH`tAi;e62{x zVW;FDAie`2K5&l^@tu&x2NUl7lK8BqBq8X#3i08S(H_{`2|T|1ZO z;5zVWk7Rub?9IVJVGz)V z%rfYkS20`D2lUM{C9{_s2eEGjf20rWn^EEKANx{=1p6v_#lDLQ?3+>1puoP00b}3X zL0}&;%dl@o#cX4r*q6%Qyd1>7#r)Aeu+Lu6+CTO!8WQX~)GPK~QedCGqFI4`hX#y& z%Ljpdz^sIQ7~%AZeT&%Jp@Y~rkFV|n`z#e#`p3S^A;G@NUa_xDfqj;W4h8mA4jB73 z4g&jtSqb|r6|)!hiG7({zWf2(D%p0ayIY1p8T=UR$%x0WPKxhjd}GKa%Z0(2jARVN z7=FpnC8HJtEE%g9O3AUA3{4D`WS$rn^#(&S^vIyY;6_FohB0zvB?AkC6q#;@BaQ&w zh#m|$WXzD^f+2+r5)2;Xs7S^H1_wd95d?bkf7$0{Z)bikyEt=hIi!(2n)z~&ZggNr zK)dL+%wuI|mHm{tsO+80FXeDVc1GrhGTqGkdUHJ4*}V&OXcDZrEdnAL<|9ZY+Y&v1;*!y9Mt+B6n#Rbu)e^n>53 zJ4F4+ePH64J*NN@how992|`W-2N?y7e)#bt@u74#VE$43rXi+|U<%(raU4^%m^wg| zQO{DzGU`o|LDLaFH@>^O8*eUo2Y;XVNaBiBD`3nd6JJPKYXQhtCD@~qVC`Z7kw=Z& zJ@qR27Cjw;BZ46B z4q_f}8Ui=vmCw3kuH-_+7tu!C|e!1ZoE%?5w+_%!^6G9wp~pyCXCT}QW$ z<+FsIC7K@lQWx;75N~IOMy|w6^(Zqhv@B*~JsqD(jcQ`K9%P)@Qok$Cu*6{9|Z9tlHJv+WljzqW*n0)nt~UN?=t@e#oVIc zxj4}6$w0Ric=eni$S_<9cegnYWE8SK9$Ma0s}tO_!VI5rSWj%W zls8y%YK@khW@E!cz1ByMZun1Z|E5lSt6TOkDiL}gv9sZs%e~zfey~@_X*8_OM7&85 zCU!Nvh+XG&YO)gXeQ^vinhG2K+JP{W(F{%K9v9r%2CTNO*8fmXdeC+n>N~M~Qz5=t zCbSMhTXq6U4EJxk$8AD4vhf+u)Z*`V=!?N4@aM$@{9QsHH~nrZ{;pjFE;XqCws87U z{IKxRVfeI28=>10F_IrPLdOl)hlfRu9;M|~|7kMs{m#yYzvSyT-to+i+RyiDoUfQS zCvc9f#n(Q%`sXeCHIJw6`_aj_{%pN;Z`A7V*8EO$XWiv-710kj9eH4sr?~x5jcLuu z!Z*KLxBtPNU(9a&&*wE0PmV}_&A(#Wo!dUxz4SNvnxZc=ZeDeuY~%Z3zVH6ykK;5i z{(a%^vQCv=o^+q3e8+^>xiGCJd{~5T_=u7EQKR`tNGEdHz^OR*V58rEu*;o{-Yr;qHy5Xa>`cYvKk=(G6n&Dw1b)$yydTxX^ zGJJG|X4HrX9XEO;h?9>TK1}<``M3Y{*^hH>-@5q~r+(+Xna`>I(=7U=a-#iTyYfcLAe)m+wt)06TeCM;S+cYcA{;a?kC1W3-39pE~^)j$h`^Z1~=>>ES;! z|Mmw_e>J|?`TTy(?~lB?J7dfzSD(9UTjHDNrfI$zbnq}dO{+Uz%{hze__~P1|U%p9G`Tbwc)W+oQar|O>>g|X2YV1{K z#hV|^TlTvvfBAL%^J_J~{qsL-isx*eY`yojv2U%uq}f+!YQJ1J zPwjnn+0v|xwZ_zCYget$TAZ~aW3|y@o2htS`ryo&Gi~3DH#%3mw9F_hUbSTX+H_;w z%&f($q09@$*{fGTt!F9Fr?v4)c!@ z-0zt%Th3|%MESu(8qGA(TC8`tf4fDBO~Y7N__TXg@%#3iIvAoe&tZ^o68@E^@m(;$ zZSbYD!Kn)}HtdHA*$`vV$k+wr!-hkc^TRDHxPXzV3r5QgIA&zOv4dq+$PHoUFW+IA zAq{T(h-HShnb%vJWxcgo+gqE>y|t;~dTG!r;E%IrFt9%wL`a-@b**{7$g zt)^uB<1nS-ADC`ai;JYYFpW*){V=8RpYKtTBv8OsP30M5#I64*^l;?x*p>i3W-{1^(3nr%NNahB?h`PII?I^a<_t zb}?UYad!SsXYArZq5ZU-gY;CrkaIwTi$Csc&L5{3U+tW4noHM(x1HQ^Cscu}r^S_5 zo1ru|#}&Z+PPz|u?cxDw(>b|xeIeA8YDra!o9U>~LBV`Lk=Zd#j4F1zH*kI(zCU>y zcp{xEB$ttbU=WCvsnVfTB`J}wegfj*>S^HFeuxCCpLqu^8sXwQaL*5cso&4)=dt<< z?UB-;eje5@lk3;Y^_!LTbtYC_p)FDpUmb?E4RURxTstAO_heQ*l~rHFs%K*F$yh&C zuD?jGpBbpn`ypb@^rCXTvT{MGOuwX7hbziNx(8~?P8^_gNlZmJ@oCIT*7Y~1)vMoDY$ed!JTMwilwUk z9Mi_@TBeD*_GvC_n;xs#yGz%f5poV|grWj`F|uGTK=T$$r@11bRMbTw_J|Oh|FoER zQ5e1#I%}s|IZfR97_I=|tn&60% zZt4k4>44mdp91X6Z9t0ikEIia9K_#HfXu?WCtYX+FxGd7xhy$Cz6=4wr z!6{ecbZ;^WZo82ccpMIT7Nic0`23V3Rvd#gAt&evo(9$WcZcx(Ep`rt( zxQq;z|7rlR1EYYlWqoMN9sP3nxXfWFclMkjBcD4NQMYe4V7M6+7o&S}G0h!0TrhW2 z3spp|KSCa_+&^Qs`oEz;AE5zR|8!T>Xvx&oK>?_8I{3o%4Bn}IexQbb$Ui)-r+u?V zo;6(8&Ozz5JuC#e@7LbZtkm}I)PAF1vq$S15T8BFA5Ab||F?wLf1hls{2-J|5k}h; zeKI<_C!Ggq_98- zFt*QFCtexrBwq&~hd%ysihBw7ttKPILEs;&OYx^X&CahridS_FxwvAPJRE+$hgT2d z)f&8t$E$R_x&yBk;8pZ+=4IvhQzwY+f(zPpxS@xJV{Pn(EdvU&P~htL%Kf4dhbmB) z!9tLxBh_fDSzznpj!(=+*{0E!S zprCYzz5T!JqL+#u65LJ=m6WPdEYH>9A83W|@WV(MGa74v{bGpbZYqAgl=5X zq5j;f42sF9Lwpc=7Q{LF9`C2Qj?iHa1+r$$QA3!7{zIg%Wzx6HnsAy=1vEjduu?BP z*wI5^L}Q)A5w_leDpTTXFZyw4<9Ym>etkuumY6AEzd!>IfxO+&KM_ox>$j&%9rq?k zA;)Lnb9Z&2<86=yI8~P4^Rmq<6&9e@A2GpVJ}H=9=2vJALi- zNFq!X9TWHCteLB2!(NIWZW|n8b~Gxy83;Bd5jZZ?Pj{Q1wR_Hd`)POlJiEvLZRhYq z4%ZxmQ#>KG*V@}_piH!Vj2G*5IK+GGV-5k_KI;%O^v+wlSw)>x5inqmQCTrg7z0C) z?g2quXt0kdLj!e)DbY|-aAzm`qx+hF^-fc>vgwW>P3g5kq8fT5>Skp{eUPSRpb5JI znmPjYt}6xoSCfME0fQt4g$O`fFxHL&7e+eL2gLK#>$A*iK({D#;1d}{7(WC9r`;Vn zahiJ;SgzA{90lkh#!1{gj9@#i%66h2LG-R^Kr*P|GadxtCC=x}AkIb1I%I9%CK zDGWJCw}UV7f0_?tfy<`(Ob@mF7#~4OZ*K<^F0<#9v4F_~c@jYF7R33*3Al|Ur#ui% z*`nFaavEABUy}u6h)_p_IwI5&p$@v`=}0Y4Om{810sWwy~5sp$lhLQZ?D3Qb+JBs zEFHCPKk68Jr(^6(z|}h7>U%)pJ%al=BLkZQKqmuoceYXJ_N;%Gg?=H+dD4@itef%C zb}#W)ddLAh#Zq2>5$?Up2>0>-;rP}m?h_ys#f?pQeuFyPZsjvZ<0shgJ8aJz)1Kh7 zb$Yl=OTbI3-XOR?Nv8RAww~p{tTj8;z^^yXn0y@b;1`dDsYUikzcZ$%S?XwT{bY+=I;+ zuo=Y)E;KW^_1mW4`mNyphrEE5FSzHMmctnDt^0f1Fz=Fg+tTzfo_m*EX9K)Q& z^x8BSO=i~8t;2NdG~U`2jjwPKpJNTyw&L$KoP|2XrOo~k3OJ!V-ntLkhNWTcN~C=Q z>*}z#?ya_|2Fm}ZcgdSTeR?|3tRxjl$03cnuEMS-E6A)NTJ#F){)=*8*qeEnkp@K8 z(w!y+-Ah^P4NAISY8JXn`P`>4PH|aHdi$6r!JRmaA{mt77!c>W4*zHsez@N>bQ3tA z!UsCILop)-t#W$MDz`}v!wmC>lJd#c-jI~vNS44HLte|wNI}D#9yH8t$(+J4f0(Ll zXOXg<%+PjlL)i47VQ$;PDGc+6JCx15qip6Q6*-7^=#d%Zk=5Jsq;@i?oib%RwaRvy zL)uX+(S{hfcpuof*63aG_qGXOyJ+i6Twvy;qbF<0N=|r}1no^SD|7`kE&up_;veQ2RK8X1 zA7fjUUP3(UTNTbCyUJ1~5N6XaF;N5@M`He~x({<5bfO@`%*{hwDBwQvO82?%QF0x8 z&>rm^(}nINxse0I=De#osL$xIiwzE!2{abZ2_m506s&2+Kmk}zWAgfag6kVFNphO2o#skzwpXMqCdv5`tt6E_3S>~0MzFXzs>jp=*7s_H#5tx<)4kKsTN3n^?9KmcTcN`>HHoR& zoBu&u4P%0-*qi@uTb@I_z(mP33?`q{=JV#e0QR1EtF6HyUdETvvvGwA)+;1>E9!n;Jep-P@Y{!n;bXlcLXoEf7u3ZjhYL&?OH3E&&zw~Hvv96sZ2BU{*o@_l zu`g$Wfov4q4@kBx+yZd+Y#ur?J(M?^^ZZf0&rc}GvuTKY2GSnPaWeDEYh@CEoY~AG zb_d6>1<~p;>~}^vCQ}S!tKt+`e;#wzCkSFp>E8nZTm#)|$6H@A^P5$oH2*7feI;G! z_VBrOd8LO52-kWHS#yN@=9{9^kZCF#!35U#dJHZYf|ShFB%{WQfUUjB2gh(~8JqJ7 zE|{9p<$k~o_P`;c*F~YprTq51D*(G9u3Osmw8rC_!Pa?P^VnLCYcX5zajjr$KCU&i z?&EN6N)p7<#x1PAfMAM?M5$1DE>v6um6k(=Oa@c<=-IVUPfI<3ktE7$hbtZW%K*o` z(sq=xPNgBdr}=`_4$wU>Xk;`rn4Z9zr_AHf0T7yk ztI54}G;4G;jpeR6$*OJys0E|U!nhv&)IWd(YGld+9i^v2XLF$%Sk6My-i}5i2F7$> znNQ1WxIRh?!LxAmaE!_m*ziG<6%blreObM(Y!}M}{!a%(mzQx!#I-ncn`Co-eeeEL zAp1|f>^}vv|J2L=vrqP)df9&t%KlT&{6`SWwpn=G8p2p>@y5QKO6^OV zf%c;cH|nD6uJm-~NXh6uVzab}oK4#tXW=^}BNXl>`4I!2K3xg%2Bd?o6#Bla|0#Uw zMHH@|6mOAhOY%Sz0t%T}NG1*mQS|FSaMRYP#9}04>ts)+e!*;(k|@nUV9!xSs4zNhY}Ib2<4HDEFup#+Ys4*ShJ98?7JP zAtDs_2t9D#y@Gg1nLbG?tmJ1#B_QQ^` z_W->hv6q4VY#`?jTqt&kFY2AQz~H$Si2N=9#51&Z{Cl==yd$SelXa`$j@pZFRdZ+S z>DkU3IE%|U&#gZr9gkAZk0GqxdRC9d>keOyLubyZ<}`kFV_@;j;r=BJiSF&Dko7a= z_Q;ThDLy-SOUR-WpDpYNS&-thQ{M?$jN-HBexzL!fp zCKA+SB&*vUa*F%fXjTxHNXi8~kaAdqel$l_iW($Tl^pC4s7wqUHZZO{1*qepFi^J?B@^YPqI(2 zA@)>CL;KoRMD%WXwO{2x7+Ksj@s(o3S24T3_vJ!~;Qq^O9VddVYn%6*ny&8jYH(Q;v`vTage zkLA~AMY3KsCxMNWMC>$i%2hNh+26p~`q)2v9d+S5cLuCCZikA=r?s=LekE?7*3Q5B z6*Ty?mihBwHHqB~Ql&TbRO#2}mat&Jiq^&#hbkQLX#@sgn$;FJZT!lX=xO5ocEdDrWqb6r z@zxf&F~2QgnrmfS(zNljTH=6O9pc>=@`)xRYlF2WS4S|UCge3y2nsfwgK__q24E2~ z&mwW=Jo+9ygi4XRRz4#t0s;nzqN?#_Ci8W;;;7R>n6MY_&5AIz^4XDC;utHDZN(D) zqmVs60%oAeMkMR6LvTJSWc}GD?!_f&yQeOk4nxg%z`XY~LgK_$#rpF|T7O>Y%=6#A zgVvXMTwjLt;1-v4{;_F02DJURJh;oa5#7jf7U) zh`5*SW`PNbe@cJr;|-5ifH|=ZFb-v{!1tr>bhuuE>B>5ou6!?=cCUoGYHVt1_K@So6jC)$~AuZMi ztv^|dKh3EBv9-_%pTU@#*bVx$QVpgCx_UQQ$D2&^-b&T*|iLE2yD{w_>q zZ^FmRiMJHat^lt#ILBP19kSBnvuIr4rsvK09Ie;K%ebD<@AI_Z7-%K& z6;s0CpD^qHAAi2AqmOgoLoz4V<-Dbv9xadUW&f4nx|ycg6AK;UqogHaqKBT54&o81 z^5N(?w!BTMl^=#iE0n1qTz+yIc$7NujRXB^6y*`3Rb~UA)-gSvFa5vqd{oI^ZNKV| zSE*mP?>^@k*#d+|Uojc@Ts=ON1XU-4s&R8Jc$lYwPO2Dt*`Gh&;aUp#`4ZsgItf2- zag1LK@VNrua}8VK6>oQl=>VS@K>EuPeC}SKZ_laGNIg{{^p~pOliM*c{2VA|3OQ$t z>pv94u>*n4zNhAc{<=U=UbA7qohjfn^{igVJ&^Tn!4)Rt=4E|La0vz>cTd&>t|#=6 zo^H;|ZF_Z`ctH@m1hJ;OJU8#v2w`cNz|Bk&a=iNI7O?GPf_v2Db)z+wPv3*aKVJtw z=C*%~!GZ6MWKQ#G7cNGZLR0rDljythk7RUyAhO|)PlRcJ zLa>6YC4zgS9v_hxCF4^VjX6)q`L7OPVrPyg+0&?P6+RHeTZ^8EM9~&K5yKIFJeh>- z0!zaoZtO0Di;>8HARY!Db^veY>W1$_AO1!*9*kPC7U_HnD*weI5eY>KzRhX;g*#>o zx%j}`G7E`Z|EPbuv`#O$v|xvELeZ0Pl!R_Z3$Em3ya-1DmcdWM1n9?^dn)S|hgdJT zzZcW+@6}g`pv9gcWhWrre_+ora|y2k=`*wbHnr7v!9^o^l@f2^kO#m7|iHD+MBpQ`D5 zV$k#%`lnCr$7?cr_2a!$RP@<1diCSqPU)S#+yiJo&q9PZ%Pv;T%wiUGh2M^nu#1;0 zyP!@s*pq$hRlLGa#`9AvkcqWmQ3kwN;d)9BygcL(&y#Hd6H|qq0{3JR%n|Jqz3cO2 zzp*T^Jinkoxj$pnFIM+T?}bM>P1=s<%z1JpSq>{!_bP{cDDN+u9dHIgY`2RS8(g}J z-9m14*5iU}B+C0?!DT1G-|w2K2Y`JVfQw{5)*=2=jiizruzp)s_1f>WM`gK|Gx_gX z)hj=D1j_wE89#wd+&3GILOhL}-4Ny>Uk zYWq&P?OO%6vjY$YBnftI>(~>dANu*j%3k{kdj$1u&f8O<9uKAdMCa8kqtumA?h2w$ z=hg1AJ@WhYu)f5PrT2b*IXS@39vENxmp|8!{7=5rYx|E(R?7c6?T7W#{?^{xKbjPx z-{CJCpy>rN|E?4RiY&HSvRf&gQ_C?u2hw4xv|g3gN%-$diBh&dyS*|t&?An2Inxcp zDVY{Xk5Hro?eS83Iq8PxqLe7I1uCYVPefpOmnKNR;F=ycTdI;%+YEQb6oYGqVL?tE zKfDaDEm2}hlthnY?~^1$;X&R4UJPUQ4sxCNBFgmvP9@iE-{Qy{RP^+(zY(NANN9#I69Agu{%_L!qUpDI0XqNnS}A?J-7jo9H2M*U*5a@PYT617jE4M+8?Bo zLegjt9cU5|rgi#9$nlX_F1Thugm7T}ML}E>7h0b1G6kZi6pEfg{Il2iE-M1d13vb3 zKPGv5Th<+EJ0!EzO-{&46kORrK=*n-1K}-_{nP#2g!NI1`~DLFx;qvqm-A*%uH;iL zD1*SP=dA9u?!ft}oU7eg&%o%7t$s%yuV=8aZ|&z4g1aXz<_sEn;2sBKpC#ukhvQ#P z2bc9F$%V$@1qHcn>&LOQDh>vHm8bPOKHi%M<8q!7|I>OMAH%TR&e=GhaX)tH0_}TU z(D+IMr%{gwmN{~`(mfN#h9u&C{ilLB`yD}ae#CE2!N>S+Tb|`hB9c(n@&$!4qV0V_ zRKy>4SA>0Q7mI7^Ag-wd@$lP+VFRw@Xi9WKBQ}WAY?z3IJGlq=*AsEWfOx_oc6XQM zbZGe3mtw|FoB5ob1B2uW>=O&u|Cay6mS^QZS>_S*bLwv2a`|?C`*~=~eEa09{Ll0B`F!pyxFH2x{0~xSYmE)KOKNy@QWzct{FJ6#9KYQW z4ybrVFCOJ{jzODk1v)-AHynw~x(Uj?-^K=<(ZQwL$jb-s^|3;LZT+-t~VX`v7~-@nRH(5u9Zq z_=hd`9Nqapm+z&roP zAQa)MiqUsPFPiKglcAi-za*6KQK}M3k^fVTw_}YTCcMA+T<@Qpk3^z)wX90s4viNcI57v*n6cZXgI+@Hpmtl|>G z#pCl*iT~3y@A>MiIOkaXg+^~H@A&}BlaHt~vapEFhN^zAG&h-pcJscOrF{AA=T%ul z`;69%uL2V}!Lmt^8w$g~CW@>THW-tFDWe0KKVsE3xmj>mr&%k;!R<{7OtW{=8BkxZ zgFf&7ZOSpZ8PUr6TX6$?6`ldmvQ`XWQ&a{YRX&n1$IFgLlI+vLseDx>HhdA+4dkVl$le;+=68fONZ)h~5lW?zl^D9-;q`1~HE zmqhxy4W1*M1pjd5fH@62f)!dTuEF;9pdiy8pWTy^HqM7{BX7X-*HS-k1??of61u!5 z4Yyv7H78>pQ=jsmnKmQ5E#}k&YV6+&O4()MM4Tnq*bW}<7tS1Nz`JHpvtWb{Jw zwvy`5#CwO4KN4AA7V(YnJEx*i=V7@jj7!dv|K76jBlODjGIVHMI5c^rR-a)bCAd$e z!8kcVnY5P)fr%ZWyf*wf(G;7(9QZI?dWI&=csEYUuW`by2@`(LFNs<&KSeL?y{jjq z7e+eT%#DPXDp8(<|3sy`miv3kMxfQxK&q$1^V{j;zv0KT>Ep!-A4jj$KE8)Ob}Z*V zk$QE^gikxdR`%)AyGZZ)9sS%i7Yq6v^)CI17Nlp?zq=&rKjxsStZVW!a8upsVLEb4 zp!@EjeQ7%Xh1Olebc;?!;f1100%JJ2%TTxhuf7JaDn57xeG%l5Q{}}fjQxmwTdLaI z@~Jfwkxange)+1HKk%xUKak1(+B|{jP%*dQMCKNh`%Cg~jnjY?kw(!;-cJud1!b!J zAu15$LZ;-E#;)@^U7I>xXLY(B(&>7rdJTVOchV=5uH`~wl0D=fn|v)7W_Eg>e67|O zv!4?DL&~UJ=Bc|}kjG@#jv+rOy6XikF|HAGJ;v<!jRvYpX4%SDGm2 z>s+&^9Z5`Vy&1n>BM;nQj(804sd1MHZ z)tGF$1+AiDrcG~2v?tE;ud~saj0H;m!ff9A3?fnfN|b+8etir_Xix-=dTFH`upxLqKXl{bHAi6*@e z(_c_6?fi)8(_c+PHuN6K24|c}un&~4hJXb5!_3kIf0z3^le;4WgV1@;c`yzOFLrm# z<~{x4&mzIy(T(?DKiwVOnF4kRSQQtQx;xT%&qkCPPcNdUdgn-Yhtai1m@zgJ<%oN& zN%eveeYrd4IImsQfn3YATAs?7a%5ZFzvHbJ{e~Q8t%y+>g4`?enL5v)HAYD&T*SV4 zlkVfok(cA+|v z{~MjjqTye|t5x2NctxLU;5ShQ+M;Zw{s@jnSF5Flz{)>eLBlY?Xl|NWT5N%NMG5V$ z;skeyvut;L=kPmm$LB%i8TvAvp}oK+`U@Dy+a>vXr8kb9|NI1upYIYhYQBF=jqN5F zt*yjs67*q9^}-5(_zc8(hKh5S9XWc^dk$`)F|rEAH}ESi{4fo*8qeo#UD#-?u_H1g z>t~h#M?kp05pI!>oig!AOVscB4(!FqdkOaI$UC(4r^?3?g!kx6$bK-9{SdsK#UAAi>Au_`ZX$1*__bKoWU%f|av zkO4|BNfdB|GS@(mX6NMB{-P{pIgEQ`JHyA>{Ch@IjOF3W1HNm4cI*OuPFWTK^&9;i z-GH6aB+M*TP91_t)VbTbI|sVk9&%pbZZkN0yW7S(d$^U^?l!J)6XQ>yJm$gok+mu5 zi!s;kyk`}YJ)->Ja-s~+=}kRj0BtOORu=ucUt zq@bU`#HZcuTtPRWBiCm!`Z=mZqc*t>%8R_O^$#kEyn=GZ9*n%if`(}pRup|RsQtt{uKNT_}>E>|J#?6f5r9h&irb` zS^Ua*z5r4f-|L|ag2C3*%~V_r*RN9Oo3~Fk@9;Lcx>N5Me@FN8)NGi1kb3f$%5}h{ zP(Wp&5ZnsGFIUH2COJYB8d+V+SZLCG9mN4rdxdo4pOB5>5QcT*ffjAVyQ;;ZT9vI7 zzyqi)0NE0h7hy<~`yT7172-zQl7^dgqMY->g($I*BsO@d$INLZb(I4 zl&$+)0LS|{Nq#6>WzEQ?S#zFd-!b9f{Hw}OP0A)9z3`(^O>YFAaSvRSDxZgGNdFiF zB!+K|J@-$}1ZvI(E)4VBn^EuG`_H0YU?lEl&e{t0+?Y()gA!h*3+}db-t#?DUF%u{ z?_H)N|0Ng-IG$4A-UcCX-OO)zU8R9o>_O$yi}3dVs+H3eqB-8Pfhcl})bO$XkX-`xW08=Wl48>5W)v9sd!+;{U)YItF3yyy<5ChrSL)c z-l*p1tGpG-c_)(d?-I!Q_XEhS(+5+bvO?a17h{Et$3n81H*YSLurCQK3B?~ZH$fYNHHZXf9`M*F z)`EunyX_itL!W+7ec=x~JL}W(u}aj|S3j)2@!(FjbeZtHOXBl!XFtzKe17@a&$}i* z@0R>r)z=B_ptjSP*N!C~0B_fh#UN*0Qh97V%{y4gQmgZ77+Oq2|AX?F0X?(}vV<9% z0PV`%dt0Iac~ngL=`zE6;78)Q+LC zVWNuOZ>cUtmCU&3r|c8vuF4{7p`1qfRMACb8{j!fAYxo&G$#ZJ7p4z@Uhfsj-Rbwfgo<&F`0$==Z~Z4P++4d1Ml}9Le4Ab0E~>h5&xAsx8dkWOhJSf?qM| zIkPXA7sR-mYz*>(L|#(Of$-G`%TmP8{lO&sX!_u3z^5aK_+*p(;ny*I_O!`|Y#4r3 zs|;xE49)o9VtJyx-W*(+&d3)%C!)OB3bp1hV@yCq7iJAqAk=ncHtH6PI}JFNO7fOk z`*Q4)a{Yc3npSx&?|lbL@-J9Qt|lDHMMb7!Hf;sH1^=`m7(DjSqIq<)@Q4^FGD`AE zb5N{B)ak_9v2z)fFpqY)n0d7QlT8A~JEH~Dz{0+FW$P`cDyX2RUPbmf8 zVDIK$BUcq~Fv`1%51atzK&Slj_6BIzly6j}Wg2KLN%o~0XOP6-llUnKW=-9t$gqH; zO$yjg*V<3xBcV?d{!kWnRAjybR7S97+�G0|dq%-d`0LN`SC_)y zxs|E0HKnr{9NNX;5y>h0GWHZjNGNpGXc@D3hwv&=jMn5;T zFY2zV{3E{mAj~y9{+2y{mOcJE{8YY*kH^E~6#f>uMF_YQOa_lrEN@p@MgYI2_{9YZ zE-cDBm7^UrJxfsajrlNA=}%KM38Q@5b}}m0TIHJ1L4&(iFU*K4hx*WZl_;;{J=1Za zTCTRD6Bi@W#lUn9ot3{tS#pn|9py^b8vRhgUCmi>BfoE88ZHOa*yUDqf(A0t5u}rH zj|zd2>nQyaOcRX0;O`3@ZL<(MGCYqjud!wXomUEh^LE;L>OyS>_W>Rz7T4N8yXL)qsc`uxwGKQH^K&+FmyA&Ir${V&&^oml%t|8nhFiM8(; z^RL!!O04~_oj*T!?G04BV&uz@S~HH~s^oe1;@2GpcQX$-PpF+QABNxN`Q~(J-}B|i zp?Yh^G3TWq+pbhU+!Ol{>m4=}J~rW3@G<3Zv7F?+Ixs0}^uTJBY1-lr0O6U@6n>(M zuPzg$^gOD&k*`zHuev@!r~uUhlrlkHP{(^PYb;-$?g18sR-)Z}c;qjc|BFRC*@0K( zekZ!ua*^{?J~bvF-K4q7oac?^%SVo|%N@$PZ;@%bc8>{UV#ID8Z8?$r*|K9TI?|s& z`fqGJ8TC8X+HlVaDp&AKd9NLHEti?TaSzuy?}4EP)j_UH5;&k=AYwk|R9Tb9?6V(F zqE@FEIRq9I?doF31btYTBxgs;fiEeqtUsw5JNUxl$e;0x18_64gyKn8o8qbf4ieaK zWvV`R2aJ*rUq8_oc?XM;F}LEKk8fsoUgMZ+%&WsD@ZN8kDE29ay(gJ+VqePM5D#wE00+0RrGVyZSYfh0fq)Bn_FV?eKjV~!x8w$O{}n2umiRBr8=$WFK;2P z>TM!zAb&aV;br*2@j=9Z3qb2zjv{}P&=_&v1YKDs%0cl&gTNgJ{+BMegB*}mUx1Ru z*9&)F>Y)t3V_BYWWH0;{Q*9Ud(lYMww9cTp$AGU%vVmG%=2t3@Ci||bUU@YK!=L9Ri`hxKR zLpi7t?nBU^K48sCfnlRv-@>5Q#mQQEZYhbsad=yjTa`2g>BM;>mIS4n>!CQ;hmzj~ zPfsW-j^SjSP_8_KZ@KOu1|iqj-H{*n3P^q;z`b#TcUdwx)14%z}DDQb0rUWVA!L%61{c{>o0F-Z+j{qGM=4muw zIVu%P*tz_47a47_wfP{+KO)Vl+hyJcmZrlJP?S~7$eZ@Z^jae#bbO31_Mbda2Ny8y;1WFzs_%+8xGH{GV z%P!|spo^~K1DS8QYd9qCjy}RtDUc($c6K6*c`pzn^cXl{Qe{^Ybcqg7`Jh<;IKNIe7^}dl+u)%>=@a|sc~a`xv97vM*B_* z5L`Q4TUJY=6PPP2)OXn3yldngP%NgrmA??(>(Y1+b{?On;q<*j0<#sT4-2CCGVieE z9+d)jbMgE{_JCm!$}S3O!vJ?EVE%}Hag-_l&+d%R~FYibZDM(~56-oblj zsv_pO^#tD){`@+&O^Rmk;T9C2c%S#SYO{@)=!xHSRG*B-?ETKMu*4swz^xo&BYe*p z=I12B@z$O}{lA?;+~VT`0$2oE3s0nX8GnW5grDw6xSkHwHbQZpWb^Vo1)}L|^GxJ$ z^{|jRY9jBwk0S2?x>KN?;X84aUZcu=Rf#BEfdtBXr$fErYiMPWTGdOZCiHlMvpefCj=ZFmht8AmrK#6zNfR}w(YsKF=rhvu- zh)KXTtEXy7JLf@~j7<~VM+_9QR>zl5;KWd~Q3y3>38CF4AtMAY=;24o2nm)id9Tbm zh@m+0SG;FC&J48CE(GQo!17gA9#S>pb<3HO)ax%YDhKo|LKbLh z+Vt>NxYFKlsXoVg4s-TV36?_BoJm697CjbesVST$RKD{z`)947mF^RJgvLWs;L0Uz zXce$#(*#R{^B!paJHvV=H5`PkQm$kE8O?XLm@jK0+N|1piVUl3&*CKdrRlMH-Ib>To!9+-13m$QI74KNZh zx5^3TSS@h9`udh=;xUsbw-!7g2JGqxh?aGQSDkAFMyeywrj7vb@i5w5ZSFJ*8_pz+ zLQFmj7?4<=i~VH&58J9 z<&Feq*NUFYw_yCjE)>sf2(L?Azocb{)ro@jl(Kc5w!X`I{s51PN9dS5G}ogsN#z%h z?5=ZOLoQ)U$aPE%%+CS6{dD1tO8+RwjD?W%VUUil^G4SgE`oWjWAPvIhDa0XFX%$W z1($Ah7+@5NY_(Q=%m(Dnp`8Z#zSA+sJS{FL&cs}uG^#`0Sraj`%zDzSz6xu8hi zgX<^eV5B?pb8GQPgU)F|GfS{+bsYtv34iRGJz7bNw|N#)#)UP8ez9SR&xc$c`5fs- z-Y&5PsY5kvd>UfoW4XH1_>4W(X?$Kd6RXQ$-Vs@%j*S}cLgDNJ29Vte%0ar_6LgLd z0>6sF^k!%`ZlI}^8_;hcfDI<()(MtvHs9E&>l=){?Jt0F{XumLTPx#mXNI|++WmP= zS6yT`dJCY$05!+8-~+$m|8YVYsf??Fb-R$`VF+(qu56w z4x5BY&!LTCZk=SQcYWi4o(BeUPCc{)n`u0&nJ9M$a*t2{fprd!&(AnO3+O^vfu>!MpY{1)BCL0J7VEE^go$W6hrg$l!U%A2 z(r_E{pA&#V2dFQtljusaG($5b{+2f7E4uSN1{Ky%LNzq%%%Q;QR&3LhwG>~~yHiv& z%#UdK#yOxL3zv4+acA?R&K&0~e=u8j6x1bsL0!Se#V~bwKXp7F#Z6LR#JNh)q61*Y zHn(%?nuzOsrrq;N%SCy*HQ6|r@3Z%uW=S5R=~t3`x2YyCi>t{)coL3YVW$9d5O5`M_lhA8(7h6B~k5U@&k$*&FhcaVhONw}Z=3Mj-#uerpis$vy zxy+LPJ|hgbB=2_KWarDBIm~RcdsgSI7jRli5^ z++)aW{FRre;ZU(9^`8JGdIrQZxGr!9T_be-zFV<)904i@n~ zbBNrYlf{=;@#Rgp?70*a(>dn&{?&Y3ix%*S{u)6(F6bwjY`!$n?=XtACNn7uKA645 zp*fKp{sjs9U#&G``*CM=GXC6`AC$n57W{Xalw}?0OOgC`FtYNYG|WySaGTgplo7u& z334O9crW(KpPA$EKLPv#7}hDiWD_7xdP=>};Y9~txN)yzrXFMKPFl&Xwwt4Ze>CU^W-D-6Autyh`;#VL zX1a*yZPSvX84o1 ze+Op{`}Ysr-N^daz?Zy$>BtwMUqV;7kp95#^g#C=Tu|R`CSP7#5!;|_lS9^(tZzZK z<;CAe2+ZO{*_i`9OS1+hMRAIuZ{gi%67(IQt?=J*n`JE*BU7=~PGAl9$NjUPqFKMv zgZk1+G`uCQ8R>7b33^#DwRm2Z&iSAe=-XSuhQG>TPMiF%P?lt4Gg=%!Tlh~RT6g)hH#C~LBLX`1#z1Y z>k*FJ8u*Ve7RIXBSWrwx_$KTP^^OAG!oNi7lB>e^;(Jsb#XN@3BeQqqhV%w>`)VElD6{pD^5W19Bx#}*VRjwSks>FCoCGFFesH#6C@K-@V7xMMw zqB)8>MM&vtb+6-4hibu?4%#1Q4=jkTq4t8K&rsAZwiH_$ZuRgBX)%SQ+NJ2R)oN-VH*6pxDMg&>9=i0>z?xuqW$ zX^ii8a|C_)Kq!SKWzk7wk1(Wlen-&xH<3=k<2DQaStilaI(rGR8vm2P=Q|4S2VNN| zq5U9HmPI4H8~v!ILZB}guYf$V_py@Ra-i*g&t6B0-sj7@_nzaz_9dK~#QCTG1{c8Ap$hyCn1>F}_bd64m zU>?x^GTHG*fZI}=Z+ZogBSg#-MS0z+92?VVmB$Z0^K%aG>+ zdIkNHMn9Hun+aKpj0tQ_yf=?yy913l;6h*_2!XjKzT95tGhCzI&kx#scVtCOnCGH= zVwNa3qP&>VBw5za8LF<^0pVeOZG51=8Iz^xXMsFOXR}EHbr#5j<0ub;y!SI=wL}R> zvN0azKvXP8Px>9|x`EstzFw08I$hyNmGKu&M5bT(n9YYmwl_vnz_3Rtrf3H)s9nS2 zb8ANTAbMPt*1h9o1T&-i8OOP+Q zGR_qWr^YmwS9J-R3+cI}x!^u$yIfmi-^gJ^5i(?dFCicR6mNxAj!c8zE!c={MFws2 z&CLq;iphHG8kE%VZ5K&ik4H3ZM4|%{tib(gN?J8aWxgGl~2qN#ql#%_t2O75P&oPRDDV%884B99W%@`AU z2}}OOCI(Q-pahRfO(TQ8Q*GwQFr_is9#n$H6D%T#z|!gw5vz!uT+W#QX` z{JE6d!VEV|odqM3hd8VQ90oNUhUQ$#?1dNrK{)sZG3FPF#L%}LN8CFhj=lulbzn~T z)dc)Wfsr61+xQZ+8zoCSU-B27s%dv9qav!Hzn%pBcq3}n@;s1kleqAcq%SbczNTST zuvGIU+mZDyIz^bhi~c~Ewqv<02MJ$7>1o`j0JCGz#FV#LX>QCr7-)NA+(I00_>Sgh03gHXXes2VsQmjVN!gWLELg?hS`FWIG2 z)qnPwW=y^{{(g9a#9HJwO^4}*uhOjXEc=MvWtEv zDB|b?Mpldj6JCEt)0wSIpROQ%dU8xo5ruTNh?}cm?U`adm?|L}A1{K4LH;#nVl z@c+6z{&3I#GI^Z+-oJzAZG8yO&AkcFBl~E0?*CRIo&`0N9>yaxY8=OP&MK5(mk!tL z(w>@K%6nfYf9@gcqW!oPDs(8zDsQ&B4>VidX9g8qjbTz_50dA|dqNl6??f4YhNqCL z@Ol=#?)*mbkHB@YuP|2;s)Ru5e92yNgN$YT;s9NAwP*Y&%2R<}t~6q%T6ve`pJh}U z4^S~nB>xp?AO}Q$J`^(!idoN>8(%Hx$} zu-DEmBE#@;G}uK;YvCMGKH9sM94WaL z!u!rJzaDYj@a8kx{(Kr0XE-(@W=(`_jjj!VMVO~@u8Tq(PKH^?K z7>LH#*GTz}kWxQJ*VeBP#~?ho@U02$Sb*(_vC*0213bx>Os94%po}s1sg0;Yl4DY7 zfeA@3{PGz!?v(M-_u{^YTE65I`Oj#TsPK2O6{1WoVTI^?PFo?GjBaQ7B;%vBLWG&l zd&L>)1Y)E(5yVIH$UD{;Zcu;A&!lFioB730(R^5eBx?%pFMb00TgaH{PQIi?W2T4b zB?~cATwB168Tn4`AiJsr=<~hBgW^9Q9(v~B!(yLk%;^MaeWUk|1Pm=#33@X z$cDJQh_i_w3O^(^agecz(7DwZ^UR<2@1^7y>S5@NXGs2wHMT*+IUK`_8xrEqfj|Y6 z_W!^OB+$2M>_+C`PS4HTRX#~;O6dAjhM9Z9%cmW0c?1YF(~!zBVT zhRci?E^ql4ak;TW^`C!E*;oDtkc?=M+>-=JAollx*grs6#&n!7wxxjN2Z(|9G4n3y zI0VbHAUTF0IWz&1|46Om+?pDSH!vt(=HR{}C?1SKal{*`pjeX{ibqu_p3k7z+Kr%i zCWD}OXEzOsH!O?!C(nt;EipXuUa~4t>3TwidDiW$n! z0CbokShn*euaJNOf_#pP)8VaKnP7OF$d9ZIMuWF^!6g-ES5c2-nJ&s0a*$$=N-5#{ z#J4H&eR14Be^%ae703IM@!J^3ugi4~?EX@XNBI|U>pVAZ??&8~#c`VoZ^NViMZ67& zs(9NJp4suA#hU~0W>3c3^v!4E&5n3;{Hu7oIJM~e`FK-}iqDrN;!QItejeVQ>PmPE zq_Z8K{rvaa9;va)HrvJ}YC zIV}76lE0GT)y$W?tVM1dP`>|)!RtY?TW6RMQr&Ro1wCl%>8Sw4?Emcv_P@lP5=PuX zfVu{)uat<2$uqR~Bh88V_A_%-fC-kC!YTk5Mg}3koaX@8RkV(sZ0E-SrmoAz5k_%3 z-WdtDKA8A%vjm44X(C+CN#tn7^SbRJj@^_SF>l`@q zCE)APm2*!md=Y}tAO+kCwSwqL!J zY`|`<`JK5EaRBn63<88Dc(LFLii+w5IPZ;);K6115h4z z0gNYW#&h93=C2DC{+6um@i6bhOypooIHWDME)4h@yCuoL_N_GHenz}Jw}_+tJw z^y|m{YmtUTys3UQ%NKA#4xX+MQCG&{n38P1R| z5j$E%RJ)tS{Hc6tYJ;kN-b{YA$wrkwkx95JX@8*&+iWd9381Y_0`1ocO6YH6pzV?{>NBcO%?>1S+W-6hf^hd|E3J!r?Jd3J!^Ux{BeB z_I$T%Irc~Z0o!udV;VQ}^@IXiz%N__I3r(+_NcC!#pf7EbF{&kmt z)Me*CISkXBhFC4qusSn_)rU=-HFOxgWu{aSAZlDN;`W(Msr}+No)fpFDe<~B6<)tA z{EYB=;y;Pke=Zky{0zMQeZx8NnnuB}KLfAd{vW}sV>OE(1-zDio*J(V3;#M^EB^y{ z9SV3IN_YjX;meQVb>EU?yfU8hFW~j%(+Je-UOz>6J#$LK>yBrCHeUZ0!|SW(!t33t zT-B2=^UpeMEiA;BPzrSwvvL?=n-&TieUgcTO8{_IC~UHO5BoA%qGjhzDVLrI$F&>J zu}-bwn7r{R*T;Uo{QXjuzyDD@|8x1f>Sy42OWir~tS6u1|3mUO0nfqO|16&W6Y}?5 zc<%e$zlrDdZG`8>6NKkqwrP0o{`7wX&xW6aXFbDnS{%cO%7!C0na z)3MLmQY70E7~d~U2ENJ!q=2hTOP)=M+FOAg+%L!rmMd$U5rQj+BcXWK>SF8KY^R6a zyU%6v>~P-0`*6LQZJCk$3(9B>7&oU|i;ECd>^Qsy6-iq^e1wjvv8^97*!pm7;W%2g zEW-UBv3&Jmqx)o=%ZD}g#LYh^lzvBvJ6r?hE>||xQaarG&&qQ|tIXaXb?W-C7Ugn( zGTSJX9ZP-r%Oro1Nm*3Ow#Q8`gPB_9cPw>vvl9EL!}ttti$LDY(I$Y`>;odk_*hO9 zek;mH9l33|HIDcEzyNt00J0qbc?^bn4FhCb;RG1nzI0rvrT-tYgwLeSXb6A`%;_lwyf|%9b4Cm?L#59g1b51 z(y0r{o1xEm7Ob)?sIr+|l(i^-nJ9OKZj16%lQuo!N7Q{NH1x7#neqz71o|DzcyAX> zAG93)2YnuwB?05a{duA#RB%ngdYG$BkZalgJhmTWHtxs3MzetT*}y)J(*8zgFC6@E z39aC^hgrN&nocDx7zE1BMI?3&R$aliQY>Y#P2A+N%p-AMYRecXpKAE0B1DzsXr*29 zG(emAk{f9uqk%2C!4^UkU zW5C|?(^q~fg7@qr%zP>8fm~!go~|g$lX2gRD9-`LG~6{uBdc%@B?j6`G$+c{;Ucwr zjOirq6j|n`=nH(v7)atyk!6d+QC3o9sg@m3+a|JX5&3iQP$>tukwWLAa$r97GQU`o z9nd{{sZt`N4;?p#HKsVUEn!?AKKd}doU?bf)El=|e9i1G8eCI7YP z9pAw=RH;ER-xEFD!t|B5X}Z%z`BsdHF+j-}ndhHtROV6z&OfVVyqHwRJBekar^U(; z0t?D8wgy$8T6-PQzC6#UQLQ8|2-eu^0J!$D8hbsb^r>dCD%TUuxXNh9Q7$dtP66Sy zTi_aJkB(L3V2yQn_hDU5o~?@?!E{Ftdg%ZhrK&0#%k~@I6U%Fc``t*k%#6p$yvSaq zbJ3FHWcbByUGgG$Yhhtxtjxpbj+MERoy+xa#LEooI4fS}0+s*JK0da-9g9twuEnO@ z&bHWLXv&6FDo@j^Ln+GX4tXJSMVb=#`BHi4#bD(@FM<_vAh5YZ#L?6fM_NxAF#o+cnq|Z^4MpLOXWt{LB_PcJ}{~VxM+(t*T?IfnAmYSDY;(9$x0s&kOb=fPCXK=>^%dy!H zlhR~q1btTCEarZxMPHPp(bjb-2YF^a{8g{<%=&Z17hc2o<&i{w3F2XW8cw91#VzGQ z^LxhoqsD0D5Q7{B*gFc-g6~_7@g;Aju^0`^OZ#@Y%6(3DB*Y3)d?zuVjx&k&rB?2S zj)Zb=)XFs{lzW?2?(3<`ok{j0DBH*fNZ~ep8f9Aom!rbX@=<`E%_77`JPGn>b7DRU zE9RqcAOpmF6spqgbg_ICy#e5>8HYTo<)cvb?TWMnefwd3g1$XVuj$)GdgPMTT0V+k zihL9{-@tTc&H1Ke@#VK9C8W3}IUj|p@7@S%te~I5@_9J5{l`f$N!=;N#j3X9sfr$#e$mMVqaQJp49LqW%$9O z4mMhgY3MrGXi@MfT+nFUNiwAaMyr`*%5g2jMJqv2^j}~_G6tc707Qp@DpiE6a2he@ zFT;(ekw<0Paym=`Vn`m^I??3AhdaMd&72bXB0g1GwzVl^06X3>YY| z@|WUf?-vE8ychGHXU~wjzFg$2nA3x0ld!O@D=ZC0$7wAM2J=m*c`%+Je@B{^nuL!gv4s!6tVnJ*7TqxAClw)c!+@cuoSp=4IV?e`X%)cKLaa-#4(w=#5mpzj)B9&$+FnuHULD9c^|gqv782}<6zOZm1dC6&Jj zl#ewduX25e_vCaP9V4EJHzR+Vspcg;%zGx2kDwKr6Zw@Y_XD#U@`^h$_!5km!bW7o z3=Lt8)>hHd0V*ThqGn~5W3y@mu3k8?SwJ>_iZOpo&R#9F`+|Np1&qc0ynnb+*dEwF)Pi zXv0=dGM=$;k*iUDP32oDZJjem?dxB#=Cq`~K4VAapoyS`B-^SySII){N27CZC9+s| z$7TFe-SP5mxGdwsZ=T~kyBIy5p%63W;0hwdK4%f)%u><4zz8}3%NMjP*2nkMVb|I+lAcqcJn^$LKHw48Nklzs>gGT@MiyRG_TL&u^4o1VEz17 zbf4z8pEI&d$9v0kz?^!b#sk}}ESZk6-EU0kp!s!Aw(HAPNG%>nTN@ll2p%?Zn z%+4k1?VyQ_HmdeB=IhhbFP^q`n3BzJhn8T%wp|URB##l@HBh>%!qrH_;wpf`^3|zv zBqay9hb0FPEcRtCGr5{0UDfQFSR@4ZWhH!N4)fIsA|;nP4dEYtGwGsNEB-AU|i z`?tj2`omS_AE8Z3UuR2Hxot5e9QYG^r<}cWBfdik2ae8ZiL(5YvEhj)kbQBmH2l%>P7ejI_xQMt~&d+;UAI`RY5hnlmBPY$-qHO>V#xz3hx6cbdZ z$!QMwxt1^QBZ3CykheK<8$nz4{fPwmdbA)w0%qo5T_C#)`o;W(G54(M#}xy%^K@eF zCP{`?frOT?QV*z|VoZ1u{XUVAEMHCt)6rwKVkm6k%g09@-g?(^q~I>aCySkb66Ja! z)ZEqT-d!YR)OqT8c`K@u&|Y^WRO;SSWcT8t!f~e!J*L^tn>ik{R zch&DPku04-7LUJ+uP&m~qXpzD%UG@Kd@GimCAQHO6=OW*<-BP&`KUIf3HjYd<|tj? zQ%}OvI|tJO6ihvCJHijx>j~%0wnoa&%EumW3p-(U3bcc1bsrBSEJAMg%{#@dR&O*1GgyXRlotU*FJ$oWipm?Hi9K2C zlFD z%59doO9lBBWAX_QalC+Bxrxb@wRkiL)+~m;vp_3EH{9(YIrcNjVU-)Lo5FTH;~SGmiQHO2 zZh*1l-KeF9*Cg&YOOP9XM?n(`&w?LKUx;f9Q%PG;=k|UW1irkx6qp9HJ1$vQ2x+>) zI_DEMIhZ)F_0FrIYkJWghIwsp6xy%nz6}Z(B&-2NTb)Nu}>iRrS5ga0X{Wo6&{(g(L9m95#v7?&iUZ_NucO4DHPt zIoMeUGnlb8VFItRlD3BvxY?15zER&Yn!w|fz0D!JbZi2f3i^e5tdp6%b9zg;QjZ%+|Z^KJ$;@5ZF2lCpL?sdj{TtD_dETDh#n#q zvda}L5g`uH7#D--V+)H_!9nWBO}j1FY}NII*0 z#JcGSo#lncAUL>chpfQp@$OG?Uh$~ec)r3w-fnSG8_n-in1U#b{)Nux38cn1ap+s1 z$rn3P&#UusJW)GGn@>sJB&hTGCQ|{jBuirw|Bc~NonS}6&|S=3uT8Kmqgg`446;-S zU=%UkxAXknOmZsX5@lP3{kQ8%v3_*!w!-aV_hTn&G#qpLjUj)<6=WQ{A0X52cS4P5mT~}zpaJk)m zaPXo%IJ9|ud2MJbg!RpuZtsXqqe`ijJb%uS*@$Z2|;rOr7Xga+P% z0_R2r;FdV1*F!*8zXaq2>*&q`zI>n#@9;V{_+p*-_whqPGi~AE{2@wuzSYvnN@Z z`~hvc8M_#gqVt+b-m8Q@!ratl}GWbNjfS0KUu>%qf17)6IS4 z?0ohlk7DvAPxT{^1C01d80pcLL!N}1I6`N z=TZ*myw-}pK+X0-l=b<<&T z=B~qNFS%ags$ph$_uWX)ACh-Cfy^((yRWEmu0Y-r(g%PPleOf7(hsa>QjA8BKA^Sz zZSpqI6+u4)MUjON(lTr#vkQy@`H?n=xnF@G_wCf7#BG$gJrXWIze^H^C$xl!K_Kb9mv67>frUnV5W7j~L`Vum!FVnjv`TQ`Rxlhd>l)Q5DA z%ar_ATVWalcSd2-^Fdxr=WKF=)xC$A31t4Y50>qp;%XP~LHYv?#@5~bsok=c_g;-2 z0YX_X=59wqxrCPI8%0Y!zZeMs%5k96PRy-G@*v@K z6^bUSWv%N033awZq`G`9x#oBsi4>z~;iMS{Dk&b=RH zd_d{}bvnu%@=iQEK(3Eius(+jlV5|Ae`{0>eQ&||sY{?5R~NL3p{|W0)pfAQk8b(#V1*fW6XqaNFx*)ujxk;N4y2L~#32-BOne~sbL!SF#^ zU`!yuU$FZdNx}-dCBLIA_DnfL&q`T3kh{f-$1PI5-O(6<+A%{RN(Uz{t}mWXsMsduZU?66jlygAocH1Z zs%2dGBW;~d)8|AyvNA;cQ4Afj@a2yKJepK>9NG=)xF{cI_}ByZXux*L7jTS2h7th6 z^hD*Y2Z&A?{pQE$7id-rJW67)a6RUAUnc;q+L)gN-H%qVs;wDK0+Q-LbVJ5(fV~Ab z4C)Ol7~TaXn#{AwP0Y!T2fD$9$hE-W@$SzUMgB&!`*1=et`4SJwZB0Md!9dor|O1q z0HA^>DiA0Dz4PFHaeJ{VgPcN-P|~MHQPm(E8Fikfxobp_q{2;1>l#m|)K-O;(OSAqZ&qZUY$7nQ*pb9Q?I=4C zmk!`MqNW7*!_P9dQnzstLBF%uD!}25VeUT#e8GX^QCJFXIz`ENC zW2EVZjhJ}O7)^-tRq0bBPt8w}f`8%~je;ID8+5W);o-+-(oQjXKJVR1vS~quj`w~+9KIK5sh$_%v*LCjXiPp+ ztLI|!9hD(jQ$ho_+b8UJ_oDigtyN{{~GyU#q>m` zYJg-ML;;z}Ab1!{!vOV}rP}(J{|0plY!F}Gv-o7!!t*`#Wt`KbnkcdTP}=xFSI!5) zyQ$f&aMQgF%(H^)U6Q{m7FT4X@}kc3(R#uHdwA-dmk91C2U2fvtQ-n>+_VQRz~NDt zMDKdnKKZyg7&@eJxbo4Ex3n!h?*7u!#UK$Qabmf=1Q zNj`|1$&^=0DJJ3q$$zcnf5>PztLKt?54$d=bM6i*70;qHJ8Ksg@c~J0N4-~c9|k+? z5bD{tL{USiMFrR}ve>yA&?1Hob_Fd$%-G@y&V5sKMgbseX`YP5Rf$dXLj4N4nWwf1u?m_Pkf7(O0njQAM z3l!0oXMU<@VTv1;+l0z5Hn4v*Mkn^f#|HUR;Jg>1R3Q}ZqWtwI%llcwRQ4WgcN@S# z>yt8~g5BJErc;9?`J~n!Y)$A;7a?GH4w_`3CN)Empf)Pqf4ZL|QFCg=ikd!Q6+KDH zmCHvbW*$_oc!}Mf!J;I6;oqH!c8b?eIuF=e@9YB>lc|t)&bDVed#ip;yjY15MXn=) z|9o{h1D$140Gd-xw6|8q6GNtC`JS0baYR$ItMfshD_zw?XRTNSA7}@P4`LWoI=nh| z=61?>F)8q712lHB5O}SOG|+Wu7Rn$`jtPst6YxYk2G-A(QaJ8T7)OWvC2cVD;up+1 z@>azE495SgQ~y-99Zc7$XW2i>rRgXNvV+buPs6LQdpFN}Z)YNoCGR|_T!+U?`}4B} zzbgx`FQMz{CP6TdMYhlTM(kpU&-!6a=11=Y>#R)r0E^34H#>&_rZPTUc!li1wPv`w4bAdLq zoED&}8&@J#Jp}Vo`9z$mly}^O@4&rVQ|iw`z+NoMCxk$uk!{X%V9;HyL;gaLy(lM@ z_C4t!jrzD&B;e7hXKf#H_)A!gb!nuEnlgFsZz&d3 zn?}K6&*n?)v^T^q@Fmko^WG=Qo29^lQr^R|McA_?|0CJv9RPxKt9$<_Ti^~3_`E~j zBFa@WOC@=WIk*yG)*`$EBk2>^epoxjHR5MUk~^y0U4b zW>lHf&ZZg$PXImsGUAY4A^`u#L_G3Pq^+}3I0kww{1~SkcTOc?_xqw0)M%{;GCC3A zqa+Fmrg|Ofo<$q1?0F85u&nXWx6iP{we~V*q^08qElIu?Om*DdK0` zOv+xsUww6H;(ECIyNO8HSq*W-!U zI83sGPU2GyYG1*ZyRrQ#8vkIXa@lF3q&#d$ z;*z!P?EQoS@c&G{yaWs5y|}I~jId|%-W6D%{B@iOy*Xq>lJ_w7CdrM; zGrQ7}Ba|{$T!lzFU7Wy;oo2>~>2P&*KNJ%rfgTTi)QL6knV&+CftugL`Vr3L)tzVZ zqxmU_j_k2c5(i1Wv&aGcjYJOa14NNCHJhT=^DEeS&iQMt{(K_|oIT3QM^WI|m#PQ5 z_W1C_l>GY_)P6d&E{T5!x(vVUTAdc3i`UC9O}tjm-v*kegW0>k(asK&53E{-d}Mmq zGW%3qek7hBj^D)35Lc{g+M{xi0ue6)E(xanbantEu2sb{Uux%f|HOIVv2{H()E<)p z_ZfvnwabX$epI?IJ{2X~rCKpTg=nz%q4uiz+BH0Z9#py-8WcheTNDBq4p8UHL)L9> z5(2~e(MKna3Avlm&ma3526*^C$RGM_8Tmt>L_&E0M_6ZXi7KASKame9iXswyE&jW5 zKv_Yqg(uH^f-)`e!7^fB=o0dpn9$6k{Y$NY9ZCKk#H9`;d+wrBp#9i^CDL=rPj7=1 zr9&rR`DCvKC&ZYfRg zvlq7`-e!2&i}Xm~LevlMj(B^NF{~lH2ltu+xo6>s)Ov@iJ4Ik{hTvNt*N%DP+Rf9#snt8^_ zo~@&6#>&1|4yt!QAOi7DB%J4$5>0@%O!KeoQD4H8IDR{O=*3jvO4~vpm=W|hN;%%! zOzm7UoT&!%^}cyv)-Dy$DSDzoxTrx0wT%+IG`ze=p!dBzlZz= z-RYIH;{3+ELn$j{95Csm1TCLIMU)Ba@$YrwvrocTm=Y(ZkLFVO-I1R?mhj$f_}(&j zoR z-ffjZ`#G*$H3+v5sq|~Dz@-h|dlTD<^NV+r2MPvwLVbR5(*vi(TUdEKp{}JW)nZo_6D8TW2ctSkI|Fs z2`TVMsYINBN@d%o@U78Rz_I5p9p>l}jWTWnyJ&$&ay!;%{$Oya1WrKQ#cjrCz3~h>%YI3hLRt#TDM7ss=EPCOENdw>* z7oWMP%0^#^-hwFYV^3p=Cp*wP?=w~Q%5C55M~$z3qa#sNyP6l^B<_igq`Y4IRS~)D`D`~u0J^itJmdE1+#^pZR@Wa z5e~oxI{$gV10M;?P{#0Qoqh})S-LNvuK6HavV^L>!Qtjc5OGpG5b+yM2mMB==*KB9oV?%HY{MxH34y4)@lPI8=(SX43n#oP0(O zCRS>vbQ!jDC$*FH32$dVwDb8LD6BrB;%lim1-gNZk?ch(_E2&CQY>x;#c+L&-XrjXOqP_H8VQkv*w#$ z8@O)})paSzC5Rpnw9el5bOSF-s(=0?s6XGJkFYKw>j&%S+lGNHQVkLopDNilvL}Aw z+Ot%~Ui~Q&PL<%~)Y%&FICMLx)tdxQG!jjJR{0|8+0I^k5KbJ#-7ts=x@N=e-p zZ?y0622I;RJD6R;#RdE58+ryOb;kQFWJ|um?Nhj|5H3Y~kdeIOz-r8n7x2nnH|`&# za=hHUV%!CAT1gfb&`IbC!2|0E*y3edp?mg5Gbi(k!QPg)r{;YQtA3dqgq@i;m*I_Q zY84iiB*l3$y4s z9SGi@Om0x2BBboL&y9BAV8-FVr z0sTJG4~e-y75_-Z<2jm&hC*>lzYy^nLeazMeMApk_$nAg57uo8J=Byr#ectt#h%3~zQ^^tLVQ2BoA5p8 z8w0){-fWQfhvx_Io%>!$e1AFd68MhZ9KiRAFOBm4>n|IX_th^qkoOz2{x{@({i}`0 z`(tDE^8T?eLdpAii$c=DXOCS@-WR^C$orF7SIYlA1M>cf_lf@(5&yrh@qYsGKNbJ> zF7f|+8viF;Aw4|tqr(4R8Tr55$p5)I{(m<#|4+CS|Cj6d|MN!q-`kM?$Naba|4L*0 z|6maRe-@Je-wDnC;~V9Fugd>pu9W|K==gsT@&7x-|BE#Kzn%D>ivNC_`2Ssv|8KuS zd>8E`d@uRJ$p4!R_&)r+j{n~d&Hs;HivKt1`2W+!@ZJ2y2KfH@{{Ir+9bRq(-xnXy z<9qJ1Q22i9t&sTMFs?Ct?~xUJ3!|?P-{)=%;QIwcbe|jELWGWaN5%K6-4WlDs5pX( z3*J`oJ+1o{;`@b4!uQvo8u0y0sR7@oasv2X_-06auYKea_>L_N;CtgIjp4iF3k~pn zeB^(L@0+}h;5%xJ9^Ws2917o`z7Z1NJ0ETg-%L{Qef_8_#CK%(0KVTwO!n#Z1|oFQ zTPnVnbVGd4q~dl|y!1^K-|lW#i0^$r5WY8jV!-#j5(B z!1s?!8^iZ4c@6Mgo%vtlyRX~`zOTJskMH+C3We`)3qs=i=tGU+yJmrc@7qUQA-+3w z3*h@R#AKh80z~M{H&lFo-4*dIQ*j&>uY6s__iJ6R5Z|9|C4BE(YQXm^>kax`^y~n> zS1b&P?*k8B0^iB&1Ni>!L!&--^utE=xtr%V(C7Xh_P?Re-6l1n&$S<^*XQ0?5=x)@ z_iG{PU|-hd^f~5H^ttPYUn&204d`=k7ZCryM*Lr(@&C=l|5V(LikB|b`2XfBq=&b+ zDE$ACk^k2k=)p8o$N#T}=KlvS#s6z{{QrTG9(I1vC_P-2+khT^8~VRN57#YdgdUn? z>gi#@;!yPP!7Cx@Vau4y(Ze3MLJz`ySGxY+6rhI}UZeH@6DG> zBju*x@0n(Xzm~kTFu7@7TDY((o&SjtR;Neir8SY$Bd{!zKjft~<%KQs(k#5Nd0tvG zIo*PV&G|!KS_@t%K^Z(27@zSgE(%SLD4tZ(q<@9z~*nvOfrCr4fugOb`;e{RZ(yo@%W3aFzf5=O_ zh8JF+m)40FUYD14t(@Ko3$No3d1==x_!*JRXB>!c)l_tsSwwd^`UU5u)rfL>4SLw7 zomU>?^U`XS$Jo5I3(6z!u1ePkAGC(A*e+{G2B0104hQzTnbh!@~OhA2b%lgQl+#%qicmdVY5Avtw}p{>Y}_HnTSflqFf z>s}02))a-!S&XOU+3ef|NuQ?CIeR8t0P1===oD5uE6)294ox8>!GiB$)Jdm>=Rd@w z*?KM_e?}Ls`7j@FzLe_h($Ds!;_1yZyvy(?zUx15r`yv7ZXhLXa(Gw5Nx9&LjL8W* zAIbHF-fgw??f?NAGLqPg4@LI{^Ymv$uPa}Te0wGpJa`#s zddJP~lW@`WXM6Qdz5ii}RjPpq6iKCgJSAyeBAN@UT84@4VzZ<34!CA5tg3J{J3T0` zWh;y*mbBNkCBuBezN5;KR5krZ5&ZSxjJ|3iTorF4&f5r})CpvCNyEDePj^1+uR5rJ zCxL$Tkkf!Sna(&Jk{8@#SnpUo=}JV0Brkc)U>y!$;rUN6dhUK-3liy{yhP{r_-DW) z{7(^3AX|Suiu61x_Ia_m2Nc8oW44%zAE)9iRNSqkxv2=Q-m`hU82v5T8+f>w>})fu zdH2O(CKI@vMdH1Nd^O@?CNDNW&?jeH9I!S!I> zbudE@q>Wz_0X(@j896gj)dW0=;Lrc_mO5Uo$*Bu|f4OHO|D`F^DuP_0J zjlico{*;X0dy>&Dq)Fa+zyDjPyxs(N84JNZa~-%ui{+kJUZov=m&>0l^j$p6)|(!c zu#>+`xd@@NQ}1f&kk>+OuFp8S*4V56fzYV32VlX8F^c~tn#EO=F6r3tc-ESPV>e#O8=kWtE`!y8c!{?FdEr{IGUUwA^XbzXfAj%n>Ej*qkA z=fZ;O<`A9woZ$JU8C*(yx=B|LT)VQJ;?G|Cl0^V}Ob||1Kr}Qy2a$g)bj4DrWy0&; zCoA|*^!T(lI`C7p@>6W!CwO#@=8ZLhf*yeacywBMK_nC;2MXZP8P5wWP%tD=0FN$# zzu+mc&=vorN(D7_@MlpEqPHRFltc)WSHj#yc`eX_zmn4nTq1E_uyFR2v1Y1_-+@~K zzF}j6yH8VyM;76Xq`>$x5(z{*9fgFp7Vi9fOEk2$2jR;hEyck+e;21yxd4c>TLF+- zhIa;6VOJtWNox;7gm|vBdq;5bx^zz+jxGWdx_L+NiLE__KON_Yt!*U=>71Us? zNT%SaL}pz(k$wq-%Lw!s&L6Ysk<-E29C~cRAM@xjl0W9tV^g@uM2{B!xR@TB@yBKK z*qlEur^gojv5+2H^2cI&jN)X+mp2a~u_lA>Mfi##t*yY}c^IM*apdTF7;ZxZ^74R# z{6%u^ya=*$v&f}>FAuVg>ZG>;WqGaGtH6Zl<7<$-76g=4+K9p8s&|T=w{STuVsY(2 ztF#+JVHW_6SgUsnt9IJW71m~Dd=NMm^LGA;gQx}og?|u7c4-SA%3&P$hUC1Jx#T1V zh^fk^`zec{y0GfBSiL)4Sk)lIY@vt_O8#*Q(hLzMQq70GY6M`uHfE!$t2PzQ5V*NGJi9 zvxFb8{mo*@dEWAl)Xqoj?%I~Z{6}!@awHXi;1cFf#1(4MSp8rhXmOU0)|Y`rD-%(u zeJLt;*of=+QpZ_MMp1~D)A;KI;VEP{;Arxm@c>y)5XPdH^914H;t9f(VlI^SS4k*wBy~BYklvH3h1{RX*$0VU zUI`uDl0aR%KR&wKV$R#8G75jTodd@#^!H-GVPJB^|IIjcuDfr=m6|lqS z0zk;!5l-vj{d_&---~rbskG?PQuzP#8mO#ubm@b#g|NF;!gLE12Ix^oK#0d*Nm26_ zynB&;eV+d+uYH#mPSV~I;rAy1WL)VD2Yk+QY0hlv{e{BQB7W{puS0m4X zym0?olTa*`xQi`Gd&Kex0p#@5lcKrC_i>H?O7u2?^tSjj|1-U9%Bf51ZLy*BHebrQ zz}Q;i*m7`e5y4{{w;RVc>$EyH0ms$~#}=iHt)(`$oS?C7gt0Znu|>ex=EB&b17qu< zjV)R?wom@y3z56t0;?TLFIUXeW~tI?uIrr@mvVLQTv>#(nHi!s>RB z6Qp*0UtSz$b^mS_awE?H`?QP9YuD6Q`w!v^^>leqMl2@mM(Bi5R{4o#qrrvgzV$=)s_W?qrMW}K~rPvU*ptJ|K_8sObf2E`UO8D4y zK$Nkv?}Kwh*~>ny_4|?#?ooHn!=1$19NbA9%z?cU>?E9RaIbVN6~9Nt$5!)=#x+o! z+2*W2U{5GD#$OA>d(2cWmLF67<=lR*hBdht+LeyY;G3ByL;17h7ss_L6wQ_X!>T>6 z$@Ht6>*BC`o3!Bd^zS=zg5S3r-nZeqF@Fj54|Uc4kL85!|6ELP|FWU|p@#Q+4euEt z`{$G$9j${$gLebGjPFKx|F`An(vO$4;EvTIQORji>dYd&wM0vUm4m5TO=Pe>UL$rr zKd%HnMrQk*6Mj5vbChX(0I;IZK)abamRVmq z?+3H|v*GY}riD$Rhl%VldYHl+Ijt6loZE}e5)SYI4NF9<&!g%G=uhB=WBlT1(eyM;y3U@c=(B^Gr5)Ru5U|654= z3FH&vn>M?&9fEyHuC46g!1I*(qEyJe8?B4w>=x^Jlwx6n@Gl(2R9mr{T`CrG!RZ-u zDeZ)PgL^iZP8OK)(ZV0!<&RcK6!a3rcjZrE4#~zpyOVxzBIGvbPwVJLKs8(2^gL!N zgDUf2Ng^{_Ir~T}a6T!!w%X;{rVMZ+KaBxvSZ3=wIuF^oE*!t`vz>py7tXtQRf~Px z-|P`=!Fe00CROPgUpzhw_yWQ_7CoxtizS1&ItKSe?X$F-zu|#$Tf%Q_?jRE;qWXCl zfk*)Gxi`iH>nUnK=0@A;$J?nY!+PtQE0Fjm5{65&L2y9eIp1kDt{%ocfqF94YoXPH z`)K(hnXVK-L~Te4)^bR=((r=6H$GLmh8fc0NP%mk>?$ z@Y6kA&-xCk7lag``-4`j z_q>6~*>iU(m8;&MdRJq;4-9-x?Pc)Fz6Z#vetHJp-!cd8Zw2bz2=#jVA-@LX3p$qN zYs;}s`IhMY27e8`&nsU@8U;JG*KE3!y&SS0k@~^W$+{$~s)vw!6JD5VRQ+&xC&hyb z;_!Zhi+!*r0)Y4YZ3dFoDIp&&p=3=N@~pUtu-=Vlhua`U2)S3$fju1Mv%$4BevEr3 zg{w(*orBm)wJPO#n`yzi*DG})-?NlradjarM;|SbYcflfIPw)3cU0+K$bACz=&Uvh zQI-q2uTawyS;hGpPAfAfLft7?Hx240Xmy>Bqqeg6d<|+VyzUfs!$3$~m68n*f)BIH z-8l-tunCYU_HirX;72xMke}Aolx*j#P9~vPbto+=GN-5Jp!c zy$mR6*_*Q59YkQ<_kp5k?ad`Ut2IK0D+LyMn8^Gz;3@2!GIBQgu7I9(gno=>d*~sC zZAU~iloR*Js}dHqol%rIGyQ~4qW$Cm7$DA4D!yTDvBOwo{rdm^M##<|zS zGPkQGEE=xcZIq>Q_}w(b7yGEH!l{t)s{l-5cNI(-rdXBf!KdgP>yD#xVo@@PFLgkV z(tuuZKB!I-vc{5$z&F#code(mQFCAqBQ~4^pcz!x*wc^0;A%j?086`TS~%aqxmPXJ z;+gPQv(hYtSfKWh!CDh`)1XR(?;*k~gxH#hsq26;ApI>~G-at#u0wDGdJ0G)$U|K;lOz!s!5Hfs(#Z?)!}Yfs49(&y zWXUf>IL33L>?(9#g_5@b+#9tT6fS3g zurFQc#RtS+Ajr7G@KY$f+y&r!UA`bH%R=orI6mF_M z%wGefU-gcrf2;u&s7em#BVc5VHEt>>$O`knjXg|qc*o}yQN2Tue|6*=`1`e@7o%^0 zUbpZa{HeL(GrP>hq_ff-$kDw{qnkIx+bDfy@9Ca@Ao7aEYB_G@66# z=<9xx$ll3|6(Z(?`HO9#-yvWJYvn~YZt8j--cG#;>vSIx`fsX;&RPC5$afFIxV>M3 zj~rfqR~noBIx)aNCH}Ht~G^&?DHJ`TgOl@OYuC_)j- zaZ1_mc;|a%vcAlslx@MXtYR!s+WfPC%D$?{@lNq;*Y%IXc(*CVFVmfT_IOJg?_BEU zDrMMxy$GXbIcwpU22BbpV&Je*7Au=j8~%Z*o0NC z8d3J3v!Yx8AuuxJ@wiP+&0E0(II-M)bJ$ zChX*b#CZ)DB2|Lo=d}@o-8-9FbMA~YIZrD2ssEHQbu*V?iB^peI^mj%$jUI=6O+NH7@de{}`zqso@P+?$ym!ikaEQAj ztn##^zQJE8)f;D$lpinx4~--fLc$eSb=LCR9AbMU<9I%nXgohi|LJ(Xc!94ff8KvOj&~&*#}^lM}F^-`l&bPq`+9@NrQyl^C-_^3JH3jqjOW*Gq zrR&a0*9qug)b1jwSDd-Lz-VI0;YhJ-rF^HEWw*HC_sy<_STeWq{(Pfqb^C!L!+OAF ze?DmG(24ucY&PZ<>g6m7cUKioh?8JAU59s;&cWtY$UOq1hlci`BQ>hAQ7ftK zO=hxv;*i;EG!GN*(rwF&lmd!~0vKj}@~O&Ngc6qqGKTX+0|ZsWv9(gFy^b63wG(qpWPrb&Y=I0V!j0AFh{D;(UE*J>Ne8q~d)hgMJm(UD0o{gX8u}5-tL}^Il`-cYp~E~Yj%i z+b1}sl77MI2nc1YuLiei;(Hj9FWV3_7UiY2kd9NLj*T~HF>QFORm(*=B^u-LKtvqx zkQn}M5xr9^Y}{vPM0?piI5rH3W?C*W@;xYDS`z}EA}3I`o6$(tEvkhe#J*M9{+^H9 zC;9{Y$(3>6Mcw}0S_sLtd}DO?4Em1;C_HR{0<7H+b%Vt8s2W7I$68N9t><;Kx)Z8w zuMgy-gxmlDX<)@GA&|KT_Pu@xHq7@88~_+!2F1`m(*>Mp`v%H3<=1spAQk%Y632nz zemq9VEOb_iaz;glbUH(-3^e97K{&e1LVMB%$|4d^`M-np_h8RvL>QOWPSJgr$@!G% zo)c|yKIZT)g|)~k?M4a5m3`1;XAefjsI{{1Kh9=7f%?E$YvB)s_Y>nd-skBc)bSQY zdwtUsBK|YPDRH2@BoH+5SkF%1jbrh^ygkdoC8NJc!SmQT zpinNl-utsgUG9M^{LFim!bdQ*b7f{B_Z{4{{vPIX(NWe@Tngu_5(=d73)D>v-~25v zqQe=CgESR?ceSt<9tVmtliL^X!#)~6I4%#46XjtR5F>IgC^&o(3w{8&{g!~wC|>r^ za~fTN*;yZ)Z-n=1Qb$wI>g-roP@0^41_yD6wGhr4e6xsFAAbZO3VDC@m{4HzLOi%) z!lUo8dfJSPOrSO8Z~k7jqQU(DZNFCs)^xt#19QW4Q68AEAD$p|Xazg_6<`1s=~B_N zGy5UF?Wja+(MdKjNAY1h$+p!fhQw0mcs#Pi5Mmyrv8-n>UpsiSB!9lnb~g4Ngux95YgTkt}+Y3!pL)vf74 zT(rMl>o(yW^V8P;*C0+(irK5RScr?=>x8bN33xv=kKTqWVf=mBktW|K^-5efZ-&ZG z95_FS5=IeZOVi*JC8i*9mndy#!bu!l2Zg)t0@Dz(m9Lkwf4ufK;v6{FtlES3Np=c( zs~ZrStC%debq=(c@MW(r8#-gjByH8Y|&FR3HY31zvU$`<3z<6Y@g94`V z&H7JOzNu!tW&_{Yz+@968cvH+$!?hcdM#o;?^9gAV?wRp3B{!i`9JvlRK)XvUZ}ec zDs_XcCE*$7OUv@Wy{{vvu|L#45W$62bwf_Dwc&Y~Rzx#uAs2m@r2)V!`{dLwM zXCkNjcd`tYp^6)bV}{(Sa*TVUXjnJVZiB)zGO%K-PsM?i2AhifXev5j6;@?N+hv!9 zF_N+GLLYbDs#;5MYKx{rVDzBg6gjiM>J{u4?MU+j_9HMp;N>E1z5D(F;N#|9Dj{r{0aM64?csckF|OJItJ>6 zL7EOBsGg;ZVsqw|)?uZ3bKo#Wsq|}S#rWUwzFN>(KG5QJULcQ*LrBbk<9%DgW>hlS z?CM(OfY;dryivN@>Y>Qqk43{vUjp^8(4%;cDzzqiye!Ba5{ffR{t!xh0*7_d zjv_qKdZSnvw1Ml7Id}F$96$oT-Rz}5&iJbS`XCVI*C&_Udk{sSoa=@1c zEX9bfd^`(vm&ULUyWsgFL!g$YoKpVQpwg4{t82GiUJD^c*>hdUXNrqkC0gG^{>xco ze0ZrId$gAP6=RkB)_KRlq0}nX1$11iyFPZt?Qs9LvL{FG8U7yTkZ-MYU1xRs=Lidi zhJlB!pIv?O440%ORfEXy;RG*@oqjX_f1I!Y_O(`T`g-tXUJ#4>+4hffeh|k+JTICz z_>c1D_il&gYaMdSO5Bz~nla25{15I2+qdTRGbENO9a)!+6b0#w;%=rR%dN-HfJJtO zvqM9T8OF2j4*G=kV!Wbv*fbGR((hKc;@hx`Y%^A40@ZP1oegT8Z-eSo9mhH@tn-yx zXK7HKa;h_(*O}1_H@$Fr^DT&2^povcK^k?!X6Xlai8-mHETZQfQ_uR60_RE&Z&c!_ zD7r{qquY-uIur*>GA~Y*POnQwCQ$DA!5Z7_CbLPo(TMRrh&Srg z$S70Qdx8Fz!~08t_fPq|a@3WK{+MvTd>%SQi*mHEU>?!Pz!@`BrSqxMDbd^K9$uy7 zh){C4qv)NT@9;J`Q0MnItN0csE!pzfSAU+tb5MJ#Y_ZNe;^zvGBRopn)Qj&2=o}e@ z&Uny?_zWkEgc+XKW?0Q`yV1-^;k57fi-G;?W?$LG0DSoCTlKnpDVTS_H(%TDev9kN zbUdlv?E9)dq`cJKHw1i%(%Qre99w>N@JB_GZRD$Q82MLUK=r4?UJ7>ZYA?=a5tSgT2zM^F9+A{AbrtMMjWF|-E8|_C4>Em1CxwK{ z6|Xd-?+zdrN5B_TZ~ITqgN}E+1{7lO-GMg9cZZYXb34r#gBbrcRo3 z@{r5)U{iVk=adFL#9$B3mZG!>(Cz~vAeJMiKnuQY{GIQ_&!9h3f{DuowcBy5YRakv*JFCR? zH*dT9?DDML7WQCA08~iYm2F$C(tEkMO9_#Wwz2&W!L96S4Q{(H0XM!xv~N*;3Ai2H zj&OU`E*9)s7_0oQ zLRRl8(ihSQxl0MS=cIj7)<%G@-P>uSoWj9d?2N@M!_KQkxdj4v2&!nUgw=!Ffwz=o z0sk}(W5Qd8xA%{BNb(1+TyT!hkz<#xa#~>g01CT%j>QD%BB9MT$AFj~!FMDJ6(OBQ z`N{q4D_$hqHX0I_^S=sR(?TNK8qa^+7~VKN{Aj6q58#`%s}+s0DUFee=UA$8IG+%H zf4@q@?^l=Mi*Em10>Aa!NcO<_C-BD8IzO`}6Gj4_BLSq;w(lc#dDh>5L3zPt#eLhf zG2J1*25{*A8gfb}_Xe~3r~pWmQ%j@y+;#qu$oIN;nqTBqn%rre@4>|8Z9n*LRkr@# zaDLZqBffV?zxv=vWI|AWDd_@G=iltjtx9j3oMWwW0a0TCjy+i!yJtVq;P6c*c~G>o z6WB1W66NMde|b2V7Q*WLID5B?(}}g@M&#N}9aPzJNx8=O{3e}M=Xb8*{O;OH^UIL_ z@eT45fA;w*lP{c7FKc;39AfQBzfS)ODepLoiqiKe^9-55uJ$~qQHca zlqK1~F0g-WgirYV22_G#z(4QjEGI^+T5q7AvUbWWik+)2C;!f<@GWZqUrcbmZwv9r z6MoFfR*@l<`F_{&NuRrh1E2H`$ipbTJkALOYlM8%_^Ut_1$go=k4>}eFprDYjnM8$PcqB0{pO;_yGW7 zOG37Ajs+%wq>3T=;L!gmAN1MQkPj4kDQm6K|GLZ3|H;eIe>u_rR2}`F3!?v_2Kcsb zNdKw@klk9Y$=c}|-laKs^j@XImug>0X?=zEm5!}Jhw`YivX*;t1CKc>4FwEg1fh?j zH5&i*G7{K#;u0(9cV$R<@7tvVs0SJCoq`9_PVkDo%Cw134oo~?PZ}0|h4Z~7`hPOt z6~YzI_xzuinD5q2f%*0!yWFRDYZo&)L6s&v|KSO{H2#E=#0J&PQq={?>0ODRL;Zsj(iVT6Cnp9 z+~3%z*c1cG>lB(9Ty3p&Z^$nBBOFYO(hfc{v@=$&32A4%U$Ha(dmq^u@7CKHZzntB zHocuOp6rb6MV*~7AfKB&zUI>Q#t+g%*c(^0A$#KkyA1Zm;IpHA&E8nL>+<%-_jg_1 z-Z*pDY7dX~f?6R_oCA#)PzBdt**Yu)Q%kCD`7WloD)jJTmLD z_Qr#=4EDy!EXdw?*Pc-J#y!hJ+Z)dfGTIy0MgL#h8*!I-5*W=Y)mrBrMUFvUll;hT zB`0H0SNe(X9ME%`O!J)dd)pGuS>3goXq-Q*`Li@R{~X}CR>O1aFr45(VL?;m{2i%M4afDF z=M1=>_q&E`Wzc}~!MhRV%Fu<}QZ&Tru)puCUaAQJ{VXp=2=n3A1F{kfz8N3V8*Xkh)~{KJ_AVo~sYPm{?jdz!LD zIiegvz=)L`0 zGonnY7;Q5n&0RBahLv+s$#H>m)w{f~SvlvXUcYi!qTHNa#~G}Sh3z8Q*qCMsV1>>1rx1!%eR zkg3E&cDsjzPEE?lD*WgsQ-<6!&L+=_ql1&duzM0#r^dCj%c*eP6fFEPc$~2qk51Mq zCnvqhE)NF>tZx%A$vNZj=wu}s<@m=iFxIgN90B<4in&88fCu#yAQ4XiRw{=8y|K5i zO}fJqzoEnNM}cdW#?awOA{_)k1?%v@;mN~yhCV!rX%4!{LAitIE{pI!kB28aJ6GO_ zXD2?)t6lO(^rv)svO)BYfYXyRrebhgf|lI}!y$4SJa#?kuk;j;zF&BGX1$U}HTb-Q z&Pvv#v@-d=0QsZr|H3tU!4H?wPFozgguUS4x2T=IkRcuOMWOvS!3e2$!vAOBr#$kZ z!SC?iIBH!h6FYR+2!8Ys44lIp47;Ke5}4-=hcBUppDPSLAvt3>Avs#7>JJo8c*3vx zcYXYHuqj%lA8qI;8^Z<^1`dkAKH|O*=8Cd;esft^n{rstW3BQ6U?1-kv-@|!+J7~& zl(ZVHZTE2=nr@x<8_K(1V^W*_n&G{(+%C_xu(&X@c2uAq9b`ziPq0Zd+F7NCq8lC> zkUaCIncas4tF!?6sd`_bm)Fgjy`bz8aOBH=`~3$mV=wq@HQ5WE)9eN1nBF%SUZU3) zjWP=TDzYI-J^yL3v=*$D>mee!D7ltlGIuGX5Tax_q>PGkTONh#vc1WUniSll!=RAc zlMlqRp9?gwz-Q&b1>tgZJ|AMjm=->WQ;?78R^N1&TBNKzhui~hc{~Q1Wjl_Id%SnX zJNhdw^&6(ALo_P>TqU@19U;eg2oef9n0H2dyEna^>frvHG8(yHERJRc4ikq4S=W1H zNEfZ{jpo$;OJRb}6!(SOh51h-55UZaaFA!_`<@8+S<%%Ihjb1;yPJ=Ux4>F$<}^c1 zxYjCF68wY8BH~SK=SqziGG#Z=Pk%SssBQ&GEP6YY;^p)zzt%W7NHy@^SFQ(a^y8*i zUq+-d$!}6c%dX3q?3Vv>3DM$Si7fXrm?AU$6bs`Pf5YM9nx&aSjeb?ePY|Mey< z7K0mCXmJ;Kaa%rZBHVn|$} zN#n5?e85c328f;8Xq}APh&PRyqaI9#5AcOicU@A)zZk!-9u;_f@ofhC_eGOxCApOR zhx4Hp`&A_U6+-@hqssq3mGgywJ^PVwbmu)tZ%IUNonS^ej?xZhHGI=$&wDyt)QIYG zcHlAlEcIIDz&h8>P!vJy48bj0*iNupHEFl{gHZ+ut(QdK|XK+{PKo#9xd%qZw!A$ zfI4wO1JwG<0IKKSO8~0L*CeE8AcgDh>U%rDs-6Zvp0gOb$xnSm~m}}u= zUhnMfkgDwA*U>GvO))5;>=*P#s-P3rjHC&c8C&i0T`Z1Q8p8#*a;V^IcL4%UB!gdF zsPjopBZftJ+48^<^h&eN0Vqjz0GiMLUk^R9S+$B}u<>0Vxb)iO6~w^WM`gnEoFTzd z)|IPvnZ=@ecCjEVr~sh>N5S4s$zsW`;lR1KL2Gcy09p(qH5w!9Z!;;C)xtp*A-A;| z7cRWVq;{9V+0^R@H^Zqc*R&nsJ?zqUI2~qfc-w=$5p(SnNJ2fe=6^{^@J4JDWnR0l zC}*Q5ZZjsJut~GPXQI25P|!cC80yMmw6Hpv>k{`wPq0d9?F7%Wv|3e!^hdC|%c6bf z1NVsB7cA4-!g72Q|Ei3uP}lht{WaSn?bB#BH5v`FWQ43SmnIaKRE0}>Ig$EKkiUWP z{6mx~eV|$pKOCr01-J93%LuwIyDy;t{__clpE}y;gW#E7-5;j&aO7!D)PIE|SIk%EsMs7OT+wM^q78iq+?4#;N znba=ZOc+gQwnAphfT6jrVU=ysN(_k@DW%3LthTkAAf?oS&zB?V@Qiod%xsnLe9c;p z=W9a4Arsz@oeM13@t1bM0_FdvA1{IX^IwoGJPc7eqys*Pd`i8{)%CKW{*t}GQCjh? zOZ4)Z(#vz$%R%2l+RtUKr}(bNSR3lhDQL|nBUp|05ZHg*bPhrlqr8SIn6*P7Bw_^M zGiOeO$$4(MS(N%I`Ewvn;xXNM3e-;wQGbSo;x%Fb#bYX8=gj5hVP{J-ISbr76J>%EnS4QeTilUDe(?{RHM-M3je-#JP=+*Yyl|nNQ-?ie{M5%N)chYj=kBotiAt?xm>LvD=KsfH+dsFO<)! zT$FxbYhx4&;%_O@&Jte_#eRkZsrn}WO6UKSGXGmc&R;OjKk+}Gf1)z~DWT_o`m*!C zKh*rMy3G7BY^VzV<~}b6qN0a*qyBgrh61yHF5?5Hj>3M|fUkUr!nri^JLP06TRfE`%7U&nN zo#8unhy=t6u4%Ac)9$f+_*9f~tyvESHs~b4K)IsQs6S*dJQ6uP@V=aP(G)z00GI&s z3+{z0)<_3;3`bDhq(tcIHiRx)2gl?Lcs}#|8jU&5?xfYyn7V7sAk=V1?ZxhU4QSD(FNYmT{1E>UBFdoP8T0uZJ>+f z9kvTp8qV@eobz$d!QC})gj=(=?EYR_7Bu1mT*A%!Eg?M*+{upami2ihyFC;C%-1qM_4i@2$ zKSF#-BD5)fW{)y30$$O6}Cap^tOrY@+)|6snI-fA44r=~@k z{#$r{_vU5c=`_OgB8Dyr<2@HW84S-kzh4TTPK4+BkY96nT7SPBJpEH6P5&i4w}zZA z!87^E+W(q9pS%n_=go}(u61ExeAIWnQC~g^5P@iYMNb67^R{D+!c%Do?A1|EhHQX? z^snE7L7J?=(+BXpcT#Nw`ZVUxFutqSTPrRWTxHX4O`pf*3xIQj3Y?BAa9TH5=&bHK zr;xjpKg}6zomclJ+WGKa&%8R6XBixt*8w#7XqyFr^n^p21R&Kp0c!G_Hyi0{!$19% ze_B&A1l!hIi38cV0skNfGR-B81I(Dr6aHV{Tm!E`!#q0e4Zls-1JCD_gfYbCKxV)vp1>(Gpp-nH1i*y3K0p3Xk$2A-rW8tAqVBJN% zg0JVOS?J?=+ojXCgktn}DN5x3t@Gp72V;|6jJRhmPMy!P`c$ z_fjbCNf|;aYq36hj2SdRDSL^2Vjm|d!NO)^<;lhRfSgHK+1bP4 z9k4E|mnlQO8N3juCE+QmRali87bPZbaJo3Epxo{*x7d@)T|aXYNX3valT8K^_yo`D zH;xLC_u5+cIL;-i8N{KPp!;a*VUS($St1%t-+nbD!_-uYaI&BdYm5Vr7EoZ3 zs(MhYa1K6j;c%wG$Q*{m6v=!(PCj^FH!?E!ehA<(&j^n<4h8(SI5sSQ<4z1B9)P_r zEn25P#u)TR-sqtRG0wfQ(Lsk4Ss$!_8R=;l#AW5gG!ba#9xKrW#RB30oG=a zKcp;SRSMG8Y`Cg6CO}tv`IP_6f+>G&obtWsyRdO$$b9NitMQ{HCQ~q7bv&rh)x=Qt z3(-4`$g6-()G_WLo6r~IKu+;%p$LK0+5)XqEhAD(qkW9rOZjBiX*IGGec;T3+TAHg z&`@P(6c5<#tl8XEzg1w%>c7{PtRvICYaX`yUa)=e5Mmp*od_OFho_aK-r*uG0jiEuu|8(oTxLW0Gh9iK&MGzYDSRm0FOis5y>`a)@IgR<^I z6q-Y#WLwT|I7bnhGM7`l7w|l30VkPvoB~7Ib@2J^z2UJb#SYU4&(_CJKhEOwBgv?` zfp6;qJv{Tb+JkEeoL_zLSft74SN**q-q1|5`rS3U?|f&pkx?LA{%vMNo!?hV(b(R{ zxGObAz6jfo)N`M5uE#$*^-!(f_fkFn++SIbgYmx`@Slh=rp_ud0)m&xeK9=WaaP}x za5DXZhWx&mgbBmUM4trkWZ_&Qdg(xw*zkT7bAJzhWaF0uS$VUHk!OyxMbv>Jm=3bp zcY^21Z=Pb;tq$q5L#pLsho+YTL8(8?hpa9X;r`V`WW_yrf(7qADZt|a2#4;x*A5EL znm{8U)>%DY7%*0&PF_Uadq2Pv?mh>>^Xt6`I0~L*3N!@t&Q?N0ZTyguFM_jza}=_H z)5=-Ec`NAjwWFZXk>)_iJXNP3y;q4cMfb_%xfWmb9J5K=@9_7>R_pTU^7l7hja183 zeUjD!=LA(}%5r#boZ{sh_!!Q<9DCOOo#LFM`Hy4do+fb{Ll&4+GVbw7E-qsCiMQ zSsc<0TsoJh#3D`HegXtQYMcd<1i>ucQiSN-d`!gguRoEo`9@|F@5o2tS)&R^YLPSb zcDOJ;ZeL~t|7LZ+0_JbnhH^iR1ke_)!9Q2?dBjF7Gw_vdu58=UI`{*Wm_Ag zZfgM-H&pp4DN4sGbx#Ra%3JqpejbXQw<-GY<2ji-8**5Es-gj}v=ZU0+!Hv--GOWv ze3IK34cX<0IGi~iXrsTfpN_Iq<6@Uq0Bn&4<#P-9+VlqAx=vnq-j9?NTtC!KWTAoiXlxB z$Xm?AcPvX6UrLL;HdO)k6ApRE#0;*HJnXLbOW!*ql6tsqlG9^((Oh?}-)Zqw3b{*6 zWE}<-cJr95RIbbWkAi)3*>D&OY$<}y2hRMcZDWJQbZ_>qU$nIfs9ntr< zY#8*nufhCxh1P$lzjUR)fvOhQ*F87Z;*96*jl%XIO>@y4U3*J*>FxZefsvjXkM)lk z`=5Hr{`K=sr1?fSGT&swd<*Y`{>~Wt3%la^CaUv|#`#Xf_TaQ5Xue(|JU;g4Waoahn%lBYR^z;-(ohj-{q3+>*qH_onJD}FAv-6V4UC19gWOy z6xMHJ?0?CQE1lmcnqN*M^Q#%6S?hw%e=pg-etvi$(s$lm?7?{>D8|_2t;Q*ytfVHn zo}rsxHdc-|_P?R>O6Moi{9+rKUktY2+t~gimuz1@KT*YBEaI=?UTANSadK@c8^Pb3 zSpP0#|3|l9>HNmh{D31u%`fp@Ex%0={WsD+9k1cy8jW^RkSyvld4O1aEZ0T;7IIf( z37P_hToBOmuvqWi#qJ~Po&Dz3!g#p6;2Yz6U;qyo0b<^c0y#RNSZBY69EeBOQP_J_ zt%R50B+OkCCgh?UHJwTdxoy#h(is<#G$lgF_0Uu^u%{`;4c8k#C|htfL`Jd+*;??x zX*6Tn+Vc6Vus(VdE6`3wD&I>Hg3&2Dv zheMWHK3mLys!e&wM83Rtg=R{4EeeS}$gkn|^8}1LzsrP`QVzJ&#s5dcCPqO{V9^+MI+n>HIA>#Ffp#QNdeDP=H z7q*4s?>DjX%MIv%>y^@fEYW{NBmAA{(DFkD(SIZDlfDxZr00Qu5OqU z_$>Kt@XaUsd4OFC&V!wkDT6dPxzc%V;b7>0nGqOaTSK1ba`}(-KQfMY=l7w{fBQ=| z#`=K8lXO7Y#+1$}KhkW*Rv+?fF>S-aTs@n?%=n~ycE9SEg^yqu#qQx2OpvNL?0Y1i zxFBsD{iiH={_M_%AXunEJKOL-F{UhGA*7cgzEBVY_QtvBALtu2GSZ~_3qp7;-Ub}q z%g@C^TA)YQK%PEoA=rMc_!oi~vhSc=(;(iX+J4U&)PC@NM&iK-EqMHZ`7GQs!3C!s zc+XPEe|@}vZ&f|8auUGhZh*@g{(aHoI=J-Tg0f}aMrb)iJZZG+jA)K~|8Es1pFqU) zbF3To;Cv4pcof?uy6?25-{*_oK3iw@q=Z!9qcZqZy&yGab(2?MjqMAmP z=C}Sq)%&91Nje8w3+EHre<}tW-fsLoZ?Sfc{lss`J<>q@t@#EVApHO`N?nhJ*jwCxw1AjQW@b8f4SkP12 zIfwt@EHlHFaE?F4n5;rUndn`l53CP{rU)OB!!>G7FU>Ok7xOWs(E%KWicnRE#Z96{_<+UP;hdu+JRfv(5oWQa?8;5gSCGX{l z=ZLh>*UCG2xst#J$7y9mX2boIDdHr%zJ`7ognoK6V&S*JS5phODZpK`o$6et5ohmU zd$W$->Q#FCiS@potZV7)j{73Dtvj4Yu_U)fD9&+8wwO&ZK~(zRu@4bQwUh;Re^!uQ=ilGDh_Ck`sJtlFi!?+ETr zX%@xZNuFXMnVD^2cde?yEPBfECgzg_BZE988Z9t#b~NjVbzHxwB{51#3^T1#WGv-K zQS5tZmHo^=47}-gyhKMyh5N&|48^Z3NZ`=x>nrpdzl;@2*L9P6N!X;V%Q=q`~HoD%*Z%J*_rvluTc*`-QR+K}at?G}f$!}qp3(iFTt#^d>n-`OI| zcWs%HXE)g1BVAaJ*A&4YuxJxNWdkJG-RX=HDeW#e{11yKXTJvb3j*=06l3orkb+zu z83QehKws#;9r8qr^EtlSTUax6EA53r`+b<}=Qul-1N@kMwfH92SX70|qW5nIc6h*i zD1$Kj9>CY7gYVlL8n{QKQf`uDJEi=HAHeZl+_Z#+tDQkEgHY#BKVixsID*2NopZ=z zEu`0H$S#l(c4@l<@)Rx2jWpSl%0xM{aO#~7uWf4rr~e}&gjT6%6Fn$wTFSPqnoqs) zz(%JJer|<%f$CWFsBT^xN^o9={G@ok1D#VUjCegGK?MyzoV5`}xcWQ`Vf(D9TI!8! zm6K>LVy+P}X$I(IQ)Kj(?F6KqnpSIsDz2lLZ*e5%UD%_niI49m1n6%g>gV@;a+Kl@ zTr^8JzoW&W@TN1&(nZ>{?CMe-fnK8!s9hQ!$FY5$V6Q> zCd=3Ch1#I=R2t7~X*eG8H|Wr()(_{qz5_WmbRNrgE!S;l8tTKW?emer`gfPKTK@!t zKl{aDit3@MDHANHe{oJx&%8KFt$cpd4&&5hkn~ssvFj-L|M%dtb0esE+=h0^TKGaZV z>--4n+53m;wKPAecu3Aaa&IxSG9`oGwTi}kQ#nu3D*qVwaH}9usCl53-cWSngjriTSrQ!BGnw0%-5^UqF3Z($fXWfA<9^(Hyo6N z62)+$_xHC;r%~k-EjiC7n;@9fL6LJDSaOt&5f*0Z@5_Wc@D;6`sF`k~GqvrJ*$oSDfczR4piOb>$< zrs+Q9%3k1^l?Zc@ja<31P~pmD!CaY7_Ltqb(R8j?qMzyQFQ4zF9j>|n=}a*BR3ve3 zwLi(Zl^h)Xtm2?l=Ab8GzPWJU`+(*>Z8|*RibPIJx3%}?tixu7Fv24lnXZ#`b8>9ao3Rz7l z$U~)Ix&=%U&+KVbjE=w}S5L?;+ApLSjaVJpY^F1Vodg`Tq)t|G9Bg_*hlA7us@W`9 zLjLkInaxN#t~*YUbi!ibogRyTfPy9ms9=mD7&F%g#3+IMVV(UZV9kP!3}iL_sTG+Q zu2HUY2DFIZ+2b*2{_n#le^+14^+Uh(HeIEhM`Slx-#`FmD~+#%kxhaZcTKEoIlN`S zo7c5i^v=%VLXBm;8*W-}Mg6K!b8Q)2mYfU$9v1WM4F9tX9b$?6djtX~7%f(7a7f{BFhTM+ zQ5q7<|8lAw63_oe^S?d#-x&Tkk;3M*U#*7CF>F-$h@R~YBkE$D@E{zK@9_}!PnBL^ zru{1d?tyShwGO=CL5F{}5^cgxLf`OJCN(fi>Xi%|Bb_|73aELUB{-R#Dkn;Mg&g+_ z-_SA_R?+DGN9F}zZ$Gts666BJviM2bH8|E2oKhpeLc(vdS=ZB$1$1T)R<{Dr>=N186(-$f$73s${iC|Xh1Q>{he7HRX_u1s7)(@jtrgBN zY=!J8jsr+G@}`ix07Y~QF{wgW0RGRaClKW`0e_)rt{giDx@#g`&%tXv1_pur$BZj{ zTp$mz`kisHBSm?3A^WK<-sFlEgTwtz$Y)ipxmF`*6oOI00daTXo+?LYc>6j`&MVut zg4p2S8BGTYU5(!pI-Y#glmx&7oCycnDLVce#D?Lk!2K6AG$8zE7xH<~gkU!8Nb5TP z==(9KaY+99hZ2c$v|-}n@odP?7c^3xh2R`!k$b^zqn!1=2B#N`3LW1c5%I_9SzSH9 zm2AJTg1r~~vh5d5F%9^IUU*Wq9$TCp*O?(~8*hG-6HTcY%gvZy2NWQ~`J!}Ty=YSG zYoou`nJ)&3-sd=J?BH)?#}La0C6g+pIF^azSjJ1aid8-ca-N%(o685q*Q5HX8Razp zOFcbq2Qf^kppWB~abWl#MXk^)mfK5Q~FX7=6F1LkRLuCDsLT&ByRWIRCOohO~o^ z0Q6#C6>c#Limju-^mjid(9!!R-U@PZpxGcN&wL(2PUbtLwxMLPCO^5J7R?7;$Zps` zwAtb-MUhAc+T@cc6g(~csH*h#EBPgK@ujrQeWF92p(0~B*sv9ZEMuEX5Fz5YJq}PF z<&5P*?my6V|16WRU=5-r4bjq5LCdlXsf2T01;hjdllM_pz}KS?hygukL=TLaUTRW- zI(r${a4R@er4rr-Z-9+#TZi21YTW=C@QGc%rw0&%)jvI!|KmixFGId3kv}FYR;CUs zFW_>BXaWA7s>x|@RN*avkB5x-c;vHC_<;KZp%+ILA9FcAY$PvUE+(AJ_(H)+8pp{7 zkQaQo%Qd`o)zOFMe=@JWNi-1uL-9iHcD@wMwau$HN5j>X1pX$x42A@Ey_xQbxa-3N z&uMaxicyb?v~@_YpUkUo4voBAhtJL6*(;jiIg&p+r?_WEM_3EX@bcqwtYd-SKM|$D zF@Xg$FyDFgVK}J2VH7i?BL&YLVeo4N{(3$PM3}p_UU(4#fAiI8xtTQP!1*U%cQ~*Z zZJAN|su>T;R|Zo|D}ybV!B>q}v6+lFyt8n@Gu{s=$)APcYI+EZk1`ca(_!+>WjfCp zrQW>y7C6FL_}nsDoiEM*0$h1s!S#n^zCa|xncznzbz}HhP<(YrDM#S;y#n!*`Mdk@ zMF`jLdCRPUJ2wN3CWB+C=3}}zci7)agm*FlAQ8a9+@Aq9GngowLG+5U6_n^?YTzba z{+y_qo3%XC`jgEipCEBvzyR$g@7;d)`6e^Q5=?g;Ue?_?++7TdN#-1)tX9RaX|z?<|*@copDSPeoG7ERVwYeVCiMB|xP3zxCx z43XMW7sD^A_mrdJ@%xYAd#CREogd@(-ROHh@L0T2+KRvXnq1VugWZHz*Wmjpy6<15 z@2l!m|F>eScL?e|qpSA>)hnfX=h90yHK%BgbZi$ZP(9Kf6yEPuZV?j`a?Z{`Xvv$<^MI=r{Zo4f<5gq7@rb&x)r zh<=tJsz-29o$;mvtP`rF=LEhN+xr(&zZiYLb3B?R`rC?KW5jN@eBNmfQ20_&A=|({ zTn6eIue;3E-#vQ;XP**WL1yRk@l6Li23PUWa4hqWrHDApGmsDG>sCB&@coSZYFh!D z*b2C+1{LrY=kBz;O@p#}LLh1O5lC(uRk8FG5g0jytxfh$Oz}V@={C z06frv?-A@JlP~HgaT;AKMROT@_Gt*@ow*FJ!oeV*jJZDsO`QMQHF*DHl}_0bHh&9C z0cIv&r`U3{d_#YL`@HJ?@rLCISgw2@4&~rE8YpkA|30E&`RQ%C?_qI5qgSAO$2Q&f zkqyiLtuJp1<;6keujtDy4a=X=m&3|cR1;J_RA1h_VR-_ULm`NMk&-_+ycv9h;Nt%> zYp^PI%9`-HnAZX8w^RqCIX8H-Dmmxk2iEWWMBFiUuBvR^idLPQ!NRhiXwil9#PgkDd~i!P$fDcxgU}{; zH9lFu@~+(4PM{9H3}JciKt->L-agwidzy-xnaJvUk8M>_TOsUVLAam z?_Lk^f!oTVXBa7(b z@(XYu`z@V6UeqepAJ1*@$AiWBEKmUYhu;J zYY-Bp4=c>&YEExYo2iZE`)!$F9(0_C-(jZ1qjaM821o)Z+$1Cf0wkcq4MCC+%?pzJ z)@QAKZXSTO^X>2V`~LB@4fpK5)_$$M_F8N2z4qQSQ^x33+&4zACvHZQmQ8Nb=8f+| zuUUQL=(YH+iS)Yo-Wa_!&OA6dbA6-GYh}%4^g8~ZSEARrtt7tA&aj#d>tMjzlYG^A z>EDd4hWu@c+D*$htu@4kE3y;Y9&qu3=v+2;cBm1Tb1zrDY%x{!xsRf%c)Q101fGGa zZv2&VBKMBJk;eba1N=zftO*w$jPWm*@;kI3v7F{bK5g3rvkS0`Z2MD3Em!A%jmD#!X2_obmPh(Jx1@;O~L-#-s0I z881_JrHr56il6mw7LL`skgTl|*$tY?h5WCvLJGFeNPO~_e6hi6ln&mB1JsPz*<02( zl=;w(dCnMm^aQ*^hLEv$*Xrx2OO41pJb(Of$I^RTGJlL}Vde(<{n0tXiCyZQ*>ZhhIm= zt|MAG)=!wL`OeBmd#HZV#OXrHBtix?E{BIBNV0TK9~IuYZtVV3+!wmgCG~s`AL=2U zi4KLm(&MLm>+!EC69wm8x?m#>eD+uf<8SyK@*Kh>SgVYyLkN1&1oSg{1#BLm9^?%ME`PMOt4_MzyNjx$* zIBIxzmle`{CNaT%aQyk64L72?cYpN}kT#Zb;^VEe&>t6?w>yfpj z$p@H^s&J3Dju(8lX0Y0`yGL-p$^g~H+T&?hY?~jaVtu1V|9fQJaTOb@DYngHnYIh;g4q$PV8^32XRYSD}Y*GYBX6 zG7Wh)#0rx(814KD=Jj{GbT1bV+4(%?hZGfVE$7sw@T!$7C{X54E2K=kudPS6rf~K6 zGL!dS6bRlC2;fGIrKGsFqTzrZR?vTwbf^kXq1ZoP!CoB86KD9&B&r!u`onweJ3nW7 z6)n%Mo^D3e_XRU=h?oBOIhIJ6=`J}evggfBEdS<(Q9jhB)&;iQfPOYo#^$Zu&pTgI zrqjR`?M~cxMC8Z*)D6kYqQ>*d_c(p$t7G^rKUVkZb^g-?|47{~{PF#K0j$-?TDl*u zW}$P-KrZ6yqJVWW@yi#PmA~;wsp*e==f}{K8~VV;d&(ZC3FItHCP&M1On3ty<-NNR zjNv~YhJOF5RvG;YChyv|=%oD*YX9|D>^e^1tQJUfmQDfA& z)wXBoq3Y>+D4Jcqs8`#rUx=NY;cG=so&;*KfOR zNzc*OgifY0ULQsD#ZFF@EZ!whdg$bH4C)6cSl>~;Xn?*3Vkg~#)JI5}p_3XT4FW_F z?(#)L^f^T3WU8pF&`B5LT%?Fgv6JbNe273(Lnr4m=n_B_ky^fJm_A1+`5UU5h@KWY zIhFB70Hla1<%>p5?>;U3E(Y%Q=U0-c^>FzMpcI8xQ7G0v1$RU}{}g?P+Hc8>m%s4D z7&I75q?Nz$9f>HV2zzE&%5%Wigxb@P=t;qOnzNirSqeU3q=(v{14hXhMzB6n{=&*} z;aDOw)UK7k@D#=EQvLZ8>CYkV(qw;b(_MaTTMG4C>|_SDHE2zvxYQ&e2p;X2qHWtt zFhVRy^HBwTvG!{icTp-KxT(h!dAI3l(T-HXCa9Ytrf`=KRtQ&lJRT zLYS(0c$=Oc?QoBwW)P~2P_qOzgAl0u=PRb!rf0-Ek_Dpf(~`VKh|!Mpv7D}(g`D?Zev+>hH{Vlvu4Mb`(lu2$;gSbIih zJZQauC_)gt=h4g2vr}U2Y1+1iOz;Hyl&D9jO8j&$GPCZKr(EDULRRpz?3=C5GjRm#G_6N1)B-vK=q|ra!^iN5g zVgRY9?>hRfrtf_c_Y~DQub^A^)|KthwsWhf5y99%n$yPZFRYpnxk@5SCq$M?j&P6%>FzSl;w@~?Ori>l-eLw#~n7;;}9zUAOUuoL<)C?%O zSOD&%tww!^Ze0xI(RAdEfOW};z3nz)Z@B~Ey-VynKefnq=arHrm=e5OD{^h?aMEH+_}3XL0yOgCGmEJpZlT0q^m$`hfSK9M{s;|1;H+F&{T#?QZqL7v=jo z-KxxIMMxvro-3w?JUW)5$4*VrXJ;QOye z_`$4^mOQ+Ymc51)I)R)iTk#pT&a74#}l}6K#~5^_dZ_ zEt_U_CM+3S@^46#3%&#o=&n8-h9cc~6c6Xb&bU2$$DJ6~O32)axtK?UuNQR(wPEuv z`gbrU{CeP-mVkG#)I)9HxAI6Rk+3se?MO|O7<6AlL^5puo!L%||C(%yf_dfYBx``^ z7jTW1-UNP3<6iSoFS$`7dQSJo%iboFVMog3^N@JTl-n^tox%+!MF6r}m29Ticcx1Z z3?-?q9?Vq_`;plQLL^oXvr)55nL5%N0X%^?53h4dA$>&rgfYx?W?qO?Dslr8D>X+z z;7<;Vv$8zHO9a-!zr=d-^wFdpPVKxgsO{N>kL5&_CrK9U*^N*tjXxTjPsRauQN-f} zUfvM&^YC1`7dY<`c~-a1pkM=ngF&t74H|$aW&n!@a#It9hEpljd7MH6^wCsD{|xVS zN}`Nc4*;R0o69TIN*a;ri5XG`eIBLHk`AEINN+kooxYEvn>vvXmG)!;xkeGHMO9Jn zcPQ~NYIK3E4r*d-a#*SkWx zaRj64ouCrV0)T{I7^V<&)uif8LG_w1tKJ*Q>b--6!;{FRD79zz6je=dwe&7wFwwMO zYFNnt0>>3=OFx#pPVqg+;>e8gor$QX{SGIhS;Ng#ST|~!P9^kFE`6w5v%`B093W)s z>f!Ur4j!YUkc+CB?rvOQgjPT9_jvP2bX1;Y~>8cHV&V}KSzXUB}O({&<=vMfV;`y*%M2V zfkpkVDW}sxMG zz_d?BMXQh$jZ)moI8zf{&9Xr9Ymqog2k|w0P9!L2KQ5bL5*a1j@Kni;MH|WBD2NVm zqX-}Y)%Y_`jx5TF$tdn5SJMY2AtH=e9MX=YCsTle7-uA?M0$c~T4$4@dn|Q~?tUdS z-w=tF9FDC1sAVSw$OE$p4IX|+R97L!AR!!a2JCrY9&kkAagHfu@kV@52)<{r>}N7a z|C4n;e0n?&yr)VNanZ~%ToB4Ggc4=4ixeWQB-{eF7)~%7Qisw&XM# zpQgmWl29!(9J#CT&w-V?nU59IzU46W8O6d9YEqh7Dg%WECWYD^Hy}<4XN<9b3A5K8 z;R`3vLr$KY%j?YQL?1WvH*>O9a!8nToRo(CYpM~!>Vp`h3=TWy16S(ppi;hX3t#xy z#^C06qEkwkX_|#ggbYc3dN0XJLrQ(d+~Zqz*6HfERp7kS?`OwXM65D5qRy+%7ZZg! zV2b2c$UIPjK4wN34Ant(f8qtZ7WQjEH%cIwqKYH(VJ zMmytvLT=|t>+`0#lVPC@Te3@9bRh!Rr(U_hV0oe2b6R3Jx^a$-NjCgS17 zW=-dNiYH2jls&@}mEki~x-7*jdqa(bxLKLjy%~f{))m}b?ww0GI>CxNe%U=by40}hu>oP)FGW-FPVN7| z&*lCU@psvu0t@-)64#}q)IR^cDJimIV7>H1tOxll3%|M@K@Ps!bq)fr+k@fUV2)vT zx|Zio<8T^>T@-d<=C6MVDa#I-_AA+E;Wzqe-n==5Jms&d<5P_p!;t#6zT^6WCLD2|HMcj}R8 zL~u@!*J5CeDqJkfVaKK2cT$GoS(2fwOl_6-5>{pCksILlWZLu6sTf{Gb5)!6ol}iS zDn;;vzt+;VM&v$X+$!O9JVpU8Lqb@(!L;`2u~TVO06Q+a;*^Rv;H+>qCobG%L>6Q? zOFje-V~?Zq&*-_RYVx8A3{7{z|Yi`D+jYTX-?u)Tt}hF^>oX4d&~fUfZFI>G&2-7N2B}xc<_a4=?dm4Ffa3(;`;@GuDWNILiW=3ufScd}Ek$`7+AUAG!VrFg&A)!FSv)i=l z#KTkXkDIyeM*krL=U}<30=cgTGIvv#xJ)ZYTVKj<*)4ir+_ZHXnoM$|zQf=Z$^qBC zwmZ|bcxO^u-^)9k!g93kGv52Xn{PEgW-GDkX}K@Qna) zm|dMgt(9!());-$OuU3F#5S6Uwl!(j-js$>u3hU+1w#eAb*ug za1oO+F-jlMW#M;8#NE6oa%cVGi2LxO$jk;4r!#tV(4G`VSZH6b=U$-h(5*VAeVx%? zzu0oe74ePSy+&sJ5-Tf?BYqKP5Hs=$MwF3<4lEKS4Zb3>?a7vLo~oxw-&scu?;fzS z?%kRr4Cl)vuHQOG0s+^)fVT-8=Ne?r|5znh*zop&tpnaS0`}e6;8**RY=7p-+ zpP3u;@s^0;sm9(QdLVA}HyD0a4RgP4 zXMKTKn|&MBZ<)ybp1r0$`v%j_N~K6tYl;17jd(npm^i^46idppnc0f7g_r+)3hw}n zV`g8Ho9XK&_wQq8AMtbQcz)KsJ*zfTaRXd1u?M%L^lOS@#A>|f5}8*^+*?CjOn)

+KN2QAhk}qY*V0O3YmiwrZ?ncDjCP8K$ zip`#glp{zd6cQ9;ul7UMbV2QjC^dkUp2daXJ_+3pZ0g0HpuLm~z17`rcE}=2{ZLHH z!?B~V@W|GJaaoaIDwZH?4AVM|%(d%nv5Kqa%0Se)RLWE37EOG102(*2R=4FXBKz4W zoZlmyYa)AqO&h~%;fIA!vrEy5#9%*hq^nQyg|>b_pG%q>z-jQv%8WoHK$`j6*#WyK z+qAzyzmJhXYG*wHg^)>uxP?PdTclFzbL≈M??`=a>e>oaBfupNiRMdQWqzHkAsa z?y0FTJx1Kx^WUBdUhmOkCuyv8%s0FT3~k|oLeD#TcyH-^GxAw8@<}6-9f({TusQ?Q z8N<^U@EkUC&l>cvbu^HBO7Cy*Rk7Dg?vT-cIFL&{dqW_TpnqoM9iWWU3Ox-0`x_&s z>;MJqc^Luw&P*C7Gj!LWQQv8VcDWYYGyjB)36R>u6Q|iTjZn3Vx-{!rXDr-wgEJP~ z*ULD-c(|%)zZOFQw%bftiUIG@RsX1q;NDrejr3NYGcVIXzj$|e7jkr5>SF`Rq zOsdq3+z?=k)u1u$sWx+u=p;+5y`)+7ei{z-)NqsJnx&a!dRW(+k~ zy$aXw)z%+iE51Rm<(*PC8kM9a`-6J_5Dr`y8j-t<$Q{I#DUh;)E9cGzXcZ80iFAoExMB3G$&F-i3% z@dWfY(LXbr4g1b%rt63v@tTnthBX|p>I_elPF#+b)At@2<&cI6qrWMTyGOTNMrM;~ zeU^-dS)#Htf)4dl&bq~-e^<~x>-XXCahgk3KRU62@ z+&Z#6nQOckNqyF`)f>wkB!J<)z$pe0Pcbkj?{dSO?1iF*i^q$xf{X*&`q}Iu6LUtp z={;#^_q@($2Y-_amYriprWujjjmQ_t6bV=d3{SV=88LIu8}O7P)p}6xZ#HxLjQ-{T z^|^JsnORF5?((TKiF38}S(w)3y^aS0M^wiGgQgi@kV`Ri<$9xZf$lmIus@v|u(Q(0 zpkX~~&r5amhJDjM>X=kd)YG6iLWDw6L4JKoN|Gg9AAluX6Ir79nU@2u?tu4n=|f73 zFD4a8!eG}T`_8E9I>Xhir)@=j0+DQD2GcsMLkNw2Af)544pGrdXYd-aH`(9_Be)np78dirRHtx&^}CfuTcvT?f6jL^I-mAXc!nU$x( zjO27?2Pj(Q|8c4E;uvq*J7d**>!cMjY&~o`7}H@bsFwBc|twVKo~4jYjT%k)qUC z?&!=OvcC#+i3jOMoz?OQJ@R4Qx=Xh(g}tAqmoyn&l+p2bGLz`|$*}zDeSA`{ zV?i$xZEa^S%~x6&>#gKuX{*ro6)x4P>Xv)|mqfBolOc4H452f`0dHckTNvpEksO1) z8iPG|C}}+ga;q0x{^c2gmVf_hM`2eqvIZitNM8j%2F>q8#N^RyLHc?o_j*~^d!ZCo_m05OV;)x>&~-! z<{(u!VTATjwr^8C-@Am~sN$)<%xDV1%$w(oO?HTJc>03%5}H;WrZO9;KbomO4kf2m zWQx+vN|~ZG`EaaJ8idpzO(}ZjK5Eus7!QW`XxVnk6lHci`rhUD5l!-k($|et@gf{M z1ZgkeREHjUs7SX+fhE4g{_sCehnRc@w}j}grT{x~Pk|XBOX(3BsqlgqlQeSgEYs>W zBlqN(k&-zyO3lc(^U1*B!6HK)=aV`=q3B%DS2JG=Yw0}ti^qthlI%pq)V!F;kVliF zj&xGl>7=VIWl#Q8Wa~jaG(;1de6{XV`h7l$6BCjPTg$FX@@xK#_H)7KM^ZlRidi zPNMjm)|%WUfTzi&3z+(i!OJ0-1@5N#B*SaQ!7DJB1wKVvzm?(V$H6z4G!Ykgx?KK( zN5;nBo7jVk;ZsY$tB(z zS@e{}P>bb{8L0$j=VCd(Rn4UnA*m?)LCWxg7T(PRzfwE_#Ya)5yU7fz)Zk^pk+CiJ zo9=|hwPJY}NFnNtT~<9QU0hl@U{K$+y8@Ywq_hLj=SLLLp^W$x!_mpbaAnF)Y! z1Un+CEGWHIPsvR+zs}ti$ZQK(`yJo}5Ht&catDpvAv3c+fO!Y2U1W9&_?>~wqXBD| zkvm|}RDz5PSt&3x_XcnkIky@whjh_hx-~WF-1uAuSf3f5*j&f1Q6wyGjL#G>H-mw(L8O*SXxLotdx?vR z6IsN|pPz9WKVSOTnXHtZUnQPQ%=+8~oZna{;I6kB93rfpKL>d5vkd;Gfa3ygqo194 zfGBYkU{qX9i2gYBx2IR1;qsPAdEZBQxEdqzxK!uw1?&^>X4Li>fEN>Zyne>LRX_VuxFw?d-!2{F|r>LVzc4ZNIW!<>}O3 z`@EL>V1|~rv4ZTJf1gGl1-LmQFT92Gs9Yf!+5B|pLA*aL5wqxXjeO?(dlu(ixmLnN zBQmoq%O#Zm?^zV6kic9)+#n$`Z!0&+ClRmmW%(@9BTI_)+QPM9SE`8Yq*QW!A}`4? zYFpN#mn`*PH;jsKkfrh^D?ozU!c7YLvijYku;PNXl0IeWeaRa7 zl=bhQtd*h@5-OKaw}dJrB>OWz*&rcaCyo4M6B{C1g`hUp?zk&KirT^*ss>U1wRxSE zI;gOM;_3tzj$e|oi;db#9Mhs=fVqjH3ogm`7W&Rn-&ivE=xpLE(2(Q0qSU>cF7fVd z99_(x;i&xN%TJN~tdyVbLHQZtpA_|TQ94FK*%HbTU%8EODgHZasjRh}$t&#_1r+5I zfF;azNFfRHC7XN+6-cN+LPZiPlF%{Gg@&*=VTHz&B)h=zXrN6Cfx&5uX=a^bqP3d-u(w}g7fJ6>lGQ!)gW9T-x4Fi95dED&E70XYyrkuA zRxR4hUFXxf+8Noeb?qkXw;e~dcY!I?Is>dX9js$co^fe>7a^{cioGqx!cDpB68)XW z>Lwg)yDq>>Q;<$8q*wAK442z3_5^|NP8Yi)+6CM(TbX)3I9gKc+oe2}z6)M}UEF#b z5-3=xb#+ix3RIG}Reh;4ls~)myoE2geZ0~wS-lOX6sd;bz7m!0f;VB;TTIzaD(`cY zch2Oz1>cVZek34#2nJ*e2)s_>7lrb#@Hj0*V!z}~S#T+ZQjyKss*hfQG~u$hNRqC% z1-np{!|`0kkd(2-Ny!q1Ykhf2B-k>f5FI#t`AXm&y|;TznS+E4t)h7J_RF;V4hFfuB|D01wOpRU73iMQFW9a8z>SW+K}{ z#eVR9D?v)EqqU;WZD>au7#SXnPIlfr?rXGcHZ(qOslD zpR6C;OcQu zeV9lqnmKu%d?tzEl8-5O($$Ata6%(l?1OVMoT)S54@&=ZQkI>Ew7d_&1xF7hvdLWD z1hhD(2Xk?9ZFS##A?E6l328|j-$j6tvIy@2ys}9|b7jS)B#o=&yA_MOo7aja8&a@W zDg>Wob(JG$sCAW_hQ-^gLgKpmx#O0};BV~(L$WX1X6Rt>@CdNpLyl{ucPSTDa0!O& z))eXAi;(xK3pcNn!ndweB_ETKC#7_;TjDDP0&8t0mRhUyO|Q}-Br{a~4qdN}l{%o{NxvXz|1>69)w{=(z4WOM$I5`=^!iKz^dvGg^Wd)RQyiqB!Dr`5#bb$m)>M*{CrR4p9o@50BIShQ1lkW)d93^kr9&%I= ziRmFn^Fo?sUrZ{XuHBAGU_a9?NM=b3NePXt3f|?&RvNp=F-1z|ypX0q)u1Xx+E1fp zk{&_sjL|Jh#K~f_qg1+wm9BuDI98)%7MGHm;k+mcs0Y-cTNOF1JULt8z___cNq2a; zcfl{TWNU>|LeMN-7q~JaTv{s~5b|h!C9XJ!B{z7r4ML`^+3#?vPwQ-=eymVbZbYgh zD48=|s~GgGk|3;;`W?M;Q4BX#L=#}0PK8)85ZBW`heh*TbOubJIODfL>9h?enVki- zDi6cXIcDHz!NsHyR%gFb$YqxKToWcit56VW|dR`(yvLRwp5^FoJwIKgF}I>XGO4z7U9lnqf~A1 z=&dY}BF>6ycxyZCDb)!HN1Y~=no*qY@V)RgDDG35q+5hGsV)(7EYZ&-`5U2ll4Zpb zgo0@!Dcklo=R$3hTp@X-;fI(FwANaPd4Zh@j+qs$uPZaTP8oKvox8xKtp}AtDcKW* z0jaf7W7^)J?|0NZiHg7!VGoUXoM9m+}JdgYqbuiDPJopab9{ehfNb5ITTV zE6y3MzJRSdG8R_X&&Mu45|ieE*}M7(kS_uG5C~1+Pp2pc%??sG%gW&FYh(W43!O#6 zDw$W{gr!W!sK%JRydra-))qCJsZ!G-nwoUs{L(@CeS^40rXo0^c5%w`!&#_#cmDW! z$iM&+cY61hyHZ>jGC^Mwwba0nAmQI?;wAM*cwwjGiKiZ694jp8T&QmyBD#w-D zr&2IjW_Y%m;Q_o~?S9re^S|{mnZ$)R4JT)c2&RlA=Z^?(8BR_v5zHP*&O8x}4kxFj z2alFchZ?wBueWRtz)Hm9;LVcq(E7dpZzeatd5^L2rDp0PzQG*Kg zU8L_6|L&A_J$!$W7Je8G6yX=KD5r&sDB#z&CwOx%`1z_-Gw-anJ*uy}z}twKaXmIL z|Ks>J^7a_(hViAnu+%NPO;%gtnK$_!A>2Fo#G8tFm_luO?94o(4i1PCe<3c`HJ`~P zTRfq57q<6tKURO5JdhDtU2I^}x`#Il=>Ks&w>jXc(>*=D-%jHVlr-I1T&!F*?5tt7 z>n&amGeXg6@QTB?ZdLM+edjgoYiFC+6vIVMmH|qxH$sEc2(Y)8kJjU8B9uof+!9GP zD-%Oz9GLnvYI&c~u7wlVgV~hiE7=M^J~0jO*;yl@y=UxQ?Wr`yzE% zrF7U64ap8QzgSGo`~*61T!)?=*O|MO2MtV=qe3|7LiV76O2g5wvs&KGq{4P<*Fv)f zlbyO-;g3syrL^D3mc%}!2_`%EgQm_=sgWj_?Cjs0no{N?B^ZNJBh5l|{8tKuP1z`G z42rq;rkY9~Q+`8-zyRV{OHAMe6X=oe7Mr+&d?a8|ht;r$?0|0do8oSPJ-b@*vd}Fr zWC3_$hqEcA=Mp>Q3Wu!Qjopf|ca?kxc!%Vc*LjEJe)`#o-ONPZbr;~~B3#gsA8Ib|q;Ku#q`maB^G@Qi#QB-nT$y+ujJ>dUoDYO)Z|>%86?>!FQ-Kb~V@EbS z0pmhfe&8s)hKpK2BB2C1gv&X+aE@thhGwN`;*OGbf>bc%d2Az4EN>?eq>~^UB#${1 z{oouT_DPD{#BqNqSgmm3R?Ea^3HY*_?;EzCgt)b__>%0?3I-0xzpe&=Zr`16TC#6o zRjuHktt)w7xON!~Q|#MtvbHx&%a%2iw%w*xx#NP`&9)%Wo2 zF_Bek#JGl0O|z8o{MNXM!_ z?^6C~lM?lOVM?5eBW6MFbYYQhJ2EEOAa0Enp2Zl?vmVW}bGTZ4iqt=ny`^v+<18q`W`z)Z z!3tDwZjhHTy3RV(9UHlPpM@%`Z8sJjhNTCw8^Y8J>zwOfifpdjB7HspeNU`aL@d+7 zOjNau{grqt%C|%A$_}>+e-+`{LHbO5avyJD3FlS{yH~Yy!ahlqa0|*>L0Ks%MGgv1 zk0nm-WnN~EjWEaJRviAR0)rd&qHNv1CmZg-hP7YLeCdFD+t4?Ra&I;tRI1iqjAm8` zyeG;l=U!5WmoDIxkrnr4Mg!gh>}uN2=P$Z_xfF8{-@5mp-{Q30IKWnNex z*VZ2(A+xp$Ii}D|;u^k!i!|9hJ(ifS^OKKR&}16Bj$Us>G`*I$*Kpf3d(!n9o;Y*c zKEu;!TBI|3;5YtTzN0z!sL|gT$lYaFAM)+M1DQtVe$%?e$KH{wFzuOB4g1b?!_Jy& zy5f4M3Ykx#zD^~Em<4{IUz{=Sq~B|WFTfw~!yDIT2w%XpcmZ@+akk25Nw&%eCddcL zNzaz779imQ2!rArC$AhQubdB@0OGPBLJ-J0pDHzS#W$gF^Mz_d;no@V&S z<4{hE;i++s-*g-7MXXycJ@bNLWy4#N8Q#`~?-R__#P7t|eD0`PcsDbyZDYshD~!;8X6cOhPO|9DQ0*ZFeK==U$qM z83{_2$A`z(l#p?9?A$l&@;)n3%Pm*Uc#@3}RLfDwcuq3oIakWK=>0P;7|XbOcx<^1 z8P82-Jof`}S)N@qmYM83$?6<3)03I$l39sb1%egLt8~g1B?GkPeTrB%Rzx=Y^*ltg z*eYH-i7YA>Z*pdVI{1-`L-{ywa`;Md_t)XbH)ap^(jwDJpl0UmVm@yX+LdjDc0<1K zHF4(b0xb06a-I4wQPDWI`zV@&C1mVpKAMCd`jSlZc%k}h-JFBg+0OR^%*G-)*4xgh zWlwcPa`NGi;%=G573jJ*u7w|EccqOR#F4LpA9I>1r|9AC9oeWWw=%U3#SVZ4eBbn) zf`*fl^*9v({LC6Wsu9NykCJoZq$b23xXK@0{plOa{ z@Zi`huM#w9>u<#U5RpP$Q7+2_gL-a1@7hnt>x&SR~Msiv(;_l9ag)iv+nwXn+?9^offE9|?ZoMS>e;rVh=t zJewy9AAFks6}3pfs%SY>(O3U~MS`!WMFQ46%b|O|`T-XSzA|o+fR&98%I2#daFO6E zYLQ?;LFrvY$l(+{^1WgcS0@beDnSnxcXCH`>!9g5U_x|Kuk|bDuu0l_%I%kp9 zc5lRel=KwYa9M38)Kkx8dlf5xO|7Y_!UMt$Ae8yeU|E>8jS`P{hjXCLua}t+>aiPpJ12OM$<0GEx(4&j=TLscPUqH75)yh{Lfstt*&z-KnMYUwz`^_;JN|B4;ZWI;pw#`WoN#TSo{vHh6Q; znqkFa9ciJHH&2XSl8X4Lm&dyif6e9b>4?`ZkIz8-hdKVV-|CUbX^6ZbiA<5P#7#ud zY=T{~vr?>q#`f3%VJ;jo$*Sio{I%_ouj4Jg!!!){T6Mv7Czmb_)+d70LRXz?TH_gO z&^pXz*adHmj#@F?q<$G_fdVD6p8%Dhn90y2TyT{*Ch#=)YAi z7?%Y1vBhxJ^PXsmD;|suX!c4M2@F-dz2gVC$R^33q?)PYDU+g0L96JJ?j0yQhtu}Q zc##1~@B%`gY5xtburY1D*s!WSqn+8#{W+mns_8va`lPn~TD`tYuRre#MO~yrER>z5 zy%vA0Vd>J8l-N+VJ@dZ2I5nhxD7QLv)|E@&q{HTiPNkAS)3!H|=J_){^RO>8P*C5bxG>166liJ`Ubgb+dPOB*%fOM7XAd4{DQ<-yugK zj$y?(`g_I!GP?3Oh<1z{2Q?jICa16&ke@ve|YQ*z=yFW(pw8WEu=wj6pzxR10KxGD4`u zjgW|%F%s!haBF+=y@`B1be1TUa9h23`?c+QYS1c9xAy1t(ytrm_R3oIh%Fplhek1d zwe3YClqfZLTM;EFO2uXNow_~c&a^vHg11#+XyEoRF13;qN48?r_Z4L1P}Lc3wQ1oK zT1rZy#)AH2aB45H-D?3B2Ea*b=#i=wFAxvJA z{l%>-N$25CdU0{z9$wI;!V4tqcfx)NFLc5SC9FGPUBcWL5-gol`8^}Zr&#GHSw1B~ zwJs7fFs4JJ1!bpnE4?6}?A)3RPqgz8`kN&R-kPZmuQ(>htn&sbkaJvJflwP>qvG|( z1cl~|0oFPIE1hKLmJ=*|wjAg6sx>Eg4pDxBpNorZ-E@N0cMhmGw%bFzB7%3v2|o9V zdmfJCV1t~3h*YkWWSfrj&j#de?WJUKNfteUk%gmQYXz}_F?zB6!u8l(!SPisVm+*- z$k+ahcNbsTASu>J)+-sk>Sc*sA(6}ErTpZuGmoF_G7$g-z}!ptm9+;M>G)z zo`H+z;%fK&R>lwH)!=ZbXCz>CqXU^1TN99Fo&ockaN?NUNRO4j0fQ!GIE99Z-36Q{ zIwnsEP$05(11GJN9OUGCHCviA?Gs>RYlQ$!&rt&MTKL8y7Ms1e1d9tr9QQA=6?gMd zle5CfXUo=6WnqQme6w23H^bVtSg*NaYSrU)hqx z7dW&fzlD0nVVZPEcPK)Rf7n7DEiHA;5`r$$vl+VGsm7s-6Wx zW=f>rQK|4Y3El?5!#z01o#zqeCeaCRgOp+3QCT1f`6p6YF}{HO@deBkyb39xJXye8 zRe+Ooc{1mGNs%u(ujPbWAq|rSKu{(Nm?P+GnJq6+`dKmxB)%aqu8cLwGGEY2#0uNAa{Z})2gsR)wL)r#x!|9_%BF z=p&}X;o%CwDQBEmz|%n@63ngB9PkbsV-XwocT{1V8M%7`)>%SeVQn!qKw~Acam8_g zdWH?F-^kr%9t^2s$x0X)cqxL%V?TGddB^5r{;I|0 zc}XNN5+T(kL0gg_UDCgt1ihRD6$ofk60|7^nk%3UNzeuXQCtPHxGgMGmCr4$l~i*i zRe3U1c{0OU0$Q5{txbZurSfZ%pfw8Q>A_B?6enHZVZ`2^ujA<%TL$h)BlZ>|jLbK% z_C@O7v|eFaS;zI*oAZtOcL*BDJV!FFj``@A!O`c;%r_0(DcWejFsSbgSWTuq^9yEX zD-J)$PR=(n_Zsz`CXOh-jzzWDnfZatlWhNnEhY_Hbns4l7$R}W^}(`u8|T3D1Haqn0W*G z=jD=^7_Nnp@65WSr>CA*qwh$O|JwENE-n16snAVZc}8GwEb2mi`V&9+E%LXS;P&q@ zZQ`q0$p)oKUVR4*4Vw5A@x-azRs)`3v93mB(ep&^=C=9j%?MDz?# z0}YUj&K4>OGdDc8O+D-TFMjXap7PQ6ZkHRK@uW58fLiz{PTiMnMhR0!-n~5~7-F7J z#cqUkf+mN8_V++rvKgNA=Uip~Nyznt>`W?gH50E~3iAI1a<1h0&x|I^nt}}+8me0O zZ-F-S57KMLcy2%CwwK|vR>5ze(R?kKeBZ6LPRrBzWvWhvPT9$l+X?t$5e0+mHX+^U zE%dR<`6zWho_0Rg@P}GrGVM*v_)*a*E~=<$osky23y+*f{y{d_SnS(V+IzXdb9oq@ zR%IqrU|lNlneizi8@T|hUAJG1r@w!DfJs#Y8rQMG`8ec!G&>(H{GqSgjdkM**NOS5 z2Ce^*meNr?;;cs}_9L82;Y18yAMK{E?f6=bxL4@Qsc*+(nu8{OIX-48Vk}^PRW9UJ z^Bih)z&b(t61Nkv65ea#KEO7JvlB@Hi>)*6AT?S3(!c(F7Lk}mtCFHuk+9=Ms}iP@ zCG7tzqIIG$Eq)(is!TzaqSjWHsrK|G|G<8iPy8J&nQ&xk%LJJke;=7z_kXxdogg=9 z@gX?_8BJZ4}5@qh6(W>V4vZ?x!=|-`wTv9yCoZLro9!0$*X8nz$>9pn_>SJTRUnSuIsXGIJTrNsGYbjb#I_QGSz4G>Y-Y<--`OywU6rtzboq7afOY~ zUM^wUUr;5L9!$-wmw(+UrCvEs7e)zlY@xG%a@Uo4=$XwbrcB7N~! z*!0o1S2f@n4f{nHke)s(9t7h#-4OR%)lN^G657XxZl*9%FZ%B2D1jY_kUj~m9)l9m ziIT3p1Y@%Aeit24iQz%j^6bUl#=Fiw5W~WU2Ru1@GM@+JV-n8}wZ>vNX=JlKQW^6FjhW@X))M@t^V8S=p2#3}WOR#yi1P4evt4e4mhPw3${f(*FpF8m5o zwLPdkiPwZa-}@^ZBgJ{t<#=Eb4|nprF#O&upO~!+YJ>ZO@@@}5L5efjxKre9yc+k| z0R4kk=T@D`#(7md=vP9)<-m9hXJ!=wqaqr2q2Q$`JmSN-KXuY2&TV*rtJR6Dr^vy< zpw?Nd9%I(R-{$>H-Y?3FYp>NkwwJe<2ea)gBaehiuzoN%7V2>2@^?r7k<7Yr+e?2~ zDrXbw4@>22RsCV9oDD6NYSPv@Z!%i7!TQnM+EAA(cL3jA{cV}GhP{3^KX-0bA+0qy zroYDR8~UsZzPx?BbLh8>f7ZV1s+|exd1Y=lw$WncJ~nh%ut(jF{;p zc*{fW6*A1~(!!s_QKGwlfx%wE?+Cfo>pFOk$9ZFicLh(XUNo)qrZpOL6-HGD(eU0v znNv))B0hcYRurYr8bv1htVmi z?T%8X`8@mm+HM^JBqJ;*hsN^xzamZ%nDJTWV->Iz9tv7R{9 zm1-%KjY?&IUrh}!&9uu#xnB#oL%R~!|NJ8C<;ONaM0D+h<5=SFUb;9cj}+|uF~UEi zuntpPh5wS_BM7f@Q+O5NG_U6@&nI5oK*4ppVH=H`lP1i6pvmK*VYb<`f5BGSl1jG9 z>gZ=@%2dj!_(ad>ID4!{n{Xa&>&j;Knd&vHH}ue(-MSrOXQy@N>hjoODyNz4usO}} zhpt&Y)36`RCM5Qj@>XjvcwH>!ru=OoOTY*Ke5Zld#W`ttHj2P;_h#YDW?W6bFh}3< z1@)h4SLHYJTpwmU3bGc?p}&0kn+qrBX3VdDc2O?&Oy1PFuJpJAr=(!F1w!`SMTt6+xGJ`5H57#3%>H1={jFK zK9Ey(repuecz-zMPmGPNctx{_=|VpPPlgLxc)KBBZ$$a*3q|RzTIC6K znIF_+6a1iTH$N5dzvBt@>nnRfb^e4s6{z;zEo>odndAv&SRQ_YZ6x8Xk=1L)QHoAg|;2bU+LDt;B9OA^7aMOiUwRGhPS%x zbnv#VFd#`Rr?CM^b|01K-PiBS z8^veR?kjKy$Lz()_FuHDGk6;h9GxcgL$K-$t6#S+sNrNB`7y2R9xJb&ca z?I|L{d$F1m{V~WHs59(etYq78AYc*eJ*hZrd7XOZbmBZ`FeZE!r*`KBqV!j7B3a4w zEkC6(k{8%be>L=1r`GFLy4JCDrd_$|2-fSr6~&7Dx2i{P3E2N83S_URk79Ond=28i z3Jq7gi*K!6_pQ3oTRO9do<=B|8t^7cpYmZwtQ1Pjh0V}3?su~O>d5TtZz0b4_~zfS}Sg6k*sfniCf?0iTgcm;%G?* zuIr%R>S6o#0RQ@3{hl7NZe4?}I$ybOG(i0k3WlDoqnciDv;DEbbJ z*7&#6V#sQqp>xRB1f&dAoQ2v;947ftl&q>aa-VHxDuuj&k6Ijfw*RN z88TeX*NNdZ_Qw-)Q>gY$JQu$lk6Ysc&ej9C-wijnJ>G(RaIx7pacyenadlPQ<9MLF z6SoL|6Sq#TIO@}C_Ll>AJSe$-lV8flEwYc{J?rJTR;~2}f-T7_N{>5nosE92=4`n- zj<);_py5nU2eo`(PV{3(B}dt7$41 zGCkfNL~XTJzIRm}5PZBvWdum)F-}5|w~5RvvbCBwRhy`ZXXT6$zW~3CNVdF@D)y#i zRfv{?kcDWTww&to*fD7}lbexCj>sj8(lIM2i)GZdZ??-3x@DyLR|pozGkiavD=x8@ zF@v-oR0LYFPr8zkOK{!bV|CI`L_9VF)HCI{TG7w9IHQ=y0qSgbF1CFxk+!zQ=ZutH_a^ZCXuZdC*}ea1&P!4F(TzLd4-R>-T^k1UnMbYBfES?lG!z z3Bnhs^d9xyEIR42xST;-t`0XYBYm}^UUb$n%5MeY<74s5a8fb5+LV_#QME3PpnIxcP7@O2h?`Qg#C^hIh%xYOCwK?UcMhvi}^S1)#urE z=v)S_sQQPMJcXO(iPnb3UY-u&sbjf*D6~7{YY*?Lh}<&^&*MCwqMp`~hjk3=k{)>^ zTMu2-wCA_*dxuj!4SIhs%^_2tS*(YnW%m*=yVStT#*FnQYbiNj>AaTW-BmURP80WF zY9A&uWl!m>K)6p^{}v&2G|)d4rv2$ZF}wA_?ZOGm2WDrJ8aE<8n4J;83wWG{lTiEqBqGkRI&d``VSH@RLQF6#9_Q_94O9iLFK*AjQ}Ikc~% zVoy-TcJo=+qnI?-CeFX!Gum;MPr6P|6(;-L^UP$ggUQaK?*Ef;CcUSmy8N~L6lDD# zcqs8A-@L=b$d+)9A|1}$)L7i4t_mRJ#=P3&l?QI@?#xo{7k^z*`S0F>B~+I zcW0E&(<7_1b$jM}oI$vu>;}{NiyYH>bPhThyeQWwRY*HNL3QjrF6$Yc?dW^I8wGYA z(!-;rCCN$`dfviSN8ow9;Y1T&yn*<6vTqhe=CuPi$5{`EJd%+Z#Z5?f*}AQ)%ZLV+i8L7*)PGW;j@K!sch`7&MW$jX?o#=ZsV?MDgoYgF&n(mRaEb2)J1iQM5EZKg4H%80lL zJ6d|99!W`3$yw}KE1l4JdV)N#;&8Yf8^~|yDBbGQCsp9Xop*x1FN&OS;yb!#~dtB1~Ljh|KwSSggo6dx{%IZK0-D)1KCu$V#CW261Kos4+ zsosp-K_Y?Zu~gPK>~CZQB2Uv`=_|ck=s{HYg4F4h9!W!$J_T|_0(H1=A(CcVIgUu1 zb=f)F*JgngYxrg0YT{U8kHe9|-KO<1n)v9o3vg{sinSnHw?1ryrn(a6X!gRnb#A%r z{r5kte;>ZAe-~e&f9K5xmwLduhD8LA+>-bTzc2qe7nU6##=`!%5#sbc`u!}wbAqz#kyx!=@$z<07*c$zkNb)DtJ-E-3_wN!ugy? zG1Yf2S)#ttLp)Ym$~uiu4GEuTvY)B~(7a;wm`%51;DQDgOVMgNkUap_+(AbFr6LytgC=pi(bq5^1PUO9hwm zjFeLP5#27%Kq=!AIC7$OeB{LO>vRq2pHfmeR6t_0>LjA%W$kTnq_4LA$yD{QY+Ua@ z5<02r-g;fTXP+J#`jGa5mw+_AmUD%EZCln1|Ls{Co`&^j)|uWI&C9(sm5*ubN5Jto zWPRVLk+(l!FYY$-_8GAbH?$sHe#saxLPIm2{w|KWk;$OpES9T>#DDJst;BsTgn007 zt;>HMjU2oge3Hz&x@U0ZM9grts3VJ=YN28E5%>5&>i7WeF2-BYacx^5j#phtL(irk#2a*Oz8bjO+ z8B9?SHz00?Okqfy1DOKI6o#ZTWRC+$2PB;#QyKCv4rD4IQtJ$c{FMXA03?GUnFM+I z`y88z*i4R{hS+a$>@>tq2tFGhkhs7Qve(c+^XX8!l zh$%~N)?Fjc`OIxMkY()h#2<^k=t`mf>MuJ-yj-yqI-n28H?5tYQZ}RWI*}`D=S@v1 z&E#zoZ+BVGME(8%Mo7X{4PT`-nWnwWm2lU{keg51M+SAQloZ)pWi!5}TXiHql>Wgf zEqLYDCca5BjIbQAGcNZ|#J3-MZK-`vPH5;xDp{rI{){Sd2L_e) zs$28h1+UpfIlj=qjoQL$?bVqx{Pv>P{PqHh8obe`EsT5P&%Vy*8pCuc!uT$| zmuTO4ZLs3GC6xbNaN;+-@lxvGSKU+nw*H!}bC%k|7)52J`|U4NF$TpAp_skix@Y&O zV$^*XQQ4omygch|ecib@eXb`{QN8CLmggpIe)lqmhT~c-FHoW7E!~7`owdAgMYX)| z#I?M#b}jGOZrv^<+mau*)8!dz?5ogEEkAn(3!pYUkG}HR9^ogk6QS37M)&Tj5dHG> zYvc9H!yTtryHdu7nW2DO$tB%#>fPId3sW~`8$)!n-mT{IWhd;r10}=CYJZ~r0}o5HK~1* z!VAAWG5m$GFwREdi1HAgCbhqmg{^)Xg7)W6ugOYzxZ~WXsVB$Fn@RZRE;;zy82`Vr zCgCSB8!oYr?@tM>kcAdZ43a93r{I5y)iq1${;#gN8ht(_^$A_2KKIX>RR0SX z8UOyP;P>8q#rnT~(W%d^T%WApQ~!TW;`crc{A1IuR{x(#eO`b1%Jq5ihRO67{9j!K z|Bjm-d}lr4Jx?E7_=BHPi!HlLeNd!FGL>h?uEdOV%9CbRd4cddrPGbbvnh#x`N+jl z!+JPHrWh^Csu;5$T07sqZ2mg2JnJ(P%X@6ByuY4uS$SXl!^=Y{qh;B?FB9Q?MMNFg z&-Ulb%is+^M5KC%W_^kKt_MNh2fsLp|EIvea1-NC=EotYKfPt#|9S7<|5-`=OvXQ! zakc)xFv$IRs`M)Tf8C`1ZxHx*&$Fedk2&Z4>;K~D@9)9{{68{C@|J_Hf#7{pMaOnHaDwhABc>n(RdJ?~i@$nA(mHT7C0MqxSRad6( z-5;LRA0HF^;8pP7yzy%N@lSb^>=~{g->3^UYapk zKMDTzSHV~M%(|e5_ge7+?dLVI#4QCGHDwQDNwIjs=Gm~-s9OgO>-EGN|4x(sM;i^R z#uGKX&8xB}ugi%CZ({R1!=qpGTiwY8y7B9w6WSLVnZ`>JSZLJkkN&UkWTlkmhDL6t z?|;Sjb@Z|AJJ|D?@@x7Djm*?Fe+%r<@4&(A%+3^eGN3$J?UZmO%4f%h&%d*IdJ0Ek z5%gJ#4_;<3J!n{czNLiw=aeV4*k?8A)&TVfN=g|D7z!7;&?? z0&R(Z-O@95C@Z z0qxc30&Uwom#=)@Q$G5*Gjv9yAFr?c2T!@+uC05U%i_ALPj}VSxbK)&hwF4$Gj#UkB>aDkMvuI2v z?)t}JK0`clH)71+cKTgU5Ycx0kODiWqKErl!)^MB>eC~mo@i(7!I#1C8RhOpX^7;-MW;wQ(n^?$}L?$_Nj%zn~%d5{`mTCrET8#I44FL#f| z!%^)gr@)Q&Ry+h{gDxjP+~`Ykf1q=chi z{Yu$P-Tqd~h7BUK1`8ib~a?K1~Qu`B%x z6d|xgBcZ-3Om!x)uw!)e)dC(q{RwXe?{~iWILP`7s@BeLjU(${ybZE;EK~cpgt(It z*8_3IIK%~nh!%LFm?|lKlx3TsvwkvzYAe?c&kLcj7 z#0)a#FIMmH&4aI=e#)91xW#z_zRCy3O+N%xUFaNdvX-k z{i}IbzsUA zf)O?Hr-qZpKN0GEhB`kHp)g!w{?p-5a}de}P+HE-gfZVXc7`aSB4@D3lOb*2hwwIp ze_O!6i%}Mi_vDY)@Z;fL{P8l@R2of-(EPtZoi9+G6bzljMUT zPb43V&(h;e`BgZdotKC1HwT4#J*+=zN(qSf`9uK=rqZ4^?SpH(O7t~I$~lUjyWVGW zQFlZ^5S@otAh2ti1le^$ukHtH=B|Gf9HM; zFe>PvK+4&OjzFGs((9XuiEOVWpFK-CE(}-}pyeq&*fr4!q?y>BqHA*!^{xS?CFs^5 z+iQ~NxsV?HP(xF4G7F|SPE5(Mr8vcLl4n^HHm;Mz&ZR^WlII`9r~|&v3L%R%RHl{W zabg+uQ|xeR^n}-Z9{A3;;*cMsVz$>xJ($Uw49%JE&VaZl9^@A!f6}(=K_P-Mp}IEr zd9|>8Z{86HHS<3t|K-VOLjYSWic|!EA5M$GS1f4? zq^6v5@==)f64xqVB+nRT5c$0z zg8(R1c>faO696Tq@buByMF*hp6vQ=18#-Lh24bFo^!i4fd4jC70$a5upMxq!^YI$> za6532O!Tz^tE%l1&OdvIe@da!InUYW23p{q=5LT3fQ<;IL zjt^m==NAhG8U>jubi~}~n`H3LFu^;vWPO%!(P|$Ev|zs(>y01SXJ?vH42&}n{T-1w z))kNnyC9FWRBss>5;jsyOSN0BJ0E~Q_k0~|(wCwUthXGBkJojg5>#iRV<`Fg#|)i_ zl36Jm^5kQNmP5&z4E;z%@^hSge|WNIFX`*u&$DaIHWMq&h1ZfAB)ytPQWDf^^0JcV zCqOOUnH{x)YNCJg+U!AgsE3vJbycj91ALn2h~i=6>G5c;cQVu~yEh^tOp50l%DVxI zyEo$R1ZiG!EZRBWkjb>MMkx2NiYuOFXt{hb>jGIbg)Fu4!M=+Ky5JNwPVf5G4%hu2 zyzyNgMfsD2dd2gh^8Pn%NtHif6z7)0YQfr-y<|9Of1=1VflcU-sdc6REC)BC{Ld3Msz}j6VY(rraS|7kjfcQG@ z0*rv8-82a8LyV>hUk^Vd1NJ;3wiIAIqHKu=D>i9GKhvhe>md-@Do$&&AU>G(KBv7G3ODa( zv4KHmT!S`)VVt2bS&$|==TJ_A5$@g+o_F3N?AWVW^b2YowtCzsEpgYgyWKPFmmpj``NW(D~2%?J2ul6?%2%0=v23?vk8X z#LX4>V;3Sxkd43`%9vl88ud{LYy22Z%Py@$EZP$yPxAcJ96DjBjP&yxMSato40^c; z62mw8lK!1s?R)%Eh(FK2BWE#{Ac;vNsT)ZoWfeFSAH_5?nm*ou&hE^&q6=1ONwM#b zOQK|gkcyhCy$Nuv0#9)?Lp+2VT(pmVjS5$>M`96FZ9r#TS+dzc>q<+a1zgzt4YFcO#Elrmmf+z1 z9O;essJi$#>c9}}z?5(W@byZ6%c%3!U*!FcRNjD*y$dRdQhnkH>%T^aPG?TCpEf8e z8MK_K!D0Sg+YmPGIdSBUcu@5qq?xMLQZ6+H)KJ{}MgY{NaJJOm+8fO0qwo0ahsip+ z=^?U?7QmnUyU{voTuecs9_Q>foP80qM`HHA#dSgY`7ihH7JL`Be`i-ga?vS5Qoxm{ z@8s?0`&a4OUOZZI7bn3;txB;r>8+-gWTQ?WPmRUtX=RSs&pTbw(tc9p`im+nvC@Kc zLhZV-^v8=&SUD7WwW`#Eb|?|?fpXk6EkdmrP1Bw539Fe_3V*>^;f=NN_@L0+TkwL! z+xSeJXLyu}-Gu)VMFB6$0ieY8Bf2KH$<6nh3^6zL?Iv8PULT7Wp7s5jtn%y)ov=vK z%NEcs-HicPtvB6_VJ*EwipRj*ED~Eo2jr}Zk{p7)0!0mW`pjpp#khuUeXlOyof->8 z7DJKmP^EEXHOg7%Wb*-B(J|hI)jJ~&)juM|bJ{DrXv(a4v#~olSDEoD`IrkBx{V+z za5qGqX^}iKP}=ga_@p zw-5u$7U}gFW{&?TXKl&e5UY>>gxP%o$A7mu)%&+7d67NqpcpBP_!=5<_D1@Yat1&R z!@d0nLwBs+>2WxuRpp{rHzIF%J1u&!ElM}+4!@|X>_x377nTB>CxG2{O|fU~v4M%h zP$e-5xFL$e@Qpx}<;|lxjX##rHe5P6p%(&~nq_Abt?{OvO=N*$TK$hr!lW^WnKYsy zdloR!e%V~AEq}j`dg9{?@Lc2iAGyYrKL%_37=L@?AIU&&m=F~qEsyM^-R;+L|BoQh zbvXaI5Qr@R;j;@P*$^Nq&zVYd7QzXC-6>Fcj9yEi5)SVT_hjH9#|j#P@(Vh*mx#F~ z-fqxd`-eWmALB`0?&ap!`*4()5)=vYB7K_WtfC-oOA8C7Hw$_fx-VL#P1~e5zJe+} z+oYU-!e4dWYsA6<^8t4W&HhJvm&c&;l0{>oTM%IInakW4Es{GKCgWUDjO6KvHB~#i zz;d4fDN*wiBu_uAb%9>W8`A-L6=OD;7h@83A*TZwHML^A4M^fD;R04t?6LNGE{^`? z_O%0b{$`eT#)t6v9G$-xwZPC~8x~KFg5!t53$_Gv21l(Wq?i3jFB3!P<;{NqyyQb0IXpvsYM`YDBCRZqX+*>RTHRcX< zLXSy);Sb*uKVP`bTSVb4Nz8zN@h!a=L6pjZmWgub4w_+D<}lxRADEltb>1eN?UmKL zks?!-Pq7dT^34K3V(&tSqS=+75RWXteP8)`{kG;Uf`R+4zJ$#F8UDPI7SF}Gyb^2G zoxl`RsgQ$d3KsC!I8%mskd->^cqdfBt$Bx0lFsi+_xY#oQqj!yoG0PPQf-f$_b^1)bnp2pI2Q94M|zlPA-joX3NRzPjW`VFS` zB0BALcqc4#cwI|nWjiYzGeMKfhU{Ee%xTX-*~r%Tg7HD8W;72=ZMn3^O zo|t#YQ;_^!XO?NmUfmhG|43n6i{vXf`NDlDNq^TX$vMN7bVq0@2>mlB*G52ZHSs#^ z-GQ?{dy$sIKbC5<=!a{*?|Qyp;~k=5mF05Y`I zM+o~AIU3S(wN=~2=sw#nMz&lr3u7{K6B7?hA3QMK=UsCftqBxnNlxFy%g+_5GWpkZudKPqz630P3SX zzUzQm$X1_8BzOoWU@7w2hrcpC{h5|s+fsdhOo=kKjBU&Ki#Dk}70xb@l*<^2(pLTe z(v5gEj(`Ct@Bbih@}`mZn?Dd}SzoUwef=jxS+{5>SLlkmvp?(Qd`$`+T`=~gP|1Iq z*F^V|BF`w0d-pqVy|_1Gzg&Bw)K5iP(8ouQ0)DN*b9GoW-6wm=UKfbt9oGBB1E3fc z3$Im{+NuPn){wma0|eB=$rYZ0hn&3uEj3WC3K&)+ZyZ}yV7C|v;B0n{Hg_9#e?Qm+ zr(~sk9NqcA{rxY~u)i5lA=#mNf7g`w1GOa?b_|ITg0g?{j`wcayRMAP!v(A9&&C4$ zGxua)a>Ce$&oW))W$Aus<53J-R_e*S7<`jS{M2r`h%|s3Go~n77^ouekZ$)HY)9oV zKrx-()6QHD^HOJrkSQYhH%{I-0+JJg$#Yny92p^mhe#F&q$?w!d-1yeCC#H6ftTcJ zZP8Z1%macy!K9@AuERbi2ef;yHR*TFzefr5{k6ESc?74Sy?)&?R{Lq6A9Rv#JrLqW zDpBox4Ic@EO0=p{TJI4L+Ky7*7Yp8=3+LzN?Sl!+m(<0vKOWt8rsKMeX#L~OKA%t{ z0)gFlL@T`xDuD?dN&Ki8`K}}F4Gp_Tv9hc)DEdd01<=NmJUvk^{G!VR+<#*-$fMQr z*k&hA>Vo5VoLA)Q5x6WIiaYID0e8u$FVbQ)?oUUSN5et8MU>wVEFXV*1kM}oRD}9h zz`WHI`RNUX`Bt;Z_Xd>!_-nq{4+A>j*Y9)D{fT9v_ZL0GPVe;YbBC=4*oB7|gD6ecGn+pTK2-{p3Ec_E$ zaM{G>LaAnkK?(8I8M-$1;gH|zBNsx|z+&h&`yf0G~SFHg%i`bWNV5Ba~JrvsqN`zIUl6AAugRRny{ zACylX@4BaD|Hw~{1_I&!pbRfv27cJdK=A(Exw|9o*F}7<9~N=HYuJAt|Hn2Of8&2U z{uc=Tp;!;nFYwIs#QGSE& z|FZ1u6gP&>uA+E*NZSu3YZDE90<`kPsra{4D<5K}?`2S@BSmu^Zl=CY%oX8j&raeZ zzYNi}4`&7)K_1k8c#eAHwBaRrgnbBL?J31C6hmKTKS+JS^BS6eL=ZI6%%RX2Kd#>c zp%LG0Y*&Q)7e2dm6owihoJ`UUTuplt8=0L;hc>_C%uW0pcNm}wJ{CQR_^5Fm#t1aUlfv-VaBJ8<9${1&ZWVo(w6EebrU7a#{0?J+D9O~2(7+alKk!eRQVXL-s7 zU@z_zt$H5UO{{?%^&OHXKPr@T)~!{v_XkMd!ztOeZZ zSVLW#X)%G_Kr`qn(D3*TXXqW$P=_Uw1IbK_HnEA*=ZJKuLwjNZAr(KRoU@bxkPc3j za*B|+y)N3DENw^Agp|4ge%gB%(?M32MO#cc#MJFK3#~6&$~i`3#xN~O<3e2cN$(z{ z&Ho-JZdQU8kYl(zD5G#(-cBb|%s4r*ctln{LG;$!DWfwb`_J zb;$)*#FQd!bbcTJ2nAbtoCuB(*4~5vslCycZSPCTXz#9zMteuTyG(odBCvOWhp(|^ z`kwhTHuAm+=HGB$-+_s0M+9h2dqkv)!VT*-6TJ$oX$%@ep_T>$CJL02PU6g zX$7?9d_GN`fR?QDaWrGK+rGn%%$Is3XkXkD16-8agUD(ylhuMdh<;Xrdau2hWfJ+` zPE8#s{j&x^SDBReQjbLKa31Tv)p-HFFGS?r?7j+ENLtwVKzt81>6es1@|NND1n?%# zgLdUBO#!qnz_{>cq9R@fvrs_yKdu!1raT2WaBX}Znaj>tjHA&RjSF{iFWxHQ$RMd0 z?yXM`yxf#Ymz(~KomFQiXrI~;2eweodh>P~>T{4aP|=)ki~fpBxJ_C@m<& z5bx500!yLfd5=dgdHzAGO=kHflLc$aSVsO6`KW0EMK5zc0^^#%Ta#MaN2J);ff4e6 z3s#f=@>Wtxkp-|o9|y_DBbRTjLKxw`5-&W$#(nYFhJug03z@o^_v0NA;uBwu}~pVOP*0P|jK& zeSuJn@K8}3(~&f_LC}d=)QNW3iN6Mk5sck4U!y)Wdztq4Z)iyAVYW1RhQMm8J#qzU zH?D7Rpd4iv1j<$K-e?uOn53!q7WazXu_V)J&7Bd z+V$M~t`6_U1OUo61r^lR9T=b1ylme3ykTJ5u*T8(@8i{e^vuV)QWY)oo)y;Q{WJ-+ z%D6bj#$Q=8Jg((g&=Y0A0|SlN#0^?~95!iHwgLv#XxTjw)T;Sn{CD8^-w5$P)VpvU zbis$(YRLSwqxqprj6>Vg2;=d0j)kXko;7bC)a2_K(C=Rj*nCGwKc1m}xYDf~Nuli; z+^-y!h%qBUv&EKUzlt^KdqWMvaO%!YxSTKmF78m0eEWD`5vHOqVwuSYpOer&YDe9> z9$%5btX3v*N{*hIRp`R-GC{cRv*s=5d02dV_&jLWtO5*7XT3W6cErPB^spBi_%$_9SXgR_V*B0A=7QOMVPd)J~*~-xAmad5Zlas;IgFk7Kv+ zI+J#bh4VKGoW!>lCJM)v4rnJ^`Pm)SqhX7`wL(%@hriiMX?v@gH^T@6);z?clw%HX zbt}0vy2-DFpLTPJBZUfswD-opJOB}}B@zMWSNQe5YVlCL?;bi^U5{k2j0V8roxHP` z%B7xDe&a}@(Qi2YmlK>V8=pMnpSk!lqMvn?o~?M4In(JVdmTcHnLe)NON^39~;;ntziVI}n6uCFj zcqC6LozQaC*P6tGE)Ci>qV{EGu02z)eIbY9*Bl}t=T$l~aIMC~J7Q_$9B*5mTuVM$ zIm3xSJxAyq-Gj%3wYR`f@&p~+5XaK{eG_`39PzVNC<&SF5yWvYhB>H2_AY3oajiQg zK#Mm~J034l#bjoO*KZSEMzv#Ql}mUkB+naUOwq`-4_1Z~)WgfiIqjR}d{!{{8}cxsx`(a|GC_-kqkV`K?$nQIZl9mV$<5a{JWGlpGtyNAx=PVyy9SI#) zN_S;g^V*{G$x z-op0XYRq3i``*k`)Q+a;`zF?V7nuC|vD@F^r(I1;NCSJzj_*jWqriOLjkq4C0|6_i zwN@YVopCAWO}e&%it&Phd^4vn-@4JFuu5;I_s_8}vhuq&l#BGlyxoT4;icHkNmNv`~d^?{Fk@1Sbrkt>-!w;lS%q{p4RkD47qBNMCVP8 zD%uV|u5Ix*(PDXLsb4=Ty&k_$_-WtD)d@(wN;|nM5D4yEWfr0ScMBp9SxrB`W&QUv zHdo%uCoZYdzFrox1r?0*Si3enUgbHx>zUvl%g39ZklaTI*7}R=eCuKFN3csze;No-#C~2{$OljJeu*^ZvO*dx2%DETu`g4}r>uB> z_^rPMZxHqqFT{@f?iO~AxM1MmOO_7RHXkG$58kJ`a;&4Fg>>r}&4E(`Oaz3g0+F^tZfWJXLZ@?8@$pUD5cTju|d*{*qSy)f**{%4^11&3z$8_>>Qfd;P~ zqZOP1ZN)-U;?-DE$ z=ZF!W0ov!JZn&md^PXY{&#k>8lpfnhWV2x7+P+V4Q}Hj_*X#UxW8vp>Ge`4zRNw3Q zJj#+r=TS2bH^at6x}ZPve3O3f(DNcq4s8=qeD~HY>z-ia0h6O}Pw*QtuA6K!V*SA^Sq>pzNbgQ?XiJ2lsFf0ws|{a=@O5PqtzXX&wI${}F*bI)>_+VN>ruhD4b<(svD=Ll zDQBa&t50@ijOfY}Lh*Tv`uuGZL1=^)+|SF*qe$?x3!LroSaMt7^hylf(@^g4PGVD* zZ4rTuXUjrR-C9(=l&XHD$rz~i+YLf;(-cZMW2t}7F4Ymio%>zeYSJ;VoZQ8mgv~74 zu{J!i^Dbe9wP)~-Gs{mvf(UIz{p+6aAz~o~b%3g=E zfMzv&MjhW3!>>mllXAWUo6o(mo-N|_*av+03pY&Thuwgaustdm41^|O<6@YEX+cvF zSOt5b4Tzy0UnP2F_YMunLr`OrUVWHU*l%#y*JDBV>;k-S*K=IFNpw|oJeAAhT4NGD zm!;HG_*{moKr5PFC)^YU!|ocoDiVS>G#o~`@0FoJ>F42!<7hDS8 zAEe#p7SDt5ycG{i>BM}ucKs6lxYYR+N|pBdcT5jwrtGPt;R-15i82 zi|pnS41!iI&%IzW#h7MXFMIp9$L_u(Dgj-sL+u7EIr>9hM?t$l0NZcNU z5SLA?3)GeZcCshx{=(#osNFT!@IvecY0zdui^nhhL!4ct zu5&LNi0n!Z0WdX)&1*B8bdT@D4~2Vthn~k7^x7~=_a-k!O*#!o>OQ-gi0p9>$io6e zv#j1hn6c$)OX^U+v*kII^Xy-QJEC{8N$$xfc~#sia^C~Yff=bLBm6-%86aR`c*$iU zFW7`Ow595+$v4r}6C;B_7d`-#qxllvP!L~s{S}>q+DF`~Yakv152qtaDAhbTS$0IZ z$z0C)FZ^(fK!MY$_IqRf?=<;pI-09E%A#Y0O}q-=^YcFUqU&(#XGKSBjSFUWs%lJwvc{Ca~Y5-u0r zw7?*ljC7w5^R3-Hj2=$`yo24?X-FcUpC2K6fP*X7X#Cr<(V zQ8N=#RaX+226T8-R;XR4Y)Mr!5c{~u=|MG~5?`G;l#DA5NL0!>MXcc|XV(TPr&lyD8F`PQ zmFHP~2|e16lzNU_ViQ1G;HjdtfV7@|LBcBd;~bJxfFT#@N%ZXxnaCn2f(2i@GWt9o zuN=9=7a>UYV(bZRKn+7JCVk(N(1g!h^Zn?uOSoQtfD04enMbwCgZ@Lw)kf@$&7>cSyl*tc6B3ST&o`{_PsCae(v$<-Ef}qozMK7ka_;$?H2Bd zPYmk!p1w^+Z-nx=U@1*8>T`>*v2V#(#1I>ZG~TvTH)7VNXtIAQ!}8mcPUA^$GQh;n zB377-6;2GV@K`G&S`B-2;c!AZt_dcIP+<(zkKP!1OKMYLa)3Mx;dQEgl zwtz~~K^a17d%oe*cKU+i^Qrg?C@*q;h7a1^Vl)JV-&t+$rx<%nD$ZtFwxZqe-U)`e8A{P_D19qP*%^-p2*@;A@~#oQ$>9|pn2Dd{t$Waa~jW z#@tX{72!~6!E3OgoBv;hL-iyfxW|a0x4nyFErp&qcEQQCgc+l4O$S1$w7& zo8*}sO($!)GzzAnS?`85e9y&TmdW2}aGg@5-5(CwjRvv;=X#UH#Md*gyzN3L$axV3 z=G#+^I-NdfQ|e@O-D)Pma=M0$GG)|vT*nFP1L<1p`OSv4l7pkp=bo2eV$?P3_c9N`HXgo;gYZ}EPK#-jN3pS9~hq@ zB&R#%kd35m%+=itMJ8Tcq@_jnrdw$7WLhd-8d`(r*G6#)L*L2O+BdlYLzERa8hbs+ zMMC7vG~_Ue@DBP|W>S}qT?mTRJArMoiNF@*QBS;L{UB+Nz9rnJ9Z%JTj_MPR{-b0j zIF!R_N~JC$UrZq{|7nv%0%G!~6k?>M#GD1m+tBDQ(PmMF&O7jaKy3jYpNA?E2Mt_2 z8P@J@Fc5qbQBhG2`L5-NIVRe0S=UG%hAI5Ciq$TP-zQ3f6$9weo?6fpuz8Lc)#Z>c zrUz>fe7S&o%~Y>?8iVT#@&J9IB7=z39JgK|Kb~T!4~w~a;g&>*ylC@AAgsAg*!v&O z2l4Cu6iHM&ytXl7(Kx0(wnEGq)!dyJ28Oh><&i6yqy98TJoBZ)}60F<;L>TV&z|d)x2^kXSh&yf~(0+rb;R^ z`zz4~19u4+D=nD!Q@4j@Hu;TpNx&G0x;?y_-9oPk!9_?v<67+J65h0&-M%>@dVL*& z2r0+KTUx8BG-=n$el-o_c=FsFf65YAN5rw}6(P2;pmnSjgycX!FNh{S?bTjd9#;3& zLB`X*{#Ro_DUhRy4g~~=>1nrtE}PSiVq~H&J3Xk&2F1f!Q0;W_P|CbPx(4Y&G^kPE zhBdsDf`ST?_cE<7(CCXw>YeHa!X~8HWX#H1$AAQZ@wM?!v}lmIU|mULC=2()2#V_v zE}F{XR$$gS?biG-vDr4LN5PPy-VDuLP(;zia+|$xivF<+)j)_(tD=VZ%bzJ~e*e#~ z=QiVZ3*PA;rOx{R0Mvze45+?QO!VMdByZ9ICQ4H}eU#AFBp22s!S5SB#ouw|>cW2# zXpuVaEBcXHfDn|kfwme;)Op`vPT?U)-J;IhL8;{9VGw$N|9ZubSPSyd3##BN{6>& zSBFvr8EM@Qj`YT0pYrU=RtH|g+aovpYB$600z2(AVGKNbyFCuYXG^X~lfF9K$Dy2Y znD;r_?RKE=c=sRWh6x4G0+p_`HypB?i|p+x@ikECt9^a&V&8Ws|&G z?Ye2Cw_El|Z(AkJeBRN0Tbi;{*={fVqmRSeBW~_tY}sM<-{&2?80Tf8^jwjwY`22u zQ&`)@TKH>w`}caScvh~QNsTLmEQ?JkyaCGs$Q4kij58?-vag3EG#hdFg|!J2rKts} zP!LjUWpB6J2@DdqrAoc$<#MyIFjw8{?Pjt!9EY6hR4N1i{01;@EG3xwAMg}dDP!!V zDZVu2wA6daYR=pbbrtZ>Ep~IE0~Dxs6=~hK%MHh*-mA(v=LA$%L-P)(Z-k2M4K-o{ zV1$ASVJe_5j;EEm6k0e%C2(?Oonz;wC+{VR2r<=Di&Im zV8%j))L*0yhqBF~?3T@ia=YE=3k@}#g&KbE<1qV_g-Dr>c4zF}cglq)l5k3IN>eV# z2J#gmV%Zy`z*HqVx?jX8RQ9Er8v$=~e`UODT>#;L%feJ>7bXd&&!+6P7i!ld?g5`- z%8p+FJx>tRW{Yd&;4B&O;O6Z@NY5oSryQsxE_B#vr2NW!v)p@XxL;kO57;(U&uR zfV7HU@K0S^egwC+)1bC(F=q$XfW@#x4U`y)j-kCjPp3Z%$D1#6KTuZh0TFm5k&|=@ z8OB|DSBC694(R&>$a{;uJFt$g4+x-+For}Ojd8o={6L_9DG0b{>y*9*&lg-NJoP7#|L=**dp={;O&cc|21K9$zHUUoNLCo66$^cC(Ve=d91X0xY6nz=!H64k>} zw~iJJEm-Qt5nzJkv4@;L+ufJ$nlXa5`yL}r>-%gH8HdoInvzx*aF{>AdVE$YWYXH8 zmEtZ!&lcbh;Kyfk@kd|CO&7Vqgyg0Gzn5nd zEj+s#(}5|U&BhNrMilO4OhCgdW!PWdZ zUgVP4hN0hv|18fHU*9ifsJ;TKoIlIC4lNg9D&SYK>^?D6b{7oAgv`|wZBqRnkXrR6 z_-!kI?usj>iarWE1l4$NLm!?H=0AP(w_7yyzpvJql8XF&Pg(2R}Jc5xcO zEMP3u7lv2+6M&NIsb1^WV6A}p`qLAsQvg|s8GGRK4E#SkSMNYMzTA!dKL;t9G|Ehv zyZQ_8{{osOF*bt4+UtMDuY>r{0Z4Me=0IBWtUMm#PB+93_ zfa>Zm>A=;{f%?T$~vaT`tzbqLI8OW7!XXLfLsBA(?eGg3iWfk?rESn`Jd`-VLt^MRBGJk%390- z+N*?Vsr14BKf$C;(jc|+BK~t)>O86OOsX_zY)z`vdF&~iAe{Gq0X~1~1)}sW#9|R9 zALd2T3NzXtI#at36LDHA0q?c9bc$_@>{*EfQC&@7H1S7(;Boe$J*n~pLKI`cU$J$h z@E<=RJF8EP`6_fyQK##v0ZT+YdCnZ%^67G<2q{M8l7xMD5a)KVfHZBvYy}`K_UxUw;I@fH#P9 z5wia2p#VNA0Y`zjtDlvG|4!?Rf~EM4+5@f^q6k$mfYI8Eq?xvze-U=)nJV@Dy#_UqP`LTK{k)IM;caY2cwj zHRJl?zbCM=ChouJKEqJoP8zw?OYB4aD1t#cU7pPY;$cXgm*?U)l=uUCvJ1hGzAI7M zS0NF+C?nAT1^m#_^9b?-CIcB%qR<7Q2}IwYW{j;5RswxQ za=oP983lrNQ1Yir)wpz4A^2%(0jGd1l4(D6!R6)*>611@5vr1kkUC1NU2^m3A!Rtf>qUjViH=wDPApkc1~l~ zd90V{5v9i38UX;y$&|HZD?wwpf}!l6fBkY8Dj)9l8(LNK}RJQ{4} z4xp7!mLeZ^{$w?i%o&;*EA_ncG&e|mKBW>x4Ql=kIJd`XMtfmLwTUEHU_N5)CYEJq zuxmaX53Kanu&xq=RQ`mMf09I2h)#%B0p{X(rtCqGQ`l~~XuJ|+9P%Bv!B9U7lj5f? zTrCMS5_G93=~A&qmUi`>wg%czpH3sxUd$`UUkCIWt8t`1aI+-oNM2Tx8giI`Qibfo zlnVG*n?z{r{XI)EcRGK*MWXEXv%z?e8=_f*MeiSN#og##L!nT2HOZI?&l*m(QDFHV z2Y!*UcOo&(TsFcRb3GXJ{yxx7E^g$gX$7*mt~S$%FBeg{Oy0c6A9nX|3Dgd5Ik}di z?=i)%go?5LxGh*WG|@u!GWuPY(gQqf>e0(L^t0htv6-O)E*d7}Ixtvr<|5pxHJ3P) zD!JWWdqWNUh95Gto^N!ld%1`jgHR#W%*Sf%N}*k;a3}@5{bX-AZa35RN4pBqKhl;Q z(O&XrH8%7d9Vp%=z&oJ#imSk=HW#X0m1Vcus`KcxK)Whc}m7yi*lHt)5n?-bLX(|LR0 z;Uwt8U3Rz6oTiikhU{^>?G5Dsfo45!=yl)X=w6i8y~ZAQ+N!SSJ^am`bo3q6zcSxN#>uMKT zCWH@c2G*_0Unc6^kt%nK6_;1j+Su6hv|hndWo11I_3E^n1; z&$G=r<(31DRE!q4%7d(>r=krlRqw?zxTZA;=Rb5E9o-T}VYU9T#Ef25A##c@G3t0jdJ23RiH{@SqE{k^}4s zoB9qeSyuH`T(fNIrmyJFhZ_WG9cFbC$&C@Nq* z1E_CI;@!dXg@@V$6Iyty*B)1ts+3!m*Kwtw-P=^<0FufkoFuFAD#y_&FFbYy?h50G z)*~0iZE+~T+uHSW57AyPjJqW66v%(8^3F27Dv$yRAhMtqbWH>yX;s!&vEDdk(^8>8 zekeGs{&#@GjX7`3dZQ!Vz0B$6X%fua;dcyjLI?Is` zOjC|Z8R$?pv;d&!>w*J``=;#D`Kz9$a0i0@q7(WQvN?7cXvIqESJrSJDoKm~4NL9B zPc)@Sg|A;JxUTXnvlmGj8!N3-&NBnSKgB%6RG|UalPyQ3knV*_IprMJ4ftBA4<3hx z6jb>vNt$=W>BUYo*>!sTL2S$%BQ+MPAV%2Lg$D$cY|*vFjMdl{iX9Mx2D3DB8<6-1 z(|3sK$T}#gtW(GotiWGrA5HtkqGM*xK3&Nrd#@A?WwNLDS#+e%DOY}wv$g?EB6&@# zqKrm)5oY?1;z)=KBoj0J~0~Hr<$?@mcT9m&!lJu z3l1A4e9Sp+$8)LQyI946M^*p;FvJTKfz^Do!@LDG1gyYZiWkEQYr1e(L;+9>3xBl) z6&kw&Ykb+p-f>gMBYIB zn_M7!2k^6?$H6LSlzX3d7Y;4-MW2zE%ibZc2hd+-lBgxxc{Lu%Z|hFSjO5;*n_N(P zj_wyK2eqNk(jARsk@1=C1v4&VKlgh*0nPdBGoA zBj`vJxllXsbijDwkKQo|4_HX?jlyFVqIZ`(McmJ`S8+Fv46{~LO%dggycZ*|w`^wvO5XPrzJncikjh@iI_B6dyS8bNQZCNsVHCtaT2KBh|J zmtP2{x2_W-=KSN%HKlz0?g!^u# zm+HRd?7p1+0A{16quqLI3-Z!knusA^cALaC8#+=BU4MpK_|BkM@6_Q5Vc*IrT;gDv zvXC1ftWs4QnvC;UHTH?y`w=pg*zxd799ReIcOxQ>x_Uu>%jv*&$N&Hj5;^!+kN}Z0)W!Tk-Of zcgPaV|FSDe3wrI2Z^V6fQ^@;(vge$$jjsd)a|rx-PWTN)Ha>2L=*+3jRpPodN5=#O%zhLlCIf7lOKRC!gt)ZJV^~oBC5(6 zKixaL3+bO4d^2)P*71zcZIhaKY$36XL;N#N3-9xfU!eL|@qo&W`=4R_(o0ic4f4y# zsOJ3AW()I6E!|zA;3nGe5>DG|7FWh4Prcb-r>pViS86wdour(Fw7*TPvST#|jL2bl zWGf&1)H)jQ2~y6JLHG{@{Ah+x1b7S`t{v&B!zW5PcLd>|5%4`2J_+Czg#X5XPm*$4 z2jK?`_<9xxy#W3pgwHYHd%-wO27FrqU(E1*0lqiFk2c`@iul(I|MMz_f1BY40{nTL z%pL~(K<6w0yP9B~&j8#|fZK&|m%8Y1L!F}p+1ZnWR%vL#AK94@~jw=Us?o7AV3&yyJHtYZK1b`B1pFER_F`VGAf6 zDQ>yJ{|L7fr1+t>XkdDNZyvnx^_MP)Mk z-1O@IMIN1;OKAOSg4>%K-$}l92DUJPV0+l*&^*P zoHh{Co)Kv`aoSK!8!OT}vz@Fh`YW@4tAHLf4Ru34b^1zmk_PJlm4I1S@&w(_ulvUF8NvaFFl@tzk_Q z1t*x&V9@Gas3ONxB=GOt7sOju4O-J0d=Xf;t`b_f|6EX{+gAq+`^v!2QpOJ;#C?*c zX7=OM>Dw`oUg%{!+bj_$$s6DPaEmMCnTHrVvjDAhU^ zv}Q|Ta@Uc6q=+b&FDR?=WAM|iwDN|`I*(`@**@^9Z9l# z9@VA9GmnNziNzFuz?Fi})Yi19Rngsr zt>lK0^TF{SY*9+qW*Fmt^goS%QVZi>!Q=nu+2-S488-g0mmU9smey+= ze})+UoMz*Pv2X50>+2Xp1UMshclfcCi2WV-i^(Q!)EHv}%z5X3csPYFiDV_orNa`j zYhnGzcy?oGV@xCYLERET>n+-E4;tNb{g1l$&4a=24f@aBgRUiU5_gfd?(}$v%O0fu z)v|oUC|F0Gt!1?}=94uYv0ptvR#Lx#C{8w4_$BY4rObgF1&T4oTY;XAo-g@_M`eA#HCN-2>19l zhsCbO^uQ~s_;l4l?M`TI90ng@(eAq`BlM8u^6hi$)bv>%$W`2Yh4bdn`x`L_u}6}u zlm^@HDB9l*?dOl?u|pSF<@7BZ{cp$5`y^q3Va~~_1h`KWw0tIhEcrv_zAL;LwPz8py+`tr2nGk^MR;OIXyiqL-w{k%Tns@ zfWH5U6nPQQVfPNUIMklb1iANV+1v4_MqJau=!yqy7-6My?OvzFqD{XR-PEFGw`LB> z<$ao>i7n7Sw1kf%wkG0ps3R@O+w3XHcJKWbSNqx|u+QvDx$NFzkyBo1bas{9g)l*d zt`mg3%%U`V0i%SNiwRB{Fn%8(^nYE4v2cPXSFs z!P;Lgf1ZRQZXoqHBTvfNkF%~0$N7h z4p!UJyQ?#hQ?cJ7;=*ywaxF(M+D@;l_lAYK#9ru+##m;u>`g0kcxN`^yPQfIjPnP5 zoPSvC?%&PM8*zL%P8i@o95b!tJWl66Ux$DmckP_7HO(TZa#xUlUn{o7P?Zj+=Pcl6 z3{SSr*)KzOpEnB?0;m||3zBd=(8RxVn&>3EeP)d1#u3cU=z72mi?*sOl5raf^yV`Q z02K8@4Lv$sjNoa3IwldCvU^i|06Mo!zZTsH?^F680!9$xBkbJ~Z=d##loKd7_ORSA zL_;6oLabs&N7>sS3CM(oLER5Yg)AvcKc10$L552I8NtPk>Q+EDa0>wO5N3a)S{ zRdD1TK&qfr^F1Cg;-!&%(bjhf6E1rZ%C7A2^$j#Te-`T(jQc7I3Fm~~jtIypKTK~! ztC<>8CM#0Ndri&OGb!gfb41`_ic~u#iB+C=?+X|^cJ#2>|3t5N~Uv63R(#;F9 zLctGI0a5{nNjmv)a^VM#L4JjKJf6m;#KAODWU7Epbb3Sj2D5Nv5|VdokTcMmHlgPO6p$lK zKe&xlgv0IY0ziM{iNMN|pkJXMK~*Tf^&_BQe}*UF)W8DZY+X8BEPpeSLe77};w$s< zY$D&qwiRgI{6bGIm2y6$`Y85We>?Okxfz&#oFxb#Ukx zw}aG%#IRsTfb7MQ_u;(*NW=9GiS5ScOD|r9;WwnW3#9c0N^EV{eqs26`MD{Kmc2tP z-RR({m$u|;ito`OxIK0zC|6^@hME9+Km5U`AVD+3R|nPDk1!o1BK-W2f0k=!J5k8Q zYY3Tuufg2{us#+6)-EQxj`4A^t`5C>Z|M7Y6#ravwZop!d0Ll*@bfgt?b*@Dy(HB| zzCU|ZJKn*_|GB07Z#xe3*ukKJMc+oi!N#^3_Ct55pgN?Ex088tpYGGg9euymmCt z$7Yn0S}VKG)y~_6KvjnvP3%V@*dipW7DtG1dg(nNyV&<4Uf+YG4U1!^y%rxzjYfP^ zS-sAo-iX4HB(7#_j7F5k8G5Cxj!R5a{|Yh|4^5myVbO&ZF2(MCsRw-8Q4Cw~oe?=} zRFYC|y%~hCYYRm%b&{MbVsd7R1#>M45d9 z3ZWVnjCoJ-yDeAZ>ijzfzyP2NE3u4o2j;9C9$-yYIY{riWWiD~)uQcXkT1})0^2&M z9p#jT(t6w1a$$`HLtL)6t%08jj<}jsY5j!N_Bc#hi!SBx)Avju_`WXAM=Mz!3y5p& zP{$_M+FlV0+f+9)R?lhb7@A9+=eA66uXcSHIx<1&mkplm)yN zFuo5{I(GLnPG2wZJzNUMx89##+@D_m-}>|C%k-yncz?dVvgQ6HaDNj1zxC(CRW0}D z)m35r=^5G|cZn>!OXd8ItQTbZ;;sz!M^IbS@~#d|q;`0H7)8fFu=L^_L;W{G{aW&B zpvJ6eNysg-H>2E^ybY{0r9hj%qLE+Kyg^!@u?vW%LRz1;8)*r&O^Y121(MLmGZB8u zaX;vWcKB#}k4cKo2UqfbQ`N_f0qw0-A$)SrmtlPJ2C?d22()= zO3*5o6T$z8dwa|Rhd5;e5_=)^a0{r%7VI<=egXd}67*lYP=7yH$5$`*<21f4E#r?) zqmHcr{MD_z4F2S={=egocD1LjVEirm2=F&*WeEOSeG!Jgo*(~5{Js5UB>q1B@_&QB z3@j`nR?}r8P*b&qeaxoh3i_n;Uih?ozv+=nde`^(hp1N@*@hOYVPKc2_m3y}uJ-St z-ZLH&Z+o#?>c}0+a1wD~6 z-%DEz1+-;fU51bXU;Hl#>EFJO(^u=coC#^<5+I~2z6v3vjh_)AT`dUdz>;u6nwT3& zM~-DgH9LcJ^f0Gv{4eQfQ49NdCUc}AatSv!-6vWoBu?$_B_d)`8Jh3BC# ze@J)nOowtRd}}9_(<4lmqUQdoeq=LS@^7di)^@^ttqr$A$6|7y$r^E+rEYbZ3CqA}7u3_YVv%E6a2Fa-HIFi4~ve9_0Q z+D!P8`;}V(&fOU09O_UeIH{-1Xt9STTccWQ)l^D`Za%(E?`PfzXvJrfrw0O| zo+;cI$b*Y!*}Xepbq5AZp3l%BY|wyde<4BTHKz+c~q#w4O&vc!AWB+#S#xYtJSV<;ZwxtW=wE zl=?=zJ5YPEh$ltzOfj<`_LFJ&BpxBy`Da3|YnhH*=QRSc&TfoO-`D;Sc?9FEJx+Ll zaXz*=I8J=}RQvNPAd^=dm*-bYuCgjnhKmu_q zF_0!tlhkDV{G!R&&rL39*n-^aY8A3yZM+9v$vcFt$yfwZ`7s8c_o>EbnB(ToVlwEI zmm=hGnEx*BQ$Ds)$Dwn-*6(Y9KV^$EW$pz;VIE@8Rjgyrf&*VY$hFp?9GZIw$0fTj zn6EBy*9@**BKoT%sSU#B;TT;x0z4`R?t51EpAq_6a<#UG7OdjhZsHv109|n^GHa6$ zQG{CU>pVWL@cl~M^TN-DmJ+ewpl_XO(yOtk_h4w=dJ8d#a}0*-t@sX)gEw)+gi>u; zEc#RD7_emm_GW-3NYHJg1nf%)JMTt%5yHZ!Hf%@e`#zU{pX3rv*-v~hea>mqrg~yI zlxvBWA)34fo8*4&H~MFRHbMTT(ui!1dGixQ&|Hh?;imH93?MM(;PpP))A& zGWxe~!F%~{qe!)MFL$3ZkCWCHh`K}jANk(M`(UrXny4RG&eo>5re#@Dv~DZG>V z97YEja>$Zw9?G0sjE#*G`)K?{0ON3d^mn zF8Y&H4?V%MmNP-pE?I_IpW6Y((qxGmiOWH`Buw1G7w0e?i{NrX=8*W4!f~ZxjfAu>FZ*6w>rRfNG`w^ocg4yBV%ly zZO-T-u|FLs2MSz*=RqOooJIP$Cq}l9_4Rb9@<6O_n=^sy*p;bQawg0^DY|U=q5sG7 z^8ccI#^wTA3+TRapl|SGmi)TCGsEs2XLHOPti|EYN)Q*0=N4iWeCHU~|G-*-I=6SKOl^-VPl=`OWoK4uXF6wffeRqh3c%aNUO2hd-s#J zlFnZn_6PVqc=}qopQYk-7>L*jGUgATLqdCcZfPEmSX;d<_$qoRKitSd56o?AhV0#p z;E0{VZ6s9Ubun;;_SL*|67YIrN9bdk`doVLed0ZB7BL%ftSY{IP=suHgRI_76b=jA zLEtOoa5TUnnRqRhla+%gv9#5LeAgB(7@UwP0OJ^nT}c(;YnKRXAj}@J=4ST z|5*F-z^01r;iL2SQOI#^aHTEPQfYXr&%4B3r?_~z7F!FB0E_8w-m<+1si^rTlo!dRlr2&E*?AT zjMPEwj<501k-Vg!!8bVP6<-UyfYtxfuJ%huLiC9`D8WF=z_0%T-{81l$j%Sp5cpE8 zHyi&By4;2EUpZ*aIL5jIJp7ov9pE=b4)+fQ;X?TyS*XsyLOI(IKQ!P$ozQar=pFqz zRR1!ppEQbiw7t(5gukTLE`Q(WZG$0-f1>Y0_7wSOltLPyHm<%#X98JdZi~Kee!3wX z-*x_fVgoHH92EROgC&JCt?S-M{T%1pL-_`y{AHu_4|9L zKx=?S)XK=OP{}zC5SM(lZ{N=YGWcw2AKB$gVj~ZcAMUGeETirb7U<%~P$_xWI!2LKM}q1`k1izI^3i%-Js^ z%XUeLBh!1{>AMH{=#Bs|{i8_x0*&>6p$FazzmL=5->AdivPZzbMsj~gM;9M!9ETyp zq^rnQb~9o=L>w>AABTDrx6dFHR5;!r4+Oa~#hbTLYk@}mGj|Lq^@Xh$yePih|B?Z< zR5U)b%SVf4$tv0?&7G+=EEa3^9$c)^XxqvzykDFWC)qRnM09S2ooygUtT@20PTq0j zR64JuKfUHrhS-zVj0TqQWl*eMgf6*^AT|Z6`>!MWP`$+=&|=^-UjFqPP~MIJFG_nI z(C_i7`7tn1l;I=o;eH2#lM9z|ebH_6^KvVVT}+|L>{SH$(dAq)-aJaQiK1J@)e3zX{$+ zVHHC72=GCryr%)6#QoaI>M&>7p4%8duWo_ zLomQ!Mk4OOK@Z2pQ{3uGau*r6tGm%L7A4rb1-X9uT9TpqttJm_)CfOn2x)|CQ@J0) zAn0bKMPbeC3?Bh}2)q!|1KkQ|RbYAO`HdCuJYVQ>X+-9aiIw|&FGt)Zd`yppM*rI^ z{VefcLxWrK4i4^7V<-(J4emM^od5oC|GODf@!sf)aab{1_p`!IBplbvG5zt*vmRZq zX=y#;#Cjar7h^ph8WCELAI_bBJut8m-VCh5nUqGaN9NUhJ*xM$upVEph+L01N+Q?e z;^i%@N1v}+T91cI^z|6d2iKVf=ZyyQt5-#@$I-X3;`Hc>PhQn*J&KpbUXL()LP4*2 z%$~1V-a+kH*(%gq2QlPIE>B{6VT}B^YJM$ucnl6fp6?2@2}FiTR*bRI)oQeUEvNP3 zd>UGeBaSC}e+C&_HQuiL5;~kve*lbvz2K1Zj=*rfF7aeQ3x)pApfJ-2N4>KTy{MvN z>m+L1yx-8kud%83n!<+Zl#zO2k61tRGr`Z}Evwp_2=Ll^PR}h7QDFoNAng%7>*<<)u;OWl`m2QRNGx$`?kJmq(SCN0nDZl~+WSS4NdbJ!=eeGk$4^79ALq ze}63Q(J_p!LZMqF_el&7Od)K6w$1IAfY(EqJd)zuO78bDIIGuMlCdADtK#8-cq#fV zUN$_qOWZo&Mp-32JS^EyShQ{zRl&TBy^nHt&24~NijUp*dxCxoSShZvPW^BqXj8UQ z3UP~S0IA44>y*KeWRT{W;!)*ysjSK-$(h}g z`+4n2amN45I?cPkCXVtCOYZI%jPVI0G2ILBs?VZ$@1s8+io09Tpm-m~;A`D3Z_|rtn+sGa(O$$`e(w<&D$<*A|a=)E4lSI3+oH`<>-WDp0a*= z5MpfQWu$C?O@)unT{Rfu#auNF3Hqzh$;9P&M5Cut#1Z|)B~8J=3vvOE%&ZdhL5#Q= zYNOJl>~#{A2HsB zZo%MsEDR_a)&AsQgueUy_eFGF6ejOS@DB|`lsyzzQaK>pN?2ey<{&gHI|{E?wsPfF zJ-mX`+l;a}wnA0{rbs?Qw1VjRCR7K4Mp-Tk`&yL4{Ppv^73UTUmopA8Pb8SURh`Hc zmNnA+YPvDo>aKP?;I4DLB<}s9c@S=I@DusJi$=ykBNpWh^@Q4_?qU|(B?a9ADL}Rr z?_%B+bQ}|--yG!nK(-n!n7G4&|4MSafe|z?6*Y z26M8}nG}5wY**f{qHVmansDHN@&X+%YX8ho)r_F7UyqzG=8M{lVQdW!?#;C=U-oL} z8JW=zNRic-_@hfv*JD(>|0>3i+@NI09C=jZ2bmix(IOq5bw?up!56a_ohZ zPocJbE`-28cn{RL^CSOMVMhX+Z}<}2d^1}`ZN9-5HwF2p9SH~hxw!j6!{uey@n#Xe z8F;}2yC7D)0CdA4B~}W8upuWo$=RwQ_p&&QAL3*bZ^lXC>wq85XH^IW*R&LqQx;)E z!tJn9Dve?O@k)NZk2r`_8c7qZ?T}1m3;Gu3*L!w47Y8=tJo#uZ=A)%L1Y>HRu8DR3 zqwxLuH4dCC_mY9*ueGaBr3HF~?(|>VO2-sf!&WU8j;AT_R$Wh21x~ z`by@HF%N)afmPP+I{Z_`0425Ar1lEE*g8JPHyT!IsRF)yy_&3GWvv&B1( z3{MMww<_;a{Jpm~la3QFgwWrk#vHtt;$K|~1Itq%NwKN-q$Z<{j|n*LJ{0fxfDMT( z_U<|yXO}F8G+{I+1)1xtCKOhoHYRPEd9|5uXUy@N0evs#_%3>R|B4>v{;Q=UCx*(IE7quLst( zE5B0phMFA7ja(oIxw?HDoTPKHKfux^F;K0Ug5TL?n~?WCR+t2B!k~#pQ-4+BJ|#Ko z92d~NAU1S8+cbN%FlrKE?<&4WeD{xPLc0cfZi|Q1YP2KQ-e6v5SO(F)4tmemP2@wm zM7!72>bH(quTcFrC~VCU#LJxSO+k3??;ggFiBCi$okL#}kqw3>fG55sTArjMX_)LP zHi2yONQ&gX8?B981Z9KdnMi^lY!1a*#g=XmYBv{_0uK%Z6j`BXw{o3TzPSoFHXNv? zSK37!rFg%ZkQ9EJs>GC7ltZ36XIuVStkt-u=wZVBmw<4LA1l-V~M#7?eZHCu!}@Z&hDLNXH8$=g6O z@sqV0&v83Kf&Zqk20IjM1*V3czkw{&(dIf0=%r#UFP?&71u&R^?n-fzWuJw$YREOG zIEM?Ks6_I}@g$Wxa?#xQ`mp=wAj_sPHt_RR(?&_2nQ@#G!YXgA=h~>xfU+XXZop6k z-n?3$wUj0-0`=OJqB2>@t5B?S%xH?rr!xUQws;G*DdiO$K_<$#9IBw$2@1m zxE*(DpWd_{HD0Kj>xr31`CB?(3jvz0w`r-&UT2diLP)qK%rRP(?-5`H>j_}0wcL`$ zW;{q4>WeA@{j~TU7?uxM4B%99@Y2SX2sqcVif_5_qUeUS`_qJ1o8JEGD!`eHi z`B4>3_UJobM1BLlNUt{0nc!s7Pfn=phKLDuUd zx~!|@<_A-}NO-@K=H7FTw3@>#QL(P+9@?%2UoEk{xD%7#~pVatl z)PruGOBXV=*y9U;*)D5^(Z%;PAZ?bl9M@O!eu`P)Zy}pm+>E5dDwJVoPr+2}mwc%E zy7m^^+~4(_cFu6T!Knzuxh)ZC!fWTmW#pncFsOZKrCW6rX?UKh5=(Cm~Id^TCPjqqa;8wJsDy$g|K5GfDF^B&t=S2Bn;AYhJfXoGQP#b&J z32DwOwEDD`7@-cb_hxUd5rd1Evnau_G`As9KZyaiEy}Mf=R>lfwP_MVd*Dk`&2f_F z0HC|-TKuM#BB3;4=fPH1>7{C@H_M8u2nNdD*~NftfC8PT1eI?CczbV9wMpMYQC zPI4sRCMVnXKu~5y4cv6KhkkBdf^%u1F&v1^)nt^Oo*mCa zIOJ5MIJ=|I2?z*HK}Q$4pr}MH$e$y79es>))-Te$8&N}Oonvg+4zoi4EPxlq75N13 z&|uqHmEQqPN*Z!5F-e|lkzSw7Nj8*VB-d|IM^`Xr4s7s+nA+TRG%ip1!L0b~>X?c^ zPmYH=_4wukq-_Vd;3BZaiVU0UCnFMB#1FIUIMA>Gco$lX1gZ?W2BLA)qD-i86_o<8 zeTe@U|J>0Tqo(i(bAn9Y)S9SqZ$k0)xaw6GQRnX?oTR(9Dtte?jv7%~Jz!CmiOk+r zNFY$1;68X3K~PE{7#Xr7M`RY{gViduBg++d!A=a;hWw+58C6zwv4|;8sndO zmhTUXQZ$DZLVMP%#+j*B<(z;QGK;Ky104p$Bi}=Hz6Y5~5X`IB27?Ms)W7(&-i1E} zyCcmOjPUq#K$B4E<-e9f(~GTILBI!UEX{q_s8NWM>_H!AYcB=Bsrd`}XfF)G&!TKV zXt{obRTvzHNX?0B73Xo04bTAh;%QVfiUWX;ck=r%Fd*=A;?C&=ETe&a`aaFkt`-zx z7Cg3S2U!deqyMEKJ5P~!9SVC3IsN<=?cA^C=M{SvTL1**kXp!IsiFJtj$9%RvNLPA zM7$d?6rOF$e}Qpj->u>2&FmMAWHIpDGh{K~J<2UAzIvY{KY0}h{3G~IK?SC^7rUNk zFJgRNu>;`&DeNofxQ)WgzY+`YNj#^svuO3@|N1NnY`BskM3LObDEBu%Is9ATp`~^4 z|Eno%XlX4E&0`^CIE&{$rUf{5D(t(f@aR%LjQQQ^GaxF|Aga_njZ`~~Hvs187nZ&T zM@KXp$Mf9Duy(IG&v=GHyZ-T-`O0I^wjWK5frw!GKX#gyrt{#ZX@vqm@)hd2hQwRR zo!_M66A*-j84b8jZ25cCTnW4MX=DfVL6x4?=Z@RG1S-7sLXXVztr*2K^9*JzT*)ei!A-@6a-(Z9`5@isZREPOpK6HrG>Ir_YQT^t*#!btWJ*qCtG z0EV9B>QF3~JUf2pP~q=@Lnx8l@BL2XDmY{kOx%w)T6okG##@y)xbPh*^E%q#GW+`n zH9V~HBb$Z8+WiyJasLLy=z{lGg{UV#x7gy<*uGf)Fyc##cFpg&i--n<*jAyQjzAxd zH=fS0#r)@x6sXS1uw(ikYms)3sSwnKx7EoI;$P9e4o*%wdfq#QFc|`v(l1%w!dkfw z88P{3zZAt<=D(ezk&eiH&x^>OVK|`iXCuGKIsZ3S{N`3X^WaRET@xyd{^gCqU`zHC z4MWVzSHhRvQ|D+cEAP|Q4T*-yahNlT^hT&OLK{4_Fxf0reAl>6B|F=hU411|<~gpk zdX70CZ01l26*@S-BmJCrXi#ykRr0HtZ}*9yjbg7EGOE@5D!IV9HqcS{F;^S2wn}q} zCtcURwNQn!Kso2HITWx*I&Nv&4h3xm)xs1cN4KDqG&aEZB>I)02FHyTeEid-h)*T= z9o)}S_PJ^`pvEu*(e*(kGj2s?qNFPm=_|=yM;n(EinT(axa38lP=sEdCb9tsgtrJb z`xd)$hIAS;Uq;0u5W*_QBWnJ3PhH^z)mrPRE*y(-2&)&w8$vUaJXhg`j)}veunWwK zMk&#incvOtOB-C6^DoZ%4NZGOlQ!-4h@Afhy#wbZW6G7t_c-yI`!tsq(6BJ73+LieTH_bT9hP* z)B`2rZ#lh2}YT+kdp-X~k&C)62*WPPsonhH7EItQXwo0e13Z z&Itw+NSsdx4kp2f($*?e24z&6@hXKbpo5adx+%GSS$=h;&!Fc!3b_~aYP{nNI=x(ZH% z|DsU0jeMHgbNfi{t)zY2;Z1Hl-aRf%xP$%iCe00R4ttYA$ytS37K!HR+<@KTf%pLL z>up!ws`emlmqMQ)6x~Z^6S{uhT4^MJFRc)ZF}tvdM3QpY7GsUNDTl0Pa_PS72z=+| z@2usrx*(kp2Aav^Tx(l+hiIeb&Bx#Q3)Q?br7cV8i68P?Xtg&s)}DPj95IBV@p3t} zYYI2awbRk}65^Y!Ah(;`u3t{W_9V%T3TO0Ln2;1`P4}(a$hGUcH-bF=!`eh~xpbW6 z;EY^_7=(4ZW3SCMGt=Psf}E%)CGmZk#QwVgVL+b0fk0NZegr=*C|(H}aXFT+kBise zIrzIW0Bt6+PdRM%bh?F|H;v36_}fhOFaCBe5-e2kkQQ#h8TuGNLA%Liio+&5-bAp? zq5YMTQ9Y|@KC0QJ#b^{z)dKyh$dTOWkOpGm$=zm8?#+@X0ndz^g&(y5A^=LlqZ0vD z6}TYGUx@q?ij&y;D+R!QB-AQS!i{pl8v?Jkvu~@o^$ws_)p~-_c8*p@nCT#))mlKS z1iLbZqm_FNT0P(88$sX-UuUTo->1A}P?kgNdQi;F{AlY6sWeTNd0E0oO}9UHUP(_Yz$$f^nF$@@wgd!7!$%QycS0AJ7BkLZ1zyv8Yz{e@vW@-}sDKxR`SKu;G zJ<0V8-)+2*@_3VlAXIa%AhluO3bgmc5kI^_S3Y;GChpJ4>BdYGt6$DFx_e|7HEaxc zb^HOEhu1t+DRiEk)%5|eJ91!@RhXm=NrIn|RmW32NK57}^H4CXhu0M*$^ARZ(7)gzt`b&+@Y?3`8Cj=g9Oq|WNTQ<- zWt*DIeH>*M3V0aILG|KR@$8h`%h0{FC_KJ{`T>fXInY@3m^U&$io&{Do@_ERt*jBmG&K=KI?dCFfO8nyo* z#c9YDeM|0(Li+%At0^!`P(pFV?>}}%)BB>uNKSG^g2NKy?sZ&+38SxZn-gGx$U<>U zeKYn?r+?fkMcmf1)k|Oo%&3sjSK$ueO;(X0p<_=^FZO9b0esND}nUL_(* z2sQ^5YQ7~XP(M?xbNSnB$}{4cZMS#{Kfe?|{TP5nkNX1r58ms;LNstil%ETNm89(X z5cyS*yhdZmjYjSxeLd%0wAabBTleWN$KlI(ez)N00wtm~t%z}+u9Ec~%nkT%9gI|( z#-~lol66ROKLBlH7ndNUms1!H)GZGqRi!1!^;|w{6(=(lg7>Qye~Hhwd!5y6IH@v_ zRj@8Z%OuegX5_^o0~ruU2MuvbvSMAutmG{;{C>woU?(svG3VHcJS+K?sJj6WV_!fu zKw#ZXzgEF7Y})^^9*>0PdGFnPo_E1tSSMX@0L*D_SAwcw7{^N`oX*>nt&+QjC^cmhxb5*a&Sho}NM7|E5XT zF~#$x&crbTN4jJXNSrav}C>GAZyF}ktO&Q2uvA#@F@T8p+OOo z4SX)(Ax40NggpLyv>6C5kDd$0%*V@be!WGEM+Zh>0CC8`KHqc#Q>8jOHS|x)aduGq zlc?Dntw}BiNHV+f4G$7#RpJf8+C#gi4J*BNV!R=6NRtOb@4oJymS?9vqT6IOBld`798phM`8Tv?Q4X{2d{ikR_|UlW>)H5^ zwQ=Rw2iBE^KKW$=jNy~e^HPut0~=!8*JsMEV$9Fh5y=D|`>=|VC!U^);Y@L(k+qE6 zvgB?gCaD!=llTJpW(qC?eDm^3&IvpW6$I~030v%Yxv1s9_nd#G_cx38ZfbC{;FkML z;@tALah#A{BNFU9eJ8gHq4-Ul0iP^hy@2tTp2`z@afjqyZww3Fax9e&;YVd}hpkib zI^);YQP&w8jyIAK<&Zqm6l6whc)CAT=);#Pb84a?%UU(#Q|y}iomYCj5!F%+j!V&;ql;ZRW9Po;H9a_{bf+LG%>Ptln1Ayb zB!S-Sy2Tj=Ez*wHd5*2Qt0@#2#(Hk2W}i9BcZfF2_oMuq2Q@XP?_uQ=?`~cm;cf!- zF2Wrp2g_GQJZn0m7X4u~!@2J%6#gjxnGeGCoz3IbL|9bmb@{jT}0^T44E( zJWZzLlAHSj-;x3aZfDdRJIGOQ{_)Y&KUFLuueS z%5C5(Offj)m0>A7J;(4!dvmlQ;5>D7ek%kLJLaL8W88meN*$Eysl&EXLuTBec!wQh zkE<=8U!7>*Q^ze7d>K4*3+6;Q-C7FWK5p@7G-pbqpd2Z!m5OiHNx)~C8`y)a*l&Q{Sw7ICt*v0 zFnL2Amf+d%oJSoDO7YY=76;zt;MM&ryH1W;(PTf(cnx>Pkv9ZxL`2dvk=zon_>pY)2=lQx%%f8mU*=Mw7Wp6QNuR&_nto0cAL^c~D1373ud9Z~# z1b53mSE<&nI-M>Nx@e|2B2TuKT}<6j&f9b(v0tWPfva5qlwY|*+2z2luN_D1dfsQj zAX_+}4EGo6gMWc2q&NG4SCHf$U?19>u!252JzUaH{w6`Ewf)TYw}k~9ugm-S1O|;m zvwI&g4Y7OQJwa=97EW)QQjh7Z5KPMFICoo*Q#PL!D}Xe$0lPHMsz&IUx3GANvW3# zeu0-W40Q2vfYl6a3Tm)UWTv)cyDQYV9A&gInyW5q17G(zE6#SW8iq&S4wp@8XW$rf z^$(Q2&lJs%x#Xmd(D?;WF>IlkyQ^4jrG_Q9=cgmTUPwIsDK1z4VYC!Ve;Hmqg+`uA z{~Y-F>L>Iw6Mm-G(a&`Fx!^ec90Wh#qTA#Cqv2=q=k#_Q{7l$LKMUdK)Q|C}tAubM zvE64>Fe#Sf4~%>0#C8Vub#iF1T$C}ZA z`uZ&-R|mO`fOgSqR%)E{0xQd0%t+P|+mW_-9jThZEt&)mnoCM@ZV}5zBDqn+NLj~3 zJlci`I%KjDM@|ITb`tW-0*{DS(IQY*BxkntJvibs|jN=OJ!&@Rc_zIz5A&aGf zNp|mwL-eyJBEVIYy4B&6)sHFSq}o5(u1-#&=uMteP!CRy&&fC%Evk>$l@EAe%28EN zjxM9D>bZorWa& zH{#+hS}+s^qluw7w;hHw0LS1XWir;ZdFQ7Jj-tc!8Faxr24HG-H5oMps^lJzD!nN< zp_k!o2$bmiALv&t&Sgp(@>9xyoKf4F#UK5XY}19>syhl}vP_$wI3LS2>j9eKB{;(| zY6~pW_0MsZ$=$yh%S8EJ89HbMW`+4%sl_QIj6T#>i!Z^S54Q<>-b=6UYAeErMaYZM z@w=vEM#_JY@fRY(E#!pY2MC1{U;B<6fEN?Yh|#M7MsGp9X1@?4eppI${4gzfVG8O` ziG|uh)UY5WR{Y|>5pVbZC;l!1@|J^5Ve!*O`G;eS%8o|b#gHz1k6P9rszTAYp$xfJjz<6rTFPe=+nE5y^K8;A%LWD1)` zs%N89yjLCWjbo*F^O8Vaa4Sc$a2$J}Da^tV(|X|hi`kvxJ&#fdKT@;`X}rP>qC%Qh zfkt4iv7~0=uE}KW4^aAPO7mfW-_7FxZqDK8gb^ec|F20uSNj8=QyEkK*QCcIB0CoE z42(ngjV^n+w&Z9G^K`(s{|H7K;E~DqPGl{ca$jnu1_ZM>B|QWTZT48smjEVy zC4hC!V+4MTaKK)C23Z?wR)9bla9jcvcamnS*I=wT;Tc?~ctKQ>I8|i-#l0?4?P87- zc-xuP@wdrhDuUWfPvfO)d1;y`r9Nw5zM+&hntmv7ocu4~#3_Mq;Q3&r{0O2>--PCg zU(;l~jchjnw#A*96-lriswB_rM&bZGobla?uJg)S*I{Et9M6<67v8HwQ+(F#$XqD* zrl>fDaT14EJ}Pc^xG zI@3uUqr4@qqFMv@=aOC1(}C)rrk`0bm|Dq0gvm8E8Fs+rOv$qb9zi}TOoMvc^sn9T z5FD*lc6D^RtQMx(T<4M;ZQ=g}$@3xU*}s$AtK-C0T@lAEik2TZr^hc`Ma~v8(>Q7R z$Nr{Zo-#8nPZxZP6)%;W_?#zqZsnb)@4zGj?Q=u@;btWiEC65ttGPf}pn4kh%b^JyMtb~b+dKNVT* zR-x5(i?7ujlUd0%sd?KyJDsVB9e@O4*@tl#7evmcM!d<`ZV2IMie1e~2Ap(Uj%j9m zcIB`guNN*_f;Wk1T3IQ%Q;{LB$<3q)Ul9s|G89)#WBCLW=* zNI`W-cGVhz>r7(LpTPkXB#n%je+&7YzB;?JcQ22Cclp=n_@0QD7F26-B%=E}0aQBq zWKBtHI!7IG?MtA)WG%j|d~z@M)xJ{G+c)F?k|X#pmyh$0gV%8W%H>O?=cN-9@tO0{ z;qkMk07btmIgZ{^JYmjE4M^vFAnaP7hj!;q9(07?*D6k6uZm8ww`Y>$#wnbpith%b zJ3q;&-l@?uB*I^7fZ~Cfg(NRnA*3qT6cAOB8ej{yE#dKDC{GTv&lIA1Zn|T9#{R&q z;c=Xgv+QC{SH6EeK?{yx1&kn?(vgqaalK|kSw55OpPt-in{_AAlW5Jig^TtJABs0b zq^m}I7+)^oedH(2^V8>3@G(8)g!(|H<1!3_oD8@Dlc!WuHsvdJ(QCB-9TFV+#;|yv zVSLBkzY|^B_ZtA^_nA81L9*me?Ax!zfwmnJ(bGo*@u0iZjS$(&amG{UT!Ayn{SvoD zZ}xf-2T2>7dVglXP4*wX*|1RKnAnZr%l6C=2T`hi{a)j2KR_DzDW+fk0PyCSFKLxsweJvfj) z>r7Jk0!+*Vp}}<~!O_`V zV~v2c*pGFNAJDPr8z4NCqh&8G@g6o?L#P_ZKWu7|;Evkk2oz_UO&OIQh5I}oi~6tB z%)tCedMI@LV|EV#6hI>p#GW-rQ4Lg&@vZk$T&9$Wd=sv-NpO<8&L%jzA$>-mVV!2< zrw~}UYV6-%@*`lk#>;ge&&v`2rX!#hNCu4hem?5JJPlvf0a5fXMm)igElIMhW?#7^ zgOjmw$mU!}+(mYKe90pOPyHEDbRU~Flhk}SKRq)kUDkrnr|>^QxQxz_5%lGdaO3C(grj~GQPV9afN*S_#(m$2c+p}} zOk^Qi{&a#!%oMEoJk_*PcQ*8ZAlatk5OxQLgI5U$^DEf*`7VUwpA#fa+!M>~w>?mQrR4VT*D3hARp|9x{#wGi}p|VK69);QUgm0TG^{2kffyvE^`nhzaq{E0BG7yf~Tx(tWnOI#0C z48-ri!#m=Ej~hN58OCG*YhqWg)zl*$H5)sB!rk!(te^8?99p2U)EUCLDIpc}uEcYb zvP`}HmxFkHE$@jQR=XL0rsreqi#5VJ7PGtDH3{+0k7Fk;YpX zucc({Oo^{CwRZh<^rONo*u#>EJR-B$V!qed?WgeeZ)o0r3M&k?pYq4;i$Hd*p@WS5 zC}6WpAZ&rIntf?u8dA3H&R|d(mdee`sh>2K<0SqHpP&5Nmm?d)_@!|E6!}%wX-p_N zNd&$`=64P~rwDr&;x`%^+T)((xnh7J{P%O&{76ygyb&9 ziWsy>a`TV|#2#DN=Ar0H_?=z3&UtY&W`?(KcAX14uhKH^>|@DKShZ75Pr(0te5Bh#R2H*9zHysQ98HTINZN4&UyYQ^ATj=Kwr` zk9R*61mUUvRTQ4~qbM;!c1<@K98U<4C9*mBIQ*+|x*2tdkel$N_5VIP-k{4%6}*|} zLd{&Ge{YBHIjpYc48{|wDuJYU~DNarwj-Ysm{^B7_owJ@+J zDa;ls!HKLlJ}AZofN~zos4k1Ta3~AWHy+#%)3T6p>kC2R-mZ|bJd7%gGeJem0Qr)q9~yAS|kcq|BRrq$C}- zt`XbIwo<|B*2 zn8bX(oerJN{p?T8=fmkVAH9iD{_cNfKBI9y*EO3@^YyruX7hwGk?`XePJ$jP5hUo% z^ms$~`6$Z!Y32T$nB`Y#<)?3qSstgAE9RKx4^0#0JzD5rD}S_w{?=V(NFy&Lzxi` z;xqQMALpFId-znID1nOWNwBuovpXLl}C~khCr|N(CKGed8cD|r^#V`wm9D& zE6+RMGiuB8{psP*db|0VA?vb(WAC!~yIjSr)Z>8PP zM#fGUd^eJQFdx`%){YOJLA&YDuJRumcCD6Av*q#5R?atG$3vm<)^GlMZx`$1-MuNscw^xI((~hgY0LQk_)`fX z{NE(-fBIj?|0ONszxq@{GyIc3NP8Ur3%lp0G)91s-<&Fd$9~W-?d-!+=+GvK4mlo;xn6vS( z?4Rd7=Ns>rH-`2P+wk|so0A@6|BRoZkGE{YpV~jCUq3HC?Q*w_PyZ}y)X&SSHt?lg zU-*~tY0B#@wNh5@Z3iI{2BJq-^0ff&_*+S+&@*vN5e;d zW&aGGeZKMj_RrA%si^vU;~jltjQw+6XuN}}&cA}txsU=?`YsOChohhd@yZDeRL$thGOFp_N<6SXXu=uxZP7wY;5#_}bM0uAM%AXMB z4Vz+?_nj!p_lC-w@0b6)a^C&&R!PhIWmDLB6ZauU-B%ocdB5EIO3V9Ydh_#U+!TGk zc-QexyF32!eyJ-yAO3w8&i}0Yd-zu!i+;cTREYoi;Lq_tFQ1?PDQvwjWSwje@8ubZy(gIyYu;-aZhawI~=`f z4Jp_JAH&yeX$+V0F&xJ+Si~62%62x6263Dja1hEt#@f>TPO=u2Iy!%F6#PGW>AdiN zw&*$W%%FvQp@l^o0u5AKV*jv{|Tbg#3%mz`MLAO^Un8`8Rwbr zl_7fY%NpMGvy=YvdXSM>sDAIf^dNkHB(~A_hu9519YQ->H`@)jJQ>;zBiW63|E@#e%Duk7tm_^@CjCeDj_Uo}iC|6+a2`?5@} z{M|h<@5_?4@`JvX?z?_35LJ#|5_#VhvZpcQ9f&vk6R2s%Ox1f?&K~^&@m}-}Pnw-z z5H~Bb63;r`s~O}bIb_!Je|q|$-ekJ2)Ue~DsD|?n802Ic!9gH^S-yC>*0~bDh)5Z2 zA$tRxUQ_Z#*D%nX7^Q-%%Z)b#HvbL(j41z(X8suW#fRN^%4#d*TQ3kn>>P=%V4|Zf zw`S$^*W0U#T&jF{>-@h5LbuKuKN)c!;LW~6TZ#w$35E!JFEXol1>^1NEs}c=#^Dbp zO73S3_`}G(5#5i%$h@-l1~FP6|+Rl0EwyQss=gO*~r7NtV?skz+z`n&kcn zJ6G>ax2U5{$}ZO~qt)}BSs9ux%|*)&*UrREXyBpjGAnnce}uLl>ag~n{nFfLoQSe|$Z51AtD%t@Acvj%`1vd+gqK<`kL$W|CWZ(KSuQwh4tpxWkUbnmjG?k#MN4-~T#9!2bKk)TVhCty{ubA%5z_o>(-z6|61<&t z2D_Kss*xf@b}RW5z?0hz`h+tNQ+R};)_jVl%zO{ZrTh*cEoQAPK+-6d-+wh&MQx z;)WeTyOBBY4&+PdZysiLMEjAz9@yuuqECa8r@?^X=D!apIzIFF2jG{lKYDux1B>~6 zanHdad(Pa^lBXZF7G)$qwhTdS#mMa?pC(y7I}zm4ypoW8JPyF$KJ+Y{xgA|ZAVEh5 zz!)B{o`(EV_*c%Z9EJ05MW#N^5opGheK}MG zlo|jfKos0pi73V1{Bj`k3*l&OMBfVh;Ng58AL}RF1X`R+c;EhRioCQ?&Z?1I|27!( zf&yMpD)!uOsT4`ev)T~m;5MCXV;dch0ki`;UWo!H+vVBb-oW)^1Y@|g|))~^}pfvJ@P1V zLT!FJCcl!Vm0z(YCck3R%7eRN@+<5YqfGIR{z|7NvqPW zV{?8^BvrgZ^9H<{jlGAf-^{-LWaJRa?5X>Od9#1U@cd}oxX>Q5iS3_;ysq=(uy|w7 zTpu&f&C(?lfFzl`cSG^X)B)o(13Mf+Yht@{$gWYuC)LW6$zNeq#?IiO^Tn@<QhQkH&h2iK6^W4YRGbYc1(Ot*2?c%9TQ(q4cGQ- zEcwFQt6nRAU_;FPlBtzXY@vUx{EK&5!gr@u<@yWaqwEp?$bZ}0>y$RX&9USwELb-% z6c5hZxt2RbFCn%bO6v!P;!ogy$`?jncJ*3;b!W*}n)Q9{T#!tMy`9~>jZx@)y>K8; z86%u7x-Q{5mX1rBG`ASE|N2$;m*?@n2zS`3@{n^gA}D*+CN^ihqbGZO(K%fN>#q69 z3SVH_-yp6Lm#=h7B}+B5FPqe(owjMsRpkb;joIPX8iTr7-HJPC@30BbeaM8D=tPI~ z9__+E)c5~uG40c$_A45>SFpXZn1r=Na$}C2P~6+)qMVR_6n9L+f?iFEP%odY?|J7spK9`m5WMH8h!9C!FOf=%+>x$igh_)*8XAX z1~zoN5MFCAs4g%X^jUBA(>T-`j5OJlR=_x!)aeQ21$gbf@rE~t_Q2Rq%ZbJnRpR-7 z!iGGAa}qoQZ_;ZPvHSE51|AkWz4B{7V1ECOy9%j?v~V zHko|`5-=SG%h{IzQcsQ9)fBWUw@Pk|L2UIj*whD1(%eoMct%=xoz;CzDwQy3Xvcq@ z2-=i`vT6vV==2@OX?Fc&l-!5m$ttb85k4q1^=q(YV(Mszs24K4DrhY2`I^!+>YRw-RO01=0fa~*P zU9E!R_X;>#GccxorQgGIJd^+mU;z7!{r(#%UjPVOcJG8*%#au8jdr>`PxtS+7!{CE z8~BraK#T?pra?JQE=9M#M9gO?W$7_)*K&6c%H{rq8)S7_Fvu@qL0n=8FuwXU@EH2s?`L0LzdONDw}FW5nWc3CHEZ6 z2~hAYML<`x(6`?6oRK>>+l;UiQk44UstAigkBrS!OE3%`n2uz>nevaKd-$+2@3|tovL+>A{4l`zjK9ArB_RUYm@gT7h#REbsa zObf*$17JsxBt0MOVU|;HA}BoF{UbHH0+TGXkbgA}@|{yO$fr(mx7pV%BzDOf@S;t@M% zMnz7|2h$HnQw+ol(p^!9U>R79t*>R#GoU}6tlp|dP7)*70RQ7NHng2pql}I6zf_VA4j|V zTADUi&Qq?Hl_b@w%fsF=^|ADo7=|GwCcpE`{KlZ>=atCTnt3wD884yJAPwVZ?jQXY z{9#|236XoHAR;5&*S8NtqY5%gat;*r9K$%8a`aY{`7`qW(-|GhLOe_yYa-xZ60C;s~b zt;(vn=JI&*otnJ-RP6rqH2Gsm?Eba#*!Lg3sAamUgYWpwFAovM+Al!@GBY#J zbIj3R1}+mxxlOTR*cf!A#}@|N@>N0(H|dv$f)5I_dx7=k{*hmY^p(MQM?3BMf%uzN zTzoD}V&y9v!!94fjd(;!)=1D7cbeH!FG%t>qlAY3{W6Zt60+kXKL@Eqm$;OGvu#;_=X-(VkA(@~l;C z&1C&HtRr@Xf@Zw65`_}S)lhYOHykM5tgJ5z8>m_a+ZPqo?8+A~sz4EvfR)fJ%UFfz zHi|E?-xd`6Il@0YjRQ<1+3U*#$;t&82KMU!WI(6m*_VAQyl`}9DM+N%=}GM5auntI zhc1gk_xJ4V0G;1Rv2)-Jd!4hN2q@qrcjKWW7wkr=ei=JQ)ZMzeQL`v0qGv`Y4#+y* z?2jIT;kdQoNbXrg&a#){>vsllvYH4ZLd88ru}^-UACZHW^Yh;xvLh+O3YY@sx951B zPnS^?=9_f-=j{-8sfc-Nu6jKLZFbc>ZYvTH`5v;bd8LI+L}x5ejNCpJeZ+NtL*XIAfJ{u`+azid;CuJa;{j0Nz25f zy}drTgM&Kzt#}?l>Nh;YfJSs6Chdo%jUaN2F@!^dpCW8vF0B_JW9Lbtk%~p(>s3@u zk}$4UkWKjE9A(2@uu7~~jpTmWz;nz$_XFn=Jf(brj4)#Msa0RM=lJU%N5kvitrP>2 zErr%A46Wh-&tapJ{f=~;&$wC z&>Mv87&qn!+lHF~#m#N()Z#|Ua?=xyG+-0hlvZLN%w2mrh$Ioc<-;%j!`6F$;Bd3^ zoe$JXE1pAljTq5@qXGNJ;zphC?!viO**glg+pI4jD@BmeQRIf_2}K^{_6HBVKq&HD zmzXH>-m`=v`*I1M7#sD0l>$p{@p9^Q!D6CtIiU!?n*1u#x*U6p@TE)-?*Q-oi_Xzi z7SF^rycH2_ z^c+Gkzm*W|$rXfP`>i^HB`qWb+i4|$m_cl~6YBVX4UeCT_*G&3Gx*i-Uy=BAn2VO< ztvY_4SRRI7?=RKyYvF4x;a8u|35Ne`{MxmsMf|#PNi=?aHZL5%f@OqX85hOGue(bL zzouDq{0c4;__g<0j$f-6{Q-V?E1Kh%^2#6MSE5D7ul0|eAHODdiiuw@JQIyycTmNv zqO|~zJ)_~5;aS2jr9a`r%6hUxQvC{Ay#?@oR2*bNmYN=XuI@tMVDYgzU?< zQu1KOt%xyZOj%BT4#1qWVb@~E^kxuV`I#(An(R7!LU!#k$;zOl=AvMZko$ft)8aF< zv0;8rg9y>k$%YC-QWCbppQ=HLL!+lwSW7VqA&~Bqc;oz1*yIWtW%Y8*eM#Y>fgffA zZ`7>#(zzXoNm>^Xf7#n%VUA{hI5wvISg4gt&&QM>hiT=?EK^V;?47q zFZf#c<)?nXtM@PcLH`r~p#Qu6p#S36Vy;iJKL3Yf>wo%a^N(r&nq9Aze^C)z|D%-$ z7sZ&rj@t2n+LC(@l+1k46-;p4>i#%I)_$WoZbu6&3tPf|jeN64OX z1Kz^0giG-jk|OpS#O~DU@Oz@5(fOn7ot~zLUiS6R5r5P96=wg@;&~kZ>#EQi^Yt$j zueA2r^6!xtZ$Pn`$k7mEU zO>t4ivFlRkO(A=}KmC}*9>r8P>IU52?0>oR8zn+EYcTz$=_g)7GHeR_*X*WB_a;rIc6AoBb$!%f4F?uUkn{em5lC2TSCKFym4 zM-b^J)bnSrbGPg*!f=59psu&Fs~*%1t_RyyD25a4e*USgM-40JN{W8xEy$$UvzX+0 z6}quyZFf$=nipZQvM9BZBX?3r>!97gU}wEP?ww8X7mJ^P zV*NZ4@x6L>jTzr7U(e?^dr)LPzrzO`W9IX7`6ytc|Cd9JK_1VRKdo^Hy{=W4^|f2A;R6uR)q$tynkW6)?=F_%>XseIlaj6Ax%c29AGG07>;?xyM%7 zxDOblnaMWqCfvbTQKsPH)fiU}!%Vj3TA_#>F#3E|>qe;A4WM@6ZRuovyzeE>wVJdL z_Q2T%Hfr^K){UfY93k@AMb590yedfT*dV#n@NQPU1rsWwndy63>Du+sHP503%B*ak zc_}Kfy4ci@dDe#m4DZd%#_gW!!i#X+mI!Uu*fNUe{WVRrCww2+uit-kzB>yqPC?!j zZA-6LL*r&?zRz9leBbI0I=hN~pq9l*Npnik=oz}R6`W(Uio^vQ8gbO3fy^}7n}>?d zlbHybs;u6Izu#_YiN2H=|+}pR?}`ViuBe3mF$XYoR`xgp4fGt#7wJ(flM<&q^-+<9P%-UC_@!EOD`MuV%+Am3P zd-xkUy=L*}Zj=*ZoQKi&PaPI!YUl5LvH3Tx{I1g2{F_#OW6b-TUYT0?0}Eo>SA}RH z&ZZX$I2bq0uD*frCVePA>j@`D=wTEu>!HWn3OdHiBDn})`~!I7Vcy8Y_68jnF&F7x z=#16FmqII+sC)-X{&!^KXjv(w4jiJ5EgpHlqIV)^z=|8_G4J zD=WM&{-NaQi?-ui%KqwnlOyMQL)v-fn;bshPRqmRoBT)f{c>6Oe3Q>N-$!o=%{P1T zAIw(_k?x>im?@fB`^7l*1T{%PLw8enzKQ5~jLp7J{wv={`S;NsTxS0SO`^PxB7I}L zzC3k>0KQ|0cI|r;60AFVa3`hCcq}%yijTa4i%d&5Y8$)ss*qnmLaOuQK&@s!o6!r- z_4co80JABSaKib#xTj0#PMfq{jQtTc{=$Cx_^+q&&x;&?0S`zmh35EZ)_u(+1q(uV!VM(Lx5_CY|zF!QrJUAwwEod(`e#1V}nV}8tj7& z_6Tn<#}wY;pUnsKSNzF<4Z!*m4`dbULyNVth-ixOPM9$<$Qd3@H!e+X50h*5sC)%7 zb|O{gf>91;Ckt*OJ1<`jdEHP3Nb*cNNXgr*73?NM$d70U*|kvG2I5L)6(__Q@kSIM zSVdzT+B%syWy8EfdO4bEuu?3CG9WJb6&X7>4L5K{quoKV&#YCFI{{PEzyQ#<38W;& zx}5F4T$rge@&g|4`@3-WCs?R8XvO*Q6>!v}*`nt?ZhC|52PlQhL7(+g+;Hqs-i_p5 zL)yCLND$S*9H~!&K{vo?HD6$RINN)7_GS8YR=A#(7i`fU~;tZsDgtN;* zz9uf2myCR!T%;!X2T&?GQb9?c06xqbn1h+~uy#Jw4%CzLllzCTvcPtH=h&n@BX{>+ z;4?sgPB`6}-Gs@~0F&hYv55?INrVktXhY`Lb1_;?DYwclYb8&IKIbyB+pUyb+xY2G zNw2?$*J}Q9R9^ZR+9zq#T|ho5#U^(3HRKItVt4dSFl7G-kZ~-+0^nzcoeKuhV&E6{ z_?6tgrkqaGVgN(~n|8HWwJAY7TCG-_XRfOnBzv`|)8QrfdKa`b0q* zGI>BtXFa>NJ4xRrEqVaV`C%MeG3*s(YeqFt!VXlfI&0Z0-T0;)spaebBl;FbiU7{- z4Dp7B_yp(0mXCSRSzI)~pvGtX0tljNm`-ph%Ai|vL zt?j5MDA(jk_Ee()^;YN8HtzAfScLjfNB{)wMw&5Vv8;g1})xkM$Co|r+7wngU@JIYLptQWcj_p7iWe) zB}WqA&komVqvBjdE+E%qMzLBKi&NmFD7^`DsueJ$nhPa)-Xg$_fUSowaI4vM&ggi; z>S>sH1xgC8Z&GAv#_T!(9M7qsqphuAubj2pBH2&Ku7J^=br#wd9?QroKMtc?kC+Es z1)My}UUvPx{6JjWAokc%!tcDCh8-ryb6koK%#&S(CIdOWS|rO46g}FmPD-{F+?Se! z`)6erGJciZ-UPbh>k#6nghjnu)NqYY(2epBb)l3~GfmbJ@;WO*sSu9XlD=~;D z59iGo=zz@_FW68YpkFxv5Zf&nRpHCwsSvk(n*eM;lfQ_)=4;hjo5fjg6N&L`BB7>D z*}|9G)l4-H)?rlnW~8XfKoj%#Niby8Wl(zDW2eNZ z=b)=Z9VVZ|s0l%~J(aJGdomPQ)VxZoXQxz(OrAv@Q)y8rEK*06SYUquuWj?%Q;kSt z7Um(gKzE7<&!|AJv1OJlXQkxcNC#{k&w}XLFL}`M-aPBBWCP(Nnq=F1J1ZpjYqazq zOd@C(RoPs=6nl>bHuNIg65DZNZ~rQM>8&Mf-jB2>C1`Bz$}%|aw5j)`=C}^0Sb;6{ z$f`L>cAbIU_dlFUA#A?i<*XC1`Iu+FV-^;<_Cs&x6w-cAp_SMJ+f%aFlR+~L5R>s% z=)Y<#s&f(Go#g3HHyJwC0jOm-kqUB5cJH(l*D0ek_n}s(&RoQUcK6NmPC3MW`b9hV zU~dT4l(p%kv7mT2q~kw%kX2F&3KPrkgz4hQs1HryBljiYd*$8R4i7NN(63Cwcxy`~javpsn!dfIevgaTjVHbn`g3 z%+PL-jfB_ch5@I6WNde~2O82~0BPp(v)Js6)Fe;Pxe-ZGMiBXdNhYL1$qhl>Cvojr zF*Z(#h_Y#v$ki?Z*)mptrJzF-gWRid%I}1J4xpe439fPQRj)^yxX*&7iX{BatCD-e;0KX+-qa*=%Cjl8l6wVA z#|5a2w&JiQ)x1SnTckPHAXiqfTi$Xw(Wc&)bON(>xO>NItraE{Goru?_Iom(NX~MQ z6wWjR$tYwWruky_izoy^vQ5p!7zGqhql=(Przm&=5adFW(XJY#xi`k6o@sOvX(+FX zCu`tZF7V_jKA=WKFq)ljB(E)#!~xgV$Yn8vYZJV*_Z|(Bd5!Zckr+yH-;Lfc{gZ>j z8oGnrpviNuccRh0WmB!NyhLc<=5*irqThC!+Y)CUOT&1QwvZOeE&^H=rB#lC7ig>q2-OA}+<_gaVx@{G$^m&jUQiQ2u~aTfNnS0%^fs0HNH=MJXt-Ly#7bw$MwynOS=$J8iJ<`}>~vkMH+9O0(CR zb)PkB*37IKLGgg^5!Y>1 z_K`nwo)Y%tZMiDomGV>ZITF5+R3`LNSF5_2osc%m@B9p_nJ=qL;NxK3<}?Ja*UR4u zicp8->BU}P^~{K{|IPRS@<$_8`$sZf8-9Fp3se`((3dCFN~49EM7ykAxw@~~hkJ2t zwJN@3RdDX1)HBv@&t9tXvVx3zk}p$*IF*R~u6ObJwFdp*0MKSB1*yq~le*gtIp9Gq zpaQq@9VwNDp2TeCrr4#s%*u;U9n=-LD-?vLxDOZbU4q_y(Zdy-o=a* zFBjz5sevp4o>pc30Xf5f0|RKcUYbB_PV~{5%6XqhpD8akYW-FA^`M2JdB&^q{4~zH z3}pm?MieEr0vVs)$jD?GOmJW_77Z^C*_)ULX!`H5vGwnlYx?iUV(Z_v{QIAet$)|@ zUs@2$-r~>Y3u2AiZ|a~_MA;e1wgG2z}SY!A$foc%VJRs3$h~ z*CfVX+{$G8K%>erg|VIgffjFPeN`4facIQ#2Bc-HX%9hK@N4aT*``Z-QtItjw5sps zn7f|=I8G^##(yb|qB|=O{davvmVZ^qDJr#zY-JWdPLG_UUf;6v53}+=2i5Xa%tCc! zV$(MrCuz|bwI`M7t?X&W`$f1eY}u}2b&O2TTxP9}c$owubFph6IyOP^{!TVMIw(#a` z&V$Aixb=1`J8#tD*K(dy-AT-U1@uyx=2q2$Hn<%8r#m8P8q#iQHFiN6q8&c@-HDZ4 zj+B3y@?CdgC6{yVH;79Iz5|m}aE{#bQw+!%dUqvYR<4=uCDjSkX56tP$lSFO{T6l7<)`Fa9G19__KTc#Um(e_vNz3Z#pw^> zIx7#`E%%_i=4*7Vsb?f%{qvbaxBa|FHKEBbh!A+24qlBYEZAVCNjTO18CBcA5)SH z*3yAG8`!>BU$=YOLV*WN81J6M%EMj;IAzY`Y#4T$qo+~|Z^`Doy}Csf0e7{Ot1SZX zspf7p!iO$~5ni!Kv)fhOctE3Tkv1#JP*txI$9djlb8;pXN;DK|%T~&H*en0MLcG_C zW6Z%^jbB27#nx^;bXamM;iXi#8+$N*eC619oF9kuXbKLVyP^R|=?PRA9Z*i0X(l`) zj7&I6PtB4ORO#qzChcrFqRdXfvn3Z7dFfGDg1$q%yAhg#Vr}a6N*z@B?7Tr`C`%pxw;Uc|GrSe?7x}E-t5-#a)W;imkb?=9o zl%Eg+u32z=jF}!eKCWV#BOLP^NopN;7$ef9tKu1TS_w1igXO)enW;W5hmi*A_4dna zFToX>l}K-3spH;#yqFKSl8=ZXO}_33V$d@kF=+jiwvcKURtOEuya?8F4!(%Qw}K2^ zl)8xtWIm4aT+Gj9`NdkkpYoxD%s590{X58|rlA|rsSffBk4EhTA&ZfpwCDcl+dGjw&tjzAF`3Wdg3eBqK z>w*?^ltAyXlJETu`~M*JzfjlzKl0GB|G3#2asg|d2V2T!7Ubmd01bsE(4S%W$Kqd$ z0$=4c3^@OuKr4+Wxf|7rW^b$9x?0~>YYjb>B9Og%AE>kOSE{p#{tUxER3n_n??9c_ zd|VOh-GL*dPc_iYtU!_wC_bmjJ#Ze#Do%$@PYO5>t`&hvXVJIG-$=`s{3ukvYB6HO z1?>5OhHYc<8GkPmOw3no2xEW6A)ukxsztgX?7Oq$5Rj1pSQyE5Y$Vs~hOi$-eBb?n z1nj)Qpt1JQ|M}*?Lu^OwJ%kON!G>~l4Lu1B1%9IY1Z|m@`vQtwHiwYzBPc@XW-Z;Q zD{||d41WWwNXVaD7y(1rZu$rG>;jyN!m_3vit-$-MJ z{%xBK{Y$2jCN{UdvE<`eatkU#$qxbw91cP<3LFEalzbse{`H`?67N#-SFHI>VdzS> z4&NWd=8>IW^$%V1ld<_184fSQGJR>B@901>CZo-Cf;fipKD<~Sc$XD;i0VX7LQ$JG zbO+S=H3P{Z5cI>c>O~#Megqlb67?Q(L?7+_!?4)z=e2z9f!OcoFVO7oo{s%~Udta4 z^S+imU&|jFyZwAE|E1}%{YPv0kH>63d0B7G{{F_;_JLad(Af3oYx!L4@r}^(Z+$fO z`;HWiUyj{=sh0o6{juLKX!$S2;q%qm^3*B4}C-gGf@*sg0e8S52zYf?^rAF=q3$g5NV zD|3l4tEiJb6uQC`F``0~Qap|LJPVo?I}=H1JIdg^5pQ&zvUl#7MYJ?pS{_jsR*hF-# zFNoEsz=IQZ@0-X)(hV8XR}P38uMm>FYpc!pgZe#C3Yz*jneLs`6euk)@zchvWc_r&7r}lFR$E0L}*3ZCY zc#SDSPL8#zJOLnqZ{b6*UZ${$JtV= z$S2v}%#pKAXmO?7NzM~o%XKJt`vOA2X=c{rL%mu_a?HKtk&^+wQ1@F zwb-PYJ#F6KG?)Pl#u|n>_8P)WOGTIWDojgvg1IRSQ}G&3_e`d!L7i?c;QCI2O5O~A z0N1bDOSryn50%MF|Hp@osFwpMa`^@2UOHkqWTxt=*{ZWwmPEIpR5+{G3Tyx(p}G8HJx8Fj%=3 z@$($(%A?Do_R%e8!uaXFnDFy)wedfCvVHXF9~yod2tRF-!;Gr@m+;w$O|tbwxBV{- zCM|72D!Qddiv$>LiUICvlKIp@%}+Ax*WKC}l-gJ041ptIe_vUHk~;XS*)f^S z2SvCYzMy##L3fA~$)q?p1r86i(UEgg(a}2m?a#aYJyEU(-zx(DF&HBD%5v6^wW>@+ z6^MMy9N%1r$%&EV4JJws0=AGmPfgy=lJ|8;-p`UZVsg(&@?nd3Hy~6inW6RJrMNJ5x6PXi`NdN0pZ`jY)OR>{h!9r@bJd2Ro`2Esd=xr|OqJ`ARhrs&sB51`owx6{91z)U|l!F{Y>jrI7N>Y)vwD0-wBJ znw0=MWhhc;X7;5#{S~=|E=Aq23NY(s=rEgq{@vjBY|6V{ega>f<~TWR%$0Px#)|i z5%jQqhmr>z-VbWtlbAPKUml(P)GKeQ`FCJGAC=ggUX`L#yP5^tGX8w0*&lqKa5J3<9aX9l|DN*rs!wVHo7h3(w|}Y zM-%-t=aA^<8FpVkupMjN&`}`v9@H9`(#D>xq`4dxwY=(YU$!4#(Fxi}XB1|^dY>SXhD(zZ?u_&yC1qPs* zr;N-}^5I-8TgD?7-$-)l4M;K@dC}olR`*4+3Ghtgk7S$CPKpO95BnLF{o)qIdm=(R zi^H^Y3ZtC`TVv8rt2IJ9KOk+;u+L92+IiE72<=SzH_=Y_Nk*G4rSTOMLsD@ZO!)Wr zYA3vsF(LtNu30dd$=#WFqM4w^;F% z;SqBCvMHL}_LxDBd;tuOXyvov9jR>(s1IlXqU1FydF79*IC`;~F#LFg-o639rR)@> zDX%bkTO6jh#ZmP3qzQ)q$U{VLi#2-tiAHbN>FMppHjUoeWf!wwy(vLkb#IcM+}3RA zKyF{Z6$rsU+Qa@zH)HW1p?IJNz3reQX_+#)fj!-rj)ZV(n7*TGM@6`k28M8F9T9|^ z)Dhu?3pyg)HtJWur-l%QJW|gPuJ3U&$SQ`h!WdIHuYz_NzlO)DqQqr4C1{USRwx)Lo0VD@*RJ~ifnQ&Znb;(dteQ}C4r@;8eY$~AY#F;A1~K{1*g z6g{bXxS&xhS(DX9bEe5y0^hV!!s_(s?^^z(*zJe+&lmik@1K`q>K`Ke zXKmh}@1HIG|9t-pjop5D|HRgR44bdz-+G?*HT<1)p7z80@3z?Ohw(qAeqz{sEr0Eu z=kw>XQv=!bQM5l7(y)FFD5ZR)H?f2wVuhx zqU)WpGx~dMGd}P!&;yL9@=uw`#Ejex41q#y9}!&eWqjY{9E(bUTQtwve~=hmx(R)p z-NHJRq`DZO{pTELMxzp{@0T4>^`Si8-Hz@m=3T_g2|KEj4B|;%oMPr@^z`w@s!&~# zIK1O~es3fCzlfK*i{0F}n&X-XoyZ*<<% z4ei%c*(W+?GeyM#RX;lTli8`=hk)J8>P=+WwOpy178kbs_?%c`Td{g2m{a&nqCtJe z;Vw^d0P><_2c0g+S#ziY;3}WT3M9ueVDy$-8Qp~IsT%3-nG02;`U^<@c?)4Ude%eJ zm@GwQH46>ewV__AI;dV7IHP)Ps8#a76>>Z1VR04yqgGGy_8X~MpIU8EE_u>%y{~g9-qHUK zW(Z<)^tJt3Y9ZWRjh`yK5q(()zwU&u(eFbshS?^HQa!0oWk7F%iI>NkY?ekec)XnA z;MOY*@g$Pmf3k;WL8(k@ZlMcNzq)p2Tzi}rwh_~wfHWy@6O?#;9bNU`KZzzz>zD=T zQ*N@}bl=k5^&T&cLnAR?C7EXCNNK3jQjAcy)!k}l{(In8n(GPcH^$&KxF6lk9Oq$+ zHtdFG>X1^?L(cD z_Yde^cdldtc2NdE=rDlgLTWesCjtM7&&>gdIEx?tj5R4YFD7hD3p@dp6zSu!^Wt_Z ztfZX$WCeYdqbB3h+7meHAEx#-}sN39>ki_S6xBzUkuYq)Oe~%_e4< z&%O9Lol5b}+uhBuhcf6sIL^H0W+Ysxs2gMCX}($cYfC+_U*)C4=xE#KOe%cFIFxxC zh^Y_drTgKanq`(Io4wmO?=bXAfVvaJPIS3xpK<5Fv{94OxR;i*8sVTxB?mX0H`~a( zV?|Y~y>o6ha^B|+tfRa%#k?k)KJv{{k{~^3&XF?ES+!%6^3}Qe5M{y^$H$WYhHoH| zGy=N8V)!bOJV+voW3WH@!Fz?lb^OS3&U3Rm#L}ULjw*&6H77F${z!L0 z_a@Ohu zst)@#>M`ft-A*m#MC~ut@2jne^#&az*yU+PyFAYb>49YJURXG4i0fF+NY^Ir#g_m^ za-nOFoWYI!j7GtDBybqg)(llUTieV!>zzzSr`0IeIQ2O%R`E?WG9L=ykfEFMIMz*m zMj7^Ea1ERlg0Wr++{F5OHS6z^sQYwB|7O6?V34jxrt`%e88p-Sk|jRyUHkUZVfWNr zauNi4^5%Vb(q%{&T1Vy?T)UZlo_7b8CNW<%QNwnVGt-7T`+6If)1=(lO5R}8+|5Is z?;&5Nxp(cL&$801;$UIeztKLZ%1F5B+{m5=*jbOB9M>r}CDjYvaoMRWt}!^9dt0dG z@zAnMWUmP@PEVXVu2ta&1`q6I1M6iS=`gr<2ESqR0My44jsK)`dSKFfDuBSPzbg!OweW8MKfmnb0Myvlsn&OM}_@&QH=F*yPf028QYv;c6s(5*btsA+ z;G5T0{&0o#l4Dhqjryw zvo;d!US-FPB+FDZ&uBV-#96aX6MlsKd+#7On=juH7dE&?C*v3oZ?-v@5sq)W5~^O= zk0z!98Pa#@=LsBl#cwX1QGzlHar&tyt(;z%;DoLai3pN9r?057ppW z1L#|QYADV9?c?I`*pp{QTxR0?(}iZpZEzj6yKgbL4#N3gzXrKQG^1VR(s7G-T+g$?ouO(&9os7SJ zit2u%Bd5A+FhyNK-aN+sRZeRA|MuAaQ?>j_w*SuiG1||(>>Y$mB3scvaPhuHKe~ea zXd%DoIJrm_mkuJK5h-$m<8^U2h{FAIC(<*F@V6#cMgOgQ586yhJ_W7p*Vt$Ohzeq(5|jOI5n>(DYbUoM#!~UT~q#uD@P6 zvqRMi(WQ>@JIp)yLQA7E+=vwGxfDwyUeE>`Bi^5WHmLpr>AaD%K7*-MR!mau$jNUJ zDKyhayD*25`uTg?Ni~qdhQuwN$DFTMbx<&B`cPi(cZRATb{{@r3_vl4kANVb{*V-H zAN~E+n)YyBie0+M0SqNz-UTIud41^5F#Mw_55FIT_|{2Vt$*(a?Ymg}zn~fQe^L7q zQji59z1|n#91N}m;j9GrGV5pj4@uD|+~?{TG1;u8xPyXQqVYAqL6d)uiETft=;;f-Y` zmcYk5(-s}fPNS0u|2W42Z=LO~>&?5HdviF$x160w9Y1~qO2l+|o_T!ybc(=K(#)`T@HEGVj zg{w)VYWz@3`22{pXL25cKD(}p?(5aq*J)9mUB}{;>W00aOHEG=KXly}&Jg zE$opMM(04i@2wRkSO16)t!}`(vUw<@^QMd~$5sCe-n8)e4EP9PpNDFqJw_uo2%lqh6iDLVO0@vm^>eg2i7b!uxo*G>T_h><_}>G z`*OZu99Yi{#Lz9p?l7;CC0C*ckp6Oc!;x#I^mZ*E~N2_t>lY3+udxQmul7GGGaKAsk(UN zQkLaRhCy^u9I$JYcwTJ9d!n+;Cm-TV>mkeQzmdM=fhL9X@4Q zp$eS0(EPte^@Y6` zHa25?vMn_K$-szI)7;Uyg4I_CPsO_5ImF7z^8IjmX?0MTm3()LS#-T1=AqlMLhgAq z6@n!0C43ab&sZnOJ+tvW(6$VNz3gDTL%z|7a?i8r6vGg2c_-4<0J}eOo^u9OZiIFQ zR_QJ?kQUDSsS&k@KKCJ`LtL2Z5FbmkiC4Mn+TB&A#B#~@g1ZI!bC=mFj*AD5s?aWu zgLB;mOy{^k%!7r7r1>50=~9+>3g%=6p!A7E<1=}dT+>)PUB&4_wN?$gus1ob@q!2U%aKsTX)LA=Hpz;Jgp8E6{acAr-hF zV!o06U0-Eq|28u3&1xo@QgTQaXQ<^%ET9-V+?oE}pYBY?b_~tUi8GjE8nT*;_7|L0 z*no@L?;r~zz2QtqrFeq;mx8UBby>xh4498<*ovt-|CM!@lgEe*VdSqs_r3d^r=+ns zTS1tu1L&tyS#!C{FnkL?f*aAlC107)vmt}pMyd(?q{^Ix`*T~Wj=#k3H2t@9TfADJ zP}`q$-CkGD4#ZCaapcLi)fHcj@jK;$RzXg%vDnFLV5(N4v||;8VyZtzg1%~Ye6sro zs#OnY8$av_`!w^Xh{`GFb#}=$9d&J2Dnq_B8N~9^%MHjlIQvA!Oj6}D(hRhUcV3MM z`}m%)WTWpJWT&1&@~r`euyh^Z&6a(r<+(UG5ces&@Y?`g+#}%R(}Two20xI5%}JL*|_=q z@UiGu^nV5oTo_w^j&Sn<4vpKht&5HzH0MR5Gv%MpH-+r-l7U4Aa=(lh#G}d&e?`4R z{6JnBq0D*TMD7N=+1}*T5W!su526i3(=EJZL@?6P`rdL-au;&ulo0y-`F?P6Hw-KfK3$Igh|9hYo=%7eSTe<_>F}taks*2UM$b6gm>*UOWYFI1C zk8TQf$9HzND#GjI@5LJ=SJlV{?nO%+8{ysW)6PcMJHM=9o4n#M^x$r6&a*)EQ60CI zOaZ>b-`}ZV?;MU(y;^(G!vSGS;U-hzCL>`1;%d(=?9Ez7xk?tw7?>u4=Eun7j73=@ zF~(UEWpCE9Qv&N$yKK3i{Udr0`D6n#j=TnDjd0;Mb;SbjFcF{8smw(Oc>MpRor(%_7ppVw}c9ucqA2N_%VkPapc*6bp z&`OYt*gwrD)$bMMs>-)FsTLs_&)#P-Rs2!)0mOMeHK0RL+!Kopx;#G`+iMc*hy?pw|X=B<=~QmB_1(33yo z_&(T%(^EmtXg}TqtVoV(*Jzb&slk8h`G+~^gU7W~jjLd-w54KS$Qclp_BWm5J7ZMaYX1m|~8Yh%tp zKA!VzfW*R0X|!PitJ|32+6ZFb5?fU7wPB2!xSZ~&4kDa^jprCNj}sXfiXOw|w;8$2 z2t0ibU6WS<9sK|+#c8Brys9=qLCXS@&uLBEr8W_K2b*~Mt$^!XNQ;NWo+)E=N-GK9 zdF3zYgTdVfwSR)=nry>c!o2bD55MfWYvED^jCsF?3Se&7orvkX0gpquDY{_oc%EiG za1?o!f$ zzrb&zGL@_or1-y}6(H9m5*LHlYuo8QLf%Pdh0kkEW~1Cnj_xL-%!4kuW9e=jCA1j8 zOJk$&#^G3tT6II1jD zA1DO*F}_o?$SUXK5_2esO

OVervpjtFWu4CeJmY4f+;s$YR@axEY ziB)5K!j;A{#XQCrf2DhD3X@0$LaP0QWUI@?}%{Sxe@ z$v5#I3<|gzcmLB+6U>kaYWtq2B?xA+`LoIVTDL#VT6p&2dGS{Gl{3QcdKFuy^{2LJOyl)6Y5M9-wvCB z3mv)~0Di|vyc6)}i7KA5``jO;D!+{$v(-yq?#lL1&d5QoeF88c4(Tql0|RYXrCu0W z*EVWlyt6lZLmgZhvEOiqF62BRNFC)UcGdCjf?@;OmcofdFaCCm4v!$_`IPr|qu@=D z@_drRHi`57kxT6rX;Rds-5!Y*ZPCxfv3oT%`BBnLUFq;1EiLO&o| z_oIQ_wR7H24Q!SKxuA-KKFU2LYWh)K56SfdS-vUneT(P1xQ_E7hs`b=2TB+pzq>4+ zF*42?a>TwteY?bsC*ijb-)RXm>T<|!m7^HVOv%4pK^6L72-RS4y^4iB*ciQKtV>l& zTCaBYkQDWifxnPFNu#aQ$a%+ybpA&2W7};eyvvq1gBHwI)BzXBZ(jSWVK;}~ZfJ~f zGAsuIHg|4{SS4;K+*DY2%Mi|1F4{_=braP55IL#qmrE|Xf1sy!kdKW7UyJz5C*N_K zda3J!gB4ARklY>$+(U**C!^B3GFQ$obUf>W{nO;NT(tX^9?u;t} z$#mbbt|11*9Emh@Cd!^kB`0=qE9WUv$;nhkC$1xMk_qJG3cY>m`RU0u7XLb=Bd0sj zks}@GNCu-LOqhQqoH3^ajHP&3`Dq8)fiWdOFmYB9+yC1m@+Fi!M1^ACNehPl6aE#x z`KQ|~zoD3sZuNJ6i)acxoRD+tFZ}C3U-E|6O8TdGs$J{CC%gg6VWGJjE&kv!{d);N&*fAEW`y6ZQ9l&y_Dbvq z94o05VMnKzLW=_(zumze8pt<`76aT=k+@g2Z3-y|i!}=NkxIcr%4J_M3f6Bm9SRcA z=Vu4TU+Ai`!(X6hez+I>Kru+KH8K;C@4rXc@ttUVlziX}b41>x^!tv)cmHG-c;6+G z+5p2pkU7>xD^-E^W6W*gLydF;Ejq)zF0S~RadWZg7#u`ldqX6-k9?(`LvHOHs#mJM zLfkXrIW*nO24I!q2f8+@gIEV$SxbZXyMYa&aaCjxmxKqATS9|a!Ul25+{#QsC$N`S z2r;o#HTNHmL}zv=q(+Hp=xi4&XZ;gB=sjB;(Jx>aH!|{K@06dw{Cjoz%}9Z#g`(x( zcJZuT+}8P3_y`Q-0!(;uj%cfzn^<`B5YDsBh$nd?bJda~O{;Q zLe|wr12dcaH69Myt`U{<6(m4~#AhUzfZ>gek^?;JEomu$H{Zl+skF7Tf zBR4v&uruQaj=qM`SFzlE=z#l}weZ}<^S;JH&J;nc;s@8l1uu?!{wD)tDXi|p=hO;Z zYvCioA#N7%7?2z*I4`=yhtY8_8jY&XsTmi}xKR+>g~2Vt;LUORNyLx*Lt| zlg7l&uBQ@foR32%+Z`xiB34-MUuD?qNrqL-MDeHqha}9l9}TbJM!AyS zXi&WsewQin?mA=PxgnhQ75MbPW?Mx&q&a}W03-cH^!Ua^g^ln(@Tr`~Pv+3~g)G~G zx2XG0ej3@_?84B+z@e9=ji}jk zI<(kj(5ek-`)x*S`c`_!5H#h8Wzg~AKp$evKMKuwIA}7l@BOsC3l~uhZLQS#1KQ-T z)h53@a0s6@!+my+wXk`J>tV;pj=>-1_p#we?MwGJ-fI| z^=WsS^Pruvk{%aPW`9dtVISwIGcfUdyq3A1^WYOjW$c28gW7)(qTZtUTVm0y@eLdu zKr>S8(k)8nS47BN6X8f1^sgj?`UY!`A*}XPibJi$c=0Bkl6Vw+kU|YVk1E%}r!GWZ z5xJ++C7>5RIk|FaQgq!+r8)^32!5c(<@=oiFr=}!?}6)7{z8%dqaHZjIPVQOu|g;m zD;{Cz`8#O*llfb3>@a_E&-~Bk?{slw{*)y#=Wk9sn?JKrH-9aLsQEi?`0vc$^GjmR zAJ^XX{7v}yPv;NslxfC}aUFSayR$%9ZfFgk^*xET&AJRaqWHA*ADzj{#`&7#71ulr z*6>Qqp3a2*uJYMUN^_`=PGV1oRkkPvkN!1r8PUqV$%e?w#uQFEe9h4wJsgt~U* z4Z=MNmJ0pPYEc|f>jTq!#34R{vA;(twlCOf%}0N@uQ2n3a3@(8%UfH?tL{D2JD8T+ zLn?ct`akRs)1*>Kvpi)uK2ZdT*tV1;dPt>&G(Dl{97&1 zJn8Bp{>0+3E@!wd7XMTEdT!J@Z3wT^fBtNp-ujHKQ!aGj<$uOpr%PJcI(^kf>*P#E zl=^@^es|u2=(Ma2Ej6Y_-6xsdUJ(6x-2Bg&Ln}WQYu%I{A4b*(Utf?Q3~KqKC}u6s zLM?%Lp@*|5)nG=WOb+xvILv*1lOmvZQy&#@)K8~LEzGGnwYTEI!=qY*waX`5t)IwT=XIh!} z=tCuELi*^Ys(-OyutlxewFjNm(J(i67^Y>P^2%ADJ>Q|fMy@D{ntK46WA05(VSDB9 z8GO;=#|N{OFJUd;cB21h$#P3M+R_#Zz8twn$DFSV{@!`MhOE-f*X>86=WAQLi)_9uZBg^} za$97+UVSmy=k~laPfCsv7 zbcXZ4c)5UL+)X^*Laa>*&gZhi(QyXOi~G~&{mnHKcy|R(v76oaqypjBDW79)c#IyR z1*i?H7{dHIJ^#fE(>KP^8Ln6H(_MF9biV5adYvf}-4Z>E6Mm$k6>kKi3Ox$-Tl~Qi{e5(c#P&nF+BJM@ihWxI-+}wt zGW=bNxgmO;VP0fEY=RB3i1dMW(*I3ns=3=zNcypb9aK{#3>S2AWdcni27MU}sdqxE zF6lz9B*lI1Zr4?X^Np|@{xC2-qqWi)vHp~%HCL;6iCB+gT1h{U&) zlb`$tzLQ_KYCf)a=}=>2TAEI`YB3HlL<5G(l9MMbX8!*cGyi{!b^ibUXbt6C{PzT| zjGS9AE>|YBYGbY%!s`Wnn#_57KvE$vVsJOqo&;)oD=LD~)Pp`YxgB zr=cf;I=v?XMvubuuLSAe%Mtqbyh{J5@masNsQ8N2E4Ljc`X{Q*Rux2=9j!I{9<-#= zz+hoiJiuDzqAwWz^TnorC(+Mu-+>N_qt0FujwuAtkQ10LIr)}C!U+7MVZ@PBt?W@( zT}M6&<3T(hlWs1CW`eI`JdEdKlFSDoKj2hf9G=(T!Ju%Y=_~$d36r!j@p!VB;o9zb z11@8QXOlV4zhby@@N3@mNJaBVKz@VIU{`wHp z38h5rcwy}klAsPii(?vd9yX@Aj*{JI9!ft;mEkLaddoR+A-ni9t01qKA6e__5$un8 z0r=_c1H*0184o3Y$ItkRY@?xYlp%*=D?M*walZ~=OwCpGp%4yB5L>V=6m47)*~d73 zyZ8&|c?x4hmR#xqRLV!|(JP_zf;1?2p=xL5u0ZL`<(NN$KsQ<&a~#nAVdS5@rS&NL z=oS)2Hn{o*_4;fDB}z^X$R-rXg968~uhv4mKA)glV_p@LSEtmo0H4h#+t_o(=r`C* zzNk$#P2gN)VV+)FkMBN%{WQ6HI2-kp(eqmhGpf{nr%o@$;DLF~jNn@#UO@(sZlJ|l zN^u>PHobgQo&VjD`N!b|E4BH={&6LJJllMEuo5z2YAqbbTnW1AF{9B#7!L@{`DkO# z?NH4L>Y$@kxjE812lZ|k`FR+H@90QBfQx$gcbux0Fg^#yYVnWM{%wKzW)%Z|taJLQ zldIr7DSejaJo!#fQuO{j^gFvZJ=4$@QeRsR{h9O{&HF=g_XCGO?HUuJ>vo9hTxxlGcK0gk}$!n-^EtR1gfj?k0 z;zoyfB+!#NYP`iA)0bn#aiC2Hn12^G(gF2{vh^E4 zw5|i`9mw^>Y-0GVYq8eSFqlBwFKNwiu(E}sb zgd$$t7?e<6Jbo=!dAmHmI`|2G-$gb+R{5^#;OESL`^$@P0s=SzFGnT-FEh%8K?(;! zWBbqVZF=?|lU~InIr(;~AO#(?&ehG}o?}t>0hE6d;4j$HvpT@yW5nd&!dTEa0_tJv zupSZvx>%JHML%{=_PG zgHbap40{L~qSLplCK2IRbc!;UY?hH-aI6R$6TWdd($ERlVjXhOl0<0n?ZhGS_!;hAfoGxi(V>2KR_= zuJs+%o!_YH&QD=Stw2f60we%E-j1j`ht_JU&Y7=~s&f%jb#_1A5~Z(w=nYk0dmq!+ zepgFluG`J@waw5k7=kK)tXHZI<9s!Q85`J6r&0e|i~7%3H2tP?Q{a(U{t|$G zhN~SyT&sR<9OB6w@vIj8y*>h<9-!vEp%4|t{SEC;%j4vS%CI1^yJ*A@U9kOceG8Bc@2k#O=b`(SIn}Cn@ zr;{km)dA)CugHAV^>4|xgsd;;EtB$=G0V$&vgCM^SzXRsEV+s`i%Ycs_vO9ovn?mZ zo6qs@Y@h2rY&dYO?tX}oU0%8k<*-}Gp2{wdHX}+o3I);TQy%>Sc}4h;Y`#D(>h#17 zf>_DCN&A%hiYeP?7yW_v)p#>cr`Y8idtqr|a7Zv#DkF=LRe|#8gGLY z5{AutQ(44m8ZjyFP3KKKZU|W1*Pq6hcJRyf7v(Vkv4n#Cb^4@z&PCLZ`KrArB492b z_MoWA-`oW?48Qy+@=-&Tlm-_APQ4GfwF2YG5qU6-5+C_ z_rvesJD9m>@}(&HVi#?2VCI>FAL8004ybg6*54Uk|IZm+`Hz(MzDiBri7fw;$nuL) zL9w>{sGYa^@(fm=!)W!nOjch(T;L(?T{haQvh}AhxOcQET>sI+;3o7{*M`M@X^+sWJo19q@$7LQpRJUCTpw0R++<`gJBIuk z(W0S`D`D`jDjHPs#&s2O2IYm1ReYJu&L~u9DO5 z60w)_?u($(;3^H1{$->x&UA4@&fs$llbn6+@_d}TZv5aag7K(Q_MiGt(ON8OT|=>g zWVnhz`pQ;%?qUU=cFn=IkZoj>E0MykDH?vw?u=j5Rb&ULt9jx5jw?9-AnFHWmjnFs z0seUa|B3+rUI6@i;z`6mEcnGIgnw4C!u6eG^DqMRnUv>Kk@4S>;{_ENUzJ=%Dl)!+ z#3%2|wd_~}0_6G{rz(I?4vO3b%TqY-Eat$e2=>AO5_^-YjR$6Fdj83l8K8vSq3 z3cReaB#Dv&w`%nX6srZ50#QX0KJHYcsV)4Tj2)Ghee|NZ4emsCaTn*=XJANg7uBl- z)9R6_{s)*Q&B2G9-E3$tw3-;3a`&OO!gJU0;wk4sYhm-vuGHXMwyxq9_tCcO!q9cj znT2O>c1}lYBQ&w9b{)2C1JG-oe`P8GKE=8ccf}nxu_^}%dc9(P2@Or}x!WmduLnwe zs%Nc7=|a(2&;|r+?}gjXxepnAeY^8+zmd;EF|TVMF#Ge2HjCf7D#VD$6+-pFuqD7OaRfb`Gd*4!7`UT2m2$K{9z+%4R5LvxI^zHy-nlfhj^rdaN? z-CY${Yz!qai<+s$cc+HqV-8+tb+?>+_UFNsIbzk|s%+mvn22XhMqaE5UKly=EATLY zN=7z2KP((!aMf2BjKMe0FW=<_=_a(U@J)zAE5G{cQ+Do}&8N7OgiB5)LBfzS?(cV@ zKvx=PMvE!mIvrdU%fBfe<=zuhFk*;rftJs6*Hp9o3n_nGsum~BH&h3s)^Um=kj+iQ z`+@JMDU!dL#Q(zInzJN0Xwz4s0n4y*4> zCfvpZqayU)hEs_uUlG`~rh_!XW*Q;%u?&e9R1L#}?)hPe&^G`(0>h;4ZG-za>z??t z!}t^Q5HrPfP$8>BFxCpa4`IxD*ihmwYN3pBw+Ti_IYkX(KLqa6hD<1A*++fdFNk{t zaWAAE%;e?xYF@t4N^Aw~@=&O|fluGSi&N70^eWznp;wpN#glycW?bwFds!fp7l%V0 zv|<;_;bMS1sl80ew5P+(VTzfuVBoitDgxeMn;*?Ic+mq`D3YSfQFbpTY3K;;0$yep&p@O=5Jy2yWqIj4!AL)xxqAH_de zZ^55cxR6TRyC{}VKKwXkPh;61Vs2=-#hoCm|*gj({jr3+u;8(7&{sEIFK|S5MnD(P`^XIE*sMJ33A`D zKy!i871^7P3gXf^#Eaw=6XB@RXtBc&R0F?}$V~&#LxMPS8B!w2OC{nUQQRT)nYj#F z0b&k3Kf3juSw%3m6#(F809E#;!_k?T!Hg^5@~RRv?{s35YIt!ey}wXNC+pBrMml|r zbOPDbNTvcL(@wwDe!({!EI`D61l|dX3qThond+9PVj|L~7@6Ig3DgX}p!HmO46-zz zF_^`I^Z}-Xi8h`GI7S5DMq%NgW4o|p#viF;NT{J%{8Ia|%OM_vv?2$J6Mw`L3pxai zOP}S_NcTd+(1+td>JQp1zY+DOqSQW4$x@2--1Mt@DxwSk+6ec7PARPqxKhw!06OV_ zL%_(UizcIpV5`*TaXvny73_UVVBw@;qTv*yHL)$H!^~j}!Z8PB!yLBSdJ&rrSY1ak z9yxs=2Uf~RM-ZOT)r3^r1!*-61=q;W_udZO;-zn=Q)6W`dF;5_i>DHvdfWK#wR4Ca zQ8Y?a9Hsxl<4W1n57^W9@#(-n^sNRYzTIv-1y$zKTyC|e@1^5~hjU60FTz5`%{-i6 zWhXNE^gV*HofjvWd2t?`PWPuGQ?l}ZHgLXPy6g&; zJL zDhyW#aQ+BfA01q|)~`d*ubK!<`?;uiap!OJO8|ua1DkpBd>{{ZXfM{e_?PJT8pIsL zg8RAc&(GI3reR$+n-0+av*-Yo;m<0J*Py)j8lA6Cv+TB+lzlm7KZn^KmVFP){)J^< zjoDve_RQDnLt%Ludrn4&F6Y~*Lt#H|fW5!bfeWZOk$C)Z4 zQ+}_w@I8X_z1KPageiRBa=saz^EYGu%04u&%^^On7xP4dAIOZy0V@EYx0i+~i&!J1mpRhQc!Wt`=SVN- z2Upn9;c0`ttOb(xI?~ti#tq1y8~5>Wdn`K!#Bi@*+yDjb?Ayoj#=TgHxE((Q<2I-Y zPp0&8A>GfCYI$QhI`|Yfz_%HaeN&9=BgHqx#D1mvro^#dX}&4(^ed17UxR#862{1V z(LB+RxK$vC1mC1?zA4?=N3w5bLo077(G zzEoVeI@y(^kTm*@J->^uSvb&YSz1!LvD7bI?n zvje`n`|j`Ko6?tkaK0)1*sqIxQ!Zq`2Kc62#C~0EFFQ8WHzidpFFO=(O$2fd1o~Kf zj&I7vzA2ZmFX+^z>{nmkl>Y2jf8UhL=$CRGd=2#7231~8^3jO=>Ad?Zt&9EV>mr}; zBH!FaeicQVO#bYzv_nTfNIP^V{8{xYN*C8Htq*Db-qpLlS=*z-O&yrxn+|GB@x}4F zJ-Wrzi3>cd-=i;e&flZoqmw%4uZ+{}QLc0TYjGMw%=v1JowBD=_88W^qR#oFSU&4s zap&qUjSDlwoNrF&{1#J~5$1duo%8phn=t9VGy&^?ZX)|1*NvCPL&~faL3+$ANaNw} z9p>@!jd6ClZx3GV4(vdx!+6YLJP6g|9aa!0rQj8q^L=LQKrP=tpi#@u*}=|+Iy6E4yF8bH4i~I`4FN~T_ z^yK(7q9>o-2lNDkPbp9RlNf^jEc+*xJqxqZhDo`TWw+f=*`KrQS224yW)J&km|u{S z4=^B+0bPMWxnu`*SV;!B*!CTI`Zc8^{hGU9qhBNS^sBz}23F}C+cjUDm%g!G^Z0q` z8{0KQ&qv=F{c7k;pBCupSMho28|z+8=juo4=~vNt=^N`_?)m5&#=v}-X7gGTN;rUC z0sXosRgfM`5e5hOReOn%;egalOyi}Ask{+bj9u>`)^5C`?HJv-_k|-AG*kYwu}7HUF5g6{`vm*c0nJqx)|RZUD5w8@-OXz zzS+9yUqTo8KQj9E;8Y?$J?|#s^Ar48bvsh1S6-+ODfhK7t5gP~I}FYETR>#Emn<%^L;vaAQhw;9UT2B5+-D zh}-RDLHK!J8owqY>_25sZ?LDIw5Oi})+ZGeGf)dJ_e3Eg4}?X48vu%58UU1fK(ZB- z^mIoe0@p4~fJy{owIBk)0OpBx@-3afU+wol@Eq}0!>yn5S#`rwWi5Zx%lnO;hdn_r z?=R~t??+Q!^_|;!@hr2${r!A!uXKU0le@_8(?$OFF80$kp+C<*rq;jVJo*1&^81Zb z2>E}=WBchYK>nw3uZ$ImA^4PKzs0h5VKypNl!2luzvoKE8M%^)Ca&bd9Qd6C|4h69 zb_o2vE0vlqt26Uu2UGaOD#(hrm({_~8pw^aV`|yKG`{RmDkL$01V|ages+VO8SG~w zj1y|_&Z7hBvbqsMVi~s9lhtH~O4Innt?;u4t8A+ZkPJD4)ZcKE-2hc_kdz@L!maV( z2ux041$#q5vyiv}``;I`F{=-NrowONXquYU4}K3)e_sf{Gt}Rg!1jV#E@ERkG(xT6 zVyMCFNUTDTOC5D%?qd(TK{=N+BSASp5mCPN0ZSZABMnD>hL}Bka(qVKJ zOj>a(peSyiGJARzG?ffn?J7a!ghW5|$GWHvK#WKL`fUdSP*w+b>DyqCWjtMAPA)nK zJc0ox;%c})U(A=Cfce}28=a?(K1G#zR(9S;-x$J&`+P@-yk&;)9-aS0hrDME;r%&( zSXAEJzjm=NgYW*i+%34W3%T4g%w1rS6|DD~p#TtG_W0^8Q zko#rajTY+0j`l{4lkH_kx}l+B*^wl1w?kaVFFHO5z(ns4u&0`8(`$smjeO-;4CZVw z`cVK~ITkH&41+8A$~da9a%?Ili09x}nt`QcsDDSOf3xwg`;-xW@@mo`dJ9{!4SwdT zN%_poWL7=B zP{`yn?9i-*v>oDv4AjhL&KV(yvVtCB#FLmSZgyCX5WrI-9O6+SbAGlUep^ETX9-}7 z!*Y}W7Gw*Vvw4RYpyZz%mcx|%H=b3TD~O+&L$NB(WxdIViVZkHx=!Ug#KTbQbm&ww zwb`-q>8$eEPsv+nJsgMQ7cAFp`u&4j%u<6Af5N8Y$GVg&kTj#KBcc6Xn)C?i>juH0sa)$-JCXawp+33d%yLwiTR+~u~GXd7+ z8Qz{=iNPE)S)#PsKnk`6JmlrV2+TC`Du{uBG*dqp28Q~h!wbSnK!U?gY~kfS2!#ccGSsa3YlYIvHW%me2j(rm{*}* zG(bA+q)G=`M^y5bCUhk!E`arVGL=uSU|S2e(U>$rEOUr;e0rJP@)NENqz$69T1wl* z7G_KarPWZ{Mz%I%Mo?N6rO^sO8tk7kN~q=~Sz94Kb3BiUVwI5THIvd&rd1Hf@{k96 zhox#D1_nZ_Adbxi5ZFB|RRf{bljH^lk`ExTi&&}#!tww&$4-ZA*h4Hu^46$OmMsEW z-{1<3p92&B_sKxb)2n#4TLg>Wfvz6M!qQL?b=i1%Nz!aI^UO548{%PU@GTtFBhl2e z6{6ZVtIHF~q~t1|`ecSiQy_s+J~&5LE8%UHN+9)=OACd2iPDZn778~Jr5%VY6r^QP z+V03gLD~pP+Y(tQ+)9vkzJ-DQkVLaAFR45C{o+)B7m>Q)Mausm(I<*RFk z>lFr37m9WhEL3DiMHY$%5MHRCZ`UsrZ^6CAW-r!a)sY$RNj@{l-H^c6D|(dx2Qr~% zGg}^YAFH|LRtvrD3b||HxJvmD^z}0c=%dg_P(O$~Q$; zs9V=7biNH#^Z(xsr0yErK)7?%{nTj#MNTE$N9v}Cv4P&4*m(ohoOgL@bj#EI|H%#X zXJ?VRV`u}Zi-imIzp;VFPFHu1gXU$n+CLE&g1AyT#c_Mp(By3qkadvz{VpUK)D0x0 zH^Zfl_-@`3UO-$nSSUUYT?;h9?4Pu8B`}U8tGnK4Evp}3Z#p2PpN(29`qu9yNTBsO z$ry$4sgwXE0&EA3PopBRG6&iA89#^;Hj%9-?w#?lGGxgF>4F6=023f#E2NL01mIjZ zKpHH~9k4W8fgv<&u*lVCBO91`50BLmH~_G{Gl6GWx%w=&XC~ybtbELp-lBOX2%gyM zz0lTl)>cisu3ytxDVXX#X!xa-0CxJGIlQzK?=6n>0|IOeoLUssi(Y1}H6}&19wPy~ z=$*}e0(Ef}@zS^Kbz6atmshiJ*JSgg+FA*stxymjU8b}Ku@JchBq394M{|i!)@d@p z%Slzif@34-mz~9cEhO&uW@sQ2I~!HhGLR7(NRQ4S-!hPF4Wz1bLoXu;T6B_HfG^b) z1pKvZD6rPo$}CzqzDjhgU@6ow(NRoaOR0ftWwk53nCC|a44iL7$2oYXT$=-?uduDq z>r?do+Q;&X_4;0Y{`V}uRIl%iG5}`ze!ado$^e-1C#OktH9yTg-UEk{DU;DzQ|&D7 z@uke7k4WZeZXx}JO18rGp2a=l=PiKvlhdG%L9@7hHLUb}2D1T+Pgf~E!3_eq46Zje z0&Puzz33||Br`+6y ze6qd^eML(b<-hJC|H>}%>$+(FtuFGv@1p(%UF45q`U>Y5k~iI(P4Xt2HOa6FW4$TA zK2;x579Q>B?@Y6Y7Ub1H%3sea5T+2->#YmYY)4qjzT zqr^&iVBU(?vhF4zS!W!{KvK7;kK>bhTeZ|i)Belv z7(9%@ea7ksGG5R2zdr#z-YYN2FvTj65;=4#qkIOzQk>k|;zmvE#!*#4OcRPNEQ|9_OIY5%R1KII}_uTf7Jtl9-Z{ zu0C#{>o&X*;mhbiKt{5%8uF4cFRQ`|{YX~z3@U2y=EVyz%!_kfOk&?JuLG1xi==!? z`80;C(2lEpw=pE8)WO*~I6GDvt&%RmtDBOt858lIu$H~$@aCpp(%Hv%av*bYvC;Cj z3kYW~LBdBaqtBqyGHM3Rc8SLK7eDfrLmiAK6EY<4POEfj5w;(mwD#sNT~dn&u#p_T zuuJVodX&(0cwowy0PiuZ!tGZqo*CJGjb;AO2>)IJL46|pI|l@{nne(#T+cy`X8*1L zLF>>mz%uN?W?A_}R(7)eM4NrDMY)}>>^H*totWBq*`pQ)lMPKj>+s4&2j2?}g_Yl7 zkXtURa;giw@kTt&sx)nR;Gx3+NR_t~*kk3;cBo^szt@1z#i(IQR zyG+|w?G?MGm0st`3ZJ~r<@7Mw52JkY`ft97)fHane=59!wrDy>YZPxab-hhgV#~reZ%O1@(Q!zH-kl!^&vU)0;h1Gj;Sm6s7;?Kc<+kgU_Zo z=@O(K3u4ftG1~h3L!3`CJpPx`c58adk@S?k^r`_Y5geVg8jvFS(xg&e*g=cgRC=#u zLTv65bb*YsOB{kKkPLPv&|6scDd=-*WALV~YR^Ple|e2_((-Cjc|b3L6vyY=@9v}z z9$MF8ci^z9CKBB!3P|1B4xiZeIU9vD1xlsH(~L)L;RsrXSGZG1fqi->zO}FoFq0UQ z7CY7U7Cn)+^pp)B@P53S$S8%KRACpVFoRN9U2Pu&4mTr?Vnm#)jJK?y8o-!1m{oxG3zCtTA|m_E-nrqgEM@?g7p&3e>AuRFPvOW6M4F~Sc94ry7yj0T>>u@+LMR{ zN=M-$fmV7S^$A-1y=4dzw*?m&f~q;l-p9X*Am|~wGF)mRrY}z-ZXKvPh_eS4Q6K%R z9OsCo)a42mQ)ekCMfMcd@n0uhKxs*bnAW#z4^{9Auv07#Jz6Hp%4OJ<%mvh=@?0Lv zydQv!b-E(01u99O0Lp4sRgCY?r zN$-cdC(M#KVNG;5QceSq2(Bz{U~*7nC`Du3KGa$XN{Y((#f^*;FPMb6A(2~(n{TW3>3VAUQOq2% zoxU~a0qtFaEn7;(X`^sPRGPI-YbYLxW|20saJyE%hC&hv?;tXM!N=&_O-XpP*&kCe zxMS?sUip9(Ed|RKv5_Qg!3V5r>C_j)+HcnQ*2V4G#p|h734*yhg+gVkcIn5W5Jzdg zLd$2;)?CXe;&`NUx0jz`1*IH3#nhwOnZwVz11kid1DG^VSFq^v)2p+PGkxd;5zIE8u z&#@p0^uQ)a1c* zGUu-I8zp2(*2_SB63bL9%7n$xweFZ0#77i}AuwnPgfSdl#J~-LTFQh_A*R4oNAWH^ zi%@$FhvL0y;22a`q%VbZR)W8lRQJh6LVR*+_EEf9RicYm1!tS65-rmX9wCloRy-{< z2u1&)*gQSlq4lYxX)ewi{inIJFX#I|$uwF0_y>JWA527T_2{Xjjk=4$3mII4U>U&| z0N7xkPk&F~gjHQe+e>8X@C?jU9k^C6xNT$xTCtsh#<|s5xv06tz$sAvS~5Bgo=nu; zqGLK*$k7I4t<-Y9#(xm|YQ=}Bs+>78iTv%_yB~Tro!?(cRL#(}*H;B9fcq@f6~G_R ziE-W`Bs(F&I@%W)B~wR{gHdKO%5)v2PcX_YjN(3S)74X$A$$#Mlui6Eekk*F+@oS#4>FsyrB*PB~lsB<>Pj2brr*uY`( zv&EI$y2}=uiI^27Wiq*7`p{yg0h!d&H7O-`DTY=KWH?KkJh6 z97#dXw`{X)hxbUW9Db!KkP#LvUFQfl04O z7V2=OgMjRHA_6Rxk7{-L^0?iqoYCsv^Nn-uT2YyI{P{o!ke2uHGLp_LL@4ApTo_t= zk>zXB5ZlqJzxKp?9-rRV?N$2Xgh#vQJ?|oW@_S;4q=op$uJpzC`s<4i{EhX+_yq#klpQb;?N(oGitAcmyf?OMeQ`}} z*ZShOW4qQD7sdwZi>LVX#gb27jM^NdzIZz8i^qh}7bkuB`r>$>zBnO3Up&LzJAH8m zD^)XCsXE20!=2&N;j+FsHl)6ovg@h7IF<8C<-821D^;9Wi97#H^r$?Fo|klS8DBYuM()%+ zf9mwu9f6g!%(ElhJFxKLzLXUJJGI|@$ki+06+MJ#m3oLKLf)VOb@W1GW`BmvQa0eL z5_9QL1XD#F()VRlvbwe7%Y$17^%jg!OE92$ulM6i7}DkO`WAmdP6DiziuKX1AO|5C?2MJj@;=1^A(o6sj(HV~ zXZ@Vg}FJ=7_ag}yE=`saTS-+*`9d~S{HjRrbWo36mjgLi^A*<9kAi<5U%V)87%uA&AhvU z@{Vy=(0_Cb{av@vA9|r1{Gcq8XLWW59n~%LP)B#rH@b!Xip?Zl`5mJ0kfB6j_fY*9 z-OJWykM=(f?|9*}H=?vk>Lal3+sf}6#%k}-taw}!Ex+dlnjmR?K{r7%jn3_k8%1~* z2b(&c!0<5y|2)`6$M+bXO7NG1;e8k`5&X_z_m$aco*(d$0+Te1@}W zfzIIg(;3cp=*xrQyqP$T@6gWuK1jmJ#(UVu^D1K@IEPNuv7r(e@U}eH*VGjCck%w5Znw~&-(VxY z_jmgAAEDp_+}*=J4HJK182HU$;Fd7(^&UMS0LaHVoH98TYvXy{u}=9saVw~a(r4_NjZZDQ8Vzc zX%anfs+`ttrL_N;#0hER*_*-pGumkM61V>`2Qt0`rq$kIwJG5Z!}}cuMVwWgmV^)f z$heJrQo>3+oFvUX+`xJg%_QdU>XI3{+P7VMtJHVA^?+8s#G@Y+kk!daIPFzu731A1 zh3*Y$FE2sJB~BA~aEM-1y1Eci!~NNBilB^&0Jm zr5?W%)K3JCBFmo9N0H1Z{}a@Mf(MeZQHB%L^}&*MSd`%e^|Qh7PU8vcTZ7@-1MY>y z2cx{iD3Xq{E_gi2VHB@2{a|n^lOp|<=x+w2{3z02iGFV|iiJ?vm~mtg6-kgfQfMRb z>cpg60J35nO&8!biAnihr8wS)m{bT}krCc~u;gzJ96j6%_0gjt!Z6q`43_L=5r)D3 z=HQ`WQG{Wzw*JUC4Ko(hxy_%LvD82Nv3nDTudCjQGj ze*89`?f(v-v0?iF-JaiTX|(dNGPtupfuA3sA0MB`;42xtgTd7U^yB0Ar5Wu1vOKn` zzfU6h@8-7+B@*`Kf==nSK)*Rfv%c1Aw__{;$s~j6d@)F)J!70^UIoLbuUlmE9EY|k@L&$2p6rsJa zoR)r^3kGUc{}39NX4-$TigUW^6B!hxLc5ou6tN_Sb>(dad9(&4`JMt@lJoN@ESC_q zDI#bII*3p}G$H)&!SGfRZ6K!*o)rwQXLupOlY`;!F}#@b4_+eAj5I94{~ip#Ez+9BEjB%favyhPyhEwxv2@BPSKh%G0@|O|pU9MqBW==Rgp@@?ZnG zZ7TzBA$vh?+b|0VK_fzz{ErRtvI6Dygg@zW+d)(E=P7M0)J!9)FfYa^`8rBUL=Xzk z{GTt>QH}rr?2@GHWU|8HUPzb6d*yJ6&OYMAnMhN<5NET7l* z<@ZlwNy3he)8%vIb!U{z4JYZK*>px}K@_Fjb&||J$x5|% ze-A}+BWj>=Ju%{4eMec&t4!76-zDv?nDXDVL|+h~SIS$NqgnnTrYtmrzT`#H?;f=G zb?OuW$`s$};yXipGuTL1Y7YBzNw>t#e4n$> z=Uk*aRlA8OUY^9HPNeeMi`h4deKXh>%f4Fn#kaB2l1m8-FGmiZLpVvHE^e_}y$X2_ zv`FEgPeu zHnby~VNi;v-d%qh&y4{|AT8*~aV^f>ksl#2t~TF3vCX$WwX7g7-VVUe)dN#n{Ox8` zNGS{|D9R*^TVgpdlLM!4U_1v-;lKW{Er*d8^D)o->CJbE4h`14?bLpZkG8YOSe(>O0%S}(aToSNYr!(?z*ePPB93Rq zFCgZ@O|O$;8>~E|U98^KDde(eGJY)MPiYfpQ;Mb&c<~JSEEz?g(o{x{XXFg-D+%nA zL>~j6wt=Ra)I*S3m@^eqdu$2q*HP~;D5Y*$P>$V7F1SEeCTGY6x5vVF3Vh??n+e|p z_)dp!B7D(|$M@#;HEFjL@$|BQJ(;LM(RZH>`JYN7g-W1Q7ZL8yqFhfqeLF7gWZ4ns=3Xx5<7L3j?BrT12P!)g^pXu88mT z6w&cDbxP0}t}Lb-N9vS}py6B}!Fj*A2i?)r$X?EFdwb&waJKgdWgyXtNIyEEE4s83AYCzkQQeJ&HN!z9d^1x~gQPs>&o+a?< zNo07sUB_3QRPL{jLAP{fh&y*^!L8H#epi7i?riw_B2~lk(O|RI_9jhX0(+ucdti~F zC%WmVEn#Vc7YFvl@2ZqWqOm8^p`19itDe}rAVg0*{epf#7K!=x%f92X?b?nPd_D1x z8QR@{_3w8ct^9?+p7`6B16$*<1%}q3rQQ7t{9D7VOCM<+&*g*~e7RxOc(9>`n{XzI1&KC!Cr&JT`ocwtrGfV3*D5#iW-ibZ%LDiTEa z6ucNK!X*fA6ycc=UMs>uzLjbsd^%o?J5I%Fp^v?UYFO^TPhNEr^ko-^#J8YYqOyP} z()1S-sb3RXsJkEv_O5^JA%=7`>jRVauELYr7O$Wv=8)JHub6q7Lrg7VQ*W#GIE9o_ zA)vFipna6>FsyKENOOs>sX$|SZWWsKiR4O6OvtC$s5NzVN-Qt~D# zvJ~4-`{4?2vSKiu+e~b&lyiNk0$O{1VA1zch+R|=FxSZ8 zolmTv_P2av%`iJ{=jTxLa;!j%`G1=f3xRyY9`}#QA}dM+Zf|)E7C)FLK|Y$S!+ei3NV8X{V~;iyV4Je0y$)r2vsX=)w<}7jjzmv#t#0~XxB72c5f!5w z{7;8TiqGPI5(ljgH2$CB(arzsw*OPw>Ro|G|MMCxfj0kjga7aqtDHRFNS5at$?|+7 zS)M+Uk^U^-zLO}Uia`x~si=Rf&l&G?CitA#TlJw%gce#>hy;shKntx;YY}3Ob!l?S zCcMr<=(-+F6V1;$G}Lqz6g4u2(4RH39Q5hW0E0vf>(Bs$rM^LrhIR!wja0^9xwhM@ zPeaQCNt_1h)aE)#coS|CrH1*H5iaHz&$tf8U?GSqwi|+)$!yj;xVRlDc7Xs7|BeGo9LIK38pAo#x|Cp1uTvW zXkrR{s!g0rDKFx*45Acg6hD}2kY+ShPMC`m3Y0ie1@swff^K55r?P?yXT5<6s*v_y zZndnK9AQ~;x7o5HX1ry^&5`gku@C%Y$HLFu=fMv?*IFkz_t1+wGCr0Qw-%qEk)8X% zxf1ZBWPr|v*%R8>n_4S58{_3TCjg@5IJ!FO+?^ofi#U^GWM@-!+T2};YE^7Clrp3m zS0l$cAVQq%)FSAO8HA@hIH@W*ZX-nQi?7-`8zHGGKYS*fcJrRhe4i&3O&f8a@>eaiCmxFm3K`st455ik2YhX(1=Q=QH_y zoXAy-*!4bxc`y(OISA^JJ&~CWGujKhq}m5!HPT6k%s|gb8DVS!n#->Hax*KSt)&fA z1sr?n^PzulC90N8k0>+QFOrX^@hWqLzp0E)ApQS?S2npA(Z`H;0nm?Ha;b3SOX59@j< zYw7gXtc}WqdO-oh5LZi-^l%7gxS!e|9Swfob^`kr^cl;-m{uv75^;&L23=`*v9?($ zmi$|#f@i|P2)ew2&W;RZWIg#R!Xt;SCMh>r$!%f=6na-g_!I8+Z~QI=RG1_^@{N~I zKg?#IOkztFOK{Dpb%S`SvG+!U8o3U(aYJU#ZU;aXuT3rJVp)}r05^0AK9X^53VnBi zh8OyDgX?}#XHAF3+;(WJ!IPzpvB9JU6jMBP>S?!dOjCl9JojPd9c|wBIC#0LD0>R@ zjr~CjYIKbK!OOwW#OD|Ng>sAy8RbY?<2q+8_nHq1FP6y6ddRqBViJt0GYy!L$=W6Y z{Y*Q&vGlRAnsX)H1=#Q`cfX<;T|;KP2+@fTijI!C=;4_6-vP5$w#i{G|6is{4kp_5 zDFYb>W&Oj6JpIF9JfQRwIyu(N!y-JLVfkAm<_AbrYgB&~jPj>97c3}S)yU|0E$(-M7oiDkrc1G2_3l-V^}itz#R<4A|Deco(nb2`ww>RkUxu+mdN zV#ib!a13j6*tf<01h4-His#!qjkz_F-g2nR?i$J9fCzcrh`q2m9N1|)VXpS%i68*K zhilLj(8aE~B+NHwwmlTs%>D#!Hlo=K_8X{Y3i0Hs9>)@XRb2ge}bKH1LNh zUzEI)=B%1%pp639oc@_)mUn=dz>tdB3P>I!)6Cd+%-VEw`4lkcgf-y$GHB@B-MqlEayF`$TTP_Y*&%^yq z+ud|1H3dFB=?lv^#n9Auq z4`jVd_YO#)Hihq@n?5a+Ltj~~O==gzX2Nbx|6fyTLbu)!AWAylLwg4gCozn)CLlAI z2fp!SZH`mVmmK~f(nvbCm5k_ zI-apBCric0%H?j8W|-Ju(xP%}~0$00htOo}IX^9;qvvDglBFfI4@I ztQRR4RTJERsruvWqjU1|F7GJv%MR)>gNFEMD2k z=bMbvHRs;`~;lO{+yY4CdAPY(VH`(_%r| z*L7jfqlr7*p=7sn;3XVM+&)Q}Xr|GX%1|Pl>YoQq@+Y=T*l0vb#Y00IgG_nfo9m8j zSf@)vA40$(DP{D2l8eMax1U(1=CJntM82i%9nN1y|IFk9e+IO!Auy%RnE|ORRkR+1 z?!b&AD$H5#4Qlf7uvZ{}gTVn6Hj@ zQwqJmkX1rvLXC;kndDE3j8!|7)*sBTk|flXCq~JhG)gvRDynju67zs1OrCkHBb;LF zIauK+y^>QcxCh#KXa6e3Fz6DW3~5h@f3(XNq9yvYPR-QX)sEIC;A)}XMB~+C!_54L z?Z7O{PxNil`gknMqg!0hMeWhnX^N*%s72{svN@oSev?vDQB@D)QtBsT-hE;c48~Qcjbfp z8E2SETWhFf_jpgI5_))wv42L_v&c^oRt_?!c39GOmEc>k%VNC=VL>mrE?-ZbA3Nf( z6y+8bw=uS3+F4%lp_n1rJ%izn(*Hq;)s0KMrHJRbW>;ZqiQ+AT=uqMe))ZO50;Ym0 zIIT#ztjeJ z6K74VI}Z-Av)#fL4J9Lk!k5q9UQnfApXv5%tQtHjpG099IZ%oXDDhtWI;mU(?qG%;2#AD}sqS{iSw`056bBe)NxjHx+y!`JMX=&( zz9E4fY}E%j%`O_+-YEiljp+^?dt&yW*yavZ8|*qb*@NdZ={a5*!vmYB%nkb1w6Gh100* zu0`F|(bW;80eVXzR)j~z@ewr$##G&=Bj#C{jt%_WgdB64MmX;lTMPsg-@FcZ=V-&D zmO1SZULx2Foe)cg;Lox*cnE)GhSlRlLT2+rCS`L*u&IxV!4*3T2FIZ30-G?al&|UD zA<8*WwURS&gf)c2bY!f#+M2|hpj_ADvKFny_ufTW!o08DL0J z7;FEH|6LI@kzM1Q<85yTw+(`Rc4E9yF%m6-psR=m(Kgi56jXLyyPVUUp`pNW*>UW4 zIOMV2b*%_cby1Itqp%H%G5oILIV!!WA z<`vDT8xuQ1xxLI>M1-15g`N|^49UBx2bM8-ALyyUijOi|>hcbaqyHo&e4*kVf#lCk zLn;Lxeug$WYL~)|+alBaZ5HgwqLw$@n2Wb-*OaM{#HML*-MszF7{?0?Uvc0BC!v@c zvs+wJEL29L4@JZ>KNI7Ybh=^lEU7Xb%Hfbdj$(lm2wu9eBx^BoCuPxGGpVaMH1nh` zBR`{vkD+NLRkk(!p18)!SKe^-VIhf7%nj`KN8WhHXaT;F)sf@ z(#Qf!ox*V!q&GI%cNYG8P`ENt^>;KpEfINWF-}@#k}~UYlpAedA?uyr`ff`kz%sO9 zyCT(r`iLIGV2CBah^3VzimhF&3)*n=LpW&~4G@x^EwVA6Zn|l;%78X!Bm*YhpZ;kB z#8NEeZY#G9%4iowCTbT7L9gi>`Fxgq#r`Bl-MuF_eEdcYIWaqe!{=uT8Vt49V6Fjka_goEe=h zS5zZ%qD4U*;e7px-Dn{s+ty2V%+b$Y8oebxBuenYfL z!SA0%KX^j@g%%H~66@`Pdr^l6L#%IN`9?OGgIEZcyXy^Nhr8obMTa}_K_Q2EGyI!H z&1t!os=+R)RSNh?h4F{F{dK-<6*M3p&-nL|SUm7ei* zl^LkkNsY_K5Jt5MQ6@%4Zn0W-C%wqmi-V@h7KTQ5V5 z1b3(#h(@;N7mq1vkz}l_PNgv0LufT3C`oIgcX@ltvb#@cR^#muo8}zn^m)Z8uqdqg zfRr*D;SxA_0_KR>6RMAY4E3r&dW7qTM7bH49!G!Y;XZdJ@$7sfay4c<2RT3Ibr~Da zJLB7?@W_vs=^7mg6lS;iiG~OiA(AaaLOZ<@Ath!mkd>wwFy`m;WU*&HyUx>i&JcYZ4l4p_7aNceh;U%b%mX5%v zT7CT0Ql)$8r16#xnxH$gK9CJDIa0m57kfz8)!Ug4DGz79tUV10-_qwLcua!QQstWL zNhbF-|BQ<9=gRw(T3~**}c`%JRgI`lWqe#_rlL$+u{wH(8Lkx(K4b>xH0WtqM4V z)sZWt^^6tjBQBvrLYaO!Y6>e_SyfrWe1(RrX?$mHaJ*KVvM6AfX2XN}qjCrCHxXD_ zJrW~&osig?&90%wg$UOljHp?rQPKAF02U?!hQE&lY&(U6h`g)Dk0j59A;`LE&#E?9 zdU;QgUuAPwn-mM`vzVNsTo|MwSrX@NL`<}HP*hV@^;}ywC3csnHInnYcl`obuY<-_ zcv5EX1P;G^z3kCy0fP04@jWxU+{sGP>$@mYpIwS5iiZHic;UgmC!{sLXyHIK^cV5y z4alz2z1u*Y|2gk%T$j{UYsgIKsG}H<1t>S4zz1`HUwlPW)||>&hw|Cmi$CE&QXgbe za$o=sJeIy_gvX~;0p{C0u>h}^&v&c< z*q`FjoItR5N%?qFpKpBixOnZnOu%TRB-?c)&@&UeXPE%*h{mI`()~B_b&AK(B()%E1LibyuSvzRzfDN8BSqJ|p%ph4c=1*-= z7f#GQP7&HUrTdSAgI`gzF_p867e_kirb#ZImI&GemX^Z`e=AiH%l7*>0U=P7h1v8B z=%xc3`(V_uq48w?=RkQS7g|lXF80)kCb6bHJh{@+QpQsLCQSLv6YSyzQ%EJAf=JTX zMMLM5>R@F`LzH*i(#j6hh{H+HO&E4gO#(gTDOZV@$_d&V-U60LwDIOyV z(nu;C8dM^rhoxZ_iBEuAB)1tHkBRyu!q}UnH2pI9l-!K49y0Q$wVNLj;j@lp6@!1Q zPlHzR@m?a{gtl0dXd9Ij=*QGzyVc6oavV9xaLq}Uh|HvY=ICQcvGr>IvgeV%!lL~u z9=Km@j9^xw=27y$pNaPK{YE>ZsH9ck3sfMVn&b+G9lpnC$ljl?0i?<3fE@=jl5AsA zaE0vL^XEJXN|FDKmXU|5M$dE;6v5YefWh2h-;69ZaBe~dxXs62f85RUs8)-?Do-G2 zcB9%y?g!@JNlHz=b72hVMc=Ic6@~qYs}U`DG0SIif65xxQjoDm%Kz~vaxO2C3K)gO zqgJWjyexozARuuk=61!YZZ>3 zov@+{asN8EkiPJ_6FLPo$@vjCElbfbFaT~s8{Kn|Io?jBpH+}*Lc-&Er`+ZgLnoB% z{6f9ZFBt+|c=4qvRv4q%>k;-0CmgNC%l^f-0sJuA>UpwVVJsVVhxPc=T-#u#U5E?7 zPkF)v>TLHo>)Ed_a^w@JKLahq^#VQd7Q)7Ovb7wL?M(9N#4lhn!_d38^1a-bji~|Z z{4Mf(W&zY$KIU=KQ_-CD$k@PWGJ&a01Rs#a-hFxoqxoA%RhNmG|=xveRBfM-CxM# z_og}TerYiSeQDUwWUYC=U+n5N3Q`x76~~*=(0leBzj@(o#x4t20(G_=X2xsx+2+Mm zsV1h3XQn1jG`S+(gCDkW&>U2fR?waK^_a6a+OKNU?I$J|eXavAn2-#TdAyu_p?zqK zgv&`&!askZ;s0vgmdD%C9T(;qU+%=p{#m*sVbPdVolBJ9kxE%$hP_tPy~LM5ZdYF( zCKw-!Wxnp4!3`5Z=tV43;Y1iox3<>(b421(&xNUuX7P%>;PcHEPw^r!=m5p6nL9&_ zJL%>5P5O~`VrK8T9Q~^x6kL`pX~Mx7V*Ab-z4gr@IV-NuF?nhTFXzr7Br7n0@Kmhi z&i?TT5XUmVw*WdJJz9X}44Tp=q*TNJ2c_%qKgYUq1}>RFFGiuawfRg zCL4mmN}FR8adF0F@a8!(q$Jx*PO;dm<&ztsVS)9(YOkFEGQ}x0VVY+*hRk2P!&l#b zJep7$ZVzn7<07{=KGIhbBQ3^Z+h^fOTq-@B-hG*CwSR!<-h;AZFDuc=F|n5e;(8hZ z6cfk8ShV0CP3jbGvtG{RVMg6mk$zP#laUr^BY98RkP_+E7dlt)KO>uEZ`dINRDQ@= zH#6D?t*@v>dn1K*`AwpuG;%*~f)FvWFcJ^DY3hW1s0l?nC5?w9&S)Bc&ekM>_xO@dP*B>(>NSiO++8w(+u1Q_Q`xSr;IEI&_ z?*`awZ^J){A8H~orE57OFKq0uYyMk;h?Bg&Ro)-l{`O}C{B&@iJchflgMHNy9_9DF z{#D){66vo11kiY>r0-F~eSKVVKQpY?I^(U4n`$Ujz(2-2>w&&2!`-~FKupFi14r^% zH4otiHF-#pAZZxUSRKc-ATxmfcTwQk&nJXnOhe2;~dVr z%lSWi65Y>%7e3T9U6O_#!@;Hyx=jhk1bP!A!KX)-uGm@)3)fEnN?>5Icrf>rn=ffcKqoxj zg#6OU;Y01>U(_Jzf$qK}t|T-4-U9X1cM^}UK3nfu1N9z5tnkrg0u-+~uz0}S0#M-Aj|~vTw^q@VhsGF7nv1 zYiDfBSGeDx`E;8+-qpD_9rdY%9`gUBvmX)^Cm;F%pweB^=^O00G72&55rc zTE9A@2Tw4e5aZfTZ(9Es_9t@ij_A*Y0^qZ}|C_&?9BSZ;#J9`uK>v#IG6px%TMuq0 zX43kTvzDC0A8*?~d))({)mx@j~{^K1XQDEHNZ1 z_QJ)$UQ=754*(AQhJrc;1@MV14rj0)lRX_aWc;0{s;7 zcTi7X!2W;a&niPpvTaLpw^MFq@35Z907VR>7OWpK9S`$qHC5@m@^rDP;65DBUK;{hR%W!06|T|5pMU!Px3$3Qn+@aG z5g+iWd#Gr<&|^W2U$v<{nt_hBvxUWyAm_K~hw9i5WD)Vhc zu<{F-5r%6Re0VI7ndu3xa<{GWW$W0VTyX+&vO_LJh0%W6r|LyJ;uRVY zC~g$rHUoLCdfa*SVj~+6a>N;i%E|{!mz>|Mjx-*VityjzRs~xGRyW~9m=HDh5G{&C zsXUz%xE_B%*x;%jA#(HkM(X&osCRbXrD+tJ;TeQGN=JVeOUMH8G1k$M2M}O+F#j+8 zC9Z%ieU7R&c13WlzmdMd_r+KhJIL;!fg6gySGR=cR;)kPb9PX^vffE@z9u=&P)6K? z&_V*jyY{@~ytFdPvXIUZla0a_>p3_^h2~bAM_48r`sXhrjP{~H&?{?1n7t34_!{X^ z4!5mlTXe~1-7v)|4=!1&H#LH<9ZDe*y9xcE8L|yZzs5hU&958dRZy@-WH=KhIKpMr z_gi2g*3=G7m7TPCn2>LRh9Lrn={J6_vf&~C(ECk%S%7FhA<1T*BybU90F!vLR>O^S zXdVS+btm)P3-nk^JHhulay^0nW_2o860rSDQ3S^i4u3j>1@u=>&{6n}goZ@Smvp^@ zkBTsxwH~bhbo!Xq=UIp-hEq1zq*yi=?9V}%=3rIJDO%C&RT1k|L6?0l{mK&nT{HAP zGZS!_yD#ayjh1-KyHrjQok;2$8hOl3xL|%w{K?7}Ng;c=LHm%46++$2@;9{%A>9+M zM&MciVvykn|Dowr$#1fLD;7Z42h6zi#bV;5e7MWc0pHO$+uom}q=zc&hJOdll4%|P zVg=%Ix{8_Wv=ky+IEhDiAr%+654=HiP&SoKV`eoAk_7Jm2Jeu zv|Z8&KVCHT^2#d?muXZ4__P->GLFj*Z{oDz7~3zjhsmU%v&wjMew#&Ku%H$1N);b~ zhQgO02TW4@nS?W$WMMZvdG)x?3`dGdcx6Ns$%|8n8<2n>cb_*B)oBsnVWcAY*EX=% zMl{RW`Bx-BQE9tCMamcK`?#3>t6ad90@9U2fz`cifP_zX1jq;Dsd_#SkEuqLpWEot zl1!pQqf#&}sS;lj%JQ1(b1XH>x`!_3mN49VINv=)^hT8e&$&$b`QotD(bf&*M7T9!iPH|n|-^`=&%IWeQZvx_Fp<3GETzC3M zW?dA%H`+78JchmdlpLC)!KJ=It(D5aaV<>YPjfAX5RY5ZZ{h60B&0mr5RWvPfL-KW zGaX=)1(;p4dXNB17y8-iw+AM_%pQs+jaj_cAT8Qa;3ABREzps~y{G?@H_*K%jX)A$ zkWw=+$v|Yt&*DB0G*C`V`+>{i-^S_MbQd2nLY~MVMMt;UtiH(e6~kp1y(J5 zl|OWLcbn!!cSInMaRVF2D?*=+qCmbprbc48B9kOUf~blOiJJd~hC8k#ygH-PXS=?} z)(5ZXj;M7wE<-F-;E!mv^T_eIo>s7&>Wx>Q9pqnIY8I|EQC|^Rb?)rI8u)Iz@Xrv2 z%>wkmVNv7rs@$I#zaQpA`cpz}73snLi3}7|<)y8-mD5XxUn{?xuHkJ0hW!1r z9o$i>Me2NNk3ci9{$Y>BY`1G@FJWJ?F}p0o96ys>J9ia9U`4ME)L>d0h~EQD)Y~IV zbD}b=S0um%;mT5rgn~>9QcjMW0>)UGeLWff^-^lPpT%jLJwE%zvv}HRQ2ccR*dH># z>F)PBn57+=T|w#BKq*`kRd0k`C(l()(_c&XZKh`Qb!Hx|Eg@&3%$qrJs_6iO{XyJP zUd#cmgjH@i3C0!05_4!KGcrsG2+T}mttuIzfjhz3Z z{a3duRcDWg=e?U#W45+wD!xA+7`jq@z8~}yU5z9k^{UAv{C?PWn{F7^QkxjP^-0YQ zEUooD=kA~QP7MBteflh)1yycEA+AMzVMIEJ$q}*}jx^nV+PiA+HM4iO7a1OtNm;jh zGzk^UW3W}h*M*6yP^-=56aT<3Iq1y->{dD)#$Sp-C}@N##R2L(kJ5sx=9alvKB7(`|WU4?nSgBYzR^J&fJ%W&K>See`3K z0D<8EU^c&7XMJC2eaUQYm(}leA?P{FuB|?09(2U&)^ee6xYZv=tYKMA8?LcmT7pbv zLF|3cs$7w#Wab4`pLGj$2@tp?8m$tB=OJtoU`8i%Jc&&x z`$eOT&Gx@xt5Yq_K%tDrX(X)@K-rKwC=-d_3l4wd1~AWTX+h;88~ay zdcOf9jdpRB{rAGpi?@kv!4|J0X?RW7cU3L4n{@J3_*yGZ2AzJoE5TRZ7adqO`Z_D2 zLH248fPKLy92D}G^*WQbh}$>~un)&@cDll%q+yrWWJ zUJpTAZa=*n#I`fgw!y`u2V0*yc&D2|68GeB4pD~ebE9S!YUCS%*=TW+j{zPMkI(In zW&h7-d>`DWNdC=SYdrk?z>KaIb(pNxgwt;B{T6$DVe49a);4d1$T&LeN{Wwe^q zI;F8~#RZxr!`_6`EgF+qvP?)M2P@awQ$C`fcVDGBThZKmznLW^42J)4C7E>QwOe;-VJY_}CQr>&=bvpg^Ye{DMTf?} z92r^6!lIk%OY{Tu(4P)O!oR#hJ;vSgr}bQQrBi!1OWe*(Y~OTc%T(jdnIO}6FAdzN z`WVhDY0Ot*$ix108@l;>=-pO(Pcz#$AfBOXNBF|&-IG?m#Jx^#-RqL>Zp5cG4$s1K zzVI+zzrA>Ri0;#zDbAh_p@66}K<2?ch&GAKh+<*z}LR1edBe8ht=85EGSbx4` z4H&dok+^m(L6D=XFb(2sm$s>%7PZPXHF2XyYWMOR3Y(1u7*`w>ETjKWPsi=DK+<7` z7>h89J#mly!kh`^+b2Dzk2U2Q8j^?)`3zM@5k9ftk1mCv+pq4B%Xzt6%rd9%MJYaY zyaP9U=<&Cak`rMG@JwkWagD{4pR|%q30S(_F`sc6C>t1a<1OFOnZD?wi>sQaDhqqK zoduz0`oipkM$*-yBde>9-C_m(&_1#pe7Y91zfv-EhD>i-f|f&BAD zWb<9uT&AhO_+kHOubl9}e)auEz*fZYiGVlEpoLgLIfN_80pN^r z;}3$K8Q1cGkPWt4RO!U0wUmLc``$CEtjY9|mZq}J_>pFL_xe#eRAG2M$-P~6LEJ^A z!JoazzuHdsz|#v`&vUYwrp4wh??wJeOZA-dRDSNFt_E*ySJh>ne%2tC?y;`nZd&4O z;b&_zBEWHyP2c`hnLXPMUJi_%x-2M}~AnOghFP~Xm1%0zXQ$Z5F;Q!!YIA>g$?d)XXK$E00bFdlp{vtvd+csYO69!XqotwP~Kg zkm0pcq0Ty^JI^=XaGr0U@KO0?m_60`z+UH6zy2>5B5>l(T$$U;)A{sE{+F2wl506t zimi+C@uI9*l_z_CJ+|U@=q!EebTHOU9nZl=+ff>tnWm)=3tdl<(qw>gEF{ zWHdQv)8^DUEyo8bGg8xXQ)XU@lZ5NMZ58JQ&wdd{AUCD`suu)S7vZmTyIY)C>lipq zRtF*!ABH~gAmB97$qhwG-{?APje7xtAg&VN)`zyB`-zXxlJCD+p;d3}B<)DYf0IhZ zq>+jwk}~Q2ffF+t@`z+Yi$pF$4bHMq=okk5dS;$%S)Fj4$57~%K9#IvzpBX?(S=rO zbJFog8c*>i54^3#LQns5{`&@g}|AqAEqAZbAg!tT(=)s zhWg|~SUMFj1XTg`gA_uyfNejQWYG<*)>@I>=u%Z@7f={-ne}8#>t#TDc<~JqU^)pE zC)a>QojM|0Er!^OL0s)#hQmoIwXbo&Hj&f6c{~NpPU$@snOGq$*+~8cdzee_4qBe(5U^>w<5jQ zp1aLRVP1yn946Z$Uo#DA`zuUg;nMBjcc0D@^UaehD{)EnXc;Sug2ejv^Uu6G@4>by z>#RW-1}fB6*JKUGc+WcXjpahS*&evP^_!g9jaO%Q<2rF*1YaIko?m6SXBTS3>ep0; ztQHo08yx0>#alIf*?h3YF`f{M`F{C`&|ErA76SYu!Eha~s|XrL$6IyHNq!&jOP3g4 zoot@v^}?E9p1Sme5sE|~igVW3kxkT}N_=0<;f=Hyc|p`t&Z@-VpPx1)-hW4P%UeIw zxpQllZP&Lak$Hfk@GDaU(aT&42iowv&ih>AQkoY4e$jq07^GSRH90s@8NwSeJ%T7N zHfRKA1v?=M3l+hxMHrbko*!BE!wZ4h(P&`ARzt2nbnn7Kg^dPIDYar)#bePKqO2$N z&6z#U7r)rBC#<|UQAN5^KhVGP@YfH)p%yH`AucKB%5^vZ8G(In#XLmWWUFsxujL%xa+ngeIA-dZU9h*4%BzVra(ceR63|-; zwLE$znaKR5*NzKAV|*80R+yV}q(@?%bMl>$Gc-0?y^`@7Tbl*{1>xi-0BZgLn>CvH zSQj1#BYLT}`WhcE<3(x7@7c{Eyii-+W9f^~gb$-(TnXpj9X{Q9*Srly-mCVu+f2Z| z6Z~^~V1KeUzqJsvYW<0fUb!ZM5On**eR#lVX`gSu2Iiu>2mP<4AMIYPE#^0lv@AHQ zg6$N*5In#dEPHe@yOR_j`J)Qmoy^=XaJ8uV-@{0MRoR#0)%8GYNYSCu{5>)vYp!FO zGGS~w8eL+d9Y(~=D>1r1@AX3-F0N|Iwhohxg{g#%8l0Zr| zewtLw_fvhJtd-WrwhJ67@}!SH{s}yULfD})!ILZ*;+O=VBPx{x`h6+A3y{WZp+U4A zF}71JYp^E3i5uOlkY__I=e}x$RBShY1{)O&QEw-(^(u`qgLr4%t>Z^8-+Sw0d`T(# z4oVtbTgy98FfY%!RKR?4Ah1Fdy==ce;)wR*%s_ITTX<*OQH z07355#hi=#{IXLbQxoL2&TqX6@9{v3)(kwW>X+KcooFJyHrv_$oWc~ooL2w#`UghA z^hR&=u{887*JKX35c`RzIe$;e7YEtpL7(H@D85aA7%Ig{0kdFB5h~vU3$iTK#|;N0 z@hk+jy7(BCHT%~HvYGcSEONFYS4IUQkQonb0xDSWEpj1dz~r-c06K|p7I*~d<g2-srH1=bS;CcR^;Lx>ctYY)+ek0$a&?ZX6YB$Wdg21N@ zqNVHMtU28*`)7N2lM)EBca7xtSDv?I<>XIA)k%w{e43M5^_O`ErPrg(HPUYTSy+v^ zfS8-G&3LXPb5k_Fo7085-C>FpoQ7D(D73=A+c689?^jx&tu1<-63iHJEg+nIek=3K z;?3@V(Kz~DnHXZ8tkigqIQU&1O~Rk7jF2ae2;L09dvT1+NT00xd}-cvo@Jb#2XO75 zQWk6Zzog)YE>1J>1n$s#wblsCdAA!H@^{{DG>>Su+-i4>yrE!@N`_aI_3B=^DSf53 zY^2q(lbyY&K-NCUiX2jX%CcxNmz{~ugs5oK&P|XA#-oH#6HQp?zv<0hP z#gt&9_B>z8c=rSCKnK2Yf%<-UKkVQRAo2Ec`#-vC$mu+lHL9S#&~RF_)hqg~0v@ma z5SC1|s`&YgJNfH=w^nX(=e_@TUx;2}uhUNlt@Z;8UPI3k${OV`+d^yLl2@)K+fQ7To3<^wl{*m)32ri z&QO7}m$F+1?|Xp!p1*I|Eng+w^Y=cOpOBq8Yy_lpr+UhodKLa?H@%tTeD^&F?f0WL6Y*#j>`igt z45Bd4h=<&keto>#)rLMnadk`@RM7AFisw_Q62OeWI^H~NS2hQH1qwXY_0&TCa_dZV zxOSw|y7W}y5WpDS_&YyIc=0qHvxkXeXZESGIn@TJx3f1}Z~dA)5_8^Ge2f)v;s%z0`?%z2^;`4jH7YWD*9%g>t}(XZ?7 zE||}xP~xYIAL)J^r1z@B$xl7KNe5`C;`5OI+}9)UBmSV$+Oc4FwBw?|i$OudTGXZ}XCeH!t?w7*P1-mAj2vr-dq>r^l6!+SS3W$lM&Z zw_gKcki|Xk0j%m4+2P-%G_j2om6Zin`fBPTOY2pD&iuJbSeQB;Jq?e*=xzLp2b zSzdVP4UFjiDOUaeO6Fpj6K@CT&8Cd`Km`Dd%tmNPHaA18EQ{dwq?lKW(}1uJ(u*9FT-R+ zM>?5LloDsD$Obiae%vd7_xdLqevgK}5nk`4 zMqC4WpW?3pwI9VF0N>e*75{)?lembOsB2VIT+GuGdq(t_$J6V00Ju`Eb!OPZ3+?8u zz32;oLTaL?X~U#MabjbGa+=RZT2Mr=Lq_d6-x{ssxiA{C-_ZAWKu4pHlF1G$xL+ypmR1aWk?UsUn1wr1=~L_K0f~kdhI(-#9+mMlN#d9bqLOiwujT%XtJ&QBz!k?>*;r_`{C6v1y(_Vc zbu2eeaOreNmK{Wo>7W|8D>x*L^49Fs-u7$bfZWo*bWBQLtTF1L-`U+w^uTx3Wg7=fAokITn2ch)w)&yHer|?D@42 zkL1D%Pa+E7A?)JrSaTOZUswg^?*--{!aPMBICHH)+zpO)M*8HwcKV=axLkFNPrT&5 zcRzritaoh5ytP?DF9`ucOO%i3nxUPN*f02E1^m8ZhGYATaU&w%(qTcb@L|FCArJlc z_S5YiM=}*9ir!nK>3Nn;=fkmz*k7#k5lWXLZ}@ZmIZ*O!`5Ojd|5MT=NTl2Au63r4Q{L87H_E7o0o4w}ynm&W{o9Ra+gAp30#kDUqCi6D`98 zrjvphRUkz~hX~prgD9eaFsVZgK@cNJKnJGJf-tcqHXIU257a0^4XY5NMZ*gx4mBY~ zWk3|1GQtdt5~Hz%{&v9%F)BogY8R$t#|bg25TeAu4JkYm7TM3x*PmZgk4EBH-& zS@3(&ixSN_^tTNw%>Q(S{$>@!^)f3}q~wkfW|}AZUmlt;(-I|G^iSR}j2at^Shyl3 zIZmiijSwZf-$&?=p;143Ff5D=HPWd<3I{3CP-BIeRw&VM;f0x0p@xl!(d0nkif7S; z=ZSxeQlvcoDO;pSS^8txT{{(;94#Q#3@5Zu1nNKAonwXT=PA*Ye@?4JXmY`tl%)f+ z3(;-r0z1DB3rP=+N)w_@h3Q8m8g;Vcb}DR*3GG8O$j3&FhzWIt`OTo-+Em6yRVKEG z2s3*65QX`<;qHH`LFf#B|D%>rIn-wg{f8p@kD|r6pLYpV;5WlLEE*&tM2)>?dWoKt z$XyIeiC6ZbB>Wjr5E>VGkrPIT`ml;&b6b`s#3K;@Pi}>h5cfxXiRgcM!%#JB6B6MG z@qoN;ci{L6?v>G>h4lQ8NMW?fL}V0(;c~^I-w`vCzmhVEBKoaeehuTWg~=4=>p=XW z6p1PPXGKR>v_v9tq*6^pc?!X-(?wJW%vur$#m!W@RM-bb@c^K|&~AJo^(dDB%yON$ z`w(RKYVYWISG#v#eDt_q;g{`8{#Qop3%R(yPG3<%w@%UOA`I(q9ts(o~VK2uD8(7 zn$tkmm?2=u8H+nB#XDWb@^-4rQIO#&3i_veY(T4r*;v!+q(-`x&8N`>8j&?4k#`(o|Mty=k3%n<;PTBe!K! z9(&>vB6q?xbeyY>TYE;7B?oKxFBhTlOdQ30;V)n>@c$Tl2k1(gu75Pn#I`lDlZkEH zPA2BWb~14?v2EMN#I|kQ$vt_V_q~67Yu$C%s@0{9-`;gj@9wJZt{+YiqAPP)oO6X@ zFeQD_`qeTSIeFGL37rQMQ`bKl{(!Bml^ru3%>8xXJUGxjFXAN0{-HZ-z?SBm%q0Z5 z;YbqN-lF#TT0_CZ_IKlMjq}MQ1}`rYr_a{i2G8d<4&Tz-2G4KSLF(HC&j8~S8D_vI zveDIV71)pbu%0Ys#-}UcKvM3i3@Aw-j#2KqZ~t z5@6;cNm5xZUFITebXh0pBKIIAIOZbGASJKf5+dfJ*0FSK28UAkkrt3oCt04_V$TgE z(S3l!sIpv06*%CGvg|B|dQ0fG=X~u-;m6$^HQ48??Mi3IN_e;DV(dys0UEZIVW+Vs z&}CU^(ogE`N+-ukKKphZPhbJ7?cKd4KST(mAiZ}r)?sWihWB(Pm+@c2P*j6qA zph1^~1Bf<0nEMR=!^P7i^XJe(7p4fiw?uYZIV(s>2moG#omKxp8Ld~(pXdz)t(UJ{ z@QGaI+h;!Xc9sg!n-o=`kBZ=vN|xd`KDnSP80BxD)zI4ndQ^8OJJ89;XsfA5DiBXu zu*@Bt<98>_vz9}GRdhLOSyPrn(tyIE?^GGsophRsJL}lpN&VER4^MHH%TIy#b`sc= zLp%d{zDsd`ckF5495uX!TYeOG)yK?SQCH~g(&Ipsmx*391yg7?8LApBV?r`cyD!pD8DmiLGNAu?YU=cr> zHwIt{(sf#zybegw{ltOzxLf;=yf482^gT^hnB}PTNmnHUUyZ%b3^gvR_H@^vz4B2$ zTt3WsL9Ho7UZqc>Rp}1dPN?yT8o1T>p2CSz+XDM4xAR+HTklK0RFq!*Le&GzV3DiP z^e^$Sm)tPEtUJvHZ&hkFfoa*SAX&BC@0z&u$h2)|X?2-7A8ytoe6-vnDxW>@T*=KR zjaypUexiSrfK-;XdV#(vH54rK96#Uzx(4NO2RdO0o=A4dtG%tmI1Si)^=}dp`8X!o{55LeFamJ6wtS()mq|Ps&~MXdQ+tgBJ8Q)Nd)vTTmCYp)HRtNfwSUs!b@$vG~vyR1M1duNy}s+CW-V*kgB# zwe}0@er=Iy_$B=W*PTHC*R4f!9ah~U)ACEQ`hP2rK<)XNzJu?-{blYw4_;1!f8zXCJrukk5e!1jce(sxP+**f`BdG2_*_tZ3;O14ZM|=q{#03740A1NOkyit>Kf>< z7DwgtX(Sh~Ucs5bT%}r~TmCC-jy)kgB#f`b(Qe<22^u(aLRLp-@3VM5_FEvOGOYZ#Xu5qnw}! zbwI>ZX%J)Df$=QA5&E1h#{(Us@&|*(00ZZ%0Ue|8zktQi0q08r4V$x+yOe_D6ZyU! zo2ViKcBMSC2LEYGP3WNooMHn0GZihyoDVg|T$R62pqWJCd-N~Wz`#HGfsWDm-MV*VqXfSNG67UYHkkiF#ia*sd{<(2=NxpyxqaPJyoEL~?|aD+3L z0Y2(?%?*UJt8M3XuPt=08&KoWz?biwMK}{cf-tQNB=!M#py(}%>9l~BsC_f8-nXon z3>K0&S?RPpZ*rzK$UhM=Jtxm&%aPn#o=6iU4Iv0xnW(ukj!jYveqem6v|5`Xj@|@1 z9*e?G9M%VeO?s5nzT2)6a{{9ee1swXP1Xi2a+9i?jL$Fj1pe9nH;5A?1yne3UA_q%8`jH6j6z}#n`ke3b?f33apyn zMSj)qiOri)Mwd><;uK7S;9(d7nTYTmzU)HG6R;J3Gz$O)!ab4SkV4Hh6UE-f>0ZQi z$RqdM(+ZM4v9L&}I@;P{5hC~`_0?@(V*4U~@WKr;=ln_R`Sqp)KY0CUz9bjz4Kvu4 zzge`+#)=J|#n1jQ)q6t9kp+8QJZb4^ zg+sB}9k@u)k_6^kR9C2QwgYjMyUMW|2aYDry6_^q$}bfQ)%!l0M>P@r@6Cl`3j(Hs zGRi`e%ukMYJgt8V!paouN(oGjEfx2E3I4L!qorve>=(-;n5L|vDxk|$sR*`q3(St7 zc#ePYJn4O^eke`(7>I$j3=Lm89x%X*F*sKi^EMr-b}k{23;d}b9{qZGme4zJrx^_E zsQGQdI(xS@_!8@gVj6Fle^C3ji)HG#A&5YDdBgjY*!cAY56c+x#f#n;=iNQ`Or8(L z_?8&)tdexT33#5|8|_53t}oo{RHcK3jnwN@N$hR!!Eh5|?$2+t$oxCf?-y;4q?H`c znVG`RcjE5IPQE@R+|aXXgg zS{wnn-A<*Tst7?orTvD#Vne{&vc=`(i|Ep52!=z>uSt|0{3QPPKL2evg@OES`~>le z4G-<(2P(Z(&Ix!uCv;0nHRap6a)I8yiem33k&1Q#$VXui(9n3G&bDomW-3cG2G}>B zTSJg7TOl-`^gudNbN~7)wo~oiO<5I59oV;VBTURRHESrJwz(kp2(9)rn;7Yu z+!0~~Dku=$Vse;Jyu5@VMTq<{nS#je-atrlR6xW$C8Yhpz9g9@Kw|ExHNp^qB$=*2 zVocX|Rj_AJBz}II;YdbXe+)=juJrmt&I3OkceZ)ix9w069!H>{P!LEGqp4Ujr+8G4 z%&}{Q5qQQW6J2wQBu0=8?$}K=45J7P&|yqOGIs<5aOPB>hBo{@xl9-}O9h0uu}e&W z`e@KS$#0lkJRK)SJyA^`sd5vzjI%8MXpWq|JIu8IXYVXlJ;ZQw8X@9*O`~&TI44`0?62J}6SH0(lV0uTN*@ix!zSD9VbG@4R zM2t?Ix%&5aXUtt`h6#wb8@SjllttKL99S5tiw5lDc?%3Cc8fs^IuqV~K`>cr{74 zf_Np#pnSjqOL2s}z*A9@OQexk5snfqRuPyIEk;rITvCFvDqe91T3PpB8tpH_bVVxC z0zpM8!h-g%gSKLr8lk>W21PUtK*k7wjdDL;QT9JlVikEQ_hS|5C=2{yU4K#m)15gZ%veaOI$p-mX_6X${VIpZll45ZDjRa-V*P$pLTB0N-9$Fj# z3~)$_&n*z0lV`$({gMG7lLS!MOa4OvIQ+u}uzNqOk3@=cm9V7!i=ygZ3VV3P+g5=K ziThDuE^f%{KlA-mXDM&6UNIKm;hcXw4ZZh6orz!l5ni50`=T}f{wfhNfCT;J{$!^o2TM$q9I-zS!( zbWh7fc3(j)^_H1c(s^%E(#e`{>!nd@D>T#OM>F3vBt6#@+(Hg#D#wFm?`6m##|BxRJ^HwB~nN-E49do!l3nifI8%;HqPOOUnAMXMQ=FB3g2o({Q z22X4NPO{Gmtd7ni^Hal-r8&0&J4}KkHdDI>w4iHTpIkdk*j$*7NC%{Jd7Zt@5mA!b zqF_S(N2rJM=DW=i)s(+9e5I-i^RAo6Mh8?Vafw)V7!HWfB(+WKgcfI~U9}H(cS=#= z$+1&0J(2Rs=o}O&Lq|chC3PZ%qW&_70v4d;cbEd`St5TOhFV1Y{u$9Gt89d8YVQtm z38-_)sV3ZH6#s2L{pr-_8}aLMIIJrY1f z{AIx@_C#xyaK33HPI+^tyeiI7$o*uvJLd9S!f5Ev{ZF%@uBEufGm>3f(}%~KCh9iw zj|Y=9!1s46p3;KhZ`k1Q_H3~^hc;OpW4g?b2z_R!1U?FL0^VeVAfNvw$RJ2m!zq7C z!;ikP5cFK2!I5{#6Wj+S8wX<~EcSpC5M`Hh_JGIf7LXP)r9D9imuG07aTW_Sy-- zPXs9u9$_S&NvVbA+{2U+Z1X#A7Y(E6OjBk{mK&pS+s`7LcfL{_a8k#*VMn@D#J5kT zHje9RS__LT-$3@@WVgj?;1~)@B?TwP$QklTXK@Ii=rRgy?u~|?SvemH<0FJZRj3E|2YLb@{3=Qg zq37cY{+^0R+fWA=gFfS5r<4*?h0A@646xWwPY23Gw0HbsY^Y9_G3Q>Q2q4m(=M)Vk zIJui11_;Gx{!A89I|PbWsuduw=YZ@V&%Jm%QEl~RKsO_zXhzsvv%-)hil|vj$Ra*|M2+=736KGC4iF)4!T<;RFhBsDiRJN%O$#nt282k@uQ*6o zBcg{wGGz9wX9YgiC}>Cx*x1%^l=>@vNhjMtwsWfO{be7P)O!22hAJv!yNm3lR2YD-NQC_sYFBt>XB>-u1KHqq zlQ8gWm%XZ%)78qpr?4q!vsLCD1>BEs_`2VgA)r|h!7ilfSpq7eWL@`HO6|UEOk5ui z>lQC7Qg8QM$r}nx3zwS}6V4i)4M2WKUXT16-2pdu=Adri2qT}}O&dM6`>e**R&H`w zG_z%;H`w@dV>gBr@lEYm(EGGi!Gg~hm*)Zu0=D{EdhkJiTe zA)pgPU(Wxn=nd^_BHqEAzk#KA2@7l%48$S?lu86Nj_7Ohjh`7T8w+GAA`3Q~j+x)w zEe>RbG0&sRBKl_&+s0st!sNL~$u|Jd3Rg+DSA=;RbBn<;WK z&PIoD`VYK3z8a(8KpfRWsb?Zvc3ulgPTsRn^VIa48{?4~U^e3cKMzRq&p_@fhaC8}cia{WU-xk!r|&>4@jz4jAp9Kw zV1UY90Pu+nP%$%bb`F>?1TxTnDm?9}pZ3n#V!jY8{Lu+fysDh7U5<`Ez(+M8Q%2y~ zgV6lZXub~^|4pBiKr4DD>v|_V{D1@~|F6W!5A>h3MGkUg@uHf8l~)NCu|lKw267_N zT}%)BRh_|5^+pu+g2Hqn(+4<<8)}XQIIj~dq7|88#K(giI8QVhla)mjQd|!qrgI{` za{@I29`S!A6KGLp@VpfN>9H)3ycESt>5|7^ikG`!Uro|IW`+7JN(}Dl`giO-N5A~1 zcfFY+AdneOWV!)%=jDL?w{i<<#1iAHU<)Fe0fh-irvKjxj{(aZi6~=xp>1S=;hQSA z3(qHuQ6TQ>nHhx#e$x4D*1+AE_1`KN_XF=?z!$y^z$l|PwQ8?xw`1X|te$?75aI+| z1^( z&}FGu6gVvycWi=R`=k6V?tMiA>BEK^zd7r> ztHjh=Rt5Tum}2d=-WX2FBreTPsE;*TWsed8s-|?Dmz`~(?xfCGC6G=JOhegK-_XfQ z(GrrnR@CV6+$rYdEGl5h7KkfeB=fK(sd7uMIRUV`#GYj#N?>LkTHo_|?wW7=1KR%4 zbo6jT_aJ()6XjU>zMu+J)Y&fb$m_TMo<0NSJ3OK$+~nckA~hesxK0`!alOR9a-3q* z)c!tMYFWb>fN_ny!SP~G>@H7@sTmqCO>8%!V;Ns7C-MqvSMT=SExmaxs7@6nFxNa$ zSg+q}v)g_AQ@GDp$F0rew2KQ(i*kS5ng(^hZ&Nla??rcGX*3#6vR{8Fu<{~+P) z?(&1{;0O!d&|aM@i_s-b(x>!l+oi0^hzI6m{d?L!*?(YC*^O%!aeMEs{+( z+^dAgtk!93hlsc@zC_YP;t_^L8c_u$@ZgTs$o=U9{4q7`@A{@1AS*M=s2!|JAO#qW4#*!GPh0no6|UGZpf~?vdT5Jw(&-cdq4DRX5(a_ z#zMC`J7gQo+*VU8ZJUz!b988!-2M@BhqhXV{h*0p>Q9+wL_w(p}xsJ z?N>el{dvXarexxS?Egrrmruewmovrl#J^H`B_4-DXEtsoR*3KYQECQ;Q%fN=ieG-N zlW3%t$}K4Kh}Q)vDcw_|mpY%|W965;pvuf^gr1K#khbFIAHR^o=y1_Gdf6L##DN!{ z#T9hizp7ErZ2aFTwF}a+5wrZ+Bc!E6=mnZgUARFqQaia~nON)AB`T=P!7=)$<;TP{ z;2~j?@L3k0ESv4iS;y$~(&tx}@y+G+r&rYR%_55rd@DdZ&0EH7WS89ZTLuZh#4O2# z2igBC&A)lInuIg6_NI{+PtLP}%EsgI+2h&<||tFK%q5R(1*GIbH-WT)Z)0o}GQGJYFNPN?9>P=wx&vawZ`?icc*9dKzn`{`agxFR+>c+>Cc6Bxbhur~O{4=t>8}@| z9Y@R&{iOqGvp)+Roe07jVH&16FlO;I5 zpeY|QKS*IM#n8oi`*?RGa-PwXel*yFHDwe}+g#F}XrbTZzi?+TM^sUQmFUm5o!OS{ zh37rFLroW|yM4i=biWE^^~T4_KYK- zdJtmQc`;pWh%!KxFYz(y>d;=xA=dR1t8m@oZ544U?{RVmzaq={SmA`y{>fS8=eqym zS?SpIgdz~hWPUyI>5T?#^|bmDPswS8zT2X+*~NrR&zkHQl4B!;i_^vL7+fR)!~jVj z_Hd#H{KS!RV7=HK=3RsP8GPPD%rY8197fwgI6htX;Q{o3@u9>T|jQ+l|E zIBS4-ly&37DmfzY{!j)v(k=FZnIi*%V=39U?O7XoZLT42rfG$oZiVERz{w)=Mj9Ia zRw1_~YfDnv`Pab%%B+oixfVZ5A}@DqNrRiS)7MM#n-%a>K~%nX@?0nL60$XdDCyLu zj?HLEHo7_OQ;d&;Zau1Wq5uB5@6i#PHsZ9CQWWi}qs7dEKhTFDq;wOiC(7d*$2r$E zDEXy>L?A}@)p`n8%1K#G+rsq(_1z+C&ahsp#F9t+hc>C>KwSGL;k(m>#{8lXKg7D* z0nb{-@}qidoNl(D&@))wYtXfeh8j!qW&^PLX`8;l;ma12 ze${^H8S5*4@IuC8^781wSpq&kmvKT$^m?RB5I0r2<{Xw^&T(nNc4|8GJ8#`e@GgkglK!W5gbQ; z(W;tqSlk?^$m?2%meK1yyv*bK0jBl!6Wr95~!~H=LJj` zwVdW@-7$!{D!1l&7R!{EN@r>wI&R;TMeN23YRFe;-SsRP3$~G$d5rtO7@wNmmQ!ZE zMCiDZMFP34m!_mpDwWK<(`p=A_K>jyqO96uW{O9^9T=Iw9l4TnCt}>NfpPzxbm0< zcCr$%(d(6Te0Ol&dSjcJ-OmquPK4N=S+|LmMsT1cUFHnawuUF3dIo9DaCPcIsE3+1 zGyn0iOU;_H}4`By_Y~bNjIzFc4QZgax&n$&JpAxyA!UFN~x_ z7TOwE+zze!^>$PY)sxA8b?#}zno95ahH<%x>jgB@4db<@=rZL@Icvn%xiB=*Dwc3$ zmy#~6Ps9f*9&JB6Xo?#c)8hQ8gv~@YN2alTrXCxt__ovqqN;_} z0%R~7AvJ4ZzL(`{gV9UN-uA+a-KH~d8-p^d@=oyy9^y|CA@)Y%pu!(5?vk{Cyl&j7 zJL>qjs`;G&NEC7_JgV^$2oz(tsh1PU@ztUvoF3wbWr5ksmCM<1E z-g#EQ>7yE*SWK`4kGfEtn~4Ce+(}BJeOF=LIYUV9Yjb+Fi4t6mv55t3-9aU`LrdMU z*(=}CITTfq`mMjFiA+`RZOI!$l{~Wm2EpJVfpX`@Z8{`)|C<3e$pPCeO-PP>=D~f~ zUE8&?4d9UzbLdWBkvOzj%+5t=lGt;1f`h$EeMn!pgP;9&)Y2oNq|f^-5LcPzCb9X| zkUMGKv+SM6tmD|5{LS+%k%jyEknZm6v_b(tKYn2||Fs>~oR=@pmK;&uCdkO+V?r~Wd-q~+*qxWZAQ4_(o-JMl>>pT=HigS;MFn>^ zhw;Sb)j2RD1~?UDM491WuX$Qb;R;NDQaq;3sX??XFZ_1&Q}ia_Nec``2e9YM(dPi@Zj|tMe5N2j&mB}| z6r8dC6p=WN2&!zY+(Q8c)Yf>G?Km+BX5O)QE^=lGtAUYAI^X7GMYg9g%iMPM5WaYR zVOS+MWbjIEQ#B$aNMK`B4433xLwDzo3nDwp!2y+nX3QVFKxc%~21Xp)XoRpvV31Nw ziL2PchjE4m?2G>D5dFO37fWX+V#9Et!nS1i6Oj125QrP#b5V3XIP+KfmYNQ9>0@+$ zMoe|fnY2Cj$v)r9fH2`YvO8;<(SALg4&zG3pkVeip?XlO<&>sEcuwek(5fq0H;O+v z(a3^_#u%_<=*i?!9J+msmp7rVMR{13tr(Pp#Fu0<+~jbDggO+eFq7z+P(J;(aG;O{ zMUYavG>a*nUbSFKHXgo4bp11)KL)V z5b8l=x{g0d1$>J^;GrX+gcv+hM-d*FU37wI;+m$=igw>saNVA}hqZrbk3t@(^DnaR z>}}DE_&0k%8P~Us+`y1rt;jVjie(^Rqn~ypZK>b$jj+W{U~3oNk?O|4H=F+I{`{!0 zp-@e=hJLGZMj~auA|)CSqSnoRuZ=_>R8!!)WgMH22?^otMKQBgw0wb5;eMO; zUfZDHT=_s6cvqI4kxXcsHIw`d^ifjN7{QZSVlYh*>>W8=leJ3kjQ6SYO!Qv`L@zPi z2PJ`l5iVO()lLxsHX&kzeBIHqM+bDpqTqS4RpOv!*$J4@ ztRT()Q*ZXH3Nni%k=zIK(3LVWi)o$A9c4meuMW-J2MtmSAw8rin)%Ke!WLQRB^{di@aBJ3mSz6Q%0w(Q7jsq@O_54;)`d@E z&`WgqTE(qrk?X&Ng^ODsQG)Y#3!9K=&+(>+Su?F<>u_yv~U(+8RfVnzku94X(8c+fZ(7%Q?W>|0A{^M4FE1ind!~xlt#{p=+ zMz&lJ#K-ba>csmHv6ICwa52~mXsP~gL;8v8eY6w~*YV1c^zyJ|mAnhB9wce#j|1lp zlr=JF;_r#_d&>!!p`VrhyGg&PS|#Ra(KxEyG@zegJ`C-pD1qY<3(ArAg}CB<)fIcL z4T!K%fd3r*AOP%ZxK+{Q6D<;9cPkgOU6+)oLk(pK7TPKBzFO34YF7hC> zkjLKHAel#GRABCrdY&k!)s5=>7%v-0l&C0y{56@e+L_QvE%K3r?rxR){h{c=eB5Gh zGd&a?WwhfL>$S12`B3f7PP5tjMkFecDVhg|BJY5qL244oK7)_pAw^h>=8`{j3b_Ym zRP%IbN9f}qVL&^}QOtGwr)itvo1HDBZS}*Cl5R6|H<5#`-IlkJ zJ93*4js4RgHin0TP5x!yN>yacN9T_O2Z0N!Z&^t$Fj+p%Bz>1lIBmm=9B9WRw%I)~ zZXa;D!V1!pp=6Sfeze1o4JHcjM1_4XYviBUDi!Gf+piPOM0)| zIP~c<2XY|9$*wx{9J4%!A7k3pjT0~&JYq361^>Rr z*mM454{gD`xwBc$r$OT>CARuGLh!5C3?kEsM@wGef^QT`e}1-x`SE6gnhS|}ihy=M zu<o&zr~P;pmh$L&i(|dN4n1F2{9zW2BsDaj^X4p=Oc!C8~GR zzcO)aqjEBvfO9kNZ9Fw|E}LwT`3}FMGp&7H@lV48CpJt;S*22om&o6GtK4>GQVE=R zTtDW@=1j;B`eF5pDV}&x@+KyX#omJN1GQoVBQ2ba)?E9HarF88@5QxRfQf=Js*frqUX9>2zy9nXPk!X36v5sAr% zKxBj*vL(4mLs}$R->2B#vt%sV5+A@#<=|^iz40&Et=tB7{L5-mvx^UiV3dEdl7=s( zka^mP>$sg+M`^GYug>r~SFE-$7OTeupKhscj5F%+GH>F{BvsHk{-rZEjs6x0Z4PhVtZhFSS2gET*ckm1`gjX2?(Y2OB3UJ@vL$MIg zs7cGWI1Y>upOb!LOwt>GWq6^gLlpYu=;tmL)b-6pod|+_eP7s5NPry^xx*$bA%{PO%6rjtrnk553SwgjU9*+4pNh+o~qpfRI1uCm=c7=E*TBZ-t)Qb&7m~f zv-i+<;Uo=hQGB5uU#K$k}b8-nNk1DjcuoySZ946gHODHF87uI#g|DqK&&76sD*ocGjd0F8 zVU_W$;rQ7Jq)V;q#}NY=Pb^HMc7Lips%4lJc+e7R&qDhAn^v`1d*ZosgNe7O*eX}f zo60ODVNMb(S1uP~!1ndrwjBg`hwl-qo}R#2j1ottMz1yF^N?%BBKKrf2#qbE0Ptj1NQ zy0)8{&SZX(9(Pck5636T=^PG%sF)mB!T&{{xQZ>>D7y-OF`!-dYFQor_9M8f{Vc2~ zbz*Y4g|Mhm$UJs-kg-V`fADmI1HI^{8Am50NoxLdU+XO~{IL;LKE33(V>N2p5(@he zp%;Ba@lU*LN!r5I)jq@4%Prk3C0zROV>^Q(4w|aXCV8k>J)|Nz5LrFY^iFN4*o~S! z*ZfP|(WpTfBbl;(rMSNmL-4^vHTiNY4~3B?y_@Xt7DuGe-lTPTVl$RBj6yF~ALHrz zHSLb&99jBDo**ePB3`HT(bJ{RNJefz4^7F*EQtv(0fEi|q3=Lih=y6==p&l5BuUIQ zT`n!ly)iWhp8iD7V71vH>l}%p@3;k8CK1Nf^vbdkk&gr63&@^5UPhR@4u93RMHGL~ z`%k8a96GBUXtRAcncwo^2=4J7#{XsHg-kger9U!G=;qlw}qwmXM2*`Z~x|vBz2kG2aRWg#NGNmU3ke@ zf_Un7l-O04I6f`zTXhX2VkwjQXcR+05a0>7{r13=xuh7q7Fh9&#QW_uoFiN1q}Z-M z_imK0IJWVr3#b^l%)zx_m$`U1M%Q^0Z-D6x7Jp2~h#dICx@0#)TKu(`F~!`F^6vo0 z{j7O|#)jY?bfORw1pVHdb!ri^bfn1eW6 z@^YT4BOGNnSoK&)v%tkXVZ&147*w<15l~7b4*g~N#`>#b29k!#OI**uEMcQS#a)jL zkm-U0_!e_#egk^4(1sGyqx(*0wka`OU2{!9L3UQL&dWgtNT7j)iImpgCshc3JjclikVrEq~<^f(^*}Vt=&`Ie_ z48r>nl^4MHnjfSDVeWP;=Rf%yW4mQ-_a~!^YMQ2evxlI`O}ly)gVbQMb$wocH1OU@M!VR za0vkb|Ld)W;yb{=zXHT@>D;=oC1hGkyj7_D0~mnY7SKS$;PZJz)xrGFD1rjCC*X47fu7Qu_iJ6o5nBO#wd^0HRwlwPT|Er?@nsu+27F zBJh&+H)4}g1i5{+LOGYga^m-@b=;k`kS@zsXYeE!Cxx&dOUn|HDoq%ST}}?Jv)dPi z(@84%1eYe;z1vY;mA@$=67mMl&xE|=$*dd4X?{2|zU!#;r5M4!akAbtz>1TM`J(Il_nG|WI{=KtKzg_S=ju!)*# z)t02N=x;bSIyUpD6Dc&U8Tl@Fd`<`L$E37|3sTXjek3NAAa5}%NU0H1{c=bLUBtY2 zZF{kf4r4nz{XqXrzY~Lk5m^h2rqUIvwg#A&QoAkI=+ipy!CJM&4cB=^#=lIzg{Iwm zcU-u7f6)K=N=*-V0add)WU^C0FT=l%k^5CNI9BO$uTJX7!8|K=dFX*GZrrx8=<}8s zVi*HML!cL$qsvn53SPIqiHX`(VXXAL%M(fH!Hr61?w)tJ%(9Gg(};FX2h8Ak!^Xg^ zXCPwrExaxCRZi{ax+_v-j(^!vWiotT!AtVo%KUT72%p0DQdWme5nRkhz5Xhyb0V4U z;T?1zySRQ@Q^0}|{olQ0^uKlsEI?ZYZt8wrC?SCy(- zLT(EN)oG;Lhsw)1Mbt8KczZ<|uhT)of|R1WLX37?V8+m8Tf{pyC0eO`w8Hi7$`^CJ z7ev6y`S?tX^qmN+KiZXt%WFV_&jet8&c+|TKxKIySb#6{xIUz4Dc_j!WO+Fj_@DjY zem(krJ9?pk5&hFsn|;O~;YF$VO;&M2!nHJy$)S5F8SuHM8*zxU9HivEQ)q~?*2byu zi|35XnYNtu{OaGLOselB9_ny0&lvTyiYd2i-^f%Mj`NyDrE(bD^7GXR3v>TS(}SCx z#)7!B)n|4rvK}pmA|G&;BL}DC>AXqC-mg!ISkU?+_ZmJ;#O}{eAyD*4aH?N@b7GL} ziNFycDbJ4damd8X3CtGlwq4)_T;Z1JLS^f&C=(g9O42^ve=A%NKnGPiGV|d2{KEi7(e<}QlOD7fUtq3vkNO3=z;Kc5M?zWA zA!vP+3{&2G8S2Vy^H~5rsl)QP`h$QFnGe(RnHN@x=R$YgW6f$|BEnBzmUN^Y0vmyE z&wcfm2RB2S7_C zTz?cXbvPtb!1umvArxcW4iOMeW;bkYZ%Z@73~3^H>4*+!h!lH-r|$Elr;20(b-`%r zBiWXKOIP80!E;+I*cgo7xTV~XB-ELS`gmfFA8{PQ9}+zdFcciKxO!?Nm)OC{$1=E z+k^~~rQJPE%GbaR^rN&WZHcSW8_PlhXV|;Cw_S^+x)8lBVrJS*&I}_7zZ0PA*;HVy z(ihjU1KzdS09Rc@N~gd{)UTL_57{WI@Pok zUXG{iPsj1?Zcp=Y2SXxUcax>Yna$l^_VZ73tq1zJUqONA*zF4hGG>mhRslY*I zkeuFje5~0w)Jv&Tb7e84%*5MSK>HB;JQu8GlGHxK6_|0;EUB<#+Q@u8OU`i}ULOdqP%6M{Cr z!Q|lJcC6;nvn~kcc3d-wzS3!+3&_r!R}OE_>$GGSoYe_8X|hkI-4@>lYb1fgmA8#a zH=LT8!=Oc|C;k>QB;ioQSKQ}DJJitVY7Q`~BQ+VWp~khBHP}b|$c;X*L9Rg4q z@_V*F;?`BWoV#k(5b~S8OV%jxnb@Hw$kN&bf;!4>v`)?7;;0laRCy%BiuV5Bt}M zEh|jIHVPj~QuwcOd*_y)=5fXeVEXoB91+7(lc5@MR3{{0Eodb||E$u)I2bO7AP;8t z3GGq5(H^<~?kx0!t|C=QB%8qTb8z+Kc9v~oMK(rDIh{CgeV(r~Cd*76@5b>^;!xPT zh{FJn8fx=v6aFJi9O|jZW%lZ~vHe_aBgfddp+LIWqk}b$Nh4mI~{J z2u5!*$J3do?6b6vi$`uS3u&TP`V_qAHPshVoo<%7OPjKc8|^p&=K}t!j4iaBp90_G zd+W@IhhL~a59n&upCY14X`+!PZ)g@Q$sf`OZlS<&+ejLuD*% z*?)*HiWLURJ~px1Oztz@maW;G(%4hY5U|IAK6p&d!92M=LyF=Y{nFr_*&j2e!b!$q z_R6R3r?c$+lGy!Vnxh>=YnK{Kk5WuyA|RPb7~5Q zund(K^aOYpLzDv}>T++_bmUhNr*}*=Ld@>npnQ+9R%s`YKRyS%z+JsT?rFA4(GxmK zt7;VjSrU~)Z_1hGJ(toUb{SzOZ=6n9^iiOwZ#ZposbzxlWL93uvSYokB{Agd|F|LZSmV414aY2TaWAEhm z!X(iyb|eGRtiOl#`}kW`>p-Zf`ic+Kev*SSVKjzg<^;wRl*8`p94#;RJ4589Qj@w{ zYr}QXC@m?wo3aiA4_VGQ=n1+@rg!oWgkr7u3>)j)3j1*1e!dg`3Q8y$>td}tzITSN z{Oy*-QfXF9{|Lk>{BaH(u;RUW*4gR#J`OX={n4>+-Gygew|<7HY*x43`AYQ&=92O{ za-G>qKg$irDE4j&)c99G#H}WHp|^1Ltd3d|`|xM;qph?9%NxF|8#cY%t+e)~IaU22 zZU6(4fkXLF@#cZq6KG}y%$p)tFV63^;tMWOqD~qg?UK4bq zf@`XEnpO`_>oGDA9SD~E9{Wk@mg?(GYd%4(X8baPt)n1y>vppejswdM*zw=Eekg%* z3)C<9bSvmZ9%4hfVQb>uqZqW+Ukf%FiBz=)bi7;QAw2kzIHk>mo&H!lOtP^rUpg_z zO;A`-j>Ws33+lYNTh`Qm%vj$gNZ2mkH0-MkCZ}WVP2zT@I51}jdl!{3D%n;C7CE)8 z-hF11dVA&hh`g18YqnEzbrlhVmEBF4>$H0=d=$z{51Ob{d_|Yr<(cXi6SB6Pk$MwJ z>*a%NBa0MClf>bae9N2p5hV!>v|^zc_P*b!_!%@s2Fml1QAoG#Z6-TZu^PjwN3sOzUlb)2A0Nz3N%B_cK3sE#YBGghIi>$$QjYR>PO1in}dQW?1}Ph znX7P1g-Ocb=dY*9q#1$_tYlMl86mHYZpq)1y)f9Y(LNn{-kfU#Ggk;}i)i5ue0Ftn zpE;)2osq7apA74d7b5Ea4**F(w!eSrf4Hs^uY2LSSKX$I%HX258{iT%J_?&|?meCd zi2HSVH~>q-{z6CbX9u?Xzjm+vF1hZL-tPDJHf$?$6n#r0^8>o<*BM_GjyTQZKQyb_ z(>NN&tl_AC3(k~I+~YpbQfzN78hmDUQJ-&%=AW6}nA~8@9_nel42JAh-^^oTw%ROD z)0M|`wWni!>%Or2?zY=%`@pDeF#O529mt!u!}UkQwwjMUjdPGDYJTMg5ZXNxzxJ2D z9iI5xL$Suvw~PH7OW{<7XJWGd%`?_DY@0oxxM^f>&%|wc`|+~X zgKfpC-4eSA#!UTGIH&Fr!NS$Qk$bd14h?PlckGbAVTYL8y6s`=kY7OIn-?}s=-sft z%u&1@Am-n@wlf(ke%grRXAL0|YuH@#GpO5v?Jw9JD+V0n{~8+n{ZPX?$8TW780G7M zFB^b3_SRfg0+*M<EUm#gd4|Z_j0UA>33kO zyFR6l%du!<$?&x<$BK=$JJfi{GjPLYqsBJ1Z`mqj2n5NM
@dHnBt{0BTuWg$;f z?N0og)Zw4s5Q*jOZ!l_p?rALA>1m`>VSsyy{}YeDC>;K<2H9lRPC#{d)*>JQ|KI4Z zU-`efj^+UT?<{>kycM&5vA+xMmAAjx-$FNf@2&_FNHcV2Q`uIy%_dd5jd`u%C(9!- zsKseei%n1qJiW8T|Bn*?*HDX{;qhFHO<0Qqj{s%(KMy}R4lGhtgdeaGd}>@@_6&CC z1nkb<@$Q5>4Sc9G*1j=5Unx@E@7Y*{v6-#rm+)+G`;f=q31b4P7cN@@CMN<@7KPvu zU}m8v`YE%eGXs#^%_cHJY`LOTiX(BFAR`#0DxP@k56!}^R(gx7+F2(R`nrukEK z)jp_KM%~3}8%L+W6l|N5Cj%ul{O!YsVsG@p;?F&-Xt^*u%~3xKnSUeo#0GH3iW{%Y zPQ&S<{#SSc=7ag8V8c=0fV=*k^UFh?wFh~;!=SazFZu-QzV=`S9jc~7yZO*SI<%M$ zeaeTNbm(_Likke8YyQ-vrm?ZQ&R9()jI83u8a#1jW@!F|=+9 zw6THxCQl5AP_@9h@St4ba5B`r?qEz++wQ0z2P3HMO{f6lXC=-)7|4@7%`=WhroIjYacsgjjDrIixQ3Rl7*sbe4%Cpfxc5*a4&2wu}-sVO} zT_nx%;A`a9-38d%@2GzOnSQh50rb!6B0!DD)!gP-ajD(2c6XYm&$_xuQq6U+pQQJb zYc7NRWWAqKGXnNg^nPm1P}on^`@L#Tg#BK6Kdts?*zc|OVFXM+t0Zq-aZ}1M34B@1 z58z&fo!jP$Njs_sQ}mX=Z_IAr75l&CLCO#5CuiyN@P}9Ji`5;p!{!ObIYuhBmw!}B zcwOl};J4aKsfY6Rk4T<57JZ?*4Jn?9`@=O4(lB<^qdx%7SDMGa0gb(V=f0Rf8U0cr zIj(mTfYfk!)~!95+%(x)ztj;MOed-B(f;X@>mGLK_ z0vAD@ut58Za{s#7DFbVc$}{A76u%ZWo?6m)X_~w4yS|PGP$ep9%AA6B2?u%G@>)H0 zoxS|q9rYd%!-_hrjK(f0~`nmyER+HeIvT?K=dQ z+=!Q?`0CatxrhI?u<0bXyJ^%`$NiYaL4OYJGa}_7$Nj^2{{*%F;OWHCv6_?Xx_aH! zuXtkLmrCsK#Xh0_yv_Jk?(Hx({i6rC>mt2tPVm&Nv%A4bXd921)%f&-YqTDG&J^3o zvmEsofyI&6I_hU*g~qnb-5o`CX*eEt$EW0N9+~S{wAMZRgQk)`F~hg8L(>#QSrzODy15n}A`R5cije#eP^to#WJY`eirCaL`_MKBzX=L=X+1T*} z*RtrVrcuetRv|PV74RT^LCZN_ACOVlbO{_h*ypbE$jT*PWyGA*XnwoR7-`>7w>}ku zxT*d(9e>`Qw;u|IFJB8ZV$|LU{HR!Tp?`Er+t*!jf8IYjt?d)be&4qDO#AM-_Pj#> zjTvpL>7dO%dO+KYrt|ei?NfOMbem(5(Y_e`y=iv$@NZ@=^i14dVqfQ2fz2=y8f7N5 zx_@7LDqK0UzhlKE_PX_U>=H)Fo#Eksb2Q8YyVN~!A2{9q4D^euZ_w zPf-|?=!TDmOE)6*+S*2TnftRW�S|G7w$H6pXhE8yzL!p#ui`&i)7BbaP$|TR21Y z)?GKkUQGs~v#@CbG;e30yGjs>YXORzv8!elc$)G*`y)bb!c6dE@*n5zIXQJ3?RAlU zH3fCCUNtAdvm}`f-}%hVnm6Gw3Nw(-`XeKtd%OFY*!!;Xg{j{EN=J3FZNkE&}she%)w9`tYZN7|3(>!I|_0ge@8 z?Jy)g;50N%`*_qYJt11Z0hR=hDa(Lz6pY! zUv8Z_%F|Tv<>EBkB_sNdo_PW6EKxgi&Zye}WCES<0qUtav$)PkGF!j%NmKmXVNQ~i zuOH=DQIe9>Gy}l!IiT7Xxwh7T=PRDL8F0|IuP*{PIDj~a01n1r=er39kKkZzDoz>x z=IrO(O>V_HDT3oi-Az+WlykoBzIzZEz!xc()*Xc2znwxlce1s(i^ZY)+Z}3tsNd{p7zhUohrj7)aG;TfRt~8h^~tbX;{OciyQj}b zCUVqw;J6)?GSJ4;U%Xj#d%>d>4qxZEe?44TJn^d%`)7_77d2hZbIE_e!6N_K_I|p3 zp|fUYI#!G!Zuda9NEduoG<=hz0m0ig-NI)u1zaA_KPALpbzhDp2d?%pNBvrw&T9jn zrdf}Lzj=*30|>?(^*hON*;mqdTN)-j`arDhsXN&Jt|4y>z=^*2UKL>k9rbI-tGg#H zTyrmmV7H~Y8wX?~LfzmwXSf?nvIjuS_Bxik!VNxrN@0`BK6fq7TzLi^NxPy5uIlS) z%KYQwX}0j{#zPdsL8-8}q(d4LT%M*$p&sJ8aQjQ*($oymPjhPgX&Ume)1ZsC!G$+N z{Dp?Jm4z_hra#X9%r{nlCaDhdJ^O3cgG>I_&6e?~^KV__-Tszh|GV(SS9810chtYf z?Fy!6%?~erHQugYak~~ZbT}HmgY8ma`i6aY@Gvd?aecg{_eo11)$m_#>GOXFEq%p; zcuVj9rM0DZCT7@d;kp-fI}g=Q_xh@MJLgb71Y7!+7cpev_M-N-+M<3d@pl&b|BfE* zf>kth8V&%8vAx$=+8g?(>M`{1bnl3VdedBp+P9ee@2JWye1MK+c-pTiyMXl_ki z;BFAo?bAZ-YuhFdz06LXpN{!`Igw3t`R)m}*U{3jtZ%!Bn_45B+C|@PD%oPx#+E@L}F2 z`_}6uZ;Rf}96K34$i8@E20UitoRo%bHSUr|R~pP4=QQlEEz8?n*J-aEL)n5d%4m;p z`#*p%vSA|uYD4&4fN}A}W{*9YTOP+ZMq(GLt5+j$^K)$IZEODkaP5!G{RC@Y*EMJM zc@3Lu&jt@_zXxTBn;@9(M+N(Pn(nqk)ZLf?RoZ}6YWM4WFH)=u$cET0&dpjQZ z2K|^XlJYit4s3(xEP`n{E#^M(Ug7)_M+|S_8U9{vTiZJ{cerd65kWotTie!i@<7G^ z6bHYc^9j(fZBLr!n?ycj2W232>sDJ_c4(&U3!ow0{v95F*(UJgN}A3*4k*(IADE+f zZmp*g&NPC@P><`7AWp9x1H6<(BP53Or*wkSP2nUT>I9{mfHO+`m%&s!q9(c6e{!k+ z%xuZG^^jloU1W32N4zMruwQ$uhg$U-c+|qCyQyc2C+>io*EE&*-wn_30IFb8_F+4{ zfrYUC;VaG(Lc~hfR2Da$pAC<<(M5K9c*f5W-zR9ryK;dZk7~2mT?xsS2S9>2S z2Uf4d3lRzK4KBM4F0K1JLTE3X8V)e(vb*QoSfr@?dlK;Yf$e}J$BHZMu>;%O^RKkU zP>Ayk|EjjV?Iw*6_XaxzzR=4XQjW2;{W5<14FuAwgr;Xq_`#;H$v~9=n;lw>^6Gpw z+G^|FMJ=@aVI(cxJ~9cE-_sdu`x{&WPu4aH^^aoY%ec!!NtwDY&q3TwV=&g-R@_(w zL|po|rwRH2^WIIfpit4)@E@~nwxvEt1HL|0H)~VUtJB~nFh4Y<=hOWs*yFd}eK&T_ zo|<08!%xcQ-dS=cO?Y+9=V%9m@KWM0S{WXErr@Qdv1nz@4h599YD>J{ShTZgI>bM* z@a~xkE*om5dKyc2D!`(oa29}PDuCF4CoXNy3P5qZ#d^e$+7vX5#(ujb>}^CC+7}3J2}OUx5I;HvY%7x zpG3!Gz+P-~G+;j22eXE!=`OpY;W64v!o4I%!?m=RjC;wBhWlwR1@}@M4G+*>D(Z+{`;jJlm(MiZ+!lzx)&e#Yj}jI`hT{?t3UCc!&m=8 zOZAH_)xXSA{ZdQyFSk^GoTd8Xd#L`|my%nD1d;d@CV%AG2eB9LNE}*8)4OE=;kj3- zeDtgM{AaxB@87n=lArMSyYe=-f1>%8@@MVwe88%NeDtb>eDtb>eDtb>d^FlO^!?Yt z{g1PZA9_a9{V9KLy8nMQA8wAv83V*GY2ou`C>(&J|BS}LJK^hQAPy#`(Y%y^1H?-l z2divm9Q?&*#=&x%83&X{(Kz68f&bd{#?5Y2$f6e{h{siNK z+P!Td;ltej$qD_RlFXFWCf#E7oiq$; z?oygXdqC!o&%-a=%oY1JKIO_Jd)vf#o`UAda94I*pU7j^X!Qa_CnDFPPyA{cqcdJT z9hWUwz|K^&)tNeXnn|DdHe2ZvtqIR}7=5DU+PFTEbXa{NR?w3^(dox0XtnpCPyBIe zqCR2vr;GABloTDSxz$rQXP~WSB21%=cMX8ry@~@RFBTp;2q1kh<_0(6I&?=S6gQ@1 zV^t?LjvoMy%_htZ8Z@jtxV+}IEKJJ6sm5soNk0ci;pZ5opVS*&fg+?cj9kOk_#ck9 ze};L0+X~PLfN~Q@4QSuIuy|rucvy|1zKwW?n!Ii|4qs}zX$iC}-?!$0a28l=zYiMe z%lX(y<828fG;0dD61SPw=Rg!l5$OP!8L@p6;}0GXw_q?_=&73xO;%9SROQ1`|9pKc zjgWtYca+$n%`S&Hf6nCq$Oj&^VPLbn1QQYm!XKU&iM5}s`7u9qNs8^&6dU_MtKlS+ zgEqQFqI*kT0|td>VwZn8>H*p^blnNv{fkqiKB9$ty*mFTD3*UwoO^7hhN7!USfd_v=s74&>{~+d8KlGV3Frrb}Wyt&uALH)YOU#{)2cHftQ7#*;A==FY2Np2kx~pq?jy zkj*jy5RCy4IA2ocU6uC5&u7+T7CD0l`;mcv3 z#$LEu2^zr&-+CO!jOS9!xUyI6l_fxL#Z9AQ3UkMlK&X!(NE_vwc@d!PHjTFZbqQ!| zx?Q1dn1xzgwpVMoL~|WB$YZ7=*WOMu;jqL-h)O^3cjg&x-^gA7&_EoZ9i~e?KCtanwK=fu*QyV5Yxt#61AibUT6oAiBQU)2n4D=lQTsU*IJOOm`#p%?{c-$abiw%DR;9qUp4Sg! zt6G8DGo@+X8>Qg8k>R^E?5mO$tTm-|ml1N8@>7KyE`V{DT)PyS=`c^^Xc&sf{do)p zAF_<)s=#FXV(6!n=JkldWw-Jo(xY$@siXdwF4iVibb_grE=(ED?1gYHLpVn`_3_@^ z9>>58~2pB3V)M(I`{lARuZZP;J7XJgS<=~(vC3+%Sd>t+5dZ@YJ( zIlo)pcIrvi{S7C?_nXoOo}X)@!|#^B;Y9zmDf8fY>ipT$dpcjRQ_edc>gjy`N;!XV zLl5WGRMRwTDXA|0+NCfQUkIP~1c9gK+(y@cyv=#rJdJR||8ZUT+`5Ak@5~5~`V4cX z%~vbGEW9tBJV4yXwYNTOlRvzxA8FR8KWpRx&0p$`?rv?=C6GWC8+Q- z0j-tP7B%dty|xtd*NvkFsM-vM$~{)%|C$_te}tFrrK*iJX| zAxn^#1ZXkD@1VXf!4thPr^hEclb>jz&Nq!t!AGh)IIw0w-NEB)dUIf6D{nhC4fV3} zwgn$s)*o2N@mpG$$Z7zXaMb4@34)n{L}F>({iE<}y|J1#9}@xLIsi>T@6il0GdpEq z%?vjl4I z#At;aoW%VG*r&0)(@~F6M{(UO9Ku)QCh(F{Obf9#P>7^8Xwr9&qbA+JBX&5B*leXH zB3-r}A2Y4<%YXdp6uS4n-xi6r&57}R629!W@sI}7w(YObeZuGc4MzArogWPkF^x62fLHSNkQ~6eZ(N8>lghWLvbX^@b7lTFivdNr@O_U3H|Ti?wR`~E*7GdrLQ8Yu1Klt zJf7dbsOub9a{|C&ZZlqzxNg6sY57=c(xzcCo{+Eo2zDE37oK1Pc}cj4sL>4&1DlVd z&4UALpD$^ewb0|=f=zzvM|AP;O8gP(a1s{3UShMg&lkP1+kLj7z`*xu|80CcfziZk zW8s_gU^>8=Ar^jXG4~Ufkg*AO4Co#I_fu#j}LfxFkSEQ&kBQ!+cYbjw?F*s z3|fpFt~+G+52|sL_@*ehMVJpLUuCE*K?RkCo9^ERHkd$3aFJE>ATp;DPtsq&qL{vntol~)3WNi45h zApHBY?OH&eW3siCY;6TwoBu!RzCApOBI|oPnIsbobi^nTFC-d~h!>(LL{QQ&4Lvx+ zMU2R@c#GhItE)I8CeQ)IzjNx;srgh_Q$u1hDa0ZN)kJj~mc98}qqiTk zO(nUoN?Xca#rX0n=mt>ZyIPW`Pz5nQ->AJ4MOu2j9VY+AuzrNDv)ZeAeE(f&k^VtR z83FhZ#?U*?-;h>zA*{o8^c_W_{Jw<;~O7{6W=&BetcsAi)Y;MxE|j) zgnfTW|Gq3sk8jNOcPI6XZxsA>L;Y=e0zaXSg*YaA&oe_DCq?zHCBE?weBk2PxW>N! zAH_A!w!}5gE2nkmc95^>fZnb15#t+w?GfL2ut$93ws`T4Uw0dEhekLfO>_)F>}3r> zv~}wth;K(j5MPSMv$OcaesSUtdr|yhixGcF(lhUF6cs)Ea54)&^q{2FgQT2LokG)- zH-kjHHxDIZ@{`xX=LPKZqJbEG*q!FR*&j*rUK9;KWbu?0R{7WC4_PPrQ21eVz82M+ z(t-PQ3ZKh7Jvnao%IAPV>&x|@mGeun(Kk6_N8+CXbzutak)Zxx<7I#KbhiMKy zdW&CDZM;62q7E_S)L);(q9B)7N9VV{o(qTm=Xk*DFUNSmUPe5ih$C@EJfK^T2YlQb z4_KSv%x@A@w~NICUig?L9`KkiCLXXO4dVfm{vX8y*5bI>aa?vr#{~g{ppM5Vz}12} zx@Q#N8-hAFP88rg*%ATx1VsRjJ|hC~-rtD;93LA2ID#SoYm=P$TlwJCu>pUVL<9a- zrLlm&G_wEf{cL9j{FSizSIdPzi)%ly8QqY?eUGT~tqGp!fnpChE#o%e-F#y9Dyf z>l}L;qkqHiu=B|Veih{Jr=aOZ;}DJCo6T9`_x|%-bR3d63PYjzy&MkxA}xII0$dM~ zsm9ps{x4(GXGU~vo{Jfqp)>xUjLnO4&lsCI_eRI2&!w?rb7q_$j8Fbtv>eURG0(qF z>xj%M69W<71^8yHyUDWuyH6`W3xM|?W8deb9_RPwvt*Kd75dg1U&O{U`7Coh-!fT$ z&$H%uzRvwY^PXGPjk9<0x)nC_m{x(!sa%ez3W3nV zy>2z>;KT7i&}@M2pggAPtJrE2|2+8tsL`pMxtRUk0(&w5_Ka?NvJ_aj{b3#Q%x50S zclGbf@W?l!ojU;StaNiVh;|r`p_Abz!ggz)z0`hFPJ_rj9! zIz7(DU#sJ$S_Dn981Q{S_D4}2sM{1&x6@)l61v&qfTXW*fkKc3d`K~sXa)XiU?kAM z!$2Ad-0(}(Zl*^WOyZXn!}r<85h@Wr!fKpqfnNA-S{4x^!D`vhTrFKj9R!m2W%=+O zFp_){FcMeG2XO^A;_Lsz2m6zQY{p7SU3&)^Y*EFZ zDs(seLU=s15MTcoCgxf!#lYRmOIf@t`n*GTV1deUF$l~Lzco;`1inUh|GxjFi$XCy z!D_cMz?%aL>IL#nLH@ztJV03dQ#BMHIlwD=L^*aneml`l1)`G;^7g zuk0rVw$9(-k-sG>0=PcoTP_(-hSa)K8*jpn3UbZDR8*~jX*3ZUxbRts6QDV$1Ec($ zaxbTx%P+;~Nxm}0|6>+(b!3S*Uy^Iw>LSj+e^H=%{sbwwL6RH!(!Mm-ee(8rayvcT zPFH^QlaE7#>pk+woYb8ZHWJu_?@4S+f)6wUr~gH&pgO!!@1HCGTYhN+${9u#ELltF zc!PJkTXplC8=dRM`RrPbuY3Y~=8#g`-1!@QM_k-yAdR*!f$ir$h-^pOW6dWPw*URM zjtHOze`5)n{}?~2)04L+ubKq}SoIwk--T|a-}%+T;$|G+R@$w&&EK3gMM$mow_m)d z*uQ^(M;*@zsg3^b1hF)M5A49|HuyUh$xVyScc}s=l%73Zl2a__PNkx6YuX&@BOKTr zr{v>6j!??ERA(y3`d6Bc(S9`@4tIW|?<7oqNp%k4j7p;P3I0ZC{SP1_(6S&mhEt8W z(MF-B4K#qniO|GxLEhvps+Q{n|G`D$O7BbHJX~!!b56oX(@TOR1@hLOz9;(4juz7;&%>e<%BT-sP zAqKl+yhZ~w;d=w5_5L|1t#?I(f(kxQ(~eE(;Fr{UDT|5 zQeoXc;Fepon=bl=c7H?Dbw8M(KAFQ|EN@Xnv)yJk8(w0_A$IXt97XpvJ5vL;&W0(` zjUbH0V7oq^qAPQvJ3n^Zub1?FA%aI9bnJqDk;mBQ-z%shSbJ^SLZhlN4z_M;AGKG3X3Jb z^lrC&RQnC2i>eC$O+VL8@n+9sK82-voXx8HJP zRtMt@3&G)Vb1a@GSGVQ7xI8TOyE4wM|4_pCh52dl2SdJle0RwkJ#w8znRw)XgVopK zl6OG&B}thv#G}kamwAO#@eBf4cFQvx`2Yfek`aIk{)#MC#yFHIcHHk*#$?Fim&%xQ zS*((Z#$>v=W{ibssV(7y?HmShX9Bt_O{0%h`C=v?z;Fx^4-#%Bhvyz|Vm-!n_P%H) z4%b&8Hfz4J443TIV*sQW)6CfK6U*JG=P?>V_JHN~3BvT|s1U;F8>s2yTER?38UV@KR zf}HGi@{6&whs`11kc=Pfz?bkEplZBjEE>W?J&mS> zl304-9pL}}iudTXa3J6DJizZUKGi@NQarw|u<*UY%6GTyDc44v>M5Vax_nkJ`IOKj zvjF9Dg**3Wx8f-$`IH~c)J1d*btgIJU=4(i7Jw+fHk z4CPmVJOCx_+Bqj85vWL5{9_4?`W_?(bchco!Tq#tzZ|m%;SmIXbkyDvb#-OAm?{Fn9bRgZ-qrx(6 zL0>xbQUz=%b939FJJhV%B|0*0^?#i#Sv!xM2gEOsfgA2Z8MqT=;3bO;yg9ZbBIfN0 zI}O~|=dQT2@&uHFr^oNaryhHcIOspJT39@4oO8B##E=h`Q=HLB+(pr;L)r1pM6qEa zBgyIEHO&9_G_~}AJr+d(ODnXaVh3`oBD@UB-AUzc!*Z|pC|9qS3*E=&=5u1CGO|D<|- zAK~vfi-tc&;F=Hc30RS6(^1}_IC%!6^~j!e+C%k>#4lV*MvmfHr;KJNbWgcMTB1RP+`6rJ$%Mo%}&ksVie5DO@a)tf`zqiyS>K-Ju_uA>$KRki5 zX0@e%xnrPz%k_9qR48a-M6j)WCsBPtOMKAD^!F1~#g@b#i2CQz9TXSxPXG}((nv{2 zfLCyTV=TSn$9LYgyz?o%6S~Oo6SmZG=1Oudy|5Zz_><*@#rVPz3{Q<{mw8c})47q* z<)?8MigD+cM|B^0y?2*O#_%ZO6lEa8C}LLXy;BiA4jrD+>BCekOs+*}|Ajwb`_(-= z$zl6{KHXz|a^b$CC8g`)Bc(Pn+nut?lxbl)fR@a9Q2`4@T*)jIu}c42jeq&TWl5l# z@{<_OamoJ{aet+XmJGq+Z|jo9XZXOwC=&iI5VhNrQ11E=3i816;HlsPbL~13dV|DZ zibEL#D5!9SG9^PUT&_$>2kk8tO#y(u5q_?~5M8_qfV?X890s$_EMr0}VrJPN?J?MV z@wB+u{Fee!#zpj$Pwb(LHy03ck4<=B%u~;Dz#3)xZWXL{ZBF@xyJOdW z90m|wxZJHU*R|pS*ls6)M8)JO8L$vZc;H!~*X1ec6t|kYv0|}2C6f=HL-1&;#BFrt z@8DHCn+u=Z5f2?{7Znf?2@NuEkr=F=KMng&<$nVOv_`h}e^i_vl3xb~%?YFCJ+Faw zENDi!qHzQ4tO9a6u#1I(44@X;v4tg;78GhB{EGgbvKng};akoJ3%i*~kp={t{A*~|*X^Nd{UZ_e@`-~i^9tB)V|x1TQHm)H9v9RD;9RzaYJ4_?J#Y9~JM z3qr(3e{(116mEkO7-ox0Lf}Wzk+`?uE9bx3DCSkWI+7LN2l#IoOZi)tT99r+O< zUl{Oi?-y^!fw&HZ`7_HvB-~6u_UhBj4}%e!D3K^Fe~PZx$a7Ebeh;^qHs{LFrQpKo z_`(WIyYb8%n_Kmi3sW&k(S|%w5-%`ef0d*gNYFY?QuJ#UG4f&3AXLVCVM%cXaI&K_ z5@`^hIgNJK(M~f?To1M@m*QeIdX7x9&j1jYMe zr;+cs$jLd4=sMC6#6(K8Rkn0Du|r%g$*V_m*Q&{ILicCa#J z%6APPE8yeZw8zuQc}EE)qinu>0gu7@u66kv68!(5Cio!J>w4WC9{ER)T^K}Dk?ay$ItrY&h3!6 z?@-Q{-OF@l@f768xjlh<=tkg2MwgBs{NaS~M$361ZgY4w`oHNcLAyJ5iw6vOZddpP zQ*H-NVet>GD6eThD{_SgfnqVht-I zSh8Sn*f4vMa-J@=F7&?X!LP+7P-vwf*9eREGfE4J*ls zs1}z;R$~pqq`a-dWU%2UNwSL8b_8SMJryqb1m!8y9ymhNeMqocuH^%-rsyt|eE71A z)_<|Yt;}|6!SLB0WplwEIWb68kN-&z1ko|Zp+CM?hvNoKpakFHM?@xv+)62ntKo^9Iiz| zdXm$Y6gF5i`zYG5QmlaaS%$0NOg0;jO+=|*lh9)}P9FvnBQ4R)^RJ=@DN2i2s?$Rm zjDwGoGA=F?sz&I+s8B0aq|Y|(tYLU0x41MkFM-8hv+w{`uUN_lo=e2dR7%Hs@@sPW zD>RmoCTzf)NY#!usuikIkm8wCIkv2QaD(2!dGLVNOIRdGS1C*aS!6jVFqFK!t&H1q z;PpASIq6f}@`H`qz>qPC_`p_U2>(V2v_Q)6S_z@Wnd7kYfj`^zaUCo+)(5XaMJI7} zD4-?9AA$e`kn{fHDoY2Kq~cGk+ui|r8=#*sXr*WLHIP9Xh1k8Z!2M#i1yl_jlgm5dBov}_Q0PJ z_#DMEvv*I%*>ihl_*09%RVJj<5rI*l%M_Y8Od*L?CWIzY5HUgf?x|Rh1O%cyhi06P zuItfpU<`*v{{1=V;reKbj$MfJ?!yLJ0Y1Y3mK47aT)aAwx(s4<+{VV=4|>tf@9MKM386Ls9p$0RRedN3m!Ub#{oPUVk>cMCar4s3GBP z(+!-&)h0Yg(bw=t zG$(-Kp*&;$>a#-&+6^3lG7KV_M28(pk%MR7Pk}b2av~EOsT?KmnNRf@k++9-ti(o- zvM`-rTAOUV*)7Qb(SDr`w5z5#DXZBG)g=l&q74FR$jRueY)Pg$c;{Vw;8X&&y7V7* zz195h_~46dtggJp)Cw3zzviu|R#<+mt`%&|FT$kB&y>`!91IhSk2c1SbuQosUf>b>dzT>}!L4qE z^bu++e(5?pCEWuZm4!NLh#|ioxzjy(Mx&dXQ7x#>9L`W$o)NP$8Ls8k6>PSbu3?VT zy-xWZG9f@){cyj{Jo$1i4mwj}@8ZuSHFV^Rj?SdwO5yL|@U-y4UI>Hvz(6_=H#QTM z6i(DNMls7-OUyX29ZOPw1?~{e1Dk62hd!=Re?^annjLk7rKEf zmqh0$6L~yCsGTT^ulJLB>irAb$-=}a{R}ozlFS%FeHs49E z2-1n_WEOnr>o#zZ8=4HAY9Vd#%vffbPz=Ru#`@%uPm_u|aaD&|&w7{cqMp`{GpaDt zql_Jb=@Xy^Sg~fpAHoZ;VjaF3hRp-TwDet+eGKdhE_BDQsM=pf)t-*0_;J+OAo79x znOm}3G_~bk&P07g$}F9f(AGBA4}-3&fUeM=NmWy2ST7$)p!mJwYHnc#%5BMTo9}hX z&G-@v5EC0i`zTx#wN8x20muMK$AWSjv<^T3pwB{CzfmhRpB3LqOnZPOaC~x4Y8m&k z2-D-gVbg3<@xk|SLewy8dw_^)wH15zcvQ-%drPhamgQ2Dc#)E}V1SuMt>Eyq#2 z={F1%HN9==gmMms{@PCUf9*8c_ain^5oO=ySL*D0zLi1k(DGl6kkwwl(%XxpZ&z~W z>OgOY2J3GZw#ME!gge)?XN^9u&D5Fm-O50HGh6;flxu~SI^2G);sbRy=Ir$ZH{i}T zZbOOGSY-R9vAg;z0Q`?q=iS?43|D5IoF?NyB_l1{tL=6bY@dSm5C?_xs#(UcX9mUHkg$pTS9>D9Oi^wUrWm5s;+78yk-{tlj^Ry4+_vGb zu7Tn(YU}23Zq?Ih@mSBOAaB#H|1m5-hP!cpn-46Ur5yoK4f&pB{242-|v>_okq zyAyHVR4_>B$%H=A)WL2}(ux3E2zUL#$ioTC-wK=eQs_-N<)CuQKkEj)xB^if^roB- z{EQpU{*gJLfOAuH>~aX-(bi4FC5TFuI@nMc2G#q?C@9Hz@()4w0+g6We(7)r^RvP0 zVAG0alb%#;$ce6+6$e~KH6b;UYKq@Rc~-+$TOyze%X?bbOoeMM?o{EBr|r@X}jkVUkmB;sc-Se8LHp zS3QRlCgT*D1f1Exj{G`7O>56?kFw@m++YPw%?I#491Ps=!!-V_y5F2cIq*+ri-QS% zvru&`-VIjh$ghftb=!UB^xov*}XL$yLF4}4*VcY?n$ z|M>b5CF=(Spe$Dqs$1NluRlGC z>8*V!GHCemX~feA3(b9ouWsytVTK^Z|6ujeoSCif|KErmM9&qutjzs?EWWUVjTh=j zbRmjMI2YI8mW!(>4xtRiCe&@=E2;orIZ7?R)NRKl)GgB&R8=N|DW7*VsVhBirZL|br&%{-9V9<{KYcaVjR9<*{YuYXQB)4bkCGi;1u zUc2(^(Y&_jka^7%De57nlQ0Zz|0L~#-w=CDyOR%WGE8dnEYtI!wM`@Oh*s2|Mr&aW zhGmD|!acS!-~~&bv3M2U22ux=g+3#Ly|4-aiM(qL z=1wKgSBtjrKg{+_YelPJW@%_&3);A~Hp@WsfdV$Izq-K@YdSOjTmh|x-fKbmI7D^4 z*3Bx@-^c3sK=?G7%vIdN)rQIZ5rOJ=O_RA1iWsY=+#*a7F>@Y6T8Ff4dq5w7stVmh zHmk3~rQEaeS@zEWmtl1OXR?zv_xJ$H0nM2=)(ARH?EIlOsvKK zhd$^w*K2X5#bdLdozI#CtKqPM z4WSTC0!kh@rYY7O-~;DlXnygtnBP5-Oo8QO3Y-t70CtQV-0mFR7RX4ZHM;+GaZP}y z`-6NI!>2NQg&6|9(1eLAP<0#tK{o(q zDch-hlldCUuZ@d>?ETGA+I78+E{;Xh>*;HwTTy4jK!)zt<0u)Qp*S!Zw^|wg;)71@LVIv&dc z=&nFcfWZDa$BN3k?1aio!WljBcz`-O;?Fh)YXo;2KC7 zVypa8mD3?AOCUgM7UAiY#yL*Hrxk!tSNQ2Q6QX)Xcu^tVOu>ib;I+ckZ62F>^ZWQH zOnucuFm(ltO$ou&$2k+GRv0kV)`4OM>ek!_I_^zzDv1bF_5B|?J`0Q`*w zq$LcH!U%;qkR#IsM%)#y5VjU&|Q^BVUq;tu(Q2ghG;Z(v9@Ac=L1_9R9k#iT=4Wu1`vI) z!%PEo({N^fsi#e#(w7;b1y{w}OVK{cF!k!)*c>9cW+Y|;DtpaupCvzELZMkvr$$e+DI2c8I8%5Sb zi_lbrC_m;#y~7Xnjf);x{7VJ-xD*@n1^yW83v6M&z<&w10~A?9zQAhXR^hg&FR)Zw zQP@Rq&b22%PF>z9f%we>%%X z2CK)Si$k}j=;uP*vgY!y8-&`Ja7PX@q}84i#T`svjXMK(^crX4j>uq!$1V4k4Rw3m7#|JDf?+^4qqKS6auq%HXAM_;Xv+i?hr1$QDadFnzK6nFH7mMGi z92nPsc?p(TYsSYICiic;e=zSCodLUZ^{xI_x{R&Xv^P+8X_xM&gwB4?HpqUtEG{sdhQ)vFH1+S4a2LJYd%hE^KsjAXw^Hj?FOy zfAcbBaRskRPR5~hQ{HocN-^oX5BF%dgYGRNc!*NAbToFC?%92&~PO%@z?-G8xekd{B% zgnw6GjvtTXh~YcDdV(|E7o-ob8Xb+-`lLPTk9_LVc z;nKJ`cMZG~Dm3DFFlt^0(Mw*6TVKkndySiVsn3j zX%Zb-kV7)L)3EKd*REodE;$NM6{ZQ_xFoJCvNv|H_-#a`4(woCrylOenn3))qVU?a zlMT!ld0i_OnNbpYq>YU!e&GYRF(+jHC2?`|JE_2wf;L33KhVqXcC%Wr0zQyv2JC!k zcppC_`2P>n``FC9k1a?UkR{v>P)i3|m_M>XcR#++!fY27p3{y32sk+j-h_Oa3mW+?>19 z$kzV5`P#Ql5#)cnWZ-%WJ2vgifu(gkV8Uv7qwHD72hzxGT`+{g@*`BSh*zJ04&9&E z5@}Kj*CDV}#~;Ug)Skv$LHr%3`2hA5moA!IFqdq!72_!!Q}CaT zIC|}BCEy<;2rTMIlT-10>BRiq zcoXpT2tkYLTwF3#EbsL8cFKn_Ky$N}xQdX-pdRt6V%}C#mzcJSlCJbPFOErSa{|@A zUU<_cA9$5C^Dw$^?XS?*$RuMgrM(NNVRxtaJ9_a;t|R!GbR`Ibyj?09x>DkfN1Ds@ z`n;oDNV zDN7hkyNJ&BDWmdQP$slX9>RmFBn_coqV|6D`P0$o|2Cg1!~EUAra3y#7(5Hq!gFNl zSLBUx`Jc=gWaD3}R-Lv;2ptMB_e~+Taw(&}vRP$<7msTzgSZ#Qq=87n@l#R=knhF4 zf2h~d<;_5W!s7~hJIk>3Qn8MOYpTAw@IY(4YKl}eDML~o&cxCB8zK?9QII}B#lh2f z`7EPCo6vLw4?tHT6`==cfAqi5+x%w>y*Dv>U5k+>blEE@`n?CX(9&qUtohu9&%f0< zqCD)7iqiQ&8x+I|g-1R^VZ?t<`CF$P!rjJJlgg@(BT*^?U3$JY%kz~Mx>m90%I$VM zBoA_Qam;uq32yZX2Z&V2bDHoK@)U#f8@bRCSUg$^(t$2K(4vk$4&wS)A5a&RnUxQ2 z>coJ>tpeQ{nYSnOqgkF(ydZZk$JI)n?_kNVLYd}ztB)O>(+_d*vnm{?1H8lFv%mc` z9~{RncFKkC^6}8!IzBw}{)!!v+$rLXZbydml?k$^%B|d*1vB(|$v?{GUF5-G$2`g& zWo|lUDp(9(Mkddj3Dff3i(;lF?q`FBPI394NVj?KhLt*9l$#Y#70q5IY48*nw6f3{RkIOUnE5_Ejb@@Wr0<32th;2;nnT+~A^0EBH2S29U1-b5=N zB~ZJM998pI@_`N%L*>Cu;rI0NuykC85~UxiNRY*q;aU^_BJ$sX)~wXi{fnTJVoOF#EOnh&I?$SD=O|+VLz(dIAb8hu-dt|}g88YA>Ns?KJ0^Zv zPo9EVWhNBGoR~snJI-CL+gwDix}Lg~X7KGdo5i1yLW|HZ@K&hG%+Fd}1xxxST5SXU zIF_IzGhQoCp~;PxE;4vl&s5dY-gFS#SAqU!Frz_+z`9!%o?+lQW>l{pMh=)&c`bVH z4j`O6rn_YWc)(jJ+=R~-MgWBGxzG=$ z-D<^=Lz{53IB%H2ew~JsNCVy90PKC2=^L{trveZ>$WOjxD(R)Q#?Koo95bLi?1yNZwiwV~JkSdGN8VsFAeDve@60m0Tcz2mC>MAOBJ!~vTB==AUb@J ziT7$3AHd)J#ibEmtYwd#ay4LA>~78|sTGi`gE>6Ufd@V2_zGo4ng7RRmjAg$EK6r7 zc~TZ1u#uxieA^V>L)!Wdme^^%_aP@(pH8KCgDI<@ht*~Y8UHt?K*M{X4s zH{+1x?SX~WDF-Pi9^O zN4Y&q1SRZC6$2oj44bLiwagjThYk;Ggi{-tUPZOmfL>bCX|+QB_x!TY3FXyc znpuHcudRHKb>%7Cgknh^K#rX9QIFiM6;TR^Q6*j5_)*^fANY0q$c=Bmgl2H^S_7556f`nkkF1C zcG@@>m2OHqns1{paxAY!L$DL7H}@xj?=0!KK0jGn@de${JPk%26nJ(LYREnbB?9{X zI(n`eP>+Y^#>9hLX4*;A^~L451cyAvdT1Q;D%Y33Z7_SUV{No(o3Dzlu47?!{kzTj z0#kj`l5Z%Km&sktn3|K9JO#{+2-kxdEm(87!PKrL~acI=8%*ZUR#tEDuj9)BS?M zYkH{M4kYty76Q!&9ztO+Y_#BoKccAA;yOr4iR)Jb57G*rK{t$Ymi%x4F8$Ysxnt=)E%#g)K^2RGOO3|r}H@E^Y2}?%& zG~W4)+a}s#tRH4OdYot!a6r(#M2Y~9{AEh=7)|>EGgyu}&XSXkX+zSszu6ga<*%Rn z34F=hqcyzQNfC7z@^=&my?#xmjopF%%2!7sMw$wsEPS4UYq&OcbaGuL{*IL{ev_2V zqdBgk+c1Bz3zj_N=Ikcz$C@K_54Wqxnd1mo``6;UN?`6;^rt3z`{N-kZBHBS+mH^| zHp6F%hD74>_JsSRKO9pdm`V`a*D8>FAi%~5n7b#uTJbf8+L)><)k?9N4z^Dai*I)5 z_iqoEFwJ^#wYQh#$2^a-639;VC{we*PDt_D1JyneO{F_L%A9oED4Pfd&2?bVtV@qE zXl!nEW;HuGpb5D&<$D`ge=+7yK$kt)N5|vZN6)68VITd{Abvzs1Zrm4KHI7P2eg90 z)WAo4poKK2uQB`bJ$md{2Syq*Tc`!z z3T?9bdl)~^0H99RVn=Px z6{O>Q!O(F$B!zrxiQ(uWE#){vuk`?6`K5Tphg8%ikXKz>@LLDVL9&`QnS70aS=s9> zl9luaKr9iA=&z5_VLh;A&k8-Q=ak%;0WTLaUr)=FTI~brO!3yoh!i>%}*ESGH+^E*H5`= zJ@k{VZ+g6BS)V~)h;29Hm%=@4h$+JH#!-unyzc$P0XE)dd3ayYj=jRj_aJV7Fz$i`eoW> zD7Mu-l*{v=n?hxVX0ee@p#W7a_3ye?`G_5>Cav->rd6;WEcq3uE0tGXimy~2iTYm* zes9S!`K@z%uc1_ukD*fWRP|8a#e=QP{3M4l^P|v?sDGPnH8gfb=$iD*IQk>-J~~~* zk3w&C>GAq_=zNgGQULEn_d|bu-^hEmo&q=;U&U8)fIaf@r0^p*78oG$E~zpcwV zLP?;Vmkye3=f35z%uuvHSn+?`*yfL&7A&f)%eNkbm=`+agz#F+{>l@PnEjO$@tS%yyH`cB&+&}&S9?7atAbUS<;WLtPUWBj*f9R1?Q^=*iL-swb{GS{K>$Lzi% z^Qtk+@gE}kh8jBuKh`cMgzBR4wpgq8gHrL>zw~XDVZcs_QJoT9%%-%})I zzC!`9cL(J@{>0zGd9PeJz~7PNea+vI>|3qve8Q|M{G8!8>geJFv(e%7LPLRrc#kGg zS5TNzU9#y4%Dqn-=q>r-m|fM&`z5d$F#I3Fyk?p95z-hPUQJVVtJ|GnWFoL zspUghuH^4gK5RD6wWYX~yLJ9y$+f^gb{G7@N0Eo`A`ffu^=BUkZvD-C55^%eybSyB zfJ-TcE{Jrw$>r1qJUg-QZX~}KkndZd-S#-rEfU$(W$et?VK8Lyj!(;dO*k&N!4SfG zL1&c-*jE~!zVsbmt9lu}w&IPg(0GS;4!T^DXRP1@f5y$| zWPV8((^}`-d*BLY=W3@fU=F;`u^6n+VkWPCNFpw-VDJK$8!;UdaO)nF!xy90$B4(3 zE)Ww;1GzAfMYy?@{JcH5Qicad^WX1fwlb&XE$?J0Z~;N*ZFMQ$oGh((ge9kW=XqvY zVjdgqPgGD`DY0EXqHuJJSnM&ufU?w&!ya~ffWja%5HCq-p_5h zl;Tt!ZQ>n9o(0kF0oe5eu;;wQp8Iw>e=svqY!%^uG5?QJoQ_9HcDR&&Ew#XdB66z^ zB!6OKI5(#MhgkogCx1dCrZ&O=%wrng%^<>i&#~TbCHn=6>7AfN_=X-59L!%5LAkm) z(iF-j{jLa<$uMwEMBg-Pg-6NednUsYOosW}68ONuZlk!?em;|d6{u!)8Ng#Yp~z{o zjM$$=$NZz4Vfc7eJ{zsFvOmR9XSy)?x0;5Y9!Yg?+zE^RN`C1u3Izfb-5^#loB`O% zTn~LKDc`&H=7ol@%6lK7j8^S07h<*jaC-!J?tRp)WaaG%4>bL@jL#dMjg_-`S=zf@ zjL|ng%NV@{zi`iyn3d^bM(%U}gDVG_AVoxwQo%n%@$?G}|Lr`06%C6IV6eG10qPCc zWBWMQ$_T}s`;*NgOvXeq=36G?(Mr+xx0k`DG$8=(9)O@^Wy%=?q*`HYY^?*4cLRM2nma;Bu5 zha+&0Te%!1-q&cH-?of{P=@JQeI)Cx6rw($lMTb?%PjK#1Quf#i^ELI?6I#SDUa#> z)%MfD(3lc+7hbn{?H%R=is)I#5~uT(wfa@1(EqQ=+40)*5ps6CQ0q;@L#Ro_7Fs0e}izOO(ceCtseIcaO|Uw>uSDJsRlj8};n zDdCSSco-D*KDuQWlP9a4DY1EpbX_B!rsVmw>lS6vT`e#Q8)D*}sJ>-XAIj^r&K~lr zO-DtXYmP5e7Tk(g)EXnC^aVMYa5}p0*Al9=;6W^siB$|ft;gZFNXmji+P3D7h#?(; z8^{wsWe|oH1z)D=9?e;BMPL19@^;*YJgEEY=$AwI%hn$IJy7{PYvmZztXJ-P)Hol1 zwxQ{@1xWt|rk!W3O^39EE_VL_#!`eHH1_e>e%ru`BazTla(DOsv;zeWkFUnezvq8w zkI)qY?2TXFI}!;m#;e%k7rzgSUma(?Ggdq0Jk&XO?b(G)=DfG*@v!MF5ZIF|2s(mq@Oiy%uOiMaRbK%|AxXO3Q?s&Y^^?EG5 zYlz-~2EG6IkA>ceR(j9j^eKFdom0|t$z1PRo*+uD#hKA7Jp!3fdrf01Bx{V!! zF_OGg*vt**wdts@Gjd2_14KoQ{Uy$^I-Fyd5GqU!8gs0X4}5838v6siJzXEKZY6nJ zZZd4!apO95{r=EXzrhdD{WjF29=;&Z>!C7Q)=)1zr~&JwL3vWjbu9StN2A;$0F~? z=;PS?8psDUNK&Sn$5$zm6b%@+vB7~+O(mmXRl?{DzLJMkuHSDogG(i@kz7N194QLO zGrNs_b71~o$cvy0$mm4Me$f~e>WH@YEIoLS?f_y!-K`7t_b@&_GjsdUW7N^_e~l~9 zl{?zwMb|rRq<4NIy^~b@tA%RIen%s2S73rt+7F;PO*=A!Vkd{Oo-uvXpzE8wYSK8r zai%B-eQevcQCoQmthrB(_!Z*pw1|1Grmy}J-kluTYVa(Gu}k56>;4WF%8tHwt3&^- zx84qrAK8i;z{!N+lT&dO=yTiK4cCw{&+%?IdAIKNevKSX34CxUN^hGH_j5+M$MkZ6 z?+v9;zSA=13nCV!K*vL_CM)`D(ysxp%<$7z`^)>;nh1E zo;|565JX|aPn&sB-Z&ALhNrdx4H+@cNs6~nd+&VbN*fvN6F-lGf7ElUjOTcswC8hn zuo#~{8`S;HGeSQHqxXZWy@wm`AFGU8f2P@fy!sQ2`tzdoKOIwl`P=&Y%i`BR-*|p( zf1LVz8TD@onDq~gslV+l{rzj=*T2(vzARq-iAMd8MC-2)SnI#Tcz^Tixb+V=+mBa& znocb` zKR*>C|Hk{rK8r8^X8ZB#?`O(?wEm}K>MwuYkiYo#&o`bQ+Z(6;OjG_BN98}J{z@@U{{M;JzVUqXZ}HnVo<~CQ+c%!`ztjH9M*n}OedGDW-)Y}?z9@eCcNou?t&A@} z#`9x);L1 zRVHKEk;&fPBROxX)qhS7ilZ$B4$iAdMFnY6Vv2WP10Ei+S>r8?_s@m*&#q!mt^Oxg z&(Bm(vtAEiDDSozp|9`nJhOk8+UFenQi80g!iw`(eH>*sW{_l=R`1@Jb_7>+lQ^>!cOKp^E{Rd80aHUf+yw|wp z!EU+80fb<4RVFpGDSxVtez5SL_U||8_Cca|IBvmzO##o8K%w}BBa!gOwEr}YfPz+XZ(ivz$7SHTul)k)bW@l=jEysB2 z5%f-YjX56lYS>~wwk9K3hY^5ge)9DK@k*!Z(PZpT^^xS`E$b7#GmWGA%B(E;JD6AO zNVYOE0mt_KlAk!=cvM(*1Uxqu6@Y#Ef&vUrf!N{OyWH+zV}T!N08`w_IH))wGRr&YtQ> zC#Mm1TdL{k?UJ{+a%)DbX?-4;XCEof-ca(-?uf|M2;50$V!v^!{cK@z+oH*wjbAzv z^ODqX!eXrizB`qP>HqrhgAcB7DtBcF>LaJ*%&xKwnarftBM4nf{37B zKqIp^tWL7ooa#M^!3_d0ZOUy@?FBXM38Ly=9DzpBi-B3eYT5_1hPQM$b2pBIH_PBn zZp)eh!SH0`;o+K&BzUT3+^UXF1UYrp*0d$$)+A0$TO7Kj7-OzBNZbj&6gR1*#WlOm z;%w5IU+!&d{4)80d`TUu661l)n(k3j>tWykFXf(;X4iglV}cEMA*qv+C4Y6I#7ni( z8nyNzdHR#B=K<>yq}&=weT0){x2)iMRB<*`;gYvXv$w9Mls=;q@meyVOV>lSz=r!=H9y+j@=mBmMbnadQEklQHVWp~Oc zdWwqzyqB7nF(%R<0wJnKAqrK(*Pi>8g)W$j78jax_IMK72S(dR3S5mdu^052i6q`b zEi4XURV)jWh&xtmP%K5|0T9c(PRTok*2D5nT`0E+x$DNLuUv*gIZl21G8D1ddzf%O zni$*z!nv0TXD0~flS@E2BTP8kgqp|z0lC#IkVpzOT?0^5JJbu$5lApW=Ed9tV|oZ@ zouG_K<91#};#uzEs`=9KSmW$ztXBf|s?M2szOcBHdbJFbTjlK$gM0YWhv^*WBYK~l z%HtWLI_nhDK$Pgc`d-Lg552htdnL|3h}{9=a}veiL6MgZQHOwB=#lqn=ur#n(Q%=s z8+)V(8CyEWs>=r z4F;z!#pBUVV7oQ|@Bds($~_FjUWa3a+BHEQK!I@>BtM z4Zi4VsHX6%bFJ}&J@+?-#bGQ_7BoH*_89*(2Dt8POX7A;enL> zjg-1mQtyH~cSv$KU-<+lW$)Gbf|M4N7VnU@d_A6#&;wy_2pc8F^+@<}f|Bwhj z92ex%+E)0Xq%*G5ZLLK40nC4(-?c*f+IKs5z9zSNe81 zQ#*y4pOf=|EqjHM_L~a$n47Ab>3F4S|~XUA8S)P zQn#mKE)DOaa|M4}qAv$ex(PK$(2(gap?x%E86goHvo;E?|< zNZP#1FqjL~9sMIwTsr&%qI*Eva#+gVBIR%Ko&_tE8Sh)OKLMl%Fj&nzO<2iEa>@*N zTo<)BBeE~h%BOYon#8E33ZbwymD6tXwS zyKthr)oHbE_3=&cY`@^&KUe~tfD6B?bsPK?68s0RfxmTP_IIAtBQ8`9kx`!PW}H+g z|DM!Z_}JjYn&DyV7EgAqQynwTUngYm0s_SBwwiDn(@E9-AEA5`Qsc>P1iFwW5O*Z2 zl#dd|SANrzGLtH39Vo+=kyxF&Pz;4g8c>KN>C|DhtxfSLOOBK|%#>foI& zqHrQd6qsbX)bX2`ZUuD-+Qj4_erMc+|Gps_v+IyGsYj4dPxgLKY9r$qu&dFj-ZIXa zy#arK3A@pQ^B?O6LIwZNqwxF~YMnc~*a&WwK;u&1HUc=byP1-AmQL*k$e-xWMt0T{ z4eA}W9(B?tsCYkUvi?LGvak)w^T_LwGI&@6|9#_0UFXc+0|ddBJ=hzd5~&f@v4SV{ z8&RDw4q9<%?-H{$F?%DlvC)-^^!}R>3!fU@sT(B!;l$F35&ayy+!3nO&(%P+$wEzY zI>~JSe--6=n2LR&dwSn+IZ+?M&Pdq+`dY9Ll6oh|!0prrRNQ(WPzOREz;wkv@RcX9 z3$D~(<90z%AB*jRGy5ns4!wZ()37JVX1zB^3+oXmv~QtDr94pBlR5#F{qKf}ul zTh%uZfwc0TgeBFT{Vfz2C-|!cXZA)Eqtuf~8GwP-?Navk)Dx0A)m}O=p?+NAO(=O$ra@xWD}x zehJ=rEpMzsMJVMqx$>(Yy-HG_>}RKazFN%_JQ5MO89gIAqwNJZxa3wofO&c(uEXEv zkks*Z-d_j;R$=FZhdGRw^=1p0=(QrDpmeU?n;`?%`P`e)JxlhU-EG9hsy_RgA>_b_ zxVUvvZk>ktn_X%~isbLSM$jWlFiWIceS)h`=v6-&;PH}|=#}FZf6N~52!E#o>*M_w zL-Q?lKnJE}hbKr!T24z{f>7$Qd(Qz4$bWuY8W3_@pKc?5&(kPjYRyHIU4$Q3E66i! zGS`y<(et(~nBc7Knt6T0rF1V_=QVhF#?y2&J#Cg!x3tgNBN2~kPjRWvWIUEV1R(Gb zcm8pYI-e8jZM}s00?zg*9ah)9TyfW5a-H~V(>!vsB+qRFVCFy2=2D&<+9t_Q=E3p> z7JMUD|aEPrZ7j!P|0!?n*P z7ux0BF2$VzmATJ@%DUBz>s-0NLKFTDsBItCb^vM<~x*W_Y|^x!~p{9rnKpkfTeaj@YHGNY`RB} z(t$6U#|v_y^;ioKAu%Fqzb7KThd*mD5u26{M8L9N&{Cg>zN2)h<|tqb`(s8N8H->3 zzrzz}#MAnZ3}L-1S#-S(^9S+4UX;eFnjPGc)NytfhKznQK^@6q`m3*un0rYToA4M% z_#6vnD zNK;8RN7%DZEc!3X%R(A6|0r(8`PKgB1i!{%JM@Otzu=a8;JPD2DiZN*GFUXg13cxx z4x|K!P^PeIm;7(A98nMor2Ja`sq-l@Pc_!lAXZUPmLI{d*)f|m-fi$HXT({Q6z>3- z{}b8V-wLYhYcP~v_qPld)W^{ntH=C)eB~BNuJyNEgUN!)Og!Ab)bl;qh**LQC!F^h zhsX(54+Ixz#emUjOJtd|6kO_H}@*s;DKVaus#Z=yL!h=v)0?(8No zlD+`b2+b!T%6jHW1oHO|-m$#q%SWIWpL&;*OgAX*JD#0j#C&bf&@!Wb8EL zZ2-%sq%>2W>9qYFiM|K%T3)nQ zn3*DRFpoI#|FHJuaZy$6<1;LRDPGDY+LTI7Q%g!qGLlgyFFCwnMp;VvW}2Cn=?vOJ zLc=7tX^Kk97RwgjY_Y623@r`NK+6=hT(VbD%gzX?p|OeF-?N;1X9lKs`F*~BD0j~K zoaa2}InO!gS;`zEr>$^}2fiHTNS?MydFoxr8>6g#yr0FUwTWIQbe(=#3+=n%)VE7S z87UFk2AigfQ^)Ca2Lse8ecfZTG+jmd^NZN z`bL;nF7w|Oq@PcAR%DqQ%V(+Q+P|-a`io`bdpOg$ z3EQ30HDLRHUf}QO1wZY3SR=u8_K!%Oh*@6dpcO9101%7jghogT?f(GUind2 za6N7@nE3Z-ew3kAChem*jrLWrtM+XrKQIH0eH_i%aBerGTVb&@MLIhhLzfs6U<@eW@}EZTPY zmM4-WE-b}A*IP&_{|_?$_Qca}?7;}@RURyhk4;scuJA>Yq2@_|PjP%cIx5ySD1{}E znu5-rlxK0jPV-H%rcXVkqMc9)`jru>Nk_@?)A2A+9TIrRNbM;)Y;rF76sd6fxyvsa2UEWfO|yc0g?E)U4-? zuq=20e|*V5T>bIK`cBQ{R4FF`e=WnT1y;)=GlS#-NMC}bv;*ndHzF541FwPGitE{Y zh&#obII~}0obP)?5UO9SogD<_+LZ{UKTBtSyrun-e+@DC7t!-=QbIce1i|P?6z?pk zXWH?Q$nJBM5#}7)adiJxytN6JUJ`~6pl(NEw>d%Gj#)`M)QP6&Kz>KS_fGY}dVNKV zs~h^XYZTY*$EKVmnA2@eJr0+x53R3J#_Op3`(p`x7ty_N?R&xZPjSUt5P+fW#+|fJ zlzr3zgd+K92%;QYDa7P0NB{w#giwD{^<@dw=c*Aszoz~k0pUtWCZ;UPfIR!UI4{Z|KUf}Rq9wAK-CR)A;;-PB*Z#W0H~U>1 z9A({QR?Yr8`7i!W)I>J3!NVsO$*Yb^qm`U4w6LZib+5%z=*~X*gezM0G`J>vGu~}1 zhboEAHTef&R0rJtNO$uE^A&T;ihOcg7ng(l)kIQDy}VhkOBb}~>ufrNZXRegs=@pN zON@{`<~=@C&vQM``&DI(y!ZT`ey_6DZftLVY2*EOL1P612l8d1=0m=n#?C$!PbNms zDa&7nzYxjFqOo)WurEQ&abNZ_3Nz-jFAfBM!#Hqc-}ocxhqy7KnN(SKS5pK?1$NIbif1&L zYAuEn4&%o;7~it;f5p+*d>Ln829-r|RznA3XH7qxMVn&((XOP`$&hUb5Wp(z-x5ro zXoD}&OfCp>pADm5U^V$;wlzSUy1028x6+eH0ap^0SFOZ#BVF$zTtOohPA@` zL6!JyUL95LgCMMD|Bnj-b}dlrgoWpW=t%L*L!&!F`5Jx!J4!}#jfC~jr9s=%Bvg{A zi(v=gw*<+9No9HD+7C}z{?06aG;N{OVWpYG6;%uM0 z^H5_2(xdZX}^L7Io0_j~%0V!mq?j=ZzpWT0mKN(SJHG}M}JcO0e{vW!s)insYCzqh6&hJ)V34D zfQX#7CSCzmVp3ZZJEEdJGh+*{?S;}RDgz6`9F+p46q5N!qew30GLv);F^S=9RCX{a z{&l7aZS>(cxpu80*!lj#$^-FV6`~TadfBBe$3je_c&JkCOXPtFrG*7Y1txdh0O4z+2^+7h z&fZqAZ^6Yjtq)qRw70+uG^y=z*VJ5J=DZd^PF!EZ2VVTqbo~jXV5P{8(4_)*7n+KG%PZL^o{r z$S7*!`5gh?Qv0@Xe&a%2jY6{kM?kp0WFhoURl9CF%AbgW5cyM#4FaYezNtJ^n16#- zpMQvEu7z-Th3_T8VN0K!OMNd9z0kbn^rR*i|JbBw8*RdP0(y}=BS!!2T0I(Q^*~6g z+o{#{KL-QaUOvRlC{guB?LtlRBL z-`UDHbcJ4^gNj&8u{PPQKiZR^_IGyeR2pv&P$&j!$ER8MCGE9aj{q?Cq(=0tSzgr@ zt0ZlN?ASEx5qnY{7PKWbTDRJ*2W?3wp-7B9>09gvVTC5BgF3r*#8zGzJJ#2?Tbh@^ zv?m<`xSm1ylJ=%qbr?%hQ<|@DlGjxLW4Bq$?Ma8zwB@v6d8ZZHleRHn2ty52NK2|n zY627i2CTcFxGm`vl|d{45Ktzm0)`2BNv9AQs(U5QjaBhH3$&)?J~Lf;(Ed!CX2*|d z@G%=dGM>@0@h|?kXQP(&jOtyQNv!ss@24vdZJ(~ps+JX9SkCo~mLcm=L)h#+mCC@Z zBKW%vHf{L(&|Y#8uU!q5_o&*0SY=>#nRJW{hg(TOsUzSgy0lE}xL;+dzNm8zoBia< zdFQCfjm|ho?4>4GDNhz#FnHAJRBM`+aiY8?)~1!?&xrDx__WTO?8@p`{-Z|P$`2;M zw`M9GJDFwGDET+WEZl@8ItL1$6(}4F|D|?;AGY!uC|u_7*-vQpR&o9&t7S8a z5VrLsBmA_EWQ3FOXYJomM(BKBUFiN5Sg-kYrL=&2hXxnKmcNKzG1ki$b3{;v+n3c4oi7-=Xu`ZSp*yfAlV$vMLs6+t9S&9 zUzze~^P$WwqRo?>hg+NpPyoNy#>yj??9%1J6q0v^d(XMtS?!BhCgdFk@~ybO9Hfyvokn*CT8D^*>zKc z1?cHTS%z?9#N|z{cMie<_s_x?06g}5OX`Hbia(&`4)!DM?HPcdL^sVGY_up9`Jt6v zn4;I1zkg-FPFX5$oWEupcJeF?(Mzj$l<4+a3#sqNY8ylZd}|fHDD=l{nUqw`{+NQw zgHXuf`hbIV78CBpt4B)rrFtfnfd45OJBoFngu-kwsYqxOZSt5X|9!OKm2$>YZp z%6~SD`(BDc=9$&D+}hEp-1Ihm>CNQZxBb zxU;y4?*(UsJAY5dD}yKwlf>g2*AZ4^#`IkBUG3GTUdQdRs_zaff?pVQm9l=k72S7t zK+RVKxvK4TeUHs2@7Agn58XDyKx@~)#&8k_<|#%!<<&U+4c1`v`yZEn2B5V>j)O+U zeB&Z)4NwHA2HM+#PP|&?Dh22>G_U1BrQkA3xko7&Kp)Pvov~(GMY?x>Z2Ht680uco zR0PsgaED#!$>}=-KL{K>2LbTV)#`5%LLMN60EO9rQKA#B>-#!~VtjLGo*vt%i?e~$ zLYn^w>XhRAkdFYE>3LRLUfW|z!7(f8)(%h#8fgO^XH9AViJ;P^tlyQceGBsh)DUBJ z_BP%Yo(gs zhNnqo(F}U*)6E*I7t7TE5xi8B>25T^W^#7ff``=N-ZP)LV>O!q`}OW2I^_xX4|_ z#ARo+zu1Ap{}PY!?4PIK~*xa%2TIN^pgY5-~rD+6Xf5~xW8>jU5rkZ_lfz7Y4q`G$YUGvDHO2^mzeg23oyc~&$Y+li@t|6 zui_rSKhk>Ap-qK0ep0LK=mOg!9(uloREcKqagIWZzt_^exfY&=sKqVhZWn)DEgD{= zUvkS6@C*B`fxrHHbs69CH(|NqG`saBzn)xrw>S!8udNQc4?XxJTvDcKP=f_yYT`wD+ zT2|Of`%viDxLW`m0z)ZS8_Q29colqgn=WJB&{KZFGqLpYgF&w-V6i=^0X9}rcQAsE ztug}RVEkC$7nTU#sQu5$#XSNbaYTX`;muvLR;LBkhL3FGvRj&TTje6%@DNiqu}-&j zu2c$c?#wajZ-_zbxlOzLIKx2bNw zH8rp8ETVQu24LKBH3f=r5Ga9`U7jP**(8xOl?)3y^&VpJXoBqKtV;H#_(-A z7rM!pb^~l;v9^XN74|B8m$SKBt$023E+j`U~M# z5tezi6QXZ4odMFd4b+TXS-*2dUK)r>9=soG*cj!yenqO{c@z3vk?7on<&Cn?MgheY zhWbJE#&SI(`{P4qS3&(K9xpec6ue-i5ul0A*m`J7Ib89e$7}3ziA;vW6(;(M% zXnJD>!um8VhTBMdGYZ$1nfOJ-UD}4C3SbG5tdisbfcjAf29}6A74^T0tguLIl~FMk z<=N#(nC3FM|6|~1J3?27=McIu>K5>)rmTJxN1XC8j~Lot06p&ZuaNr&zyVXGc$UcD z07;shjbIY=FCgc{3neFhcEo9tfp2@ZCE%xx*^>X!)4Y}e!m5=q7Jq&#d2a*a2LY;E zBt~ik%r?a-(^A$p$#Sk<=(P?* z#)Q;9>OL6j>Mq;I1A7Fri&$qp+TA&lo0Nj{q7g;6AREbyclD^fAfR6#PifB1s9*_Y zUB9DZHf^S&kIztQQVOhS-5a`%Vst^dlV*4t97>vNg_!)jtdiCj=;dS-GknBzJVm!} z5HqD4sY8^9(a9Rckt}*UF4#!onSASyv;0B+sXODJc00F}GBp$FPo&k@p&CzXj(#DQ=4JV;F{14c+LrNrA$%l)?-MT?iG z>S)p8lr#Saiib0Y#C z@#1KA{zbf12nR&p>{b41ytS?HcWS0TG0nIxXjgZ5SK?IqXF~xJgbS{W2wE2}N3h?+ z8j7W~-QPr>Tk(KUg4JMQwH+fEVvchN;wt+p8Vk&9k3tr$a#tw@SGA#t2{Odp9BB^~ zBHq0goq5aqg_oa36fcaW=p-n!3lFEnQ_A_6G7nPx@fd7-lYc-nb-Uz!9C|+-`i#(_ z|6>sk-4IqNK)i$Af$259vXh0CvvQ9!vDXI(n^1N$>vfkVg*GlauWfu!Bb^y6zQ9m3+;J5XaS z>i7H4W=Z`8Y67*sDLJT#Wly!k!ZsNc>oN~79O)YbP&^De((Fq_oQB$6OWKijQklOy z7T5$V7!|ku^^K&r>324wNHZEJ{-y}vi>mf*&~BK}ZZ}Ni-Owd)4|QJEe*di9qO6{a zd&GGwj9u{$NS{;txkGF6$KWwVJC?3}?@z-(zNODHIu^eN=$MX!i@?)53DuCC`(8{4 z&paYJ$S0=_O`QfY@+7-3_e|vVb@h9Q-p|_2kVwO}=AZ81 z9|sA#OHz=1=hu8C&Hi7+>_x`e$2lBWQ13eIBjvYRAn>=JG*2;SQOr$xtfl+pMJ+-4 zG_}8m`lFKmy2Lr$wDY3BI7~_ZBEGo4{{u&4o<58b%`HGZ%Ipmt$1@Rw&_K(X=p1NY zErL9}FaD8Ng}$t66|bV}2Ebvy+BjVqI)6g*v`|Q#hjD-~ocEtJ86lfHA$Jz>1)~b^Tyn=OzvB`<`w9IU9*3BN% zM3o$WId zY}ya$+BQ=9_2_6aqS-+}_|<~<;UL60uLJ=l+fuO4*;wAvZH#u6b zEez*w8r7uVLDaS)YVQp~?Q{9$pyq>Me71WZApCA;+}oHaIt1(LbrKAJ_YxP;`YBiv5Jc*Kc3u5DON% z2MaXn)KV1iC^t~6l6FFKZQ{q=2eaYho;}l*SvzgY***G-tH#zDYhha6h$n!t?t#^E zZAmAPx{lfq1Ip?))mjAt*tr?3iimo&>#wu$HNidURNpxuZ-Mm*sCnk|P8MbH2DB^p zJsxi*NlpCA!8MDCw9E(JfIiH0<#-v0ENcwE7+N(9<2X%!yk_4b5mWFRInh?E)oS^n zM6C=xO=#8bnocT)#+V8SgN-f!5TeHaZCj{5n0=4Eph2r}WuOv*4dm!3o7SVRe;D8U zV1B_q$iC6Y?rPI+1z9o_Sc`1@?`AwSDBZZjc4?2kjn%eGq0!oRL(-uM&gcr{+s?m| zsojnmVUtEQn^!-LwDV1?#$bJkq}hH%d{@H?EbkBoM*`veh)bjI$a=Z^NW%zLhhDi& zXFYLzG*UdMV=J1klSRkGqQ*0o^KiNG_`PNqF-7Oa=rR{TN=H^a~d2E!xjYI61b z{@R#CidVp;UAU zA3M0IfEUQabPY@I`|oYl_1fnQf9%?uVdvwj$D}{@Q2kAICwP6nZiPv8PI2o|&Z%xa z)j2(1e+Tv_=Y#n=s$4j8^7Xgy?-Tj@+xWMaTc1mNgR@6cS%98OJ8`=p&HT}I2(X8m z=xo)S77>fD%{&jk;5;x1?JXkB$J*@ye{`rRSz8|X9Yv%T_>E(!ed{muOF*RXj*Z8- z5h*wvrQjkfMiYus3NECNRHXp@OsC+;l!8S3Tt=U$Z=CWLeO`{AZ`0?MRB5gZM%*zP z4B~t0?LAcQ`~=kFcKA|q--yAZ4ILs+3x#z1NS;DsP2GXS`i-)BOlOc&oITR<>D9d< zKHc3KQhM~tD3+3>-zk49MX>l68rhi|QUj9qe7;oBTU2&JeI1hBeADGfrF)Vk% zp5g2HP#2y7{nm)s4(9Quy#7OAJ~qqwC~r3(pSGKix7y9e^V37-zczn*M}7vA1V~yY z9WCIBQ^o$Dn3kto`3SU@gX%V-E_a2@yNU8rE?p`yds^D(92Y05Q46~Q@u?+X0<&wC18^qPcAg(5hNnD35CUN1bxg@SN7L&N%wJ>o}M5I+QG-2yI2ThpVBv~V- zQAAYpwv|o1DcV5P5%J1iVh$G}qxjc0-mNfzp{V8`&Ou~%hW;69JK0L#<6X=9T8w!k zBe4=cpCo%V@Qg+FZXwJ+0fU`5SdM)W`|_?UQ`($OBUl1H%l)xa5=46ZVBUsT~1k?F=aiBDeDk$wqYCBAL6=`ZuJDL(dxCnQ1Ob(H~ zVSO~g8GS$0P|J@*A*op_mDRU(7X4qRFK!u#&YEqoE(7xCNQd_Ux|-~Y*A{0Vsbw9^n=cF~h=@}?!m$~?U$c>99={A*IQGwv@|=9u>v z*{W|rEC%j0t{d_SEf&f1q^F7ea;(hGCRE>JBdXlUs}TdD$Dq@2=Km6_h!J~nYd3B@ zjCkR3y@?XCgXRTa^2N9F=n%Kvh{vNlFEN#td|&6$P6Ch~R!-w=D~79F^W6O++x@0| zHWqjRtcP#+G?Q|Zcs{9~^qNE=e>Lnfw%xaU~J;h<-vMR z8(l5(J=}df53gss13ZRyS*X8ZjPHjx)?;DWc0&G$-yn=eu>MydKQ<^o9`aF*4Zjmq z&1&E+IKD5*eLt`!*TO`^R5D6IoyE&nq8aJ~i6HZF1T|+Ijcy^OPaVKF79v=lMnlFb zd(!W*^ShRc8owLxNZVa3p;z8~KSm`?f<79b&A8kq3egwt@&~&C= z=fxrk$I%2dUP181hr*X24p92kgh>EgJ3RmZEjTr-{F1JHKfy-95!L)UeC^C{LtS`P zD;~hAu}u_l)OP~c?^B;ambmD^NsRKYvf8ZJx*dO#DY#9mg7r1Z=Kck!om6Zs&hx#+ ziNuv1$w!q%x5tn;u6R~pOJio&^TSJ}QgB5Ko8&$BU_EU;_dP^^(co_~e?ljyo+2{L zp9$Fi4dzdb^ee73f3Cp7iRMoXy(-Q&e`3605%G{bwo#lurC=GbTpM~s0F-DYUty6q zhayq0!+f&KsuaGFCrL<{M}^1gYro&JWs5^A!6$_(&&D>^*|eiI=F*Ay%uuC~Ca2ef z4K^Mc?Tb2NA#RERvY>?%Hq}xDef>`P(gL#B}8$8nb&HPM>wUa@@X3 z8AwUaZjfou!=y50pgj*pF@p&1aVwhemXa34th)YUN>s>>yTAs)HOb8o$$tllIHGe8rX*C=lmW!M_yCdx_biL^|q8Zmb)r zG{bx>T_JN^17+_g8SQ)8c*gL)OZKyua!O)#wuGH}9k+$H>JIO6+yMXuI9*AQ@8_7R zBKrop{&50UU5*>X5fy!u92cWiZa3dz03+Ju;0}m>lONlEO$6ycsLR>bq9Zp9z_2H+lE zA{aKFtO_yhMxNXcDcKjxD+6FgD}`K>C%-{lka5z~F_|4`l$Z9_!Q3pK zXKXTlyN|mjI?2!uLirl{I)STdb}Lut$%~C-oJS@TZS@n01L}id#x>RZdMJx<2 zZZs%U3Lat|NTpypexM)=3!i!-Q4!DnvECA-XKg5?osen3FWbK4XQiS3A&|)4zER7n zC`3QDl@C{_A$#ln0j1cXr2X=jhbxEgff0DXa`^s9N14!b7EEQ(F>B>Tr-_h${H~b| zO~eH-L{?`(1jNZ-2{e3tWYF-PeQ_*(Sbk4ve*ql0pZ^j&6V8a!a0X5)gX76=I9z32 zHw2Ex^~AF5d&KnbA9ex+-;iN`I3(I98J zZw9IQ?D5~PoBk_&amT!2&x@~4BmY1*@cT@d_?NNby}#F+H14JK^}qlMOZhBO^n`dB zyYRTk)ydK;84?riNvIk^&davaG(I)}=;NOjz7*JiZPGdFM6kaW2Mm>TVLs%hV^!b0 z5paH-6lyGC;B4YluJd|Ukdo%^bNp-4P-v4r&-#e6j76O@7)kdDQSyX#p=$RwhPk)$k*2ZSlKn~WI&pS-b*nIktM z_*@_lb`p+0q`*pSriaypqBDyc*pOHU{fh+%1Z!Q3S$zz0a9#!nj2Uzey$ESmR-u^| z-?Wm|q=)?C`rylzVa~5^JPR<|p#{dkFqgH8=rcMuv6< z8p|ofb?1&S+X0EKAGbuc2llBq1S@e-pL)`sTYeX%(cGNJJ<+7Z$)2JyEq^1=>7LNX z{Z7>1J&^&8H|!K(5uf}WO`-@i0y8+?GcGmW+a0ti;S7jlV|n}V6H`Z=1(l!^_ZC^A zib{m6E!aS(@HR%nyhVz+OL_ZauOZ{&lXP;8TQ3d`BCY$Q}23nTI%3CNf*EJpZ&chJ1w~s#G*1anS`F4b13aCPGdpH!N`W*SSqG< z;?P>eBvne96z3i5y+0PZoqB@oa~i~#X|y{plrw}$D>!Ku#lrPw6v5sutyM6Xe7x8* z522a|oaQ%&cOK9<8Sjd>vU%_`TESUN*@Sm&Om>+p1 z#U1CFeN^fR^SUD~5466mqIHRk!jD!-GIv_X-0O_^sH_kuM~5?ue!bd6-#>URoH`q0 zd_Je1rW4AiazOxnV&KRirns zgEkW71Ak_#iNr`oqTlkD@ubtWZD@i{LW%nkWN25Qfq~YhdLLol+MBrABgoadQx;xO ziUgo@ZG*e!>y%|s)p>pXX%t?Z!(gqSUw~`<{UAjiVpNsc(kY_qo3A*{`iWBR*s%x_18@;9+#aAr;^4Q&^v zZ6QTIry@5eNeWH~o7XFNy%Wl&K$3VZORje~h`mYcJ=Y?|J#n*K z2usQ7t@BGh6&w(|vjPYm*(-q1EBTumzqD7FB~R~<*DMUX-VaR-Gw31&NBnka5F84* z=;PH2d0olizzyqd)@&LKPXBs?Uf|z_mP*vao3nk2%46~Hx zfCrqsJgA^^B0(ofWX9O~pzar=^Cd0P|M4059^F4T{QAEw96dn!hR4J6fu)k+ymA|I z0{Z3u+nILyVp;6G!hv2e&`Vc_wviOYvP_`9IHI})N-B*M7pF1P*Q5CebSnPo8DYv(kN07plqjP0st!?YgRM6Bl4p-t1Bk zZCIl4>_b-?BgNcvw-dciJH7Y@#J!Mc-}|z{M^qov?f>@sEpyE8^hc()FFT5c7~ANh0qg=KSvLW z8@H8i8h?WsTOqdg!Iyt%ZUy`KtX-bBl_&lHcU*LCjm-hI*9UaF@ ze>kPLnFBzHf226~Y>C=#XESPFx{U~DgrO#H2t=tSShb%R=QxSa4P6kQ3zn!;tJUP6 zTt(!kIJ?Z%pOERzT@gU$2(oy}Mr4})aHQkjrM*d2;yV;<$Ln-IhgL`0uVO0jYps%Q zv{fHkPqr+af0gRHwOa%R1KTyfuj=c1j+*@K{0mff^Fq3k#Qe0gtrVz5O)gi)R=|0+ z{Aeu2VD;>9mIwWC9l=nm2*)Nv&o>sWVNki~WbGHh@7FD=_J`_AiMR~)71)uL&($j4 zEr?ik-VDF5y+I^@&N{k0+3(-oPB?b>YKY%I2zS0&Di z#9hn*;!)2zQIuU)Z`c;jK-Wv`Fs7SWVWda(MY?zl%@Ul7cs@`();S9A06G1UKzh9M zGQ1Mx^uGtv6HwQsCfNTp+2401%@D!%Ag11evx~pso#6QFd*>s=M0Y`k0w=qj`&B$R zUfRHfv;R0l{vNyhuA3LNUsyWoS3mGiQa!qlb*s?_vDg%6=vF&3rM`65+oXhQql+I` zj#zFK858RqEXyQu8PYqO_P*3R<9k@cAv^6CXVGx(wgePrX}@?olNDy4t|ML91k;|x z&=dVa3>^AD>qkHSaHxLtL)gX3Z=3qjxj6{TEK9)E1Od}C^FQcEUxw}6%p)$P`_qZk z(MPN}`Av{~e11;2=@Xyb4DvBL$?kYx7(9Q2_66qeTx-Yq`|wj%i`+n))?~ zu7l~}gFSyq56y%AXL@kVHu*)YX-{$A8z#TVAYeXfPYb_}`)hs?Z`xDz;W3vQQ-2A) zk8|QXh=?a{rQToWsb9r$Bz}50?7Yt%7tWrdgMCfyuE6|17lwa4u#)G0oRuY01miu) zQ?8}!uiQ;A*TbwPd1Sz+csIr*55BjM(>nNdW-l8p#?ycYb?$}JEFZ5Fyca0n`WlyC zY%$&sJm_Obq{ReC<}q2en#ww7NjCQlmb=g_m*|`z%dOyYY5cR41d{0OufC zE|1GyNcW|0GYKl8W<5o=((jmy-#RBIl>X||nf@-{rH8TSv6Z5yvD!-H|8sEu$}ItV zAm;y2zji&)=|(YrV4Wc!|0nwVXeLoew4C%$VCYZlqVaN={4T{_7~MwN4zLx)k<0P=6&$Z7Y;`{;MaGfCZ| z8~WgQz9V^{<-xG@{elYgeQA68{;%wX-do1@LdP4z*$WN)MK|q*RxbO$wHF$4b9j59 zKWCWsLR}sVD{;ItmdR;Q=6_``G>Yu6UJ11qx*QYNa$+$yc10$Eqe;7<=U#;))_cZ& z^5pEWct0Ni@!x9fC;t_`uYHs8J*qJrzB}&>;5+5b|66>o%LtEe&wVDoUz-(n_rCf@ z#`l+h`%m~LLZ46a(2`Jm--C%aabmHtcrq|?I49;Ai)RETCNAMMai7i_NkxC-HF3Lc zT5kOgMf`L1p!j6K{~rj?|LrhO7yf^(k3~y)zP2@oo3A^z2j&I#4 zeAUhj%gEemJYQWh{_T9Bn~dMYOJo!0{8Ua9-@DNH`6Zf{CB|eez{EmMbQ_a36%*&Z z#FI6XnHmwnO_*_~%os1Z9j4tVebA)HnO*JRnH>_C+3qjH%;MFdDbGdB3_Gu{j)8eS z)%L%vzb{_n`PBamH=o+Jz4&h*B{N^5KJuL#60X_;5eR)If2P|su&pIvr@i%bVk^DIA?TN`^a|w zt0&UEt_{w)q{dB`e0lVG@FRI(*yhkSID88dgnQFTBo|B44ppm4*DA$5SF?)47zR|UEfIn#t_g;31E)3H-A1UDNby!q=-&tmays`l}_F>?<`d#-D99K?5Bzq z2NCI0y4LBnADi@@ejnl$p9qY5;R_J2Yg3H?anG`Rdoh)pkLBz^&$XpxZGP*?g2~S5r@j1W0pG{WFOToURM#w z+1oNmoMh*QCG5Vmm>+buI#5@YLbs?@c2bEGXDw;c#h)urno{{#!`P9f^51age|(E+ zHkxMYYSgyaQ6ydeAQsS17^JP2Y5Og#5p_&@7502D`Q5*agkNOBSbOYY($2||Zlwd2}6euEox}ljQzI(}(vEkjo!J5UjiW^(dT zESBtArFc*)BE!>waeQ1?%E&!3AP-(bd&#M@c=Csn_^{}2ZVP5HbOo%7l34Z|@g-!| ze%1cI^AjRO@7SSuzY25ssOn8k#E*ydYVbl9a>XdZvH6#x`OE(Jxp~KPlgk5Z2=#X9kjQ8Ul)HW19 z_-KQxLReIS+W94(34g+b-!hmahZEu1SJA^SK7o<3?!Uipob-NxGd*YNcwaD*JRjiw z_ti-K*FQsW4CO@w=YRf9t}>>)$Y<*Ap_jnMdLIAxn9!xn1Ag^7UeBMse=__%@aOH| zucug!zcNRjT{Z#jYVqddtrvjNKC>b)oTtJJXBJ!Ds&Y89y##);F`R7vJrKX2e}RW{ zAu@*o`QzGl_XRMnN2Y}3Qx(ZPuAf#&J~e(yI6fs_mE}HLLE|3~I{yE3Kk&?*G@f{V zT?YE50l0>u-SFLE3GDLZaQ;Y0?t=QW7y)GzPtrzhe;ypCG4#Fe3_?^y_SKJj-yzdy+ z7njECPt$l7OJyKkcsaq14d|)7Zsw7sSTEyfymAw#K4x59)V>}Z?{us+i$mDCn-&5x z=BEJ5y^bE;cF*r;Fx?9SOgAOu(M_dSx1ja{e9^f&m`NVNgZp2-zkM`4==`S?{?^Fzkhb~h1 zw++7^X1~di(pyjYBW4v!vDeS;TF5TEwN*|Ni~o2Reg*?oI?$a;|c z750^m{OqsKE54NQ=asD6&GX9I$ze$_Qi9>V;XmHb;942Ri1~zdPks}9`S+XH#F85b zh!^Eyq`43y%9`XdOPxNw$T0af{deF{L+q2eb**~PjXhinwU(1W ztwmvL+5Q&2Hdx=1!-=TA9th0God6i`uxh{EV#I$A$$yseoz=1azEl9upSa?9*yg-VQium=FSxy=hw925Ysla znNQTD9ys-}9_{oM#2>?349kfERA56@Zam;R)u|JDAmX*{t||Cx?Hv=Jq0Q=>VAC9Z zAw##(OLUei8$cxvO0-->uQ8#25FEJ8;>&W(ioc;3&kFt0<1CxEBVKq>Jp~?20~X zID4hdz~r*pr=fvq)WBS9pleVA-mAzQW}0sJ>;F&m_vo1r`cv7V5&0b;e$m070p*{o zP958WC1K!(idtdDpXp!^!tyu!6^aeejiG)3WLDK_2%KO2s6~?UA$%j|ufhGRl^jP7 zXl1*)WkU^j$j$_)^*q*k9ctOUFn_Fi1oA}6BbZ(Q=?-7?MbyIZf4mlBSrdh*njOi{ zl|NY68(kl-MXsgMk%KMXwCecUl3@J;K=$$u{L-*28Y)(5z{BwtYhg}LGc;|;{)FK~ zS>p!&Zjyhic~&L*p>O!(I9KX~ocz-r%tk-ZrWBbzA*Pwwado*G*S zPZo;%aYQV*(sbg1c)A~JIiVKNJvl4k{Uc>OrcV}xr$4EM;ZM98j3=_k1Rg9mOz@Mhzc;dlbX1;vX%$OniNXIl+CP`YI?c zzWhy2u!&qL52rkj*{}ab@P8xWUv0p@TF!G_TWz~}zSGLEhqedv{DygdXTX1@0e{00 zg1<2`6#j+er?_WfGr_+}=tDx_KLN$X=_7vy|8~s&`3S+^kKteckM{5n?g0O*mj4KU zmN|miku(BT4C4``@CfAlAqEKc#jT-08a*qTXcVhNa$pn>Llc)Hn8TrZ#?*WJ#$)Jg z@-1k~Z^bzE%A!N`5E>l`V2$(jcvtmZSPg`Rhw)7I^8MAV$fz(lw;{J9qe|K*YKAnU zX{>)Zn(>?RVal2a=y2n&gqPo&c>R9f9xpx4FkUV&@$#fT$l{oKZ#9AF$)>xil-bzU zh+k>!$q{nwHG#3OYoxJPlE35ZQjsz^;BkEogL64HG5pt{@ss5}2A^M_XyH{deHe?p zPgohcBY)PiId=u-s!wq46v9W=3g~+orq3D0=_)!-fMl<08ON;Fg{z_s?|PQO8WwlZ za^9yDtVG3I?^K?H4a{7`mtVEfF*?`-Ez{S-Rsj6*UwD1cIEUZ$ZhJg#IW5=s;NbOr zO`F7}ziWVBT#W729VT3!u}ECTT1{L&OfIi`KH7lZMW&UBKQ0fLydGp+?uTl^4^2pS zn$+%QOkB+EuH<%|Gw}Y3+q=Ahx&Paj8{50{CaCoy)_MbK*}c=^{q=1Dy?MtiNN;HM zZ`@-_dYdzdZrbo>>(8s3bpJZKm`X{ z;^|>->x>R^QY;QI@(e2HB6;7|)-E1#(EWj$-^Qy5*P{pIg6J<>2U(N?3N17he)m}I zy|p?XZBTa+3VW=^8xn`FZ?*L8MX|Lj>B`=9PS9!2IG0YnwQmnRyd~f$j`92E9Q@D$44zkoRshMwRn35XPjrJ`#T%q$!GkpEyM+Ue`wm3t)WyDVnJ~_@iMs_Wk0| z0VWUW2~8Yn1GHd$@{A?h-HwSrW23&fC7I!Pgjhe2gqh0%_hM{XI`shRw&tHk+|3{6 z*#~%YpB`jEm!v&jGZ4n%nR>jY`l6~ymlx%OqU&wjgEh#hRnB zrnaYcnZZvk#?-+zG%oSCu;c2++9ZYtB8;Zam{Lb@BlkPeSCoqJQ|B$lI&VGTNW-K>R2GzuO`vW_TbG*6Bwn7o!UZY=W8e~B{ zc{7pBJxxS)akpiJ9l_)MX#_hhM%;H_${pBEUxu`w0jVPZ>H8-L($7vvsb*jhr2QDu zUf9IN`-AtyTX;__Gxx+>hVbsTR&G=~js@?D!{&4D>jZrk>U^)F_m&Zyj;p~7$zIng zV^941m$WCY!adPGphdcr%QS*tn(Ob8767i`U&PN>(pZL1|ET@G_rod4&ufGE`3;T6 zzV}3%!KdO{$A#NZ#3lXs-R6TK;T&n`A^BW{6O?_jNuf5IjA1Yy9_h7>cV2jbuVm?8fFTx>G%EIJ{wRYvu- z{1qJIvBk+h@%gWg&ri>^mwP_?gXQd(OxisE6w>D7K$|)I9{jT!mh(hh(ySYkfiN<= zB4_uicPPyV`t0yO5b~af@_z%a-@SyN;s(i|$n6nQnlG+1!P@hjI;8d7RcNvPRH8xd zOO%4wxS6@wOfEG8#3V_;bB3Z1r(|JfBKMJG^zl4%()Vkb|J9QE19l<$C$<PTz(m|-EZUprk_8USuc8+bKh%s zfcfMqPJfQ(6w^~)KQ-9mpo2@+5^-QR-X)&*Bi$uF2|09^*!f44k8n8NtR>n3@*L#H zyYsw~z@g{fOXQ&k{Y|^4Ce}j>%b*3}-A$AB_$fJQ<7nwepYqczobl`IndyiTtVqo4 z0|#*J8^8{pNjc`;NQ-k2G;$>Z>x(Ysy5$|~UV(Kl{)QBJsm7ltsm4=WB|-IZ!{x9YCz$iX797q2W{DW05t8ub})wZ-Ic4Ea!J2h>0L5<-1(Z$|tCht@LF zl7=VP&325JaZD^-(4vE2Ht|l<2XP^>H8>xTz2IG7*ZWWF+6J8OOI|(Q(LlL~UGx+a zP`=?K-s)tTiHr4Dn3kFF$vvrBx!7r=<4z)~V5mX%i6QmReOuO_7E&L5AH68+Ya66E zE&07W7wMnvJU!JEs3{|nJ%CvVzn(HeSe2fY;3>0dRZlz){bAY!t-KvJp2HdNuEWgq zsf`$mb_gusJMQxDZwt~ReDp_C!$#^uVQ^`2%LODU&u4!mu+d$770Q_Qg!cMW*HL|R zqqp{;~v^&JxupeyR&@q1v?iT4< zH56G$MLO{F@Z||-!ogMkO?m2{3-<}%ey}|$zC`a#*R}w;$HH`o$?Q4h$)=eDX-?niDTCGUEb2wzPSSwSp69!8-$bev&%F$dO+AU?V!=f%9Gy>- z$A|9o=*0u~0x_5N0_Vf{PxM&R&fY;rmz z{xdXT8*jTk1HHZ+p;_a1aIH$bbei5r6J=DXn7xg(RNI@&GX1XIf%JzeXGTCcqQ4gs z!r6~^gxA&q-om!9@TO`00K5jxAhe<-1ll|HgooDHht3KQ@1b-5f8pJ8=D&iMG)KkA zP+&jW9S+!F`7`d$?P1})>c51yBRvcV4{t|$V2HKm5QtlLg-5Rm?+4Xk;eE2#U&3qn zCp=|9D?F_Rh^P;Iggp}>Fm=Ry58!Hj7yt==@HFW0a_aw7`~D+(k+p7hyY&N{3fPco z^?V_{I{XTI$G+D(S+f0)|E)#;cja3fjq)E-dB+tk?d1nq@(^En7eF!e4$Ev1$O5;e znHCWVgt3|MA@{5v)--wvMFO?X;gL3qB(`1kNXZkFFd<)0bySMcA%@GO5N zJUnBrYX|>mF7Nt_@`qE6^1lXYp0bTMH;$>Ek+q=A8vS(XJ|jZT$iFE8UdJY37o+Qz_S5BKYQ?M6pJ5dV(07| z#3K(hksd2*Js%e%;TIcpZ880Y2Gow0$@3XZ$3!?n6@uWugarT3*P~FKyM&Zz8*n*d z)K$RP4H#`*4DZ>Z*LKH!4I~ipO$%w2uKI}egcm?vM{9I#5|Kok{okWaLCYh8k2(BVJn&Sy!mbu z-`ThYb~foMA`&Kl%}Jc0P)3HSF&NbDo1yKZkT2I}8#bLMTlk)?2ZwvV4I2FC*Kppa zZH7^ZP5%)SmPu{zlL;_K#ciSHAE~Ci1b8Ml=e+SicPB{({2m>9(w##|D%nUB4{gJ1 z9nT2*<8vU`6X%>H{qQQ!6vK&l6$KKYPRKge;ai?b;~sVa0<)`^T;=4ab16GhE>30G zK~gP}KCVB$uSu^R!gNQvWwU3p4*QQVDPAf@_Sp||DtbW1QeWK1MEosV2E@wj_2iqX zcm|=bWQwf(^u-LzGIo!iC?0v}r0H&YT?Ah1-+G0fp8TV-D29sA&@z1YcEUxgETYiG zcpOM?Y=HliPb!I%)WA>t0jxl^oaMDHH$<0S6CN;tQj?VHHR6LD#>4fuiTrAsBw!cO&3`=?Fs6Nr%^!=y(73mFa-jb49I8Kzz@U^foN@_< z>PZ{9+(GLO-N2foIPD~dn7|JS9exTXKSERPmH(6Xwf#zsH{>v$OIVbn8l+mdC-Dx`^ zbf!>ha{lNk_X1WB7s|4Brb4a(|* zo=9MYQ3})WUe_-`r>R6xYU4=?%r z&QMtG?1!(MHZ#O?nzH&rMOodWTblJJYc=)ix1tVm@^|wOt5)F3O^$)p>B;5u*ZD7J zFZpm-z$gSF9T4chfRTVOFd6#eXmoK`d=rC@4vib}Pf9HOT1x>wcm7b@-v4#F);J+U z`$qdN1GWHn-3;KaXi1~*2A*Nl&!y|%D)IEqK!9+F|CZ(`c1xt0@;FsIGl5DA&H1cj|T)xSu9x^w7(T(y7swNkbW5ZV_A#eiGSy9 zX(ipC$RFrYe}(Ax_ZG53LBuytLDBR$il+OgG|_aWd!LAhvuTkxV+CHyr~7geW?F!D znli(<_JSCS0JqM)=;eCyWQCUuGhZAKTZ$Ucd>i$EaFtM?aJyh|L=_fJ)L0{ zVdv-D80n)ajJ-qyzDL@$?=xVB%dbQBp5KC@f}Lu9OA!eJA6YT&1^E6F$<3Jr9er1h z9Zf)x=LdE4_n0u~MN1QpDYel-!l#lUTBQ+k;*J4`a^MN%C32k<)U>f2!7tc(&V@)7-3%Ci?4y|8Ro zX}CYHyewcpO-?V_8?+=h6Y@7@F?K_om7t*$?9o9rH@M^zMWE<>8b<` zUMZkRClBl!fckf@4z>Wrq~Tjmy=vW%Np8e^^AcuSbytG!iWgJlEtkKr&AdP6&h0v2J<*z%zkO8JiEsvvof&9Y(E+`a+=r!b|9;`3* z4{~>4E1k<4iXVI(!z<4@%m%1GOA3QEcd?xfMy5!2R}Qeb^(cyq`RLc=aj6|a;imb! z=n~C4^E)xDu{~%A?ZybK5ppf~{l^aM&KG^-E2zisL&(SZdgKA$%g_Yt!W2`iDy>S- zOf>Z+yt(lrGA8iafYz&Oer`NJrK3->Gz!|naX@^<2~5DG`}VuoD?{9%4T1pPG|LZmghHHshe?n+j_w1Y@FPG zwT1e>y?#An?^$c8T!WBpi-Z&O*Syd5qEebQRO-M|?_;S~+6?{nQ2xO7RJ(bB&>?!i zD4oVMTt)|l*~0Z}%IS?Xgrum7CXWZX|LnZ#Ta z?-<~Wo;QK7&(gJWSPuescdcR~hR(RdIR!1yVCfaY(j#O0!IRjo^4&|xxHk{mn-R@> zTBy~J`F_#*6=LSY_PfOk5nCNaI z7Mery7^Aa$|L4=tK)K2hxvq?kViRZ~EREOA`jWWzi(Z#7W%2qf*+gx(kUr2AoO6t% zU&1r2y;2$?#c_j!3>QBx5@qndiZ)uv9ICGPbIw&Gshs#OB@nj2TY{O9Tv9urPHhkl zdFFp2fGV7jel%Y;rFf(N-cbDvJj9jaL$a#a?eTsqizDT0?J!RbuVi z^*+fFiv~ z_m_k;-|oKX%mlpZ)OOlnXU?yi!50rfbPoFk_9|{mL!{YI<84Ajz?THtcc!5%s%w?v z&|4HLrdJpVjn*lJ4m3<>gUIguL?XLQAU9M8NK5WcC@^5~IB}|v6d3TA6QqRU_pOoF zF?dzeuU`{eVVXpq*A^lcP$Qr8;mN&&$jl=taY1NF0?H#xvN8+ZZ*>F=FL1;p1F$)C zJZhsE8NRbL2)M$&+}I&LyJza*0Hnsc$!?jx4LNE~V5wlY1mG{dWw&|M?V`3I2?o0C zDZ6~A&1@k}YY^*GI2pre+1#!DSZz!SQsMBB+gsiPbA`c5!DcJjI=q863I(6>kN4=u zYy1NlG3iXf8siP_)i01d@p@tH!XWbeqZP<=>lst8a2^#ySCfXn50-yhn>wCAkn|4P zP8wqa@;iJzLz;WWlnlcS=%b6^G{2cd$TMuT5IV0!U%1)1FA5?!rJz4H`@rS|nA`iG zNAVOSgRK;M`2)$jqRyDpDjK3V?L=!Rufv8fzligGXyp}jxeZrP8*3J2%f`NFWtO&5 zHuRn>d?pnNdo$uS5INb43Ns4mgu+T{^)s!Rz)@|tgN!LA@2h12b0$<@LvcjVKxqv{ zVn#WN96k6O+JgL(IFNk>v4|g0+#E{c<|%olyp@j|dnEI|KbzKG9Nx98oWLQ%N7zdD z@#HTpi3#18TUwer?#t5y?;|X!2bp(w=Jr>@(wP#J#gj@wYkp&=2ww7kHbHtUp<8%) zwBo1_c@ZhXumRcHzSJtdziaV zqC#jfvNkX1gvmqy2#R{?)BiqF?gJZuM58O89c&q`-)f2H*iG2Gvj|p2Uci-vIm{0x z1*Y%Vc%Hs4wo(kl=)bl*9wb5? zCMO8a|Hs{%fJap<4WKhwhb5kffI->RprELU7)%Vo49vg;5=0<~U=$1>Dj>>?pcocs zMsw~k5EtB0+_|ENUciU~fj|-zA>xLjYzmxV6p$q=$-G+noH;WY@ZSIa|M%Ya?)P2E zobIl!uCA`CuI{e7P)+1?1y{BYGne+}4lcqDp3rqL13UN;xyD2$V^BAe-2YWRG@1ZBi}@@XY%BV9J!X>?Dw?E`K2~FZj&YlJDtNmBy!aim5|`g zdr>)_kJ<)r3+E@MZsQ;vbE4{LIkcfWu0{A#cNuSjtjh_@_q)~O%_Y2hY`=)ot?pAs zr(y7V$%QWOJa-L#Ct&@>_orxW<(Ke9+JG`BnnwDf85H}?ZGh;7)u@*8mn)rXgVUF{ zOzi*hVmrkj_U13oE36RbtmZhz^q&UGYGUF`QkNnCk9B#TJjrC1tx#f3v^n>musN$t zIkK?eVgH#=v|Oh6Yxo%8H~1H?^kRgb>InWvc^~@xra?1L{)_(Iy8ZppmB<(OhVg|) zucN2x0{(@%`s=7Zx(Y}Ph1XvP^~JO07XHBIk#v7$JIp1=JL;K(siFTT?ek=IcsqD;MRq-k1()AVes;51b!jx%DK zwqgbd$>mn3>2aQ>s^B!e9K+M}3{BJH$zq;no}qcF!pHarW=7Z~B4fV+xEaENl z0lvK}z4W4|A|~aB)wO|)a_Kp|Z(tf%AgwJsrQ(C(1M0cSwfVNfT{gK^oNZ&u$ZzF6 z1fatvKL*bWg%I*S8)v)wf&jW9q^Hn`UO*lR=_yoE@`>!?qJ7T&G1;!b>`^eL9xx_b zVWn9H;n{NA*3t$}1I-_GAis@k1 z6RDeFd7R|NSNzVvosLT9VadLkvtM}AF8%=2jyTg-zWx^ih;sC&!w-MI60pGc+q@n2 zu823t^{HhW+8FrYB&NQe?l?ZUj+48P7+FlMTieH~0aUt+?z0U7&eCu=U(_+24nn09 zoWTSq4wxT@5f>arwt>F1R=-SE@yiW|J5j*xB=SQN+~c1h+-S*tC!kkK_V(?<4@mby z2&BPfR5xfs9181Xp3p;K&`{Xe27fv+;1eMN4vaY%KYUa zaA1Ep_nTz*R6a#nCQeu2&(}l>F_H6i4F6ds&QySPc3o5kq`TH64D*4I)q023**Re}K_|uB*+`B7B5kX}unogt?|tlHBv+0jssv zRjx9r__tUR_HCEP!>4MC+<1BC3v0?J$UjN09Z=*vU?SGD$HBTXequBmzcb@{aAg|_ z#>WU_UmU@xdaosnFXPSy1E_JNxcBZzYxZ!JeKSTFy1`p zSYqt6q5W!Xe-*bkL}=gcob6-n>Ck=+w!b}w+Lyh{?Jw_lA<%n*`pcW^z-FM18u-8;#+K;2v8(dkNS33gZYM)8hUT@K9tQ$Hl9hZ*C;c2g}B}MbG(`WIgLYo=m)t98YB`^G||N%>}58 zAxiH}vB>-zSkE0-PTpcPv|c)k*CVe{uu=Cqk4VnLG48VLV#-yQT})Rsq(!e#Gdf_j z$h42oL+A55oSU`D4EsILQl$DoX>HrRwRk>w_5Pf(b9hSb)lJOy;IB!{=d4fui*rJhqbPEXz)Zf68#MgX?@_>X%yL%dINw6- zrosv(*LtjXXO=5!6_b zHd}Q#Q@)#VnoFS582Eo2@si})*r?q!00w;b4P?M7GSX7Zl(J*>A}{ufjp}*2MXq-q zh;bfJ0o>T22QVoFfJI({45rLh;aEKM!5664=oJp{L}I3ahCoeh%37b=_8I^SfljO74s^=*tr8h$S{2JD@n( zC9OTBbmHUyGFI`mJ)`FFrrer<&&RI?@!8?WO*5tBB!Xi=eB!@Ph z#*wa{%7Tc}rprexn18!3OQP4Xf*j~oX?ropxCLdfOtnEWN$w}=afI2LFp{jhebY61 zdDFVRrrYLx_l}VFLlKLU#5H6yJ87DuX#a5yNBxR^O}GDeMQzv)8+ND@@If&3L&Zt* z>sXy$e$n@!{8Ck>uL^3FNNblO#6H%#W5vI!P>lYIzSu98a*b8&RHd5iYhOAoI zQusVIf2fw@slV^F=9clJ{zv|jhyOi)$wTBX$x44olUJJ^AAcZj-#1&g(Y0phWYRBqgZ+k^_R&$N$D@^34a#h4#;-ZiaLCO-jgLf_KQx-a+gx zp?F2!*l%BCcS%Z!y97`8$tAHx=Pvm&HWn3(&SDk4?n7A_cvlsH*M@mfUXI{JA^FFClqH%U__)l($>_6Gxi+3&E|9{{=Ib0Iaf3p62AYAtUmH%XNNr?Ys z{;&V3|D;Dr#Qp!T|5N|T_lqOJQ-1lM`cK>u;k_6R@89;Hq(y{h3y1e_`%exniUhB2 z&;P3bq}SsASN$i(ML~Z>*F6#ZCsWVmKgrte4c}Qcy}>rEyPBpd*||}4C=v>>B;l?Qyz}sPq{1T zPs!T3FnoIySfuUxA0uc+pI!om>q{tAIyh?ixk>Se*Wo2r*3HN?x(^SoY`KaF}X%g$Xv zXM{iKj8MHSWk&Y0*kNO~+S9_zqS7T^j6qe)f8%T!^19fDwMT!-LO{Xy;7N^6s=wtu z;cuC`OZB&O!qHjv{+2H1^0&krHGj(=-vozf3iG$@H=M)YqQ6fk?)l=Y#WlPhykltV zw+lYIca_ql=WM1>WcEAo?#`&xJ^J`TX}`Y`uO9szSGGS*zBBt(V!Y};6g!D@e(x*? zo2JoG3Ve;-_JGa|wk(4o%3!}sdh#~3ir`l?$cQ-ccZ|XDKI~V^yd3L-6)Hq zNk67*`jM`%Ld7n$F37>)x0WcGBS7Chz(tP(jPS-$^jETQ4g|rZ-WjKY0Yi^>bR7l& z08NhU2IS{snkQqYZO#MD&P~yHJJ%cQJsf9{y_hM~CRsP*8-Fx4vG#a_?8ci#%IH~i z|7y_yie~V$1&kED&AR~a_JAGsQzb2WBBi{99xKQ{FF0T@BpRrSQQQHx;9Wy{YiUIu zrc0S@G8ov_n_Pmy5m@F@9s|Z5McBaEn3Qx2?;K$IpGRYfx;1B7k`sX+jC5yAa%BLq zJWKiF>Y7Aho&BurT1s}k>`&U?bZkwr*UC^KFTV8lTW(Krjq z&G$bjp8(pm@GaZGAzD34uoCW(Z-zNou-;BY>kg&%C^7EaDBMNb4w!R%s(oD9zo?e4 z`8FY$a|M|_OZzctdTONkbP<#0_*o5`f;E0@?-*ZG%8N{j4B5OaNA5+WY1KkP=E*xfGdi!*Ia)Gxa>+?RSwT zHfy}k9mgZhr>))4LFS|4W=)*e+C`65b`~x7d2FB>KKo5pNK1qRs_} zTSkRIT=ztL=wyZJsMnr|sH1G1w%I^S*-%a4PuYv|DpYfQyAL&2_n_uVX`#7Zoxz&x z&VNU9rAO3Uba-x_Fi(G4Ao!GheGw&tKs_VpsK&_!yrj4v&5qa;0|#o z_h-ek1aIPk==n#JPLIW?ddwWE$1wla@6Uwu*We|p;cX3SvG32T#onHyX|d1dglMt% zpKYndURJeOK~Rg02+?8_!?f6Isur6=S}e=tWG%)6SC%}-Vfo$9okhZrQkluKv?r4( zC5%iHpW_S5?n2UOqe!Q9{~zeI%)_kHExerORTX} z;fHe^Fd>NgKc5w-FK-O$HA7IZP1DPB)^Et7q}oWy(fy{^kxs+4MYm5mxRA73jtt6Z z6tQ3#mK0V8``)Li?0s zaydRR%5#^guUTN<4gYmM789K#H`CjdotR^L(khG?gmQ1{W%#%QqX+#AKU4Q8E22=L zIBusfKEulSaZU}?qUy}MBF#LYra6EkBWah!knUZPWwieAT}bN!g>ApbsAz zvJ{7@7uCBB>kZUla&RNPSiDKAXQX!6;mxmo}IM~z>57K3Fv|LAV>%u7@* zKUyaOqL@nu6R9sS@|7Cf4|VwbL2^|J(R34h$XbJ$LtO2Re6;ZUHK(*BH`StA;tv$! z17G_r>4rflihhwREb>XEG6$K&J`gH>j{Kk;{=P2YDf$IEDaB`xl4Z9tatXTkwXmP< z>qMkWaeqPN48{D{SIoh2W2jR#o;bR`W$Sij;|2h0-5%0J%g_9HGH|rx7v<;`(T*Ij$BNAFuUB6*ZW z&yi9}-L{S`mI@`ewjmHYg!B1R0+ej~2cDz;%5e`ePW~_W-cPQ?;Kp5SY7OB?#En;A@2R#fE~A) z`6hrRPdAi#EC)dl<|oJJde-_QZ-&!;*Z2Q{D^{}Pr0h7JvhSN+>;#2*vL zt@~7m%oAVG`pOK1$HS=O*uaT4&jmJ5JD7_);A{}OdGP=Ix{_g7@nUPjK`I+p^S|W)iC}T zh-POqZWYoo#?KlT9!SgUOXx=#DtOm-F;Hba7DU;vhq)Rt3+L)8z+|0U$Dlu~ixl(0 zOF=ShGMQVs-n8<^Fy`Oz*`QUcsH-IuMB}H@S}i31t1)zAr-X|4**w{a7z+e`DShTN zh)2w;@aJ>%(_Kmr!08jH`0OT4ZKrU-egfovk$+Y5FB|_V<6mRw7@iw=yx9z?SU}44 z^fn2PXHTVswRTiq$$dPR78N>3E0cr6>FW?~DuaEqP&$GKZ%Y zNE@w@R&q5NF-QUbG&&M|UDS6HVBF<+W+moemTGSYza!o-(mXm(_T_lmCYwvIH_-9N zQ=94oPzw`T;7B);wmazdSg4SGeKTj)tHclqN?+fs?vHyr_&Pn7n z`kqtIYdxcqO*|%INab4-KWWA24xvQ(!5)z)50^zC%GWL=%3XIq`nnJ8LC{}-F$@5y zqs*U3JT+WT7Z{Tc@3{|Zc!Rk=oR-L?v=0T$(rFinCnUI>$mEa#&up;X%qA3n!`IeE ziuK6)&7o6*3QKr^V0fBP15 zSe`J3PTeDj-~;ufl8NVa`8_nRSkp7cy!|Z`XistK(Y~y{Sr;G1L--iCl@5{-TJKB&I4ztk0`J8e>ZNZ zF5nxaJ7mnHL_7r4Sv9pmfn*)=7{1134KLj$N=rt0GJ7h_d1_T(ORg)c68wynEJMFiX*5#k89nlaReftx?Sr z$<-Gri&B8H05X$}QN>qMJ!V6l!a*nPxXMME2CpNWtf&Jv_aRu_1cvSE)tcwz|=#zr` zdF9f6yh$86n-d-O$5!HYbA^J@wP1F?g7LjD4abL(JJvaZGN3Ov5OfzaRNWNpJ4>QH-{B?VAUv=nCTyZHFpTb=O3&Jxm2}6=AfBpmIqQeyW z_1=ZJ#>huGM())&Bgtj)k<4}5h zL@|_=0)QC|Oj&RW2eN@$xx>bdt&XpREnXGhXGr88?AP)r->PD|uDC(XTXzc}=sOb<9}?rn|4#W)f0tb7)9w+FzzVOM zF1g2|r=;!cfXQYK^tI%ELbz4sy74;X^cCsjcpeMo}9_h>*nCskJ51kF|i*W=&wYtFOGuM?(5g1>2j}EhOL0qnUDZm@ubWGC{MzB=2>iNa3U*u_ zfnO_D==gQ|lt}m~Z?0&`uRDA#`SseDr=@U>td%IeZK4tLK3Ic^F?S`BvSALO%HW#1nNPze^@z{4j6An072beYbj> zTjrnlc1TytKPrt>o|UAQFa9P{d0O3tqI~1l2;~~bQ|_tLl)&2gM+*6H_f}HG?RKQ4 zQ}q5w4kqrXQW`1BmHk)o;S1$z{;(ZCC|$mT6`218Fo3fmz;|?<4b6^^aN!)-SVwV_ z%<@*V^Uo;dK&w+|X_$L$waC-HP)fSePM)grren1_ov{OM=*vAQII}cKh)}G@rVSN5>)Y+Yl zU^8IY7Ukf66$O+1YUp$K#>MFJePu50Mv6=r7>{Eg{-oTFpjAvfMZ1-nJDSM?*Z?rd z18ct*I;h4hXi{&wSF+l`{oD$Wt2d(mRx!`zaC-(0MGk=ZSfPGDK_@a}{@Kq2hy~@8 z4!Bw=hTi1eJ|Ed{d;Gsp@T)6KrvkF|3*RU_4^6G{wGU|dHiP!mkM#77;^GyRq6d1K zBM`_UZ?gLYz2_KcBC9zTU3uxShdfKum2DDv)vpCyQUQzMU`Xbd*$NYtOaIZVS!6r$ z@W3TTK03P2ViZChDhiu%9ZegDUKx^*&7Lx>ZB% z4Ao3mJ_&-t4Lcp|))*+%n?XtI9bofcilP%>nT?iEaP6shM=qg1xkp>}iotM99nKDP*#qGB7DI^7}_KgZFbwKVc>E{5|o8 zbu&WM$5ZdrakRwACFatOrmw+V%>u0EL9Db(UmY;5CsjGuB1eXQ`^+^n9PJvRaIeLDFXaQR&Ku(o`ziPkM2)QQB_bPf|-KF8XGFP~qY z3@)GLG|w|k#EclvFvmF;S5Glbd4HwsA;IyEM{_)baNRg&L|!vL>}jB9mfo~mCg5TJ z5v1y-d&y90Pf=tod?vPZI(~E90X0XoVRpP^V>9P_-9zk{6v~d))PR3E(K+p_@fzsK z1llaUUV%v;0{&I{`REjN2xG<#PxLtBWV_xB*-bQ6{xP2na# zab)jfcWb!KfL3~(h6|iN;+hW%;5h;~hjE_?Ql*B^?y&HWM!p|4huf3{p|=ZzesU^9 zzm02NC!qT$G4z`mdJ;ijFQ6|H(7SM(8@LU>6;eKimcD3poqE$&-Ng`JVu*`p;$@HD zns`P;S&=9^g=LjoHd&NS5M>yw)whDn=7=(@D5Fd4i`n6L+1<4A4i(VK`weQYLocY( z=>b1By<0SGEt=wmSl_6SX!f{1F&lSS#*4JZyYNOMZjWrrr6+j$expScms(@<%!7N#H36ke8NS!Gf=R!PkCf?FhP6bwxDDqN zc!5^CwItU&r9H+!g(YZ~)?N%4C5(&5jG(Z*JjW5AxexM1|KpCm#Q%tcH?7-vwbmrA zm8#@&$AA#^3(Gx)7JA_gHqT=wjG3yu)S5D;Ws&?POVJiVe?TwY->h$$;a<}(ju!!g zNn-^DfOkh_-!Z~sMXr3p<}5SjWNgJzIlhI8NA9F(*r{d8I;g~(LANR|wJ^46-U6D5 z9FUsmzUf~`B4BKe=f-3l=Avte;RZV7pmPreEiwxF}QB^kGLyYt~?qK~tsc~CzxnOkTd`vbR;#!?z@m!GW zdBkLOZ<7{nAz5afZ{EnRuv3_LY5jGmV0+ct_-2@2np`r_8*>wSFwGg8=8ga&#`T+L zpVO~(Ht4c`-K5txWxICRHmb1Q6~HR@%*_r?>M9nXu}EvDngV9;km~vMt)!=R(4+1dM(;5Gy@6|vi86bK z@)yiY1s|f-->*^&Ta43}LcbUC?^yNshI2kk1fXHb;)=|3;vEBy2pMt25hqVO=I&4f451N z&72*mL$~POhT&%Kfa)Qh_6IHZRV%Rgv!x9c1HG$X!QncUln2TtT8!1+m>GaTcMzgp zOH%c4k99lpTAc3I~ zh4EkR2Nfi3Xk+sjYAiM**i~OjE;DYBqzyBTHI^Cp=`YE3T@*fkJ{v79O6I~jQ8kv? zSlBGNF2qbJ(uRT218OYUMi6C{d@c_SVx>i=j8r-_W&rh&Cs#?Xef(`?EPNY|-@cJt zUvZ0gY0)Mw9Tx|sW3cpR$@L+Bn;AcVkjj%!Nv@Z;MJs91VlJIy8bDY9zwMMEa8YnfRMA9k3%xjEGX zyXG8E=$1ew#Q0)?58hl#`gsqMxg|-8KFrmG(jth?1m=}vtjl#?MKt1 z5L`zA?p*K)1hH zF%`bA1B@|KHhoX$@1x-RYW$ugzIW&E{dhvNyTcxe*-vs|&RJkV$u&Bf@f@jf;XXyV zy*8lpC*vn;J|Aj_CoRMI+AtFL>v@K-3lO`Wi*PPhf-^Ed-5?dmh?Ef5bZZXicwqR5 zc!T|Fz^KiHcuF@RZ_1IY*{DwZov62X2yRdc#XZ~F-56K#>Ntb{ORR%$sL_h)R$c)?s3g<_*Z2`}=v`$dP&VuLm4@xgt^KJ?Q4)e*}m$J{oN?rAPQ z>z{_VSwglu_x?$9@7~e~#V^y1mbB5g@^tLUM>OZX>3iaAc(WAIY%=bo3(xZW-%6BA z7@=yZVu0pD1`$Q8iIz8lcr9G+h?wT3fJ;PV6J)>%O;HCv)2mxKR`kz8-_Um)!lMA2V~k_&xc z@K<-q^(X(8ExDXA^w)gJwGHdlZZRaHTW%q5$*XM6*}!YCcj|E6FdFeS!&P3X-^Y`< zO;lBR1(tTY@@gmMmk)w)1kWALBRFj|I3Ei*3tGa-Rz?vV`xHbBw;Ha0#WI!{fXUTY z{D{FH##r@ouD>7p|JM%F6!AV~;Ruq+-QoKO!*mMRK7zd@>otEV$=$DSCApj4LhhRB z1PvylweL~=dmi+BfbRaj6Y1`#JWY4!4AbfEioLXNf0?GpUBCh%dzemx|9%^<7Q46` zF))ABCoRvDx8}*;>GXQOrq(O+SgltF)w(5_^msyUV&94j>(+ZhAG0Bz;W#0N+ z4?y-zTJ9@q3i_Lg2fS%(a@n|`3&qw_WfGHrk(<15)>0}*+h9wd@mf%9d%~$Jz*$OYG9tqguTT`&eCmDz?G0p5hA>+ zz-KRe=Pnl=zb7WEvWX|_s@od0v%MFi!$tT@9gJjreEddT!1t_pM=ov1M|X334UxBg z;jU&1m{$zJ>s&qz?Z*o2nf0u|MldeV578;GcW)Jx0O-EdrqFl*%|msz%i~;gQCo&` zdZ^BR886CCkg$ylGhy%?giM&}Y|k2p>P(m((9~B-A~K=t(1`ZQ4GYOWNe{J81RY)< zs?y<=JxqrO&N3Z_hST96UoahpupD2Mqoc!X!i zP}FllQBOB1KMC|O&QA_(TxdQ!@hlvoZ$$>hK1N%GFOZ@R*dGzPvYFx&Ttsqz@npp7 zqhLHJvHqinAWxX6{sCHJ!);30+pyp2Ndc!Er9$=6_z$t(y%4#%uz`v4)*q<5S#rKS znAc%9G2MTG5$e0SL8red3)2h%=T_9;m(tbX4&w)t0LUAn>hEiQV*ULr*a5yXb=+Cp zaST!1%=){Ub#jY5^5~@Hq*1EArg*2$hA@3yQ%>sIZm%f^P5qStgKC1JuBSgte4dJ7 zHeD%EoPWjj6#W58`uS)PN%ai=Q3-W^kK!1eu4iNM4*>W{EJjKNfZxQ3aQ=QvNIaQ$ z2Mc|OcI4wjHGTN%V4)8wKhIDmCB>=%`9yxFzm8$w)4pn_GSV{H0Gl;E#J(I{AqehJ4EM^AcW5!z@}fk0Z)mIo!h| z*h762-JeR^U={iUYxLt9i+*AKvDzy12c4IQva+T~`s0~sp+D|m{jtfSxpiRvZRm*c z6*mRrTG+;UnJ?5f*^We1utAd~wn5WXosKrB{9O#oslnCXG7)nqgG+hP<}41x#&(F%+mvCFf%nY?i6VBJIjM%CRFX2VHXK|kZ~9>fEnvY z8v~6OxZiFI+A~)~QO7!aW?NgfXAZmw_RPJF%=em}y+fyGH(sFWS;ef=vqR8`8y=!( z5shWvvYtJRI%E#(Sxv7|{7!y@M0YBEcQ(@dA=1NXmw+CApYobPwb>j3!IjSXv)4eO zi%I?s682zZT|8FR#p9azXsNe=^#_OA?{3!3S_;Ejd>i8X75fdv`oCzPs_$&_Pc#Rh z@2Z!AzOyx`cIq2uqQ0hwZxd6yKTj9}embaWU;ldUY{e01@xB(%AYbg%LY%DPJxQ*oFD@|vt8BF}ZF;ct z^FX$4EB8-P*2)<4rCQsPK6>~TO&@)ht=o_F{f5c<1Fx{P*}9$BSOIRG@EQJmi{6*I z%^=hfCIZ*_(M;4+LPw^-HlSSECd40@-9H{ggYLiLGXeKIeKm!rfD^qcX8im#fQ*)2%7~_*&&X-g*zEgZxP<4ZC1Er@3B=Z zS6g%Yxmh}EdC>rd%}>tKPWRXCgdP@UySVIVf89=Kv?yEKtoK}Ms_Za3F6}o!yM`>? zHmK_WQez8(ajsZCE$I+v@U7tZUi*~Y2Uwb*(w*Go9PQ zXWBE4eWuyZ=zXS>8H+*NDN?0ho9Duc1Pu|85o{1!#wSCbCM`1Bdx?c|yA1%oFlXHhE2#`JN*k z+o9FXM4Xy_>bB~;uXtP44@?W+EU~fu2qE1XawSPzBT*xAG}$l{wbEC8A66l`FhUtf zESq-*i19H|e3l}u9T}M4aJ~IO*Dl9hx!ywLikx1jE%HI-<^Ht+OU8a!A=q$Ez~H#_ zbJp0k%CNch0qym#df(-}R&o3?06FW<{+w`iFt)T;;6-S#{Y7_(=4B~ zcA&9emfguef1W=lDob)bjEfTL`*-N>gI;hm+5(-ULj57Bdz9{Tz<`=0&Q;_}WmQpq zU>xn>R0PMRmW9Rbp!+wAqrwm7&x(u6Q*6%u26@Tj6fdbenr_~NY3cC>i+4a24Aqf| z%xg{kldt9E%G+|ij`_+}Up5E)HwwEu6I8r4m7=37$F4+?V3RjfoF=;eJsMYF4KA-Y zfK*UjxdSTYWNe+;+3E^7Hll0Y6i4fCK2-3pRNr%@{eBah7WZL0=N!DL`5L}tbR)Ka z=hmBuhZ=7dSNcMcw0N_l(!U-YPuEo|9zY z49;4`qkXC3*qc=|eGOLh3{3JQx_8aZ!Aq&;^(0xn9X9mEaMLD#Covy*U$w*HEX*dd zwrx)wB|z$|L3%(yvSjp}w+r=OU!8@zD3jdTEp9M{+*i(H&eRIfbVxTt`7B39vZ*MEPpK?s^oz_asH9y-Zi_k4Ii}^p3If=s+7B% zn*#VUIvlIHj~GiXqMwk6Uk~i zp*(m|GrbGVmH)EHM?t13zplb(Mvk1E%E9~|BK-YoX!Vs;UA*uLy&|TrKPWiIeXyiR z_XP7PVf7(<>l}0ifRdBr^E|dBvnRtDh_T<6TILKS+FN6F?R}TI5>Gq~_KR~oW)sc7 z9G64E9f0)=aumi_>WSbAY_d_%{kqWh_&-OP|NF_DpFkHMSU*3o#MD=D7SEM({uaEw zM|pp(l#jk@p4o@z>a?%QoL0pTuLgo$dL1ytSr+*(-K6Tn16#>7EI<~uX8a*7!h6`I z8H;{P&ns*)*Zygdl}&ZsHZ?@`+U$+lWh}E~D30amyXd1cD}G5QLo&(Zz!z8YN#z<= ziHqk5IT=o8M5jMK|LjBPwARoW!RYMJnLVB3mOd9@szc*(d1Kcj>uc|C zy|Ej@et#PFdu`b7(y-raLVkPVIhf7t6ra=qx(t8s^dFSJ{x{{_I-bA$qrWNN_c!Iz zW0A^lp>k(sitqY4=PJiyrMe^XuP{)=`!pSVoA3BV=j;FP%jdN}e|g|<$`_xPzBb>+ zh~;8@HswTzzZu`o+2^AljPKD%!kf4@yazB_6BJkR6FN#@FU zTBzgf^|1XOY{D~05`pwYVF{aS6;R>o6cL!HLh06%4vpuaA@^*Wx<#-*Aht6pJxME? z<*iDOC+k4S{b?T#)^m>hi~S;-C%;PA&%O^)3E0z6FyuzKe-qOO?YUuwus{VP9;!os z6d%9jA8QEsH_-ZK`F}CaGeDJarhEFf?r#W)`{xt_O*fxZABlB(q~yAmo?^t)m@lKh zXZKW!@xhy1E^PJ&H@*P{a}Qk;bMJ;et^mQt2jb#=71X*BYVo1A&{y7semCGa&`Mrs zH-x@_no&pgdcdV`MEcBG*X$UIK|W*X$*JUeg+6wcT$|aNIGl=5G3z}&#?V1c6;S`F zgIXfa-Is*)#P9!KB}hn?bu&XD9eI!;-Q5zu)IDUA==`S%qf)h?Zc-?m*8z@iZ*$oD ztz@c?mxI?$39bJi)?XP~-xGUBkKUwK(U!A!$@tNJh0R$O<4+Lz0Lx^-IYVk68`z|j%2KXEPl44Ws~|BJ9QFM>tlc`QXanuJjy%3-AeD`l&OuN7ZU zWwSM!aB)w35b990M^p+vqB7q`w>T5UbZGW@qH@zEv=$xhu$Z)W(4K;F*zmcbhW`cf zTa}+Tze0{&Yo>sJCgq{mgHOTG+E(^=ZPs3<3^dv8U^h>@3Nf95;h!p_(M(h>eY`nf zmSd8Y5t}pt(M>s%Bz{Or7;p+cg6xX+=3MMS-prv@fI*h0?(siE^Pvhn{(td%KfjYWUxP`{Msdi1or_6?!Qtc!Er%?q&b2Fx0dNhaPr~^B>1%}lb|R0 zQ_7eGNv|OZbh~H^k9!ZseZ-4D!I66h;K;k`2i^q-?(P~o@K0diucBO19=JF-?xW

yPm&1v3sO) zjBj+v9&hsuhDnUW#I5)@tkS3g+Mm(iIDmBD>*+1?e=_{a!HFcGeA^E<1VZiM|1O>* zlF!qT#HHaHz({-l#18*=(OX5rY%cPV+=PLxCM=yUW%U%40&D)FSuZre3Q31C8hiLshu5n7Xq`i(J zdA@}cKZ%d@+Zg=K{w(sw$on=>)KipIj{HX4rDyjrbT@EH6A~fL{ZY;`<2c>^42OD+ zXK>7Nqdi$4TWfigqjNCI)sqjX@hnfN@p3Ty!GjOf>q22ofL|p+`jUSrgv1=Dwt+^D zY6f%n(p4{u=l&!V`U8RHy;Qmhy6T=pd$yqZ$J>1I^v-E5j!#T)(vB zKl|+#&x1)C^Yw*M^bDoLw-@n1#-;Hv}eL&nP&K33}GV{Cd?v>&oky9zTxzrLN#0ggYR4csh$hp+RR@bxFy1pO(G zZM^irNwuk-T9);+kQUmU0>}CK)o^57Wp9EvZ)M!0ieF11bZge^Pa|EMFy2# zK*9_l;WfJT`)oTN@TE5f0$Gd5o1JJ>>*Khi#JIl>Qn82sT9sa-MDt(zi01ZGjbc?7 z3i=CX!0o{4+ZqGr8mqZGO@QRWqz6PhU;ohe`<|@7ZtiW69jg%O>y=zIT|8O8-i*V} zUq!E6y$NxRU{Sr%f%XY)Jl5sTG85WX_b^rDeSr?EkxoInX!Fy;U*pMIfUpZs*qp~q zN)e#$Tc&%jPjgN8o_HOCxR)o>ZIPHBsA8oyVew`?S*ZYz4;o+u2{_Nn+Qf!H0U&v* zJfF4~v(ue{cw1p$UJ^{}Mwr&Em-IFi;Jm7K8T^=42f%*M=xy*FNo)vc@sL9K$vYUi z<&gpTtOT;5T$J=qTkXCqF*&Wy$AYL)C2t{v?ux{c&t_12shOK6rE<9 z|D?vxz>aFjo3kXsvZ23ONoKiovKo;tTc3X$#-D-nSBCvxtt?EU{=0DhWzc^q^gjsu zKX4KCALlG8t#y2a8-*Td_^$w<>mD)$H+GK}QUtN*S-(0DH|tE&Jb5Q9ptF1rYzpYR z`QF=MOJXgS9SfZe@nEV-Pqm`=WzF&P=HwJs*t~6V8Rd8CZOHZZ#_Og^tH~lFZC|{I z?a7O&*%T$YXX0Xzt)Nw{^PLBbo`Qu|d6FSdF6xdES3K5Y=T}j!z5=*l$`! zQzgihmVQW}L8tWIhMdA0;YzZ9D$U`!sgA&K;)9!3Ny9R zCdY2<+Z(pp+qjzxu3$0DF2hjH3g*aCyYheZj3B4vBw1D%acNqNLu2kv4Ca zuR8WN_%G4L``f4VEJQP+yk~C4Z}x7Or2eF5AkMIVd6a!0ZRp~lhW|Uh@0PVG4d<@I zfLb~s+=D?FM<+A{#@Q-k--IIkUvPcqgKZ=Bw)2I)8l1>lN|eWoV#X(x)^Tf+Vkg>w z6gzdk{o&b>a;^L=SzH(O6%lK;+UKD{pA4FTQX9ceMzZZz-U4756|}ALVt13(V63 z@iQaw`~w#JSw!Bs*(l}wtQZ)@T!CUD(mMivIYIn}{VjfEKOl~J{P4FbKM}u|5I;a7 zxYs1?04KA28cYRca%;k@zt#OaA-Baol8`yBd;uk%;U@R4xgB%|tvdTd#s4?Q*J>y- zQAdln&E@v}0_1%N61@@g4$8azD;Qh7z1&AWu}SIXDsx;d93vEMK-ts^=u&vfmQgXY z(>V0|-C5x4(pvMov0X_gtYP-|Ip94M0bWttzk#Rv^MH&%IDjR&T9F^GTaH|VG5g$m zB-f{6;YrYgIb(~o7~^$=EwSlnH`gx7U2Nc^%w_Af2whWh3cnLx&7-YZN@sx(kh;aQ zU^bV27?cFcr+3!Ty}rN~yn&wu{e&DXXB#NstWDl+pB%cc2mBn{b(?*dYu}83E}qcm z@tPA?(yT*qiz@jMiPST^RSOiP*Zw9l2e)1oPy_tmZ*~TZIhf=HBwMF4v!`R* z0LIfDLH91~gYxh!Y`1-6jTFAg6ym|J3=%G;S!SNc{?TAWT zPG;w1dl#Gm2egWCMT^c6O_9zKG2nC;q|-*E(*hlxejZC?{TI|o>=IE2lyZ%y!{bw+ z*uk(I8Uc~{rvqyuxhA9Jt#L?UI3f{?kMa=bU#Sss+Bt}r(3*&tBZ&A36LAF*@g4H_ z3|*(uk1{ZIMIv4)dgyv(kcetNDx_lu(6K+#@!E9_5re_Tb&g1hm3I(N;e%B82QRSW zkTY@L(^qw>qSN%29Qm~JhOdckEWXr3MM7Fq#wUe({S>Z`guma}410Va!eCSI2sVPh zpY>Zen|val5)HvN-b3dt-DK1^|(gP+oEi^E}*$wKQV|fa2QuO&vPY`YFG@KY-U)0i!dG+hPIM~ z|G-&k#N^oX#9h_rVn}J>Ru1V4^6OU&*_8mFb2I85nF~LlrZO#txC6P;X7Utl@fhWw zokxus<|njsNnH@e9)+oL>`!n@}l;Vw3Eg{(6+p-;r>BLJJ{ipNk89Ql6*CVD{!F8&mgS z^wgZ6@H%SSD6%YUxrM(08U1a^9@ql*fFJFh!&v2xC}Ho^D0f})zh{qm2N=WO>h58u z&@He^a&}|a(~X4P5P{tVlS?At_nA?{Z^PyP3-*{jQ?*BK$QMV0XGa44;yjpo`P%*G#bjC=j@jjHA~M`IO_onZZYuoA=PKbcBj*(IG~K*DD1aN`(5&wL^k2D zpR0_#$}h)_o%=Jxxvm-Ux#++SnvvJq7&P*Bji{pw-Ru6<&@%A7#t#vY#!QFq+b|>y zOxaZ&0Vs73-ZI7gY2j(w&o|cxkc4UKK3StP(6~wzCJA4g4Sr*}F{48A-hk_`u*TAB zzs-A>(c&H7oMZgO-!{nqIM2rw7qoF^;|%`>GSQ*<8u)#azMt{j547-o7~Ic_^KzU_ zd#VkO^H1A*0rM&c4zT|IsmpovH_-FcMQAJBZVa`M={%ttKj#UFCyo!Y{|@b+vo@(DD`_D8`?v|t9h zg8gjB<#$%+z~ZrPck~LHaVC0i`LQsElfwDkB5%r-8!UV?1LJc5y*3MLawl7pRXK9! zWdCL@Z@?ApBF^V0x7G#xPZ3t1YBoD!Be`?XyYqO2@qR7Bs$2Q~qD?NUvY;LG$miNh z?nMJHA|j+K)Alvd4r|h-LfOSTy5OShu`0r%#QCtuf91+YEvz|Sr!e&P9iF^IPyQBs zS&Knma~kQThFq@f`-q`L(CaYnYc7m9?sdahF1!hieX|1*@?nOJS2tM96AQzwOb(iT zs4JC6)@kGKj>CRrFAsbACEBpNVxQN26mi%zuex!+N3rtlUx$wSwgz?FM^>J1+(s(JRd?vI}$9Ra6f;e*ic4hnJ zfzZ5`ur+)DYye#4h4dorv0qrsKAaR#Ly|i#%gwyTZle6~A4Xi+29IMqlLRZhN8u># zU)r3R7z4#oohx7C|BCNHv?*2JBHBO2NoXUwiw<^Y?+^m*4i0X40WM58MzW)DcJK>8 zOD;@4>VIB;Kj>6KME-eVC{fIOKL5iRisy2ctdt!(s&&*Yn4g6sqA1VPI~mDlC$bIP z!_1UO0-%Y`o?4OZb%#)8f%JSsi$^Lbi zSEhWeM#uM%j^#gd|DSg}7acFGXFB3pp8KAFhyew^0J7w|hvQJAkmhYlJ@fPxiAWbw3+&A0c?Z4liXS`C3;u1@!xs$nY)}$*Ls0 zS=rs^UVHxgc%k)s`o8~EUtf>U#%wa54=HaBrSl1Uimfsm-<26Evr~xUDznr1UxWB5 zBK2qENs4T{=Rp>J}zPsu7le+n*;wE^=^ZfGAUN+OD@Pc z>HI5wsQ%i()$l6_U%$7b`P#@CyE#dhvVKS~E!971In4$zn}TDr@g&%sN9t_7>Yba0 zZ}LT-vaL|wp08f>G1*f$#^C4{)HjJwc>jp-**z|Jt|wA~DgT(SAIS&UNW_(oRfgS3 z87q3?uXxYE6kuO+6OA5HueynXc66WCNaUPwBL8D>U9fJ6aw#;`+>V# z-$+->rRfp#ZzQSZx7^bl5McahHntGDc*{P)2ma?W{)cs0f(Eh$H1b#1x5wX}AR^6p>t z0po6R3uyS1hm7wkI+w!4J?!`n|7reQPaM|w7z*$iDP6z9tAr-sIhd@IE2oHBik zO?K>tz4a=kPd$d<)^0>h6qjxmo~{qzGcETnds~|ais0Mw6U5SE#)bJG#C`A_`A7D& z_~Ry*h&*~GB^n!Ll+U~lf1#V~9{7-(F~bC=h~p<11euAmP0H8{Nln29hTMMvi|3I*SboO> zUq@?%{K*ueyk>Bxm|V}5j$6$&G0D+(8(;Uzrh7a|NQo84Fy;I)dfzCKd6LU>4CPAK zV>HPRf1^WRoJZj<8A&yV8u2?Fllo3aw|@`6e|Mhzcq2KJi=z~iNhnb&MKO}zXYq7} z72ksul_^g_5jniFNKY1(;g!{cSRb?1def#wtr-qNyoIdH%DXJyRdCj z6u-KCkej0#O2-ppnE2l$zqsGIl@X7>6H&D8rY!cfx(z{*I%A3>5 zkMkpM#*NrrCz=>P>L9@yO;IeOf*KNYl_*oP`TBizTqC~y-G=|b#swcABCFnkgsQYw z(ezOGm5@L}Q0-zyTzNg8$J0HQ#`D9jSo&PdpWTCDT*}b6;7Bkt*@jF5Uojcc%CEf< zejWfQm&G=!CWYj3#nNL5lk&hr%@kiM7Yv9T?_8rYRw2I#zA2Gh1yC)o*D1FoECq;Z)y(!f%ATzjHbezVW!am~<;>+9oUT**e<(>G`Xi_r$wNgG(ULIXkv zdDp%4pMymJCd%xew!l3#Mq;WP9v191^Xow`Ty_v6JkGP;olzv37S z`jN9*+$30pRkV!EYonC@BY1)R!I5xY1fbaVD*o)g7^!eHD!9Ph7&?_MU5x4w;DO#m z;S~m&%o}i<;_Ka{=08ceicljdA9@ShHazcCXNE(Lr)hOCPRBi0AfmSl29r==pqa5Gb@ zemGNWAYS_X!Wg91?PuBFT{>D5Ja&tD13vQi#c(XdXzFV(_BBbj2CvMdjq&3ILrB#D2#7fOZs+Cl3IWIs7nU2Kv2Ql_F&nDjy*#{Sxo#Pha+OyYzUeGZ z_RlK(HHcjI`tb82wjavq_2TB=>oxuJS1dA+dXB0LK161X{BNKDu6M^igK{2N->hb%Dcm-%UB!Q1SaPtBb1VfjwaQNzd1X$g|h$ z<09$#&3%D|AJ*ylGS-Jip${KrJ#Qj?h#CCoo#2LiZ2RtJ4*Zi(b@IyaY2c$K<@jcy z6euSi5Tk3Ru%)(g0u_7(#_uG=8S#e)>p3ZTh4PFdROiREE!jfHeq9cdrTuU<{@4#c zdly) zQe+Qk`s408q#q{HT<6u%?}@ZJj^WMcouxN1`=>(f5XK*i^;)a-@Dd)uOF?+Wx_UQY zy@MQh?;55{EE3r$ANZ+lC-U(~(i5-q6l`M)02Kn9}tcp>EtE>Lx&f zh(>4Vt;(LjPy+vcd|ptxQ1{pt%;~__Vj_l zIU9BQW^v6qlu4DcvJcUxM_mYgs%|7@QssLrpkG&mYa;5c`?14zWKbm}QYm%$PiV1~ECPV$vTl@oi`b+kdOtXTcxcaq;#Cdvr+^Fv+YDo{lze>`Tyt zt+0KG%~&1SZ=>DjQcNeg&OoblucBPU*9#mrFSP^hvi?jQ73JN4b7UVE^ly6HTxHUJ z-PtqhwLvD#wTNNxVt8NRtV@(!3yGlw*tF{_Vo`V93(%F5sl?kWQ_K7x>+QSIDt;Rn zzkSN`GSEXBYH0$sPg0p1@8`>U_lx)+T3`LJ9_Y!_(ASfqZhx-ZUpJ3{cM$gKQ_u!R zp)9$98zgXpgg`LvsJ?!m^|4%klBoYH-`)SUImmA+J|CPWeDV;VniQ_SlB=)O#e>u8 zKhE{9#QGJY{#>pwSkPAIr)#$9q>)j~Rel0=ph8w5P78xr1WoV~&#K`0n>w}|PT^8GtgO49hxTw24eepJ|8W%2=y2kZ z(J-LTcgkZ88w5_ntGi)nAwV z;CuV1yl)Sqm3y{^?%TN&u0!^z`)60=E158emC=j)Prz^2L}i>|#2y-FuP_~NTV(>u ziA5{us(s|8MrHl?>%~OO^9Ri1LJj|n_{dl7SI=L^^4}H_)qD{<-WWaPYb}bA zFKm2M{{5l+JO6!tch+oH|C_P$q5t8R+4ogNYI`={8ox?os1u8)N*#)X`fDFn?xFt* zzP&^E-t=$tceuaj&RW`t1M8_Y^#8RyC@Jybu0s!T@cFmKL}kQyPwJD=h1nO7%{tg$ zcoe-No@JBo*o#e)uQv|-@{w97jb6#1q!C`@;b%B>Y_S;1`E(KfCnEQ=52;HO@%AHO z+na(8XV~*Fj<73lID$%7HD~`xBM;1gsQVr|yK8zH_C311Ih=WEUUh*jf`8?i2DX*; z!-pCuSzc=#W*;3N<9@Qdu0E;W8S#+Dib znVM34A?8v;?8~-5#$2MMh6Vqy{l}mW6qWCf{HOiLxnHx5##?2)-MD3X@?r?@PI2mr zPFIO3*Y{~Q~+-Z+0CYQ6CTF5~)~iCS+gx;wPq*b=9# zH%`<2_Iq&Gai+Z7_$pprZs;<_m3N2lI@;rCwZZlqhk_U*e|X)01eIDJcAu23GklMz z>jHPysB*Sn1bTMo9LDAQ*c3JEJY#;~0h-Dui(fZ}Xmpx5f&}UvFYDOF8hNjXso3+` z+FUvpbA58#ouKPvhv5-C$7&NEU|hj_#G+NL(Y6HZvYJABEn{V!SRZyD1@*skF?Hil z`;f+Exe%do?LsU*4y~tD71?r5QIWq!TN{GeVKa6r!1OI;i;@Rr^mpFR(0>J$X1D6% z$hKMZtWh;+vvR8N!>y!2+lr&L=a)yS+H((Mc4d7?Z?1^an>l|>z4>0O(8RXI z&Yi_9??#i8!z>n-Wyjf=dh@y85hjz)Mr_w4A|EccMa5f^@AS_Aq4p>fYIGhW{00Et z_c3nWlFfSj-lp6KO@~W87y!{aX&qxrjMA{?}$Ez^2Lw_Cc}KGYmlt5~vC= z#7EhwLir;G{9yuYXPhj+_=)JPot={4;?A%)?4aZ=l3S<44oWI}hwk65j*|~cNM!Zy z8xmO!Vkarc%)~)t?dcsZvc{0eYLe~a`o}r|MV@^0iMpRXZz$td0h3R)B4gqNN-i8O z_Ba0Y^S(1|@xo-$-NhOM;*0ZWe`a-<>MRgC^ByZU~bhFI%8ry=f9b_?X1T7VN zvR&gKW_=y9vfX3fFg1=Aol5*xhIa!f(=#^{(OXW(6hpTF!oIkO{j9yG*~nKVk=*{< ziFkh!BZ!v*B=x%^{CB-@I;Q;j6w3S7GT}0ZvZ$g($?^iL;>xK+@*&B@i})U+>Gbw! z``|pGnyygfVV7JVPsS${y!$XO?6P1;+e0kgEi3_EA$2Mt3piqgHHD(^moX`_`a9*w z3s3iLHzM6m`p-szgFz$Yo!~!6MDN7bijQR~nvrc7^U+7Uu+?|qF6=y6D|)vz zsBXeOm5H0MxlF`321=oQxy8T4>=&=~gnm(}=po|G&&#dwa;4Lhg{#EqnNA~v`Kx`3 zzU>W-ki1$tuQ)bbp0{hQ&^pf1m*ul!V~)FVhQ>w!x`)K+>4ewupJR@UIK=Bt1uv5W z=J^KO5^))b8zeC*T#gA$7_9p_rnvEBPGGx#=dN0ENx5-dPyD7?ho-hi! z$syz_e*?&s!jN42;^b8ii4NULEzSESqNR-)Y+nx<;akE;wSa(BYomngnp+9C)OOh+ zR?PL)Sa#v>PoPgsqK){{JuH;LloN?`bze9ROE433&+P-)JoOIKDsDN+uawdaRUAj*e2HSB?0sL*Z1MBh2-ZQF52#BBVL6Jaozm<*Rznq9E z88-K5mYBaMt>&tJiZ_#bF^8!ad2GfNrT%Q*xLp@QRjUg-jM;N0v^4q@b&z#GhiMk; zm8H>{(QIji%OY~J31i{Fb{hY+>dI(Aym}IoHnkH z-td4T!wMfzcI9su*seUC3h$wKZ+O?{cud)q3z$(br!sljf>k~ZMK+VNP!QVRg!KzL z{-gEZPxAUt-rK}uKZVvy@Nmh2dL+B4C)z+9Azh&jXuAdDiN&)@eetK5vT)PQBq2uD zM@Wc0h-ZCdyrtC9KgHC)7TioCqM}_nH~QUAS3PIQWwjyS!fKm-i+s+oA=G|x*+AA1 z-@Rj>tQzs@Oj3Z2 z?8@R9@sHKaLe)fj#%Dw?zPsk@OYR?T^FXz8ffh9EFTYbbNETeFchE{@D-fd#U?m@o zA07~-6PM7@c(~t(yNRaD;bqIRh<HWC3-lnz9vA4x>r5`Y2UpSVixI^YEa9(8uDZGI?aLNxjpj+huZ8DrU921{Q zXNC_8-enhRZraf<+c}64bB!Wj3OzWtb?uAlR5jiQ64;i`s@isCT{hb)%+k||z3{z} zmemGHh!=CV;t@nOI`RN4NLtI=oTB3!66dr$9Z0M#ti}Fzx0%AQnAeh;G|h}F{JFJ$ zXSlp;JBodvSG4q4bvYwxs621y)VOE)>8&7%Is7Rh%b>p)*aVcA<6U(r>imx&6W z#Wm=WL=6QO3)v4(@!j!KFycKw;yn~U;vns3JL-fy;1^GuH$n7P637#(#D}|v+Dx0K z_9bSRaW4T9e95rP9IiB=VH+S%W+Jh-W2BsGaVC1#%S;g*7Z?RvzGo{=oFBI=)hLZ9 z>4xs58^jIgaCoL(_#O}JvVJ;@BBwG>s5f{R=bX@3s?qi|#wtX^NO#t7IganhO4$=y z_#Sll`NQyuG!J*j*4c~uX>8ZPpWYgE6a2C7kv%9)x46RC^K;i`IUYjqn0DWI{~GK6 zvMh*g_esp&ffnVLpi4+&&4e?qtgZ*NFUJ$wf0jc4z5*md+x6FJOWRS$WTf1C)giU# zwjlYtsN$U0A@7MCo2%PZ!P>52rYElP6$XixloyS{1zbmn_f~{%79P^Do6SE}V<)S} zV;F*0MuB{jQp1`!=UAnS=B9o!w4_CfP}10wf_uQ`&a_&xte z7(rjUj_&Fkbu{~>v2nDBGLad@y0A>`T`1F6Ob>jRGU1{^@rX{Dj&x!DRC5;lX;$_@ z>Y$3&ahBEd4AWa8$ju)J1B2fAyek5qQi78&KIdi50c3E z-~g@NbFOi5w%&uM1# zf4l?zCo=l4`7VM$`8^~mG{G3Cd}&dnX~)}y_3U&?97zo3f`7$wgCh4gR$E@j+HPXC*?OUGPK2H;L;YkkgIeX^5H|mI@RaiJ z?u&)umpFECr8p;BU4xNi?h$6XRRe3khlCKlo)6LMZ!*35GAtI3kz3%}+@S>%TNQfQ z#&n?72Ol$9^`FaHb*zH5YNO)(H7dT9a7e6hDi{@}Uv2-#aLnJI<|~HPZxtP|s%&9& z{Ys(hCvzAElVl94nBS14($(z~{tf*AbwG;0`YH75g-(^n2*=YJ7LB(DlGAe#Jo3&wlxlBc~ zs}K!0H;QA|NwQT&H|K1!1?MkG=*(lt7t9X)s{XKI>a*CkMu*Zi+mR@Go60-j--Y3C z`%s~`R>t+c&t-aJUD48o)|am0et2gV)L)D^)El92cx@K11JkmT^tx@kQO=?Pi9|@1eceugq8qu4=%L9x~csaQr!I1@6 z`2FdK{OSKOzdK>j0$h(TCY*&4y`%l*F9zLwLYh?=kwF%%AUnZM_ctvP9XA zq1(Sp%)(c*sE2({b|b=?>GX$OF9QT0x~xKbw6ECbpTvhZAdnPd)1QY*j29_;?z$RHEBIpT1V?) z(?*lwJR~@MXuXE_bvK}~f)NSg&d#3Smu}`}-5nBxCz#5y9xoxJR>+qSc4^q{S~|)= zbyZV!i(up&#on0MDIl?<%X2@ZlQbUodrk6I`36`wL~(Y7P%e!>q=)Ix{-MSM$eF{_T{NPCZ!u2X(Y%_I+*mmYMii0`o>ymDs{! z-vR({ZKC3LKOAu_0Pv+SheZ+lg%jr~`-!*4+D}X!q39$hU>J2qYZUo1#te&Mwf-+-~Xk!GcXGY4?t~&YXH>N?F`gd zrBCI2(N+9JhGpAC@s^K^Ie@vL&i`≤4j4CFYgqP>`ZU$2xL&?ZnHXLQ#p;op6SZ zUD!nA&Jteh%l7%V+(zSV9|;m$H`EwbRw0nd;qlP2|9Xhg#3(kPCgRF=VuG9I2f+Oy z3YL9Td=LgG#=DVc4#+nh@`;l_gfc4FGWk(X@V^9)0N-Zl<9#z=2O80&)U-&jHV?&h(y~s`L0=$U!A;#*8K$7*uA5H%;w1J9(jj@F z9s0x&x`##j40Mej{u0qOQg_R<{$pB-KT38I>v1a^NiY5+HHU23(J;z|aqO}W*P(!p zXLmh?;S`%R4gFTJAf>|abbJ8ycz6ar^oEDt_|Old+Tp_>co>8aIq;B!4`z5U;{!;_ z;bZUtG@;=W@nO1sq)rPwqRpq_(`}Cf2XjHckM! z8dX@~=S2*XGwj_kejeWs{9K7=65bg6d{~BQ`^I%}x%J*wW@JnVrHvVg|GyJ6#GXInrKD2BSp z!oDbfG49mYJ+g+xVic(R|4`0Q=ldiWwLj*{7WW~Z-L!O%JnOY)@|Gkn*nkRDLJfS> z&^*l6XKWjRg)mXhvt&=hRV65q@7zI7Ul*ZBb9iKNw&cl>5EGiIo_k*j@2P^RM4Y~p z$+5}uTC_h4UVhr_nt(w#OVCHSdFBlU2_uBxA%22aEFP%;<#R*SBfGgO`yPPGXCGO1Hj)C=NoMe}7F7L-}DA zTOuy8{FTCuX4jNj^jX=7;$N(nBq5&pQVfJ`M3WgE&@8b=OPd^cX#|(%H5z)2-tsXV zGZ~uv$?Cp0u71Wb{1)CLZ%AxP}Z zH5zhI{w&RZ7GZzX7LZlg#tTjMpSe7tipXhi({iP4Y{Eyv(oPL}G(El+ST(a5ZBATy zRVD{WM+!A!Z_}|LFK8}m+A^OeT;Bdm8jCdoOkVaCs^m{=)!~6VbMu=661z{+aeo_N zCNM|&R$-^G#|YH^qZo36iTBIV?IrrO|La>V~RM!seh zRgu~5kEii&4s2^jQC~VDbnif|)qf^bE8a)Rb3H-sH-9fP=9>4J>#e?d!V@vwY|H*_ zetxq>+4QT<8~lXEMi=R_3HRgT=M!2ba??%@C@8*Rk;XDJAkKM5K3X1vEc^lYc23I&;J1&251I?#JH{vE5e|OMXo8CH7X&$8pUmhld6sQL-9s*A56fp&rL3W;rw| zHz*l)I6N!5$PuDvO_C^qTjQN9)RIH%@cA@yy};m)myK_P1H7;s>T!ID2U)4NVHKcR z7=^P4xWR-z76xR}A?<4#wvc{k3@tL7K4aJ{+tP$mL;Q?R!e)Gl_Rm-t)`Alw_aft+GrsVabDsn*rdf4B#E%0H>KcUfm1b2mEmh!G(PHK|bF^ zNw()c95%j8hVx_b78Ub!7qT7cg@^irX#;Mj;;I}zY$ULP30RDV8m6`4m^b8mbyG2% zp>i(G_9xzYqeTvuyp2xNZHXF9$&ehbbh4I|%rxryQxj?C?O{bs578+fo_hsoFcfXn zaHUpqm6bYt3-+nUaCTS|)~ELTN;;xmT9{@u z{!ti_3w)c*PhlQ*PKzS|SKq6$|D^0*Uxc^6u$SC+^b@1Dy?=$$_NB@_`D{-%AfH5? z^bpeRMVf9TdC!L{U8zHvoSOz&J6EXZyxY;)%p&H4=E@i9`70(n&QPP5B#60Ro|P;! zZ?K?COgg-@rD;X}w`L~5_v@HxFWwwN*Laqh9QJvK2cT+q^1=<~nRVNBX&M&qh|Ql} z|G^mX>|%1$#pv*WPom8*+6TL|PTmK54=o267k*D(-ikJg2fqnQ_WJhhzoI)vhZIjO zh53l=vs1f$K^Bh4rL|#PT7NMh`T8mTrIuhE@+mvzVA`gQoCB8(e8nFbh4+^yGXK!N z8zSS&XHOg!NoNe4U^4td2%IAXvOg-1r0+i%eg9q_N#Ac2`kvk#gT5y&%k-TbMqjgQ zp-SIj@kn1JSct|xkE}1_o+XUdKexA%S!pRbnVHMl_S@&sHuPcX8ya>m;P~f>^0wlZ z#bH#N-59HHdsk53uB1O%_(u^Lvw_M+*Oc}AB>qto`pUnaEThmdml%X~H`%9#`JX6+ zKEFgKUXGW&pFo3eM%$>F_Ypbb(Xh?rG~2gp#l4q^B375lDbrNxbuvZJ_CdSTD%6As zB0dC#d=90Z1~To;uMK1SbM0YlKe|a_``T7E9*60n?ga7YEoUX(#Zir}5qjV4s$KBA zn_|-K-iBz|40&HFB6{H;AmZ|(5ZU?znN5ygCQXI4X{$=G-8TWjtgT`862#Zz;-)sn z*WFjb_}YDw!q;OhA-*ONe%~Q}rQz$J3osfks`(T?JQb(-`Qu7AXe0T6JQ#UV?v;LR zNBR$v=^u#fl;ZYA;K3b$YMJQ2F{tj6AA;;4p6D2KMw?8}VS2K4G_4osZ;l{2uzkG` zEui_0tI#hMC>tsSA-&7aDvN#8NtR=*{zIXqQ6JdNL^^#v%1Lyfqrp}8ZdR~e{z zDyXt63>%E&@k~V|GXG*W-~6X^%Ne=2I*g0E&H)$uHe)AP#h|8ts?cQtvm#5PlH~E} zg>e33$^Ok(Z|UAo6!iMYwvEz-t2gEiHFq0z$;B-BIW>77Ept{#mtvj=WX)2(-;)&cJV38}e;{`K z;v>p=MN7>3YnAWCvFqQUeE&GHqkVMbdSL!Dq4hxF2Bt?_hJc!l5dmz&AV=r-fs82{ zrmN5Uu!$+~@cdvNPPWT4^%`3?dL~+uph2rz%9o4z*tec&+q#h;Zy|r_cM=R6Y4Kva zRgxx?`pq^Jh(5Cv{BPYI+kRYae{)Rw^j5x4i+P@srB}YcknkVv$7!j8_xi@;PB`8P z5Dv+qIgy0Ix17VmRJ(!i(gLH$(gY3JU^jrC7913$v(EfA@>&(|138Y-u7xev_p&CW z_cLAc!!@!li8_B8$pdp58NY0!UY&LQ$9O!3u-_aMloV|!agFRikcOV)m~A6ul@=w) zSrF;xX z8u=SV0PW#^lCrK@LhTg?PId3N?~n-v`;ognah?K zsflkBRSP~uTTc%V7Vj4gM22*G^$;z1=A%a=NgVtN`V~6EZzVg*FMS|GU4^s;b~M>W zh&$tgl98+kBh8!{y+EXT%E-D19)TrC23og{1~h}17k#ESt)^Lps{5vUSaT%ri=%@%m6Ab!v2vF_q~4- zaW8y;t9_)bEenRg-u$vic!e9~{7YH>{c3(A>1$b#)` zcagWP+o3e5RPKcD3P^43zkqH|nmN-xAn}R*2xr9;uDo4%v0L7JnoO~sILA;Rt84r& zz~RR?2#4y0?Aqn~{ev`rq~Xgt6G(`(2z%gIf$(y!dXKeZd>&p7FVI`a+baIJqeR1Me720O@yQCE=Qcrs8h#lZp3&GK%Y{_`yk#mN{6VCAr${P4fmsM!EFD{m{AMKP+81gXYwj<3a# z7u^=mBWaNUAit#H+!n-sh17MtPSoA`E;-aPc<8%0(f})_L+bsbXnDbkKE#T?sT38v zK`B4RZC2z%vMe781?(P#@)<=k92ut3K$D&hGXFD;=9vS@u_~%0%YSW+wqMEq!@KIT zUp~teE|c9gUQC>L`5iJq=U%PkUh)j8GjmC-Qu!F)Jjri@0?dpE}UYfWI*JH{4zlPlezexC*)|-W+bp8?n zq(q}auAFRE4(!;$Ht?noXe(RF!j=ZUm`+ZppLstBG8R4u(Ru};r3ekx79t!aQrHN z!t(|HE<|H>83v~Fd!pkT9s(Q!BpuVuuKR_&U1G|g+EGkf-8ot`1^YXV3-4Pfu)Hun z(f{AwFf=BI(<|djKcjV!fz)jv48`9`lGMYW+1*S^s%rufHBr)*t6$ufLS6zKzv5lSI@qJ$4XRp+`LJVotIiV@t0UZ(@)B zCAvp99wUp{8;Kmp#WwEVP~Ve#Bs;)Y;;%0`vdOmK7gS3R-CdgWa|v{Gc<`a z{zxu;#u|>%MX=C$g-1N|&$AK$C7wJ#hwb;wrm9i1+wx`@);}B#bUdKW6Ylr%Y#-et zzQ%z0{v2dO3%Qu+mKD+$$awd5rWtFL`&4plWE{xf;rSolLz{V$IT7xG2W&>c$1Yfc z=D_5*Z=t&X=~<7MqN9o}bHXsamth(n=BI;wK>Jv-6NO$3vWb`DG5yJQ)aXyxlM$#?mP z+&$Bx{4$%4L?nWkKia|Ht>SSORLj5Bs~*25!SKFQ)f3Uw8M;PWsXV}&aDcCS4he1R zql9eIi%*rZ(Y*%`&|rP>9BRTx)aos+DSH2DXvUU*U^5z{nqfF5-U$uy?~0AzH*CY{ zvREi6cZh98%z__E_M1vvzsEm2Z2hL-Jv;*MQv%^_2;u#iJpyl=5Q%pP9^n5|guG~6 zhb?K3>S2J-^uqz#2JIaMtHb*65sT2I7{CC{?;v{t8E=<89Phgs-aSKjx2=-#rdZSp zvYN@DhU(P{L-qrjsQ*Z}^-j=f!-euHpu-{y+SSi`QrqlXMB4 zuF)rV=2BAOH4a`A;k6UIc79^X{6EfnJfr`;ch6XAdm`g0P1Jn$pYV^1dXJBK*ZnWw zl{&-egKbAzDDB4s;ZE=mw?A`#vm7%vJ zSGWFe_to%$N(~)~u7b3&cuRIXjJFB2(%=e`;ork)mwB!rJ)bM+wTF)4imjPx@H;Lu z1AZSvtHp!WAOpWd4sJ#kZb2Rzaks?ivSxNQy2fRu8(oiO-jdIAD}FG#=41{q zf^Ij+2z_~&S(uZV^A%FgEZ8#5`DSj#A+xX?O&$YgZ$pMZF2BsE_3(x&-dmMs)PBnw ze&oH?@N}9tH1b|gic#CZ8(Mg;H$}~-SM%xBd`W7)B&a9@s>lQ+dIJu38MXU)!yewd z9iSfN4TpJe4LmjRhD*G+5}-=Fp^f)$OHuRb)qHw2Uy_;+o@#kR4TExkH~hdb5AueK z49hdTp_M_F^Xb)mdNp5?nh){q4fx&#_+|jUnMUpRykR$^LM3lF#4x|W8!j^}1H7S` zQAWdb~?f;CewjLQUI*d_tnBw(8a?7cV!03x+IyYINn z;u;Dk%qv@QC~?oGks-@Dpn5^VFRhZ)VLWDiM4=!Gvdigju`aXY;I!0ua76iYS^FS& zcReeGxeF+F0p$*ca_kIr@@W=mq(=GTpapB;nY=h(|-|Bs-*jL-*Lq*~4~j5d#h*!Cc`J*c)FL}Si^ z(Lmrk;NLv>)KCQv`S7)2H>UKHd);^N??>oTN$_uU8vIPifS+moScf_b9SRR4{DWnB z$K*<$!Sf(=l|=YS(dEHn+EDwT)0hOEWoO}Oi@WD#RpE%DKlG10;ef?^Hp3FA%)oBq zNCu8UpYg!U;;xpJKWj8OI?Kv7_ydxDYdhJ%oAGj_5qOXLahKs?FtlhX>qutzpED^$ zg1g&P+^utWe-i%Wo4EPgH2C}NcqmzH1Tg?F2j0gQYvveyIVR!DxXkJJ=dsNBn0AKB zG*Fp8mf&-Miu|bzpCu~s^m_bVx-p-!Q*OxY$)~)KbzeT^gpBp@v$QdvP$5jXV8WyT zenU*SVoO>+p}`eDjFg?!&$oin?Dtop9})qZV#KaL2LCYh3lM$y11Q5UAPc{QW&xsE zfXEdfUImC!0isl(g@0Io0ajfwsUPCDyNU3&5ucZ5_Je7zU`T~Lq_lo55+PWHKfB>m zg9m?hU^lM8pS=JAzz2O$Z^JM6a~=Ww3P05I7aZz}@9p^hTMB&lpN7Q4l-M6%8?FX^ zJO1p64gLy&PeMpH;`?s|J_+HS!8$2%2)ERMh)XjM#nC`LYT1<0NPWDit$7=M1i3U}hq0lAA2UxFYjgJ-eAEBJFNM59on zuE_n4h!oMNGgkw@N9Ic`@RG~|rBT1i{f<~1(WuiXsq3zgun4lz8f2q2$VO`rM=RuR z1Qcd?@u?M`4#3mwi}=$9uLZU6Gp7>AFwlAqGGgvdc$<41A<#e%(m)T=Ko7z|HVGk`hhqax?r&cI42dOjn0#=9f4a$!o$PcK7 zug70Yqbw#!a`wYoe$AJmH?vk5x!KqmX1C-caGsk}kHGgJaOBin>@#ywWPl(Fq96*Q zAPP<*U{D``KwOY40194!Do7Nh3Dj}~0YUc$ngB^)0>L?eTZJK{U~U+Qlf+b*-3lIo z*{xs&^sR;gnSemU5Aqb)43pk{@M%#FWG$YSMg_*m(_z6Q)8VPO9~PZ&c9)ls1Z=QT z$87i!9uLC57Ra+WN3N}S+8CwI;tA8`ii?Y(^fy>qK>}$8AdX^}JA^;~Gdz%?4J1$Y z!+RS;gdxGI57lg5d6r1yKz~ra=i=1amb1^((hI3;_BYt_Dre!%d4@XV>*F-%v(ahH z?B~mh`)`q!wMJGiiu-Y;Tpa0dcoYhn2v))s51{Tk2a}3(ND5@24ni77Ki@X3JOQfn z-~TadugcRXQgWq#(8|bX$ZQD5e;ndZ#Kt$wv+}I+hLmg5C~*$|f7msn_-GqRWat;o z!{ss$N%|lGJgOp&s^ayiDt66t|ADEBH-xE*QTicXd8C!YFZI>Ui>qc7J=*=93zAfI zvgpwp@oi~ZRb|nmgA}#T>@Lm39RN`Ord=TQZx-g}i$LeX-*3lP7Zvj*_;}!bc(km? z$72#aKJ0;?N%ht3kQII)XM3?09w1+Pu_XhZAX|I!!#VIXX*w1=Lm(RvnBgiEP@(hB6oIeJ8J$oL$wPW7);#mk5UyJ7>*d^2bGs4?nT~xrAz>j4;{2+b> zh+hFFrb#~BG{Z+#+Rh3gTXR5P;JE}i#IA2vcUei$BJ@`xYx&}|RdqHDL zb^C&SU8~)G4?^o--7R>&dQHQQYR7_$Db>HhQm5MC-w_V#bU3IBDk#T-#v}xFBjoom z#1v5O1s9VL)QwQf4!}G!6MjHCKvN&brht+#z54E8UG;|zO;we7nm~20;IZnr8>FgA zlSYCXPD2fK0OnFk_4x&Lh-{sLYy;GA3@X2zQhjQ{0Yvs#IH;y@P(c-xW5J~)1ck`f zDabNV?gf{V5ELSN4BG95cFTxl1$RQbWqNiGo&iwQIGIk+I0dV70E%jn5r$gAK?TD> zomN3PR602rs39sSw~8>d9jHfiD@M8%6(Bhakeqi14?w$R7WWDsfp*LE>>fM?HBjSZ zIzi(UWG_Gs#6P(!0RO^4T?hx&q=Is&baF6ILsU?1wJSicLj8$Z^RlrGbbwKj6@*c7 zcMwKJui&KmDn5J0#VY>p8Ex=1+Y3*#4^;7~bN0f|+yJD_ISM~>E34as+UnneFnZ1h zVf35|;^>*Nzk2SBw(9oT-sh?KDtJ~*RR?W=HZON$mIV*REp=x+) z?#>MStus{3?3H^N(tG6|gXiqrBk-J^>oEXbhhYb`QU_g59n=bh?jD3;+ARpfG$FXe z11&JuS9!b7I9SCe%m9?T&priDvsf#;dsb9cdW_uO9$ASDK5(vZqP4GK3>SD|OU>fH?ktFs$&Gpc`U7?o3f zuwmYGd8(-n7T;IBqIg6GJdDnPpMvS~G*aEY;g0HV4fCK+H9V42ozSpCEwiHdckmCy z(~9B=Iq*}gmPv(r&Nn=gQGKdmDl{^p%zXfT0@gDP>j{z#xrW@F>K_~O z!^+$W|3+h(5m@HQ>0vq$?GpdJ6?C8-P|06*_XId58|1O#wtorm_(OFEB!V z7{WQfV|kS^F*w4Nw$Q~KMNuL786pMIilnLUi}=XxhFI{sH@rh7x;1>XK6Vd)*LP(G ze2iJR(xt3^k6M3Mtlz`x_pthxL;d&t59+_W;V%Rsq2cTL{~A8<_f-%3`%OLJ_dVTE zDo5Tj{01O?q)#w)8Hhg)@Ow4xe~({k1Jwil{4cmV`5)j5f4|VMqOT$duJlHRe~yfQ zd+{xZe-6VxhvAR(J*@q|$NyBrKd6T4hOejp-?Rt*zMjMWelwlg$AYn!m#F-q_LZ>q z;U6Y!eRy^B|8@I%HT;8W%xZXk{r{Wx!uQup*xwhh3cR6E?ooNGqp`!>?dpCJOdHxSm?=u&-&RdCOu%jZYQmiK%`VsFr`Iit9; zs071kaHa3z?ce8LhR-fjNp>8iO@X&;9X>kV$Fn(iR{!gRQjG&$KH$Ys^Gt`0ygwAJ zm;EWAlYw6-AV;NHaG(jLm~$=Kn3UX&Ano<-f~geUp^+~%JohDW>MGo54;FLJ>YzfQ z(O!StF7o!~8yEe~ox9K8oV@5(doaUxli8h}U!15BJFk-6pBz-m^G(${TNkS!n(g&% zD~mSL#xXZsU1wQ&N(!L+x$|5p#<#+Yn7g{+Ws@qcS-TC5 za(1T$SOrxYx>M#vXT#DVOt~$NY@rLgx=f`GPhl!lCIyjJGvf)3IL>hoTGKUEr*WlE zkaYzj)d2ahDA&Sm!bX-T+$F5Fun&X^cPm> z(=JfRrpBWb0^;uz`|UC-3!t5Q;K1E^CpRf-Xu0`8m=^!Rd)RKcbRvwtPp91W;07{LCOb^64XIQ<@ zko+CL*6c1ULwu@gS^5lWvI)}1NT4AAS)sG$QDUCcE@}lWd@VBzyk0oszaziQwMLF2 zx^6jIVaOiv@zh;B@p*vxU{Ksg#j^L}E6}VR%1?pvTULY>e-n!PzhPa#$hrXfgY!Lo zCE|XHJN0&|8T;Ip^04wVl7yIby|1kKkO)Opam!qTne`0u7=5y4(&s^rRJ25q~ZQLo8<6a_npV7r-P zpl)XlMu5MGf&cWG2=F=;`~`;kTms)YnwJ0i3_%x_iNnjACEq)A3z|hXNFDl2$ikQM zmOx(LYs&p_pzQ-sv;()X3~&^upG_?uU=dwrezc zZGS+=l1u===+&A9y)T>MivO@h!+>l}10Yzt?*RF&N_~4Nwga6~M&S0#r-0iTVxtdp z50jWZ{dkjM2lMx*pREylv`Es9u+flT=4cvEWa36Ok{g~=TQ*inJKt1yMPw`gzlz#L}xT+i&-Do{2~*;e*{?Z0rm zE2(cHxl>{DMAL|jDF1~+HZ@23FU;2bsjuRJ1^quIG|mdWBRcNv5lg}^?iY3{S(2bB zv>80;e?7b35;^V<``F!FOs*it6nhioHdMj{}NC={b-{_)KvLo|iGp;m)Z>khO>t?{&xR+Dg(iVV`h;Mu|W zpn;L*8xzxi6U-Q8=o@i8g=NZmaWNWnOBk!be$H$ZOzW-ggg=ujB}m+}1v$F8HygL0 zAT_IV#M)xnAD`8glWr6)vKRzbP;_qZR{CULO;b(=df2U`+w$|%tS&P;oDm;xlB7o` z;Y}sHwrl}LyPt@>@j~-+Ht$gUjJ@zU`LF7S3GPCeb5cK)oe`dz*23)U$dzk$XUHLH zACq18JP{_(t*(TN`F%A;j9a1vOITTtI-mF=j9lMbdE6O1^jJ0?Xy9aC2%lb*$l6bo zko`(}kXAyG2W?Z!Rxmn%=1^sxtoq+tS*qZ_cP8OKzm)LDdu`{hF#LOk@UMUzqacUK zPbOEp2q`Syr^o+Bggv6gULTBznGlp6{V<>Z6V3K)$=(;I8PrI+kqR?+JzkC}#4z^> z9{>iSRv87n?a?4s7s=5;x7%RE2NcLm3ZUi7#osQn<+; zOyHh>4L6X=7i%}R^Fkwh^T4-~!nH=Od_@5w}C@(yG2{*^ElO7YxkjB{6k)|^5oUrXp7kUgm7N{?t+ z3oyo?Bn0edJUJy2)VV*1b>1q}3KzZgiCBl%-cro1{up7!F<2k6dqC)^9qDQh#&OR( z!{LPyg>w`KscCV%&>Tvf4T*p{G;QSxO`}HRJ)M}mM`*$NE&}>V0BrRztTLWeIRw!P z#&geST(utbRTIqs&Y@+s$MZrlbn%`}Np4BrV-&oAfcHcq;UFOn z6;OSJj^R))$7i(%b=>o7HCMyqh4xr@EM=)tondA>VW;DT)==Uo!Y*E6d%E{@7h*f$ zw*xqzk{kpqjmg_pL~n%!YDEG=664a+d^u%X#TEJ!X5TMsK|gEIizxT}F}w`ZTD<7@ zD_fD`iw9EAK+Rp)2Yt#bCfawF*lRt`30h~8?135hP6Nw%gL$6!%!QSpMN5vts397wE<)Cz9(0G0q~e~sf~V?6il{X{IR4?jrpi>DKx$%!Dg z0i75i0_pbJX4Zx-VQmO&QCu`~I!cZw4u#7skR&T6qojKr%lCkJ#9n)afliMA-6afk zSVi&C6(vxzj)pwwmr6Hal$29tEJz-<3eDr~Cj%wF1Xs4=N(?&(=q~Pr*Pc$)*rgte zJHzuM-C*XK33HIHxugGWy3a6{qUgx6PU>v)&90l!kAs;mUXrJ@%J*RIsTt0e^KOi( zj;d_0NJjk(YL-#&}@khjl3ChhF(D#nm(#^t6{(s2+3lFH@ z3SE3fE?eR(eL8B1v+f{sTa>k`~G+wSj- z_clTIaTRW4SLn~0jEBxy_pXCD=j6$Qg`BtO$lD9RXPvui}xATJ!6+X)x z6=Z1vAA&5Ne3F5?8{z1DGZjB>SrdmQuzUY}c|5;KD{iKYAe7Jr2LH9}qO`YB4IUcq zFM=1E{3qD@6{}lyT=wcBE@ai&s{C8!cwhd_CgHSyQsi1C z2dqAdN-mo}KqpN`a_riL?edgZF`#|G>b~oGyg8M1PXVfsPn4PM9=*D4Dc{S6W#V?z zOBt|=tmiyubx7qz`$^8;2CK*NuGc$FFFMDUaiw>mcy8lxTML-{2PuRuZuvdt8&N47#kdT1lSgZBOLH2#Z4jnx}q*lW_k3G=cTLVSrbhT$ovX4SSHD<&hdZQf>sLR+fN_?mKdPnh^DNK z0A07Bpo0?IO)qDN#gvN^z7Mr?%3qM{2Gp1)ulvT8~lK7ueSmV0!Ys zo&>vJTey<9pVKnG9Nv>@+!s5%^=ZlbdG3q4H6I_vr%%dVbsI2qy5}+W4La`Wz7s82i*i@-`ly^SwLc zdVUL!`KG>nqVR3tfU)PcK1YRZJ-^N0zoXC5e7^6)I$;~+KhWoU00SSthcrtZZ$GOo ze2E8wa1Pw{5Dqa)cu%JRDfT5_Ox`zH;D{Jgl=C7KdwN(`xu&Jr?g6FF)`gtTqOh2( ztXGBWB0Siu^b@~4c~SD#$CDd`7~dn}8GeOBk(TFlL4R7B(bZpk1^{H2&{R5Bw|%${ z1m5Usm~d94@+*UQUyXZyq7E4&k%yjRe1)VTza3v=A>2M$0iW&7*K?~M)8VOsH8agT zc&FJtvTF|aa*bv1F79P-j(#s!nyo|8al#z<-hP4`?Bzz(8YFaGmCmH>uuOZ4HpjV4 z$CZ}oC}nx3*4`2aNeRrmC?&<&TjC)pkt=;%Ns70(=pZSHE43?0I(th3BQ8In@C(w$0Dvc07fB&Bkt zA1X@C+oQfGEsj*_mix1>N4$CWNrl2Ys~sgRTgbB&UeYHtBKyzE+7#VJXh;l0Z@ z9L={iys(Gwv)AMv-DnhwYI(0u$J@PHldzZfp3yMeJoLkBbJJ{(!h1ZvXW)B=?IC#A;d?)P?`InV?+N&xgYP-E zJK;SM-^bwl7~4(oo`mny@qN0j6F$$!=lQO@UD~}v^I+aK;v6`0uZb&9NijOD*XZ1% zwS#N9=kKEQ9pUM5nBKKxdOW6I(R4`HVfrr}(-Sa#SI6{3OyAToJqbFZ`|lmo^_X7T zF+F*3?TYzPX`LYLv8c3bAk7?=mI7%5qS8_!4OHD7_L|O=Jh+)YtVn_N0Q7~FVNUB% zopoltJVdiU$WPQf$dy}flWar3K_4L&kdy+(s(KN=@O^3?avO(e)%lf=OSV97;mUUa zEN-~d+EcO&_7;j3?E%wm@g~Q4sKbLADDRB74a%EYX%gzpGn*cAr^H$8qBgJ3zESWP zk|+1-3}(AeD;IL|TI=8)g=ftxx2+=8yNoM0-dhal-MhkDuTQSc7j*s2_6u=ld$6SN z=UBcsc=w_tqi~T)mB9D>7HpF@7;it3#P|7)KhVtgxs*5aN8ihB=ybdc9z4Rj&}^kFn1_bK1t1zKMvtjuX8;KMcgH+Z?$_Ypp!g5!G&pKwRq_ZmLos)@X0 zeuEcpeXH?_Ht3e)6HZLN1^7g9Vlm#;H+Xs1Hx-{yk@bzkC!D%`58)Fk7rvY;ZVbqL>@SDsr-(;PDia22PS*=C4Y_?(Vbg*9X7kDDR~3t#1+_P%;-GAdX#u#hm{ zXXMJ43s#abS}?xa-}~3Exaf6y!RAHkMbryc{j)&SlO&YkbH7A zMPliB?x-hTF;NavHc3;T@8K5soNDvLH4lN*(IGhF$&Ops9J zu-37_3`XLw8AgIJ^NJa?7=Y(JsZYK{Iogt2g~L>$-FrGcIY5a)_-<&y^@xz?f$VZ# z$(US;nJ(dS%&<=|Rf+SPpka9)m#K;uzJ)~yFYEvf0|czs4^0ED%fLZgh@pa}K`(c%3ymhqr@x4#IP@dFYf4&b%^S2pXMJoJK}<;gZt? zL|!OyjvmBa3MAtkJ_tP)IY$lvuvq|hAi%uOoPY!Zkb?o_5GXVhW7C;e&Mm=ovunx* zT=tj+BD6W1_RF4~NUk^>*fqE>j+pPptoe9$Ib}VAXV#ML0e`47*^Ac0N_I2Zvs7Wf z>*xSY;V$%rC)UdBWt^@?tbwH}d#%i30odz3iSD>e$*0Zk3ar*5)DWu=ANTfJgBBn7QQi`GK{i3MOVyJ)}~orPLc&qDe>_wy81KR1h(4FV83Po$Jg5N3>Zp;(TShbg}uEGswA+N$B>g zKl08Az+ej+v$&vT(3z}SlEd)0-V!)I+Pys3Xt7JpTlkKlBZD4t>#Fs%A~%(pggq2i36IH+n6=FqLqd`9A$G;yY~!q)V(BX^~pi{46>jRM&JhT8Nj%ixw(YlCzv*h-H}b> z<rBU!V%}_ zEF6BqMd!3ZFaU)k&{JRt!k~tcJHR=5AdE!kw4uIlTS4onkNsK*1QTg&uql z^j0184Pg&Tact;CkizjesblYe9>EK{;ZuT=CsE0hgw0L%Dw28wFYH1|t!+d}T?OM} zBak7_y+kh@g&A+H(a@-lu6NiL?_Vb4=o+xuti33m5cZp4c?}G6ULzbL`T%7>n!jJb z?m|p)J}e1dSjb8+{+%P~`B4e8KBT7WA>H{PH2G(faKU$s4)F8V`wmeUuI~+XzMEls zH8hbYcuZ)5r0$et*x~DmPOpR`lz1D<;D6RE9QCikpA9G+f%Z&y%d7C!q0@QnC5Z6J zG~Z|K%-w5beB`ykJHzZl+zIB0cT^6AMl-Uw59Sx{DZ0 z3!{r17fT>#$a%J6f8kD;6#+k_nOfbSW3NG1T~=W)fXfqhgQnU{)61GhqLf?2Gd96- zB}juXUF_R{H3rOr1MSDeTS_lU6vy@_EBW+7-+af!)z30#kauEy4KiyieTD+hIPh$| z#?cJWw4@L_>dDx_kwGMM=xNd=(B{FWG^~irHKn=MVF{?mu)|PiXmYN*4oAOG4dYl$ zcM?#1OmjM}l^^v^M;FBJ%4&wmKXhcrDAL^ed86!re(H1VYYBb*9C491JOko`^0&v? zH^dD0q5`5t{xDKU2E3eO7Vd|9g%r|qt5?vTh^f)yo+9zIk7@RbFo~~g4AOkAg;q8r z7T$WD+1}RK%8jT;V#3#vZr$&!!rOZ4XO1J-_1wlxEI?}>l< znmBg;06f<)3rD@gcs|?7*!SnSD84=|e^|&2i@DNVx+VD)GOqKY`REG+BpWd=pKD+} z*4~V}ElD;LJjG#-5$G4h(GL}7?%EM)w!7d%JbgevA*9sdhYZ_wl!(_tNq5J@e%8Tn zV?U6i1Gya36I|{iblhH=fHn5hLOON_%it<*^;?v0&A#vqagd=_$zZ1ppL@bHBtnJ? zCBqChX>JV9fUVr1WMDxk-0P!pSgT~nU^tXT>NVLrSyd)ZjB}#^$D8pCtqj8v} zWFW8UF8B0k940CmXg|lbW=u2=W|m=5G1Q+EUH>5Y;|!>~Uv%BQ<&P$)HzT^I=rk@8;D%!u#BH1ZFR}NF z^Mt+3maEMo9JG)Zd|2?Ac8T{jU6eqd+a(o(M83#RpxV(pw?*)Z?^EQHCt`=+DlH<(o-z|wsS^54I#5i>l7&$oDt4KGx4P7XK7-6o9KV{%}y`dS{vIOvnbBYitH z;qiE3y;{qaqFMWJuDt90fWliNsb=2H?3<5^f?OpAe!&Aa)8KaQc`RKaIGxVCnP>Pw zgHdRvGv-!rYkZ$dP&n&#efAj#Hy6GRpq?_aKp>3|F`w}>(PegF6>$&KYaga`vcmG^ z!be2q4vZ>hBvi^rD5wMvGE{o1{*2*siZRSD{VkTJ5S_2It8_NHXKC|?gmx%(F>mlWrz)5M!rH7f2vhv3Jv+$Gh0`#TzDAD@_ZiD?@a z;d`MH7=(lE#mSeLedh+P?8m%`-^|n5$S?l%kbiRu@5yf-&c0yeyg#79!fGw{^__+$ z+Cao$Ow%0!@?hNjY%|SF(;sXkpQGZ+M)J#E59#b=zPW=vBh+KAF;!?}gil>_2^X&w zau6zjX1f+0owko?l!O}V;DCjDFle^dbP`8C+pIbQzmM)`RDrr)OUwkk#kSGN0Dkke z8ayDspB4*w9`SL$QKpj7m50{@Xxb!irINU~?IM(6A5!FL_MfIkG?cG}wQ|zK)D2SL z4~7yKn-8=r*`NP0C&3MZ)c@-|7W;DaDt8M0mH%ERmx zR^b9>6<<0IvnbodeK+iUnpVga;<(jvv#V`)A;B^`Bf(4+z+g~ZRUUsh>5^=3U%@&4 zNjhhU$-aIexN15rAFdY9K^lvEqQkeTdI6mkFVkCHGoh0%%s?w1z4%xz41CaCT+7lh zp|dy=5(JNBaJ7|tP=w@kmOLOl89Vm1*oq#JsZE#~5cT1y7I#r6@ezm8daksSmc@{G z%vHFHxQ^{k7)3I-Cr&~Q;*$>QconUa)DlK6?7gWW|MnS_NqBxY4Q}yvu2wQ*9RD1)j2Bqh6F`bJSKWp1|;9PU{pNkNPokI zV>~QMVLCy%!<@1LzmmjKBk*7wCdhnUhzTAHLCigP1k+{4PL58`VClo7(|fb@zR~IZ zSo(F*>4R8$+wiFL9G3obc)HoWtdnT4hk})VsFhvgVz374s0VoLs5SDG-Gtp_-$?8< zHVZ~}O2;51T&aodrLr7(Z0c+fQ}$7Z=mm+R^3ZxGdkp-sxW)l!T;+a`nGOe#Cxy48 zxJ3E4in^aZD)x-*M|@mnno=@|gwr%(&9jX}!@I#=Nv#)RC*9`dDC*tc=_Vro( z+8e%J7x6VGz`7IK^J9*FkfX&Ko}-7HW4N4S5ac)+k>g4e%h6NLk%J*y!gD~I#lw^d zBj5P0D0bWKtO-_yh&6lIwD|)Whi74US0H36{PBIN=1=vE#D2Uksvjf#zzXQdZwBRf zwg095Tg6B9-_Q8y{-RwO)?YNy{pFtzbm%YZX>PX-#1K;>6IlQmOc!@kAL&880zTe9 zGMs{2exuHDhuk^7ExAhPkp0`Z$X8HS<3mw2Uih0rV}AwnGhX)4)#z1I6p;Kc#q=+Z zq%yl!=*fKb|0C{Q;G3$F$MK|1+W;*$JS~b+wMq+!ErK>uw1I@)KmtXqvWxhtqPv13 zBs|mtH3{^3_hNmpvMP%1va&u=7eo|D3$_Ij3L+w?R8+W^@F=ggl(zZJ%(*wY31#>D z{r!KR&)?5SntR@7&YU?jbLPyTX{AlrBj0V(SP0JodHuYG2v~N|uw9eD&NTr{1cbt6Mzd)$_DT?td1~Q_pQk;wRrmwDuokh}PCR8Ld4Dd@Bp?^4@j? z-h$kag~g-9XTP>hqnIA;180=;Xo*blxO^D41u!5B7Bmy1am30Pk0)U0ko@QZ4a9N> zdYrx( z*WCGx|EkYpV0d&c`ue}YCg(#4Y@{Ab~ zM|rFxIzBnC9uZHGo_Hookv^pzFl;5rVY$1B65zIxr=22dlCDydQm8Q*NQveN9r%G^ zOQKbX09mPu=NpYYc}DBW(-kcDBaQIf6O-H4K)Ekyhm(DD99jCYAwv46sr*Q@qa%I< z3*R^o=s4QlUy2`TCwl&U3_W*?(-=>YBA*-{2il*9cMRHUeLCV$H=NAs;qd@(e<}*_ zDhBYO4hFqu0(g^r;UodPzY6fwr#mFyy7$t%><*P*u*+&!m~kJfHU@JfhPG? zmiwl*00&}npJBOW+5&9W=ORoepNuR3#+(B9&$V9){vXW&_}_dH{OM2-N1KKx-FD+Z zgU2;I=`Q^H(D0no^2=s=`Ay~PN9J+BMZ|1aZfzk!Xfw>rMT@#CAq#`pRtrdSxIjc?^w|7m>h+z=gK`u}o# zP<_=XroZ^zB~)6ip1!2T_f@9yzUJ=U6FI>c`~@Mb3ooS>*hE*kN$84?H@fqZsbGpt zv@bLZ3GB(NdZB~#P)c%P`}pcQ#8se~6SNO)pYQG&V9oKKuojuT3ccOqrJhbG-Uo_L z5lt0BiuU{#&wUt$)@aOol65R3#_!Q1+QR!5SRl!}&(rPVcp!MY{*4ounmY#J6-zZQ zjIvO}C|=g_Qt>`iU4mGCQVzb-1SLkNQ;aO%+;ju?GOBGt@9O0xt$!9)$hW|ZJyKnBF57yK{kzj(I8`tiY$ zR7);=7Fk-(3u~Cc*3w6r`4*XXsqd8G?=to83R(?+KNDkb9_e=;Z+KAeFLxObj*w7= zYzy!m^z>COVI{K+b1Xzp8+h$Px)jV1}0=^6=qD@fxKw44xUiQN&o^(-wm#n-@1Q z82l{1oT)|}`Wo7k_hv z!l)dCO6YOM*TDd{(j8&(GV-6(4nvg-)i%cCa}m}53Ue=kZ+=94!W@sF4jG^vhDZOm3P*KUT;m@W&<4N9h9$R0WJr!FS`6 z!`l1a2#B7%Rt3>2WCRqjq0FfCOh%oyd6)xV|e1FvJ(&2RW$^XKc4~8 z(7l?z8$@XA|3F0U9I@bB%!>QFF+^C5$y^i>I^!+Zm&+hiIUUvOs+}Nf@MM27p2J;% z?7XVqhV8NpCkF5;>K`V)JgYAYY4fXV_)0R0qxgl8gJ<*s%tQwrZ_iYr`?r}YbSq{e zpI@&U#l_mgX_GxRr74WI80o&2PV6QBbzu(8-5=)ycVmIC@5eAy_(E(k&5gn_HD0ZP z9=Y_~12~(%Uk^~iAh+uKq%S?VhRv;H$}@?IDam1-KJho=?Bw?@Kd;FVfjZ{G!*Cle z&oUiDg3BXc{RBOzyMgW@@u?p@JEE<_1xnb@Jot&7DSuM&C5Hb#o7>24dgP?yMolQY zfxHx)Ld*u3x3{IJ^|y@Sc5~o(9%k6mL#PPn$4k z6EA!VR5c5)+0nY~T{1uT-X%z@FOccMx(h^F(eH#j{nJx893ikBzixyt7$fgs@!>=& zV;8B0OQiZ*eAPk=tz`y4o(5kdKYa7iu(%WlY23|Ac1tEdf}U0^&Y>#H zab-862w6>wOjjQt`w>qi?2uK(TO)A|&AV5CoYtH>Wiq%+KsHo(VLu{X>GMQ%mi)zl zgT4j1kB8Kau>1XHVr<{s0_|+X-sLxEG}BdGEc31Ebqmr-w?XYKC)-AGXd zxz2&Mp>WB45yon7SY^vLKwv*+Pr z2RSz(gr|70z+c(8wUusZkk2Vp+C+!JX2fTj``l9IYffRGP3)U#6Q&q!f+^Fu51(*~ zCf~>*rtTxlP2HTnDx=eS+WR@kjGR64#7X@iIs&~GyM%pYrsJ#UCi0bX0^hXHSD#pe zyTedZXkyF&>fwFO#+pJi`=*zP&51RIsr0@&fDce>3eiWV!L}ZwhuJ=-r(@_i2nN3H zcKFENZ2Pj6h`ak3FS(vK=LV;8j~5@C$~}>G4ZvzDH=_zevLq$J2dy(>T`gR;tpTH? z*sZ~t$LzwcQIhFud-g9j>yMu6Y(mg?-Z)Ay`Rr1G(bLNzRQiGmzViu%f=TowqtTg3 zbV;w6(f&o)m%BALn0r850-?qsnxPYJZT_ux<4)swYXi6TJZInRBvaP&PAUHjyV&0$ z+~4Z9)&bzW-%vo(+^sY*+qwdr>6Zpl@ocrN#~;d-cyz39Cm4`LeiQmoS|6qa(qtUg z-YdrY{~%`uzmZSAXiW@DPW16!gdrdmgJG^n|(xxeu2D&w;DX0v*a7z z_ycd<#w~tz}CyRsZZ*4-d&G-}N{|c9EZ9@Ji&c7aiF4+VB<`;2(JXG_Gy&1L| zQzpWP$s6N!@=1h_mgPkQ)#5R1TUQN*%N=#qj$H=%ra$3cDFzOF*i0x$FB3rUas9c2 zuzsw)+&tWO@8G_BA=;{Z^idYqRlUy%1n1%`BKD#%&V>Y&Ef&4mxTg2Yx3PC_Sgt^u z4|y@9 z2~@N(S_3~VJq6j4f}8eQegvZ z!`b8P;>sV1^IMK)VDiMjwMEpDADR)MNe^9@j7J2|+4tFe;j6gC{gNrY{uYc9-@=#l ztl*7xh;{{J80P;q1r5V+0=eAtpI~9uI1GQaI(+$y?yA@kn2TJIzb!>Qhh9uk*^85M z)0A*Tx7Y=m5d+lb0oZ2({S3Kd{O=u+y*>}OPqMM zoYzO?UOpmoJr;3tb9csMeF#(S#!G8fQ(GI@AlqROa<^C8!op@4YWWbRoz&}6Y{((pCZzL1ky!@`5bMLQv}t{y{RQw!vI{1HdgLy|m7J8P zO+lq*Bj-OzMt8LK-map%)+IFH+d_IdF0H9h4g99R(ni*bCHv&J;ip=G7YUQl0FI{f zrim)8pnj#l(4VEu3)2QH;VRiIkK$mq)n$hl5^-LMlAO4Kx*14iRvAZ1slx@yLoI%1 zibGPI6K+{?DUJZd&6~QzTub)xCB?NUOVDs&bN9R>5OhhIyXFsZo4%?p85(0pL`{Ci-? z*po|Ii7AfU<4#{qI@`D69J2#YP2q@57DQv*RyFIcx#1dlOBV8!rf zbpo8FUm39hyLEc)f{mhtGxMDfd-8z=!>#B*MXO}P?X@lg!B$9MV3;m0Mc{y(%D{|G z$_F`wiHf#smNAT@et{kbCXjSYh=~3nfKQ>{@}HIx*02@bjGB@pdF525(q9gL(K^0t z3lfu#{2TaMja*<#BKm{usYBL8KvsGI%c6(M{vOON;xaVv6z7}m!UB`L;6f{r6ONEB zEjR=TE_yy*|D2BC&MRpOE8SIlMC#9#g^gik;w(_V=R`1#=8Z_yGYm0 z)(4@T2V>f~A0Hz~v(2zjs~P=T7p26J4J<4V;(7?vTL>!WX;Y`Yn~S}ml)k=H*GmcZ zg1n3BZ=|G$4*K+>*4LT{q&GobBl7nOv?!4aoMLGvE^vf_;+;f1I)Zdcu)_mpJn|q# zd!3C2I5dQwP-^`vx*^m*0D|o^>bXVpL?$(tK>qJPwP?ff-ULnjmP4K_aTd?&nploa zSYe+o1c@;49xXDH=Y-poJDK?QFRI!fd;d~b^}z=y8ufcQsKeZ^o#)Ry$YR2teGwzB zE2G*}zD8J%yMoaX;4TBT2#N2Y-7N#SS>#f8PMp7 z)~J6vdGes6VP$1&^!!f#oJOM5Vp;`!``Gt7`u5dlh(svaE>;GyIev8`&T$QbNAF^F zJ_yT9K$X(orXt(1$ZB1Y$ylT_E3$x!{1c13r7Mz$MUF7XX!lT&AQoAvE7A*#Y=I*B zJzEFlgkj9%X!<7JW#+|^7T#Ci!24=J(iv${idn14)anUrb%CzcP4_{okH^^epkQ;f z8To^k@bpvKnvoYuewV@MQKV=dUls_Ol zwER$(83Gn1NEyN)MsZ=vO(0 zN`&qut-;Y2DwIZr-o!!yU7>0yqxI&4y6O&+`fONZl|T!y6!ncLJasYo)8aiOk= z7mIXN5%K9rknjj6zK3z*!UeoSmQ!v(Hrx$t_*TzV3#|rEA7@D$Z@j=;fADlAZZ}Jf z0bYUp>rJ%r@yhgw{N+|8$>?%)OIDPBjnaD^zuk?7Ix;biKi)Ql_~TbK`@}S55SAOA z7&~jUi+yv77PLJwo>=`xD{7td$VFK3I?n%UBKiok360#6S#Sa@y!}?@!9W@v#;Hs8 z6k~{E{Vp61d1wcU{21_I-V=O{V>R&-4%F|V=fwAc_uRgHfWcjIf|st`y%S$~B#A=? z;={~PZ`EGLS7UKn0ToBggbdFlhjotGd((Fp61 zM(17z&{53u&}tfI*-;~O>T5~i7QfAidNM2zu+%T`#$DlGBJz^3&z@Z`IAH3djPyVT zjUDUx>{`3AL2#@R9q;>^!!~r)c(9#m3-`keAwk)xSsKD2PO5O2oixHaP<5V)n#)HL zBI*4x-K3`Xj+XC<>4|E3k7&99(^FV_;C`lRUq`ZOO+hNX4*!ab`AgUFC9j}$x5)Wg zdUF1kpi60{5!fc-GfV~sA3dQLnrc!=k|{J1K||9{;0<}{&b=04j9D0&%Gw@jVSjqk znebqTUF5`d;eCs=wV`(}E}C@FXl5DzSEDiLMzg1_qtU!=jPD}(|Edd;FP}sf(z^G?#!;-b8!L0;_VC?F zSr;05P)q+rzo_JR0N<++>M1Y5deViAasfwGvap$SL_HbI5Vq0`BVH~7>8P4?yvQW9kwIfcDY<9MGw-(r(E-5)1`ZtmAVu7lKjleB4^ zy%0MM(##x;OQacXW5>q|Y7v>w^*&vTe~7gBHfdG;j60nPrDGhX=SIpsMdd1>Txc(p zV@0Sh9f2;xkzurr$5Fc9M};8@(JDM-C%)S(ejj9uz;5r2 zK>dn%GK^*ZoF(__+}$cj<*TUZYAWjaPNj?gx*mu3nr>u|+>Lh#(RTFX(|j>!K+CN) zj#g^7`UJMy3jL>SjhtT-;@c$`cwO#}y1-NIv*f0P^WiC*JFsVKohUR3-=f}EzV?Q8 z&G2Ud3fxoS+|kU-Iewq~DYRkTTsq!Wa>fS7OApN-z#Eu6 z)Ht+X+m5Hpe!OmvHzro^+iADsZG1G7oWq8Ws`{xgd)^Q`@N*Ntz}jpX&Auul=iDz_ z*brUn5G~~A^5XIRAv(T$|Lgdoo|$yx!*9(V8u*!;@Hu}!f6zNX#V+9t3`{OFLO&&E zcx&~nK9fLYX zW)cZ;n&i!^&TH#m!n60J87Y+H9S+F0?7CzF`BZ^@XNYh@^71>G3Ejh38>&n(k_YB= z)g|1N%}!r_Cc2;WOl1xzRb~uTT;3hT%4vxzJO%GQ`JJ$4Rs`q3J}#gd=JdQSIktNm zklg1^V%Yx8_#L5dX6WUZ_zOQ}Vl@oiuzr-T+M$WW2tO5$vF|4MppV~TLjD+}Yn+H) zuG^4B1+)^}U&6i+ii7ep7_N0QC)MF@Dfty~FLHlL__qb|kMpml-9&1v?{J2QgN^e3 z48p;^z6;PZdUNuX2E*K65@8IeTl?0)Oh<2m8ZK)+x2P)G%JYy0rP~2erH_ka&8Prd zOKUsXg5E+$;*vZC;-1yG5T-9uf>R8Xa%Tqqy7!nCySL3Rpy+X`Z zIZ@O^@mv1-EU8814-?wLx=pCN53F$nakyx?+NV`jDcUnFo)v)uwikU9LhhEL4$@M)xfab`t1^GPMF= zE$3a;jtpUKZW!J7{D8jvSH2n60P(O`U=kcbHWA?7`#}q6onzyR2 z8v&W-Gs5ec4R8-Mdbzk6^Wt!Na{17~RwAtTT%jQsK4$S;X-BKd7P~-7y^q?+w(>FP z7mw`DPHW#x&i{siDZqBU9Yc#*`_%X=Z>x6>o2k*UXyLgLHV9=N{h9TA=yH9}-bJC; z*}cYZK+V}=5p8FVa**tjb9;=O(huQuGM-G&97k|Ur46Pe>Fp78_Nz|e8(ui$5NZ%! zLh*Jv^j5?k!hesE?nb=V;I=yr8My~=f_#VXl=9>ZmglhX-P%V!YT76}Qi?;@gENLhIW-;#}geIr3L$_Q% zUeg{~E{wb;4_u+)8t0#BWDB$SM@Bgpq-!YVnWL=>y|&%kLhJG%-J)T59k=)}dz1gV zirhU{oYx3|i)X5zC)){99+%Tsip1$-wdmG2**xF{e!Sjp|9jnW8@t`U?PL*J$*Y`FEoI-7urxJg=5}tKWtD& zvwE3*J7iq4vC<#Hk#|3BS75PvLsX=tVhLF6I@Z$uD>_`&gJYmb!hf(QZ-e9R3|%0^ zuJY__qpSQBuCn8e=qkUhuJT*^*((37cjSEZm(r1-Tj^uXw9=1a!5L72R-B?nYAgP( zx=Po)5n1s&)D>UAPA1PC>WXhti+;bn;}y5F>~+g&#c%=IerOGc_UPArQ)Jy&Uf$ul zzw2b{K9jW~|G5#@J)%z!osF))LtX#NFS-8EmNMp|^;h>|=H(r)KUSKcTmL@EkY`($ zh9}b#P(odQC+Xdu=n`|k&=;4yo@u?e^yv^Qw(?Nq$M%ls^A2QxgeDy(9J?xt2{N7J z2RtP%aUdQklbvEIoR9k$8obz3E$J|L`b2Pm-hZ(-ash;pYxqFVH1(ao<@NI&aVNBI zz7BnPU%BXnge;dJd&*Tj@C}Et$Xvin z)}8~^fISHJKzfJ!?zuub=hU^oKfOa3Kg@l|a*vNI6j|IuF7=R4gaefEy+v+XRw--6?P89VhZ zf$l;t!?#;F=lM$o@f7gWTjmyqW-4=a{8z8b5R?%tk8Gpk@@1n5)%|92wxFSUF2Gp1 zoh`!;y*pY4tb4!#rMgAVZLtu<7f&}ty9E{lF!T{WO5~U6clo_j%^k%5@TBl!3i7LH zhkp^jzsaBz#9<|E7-=Y;&=0qD?L%}I+4su4)3O3~KTw}$Oj*~_a! zoH=s;0mG6nJB3owmeM;&`0{i?NhY*Od#`PAmYkIzIz>WQ=wDHO3} z7BEH}DrP9_poT8~)NXzJx!C=&ccS{8Gg(qa3Mv!6g+f03ohCivXwRsg z{*@s2VpVn>w=oD3r=!nrwx&Hc-g9tSvTnb^wL9g6ZBXgDc* zjz^zg-iiqm&FaM~`JN_zfrdDiozI+~d`|;$s!2dyyG!`9nc_M5@{C;IuY^l_2q>Ta za-Ll-Z@fd@la=3~Dq-#vCKmMAK{U@Z4hKGZv?iY7Tav&`G*?{Mn!u4uBLLxJ-4TQ+ zAFqEc3PO3jjZOvfl%(lLjYOv!b$rR-?vZ!Pf6-O$&MJ>$J#?Z%$O5l_RafYE0TVWs zesG=?0Uw-hC1z$N{DsQNtn&75Sb2kvT2kfJP&rhI^!m%Gar$(gG#3_LOy-4R8R0O1 zFUi4JwDS6iE$FQ}bB757y2`LZXt>m~poNa=){(fOW|?u5aIhS5ZdhJsp?5Dby(`i; z^#p6`CxM-HV?p(Fpq zFdAkD7+AtCacYqhyff5HuB?2xS6RH3TD+fqAJfr4h+~~goO{1poMR?o6`}7Griw4eD8XSb)AO<0B>_K7P*8^J>JF*LB$P>}|N=kC|;@uf-`A14HjHyQH*z+i~&I zexN_Y@K0`#Xt!K;JB3ya%IN^4Nq>xY|AflQaa@EW)+6R+x2ZLMKG{kJ?y*EhO6yJa zyz%u8j-aFN6`j8--T8&0?OFNjk6O`}$o;(d6g^PBmMN8_BHj>ljzwo#>Zlgtqv_+mg2woGAsxgkjlR#=BQKYuj$1HJYwm$i20J$!)kG9CK1IQ=+5!R?a zvPSIvjUZ!UBnfIKQjcZvi#8yi7HyXA1mARe3m+VjkCMwhl6App*-!LW|7&F_!eLC5>^OsoBv`PWnyT-C z%WrKhq_1Tckjy(Oj&~||GOSn3ZyX2JCc5CL@g2+ah2ePjuC{aIngsW%sodaZG{fN* zKb~YT)F88i;i=G#wQL8FV^gO&i#s@#%c_IIJD*DC{228H5=S6*yQIg8;A}t5bt_&0 z|CFwC3n$fce9xYwQL237ik zIW}K=k*z$%(+#@*P3?N{_NPAau6A2b!guis;8;1|Yh*8Zz@2@;DLq}p`@+4sMQHHq zl;&7?>lt2}WwCKD1)bKW0`8@1o4JAW|Auz2!hYCB&Zhk~-yzOr{ocim{@#t6d#{lk zzNa{SEk>wpbV}om_VPrguq99Mkdpv836PUuFCS&({5CZ!0kRSyD-p61Aq$w)>(#78 z$TC5e39?L(mB^y9VU`K9k{~Myvd|Z@iM>|EtR%=vhOA`BN`|Z?&i|d7l?+)akd*>i zDd<6&z1F}i9J?9A=s}hlT`F__H`OdNWOah9PLS0JvdrxC#jH+{)fuunLsng1V@ z$(e{rcY)+oNKS?1R7mdPxgC>JAvq0_(;ztwl2bidn4AX593*#z z~EcwGQcHhbPJWv%3t8;g|4%Qjh8Yn z8zY%|I8aR5eP$H*gsCUgF^=T2O%|uG%Hp=}nQGff9dZi3yMhOFJ)-5_hn~L`Z~1ooAxN&sm}g5@9*#B~ju( zSYi?+!a~kVro`u2VlpJc63$D3#M$#9Aq5g(;pUk!VLBw3Apw?aUMEbr2NF6#0xZ(J z&X_P35;{WyEX}+wm~bm3bb$m|lzFL`kPQi`kO0duFAWpYAt4PCU?Jvp#e@`Yt+Q+S zxUTN(wsBJaz|eF?&u~w}o~q#%p_&sn@~vHPGg{I{cER^nSv9ngVN+H?v0B=t+@je= zteI7ZjC?1xovb>jk2)hw@z&B1u%iMDqUIR@?H8{@*N?y=LdR9|Kid@` zB!e8aKr`IdRve72yf8d#m)m-b^Wn`m28_!T-K$)}Azv^lu+>&>%Jh!sMfd7_>-;Nu z>;CG59B$Evv|b~D12}2~#4At8w=VeAR&B~8Bf*aDW#UTD-_2XUg{Ed6!AVsoO@0mCAEVPU)?Dgs4jA`Q@Oh<6$~+2Q9WwBL0J%e zR|~KNpeyfd@58VQBVhH9vy*U~ImqWihsyPuK2`Wx)%#%(9UF`l=&$vg^3h@MF{MFs z=dC-R$@z86lUDQ!RCX*KZ2*N0JU+J&ui_mUVv3sGTOx@?>MytOBQ(@N)?903_!G*mAXkBI>mJUB*C@D( zIAb5xfkXIfZtW=8(qoN&5CH}PjnBW%3ukU1i{8&H&7ooZ0pEP&p=p160vhJhZ(g*Z zb7qJ-G49xJATbr1FgZDu0*3c;}&IGx44jqcT-(c zh+I02G~A6ryYy(CyW}`R0P<@++3L6H9=Fgq|mP-oUk57H*-|NZ)4@A*1A+i>%k%+!- zw`=$ZSb$q}rVB!b8uwG@oWCBD{J=1{(oiAQ4(mNKjVZ5t6ONAWLzKsWIBF}mXNE;W zRNhBT8u4&*`i>jPi~4e$)=25jETC!^TnrWM!t!)N#SK_K)OTuyqkJe~yijM)ma(oI z&zW1huwE7Db>@%4(jvl*Yw5vt1UI+ihvwKyraQ$lx}B8(B2? znsc~k>$)O+UQ<9wg`0gOtS-nzB?Azsj=*uQyrvJr#b(THj4rvlNcn@g0LR+QAat8( zyxj%EMCZVuIA41@26*NZ3oX7Xvv3(qW*vK=`KSs*m*lA9H+{36{l{A?xy8Gn zKBcJf$t`1uO?-~`^IF0P)&@hZ1f%Gp#0jQLu}ht>R;KH(3SJ0 zT^56@qCueJcH!_zyp)t_Os2b5Lo!2I&R#20whCtqT$4z1IFy2O=-JIyZi&o6hnfSs z5P|rgp=LW`NIYt$GwvaSp{9nDzH{+7scsC0nShcEm-Q$)z9j}gJl^FsCaEwwPCq~# zyYdPZK=)r3zr5la-2ZUW2suhZY-3!&-6={3JU!#+Kprxg3)yQeV>4pJqBGrbvdu`e zBzGNODc{aF?RT!zXoDS2Ie*YDu(z_DqXo+PYH~0-p|h!;k5GpiKCru>+{z2zdw)Xk z!<2R4f7K8Hzir*6;17arwbL4$J;Q}6lbt%)*AZEe_R;SC9tZ^PbM^26z|u;bKNDFA z7o+2_AnF=WeUa(|t)R-;L6?fY#!3g;LcZJ$1Argk0>gUXQ}6UPc1?Sx7jm<;woi@9 zE6s6;eR+zyXn*V(MeHSg)ODNCvqQuNfL;NZJ*FL<7!nyq^8!ML10WDPLZ%!Bx^BxrX6yoxT*+<<>G#imV+`x(L*fYv>wk?-Do!WB~-Jyg8(P2$!d#I z_aYPQq_P$?_vL(bTy)b|gu6^IX5#V1rO2og_X=P`Zk;hLAlMi*fH3s3Gcd>x9H+UV zXy6or7#^f8ibZVdFfm@@jdlhebVSD|R0np!?;Av-nS4o2AUHo4PlX)s0C5e0N3t8j z!s+amQ0mkQ9+rCHTw{(m1*T=)>pdw@8F49!B7xmp`QMUpJ_Fgzn@!yFFF*n+ff~{g zTPHCvFL@VE4QZ~Cxx47Xyp9;H&7OY9!vTvoBW$fJD9`o&4xiOF<6y{6ur_;tu`VTa zl3EbX!VaofZLA(0&XM@Ab=!gyyfn^Ooje$q@a14Wq}Q91 zv_m>E!XYb1&PJGMHPaZ!KC7Izvf*qzlMqUXFqA5jaF<0$78Cw+L*=Zw$Dz4}Edw}z z62}B8J)5VKA(dCpz-l6`c-j<8<4cI|1yb)ow^kOV?^(pYgDPQD`5j&)W?Ud{(B``U zgy!gDh$Sg9>4S zy?jju?YnBUG0(0Pz7Ac@?&rC+j;OdYQZfy6l2?QiM$Z5o(Q&u+(476iDud`5GnUuS z;#f%146R4EXVjkzv}_F{Z68EIN?yL9Odg|o3^+|{-mMK`RZLQjoMwz%!DnQ|p!{%( zo)a@+bjfyfQiLiU=j~%B}VDz;c%e%pe z1q2(mt*^Z|Y%L=0+@hhWcqvlUjkhkP(^Hzz9dAT%Oun!QGLj)9seD2b=O3bGB>BQgkdXoz$>kH0IX|anB>Te2kYR?5 zl=2BFoc~x4mXYEMr$9z0$S{{rFmwJbY7Mh546E>DXUOPOKA{tUUd`y_3wMHyE|Af= zd_rf=KVQx0>j8w?zQa+&zh+1k!7hf2-kSEh1Bei@&D(BBrGg5uwRLJNG8ENGc z(l~!FH6zUzPJ;{%GP;&e=*sy|tHbT;3v-ZRfefyE0>=e*su`Ht4N@)T6D;fTeXzKJ z7)dnF+E7QF6)tD?PHB9Du)zQeS*e($c}8Kwc1W%~ViGpgM))t^$pnFl52QwX;jY{w zKOJV&6#D*aMV`r(XXF!!ry>oogqi5XmvyajTVeo{ZNSA$<(@F1-v`2rZt&ArX|WFv z-~eXs6Qx?F90*bmtgdkfZ>%+Lvq}a!C{caOtu@=(c~#z38e_C!i2MDN&Kt|e8Kp4^ zlvKl#63WLVNMjNy>EA3Vv3y*jG{!_pZ?Pm(`8bm_CW(@ku%x8&aY@pcWJsF706vq; z$0bW+Qt)#Ie5RC-OOeKy@$&)rG?$MvOJh3W=N<6bseD`~X-sGQ90i}9%g1$=#&p5Y zq43$Id|Vf4Oe%g}2cN0s<5HzDY53V2KGS?vU2WF7@^NYNQ@FLT`lGvw0TQ)D#$QYJ zWQ>N$xG+_+qAy1vSZ#Qimu4ia8;El#|BgIwVTlrTEy@dx+>*gb$fIKN`hJ1X1YVkN zY&u%mk|flLOEI;wrC)ZPFPKxFtSiNsXLDQbNzB5q*y$Fce?R6ijbeo3ZitP4?!Ph({Z$k)w?_Wmo*^JrCNgMjM%D?ZIwIzT0zFDR{h$bXqpR1Q$A7{XWkLVuCeJV*DgQs>;IHn59}mI z9IKN#2b1KO47)Zle`-0q#PENYtVuH%E|e~1mOjeZuVz(RAlWUV6IhNNfW!l%!4Bn5 z4-y$*YTO(cO+AzOb>rYg4)w4mF7S^u^bAHC12~Wh%Gc~-($@3OkeY3cxlNVr#-R;i zD(=z(qo2%D&cD73Iufa= zJlU_nANF*zvEZ@UC|sgMYWo}+2iR7f?I3W%;b?36MX0k$MLEEv*a3ST#-KgBDiE{@ z!SdeT{c1H>W;oP}I`i453l@freS*IFc3eogx$tXfpn#W`BIe*cwE>wY{!Hr68vMB1SIb|IpiQXzWL z1sE6TAx8v}s8ok#m6OEM^@f5!?tH$EHy zQ9!Q0PfawkwCmu1A}are{;CZQ<^2m9lW&MH`8w&9R>yY&kuK>91-hNzph7^IeL=rc zcbk<(p(uN5VSFNN|H3gwaCV40hK z!ke7vJ&wENIJ-_Gsrshh;zrPVSo2UNrM-BRi2#f85)7(%VQ0K#RKnn+BEVs#{8(IT z!_P!pvumw28nxC)FU{K1B#M~!?pNa4n|LN7!#sON#>~QK=5aZi`B5k?^IxH8X6324 z%r8zwGw(hVmpS1~1aEV96Vl#z9!|b_;lbR?K|i7=tgZ4(vvv>Lv>dU7;VI+p*U`4* zphW&e&ff`6`20J$fcK1Ut+o3JeE-qG{RG`|i#h9}_Y>9J;(|`9s4Fxu@m3n0gBx$? zi^j(624M2{+R6^qt+L=8j*p!jM2e-N_S_)T=<&WPRDQV(#`4q@?M~)| z%JC^&Qygh|6<|&Wyz*`{y8+SXP9Ux#?Kv5+e#$BOJs*Gn`;_iISxk*C;m83Nd#W+k z7Z0`Et+6MXcAWjT53%0D6pxHZbaKj#*^kIVXya4^{DWz&XU<#_W`^;6BA>=%YC*csyLl0 zj!~u^R!ba?E3wI-3_79awOy)qF$^+)BNGMM8&rsvhZ>z*I+R3<6W#56bh{r`Qd{GWhT03H zz#3R>OMux{nyPK#9Hk+8dvFO=o1q@M@fDciu>Ucx#n6SggdbWWq!Hi_{|*NEBT*?k z6IZTReFv=P`BQPF4#x1!aPlX{C-jU@cs+j7@3h91XuB9IXSqAT$je}Jig`-kw&+MR zw2_X7k!rV2{wD40h80qt`8~G(&}umT7^|BWSjzg8s_zS~IC66sTRg!u*|&xoH0seTpt~w|8Mi8x zjp#~SNot8~3v5bxw=wd~E|e2E|6dz*Q~=5>YQzy#Dy0o1?8^JHlv}h+hUK8m4PMg+LCK2^5go8P-cDIdTyE%3&qY z-GY469@z9Dqpvy4**EiLTX3`#pUmD5RkEz7xPCW%qi+*hN~d&(C8l}Z0JP4kzeNZy z#b&A#457sxz>C}kHF|G>gR~yHy7Li@fi)-vZqM z)L8wVYJGGO$W!md^{gHYAW}86vyx!qq^~ecNH(7cSwBMGs~?7MK|>@U>e zxnhO~jd;IcZwEvgOUH4#gs^P>M!}o-(A(O3N%gfP!`I!OhiDa{*Lqln#F=IgH7sb? z;ysNHrGZrDqR1(i^}tzv1*DSR+unpx=XrG0wb>#&b}ID5EOBhJ(|641t*GS&(<)iS$ z?-`K(2xm5SrKX_e$=wlAf`J3PG$Uj>S1&!DG5`OLXI+s4(M)$pgyBpW>n^-=s^Ivo+Hu&xmG?(AMp<}!B3kagO_b$4 zUwOVWUv?hT7W^4C&W`%BLt%2~ecn93cUcRSe*S%$=J`%s0PEQ$|BDossj|6~&DY%7 zn}WQPBPVF*HUeARYx3SCpoklj49p_}N6wt)j@O*BQG88DJuhsd-cjmNM_^`shn|Hz ze9VFBj!w{+a9-D35H0Y6^FFRg9031Sk1`pe4QucyYcT+?3d1o_HEwwo1%&0+?&giV zpwY_jl3+hkLvufiR*&FYO>#!Y4H?y=P0Z_7MKL|+iMC%Xro?^=x5Ry;8^;E&G9`L` zHe}xt^#{QG8)`)=BJWM<8;DZ;dt#7YF!SSE)b^--?!$(~)B(0n1$Zh!^gu1ZFQ)c+ zUyYeIcNH8JvGf9tri}##_8uHCM_!0;6{NZ3-XH^LkbGYa-k^D}<)t)J7HR~B0|eit z@%}70ew5$L1To6-qf?k-(0M+HqaRxThqki**|$6k+FmboGa7u&7S3KZCE`~{EB|6_ z`Cn`QOZhv#R_E8wi?_o#6V!LS%GT>CsE9V6$O5wUVE%e^4*G=)%sIH~FJH{h%-x;a zNSAx^3Ae#8Vm1uI(<$Qh(k(ueiT;bcdz`|L7~Y&bw1oPw`o3l(=YL06%rnp?LQSct z-FK+XTaWE;L?K5WS)>*U1Rb3H9Jcr!Mj{M6L#OYLXu@9*Z3fQQXEMzR-V@XUMOZXI z3!$G`zq+^djBbd-bGPC|DD37-HL}R^SZ%SE0wqvbIgtHvq-y#{3MpjmPDF&>OlN zLc2QlkgForp9YH$!Hb0%==k6=dV@AG105NJi!cl^CVLg~kwi{5YB~RvWItIz6|UgL zDYYF}`kma?ps=rb=h$hCtr)y?#bC7JI?i7fS8?5?E3S)HT+R7!imSN#(iK;`APqUD zD;1Ojbo{JnaSC^9Rh$e`Ud1gw0mYAddn-ubiQj>8#nEyOt=z+AT&c@KPicNMq1>IB z(Sr6&DtW{y-jj*WhrB&P4%S{rOW_h{kq?qS(Ri*+a#lOuIL-!?X!M>`QPaCUf+erH z#pxDw^+VxT-QoiH2y;w~f|&z^EE>oH-;iM>!O@~EKwtq&3`P@!Tp*Vv)&@D>XmH0HRtW&rrlNf4r z32b%?u>a$|^hM||0|e)9#b_MTC;f4Z;v9C@<}5pmLw8wE&j~uq+Qa<-Sc?!gZlQq{ zsFYJtZRE5Dxy9GeM(*tb06u$|063o<>Y~H1pbK3))xC+S@IlMm+ldhy#CvuCqhjAd zezF2GEREAz$N4a_ic6>rb>gkNIG@o-wgUi)bc(+Yl=283e%4*QkT-J>3Qqm;m2T4y zykx2k5nMpd=)%*LReqhp17~n==09$f1#MT6)kz#};jM63kG9M<@xomeb-+MbkWiMq zi7`PU=MIclwZ7PB0Ib1T`YtDoADxBtU{0iV3-J@`lz)0dKV>K?jicg2FzTPRh8J39 z_ETC6(f66+SV*q~u8Z@PQA7K1(y;#yYw%2?hPS+dw^@!_*st{ckIhQN;TB3|nCCQB zyM!4Tb^&&jG%Aa)NhI_XMy0b=_jE;ak%idg7N%up*{Xqf7!k>om2DBb;7Y)KUzHK< z+Ihi92A~Km0KlSclnMX8Lu!^H_I{Rl8~au}=_+!PF~IJJWCMtQuAX{TYI~+0TI0%?#m-Sa5eI!txRE4w$zf}2|IqAf(MEnjG#(oQP zGUD=MzjglpNkTBo>qz(~C&opn4_Wm(!tqm#Yt2`lNCi>ZnRsWW#X0cKOrQiFAnVq> zNc1hUlJL_KJI;4sk2!D3HnVm@(9S+*q8;0@V>@^}SSeB=wQ~<^C+h59^C$OLqzTA& ztAtGW9lD>x@ntPY5T>p(Wbj;JRm}W=8oQ(qu^vve>~}^*PR#C3(GOzyAYd~XT|s~P zZsCMVS5Bm&iN4cXJ!d;Fa_C~ydWw6NwieKI+yiQh{u71`v@(z?+%9l@Ay{L3HI5Olnqf zl%^cjjc?YgERMnCO;JMPMkoHpuj~S}Cp?;=%50RAa`BYrs}bb?S72n7zwOr?la zoW`aQFp3-#uZ;$+j3plPv@%Df2mkIUK1Q3S@>DS(NnY?b^odq#HqZ$yIxL(You%d; z2wf5NuiZD(CHxA@=M=Vc{^JN)5jPVc4kp(1Tm_7A=@Ji_flOhv(2txA1UQ zh~zvN=XXf;dFd&jflp>(?(7u66c-3TytUaSJdic>Ca4PBIMzm-8m#uzHxoOU{Uw~} zS%^0nr4w7`t+lhhAx#BT^6qwurRi<~SqX;)2~lxJ`_UW}d1;e-kq>%nayN&bb**=k%?oHu+ zzc!O0=KBuz~Wmv#nA<&45#9OdDV#Z@gfu zar9&;F(nmU8Y9o8Re_`qyIcB zzEPgjQ+bOvyBiW}VbHT@x&#?!hGLcy%S_E#sxebx`o4R;67lNyG~LW0Ggadf<`Ofd z*rRmNEi_-m3NM<3p(#ePYd*w8AcxXOqjHqIcLINOH^UfZ1rf!sLI*mCe|1(aYYl5& z;$WDEP$yg?=YNK5kQUymK<{6VeEJs7Krm$J?8{K@o5AEh0RN~NstEoxuvepIP!WK` z%@P0T@6k!~G&xE+#tAAf4vm#>RJos_OlHY|^@q^Wzckj9`=ZB|Hh{ zn;{99+R<Aw=<2{FmG*`wIer39=3n?ev5N*3SFJBWpD|mqCo!HId#&$h&&RnPjX86G%nx~ zknYS*apTpWeH$GMPe6y~*s;*a1x$1-=zH;`%bphvhEO*~pK%F1AH#tl97B2k7!F~m zI)s1^YEMRo=d6F_wXnvHPLaqmY=VX7WTwhEE6T_85xx>C9mW=^7Z;<32zHisC!^;=dT*f$_OK z!k?*p58(S*&VM$__YBR{5MJeb{uPh#`EUeIjqp8j_^z@X5qzKe-`JiF4B>}nGK8N> z2p_}t#31|>w=f+x+;sE*0MAXCv3UMS7O{vbo~K@n=l$frJP&xD9;LS$|MQ^|CU70_ z9@hpC@?8`!s1e0Br|?}QUQnYdqXGvutn$kN1~l1R>ZtW}LLZqXxA=Gg>c;%l7@u3@ zkSn9`8%~vlHOpTvp&X}pUyf5M%pvLJKmy(I7`a7TjM}lU#ht~Y+y6mSe!<5jn*2iN zT+U2Qe%Zk{{S3!+!zJVwPSb}t#qmshDFjx%)+rQ%@Ul`j$Rzo0805mzS|C}z6^OIa zSm2zl;DyP+*4Q3JQN|@aJ&6}yW1iSwTS!OsV~FHAdL)OxMeMI{nZclL;!*dje`lzF z=Mq4LSy|dXn4d#26<%o2#AFN)#`(LcC(1kt3yn5oDv_U2c^+VRCgvgCaSG$iPG2?3 z$IxT}HB>I)zIE3kFMer$k_lTAotm)K9le&PNi54L&7th;Mcx9Cy-dN#PzZ5tD*3~JDwrtxKNSPdn&E!rOd zLTN&Uao{-VW;p+Q@I~CZE&LsY`NH2i2&?*_v29G&TNtCSSvaEX^8q+b+E(kX8D}tf zyKKIczW00BkLPGh?p2KoIDZS_%*_;3pxO{+63lXuNI6dNUf65n*j}4Ld#xSz8tUoQ zz1Bk~pIRF@|2GIIUnO8zAKiXSggoHvaK8m{zcuhp`(VFSs{5@G_uC2FZw<8HZjSD^ zyV!mk&9nu`8#{qFc50k1`q-0yeGXvsoFRJeg$(Xi#7DIh25XW_pJQ20X*6YDFFt93 zo-b1>G<=+8L0Xtbd*uh1fQpaWLGsLV8a_VwQ4}B1!|TQP$V)G=?LP{b=x~Gr6afX8 z2ozw}|62;69h>o(@A07N1?^k&#ru{i3;+6|qQxS*7f1xD3SC0;8U8)xmp1i%R4p&| zK`By>~Hi##$K>D7*)0k2p18b>YL!=)>D zXpqpD9fD6v!Fz{bB4JWVqAlXIy!onWXt=kjCDzg4w|quc*e)VvH%i z!X-WovenFVS2lVH0*SxA*=1~?;Bq5@KS8{7i={bHx^{N4>YknsGdM#0V~oU*^wHoK z!;hd8dMSRy%Fdoxe&i92AAug>>_V!9yC5*uh(}WpR=0sg=LC*^62*gt1+6J$7cxR2 zplKcy0eLioA9xo)=oS}-LYNP0UV7=98sRRG^66G))-bxLDTuwNN#Vubyf`j$O+zO8 z!Vn;%yfDLz{rR@=vG!5GPksO2_^DgenKKTF^P3TkVtA_0o#GrFPxT1#R9ka{q3ysQ zeg8k?kDfs+j{zsFD6RnMd>K}F z=r=+#kkOX`PE~LLFWYy()yC*^urcrDw43>k?x)=1^c+CO)p#?E3LV5Frv804fQyaZ zW=aw3ys83lF)jFh#tIii@eWP~^mB}6XkvGp=*qO9`(#z_JKtmWY-l|TeNshG?%}LZ zZ3XH5c@+Rd16ea~jxHFZ-pCD$SJ0Yi>vgf}ZyT)FH%{r9Z=n5C@e6he zhJp)*%__I4iaZn>ql&yrzq(W#b@52l^;+}y=z8S_l{eZiYQVb*(igRN6yJ(3JBFch z(H5AQDKg_imk?kkgAzu+@nmq>j)h#dyMoIuUcqHgS-Pftdm? z&0}`81RF94IM$$1oESHxM))-x3+}CTVv1r`2}(JDRJL%_9G&2(fWL{C-HKj zD^)#Llcxty?1A?T27^-T5tSzU9AQpT+Sb~_^>j3=pbVL3o z#@-32?Z=QK5B;6u=A{dptCK!jjU0_uQqymx36KoMpMyVS2=c-itRigojs=<5Lz%-? ztxht?kCIAe!~@uF>wdLRfERm>r$VJNRDP%oUv+u%gv}I-NGG3;xEkx7{f%VjZmXHV zT%B||{!%siG#;-EWzT&CWz*?h#jsQG8!cLaKST>7hws3!4@ZG zxP<$Gkkb7_I^M+t8={GXXONz<_kMRN;c%!fst7xP(J47S0t>IOYZm_2pGZBMSco$2knAs>gP*C$X{8EiQowXmg_ z5xpunqpmMokLn|^9y{6mX#X+Q72SnO<4Hx*Ei{z*^R8r%fIQdn)){8+N#0syLAw-y zED&7iRX}G_Sy6m0oB>Uc)~0JHx8f4@NiEZj7w^uZkw)uZ93O&T*SPK?t6eA zJ*?+PrzlgE*zdm{i~XJ*mtTD%_WSzNvERL;-(}%`;}Bid5GlNHX8z=|@TNUR1Gfaj z4U~m%w01Vkf1)hBptUokU4S&{`Gtw+Fb2-%Pm>J3^F~iE_`UB_^plyifs#ycgf`F% zgM)JR5R%17WnNL8Y>@NbR>BL18@xxU07VB&(v?x|7>51rlO!^8RdRLp%fi-iDCFL0 zLTlW=(f|ih%FURvq?J-`qLfrBbI3@O7)Nbh`9Tt7l;Gxn-kOLRH?WK??_&mr8Z8SC zW=WMFW6~{XFY$swlLMva>#EO%n|=Ut^1>Uav(jcT3P!0|Blodo;kn%ag#ImLW4w_@ z<)@Tq$&T1WqnZyX(9;p>32n8w=X)ez#xpGA@Q0Z3?{*wjDNDLQNpI0r#<07fqO!DI zcixnRucx^eP``tzmuxnlKJQ`8N2r&dD5F2iSn)1qjH6z%SkeYc8V*UJW$Jup(&*n$ zz;>^q>RvXYc?46A+I7(AXR!~jc3JplHu9F$m{&&a^rv=)vW)jA<56m77)$z;l8UjN zk0R|{!}3C_FwctZEP0dK$y7U_j0|e$I+n!!3zO2Yokt?=3}<Lm8%R99G zn0bH!-3r5DzOJQV(_t#gL+rN`w!AWl{ay--s<_y1#Al^|{Vs;ql%eeR)zGbybwRJa zn_40Ey6c|<@S&oRF zPQu&z8lu89AGeF zvL_Mf=M<&$;rMvgQ-(bisH^YkzMVbkqpI!Kg@NzJ!%wJ-@J@I>=ZNOOR`1VZktwj- z`@PfGnlgJ=*<3TaF7Vbmr70;$ZZnrD>iglJ(Q4+pYweS_ws`0Me$VIg{PXbn(43cj_FjAMwfA0Y@3p=wgz{wNVs9r~HDNJZg<6ek zEbCazsZ`9H=9#mRwc5;n zw$F34#}nxhy*JW6B?z}C927JUlY16zIIOxLJ!{1;K$G`gAjIu5=dLFuxQ3#nB2-4e z>HUt`8r>I#{+%{KH92ce4yD5lN~J{a>FV+HrnuEDejPbnsN+7zK~`%(fihU%qT~PU ztPS@jj`qsAed_|4HV{gMD%*W)EKYWnQel*pnk=;$xIwAWE0v8g`)S;w{VVNc;f8vt z!O2z?AQ4X1LmSVvq}0`|SMP?J**yru1-!Uq&Lmv@6Qf;siw*xyilsBD5!HKNjHoofo_g$gAWOFybM*SrMhFXt=Z*aW?FDkuQ7iL3hMKu;!$g$d zf3Y1E80-d+F3cG0vr#?y5_&J&8nG8BamY46&?NkOXCr(}E(UjPC!a*5mcCY7UW!s0 zs*LKJ68j)*r17Tp3lcwEzpOs^f6*_O{X6vw`P=oVO=qZITvLYWlh7)@kEN+q-VbO8 z7-kV$y@Tb2iU(qPGCc^lN8`9e)L%mNTCj~nFAog$*VDr5aaWc-?s>}e0+?M!D|h5$ zm%T`=S&@YY%)S+tV6h2vl|8K{_2<7tXP?@|_*&Zn50>QPkN8$`F`prdv*w@`5$dFx z+=}zWCDGFp3^vGpa%1!;#YLcB!E7z+8&Iax=Nb`l7WRlwhLP3A{t89lQ?im59v6)V9~SI|hs7s9M3S1v zbz5f%26u=aQAmUSTii43g@;y_!Xwv2*|%EIJTBGWz8H%5$|LLg2O>J%%8Pk^)P@)= zD{0)+M1V9K+EqH)31@DfR6lKw-FKJZ&I6|3GM88v2y<>!C%ga@79Mwxlrfl*bEA8{ z*-zhUak5|vXrYldpgz|cA+J&_Eh#`Mj&F@fbg%7uA%RZu$-8mw#2n53Pz7x~9N01` zP63SF4eBqRr}Gdb!1b6zsoY@q-4Q`3qXvu|YtL;ZGJMqnJ!e}Txu?~XR-B-mn`ClJ z7chn0w+b!~s7C`hCM*4XJ!^X(X;X_=Yu;5wnj?rksL$s8Nt_o-#iD$7MZ*7`I6e@$ zF!{&>mT@cF>SVic8TWLhYHZ=U9Ogoc_yS_Zh%dHT46s}vAgZS~Rc!F;Xu}3yxE!!- z%Tj$&_D{uz&*6FZR>3O1)hBv4&`pZF1{ZzkIEn4IvOWjn&@&6ZR-7$j=mJV zc<>8dUB~AuZJz8=EA|dIPs*&->_=bYVWvtLTMl;pW)vRCNX=JUte(As!WMG^d(<_4 z4K)p&UN8|1ZcETIL5O`*UL3S=F~mYbiOkB;WJ{-%=df~^UP7PHoH*?Ht(HxgTk1VA`%RCM7Vs zN-@D-v;;=+Etqp_Ksr3?Wck_ZNeyR$FY<7Fq0{Uv3os9^4?aB!B zP@+u2Z!d15dnkl65&AFsFP%lf5w~mBeVYS9n|iAc9nFJ$04XIC zJiVh_bu`HC6cA$VOF~j{_}=uN#xNV!BT9{lqW#_p6Px92K~L82HuH86veX67f(R9u zeK#7Rpks@t?}w=IrP%P94zc1fHgll&a&sUzFvqtJMTx<7saB;zw!r}NSd|$L!gyPW zdlsuXvX^%$js225b2m{26l%o1jeLz1QYZV~+jJUZL1Z8k_!l!D-Y#Zu{uic1s%?mK zcXA5~rp${_lj`rE2}S0yfHa%-tP}RPr#M)*`pJ7Z@d1xAWXeGSrg8j9$_QB7MyQ`X zh078qNvl6IN;kyr7Tvzikx6|0LVe>MS{T-STc9U_*L4$WA3k3f>1r=xl^s4`%V{)v zgMO7-bb2n&JkS)P4Sd)5cTy;aZJ2}R2>|k*;>C|}i|{a--U6?|Sfte+=J+i2+jo%{ zZAL8}yvFCngR@DtpO400K4VpuXIqt(FlnE%75?Cw4APr5+p-uWE?4Pd&uUAGJAE+UH`aR=z!uE~LDBZnxtdi-JP9@h0X-eg6Skw6p}{`=+93=bv(V|suqW@&d&kv4tRPEAb5(NyU|o*`$oNMq%w+)eH$Bo zV~Q(7e5+{GVpcR3WB+UvTqDG0{isI$SSeVlClZ3C)R$s&rN5!GvGRQ$WJ-MmW@>o$ zz5|!`52W|AsPEl662--ae3DU5$K)l9f2fJ+C!u-823&igVZDfkDtRKkVuOGpPd722 zdYXkIQxQ8#d=l;HQKG^-$#cxGr%M`Ls+ad1kw(tSlvt@AGH&Q}je`1zR+H2uwrlNz zZ=ne@;Q=en#6Adx5QdWV%0Y?7%k-c%y_e1debNLw+P~zs2?L(5dp!dN@$oeXLHBj9 z8NTTA1Pz5fGlF_F(}=A2K6EbviKSA_XsrTOwG+~^-3Za96pb!CX=g>^ieq+P{)Mt| zAoMs?gGGX9mlp|IZ#_dEl9!&<&IgrQ6JV0?^rxUJ#LPUL7~NMWk{;&OOQI9%`7onR zOK~j(K8@oqdHP4WPk5${a({c}s>~^?^zJWVD!sBQYf6>D{i$-h;mWEBQ>rrDyA&zo z$}-cGGUHr6Uqx}CAV>EbN?GQWrPe8B`shnaotf}pNWA+fRMZx33U(S7z2*<&!YONw z&Q`(tdQD7C(B&JsfGT@Cxm)L^4A-R&c7?2@1}zqW1kGA%DGMH*Y6sMm7Zy!Y`HFejdKsPiV=50-stpRSn9 z%|{>9TshRYZsIjU3yV;0()zfCFa>)P{CCrSN#BY4>PD@+@H3FekWj|wlQcfW0)B-?*e_B}pf5mf z>c16GF2ABy`E4|pkS*@dlT=^RIqDm{YMA6@vHE7IZ{+_^_02m+efLcoroK6Y>-)weUf-O5y}o%-`%pDI zJ67L3b=3`t_035rw;>e`5FjD|YM=LMzmTUJpU10vJ~OKAFyrWuTn2$`>~M3 zt9@gbljJY1Nx%rg2j70kk8+%>jSd>Tpb0wHqmXF_CEKRR)nsQFv~A!bZNueOTxT{kSm+ERSb`tIG zH9m`n&EAVr&ce}d`-pQB*C%oI!V2~RRMp5)YLlq4^f za>=-5qBhZ zt|17jBB%0~dafb3^0f@v`d4EVHe|1RX0v=TFp)_KHgS3#9GTGIeBdT>Bed~m0v+>#3_i@u$ zy_P?|2_Dbqk8wz<7x2fI!{c%~QNjI>_X$?%bUiQNZ+NriQ5a--nBY|sT1%H;SNbX+nExdE#c4d%m)*n-f;~L^CdA5 z0NiD0+QNg@993^v$D7^zv$KiowR#DXm=*pYt_N8DbQolh%!tP;vQpiP)B85KX@oZsq~C8-O07|YcN;Rw?WD7!+UVYd zW22Katf{XyVSNv%;~?+w>Lzoy+V?2&xKZswi^uoq#2&wNsp2}`_rFI!7vt@yah4?9T59{Pbd{$G|}eUS2|S8QpAQeya%jE&$lG4VGWO}-D~koZ_N_4 zsv-a3y7jW}#>^(%YB+)$vJv(9kIDT-oq;8>14Uk{vuQf@V_?(p z>v$f+PY$6)LLt=slv!5XA7KflZ^X3qe13hO=7dt{$k1_MlO`BCg?Q6^n(G%_)4U2F zorOm{`OWZ}7XF5J^Xn4=#&?V)G;IJQ zRouObn%477Lep+MlaLBMF4cUV>b1mFkDt-x(fE4NUSVh5cD4_3GJFqh)12}gB-Yz@ zf_wm%y`DbdqJy5TsbS;dCiUP-7*_>)(fbtjT~RifQmQZ1(c0YQTB}rUlbAjm)U|)C zb?1o9d!-=mXXskc+(=><4DT=Idsv_|Kui6Au;#OMNAMx&_ZVC0jK`lYu@Mp*OP}(5 z27j-Xti<&_{M#0^J4wKk435xDZ0Ck_D35yv1T+-5dCep=4lsfuwOy=?dU6-HRUv?U-ykx^kK&XFL&)~+Y@_aqbO4XpVEHOIBcF-s@4?a!U zDU8~)R?x;;x1{HQoVlxLCUZT8$KFq4U5_HqD`~xzWaFH%3=Q|DUhTtq%cwqDOS@ez zC#2ZLQ@ryu4TORm{Qk#)rFSD@{EuWZ8c#?Hv=zO|4M#G9MQ#tsyf7MhG+ zEGIrVWMxOW0X-CCQg^zLd^zYRE)nfyU@x!RhIw$!co_SfM0w2%S*N|NkVQ@HSg795 z?u6HDgZYJT)<=Raa-b6u4N8?R~b~Eru zv|l*>uKEzJrte{Oq{t^Qx4VyOsK$9wd+OovccCq+c?&dZ8M{}J^iGtj>mT7aBzh>? z4c1OwqutOZ@xzZKd~=_6gSAtuv>U9QI$gWL+No2u8?2q0r`=%f)GO)ckhzo3=pR2E zLN-A9hREl#1-tS1D|D+u40bhC0f(0@L+&7Ae)#$k{|{ZYyXx8|9^)L}l*Y1PYf{6NUdL z)G4<8tOPfWW|MOU+Pf?)&8(E(Cfm}5GZD#5^7M{gX|S?(Z9S1tdu?as#W% zP|7n{l~E}-vZ_p_Jd;&rDdkzLYJyTefmN9lC=Il3R+Y`la=3GkRvxs%ApY)!9-x8v zmATvDvmK|w9c`snJ_;+3!#K$e%B-#zCY@C)LXqw9{guJoRsPbzDKQf$S`r=W360HOEU2#`+iECim}OUis0+ zp!E%{T6n1g*2w7pA4wfMfl@2W1kLTq>y z!-xHZwYjrH+u>ys=o=%&hVKQ^JWOK4mjn&cios`#SXf2U${FVMGB!H(pT;s+BpZ5t;~HFG%vrr@RS_*I#oUKMkL}nE#S$r*l<@W zqR+1$u(P)5zKpY2Rlbj~iGL4usnpf?k921y)sLRJPS*FET;J9ALw)W+|Ly*taQoec zPJ-zOyRQh}t<6uEdzTmeQ+T0n4 zz~1laI$LCN|6TS~3GvU|+aY&{Z#i1C`JKW5G_ss~Kt}7M%68TG2FkbTblA-f$x3DW zbfu_34bkp^K6|>7Um!N8L(7_FcGAr4L5uEj8$CUO#QI3sv4OU1XiSHeK%RgW6`NC_ zQ9pG|soT_+uH)zwcm!|v1P!5maU21$d4+(Hjw}t`VgmsrbGu&XBkDtrKLU3@5xv)F zeT;ns93+xl!OH^f`2o1+L};gWeu!-@*nM+BX*>+|&khZ6ziw*$lT=PG$6+dm@@uha zZ^EWMEH;-3#5Qe3(6Fr*!(R6oXj2Tf)C>LL-X;6Wg!otP9d`CPG%mRsL;h4Bpb6^Z zzB2Te7`_4201Cd9wt7?XgjaV)c{SV`9bZ3^+K|lIVnYs<6#7%5 zzY>nQSo=Vl!&v_1iTgYxvX4*uWZ)Bh2+mUBa85~j4}q)0`0)*SHE6DTCd{tyLUzHez)aB2~M)- z^3=8HT>pS_WuoghB=aB!yNm-|R@^PxakN$QG)BN2nN^$63QY2pw z^-~IqopFJTXcKsl+648&B_MDlO%gta;ShlzSPR@oL!(gQZwA@-1k4yyqxGZo0WG~G zY!jQ2Q$mj169`R2){=H*+7?7UQR`oiL_)K0AA-Db>IbdYAatE$a4%8Q;A7jiZQHhO z+}O608{C-xSU0$_ZQHhO=S@AkTeVyJzBNfG-HGB{H^S}`#|R$muV#C3x6hE4~nOFE9qazY1NdygsEMxGb1-=Ab9No3cV9j z5WnT@CFs%oC6@>6u_Nh3c^DNS@vAO4mCoRw(SkdTg}S&IO0=wqzuC++j=XXras{B+ zrP9eCu`;Qzw&|r)+Ct>cYtwxWsx|!lYVig`xqk6s@Dduw(fr?EIREX8o2}IffrJkM znrqPEHv7C!#+_x@+0FDcMExshlJ}sv1wC}&*s_~KF)wl3E+qAP0WHYtqDN5BaY!)g znG_53kC$05rqzhNdvgv9i@%qxCCK?&3u8G`nfix#HXcXLUv?wZc zFAZ~ru8X8`RL@0=r}c@PNxp)n7n1v_qKuG{ri8e ze@~s@o|=-*32dEyCceK@k-vtKdm*_0vtB#_dq)Pg=%m)zPDHD4=rTg|O^ME0R>yV1 zcnK#BHRxbM6PCV6$nnbq#F|7x=qIs%wZOZ+SJRSX#%l;%f$Khm&fmM|^I zT?77c{DowTX(TNzq~3(w^Isi3S5yQyrpP8vCOg{7n+$O~H)h#9F>0b{JYi2hUCYCI zfUSd+HSZcr)=5s}i%K~X=9LXIf9;7+)KW=hEY>(g?+aGXG&JpTT=E2;h}IO*k5PnNZ za%EL2wTWi5kHWPYr9*=)&oq-@PBu{2-!V^ZNJo%Z0uS#ZiL}rd9uYuhAym?UK7&Bg z)I4g_)8Kp-V< zPkFJ=`bpCK+hvG>7et40#ktKP9OsbWSkMjF^Ha(2W6lq)omPSEOj>Nx;AGoEwFY zgM6re>5D>g79NCti}OJR)-I&)=py)PcA6WLN6?6I22MP)tcQpC42>WqAMGPjFdIJNbnm%Z9lJ+v5-Ctipy-?0U^+N3n-NdO z-$UL9_a$_OyBiLqPAjQtC4M+n_$}$C@wE7g`mmU1CuoSzzEGP7uN>_9%hM)2|dU7R@#S;ZES-WzV}YH@6?C)R~$oa32UKy zqIe${Iw)%v!BdtV(`x}YAeU%b>u&@UOVC;w5$jZMbx&F37+QPXoR#S~{I9C`ej*te zYk+8LW~Jzz-tjuW+ak|0%M0*h`K}i}KKOCrtiYPf%TR|fknO%?@t&{octjfbjV@5P zb`lByA7K0G?LK%Dh*zAG1>CE+98L1Q(sh2DJf*>_ii(mbNxwWK;oRy}G+hFKNTFaF z1Vdr$e#+qr&}%noH!v*Rg0mDt+95F|0xTn^f%_u72triTFYVO2F(PWWaA}?p1bGXF zGd&M6$mL%<_Wl}2HMI7msmfn$+{Rek^p#}O>>5v%5YE@(gfUZYpdBNYOxrX~AV@5z zL?0tMnJ4O{8{)@#hC?SFsGrr_DqBL`Re1j6{Q)sov__O45>Hi2Yo-do)j3LC@8vi3 zIR2eO^n@-6IjUKOdd!OX57O8<(jXbc5x~smi?Wb-}Q5sLfAM*tFw8 zz`d%d(x~Dz1yoclrsesK)Ulo|D-&1nyNDpeZ|fv&3#|v$I`e2rG}eG1m%CV5bRn?{ z6Ti{a4ZC0r9MptCxY?;wu}IF|4(ilD@B1T)Ub@&FF|xlxl0l4k)~b8;=}YcT z!}4?rQ#S5cL9Z>Hv)mk;>qC&(5lwb+n*q?dV$9+oNq+&hvwm8z-B>7Nu?t2FCF9)P~I+%%&(jbF>kGcfKYg{kuV0 z4Myv_R*W*K)lQHCv8e~jp)Z>S?~L?t5LSUMoCI0OM1<%>Xd8O2Kjk;rl-Axu2b8}> zS@8XNGW8q%wW_$Vr(BnN7A`U7U*H0*zpLBeczb?}O~CRNMTYrfKJY)Q_AkB)sbldS-rOkbNQx)IvNuK%9eH%7u^Evn>`RKmrgb8Rudl1F*ha%%RB!*bdd zYj})l`sA2xN_kup%0#K6bc9(t-x*_ELy$g?Be{;$Kr(Nm$VcB-`yk~2n|Ez<^TYa< zv`q(Dz94-xhdd3G*`#nExzOBh&+;jZij~2+GLYCXO=0WrOd8e0WA<%8Z;<4Foa=51 z{$*c$10A*xXrsCMsNO&nQw7s&rft0@w8F75g-p`7??`ZB zFTbn*gAIXc-EEw|nB^UDKpJv}AsnY>OI3Z*QrI=+1|R;9?MqiW^B+4b{tyjkoVvDF z4-a>I@W;)>w3Uk^HO88*bYRHY$(9atUrQk>_Hb`Si=m`;1Lw!-$M^UDaxyh-#VR8g z6cfmfG420Z_8-B{k*)h<{g2|+(f+W%H3jilLm*rC)p6*CFyCfMY?=Nrg-;;0mOtfL z&X>sYJ}A2%l7#~|YrUXDMd&%8XGz7^V{RTZS#yNb^2+-V9ag`ivcse}27Zi|MZe>3 z$6j7cER6V^g^Qu3u4oDC=#!UeQ(^i!Zh$yp(ATI;koT^kFw{HhATR6tC?8bOF0cLa zh1IaAk3(oN$mM>p_4k)Ul3OU!hs#7W^o#t?2L)0$boMsblmp0JR?qAI^WruOX`hpN z$N}a5SML7LC#4oGZ^4AVRr%=Jtk4UJ!Il554E#qyGyPd_AB5EoZgPPUS*uI*-xyr& z;Q!a1f3JAb*_Dph*@|6--TWx|>Hup3w%_4-gUQ^hwrtl*1zwWl>`Tcp!6om4PZPiR z!r=R$el_!g+SDB?TlJ<;I0ZBu@gi z8Y>(51c%vd%jQykaNKhB^+SHth8*IGN`25!`z$oYH@wqP*R$ngIYu}I#j*Li{L|aA zM9CyWq}?~a2mj5y_ic%Sx9=Qu8Fh^H`YTv8T0QK8v+B!}!kv874ay55fhu;Z`mDCO zUUgw<6@qaN&o&6Ic+!aJSp$9YDi^bTg?1oSi_?IDUk~g|XU(Nbs?=i|HU^M(Cz|qS z$=GJwTSc+FM4=D!R~#X<=+Do*Nb6#bpY`z|m3L1e&C3 z^&>tkM6}wSX=ZQ5b*C++uz=BHYNdLjX|!ckX|E>vV=ww~a_2 zZL1IK`xrY5vA>)4lwL-0;pKF~=Oxt^(3;w;8c=<3=uBPv!g4My3+hTl2DI(g^^4h# z(SG)Y_ubclYS7?7j!+{uF)Odts3IDSc55#`_OvTpZQ(yF;n6&Z>@8twCf^i?37c0=kUn2%#IsoBS45rDPss#%6VtAg|xoNf-kbO@T6?_wPh(K7J!&wV~-Jv zpE!cR7o}~ETJfDJBrG{v6|$&)s+3TNSOMwM`w7u?*27_~mF_;p)Rv&qRsyraKnJYO zw-B%S5Lt*z4o{mLZtjrhF#yz1+{CAK5yQ@AZFPHAU#Qc(ZHZboiHE!ZW*T9eth3)e zdWqozt1&9TDh=G*vYJxP8%pXFC%_ zbGsjX$fWUP0S0$Um78fs->N6U9N*WIkl!Krnn?7~;pKFT0zfYCf-ms0GyL{UO8FiL z#P-*$Dke=CQ?d+nF{(@JyALXk-3eDm-RL;DD|4fllWV0)8KT#~eaKt;OYrJ2Eq5t0 zGKEm+6Kgog;p{NrU~^&=sI6dMkdSAGXKOUz>*@PZ0r1&w%+q?{p+h-ctiTZCez&MY z8GrsGrCMMtC7xXfsiGcPubz*{o?TQc1D=0EZ34+N-Bm9;*UAAZOQOLC$Eqh*z$dG! z7=|vA$vc@9!d?h3+XdRvfi3f=GPUSFQphV_^Hu%Iy+zC@k!i~3$Rmtp0B@o!us+hA=nnaLhuy`Uo9%bCZ+-@Osa+A zv2&I4+h@_503iUomiO$|0@gUQ~q zU+#A|Pl5a7cWBb{{IxjgUi=piAT%i)ect3e+@Xv77Z|b{RelTOsu}h^Z_RuS`>JD= zsPzcY7yI0&>Rx-1z>luqh`16qyy5vNhtuHs!*<*LTM0vA~_@wa6+a##ek z#aDo_={Lu`2!-#dc2=Pa|4)`@B1fInEXj@Wa;#83^}L{lULb(?bIeQ0W0p4ttf?{F z3U@{mn_LI?f}HA-9g`=NfiDr-7>?ravPNHn#=mR{=SRSdK4{2Ij`!$r1AVWR1j3rP zG-K<<1=E|73ZH2l_HRB1_0;3l8#A3nEj%potzi%0rYu8em~8fem0<$w)$TW#<6n3Fp!8mo4Bmu1@T91iDv1{`KlSNp22 zy7vGBNiFv^1gJB&6;=4-$pe@IP@LtEr6#;<+DNanajq_&;Ej(ORg?nzx4aNlBeVDD zhr}ax@bDR+XH{uISGADG$OrvSI<_uIkNvs@#~uLo9$Ld=x4V^GRcM!f`1$^p8}LMD zVx(Oj@a=;7Yc8*6P9|7VX{qT)9DWkJyHH+Zc_<%Ka^IfGx=b4OZ|v!2iZidDIvH8h z&1SR7V&+~cm1xxCZEY;a15_24`;#)%&@lZi1xsix+Ox6V+p86!62s?LFwC87@iMbaOD$s5ccU^?W!~}F^H$MGR zMFn}2d`vEH(2h;`t%uYA%Ay<}XmBqB!c3 zoMOMT9MzW0x98m(B*>KJuf(FeuLWK?uW!H6n!}t-d!I@F|g$$ zyx-6-6KcT7F83oNRzR3V)(4MYWg1ooCRzZE{kmsh6EQS@MsK8(iP5j9Uip`g_4Gfn z`$@?fcDS;(OO}PBcC%CjWAoZCRT$OsC%;#!p$bVjuBVRjIW5KqaQ)YKyIG z1{ICDq1ssPd5~LWYCslEVc+dyy`2W1Z;&HNgnv+wkQFwF@^OytEFL0@)O5HISIKL0 zZd3}k;$bF%}JbXZZXLLXBd-qq?u_H?(Wq_Vy9_4d4e0FtN7x?oF^7W@PVgeF8-97DQQ%A|OTCqFrhuCZJ)Z z!5oNU?C2fC0xE=Hfe`n;sEIauex@{9ITPqK?(AFb* zeBu9Yy2s}mNg#TWGlFb^y-6O(%jWsW7G z6{8f68*vsOw;QZ-d02NxaSXs@mQ`X)ex{;RZ-3YLlX*&;dxq8?uxBnMs`8(StA2JOU*c8wKK|n@49o z?-B~k!jp`2N{nrUqxdeLj8^kb4?x?=LpUjB<-7rLymK>rUZZ#6rJU>dzv@5rF$wxV ztCC&GjIL|II@Ga9jSNp_$^1mrk&bweYOO8(|9T*f5c~gSYqLkO8n-8^c96i~JK-hF z2}*;QahnI6bG|Pgg)#j*k@~!eXrg#qrm~SIz*^Ey;TwT+YJbCFlY;waQqvJs@;QrS zPgbm~#dT=0%%3&xwqbwXD2`SV`+;W!&Qk7k3v9nx-IUj5m?+M))?%mpfbI2QQGng* z906=Ww^mcmw7O~@^&W{}u8k3l+G+L_5f3os^IF0IU#@aOSmJ;adA}E}eTf<9cHkF6 zJwea3030wznwo7-vPo>gc_61Xlf8eBRSGs`sAUyu;VC8eUc^s9Dha7cfRrEPp3x)j z7_|a(oxI!+2tPqRfB05Quh%PV{nJ?5(+Jmb0=>%h{EYvtv_3gY8Z#Jm4@k->+obJsJhA$ur-_Iq_Q_lT~-y|t1C6~ov8JN zsq}@ok)9k-ISRSkIs2tdUt!xbgZ&MNnd6{>n|d3Zcmf#}9?3de3D&avPA$EpUYrBW zIX+m)qhlyfKOcynwx!o~-F{dnxh2ez9eKivhmJw7wokUnvu^POjTHWRGd|o*Sn`hMbtXIOl6+aUn9y4Qugf z+N0{E`;oVb1b_}F)m@zeA`foA7!NelleftQ0o&Z~$1&#xj|7m9M~-;+o>PVcHfwA} zonLtG47lB-HF3FC+whl>-~58pEewAgxM*~5hKfE=J$cRo2Gq%x`=N|qPCr%-B2>56 zHS(NXtX;p>Bc6cQZMPUVn-gp+DyLNFX1$a|mw$?49%F1~s~jiJT~C;|)=~At7ETol zg-c3sIP#T$dMtr;GZpli2q^)xv(BdC0xTK2S~SdjdAo0gduvvmlt_ee-*@krmqM0z66J_N|N|(smH7>g?pd3-BVUc-6G`7_)88{1h9(Eaf^L{KXp;XaA(RN+^8g zwJcM!PA@BVEp#jmM88(`a&f}wqU-)}7j${D=Oh4;w%8uoWB6tzbz}3&T^5V{iczAt zjrAJCJ?Py;UEJ6>@+vI@q-H3n*0uow6?_ZW#gk4`6y z&BKD`K;-RjkCw1|EGkadG7SCJEQ~Hld88pTV|p}41!0rvy}Ot%<;xm)1WkoA&R?d` z8u#mLD^}E`w9SQO8#8jv*oF@$&9J~_5PkuQM3l(-5%Ys~)J5X@4Ug5Ep&{XVm?BoO zHa{+P&iX9YC26>f!RdGPHY#o*hJZw=5@>vDHbTVEGe9Ko+j!{05Em6kKTA+F2f>C(!;dTt zM=}Nz`5fZ1!_Vqn1peF8#TQN<@EiVf1obaB=hnzO^LcmJOuI3YK|+9R^w31D=yN%u zv^5XU#^*;V-sC-K?JadspA3;%GX?3s1_`0WWYVhv>Cp8EGVT^i!i^~rv`1NCzO%|q zW^B-;t7p(J6R_el)Fk$GZAKn+o1WO=h3}CVfhS6V>L~s%)xCY6>NtU~R)ptp!tp8L z_%6=Ig{#IB2(b7#aC&ZL7^?yN(ZxhF(9(||G_QLOQ9BMx+3PDR8YKkrD)co{S7B$a z;b{XGzXi|VpiJfRlh42elQXKnsW#;6ga$P&)NeWGG;c@zWWueiZ}O!+t=>dwA=y@6 zSTEU}XM)x6%Dm`D;Uv(#O!i+CFHlC1n#GUFe*aF9A zicX*0CRklp?SWPlY3n)OHAsdJA=A%6+YemF_Vie<9DjS?DF2c#kbp|Ms;KWc?f!t_ z<-2WJ&7OaFa#??-3xoTMEA=7fW%dyzhZ*75$dMfP!bcP6oIwMaDg#O^5MLDU#v2Ne zXAF3TKKDhVn&5p=qwM4xKxibhAVygeFxWvegE<1SBdwU&PzAe=DE8J2l3xx2jByOt zWL4)~=P@np6T{-9Gz&WI*GPR_r~$~^hQE_>ne5+;Br8H6)&)lwclK^*Wv;Ix_t7Wt zarPrF*yjQBW6X69ba|{Fr1<`Ta zTt_!9%$8p&cUhY++U z=2i(9#Ji}oc%PP6DQ(U1-`*MQ9kynT958m^?nB$g&y#Xg7EmMWU_nd@dnTjmph-7H zD83(Fd3v+JA)*f-OX3CipxnJfPcy>at0IN+vk6dl)B8?Sj?j4gU6e(IuWQgQlpm6@ z=3qIDf?(mKPT{>A3%!`>2s(Me*|~R9^xf{9o8p;R=X7b1+6iHmK56G7;1k~Ko*DhA zqZUeU_VOBYp z;}h5a{mW~7-a1fXX%HOrp4dG5=Yo`5F?n>^RJVVnIO~qh)9!)q_H;>>=Ida7<*|sl zymR`>z8ZbGN$jQE$qv6$!()k(mJnPm*Wu?)5++%zZPQS?Jfcpg3%SiFL$E!2ty|ytgin;=_OTC&b8*Rdo^-Js zFhSS?8GrSQ{|{P5r`F^&R_e?s5MbJiq;LbHmRsKzKn~UhN6Kq-vU0YmP}|BT{xrGJ z&$J?*k@LjEMO z&g-B>{Y8=rh#@YSyPmDTR!!W`F^1Kn@Ljl88$^a+DQ5CX&4BFn$VlKaeR|%czrvbp zH89bXo=?Z4IR)PwvJahVJ8Dwh`^7~!b5k?5+V6J&h$I}ThbOPLdEp9>3iI|^(Di^t zdRm1l$Ttwn+MosY2$W|tD`he>FTRt|Ts{o75?kcB61C^0BueH$co&;5Jy#9NjG3KK zBhwvMm#?sG3u6vbimUqs?iK_Ym?+LF!W?ORRrg8WaSSq`1}KT{53x>rnWzO!El(Gn zcbX6ZV1CA?Q0P9(!!o z6L!IG9+27^P$UDpBfk}i>h-0;^cx3a8x z&7VEGj8)SOEpb2krG*xMnFU0>IOnqe?3yCsMVFQ`2QUVBJ^{UrGC3=TJ$uKDRxCM( zrk2<6P46AEt$ihfGN_woczKu<@XJy(?B>dgds08{P1$K?+sa-)v!1j`3Ox_pR>7H0 z;!4CZW2cO*)A0&h*+9Hj@q2srdoyASS+jzOFS@~#VUEb4j>R-Wpib_$_cK9Yolfb` zOg6TX8pH#%{4_I@mEw1e0rkIhm05xi3@1 zs&5Idd}|7&6|ei%*QnY%P&$Uc6cNm3 z2l4ucc!>nFnZ@2S@zUPn})XC#S3^l^#EIrHNgi3tdY_Z>~|xC_(mzp z&CN4)&=+ zB{QrCc*!AnEsS3ddMV^U#6I(zA{3UYo`yF<78Eg~`S-UOb!47B+N7V|sst{E%Lf^; zXcAI=e9%)t(gVTJU*=?p^+n5;V{ag89_J~QJDAlLQvT96?rCdYam4WD3D2d>qd>@o ziUQl6;IXT4|DLJ@i!aS2@v#7N(&yIg#EyNFw)7 zu<7vuw?#M(aWi|UGF_{{7s3+lgT%yj>^}!qRs1fM&Ze;w)HEtB5Zg61ew0+IS&z#2 z8_<=Y8BM>)Gzj%TxWC3h$%WWE0>8nywK8&-2{AJIL8CHWOfo#*TJ%^ZR5DVfhP{z? zyf92cfSX;2vPzu-LNrgJ1mykm-7HiMO^mx^(3`p#fpeWCokOPNqMA=82yL5U^ldw67ESDpR@yZp|nzFvMWsLg(Ucy(hkP0p3Q3pgOKFFR0#B7){OzulXjLt zGcICd-5jMKQuf7s&Gj<}Ow&<{ABiCMU$m?iR;#BcUp6FNvB&GHWYZe<2e*R4(deCJ zhl|_Zuw^^`@~o7gO0~vt##1XF3Qvkz*c7zbPLwU~akJVSdF3JU8+S(TVDNW2bq;3{ z%SAz4FP(3Lp7yf)?YDNGBpesT{4@UC&z?Jx{$}TG{(6YyO^UuK`S|;<9sS0N-7epg zJ~R0%94!>9ei&GpR|u^&^1dJDi5%}B#32h(>--4=Yg#cu21|ug&2zxNRP2SG>C`D^`!{TNQa-nD7mVT=(%LS&m|Sp|l^o*eNiocmdeU^D6-syzzHFIDos&F)lq5GXZ2 zzR`Zrd5=*P2tPXR`l%1wZu9_Bwri5-G105?!;n7~HX={q|U2K)Y1fpp9ZI?0sXu&kOn~OL&>9K~*i<<~I2--GY+TN5{r|5M&ML z(kOqi-NE;!ZnVvj%BvfR9--DuGig6{+)xu{svn3DJ!x4~e!tDTWDVR$NW_u*-9sDv?~iNVKk>s7iV51xV1jWuk$pXkwxe)LN^*5V|x>Ay-U?Bjm1P!?Oe?_$ljO4g28#2@iLi?;OCx6yE)nK=Vat57jI#bW z*ic!N&AF~jCe%1hGX1n#m-sXcjWu?kP>F2|A)9Jdg+y0=gh)_i#M*WH42-$V=$aFR z7&Oe_2POCr`}%knijj+(`LuKMlrwsp4&jJ9kuNt{+-FCR6&3aLZksg_j=#nwWIk=K zT3t_jKUQ^-?>>#J^a<9kqHcs~cOR(uBtF$l+l=0W3X4s$k?%Aw`DY#eB^bmG)%V|1 z*n~g_RVO@D~FUwI4t!R(~Z?Ej3%8A`Zif? zFmVB7{#*3kG)OqZ%~FnX+6z}^hEV`T$5ZvA3`ZCCo3Jea46l;;@Hz#nDHk=tG_?5eOlgKNk4F?+QEY8|QI4af#y>-pxpA4XLpTpgw zZ8HrAUwCksN?7Q>=qM_o!kk zL}fKr`6X4vt`C1k#;Vg7Yw&BW(0Ge=%_|(pJ~p?iT#V_gR#(!*tCN+pi%1TIk^u07 z7qkuwJ_K+g0}yiy>H&D4YJR9Fr1aTT8XEWoRBm0&*9Jkoo%VW?f~vU@xszCE28|Xs zFHxK3)yb&^89X_>^caG2ovn;0jfi2-eF@K)L))SmDUZ+XQw8N(+U$J6U4>Z0fa+wD zU6OCI4-V!5(T`514WBzo6^)FUASlk<>(QB0h#bi(_Svj&oY85>g6D48QlH?fIPA`f z>(nIXl8GjJVON^RN9W9J`8r1W9>s_{t45_PK%8gXg?t5j$LCgS8>nv!~OQV$>024x{&^J`Z<>;qU!RInVgo83>c#$kPKzqX0 zFr0M(QBv$OgTWNiWR}v-lp9MdtyJ@v!Ad)f<8)(CAX+znV&V@cBJ~6AKv0-S?Zl0$ z&4Wb%R~bV-$qYH%VXR8r^CYraCe7^yrLkb{qlUuVVp13gA4R3KlIrpg31F^1=7S#} zO04`Gnj|yF%Er9vk@swuLA?F`qNXPY$*)-!(Ia(*H>sG{YPLGo(Qd99-qCKMODc#_ zu9orPM&h4-X^&g9JH2Ndj3f68U&sk3*{q@B11H(c&vV#%!1uh-hFHS*lR;L5KHmWP zA=BjH&Ks+NQBIovR&YMql{(>HuPd5b%Kf&s-pSFw`ysVDJg)1y8pg^`gC~TVs6Z0irSnn)m)u27BrNBae zfaqYzH24HdEQ-uF=;ekPO3f%Hgl^zF|CNdjrmq|C(8pJluKg)Udaj?zBNpV*#fRgz zVoW8n!@nRca{z*829Y#A{g4Vj6!;j*d*K(|yUfe@V90>$sb^B7Su-qOhuhehZJK{f zzi_#BtwGHi=K_k%yr-ykb(uW@_0u41{i8UzFliLk@aZ6vh@WiXpNMqLsc=F-hQ^<&K$H^y%}FA^EMC5mG%XxG z`Sn%^QM;?akV7F!Q+%(JJn)V@VTnKmWlB+G?CWgbvZI>?xr~Pxk>b{KekIGqV7BVt z!Z*m+zH`Ubt#-?m<#zD&c0g{?4UvdUM^Ve5Y;Y-m_pd{+v~M@-jlwK{f|ZIz?#qo{ z<4~l6Me*7w=EHO7!=u7vRLKIjO2)0kt6fDv*u=6=8wY^=-a90av_g!#ZfZ^_Q)x(z z6XPR{@~#GBOGxiO5DjPBovcF!u{Lk6D8!fnY8%ia*!Ro6xfWFf$w1+wp|qbzxtOv& zjL4UBsI+@#EKX+?x#G3PZ4dZ-!0*UUV+niZ4@ri{zW6y+g;&I68ZeqG^vxNrE6GKD z$i=sN*0gMd z|`mc5OGl;Aypj-;KIrfd`peG zs3jBO-p8%`ieKxjJ^?mW&Y;c*P|3^RAaHHcKrRczG-z!io@)B^!W3)R^9G;RDVk)D z+?OoT9Z3fni$QW`(A|_`T6x0|Xn7NU*)(VJR!wXQ)NXgAJ&LLtXF&VomW-fZ|_@_{-6ek*57A2VpqF|yE!%CO<69D%9O;B8l@ckKfOF}f6wn;55Bq5 zF@5eDqTI#YJxNr5;;j3FYih#0LLuCi09{D=b@veBsr3tqUlS_Sf!15IAADg)C84v_ zt+_-CXBkT`55MCO1rmcb%{XufJ{%W?M(Cn)wD$N)L?tYK!3)cMm6ZwJ3N?3NQxqgC za*rg74^(O4H=cXPk+G~t79d2`vt2(;NE*{#&bo5M342wVYSf!J*^J*I@M7(OpKKd2 z54fsbz=Pzq9%h2F2~!#%UVu}WxMY*up@8PK*Sx|5W*x~9`FBwy;r`P`;7sp5GU0YSq(p+Im`Nvq=pcm}4Z^Z+!gb_aKRVoxxMCy-hjs~eTF7rY zbSN>7J+=z?bDA6wv?PG~#vOKvzuy{&mdn2{eTxn1k%gOAxPXo9R?}V5q?|WcdWwN7 z69s9^BS^~OTevS_-$F?hoct+Iv&xr}&L>$C1&`%9$*FJTvtG%0u;B9sGeAkPPYkRe zhq&*20Y1qYrw;T{?ifm^7O7{!_4nN*%^w)@k_Z}vlr zyRiV-lI!st#u*C1^Y}%vr|@n;Hyat#zsaTyp0Clb3EixU@@r8eQI#+=+1AjVHLkuB zPKKkmzDx(fOWqx<@xxHG)&u^C;hB=r-tkHO*&C&PLXTaB`@2PPQobjW&;= z2in(*#0x^tvITPg)@S;c2tW&FAs>nIyXnM=9d>KaMlP~}<_$Tf!rm@gY0WRk792=V z{_sME^Qm_4UVujBlou2kAbpxcWNL5^M(`t1`xV+48!-V|)cuJ@*TC}m{Tmq<+uEKc z>S1(Un2G#?a`*m}wKeYwKE}B(1DGGhIgiot*QE5g_JRE!@&-nh$97{S~VUB^)1EUO6C2mCaSp8&?4DSd)Wb@856R zedMsI4DlA&(Y=y_7=tc`4yea|7yU-84cR>EDf8S zdqZTeS?LYB_VU3QRHuqE_4FU3i)k@iFebGgph{XiPMuG?-Zkj{md|dU8+3TiMxSif z9gY{Bok6BC=RI$(-)I131VLDtJ9%OoKgDeN1 z)T1Zl+N(b(vaN?z>umOckEaTQV^(W{xu!E^WoAOUk$+&+MIYV#5Pi`dmFc(lV?rbK z5!c1RG09TVkpEmIqEwpigV5?;ZIa7fX#8APahA019v(qq^d#RIt-5ThOBOF$HY z&k9H13L&C^(%v-|dukgGgMsT%xxP&oDMZ>e=C@5EA`uF8I?Oy2{puwlS778UtJRVt z2{a!IccPRszRQCF?{zN65(tqSu%to4i&aCKkNqd=;KqAA% zhewsnLV-oJAt(@2$D-=aD&&Mx#MHP&BO|2y$V9+JiQo@qz&s&1QA1v==p>P8(`bao zPF2DZV14**o=`{0ZquRmldfyAC=h!NU>35BfARag&3}ej*W@sV-fAv#ug>hS3sXsq zIG&0BRqM&jBM`0v#f!V6l^YKuk6;JoGcl`q51fL*4e^%NdhNdazGKi(?FaSm_o&=Tw#%EqBSNdzVe7FdF zu;1tPCXRvCzNJ8VsmR*m58du>#&*YKatbnq$l*a7i&? zFkF+tDlD%9>Et0uhanGTCPr_0D%^q3&UHxBJ4m;VBfgO5bFkzv>4eBh>G~OSY!nEV z$~GH?seYIxH_UgP8?xs#4BUvZ+uO60L*sFTo1BSY*}1!!m6)(xzhJnYa`P z2)2nMm==yEwTffM#)cmCkgS|Da2|oDo`4ztA2bLHNYF~jJdt0wzcM*veSkGarvnqA z(*LMur=??IfqetbN23(Vpj(%*DgEjdwvQ4BJVTic?lz%fs_CN&0y~t2(IJxrE)ki0 zRo(tTp%Zh%`uXXRPxCnyP;$Xq7+5k=2|Bk}nbS%&RvBPX%uLJP7M?2UfhO?M;T0u! zKn!KPGVU|C^96-qsji<}Itxlt%M{}NT)Vh3`nsFAY5yK0)XHz&0J9~wM zLuy3=m6P}l0ry{IU1M}6QImdSPHfw@ZQGvMwllGfiEZ1N*tTuko89m1pWQ#zeY(2) zoI#Thn0Zo?_^lg~3dMK~sZe=1Bc`IU(-Xw*&&oY5h0E&l20WLT)v`+T+3aTKvj+!q$oPA$&-aXs`S{VHp9ia$k@U+H4|rv zhxcg5Z%dByta21N`8Gdx9E5OEkrjOybO~56mwM0RZK`l7UaCnqmbY+q`#dQUB%o?w zP!a-0gFyRmg(AFNqLFDRJ`B=HUB6CWU;DuB0_u%Mhngo{E&&g5BWvVey}5N8np<>P>pvblELO z((dxwzO?K&rS%_wKxFOtVE`nJqB;&_S3W+Dy5IRs8D7`BTBL7sHWy?O!b;Kxjqrk+ ziYI~N^XJp{s0&@rD{Kv@DHmqVBhv-U0TXIgR+MPF2_T;DT>;&Lyg*2`ajOd@)&(pP zkdMLNFfu7YibdE}W*P~D(qUalF=uxu^##Tkoqfk_N9aT4MP z(o>KV64H{<%I;^4;N2;eipaePjmCkNzzTtc?KzE@N*V1jAPx;mPK6^oNP}8}kkdu< zzhD@QV)_0FR-@`up;yo^R=2`oS(@clDJ%ERJ)}^gnp7EEs5{uRzp2oqDFP1_tBWSTom@_ zV;6XZkC9w^VTdA5bAv7x=ojzDr#KhnWXgSmK2ba^N{M_x7^XmFv{1yb0a)Bv^`6ED z#AcJ+NybSL7bQ6V62VC!oh%%bjadRzXN+T-CZYUR9y3wWX}AbS3f6%tQmI;aLvpow zoDo;%nI)k5lFuR713DpCz{#*A0Qn^;Qb~_9>$)Zy9t;D7QGGCBgrh$};& z4y*-rij&bPBF!)@%7f5oj58HId9Z(rsbi)~&k=v(9;hj{JfCIoGQ%83KNF4YG4$IT zGfe5K60->!!bprm3z{mkBa->wEK*?EKF3Sssj2E=O|o_}KW{%Ux|`dvoK~=73{qh#d=c^ zGlJ|X86g{S{NVrl1RY$PqXtr3ew9@6`w)=WA7Y4ysDQwgy2G_WPL2*ZYKky%otE=h zvz9|ia@XeXA@2p|3nK|(1_<`~hQ~v}_KZk?Q*)7Vjiex=AYfn(0Qv2IjvR9=DqDMl z5|dDHQP9!&?_yz~Oglc^G@88hL7?9OIg36ZAR!dp{h1!YpP<<3fmsK#@6|C=Z|3fO z17m_5{ZkevmCm6fZb(o!8jjG$WO$cO;d2AxhndcRuX8d@dN&%3M?e@y@Db1t^bse? z3@r57>xa%p%tAww1hj02Vc)h5#DXmYl@dG8OGB6h;=?(EjxtHu%ipV-3YwCEjDj%$ zWF7;DMlu5A!SIuBpBD)9#)c~kC=C6Rd?bLf41|JkgoK6}lD7x6g){dzAAA~x^#BHi zH7czzWGgJAoH2AIZ3JaS?tz4YBveQVDv!-kfL7=b?nvzimnwXTTN2AAhhHSt*{F0L zo~2F72tb8f;H70I0S2EF>0NQQlqZf2sb!GT-Js`QT1h>W1sueBXu{=8 z4kPuTD13zo{QXiVtVwwJt^qpQHXa+lqkg3ycRNO&h9A*~cD3rV3S#@L`4{Gg@s$HS z!4D!Zdysk0|&LP*K%dQD;VDKL%vG@ z_xK$rCT;ZoZZP3yLqvP2fu2A5-bzq?%-V0m7eDB1fQYnS$~p+7jt~!dTn`GU<$neb z-1*Pp(|T|9JeVJP1UdLW+Rk~;WpFXD)^dTa{}~)P1&AY85MkWTur5!~!Tgw1zO+|T zQY#v$o@K$sy6IRwFeaxjdEH_-F}**)veH)NLE@p1KC#4sL`ZOOmkJX8IaQStn0ME( zyCS2r`v%yMD}=fnBQ*QwkKP@ON9jl(t`H8GZV-`CDAZ`YLtN0fh!EJ!Dal)cav&_1 z7e)jjjO}Wp9d-8tYwZBlPx2l97_Hu9|;JiXg8n`)DO=%;kA{0-Xe&C3qRF2$ZE}japby zk_moa($a~#s1^KLFep~STtSNl zhQn}nFtQzC%Da2AkFP1zCIt12Q+~++(u!0@qE4f_7u4Aq2uxJF;f|4&st(JdT?_p2 zX#u;V{_m%*REQ^th3s|oa6`wg2vnTSgQfq)UlOt&Qa%QlC!UQQQ+-d?9!6GcU0*&& zLl6u$Q7uFt3O{VO5o1pgmb`)yBO0>~j*hN&d3WaIbdStU%z>WCh=GO>u8F}MiV+;I z0pX4Xav5|dY%4+;ToL68wwD!*!wuaGDHJvzvSl)@&wNyZEGjJu=PuPA0x&gCPe4J? z{1+xj@DnL!Nl-)x?*eOQwl9=HPf6n@NbR=BxxYYT)KJSsaNWxUnlp%W7)1sjzxXeC zU&iQ5m4b6?MNn)MhgI+HnDI$mC^1;c$*bD-n6CzRaHLRdPC0hU#$;l+f$QBM__|1VticyoqN4QYP zEe~ZR4KN2Z0Q~ayY2tx#dJJ#y%O4oP3?u`n#qVVU^h-g#!aljdv&RUj!a9q997#bf zn=U%l0L==J^K9&BJ??Pb;Udl4$iyfBT5hNvIWY+Wpz{e^ToD6yhwSD+nz1nkP}U3v zO(lbYRW($RkpcwAlG(z>9xA4I`)(JG_MoNY6Q_|9ICTO5nq8dUk4mf%kl-}sD4&>|OFr*}0dUW} zvs48wEY#4nDD#3#egg>m5;%;~%oLQn9-Hwi>1f0Whd2-3^i;9*sd?IRK{%yRg_ZY1 zGnz_ES8yEXDKNzC10bu{=kpuQkc_y)u#4fnV6d8n>SPAyWR$&p{!jxImh09hn<;lW zA|oK<;-CrnnX2k_O5s{sX?>w^icG7H39_N$1S=>DiUfrbl*lCrAmy6HVH$FGw6?c- zIX_$s9NvOyVEOC?5@W+!ad-%JI9Yr12vd$G0YdsnmLg~3;YF)qe>b-btuxOI^n+$8 zz}Ejeph}x-WO0A%Z|_MURHfn&DrQf!9KcBcRv%318)Qziul?7=K|~jbCquLYG~uKG zTB}+DH}_bNmTRxK!~qfnldb?6#N2ZRCihs5-bx}V4+l9^qbPQ{caF3tg%{&WlW#&EkHT{j{?iv}FX~q&Vykzrsz09br0XbQS-=m#McHGx zY!X){8C+CF0~Pmd`vevZLIiBDH^3WMMYKIG%*_9`679P;?pBfZGcWxd|H1G@Q%WBo zwitE{d}C!S#6ur4T|97)^a6`_7Y&7l?6ikDmzFRcHw4Qw%|^7xQZ2?K?D-q{XDjw! zr0KXVjDfbzv=M2Sd;RGi#Woyg3QV2;qcE_mS}V@PG)v2NMD>Bi7N;Ooh$WBQhILH3_QP~d4 zxUk6%%s=Ax>?ln@*jy0Q5aA9s zMf7l>SBPi#F9jlS8NC|ebMYar%+E9FH7x$hV>}qC%{>8?GgmM+hQ$(tG6X^tgqrx1 zhGAN$uOp$bc=z2y@jo!?Ps{x^hHUHdJP!Or|;qw;FM`in+Yg)?(pULEUx7Y-eokY4wtTwxxQ0J;P!DqOV=0-zlf*ND$se+Kvr70Rk6u+_a_Wbdct|c-@1)7i0 z2&uS_SqsPWr%sS8H6>IEL{2a6>p}{Z<5T{dfN72!fhn+n%}k?EwI%E`*P3Db@?G`1 z-;?S$>uWcl%%NlckC={PCQf|RX&D8R2KO;D{BRHT%o$v5w7i*34fXl&D+JTZwejBt zAwR-rmUnU-c}GISB5@x+5ajSe@VmS=DT|fUA$hC)LP5a z^O~M9>1XUmabq2*Mn*boUq{oB^!W?4w+W9Po4$D}{8jPq4z`7yuO98;o;bb?#IJF@ z%H-|?emOmEpUiK4wr<-umiYai(Gaz_@=5$}roWX^ZvsQEV&kpe?kHFM_4`&O@)w{0 zQ~~G%R2423Y}JfH*N;C)L<&eH6O1uh!nwEHT7`>~(XuiV(K1o8vXjv=(UTJs(GyV> zn5y%!p&A92w3sPTWUzbmMRQ^HO1pxV(Na`XJ3@Xq*~<$#ev^-!>M<($;&Y}JzsjBK!@Y(I?!~aP*|e3gk$L5jU{Ql}-TJOti9T!moWn3Ict1_QdBR z6olxrxzz@u8_pph&7=o=0zJW2Ukru3g}HhPBN38$@i-)w@^nS{HnD%u=azCO!!1fdVBHklI}>cuWrE_7^E8?%-|~Sloz1uynwD(K&sI9~?<4_H%8_GcjuB(< zfS2T(##>bQ2*FG<@=GrkPDN#*4|D}p)W57x%jHfOv2WreKJrzTs^6)t7;R{D{Ujm- z4iaCcmR{;=#Amc%;LS6bRFqX;U`qMqqo*1@lS=^2DJPh~=g@yNkqet|aK0GQOBTAgw9-mt>6{=91t;TjqAHUCf+=Oz@ zV61q!>c(8U#X68ZNK#V`>Q$=2ZnofQ`5i-pO!+8D)~bU*sm)oFjvYs;3uQxvy?8AM zNPe+WByGJ&jH3xE2)3tzQXU8lC`#j_&d#?7=6TP}m&BjP%BRM#(4H69gsQ}t%yNl= z-6|7H;NzC7L(Z|l4JAi9at=I}c%GU^iieDu_)i(MQH+LLuCv55@CqwF8df-x-$jhs z0o;Lhr~&tRqCX*AEx;WZ5-bju07L-JpAldx{K2uq-(0$Z?*E#Wf2Ugj#OuFW0H6gp z0m#MBl81o}fZW;TzB{emzkxE~sLD~htPfKkQzPcsS;oGSlfHktQtS8j{+SR>8zgvE z1$`Y9Ew)G6`v?r`2j;jH1ac6BLXn$K?Kd5O7$gN7Jbi;89UdAW1;AqBaa!(!t9oCG zjEunUsXo^5`uStRn_DQv|M}ZGctE<`LlS?QSoMIQqDmqBYV@x{Kb2Qx^CA&VZvB6BCHmYArF7wBt% zx}+km;7(-%-Q0nW`NHs5)7&5*tob8FOaP!I(3qLo0NZ} z0%P290K`w?trNy!F-BSmTjku`EQ_tCh7odFP2uiXOM%pju1^~>rS9B z4w}=Ed%Z>czGE<~E=ED<%wq7}6HH-_?ED?6e7@qV^7VD7N}U5Elpqk)j}LOg$U&3P zj15-&tH{m05N5e?1d>ITi7GozHrpm(eC0q=ESVSF9`&X_;CunKIcMpmOaerO6?4&t zdba(76%!%`adq+JMoS8^QtZxA`^T!`2L;Z<#Ky6rvMbqDa|>6Na<=9mOvao6%O7 z8`_ZE$+=$TR+qwjbPs5-Axvj6@+DP^v0AytkrAb54r16fju&Q0-4S?SP#+-USuUXYOfm7Ev zE;2ifO<3e)ib~r1wwbNid-$$VE_q4yJg>|K^)|;;k|#%^>7YW+&3aS}bnseY0)EzMDAj|z;8Aw@)&nnS?QDJrRuX34AumJ&v)M=d&-L7?i= z6gMl@ENx)0Bqw9ySF_29wCrK64=R7lQweMj*D$#x`>8l@C6(cQvSy8lg|3c}%4r^- zDMJ!^S9nwq@>mis;b+t3YR3Y@lFA!!IVsQXfu{I%0ACGn$Z4M(p~Fy0;n%pQwxY^W zihYvQpL?YXq$~Pbp(0-6uUCY7k`lC1DMPbi`A1!IPQjJ}hOd)l(cb4oIuwUD=~evE zgZhAHGIAUfAg4l4Hw}aU?GLpRtm*Jd2Y3Yt0d+EiT?2*y3=lU{g{_?c0SbQ@z!{Lc z0qhCTK_5p}#CYN=0KOL)00Tf4;03?}t^r>_KfAyIuYl$P1o-@U08D@~fDzCOumOkx za4EzN`-(N`$F1MOmqDr7SLD3H;S)SB@ee9=n;`VHrC#Ci4qI>V3;pPO>{331*+0Ph zWMWrjYZnJD3CVJvG7U5s=}z=Fr8=)-J|~}Y)1(6LV4|9%QN%IQv+(=}6k4^7Sut>Y z2ZBYV272VvLm;O{j1>{7XCa7Z{@G?s`>Wz*cy578KFDjeg>6ke^R<%B*CNCX2>=~X z=n0~?hVIRSma@`wGo90*>ZUjThT1JDA9eIBtpL@&6*>5yziPmq=SG_phH=(4?KT*02_eJ-xGiV*aG?h`8hETXf2cgkiR#;17Q5~G7bRW zfb|}!L{F8FrE8P3UL(Vlb5aoCRwIea{n>PoDY z3Djl%=4URpo8y|j2BKKGsQe%ddW$i)ak-#m#B5}EAHROx!GKPO?=TXOadAPe7OReL z!)9SX9mWY~QI$Zm5y*y~z(E-+6l^AeXft8!1%cSWPTAsa^62tzaFvf3 z@**t|5g_F!hS0$Rh1|#U!W=1Y4mLhBKsh+?1PEW65RDay%143qAAsXFewV)ku$atq zGTxxmBd+7iTEBCt#39ev3ceenIeJc6#;&C>rJ9(e`5&vMP|DHPMpfxn!AA}XT|(4+ z5sp;FFl!KvY>|k4|ds1Qr(9e2cWUs$z%wg#Y%%{p+5)Zk;xJC1S9A z?~3{o2Cu^V@84b0KT@tKs&Rg;UQ@LMFZ3oI_3>3Me7fAR16O=5=;Gls>m>;{?Li{s zArZJSe9|ROL4UdzykQRr0X~Uo!6u`SXhaBs+2sOate%TVR&AbFS)Y$tAU!m5^KCeE z4k0*nTx0>5=qNcDhsXGsBIE>Ykn_tJ)Q-R80a;}TCx(DKAY+uT3?CV%FQV>Av787N z&DMZbDKk+y-YWAI0>+mv)r^bMt>#?&Vz!z%IOr)JXEHe4h~jFVz)aETK;IrE4PH1s z%|D0!DZ(3>E6PBiI^8(LT#ReG;1~qT=tZi215e&(zc&Yr2o*ivyowFmMz7r3%a+iT zn-0Z^CS6r}v2|YTkq9m);rQ%TweG!{T#!(!e0rsGN=5PAv8ydXGK$+m+JnVlD(Z6T zgg_#X6Zh`0VeDy z^8};;Rsh~VHY))}01^j5fd6C2vH-ZR8614oW_*>_oF%l=o}PW*Vc%Q3wVYe2DSb;8 zry_7L8@8Bp9Z*mSU0r8aA9L&P&(umJRj0QO)=6lKFgc~$FYWdi-PtegAn+;ZxvH_R z2yNBBVS1iT0WJA2R8;)sIP9ArP z_W;$s6Yi=j_0qAy&HPm&u2UfQg#bn5BP6zkCn7E35{xhaWC65UAw4Z2dIJS}93 zQ4{x9r3n(Z713$KQ)%~2>bnKB97*jg_MQBWiuo8Dip5yaY*&dIb%N6BayAO!#>yYyh7kpp9o_>cW2SxAS`@@=Oa={w?oLI*BT?cT zV5Uqx#y&bd^VxyAh<#GisRLH7E&Sr#E%2M~CN|{G$4`dNgc4D?*u!`g>q;FXr+|@# z>W3dGk3?Emf1rg;Czo!-vDw7aD@gBPqA^GpOC=)i#F&MMw6_hg6Jd<0BOxSi3IqzT zY%l1W<;R?x_Yq`y4L^;ez?mx9z{si4*QAM9${DN@0^R#y$kjmEkN1)GYoS8my}vp@ zuOn9-0k}((c{2~5GwIhM$TUJRB~|Hhw@!Bwd1YqK?&kq?&^-|+UtX^K;GZM&6QJkJ zxhcMc7L9!oGhmy@DQRVT%LYBsQw989l_FuE3z4d>frXKai;bkVPLjBq!4lUP_Zfe9H^&K+~%fp>xL2) zTu3aMLgCr9_A&EWmBC0a!7%vFhd{4YF#jq*60LQC6@x#<3-fgd{vnwtjC%nWMM1TJ zDb$0mDG~mZ+g2Ayb4tLH%l7_hfdSUC#bPN{oTHJ0v_?Zc1i7<9kv{ZB@VNwyyCpF{ z1+MyI2J#AF2J}T(YxWOW#@qn}!UayRjnh6W+g%hi3`4JuP9|||5GN1#O!=mex>5$s zgIt}B$#1PZ{=gdsgP(>4W`d0F#LEoFjBPzT4c>pIxB3sffQLUhQhxAd!~u{5`98F{ zG3-Cz{hb1&Pe61Kd}a}+f0k&)AASyb1WjRB*y_Q6%Kr6}+2d6NEK{V#->4^4%ji(M z(Jy}x$TcW-BQ%T@ro#v$!A9H-q<3^Ts0KQ?J1~YSmiA|80B>l>Eco;E6k7owShR^J zZ7SmPxuMKEH8aA*?xYU`5P7i2%Lx2a0)7I?eMRzuVDn%Ef^)n`_moxJeGspVl@f~6 zwH%aX0!IQ?b*P$nVYkWP@9Y?I12=n;y(&4Z?KSO_82puy6zNqB|4JqXAVwEb{4WByDj z&18cHgqk>sH?x(KM#^>q=y�X75xECg%1n|)adNh;7-fg4EBTQJ^`KgGWZQ6sAj z>nOn@RjNAEk;YMj-`d8%1RW`vC(QCPg zw`diJXNUEPKCYq{j}zlzs?m^Mj4J_s_#2?TLT1FYriTT@X`nf1nxj4bn^76ApM`)I zWYn0FDrH9WSxzw|6~C2aNWdGIs1K0At##IpRwPmElB#uPAU83bk;aAYlz?=Nah)0w zXhOJtXgrl9wta?gq_DzaX|W`S8~6)Eubrk4$;JipZfY*H@a9(;y9_}w zF1T!8yCR_-t>toWuDgxWL5V0tv3>58viuB_I4_7KDDUv#J0o%J1_H&eTpBHc8h68X zk>F6PrjvX28)UZfKmk~XX;>1K_kvh9%VErdKij>A5sY;z+0wHE@jbUu{(m?^dpW)l zTxbHm90(^6geZCyh0{G6S4Z^T4FgOSWI04`WRDcEG2dLD$9*2KoggpW6Fjd(t2R=(3NA6(xUy~ zrme|)bA@IbZm*u|!cSM_*KTwY>?Ld0q~-YAF|zEXIPo<&Cn7eQm1(l}JnL@AU>$YXPk?kGkp*6IO=IIeT zn%*zG1?%F;y&U}FQc_>!JzoNWW&fXK-A1-yCTjB&cdzke_{Ii3#8d#6(I zyq~Gid+-*Yj)zkH zn%WMYJ!Qf#oKdkx9!_@|oGzd1!T0IkMgB_3MksZj(+oY+r|lcR(Hibus5Qkd zas9K^{(2A!t(e7yX1j32L}c00{;19#{?{!&FJJX#$aep)TN@o2ncP&{2WP2lKdGW2 z!BNgV=j#4d&hca^VBUs%)6><$&cI{h^D6X={<3-4(YkQ9Ppe_WXu6}3>*X2neCcB| z)fth=!o}nBaah8rSC&UfHqWJtud&vczP^Z#ozqfRv{2PKlD{s?Y=xu!W^3oM*{^<-B@?##d1-jiF&zKe@`jD8 z=ln2GI9XLpnLLcGh3_%{IziV)VXYiJ*AyVC*nQoaE}!%HP&d+inzTqdXQnqDgq=!a zX5;B$C6SexsA<^MZMV6=@vo-#lZmtfT^qyd+<5y;3H^QYzOp=PoizyAX@0Dsd$_m_ z&&n*AqK6}eQk(pn_1mc+H8BMmod)BvLH&N}k)IU0KK3h2pPA4s-jR0R%E5J*ZIO&L zO4@GrIAD6RHwN8in zrvBwL?gO!p>_{KIg9+>FtS-*9JIY43mXphUe!u!8eJsvrlUu`dXcSdfK)sRMYGg~P z=Tk8m{+M4jXWK#TazqOI*6PEsF@tvfb;EM~z+m{1K^_&U!Vb(=m64 zZ2E7t?cd`d%EsHx)Q;R<^KJBk*I>_D9A$W!1OkH$& zYS0@xI~gjS4u&|Xy+atlw(%e>oZU z+YLXKD#qXFNlrz(b-yp#y;di6v6r=R=}2BpO|DYwVg+BtslNEGR@^N()mxuWObLa% z)$-ku9`8l9tvS{`Zyw@%)1f*x$*M4mTxu06ylc_EPMczvjBahA+V&REpK7jdiD^pFvGRu3CflV=>$LnX zoo2oBN%gb#x@b5qU8B_t)wb3!by~S~>}R>6b}QieWp~KHUTA+*h2~@B%TN6^FedMP z8+unCJr?G2`nL=Ip)FJ$S--qV(_pJ*5b`QeQX76MLa)>OD=Rw7i?fGX5$9Fgv1&1; z%NE+B#4gB&ep4Fu=a?b9^bFwb8YAh#570C;=%AOG)fHjJ7sX(Pls_TIXPo; z#hRVwAx}Qs>Qs9@&nD0t#Q69O@7ptBJ6}U${+ri|n_;u80bCu#k7iNuhu%>Y8lVjKH?a*^vai84v zy_+Ms^fi~;C3Wd%#lBr*|3$x9rq>(q3tw;kw%|7i;|5GqBDpssZ5H3tT{r8bQIHmX zdacz{X;G{w|B{!DyaoSA`CHjYO1Q0?7YEMflSOhz4~lORAE0VnB##vHe+ob91Ll{& zTakc#Gzn7P9>P$0c^a3kj$?HahB?fM!9eKJ?Nxm3`H?oz2N9%qt7S2RdX=^LcbYQS zZUs95()<5}xfZgV?E5f7J5(MP;BUJy8dnMjICY?;ATHy?BDGuDd~`bTLpPKHr{0Fe zKnFkwDfzOA<>cR^?VJaean-dj4(hyE{nprbEGG56Xs0 zxT>ujw|dhvO*)?e?s8c3DHE6w2(+v)OpvQ-6?$IZ3`hTd$GRQf84b&*ZjWJsNU1VToEHkw<)!H1u;8IS+Vx5>M zXOzJ-H3j@OvNW=^)|t!~YqBif$zd$;bYW%&InC2TkgDHt-|el^)zsnSBDYvkCF3<8 zm_{Gx;LQCJT*!r*9b`2A6h=xfA3;rdQJ-Iq(kF-U2V^w`yn|6m%f!M!?}&gFgS4as1rybue{J~y&f z!R>Ag@LSu%)M?d)rF~`Yh1rNK@(g&)^L5B$$T*aaI0Vhd1u|c-h+>uPoKjHPEeP8* zI`O6O*`@4++?R5;sVW5LSr{R)8>Sa1KeoMYL8 z+XOX$fOpj}2MvGJfQQqnnrOym=Pk^470L42m5OU84Lm^yHHkP&X9 zkERcmiVot&AF(toH?!cuo*1~#hR|9=?*v|19sjemFs8Iy4G!mXSt1KR4ljg{yF4Bf z7(TYK^mCR%Nn+P+$bF23ioceJB61CJW-RiC!<@=9%Obdhgi&Le=^Z+$A^0Dk$xCdif@SUT77p!M2D~V5WZm$SxU#f{7Pn#a04yq)`My^ zSCZ64Kb@P$c@mvpudnOu*LvnzyWj0GNR{m`jfLT<#nH{fl385j0-;#Lxg*K=Q26a8=(xCT)5QHQua}a+J|VRk z&#j0nL!|d1-iiCvHE=#y3;Ny9y``=$B9nSvjn{e=9E*j#W(|k@6@JKoN37cP3oX+l zgEy?lB3D*r3kV?ma=H3tUAN8LoKC^ZD9Ob(v405&8EW)ztL3Tq8UmR`0S6 z&slgr>|Q==2BNPq)jlPwC_m(T^)59dvXj1!lgptmB=Vl8D-)~U8gh@!1vzU^K5mZ_ z1C)JDEp|#fp=)@4T|{iH1NmE#F6Xw3V~raJ&*7nN`gKnai6aT=aD@0ZKG*DlW~c6_ zSMy$Y`7&SCc=1_8?6EHcKLlv~tw-+=gV))7>I#|q=k`m%tn1n`{Ig5Wm8U<1<>!MJ z9(a7Qx(kLfv-2&O!#7b&pAfNnm*-#07C>A&FobA_}hlvHlLy&E%EAWJ_p`qcy!s0j+!!NWLx zDZ9-MK89%HY4&Vkclizj+WJ+0_17eog+k7=FRD46hO_KJyNT)WF~H0UQ7q83x8*%N2sed8PmJ28XP29J4JV$KlH%zbq=iRB)-3XZh|L`@cES~hjDhE+g{w1mAeo3 z&Uv@7eMuMeTsvA2&ex_N$}DRjL|{`sova)0%9GEkazjg%M1 zm+Sqj|HMDrOMAn*wpwktu&#tpzx;PPSvULD`yaMS<^8pxX^a;OiRkkfMDOzgGfVxH z7?k3UpiJ+@z~mdQPglP--T)N9#{F?7rZD23J0Q zN*teB7M{zG;%w>9RGw-+tC3U%$sm=#yTizE8d_XvY!ps??B3HdSG;B={ZCJ!+|8KX z45O~!m&a*Y^7G{MPi*@8?!%Qoov@3`-d(p=$YU`+C{AV9nTP5vgmW5|Tf55~&g(@5 zvRGq1@o}eO!`Bb3HoepR0=?EAH3|3YudCdtUt0OpO$|?{C#ZS#kSjR(=;v?E)DMFU zM-6jA*5RE!y_zoN_9;EKmXVHJbdlVuUf&aDH73J~#f|Y`&PnX9Cb%<3ydRd#$5WRB z!MZwYvTZegF9D<@UAWgwy6D|rENU(ue3hM_zlb9Bq~W8p@4dBZ?{nE)Tqphd_;Oh` zxjQ8<2`}@r=x`KUS6EuL7W^LH3XwOxIrzHw$;Br>5VNh$Pg>XtjR&jpc`WXGyb4<` z5ZA=No)cKEGZ+)upXwu5m<>ek36*WE^SM-+yEG|KyV4?Qx91HWNC;Oicb~eEQ9sn;11b*m>0O&W(BWUne_re|2Z@KweqkZ&>RGZhYs^Y!bjG)U9BEYa(8-{p8s)??%8JwH+R$-9eX zuF`bdWVD)I-xxh5-Tq_aIYy7uXLY(+(FfNOs=339wcF&?FZhx$FbLOg_qMmNP0r+cLN^TWhM{++YxeAzxh zvy_FF+fQI9cvn^v9m@{$iqbX5Ynfe5Q@fL*Re0R4q{a!G``g~xl6yyc_cV;QZn=Y2 z-m}wPMq{PoO>As_K64wYVdz({{3GWQx5?~D4EqwRZo9Rz=h1oaI-DfCulw~Lm)X#7 z@eV-c2)_?mB)il4bRsgb5w{Plc`rZnef0fQBIWN1EaQpS1ZU&D2cRR5D*cX1zkiP^ zw%xPaM+Xga)52nCxYX8aHH{<=iy2{@vUh7u-YX{`koUj$-S>xG&u%I0S&1e3 zap>#sY4q*hU(J`qxxErYUIKnCf4=K=>!JPDbG}p3*#1W!gE&4+oA8LK{d-z&XyITk z%dKdb-gygKGwHXlLrTznR~RMj!_pr37 zNq^twU$e2;&_mGncPCXbBNZD=&lY>F4n%hlXS5NDEsc-Q{k^{u4f5>dLwrB} zG|i;-&D>VwzRi>>V%S_MTO$8uXBX*%x5$w58<8pvonvL$;p)xw-V9E|)qO0gY&pxr zR?x!bC+@(r`QRfq+6i$Yc8fiavEk7Aty?u2aLvRNp}UQraj zJkd>L%(X6eZftx!CH1l?CikFs>4WK^vUICar8l`P=^MWOg?X#KIz^My97rA$fxHga z<8uF3W&Z3`>06N%tVHdZHY3jNAXqngT(WfV$@Ocl$Hp|MNq)`evd4yJ+T0~H>%LM= zSNdw+WQ4Kd$WK4bQD?5}y~wj9M+4^+$yfCc)Mgj-l-#$M@a%jDKlg`=hsHzWKG-JS zC8s%}af6O`%h$h*y!FXxSq?UWw0MqL$B{n-vACw}_3G9bH5_UzKBu3v(yy0pC?4=w z=qrW}7r%-Zj+fq!pZZ3l|9c9w(-Jlay>~O?*>7P=u;0bo z8n`@FrXB^yyoJEAb3NcqS=P@SuwZkaP7AgRui>(7Syp^%@_!uHysA* zVICdD|4JaNmhLnUS}RP39?7@htFD(T$8oaB=_MmGb$?rXrP50c0YZ0HQ#3}?23hUe zn>5ZDcXu5`rFu*Fq)C^ovW%_h*d$=sviICa8EIG}mP1r^+dun!EPB|Exmc_H&SYVi zPh6u!yH*oDx>D%cnk>XeHu}hM3c4>j+#0;bQBp7VLH-S6pnq4s4x0OfE+2c@9HP+a z&)(gh4h$w{sWlKQvYstlQM4KG8VMpa-^`ch#F?}d>u`l%Cft(@j7-FnIA#Z*(GjgtIA(@ zMszUHl#kAtjdSN@)z|J^rC>5gJQZY9zFK+FxT+j$-w7#iTr#)`r# z__aG6YzLQy#7j_dFy>ZygiuzN2Ig$~nglQ3-0HR2dYSzYYfc2NRx8Y`vl}gm5M^sP zH-qDONDAMB9I7&oTVatqenoh!T1$pzrk4mqr*^o$5WzO`u{AC&|MSYhZD`@|u=u`j zTz>rfYWI)LPW|mllY%nse2w}z;BmQ$*9x1yd*8a_VkB(;_$A>$nB2qaliPW7iMYx{ zK;e2<9|_);a~Ai#{XXt^j&7(yNj=-L^uaXUg=_b}06jp$zp*-oXHi(vqhIkxW@g#= zl|_m88r-d#5*E~DX^((x<%e&2jG<5B7Bm1_rg zxV`IK?Rq;O%^jP!@W9OZfy)H z)t-YjA6;n@;54$R<*k5aU3=BuU+2=&R_*YW(KS0_oMwGmbbHU`wa6IO<@kTU&ai~5G319?EjhPe!tBI z$K7og9@#%H!l-?(iFNDFw<#$ZJTXd?;5pNMR&H>~)T3SJb8fCXvFi1B$0?iK(%2K7 zy@~}}S{(5k@>gC-L#sHc)2=SF9&WsIXz;c>wMDYq7v#-DGnWl~T70?xq1ua4au zV? z*nODWy653j_am%IHjldfz-Zmf?aL1LoEvR!FRC^)YwB>;sZ;%*yPWW`?de(A{ngLr zo*P@0J#H|}?97Doec7{J4y7zyXLq}##%{YD?vC1Kq1}!8&ntAXS$oVT%ZPFOc;Kyf z!(Od?!4Q>foNN1}_K8z-9*)00c*kSuwX@YG!qr1|^DSL@N1MNPUEh@7-zQ%7-KUhZ z-yvzOrC&&T()6&7W|P(g_qrB+MR<{aVb$fU3@eX;7ni(hKkM|)_`Bnpuk8?NlJ4KQ z*2@tEuZlXZy8L5Y_Sftu1C73POB>(6G1^$NG7+{o%T=Q;YOR zlcm)f&ynss8(HhhDP^6nd)*grv@c)Zrl8yL1~b=pi}~Bc;;74uCwprc4Kq1o5;>$< zQM$*77~j=aX^ANo+s_S-|9rOJclY7*Rs@F65zZ^R-Sq4C>$Tmlyva-Q4q37{@K)Lg zY{!S&9Y*IBHm^O!Y35?(d-ta+KPA8BKDaJ%T{4z=a9MxLhTA$l_1d(r_N2!yd!zmD zJZ>d3%Xs@#E_m}YV(9Ghkh?V=taC3vpPKo!^|wXNH!tscJ*nF}M(BKFo72OlxMUrf zF)``KFt=s=>U*+$Jg=Gzl+79%J&e73_PPhP+fJE0@W52rycxbmeMjf#jh|`yVW#-@ zZhVn>!x|QzMy-w?9$s_hphC}iNz;dy&T#tqI0g$|rw~@Xsk&RxeK$ zHlj`-Gu_tinqB{)r;qUy%6aQ{?A#ExzF)^qNWsuE$DQ1+F^+}%);|iH`%0> ze|Nzx8inne=sDQ<^X=f2O)e2{E@nJFwYu!+<_}vom2FyS6e~IE<*1nUNu2a-Yo`+B z=kS?Rr+2D1^l{+!%`H6!4CD9wv{Ax*-8SAQ-@oQueO>m%rA%AMosg-^_7M_GG_tu6FhFzBNn7 zMc1t#HrE9!{l?sK&Ed1<@QXcO9f|XM?L9bre)mC*`r6jE>=^mlXZbbvrI+e-Iqy4c z(Q>bZr>0T0Cv4m1dbf1%)u2h2hBU;_n0znm+PU!Ual26^tv)$_`qtHV$tQf@j|W%X zV!BQ-XJ*8e%;<9R756psm1xTf(KgT5-8Qukc(v~iZ`2yQ-tu0y8^yQo#7%NiUfOf! zmSs(==k?mS3~P0@LGF;HcfHO8J{&5GIX`Uw=xdA1ugV2?>xgQO9m~(T=+jKHWc?cN zP=4d~()MN9sZKW|r#|zR1Y6Ahxze49A>|yQCZanXN(|6^p zPY*2SnhmU%QuJZ#ZHrx#*7RvL!)JERq;2)Lui`&DR$@G^_8I5D9`$=vzty2^W&T_b z^Io1?T)%Fs({$dkeA~Qrla2>*=MHUDaMY0}&gnnVYaMg{`tPo6hdn)pS}t+MJ}tZL z-+lpeZ{0PCk6+r~zCU8@UlFcvCwR^r?P__h<%@UBjyChBKl@yKdfKOrKa+ZmnDDgB zD5-pWqpfEgE&U!Ta>mV#+YA={1fMPc_IH6S#y`q z;WGLCn7ytodYN-qDXt`k*UIl5`DXGsldpM^;uq~s7CugG)z|di*b^+@f<{{$(;jsX zF@D%__N1dP#wph2@mPw{C1#mT+V~!RZ(*X~9J%7KuKUSXKl&BT^=(^xG;!GVJu$^u z1IBS*r)*6Pm%g?MQ#5$&FePHh#8!_|ddPmH^yvPm&OV3N_pZE`@3P4)+TE^0ml}!t z&u*RY^*~L*-6tF8e7F$wI?IXukmKE@RM7Dw&t2-UqnmS+4?PBacsl=jVa>m<)*YKx z%*@Y8N9FS$?-kwnI;!J4>^%K>~?Zg)Bw8?MZufcogRyy-Wo7B ze8i+5WnUUD^m^--wOGU-HaTl+i)S^WAAGc8-8oxWu=q%n-&xM|gpSiYUVYC=-1=71 zyz%gZo%ju}HQ#C!vg3Z@(DF`eO=dp2G{$wHzscqm5#~)>y$v~Ad@$j>CA+kD_()gb z*w5c?g_O3vAWXSAWl+r#ZKm?pbrwZvL(9>{qVdm{%o|d(wKd1i>zvz&@HWz;pRBne8Y*t{l9ztkPQ>g3Ed!pDWJZ-|yFnoYB@ zd=!1budF!L+1MDWh^8xqr^NRqDR3Wx_*0kI6Xa$*qH4o2RE+);LpR zogmh%%y;B*+m6TD_seSYc2cB6ZId23i}MzFT}m;=;#f0|&0fE5vBGG5jqkk(e#Cl=Q|pZpuj4mcDY{-OY8&!-ONfu>lDQXyZ7g^P zo|kn!VO$(|I_dq(%$Squ@{KRIXC53-<}kkSn0!TvT(;}cvH032>Ysk)k`hq9;P9-R2=1Q-OMeAeBSDv|h-KT7pLoLCjy20N* z4lA;0)Xs~!YEobO`xh+}4%?S>uDyMWO(WZ#dqg9RJH7d2vcaZo^qwdCtDTv-WB#?e z!^}*YG>9MhvOzWB;rq*L%&q&^_aV|W%b92Qxt=vXwv?T{=gI5d9;2$??w!b=kh|~l z{2z6gM^=>edHmGAwC?`{1wi`02En0KRa@eunNc?j_jOa*5%wDn7D_c&3T3e5_)D&{ z>?;COF2ppb2bS%UhO+?E9PAYCVUqWz7VSxfYQ|#_j}$(ihW`A``|Pu)0Ku30Eb+oH zaXg!(&WL2$=}!bLM!}$hfiBfkOX=y61Jm8Mgb5mSu}FKXfPXR5C)5=V>>g)i8rvQG zjgniO)L&i9lQE-oHa*pq$*zz93G6{GFfeGa?aZKfB9NBJ`!ZPq9olFOxYGMF48f*B z^V&v{*Pv+c8#$p1t~yJ1jbo*_#@D0^h`k5%91%5u8OOadz`nzXh&y}vsb0{=g}kn4 zJ?j}c8J&u_!{ZK%#7GA)Ex|^o2YkE+b@NwkZhS*X{2Hn1&xaM)TP0Gw!3YHHq-`P@ zC)-B3&_gYabz;q)5Sw*TCt66zb$S=%oJn7|ussaHc^(?3PuKisp#aljz^D%obxeoz zP08)wgZV73DH>ykjWlsX_|nKJbP*R+ToL9M>3U<6<-_te+9#iSwGZL~+G zFEb>ic3s?CtdSU5BLUpUx^fyy|5JqDVqxM1nO`P5=HErYI;Y!Hts68yidtfOgUo5d zj)@WDI)Wz}*C1i(RipW|kI!|O|0DQHfSW2rT~|*cBY7sq?4CPK9I@T!p!*`)kzu9; zY)vw7qZe&AUbfQ^msjINFz}d+?Hq6-WWMWU!yMK#cTrt`G{YvS&bdTZ)ksyMRjuI0 zfTtirgdJq33Sb6-F@fwj6hlBu-Uns+opS;{%-9vR5VFtMDBWgY8@>LXoRLMa)AS!4 zqTfqhLemF!)funlUxW}yRQGfDyU{+dlxe*+ghs_oW8)pI*~%c~0Q@8LEZvqcpnz@C z?$Qa{q~{Cv*Y_d@mYr6A^eM-da@SvyPjG%I)N;G`48bJnOEDpdp{~VU4U)+Pr{k#j zi^WgAeHbU4=w*g#s26W2tFMk**}Qq1OQ8)UW@->!n8e ze78NxQcyc(As-gzaSB+xq*wfOHB%D2;oTLGz^d1D(XvN-O9Wk^^XS-eKHu z?y;}E?fIW8Z;Jk4Hx(Ae7!z!4@W1AkdU4RA4iE%-P0pp_3!6@nq0hym*jM8wR}(9{ zD#PrT5#5NcF@JvzZ2msG{TdCj6*)jya3-n-%}1Bqr)kM|A{E$Gc!!myyehMZQ8=QW ziJcq_c8em}H`*9nn-RP3$Hm+B8OR$Ae_N2|2TM40NQTX*JE{xC$KYL%!uvqZuu>T# zdUFPZa3&)FaoU{u3CWTUObe|?H)=)a8QXz&#nN^$wV4i$0Jc5t( z=kr{BkwcK#YiaZWho~15>)@Y6n(g90y*54X63TfVLf)l*K5WC|p7%EY&9#_;_xa6LU0#!#t{F;J z3xQ&+#%WH}I~?ebibWmKoqtA)d})qb>K`R?_q<=X#l5rauI2E5h%qzD8Zn{p8UIOL zzHem6<*E}bSb&+~Am|dR+myYl_Zss+k7IWH9jNsaAHYxRtnTrFT|y6UUIJhxHpp-j zb=THsONj;O1DLi5CIN#c1^m*$&k~X_Ksh~>ro!RSAHt9-8Ft+9Hxuy*vX~OTmj+(U z@1A3wAuwm#%WsSxxE8?)(Ba)YGu0v2)nb?cfP$gZTccMq45ZDXdON7| zCxtNh_D<)DwQH?yE`b`6DqCzl^ig0a-q^&J`9aLwOHSh93cxNg=d*=mZs!uw`1n#B zciSXBO>HkqXhH7dNKsYUA$^r$acnBF&4UtS8D!y5E`q2(zjMrHF=I<5_BhPwuCAx= zI{o=JJ7WH@dBC6H{UbY6ChvlTm7rv96D3#)n{hYxBQ6x7wwBe;6br0cGsdc>Uez54 z^S$VxhAP250C}CbU(_@%_d3HB_BHGd5kTtM3r?fn!CSl%9GOP0Ny&s|XhOM{jMJ5^ z6Jm8-P8wDTMu~rO@~)(mp0;=#H^!88_2Zs*=@qn;9H+_vq?%&%_TA=rn0ldg{y5APfR)ikB zgi+D@MhK6vA26RTg!!v{im!vKUcb~aqiQOD0lLp}(&M13gS-k>Cv|H&4u>bR7f@AnI(~terZ)2R4 z+$t**g?q7XQLCc{ek(V}(r>`NbEa*Hx>Zm2*=E%NvEekLy@3f@7 z0$QqT0^X>)i$Fwwp#8p7|MMz(z8#C3y>d#oPpQgL?3b2@V@rCIxz&dz>>!<*d=&8Y zZcpTM7XWDt=@Rt3iXIk1fg;d*I4b|P{8cwOP-E|H?NDL_GRG_JL_7d}~4)LEs>yll{4vw*L52uE5>>iK>e^G9m|Vb}^v* zTte<&Ao837dO=g4qz#E9K)7g(sHmzpb3sriy!3>#4y|O{zBMFRRP{yUp)~-s}36Z9%h{qcTX9LUTuc=q+Edi`} zJ+vF)-%+a9e~dwfa#|VpcL{gD`j3O_t!H7#jx7OCyQqZ{>OZSKngcb6*1`mkF!&^0 z?q2Et6Lw2(siZz4y%PT43{#qOw&gBdkC=g$-KDCuTx>c)a#9=<`UBZEm5^%GQ;uz5 zNYn6evb>{Hpk#H7QSZuz8n;w}2+;4f3bc&6vt5)0Mh=i!T$OW(q21xE?A~Q6lQs{Q zKq*zJI$Zw|mNr@*e6SUQ#U!QS@K~FS)VSY}^=pL&v|?7-oCIEA$AZs3qwY4)4YO7; z!>n-ur7jnMO6nw?@b8Y?cE)laPVPiKUfik-=8qklm!|k`049ximmG?aV8HHgCR_Ok zZcfSQBElzxVP#qrSvfOdJvU=O`e?(O&7e|>E#n)}f{?|bmnVCN_9ffP^_ziugGmAj z-q_JkDqjwnbmx64VjEFKd7z~age`d?U#0563xis7+x^>yIcujEZNqDYgoc&UaD0rOk0FXMH8CFVjS7e)zr!X?i^! ziDq^9k1qHOYqOA!<>$2(CPvaYWLgxQX8icE2g~8{Hrle-uuV)D+1uLHRCbv?`PCS^ zJx6^M(N+&W3JHd!ppA_BDlBw1yy#O?Cy?!+?tVg#i2!>?K7N$+CgVYeFp7lOibg6* z*El?IY@&Oio2dmpUgW8{N@j2R!R*#SIGQa=M<(MsWvw|9heNwerz-cn6obIjAP!|W zhp*HQ5jJJ`JyoSVwdZfx(m2)Cs|~P>XlR7f8l$~|&$ed~@XX`SeC8iaY~V*v&w0Cx zDH;L zM;?@kgb*8lO6zpE{*T$T+e_0LAk;CyH%I>1MIs1ad9I)I)eNNc$C0uR z+mhW4bHUkZE{AE?NmLL{b%)zxb{&x!o)7-#~vxJVLmgN-;x9>V&IN?yPXW!f7`ABY`oJh-0XC|2ohSu zH6Lf6MjFdN)yMXP&ra=Z(L)ajEu8ydg|wX`Z+Sgwev}QpcPRNbr|wW0EGgNv2eT2} zRzr0c+Kz{6daAqMrHZiZO)1N898i1@Z;Ong`)$lkg}9>Ys6vLfHk#=8U*9^1_KqA3 z2jtR(WO2jOyD33I4UaK+`wKhSYMi4XCifTVsls%fmna1LVMkUO{M01ozDkvNP5p+h zxZsvZxf6>V$qcM#W-|D^LlOlTpEm&kptty~`=^S!vxz{P#)Zy@=FPgq`ar9GmyHNy zk&v@sC$`1(QB&0oG>;=V(#bI$+rwssHUIT!P@dLWItFY1+Wa*X^YFjk)?1oXt}zqf zb+_>4SOoT?H zvP?jjJNjCqc6S?H|0P$)zC>i8zGI2FEM^W5bT1#raLglXD{Em~RM(3Nw9C&Z zMIv{yDl?XY|C-5K0@){#1UutnE1=XA(-}qBDCZEv$hw5QvxAAt4;V;eIAJ2iEDJ77 zwDc4hP@~)5RH}jD_l$8_=kSl!&!`8r(-hkb`PkYE#g0e4SE~8tP~w-tDOJb|-S^mf`J>jpn!V#ql0Tin<^ZTb zq4bi@jR&tc6CO?Ug;Gq`35&o}SxJ}Q^`GRmp?zO-Qn-(5Giztvn%lNAv!ohPw4jKK zRaMN@+fs*k3B6Ly*ql4G7Hesu!l1> zTdgVo32v0y=e=?p*S1C7YMzo(SUyQQ}q6|7cUO6e{{wl*RB)!Rzgzr$)TZywCm}mTcHq}rC zIw%=_?Of+Pt7~^IHY3F(9|+@+=yvbS{(`n)xsq+Vi}ewXi^O#!D@@ERkNRx*;!=8l z4Kr5$o(j{H{WSSeRBI%SEdXjxB;&?3B2bL%-3?nS-dk^Xzgj%yECqWtU~xclZ+bt! zaThmPvY$!}L>y4khd@)Xhv8BML^4cbA;^5cx7*G7BF?5&LGL%1h|4HKT5;A-{nplp z?+&5b!hUEoqL%Bx14JTXb>^G#i-gfKcJyC!KazQr@Y^`&oF;kq2BmR4I?5VzjS{jxYolyQX3{7l+{ zqT#`0G4uaY85nT)2$cYD{Kznx5UQR+nXbXTVnfw^1Ig5-sminP-5 zPG5$n8s#f99#R?&DHvA`Hm);=Dnjw0;QH$pJ{RxPw1V4i)$G$dgZ|-$dOu#+`Ovr zZsvcBNo>xj^mwexaHloS99TKB_M47KgAuT7*KnXg16U>-t_GcK+ER>U-7xzBx6z;b zQY@fMnT9Y{Zx=C>2CHj@7XX1PJ#M69S;$DA=*MB6sdc!DWwaNCONVbMZdS${{M`J+ zHt%haqoF5K-bIN)Vpm`3pWbZQ{ed3IAl`sP+J5BQTDIbCIS)4ppuNZJ@Jl^dm~sXk zsFyZBtgw0ipJ!Fvhb$cnO##rQ)tNQr#y4 z4z+}0!3~2r6Bc>(rOH1N(rb!KZTLpT`HH4)BQ}j-`g3U0!4qvmjg{nf&98v4k0HX@ z)U`jY=qMLlX5ezwvrWDp&mdk*SZZ{rr!{cgxuH&eCYy{@xz80kS@ROmOEn>GUT%|* zjB3m=-`qjMwFFK0iiZ1MG;a!-W6NT>xnWN4ZwMxGB?*r<_Qv%b6Wd@C6Rh{N`a3o2 zKq=_%H-L!8g--Il!~cO5OsK`f^~-htTR?A4p+CszHOk>G zGdypb=nEtRtts^*$lIm9nEsl#g5KpXmR2Z_R`Zd?VA~2=3|OcrN)$@RA51?WvzR1i zJ6d)*Yz@!jkqgQ^tj#*xqEI`P%Lp%zC>qa0&n#FJB&g}`V7!a$jY6QBasr>nRMqt| zW$e`)*k9p-&pI%2reg)1aj36qH=oB}yxh`?&C?8Z%b(jnFpjVoL@x#Z6d+|SKF@?u z_X*Wb6!`GHewbVG)AH~;^f%a4qt9`zhH^FZCX)UqxL8MmU0bWu-@%lSQ##UJF=#zv zW_shmh2WaudvbYPSG6_`-b%Bt4UAZm3+Zv;t8!Auz@&x2xL^`B+AA5;XGJA)TWoMZWyzC;YRG- zbVHKEY(KZcR4u^IGo0CM@c}jP?fYx`M)(&ErOup$fnpzi8xfFtwAF`TuzLe1d05q^fb>GpISBRz7YPAYz`hV zKarTLk~jd;!l~(uKEd33CGI=Lh>4di8#)xa&6?P?3TyN2r zNV5b!Q0VgY2zYMWjC78i7w{apYG559NK#IR*Z>lR>ebXp`3i*j<{s2`K$mTq9g$rhCq# zzOKbg^tWgpizkC(_tPmqFnX;q$ICv@SE-y@{->9`K@8QvmLz{qo9CZq#P4UBJ=lTQ z{f!_jUBkzcc(Ytou6$4SixTO3Ml6OOz|1f`$LU_k^v6E9YQFTQANf2!P8>XXptHUJ z3`OtwhdxXV0JFm4Fk8O}{*()#8dC2hlBzSoGF@opGP@SvI7Id4SZUy?XvXWs&G)Zb zNnaC-H$|qD_|Jag2?{k9^PLy>SRNql?7Cri={zUwE-t>4)~+nvR#4R7k%7!!C9v4*Q3zhNpZXVAjQ)(`_!tkOf!mwU??5)EDxkgqVIEP zJ2$%#EV48LRHJlE?cHVrEnV`(?%HUn`9d+aXJ`;$Hq^G)Xe@2b2SO=?3<*#7sP3_7 zcLWr5;^eMk{)v8HOC}8miFa|Q#>a~s-A<13p%)*XMg~t`>fm_W`zj>-c=3n@i;~61 z$sUq575$*Mwl%Khy^-5n%w!n<5?)9ww5ph^+Y>$!p(GG{p_@jX-a0Ktz&DHZBaH`K zd^hD%H@Qw!72Rs6<9VX{fTyk#@U;OT@@%(Ti>r|+@D#ci=F=Hslcs`^(;3TAu9WfN zTui42znF!O$Gq{&BJx;qZ5Yk~;~{H=J6$IcEik70HJh6wpf+6MUqIOf-(9o5*bOp% zAFwO1P3j<{ZDhki2BV1+f?fKRZ9R_7k4Tqkl0(6^&KK`*F)Ma6*+IDcXZRP35jB2; zyCF!@pD$ICOOvOpMh?eot{X9V<>avN92WNiSiUjFjZN;`y>Hx`2?dL<46-XV%qt1GMkUo zcks_ZtJOja(u1=Yz+Z%TpYIT;`S6{(y;C=@K>7V06a3r@xq`PE@R0>_7>FwIY7z1y z$N@yb4fB8-rQ`c8`*)8d-cjeT4CO>x?;+CM)SP^ob0k28-qD8rGP{JEpxipO)zR4P z+!1<>0E8^FbjZyzbDPG;czGt$SEB5?VM6L2I$E8C} z3zt);NWSK9>c8_KSZQIcDv$Ozp54+A^nm+U%=GI&N9$|A4Dhe(_06I%X+hDdfSrkuia!v2Z{5?dr;}o-e6>PxlqV}#aDr( zeWb$fuCs^5f`3j6KU19XspxGSBTnOVX>+|7SE-Ls@fOnI*(9Ob+F>IRpGb2%f-znc zfN4Nutkv+?O<2O0BF8V-<1uvZobJzWC#kN$D_$G6jxm%n-qnw0Pky-MII$|D0}Joa za~DN7;e*LKh35B08dr+vXfE`(1 z4@GZwR$13kQvA^9E5O)~|(x#Qnen}h}I-pd&Df3Wl(#0s4t5MR>QG5u%GA7#` z4CXwnn+2kLRY%#B&CUI>otV`d8%eR)qX+@o4$LnVRuEGIhX6I_Xh7ULKVxC&%;|WdI6|oh^pX;?DxDiV(Ol1T9LIj`leEF4 z@krc|)cuVid324cvp#MB0B(H+m=4|AxXM9am=RUu->xa;b zJ$Aae)_@YmO)WV7Ms>Ztwdkigz5XcjxG@trs{lhB&U~i{2dvIF^HzQ&YQBhKV;k}` zj|Ic0$aKp6odlkd>Kpy+ZTOKHj*V#Y&3>l%&_v${<`B?W3rpXH+4T4%48=6~SFQoS zpjPj8>2o8)p{$i-QxghIbE+p0HN4epUe>ko((UU#A|#$uD{&pF4(v>s)Md{dW*s<# zDnQ3ndoS%8U+$&d)}=q{3p8QOHiWiMo!vvE!1KGgjw*dSI4%xQ{8v(UTO4Aodcb^- z3Uo~T)__D}~-Xlv(H-vzLC@uzK3 z?8n_Ldb?9qS9AJ0Rik{!H&Mb>P5eJ!F*RL2uPQ`F=qUe{WJwXZVZ&`Q`qP!nBkC;t z7$4g79a(*&(A=88Bpz-JGE)F?w$Vg`VSZ-$maj!*phDn$%sRY{))a*;pj%oCDC7yD zPQPIZw5+E=SNdzEYByQhO3`Jv0!5MQe}WaQ=DwF6R`r58+P0;{yJ+-Awe49Z3?MCG zS*qsexZvm+tRVh9*o-+fxL?FDg~CCKv%jbYVcXpGHKzIKA?=vt$~SRvIRKv|*fco8 zaG*COtBd2atmlK1dwBw`@1BIBKkD~h(k;C|{U&0XAjpD!=&Z0QJ zW+G7jOq?ew)VSd!4luQQS%1n?!MBFqJiTLmaI+87s2#$tgBR-d$#}Qm@sN` z>G*vV*E2QFum2X}wLt9gFPXPcwf$4+p@%zRYYp$aGAF4}?hF5Keu z0+lNG@8mFL0g3L`kNrKl%m0srMuyqb7T!$;sxm>+O5x}%4Z3cW8T`vfim&svVrJ|9M>IFP3SqVS&aqtc84l*1M7a)s6%^a} zIo7{nJX_4J{{v?sK^@ejrNSIeBJHb>e^c2ShlYV#4XLRu()V-+@eP}b%y_1_MgJ_r zy&~tr*`)$Pjr2(C1WgAs!rF$l}V6%-y4j)gM#mtP71g&y3e?sQ@$UA9?Z48V0z zxz6gF>?nBo9oMQo7k7A*sI?r0zdUbb0hGmquu{{;P|q-}JLmc1+A0r+I^!(|$bKHbr7anX@-R?`Ml zUu>L7l|SOBTRGbOan5cD)ebW2&lcTg$}6dRzI|Hz#%t3EM&QsbZCau3<7%UqkL#nb zbspY>9LyA$^*_Qh9I(a7RyIMZZ=@Y;sZnJy-gF?pMvrWDAA4?wJ8SKg6xb(sa_8KF zwnIWWvhxszbxuXNillOVg@n#S!VfpU+i?vtd&>9VcsKf*|36ZD>fq5C0YZI@1q)F`DQOh)z;$%DCYVmHkCa07P2;=*MBC$PKy6qK{TFBiiknBsA zhO^AU2LXd6w$JMHcq0h;R!KVFOM7=6bbwF~x?uN<^TTl+2?ASUn-A49mSNt>xI?$> z10YA|T~rcvw6Yw|bjk0XpYJlg0BTQ4cOs93bN7X#td3Rq^A^|IQ~7bkldv>L)VDR)k|+&768c@O4X91>h2_fWM@1el+CLzjPhfL z7&XYQyR3%~s8o%e^5B*%V_vBMOb({7Df;OV;6Q)zE+cZc7180q_A*Ye6k;sws*L4? zJ=lm=)7x={l}risDyD)4b{T&EPkca{<9BefOd;d&Pe;XB5C6zC8>Td$7N;A$QkD?^ zD?~L|gSR78qJp=`3f0wBU6RPNE$kv@|3bElYeDDyNZjMsX^ffHG0?8u$^?L~+dKsw z*g^ptfW{7ES^SByl#VQRuQZts5(K2i{iK?rQ%ISRp&sDa8G)AQ*YT_TJU1;9P7S3Y zGZA5^>0@X~&}F=%24UU|jPH)0{7GzvuR&3N{|+@~Aidvh<%vd$Pdb<~{s#;j>dUx1 zhhlUwE1x2hE!^#mWmxZI4^6M{WvIstbA2d7{&0N+J#q}ZuE{1X**T@}k|t%flvjl3 z1}y+#2@kkjK@YLqc4fuoiT+uV|AJ||i)bdinp=73PqB|@9yImTwYj2wd%p7lW(8N!-WmDAAEY#ajZ@7BcZ zlA1GjYOwjK2};I@aTROIhggw`L};195buI+?J1&JSWd6&W#c&W7OrQlUWXIY473aH zR4CGO9LLC|bTLu2OlqxZe{dD>TLpdkZ$3rFN{*@4shh;G0tG?Sz?T!c!^ z0?oVeRNjgD=uGW(C1+zDG%FxD()(BA>)GmrsTxfSV4!Y8QbeCz!y9#Qqd5xxmN;r- ze>MkCOv}}F4!1v)pNosOsJaj}upfHRo(gzI!or_aZ50#+Jj#K0U@sox}pnyCM^wSX#ro{!f zjUkg5vBTc}P@XW^P90orO%O`)+(Hv+Ra+)(1;?EPkpzX{=nCSx$@M7{6SU3T+32&# zmbB(h`#AVtjPT6pLF<}Y{Kin=`ee5kbbug50N2A)3FznFID#fOAxoXND;0i0>MhUb zb{rpMSRB)zC%x=ba{YZwVuoSLPbDr@FPv_MP0ddD%MeD5_6c{EbF=|@!ebFj zq4SHIiQqPp{R-N~%j}V`LYLCfT5qf!yKeU!sZ$;1fae|`(-Nc*L3hvZ6S~8PEp`n> z-?U3NAFGWR4f5i6Uhjpt^w5r$;}2I9bBA2icz|iaGZSXv!OBZbLT@-*p58!L$cIiROeRtl|7wy1m!RYbYb6bTo^AN++3 zSQ*vEREsv+!S_57)y{~&-An$P=?v$^NNzk6|ATF-MOs%;3;h zWyOKI^#@|FJ`8C2B?|~tLkVVF`5gZ8d!@f6Qu>&ET;c@4#bB?R0UW&&2?`3~6=g<+8HNfy z%O*&D))C^^h&d>o3vTwBv#4B0y@*Bmr3<)sz-D=C;B3Mt+JzL`{@~RQ{yNi;()zGZ z^GD1JqzS9BAIJ=LwBx^XqLT^Ak31m@_Xqyu6Jt@Z{gS!OJGL1s3=vC6jN;?+s$YT2 z8np4BxHVaj#^H2sSeSUMMa65cxuk zF^W-yG{Zp8Q`{$XH`&1(IU0E2o#6*KX;|p4kH^e`PA5}_ql8<7Z}Jbr0@OG?4SPcb zwmpKOT$qadvm^t)1Id0bxHC#<&tca#?tQ;0pAO#vT0K|>6Y~OMXf9%WL!b?@1;~49 z2M?;6ze7%skNZyPC2hhz`nSn7ip2x}mXrJC;Leo8>q9txr(oV;gOz38&P6quq4pwt z(zJ9?eOaa-$qvVYpldMhr@w7g$R+gv(9dyj@jHl`cv1; z#)rFLMo+fpAs`6pnm_w%vlFLQihtB5SC-)XtACmr3$g+2o;OOjPSncjFl}b(Q_LhK zdoKV>k2`27Y9RddL3#}!!pw>@I(qav&vl6s=li5NWzT;+!xS9D&MPv$$D4V!ek1lq zX6f{v5t}L3RHY_*d-6s%g4YD8bkcJlrI!A!{Zc%yLu`O=rb(s^K241wy%I7 zjovy{<6yqF_X#V09G|LCIVv|3tc(UB={O9=DVIec!H*ON?vh1ehu>x-&Ed=(-GfeT9`QJ)kA$ z$nEvm&344aSRMYU?^3x2(Ww*v4qJCzJTSRe;JQL2B)tr`$<$c%>6*CBqev9DX+qa7 z4)#XwL6dX_Kt&KgR|LaUo&Qwfu>y5EjcP;CDj|UyLoKRG?(Gq8h^wuF|G zOA^R>8|?T^S(f+}7-e`Od7P)lhrQiaoA9CnuJar(imbIc6Ak~JgU4yL@;#*K{$%L6 zuucZMcR|YdPh+1X{Sqiz6q;yn_^iISI$mGX0VnLQn~|zzLS;+^C_?^T0|bMyq{rSl zmgcaBXr-=FJ+1b*^b?92m&}ur`v*;vOkJK8KPkMd39;!=EGf}capq;~2tHi5Fp2f= zPm>%;l;SVA3%g>kiF!Az*q`KumXH*+*BO6F1bdpwt3T0t8Vc<-8`f^fJ|P}pYw>mr z&bj=N+Tkaw*LzSXo_kxAV^yUYd?%Rvs-f^hYmr-}I=xRjM32Z~X=N@emddaV&eW)# z6Hjc78{Gvzxg?ki828_tcelq^uH8SPyCNl%8}dR^G^0vwRsMjS{b9Q$Wx5ABpfiaf z|KSj@IC~LNs#LaS!0P7mlpq-tj=mOh)i>S}|64M`ksgkR@^5b!)r_9F0Pt zhnKv@w4H0~{Zh_;P%>1>vJz#b{06<~(1CH;73xO2#uw8~I3%`GIUwpmRQtkxPVNo& zyu;`yW^Ms1HzIWkwrAF$2jWgN3LA9=&+Oid5lEWA4o1#B*iJo5Zf^Y2nQbwqLretE zR*8w7skdJ~u2e*Se3HJa`ysLBvUUJ7;rmb+C=~#Sm+XNx3H5jXb?x1;X$4UbtB(LV zK*qnZ@OZz?j*^2X%P}4Ab{&Aogr%Mvgo)nwjFpcq5aek4gY1*B`PgU`&RJ-S|#BwPG5HMYU%Q7&0&|k zXway8G2o7vOD3ypeUcus59#0skD;Obr@MeF{Q$LR&6OvAI3V{2m(%B@iCfMGM2SDG zgO4&{B5agf0-!s=2KmQhmB%HpdrmQrR9_R(#_Er2Oe=OgfexHMex|wf;XizFf;kQa z*d`fRhFBnP@b?U4^O?fyW-1l}VcZBFdUkce@=_aznOPbRW(AW*JZ)kZ4{Pds?KdaZ z=kRSJTertp7sWeaksm0As6gP1v%9Cdorr<0+`VN>K;R!|$%{J8y#94lAMRJ9`TLkN z^C+-s!Chq!>2e*xez36~tNKbif3AvC7hhf(aM?iSfdRf;uu?rSLndOq-t~eX^IcSd z6n4cEo;ou-;3FRl-cL!1b^jpjw&lJWioa~z;fMQpB5;po?KOI>N5t~X&;vNC)JOw- znt?d8tbokU7jB-&$D4v3z5|PEb4HdgL%?8rUDhu|wcexW>)oPh|9uBjqIn`Q&jOTf zr8f!tBaZqdjodz!xOGt#zcITWX6aZ=fa6Vw1{LTuw0|%h}$m5hKBJAOkFpV4KiImc|0Yx^d@b8)`c66`{mUj}249756$J6Vylw ze~;VjOTcY_H6Zneg=IJczHt?E3alM+_C}7Ot9v5@bfw9*G1$W5P~#mghkOI%3?A7)V#Uk#bZK|6-uWmu9F~W=P<D7UOM)l1gpnBvHr06&EyREkDV8B>D8~6`kDq!_V%I@Ae~;n z&;Y>Ai!~oGKj!#HoeG(rXa_pfLr*WBXdeU#l8_cER80q`pq$l{qON6t#)?J$VbF{K zkN_LMGZ40IiojTvbSX5@)xlZ(wRGSW;%D~h68Gi+0*1W|QvHY}qCT{&DFQCY@c&HV z(VTOh##v%ZXorcp%$6MHh1!#3(4zvoT`fEpYv?-DD)Qs-pQoiB zEdz*<^D&--DR|*=KYLiN0!6fu_6!wa{Jy=7W$zSItFz^9&80^p9bbEmY#=%%!c4kF zMtKOHI9UJK7uASzgrDvWqE8BUm)F>Gpz9`{Lcbl!U~_qbuYj|o0CMBp+JaD#9BPz@ zUBjo-<__h$Rs&dNE(HA|Gh~fvS)XSarcf2Gd6OQP$^2rjxn#yPDQz*<)2{1fPF@1D zR3s<_mH9{Z39HPCBq9StNF4 z?#W7>;LS>1R4~46dg50ZUEP1&w~miZeK>Y#O7-L|cE<^n zN1S*Evzc96O}>IQd2Ut=I3?pi$J+BZEnu$XW);5b4HVBR)30aHfke+7_uG_F%oIA3I7T)Rks2;;Ao4sTXO zh<6nNYQLurF=>owwX@#R`UI6i$rnv+i5KPYAmCq6b_uJX3v@sF<-3hSq|9f% zCNW?Ahk+(o5J)X-Q00UEN3S5%za5htf$p zFME`-4lqT16_k_|oEC0nDl4gaq|M`733-xA#R zvfx%MnmcD*_}40izD5+Xf)Cx;umi@+1M&GZ3`QBA*BAp_F7x;oL8!Wjh>Le7a=_d- zJdY<2931Hz(h$a=_uy!ReAM8;?5z_b@SB;7v_cYnstcAtLP%X^%NzS#`|H+xgo_Q) z)S6bV!iOej;|(IQ%kd$--#=(P1m|GXc}{AA&`Zmr(My=+^$~7+{|YrNiuFj z5o2jjB0;^Uv?oC+tl%2bwF5U9)525&nsq>|`%~Db?&b4+@_$%y%4n8{98v`uZ_Yat)w$%p4*gTjP%Rekqs7)0 zA;=+;2fCgAKA9Y-T}xgp+T?}(Gl1LMiJFlpi7UUJfne376kQ=JkW(R~{nt@Y!)k;w z$YF)JYRnh^O!qUP>lE@6xaHI|IHFq(I#$*yfnT}+X2t<4YVUZn|-u#>6CMGGg%?|oMsfg zSHENs8VujAU4|u*Cxg=qw?3=d>@=TSJ-n}6bzgq%R~iUJ`Y%TK`_g&sTaZ_yZy6gnlFE$Z*{1_MWP`jYbuc<@Jf^?zGVIcSSrS4yFNVgUa>CV4m8;!ttAPaHgdXU}Cn+!`+ zG2esEtvOV%KLl!oCfGv|1DTWaM6uSusbMFCaYsggwAzaggKnGu7vDz{n&jwyjX9S5bP1^SqFbZ8ebZ(D^lD>nEL1E$*Tqwt*(N@tJZyap1_ny+ zUk1YQzRly3Tm5|x=66toa3=6c#n-xjWtY>T z*nNre)On9{z+c1I(icr6eRzZhgK$M-{O{#D_vGD{+g+teB^R+2`;L09MI!D#w{AZ1 z5$YZWq`X}purM7$ceN}PE_fu1XsWsbHNA0;U}vCp-n1ki^iJg^Vf|j-jj%sd38W=R zNZyqq%w?F386TqG4sLNbibBU<^R1$nmQTmS%J+FZqyd^`(+^p2U9g%hf4dCxCyCR9 z`HZa5BS1xZi2~DJ5;6rsK|QT4SNo}g`s4Br0CRy=3i0V^2u!6s3u|ZmVOtDDu6$dK zCBZRuD?mYuz#$y0u0%K;4~nFMS5c*m>bbmAjQ3ew*#X~RTQW~PDDUmOX2b)X56whL_P zLvv=i=yH(ki-dr%4%9KsHvM@DNWGQyL+xc@e5V1wKn*S@(af()h`}5!$j|JwQ%79^ z!dl3m365o~%BIJVbFu`UHb&3tXJk-G(C6aJ?9Df#V;e|16oBp6t0_Q;3$VP$HeZO=G&=Y!Ljv2r?woO8v-dK3{?Xbucx>>npg~lrSD|1~b=2bN za7l0=kQ*mc)ho>lO5!sw_*egVP*(Ck;=Y_ESYPWWG&L4t)=25Z?r$e60}$HAE4I;7 zD(CRsQG;>1=O83@AVrgn5db>>k2aR%NeUGvEzSUIa0EX#wkQnHma$kyTGqOJi=XWL zRfKGQhH))li45tC^ZD%?;|p8_?Dp7Rp*Jc~QrL{=IhG@9OnjUbQ=0EEf`ROdZWIZX$W6*0S3_c-|>G^79+b{Uq)c)SNzagQem zj~b2YMuf6qrnck1fkznkdD7+`%Tae?ww}e-mH+ss-qTpOkZl-L_q5-^wo4A&0VpeL z^;}#{R#YES&!3kmpR{wzXCq1hDb z`E74t!&I~C7mpzgchB2^H0ycDw|yWcYMWK4y*(kbf&3{8AWW>mhaDH6;KS>ls(}bg455KmXiJtOHo(rAofS{& zaWd~EB2xuPNbbd>zA>@`@sw~-Po-B zu|!)nlI*;ba;KtGg*VNIZs1-A7Gm0Sf&b2s()F24KCmmSH20=z5Jf36M_kebW(BqG zuX*4Ml0{h`gNbLL7j9_fT1vO*zGhiQjU^tUbEY_z92A4{!5qDYMkv&na?8;+KFSW1>c*iZRBEJVYXfWiV=wauo z{?}b%={po9M?89d%Z=C{;1 z9~|3CHqKpj8Y%wcTaN`iDzdkXqy9Gn!_q7lz85ZfmM|%090m;D3GjzR?Z)k_N&(C3 zv9CL{9KA`1HZ3Szi@kxb7aH(>xY`^$1`y}Odh;QZf&4;tR%}{a0j5@PK6Ug&3R|IG zc@F|usaT2E$TyF*b+)9N=(l6)8p96^4EwY$AOEL0=9TUH6qaG5^?;bGQE`+4|F+)S z+-Rtb5ye}j&LB)xAllfP*2ENUcf!h>SuJA0BFoHByQyeXw%w$(K}j1?@lPy>rTcEQ z8UNwnMT5j+LjyI$-zgqFE>uPBdY?3r%a`#WgvyTqS}Qx)_NpW3UKf0}GOslu^xRY) zR@Ns6L!k@ev31epUY&=veSn#BHpIuxTh*LZM5T9fzy$#l|H9^^ecFwmsrMMcH7yPx z7%)tNY|Po9Jf#Mn9jfsj%pt1iv2{u`LqRg!pS}(#6KyJ5b)=X-7rhqQ|1%*)vbp;p z8V`f;=zsRUc@l6p`<*t`&?8W^}N;lLP zRH!VE!c28|AH>rqp38ALTwnuvP!Nf1$wHx?J%sRtHj)-JNp$m*C<^*7sw{JG0Ty9! z!sxQ2x_?8=h$q9WqwS`Sfz3GwE-RinLh0fvg(cKfdCDOL)|(Bp zSM7m!2%=BE3rgeo@4;YX@AiZY1h7_vgs-tAE50Zc7E|?OfhV#YtRh)Ob@Gj4KP#nN z@3W`GSKe_HvUiS)08E1rENC$?UDZqQBxuKR9JUvRz|ppy2&KXU3EMG0T}~KsAqku~ z#tgt)?R?$Bk0)X46(Abc_%~{ZQjRXIGnwaacvr+4AyF*OO#ZW&1i~b;5Sy*#wT;qG zdf1S|FUmRv(Tr(rQyb|AEfG)F3A�g1OFbIeZx`dlft+>t=gFEj;2k4KqL4R}4t& zjSzXE$fUF7oEgjBh$C>+s6@FDk^u`gvkQchrt&0v`e~-zJ2iX&!^z(K)*#v!PdcG5 zzF%S}bHUphwy~liR#Bfr4q634qSV z5xyzeU<0{Q7rSzh)k<2IF&Qkt+rghm^`a}_>yYHT?&N>K5w3sDq@WTvx&!Hf&zEu7 zZ5-aHN`22vXw^UTYp;@_3O0bx?j|dOYE%F)50<_D*qZf2JbOU8f}AA&<;sRkbYmpP zt`DAaVv&^SQT2xv2J37@lRpE5F6fKVuwEYTFIajGf0l{*A?c>ohxWX>33}7>Ozz)h z#FMmTY4DUVz%_Opnu}wfUhrmQcxHAbr%k{JXs^ut%#U`Ewl2zIvo*MV(*pF){Qct4 z!1@l5**LcE&B^bJnr6{*EAcA5%yd`0HpMdzhcx>M8L}|STJnAOqL0J%^MSuB@Wph2 zMSicYr6mQfR6Ikvs?vzXR%(FTb(igZv+cl&b0E`EAIxU%w*4h`4PX!#9cHX{?a{sS z!5)+GL+%-39l>Rnoq}LKeAQg&=1Z8Pk!;evQw)m+j#FqFL4z=_D1qqla08c1!QPSE!(3tqF{}Y> zX8HcZdIi;Y)u^)fUA~f;V)C#gIW+*~$a(d8?LiZhJ^HvwowcVdCnG|n#JGb;8OuH7 z0u${4JV8#*M1J8Xk(q7#t+O+AzlQG0mZ~M@FOc?q9e^@vqv9u? z$#LbE@!1f-cFRDnz-KXu!U)6iSb6Kyo$scxc?dB4Z6c>O2H$>Sv(Rkqe#3ig&ihUw zkZFwtD*1KeW+iudOl?>?(i(R0rHnVMsAUxVh7Dy=qAP?~)J}RcNh3n$<7I1C)~YeL zLlv)S!~V;d{?4TPzDJ*s+K06hEipm+>0)L6&hhVA>XD}QO#B6UM=;Di{SOOaBx9jg zSv6ZbZ}Fid>?A@{jTBod0eAU(PJ`UMoyM|tUi^D(+QS1WdZ#*Z0+Gr*@D1Ko+y;0` z%GJxucC>8=SjyoZqc>k-UE%!}^cwUa^_ujxEi`?x#l@DJ>rCsnLtGUaH+N1DFqtl| zLtSUuhbbL${h5Wp!7J~w_Ck|;ONo~7agKcQtsWr!J|9Y-ok#Nomnv{Vr}*Fur0cV` zME&Z};K9KupL6+MP@Q2KN09(NCVZWioL|8+zORFYa^zc*;PC{n*0!S&of>tyCS~W+ ze z`S-FXW{s>mFP9}d!RQclcU1UbN-KwK?@-JeQWHj`G%Bmn6+W1d!NoZeRghWV<=tWP zbt0c3_7~(!i5*3@$WRBWy|8$>O0adkiG@YfpZziM z|CZ`99g1nb4cu|0%zy#XVo8x>iB`jR2Id+>#Ct=rC-RJXESAYT$WX?_&wT#f$HF5> zost`!$R+tRJa`66_h3X?N06mSjV!F7y9c-jv_ShhkN;^cHXfa%Wbey?mOi7hv@bLI z1fo}V{yd{IEk_6`$YWo0iv zQ1kJKL~c@cspNhay=R2cV-u1S$(O&~S zrjc(YA&AC9!FcYEUiTLN%Ktt7AK$UT&W3eXm;6^gk8 zutO)hINUn!6+|=l^vAJU&|)5s-$UxQqjsGIiD5EE4?3c?3%xIlkpXJkamibhKXkN8 z*#USf;%aAJ_D#7=iIuObzx=8}B5NlH+?#ugq}^L$gZb*;wEUy;`gd7dODqNNLeReO zk#z=ff9RjbPVyqkbDYQ?{QJ5GX23nHk&xdKYy2?y}I-( zuWk{JJQ4T3XsiAv_1?E+@Z-?}`m^AL2*VnkEfhd|Zdx1*9Fi~tR-1#P7NIu!0rN<0 z?%B@=5e_)vONCh2>P2~-)YQu^!4X04!YMlJvE%IZ7B-Cp)K+5-GEG^JY7L*#%u)7j zF^LJ}-H#7?5i)GNn`)xPT)vud_9K`3VD-;DSv`L(p|T;C2D&8L7|Y~@GgQMMB5oKd zEHQ}SWG`}3cib15DkzkR?uA;vdq!fQb^!5E#x?W{rL#jh75~Y+?^7BK6EbnP#jy8d z;J-14aR-=BXIe_o!t8q8m8w_aBGzqMINC1dLP)3&OIz=u^E zoa1cjIku8q6FZfH=LQGwXi`EhkqC2XbPfr}lAc+0Uc;MG54nT0bx9=9vrtEJU6v78 zL!|Rj$BmqHXPg)CpFPt4%^3ej;wT_9I03|Yyub6}fZ~>f3W(6X7X6}~nh9Vz(2z-@%v&!2e94*Fc0TT|EVzMw_%n%0&FpN{O@wwhVE#iuc!;Nto3 zMQzhdmG0c}p7cDM|7-~JOyg@0HlsO_)Z2(1y}u32xRxGLC4S4l_vOKuoq;G6vPT8W zo$7j=-EhaqXdMK{)bOsTo68dzm@CW&#)kfOyvKi)nkrB?lY47pt4+}-+(;R|N8s9+ z(VZj$qJgP|Wi{xBGl@DF-7z%@a$LxeAK@EXE2ZKd{?e0|Yi;b}za@g+r3$_D*7lrV zpLbNtSrn1;v95;O%}G{T&L%&3H4`!(Ll0E~p^3t8Ot^E}&z)7*#5(R7IG2QiYGCr_ zRUWlh`|^-sa9`qlWbLD&(Yo13vvX(jkC95p=ZSjs`m`Lrl zSU2yCQ&u1WFNvY9RYIfd~x`$&*#~xD1 zi(n(v#E53sBBd6+IyVP`LTjC)`YyDr2QQ|80lDM%06zCV970L#Ul%rbCeJ;aPvpd( z@E~$)L*I7Kwx~M$RVNqBIblRsow&M>AmCpDZVC#S)8!H3W4p}@f0w8ds_ZjlCdBt4 zi&)DPS@=*aJ=U@Rcm&@E&2bgIXn>2& zaF*9ks>3jU-Wrrk{pckT0C_Up>m5DtpFcXi+_q($<~@0&HYX_4I~~-+plR>kYoZ6q zxHGk%-!9vi9f0ESdWYd@f=b-J4Ou-=SR8CulTPkyFJpvsGqZWOHf_`9qP&~qGibDi z1^o9DIf*wJN55wG6;n9eRT#6AV#RX)VUUDQZ9&l!as*L$tFykA$w2U7Zhu4!9gAZP zF>TN;Qqj6UL5kvrKLCQuc3Nnoyv+;u2Mu9W#H5@dVD@k94x!G6oAZ4bvMu74UnRLj zOrr?v*~W}^lAnc!`%ld^bI?$-UZIb*BrMNGI91=e!}(I5YW9h<&Zp$LwIYv(MiUMR z`2MfL`nZn9;Zr`2$*w5tO(r4dO!?F+Q)hGOW&8b|qeTg^=o?!*uZt_3c%LF&@kwt! zAN1Ig2Y-+#=!|D)=%)OF&YH%F%pK6KQDhL(GW;3v_p}?*QVg*CJ!51=q@JysR1=Q{ zH!}NBzoVJFO-)mxC6)}M{-VOs@wi1A{4k}VA?$WE*IS+{oUBWSHBNhK*Opo1*@6-7 z%|!q`@dpyQB<>nPS~QoJE=6Y043(c$u;mtY(EZo3FUOo?mLz89Vcl-a%tP=^s!ZM& ztjAS)^|296;)NPSy_|RFfnel@4+D5Z_8^WjU`qWD40Q^KWO~elj#N(L&$>AU4-pJ@ zj6pq!74fl5s!cxsY5ZTc=QAfKF$I#mNAX6K-y+vEBwL7lYy9SAPef{{Zt=rfMjCn? z(ebh!Mp_Yap=fNC-DgLbMmo{!@{%*h%_il|oTfqCmAm zj*$U1aQgqZQ{&X}9 z;W#xhP-Vq8i~eyMGn;E67TFwgk2t#vU4pF{Vz(Sy;KEjZepYdjzrYR<$;4rXZwKeu z1iZRi2z%CQsgqt=1i7D@B;5Pz!6tRL3*GEz(?kuG*1(+?nK_jh4J4lF4mM$CM7gxA z+piZm&(p(k@SP0IeAO=ENw!+&_8lj<5eZ%MDfH1 zZEmSy4v2@{sAxykm22DSFT95Yuu7eE+?M{5Y|EN%d58-wOiBHGj_}ZOj2?%}61deIp z_#Lsx1&|g4;(LdAVLVTao^&z={8M-eoxCkK*;JI1Pkk604uG_s&VuIXXkHfBni8p7 zcz?b%m*Nk%0-jq8^aE4q1nxDGmVBctW7aev%tl{ZRIHY=a_?1Qu#tNoT^e>blSGG$_v4{2AH+{ z(gHh2Z{DGaUKoYc#I_C}ljb=DTl&NOZC|Dd2V!@K37XN$A4}>P)q$rKhSl1*q94#!yH_pEk_^m zqdQUEcd@G`uh4m9)sc8;wJP{?AtQ3BXlE-KW*>~OJG<{#gVCnj5uCC*ZIQ5-C!SG| z;)s#(7~Ir<%CMN@hK^*lI6Rm|F`b2ryn1777f-R~d;iLv`;#YHYNj<`rRD2c+=rjd z9qfEy_1pW?Tv>cqGV3NHJ5*c!HOa(A1_){6?dgF7L@s{{-}cIsI6Nc~I|m%GTVQ-* zN@}`6r3TIA;?zk=i54kEpFQuB4h%Tc-5b@&_$cB5uL`{a840lSyxOM-FFmn!guVn2qL_k^=RAIw8fRT zGUY;J{KUABQsbH;q-!|5>;ck3+FkqBkK>?=+*2cX>Gx>rg$tS=(bg)0R_vB-i1y`d zohM#Rk+D3@1OC%@e5QV+A5{Jff$HP0(kqp8(*ccE?;m+?GZ7CV#-XEmZS2RX)zDf( z!zD7WIB@?PU}?OGW;_6Wbm&FLn*}AmZwT_TWl8#YV3CHV4utk>9SCqBcC6P$hBCD= z+5qN&iknhnZ;=5$Z+$ITNDkdo`O5XvjZi;nt-L`c@JX>;7^GoVzD3OOO9|RD3DGBeD zzOk_ihV!Y#Bd!@s&s|YD`0&JPF!q zQZfCY4`}+Q{k@#4ew~EMcDS8!GABGscH-;m+z9gHS2JdC8(UU0*jtPZ{5P2Ow8P8~ z@SLZp8)Tc1Ip5zsBq(&Th6e(~l%<0LMQlM+x@Z?%UL_M8aWog*W{(Yk7gbY-%fuf9P2DP}O3z0ULiDK6V=tUK<KKQZ%_~-pz;r93WT6uZEgYBI(_z&NaQ2I0WMjmsN$~!3cI~Fv*s|^~RwC9u_nFZ7+t8H-7wnD}G7#69cjUvnH9DNXEK~5s^ z*!9q}+MnyqL2eC(qbtBhaUYw-a%yJ_uxw43-PZqnjV|8G#re)Qc^qWd^}C@icaQEc zvXvXLC9LQ!sb%-Zw-`G7Ot8*oiOdS`Oq3@C5KKc?$+OEm`9GwgahaT{SmU7=+rnFd zyxFKOhO|Xe-~l50`|EI#6^D|?WZ$nD0xh)W%3JC@@YpIevto~E zMo_u1RWSb~TYHct@r0T+UYRcYyqkV$&)SFMmTlA`#TSFgrdl{&K)Yp+mQ|;4wjBtZPy-w+ezIL~$tjo~*L646&{H|Lj!Eq=xuzM}db)jW64^ zhb&87DMSh&%~$9|u*O&+eSL6zb(Zz{A%(6Kx3Xsgk3@%!)x2Isyr~nx(;|`C&b)a- zW(S@}LS#IBcJwWHC-MxoMAUKQu3=ssN(jZjHWDHB>eEk}ycK@+D`KlOEZ=B0Prq3Y zPiLNUP4m~AW`p=0dr2|4!HUE?a0XFbk~=a8AP96qZxtfTD@e#1QvC0EG#3)TF=$+n zlyk(7QliRDJR6TH5YmJV?3*pY-(=74Mx}VI>pw&B&U1Ui9%3IAhBG%O*6eEb?brdU zf*=-VYv>snfjWcRwVBdUq9b65OYplo47>G2nslyd>osRP-V;Z~Sm4mjOg$_JxwJ|=7k+qF2GRa<)Qg5k+taH3-W}sp7TdIOWtO&ml zSanJK?Hq~K-yG5qzW8T*wTzJ_r~g)b8eKM%-ZEtb#Wf@X&2%OZUi`p9vAdl3{84U_ z2F0&hZbKa+uDn10|7&)*(^dWb!bmatA~GY%{l`*k`x)et5dVpHNhKwj+txg%Eb$y+ zp@lBV3xV8xRMM>?d?lTaML+7BvDPQPA1q|1fdmw_<_GQg;L?Ud9olNupNbA50g~lO z8;o$THpv42`b#VB5A$GQYL0~ltv6=Sbkjd)cTCRv-iUE6OVv4MYijjS(#fdY;ZL$& z5Byk6@^fqMov)q+2W2yhSPiKu$1bnMVq4yBv%=G^&vIPaHADl|8H|QS;eO$hUtox_ z)ungt#9*p %wpzxBLHFdgeW@lLPFl{n%2L_aqon7j-xIGghJuJ3bGxcnPHZ^@{f ztX9t?9vONG@vYawjf%fSUjvwg40Rs4aSHTl#Kv6=dw<2j8N6?t=b%Zry^f@bSPyHT zPJEz*&_7)i-Y2UO0fWQLjo>r>DM=2abGtw!t)=wJmk&w5!iL@$&Y@?<36Cg^M?@Hg zxJx!bu??i+rRWA9j>xVjM^pE)4vqU#dN#ZsCPU5J{3VVA#yp z0@UpVF z)~Va3FK5!emxeqsEgEQ2Iv2$ntd++Rc)7lu?cUGD0hsUZ*7B>Wo z>H$bzdH!=CtuXX+>*bD36=bmve2);=?1TuI22x*^z-|uNxc5Z_cv>=Fa8ouJULQ>EvVXGFix@Pa`Hx z#E?vjK8Tj~P>P{A=ea50AOt@uE5^U6A3kl6eJawUxCQzvVmo%3aaJf*T%#w!@__gcMxPG`c>1m;RV@h+YlC79;(ALJhr|gN@{2$vew(v3f{TuL2KWI%@1C++myg)>N8>x)M(i*27R|J0JB)Un{hr^oy0vo-REdn zUOE9lSAvZjJf1v97cn8LwwZjVqOc6|i1@;@qexuyBo~>HK9R?U*2_7ZdD>nc{M2K| zm9^be?N6)=xMOP;wO@W|{Ge`R$??#-2zPU(EU6J+bSRd3mWG;GdGld%P-dzuoAa9Q zB-8JtFEV&By(hzzGZj6s+9@${^3{R$CxG-q3@*nP=Qf4B%|NOToi;6cWm@#+bbso^ zzdNw)jdKkc#HdCoSKi>TTN-b57O0E4Faq(^Gb<4c^Tsk_`~TRT5?OTxzge_L8z^da zXFK{jy}@@nmnnFFvk3a%>ULxIj_wrYO* z!7h_zVkjao($Z}0_%>($Mw&&th02})4Zl$AA$o^WSb__bdQ%Ul6a+A$rl}!7x*R_M zJd}3sPJ&Qfx;*&o{M(?w^R$ylfm8X{P4Y-$L3Cl=3H#(t)*@5$C6hgbb3|8!%W?D2 z^1i9XHXf#n7}9rg-vY+hr`Ed*ykU<5sXDr$w$N_vg^=`^f^M$XCB};#!0DnL3oC=s z&hYU^#5bcW-IA1(#|(h_j-b9B6_ZOIbPC@1^9+I$eGeSmf*JqH=G}x~8ItegZuv!W z+1DD!px!CIjhr!?xq^FfIO@|Rwhl*4Es}L+)Rj9_>}HQfr65KR zlT0$yC#?G!O&=|+?(Lh(=!QVJnrE9EA>9nb{*Z8e7I0Z({PFt9@*p&{i;OZlGi_l? z{_$5#Hf4P}$K_Rb=tGl?=Wyw37nN5hHdA1~&TOL_{LKk~bA0?J!~V1WDw?y3KqO+J z8~?Xn{Cp&mZiUQ~#EZe_0`UPS$B`g(s}n{mzaYe%=q0Mcelw7EF{J>h6s*vWH+kk6J!4!IBK#P1R7l4> z+ZwH##3W>SmTzRqLBeO~--!50i}oawOD*9b-*KhbVr!bqgOnF-vb2HX`Vo~fmTzVs zr8?E*vw_kUy*+}PgIp$zUlIMLT0p`Nj#xL;%Wbf%RT{|iWC{Ij^>uAR=(^`hM zaQhP*GXkt9n^~2?M0*1)2)$s$4~7~6&49^8SK^{RCZHXthcoWh^AIhS`x}!oO7fv_ za%fwIOG(OI=0Nd!tnspu-%U>^?hkgms-F4fD8fmK0Q6V86ZafINR@B~og`;n~n z8lG0d!Z_HasvQpt(#O`o0=pZ*ItHqEid9)6q&NvL+BJk}#zRxw5JUqQd-j$D2vLM2 zyWbW+RH25F)mvLasUMO&&^@zm%gmh~^+UMX-GPOkJlBECUz1d;mtG2a=?87A zYdX&gv&T&l?h6<5v3eBb-mjlMj;{)pi;7dr+q3#{IPq??^j$znT8#+H>Gbg(KeJ}j zB+Z+^MdVx{xR*O&|C72&Ws>3+=C4i8LBrYew(k~hr!f0Gm0MuSxFp}7`~SlJD%FdY z3!UNjU#z>${F+6F8^A+SCPTMFQPgXW*R@qJMRSCGx>Ob0j|hQvlO`T4rJ$`4?8Pm_ zH}LrpyuXwO24=u74aCX2Fs8sB!(^*|dHh~PaZq%t9ozp5d8QnPLK61B@tfv>onYz7 zPtpLDg^Al2v>)X&19hCtjdM4s)7j4XeQ+;-Dn+NQY#T zV>@J#Vmxi3+JYHLRlJM$sQ_IY0P(`whXseNC-3x8Cwi+`j z61QY7^KJ?+HGKXK6(|tXIeX4fL*OTZ3gyTG$8vaH=l4JmU1mgMayEp zkn)WtLoWO?pZNkZ@ZDZ*2QWLeFJ%nR_#wM{&ynq(v`;Zm2XHT?>=U4MMJjP4Jf17C z`!KIpRW;>t=XZP-HQE|*Rug-&?Dj=2ApCsa8C~d~zR_66P#X_K>q4ME~^%Z0Vo8h{f7^9N}gc+MxWe~e`dNvp+qV+G}yG2i#@zp zDZR(gOCF$*JVio&<6&?=EXa~kcOWu*bH5UE0|jMjVL#zHvA z3fFcx`d2IKs#83u2WN^H+q7j6#ld37ozSxT!QzcO(rYFX?cc+;eX`Y?1AUU(LoyKY z@UkzNbH0FkpjG&&C!ZhACzI{>-kWmAnZ2Nh4J9aE5@##d&Txk4CQEp<(;LGd!Rn?FzY2z|IxDIGa71>i{R)K{o&QF`)Uy4h*1a|9oLqje zlqKkHG_N32o;MFlt!2F9F^ga(NPH#|YYcJmOqFr@X30?%q6!U@$Ih{+BfW-+Srs3K zE=BpA`7f@b>I^hImEG7glP)fGQZ*)R@P>u?<tZdAB9!tmIjs< zRyomMa3Wu_1Ot>{sG_i8_(N}j$$Soi5)#jrO126TVi*!UIElBKTOkPvd^k4+vmP;E z=RWYxFN4Jy>i05Ok2Nq+HPw+cJRRVG>PXr_?xh^Tq)*R|9_KX2;qiJUy(%%K^iv`0 z&9CE{ULye4gj}6(gG`L{(+C6QEi;J(Yo@^OX61>Q(+y!Y7Oc@=r0)3Nz@i}gv&)3V zNJ{63w68!=u_n`)fpaN=K-Z>!bQdZVVxmc9@oc^)E*kV#8pQ6Vyrtmm>6%i#wq3p4m(G z^hCiIa~x1bNk$m636K@O#>I3@Szc{mHX1S!a9AxO(3m)}gyGQyeriloN1$6}3sprc z9{NY6slZ6^+8LBK03Sf$zcM6zQzgTT3Hbij@`e>^&vgpx;iQ zZeZYN5gT>yaPYgU&Ta7ab@5X3wDvh@ueQYBj)Ym6sQKMfupkw}dBaKCw$atw@K1)2 z_s2AjYS>H$2r^&{oGXh-Dqx6MnXf5!qm^#>nmt z-+RQ?AsC7W!Tu*oB$Wxn6?7j6V1Uhn3*Y~a_$`EG9PDu00jU|&ShFFBDYZi0?pE${ zhC*&5l9jVtKyh#V7Bd*p_koVRTig2iV%Y*(3|<{@Bokm8zfJD_4r13e2XxWC5&(?! z!@}bPnnYDJr=8-S#5Smr^xiC1mk@AAUn*#i!qABS*d=jc-G;kYB-)RZhH{@MkGqOA zy-V4dwdxS-V9|s|j-ngY8uZ989OdAvN>3#dMT=NwrI*n90l$6}U4af^s6L2ER^Abc zx?oM@ZF+iwZr*UI8T=PsBoyQ;S3n^FSH?5gKXN`cJAri*n=Ym1h`S(YYSnevaA|*j zzxMl4tf`>Z5*_II9zpcnFc z^%EVACk643C$8>v1MdW*TKb*QM_zCIE10c+<9z_z^%XZ^(OMuVo>GZ;DQ6xfc#i2B zgi&yEGOQdYeN!0t^$_&8ep{i)lOy(lZ#8tpYcWE?THM=`^{@2XtYF})#*Uhh{y2+J?gC!dCr4U3z2Hf=EGJ8mzJlh0sg#Tu^l;9 zDLbbGphcS)N3D(B*|^n6&<(7=!Y)01^*XCGuhV-VE&QTy?G06Ck^uCd6S>tNhd!He z$NLv;mZWo~PO{Q-3a8NwH)&_G%EjU|A)jWy1`7q}Mv&efwn*vwSte$eQ2R5kbWCSqbH6HvOla=4NGei+h9hui^qBe8FnYcey}C@2p_2sB=+Cf+`$2O(?sb=m>)qV@Tb+%I)!k>=thT$i`i1 zH!MD(hk6A{@K(1fn2)rk09AIVL>y&k&iYb90$z6E3sftmyV9T*#V?XinKWV=Y{6U& zi!8=hI?q9pe`Xu~VQF43mlvS*3&#~)l3|r#O|&3AQrjN~Hye=zdk=4o7Wqi__k0WH zudKmsY1pZRPzr`6#EQD-32*#Xp3+ER&|8dxU#?Z$UPEiF5p)s2Z1pFlEHotjUIf*Y z4Scq6H402DnjOaaxX640ivGsqV&Elx1NFb)u;9&%GgOgr1$g&ebQuf|IgDiIH!B)9PE^h2jqz__c zz7*J_P`uc*D@N6ta90_r77H90XevQ?9RAenjP&2Yyg>u5X6OTO&qchp8>A$P>TMeF z!X!-48WjgrXnR}QebUKd_V3JyHA0pyY3&D#PsNml4Gm4mJm+{7Xed+vRS-^lTAA#% z)xb!jRGKIhRh>(}ch~z^rd6Sv9Z7^5sMl^700~TCFe{nVOTqxXA<>xHp})j_$W2p! zyfoZ5boO6L*P(tLxR$|a2-2p2lPJ$PZoSLn3C&jhwxtC^e&TZM_jgaE^=Rr9yiz>0 z{rRPO3Fh#WZ)g98D?=?VW9iMLuW;n;$X-8+4t*1ZF%qNt5d^gB&wPJxKfyVo z3)8Qt8CQlf&TWE~{9(`xnW8*Xd6@E;#X5nE{r@VXVgi8E~1KE#&3 zmD`R1SV4J^^s2t{Cu#@7p0F(yT~rq2j*x<0E^lo$auUR%;dK2mLYAD|751A^jkwNc zPA8%TcFmsRpf6s(#B&5XXHF<&akg%2-ByRVGI_b^VLpEGQie|Lq{QJqnH1f^4vc?U z7w)RS^NJR1x$VRib?BP2zsF-`HngoEZ`=B>$B^`ULI*dAJ!l*@{xW~AQ` zf&vDt2R;Z(Kiv1vV{>FlIEhGlWgkk$5y&J2+{w!;1I00b`sCd54brSE&+UKq&vektwNTqAzM0VP_1}E-_oZ7nmE9F-rAv3;+STL8ak|V!N ztg)_RucDC#cj)OI+9811_nPPH&kE&0PAcEeW!3?&*NUm|AHFddzd*L{psneEFTLn* zk-V>Lp##B`jNk}^S%(i7Pyc|ItJ30K@&U3%yk=x(yhF3~5FoEu{zCEc3P3Io-%DSp z-oJx1sp`K2lhCp70~QF!l^dWgmk_!=$>c-nrt>(ot&+Y+HfF!&LU+3Ww3kAHgK%pT zUqysffK?8ip4P|l+|Ed=c5Wpqf#j)E6C~?p;IIM&rw7E3guC0?l2m_Qb&ZMDhhGa(f87PxopbhHwWG>g`w%Nls)-9m+Vi;}CU% z99cL313>)0U<9}FjAKw^{(7{?|6^WF8|EP5I!hatokB;3z3pq!n|=@sVS5xO5h^us zLJq~xWaj@DeL~AN9va}@P?}%dH6nJ31}yed-!|0fp1%3yld`0-sm_* zrI8Jht2lMXSiN}CSiHUEF|Mp*x!+3OJe(E+Q_>%Zv5W1Ash|(0jP9V8|5%inp*<*U ztOFku!PEZwZH+*8c8C>bIa3y(GoWvB3+xq~Jla;9Y?J*A3H1uZqpnL&D><66fq5neAd*Wc^ zvh4L@U;5(px3?oC%xmnXdZ+GVx_=7Jb>XH^^08l}XP?|W<(!eclixxahi~<9l4h-0 zcyIr5BX}CiGnOk%UxTVRSOJmZ;|vk#?LyKes&GQPCM-*I!&x6ELLJ4zrF>{{U(0)W1;$Vx%dk$<*4Ym2DeJ<6Mn_xz zi5QtXPd zSSIG?;qd;+v#@w!W4@4IZQxROug_7=?ijD^lpk(@%I(l(C^V7AmxJTtZIcoy>lVX8 zTHv~1Y8jGu$I+Q}YewTuqCrVrx2km0N$?*~Ertb1>kiZwpB2H_%?pMz5W~)t zAjq1=whVVWK#2QxnGIQ4NNfV?9sSW_g+{xqwuPNmL}A3%d>x7Kmo^F+!+oLENR=`5 z#>lHl*JhcKeISY)-8V=OWf?SRFUSy7el7x(zNkEmmBQLZVV5ynU{pP50ReTuKQt0s zS;Z%D9e*c*lzjNO5ZfAB=BToLXGJs4-Kxwl>`+500k;uoD%^`%tnm+amMB5L(JEz4 z1qBeIlGITV3f~VAGjTb8tq%}j8H|#ZjAAqJdnL$`YzCh&&Z4r&$V(t6%2|7|{VwB3 z2&L*H(jKqB=PpY=x$QlwMCOVx#dFmaz-nfCF$b5I8Y9k#$XH2S4d%f?@pEI4VdZ^0 zwUw2b>)yJJg*IHCrLsr6j6&%2+_M-1W_nAy63rYIy(%ck_^aBP^9>WS?mAosWXSGyUva@?t#$-mJSaqwDsclrhbjExkddgCfWqsuBb~0&)7H;`rQ6oM zkZd2Y#q+n@Qrw~f;nd!a=J0LnnRi=w7kRgab5YF0CXS>RWQf^bG%@ei2jsMn>AI_i zC4Or~51MYtS*50&BL+h2;+#b%;|jR~pc6ApI{Wsc_6A51D&0}Sg)gmKt^sXNJ~Pd;1T9OZ#f1KOIR=qfejq0<8Glk+q0vJOrl#|JzJidhEp&30 zypp zFe?t_zab3=OX2Om;n2_ce&9;1DiHv>tuxT;xdDAXDjc1S8oSq~AwaKVF z@gTSoiLaA)hZKQ%$m|_I>|GM(8QORekwEaV2RLU+klNlp+0E_i3tD;@RY9FH>mo~1B#2X-Dh>c+2EGSNEt*m+$3w?WNi>c2p zi)Yi+Za6#)sV(N;BrC+_KM*d4p`R*q)yh4m9Ic1DcDFe2H6OS_@~d`9XU&lPuzubv zPm=Kcr!E)#ZXYiqlAE>$IB>4_Y|2?ATJ9H8sGRnRXRL+X<6bKxUl!%*NX0%5W-;rR zF-J~I5+lm|suRng5Kkf#f}}gvdryA~Da_aUuVJ)?0-l^eYp(~S|DS-XSaZON#_cW= z!+uTjPEek|7ZF%0K9rY|T>&nT0%ypZQBkPKIohSWb{>_K+MH(j=@`_wW{ve~OQa^$ z7jzIZ`X^xCovCJy$7H!vN`t@_1u!FE|2Umx{FlhZSX^GU3g5+xgR+@Wm^i{O#r&Qf z20z-tYA7cwaR~l{l1YJO`}d-%xRk6^Nl)jpf26Izg|H@>xQ?ld?pv?XBYo5Py38OlsN-x{)ZeUvAQi@zU$t93s`J1Mw!6FND^_l2I_uK+W$Gb|u#Yiy*F?Z#LT8 zxom}4@FY##4gh&Vg}>(VWa{4u+?5jOJ#6xuH-|(U)TuS`ZEY;JP-p%v$F$@4)Zd@s zkoK?^#5KJWOpOV-7@o_x6^aoNtU)LS@ED_*_4HQri7#N_zYKa8Fn(j>dN*YgVMlif zG%AAy&|dqp4NoovkvsifU&{D!?Z*92jLfMmwF_NjLtJgj^%ROYOgM6+=hQn?CXzPRY$u%19Jv=1>t zIoD@uI4I09M2(G9B_zdNm_qT-r!v08lcDusP2t9I(8$&6g9|JDv8RAKy67|PTb9#0 zo3lT3d0~x)v-)DR{rqXs&#$K7oY$GO)A&CvlpjH)o*=X#_Xx+Lz8CJ6{1_=mz>Hdj*tAJy;StM zH>3>(T`t#oBUkHJJ(uZT`=aZ$>B=}2;i8OW0d#pX_V#wWGPg@V|F{&7GcB5{*J>h7 zQdsHweGLf!oWj^qhdK473oRzNI^DediaP|I=_w~US2~out~(}C?=>lgAd}AOpL>C9 zumk*_V|u={X0ue_A0B$6gx;woKiOQ@c&+iC^R>ZZP)U!q*ETd(|Ec+OZ|703=qdjg z_w0Y^6%*4mQVJX|DY4?Hw!t8GGN?NBr^qY`x+0@&l}&7PC`#4?kCM~SW`Y;~QGhIi zJliyz(X`{DZCuRUV9ZPq3NF@NB>f=^V&NUDrz6pmI18?mvK;4Lv4dB)_lriULKMcM zT13|s+dLA6))qmlr;%u3j9H}Sq{wp3Z`bwtZSrCaOWUu``m#YG=07~=y`(_ zGKpy0&8=<%_n{GD+Fs1j^j}x9^X59Ir}oItK!dc^VP2YtgS-xJje{D zSz=;`vCh478|hB367YEqD+e8J)C)V$i!T*j+lpNaQNxNSsL)I3Kl(U&UX$VK&`Lc-qXDz+Tn5yG4$Hrc8}=7OMHOu zNu||o6p<3BYw1~yeR>C^p-HnXfajvNhmKiJ8<&}9)g4Rz1|RR4^u=eg{I1h@Inahz zBUiP8wFk=;7s|4;G94sOwArw%kBTQW2xyc?EpFT>W~?@!%B_G-u}o8o+3B6 z9Jwd7(vbuU{%od$>8yq$4-*n8vmQ+hljJIFX1PkblbKm-_<_~=nWx^#CaV7W z!lNfJ5{e~6eBaF*4Gw&BKem+tvvS@GgWjGAGJHbQhZL83G1&ul@TO!-R<8Y zr+Gyk6d{KZre{hV%;bm*v zrN!)GKa~`sICHUh3Y4wLyq$sJYXCHy*!Be6a0-O9rnFZjM!+GxVBTs>R7+HPkWTUd zY7nA5G0P27r%RdS5p|oB{%~Co{T7&daQ+iq@n=;OU`nMh z;i~HxNN4xbSgq5lD09c>*ZpTw+?dR-!-&ioNlVSREwE}Xs;WN5&;;yn)H1058O4&z zp&w&9AQi1a1ArEPCKyqOJ0?!BLHr$Naim5DpqzPvA^&TBZ zzo4yT@*`&WytZ7Ps)q&)w`?3h7xE;5$u$(F)L&zXfu@exsjs+?()dxrKq`EuxxPf& zqK940h$B>+;FjxsL&1G0WZP#16*;)8&AZ-V;Xay!@j{oLrz;>WWx~^qyOw?V>Gx+o zd&z3cv5(iKT15>V%8ovv|CPU`E8UfnHeT?rIr!0w*7AebCi#X9ufeZ6{Igj)RD7v@ zd(wf=KSu6d(pI35$t=#CQMh$yK4aJut{?ZB{9ItcM{i0>cjyJuFGWJSMom)v)1>Zi zpu()-Z9ztI9zrWlGw~KU^&|ip8Jm$7(8_X zC92H?nn8R(Q6`=zyX9mx@~Q*N>J|{@ae`$t7%_Q(hAw9g>W$$xvu9ZDH1Gko z@6pde6Ta;;Cp=sNCYpd=v54PLlcs^`=8pXWg*Re1lAFE~3>NZ{M{84@`-12UeDVie@_zqj!t*6?0mG8CzkpL~5qaWMb`IASJy38;df&*o}> z+d4;&L%HmoF~e3kD2B5-rW-bDurC+m6Knulecbq2DN?8Vb;c*WEuJjGhq8Ji!Ln$g zadjW|2I8D|1aY}*8y(B73KBS|tktrruDsxSDHEUjTjx%}rN?4=@q@(xs`gno7c7?R zIghBxhSZ_!9$st#-+IHg3P}fwqT5HIqGgd$?$#h|BozmN)3MQLP`O}%bc_M#v10## z^ZK#%10F2P^WCw3@7kx4{HQs#>S49{^9Q%xgQWL;{t@C(#XxUA2J#2Zr<{j&DK$3U zu-ZXa<%K`^wLPrZ^Ip=aP6D5SVs8fGZ)BzP&ceB}O5-j1UzS}J9sn(;!<(V74jaQ0 zgQ1H{8ei*v?Y(?5#yqs9FCh`BI?v;GwVMYtI_lMHjKDA8 zU^%+w*Z3P0ki*h#@t7MBfZ29(wPp`5xc!3&cJuGUOLgAYi9oBuT|)-81FY^_;NMej z&CMk6R@!S;&qC7w>gFs8#pF&hZ zzTJ*1mzAwE@wDj>sTYh!SN_K;`%*%z={??eKc(-K{^O2OFD3u%#KL9;zj$%+hS z_pl{SE!V{);z|n?CjCRaDVz%kvA& z1b~p}l}YbgS;3l05U5#{{xZAi%*u%^t%ZW#_u*yg-Albuvo(zEVy77n3^xBzK}Sws zGy%xthHNG}AaNh^I};yUh13w?dKV=o2WqcsUGXSAf#DJl9wVjKdL7{aV0)oN$M4?Q z#{gQu6i5r19@Tc~lkVF6Xk!gwM$tiHX^7fAiNogvvPNk-YGwM&ixCvf#k3`qwXs+> z|3W|;aFfr3N6KsE<6w>Ej}>3uCmBE0_|DI{N)2-gq!9zL-HHnCPSHH4Z=G>TQb>nX z{qQ~#s1{?^NjOyb=d>-ub5wYq4GGdNE(Zmlohg%w`3jg82MPpKQVp4z-hI*;!s-aU7o1!)kd9_)k}Xv2qc96{ZA$SJmhf@@*#sp zm6movaz3ss5V?qgUH@H{@jX5iEl1dRpmw9l05OLGCNbK;Z=!=xfsdd*Ii*{y^)$Ly#Bq(P9go8o=3f9N2v ztHoqXX~*uw8HB+YJ>`=}9Pe{RoaT~X1+}iY$7{j(@I%U>I`*=8*6h_zBFP1!Rj=+= z4T}PfOS%=|zy9DR?j&ut?*Y+G$bK|A6xd16b;Xm z&(w)71hZHSQiSZAF%7C=QzmvMDy@gSj{>7_$khgmT&vQ9{6?%!{-TAav3YlIguE4# zxdo-3?-tlG&xPFA6>9#GBQt9Ov;->?FUPbT7~_9bDWO#XWv19>+iz*Xt>M>ZYtVAd z3X|pQm^E}8%uq%0W$dN7w38Df&iQ=?+teaTBLJKK%9Wtqn8C>aVW4%q8IaflZ3LEo`xT;QD|Ek6{$4Ur2$TRuUY=J^oFy;Xb}G4tTr=%vxzcm9Yz7Rn)#WmW|tsu<;xVLC^b;#!T0C|^Hb@0-x4^O%1ICbOw1F!R;{`?(? zb=aX1Uhnea!7i|-v&E(b#=9#$87&K%Gw%LhGDe6Do+PKXC(38T@f;5YLEYH%Z|{s0*r=1}5fFRNWuXR%n1zH5 z#dvYHf!?zyRnL79LC6JLU zySLS;gcxa`8FT}8ekCKme&X*q2M6Io^wNTm&sIO5&j^QM3E^p!DNG;%&#G8 zHkCRc>>m`125nvJYUu<|nTq!ez+vPeN_4!29`u^h-8P+p>%Fx8-3S2M4sc4m!a~>VaseVSM8ZkTsn8sQg z;#}u4=sglLQtEn3G24c&WsH0<2<>P_s$3T520850w2MV>v4**u!Nxu=P3;5&u0}}n z#UhuVY@n5PkqR8EOc!&1Y>O^!oeq?hVwZfMRKd6y(}g&3mjp6JBK|<6s7r&5Ms9h! zzp~{Y*cS0|ZQEoc6;p1uwOM?OyON62co$NnyB-p85phLF2648@=dpf!tp{}vb3nY$+R=Nh$Hv%< z7dUdJvH{@>zw#8EuV>R%HFY3nr7?I*5DMnmdAnK4SUaa_rIJ&`OUd?_R_IcCQB~Ki zuBAO>g@hYeD%A|_8gdh_fQ0|FxgTA+@e?sjBr%Qo^}eiKDk{8nL+B$jEZwn`RZ6R) z7k+uP&ijj!p&4q3VNzFGHG%aj`gCTL0H~HPRy!h~ZEc}UP9>_-D4&5x{)*4v z_;oPqo+X%Om8$TBqS!xuhESsXvmR-kCSQAXAl{|8lORCC@}_OT>sd4T(=(iNvOXzz z5#@%~ym05E(liy6&Gbocrw$y)<7+@S4deHhJC~C!7v9i+y*c6+?4oN)Z&3v)qct?2T?9 zDwG*jj+bL<2BK4qxrzvCI;tcQEquo33_KBKxGF2qxF;u#x0*7%j_sTFsz@O_DYHOp+aH$1xP7NCQ=NwG!Q)ZU>3{7WtCb)KCkZ;%yNF zn;g(zq@-mR$Td;}g?zTOw6dw<9|Q&h=Xi(rK{5K77aL8>TRo^nu%Efwp>O#Gk~>0J+P zj|Hpm>5-eB(WytP+e{U90!7#Uj1AuYEUs5W74M|=8Q(AlQV%wQ<9wfWC0AI#h{*-p zwrtc`Y#r7YDSFEw{gLGO6hH&Q5qffpwzkWBuW$H<10b5<{>;XUTs7)`yGs9deR{qX z+vn@Orvd|?*i?X^XJj)O!Kn_vR=$3NVrN7o$hbk+Ad8N@GEb$r+eig^5;g*|W2Q{q zgZhy)AQ(q+hlo;&s;bJXP067nuZj%JkqizZKWZ$;G)fdQ65k8Lyc^yQqS`_)ZWRTf zbPVu8#7m4zbGD^Mzof}ldjpj~S4CY|NCu*LQl&@KMJVX*BhihZ2zW*DAnC6IkDj`*w?T+K4Rch@t&{pnuo($`!F3KbC`v->P?E$EEMh1=`kwing`9z4#Y zBdjxXR*qpoFl*a>6_5SUow#y%wE{=?C*faxWlwAJE=?``wPK)b1_Mz?)a=$U&|<$rVfh%(A2GI$f^&MzNY6 z56?9cXTgQSo;0cZt$@iE@)OYBh`|e<_qb22$Ty;w`^JfzqUYEmnSi&KwzWmM-8sfW zF{#6p;Kv4gM38JIazZxC%|=i-sR*q)V|QeHwQG}XqsF{V>12x^hkK@xrDFTgDD@`U z!GYjFomxxS^7iTxF+{dyrTLSIB8wPb}@@(xEBwAQhzN2 z8by@9dF9F-1ykapRPMN@>ylPT)UBM0T)ydsi4LT_gh;dvFQgW2(TG7!p-=$Gj;eb4 zmv5{@vnSW{EUX?%hOH3GTcjS7?h5!*TU(x?ja65e`z)X58>8JG6i%ki*dd%$p`=;V#W@ocX(7K`z)t73ipx&zj}`0 zOPrynps(PuB^K(~kXJvI?~|f}by`Nny#-CCkQWD0;+MpE${%FQ7<{`8R{N`FONKIz z9RLXfS>;l1(i)V6RYUgU9#~nQRxmd^ZKd0ZM=z%GpSQ`iauvAzkq(-VY<FDA{7A=ZFR3CNhV#;Zg8fcdZ=m2KRaY`a`ZM+TiIfWfG)8 z!O%-z8E=0Q9{4mb2T!ZUDH4H1Fn1Judsc>{h)Tn=0m zLZB+D!HA~XLRyF&8eGAgVi#z)^uR7xB`A-NoHgVUgOF5~fz9{_feaC*Z=>^XQ6E;5 z`>`JTbSqLre9h?et?jyDH1I-lVHzb? z9rV6c$qCYc3rDc62GYs`-|kPxUXjb!@I<;15p^?A(l*Oe-(GcbmK#-QGZMDkRHP-K z$gG;yW<~UV=f+0qYn0%XsnJH6ao7YN<@QMEWfL(TY5+4Y<^$y&P{GGR72f)+^!mxF zO>qm&M3-1t4tLE_3TfC+RBZt<0PP_BU}tD2r0fK=Sd&TfG|a>HM!n@W#s{E$M@NfA zeC72RfRAQ-N;HCrkHTdwusa*V@6IiqhESL$bBc-(sfSfe`|qVN_!4A z2~VUkJjM(Gt%8J&d&Xqq3I%W1~HhZG$d=`{8iRq7W zamXQEBMvcuyL*58zR^`)P-lvj5)lZgpzj}rO==K+)(+SmL#Yu!xY=wlbIN>t>oB*o z(J9^fAk(uq{#%QMgIpaYwv*9Jzf##!6Vo5c98tjgkfc{(#x{nzi8!W^dEvm`hbn2R zpF=8barp z^pTjZf@avKToAcft28^?I^t(3SiHdM%tu77iS=srdwgb`k=NYyV6hkMn!K&b-Jg4D z?Dc0fMnUtK3ojFAsrT`Ru^8)0Asu@DkoPhabUd#H6~v&C<@se*yTcn#w09cR5SHFc zsN`jxpd4xn^4MVTLg$=+^8Gt1WuJ3BJ6yKCwx6zl?6%xp>soHM?-VCrnNnRuX^vg# z-XM132}Uh!8y43ofiWCnGOEE?3GoRzuw$?E8u^Y!jP}i?`=3HA7IBk1bweX)FV~zO zUs|-vDVEzsva2H&SDBgDKMC6h#WHLIuTu6dPF#)w1$fNxhEU^+tJHG}!6##aewVWu zW%oS=obO)C=x^!(vo~Afjr-3@$ddX4+q-E&=%+(2B#xDD-SkoVP`7${SIbR9 zXy~XcYPvP8V8J#s-r%wQOBIl_*z1|^yX@-`7xR`oY{dPT&L0n?EgwO&SJQNtxwc!j z74*Hxhb{5$!sCfL+VgRAJe#|f-ju}>*jpkGU~8pyD_wk$R5cE^jqt@hsTnha`qn%? zbjgDj_(5E)eSQ!$oZ08Wa0dy#ra4O74%4^LhBa{1HemWICj;7cj1;5@59$X>3sc>} zMJY|5X|?MeBbZp+%HRNhr^ozB-ws+nr@wmzlhI6i0?mGlFB$<9#5omm8XSW@o z=TAQ;eqO%WiSG!Q2rkPd(g#7F)qSG(@yWRnf?>z6Q(2LL=4ZTN111y)e+ylPDrY4J z=MWe?YQ4&94IaunKD?2IK*1}pp*f=aFDt|2c*TCEzD*DzvDhGFaS}?S%30F>?WG+hG zN^VjY8L*;L&BRKyATT2)#T~<ygC!i0;sx>@qHFmA}$iTTc1i++=9nb7gk!ydfiA{lo^EUhnv{ zr&aiXa@`oI`-sM_i(ta4yFw2@GY{~yqvYoDW+}QYJ#~{%n3Nj#w#*}>&L0CakNf5a zNYIU&08EwJT;${kQ0#a zGAiv4lk2zCFA6gcs*W3tQkWlQGr9Jj^{EoRSL!FXKAYL=Eo@J zVuaGpCs+DPdUey5KTyGhj3FeN~ve|`71}!v;I^&x! zQb~5eJE%z?-&+R^X+rE&IOwk7fXAI^7ev2(DtJJzKMwo^z$}Il-<~e;va`3LpbuUO z@qp|$i6f@0Fq#eY0tLT3_)Us2ENH}mMsy~e4j5Kfm@WxN7)H=}zTg49cnU}!uelwi za^zXlZMM3v;~_ZCVwo{>7q;cj#k-&VZ`3f0;I|OQHkrm-ZqH)kobt~@^T8CkXf;kQ z%EdO`7Qkd30V%lRLfsFs;P|SAW`QY`c$eM|BM28!=lCmw+J<$bGpk z8|Js*o-t0>G%o&9TDF?^I}pQCqmS$Ut6FUTmj!dl_3w|qzX(Q zhYignr~VC8rCI&tHL>=*a} zj=v10)1{%i_Ta9-&*i;%O1)5wz((ADgtEzFt(-%my2~p35Gbd#&OHO#CJy8oq=P2l z1D%M&Y$f;1{aJ<590<}Ajq%{u57hY$A3Jlo0rTXwqVvVVjMPssI{G^WL#G;_3S4G~ zj%CR`*X${O2-_o&K2_wb6|^YMD+%LpBTDG8wn8*=`6iz&w=QPxz6o*U@ZOe_vCxS! z;dA=2g))xiRDSntLwE0x>9aG)4ki2%8|~wBbIvlu4~N8+YOEC&6bBA+r}FY{nVB>{qCLwC zN3>*L?^(sTl47YQr4I^pX~3!!45h%mrsJtW4fL)^oCYD)mpdhVJ}|u&&%y2eM;c3m z%r2`sH|54+bN`U_#a`G>CU6+kX297-dI`FRN3kFbr8P#><57NI7~I(6F@WH+v)So+ zp;+b7zJX__ba_9jEl-9(I`X=5L$zGzPgD$$M$lzv_Z}BtxdL^_mC`2HO1+tUP1V~m-k`!NTyTfIwV2RS&moD@52jzC6>U=|iypYl zR=CSdpSX$*4=EQS+tNf|?F`Yr0EihBLyKE;9i&J@$$5nDDa7gXc|U1PVdJC|OcXj4 z05L$$zm8U~$uh*nn&O#vQM_-3bSIK|;RRpy8*BypQf?NW@#G~!+O8(cT(639^l&17 zl(NwnAzis|9Viy+C&<0(@QjTz5`@p4?*p=*?nPsLle-}VaE_)oOU%{skKbr}?Od|e zz@%A^Fvst~H)ZJfH#;V3&Lz&q>)xB14i_xfD2Gyc1Kqq-0fki#TSR*^r-KF#eM?F5 z?ElmjG#($fqGtaNgd}ake6tFr+0gfn;%`ua8RCCTG9wo6#XMtDn|4(+4SSujYaw(%_w_>#4RjxNfqNk8Cg{x&~3|%5+~}sZw_Jt?>lQA8mAJj6P}2iTo!rde2>xcbad=k&S#H7T=he?PIG+$}g7ReWyfR z5~FfJG$e+K-33+Q^^AxQZ%VG1*!j5 zhlgxZobx>J?TYMjF*r29;O|}ss30O_0%T&GFYxG(Tz|k?>j1VOq2_-qKGa~ zxm6dKpoX**o~3a>yGN|koue;V;5%l26IEM3=E>Br{0AnoZZz&vHkP-nFYJY|`h6zd z8fm!lJ10bJwDLej*Dz(!!ty@n=mp8@F4O>uoBtx6AeRNw%P%bI6yEv|bn6)PuBa3F zAIGB4^#8kb{Ch}Xv$hOq78miF_nyuy8xqsA9-ubJ^S^rsd0%QA>rl_T1fbS3jHDg* zWPKKsrAHPZ5R-otXkd0F(Ues7#S)Qgyve6=uc_2w9%EHsZO_y(txE*vq^ms$sp=9yTBgH4y-fP5z>hXv5};Jg;l#2LbsUB2#; zB%HBa9g?$eO+djd8A8*D`7+VmgiM8wd!ZlJCf*2id=s|ZX2jZNqBgeIIGP*3zJmtK z^?u_N&=0@M&E=qFS@vx=sxV&Zv+Kf${$L<)?q*>N{WGBOK|kBvCy0>7RtzMaioYFX zx81E>CO|7@YdXILD^i<`N3s4LZAP0$gcM?FVXB{x-vUVmwhd9gxI6;Nd&B4)Hz7vj z|6VdI5&vXT_UN)7mwRn@uZu@Ifz;;Km5W;>so$yY#sjH8rn?*+w=Ud zHC0;fZ`b`%BDrv^QwY^5DvOlW<2Tz4;7wQb@hFQjEcCU(gMd57ljM&b@7TRz+(BSs z`<99??kAm!PWN4Elq`T+V0`_|IXGpF)b4slDLPLyl$dS09>q!Nf!^}M<)v!|>e{o% z$Osgs$UmRn&<2nA-+BtpbvyMO!MoLmy!*_0yPx9yt$QYpMWI&0IpAnEv>aXi7krER zCSNNCs(G&_6vf()V90knZZer=K^rK(b~+jBDrL+P3^ zB)&1)E&1oDmEeog_XC9?c$h%wFQvRq!YTgRZ71twdL5Qj&45C_9e;@kHssaPID~ho zfiE_Y7KNx%O@3W*Ssk&};~74fh}UAL>c>d(@vE>fr{#nFQ`TXKoUZ3?m{ip717{}A0cBs>QZG?+->iL2Mi)eWe|OXQ^$>ltpC)Tzj?U#R=6wV>_o`Dt-A{E9 zp{tv;a5uVHS5aX)LaO-d35$~ZLCv8>B3U+xOq;=ux*QI1Hm~E6eyHQ3G(F@@*dJkC zJG9POH}1!VkztI-TZvk7xyuPSobew^bl$2VzQAh*_+YRi(x(yrKC)L~zI0Q9)(lT8 zt8XRO@k?LoGp!GmY)|5j9h?TQccY5uPE_F&p_IFc^i=4#FPSvf5sybhq@t>)X#oHZ z0+p}SNgWFkj^H9!l$C-a)1-Jfeu&ITY*avHEH=>#96khHB~ z-C}*@d*N{t;A@6YT;wF{Z`*89HU| zHk7u=q~|HyE6>o zD1|?g!8_FIpQLw1GD8ukrz(WFp<(YZ0&oa!sQLJ<8NH!}^$POXTJ>S`MRrVK!$u=w zXKp`DcTRjI%Oe{t5Udl0qk^Ss5_^c<&70&t{qBpI7UzR?7W;A4IYNq$Y|(fe0o@sO zqG^>V94Bf^*3!45;G8&T?A3{KmyB!gK!}>3w4)g`MViSJjDXdm@C;ii09MSOV2JGrCxpR`Z1VD(ze-!j&foxGmVs~K+3JgD1#@lU~*Knuu&B+%33Mfu223Nz|Jcbaw*%Br^;W+1toRU9a1e`ZQtzhU@ePFPG817g+1O zd(tkRSLLSV5nmsAgoa4)JbYEsp0iah_M#VL2jQZE&l``knrrzf>v|c(w@XHPA{V25 z(e?n0mJhw%Q>zPz3;^t>J)lj}nrbALBYV(b>;VA;EMf6opAeG-(Qr_8uYfQ>+WWJq zxyF@0xiA=%eE@OnbPOMr+iql47nFj+B7bqhqrvrD5CQ{S-cu#^J>&v zjK|D9n>`C40(b;7jKk^0N}SZdsV2h|mj^VpkhB$Rb?X!1Y4MH|`S`+r-dSCbO%`KvB`qv$9{1vuA)X|!P%7L~9p#=C0!GzGeKc|&!vZ?59 zk4?+qB!5GoB_20fv`GFvZv)Ctr}&ma_{oK3)E=;S!aMt@x_{8iPi)j|>8M`{wsQg! z3Y5sMq*Wl{sxu}JMq>R zxm&mjsm#$TC6p5>FspGvQp z;ylgpf4d-y^R2V#TN#hrjnjvm00lt$zmjUXQ5v$G$^2E5MH+HUTWKr8s*ZLQnjY;G z`H6HK*JJw-i4k}lql@{pjuM-56Dfd7uSnERApv;PjX_l_;?mnff|=6|Jp1n_IKRhy z2d3YTBlt(lTNd(E)ntWe&o};<`+*9?CHUoUP3`|@Ps~HGGzztNm0-N_`aw|n{HThy za~EY(@L5(n)(YemQ32y;8s!HEdUah8`52gu8!tSVnol2@|lZziO3!Z2;d5C_Y!9VlM3DB zJtIycbt9Igs5UFHMI;&7lQ{x(|M{zrO06pNd-EZ^E4}zhLkcuq8E8n^ERY@Xw*BSt zyC*#ypHC8quv&Tpo}l*zt7gO_!SXS88Hz7gW*Y>@T{YT#D$b4)4TQ^eCbNB?tpK(g zZ=eS=q5gQ3U>~z-A_HBjH|&G;K6M9p>INIBoL zz6oJP5p6r_5kY`dCRnJc9w)Y9W#%O26_#O~t+v-iNmZg>JS+a58++CTl2P)OUxSnMc?n+P<4IT~fogGpYljS*a-0mU~My(&aJbhBv!M@?mMt z)3vTYJ^2E!g{mM=uBX>Mt=qv(pU4Hxgh%-|`YVT}&THmzWO z=u^!iI2XtxW{fgbzLunYcHw@nMTultGVzI=0LUf%rA3~Mt6?A6>o2c`Qzwdsy!JzR zE)?=eBYB64Bj(0glGqoUor6mD(l&M6`=NHuk3t{*ZLkdimJl4wqoYv4Yr(%8^e~(3 z*~|IPj1vIXpj}(1mhVU#uxv(IX(g(hL|5`Rk!^A&IGhArpLu zOnmu%yFI`p#~q*KyNm&4Hw3Oz(PsxXDhvDTx|}4|^x~*!pwRLfyui$!>pG($&D&oQl1`65x z`43Hnfb+WBCu&L0Yj@fKR4_80AYXF!&ds>G3u;3*54vGV0ETu5p+6?iLN(fsxxSJ( zm4www(bS^6q3IH6otv0wGHzd72iajb=B$~iYAiMXLl2TwN>1Jvz;o706y)XV1#WMy z7j$M;z0usR7BpcIH))0eVzXWpsTw*2#d|GvS=@G+5?U8r639RD-~j1$h#3)#_MTR> z6QN^c+Y*pwSA&>VKfUAXEkSltO39^MUURAmDxR(BWs6Wum|8&5pz&?dq2#^VfV$tI zGn5)Gtw5CdFB$2Q{+UpgmSmqwr6&tqw%sLR+K|QFd4d-jXx{sosd-dl;XeTdPUz4Z zM=fKL+!4N?2oC&33=7{SKom@{75fAynaLBvUmd13>*77&=`*0T%i6eHT^RVCR78el z;)t5|A7@lzuNU_NhNWK{B-Bg5b9d$or27WxM<{C2Gugp#ES_ldto8IU{klfyu&R>f z_H#6(-2=OF0uD zsqr-}Iw`i<(nZlB?={{7rjNU`C1SoUCC@TzkNg0__q}NUEmzS9#-TY7(XkCC0rnO& z8I4vmuT=ABmaYZzL?y{LZQdExH&~-nLkP^yPEi`1hPKOQ5FIWk=QFk(SZtG@)?M-W z#5r`%xaXEzibO6*Z)aY+Ymxn1eaf;WKmzqaC0f-t*}~*$SG-4`7-9n|xiNkD7I^P& z2=jxQSBJHfZS?tHZ09i8A}#FC{{XOIbk!#;{7*LdWq!LLqEcpelze=F$(GiMk6~f+ z|L=(%a3N4*K{0%e|H$$_P5X|m6QnJbZ5PhpT6>cm)n_U+eTE{*I_nq^n=2-Z@;}7n zPccqF7gtj*_o%j_6BWd zNTT957Rp0b?(y^rB1>~TFW%g(rmIfUMWm~z;Ldhv_Nq0S z1ghn!1PgwIJ`jqYBhOMp!xMhBVfS!LYfna4~rW7Kuv@~?mo3>V_Qz; zzp#^>cqG6A-85AEd$mF8soap?m@_2en5c@oluC&-$Orazo5{kLz6ZMklfoEhx^`uZ zr~~KB?|D52(V({1&ROycVcy7HI*AX?BjRWK6~*YqZLLnGzjSwxa$J&?57LRwgM|rZ z2_Aaar}C4O+wP)DO)yNCASfFxD0jfx<3b2~EuK50a@pkn7=pwn3&wnrofL#P2*;1i z9!4*3Gz46!P)1?${psthX7<51wKZOYyt;_d6INtC$>GI1&dP`*F6ljWqnse)w!2@* zJh04uNU_%g54p@Aln}=_DxCW5&Ln3eIBazz11%X?B9JV#}}L{s$+oDY{+!~6h?Ei#Fd52Vh@?h0N$hJ zI}FMb(!SaC>5Sis2Zne=MATg4e6ro$ZIW1KN;mDe6@ZXQEsGPldLQ0-Wo(Fg3j)KS znu~~4HX;8|j!Xt_v;+&Jj!++LhJ18M8djKMpcn=I<*~a6 zHr(usPX8b$Ko}tH%~gN!g^cr+>~oF7xWkLzpIBROwh|AR-Icxpjx!Dk8ol)xS&?4l zIu>PXr1*x})>vVj`RndDwPu9xcci@bVTZ~A#=D-vNi?r7uEXF#4?FKtH|AY^S8S?L zDrjT@W)|rlsuldcmoQpI2TQd`CDA|S%v=+5fl4$VcZQ8g_s+Oc3mY`#)N*>y?AIJ+|cZVKVzf z<%8H@NmNs~?!t_44N?=CU)IxmGxi8u`5bt%eVKy+EWHf|@o&yF>6{;H>W)t#IhhVi zd~a08KXqOtqc&pcB4D9}AEafn+tEXK*cc3)?@TQ~R>gC4Ohj-9fkL1>`~8|;@#JXV zw%=PrVWmlzJ%V4<8@#D191-epS?j#SyJ@mtMQb)&(-{AdWr+;Xx zHKcwbfF&$He$J7U+lylj*Rqz z56}7AK)<1N&>x}YGbh~Mu1WIgXFE!Vb!uKNRkK-l%M zoyvC{{EnsnED?%I%VfnIc8yWawbAenbhT-P7whTFIdE};GxZ{npW;7U$;WF}x>Sx2 z9r^c)ZGfC8S1PFSuwgq?XV;W113*T`G}uSpwLT|PO-_6FtB5Ig7eL;>X3n>>aZqtG zYr3D98ak~cJVAn!t?1yy$bVanhO1xhnrF51Mma0`()qCpwS`vPmxx94%Fu~0$(SCJ zE@XILOjbD20}N;nc?|o6|6KTG2QF=^DyYXYE66pe zyhe0S7-Ma!2j*R-11Kh_MjCuIfWEFhF&NVLQ9Q@}KM|5_iG z!-3nQ+1LfZUuB&pfGJdZ1zkUsM#YG2^;Z!-R|%V?t822+|E(gh1FI~9^J(445~N@` zP_k5u>2~c+aeipiXYSOLwE%lamUD-jaPP@=mqE@zIv^Z{Z}&ivkNxV_QqwTfjaQXD zZ5gjUKg@U6vrFnLc#gRPy`@48k|YK|jFqg&{8s!-6F9hDE?0@qOzei>d*quYHNDVJZ1%3Y3^mk#9da|M30_>yW9O|L*V) zlES8=*|@1}{sjrp2|-&dIM!@Nx-*&18{R)91{FiFs*jYL4=L*q_u zk}389qv(*m@k3)jTY*54G|BJ#W{=(@q=NWYpWEm3)e<@qx*faV8(C#OOSdT zvU4Z1U2N+~;NKi(?oh)i4^%|Xz^a?AYV4(}wpYBW8+;1G=zJDdEBjNW13I#Je+5qG zLE&uKkmQNOC1rVaej1}CSIG+o{U8#NXT=`Y3yVNJI&N{P>+9(EYt1omw&uR3Y%-@q z3<=E~g`Yi$IV|xLj~-U!drOdN3rSFWa}WgK3~6CfkN^)inw}{x2`2FHScfGK-2DCX zRidBLyy3tf`^not7-b}G-T5`Vn#HArTBO`KNy_YRIXw_5KYO_f+A>AIrkNR~UK7Ob z@@rmNP^mhT;9EZi?|k#!`umuD>eDI4@35CF3wbK%B`n%B_j zup&gnzwweBkmO0(4&mXCAFBqqCMnx@RnvsRj&3zMoixg+@jEm!J+^SMrqMSjz0;4h zWoR*+hg1Zj?^wk4pOx-E+y z@+X7XHnP7U^L&?cLI#Cx0VnPV^uvTJZNFL^3e5D}OZmLV~V()EXAz)pt! zcOsUNEjbZi?P|M1d8)P9nde8`lmG~up^G4lSo%tY@)Cs)SLf53dG!M3WUkASluJ?X z=j-18n8M|l4=^wvHh4C^Vf7aln#-6H^Dp&~)M8bcJr$p)yA2R0*8L{KBii0kKr7c7OA>4Pa9h>evg zpW=|?7?cbE7NL~DLH}y@< zU^0_0S}onB$4u0{6+u@zQxd%mvZhPPS1=^FJn1kFGTszwNKAR;mK5^N)FtULV`=P| z(NXy?fsH=i;t-gtl#)1Ea-`5^t2++0qB~J_73y|3>TrCsRejX9SZ)Xt0v4gYURB^A5%5Z zDHdA@VfvaYEFrArdubK(YoIW}$73wqMFnrKt<8KwxQt!96>zMZH!jsa!eaDCVfDnY zB)mqLLQ2$6ZH3}G(sR7uyfO%k-OgdxLGqbaHI-nUG#UWc+e+?{A9Q_YJes+Z|FJezU%TmzI4exxm^88TeEQ;Gl zdrxfCMpHPRA7e4^#fH8qdZhy)+L6B@_#=Ck_p!rNvkiCB0DTYIdH&ctZ2XUw$tBf5 zj9$ztWB<<`nLtJ;B@t(ge%WLm&G7W9wv0+{A=j7)C5-Txnu0`C&y zrp_Bq#~zfsEV(=+K(){XosWLhxu1ZpDw*P2ME~7zFl~B-`<}J#;)Ne!6Ev!m*?y_5G4d!lwnu7~dBGRu|cF(G_mgB0} zz9zL~L{>P$yTQzJyhV&h`9Gd`iWpd;u;lBp?ca>oUM=i&8X^97z%`}CMA8wP5O|h$ znkl~fi+u6K`w{ytK~c74KdY32R14}ug7*xS{omNsVFS)ro^dW)+)Xfr@CT6UqN`19 zj9v+;D4jpF%`;j9Feu{KT)BlaM|8Ov$kx1rSAYCC^~EXVIFQx)vPo zjej*m!Ah~w#MtCO%T=&-=D59QQG1trj*p3BxG%U-R>tff@Q7^`NK{s-bvh=Mj*_t2 z{304}gSO9n(>p)g6dh$`Myq2=K`7NA=rm`LQU73KCaoOSZg?iGgk@#3XX)!wV;n^ANS zM7PMeSknWk*$@s3u>eiCQral7u##g0$#n~uqC{tG`S+-D7^GElD@=Snlqi^sM9Af-# zyK4%@4!A&MOJZH~R;7C-BnL~VAT2&r!=Qgbog(8g`XTAm8&i~0#wn=@g43oaU=@VD zm)mP|pVsp+HiH(L(FTd*8`T3YOrvPn98nHCSXs-cGQs`*YtETtw9Wsj+N?hai=#9e zyo+Ux2dVM+0jLt*Rk&*Tyk`}gHF7oJn1tYq=2?d2Qv=U@fFpg#>4v+a=$IjmatOTE ztvp4%x8jZX!3(^n068G9)2pUOZrHs0?*fGqtQ?VPRF>>Q0#z-30pKZ;cApM~n3rXb z2R~!)QTI$q)Q9_;K_&c44te4IJpW4raz-{k=(Z(>W828mZ>?-2pT=yti^gkOYhs>&;nt)#+DoEVMH%@tgq^*8K1;KT)5PAh*Qtx zC|oDtR4(0}8bvjWVFd=Yt&wRY%{F_@ON(0;a`RSU`|EF3e8hN?H6AS+qi`C8AA^?`&(yCXIT-Dnd<`46us;WEv9*7M_BTJepUNT0& z7GzM=7+eE(9ypP(7kKi%XuGpF`4&;XxW5%ci6+5@rDS}_q8Wt4T;g$K^fiOTu-2ar z4({kYy6{W-A&e|)T^m(Se93fO_K z*}#ByC_e+(ad+t3!(4sFZWyo~yvaL&yXaUB_HJ*?l@m>%L}D1$Vmzl)TM7{)UyhNh zej9Ykcu4J0KH%FL*EldnGCZlb1OkN3w7H;ZV5=b*Le{Y>Zu<3+Vb${vt>zuaDH;EW zstQ-NhYn(V>K$ zZ3JE#`S3s&d+cxS54Bq6E%X$}84@626#<>Gust??a+$HIyIMuQ0=X`q95!7ti!U2!cP&$fjf*KK>S zocePLZ3@Pz^@}ntnbYlpj4XrD-GIYTzo4QJ)Pm95D5cprbe(;aIrYlvpt0zDm>CZK z<<{Mu1q&Gi-?*%w5&(lTI&f3{JH%*Wc4Hq=K;-}v8NIy4sP%hd%rAl3h5IRAot8i7 zSx0FK;Pi60CI@*^ZfimbvfDd{ynA66T+;1rYmWx=`?CHo&Rsbt}e)8D@`2|HFf)|t$H2*MFbjrruNlfg41Ph zvd@tS`(kd^k$qgH4<7H$a@#Yy$LB|~bbR^=ynF#XPzv>s6_Y$ds%v3jPsPd)pEk09 z(`=hjVaFp4r_-0lsbZM?)}3EbFEJ9bN=yXYElPv^pk4!Qf_uzjxaLWaI(0QSJ`)C<>(0h=MretG`P>pFER>|2|q>^aHY^ZD8dt(DL-XuQ_^(xlo^-K&`8 zix!PB3WV7_q9r5X5PZ+C&u)I8D!`=}-@moy=VBxhKg^(uiY==tDrdZ1NZn}DEN@v- zmjiN*JONg@CSc})8VZ37YyuQON)(TU54Cn}I(59H)LdlfA>71x107Q8Ehfcp z903Imjw^ub8B7bioS_@+s_^TzWZNN~FEnuJPB|YGO2ixP3Zav#3;jMD)9$BbP8A(a zf_!f7=VqCK`XlZQlD7q@O6+9Rs*o)$Mno;udMDqN(M|NBwnn@0ab%5

B5#(`6pC zcc~Ng0H`zX;g<&?*Qu-*YXAJ8g<6{Ke99j4qi~tyOHj#c4WGN>PSUNu1jXR=!u)qu z*$IBcU%O=|QX227hLVf1PhAv?0!e0F4E<&tSo9m~e1({1v8Swt;Wg&0B9~<%-Hj&) zfVLe>JdmIQ6!J)kMQ%2ty=u7>oU}2JF+ku6OPn(PL!Z*4d>NP6gGnbI-E)ACd~sFY zI~v;2`C$X1O+Ee2;eAYj&@vMc4sEKV_kNp798dLgF54GpRntta|2?m9PH*J~^rEWA zKhAodb!Lgi*r-L%>XelZPJ@w)z2l7`1dLQu%H*%cf$rv3=-2%^8i{M`8)uIxIOh5* zkEQpYtP!01hOwGJQnbtiz9!H7{_q_q(EPlnS9h(- zWih*TY17>55$$8C}C>vxo4-o<3IK3%18H;2> zZ^C)>WRaa2<=SL(!es&wBU;$8;s;Az-E(mvqMnV*#6-~)s|69HPO#|l*ns{*rQMR| z+yl?3FPQ24&mtOSsa}i3h^;?=>h$+qsatA3ciUoTMe>vT0uR#4S5h&W#zy^zxyp;j zOveiKqU>A&9P>t=A&ZDv+pa(Y&HUjv%ns#jfgh2roy{8zu(^%w!8}+Izk{S^F3t04 z!LqUSrU?2d`sqtW;&t;M0c~6C|EHPSVy?v7y*_GP?@BkW0P1YK1x=lexnHYN@?g~? z6uNxgkc1#1%(6*&DEqv_Zl)Y>v2PoEHh?J~Et zG8`dL?IaF;@{fdRiVe|7;!W+CLkGbpiAWI6+RoP#K4*paYpSHLFt2Tqu4}Z|dhh&l zga`+hUbEB)78PNJnd03hkN&)QOrzDu#Dwb<5>6Ie#qPlayGNmI=o^VOXsHVjO{<@u zuEC|a=>fmM>xQv8;Z1AG8@YuhOvlw!IN8FSsfJLz->Ij2D%S3bVYhZ~?RbY`V73FL zQ1`#vhEcX6zPC&hZ92^G`<*r+qmglJvf9UvaqU3)!`hyM(a+E}D-^tw7MXnZg0ILb z3(l9=k+hf3z=u~vKBC#8I%n|>X%J46so#hQ5&0`N z8|@rPhV(kh&XbYFk8>@3GBo>a?q8QTu$N|~c|gFRT}Usiq3SO+SKln2WmUZ(M3i<6 z33fv(<8yf$SvwZtPrHW7@Rq^t8blBk*;)ijA45B_tgoauv=MCByZv$7$FMNXTBhwE zlR~yNvXY$W=Bi7flR0Y+y*sBd&^>AFU~SMGh*ohMxyl1eSF;^2h85Nt2D@a10b580 z9vGmANkb=B$H2kV8rCcW^!YDZ?f*V0kRxJ?jmeD-w-+BC0tO-#08zinz`n2g4HJN5 zaqXY=+yaWV!fi0X53&9rX7ET+&sSR2;n8>y_bwS)jPI@aF?2giy|0L#3?*a-bRT{P z?2EOHF)?mVoA@NS#83SyYQso~-KAxxx+L)b%pOlrMybp49HJukRHZ`hozj)M#4Z4( zgNcGYx2=o!p%Q^P7<{6ai0XNR{bkUnrD=)0pklI-_nP!4Q7f;usNVd%ElgJ76X_69 z?;*Stog-0_a^+0xJRIda%)3Wu@Lzf65q`ZMZnTE<9t40Rx~2{;G-6@}8z%swK7x>M8Ja|C?MDIvv9^y;5E?> zkp&{>D+SvQOVQpr>S)C$J$q^G`9J%bn^`VgQT7+B;KC?Lq)kWt zr-{!LJMG~fgjY{h*a+tZMBz@*!|?-hu4;wV?$^CfLgmPCLaTG{Rb@_}6cP0noll&@d9yR4 z3GvLASn{EWdS2())U}}>md1Kq?ehq82jS!UEr|;3REi)F2WEH~Zi2qZ@_i$h3ik}9 zO>pf%bBaOzGs+05x$dKu%9|E5V%bUis-`W}hSCI-`of^>2A06H5C0?8&}oQe4HeMk zH<_u)6bvK-B`&E+EshXH`Dp7o+Sb1D@esWgOGJulqvhqE8bN4(P`Y%*a8FsmA6O8nB0>kYl`6s0|BnOflZu1WhYO=rU})AHU6>{9>`IK7%v2I^D2~v^klK zoD9pSn5tS_H^hN3Ul4T<$?;xSQ0~sZ7dpaU1|G9o)V9=h(qvPXiR@yn^JCLn#*srg z64JFt8?@_XM^=P*d=NLxj?o}yDhyX^JO8Xj*wCBVb@?Koj~BW|3<~d#0=9NhmO&cS z5{4vn#-7AguQ+q^#A;hMTJC#dUK9qkMpx$tUFkEvcnrjCA08ay_%9;wGkBJ*fRVIn z&+SW!OnN=R={IX&=ihC7h)5^ATRN&EQ3dyOj=Os$@=uky-@})&iYvraffmoONu^bi z2DQQGNN26f%f)XGcy%Yp*U90yfU8bc&D%fW+G&i=>S>w#wB|=z%LS)}5HA2#>OXBn z$|Fu_1%dR2oUXKe@?t6*xxYt@!n)bXwBBG$!{&XI*pjGySd>Irg62+Jia#1o;ZBi- z18^v+J`uagv=t$_1ImHX$_R-#Jr;bL7Z|D-p%Te~Fm8xY*Ph_Y7WF3j<94QzFLRFf ztUPtJs6B(~Dp^sgsJ=`u^d`oyZndYG$er&NbMG`P<-h=jih=$lP99rFzBt7V;xV0c ziPkxe9p5@&&ouk_Vy* zW8$nP;e-aW$%^bpBnq^)*lI{5`{SR%9}-|cJB}L9{3ceE^U-P|hE`6YrD_=su6o-0 zwLuVfjNjn57_%nXWfVo}HnDnDZkxhmgcNa$v??riN=0PjjdQi+s{>k~`OH*~)h7g& z0(>_goU<#r@bz=W*a+qTm`xxIpQnNxrf2D!@Q1*u{Q&{PI zeF}@V$c?JLW&stq-*p=k6n!np_vDb5^)#5Z*5YUA!w9JP^wdwm9w^n|H-fM3d}xH+ zU#levY_g@X*51WXcIwFNT1t+8LCh3fKDQ%Mzc(-#L;EY&CC7NRZ63DQ<+HnBK>d!6 zNUYldM*%KLqzX2g&vmQ_O=Ecl{S3G775o2fQ<0@?5PZ;rb;GjxB?9%_KPj{WE8(zQ z7cRwQzo^iQQ_O~fFxY9?#}#Qr!WCErEvlyFEV4lY$P-Xi+}UUEi?I!R>k$ zC3EH3#=C2#s$F1))W64`Y?`~ba{We$`y`>Ue}v?`|Jb6voRxL%h^5G7pI1ViNej5< zl8iR-U7DHA_D<>q=5LdKyK}0D(N|;*r$_UPIC_g~sO^RmV9g8$|IN3uKLyaowz0mS z3M83_BLWMuiD7q7IJ+A<8D@|k=aq(jtp5g=0JScvEPYDAsZ<7#qN=6Qvp;W-Zb$z? zuw8@Xl_JCoiwjis6ygQRI>)6rwxCv82XvYXXr9iH5KAulnvH9Dw53;E=D#NANXiWd z!;o?uIerN)=V@jxMm%V&StSM@z6JGK1Eb@iKE7Y54b`0X>oH{V2V@^yofiGL=KNrl zii3^t1^{J$`YS53`@m4s+SlKft@+HABt??>HCEY_^g9@d*?USu-_-(@5JRQRg;P*I z^+WxA=9t~7Gz?H|8XneSJ&H=kMoU`KLw6+5#A$>xAhRhnK%r4Su3NV6+laL`RksZE$N* z#dfF?)TwFDUliXooJTFUb*ti+|05c9n%@!-oi$yOi4(>mUxWIt$5C$C#QeaqoTq8w z$4prbHGlPat57pz%6TwKS^t_8wr?UUrde$Q?5{O$EbRvLM(&$env;Ds7ur~UdC`vo z1jR(krp{b^K84s{&0nL9P-Q^t%3Mi|-(`tnso@e(s4Wk~9M8d&8%OSS`v?lioAJ(( zHQs^sSV?e*PwzUi_-80znxE=~lXQ*rNsx>;RX7{A*phqmNaD}9YB^@3d6#97?SN!h9;4ZRgAY2`3pFy+ONp#y1+xzY4KFmVShIZ@?Wmw2io|k z4ymY}#CEnJo1%j&|G|NF(%34jmohkl`H8wO5&XWQf1kazDwY=Ha{Q z0NBbBUH8=&@vP^cpg{Unqk(nR&B!vD#3OK$86tE6IqW#s<*%slgAS?7uaQ=)j*kli zs=PtfXeN#MUyI{q8Q6LnC=~}0&c*|L{Ky+N1Z9ij=Dy1anA;5K53mLm(?0%Y;MNxy zPzz1QIGrx<%1wk;{fn!M?K)9eN{K>x&`U9GJAe0rwL*@M%ig=@4>{7PFAnhm@}goQ zP`U50GVV}N>RC;wiKlHScYoN#l9cz)qL|OZLkV&@a##7#xsW;aOa!`0i1L*q=%`w+ zpm+l9=1l-MK*+y?61^o~n(pDqak6@Es?J%Gb_>vVqL*!cDuXwq>TV2N9k>Qo55h81 zaXL2=nsSgcbUc>)!LD>CTD*zxj!VX z3>M1hd&r8&^i|_b?aXvheWQ!zW?7?{T{9PWqK(Y`zsLK7Fer6fglWmy)EVSxfCHog z^f{V9ym#pzuMd=nrrwT^OQ}G{eN++c7tJCb;=})0&S-_==j7hORU4{cW3zmN$d`1+ z@+(>pibMMbAdU@C#C|m}5ZiDDT?yG5{d_ye1)5lYC~XahePBfO1Mv=`v6H$Ndsn7w zFp-6=pcJBS_8iG&r45-UY@F2BomRUbjqKU%Gra%q0wqhmX9VqQTFO=+jB;aJ_n{Vf zX7!{~eM$M$5NNfRWJYs|A*PfBblTzgXdD5JlWJ6gV!;~58Ou5!J`Js_ocj1uL+ZD* zBV{!eC4peBVXYQggwd`~f4ID`U-T(Z9cv?B{ISLh2X-vfWZkl3g26AvwqEH+Q4L-{W~QLveMCi&P$6P?Z>j|a&o9EjcO30&rtmNjIoAsx6`%4kg}&bA$BB6@n0{xDat;Sv z)gR~v(|dsmm8|lxFxU{mPUax1TA9#Dn$0P9;Q&4lMrl>uAPiS3O(BL({k$!am{eiU z;!XiAZr3n1=CR^c(bf9wrr@qxLRRW~zrP3)H!O!|JF3R@Br^F!gWx9DSQZ}!rP@>&v(gOT@ofRX-cUV* zKJuf&7cjlM(t);fz(d12#BBsGA!KodeSUezQsjHpBErFZU!%sg11NJiu=j}-bpUP%D)lv7t8iXpT=OE=wI>{Yay1nE~;rc)rslF!dD5 zSok%CQ7)KUWAW#vV!z=w5`}z(7=%Ovj>V_;wn{ho12;cdML||>m61<|P;F8oNI$$Y zmZV}5fIKpvnrZ|;uBVr(ngfd8J{a0?rVFZ49QSBjEu2LmQU2hYwp$p+~@!WJm8g1`wxI-uRW*FP+a28wASZ@pk zj2=nE_1F7{tFq!N?D!GUom>t+VCTfQjX%S@Q7SpTk(w`XE$K4Bm8YbA8x^q&;|dEJ zQ6l34$h!Vic%Dvaf>1i@xIq5o!Tyisp3mUX{E}7p=(6X)4PU3;PFOX;&lN(zDhXFF zC&y10>I;+WcX%h~S}L$!%dAXLn;Rtc_Xz&^RHrNgPtJ5bIW+p#X=zUJH&vinToX)~d?@XI?7V-uolz5q~rvta(YwTUfwrg)J6| z<8VBzuT)~)X||WL;IvJj?3)=ipZ8Hc5_=olj7;f;myL_Fe;JxtvXJ z!k|+H300`RRt>Ri5!yncw^!LnXmTrO-P5ENYpDW-d}Dq+J4Ow2D+9+T6lxRX7%RZN zzcP+zotDqK13aAFqlt~lPt3j}3PO^5TvkoScptRXm4fmR0G*bf8U;PEc7K2Dg{3X2 z9$u6PZ$d^`eQP#V9YlR3iJs8sNWX7$O+X7#eZ7R|&;y`{aT_uV~{iG_r`uje5rwKL z1oJSmh|!|WCWJUsNoHnCHCKp`_Eaxrn*RcC@GGuglp@Gwoh;H?&TaBao>L*Lun zvI|pGqr0=8>S+keckko9V-Z6OMv%OTHJX9s($@n`f~U4ao=>hbhNg&|2eWM<+3o)T z;E3IkKuEn3pW1-3LFY9Vip!@mrvkosiU1V;FyML{G_$Wi=?vXk=Lpq*QLP8o88XOn zN|r#43Y;PjqK$_fHpAi@hO4sbLTAa~*Njs>k%o^&Q>_S&Y##up{}o-o=Ek^E-4zl6 zq%?w5ML=6UEuC8bkU@}2w(kG1i+^B9q^zS=DrcZDTxPwDXfRkp;57J$gXZ@0>nhp; zNFu=nT|Rbe{H~~Fk2f%K%H`;BwSc{oBqg@990aRVE>)at69tAFW=>MoEI>`@H(7u4 zb2WdXrTpa4eQXcw{IH6wRHbDptae5xZ1wvWAlLV-*=sK@W=A3rh>!e1?>p6nvlMz! z(WiEOr6=RsXM%ymWuVY%!MnjXn;5Dyk}5eTQVW-4>NOQD9GYn%{ioN$oOdA2@;u0y zSzs9B$GE_93q)y~O%9%W*OwHi+=5>piCkS#o(ExL*Jpp4FjJ!up4f)3}tAnXrPsgZ`}+n(#1a0WBAZv?&TD0-pcMWxXJanlh&7 z9%iSwAnE+mO)wxu@ApS5AKFE(K*j%Ch*E#xbogX632>SBI3N0^&B8Lx#c?O#RsBqTTMtIn-!`Snw4e9oz&nQa#vm1QJk8wgyvHjOE<=AN1zxLZ(DuyDHa zVV-Y4Q`Gv3$rgUlYD(+pyOb86j=Zou+0N_Vi;RX-+}zWWrUzj-q7g$EWKNrCeBcQ| zw%qeAe^}%&va=BgG3pwG>O;;99n?f&19+M$fkjM4Fo_jDA2&VF3pw+#^4E68auT+s zj2n8NsVCvW$-ll%zn9rv3{V0_4-6Dt!T&szN{0*MY!@#Oa+DmK33O5cqNsq$DP_#3r*-I@;ySe zLLi5m>rS?)-}aDMf>$7{ddKu3>z4LUuXD1h?v&=&}&|eeM`2fN8P7FB%-l^#2+G7(^1SCefyqUnwr4^$P=Y;^WB_^9*8%8;r4WpCsmsj%J3#noStRYhTV7Vts6So?26G(>MS1j5@C zHk4I>f18$C{vh4+nxmQMR_R%tOV8Ab9G9^1xR3iUV2DHmqzg1Sl55DmIaMK zuw=uSq>jhuwTd1ErNpYF0+3xuVAbKdRYZh{ic!s)z*cY2k7&x|a(M#Z4fl$qonV?? z1P>o+4kuym&4S&j(-`ldW6A@Gy`yYBHSV{c1AX+XoS?L)j^1Yg13>)090A(eOhlqp zlSwxKQmIiPPqxy3=gaT3X7+4dX(j$F+Rw_M#bp?Q?_B;)5IxMLp6C-O^}LO>K-_~? zg%QhN7vAhR1CkYzbT?L@`7xclxg93)v{_V~PPZu(RZ}0QZf`CazH zy+VY7*bk#}D;({${3qQyC=~#tW&q;A_wSXm2tnpMK6R!z;K6+otRQ1tjPviEa#Jh~ zsnn0(x6^zcXzZ|bo8_g%`7=6eOf_;p)h!F>Da-9!4Ig@ff-LA z!!jh`$U(^9{lVZ#fj&DRcyl@h2)n%mSp{et5YZj{FiCByNxUP%Lx(^YdO>m4o(OQ6 z)u3qKLqn`J=NR(s|h z)d9jdb;KtuDqIhwILK~=Gv3^6z7_l;C5dkgpEEyT9;Prp5X0@1c|L^`3*lCAEWl_S*a1KTlC>4F$N*g;?eYh? zgCd-NBf#QtL@)EA4wX*6Si$!tw_{YUvHHp#4+Q+88;4QtSxsFL&=&LH_L&uF(=V}r zQ9%lSQpqxY6~OG^s2vCkd(aI}zly55ZI&h_O~;9{nf_$>+g^3Id8^aPcCq4gX;;?W z2I9WkHEpT$FA?3#=UC@!2WS~Mh+QAa2y%XNEp|8ThGMIOU!kX8l$~U%m@^6QayRUE z^syI=2_7r;W{YzJ*!x)Qo}_B%t$Q=HHAEN-yVI9|34r@edS&kn&Z zv9-dWHCDFAg1u)P7R}Yi-OMsNu7!y%<0(}d(L|KGdK&sCBm(VZ=?ZtDBrQ2870oZl z5-iSAp-FbYj-*pXaZ}WtMUcX@7!a{!dla`a>Y@j4E~US5xSKpKp5+p8w)T*!MCDNO z`v#Cxy}0dq_iv_h(VE{3JBJa)`0Ahb?Wav1bnIul1w}&<2r>`fN3oKZy9dkqR^%yS zkLd^C^AEajQuG7SVtJi2u=l<*Pr?41wR<%|m<4A}!&$qH`b?;;RW=7PFA2lYIsnCIP<08<$ zL|szaGc7Ai3D{2*6OLz-@f`-i^rTy>xb)!5-1@acLlY-~V{s#JsFNLG_bqL|N7vyY zy^Nc3Mt$cg*5K5HbRVqSVCrrTgZH8a+JwZ|BU@9-P_PtdiJN-a>qIt<+_ZNgvHJOL z5S(32Q)Kj;_X{?wrb(kWB~Nq=pi@G{$95W#SuBM zxV}yppsuiu9UAf=4-#I(R$mEi*;}H*__YG`?Y0Dx1-Wv!8%*Q08RF{8P+EkbKY*PO zleM4qj`z&>RAUJTh~ke!JDa-mFP8#GoL{Olx2X#+JT>xK@XJTe-lc-9 zq80b!_=@oZB0;8m$zTw{2?SNI!!1#-Mg=VDRz`vH^RJD<`jqsZl{&QghhGrI)Ik=B z31z*1ZH^D<8nC_QzF#6|onY9+e_1<2J2+P8leTT-;o+fXky zuGjA`aKo_g4_!x8H(kTmMr*Z?)TEdJvqYMdzTQEXIHgZ&REU#~W88ZLpqxSGKgw6?sz~-na2Kol zL6C~%RHEB!S11dP+T8sr$Gq+b=)ZR*^ldZK2PFyOzeK{QVrS^wkHeb5kz;T;-PNU* zTJ|Khyfi8D@1a_%3 zzkz-7BkxFGk>Ut;!o$@erTRk)JA`t|V4#M}ahr_twY-&}!-diw9f*m+r5})2^aBPO0(DM$)0v zg8}72di38xwP;dH)x7B=PB70TKO{D85*GM}Lt$+TI|FK@CVY1-i$%yAubk(7Qn~@e zCAPa^LexeR4#!J-C`-haOD|SbQpb~oWgtqTxc*7`BR{1q_0j(87gI?4yuKT0n_ZxI zVr%~wR=u9dW1T+e{!pwm+qztiOlenH^a~r4R;U6=)PZ{}g zfRk(eZ!(Z)WgI6{Zq9N-@kC@GXEazdvZFNjz+hYvxU){`?Z#^=zIW`7NHa0H zL*WK-v1auj^fLNRWpa_8Mx#H%K?Dy-v@V*M!LTlIw5Ob+lMlMfJd8pa))ZVIZiW_Q zvsZcBkWP!NMIC7}1W*G`_N-Z_2NxxmoSfJc1;c;lsDBgz7ruoctBVURo>xvNtRxs^ z)nqn_bR|O#iKA#<#cp1Fidx7yLksB=zg4y(`GsqQF#y-}3r)^cnw3JK@jB@A*ZI_i zvRsvKk-FOdnc%Q{NGHXu-6ocnh?`}?&$ zg&L(~=&4IbA&VP|Zkcu_Y$ys6lqNQUeA}gigGibl=4~N_p94_WeTB_Ktp^PPt69bp z+DNW9l&w-!=Ig>qWWm?@EF4UH%&X;!s zPM>xI9};P%eJGeyg#D$*Is)CyJCxem6w$Euej&qziy}Xhe0oQzQi2Oa{fct-US7TA zY+2YlTs|!<^7Vj=k;QDgw?I&hHXNaqCZR%Mws7hh>pEDhU^gVI4w$=qG00RVUR8|1 zjDIlL<{)J){*#G9SMx{l>G5p+(iDB1U$DzhEo9K_qxV5`j`042ty~dfI-nyzWJ9-F z=Bs#$)11idHL-Pkzi9OhQ!aJbJ%Yvv<9141_&3jr=CW0pOogv9t3BUDrIYkrqV>a$ zOY1LU8e}JrO-a1m@T&K;dt0=-aQRW?edVGopXBq<&+m3z9|?!$?5=rz7Tpe#GQoa* z`l9%q1Cg&VyTArw^jT|XBK6|+A7&k;ZaJXz5hi+lfJGE+P$ZjLbp==2V4g7=A2Q6y zkA2oNlua#UgO<{LYLCmMe+S-so7<6PSNj&1K_xvG6aPT7)ty-|>cw{cu$NM6{M=fP ztH@|+9F3A3koCtS_X5;&mm$LgLPy4D`-#p?gReIw|J%?^u3+TT7S2^d{d*r_C z#dW`9u9QjCc_WGYI1kW^7BDeW(BzKPbu-^w+p2d@RcU8wJeKH|Gfrpy)AHNhFXiLV zneO(uyKpbA%TWfoQR}FxlLfrQ@0s)+-HC?Hj-_YwBB+_}8pU*iuDONOm7KFMeMQt1 zY=Q5=ybG!Ry6vJ_qzz(k! z#66^1ny}qI!S=YG)K4G;dT;j${u;A9nUGJ~m^}Hk%8RQ=6B>kF0ZWG?!J)UJed8 zggHF^#A__F+=7CAYRZ@)0YN6mU1ZQ`=ztTC2!KMx*k%AXOs*st3gn z&PKhA@&30}T^dOu(>&mvR5vnPbOQFZ_qO7;8ls~zHh7=)4vD%u4Vo+E$wOrP z4M!AXa+#LL9kC$3vllr7??^c~v0ZbL41Imwz@jPq!D!GvK$o_KWZXN+s44 z4XMTrK@}WiOal7@u*3w#Ta1^46L)G#hKB@XH`;e9BE}n;1rh5&Phce_OH=*%~@v# z$36<6pee#~0`GAUN3gcU(`(P00C!O^BB(efD!*G~LEl*&)xe{t!2LoIsP1EMOl``3 zq3qn*VC{#v3soeoAW@Fe*17_5rU+y0>j|1Tc=Kip81<{XS8H6ns5udJK;( zx^-*J|5Ukn9+uDAREe*tq;F|O_UxFf20$4>Iv{$cBKEVmK_HNj0F4v04Qo0zf|R8)s;a8T*+-marCy6t_5Z~q zFEIil^P!{vYI+pt>DA6^vLfd40l2a4Wu{s zz6AYA11Q~0QPQ0pZ6TG~F#k2dDQ^pl_XtI9>O2&TXVP777iAveOBjR{2*fzw=Uh_v zC#128bO{c(A2US$UZ7CAobp=t6{g2Vc&I+-URNx9Dw;$JKcr^mLOuMDtvgIdMOV;a zi#@(abdtbMI`JHxKIW!?^Hbyl7Q#0an=#~u3kqQ=@A7j&diPg7@TV!KkA5MXY;JED zJ|vZqk$Es3zOhd|6>uu1l5sDL)-*otGPkI)o__*`*XS>@#dqsBypf^Sx$rb=Mi%%5 zny-n901ClEefub|8MyYdNiWIhf8(v7rNDfUSbQ3tRZ&kBv(^rI*|zUyP&;|5ro0S# zr!E9OB0OJ_8cT=8X$0?G9NNLQb9exQ4tiDC{L+QbRk8D61a^FU4`YDkM$0M%hi0yR zkbo;835gtXP2gy-JRGb=-L64-sJjiS4Gmn4-fk{Xf|5y2NpPu1>V^DUt%PnHLLI3s@w&8&Foa49qsr+)XKHf zvKZFmi{dP6;Lr{t;=rU08(*4ff~Uj4arFlAKMWW&4O$Oak9E`6A`TL}PIWd>>CA#x zUT`YNnG9HUB20|x6~CV~=x|;^d$^6XQxsqf5I`c&svR$7jSWPT&+kkVW7#Uo4#EhNPic~Z=?Yvd+f>yj*x>3|bTatT3*M+BPwGOK9=inx7}j9`rh>Oog!l+QO`zv{=f1fl}eN6T6AYL&uK zMT_ot?dNfeX~)$G^U!+G%UgaV)R}qOxX7XtIntsLzHrq9R5tolC0!1<$9^7 z(14F~iCg`ErHG6O>`|W1Ld+g3VJh8Uvi(4qk2(0~0l;|LJb+Z->^ePbR2B)-fTX@uS_a_Cxq0;DCGa5h-Vkz{H1B(C^c$O+&anmdSM8m4hRnV z17Snb9_N(SqmruXmUO&O?@c@Oz_d*fWQ?V1&eRV8eCp^Jf<%a482ep;toy5170Zx* z&EM?5uT^HzBTB4X2p=c&fJZOp(5ITif8|IDM$sA7h#nA+(!`ooSr+J;%IVpGc>{vO znk;2FT;+ddSAY2Gh89#G;&Q}~R<)Pq=bZEZDyvRyB?r6PI*F@=f|7<=BL1!%1S%2u zHH%CJrQ-tMmGesg$8I%aP2R_Y+w=$uBe$mshj6kd$}PQ`%fw6^4e0(*b#BHBihd;b zoX6ChHs`msl7uc+b$hP@SfCB<=NSKG=TCG_S@LzcnF_)iQ zap7)Gm^VZu=*?@k>{v+4Rqp$*t2Jj6e2&AuU6mDPR8!H4I4pR;Q7@I?#yZ8Mo2cI* z8!C9_C&Js(PCbw>XoD>~e8^e44mFd0oK47`j@Kr#4zMAcU#@#5uawg}Vr9kq!q9V#?C6aQ!jJ7fB1tI`8eGWRkkZ5VNvy&29+$h{sH?)qWlT-8uY~Hw;^z=X9koh4mm6(A ziNvK_l6vHO>TtUOKBT6;aGBuWhIv6Y&s~?b0DNhzK zaQ@3*?51lZ;s_)$&=m3a#{NZ5snnXK@<=6p!L*!R51^hW3p14 z?O3y=uKY}PJ*p2EdcvD;##?a%Dh%4uunE#%wQ^0MV+hV$)eLL*mydlQ9tRgl;2j!JrJK-h5Ise{HxX33Ug7#0=+{1qn7PY$WL<@KRoN zrpm9houmi);E)%CW_`}X71prJrcm=vTWMJ!w#2VWEKh~U_5wdvLP$H99o5ci0acfj znb=md{@0tM(Sa-~jOj{~<_|PPkJ&2y6eg)r4__ESb4IHPbo&60FS3c^faZ99 z1<~b*s4N2I1LEZEof;8`JPSq%cpMI!TgR)(aZ<=p=gA$+$0$yeiwOEW9r_<2PtGA{ zK3tIe0!(MK(Asb<)uau97|P|sU76i!TywI7sia)qgeN7rW5nK4$a0t*FA3h`!jm4LDeLr@WI={j7SVCZFeDOE0rQSFtN-Uzw)Bq-= zo|U;AA&HmbtM961{82T+Amc)SEa5a?u;vhkI@MCLD5F&9_Q^6})e|N2j z_C^xC_UYXf4+Tsi1%fMI4XAc{q+%=RVlYi5yY5xVKHy)V5jLAroo@86e)c_{j{y^_ zRhV`$nQ`M|#H~l#%s3P!{pyw0BS$ry%C(8*7H(qBNQTgdvBzXvYe;z@vXmDt&>az& zagXU(i_F9M+y!IGkg$nB^K$-?R{wx}{P7x;VC7NkBBH>XpqlWUszh?Tzy_ZcfPV+5 zHV#(Y-JY0Si$fnSQ5N&k8DjqktyIP80!|Ck2TZss0paA-))nE=H>eqVs~SE;wnfeO z-hKv+@Y(}jXzBB6nmen!g>jlM z30nuca-3#7j1`bZx>bw!iKvU8KHjPZSJ!?mvt`wbZPo~1fIH^&CAGo4!wn!ve@ECQ zuxJ^8PN?9^hXkyzbdc6#ek&~PL%7W&0`R-G@BY(-D6Ty0PoyPY*XGP#262%0Y0i`) zY-WZ_1YeGt4Ue6+7dY|>Q1{y3#s2syJj7_O>yJ`rkRiIoFfvv+TkIO$9~totRCr9f z&s%~w&r}q#zJU4fcV@7`V3DbzK*ynp_<)H4=-^AjcIhg`x%3WYMb5nY1;?7AiVzO~ zu5iQzPjYcfh$URR8RGrad}#=T=qj>zWQGR4Us;HTsJh=45iYxJOD@Zo`AfBP zN@k1AK12Q+nu^qCDhVht9O9`T10H8;%}S#~4k|IjzSBqVRXJ9Z9qM!HkBKCpUb3$> zpk%X0f=sn#Ql^K;+>adCAjM%qJOHZ>uv0Wk{bDd~(}s`Kb^`xHp?6YjVlMP{ zuikWlQ>@MhWQM9|BGq!`a#3T~`s*-ry7wA}t5=4NLC46Upi-JBAj=_$n5`Jh%pxUo zc_-ke@r^&SYLNW8cp_i&hlF%6Ra7U58NR9#q>Q0IK+w&qZ;TTIN z&h~kmzTc`?%H^G;g;8@z5mnb z>$225K?>Vq(L>U;m>?xIXlB1hyU}8(x=D%F2+gYo>BmX)5f1Qs?S<(~MLU37wVqt7 ztZl*LG3sbEuoAy>(mf$0T!(mAaqBo0AuZ}1b1zDc87h12bn58F+dW&c?XsQY15 z=pztdW*SHZ|NFb1{CYxV#!~%R`;iVXua#6WKW?vXIXn zHXve-8e?QoCEz0;0C_813SFD7nNv6lc8iWv-4n^t`4vcb^ZICn_FR# zTK4xv@4s9Dm)}uQzoo=twDK5nutChNAuOan1Y1-L)~n%b=~SvA{DA6FPA|ht=#5>Q zCGbVAJzg7+!|TU$yM1A<=t`#r#^T9@fKBGW9u*CLre3wfu`jSj&-`k9rdg|&&Lom>vI=99fPh%cM@^ts6cCd8DKH= zF#r^n&F;)01!z}O_7QI+XEQMj?^?_+a64%heqF?TBuTd^$4O2|{SPu2vVHF?QnQn* zSq1cN<#7-gpAouWBG3*q6_sVZGxsvv9o1>R#C!-y1ZZ|lCax_*aX*kStVh530A{jL zI0J3nY!TBAKqMsn`2Klb1dFL{`bwAm{W@nJ3_g-QFGnSRmzBeU>PDs>W*Xn$%TZJaxsk=p#}giQfns?Vf?SA}k#AX7+Hk%9 zC`srSyl>NSCP4~r9`*cEK`e)^VN~aLp0I};L<#>*qt%^x&~dS*h(u0nc9O^O5fY^a z+&*06OOvhw>Yhdn>1|EQ9GC8RKeV&+y*QIiFg>Q*5$;fWl9S=X`=jQ;VQ$iNKd1QR*7b#>DvMvWeU z&SK>zE&M|c1E4mIU`q?L`xt>zNG$kIV-Z;>&4u7^d?=4KnZPh8e+g_}W?rdaF27xP zn>~F%2ltwQ#lg$Qs&?&@;ttIXkip>EBCvnQ3NcI=449pcJP_4FraO`n0hhU zX25D$%ky{m^cnF)OfB|buoiT;={EIMMLX86j9JjY$EmVd!_K$T45MF^MtdkTKaq!g z)*xGW_z1l>mSK8I#n*Yh7>rreRN;JW#LHTqh<8{Bi?>@*nov-B3q7F0`ZZvZ@``gl zee1AnD$c?jhI!n2k5wryZ+2q=Gf~vUVUUp3T7WG5*`YSLvN$)W7+y7|WX3{r|ybWS+TnX+Of{M!tM;qKnU`by(&r20y$mO_-b@Uq?hs30liQx$TAW zN)Ru|e3I7&XZj`u18jH5$Uu?WpmuML`tal)Su~yMcVbscV8{!`- zhDM9P*JG^+96T`0Mtfd_Oh7bLnfGh+NVUC>8>e%EjFQ8GTh%p$lEf1C2!I1KdGYWZ z2TBFns3a{+$6RZ~LjiIQ%k%HIzg3aGJKvIh zcjq#v#X~_sdXw}Yry7-&sP=r*yOIo;{{QaJ-eg|AMd?FKq^ZIUkxlif%qBlT%iHz? zll#4kl6>awsUDR#Ji}p`_fi_xJ9(O1Ip#Q`2`x42VJ*r)X)#uZ!CqV*ebFEr9xQYj z9&RrDBM8|uMoE#C2&pM8}UB5XHtY+zC{Y*%LQ36OTBQ##N8?Naz-ja6?7JOu61 zw4R$$65gmf+nQDi9>nQ5UE)VKFuy1jhmFCt^?c_Dq9gfh%7~m2ou7jOzkeCOyp%CA zCou4vo1LYcGmBaUJ0O*24-zA5v?O1CNgtnh837pZs%Vr+oSfBZi;xlSa(Te?td*71 zi%8H6j?Nz8J{WJ-9_)tR>qE{T)@zVmB2Cx6=++0TT(??W{%J4vLOI+BPIIpydfE@H!mASKOS=NY9?t$M`nmibS|FC-j9+KO;A81_O|E6%qGxXGTxA}lP&;%i7jiZ;KX{veb)foZI5rs;0evxmuy zGK#?6Z((FIIxNjQ3cA}6*qy`!am1u}ODG!H`9WFLOzFkbrF-kXc@PPzFm1p3>{rQH~cf8EwTsQiSG@xeNy#P*@a7 z5}Ljb)<0{!lE`6Kq{}C7{#Zyj^*eT+wfAuXOJ0szTd4*{J(7Z?1)E*k9!A9e=Fl;B z+HqLakdrp!qm6AA20%b|=>r6*iGnu6wKvxBa)Wu%OFG5U)t+Y-riVMv{n)(3UQTIh zwi9-3uH`5E<97UDbndIlm4cf_aN_}$h^0&d-euBm2L{~QYqIzbT5BV*4EgJF3BTQC zKr*DPB6T?3@pk`8bkCQIs3bj({{5OY(eo@&kt@2RQeC%A2&ghREXlDj-rX&s1Ps<6 zMJ9`(@Akf(`7X>A12^~{_@3^@U z23<+JWIIccx_T6&lhoOg5k*y2n%t!@pAvHLZp-`PW?{h+^fC%V zt}o@Rh|_jw9(@MaQY8AoOB>AxkE({k1HrO|sf_cuOg8^JjDWt(gLUO&Qql`r0MXo= zCE9%rMr*5F`a)J5N{IaApSjz6E490q%qWgQ39ca{p|npY*iN>GTzvpHv0_O)#^E1R zfF;`>n28=#P-`np{h^T(+{#birWZPCzytCZ0Zw4Vve3Ah3S${xBt$GB#;PFPuBcKE zQdUmwvEzGG_;vM2{+M6nsm(KR3IGAc7^%}mOWHvlrfSWmvTHn(SM{{JVElg#E4_|W zr?w{6jE$?Ko?2o7HiV>ef(Yek0Y#XFh`mawnX;R_jB%#0`jW>was0oYY#Cdbqf+5H z5{_mO=k(vWyI3WOR%3bb+pB&2i&GNbvs(UHTdw!1H0vNUe&p{_*K*J3RwxChaV3Bmgho7PyEtM$yOgwba zPRe+s5XMK{J}_VmIo@~*j1p5mk8t0G9@;C=%z|YbrUP8Z*3s(@{2Y>E2fz?e*dqu{ z(;v@70L!X6*NW%0qq|u_`#*1j<@d*jA^{o;SVH&eN_jg%U|m6g@I;~Br_CZJ;c_I_ zS(ZuaNfS9ASMQN2w2E4?^%U|D29KV%FongfN1|n*2lhP<`843-L+f1%U8*ENL^N9a zQ-7ZfYiNvOs8%*stenDWoVG{Ky6Yb!TI`~wo{0j^y*`Cwl&l}x1*&+SP*N;`mXa0~ zS8-SS(?lGO+4vzR>*1R@GKXMLz_{3X*%4J*+JI;E#$DBZbM&Xj4FoU^;sY9#^#OCv z0y_9G?G;k*kzd%7K*4`rERT*f2Pa)C{1x}q@8r+wHkunA%ad1hBhw zSeev}a+L&3=dS(p^6Ad&rHj^O?$u0bxi$cem@-|6H|fyrQ*FGtz6UdY<7j7Gp9i_C z=?@n2QPs?rL#AyrM~sL7B}2|{lG&a=I4~*N1jo^xfG4s{ik9{EFgR-$co;sd`5Y`g*?CT%Xcu2>ZMN}GvXM1F_{WS0xgXxRW7q#plq*r3pQ3CU@@b1T%7Wx`2S((s!5HH)9q%T~$Y_sW&$x zJV2`_<&rT)d;VYWIf(XI+fG-4ZQx5!j~gv1?hHla%!bg+VG@c&s%J@&_p_1CtL>!- z8JqAoMdv~g^&UbJs5CkY?7YtP`kJ$*Oo#msb3)Va0_}49aNCYt*ZDO^>rtwB!dD?5 z!N2}Hzcuz*#2;YUYqanmI&;e)#3Lxt$)go1JmT}3K0q`av~n!S;9r^paemG6afv(=K003ZELmW)}b0CDxMGo&{eO_2gedPY0n-m6@uF>GG>; z(Te!6mY|ExkZJ-{!GcGBmC{zPQZxtdRC*M&MO2>nyljC|njtR`J$Ej)Iet+HXxN$- zX(3eNl30acyK&ed*PH*;_O9#XU_N&8K^X|43r8~UQNBek%3pf@E9`R7W-YPMbT5yJ zQrvL9+BhD6;XIg=pB8gn9Wg#a-%1#=Q4kv2u90#<%el&1Vw1{LAn=%psbFY1RDR`3 zrNs7APMwZq?vS0)~uOjt`%+bo^H?_9nUk?#2|UmPd-vyZ2}FI4=4kFSB6;6YLQie@;m z23>~ZLk~#yIN3+v9jJHRa`yl`K*YaaAWU)TsPfo~2%XqT)JAV`T^Q;aW|Crm-N)+! ze&jJAyo{$H@#o1(jhO)Ni^19=-)o$D^BCWP+>kFryzfyqa9xh?EXAdPnAxCvh=LvL zAi3I|?0h?oKJ3Hj*U(3Ys7#f^TSPTA)t}&^>g{!O$ZV4x8%Di@BW>bRD%nOgkk?tu zy!sNc6g^?55n&Sc)wBMD6ic>?L7y zuZD6jVzuJAIB=>b5)OCzAgpt;@6cP9-+7Dpe}O(wnHh%iYlTMaI8Yxo)wFJ}Lj1*2 z3UogjyDSyEI4&tPs2Bd3>%(-rX5I12Ps;r0<;~WF2Z+9)=d>J5$lWie6}=5nfQR8Q z?18WGyhJ_+VG5{F9u%&JzDF2Wl?`%3YWEiurfAc{2gZkfe)Tf*Qd0Q(It0?BQWYWL z6wI_rO}moj-nOI`IH1UMtF|dhId9||0E9*RmDfar?OUZ)pSOK(GFSZH%u3)qQ^^GS zlR2+EM^ni3J)x8926=Df%a`_1KBxH19>EO7IP>C?jK6)Ob&_oQviJMM*S ztU@Y+0G+q;`5mUtuq{{~-a@O=%GY7LZ!IR(ZZrw6FB+98ma{HlC47LTs4tLY^?cd{ zt}tOa>JA7gW_3ph2tCF12{!iBK~gy#2+Ly{mUj?1`?UOKyDEfP48le70{MKZKs_F7 zQ%*J~v7Ap6vzgi?+Fq{_&|GPpy{~wpvqqK<9r{2^*_!+)vl} zA|p-SFmx^Q^1!L7jOLamI2mYIj*;>2@fQR_Z&)?9wc|lUygz{XZWK=KUR!`HpEVui zf^<%FHJ`#Hvh(c2nwqXq{E_~qVUc|L6UJ-C%!yj@GRG8)CW!;>XSCw<)_e}MR?cad z_xqUMEx=3(KKa)y?(L|<;O|2HXtKcn^N{TtBnVdhr-A6Nu>jb5tCdm>F+Nss5XzXO#y z5Jln_%~odNPRx0J1G=^{R!CYuP%N-+X10p`<%rZZu3_QUO9dH$~LX3j{w`m2vcz1Y>#&&?UgrrQc)2T(pxa@4Z7Em zoRA_L)Na$3hJ0AJ_QilY=sc+ExD9VhOR_iqm(mR%EmG*OSpM1Dn2a`uHJ2x)RFujl z!fVjNQ_fB}E1}v3MqYi1op-Pj>~pY3KBcYkR#&>aEk?FC{>t(R{%IF3&g{P)JtzGK z_B=S%!HOje;S&G%HGm*hY<|ZyO@&zMx!mvheZp;KhSq5#UqFSb7X-il(JB54N0Xj` zMwp&|qT*?mYCZ&I9^;}2&?E|Ix17(;r`tnrjWa>@uk6g#K}B=j_g2b?yq4Z3)V0H+ z)1GTj*+`rx_%6GnRAcgI7zdcw(Q) zK9PcmH!S-7*zNv|Z|*+xmanC#KiuUvXTTjgu&^<;Ya_5X@QEp(m5&Z`|n_{2%gZ7jbqg^V`4f| zPnkr!VO2AT!+WA$ggvm-UI2>lp~B%lozo3#XXn%XVPzL!aQpu?4q~YJWb#i!&Nlaf zCfc7t!^G&ezG*N0YWo8XaBY*hP&!YcgAA>*K*;Y~2voiYGFMxE{t4rP_*&toKs%&( zc9r5*FjU=3Ef)$r@eQ;l8=MCJ1@s(-FdU15Zb-d`tn2(UX=8Dzk>^M0?s~q$=TK+Y zkouLN)WdtLaOQjKV6JTidpTq}V;8CtP!Ka8qLV|0xT^~{JZ6~{;v&~J-;r?^c8fKr zI~eghk8&OA9yn}nmN~ldvbmB}Dd#9kMtpy3vIz?J)eqkZ;QRTZtEsJrn>HdJe1%q} z{4&|)06b=7u#l@v*BbJw4^@UE5vTRln{AlkHFQ5l!&x|f%q_Cid&)@necUlG?41Zq z%@YuJBjfm8I}Nb+(#vkLtqm&(s^qI?S)Xo}4mIA$Ze|ujQts6r{TsHh+CiJelg+x; zhS4>Iie;4%ofXnFeH&}^$|7LoHd#;pmYR>^W0ttD;CN= z-QY3HuDqoaFXg;{BgU7)x+eo`E6hsu7rDjC;VMtAMR?Q)NJrpuS`=aPzN4+0IQZamKttRtfC2FyC$@mPleu1DnyiknR@1QdV=@{)ca4IO`LpH33KX zGG2o`lg|>f0|(|ue;RJLThJ9GoRkbtUV`}6P;w$xo>o2qXk|L<>AUdB%IhPfCI-A5zyw7NzO9A|rTc~>f=;M2yZwh$b&H8Att z8T*x3#F;-dHq+%-$W%B<{$bTuPOiQN)x>o_!@JAK*gSp5Vv>sRfpQ}#8u6x1KL|4} zx}P2dm5AStL!3$K-~2?n?UBRWcy{nmy5fgBd+-~Kltb@`IKe`%KoQhh6T+v_Zj$o^ zQI20}_K3?cMDifPb8UNL;0%u*WB%p=+p?%3;>$_ z;i^%oNMDNWT8+mhR^uYZm;_@dl^_JAI(Ia7(RO+dFzeb4*){>_we9?U&z(#p)pTgG zTx6m;jx>4hC!xC?UsqGiibx(7=75EM1C`KFCMkoG^0K z4l}llknJ`BQ*YIe?g(kY|JT*GCnD8t&9#%rRZr-aC#DfzB7!X87|YS>Y+M?p^@I$r zfkViouYc1MSv0_^kpzN(kv&W%X+9r^N0ZI)^t3$PslMmatv+YarM=j^a|O7;GXQKz zqFs5DSS9DatA<@b>P`1HG?8P31 z+i|H+BRf8iw?57W0sJ)&#<0DuPIn=3LIqEEMVLM|=U{+sHn$E%L08|~H%Q79?&NPV)M=aR zWv3-L^p47nR0QjY=vquppPZSj=~hv@I&&$&YPujit=f*`qbt+Xjhh|mYJ{09bFj3m!>Sorgck`Al%4}#&ZiNf%az0l0I+{gJ#Sx&@{cEMIk;Vy zmmEW`sj=1m+2yaOd!4q0^faIFj58J`5@gKU4ZQww)CCgSP^8J&jbcz@I!rfMDbWMb z$7wI?2RKkZiK3Adw7uTp;np>R6JA?%Fvj5;W=Nh8lE`!>E9&d#7mc;nD8q*j>;Rts zdGJMs00}_$zlCkBUJ)5SB;Rb$PpnHmIpG{z@+q9q4wfthia4E)O}<$x_ukNK;6@Ed zCp6$)+atL>N^O(Ps?+zL2@4C%8HH8vmvDmk>!?0m^;FQ?QXkE(SpL!#?^Z*-zBy#A z`=UdzwDU_q=(U31+U7Y-y_aJMWN}Z+>>ht;<$VvqzyH#q#bsDgWXnwY5kmJ^C9}YY ziZ*60f!CVCol{%(_Bt=ngOpd!PLr8B-y=Ww}s_E99=po|wv+FZR|c$V6%k zd@4){qXv%(%b=K`v!YI9gYLd?dsJKz{wiqUKpjS?z(IR@#}A%1T@B*>uH<=RpWQMb zd~+- z?l$l+V#Rc7ns=_|ii4}j_`g~3<~W;~%7e|*?mgr6z9+k=k{AGQZ#CvVz2ADw<~zTg ztkE+Z$jh$WVd&-wx&Gt<`1MnyN1~8dZZs#$3dI2%%o;{6Bq`33ikUA;A>9Of^19Tq z=^iH{XTs>wCoq2H+#ncSWYHmsj5sSyIMVGELi@aFHUAlQ_L4J9(P$RyX8J>4riy&cohBu$Xa+wGRLmlcmV&97}Q(26C`$2@-%vxWS{oOJcl=}MTo8Avv z4Ag1~hsv&ry81(%iX<%Yd`iZ4dSh}@0%8`s(YPX!$KkHBL#QGBnD_9pTmX$t?c`Rz ziVYutcV9IoBT_Wa1bZ1j!9x8Xv-v?ilMl4A{`X$gx?o*GZ!#x|&ytL?&3*#@QZr7` z(5!z&r+!)+-t~l9xO9!T;L{I2u04Eeqm3z`S zjhUI6j`Os#ioU@X5Wp3`TeXWYtD+9Hay0kdXg&8pqh-_U;ldecaT>rG7X~ypp7b)- zwAcahS3I`>ceExvLdXMZg4m?U3mq~z=Dx&<0+x~d|cJ>%d!ukYun7r_#&oa_5+Z<5u zJc`vW@!XwBk>H1%&(Z={VwWt(eqG$%*H8;?m`7wP5r2U9y8Vdu=?btag2`CmCp~RY zxxZ>ppF)aMQKaY+W>wQ1*9x+b{ts>dJlT{D7}*^^vn@QEYlduM2RHmq42 zmDwTxg?XpW907&4 zL5r%~SCP8=J^@b$HGlr1#5k%;ED;+;L#CTxYCxe2$dhR^x;*704bR{=S|hrg8?gpiU=o8y!WNje-Ap&v}nSCg_0mH_met-({iwAv*p~??k7T z6IIXk=karEXN-o9i5A8-EJTdQC>&>fF$!K-<>k+I^igma*g_IQH1HIr()w2R&_oKn zXdos|-)_!s&7$8I6|Z4*KvazO|iadekg zT2xZAMpX(eSWpo`Ok#k%RmJ-l#KoTRNf>vn+=nc0zt;i9GRd&AZV`z-&82G7d&*H- z!g$j$K5m}cpc2N=)rz)Fr56Oa?ZzTX9z_7oCn&TT;J=@ugp;}jC8;mLX{}^#S-2sl zH9MF|%ukPUMs;#vHwoTt{=fIg%oHy@CX`+OYgQ*P?DH+g>9n=Frfxpf*YT8_UdpN( zg}0ZNJ8@ParHWDZVG0$;=vOmoD=h5d=R$Z$rD$knum6pK{Al2Uy<^!kcadd_T-szP zb0mBJkZ_Vg!kE&h$x9*qe1KNEm$u{E>SRR%cpUG=NrAM_&FI|(CQVwW@N1N>I2CMv zO%*51A6woqlFXHYPljgc8HLy^D6xDR-6*&^mqxxOc8#F2)(Gc7jjwC-v#w2{A6Igi z{Vnn@bFqbu5QhM#0jr04Ac$G0{;H}~2k>e5GRVyEJO4Hgl}(m7Mzip557LTXSl6|) zcacq>0YC3xmxIqm#fXBl#V=UVYY`KhPL40{AR#j^8y@*1-4`k(hgQ!TosG(c5sygH z-iO(J$MUTtVZ>nYIWS-JmEdu83)kfj=U`$`hSh=v(NOoNUe`Hf!@6~`(DRHb6VM?| zR;Tl)(o!$Y1TPp`R^Q^96y8twvY)5yUJSUG$lW4)9Z9QyEHTCyY}c7r8l-AHUk%Ej zV^h&3b0BfTQnJ2fqAKXZC^75+N2`=cKRAwJ*mi8|Brw2Gu{PXktlCp|ltcR?mdA}s zc4+*v`#`i^nTA^p?#0Z%d1%(ti>j4Ei6RBTb!r}^KpM3d*ryJ5d@>}LMRx~+0b>Sa zar&3TzWvrn56Khzoyj0$@@{BzQmoeNPWJxX3#G;~{QZW3vWLz5KFvQqq)M`yjtICk zz$V@5Mm@dBP4A^|YD|h)o%y0{q#Z9+)2rRl&H5V7_``B|1=N!6P7K+UJ7UTow2RUa z++K7M2mDZ2b;vz^t=XMq9-XxNF=@{E(2L2Upm%Jse6MRMrnzDwE%(fkUk1-{kYjo2 z%mZ>SbIbXKNUjO<6oc;UJA7V`DeBwpiUu6)tbly+fzt?ENigH+YqbL{h$52mxc{#A zi?77?IRBN5eZ?O&E#pw|M6?%DzBOh5c^9R2QqcrbhlTb>yB$t!c@@SYR!3XEXBcUR z1Nx@W#zZJqU4QL;N1_9p&ojo>AvY~RT)y_^X#MH8E-|m+d_m{Zk|!kf=MI-~yh8sT zq-R`h-1HwdN6=k8c{WV%bqFe$tI43m9F z;68@*(Qi4?DkeULD+CuN3EdvSU4fuoJ3h;TCY$Jhj+Yn+Oc;G%;X!mq-6Xc_N+V@R9RiUx5IIb!i zo!#~#;)oW9WQ$u!RB2bL-Y!cgA&p0Bm4zOqkAbql@^?5 zB`l?Zo$0~!pH=w47sjL3r{M^vYm6jA{g|_qHz?7O-TrR~(ka%`S!#+7Q;;;(LD8v+ zSx`6_{Psfbl$}QnCBF%fCxWxF(>fv?m$;L_CRuD^`uBBy{GMQm*}@)eYSu-PCw-V> z)*?hJ9kV{I%5zHQxh$i>Ds1qCx~`F5TtwMF0dcR@q4#P=$?Xf}7J}xWaC6cNqs#Qn zAq0oNuYq~t@JWFu$s%1PS&VawOBFu#GTb6iedeFz!G!A{Lr*>Yhm8oj=iI#O2gS5M8Mn-UbK?3BzbQF?|Of@Q76TZr| zq%VQeQXbXlA2Cyw5OnNIQDnF^MR!m5NZwm<)~&pKyon)mIB>~|?g_`9DMelQuUdrS zekjo&za86jh#!5{hzyyFSVrm-gEoYkWV&FkL)Bo;%l*}`S`C3iZM2Zg<<|uH*g0o2 zd@~JL8XN*ZeCB9k5vBki3ZFMSLGP87tK#VeW)l9K8DB4ji&tk8Nw_R21HkpzeOEUy zWR>@b{>3TuD!79(!M)8U>KmM%GX$K5pr2UZnK!XGl|BPB(3R35i~5Di+?~BY_ZQzZ zW47HQ(JS*z1dD~Ii0@UGD)RoFnooc&d@-Gz5@-*m?uF1^$C>C9cw{#~db z+A!lS+4E#`Rk^;|zIb^Qla#AhPDdqZAp z8#Z2b;8cSFmBp3WRI>7gUvlhDE43_rVM~OyUqa9IRE6yl^Z6=1aA&K7g7iGWPQy%I zaEhEIs-3WePkmd^(CN{gk~rIB@Agebx*@7#2W#Q`_>-b|(XqzAsvy+@w)GEsIXT0< zan_Xjl2=d9n0ea5;myeGGzb3G*}ZfG#_9MH3i3;k6ad$&<@G3@?>(V3vPat>#=A1feV zW8tVXi^#17*@^jTA?P##jkPHtjpo9WTIFw&9QYAHZz+!|M)4K$Vm<10(-j$gm&gI5 zWp|!{tY))`ZT9JP{+G;MXZtR@wxl& z*9`_!?AMifd1Y?R+Vbu-f}rB_8c|d|>PjD^N-`dCcIM`BehyXl`cl7;QO&$6#ylbx z(}4J0>oO~TEIuCwB4~kA!g3xSb(;3^OHM4ETp`_`8j z@*@naYIop=&85@Z=n~g*f8)-LPWL7a$0+-t0TZ?;$Kd}|Lvqt-Attc+qQUn)BI8~w9<=SVPp%{`nk_YAdVuvb_*nJII6z9*V>TC zfo3gUkJARjKUF?ZmtUQm+kEm4Ta3~w#f3hb!$(rDl?NFpXx+W*F?duY^nyw!q9^m6 z_(y(toRKyxImbg8Ie6s${uU#uxdHLg2~ZrIp4_}egVCH5>Q(#4cg~|R;Y{%j_NHKH8|U7J^kAvsDGc=6Np+fyaK3_cm}{A-Xwov?nhP%F;%+ggW=`M<&D_k4R)ib-YB?h0pW| zCz7gT-{e!(0UC1cY_wfb44hGp-gt zkEC4R8o$fJ!%$+Zxk#7dR3H~YcUXadFV>hU}CGLrK+$sGc-aaZ|a`86VF4KP|K}#rT&#er|AC;#F zFG6FP;RguxBr8^sM6Dvx<(&Ssv9b*i8}-5s9aQB16E^ERfq*6jN-VwB2pd*irFv3W zRUoGhr1+`q3q2%nrS9XiSpT3GIE8yB=4l#bKYg~=wP~oiyLJohBWF~C>#w7dNL4;< zK_p*ADh@+EwG)J(qB4-~YQ>qcXT$FHD?QLqydinBI~$7%E+8vLvvZ|ZO(jQNX5@E-IX=?-4Yvq3LecBI=!UDOOA%N2 z2R^Fu^bV_u7ROqz**O7(qx^{QN@RA3KXYX!X$o-&s_H{E`)Zn#~XU!0IasJ4YM1{id);{OAvF zOd{i3-+t&0&t4=wV zIJ$fcnvC~rmJ zz7WjSD|XseT-l1K9ogv=A}$qlJhFUp4qKPsW(=utc3u5b{9huixV5#e(%$-W|81+C zzD#d{XVvT1OIK=1ve*v#fvAE0M#4dGoZG_%NLG;vD27`IE5X-$jfGY5(X1aT=?SlV zbdz>pLYf%5+-W=SWpkFFj^Yc+^#SOz4lq|QHG|b^thkf6vIWhFAJ3x9K+H{an+$yW zBc-iHjN|@I%$X4EO`5dEY4B-(H(WCR$!tf?SR~!YDQ)nhD$|H5Xr&FXeMc={FHkFEbFPFYA59P?uNi}T`Yvtheg!K<7ju$&afQ?jhBof# zLTrgG!V^LLP4Lj1=dFbSklv^mUudk|Ja7-SEzMuiC%GoMiPaXe1BYMM1nlx?}Y8*;N5**K9GB_z_ zYo6J0RBc~oLp|!7$c+C4O_w%+o#@Y@No>M#gPhuJ0xY!BX3*L?0c^_d^Q2ZNa*z94 zlTg^~d9?_$;W5@sr@=Hdg&C}t#uu4H=7#pzgs z(QGB9lF;6Q&YYdcx07ypS#HcHp1U!UaOB~jDW0l>FblUt=7l$P8W_L#39XK0EI;*4 z(SVoL;WZPyW*g1mAk1mVzu>z!3TK<`zV9f3+*8iwrY3`Y=X4BXnvS^A-~={EBHRV*C9@y3WgY1+{3LkN`=tkWkSG%v zQfD+tgT&y~pY~gIc&fJ3-4mu}*iclQ!FYdxq=b*0K=ejA;{1>(86BUwM9#8kL2Lnd zSwqM0BjCCj4&xw*$WpPdZ~Eg!79^ZtK7xe7m0fceQ5^Ul%I(MTcz|-+-lF&vg0-0VizV+ zdx8WKFvLtk6avN*p(4#`m3$-`)-TS=zEaV138h)o7N=toT7zuQ?R^;SRb;~FiOxyj zHuWd942jG+g^(DbDm>V$35hV%JGUAu6w~a4tM#*4zlCbXq`vZK3Nkz&RHSV<#f`0o z5yZE!PV?%-mwcBg0eBqxxqij#zWHBx#3INf1x=zA)Eq`C?2i46MJMo18Q%L9;Ts(p zQ~$K|)@DfAbogV*&>FH8P{%dWNvuZJ$+f1bdJ}^Se8)mKS|Vut0MH}{>ESQP&tngj zA4J9*u+buqM2_qk_c(pJv4419`ZJr~SOFQK{oIrZN^M^`vwXxRxKi$1HBV~_lNyN_0I0L~k7?Er9CIP>R z<5Ugq2J%W^)JSl-eCc=gh-`Q+fMl%US~t`j!9LaD&znXtA_hkdG@JaaqqN0WyF=%! zMekXvv;h8y@$Fe7@1!Z4XoIx=E+jx=!a7m*({&|s+olZ2HvseEONN+&&iu(OHc0{h z5}|}~=}H`eb>)?8_*|5zv7{d_N&*WsrzJ&TgENm}i)`mB^X({;^*>1L8jC3k>F8%6 znr(A9v#Oj4Cy8eL(g|Ea9a^>Aw3Z~4LrUTr`}LE-_NZ@3P{#;g*agYL(dx=qt}_Ra z_t)u^-F)y|rpY8RS+%5`HHmZgMSeE735WIEdVb4iFd!coD7VM|OGJV*fULSTJ1_^K zeml|RO zn_F`gTz^*5Jo^>>32#zG?SBOdBq)Il@ucXD{VV^(?=q7ysx`f!cTOP=r1}(dZHY35 z@#O*RXLxo#fuR@i_^rv67j#s3+HA|JQJpkx8&bFFxX0t3T5I*+mY_1pgpbUWV$*yyRy`IYafH^p5GCR>a(y5~T@>F^PBBC9tvv(zJn+SLpf}q)PVpy4S3aB~U&~hL$PY zOz7OCUq~N`aOR>~YD`nMD~qx@olD7U?58o>AiUOQcOe~+5q5^`warRn5O8xSV{&Ml z!YCnp?j%iU*q@=(>vF(5-HqEENG@Sg*kl@1*Wt9Q#Ky#|mG1yXYznIF(l!;4M|VSp zKq+Am+CMbtC9hQS)+POo+4-L}h>J#iuSN%_&JJ_SF?yGQO3%W8YYC-#!&qc9Xqc)r zoxP^>oAPQ@*u^uO{$grgI}^7A!{$&(Xl{L30k#CBzIj#OmeNe|+U{J)TH_G0#k}MF z#l!^pEJ2Bi?)jAu>$ZaKS(Gf@s=}3JSA;J9{3p|{gi3dI)Mx`DH_6yLiaz*ASG^7M zl_ZUps(GcGZb`17mRVE6y!p53zpn?A(tBHN+`&3MG9qyQELl5tJ_?+e>kYcgdKpWf zj^`NfHf$*n-W(x!+Qfo#mgs)kHh86Q(igT1R=1onoXzl%{#E4n%l!fKG;@BtJheaU zqNbvtaa}z;@*}`?n%U)o^PSv1j`MJCvM`9S5)$5A{UAfK?V{w* zFi*p{WUE%AF|n&AU7Xp>@OOWd9hkp2YK3NQ*NbxrZRZ&|k0X}Difrd*voNT>f~3Hi zv`!2E>bSRMNc&5?gUya_ck`mREE|{Kin#Qob|JJzq#wTZePOxM8w1S<)iR5~7`9V= z)N33gc_M0T^d!V&*^=Gf{{wUKtF>`>C~@f>Ru%X%rbv&`Rlw8rkMxC4XeD(g6e=Ra z3^r$=LD{BX&HuPoT z5*wibSehMm)DGe-tEYI?;6a*x2H_3rBTz73KSwZ^eVi7Z;5yyS`L55Ty473*NTg*8 zMAuy2Wb;RAmJ{5;SA#3)D|-hjeAVg2w6P36Hva`O>r8s@sWr+?`VjNRXn*-qv$xw= z&jJfjJ9iA(mLEJJNG|FX7~t-(SF`^)fb!tYB+0qt`=dr$$u+&2s|}|D6K8+?jKy)O621JA%izAxn(xblwZZtgpgmVm)j;9DYxO%j+s91$&cUvW{SvnSJX=9xn7LI;FV8nWpPD zwUOX)9FiZ^yCz$Ep5n}areQnn^^SUe93Q$vfbZqnjTIOdg6bCe@Sj;Ligy8P`~-p5 zBdKWfPiflUMEYA2e`fdFnQ>2fDsSZWk8Qvm6Zq+^-n9sP&a|spno$Bi>?*Xp5ck9y zHl#YL6wyvg>@GL5hI55F(PW5+$zEBwD57uRJiY1@cMv-j5@PGdcgF5Dr zubKldG+>l-dh$4mUwJ=_6azqY7@vJ){l#%y*mC6x7s?B&)5M)rUL~);V zI52wstkgq4mJAo~)7@A}ffQb-+j5?Q<-Mm+ho*0f+Gz(#0xnlFUr0 zNwcS_Er^G8r)O+2w{D-Y(8cc>M-a=h2Dsr2bu+YnuCZ55am5)w{R8cLi`>9Y8N4x1 zGy~nWAY+WDQf#J^{kG8KEXa#N6JN6dHR1R|KBOhxRx#yVxUNlp@~zXg-tD`2gB`eM zSVJyuR0|tm-KFc4>mZPw%MgNL5!pA-@sV6HYgtSU1PcxisVK64iT_1{E%3S`Vfp~( zd8{f|^)`+dh=L?m!#)u)9a6dY@$VHLhDe?j7u4Vq)S)W4G4WFaV)$_gl+i!hs15d+ zosB2l7N<&(9L1>jF-18XwVn0ouwb>Vy)^og$&aGF_|%*f`Xunv(5WN$F!fwkn~dU< zp)d;&=DSG7QVH-*z+Mrh4u!vWwYIvC<>?HC&=10iGBIG2ZlNU6O` z$A_kwvV^8c%)8MfH#>$B5pS>f>5MG2(L-spuZ#)1qeS4ZjMQKdBkdb*{DGj~uKZA;kz+bxg9uHbx!X5%^a*nHp#& zWkdM0t%01`#%mwcfPGuTiVuVue>n6Jm>P3{Xce{798i2`i00o1O3I}mq0V3M*dy_t z0|(Y|4!a*lMLNwD;&3aGKr&Mu@X|7yu?iRn|CYC^#8*6qde|~2n#8xSob4)@4P!B^ zUbCO0AU<5*jE*zg=*?z=Vex=A{cGtrlB`$uNu&nRyV(9t2*XX3>>OJ3D{1HHWV=3&To2F(B@~KSC`63j39jWmo=h+ zdOiN)Y6*SIt6|?*}-lH_u6L1Z@i6sx=PKQcom(J! z*);^zR%Zod@VIUcA^5h7is;qq*`Mq!@eT)Shs-+#m4r>>OhBhNQ>PMM+o8DONh{K= zrN0zWARp{DkAk*&>$qmJ?O->f$c+!Nc~=USo7HrP3xTxVoDa2b84*Hdy>Wj z51ya*jlxB#;n(~gW@sq!NTdJI==6NxIE6Wc!qjUShMFjmpcp1|v2kW#Zh7xA#*mQZ zMown?_+}o|Vt}JgCvycLN_1?L6DO(UXM4~f4%}<9BN92M1q$o&lP$hM(KbI*EcU)KhVZUa`Is7W%nd(SQ1=SAW@9o>Pa;@0a++<5h%=Pcg? zi2_WluQ{BEV&kr>1Y{I3e1zN>cW{f3zt;8Jk4|;f*zuJ^@=< z9Ej}GDpN_5Ojl9fMtieG)UhqtpCjLgTSUh0BpnrXuNcQ!Vm$Pu#G$VD0={t9<-kDl zGaGXBkVR_h1mFJq@gyPV??sp1JmvV=M(|}a1mWlA$I#@CqWr@fJ0*vjGk%0a=0i6~ zQ(2qEFVRm63Tn&DEzl>!Hfa08yl8rY3D8&scluLvE2N!GN*xOLX)K(7!;@N6cX$~vu9vC!J1$n-$kXDOiTZ32hubF z1tg)D$_W%B4Ek5(DwG4mt-@l1WaL)xoqNVsV{>C5yIQ4E zyivuNz>}*>=3lY-Nh>%Rii>&13)hwDTXG11)k^CH+_wQ0a!KaX$q+Dj0x$m=C+KpO46E6iV zl3R)}rTILoyPmGsq;2(kZkRR96E*)_Yhyay_n0xd_zb&-Z}CAQ1=P5|rxV`Tbp>{T zp?aSdPWLt(?FseIJ~|eWq(D3$*6M1i{OI5Om5dc{q|Fm<8=0fft}iqdBuII1S!-tA zTe|bzwraAx=29I!-3W%~p^J8gWe4gi+b~J`8rF(LhtVe4% z?2P66ccKg`!~*9N)WEb5z*FJmRd>#bA;ptpibjwf@vLz`Kg;q5j2pa&z zmH&KHw1|McOzvZf44#_zGJo64M2MGOr=jgdTaHp9> z4_<%sHjZHqlX7Xkdw?UjcYG^Q|0l}@B)H{bkT<>Ki;KFFO3p1EdP~+fE|s#ya|#5? zt~k^#C7ue)JDtO8l|$WQEi1DC$}t)kdO=*cGjZ0)Li8?>M~={d30x)V8q1iUb4n|L zW7tcfMM@swFLJ_**Wl zYKX1Shrt6ywsE(LhZbC`ML@@I{C%+P$^dx3U}G$!a>=>1TPY;EZJK|2Z zpLYq3ozcUuy_4AArq)-KP^Z;ZVlI{W{`idlikjN{e5O6k4Hhga_9IO1>X47)#pX0x zU8boSRfaV^iqns-io^#eg$7XCuIIOElaL%E?+GkSGI{8>{_HV0$qxH#ABUh_gH70I z`|z>m!5xM^@ zJ)36@n8(ru6XDV_Vg)mAdcgCN2`Xx z5sA3^f~Zop4%@WkMN`QC!>56!0=}bZ1nlsPoQP57Dd?qX69b(=ss|SQ##r zZYgEFrH{POvbFYmCk#+$tzL>Hqmf!&AVxu?CW?dDMJ_V#Z{{;PNIrL5pQ<1ykB7Wbl8*|02rm{@MWV9?*yH;USRGeO^v$E6**dR zG!eD9;lM=PKxd2SB(Td2PaW=4dI*VfR=D1w$#EVsJ^O&5 zu3e*n`}izo^0w$|X^@oYuTgCoI)G7btiFMOkRgVDOVZ;!MsMGp{d{uhPAGE9WoV4= z^h39Oi^ww09nt5Tkdrs)>+&xm`Iys|m*nL{%5W^PF%iotnN8n$j)*0g@+No+blCAM z5jsta`U)quOgI3QfV4VW5@gE;QP5&m!cjCuSe2xGS3TD=M4!JF3|Bd0^6ahCs^8ZF z;db27*u(KQsK!)SN_Sp+)RI1Ow$$37me=qPnSQ`_fx&)9r zxh*~1U0v9Dij3qmf3Rz($_MGyuAZYhG}~AtSFf!H;HD1qGcwX?@eSaAW6O7zCo(>4 z&!8O+e2E|3yVfz-^--pF3?mvDuVLLvUaHM@p?SHqN8{V>IIqGs3W(3@9H+hg5XP2cgsL*BcRBD1*@lHbcDIgqZYe5MBs4!Y zdV@lPhH=O-#|dhX5XXJn7PwLSk9So4-UhGizcXtKw~~cl5Ip`AVcKD&{Wtm7Y9>rk z<`X>VI9Y`$sAJ=DOXZ!OUF$Bn`ph5>JGs6k8s;TmyZ=2s3CtQpUS)q_&SW_~%(e_* zwCUxcgR*KZ=f|PNhu2jCzpdU!{Vq*tkVUK+0_YL&5%X~-b_{`2@3u;f?-%<2JLHok z{YS4d2ILDr|FvAB9p;V(acEQz{fe+|8Dw{l3MugIvl;+LK)Anyhr>t9W*O+pPy0?7 z&feFm4iQO5vs3)s`7*XlJo1=kXMi8M)*33r3K)c99jF$uZ!!Vt`1S4UMBw9>pG+t6 z$)C^>AcX-!BW{k-7hk4UJZLL5DjudTl&W4=yphr;EmHruA;){3Z%y#0zi4t_NEf85 zKgVcTt?uA^+rovPHIZ@uLU9Gb{F}C_9gW?$sRcs*X&+zz)YQR%9w46AJ z1b|=mm+jj*9hnHlH?ScF#Ej>bHRe(rsI&$QvdD!~1AXYQ+uu#tt*=XyX8WDQatHTW zaN%)qYqcFZggOU0knxH;14&5(@E|GE<1L7NK#9+?3O{3!QIC{wc(Jf@$WtbGFx)6D z1v-%1Ntl9&yV4nI3|$Gh%YyfY41buINTGO)Jg=$4;L`&v(GP8_Kll!9bY~^0X5su^xQ{xrB^6N&B*w;*R0R>*;yW5;ETsUhrb$GI9TBC?(Y`bP z^aeZzw01oWny$LLX@_FycV{M z?mS$z0B1-+c%qy1fiekk5C^4XoNbk@fqsf(i}GQW!`*;c-&_?`dL(bsmP*^Q0MUm> zPEL27(l>?&E^eM0YTyc9Ej~6JRHM)7K1h>vjk$Dxvpj@$M`^ayk!8xdj0#2sn9fqP zRgZQP{-|d4FSVPK05;J1ClXyz!9)^fm~HJ*?AmEkNanqh1`UJx9#dAtV_a ztW3zAn3)S)xOHxvEl8-a;v5+|&|#TlFnNP*Wym|)k8f@TQwEvJr!&!HK^%sJK)z~p zb}~SV1LOGX8r8J7T_p-45i+y|Zk|B`EESi@>MsmqwnzVV-r651cZN|=3oDS%y10Wf zJ22ZfwN6Woyb(}!E%b@Zo6!l~yL3IYP@~(5$!m0FCm9^IjIT~M>~gITmSx8iAPm&_ zAyEV{o0og)x0x+%%{%+%bj^6p{`=2{o?7G)_9Sp)^c<#ml_F3*a>#4Yh_Z}DRA@Kx zhJ`OQp(fk8p5X3F&;LI2I{OT>&BWl>$h4ho!MF)+-`U6qLK?b2ZN^d@_m$Ld+GJ8E zpB4`LVuO9lGA_t9^J6tYPo<~Yt2itIt%&fQ z3e!KkX0XrYPBrlF+G&wOY~v&~Quv&J9lliovrlgm5~sT#0k0-Vp`mL~wD9n5KBumC zF^iJfP9&;c@@l{0OSU&c5>E(?2mS2hnZAx_*j6oR5$0Cnb?6s2#2eRyI;uu3tPIS z?hQPenNrX~&K9BOLM^cA{&knGwp>)908+4W>a?RVeQ!{_GMl}Ncrc0OmnyO3H($KS zakg_x1`Gge(5GK^BEPj0a8qg>@LPacnA?Cto#Lfu;~Z;uZ;?|^CidMkANEgi1JQ+` zjw1#_Yr-HbIX(yBYVEa)gJRF{`jU^Hjl{k zo=K`oA9An{YbyKW)u*#)9fguQeF=!C6G?CBihm9UypuW_bH46^8r&M9W%Rp3{SE`~ zq;CtsyO*$s$E12B+II~0GRBRIK2jsXYQp<(&v`{O5&!n2_D}+tJY^+VxuP1pehJ;O zIy0a*laq)CINP8%abEQt6BaCae@I9sVe-yDHWSd=Dej@qD z#YBZhhm8D)7c*;V4QG0UCbqm}9zrwmwcy?+$T+61Y%~zF3ux!o)6Ue@mq2ge3!@Ts zQWTa6nx%>5{b$(ofWgpGPbG6MtaLmxqu67Ryyq0tmBps*?U{H+OGf7|)72gdJVNgr z6L!pL>UMr17pcwXAI{7t7C(uKYXR62Yfs0Ygx=P?VNMkD;ul!@Ozt5FWI;6_k{QMe zwCaNupNDcC_R5fSvgc~|^Vy1-_g4|5P#G8=emdId=Hb_=U%4iP>x4I!5GpQIZtdg_ zdmeSH#Y~;Iy%9qFa%B0F#!ueJ($jK`Wbj1a^~-5{;grAhq711$;*==y_$5;s3)UBT zY=1)vUCYUqh6LxE1zgfp?6=V0Pz$k~ zA$&<{OD530mTXjv6g3I_D)ifV`bVi)*VfEQPK~s{fK|PSEU=INvUAi2=`~Wv^NJXv z#Tu(mz;0KUyrT~TFMWOs=tjxt09G(?%*mxdv=gw%sbGR_T&TNnPcUc$5!0Zgp$!SG z!R{3#aOiMxN9tjO{x!P4I2q}|ngMxYUqSpD^Eh^-4-UG#yIjULNGwhR`wkWYDs{@PJ^-xA}2;$e`dy7gA^4p|I{aVyq< zq53%rZ*XqkQ!T_XXK=RTYk`v7ZkJ zF{P)QgpLS&e-xVShE>xCHp57-t41hbc@EDEGW{TP|Iv%FmO1u5OH0OP_^&1XVG z)}wy%Ut3@mZPAvCYj8zNNf5Jp_$57bj*Io@b^!B-TjaIrT0I3_)<~4m1PUS3xs1#n zX;S;zW7NlMpGd=J3b9!}8y;k3jCg4mIJj)REx1#PB42GI=GJC#Zc_+pVH~Oh37D>C z4cc+ZtS_3xYpAmUdgqcbz#xP>R<&<^0CNQbhYIw+p|U|~D*spp2xAKrHmf8wV?%(4 znc`7bQN2?FIzF64cFkMIU5AfXBb`0=TAPOF8Gp zszIEm5bsy5gKAqu4D_aM>vvKNWiw-lYw4dc%VS?SAI1+0C(*qkgbXD?!b0f@@l(-f zU9BO&2n~{8rGZUyMpvIAnKHUtlDPl3GB82jp;3%cOxZ6ycmA6y{q&JBIpWRBp{>@_ z7Cs3>29|bM^La%WTw*rTv#69Tj(tO1z_}u6$n)9QBjidwqL6Sd!7(iNy;W-0^Yz0@ zk@;$C{0Jp9z3W^X0C7?c*9}D7&_GAH{bB)lLHLez`})S;4*uJyCl4_91>fK5$xgcd zUsVQum-;21gKxzWG>n_m148(B+wCPW<{OLc#hv5!K_tIhif3JF=57zLnd;D2&u&G! zLu}v|SP!n;s!LlBCq7pWp~Vo5E?rVOpb3yWT(oS`NOYavz|F9>CVVsDcA1=%6mmwB>*E zII<+5?wVwk1l;FD*uj2YRicbZhJK)`>So6P13>)0JBEc|^UM0S4aix-y-6kAx}Rhm z1a3U9mHP_g0*sPsv$%+g0IgP2o1LB=&+(;aOU~HXrAIDQ#>}?gdgvbpb3uV;nA2~9 z%vw=^Jh52k0E19SK}RzFYITaPwz&w)58+YIN{u+rxYoI0X<$`KGz|ThYPfi38so>r z?7{s&*vx3#7JcNO-l`Vjquz;M>At9xm87Qegew+%T5^5deWMr1tr^=& zfEkaMf)iUdlaTzOCMD8DE?sbNP0|o;4fI6}*AQz<*?GVgvLX;54?*6KWyNO6&K+PU zeto>ssOJkK|Ka1zzqS7}0Bir0Gq1@qfeFQs*l2hjHtVp&s;U(&=+oYmQGIK!BQCvA zMbw#7>W(qmKY)v4k@~-h*)5?;q0l6uUPQTPHkr7wA8Jj+L}_{Kl{aDNS4tG$kwmPm zb0F4VkpVcUSuJw`k29vAIwhS~?3~rs$$Vbk1`vSnu~_*s!2%jw=9X_qDP=WG)4+FX zGoBuw8?prv?*f3}`HRrzW1fH$#vw2NnqB9BI)zVlwSlI+Yd(c>-g^C)f!{&!hWZzH z5(q9ci`oprx#*kO1(*!Y)zzx^iGxA8TmClre)88`uu})hk~QF=bP?NSos#P66UdBW z0`mLhNkZb;^0)YeR49_{dWHK?{|V!%G1={`Ig>XCf~qDne~xKnT%EgnV6ENU~m zt6jSoBI9Nx%`JAz$S?H4Iv*jBcCIQheo%DKMRUEm>pvmFMuJOxAnX%aH^C*;nIM9Y zXO>>%l=jrTn<{W=ggfXDIA%c^lQ2V_A&}8Iq=6PB9#^AkS`W0bHP#UBD?{T=ZR#Y0o@NMEfEx%Jo zgE9ZxA%>kMX-Ut1=8)ze6;f#zdSBZre*{=j1C1)S(G^R1?&ehFs_f1L@T_k*@Oth(NrdsB6FWXEB8dj>C-TuW=R25 z$Rom{QUd!;V-)VGUGr&p^zcsyU(7m_qm75|bkSbpPf-R@=6Wsx@q(1=v#w^PsK7h{ zQ9BL5M?;cc-Q}=rl${9Pf3vhP$Np3Y17e{4nWQ$M61+7UTDW8oeD8#8L}OPUzcfp2 zp>3rp?N1z{+zFk!e}?3ZZ2+v-X5poxbB7rQVDOKkorVuUi=0`+LkRaXF%8Qo?zqVj znH}~Dj&vw~&HRz=a`NznT63e+oH0)!g*+uC8NTHoh<3c_k>Hrz*rLN+SpKgdaeDb% zbL~<|a)Cv&KHTSlRej|yGx(^Nvf6q_JK%%v{8y}bMmeZG6c1Y$E_A1sBXR`p}^ zAy5@is2n{|p}P>P*x}$&A{@nGX_lN4-e?bzfuu%qAM%kUU+zS=E?044UO&@EJZQc1 zBFSQHO;-4g%S)ypVc!Q`d%(t;I)j(H?KcxI*xrA>^6(|q@Gm9VPUE&Y4%4F7q2EC`O3=ZR)CB|b^a#{f=Ax^0L>+{aH zoWRggkjUX)fugAufU#bC*FV4up)GBFo zB{IHC#Au+T)Oa)ixZ6C+F6G9Le3{b(>}#d(b1k1ohtu+o!9^nX!3cF>r;2)Myf zL_qR&Wz-x?WN^}Tb2IA)X}yLb`n|r$Rz@ph#H`X2$3gCTOS8cEkCbn}b4(LXMLqDYG{t_bxAalb_|>Nk zHj+^~|gp<^iBvuS%3bz(h1#ZJe>;>W~ay!W3nzS^DoVCpJbgNMsSbglIl z@p^9-u=E#$uI6;?&iCWf`~6FY;H(z4{iE}N#u;FQJ+L4ad|d7l3jxJrE8Vncx*o-7 zW-`qKtXs#@d4kQrj z%}o>yu_3|ky4<*m1#}KzTHVZvyCJVL6Y-&|I^gOh-_JyGJ8q$^EWDVjaed-`Wp2rD zdQ6K5nbg}!HK4%N1+2qZ@sHojLWS1thp@A@KhR|GhV~o>BIOY%{~vPJ(u}Tu{sXsx z+q?Ji)9-ZqJYDO}7Iw|s42*zkRw`sUPu64rG*m>l0&n^#1)M44BRM+l(G4W4NtHtko2R5 z&n*B#e=4@DicqYS5CEX=I;vQ#R*bCOX;;`Lm+y|5|bd~AI?Tb5Xq6KisA)X^}Xi<$JGt07*P;mt6j5UJjC#Yg!t&68^MBnkoItIt1T@=p%%bFz#l zKkFd5P<_T#?QsQmjxCzY@d>w;2no?pd}%`s83)e?Lg9ZB#DRVSJ7&e{I<2|*sinKW ztWgNDTJ=_yR!G)Ir}ulQD_koV%p^1VynXlJ6`d!Bc9*wBqtgR`PZ*)oHl8IGNIFbf;876w5F*x0VFc%Q2md5#_XD88I}wlmsp6Js?o zE^MfAqiJvp0k`HvXQB_;L_^}U)@KMAXh@6%S)I-1pEwnBl_;>v_~GC2X#+oR5UH z5{mfX1!H!}rLxlYcw8c4!J8CrZ;2XU-drOB<{!Ru^O(MS(I*&3s9T!!$`+xj7=;QN ze#3wTCzdgTY!r&g@i{mn9;M(kuC({EXNhvo*hPn|M`zRx5Sn*%D+vwdPB%kJC&`Oo zgA>m^V2FdB$it?c&8!wtHJ=$^Chd;VwpqJ4%yGa=FCX)DFn+~dCVxeAm%E7D#>lCr z1_X}DPJhrTWQ(fo1$*TMjeF{Hc(Fk7Ad^N($_WpQ-wEkb@*tqoND@o2SFb5tOdPF8 zzkQ=`emKdXNd~p&W*_wBYe)?&w-S^!vbaKGYJil5j&#R0%_pY1hw5)r5`d;E_rd?Wzc3OWY8IlH3vyIYyi!_O`hvcOJy2#X?9#s!L9uDC$S4>Di&1 z1iaqJgMZ}tM2?FeVg8|a`X}@l^=d&b{lmIW5peqd#d~RqUFpPE{uB19v8+?j{Y6?> z>JK5`F*Hz^Bo4uhF)(2rU(xS48X=w@3petHzp9A79ZgYKviZ8fkcg!?9+I}Xy&j-8 zDGajJZJ6G6)z6OkaB;tewbw5##1aqB%zM2x`yA#MhX(xSH6m9Yi`5f83y*SlG;(u_ zs!b^j6^c^&J3}2Qz?PxK&S$*H#(at{T;wt9-ccNG`$XcCx>n&RE*rP_xw$HemWQHc z&L7LWMrqE)<1jz2=eae=_$#<_|0eFc79!0;{TCo!W}wos_;8sM9x_QFc`4dl66L15Hj=38!iU0 znpu($d0%Qz9<5`e@lcvkVuXv}2JZ1?a)(ioU|yK6lQhbJYGg>ds%V$8f!3!|jq8)-tUoxHfs-wZ!KBXj{&?BC4J=7A9i zm0pAmz13BdX&`xu%&2v6H813#Rs97Ad0GDJ=vXRo1*}f9U_i3hSK%6Z<2D>?r0NpeVzk31)-NETG}RyS zgxK?)PJuBV9W;%rIU0tf97?JgTkPJQfM%02O@1%;S%*+IE6oXd!y?} z4&Zq@n!-!z$%Vv&8KzkIgq3Ke@IBNtqHvOYKzPGXWMpkoJcqyvEKd5Gu_o3x$fdf!dgrZRKY>9cKTWX;WBZk|_^dWE}Fk>Vn-oC7xwLA}V0Hf{}2aE$j!b zESt?piwts1zRA`IZv(vLY(T#dua~LlHWL33J|QhagOnt}yzA2@d)Asp$wZ8~N4 zVMtp4S}Q|6|7kk9a=FBT4eyy4`std7?)}PYEuiAZx|u^}9Bej#^3GY9st*INH2l=oy|EnNJ8h!G(R`pfB>YdIn*udZvZ>Qt`KyyE zeFnUF<7kb_)ae-fJ}N=Gfb3W`2xzHAmO!}!*-r(go3`+oq`sOoxI9;jARPR(Fy*tV zJCI3I%er{!hky>>%uX33HA2iweI*xhV!8EPib6Ib0EJDC-(Ue%k9w|LS6 zo<_Hn2bW?F*9C?bfK_V^vv72x^D9dAtoOx+Vt*f=oi>NnRtD5BPd%?T!d8FvBZ`bERLZr=V9 zP8$OcxN8`q2g1-*-df$Zxp(cOl{`xKUUK;9)OC@LgM z=llN;{^e0&kbmYmKS>7RRbPLi&bRk`!Qs@Q+$}cZFA{As4D(&neKV~01>f72!#Q?4 z3>@3H1FZED38~bcTyRV3ojLa7s#9)WN34q{{Cp0&`d>GYlp*5z77FdBrAC*(uGZBW z^gplpMI@5xvOHqM1FBZfvs}Byg42FhG#MB%WUPtgixKiy>`M@#xm}4NCK@57-A8%HdAx4YcM`qEWdATDn8AI!3KHx zOZpZ?rvYi5Crgh@TSGTjE3{Wm47P^jwkfjxbtkOenf;Szv?(TW&+13)?%lG(%f$0# z=~-o5LWBF!joNSU3g!;Lf8SQwGbNIY>7vNJPc`!%52B8nw?$FADv0vGy+tYgK(}2O z*ThPJYMNk#z;B&0#9ibY?NWQX$#g8uHWAjMkO#e2`{nnhraZ1WLlBDV zFDlI+oHBIk$EnOKm-cpmFiesQ8>LKJDoTN}fGIogVO#ZhFHb>=!Af*E_=XBSkgnDi*xjlzFo66x!qR<~-aWlNVrmTcox2jW0MZX^{FrE#MViKN!5;FCfzc4xE+7nQ ziw2&4;zVm29Q)iT9^6-~)?~{%h;aYL!i{n@hczPH#V-=pdHHwZ83WMuBgc&G-6c-w zehP@qW3aENkzlWMHO8ktKSF=r0)*WB3{s&QvoCqjV$KIxibGBtj((aD6L`{qR8)y6mJ zx1ZZI&Unh`bkGYSKEXDNPpDp_G^Azm9rhALP50m!G{ZKCQLp&%T4fmy6HV{VLRT3& zR13;rRU@%%h~;57!!Yfb{vS(XW__8E5WnMn-KViaq6@fNAX81gHrEe_znLJNJb(#hJP|0&z zXA%RKbhc?`@iE(vgWY|OLEW|o#nu@wN0W_+IZf*)48A6=euTKI`_n$P7f-GMD;okQ z{DOu^&xbBXh=(m+A?%jmzMJAg(wgco>nQvoxL;eCUD7HZ+?V0fiDJ4ikho^GRmQPF zPqjh)<6&*7xh||YxAa4lHp*v(e?X)I_$^|XhTQbxk2YdhE6!WFR>uI$yMU%`VFo$C zUQ$9Ka^@};dD4=ohGQ!eL%(A(FY-U#OG2U0oIG>0)yrH#M1ra=S?Y9DjUXz^ktMlu@4$blPIcct>RZ8| z4AYJ@q|KLNTGT>(&NZtz4G^pTg71^mpG6xHf^BOBM#IJLbK%+%+WV(PRd@n)+~x<> zKpbGQy;%BYaO?gZ5-Nhdg;x`q0VQc<>RYya;b}zoy`>%J+^-T^ROVU_0A0_qDKK_&%veJ zRv6KPUOg^a>Jh~x+HGsi1QgO#=y6659D!3-X{ooSky{_n-aulO+zHV2{2bCE8`aw( zkU{nZDuX03)MnKVjBg=6WL>FwQIkXj7wlgc%pn??R5>#SvnHp30Nd;V*t?skPsXrG z0lBu3Ls|yuucPxb)`lm^?Guh_u1dlPQ_;^2jEgX8dgIyI*3)MvOfXnCVz$1F^Tvxx zho1em%o;yU*+}vsx?6LplCba5Y~|1@CxkMv1)R1wbS1(_X#z*^=j`mF%Symjia|c7 z<8Trf3?(<|1D#H@Y^fn-NfU>B_mH1|`h$^Hl&uGsj7+NM@}tSYCc4Se&IffFK$iXq zksSW5>8WVwn!7npF+D(pIgdB~@*r1-wJ4@Uy5Ir%GmJx0HvT0M>I?GGo#w=zuDyT1 zzMc(q9-QafLfk#QZFXnV?d&QDVmXk7>3GBv_GNG{0|By=q!F_DPx9E*>oUv4s;ZzP zKLtST6}PQA>SKPQucUtQtZddF!5Ul=R^0fMH>5;kCN^9=NIH_r+@KJhKfh~hE%UwE zfy%`ovy}?9NB<}eVsSI@&H+0_;>PL(%=Q!qxy z#!M~jW!F6!c(RE+&CDj>xQ#zZTAAb9m^v}jeOIa2a&^-StCVsGWfq+!d`Obm{DGStZq5pw>4^Av}AP2b=KKNXeTJ*UQaE0_U7rFuUaWKkjFJs-j&5I6W%mGHge?yA{m(^x`>e!#@1f{rHa+*|yM-ttrnBh_d zgnXJs4xW&(OVITE&8LlZQ?hNSG~t)Y>VvyO=6+iYQ`ZYXmQsrxw1=;kpZVkN`yhz? zH2oS0Nsu(zdg@Eqy&Boch9ROqD;4 zWb@Ol;i!Yd+(@HjPihP-^xNAo0B;w)DqbOMeWL;8O}u29OCNj?N?Fa8B-09ir~BWC zc4Dk}`;L8+XUd`_aOd=>(b2N#{d|&Cb4S{<^)*)@*`o%Ahl~?nL(+>u0 zDrO7x`{>Qj=HU!9##a0Sj=Hq1bCq|Mn2&p3$tA2-*c5Ox^vX?AtlilgMD>{j zUxhsjWP+jn)v1KfIiQuQvQ@C>1Yj874GKTwDMM>NLbY@*2oEc&3c22XAyw+jgEc?6 zSa4k!yiM+f#FA-W9y{Y#09zR&)b*)`x^ERfYCenk^u6 zquF)wMF;uk(-7Ez&|fhxJm?^md)?_U5w$n)3gY>Az)TRF9>*@tIFkNy#Arik5AWwk zV(qo@(;@zCIjJ)(v%;d9O7xudMnQfL|7cOoY1%Y0OAp#vKfaz#r#)Yzew*pj10;4i3*kqS|fI>BdbCyRck%H-}Yf36Idkj zJ{Dzu$=aJv*Piq+!eEnkh)L>n9^)PCrlLi*KjqFHSQI}0Cgl_ilaQSn2I{Nu-rz8swSDXW0;Zn99V-*g3;Cn;w%{i|@sQrtxNqUD zAMjr8x;W3q{wZxxs}(FXoZtHw~(?}z@+4LN=Pb;nsxovpUsQL$4cut|4^v80(J zf#C*V${+WT&UbDru%=@p!%fCE+`dmJ0%5M{Po2lWY!oqhT<|d)n@V*AxcTZE4e1y{U?tB1u_S)+p?iMMsay+~!ZK2EheG zffhgXBxskz$TllKQZ`?t;=7X#l9OaynJZ)Qkhj{e_*7Xb9L>J#c}Oo=dD}g^$C&Ae zUwdjHXFeGLyo(7R+5Cgsv85>4_W<^h=@*3KRWzxADLxUAD&9M>21wS)%S8WG2duUc zuoK2!1Q@kUZwiz3!Z9r^Qr=}QIIrQ#T5p!TCucBZSMZZ@99?h{UXF$6J4Y|`@WnN-L z>->}2C&kDiNR~`7>hu*nsO_EPjRY|BdLYkadJukM$?L!IMPT2#R3-+ChcgRYsXW|I zUXEP@!pb*F<%|pRfT_)gkmRiygCiXFHu(LF-g+o+b_F+YG)NhONTT%Q7E`#VN&T(u zT&MW6xfQ6gjTIXk_XeS<*68tp((3ZCN$&7I#65>sQAWbNh44=;v zazKI=+#7VQ;?Fz;RNtoK4zm(O*qcZH7VwJ6-{4Ni_WRQOQlA!zr|B@snMQ}M*rgZY zNm%GJ9iMe1X0-f)bsfO%TTf(A-d+{+$^c#LzF+5aysTy|XWIb_C&t*hhIw1{P=oJD z(&LYpbJW%`VW9v^bwWi>%<0eHR|Ve%PVj5*QQe%fl=AJzoBs?`_<7W=!ENO;ip;V5hqVz^jQe+Sbsb5Luj;*Ijx8oJ83cq0{rx1kHm9igP6L zpS#L~$T(Z#D3yt&N2S|n>EG*4cLlY8EbBX!N^v`O?)K+G7C#~`P$K8mVL`U!Zx&PU z+pxtqoyQ9^_;BqN0f{H4b>>i*6T-~S|LpqLC-(Q*PhhAPFiiH?Ol0hm#8G>lCy>{o zSXNTUAH^T-3>Pp%kL`+OKI>W91um3f3nDk%4WnQ^x`%%*bkQ6!d~O@C_U`|4WJ1L% zX-{ASgVT|FBH{me@V)t}6=Od38T&-@@nz+n`I(U+)*ogaiX7yO)W$oAj0D+qJdY7& z5F3lH5AKOLalW&mfsa2cm!M=P&PCxoPxtl2D9rKD;#-*W#|;eb-oquH)4;9teSn~6 z#2}bqKv-JOVUxTLL;%(*LZ&HqQl@LSu{Q~9%G@R(O@*s28Vau285Le}u_mCzx|xx! z@eX_H2e{v}HHqulK5t3&?W+iFE&t^*i7_X=B~FJF^GWn`KXq!d?|J-0jrf!t9ZlT8 zJH`9(NzkB}HVf)Uw;m~tBL2e|Qr0QKL$w6BH5SFJ(YRU2`e^3Xm3?UtY)<`l2#-UK z!-MnEg9pR#Mawh^vFyS;Anrz;EIA~PYJ1+yWgA0l@o#G9Cm7A*LDz&dPLI#Xxk@Az zgZ*woWTW1c{6vll9I&9)+P}@nH19kS1VA{Pk@^ow6{Pzz{kP9udG?UeSg(s%g997z z{=mT>%3~dEp2-@d@RmOpQ#o$A5G`nE{QOZS@UyA<{Ha5^YVm!(x7b=xmR%NivxZcqmEP-SWEG1*8plL+Or zuv_o2^iP_UxV`g;JSKySAc>hB+c10-s{`NEMEw}iJ_g9F)$ox}!j1_eh7H`f|rqxn?>ZLl8oF9i6T+0bZ!gS2DbI86^eZK#u3QWMbMay7Ng_FXPk2;i zc<+wJAk8<^mARUC-{s-)XsBt=wXl)26p8-hDB!zndh5Wd#pM3s=Ri?@X{GH)e;<-m zv^N1oPlbWam4shn+k@q@&7ljvbmVPb-F*Gzt~nPF=zT*`LS_SEte|pjXhf6Mww(Lp zOSyIHq31-4ZVbVtk)xgCgb4(Xvyc?bH-g?OEJGQU0@$LE5rcpJ(y1lWF<@}S6zaxwl z7ga_|3Ujj+e{#w~*wU@kIR#_cK9NHnrxd;bXTF@UTlW0E*i@a?XwN$sH)m5{r1#%& zGrPA^aa&Rnzp8tP&&}h}*4=xo=a8-afp|}mC9C!<4OWk1&*e=8%s;>jxlm)U7^e{^ zlm8LDn=rEgFLX%+v~P+Kc4pF1sNlI9fWeT7zrLB!4TdS>tE#HSSSOs-P%7Q&4+V2D zjr1Rtgm{}$hAH?XYXQJKX{l13ut1G}GRxT| zHEM|8XOvnaxWcYq1KqIXcVTdSQ3|qN5rg(EY%?_ecW3;@&BxtvXoKE|3y1|_vA*5J z*~)}9G~!EjJseLKvy{l@!8+|R2TYJ_Pwag%CNj;nm)^!noz3p-wXR)*K>pX>+3iqc z#sr9GdR)8iitOo*TemONQ5c~-j@^5q*A77RE*eqSK=fu>hpdo^#7E+M0)Hw|HVW-A=kRzbT@L%JLa zsy%JJ$sB(jKGH2B2$UmEwTV!%D345!xsTF z&-MYFf%?Xp)Kt9>JiVscZO@`ffM_suk%8~>uO66x)_|@UC^b!G!;+{f8$?jhfhdy+ zJ-=tLEtkwaaE6}8=M~~J;WwLa=3IK30*p6~%PXoGMx=h2%tSI_bWnh5`iRSJT9) zEV~J6Q5Lc|R9bd&c02}Z3ctBW1!T^nH#4TOn8=yHLSbp{1r30!wbrbq5m`T$@naAm zMpt3(x(XqdrAemQHjERe8)HMzu68S^q#S*L&*^J`^A47VRkdr}35k5Rdh6#;5uoR+ zQgEBR%NTZ;&Ecg?U|RhKPcz}YZORm{kYqB7lp;g^8f+{}#HK6_r~ZBT4w+|it9N5* zF5F}fF`y=aOlU5&Np`WI&cj5c4uX*;%lF-=Q_G#YyX$o3^D~`h*r<3Y4WqUh(A71$ z1V$cjK#urayAi=xTNW*mp?Ll486o@wn5krUz@8i+u1V+u)Sc=`o6w)+ijIbh24c!AGkZwBXU%@3 z!yN5L4~MZHbQFtO=U8s$nG*ldW6DGYZ(pDgwuHy;l^Fa-?TKw#Oh7%v~tSzA|Xgp=4hiu z5V9C61wV?O?V9vqxv?yNn{bu@d%-U5e{FhDB2!$NHi;$H5zG;2iWpZSMrD5d!_C_-$e%3Vw_6~4-BL~+I~3%iM|hN#wn!1F z?~~W$1$CA#{YKP&w7=QO1i6@S8b~mC8-XYuW?|$Wux4qYv2S!#DF_o%yineEB>bQW z$?s9UeH^q4*ab-`Q|#wksF#nGgBV*zZT&!K`!MtN7RcrSG!(L~`uMc9%n#FpuS@K0 zCOKD#BHw4kdwe#uQF_us*yCbllBN{u4-{t0-`Pxa@~%xe>zKa45z^)AK4pPUff4QI zu^i4T<-CRv8QKI|r^ZaT6om-}jD``=>@Oqp z>*CN3kKHaR#hwrN|Jz@+bu+Y@tq}|^q99Qzf99%j*_@RRZ+xU z9aNs3{6X%9u?=Cr)m}88ouA6A_FblXK8BPwsCszXom0BCj!$9 z&FVMC?RG(GecY_2HVHj+;U;&pe&TZdW;dXL6iz1k_j#)HMczBTpCUedc1Hmo?c%HF zf_eENcK9>)MRMHk89Ki=VO|xR=1%EOLCzDcpc-Wk93<7_lYS!30xE7AtIB?|N|zU} z=Q&SAz1~JtBolVPRv{|lS(sEO^v(}7* zcny2wrYlMHc9y^vB|2Zgnuz#I9xJ-=P2eoF%^NggQ2kt8lOU|Df-^HPiVm=tQN%cu zLi#hE1j#!H9sdADK)SzYZxsQ(5m*tV&7rqs$N5 zexQe3+!4qCWc|;)IKgt-KFPp!3r$zCJ^fn;hGRnO7MB(*dx2bF zIR*jpcTp#R~7OGF_DRC70IJQ=5sp5Cz6| zDg#6n>+7SD1BAmm*FKE6w?wnJY$bZGOVSd{l;au|>wKu@!CTN~YrVe*4++c8aO{b7 zlYQ+(1}FZ^9bY{fp6=3^TXcj9)UZJ9RwW^tLfW~E6ou{VcJ?-j(cTQyB3jO*=y2t^ z#5;KduAW;&(Z7Dw3mOb8Kk_kEgyAu}z^Q9LObTF4+?_=46)btJ`n}9bNPjk>Rn8S3 zTLq^)WeB5lh*Kno`Yz|k;jCSrDG9nV_^uLrkr-IDhG;hP+>SXWtHo1{QbWyq5# zg$qZNH&XQWKDG>LUJxmLmDI3VWn^PRNPUiqcu71bsPw5+&NR4eK<=B`Z}<-;93Fv6 zYR@R*P~mciNE@TJqaRzrsl^cSZyrag){2ok6UKO{|D9ab#ftJ}IbR(d9l9^IGI!wg zY54~#7p1k$?)>$*@(^DN60l=0OkCe=I>-~k`S_8fM@`kc@zdqZ;n{NIr>lItkqz6T z(cj2)-2CMjMmGSc?*(3xrxj^wArW010`?=A$kZc5@$ZCQ4TzQ#&23t|BV~)1JQ4i; ztkFPdw!7$L!LV6`wQ?RX^!M!OIHso5n!=few?<)!tMyPvl@4@yy`9F=`DBRRAptHU zh1o)TSuS@MMN`GYwpIJNoKZ*RQ@agdD1@H@H1QgFf#7yAM_LFB`!!=CJ}6d5nEvMuqmx;|oYltHM zN!W-MWh*m9^8$IBM_@?7cMKFvHC}jpQ>cM;yR=_P;qTW&fEXrF9gIRRp_{9!)RQKP z3@%W_SE`G9y1nnOy_?pW3=|j-tR+IKXCU+Y(F9v5vG*iMu%r=@f)tTny)%;W<1BI; zE{|0mdYnU|Dlmp3lm1kdzSOj*d`WJ4RN%NF=(yl>N!IpCq zF_B9|SlK`N>uSMOZsqCRoUG7Bsw~saNO9}vAGYtu5H3y6m;c%|*q?`!9ta7cZKEhy zlZ@AZm-`j4K;<$6l-2rX^xzw{*U54A+15eLt@;43w;YV!BjWqx9yg@#AIGPv-aoVXfU3&KF3NDA4cUp(h zDJ@Mghsrb8hsHkK4*HwmF=`<;^PM#gHNj2fiQZDrpiFT{At^YehzoL-H{To9ktGUI zD#?39TUghJs(E*RDCGpUefEHQ{$9Vxmlns08OupXi5z7{CJ#NSgs6B1Ae--GsV(L- zm;i?xH9SkS#NwatvabwzCNoED4_Hbj0t+DpNVwvZp+e&Mri(a7wU1wyhy2s(!zYLb z7ByN3O1IJ$L4%g*FQ#*v$x|D}fm`5PSckV#2)?MVTau?4O{o3)q;}K>Qdojnq&W^e zuwJ%mn@Qe3ymC}|oAJ_sI&Pd>Yp9=rLTw0jH{_$}KD$shX2D+vGO_#=>vi@t*m?%Z z%&7e@>E?7KYU)^LjSrRma-H1=j6eb5gv%i^;-&pd5xFdMPU+%bi#2r)^p)&)1svA% zb3IX174T-88@*@ERh`FUjjwoa=wtDfz|*TV**S4-x8%NIvIK-_KId?0E*%leUtT-1 zpNEiC*WYq%P!YlLykUjVmpTVi$0 z^`6K_0{9fQKGjBR#=}Q{JV$MRzPkvxbd#)LrI%anZtT;B21WWE9 zP0JHTYBdtSV~Z$(;BfY?ArcmA%SusJdVFJ0y!BP5g)4yAc01h+C zcmf>VX4tV)s^zA3Z)VS;6JU9_&*e54QlqUJIvs_i`g1WQ-OhMur@&8a?#O*+5h}^g zyxm%9FC3|kK^k#6_fe9)5!;{GF4}@TOFkI`KIioFM4GQLG%yd|TZbFYfK3*xWCN&uii4|z9KItRLod@!Dfy-nXytbeOPe3iNVHrq=mcS4uJb;| z*px-q>MI9}{e(?Jg$m`~srNsadsBtIRqq&I%ZVgUdBRFO!f#;YqZb@goVjRd@7e5SUc!N#duNR5o1Wk+vsD%L*s4|6` zvXbHCSlc^_4abICrsz9BpqNf?4qtF1?P-LT5(p;laR|LI+%yGSoP0>>TsnsgsY8H! zNPT&64&67pi9it;1hkO>_(i3AbeLEWJ{*$e8g4q_#F(C+Ea>JoM$|(PI>UUp9dQNh zimGE0LvO02L?;ar|3jO1cdjpqz0-o9+>FT#{_wo9#qs8UU&?Ztl0<^0W6YZ2+X)Q1 z>c_%!Tx+d`H7_s>Iwvk){*L9g-5nkv!-^;i9ffJBrMJtU$^D+o%PVlQ;*U>w|tZm@WlItwLTa3$1;Wt+80bel&@QP zjO)@-eD<|J;!a7Kh-h_n81IV~i375|w59`nSL#^O?8JgFbX6;OV>-$iHC4^P0@OZEJJn3VH=rp#-Z!ceSHKpL) zEh*bVfsa2@R1Vk$Y6CX39^?POlOF%g00s(my-1p8pM-;mH1_X|+F;%*b^u zG}P3P0IdL^?8-|E34-eo)Ej?jIm&mPm1Xx0yXTonmYf)Z*EvyCL9*y#f7l3N*S>VO z@8!_&BIu+Mpg#Bnhg#8kWV{c8#!xj|F}v(*=eIrVJ%-1>7w|e={{bn2YnM4h1t2>O zy$d(We+CadUVmiXJ@)C8Oe(YQ+aii5_?G_gzpp)c12wUru7D}c245+FBeKf=!;V)z zaAi|)J~$ktq2Jc!qk(?Gn>v z^TFMP!6uwB=|;o~Z&cW~g>6@ze8MU0%hpVY$VQUep00lt$zsfno zSfBrUICnALoWn}_cRq*0S7l#8FsL>LKu~9F8SizWvd5o~$gF^Eb!zd9%mY{zB zoug{CvkMM8Ns`?km2>`2% zLik=5@DE0MKy8slHy%61-ys=`p zMyw6%txf~;VVlwjSVwm(AG8upp3E}4)|V|HPjSNlq1%T5xxInJtYd-Ikria$U~Y+2VIkaU8*<#Ri=|pyYWrQK|xq z@(%;TvxXC4KPdaA=z{+H!>jY<1M+<$HcdKF!^iGnCwAYzebM*uRUlGCZk5B^NT-k6 z5-{Er%pJdD%b&_RFHXMCoo%JU$klv~^~JQy{gmF1KUB(R}hHlGaLcYT|&?a#< zeL*bxj&F;_`EXk{g>g5kFB)en$vL&g)6Ubv0}`Xz8*k<7$n&4XZV8L(97#hm=8aR-#V|;Xj$Cx*EV4%sZ+3-D0_O%eg>{#CaON!oOc4qxk-iUg)_xKc^wA5 z7;W%_s0%(r2yn7f-lj`Uo*G$bwr08Qpih5ib@r2cBS%9UaROS7V3g-C9t6CMJ@;_+ z5U*ADWOcju)YQ`FJn6HEdeYY$EIM6vsI~WsAzLM&Xr8iEXZMPJgEO@*tDjeMTCmZD zL)Yw~RS>ilkzPb}k~XqVCm{92r*bKZ=6Lo_GYD+$SFoAzU_nlrw#-tv5&k@K_sAhS zg3&Kk@R%|$2Y=sgO2_{BW(ABv(6n0_;7R3B*a(ukFgbiHkfN&mL2=;In)c+G*P2uP zpFe?=KSieZ!do{;JdMScPR3oWZ!)t~<3O^h%C9=9=b@|v2^$@txDgU(m+MT9g(~Ao zKB(-idyuX!i84%-w7D*}Dzw!buB#s6KHh&t;{DjN`6%gZO=4srpq_Q@M$6Pu9+aTtT<$(qKr_x()@>o7-Q%rG;k9;8WX^W$T|Yz%aNM z|IBI-^M~CKTveZ|>jNWYaDFJ2Hei&DHG?k05rg=60sHQ>*^N+AhPo{VH==yyt49E% z#j0Ynryh%-(qWo~6pOB7tCiH5F9nB$4|F){PM5M%;dG; z>G)_M0vfe-KI;jVhcu*8r}6FyBU!%fp(=0JVsTy4eE<*bwZE<+-JZy@6ka8lwgUrl z;Dx&XAXo4ue$mCY>nKrfjkN+3;Z)ZxB!Dc+;rgpz%Ab*&MIjoj`g)Vnq-Gx+sSKVm z6YL}N$4nWJ7NLqd6eX<)p0x>$tw6pChEV654c0eIL%+rduv2V+>sAw8L2|X&-guRx zoxiw?vyWajj%6Yu%lKL%&TyN6ftrGf0RViJ(jK^zpE87l>O5bfJ~VtbWmsolW%xK3 z#XcR-pkb!2AtO;zMs%2%C8_L@rA!FxKg%7-ox7<(!nAl*L&X%_uMA;3D%M_Wgg6^F zFHr|!f#a|Ww;NZ|1*a`rxmLB9kD#%pt@u9W`|RDPEHH)51u>L!1yc@PC8;4*O;unj zfb;U9FQa31ct_2!#mF_GE!1l2SoD`IBx~z{a4myI;&D48&HYSu&&Xz*AoP+;opTz@uhHq=xQe1*r>+Um@y6=4UpEw`dXI2mwIF0c+*D$m zNv@uLeuH`>IVpapBn1hKz^-)iRd0G1 z2`0*g_@2iGo%ZdI6Mfu@9|Hg;_q-0~Mm!J>M=@eCxPn9pA$&;6`fYFTX69m-&I&F< zAB`W=sYWu=L`hfigB}}8Gk>Kzrr#5RT>_u&+V~8?5$*t}j@K}C-Q%^?JO%5;LyXlv zv1OZM78Sj{Nv*?>c|x{9j{TV)u6}U670(hLx2qZ(Ppv5qT~r56=R;B9-iydKVlkz2&8FR*Qg$JBbDOK&VkcP;#1t*0qEKesbRS8Z+jlZzbb!C|Az9@E4MuvYl z;(I`A{5hscFkTKNHUNg%HC*RL7R+7PTk{M$Cnjvt&dQ5IXH&Hu&o1JN2A&$1*gXNo z33-ak5%d=Z)g6YItXEgNT9Y0PAKry}<4qN`5dGe@>zp={4#DnSIj5+@?xwivifnL8`1i80HJQ>3w13G>hS%Y&#-mnQ)q!5GdLGf*I z^w#D%puw&#(U=hry0R3kH{>o7Kabr>PTM~i4>&3I zXYy_qO{R@-mX1F0bao&N7#Y0VkFGPP2`&*Hx2|ebb!M5zu>s-nDnoh3-YwpbRqNG1 zbyrg#EjfXT72!IUYp17?nTbmOyKh0yok|V;{bimK-%XTAp2B-(XoWo4k{Z0?-rr zp2;D5f*7kCX*7^Xh7Jap^T(49`3F)9(fESPmP()F`OrFf&2ioJR66|9HS~mkcr%V$ z#RE%oMmVN*`9kmTxlvH}1atqwha7a=p{y8y{!1;VAn{f#b3O$*^%(WS%$+vg)OmNUYSaYS*7#rJEb6%sp_#8 zOH)=n-~_*<+|!R%~(Scwrr+ce*Ez|Dk9=49Mg?_7e)35CL`bMjfUhHS>GB zWwQ{DA^!?>>emkh7y^MMsk5Yt{i6luv^|?&=P9`RQZk26EQis}Lm~ix?-GU@}QPDc^f_(ww5JA@D6x8}Gjgk~3E6^@jUll9y z8-mLl6dFj{^?^@t$Kw)#ga4o9U7X~U$i7=LG6fODh)qSzQP7|pQ;r{;xNT2FI#$Zo zKmIhiirQEeB3yr=@N|@I0Y!ugihavrP16?{Kt~)9Ih~IC?kv?QCjZJSEMmbjKF|3D z8VBh-Ub)naLzERCro8*#-l_zq=C&R{N#SVpCoqMa+N<9xG94nC7t6QysgCW;O_R4x zc6;0n7ztiLIkEiR!hN*ArYUA#ky@WlUd6v2L*Ccj*y6S5l@l7!Ons6Rg1Q!B!LpGIUOZmt;OCY~!%79jU zuTxHRb;jUI2POsfD%tG#&e)Y&bkS^j`nl%h!zH6VZB#w3I_a7nX;&XJ8ZMpw^O5K^ zJqD~l2{6ej9QW_w2DMF{3e<@O|Nd)x_28)-T($q0WzQnF9S!@q&0h zrk~>6%EkvZ-1f=PzJj=KE%HVzy z7MgJ&9{Jk#DP3q%o#ZDo%)wq+pj11S-t;)L{k-qx6#x#Vjk`_RBYhXG7p9k@N^+Qv zRkrU{)Z2JoKx1<$0|d7rNsjf+J7?eENODymFoxjQVA)j*Uw7B7LM|`7@2*T#19x(`>a_ z%B`p}g7ex7HOLU33J9gM9~LV*TlP9+Tz|e1IzX2>Ky@symKzfHY+{tcKA396=N&`$ zJx^sQZvOAM6-x))Hf5H zh%{QrD_{9w6)a&odVthHgg=UC+9Xn!o(*W}0PDkuZp~nI!)uz3sdYHQRMk%;`GDL9 zbjNBHG)ruO20|r|>gAzWqz6V6`WsaF``4Cwe!N{r3Ry85g=%E@ZSEDig`NYzKeX2| zqMy+O4w@4V5$*3 zr_yv%tF@oKc~dWF0JndnQ=a6fsY*cQIo)W(smd-_&#<{`f+XlW5DvP;qcKK{8Ny(q zW{LW%g#X3(ua&GKzHTiEZKSo^wwoM{PwrFNVvs@a@0YvKLXdmC_6&o7@P`k^rpRuZ z(P2oc0WTn-+Mv_b`&#mJ77?)7Ap086)&OnA8)rZOP-zpJ>V+iSusV^fVeg&>?JFN3 zo_TE-f((V;!f`S;UqO=)Sx;=~2K!nee@(?8Rghp&tXH$&@*ip?CYQ2mGKPsIjG~MLj%y1AMlplfYe?+>bbShE&V>iU2HZd`kihC<1g*vmyvFg=ygS1Dp0EloN*C9 zAs^`MdrT%*#PwA{k{fjN^rcB?d#reUTg0EDa2@$o^YySqRj@hPF!}DzYw!`_1&sq; z(uCVl9n8Arj)MdX;EtY#&ZC_l+HS*FAVI_Q!r5dKxqzhBe}S`QlF44q{!ttEW%&xL2Zyb(ahk|BN)nTrzWu*?(T*h!_aI8?~kh*f|m%O$)m zGk{j~{Yx5_ywTidWp8jO^q+Tnm|5_c>As8SZD`E$jL!D5{XCUQJgxXP+65xnCe+bE zqJhUh@xZgFVpG=+HiO6@$-h$C;7^KKP z#ZuNYj%}3Nc<8e{W;b8JQ-lbMbO*Gm+Sy!kUTs)^8TFxA%0tBH3s6TSBxT~VHSc(R zcGqoh(!~jTKTW5ar*&~zq+ZXB{U`)&bkU`Jxg1e=5*_C`IW{)*3pEf``7O_a=c-^i zbb+%8h-g0Kz(}VCprzNp9|D+|dNvLWr*@FSw_*SxO1mXN24m;$1C0<$sz}<P3giDl}`)pD)P^@u+VDVh$XPq+|UuM(Pz1Adw0*qYpjh~ z8Es6xyiY?E=9oUp7yX?|PL>Gywn6et&EqdahJ$=OW4yG>I`iNkpA$Xo?5VU#A4=TKY0d?&iuv^x z8=ax=;3IF=al1v+{mn+rpLjFMK;X`JDUO>cXbP@IvT2;E)p&3uJyW{d`?Pz&LN0n- zI5lvZhqTXIaDg%l(>;uV;7>qo95#O6PGCQrWFdP;{x9V}YQJ5-*;Gp-d>h2s!x1%s z=5!NSx3a+&zhfAdYMvekh3AlT@rGTE`Ae(hZ|Q~83r^QW)5Xf4?n;bpKDqM> z!^U$FAoLdVK@ypFq*vtXjuoH<+->gLSD4Xp7Q`4NQYN!l4Jx5(K zTq6-m;zd{(!KenYlhEsivNr)x;jBc?;fck3>48Jp)A-=Hq{U(po9S8k$qxPZV$zVw z6Fw-(RLJ{3vL1{Ki8EBEQ=V-zl>>^U1j zgr@&5e*=6lg*<24k$-dmrqD1IE3;3GMAwQE0+ zHz~Q&)+@wneOBA9mLI!lWcY24vO`E3TmB9}vUy_#yBn5kqNZ^0^~$z6ZjLeyY-n)P z$X|)i2HJqh{bREbh)sZS7Jndnha$d_QejSr(G8ow!lRem=vrdvhMPn8PFM*4nAqH< zka6BU?pPo|Y$ztdv4>w5vamh1Z7L9PM>EW^4|gwA>7ChLQ`bx&#=)2T&IvzA#J1D; zv}0NouO-*)ARfR^(v=J!66czfbkpDLw|DGv+iiWWq*DVsod^EX$Z9SM=VmYkTy79U zT5Ft)o8|5G#?w-+$=M2+G%ba=8WRnX z5^#ln;H*NZVL%z#K6#Xdr1aO;KIkw_|L*x|a_pW3fSn9kx~!1Vxp^K~r0icFYET^~ zRKAa=&p9NHEGwf<3*B>b!EqMGRG>r@f}%I^Q;SX;3U+~>JNfw}Lcpdx9%s)qOL!QR zxv9<4?7M+TIIFUG8G|@2JmB)T{#5V!h=BX2d>PNhmfTH8azX)iad$8ZRnnpz_Y6&# zm_}2^swz6TH|cDfjq_T=9r@{*D8I;qt(YYiY*V$#YH*I2+=%v-blgDvN=y>h3`r(j zcNiap>`~@Qul=;cran0hjx&ElGanwc=Tl%SgZnwv6rN$)0_R5H_U_Hmtek~?0AfVv zoVjDS^|ef;NUON#-Cp(5bH<+9@A`~%e^$}#yoU&D8#qS7^LXLTmqcmHW?~-WC1qX{ zWPNES-Z5E|)L0ID-}r|TKet%WUaqlC!=T=T_h`eQk(z}dq0pOXJci7-hz+o1O&>iT z!5li1Rj3N*;5h2&uEc$qKJhoLdz8FqhSLx(?lvmU2aa>B{CCC8a5TvcM}j97w&C6q zJ$Wu?UZOVIhVkI8Y^v7xdfv~mTrPcu z!-wK`2lM-7E?mJir231V+jH1piv65*%#Le`gT`~LeqUY!dCOFJvxp5KaeRONt8YMmz6-I zR!YDBrPoYiwAzYvWC`W4Cp26pQG2Pq$%4{cKmlY$Tr48j_$fB+3f1TqEWokJTVvUE z?V`RfRU~f5kY^%h(D7xX@-sXx%=~RWMnfuLH%Qw$MZ=6MXVqneV9e)-{65~x)_y0i zP9te(^YR%BfD2p~L$8|RrfrLfd>$WO$45q2w~Q z$z-bY2%^ZS2aN7NIp}YhNe$sLIv?pjnt%_;V=sUlZ3$sL4Ru#y6rL#R{)w1=76$DJ zIho+%z{%CdicnVAp!`OIjvG%$V1kiRCrX-nhb&L6{^rVZ0KIWR8L9@%jo&z^>b_%4 zO9xu*$j-quS=QFQ1H7u9wCtu8go@t6k>0A{HW6vx@37C6G^-=-MXwP(Mt2K{-!h@} z23>(%!6S(_q+t#9v(CfXv5)8-DN?U_A)y^~c?Ieq_x{q1A=j`)h#ii}nFwsx+AVbi zbrqPGil(7Y*9+E}`&jm=qepP$5rtpYhFu?sZCXo0yrj5&`o0=7EbD0y#mN)sIlq-d ztBTYLmT~pQBTjJ7`;2~Drm83+eH+JvJYFQdvO%E#R2b4Uh`DL({yUbVC(?<_r+wb4 zdX$;5sc7!o>7E^E!;WbK@CXWrB=2~XKSltwgqNFoP?H)0RVx(F20V?25zVwfKkSie z=WyS`Q*}z>LWL!&QdKfD9ToWZA}Fc#EqI1eKDij-c9}fVuqkAd0n+S6)NpS z>Bhfm!8LI692WpISiH8j0?~uBQgPx)xJCv?T50|_ZElg+3A?3U0T&HqohGz}A{`2$ zR)LLjZPG?jZ14W3G*nE;SFD{ep3_;r<$?ljNE~v3@&D~kf>^u|uNyEi09OF2rZ$0S z-rXFG_o`J=ImTIWlY%C%Tf&QU(mOiCE#JEkEB#BuvX3Y(9GSgmp5MMfhO?Ft+u zG6+3JEtGmbeG2iu$<;va-VuD5?S#7j=en}pjn%3-ap{wzA8kLDCx<-~|7nGR$~s7L z4`7=6rHPA%D{sKVC%jAxoULOM2NNPdD6B{xi#4@(l)CJc5-}JT+7_^%C zx|lFh+L%FD!Ng`8{c99_!B49QsW!hnObRbJc+Xf%W_;CxpRgA3b_>kT5(CHViV?}?sRX|>(FrsewETii)J3a>vx*v{gL`2j za-jQC8|YZwP+zeBTFxO;wT&Hfs6^w`xR)?K)i(ZZ5siIN=XH2kFS2dwMBzhozU`~GGD;#=9y^U^yTT*BdZ^1y~DlcV6StxiYvED$m}fOda{iihueGD764#aZ^YLV zOQb?MeW$}Gmyx)LVEx*?yT;Es4bp2y0dyL25X6|hdUTATj`qD}=aEc!nt=$)<&}IP zd05FQw@A-@%)8!Sd>jc$QQ+_dbSQ7^M9n6`T08g-CF;+nGOmcl>joWq1jTBB_cYYB zQ02$cdvaXz9IIil75N;otxGN-Ii2?GtxX&L9Zd1292@2z)oZ=Xd9;u)>_vb9*PrmlrB$L?aU19K1QH>K#0S8XJMcb^A3$}zfyEHEzlISYZPfGOSp(}}wC^@jQfWfhJ0i~UX z+>;h&t8i<9bOq6f%`=@N>gy%=3T&!~+pe6JZ_e48e%w=9vEWht`Ik73 zcwLV8u^jd8kksL*d+ZuK%UI0dky$1&t;3&;ZZMy$pg>S=0PSBcZ%6w>P@T#ndg{LD z2ArW|yV?!`SccM?MrUb1DU#dRj*5*r*pw(DamTv`qN6VH#&L#InBLdc_$}ne zyz;R|<2)|UP@ReV9bnpzSsVM_6Un~_c^~GVki>y0d%T-ZFfJK5vjo97hwZ~*T|an7 zp1$p12D2@`=>`mree1wKVLsY!2`)01KKj^4;x^Ohk-h#&T4mqF%oC3g0PCZIeUn1M3!2-dWUQhhGTSeo}(V9zJ zm^N#Z_qSU%g@LISq_IfQ;U#S6zJWYb-GLJzc1n6-4WrUbl(;!2DM83O!a2D+*yU?vwccQRE{b@i3}pi{5Ou&iuQc9b#!z9Gns^*<#>5pSR|l`ybP z<-!H^y}MZ=-RLH*Cyy0V9)3Cde_PBA<-vbj_5-f@@RgjsPI#$t2V^9&-q~KnVO0b7 zN+dDfSLl8^FEfXT4W28WZs7?@V4zm?5ZO{lFIlN~)P7KS)@37mk;{eAn2Ai298vg> zr4DoPqlO`{odU8ZmKac4x6T6DrZB5Ji8i~KnaSw%!J?wRx zxSn*P%1mCd z0ii$pOW~dJF!Y<9i7{oyTo0jr;f(701pryLOjINX04~l8>x8bJ%MLAazew-Var|FG-<`~ zd^}%~6&d6E;CaBOts1bsmI&lu(vk#`XGV>7fp-#slC8*vshucahqQU-GVDwI`iNz10$6+Copz0T2RCx zr{RB}K$iD{K5lzTv(PobdqLx;z$b%iI15D4Vgsx#Cby46`OV-=5kV}mpp~uh&ZR42 zL)B-fH=Dr8hU5X#&DJKDlK(=#dPg9Kk}4^)+ytP#4)E>zuie zjbOj8iBqRon8?OYJP~%Pwc?ksj^2O-6S-r|A`pIT2Fz>)w%R)Y`R%eqvCA{k7592^ zWEaGiN)ynO_4ndd$sVNda~LcH6M}d4wCChXH`R!j|H$QAJ%WH^UXd%}kM6Qn$7||Q zCK2aMVpnF$TF%-5@p=_!qNWk@y#2%phMdGe^WEBIN?_?ypX8U-Dt?oY=jfa97j7~L zC2oY=z`F~2060I^79O!bU<1$9@j`Uq0QlCcdmJ(e6IKhz|dZj0dzfFda!M?ZR1}m zoL;_0Q0aDh)w$L3Ai9p{GOScmqq%X9$`S8arkMXeH(oGjwmyOYq2>S5G8g3$dq-Ce z{5T~j>(<}J?m0WmutQu|%r?t-C-G{oCBz0Tg|Kj&8zNkcFZpepZGz68PmOHDS(CJT zx|;Q!Fl5@**p6HT2eXk@-lky)Seq|uFSXXESjm`_LGBK!OdfZun%&L5A-s6~U6zCO z1huo{{I3F$f*e*MdW0|q+P_lqEbG#YxtC$-#N^;_a-sNa2Q?i%9_~j4nwZhp5rKjECi&~Cpii?(M@6Fb;$PcVr+hrSZVLF=U7)X2VELz>5wH9q z&)+6lyJQ99@($v-0?T1g%1!Tb2GeDVgQv}ger5eM)d-8omDNU-Lt9KXxLiUdhqXE5 ztLZ&i5T=g41KgZ}NJBu3&h5`d5+<9WG*ZvB7{hp(Oyd0(&kIpq|H7hm%XccY50)n@ zT#1}3r0uYE=?v2*%wigoQ>gt?;J{Fpo5myaGseFa-zTC-eXbiwh{q~D2Ynd%cz8&C zfPa=5=__oHZEiEzVOVxUic-!@m$-;g&{zL=5mE_Z8EX4Cfb%cZYS*K5x(fUk?58CI zNOpY4n}AGvb|J{^4|OcE;4qq`IvqkTEPlg4s88s8KqmPd zZR7EK_euNW@{nzgxekh0RaTK?Kaqq9bS~@d^1}V+*QDL zTnEzMoY;XS`VT6v%l&Wa73hp-h-CYCh(e?LRYr)1T#TYO|92uBHV>ySJP+ODv{)0{0Ph^OS=Roe*D~$9r@N z#*ABbirC6%p72;}rp17)eiHUk)Klm6#tvJdbW6mzvWZgUpO{B;{MJ~6gkN;4a2)n; zDmLsvt*Eo7m<$I|+hcpc52ADQHB9t#AuR36oy?k@6jqA#;KrJPc=4F+wG*VT-VZkY zumGd3c$ z;9!D3K4+Yf%%N1}N$dMFOHB3TpiJ4H1ppC8TYb8$;c` zsx%m@uEfp!w>g9)vEGE3St>kf@s@`T8Ao}vN0OTqa9I(h*|}BjW)*u4*PVW3Ud_w* zc&(H&z|@-;tXTuDBEo#Hx)D1P=y;-;pcq0S^f1(Ji26k;<<&U?w}o;fu%+p&Jp z8tClrct1xCJw5i|FV{i*Vymx!xFF_(gB6p>YYCEUZ;6jr6QV(@{M^U-;$9mE;6|0+ z&kAJ+F0iplw}HStpzQgeL7v*HR9Y7&r#k@;-NYIi3qYmQ!kxMD;+d>qN4r{yqN_rm{!-;5g?*JEK{PA$Z|GV|d zoX2Ul)G<2^H1?;fD!k|l;U%ZEim#qb;!FiW~on&JC{pQ2hWbse`T z)Sesl8A$6+Hs>>JbXVtWP&d?N6f^{!kq zg?pB~?(b+rl3Nw#x|hWDG;Pgrv(9YO)z}^RdyS`g#p)_zdbV(V0cBlhKgAbM;b+5Wn8s!r9;mf&~NEKe= z+K$+uY_@kioG|1@H-Gj8SXK4^0c4}PH~fWDnn~Hp;)dT_mAveuvT$NM(@K~1WUjCA zEqnfVS=a<~Kd?&wvpp&bbOMKphldL?7Y14ZKPGJf87iaBFjT@AP!-!ftLyiUx2a>3 z_J%qHIG@^RnMH#rhMwBB> z)ZBpp-NL>t$STASe?7_|MX&vZust90+DS!#K5e@~T&)?3otE_x75&>iu~KkgkSk~# z4T%vfMKr8~I31_71vq$H1vsQWvL#1%5+p8l)<&i1#kSxmW)9UIA-M9-RGls7JyT~&fOCJ^$i%$9@;1Z;iM%_5Ua%Q5|!? zjupB|qOOc}h=G#=O#irAZN=?1R$$>Xc@30)Nz-Xs%WZ}O0nkru@6D4)SC6A0TcN}H zkmvLJYmQHPF6!5AhRQm;PwQi*_friBr5*+%KX^v<=HGxM9%Kqy+Qy61Wr^M{(L4$J z&sDUnv`uA+?ycb_n!h|j)RZzeWH%ov{)2#IS8{m(LBot7(**S;L++r%UXEI0|CpF( z%Im6dRz1dDNA>BR08O+1W_g&-3Na>L8Eg3YsEjH&gj7SG8T=5_P%3m$u-Uo=!U%@LlKu7AKvP{pv)`vt_bR;u;?1&3zs+may34ri?m{m`Pw# zux#Xwrz!_I*qRBExR$Du9QhX}6E4bsYjMaABZo4TUT+YmB4sWn3&hFMl)$$_hW@&_ zA;3cwokMdNUH!gX-le+X|B4<`Ui@RDtmJ1=iW6!$F&T$D)fbFZ=+!D5__CFY9Ir za%p&47Pa>ipt~=MS9SY4AeJjyBTpO?zDBQaC++jUBO@1n(Q`zbMq-~cc!(ZCG8UvI zdRV9rlWbd~Bte0|+qmml^{E&uoU-k9S2Io~;!%9a&0Hj}6k;AwTLH5c&Vo1!V8t=J z1RlZul)GVRpriNijaz~{CUf7hI5)$-A4*`Ed~~g7nbbR)G|5!a2C2s!Z%{L=j}BIB zl2f~LL^(btzWgX;G;}x|)hHOj{}M|+;Y-jwd}fF(`Pr(&&me&$_}neUGL<+gE{R&0 zpkKH<8?va5~^Y_L!JA5EMvZbF4;N3qV>$QPSsR8CnYWjx=jQ$xO-7c+4K|%z65hWjNb9=e)dn*1N89x) zQUN9%AT(_+n)r`DQ+Bu)Sfr0P48vDZHn~qN#l;tO+>BA?toI`CWO{_Z=7Y4}iTdD@ zSVai+)C62AI+6TorN_>nlcTXR&Kj}Ee#J! zPKuJ>UXBCCRMA?GA(lePk}W^-Vl*LY8-Al_^jV+!4Cyi`r@46?~oFTg6F zs@T#zuq7LNAExjo&I#c33SR1=twB;O&0S)zzF>?3|yESaF!f%5Ik z2Nae#RSJ^^A--F_>kTpqe}R!CwWa^3U{MAM*|f;!snYiUk~vSTy}{`^SfQtSK;nGk zm~F4%v^c9M*+N6*%9tDq2T-@@@>nC_uZGU=C=!zDR=Bt^{AU~r5*aE5S*@<)q@D(} zvpk@G;?^=Uh<)7;TaJ_4UHr=|&C{5?u7s6y}3?wIC*0CSQEMXnxXPud3KYEgi=!R|jq5&5)2vf|H`zvDYH^-;J~cOt63ZLvP{hr_zdIKwu&uwZA{j z#I^)cvFtQ#h`|HNPIbUAu*xGX2x~L!ISxA zNDG5uUxAM2Mrh4vDhL8hpA$g-Wy3C&%fGu)t2UTxLKeYv8Ssr&o;8S-Tcl?T1oHFo!Ev}?dDnP`MB9c%Dyj#?-H65m8QK&UA{+94K zC&1pJbc;sEy4=T4=Mc~6Q)8g?K>s!BX5Dv-`VCAA8C-teQiYp$Y$d?K35K%zJ#^Ho z?vC|jJIQQ&IQWiEJ-@L`CJkloVc*_!37sEI?F_fS*U9&;9^OF~eo(+3c~Iod$6S8R zw_1=){73uD>G2E|2DbxnQrd#G1r=zy9HzeX0*zN+)i zE8RLpok0ht2@#aZfXZvTVjidZ^D)E+6O0u!dc-T4yuM0VpE-Ezk*5A_q{~7-HT$Ya z@C1AZuVu~c8cBrcW}_(@zw#j*y$;1vHGtEGnX0|^f2SLc8%8L!GBuE``V_5iIRQqwX&U5S^<9OUsVax4P}(*tPB$Y=?9&ecGF@ObImy@uYV9CeR2PkdA*o+|rg{e|YS zoENzsZ!>AmLPSs!Cmda|YoHOiTt?rw&45`kqXgKv_v<_QlEgb{1jSP#?yh(>qI4w9 znB0)%Wk2RxgxgeKV;$7goQa%0BpB1U4h0{R3zWp23)syV8TY!|14KVzZl9ab|S zbKf&XhR~@+ypX^>9V}GfXTe#j@0Q7eP5kE$=r#A_dD}I7lam*F{oBR(C7waH1bZ?c z9(tWksQis7n{S_&YBuSO@!!SA-7ClhjaYgc{hcxRG*tj-fR*dmwN;AVfy!Hr+J>lJ zO$#a|0SZ&-p`hwD{BD-n1RK-yd~VKO-|3p7 zRr{$*)5n0zR5CLG#+&IXaL7 z?-#MtG11Xw6WW#FKG1r@XJhe(-PkNfo;WvO=3si(1o{KP@OZTJL_@<@jma0`*zJ+d zF2Nrouy+>#%e)7&@<*-_LZbK}jn9MG*cvYK9b5l;v;M??&=LJ;;luH;kgii*0~iGC z1$p~Wh)6&bcgf?0kBwri?fPd9EK_J^P*H|1u%>5fq^hhHLF`jNi^Rx0PE#%v&%5BpQk(Vo6ntQQOKaYZ)?I2#z*US+a$4qcOp zs1EqZHkne9Lh0GS<=})KL11VZlT4$o(}dflGN*HJ8N1c_z}L@fR{qks`hx58HeYu| zSd0Jt%nY)O5W#eVU$$8mns^K|BBZpqiy-K4>ciu8{&;RE`WMJM1bb#6J~4Blx)YFFgjvO!)*eSPt=MMq_90bgu6)e7UJW zZEV}RR$^vPV=QpY+%u3a3eYCtZ*1GPZO<9oHqY3$ZQHhO^NelV#{SuEc7qLe->OcZ zst&q4Yt+NWV8wftjvc|t)@1UMV8sh&UI2&k-S3#QCgHYokrwtUoU{t3Yvz@B$By4Z z_Mb$W>_3J92c^%kH!kG}zzm4S^^9~^%ngix{&r_*>Pz zrm#wU8>G+k5MDjC6|+DwD=k9~6Uw>PZWSavX5}b;h+6O5$|5}M;?`r+PVQDV9doNh zy&X;JXHxcuZFQykiBo(-pY`O?3?U9qqv@SCA*gC4BCT5Qe>l?t-yLvv@G<;wH`T9d zxv48x;UT9&`9(k6il2bg82RHiFn-nFdLdkS6{y)#k?RqA3(+cCsQW_uRm40?-Wa*% zRN;Fpr0z@~7fxtD3DCLE{{Ay}xaIBm2BCV7?;Lq0V}-?lJ0gx*>N_D4J8w@D*?^`~ z0GEUcX2{?^+RBogRt@oXWHjnA1>UJ}H;sR+vVKYudPgwiFh}}1@?fhRoMygY+~4>n zWr~-|F_+I#+2ea5+~~b0Zr+XvV@vq;bG58rizo5*);G8&4+)X_hxr0KU(lajaA(Qi z(nMycPisZ4sy%lMUj>Dr19uNu!SZ`E6FBb5(uFrD_#m^u-=;KI#;^qG{nF5EfxyP*cYQmGF;SD_AUWN1f zBS2GE=rANNstSt&=Wr}}fbvVYtJ|(}k0;nVXtPx0y5SK_rx@sn5rx{VXdq?e5lf!s zrbTp~OTV0z@>t3ywt%mGi^Q3WM^NilisJ!VPqhoFAR2!-W10GmM;fSufg1-x!R95H z9Lfn8h;^78H4kFvX*(`pjslb79pU~3jD37G&xu44HEeNua9~n_72Re8`>HS%S;?xT znb-Dx7##eyFVam&z~|@>cWkd-jFzwg5n^dt?uBxA>lBwQqNg~s4d>$m5UDazb4P=T zyjrZl;zmGtP}U@WLK2}Ijc@~UK(}Qg1E2C(j8@TmYKV(*So^`4#`q?cdMyFO24)n9mzA|AZ1iG9hp^ z=AzSaQm7+>AmHqpB7;Rbz~>Ax<3gQB_B_e*^AB+=TjVb*9HHN;GAHOhuPK5BK7zQ1 zalU2`SakpuY!b+zl3+m^MK^iNhohB)`F#dDoH@!r34VMf00zMQrKSz&C@yVD7k4S~ z^GdbEr*TaD;7uRARRG8_Q1O)4Q-$Gp)k0n06Ud}8l?4NNZQL4skyJ2}WE7Ppj{j=sqDV%5mi&SkBf=++aD3e+fZ8-P?zURxyf$_!WFP<2vCQ#i+xdgbB#Zx{slUvm@g?4f~ zv;7{dD#<0=$^A`7^2{^hCg2EVVxY4e6(60&F8y1dvZ_mIpDwi2r)a`2v75P}7h zEvGZ_=sYOjiC7?6wep$h(Zn4c&lvt;US_8Pwmo<6U;j53qX+)VYkrDfzMJAVOu0G; zwm=QaM@R6M3E$LErX1CEtm~ZY!!7HbCThK1@FODe96qd zoQ-%bY}l{rL_BrLtIjAt#YC~6kX*I8Vz~m;dX}J?kz-_r(g)^{tTk}%*}-&3&sT;@ zA&8E`lvkjy9Y(T@fu>-X&f^vKi6dmi;8-88eV87fJfYeN8gk<3ub`5igT90@>R>Ah z3pmm9r&$)%&w$&VCrkarE_srd7wdbC{nPa!fm&A<80YiBmk7Cn6(dRAaW8W;xPKOY zkv7&d4eG$8j+y#~nrl+%9jW$GRq7|N;?erPcw(>ftLi%kdb9qrP_4e=U|6i*c=#|1 z|8(y}|2A>%x%QR| z!ZOKA)o;jLP5M6Z6NecQCUHfn8pN85iQ)H`3j7%AVUvScg$52wkuvDC|C!qrk+xt^ z%&dwtgC-tR8BKkSE2(?-wU5W!+1r#)!xN@u=%%kLnt2)~{gIpE3$Dv=K%B`QP;XZT z<(Eyk{5P?EiRWh2HNN8&P&txN>6RE+n?Mq@5_9i48H_!K{wRnc8C`dic}^KYxk_rw zSLS(RjYe}i2%#W@7b$o>qCTe&>I&A%mFtk#7Zo}P#LMbK$B*bMO*Tne*voWZ!FhSo zY;+nWZ^Yk(!Ws+q?UU; z7e!|xVo!ZlR+A}oIwoOjQpTK{q| z@FXv+{p_OEuNzl$?}v}H~Hi-%Hcq^1_+5%fKMS90B z!P%u@VF<+TM$j8_58=Dzt)XcR9EVruu!n=Cb6hfAl84?`EjRzl+iAkN3FZ^y$F7&4 zo2yc46BgNACBTir4{~O^j+OT7Uvd;%F)#igsIcIAG|lt~bRgJ#IpoNIrq>rh5_SGB8*!RQyN znRwXlP8j2;k)azflkcLl&trgIYA*4Ht9@wC*I+tM%U_RWAu0q+#6(cNA!Kmm9SLk# zgAwoTL@5D%ZZlD}q~hDwuwp8F8tzl#5j$jYNOzWPwraQ(aa3k9lIt1p@Vt{h2~D>EeD0UV&s!|}5orRC z5o2i|2{8??;|3+!zNNs1u}$t$o<*v8f(`CuxR+Au+aqUaXQI+O!!1@O3W*r&*Brc7uk-h30}~%(ztb&Sxl`7k;&-` zE!ysHqFy1CzA$I?rlWJcx*pyD%a{jYRF<7n1B+;+F$9?+WE5*}ls{6TGsk{%XP0=O zcVYF=%Wh?d(vdgK&y9_W!(^wxVXGiG6)#UG?Npp8Adds+Ib(^&3u^S3P%oCb zv~EjKK{Y@{inQg`eWFhs^S-#FflOl|?=&BN&ez)HG8LgD=fXxb2{qu-a;|FEP3(^P z>;+{k4xGC;!f;QRPYEs*<(FGd$OEEJ0laZw(#8-5j**Gz--^x2PF2RQ`FH*j+(_k^ z0$cpVYNMEU)Eqrv>3Z|CP1I1~T4mt* z-PT@|xN@T1gYu+ltv|rtuZ&kz4GCo}_+9Zz*plwN_Ix+wEi!*$GL%}jWSwk{@NSnY zCS93Ua0^yM3nl6i2Cd9!FERl(3!W8rXN};I{;o^5_B6ka%cxV2`cqsZs(NZ`OB~&5 zJmd-7y;8V!_lncrltUowXHoLQ^z4`xAR0Xp&Ov-;{B zE+Q+RbD3(3 zM+c-toLV=Pzq4j){cO@GuHvL#7-NkiaC_OSqU1iCZo~RzSYXoS7z4k&E#)COX3p0C z;!EpdYEXu@LPC&AT==M2-+$%ADH_Xh^c|?hdL%u@I@huGkf~5xAnK3G?8stI{p z4foeg7%H9!b?66Nl5p@1vD<{yvQIDUbow*PikV%6I<|cEqk2#qmmF+k{I=|JOFM6n zzHgIkk^@k+vFAxE|M27?)y#zz&}TQ? zpr@&J_VMF7Mx{SfW#%PBcr`oH$V{StFj=twYnZxn5(=E9nr60f-*<#z6QfrA8 zhWVKizCSZ0k*WOd^?0sQG63le4v$#h1*>s^l#Umu274UJM)w>UW!bkWp3}4Igu4ZD zQD8{*ru+Bwjgybi)Be`Rg1ZEA>CX+0#ODn;yT1jm*x_uGB`L!&nli7LQ2<8(z3duR z9;+s_;+8Fw39ijMmCw);@vCT z1#imQJ%aYi!4(VvD%IFyQ}gm`1WOBV%EcL#n2l8sxf;9Fjy%Q%;=AM-k)P937sIG; z^)r-FRmtXBZ_Rs2#cB(w#TZ`cE2WS9MMlLWj2M#J^id+NEQVJnUEdO>S(P-e$3oPl z|HNH8hqyFz=7luyBu_ar$x+4U^%irXh?=o&=ePrzd6HoU4M>}c`s2iz)Uq&XXA}Fo z)@Xd--JLPsLHo~j+&v6-vvc4*gj4xQP_(6iXY>_;34LeXMWo@sFI4aUb2J?2Wy)ZX0 ztIp7XAGZ$Q0y1cf`Q}S~rq$Nh0U7yvUI&PWu6AU~TR{0Isu32t3njM9- zQi`h3ye;bC`4Rd@mPRoI?r%dzc~lB!nS@IxtjJ&r+KE*GA`V=WOj}<_TG!>_W}XV> z4=+VgV%v;-AK>7{x}+)y)3!{d@b7KJi`kVESW(Bn)f$YL9Gu#=juFiYDX3{HVm6cT z4yvWqVRW@=Z-Jr{ppA{v8hg1M2v4F>Z|wmn%yO(1m2rP~swVJ@j??FoStJRJCsS{C z#Gzd?l`3==%bqehdxUAcDJj~4bss@|>=?$Qw2Ha}Z;hg}k~X(0WNWGmB{*L5BgvW; zEl6RKSLEFAC!N<7rQZq)m29VISgV3Xv!MyjTKkO%sdIBvF%);Zr;>YT}bIYiX z=z|`sliZoWGnO8P`55+*GZwD@&U~|aGygTaxniJaC&7PfI7X%}-#!hOyX`cYPdgY; zfqBRq$HMT|P5>Mt2L+GJg|nnDad@MYfu^9-*F5ycoD&v#PPw zD#L;eBOMHLUXLBQqZblgrg?+_b8Nh+i%3~M*J98eb38_F3e?+MJQTM(w!uK)Sp?2f zROh4cin;A9GsJ#Tc}&1lq_xtJa0>ffN#uO2t7p&*NN+pd4=umd@*M)?&>*mtJHk)- zhH$sGi(Cy97?||$l-+C4Z96B}j6#b2Tb#M&B$M9@rtLQYDQ~gUkT+;rsA};*&;@a> zS^Mzex9!ij@R7JvU6cKH@0{c9Gw1m9{isrr?Q3CR$LTK&sqPOiYDPB^xVN0we|Xre ze)i>_nn2F&$NJF{cA4S=3i@NmMyIygNLc{ZWZgXIkN<4e!Pc&IwgW3!?i+pm-G0iD6+0Evi z>{CQTs>3%lv%ygU;A%kOcH^8eLH0tf+sYA|Ww`QSRL-$-i2Ga<4DtzDO@r83*>TdR zi+YD#^$a-fjXgW_gbb)GZ?HqH@dNBJHto#x^lK z(0@^djnSCxiP~k1!opBOdo(=`Q)&9^2&V&@69N2o&+;Q(#AU&9Ah_?TE+dkM9l+=Sq zBImG9cO%AlrG~X|5^bD@MUd}MFN+qTGk5&$`8F1tLz?~9S&RF+mUE4gUb;b$TI$IB z#DQ+pg9Y^4>yE|Fm>}Q%^SdL&kmA_{+!n!el35}OulO#2x~0x+{?w_o3P#+ImAKM2 zY{@t%Rt=2yJ(OrnF9u1BQ`n$xV62S;EA~!sETUtrrZ^R1D|VO0hBfg%C$*j|345-R z)9-IEs^8QHZi)=p)D){~hsLZHQDsa;pQMRlun73zGMiFGHklT{nj&9=={E%ZEZv zv=w}A>_JV=L=Pf$FUY;P>$gh5;mpPvO6UslYEi$HfT_@e zwX5Q(vuxPe31R1;bE-i+L5uAK-nHkvcX%y}ZoTFJu?C@S6@RTh3Xtn)bzLS}_hb;J zRPt9EW|VN$hxLaqbQ>6MA(q3AlvE}>f9vTIlZ zRjRBTj`1MzAeS<#ukvHGda*eUNl$&C;f)9Y263HTEynsvh*Qn1DcUM=aqCl%{b-sM zv@FW%HZZ8e84P#V+_b?#rdKOE^ApvEIo!eH-4muX7}PmKDtE z@`a4Cu0uKR<{Lw(nSzM?pxO{PPaMS0>Q3-E$FS~lJ=w3W(_e>jIn)a275`CYcY(J+ zpoGD6vL0-`!TG*%VHdV(&3l+BUL(@5N~^G^W7iQSggD^$ban{S46Od=aPE|s{K_?g z5Wq;EzWS&$PvwB?1#_Sg_F$N+MmSybODChE$-u2!udMf*H}H(!%NDR)*={=L)LY^S zf<;7Yl)qwUBEZApG3U+&*OCJa&yoL`%rU$hRhkB26t?nN8@tHZL&I4jVB)N+(Kkoo zYFsQMcZlcvjGEPWN z0t%wTKY6odFU^}H9cw~>iN;IUrNhc%%dr;l?n(6kn?*_9I@ePhr8)Z7^%!W^1*(=3 zd(BmYV)}2+Yagw?hH3!Fin8i*FVCo%0O$NfiUnkt zH%*Gpu%VvKi8jeI1fty(UX}_-D6j^$OS;jT@-fpHX&2P@rb~&h%>!_CWzl$#sy+GP z*8`UQ-nPleutr5`xYBCdFIn=4kX< z>w*we=&ba&%fv2JeFp~zA(5%g#e}UcJ>`vSUo+VL+0r(x>PRQdp<=#V2$q%@cHu-4 zM(qy8lw_{9uJ{?37gn6U#s>73|z8S)9bAQ6u>M=)8>Nx|h?FD|XQqdE5@s=`~OuMU{ZpRw# z2=jrw1SATKlNy;XQsYQH9eOV!j@6l17nzK*ZB%%7k6ntAEPtr~iX_>d#*Ulq0}|eb zlL*7);l5)@4KMcME}GzLZ9zS9;5;JgT%m5P=@3k1C-EUfGIf)ReqeYgClgOpuuk9TP*1pH!inB(VTa``J~qWYQ@+JK%soRy7;A>!{AnC)_w^@TV9))Dl&VSbK&_yWxi^+Th3Gs*`3KeVTz}DwzTeG2JkBx|JYgnQc zKi9oGSA5w;tB5+MwNwCIk^$z-sx31MzHKQz2Am>xz{OQIU8v#KmO9_~LXa-aq3zLiB%_Jh8+8mA6xD| zf!ioc=1@7F?R?%hRg^?CS|2e1wo0N1!I^PDm3<_1Vyx*ggk7M&(Ip8g#Je)nn>43;dYi&X{+Wr+Q92zkkR{f~rzza0A!QliUJyf1Ai z1zr&y)==k#;lTn@&4;?n81bW{7^07j)$t@Gh!z2m%&$C7)&!+z4@$0d*{zk%W!)nw z*M|_wpNRJXl7mh<`4bw+MYxBRIHyJzQ`(rfc$$na7qz+7XSg9m8Q;;7P}&T=-1kG8 z>&MVXgk&lA#RXD@5Qh;E%-xNNP8628WKm|ASmvWI8c}q*BVCPnvucwsPi;}lLa44` zasI6euGf-7Po3y`@zvFaFM^6f*LC*9Q+L5(Je{^TH1S`V>$ce+rA@)y8@f?6X2;UIOljphy^;`JS)KR3RjGR zcD^z(4KEn~`pF~RYJ#cqSQeyV%=BB4E6~-oJ^40cT<}Z0^XUK&uS{Ti73Jl0MbD82 z%VvL1C4jTxR^3b3Zx(?E0uz|j<<7qJki9>KzHdRw5VwdBkte{m1T;f+(a4XNS3|AZ z4wS=&H0Hn&m9D5vL3zH!*c4+yDeS%6B&buOAOxVi#76-yzlrb6W)3;D#XI202KSPi zbRIeIsQm)B_XN>@&1&JQQ3ZCFly<1kJ@{0x*YKjz@5aQFI#(SY9vqtrjZ1 z>CN>hBUk^4_KE& zHhM}NC@$dBe+i;bl}|%J;)CbzYmB@&Ctgr_d=_Zprs|#HF_J3$0z;b>=&EQQUoxY6 zXWF?thU&r3RsZ=P#C;L_Iai_XHy*>q&eU7L>|buSX*DkS7iV*i=IBfAOY?LuxzizQ zCbjcjl5I^vL2CC$3NYAvFLs3q^_7&Oqgv+A2IGd`ig>_=#_2|aa7Gd^37Hcbe)ZEw zp9Zk4Y6Z~CN~Q@KXaaJln>%R@AFm=+e{+Y}b3Jc+**b*gHs{9vVD_g~)KZ&)sIR>M zAN^=0Mi~*U%GNd%%uC}AgSa$_=LydzfMf-ddCC=C5k&_71(PY);OMZN*=WS{Pr zV}_kx;6r1Y{UA7JdF*{OX{u*FqjdU>yu}FyqXVtU!apGT%eH%XC+usqEj0y0QG|3# zyr>b7SucfngHS+z7^?dM!toaFR>LCEEzYRo>dPIGMMI-;W9_B&d}dy-kPU}*v_5*+ zdz(WKyA|1c31S`-NQH9YX`xBh_ac!!QEiH9YD=OL{}v9ke<3doew!qg(n zVAHaxzB}w_mAr`IMGO_Y`HmRs!5<^YEddNGH0jO+N*TS|3ZgneVPUAKre#=uoya~Y9w z|GsQ;vDPra@6QF?4ooB@>>>BhJODw&tdV)?mPjYajOE!LY5qX*e

N&ab}4s3C6a znvyz~o9L*yBTlyD`yz~n`N*Bjxb`M?mBL$%9=TH>$2@6R8FXZ4v%6|N=~_JF{rl#- z4T)wJAJb%=ji6W|wHr6^xUEZK8%8ilCcrPpB6uCfgYXx`;)jbONrs-$uv$P2OoMzycI=LOdTghv2NRgcwh}uMQ{75#h+BU)I-R z2>6Ilm>|83=Hy#t2-6eIyN!?rcVLhZc}j1moh(PNclOdg z4&S3-;raM2KlPfwmKM$>96@sBhwvxp{QW_|16bJT+}0kOVU?{{zy+uVT!!JUkG&9S z&X}-xbEjmiekar^`9}4c3lc{$i8>~`LL5;=ub^*>sPIoSF~8^;lVt9|&!^z`D`!zX z5a-{pVp~}S3fpp!7H$V!*3QQ#YQAivu|QB93S?h!G)*}fMfu*k?szrD9ckLMC2F}x zmm2q#m6K`he0Csm&;p?vM&urAvpG?3O*fUhVx>fOGPZrE2I=$#AA+un>;j4)g$Fva zgq^Gc8^K#CrBd^rTzAo68@-1~)1@{RQY83^25sB)O zIP39<8xQTp?!IUgNbyoXud>oFjAm77%_}^Gh?laRszpQ0Q0aRat96?z9P;~jH7eUD zCTt;Sktmo}4nM55Vq@_Drv@=;h!1swz1@|-2>K(u_3LSa2ZRUd;Y<@B+Yfz4!G!*k zqTAV3Yc#);Wv@w#K-r^LK#qe>407Ab7f|>c+TCA+3OFm32456yix4Q;ESZ|YQv@k& zC`++f^1=EPGUi?Jl*^u_myw{Fv=xq&_uGj$LP~tPj~%o1%RmkNqYjP;MN4yb&5vvQ zml6YW&j3N3{+)~+I)X_|O~+sR;zu3u!f(&W$Xn;_oX5$>O!{NQ z#ns|^FYQa!M_6~jhsnVct@YlriX+8`hb5Mg%2TfX5azW#PCyy%f~bN*8{2SJN0ars z<7hnT7y&2I^|T@2iLTbRL?`eB%M?s^SdCu^pg5|#?LOo?4OzvvJk~b{Ev0YO+ANV$ z9NiV-I{-Fi=iQ41J!5;z-@uMTiW1Pmo(=xPb4A%MW=3l}J$Z(3G`|`0PXr~Jt#VTI zZS|NMlQiR~4G%k0H!PcONoHmIh8|R3(W|LfAKeRTNmlmp7W`nHp83mnzfBWZ=*9;y z$sX7rP^{`QAux!n93JgOwE_mN;ef`q9s3#^>YlHpHw?1JWR+6N;`O`i0xT|I?hDJk zwLG3q#^BpbktpO+fhxn=$`KVEjuf@xYc{_{#_!J`4X-Cv7}l{hf6Ph(l32@AEttps zvd5aTW32K{X>66)M_JHzZ?fS%gDc6l<26iib??hT@wW(pRK0uv$H>?WWs5D%;&Vya zgxfQRiE4tY_})JsX6j9v0`u!~#NFSFqow|K9tfXQ67Dm*o|?mFbXCD(;)1)*{;=7M zxpnyb=k>@$t@?0qsk*r@x@mR+CgIK2v3~;gm>aiG&}-c?g%J%$OY7TDI^2E+HR$$l zL_B9wv|)*k;=aZ|;dM6}5z~`$^VinC0HP*DAp`ESWDb|<&Hgj*%Q~ncr%+84@;xTN zrkIcVak?(0c6v_qC(#!Se{elI5a$us9q;^%9u#h+={}GQkbtaK!JHiw1uEzAarYU?gni*{D58jV`&L6e!@ha3%kR7nt zk3BDM9guU2TjSUS{3hjQAQPmuW_Hmf&rVpgZ%@Shcg2L^BY1-O^6ENV*YjLFogbGt zjs82Uf;3hce^8K~-UI%Q_=!P-%+GREBr>iaX4!vcEMt@1WPR*9l4YA_v}zMktTkdd zjIzwWg7z!0Mq+tD+)ju0OXcu;q(k-0%c+Yq7(*cN*hJbKUO)PTK5$SQ3*s1`{C+%G zBl0c!N>hdSX@6gsp@?k-mqMF&4F8BOEzI_t zQ_l#J{v?8yvWCA3h^BiLl%OgCRgEIa=y(1GHSHLj1Xcizs#MC5&m^!S=`G@cS@n49 zGx4U8ic2cFK-5oa?VFksWSN4J*T+RuJu~)k?Bti@m=JxTD(Zch6^)&*ao;JGWCw5d zv&jLUZ=Q`sj1A3IcNG>GJ&ygNW58v%MrH;zVw<#@c=8&ye%&! zT=iA<6y!o6qQMuF>*~&L)vRO7$DE*4(F<$}_#>DcVE^J`?9VPqww>-lt+NYiQ{=^R z%E-py9kR}TF)FTF`ZW=QiUBo$-|q#vD+onkWz3fU(0d5EXmbQ=D=ww!^l8fbP;e*8 zGs{2Fo5b_P;RHXW z5u(=O)%s`M5;xQLT`rzPJfmWJOTWp)khJ!O-%vYTha}2+J#dPIkCwRw3#Lr2SzgZr z`L&sQnxOVQ7GRy23B+Y5OoEf$;T-H2g`t%2=B!Q32E;!S=NViBNY8`OULB--4*1 zW%{G+Psx|@4TXXWWBia|*LK)k=ViVo?Jn59g?EfPn+WKGb+KkfV5WSwV&ogwaaeLg zGygNp#%yo(js;i@J-1w;pYkT8{Mvk}BOO;Kp?+UuP`7+{#5x%$oAoTY zZb*(IlM!W(*p#QhU+|bncV|{1nZU}3PB$QO{PGg(&~mfeq;2Sd3=io zAFn-{RxXwzVK@=)EekW$MCR}Nt^}OlW;*vE*O^a%Jmff~iyZU< zFNAz}QH+<*c(!Kk7J6U~A@G^5q`qENBKdNIgnth<=*XIL`yt->2gPSB?PZtyrVecz z^utixDEXuV`{mvwDgHC7ioo*YhYzJ7uJ(-&&S6@^JVJlQ<7W(AA*TKFpF;GR3=UAN zJg!?g#2K^8Q7pxQMC6zWHN+pzAbv_1=fo4qgon~}+(rvbi&Wc=*!c2>!J0eHoWM(h z*SWfQHAonupdz%42#LELVDR!1sAUuH_e^+yJG+c_7&^`X3g5&48 zS)-c8b$uCk_ORMFd(4j`GApsw?)eEhQyr2`?B4C*aCBbun?K}w;$@cr_<0z0gpVj> zs#e7h3m!b{p8c=H26yw;Ekh^%t_4HY98#ZbDx+?gK1!J4G}QH7z5XCvu-%|L6~#RL z9P5)C-l(C@?=QHk!#2EE^kC?y*o~_Pvr1(5(Cu#}M`1fT#M@NLKV|0#e=#Y3H|-}S zXYxoPC982oPYTJL)qjt#w8HbBG-`Z(nA&JJWzxN1kH6l2R~^_2Q!SC><+8U z&;yQY^1#U&B$~-i61BFn4Cf%CAs_sQO$y2f< zpi39(MxMZ`U>lO$7e_wg@DJvYrc|MI3QK?s@%ug6?T0DF0G&W$zgSZ_sS)X~DzGhb zbX{sC=Y}7;0o35hruYmHmO^V)7l*rBGVPwndFv=1wx^3?<_c_GUQUX__0V03K3;17 zx@@`ay@@R2a6U<>gD-kDZWRBWxwn|9It)?bi z3|jAn9C>|EWbiP}kNll4WdQY8YwKvZjaP<`(5!fezW=awLe$V!k~9em1yPgsiKA_J zh3$6dKzGQ^h%^!-G5pbf?0|ExVafe{cj?jsgKGpTNlJ0ai2{rQOg;f1b@F+}NB1il zN+;y6Xf!>YT&WQ+LE3%ydFMzWTW}Ehpqz+UjkNXkMfBm*^tAZr+c|#YA#SmNd`}|t zW)WgT-s;~`_LdEf$b*U#$yl$mY>LD2+3%R2Y5PW#VxGODB%<`9ifZ^3E`%y%+P6Ap z51-e)(}Zh<-?hCm$`1|1)C=dUXBe_M@-eei;>~eY+0zVlzmmSHb(kjcFIk9N4`;l0 zCDLM2AGh{<=^S_jR6I;k^T?mE!+11} z?Qo(%y%C@P?4Ws&)F;Ij+@f(Opni2BxWr@v4HWzu#I=>YDADT+Yc=X)XXXjnSPPs& zmPh8BB@AmVaH;GKX|~&%itwnoa7q!Yhao*|lJ$d@F&s;lz=ZU6PRRH+3>6Y2ajxK! zQJ88&p4j(Oi0=33v7H+I6v}L|a&2Qh!BW_PQ0S2+28+<(Mp@XqcGGBX08IfSkCW}= zolo()4{j^4!8YmgxSP?GtS9|Oc;bcy{=|r zj?UrW3}I9Q};;fUOBm< zQB{r-SN(WQXtk7Cb>)X`og=Kyqjsq%vPYur@X)h`5Z!Px%wLC)|LU)6As>B&AyGbq z)d<^qKnE_0c2u2vPAJjO4CmX>QjZ}GBmWs@3{5A|FvUQ&nUGIulK_GPCjrz8!7c{< zUE<}x{O&G}eQ^u5+$)&Bk-1{5N5-tH9)=Xd4QK#9ln~%vttuaxe(#{lL)eqMc)v6Z;8Z^e~>=@%m2=6prs**z$M{LY|xJB1d5#; zi8SY%Z{6xeA&{xJ)quJ_qc|RAgFU#$!sIOq!2eJhRqJ1gqd<}mwTG-;#`-URnWbfc z)lJ=~f4isPC6dQHX)w1e)7w|Q!K`se@x-R9NmHqnoALbA-m$i))2sx_IPM%C+iId#p zUE2)WDcKI0K-GJ@0}wI8wrn|6e|9mB-M7+TY-RZ+O8` z;5X&B7S5tBoqiu@&t_gHW4Qs5u;JswR6;Aot|!xW?yHhtM+ z2hM^j7aEfM0m1}Ar1>vD35_djq{ZUv%RW%E@Hu3tYb*0#<)f$rZ0%=IMuh726GOXm zY*c&F1na4TAn)D+HFLj6U9M-QkUgj-fB8Q|SN=cJ`}%>ss)K)#W?#&8r5GTz zy3^~7eK#c>7CxH5s;H`4`{Pfr$>ey;D>!_1DhFN0eV~J};qwlCINkosUxl+Sk(|!O zceaKUTl<DU9clnfJy~NuWqzqO?xe|Dq zR08Sp?N-`$r~ASBc|y0oy%7bS{;5jr;#^~Q6K+I#Jq|(hd)m-A2V>$y)XM1Gan9tD7}0Ru?=ieF*SCv5^gri| z@1&mT=2X5){9J4IxnazCIS+dwQe<96Qe`{MB zqkfL`wF+;Ngy{8{r?UTf23`p0FWGx`tcc0L)60UB{Jw-Fa~JDh|hb zTN3}}t1x9g<%%KPX42Exrj&&6XG4Bb@Q=CCgY5|s`HCW|I+?FO=~TI;GUF!(b3 zmv0poiP1s~1lF}(tLU2dfLg@_19q|$rXST<6%6fRADAa{y=?V#7?RUsO@~a3 zZ;hk#>>{XUieYXia{OPujpk+Mv|j39b0RPBDaZwoAr}yRYcr_0d68bcz;`*UAq|eI zFt<`7_zVJ9c6l2N#R!NfB6#WENJjVxFj*rZuV!Co@e&^YL6VIMy;v(xP_@IZBvE`x0@$W)dw5S^X@_IUaGd|;;|(!IOQ(de z!(u;Z1Nf9JCUVDIY>if7RB}dz#a%Ea8#fnTem88VF>|CadZOXJ&)ol1CWYyRozPZg zqm;Q;{FlF3z~78i#W9<>ps7@G?*|=v9~?&nj1z)V^RaV5LtG}B08Mk{UR^k%`SGAA z2&~09D{feF^)DfQl)9T_Il1@0d<5((kMG(Ty0q-3+3%HZo)qjCjW2!mXTs*!n&Hs( z&9>Y}G}Q3Q0PD1vj_0ofTOS98IbCs6mH93Mi{VAm*#Fo2p9oI}X~mxzz)FlXsJ1u& z0Fmm5M`88@^fg=cz{0lySbYH@83D@xQ4?_pt$Y`N$jr$uvCJoc(^dJ6t zm#vvg>Nh}dR#4WgtQ#Og9T~GLObi&wdav)2BmjtHZzZ zTL7ZPcDZfj^#AnRV|8Q9`E3CAMoqZaxvBtx$l0!HMdENM2=~Dg+M5um4D(`b+@8aPJpWT-^eCWA!5| zGe7~lb!KjYh%NvH6*xnx^XZw>QlMTuzyK1^ zF#hj|n|T{h6yn;eMw0~S+a$Z-d(Q>j#XtTp_TD|NsUvLwo-yT8>$=u; zU2D6pwXStt>-Wr@bCMIV_5FVD@B8nIOwO6-naeZJJhz#1GCtALp+ei={PO(FukWK= z|G0k*S2sPfeV4?Qd7*Ft3ZI%_S(x}5dL!Yf;?kNDG^PKUDetX3iqdy|^6sT`2hrF6 z*>vgDy$fhwZtBxttUijgUw-rA(7mybY=6t#ls~-qssSBrS-Jh|gfr-=WhXwAFMoja z-v_SQE6hZ%HU2JFaYl>oy;m!lbMO>esvO)H(7g^t?h*gvZ=F9pvik1uGOw;6pg{g{1xs|ZcFACs~(xgD)p_|Ae! zi~1kgeqdrup~%5U`@%nd^Z6YE=&xIs7Vp^fE0h#+uWQK(6`KEr?4I9OSCF;q5BD~I zRE7TDa=0e{heWi!YzgK2c-SM`pZ5=WTHd=~pnv~q>d&I2TJ)+&Npi_E%_YpyN+@ z*LI;Vo~ggLFeDy5vtwK8VS6lk^_{6#U(KC`oJT(D?0!ED6&m-vd48+`IigxWkAD5j zN4CHB#=gzwu@g~~pK9IWf9yg(41DpQMQ{HeT|QL#kA|naQB!Sr%=pa@P`Ujv{g+R@ zhrU~|&(gi-b@ar`|J0t+Kb! z@BXA%Ge0L0T?;T2Mn(M|CGYXIs#j*CuQpGV3Wo$}VBH_LO;_JW?@wEF^LqB9<^Ov7 z@1NRGv>6#5U-qY!r~Hv?$I1@h*av9RQ~L3@-oA+*wEI1gpL_wG_Zd?eureCT3q6_kbtwPF6U#o0T8CoaIB(dR^${xi?~{%DA3KX4 z-}9H$3x8aMc0O4#bfq)}{ciR1_fM;~p#1P_Qqg0IN78Tq%P)2r_3=n~W#0D2FFrww zzuvhnXTw!=`Nb!c)}wY*yZW8v;MV`3-Q%7MZvNd(wDaWIKSsqLL(hFEN|?OC=aKE- zSlj=?gn=Q%Uw-*s+*4-9bSvNY$;w}RfKcS$ zp8IU>pTTflj>>raPmgT>{(lOtPpn^sUMxygeU$eW`me>R-SWi?=-f2LeKctSO6l0~ z!kF31(eiITuS$F^4eg6rvpHqiZuHd`vpx(t|Jftk{}HWP#p@qG7=!ML)S{9~8Ty~PO5@jBJvtr#%&>Iw z5p*y+{+<8%DEN`>|MqrhTIyS?(3{`1QI)rv(CUzoC2=(;k-X{Wc_x1fty*eY^{<#8 zQ0%_jkG-DHLqAWulYVewJh~A&bLRWs`#-Y%15eGr^Gx43H1zU6%K~=3k1~&b9x?Vv zHF`TfJnUNdpV5m8?{0gw?KB#=SNM96VhY+{M?H7#KQANQe`7!Suy^ev+kf(dBlVy7 zYtXbePAofBw+a;$&Mkeso}O z;*_pZQQRZjpMCPp<}Z6Eqs+d?Ycv);`tC1h_Pm|^CHhO?l%b3de~Uy{wj8|rlob7{ z$9GT0o+7m3)rDQ6Yjx<>0eSCY`|(G%@9(&q{?QMWXtjB4`xk5KQ0DoyrhiZU3-W82 zvGml_Ip`Nh-@Se|p%0zgqrCl|s}IHOo;0E3SQMJ{L&qk;+DFG%FV;PO$L~fsDwz6? zyku5BT9df8;+51j=wO8YV&9>Is4GHJweENCB3EeM_Bxvo4O6WVtrs_=cRF9)yXNUR zkM=KL-s_zkzYA^pd8nj5@?S`|P!oBvayt6_jh`*IeoCRLo+ZMZ_PIbaj@Sb+h6zc+_w`=-=Lu1uGw|xpHtAr z8@DRQy)X+szVB$&klz6M-9LWW`c$0^E#rkxT>s>I=*-%ugL-0DqOiG!_rvd2J+gh? zga7O2|J=fV(fGC12U=f9d^vg5%PFt@auM~<*T3kWykT$jtzW&jbma>-120X8Dv#L8J>;Kum;qmV>sbl5?yqaABtqNt| z;@A5>q@Mj?&CJbbI?Rvc!+*vvUb66T;kxPb{>ZDp6S(c;%6%t(we*#`w+}k^k6l^T z9{7lm3FXV5D>$<$chT>^Is0|)jbAMJL+S_1{7PTgbA0SA@ved1q{?Z_wr_l?<&NsF z3z~DwtUljPJ#5YCu4sO|tbbzdYutYhM&a z?fciSeiQV>o{y5Ap1kUw<9<ST{3X>@dU*V8c0Sk+-~ao6c5RP+^_g1-{_x>{+cL&Kgw`KZ?H{=J@A(BS z3gPeFd0Qo$VAj5~|I@>t{W>eST$5&f@ZSEgkVDhoJ3LPo_2PkXzDLeZ_%I^1ST_7g z#FV6zL#1uc?z@wA&yafY@250_uRRt}*RDM`f5Rg5bg_4LF|YsiSO2Pxx{#xrG9|a_ z$N10V`S|ep0RaL*aByg7SeQ^46%`#F6B8E~AD@_*oGccnrDbGfX6EGN=1L@mg+)a& zS#fb`X<1oiWmQ#mHAQJOI$dq8UO#S}!C*3tAK%o}+&ppOq)C$}Po4VsP_P586|Ni$UPo6sU(MKPDeD>_QbDw64+KrH$!Z0j0 zLQ;pBVgYiqn3goS7YLcDz%U^zg^bA1?Ft2qIMDaB$OvdrbJN4p!A`D#5ecSGHwL%G z(j=hirP~lvY`>l{v2{FREkFzh#eAMXjCcaL7l>)@5}X+B#JC*5#281)@C3{(5|YG- z;E<;-U5C`AVWbSA+h@ws?J#XVq(8>Mkbdd@n6~LQ=spFElyK-LZ6ap2k!MVZZkrh! z?gKb8Oc5wXi0OOyqr_ON#P|wHJPbHo00B&hfFyGqn!qWqe!1@AIOcPn7-8< zdVFXN=QI7_3!%Y>>5plHp@rw}mw_>K@t8h=DZ{{x1;7Z_9r?gLVLMzGFq4Y8xB%fY z!}OJ&3{sbc4KsBYFgW~Kz|I{>6dVp-(beiBQb=$W!xi#?hXhy1L;KLMkXCc(536J#iBQb=%x{s0ekrI6qX`9QH-s>EavK8%I%L9{OV z&?XJ}@F5Ec@?yvX9Pp5QXwL)n(&3s8SGpkN;hGNR_{8h^@Bt4A@=y=(e85BUVb~uD z%?cq!GSdKgxQfM*P+bTq5~>LyL0$}bfCC;bgNR9Oo z>O;Z-moEw2pgttX^B|9-ZnzRdOr#0=gDW2kLLd(d0pi1zABclIUmV;44*J8vVIbrI z&W9^M5C?NS5s!}10fG^s1FIT8bCyzkwh$!M8t)R z=ZCKdxJC$xKNmn=0C_=#kY*g7A`)*1`D7rFCxd}JnH?cqBj6eVaL7Y_KGcUK>!I+x?{1F3UY>^b(14vTFb;FehSDthvgGLS*a8OSOi8e@FNFRf4+*Z&Kjev42%&!=^e=?|ArA?z&_Co!*FyLbLjOYOA8sMR z75axf;GiD#FNFSu&_CoM!4>+4JTrcJ1;NA>I93qARY2koBHZkXNPuXk2YATCm3A`N zDiNU-&`uQcaD^nG9g~j$3o0Z5<4ge$c|KX?;0}W5Wf2M@D#nLATp5b%(PdXNAQ zc{&at1_t`}70^E2m#_@3&<7TOU#teuH{c=9gFL_i=Zo13efwf}27UWxW0wf^Ao)Ps zzR(BYg@6|VUI=)|Lp?}eHq6Lo6tg-n%v;goiTOq*}@Y#URCPRfQ z;2|%jeJfm{eJGy_c}Q@D_8~8Z@{r&PcqpF=c}Q>td^X>`81sCn(V+kT^P?Z$ql^6i zpP%^nfU^6c02GWu!DB?BSd@g4Q3lFEd8iO6P$jBD8dQh$$b>AY1+^kO>O|8~51N7I zp@nEET87r5^=K2?ingO&XfN7_4x=OJ7&?y5qKoJevkZ$U(Mgd-^$;?-@`w|Kf*uGKgB=IKgYkszs|qUf57+k3-k;2i|~u~i}g$J%ksmZpWSbk-+aFneyjb~`fc#r?6=)-r{6BWy?%%NPWYYlJMDMI@0{OxzpH+? z{O^1uPHP6tFp9U%8k3 zz@)&mz>L7Wz`{U9pe9fs*cfOEv<9{W+5)Es&Ip_pxF~RG;EKSFfm;K21nvqv9{73S z<-qHKHv(@5-U+-P7!{NploC`FR320nqz{@H)Ed+l)EVRq8Vs5fG%sj=(4wGaL92t- z2W<%29JC{7XVAW&LqSJ^P6vG+bS3C&&^JLhgKh=+21f)(2PXul1?L3k2iFAG1&hj4zY!_hV+EELS}_53|ShoGGuMYrjWfM`$7(c91J-gayjHm$n}t$A$LOVh44as zL&HO(LX$&NLi0lNL#sk-LQSF8P)F$O(D|W@LYIUt3tb+%HFRg_{?J3A$3stqo(er1 zdM@;Q=#|i`p|?Wshdu}m42um*3QGyg3M&j#gjI$$giQ=<37Z-=J#0qUVA$NSC1ESV z)`o2i+Y`1w>_FJjurp!j!p?_%6Lur)ZdgEgaCm5VY_?hst;h%?J3%?P5FZ_PEuP|JgEKCt*2=j#H z!YW~naJ zUPNeQQe<*uT4YXSUSv^ZX{0K$F47Wdi<}YZikuxeH*!Vf`pDgpdn5Nn9*8^~c_i{w zC zOnFRIj4s9$V~ugd^u#!024hyotdH3ovm<7A%)XdIF-K!g$DED%Jm!4NrI;%**JHkk zxf63YCNMTUHZ3+QRvxR0of_K~J1cfh?DE*Pu^VGI$8L+=8+#!3aO{cLldq_CaibC`yzhQiw`Ll_HHuFKQH7MV+Dp36aa-cP#QljU5-%m*O1zsGn3Rx|l9ZODNGeaNNm3;>Bw3QC zCbcCwl4c|gCe2Bjo3tcpY0}!H4N04mwk2&(+MBdL=~&XIN#~O;CEZTCo5UL%FgAE> z)Y#au8DsOt7LKhNs~_7q)-<+f?2NHX$F3N=cI=k1yT%?Idt~gfvB$@r9D8Bx<*_%$ z-Whv;EH61UIXpQbxjb2)+>kszxifh%c~0`8dJ9%I7!Q`{amy)k0 zUr)Z3{2)0(oFGmTXNhyf`C_@aPTVLSFSd!>#CEYmJWD)Ryi~kJyi>eKd_a6ud`x^< zd{z97_=fnN_<=YyB`QUcqDs-Hm{M9&Ca2g^Tq$!?7Nx9BS)Z~cWp~Qnlzk~jQjVv5 znsOoKO3JmAn<=+b?xpyq2BwCmCZwjMW~An&7NwS_Hl&(T+fqAIU8#eq^HLY3E=gUP zx+!&Y>ekf#sfSX}r(R0ElKM^RomAhnz_j4BthAi8!ZdlBB2AYzKFyNWlGc_sJ#Asy zinO(9+tYTX?MXYFb}a2g+PSm~X_wP(q}@upo934uoF1DVm!6Yem99$Hq&KG9(mT_q zr_WBGn?5gnQToR8E$O?{_oW|9KbC$n{Z#s=>7S=xNWYkVJN-`jy>wniKt@zXTt-$# zQASmUCZi$4l;O&lpRqh+b;kOPO&Oarc4q9!IFxZJ<8;R78J9BdWZcd0%M8v8&y3E@ z%FM}BWL9R@WHw}4GFvk3nT||n=A6v=nF}*lWNysdoVhD=Z|2d=6PYJ7&t+c7ypef3 zGd3$OOOaKY)tF_;vS!(`rf2nJxw2+u&C6PwwJ~c`*7mI3S$ne%WF5&mmUSlUa@LJ3 z-|T?w(Cnz}gzV((lgSp3Y zPvu_7y_|bJ_h#)yrMjLUR7RAUPGQK&zk4R>&bKG&B+>7)Tl4MtGxC?_Z_3}2zaxKd{=WQU`N#9m z=3mU`75Eke6oeMU6(kpA6yz7k3rY*h3u+1`7ECU17R)Y~SFosHb;0_AO$A#EwiWCt z*k5p@;Ap|Ag0lrz3$7RV6$Tat7e*9j73LJG3U!4Ig^h*d3nvyj3kM757S1nRSh%fl zXW`z$qlKpn&lH|7yijHC%VK3ovK*O0)*_oKYm-fv zIc0;gS+Y5@<+2U3&9d#XU9!Eh{jw9Xld{vYGqP*4Z)CS*cVz+c40)csP_B@x!;7Pe>$TLZuk5uq&L3 z8H!noIf}W8`HJO=6^adtjfyRb{fa}1!-}JdbBYU!tBM;6zmkBGh?2CDq7p?(X-Q>C zV~MH6Qqoe=S~9()r({{lijrL=yGss~94t9ia;D^R$+eO@CA?DK(ty&0(wb6TslL=y zI<>U5bg*<@>GIO8r8`P@mYyg*S9-DZX6dccJEeYQ!DT6B@-ju4s;sW8q0CY?vCLLx zFLRd7D4SchqHIIi_OktD$I4EWoi009_G#ILvP)%`%N~^R%9F~|$}`Gy$_vY@%2nl! z<&(>&m(MPrTfVS-S^3KHwdEVjx0G)$-(9}9{9yUv@>As(%CD4PFTYWKuiUpHsv^1~ zxgx6~ucD};p<;Z+w zu6$6*Q-&%dl(EWWWsXvz)F@3#i*mBkuAHu%pe`+dP((`>I2n>st;Emtv*(L zqWaV7^VOHCuU6lveo!4;6IxSRQ(040qpE4CvDMgXI%{Uv%&l2gv$AGa&Ec9;HD_xs z)Lg5%UUReNeoX?ELS;~SR1sA}X($~vm1?CrsUB(xwUXLO?V$Eh`>8|JdFmo{ow`Ha zr2s=A@NrMj!~QwONy z)XC~Jb&k4Ftx;RlR`q0ctJLyXG~8jZf~b z&U?PgZ{zess(U*>Uzj{Q<>{!nQ-7WO*{gm>rfxcTGT_`(CAm2-Pm8_KCw+|9va-A- z?9OLj7e*Ha|K>qwro1yEdQ#VtaqqpbeZ}4O(1MWjZT2dvG+i;@a^oMEuJ>m(P8gL9gc3hxZUn*>vurW{t>k>7OJ%7q5eFSm4mV*S*XJa zJ!EI0TD?Ih?sGbN2cQ|3vu(P4U_@;iXzQ|f_u2cQ84kABW*-{pZ!K(d_Rh5T4*=33 zR#u6p_2yT%_p}b7S7z}G!dF!;c zc6$f6ufYzSy!)R$V0R7l1DgulrcDDBeCJm?T($vcpRLjF8k%#X@qnW%r=eQ zT(4`gX__b5dO)N(#hEeT3WWS28Cwtcktg<|-QL>YJ=?}1z|rd#$Nv}oRC)C?(Avp1 z*NQC-uEM~wp}2(UL?R}9x6!#Anh(?Mvu!k;_Fir-+tJh4?Pzlh*c>y(HDV5Fj+wYf z=Rj9BO;L`xLM$aDF|`r58+xV&oCB@hoZ&KBNR$<9QjUj5TwwsEk||2O$OKSH@kse{ z58Bb|81Uq<>GQ_B?4S&;w$=e`1s_tMVcH|)mdwbsHjmI_0$ujDeo!HrGnh;~0%u1y z5vIP@_8jEwcXYBs%CLc95deDH>CwC8fa%26?&t)i2M^K)Mi!06w>kTWJaPH)@8}*7JA3I|Uw7+3hqJ$j&Y8?Qqs3y->YM0%uX8}$Iw-5db>0gl0yl)?T3|9$U4Uncl>V~9*FxjH99!d#k)g(Yi>Gz&Sw33YTkXCdOvz`lv0FnGhhR+r1(E^Za~4t95o z`&#>3d+d0kd%*KL+r%9(jKO|8>UK=a&KS%=;FMj?UUCaov?9Yr{@UR?h~z~K&;`2(rLq_cim)MYV&~?8gX8aTbPo2nVt*ngQ3wau)8izA zrf1KCfw4fC0@Sw+_Rr+h0LFSU5Cn1PY|vOry|ZN;!ZFt)Z_w6qvQ`&hA=ZJ~?Hn-quvxLI%Q*-k4A#1?R$$pQJ6MM{5MMh{eIkGr8Pc*L zEUF`A6kv@0i?)6HKmh4iZ+!cp46fOb1`d4tpdHc}NNXTH18Mlsw-26xB!JWh=|xBb zAAI}ZJxFgrS_A1BNV=vbn+xcH-~q%1p*{#91{igKAk2mZf@^ms4>t*Vqd1> z|MCN3SY$K6EH>BM8ugY&RkNlZsjQQ1(6Ffqsa0BAovztNikZ~obed+HMrAM%1e34V zskAyM+Gw^kH|dRaHoe6*vB?BCbh|i4HR#nyZ88~jD&uIhwYC;~U9-t#gZ`myjmg-o zo78MGnwo8TqrO?MGUzQjEz;^VCasR4Osg{k$+}4zolc8!`Z}Ej=rn3=4Z0RvbBkF= z>RYU4v#F^WO6ZK7+TrycKGVK+raB#> z=Ya+Qdy(E?(AB97HnqN)nV4Er6ZGUkFch+RO}(l~1)Vin+@KLP>UERc_xeT;wH_eD z95!!+NzrL>kvctxB%>9knVYdqP#LT`H;7@UJ8N#%n~ZRyGPaBWT5Kkml4cz+LI(t? znmvu+PN8pZ9n62WE1FWmXO%^x*Sp8cAl!__2!^9ZsH@jvj@lTOV&=j`);4>PtRA=_ zAe(v*zR{%B!(=wO0WgbdPL|OsBCgz4EIKnxPqWEWl8nYpH4U>G^%GzMRfZ8XSet9h z7!W8^GouEKIML#+Gg;gKok`=qw}9Fi-DI**nv0DL8!#8$UAv*Qnq&E3sJ!w$%t0%z{hQ>g$lD9;_9ZOCz-0 z+@w=AvfT9;GKRTsJ?G>}ZG_DSnH~bG*Vok}A}XWC>2AVpF1d8Nf#xQCBUs@HI%GA{ z)(mC|OUMXe8^vF%vB7AXXk@!%vyD27MFo1r+H6CesY%~l-{>x2(bpMO&DJK}LvUD{ ztVTCCuzG{E)B?jm^=QNIg*RK(V5Zzo!3#$KG{qx}c|@1F!$9oxYH=)pl^C*3rUtk- z>j4FfGZMAsxEBlKUW^cUxfKzF#@eVeVvV&~EjrWyk`4%9E-q&CZU>~f-ehSus$ljk zz<6vpv>1v9;0>`@HT5>U^+bz0wiS&!&>3WDu@LubHLBDGED|;i$W=2mXwo$3@T{;t z_AzLQ%O0p-Z_)w-z`o*oPz}tJ(a>VU1A>9bY=&k$AUWZT+hXBN6Od+ZGC`~WVP~D& zQDI>tCX?gs;R`yOsK@Sn0?k{i!D<1QJ^|0@W4b02c8%DzgF&z~tD0e=m_~`t>y2hB zt;J0`tyKe~V3|!Y?p%|~2nuTiPk@D)HE2XDY?kp>Rg(^JLypI2#zJk7aUB9Jn1IcUh5lpp*r@(Y({{y&!bFfCnArzi{ z0GO=Jm@sHTWdZKdbBt}Crv=ZQm#JV#=ECSfjslY@842W!z{`q2*91sg3NS@Dw&Jpv zcwCLjhPhMMaPJrO$~ky>!jNa1K2>!Az=zBip%ImYqL=|5qoOFO;)^Y zL~SgXac=E%N$N5H`cYpy#5g36Oh4h{wRvqa2taXM)>q%qdISpg|!DO+c za@!{G=Q_++LOPazQpdtt447yvskGW^Or|EVF3_dL8=;lyZ90v)!P6>HBk&%us23O^YG}RF+kB;tPNHp>%%=#$jTJy z*qx&XNUIv@f-6omm@d+wt!4XRh$Vr94to!F6l|kRB*HF8iJ@RZ5H!-7g4AhEjcg#u zkjojAw;lF`Z7|1;*oS#TVJ!fQUufAKdRRCx&d|J;n>w1Kc+R}w5Q~ij+O!PJy>4o< zdBsO6EtW7kqT#GqIQ$p^C7}?<>|=lKS>4pt7TFp-BKpwgIm2>;EEXFClFiViM^fE$ z0KL$1h3$W%uF(WO)PRFh&jtYP#=WH(2Zk!}c?{(k!uVRUPS9#(>`dUik=7_iZ~@?% zNu~zJ01!v?@R=r!G}r5iV<*g_3$mWsE83#v@27%PPYBF7K93o8`%%m+dJV<42!kPV ztOg%hQf#BQPM9!FORq0*2K+9)yulf>%3_%a%0mJ@#JFIr&{#w1kQTyIu;$I94F~X8 z*MN5C~yG`lF&V5)2N8sa~|Y8$My(180iZ~%lSZ0H7~wHlko+UBx4~&r8q+uDtd1ir6LiiCj zhc*3;Dp=&RAt7fvF+Z3QB4(f*0VVq*AdoF1T4hSIt9Mu>(Cc}xz}agHkBjiyonwV~ z^y<$o58UiS$}tK*BBXOKOT7a<5IRf@#eVB+8B+LYY zOs@{;bqrBTO%vmQEHEOhsvuafZZZ-C9jV#eB1$hZfWf2}I&OvF6wE#b(pZn-^3D>2 z>5jM`f^)A3Srol;^v;fGh4EE@g=muCSm%hD&+K*7HCeHn1wxAe>=jnRS`9?pBlpaf zs|l12bCIYa(NKu5LClTBH<9fkH+RTZViR5sunQ{2M>6{#j5xA1b4F%Rjq0k=3ZQxw z#5*eNm~nyTi6(lwEW|m{q=M@=8S(xC*#=PSVB|VxHNtKlbLYrx8;?vXotf55vgRPP zl4U+LghvErn3-oh(5M}I*2`<^bvvinlsyf#F82rC!yb z8(j*A1@+cuEf9o}Y*lWRX1wBWrngS4c$LS5_7frM$tk)R~ znD+5jY)mnP)0CMkRk;wXV2pR8W9X)4SVcAHIi`+)ARK6B*7IJW1!)GxCUleb=L&UQE)KkEoN{)Lc)@H!P1}vZji;=39W}F5hYU8NQUK;9NWNSe3fsZ;+8Z9kqh!cR2Ubqg!(+sB&=!3r2 ztq~Yw)2JCWA~~#Do@FDq+$ic`)JC1brl%tTGk(*91M_GLYit;9nMg=e592%POd*qjxNr$9#-i z!Q*kWm+_IDVUL1f+L?Vw9MR%X2WtmyBWaVtwCa`{qCK8IXx;I$Ran>Sn!rs1@0;o2 zXkn;sgU)yxq_xtvLgyI{2S)6DJ>G5fz#g{a?J-e|=O+8btOgSpc5ApQwtTe1VpSH@ z#jUtH@ZfmcRA=M*E98#RNAZ^qN*N>T83GN~8$F`U$O+M28jD3@L@ijMfrtQ3K(W7q zrm<`y4jKwHsoX}BL800vRUKYudt|rHgrAU1B8vzxqdhf1vNHXIbnrS2eXfv zucfWD5g3kbC+0jC*{t@+1;{#%_4Mr1M0)pV-qql9Aa3_s_j>k_*r=H?DA+H#<4d!pf!R)I#t+|&c%ewv zgq~r06`_5D70eI@lSc={9%|Gqz@lryYd!$LWNU0ARw`s;20t^VO|2P=17b|mNXx-m zEcfEhOQ48Ca>{z4jGnc@2o$FVBaE#2FmXPph81sA6W_+nHrEDt>=;|p$PmLVJ))>Z zW0~M_ZWHUt%EV0;e$-CC1VA&7cD1aCai05okc^PA%qB4$i9oe1OsAHvm7Azv_7G`*e#Ofer z23j4!Ae)GMGcmh`F5q61jSA}V%ZGB@JZE&cVf=&v<0i=I^$v;CxH>ix$BE&%D6Y=2 zGWuiqCfx*6!%yIuQF!CY<2D#NgyCL@(SbMKc;l8B6_a>j*`0XKGkOlz+w*v#IImx@ z(StYop#}*t#+TimWH-eznBBm{_YWo7_7E}jym{h{_qx}+Q9wP`p4F}j11*oV}^dn8TRz~fDGdcTj0{*k!PyNCr=nR<<$ z-uTg2@rLM#5?&USiyGy0xk$JDXQzgXWHbTqmDCzc6TNWmJyDf6*n%yO%Bay9JPaD4 zakK}eSLqKgfR_(e95z|Z;M>_!-Wo<5>W84LMw7Z(r6=(KgLYdDj@NRd@F-zT^B6YT z*jBJ`&NBeF$;QjC5ndaU$h=R&rGiAzxHxB<0>>*4T?@M3*`QZd?6QROG6#0vRFAlW<7+i?o+1;E!hPAn z^Lmz>Lk|Ov*a3TpUEzS-fx6!FEY^)=c8fe4c7!7CX5FaJh>P$FbHOQ%+}U>rZ6n}b z4Uy*ulb9Vd?~Ousb>@DvjAQfMFuXLzTlK11EC_WfO^dgFf^1pnSk}wJdQ<+89e>a` zy%|4lrk^N!KQ3?+HX6u0g(UkSc-_Q2>>kNS20VgMoG0VVJ;w|ibQ5$2cZ9>>-Mbf_ zMm>Q77s%lq4RMFp^qW-PKyMMlKPVutp^+Q1x5e(J(yxBN;DN=k_E9zHwVr4S8zNGL zvnj}VqRH-usf_Ga4YSrH!o@ze(9d!+z-I< zqYhJ3i}(8&w3Qn5aNFzI_Cp?dH#b@Fn%-iu(s3=_m;3E?F9`Fv#SJrifY)R+N2q#(Vc$TEQDGv}2$BG#O@ z`ZZq7(hOs*f|s(9cjZh?bt)tKo~Fv^835Tb!mp4Mdy9=Ed3lv?hr1f^K(#uE$_(_w z!%?FlJ4SdQ#6~c91Grc#d3Bq9TZ{ADk$L0BVx<|#w(b2YJ&_X2h!GBMo?BFoFnxjEKR*Nj|=>fVsU_w~3NTQ1&x8Au`$P=F7-$jk!Doj@iWSSu3_N6Up9*2;x%MHdUltSA;f5uy+tUaJsZi!KrVcWsF< zA*NJVyslKJk0}#&t}7GHj42ncT30UI6;mO6e_e&}vzSWZt#y^cxLBp|t6`-uH?~Sx zy}n8~HMUyVy}nwwFt$dxZheg~J%keOSWgL$$Et*1tXB!Yi&YExFR6thkw#eXl15l7 z(h4WPq!l_vI^q16bi(_7wL;U%|suw;HrWXdS*9-qAdYmvI zL^dY+dD)os5c!yr=jCJGLd9dg8ZI8Q5XnSOu8@h|LUPgY3b|-ac(DlJq8;H1(UBDj z(I??0qOVqzh{A-WqVyL_MPZ$qQRhY(WDp4MZ>}h(a&G35Os%Ciq5}SDf&jJ z6n+1qQY46|62+~o66Hozi4M1O^!Qc9={b>#^x9R5bW3DOdh4o^bZ2B~ z`rK8e>5C)F(pRr4OaFOfd3tk*EW7)8SvHI@d*1W%>?cBsv)>vn&i)E1vR4(zB!^bX zB(D|7CGWm1m;A1vSn}E1#gbbE3W?utg+x?XBFSG^BFQK$m6Yx-l}srtlel)5N#+-p zOIGeKmuxMpknG=GA^Ck_rR3`FO3CR+rR2&grQ}{=mE?z2RT7c3T9WxrwWL~FBbo3{ zjbvC#Nj_dlNmfc#k~iK_N!CTECHvk{OHuO8Ofz( z!*c03R4i>DE|yM53hBaOg>(%nk-jxtB0YdgrN198m41cFq<4nPq(QuLX~z6=X$7xB zYMfsooyMz_y5?6(pXVv1JLW5;`*~H;lk=;j=XurA@8(xa`93w$*abDxOdm=rTR=(c zd{okj3sllxAGLJO0=4u7AB}X=0*&-tAFcH40=RPbe#R{Q4S&4!<7dW8|G{sP-hZY^8trG1<}9*EResIV z#}+k9U4B;Si;Jw%jeZlPdlyZR9`&0j{hvh>rC0qXN$)S3B=z@iktQr|krwzrCaqfh zm~^WDWNG)}$tLyiECFh+KK$dAagGA;rq{<;BXHP=(UI zT%mkAv_!dXd5Lm+XsL4F@>1o8p=HWXmX|5JL&}wRmzOI;!zz^FODmK~VU^0V7b=zG z!j#InFDR8u!m5-TU#L><2&-0p^g^|AZAguB{|hzBw-BZLYM4?+hpLp#%T>xJ!qm#I zmaCPuaWZPk1{w8SoSa&-K~C+AE2d6tD5gG-Q&878D5&q_N+|xzB~)B|DV6tfDOD0* zM%BMuMvVzCrzX8zPEC)mpq_uZg4z^cN$q;Ml4?gvYT>YwDoUuL>NZwU6B4Sa7dBQ? zZzj}GA8)Lot|U;@-Hj9#lc=JKUQtow64lhySJc#uL=Cm{6%F-TqLzAVSWE3o)KSM@ z(NWhEYpLK(wNzYE9Yt-bqxK}#Q-9l3Pu)w>Q_-*LDaqJz)VNp2QPajYP_ticpjM1E zP+ML#P#=zMr0%`iNQEaGsl?4jN}p_^`Zt@X6Uk=k!e%q|ee!rpAa0^0uQgFxv4xuW znuY2XH&ZKKYo<1dt<)Egbjh>eH0R zsHR0I<)R%8`P@x%} zRMWOj>g9|s>fp96>YEG)mGE;1rObSin)34}slR1Srvl!bP9^kUxtu@0vwT&mW@R*fm5Q%%4r2*)^NGpZ_yT`1a4J zjDk5->DzOtu7ani8E-#DeUdSkTCjO8HJtx6HEYY$RDRMt>g>z&sFvhm%DQQoiWkqP zd+6&a|j1|{?5OXMLyIl66KmEC>aOj<>JM^4)K9+|qJAkmRDJr*Q1!L!NcEzeSasfxSaoAgqFRuX ztUi~MqVCGcRKJm%r*8U1p8EM*i8|sJ67|1w^3^kbk*_YzEl^M0S)k4jkZF}mWZL=w zxpu-5xz-U-to_-NV(t6@g?8Bzh4$5e6773SO0>TVDAk@_QmXwjpiKMKk}|E2pj<0_ zwp^PgsL&pYl+~YJC9A&@DX;%wmApPJN@kF+mKkcJ|U)hh@#bo<5HRV zlXqn1OH#S{<~wq8P*JfteNVAjR-`c3?@^c^D=IPf?I|%YEh;s?vZvJiUQwC(*q$=; z7e(dff9@$ahsi3;>F-vUm9k2+@!d-ElQN}w{<})^%d#r-JMUJR56P;{XWy+hUy;?A z=Pax-e=noVg1wYERjx7@?Nyl@=_I zYt7&8tu^zD>&$WQ)tPIG>&=bt)tjFv)|&_4)0>wSk27z4Z=CsHafA7z_ZrMMiw)-F zeFk%pqS0*J*JySqjOIoAjOO(Ullkp^Ci5YM*?e=c+5E9$ytye{Hfc<_tP5g@u6ZHy zuGAIsE?IbS7tBG|7h$sQlILaJSHk4oKfECCP6(0>yfrKvs0)z~3n)Y1j&|py6(30oMhSmg?5AA)f zeCYQ<6+@pqS26V0pvs}|o~s-R4pt6{mnnxzgR6!N%c_Q652_w&ey)1xtq9puoKsGE zI~~1t=0cA88t8JilY_1BaS4t#l%4Gw=pSqw5aW|~{=w^@uatM;BnvZyAkf z8yFe|@3XbLS|47e&t`9XDDX#%x62fYqH^{Go*zN~SnYmS>mv@V`agai(&0mB86u~C zdBM7-w|9^)n!a4;-~f9}5?d0^a{%q(!2esQfZJqdxq9RgI_OODP({fOo z)85hHnhZVV7ft0J2Z#Fdt7lHm&ISxVSc*9qi6+*D&yKSV{WjTZmZ6mF4yg^4ve()BGkd=i_mScn zaCCRuI-QQ*PWSm@n60*+)_zx4Yd1diOFlxL2;M=?-ot1adI(-_>vs4GcJX z?4$8a12gTTWS=>9F`3Pe{QocLX(#RDp>y+i%%lE4-1;Vq#l|V*Aqqy=Z*RxP8F9;x z?C<>%`vW7l3w%fXdywq?&pP6B|AF^EkE(QcwYs|KZ%~%*uKa3z(9^U*d=w%Pndyy= zUgKbonuaj_cXz>T&7NillS&TlGU_IZIlEb6_FX-4V3DTDK%Z+w4t!!>vk~K+DUv4} zRV1q&fhKv*Z|yYR*m^Id`=?EfdQHB@pd&w~)a2tg=JCdJKK=erzq2jBd3K+@LR>6C zT3tiFzPH0!A?DbQK6?*Nwr&SLXpvI^id*t^TAe&!qt@VGi02o{$~^bwGWWeqQ9|zX z@kgzz(;LO~eGq-o8MR_WA0kJT2%rDPD3W7l4$=+v+1my{J)lpOvBhi9?qPaCX*P3q z6Um-D6!LmU3Z_7uRw}07F%y&LKj2;Q5qZvOPr_Xe*e01Ae}Mus{2Za$X|mP(1P@Gxme=WHWyjI8Qt1jMH0wJ?CGl)Ecd>_96Ut4tO7%#=`qRb-WSN z+gzpew`~X*fjx(#w+~w?e5@mOG1-}I{oUjaj}b%hs69YhJjsYaSY0C;FQdQN#>tsn z7>=kU6FZaqBfYaGczJPM)=)*xfoP)jX_|V4DK>Daqx4sj(GqFE1!A1gR zw#Su?+s?0sk}i(F@h;raZy#~=)riLIv(QiB%_ft9Zp1S#98u86=y75T94?F;VIO!E zJ2@h()6P9F7SAU6vbFW}c?pyU(>(|Xqe)C2b&{>#hm`{D85vEDlJ5~8-iS03_poke zc3`=0w|BG-b`KyT@5t2yKE;OP*O5K^-#%o(CL3@GAKEU3AQR}vkt_+IER>sZ(vIxdx!y3f`X+7(1bq!ei+VN3py)gUCZNGh{ zbGp5qF3Y}C=Cb!YTDy(pWIR%$jhx-5#T7_F_a?q&L>~Hgz!IXhdo-cFtvz;@Qu0iM zeV>Q}(ivg+ItwBX#=OBTqg(77oJJcoQj$5^fj;vPANk3SX*>*J21s#-(+#0H;u<7} z9wNG(hsl}f$L1Ft0ipwG@ZZfAjmnIVvuV&HSKsv3Hg*P_{a#p=2DNqJ^Z0r@A2N}Q ziNqITfbk7bKH$E`9V2}5B4;cb7ziSVNje5*BZtd1Xz#b`wd~OFZC~FAwH811lsFfl zBPVw3gKy!If--pAkc%4!h~u7Hans0Y_E6+vuUQh zaHibW1_3KxF-eHBRv-zM_6i`H9L-pP9IXmxeqw+v8}}*7qrsPG=lS(WBBbG zS{2DtZ}JigJ}-y9@DTt;=((vAbM!Ra1x9)Muy9fCVGQfK449AZPx>^$>0$&LOiTZ|--@ zbaX=~BAy6-w7tJ|RxdtGl!kdgDzCAqxMLw7R%BFcs5wG%CMj+|c#%o3L3)R|;(QsT zdPvHWM6uP#rmgwj1k&3vp4b@?i|z_jk0F$z)8X-DqL%hKi+9ReL)u zD_s~)#MIe8o59-K9p={l0X93>>%a#$H`!gz!TvTo$0W5EILw}A`t6+{i6G+b_yr)X z9+r*GEpG8sxp4es$EPQ=O}pQDKm)M!vw?Mh8SZfm41hS;huXRav0OY%-FoptAM}k; ze55y5pp5nc_OFn0j|u@RW{9j%GRG-bBeT0F9orL$ALugxRYbSk)K~_m4M1G%a#Q67 zu&lRO)y+*h9m2=*qaH`Eqi3)O^|TI=YmFS;$250V+}PUJ=lx$a zct+gEjJTg}ifxH$$U!$m=h$YLcRjcRdT7f%t$nyuYz8Z28Ifn=1+BBE)zQoJ z$^;Hxu!r_A*pDNBd>V6Gw*!O+xm?}MozXtvYHRJYi!m1L;Y<*aUKik*d&~br+?z+` z*#2$b+cBqPh$KVGkcC+?)(1Tw$G=_isoFixieM&IXeG*rhkF> zzc~Cq|6tALf4$v5q42Li`_Eth6N~@#A`0h!y~Jp^8AN&*8Jc54P9pI`VCuF+y85pG}ltT zYX0kcU;l4kt^UW)wU^od^UcZ$1lQ7l*{T|-jvG5};`oso+6&DM4908!2g{Mq{U6WM z9;>bMe|_GD%{u?B>#hCg>#+Zg3$FWbK5zZMd)~i!ec`Jhi3Ude*F)p z|LfoXu>a@ZW%%cJwmSdo*Z+C5|MlB$u3P@&=l?%``0EFK&ppcH=dk;}u5W-F=2PY5 zz5_bu@%~431k9Oo=4IOsU}}hAiJ%9>xfLsE+yZud{5D);Dhu6y_o`ovW8iyuMB58X z9_ue`-zO+A@OpLBCd`Y+sddp>YvlzreaMm@Q^X>-dH;2#77nA! z#qi~|DKV)EfpLhBBtF^Y6ny&#axT|AtKoSs{GQq95Swke@vRp zlrSrgI@`w1l}ow;FF- z$HLzywf@R#;L&jX?-tD*v_HIf^|_cq+|_g@ZxaVCyQ4+ttAU;J`Oj`tF&JH`YLQ5R z*mf#6r6W;-rbEKH1M5Vrxmp@Hu|a^>^y9_F{aL(wue_x8qJ)x5lU~No;jr501h-2~ z#HL;2Wa6tth9gpJbnUml37SO%(;4%Yi z36Fa3@?PjCK<0{?s*-_(s*yei_CmtX4-W*BY6czaBKkkSE8yUy=_dnAB$S#AOI|vh z$E+CvX6!Nvqtsn`)QlDIbZN(eS1Q?*UEQDxz;J;_ytNQ>9LY$IdR&?L<`u z&bp1e{^+tedS|zeJ|jWi@$=`+bbc>?&4^o*#$uuJlKs!DS+E|3dpy;tGtP{4Q$7uB z9C_$gi@u1cH%Ic1ofI+acl5SGZ5|P)%AIdT&^d1?=*8uWI36Fox7Qg4vO|=oTR)Mo z;!jmoco>H&h20^8RYb@Y1epXp<=}g_-(YKh9-Cz?=9oMZ!SQ|f-m;e<1gjM-*XPkE z?Rm7O2aBsuDm!X(8Qgw*@qU*jw9ZqCdHvn*yvFdM&$5`}IGS8z;N` zK6G;)hi=08_&bM1^p7|ZW;T+?pIDQAAC7XcQPJs9?<3+)-}Qp5K>yWQI4FOlh+*xI z@{N31_2tE81ncoX zo;+`p{>So4+f{MfrzRdoQj4N8&CQ~ijpZ?G8En$6(79{Jze;!!86U z(+&Ol*;c{=gF#6X1_0&XZJG`@0~3o+^$M>6s#SZL% ze)~9~l*PWTT~bqKOL)+DFGt$IVqlD>L$^vEWoHMg7d~TAKHYR>eK3QfvBz&*9;FQBBF1N8p&x zm**(}OD#V~|JD_tJHJ40X{mtKbqcNj~n*MrgQk-8T)t%&c#`eSK6`Q*V^Q3 z(0#pqczEo=Z~+A?Z@)TY27LUXvdpRvgUvR&0VQh%Jl9;B>pxw9>egTRtLy|AA4yqx z(}{_xe}Os{qGoWBC@NI#YyBACD5csA2r zk3-Q<{|$XiBt+H9Lhm@Ry=}FZ`Cb82mbKL=O<<5SE#cXSSq#z}T`e@_Bp9szeWOFpWqRlt$~`L~+xN{}7# zE9G=PkHabV7d3P+IIzs9ro~gjx9l%lZGMW_%_{vy%Ow=?`t)9h}hGs_B`4-=vGV#A2WnOuq0O-O1(O0`?lVXO<6quUFWp0Lc}4y z{jL2d2B9j>m(+J~_~1R*wF~vLdGr{0%`H48S6ir>P(N?ZRnJ zPFd~(x_o@*uXu*XUoAbhej=dI5$=4$jQTM7Tr&TI#it=hkG`HL!ZetVzVwH~&!v3P zl>~rQY3|YZ8<=S^^udW2EV4|q`+J9q(382S>#qeEP2*Nipq@?YTpO~j7p=QWr`b6h z0sHg6B#$@YajSLTtQmh97?0^T*td=L$Dr@|en)sT|LPaC;0A*}z7wp2Xg%tuOsZYo zA2__j=1NBw28HT3_NDjdksPvi#}ac654s<}ZbiMbb#Um3PqYvH0#>_z?j|8I*QMH( z?la3|##ZBXrACb zM?!k?(-u$4!LBXES415aw(DEr*2JN&yUJcu>XF@b<$-R*MfLq7US8_WqotdspH*LA zf5&LY9Yqpu6z`0EzmWPN_LYNL5QmvI7&$0_!=vOU<|;KT6ud@wC{AKfUEF-A_j48< z`)Y$n&gbx=ZL6Ko#G;ps_K!)1B4W*7+7!?nZE}AfwNq2V?BCrtsJKe#Bfo5X+#Uhf zuKU*&yapN?if(m!vKZd1>-8g)N0G)t-{(~l6xQ~ex02@4Sh31#EOBVXt52sj|8iKX zcQk0wdKTmR990wca+n{WHM?c2h$FKS17mJe57%oxpWA~+x?`6L(SpIyD!oAsM{A+Hk-j8pDSw~H!%3# zE_=lxk9fs#@2Rcv#7QcVU(Ql)e7rNqAes1Mv|sPby32Vq{*B&wJw$+QTHlkdR~bxv z=@+EEK%(9ZxuLX!#mZ=zy;o*3V7PU^uI=a19(~PcfV_lZCLud_y`pvgutsgmT@IGj z%9Cz>5b!f$M&Sk`LPo(C|(y%p*>4575}y&k-mpSazUC6i@5B^H7kI2zSoa zO}Cjr=gp8uU6VoR7mH0tfRIJ_0~+QM>8v~vgoQ6>}%=3VC&nH zb^E)p7~9z3e~`HR*<0^5OQ@e+_StMGr2W>CQx{U}SuAns9(Gep!tVSrlmBd!pkt=c zk$6l%v*Kf)LxcYL|51$wg}o)Ld;Gj`sv$5x$70|3F&swRYzgMSG8huAq+bFKay!p_ z@cu~L>!-C~>umuey=oQJh-1^WZf4wE%tKvA`JgRv)}kVn!xnvb>=uWl?932w&pcHl ztdPZ5O|A5`7kSLP^rtM3xN!KyWy`N-i;$fp`J5-7&06(1cilJY`E1*fLmp7?Ug>pr z;6e#c^{@2ox&i2R_|>`BhSUc$Jo&{z#2fK$8djwYVv4GEsjQaZ`rZ81=SBus?k$=> z?Jf(8X9e@6CZN8fk8kA)5ug5c@3n58faPKS>3fLJ8yphtU#w)XGv3gI%JsW8^lV z`@wCeFEofq$}#_<*+N{ren!|8f9Qb@xs%Fm$z2~Q$8)Oz*G>$op~)wCOy zNqKD*aD36vGa{z%s4m()g24cl(d;$KvD76?nvZHR=$kuMbNp=v0S$rCp2PUP+$@2or$b0+O9G^Umd^xqapd0n?;ozF_>-7b=9#`9cTATc3i~EVm znhYL4u&#$Z^|q#S&o`Wi!0Kv`ds8?>t^TVVevv`0k7dV9XAyCoH=8EqQ?FLIY*{Ta z$dljN(9%P~8~ya$bv-$(?=ffo^huN}omwYe?v(KSl#$vD@~L6>(k(gzfrHIP4l@*p zSJcP!duK*Hn{saV_Flm2UE4yFeMQWB{VV6rU=~M$mClTuPd&chVqb-kfN}lzcKD1D z5z#)pdGl1DZd`b*!6^w}K6Dw9wVB5xvy~gHHGzFX+@%lH`)r%j$-MrQzj?~5w@&!y z@2^;OueIk9(mO0q)r^Im`LUIabpM0yDpk)PE@69pT;+|y#Nmr8ekhYa1j%&`KS%3o z*8hD@4RNi}nN9PiQU1l&Cj8+X8R&JJysA)xyh`Q_V@#af#dqP&qRqrS#FaCPL=RWw*RXiXF1qk zls~sl; zJvn_aaA{cmHt`aR?K4Z(8Fk^Y$8p7hCj+U+wYO!pgUhQ#@KCr<7xxwcwE1G>(ZWd5n1Myl}Vd9C|}fBH9VfflU>FYTZsF2pT8mG{Nm7K+U~Z+ z2_kXb?nQTh^XO->C$=l)Nc&!w1EXiKSf7|+@Y9>eIt_J?5bBp_J$x+R)Cs8S)38H{ zcGB5&+NVLSDpv#?{869wOmBZ+u8~x@|^F--1fN56i{jP<6P4k0cH<_w_2$Z zR|T55jd5jAc~tw^ul>ZQJCxFg7O{A0s4L&@MjRho)Ronv`&3Rsq$Z1te>O)|o}_%! z&`3O%D`E1S_f1cUTkkJVm2{R02wE~}Za*a+PkzNtJurcV+{L#t7Z%WhpFcb;l=xB8 zCO-DU77nXSi#HmPw+5L%I)1@Mg7IG{;Rl^RCTNz$1_{MYv+Do60&Z_TYpEt9pqP7Z zouNp6lQ7yVR*B}uFwe366oc!NGu|E@!s1DQ`2qLy9OiTuPgNv;G>rU}JY=MVm<`A0 zXUzxt4C(dd1D*F2r8kB_I9KlYIQ~u?u(JaoCscam;Hb53gA!3SB6_cUS#V z{6l%BGQXs0Jxes|wY4!gHFYyFg(xP10WVW~0$NwxD?M~Q@ENA7*iv8CVlN>iOblfwZ~#UY#eQZ2fh!(As9!B00S zKW*c|XgH}msz|VX^VTgniuyD~Iki-thkTXN%nlO)?%s9d7eokHKhfLqpRW_>IGAfL z_2aR^XokHOT`zf&1ykwA;MBeuGL${i7%5m0f7R`{rq)%{IPx-1nd~&svl0xJa8TmaH;tZ7ex9K;>xcr2?+enal*zw$manKTN+cf`SlsP4<>dLOy>?N3 zIZRJoGAQGsh^`LSreA)LcZ?Z1!i0FMI4)Q#(v5}nP#0PAZsgS^-e)eC)BEqA`r3&& z^J97A>%YVU_d^#PySIhru*hzPMlc5(jpNFeBIVeO?^)f~6DI|~@~gZ+&#U{ieF>er zEUl--GaphBb;s!&7VZ?(}(U$%hQ;Y> z{T|nr@TeR)X4;%60iOoR*umVq9ND)T+tB0GVyuJZS*UvS89UAsQwJBQ(;mK~+%^_AhNOzn54j~g}*9CIr&%m z%H5~jnOP=p!_hpg3ElZKm`8)z)psZ9^VM9I_21tBIJzXwOCgT-4@y-k)+c{iy0dm{ zt%$0*N`JQ`GU%S`=HT#_LsF{e;r_+Il?mDPUrK;iva9zGNMaB%=v(q0djayFmA)?C zz+>0&t(W6M7<6B4q(4rQ0^@6GK-et-C3hqgq% zecf6wn)vEwzJbQKg#v0eIEh=2182A=J3$6 z={&T!PDGlK%cf;K55--&UyC2`a8W)S`MZ)w&6+b)m45IDoSEEKK9lxoRfWSu^6j=j zS*teULwaIV$trG zqtR~$2d~nD{_-6uf`9hgR8RS8;F7?`_Fi1k<(j)xUTmcWG0}na5bJ(vvJH&%Lv+%ZX zCglx}5rce%zgIboEi=$q^;tyC{KgrIwjzSFdW`Q&9I|`6)(pm#Mbg)U*AAC**nT+M z;uCS6{fNJ|J?Z`UqCv(xCNbze=91l{wG!;QhzBi|fmQl@et1%T%ecMVAHawZ)0LjN zo)YovK*YYn`_xY!d-7^7OIUv?&$o3TdG^rw+oBoyOUniI{z@DQO%J4&-Vl(j8{50- z9_82Xs8!!RDA&w>TQ!i^4K*Hju6GXcl=-5FBec=O#t+G~*vp`%>CcRr#7Qsu|GhB$ z66Jq^+pIroEQ}l!CtIqrSRKE{(1_;Nui&$OR|6iZ=Qg|SqH{6yrt^{uG&e^h%=2uC z_hauFYSr!JQ7f%Uts>5jGJW0tv54}v?$mC-AX?9uv&+{WVo@f4Oz|P*O4xwYlkGhz z-}b&MiS!n*_fo!drH2TcBfrD_*E2Y|k}GngbNFS;wuB|bS@BtJD>wT~SnN1@+E1E? zy}CPoZ+0P{n9!)_8A+VFyC7^3aY^R;%tP+U#C7F2E-m3DWSp|sx@}2bc-lQbQkKU! zciF);K@uXeYkF)~6S4YEq0hxI30Fs53o)x;uw};aWh-c2ZK9UDzS9Tf_mcU zXR|c=bPAA3-C;Y%pXUF=@~rOiBK6rjleeb@9C`IuXzC}ReA@)x{2GgX;m2w$)fhaf zPt;4UAq8wxPs^JjU|{_^A2aGzrAN-jCp>u^x?v$D?W25u7i6dZl7UJ8PbZD0GKl|Z zyJ*lB7Eu|kP4l~o=(#C*$35azVRKl>F-PL?fxRBx8p^iM^L9g)MvSBThw|%vXYTd|VFP$$P(^W#RU7yo;hVz&_U-Nm>QU;mFo`;0>rnzv* z=3BRjuy*wA6)=iJw?^laR>v4zyr})SR6{`f&2+a-duTt~s*gV*UYn&iA-BDn&hgV( z@3VeVFLy8R(Y=O8y=v0UjYl~=-`HUJo9;V){6oc1;#1wHP8v7Ii3p3SxOMfhi1yoW zc1^O7P}coGPOll1e?wO-N*KgpWLIEXV5|Ab z6+o}Yr_(l5o)+eRi8DS(x!QU1@{2;?{<zBI?O`+QVJtd^oK1P0{_ln!%KT(PJ)ek+96-pm9JEv2@;DqN<5wACN+`j*v!8B>*?^D6V zlasryZ}E{}UG&?)_6&<(l@FhOpn4v9(bFfeiM-rNd(MRgbS`p7ebBIFu(0apsgb<@pTY~x2H!hzmmaVT-YIZMe@Y* z8;-Iobr~prFWgqUhQ(Pu74z?O?o8C@hxKn}v8n1rMd=^v)rea``xc1kyz(RX;UNx# z9nWMX8!&LRxA>6zNkDqHDMxq>4)UpS6`DPWUp_7ug!iVg`nUZ~=K%7o?{n)jX7fQP_e|7pL>@Zl?f?6 zyiP=3VoHN%xCps3HxI`V7e}7#|My zWEp6e4{FZXPVaB|@#c9tFGUS9YGcV0kGflTm-l8ccJHIR4y!27^k(kblSK1eGNVGC za%IRhugQ1FGoxR;Ii}c4KP0I*O`FCpXfZ`myx>u{E;*gZ7d$y^j~Kgjv{ zLz?f&a=%k<_#zFm9bK=?Eo97J= z<4`=~ysTs>!LMiKlhsZPW_;4|o->Wb)$x1ICMgr=UwVBf%z;Ntlx@E;bYGg&{pD1n zsFy2UFWgzk;ew}q!!|i!%%0dduQd`{b+66umBwS8`he%&It)(dtpw(F3H>vhh$VT*Z~j6b|lYd`QKP}AvTDS1uxqq&nN^O$+P8S^xJVPMr;`4q^~RCOnuM= zzLY`zjQe@BcZ%4yc)MkWlYqlRcC6A0WzkVGTA|-10ju9xZ9ZECl#JBQD_kT%t*(E{ zqIJO3E8e?$ui{aCXX;jE>Zi1k>zDi{j(M0D^e!m^*rONR^&Rc+pFJkKuMtlTc>ORv z?O%G zAYf`ppT3Fi4C)Rz-F!;@_`hTOl^%;CebABqf}<41ERsEr;< zIkV#2>SZ@X_}^t21CIJ7*-5)!D1(F9U(2E{vG^1v9+LyGDJp zFyr`~JnGZykzM<~BY$$K_Vi0KuLy{CvscWHaAM!C6t&)!oiI|aC2 z(KuauoqEyMy{CIBaffj9-d=y|4I}4r?;v`gvdZK24jgim=e&JQdHhaj3|oAI#iLE- zIq`Qms7#DqlXrnZv~#9c8}-41K~3YcX8^BSm5&cN&En}T+k583-Q~EZR366S)X{(& z^)zR>Uq|N8p*{`xbf)KnK_c#s&5kOhUaaQRb1TSGLib%syyHdrJpEOi&u0c+b|W8l zc9ST_Mi$Le<58zweEm}z1JQIziscsu-{iH*=S>E-3|KTOZ5@vzA9}E%uXy-}d&YV^ zraYUqe)q#k)a%}TmVLAnk!Y2!Ts|Kdv3kv#06Hh;QO-;Bk|f+Y6(8C}In0#C%g)Rs z9+uPJcJ3RGS()AB`VSSMcyyp;j69F;d*0Qa2CbWB-vdW{fXG`ndX2Cqj`zRsS@4=e zVv=1Y>my>?Qw3LxDLir}w>7N0DL`A+=JCZ999FgFzOpAiPinlUxO;$v_w&@xHd+A< z>p!mW(4*_OhAui!dGKZJ#$B87h>gf~0KZQ$Bzy83gNlt{J-^NbChef#d`F=lkIItc{)4h=0b77!RPU-#rK3ASUdb&uQ2qSM)>Z1frqS3g!> z^8O>D&sVJxreh@x_Zazd{0};(sJhtr1elQ6VARuy!?7n{r>hKLA$Fp&FN8^;3%GkD~5GG z%I}YkXK&MaG^|z;j&%pDBQ8rBM=5vPiyE)jQm=NbQ1qt0x?!KMyKV$MuX5O*-QLtw zS}*Spp}ap=xZ<6R2Zym+^1fCRA5XmA_j*|Xkd#&T&}%24P$`oc)4}1di*@Og<&;~> z8lB>99-kFMlf(+@hlcU8^Y1fQa@R@C&>2W8>mI#0j=ZOFm!UfCzno>->~i8ig-zG* zTkYd8`^=NfbJi^Sxs2nb;XJzSYqL2d$08pUEAu9DXnO9x_&Hs#--p4+3|mAD8}ls7 z$)Eaz(>(7pj=`<3=Y4ui<-w2JIa*~E<#Kbym2f2y--~r(Gz}zV?XoUBbV00&~||A9O1r%C)s}6rV#%;)HspPn^JJLrbK?I;t;!OYvh2< z4BlK0bCIL-xX_?(wNDt2n(cA6U*bfZI5D>RBKd7y^P2i4LxFK!6ko1RC%;muYdt%k zg?!ywD|Pbovnzu`x}PIY=>GNR9AiK<_?#&JfPwzr+b-FZ8;hR5x@+}R#4WE{#}zjO zoUU{le7n1Zwn2Ke?1e2KhKUH*+2Z_%z0FF8s8^0& zvs?b2_(Mqs%SBclvrrMvFL0muAaaZRGTEBs-q65E_@Th+&OdIP>d~l^w zA;tl?y8Y=9zORTKCJ}o=zH+$etCY2#{3)z@{ygRFv|bO!#^jJkj9KFq+gvXpU`*@a z_h&>L+~}u&D4B)MrHA%j`s9uFv6I-t*cf zHW!%&R7vG&e@xd!8ly1hib0`qTu|rG8eFLHWFdoNeNM;7 z)A>}^P8!~gdiT27c8gUqEM9(y^4UoKzhJ|Z2eOnuXO}E0Gylw@O~d`FJms5i=;%M0 z`7G|0aucl7SOliX4|@7Z#MYiW=X@f6S#J4e{mNGo4iq{odEEhIe^@H5dqVqDBd6C! zb2(+yu9$g;0AHE2FYZ&$O&NE{*r-2??)%;5yh)J|^!=CAcd&>L2PgZ65>IVe7<@mn zT*P6M1q;hmfU)Ckheig77#f#)wYrsfVZi7(#~i>jwMDP-j{vdd;dj|b)Vqh*v_7N# z-`=6)r52LJQ;+$d~pRy4uFo(s?d-d+==|t)s75%lXYbhVPZ#Fiax8 z_G+%5d>$B;yV5wclJd9IDtW2MpzXth*w4cRn6FsooWlXDJ6R6I36`V8ScQkcABAN`>K_fc0KKo79BLbu`=wj*h<81dU3DBdHR{|g=d%oKrrcZ8qX&cT2ge;z zATLncIyQC}<;eabB~{l3;ux2ii_X#KX?%2Ixs{WMee>T1eAz0&F6^#xSs@SAQ6+lj zV+Gj6cukmPz`*DEL0iWT@&+3jvv&ss^t$XFF_%31wb9lk4ZV0QJM?=#)~Q$MCIL*!AA-6`|g;udED>MwFwGOKlB!5RT=orms?^cG;X zL}x&J7Xiwhe@Ak*JS?g2^J?R_A7Oj5utrv<;xE8As?R;9`fHKsX+5ktA(;7m_=U-S{aT>sLIQnY+x?!VuSb9%ivbT=BS#t20M7jN;K`8Q^ zMRO$cx?g1si&-^`%GxN`I??PyNE4Uy2rKhEY^K)S+D&Em^$0DR-Qb38=va=WDbWt zYGXIY#RKO5hE>izdvP&`g+q&TT+(RY`l`gHcIOZ>d`C+6#pDS? z)8!T>5_b;iC)?UnKvVRPlA)A$5Bj*@H_+wKw|Zb(ju8uY#p|PYPhjz8-`u-V_eI?P zP;xM=NQ92!0f%qIVOm;e?XHtAUD#MD2SPpN%7;$k?wESEj65~%c4ePz0R>5CGp-dIb$<8u76 zL)7;n`&YKcpXV`WN@!xnELu!*xu@LYqlxPj7d4Hh zbG3Ka(6>vUNqAReonrTb{5&!G`PT#vhd%N08@>W}UFQ@&r9Lpa7;Wt3LODH4%_N4n zIM_+CbwdvkmwV_hR{uonoA}7Ec0UJ=i`^Fg@DwmnFJ@e;v53(d!)};v7V&)HpiUh+ zrw*%Z?WQkh@$BYn-Q=r0ak65+4Lf*rHMHGaO}UdY^;g;82OQ3=IW}=7`ReAXl9qF# zh+1jogS#y}NuMK`V*+CC z{o1AzAz)o+j(%GYAob!JU$Z6&V`XDxhK7r9<7@NmC7{=kfk}I(l6Qpt_$-&rB2SH* zc2iHp{nI@|-pL5qt{Kuz;Rv0RGobYX&QpIT35^%U zJDr_$L8Cp}JEAzO`BQ6p=o*9efpc>}e7$~Y^b#9i9%ZkLGTr}BPTD38`d!AN=dEt7 z!^Tujbm9Q}EWe2lMT`^Hh>s!i!lnmEN zbUv=EwaT-yrF|MS!Fp>S5eioi`krwD2D@K~ZoMpGR`4^n;V>}Eve7ldnuo^YO9!=Y zO33L`eOa0OZAkEn6z?V;8&)q)-%lL!yrF8nM*=`oS(k&NH$d(Xo6^EjD5d$I?_3$zR zmP?xhUM&>yYwpLdqqJBQ?^u4I^b!XXExjd0k#r7@OnK5t*D3xrr6IkHgO}pG@NP#q zI2!irb(%b=_4WAcE{hqshW+$=8Y|%TZn*&_6)fy^eG{ioCf~}-KN)I9K6)gqPs=G_ zyKYK}CC$a9&oTq@saJaqy7cR|n}|cVpY-5X2`Kh|yX_lsUEW5W#*oiE{+!|5t&2sx z3e@v>sV?G~%>?JYjS^-Q>zd6tN*;W{&N!|JsCKSB;PPBT&BOcGO{v$*%HrcZJUBcY z8#izod8m_HoSx5T5$*=&)w@=)D6BO-ccekY&Ew;?TqZAwJ9O#d9m+xJV2_<4tbp#7 z=Qb+M0QL?XeOU380R2xkvBqh{b6Hb9t#W5kE2ng=B!opx!kcxE3kAeQwyyL^V6k(R zYpjJ9gU*=OjxT!Cd^(@*zoCUVdULZz-&z5`|29jX%{WxvD|s7i!sD6b@GG~JgbNcllZv1`nY=(Xr52a}nBRstse7@0j;yB_c z4FBPwae~9XjRT)PW9V~xZ0*^vS;7(D(`V(!vzR~Z?Tv%fXZgjx0ZkzS(!Cn4U7_bs zmzmUkFw5d%_4HdFv_2o-yv=o1Vvyc#o7w)iz!vY`fw6S|T3;MCtJraH{(C;E$1)B_ z)Ga@p`N1H1Wz4eY8;Jw@9J#W9dTIFX8$BkTmEi9C;=D?fgjd^g)*7ZflfwdwLmdg ztwt3le>mtpOUno(UMy;k9XMtualA&~2s2X#Ne|zzJ3!p}?(&=knZ%_BKG{dDt=Ok;x-5g@Bw{HAMd2St(ymopid7^dQ z&QI?J3{bKO`u0b_1tIK^4tYh?gMoYZ&^eQa4BQ{y$blQ%`q$wr3%%sGdgrOP)H4Db zzN_=dbKhMXszv^*TxzICzS}Fn*K%wx2Ep@oRz=YL6(8BRyeg77`_r-o>FbH3JHrpI zenI_tW#FYiFCHb^Kc4Z^0rYNm&3ZVH2j8W~+KnmHC%I2l&Eo*yfcIbO<9N8P+8eH5 z_s{p~9X&KJL&A;m`aMMQhHhmFY^!-drM3|Ml>QnkuMC63*M~LCD{=_WpL*(6QZ+qxhGRT2hxhMlOP zUNx9GU{$^XhaFP+$BQ#*9_nMhj0yofGH2-Sdd1_3-P!Bks5hSJyJcMYB*9Ej6&!C# z*r59M{pTO#FXvih6X|@)mZmNn`jvQZ^Hi0JeL&vm#lBl0g6)jIQzQN(E+Tv8}VG>>{oUk}d`&+iE$H>$y4hfI;=!@q? ze2&PrJ#&_FYRXx$YaWL_Pg3sbkpG7HzS=%Io<4u9*RHjB0!~}yZM7Qu+}DCM%|8}s+AzldMY zdH2-WE5aaoR97YPvfY|@_FtLAK(9gf&$3fIzT1VGhj1(mEUN$RHj@x}xpBv$p$x9M zJ&+AZV(|BTz>dLp1#C0EcSnJG!(A)E?JVu9OWxWMD}NCe2J;ize*=D3Gsd}+CwMH1 zPnbSOM1g;l<>m$+r#=o<(Sy zI|Ux3=bH3rCYW~E{`izF#-DqoZtW=w9&8n7yG?-kLq`0 z9d z?^BkaTFHWdeB-i;0W7dPx_$8J9wdD2^KAVS#CY*s(U=7` zByc-$r&%u`VO_@(-q<|^#4Os|)zcftzphbHYZDS49mqV~0`Z5#z*mNuo0zcj?1qGt zmn>L3r^m#!8U_%#!T#<$7;x(B_sNdPBfnRFo;9f{3y#b^6Be}<<9O^^bJiWiA8VWM z_d%Y%m=K)0ZpDP+8R9igCjd?wM7F++>pebUP4^v}NEkhJ(Uv)g?|1sVG46-{A-jL{ zh8xKD-~ViubgqUCJ>I8YIfQZIq0qQdg@OUHYtHZE6Io!i?bnIJPboOc@7TE4I})b# zPnZ~Sgavz=RK4fMV|+i`Y)4mrCUiG7-{OQgqTR8sO)j}2KQ5tWry@SEvk}eOkNC#- z(4&G_j0gT%!>=yAfcN>9cm>+A;EC|&v53xWw8N_1clR@4IMl z@+GZL>RK9Cwv4bjO+dnyylHk{q-A_@0$ zmKLO8TzsS$`OBy|!1x~Rx+Nk04Ijl%e4WVz!TF5X&+`eWXg}b3Cn*4z9J4lJCkf>* zle0cj3^3pPF7rn-0=#o92F^`Eymn{v`c0oP{>a+M0@2@fxG+NUh0lOX4xSEOZ3zfy zwDV&Z8v@!$gGg2b5*jBsjdOHo!Hoj8XX5RxdS-+Av1SSuMlr!eD>V&hJulu>}Ch>DA;S$@>2(20vc!TO=y&?WLj<*#x+E8nL?&dGnrxZ}#k3 z7Hk}wQk;cxrGO|}dtf2Jg3o5@7xpos*{4(UW>hd?=gv!Q$~gq|eB>~A9{S4-5#t8t zTLBm~@M-?5lns49MH!YjQ_%3bQIn$Q1blpX_iblB6BxypJ~r}X!Z_izHR<~Rd``BT zCOX504cSK*U9iG+a%E;b_F=%B#-D$NtYpI;UM{B$#7?9O9BR*T!;@~s)rZw58+WQYd0)o6w@gUO z5A1MqFB96y8_%17{w3GC;=B|352IGfd#BO}C>xR1MOelL;g&Gw@Zk)2H>b%08^p24 zlpR(YZze%LGR@`r0Duo`w=F!0eE!sem1`gSFd=T2`R1FJ$hQhsi9JwXw>LFt%}HTH z;jg8GpCJDhG@N{E>2~xZg)rUu9zc(wHBT(v;$b^x_Hq%jO$S+9`0``AtC!v_W`SVupqN>+sgfjhhC3+^6&)u z84uNG;qylnB$&21>vkRex<~8ybhLk8R~L5LIue`~kJu{>r9h^-7rGkx-pnI&;sW0@ zAkbor<@W*>JZO1)PfJ|Kr_cPCt45HJQ?_h!HLicw-p0M6^N9Q4by*c~H ziw^{BJvFY(?LHgYEq68@i}UuMx98EtJ|uLwS5jTMfPyrJczF`;M}XO##P99^r%R_L z3nS4Uc8zZo+lvKa=kDj9CsHu!ho`)3IY80KC_;g8yR?kGz76mzOJ-EDk%Dls+gLvC@4V|bH&-r1`|WL=yX6ZDj=PuUyuL|5`TftWn;W3s z#@RQMcVIy`r@X~o7i0W=*(~}n?!(!0E*4FI0M$h0qjkmrFQ&Dep)e%D=uo$5rWgk% z8=7}`x`cvNP0Jt5Nu{9D$;-CtB=WH6xq^FBD0s6ljhcqMV8Vvu;@b$^yY`4PYKy$A z%hbCjk^~kUUUj}>XT;Odf$Kat>i}*zoe&+85wN7R$s2J98yW>3Bv-vgyi}O};Cdq_ z3`yOWFk&kKlY*iL+3Wy_Xn4S8d3zSL5SmPFjeP#n$BgR1Ls&5T>-Pp5`XgSeI^Q(U*hfjiEmG2rIC;(`<@;yPh$R#(JDr9lyfd zbt(a;e_ik#mCArgebUYM_Q$yWcwLt7J~mWNzn1hqn1q^l`=?}^zret12PH@3WK&Uw`IbQxvjG9y(VDb_`OXmERnak zzW#C+{ftx34f)w8NnltUJ#UG8`_#Ps9V>DO;EjFAAA$Vs(#O~V<1l_kExgOURe-#> z_XX#rvzahOo}1P)lnIHQGfsy&BL3vHI_o)xf@ftjQXlxD{Au?{g#9Tf zJ(jy|<9iCa8h_q#;w18y8U9uUs9!x>r>?!8PQv-HlvdxeS&+W5;=#f-00-~ef1A>l z0crQ*zvXuXcoJ#<+WQU(RM*7LixKZ%u^!+w`vC*;?j&XK5O2)-K4yjcbOO2-nVhrs z!#I%m*}`QR;)U)VZqeroZb!=V<_u?pd5$}G>sJzvwrCzbYZ(i+cKe!W=L4{@FlHqE zckn(>*`3}ri-0|!c&}|5vY^?TIU$e}g z=C?t-;(W-Ga3W#b&TB_{F9k^Lc*E~&LlPQ|3d}NJj^io0zWp}F2cvFJyH&R*;OVqb z>8(Ht9IE9jPXiNj#%@X+vziSp;?nMOwo&kn__8HsDFcR07+-8~hk{XimNk8haqcs7 z)Wug-6#QBlWaor_(DB{qY13Ad@W$3+Rp4S4#9i&~HTW3|h8YffW4?p{bIZgL`QB{s zd6Rh5&Wa5Khw?f`p9SX=e*w#CR`et!^m02fMy%c$NoSbVLo)s z%HAsxcl8GL`2}onX?tx6KZ6Nv-*@P_;X4JB8|@J-I>drW_s;L1|BV3#n|rT#j&Y=8 zhYr2XaouRGem>3CtA{6fTEQBQBx9!CAi z?Qd{u00Wq7N0KJ-EEqRx@rrvx8PLFN`#_raDvwkJ5KWOcKK?nNib24tPJ#O$V>}st z6ZvxwCL|cYzTUuv3Ey_CZeE0XU9n-D;uz}jIgb%D@(u&=I~P}fxWt4$6%95Vf5?V5 zl9Y^l$bV043@u)cd~e&xaDyoem>_!BxVZW<3894Wpid)!i&LlCSVxo4bkmxXZw|6y z^|z_Hff(oZH@zgU%w$06iU!ig2PioIy!HV#jtOH6c<0_CPHHL|D;n4jdCAqLK^IO? zaCnKzdnfYX9$m8+4$Gs!t!?=2(cLKc5|hGiS_t6XgzIdzm;ntpc27Tw{Ox0l(ZXv1 zB(NSe>2u`;K#knH^wBRC+$``|KF9&_!s#=nJC0K@XwZ0<4Wrp`WrqW=2lBO?IrGo- z7a%V?WM?`*3HRqtWj8y-k!#+~drRTf^ZT8Rj}aF%EtC)S!}S>x z`Zdlp%vZq%-@vAc`5)Mj)k>k&GP$?q=F$e`v>q+&8jk<=>d_cHz}~)z<;;Cth>5%3y(O+eMK% zyO=Qj(uTfCO(-DN!jgoKEGV!Vwyy;MjLmH{UepESmFxBqjp%>Z{F8NXRXG8d3#2z2 zv;qk1X_mUei3Lr>=LU(h*zkSGk&{O*60nerIctafzE$SgiN`)54}96Cbm%tJ=WfS# za@h=EgkLi{+6CvibHk7wnIxPlwQW4r2K|x!o}lf6NO*a*>zQj8DA+i0l1*#^~AEb6zY+*!uik6pICmp7d$ArY!~WqUpm+xh#pXtj0vzWHcemy54 zZC9buL$rtVirqt(jbcLkDQR=uk++=~-D{XrB@=FL3mTh?{Mevr*93R8V{Xfu)bG9Q-9x%3Lxdq0dZNq%;v>@Q&)QwjgA-;6m@pW6zV@#N~Xn0y4 z&f`-_yOz7RvEkmnZk)%63xyVA3;L}_9P&JG+B5WY4a z!<7^aoYWw3juicJ;e)f6P=6}dHM-V<1YnN4{Ojjs28@6Em75yLhT%O7&5w{IG;DZs zHZz%o&L)k3^9$qd(Bft76io2Ddu@Gs023NBPo6LMO2P3h=9m5aSRg+>@6+z-0HpVi z(HR2)c64m_vuz0OV`6OIN#z*VQwA?f!9(ZwWZb_rLu%jtNdp`gom-MIN0| zF!;x6fW|gmKE6z5g7lQHNQ!)7e$Pewd%q!}@VfPmhW!b6wr}(yqn9Lf*)ed#HpE#f ze$Z{E1q+UBKNnem`1bIWJI|&R1K92x*UeOneB5kPGpRWn9A8SW`9vX3f$Sb*F|JQf z?i>HAgoKd%JYNOICuk5IHx>E!gXgvn9_?g9@(#Os!Io^WExWbe8{=$a%d%XPCoDMo z{l|_IDFCyCv%0K*$AH;M(@eAHFyY;y&4oFEOqhG;X&=`EB;=;Xaw z?zPzGW=oL}k#f^!W*7skDtA(8$5_ykzv?HuhW_`5{m$P&yjRm~Xt;kU# zPtW|G_w+Ff-q}<*^lrw2cE3LN>fV@yK|JrF1@llpFLWsxbrpHrkcYRj(2l(i%AEU; zWkQL6a#$-a39~!SZ`7uegsI|IvauEf_&n^FeeV~5dtSkn{CGAr6!{L?vIF(M-N?O} zF9=9XeEj7Omw=z<_m0}kLH@9$<+A051h~y@V=tYIxO&Zvpu9K=K9*IzJB>Wl1iGBM zQA_{bl187VyddGBV)l=Mb`Hx2Q9v_0sZb97*)224Ra?gFX+K0ftzL5 zW^5w@T4uM{kR%~s7SpffX$b|Yut9romouPXdh(!$=x+}?@{VkvHC&6(2Ls7X4?Dt?QSr4A^%s)oEB90Y!c0zdJ4^K^BoV=2tQcPFN2aHuM(2 zVl(c!VTak!d`@su$4N}Mz%zVQihf|C^^2rsN^IAm$%QpVOt^e=u3^+H7AO-#JA}#D zFzivymm4K4cyT*At?^wpY>FA~V8{V@-`Qz@7{-~Yi&M;VrlUXJ{3UwRD2xX;*XOn7 zQ4o7~|7mV}HY85IVbfK>f@8a56NDETP-|fMJaHovRQ4~Es_}Z*ZBy2bo`^d$nh5T6 z0_fZP_N86O_l~7N)7RNG604K*y=#?V*n-2>p3eY=pSGY|qNb zmc0mg=@g#b;vImo&-pK>8nMvq+V^gce(xnA-!&2OVc2s|%SLT+AJ=a1>@b)Ko6qn< z9yDTr=h}yF_vNu6tbK=d1#{8=#8-zdXh5R>{9!`>eazEkUQdq9VS(kV4aBW{fS_ja z_C97=DHH61eQe8yQD8w0?RPSSiTmdo z<8qAxwrH;9K4ZjRncoL5v?QT#R?N9==m*}pZ9IB7g#l~1J!K=66*hwo1?;mhqkJ?&A?YCjvEjOpMdSx3R+ zypw{3Y#V4d=lGhiA(85gIL1*voJMJ-{=t=Dei(zaalns=Y`2^Tz5|ty~u%Y|L zF6IsVNEqMJZqs}b@|seoPzyQonX~d?t2&X;{YTu^jrIf(Q`QZzSV=(6&NI8H6C}hP z*=3z<#e_FgKd$q}^*bN_*mA}OfNK|;2G5?wgdLe@?z?>_pwFR7);3Q9LX>Ooz<%U` zsftcUb|jQpWQf)^#`tr+)3rhS5LY*gbsp)8{I`wuh?7p}A1B`aITZOg`J`29STzMr zYj5r0*s{U>>~c@I{fn_vjZ6V^8 z@}=#DoQY$=GlA2U%g9%KyvKS-{8`|g^UHepJrYWKteClb5%R(w)`2&WKkUdmSk~tx z0h=<7;wB{_FCFJqu^jF7PU*O+KJVziyJP#h?|GbWsBO_jn0~XYIoY$mERsbk?5@P2aA`7(bkXgOA(1;D;lA^6heeXD9`4cNbml?#Y6a2NVf; zBN)&~7S)4|cJOYY-?esG6i7-oz8hFYLZ90n33CVvEX^)GE+0if#XQUQ2a}OsTX$?a zOvVH=`-J>==y-zA~III?BR$Z!U@mdBnO zxR8KFTb7o_V|h=5c;l(V07eU+ld<;+aPQuD{iBBfRBiaIO%xlP=RF))bcKXkLGvLc zofwefxOBnEJQlQm{_NDXVNB>9)aOu16ba|2IUTA*Jza42p3e*x3H{irBy(&Ms-yp8oe2t$9K5OgJ|1 z+mUuS48$*UGwsJ^J56gdfZfVnKN4R|B}iS+H(1 zh>0C6V0LQNJO2R*hDVo~X5;$CIJ40E2gLO5d(x0;AoW*KJmk;BatV^TIfZv_ARgE)e^rU|JX!cQ$h|oU8I8J)ia|T=k~8mNr@qJwMyqaq+Q5L+?3=D@ zkXKwDEuVD)dHvw!F=uC*GGY6o(&aXTn9$ny<$1>~Bs_MU-t~+v13qPJTwiLHX?Ckj{7xCq$LiVB76G*uItHat%T=<|5^1kV_hx+dUIA75*@c2yztRAzrbOYjc!Y_T92;*_W)frblAfHbR z9-_MHPr{0=SL5a6tz6g=P?_s z4_)oqF^YhGXP;zGFK2^`i~IYn^xuVeBsh5tVL)Zyc1K!XX2J50_UMoS_9-cjI__)w}p{F<}K%qz0HPPiw&o|ok{RW+-Gq4JleJ0(29c& z0Bxn44O%!MzVHk?Aa!8@=So<&O`XvH9$lIG<23G5`r&h`WyqIhRx6D$UcdUhL%j@tK4ke9O9DFFe3LfU1=oAP;`yNl zs1NO&Zl*Fx_?}Uo^d*8qzjP(nyA;AoT`L4=$3)yhB&3u9X4FaqKkMypX0bn%vTxaQG z0U3^Rks5v&HhPq@_|GM>k8C*iuub8uTs9PctjK>wvT^R@O;KPm!jYJRQ94cd@38{4zYm-EDE+3zj=IS3JVS^TXk7-lmV7~;o`-Wh!2K7 zwTZe;LQ0wUlY>uDPvgGcZQ7TDfVgvADpHt`{_(;7h69*T^wD!vw_J<|$1RR#9%ezr z*rB&Jq8~cZ?N#KGSrk0@an5pQC&V|w+uE(FVSwx%Z&?nGqwMlvBb(XiC)z(e)A}+6 z3)i*FeA=6UMvqmouDK+fR89A{FvPgmf4)@)&gX?upKTqHzYTI&oINZ6aW2ucOW_F? z*pz%(>zPDA!H|B*4ig#ha#8>C-d`ASWZ}|Zh8Q;%7G@NL{$fD#i^9~qCJb0V{d@MT z1{im*y2ia&}>bxt&UyY#{I}cFeCmiRUR=H9zl;m{0<3g)2JGPf;rzBvNXNWL$*l>n9H-~O|#StpL z8uNVk{=A~VOq+Cmr=PiU6BZZ36-DPANhzi} zfK0JM9L`{cg1~o(-%avndbIs^x>IDEieO9X3RWR0QJbyacf2?`w6E~XjBscZ=9>b{ z2_9fvUuKrfn%+ZG!9q;sWWvG8gDD~$FP><~_st4ZD4g~x2lUsycu2`Z$~)vIb9VZ# z*OOL0`t|i37s1JT2JF~bEO;z@6#ceoB@U}k*gcR&f~-;LQ^l4OX25non}dtYGl-?3^J zXPP(JWILffhbO%;9sJchLW0kWBWv|P#Yj?k^a2YdnKslaw$^2mrk;E{4;MQ8?fB!u zqw4Wze68m)7oL`}cig7}mIZxDB&5wtlu=ErzmbVhgx3M%(?z70OHoc6*N z6u%m5>W|u35R>*i2-w{5ankXATDlgGYV@X6{9htxr#%@-3ZL`LI3VL69guYV@Vkcp{aXK8?Uf9tMHROpkiy3b} z$@1O(WW@Xfw*&FDJl?o>;j)RT^TR)-a$^7?{5xQQ-)vZp32+0 z%00?bv<{qk_CF#|0t(xoI1C`!bwGIs=Fn2_nV$H=XTzF4Z*Go7bwV)bALI2O9rKOj z$a}IY%vx@CVh@iwO*T(ZKV%K~?b`f_nfQW0s_rw(Km>S#IV3x>6Y4NHoyJT>yCazT z_Kq*+=T7ObwkD<3)ZV?l^j?0zM=i-eF?l57J91TVA}W2WTw8X-%p>w~uBJQJB1Lt| zHPPuJ{K%fmqp<0Q4@j?;cN^_fxDi6fvph%!96>~u&0B@gPhd}P`=o9DE0LB%+?O%o zEwN4A897)7?*;33TVn`sS0gmQG$P+CKzA?=uGS=qt)U9#w@9(hW?S=6Z0`wt4nbU^ z8*y{Md#NnE0Oc{;w>m}g6mMN0voqhl2HXDJOniYl?~;2xP(X-S%DJ3hh$0N}=&7{H z2RE|vY` z!LY8TbWF{#_S6M*d%Cr*Wh;lL*zsCb=txH~eJ}wKOLCfMKp~H@rR>BN@=|JxVDM2# z`vwn2sMo}@4$%%@IM@U|`;5QAV&3CBE>4Z>ytIw`eDYrodF?-i9;qthieJh03(zKa z{eB;DZtj-+@@8M9p1j+`k#*ydo%*F;aS$bK}>ND3fuD|{6P;txJe|0mh7zh z(q;mz*4e_&`ABR-8F8U~{C|`?*>NmdySST=hjC{Vsq-@ZpS%GCeGkqj#VhSO%TmqY z@3blA?fdsNyt4;MXMhS|56`I^eT6$O;W_m22J=eHr^xM?=awwGAlnKkosIX_MIYQt zO!(=?>1*wYOy2BIzF)A z^9%d;7p?|U7B1FZ!yCs-!Y5g(yzDRKntsU5unXUFG1z>{#=F67WIZ?t+x1L^$`)R- z`O7;fCycVX6RfN4%vMfs(ZzJw2|d-#8^!iQq>-&5o&bDLH_xJWynx4SsYA!V&Uvg^ z`o*Z;We)D83{q1zzIy+0jTXCJWwtNQhCuIZR}Y*LrNu2&I4yfjhUHUu__?nljWDs! zgk<3g6dvm`YF|ghGw<&rg3; zM}hXt*27(%SnwY1$6*-C<@c6xnu?kk_f#LShlW~GZmoDmUpXJm@$JUw{@e-j%tj7{ z&E)BH%UT*XdN#PLh0zgOZb{$J#}NLjJTRN4ycM@;nqedNn(8Q(0|&L0{;^UWGPEIE z*ZsRkOM&$rdLrOYKu5Wm$x;kq6%?-nCCHgZwQ%l~|QcfO2FeF*e@ z+t;6h4PP+(!6i?g!B8EyfZ&a$0tNN8PdEWwUX4|H;x~B0@D0@6uLIeHkq| z2M?7z9T-mn#_jNbTI9u`hj57?zP;-=FPM&C0Huuw#7+Bp zr*}8J=0oN0e~jiC17XWiLF&Iqa60#cLMDN~x^)vSGpvKue*AGCbhMTsi}v_Tb16oC zd2TwpEIHCab0s71Zla}lb}!zdq`jbL>>FfJqe3lFEA=S?xb_-o7}xFPiT~liKGo%f z>buB??{$={Lx+l<>u$XBjn|DO=VJ(or_Vf>6A~qWz^5l98~Wd`60Rq19?{y$344EM zrGAms)9K|bMfj{2v9h0u_szDaJ4AS>$)bY8npBUUc0uvnZwn8*&oVS{vEa*96KB9l zqA?59*#oa6l;+iqviwE#w4VC#zj>^A*n31&5E z_(oTdh76Gg{vQ;VM(Am4s3mnt^v}szMF7m7-droL$aULWQRTg>wGMcC$hlbB6D&h`a^|rt zn~nd+yS8;*gLmi>JXOO&TJ$cA<#-q@N-e`I%-L|phl-{iqVYUZ>uCytrW0n;z5BR# zvExL;14vEFI(MYQi)PfFXqa^P+3VWm=){UMjXfj^vX1k28oj!v9~-Ck44jg|)l2g< z2&Otby;5oL46=At+%}x)719Kh4COm74GUvlPjAyCYxwbICq}{aCa5rJs=hn+uH|3d zQb*K>&e>b8P`M$l)hYE}!NRhzE+6DR#r6v=S+mrFWHkl5)EYOsiG`1!G8K^~2`qiG zN=~vzj%p1$C7n497kN#>WI!5(5|FkHK(AL~(EIuQ3tvoNr^vQF17}aR@yC06B{#pcJua|m+7WK6d)(<#Z@uDsDUR6~US1Awh{>#Rpq43{ z9Sx_UW;Nh3Ci0QHGiGn3j#M4Rm{XkTHtEr`D}L@REO>9+M)=pb&0Fhl7Pc@+c>{OF z{`KN4t|5)q1NsqQNO5=`dZrAR=?C7^1Jd-WOVk&CPMbh~PTw?$-W*%1z{4?SPm+z! zg_`*7dcNr*o}f+{j@CD}j_v{&(%-QCq87B0uW5WIWSbP4fF4+`w)pB`#izxRQtirO ze-qZ6>0;a#AZ-2Jm%xy^Jb5=qYz}k1R(ptXIkYnhK3qiidSGm%?+7U3+3sy#PywR| zBZ6yp#|RTMiSxf3zMv$|Llv-mGc+Fifnx2h_J73*$y2ax`)pmZKXp)Fo!?ti-7a08 zD_!I2E+V!7PUa~5elBHh97NJES5fV%OZkVM8sM;AQoZ~= zb8ye(u$zLTaoy>wNv4J{o~IP*d@X}+!Zthhfl)5}89a6TMf8PVs<6s4jdMY=_m$aYLh-a7t zA;~pHta?Vci+IL2b670&TO}JwyxjqHG5>fk2n72^ekUW97f)Iv5AAsO;rj!0_mqKr zd*|qH)w3YT&F$@{&MOz$z}}SlKSxxXf2j{xL^XWGGrF+k1i|65YWQyS_(c-)gLA{+ z`=4TEYwRDHqE-N>Kvp@q#po-1K({fC@?B3<5U!VXFe>tAZ>Pptz)j;S{2gM%y*^rURQisLzKe4t9%d{m-WbO3DxZ(yYfP5ng;m$PEb1 z+sCLw*WB)>-jIXI5%D6=WXhncCP6>wi_?TC2=Ma0=rFqREdkq~zP@NjD-;<;#zs!d zmwq=L%$6kn#QWrv#tn0#fv7om{DC>wahaDV^G}?C3QHuR^ z<;_pIdZEE=*^yRX{;`rj=8A_s`T08*Os13YjMrEROW={I5Jp&=4{cfYen28UT-_Oo zcLjo=7Jt<=()7I2pa{Qy8Jnk$Lk?7l$bQw!>ogf$AgaY$T7mCTU;jRtt?CV(wtar+ zB?W8gml7{sZ4s@(`1P!hffhsNmO~%$pKh8hUF=1+Nl+E9_w%Y6?1mc0rM-tdfIIFy z7KrboSsp^W2PD%2sRmTYyUc#kRJEd^=XV4glUDS6MwJ5}uL953)zALPAC`Gwf!OWX z*dvJ?X?6M6d3L){TQQ|59jK{wAC2>#(fD+Cq`)4C3t*(s7@Ad%5Oy1rALdI)JuAgd z1hvui*5?rhZan)9_8Q0up(3CvF?&Cc`(7qdD~`rO{?J{KH?GXbxPS4PaO(^jbJ7qR7R{2mPrjU_ zsYVaR^_=jItY|F4MZStFmBy8~SR|(xj1xr!^Ji-hl$1YJZlt?V3sC9!!4`!Z=%@0s z3SzJl2mBRkGJ`Xnu-~6Mzfkia7-tHG$9_VeuBSNtP2j#1J$EZAo`&f%QgiVg4^C;k z9`MR44^a2|ZW6w$)ayTQZvAtO3F8sYn5)B0tpBx8 zkMPCehrGe#7WVP+n|a+>v-Vx*VY?!&sYwE&6k<+{_)`(7F7$sLWbXB=L~a9RD>Jnll1T_k zM80*Y@I)xZ>Q!S0>dP!!DLyyKA-sOJ*PL5XKAw~SwPL@@@jqhD@y9N%{{W_#56kS1 zjt>k`#b(tFNm2{XcX=_NYoR2sBT!se@yD`e3BSv{h7b8tpkfSFw|DbKd%@N+<2@Zn z-C5K3-cuI#R8N-ekamuA8r0!oa=*AR>br4<0A*!Bm(}9gVREnYGpXMRAW2QVo24NM))9UM_35Qh}>7?-B zo(IBZNwf`P@VP!Jf8dCtJkj5^G9u_ch$7iMk}; zq@xa*y?DoAkRoQ+6#>^Xqxd~)?QYIJ)%b+UC3iFt13Cv{TzdR%ckOROXfy;rKszet zZIS1hE1`Q=Ytn15IK%rac%aKO*B%S@*^9%&BNuBLa^H|$i|VnQYj`1t0Bqc#+E>+z zxr7W;Uy8?Q!DfvAz+K6it3j-Hm@Q3$E?ERttnNJ+U#Kj`!)^Q3+iL5(AJGl$UCDP( zzi9(?YKYLRr0W&KyCF^lOcL~x>egR+i>aa8*17_KFHVk~qSuNQ#-V<#dxKRqgW8N$ zuZkqve#GE9ahncRCRxLhKM^(>_K@0>LX3ya7@nFM4*$?dv=Ip3CtDQuoj|&R9=VwE z%gtFXJ%1$9rBxIZ5pT93psL}RpjXVtT1S4e;?UamPZ-7Hmk*itP6XoFvR=d|tK2ac zSbif_=lhE(XbwzP9e^aFOu_YuG~l=l*t6RE9&6b-P0D9gHqZHC9#1q|X~6q3a~z8% zjhhWTF#qSz{?)Y=Pfl1^PfLkNvE?jSR9hF+l|g(K_upp#zmv(wC{33T3R%fpaYq{U zD$$y$$TS6x9d)VOoy@&W$Wr+7L2Hde1&G6c(ai3D6eV;U02U$BEPiU$kAMrEAnThGso{Ne^mZU% zE0CA*&!Je#YY?HgUiAbN&loQ}q%-f?071Uv^h*mDQ6Scp-Ls56F<-;@dsl7>g0371 zY>%uRB+*_`#6RUoe;hOpp{md?QF9MsR z@5Ve1)7*h$lrmL%gNn&E-yZ1Z&Hh*HqeZ`rO9vn~BrwMPj+VB&mTVFdiG2s+ks)GyQV0s;YrVBt)O+nyA>KXj64PqvlIo zil92L-U6^seFv4EJLcnF`+B9({+TN8x6@W5a1x|12i-~>?#hYXNONLe|n^5w=@ zdJ)L_i%6$rYr|$I-p95%5d%8c&VR?8tQt~rF1%tAMUyyo8LvI-8#0BIC}|Gj-4u5u zJ!B+>lp;p*{AZ-X^;lE@65;n2GmL+yUi*i(>K{ilCqq4=65ke04p)UoP@HQiA2IQg95cp2XYJDzzGTSNx7m8I+DbDb^~N*h**Q5^*= z((nEL+Vuk^H<#Ik5tuoR=-)j0xzcIje!ZWaBUQ#I0%`CB*N(k$8!f5wx= z?a#gW#k@BID@OiJ#=vW45oh888-( ziu|^3LuXq!Uy;_|>fHden9}dfzXI!lcq_gs;am+ugHEBuyK;M)%fVfkJ{G?Fy>KC6 z3Stx0M_DFz1I>z~{@6&#f+SQn>G^EQ-?j@hB9OQ#9sC_ zAGx&J?2-&DPEV7wsl1FO>)xyh$*QR@eQGG2JXMzT3y>4*5St`NiIx30r;oQlu=-OR zcX8ZPbs6SE5z3dMo{lPM%UUlgNTo$RFJ`e(eJ`uD*y8oQH_}Joq6U#S>6r!M`tJdA zY}d<4rqy*$KZ`GJ3LM8p9=3eR;F59CkN5>w}x%T1Zv(0eGlG9v)nmtbH1_$zXDUP z4YZTO%Sa#|y~QJ@2(ek@RDB~{^+%GL|3QNdUKy}Vt5!})Iie;s+61$UROny3l~rk> zH7>>JmC*=cbyr=)D-B^FcgEdY3^y98lU|&eQ=cKSR|DHm)PUB!yR_=h^_)qNT!*a69fcVMh;^e?kWXm2D!9 z5Q7F~OF$)xml(}%txDvuZ;@;@zJcM1H)v?CFAeee!?GLo%i*Pq@iMck6!Hla@zHa&S;Wae_YMK3K^o6GAmOBTfSK}S-(ai-;E(n;~t3?Gd_39 zUwUQ4F;0L!$ZxlJzxxO-f|__{)O2^J4`6BMV<_=zTs8Rj8*7>zWhpWF zs?;h?dRY~Z*w@w6l~cIOUn&-g37Yf}%yb6%wLo2sZ==g4k+g4Gt-VD#3WkI_-@l7Z zn4NX8##85;>BB!wcF9YOl6P@n%-0Ke@ypnfe>NZ2h2AYK#Gd(^d)LV^jTIJXi z*>O@edZ)l!!W&~sDcqcWRrbsiQhMy8`=jRy4jWtRJJ@UmX_vRgRGo;e7y9H>S9b1c z+hR|#$%~a#oOeBcB5ANihii)1af^aTA8S;pFuRdh$*(=?xjI#`Bb)Z*-Xwa~`;9s9 zGnVa{s^$2V+sh*l35|zst6%Edk6XJUfYGbxBCpBzzLKU(aeKE>sGd>}0KU!2M%;I6amH9om6oWG?KUWoLNtCjpwC z#1*!0gn$o%D6FCZrOnUZ*J|?jMj(S#o!(?0*8Lbv+8ee_?ZfNNmw|CA-Bznqo zT(bSTyt5aI@m4aOsK049BWYYl)n}-tDnZ@CF61I~xLisy#J4)Nd-wOXR2TUCSdOVK z^l}$~?$%Y`Ukj14z#PrMZUPDu;eH|O4tqDHXArsFo_bxWhW*sZG1QUYOzKRC8aw~L z={~=o6UK!CPgUvEbt@&k?!X*UHKDCWnwW!BE^0R&z&-30+$G8LzDM9qWqV7%2W)f; z75}D>c9Ts+it5}s-@J;)E+lzq#VvxK*q?%ZbcP@bT8usMYcR3qUM;g-gaxa}G|i}- zED)^^Mu0e3iW0M1E6O+M1F?Y#ivp|$$+PLu-02-3B3<#vZf@tX>`J;H4u<^@pPqtn z2+8AiH|*RmA@&;L+TU?GK&p{aEamd}p}m&}^s&+iyxG||K~LD=eFRDR)1pe+ETXpb zlvj^;Z8~__t|g=v>C{HnpQq69fmvSo4ln5H>8jtJzF`K9=&z(Yv2wWD)>h`$GHb(P z!^MOwTg(2UYTjkH*4Ij&)$rW_)>uj`-bDOBN}hZ%D8BfcV`$AR7=4qcC~mTr6ryU7{6g)tB8{d(XiobuK!xTL z-eW?sXAA&r{+cD&qlsPCNxr9GV2n`&t%x*Q!@7u|eQEXtx7ZJvN&<9Z6QTZTZULuU=??P?d~YnZUGvG!m9hL!~N| z+$T)^`FH3D25>wy!^q;D%&^BSK*efo1ZM!5H}aX9R@zqVep<&2uYQhJ_sYKwHL*|_ zp>2?!A7ZBYppVN3!+jtmM{wjet64m*U}Y6ZN%#i0ASAA+%ePx^1)~87~*=_N$Ow?|Gt-7bFqB!!TUI*U=6)-`fp__`_(azkJI+}=cxoIqWyeV zZR(~j?jhc`cbjl9x3hRPcSZeDj#M@Y%_< zMS7+QV_O%0{fk|_)^5>anx?K7PIe{vsik~EFcANOk9PZ-yhwHBGn0U)0q3v%kD*kx z^UV@{t%UQBC-n_R979@9+GCq?OCAn>hh@CJiI9>M+2TBTnF{~1q*0p_9fcnqmpBd= zWA{%QGIsE5q1i7`i&c5Jp9_#jHcY z{yLNSohoN8+(iwZ58;)f)HF>ikkqal)UB1^>(l@!4F14pg%`M`~1B04xdio7a6mIa)0CY-EKLw9j{%N0fzyeKX~#* zmIXmRqF1XgydnJ4~>RIbh%)r>6FP<;(qe zxaBLRpY%`7u`E89!2Vpo%Q+zyI9Xqnw;S6B`DI@3(R<4@#2&ToSFtZ0BwC~zb7Wln zscvs;>TRN?Qmxx%fWE?sj2U8*`YH7|N4-wHC=R|1ZG`}KxBEUh!`1_XpD=MKuc#(7 z;0L_q?_f?fYXR1w$-FKT-E;01=jRy(wf0)YIvB)1d{}|n)a;O2EU~v84%G$P)wK)B znJtOS7)XCW{=F#1a{gs&dGP2L2)^#OJ(S`nz3wgz*c7a8_h_tWxT53oRPArGQzOMFAtKzE-}g-xZZyoNA?STM?MRF9$H@+X7o*YRu3mNaZhJZ%Li? zfBNSWDJlU_q~+;$(M>#g(}%TO-1m3r9_U|6B`Qt0|9R-G)6Qmh`uN1>E?;%~oERor z&RnJW-gxpuK}o;|(^9vO>h3#wnO@S?q}9E$9&74Z65(;zg7n_Kad|P$TJAEgH%%9{@5)s&=AOz zJWVms;9sUYo_064G>JoV`=7Me3hTFG#y(5%YU(^`q}-s7Y+5-%==5US2; zbJx6AGAE%JbRC6M7aoTCS#>T=BvRc8Vj)YN1Mv376KSc( z8WXkA26iGOSc}q<4c$`Ws&u9(^8n$x{*;r9n>gmrzAOU)PWyhw4_SSW_x9Z$z&N+z z5~*%=-GQ)9|Loh3+0+a2s_~0-a68yqoGY_sDDezv%=nJ0R76UIgnIP8-YAsZg1Y6y zXY+*nKP{qap=-Nxly&`G+<^)fbkssAmavj$UJE0+8?e?^_uyNrX05%YpxdY$`>$Wo zQHQC*dOk|lfxr5LG)e8d(ltT*;_Yp@x#Jv~G;?!v%~}*~jTQCIhvgiahZ3q;8O6@! zv*$nFS$o~QJN!jE-1@$Z$$Cw%!&g;6)8E0JX1F6f@Tg^WK+4KadC9dcaHQ_7x%3D`|m5;k&WHcbsMF8D+h%@$DLnU zb0H^$$(CG0l-Zxpk$W!>Ru;BeOFC4wR|Nwu&8lMOUgLoY-OcwszKlY>-Z4cFN$u$3 zw7TFu9*t|e$HuRinjhL-)MTJU8xLD>jruZQG_lkc$8-0JQ~Pg>rpq@$9Ehi4{n_demD|-!Ju=YJUX=b%b z1F7+=x_cUzX<5CnoR< z<-?=K@~fBogH+aTD-y@y@ib7Y`^UcY9lECopZx|~OXmaRf>ZC0;<~z~*URwo?2=DC zuWIF^yxN_7f^s{=z(2hj;2u2#afm{Z6dxv4KL)QlHE2y zm6jt+DBRthi^kz61fw6AI*LY2l0@6)T*tn(e~3CKJ3;bD_; zMr?c!Dy4fSN^A$P6FaLc#{;xA1$p+4^73C}6b%;JoPVA)cfjB{L?D@^pQ*)$AE0b)d z@09YCx~CJG_z(5$7b%#@Y$QaV(p_1$lIFbF66Qaf7pbZ&sTuY`? z#gr6a6j-=q1|a!Jf6+MqVreKbEFRk5q{+teS9`vr*q>5S9PA99K(-+_=6j#^sZB(L za0E&(mC+cjgx|lplKe+9->c$P8=BS`WDW$6KDYY9o6MrEXb6jjYMNmT(i8$px}jsx zF&KBCn3U!5prR#IWV$EuuHpx;D3h^Ue+eDZSCk_E3-+QolHd2W)Z_g4N#z}|Ip8(q zk#&rqk`HtcN$yah33Ehx`{L!q*Q=No(~FW$s1rk#jN++_2f1VR^kglM6;+_3Zs_4K>GJ1utfcRChi2tk0nWnHneHI9nJ3=yw9I1n{J9I-C zw6WeEYqTbzubnQA`Ji62W2cNife1GN2<;)8jS!AaT>uZ=MNkt;NvxO8MIdh9Oua!} z{>3p%F%B{adCUE^FT5|YFRBj&OaOiX#sb5dkzB|Mq%(2{`5tAAB0*`lvV;(aJPx5j z6QSwR_-N`us5^jQ5GG8oC=Ihl25?6MlkUTq6(7S`eQB8#?UCFNrL{4&|Dh!C^J?Yu z^b0pF`6Qb-oyLDJAo>5I%>(3b2yVD6;o}RY^=(p)oL`0z9K9=Bobe99&7V65*j{`+Ol18INRaou3>EB+Xhyi$zCX?n^{x zKBusb+op%_e@}=6Q1&KxUzQ}5Bo!xtbt~_YsYpZQ%%W#C%L94w(Qeyvp}MVtt%|LZ ztr}@bG<2zQcnKyAm45Hjls9WD<}DI7d#Hj@8*;!T62JWYNDI^2mz z_CKaM-8XwN5D()RQOPVBgKi;-P!`tCt%X%fA@iY+Cg%LGR?GYqgw##R1E{L?*M6Kj z&vGGGTme`XF|IJSR$4rLrfPLf-w*#s!>!s<8Gm`t#Q`L)YHqOeO1!ob1}F%?WRdi` z2*h3Mg}WHBs3#p)lnZNtIr$O?J`Jp|6u}ObTMU=d<64Y^=!wUbxFgRi)zR#G${l+$ z(0k=$=)Lo;d|lJy6ZE3^rmXU<)=v5Rey&@eU-DrW#ZzO6omoRK=kvallNXe`^-KYs z?ZAsUJ!_*71emn=MIe{gvC^^Hu_DcTaVBGM@17?V4Ir7m{k|5z8a=7HLw6zF<6OM( z=D2r;m~;P|n!hI0p!JD>I-efSgH@5;kOy)FW{wsg+``nML34exhv}}C?m|8E)?e#Y z5yfP#DmAdDqG9;@PgbI$^cYFN&sr%~5?9Nco(4v@GxZYpGW0$!g9gEFeVM_x(8tin zFdNWBYog3*(vdO?v`cAOQy+VU$8udR;-Z9fUxw{S5cLw}eZKN51UG~-L~DoEs{z_x z(#>}mfVCiTIEyhhpg!$_HN`d6HRUyRc?ENn zGKw8#kD@>sqF$kVd>^5)(d7U2y%i8J9vJP7yhM^BpZSt8DMiDciAE)s*dvpW`Y0}x zGm4g8Jqg@j#09m5twGmdG*B8?CNvZF70H^WXsg%_`-!YUx*_+G49Hx7{7uGm6Z*mW z^zcLQWE1-0Rza&@4Y;eC1RylM&wAf(P@|rqGU2XvCBFOW9LJBI^3_~yV%0fc)Y%j>2G5Z#*EIJ!w5k4L@LREhCu=xD_ zw)!RFyr#4vQ+i#O@v@USJx+i;~IvTz*DYC(=xi@UH#i$GyF@9e-EEV9#yx(K9mDewoB<7)ysXw#aflPYl zBOshlBtw?nbwVrdwGkTUzxQ(Igxq>6-I9ZP(@D!8;@yYr_I&0p`^ICxg>r_PZMa+4 z_*wVOG&cO)6OuP{xt%*jZ`S-4N)zI-g`IP4$&Y%b$(tVtjUK0k^4J>Pmj$a|r=1E} zug#vDiXEk5ibNW27(g7N^-$UY-y0lj;;petOi1d?PBm#?bSI)ZB>&zV-E^}fZnPM3 zlAC{Tiax1nTl1BZ4r#mD?$s&0qpn~v>A$1@V3Ty@b@SmIw%2l8*NQglI5xzbKlgfA zx29>ErZ0BIR=OOgs^0D{Eq8TP@bs{5Sla8WD6kd)e>@I9uT+Y-b z2}kq@ArlsymT9!;Ba&kX%~G_F%+_HkxhA_#KNZ64q0o>-z|%MOI1?cw9kZV-`W(so ziU%$vwOb!OynOB;lS&{YtcRD~>bGl-{OcW0V{@J?ab1+XGGuvrDJoyul)hcucWlMt#{|F9kSpF!5fz6Hp0a%?ur{YPEExBM`B~dzgXl!p zO5IwwVWVi3htX4P35}YjHSmObzYNs_uFqU2pS~IkF}kieHQYLO;ja^E8=9e4@3wcn z2GYm7JSi8+9I~LdbkXAI*GGt4W6@*ANWaPJ1(Rfw?j{b32_hIvf<>=$PYFdQGaH`w zE8Kqx#=gd{XJNJdaZb3&GLrcbjd%U@RA@6`1OaWc0XEzjUDup?ZY_r(V%TFN}zES72e(q4qzK%QE>)q2fQ8mQ8mAx-$PVAv#EtfU4OW)x3 zp}Yw+BuYO9cv2#g_)uBvw5JB?Klt1H*0Z0?Q-HgYhx;$-&j7Vxk)$Ct`d2PJn~BAu zeLT$Uj!%LTHWJ-MC%uN{=r4l@8;2$7A763s!!5}jOp2}6YHT}F7s!L^xbYx&CY&@+FeiWZIA>V z9Z^wT%-^I_U36;tXxggXLU&*|=@;|X_Mfgza+cr^6)zCl-2FN#r{h~_WfyVp3+pq# zQz|i`r3Zi!JV$fZsv%hN8k7?~9h^3FbBZTwE3l@0c9J8G=(P3OIb{`H6OcEQ`*LS+ z{VwP0gA1=2+*jXeU1!mMb+f8J=^WakPjX8*x+yrN*|Z-?f-QG%yKiDfNi#gzSQLPb zcaN`sof2=Vr%f%4c>0AHBLdP{YMN@ceMCEsC(p)c{C56{*`E8J@#W+q7<-Py0Xi!8 zF?Z>8o7ajz3veJr*0HWng5)XbT2v@%N7t_#RpmDA}a~{v-6>gvVtts?<3znQH z9P7`NZ@4zPMxAbLWx9;lUDHhb1UyxfOFFXcJS#tvsod5Pi?)zsslGM}&@p<@&DW#l z8tIO`$uZZ}w%`I|pSB~b57&9s+iZTvKl9yYdUCgyd`Z=O>L7(hw*)z+@Vn;iT(SPz zYp01W*CX>w2GM}JYx4;gHyab5=A7I)nYwK^vEjOF*MO&%XNa6(IAh0lx7cx=q-Q|A z<=HC4wCMWt#L7M}{7Agh=jx~zZzU9Z*L%c+l#4jx>-4#X^cQsjAFj#t9MJ*#Y%WPe zof{sWnCbSpAG~CX+rh*iYa(sJjvSG|=pa(XOEF&-rAtO%(t%#mKdbRj^rOpr#gh=w zdJy5@rG&4o;-#Q(lIrDa-(%HFJ6{&HOZ(-Z=MWa~t-z5i-K9fTzR^I?S zwi87?OS=drqA73x=BQ8KMcjE#=y&>PU@Su(Y(=xLWuLFxhvzgiK4h!=!Su(^lebv9 zeT1hrx!i2j`kME5?oETOCZ+()NAf3LV$NI`__<@@@1(iAf;+PSpsr8Qsf$=eX_p+L zWA4tNDU@%`#3wZ8Z04?T8)s|7^+?y}3)h+F*v=j2S%4VkQ}uoQ_H(YY&tk*e{ibKh zn5yHNE=>eRps;(kr)+3$YLmvTK+DIjKD4$=cp|6^*u32%+J7E;fBNcnEXNXkd^MhP z?7#CZ9_<{^cVUDEOsw1&3dzDU{X?85`T*l6@3#^ibCK*LHoJ`0FOub7d12#G-JC z>nl`rB^P#t_23D??$`76p#p0bz_#FBOm$K%>vHG=aPaJk>nzi+VJkb>X(bmL^5}XJ zqArq+;kWHv#t2-5`0?I(UIzIksRA30T1F7PGmXb9dDT-j*FKlo-QE~epY9wA-ixnd z!)&(QSCU1TGY|%II|K>X`d@6l2{c<>{5D#zHrfi>YAdFstrjhdH>&c@4DYzcdfl}l9L?g?BDNs zp5Jr!*;^BDxh^GiP+u!2g&%29$wI%XgR9M&EBPni7fM4E|qZ?(}^ZCsp%v$#@_SxAB@C8ybD9kyV(p+@?H8fSTAB>A$Znv>pQvS6Zv?&1@7rEwdH#t-+*pPzlLcepEoLCwASm8@?DK`q6;3)cPV zA$~PDc&Gf&ANSif+e|=kX7}PKlJrLAn#`W*$_Ah)=$|ikHa%L;x zhjz{%gkI4OKdoxbu4=8WYLQp97^_-)9^t}FUsP?l(GO~pdzPz)!Yp2BN3K-Ci+u8y{${$qFUbIJ{KC~~96g~? zJo&V~VdEH3ZvN%7raaFk*vLOq9p|N=-K_Xm|6+;By_aXdh=%X!UgXY@#=3+fX1?v6 zBUAqGc73e2r5Ct=lFm3uw>bOK?IP)eR^z$wK$-tG$pzB$HYTxKds@Y{r_>8hC7T>i zwL9J`d$dpbC@=cQY`4~SzSj1-)^@Vi_Jr1Uz1H@jmZ!x<)w)yl-%iyRpQ_I}m29di zd$PyU%BT2jQc-DtU|jfOo~)w%^ECQlcz4#>W4v^u>pzXIf4Cm2T=8&hC_Q8FYC+3N z4sgk*eCDoEiTs}k??sEhIQNXPHWy__)kHO?)R|{astxHg=NF&3sj7(@4M)UO%{+5g ztr0b%Mto(j@H^d7CnB3xxRPH58P#ZJKf#+8_FWxWie2;Wq^K=eR5w&R*sqLQ)}|0l zk~|EhiYW7sW`~*n^F$wO;jo5MH;gyI+rc zpfbIw?>m=WZ3EA2>1w;L^yfWp^#&}_;5FG3W=(^76}{QqnFT|qPS3d&p}VVjBGE(D z*d9fvx-7K=kFiAlzE^&gL~rrbrR_j$S+e%v50X!Ukw!L*lDs+OU2EF+qP=jeBoDQo zD_-}aeN(aBJwm@76X5{yS@A7WpSqIM!dES++2{B{6t@bO{l*+f-_g!G~hfZD?B23*2h#HC#XMEOVNEb z*75R25!dAH%#nZHzm3@}=$ISra*NOMZd8&F!d z_7OjMajf-mWKzG#(5K<_H51+!Ix62p2X2*vU)~lq0ZqPD9>_0Gi>B@lU^K`2^v3DV zbyv%`6yH+@o#$Le4Tz3O)FUs|`RZIo4KpIM2iiCW>Cpw6HR!h)bl*O}4b4Ti#x`pf~a}9{+7nh3y?I&X&kbLl* zH1G9ZF2pZ*rV$@tmtap$B`Rp#TuT4NnXud*)$BUe*v2xyf1b&oeXG3P%)EC$uM_oA ziL<5=mzrKaN%#feJX0h8yVo4HQDp?x~#ZtgQKHaaaA%8+%`_Drb%19GlAr6H)&R8X?9`7{Hn{QK<=I zlK-{wl^azfw(doB6L!Ag4Vx#?lWP2KG2Wb(evlmXY&z=5n_s)0h+;;@YS*-kT|b<9 zke~Bgy5)pfbtnhneYyXWIV#W|KC_z86%JnUB`^Lj0TKf1(9VTrAGQ1YD&(A#&IKx54@-=1Ht0e)F1&H>ZZw__0yXT1W2R_S;8r+}s#14y>z6!nAg!_4 z-M8VOAg>Lx)MEdbbLUR{?zz|-Et2kAiMM848TP2h4P7rI||{lp++Dp@nsd6cx5 zAvM55jR5d4YP+c|mbdkL$A5(=EB3D3lTzsaBP8*R#>cQ6Ciuz8DSe!>vwk-6g7C}d zOJjO5^GeaQK~X4Fb|G?RwaFUDcnqx_}J#D$E&J!$-mp`7)|2 z9{*+XON42chm|dJ@NbRB;n!ylscr7FZR$adf-0*k8&d8QgX0PA^XWPtH!N?;Md|q7 zvw1o$dj9pfc8{$OMa9AIeUqbGv^5$ZbpZ~9x`wYa2e`ZoYbt_^T@T{@Na7~DNv_@B z(kQPfHkEq5FIV34g9N|GjkkRBcWSVV_x)JE{qE-{t#p$c`yXLdJ~T>yU+mDhRmNxO zojx>m0zq_~^Z#7aoQZHRP9-e~Cm4T0_(|V-@SzLn9qOXqrP%FiFh^j>J+>q~TPa^=*3$ZLb)c{1X}?k4Al)C+)3-u5d|s{tg~4 zOE#`b4L&Dm-OgAZD!eZ;CN607q~f(uozuJhnw=Wk^>EgJ&fD684SSW*zb{{Gi>p-e z4cNOA+dq5_;&#lo+OOUFdHAd{r^~1SdNqQlr_d;B<<(jvWAl1R+WU&9IWYNjq6`pcmM4?>FGUJJFGM;gWQmQ&FyID>tnNYkG-v)QBW2<`b0>4 zS$Na-IY2M6Hd9ANc>mGh#fe|e!BrIjuT2MDl(ltC0jnw!XWIT7-!uMIrJ^TqIM1r; zR88v#H-)|Xaz&^%{%hzUpKqs{PX&w?hRM4Lxx_g`JVY%?#m_z03Mkw`lS;|L0rc&P zslu<>fRMfjugg;ocV;P@Ni)sYZHXBXD~H}ZYr}fK%v~zJ&)W7? zlt&iXlic$igI5(c7mfT~#h{B54ELCr3UnOJp?4!yG`mLmE;?WMt^WcV<@d2E>iR3N zrMZpHS_AXwlJd`#qKU$rOVntbQu9C!Gh;WG*L+DjfsJgSqT)<6stPrY($2zfRdtos zTG^XziB6f{%MKO=V0-=yybRMesm!enG@MhmRY-}KevxTDpZ93J$jxRD(psP$nNlNo zaFFk~Ch}C=BYU|lbF`+%&TaX;<5F6w`?45xShwm`zJPFK$;IlfYUHyc6wyct4OMq} z2Zxd35~OE#iR9Pc}SyJ`4O(ZC+s-m2~Om1<3FD zpPM88cF^ZMz0o%Ih0&<5pZydYbCZti%wo^C%azJ*2$&i_wJvR`e7U~0_R0I<&w?xH zGo53yYj2Pn?CKKLTN4$k;4OPGZ%yQu?%CnTw$>}Nmu?j5+Kks;H|+U3R+aBEY4-T> zcx_?Lp>$`=@K_qk+2a(&0R+v?WfQr6k|kgO%;vGX>N zoyb+wpApQ+7i&Jx;wpr&={uv}lwzi3Umb15Mp7$vyVz2g(iYzviK7jxdibuVk)Azo zDK?BzQp|UU!_$8#I_=c+x9&P1JV!NC&XyYCOBG=WvQstBcpJb!P{7SkBA*Ql+9Vk( z!^Mko>qQD{-b38O2J^oW{U2UzZVoy%ZWO5$d97O$vko2{7*5%c)vzap^J@d1HKBVB zD(q0t2U-&o3Q9$S&tV6W_+I`~Q4zl`;#qudwfKcQ?GlYp@?z7Gc-#{IOX{J>!b3O4 z`@rdPLHen&BB+<#E#_TT%5vTERiA-wK{{&hl497xYiHWJAf3C+?PcTk-A39!U#*o9 zylZN)0SEfJ&f&vTV>%w%=puJF*Oz0z%e}O14zk*Y`8Hn0SUc47ai&u{e{OO}Nmq!O z)QBzae>_<45;rhfndTl>(!WV5lJB2YMb_GAQBfw9Uim%j7qlDixtlc{>AKwF6Kk@T_k6@c z6@0F>L?k+Cn1(7iF8e3Vxb}|M!Qs!`Ws$L$_S!0a3PS`kh$fADA?8r;yf^s22whqEBrqgTo+ya}G_!J({CUq0->|Mo1hwh(z% z@deMYO66qrht!tRwvlB~CEbWBCE)7rgkb5QUPdiY)~=O#%RF`; z;V|NWY5AH=8LyfwiLfP`M_3v+j~feGKikFJzfYEtqq*C=U5em!m+PO^crjNIx9=x z0#WPP@c4kV=OZGbm)2IkCnZ1eaJL<-FKO^EEAwM!wRwK5`fX%JAdj7J@(w&Q^4+yA zkhvPem<-?VEwJ6UYg0Ic9dJvLA1N_tK~iM)?O?6AE%;}9!wDZfn&1+a)}kJ8;zNf0 zr_}wiby6VJSV@mvu@*5D>Kin7MaHfY;WWNhS-3XUu=s}^NTPcX7Cd}|1Lppw^uX*m zK7`OnFrAVHZzP8br7E))_2tl0R6gGdwk)G<%QtW}eZMEsTXQd|JveCiSCyRqa2QW; zH6sasloU)^;w*scyeKz9HAGX*gA;e7jl#>BgJ>^QmaK=7R6;KN?K?{o&|HB7C1&N4v3ptIv0G z5njJa%vOIugFz)kCN%e7>|Jw{Qe&bR&%>|JP+c5)E}P#S~Rx ze1WLwz388o!c;x|Bt7A-n@8-P=W|xNjeq z{f5Y=)it9ai!*Cw;uTZRTYeAQ{Ubh6ubsQvbzcjKod{OkerS@|H%}5&3y2cc| zDwXg1@R#n^S=`2`bYQ4MM)8#4jvePkv--XhYZ}rKrhR?4kY2OpnF^}12SV<4gjVw9 z{IuZ>MNglu;l}1OqDy@T1-?D|U1pkHI>s4R`i3|5Q2e$T9)dK!y(bPpU-IDS@X*+c zyJI8DpzA$f|G09X#8t(@W1_m(IV)O0`DXR|auwkb4(_xIm^!`i%`sZiJ*mZi_VnNW zfU<$H>h8|+o@-r?8#E1s{tnlQ;7fx1QwFB$3u|(!$~C#CwniWQd(($unNdKO3|KDy zd8D*r()o(qGr5N`PJ<(zooNjQuVo7-t6dr>gQM<>f$Y1ieczosjlP#iGrun1Fiak< z`5jP_DLWN6Yg^OB)9A4FY|Z0f<^5sna?kQdc(ccyisf;*=4S$z;bJ<9KW}b%ozUQI z8Tg7e0RCgNy0*t>W7_4)2w+6O4tRGh{Sqt;EB4+b=b+`o*p|j#PF0xWqKr)I;4M;`oUm zjzp>Cr+)GbDupZ@XC2(d^qR8khEz5>*hLG$uDQ0XR#VmiWj}D?$I^~&C0e^R#)AIE zrg66OWzf%sPg1+)y2#1fQ;TE43Gwz5H29$3cH$=Kuzkf9pvyQ|iu-!yfl$%$jk^7F z!Sybrx}vg#_M)$K-^Ug6{V((g$_;opxjE|_v!{o9S=WbEEDC}R!zA`+x&8J_mjBgs za7HEV%=98-qFky~UQEgkz(z%SLYFtPeka=6eJL<0)9+*7trMzIauI_b08lSd$JaTd zEyso1Wp5!?z0x&BK4l{+BARqjQH!J(JznGElb^Q#u^KwEAw0-WuI#G|X z=X*n4D%ykP2#>Re&pJIG1NRoDU#ZO?)>^HkYD!g@@}3)yAB!6zbz0Q@%^Ox2xA~ip z?bS7KLRIzVlUO?^cxqG>q}?cHqSmyn1o|5;RweRl6HpoD$A_%=m)f{#y!^2;;&n#R zNA99B{VL+gEB%Hm1CJ_vgGslC9+RA;qWz=47VGC%-WbbGKe~wM7IBFaw?F2v5xlrkVthK^?7Uk`YuiEH5(#GJ8<+`wBS<|(B3RU8LWhC(N?&dJ73>FZuJuC9q zl1J8o$r(iS;_(3?+avo{fMImr((**EN#g4){#Axny@|Wtc^G4sy5wCDh|Tw|h|w6_ zu)R>v1mpj0LN(9-*GzJ) z@`1IRVbIq~6{i9@G5NZ^JqrgvL8%8rH@xop{;JNk78k0JenC-sDY{V?H}134@b!*z zWN{Bh7dd|6M1#P{+OvvR8INomVicb(4euk^zdU`um}?JU)`n@9^FA1rwmjH+RQ39D zqwD9>B}>%qspe|;?SP;fqw6v^Za#E2OBoAe!qj&2d&(ok!?k;JY{~-%N?U$$hIIr| zt}aN3EkV%U-=uOS2g2_}+E%;43>}#t^{@p|=d|PCH;2wGJeeHGUb^d6cIjhkTJz`8 z%R7uO<)RIKuB4_}zao?I=Q%}#t9grUMXxOLGTKjgd8oP$fG_@Sx2`CuhShqPw7Ewp zD7;8{E9R_3t@4rruB-{p#a0=?b{rRn^(aYD z3bsUV4BHZu4J^NERE+TJyw2+gxZMpsiLz0>R;4;}_$7buUGnRqaYk*m|B8~Mp^d%+ zvvk2GX-uULFO=+KfO+J)+bN?|x=}tXy>>k}skDUkL(?YtXmZz;E?0^nwUjDt2YLV6 zCK$Hk!w%p#CQ!~nw&=6{g4~uxjPW*sdY{4WuXcm51#t+ywd+PAvth!UCfn#l5gpF2 z-r!+(2U58SjP-#f&A{Ex_P)Tw_=C;b%_)52bb9o9P4E~O$=e<^99EQ5H=8GydRiF6 z3e#q@<~;-7tJ_NPYMpM$sGt!7_ap3sRhjkn zo0=uwgN7rUe|I8l8xD$c4T5(wtJL>?y3}|)VMhos_p%B-j5iWhwvsK);GX8hOraYA zFWOruPWY+*VQ^6tri3w^^0AS!ol^8z)n^Oc4h|qij^7v+MN=XL2<$ADM=!XJhP5;Z z8t=0!aa*)^_+i7n6tvLz*PJPH!_F7oWsYY)3RuZ@dQwh~I1q3f-QV!lq`hy}X$Wpy zqY%SvAs`JWA9h#XwMKf9)UJRedlb^R1=)h{Oxx>v97JrjbUFp?*)f=9Knb7i&xe?X zdU{{gE^Nqg9MGwD7gE* zAy4yzM7D+Rk8;8J1h<`_$BdhR^~*)k^WNLHR9&^h z4Txfg@!DLZ5!pbeY@#qbleIU^gjcGn*`^b(x=x@01~a)ka#@*j(2o_ZUqQfiMl z)rR0PAYSfnDYvVBDb~F-l|#dUyLqO*`|?4$v7BEJaCbW*i;Td?0rv$K%b$Uinf38f80BbVc}BQDkG)^%cowS5pY_(+=(kyS7h z44^`}UH==*kpSo|4}wRXf3SE_yqxIoCIxIZ8RasC zo=^xdS)%jo*2)XNxq}N4hbpZhFaid>Szsu4G|@|S%{JDTYy5;mv5+M4jN^=ELpeoj zm77~=;Ko+glhGpzP=L1gFNowcrhxD^pdNdOB%=3ZL6ngH`L1Jv34>G6z}Dva#7pd& zA?^B;FwmQ;pa@}5m>TYFKqwlJ1m)F+^1!~CXa|7aqLF97A#=N`{8M_3A9IAnunqlB zIOgvK16Pgi7LeEoK7~Igew0abX&fXSGj5a7Bws$soN#`%UESq#Y;0VdGBSclx_}W z-A{Srh-qIle^ScOnxHK4JT9UE%I!;(o#6HMgo|Mv0zxSkV=%+I77^!`^Psm4(4)RY z=malpnpMuKjNOB#?cu*SJ`gZ{ATanqK;eNv*#iOZ2LkPJc5*vgmzdeokm1{cl}#7c z!!K5gM$Fiaa?yw&v9J@FY{dl`4TGy11~MAm(i+{@G`g>9h*<_S#1g*l;nM;ia~qES z6ILt~G3^eDKqGl31ROkplCi9^*rGjX<{q?wGqKYiG;a@@z32I4g(Ck!dIhdWF9swP z9|IuefX$OoAgzVJ8qzimGLU9+|NnpL;-{D2+fN^F`v0C91{CJ---%@hW@4{qVqG$^ z!kJjBOsrZacJ4t@^qswbS-i0VuHR1BAWl(RPEGI;J>}Lz4v@g8=C{9tKa=CYS+U@( zIIv?Z*f9>g7z>V)X|M~K)Mv<ap5ki{0h+RQ{&uWo?;P5 z1Z)MFj>nvEMChWR*$Kb4z6JLcr}V$Q2znzt%FE5YX9-oi=MUwLCW<9f&UJ?JLB%q} zKw*FbK3j6ih>1Muh-jDI zI7XEF4z`!D2fYauLJBy@Ny2dfC+IG8BWl;kk3QRS0fUBQGIH8|l~GTmKk6=-D;OOUPb%<@d^~U=L4ewte_j}eYIWeuu-;SKVzK&;>%ejTf zk8<@?eUa;K1_C#cb%-;7MjdrvGv^r%p^Y;UW#!Cb4qZsE33@^dnz^C7MwPVjdG?)k z#BuisaY$AJCx`#&Q?{~GXZ4EN{FFoKh32#C;T^_hY~=aPLEe2h(MtlBBeegu|LuwY zt`S!ml*=2MNx^4|mxBR~?I;XC8uvy%2sF(##+3x+BW@otxE&;W;g~Bzq_SxzXq4Er z^dn&w$uCK`$AUM30Z5SuB%lp(UJ}WlHKW?SI(O9mDa@tn0?^S#4iwSM^OqllK90uq zKIB(~WFb@T^6$pLWTsELG=QS zRxk$4SR;tZzVZqJ*oOFr1DIJzfII8N%8Yb(vj%iqe2hmD!S9GT$9geaCnf~t7KG9@ z$f0Rl4%rKkhj@HR3OxM@=um#4B?@00^noor&ZF)5q>CqKsqa2bf0av{aLJy76(872 zU2c{qvk)SV2mtsS%gRq|aawj&jbC){^aoNu|Ng*5&|7gRpEopH!2B?tLim<~>Tuj12!iieg+pWFv!n22VWe z++|Iq9om7y4v=S}nicbJ-D4v7y0129qMm*M1Ex2_$n1Lpup60^&v>}8j~hRsBCc|R zj{^ly{yTpr^hEnJ-`2FleBzM9jTOZ5gHAPB!q!6!#GpGvQ!hs#KU92IV1Wn)I6%2# zpeF{Q$2fAJ9eEzG=ap~_OTGgNL4w}M<3iAgGn-*o{f;D~VSPi~vQo zoHbY@%2&7vtcQd;3<&^`|3n~9wjo3$TXkZ&6+V`*CB}Kf7)=JJ4&a^eR&WI345~@d zpotrLBAR$MnIcVZ2B`AGz;ST2G~UmQCA-1r?P&lh{gJJ~@n(tv5lCKx786|PRa}^G z&~c{|2SR{vno)vpC4tSS);S`8C&t?D!L7g7Gnhya0Gy$L*p{dFjjlfk6$oQ*U zoXziHbDqE~kvC>jGsvJx1IkNd>KJ?V^Y;TkZ=P7?3WEZOP%a1Pi5Mu~Ae6Vv zQ!I*jTVi9!vJ=b4IMV+jN(vN`0Uh{BZMv+^vZOD#^^tNoYQ}4ni`!MTh$YER-d*(| z_x-)`A3Jd6v5Yi0UE2~*=(5k&*pp9p{7pPEao|he`i{M$3eYZIh5Rhad;+RZWi*2( zxJ^Cb-A(eiaP#&K$nbKQ?)6w6?qXW?VPI$2ooWutVL%ZM#oQJt7lI#i;uu)wX$Fyb z@toa!VM2iD2}CvJO5NviMf_9Qba_4eYPHRd%B#R;FV(F5*WVAM==oCOpPHeHDhPox zXhyWm%by%T`eQmu6aj@@hjO8jCpRO6tA!Rh2RRC!)hdz|l@m88P{>#H|tgve%? zFeg}Myl`PFIRXsifT$6(S`y!S^Xabo`cLBhREk&<72ju%;6Ljt$JI|tMa3fn}64hL8Au# zP!#WiOArsI0w=D2R|jz<#~(u`A_t2y50s-qM~%@IkMQEjLQv&<4G7f;aUgJD6NCT! zE(Wg|q1@ts(%VxGNHS_b0#_ZL-FMdV7NYzii*tZN=#l2SpFAbJIibH0y8UR(`x+%c znDX|0T}haDQFzsu@Blx`OUHgj7G7#Dbuj&VFhJ8nlRUrYTp*5y}spF@BXy1pY6@zkhb2 z`GQ!??$(MS^h+6v#m((OwDh31APT z7|u~{oY=wJPs2E(r_8BY9w$J1kD= z0Hx}a!~fcwkp9NfT-T>n4VJav5@u9GLrIS|kGYL(<;W>|=VPs^g(NY3^dn)>#Y~|` zaM0VuJ7aNIFk>~6`|4Dow6CmMm6zO>36#-VU1j|0>Xx5q)v4_GUZs#X9;!>Dm` z6&Hb2<#*^8li&a_2kG)ThwE-`YMuAxh+E+QKrJ7wXy3{(7 z6EsbD7nSQoS)gV_!~4m(bpchD(Acp>)(7!!Rtgh1*#6fe;i>PNfTqh6`3q8N7*5?4 zCeEG@3{eABfATRNfD@YtIUKEe`BSr^L9=Xkix+NRxO&b_axeP~Xe#CkB|2jaoW2 z(X+$RBg&jg;7bTWm5YL`75UvPDK|Kf>|lZ4>HTtIbiQMUJj5PZPI zAznwsKS>GT-mUNxxNtSUC#VDyg7rFr*Wd|86@IcuMjDcZJdf(W!q)j7c5&3}3gvBR zUa_gVSlEmRmGrdI3B1AM4LxDHE3l+c?7+3b!_9lu5^|@26J{I;t$9bD_WPH-nxm(& z@ej=6UkFg~Ux)z_XrTV_5V828vMzO_hU$q?06xK{jJ^{Ga-N~o|K-s$6Uc3#Y#I!D ztX~7=rpYwMp+bK|&R50ysC}?wxbc^R*8EYcVZ}wal1UkU)dGMSF3_8HB&fI6dQQ2Lw~b>1a6snA|4}oIJWTyf z^vWFJaoM`40c_Ife|vy91rE8$$@}Si*B|Y9E^@f=3@8t+$*bBJQWM0tf;a-kN+!P+ z=deHM%~vExM(|fJ-HL?Y7Z7|9oYI!|ol_UWW(f)r5jdB4Z{7_75Eydd^3 z7#DtVR@-5(%JUG^F4ukdnF6ItuM}nnv3{~bn%nOSz)~;Rx3c+QK|b{Rt3?7za2;lA z@Mnl3+Ykf&oXvtx!@i8PCo7!OBj3T1AsxO=d!Hoem%E!4f!(*hKt}ZNPLPhT4C%dN zdgSZHbLkRv5U^YA3n}9{uI3JN{c|9Hk`ScBa`GOlm_A{`>&J&|Oz9nOrLg zOXHi*i}vzJPlZy&#zZBn9%8=s>UUy0M!+2-Z6 z;-E)pJ91t0J0URu^OXr8k}fiPZl$^ygAOzgXn#l9At>x$*^PgQ@Mo}}_MGD~Pr`y^ z53A;RdPoDCUzJjyvzwYOR$6woJ15xuS~8(Vr+z&UT$oDV0WY-7+dDf2c^;mq+*SJ? zb_W;w6}fzr-YhneEr5gHuWPz6q(@LF1}q|Yfrl}3M;(z5I8(R{} z=um@WZDceDebK3HLJo9h-^lAA+2BKZEh}n4g%CouQja(0(CqCpcUd^ULjMr9ZLOVr z(fNjOmlWc776QnrhN3#$k5R>A$D#3CA>w(`9A8Qk7peyG8-CoF8$*E}kMUG6AaZms z`kNOYC0tT0?5e7(OxH-%f(>H*)+z^%j&ZxTk$Dg~&t)OYm)I8Th1aY4s!r> z)Z;sN@;h%?P;tgfSx8v(o%M3D42;Pt5AkWBEWrj+3vonqQt;v7q=UbM=uX{VjzN+3 zvex}K|G1Cs_tbHpl#JBkIq(>tBhrXkBpDQ4X!?C?;5Bv+x^Mk^Eo`I6fxYLP8ukjg z%vX+GM}A8P+~R;FRdOAC||awPQaIc`pw zFazS$3pOPLe86PxTKD5dnnjO8Ke*}z>kakIVmn8g1KOVnte3(14sA&k>bD$kT^^dj(?CgpqaAecQ1%e#M1U1KY~2%o{1aSSV( zE0c>8K*g)dm23ewIYr2Gw8!A!km4oZPtvX8scX#UW)^VUO3G#9|@#_&=;`ddGQ`8#J5dn48Hp9tLgpU8T4ssN^N`~;?i9f zv?1vb>DV)Hm~R_fmKum~BMt-tqp&|zfbK2)S!2%8m1RU`U!_9# z6xP#rx-#hH@iSbbT+u{D;FL}SCS3eKBV$i=9q|tsdsTJq`6!d>+{m?v-~ld0Et>vE z#o(y-gm%_~8woOM+^_lz)%avrr<~oTB=d>y8It1y$B0aAz5Z6l`=!)ykO-5@)8u2Rj#{Q9`%{z|kc5(zU7^wa<%sJnoj&noM z?eqwwE4oE-B4)@xNR|LuEm^pV5c#&dp=JQS`vW{bwq(|01l1ew*p#xyf)I7|%0wonO3nN%E*oI-vOu z^xEU2gSvk$TFQH(b%v_VqSM9i*1a4=SvL;=OfxMqfpyP2Boil8PB*gqynFvnmX$1R zvm5L->YP%S&?yJccQZ8iibKYTTR--c)kwcFqV(uejfxKc*o_kf@Gm2heJd+cZNbP! zcPBb&%2W*)f0Iu+rrv;Vcuh?VS=N-0{{e6F7?!mX7HCjN8LDb?# z6PpfHaP||GvXr#c#_Xi?9lIur#|mh|JQ1Uzi`a5sA^+DhW`epD88iNr85uLrgmpa` zvrl7DGG>cQLkh3LOQCm#Qc^5()L+!)KNJV8%}@4hOEKQ0FUye#Y3z zOp-o{Ak-VO#w%(9(XTm}uf?&RzNH_``-A4R zgenexuz#KFyt^UI2y5Fvm%D+fG0MWwBPX5HSiF_?<^9R`Gy|5AZ~T2EfrwA?fb56%;FNQnZKrHM^|ynx z4WPYU2ZpOU8$bs0!rYzRJN~5m{)z)Syg>%V7yO+b@rx^BTGi9S!MBz(XfkSq!C;_# z%W*-)-gLac=jccr;=E~hn({}=JHX;V%Rje9uP;D%i`7XP>bkhrqux0whW?7s==10M zo89-IyT^A!B}JCt`FAJcmAwv{ENY{zX7}4jx?O+l^8(I4 z*2`X--LK8Vy$AhyMB)>)?Wo`oI73shzdD}0{4I0B$7eC&lS>!}W z%?CuI`r93WB?~8N1iot+l!#NlN=HjlH81ulTxHwT7=J(XSO`*GSfB-tF$*T7ejAs| z-(fl~&iKme@ov~WHeNpHhTMWbhkqgAo`wX#=)6B5kv54cJIcc{S&wfQMEc(rqL8|F zg~{?YDGd4QFvdVl3`<|E0{*b?(l9&;*Sdd$+6Vu|yh>k2yuBg0oJQ4SU;!dr3qR2~ zx$>>&8czcu%=WXCE1wQ9j4*(5#{28bhhHdfZnWvo2PD1HpJ%Esu41C@D)w+#Cnsqgq@^#?G}H@1*J!3c&! zd>V6z)pn5I6*B4Da*_J7RQrjS0*3~d9!8gkoklslNhEYKp$iY&Wkqnrv)p^^pY<7K zpHThiC>Aqvhxv5_P+_?YVQ_-$8vP7gVC_D}VevjAOp;MlHp$U!bZ5eS%>Z@=#)sqx zv%9wCv=TkqL#;C_?E1SY-+DF-yhyn^oA(&9T#%`$IAQ|1B|55^D%qHbl4t#k!pFmJ zZkZG;4^eVZLu+U=+90{HSWih)TO$#?@Qt7yO~!vdm;#^EkQm|CxTa&fWukkh@GCkt zd?yL@w<9R~){zwuW*jLG=8HOJLLW|{)nA#;Lo%V>V!*0VN_O| zpS<6P04V0Ys&&y#kD^EHSF3l}hO7SOjW*xFz!K5OicfQJa($v#Ms|T2I0E#|g79di zr~}Q6>(q5Ui0jnb9h5PDd-X-vf(GrupLflhGI3>3ric&0ks6k{M~MwNON3YOnnza7 z!BHq^E9w)?{zvT<_d+wiB86bRd^0P9_9C^e7dapJaaWjyU^0|ngWtuB4f_KG1s9jzzafL^BN+3$0*9{h^EB+((_$>a4rLT zm61betu}TDoE4TaU$c1?qz48#DHx}`O3@MP*##wxmOcu1cF@ZXl+)9&^;7b9w$}#- zfZlQ9S90n>938R{+_W#xx;WRor%Ucbi0&Qq-cetNxwd)y6Zi(GZaNNqWsGaw>h0a}yx5Wt)fEra^5%oa zuEn(@dauP<8KNy8fiqmKqB>sZnW+E-@*g=~+tn9qupbcEIlZd0HejBMkP+e>VttcQ6P613?l{ZV~1pS?R^RD zc@Cn)o+F|Eq#xD5h({Ksbo%>tB{g7dSACgDi~=RW#f{)zRxu3i?e5=UZn$0R@D$^E zKsO26R5CfVn;oZa6??gGuGX``Xb2V6czV=AQIIY4zE6|)N#aX>0ub}Ha>MClo!g^b zO5EuudzPWyzgkYGLvOFIH(Vt7YEm8CtdZk&N2CUEDf=2~730S@IGuMJ-|KcO#ca5| zJrB$xg+PUrV7oNP11%#L6g`5>z;n9Yk!f{ePTwu!^_T=@j=%3iZU0Wf4GaH}6Cl+y zmYJeG0yBy5RUa*G9`O%Jd2a&+@aYU%zq#bu#v85v-ik)|-e?xNU@JY-M*BuU4fcwK zni5jmBcFazP8;dkD>fck1XJ)2&aFI^xLRU*+!-0lyISgBgjW@OQX7x_6Qq0q@MrQq z2D7S86S~hLM=tpJF-GSe$ECq@Dy7pRZJlo)rtbU+8cV#|Sk$VHtf1vo$_s|S2(NV4 z?H?LVlSv;gt(0PQ=F5$n4tEsY%52u#iqDgX(7ni%OaqGze!l=Gg2K;Q`Qb5YF6@IuLST6qb$hjX5U}o>}5qRt9b3ZB=@g^3@|U zZWl&U**6#cZLOWP+VZRf(cIM06bj9Hr?-!ZQsqG}ax>Kl zkX!hkfz@HZRnQB0TsnZBOE3%6G#OQ!TZZeg_M#)%=8D1FqdoHRDb0Y>^-WVLJ48$r zUvxQ*SoM~6+J@p^&?E$beR4IvO0lOwWIq#NLMm3;ep&nW2g2_bt%^t0+u&s?+p>RH zb(MDxFAlG&w@p9XFD_5METR~ct$wwP$Mv@t2#cfCISk&)wxpO1g*{J+&tOG>c#^Yu z<`QnBL|EB$>c`GB-;hNlH)TPUQp^fmw=lj5=fenKoW~it`c4v^d?=Sp>z3->l31*O z9Lt!+rVP-NcTKxRmP)n}BNsry2Sz^r=3z5p4$Q?TCw1U8EYVR#RRi-mE88kog9`nL z{qTL~M$gkUNPuls$mK~i=*lA0LJ;d_lFAVy79giI3Y+kTZM_Y^%u&E@4vcpFCYA>M zv{%Nal#kk8IYRQi@7iE|GD$~V-g}SJcT2i+Q?B5R=qv-e3jqS0RPO@-+xFg#fR6ob z91{A))g^B1QrKzck+o8m>B&#Tjg7f+KafJ!3%?8VS#4XO}K!E0W0M=o|@&+bxj|)iu zB3&Z#FRC-9;ihUW8@-dCc0)hj96npwf3py71-J^Wm{=!BqPQS?SFVEYwY(?z%+4*C zzt4SHOJGz=2PE}{7KG*b=o#_GLbeHmySkt5ra{9`BK=Etj?9{gxs7z|N7O3EfByq? z9k<=}&!9q1k9>FqxbNJ^uBPAu?a$?d!C72_B|+K~r<7KAWXQZiPWyZ2xqzer>rjVMe^`X>$4OOc#}$7Ah~~ zi_(v^`44*WESVR%>B39zR&cBApU_U4JGAY_muC4snm=R}uf%$F!L%%|YxU~7MTinnyzlmWl1J?I;)FX-AEdqSi{vh^_y8_zL!YTt*8YB8B6hSZ zu|J!rmR5VzDaec-Fe%2BzoeojU!wmw?u>51z&U_iZIwhS3oa@ek8I1+ug(fgH>8HZ z_HeGe&wZ%ElpdORh^S;aowz0yWFWurcl+4t6vkgJfis@8JUpW^`r9fb5#tD+C8&$e zp$L6DkMF3@sRnp#J3ZH$wF*pX)cjcFUG*i{P*~%v77$zQ+_XE>NFx3*7u@<8@wRRB z(_qA;mcDH+v-?Jlxi7u3QcCqy5fiy?Qpl~_ZywZpxO0h$2WAnT^Q*c%dQq^qRfu?d z%4XEi?6VdHhdGugj}w+QR?3+Ky)rZ5RXMwkLzBdtK4B+N%^Z{p8p(^y9DVh+^1{Ji{E|3c$?M49BI5U@!8l zBqYm8oC3xnaM#Z1EXMSsJ^I7hOu>zbhqDiYF#bNqM$->WcYGd6m5-u>jBKkVPh#|& zydFL1hCbUh`wlr|MQcw3NgWvFR)0mm??ky&tFJb9Lm{E3{#I&tfair`uPE6b813Eb zF%I%37<*J3PcPWzZb&yp9nj6$E#96xmSMZG~GbDk_7C z=e^Qiu^?epfsOAISp{+_Y!(FFDb2CNt#6HW|CLltAvLYzFuISM61jRkYV_nrn=|l%(tPR{_ss3&SGk8fStyN z%lVujcwSM0MvVW^?M|G)WSRHtdXN6Nj$eGygL!}N+rXIYongFsfbzO;=!)7$y@`~h zj^Ll|^t|mSWVTqfua+cn*=zL2VTHcJk*+}t$}kSQT9SUSsgD8vVs{|K3w<&07&8h= z9WmoYF7K%_u5dxY&v(iVATzew#}m|g{c%Eh!Fd_66yH5n#C*DSGTh8md^!#I(#vi% z*w3T{(+Pxj`lz(NciK>%kK_x2pC?{@#g&UOjr4lhTgr&LPdh+^zc+<`{H5!}w$*b{ zO5{xrB9Ca;0Cs?@p-V;K6yz^U7PL#ticV*AR6{hF54mH5sk;Z}-d_I`WFFM=ii<6c zkANl}Z1R3`Pyy{Hz$I{xG^ENJiHZoTI;tQ5-iZvkGoN7i^E`QM--r9gJ3>+x8-myxfaCFFggVH(=!; zD{%XXjC^Fb3skh~B`Y>qOnAZR{$q|Bby>(_EIdFGJPXTfz`Bg8`<^etEZA7nH%-@ zG(R5JGZ?FZQi^@QsBoua>fJ$APRO*nf9l=ks+{KYe3oUGm%iPr+>&j*a-;mczn3%s zPCx#TMS9j>>1pn}H*Se0>2gr33P(sGOpF$#m8qm!K1~3ix?(dC}Es+$vyaqO_fN_bT__3$m0n zX@H)Wz+wBtk1WfWjLSNL6y%E^LyhtPr``)S-R>Q$?yldaizuzXZyePA2{%n_MyzPp zxT)fa{_~nsfE7Sv{B6omxXMb?#DD!P^uGTXX6-}I5f3vhdEZ%3m8 zzQ)TOl~C{}YfRN=n}fvw{Z(GZ#z6=kUss1B{7W9|8v$PAGXO8{=eO_955^Gx2=|Xm z5**PEvH}_ZmC+g1AAY=-6+UbDILzNyFJ}7Tu_o^{8a#L9@9bQ@lLkgj|EsNGP@7wI zvj)br$tyWn-q*Nn`T^I5giDdyL#@-uTyaQ)WUknKCq8g@sV~5BJV^V(ItM$xsty-i z4_Pfvhc5s~UXSv5eJ^`r%;jD} zEkFhJGliR3D^5XZe{5H@?!_fk(_HvW;fR@>d62n@6=5@XBfLqg0m_tICt7YpQ@O}& zbI=^>*c3Gg>|bbSc)$UDzAy?JB43iQ9JDMYcHji+7&Fl{fbd#0ojhqVsF*nYklQn; zXmtI#U%Z`4UF=TnN%|FYeE?IylBJ>WO4wI;$#4- za%7NQ<4?al3%Dm}H>~&iZcc~QYvu8pl7b|81O1Y8SHRtwHu)q2HvA(KmV|??yEqBG zNxn;6jIZg)k9*zhvDWa>pW?bhdm<9~Xsh_Knvs8TlC=1$_oF>(^2IxW>j2IR)DQ(C z@m7G^ZZ0Vnp?g72X4`=$B_LX+RpXQrnbR4gEe}vn;6CohSYE0|c~Vr43iIKfCZ9)0 z8z7H@$(K8$%RC9-T$6@TD>>H7Rctr(f`m-JISmT5uG*kmVAsC-b#RBD*HKey5`n&8 zjA8Q1gNX8?v<@OwmgqUMkxwhZQvp0aR1^|m8PGp-&;Zr81zxK_`)LP zpqLAr>}l-iN7?OKDX}2pZ^pVGJpr@ihiG7I{k`>!rWtNXv<>n=?N4=nZ>)pcMGo0P z<>d$=mGu}vSH5W)Wfh`Vf_9HW!h3j8dSrVC_IAh55t&#a@y?*1I7nZU(=SPN-PTnf zpciB1$palePv$GgR;_S*6~)m{r;PH&yS#?n2hCeP{CvjR$_#!S% z*FyECF<*~5azR2lV$>YWWbhogNnGse6rzv#9YHaDfhta|0@QD@Q(npQeNvACl6TWx z)1XNU5PMs7sKF#9mP8!k{}1=BXeV#@6u{_8wle1|MMIyIfS3j;nVGi+I=QL zUBAphKS|rv%5|;C)3Z(yPxbz^M`#c1#jxIjI#u|J)tt>8J~d@kxow(Dir1le z{SaqD2oC5c+>wjg$G}n%*k`<}O}>E20P;Pc+w&W%92md>A&$a3tu>{v8>_g4f5*bm z)_|-eD~Pk)3|~_h_xFAz4I~F9`$G>%rHy>rt33WY!xYPqFM52?D6hyRldYWnAA2#A zX!HZLqc#x^fa*K%;=mtZ#TLVPN2j-VUeO! zws63>m8aPlrbW5d{9ZwC^SlEYV24qBTNIZ*0Mj6mx}rhrR)3|# z(pyG>qToEp%`<03u;J|a939;G1d&6R?jy>-6q<9o)*wcCB21jYhZav9f?VVgdWxy7 z((1XdraxYdd4hmB9%9JJ>N5{l5D=reF%wTdUrIx_cF3tC7R%pz3J1c1F0$LVnFNMX z0m>&rgXSBvk0Xz;5^jWfn8O_q=i#2YOxVVrtoD4u6g7?%t$Th+>L<>Whj+J7*h_=z z;^^x&hg^k+nFW2}mnI=E0;e~1tH?!+)w|7R_x@tcQ^iX=`@_yGWYg6RPAYa5=vT<5 zMa~q(apHf7Po#fVRm^6|63{IUhO4-wekn!gaPPgIlRHaHLoOLR&`s(BPf?lD8u?cR?OMbA%Oj0bP~6CA&Sa@2`m)nMn6h#*UE_n zMPqhq_sakh(_(t%ksPj1XMWoFwEw z<&JP<1Rp5Fiq3lyl6PV$vG{b7yT^CFUX0?ZlMW zV8SW%UEI1Xw!CwqApra{=m^XTAo~9rYV-bOcI%}=KL@bFYDtPv4T-l9 z>Gw0Wd+RP0h6FpV{X77VrE+AcvD8*sD)cF%lTmPx>hRKrejyznnhm;UKq_9TKyOf!=2ajJuH z1_Nl~zrmQPxV+CJs9K&TXET*?qw{uAitOxaQK|}8-YfZF@6UvH1PO$Lz3?wd*Kb?( zRaBF|#E;$q9k8##31z0?BQMNUjHWY|Q^p|!4mITk%eJIX-wrpSx9v`V$Hsta9H9

NFoNay#ORITpI$rJLzUp;+0Gj2^nHW#yy7xYoZ7Rml z0RNdd^^_nsV8VdDvJ%J_C(W{6UcGN*>F@PWEx_0jqOj^CE3-W!{8ufqNr#&lQk_r{iD-r%d z+Za^6J<8Q)Vn*qCpmDSv`j+8NNm6h5b)j5Sh>fV1OWeJ)Zoxas-MX``H3G6JEG4W8 z3t$Iy0Ydm!#{4r8XgSzr-CQE;*0y5TN}j>E=bv_q6?99U=r6F-p!Y6hS!^lyI>et~ z1CJ~xi^{)LCQ z224F!o{BsqB60%&7vjY~BXw?*CKq@V99|Y#oYRs&wPO z49uIavazr6$UFF%k=;|sJPTYB>*3_(-UqXq>%`w!qiKHHET7tp(4uA_?sPt2CG}c1 zsgCPA(O3LKv}m~UK69|@0o{*(GNULRcZg6eg!>39F%(#0$L~jGwhku6%FatLRGum z;eElxC;QfdLnC(7GoZ3JIk{_)7ZTf*y^w=eief=#qVwFOh3nj;(dPa+Obq50jMtA$ zM4n%{${6XAK2Ynjs27`42PfF!JR7diolVuGX*iTPtK>nxWp7eCK)B-vz3*35VEbWS zFMJE}Ht{O}g{wQ-C7I{a093~tO{eSJ#$Paz+tCEP!3ZIbbDmgA_K(NJfW8^r$n%m6 z$OY_qIzu+{Zl@6OfHAi>2I&shta+~9^81^(fj^s^-c=94YRQ3JKq%pqKplrzaKng&MEuX%8Q(PBp1rJ^ zF^C4HcPZOS#$#MoPp6^wzG;B%V3$k~dsXlp`QaBgYLqc+AjAf0vGiLnD|WqSubb}m z3+RzunK_vk)2XuhaS3rj7vx@!?&>Xl_{|uQmac=z6oX}Z%&NK(`qlz;SCN)cinRmu zh|(aszmJOYsiGte@#!Dh>*kV6<*Vvyc6ut~g~tG;383{_x^Q`~lcocq2j|iD!%wrj zPW2A>%i&h~x_$usPAIuOyclI8(&eZ#la>k7gr7>~ppu2a-v==9&WoL^J}NS4J-Pj0 z;;wLrUz{`~?=^6)74_}h!~IRTo!@CiDfwFssldcYUXQ9O?u`KQx0v?0+K9A%36Ls< z{!Eo_X;O7RTzoD{;^hC$n|`zqIH<~9wF4xq3>SX_G0wO48S+Ip*Q4j$i0vJ=KPk8f zn-^4Ok_Pv3_n!#42IWp`4J3tJOLuKDA1uMo-_9|YaU1Q08C^dw*FHLpst-7q2n)I( z`4T!ETL1e^PQ_5A*nEM$R{ZIug?uh_;Z{AvUNnAtB4elA;O$|Yhk;i3?*madm5335 zzsP&1?JujKUFme{w~~=+82##7&lsHvuYdBNRA@%2${AfV$;M87t^buGjUO*_P)WZN zrcw@51Kz;!X~ZF7{L?TqE=<|+XQuR?pG=T>SOvFt1UokYWq zFQ1|KGux6_VOMT)2Kp6?G-rEpADAxRGL+Try=wyvhtZTi#aPMI`}<0J1OWo^OXriJ zX(uw?bUVNwa7Vj;363B%xi)%p?=eP=A4cBR#n%Dk-m88$n4$aOw|Uh!a`EGKuL;20 z(=S^sxm)M?vrs8*wC6A`v?2V^)>fLB?|~8Q2R@0Bvb}P1J^!LgqTkb`1N8l8xgH{z45^Ak>#1E8;95LQMivd=U1lQS=s1XZa7R191R1e zxi{Mg<)#9{{rkL~9{89rKTPNwU(7tbWJ$ZTd?fh$b84Pp>JHXhP#%KonmgwCFBW9*vADHq7&R$X_nw+M>wV9Bg`wg zO5QiwOhRC2^;zOwb9f3wo$lTWFD1)+8^?s`BJzBEGh$?xC5H?QtG#||A@?;RaYEUp za~N1qw|sDfQN96?2sH^J{_r4OTDa?WwTsih%-1Ge0EOq3pz%r}%@vUm(2sxU$<@XU zYgg{~jnK(l(&EVVh34=jDJ^Xl_uT&^4ryP0IRx`oRlqd^v|>2eq3XR7Q{HQj`S;%6 z^yEnKhpD>&T6)crIex&F|&)Gv^-g7`eSZNlHJsZ?Qb z>hC(wm$qb&6i@yg`z+~bF1Ple?O@4cLz_C=MZ|5?h{=6G4bRgiwLFTkT|O8e)Oj(3 zIF3$HLgz=>5G>JntA&H(sXNyhfJcMY^_jCzF7q6h<^FFjc`q~1XHVmn+f~R7|G{Hm z3*y`|vuQJyE*L?CTZ*uPs{)>)qymF5@hpeOvGxyzR%9TfL}Wo_61wV+>^96P(cbes#QL|$pf6y)kf z#cS_P+_hHJ8PHEZ^^C)A4m=g=Um3~_o+HUaojsGrHfkL%Q{v{ChfB{r?uK^D8+Q+! zTT;8ky9@#zP>u->RnSnln(sey3+ zS@Wf&ZIJDOk*;5RtF}QhLA%EUyo1aKD$BY;C`W$bq9AY3k8fa4|CM_$;nhli*SBlj zgo91K50xwCEd>?6UFrP0IJg0cdAAp%&jmu}v7aFmx?VM&q}JA~2dbF=7;R7d#TE#d zch79T{)(Zr8q8@cta?Su(FA{2tM*y3Re;*m3Q-=Ak(=#oVEQ6et{4MjO9+`JLod+O zTC0l?^2O<9g|rnlr1S+-BhYD(K|fQ9vEoElr3~k;>>r0Csv+00t3HY`(;0gri-d~H zv3o*;E&x@|u@7KTManjH9w>Nki(Fs0SZ`=O%%Dhl99)iA^^r`Rjs$*s@%?=PY#%N; zfc0uOQw%;(yH6&l-J)>LX;T(iectd4+R6}1_V;-*xxO1-FvbqJ_+QW50#n+uMt z6P!24j)G!VziQIO}%9hWo4GiVB1XRe6e zp$}ZsRk5jKNh9&qciQiiiWDO!wq|Q(p7P z{q1QT$CvI{ZL)@Hl-lTW4Pl@63dhd1JJJ4<%$Rj06mRAgSSndqPVB(q*yVF~ZyL5|6Ixz{I4=)pcc-WlbI(;n5VLg6X3-lV zzdvC%S0pzhHP?i0xO(Pp-wsG?1@4&-Ut8wx>skph@Lou6vo0Nc^i;s9r(jZbyML}ZLzvlIaXN(0=OVi{@ak`@FsFjWocbmE z-PDj-yh&dKTx|sNHYzUc>-*5*^jOqaNh=fE{_lM$337+$vfvH@Va5}=nnZ+q-5A&9 z(YvV=jQzhM+jmxMl^f=@Aw;;_QF!E^c$P0|KubG?e$F2yzR2(&e|IRCb+UO{d@HrG zI`@Kzo5?*_>i_vfj-UWqflnIPDhyyvKolljyhHh4grzw=>T5Gr@QG>}J53%s;*PPe z-b?L!%1*O`PdL7bfj2G+>u2q0en*D|f+dX%Y@FYf@|c)aZpc&0B`Wt;=de1W{hgEY z)>GQ|XZZP=yym7uJ;><(gLun%gC4xsv@MEhSE8ukFNMej`}Y zY(_H!gaP$uBhF?bO(-idkK-57&hHGkmS(MO ziEUhBtT4J1$7}HXnbJaMiZu_!RDJgaf66WK;8tfqP?F3`j{r=TvYpwSH>+0r<$$3p38b+oqZ7D<`YC!5HHnmK|z&-R?PLZpMe9 z*hg+$N1kw*b-yj`e*N!OmDY@x103ACklkms^JZ~pcz<>LCJ~D%e5jE6S1`Fq=ymx{@Sw0zHt{_)Z2p&;qiv$oXy0VT z?u{hp27}FIE{>-&NNQ&r9^vJTpG_6bShwPuW<*zNS~jz|E7pRk$jY%>W=)KaNac;v zhW>uM$*Tmfcw}rAH`h_ex;3!r4Po=i$GKY| z>fBua)>^Fkrx&i^xU9cLb)pTTViK(ke9LA%@EAPiBr20~FfoP!+PP5kim4(vF{`ez-Z+f2tc zdOuYl(n1Uu{CLWc)MWZe8y!@3KDrLar!U^WyDJa{N}7n1l@M1{p&x+B6G?zYtBh2_r~O^kcb{xO_?4n7%T5Hu(0 zI)jfr>DLfSzUo4YuE$5bs*J@&`~h!|cpt>5ot>{65_}6S)p|tY_2yGt zv47HI81I3tH{-iqCi@U_Xl~gyr^8o*8hm1yvlq#GFGe*rTe!;j3Jjj4j^}on&B-vj z3`P4y4NHc?%3MJ|?3myA#-+$m(oaBk_af{22GBO1TO2U*U*!pQc~Xj%^I{JVvAESl zv(XCfil_CV@P!@HjEuK~TAttnC!@a$&eRf)Orgct5bRAsR{6YZy8ti}NPX~vKCA!i z%yC8pL->_|4GH(G!kT_pRev4cxM>ariYUMu%3DWK0o?OzstZ{<4K-a6JmvQ%J~j=F7$T zSyHduV+21AlBSfcPtu)C-CXf;Z|?EG5g(WG;Eb+z)0yyrMb@zAcQjnI%M?Ksl2F+uc|9QD8-JNP-j zvsM3YUb%;$MiP;=!ZL5W9p_W5j1!RGj3(Y4qpOzJ$4kG}D$uj$)c4zM;=tyg;Y;m7 zKf*s+FphtdbH`pCfE3hZthJ-BTcY^COv#uRlEqgBcA9hey5u*~itj~X^9tFUch;TB z@jqJdlCFjz%q?mX=w4-@)zz88b%IgJ8DSt-y`@FuRLvf}axdkY!<9mpSMXgUG_Yt9 z{F=b@pZ&n%MZ?=fb!n8S3!Q&<(TxMaSqgBiUXy{{MAE){>ANb*( z%r&~`(4s^7ZrGxCI=A@{D4^SlKD1S88FwnQ#xhPVw0Uwj#rQ0jklf;d;~!eQ_Vdi! zKhHD>NEb2RiEb>ZS5GLwx6ZtSq82Zu?;c+aP2c^$E$&`hG)v$0PhcTbO`o^8sG4TA zC|<*gTDuDrwk*17z{PhbP7+?VxCZjO({C)1lUN8~uMs$2C@kv*JkEGBF(ED&xr*;jbHn|92M~d+okfpG7TRtJ;z=2u#PeI0W($Hw9A)1en*)<7+j6 zY1n8-ReW-^yDI)s^i@^-tLW>h_?&2aRnr$)jsMd@H=qXvKYKQ%01tf&-2Tg-ZC9K{ z>0D>sSjv^0{no2jguv52#1828I~4V^ zFHS&{|8=uKll4H8ijBPl_yz&k3j`X6#kchC{`U(xyJ($kb#`$eB?)+WDH3NFaf+^@ zvx~kdv1b?cQrdrZ2LMg~_mdNy{$H(C{C`>_xVORI8lir^jMbXj|G(Zb zi`VJLnx7B2#DWaD_Z>OK_(1Ii(IEpBO|;*cQg{5W`@aqN6&Nv?hxc4BFmEkOsze*K zWJWKU&&A~$gn9}rs4su49T~=z;!7z}nKhtMyOgMd5ZG0LG4WsD%Uc0d_K5{g9RE>R z#|@5aH}fnBfyi*tLc_P3G^w}n&z~>(T{TrVEaNZS^1kzaodoKBjb~A!VD6mGp@ss> zK?VPiX0fwX?0*^TPw`}L8>M9M4s`1%SO)8)>d6&ejx>mYEBxbioywk;R{~qVXD+iX+ z-O1c1Gr+o{GHu#M+LFqlUF%4yX>S(Q`9aam3DYXA)#SU^bNCVxrXCu2(Fwv?{!PwL z%(ppuWnzR3jvaw%zdHy)iEd;NemrG)vTexSe=jf+E)Um;FEe-X9s6^TUAFq>jEBOG z{&-|KFGfe8VD7nnor-c2X8dz?EuMRIagy*>r2HWW7^Ay!i@k&-p{MI2@Kv?STfS-A zxG;rEi*Pj*3jg2@HB!~FJpydUGSPUVK!7a&4&b0RAM>6Sf<%4iAEJ4ET6R;GuO+G6 z5;#O@O*Ni)atzx%%WSH_UY#;P_t43G251*gaF*tJ!KCPVYYX_)Ul6N>2v^-IdTVb7 zH@~zYYk(Z3BDmR;l;n8#D3F;iI7bm`4~Lg$F9c4S%%a6Lszv70ggN7h2j+28js|ZV zh(`$MV-Rk9`3-$aG*w_0C=&a=$g^VyW_WA z>`pM5d1IGciwYKTwNOBzB75x$izTFO zt}=M9_6uq#PnOhcayn>QbNN>TR#q4H@~hD%hcUqF>xFAq!z=MtEuH9ZJxk!MNzc`* z16u4Z24}mCAsgY;3TK!G|H^6gGoC<{3g4Jdptrxt=`A%OL!1f;obfDSr&FilD$KLa zli(CH6(79z{Tln?+ISIbXy7zSV!&v^G>2+UtlPMJqMX&4q0uRN&R(c#MyaLnBwnO8 ze4KvR^`CbXTa9H80p%P0^6jTnj8FDc8a;$FMkvZ7hlFY;*=J-+;UxiUhl-sL>98nI zqTbv#GNbYqyT-T4HP;XX;kzxturKUX4kHe-$1L}=pA*z`nI zINnvGI+AGDS@Fz$r4OkIm&uWhnPHD#I}L4w)X-zQdPVn~wZlY3C+pZnEFo1_(L5 z3NnR`LL)!xk+qdEYEl*)%pWB1eHL1B-T*Aq0`;fqpj}KmVTJvrTZh?bfGDk8+hJ48 zay&d+J>{N}1^bz)Q^5C4csEnep!xziIyz-#L7NO#@+ic-5hJerOFq>Ds&vQp@ZPNn zHar-;nJdql(YHLedMPbtNWH$mVV%gKaP97Lx?-zHX9&TYgjb{a=G&o4yP%ZH@zdLJTRk4L{S>+tnu z3GPX6UC43Bk_mAzR=xea~0I z9q(wJt6YpWN$mJ*%sF&HnT!_L#|h0~)d#V*u*!p2JJ`E}=)9Kty83{)qoM7V9Z4+~ zaaiEX>7||}QHvv|UUO>T136@eJO#G&R7Cn}&1ozTtr0RIo}y=Rci-`&K$=ram*Nf9 z1+33tbrD24K;#h>%FEpWC^1Ta)Z-C00F|LPXQyUGbK^}11{VH}e;p=F_Y;I@qN7@d z1f#FMlwF&Ox~vUo5h0tjtHf*Vb{;0XhM!I`+|CKfx|DF4R*<~@JNmVz7J)T-VTBvq zw=}drerhO-n?bhj45f0MQ5th>Bxn|9Y(N2;(yR>MzG{zHN>cSb0XDy#lS#ClYpYpRhu0L`#ppY@r?R#Ptx=*W0C=Br?kNNLx#q8@{nE7uH|gXuJBn zqm^IMwJu7g^3`KjkY<{M6Tu)2K+8QmgLT!tl!k7J<2nVq+ zv>Im*r%S7usS!VQG#9AF%N7bTahN5-H5T3J6~pN}gN%L}5?&~}K<&s5`BLUGXf1~v zqIhUZa)q9WD8%-57LJ52nPm^}%Pth%oKfiunYsN+`-q(9&%eH-3V&hX_pPZtLSXHw zhh(%;J`h~&Lj`Ub2G=rh{PGuJ9WB`CjA-|nD`e)|`paX{s5=Zu_17!8)*{P$&h9P) zd{9GsPP9?jYV;P5?R$rqwUh{BiWJ=vc<0(__hvOO$hg6<*wC&rR0 zpeZr*VsdA{h}aJkgz;Fec_Ii#(AgAGqP|Z$x1nhp#Owzg*Bcpj&!T%`A2WCl@kfGO z@Q{Uts&AZ_B#n_tz}t^C~J-G*LImcynao(t@dOFHriloN^7NVyJNYgv}?XD zVL4Em6C4}yW?qX9w&p-`RZNByve9r(m6S*2U0(5yq#-&(VKnO4G4>GTDV7(P8&fqK zUd$bSE;32yMv+9g%^}7^lL5&mXps)+JbOcP-pW>Ge0YVYVmdq5@eLSRjym;dmoZeOx`RATVLJcI> z(QYSGr?D(d@SMHGSQ6lg7Cpi$b;7?RI`_~l{uhEsH=QYm%ykw-9lD}@uL4D?#j9Qm z6*yJ%e(11DG5DlAIy4U|VdK@=XeFjbZd|AJJZm(}X+V`O;$TVuMICPRWlpqvhI}$@8q=cr&jqQj3X+I2Zp=%)s1|Ynh zf_b6&g4BCdnM*ZRk4GM`smLSbQ>@6}H3h~8SJz zJpA~KUGt%+6maVh!A~#r&hCm0j}6Acf3xpG4Y ztJ)5>-xB8Xf91FY)yI3BIVpN2+(>^+F^GHnc02Ch?2l|M5vG;>pr)--M5J#P=+1x0 z&wKk^`-I!2re{x{ge3@CYo)|(lytrOMjnVIUdc21hcxc$jkenzJHT9wmHAK*<12gse2)r+3^{PWgAA(!;|7s7&?1k>1lVoDPkpFh@C6H{eydZhV?ePXVtF$JG1}Hw}KDz;jb+cFiD!0&ttvjv=x-PqQx7eaL=qKK`25PrGJ4%u(W3p-2LqmvtrUj% zR7x_KSDv3~Z9#BV{qn}9Z!*k zPKli`p3*b_Fm`)DFy|ibj<_pL8JI_J$^Si<(e+tvtL%On`dT8FO8%E=UOcEydfBeM zb-WxU3EMHx$(+jx97f~1?M8{1yP{{iceuO#QN&?kD8)RKa!KTR{GVoo(Fy_CyPGfW z8CV_vN_G!fMWRN0Aye_RKs;e!~a=mw#759*Y`&0fxz^?POA9_<&OKFGC(nNRSP zaP#XYP>G*X4_{%ETYVi0J;BKL9=~TX2iI)73BSOP@ayAEhrkGbIphsvl|2wC*cA5b zHlg6X&A0M3T0~*0U2Y4FUv)Uzv2c7&38}sLPgQr-3i<1>kt0ko`ajK17DJoT$k(wAF|hyk}K8r%?lz&;HDlXF7Hsa#gNsHA;#bGL+4$|{L+*$0&n3t^azRC38BN#(LG+!kW)TW-U!F}q*RZ=dh)pZ9reJCC#Nyx-^jdcB^{=j-*_I6@pfw(=qp->&#= zkfk}l4resi0GalliD?E`2nEoiUp?rWoOcykqIo=5vUn}L_;;WP6W(L0yj1%rF%drZ#jet8aDN5nY~cA16U1vh2fYZ?qe}-+ zfmh&i(VNVhaMS=Q+ZHx-Ks|UKYYsauJ_T428rq|%SJ>EQ*edCWG`;zV&X1|^+Uuyc zdDVO%mNLW2dGvadeaQqumJgJepIa0B#WL@Dtf11REUts>go5$Z-l=$OM@|VUUw5wE zD?xd>;kLUwz@0T4JCFTfFat>alQhv?^cNwN^~rWs>{%8sC0?gte|LUm0Qys%^wxAe z0G?3CJrkJKb-D+00Z_on&G6q95Em6EZZCe-^SZk%(cN+HcW`Y`8@E^XAncnK40v<`UfVOimvxks95M)H44Um~6pVQ5#_YZ0el z+A@lB{UQh}PY?{u&9x}yrp@!Sb}}w!*|#5RAA{qURty z8SC7v4`ppva^Qk#!))u%)PUmI1A8L^XbSYU>W~a6glMG0+=#q8#VQhOB(36k-5p)4 z_BpNCw%Tjx@^;mwPHpI-+gAx>=S=kvDC-MyN!47QpRgHnqLpezD<%OSaiDy#LEq$< z!shLNh);5ZFJ+Re3lAKaq_!m!_Z0uRE@)&MY)ptcEY*A=XEy^PX!Bv4h^?ezNvzMY zWFqr38^h`&)xlOa&k$6OK6v|PlqLvG~JzQ z0Lm-BApCWhsPt8GFz>@dN9x3ppiwcw8BP71rS+=j3ZRu)vm~1LpUOha-cE=53u1k4 zJ+$O5#QMlXCm0jlJs)7g>{F6>4DL@EJ(MkOZH$ie9fat60^DWEd-v~!Zwt0`PI z-s`ukEw#l|&A(_uopV*kK+kbgL*F1sb=P2ai~2LAkmZJOnfW93WJ6$AyQ5aCXuRVU zxFoZfaD&8hI&P=WL!s%9@Z7o7N7$iTn8D`}9eBtk5!fT|x`|2^yV~imb&dd7ew^O zn-hg zNR|L)knO!l4Qy!NJ-Bi?Tx7p(x?64+vtI;~Q zW|AFDq$WhmX2R<^c3$`(o~`o&!xHT|w;0;l@U--Fl6niY=&5B;!1LU@NtORlcf9A3 zjfvM@V;(dNbX5H=GO7Tq$`&_`t5Re=loieSHtmtbKqzZBY%c7jSHDy*DBR+m&j@h5 z4P9tBz1#W8B1Vgep)5eLO7#k>nVaECAUMMyaT8rSU zx-!*rf4-{b4AO+TAt)z0VJDzn|3&lM;*U4M{DPFq>t#GU(x}2jQI8tCMJ5n_b<1`L zYUCO1pIFX2zGHZrII|w%xA`se%xr+tmw`(^iLS~X`pCEEF@2KT|2VoIpN%#Jro`YU zF2$!UIm}m`Iszb$8shle;Cm~zrtvW4FrVOhPPfM@b5Vmm#%^2dE+V_0^Pa4F@LGCA z@GEfRP7hUEnsW@`n<_~@miGoA>5uC4Eq8WpL(h=)iUe~I3?$UqU=m-d}3t6?E9l=-ju zKpZ6`z8We+{aFq)cP+|fiD>lB)17*BlO30qRM9QDZ(u2GKeFv_=Q5`iHNJRM8NJZ6 zwXX3DSaE{edHRaZh_Od`Fc6|Zpp=YevNIdrh`|Q~`PtVkj;YPpP^isP-ew)}#3&?g z5lpJCe%2K=9AcaB$jn+3dF5-E*>5y0bZ1TvCe~T5FnlJlTW~MBOX*oY;E^DGl#J>F zNKC&0I<&W}f^2W!IcCVdZ844FwY(xC7b6@sk#?@uvu4f(-jb8GcOwgT&Eq-$99ubU z$o_L;eeG%;JNsPn>60rtT2d1xU;op+%bC?mszySv?^=CPyRaBbxYa+KWgWm_9D3YlLo*9yEm$s;#ys{aHIMd%k?e5m3x-g}i8@1a&YvNxP7Z zNx-y*5dMmaFQUueB?yZ5h`)fWzqC9bJup)FZ}wd8(_cM}?}yr^=Vs!?T$<3(NBeGwn-Z@+X`QVNXuo1s;N&E=4Z?$emdAQJ%Y9YbIX7n zc4}z^>b98&=Jfy0{@JL;9 z0C}}^z^h%s>GB(OiUa%)X-YeO=yw&bk|7M-PqMPdL|4%I0YFbBl#y$nhtH8rRQGAGpV+ z0To+w{{8l3pAXZC|ENH5>w{TgB^!t{hV$SY2&Zd!*}U)Y$~kwiIqcP=IP)^3O*JNl zNOv|AD}1!@65IjS5v%3dpV5#@9#3APWZukvhyLFZtbZ?2CpA3-u^F*-&&U0|H-Sb+ z8*VOAlG*#|+IW=b!g~R5TL<^gor!+BkhXpJ+MTY!>&5hCMW{dQLVerZkoKJ~eW4#g znoK-+dOi-(%(!)j>-rA(@j=?yInb6+?_7H!oANm}D|CpUYOp?Un=G=(^P|m1Ts_I! zJl*sHcVszxb@P6a5lfK!qGIH{E=~CGk0RcuW0yzM{5^-!wIRPE(Q6BgGYz8TUG?Ll1Onsjoy(&KK-GA53~dKHLO#5Gr8|M;!=5$Q<(zkGVBa`q zA+sc^lJb+SDc*e5i326xFYq4fR%Nr{hG8KFmJ?6QA`dMRY1LB8JFOrm;nLH-b*Q9r z1Z`oHe2* z(!6(T9k~fwdY#?$^KqrKs4%g=R&#X+`I$?^17d0fad92efHsa}Yek*S^oMIJK&J3~ z^F+*_CsM_UrXNVX@FO9y&oOf7d(O|lPD7_VBi0Mumn?Y$XY^?6z5+Do(2GP0D%_*^ zTG(sd70gcj%STcLJootc@+%1WS8#@pe^|m+q`P~tw#z;?uF7@tg$uZ@UNZknsvy3v zC2*r%B#+MM$NE?zvJe|A_otvVSrzmGthl0_FV&0?52QV}X8viqbWQFCnr1-KnByya z>@E^R&kCm6VG4O`kIhqWFI+>@d?7=f&YR5H3QxLQy=Y)Cc?2bs5_OJs`c}?!h1a8hi*=fr-fyf}%ahl*x2H^N%t+A2{vZ zunus~Wp=5a_)|G{_J*0P!+~mW==i>q%WADa!H5{f1+%Whx0F-1%fALg>pnF5N$znQ5#?c2a! zqp29DGN*+#)wWs0NVf}7s_unZ>!7CQe?GvrlTu}C&ZOCIO}c*#zqx^U>aoV|csn5! zd8=Vx*wQYKul)NB!#$uz>O?73Z!0f5uHyZU+f6~}4g)@!(bifh3$rNoI%@QD!s?wD zj*nv$ag0!3UKm#+Fd9K=)6NAR!#U5s6HB;%XWaIkv0+?kxC7-IAm$=DUQJ~E4JcoK^6FeXhx*^eQq~dsK3QfZE@zhV zvKwgOgkq+Kn^X&*ULb)WFM_Nm-J?EPwsfU2ZB$nwH(zgB#nkLz_qz%kLI)smk(R-Y zrIGsQnW3{8@nbR*Q$I+2;xglcNvz{Y*eWbtM7&GSQ1I7qf;|_UN3=v>^hbd8JkGD0 zTlY*`QMpLXK?G~|#S!|=fvI0okVhE29cAa7?_8MLf1On#e1y?;LMV7S1nb?A;Rn=# zLMr{}yHlQoQD7tnyj01|=E#AMF92qLxzMo8e#5dmZyY*Va<(##i*(% z#mwz8plG`W;#Cbt%3Jz~e!vw=NUq0L>&MTLvSY}zHKy?-f=Au&%OXJgccI8@^3gBR zn@522WQc*tOz^p>b}01I_F#AB6PJcI@m}vj_t_#vbFdi+7Ue7??&uJF{%xVLxXw}@ z`EjWsvwJ!D(hPlG;siH(WL8rLxr~=*96=)HBo}5K6~i+qDmROiRdWIB<0sYCa8~n9 zp~0SGMXPWnwDScZrH6Xh@-NY8Wt4vN00aEPf_8S1=fUP-n*_w-;Yd`_hY_x;brN-@S|(t0kK|k8P59 zjPUQ@*@v!G_DoCOdlQY<;a@kVanVnYRJ*B*y}~I+x%4B@pW>a$<9D}3_4u=3@s8(It>$2W^i_1*O9y%&Kv zx4{ovMeTGapfNzr--d&Gl-#=-Wyk03)!;m!2M=3(f1g`{x$M7<`yDrdcyzsAs2Rfd)8Uzed?rXms0j~}N8_OM1Bdqz4O5~Fjgu;+@!R~4PgLLcLOq@FW z4_AKd_1ot!&dx>!qYs~;$6tB3?~qjoH*KF%%3-GLZntocG7yeq zF%SFJrw!82A8$$kjVgL43rpO_dcyt_DbLc$RVAQ4J_xde?VC_595ZVymBTfCk`&lyB;qfMpsZ+S{Loi# z0?}$d^gAGrQ>kZgR@=F^Jjdc+tq_^BD<8bqT%9rSd}W^nUI8sDdhb-go}detoWxK6 z4NlnH@*aNekVJLW0Nl>GUdr^ltC}Gt)3l{$d8)w-yOQzX*-$g{>RaGgw}ESTkCru_ z(|Pl>yQ0+`jCwZDYF-?rN5K5)^niesYud*PS#iXX zM*pCa-_L<$!<$6g{G}h zHY+WJ^0PXqo6q0M$e4iJKx1kQlxBiphlW(``F%>-NIktL12gP#ePG4xGqM5`hnYuE zWw1S1E{}_SgDkl%xaq5TeF5OHkiGkUkU}OPnJJ%^33ck?jY{PZ*D2Mp`>$~6>}XH! z6&MtE?$+PH9FhtvRY&7Ti~F0esdj!C7UR&9?i!i?fcIy0^U*4xpm^z+pIBVWgs^tuRbo$y_qzh0li3Wbw_p z?gLza#*4*#;FuqGGHL4*gJkt+em>BLkyglEGgBWQjt1FRB|K2conDM86WImhr#U$Z?Hc?Ac%TH8Z@xFr!9NE87awPOimk zwH?pR{(@+plNruN$dj+42Y~#!+!go(Zfan=Q_k))6oUc^MB{GO9y!VtJ6XVDnHyJ@ z@Rh3TAvb##zUSm`?>tV(Dtx4M3V8OG%yoFTSgd>gV>WQFeE?$d@#J_%@L5r;tcATj z*$&vbVhDJwAS9XhhJkd4?qSc@vk`&hDJ}Q1qfdlT{tmd3>)M8{QH|2Sx+7q5GR`}i z0JQGy

>WR|Ui8WkqxHlsD(sDc)=BV}t@3m0 z!zcEOK_kL5$j?RMD&D(4WHjybKDE+(z6|shr1g zX_pLSLiJ=&32*x85_7A-sKinR5<*MYF><=01I(j=>Hui7QzYe{syBC&bf9G4eW^TAL+n6Cli3E@KQV-t`vJXnC-wNR zyy5tRfpw|a)sWcH#c%yFS?j9Dp~u(fY5x~Z>;NGYWpM>n5~2h9l@iX5zcx9I*~R~> zYnHdFqyz5Alb)hkkQ#b4i^MWeu-zGW+*VnEA@2S!&KM_Ex5>akqvng4UC1l-tKQbP zw&BDi`~OhH%GiwDCt~%kD6aw41o3Xs&2qv;T&BJg=Cc6U*PE9utMT+J=p=a&P)(5f zvNd$j7uH~!c@?g*(K4WV4tu8l{{P0v>cgqZhzSE>p)$4MzfULr8~p|0Nm5?}py?y# zb|>&{4m-FCJti!y600V0t6u}2z0j{S7?=Ofi;(I>IM-zbN%J>V{!r*R`KD=JU(wt)H!cWGMu!deD1N?U- z-fPX*2mHh4Q4swS=qrX+`1)V?WHI#^DH4CU(eT)BVH-QjA}5UeNq=&I6ob>+exGv zjP%6FFbbo9O_fcNmaSvF@U|-;e&bh9gR=BS$Y59aPe4w5oCRBnTGh`#;iO%MxPsZj z_OyP}u&|r5K61!uSsFK5l0?GY-&~oo9aT1dHt)(1wOB$0GIg11kETQizj|&%_+2)+ zd%7%SG*n%zGtE2JDbi?*+xjcPs=IuQ}wFwr$drXapNTw^YI$ z$~jo8D#N`17QQ0N^X!KmNXa7t&SC;s80QDKKrcs(9`ab=ZZrktp+h_Gkwp*Yee7Mm zUVO-cF-Ss9=N^3Ry$+{yWy`VOcH7xhpL8=wWCCyT;L%O^m!PJ_oeZ%+vnj7%87(3k zf+LsJlE#8BN)OYa>i7CB1oi`R1sPah;xj4h;Tr?i7;MIbt5^jb0|Bg0p4_|v73mv4 zI}cd5xdi@ncR$0q+HQy0=)vB2fu9UD!+%csNAyIRcaxBoz2EW`^G@<2m`0J9b{+r5 zg@hk{FJJ~?5tJ!U0}s0gCV{6Hs?^2#7M_F4zp(UO4o(=v2IBm$e;Ad^pRk4tRr2E9 zu&1~7{(6Ze>4_a$WOY&azM`Mr%elugBd<-WS;7yoj#Vu;kEAbW?f}fvAJCLqqjK?S zu&5(liZ8-47#E7St{Bw_uPv#KVNZ2UPy}ANnBiL&{$rb$Al}+HB^lHt(b=ohSBzHy z1;}dK8_wA>@1AYz$ANXY+01V&{eCX5bXq#GqMuy>V(&l}w*7IANHnJAsh~7iU?hrs zM2~}!1$@fc^zw50qu}~}i3UrP6wdTXs@dpj!#o-?D^Ow8lA;>-VCNwvm~`16aKdb^ zybQ7U+or3T4?q)y5e8p!v+*;BBw%Ed|8;8cV0FtosVYk3&z0P*dIfz^ttW_exAZ-? z_Z96`>D#FWLvCbv?!-rfa9ybqKfV$8`=(|~Qo%Uetsp%eFybFXJTqW^Tn*yxx2D1X z<-s;1@5M7?fzhSjnJ*>a`ii(Wbgu{lUClD*f};4_tdrEu*+DUsx+!lEYcGpiKINTq zvNuiI$p+<7M7Q#R^~??ERu>D2r(vV}f$omVWR72l<8P{T`@Mu%!(d!T0xj)3r!4r; zp;aJuj2r%Qq@im_813}Vx-ye+L5fKmHsM>f8Gds-01C zwGlVEWY?Vl?_}bQ*|GK2*N^Gv%@P~TEnz4f9((tMj2{1nsajBj**DmgVPeT9JYd>7 zJZm?%?=5T!+WXIFSt~sK`Bd7^D=?xyReZR;CZ^W-R@oo`dtFx=_q|dUp0P z?Upko(yN=%5dI@@{NUTC&%nM0tc@&(Qt67ONmuQSUcPCAX7Fj|)8U4}s8qSa2B|J6 ztTa${*E1AZWGK`pf={^KrhZS%7ch6Dm~r32r~}1z>|FMkPYu!GP4u8+>JzE^%rdP= zOf!<`cTKzymBkoaboY;h8ckO~lUKYS4wfg`Ui<|p*IUq*%3H70!n@mZutUG7OqI&2 zJfS+&$K5vvdj&-F^@5rWhd{eVahK=g{s3lpzzV9LJIuAa;5db<(ctL5p?>t z&srrI08hYL1^RY>nrcR`Z3F&>@8X4zPCXlt$MM6UQM3*AW&vSQ>jan>s!<*GG_nd) z?2?{n<5UvGLXADyoAq{-TtUqLWf0DQK74k=M>0BnSq$QMmg!%XSWHtmmHJXRB3`{5 zY#Jlhu|>OnVX$&9@$yBgqTddh!l%%GA2hsv4oR0qOmSt4zu92TpR`rf*>oKqZXm&ms z<_3*&Jp`pVwfvG+gW1EM(P6qU!2G17Q@MH%ra^=gcuy$l+nMgj7M%vD!#Y9g1qMx+ zsKif{7SQhP|aa^79E^y{=^px%zt0zARxA`iE&Fz11~?^IX?2rR+1J@FX=K z8ZR9lBL|juvt#%8B2>kl!w>tV;9!GlZY9Kp%)nmNPT>MMm?$WN=ezW=!6;K!ha-_q zkwXuspm*Lx5g@OB1Yrms%0H!>&?DNCFJx}dYP8p1{2y&aT`3vhqsGILWNJPjhX|P0 zpzrf;^-PQcK=m7u9?U1dC3dt1QUjNX7MxSe#l$$T$cm>9&r<&aZw+Bu8;FuTbXav> znVIxV^TLZG?w0V@2FaqMWk6omHjlONC!gc86elz|k04u%dj+bj%QU0r=Y4;?Lu5W+L0+no9J_|li1yg(NxFf-$)di=EP|+Xgc9 z*FtsT8@Y5$&`xF?scrbEDP{y>vNBsfyH0+#mttv>J?7O^^_r=rtmVi&Dd-Cw#;N=_ zk;230MP>UrFQ2@z0Zj$gDnU*Xg>K=f-%a@6wj#?uAPs#JMvQ=VYLlOSTrx@;I9uAb zbwlmm>H*i|&R9?p5bliv%CXU8dt%MyE$j3a_R5UcN%*`^`22j5K!oZA%C0J(3N0!I>(x&!X(^e?(p1ztQaIr7u%2Nd&mojgpHET z#x1a$)xd9H^d&TtL)g){+iwdT>Le=RvsL&LJo^kX<1!>EY-I=+_&gIf+r> z+G*I{QF0#34_jC8_`sA$=lf1Cv2_`ngEp|7HTTxO6mgM8P4v~cRN|7u%8+Q(d$49( zSYoS?oXj|S)F*h7{zu`{sq@DMWq5F7$Wr&g6X;c~DT|nK+TB>*#pp3V=8G`Oxd7*V zJ#()+%p?$FVq4}awb1nr22}A~{|rWqM*PmKARqNF$pQ9djZhJJNoJfnCmC297tqQ7G0((kT zc^L_C{I|DyzwPf#!%QX1cejeSdA1ql>?tDgBH+EN2AX+R~1C<(L8j; zWv0imiWD_PGmG)b3v4rsp|muru45*~$>5i;)|grPhB5fp5$_JUFR4C5GleqfG##Sg zar(4_AeNaB$6S}U_Ec`Ll%>#WJ|QR5@QuUuU$S-Amd^1WKlz~X42ka1`n_Dt;H8Oy z%jtB)5zJ5NLX!M#H>sCnDr`-2zGnk*+FpJ!4BJX)(9VQw=gXWdcbcU4dfs{36}Zm7 z0Y@>mv*C!~sh^I5M%cSEBo9juM&XDgk<{OoJp;l}j1-3sSa=kxqX3IYEiM0Chm zS4yYem7*ZFi84};g^OHf;DParC6DLCS!|yz-(n1bc~QbPZ}I6ksxW{VRw(mcTEZ8R zMyzCJBzgZMsj|5e|LrC850rCTbolc%BGj8;qBwDxRnhiW<{l!gSRDa8P5DAF{fp$0 zY#Vmx#&8K&%GD6SzCyj<0zG7LrJ~K)D?x>ORY*e|>0}JXcEIjwAKch-v+TlKv?)E6 z{nvUAlGr;^)^NcqZ%LYyv>DQ9tOldT-2cC)z>-$I3%URABWW8E8g(|Rx+y~MD7A(n27A z&)mLEb-Ktb#jkPH@gI`PhVQ|?j8Ixf%yor%v4)rm_enLwgOT4Jo^4jm7fA~iw3SQ& z1?jqJ9EH&a$FZ7?P=sm5(!7lANmxrgswKn|;hAO{$m_i^I}veHS5*s+zJT(Ql6=OhSLH{VG#`Mb*%7(5{j-z@4R$V|1Tq%vWi2hL{v?g{U@m61-EfV7x`AS9n~$q7}`=0Yt2$j4>OZV z=Qgpjl~D<Iu&*7zPC3Ps*P7 ztg+pNfV@nkMUjU|(k(D%u}0#)7sq*&X>L*gQ>q8lEleD5L23rPrTo*m_ z^Nd-6mhf$9;Zh_iS^w^0HQaEvFp{OYyyqW@#mxS_GIG&kXO0*g>oaCSn&?~dPKmzI zI>KUZHC3QMpG8EPfrfl@;JOreCJ`@F7chr9%0| zCX(SS)NUGMOX`L$0q2T{AkffM9(^AQTxmXhi&As^whFoh_ueS7jLps!nVa;bL5+rG zMJ9}fq5H5a47rh^ zw(KsuAc1y~3oY}sH~?A)jEXrJj5i3*xt5b@{vgKhl-*<7cl#SR?|-R2OiXuRfE9-V zE_z5}8Saz=^N~FcEJxKX-!?d557v(B{>0w_HZTt(IX@&hEai=dKzTROGJ0bpei5pA zf5m=nO&&WnqCSt29|AZk->P(&I~DUT{r0n8Aj+HTFxO?11+h4OsPfx2IL%#lm)t$( zwXefiN70)6lM_1sMhlS|^us5<-Nv(`9}0nWsbJBH$nMex$y9Eto*f;N#*pX>8xA@+ z(6)2L#s7nR;fLO2a=2$F8X*M5+@SwhwS z!yB!1xQeO(ws;X&r0W<1OOQY!zo8#;@3C{xqnO;t9A}gDIEC-qSxEf>T9()+X1u4L zLyy?94v-icoiH?7ZSoH~ltT~7caVVRf1pnNo0>g+>~Ts|%$KZd?VN9NgND7=Asn@V zi3B=k&rWV)!ZvPwxRl;6_xvqTf5%8vKC#{p?Q2upaB| z3d|4d+s5jg=7xTvRl`!gK}W<8c}fngOXIs#`GXjn_^lfD{SO6ps95-s4P!YH)pt`; zhBkVP7@MyGauE6YV-hW!E_q^xH9IUC9GS)@Cw@DaM~|;ZQBO_!7Hs*1aY*{cF@=NM zdN$9RCr&40+D}Ix{Ha@O(fM16pz$zA<{ZX>#kZdB zKCE3bykG}dG{FRW`8((~LuX2f$hdcJ=ES0{)d;&T9<-q08U}Am>~nl zEX#<}t^pu63#GJF%%bIQI&n{To=DO^X50b=dB}?^{iL^^heo*W{he8~=lLQ)bLvk1 z)~V)&)WdVv`vKct0a4qy4`1FqF3}1)tCb46cV%m?W5>+7!1L%QlIs-s79ii}(1jO_ z9X>&yR;|mT(NA(e%!UMl`Jrc*fC%(X+o4;V_IS)K%tXcVUY#HMbcRTX2)0087Q%nc zlXDdc04DV}xKsBXbk{<}3tyi@A}W<1Eq%_y>P}prB3w&v{x$KnL8Bgp6-|fJA_0L= z@^2!bgs(j@4AX)d<5P?Okp%cpKRC*Trm+nYo`++qm|rt^g2%r){}i?ZJH!uI7I0-1GyTwc{N zFM`r9a({61cgJ*Hahxfsbjln7?B1O;LA>5IzIwLqlK7Goo7DG)rUp$|+XH4? z;*8MPh`e;~lGxfJ_YLoKZr1l-Qe`wrQ-b z3!#M#f)nMm*~^XU5BT9uHbcdcV_W{dG}n(PAI08#OsPHbbAmMi>3*4-7O|XlUu|@Wf-%hxn~&R$bHZ?^5eP(ZcfxRSHWZ z;%an~7(QVQ5D$AzXoPo06}q7o*0%Hsz|$oqx}O1UCSvB?Sta)2Nq8)=@Di4nylT8M zyZl>u)U8Z2c*$S)%g>RC{gZJ?wvNI(W`}fQrB!Es-iyF6b@m+?g}F7}NVy{WI?LdT zHg(PgYcCDFw8QtOWdNG|D+Alk+`?ApvcE4??7aJ6+m6J&u;hCc7;C}PRp z=Ikm4gxgS0juds%J0TY#KhIVypFPJTN)x2vx5P|V=wXfe%dF+qG0e`Q@D1#NY|~YX zYs4)hkWt+3`BQEwp|>f_F`mzQ=;gF4ziHfm7JxLyh}O21DYh3Kv~nBu+ZQU(;9zu~ zvlBPg1$If|NO$Px)20k__%^+H=Alae3eaH3_CNXZgH*D>lo52EJtLELLH4mc4k+ zD8uyPi`1$s*C{Z3fR`28cZ-G8(l(WPw0rvI}5u`PfGVAM>VhEj%v^O_Vr>CK4Zb_ za7y)GTWh!Nu)f+8Ez{gUri2^oETayNi}$GCRxnWoc~dYXC&N$dJ zd?7ODRYu64T_zR43bV@AM!ch>En+z`BJ6Rw^?Ok&jTyODwW{c~RS{?B-AXevr8B7fUCWnrM8Q$e!T>9in%KmoWP%;b$o!UY%z7hwi)sNH9r= zO;VQU8qsb;L=r)~834CCHORKy8|=`?$2sDbXRl<~&_Mk*PvZBHeUMc%N6UF44UFzI z)tPSE`xBP3MtrgX-UNRXmGcR7qd3-H2u5C^MgOZ^os=9jx)D7*x^2m$6V+7q=gbIRq<~<-^uJvKTq|||JBshor3gql5Qkzp#w~UoNJ6EWy^=`| z=os|Xq3E_Bloz!Y7ctd_(d%1*`MAZr4ac5mJ9 z$Us2pn!JN;jpyAb63tbMw{Z{fRq&;o3_Sr5(zaNsSS4|e?avFzP41P3>&oWaBk1*i zs=3_{c~UcsooK`D>)K`t5-{+037@lzWbEKIzI1!U~XI@_lEAeDS5e(aVWTleJMRGx^w}ZO&sEg9)FYZD-Z5xCj&72>@ zoy_mII-czLo0r9SzV#K4;$$+q7y9W=+Le{>m@L0KrFj|XKKt0}jC~auwGDGNg|eU2 z83X-LjYS#NT$@JWE8mAmLKaqb;bj_$96MR``jEkV#euIwN-rW8o}v8b_llC~5n-`?m>hQRHC8nQtj_zEj6SsdapwWD z;o3Xjb{AE?vkNbvs^f<(4?`lmnA|UmiUiE<3^H`3`(#1$)jH6Qy0V^Y^g4AJoBJ|J zv5Hsgy}I98I#_c1r7VqWR349W?JU1Gle7r_Yi`eYQ!W(Mfokv~&c_S~p!G2s)<$W7L@ajrC&%8tG8b_8?% zF>a>AYG?T%^RzB~1i!foI(=yYH0}7`Cx(-WOT*$8@VcPl9;C~jd=zWJ!Poy_}G;B-7c)|o{f74dF9%#GxSuljs7?;NM-MyBadPxY8C2!~* zt|yWD9SxXLL@y&$kiV(mxCf-)Xj1*(T*B3!I5S6EY5mkQvZ2oXPb*|L zd&=ssbACa1<}Mw^A6mF9tHhDHN1dma!%-F6w-lY4Ff5b+7w@aG$bZ^)@RLr*tBe3~ z&Y{kiiD8^2|38zuf)_N(xJCQDsMl&!?u#$jIiUse;@7#qSKV>_Uf7%}Y= zH*2;h->xIx_3pR#1@ClPDaDcP0o2p&9^YrD^d{q0-{fe4XZ1upfBEN&;0t<7`;zUW zb3+$@{IoemG1+jY;q0`8;`j}N0RNe6eG`;lxE?C(wBkPIpfDAGqFyInd4CLUGW`R3 zradKcx^t5w-DcO8FPkrPmt7oP>J;HF=+(4YrZPb?jcbLaiccMKBl4b>pc(d28dEs8 z`Y!k3QC7b3DxQ>T>6wMwpcC^>9CJPT<3Wjp#B-5BN$aQz;xiV&37guS z{kbyy?0|?|>;k?T$RqqS`!?d-zQcX9)nksfl5R>rL~Ti5?I!c)s(3K(=!pKsmG1CA zVL5w<4W`q)2mDyascK64f5WR&y&fLM!_YpPs7%Iur{MRdMH>d@T@kS*5!B$H@k1&P zkt4J}W^N#i&r=*E{xd^L^>Q^!7C+)h{@g9MCEM)U8vCI;z*rL%4@(sfs0txO3&k|% zHwLkKWf&VQ)lq6}{1vmkFv>3_I=bKu^L4>}DU(*EC66#d=E^cbfjEtX4Ca0rh-s+? z#>y^Xs^Vd$La?6nDGv5R|DH?;TL;}^CnXv4(}P$?=2uqcr)UL14^4q|ev6+)C^JBD zXVP!YKR{zA+VA&C=^g>me(Z(l!j%uv4!o;FseW|P{7G$PUmJ{jPf_d13;qYN3 zQH&JhbQ&gR@bA0YYS!C?IoCMMMNG?CK3(0bnc6_wg9@i zhB3QO2x;MTU~3`|3V_GXt6K);vazk4l%DbnzW4q}3q2(fVQJA;1(>|QZKt&PTWI8F zG_cP(be{+$s~KP*Gt#4gPF7pE^u(0NY;X9QYONH0!uRLX+W;tU(4r_eOLtu1Hju^N zbYA*hs%l<5N{~7Ta`=P*Nsfy7b`V5iLKB2jnBagK&{-edzpp@M##g<^T}v~8kaEt{ z#KMun``n5%?V8__hk9V4dW*yTb^b!9C*fjmNBwp_sWXz6wP*uHXzHO4SMK@Ng#fp zsPb?_vwI@>#-{5$HkeOYrkhvaWQJ%Qf;a;hOZTtCJh$o`;Imf)D&T4z{YjW9Y&uRX zOE*KG>@h8F!3}i3Wm_z78#rgV`mQ0z>;SqXzF|fY&nskuSwg9KJE1_7l^8rx+Ho^& zk#O<&)DH09d7esz##inSV#$h?hzdEte>bt~j*lpZc@}=Fr~$K~eny!o(TBhaUJ)I5 z;GKxU(rC+Z%z7p(JMOldZlQT!f7R7#qI|stLRp91@M$X`g)0s-Iowk{9|F}QdM9G{ zN`TAByy~eTAf|9Vu8+Mf1fJA^ImCnSx$`tJo`cG~-~M#OvUYd9BFlk$pV#ow#Z_B5 zUKW_XBQ&OlB2vV_zVJ2gj&v!{cAA-MwL&oJo(egX;PWJ|L+7l_S(AZRy;9ZsCMI$; zcTTGIANLZ~VwBE3=7A73>WdifXQGcRgH|9_q;#pgv@9)}YJN~nVsZx?Unq}AjLTY& zQ@SY1uY~o>R6?wyv{4STGmPNCP&I30$T7jf@MeBspM%dB8^LmIid$SOXZn6m9N?o? zvS35>^Cm>h)v=@(^B}2RRm#@OJ^iT9%5>Ns0REV8<~S(F1j9w1MC3yHgLv&*(X!t9(=y;dg)ILdl`UrPjLef}?{ zrq)b%cNB9!$kbC5&v~E1Wz1U% zc9|GUkJHL{k{ACuHSY2{`k3E+%A`U2k-lSzYfpmp{NeA>0w6QAyeu$ z0Tzq`(KlhG0e9Kg8t;L1P5cU!9q1r3rypn_zw75V7swlKxDN_Y;JZ*xdoUKm>vgJv zKtm3tC4!4^#Y6Qd*l6B5AXyfg%l9!V8+-v8|5!5=P{KEp5afpb_zM$F6|MHuCHBG`BY-$a>69)IboG&iXzlQ*UzZ z%WE>oLKGE<6_Z~mzn50b-@A*cZW~s<&9RhM8OV9ipu^e2Q%LKAHTp`iopu9*XJc6G zD{D~x3w+`Owrgwhs{>^ZY(TB=G1K-1y}5uwgu!-iW$pWd7fX77nci>blD^U6ya+=d zRWuGsjxn)jAL}jkh9e0EutBo!MZWI!Sf{}7EJ4eZN2SRZC*-3;5*5CR_l4&AEw1GJ z1GlZe{V@7yCVs=WOHVqxz=QKH_b}}5(Oo9o4>tpRqvc`Uw{yyCyi3?`4qkRt=SQRK zo7d!O-+H%8+-F@Y3-8owo#T-k%7yM(rrum(@o8SAAH68@aJ$1Nj3@UZ$upr^G})ky zlXw0rNl&|y^pfrSVg)`mnTAEQ8}2E0OZ=i+%IuEYztZqC(95m7MN5D^OPVXO&zXBc zTd%G0qT5-u5g%Zib?hoDN~0(~-tl(n@{KRRlBhG`OL;gyaSgKV)?2XR(i>sio+B_t z+1DQYDQ(qd_d+3(UuVxowoO@)^2lMPoiZ>Fuj^>?@6Y|R8rh|qqdkSlJE>n3`^`yX zBhN8+ep%ylrD=a-jrQf{DD{pHzM8m(_ILaW<68!~uXz)BX{2b0p-y}L;zROz?a#?C z(*hgqju=#n(#6uEjr^~vlpNv9B$voQIXmc{d#Wo)T1cIs%Ku`wND(tQ@?kf{^Y9cP zTPBeP^^D^P&tJ?BBNNY!_P<#HYklwRSz%!ZIH&2>k)#2Ecd=yDi(}Ow zEY7#%)jvO5yZd!a7q*doYxli={@3so#8sB$9DR^szpBl~09rPqN~;u=-5|r-lH|?g(M%L@MIi6(#vP z1#@!MXesL}V3*3rxnQU^MG0zq(cy5gkp?LJaN8j1=PQPNi6|LaM9HAkInd&@JROQ= zU?#3;{ZBwpcN5>>>{nIv>BPePu4vlewtKLOUh|4uW#OF4>WhQthu5^JmqR9K(QbeC z+^hSI)8ZrA|k04%x$+bqX4}a%YZk0IbE=?;xQXm`ho5^WEN?Js%voed<7}b znH{*Llq_;GFJo385+)U?5$c#aX8q967}_JJtSV>P48PzLP+Bmc>QbN80eLJM!`ZJ! zJEjTGKA1LLx>Rv3#qj~azKab*TpvjMNpl)a`>7P;e?eg}EvI0)9q#8FP!^(>LHfpl62BwCbjS#b1zHrA;fzmH-4x8@zU zJV+_RHO=By!>ZYAaCSu!i` zknn4#hE~_LUd+4QB2#R_W^&o1}hXua$oc z!Wc!aoW3W4wD~MX)NXOPH=BJiono-3cWyGN*DPUJQgR?4_km$vsr)dfbfQU=srwXW z={z_sk?+3DN@_{;t#$Sc&fIG;E|H(Kq9%TMk%Rmer*hb7v4{IwamDf&FE5NNNxDrb z7SW3=3dYr%7M0T3*-qRgY05fygK)*Ah^ESO^ngyuK2uuVH4W=btGz-rvuN6M)RF}w zF_UIl*DF@*PidUBK3w=$BG{w&!#;F1F(*8mov7uEePklXU^Sfwq9Z8r_SO-r>x*I> zMJeH&Ymw#;pRI^{UyIsPTEYjUY7?*txRRlh7UjDLHE*J*B^nNISK~*<63yqz^8=m= z!oS*-$_GpAqgI%Ontu5%Cw$J=8ryZsP&Y;3yd!o{VUqtIXmdhi^3!g&shfr*jAlOJh*V#5A&Z8I$1|rru_tiw57VPz3!S~IMHmu*hPNdAeQ->L^fP@6WC<{ za+6x=NcJ%+SSkMbYGODioTTSTlJ;K))NXz8YaxA*4%8)?dbV`_?$Cn13wzwTZ4C;g zA>#)Y*O2nraDBIgW6Ic{pTA3XAR}LyDbw(Vh0VUfR9|)zO6u-OZcHB}gET;T1n-NJbxiS?J4Q>vt)m`{ymb6uHgy3>GZXnHD zu>#H#J?gv==>jUn5MA5Q`joBvhisd}dS1E674 z?E9S_7#CyD{4|z$1$)8u!Grn)K%w@T^^<{weBfk2=+__j5BIl`olKkjzeciG23!kG zfF~}^-!yNqYN>(b^6)q7WEGBx@k5Znx-$wsnC?T^j1ixr8YW?e`&&ZKpgmi1-u>dY zI46*qxcpFJnBBlzM-b3XTZKXMct6g7ru_Q(M-jNu|DHr+iw?clU35Lz8&7)~PT!u7 zzn!cTIoj9a_wEK|5;%!(#qRkcgh?URFEA7>!IaL3{>pu=Dxyx}j$Bm=UsB+JwLaPd z{D)_(^JO>S8iT(hjT4H3w#}S=sP69C7hIOL>ww_i2)jNr(oqO15!h*Irm$5AKD**;H9rFTsYxnVtUkQDctk9qUfh~rrfnu;oW?&|;20tcJ=c~fGRY(_h ze4xX)y@Oj=Pn==CAw|p8k8l3Ot;@{0Qod0{qJeK%v2sV)1)`l}r!>b@x2fBN<}uDK z2)>pxGcN<2;8&v`q6NJC3M_jc0B5F$x}97_8#~5b6+$h-ajR?_X{xkHBekbKZN{Q#NUQY>=OKvkDUVrI@9%EmLj@%53bOUtY3&f#AH=8UPlsW}WAIK=&W!Jcpc9;Co8M;ZNmXyevr?yw5@AsHR0tiLDL*k+)1^w zie?L5)l(lt^Ca^F-2M%iDcVUlR(RRMZkMTQ-FX1?CkG`s-sbiC6}XO^Aexc1FW-t; z4Z|k%LYUXE_r*Wh8HyDPnULlG+`7(SyH=uabtYE&8~OKjDtV2b`U9|YgWTF9PZErI zwIh+0+BQ39vcXjIP`CAoo|0zvJ;>Gy1ZOEL{GBrY#J0?R0$^#`cCQyR@Zw)Lqr-;G zaO}Rk<-AxVsFwS6!uh8VSr$dXU;11FY#@#=-(7w3gm)=#4r{Fxq2i|D#S);}U*$c0 zXk%;Ma?pF9Z>DZW?ay2Kb{l^!7V)3CKeb-En9df?o)B6_2$bt5eZ=(iA8dlGZ#**h6dJ+#O4pvs} zpL&1gO6QBbWn_jO384eEjC9riwJR*WDx7@w4jdMBj?#Qz3rkmXp^CzAzJ0MhUFmD9RR@@0pgVC+emb6z-&Y$3}pOjl7 z6?$Z1@9odkM}zITi$;!Vij_KHEnYUZeyfz1d7;-D$T@^T&wgkKShJ;fjxpRc2qGzL zgU@p0{nek#{zE4wh52e;ZzYQi>^wvV<`))I9>P9>zB+>s6@;g;oy^VN20cFji4_@Z z&}du3Xr#NL$I8>=0vKZ1_cbUWl?e>q9g{Omjj&BNNdb0!0P^pyzF1_D?*zJ?f8AP6 zAWTP?;BFW8EX|&9)?eJy?PGBK(b@%e@aleCzW#4Ss_~QH+e*@Ird`Ikrp>>!*;{Qr5Lk(`_%_f4hY`fR{OZ|2{3OewJ1=9kZv&2uYk%Ah`raSPfRZohZAc@0N8)tI|N{rnrofJlVF zqWnY}r5jVUqDc0h;HmyxH&JCa$vu2L>=_jKtGj{mN|Pq|b3$GXk;%L^gnvc^S8IGk zOQHFkE|Dn6d?u|e#ND%8#-G)phyivw8^SPOy)gi++UOzl3o=tqWI7$8ZHskOuGos|8Yjm9LvF^*x*o1qS z^+3%p9b8Act@V04ra#*~k!R<$xU#aXj`5!}W+a#t{tSuXe|x#{V(pB?{3v$)iYl$W zVr`zNWH*o(9EWJ}DdIK_@uPIW1$W5p-KFw-{hGTM6EP9gf#ut;Ej;%Z*H%CNAkMJu zdmBISO~#(|v|`5Ocn@fNznm}xnYoZ})U>AbaH5?yR>*ZKyms%h_t1k+5Ie{YwglBj z0Qx)*JAuy>y|C>S{*DdvqBcza=%&Tc;iO)dTGXju`CP4P2W9Nf zV(Tr(PFw(|o#yx!@?JI7h+nJCRFT6H4y=_jM#i$Z#|kIin4^^8blV>1=l!Xb@2?%` z+2px&RN#c}O8arCX!-{-i)tfudEvH%t=}^-GipAky*Za@7xo?(VuZSkHEmi*VUz}J zeGSS(%cY1_Pm4tDyAyCl=!~bUFgg+p{UO8076a3k^By0PD55OWqV+e!Z%l8i(|WJw z{B^4V)24KMaeWJOS9I?7RLWbh_i&89GA-yC7hT_+xpI3{;VdKGu9WNST+-h~JV?t5wOUrg#bxte9 zz%xbl*4G=8txue!5h=0y(jaMAHGF z{CzA1zJC^NLj5}{Dg-$72K~%5{s`Op(tMQ)_j|BwD&sX zJaExVC=HfrA(M(3&5f4*No$|SW>hak4sc%l@7I4|)!8}?6k07po6|=gWY*RH^Y<&R z2f3eV{`5uDIaK)8sSa`&rD!Lx3AO2wzJlA+Jq()>nKi*{&9e;CpK;m{(UFHjp{hKcb9O75=-Va4tM)lOi zroxEV_bsD3MpeU1wzI&=NxI!UFg|W6R^=MDfd3J z_%&;(ym{#IaRJu#$*NF$tv&#uOJmo<6n%xWqv z37DvvTSt{O`M`sOEf5@i+4XdF46Tj^z6dIhR9|la_A7m5{ndd6G!-(OpW8}|x8DTs zmRtKamMk|D{NB*&w;0LMZ6x37rI`G5`{?2y^fj^(!+F&Jkj9gcCO29pJ@yc9aBRE9 z)@tNdVN+q1McJ=l3D$6`9}DRo-}XQx8^g{;ZUL+O2>v5cXo)PMKYlfISc=FnN7n^k z8aZRDc*(?O?7#+$r&>h&dp>_Tfa8$peI%mx8_j%?b|!JjZPMPZt8U{?K!Ny!Ub;%} zweE+4kRqbWTMyxzhrsK8;`v3|J}0T#wB%)iY~NG(;kxrzgKfk5FNPy_8)>&r0t6Lq z4|a}=Jiz&nUTNAMaPcY2t#1o_H!9^Lxp~ByOV(T(C$?$|S1_OwIyeDW-i;Jhl<&Y@ zt2f&>3=x^q(apwh`Gk=B5IU_}=oWW#cDcq>msL~0mX7i^>BH|gJlAvaA{vC_FjVgv zdQ3&@T;Jd$ignsReM@DJ`UfHGJC^zvh5NLQj!r@OHRvuoi5s1_!J#%m0rQYvr9l-Ol#8K zKhy(Bi});dal#{nY#9KVeBfa5H8f}z8Ea3rZ17!=z*9;vaY2uD$xA`8#ZK;K`KP~^ z^|J-vc(VS=FLIq+)q5>SIBPLvDJ0W!;5_~YO*wUxa3;22L8JNNpYe;i&C`+U^pI#? zS-+?P{&iq&PiOKJC>GB%D5%F4{`A`^2ywCUk&rj9-J7jgoDs#t)G*x%36`v0|8clO znbOy@H2k9A!;22A`{VlW*b{7AmajZ5TYIh}L;?s-7llx6oWgWYmnTam~Zm^Ow+Gd?l!;d#T9DAdAjM24&PH@ zDfD+XD%&yYCs&9U9m49^8_L(4Xqjb<$zk$8xP^_y+3|w`xxiOH*a=7w5mqSpq7FR+ z`BJoku#RC5ekS0a_rd6{;F_aj6Jq<RiQYlLQTx$fGc!=|Q#1I=s>ddMV898Y0Q-41bbLd!W1zA1Y4owa%R>-{C;{7u|71*^)qZF(7IZ&oNS4^$sU1GoihvU+xQBZcLM_g*_!@p+^*DaUd5EY)b~1b2b-?<-Ty0t zC58032r{NBi_$*posskuX!usGzbcY$>y7k%?CP0WXL&s@bku1MyRTn@+ujK=Qw+cI zbSN$LA6g)O1@k!g=!j>I$C}qWPWp_GCD&;i$mDINLBJyf}goFMdDa6$Y-9sy#hF{$4nv*}@_H*;Wy~G@AcQg5Z zkRv%wvxc2SU#qZ`E8uHiVsc`(S2r(Vs%jFa7vzz#_e085ByC;@ly^YKx+r3tqnBiw4EKfWW+eME|*z;}zn$hb1gU!lt} zE2)yRU?cV0jkcACq+jISSi%kzd-I?oPAo7{M#yMLNp{tXh9XI4K&=nu_hQIxL3EL0 z^5=)@SL0p&Cvs4gH0YzX3d!H1SIAwB$^901Ae6|5O{aiA8__nD z{)htv%Ps$r{qvR9U=+9Bjp*M(^saI9T$PFRW$>#1q;gmNBW|j##;tL3Yk>&E6y;EY zX7pcU`cpT;hGhd?48wcHWz?I67azC!UrJN8o4LLp*LFZ_f7gL`*QWcJ^?S+t2Hhql zz9fzN`O9vF*D4iQ*&5Kxub|M~vbpxwBP&`mwbZRzM|(RKvXA8`99xzWzY&scrpt3v zKSQ|eV;PKoCeyox=*gz(9Sn{d_V2w!K?*hs+n&McosUm;dxBM&{;BN&(Dp>;L64^0 zz7ua$!5yPE^j*ParFPkJ3sptShFA9Sk+psRbWUeM2+?&V&SXhR%EvV?p9+#4!cC!7TF>az~*$eeaHa01On^gMETJ znh5enyE)%!;ql0r-)cw@ltVHA9eJh=-5B<8 zJ#|eAhXt-q_H*nqCS39pduLV?!a2TsEeDkP_-x?Rc;);C8ZUNx&KvFu`h;H@>B1s3 z4CAkH*Dh?*@-aA@^o%bqa7o;f<@E_kcnvS*x!jY{0xaDN{v6*DByebxL#W@4|HMP6 z2{3=gPc9kn!!B0?aIp4*oaeK-FvdZQ8;=ZN4o~VjOOzR8FxwNmWkHr~kN|J#ntzLk zp9x$Pnw!l~0o(j8*go3u14zGmL)|!CE(){-L!?ML3TBMm)4Bni83KpbFpoO?Pls>} z)FJQ{==T=e6;1Yl5wrI?a^^(kJ+V;#aHO#9SOG@i7U%G=JxOTMR5#s)`A)VN>xs5p zP_3W=(*H*Tbg$7zEKQQ*JF6k7&B4pr!#sxJ=57a@a|J?jdPxl^mzzZf{i+Gk88L33 z1eyeXCt0y0O0!J0Ylw@WDd^37K2z9Q5)AeWEaBKzIEXGy8(K+JZ^#5kJB(cwO2dZC z#>0+VPvEf-zD1mJN2JAX!_KFIju&DEpS+dZ{}#ZRWJ8u5)54kmg5skJBx%$M1*%kp zg_z;$aZf5Hwl)9Ny#`8_dru4^_`GHcQSdOW*6y74^VFZD^M8)n4;9}+|eCk7xqD<^}pU? za;xRn8v6=X7bH76K?G?)pw8nhJw*70D9mm-{3ejfTf7-xBC7fc5-e@1k@=!>6|MzW z-GthZpqJXUV2mQhZ0{uh&EwTg(7yuN{KWhfUCQDQhsBTG>?R3YU+ZCPY5j$~(en&o zz2uiNZM*RFJ?MGz@kdO#C4iJkPW#0CQDL0Z_N4xT#^Dc4$zusxn~XjA-RyS{QaEfe zx?r;kD7PAkZ@{ z11)FH1o%IF`qcjj_)(Qza-+4ErP0Cb$zK&V`O+Svxaq1C|4wM-8!cEg9inXTc=vQ!Zxa4-dAgUe8S5yyKi|wF5yB;v) z$kStQyKbF97jL=bG`nWUm@~&lKBmOw3I)C&u+N3urMayn<=P98fA{0;3wzvE39#|C zn%_DQ9B4?Hu6S!Zl0FeuY#XZg6uTKt-PXQqGPg3NuFHw>d%qCE|Jf=K6@9OQg-;x0 zsRp^*nRvp=o!3Q-O_<=ng`QE^WV47+DMD*7NTR>bnMpf=bs`InUH|;azd(PGtZ9MK z!DiSx^1^R>=8j?ECCZ1-0jDHVd?5z=H|$(?(G|9~a+?KSCH^S$)ie;F*3Ro+&n$`j z()jhP=U+gxpCA9WiyIxNw;;^a5e1|>YdUd0Va!}wn4`IEk>_fCT4zC;$SoG1wqM83 zt2Q-3v=!1jzB(-^IWGzAII_rq0?vGDkmWrSB`vJ-CHGqI&#(CiI#aUvLB*-~don>i zi;PP*-coDiD+%ptgA8Idl~oNNkYMwvxs*1^Yl|;;c-NfzHQ*D2MG%&@T>v< z?fWxCY>ah1{vf~;j}yfoD$k5{S?fef zo=cM^I$p@x?gu?Q{L7+UM9SQzWst2H8$9f0GW+mlvw^lynQ!jVC_MBM0D~SZp%=nnJ&5|l2Nd{# z!ED}LGX9>-*4;BU6RTn|o#2ve$5<(&qi$%-Q0uf)zd zB*E&qTxITBIUOSZ2Z&~GZe(JWg}iMM+c_=Uh`UiZS#7Z#YH1IX;XwRwy4fsI|El6P%p9TJ20`rjZOI+}139&Zy1bX^ z-aP*u(60eN#z6;~o#6C^wsvYleG*4s@rkkycGC_X$q?N5&Hl{Ne zUXZ$Lk}pSUjsb}wpfWA#!3Ck~B#Fgw)3vu$IDY4*&jGluhwZGRx&8u9v(*U@(J;%DMK7wmb^%cCPzZVT}q zfYBbcpEGkka?RNL0RXu}UWX}z`_D`K#uOiuPo&8?DY_h}@)xTQ-{$OXRH~@XhMnp} zC?C-$eVl8LJu)D}MCwltX(C^p%>RrX@Av4+?LCXM@Da055^CTiQ!rI3w86Kx~69(uOHY6Jt=$O!V9Vg(L9$$}aIG6=i}!RgF88n|zY zq!Ci^onP_79d>$ZB`IBiEbPnz(WmRzlIyX$l4Tj!oY9YIwyDZD z7}-T8rC7onYz(kFn{Cu1QHt)@_3u%-jmQ*Kj0vOrAA?9C>+_tu^ozS)2EvM>mDeGI zjgu?bd$KY=s@e&h^Xnh40aDAY{F+cm?fFjFg2@$08okpx*|;^OW^%}AS$kgh{%n1I z_~BOAq}-S9WOHyIaG={bi@C7Z8av*D6&<0i2H}&_Wn}bq%(^(3MHn;HSox`9*~H>^ z_L1M%*N_)i)Kym-xxW)aVF)iYT20-qP4Bm;9o2j&Vya0gF73?o-S>)Fr?qt z;s4Vq+wqFpH?W~zO$<2J^%zTTc74^)t`wks+8Fhik!AV7e)ZK_ub)ZF0Zs#P0{OtZ zJYJhbirT=MpLnB!QskJZyD@rM@D`ZagmJDaZAKQn)b;_#p8`@5GJLsZcDhEbgZizW zHJ<#kA_rc~{VMzJspZs(iU@z8ukZ z>^b{*`sKXRUbGHM4>7gma;Aq=0YTLg+c#r8n8iu2wzghBPk01phP(ci#j!X`! zUjl>&)`p#1s?eQ!vo5%07+nkThv`T|mn^x;GO*p7F>X>*ar?yO@UCb7Xbl{di7Deu z(&DT|&YM*^EY11Y#4B~x(Tof8w3O9|;Tmr4J_)ige|~930^z@saf=|nso(IO=XOCtHS;II#_QrKvF~o-%b#fWuY~sXhoGObCe<^; z&-ax#p?xO0s;Ywm%(SOY$S(BOJ$J^wOLr4>1uz+}MPmCtd>u@ArF)Bn7&oDb**uK8 zN_^+1b3(@Gc9ge_*`DaMi*!TelZm+PC`TFU z&S-XOgRu&>{JYZ!A=vm*h%n}uu8FR{XKi0{jXjNuPnu(KMH-)In*9BAQ%x{Ok^VRG zC5o-Y_}rDb@+7jxoCAB!TEu|G_lzl6Bk*7!HmBG~Df#L(tA5kxbWHx|I$E;e4M8{%)P8}DUtx6Omm zl&s;?*a}-P7WiUpt{Z|#%T$|zku9)MdGo{|DUKA~DJk&N#%^uN7+UlPDf!mPJma8E znYo?eKdYq)u2XOEzyg9MPh@L_WBaE@kKC$Scxnx59_7u9{1;(LY__T2$d5L%swgeR zxITPstv*i2Q{Ls{V95ti2e)o=gJYeC+Ci#;cKrOi&IXp*$pxa(n~9(uh7k%xCpr)t zSZ^gP!zd6>_|Fm6Ky|0y0_7J8NQqr z)PgJTteW&L57)S`0NQHdkHrF(E2h*)(bfc4)VrGvG$00}LoP6izIzjdi#nsBnjo_l zIj>I0M?-eZNC6U6*ND;=w3TwbImKJ##x;N&;GG+u8&Fj)vC4{tj)Lr?g|Yze6&*i8 z{dbgQwUo$JdQF0f3j;D2?Nqpo)Wud(=g5~UgxxyH-g^Whxuq1v z6l<6RU?BLEXLPYPaG?<;leTbDAbvjU-E#TR2J+6@GlYQiV;1TZqOw$(%M>OaPW|K z9%RQPD)QUt=Ut+`9K>I)3b}6>SkDeOse%2LD5KCD&9+3?Z8Y0TJSIaSE3VmYgl~zk z-U!FykK(r?WZ^qv*mCflH!yVBy9}~jb$pem!Q(Cimaquh>42eYX`)OyZ?+LvUZd>( zD-<=XEzqmGidQXuIo z-anG$JjKIHPOyE$lqUp*wpGGAGWP5$M3rj1Pl!Jz9+Mix`tg^e<^Yf`eW=xLn=KQj z*uKb|DS1gEb{mrs>ky}7EZ1nUfh3*oFlDr0CgN&W?wJVAIYZh|yYcGlymCKOmw6zG z&}%tn9$5c*%u(^@N=pu=2mIm7SRLdU3wKQ16+ap7lbH_~u9j zW70Q1naf2%UbMC}OPR=$H*7iB={Li`IO)PSE-Z-!h zb|NY8OmyyML6<+hLfd=|#u?F`XY7%IZ;MHjf^NPHW7_|Mf?=fQ=6?f5&eK)Jp$VNu z)1e8Tq1j_iwi0MMZKAzSsI@vABtA1OQYQYd#sr6sE+_Dlf8eHRBDKrGcc|Ku~%b&MAA9VEf7-d5{28w zOu(joctd49H{g&AXijTUY>sKyeEa#Jp0O~ds6s=Gc0m)&X6kZN4sdVVK|ac2-{>xw zd?>P8b=oDd0(T?T_0U6bsZMNiPhH!~Q4`Fk%k}8*)pe&KNcz$%Q=teX=0FDg@vo*h{xsX`1wuhK^;7 zO{nMr`}-}Q(1^6Y-@%fgJ;FM}!_3pB@R&tOLtIPkt&@B$w+L!-T{%tXO5SNpHo=Cj z+-Y+eXiTcD*8@P@y)hETp4()C-^F~S-!i%k4zh(YrWBHh$4x-R&H=*~0L5*+~kyfh`TQ zBU|c-)iiQs-MptE0?JwPPg_I9J>u>NXdI{pcgoESbni1z`r(ja`^!-rJ&rwJM*o@> zOYa57P+0_FJl-EHU=G{VPXQn^!)}izIrJb@1Fkf3w~taZ{Yyi@zbL%Z2+*p5bSr}>lZ|20#{kcPc z;T%B21;UHR(WM;0gx(&>)}#i6=u-PFoO+Wox2xjua4R^L3`={K6~kxX*jZ73bzVW+Ynt_}q7SqvyoJ9k*6?g2KRO;BU1{@{v_m zPW-^FNuu0szUyi$elC6(5_fRmR~9Z+|go_}4_j96r{i*E)UgsFgo77**HyJOfJ zNa{6AW%AH6;E^}fTTi|@HWr(V7Rv}nK+qW(ynI64kpMJtOF zos~M*Ic@|XmTd5%n@YYMK7+_|O&BMeK*uMF@(|OEm8l;|f4yRak-PYM+0U^Px?CLU zEwhrb?4x=gd>$r~d&xM&+A`rW?{GB%>ex9`^yOB-EiSfrel0WDSpV z8IXo<&}gnI4`pVmZ=z~s2SuFi#kMp+*U@Tdc9>^6DF*AfX?I`@W@+|;{;+2Hu2Nhr zyk7veD4?72x3imkp;|+mz)7M!fglHU+dZExhusu+~bJn&m{7a zoRGuEQpH~yc-O`+U$bL985vtS%yeHg-%2o!EIL<6yrqr z;=MBYx3jP~=pC3s?Zt2bzxleBL>BdyUT=v|B#0eu6NOgaM*f?H&tlV6i#<${C3R^x z*s&iYf@Ad&9#% zy=Oa0a^C>{tELx_LLI$8(pyi1MDr}@KQs*nfjzWO9L8T-S@w_PGbuOB0MvG_I)A1? z;*cf@B=67!upAHX_jwRj#NMKnh|kv#^O!bMMr3eCLps_vH>j0jRQ#q#1YNp@4>W^m zMe=ZWp?5(*1epn=Gs%wy4|5{Av5n=1){tDi_mU2S`5>#&z%b#n(FW@%G%u=Sa;>KK z6W*T_8qHI}X>+T-JAMHLE*+z4BGXym`&}wny_{el0*)?S+e{tr9230ZK^s`Zzqm0! zCrW}MI-JnunV<^Z85MfR_-MBJC0BIjqMPJ6Rn^LeccG(jG`R8w+lTX{K`8kp%KEl9 z@i}F8@;Us-vu6PZy!hvmBKtjbsq-H;Cddyx4&MB4%8pDb{(=69`wj$lrdT>q+BtG< zWADSNH)&g9^e35rzJ2cd{i>U;zJdL?E0Y25P$E73dT6Ph4tdU|TsBm+sOK@LJrnG| z=4F$jtYtelztjmYKb522;z#=YVs7l5RjGwAG&UC*O+wtMGk_Und76U%^fR1=-(gm1 z<44PTRJ$Of! znA@&ZwSdEau@mFs*I7Mth@y~u*=OL03cVO!fSY00%FG`ey~?#|Ef-2vj#ocHT~XVB zrGRLX%bkf;v4EK>Hcg9N5w~ONm>+Ry`!e}O3ExBh6 z#G><`e{{Ro`t71ue~SKh33ME-T(o{7Bq{Pj*W)G-d)>%2zE#H7YEO>z5jjf|Qoc&G z>Z{*p1zksA3!SFmlfox+a=kAL&IuuH$4LJ%vTn}*fqvMkg~LYUiKhq3F7U7y!W&G1 z;!_yroy)67{NTR~p9p3hsFYfhm)=gb0XMf8dz|F!o6C0MzS!2~PmUxFq7FI}s>%fRnom%jVNZn`3Er#HMP+S13`M%=@{ zx)H8Ogvuafsk^`=kg5^GTTfGp0Y`qzHx}szM%?Vcs{L ze^Evq=eVvJKF3g2hBy82Ke<7&y1NdO6SSVOwNXC0|Emmzw-eX}a%~CD%t43hW3HdP zKnt8qZF7HNUiI^Xt>H&tm0fZ<3lEytAd^G;2v_P$YDQdv%%O{|{I@SRCC|B~4oJAwWwx^!Q3Jloy~5G%L$=LD98zstAl z)DWm2#-Z$D=9QUy=WogiUcP!mxJeU#WM)rc?iFYQ%Ah>w03*0d{+SZ9mS$74e!m82 zm~Il@9{>`>1MN|PeXm|odP$FD67`$CiMx+4r{n?V0&>bqh!^=opu80mpjSmW!^)55 zhSz(YIslFq+0I#gTxba%Od=h;M7zYJAJ^Z||0=h`M-uBYi$2UkpZw9_K_duruUOIr zJv%5BzvP;GmV{@B8vIKeshcCF(VC!d$V}c=LT;!d6|s4_TnsjQdpW=DE6VPd1otVo zf~?r6ypE_(mqUv(TCR(Ma;0GL)V#X#5AC9)$Uaq85*O{ zkrun$t^87yDVFC?E27+{a_o``!;B4KrJ=tsN)U85tAV6x-BF~`e;6UgI(~@ zB8L7Jw;B-fIKUZbf7#CW0>Pl zPr)lUr8=XUmM=F4PSQ=iTDcTFod*}D;n@(yVG%*^l% zFsHerQ%#p8dZ9k6M`+NiXD4oN!Pp5_xjIYb^Jyve;7ju{YzFvx|C6h6j-SQ{QFw9y& zJ*)|oe?mQkBiqC5DdaHH!xlga``Ve8`B#vu8gT^1Pa%&lGKUW9kw?rMSeWJ&*7!YY z__Gc9G7`gj%yO9I0Xx$ZFx@1EM6hw@rpHrC(wz3Lfzd&eG!@AgJus3&xs5vcE zqwJZhhE48eZv=7I;CdSZpD2G`GvELi)gh*BHRKkIcQI~@fXo9DO|fy6Lr{qH+YGTh z*n3Gf$Z^NjSqAq14_EIV&-5Sv{g*?ON_~V-R;h%tN^-VND2hsv<0_#N3&k*7s8r4g zmBTuaO048;2g>=hP~@~N!*ZCN%+7oN-hF?+>vvtZ>#yBzj*b6JYb|$f_Ls)u2;b7^1I$ zwp4OcgsjkCvftB-;q8NQDD$p8U6RC*J(4*tDrT?+X`B`$!2-ncFcWwX*BxV>EDANbF`AnwVL6s$I zD}FNeUInt1`5Wo~JJ3%Y6v61_v6!e~X3A90De#;#PaNLdGPY1+ zvDSZh??AWpI`Jst&7g@AZpNa2_E%!`Wv$uj#5@H=;p8a{tarf}q-tnU=#9V*WZ-VZ z*(p}h-=zT>^Cegt<4(wWcM#6A9fPy{HIVfJ$6=~JTbGqqLGsTcc@;-1;bY?aXQ!v$ z$i;8E8&&as@%_`~D!33W&{>DLqcwXsF;6n~m#eRUF)&yhr z&BYW`gc-6Cd|;v34MAD{Fyz~fvsnNHS+BxhLNWIK8fmkmO9#IY^-tem`Ft{Gk~Sja zl_e(I;BkO0nuk}BLxg*zY8XJZbf?LY8CCPG$ zDef2jZc?7qZz(#tIRpS7zRtV_dUgKY`P5qzN+q-$QZ-7*u(bYS%7V z+=g^e@3{){xH$=4Jpe92Zk@8^+^qN9A@&TdBq)pG&(>OhztjmVp?|IYEm-}1qwtPW z5ioii|D5wGUUdYvnKjt`nlLzMTz$tN-HM6X7L9BQgE3dxTVJjv>0cFXlk8yvn+-9I zQ;REdN9c|bF@`QQZ-(3+VQ$%96rFnDdy4q8X15v$i92l6B zh+zZ!xU@>A@{t1;oR3zh+YLW8OO}&;7rtaJXEc=>u%&qLiG=%Xc_LO74XL6qN))|> z|F6lIH|T;S43jASutUIs#SzWCPQlQKx##m@PzH?KXq)w}GPp7&@ei<;lG}cMChl{0 z@M!%&0VPlz&5v`M9uPvuKhH`T>o{Pf;f{jXy`9siW2g57j{mMTk-SIgJpr@n9QEnp zv!f-soe_2E4?wGjjr92hk+Kf8O$pW_`cmy;t$+pEZllgA`RgTEIML#CYSYTX+RDM3 zs^WH}u8{bd%WxUj1yqF-&~W=4W=xj(1446IdBd;1l6z|4|jqs@ej*e~{ry{yiRbY8I4%KoG(NnuM4yo<$!0VdYj`@SRT0J)W?Pp$Y8rBhcnfk_vsC6S1JKWvhiq z`3JSg#VRKeJGJEEAU=h9wf*;(6MCimOe(ki+2HHEL0-yCJHl2T#&&8V?fab}b`r1D z%Yhfe4oa5VPlye^y1{_8#QYSo%2l)fL(iH7PE8Hl&tYqI-7fS4j z@i`#^aK$N=09{{r{(vyA<4|`cUSlvm>JKrc+zQH(*mwx)(T( z=__Cy&p2WwML!+-sL92E{ZJJZtVG$EAaJ5NZLVbwex+>Zq&4$AGCnQ7K(A@dG!goA z$kf|H!)2yUecEbP!vKnMKZ5{svDp6ebI zluK+cw_xHwJr5tVinXKHL@t4SMXVT2`_R8Z233HhHiuWsf#X;4(>@B-y3X^UK5IaxgX& zL^zCVw@|n7_Y&oM$Rvtp#<~RPts`HoYl%@i`vk3}@0D3f$X}DmY5FV$YMfg+O*8dD z6*&9ffY(Oe>IpN{3liCG9bRS*rViT7jS>d6_F=1l7z@xvHgSXFMslpF`19U@0n60W zb+aFC}rp{n97{^!-zsjWUPkL$)MMvKt(ih-9!3b8z;8tPO0abU zo38)BrVH%cN)z%{7)l0LMHf9Y({Ck7@14dJg(Jv%@ZMXctbQ%~O5jLXcnzWGr4DPD zYVd=Z7c(QN*h+)g-~ATTr^^tcYhYXus2Dx%Y6<#5iHLDUsYSA}o9&eA*(Et$g_AIR z+LFAgjjkD?Sn#E#ck#G-6-XgWu=p)Gb6kmNroejCBr#wcY(x$vf8!``MBY%MY)D|G zHrec8%T=W(m5Qo6(%q%eZCI^3X!;qc^}ftA4;Iv`X~TqmIblW>g;1hwO@M$fpp8PQ zdNMs_2YXZ%Z>2!hO8~Fh2*eAMIexQ0{muJ{b!BE3%+$GHpBu~Wyrfo+~Etmyhj2&EPb)F#PM8ssO3 z2a0}JPTC21N@Vui0jchN3u`8(DMT@Ya@T)1GH^>q?Xj3oDdOL&DfaBjPl6h~@uUw2 z>z`D3n5-`ZL#jBv8q3J;$GX+PHb9%r&^+A{kHt{CCC9^o9hd_{PHf5*tKbyGALY!q zfvqqo;9urSz9!Pm?E6;xn(&G9D!OAjg`U+B0;b^DzgFa?Xy5O_Y!qgJ#Fl`lf*Dv_ z<0lTXFDegS4<649rp7_jvizb{#g9vIPXGmqDmas79H5|=O1n0RGU!{FTC#3Tfw0Bj=0U?t{Ma09VU2A?uEa+K0w_ucNavfNejA-!I+Ff@M z(3t-BZ^TfjWq#$swoe;{HR>UJ8lBvwtBnwiD(>pGs|qO5QL+QSS2@m<8M7wWfYT9p zz#l}e-D!@4Opr)jFW+a44G<4jRqxJ+mgMATKledrk)qv{h(5=}h;ZB7W8%9C(e%B0 zN3t%3_%~9DU!+`xnxvAa8PaD6vwj1PrEbZevXNoQR9G9CZ$rGgsl1 z@zfN^BsR*7Xb@h#MzEU>N$_0Khs$o$*zd*LB@9FQ7rRW7(JMRCtR#2Mds0lZTLR!f zfc4kGe01!u%T#a`w|B59TU&vv<0*=d#6D1)wSLb>F1LlwtB{nXtiSZ!$z$ckkFS~K z2i#u^*7-2+4yJdfbbdUus-t?iU_dT_T_+1%E!{?zAslp39oBt03jCiM7RXoct8giN z@|KBCSb}Gv`|q2lUs7SmI5g8i?-k9Ef@seGS1VzS&=-881>8N)S$^%g!|a%XbK&qB zaFW738r~PbT*-92L{F(A>M_ zrOa$Icr{nh-fHx@#1bd&;&bEIBc7$Xft>h^dvF}@;`q~R%WH;wX7XOyX}Mizbs=XX zgsZH8gkfcWB9y?dkbf7d#0$MDHytYO1R(eXRecamoVK@y%t&fV)*55*zNm@{cAcky z^$QZKT*@Jx$hGJxa-~D?{jD0y5-^GZTJ(dWc7Pf^=f7iqK1JN%LjU-9*e7cJQV}nU z0S9k)!uw{P!xqKPtEbETfJ;9WcHj@{#4Kjop0GrSMA9-MYBR;KsTmphe&i99 zrQFN(Nc9ua$XXPHjrmKT7>w%@K99zAvp{jPyGCDoaEsBN;JH_odH>?p+WzbDtcm_G zDF}aoc|8+2fXTgL2e89>Auqmvr763_W%iL(^~%|`pPv7iY&PWI>Di-r)#lGk+(_zt=`(+e0h_aY zz%BNU+)pcm>237~71V}&`jrte9dY}`9P|zP$##qLc7LkY)L$%6Hz?xZdhTflng#g& zkS5>Z-rQZ#&X;eK>i2Q`Tu#{AH0ycW{n@hCzLfeML^#M^6-~mARfIF|M}BRE+i}QN zZ4C!WTbR{Xj-~iiOd42pS9n!4>;_ZhU|tq~7K*$uu*r+pQ#P?}sURgRf~jX%FnaH9 z8$shJg9Of($&G+t>n!!vCQXnP2CcbqEY9McGY%ULh^DG&SgSw28cVWJ3JU*9fd}H! z{;`?GxXHk0o6qI?`M5s#a~rs4}H9Y64VUq7~J(*7Qf(F{FSQDE^XUN zk5^j$8l~w0>h%1zk#BSp>9N4~JkG9>d=%+fpy5n!c>6F&N`yV;#0=f%g~@z-;tY9; zk@&EEd`L4a?wi#Wez!2`+3(4UgIfUgh_PB%OJKb%MuW#+Rg;fYFFYrOb?ggk66@Ng z>px3*jCM(4kvy&vVTf-rN2RwvWR_)#2R$g!7fzBAz1i!NEUxPz%KkHkZ32{}xhcB~ z=9i*#_?GDiyUx$hJ4vwHBlmfs6Q)U`9h^(!C!;sxpPIZj{42T)8)z}K4NH~8^hKn$ zhwN66z276(2DQ=f)y@d_@yJEm(p)pNZo8$OF3^;6kMURR@WylGqp!aTG6q|8TmKw0 zHU7k8Qh1>gcnJ}=YV=K)hS8y6zTl*U217V&J*vq0)f;xoEr5l)nIDQ=z046?86}AzE)VPF+(`A z0QcDQJ!=hO2OkEehyzz?R_;1nzk=K5o&!J=3F3`jF3-FYd#NXMQy12pmBO;niw78C{VY$FB|4LMDo#fNJ6bUn+I~n9P>%x; z1$1=s4C&@4H??#t?9Z_ml8pJ!ZekcxrOBt5x8KdbOID&IT){STd-=k3hvDm6syC%c zm`T6(9$|Qu9R6nROo92@L~Rq>nVz)vpO~wKHi=YiBJ25J3;F{c`n%lEC z99-Od@Jm%spA3%vhfQG}S^w&JA&NKNQfX1>q456v2Ce_BGL#tm?3gFU&(tC{=K_8m z2o}HeDz#zHJOXDY$#;1A_A_|VXZxZlm-~0*25iV_Z)yol38l}@sfjLG`-J~ZpATZhI)Qr#Y0ehaStd^v2GNgLo~NRZiL}jobt$sC@2Y?zsjiAU5H#fBB4Zz;jN`U;==z zlZl#&dCG68jV#N(d?}<^Pl2_YpdV;UKs4$F3XUpN0!i?t!BfHM_VOur{700JCA}C( z!j4(Z?p@o!3f~t3d@}IsjVK0=hf&D83$+OP4`6R@rJ2d*|4Pk&1UHim^m(nyXO_t4 zRBpKtK!xfOBfD>74$gVV0)zr)LbBc+z13#Ayr5Q|83MEx^S;~@{5R29`FOY?OEuso z5nOvFz+J61o5f=Wm8-(oy2;aR+lGhjW*1g3lwKt*udRJ5py?I@pIVWP0Ru6%mnkW{ zz%%-~T&aq5B^!4^YWxv_p2-ONdJa&*(ZC7rX>c6HP_S4`cuVDEP3>;pM6ZS5TWS3F zvd^QqUDY$rd#^dzeISH7&%^f(WfYz2ZXBCL%?FiW{$)&3CbIkTmeBLRfKu6V`YN_h ztYM)_;%Ll0Nh_i91$44-Pa5F85gK?c1tP{KL14&9G1EHq0RU^LOW zY3WaH38vD!NbBiz;z&m6Dz^GPtoCnKi*)qeg6ARdYT)eMU5)%W6UUioie8BgnSfTm zm%DJ_O^5~ToxWcThxG8ei)9%F{2#1lNx{pE5&LoE4>R~>I~Osf#OT2|o_fZ*^<%sV zYTD<+(49(fMI`{=WRnTt6bN8v8RdL9+R7hTUB9eiSa>Yn)|bW{RB@0)*6O=m0|I08 zioUL#k5Tnk#jgX`w9x-)i<4g?={)7FUdvbxmawps|({S+oU= zHN&?Yf*-E^F^O19?1oftfiLb@U!wi=LlSkhD5?WdP;Uyv zrtc?4Sdx%AFZpSfWdR!&^u9H0s&R!GMIK>b^Y;VxEA6(u$K&nWGrDU^gaEA;W;F|T zt)^~WT6Kmd_v(Dxjp!_i+)XmKr`ggu7;v6KnQ|$7reWDKx z;U!I}XZID~)n8s)aYr7$y9?O6G-y7Y3 z1mQ386_?yT=$Mj&WRE7XiJoS*MpR8_G0$X1v~Gr25%Dhm#WL(<@Sn)+zIROmiN{kQ z4dxMf;aU(EgKHOy+D{p3jZ=c>qpm*zR-hN$vS%cddGh2w3%23 z4!NmpB^gsR^6AX|Vj!_r4L^O;J`>;Db*uJ#`Sm3^%j-F|39glLgJaf7x3ZTDg0aIt)i?S*Amof2#a86yvg+{3AJ9gK~4ceMSjuT-#A{(3XNbJ_MP4B zJ%F)j4@kyhe@X|X5ik$>LomooAPpQ41^b$mhC|<+)q%`{YC1pY>6?Our*nn=2UvIa zQ*NbnQ(JD@g!qdG_Z*X}PO118+zHDs`&AZ(i&50Jknbhy?dtluyX{Zn*Vk~j)7Cz& zgG5|6X0NGXe@p!85r#&{h~Cr@YGb#xqZ8* zRJKz+UJOxq52P{Nfbkb6dwr*gw;Rmce?TL3e^4@EBVw5eGBL3I>4}m*>gSY2#GD(9 zUu~kyplDRO;OF!VG0bpk{U_Hk7VHE~mKw*}zH=Q2hmCIvl{9?f?Jqhu0N;?1cQ8zX zR%GX9vRF!#%^q@9JT^Q;lra1t2S6?4eFJC0;8*N}v0K4aGjabXaLEATQ@{XGH4{Mj z67_$7Uc-F*w2HmA#`*vC6CmqFXb*#>x0y@YCuk+9KHchgLr{FIem@7MzzQDsM$;tU zx-i?@5SPG-;rX`mX%))9m3G<{a2_Gism%pR$CpIVZ^CsZ7$yI%I#33!oty&^e*4UI z0{cHg^BigGHVR$aI;mc*F9y5RrJv#calv2(ADhuqU3<~lPPVXx?0;yL z{~G)*Yn-=n+oD>;WE`)yQ}it*c#WN0yd^LhUUjA~`i?1cG`l5ce}0y8wLtX-Pi1!sG zp*Y9R{^oz!1Z2rnX4i(g+NjTqh6SE?@kZnhS&nV~a-Uz@Hd0`SH>I6JrTgFX^~w@Z zP217>C@4|0i!`K{*P5!5lRt@3WGNE+f)H*kb=xhBZ8Sk!t3|A-?-mt z83S71?5?D|y&dFR;<5~7kC9z$m>v=!8krRP^{Ih`p?Qf4J!uo|LzkR{4VK*cCvJrY zjCKyOOg4vGxqPLMM|T1m(p-u+_QB9>)Y&QXn1?dA_lILk+XtI z`%W*L7 zU|7z}D4&QR;qCV#_1T5`9s6XV6U!CLlcVV?5h|{XZHv)ecD4-T9%$37{&@8}ThP6i zVP0COvM>CZMs(XnhjXH41j%?G>&AbiNl@?!8+Hx;4d~z-$_D#T6Q&ho!+!Y5?Bk$a4tVqu=Apxp4Nk^2g|bHaNZ#*Q6Qqv^*$$rFOXVz*0vuUR)PE^h zm!H7!D4G5V!cVwJ}zg zux~`_!ecau2Os{&CL`|j5W|Xc_AN)f{!KoUzMJ8_WI;1jL3z?1G({c`#3e{GAdP6dG-*f^KbQ%V`xDR(8X35*Ktu8VgZ z)K{pyH%sSZj##}KhHPAZSLhQeCumpdMwV{6JAdPJF;HC<`4kO*C?$j-w?(~2-kkz| zJmFGbMlJU9H$#E^7(bE+_Qxq8!FR%9pfC7p!gLVHV2#T@7Ab+MD7Q#kZfc0DC_Rt~ zm>?afr)?zs^yCGvJu}8+7J}Z|#FlV55`>c~>}} zB=Nxw>-`2znB!vuLB3k?u<#dUzqmruN)@@~ z_N6!*aV!x1Yu$$lnpSu0vsSDMFdV;+pz zQG3WH+}l^PU#6`UE}r1E3}^yuq`(S)aL?bxl3VA|iNH7Zan4oO5mT+fC|CACv{RLA2^bebU+- z74L)jIBKlu=EHFf=qfU!O1=`Ya7C6jB#Zr-T$&uE2(p|+uQ)ZHhfD0KkQ*8-SZQHW zoAhd*;0_=e+@p?kNgT=ws9e2tC#9K8u-QjHDjW>dKI9EF+pMLOB@Pe}K@D-ukiYBT z*z16r{}^I`YhMSQIH7|$Wp!l$a{|q3MVnlQH#qj$U!BMYbU6M~7}6q0v>7QKPKlq; zppF?41!JIWST-4#i5c5FC+&ZV#Zult}_~j|Hg*~t8X;3zhF3ZJ~J-)>2cE7 zwmiGx?S;!g0fvnIvGnz{SrMX{LEXt2)Fgh}3!@+9c)gcebc#8PCZ>6;hUpH0@k_*0 zJ|a$VnRP}kwwSNYA5`zc{{E_e74M(=HO)-?JvyFqpl0g#_ka!wqA?TO{%syKx3aGu zZz$TgRY;D3FNRELDk@mQ}}F6k<>-c@uwL*t&X7 z*Dk3|#+!apF=N?L7VJ}p+C-Yn(Y5kF!LKuv=cY9)U^fohpta{2SPyY!ChH-Jjva7K z{$(%o8h)Mkz1OcqYm+W8DIt$BH}go3M6)rH7}@`Fo^D5)vE#Q3{b`NIKqc6|?LyE} z0xARkr91$Fi60sFFb6-QL;CIW>+%tY=c?C4m}5?L*F{x~zo10tZ^q=OBbM0{O|?=7 zZ16nNrdf? zJkcJCv!FB@DS>^U=So)GQ5fs@wF2$KD+|iQ8@xULES;f;m>`n52>JS@Q734PfP%H%*kb5S6%P;@3|kgd zMLrHQ+#WY!eE8WRL*MWB{u(CH@``UU%CGtt(!M*fu59uKmM;^ZpR{3BQ4IN$j31&r z8pNerkUhcAcNKliArylBW_5z#{N;wL`kZ40=cF2Jmw_9lKVtQT&%9OkfYAiO1ARVk zCRd5t9CRbpGD-`LWHu;tUBcm$(IEU%LiU{eQVC1FE(ymFC}s9XIBv}lVen|@R|tc|OC5a<<@{A!!gvrF^b9+M>EIu!IMsQBb?DYo z+i_yuo@dV^^lN_MqTC)%LhezE%mdqY{baq^74ggd=_K%I$*t&V)I~u7MBHjkjXAA? zUehFL@3Fi#9N8YMBDREo_PvoxF7Vk8>n)6se_9mnjGvCr zi48uA!flq?;R++{_aA3H#w74FaBLFlY%m&kv4e5O3K3tzjKf(-*H|k9h3>BW&ZB&j zg;!%2fL9`jv_jx&g?Ju9hhP3gwTkOUr@SlhoR=Mg0jrx&0l_F71!5Atb87vCM(|9g zB$xKY2^uE68p$Se^@`Fdr*{Q7r7_rIc1~C1g#xIed91i6i!dqgea5S9LL0$PmftoI zSLP{ERC-N%#z{MS_kN@^kga862F=UP@ScOO>s2!fouY2&Ys+6l_lLEkGVSJwuW6QCYGIiVI_ zPWF7aHO5+j47ghQZ%G(-qLptHf`b82{1GHg0|{b^?+m7RB)xOs$LtjlX-%NUhd^Bi zrQW_T1?6&7Q&5-VZ~(M4gJx%NpQ|Zc-hhI!_iaZ)*)KPsHiJbU7+q14gALC|$px5LzVqi3i=KVH^!>Z zhQ$4{Iup3uunoI`ixCmo$%O~$2lV8r@I!qe4b8BX@^Au$z@Ohe@%a=OZmRAZ%wl-K zRR8w~B0GG{a`AtuIy{nF%H@cya%n!j_*po8-0f@k;r(G_Af}oUSs0HUZ{{9l?LQ~= z$%PC^lHwq9W0e-4OTu#+n+Cx+-ZS&hc;|L`I!a!Je!|QE8+aPGpV%NUz;%AAAY)st zS_E#jHL1~MCOJU)a9!dxK!e&?BnXs8u=Bg51t|qU6Pa)n7j0%L+fR4<8;-Rl1+a+Z zfrXZ`86+l*JE7;pHu&pf;p#;-F%Cr4ur$q+U@PReC=f7%L}=I~keUS%%T6pK*fmvW ztfE!gG%v&kK20h#!Zdsu`-~CwJc}&%XFW$`KW{P}n&|{Zl~Jd?WcnZPX)~W!wgul* z@Lz!{>ZjVGEZE~)Z^2z&@=wcgslWwtX*luMUM42QE-vwUwn65YvK1G%E|mwLcOT0e zq(5%JK4zQjv0V|bHM`~P>|c4Ar(T${Nu{T5hUSH9So)&kd%xC({wGV$QLa{+;8eZn z2(v*;;y~W|q;Dy&j>Ej_Ow{I@1M1wtOsazDpn<%4kDJxLo$wsOwX_3CE%2m7!*SkJrB5)K53-TBXdFy6BKRWmlGwr)B{06QVJ z5G|#eN4(!VDCQkPhSK(3m@U5`1A~Mx3bMuBRJ^}wZBPTu$q3mf#jhY`yu)^|rswg+ ztFi0_nk@zUvxGizJL*baCII&T4yCj=gg3> zR{>#6jZeKGs5;!M3(2nYoz<-hPVM&SsE+rCfQrTA#FHi~g9=OH24vFGC4p+qulyXV)qE-N<# zMyK5`N}*bp3!;z(_b@;IV&)2eeZ8ih;(5KjasZ?zPvzaQq6(j2OD>#<1sK8CNl-T{ z%<3u6I>vn92nXcayRWUc??_YXy4V;ojjj*sFsIlCigzg&j@7q2`c}Ah{8~sTx>g-< z=>5YtJV-Op^~_zf1FkL?wbh69F`tu(x^@)pgyFvoOT|80KCiV0XFMiFB=&mn4@5Y5e#cveOc>eY(ieX_aTD@!odk79TEKS8YD`{>!59Q@%U*L=zp-mG z6Y{JGhy5YX7&IHrp|8t(fz1;@o&&21{{8w4abH?;vxx$k?f%LNN#GP+AA;_OFYwD< zrX#vPQ1OKsUo1u>kYR1 zf=OIUU{UTj9!$8e_KcPGJZL5qJ>|)tE));DZoMi!UZEB-pCgN}_k@{L$$TB6k~p4) zoE5WLOq?c>bb6+B6n;uEsX);FVicXVh`I9?Cfh9bxjgO~F6Zv17yWT%XVt%CA!s57v1u z289~&?zz!N?i1yjLu2^TE%$x@6#gh({BC4Zos` z&r;QA0*n6%3f~nB@A0qZtVbEq{ZXSA-^sGW%Va#$7u0S43O4l>Zom7Qp)|Um;O=(x zHpl148?q!RG5HmIWD1mv9^D{%0fewsn7p<(s`$eSi1a4VNIJ1m_*&I=YXX_%Gg*Z7 zsSk&vHqwWLkf!K$qx^=TT=mLecafL9sv#If*S&4bU_5$uiA9L0K`J z9A)%p)osBhC(xH%{~+UxRkBpn&{Fu0Ai57~#nqEA{!MYmA4{zw(r9W)5jQ{X#E=T8 zj|8Ugsw4wzd~_klbwU=4NGwQ#Ho1gB!xq>U^JZk-Pk%0-Pj|Bbl|+`(sWw34RY?9 z<}1f^p`NcYh_?eg#$hj>vMw~z3uj70a!PWSNrEf(3ascauTX=M)gKFf%4xOuO#||c z!ah~JfdVLo&p1KzQu)qN-d^_?Tar4gRAlo+ws^x9`USI1fU8?RyRHz}QCM>T>vwM~ zI{(&C%J`E~34|s{6G>mJv+2>Tzp~t85dkRHKLDObpi8=Ykxs-mOV%3+OZJ1by%u_@ z;;8mNM_cnbY6Fww$oDi=e#efY3kb!(^H zAfE3k?TQ!z3iUK~L`&tECbhkm6F@ygyq2A(qz|moUh*BDN+QQHK3oGrHf;aWg*BJpL!GBT&+nBpJZii;)3+_2*>t>u#n&-(f(TH9kDwZLT1v}3 z5md5e3Ud<*Q&Zr680ZWqD_gS*+bn_#)ITiymn;&_OKoV4_ki!uEm|yj3-3UsyJt2w z3#}c79zhIEJROw6?kzPnjt`_ijz%IX2SkR#Wa_iR{Hn->oJ?pMC0VIwW zC8pwWKAn93-uaPdp2YQ8ShIrz#Z~dw9}IHrKp{P_l_FDoeTq@%)Av;HQWQSFkdn}-0? zV~V81ayUiFgJ(@&bpN>Cqk-WVW)*F3JV_5Om_GsA8-QNds@@r`{^YMz51;j`Io6Z# z7XWSH7)JrX5d(``q81Jal7ATUdkvHk2Nb93_-BQijW36`J)Y%k=GQ&&y!PSlIibO? zB-jOP7OVV2Jip$y-Hyt4kHve`laOsX8!omuilI|36+IourWt&V zCzhOkZ&;D?z?!@d0^4A~&E{>=6@EOy-YO^qikls4mp^>*jfw# z`plKmwbkm}5$*q6+zu{bnn?%fc(L86AXP@!xCxvDHQCWJ@OnLta0^f8hu~Hl$xV=U zu-@yeBp1rvTPHSHy#qOiVZm*wJl693uLx(*)#Sew`H)obiSc#f=&5Dg_utq%m4&Z; zS)}0lL$d%_NFSj%wt|XJa1&WkJF>BFi2^MVGH9jfnPH&bcI?_#kZxARxo_54YypKf(j0Pag*c?MoQIolN*oiGXdp*wbP70 zwokxG^9p3}e0ZEP?s%Be_zFw|$$SLZmN(aRiIso)Ultn=+l*Y@oC5AC$o=8H>0au1 zHKEB8_l7*BylE^!k2Q0HD@kCjL^mk20>_tSS7@PrV$|7pOYX)nHnWK!DV$za1neDc z@4&L@Kf&0ywoD6V6(5Z2&||d-C-?S?RFI8(XESB$NHw;_-?}K%33A;Txhj~IEc$fi z@z|913Y?Pe~*oveM%RDbiy>r1l!R$DQ zFpinfVXVxDUz#j84$}}}?olIy9f^viWSp&fA7;ITXA{p$84KIxElJ87)3{YyDp$AF zO02avoGcyV$jFa{%x23G*gf@?qs+?uWsv?F2>2_}>JbQj7iiE?2v%oh8)AF9E~Uc4|;DiQ(a|GDnc4 zr`!{)fa?@28bnqtbpT)-n92clrCja#i<76cPTh)NZhcdj2b*S5s_3vbVMZhGa_KZpDvd4H|rdblMZS=Qzquk?~?(60iE# ze9kREV;%pJBgNvZO=%rpEQ^hAC%qCM zS7aA$w@;4bq~MoFS4;P^E#^gUIe6E!U@jO5bp0fZbX9V@`pqeD8ardB|Ecz@^f=8h zvky@E>GX97#sw0^NL!@zhxFguJ*?=qZiM3jrWXq3qz>*V7=i)rarU!K)j$IT(uVEO zm44}iG)Z`kyJaN%G%)Mn^qGV*HlPZ^870>J=FYR1kaVLhm!#;gelNFa$5~s#iUCD8 zyPW<-uLj1bOIjo|jO~c6%#&7d!luJA}A1d!Iu{=lZnK1TavzUgJemw8eW_Li_H6qju#H%XFjO8JEe07{4p2NuNk>?YDVq zTJ}w+XMoXxIrpNUP|?GSeeXVjf=%}G#oBZ4CTGDkZ2HQm$+WOG(7FWnt~JqXViOQ6 zN;AAIk{2F8}E0&;Z{7mM@Y4dz-8pO0Z0E9{zGz za&yu17;b4~qgt~|NsQed`dyTj8#Bp2wwku?w1Z^2D$zoTaR_I1 zOGN)D0NEjrairKpP`?f@Ze3^Q|C-)yi_#S?uf_5j!l6(OTl)5gy4kYab1f=E84TKI z*QAC45wbM8vAw}80ES3=mhn)6XQu!aXp}9z0A@x=i;Ycga!IT`Le}QY^@8zYFTILu zFK>}Ckaw&;HtAVakYPC}DEa~Fjzt?MqbWan2OZTjhvV$N8+HUtA?Lj_8zpS=f=&L> z=K_kv+oOz(l11e>o_sS{vNg8Nsq-(E44#$dJ$vMY4G+Vf%ius}c^8b3DL`TWVr#c3 zz>a}S;H`MC$sTul#eUWhHsou92Z7^hvYg;O8$2?P+KmuRC$-lCyRQl4+G@z44r zBHZlOGfk3vtkrX+p3a(Aa)uzwN$^Zf-}PLKN|74AtpQ@W(~xnOB3dgV9S{3+s=W~Q z1mYp&e(5#4b3wdVt?l{-m1!oG9kN+w{JfzhA+45ow)YCAJ`_PS3o$vjsv?HoMkPV= z<7|l>v^5JJ-+s&@2slZbZ@SprwkDoBwky9 zFf6>8c>8{-YERtzf#z1;oEW-G+`WBCrfUbx`7Hm#E`0O6$HtCT%TtU=)I}i}*i7Z2 z$;y6|BV?1Os-dA-qu9AWfm6VKBZFfZl6R94Nbb%nMi#%fJ_Z)YhR6H`i%IVI5y_N_MY#R2MKycJ+m_jpP8By=P-x_VF!4|&Ns%(`J@I=^A z`P&^)elGigy*N;TN(3h`0X;r9psW90H$`>Yq8Z5OOC#6ib~XcalC1Gs?@Y%}qGV#- zS9)4(3$rzZ{HN8C+0GU*sPtLV3jqe?&{p_9&~Ya_`SL(Oj5e!{+K zU_;Vx0a)8m*>b?lpCuod+fdt=<2GRDNzqBuFa6k@2s_m!L0fIRrv28Wjl?PI8~=T` zajX5tRteF|%KOtl6qaI#&*z73&MlSv^}9LM<8FQ}3^yFfF$wi724)VeDv#^%dOjHV zT4l)_2i06Y0Sw^sS9f$5S=`mU%0-3#(WZ3|EEL`|&{!-rLrF3r1g}kQ-XgMrnmQgyL0#LQ~qDQji-5`(zqM2{UM|ZQSb{DBoOB*qNITB z!Oj3A0EZhfaXFD~-Ldw20GHrCFzoC-UkK7LTUguC+*26k7j+O=+hCq{*Zy9I;X>$o zck!vTYX>`g4$sQBEVIN0afKTQQD?A<^3ti|_ToyRMeo-(tcc(pWPJ?%a{Tew58#Z1 z1ACYE{Ko&0wDsNx*%fGqhOEYr? z?=iP7jDByim;X@(zQNaw_lTBS%<*^aMiihT3|bGB z#qZ7h6&$_eZH^eM(N^an1W%tv`Qo$uQ#4i&HFc!EE6B`HffM=( zu@l-avAf1OQnK#1ND)X^mGLR!yF8gCr_?6;$H!lHsN>L!v7su>rT~M&7*gC4bZ_vJ zY>;pu*&*ugfi!OP`eLr43|!EgchX@(8Bvn$;nph-y>Oe{#`M@7G1&c#AlEN{_Asf^ z_|>2On&a^*%jc!pI(|{$MHz`O!f=T_{|je3q}tIG;5T0_DKBcne#8ae{%|3hqu{TD zKvFV=J4lVI3HELah()=B_$+!;4T4xjd24p(~EzfA< z^PxYis2pr=`u@S-Z4pA3`UiH^6TUL>=j85z_gaFdY~^5;FI(BoEq`G5*#pIq;<8&c z1+6%+Yf{$OrL6zT=;T|P-5zV2X>my<@>iZ9~&Sx#(t zBD|N+Rfb($6WsXq>)1d<63Y(3*_bGv4qL-HKY3fNSK@CG?!dUqbe_> zFMkNZnMcJI>Reh9QHqqZ*gc>3`e$nT4;KgWQ_K zdUj|+#}3y!Y-9RPEL&a;e(-P#Jeb5Wc`VwC(oz<^;k$J8>!@^nvy6t9->zOt=o0Fv zj?SZ9ZNV>9YX%ZFSgnKV&PzT$IofT}neb#8yNhf(f01S^|9vV^bcv}ZtxdO50@h2EvwujALb zY+V3O!$Eq3+}EN05tJ>b-TyW7pTiwFrY~-qPKnBRXn#Xqbp`ym1*40807+k1+NX5$ zR!@2%n8xf@#f}s`_gb#G5}c8M(LwuUD^57G_}(2`RM~U5!}4!?WG9+`dzd+Ne~{zl z#`0%YH|@=IeEwV3s1Ld-@9d5$@1DT9%Li`_6Di<-pgOS#r9;2;j#W$c0y89etxj&4 z7;4ak`cTv!1~VA6@XbiEGrGip*x6$~?v5if3DfWSe_POvj8MV8?&lhZ8>_qyc3HbzZz<-E-tgw|zjX7T zP*EKx%H0zj9?j=2woU!aU9{7D#5BuOBw>><@!pcDi@!MZ3xdP)_&4PAlO~+h3rh*( zWqIXBUH-e!hVq7Ai{Y9wiSk%H?D{nBPsNVgUa0PmcbQ+Q%aWsG9B%JYH`=OR(Uvv) z{)AtfCOMVm_*>0v!ck&jJdp+smRwur8#6mqXC+6tkI9Y_ujRK15&%aJoZQ|UDcwO-x9gylcvk(jtizIW0tR4LHPhF&8(7eGK#y+YZJ!_>7#f25@lcc3Y zQCy~Nhhv$kXm>FbReWaU5sjhhG6F#bXAW9h6`UFRx5ifs^www+kN86W^dG16m+JCh4n%)zP;dIchNo~iD*1mK7LKG zigK>b9o7=+|vIeOqeUp7se}w`KN~nKBvO z6R7WMS8v)XZ5)z?7hKtKeH{L){OYnKgC9m3`}nN7q{2uO_7=>K=G=gw+q%Vf%0GWD zk)|{BLr)(q)EgN`9DAAsDW;_h70r1|&RzCDy4FkKJ(EkFzDrY|m*Mr8r}9IL3;YV? zE~cRH!^;ZBRfp23c&Et@i}0AY9_>F(zR38Sh~aSHaPGZ_Cy-!#hfdfjcQa%A5J@4u zxDBi`+a)RF97Jnw6HJvl%tY)N@*51>fYsKT&2tOT=(L$SDus6_kQ)dVF!vM5jK~j8 zoil=|r2c+aRc)%-v`<|WIX^ukEpa$q$So>_{{&x$NrvYGCRPy-6Ayz2o7-N)^`}g` zej>Eja~{^Px+FY)0&xc3FJ{!z#YFww1vRmeA6Tf^X4!IkXgnd=ru{#8Z}K;< zioI`IvBHxnq9<3H9J-}%U)1r%E%%I>c$C_FdBu`6kjzruu#(S`eb81j>7GGP8qyQ+ zC`%ngXV|a!3Is=0CcSQK^lOa_>ks-R10$!Huc1HQkUl+7b*|^+0=Dhdl?ocPtL*5U z%`K~)&>s)HKy4+WDi$B- z?RSAsyg;b;5w_rf|CW!u%9U19!|fM0%>w3q*KE^B;z(~B6s%1*Ik;zI{n zTZ_Nz4o7n=HS5)@<=cyD3N63vX;^QV`Zy}9;_LnCKEy`!8vR?W6LZl;Rr)itE~DD_ zaRq-l8XGmep#9_W3fKkWfsXG$^Is>8$m9@he)SzrnZ}spGX@*G8!9+5I&|-@rLuok z06yPgk2Rf0I2x}EjO{qFD~eaxP-Cv}9S?KxP1@gM{pQ5Xm!Z9-aqG3gf%&MM=~0~! z`^Jn=Cw(C}ozE3sAvpcgm~%Fyg;{MDJv$P1NGILDuAcmA?Ka1tcih>Z& zy4E^XTfaOgF4!WWNaKn}kM#J+O!Trw%eqg$idvoyt^M!ik4mE5S>|VE84K!P!dMGX1TiG^yX9*ejrjLEM!QmH-XMwmr69d6(Co-?h-Y%u`1WM_cHpwqSrOhhw{_& zXsh;iWv{u4t4vQn(Un9WA5)G2&vH$jDf(wz6`>HlNx!22qbAAr%7Qb$M6GekW*N9r z3Qgjpla=4aOp=PsRi$52wRR|@#N>O>=<#^Hh<}<~5-YlBMV}7L`Y*1M`k^a&u8^~C z_E=;bKii65OM5$O6|=fU56EY4jf9=;4=Z*5w6gZst9>xyGKLqo-cuEN{xr#4^1Y}D z=NvSz94d>VU)|oG>AZA!ZZ5U0=Z~w5)(j|ZAAZSt4wNzGu+Gs3$WO!%zl`Qijla^|w!1V& zfvfJ>Ic1p3w7~WSn*25~pOE%1Vh)>~o)B;Pj-~x#8cWezWyd}T1@D?4%%L_LGCT|( zw-_DeF&63fvbxqu(QU~x;-jTlu~ie=K;@KZaEyvpZ0bSxeLb{#_3GU_x)b-yY)ce> zjUP+rhSDr7GrRFeEPYNyqLA_9`-iSF%m(Jq#UJ}&cP+@H@J9QmQTC9y7bMawVHY{g zdt0LyFU=b?CEBKOXKWhziRU}UEYR<{72!rIWquoaGD^11N zQ_(G2eBUqCt`Qf4M?%{=&-ClwOE>!C2*+poh5s1$X@n*}#0f&sb{55gnuD&ALtk#C z-}dn4E=AyEjxuHcK90&C%FSEgIF!ln*_8J!G-nK1$9o$Ca~U!2EefRELi(D?d^2Nr zy$)J}lk7Y6c`^EB5fX@5A;Q7vxE&Hz%bH zX3LUBM*um2Y>G=N*Cz889Mhm z89Y{3=hfgb$f2M{jbzh7IY}}*un|<%UHQp@V?@P45kSsK_DiJok>{Q(k5!}5)(Xv@ zgsN@SAu>V3dlua1k%SAcFsZYsN%jfDCDwiqA`k5sR$RZnc??~cq&m)cB3~Zvhj!Cj z*fR95O0ChN&ZTj>3K{nwq&$fES6g5w8lr(k?l=}$ydMf`O8rqJyIX};x^5}luB1| z@TpuLwCIbN)XrW2{wzin-(2E`$k!E}M%C;zw< ziR*~}JXW>zg(;EWse{IGqfCB=%7#%19o0ClkL+i^)b$RI@ljiRK2iv-`{90+|79@W z`cJu`Oei?n%( z;|{3M*BY?nM6kA>O4DazBk>R7b&TvWC;rRiyDr5WBE1(>VKeD&?B3DrpZI*CU04)% zr_7r%HkTa9xBF?|&CwEX5_MSdTy;9%R?jI^iCs zk>}#-Cw3Dy!t5!rMttnaNgcDuU6T4RFSDN^J0yo1o`ehz?5u8+u-!Zr?Rr-W^Ux0* z*y*-3DjV02c>8f-_Ql-zd8M|0r$@XR75fakJcZf|&5*WkE8?G4t>(0kigu*?wtK@Nk-8tuP;fcS1}eIH5S08l zq34wV#>P)*XeJJ6G_yA3WE7^C5QvZ1c?c}sJ{B67m3wCDKMI2s&UIu+o|(fQu+%%t9)_z~!w}G}mnv#lBGC)ex-m5N$mr_G zH!MLO$i=oZ(+?IR=ngX?amjAB19-=7R0mk%fYW4DvbOSg4Ftrh{@ za1?OjfGEh2oU-&F@gm~1NBU?3i18QYRt`5s=4ERaXCRWn| z2}(odi_2Nn-dSO2qBf?+33753Z>>E)x7kU}|7NMP8E^M3=*iym3A>#g4D^4cmz*(p zq`uH)z7Lf7EP38n_7|zV{B*ki4`oOV>v(pQOSb+PpLvkCHBo_%DiZ~T||vNp}g zbt0Y*qmG^aDJMT})}e=q=(E9QHGQlvHmq(7V1F3eB|^C`4<z7VtFfa! z;>z+KYOoWo`2tjp+YpV*MyGVFDsn}*P(3OvI%h61aO_*HpWTV~G5u~1sDEfwI7M6h z2Q2tUqe)?W_ho0q%;dhQ%1+tTw53@*ZN!9Fy{>McVn2f#9`-AuAtB7vwo%F8n@U_~YGvyj(*PlxM{0HbS=w}VT zGq+>H-6k}XcSqt`2oTkY z6Y#?hdl_a&nq5QOhUx!iK3utF>aH|1^Hk|Bpa* zx;x?;+XK(ZEiBnR(dnRQ{ia5MuG zaNY)AGgWEKa|BB7V{B{-%i#bh^I^s%#s-06)}_#-StA#tlKyr_93L=yYxCB3MjbNVgC{G2aVPtl>Ro zUCzyjd9D4;hcN1vx@mv>~l5)IQMo20u?kDAlPs;KxbfzpTjY z4Adr2CGn%ZV?@hZ!b8>>s#9F&F;AE)4}3Q6L!&p$5J?+{2WruaV2fr%2TX}GM0$*I zm{L%sz$Z=~G$uHZW`l^gF!L86EgF&KtcN)ny!1pLgio*2^~ni8#y%u%Ej%1JT$?m| znmB+NGvd0CQ)K56!26KXY+n#3gPtF^Uk9o=0o06Z#Ar;7D@3XfYh#V(W}q}czYEHA z&qAbYVK*rQdqDPUiu@4$R44QUvr6JcHr{2b&y@GiaJq7xpD)6X>V*C9^(%L}?GJl( z`WeLc#Pm_CorFA=fgwn-n^46n&dyM-u|A7vz0q+Bod%Tc7dGB#@}L>PTJ%V{0rZ;4 zp*neVwiAepJ&q9xnk*dMA zQ+yAA+PC`oBg}i9>fQsbRs%_cPL*PGT_HLe^h=0(s#9HOqaWiak`@jh*Ank0&BhW_ zfVK)!Y>8}Ox%c05iF)7ujJWvr`SJEm0M8ttI;kE8yjpqE0Jxhr=rkljH?onnGdCk4 zs=J+$K;iDbJyH=*(EH<0UR<*o&rWf$Z)omHqE=jB)>@r);!IX#9&kW~)5!K@)Z(^A zGsY06{|rQDYn(rV)a&%+L*OOYX(VkCzJM`14=G;Fi?Ubmb+YVy`MrYR!p1+HJZeJN zObWY6)Wam+hgoguu_m+p^D=IxnN<@V*!b_0=S?E*NMRdc@s#!pkYas8F>7KEXkfKp zz)EpOcOJXU0I~6;Nq-Yw0FuUlmubb7!mKo4)|5a)(7+bD2jaUdZPW_P@TAzEffR!Q z^K?5eW9Q$AL5KmW)0WPQelQ~>?F`()j9?)#w$kDqqg*@)GT$mLWc|p^*qM?TjEKs~ zI63Ly&A5%E;oF$U5uTSWG9HfUSu=tFRByxzU?#OJtQrC3grnqg&wqQSO;W- z+4-QI$?Jf|Rh>t7bn6WCW*nL*iq|$@9~X!)H*smOg1ry2rKiFgw!$`0G7Uk=T0lG7 zM|JA&lUEVNy-xj9;0n}L?8-HnjlXtOWx`uW3X6ZL(u&J~HQfJ*#`Ht?3|DH0D18{$ z+4zDRN*`D+1ejR}WHuQVfTSvolxBM-;k5UZ$8 z^Y6LdFyM&-pVGCgQ(z6{AH!F=omvZFF{>~y<9K%bMMOB&Nw=@9k{HOwXWj_%VH`oy zx{uaz>!XXPP9?p-4-KhjB1SL$I7ZRl1By5Z#O%w>I9>$AjHNmStOUqnuR5 z9j!Bga3(!V?&bmkUiLcK_9vIIbh0xX4v1qh^ZP~51KAz!*tJMn)l7#w;&mV+-*D8I zr@9mmG^;?Qs*0%1#*ed=9*8ljQ)|Bh$I8mdSjv_`DS>-ITod67*3szT1EX#~Tq!d~ zAg(#^g~34(*ATeg&dpdVi1$WpqdLi`cCU#`Y`h|*oDs#5=3t%FdVz0tv!iJ#M**7N zE-{VygI+7g$Z{E}>JNbqtB$Ji%yF{Mm}ew#3tI;}KnY=qPetP$r^-EHUV@%a*WlZ< z{EFI+ozx?|WJ$BJXI$RLUShbh@rF|^CcF?N%?qy3w#M0Q*53voTfD^X_3V>XLdp07 zCcxDOe|_yo*?*CgT0`+~yF0MIC8d%Z6z+rm9`O#`w!{4E9$%+dMM6)O3o1=`x&l6J z)9N1z-tL69Y}^^6cjS7&T-g~UtMfuA>wm*6%;p{@`XYKD#iIN0p|18yB8!d1TfA?7 zPV8eJqk*O-K2PVvVs@=k14@f#oJtw2%VBK0RkeT8>B$vJ#W9tHuwzR2x zu3*Yv@%qn*t56nxi`!QI`^d&e_06xM%p3p(?gQe8jrgJeIMdLzT7Hj_=PRP9v7QKz zY-?v`;rufQmjALTHNcd9ni6XWe$ZhE{F8YA_qbz=br8bBO@;4Yu`+txL@>{u_#N;{ zlD4$yMW(Qpa#eqKXnR6;$k%EvlW&x6XB?r_a^hBLLZSc5% z!A2s98>Ho7!jLh~0!j0*VY0MB5oKC|ZqQ&K7yw&ClWWw!k#gDi`N^ZE1Rc_B0ay_xz%~Ps+N9abfV~b@lEv+-k742&&c{SGj;Qpyg1##f6mD>jGs0APjh zE@`7NAO>=!^BCVIe#5O)nMw6gGtQ(49Y z!JgK?DPc@ppJFF#|x1$rfYYNzyD9 z-mVR@J2_nGGPO^u2w)*2A_8LunAxq7xvajN41@HNT4Dd!=a`O?7;oehOREVjQtA2k@h->@s8L3T>c` zd51}}SBWJUTQi#kb;z>rF<>7qr8)>t{CNFLM1ld~Zx$vuBk1qMbBLKHtu^o&p&MD@Hs&KCrbJpaQB>{d@gOuuVwX8+f|5-&2^CE@M08*j~`K6x&J;kRiX| zWaCIZQkXh14^!d_Dc0n@VHIX){J7uxjOYs>$CT2Ou>nc51IT8~5mF4|Sx^*bA;lWJ zLKXzjQ_{D{M+VnB=-&Ra2L1ule-YyU2c&{!uoq;$uE-a0jOye#g*1kg!Ws_43@D;~ zAoDeTrx4evPWtdPZR;{vgE8y~1@Jk|H~5`Fm{FZFx(fbBIvO7ed>8;wrB9g}^Y)X% zHW5uRtw1f=2#ckJ>;m8Zk_&GH29{tAS}=Wzy)(pLhmgXW14NIMa6Hi+fQimV&lOAh z4qmHmodavogC(w%Yw}h?J?nB_#={SQSHorFXQo=rcpFJ!>jAx`^%TT^J5cOzHa$OH zyUzN`O5T}uZS#R4kTg?xg_ih4Jias22L_nUYPhC0Jq^~N!`On!002VW+6gfP5XRpn z&xs^9{@Bz_GoC(@W&|iNJWC|aA1>5xE`cq&5}h!%=OJ?|E!wZwDZHz_mbj?t7~!u5 zp!8;dH_`&069sI%X!5cd;W#NQ8R+Ne#ju9eK+}jd0!0{$(JKHvdXA@B6ahf>mZ36< zJVFZ7B&mEDYHa+)8Ng(h0K7p3O!i;BP9-a*+dVs@>3^mhNWcV0pkDxcb+KzDq()9g zQ!ZfCzXCvF4*U`hB<;*oX>s-egt2lfXUM2K@L9Z%6qZO*eVP6n){we#OMOz9%gQZR z3ve`dq$=)Q8Hb$`Sy8r<==xqK+)APa04eMP5~%^ap#~rlVQ;+I;%*=o42q-?Rv6CeCwx_l^=50d9>3j|1<-?tmMi_c9df}5X2#p zjo+T~L-N%UV|x*DQj3tzVo{yaKZtx`QqY-*lX{Watj&2DS%2$Ru9S@7l(QKDLYg%u zE?~kvCcygoVdY)+b%4xfNjn1f0URKS%j^~6Rm_+(YNNiIwFiQn$2@~e15}&%AOJA$ zb${4FDFE#6s^&|GuT&?a&h{$6yT%WFsJ#S>LwZ>=(lNR~HMe6JVM+j7u%<_B3Dn!* z`yIQ&94!9zT5$%9)(c2m0mV+GUZxjzfOKac(~P^=r8tYyFVhKgaCz(ZwInh3szr+X z7OSCpu?f!=sWx=Asw=sUD23P3cB4k=1xb6Px!}k{60Mj@aHK)J*2#Vi@oEWY_70Tc zE{LMppolo?5IzS*>}OlkiWKl$MkRJeC+Qk~`SK$%LBgzA%8&R3i#cK=pgwBz ztSR%G}aW;`2CIrAf zy*niYw(Vch!K4N=BrW>iuo#=^XXz_Opkqi0APyQy(}o*rSwDuw9AK=BwVBTgLjc)o;UrsbC#d_KfwEnSCII z9wZ15K>!_N+#7ZaNy~(PV16g{tsuoa(?P)RUk`Mm#{f3i>URafr8?O&>IytdGhi`W zV7n!&_>|NsA|Kx}R44`iZAZIP0kOEeX_aOWYF@%k8o%A<` z6vJYyVY-xL$h5j*Gogaji{5&tPB8&u zU>)CQdY2vpge$ela$lG;l9mdO{NHPAh6U7xx;E6iOXi8}4LQ{6_=EwINdcfh7^jys0{8oI}OeJ@z~&*#S# z+jztkcdPsXz(NUZeAVQjF#&j;Tf{919Y!GgwQ*O$)&j}w)pARi4+AL}`V`H;O$-ac z62xL0P;Fb?%OfAsgs32mn+Gf&1pV&PHmZ9NwAG4SvvawHYf7ZI>DDMDV9Q#OD|{ zF7hdo@cdTlN8z{a;;TWoi^#7n?iJD&7nvt~qmB!b!Zr{=T7LO$R>j*RA38vL!vUuF zVG4ObF44o>d>HV87~1Rd?CRvih)`P;d!~NFQ~J1>mUVWLUhy`b<_e@ZzKW}00Ydlv zo9SZ&qB$FHbN{0q$pz4A0VN~3if9I4#W!rQ7RbtWxRNwAXab0yVFpCA1U-N|wz0(f zwEYTTR$y2$_!lf1Ns})IAEd+{0GY2U@7LJ4yvhHCqHj;0mf2h@}Ts&2*0sY+AAmUxSX zt&Bqy?StSlv}rig zIMQb3pUY{sc;YT3?KT{0#?v8%#S*_@qUAcbzv#e0+Vp>54F>^Pup4mm_X^u5f&uP0 z$^^fScAT}Q8BHBE;U)HMkL{fCEREeaT-i7!Fe3n5rGFxw)d?sZo%VR*krndjGl5`W z4Gypn3KzhDRe)5CVB@y|^%TNx_AEJmAHb_>K;oG4HyLz-jIIMZjLMvh<7xBN#8@_d zU^2lh^5FE@1{=m6N;`lPYa$=8901{BllTp*V?StHdg5t>ORq;gxR$7N_hZBp?>&W1 z23>}Y6b&Sx8n!XQrOV}P;`4~932#`i;9P?~Lx!n2m)trzx}q{(hI47^D1xO&j`*}K z68tu(K&&Tm3o{CeituIZ5QxG;M}eGM%E@qWicb_F3_;vApbBC%8xLrd6~uYK%)ZyZ z3~M1hKhE7OPGwKl)wZu&Tc1#YM ztkkp#I7mxuKnepB-GD(N$S8=1VRak;an<5{5r9r4>r%WVS_uJcSjCH?gzN+9Zwyt0 zao*)+sNdh>z2dKb02mqYiB7JAq^>CX&hJAm!vc}C=kV`ZbPOzJ7fhWZG6dBzaA3G{ zsI$Nq7KEfNz^|?7sVDdL+F@V4BKGu}J=avgK-D?cXK~Xlo^A#7hi$rZ55r#(PXl?= zm1Z`0(LpzPHJ}LZ%&~9@elSlU9%JM4CN~&I8cFuX9A#{zC@!G3m#wirjR0t^s`H{3 zY{YLMVRcL~Zrba9+nST(8h#mVn@)CV(92?7`R{#w7;6Bcn3^-@*&}Ho@Qd2w-IB{wQ=_Ilox1C1ivX*6 zqUW_|NeB2wxtr+n%D{m3%5V{&uIz|>vZ6H;?(0^pXy9tUdd>p93n=hnypMJfKjzg% z=!24%;v$%p4lqIhG-1>_aWQKN*>)DXWJyi}#D+)U6@buzGMoUMAPrr`kBWV^3rXK< z;Ngn=qHE8o_0MsVDM2>KnJ$?!*lUqN5O?|Rt9yjvrPp{&>=>U+pT97nKR%nTd&)s; zoSm-g$a5zD?z#v-!>{x$ryM;wODE&3GJ*Lf3MRKl*tq2*eKugWbyha3HtRB5HJcEl z*|J`-<14qm!;1SQ3hNf6@O~9*kpW@W8%hM%Ri5ZDH0@5mXt z!|97dkM$!&NMs~lfXL;;tl1l6+8u}4eH*V4}#$=gZ zlbYk%3K34s|5p_?QYRZy_zDz7LQK0#i9jWn$(N>pIXv#yWJhkxnqGy{fRI7>`H29i zvO~MpxMW)_7K7 z9zD+c_VEq43+q@{5%OKRbN=cT5A=ALdcz<7C4v#5ed~^8`tPA<1;4TXDs7|?|Dq(w zye(~0n3>?iz%PaB-;naC^k;d8geBVVh@4Njx>4EJ6lNLs^8{yUg1hET>$G8V=bvyX zoN(Y_G{jX(tR#gj8Eu>CMO^4<(QCFE_6@6lTGZaWcH(B=f_qQG2Alv`XOlnuGzWc^ z%C{3X@UqhXR8kQOg`5b4I(VWL(qk6TNlx0N5-?i&6X{!IzvDWBhY}8s^pCd$;EG;< zOxQ4Dw4Bc$|6-K9)YBR?R3*0?8DGdx5%r^)GnPj3@R)g?GLb&Ee21?{`~JaUERh}} z4{=@g+cOochEoU$!NOa?gAdT!NljuxoX7-P6dNZG{j^!ZA=d&2p*CtaWyAcyBH`!O^6^%*L;Rne8%xmhyrl`z0f+cgovK19dqU*v zz;!M8x1&4M{=~K9ilKy-2O_S+{H+}$;$^*|#ZB{SNlVtrtUF2k{_?Sp;mure8R#lG zv+s4;*P~<$*t&wT`JO9vnSBbc@R7hUfBES*9%oY*LjQHpxR|awInS^7xj=u{p)Z=~ ze|eJ}vb&#uHhA>r54Gd}wB7t_gNk>m zr&leq`jr2hMAbvu!L3QZ`JTVddgH)9{F}+grBR}yAhswq5iJlONxjv4XnRZpr%cwW zdbFH6KmAhHiY^aR@JB2%=I4{*hTFfqTAMk3%}Q!k;T4d1av`MjZvSmQJu%R37}PZO zDzu2UarkXd82e_vG~{PAPUI%LZVY8~p?t7vv0n(UB8)pq<5CUaDs!nd&uvK~7Qa1! zD+4!=zxt?V2>w?Ywq#q3?DA5(9nnm;kXZ=(AIP|>u{chtt;k~V&!M&4_ZpNr-5 z?_Y>5jK8rH*W^u*V}1_Qf0hNV`8HDD`7yZs$!L?^;WRtrG+1Fi(yx?+rvx3*iTxPH zzPbpi*xNiAxoaENS|@Jo)%@Z4u*Eny)O*mYf=3_d#X1>+bVV_}H|Kta4J&GH)yYDw zb}A11RAkTJX}b0U^*-A6mU~cLTJ15qFNgZV=IMc&&a2kj#JMcJSxtB(bmryV9CG^X z3y0*;Lo!K7aAK>n_N6V_$bC*MdzMX!#>wXSHf~)aD!&L`>pjVfdwPyU@S$SsK_f;` z)QhsG@%le5P1BfW<@w$G_7U($MNRBM38EugU`Nk2h*OVw&H8Lg*x%xxHFT>Bm?U+$ z0mKL0b645+AW9l6(v%sl*MN-?c#X+Y`9hp>B3G(9uUz)Ew-{>oj>E~YLb`{-n;`2Z zCn{esv*pBmDQ=pt{Zq&~T@;@`e^LxhRL(gKR^$%`tF^F%>>)QBtiwZL=|!p3kV7NK zDqOh>wuW4f8R7*l(yg1qr_yl;)9y~pFXTCio5@ngm&yPYZ#nrP^19DAm6|@;LiQTk zNXO;>_>LaNbtYdvPd8CJ`B;fqj%QvTEXg0Vu!%AV{-{K@RZaJ|1!^bXO-=6D;m*5W zI(vgcjr9mJwK@Y{Y^kdAdzU{JMn;#u6#U2W!hE5d*r1M0z8Q0kM#^tTH4LeF*Zo#s zOutCQjtxV;D3(nc9hZW;OJwIi_xLQ4ySmvuq#!+cc|qQMbw_fcG?)<+H8V$HDyGkv zh?9W%fKmO3<)S*(y!H2<0;^NymYbW`3?@%0=g>V^3t4kWUNQ=0y&l+Mrr#s+fy|wV zmc*tX3iVa}6v9RkSBbJ;P_&TjIGEw)I+X6M3J&&E1-s&)PiecLr61Fg!9*;3AZ%JAV#I~RoaZYZ9DLKToy>4Mq%L*zG5WbQAq_j_0 zCI8goeE*9~xZp#YEjPcd6xJ`dqRI*rBKJd;%1aD#oU~mkKd!J^)ltrG@vTxBr@N&!5l-qPg~V z*8fCCi<1-hJ6y_BnC<4>;ti#20VCORj4O}?!uUI!3219;46M9=o+(UrvoE5Pr)5nE zkUBIuZWlLC>~!3dzjS;ibHagU&uqHpIOO)J_+Y5O#JHC-B8~+7-q9vn)eGkKV?^$6b#u@ zq5U(xWWIlWNW)(}H_qiI&O?fhyRlklX)d|3{J{~DqkBnTePI| zw%?0B7>sShiJysH++7e$7ib-iYsk511#tn34sDyAIU8Y#u8L8Vyys9w$+5RVVuuOP zOti&;>yMrBbWi%~iq=p6rrO1^gxx!5t|X~{8xZTcR~W>*i_9fXUxKW0)^~$%DgsBy zDOG{I@i!L00WdI!4xf>_&l*WtjNMTqWk{4O331cDl<#(>h+n6Ac=Rwsd{4Dqa_=*) z86jIxnAeB2l2v63pjfQP;!k|6(hY}ht}d2NS634Bm4nQ8z+`HR8To59jO+b>X^(PZ+3LGNjs!ae9c>?4D8kJ;KxcPmMRv|mpt-R zugWV`UiJgZiiBd7i>P54J14Et1$j(cCgNlh^6{HHG-t)f4|A= z_qAWJ5!H@4wzXtE-0x6TWxE49-G48a550g-etM;dqa`>hS-d6_A1nKH1u`-tX@A_5 zTeaqnvh)7ZU1wATGXA`wXuKt26kV*}KG4!&m?e4|GaNhO811jBaqCT7yrB8TTm1fv zE0X8zivGu@>Ft6HRk9d+&tW=bcgxQ46a!}u{6ZMKjb<0NsbTZzstD|ov&(azPmOJ-&@A+_TmST7Ut$Ae^(xK+ViHtY%LI&Swhf06H^iL;kAyavw-b5i)roKO(#{m% z?U&Xh8lra{~ z`f)`Qs@9Ou(#~nVFO=}~xB?w~C-TBobD9y~)`6c(%P(X_dc z$j_QJ{oP2Z?5bcn?xSw=R!ZN;yXK?aAKJB&rw;dj4h_2fhXw=>PpaMZ!yTCOq1|H5 zflI`-ky#u(12ytxH* zZn*C!*QR*NR?_Z~XJVC!-fO(jXMNsn(pb*jrJM2n4?m3qs;K#_f}V&clAfV<-bFPx zpdDfkv9$kIUUB6i^o95hVcn0$-HWHMyx&>f{;~oY;V0Qq(##HYK1fV}g1C}w5sKUQ zX&7cXQ(@~y*gCw-*m zLg`mj$42gG$!=RiR&KSrX*$ds7fe`U`1$E5Ra&6lBQ0yH+vYDL6zZK;ac`N~p&gmIwbN%!?9A79l*tE|Q_QDoH0Ne;Z8|_b#?l@g8hUxf zzjc<0)6kHnj^X&z47y`L^+;uMFZz!w(gX`Lu}OOz zCMV)*uAVa3rr}*;*M|;-SCR&&Nq2_}x7uQsxX^~XkI#jk5n3E~^->W0aHVyPG=;8m`^98o-9$4ar7&t3`s67qi=r$Kq?^tbEot zu@=L1WnPo#vqX8N*z5nr@qc)^OsH_87F#R{&Ks~Ecg=pQ`a$kk(~@5uZ~M8Z41MA` zBOm{G)>PXc(n4IEr{gu7-!VU&r!zFm@Awv0dR)8zYtFHVbYaMmJT&&!;OWX0aCJbA zUxY-6ITkdmYI+fC`t#i9-B-kwJ6>x~J|8-SoYmr9O>sW&+3Q6gW^A84e)b4cYo}Vu zG0TxBM0Y|24C+DA>7nW#`*pUTE(Ww7Dh(=%q>X*h&)43%_?t|$M|f?HsZOe z8Aq+f)QL}coTTFVq4k%dAgv(|%>}(>CM)D8ti2bM)$(jrutH2dFQf3+-^b~1LrKMmszAaM&fIKm z$E6{C=4(%qWld|;?h^{~-3oFAc>u*x{yk{gH^_J#D~{S%LHd*Q5^@*4SlwAIRNG5h_v2pe=&dcm`E zukv2qlA`53b8v{8eysW`%VFnH{rw+vvvz?;sAvAEpr#f{r)LL$JPIZ|IyN(R23(bX z3+XF3m(rpmL?{96Cb~QFfH4?R5e2( z7f?I!vk5oi_u&;n&-luUI>isBrQ3Atxt?zq_!w>eCZFLQIYoce`pQo1!*2ZE6nb$c+W>*2s<8)?ad}t%X#}Df5|&YltLw$_d?J zqZV3A?65nV*2KxeF2k;^smoIV`5;|DO(u-K)mF&7K)xVn@pF2V4QsPKcNxF+Dk%=4 z+EI2G&gjmA5nuv<7*3d71)bJO8bxh`i9D8jTqsp;gXj57Cm|nlJ6IXWER-}o9tdv5 zrXZ8B|LNQBwSe!`Q_9}vidkDI7Q}!NCR3u*Wq?U0G95~JKx5frEwM1`N36hVUmIxV zN^#Y#2OV@L!6w<8#+DrESyqa)IY+vIoPtb7IUT44cW1E7G28!(82*DvqSjhd7Bhds zl=6YQrwz{F=~jNCbmL{DlM~>WP2XUzPoyqnp(#hUXe+00Ap^egupA*=FwguZ_LT*H zp4MOc2TaT0CsJxMrIbF<1>8E}F$2!VRfYoj>!$R7m|*`>JIe|I$^HaDw7;*7m4Y@quebI?Ya(QY-`ITa=dk1a-jb%APPu~Hp{MG}8F2mX*1oW7wU~hXISF=?9DX3(i zp8asRwh5Vu^dG=DJQM_;pVHPW$G*^WOlhDsO7Bq3`5QxZW}tdoH#9mm{W?+FfZdz$c-SY! znrcbHk*a9FXJTz(!Wl}P_IPUpr@j3%;!(TILJYQE9kK?2jj51#2w{WLFB8?ZZ5MSHM}0lAI0rnoB5Ey>hufK~8ks#fds(uR zS^k(@uqi)Vl!aoM^BrK8YzzH&1org;`C@z3;bYzO@EGRWIpgVAA!`A4k4!4``JHM( z!H-gJOAi|=K)y5Nt~)-8XX9b)wm(C9h*EUro3X{T3hM=U+w^m4_L7{KKU)Ybu^S*X z)0S0?W|^I?b^XMZG}6MZU}qS^Y1`jvJ^#nKaW`Ma&^-p1g6=Y(JNzG@L{>7K=DLk? zVSr_VeaVnHM_^h6p1FhKqtZdk*w#y@bu-{mpzvg$w#t^0F&jF}uS)tbV<>2Lg9$1w ze2^CdZz{thJ^%G|V!z7NSO2ub9#;Fq0l7-R-T-Z67QPum-A@-i(}e;p5($hu=@_foQAlNjK`*t+JqU(eK9 zRA}2}-l>CaClaOq7*O!4MA^1R95JlYT^l7d#g@*EZ2G1yB3Us5vzKQgOLZ)u@BzMJ zq5SErl`!D}l7=iC=c1$El-;dPIO> zCn;yP+o}Z0=9}@~=u|StZ0d4&(CrS-pcoI(p$B`(p&0(q0(&gG+#2hbG*f!S;0KQ`$7LVnOkGB~IZJJfb!PpwL4$`MeBlG?eC$c}dEaiR zm3|vps4EBGwGx6-4FQHoQrLr?EWrB#f9w!^?W_N3V=tOV+^GR=_%fJaXF*KylLT9i`qu7U>RTPf$l=n^eD3oncj$AAJgCBa@INx_q#A zRSjMc0pGn)X5WgUstcAN9a#pG#<`4gU+;m1#V`ZaRDsV%*1j0{Qsl6YGnQ-t zzth9dTiQT7O6hp4V7Zv`ZNgOU0(jCm%VEx!G*Q|Hsx8xBT^05}d6MA;9j{wB8XEsE zJTx3UG(sb-OFQ};7T_0tA-p^?c#Hw~do@D#2CPiY*%QULZMSupWoAd^2}j9yaA67R z+TH@u7&D-tjhxByRbxej0K-udZMOW!7G{d{p`js16*fw)UIT^`ohbV@h+oJym)jnT zlhK6!|0s8kQ4Ei(>4XeeaEF*xY5oTrW|_FP%;nGm!0QGpbvl(zmkF zx}K02Yr6SCkIoNN*IHY{CGeSYm6P-0-#PR!A$wR{bmL2Wv0cC=f4?}`jD8#drolgb z9&2hbnt_bv>e^5YN$e+rd4FCLT2<)FtvOO$jcf>{iKLCO!=5s8w?8OXtP?z&;2i*K z%}+sb_`SoP0I~_|INFUEea=gLK*O7`;=IR_>WZl&Q5N2SH)T7smH;kbihQ(j6-ev> zwNd7Orod>F8%-ng6eK)=ZXh9J-74J~kngsY-C0iKvOua@nC^BV&(4)enklh{3A|vM z9$-{HP8d!Tsp_GnFpJP~Y{K$M{a0`{kF}ayx0CM?fNI@OtK#n&?Rl@ag%sbTFKx zemJ*Z$FsAF!%7${Ka{+@MAP~0DtVYne+r5NBbiODIj{}0CJ z(9S%`?$?i?QFVwpsbbbNEh;3`b-9q1N9=BTM7{g(9u)=DbNH#V&i&e_ zX3P}mcLV75&53+~+N$j^mo)*Ltp%*BP5N(_X5zXiKTSWD@OiGJz%uUOpe^S8L$WSsvL1x`F&&cFQ_SG2i(c%)E}p12%z8#e z(%lezZ?z0uw-ffkq*>%l$a3_H^!L1a+Do!Zs*2AEgs0h*^3=}`MT)~e@pYO>&&r)^ zL+dP=j=kX4a4I|Y^MJZF|53mStQ~eGYbUIGqEJzr?)oz9aHuC9*wA9e%#+&7U}~+s zSWz0(DErKJ85Kx}>vfq@%xPWxdwlS!04JEP?5B$zduK7WD*Y2LQI)s7mc$?N(EBM9;zXt*F)3ABT|K9M-n2~tnKU^Wy#CSs zG0M_r1aULyNwaOSu7(vc1(s9$>nsk`Wy`S1oHp~NLQcJ>&44tZt#Z5~y_i?=B6dDA z<|`O1;5Wc7MkZ`*M-Vks6MiyeG%B9ilQ22JC~21!_Mz0I#*47Esl_TCm_79vzbrw* zU!FsT zH>nCM%|O1=nZUmAtoklP<~?hTFOza*O|$)Nm%8rDQc-f=Jl(a=?BN=CB?cpuXIMbG zK=ql{KxmA>55jF6GyfA;=|5!EdoEfcbEj18=peR{)0kot-6Vn z-I8eq0oH$NeQltXqZun%4`iHSY8LFRg2=7E864`>eLpF7o{L9_*{&q7NwPh**p8aE z%rNrjSkkedK0% zg`nk22DPaP8yF9nSk`OL>)%<%)K8uw7eze%s+XoY4n`WscU6mRxx_IFae={+@G&t zW0S1}c4zNwCng$%XGAd-(PVK95ch!3Y(W_rMjm8vBq&?xgqb|;f zO`PA}ulqT5(f&zCaRpQo z2PlIL+qgFPM2smeBviL@iZOMNbdeR$n*D(Jk>*y+l`^pNa6tqe8T^a*{EaLHi~^dP zR|~ngYIKJFdNH4tn3Zj?>BiqMJn^}R+SkoWf!;Xm;t!~d8BOxOPLDEIVBzSCh(p;v zw>iNV@Dd`o{;UeTu9+$P`d?f1!x88diq!3uX=a5$3^~5 z-1jn}mSVNgA+AS)P4U4i=h{?7cE;qh#1r-OM7FVKy6|?QG|9=>GZeE=XUtn(#%K4c zG}BJ%f)I)Lwxm-M`$0*rDn^cemR?rf#7RSy+uEc_ub5;r4t1zmHVs5SVf6BeTLns? zEU_X{68q)jOmr-DxJfm>3A6>q!^Yrp3^r>V=S2vM_)boGY zy^`V;TC-ygdDA^^k>@lCx|`-RV9RBCi%KR!2jVd14rvmH(RZ9DY)B%cVZ0O6EyHg! zly&>YyWQ0>pLP$sYh;(N{6^u8Xa^mPZ_BTjQ=RR*S1F*Xenqtl%a$CUHlfs_LoK(G z^K;2il!b0l<=lztChW~udJF;?QI5yuno$vDWZoL|a$qCJh6+Q_%~9J>?sI6vn`%$vcGX1WWkhP4NhO9~?Xjt9Vu0rJ0 zEyl%Zn*)Hx=aPklGg zb4R>z5-Kz#4A?lNu@+IS3MkEBG?>KL;^)cRe{Hq0wu_m?a~G!haeAviTR6MvXj z$rFVyMjFvAsEQbQ3HR(6jko1%gCh+&F=hAf`JX0Sk2{vYk$QH$BaWywA=()hqh+bG zjY$|&T7&YjfBQWJVqf~8(UI4K!rQr-73XTVN3NAoyDC({KpR~#?JK%N!s7WZ_Pd$i z!>Cu^?$B)MVtuU@Y?1pY-%RHgQ)8LXOsqPGn{jSV6LA(f_3PVmf3Zcg)`n%O-%Hf( z6HOwE8M_WBl_!;;x9KBa{K=J5dc2ISupwtiklhaDMJM$6v5+s_0 z2y2J--VJwGsE=WWc)GkXfvb4y${kE-mt$M_Go}zEp|xDmL(c8*)D9 z6h5(juPR=q#0&)rJB65`U(xPAOLEu4x1UMxa-W-E8lg7LPEelg$bdOxfceV(s`pr@ zA>9J{%znjZhlexM(Z_RzkDM~Or-I+{bbZMqT%$MD=RD!qb!d3MHAd+;w~jDIe(dMAI{Wq%$H`66{3XhzS^gp1vW z|F~^7SSmdnf?EYzQf->b-EFwi`cF4X_^yVwr!CS{G2XRItJDk5SFFM!3R0Y0+{(1l zI9XE5N8a(9KD-Y)rWxK5;%{X>FzcevM&-#`cVdk>BN<^{EM3A6Bxv0X{X>j?7EyN= zKoYO~%fMRI`RL7tsg9bdU%O+UIA9#LENhm9meJ%}5@=WaM^c^MPf$IDi~(hHKwe@R_pAA-G=kD+Pt#(yW{KJAt$)7+om|{hXw`e{+5`ssijlwcZ|3O=h^#)gJLuOee&>=xbA56wl-a7 zTw)496oeW%C^4sddsOL`g$d;_R37gCd;0A-8eo zBZ`UZKJ>?<-1Y^}X!mEXgMq{8nPJqp1?W%B;DOfBvqYJ{=%o8`ORO0+JcW?x1G8Aq zp513vMt;{MC{3jQLf!d=KzcpuzJ)JVEl`}8icaBMAZ`2N=6M>w^>(+j&SEYZ4}FSj zSB=iPYTA>SH!)sZMLc2h3E}{rjAdYD#$%32hBXSYsb+3`TNKG|#)rT9J2sQ~(~f1B zTdx@xV+`HG2yo@3_paJgx=mmzVio7RpDVwgV5c8BNeu_SL?|lo|JjqJ9rmG=hZi() zcxT1e^Tm*beh6shev4AOs8ZMuZTSn4Wzhk)$k$rwbFHzgQ*s|Eg5szkX+<_e*^&-G zUT6)08d=aDyWkA81mq^^2Y@do#jrZS|r!MIs`zi6Gl4fD~_ndfnjRN4D7atM@+~=$Xh__j#dq=aS;eiq_{+o?G^?+_*iZerc--y_iFod&tmXs3lEw-RV$rH> z3(UK`F0WA!O?B5@Tthjg)Z~5_8|y{m-LyDt+QA={LzNv^PNI087W;+1nP+fN`?pKN z4m8sLq|UAFIv7~JlNvd(ww;K%y<2W~ z{M-h#ZQqH~=!!T`&rY8(B02l(bK7xT-hrwoT#P7XTtrPEdn)=%PFOvwow>p0|6y=7 zVCU29&d6ub_UEa@55+~V=CZTtyQ<62iH*w{38B7%M~^YAS0m7Mf|!ahg~G&p=`;8I z_qu;~|C3(Z*44A)jWyc7Z{k)yXh|rs(gYTfb*wMxkMS z_X*R2cIXw_B*RGQ$g58kNFOABRB8X+Cu>%tJ&*m4MQMH1Iwne6Q%6~?8Sms7Q16lB zu}?X6)AGO=n>iJc+`nCiZ5d?EBV)eD?x9;MXkQE=4CHz(`+dhR^xr!)Pdm~(lB|yN z1Iyy(&z;Q({6JBzL)&D8_SPfJK>T~^jeh)}39onN#~I0gBO`J2aE*^OBFtWU23!E{ z1}yt-|GGonH1DT=cuUKr@q~tL&tJ)VP>rJTk4bu)i7}hUkS^%dxn=0iZy*g=*Jq98 zaQu%obbkwU#q-kh_u7`r=NYrd7kAZtB_cHy@!P(23}#fs@P&sJpF4`h{U7%RDpC*! z8F8vQ;Ic?>&~EFyUHrxBI`3vqwJqu?eE|-AQFn6j=0?iDI)`M6N$slU(^j>c{$mB? z!VR$+r>MJ9C7kEAlj_;(gD$ls`PqpbPar~dpe_k2* z{In&`u>P>E19k`u?(>NIROC78O*O2|!o(I&NXnn~Eh1kppbh4a7v?-NoanM&i2Agh zx!7QHz4Z&hb?QJ$$vd3%*A4aQa+10%+2^9wV{X8IXLWEOmhWIf>yS^l{Z|-~u8#X% zR%JuJ4UXq0xd89$>D$rEKBNZ-Wb;>oO|}5ZTof_ckKdQ;;surpS=0g6-T0XCpw^0~vXBqmg=5P&YD!&^5t^>ji+g}8JT}W5 zZ9m=(oU6wtvE`KDi}TPc$8+i(lPEnwy-_!=*O?aZL~%+qsX`oD?`Xsd(gYXqH<9a+ z%lH%O^!MdW@-1DaI>(8M1VcHSoik&aC@cCc{4mFfw7O&G@tc&flDG^Um}IXMhka*@HFHP9G1J?RbKiLJbq7o+ z1@lnmL}TIvj+5IMWklGlHIlA{U#qpYH^8QHBjjQnLq)nV+gAOo_a0_99cp51)6K)8 zj?;gpB5%Gl16_c z!Xi}e1iQ&z*qNxhX>|dVYURi+%=kuhCwasj^NhN1EUfU|{pPF=+i%T&=U(*p@AlN3 znB#9RlP`d@r_-tObo)A1aM)_mJD6zeYi#YrU#-sn5k93NQo_(w&$+Wvx>#tR4B4<`bgiqh4Ru38q zx9M~rJka`k5kmo!ziYjLbD-1{`5|si zvGcXZvaxPVD2Rhgg_e!y)-UfSL)To<4uB=_B^s(5U&O)5b^H30!szkyYsa=DE4Ba^ z>q8r1oj9Xj4iH8;N2Hv1;`NtEZnpMjBz;nH^&k=bHYyc7cfsz1N-kJh$#V`{gZV1S zbsoCi(mfvN3pXt4!U4LF31c2kBP&4Mhx}`?$ol&tvR&~$BoGH2ug|IHnkuI&aIf{f zgZ!Ot(@o(%{Lu@NLp>bir~hJD78iDj*_Tmnu(^0ob*tT7AGm4Evxcimm^O^Mt_gSp zOv_ew9SnGT`}NFfo*Ak>Mp~MtSdl3q#n3-LrrAPvJVXJX11=th@}`b#rK~57vnUQr zZ&M{w`;5Qv*Y8gB;Ft(peasAb9@>;;f?4k1*;mP%euSc0ZlS#GE2d(?&UDy)iVEBV*Ls@0bxXViydEa2P;r*G zicJ!qf*WG?DrkkFOd-k$!_N2;h|7G5tQn?1K0l z_z{>Fq&iGU;V*drw`si%UBwuZJ97?h_tfYgR0wy2M_PLBzuji)S&fuE(&dG(O*OL89#2&?8^#&;=82K6Lf>FAIiAB~#y2I&0#jBZ2x-kzQ-C zoxsX;1Y+cEagVv;EIp3mf@c0GNk~`ViI^9Eda3)lX|V!XB6Yn!lEa17a1s6q^i}Q; zqbT?_dfNZ*pxV7qvbgX$0(^jiBkT7&YT@D~PNRE$I`EC#@)1>K;h=A+)YN%1pYs=j zGT?n8UA|5vbUDV~4qY~bQ2`98W(x3p`d%)`abw?n0bZo{Pzw-wAC7o}})m)8txSq@}7&&Q1Hz&+$tz8S629B8t3%6pd)sr%e|NL64{ zh_b(+DI526DMYp)-`JwxW`Nh$_=-5+w%StH-;jxO$K}ob;=?Uxuy@GqQC5vPSxQk7 zULbGLNV(T%vsYc6Q`Wrzvt=U}Sp`vj0U!8&51%p3OX|C?DFZA7)-XErxL$rOGa$DK zMzAEpbwoH{ZBj6GB}n4yB`h7xz%#V-fWNFn@zCWz(6OGQ3#nEPHBFT>yEy;aSMok< z{?_@?`BK5Xnv7TWZ?egi9o(x&nF3^!Gv#1S&0ysW*=_+X+O7FVPA-DtWYNV^qUYg6S71oJVh%h2b&R1k-{8*7zjKPILq# zB9mja!Sb5kQX1PzK#0x*G4G@35g!YK_gj@EJa%O^7|58RZNl@a1W&6*kr8WIL81hc z5J=Qe@Q^e$$Kj8%xcx}t>k@($=PTW`lmHw8j-W)W)1iyd1MJF~+o&5(!mDrPABTar z5qblcFbgdeRdYX`Ff<_>{Za@rr>i7{rl}dZtzZ`wjh?+3F_eojFPQdCD4~YW!y*3Y z;GEkN4n9=gkC>wWJ7YgCHzw;zzl1CV3xPBBrj5Qv$Tg6uHTIy=E@W<#(gBB>^E>}bCqT7jvKRQe&a1E zun7&47Kd2kvfK@8HEw(vnJ!?utBc?CV=RAbKpR|dtNbq zSqou*BHox|qH`c(1bRVABFhN1fMf#?XC68-N&HIC2?MX}c@&WI+SzYT-**V;JBQ_a zh(o7otGEWraA>A)oqiDh9(EyWfzE@tLEv~Ea^Qr1rPQ+?=o1Nd^)T8>ogK=MXphUR zrc7(yYF7bKf^{H~7)Ep%RZp`3ug6njGOdut57k7qB~Z_hN!7-}^MLPm%bO)gw~&;S zFV@eseZNBaCl^tFgVRcb2W66X5tClbH`LzJf6539Z|)dP%AWy_8Br@Q%15`$M>mmvM>AW|w1%kX zxl}khimRf%rRNW`MEs^GBfcePz7AON<5k=tHs_|#m{#+Bkwc2aKyV_?D~tGI?EC>Y zIjD^SMvh>^<6=|haAYj-v@3XOnp)G?>*a7R|8L3aT;W7wfTo8CBiVxfQ*}jC0lupA zR?5$WuZ%JAzonERb7XM1k&Ct-Dk@3 zjohSx^FRC~!;$_=qoQ8B0y8l9{c@m0P?bna5er{pO^9YpshuY}GE>ZIEW9>d{u+N? z2}KU1x5jEWYC<)_9ity1!cNApSlh`zfIM+E{$tH@fo#i4+%%`am}9EjIKhtRsof{V z%Q2QXU9mKuHuyVn&Wz6A2cUx_kx{~X%$AQ7g9my6^w@G(!hc;rs%@lePda@A8{hAjr8pQgRFgIXj4Tl)2ym}g7f3c)YUk_)>KF{Bum~EG(*zTF3=Z~Q_?gP0S5m>GD~XyM z|K7y&o{Z;L^bx?4)V;@jSqf%97D?HpQ4sN2p|6XAIe@Cu=Br^o)-TM2vtZ)-4Y3mu zn@}%WPOdh7P9Ik?gw)8_YWg!w7;DpE-f*EoeN_>7L-v(48dl85hghN-V;n$&+T*MC z#A$mTw_ROt04;>A>Dz?FDWTGR%-|oN{o?QRcN%t+?Dd1u3@NGY^t{kpz=rX;W0?Ob^)%RdVlAF=>P9@qcOVf;z8(AWiO|8+O` z{VdkZRwK#6D1wh{(*m7|VqGO~pREavdOApqY-uRO+fZq@+Yjh&ACN26{%UK0UPPZ! zubhAsdR!4!Fn%8>6z;Cn(7S~)YaS(tR48TqTG7S9U?#c7+wPK;4L$)u}!d( zFt-zJ!llSA>N0faDKjfo&aC6a z*zMe|p^a2Gb>-0z8l{+##%>gqlXSLwSzlF70ee}W2U+{#r^XFuV<*H!tsZuE?kXQ} zueYH%i+8uY=GL}=hw}q&=3KYtCOqFwJX0aNAI-NYFt96mXZw?|6K@Q8ywkP-wTw2n z!S?r@MQ%+!9$o1IGqA3YkesgzqT`jmo%#ce`2QOH7MI2O;E6`ef5dywLj=5whnlXM zcnNJfHap?cOxYk{ZD5(M3ZNbr;s(rvB+FoM86ZnCf%&>a#9-oTLHl#;_e$QjsfDP9 zkA5+>=*YVf)8e^ACt-Y=?oyFC!4ftwo_$~iaQi*=mMU}u+FXi&L^v(QVzX3?%7s42 z;zk0TbL>Vd#kdw@xK#c%|NIV;qw@L7esO;<6ibQyfMV<%-{zy8Jq$fUDB3%%=D}&6 zRoB|3%h=#-ow?`Nvm|mQvfu{Rn}SUBIn(0YKtD#IZKGID$Nr9vxDipu-M%PfXD4XV zXYJ%!VuQ^zL}hjDW<;IfH?;Y|&EPty;pe<`g{P?hXu=M#Y?vz?!mjH4huT~otop*>PkMd3=rl~j=6^4-Qp-=0-_obg zXvor+Po>ClCFO#VUfzdXhn3DDBZAR_TyqQ=^nl$n1@QD9u&!Y?@;-9R~a zk59!liQ1=9b2QUz;n&?166T-l^YrPntWq(|G`U6kd?_bD@UGqvxL>ubyimRoxfgkD zqCCf8dTKA-5BGbJ^w!!rC;k||tV6XW47x><dS-a-uJ?T4 zyP0?9bxY8^ zcp~MdOl)>{F6{CVnPj!+Th^FOIYa-rsp2r*n%%Nj_27+y{^lV4FGkFNq_>uZ(Wr%8 zo0~d1!pPMFP8;4(WDLW)$jRC#>)!d+<{>xoi<-XiRX`ZKnN= zLi{X@P>4C}BV!ojqzYN>F9l}0E1xp;&A$z~0Y^;Wo^4^a%D5NV+O^BZGcnfV_&xN3 zD;nvw=*5~(NtCAwLV}()PCzJZi}0s3q7w@TcT>y{1+M<-PS@l{E(UJ{hwE*d9Mj@> zi(uE$_<#FgecL>YE$s54fcZfzN41v!kdNv;NogXhk@J}5o{!Fc^9#@vHGRr8+=^jR z%WCp<1*bHU@tm?mC*1Zjz9TcvQ{p7o$43Sz>?hnGHfaByStqHbbLttEV0s^iqmr00 z3JDn@%F~-BiftazV+7@b6P3LFkj1cpp^5{h*i?7zC55EZ%4NHr@q=^66P$r&SjlM( zXRl3r_waUMyBnwS_Nfa@XlpZ&xQ2^-kKUomv66sixcVNR6n2`)SmOHcEE7&6Ee$rZ z)mlx}l8~Dcogm1S@ye$NM=d`qRjE1C3FDe++$|r|c-N;}`XPH~kt=ke?yx4Xlz-^gDA6q96B*AAG9iPvW&==^Uo`p=HS{)DgZ{y%{|g_e zuEkbN_maNLLq`yKbnjcW>!^NP%J}+6zSZid6#ls?-H8#=ocs0J7;^PT)e>suWK#;T z*GqKS3ikhuSSuZp!&tKqy`~b5Ep7=&0FcB=2a@1ZA@Osu7Je7H2y3TuY>O0SAzH+@1zGnMD55 zU`{5*wcmxvB(FymTup`X4T@A=NAme}63fchK%$UW0jBenlL4c#mHnM?&y8lk3vl_t zR0I9XZ*m{$8G8_+RG$!4=Nvhx^AKtD?6ZPP#K*GGA?hmBv_f*o!qB<%mx*OhEo58)t=!S>U|lo!Ti1Pb<5wxkWcT^`pyIFW?We41da>mUnCV>4CBh<~BN%rGy~ zj@lh9pq-*n*dNiNESrcXt`4pJG$$1l$y#s$RnCU{7p3FSUHbh@>=S(rbrt9JoaoMe zs0ivPxxNj$;RE?XZ^Yg09eUs-FuqunDRin|el4?t^KX74{zbIvc=b-s_m$-MBtop} ztEDbtR=ETH>F)C1e$YdO_fQ$Ezr9E&G&dsNIA4I2ViwOQ#k^0*4gFHmG+qM?QUJi*H3f47Uq0CEF7x zQKFr+fxqW+G|~$&!ktT5Rcni`{tVs+*uwtJJi1s=wb`Im+Mp5GtccFR(bt$?o%?S2 z>brg=K7S@gHkBtCJi*iB9|`Xl6(AIYfD<{j=g&8<><$-@vaPuW(IMb(vPjkCAZaoW zwZ%VA$Qgxwr@!)$g7c4c?5VZ^r%H7)Vii#ZTi6 ze3|?*mPHBFY%E44<60=_vvUdelF)_+CWcMk_{XQ>>k19F=xr6nZQPL0)@AXB>*guv zmnEcxHl_pNz(e3~UL^&k^&?thj_MbnT{)G#9%&8b*Lvg|fcNSaC9m`FgivD{p6Ii3 z6&HO5y7Ry)NR)O~Xja*AY5PsF;3T$oWyp;%hX@4Pa4$mN;m5jjEaXzFuXK9xZu&01 zC1xp^^(S->c6Z`5kqTpDJ$*Lx>4YmEdkx!V0h$48g2d@IO=r_dq;NQJ*EqzaiG`Z{ zpC!UlTBd}edZGUY?J%$f-RDf}dcBx`(HASizUPq7#Ah}^DsKv;F{7>uvUNCd1x5zQ zuna3H>(EurU=Y!XIOQ*L6p{rdpuZvjCB;Ed7E;ADrOhWLfi(rt@1BGN<{?dDz>B|0 z%sKtl_WcUzC&UOS&g0MuR5!S(YDJLk?GDn};c(%3@Hsc%U+fRE}nGsQ}9QjFSwNz)jh zdgEK13+d;OExTcV+f(>FA(iM6c%s_K)2gA|h+sy0;9jOL&V|$LvB_3KToW>zc|*+k zO4%*qI~Tx_Wdxw3v>?{vql77XFSxsD{)HgvGu@^THqk-9fcU-uE^~MrXviC#wj=$f z6dw?sP<#}qH7!cY=Td0r9DgqIsmyi*V||zVHs)c~u&~M(?mU~ovI?nf1g*yD5AvNe z8Gk|x(1Tr-n#_J+FGAFQX)j$En&WUX=>zpuXGwxB|28F;h!<6>q6NF4R~67lXw&t* z6`Eb32h1&%5!G?CL~M;+Md{h4nfb@1Z(@^YdZv2{b{>s%CP=ypk;KMq!&nt0hnlyD z@AGk^oSYVA_~3rnuP|Jz&;?xx;@6Bf^d(6WVmG4S->HN>{gI7)2SOsO6TFxdHbqXE z^REVGo8=b7>u+SQRvpLXVAqMRzM%8%uP9e1%rR}RLxw;LT)~LiO^qNN@QDLg7t`=l z%8!-}$SH(?(@QmQ>5^K*6@^-o$!RgT0*b3G6-`o)=BW{Ff~VJawyCxFr0<9aaU@2x zZwQqN7qH+{aoLpTGFt(CBc5Q(z*S>64!LJael{LcV_QKxupxT_+bK;%B5=>Rh9iZo zE5+`6)Vfcl2hjX21;v=3ck<74fF~7*+cbB>a^=W|DRyYQ)^TH4-WAdxq*UHXPf-oedy6HoI@>LTp2g5Yzcdlas1LCEFWbrS`_pQo(MF>rBD z$wl4afzL|d%Vp6{c8|3~U|6l2M*9}&p}7Sy)O*LEG?o#~L}yP#3*fYh5#17o!5q-` z01igLP|0FrL-{Z@Bcecb`^qk=_FV{m<{wd8mP~gIY}mq~r68?t}6Em}R{S;8=>&_5!bJ5a;gu9Ab)7rMo0_+0p+V6;D$PQ2mOH@i2hf^i~ zRsb;QfX)=;B{GunT#nox$TDJFUo|?&mv;;32bHy}YgKDGj6`VLko*(y^aMwG1B-2y z`v~|fky;K6vf~ZwNh29gkck8{oqB#q#4)(sQk_Uhv@&LzI6Sid5lGx57)6vI&A|pZ zfBLGa+X!n}U;&4d7=cxNx)E!!!y{2#7<`fYS12(y8#0n^hKw#4h44k^oOGBRMA#T4kFgKOKph3_sr9QGU^Z0OD_{O65igI>MTy-tGJt9ho5v(n@UzqVDH9;{*^P&Gdn zO!=?Mak-h(C;bPV)095_y7?ob(yZY3P-mx;V(gQhqu`tK(RP{zz(wyi#+)_dW={Uq zAZ`3KT)wCC7zqcaBL9z~GmT4XYuk9{l$lalTB&f#ipqh?$_eE(+VB*Wr8!brnOP25 zx*dSjipr8xW@aj!GQ*BjYUYF>nwmL~bB=%$DJmNgnc?OA%BRh5@3q!_U-$pIRx$-g ziF+^`{bha6$%&A4$mmS1LSg(0bec^l;WA|fQFNdvmR9b}yB`-X^(;CV4;65IQ$Amj zDe}!+l%M~2;4@K$Y_obpL}bbu+nrL<4@4K9bT zQ@*9u%tz4m5!Y>0)szE=RhXbj5p1raKGjT$%h}=9zU8xcW8V%8EG=3Vcx!J5wV?~^ zFFyhf`?1axKWMfqi4VqXHz0Kb@pKZ}*U$?jE_)l%c&NWk1i#Gt(7aAL5 z-Yo{bBQS5_cA1jmC)N1egu^x@R>9% zs#!*L9f=@JGZvL21ru6c*OvAE*p+Pt|lwYkNX-q3ojmUkYYo~nQu=Nck$O|v zC66aA?HEygLM1K*@OH=&q^4XSu$wcm-P5kRIaUkV4q1bFthx#;SeBI;Vts)R?}tXuJprqHT(WM|a?lU?<)sT?ISL1GI; z`7`MU>H~@L_%zKuzft?J7{C^&&MD^J=Uj(h!v2z`J_cv^YR2ZPh*^PBtgdPY((B90 za(;9+yXEtn4>(k;1(I3m!%IcAphqD+)qe&9-!|(Cb8Hvz-_mmCPkSNXEG;Aq^z;A9 zf{%_Y$oGhZdx81OJlM}bZY=O$+d|C-)jI00{;`5ctT#YSYxI_vnY^C=E4iT-B5T;I{I+70-h%};_ZS|hSe@-+O!9=xoMn84j6llEkk+Sd)%(8|*5 za{`r0S_)VnUMAdzEWB5-vA^a7qTVX~?WExoH37WZ($6hUo9|Y1tf6jodCu$&n|+g= z-}VKE|2P%#-RXwwZ^-wu`G`fh3l`sXJa!@9(a(r@uLM%>31EP`&&Bu^)Pi@tzO_kv z0u({kVzXb&M)D6h6}BAn%nhF}wO=3bFv!|RiOf_f-i}Wo7F-(*-8 z#;v;!XZy0JyPxMnJ)I0j6YsB;oB(@vdIoF~_YvSXyrLs2*Fo>x7#D_c?r7pqAmq(% zBOJm0HwdL$V_y%k8GqtuN_D96nT*2^ugo`;TKQy5; zhV%wh_^T{`_|sKwoSSG$A$fLo8y^lMC1SMMg;71 zaovHD+yDlOJx#ubR(YthZfwK3#nYX?b6v{t+Y~79()tcd1|rdYzU-bzrJ=J9p_7A7 zXIBv54xUl67EoegG84~V35;FHm+;v8XFkAgEbS-OtR_iY@rc2?WcTwa5`6wFXxU#? zP68MI$|uK~<ncS(5a_Qr?8Fa zAb&#@k~;gBb+i6c;qieWM)`Ur^8zq^PQ$BSPHrTATgqM1^v45tCUaTIh!ga|pS)ud zElNmMpdSx%k<)0_MwYYTb*XXKDl!D&f7?&IZY{XC>8KIMf#Km%=t}!3F0n=qHfdrS zDE3Sr9v92ZAtL9w^spafM11Wg2(zTf?ZMJscj@4*>Hl!5JV%|m*yJu{i||>{FaBI9 z|KGlJ<417(B5&({c5q+DeU%~}cq6(pHyW^#hzk+Lqh{f`sP>=v_JY~X_+xVZNFaDM zj7LBnrY{TlHY9=nQ73TEe3cBN%LuutN#5SckpoS?iiW@{T7#AvSTzF5erw9vS0YFiXl zXudbY(8H$V7O|DnVxAHmwX-%Udb_-y5&Hw%Dt5*&~bca zJAGF#g`_xI+lVs)?$%7Zd_a}ZWHH^m(P7=v$Tzxw^Mm&h(r@?@BQ{Y@kU|oo*_~ao zt#AIu3~48FsBz}-QRjcXIG@G;%agl|b9CnqLLM=bxaKl!0)LY0rSeZ{@6s45#NR0k z-jabqru!{MYtx2K*_=1HXf##1goe2#tCno(Mf;D-E-Mty$ore<8n|4asbY&^sHC7> zK1Mz9*K%he2juTyX|Icwg=59Pss#hT4VGBMBzM|m7K%sUGLjKK_+AqiFGea#8r2CU z1V-#(`GM@uQ>xkmClL6;l;q6ze7z?h;|2h@w~cTuRHfFW zc>%sJ(Twu;I<}ZsXK0pj*Fdx3Pw6iqp7@#rAMyt}71ha$-pNT*(_)9SbS{sx3|NdE zml>z5CEtnqN_X;F!q~f+E4iB0&)f4=wPcS}yU@+hE+ckor`@7=Gz^y=^d&%32Shx~ zm2X$=du$d1;n)MqNt?JlrV)tb4c$VxUQ-__%ZWLO0-lSU+lft#?pgDh(Xr*gm48JW zH1}YOyP_B5y7*o6uu?k>W#Ko5&w+*YJEh|;SMv2c4#8@AyLz1udjJu|y=v|gVF)M* z&zGj3KZtr@ruX)dW7Lf&nDPDu6agN3ph0$~kid5GSp3|V9lFm;?4bT5?S2HdKy5I$FU-!VEPRC;|6@@33Wzk7jyyr(zziuaDuVz4D4~;{C*+Kt<@@rMQO3{tRp{A?RZe z_VBc^Nl^CLnup^Y4sLR-)MJXzD@+Uu-CGpx9$kN`|59>a?39+-S@fUurR?IQ^nkrs z-I=*6@X%o)oS-rloU*~ha;cfXdUZ-P3Yghnh9WO)GC(8Behja`L!k8ET^C)+hQy`4 z8fFkvpX%^-e5Sdt&YovWpFyh<+!G^$+aLE3W^l@-d%%P{-7T^R05#@6T8Au16n6sn zdT2d2$P0KZ3MlM1`p3;N+SzZ(H}jFp(}p2XQbUWU)CCH$HNMLc3*k?im_zKA!?r5R z&s^^QLv#i)EbOs6bWz+MS8rmQr*=lKAR6z%OIEoDyveqYr;ClL`qphq_p$h>xee5`sW_8ta2JB zxB~u(@hql2h8Z&bW|}ijvrCL=mgPx7#a|P96=E&m%DcwXyxr&qD~zVHtrmv^3$E&p zxuKH|lKKB7a;>G2nB>jlNK7%+MfZomolZtdK;dZokJ%^Zim+y7WB`>RhW=w z{xAmu^oghKYnM*c)C-4fjw)4gsDXh{6%U*A9^A(Zbnf3by=AJgSQ62CdjJ*|i)ER$ zs=Jj=!t7G~zmr}(AN}U1z9}MK8F>(#==Gc*^s2T9u(&RbeYMoJITY7)Kq8IvTFiCL zGEZRX6a7O)-)hIKG>#G8=CJ0Ac5e{91l@(m=jB$lYPBuWrE`W8GQooY174mftrE94 zVTa;kG_&NLuFH#(-^xdg9#E20Gi!k*Z+q$noW7q};(UIMye^e45$>|O7DVE6;SECk zTdnfqT<~{sL4POHCam5mb-kzEAVy+|9Uz;&#m7gCCV|U!0CCyzcr*qj(kCHp)TnIbiBo7|I$&E{n&Wtip7?uDmGb z!3#MV*_tVvC-5-xsdI&N8S-2sIXbprXN2+me(DZ+WTAyagL|uLIZyvy%5^vD?Yv&= zIgj{Nh}xNEsqQpdC>>md&{584wf`!!D^w0NuyYB|Zz98AEk@9xxXno6Tir{-KTF^} z*eHRbv0j*=${cUjF0KJyCyx|&L*aG8G)S~msYCkB5i`8%4Gk3%%FH&2Q_t9^`;=5p z2}fgt+<1Q-KXekeP1dvRKLXD0ohh8;Q@mlb3(Ff3%6o-ilUx`5$|*cO>@iXeH%&f?}RVD&jka{ zZ<#5H>2^!w;E9jsF}?VHw}Y6djLC}B4#6`_pm09tX@e(V_m<#Ed(NNj)advqB7w5A znB3XS$xZ8#Uk?a5hHc zP)vVoh>j0a5@}}ElS~$_wswB9r=pYLW6h~WH)14}n*O5pJq z1PW^>U*Wxr*1YOZR+A5)fS5!6j1XUdNA3&V3XPa$s!!6^DzFRTiGBu#u%G(2C*n|JXMoMu1-DzZ{_ST^ z(`Y30KW=ZMZ62fx{x0pigL}XY*pCu*Oh5(s#`mI9Do={CBKONitbxSB^HzQsY74zf!R05pGm0ytoNs-e1i!m3m~x z=!>H0rV{)Mlp4Pdi7zQ+j~r}tH6$T_j(uXK)vONKbamRTc^Pk)W-h~r@Err5Yuvz*NWZqGIcv+C_n+lHmqJZ|pg$ro*l&`p+08 z`|2+0*GHl}BV~c?m!?zyj;^Phk_y81ALOonHHPr6>6T1DbBEdh!=Jn>8W91QiCbJn z{hMAGGr0mN$i_6k-J!f-{c`GSGLfB&wh8sAkQzXPRQYm5ukTu>(GBjFTGT3p0PCfE z-jI`?8NlD=kiSA zl1pRdwTfWAX3WkWi1EaKRhOHS3Idf16N$IW!P>c-ik*^;;5y#ie_rk15rVl7wn(qi zzh{&~o5zH0YQ}A(Mk6a^TjIL>x8nbkC`>smD{oxhpk4X`1j0kh&RdT~JvRt@4AWxb zr~6U7`kc1R*YiigdoKBBo#E3`RGQuI^C>F81a+5bEE z#$?SKN!Buij`%z|kxMI4%pX;2AyGVf0ad0nJo%JY%mu; z6EfK?Y=!#Z9gw>RWn5rHjDUWIYf#x&k+VLiU|lA9Z)K5v&pRdH2uJTwuMZ7Zh2en7zI&mp`KAz~*+-XqGzznpa`YB;0@G zj+=5lHY-jH+H?tt@5TlIsOEB{gR zPO4}J;C3rw!-MQ1a093}a5pp5kVj3(s^AWAZ9VqTlzWOUac>fHOds4iiEJu91ztxymRL%cl zL~>|D0;MKZf3c*)rs(}lMkKEwAUIyM3ptV&Z42TpkTtzOz4)nZsmgps<;^rnoC?`wtYDh8saH{> z++|!l+Gt+_MS$TcUa+N?=>=5)9)+M8lWqs&0**NuYpT@})55sJ;}gtqA%^$6jPsOD zYjFTeCr!!gpB;Ap$@4;W3_o`3s4=;CyrTYvMEy>xUMaHIL-UI!#xO9_&`m!P=Mbso3Gb!ywXB_O&L|9d^vTNwCFz6J&Jx^nDdd7oA&3`a_j%7~qbi z1^r^a&gQn!rus3*>GY#C>nP8L?uyI+-l{lu$t%2jw0f&*cUV^p{~f5fi=n|8ISg8_ zH#l~EDf`ynP6rY+A-6^Mc>%<8jVsRbR_vKZx6p z?HCDDN{0-&A_KPC+)5~=BUZAv=OspVqQ6+buGCaU7uxD$&U&+xd(@AK!p>)$Ui(ITnyeQx` z+-D00m$3cHCqm96Ph9qr53o^Q{~tG-(~=oai~XSz_u+a+|J|Sx7htPMraD8=_X0n&gY@&Fd{#B(xhU4L3LFD%R?E=r*#s3=@WGd_Q;_R5i4h+BfncOAon8*jg>{Liibx zk>G%7P_1zHdQwaW9a6Q7w5sm}?j?CrDeW@)zM0lHAM<_gaha4h<8qF8PBuwUSM#AxlqPA&pg6oNI>SRv1 zc+C4^ai~-^DgzaZzF~{(ZL27*1%W`J(5dM~=MdWfk0~vZ97}HKl(1HjrbA1L^f2gMhWf<`@o8W>Aum~d0oV&bZhe+a zs$Ac!MdBa#GK<;k`E{_4r_;*RLGUU_(m3btg%^#l1;)s>lQyS8?~1^!)lC3*jBY$ zwiYgi1ITn^ok^k3+v4XI1DLCyyw*Jub26nhe0JHGMR*))?v^_69zawH|3QJQVvUv`L#0w_Lju z-my6Ttb$!qR#-bL$6f*yRxEW`jqf-NHSbEbRyJ%lExGc#^(AOSCvR~Pv9DVzOV@`U|kg@(#+=iT5 z+eV8h+GF@zK2O=!yPyulyL}sW?P+ka4Mhf-eAY!rIn;J&I}qF}-HU}6#0@!x3P)}P z5P|J!#OHMx(fY?f!c{(Mj2_YT>Rjx0MG&AmhY!1+Kg<6nxtAO0nrAK}C{*W812b1x zXLxmi3ich1&<>uzKeVyO?YJRutuV}NE^Xc}1Y@*;fB#~2jt!|AKLR5ej&foyQZwj? z17}n^{5SE-iEguEk!OS93x#vg)GxIq_U>5YOnHe!#ZVMC=2rS zZ*s;AcfV%bI_%IB^Io6RcueJ|G#oRFRX4Y)V@=hCx6CvJW=iNboC_fSPnYPE1gJtAkV#v*^yYpQFS zgncMxgX%PR0W9iKj?F2s;6d=V75+?^4$>W;_A7Q1RZ^pA8|NU`DRh1i?G2ePi{9Da zV9#(>`aeUP@R@5|^Ss)TC<9W~T=N`P%!Yb`HF~zWhrM=iPrCNY13@G|ZCZR- zgI}@AlK!@!J#2nU{Nu_&fp8c#z0?rZ9>$^{J0rxqAYK>iV z$%y^g*fUU6b0fX}luMO<`yt_d&_doB+h5JY$~p!KhlB+LPUjUIX387*SgSz*uRmYS zy_`MYs$Pax{B%V~wo6U2n8*p-!umk+yM_Ps5zy8vx$Ljh2HXH{yc^X9;9t#97<8N{N=41a-9lniS_%&B)qLxP zNARTRMqWc1N+v##`9xodjsvnZ>(kKn8a^TI!Ya43Djn{{rWPEQ#l+hw{nT;9GXN z;Rx5VGgTG;B8<;}kG@?N@guU$h4zwcHYA~#uK)1y9##kX+UgIjTj};U+^m!{2%984;ZDB6%!LIASK~s3xX~21#^srV<5u9! zXmtnotbk49>;SwSlIcwqshpSOcJ!Pt_T#;4Vo$;>X>u^VkX4|W3;9K8GNH02|zOS12;`XOT&Oh!C?5jk?n@N!gdqN4@sNWj@KjJli@!efSo zC7GG1;`}v}jqnF)m9)&}hGeEBj%DUrAb3S4vX-HBcn3zVp#p95OjVjBH+}AHvKbe9 zE;|WpNiu3O_keohZwh);<6ERm*T33x=;>sNPaG>c<@j_{;V}9LW_pbxh<=_W82_I6 z4%Pdmdj$!4eJRXKtxcre))jagaldI8IY#3)*pVBak+i-k7LNhdX0}$3$ zD1?B2fC`sqE3i`PY*whCGy459h}c#AfZ~1$ofqvCRcG%XkBX@a{7$%kMlb{Fh<|A; zC2OhwfWmj5i0AN$F<0FMC|CNa=>z3a?{NzTz|**R6V5oy1Ao(R{GZd{EN_Z=O)+fi zl74AxISTkS{`~N_IQ%W-85)2;FVDmBA=Pc}VM6!Z4&l4VydTXp%^65_a4>R6vS$4X; zW+vR-t-8FNG=EF8Nz4wfLWjd9j24+EVb4=h51MafWOKh7Te-xrn2MD|WGnyS+KNE= z7z(mCxK4s()@7g41+ei(6Rf2UtWmbFi)lbqZf+(+ohMo0`oJI3QB_W9_YXeID2bctbLKvO$`+W4PL6!4uzO4i z5Q(jxEx-Y8qziS!bC~glzp*tvPe$Ln-dd)_f5C*Wl8uvN#?3(gWDqc+ zjj^LQj629g6AS1D9#~|lSO;AD(&)~Ay8i)ulLR|+ij%<}9qtA9aUp!l!O@jyUHM0G zU8I%5WhK%7;+synJj(Om~dB13vWmC@f*KBZYEqZ>cqtc0d9^c#Rg=vh6p;I!!f{# z-Gz(ONqLY4Mcirfzi2(~uqUGxDw>qgVSXBb)k7&R+C<&gJM&c5lEa>6$KSUEuD+2L zE3-MEWHU}V;V+O)QDphS;3~S;2l+I)l{|oB#gr>pWHH>Et1yQeF1at^ja{gO&ygAn zNg(y`2gj^c2+)`~Wp*7@=;c>zg81!p=~k-h?{w+&LCSUV(ubHyO&e6KtZv^ipsBXA zKF9lCi#XF@R-ekKIfgo}eoka=lHzkvbf2AvmFmvfkHOJ|29NGS%i4#kBZAB1Tmz(v z8c`%Y>RcCSq^vB`rxv`xP-Y;+0}HI~D_|QkA1=dYb(Uf@ybmi#6Eh@qdo^m((U7b- zLl{%oQg;u_GQ^ya>+nhh)qmRa>pihjwSsytn_dBY7k-<0KX{YqOg<5mcd>bEIhe^+7`=@U#_1GjLN3-h4$1R{9QO5 z)(%(mahSC+B8T4Gg@ww?8(9nda?(nso;=owJO7ofG?M8 zb6&D7d`zLr-^d;}`L$^WAEW;2KkzZkC55P{19uoqRh<<5cK{jlD&kF&6-6T&N+>0c z@1y*9&j>uN1aj6R&8TQVAGudU z-?>uFklu`+y!(>B2Oy((b2}FXO}JGmtz;o2_$Mw9M|koa-+l7Kj$4a9XQ`X2ye52h z59nRm-5UBJ%{IY*^}*An1GAyOZ?AndTY4$5YH2O+V#)WS-&frazNigZ7iZ)-Tau_Y zl({h@aq!HH_SD-#p{%M=d4pt+ooZ{#uxH+-Egw^j*07tfJ0A^1k1|qUu^y|nD^7}= zqiZchH%&w(2JDRl$N7s%nGyKZ);pVllMAVQeIn}gyO|a?CDA(97PrmILV=W~6Qk*l#>matR?T%eu46S5UP(b)T|l{GcBQ&e>Tf`bPZ=~ITi3o$ zBPy@am^G^w_MjFNzQ$0Or}aXKr#{K;6w(2s6Z$*3OQ_pavI5v;Gq^d+AT(Z6#!T)y z!`u-8jbDvSiFnJ}h=1G=h;5&afgxkoq?z;PFg6X^lMoj_fZ@R~|(TL4bD*v7hCCEeI z|K7gN=v8*pb&Ju>gBin3f~!`k%tx3B@JkR|0B@g`Lds#^%%w)i9Rt>(xb+jK_=34r zq^IX&f2)GRjMY0EFD1uTB)~tVPV8IP<$!8&@ z3jO0IH&9N@ME5KGw~odEuwmzyKE^D4h3PQxrjqzoYxtgHW2=0w$?>OqY{hjHn0Jp= zWjZX;&81jQKYid^QpCYFZU9#E5O(Cuy1Ak8?42Lgf#A9hEvdy!O{H^D%AW7;XCY%^ zI}JoS^&l_$4%T)-{!b+&>?+NrG31y_79`a5@5THnUG%9XW^SOvcB-hsDSIFP!e(l@ zPY;aBKGEYcBvcTbW;0@jD>YIC?RN)XACf|ec*?RmmFTh!yI(i-)ogEnIJFY>Y6Nn{ zDeN3JTQ9U4I>+60k{kH8--g{%zRws`{+ecI_srQBFd?iP%2c~pRa;;X-mk%qDg5*6Zr|AZFzZ7vI z>a?KWoz{>Z9k`bSk3gl161wZ`@#kwlu2C7kvC3{mVNGPk6U_^j4~ZjIgB->3P+7_* zH~IbHI=`;wZ-QdngqY|KkgyB?e!5F+xOQNISharo^=P@;p)v;cpBc&Qr^YhWty&3eBjCNihD9Wq zPraIAsuIkST<9NS+eXbN7wc$={N|B(>m_n7PO!Uv2%VD>2mQ%`fUmz|N=EK}wVvgd z)V4cL$vZ0}V3oSVyWdSf}j6pqn?7O#`{ zyR`0c^RpvyaYS$m%_&|xckBC|e}td%isy|D^H9@xeT^IuHs`7yVW*nhx^nUDos=^F@DN~2KuqhQtba4EWgl| zey{0smN!w3XpS7~k_7)KcatYI7kn5EFd(_ToXGmlrL#Myd+;%IhgG2gXBGxj!jm7n zcS&n1lgIt72RX%h5ZcK#3lgd^^8VUhm!7>1+oL~Z!;Vfi$x-TM4kyR!64=vGE9q;g zro-_%nvsP2_)T!awMP|WVHhsSp58uv=s7oNq(!E;?VmYcJN@W-;{^W25nX7rcG9V{ z9mFz=U^VC(F5e}2zg)c*`7HXAO26@HYRz9(wGaDmEv2{t&$-&h(2u3nOoT&Fym5Ego^!aU5uL5cx zPc~roG3i!(Z9?F(QObxp(pKuEc_7S=K}O#L^N(Vf*EK@j5itn z>WMUytVCLVSp0&jc%jKA8DK}&y@dP~&_bT6)W3u`orL}Lb8ZgPA*zo_ugk3=fFA#> zP!;b$K)sbXX{stpbPm8j7Uu#%mLNn39OTbk@K57woZ(dQB)ch9b7c5~`zYL-0?T}H zUEnKHmKK`h=EyVfMLxNky4YsPFu^>V{ALcEAG=}9IBfe7 z4v~`06?dH?ot0y5n$~x_V_wvIWSHg@=X)acpkz3$|J%EU23CiAyxlTr{s-Bv>q^t% zN?Q@@z@Jp#!UyrcIRTEs{x^N+> z=ZxtCGIRPN<~o#6JP8iH`XJaYU$Y$jg7Och%uPG=pQSr)*QKr>{1=O}IvO1v2w9zw zgDLllZ=u2|Uq?UdIbi>qcegYOO;aEd8qd;k(S4VCvSBFw*ls3;&g_0O1UxG z$eXzUc}h;wXvRm>M|qBl^w+9ph6Cnl)HbQn?`-Mxcj@pQOnB-&g#2=N{zQN50pZ$R zUkr|OPuQqeMUbjjA+h@&1=V~{cZ%1}CHKE)B{l!T$DlaT#t3QhZ|wT|^mEi%cQK3o zbX~d0Hzv7_W7hSA5esNniZ;Ds^6Oc8p`??xuz}r3t` z1ms@NY3RY&@S^Y2!?HC6h9Bcz)>u&)I|tn@SCHQRE&3sHu|)11F(c`RB9AP#a>o}h zB;&fyT%-9>5m8#ev4xOW%gSlwGr^_BhYz(_2Ld7BdlUx#MIjoKd$u~p8%`Tg$=}{MjndhITMaqLuxHB9*}y!p zXN3P32+;J`Xby5UQWd?jMO+XRE-}fAcRMEcTM=5ssI`3vdC&~~7~lSQ?Cz%4W~o~- zzIa)5%7-;mCw^2JAzcrDh2eCNy6Vk}j^oe9(-z+lt1Rryx4lL`?zS2U9NhbKG5+LT zz-=xOLVQfC?TAT_HD# ze(Q#&Ewu!)8QG0XUo4Qj1~>&e4zfn&hjo?H2gl-R8B;*6811l^ymc_{wrE|G{_o&2 zW@ZXOR1!JN3fo7_gYDqdqi~2X=*<7$#D|-Hv=t=PJ z@YRuD^K<^9uS8a^?=|BGN*Jg2WLXJgoFjuVvMq1Q*2v#H5pN0~7wX^=vtUljSN;Q2 z%zs@^O@6bl6m0f!T8O)3QkMhhXUx6aRc9nWcS>%;fh;G|58bV<<9T=Z&0q>3l??l^ zM&Psi(WDqdynkZAsLF#VZuURwXHMa?Fg_4+|ZObL|oIHYrt+Q%hMM)-QQU0QJP$q0FR=8@ou9! zp@Ylt?(u3%q$>Xs{}dyMo2>e2r;l!C6Ln+dh0JrxL9cTyku^Szn!4`RoBrv1^0**l zY`=ya5scNIgL=6miSgM7U&WAvR#iqI1%e;_0c_7;mTEPa_f_q!4^_b1&&H$1ZW<#7 zw#gXnU9Jq1x@<(3W1Vc`D0>>8tg6-U9H-+xvydHIUe~0k42{3l`94k~0eTOJC%rY4<4wW2>C(d5ceEHt9Oid7&`3}k~ zmlFqm?4cjPt`VH%|2L}_TIsvk;J2Lg^z75xhmcQ6&H0VO`J?PT!6fI$lf?foh8T8Q z=&RVDv3%FIz@D|gF?}BFPuH);#btL_xr(fo&(;Lw{Cq(ss(tma2 z-tTq{_i)JN;=sG%F7p=!vESApwHor2FG=H!q)M~4xrtct_xC~_Cu>J!-*~)Qc`BT8 zjVMEfkHfc;S%RJTjgw!FAk#Cn=M~Q+YuL$kh1Q=ZhabP_8LvxvrJU1ce`?KBX8sOa z3;4ytLv&lKUTg*97{!rP#LFzJzqpa$+X`&wkozm@3H zcikS=GIyDQbpLVHD)*0P-4<{N;EVTcbZXs;O85`ll)1O@XbxA$fM z(>D72DiB)|=cI(4ufjr9Yk}Bpy;ehaXsv$LM8!7XmTqI5bDv8f#yxiG z8E#W+UEqml{ZH=KyxhvFDPZkR%!7uKUu?o%ML^?^^C`j>n0SGz_D+m$nY0U92`jflYBe!RtI&c?g zQGd}N)}f>0an3lOk7a<2&`E<2vE1sVmOE)ExdjD&+Ufwy;eSb)qe-JHNLm%@r`T0J z{dpCgp$<8VVAaI6@JPeSWaX>g1bt31QG9Wjz2twH`QWZ$q{MRPp;56Taq<+mDk8jl z-#Jz#WO$kUZyFt3PJxN)aeAI*76ULF;MgFSHET&i@8lpy&uBT+&A%Q(+XA;T^4#|) z1hM-k??30l6X0yeROOLIN5$F_{D@v?{KK*A7ou5fez@x^i-7+4AJbK@XKGW#%DU63 zd81P3&tq@!PK(!i`dO@_`0sIoSzeP27sLk*I5Te-czS>aqli1Fe?5CdpVQ83X z4v6baxoS{(t_yue3v!YtX!TZ&kGU?&owCh$BIS{ijl;J-+*^>l;lEf&LY^J&$P@o6 z=1`vSLi)v*`(rZza_%~g$$6)q#7yop)xKd1+<_L>gT;q+N!CYe1Rizc@Tc6u^u){? z9tK}jk!BP;dWKJ>0fxhl8TBv%$a+BOd#0JH$HD~j2e;M5P6x3292%9qCpCTVm^p*{ zp!sqIQ}jm*{eK)?dpy(s|1HADJ@;}=lH4n~k5!6L<`y9&$t^SYxos}7Bvy!WTkeru za+_RcQIgAY&wX>7`{ur}*>B(9|L@29@xHv?@7H;rbDrlp--%?wI4>@9pp9GqbM(0!KJrh52TC z&fTK48Q#=?H{1R34LJ#8LnXHl1IQPS)Tcspu)L90sc2Wkhic*d+TIU9Qv&jCJNQ)C z$)z=KW!>4Fq#A?HUA%V>oSFZ5Sz%NT<8)-u`)=cJBd3 z93X?^SHr*~MA1PX(f(2A5h$fpe*QQPViL&)a|)a};09!EQurD6guxGJjAMT>i=14R ztN()NI%(lRulo4-Z{Ua*$WYdVMS49I&%Qc?3MJR?Nq6@!8E!s;8F_>--x|b^-nPgc zj{&lQXVMlIsJup8fM(4U1-ZY5P1JK#dJ}Z^P#qH40LA(0L67VQ{fPjG!t6t0c%1|w zDeT6-ts`sl%V+!$4)mzvmVImIYQ`Ru9A5~nb$w$dXE3C@3d8rH2tbY{8{YiwS33sy z?WQYvSuyHt+I?l>oVP*x>M7Cho4ls+(fRrUvuA+y0C4MXnp=g*EQ$VVI6QBK!~Iy( zi%48B)S>8*7i%*twoI1rt23WDYMt#2(7Vz;wNVUTS+o@CYYjX^9XcN#5T)aml0BEl zTaNTLTsjC>Ec0a9sQya=pGHRC&NknW9Gs;J14w~ zVwVm}K9=3>C>lB%?~J-p^77DthOaFVPMZq{o}N~k@P6D!sD=MHb{-F;Grn2y0x{slb?9Ey zYA!bb#xHR2r8Q;=uDXW!ihuquy4Jvyzgs`SXHn?JCuN;k|MI-)_|QOg=-Fd?K(#?n zAh;m2XMGHW9iFp^czTZr%9ZXR9kKJN&qz!jX#9rX{wBnVoR`)+#@57k%eTA>$JefR zg*;#m2y^|X%AA%cO-r1Vtp<&2Afz-=>TgbR_qBa4z=}Oppvz859Ao^4Y)$Q;RZk!h z29eviO?pO8S&F(ttvG{>&*2xFb8gF7!aP{&BgTMYy0?UJns#+jl&pn=kL*bE1*$Q5 zPw(}ibFpWZ`UQqbTIWgouXu@kK@v&wKd`}_djF5S%*r3|T?as#WU@d{nf;WKIfIa4A0Q62Wj3B8bLbKC%bO z0Ip?Tx__s&7rXh0@lEDL1aJ4Rr{;vA3zZ~B1LjhklLI2{E4KAJ6k@tF{gpC^_aVb< zSHj=$BjTMT?w!Pb=ti#Bp=fLy9mrXdS-@{bbcZFrwr@nL) zjmjnLL}l$R32)M21gO|wbocRr1UCJ=(&;g6V^{*r4H12Lr-c>WkZ~l2^DV(Ir*|DE zY4**D5f$~JDyI8!9?b;_OX{COf9&D~nWRnr!h{M3vn)^Q)tU94Z~vpJ!cg(?7n7Y; z=va_bG;k`Ift?1+m{2{KPR2O>Z7}zghy=zs@?YZOTu112e7XOBWZg+w{6`}~dlgPwjxM~9Y zJk7`ga|3sBCO)A=^ioc(H@5jq)A-LFX2t55Tea=Id zBPkSvisz7r2jsvIJ9Ad`y)$fp+MBJY25o2*JfM{+SD+r~E(MV8k$JY*YNE5n0Coyn zk*F>jEPz3?&V)CikpWWBy=y)|%~nkJ<4Lh-@oByIp&(g`&vMB8(Uvl#ZqXmY3Wc00 zi2@8YZyvst``MxoJxj{RJGQt1$z8JZ5pI{EBexdVQzLo1X2fGX8fj)m@PlyT5Adi{1~E^wsshtnzikag&^xku!^W&O7_ zDXj0Kfz%A?F^<$T{4PpsB2su`irexwQH4*YAYlLQZ&!9{gu#K|71Trg!0uhlnEJ^n zV2YvRAp8x2*U*UK#a%-5e~Xa@95h04VF;cz`)^-#PXVah8;}d0*P*ogEFTPcxjQt{ z6HLm8y*?c; ziTiL4jhg}jH^JL!Ey?$a-H-Qpr=*yaa)SZLRs>IsgKNEMso)@Pme7TG5- zWgwXnR5yX6qX^CT9YIRQKd%(~HxQ)Si#_>eJ+gq4JYiFELq}hGL+7OLJ{?SlIs-Om~}#0}w&Sv_MUKw`MMs;5*rQ$x&nls>dRb*MR?9voXXCpJD~ zN=73nHhA@bh~j8cSced>6D6^3kH^L{8Gv_7MsxqzqSjG^q~hq+wvPBA7)cgCpx?C} zrG+L0``GIwYC}8vp9i209;G(D|C2s`EEXd{@J!rf__H8^YNbRg3uG}zd(UINCVFb~ zGqL&}^;cp2e77mr@K|~|AiVt__jGviBV;;ucuANfgdfE(JN=9_HHlN$m^TFJLup_u zS4Xb=kE;3l%zZ@ucvGT-SEqV0U^&Qwk$UOl=H2lC%ENg|&1Ur#4hnGbIIShd1S4?h z%|r#-^UkvE-Ppj!lHm5((CggO&{IG4*>_7NreXS9h}#3!GEa4H-iIx~KZ9I*l=|g#73edJ11+)N> z0&vY@j%{%DFrc{gxrKGpe`B||l#WXF7{7KuQ4H~leFV%q`icvCe*1luCZ`VGrTNHY z{x~Oj{g-7916lLbk@egGRfPIDMX`wEc+3x_4W_X7Idt&NoaU ziP=59>Mg*>&?+JNyg_Kk<$S-xkTr#{(oVMZ@5sb-!wYhEBC}oTd@BSyTg?HIzAcqHev3a| zKl0wd_CH*B(h~C_&Ssr(PDj*0QyZ5Ue+^55(w#em$Tr;v^(JwidNN+qO*Vk8nW1qGKt%jeDGC)6!ICM+qeajF3pi#h2`VTC1 zOL3Y<1aNv9k_Md%73^(BJy&zd>Ys>-Zalxl1q$-2YAOTp>Io#TvIqJfXxcnjKpgB4_U;*2Dc|H7e$2CRU9%kMi0 zuMPzv3;(N!VXcXspi=U1LDMv=^h=E8f!N+9=o;&qI#c?PgAA*rA^ik8jmbPT+Abv= zFLMQfWGv*3*&n~nQC>P}c``*yI=C={nqi%WB6TnsO!8I;ca5JF*$I@L@@2zn z)RDKYP7-zTrI(f9E{APDrWMv?B^+SopT#x{Er*=CcF}@l^hiGLbdyY@TzeTjGgd*S z@NHftFeazE;OM&)_Xp+}y1D=y+y1^VG`RQOTXV*XW==}hi1a^;lQ}&53jYh&wsX^p z6AE{*!=q1`se{m5qmLb=XRvGUfyWY{z=+0pxU(%y8K0zPg`>V4rYeYZs2uEe5I}`2 z|0JkNQyJ*hM?>bogxm@IYW3%towl!c@niKX>U)>Si`&Ao$jGdC4mVfq0*)6ysAMJc zh)o(8@y0SA7ts39mh^>el)dovg6_uLsy>8h$DHH%E=K@4W6sudxYFdwY-i$^qwzqL zH;VpA53y~v=KfH?f>=SFXFrhM1NsTuhAb$fHYVvj@UYk9lXPwVJhhD-5N?>F(Q2s2 z1(2T@K5|G+H!lA2Hh;L|0a>p5*n2#t3{U&2&R=YzeEQvlZ zm#N-%OlCy~NCdP2cl_@Wzh*%4s~_M84i#O%3{fW8Re-yrr;HWSF~J9Yj3AF#*UG4# zX?n_dOH2Th&uxUF^H4nC4t@N@A$Lc5TK?NpNV9{;|CxsdDG(v0z^;409pu};fNd$4 z&8mG6uwKSS9QoF&OUET9_)VdnEZb(Pe|7SVEh8JM%nb+E=JuVFb!9s69JeJG0ZgG2 zxFT-hX&vRdbBzzKzI7ar>06~ep5~{$S(v%Ar@@dw6C#v|oL-EQoFW=Zct2ea;%+wW z%Ebi5WqadA&Usv5y=aMoT-<3~tI<*iY_@8VoCg2lg#DW4+(lO*>eNOX_6AnmfqS{e zl*wa^*0KG3fBX&9D*PJQChHcv-G7MY_(W8$V&;$9cnHJqpGM|$g3W&5H2zF*f>Fv< zb+_@HeesnT=U=N1s#WUJPpG1`7yz#{Ptww^beYhBYKxo!CPpG(c#8Vl>K@MC7E|eu zJgvPz9~KihLE#hWikMcls#7??{Ua@sgl(*ewk0R|n6!(10Dhg-O|>Im(t3g)=se4Y zq@QUKThU-h{am`g)GR|G=Bj%$q82mnG4hJ6)VdB&A=?xZW%%=NM=nIyl;fATZ4EbE zzcX2z3-5O)@O+v47@4R6!PDA=+=TI4Sv9#$_fGD-fVj3Sl}@(!?_z{E9ZAorX*2i= z)p7$T?N{sH#2ikV3Z2Kw`sCp!j0KvS@TT7zrvh3D6)Z+ZJM2vzs6&VZ+x!hNFC{4X zQh;k!=F-S(`ftwsjuu!ef89C2zR>%rG74UJ+0_zyzyTHLz16mRchJJ^03Mbol466&uAX1V**!1L|^w_ya0~Z6}Po@U1cvhOE-D5E8;juCZ z^64X;?0w@__gCcZI0fkv(E%cuSd)8Y`N2y}8g(B^;-qiiB443hA|I`ILw5D7{Aq88 z~3!*nZwf|awV8K6o=+=@P*%3cn?m~eby5~w) zDmzp|nGFp}cXh-!4KL2mxc zD_0H}S9eS7A2N3mAGzp^m%phfZu!(hi;^2+>?eHc7>FsU; z8l@o)NQv}&AyzMjrs~wnNHI6G@A-5DbGY_K%W`Z}tO-(hlQr@Ks5q*PA9fv02+N+M z&ft=D4g80AC~QZ^oA|L+TfP@r@6Y^C)F8#^mxM`}Y@VVz7s})W;IC0~mZILr=`%CQ z4_?}Y;i2Bl$;|D@Q=shfwryRjm}ck7ZVjfeV?}+Dd=ki8M!c+a3XD_MaiPE$!p_V- zqfXhhPQx)vdmy3Bt0r5op!{d&-k)$lxi&xcmc8jjT**Uj>-`5%ZL1E@fwCvwW;~6& z3)#hRZu~s7b{G(v*9kR%-qxFI=AY)m^h=&_W)7s=_q4Sy*^-GzcB^ z(l0>+v%Nfz1)xU&s}pq<#>I*(k1M$TU!Q2eLfW^P$!Fw|USaa%-yb=pkMkybIA7p6 z^I>^E=3bJ^V)U6Xm39Uu4IQ~nS6e5hce;X93D1j(qM36qcZFZVGlt{C_`}=Cd_g^6 zsuM(cD`{89`yboB@&qrKd#h4-AH|73sDa|Zdguw2qlcj}%tZ7(vcZv4#{;Ph*Ty2~ z2Wqb^=EdB;IP{mXq8kI?i~UMWUg%(ai0q=*I)~k*stXc};G1TfW8^e16y(&8x_Bc) z5b*8N{fpGBPt(t1%k*maS@lngK_g<()i40l(qTfTXK3gGXbNP*#Rf@#?f3la3ST z_a5I~I^bVC3{&G+HTbp}Y80(1I=&hi&jH7_e0&@9AG$n$c^lZVj@6J7KGFUi5SvpH zwC3L7PO=W)X^ql?2Hu!0V0HbqttPhL(bDntp)H0syEHE85@8Q!C32tOs0LU+Mr?r5 zB*#B7Ks>A;2CL{8CQp5I=7?1YaP3Ik)4{r1qWSLLlHmYfwK-~gyL%Pw@Et>yEZfx? zoISR&AhwU3P-m`$EI9U7rwri^-!!s<+~rXFZCnsys<1kS--Q=U&+~zBU-4mn#`IJvIla13b z*{%0C2gdLglzw-Nj3%D5&g-!K)bxo$12No-k%#|iQ*MA{cVz)*&Nk_Z0SwareB5*c z|BL^%ySOwDh3PYY7gj)rpQ|Ln{ZV_}*%aw9EuC5v=kRp(K z>G?TCpw7)G*9{VWOzPgr!3O+^7xl9A8ftXqo<6G1}liECT zpasMDb=L8*8AOQIM?PBj$izI7gc~x+XVCu7-AR9Zxs##`GhzKe{f&JZ{nw*qd#gxv zE=fLi3iWAw(a(h#=Oj~bP$NINY0XrW|Mjs|82K*mi>AOcq_T&-J2#fz22J7^T0$Bf; zjnjjhdSO5HBhGn+vLH=ivX^5N(9D;Xyb6gEv$~Jc4n9EYVM38M|Kp2iZdqiedK$?o%X7C#G&)-t;1X=s#)>2 zNH~O|nQ@Zj^~wF<79(wTc!D1QJ==g&m>4(<2odffgzP=e9F{*)ckT*Q`xN?3eKD6O zf5Nq%^gT}ZAd3Rq>co4mIwy(gy2v%;Iv%vJF+Dr2xE4M@DomVBrdfSwU$i9mFu9Sv zDa$J2U%Q7Gd2w|f7RkwkYGK!;0BREn0|Y2Z@NhyjPND|Wb##;1{sB`QW(!(`BE>el z20W<9`_GGh8XAmva8>T;rh4p4f}gPN224-B=v@5u@^`;KXjSuk*Hb(LgCrihW; z^7;s_#Rs^Z`f)tYq|!*^^WGT`+;;-5VRo0FfgNl=;>v zu(WJ`nNx>5qbSNvV)CjVd@Rz`aH?PWG+4)WWLF=XH>59B7OzmUTdJ&^CsjBXuaDid zvH~)5U`{ivDluRG=Lq_*-R(pq!S&_N9QfiQP6$UjIITTyd4#(|z0ay$RUAD3Xyyq= z*cNCymTdCi<;;x8(_6fBp;FFpmxFd^@lZ&+Tw>`jRK8Yn=m$ z6xa*|u{1SVn>|9kVb+hLhZ-c7FykwiNG*oHR)MBbhl^|$zyZ%@gDC6{h#5PAfl8He z>}t-*o%1u+WmrULPRXUP;W{v9r(`qY_3i(Dm%vF#h<7Y_zI}Pk%ibiutT@cQMaM;_ zIGl%gK_OnEuTW-Z7n>f^b$GWmD@S{jR%&r#UH=91uG4%yPDBOHln2j~t2K4?&yTgs z|GD<)@2uYEAE2AJKxecke$s23nW}Jr^j=(dNl3roa+6KluHkF~ez5-BiC!i@oFxjj z&5kB$52DCK=3lyS?oEi~ta>V<$n0&koMJIsls04mTPTyCCRnAu{n%d=y?cLqhMIs+ z+Rtz`Jlu|*lE>Wej4^Q)q+7c@jH5b!6LLu<>{UABM-jS3az#PXYq?YcuwWuhg3zB{j`N&K~=J4zUa5jp(7c@^l{E6R= z{&{TI9h;M1K;8h6tSvyzUQKn9(@~B84DI7gn&O>>hIVhD_qhk#4-Lo{#@HNyg?QuD znU`OCuck(WsCPb!H+Hi*Fr@!B7MQTX__VdvLKF`%=OmsM>ZB1#;U6FjfGUiLe z0|$lpUx=T(e-M9aFtVFs2JvJoQ${#)!QiP=!8Gg)2nJdTy-Y#j2kbx7Yx1dwfxol1 z3f^!ajev`hds>in5!$N19E)NZQF0B*e}RTk`pdU1%~4jCOJ!E+ z3S_E2bopG`uKW*I9x&~3Znytw?Twp9SC1S11q%Lw^OndgOC?^zA8d~xp|g_OoX3&t zpL4@8;Oo8bXmbM`O+VDlgrMBL@GP?rbDV3ZvIGf$h|IjyD7bk1)vmuSt=R=kx$$P%@RHpi*^tGr6e``8u5f-Kh-LcsQ99b~^!PPrF z=xs@+XFXox5~B$NIVz!3{rGO{K9)ObnGyj6sHQN4(3qHRdHvVr_}b^K_6d8g|K-Q# z_bnVSg?uX}Ccy2nk?R-z*W4iscLrrB{P@I>Li*Y=N0C8>70@s7lQY)6`mH3aY>I~o z4-<0#5W5L25SILFJ%cOyLIf`OOx^eTKvXH0p#-F z2HarEhf5m>GD@m%LDnN<^Y(Zh&8gg_ie+y1Z1EhEDo)+}`(YtU)|bSa-)h^+Kg*JT zB$L26L(^GCqdS&rc&2&ahL15#F%n^gfy=wYCOWg8cQM=kF;mKSS{fPVKe@b%w6tku+Ghaw=vyLWEpp%cu5eV8BBDEoYxslH&^?NNf6C zxZGTvLJ7SD^Y9|3mx=$TQkV{v^6AFgy3A#wCUp3Y?>VdnihQAl7FZ83;d$VB$&4qj$Q%`jEhWy5m^9nHcV^!`U+y`-!**AtIBPpHO| zTRL3$sNZ=3vNOq#+43_7&d)9Fbux_3?+vMU^oLnuB6$hp0n`qe>!@_FQ^L+iV^;p- zK`H)xTI>++1YR05bxobze3KPQW97oZt|Olqz7za?bYP0Vyy}cp{-9nKwXJukW>Ll* zps52TtD5o8h0RuqFZ7hoQwLM=in$u!q`q+iHjAhUo4LIm@zwVZ*zu)PpYx_!4f&^Q za&<5o>+DFg;{$^}jM#opT}Mh;^2sMP7LeI7dlN)YMqW4a!mLPc{+BC@CwN?}KI82E z=Apw$?7Z$V|6{OcGR#}6P8Kk$y0q;FfI`Ga^-1P3FFGUa^f10RDfRWF$v5PlH|N^9 z2|_ZjL+cdc$}Z`5o+~3-4OZ_rFxyG*VkW{M?YU1Qq&bjgFpoL>UaQ*JLYO2;Yu=#x z-LRE9IjsMvFBD(y7yt|cpq(A7AI5gKmjr)<4_;pObSu8A@6WDnx}IR81K2S7$LLJJ zTja{&Vsdjj(xHD7Ny^c`sd_ILC1avv{B>N(sLn$JZB%Taxg}>!H2&Oj-Oup4Gk`TQ z-22jar&|6_n0QV{e~q#`2DFLyt~`Ep(t}rQtrov~q`pMve~RLuI`e_E*0g&bf6F>K zHN9uxk#W3loJl%6HD#wU2+OqOnrM&KCU;rXTM%}$=^gIY5vB2EsI$a|q7R+0qDd+;6 z2Q6?YjGo)%&(D}N4dLB$>S0h>UgA`kHW8LSc}#V8ynadK77?Py;M12Z-VG$vWG89RM0aKBO!l@jP-MowsJdF_w*r8?L)wdDzXg?BIxz=d0tU7`GOb za7bTlI@!DRl`y7iO8;~ zhS;1=VUo!TvOB&E;?f!FFb>fx**)a=U`||Bzls2AgK_>r5eKqVTIyn0u4}6imh`KN zNKOO(8VvEW_qR0IX^|bq=JCIV>87%P2+mK&%*Dptrr&M_WITD%55JL)l9oC#TvM?m2V}q#8*zsHMfd0AH+xl<^3WG~ zP|^tmvb%Ys@LSu;Q!ZK6iTCqi$kR60SLAJaIV~Kh!Iqz~ z=F1E9@!WRlIm8>w2}mQfn~E5)i(59EPfc{2n039pgT+pG^bJ-3m1VzWxJLTLvs8N2 z*r8s>*XNW`lZaHpUX)TPk>_RDfGDnIf!VYt0SPzkBS_6RhjIb7FK@ZlMXN&}XLmF3 zwE1YGSzLYJFLYbBT`Gjb{R12LmqElVN|$9C{q~noKGG2z51vLY1fKF_y(X=lSxXDC z9RSA9aY=IzQ|i*j3xX{3&u`Pfy}F;XI?Dl6=bf33O^#QG|6PugI4DhPmZ5$~*NCk+NogP$M2r1EWq69t=@vEKo~5Pewv# zwJ6CSJo74m=dh*ec4R&~7;|S2K6(nAYm&P^Y60mCw*^O4MXRArzoQ~G(Nxa>Iur=~ z!E^(MrJ$C`n7=Q3j()A@o=68U4uUclgb(Mm0`=tSbN|j5Y+em$|9lG6K~kGyHlskj zr3{e&gXd?_4Q3f|aj1{f(z?Oo`cr0~=`De}CUi zh-Pg@x(Z7m|3co|bpxKu!Lc3wzz9{#V28B^XP?M0RaVo$76akw;N%sBn%L%GOedSP zld0}fc(c$ngzslB$q<$G9<=>UCXrc94Tp4V-XEg&$Z-@O{G}Iem<5b68mYc0 zNxGYHb1ag?3c7cU!o_;}dfh--geH(`>T1InB@g0@xt)tAwEW5w~53>_?w>2(Z;{p ztT~Ep{oiM{!YPf3efw1Okf`AMa7rkDdK zRJXBupUiA=r@}h9D1`D<3igo?H$hR$5+U%!4y%GCE`LbMOJURsxxbi927zc(WBjxIw|18t&kre2XNhMWY%EH4Jktq zqG_GXJd~n)At$d8X z@8B*cb_%r11GFd;sdfyX{}AD;UaJWLRn)nlM}tY7eB)!dAS9giA`pm+!xgg{kxGVcL* zCF-&CNs?$g$*|K=N*BIH-gMgCg|Cu(^rG7NfOZJi%1+XYehfI`wn8g*bB8MFmvcBqVw-OXZU0*;g!PTt6;t9_Lw;#t;Jh>CgvT3%iK3wqEa*O zT@VgMe~;kVy6O3sm;`maY?oo5vBbaVwa*Mvu)nhCxJIVP6_g~7z5O!0;I+E8@wg%? zZ8yt^^SnS@qQ2zApwt>|O`X#IJi{5S?gEce&BbjU#}*pcs7@RIXmKr6H#^H1(C_^9 zt6AX`;9U)tlyt{7+)RyT=5rB@QJ&vC&=HsLm2v|W{}Fl(b!r3KciD_??HrYbv$6rL zo@q7zkYekskBBFHznNS%D5S^~IdAG%S3ir)?n7&Q^nueIAyg4 zl$*2jyU>!^k+_EumU#+8(q4*_yE##APM}8C?BvlzJ9Nssx8rkTq}R%#kkTQI&dn-5 z(8Eu?uF%TA5ip&~XVs-8UK+dXQw9aU)cm__>T|6B6=K&O8z2ymvFsiz-iK(evYWip zSBBz$(Elq+<+;^xo4d@y_lVAUZVLsF{|6|EtQQ3b`mpYVMzA>F{o;2D`#PaE5>SwP ztr&w%J~Wqyl&G)x?@<6?4?%U&#@S(^S_I3xn{8#PXSctHpQ+*Z;HgJW=<8E07gb~BPo5G+2)>RFjIdWU7dQ7$!#%F9bjFNh{EuWz+B{7y@KMTaxMt=n}4S}San7yj4j z-*eVgZ!?LI2F11Pve7AboyzuKgZS!)8P7OOpPpV$nuYwDc;QynN{44}(N8m%_< z-XeNdSYPX|Lz|y>1gJiwpDQZ%KW<|G!x-$qtl+-4cVrE>u^oM#OnC(lpWHU~YiNcUAKW?9;tob+UsxyLS?lkpp-vJi4nvLnkcJn*xR z;^fmQDe}I(td-dAq&A)Y2^K@h(Tg9|@OGY<3;!Gl(SVa&?BMMyr013vvHTZ;8KG|j zZ9oi1C^L5Q3-@!s!a(A)(%%;oFq^hbcD7&j?uko1@V9*3@bj&bgWcgLMMO)~-zpDI zRp~G<@2iE%pR!@3(bD9XvL%kG@*94;>&AnhX^$*nTBkU}4j<<3C>{Sm?v;NF-LBVp zsp%3Ljs23+diN3Zt7SFa)xE0(YGcr1x;%p-}+91p( z!H(Tuvoi_nsWDZO5;c9$kTR1=NWbx>FtM!ohE=DnZMLztN?}EYmF%7LzPC%|>F>2{ zH0vArP*>MZ8D)tnmUiyBS_`Cn#l($p&XlSSZC1Vgo?s1~gIsv3m=F>RuEsn_cYIbI zcHtWln(^@pPm+<$inzf^X5mybT|}#U;o!8?Q$c|JeK$`SorTFcDq3HPM=Q%fyNFYuD-#8GYIw0PR-V5Y<%Na7S?Ky zAGbC$u&($vqolzA=AI@;HH>Xd|O_ldL1nz`q#^K%%0L%u}4inpr1*^rsd zq5*i94M8U}=Rr9KKlFO#o}?nAETLJucCoAEjhd09#s1U(s3+h5%b@0Q&CHH&?0)B- znB=*_Rq$xUVA#z(3LL;w{Y1WJKlZmk{;TIm?Wo(H-+wgbf6~73Sjt#2mH_hntv%f{ za=B3)_&OxJO!JfNhw+7D^Dg^Fr5`6OtFkB`oTm1_LX~5N;C6*(e@(Lg%Ih2TW8r)K zh8&l^I$D`eJ`|}KT=0E(oL}ZaN%XQRwZ6pG|31;Mc7(m2AEC{zr}||d?T8339uS9U zvavM8vLJR@GxfTMrB=>WRk=1=YvsvxrZ=v%h9!XLk&tn1H;oov)U?wzoaN&U42scOnC^QyR`io0;`>JQBO{-DNw)oc@lWl$-nC$dp#6qT?!T&2lC(zN+Tqe#hwF zn@?UzOtz(17-V8^+V|;%k<*l;{Z4y)^OGrUKwW65P#!V3?`QEgCy$kp)-}$a7%g}S z`%n!7{8WxS80zsrD6ckVW!Uq8cpS`!+#W{x<*AXbt`BKq&8~;YxYUlSVG89&d7VE& zp2;~x@>_}g7MbxqKXi8QWun`LJ;>~Rrl!Kiw=i^P%4Su8gwCs-#A(~(d$xMmtL7>) z>`pun3;SoT_bq)keX)m*H@lXo()lRY0c+;%6NmU>TNl@Bu9A%>C_QfY{=;nvZqPYh z@$ELE&dno%&uPj>QBl3={`rAS|8if&amNZDd z8v4zvr?TyP^qX1usFt9^Mn!^O0={asl-$u;BqKSetCX}J;*V8t5zTcKMvD9w}MAKB8wG#d`i6)onul(o3p7`#U zAfo^JPeKi@R#Q`o@WHUDy`sFdVD_C{W96kBuBHgSb+3Y@=I8mJS1)JRKF#_bN_vpp zZ#{%nK1$49%f0{LqM{-4%lqO$L8j|nmA~aJem3~Mt=Fnxf*W7m#JYr3rgcMvr`?#n z_CRH>M_p1}M%;?fd$84r<7Mx6-l4C|^aIfP@dpkwwzQ{yR%c7BsF`}VoehzG?~a%< zpkJeJl9I`*o1YWrZ7ZG>IypZ+V-$1}4E8#i!2QR&IKU;t+n9aAsZ{~YJ@3L=D z@6Gnl-yTkQW-T%bWn^r#_JE&DZdzU=u$7qI;=b3f2)Oqm+g|0kZja{zJ&I&ZjJ1*)c`uB!--f0`9jN7t5pH1FqtIbu@^Bw29=Ubn#bkW12{%xYe zyW}Zn!a@73x{tCAy;7^PDaFdgDmfz(t$Epv&z5D%U#KkchB~$BC4V_BocG7L)3~|v z7q6pB?ephjwyzS8Ry8#$%rYW|14RmKGd=`4MuI!Oy|^j0O_pX!PpEra}URHhyHrTJ2Eb-$5Asb&5n_KV+TUCt# z-srlrVp|On9`qZ?sTO4Z^BN4 z$d$^T+(?z5qa{_a6)yCtKS!=gXB%@nFNXXayX1Ir#22nw;g0@oZPH@hMK9O>YMStn zC<+TJc&$d@zk$2{6JyD4IQwnII_+sW>d#C*Q>=R4R^wy-lk2!d4FEfkhh`Uw&*PG+3>?Whtn#v6DzS}PY zOB(~deTu5CD&E#Ul2ScA%K24!WUNeaqJ{UzCGktzPbbdiRcTdEjWIYs)sxPAO3|M< z{|PY16LWzzk0mgy-0SLLFyrB7&+N3$PdC)VLcjKi$Fg%nmBSie*^@Wel-B%g{Nx6P z653jaQhoAVFNf6VCrrC0JSMKD{1oQ7r(V==s^qhOC0+Bf$(J?r`#rfWGev!0vWpx{ z9+p^N)AjL8%}t96+(4ePIpSD-MM1?rNe?@k=N{R~G7v85k0~3Fy)4$w)zXtpIFkkMI z!z~22-Cc*-ioZ`us;-KY?W1LRXBKMP&MZwYzaPaV^8og7v?z)sva`IV1E6{`)c@7Nn`_Z<<_j z@k+y2b`=uuXNh|~nUjSHh3TH>+HSvlK0Mp=tMY+X-Hyn|wyujlQP`Uu3R+2l6&BLh z>cuYuWrx~Ndxz_q(@#e^?M?R%DVt_od~qXV+1OQq8wQ5A48ZDvS?eChZcq3I2 z5d4l%R19Er{4s(z-cl-Rn>4QX_%NO6Xx19qGIalW!?Bn5J6?wi86MW=F-}4aJI`lT z|JK)El;x^4?fs_DWrl6h<5m&vAG&-yIRaJjFV@}yYL|3W2YMF?sKmhItg4#EJ&~Em zKIp*eL4A2hc1JZDr>zWx;cY52uI=R2)&@u2#;hrGDu z1Fm{Dg&zMyN>nV(%s&#fAXakUk;N=~Bu^;t=n+1MYFj*Z{ch0F$QCp2)-WWc=IAxhs^bfGP2a)_RqAp|Y3+9{$Ajt@v?oss8nh z>`UiiB+Ki?J+IE_0bm|iirDaHt!*aVio0i%h@lbT1yrKrLxp+E<((6jiQqc`^_{=9} z5UswH^|_&Zua?MAW?BAz(CNo4J=kK*X&gh*Hncftkom}+qX!WGkiIXvLs3#--Wg;* zwVBc5b_7);_raF7JmU{)&XK2#!Ib)7WfUFnC+7$r%7io&IYq~4vrv~qs0u;XA3jxk z5So@nm9)6a>vlv!E`M0n-)IMFumfB%iGx*CvYPiG1W2>k;H!-1jUf-10~U}q))Q|fD>FOkE^r2(V>W1+q#BU){!T_E?VT<+{>pDwh+~4haW&^psu*vx zAeRpGE6Rm&YB(b(Ixdv!#TLoF50d{#RE#!Xu5ye#{+f6OHRugWm+U(e4z?@z?z9YU zk{Gm*qJ>qFcT;q5d9{>vRC`)oUA{=K!ELXFmhcmKxwgsW)VpmtgmM~;kfMrKss1D} zgoTeK;tN~&#$6Y=(D10%ljQm&+4arjDj!45hyeUP@8@Sj(tpsj1q@4|->G z2x9sk zIS)$py4=dqe>LM=o#I2N>`)vCkd-d(O6M3a=WhiiB|IOh?-cjS?$Z!*_K(!-B6_{D z)280v)yk%(x=yvP9D<~Z4(-u3?P@#A5;Qu0AP~%N=>ucB0;WFM^Il0%5o533kbSKt zIN&5PU}2*rizKyQMsB4g1->d^04SB(z|FFB1oX!$tYnf-_dQ`=(6Rx9M!MzMeSauXhOo-n=KJn&exJvom$qaQB$R@vKlXosS8g% z+wI)7O*R%>hZ){z7Ff(pwT*1OHmL)z)oP|v>JQq2nymN2`9fLBw6~ZXC53*p@Augh zWH&x0G;40h^FgonpeJSP)XU;61gzA*ws_UNS zqU(|!B%}hB8}87KVN_oS?U7t@issm<{D*rANdmuDD>c(C{9KazIW1vq>V(l0aw(&xJ2zxmPDm|` ztw-CK$xTgZX&-SjS%c^+QV30(l7&K-_mQ3B__j!Rl3p;pP?CA_CKOdcNhOw?kiDxp zy~6Abp;T~gpwd1Gn&nyQ^|$3-8kY%~Il7nrXMfCz@0M&#Al-ayjH}1`7XO3@-ey9bCqTQ0+zw-JO6WSTx%+*<-yV2QdZ73hkz1=RM- z$!Hd4JpTfII{SoO~)EeA?_cS;)PG}#q#4NW$Z zr2OmjnpzvBKhOH01nUF!x~KZB9kCdMEB$2nho6e}v9jJv{VvsmLbiO>+Mk3j5cW7c z!GOD6(Njo?FYj}|d*G6aHYTl|>gf4O=hl?+!O9jV^x|GgdcV{*S9%+zw)XDlw{(XD zs$bBfo=7Vy%%GGO+UyK@h9)dm0m{{^a&3$BG_}Eh+>eEsB(k7RYAe^2#avC1R8Cga zN$o7AIuAfigv4RU=wwmzM2ZKcdJaP0E@I)rh4%bg-vut>ix=vLUA%CSf7qoLZFb(UIzu$Onw3u%wRBi4Fw#)< zx&lWjHcNTOVXkgh@1c#TSD@DpwZ~;!vO0mo%V#7YD}s8?)du!m@mWaQ>r*Al3Qa#1 z2@KeivT|Uv$zDnowKHc1DX5c{n6*yPUBdkWeLJfP4lHYwq(Lcb*l5Lc4`0i@329d5 zW^ZwAk#>Aly|%B2L2Oj04&^@u;?zlZDeV#tI@M+BPNPkz zGIf_h>1h^6O=Fp&w)Vp#b)QsO)cw##+L=`bJVloBAIL>Sno6v+j865x#cyn=;}74P zRxiLWjd;S2)-InfkN+V)t@h@wv2z8$;-s`B>$%dtXwjnF1INChg`4-i-^mVT`uJoj za9*&e9~b)brQ1@`8-HX`B-fKwX|Bmc-%kVXh`P*(glJjS$IM2l{Y8fCWoNm(m8RWM zABboh5UTsVMk;YxwWW8|XmF6V|7K0=SO3WQS<(7C3tTeK(fgZJTP1hM8;n@WW_TjJ zfmK&or5%U&Z3@lmd`-JQyuUVdd9*u}mB%-|{pw>HNK>i zZCP;Pm6W31+59aHt=d*=wK5$KK~%BbnX<4ljJYF)J#DoQ&ev9%=KjRn_q1ZbO7paO za#!yN_&u7>0YgWF-&5=DYz|MXm~1W+1B4@ikP?ldS#t{OSfUI|VTU%bsvAfKmonRU z+|B5B2Cb8-tj*o28E5#rf`)2aBfELCXn!=?HQP#qZJyoH*-}P{$o>mcj#>S8+OgFB zRjDJF4}r@0)AHO;%X2<0@BOqq=hJYz%R-;@{#%yw*V-TWT>E$7`;wBftc2{p=lqr< zMg6l^j{27cyIN>U#Y$^A0l z-#L=hKYJyqe~DGpTf*XB!tc2?mRs)Gzag{AF=Kkr(W(0ApaXr#>dRT8$KCrA8Nq80 z5GX5oziN%4v}Z-KGKmWT)u!H)^Q$4K)tb^7J00Q0U9ONoI=g(ie}+6QU2Y+OvNkU+V=|rOOxy&HlqQGnqU9&T(P3nldoQE4j?&B z9!E<7UzGGB9qa7jc&^yPbo!t7FkRg0VS7NKH^Z~(j8?yCUC^3d*5!x^jH6U&@1fDbY^&d(nFV!IV1M@Y4O6-(djwA==q1=foUQ! zbQ@5vBhxrHyYjmR$D>h?2adfs@)et16K$F1K0nl7A!WP2kE(Cxf_Q^ z;W?xU>a`*BIXde>RU^f;lRqX{KTWL-VU!hGQ#)^V*A*;#PmO37lmK%DO*7dHLrlh`T zBXE@mT7N%oEW81#N{tt?L;(UNtcTTH;`RBPW^QKqQp40AAXZ+w` z$+6Nvdzk=a!izi3TRRd%(A`=%?$N60d3J?8P51Wo;bebs_HLWf^VX)$W)J@6vj*Lt zHR$-PLHb#ptt-7cJ^ya&yW3rCal5U5-^%Xx#aFNrf;WfBogMR+hWV}6kS5f+&yHoU zz?)l_>M?(*b%LsN@%sFPzsWu}w2d`5^K0nO{caf^w0WGB-{4R9H{iH?pmk318U956 zY1E4HP5y+x3CFV+Lr9dL;Z7vah*E_o0gX_xi}Mry88}z$wfsbWQ?Z-!6ZuWWZpu%z zZYuRlej>R^S3~=z>_mQ3xg)X@`Ay}H$WG)pl{+Fk(YmSF5!ng5NmA&@FizAc(r)94 z96#P-s;UA>flPMvXt$v3lIvO4wsX?)8p6kbb zUyX{wvB`Y8+x^T;TEt&ipIpFP>w3xQh_q37@fHX-$U&$)fn0YYzN8wZNy(`Hreeg?av<2mQ212e49K<$;%wP(ZQj>eqz z&qqDXoW*YTq<%Up?{@-Iij#=w!P*DszU6Q;l$$m>?st8Bhv`Pg$xcWgo@nK#7#==t zV`K@d*T0vaNcwX5U8A$pvJKZiKJ6IJ{_%h;R<(R-LLGKZ{aPNFv*G!%UCHC}H=wm1~da^b)%JBI==bzj8+~! zbXKnivvpESsJvEB&UMnLt*?-TGaNH#fF2`Tblo62_*lOoU5{3lo*7u@p^{ifM~v7D z@ibq!=tc3$qoZ?k$2E9f1^4YnjWKy4jaKL;xUi~SQ-0X#=)63i&W<`v_0!W%x67!* z|8O=q>pF4H4iRx_Bbkv_<_(YOx@;Do4UfvcfV07dqq%^d)`e69Nj+V%lZ@sj4sFZNJ?U_~oSz<* zhvocqSeAO{ryc&7^Rtc~J3m8huHuPd$IH(~gW@jB>$0?GCnvnxkeu*JNIa3LH0g=r zJ2D+DIZ=E^u1=;WO7F#W z*L`Q#`PAHf|9*Ggdg=HWZA8HdUisZ3GFed7qcTi)%~E+DQVa~YzYC`)a$2Z*9|AMQv^5ge5J1D^N!7Y za7GGg_%7uQV!b^(m_E~M7+sp%S-;<7Ow274d30!+!_FYdSy!7!=jUay+dn=Uc9;vt zgRb`+50ARkz9DT0F7Cr768{#5DF_3pj+19+XXic6TFCD@ zKBrl#&}`n{yA-dp^A6qPghWQbPIW(^e@Z(j=_2>{l4;v<(TG;x3yYik`ssF8L7VKl z_I%VaF;35qJ0`~I+3`^;_}IOM&yMSW4{p4R~)FaDj%^+1A=pXUv%MX_V+78G2eEiqTui5bd%?o~9pMMVR=YW5X?B|ew z9`jFW2+t^bQ~J15*Pi=N7xK`v^dLt#=fP6*(JMaa7woC4rbk~-gI2jv1OhNNAO%A2l`FK zfd1&MIL`Nj{{=3e|KO1J%F2H6U+f3PA5Yc3;Xh=v>OC@C5dZL9^sn4RfABr@FWVKea9T96GY?HK?75CpA_^S^ko{_)mU3 z|CQX2BU{o><{Q#wo~cJl{datNHtfY(+odbc$bz+cMXe9lzA4|)VK0paM#B|crunE!&Iv%hBr$LS0?0(qaybFJ}FEGDA$Nlc~KfV9IXlm!B z`BVSd``&AIcQgK31K;b?_lEYJQT>MlUYLnu=0ptZWHctb~sB%UF!J=xciP1=e^ZpvW5G0Lzec-#r;iRNXnYt zqM1%sb<=1aDf&*5hBQ715z#5`B>b^G%y=a5_pIK>E8x+WD|WQ5t2NY~_*>MV@C1LF ze!8;&X}b6GA;i7hu`TC2&qdp^ACdk2am&xHrwf62m9gs&mGY|(jz(w@mG37>{I!W#o+k3 ze=c8>9MPP5O>j-mpT`#+AIqlqn+@)*j)90&?;6UF^=ll1S=p!Hzmp?(lL*jlZ zKMw2UC%l2LG-s!Wx8wl-7z%v%P_}SRk2=D2r$-$gkyGf-@?Z{#?fFdrTYrFC>X*|Y zdbRgAOe1@fjKM_e@^>x%$}=4;mi#md_JK0WeIs zZ2p{*ou3pj=33KtgF|Jiz9emr@boov0Ud;)h}z0)t{bD@=29jz=JPu^E$#Ht|*=g?E?_K&=Kv!{$1s6=sM2d^r`$w`Qf;h3XSS)QSr6z z;A*->n5N62IP7+`P!|IlI>p zvUohK=ixBKS9WUqGH*^NvjZpJyDVf_0NKgik`H#|%AED}a(EUwfH1I}{s!PR`1rHK z(?|NU>6648&hDRgo5OZOz(WI&c=Ss^l{*%9e zI_CQK31;8*&Bt%}?8$y#JY`A;JFFh-U9*?u|9J52WHf1-YA`u298(PznewGu`pxqa zm^j+Z6K;MPdua7M^ySkCILi6#Bq?O+kHRen0YNn zVV>9tjR2I+6|c9{nCfx9InanoS2MSTTIlaO@`fi!z+&AoSoHg6Uaw2cBxz`JWKogP zeL#4N-kY2ObfN8%$EFamk#{8?dQWP2Xy%jE~n=6e3K zC#46-C}GDdF4#?Iu3P5689Z8?f*#F?H70GQA9SUh*;TI_AyAE%qALuRaQ|=zhR}9j zl`df8*Dipg;cvS7_wn8J$3G{x7k3w5|M@jxP-w_C4g-8T8+6z-#c9NUNrc7Hz%<@c zt+spG@E1MV7{2`a?e2a;8^sE;C>}HEBu5_TGe3NP7*?g>3CrE+*!LsNeHe(|NFIr^Hf7q_=%5$+pv^X+Q-8e8^odh*I~LO)st$2bhv_j_6}d@yK{X@8WDP8YMnq5W*BT_3gzyVE}l^WpmA_2tFGm{0m; znF;nnS6*G;KWJ`qc5<;@W8!OxmIfS>^6%tvIepSj*)11a{;Q^cU#V;CU!3V(-`{`7 zAiI}Xqe)W7qp`v5E8P+Kgy+&xtd4XGvnQtC&o_^+u4&_b_>&LmYrZ{vf~<=N@Dco* z<_T#N7icD8NcdQhu~4XXfaqxUK%YCJbzW@H=;cgrnR1aQl z&ilusorS~gLxtz)B1|3nXN#*BixotfWT57YOuC{Vv z>2>C;PtUCz-%M>p?~R_dUQBmn_=f_P^zjX|Pz-s`c^#@?jm1Md^;@ULWr|FD6llt# zi|g$x;>5iQQf{&)?W3tMyJP7%J;f*F&uTTTm-_dQdbyb61Co23k#WLGz1%{k#iD+q zj{_2{S=Nh+X>k79b!WDW1}8(M%z5V`AaUuU{YH3niq{+s%lOOD(b_>h~6d-iKOnugx!`^RtuQs$Tzc ze0+LJ_VgS6IO{RONfRL%Ad~)1qR?_~v&6R~hUk9!G?)Q)feT0zg+dA_Y`sTRl#i0s z*)9;vp7=yHN8*|q7(wAx?DY2y!UKXjM+F0*GOiW&W-p^uFjXXfktHp zE+r`8{0tag7dL+fnW+bV`#!!Q)xe(t6zya*nKtvf7SgD-9aFp>t+#S^;c(PJaWWj8 zmuKnlxFd0Ecrq$S#)fA|i=1(_p~Ykd!8l{4JPwA|FGj6;3WT^#47ucWN>r`_0^MOfLD0Hx<5r z|8rXz!AtLk4++sE^Lw%lyxfQOd_!!TUjKy-bYG5d(jH~+b1NKq)(%I;LuokjtPqYoD+f7qFLvP_WZ+BLK63SEb z0eL<3`u4+^uQ|pg1OEhS*@yA{0RMcx`bhr@aG#gk%crjwm;Y1mpMlu#F|R5*Wd1xQ z!%*rTZS@y@YSqm7N1_Sg080k$3_)wXC7>?TkoqGMoY)lCP!T#Fc-rxpA@sWsWWitG zO+H=Re=2Byt+QTt)RRT;6;@Ea6N!O^?tioNgOsBqy-QJW(2 zz9T2ED_y>iTq}-#-`|TE|rb{uD)zgok zFfTt`++Sa|tkm|=){$U*4?}`#uz$#K#@@@7lXN?EM;I!}3kF?`<9b%T&v5-;UXE#a zke73JefvYB2vkgt<2PI;N*wbxeZ7HO|+Ck7sUV z;Mv~x;a|)#MZ)Q?o((cONgOzA@{;+-7GFajnH&gLOy`IF=JAmaZ09vQ!SL_x-M2sf z6pti-eanR|7pTi1I3RYkThJcy9p&97x8u9(Z&$t6kwiO0J2HYrJrjHHE&7e_8+PHqTMr{S4&pX5GtDfRqGl#U2BjE~unmfNp?gzMfy*JeF%fqhTqDGp;chOWU z#jw=B%>V9wb|?$|qfr>whd*!I9$tO`jmr~bb*QH5tOsl)}diB}Sz@qNEA4|?w)4(;45Qs3O;JO90rBcS@hBSsZAU&H_6LEng zhSbv~cv5Xq;zZEHmk6^Vw(l?Gsi74HaE26+9dfvSIXOQ)hXav8`itpL5`ay$|K+rQ za@=EcnAI0pArsOEmm9PVU2!BCd`TPmn;2g|VV7jRRG371f_iC%ETH80*7~ZV-e{`Q zDQd@b{@0=270;GVZJPQ`!3+L)^Rs!4kmh*&QHCjNx4l5FIjncpW9j|n&($Wh-z6tX zU&|*tzLn1vPVg5M`<2fWUbERwu-D~RmfuSkV-n;Sm#-w1rPZg!rajKr`(?+erDT7$ znk^^Ft(0@66a1p~Jdopsw;a~|sJhVH@~b;9e`mrLCWTDQb~_XC(LT}fVWnAT@0MHK zTQ!ST52X`7J%u^ggtTEhEU(Otar5k0vWxc^FJr%HKZ{gggn>#YicOvUoD5ngH z_j&TiAM{_xHS|~$N*(uioyDD>kNX3w=u*63^15|1m}VnwQF<&cjCiaeOL=i%)Nwqx z-Br)Di?ptB>8sh9-R7`fw2rQ(!g`b)2B}xGLKG@L106hh6fdSmi3CS)LZ~E3_QyRr zWMC$m8^XSLmLH>iZQpSrR|jBUjHm59dkfdRB^QOqz4_-!>)r~@%P-q*epI`+l3S(p8#;lS%W9HYPH%js%M z2go{rJD`yiHQvjni`Y~?Vg*>m3YbP>+(Ie6}V$n_o631~!*|3bTfBM(1T zWJ}4t;SgtHBU3CK;TK5QF}s$}^iR47BKXC^T%a%ZKja#YorS~ts_Z~=ryMySw4aq< zbJEiq^s=5CW+g$kA$4~+1*b367eVW@SGaYJ9l$ar18&C|68mj0C&8c}WfBpL6-=Nd7q;=xc}Mc}1Gx?f7e+FeV-s(3S2v6tQ487VK23u{c20VH%}3&dH(e_N5BRFH2OQ{j60m z6$Y!}1db!~V%RJfJ*l&~x2zHKW=5znPJ$l`D$LFGRe1Y zd!A?c|KU)&9KPWaIbd&E?_KY@5|NA$1F5A#rXRlDjroIldVqTg8s4FQVuEoiOmmCg zqU^FnOpM<;CxIZzgKgZS1RKKKImR}~ZnH4-&rLdrz46Vj*EeGx2fRXqJj{jxkIV)s z-?!N>dz zkuWL0iJK))n3qW1OoU)WTBbnH>;D~JJzRf<*H)hO<=eN<-*3h5XaS8nj+mzTFKBf> z%Y*oCajNrgcK3fZ(Pt^`f_*ulaFszR+CRO@Auc%!ixF0p68QclT_dPq+N4P2)dodq6_~ z-4S9Kqa$sUk&J=Z@=ooZ>EhpEnsB?KOgvo(jm{oS?6`OrS5v6=8cMYG`&OfsmdW+t zR6>lTq9UR5a~>RBqTSTpcuJ@jyI*Z5G-jit^OK&ham~}DUP!9wnax~ zZ5@TQ;_y|AXW8+L-G$`weL^_L_u&8rUgorMNC*-vKo61wD`zDmn-U~#0M zC1n0&4729p3K=Xcdinj~5?dSw{KJ>)n=5=CKDWgO-pODV)?goBi2VRMU4Q(_osS*R zSMuhaSCX<8i@soFG+#E+1s%r8+ zF=O+NMbx=!+ora*q#(N47=NZedK)NUmx1b|CHOHZS5{qHADR@K$1p%~ie*=r?N=sG zL}636r!s`r4EoOCceMNOcs`f~5z$1f_!)3*^0?J%7)%Ze{1_dXRES?X2nYX+t{7^NN+%CbgPGutrrB{Uky}Q{1^+ zJ@r%Q!hGYlL0x@zAPw+x;e4K1jpaqfx$2keqGF2wXP z_aIa{B_mIm5c^l%3WD$(nHP8*c#qjTX?@4mE8_Z2D3E4qyB9;nugSV;r-GxuHTNeB zj_Idy&%}yk>zG|8AnU@!j#e>Vq{|kfsoHLkxHjg`^n5-&5+9TO+kOCGaj^6mEP?O) z@C{0@blCDy>Q%uEkZh!2G+Av^wt~V_*e#CjF1Lf9yq%BX60j^d$EzLaLcWkWJOWcZ zZTR;`;+6m>Fs=IOZmD^syNac0YK=*a-zS^1MO%$G?+vogp_csQl|~coN9vRap>ulC z6Sb$>nfb!Ky4^iS{U8waWJ+pdVuv;9s4ZKk(}rO33dzO%cb3-(3HMyRL90r1I;1CO zZ(CPSn|%!ti=Csa5E2=8g<%?tK5|BrF>x3{8o{)jyrh8X zc6K44tH@l))Dj7qn-|73tj(5pfkXtCj)_!woszMvZi%8a#OSW7)q)mMNbqL=1*er; z&RobuP@MuQ8h`D;=@p&Lr&0AyI|_O&uv#L`(kw<@*@Zk;shADHL$)Oso14RZ`w|oS zO+0YSY`Kj45=U(wJAkEpYf+oI9E@LW({OQP`5-X{&|fcalCWUkxvN$rfNbRB-Yz++ z)=26iR=F|9^I7q@jb`%xxt5?Vlu7S@+B5dfI)z0PCsI)}XNRYtHnSOH3hI9{o7bE* zYZcfN4yCD%Es__4Q3BI8_4_aAqT$u%$NFhET~v1rhZy^Ap(am*VnKpKO0q{!$`(8* zSvRuhg2gWOJ2tMs63bhnFF)3DRzxfmd#kGIMLh`1TCd(TJZ)LUfr*JX*K4nA##Pdu zpc%ITwET2*sA)YeIwVnt*SD616H<;Q|2sHcK2N;m9DuRZ$+w?hA>Wgn?fqo24f6}# z@`H>{XOFJH2nh}1u(bl^PnlzNg>rwp4GDSMm~~dq=PQX%u~jP39(gwJK-Wx;ZQ2H_ zZgAtUBMA(O6UCFV6YLq5cv;HMZWFzjr0xll-bf}w8F>wgTjESm;$T5_hHPQ>4Ck6S zdL`GyZ(EkJ?-9n`k{Lz+ilNd3+9dJ*uwPgBH3Yb-o~V^fqqwW{4)l0HVjG!$?981v zgUPJ&uB+!wquc(s%ZKqpNH!+18W+W3&C4RUTx9@IV|)icTj1-W#3WXo){5u# zZnFl;5%(qcU{2y7?l5yxD8(wxro?m->oBCuF&01FfGw-?cyYpV{XAkQ|LPX8Caq5FOAYob2j!>g49DUu_Z?wY9 z)oKE>*`h9{xn}1pL`l(k_1Z*7GEL#_4@!&n4GgjGA8CBPLIL~1pF);oj3aPSZ6gRb zgc0uqaTDN##;`3tqrS%U3Dpm-uq~l;NnPUJ)wZ&W!7*aN%}Hj!uC87yn-cg~9&7Iw zZ=MhP1xfQHG8J1Q0++@`l?6?Se&S5;#MR<^J@_(OvKOs*KcJO-hqa!Lt{cdFjwV>l< zvv8&Y?Kv*+!m^0-ucC<5c*nagT4yj1WASnkyoyXI-~ah}#T_V?UDyxdl~ngNBrb2R z+Z1j@+OjXo3YQ(|?yu7*rf>j8E;Cb2|2mMCz+fB>_o(&{1Pf4=wYjPi-!4NB04hta|9mBaDXHLM%B%VgKxR zjAPT!aE>!1TBx_DFyWvBF3IgYiSL2u84>0bYq*QfCg5d0hwNSWo z0K*l90Tn_2qRuBaTatj1Bpk8B7Yge<-KwLXszoYR@*(yiYDo2OTof6~F)Fz5gxW}n z_9dWtw&{mCa;gf2=C7=#uyoa-(qTeI#}9md=DnHD=lH(1A|JK}tLoz=s3j3@r%m%C zI;=IvoHrbfV1%dIs8LrRi4+sxVRwKP93@4Zq3t7_GGcO+jn*Kp?UJPIgZPX?dCAma zCJq_UZ0r_MH+#Nat)5v}MfJx>*t$YagQL+|50Qs@%fVn4GzepC5b@$eNHlT&7TQdL z$4Gp6;(=b)Q?6rkloR7XUB)Jg zrQTr3d^JW{R>7~R#+s0;Q5!wU0E7s2MJ#d`R0@VIh}{-fTk58x9lnxfo~b?@D%MKQ zJ_7iPY@gjaXkrr37wARiOwhZ0DrvMA#pT;r1XaT4(fbojFRh5l%T~zwWifEA0(j(Y zQm-pA3@|Nds95rg5mql$>a)TQbF1u5gf=dLBU-GS_GqYzFzsXGW&7o2v;z=#Eu2Kh9csbrV3=VYQd_$n>t}vht3D1om+9#5ot!6k z>^lKwVdy{&%|6q){CpF~0Ppl(lDcgp2&55R0QKWig4F%fU#qM>C5iKA9;P7F5rZG= znp|v*loMaCejZi11rrjW0y>YJoeoW%qg1T~=8jkzd!L|@CW(CtgJPAM{@~oYS2DwS zVSl|ko;(CdG_EnnjxGo@6~wFwkZh!axX}@-TB(O5XbF@?T(TGqU`!ackvQ;T%r|{i zKzj45plJTJi&J~I-$EII*k&Y0z&)$sX&0`Vr>J|nRGbFY?JJevK|{=Kgm!W~*ZzCa z?3=EQ2XIXZ*(tY@DN;PbnHLL|VsNtLNI^0&FhYzoGsl!^w$TTxp&%$`J(5k}ur;z@ zYT}fMl*TMb|8;Xc`L_PP{<3kid@xF)Lgql&HZS9rPdZgy;xwd>BEyfXWSrkK=!*8U zV4bA8BdMvft0c;Oh-yl7QazgLx+S}jCIKUL5W^izb{l5)HSIcqLGhkqH8UrxMcjyu zw^Q2lsyVFLt`|HTI!NN7ZM4IudkIMMm*k+wW2;VA&|gx_BMFK7(WFRlj*h_0dU)P# ze%z2bd%NM`9KRuyi)j*7AY9Ao!P}oO5z_FP!Y{)P5L>fyIw;1`2`7ZIw$Y@IOY&K3VlVsw$t3ue{Bnqxa z8D#@_BFLO^Di1sg=tvTJGQ!-Nw}PknA}-k{XmYe!Cew_BkEsud>p=8HnE#~P_+21i z_v&qBsJ-;nf+y2mS+=SmB@EQNSyT!+3cVVss#7V`wMB8OeBhJ!<{X%W^S{HV7;Ke=-otHUeG=>m-d{%i z2itmCRlqdD!yO$kH;P4mIO)eeOGe!hkfwDcZohrgQA33c`};@zGV3Zm^_7a$PwS zd(o{dTwU^|X^c_W4`ph=79g$l>-UH8AI251v)HPtzlKl1?#|D58?Ng^T&C1q905ha z%okEAX0_x51i-|ipeonAv zY-!eP7S4epfC3scZ0GCE^AAzLtC%<4GC{|+rpU$vQP2)_adA(3&jU&yft>MwU+t22zcp7nK;5N?Y zE{c8@{M!8^JK5>n_D-#K8VS!sU_cu7DWYLqrSD<$*?PPklAEq_`Da*r1x2wE6Ll(I za)VR{A3$=Or0V9u*LXgdY#wJVC>6=^4>#&h#&jn|^Z-gpk9s3zve{O1&LlAi5vygL zo_EpWSTAYsh#WiAXrX%e5YaaefSua!+7>so7q?>`nf%@fl$~^<%Zn!6oL#e^dNF4A z?k=V$M9>56pKlBf7%+6p1*H=EP=HVGA_m1drY=V=BKl^_N(<`pXxY-DZ{92F_2Ic}!2l-OF;z8cr4v+ZVUbE-Ogc_d;eMo4_sQBrnD&_iIvENN>bi zn5_ph09}|I)<-72p;*qiHuoX|+KMn&Iq!rVYlG9^E~EvHjkuat%PA5DK%lnJvul5Cg{TJ4ek9S?u1;E$Fi^@PUc8m4ULJ3k7sV{LG5a8 z2$vC%_&Fvi(tFt|=J_nje_v#DmObix)^V0lZ&8=b>U}^2xE?qF=I z-EJulUKC_|o+O;YUNf|i0rO^<-*v;R@~jfFv2xh&Q8n@DnMrn_{jz7m&mDv^rp^ih0Io2z=4}mXNH4!f4ax9AdovfT--lQp=Ep*+x(2;H+ zbAg2a|1v=_$xj@CgvwGBX%cN{E2YIT!~WjTM1HL|hXxI#2X2~rU$rK|lX=#IqzGQf zj@)*a0fmBY=*7>)fY13o+4%L_7v#&YRt$8-YDOr0%j7*`H@Dvcpk7d5VG~#gAnYq> ztS^jH7;X;HYgIqY#~OUA>-}b{n-^I#VJ)U1d$y%uMMlq$%6==x$etq6s=A_GoST*I ztD^M=w`A(M&M|fUR(fZeDW{^vDFlP_64W(t)!) z|2lp9X2~C~=6+^%PJ!s~f589CJsHYif}sU_0}?tWDBi&fmuDhxxU-+I8Z#H2aBN{F zY(Yq=)bf^CgqnYe&d-l@w6ZRQiM(jlK(?D714cQBnWW$biLv5SmRXzyQn+9zD+ru% zi3YASSA|Q4^FtaOUTp+)*Z?#@%fFowIH?lfXy3cdogr42%V@Q8v=pq9gX&1zTSJXQ zRDt@QQ$xJ*haL>zj_skwzP?wtgbT;ji=txy+Y1J6_k*`-UN`M*aUXpzEJ#fQVj&S+ zWU>K^(#EzDhcYEF8M;wmGbNP0a(&UD(5}U(`NrZ@tO;a*oX+*fH-e$+aAzTJoclwEi2TPE9x}({8W9<0RA!lOl>XV3(?O#tCkJ>*6AfK~Yj(l=SjC%17?pE=l5)B`* z6NXLCXT$KWB*x?xodp#Ik73e=>C z`wUYf$wilppP*l|)M&Za+ag4>@YO^qfOy%-6X6G1KPd@&d-Ii*tIfJ%*HGwb%)Xo| zL7WiH}NS(8>cw^W z*aCR$jC_gwHYNfYeJsT$3*|B@AMO}dYanS8*}FaLWp0N8s|nJN zcZ|hm0=>D4sZPCFRUHzw>=mqkhJVl5mcW%03~d)D2{Qrf$VWGrUTZa-2Q4dO0ONKj z+(LvL$sv+aD0?GC(eaVf?Uu9G6GiUvaRNBTl=U&VHFQhB$%zmsO~&g$iFOntl8KvMW-#~2Z-JuA(c6`Kc5GLbNGHcR zC5y!vETR&VL=TOFOjX$$PQNH#ZNdh-03|?E6dTe7@DwE(c zlTK^H_k;bw>^Lz>v098;Y`F#RB2bc009f{F>If-nP2vTQ)M&hDZKFWPr*Y-Fm=^%S z3hYg<1o_egtu&7%ENZdJMUU(GkWbiMbJQl|%MzC{0%kAua;fp74C>93S^^;qXoLID z1e?Saed%FG;DNHN)22d)WMRf&NI7yvmPoWQ>+W2?=18`sM|OmHCIgtO%{s`} zKIenrKE3F&Kfd*$S#uKhu^eejb&#F9R0lpl&t!t;Wq))VlpT-7XpnI}rU_-#@iQP( zcIZfaGK`MXtuaJhWD2X*z7Qr#VRpi@SB~*XqHOMaf$d+Mn~Z2O___y8(B|SjFPK@j zG<6@aX(CFEl(RFooA;MA174$pGAmA#;11CSPoR{>5e*S0-e2+b#Kq1$m$us535BuD zu|B38yA6;p?Xl4G)CitV`36ytcvKfXeF85q$yHP@r$YhoFf5jH8Dz; zfLs<^O0k0;ft~QNF`Rn9Pgc*Ex~!a7&|aP;nKi7A=?20)5peIUC8NC&6rC_K(2xs9 zz|N(P_Fw*WMdp5sW#R5}ltJsY!h{l+XRw%d0>OUw^T&?CTQQN>>OIS1Lh0Ul2euHh z!BkA+me8HSTS+50|1JleDeJa17_(1|ZQf@zgh7>0$oUXYkjBz3iR718q{u$k2w2hi zt>l$j;Hp?jbDu`t1y35C=a(Noh*`&x@JODRGJL^vU#}5sh4)B_--ew2&v{yE_BykZ zbGmqOhio$yUC3jCV^FijTXY~BCSXfhEW#E^J+#JhV8t&Y8zxq$k92B3hy3 zr{UTP=U_=-l1?H{&PP@&Q@as5Ys#`2ZPCu}a^@l(OTqvhIVG$GEH-KOcNBni9cXo< zg3stc7e$MEqZBf;O)>O1FYcEA8`DmrEHS+!i0~zXnwHZT(;_W29LRNe6|laPEGrNA zYO(dQbku2ep}1VB7X!np9d2mE@I+m zOUONfdYI{?p2DFyDxJDe7si)p2{w~Xxk1~EXd9Dox5=qS2A`|JPh=CXk?uhMT?x%(Yf^#0H z9%4%AE(sS;8^V&{f%m(^obkZeHX0XkW?`CQGYudpS+l&=V?N}ZO;6Q6qon08gkVS* zrUqGjJMz#uu=^(md-0hXwc_>@UWq&mosfkM{0_t$q2~OKZ!J#5{7GI)Pd_(Y|Mn)l*VT){A9T1RRt}i{N|Abu!G-IL|^3SK^@H`NQ0x!4K== zz>(3|g)V=?Vhz5?1U1)I?$zs>g6Dz<&eIHw8F$0b9M7T`KDiZH2A7|!5|G$vq#bH5 z!g{j?fFfa+cF=^wdNo~7pQ=R!DI}RzZCpZx#`)A4ss8QC(XKKU!JUW5G4|=(yR@3}WWpQcM*O=XY zey%+`vDn%G5wodXNFwqU>~J7q30ns6hFd?IlUiR@;1iGNg9Ks_J_8+(L0-byelJWU z(U0#8yM#>6l9b7kXJ=H2TA@gx5c#kJgt{}=6Z}POtPeZT)=?0h8Gk7Wo^yqa=7H~_ z1-^$G_&%9H9F-6*fuf^DhxJ=b2v%=8qhzU9md`5gg&YC_WhI9QKz_#X4_3B2Cf?IP z$Z9Kz9Yjh$nZ`sR_$+9eCpFJF*cPCj3gO<5SI?H6o8<#}s7oxwmgQQ@%x4o4Ou5D$ zK^KkJP`nmj1fg!L#k^uzO7!#1nF3_-d$AH&b)-81VOX>?1?cc{^OWUZg)CC`k#ltO zDMIK|$5<8oVTr0A(NSrok3grYjJ(q>(8GC6;}z3hlPaDF2?mPBCd+Oe}YL6x2y`#I^@%qRI;F0@@H3DT7d!vsKQ!N|gzpk}a?;MEqr5Ui0ugTDUodqGCXB5X0PvnB~NY*_m~1}gDJ#CT!?7;q=M z%2H1mgOiXvi_&1Bvq2U*bBRMHcg4-h@<%3{!uqWhmZe)*1Tn>+ZR6ftjXy#Tf4?8! ziSjNFBc=Vw)w9ivv6cI@@LSG@bt}W!H{x_jyStc5{g{30n>0FBohV zYgy&4*r1^`7&6QcMtdp)Bn<~U23@vJT2^B;YJ~+QAk~zcTnJ6=UHkz8irwKMULS=j zzzg0m3n&*&E5J0}S9#J*ZUdTZ(t#4a7Y%nYQ1LoX*SuG|;fa$>kqVuJQYV*e<$&fz z_T}-cGa_hABw~TezNh$`1bHt>t26jhP`eg$ETbn=!{Kjg1%*&x7(-r25>Tsz(3u^V z+7>}Ey~FApbK-3kn8K?x)MeJy0PzV@katE-4YXjOy5jh=s7g3 z3<^tpT>=eZQR!*#qHDY5*-P;CZ8U#J!~(L_X8hIl{q2{FKOYc)6OI=elAt-Q7RQ2D zka75W>vDP{#M)z_xTc6>Q%E(*R7Qe~08MFSDJ^8hNigv#OW>D(GSB?3d3?-A5yvoM`ZMEG!K8Fq-p!hb#oYoFF2h0tt$!vSdZN9na- z$5`1avf1fRhe#CFxjefRn;?MV5RWyVyre6QeBBQQrb!JeapOzq!2oA5^w$c*29BdV zX*Q##5meUusfqowkeV2_kAIqx=q+fbcPPqoPP%cO zu-_}15jDt3K)#zQNy{H;3O>VoA}IjAa0=^1K04r7l|q8<4vSoK;aZY3D+;))n!`${ zsox>7x0H9ol!o~b%^zMM$lu1VCa~QZY&F5}aVHrdE^a>Dz7yFCuhELcp)?-9-abq^ zRvbz3Ub`52E1kGYX9ByJ==)lTiE($4`X){A5}gS}lUDnKrjfhUiDKEBR-NfyTK&-+ zFm)q9oKPP=vGptSOq8>cjqT#8ohqYy_U(tUD8H%IYQ}C_rzRljY3t`@0Gt3X7u-%S zff!*@QoJ-KJ6QRNc_f9<$26(#&KF4yEZ8k&1`NT2tL#q+1iUH9Qnno%E#(jU$i$IF zCc>tdP8xya(R|-cUHOD2)V!e~e$9QI1KmuJCs#}Qzm4=ibkMZyb3^yj6`B$^+f ze|uy7(p){Q)Z%6ke;a2!8f6@uj*)s(Y!j0iaXT?4`X{f-xHG7VIFW$2wQh5 zQtD(3JC|fW=Q~%3A?7gPVyIYyux!mVbHL}E0!&uAvF@efJS9EkjmK{}Q7m!|sWf#@ zBM&hrFQ^RSpXX3SSz(7_T1LWL*OhKX6ymKp@6cMzn^o3Pv@C1$j zGha&vF7_CylwKuNhl&eE$pS-2ffwP$?Oj}l31rT=Zi22tTM-wn5bPlPHuHTrty=4r zq*y+xp@5BK+K4u8Nl%jNF70c zmQbherMjv|Q9ErMWqq!;^EgOky*K7yE<;L>DuTy$7n1tmn*|9-a&dstR3*YjX0x9B>t^Q=Tj z`JD)BWW9|P`_ERz=RQw9tp~5!uEO7)m>P+#l)4OZc5vne=j)-j)Qd^<(6?PyLC;JP zVXSJliAPD=zS?K~-!qBalTp{r8d|WWBxTVP*EyUfOf_chgIT?46)#%U%@%Z685W>e zqY!p+&UNcG+_x5sGsFi2gPhJ;Of10uvUsk7gP+%%rBoXJ&8b7|KS6iEHzsqRL7y#D zPb{csg``o?FD9KOTu?IV^ufQhp|*2 zWXh*eR3#i|LryN|fVAz(iRd#&@0F*E#!PhQW92hZs&i(s?n{Z&#$)yQqRqBlG738^ zz3z!I_e!yK<+o_gEpS{cKCj-A@Peu&Zb1JT@1;+%!QewcKF}#sAW#HGZ{$q0r*;ua zwXgw|bp=qA<<`u~rzEei?JK4CtiA8dmY$yhqJoImrIbNa2U1PvQfiwP!Ai@8n}Y%K z!yu2L+&p}_f9+5DSshcKfN%azli*Tap` zWQb(OGh!x>&T68(v3YV=S(bQoR&W+I*ESSZMg_o{n`kUk*ttW!6OpV62KVt=59Fer zNlv330<~zFmjA3U2k4ct|EY_H(~B0HgwPXaTT-w^BD0}7ETcFwc-~a%S_gz)$@obL zUFgcXElizJ9#-_BA#>DRHYs|a_x-eJtyEnli76|=2tk0bwJNWR!%kwFo@4)U+U%Jb zJJx++ZpGBO;Dfn9JT*z(&?X}x5{|WFVInHBPz{U#frEC|>&jOv`fA2n+Y_e!Hm_Q6 z7)I-SzT7m)M05L0!4({J`~W10CT$M(7gYhyyC2rEQLo80bfLAj{nbQgDFCGKz!>OP zgl$q^P}bj%dPm*9->2>bMJ^cq;T8KrCW~gHX@T;AE~~-SP)=;-1oIc@ty2tuD6-6M zle&AlYCn(QKpyAz%y|II#ZbjNW&fhje+Hc~ElE22DOE|?9Ug1iM`jkWZ!BLYpaV!d z40VHio#dl{b5dtM)FqZJoEJ5AsXSbqBP2KEVL~1Jj7uz5#&Jrbt+^sS?zwb7ww%{@ zrxNeUoQCd$3^|-s2t$)Aa+smv4%Bf7E>A2r{y@#Ui6+p^HSJ0FmelBbK`%|+P{NAz zJiTN~Pby}mY!Z3J0;`0@AunS}W*nBAevMq2WzcLOl>61C*G;IW^DJa#6bP-<2=V78f*w0M?kL_5bXnvni8i;S>oI-YN~ z|5IvG<)kR7p@SAe&}?Ga=3;!Wn*I#Cs)zb;}%+sBx!pyDX)a zchw9;8RE#)n+L=dF@CKqgpwDyUI%s4brCkPRj3rjgAk~UkMa<3j$ey))Oqg?Ra6zh zY5JL4hb2yBNGG@;P}ewTmdcrWLe+1CaR5HA7KeE?L0P*zvd#<8v*!ZpTyPhJ9d$o_ z8qi#I=^dI>>zHmMpe8ND-4_T!x9>`Wr|Ui0JdGNOKRxvg!@@}`L}6=R__)iX9v7pjhGVE7+mhR2)2}bRD2X9@gM&-_pQI%;;Qf^(ojh44Wz5Jai0jmi(x#vYi+6v}RXP6c`}pQE66&^L(*e-5Rmh7= zF%8hiHnUlf;emw8^GWCsCEzPwiHW_}a*=BJ0`~a0f-Nm?Kfv}~>h!p*CHL9&|5;g6 z+inL=g$w9jNDHeEj`aI>DiSg!)o1;wMed#Q=kA}I%y==ooPC=K4*i{SW@b%F-;?RH zoNnPJ-A?DA8{-rTj@k3viJO&FLr=WOzO7l2TB8$UE?ypP-m+~HFsedpV&}{j%Ey{X zbB<2P27DWtt5T>c?LRI)z=vSIrkHzYXL)x?Oht*DazJ=BWG=Te>LQ!AIs=p##=;`$ zRMh-pH(F&`MN^=&&;n~KEnX_IA5qv*_V2)t6>D&KJ=R>9P%qfznn}6<&B8ocMlf^% z4#PJl?h=o(SLhkdqr+mu2sFh^<>cBn{%Q$Cte}I)vP00f+LnOTLg$r`C>|lnwLrAP zU6R>7PLl-4mOb<|W7DGaFnn1%D_LI=F{5Aw%!kxwNKi`w@U?tSDQ&WpU0+aIj4}?K zaTQH=r!xK0V`5NnoXcslgIG;Tz>=84o_|%!1haFDH9GINtP8W^Z$Cdv)SA6s;=l0FBqO!(iex!5!a}2S@S<&O(ZM!Xh{d9X-iS4|&I*46C1;MP z(J;U~0_FGxmt|QkGI+~Y;a68J=91w2bkMRYsQFmb2DAy2Brzhqo{TmqN)#8z*vu|C zon%;(qR4rUeb`@JiMwEArVOMhJ6TLAV$y3#x~evh@1;76bN_d-1~QV?82L}Jj^Xw1 z@v{9xWP5?5tu0HSGN(1Q?o(f&n%HD7UVh+?WN>vFONp4gpn#Iz%+`OpjF7pzn?CLb zXXAFk0t=|7Ied&?U)GNmdkmh@ZC6%5yp=eX`+1`CZlajeC-r5=zR$f)7VC9I9~}DY zzllVv&T7%S5Gz=LPcp>^>P&MwT(k@1;5&-Wo(a=f+ZTnoacqkz7k5IF#^xgEuD!EY zRfU)~s>PX~tGC4gIq06H($e)cM2v-zBMBm<5hJjlxMH2@ zPQvp^yqXJyPD8-TRVAV2)*rO0lB8si;3tAu{9(YF4$vF+xPX>BGYHmlz6~BA;VaJa ziJa@y62DsInmy>GkS?1#m}<9+D+^FCPg0Fr*dwNE(dTm_2aA5dFA60_7-~$Fv2x?$ zp!B82Lxip?$3BKA%0`)*jbATrZxuB&4n_XI3 z9w{yHkYc;{30qvd-iBr1M+hl52RmmR6Rp=Ro}kvIu&x&+j|cYd)U8Iks+(QvOII3> zDtU(GmAX=#Bq*An>p8n6jwiPtK3{ztOgR6L6so>WLC%^^ynD0>ybiT!Jg8cBB*ij! zfCS^>k_o=1tjZUgPyz9hRLNz>NCL3cxMyH?E}9N?hV*lohmOvVy`5|gR>esIlB&CM zdt<54PM)!T+y+9|sfm&bVdXMUz*uBz2J|XK40}0v7`&pUsI{ObWpRkmG=V0kCEP*< zF0RXz72p!WQz#M`3vbpq%h>e&*-uT<;hYDW9MOk;pZfWf=Kso*uSWGm^B`Lw_rZH0 z_ZCE2O2vmJy56|bg1?wzp81WvVhIb0P^hb~PDt~mc9%UisSm>|p_$av<5>PWhIUGZ z9SaYK)ocYn#ML!xn3T<4D$i`HzYcJIr`4H@)E*?^5xA}rp9l4T19zBluKAFpNi1uj zdzgee1)a(L$y)KYf!Et|LKYuiNRva-`lA0dTgi$(OhnvC=6|UBpv5^OCtR#2BCZya zaz51>b0IR&hm?-ElBoGYhIf345B4Pl2f~76?oV5c2jNaD~|BikFC5-c^e_Z=m)G{sw)jfL-V z6ua`+3q3OW$1Ox#9bIr@Z6s111%o(NBWX$j7%wq(W_)EG0sRk1N~FCJ3~4=>w{&St zbpufqD_^Hg5R0j)_K7VIr*5D-6qtO-B~X%-&13LQ30F!@t==ufT!_0f=>TGs4or`7 zIp)@N$Rif~5nDGd%4%E}uTzp{!mW#>ep0QeRC>XVABrk zc}j=KSn_w1n;A|o=stt7BF;ufVi9>tQ@Hecv)N8KIa&6^ zaTNbHh^KTvX$b4_iuW~=rAdf1RV!^BMdwc``PzAm>cBP}{Jre%^;?*7Ax6%{u-GH` z57@pG&Y)n?hs&GE_nXf*-~Q)@JutelGRERu0IS(nt> zBuw)Z2rzC!N@OyrTa%c)Vrlq)DmaYCgOQkLeU&x`e*G*fNm?ZOc@44(ki31ci634TOxNwbWwc&Bh>V@ z^dn=D>@#%7JGEso0w!yndpn1rI5!Yax)@T6YN_LY{w8T7AKJBG`N7!;J;)Idc_DnoH(wSUD~A&FdpI*?UU4Dim;xbA07x^_F+Y6BI3;-5+` z7UNA*AX>7WXkZxzV%ExNzHGMw(>i8x%*ylVCxx~a1C&qBS zk9X_NWpm(N9%&~U7|sxo>AUHV6b>C=%ry63-T|E)-|8}pu#@1Z8ZC5HG5!_hkT(a? z2RSiL03wZZZW_Q5+gI~4Np~=zrU4qAIpDYk3kOBGuvJP15|=N@nD~N=STafC?aQj# zMj;wAUt^>nPz5Y=vW8y@{-jgtgyyiS#;asodV(*q3K0-ypMXWB4$BNk?ybiV!E-SN z-~=76Po@ULCzbd-lG0^ski)i7sc>`z<}wT0Ug{negHgfnO70j6wt3d2j*{38prP(5 z)xt7-)sgs2#hISQ4A!KRlX@lo5QyA;oo@Z@VmWCgHt`V@)l93V)|ms71kY6*T&FxA zZJLkZFbf6d1PQASxmg{dNy0OcPEL<*;;m5_>;=F~ozC!0Bx=Mw4G~6x8;E7s%ZCki zVfr-sqWiyg`-dMJZ=0HrAWZwmWVkHMBE#i$Qx+GPGFs9(5?RE8$2q|l>!=KIWlQq8 zPAEmCN^s74nklX!Zw+gmO(n%J`X|+iaTw7z?WOH8pe*onUrQv~gEv>{BQt7lMdZ?$ zVh?s+R?may7^;{ukXH6~iIV~5lmaP}MH?)y7`+P8q-1G{FB9}JB+7V^6v-HrH0| z9k)PJfxkPf|0aX}HYxSi9|As5PuKfS#h969T1C6f*5c~wtz7Q$Wvcte0)-xI&qM_& zEc@k+gPEa6~z(`^~u(w{5x~kb3BD$9nhbQ@yZIJF|!LC{4t?9 zr8Y3|MGA`VWQAk8L4=hax@yJxv`*%!Ghkp?(5KORmH4f~KKnQYmZ6wN7PgrXb>+}^ zQqj18I8hnplTouGEDp~x?V02L`Y+uiBqYq<_f2? zm2CfOhd-W)v;k5f(+wGKdr>+VrJ5clm2X?!j@u%s_n2o^$RGL zK>Yps{+iU4B*2*`39x21z@ds9qe-iP!mLvh5Z;{^17O>RQo+xDxsS&Gf zZTSqov=?@7B|(Am<%_1G%LM+jomGQkV)7;}RTEvE)|J>WH;<3(iwJt?tQfZ9=cZ|{ zz}9S+)459(H<@eo5`H#}a&Vde38&WyOV>bWx3C00Q87{-pEX3M==lIT7a3X5$9pC> zBcNBh3}c{u-mEI5Ht~!P`=Tcji|V>4+~&N)emWdP-66K<_X}Zu;mLZ~Z=7;6r9(B8 zigv;wd=eQvCuEHI9oV<7=r+)}o_HUdxekqHj+vI4rlLQ}UKFd-TcLDAKePe90{Sb) z_%P&6>J73y@Z^;t%}oUJ^ss*hC|m(64r@3SIcI2!8(YO9orKJVt!`(XUEh4fyj;## zZsWt-n8=IQpk$4vRz>vkxXJ{j>E?0ph`W@z|u{ z2o8OK8V+ksXsX)C>j0N^dEY)(nPC06%v!5_tV@!T)$4(GrhZu(t4kE}nO8zGYeny@ zaB=itW6yyDYI1w`?cv+yH=~~)@6&N6b8f*6!FJq%xFn^8hawgE0fHh^tFxM`OBUB< zwR9l9j95NlPoK?=sZE4!}(y#BJ`#9HLEYEv-wij)rrnd^2k>S-obV1z}~v zw&UfYimWMlj0}XChCa-h(@7~=?@vYn!5UOi|B2e6nED<^>lqoAg z@pr7jGebz2N*jv27>#Epjd*&oah>o~N7J@dgajCZt51BnuDH~6kB~1edZfZeVK6Nh zAdA?wONES4M{jM2NtBXQYUGb6ehhn*DE#*cpht}8FAxAt)V-&q<@?P3q z>pgW?O#2CN+njI?k%Hh=dw+=z24RUN3d29HFHFPmxQK`+@9cDEB5;NbA17?21l38DIm&nV#m zWaO-*#&L2xbQ#5Jhe(t;5u20(E)u}mt{yX(f+Zp}RkR+dlbIJv!k0QP(NCPQ>eJbi z^rwDe9#8&va9*n5ve+;w@9ydL^?s6O?uaFGA7~Pt(H-NQOhrb7ubbi^zJyU<8~{ z2&18PxyFLUFHO@(Ty;s^KLDB##bn}|;5!w!GS7<5&gEq2CX`T4bXM^x<57yR=cG2n zYH_Sv=d@Y*?USUNv0e&E88ijXSmZrD84w!iPs*=lapiHl+74yWBF|~8V+z;NKnH0>tS8C859C0=cU$w zG|zSH(XOl?Jf7TVN~Ds-SrTtz%{vM+^o90eUTvqS{%kFA6Loy0E=TN#CGcZj89 z9XVx)+%!8phu@)z%gyG6lwn4bM^Dj?41x7#w-R?zm$ih?5@YDQm_vy%2NOnDYTx_Y zHQng2Hfvx(CJ+~!jzS&~<;Aa1(;JxCNp?2}j9%K{L?4fG{8OISmHpPTRK#(O;iG*B zEOP44$zU5UO?A=@fvQ?`nZ^%p2b;*2iLwhA3dObJ&qe*(u1{Sz zQjwm)u|Z>F-hAvO9QuQe-mZO9HeWMjZm2W>#FHvP2vWh#Wb=ns!=F%k)OYLCp(b~Jg`8{~)|W*VF*E`ej7>s;X#@d?sif6!da zfkfNgBP)zyU6{p7>IFiFI}UHGYc2_9>U~PAwvs5!q*e2Xm4pIuapuKPU!srIss!N3 zGdLKoe;t?Tlw?|JaEX<|kmaBfvJ|LD*@a>XCWmN>@Ne`&PKiv!q0OF??J$+EjnnmBMsafmkQq4`>RnGrS=XtfjCTiL$> zkb5yAN2rIA4tQr&C5FM$5#SriZpgTrcnf4>!{D+Z74FfM@JVW_)FN_1>CHgd`Ci5s zsu%@2AKq8eK;ZCi`MlFGv?r1M^!e(eluVn)M63JnA0F<;W6i4$Fq$K03w{tD#KXzR z3vpLgBXf_sQ;w~Jv?rG3ihmcm&UnZy9Q{-kN}{$R4Iw5Nhm|69-5z3QC60CUtZKG+PzvHPv+^_>d-p@_ z>XZ*XR3 z7~DuwV0{!+Py3YM+)atfi>O2u$VQ<~V!a5%PHQz_o+q|gd-Pc?QEy*#1*_G=8v*ka zBs7=7OqDQlk^~SYVSwlaFclsqBMyk`GNP_sRtGmKP2|)4HkNYXu(y>qV==8sx4e%O z(v&)$!)h(E>E|%R54HfvdW{y|}u%gAQ_Y znL-*}>4+REYqSrZv4xt=(krBa-Ul)d%Qp0~iopSFgtx%~KFj-dbw8MZrE-y#Jc*Um zoE6w%>ar)>5g~HvQ{!!js)!NuY3O$M6gc#_B2iAO5GAD2AO2VI_b@d=G0ANCTmw?7 z7_&AAz~SkEEC}0rM;QC`*J}TXJ`0A^Dilvjiu|_d+*=$KA)KE?EqR%@St?`omuU8S z*F(C$w2?W*I2~3(^McGj_Rqde<}?$HQ8OBueHR)QMk`K3ZODA!;@9n@z7$>p+bwZQ z32u+8E;8I1!_}_Nl7+V>(y@@g?CkJsdZPTrS!rzFjS{8;%?p=okm6=9ML~A-j1Tf{>YV(8wkxPxhM;a} z(a+-N*=ovl&7eFLr|fYW#Ns{YX$o3|K#fKc>xZbGaoPm9rx9}@_>k(L7L-J3jZcX; zQVI`MWL9(4_@KdQ&5C$H6Yh`6l?LdO@amD#osT!d>r##M)X^3p5OAoxNnlAr!!%W%jhE7Z&gdb@$z>M` zCn@6ww4hZzvEJvaW>w9fiqC&Sh{FCuZ#MiC%SwEli-pp zoVnx7ldwLJ26L$<6zy7dV+L>ZYBoJ<&ofO~(d?uYW}yj{&c7FUxnlGL8;{gM2mf+_ zeCM@1;7IdLadquODzMGMOrY?XB{d|2Voc_F>LksgU}lV{lS{+0MyJKgScH z=3>cYbEka3c?pS_srR8!&978+1%BLhk`pexgu8lGb6g~>!dOEnkfC*GWKsh)7KF4+ zujqhIiA`hdR!Wq^6>F|5?s?{&r+s0%%`m(cH%s|Kd=WU8?c$9xs~gs?cP|J2kAO;D zc)~P+4V!YbBuXw&lEDt+nT@_A8Dutqt7sea{sDt&U4 zkUpBTWF1yjrUVv|$%>b#Gk#nH*#3Ju-^^+on`a|Yi1g~yy-UUVotJ`3PfqK!sAPGP zG_s)Im)w(qYfB35G3HcG`paV-xAECA}8%o$>k|=(x-5g|AGHHNd6PBj- zjI_jS_SgDYG}I~oltk-jQti~}acq+;Q);#|HZ}6fLZsS5a4SoFI&E3KQ!O>-lt)r( zt)Q6mQWpWvAsQN%oltqjT~AlP|F^fZ!kZCTA&;uG!8t4 zg#IW283aY4K%OU<%^UGgN9R%yG>$mL)fToo%{=nPOH!?$$RtZ@C(i;)XtKTR4ptp> zT|IHMImV+~sh%x3o2iqbvj*~Ug%WjnK{!N013of#Y;Q7o_MxBPjC|WQgkULx6A=6K{ z10vB+7@(9&G5XB@%o0=tK2F=pK>TOqcWfkx4k??EZm*bq^@6dBl9%Lh0I>>;6`LYO zq&`WmiBMXB3y|ozzxaJD8=uoVLdv>?YiZL&Mt6dHwg?z=lVEq89M;YB(JWWYQlqX3 zNr2}ArnHvOr-mz|+pQqLnI|eD?#a-OfMSL`v%`ap?qJz_Aby1MGF7lvK?M*8*)6#@ zDMqPdrhbG+nI|+W?~^m%9Wh0S13u4Jc??Vvx$v;Gpmkd@D;Te z`-t+j;^T(OC{q9phi!;fmS*&Reo>5dGWc(mp~N6qZ|XajA6~l#skedA(1gftH&M8mf?&S{&b4)Ht(@Ioja^gx4*|yXsnq^vp}^*cC>xn0)p+R(+&W1eGhsB=fJz=4wgO-ssqb-vNCN3e z%pA-;)8&5YV2Mwym=A8<&6&c)(@DpWzN5mqc4d{c)SWL;4h<@XaJ%YuzL&#$6gArj z6N`)$DzE|_jZ?QGtqUpF0qhMin%KWhM9N#3+6gCDn=;e>c)Ymc)gJ;TqZ8X{y)z{) z3X)9YbBLL*S0e``2spELkx!?Qlwt(T9yci4jRv60#4M{}acQ9nrPS%Y;ZO zyqA-!`*1U>DwaS~DF~?Ya$iyf5}&oNrQ}6QD0#zhTx3Csmll3KxQ=GRIl7_&Uy;=K zQ7?>4S75|0k0vM%a2Cy{O~o`Yv<6t=f2tXIV&-Y6K;r%!R68zs1~?9l5?j+(4LnJnz>@cfUBKAcDExhm3E*#(5`~1{ zBCyowrOzQyjC4_Fq|Wj~|D>DTEh8d!%cj~-n|k3Upip1em|Uy$Kxz=l@zk(=+XNMSeH>eX1(0m!@MhqVi3KA_oP{7 z*eWaV$%z(l)?W(^HHO*a9YasusH-VWJuqH7x%puF6 z0($U`Bu}+VH|Cg)7%|ug#c!a0&{&`ul&w68Si}~-Q(YY_^0}o3)zyp{_^=(@>u-G@FA6t$9Ai$_Ju$G`6!d(F&B zHV_~HMN50keSCu$Nproe_O3ta{7}qRqwMwClKW3c#50HkanJ~Ls^iBoYV+I z^w5woiY%TATnM(8{X((ln59+>E{0%YjP`k-o+51#)<_iDgtSG>l@n+Ip$hp-)n*<2 zCs9N~NfLo$=8JPM^mkvym23{|iNSAubSJ=t3KL|~h<1j2q|#o3e2gJ+!HgeoaULgZx}rn$1ybI-(VF5iCM@7emDQo)QL zUYUVbGL6~GQt;pg3!s~O@(eQj3*$-*{kcD&%f^eDU1Toz(>IlSQ)1* zDuGaF2p-eiT*&-Q3=8IEEKCPl zgl&E5CXW$@#1JvpG1%ww9Kln`4WkTLR`PF*1$k&Q0kW*!p5k+5EOIgK*Oql^Ox@NuOp!JYuWr$>% zkBEn2X%I9Hd_FP*hsRi*A1bf2T7kbsbU05lSD5a=*8i$~5Kf@zv1g9Kci^mP2$TdC z2wk3t4|E@((n$dRxXEz+Q(y$l5|F6({qWd}FiD)yh{Fw|j!x47l&2t?6k>mPRv8Zm zLun+=8YIyZdRgt$?GCid*vF|yY(Ryx5@v4w_5TcPr;iRrFRW-}_HL78WGJTBwL?RT^I}|Z) z_SnUi335Oou9Ml)Nr*<|!Xcobv zB-O^q29(eKn+m7XPJOKo1segI!dcK?-)pEY}fi^xpVV4F4G4eBXN044w=J#m*XatGFc*UkKWyW zm|W@HzL!o7aS`kvkjI6WzjuY*JtBbLp}ZiS-;r-MyLEfw4@ppaiAM&l`Q4*F)vl{G zD*%19y}EBW^@XMComvN;M2Qch0B|!^d4TD2W&`EL+dbN=217`#dt0|LV`J8DH?ZwW zJPn&y+&?*|wz%G-5pK?#+&qw2&-#qP7^OTjm>Y`ru!-uWt;0cvKu>!0-9_iRNef>T z+ve6mE?~>)OXSCZBZ~YirK1uuKZ^W@g)o{z=E*DuyVuo4y3X8<6=gg^P<@Vg->5!WKOnqZeE^&d-J{ zHnY?3Zh%tN0pD+0eS=lC?I_4u1Nl}L8>FM(-kA-~gIx(2wR^J=$@=I6q6S7dm@y}9 z1Vfd7h^%F9ajS`oEi}D_Ckz9dHFIe5glyfuPIj@C{0jcXM1?wS3IwY=A!FPM=%M-e zv6|q*b3y$9bY#CKnt74COY7|JCu|Q_E3mRo^bhIt!e7TAr{Vj<9vf2)Puo3L)e19t zNMLlOhpknWq(b56_4Dr78-*5jP9YGoY}Mz3CM{svZqQ7*VOXoXot|jWfm%$#N-eVIVOSj}EreE+tHEe^jdtcB z+G*39J&v-!Q83fsaL~ReDAb~HGZw>}=j4_(rmCuWs?mt0Qc(37gGz(bv0fVzBqC79 zJ{+uT2LH~j=mj|%ih#|cDFeK!gjiXUr+VrH5ouXHqqwSk_CHWd2MA|e!$7@)5dhwY zCjd}O0|XQR2mlBGhpvlG000000000000000F#rGnO+{2eElEyBFK%IUX?A5VZe?_I zd0%v6Zeet3c4c38X>Me1cXMBOI5uB4G%jRpZ0x-WTvXNjKRyGfqoQ}nB}~nsrh-y~ zw8GMwNsv1@qvVDaD!G8>0;#!_3J9dPX^M8+ZT;A8pUP}!5Zp~vR1yWmZAL^;5paY5 z^FHU!27_YD_y7HTe;;1F&pr3tvpnZH&w0*s&OPUj>ESGeuR@{lBQ!K96l)dyPp7!@ zpGK)rbnNwGN5!{ow*{?L8n*?FnmTQcCV6&J;_OFfY9>88YgW=+&BQ61*-y;UOq->N z7(P-nGimaaz8yNW)5>{lo!F&nx$jdSI56%dszbU7Eop2q6zs9a9@5j=0^u^U5q_gKwelU!M&td7W8~4=v61!%6 zz^@D0{fKFkrXnv#9b+ODipfu_6q{cBD$WsBrMN-U-lwBN@p+&^F{V8W^IzXpp-?k# zsuC{zJ_S_BdYYNaVOQz31MQ@AXoAOV?5F_fo{LK&_$_1?AtMFm_eC zHHuad+*hFp-F3Yp*-M*+7hJChMt=1CW%mc!fr=Hr&9`;mxl?ims~zJ>u!;zb}ubuw6@quI{B!lo7L5_G%n2yKgd&$m&RS zRRq#?=?zWuOH7zHc@AZPfT|lA?-g&{*fieU8FS$GVIf-Xd8acu7Bb5l8X73@z>Wl=GMlhT2)KqWJI}03Na!o{%D8fV#oqEOxk7$QYpI{0HmE5J zA+}IZ>ouaqsMLtHg<^WPno$Yo4?*p(q!0~m81WQ0LRlqb{=v(UqoD?Pe4y~mI-yrf z@-IfhV)y)uS!EjKFGoX-@VKOz{9{X-uL7KdP3r-tR4#0LrtP|9JAVqDzj+pGm?`C+h z{}h~amt!a#mVcdGm9Lv;t(=`VCc{BC2kM&2zgzu^-RoEEUccfi)z2Gmi$PHFxyv;W z9F~8h`dubFZ;b{A*_3#xAITH19Q>C<+Zb^Uu-MgmzQV%m9dNLX4; z?inHJTEYZ1If)u_l0wOuuOlZV=1hc;6crD5c(Q2u4XMPmS~2~+Mp(Ll@rS6)c!&8CH59HB zmab-ZL~F)+=9{mh@OWWqA-hY75i(B71V+V23bC7Pw_jugff}!**vou5yFA7>2)k%8 zxYd*hMd@a7v&_EWZuT#dJ5#&) zyP;}5)sIGI@(ndgik5jKUMVaopWLAV z^g=*4Gys=VaQsiR_;2JdF(Q<65Io*;FX!9N1DQ+`9m$nP8x%hV^_J!r#t7sG|s*B<4PgpaaZ%Pc8g%`Mt-^N?4I`GWgE~*i-+Oo z)`IpqP?WtCdULwW6JKYcCd(rlJ+tK~8Joekry#c#fj|D4Muz^ng8h_!UMXabu8~4W`T~9(x}2PmE690d6*-gEk~4iHIdclhdCCSE!*HjJH0%7SzV=IY0{aMMgyR%8N~>dT40GcR(oW_{Hdu5@gC)jC*eGlcvxQOAdu}N7rBGjb9;a2~ zzRXwHXq0xEEHQo*I9aO+v+0G+cw+O_BT32(ZK%m2wu+EOX^nbe6UA&T_k=JzMU9Y* z+Bk|yaoQlx42om4bQ!Va21TTKgdZZMl_?ZDKSW#vMibR@L2=R|Z45PuN?7_^E3oT$ zYkk^ztbYj^ozYh<5*t3#kNem&`68V`-5Ug(-tF^Z~e&sQ}x!2W2%j1V>4SPk3>p>Pv*aFb%Ff5o$cF)^sj5rR8w z;n^YtqshouAOur8!FNCirq+S)s1OXElCKgS2j*2uYdFJDPRbaRlQJIVq$Hr6lvz~H zym&BONMB!1W_Ef`Yayea8bWV*AV1n-3|L)TXtnv-Z}fxOaPe0Ky%(X4XAF*%FwaRc zYLgk4sp$o^ekEfuHGCFhFWTLoq(j-<#MH2w(z)~yB}~Y2O-_-#2POAuZkDRgmXmWu z=46io#ih|uWR3I<;jrUbd}2xhR{YOCR}a_La!4t60f#|n5!xw-rkxO&?|yZ z4<@iqme0B*>t%(kU$Smi3h#W@&us9f#K2nxr>^BBl2#ucFV8_}m<>Nb`nqx>(+5@T zzKBtPp4CIk0uD2`0wI%`5=fds8USH6%xyr>T&4<0tLoSai!}lLG=}(*MT2Yjfz7aM zX;9h43|sQ-XO=B_jxf`fJSUlLOP*S8+$PmEol~bDR|)CW7nsIS2^m{4(pe0t7%WMd z`w2@iKM^u^DwxsfP!^bF5U<_VZK1!_@^aTftmBZ;ajlDK5ttlBU|!_LKHFUtT!kiq z+{&GdWgxh+=!}qp+EpGV5362f1^HP0Do>LS6UCHym`t@6mR`%JP!nRvPPu@p3K@Pp zC=pZpTt0n2)pS0>$VXKhHa2Q$vKNHR11?*E%p6pRRRecy-QCVA6oL62l}s~?u5>Aq z+k=tL39euS+mjWI*AKjw&x$VdD4Las^5tTv+CIWk4L364_+jqs7}6FjF7J~DuUpR&PcW0oL{KxY0i!w-%QP7IuQILQX{^g4s( zMo)7vqDA(;!(>zcb(2l+dYu))P0N8J!S>4-rj1??bP_> zIZsaV<_XXj)Tfe8kyF13zP%(jvyb*7Z$Se&MO9pdP`|79j9B6=VxgCa$Gk->^%Akt zTSS{PVl5sYbUoh4RVFnW5(GA6vvnY@1DKlDGf5@po^dFL!>og_MIqLuSXW_7LnwL!H(TQIs7CIn59`fjcr!&rWQ8K^ARlAMXU9`!O=G;FfKv#7h2|Q2C=|Lsh1tEX7blU> z)F~v%6d`&01ba^}8mDRhZ+46!aZzTs?1jy0A#*iEkJ?A!)|xDP)=<1jSxIP8(h;K- zjmty}Q&-(57jDSrImwJ+!cn=Gli?;I?p!IVqV$J2X)eLyMC$WBXsRsnq{$&`E}A0P zT!rTY`Y7uY&J00csWU^?aK~I37J6pr&M~=?XSwM>W24oRtjYvocBwm6dZ8%WgDSk9 zid48alSz$~wdnYxvY#V_FsKSLvHCHZ-t)6%mpzN_Mj7^->Pg4K#ko3mNB?OtDlJ@@2`-6#TYS z$f%ONHoh# z)3t()-sGuaW1-u)ZJ8HiAEs=&o_;NsBiJa{!u1xpy%%p{D#F_ ze!PxJp_H%Aks2JcZKQk||B>+?t_fvpqN~H0z{0tMpHz;{?#m|43PbuqLCC<)8l$vX zSkp#WGpvdVP%a-*wY(!mRI;@tW?UFUekb|CQn-3d)m1W_M6^m?CjCz_E4*41|3UUd zShiT3BcPRE$rs5G7ejKcU}C6m`9E%Aq&IJ3biSEPj9$%*zOLIV+8@Q9_D8X&{ZZV! z{ZZ`Q{wQu_e-wMzAH|LBkK)GmhvS8|r~TpF*zjlUkIU|-aM>SM*ig}|{ZZV={wVge zKRn8Gvp24e!bPdX)BY&o_D2a9ppw7C z{*bK>7$G6*DS^LEMt5tXV1lRq{Nbz&NpKWTNOx@eaK7r?e zyOFZqxxRHd`$SRL$UcFm*qf0u&s%=Y+9zuZo7yMxAaMmFrDGHQ!+HM(w^tnI2o^bZ zwVb`+tgt3ZEv)ItOr6%thpC%2b-G?<)3LWg|LKNKlOf>h-cS2!M~>Ey(vQ}U5iK8T ztVk$4+u#EZ-iz+r?oxbiW@`?@>GGpXPr%HO_q42$uiZ~WJzvCc2{f4Z41!Tibnf8F#epk&GU0H#y-(~{^SsLTPI(h?FGl? zrF&WZ*|S<%bzXGexYV6Jw-w3HE;S;D4P$t36{(=sP8rZlCu`};WAcvToqz-|IF5G( zlEUE0d_j>!F_=V=tu2z5%cBsIAr+CdMsT4F-rl#tdzu+DxF6UL-hJ%lphK@#9>VQ9hjOhvf!ka@ z`qAxK+~)CdkeMfRyPlh#5wJzc*T6OjxC=J*iq^~yb(e0h3sB5Nb*zb<;x;L>L$TU{ zKFExsbC+M0LisvJ=dpKc${dV;UW*t0q5aBCd!QThcQj0cz#8IjXxv}RxV?z++QhD# z=GwP%@6|P`vpxB~95!8WH3Mq_FLmQ>;e|Vl`S$CtI05pO>0j7(&-Js+(w$8i+mxA* zR5<52r-*N^jY+(AdWWR(I<-@{@6TZIJP4L4JmjFzNfMR6w@LZ`x5mE#mmmMOyN!R? zc;p%bPha&I_$lHudZXKT ziYROgv=F<_p4dpf8;!5-aX}JYbM5_{<=eKnjki~-!2jJb_{#d(%lPZk(k=z<8gN}> zEgNR9YJ_dN#Ns}}7JH7c#hxSVW!7ChN7w)_oiXc5>pOqzc^QxKwZvn5En&kz2^$7V z*!Wt)#@CX+I=(jBXjd9bo7BTGmNwx8A9t?tX25t|Kl=;E#N5ZvlATQ%ce(M?ZRjla z96F2Tp;PcW{ul4?AlP&KEOsA1Oa9#WnYmNQxTBqO$(u&I$jnVbhE>70ZPK)yl(Je_ zif!Md(AD-f3fKN`buSEE@`PQ8E#!0dT?x&-ouJ&c%xH({g$t zK<&_U_LthCDm%nw^=Cdz$oc`>cg2t$VcBOsO~|71t3abvPZ^70JMB~1R$00JpgePo zW27VPABcUNUKSYiZd*{B9DSvPDH_*4RJIqtK}GWi$IFM+3jXpz2$>5VyolKxOy`p2 zp<|Xu46{7qndJeS<8z12@h#J*%=lQznrvI7lkG9W)2h~N$EChQ@ zt5nj7GxAQUq_bz_jZ#VVMoWhQqp&g5Kz70TWZ`Kg4So8wgV-NR+>gn_Cf^MhBb|moAWBS$2 zYNiBsf=f)_tq{|WfGsRwlc>DJR+%rW{t`TnjOp{Qpi>h(&ZHSiv#u)ls;kPq?bT(j zzB>qR^13Ls`T*G2)JbN3*^*h^~NaCN&HNCu#5Yfgd+{a6VOkq7`riPE5$h*XfkSExV zrZSGJjXW7QO8VnoTOSd;v!q(#AdZD(x%D6bE&?r{>xR# zFG{HN;9cy5)3Ri4)@`uWu9FvxBZc66Z=3t%H{9|DKY8?)7wqMw{sy*akMjg<;U4Ss zIP0VG^|bfcIVyIDYIYtw_Q1Gaxs!ac@qf8!_S%i^_&Fg=K)^Ki?ngd%wZO-d9wFd9 zrr!RPa%FE1$+N-sU5?w5yH^%^?2oJwB|EPI$R|zjeB;61;VesF<4@9FYR4GixwT3s zo$NFljhRn>sNkpBsQvcwkev#Cs*OSp@eqF0A_;c?2_AC2e8P=F&JyZzdh5K^)%bh= zL<``>0Zzy`j(0H&AwVh5r$FCTtmYF9yX*xD_wzTwhitXKxJWE;kw{$z#Ig4^D;0db z3U#&p>sMF&-OW`=NGd>O*!!A^ZsT3>S=;>ITMvhcVc8+N2uZ7*iPNO(azs;2=dzkh)=Xw#Y zxt`HA*Bj-U>&3a|dXrsqz0VuZ^}h9(>#@n-N;v>mZ|^Yp{?GIiD%*UDVJ-8^%oZ{q zMr4aot;R+Q?3)cxl3~a8PEhk#$)VK#>=kO!CdUIIKu<{m#46(etFUZ0drhNO$no$g z5+zJCF+Eq+=sX{lFK)i3bl}t#_Zz&BI=%!jY^w=9orzUkL_-U z*U#45;Z@m!9A34ptHZ0YwRd=RHig5RYCG#Id%N3?JG^(Pwh*q!gCU-qAk3~-id1ucwrS3JzXZ(J8hg;i za!Af~Rmc=Za^_mDoo-s`*CfA|<;Cc8PuU^@)KW}5)%$B7_x^H5Sn{Cy$M){+ve2zv zHY;sAd^rC$``8*BUO(Hn4zJ2)cX-vdcN|`gZI{EVvn_XcQ*FQa$Sn{m5-B(Pc$u#y z^_l9+TO!%l(Js6#`13trIL>D(cBA$Mj&{JjgiMz?b*OYx5B0C)LY&qli)zj}Kki`* zvb$L8u$Fm;T1JY|5NT%T`)gtk_MD@ooh#^M(#5?! zN~eIAE1itYRyt*Ux6;YePnAxw|4r#+@5_!*jqMYqlhaF-GJDlv?*^HlDj&{IjSu+Q z#=%dp5uO;e+j$aqzFvWyOkBAUs{u9k zC5-=SqP@L59+57pZ*@NsVx2XK} zW%Rs&F3+;drcZ^~9ZeP=@V$|a9sJw3YpZJ0x|N@=kCJ_7%>n$meHo4YEo&y+5!|7p z;O~xBt=ssyLn)_qpnD_eBS&fyl4ffRv*u2jp`p7z!wkJOgGW3RGirF0-ZWA(aiM15 zqshs$lXOYRQ)bPXGox?P?8N)7^p>XC_xEFB9j_+zv2W4fq%)Xx3k>EJb!-rQTxBvp z6dq~rG|bYut8u_CVd+IB!c+0eIMO`Q&k>$}n4S6wHCWc-OJ_eosa!lGenf3C*1l2c zvNx`_yl0y!Z!QndXPRR*wof>U?(8VqJhjdst%5aRAEI>hzpE|ZU)iYO3!D;_GayLe zq|l84PAV*^ScGPa&gQ9AScK+_(aD7dX$5Q-nMRYevXEU?m9oof8wj^38m(t~p`R5k zSAlGYb|)Je80inPVBG==GdRKqxrn9U12sH2{bojdvbr(#4i39mDVi6MK}9hoUaYiw z6}zlm&2nBCY?Rjj@P?e+DE%&mZ4%96R7P`aqj{LxXdV-6w2TioS`q?{(kQLQD8*?h z$z*M)C>gck4P8!53{+q~L55PU?rr@9{doO2ecZT5CX7F`ZmX9+eZMyPeiQuk{f4Xb z{bKzi&0n!yvFQ~`J95bq)>H{=E(mMvk><`dk>(!tk!I~lVNI>D=8~}Hcx1m(HNKJk z;_7`P&3+VIW3{V=H76qbO|J2^9#=>9i#X|PEmxZ?DQc^Yo}3caoEO&Ekjw|k$eDc7 zhtdgaE(&YP5w1kI5-EN3=2(AWO@pweLT_$EF?q03Zzg)p!&MPs`4M475n%@-!g4^c zkU1P$&PlMa2BipVP>isKm2;YBN4F6*l>NYZy}9?vv}1|}vICF6)rXzehn2HhfF`Pg z-rT%%VS!%hfm&IoJjiAg;E zqI#lZ#z`Mn3Pl4_C>r!(Rh$~mZDK|by?HLkW@)0-NJ5d1@yH;DIMbOp6X?zvuC$%< zZ?Zg%aYAoikg7LNT^VVP$&NISE!CUD=~hFxI=YRww^GRSyGHV1Vjr7WlTRoj93ra- z#7xR=Gch-kn&5~k0!$Uw47`uHW<9DR`LG@%zE@GE{vD}(qF6p*=dE^nc9HT@Tgz3H z7bV}~RtC#~$m&Rr|Ml!KWf!OqvssUkd)it??gi_?K$@mUNHW)q_u^Gs2IkZHA5-a1 z{}XeAa`Zt(5z?LrsYGu*-X_B0r_fu=>FVD-B4kfQNQpkJw4VO-X$NshJKCN8(vHE^ zOF4R}h_&juh_LJkBJyYCY0VD&gVvih?dH!Sq@#N4X0=qQw{B5WF>UqM6K$wn ztrg@J(IXX=c?~^OMT8s;spKtwxx%+fyLs7q>sEC{nAN(C+|5>Npx%6!HHSpTsy8pu zQBb6rNeJDi=3VEX)Sq18HPN57OrcbG)t~mu=ufVW8|zOhhO1Df zkd;h{lKv!FajDRQOi{S$Pm^U0O_>$$YP9oRcRSPPX|&Q&J#XE`z0 zU1c`bm$iCXUoyRk)*%Ly($q6X+emqn-n7~%rxOm=^+56dD86%R(vINap+M4wuk)-sL3{HqGLZ41S=I}GM8ae~Jnoia$(h9IjUXsc1$VwB1Z zR(l&GDN+MzIHRRUTVu!;V@R1HsLl{{K@2JvgN_@*3Jqbq4AN@ zu}Rvfx8~91kilw08UvL>=?rEXG_7g_scOA7m#$wj>9l66P3Bj#ncAj=>qxa9G?~9d zTSSDdiwN6ipL_`uK;8dBeSN-HbNX7eR`?o}Cv6M<2DN?p`ngpO>zlVQLw$xeSTx7s zQ;@mgjYj|{zob*5R4+<<#oF~)lN2(Z!>1u@H}IP@+*tbTZHLPjsiarUk*Vf!D>ZkWafCo85X=#lA_f8qWq+& z&i$fPQdD5SD1RxcN4iysjR5KA+6m9TNi=MTios4mtmB~PGit^Yi^ixKMH7gyShY+Q z|DX^AA-3~hqAQTm)q~NcVRUKJ&$SUU@O^V4uwRr~iVE%*)me(_O~eM;pJ9ZG{xYHG z6mEnLBXJrN=p=SH(H$Eo6O1*e1jff6e~DfVqgTu54QBNAPCwUL$at93E9IDRYQkB7 zD|gDUQ2}aa0Yrop70PPazh6|K6cyets)rOc5NBBIBCBy|N4dtRh`d~Wx$k+oRh53>X$LBw#FX4FX*I z>!mh&iIl!XN?#%^OpF&Ym*N{$VcAA#HX}5fku+YQ(L7nJX>KBOm(NH#um>}T*GL2{98#1&ydC}4+<&k_c zaji2*+x6yURcy)WyeZ_glZx{oOIV7Rd~~A{aMPXZK_iVAmPOhaBb*2_+kgR+%gi!C`9h}HQ1E*nbx%qEq2w7>BrTE+MFsJ+X4%O%+Jw< zNhHFhNQE@cPco^b(f)i*Fv~Q+DrCII2G$u`T|$=LOtvzWZKkr#?q%;f%`?%t2BE-aI`pxSD)A=S z3Tw?}49B8msc;uHA!8Yr*XCr-NF7MS_+U$`aB>%6EfYZ@^L^CPp$xjomYJt0Mk)EX z7o>FtiH43HG)5Vv?RcXuzh*av>=5~8LQmm|9#wx4?hqD@D+;+u0&W=7(P zXql_k87$FiFvr_|RFRJa_mN3WqrxomCIpg6O+&)?V9}Bg`gWu5H>{sD=_8u;u2lQWjaRT_Jpdb;L@c#RkfXJ`DIu6|U)?2>;BR~AG7C1b6_N%~~+1ZZ( zPkjG<4sh_&r5?bbmtOi3Fy4IgW5A5N?s^zla{v8$;B0NJ50Jif>u_Ld|Ne2ngCBiV z4eU5_q(5-()F~BE_W0v30iP8X&H*Ndg^dJm?b79bK;`G>54^Hr!w6u=jWZ|Vo?OL@GfZN}A<2#^Zn>K9$?KRij1$0kMeG@1tD;orCD=)tp7J^dYEZ13K4f&SNB*B7{}d-p!T{#mpB4Sf6X!zY19 zR<67Nd^lo64KVNCdriRid-gmEEdJ)3-+>V!Ax{AZ4jsA=7=Qcij{}`DGF}5d9y#(X z@WYNBQvrqiXRWB{4d^tQc;L{*i@kt_Uw^$3So6pu6+ppbkNpZ*wrm*!B>wvAAHeJZ z1BL=GM@JU}D~!ehVD;FsHXw8R_GsXz{QT)aUS;L2!0=B$Jpxo*xX=}Ndhp;p;0=>$ zFObl$-)JBpGxKHOnaIfPfc@M#HQ?vt(;BExNl61%#>7+s3kD910B(Bs-5-IWt5zKX z-hc4HGr+hHKBxy$b-HYz)$`9k1Kj`Oi(ddiZ@>LBkQ^Q^0;kTN^#dmT{BsE~f6W>z z@YTM3PXG@D1x*4pYIP`Zq@m$D;L@2h9f99=?wkpvZQEu90{#6v12=c-bSJRx#EIL1 zUnfjB2;BMFYwrWgBO&I5se6&EiAejY!*4CvaveNW(vqM~^~QEBPDz~npb z7z50`=bm9e=XUL`2VOK7iU9xCt=j>jRyzY&8y9yNcsD0!9PrDoU9*6}pMPEnwP3+=Ao`}8l7P6reP;vVJ$h(?*pELx1(@UBY_ZvH1l~Jv zU=d)OJNGT%zFxh8fLDhOJperV%P;$Z4rcSyfbh&SnLufBau%@YzWW{k9{=vUTwrTL z!lyu%C>8>(eSKR2wKX+Lphw4!*8n+>KKcRh*qwJy0KSZkJqJwt;fH+S+NYm>0eIbN z9Shvkv*#_qE$_VZ1MtVegW*7VU0o;OwiPSB0d5Eg=m+eaHti?ilRy4Q2HHILoCNfH z`Q?8BecpU?4KVQf>u&(MEL-*>u*q)M0Ly;=-2i+#dUOL26CAt%_^71hNx=8nXP*Zi zdhfkDpsggO0@roua4j(9x8L>wdygIs12P_XU_FpOb?RDRsb0Sh2zmYW)xg7j`pgHe z7lZ)dsnx5u0INogx(Lkt>8CwF@2*|%0VXY)|J3Q}NqM2l#M|okel~CO(oKucKNMB< zMZx=Dz4hVj?C&qVbn{*J*KY0q(UDV+7lw85+i>H$z*k$n(dL@evhrL0{Y2aJ-q&@X z_3+9O_wM;74m|jewmTyd{SZtzPswd4|LDJ z_;&c&pV#aQQa7C0xy`@Ri3zVoT$}r^C4(l9>ozAoZTS8hZp~UgwSdPqkii8 z-}?Wz{{Ju6AM;jxiNlY8WnRpzGz$LvrC71RlE+=}%YXIqUw8TM!Rmj-088{!MgrGe z`{6p^o?FJh3f%kI^esTIwf(!1P_`)&suJPdEm2dGW1RfZt|5Sq{81;-P&&!LR9kfU$Ffb^(UcX(7OKn>I!O zPvl6C0oBhfTmZa2_P*)BEk8Vd0QmLb@HIfy@-MChHfcUj02cp#&$qy&lITF-$aP^$ zf!7}F7Y|GvnGy+nJ$Zg9@YD1!Gl2TEl|6uSs@q-$lEtZGf#)MHbpu{5p0NxV@l@US zz-M!I-3ZJc>Qe?B>h*jqkTCkQTY-O<2e$!k|1QTDcslQ?H-KGtt3CokgO{BKZv5?| ze*-?gkNpT#bX^(;JhQ#WZNR@4e)}SDrp@UpVA!;>_kbz;s<#2HTm8Bl*wKGtFCh2h z`e%UteMi0xynkl&N}$y<2k!trtSNs2XrF#!0HBMn{1h1f_`9uvUuLNv1nvuZ<{a?q zf!2e71NXK15-=V8WeD)XD9d-iB74%B5_zX@>6W7?)Zoetb_TZIT* zACUYxFyrCb!+=X28!iAsX80Ij(3iWf2kz=K;1-~r@W*6eYTU;`z?+YZ+5u?q>U$61 z-!AcM;HB7O0`Tq0S9b#4-+bX6AT>L0IPmCx&HKQ^^M_6YdR%i|SD@^rAKL-hD{nml z47{OqFmO%xYkLA6+jeLVoHlRx0_d}5)ek`D>p$5D#GdMw34HZL8zXS?^#eZvAEd-z z0&0~vzYE08KKnBek#KJw@W_QF{Q>jfka@tC8`tj#)>K@a3amYRY7($FZ2xB9fk~Na zpwnwN=zxs%0U97PTDut7`O}~};LCG&1OOXC-kS;ZeCPI|!2W-So&h>MJ>mo~-+Cw? zxb2&+Gk{N$e|Q}Tcv(CTMA;|b0Ia(BY%Z{}>i+A1F^hW_0yp%_ssxr9hHU`;2;X4^ zp56R@J+Pvn{awJvXVbO=A3a$d4D9>K>F}ERsly}nKcFYwBhIyAn51qDxk89 z?I`f$u{Zw!qLc3W9XS4O+X5hD^`bpM*oXiA3b^(D8^eJ0AH5t3-1&Y&HZXVC&vyeC zmu~(TD9y@W4fOk0VHe<(-->8Je_~%Pkg_eNBXI2tHMaxvOnWTA;L6n|VC$!*QNZC> z{Yrp$c6@gTFg~`tJ7DeIYb`Kj%7ZDu+vAV>0m^6Z90#0VHg^tip}oE@@J(XRhk+N1 zc8&n->M{L*nR}u-0dMSm^jYBU_xzs(9;)m798lhApalH>$t@>=?;qWCCvdRGkqY3+ zsILbCZ|y932pD~O&iz1O=ie6sb2i-l8!)nV+i_sRrB5FLN(Lo$1{OpVJq>LB>P#B& z!_?gSfN*X9-ayU=x(Ptnp6g}SOzXePl zW6c7d+LG}ou=|cR$$&T`ZZ43&HueEvMBB%9hu-y3_;0!o-bp*N_rjt>1D3vdd&9E+ z&kc^qEw*nN_0-FEe4KORpp(W|+COl9-7r<(_Y#8t(EfYgBkvn#ns?kZyRK&P(^FS; zE%-2Q>$oXvUYwCE#$2lSVc+;JLt_7Bc{?Mk@H54;KksT+dw6WCwckXx&hLHw^PBG% ze5_yn;;ZcR%=CL+2+$8t+<*MF)7yWq`u6CYS3jS7!?7>!%q%;1x2?w$C9|SVUH5p; zkUS~r`-vSt4g9tH#WD8{UB7do=8dB4`UTq_uI_bURlntJHiUhBjryTmQcp~Xe=y~f zZr??B=yT-BcVF7{R^`gldre>7dUo~3@}CB#k9_^nYj5*g^5gtHHy@nVdD7@0|HaMO zMGpP{U;g};KmX;=KgS=(d65O-cTKpfe~V9w{OJI5K#afhALA9?&tnW>?_P9)-VQn# z%?t8daM-~lt!4W(E_cGf{-e_Kyx`wTzeN}RPt$*1(Q^8YmM)nk^1+Rd_Qkxe$@x_M zjek@1|Aqr1TSBkyfXEg8TCg7^Qrbk%^tD#@iTuoa(CgCAsf4AWjJZ7*3fTSeh)NuA zTF5+tGk~>L_63NHOck~GE|2q|uGTix{$Y>~4YQ2eS_N-fnc`wYgDGT}A?=W&s-a=1 zMSGpm+(T;|P#`SBwm5rN_I}jzp%#ywkMJp&XenZ=${-b(%*5dyTG8Cv6joyjYcNWA zV(lS=lp~sz2H~sY25FaAd&Hom`ynbZeJ3@`Zj&WNX}?k7I3HzD=0%(R>haK|JYb>( zuc%dyYV>LPKG>r^yN8)OU%X6(l=~=uL;5aO zHAlUEY*eqNnK&y(51Z7_G~jn(>CM>k5>_Lc{j?@&yE}c+-Th8QOMbP{=7?$);u{*M z%%kjcHH8Xs!#Z|SJZv2rRN5YG_NSPgtHq!{#GssEl7F?>b~BY!%TlISD*3NVyaXr^l2Ax5i+`|ybE{f6@?pn&)LSN=4CrtTC`RDX9S$>Y(*Q3 zX1y%&5Li*Ufnunc04YnTwf%VHfm&)$kKd2v5=gCcu8Jq;1&~lh)Yo~o`9+(vG{mp- zD9#~-R2(i;A(sH=I0VomcU7a@>pXH-$vx1|t%QjBszybWqGym0FTE2|^fy&S?g@=+ z64t=d@8!fsO9L-LC>{NoI6+9d+?f)qa?7+{Hi9 zqQiW-uDIHkMR8Zz#Md-!;=f*7rdL|r@V>pbdI|t^-}y^WNYsB#mW3Cg0Y#>=t(89b zYzVaCZAwx|YEn#rV)qACOdCq0YE{wQ<4&oGq#8}Mc~nhsboUvj0;8q4T6MHEqc$+Q zd&DVv5LXi#ZJtpR9^HM^sbHekkLk~~OmQ%+N~u{sJV!?sP_8T+cJN&kW|bxg^C}XA zQzr;1etro;`DS`Hstvwd?~fS! zd5n|@#3?)$Toc{P_#UvF?OG#Fm|7*GIoDLZS207!e)}0dW%s?QW>TB zYNM1Ah^_2mgH_1qK}&jgm>WZowCy z*v~zXR1GeLQw90zRLWfP&$E)u5qFCIAW&$Ljh9zmMkpr|=16EduO%5tZC*yu|Li>!&hM#tl@FT5wKGcoT7%Wz3bO{aA>FQI zTN^H5MEcC&Nc6hX3fh@yA<|wHJiv|45=M#2jkaSfgoJ!# zAt#JTEM-qB-Kd_#bGNxuJ-teV&Kt^hfnpRO-WmcbP0D&xH!|er)Y;WdJ3^Ld#;2!M zV#s=Wo`sJ8CnDbAz|Z^uRM74bQ9ir<=b?(?3#r+2-%vsQJa)ZZicqBtT>*F*8*-*+EUM zqoy|0mWb5DthQE2p=mN68_nH~=1HWVr;~o3BkSjG#_o@jdY(q=`SCzrJljxADw;^2 zFRSN`LgufS+ohK%FU(7DQVW+#H5*1 zW-rv3o|rLrTJnr(lOCNrEos)DSswiJ>ucK;-q+W@hf!e8%Epw}9FecGPv+z6)#jhn zDC4C)JPK4bUXTp_hZpqzh83-IJ^#Xr*5$`v_n#in$L7_|$MA`d<>b|ctPbYawc)rf z2uT(Bb@^C+&6>)uD;Dr;{Bm-pt{^9Q6*&vmic%iT|Fjc2oV1WC#q`QlG5ut!A-y8i zkamn8BiewIA!JMY=j!jQ+$3bYfO=9A3eB}HNs3Gr(+=YhJ8n!X;f~IRJFXP&6c%Zw z2JRC&xQpX+^-Gd6VG9^7pKEbygo0BdQ{l1^Qw>c}ojXru#wn^puw`ntW*u zyu5-2cplf}>IW5M!@E$&*qWI z6V;nhN28XwGqZXp_Y;q%RPW<{%4A%18TS*vMpqw)-(Q<+HC3PH!Nj`A>hs)Br46di zVdQ7?5VDzS^}k~Cls&`SBten;z~YA{tYoC+T`EY|HK3{cR8ytT7MSDZJF0j^!N^p2 zspdxt9$5)5RsBH0q-=OYh#k{Q;d@*|{OAYYLLD*WDdI(+YJEJ4iOtR%R4|wtDuoq8 zS5`pYP1OmM-^_x7T>VYekE8kp#qdr;?F){;`zR`3a01>isyi_p&r$V)OYAvnUXYt> zy-6->LMhc|a6FYUGL@W1R_67op5@4oNY9q@BZ5oi45=1!e)L?!^TTtJ%Ebli8BM@~QC!mveFl-|182E4Rwbf?dSDp%9&ym?|M-h%I6p&2c!nG+7%+ zESrHNEfqN7}c^>|OTaS_WzVNm9pl)KQtWh^}niG@T7+VCVxtz$%? zAbq2rTY*vGi9&bsw&20&wRkYDj!ouVMD8>yOAXd? zYRV4PXAxs`pix=q)C>n$ZsCarDVNbTuHX=)b2ozD9!nI+^o=e!?#f|-Q*p557T5}T z=VoQg1Vt8HaOUwuR&LqihqAeF=u%hK%^mGQ z%V(z2ome}&RxKAnA=gK9NcP2*;PsD)qa#zx1kbauQH&{>fMA$1Z zj)+T6U9+Ey9B5w0QaXi?Ww9}lDpIgnZjr^+Ppo1k+Z-*zL>v_VPW`ozQK51Z)I>-s z>j5Br7ioojmTj<`^r8~`@;X#kv7AL&jMojd_ywRoN1Yi@HAp!`D$9f_R?eDjxvSYC z3r-^&(RY+KFxNV$pq{6^6BRo*YpX#jU`j!W4++S_cCNKcftB@fruStPE$dZTGdlGf z*HcUjIrI*3EkYZgXMy^#E_PQRd2i3z*;Cdb)|o|n^In;=tEa3;tW%0g;63crDVG!# zFUgvPv^Hs$o=(j&xQowWUoI1*Zz_DbQpJe$PZmpREGpHlY`P!5(v1MRkLBYc`DoO% z%qbuO#uXhSQg)MXbWtUdvd2k^si>An+3O@FvIsU0=;s7M_d4idv>m#n$ac~OjUv+q zT_TeOH8OZHqbRcAA{#URN9mxN-{M9K-6m<=2%`J>I!;x6N($MSDQo}~rFuQ)s6U>i z?tXE;DA|NyX0r>y%w&%cf|F_DyYZPQr?0)@!4CLyxYK*Z7PL`=7*Vy5C5my%O#A!DU$dP3u@ zp3P3`+04LzVfaA|!w+E?4kAp$82vbA1`9DWNaHhuXZg&aK1EL!ZzU`^N_v(QM|}z< zuc+Kf9?IfKWf6IT@(xrUrYmK&yRw2jl-;(<8uCzy5j_~6h4*nZ72Z!b2`h}gFjvU9Ur7T&A(`|WVb5iQ zvLa>*!YnT33Wc=60gBq1jQUc<(lDD15p?JlXOT@jy40qZ$gr=Y2Ol&04YgD;YZVJY z!dLoKK5oFfoDUE1X2G(dL4^Dd!nz?_ht&GCl^7IvF$*>CqiQ`hKtX;UH~68QU_&$I zM9-)R3T%1I*oUo+)bY?{M_D($L{ZYXL5YMk0dsJTXdsp+tdQ@Crgd~bYc1JVNdEgp z@DQaOWQ-KSMT2rb%$YLAw)&(Yg`jXNESBSpdC0I19NbM>ryBXs3Z#iqAx}gJ!jU1f zeU2mUSTM7~8Fv9qlB#&zB}vb%U{o{Jhy3}R;m&F{SdUh_xS9izU*Tdcs`2Pr7;j)| z7Gvrzl(CL6m7Y#xZH|HtA#I&({&%S^N76vn<}3l?hNI=3amUItBsng#p3XYre!`3= zk^&Z8o~lqx4AR?B{VbQr7NY*E4bnQNsDi`G!C}S@aF|$<@1V4dGng`|a+1q?9X)zszzYx6DZ{Ys%4q4x;s0E><(vt_E9)&;n+Pdhj*hgRfPd zd{uk$HPnNzF`j%)_T;NOp@=?Yxr@1|*D6M=Nx2_9Bs%vSrBWwfJ@~n?DL-wV{8V}J zGuVTlq9;G6dh&DmW%#LKBpTB5pndhrWrpU$a7Bezc`_1YX3MoDMF|SGBAw02Ax7qI zCpST6p_7}8%)`#OjLZrrHuiUNzF0aL#5-C!jLE|wHjkN;FP^7yEJ%hmVgE!17xQ! zR~%%Q6#4R*0A?_Mu2^26qwpd};a5>C3`AxXGEGX(N?Pm3TOtifnI^3MqQ9!D^#vdp<=qq~{}Epiv@9qY)SRQlx{eDE zHcq80g@&gfmW;4b>>j*M1E@}I+$*FH<~6FKxOGHgS6-ujRHF_aH40@lx{ji%s6;id zP#>yLCyxq+^9ren$UrKu9VhS>s!%r%@^qX$4aE(nxS>?0AH@&i@$aMfK8qFV+f)d%Y(~J z!si?BF$85BC4ClJ2j5z<+OFjxE9mKt{)Ql{Q8KJVyP2dN#DoAIvWlMW>&x?5O{}7d z?4z99^N_XlbX_nn6O|9@VNP);(lHxHPDDqPDKwMQVJ>t@G@;gML4IT zD-Wpx75#WVbwq`ZQ=w)d0U)9eFH%KBh@6OaJVcFRZ{dXmQn4|-SU(;@)!9OEgLvFv z5OE(bG8D4GiRjKl!a+nhCqf6Q;Y4)dAtI=_gXa@NR7~Ym2s~sgsOZhhj3+9RITdYr z$W)YjGcP5X%3Z+A_2nUSmyb^hvbfAr2O)!4u`jTZIxL%$avcxL;!)P!&B}hAZIzeO zR`Sa#ep$^gYw6}(`{m$N-jVukc}M2$r+fBp*O~58$_l(m zbx+UM)@QoXvnBQ^uJmluJ&4i|;_2gYeFk#`ft?^%B~ujjnO zd0px+<#m~NBCo@|>bzF-E}{xgmWmdpTlB0Y@-8JkKr`{A=#}J*$R_9E(!5JWygFx# zcr_}DIQeJGX=Yx;NxzijgJhb#Bjv}DREMPTNXlwLNm&gjDWm)VPs*tNrD!S3es{S& z@4KRBSP=)x%ksV`dNG@am*;&{^hT*dPWN`vQkEy;zf$y!Mk&X)6uqcZ%JDOc-iT-K zB3^|2uw;miXn1^}{7CMh#RHBg^LCV<$SsS!3#$ns*!;j&k!rAt!IvYHBvX1J_?xlp zN6UhLg6R+`1g9_H*P+YF8M%U-M^=$DX)QU^H zAF$=_M}!4g@y%bALWsyoFV$FWt(r=>(+9nLxhwu0-)F!nh&GL&kgrPl>LXu$-JZMu zHU9k_pGNR)1ivfbYWFWdEIc+wB(aEW6N20R1yOh)uQ`)V-j?9pq#$ouU>g|Z?F{nf z2GPRI#-_iF^UrQ*2)=Y}i#Ie}&G{R4s3sHpag3IpzWamNsT=!ytv&Zk&YGbOZ+Z~n z<*hb+boH;DMIhz*Y=4D<9Z4`+-bY*F_d9*yM5}SyVU;1htSUiBuE55igM=LkLjA@B zVP0M$-R!3u3YfQz5~MtkOd+`xvX4UWE08JAundKytrSQ-Y~BF`zL>=W3mh3C@Qsx` z&?Cd8t9hVjp4r~=%=eaOskc0dLe zBDDBP*&?GBzrb2FN*m8|jF+FrX^DU3IfN}>|72|{E;F=QxQx=SM4XBEMckw~?P}b7 z%DG#fiyN7%?hky}w#+?l^4~%P-qKzkK)nN*m>;)$;7rJUb1` zuF9ERD9esBRnCqxQ_hYvQO-^%$FHVs90rdGLW+l?uw^#s^Isl^ zzqVfC@K=-lxY5*8*-vS)e#C!-4qS@h?Q3*zU)Hih(F3XEL}w90u(NK&O7hsr^YCiN z{yLsMo2PecQr41`J{#%t9qCIQ>8sq+`)PS{w`}QJ6#V!9{=V~Z&K$P7w|xr^ z$F;3@9*$!tLZt0(znAECKCb5K&&T!u+s?-|-Q3mU{qKqVw492cmQ%6Qa$CB(pO#C+ zuK3}W&IWcQ<1Hh8Btevp7(y<%AIS(%$VW0hHz5LR!Yp?Fq9ghZ553AG7+%IZw|(_Q z4%<6#!xymFZrH0mRERlxD;krhhLfQf(?_8gX8F7w6fw5y)4+hiLBz=EZ*vT&j=?iA z@xR08e}~WivEkDw&EUsHbbKI{#(q?I~e;hZU=xQ;6>)L=%& zBix4*i#-%qc)u>Ib$yiy8}=KW<-5=QET2i5Wpk&a$#KQ|+)nQqq$RG?ijGmkJ!h3? z&UKfq;eC!8de+X6zCj~o4sae!^jZxXp027E9h=r7+T!2Vbqo>I+Wj2ko8?t5tN%46 znlUAq^ba?8?%2%vTx!bU)EBNK@Fw5!EfxITZsWU4FnA_o)K`>qYV^`35<M9?Y!-8|&K*=jcG2cET*Xo#KGY(+aZtte=3)Gd7bI(mok z-pSfPVQJ|_{tGH06{dg&W7`HXq;9CCM@OSMNVJ5I34?Qx7|wNuhOMTsoksIamC^i! z+Gt)JXz$!!{{83{o&Tfx`pB((zTVgpvU*rTFodGGAw+7pd0XhFMz+vhUeqX)uYc4& z@^&tDJ3pL?GpX>E*|3med!;QoP((H}d{4ud2Hzt1=EFA|zL{-#hui_DOq6Ox#zqU3 zVEeiTN2jjZ2)q1dE`#MU4Td__VsrbJMq3sc!>Q1|Hqo-^_EOOt*OxTx21D&pjDb?2 z0Y5S{>UJ9fAeV(q^rX1kt00gRyUtL1%pm2msJlgrNx@>uN!i>%S%wScBGlP?dLRDW zf6t*uN90KT1345J(hC%>vdwWFX-2ctz`9L2E-WreH%dJO)&+OFC_sF*CmGJXL66E? z0fEgH5GSbR64382S5GnhoI=P9WhON%&oWc(Hc>W8w^PaZ88k2F+b-KEG^eAs;FV(; z;g-txXdPePItE8enZgb`TZ&bM$C`Y5T@!{e8q4GwA@@2osk{F6(mqDt-U6wcTwr$Ere}QdA%%6fG$_dw`d|n&q9I zD)(&bT=XQCx(Kp)Bj{T9gen(3RYVVpB6{ka>8{8D)t}=)L|?Vc0=Mc@OVL=~fkX~; zB(|u|L#|w9o`Q-6!>OLAAawt38kh|h|Bz_QD3V&qs8x~t`iG0`bbIX|PWzUBRKVQJ zXdbQt8&G?rIZZ%=u>+V~*Pz4AyYApbl=V>E~dL8(;2cRi#RcwZ({T z>~tj-aj801j~{DIW%5JzrLPZa&h#f#ECyKUO`-~gi;QE%SIQf)%%(aqtcGSyCE_(nyp>qBIhvLAM>SS6I6DB7Q#7`}s`s6vyX3__sf53vvA&wyzhn z*U?|x#^)&t_ZavXh@rN?P`leuyT?$w*ASFz2+A`Atv3YGV6)HI=|*Fx=~T-(RLiG` z;#=5>b*7wlHu-wUyG|R^%jN@qt&Pxz@{i47e|K2+pZGpUVv`Sa3~zx2eq$pGynj3jl3X5UGW5|hhTlp4> zCb-tUg`~t(a+0#hiFK@a3rQ2MYQ38>$wnsm$i!`>oASA;b#5W)7G$GFHf}52l+D$x zZ&$k3w?)fujs?CPeBorO`v;B^@PQ+Iw`CR?y15vT`6qB3-^2+P!z$P>r{hOuy?)@R z1)srT-)y=1uN$!ejoT0Zq2D)(;2*cK^l!~9E$}d)f_>zuJ=5?m`@!;(x`8S8!+7CU zSwSrx1*shFlFFrWhv94dSubkY&oA~mQTKWKBeyYRC(m% zwuJ2=;L>d*tSu<)UmQletu#KTD@wZaUzbUT^*8<;?Lb)K#@g;nr9R;=XzkG*$+ zZ@Rks$CI`|z`{2bbY&u@8eMVdnu&YL##{oaB${9qRJ<3^3KM7rbihl)|88v&Hw-P@S@** zFXx=k<$SK^`#tB72{`xqjqta(>e$kFSk$Y$Eq_s+R=KL6e>m|xD>+*azS-n$dF{mW z2K8p6&`?(8biQ+t?{s=wUOI7W2H)xEw$pJ7tlo2(_+XP2tWzr3=v8KiYr2k-X1#4} z8I~_3%pTS=HmO{{m`hdw8_JY$r$WV;P)S!&*~vE>fXXj3)SG+5cICZH4zKUgN4WNO zE)*1&MOYLJtr0Q?MkAHcFbqH=eE=F6VKf*nR%iSZAl^bCQ@QB|!9Wof5EwGmcVuus z2!nn^tOE#;5`X|%3XqbQ72Ntx4wS%41Bq!m*Av}CBKus1q#V}=`Po5&mPyde-r~QU z*z730HB)r6xA+ey_6s6J7ki6eJK=V?jtW+a9_9rjecs|iGOJU6e!}l?ogy1-7JiEh zwnlLD$m5TDr~ds!CqGtWoo|r3DC>sf`~IGK+ZInRUO2ddjzSV~l(@sjKlf(y&m*%P zqaV+4jLw_y82z1m5|HJ#q8p{`+s2Z)yvJ<|t)Xdh#~=T7xHyQ+(lGkM33zbFDa6m+ zEjU1T?iLbJ8upP7rN;Wm-x$=GFTzbON{{d*9k}xp8QpQ){~ihu{rsp0kNzRVZ`QU* z@juPyryu806}vjpB^ZRUF5((CRIq*dL1Stz2eT^u9bHsM}486u{I#@}Pd-R(Skz9xG39exyJ zk){(;=ECc@M(a@yCTCnR%VcPB~coU;J4r@yn4oEJD4c4hh3Es2aiE z5O-LOhDXM?|1SFyhZTh{Z(Xg{mi&MfA`Asvm?sWtQHUXP*rF^W0}8HyCFdLCmW(I} z=t9gmGtj0)$3GZfC;2+>2s;oV_`((t61KvpGQ~*xIY-=u5-4Vjw--$}vpE`cxcwO_ z_YEsq)3pbA#GTjAeoSMhFw<3L>#0K^4w$^m_O>{o_;}UD3_8B@BuZizl zW5>%2PHB{NK;w?0q*8n*RAkrLT^eT*Px>lo(~6IeLECDPzf(ND=CF=uW(`@Kz{|5B zzg`>&jqW*%USXeFw9d!^6mNP+)X15#%Dg&i)J$1r+ZZ)srmV`|6g66=teX9HY#Z*_ zHmYLVXo_v4Go}qqY!mv}CJeDnWW=Ijj%_15wv8Oo#;V!*an{=pf4Db|zuUW=Kiiv0 z;N#r*0cc%~FgFT!2v0k?NvEJ;ckM*YUw|ckpr+PDU0F`e%fUosz>W6yS>W6x> zqdwGg7X4g|`^dqLT&0$!k!A28CNmXX-Cd%Q{|1}SHu$o@-&wG1zF{d{EPHIpS010U zZ2n_QE^?`0_`i#t|Av9osP%ilsfoXS@8hgomo2XnyHve^=rd{o;UvxfR~Gn9-8}ew zI7iWyzaK}vY$ao1D>Ac?eK$KdO9}SUX%IphEXARUiezf~bLp@J*L65F;c`0;yK(8q zp%a(;arg*_APzlf_8Zh{-ZVWxkzOv{rNvndoz-Yb!#d~>yH!y62C5xAP7jXbK`Whf zYtb@BTY__ zKxh#Im8G3rbvt9jryhX~wNg%xcO>BqXjdmujavF6PBrXiqLkoKdT+aeF?T`OzmVKqt_^ zN4PvphxX$rQX@ceVXE5)K)|Y@3is+!Er@&FxNO4%RLg6^WfxHU2xrxJ(1Hh*p?lmY zSVuCI1;%Fy325lRS-W7Tk_R&Kwh1z*Y$f)j0h12la712_7BHhA!&&IFEEs_-bW#>n zi!OKJ0G2m^BvwwUU==#p(1!!q+pt&Yi3I{{mF?{bi8yx>NivJbZs53&aO_3NhFoac zpfCyd3IP*rQ$cPOAX&XFqUHJ!#Ik|3P?OXU6KQKdbGQVPm5HY&kfx`(L@M3Ad;I0L73Im*>tbg@1=*p1c9bn6~etlIf}iiqu8%n#rBYk ziEtG4F^+-^(9+t35Go1MMnOCqSSYH5>sWdsL3HoZqio{4Lde8(6h2|1BOQf5lvh-d86}82Z{R`gl&L{L zWhe+Z90~$T)gYiU6a=ucBYcDrW#1VTk)e{;rv^QzFEG?O9Rk@HC)7C|f>}RU;IV_X zk?O;hVnaW^B89Zb4m&N4@P3wI;SuVjY#)-54;MhVu}e4^s%a55j&l6*jv9Z!rRTS? zJLmXA_;`*qM6AH!MJ1ndR8qsI%)7Iw1u@I7SEB`D9NTLIDlu|vPGx=9DH$p?1rFgiI;*>J-16$-5nb*$^Ck+oRF;1bSrt2F%c^rE=3_R$=?OKQ4%raG8tD|*!UzN7GE<0g&xdIfv=C=-?$&B}yljM-LY zg4LKqM~5+=jxY1bI-R^~+$c_7=ZbXlmho+I@-8pTPA4B4tN1P1K}+G9P9Evo%B$3D zcDlAZu%2W`61JRc8fGi-=h!BAF@e=}(&F0T-zUDqo?@gf$EPDxjNQcho9qQ0x^5Cs zy5t$!`fi?~ZRqA1+8NzEL)+ZVOCw}cYu&t@K@PQuvt`D7@a`#tKkryS8@Xq>CbgEu zt+9M&af=*f<9rq`dl0P0`GY(qJXntNM|oa&up8$Kd2)DgC(Z$c&JWW0s%JHr`wfN> zm{nQ8cGV()T=fueUF8I-tDefn@$MWPAIoAw&*EB!aZWTpR6SgiEF3WJsY54KlDtMPc? zs(`aQZH4o(kTb7$`>(d4;AcV{LrQGOB8BhODQn*#jd7L4F{pd|MZIiL``|+1#*&QW zfK!vifW)%~5pLyLIn@+dC$T%p7_t8?U5j!Eh4SG1``BS4Sm+a88e>=%>O!(E^+>9$ zv=-8SN$V~RyK2qVgh4Pa=!Q%`s}ah_;>|G?YaJJ;VU=O+JjTY2jX|X_F&vq*{^txIn@I|5=#-*@ws_0Qnr)nQ@#0Jk8>;Lu7am+%VB8sRGhV+DFyeT$q1()SGL zyT;Mq-^~lQV5c~JVb(+iF>_W%zrbCilsbzR>0MUiL_1daceAd4PAtJWNB_cSiOMCz zh9-;}F?@)2WU?+vqv(btjJhOAJ95M@P4e)eiHRC*!q8z!LxzvIM3+2rlyY~>znfzI zDKY=F35goizeG2D1nMgzhG<3&Pf8rA8!5V$zVI!>ITr z%BYda^gMZJ!pITYQNxEQcpmXjtBL-X5ObXvbDb1(JtXFOXv}r&`my-z?S&ZF+L(U{ zG5?Zc{^?@=5&IKIjUWoC13>@0n153jEW2Up!Ugjld(>dbni=te{wp(Q&dj=Lq9JS1 zf`tp_7;Z0leQEwGScu}|ohZ26nz#a?(8Pmvej&|cg z8_tcR@u(KpHMtnF;mOR)db;;TdqHQ$nm#wW z;)NgKxNagH|2YDH?t;}IH_P#Ye-gDE@b?e+IFc}!uJa!fGs}1dS>@7Wyg{))g>Qac z*{t)Qpi@8M5&vPF^=M?%Jb$Ggx4d)~D2v~|-jDR_pKuTUw#^J zaSm#l*XMtiRXXpiXf$H}-<1o7OU_&##SjlYx)F~GiVgG!T84}QQ-^B>14Wn-B*_Ru z@$l|jd@6@QO~hXK{*sW`wwexG`oxGM7spqAo0Utux=S|Zinq02S4tHM>AtSGm0TWK zW7O%AM1+EBWodtsxGyer;v)FXXLTXXjItI&P+1j)sP7TtDF)i}TH=W(BvUQfD96n6 z4hoS}4WJZ43>WA1BHtMV_~?*sOw+4YwwTL{k=+}P4V6s)2HX8&Si@`oJWLphgW?lL zJ_&|T0)N&Fh@_~K%7%q`ur-86a|n%{JZ7j5Gk{L^%F#rAEJoV-i^2g#%+W=%;}pq> zE|MQA!cr4fFnfW!(_YY&0YwXazZ!pj8*)XP%oX*>6%Cv#W^k@(rsFKm9v|fF@lnnm z-!h^-P7!7H_#tPHu1=4CNP&BZi>>$vfN6Ojvv~ZT)7a4n0MZ#iIs-^&0O<@ModKjX zfOHi=x(Xou!%$v}{W|a99W|;gR{`=^Oo=^Hhxve0eSJrZz z{3%jO)*-_-q0f(y+JY6-*4+Xx7d78Pc<|t6|24X1C|K+_+k+8>tK(GK!Mf5yKOQC9NKIi86FVl6E-d(aEDmCdTH& zn*8hUf5rdb#k%EJ-?g#nZ{-^2Gp1)Vw2n1f&%S-;oz<0^Te2&D(ERgvtjDh(vv}qH z-)OFBId$32)UO5WZyf2_+Vc&K@ySazzqGQY@++@=cz57`9@b1~9%0_lzTmcN*8ZV% z-mi)^TmF&vg+)86mcNnUTlv3lUZ(l>-yiwSvIE;s-TY-s^~UkvM?OyJbrSMV5TIWLeyc%8{CJz}kOfz!Cuq5rMONgBc#v|VA_{E<;_+i0SYo2|^se9%6{0CzG zX>#7HonZg3e%<_k?n|@$XOHIahuL@jBu9<$|wTHg=LEp)LUU~oPt>$5xg0KDc*msI862s#R@%}ryzH22P@ zv$mbx6L{;t-Y?WVGV7A02eOqfX3a<$Z@KL<&0|Ay4i&UM|8mz4k1hGacP`b`KL4|s ziR0FnIevOa_Eo#eHTK6&u_G3#HJ+(|sG>AWL;au>yB9Bb`mrTT7c5&b;e5Ks_Vi-sAKavgoNf>v#<+rQOTIqNaD^~8 zpY0TZ<3IkGLlFHD|1=T*&=&5+?ss&){g2MKA5!dJd=RgxZ?MY4>cWcmR4^aY<-J3{u14dz1&P?>ZJT5jxP^&TrorExZ)l?b_u-V z;Z$4E&COW&(^YOS*sF0Q4t2Py9iyqp=zH`|SEFO}!>P{oM;waPO@HN>-KJQpF}uyF zSQ|q#+zKZe9fgk)0vP{|apvcaw?@uennqG%*R5&BR#|O|@%F zhkFdD;y22_?3| zX*%23T~2L@J>@VjEhsPxA!EaG+Gwdwv{W2*pO9=9_e&$g)pusjQ58y_6v-6*tKqltInQMEmpB5jQ29e zb&T;|dqIQ5cxwm_VZ4hmu8F|d&Dc^y0Wo>~kn_veIbH3}D;Dbuf1Y1&ND3iwHn^e$L-XXIa78x zQyQEp2b?KKoGB-G7BA4iLM_jga`zIrfIX!cH>yG+K{wm&DPG*HlQ*ef9Lk?>HSohD z_LR-G!lCw*-Gna|{Kq<_x9V65ifv>mnGDr55H4gIz)vjBOgW13=NX156EW02whACw zbDM?ch8nC84K{~0m-V~`YXEVB)zsiUhDhl`gViLqLa|zeVuc#45)HP56-%h0gUli~ zSWgW;9MPbM8{7d-vD~)`xeqm1FB)vMYlV4LwZ!4tg9gJ572C(wkfCE`z>Y>u&Xj=D z<#nbUb*7wlNZtGLP$x;B^g~Y9e)dD82c!dDyE4t=RHhxJztgt<3T6FkXW^ZOVXHB%Tx|T>dYLhLIaMUJSBH_s9_@)4j zKyts>^tGIc;D^m}i5O>S`o9~(9Zgdt`U7AU$Ac9?lYRi4fLPUl;Y86CiT;Lo=!f9Q z<|1g?5vD1?4}EeGA{@)0U(qy0qCWsuaXeTNG^GxJQ!LhHU^r1UMWVkU9{M3TvbhMF z_JnDYVkMVR;srpU;?XokqCWsuaXeTmP0phCRx`-fNN5gMTg2)>HNpjY-~!424Rn63 z^w;)TIKPY4@5{-<5uoP?&aLork00yrm+Rwtr~c(cEp(Y;hAOzOZBTG>4M)jp<9fF} zr9DEigRzXoIz^FuWzD4aI&H3E)UP9-#!M4R6X5e5a$${@zn^Fm3qtT>EfIZs0Nu{c zZ3kZ;;b+x+ege>EyVl^hd-QOJ}N@7{3Eta$ zBhoPXB5;kQCpD5DKCglb#o!u=hb|J22wWrSNsXjOaNnsVM;LIpT7)}vQu<4#%z?;o zci0`)33o{8hcay^xxp^_J4Jsb><*WmMd0-Xa)Z(xu9EI>`p0sI=g|*}4@(}4j1R*K zqehC?aAi`Sgc#@OUqDYn`zH?SKm7?q@QKbR3USr=v)NI2^A%Xy4_TofpBzwVRQSzQ z+~NdI=7x5$xV(BzG3kMDO6bIzm^+aYQdDE_`N-xuVsL6b?7BJGqYMSgOBtCBJ3(C+ zZrY~ip%EEa_`W_W&k5g84QCVK{EjT7-P0U~Uk>wl?;v9QYx|C-6I|5gtOf6w|qD(DUT}m7Qin~7(3IFD~ z=NA6!*sTwZbQ`=sQQ%rl?D|8o&V?j@e$pqD{K)TbA^(&_Q7+QU5`K`=2zN-rqmrtf zV*i%Vh{rCz_NwJWyCe|%$u02)3Pa@v3Fk_>IA1`YA~&L(i>~#dex`dBVz-af$a|HP zX>tCv+g5nM$GSl0%GqjPe3Rj=|G7iG)BVe;i?K1C`V+S;>PHL7?NQ5 z1r%9;M3}UTlX)=$zyc<=2jc=HdyF+3fd;nLl};C>Ftvo8(-jbFMjh^VlGBIG?W0Cc z+Q$YlDnPkQ6aA5O{}TNil`?ly-KyYxuTM6H69@a2%?3nP6h|P1Ap14Do$OaiD3c&d zFJpxqx10kHqyx-~!_rE9it)b)U;w>LjSdTY0`4fE37-t5IoTPg`vC&Qf<+MTU%5r5r*c`}HYC+#;#~G8JkfSJ=}Y8+O}y z$c>SqkeR&Iq5!&Chq1Rf7IU}vL4Qq{bW6C(qo0KhtqYMtcM)=h_xQ{dpc}A zHrk4YOnwl~$gKN4h*#FXnIM)*Th8B0=QJr3^677Rr&XDNciA>Q9kTHjd^TQ!=inXq ze7pkB#~bkFSipbCu#fnIw#B7`eHtYu zS)KRugvz;?443z9OnS>(l5H!xD@V*S&cfVIi_*?>t})l3o#$&~szEzuhc{rRK|4=& z#zcem<2dx$`b(Ad>-+hWgB1F^Pfv$A20ARj7dlUx>98VONuOlI3v)fcFgNfE^9+7r zZsr%}+5EyBugw?oYx8%F5MO><4iUuLa=fTI$NtqtLq51aivQpHg#5?1fX^QstOG0& zj5o(JK7!|>4^+kzv&|R4Y$f%3c(KInX8!&%K7T(wu5R@8en(}N-PHuRyp~Lcg-`Tt&%MSGS`;G*6_zngee0zfj=-v%x z+TOx{wK8HHK18PiU!}j51}ZVtOo#h(=)hZO)9;&`@O$^uWxgAp z?(*&G^ZRb-tFjgS$PJU}aW-^$Jy{JWy_Rn_c)XTt8}@lES2i5-S}tqoAS7xDiTz&B zL%}AmXHl@m>v=5Lh|7b_(*h#Z*S9K#{_Zo-VUC%2@Ci9;FS=%s{Z*)K=4)6UKOTQ4 zwMOvwY~Hm8T9~&(&WPV@$F#=q_%gxc>*;R^czhdpd>459rsVN|GmrN^ebV>Mr|W%J z((m5BCf_&v+`cROnk27%yBdPN%7z}V<>rP8ujPh@8ZbHNwTx`&A#`>TI{U!nYA|^Z z&UWE!A9MOya5{tn0Lj+Wn&EEvOnLzM0|ktZM6qX;tI08Xn%*(`_EgSOYGj_`ksjSC z^OOd7^rmp0;*nNADo+W2uk+<=99M|Oub{@S;KrR}8|(#MZad{T2Uyg0%1P(?28Uv$ zrmdoBt8CgjcnY-9F`>yZmJ(mDV=N(BZs*2r1#Z&TQb)>h8v-U*wKL@uH=ad}XC0wG zk5kDihkkRwjD)}em*;k-lznpIbI(0KLHXHrcgKBsl@j&xc+FupLQ39Ho2#3C@SQ=f z!BMz4oiecuN8yZ23d9cV=P~>v`|)+wDytDI+&hf35sM<+oQKwt1Sqg&?Q;h<&)1lCW|AC&a@Pl`oFn6XN{ufA5T^LM5>(WJ?e##QKAc;tbd*E(2S{ zQ3RH^IGc2f+9a)6XB7M1ta{NXw!B&OvQcbzv+7l&*y?80>qfDs&8oMIayy&DXq*C` zz3+q}V1-wP602nlMQnYOe{ej5>g138^KJz)GqM?|CP4z#CkPNCxN0wDWoAuvtuq=D zkiWfaOy}PZjT!vA&X}1XHqFTK|EdEv;c2R#-~0~MZ2Dw6MEC65iB+PZ#+q+X82(&XY3sAbq?Q3jNApw+6Z8) zCC00@=iv?l3x9${u}RiyIv;lsZ}_(c1Wub6CsMVa_cBTXom-UkPaw3iE6-K&)|t-r zZe{&!=>7UKWgWijWs`uZB@`(S>&9j&I3RAGj=W?#@{;M6tBL@XEvSL|_op_Ms8GRQWOOXHX9WX)JAzCyg(1ZFKH@ z*||BgbMs~AE|;ClPzodzThTbfR+Lc zGL9H9@z@Gy=sD6TyeHK^q$h0B0S?=Q`8S@B%;tW1%UH_4FY-)&@TE=`#x`O8k`oxK zjCyCuGAs#Pt+_a@V}<$3M*;z}>}|6YX*0N_3dn;HkVhdP5|fjM1T7kE1bW`%!h>KU z5Ef6b%vJWUGm2fzu+Y%{7mZ>UGc2as@G{E)oxI9+khBs@e%FhI$*~0B{$jBN`Thzq z^@nN*OHMVUCFdF>{P#iw9?6b^YlwuK5ec_RZKYFtfpAx=H8w`W{Q|)1qrjRY!DdH- z)u`m^FO=L(D%iJU$c5@Zl7YTmo^ZkL`TXBUPP;9he||02Kl`<6h+8wbi;m|;&8oA` zR-JW@>a6osXT4f=)*HjlI#G4jw*%Hp)mdY5)~a=qOR960tP+lwVYpQoQfT%>_~&{x zh&K4I`zYjkTi>x2COTcs!sV(Cn8GO42RmKeVK)pz;&h#1%ii27^g?J8+|Qqn3jK<8 zH0OvoUu}(FWl)u5OOo4qQJx!sZj#vMXw~>bdsIY_gSZE2Sx+C7VJ}Ll&hi;JORI2} zY)N+=lw?Gir*n|OF804#;@P7YW$l-Z8>CTNEsfenY1Cd(o$JFbVFNCfM)?$NxTP!7 zmCL|oYv&*#Ir?AeifglTBzZZKyiJn4HzawRBzbR3^4^#6jKTk0b9BxzC_@4L&gM|Q zBJKTEo$-wHV8-&Yx2vHQ4OhO;zn{QV#xv32UI?n(T6OQ>mhYT!&1#M8^O|T)B(eHP z6CKyToD*!0S|fi@6PGu{}F zJ3PjtCFAJ-@cd&uE6zX0^WNFPF%m-v$6EjWgFLh14)Sb@^@M?@h~b%6qX&5w@ud!J9_85-JIb@8Wsp&x`_3`S^9za3`A2yO!2!cO{rjVac{(qC zm}ld8zi<4v58*^>(Wf+oh8zJMW|zNU*A!kHO&2*)iX?=KLi!k5o6gwqsf21P5YqZH{XvLznK*gpu1zoZ=Q|Y|ASdAIke($IG2Cn{Agc1 zVH`MuqmDMj9l=>6!3-S1*%W64XJ&(x$w5bO;&s=R4WHEr&NcByaQ<{LBRI|RMsQv} z&j`*SO2_(=u}83I@ZlSGoZ*{O^>K!84*uWp&F6af=ECO}T<8!@95?y-9HNo_@U{A= zA)3(_GeomV9ir*0jUA#nd{pSr;6pS6;>8bY<5w9}?buWl)LxWl2B7TP*ddx*j*9Sa z5Dy?t=khukw1$Rgn&J%6+*K#Z7<7on9cPH{f)GaQ~-8*g}K zWY;-{XR_lB&%At|;TaxEi#Im&>z&c5!l0Z4=%;qZj?K)wxUrcrZZS49&i#L5Gyj6I znUMXx*!3-TXyUJL@i&$mGl^qPucL5&XY|UNfuC#Se3OfXH+K+tw}~6N+lrF(yeMtq z?9cHIoloEOaaRDnfc2d2#!ds0r8Dm#o>D?O#c1CGA=89bv9OuQR2^`djyO#xoTh%8 z>4@)C-cb6Rq4%9yWi@8dkyM;+B+%b&;-;Q&8f;Yk60+quWa41P!KSQ(_}aF3m@-#| z?HVehtVh?_wv_NVNLq=q4(l*F5)&MwZ__zO-=%kqzCYDguv_C)-YV77p&ExhTD$Vr zX|1g~Xsd3vtvN|mbjtcSFpQ*pScZl4)=kiex4oH|=V7+7!%+T6h|>{hDSuo>w`)4U zEL${E`WU2IRiv(VjQ%zuwSC87>#mBsB8u8TJ&)$ z4h9_3mG$e`M+wE zJb3bOD5*?EQDrL1DpQ{UxgQWqofDdzV*}2yN1bC&+c3GuH7s&7aGMb40M>-DncSpD z*$+PuQlv=Rb|L!1@g>h*J!|u{3?*lNd102O@S&tar>^ksWZUFEW!5G;Ex*! zU;E=*&xYU|2EwmZo=k5nxA@~^s;a{Dy z@re-pNdw`(^yDjNL+~>P!vFrX6~7I^pE3}Bn(ke@3jgv5@aSzwforFwrv`}3y>|Qw zOIjn91*5ZuHxd?BZIi{d)#DyQ=%Ccgmzw`#c3aZCgfYnFQVmvZi^bLGDIQKpq11~h zci#W-b(XZ9gf{`@LM2vhv&H50ct#OsJ&f6#Tiy#;(zX)v=y@(xW7W12MxK%}#2$h% zk|2y=H<)A3oh)qclHn^=DO59CJl!uJssM$J{uudMjy0w$S&%jWPI{(P-@h8~|S`??N{Ci1? z-4jT1++-cEvwK>mIBrTFzoEo=Q*sSep|+f+DqDYwo?Mj)$HJze#1tSnPq=SVI4;XDY?|K=O&ko=!=FQXdf{xX; zqPKYL&9uW&chuPk|=C+w zIE`jPxkRJyQ=_S}s4h;UW9XkXIIY^K)eO;UDYcp@i;jrXDu(dQqSY#D z)h3Hx5~tPK)M~b9wT@byEsG|{X*Gvh%@M6OQLFQ1(UEalolmXii&im&wpn?~8mqB)D6);mna&?DEq zI;;sthQw2>+fb#vw}*6}K+;vnjNo=T-6qRp97lLKOn)pOC zI(E;&41to<6i~4$yjS=6MC|j4RpGr!pHIX-pI8-s;q!^tzZa`D&AgLUeEB*v5I068 z)TF?I+HIx^emh&pa*O+5k_-q&=@_1j3Py3za4>_S0)SDW4DrbX|9sLV+I{suyMd1b(`xW*gg1Lxk~O~IytS&RoZKEm4+K; zjz);Z8+IS1N_oY(*vOl=d(N+$A@-*Fy0ut?V|faW`!wP%ww=$zmeCE=*wQmtFHG#M zB(`?p?Wl+3%6+NDrdD(^Sw8C3$(p>6OogblvzSTUxYEXDXB5E`$%~cPLNilSY{$kM>z*3I50~g| zMMG*XPtz2dlk5d2GnBQqM2+TJjmDhxO*&|lb(x9SEG>{>dtcM!HmuGuZ^FLTOEPd= z$s1SC<{e09i{`Q929CT-Tz)G!-U=@7VUD*k$3NnwJ=*Dj0$C_w6#|E!juig+NN-W@ zZJ@s+ihgA_{T^T_y!WX_0>QC3vMt_;l^e{G5=N56DoC0Mq}9PYUxRx z)MPrZ~8;2J_)Fl?lj49fGDkyLIBU@Rhb6%>{je0d;@ zWfQ7y=4Q5uX7qx%U?niosXCCUDVIoDWp|Z{O^O9}+@xS2+d<}^Fof+wX2cRnV4{#? zfjoFAC<8NP86n52>?!#NXQ=HRjFvwZ`zYB$4X`0p9bm8O1UZ&ANqQ~<2#jisqZ9(h;(S_ z_cbrw@hx9~Vnknc@QANE_b}Z#t@8!iYkbuWH%uC?(f9(vUB1AgqrQ`Q#lEuOQTvu+ zY~&dDfhN5Ezqj>1rCsMcwQ519@8qI7Uvsd|+q!}{i(@f8o7q(YC#77oe(BC3d~yp z<>!+iEpdCN4%Zx~vz1L=%kV|p@ML+8w{;Zuws8Gy zZ>!0=n@UX%ZsAtEty=0N(F~m);tHLj6*O!1wobMN^#G>lW<@Up6mhv(w5&q2p%{(W}oDhwA zXVGeOOsKOQN$4eQ@RaPaz!j=Xi!VRnZpmkmjQu>VwN@iK7{KXrUyA5-;LE`0O(stv zyI*CL?0(;bU`-^#hI<<$B!HRZ8yh_6wPfC_3WQ~Butk-EvB6fa#{~Uj7Jy$~&sb7L zuVq58T8IX-(t?J$|D%|wXM{wru<%TqptR_uWU};dx2WR6SaJWz?jx*3*FbKlvh`BE zM2#5|LW(6vFa$`%NKOd8VKB%`G6sMq-V4y__wE!Tq#{2E{A`JzCH3CcIH=@Dp^_Pa zNrWsxTFwYzV!;VX6w8lNu;*-lzkQ8|g{fN`8y|VKEAy7=I`1e-(y@dHkswTo*t|OW zYo@<85v}1HQbDsMVsnJHGzrwTA$!4Xu$)hZ#CVqO*NJ8poU56t=w>=&n}L9cX0AS0 zGw!HnzB)KEjhb_`@^&;bhB(NWMKeD>S2LTUo5>g)n;GY7W@B_S*#oe7Zsl?`6VOHa zcZfv1CL~C>QLGjNpPW+lP#cz&L{I1Ryznd0{U$;&CHqDY1n`Qm8vVOg+@00a^DC-x=`oEX; z{|ZdjCEl8#8&*E_#iSWS^rLntFOR$@Id%9R-K!%WzQi!D<;vHu${nA6McWtON?klQ z1P>@ zUmbr_j`7#m9{AFA*Ui7?&8rU^ZcNL+{=dI`Y{J*lm*4QWiDz!SlH2#V+m>Lr-v0b;rFV>&;c$H4S#|5B(`RM<#9C+hqWQt8zns=I>*~87 zz4y2G_-9^!*TU>S-Pw8nS07k*-+QxrANu-(&piCkM>NgFP3u~2ZPgtr|LDboGY;s3 zJ33xIe9w_o|DM3B?GLvZ&bFL+y)U;v{Z!lOw@xnZ&FDUQ^xe*e%ph!IHI;R-ah&{<@Xxr@YYjy!-bFZ&=@a{tu;p z8u4exAHV-r)$cC-?X1^-@|(I%U;NdBul@4ZO@F=m-ACX1?f>}y`})5ueCJPZcmDmW ze_Qtcdmr?E`1OB0^S}T6=X9H8rY&N7$cct&ciopg>yGIb=S;)g6^1Eu^7EJEne*}= zTfB7X!W;9JJbp`5{BW_~tIyHI|6bkBO1WCRA)noAE1ds1{#<>_HJ|jeiO@fhiP6o9W-?R)af4&Q_`dwgkUT6|;AwE0rb9Q9hB3~r~afl`I&{est&&|#>u zc4s0MmR*^5Uz5*urW}TIx7U(}Wn@>bqWHmLZPz2T`pDO-zjGt5yS4b-eR7X?uBow{ z{DIY&>9wQ;gLqb<@y?ylxPu~!V2?$4X(#TTPVvr7Yuv*yI_*7;-nnBNTPQ0$jzp_n zu0f{+S{C%>=9k!0t4MfK{oj?+F28#Cs+7z4uok}#OrAuu`IBm#+oTMX%Hc4>F+CaPxW$CU{9&@3Xe@7xOCj9yR8RUh35kXCM&T zGC|%UCot3ywN@HvJ>B0=@rw|E zQ>&gKWt_*{X9BnjNR_|9q|nK9J<~&Hc?QuJUBT~Uh#v0>t}@f%k!+wu{5I_^7iNtR z+gs)9I#Z0!Ty85`l|vZr-RnDfrp@7c-ME@<{?j>Z^$6R&hkVs!>gJ~G4f@K?oFZiR zcKbYM_Sg%Gb6_ErwNt5vZ+Xg$O~M{7Pr2u9--^5{I(gVlCwu86w~9^<(aGW_ItkLr z)17qEP3riJF+H&LRDZv)J+N5aB^NM%2piHAoXSzw{)HzBCK)3>#iB5BHt!Dqb~^vO zpGiMfWx{Hk_}wsD{BE2teqUKGeseZpFy<8u#=L>SnD;59W$uwjS#2qto$M&OvUwQ3 zC46$Vean{BiWqy*+6oS=W|6QJxxhYWk=$9hQti_W0_BH+3 zAy_HEln}>+GsJ*Z*^Z+Bpx?*Mj-n60V8-&+r}FYLY@W6xB9hL(WAL_EG`)0ouio2o zpQee$PAoX^`J4|f89xH z8vP`?h`WcohPwq_;+;BKlSm}3vbZ^#7yL_?{7aYoOPBmhm;6hY{7VP_(!sxU@GpJ& z`v?$3pn!lO$BzJl1BoI7c5aE?>n5aMMBtp&LRuX~i~6CA8y!V+Hu;_mZdH9NVwsRr zo)Fy0jyWyZ$j*5zd2MpgzLel$cwXP|;IROLeEELyJVwE?TtCI=#O}tH{z_&2wtlRF zzA|r|QRroJ@Jqs4HV5W(sJNzQpwVHrs5XarqnaErh|LZ-#HNQ? zbf6FBC+_}8@$_%aDa?#j*1ntcFIB;ZlcH2mDxFY3(nQj{^C+F8=xd~EmqXQ{W&R?h zv2#rtx3>lCTe-?guB{~viU#%Zwv3gkMoN_uR<+^KNm8{8zXJ`x5)>HH%^iy;rlvMN~G*xsUt1I{5%2f_b zdCXfTVlYdmuWV5*2bXwPP%Fd4k|(X(HDuErgn6so+9cvqJv_eGV_Foz&CV2z3Ir%v z=<${;;GrjS0vXf_M)({?9wR=@H7y2m>Sugii<-UjOxB7NglawvIhNdM4QepTKsCU- z)+eo<0+t!=`T~ni;a+J9hcccggJs^5`>Y=cP&q({9YMagbpa8;pt8M|$%{&<%?{kG zNx}G8DUkA(SgidTYUluBA_=C+>%qt)#YFBDw7-qO9`Q~yS-0y2zds0`^1Ty0CV6EE zR$|z#l`7jQ;PRv_+U1=$+}cfb7X)|t-l1?w@Xl+wGw9)%hQpcG6suq5VKwQHW$dD@ z-g$JB(c6lsy~8_gveK!GWu>=uEQ4bvGW*jORe0yoO~$tZ8NorH``%}adPZAAd@ZM@ ztkd**41{j>wx+4D%?#T!VbKonyfkYaBfUd{r5K};QJ1`BX8Uya_C&WeK}9B;kpZi{ z^CqZl79&2OKqGLU4A6A!O<~X_CTou#0puwJ=ERwU^n1+L1-5a7DL4k!fomm`l`0*_ zt9t6@;;8EY1xUE}@AE<7A*7h`qnLliG1uCJ*c&m|maLgK%&;4#Pjea;& z1pYGmV~q(6@P8(^P1(|AOL5y=O7w#J98bjCuVVcPWt{=fwiILN7WI{atGTdZ)9$uS zs8!Z~1P>(U2uHRGaD&bb{+RPW;J=G5Ywa>n50A(yoSKwXc$d!7<3@KV>%I&jU4!1! zTh?IM(wsHC)UpQhlC7-k9nw4@> zCl$4(`KK4XI$o5mF!fqIJ}O&680cM{Bsj@dS3OYhbJ?b%55JD0+U=%Ji^n@&yM+)K zV|Don0ln)a!3Sn2Sy%M#&0Lc1_${82@!HJMQ`00OBcZ(2fAh% zsY*bp43l+rELnts!PQF$P=#i2K$3N9(W_SuBfcA_OL)w-y@HU0uwAP=DPpkghv3h%R6-5rmaX!loC!e%-+SS}=(S zn`O*EcU4Bv^~G4a2!%}Qu7$=)L>8kE>aTCcl6B`4A`1v)5?$pXx-3QSel3Qun$+SS+2Xr=}G|9+nH$_Uj zR1usSAnY+z3zU*r|0WrQtP2I96C(tD0NKT?c&vp3IU}0jCm3>hbBx{(dVcu|D)FeZxR`61Ahm09iCYn3Nk&?3zN>e8SE8v|f1cz)IR3}Ps zd=Z>sMP`$%cb#TVGH=)>LV+Tg!?;FrXBJ$N2)nx3bqM~DnS&xlF{n6FbQTKF#4%Vq z`w*5KGIyj=QG(-(U=FLd(5!J%v(lwzO){Qev5Zo&3_`Ics*06#KE+CB#WJ#DWvGg! zy8y+S$cmLN6l+pgv69cHSQ%2WCJM#MR255qKE;w@9jnunkdqk$#j5qpEsu3F2-Re8 z7XTB(P9_!lgen7^Oen|`8kD4ZnQ>4ht`Y5JLZP0}9G&W945UH=rx+(A!#trb$zdNu z*7jymA;!n3As(xVKICK4p)doeE{=@v0wTVXK4u~mC5C2!k{aGg#YheLm<*^$EYSk1 zF;b89LL60;x<=i5RkFaK~DkDjbV397s)pD#X$)V>_X( zp#iak%h=8@#C@U~LnT4(2XwQD?3iMf@QhIWq$636p;iWUAorw*=Va`Z z4DpVomE$=gwTu0RD$@GFMS2p53|o9 z#vqOENQngw5ki}oVM(MsJ(2SCM9R|>=j$!542z3yx#*UQZn@^`@u4(Eww3kMc%v2G z=H&cSA4VYy>?fZ z-L=E+X&LI&mOF}WE*ytJ1^Pcb$x>+3Sqih0UHdGB^YyOnrrl|M*kJi-?3J81!;SeL zS{&?p9Pc#XcwYwQq;1Aw8xFOY*h01Xc&^$$oMEC!Uv;jsD#=J^r#!r$gdV45bV~A! z)9L`mayEJ@5->5N!&Hme7A4rpxx%$2aiTF-S=elA(z5}k^i4G&? zb1bR4%H;DD57Bmk@N&u`DQyWU(vkRv~`{e zrl!KQ+qF573>#VtFy)|Tj8S_wZ6BKRnJQG$^D0f-c>WiNIpPWu6H?< zcH)GAi0L-9f%tA9u*uQVs@H>u6;x|`agJ-2?sI62yJiLHng|RRIQSp zS8J-}Sy-Uv1a&?+U+#pf)c(gzHST`UI9_mps5vSHU?T2!GBsU9GGw<2Y=9Kdy-rmM zl>E~jCgvqb>f6RrK;pdcj4A^{&>#d#@U&n7QR9;$FbS>oGUY3oZ+ zBP3}3-20hQ&}9F!prQMPpfMMSnqDaYQ^JCFib#f_HGvHf1G z0b`7Gf8lna1xnDPJ({)!b&H81MC)#C8&8!YHb@aj;+a)UIEbO}1HBK*iIuAK6Sqy9 zxznL6?H-l({M{Ye?lh*F=u1X#gSK8%Eiplhxk#0X$V}1I3#(jPAk0dzTzlNKou}T3 zj_-48p>4r(>ccUFR)^5wylU+!?k2JyM54BhuuaVcPY9RYf@|DerY%ye%eB>n;y9iv z*g?o_W5qOe5D{9~j9Nm{z!qkq$F(2As03@You=JD=5o<*ePvo`OR$O%HVvHvOo3aiPom$o`(`G_e3!UFjxMr}4 zSXl1b4u&ehMs2q#$o(Z`u5X7H(jKfQ%qNjx>=PQ4R{wuflNt~g3_}48DW;1 ztsR7*YaijsQq0B)1}za|n{*}Qbz!Az3(0~K>=t7c_HzhWP~hL$lS~@klqoRo7w0DPjN3yf(?=um@E&Mpq?~{F4;$D z_9Vc{h<+rxGSl`kneC^DI9HHFh}AWJGkRzXC?Z;>!0UjFTGCFI`wkEQJC!)j2m>Po zeBl;~H#%Ucgdaiwl0FiW+fCfPZtfj+Ez+>M(LvI~%YI^G#hu0dOH}F5T|m-Jq@lZn z8s~5CAf4C`dWe!_*+b|Y*(=;3Fk7VN=nf(I3ro=*FkPV-p_ufPu&m-hAy~$JGe-6e zuq;Cd3C}INNtmuSt#n^RRVsQ5NR~nDsJDc6=T~)*u51TAM9DbWKj<3SC)^z{XQch; z3?XC-tI-)SX(8WVO8QGkdRVN2JGf_zvS)zg4s?(7$g-D&N%Lu?TOz8`*&Ps;J0Nz{ zThjJ>$o895+doluP*{0tS#|ak1sTHr7pwNaQMLb4?E1=u{jVc>CPmo)64m~bhgBVb zG;-1bRK+-eI_`!kv8F#U$p}j%xZ5WzzVLOb?M#X_LWGJAhCOW38GLfA9d}NyR@PxD zfC~+n5hfFkdbUOgo}-LWpNF|Q2Bw|^=de5F5So!qg^dqecSd$xtVt(i4a_FQ=22Yg z6T)f)mqbV&<`oC$j6%BPn6%mqcw!tbWw850$RFkX`l=a)N#K_V_QPD`u%B_r1Q)^# zV8BzPo746(SWBYDl9-=lNu(SlF+17fqD!P)`AM#Bi_4~SZMV3xlTEuVly#-`1>Wrz z-(#sPj%j>MsBsyhTr4}ooD4HDO4Kfrb20KYDOSENo=IR3_=HO^3b$}g=SsRv71NO_ zrXy8MN2-{PR5AU5QfJ}PI%nY%P0qr3etY2)o%X^9dhJEmM8Cej$o!PQMq4g&RGfTw zgV(a0vQ!*ze*?#v@8dXVqS`X@oQbN}vx4-_Ygv)^up8gcX!-MZJ%5+ZUyt3F;cZzV z^bVixZNmqjwfNL?A64yZ^?IJ{YxjB<^mY4At}+_wKmjWqml+L_@86ZT8Yr6awmzve z={PU-rS(;qDwNmVLwp}BZAj7{Puq`gwPBj^O^))GdzhM+dOi0cFI8SYGQ_t+%}aeB zEZKP#yp8hKkzuBNzTYoBlkBTkQ&N*pd40zaug63tn5wroX}i)m0`Ph!kbOpr&B|Ll zhN(#@ZcsX^yk5qcD2%f5R@pEUzW-7JI+1)5%3Dp<%Igh4xiLxGnYP_mO*tm{8A7>) zNF1gnpZKcHFUea$krc8H!g$wJppR+WMi`$>_EoESr=U;8_)dzdj;3)ipkjPyFu@ZOt*j02xefk=G(%!JB36Q87YN*cjx z&NB(|+N(e%Q&>xg_a*zv)D%-tsUmJ~??hLEOcil^uq&cBC*sCYK`r_*+=ck+|Fljd zlZ5g%K^&pHFA2Se`5T~*Q1(T1-6Wn%Ajw;yCXHx;t3h%rp z6}JxGe~K*8wV$OZATmTkp$u|5UgGOn5Y7;V5um=1B1bk}VCxG|Zy-;ErDuU5Q!j9& z2HLBT9>U-=j%4vAj`J2M9?Tk-38HH|%SN+E4hfNRiLhr5g-kLp6k5HcX1tW45NCbO z6fYq=RGA8fs?40zj+kqT%FHkez4PWM#hCRX(n7&TeC-HU!t|-^B+I5|gF-=#!IiS$ECA*|Z zgm~(!rUVd~pvq3LQe`6VEEhQi43l7~usdpA zVT8qz(g0fmL|q-eG9-NaC8IaXM2V6j*AOl7QSp-as=m}wDRO^rwVPs%3M7OYsiS8SbGKEmM0Ob-OW;GhfG;BYsv!3p)nn6SAC_X?Yv=&ljD9UB`-0QHEJ{HVN6 z@I9L45r#EfAbSSFdtMVo2LTnGzu>szFh6 zxoBIWW^>UtSk336%`azkzrdWji-;^my20_KurC~2hK&rzmTLYH?fTRVB-%EsIY_iw z7Fh^Ku!3bAy@vTFqE|I9iD8_~Orj02nwt#Z7iA|LBnzf-fE?zWh>z8LC7OR~#u9C1 z)tp7pE?{3|YPaGTR`HVghnn9FfbywuI947%Esn;Kz)!?MO0NEOsb9Qr=tEcdMDkpJ z_|WQHfA}ooTz_~Gd#*psIl$Zl%t63hgxKrY`3W;^7L#G_!n}D;7?v%Wvv_H4-jYQh zV~W9r?yoR@humMGPZPPnLVvT|U*RMEp51JYw`RZvtQQdYo$Qd0y`Dj?z^f{DKEn(# z6JdG8OoZgnOoWex>t{WI;b_a{A7cW;VBh~=+3Laf|CfL%S%nYiLqKYEAKr5zN zR*quZU9w|#@mgK4#a&%pb=7q}@WK|FmaF9|m)t5kr9cZPv>eIrIc6r4X{n&)`}zO< zNILI3@AJOT`@GNjUNfPyQ@SD9DUC^SN)ytY(&V0XolVjAUHNy5fx$O)Gnq2hI;H+D zf6(EtbePvT%B#D&{A*nqK}UH_2Uo@xr{8LIWK_EPR>7-ym#_o!;8iCm;Sx?c{8i4_ z`%F;4k+B;JB;bdR`i7xEj7vD+$T;Z~zHtfPih&t%F5#$Cni}WuZ*uu}L|tQWX2G_N zZJU#colI;ynb@}d#hTd0#I|iG6Wg|Je0jO|y{cEWt55YltskfA^zOZSEoyqMPt}+x zxs?R;8Gq(5XVZ(bj$rNbU|t(o8JLnx;N}Wi=XB{_ND5bDhz>4gX1`wQ4I+mN1RmI= z>gAoi8lKZj3nxo?n~ds;VgjvkL&3F{onI+9$uiSKu)ESsbe}1I)Mj@vccOF0-fo!`8HUd~|f=H=rX(Rma*z(pgX>2ofO3u#; zGQKfRQ1SzlHLh?sYKAZQP;nl=WSP?$I~&TO_`219d`>Xdao>tAE9Axa-AANOZ{y^I zPG)QaFP!Oi^7+~s<30cKW+xA~FOBpZzLg`nSgdp=5ReQ*gF;~(M`dA`JT z{p9;kIMYuv2VMAdfmWe0r_HTt*qIvz*)dVN@<))Nin}TuD$%-Ylb07p?XT)5JPQy5 z`v<5_b1<4~Xa!zXDQ-#R*6U6twj5m2_S!LX>q;?DO%xJAYjQC*w&U?tp9dBe+D>eZ zn+H@Cwp?RY99(#-%>>aOk4_kE$m`U*PnI`CdG634#pq#acfr_uX!yyc#K|UL}@DtZjkf1jWI;zfr@N_^5Su~&MdyC zFF_$Q0d!60ow&k-$OHlLYKT4L=2Cq^FCthN%p2ZB^i+Fk2y9XF4!quKY>m6fczXGU zlO8OcVY+ZEh6y2Ct1P>UAEb%VFlyNniHMxVpb=T@TwG3S7!&=(^$(*Df2UK6QXN|O zb5kqx++XOZInn%PBuL6Cx%~LSeBY3mWM`Rp@tMWx!K=ZoL~X2>J?`m#Y=k176lFCK zMGX+kXh+m`d83K45IS6V#AGmqFSxE})bi=z4%(xj%{on$*Ron(G^-lEa|(qOB04l? z7tqN5jNVM4u|eL3t;3J_AuC|6nkEd_VXL~ruZO*p21$j)Euoqw4%b1iy23D(@rM!y zgm0azqCdN^c3my_a_>kgsM3FD#(C~fuBfDLOBLLD9~WH)zBNm#?>l;SXqs_T5E%w| z?20kEG3Cfmj8sGxNWLdH%mB_#3oKJ%qF2N!zjchjtN_=Pz|Vik*Pl0A|1AXCN?vb` zLth+*1{*$?k{scby}W^A0=mzI_unw?Y`XW8$4lZx-1!#94%e4myHLumrA40}Aq#g}B*;Ij>J$JuWLramv z@xvqb8nA$$M5B)-4@J&5L){drBWuS>nsa zlDiPf?G-k+vGPCf&RTT=FjqvMFG1E&)dJRb8_QAE;r9)=1mb_t1k9k`@b@wuAj_)w?Y4gAH)u89wjq; zUG1j}_DqR=Zh?x%Z+=-`NIkbzRUEOJk8_t2_tj};vc#SrjNP5oHQ2=%)l}blzn=YR z{HS{ErM{k1)+1l5oez?`BFV!K)04!1|D>%HA`$WgCx1nqvL05PI_4XX)>f<;4!_E7 zD_FP!>klyw0>L9K#h)h{;*LQ1t*bqH2|DdO1;a7j1^NciT{c~`S(9eBJ2I`N@E^^n z2>7k0Wc>X&CRPMbWxM+46vps{sTCtKrnHmNJcXUKyaEbzkA?Vf_#0{4ea<60CKVHO=uQiYDolFfdh+>&?B^=kU z_&?IThkw#-{lk?xesd=WN&cY{W3Cdwm!E&a_=t$UQ%O&#+wWV8<1YQKLjNijv`YA#DWh)a19i?5c~K0>qN`M^*u&MiKfd~1gem~(v4DyRKYQN z@85}BApxw6FanW?pt0}!w>TxX;h^vP|8Z^r_Jl*#2tk-mtg|@5W;;Q8G(~(EGY29ICK~uX|17WdJ((xHER&QwFMFyEW()U z2*N?VmvOcZtPiv+OE@c1!w`|YK7#5RBf~L8s#Hm{?-csjG;|p5BvMo(Kqp?y2oeym zZbs0miObp7mkHtv(@FqGSt(`?p`!>YptP7Nf<(g>rBBMukO|Qec;i&s@4u4Ev{3H7 zBm8Tl@AAsX<}DCgPcD=t7r|4ShmD~Zl;mgaNjgXux{%w6MDB!?tOE|EYdEYR2aV26 zRWWLY_9D#o37asw#UCD$DQ=GB+9K8DnGb8#z(?%TKWk4au&1_Vf9O4v9rC+W`*VM( z27b!Y7|J9`bLN*p)Z)T%T+F)khpV~vw{U08L~-}IRe#@gx8~j*tXY5+@e7b&nT_El zt)S1U)viI8Z?2HXSq+gNm18`252p-M|GuKLoxgl#H zQw>x{^pL&LJx264R7#9I}}c62^;H zNRbnx(-=kNty4tgx@#boAYU2G zR2sNRu&m9w(a8zk21I0iU=d>dqKKUp47>b5r zDW*cqWwbJ$*pSBUrA=6PvYL{BxOh&}52m%XF!qvJ_n8+05g)07SZn$A*=07$gox0b z2g+!ES|!^D&BX7Pk7d8!GJaFgSTE`0d#TkZ1tF98-{Gk>uI*T_so@6TiGd)^(wNCsgDNg$JQ|pX4kgAowtOnk8_LcCM>S^+l;c2M> z?Y&*m)+^jcSMqvBFh~A_IGm5I65DIxb{1)-ErL%24CCIu;1CEJ=Cz*i-j%nBihQbb z2o&WR|D0*~kAv`%;@L~2gRB~Rt?moGZA8u0sfR-)uEKqBO7n``v33Rn>3mS1@`XG7 zTh>`sY8keTW`}!qDZEL(&~PC z4kr*Il<30_nuaKHa4vmeV;?lz=K#HtD{i%~qNU3V4K4VrX=nxQX<~(v9EofoMe_lj z$C}#~!1Kgo|FrUcX#TjmZJmC~S2Mcd6r81pe?iZs`Z!ivuF6lr6l&;bM(KS}|BFvm zf#h*U{*Xj1B3xjtB-=`w<#XFx-*ls+=KwIr2=tYzkpVaN@-6eSmPl!QjhW=#zJRCLSDt&mW3RHL0GlV|q ztAL=|#guRRRL%Zx(3SNabh)^`w|M<0>Qbn2=*@DRBlq4f+EkL#+oh_tNm%dscfMWJ zcyO!U#`?=phDZ>+Ma+xUE!?NYLAfa|T+R@6H)*5?|8GASuIk1DOUqIc$NnFQmGh!3 z8%w=i7&b^=mCHwawG5~CmR8pCHf7Znm3j{2hK}!`%PJcCe6YAYPhIaj=(1`^^*Vg~ z4!X)$$-U2uY%OOh24++;)_dm9Ym_nqZ{5u=28%@tx4V|u?8T#3+&MT@Nm)SaV4ke# zIaK%iY;3Huy4B@~3}aJ&ISADVJ{48w9NYu2uYzZX$PcW#a^F0n|PpNe#z>Wi*`0Nn^|?M16@w=)}nuG*$XdKF}IWXh%d# zB}~wo^0(ZP6;$j?+^0sqU(h2BoNq8dQ_rtSdn7=+a{`O#N;QSYI5l*&pI!y9pr@)_ zp0Uq$wdhv@%+EVy`1S zq-n8j8?%*Y%UEjGk+YU)8?)E0zFSwShFedm=y!{+XxGR{gqE;q$Bg3Vl+ybG1(AXX zxPKG5x@$`HVtr~GNB+^itnS4c(=m=rA$SzLg@ALvpRExGp-{O9=Om?D$?JE7#4Iab zQmNR7aUi(b zSQC~X?5&MBGGHz-tn(o>AZt>|6&r%g?AA~h;QnvuI5B5*xud!1QvwWy62J}B z!U$U(+Fz<+d3iMBkspzD=z>x9Ak4VnHk_a#9!@D~pLs0X2~=-+X~Yq)|6fp=rxW|7 zBZ>P+CVuO`cuiUTZQVwY?GZii%d))7rn1Y~F5cnhUHo{f$IqCTaukzCo zbZpF^v+3}|^EFW?=yZlDvKKGp^k3Zh=V79!opjbdw@qxo)kK=_U0bm$0Bf@OAq9dr z)=akKv)Pa}TS^A_?6ZHXlfmrMc9;hXP_zB_HXXX-!EWq1)Q%`~GqKj%LTG%uk-_}g z@+NRS++brL!FV-vZ!`Rmw5|qqE4zLsz|}qEY&-lgur2R_G2Q;@{15u?ZRcNRpR=G4 zp4$oStK`j^5oi11hsqXw{hUY|}kea-QdmYh#FLBG7UXZdmXVPsji!PC-q#`(4LUA~LW0QkfBtv7sDhzaIJ`-A zbG9{xIJ@p@H#D^}x^v|AI0bZqo?g0etoEH z#W(3wm(ePCar6xvYl+e7X7T0CgQVS1y;QONuBD#uP<>tT+s=iSI;veJC!TldQb(&% z`hW||i{Rx;{my2S`xWd|HnBOPe7S4-%iO}widGFIk$s(3;MLl#8=!N*3ZC2QdLmeY za*z-xO_L7ZoIQ3JSXT0TXB7hRn0N?M=m*>O;Te06GfHQ;c&8jdB4N1)iX~=9JqZ~A z$nuYu5w{h4y;wWcphF5wjkxPZu95lv7yG+o9^xW%1imlK!}ZAp7uXN5P(|;+!gQ0O zf)P59(Gm>ckD+$bwu-?xaiSLd$RpCH1NTEF^@P&*0muYOysXh@{s76W!@N0q-LLzT zw;Zc}yOiSbiTTIirU%)+4J9ATei+Z@)w&-U4BO^)^zf>C=fbWZdz}~1Lj&<^<<8Tg zafvAq23!bWBd3bJidme+lo{YIW58=6wQnQEb8Mrk^7VSOF$k?8^TEX*?- z=5Y!1QLQ7`EZCE$3v4TJyYx3QBYd^)u389oY0H&xl&XP?>q3Q688pL{e6XiH%*uy# zx~H8h02N9Y{{|m%VPGn-f^M{TaUWX*q*AeFf|gCG^taST`h+`xwv~NPccR1Jn9!U!k{#_q#RJSc=3SNQzZU*JdK-*j zbOe7;_^ly7PBm2I&Y4!XES`SvbH+gw3S)^s-gX4~KXO_aMS-&XsO|!S7zGLTq1oQ5 zL$Af_v$hPaNwM#fR z_VSaZex79STrm=VrO}dOei$=e^SDjC*qb58zp=LNV zN$v9EZ)XzCdLyT|WDDKPm47zVPr5@}ZMiA>;A^Y;NVt~U|Gp&3I$H-Ul`HTH(mV& zAhjd&jJMm{Tmr^%UGZzu}itj4#vQvuk`w{=~2j91z#JeN0+Veivt?#tkO2YdI@>t5oRpb1FF@)u7Rlt1* z?eqAE-(j^*;;uknTHb+BiEL{y)zr;&VduaxL=j!e0DST-j>FvaM`SoVENr;rEg=tu z>xtFX24n(roUFfzgrr+n4b#_E`m+{GC_=Y+^ELnYb=1ARK&W37DWSFG$mdw&T5&f2 zihE)3+85FdviT}^pMXge!%YqC$H3{}f&T0Do2C1}mx#?VH~IS0$M@cRj-0EHn2O(! zCDofC67BEXg^A382{ao>^iT zRpOPUl>Op6!$qaPrAeDwm@G3lMqzchV+pyb5zu5)h>wGEA4k5=Wc(ULz7Mfnne?Cc zmWOCfPGR=kJ3KHqL*rpp@`m`j-^gr@qtTU#G-4+c0#mwJ9+r$W7U}brny(;uO!OfwP%cGGmh=GjsI1hE5qe-ds7B@Gu7}BbC3b&PR&oN93AQ6CUKZ7;wJh+K2NVy(PApJX}`UenCU0J=;eQ$B3?P#FI{M z$0CIHmqxcj9c^rcFWpxR=$9=IBbq|yBjLE~LWYz9NTFl33ceSK#P0_ZSH<>X^TCbs z0#6gMeBc?;Y<>jG>mQEN3d{~2admci4GJ2f30Lqzh)6ekBaGUV>R09#+Q3a<*IppH z#>~ga2TmgkbTQWiIMEuMK!YjH@C%RpjBfaBfVsDicH|rv>p6NZ*;amd*2GsxHB6~72*%qYE6*s8e1(sM zNvyRzbI4)a#K1|};pv5e0Tp;(I3i~oIP-l-Rh|G_^xf?=lb3XZfSiB7%+@}88x?}$ zmZE>*USo3&oOrN~Rey5}`sYkJZu#quYB+8Ro0Bme)6^n<^f4SHJgnC9%(Ni>`p^X4 zzj3sWKWlG{q-w??%J8`Kip7qS{hKmi<|r9BNrF_N7(TB> zh0S_P=KBHMfZjl1;JF@*fN@S}j+A}>^rpKJrz8-RJo#hMVCz#8d%8Z-sy|svOjQ!Mo2@k zydl>p==a|4cPhdIynb{@`#-cd19RBpB%v>&(Zv{{zZ{V7YN2nwA?^CT*uJpyyn}4W zx8ouUcPJISgER_h5yUAu5!=Z*V>!vlLtn>~!kZG_>?2kKHM6lA`C*NU%kRgx&gmkU zFUp9yh^8+M33k@1b{0dlnj)_L>I3dC&M5dVsJ@ac=C<;E50u@;-&=Ez zU#*~i&9v$h^)*pg zO=k+UBDqF>vZsiMju=mZ6|Zba_l^V;9-O@2HnW5Eu^NC_-JcT;7kRtblRGZ*Ue~Am z(!eNOAXy?$?JXf1akhYLUfwN^o4Qnv{5_Q0nMCllMOXp%BcE0RVu@tVAB_wVx=GTM zBNS=YBs&OC) zCVZJT{E~?p8i^#4$BRbz?|{}_SlaKzPICxBhC7J%J5i22$T#cG)OU7-U`=F!k#vS} zCdwEY*p83ME9TGkcrb;u?VHgRLXs^<%fJpXKP$gaT^vw(1rj$6tis1&UD zU;C+KIzyxq&`P42OCt%Q8b>9GZedER262JN2Abd>F3f7^yFnJ`(+zhys_Xri?PCIb z%L>N|5(3YS>ZlDC;coHf43!5XOMojK#0f(=%iWrRcd~`lqMIchoQdCPA|&gk6&A=Y zk}h#6ZGBJqy}_BviK}4a_2U=xQyt_7Ck8C z%n`SyxD)x&$OaKw8xOO7&(+aK(Z)xxTi@4q{(viUUx1=F!%szoBfs_a- z>R6)z*yeiVOqbj(l<)YJ?K9Q0+AZBY<}{1$_hYs}R$`ikkZ;`O;r}Xk=AVZ9AA_2M z=FoG-N{Zd<_p}?E2J;1&&l6o$>N_A?Lmbb|L}?jWI(0TzwwmSRv{2HOoUh>3*C*fy zZ!vvyF>O2cd)H^TeQo!IVlSY5N)`1}+)0$L6KlXf>;t zIB-CiHi(#=pi<-4)&j}}<{QI{lZYl0Q$?Ng%|oRtb)TRQ?otY_EFvKS|9UNp2KTwc z@s0wD6_m`m`dP)Bp8g_fRr#D|bhw5pGL`;tSeYuM8KL}Oxu+m4EvRz^rmtX!koxsd zwLf|iJ^{fR%7?;0U4*IL{H;OH%+t;};%{3%L6EFE)ZS_Y@RZs-y4QYYAeNv)* za#FImB^eg{rN`Vx!tY`!*n49rP}jV17ZD#y@-2uJ_U)Nn-}P8H zsk-5{a?ymEas9zk8j9nB#9Rsuvq=kKlz|7v9F2^bB7{81hk4j_wW_ik)&s40^|-?i z@T_g?XEtuvQQ|iq<{lMM@tk}CHCaa6zyNcsMhf7B79p8(-*VrY$avK?0L6D?kR@Vio7mxZ%hQ;9C{`3^=Cl60Zq3)+^{4 z2cc(PkR~%lJq1I*LFo}#-*~86g4@ATQ;24MI zbm|NtKzXr0AQ%6dfXKFLT_lRKPb=x0`0I!J0shEl4}{?Pm{>Mw7vzD)w`+oUX5WC! ziVAGQv&!+mlpS^>WBSDNgbrojK=7~-E#vv$#^kfhe)OVg9gBm(!Rc* ze~e82NA=W#0hd@p{bU8Vu|YKpA_zj#LRTx3x}JY-thbSc>1qShJrpN<9JS9@|l z`w;z#@b>UiuDE!-cdngHKb|Z2{%mjI^dpa?+~P;P#B2BMf}nh1^Bs7)LRmq- zeAM9he1duJUQ_;r@xC|U~-wCSdT!ux2!0|kZQmo*85b{IYOE4UCFWw(4MCV5}rpJH%RevUY) zoT%A)cj)F|A9c_1P#$Qzg8m1(2F@49mH!lEG7F{!RC65=&kVHFCDw5WzjfaxiD+sBTo?@b$;&J)m8>+wKEArim3DZvKY znO)vVM%I^-OE>)koNY;gVTSPF5ZUUB`-&0FP#q;eWRVJpBA0{~i7^>{MIE)HIY2>V z5ekT+9PLI}O(eaxfxHXEii3rqW;!T~he+RqJBW{m8r8Tu)&mRQ?u+9ii_20OWydoy zhfH|aoQl(DG8w?fGjR{pK^xC{i;L_U8G34@8|DK9MSccFR@p1jF@#5S^$ukp8jX_; z{Z?`Ls?5V5KN)5{xW=vbC&g|7NuS5i%7x)F;S+-A6{CX3G{Bei%OR_uyfOp_}j zj!C*IP1(w-O2I01QT1j&_#i}+X?Knk0*hhX=dRbbyuG}@qVMi4@)DJ1B~~5!g59bd zMk;cAjCFo=#@!hqqOZtVs(*vk@f3+oA;jCv7ZD*fM{HgR(k2+B*Bf^2oRN;iZnRO_ znG3FJT;c3r*<%=eMbR*jVzhI-hp}?x`#d#@U5GDTx-H5>Dx^v6ZcUz>wk+CD!on7w zQeQDj3DTn5!y?}2b}MSC+y|!6#~jbeXi;A>HHJ;lNC#?bQHEfz=i+P$lodB5vEh(s z+YiXeVJT_-l%C9Q6hW+6LUgZt z7~k#%cDpU;2pt_vWgF5TbE7O{P%AwS_j|Vnq^&h1_HTOcgyv7i5|`VET87A*wxNuR z+xcv#o1S4gjRaHX^n-V&ys7{%LH?r1yu@@&z}=!zUvoFqA6AYqP^wD}qyBy)UpP*1 z1r2BVu=l%Zro^rBd#u^b)NFrr_oKoPdx^j*#N&g=IdmP}BfPee9~Ra^kj*@=Zu`~3 zh3ab%j(wo`+@rrg%7{>1NJjh-4@Pqii_%_{5d0rg)yCoB^J~)xqs0)6Zy!W^g+2de zhzJ=${tAW2zx^E|?JrdB@chSU-9jZ}aE?VOB6yct5|&YTJ9Jb=n=)-ryFYTq74#EJ zBHC_uZ&yslV#O#=IGjeuwsht>o9LxtT?;hC43N2)XW|LD2Q8=^-)QJ z+L1EN>6|)abTU8)QYr zAo1E8?}O^ZJ9qps4qVRME)KfWKkW@IGznCaTV?)`1whFN4(h9)OUK6-fh_oV`|PT| zG8Y~RGWz!A1=pI0GX<{MYan7r5fL#oOBq;GZRD$A>0^F+vm=;>b4c%n3U&EXK zp*-5J)6g_ff6~XC3p@8*O18 z@w+ABFfig-{=y_se|8QYj}RXg#3LG9!E_>6K8_z74A7qN$>qQApCTXZ^{!?l35hf# z2eZrD(KrezizMl8b1>B5K+XS~P#-v8S0=k`X8m>x<3<&y#e$b){GzwuZ+eul`SY?{ z=L92)JE;{~=HdVivRPb&l1>lzC)y9g)P@#50(IO}Gc@DWps9aQ)(!-Q6CB=%ldlZosC+=z z*_(fmb;Y;zT9 z@Oq!tF%piv7bJplyt%KyazOLI%6goiOe#!7+{AhOT`_;u8J2xPJk*;Zm)i+Zw8lr4 zz9C&`r=qGzo#EdQiQAU`KFt3S(2GTe6EXd1NQMgXVBjx-liLBhPNrGWF4As*@%idl*&$AkFR?ZBOP>Z$i~4chkp}j#%FoO=+)nZA$r06x zR@1(D0567gqKgwHv%+y;6Cw~95 z(gv^ocFp0MbNoZj2Ta##830{UKAK{!L_04PZHd(0G1MdZF&06(F(2`8zRs8r zc1bm7YI8Y3tY0Klqai!E_e?mW*4#Q6VZ&&9K7}Hr%FFf1^y3r3T&lcyGW17%B~TN} zR^rQ(zk(uVJ<+=$IBd_7%*-?!p~Qxxw289he-$PoSjwoZ+08BYK%AmK@~_46rv-Xii&Xmi)$7j%DOTDY|qABnMk_-D(t|S(?zswAkDIobZ0GOWGZB2 zD^x}}RS|h36KU@t%`%gG{`UPUWMnS%!kgD}=_K3qm}lML4BJwERn&WhjZvT*!)eniulGB=SZh!aW{G zWG5NRRM>!c8XxjNCGv(R@}^bLi95#`jF%k4TDXsR3K8VX)H{N=V~luH2L;Hajhdqj zaS0^NvXIHcD%UV`B3B?Echgq&P2Y&z}MXVljue%X>pCqc~d*lA_ z-p4Y<@tXnf{dQ{jtTT#s+~*oiq+}BV!N=hfX8&Lz$qM!~Fn~}L!cwm$n<*Gycsh=i z2fPohFiY)>nt!p?^l+L|J~6!ix)@I@#0RHN|9#k-Yc8Gcjq7$W%T`vub)Np4y2SFI z_2BMlcWQ{j`Kg!POCuXhVehstC_>{8WWPpANEow*W2KW99H9%!)J*Rj^PqXrJX&qQCC8N_j&nOp-|n zLvx7y6(-lP%-S_JVMXP6<8W9l%vdOfplD`=3QLEgIZI4ds7BPlvc^kgmCU;hOIQs3 zpK;GEyv`IX&BQ2J^H>o(eBm#WK{wNeu*K6B@vvwudPOpoe>l$W5{Ecs{VaMLCqPcB~e@f^e|IL8Rb!4;`P!;YsVw0-XP;3Q>Y@LKTp5${lCL1d~h2E z&@WBdxd?}Jfo_~^Y4r>lFJ-E)1Xnz@Dx`LS4+!2N!l0a5`~>pe zrBUKduzxx`K?<-1*CT&rbM{u94gQhZ6`B`x2HYi3)d(E2|C6C+Ouu6&?yS1gjtfPH z1iqMzI!ZPnSw_F6nh8y zR~)wR1%|&FGIc)d#5xn`$`cJ&4c$^GGC2o*IMtm- zH6@>>LVLUE6$r@6rp2#$j@RE-gv~yJ~A)LJ@xU| zcjy&IH^0Rq?bl3dkjsX@!sk}*g}O`BMFUKN?7Qed8?1T;!h^@0S5pww{hF*?Fdb>0 zaUa_Lk!o|JOHx#Q2|@5xa&77D&1{(H=;Vd?uHdO`18kj&K5E5H*7H;Iw5~2|6W12Qve~--ITVQL0@8a7QoBW66a&i3%L&=81vDbAbv%QafcY?tH=5g8DY?jv zQ#d9=R5OOe;__p*6At(D&rmnp(sQDcTrig@LnN%*s;6P2`}un1qp?0ImAf00N929t z=t85uE`#)jc8Hu;{RZ$DHZnckk+lmD+AeWg_TqVX+aj%kPFBEsc<3m$K#aKg-aM)g z*)A($u3x^=i`~JH%V1& zc*pTQK7w!b+P*DY)c@)0^&+yCpCvdKP1L`z^li@c&Sc~JqItL}A{|^4vN_=1b^49w zZ}4i7{d*jF@Xk+t?3sZ@A7&3n`<((1e(JnJJeTz zzs;io7!%Q6z2AQQmw!E{$%Lyb_+MGu7mjD*hj_7GWv@r>DRW{OT<-0=CsC-x)U^qY5^Isn$ zdJA_<)+4_PqN{;Q`HDX^Y@U%7z$IcY#|A?HUAT5T;r?l)Gn}?#y*O;{YWQ+ttqSoUnkSZ!b zjhW^`y9ioELCVvO^_@)1nv5T(8~Y9?+gACaFK_2XzX_mC&58Ybcz&cSf7?W@dThB6 zL2$_*!*+P}s7h)1AUr8piDU_;NM?vKYMtfgRD9*#o(6_d!L$EFKV#=%7C5@tk_!mp zFXa>YmlA!`=pJ2oVKq5-cqTP^ahQG7N9AcR&a0}?1vnk;f=MOPpu0b`w#|s z>jVWRC~W_{%S{koG)W|0m8y?nxTGk6)Zzs#i5f^0=lV3@ZU4-zkLdA)y@N<~0U=n( zwcfE*w*>f-g)MmM(L0$|L)6R#Ff+ecw{gz+0-cyJ*uGwE(YJD-l zVv1h$0K3wapCGARqBk1YubL#1DraLx)w$N9t+Bhe<7TS4usO4bn;h}E8S`L^Q{6fc zGFlRGg~h7Ox!Z=Ln&Yp9k)4`E`zw%#XIO##3hmJ5hehr9=hzUm(-b|^OocF73;@5- zBUQ01wk-WFJdUp({7Zxc`OO`Ke{vE){aS#g-)dhvL?4r)@_>=5X<=Vy5{}Eh__u;i|x1tZ&5UxXC*Q0;H2?5~Km+-k1Y1!U#{ktN^PpVl{`bo`rX$hoxEx#Vu7p-JIB zBugJ3*&`?m=M@n2;6k1rMOVJ{j?=0Yp2ePb7nt2P8;c~rmd{73FM0K?>BF1|2K_Hj zZQu?dq|MsG`YG@B5$matIJL*<1a~j1?aPW{Yqsxtoa99$y=tw-ugT(?oS~<2@$NhY z#-_(+cDN`k;DeiF{c`-@CxoIKzr9E3144UNqOYLooQBq~FEG5w36?u}Oe(SPk=UTM zABS%oH}bc7tV<{9hBLiIBS)LE^3@9@(tkI0z%iApXb&z1aa1-rSk#}x>*c5dgXb1_s&XeD+5r{vQ0TNk)oA5RUVnJ~|o_^|(ogMT+ zCeVw^X)E%-Z(K>-e(+if)`naS0NLs+RkVXNn)Mz}Fgs*On~Rub8Wax91}q<1ah2-_dnV zTRjSHzBRKYv@h-Wyw|O_@A*dPk@unx)k$Hx%~eC`BW|$Tm2*$- z)9b+BXrJ!rf9~(eRnM>MW&Mn2_mLddK0jNiB|`J@1Xl9*L+ge9{2QZn`OK1iYPRk+ zvRBG*k}SxJ@i2y4(VYMhYwUCXrgZZFh$B!J?yeD{40$m+C#?iA%5Ib!$}G|^qhxkw zx)ViYTFB*;u#|Vzgk>uL4gka%3av&#EIA})&XKN<0z>EY5U2q)wQ$y%ewMu#JGb`xPhKIcSq92 zdqjuP-62aa#?0yu*Fs7kLPRX993Kd5Yn06{$lcW%?R-vK*?=Q3Hj7wr&bolmmt@&h zW+qdV^ofq(myjQl+@esn(I0(8zbqKjn0{Uj&|&!EUE&VD8T{j=(u@;nu0*p*Cx3s* zG)3*nxJb2{Wr|80jdB0e7S{t-hRf9N`dod8DMI)O=P_TlW(K8e`8YM6{w~7 zOnPhrP?1vB4Ba@2EkH5X<{+5}^04lVyNW|S;31yS6a#w$;2zI`p->pICMmgoEt_2LZ#`RP&T zecM-qB(FFwYXwv9tJF(Fc8_Ms@365a*`TP&+Ruc)S^e=-KN&T#=j6hOI{k^;%Vm*O z?jL%U-GA)-IzY?rx)Y3<%o0>sh@tu)06{>$zxT!Dlv}Je8U(0>aFF-+;!;@{Hk=j{ zObS-p%_UDzRk-XdSA56u=N!gGsy3oQICLc(3X0^9_Q#{o)JIyeU@ue>UeFA`9SX+u z3!72t%q(!pM;hbV5OB9>f22bG?)`BO#yi&M={RsjE)4_j3=X8emX0Kkwuu>5!YO!>#9t=07}FKz-ff|YgVMPEAu;t^rJ)C;VL~iuNR$n zn6eqY_aGqU_Q&l)J+@rT-Z|$7oE@1)RN$%*;!07X^B!HU>edsv#e-@#h^o-++^ewM zy`X}h%MgG&=G)|Ve^7_dt_)vKU>teNAs$0J4^$eUFHc#82YqYg+o@}eF7ik|ZOrHX zdvxxxUE!tGmy#&sfH@{lMi#2;5BfA^RSSJO$x39tr$1{=l=P+V8N#rdOpld&S2CYgLEtu2Aa1ok5*yl5h-o+(>P%jP|X=4m2DPIIie@-P>2 zWn*NYJ1z%qg?j3*apEM8PnoRQ1mOrxVlC&18+qiF=)%>2=w1g^6rU7)-I-}F?N4J4 zSpBUZX=XGo^p)^j&C5gtTib7Aqxng_i5+z*&wuudGO=JcVU#8!kLM^Jm%tHd@LuPnqTsvY*{sNEr)VcEI=&(FZe^)@ z&&op=yTe+uN0W?P%E7tf8Z0SZ8i@ojtkE ze!4nyZzrESUkp6M_eSHSYUV+jQH7>l;D~_;FC=+=4##v4&G=bZkb@=yfyh8kyB^mQ zP}7owZZ2se5eDjJhP|cVnM<~snDZj5wvron0}jOq=PnlpgfW(8B*)E(LFClQxa( z;qyUWFmO{2LdQTEIOseQh5D=A(k-V=O0bXReghFdCtm5T z$gzn3{nw#%Q2bw|pLCrbnRLxnjC33OFFytAD(_f%#{HL$;^_0F@woqT^sn51sqr5O zfVqTpuH;VQNjqC1&X$P&$CAY1j}`i^5T!1`z7|u|p$LKSqZ0z-3Ngv+Z~n&$0Va!J zo@@&yOAiC~`Y}7ei3B&PEjUpRC&IrRfSX8gecOVYsD}g1@K`RuO(VEEOB-<0^l*^4 zDGmPSqt@skPqQQcKb8i$$^OrIeg^Y%I6oQk)BT_0K^ytb`;ovm*|@-!Tmb67$>RU~ zeGPEJf)vchJxw(2$nqaOQLFRs(!jq{H$COB;V<14W%xnPT+&_?IU8+LZI-{wj-lw_ zrH4(X&*Ea0xdmCC{=7hr?Bx_cD7H_o+^-lsHyOMu(csNm4#xT^V!zan`&j4*ij5s( z`LDS&CDj{e`Pa(wa$K0Grsn1C1-K8O?xeuF)%MvQ#onWnCGWfw-E7B_Jtl3oX_bmSjN{Soa%xnXW;#!QyonXi9CVFs4*NgAZ5Hf@@@|6@8xDopc4Nl zcuR@X|Hezya$XX91^z_)u}Yz!BksklmP`I(GC?{9rzplGWp0ok=8U+4u9)!|fX2y_ zIWH-amn`4JdC8HyB>C#UAQDu~pvYt2$m2zk$4-$)d*pH0B7XdgoRIq$YcJ-!qP(zY z!HZXJ6HM~9yaWtTIHU~xaRv1fBac9rSHKQIywXKZeH6-+e&m} z>`Hj7`Xf%y--~=(FcjHkf(2j8OCaGWJ|+0ND0^h9<@k5;?=9$DjY*f^n4%1>C(tE( z4Y{h%oWx<-BJa4<#9np816p}8d((^9o6Zr`En9d#pVw>MxR+Y9Wt}e}TQ;dz^m?si zhIfOw9p1jQn;J8N^D%6)$)s)*iaviGwO`7n&R~#lVddstzL=WZ-;_X9w zh8$(V*=}fld9;7{QxC%Px}K_kC?+|i^>v>;q4yCD^Fied)3s;xe7#7UNb15*7~M;K zd!-D!;gaa~{uBiiJ>Pw3+t1E-U&*!3xkR1se)Sj6ci$5|-#wBxlS!N8X-`E@c(2$V znO@qlFDgzC0ygr(rROx^-Q2y632$Y7<22T5gL;RQ>1Q`uj8ES*-v4v4LJf8ElF#ZW zUV{3|?Z2v8pW65|~al|>pfT$i~H$CJ`cgCKorB)t{W7e>-}CaErV9kB`B zazIWi-Fz_qR_y~q_RNDcog;gs!w$MR4`+~)c|OR_w?N%>=W>d3)}!3zbNqrrByGiR zDz`i2N0mnlM&8XmS=vHl5}v-eL|_*fY}j|D+s%IlWoVkZ$Soz%8a%B+FPauln|g`u zZ~@zc3)oMJ%AE+s$bRAFMI+Pj@Oj|+usA9dHkls1S_I!Uw%(oH@W@5pSUg7#UZwo_ zZ0*?kwF~>8xQDBh)72Bw;Z^KO+T#iy=q7ECt0P?D%!UYc?htkHzwUOr(8^d)-3P#1 zDe>^c2+aWp8){bH`?K&}ZzL0VZ=Kj?K98qY2A&rE*_mYy{|pm)vylxy(d1OupSSAv zFHTS8`G34)LEE6Xpo|=&z_>IyDHIMv>5;{s69+@rC7KVpDal{_8e|l7116a=*u>Cv z9o|N`)FFe^j3sZGw6hqkwSORm*T41cqlyRPG%gtsc~c2bvgCQE2@{usNd%2|5RE=g zshG-25;fjI(MEswUcl?$E%BYg75xmoUS;@0 zDXrSaYoBXj9}Vz#GyJwhx$za@O^+4Y~{4_(j#eq-fX^`GJ2Z3+L+vx4u- z@H1P&KYUj3d(GVcmhjh~75u9Vf2dm&d|)B&Kazfl44+Jb?{K&O^)T>#hd89As`=z3ELQ1{q>ptKL zV6-4NeMsIuE}n?#@(UA8rr{r2an8$J0&h&i|A&$NBbQ>_zZ88lbFyIW=Co#+likVY zZXT;>PIg!@XClrN$aNFwz-Y-FDEnEm5@4n5NWv*;@>K2#)0foInw8q9CqCO&#smmuH@VKE|| zxxoeQiOmC;tM>^_StbTL+r`+GZYgf1n4MVA$t^ioCW+ZMB>85AmpQW6&Du_LUf7l1 zN>^UT7GAx(O=MyK8F}y-dh>b+x*sm^0=6{t!vNdB4Kek1W`+SHqN;*c=?1&`nJR}l zIc_Bp8J;{VnauF$T*-v4yOZwwl!y;yQR#YpSI8pP*d+3SBK$@$xb*!y(_K)7==MyG zS~Iqfo8VBd8}ni~vsE|W{z+GwZ@;56&9_?|lP4>_$aI`Z*`jQ)zZW;y`>Ye$eZ`5_ zvL$c7_15Lb>%bRuN=KliqdXMrkSfdNKF(N(mlDH^%5c-(T4tBn- z>(fSq*uLA;#kkX>4$5gKgLTuL_$V!}j5X=K z1)d+L3A6FaPBaKXaaBz^A%}sgQtFJ&eN42>F3Y|DOPjywctd=*E*6t>c)HWnu}BGg%0dTjeKD_EzjeIHaQ+T z0Lnv#=q(?&4R7MDXS%!ACW%ru3dzgy#@(^n#$j4Su5n=;P)m`pvSnC|+A{33<~zcb zuI!)8k3S3|P~V?671wGdAo#%<4RZ@XL?7h3&@ENqz)sWmHflT}iX$DW#c^~hzFA7` zv8j)VBvr1WMT{`k|ybY)H0PiVM>en0kFlpAx*Z;?4T8jI6>u>T(F1x-0b-7+0 zNF3WllgU1!vLH^{RewqLu7d7}b~2zXZ3M&#W%G3#@q{~r(MC@}<$X183ln?^^@)d| zI27aS>rsO7((UHb&LQOhoPL<4N<)SwFJ5V#Zc1iI0=X;mlXyx7Kj~9iR+#AU7cC-s zHivyz;Xb7P@F@5$a8wmNW8!a3^X}&1G$HHoFT(8lftefBM~g#Rhz|gJA%Z4y`3_Vb z_dL6c3$skVWVdu<65w$asPDR0|6(#NSMHBg&y&*p=+@}@(Yf)2^eQ^`b%*{4rZPW@ z=L&90;;ntr1?oo86t_S#r_-&r@S-LYUmh9g@*8mdp!TcD15~b-qF4jZk3%l> zb>ZD}4&AkG5n~}q{2BY}v!vo@*jCv~R~9DtDIagK%K+}W6t|jnt16BrXn)=={xlI0 zcZ~Cs85bl0pkNyaP7HSICL|l)2PQxGXI7UsLti87f62-M{W4s3Q+9{Q%V(3a2OqDC zdMzh!iv)YHSvg3dtx40YRWi;s*rSR%tvODdmAX)!80CpfZJC2L$8Y!3V&H~?PJH+V7#7bCRP#ZbouA_7k0vlwzfqT3t6DoQL-_H=Y@1dKx;l0+ zFm#q?sbUoK@nE^SegAUc? z8xPeeyDn*I_g>y47wKWt{OWx%y#FnZg&DMJVq8yC-H12!7oH^4G~lWZi0%^tg#0soU_+(PH>KfLlyl#6*0LcR1iBdlVQbx5#SsEcuBIghNXG zbd6#lYUJO%!4sbYJ<|2LQNXgG0e1^Og{GmT41)%gj63*b5p8uuAIEOo&(VrLY*JP* z6n!|uMC1NhOk4VJOzZlvOGiP`hpYbZ) zOe5-&rc|>!^5M=U08)RgC{37%y0f!-O|f=kB8vpie9~&*gwuHCFi6D>6GeYsu6%z7 z;Y=!y_PdWK_nN#vF!uYs=>1)?8sp_XK&r3kO2XwhV+x_5ouW!WvbeAi0_HCXI0ag_83H=``r~1^jc{|!(#N44+?1Qr+Hv!oIUIPb=MI5BmA^3;Yi7!eQ*>Lm+9acvF zJsSN1E40yn>L2RpKf@dh1HVu=@Zh+rV{S8!xvi}+x3x9qoxAdw+s<;#ZH6%iYU6+t zrC(yx&YomQ76!#A1B;Wos``4QgQ3Xlqsv1(u1-1JxvEH$2>XNsL0y ziJFbk^_IZYcwQUEDqO4NVRUuoVYKiU1bA}*b1e;JncII&{bFjQ{uQIWO4D46R_)c7 z&6glwcRba{6n&syvsbjl%*IOhwyyCV)mZ7AcNAT=SZHz%#`ZMH;@=^f_?WE=*s11N zvQr`aLp~qw8ch;7zGTp@0J?pVsv1r9MwXkmcW7)cW{R)7L2J2RMmey^+ zKIp&yhC+}36XI|Gd$i~Af_DVf<5Mi1uJ32K|Bi@k%iNA@!xUcpHRJSfJz|J2E{|z7P%KD*$S=P>(>R03^OU3L+P3R~ ztiQ{b7nLaa(nk5ilD49_$bWWqn#-n60LziX^r;^=^+D8bKX!?>eS;+a&gBY!Y+==d zd#ttH*T}@d?s!UGJFDizgE(90yh$VU&Q8IcH;L7NytNRMxW{@|cixDaTYOT8C>!%L zx`qw-8`ewy`xA|2=y;!&>%+9<`OX8;Gq?u$hZufo3wQ!_z6}8yCTYxX7G*-D1pH3_0&(&ijy4hB;-Bvkr6C zLCzk`*#kL^nA7Mkt`iDg$2z4N$bi~!OvV?PF5OT82I7y99BdT*f6T#-Nci6W$BA)l!azUtiFzrl4a*6JF4k}}@JmmRsTlS*BWX?5)tsIH5(y57g) zw{u-fw7NcPqKqk8ZxvVf09GDp3;86lNCPV`qQvv=?pEphYUf>S= z>xH&EU@pm~d7KF5&m3(P^PS^}Fjttk-kV=QsCB?BA5u$A^2iso4}ts_%HlbJK>TCh zkfyOC(x{_Xs}{%~KT^f`U<|kL7`Nb4FhB6OpTm4XC>rypjE_-gvlG%pdrz z$}Qs3PS&&6DXTT#+I1h$C&#@VC^dag51>lE8KxsP``YIUp3(G z`AD7QH*>J{+jaWouaKjCTdU3cqxn2nX1I7%^AfQ61;c3vbzpLgqQB@%S93Tx3Z%{q zUxF)apFC|c`QujtfLL5l%hv!?JnB8Fp-(i(O$<^(Ab*Au#iOLw%4Ku8JE+`)RPJ}J zmCNLEL#f=2ghJGL=*a!LQ%>-)8sD|JVRuBOU~&`;Af9K@;tAPXB+@y$O6&Me+dr=1L|P6OKTTYrvp^ zAdY|<4Co9b@&+dw1U0)JL{W?f8es$M&bSQiptlYpXt z>n@@zc)vKJctJpAzN+ee=e!BQ{eQpT_vQDSy!ZO3uI{d`uCA`G#uqeh9w17tlY!OY zH#)-gRJh#m32@XW@-5Y$hvfP|_le){n7((KzPB2_t>lAQn!e;4Je+gMDsouH4%5() zRNAr)NZ+9j{qT1=0=J8}eFZKS3cE0X8?cxBI|rqKruVo)hsbu$4*79%2SD11E_qE( z2H84$#brswhZp%=So{~ZN4yZ>zOXHapLD0l*C0ozRn{hf%QoZr=Z~mw@%;0nc>Z}6e!eMBpy1c!!GIHL_18dB zubOlXCjHGSmgMkOmgINX-a#ng!o$ia6c?jAx zTwnNcrPNtI@d^6!+=T$SO|_yC4LUk2JZsJlO8*Te+d^gg7mW3+DHD^M%Ry$0PZ%ba zeSd4e^0UbP`Yq&>Z`k@_yH$wN`w%yOSHQETz1*=6A0fwq@-kM`{J(~za=e9VR#9#A zRnl?VB4hlJNG#bc#!i5hO;c&Yi~Co#=;~o8 zxDKWJ@IpoPQX`sP;{kEjRO~J?Y`pTKVuVPCQ??%^kAB*vc)I-0sQFiG`SXqW`xhp1U7M?lY$qn8cGO{gjlJHvYKBE{KlS3Jc4{)#^zq&-wiMG=F}>pV=8Yc#W<1e0dZqKfrVMp9Q7zt4C7hZ+us){58DL zXZdqGf9~hc0y}$F@aHxBc|U)?!k?e;=lA?M#KH6N=RE$rpFiK`PnVND^LQl|gx{%nf?pDT znegp_-vIaxgWqQmb}~G_g6Gxnyb*q3_$`9pQur;0-+g-+9Od^A{GNc{Gw^%a6!$9p zw!rUo`27oh@4@e5_^P^3B8v`Qd)(vj8Gm_c_JrqI|ay=}e3i9q-bCBaqo{g2jFHu@Ifcu;!IQJe_d#_AjWe`IegEW8L-w2CL50Ag-|o}e!okU_B0;$o-YWCPm!mIC`PuE;EKo5BWO0X0mwssXDsp> z{b9St!!7@^35$oyy?$hZDRAkREK>O?(-(2nqeHe<> z=2uDwD#q_G_l)wduJByaQV|<9x7>4TOF-H$N=Fy=@?7G23~&sz+nDJo+a{DZOz$Ur zvDZr0Z--_r^D#U%WZjb8721p5Ds| zEze#5CIQ*EAh8}t)}S2Yh^<5JF0@x-cZ@mg|`8!p(5Wm<(~Y!eK45e6QlwB zEa-$!hJZbM?l+z*+EsKtgl_Mp1W`=B_D#T3-MS8U;FAA@_g`%K#xtg6bVpN;SoZ$H z9JMxyzm+US?7c%XhpV4;i z1wx{o71tZ<&NG{ z8YX4=BH78;#N<}!j3r-{gVZnq-5g=hRV>@(X+qKpNSjJFN4uP!dv~$mC8xmR#23yC zvvs)m?&*4ckKH1>5yYn+53w|()tfX6iS)Nmy5kbe9Ao8ALJhtUDF+A0robS&F?M^-%GV-T9JojZ3fCOFcXv`g)nplez3juAJsULBCOm2^=2q0ZJL zc4-qv?r7KqIrAYW%5(mf&+LACyS=4+^o{_;R5ayxzGL8Mx~rktro$ex)dCjvt!D{H zSc%?ZJdkAmNgSJrmdK8tdA5z`y*d!a{mHcZLKiqfx;C^KO-39@L?&;QxW6$|Ck@A+ zDJnz>uJH?*U9K|Z5l+s_cG$>bK33U`_P();R^we=v5Hl=cn0YY+8G4sWKcR|d=ikW zAOCJpI^T^30I|v@tdCd@OQFliYY18?E$)LhA2-=|uSa zhw{07k{3QVDxVuB`65|OIKtuMZvZa0h>Ai4%43liYBtU6S4@rwx>C(ZNKQHdw7>Z8{uyng9UF5w`Z1JjcY4HjI)Lzj*+ulvS zM^4A}fd~M^Xp5Vb1RW&uRFc1IS<7u%ytXj*35!G~MAL8q5Mw>d+GuRTawG66&40<3 z^4K-^(IFYn8n=92fbCY-0?yZXHrT{uD`Fc9v04E+c-CaeBMDjN5Dd-eYC~k5?%9wT zOt!1FgN5D%ter>x-cOZ#A2*Hb%B9E)Xp&fq)lB#sCXPlYWl#^i#u+Nc3d4$tbErx= zXmn7Abw~^#6cYL6v3U)!s3X)7+W=c8#S5Z86bSPCDJbBSV6S3`<$kO3#M;ptK3tR=wx$ zH}a6<=ly-)PsqKN+KLa_flV)O{O)@@tSh%BKLFmn>|tsv7%gJ_VH|P6Xsa96t&5t0 z=Ot+vRZ4-bP+7~ZXYdp-t2J*&#mAw2t)DwQYc{p)v&VDu?OV%7w+0$oeC17!a%mIr zkbieRjo81sosWEeUw87gptL1;XrGuacSzcVEa>0m(lz%bu8$tEg$trb>}Z{ejSpQ> z?pf>aO5|zZKx3?D7>3X>nw*@Y~^-p6-<^_5GqRPwgK> zCO@$DlnUuvWbf6T?`EUyau5Sc)WEiElr7hwsWB?IR>Egst(V0;lFd4p9kC!1blrKV z06Mq@tpR4kYoOssbx22@qNjWZjjo_1c8O&jw@$_skk}51A$wnZKpLK0Y6Xm=`!ZWQ zp>dq~O^#(-BZ*E4*8p5!&J&1g3cNW`PTYV!)en2hqxOtvt=NS+9R7(2Fw6>;9b}l| zR0>GkbI^dqT&?{q@v-Qd3Eg$g0A1G@J7j;jZ$p_c+)G4(vmGM{AoK8p6D{Nj~1&$1WcEmZcE=dP!+a8CKr=UFyAiA~Ma?)f&(6k4A8*Ax6DNJo8V61JqEXbHMpt0J*}Nit-bM9jQUPgv#wD3-y^CisQ)QRV(#&=&w@L? zW2l#t*?in}7kGJSdP-(Cyk{e8?5w76=lxk^*l&hbOMXBSYjPv9g0`%Bz&dqC{{xVVCnCY zAO4!G85&AYiR1waoa(C+#aMVY6wW>e;y^@pxTa;D7pK~aR)A4m4sDdu0XV3KuOCYi z-fOb&qrP;Q!_(#*Hdbl&Cg(5`m>MQd^DH~tB?#-U;mK`~oDF?eOWGnTh7@f=MW@%r z1U-!ypf#s;fpA{MSOSzK%XtDnLKzP6LC=ONnwNqVeC(*NFHz~vuHdhVP>r|}E+Lb7xO+v$Te4M(O#hT$?1 zFlcj!*bO1)Qm;EDcjCaWEWa7+l^&*?`xTH_a-#5`~uP zM6RA;0-TDaAhvEgh%Mr}{|ji5vfZAg&m7cBMSfYf8(F$%=_8nqgfCt|S*L_aQ5+(A zhk4+%G@?{La=|u@iBc6!l&8GuB&I+C!HEQ@P;A~R5Y=v7lWLFd!+}X4r zrihMYA_IyE;Pys*2Y2f>VjJliW#vX$cJ}@9&2v{g2rKjpzOoAs?Y9eKzqAW06L!H1 z->sk7g^S_&Sf^e144!=<{?bqFLhg3}Bew^CwqwNGa~XBMg(P(~-hM5;3m+y&2GrfH#$l1X{I;8Rqd7;>2(now?X1H4e4Xwa% zbIPS{$Zy-GCeIS%_XwyTopVVsWDH5|@{I-Ltk6wj^Ert-*wyKp{+Sfuulg{T5cwv$ zWblB9I5)aP6e1_XW4aLW!XqR^2EyZfA<_pPRYD{Om@KS1CQiFBd;*kvx!B%<_pAp? zT13b0Hpiu6{5%_waLH~mVf{Tqm+gD{awD0NZ~0{|Y-0^5UA319TwyYpq4v@nL^OWE zPExo=G^slos|02OQApgr9M3E~QG|z?0%#5Y4EOc;K6P6_I*b#%5183uPjPF*;kl7K z)gz3wGm(q;DPRC8$fVchvp&WN`~d0|ZyiHw&tclru} zc4rZ$>AVzCX-LkET#3spAFNiEoEd1#@CD;n*p?;lsAzM5XuBDLZyON6-N_#g?V+6t zX`ATS^X=2+j!l7NOX580w>24oWCzSieMvskK z=y0aa;>3c!Xn$GCVmr*qg3^uxSfOQ@#M!fD?uuY?+cDb9&f5!825^TbA2SBp>|==^@bWKA9uvUZ zmGLRbKx=0q?sLu6#(PJ%{G_7E&G`w8UUYLG8)h*61IjnFAlra$4>(C|YY({5GE>j* z;;yVu3bYW~QaRvhweb+iX4>y>{DK{ic1>BTj&D*VxGL@+#_y^xgEzdZp0?15OYm@V3;7k?q(5>3Cxx=&MeZ#?~><}*?M?L^N91=rk*mW zS{u5bAd9tlLFLB(Q1fB`OW2R7g{kyVKX z|2L2+Zn&tFE20jHtzYJf$v4EZ0}K1Io1RMbCbeXYf>OC(EIaMin?)&D$b2=4eX-O5 z=$cuKPWl=>Wqloz+9CxY3xvuxEyxq2t!~`?j7}87Uy9N|oJp@k8{?P_7x2p-K>l8~ z$#W;|yT*fEVsaZMj<|4^{!)w$NZckyZz~o;@!N{g`HWKX8?-|=4B-vr^8eI)#J@Zw z{UAz%@K*EvqI3aIp6tVJU7TIiPcbt*kElPyJYAS8EIER)OoRv!0Dapuc!OWrG`HxP z2LEF40f2kUe+Iyb>{wlX(9tHwrUFGxEf(7yw4jPjErgjc@dxH?-E5zndOm3UyHr2v z2*Wwi8|S`WA-slKIzK-oWlERiSJHtu)T5QPE_Blv-kX>zMvwGcFbI|*|AwVkCfUpc z%XWr5)3yPX{JS`jBTDB|Oze(gmbqhoE*brlY!V$E3H$i1+=J!=V(fgV>-mMDSTBag zmfmbjH=oR52Fd;KS`ld71Kpfev96yO|3&e}e(HV!*YpYbB+2)cUmVeo$3{Mf z$-qt4-|Rj$Ly4U+NT@4MEh_%2Oe`+?S(jLLWWf+o3iz?8%yqfQM0O|?uD^`=7u6y3 zyf01l21{DSvaVYP1>??%*o8q@Hl}z0>nloAykg0eLP+Ylo9=y*rb9Q`k+7v~h=r;L z+#zXtVf&PR#GVP<6f6RwFdF2<6T}#v5x%&ZZ1rb)(OX!_%tEno6Il#5eu#q^heYd< z+}O;10mju`ue+p-!#ANqoN z@dsu*g#&Uron|?g+RGK6o}1i3xeQIl+B?Kd(60_jL0~{Lyl9F(GB|ODJCad3F_0g2 zS4ul8rL4qfLCF?G{ocZf;Sk;4#j4Abu*&bK26S{rg=?0rDeLKTZffm-%B@nz-ZnTicJhkors~Q zeDa)o@Kgkyb`ZnS;BArBD&W*0v%UyQc98P5iEV`3pP9=w+V()Pi$k3!l67LgCX z!eDHudLO;cue_HXNl%f_fLQC8YoPFOojwYCaB3ZntK@~ySu^W!O6U}$x4ZG6FKxIZ zQ##Kr{|$Qv!;vXLCjy|t6)eA3%kO3RrSnSVe7nkj&JiU))VsbA8^Mj1jtg=zawd*c z=ubP{$bs{hn+1qk%BNV$8d1WVHegIMr`Y1e+1?0zOeE0 z|DH9j$;;8St>b7*lWQEjF5Mc=v8-fL_malRKio{dER$Xl_yp~-Z_vKKwP7Skjt$1|rzt}Ep4RT~=;$*iu#OaK z)R6O_l_xLkZ4<(d*bUIlm*(Rnxd*0a#UA$tcNnIP9NM#qEpW##cFVP->t;#!!dRM5 zZVLh%2Cc;<>l<=cwxF)(5rMrxd z;{deMaWL^5_d&3i$xj424?wTaLb}@S(op-DsCDAaZX9nO<=}?=D{>34Bp?00e`vhH z8mKyD{%6XMHyA@cpU%E!j(?X!+bYI@Br@YOZL#XYWWgoPsIT<^HRR(UJ&6h&zcpEK zt{OpZI!p0g3g#yZF2ab|uw=pVW|ZB-xygb%;Vr%rt$KBbchtEL5BPUoOIL`T#?K)> zaF`EFi|OulQQ8SfGbm|R=lh3^*1)iS+Lr9Zk*0iW_a8|?ce3FAwFTr6Zq{Tw^ijC- zKyoTw>ys>y*3x2QP_kery@f|53)_Q!WuQw}+nJA4Y$Oh}Duc)GEw-1+|Q4X(PNwmjEV1LFzmiB=CIcE$ClJDecA< z9LQR*kKypz-|e)vyZ~U1XULwIENG;+@R`YiTj4Fu_A?|Lda3^4%U zO-2e^A>7hGgRXs{iQaxiX`&ofDv+XWjYTg$LkYwGF*jHx`a{O4s~A@DAxCSltOG>X zlAX%<_pEVdH`#!-<%mg)od0DTy7y?^tj7mNpzEWs>R7rI)t%B#DevyLVciT|*RwXS z0bb|A`vlLLyoK=I6e>D{7&VlA+W;x=YD}5~k+U)KJSV${v+>AwSkx>YTrO>cQ6NIp zdY8a>p}<60dED0RuV9K})UYe#lNo$E$d%_a^-WU6lJEBYV^NoR@k+TwH zX(4D4B%PT)>Ab=6LUzwvq@Z~)pE~tiCwlwf0g+2Vj_}QmmI{#>bEEZkAu=5vP9ZWC z9vMPp5;$Cg#XC?&`3#0pC zB#^#AjxI?S%zKgO^@L=>r7sad0u9f2NuyWwj+f@L^98{;^_tdOl1o@NjC}chjZ26! zcCvFhgtfcwg#Y)E>Sp=hV5m=C2H$qivRp{xFP*$-*)V}Ez~V#d;5{sQ*5t@@2aq}D zBr&459Pmnrdt|0&-@%LM^n@3}k-(a@G4){d1W#^sr>AW=)FOU%u3!{f!QoH95 zEzy=!64yn$T#*XNwFLm+<v?3LrKA5~9~tVI9g_6E0Go#azwNVL~jUc`USebn?^iLB3W> zJ)tgGbeNcsMvtfoDpXuSd^7%C;Jh@|28%0}ZLO3Jcq)K@7C`gk?j%&^@;Au;g1L?eFi|4+UWJ@5HffM%$%`Q%yMC=$U zj}k}%=I)%oDHE{mZT`+(Lk{nxbC&bG8 zzvkX>_4lDb)@vy1Igr(KN0Dj#!n#cQdc7Y9lGFi%S$_2^lt?nfSfT%5yB!@HXMk*i zQb$ERU~Bi|`E`K&kLOVU>4lO>VgSBGTcOUkK8K+hkcx?hy#VhAXCvM_$=?784f4dV zyOS?a|96`k@2DY=W1ds1fHMqqYO#mGn>V&7?hKE;LKXB)C1Y+m?2pn=) z^hh$2&lV{GX%|}j7A3|6lCA9RB!~_Vj~-2iT@__-+|e5iD7@@E?}2g{8DpfkvD83v zQ>bk7{QEKWpl(uQRAwN0)b0s=j|$xo8bVZ;9gXC)InIJvB;Yujm;sCS4E7mKW<&~? z9gOrVFFO=o2>D!*tUzND=iBhL;v71~Mkm)x-h#n<^>!TEua!%orKR$JUQd!=1;;zd z_Gr>Q|3NI;6No#{h<3Rn96nwQK(_;z?4~=TVZbjuM}BXBMmvdFq4-i(hV)jbWOFcH znM>=N$h{JWK#9r7fY77UbO3b5{Vl1)6g6MIgDQTO(fw(f{yr z)}bK4yzvy>#=jDaH-bDmxt*7`L-P2sE!Jr2@M+VdbrK zXZTpVpt;va(NXsNUU|E}pB+Ti7IIStuu26r%?*#TqaCH4Uy`i8l)LzB$tRB@cyGIW zA2W*bvmC)E;M2u?|B=(sov78=yox~-<<5s_J{S=rf#dH`6ZzlIDu!VMA@cxtK#0Ey zmbXfbiGKO;i!8F5MShNsL0!8!_-YRR<-aKiZ+l!R9v#PZ$VYxF@Ty(P5usR3Wk)9cqqIEWw%AI|+jt_F0SR(FYm&3{FAuvV+>Y*xd*zRrpD zKL5=EQuDkxoc6n>u*iR5BtOaKS=$>Ed_4kh9S;v@30_J~u!0gq%CPfmS?B-bP3-(~ zUFSa-aS7^C7hd0o8FF=rp)BAt=zZpCn?CD#d_9mL(wWhp71AFGrFfv<)x?!n{c>{! z?V>@e^mI@W-D?=~je&en2YaIbc)EDoCtv?0>sg=d-RepaVF}*F>Xr650&MCC_#vWCq`!J1wFEjZFuqjPZ`8(=#df&N%cYD~}o0$(Qz4=V05_V!Oqlr2k*zj+8 z5z6;+=(mEtDJTDl#tM;2Jtx(HNE=G}MSoNP_Tfnx4VJ6OoQPLpGzsjFCB5|_HWJAP z*FY7e9jlbWI^->@*zP4()w4Eu!rYjc%~CzBOZA8`RpDGzP2&)^X+yX}F1;2>ZwKOx zr@-Wnt%9H&SY@u&uE)1$h&t%;%eNfX@@#gaa*OZo?0Qc;SL*-A_-W(j|#OL&tpA(jpa zeJtS&UBU`{ceM(P7)q5k^{i($IcVMLS(DZ9!J+I9&ze^HZnBASc1bff zF+23CbO>!lqe04YijeolS(`FH!LW8XK;-`I6AYo2M>K?9X74pf{w#TieBd$O!k(p{ zQ@boJs^aCoU?}$mtz3+dujS=>t;-d4hB6E7YMFE& z)wj!8nLg_>|Mjp|W(U5zm@&eAJVS?US4xc#J+&CeHb@KU0TX341TaigjqiW5QpQ#M!#UM;~H|*@*BhH;jlKh8oOS z$#BBZPM1Roe)S;@+6xaEsJD)hIiIzEr%QOZF=0JRD6)jt>k?k^kepo3BNkqe$+G_hN)-?tBH#eT-#>k!A?S{v?rfRPA6n4`)z z>vTDveNd0En1Zka5k~Ap!uF_&t%&22A;DbVZ)ZO1)?&juxr`pB*BLOex(AeRse4MqXuxpAN2rhQUKZ+zzmXP2;fE)V2uKB;+0P7 z3t8=pYZ0tOB!Yp-DmNp`N5Ax^QEb?FcUXSw^*`O~rTf{?$dDaMK z2qsroQ1en6oHBXHp9o_l(w_xf$OEb(uFl6v&|6d_3gP~A%M44IO(R8ax{bLO^UGff z3XtA-#HNacB4Nr5Ol7iY zX^Q?+S5>u>3o&CU_Vn_{jRawx-m9j+EG2y^XrTYgyQd!`=nlE9C+U%X^UM;r+KVU; zlRk%9;~IEl&iep*)15AUV*zwwu@Lq!(`g9}_r|p32((a7ZMyE@pRR{r+ztHKH+BzS z-`t}7+!pd=C$~jh{IU&NMMOqo>D~;I_OIHE(tf5++FxCZ(tcue#rPKJrfSwweWBnG zwXsBZ$N&qLlRqYt%A!TI!p4LT%M~Oa;VLpP8B+mM?r;f{T`pLQ^ih^FdpG0=Wx&Y8 zSIQr6W;OXS?BXIJWG>}3nZjzaZ9ozXuflZk<>z+MYo zg06m^pMFmLCjMkuzpECf)GzrDwSG=szrj}Z8_Vjqqt#r$qjN3lcile>_50P2TK&dG z4E39R?EX{gMzCw-dDkHff0$*JZk_x3A9EWn7C+rL_vD(LP1@eSDXEL8Tk*kN;Yfk;fLAi z_Q`9Rmp%)2)U*7zrq5q)oPRDxEx%Zb4WvRk4k8p>zU6)8iQ14Ut$4#)Ul7c z#9VBEIxcscYIE0PHZI`Ook^s28*w)i+Sq#3l;Q0M*iraWT5^SoYUMfAftS z4n&{FY!-7?YWr|5^U+*3bgGn=>a)VoQWvn6TK-RSOKrP~T8b@VDZe_BXR$nWl;=ZT zp5I@kjO2F&B^#g~ZN>p8gemhcSmyI6^Cn&9U#OWeH6)#Ynb^>#s_dSltSqOqfV+Q4 zCUKb4U|q}x>m2%Eda7$#QC(ZGsQ>7S`tnL%RAK^0l8&a=VAoWue55K_-%&$=h zX^l6RR5F)2MQOksDwa(4POoWJM>9cA%w||S48{sNb8ao;( z@et9qicg$xT*y=?f1abX)N<>3CN5ymA6|gXcL&kn7S`_5Ns2mAM?WjCP)1e2jA=NN zMHHJOA~+rUs3^B`3mKc?j6+O6z6zPRo6+VhvL&4GAFY3i6b*pavnGaW0lgN$+ti3b z0JLzkp1$~c4&yipZ2np0Vb7b&dq_xMo{r?K%0rQ^TDT(<#xv?`#xJU?( z1RzDNa^c9NfW{qmkde@mr}Gt?LY9BEk^E9EL{3U=ZaQb)iC6WqYhyVP<%{aO*hNq)MP<;x^^iECNl2qlm%`MQt=KFk99vagMGR6;eBkT@+d zns&Vt#m#6bni$@UYsbV1&Asr^x7mY_+-4jw!2Dc_Q0;@J{PSrJVL3d12fsD&y92(P z;P*#({}cRH!|!?c{T9L=f?vA%hp$<$`-(V1j=btBhX70e(gnzH3l)K)6c+)prUN#? zYC}nbhcpXG^Q8vVLFEvAyJL;AqBddSW|33P4@vS5k*wpEVwmzoG9^3~3|-mX!`I>0 zpvSL4gWb?${2Gk-bpu0>U$^je_;fOIL>BX z_cBx~yG8X>LbXPZ>fmFcT9$(9Q}sB?L zNgv`1X0DS8zm&<8YSm^%*)w;#Q>VTZ8(T4fd@$(bT1>W6amGuuVzAUt8sKv*4}N3Q z$)=YW;`ZFB)!>ag85Va14Om25-SSJiWY?O|`OAyx&`H_v*Sh+uNlI4_(Zx~waV!KV zCSf-k^jY^$f3<#R8p4l$t%MH>G!I4BKkTfOc)8!^Sf9&}4 zQ+;+G&=ZL+8{ay(N5e)v5m*xcq;vS?`^uy)f|y zOIW4{&Sk^45e7MKC6vxb{DvCRzOqEwVxlc~r~BCB8^lY$5J6#ghi>u9%`=|7N!f}$ z(=Zq-ztR!xugNl_!1fXm>>aPFHRM@8m67+0i`lN4NQadTN&di- zJkOIOPFtS^|2&TRxOu@>e9`mzD0x1g!jS!NQA&N3B<(!O-bFpAPblTS9)^b;}5i_X;XEv~|Y~yMeFuAJW*l zd_PLi=~7QhMJdz3l6!)LK|e(;BKfb5Ig8k9sHiw5UM<0=rh&4;pVnXgt9!)1^pYtO zz^~7C(unt~g2>9N(#!K{;qh5_(Pw(0=Tt*n&8?aM*)OHk5tgNPQ&exYsHukV!=LJ8 ziX$!};ZvR<51MgEhZX^}4!X(vPBd^Pf){z>B#*31#WAPofU1(SF^~QQd7SfR(g-6_G+o` zn?zezq|s(LjhLruv~?SNsqIyZty8bI*9F|{8kI}D$tJKSTbI&g!=_P_4d$jY!Bq4a?!xa#Zmgmy{{k&BHY9ejUEIU&qqc!CIg7!w&gS9V?+ubXD7w zX=~RVZ1d2jX7#cjz-_x)3wtBIeyK*$hW1b&h39cC$DgTxnNml)PLSKH4tElYUZbHc zBR1y3l?-}19Mk$ zI+$HG=iAx$I?^w{yW`(8KG0XhsAf7{K~X8?iF>R|G07YMU1yXx5ONmrTF+X4hJ_43 z?VYMbjWsFzv3hS9PXMcpI!V}>k&&|WBkZtJ|8gjmTGl_*Qf@xB=*0u;GO)1Nil>NM1>Hs&}_)Bl!XVU+@pAMh*ZnDP5-eY{6$H2#E zx`Eu%wL+D}7uU1dvszG0qW;QxXSF=s)=zJ=7tRqI;fcG03u)W_1UF3^M`+r3TQ_aU ze%iLbnt?gtcR8FHH@2?evsp$xqc8(5y)Rc5`viU%1OF0%59q+h7h^O4oCReY#`CSFuA?g~9 z#azd2Hi|o6Q^Q`lQV`S)*BVlo-+6;3O!>GVaP(R>QPi>teD4A-3qpokq{|K0DRcXT zxuyz_zd;aGV=cdGfv-OyQKRnI|DdzNR}H~cP^(XK3aX^zl^>X+Xk1dP`>gHjs->kQ zzLiU)tgUDA;FN)EqV=Urv~~Oh5VJq(kVRGbnSNPVZ7mL1QP9A@=H z#9_9=LmuN|i)eNPOmubdwwW~6shn-bf6DT%6#ZVtaTR#pD%O8WNWFa9<>*|L#XkHe zfzH(_M>Bwua~0}5g$4NjBq+y};2#Kl`JT1x&{$b3@!+Qc$;9s&7kK;< zWKPjrU z7Mm8AU!)X$vFwkq^pvKtwS%vYuESNTMb;S?>9HQ8Vcj*GupZO7o`Sg5YZZ(9;Zk)4 zT_rzKXH$+&8H$x7W7v+Ta$NkPOBIvNnW=p)j$!I&-#a$)3RGx??c)pBf?7j!21C=A z=&xVqdF6?-`7*C@3le3|`AdVDzcd}2zv%O?Fy~*{L;j==@nSy2*UYoz07~?Lg=slJ zxwU>?)NSn4qR!eAMJp=^E#jf|5L)DkzM^nuHa|LT@GPRcsXS|Pvtb7;0J>M}wZOs-B3dk5ZCoG*0=r&folN$mui9~VRUbezE^x) zEa6K;Uz5$^3`v@;H)R`PvtZoYnu=?|D*u99wtAzj>nvghenHX75o;l5CTtAtV0%`( zPKIXoGXykRu+lFX<;nGL>YF)AlLAv;;`S+9ivGZw`I6sbGgs)E`SNkp%vB*SgX#p( zKD~}9fS!ZI!(3e`B53X$#ni@HLe_I}wFQ;^$)2QdH7@GXObvC%wpUBF|9C?i;3pVj zF1bRT9z=gy`we~h^*^KhueYZw|38;&<%`F*AK@qVxh8IXP)J#o_XwXWx9jzNBMfmB zm(##2RZE$brWBKZ6asaV3X1O^hYZFhY9aex*J;C?*(B4x{A-eF--d@=eWWilHW^~b z_S9txZXNRFCz@0eX8RY!XDri2&Ui9a7ulC>daCYR@)*418J{O8wFqBfG0L9FKISNU z2icfwrGAwCdA4DpXK71fJ^s3vhgK--r>iaHAtgFQr$n({J=YcaNKM zkl70Gpu}0stc!o{+y^2OOV!&HoX9715?SoQv{dQ$qZR%nHM}gOv$tjO&8B#5eP^Vh zT2Y}Dt`vFe7Xq`|X9t=tVTU@0EH@uWlE%=U)I=LwgfMbbFq_DH>acCC%NU4nBm_~S zJ5M4q5C;V#2v{BPQ7~<*LefRn2~I*}*;zavoB!B40ury}a)3{hKTFkcvThqB-Nzh; zN!*9fYCN!uQEW1s!|;BQ@czB7>MKfUG7E&%VzlLFl{_VrjWoZ>y9({I@sh@VtC;lF zrqS40aY9^pbX-wk}y! z$eMY)7ABuHnZ&1F?CaIBv>iS`&CK$76Ms{LncB@p{7v%}I!)mAV}12gnCP|CpB|_D zJt|fH9@S&{J3V~60^h7oOg#XUrvI$%>7J`NmOWkQVx2vmBh{WRjeO(iUcRxmCtNky zsh^O-IW!qX>+^esUGjLjFL6k3yFXt!Ea&Bz6&>Pjk>%hKOf|+TnKefzi7GPe4~(Bn zL0?njX0s5g`)jh4gq6c|i`0;HY*Dur8f&S0sGXgdIx!{MrbOe`npoD#IMct`Nw*b} z`Kao1Co3o2MsZMV9!ge;i2=CC5@FT5-p*N!U%sipVX8^`{l9uLZ&Bo#@X}P@Y-ak@ zvMRaGUS`f^vftBhG|{Tq8^R@BIjt6GjK}kW!NG$3#GNid+a+;I7Dw~Ctq-wcLyaDb zTGc3BRwdVsrgr!l_4$nm_4!w0PwSK3-jNRvnGcWF3%ThdcaqW}?^TR0@cs-n{Q;i- zH<+G|VlXc%9rE*Pdb)$Ynwc6-bcfTQkTq$!7q26(f3ditCj|lT7`X*& z?ermegCP416J#%KBtiCdc*ysD)t6g15e!jkz&y<2j`1I~k3HK;!nSB2E*#x?wj$2b zPQn*rl`g=)rnQpWc0Z=|z9{_9>#+eSDLgzC0~ygGH(|@D8MdmtH^h z;>P39Pc6H^T0iB|^iz9#q@POV9}*pJRK+~`gz??4KMnRE5APrHUcG)OtQu z{(iFEq+#ty(`5v9I%j=bp6}7s3;%Z`7R`Zm|iuYO;(vr@XfwTj|;w4 z=G6RXSG9oU$5$4H3nU)q=NfmC-yN5ooU8L@p1A4Ur)vpmp8N8 zLEc3A%H;Q{a?vWcsHv&cW)d#b@QWJbc`#0YHK{15W$Vk*4vqshlCvnP#Z^Xre(^6Q@XhZvRWkHWP4JkIqd1Ng(wThAKl z!QOlv^zimHYkJ7g=wV|I_meEwr_OZi)AaaHS$iD#7o@>IQ^S8r&+rF^AP?}nES+(` zu8n7I%EycU{TlrqNyw(`d3||`UETB28>h@)^`_ETDeDZ;E zfm=&m9H6~lufpNcoldC`?yG<)1%6^<4nTMi+o(%c<(t0U{0NP*lgvsw=qDklC=q6M z0-rvkh*9{}u_2ipiC1?r&lyUQK)p=RR2+oYC7rx}16aVYT&HGx@K_Voq1dLD+Q3RR z?9pnh`&j!8-HiY7Et~&;^{uT3E89$-;7sxJzPF0+7QWJ6mg=}3#2euqBl*uuulUP+ znfXC%zQwHfJXqiotX1e{axmG?0beN;mf*xYz+fhj-{xc<8uCp5&G0Svqk7cdp03f=s0=_OTn0hSN>UVO-OJ5Pf~LxqdxrmitGish?^6@sAdon3#F0Pg%n|(y!qyPI7PkCYd~u?<}&A zwGe`2SJr_Vttw_;{o(bb(He9^fgq244>el29U0wcS5D{pb<3rba4*x&WDJ1PSx?w_4%}CRKoH+gLyy}xq4<6 zxz7wRwy3&Jlz-Sp96=di-IZUXFMl>GKb%tjS%&h*@bW*tFirUhbNQUARo8qzxpa@_ z);9J2zSQ)!-KN(M9r>rz(@KVx^rRC}=Rc!2cbJ6|tu2yXqMbU)N}^>;k7AQE{9?_t zsC$1Brr|roVo@sDjZP+p5m!|WhDf(l>*qsB({tKO*Bj*@&74I=?w=~9G09Ac2vVq@g12Zs8r~ON> zAGROgd6N8e>DB!(B!GN%iSAa{irO;W&DM&Wt@EeQ#NCzn2w2h((}^3mwwkZ z%edoeSWKm{SHZ31#`S{0PRi3JoXt)Q;jMXQucM=KUBbAQ&UQZM)LFYcB^VhD##i&q zE>AznAsA5;3>%+o>iArP$9}jSE3G(&`Y?VGnwc9-WmtB>lwB}n_se@v#du~Mr$KnZ z*3Eg2^!g~|68+4yz5n9AM%L4fK8q&VR1niI=$n>%S?>24!1dMb`60B>Od~Yoe1PN0 zhs>M%B7d$SM>WmF{)E1}{d)6_TcfhE?-Ko0Z}X*WYWUMMn?P-;-_31s@R`wV2A&2-X%Ipb2J{((`NOA1A7?L!-oDxXNsO3NW=zcB?# z#gMfg8rDV5E0M||RXM;f7s9!X_yFn(=O+509+G@i;-cJ-NG8ZQqeeb;Ev@NEMyi*0 zPE(v#)8~|P7_rs>v5HtQKbr+_B-~S&HzakCw{R4-Bsxx3j(4*8th(oVL^X3u+|N;dnJ2&Tq#sAviJ3+*iAC2aP$mVRlN%6*AvrE|%Z@ zrAtsIW(Fw80O7=3-=qC??a+Fy>fngZ=M*URNwhs)*9LPSxmXQ}#5eG$Pq?Vw=yD$F z3P|Wqxl443V6kEc=w$BmYh-)2D)j44u^$h9EBgK4;pa=i@2afh!mnEGH4Vi&d4FF8 z#Xib~b}E*$*^w*6@@1kaBcR8XqvgPfbKP$C29BPD=b^R^yP1bb~V6D_DX6%yT4dVuDFxnNy)4vqD$)6Cl&Qc z$w*_mE@`zswgzJtova0roBlyMtm-k~mud>tig%h8w2}u+(1KR;py66jGY{G@fCFzM zqf>Rwjj|ZK7|GYv*lLdWDP6=5=^_rLi})&C#NKof8`DKBF-M4a`8%wiKd=)o9B~D$ zpQpk@{_=IrKqcM%myDE%X)!i9yi-|Rc=eGOl|h0Sb64m~R@IoA`I572=FTY6f|M@n ztpzC^dvv1B*!lZ)0Yn~SMwtlcHC+luZ=+4L8hl7gL3Hju*)$S}o~%VS(xEE)R;0j>Ote5d^y6wcH}|=~GlMV~WQ=#ebcb^AX8^^@*)z z+!ol?xs#uxD>f}Z2eCZ&Ant<_>7P78(&O_eDJM9lXswAt<~}y7(}v7Sx4k_ zi!wdS{-m4Mxt}v7Zs{_CUl0>6vMBEs0?rkX{cd4(0`^^BCoso!#VH#$etFXA0zWM@ zPRmX`;Z0W6;i{L#KJkc+r~CEE90||Tm3-(>8zI720bJ9lR}3=@(7zFkhd<;fscYEj zH*YLJr0;F4uR*^Ni%^yONn75BqjlxEetvmY z2Ajl$0$)1>_54ad!FmXr&)>NQ9Z&}_cfuHL#!WOKd^7py8NZc6lPfTgRwIkc6weP& zkbqs#RZX0A-X*|U?`u~$Yqid~WOcjWO_dy|-U3w3k788Adr?euC*~nJ7RxuIbWW-u zt(;d_InAt`)3KZ}-IwEKF+}?@w`VOh({(MfgA3{)5ilVE$SG%RU%^Vhr>czGNr#Uu zB0t*Y6y!ZWq`T`aU4FVh1M|Oml$~l;RL`pj?H%&Tr>Et&Y2_Pq!@+z-ssU)IqZ4%s zwbv6Afy1lL@&=KHyPkdCqlqS^TC85AL5ZeHi>T74s@A2d(Y;@7*WPP!D~G?|C_Ju} zSEq+mulpdk_!=~_8nfx5SLmZf*4p~-Ri^LNrtcR_-_54)7Ss0*!#BU!yhC2lORY=8 zJ|o7m@e|$XyR}a=loM8{Q@_G4PSnAp`sz?{+YA8nP0@v>Z?EaQ*!1nwf44Od!)|7# zY(zedXW6vGep6=A@U6PGyE;!npIoY8(2P8-Mm5G<%9XrhbxG<>1=X9r8%*C#rtcM| z@0F(SRi^LNMwDLQC?y7|K`lI}DOatV;({Qe1?}KLFKQY0@t_NGwd%>H;t!aGKctH|lrG|{bP;>gMQltLvBV+*wa(0e zmyeq(=_1s6q(uCXF5*zSh_B2Miq!IHJV&JfK3=1kF<(pTSsOzRhvvFU23iKKa#hsI=P?jtt-7oD}$szeqiKiu@+jVhRVf;w6Gra zGT!cIcj9Ol)Tk*=*HU~zDVCa3^wm-vpcGAN3e~gP>`acaaaLBh1&J5=HIG>;@HN8( zgMU2rW-Vn$(r&#FXfvBK5i1{<0f)C|8x&5H?8@b25RGjYI&ZmHN`NbxU|*7f_&ef%C!FI!1{J1_o?=RX8KisTcY)Z4&7)MGjV zq#pXya@FWlx77&a5BozqmOg+TOD|-{((~w8df1*A5B+iB3$_GzX-^%Ahew2nX&)TJ za0yzZ`Cq$=?1cV zW{|pbud^HK&F0d3k0z5Wz|Ic)>Hy9>DLp;wyxK~BKhrEkTg&x3vmbADSh=w!2T}4( z1lrPrZo2tM3i>CC{xEluH;(n3pHBu*HnTm7ZYoCrUd`7LjoxV=k`0RWc_6btoho_v zV>(hAASyRCSIKX_V8=%8ba8pcWF{P*j4FsH*kSJ*Dsi85jiBw5+V(pkBpkYpUa-eyfG5f=i9Si*{E3_9CdXuiu~g&j{rhbRRWNxl5Mnyq*>Rmyo&6d0$x6 zS$>D=ypfN=3KChJZ;)JLK;R z1il+~UY|6jZpzf3^&Y9FVQ0h?_f!Bj)`$UVJRm9KgFEEoZrS8dzC^l&4f$u3Qp6!q$@dphTvH`183Q;^4c$VI$Z2c3fiip-NiU*y?3ZswS7cQK8bELZsMn; z=^F4!Skh@~!tHO{aB3tTRHCIf6CDqWUVo0QFKy~J?w`6j&N3@7u|KA$ZbaLjCQGJ z;j!cwv^w@OUmhR0h|o@&f`qnPM`&98u@;wW|LM`9Y z9)hM@kvSE~X7AYS{EBdH*6NebevjFgiD?y>W_|Whm%%#kM5doVSi!6&pBt4WOn%hH zg@~SURf(|cBZ_qq4t<1I7va=L6zU>e`iOj8M20@Xt&7OiB63ZK;f*soyn6mcWZ8h&@cgGolxd|8&}zATUr zc!7T8XdQn~m9C2`EgLViC&Ic`8(5xTlt zUG=rPK@t2dMww~1$Y4h26TAC90~chGj%R7%;~dPnv0Y)Y2m%3!W-2cJd5%V zpV5=_Z>j0G_9*>{eU3r1^e=OMi!e!Sb^8 zY_n(o=l2mG7m0%~0F$x>?i5qm6~@(>{7*t9pW#<~I+v)JD=xpF8^dNJLO(9`s@4@E zY#&FC!!{uIuXD`E^cr;5e+>kH~ZFha+TPXJ1r&&X7bEe0xYPRM1Eo#3G zl5fYSZ@)lF``x@(-+lz(N4I7BEwk&|uVW886m=Z!hdNF9z_|47=S)Fv*ZcbRBLK7A zY1*$;*M93AKd=4J4U1ekE`9rbP-aH))c5u6M*v>Uv~0f$o1XqY?0NfX_^rYcpFJ%- zezr+w{Ofd9+ z8)Agl%Lwmg6TDN832*y01H28}4Dg1T;3Zwhg16WR?H}{zO*9xAVRpzJgJDfQu z!pKLL7dY2E6tQVC?#4U@zf`_v4-}K${A5!4Iwfkqme|3Q+ z{AC>eJQeor%6=j8P@k2#C{>-HJ*=d1aXxjo6hbvqp#SMztM`8W0`e{ieKP`=-x))gN6aQ<>ze=lrPD*~O^=I5d0|^eMg@nyMvy-m8LLIF`rqlbS8mHT+VF-=rD%pR9 z7>^L+PTU!$ipWd8V~VHJ#0KTyyrq76gQ{Qh$vH;VPO58xQobosD%bsp$0t;c#c5wz zXe_$Re_$47Q1gHN<+1ZqKKUO%4_|?wbLQnWc3+;JzL;VMU2ACHYFNZBv|IS25 zNRthmup0@{M8B6`lG6?S>h(i2FV3-+r;Pk41zvj#S}%0N&;4@KtAek?8goAvO&`M#&{JN|BfKPoMJj~@Q69>cFS!r%H@ zI{44Ori1_69>aGV;a`^){snsY7xozb>h%WrS!v;awM_@VPmke?M)*%{O9%g6J^WqY z^%TCm&Hz7{7XGPv_`mHj{3av(&aG+bf2+RzdknwW2tSq<{>^&$eR>SPquEIRY2kbH z@W1)C$MkQ6zqKPR{def#znBKTj-O_4wk&UVQgIqcpSK-hYj{<3M0s+{qYgo_q%r89 zaO?JxXH=;mtv@`~O6=Eb9fICxPKR7Q!gA?Ke`(=yQ_Dp)nwOh-PWeJJ;MLh*?O;K#u}{pN~^)Flv1yGi3?u^x`=pN&~61C!QzswH4lY1X_M~cyg?dstv zs@WdE=QyvuT;C{X8{!_^uhr**V`1^W*NjDK(bc@osTu3!vQG7S`eONwkI_IA--=X} z3~p|U<&u-Qo14qj8>f9NruRq4H@MjN@*!p4mhk0dcF#+!0?J*V?+>KKYz)2K5RC03 zRo^$3?vc8wPaOiTm2Z%B^*Q>HQlD9uDObgi4k=v z|NPBi9GA7S6D!HjTa&8Om*F0$?uqPJ%V3egvZV0 zKf_vveYW7|ttk4&?<;DcPGGmX%fB5)2cpKK=`)^LZxhC0vDIvmzhyWs@?*M1{)K-h zeSsadiE(}(;QJ@gK@3tubdtTyAvSi&tNXsaj60OdXH+YsU!fIOMVA$_;^qxE>l8j( z??e+2ANLUduaA(j;JdTYE^tpHRd3ofRAw?%Ja>60o=!k0_ahD^e`6kY&*5l}^981g z&tsVU;6+T9=rH*O!{jtxYW9apt^6FP(>nLt1deC!bL_<0cfX{&_4Jv%@E85LjbCulmiOS+ddC>c;I8nakVui&|7voEr zm$w+Y;-c6E@2On7PVWCTi>qgd9(smx>3Uw|Uxq3=){9N#5M`Zk&oDDN{HvKdVZ6Xf za}867;P!n=bNuZC)*Ry#SF%L=pqS2ge^TOizRNM>gX9(!##$|g#>*c*qVo?EY%ID^ zE#iD$!~}jMpj&>c9C=|C6eZCuKlg*OJvK@=S_{u4`b1y4KITi;Coo!{W@E%tt)>l) zr%_9)z57^W-8R&G5a=&!X{ct|^u^=#HH4Ylxyqwyf_jD zFr5bQM1P}A6r|5wu6Qwu^GkcH<-d%h3i+ATIq(xEb#fu}^N*R+k}AIZiKDZ}PpDeq zUMzp+Sk2AmDptv7hL|h4X$>hIPwuQFi*4RMub-l}z zUJS|U+&)2bIv2;OHqN{}%4ZBznD%vnW=P;=S44>Nyc4MSnE})RuL8+QiMX2aptAy~ z+0;jz$0CLYU{Nis;togeD_YG*qbQFas`Ypg^A~$JH3qurP#WWvSE$F)7dbNoM2_sR zqw7XQkI@idk6`7Fff~fI3}R&p#5oM&M$Xx)e z@@2x_#<8!JCt$W{mm_i_`*y;2pxHqs6_gESZY`G_qT^St&TJqlC6+tjBr?6HP(!6& z-q)9*LUj87u=XV2Q4~wh9)V3DCP092uNpKU%5tcSh%6dp(I8P!>d*Iuo$2YW?yl~x zuCA^lN=+G_(v+Uql*FZgesuRapr19nCsL_yDWyJt3QHXY*x_ui!G%yk?e%ttUr)NOp{n96yUW zk1|D#TD#(qo#0?KE~0wMv7URhdcJ$YR!=`w&)Xkp_58-Oy39PI6R*i4kG5pW%&Xn& zok{fuu-?nHdS}|}eQ>6%cZy!`Tn|zH2u=U9K2~f(*!=lzxknyzoJVoP4IK3}N}nda zrFev>`J-tVny1`1@AL7!6Y&T&YNnm;7^N$z7pL}cj?#)1wqAT%>0yHNZa0Jb)9}95 zpC=SWS}8Kb|I;dG$hiA_?2xy90Z%b;B)j-y>qu6b%ynTQucns0m+$1AyV+Ydu2SN6 z*Lzlb5(TVC!?|AnO1Ao4Etgo$iw!-VpyPakJ{qsS*B)MEf4-kqYvq36)|7*klO5UN z<{M)p6`F%ekdK|YM~4$9J&~KLb@BlBAZF=6a>XDB^o7i#Y_nJt?7;ea>@6B-Wm0|k z^h3Q5H{FxcWv=eo{)#0~p$f3=@$a#9sL;*#k|9ufCw?|h9*rrb)V%xkWHOOOU)Oh~ zlF0m)BE9@Uqn6AQzxaVX4o{dC zl4ar<@;D*Vk8IF6%m!hf*B7Wh?@fK)n>FYin?dS%xtyqReZbsywr2O=YfR%)q;uY> z8|s`76+h-Lv!u>>g^c+pn_XpYeT_~Uk^1rZ`?l0KOP*sDEtBKB!7}+lTPEK>io5nV zd_t)*AN$mA)a+~*D~66Co;H8rmrgS|<@OHd3(G$_}^tM6GfsstT$ z;_SH0x^w3}VeElV;zi18!Q6yLw z4M~tF)v9<+UYOhbD5v#0DBV4nFIa6uEWaxgHbg4q!^JAgC8Pw>&yW@da6unTu{5%e zH{W-^>Qg<4gh)sy{r7O)9daG5`#r}>z;5>V%&?akz;s*h0#HC<#8z_GAdgh`2N>XcJ|pftuE!P9AANs5)}B0l?*FrUQZA&w&EgG zFx>m&gwoC<2s zz)$3Xjk{2hEGgs@ob1BU3#3H+0`&#bzgsMKnst|voOJr@ey;HBB9FEIR^Uy~i;MYo zM{Mm>Y>F6PWf)jxwq~(mVElC$lP%k5X<}*EpYBp^RZUAYNuI2S0JINR{tN@!Ctk}u zVTs1b+`3VnTO#IR9~qY0Zfa-u#nNpzp?71ZJi82V)SmqbN*4F!YLDP*_f6z&DB?)N zRc0Zl7U+zD+jwIWO3n8KsRU+EFa$3fbVOqZO?$oL$0%el_gZ7+@RN)kPR6pi@Z*Vj z6Ej7*8x0D;nxF&yMOnbD#E4q~4K;J}y~wRx2${a4OlM)H8Cs^nlnEo){6Lu|W2R~? zQx;`X&cJ>lXJAKVAy?TL5M1Tb3zajlXj}a+g+*0v5LfvJRk$x!_>9JA-F6Rhl^u8* zmsz9n$qSNt25py?<8G1Dv`-aY)>P^dpWe@e8kTD{97;9($Ixm(`=pyCoVRBYV-+PM@b|dj!s<+lDlO&&)oMc^BYnFl}eVhpkjK6rUEA23A*-CzxnneLkIdf0o{R^ z|6Bt)fq)LTfc~)&few$aR7p_*bZ(U&4;RPKDdgt3oM|J_&lSOcJO%!@OZ}E}d<3?H z;D-~Zz+|s7Vk0U2-kWHQ(U;_x>RzFbdCVOM;&875QA!{VPjmz8ZI4QiPbiDRC-&A4gm{pqOxqe#1OR;np zO`iDnU0C`7k%Zoq$XO1H$?G}xQhi5{`RFe6Bt;IQaAym9W0{G;qVV}1qH!Imk`u6! z&RQiERLK~TWX}6QlO1+nDLC@t}*ss~Zk;*huw z=1j_(ce5u8vb?#B*cRyTSU&KhXyl?Xjb1}~Y$`XO#v-v&b)AE$cTwt*_S9NR9b`|v#E`cddg|$h64B5} z-{IxaMc+n?y@Vt?{(BY^`p4g5NdsL1986y5?)Jb$&ekUKgGm8X9c&&Q~@72F&cBn+qZh78O8eJ44L8nVxAw0({7>E_XkoKI!_QXmyQG6} z*H4nCtNMWX#9Ws^8#>n#2}H@QWk9;o3b^{;3cO;8P+Y$+_Pr!H}0bkdh z26Xx%Ho49OrV@qe6zrVi3;yIgv=P$SasCERzg+G~)D#d0a^>=?DME+~nGbxOE+&hP zn%M=p7%Lilh`_9=A0LrURBD=v+~s|{@%MN+k@Sm@r0=7V^tWzCQa4Cm^Ay!ZGUjY}4N{a|R6^!7ZcW+;^?Kd*Ia{-wi@ zEQwC+#oJc@F=pMK`AUU!5u!=ehR9S)OURM{Xm&Ir{VIrd_$Y z+_}~}uUsAWdMR=3QcU;iE#D4~`cWZ&JFm~5N6NFsp0r}O^G|GYy?Cn28qVIxo4d6N zj;#9-LG8ayx#B{>41Ip?_P`(Lb7! zI5r$F-a2om%1`V$K{<>n$jD`WE0s1e;VGgKoo?fA?X-sTMWg`4Az!rS)H&;jGby7$I;_Dv>Ri5-!JN;=AULd07cy($SWYVk3355AUn0Z;3*r<7 zu?s;gh~H!w7Us*$UIm7s%u&kB>HTR6m6?|mfwJ4oaaN49ANM^-aV{j%=CP zcPw4>i@YVJlp|+9gIFkU;#n9dk80QY;j&$qKZETmOe=5H=OYK2w&u<9{%zfN@FjVj zirWDZ5Uj(Wylg4e%+9UB3!2r#DDt%6J-ZeSkw=-VuoN;F!wu*4_2x99&I&s{E zoBX3*zRI<{OH4}l^0n3O+?#_wx#WsAWSsd+p5caH9e&du!FS1p^v#W3g3T293u$w* zkLkv9rz~h{dq{WX5 z<#)kP+Wo=u_ZCKf{k{3Lq=#>Pa*of4*DDUtMtE!DJgJVh*IG-uWL+-Y@DztWsXF~%dU|Srwv&6RpLKlyNhJ6 z@h1K9r)UOO5Txl`Wkvn?>3llD^vNQfVtZB}mT4r;w!t`*VX#7bA;YMuA75^ra8&M0 z%*~;%vA~WesritCWuP93&YXEmwNbZ+gs$wU^{Wd3dsqPD2&{=|JbsiER25z=5^yh_ zr`hgbW8!hW|G>*#@tieyvI4(1!CdyR$(CrC* z`lA|0dCp3sBToNvIo-FffH?+he2fD5=K};1E1%oUF51nTA14DqnB(yPUecUy8DM<% z2;%B=D2OCu*k*ImGo1}q4sH8>BC0~Cv9Zcza*~e-6@bx3 z3_u#|2m5iT7&$~sZ{f>*C<0lckL%}Kd(&ss{xW}i$1hK$Zu>R0x=Ikr>Pi)wx?EXR z!J0A;nNLHAOU_~u!{&t#XdhJA$@BUt4Ypf(F55F4{Wx^7DrCUX&G~V_Eraq>BQaGzzDplhSkQLxjW{_OKL9_Ei@`k^*5Je1w;#aHkzlhb% zj`Uj0{JHHp`M=9?aptN|yQ<}~t@-TphNc*14t$2?MwY&J(9HN%>gVC{^A8q+$Q8gp;IZ{G|*hFuOns+3LhZ0+n1vwo8632>Ulc|!z8`t zvCb(6cXxct1zX1(Oq7l-(^>%uIrF~eK`7J%8xqI4vJNG!aewuvB5~8m=A8L!xTR6r zmp)}6_bVW)BDs=X^+N&eG)FOfaTgy^3w9Kh7ZX#cZcLv1H(d?f-Wx4)Af_A64u`BT zO%WN*|D9_Jw<#ccgCmQK&ip9y0X1oWyPnt9H zvUmpvK*jZuuzVIrJi*klh0%d0k|e#tp^v_Zl!FlBDd1kJ!BuTt30?RzaEwEFlURVg zG?%>U1v(soQ?Ms?8vJ-;9pJ|o8=9DwU=I1`u5xP=sQj2vX%H$0h#zzMQGm+CA-Yyp z*lbP_MjnBLH((VqomT`qtP?Z`@01gNexAc>0$339`opDN!Tn3Ag^(Q23rGOR z-KU>pLb4KdEyZ!N@mLdboa#e#%n6#A_Yw1hB?YCVgFSrt{$B0N%JEb2%1u@gnaWNE z=sWIfb+CnC(Wmc~$2jSE0CyKWz_olUyM3KjLH+oSbp z8N-(zHp&(JgEaV(+>`K7)ssPi389=Z#V?N;*QR8F3`@_aEn&yTEXx%~&};#!#hEM< zhBqFSt$0X%r+VJIL5gqm!M^7(IoOSNYJ1+(-DzVyoqf_H1P ztqu<2b+wJQfhM&8H26gAw(CA5k-|cSj*pf>Un!p?PKb_SACNH-mV1ZIo(Za9K}CNM z;ScSCilBKlLNyrbG7IX*?-NukS8aoe6tixg+8Ban1ff8>G^k;nQi65-*K)wRFVo3J z*g-aci7D`G-G=NP5{+GoXoMxvcx$PRXauH#17iHLdZ~`(HHhUks*QKpzBb|4CiV4- zrMC6jLhBWNL=($IV$C+MiYbgGigkG7BX!Y;>T9m;>umfwTYdfUPJNO@9Fqj52r_Xv zL0Z)WiNFNm?OY&3a%nVWt9c=aI^lzP{Q{cZ+fz9iXh8+G01F8QNGdBFcDKU>4;TN` zgXQACY_YcZXs`r30<4DZ*qd@* zOq`{i7p*hm)Qi;_5iC4M;G&y3TuZVPi%K`*fzhyDoA_0k&B&N&#y%(^4LP{?&+>p z!de(+{s<&ufvnt#;&H}q&UDV5Mt|3`-`)rljbpckrqzw{TaWFmM^ePy z80jd-DI|log{+`$gAHXgtu6{MXOYS|HpRSpLu9JVcCN}cE!7HJ@lD<5N`r27y7sFi zQc0>^EbESLM7>5sdwZa?Z zwy;um3{|UcO8vq2|qj0S{>4|3{hD|>scy1R!4!(YiOvlzs~m9+Fi{p z`tUk#wpGAF!5qtzp$~uV|PJm`T7&9C_sjEJ6yfA`uspy|&tI zToHWUrhTUJwWj)PUy8ZfQr(R`g~V}R`MOP_+|rk8Rl5IGB}QdbCr#8h^;Rk=UJ0kn zl|2YaT!tBmi4sNPLuA~qx)aur5!SGf2C!<0b3&!=PhoWrh{ElhdCd$(bS)xYpES^C zT?y;vN9YvYVs=i7LoG>Dk==_%%zj-ALuq?OQEXf}SjsOQdAoErQpCDByOK$_s=jK? zGDttAkkhNGj96!qI+N^Fq$W$MNU;0R7E<2ZV}I0hzsbjY*^JgwrK(}zkzoPaW11&r z(+kjk^L%)jyQUy4a$K1d8?#`^ve&$xXP8_1cR}LP&~>?Z1@5a5oUT6fS(ZV-ExSQ0db6C4M>0nf+8&Ju-&8BZy8pa^?iA$_ z+`JC%6qULFS9<<6#G_;P>xzj@Ce?vWs6%Pf9;m+ts*9DIH==WPK854@qKiNCw}mWP zs$+k5va1GEWZkmUL0wdW7KoVekX6r9zo3E}tj5zN!Vm*J>!`U1azY2KBf$=cz0>2v zeJQAzx)3}hQay-G7-aJb7CIo{=313SO#+^=zkZU1)|RTOgRf4a$>5Zm+8gyvMKH$~ z5dvkAwN#`(Oe9ztFb=pDCX+8H%yh;!$b_?^%r+}3v$TFiv#?)f@zVj0-2xCVVtjNy zkg^OecEfzJ>=$6=G|3kfguGGFV~f9aC-X_%>4m*p!Lwm>PczI(iP%)rhR# zx=^;YsZnp!u6fC%YCxmafcxuI1Ild;n65WqG&P_+J}G?z&}RXocj6}sz0++_yXjCt z-$X#29^YkX<9E}CSvb9$^u)fL*rEd)FptClO@@+0v{*em;8%2ZanhK^2o}?iUed#m#0>G{5D zkNHr@iF96^>3+Ha^=n`wSi8IZ!t&%|qs&CFzXl;ieYL^9)}ll!U_(si>XU})u^P(| zrDM!3Q~=zU+)Etxa-JhHSoxW#0C=lG8oD9nm7&Vq^%3F;?rJZWl26`cSx75&Y(s$e zQWboHTUZj1TJeLa_O~`c?Rg%n_OH)C%bMOw->1ra*Qzx`S`j!MqbPi)-gii+Ry6gf zy(9>Md1dyv%~GK<{|v}`%q}V|45p@_la_^*$A4GSzSl0H<5Zy>oC)=yI5)Mwfty>- z{EawAq{10G6$X$>XhR!=`Q?AfYk~B>YDvv?;M=d2cMR-SY0F%a$_+8sx6@{WD9~JC zzH}`sM~WSxh&a4OEYcI~kw1$mHwas8w^|;!7FKO>!En4xDbg=f zM3&^jYD2X>Xr6YfY})I;$)N+SsKaS_!iSyn$o~a;cA%tlONi;IF#p)fdM444EtUQ0 z>$`to(<2=eh-U~yB;HEg>TqBW@k_4~x`R1qeKKjo#*uYA6;S84@<7V|%|xHP0Fm8& zf1=$x`lYYR9wm25H`Aw5!ebMaWpSZ7_rJ&jjiygGAz)sF3ih0dbca!G5zuR2L)Abm z)b|!y)zep~D@a@vldnPEuE@gTlQoEfq7)Zb>CTgU{~py8(`%{%1_w#7lsv?KGq*)O zkHqIo_Gu$iFgjvPVCG4@HzDw7uDMS;Is?e0fEoND$y(;X?E*`muU;u5GH030*hkDQ zYn|8qn{86I(6CMBBU

l=EFNtuxB`ggvKQe)e^3o%^O-q0W76*q6^qrutWy;Vfxy zf9bQJh-=PSp5DL0#cT5%{uRzV#pYiD0DN;*(#^j@GsrKS8SGL#krG7sq?r-CJz+4UW zXz^>E{AwISb9>nA+V zpDw$reQ6bJ%VFNW7i-%$yYqpYJ$X%9(_93n7eQ(pfJT2i#AlESwh%yTxsvLCKB_8uB341)|#e9ekng71AvvbMV?TL|Kw;3Sb z)gGQ{YdASM#c`OMeiqGAmfip{$KeeMcGgWltgZc}z9d)4Yl7feaX0=|1l1dv1ZE@S z^Pg>W_{1-Gk*Cpr4@#O=s^n2+xMYjbpHV~C$*j0mVy-4oyl~RMV~1ZE%UgL(gMZ|o ziT^HBTD2wS^Z!FpPl&2!LRqgFtEKesv{vk&-`>CZpQ-+dJQ_H{kSIP%~@ILXqitH^mwS#Qm zZoU#(>#Wpz{>XBzXAkNd(?)&$slLEpuFtWy`*a|k$gY{_H^7bl=}pU774@4V0xo<> z8~LAYE5CdH{oo!O?K3wA+U$U%Ie865|oxfo2;RU^~G>j|vOh z;=uz(Pt(N%SU+9SPds#oPx~iI#p5o)$j6Ql4@r0^NEFe&w@OgyX?1}Wp2We8ejl@& z)9O4Tp`g)gPw3LQO^SIAjApl8#IIZNY-rgHN z!gGI-(Jwx9qz`6qtyBafkAe}~a6Kh3O-c^RG@tp)flsTc3L3F)fV<&n88#;i`-&jQ zCNG`;I#A2>=hVPXhs6aecJFg}#7*a{>iGJ|<}193%`~SQvTN<1=6DnT5+v>tOa-s- zMJ;~ud$@EBk6NqZp(m=cikwlIGiQ~*1zQ)tFv+eiG0_yQOi#3av~!>&dNm` zZhH@MxZR%v4mV#DHq1E=osr^QRXL3Q@M*+&R=`Ubr-F@Zo#x=zEGO+@nDeE4ml802 z9(i^qFM7_qb9$Admy3%s#|;KvucI57iCmX3mXkBZ(Xsz)B8koIw47fUEpZ<%S3 z$}~u38k9JIGC>y zH}RsL#~-C3?IUYv`_U$|oir%$CK`x?P33=4O7y|Qs8AlOP!1bQ%y!EW4IvDwLu>D- z#yp3ER?o41L&^|07()fh%3zYQ3>tV9K{t*U%u#(!)@8AWgg)$rRY zsA0QS!_s5)8ji+^Qf_CMou(kl6^gQ`fYWCW6dZN^ekGnh^F>AGA9?ch304rfw7k#w zP2yQuL6GKs*Su9&^mqZmY|2Y-1u;$E+Jd+!V_mKFAIoGG9C>k%ofmi8Z|c#X42Zn@ z&@FXC_RY_TrVy0_H!)z|L~2R!ak~8@1oN`ZUwiv#>jm<1rU4>_qVZTPCK! zRv8<@n{ljPVv^jc^W(rN~p=;{;u@zjFy?KW=Y%h>jDc(!vDVT zMU$^VhUTRu3?h`Wu)4aI1(?K5+l(? zabh?|B1{}e#rBiqp0qrb4z|XY306!#DPTA^G`6H57KiQqu#uGhuC$El1=Ym0} zI2z0p^ct>ddgIFAv#Wi$0 z*sjs(_+j&rD3U$^S`vcGR%pq!1Zq2h>H(lm&$fiaIuF*Ce{-G$X7~qd!eYXzcdQAE z2^)536L!4p(Jj=B->4b;Kr`O%D0_4(^{97BkA5G6!2^zGWe2p^*idaqEqBr@FrThw zSwFX6c&jv89NAKNwiG{_4b|!%kPBX3O%w!tcvX&&N+IS{&RC){p5Um#%QcjM*cmU3 z@1ibTjC|ImS{DwbE?mb*L~uT?pj4$?Jd0#oWvUTp&Qf=O-2MriQlU^Wa+Wk(uPdh6 zR0dBNsstx)d2#(Fyuf37aM}KQ@(m-^3wfIRc9vY!q1qqIBm;bw1NA?mz9U8AOcsdM zLqV0CJ2Q==#Xa}Wna-cksq@Hro z176tsz|y+bNL$x!6b|w%fuT2V zFCY)*4NUe9D}~`@_dpF&O-#G-DP|p_F^z`D;4q;$->Qd_VhVzN>N3L$sa|GIU6N&x z(f}E=SZ&k?-g%al9u2sHodvnJBeBtoJ+NGWd=RtlP zls8Lt&t@=Y=9M$W%ScL`iHj*h2t4S1Kwzzgz}TTKMDhYJ)qW&U0@I=d-D3zW%5t~+ z0N5WIu>Eboh7z#8Uae&}e<~#w2x%fg+IXLHTt7ZUhct{Jz2(s$_3Df+cWRkg#Y%ID%G~qw$EUFu1^s+VbgZ`}Aj7HTD!PHP7Js

MUz7%0!KGpg%(4{Yl0(&=Jl^!8*_b}%hO8GK0xktYY*~PIe>QJ9kz?{QimN4J*)Jg7= znmv%0fQm^mk65u(s;y@iEAp&#)K4h6qA*!>5L=A6e_q4=W5e(u*6blbD{!(fb0Y;B zFK6=EvX~EMg3wFb|H)chPI#9qebh-mqW21zZve8BHR%)lF zdShr4SH)rJ_b23hipZI!A?L~=euHRD7ZhC}N>F45&=jQ*jU3b1)sVrmD$K9;C-0J6 z>rMO)_v7${$YB1##4aTL z%~!x0+=xb$P!XNgPUfju;l}X-0RzB!Us|m0Y#Vk)5q3Vj*NL68LJ~W_81@=oWZI9; zRetKL$0>K%B4f>zfpz~ydQoYut-j`V< zvE+0jg)M5fU*eK$Am@7WBVv2t(qPVb%qYDSd(cbq!7hcrJ&n(=%sYKN%uu%vifanW zP140e&n@1Ey7^m0>c+kzzF2|5on~3~+cJ$5nPzlV$aybNv^>*_&wbdy;G85lZWxC&GQgupLw73dGJQPta)cZ z9?d<_94J!j&%$s0A!4We>eap?rHD40s;DpJqphr8`#h#GgRzHLWEq(GNjoxMkY+Fa z#*t`b8D~;fGWy|GP3W2?)f5YK%}W{N zwTa@CyeybZ3T`0PwVOrHYHj39%1?m;MnP^-h7pZCGfRof9ieWE$GK40`jd#Vu04}} zKRh-qU)$uzx5Qu^ zjik*U##RYnYd`c_>Y3S>u+>4TbkJR>l&+V$?SCf%@j%m1Hhn{r!%N>yq2g`Rb9En; z@mf#i9H#>K%oWI2bkCVzhZeAFa9MeBuD(p3tzp8PBE=Rt^Uh|9xiDo@xNkCHW01f` zl&H~rM2%j$mF^Q43u@HuN2EqzPMg8|3m!B1Ttsex`0ltQs$2)xvz4l3kD=L;JuWGx z>yt%%VC6;^D9sr-ZD34r*LHt1DVZEQY!e5HA!nE^X190LMj|iczNFsVN4?p#Nc1N9 zLuU*t9>0Y-#!}IdcFLPnYGqeWA|$P1mSszn`z|*B2%zj6s^s5P$qMmf{%?C{zM_V) zf!R{embuAhJZ$6ROLUH?D|RA?PJ7^sw_!5fiy?Y4cIxPPhqsp|CjBdPOn&``ycw)% z_KTwywDVVL=l9gkd>Go1+)h3Q;b7nrWt*f14j+E(7eJ$n3Z~20JX_UeUFZv-Oo?Mv zXXV)_&EU(vNgH>^6xe%*w8R^>*<5#0XP4VMO`k(W^bMJX<6TXk?D;tZ*fV69-9DFw z4`@q*_gfclmck|(0d5*}~zwL`x+WXR2DAkjfx>JB} zq#^V3r5bfx_6%Gf8SzA{IQKh7OB#kPIf`1+e5bP||K3*%YA03IFDqfE2jcl4y-h!K zcWDzeN9tUKsrM9HgH*44epbEmWypti@Q~;gZO8~N%`*~Lh=={*p;9~?1P{~2!_7nc z8tQg7SKZF$Ml(+E&|C{9Fp43#eCesa2jRXmK;oi66sasIw&kosRl>-~W{QfKFA8}R z{=^HzQZTIgf(6W1yYtMBVtld~)>&8|@nZ}2z=N^_-W-M}7RxY_?|w>3Y3Lrs*XzE^OGcq?@%7~VJ@Po5H(?cPZ|(^$vG!c+BS<{| zet%v!qd3{wmij-_w}l}zPOxV1qZ1shN*v#21VD%CzQAfZN4lRvt7z@*&Q&xZi01Qf z@-s&r?>AI1iLFIRjz_bf>{F~4CA{5*78a7+{1eet#|WrpD%DbnwY1Y}S=$dIprH0O z!MJ!F+IfW%W9fZv7oSuj0i}t#&FEYBf-XLmJF!0nx{E>c0W3C+8hSxT6_U?0@;>&NfWdGzZLi$69kH+}Vz@ot+&h+c7pR%G~3Tm4f%cRH`(=1P1x z#C?q+-s-6C8CZ8mt?sk>IPsPnnhK~uKKWS70>PoJ|$MLiBr;sC4*9 zwFe~8F_;rvVQL+@AtF2SRih+{*y4Xvi_cGK@zP$-79ZG;;(5pxPsvd&E-`ce&|B=K zti$a}JZ-o4wXd}HemYvV_fUI#52N-DxY^m>?|VAiJEyO`z03E~+FN03?+Ja~+Uus} z#oNN&t7NOZiT!L}t{G=7{>#TDWtiG%;qL8=(dC}p0!O!9c8~Yl2Y+c}zeU$#ayRR9 z>~7-OruC1~fd1P@!3kD|vRcG`dC9VQ7ba%kDV_}$v$XXJuyI&GZLg|;pzxPVBBvuxuEQv0Y zY)fL^G44yE<4W0o(BGoQGb7ZwS z^Co+&BFakAE3cWyx|JRcrRyVwAo?P|`qBy6B6ij%W}rCSi`t9ga}AP(p(pzFC2}^& zlwna#d!8!da9-~pLAAJP$>*%1c{fdZEKJ`FYz2g?@mB5)hy`^}vF?7l7S>%q?WS^k zC%=JVZ7Lz?P=taGMMx_NqPuuy3v*W10hX#i{e#`{Q*!NGNQKE5&taT4ao;JxLO?D6 z^N2ism@LG8If?|bclqVjrY!M|*iyhWz$lVkXB&ZL#Ws#_FNwmL3cDvxG^7SMr>s3X zDPy)W=o^Mf`6M~@ghNODrc`9MnD0J|D#SoEq8L3+g1D{=t+qrDsa~x<&7e=kOg-1O zHGeUjrLq=gNxXcwu+((8n!IBf(bcY#_ei(WR~>FmUa6V1kUJ`Gd80ge6O~FR+~GKF z=3>gFW3`!Et}Wo zF5i6ED@B?hTx8MXSO>5#3_{g`V+5)eD`!{1Rs!GpA68KaE00?Te_pzM^4y2N_?LytA2H%?Ajb|m{blNTuF zUdKv4O{UD|JSEhG34_&%kZDMS&6lDaG^&J!Op_Y3q-M)-C9_2?45sez+9I;GFEtO# zVI?WI>ypj#`U1qu?P7iAiXYIlj&*^jXZFORJRSe0?2YQ9e)Hy}&DQvMT8$^;YW!TA z?AaaA)_68e2PH7<_M=Za%TQk+{e@O9$8cJ-&5QSxJ+&~v+t1T)*bC_kay9KD(sr$W ztGS8Kr6F_TP`pal=a8}wk4iLjh=S!7+-U+D4ll@_F*tgGADlX6FBOY7XO-W>9y==Y zM{{9{{Gv^fuLG#;19zN&m@4xW_K6jeS9rM` zo*)GGB6V(l2J0|d4N+<_X%0}b!r<6po9F^s`qU~nGoMmEJEkTEw3+p_$5qx5Y(kFY zZQ$y5OmM^IF{{W}Yju)cfQ8L>SE*uNhd-G#TRYP4nr$NQP7hxB&h(J_C4G7pxr!Y> z@V2U}n!YkpzbPIK`@ZrWvKLu2)--vUka0!iKco}$2y&MV*ZXT>m$j!A)xup@gpI5- z?x@L=+dVrBVNF8+?CtU_-sdRhSAe2mdJ_#&ZS*jhYh78-xCZ9h{u*;|MLWL%?#v?fXjqp$Q{*t; zZ;v#pef+gnCKpv{*Zu;!XfyNq z?dL67k_(&HL0F#84WRq?GSc#WYW6==j(QIs8A#1Jg13hF7rY%t%?pZ@ zm;zou0&!A9cK;;!YL}6Yr{pU|Z zoh}h=OaZCmWmN6xBcm{&q*k<{98bg>X`&m<=jHR&as_q0g1cUUFF7y3Dj%#C2Yi$3 zgg)2oJghRCCOZ`@Pc-t)6^C!_7uMJ6koMFTjAyV>#xa$IO)g?3i;?z})VLC*g`+iS z1bCRLd8i$=2~EARPturIDrQ5h*s|)`1!dx-aJ$F=j|aqZ;pecNd7zdVugE~Z$59W7 z@UNyJJNhc;iW=ne8)QS=kR1_U&3ZqQ^U2T%5*#3adIB&u1wg`U0T>Mcjv)YD+9~2L zm5|opZ}%DAM26XPaA!k#SeWPEhA3+2`oyJq%BIe2TG1KEa5kV4=yjaJHgpZ>nU-KC za4yz$XO)&-Lxq7K3ois4nKQumf}Z0nEeGtiiNm3zw^2veR5?59g^og$18hY0QzC-6=`w z&a|XY@6I^r&PxV%2No{Hscy(vdU>8<#k7=qU-E#=@5~60^fHmB4$#OBSb0%YGlC@) z^X@UUqqZ_@q+eh|1d}jFdmj#b!gTx8c%4xG$V{9XZ;2ms+$P)9c#o&Xd-~KkO;e?0 zrouTlhQiz!&U0g^Kx+h$`92|Y$xJ6Qd$dAmTe`8KeK7kvXwAg}$pWEp8=6Ax3i)Y_N;~z6 zJbpSbCezNW#1nl4EqH1d__rC7ny`1s#3*R!ckJgRrJ5Q4z%}QteX8^?ZS2!f8>32mss+=TMoyw3h=n$BW5t8^H7%CjCmHELyl?8CrWeK z+&sQ3xbM}@DUmm7V2!ZPiyCn!S2aFPH7=qW_l6p)Txzt|V@*|FAl88& z^8=Oa+^{>R6nJ9o(Vt!-qtWmwin-hJ8Cb|MzC_9*0k`{f?Emb7fDy}*BoAFRUMD!y zwE0PuS#Y`?Mb7zx;KxU@&c1yP?L#=+A90v7Zy~X)!9dQur(qZUFOsl~14LEv-wg!- zCP4vt%z-h?bh zL&oKh`D$6Gz@Q&?vLC6$G%9gDFy~Lo5?_e&iIZja0B(Gg{c*~^4YJqB>@SPAgfe&d zopNT=^mrJWa*E7Zcxhf*1C&(#ak$zY))e1tp$lkaAxJlnMizo*@#tCS)uSul>|W;) zjlAzRq>&46V;Tv(0B~>O7rD5^dGZznQs6gnBBVUTc`6{!QSxpV^WbWb9o(-Sc5o`8 z;(g!&s$T5O&eilHtOJiT7dK;ckY9iZn6|KwXy0#({YD4zJ6QaUxee&5-|P4LJ%0M< z_1pjX{B8g9>iA5(iCE06D;%q6?p9c@FcV|eYj1f%qE{fsvq72wTQm#}5v_<9{SQX` zvHtKAhz&{zrZVI0$~+dhf&bL=X2~}(N*QtWt*3Z9Q+j>RV6Sw~qQZIPife5QYdR6%N|yoNM3`oMf)qFHNP1_4yq* zvDV$e6HCB(k>Qji%EVi zXcLBHqSPDff^KLx&CFA$orGW)GMe;+9Fed}OMtUB#*MBVzL8G2n0k&p^|qLwj`dpy zYn2<6-YXN`=B`m?=41k05}*@Ptm1sLT35`GA{=@E>dG)EZxoB`Gm9HC#zG58o{C;3 zE@Y3%*oL|5VD7G6c)}9DQ6~N_5`VMB-?ifJ0r9t3{5`$o|CcY3@n1fwblV+t{+0q5 z0eU(UFMRQDbPapTIDdKtlwnOS`kMd~#j$DjBdyS$?RUrzySCq%_Sf}mjv3yTehtEF z16-G47I!nW;;^ZXDEn719J!dJJLAMJ10^1H{&k++|%pLua=e|f!jcM7x5{QN+^ zPyp|ecyRok4vs&~a9)tV|DBJ0x=OGo?=R)rs0fY{;F~(z<>XLHs+_!`P~KSYs|l9B z+Rw0n|GuA;GM}!}Zt286fpD#DH6M7}XIR%XO#$*7D+S;AYrTdho37KL-?3jBLA5>~ zZK#xd$&rKx6q%Ev#Y|MIAAVpv1+w96}Dxta<&(n{JtYwZ3InGFe+xus7wufos@1 z4JsV?&|EdNnD=k<8tQVc5dHBSj$!sP;twmi1l#r%f-N^N5o8lJvy1)fhN3_c3y?O2 zOFRIqao9FOVy^}RtvWG8!2UK+5mamuYYPp4gW#{r@YMP{C6+5F-+tmXIIMJwxqV=Y zc)(hJ>iqrt#T4QqWG>t2)g%{k{kdMhGNz=TOF)$MT*A+2^^QJlD|2BW%Hbjc6(mqe zCP?Ei25cY7=}*bIH|5-$#!=)%BgRrVm*>p)N-Nu%s@{Rh;~>jE5P-_e_qFG1raDie zKcw%A>1BF)wmD{JJM}7?jb2iuVov5|$8+0qY4z$|9iVgvTX0~Q8?o}Br=XzdPGb~Zzsts3GlYacz8BM3OS zQ5Q|w@k(k_Kqw^ovRV72jvD~OTX~2L#!)UXatTJh_9-z|0`NNk3{4N9y$%3wx%1?d z%YzJUDG@|%2BXBYyBv~-(jj?h;!rdu#YsE6qj|=^NzB27yIH`CZ~sS~P6ZnIM~zHH;`Z$S zUnDO3NASEU@Jjv>Jbwy2;~&BEq`+Iz@&9QBGQ6@qh3DT5ybM~=xqAw44_9N?5HEV)pDs6!YTv zN76?tap~>#>qhux9gQw359x3?vIq|1D7SV<6{OX%+jFbOfq0BR#dPrwi9U zBmOqVkD}v%WC6>hvICeg{XjF57o+cddOhyHopSCPs?FvplMBAH8siDm+auJc0|)iq2gJtXMjK zu_(LFmG^rPLNPf?{O1F~dV3O=A9Mz?fhUU}bKrk~4aBU;Wa_~z6+YbfW_R%$l38sR z>*l6jo!E;s{W5h8{IFBrBJVfzYqt5^?nl!pWjo8~rMX|-U3pUd=lf6_>zzlecXpy1 zI(Wi!M-DzB_YVTNGq5lmasanO2)9R0LEK&-e$3Y%lzyZNr5{mX)OW-{eDM+~`LiE; z5LPNzVZ_Vp3fti7tX78fdI8!hC^$r<7>U6Q#eChLlY_B5ei}(S4B}}}+s=D7yZVPz z{#-d~)%i1VbK0AA>XZvt+$$>g+!dv3ef8+022~$&D>LHDm|h1|S|@%5rGfhKxg4u4 zH(FDbZ^XLMvzSI&hG?$Lk*np%l{rN1(R@`eCaNOhEgw81agN;R#z?N*Lb!@LyAaeZ z{+vsH^6mY}*BTSpv)1mu#{sQB`Kmtw>QAohPrhNHcn|u+_c^E!5fpnhZ_;Rg-Bs?; zJ(L_RWY%})2@rpf<2YjI&1gf$-eq~x@Q{O2vHeX(lh5E=6Qw#`sJ@q>>U7S$8x1Mh zQR>%tWm`q^C9M32E+l0wMYO8q{UKQmrAp4FrUfGDkDi3UWS;X$LUi3OOqfnkRps99 zWU;hF!;+gg2)yWjC;f(oYAw{;d21(YJR|~YxDcsVdQ!@q!{1lX_e5WHZDR$uS)sjR zq1S$Il?(uGrDx=#GT+`F`us_bl*sbcPb$4abB`i<$sm=9aK59@(#iMSJ2ImHB` zkXslcU#=)342i6wx{8=LQG+o62Iz5j2vqjQFWywsBl7ES(4QkOC&h zDmg8mrxqul!O6XuP3GiroE&6fJST^&L5lUD7CiegS{8wg9DI;Y(n2V;%!qGnO(uDa zJe8xdr*tl;Jdm$Jm`sGte!uZr5#+h!-;OzsN>4kwiqlCsocvpFCz8| zwVkX`?qr4fPFASxWa+BW_T55BO5+er+FBlHR2Z<92a00@srq^jD|oRMrrPL5J(EnD zwkQs+VDuIe<5mHJ?zLi;j1*X1bs65e^=i-6z|s0GgLIF}YIaVk91@k@YD8zj za8Dk0k%PcW5Tr7axr>X=uIw%P81wMFR~h1$38I*|h;y+ba~UWypTJLJRNARm9KlZ^ zl{Vsvfq=X1`ns^S7b`^`-5YhsPG6Dj5Sfu(Bs4`%)UV+RAePr)>R1*XB+w)q*|oj%tq3kGAu0X zli_HW$uQJqBK4IcO=n2)bWluwJricM)-iiq*sz9#QScbAM6Q@N8}h`j=4A8(*Vz1!Lo9D46m|#-7 z+h#hfUQfLGPceQlIdkSO=dl6g2P*rHX+%qfhOXb2gRYWut60qI3k)m6QoamJSh)wewzWJFX)hNC?Wdb z7y0~q=I^0UQ$w4POr|)?cIFS0hk>2#4a-Ut9SWBnl1L}*wO3Hl$Vrd-A|C)Eo<&AO5^Q{}Cx_=#>N$$Kc#n$+-g{CUL2 z+we5~f7Ee_Cl~vrkNC7G&Itk87=H1g5!$FYm zEYO`(T6-r|X&^&2s)@X>7uPo-2spBRute8#O7*W9#u&yB;R;IGA8BtVLl zf&A-^AJ7tyT=R~?ThQ!z1D&pUhptBw{s7OoQaq!HA;{|CR~k_yaH)}N@koLd9_U<) zU#>$!I7NxrOT|$*fR_%)B*@6tO|&{GS0q5ut^4!4LH7z`Bn3{eC+0!l(FzMEUSK~k zZulto+VBCKzd7#@#78h4K4Jr$_}B;gKoV~;bxMPPj9uM#hl~RTIFa%GT13XFe+wD= zK;5Ar)#>v9CZG>y!M73<%{GrXZultm*zmCh`30E6DOk9u(20fK*!Vc!7(>&|Kf3J> z0|N@37EOeUTTv6k-1i%afR}UxxT#XgleW8RDAg03;Z1z17T&aO ze%=FMee#_~gu5LLXe-h!o<)_mwWiXCi22YnHY%LCfxUsExCea)b*C@iuZJ(}8u-TV zk?OP^KLU$I?ePDzJ{|n)b@-kCNV>we>&vI*CnhcTxv8}~wvG#*@HzLZH1O80aD|uZ zeyMb$=1~J(Y?!zHoQYT96JM2n;Yo-70GJ8?p;@}`NqMZiB4_E4CZ7?{24@+h_lcf9 z08NFzf(y1QvGik9dKXF^yjtm>UKFKaX`v=4NT|Hz7?vDl6Qmbj&LWN`aVF&OR6;1$ zP$rYyvf}#M71ne0Lt4rEK9(hMsja4FT!+o5)S59}G$V1W;DU%{DuB!{Qszrf%tL26 zA(=MRXgpqg#brDWqSnor zme-Dm`J6b3D$0(qM%jkEb7xgEmR5UP0q=f^9qme}iEP z4rRs;=_$k)C^ridtw@ z&hdH*N`pJzcvJfEI%C7%T|bL{%hU(t25QE4$2$jPkZ49?Ok%V?Ah%HF_m6iBhyi)K zg2Y1&$b>w@um)uG7C9jEH~o|Kv+;Wxk1JbT#=}podvl_5JkEMfV00fCk3RPGQ~HB7 z9+^~n`9$Y<>m4fnamskjc-Fc9{DsxApDEEjF3m_kz1c?4K4X3{Wk9}ZOeJVvQRY*e1Jcx(LeSTNt^1rjjgG0g}IKep{|6AqU zf6}dv_dicr9UDGxb9K!6a(Ba(@rrG@%Bhu0i=D$YYE|lRab);wiygz&X;sQ_9dVOB zTnkM(T(_)Cw|;CxRW45eFjVM3$1HxzC(a9B(|BFK)@8g}nSV*e z6}6`P8*99LWTm*P*g0Oqpfq@DjSpt4p9*ckR5(W9ZN_huBg7%>EQ0+k!p_vsKhFr* zAfb%E$vY#Cb%oh?FQh*q^Or|_APo&jgrK}CXWo4VE4!FKciMfUR{WUB0#Wo^92-i0 zg{XWE))c03?nVd0r7f8%UQ1$RVnpIjbI_wcq=+zj&1ODEh^^U~S=kj5`X_VX?V`%| z=6!;ZPpIaS7^9>`($8C7!toC|Z?9eCEvU0q>vFT1dj6<($(07)iamnol1oiZ<)1IA zPg0~O(T41?pb@^E)x4N~iV6GQCZ9DeB1SBrPmk*joF1Qza~_Wy#hk}II-buHw}KG* zcpQB{XT?EhIS$d(>9QCU*h?tOn(djW<(q*HS9uwaGUtEKr(;gR2HZsrxHP2!he8AH zfcbeV4`zvZ(@!*+Lefw>aRs>j#q%ZHyb@)g1kj8Vvo|9?C7EP#-S928;xRnhg(!RA zN0E!}mpCnK{`>=`fhq$eIL!3;y|w{o?bTPlqXG9OE~DOl{X?crO72fwl$cKKTJXG?AC+9RHUX5%Yd=+Lx9s)UJD9 z@G%(}zb@0zD9_i1k%kte;GgBg^|>qz517k;bNI(fV=5c9J3X&w`3Hth0b}LlWR|$Q z*z1cSdLRiwdF*!jZ`|o7fN;n&p1*ZZ*%SH5gu9*a8Zrh%HCGDaeTeNHu2$Z4 zzsuqL;~nMqcD(;K&I#|DHsHbc^Zw!YasTl9{=50!eK+5Cw6^{ItKEFxnEv};_201I z*G_HvZ-|6)R#wBESCi?N;U0R8X^av{x~PbJkrjdcLzQ%n7@{c1o~d^tZRD=0>Wn#kVJA|QK7jqGg|yv3NtEQ4)WsqiDW zd!%;=5HEkk>Jp*EL@5Uj(Y-Xa=Z;p6%VWN@rB(WfA0LX&ZKg})t>)L{;BttEveh%d zAt|8;Qv1X5Q&aXwI#fWicBE7!O-z!KW4r*^k_DYRi%=WFv z@}ZE%zf*eOChEKMzSZ^q?(df0UHuZidnPA52mN)rbNL(f|I_yWKft&A*~GfZO8_5p z3nBC$s+N#rs@-{~?SA&UJ8hEt@0Z)}XRkMQdq3;EU)>zl?pG&z-Y+;Q zv)lSEyu*g&x<}L9kNo>l?SACPR`>gnRQN4RZ14x9g`cCte{%Qmxswm|lK1Z_Qss$D z$S+A4;|~up+*pmY@OJjJ-RHNM!-U4tT3o#n$}fcCFTXLTEbeO1o-HXfP2zDM4$>N``f z?}RKz9+{-t1ceJ;C8|_VOwc5M%R;|6A)3n!C)$;CA+!HeI{?nzMPW_le#|Lm9kV*F55N*&9BBMS;v)u zKic`In*-H%jFzZA)RsO^r3-Gkn-96=($UB*Un+jgKc|z^m_&bDu?btO(eZGT&p$Bz z^mV2Gfo`ctDKy`RrXXeM%~Zq5AvCfge~N7IC6yMa@WJCzkujJ!*a6~>*I4R7MhqTN ztiYp+wMMa6LHhVVY0sw`G17_pnN|{8b56XGA zVvVm_gED;)p{!sqqAZs<9zvNYc#y5&m0H0WueNgJEmWlmhw1C zZQsov>M4ixN@^OHK{GICA6DP2g(UNxr^wb)Bvda;W#ZD;KgC}EDU^CmS6lt~=?I({ z;;;yEJb6;=A}f^^DTRqnGD*gLDBPp<$L)Ewvy}JvT-wml27Axrd9o0vRFmKU(8(|Df~RFk-}Nz)GRkl@e#z2dHOUag^1nnzVdbInhWcd5bw z`(p*0+tvFddm-ok?`+E*ho;&857qE{5#hJu5XA2dhrs?nmGNtzM-w&zensOKo#>7q zSN?Y|#Xo92*=M2~1Xun)+sXeQqVtda?EL?R)?`%S`1#`;l{opoY-|1o_h9~h+tJaK z!w+%CZ^0oNe$OTRZXJ&J{XzVgcbwxof4^Oi^Edae%-?4_y3F4KSM0i$@8RrI`N2AR z?K8M|y*7hibYeFx{FL@Romw>Q`t0CnU(4|H?R#{c*1n0UmAK9?RY83;Y_5JU%Oc%j zK`+B?thZ5C(MZ^RJ=pSR+p5}fWJ>$o{3T<_H?!!jDx$9>0b|J+8w{s?Lzd)6W^}ak zBkl3VErF)*!TRdPG#rVv#DQFO{(^dcL9#{s)*N0iX4c8kbNmHjLho!(p3LX%L{H{r zJK2+YBka71LmCId&)~*k@Ke}$ApGo2;YE5j!WD9&OCxN}iH?m27#M-!AZkqh+Gz_% z3t+v)-@f8+p!gdi{=(vKg7`aH{7v2o{<%F~uh)lu?8Spuyo-Ms9#1>3Hz59b0*-&} zwP(n0Sp9VhT+@!nQJIEqn$)7@oWA8Rc4MGjK_jUl#Pg1jpQq-*=yJLO+P^aVJ?F8^w_&J_JzdU1fgO1JuIo%OvN((4;;`}SOu zRo~-HuDyi0wny?6c|L7#olgVu{7Lm^#IG_OAAi4Emu8}R(*8|I#ZlWLm@Fxm`B9xI ztBF((3T6I)x#LA_H(m%A7T3?+$!ZQ!^jB7Zo5!0gGWnwAwqb1?kX+?|?JOdNZ3-n9 z)~eZ>@gCnbg|wOxZ(H}pnon6FwF>&B3aXsvp9}|f;JiSnJIt2uAN+r;eF=C|#TM{Q z(n1P_mZeZ+3s9hnU|B5!nrhLY6$2u#0!o#|qM#L8r~*Q5(biG{MZq0+M16t_n=CE( zpx{DPTtPrVomf$PF5uGSpL5R4+_^V5===Yl?+Z<4IWu$S%*>fHXVK2b$z<4_oI(6u z2E@ObKI+voDv*?kMd4@=Hm&VLBsp< z%$czE0Eg%)X%TE{_-%YIIX7o$3SN*?vYt#eGwc5)JdPkIjxI)~oD7SYjy$zuqgg?{Qn>L_!l^E;$I=I8h5H9lJG5ZJJn3#ounwOk&Ukyh@ zECEq8$aX%CihMpQF(n<}+U&CMk#>~u@EvY(ke%2=8fP7)Ug6;@Xf5*kuZbnYH$6$1es;@_ zh=5x3nz^KG@fa+w-#;}Z#19__e3YS^SlB|T3zy&Ok^W`3!6q|omo|vx|Hwh~=&4S3 zQ$Y|ij=zfjkupL7w-r19Uprrp^sb3+p{ZLX9p<9{F`+b<%N`Jx>v;rr_I%o8^2GVn z<45UbxV_SL$AUAc{Q;p`*u1Mmfb0t4(ky@*qi`%V`bR>UOhji7^Ir-_>YJ%DkDq`M zmnk2<6i+@|2daDBx4F;fAw*R{jR(#LuFUe**QPR-hRCnJ2kZaBFahOIiY96l3W{Mn~Mc? z)ObNN=qX-A`bnfxs|bicQ1TYgtz(E4yS`W;mF;;rK^*7GXQR{ozA9yvDhR@auIYxV zmTa$PZ4Z|ca@ZO=GwCUYj@}^q`FnACij+LoWeumTCX_oIF(t?$&qh*W3WMx|*Ay|m zAg~f-pC8v4CLbIBy41YIpVRrz>`GLa_4nVyUVbLQ+}V;}3`eAU#%G?AbJ&Y2wt8Kv zZsX6i*M&N(aIgb@;tEsE?G|98JC0xy}Y_#ug*n;)WG5M}KYNE+5pF>$I(`MTu$ebQaWVTbsjT5YwN?5Z>g?9DE6 z-CqpsppUFSj(4)_RL?3yjpU#?JuQ3^Mq?jNR4_C&&g>7rUoCZ6^eU&Zk!Bw<*!$fi zq;su(NT)uOMD^jvQtZPRvkz6BWgkjAV;{zZHhL&2z=A8CuyfBGw{UZZ>qEJy(zodc zL$tVLB6=vVc~0)#SmvwmXzxBEw&z->y<28tjk;IVavS5;((}Kg6<17qWaxgiru=Nn z^R(spSxcWp(l|g;KjBxQBOs|lL(<+m5lOicNq*+-EHM)9BoQ~JlSJI5oe*)kp#^T~ zT{XbIeq405urDrk=qv2dYbv=)Y3KDf#9)D5uPJ4m>7?Pafa zwI|<|d`~}R-P0XS5td54`&eRUTGiQ|=_cbBZBlOY4s_-Q0;TAk&nV0gAP6tb-%3!m zi1$JkccU3vGFw((Ypsqb+PBlJzE-QgR#lI;OA74#1fg&Ced;(u{RJY0l!+=?^!HF_ z#Y3PDfGBsaRt`J1UMsg&mUBE`qFq~-yJ&h>@xQwabpEvzlaxk-v}Q>nsot5L@tDD= zxD7H$1=cX+J+IA7w`3wnfR;(jw<_uPN*Qb~VBPS}1DSTb$;ePU26S#I?0E=^;u(wTx#o1I_ob%@6;v}Ob3GiR3 z6ooa^+zt8#42?8%RCf6);yzcRd60{@oI^Xp1&6W4PUbl(YBWj}2k2Ip1%Ebmlji(# zd$g?{oAzt@OMz;-iry~~@6C@jF)Ualwpp%nO+i9n24G_n-3LE=&0P8A*` zdV1))CI*QOWjqYgDo`kC;Z&5sF@wT@1!MFcb6mp4&e1qbRX@a!Ik9A}{F=Pu0@;_l zOfEx|oZN=ki~3G1p_mR>m!6+Z`MJQAx}xHL(|sWLkhx4jV9b&_n9CH6Wuup=s?R>? z)l^6|Ac|r|mNeqR=}D&Trk_Yt3-62Ph5k8(UR;WA)RiNW*_gANI!G67_m`FC{Uw)O zue73f`*UM%D0=-oOZ~1<_0`qr?q1H1xe8v%_qk$WjUUt(~ z$+=84)6MS$C3F zx9tomm^eO?pv2#!-J3prObR>uw2>^sP}r5;rd=tywS8CmhDcBorlzHk(toJPMM-&V zSu2^jV!B)2y%vj(Cg4NUwr7d+3{s7KkAx$``|rVfSq=po;rst66i~mmANMX~&_AGl zEp27**GTG@y0Yi7vsEYVY>hn}N;cYT|9+7*5)yH9ofXCX(Xhnp+B+a6A$bf&{$U zIDvt@vRw+o7zJSsLCEGSKi3u0-Wk{b1Lpca0)tCRHP`SO zRIwgD?TL#Y%kp7oBZgTiY_dBl;7Voecq{h%vw7boy^Dj`u2dd)Z-e~;Y}2Ph-`?cC z>=0S+zugju<|2(~+RaACl5^?QLVj@ecO=Q-{Aw$?B2slpU?%H!kR*X10TMw8geW}W zAti9~vzK00L3i`mkNa@BqelfSdguU&^J1#&y<=fL0l^styg2?r8aZuRqD1*z{)aX3 zOZz}SI;<}`Lp>kS+F!Oz?o5p-;?ZQw`_g~&~1U_byUQXI84& z+&`RW8~=$N74gAaL~yCVIg%jHmQ?$;98;h9=a8!H;}|MK`hE z?-jhKO?cu#>`8~}*80U}G&?a*gGdQFoQ(6-xdor6>(8foS}yLPC@*AEa`;lRE(TS? zC?9+6GbEMt(3RpOqITru7LJpzv)8FWfSFYDwg(kGa(aGIKHErM30@?Bo9Y&GD&9vw zm4tdW!5B!bpG`toyhr`_ThpDtlyPqRj82jOdkgnavzPcDpCsk9BlWO_0RN*0R3C$^ zcumCIVO1YTPgT1s_R;;RJUjbDqht8XC~Vav_p?3~>$MF?)J-Am_)pamEdD!vV4XfZ zVMQOkOu1g~MQV#BsUJc)-}2v~MiIFO zoN7)Y3>*(l`x2I=US{)xuJ8~NPr+-lY%+uA6NzX^8xSe?HnB`w{ zE^m_;pkx|F6SLN6#MkJ4NqiHU%GpSwgkCGckD0#zNYM^Q@%3 z4*(HNlGU$DBh(45xw)$~sek=ax!$wXLOmZCmXYN3{pkd|eDwF5 zJh(gMnoIc`p<$Ox0+TA!S$*_7?B4gS@35;N@z;AT@33t?B$7fQUX_&M+NerqVvV?! zim(j*9rn==q+=i?fsPM(qz`@g+2`sfhVPoWV-3m3lQqTR+Fi*+>Iq7fWRO*=O+* z?Y9U;z~cH5l(mg#h1Q3q-oo|-+Bk9cD_j~O=x@FmtY;(bz#vclvFI;ae>OjWvC@$E zZWjlaAA5_hCqvdG$j60RHnLn_%uplX(iZVTa}oR5tM{oz>|?|471Am8=2nl`_?I<` zp_H(4pHNDuH&ECOs>z0Q3G~P!$@K@=70;^qC}3TmK^6%d;HknltHL(DgMs^m-U0Ef zDvlavrY=@s1MJOJk%*$W0`}+xn&zOG=A8G`yDUe_*+yZaZ2P{au|FDrQ2idg{3FXY z)I0%KQ~FQ`^(y=Quaf-mAxkhSUgB?|UdQxm&+=-Zrj@DmbMhif0f^H4 zncswZRE1T?kSb$Yg&kT%_a_Y39<~cCX~; zvpvXlq}BT*pGGAfRVBJoiFE5-)`6c8{IQoMueMq$c!gZQ1zCVKq1|Tl^G!5Kq0O}u zy=n7-@!f2l$3Z6e|5RQa$)8^BHkWDBu^xPF&|Sy$RIXz>QNo~jix-&2?wDR+oZ8FR z+k51y==Q)I*3#atI7@ra8>eZ{It#VzyJ|TLgkO99jsMYQEH+ixt+^KeCjZX0c*aPY zX7z7!3$P~6@&EyEh85nmR(RXu817+zRg&8@wM~e+Tc~e_7yd`8X!`G;|WcY6OYFS*qblff?V7(ysgbOZT^x#lkEA7;`GG5o^- z=&Ja#zao*H+V5v9ztwnN9!ukK5C%C%S&m_!;%Aebtl3Ue{(gtcwF7?sSN}Fn`Q?9; zzuJ;-k!QZUz{$T2#Kg&fQNQr>1X3FLvKSF6LuVn_%W6Rv28LZ zQW-zjQHboMub*lKa@*us{(2@(*8KH+)$mXK0d2g}0>52M_>oB({2hM}pYRANs^uWGhMcsk^)S;|`Bt<%pw9_0}mhi$&MUUFLhhIC$5rA$?RKMw}173}uWBW>_v z*z1eqr2>O~i@3jDp`ivIUg=^ngeT7SiOUmIRLRM+qXNZOVx0JYvR_yn_O-VAg}mpf z7ET+<|24ex{vO`{;7@Sqb5>}G-|TV@WU3RBWkxqq24|Oc6$^IP!s#-!CI3D{eNWoJ ziEe5|`OR*bu%6RUd3k~Pd*QA8ds-N~R9Q5dqyBWkAKbD zDBWb&vQ^>eBhCK)KsdRpi{s(J|CQS#%^!;7Akjv661#DX-8c0&FP6IGIZk%3>*lm0w4k1kPQ=t9xoQB zoxsOYsXew_>4^DR!7Z|SAGjrZK{oQ&LmWqgsdQ0WNH!q(Ymp3<3x~mQFTvq9`>Uaq z%}(d@aUR6?;%CNw>?D4Y@`6SI1TKQ*yM-k zbpaDCF)&rvvqX`#W|7iU!P(f`QPkV1QN2Ce+S_Y7YrRd@dwX;x^=hM`C5~K5>K>3N zVtG|9P74C0`~RfgVDFz3m8I~mx&@LR1vV};uIPb0<27cK7If&7hBTEI_-pzf8|t}gw0$;*m7NDunq0xw;8|6 z;F6aB8B8QHnDb{Vn@q2Ju7wQ7DmBNU|E74Fjme;$`SfFS!6y<&Psy>HHX+1zndHD=|RbS~+@m-I&H7cll0N9>WiO^mFwUd| zP<%L)>C(Ih^t${VN{{&Yt~r!SCSYT?x^;nX(t0B?4v*V%VH0@t&TxcIRj;tD6;0o> zoSIHoO>evtWJe1n`Y>ZiEglqoBt|zS%#F#htxy7Z{4X63B_i)FeLJ^TkgUWW>_4*J|-=0ruuDIYUQB-OCcgp zBI5HBg$PR6Q=(TR<#%3<*ok~-Y%#8Fwp1N==~Q!xrDg%tEPw{_;4LV3)r;ZWU@1pI z;h|hIif9210#V`nXf86;n9eS-%tZ#x#VkG-_tISK{2k}w7ygH>=mc{SdflKn!GxiC zyzDY6doImeKFr%qoy>XLz0mBRlmgRVfpmKNlNelSDPAs$my6Qc)Zwf?}s|S4uH@<_qRd0olNa1p2?j>dIPxxEweYB`!z))xvsaqM!;mqhL;z zaMB*24P-&<({M)g&(6g{e^UEMI%pBiphmr?>-sWD8~63a@uN`)4+Rd>t;_m!LCAq(4XSlg%{S_==08ABk`>;yuKJ5C{aG@L6s>b}@JcLA>N9U_Wc*W^RE+)vbZ=|Q zV)lZF>>k8^y`2(omnL754Z@3009rtcDiakWvdIK?I|^#E*woSR2Pm^vED)1fIWSd74@BfDZ|&R%Gc8h z_+P=9t#yS?#?doIz9IG+03hhDnt2526-`mFS2RR~^(#*SSgfdur|^4y-Yj$q(1a8` zw;zY`7wF?3iQ!5Y#ZgbO?qI0fODJsQD+3yMK7Bnc-|VSrS|f}Z^Tjj>y~_%`N|m8v zHQuJOlhzY+4_v2LaGmlBf&+QH8I3&XIC8gKbi=f)>29-r()tQmhP`S6EAaqFvIP?; zVv^b6A0v_aV2sLvk{V!mvMDsPtdjU6?cJ6N7+Y4-UC`D~>42nckD(#;SS4{wS~mkj zYG`)4)9eiX-ab3~a&UIObz6`z!#uAA@W!QK;o84yaU$Ij#z3^A9iT&Y=?|6>Kc=*ehNjPxO;Bbu^~_wZs$VmY3o0`e%TG&=BFfibzD+X<$ljN!T=+mWbb3L9A1` zp4dl}T&gENBucK-6XQh5RoZewHNUrK5N#C0P8E=>5d829|mar5stTHdQ*G#}06KvlDa)h`6T~jPqH)-tdlzv2J0SlR}b*VcBXOGs+ zMfNCNXOA4u6Vtp}bcS7gN^FSW%tUG}RG|@#D6#*P+>UVxK0!;c2QI-{Z3*U#z$N&i zdZ)JD$%hc^nHf%*o@|PMmj~od=^ROW)w+L%8r@T`HQEUp{Um^m4vNtz!W}}Q6beVZ znjoz}NVjR-uOdj_uJm@yU0RH-0=WY*&lPB6 z3Wn(wC~6hE4F8b&YKN`4|wk)#5F4xrMOUrG_35zPqV}Vk*oKqJ`wD z!o|tR(U$zQ^h1Yre9n35`1ZQYI}7{eS98$;VVC2bxGb$$9Ak2YV&OTzS%~X$nw=5f z+KKDOi*a_EH!`;|5E8-)DLfNeDHg3@4o6I}OIt%feTymvqjS`%EJKk`RKZ#*h!B^G zf&r(3Phi19?DUn?>6gBNPH)vZeI9lC>UeFyySiAJn=FMI-b8`U1|>3hPbo_)7nuu1 zCiI8>$lK@qPZ-5SQoQEGcGQ_{hiK=2;5^9iez}rF;t0~>iKdGs=aCVc^T@C=4;j&% zM=@|7#l(5QdSFw&m7Iq|u#c;wvDntnIwre0$(AEoMt&~dO6TJD z$M9V2_%%NleM$1j&afWCaAYMyXE+;ie>U#yjrr7H%dbep5oqNFM?mW#rPq1pl(yNsXB8~z$p(BbD5(1YpOsc;Rh#Hm~`(mMP zXfF~2#bL{XMTX>ElBSRhag|^|%yX(DhqyA_Mk;bhR<}_6KrW})rExh0F2&^tPAEgJ zGvabI=Es*K_%wZ+qz$;1HC%;P-)6gsQFFoQnu;F=Vu8Mwqa=<0#|!tC;v&E(0ETH= zPSe!;C{EK@{)au(6s8GlNf|J(L2G6r8i6#;x8%N-JN=uJ$oH9foo_ep zpI_(cy)F9ul+WghSmWigSVf&g5s+cJ+O-Yd$Pb{O0o=3kp{dl7FMK2xE$k6pkb$PxFJ=A8bgX)(> z{;cVrUfM{L^8fGpH=FPH|Ete)2G;$+( zEIek{a=3Ufg+eagWd!@76B(%OLqF2haVT@1jS!jHNr@ZuvrhCgQ}xs5T%av%Ef7PtF~9BN zh0L*7@;J@I+iN#V6Z+34g>_G{dnXCgmYhn?UIg(`IFC(k={BJXBZyU#&=rr*AZMLa z@Uo&vMA@F?u`ScpW+nxD9F4?o3*D#E9UR(THX7UY`@~*{UM7M9zqysm>!9 zUx-e;+X7bSsH~0gT2`M!JnJLM8f48nz^RIDrmXf>fVGhI1ZEWt0z&lDPQ3*Q@lkE3 z9@pPvNKdr8c%K0Z`oXfcwE_T+)Bp}AfRzaF5e;C16F{&dAhjYy!?OoKO~U97`~{%= zV?zaf79~olMCmD4(jr(}n>~8&6q)NuT7(|)t4v3uD3v819=PC#M-tT)*r*Q4Yq`=w zgUM#wlF>qH06%gE#1%EuMp54f(u|^>4uk;;D^lG}QfDO?yKQhe5|UGS=pn!VC8EL| zvRy(F2m{e1feXbT)0OjJskaFO$wA9Hsg{95di%d&@a~mKOkDz~NDC`8C*m)fhMQ37 zskTKE^_~afnb@Q+yO!SUcP7eBAE{v~QNzn$05!DJ_PIYZbZTfq)bO}Cad<;*3j0I- z=~RCa*8ftYrZ*kx*I%cMUC9u63_WHC*~>XF+0yNzHs89;8u4*JDLjaRlx3gtRT=!F1*j3t6SEo(c*O+CC?H_30Gt7ClzK@qqhASyG)Cf@3Gz~a zHQ&2ectr_#vb@S8lf6><1a(5==@>xnBj9EGejzbmbPs{lDqXHOwh)+1@XAmaeL_$5qgr96HISpOFwspuS7EdR!(-YordJKuh9Z{d}gO@2DT zP<*#0&vG=S7(?EXIls|0Y;(5jn(ep*yKB8cX)5gRrk8{YzFj8AipE zLizmQ3JnHR+kjiPJo{rcmi}%#f+RcqWv$#V#yc0N&NWp|lQxWK$!{(*GgQ|u{6*pw zy4J<)+GmrfYh6O`#%z0vYCFxjKsL5bk5Y8CS4PO0N{vCz4`|nLR#{GxpO7$0xK{JC z8$MEI<5?RIBJNY}{}4C0*ZCiI%dahj?xUYoPP0#Fotn@{OUDUy96;%Ai5?%j%{a`zc>@e&J`=3o%xn17!A}j>0pB*dp_!KEYgE`}CjW)Weqi;fL(2tyN3gJo;OHj&chG-t<1+Pq5NiZIW0BQTHN#OK7c6 z6#^a6DICFVz1ZP>L~~sL*V|zy8`o&84qeh&$2mRsux1sx(v zY<`j4D1LfCwgg4yWY+Pq#E3p%I@F*Y!tYQU|6c{qlK%U zzz}az=|_g+a2^(gux4ciar8!@mt^{EFmG}w?g@AWlNg{Z@K)(=Cw#tT>s%f`&j(^) zp$u|i8)J|ep(RGtAU#5qg-utKKvL)zVS|?^U0PPAlEaLj2SL!G1!p+G$0?z(d{GpX zI8VaED$oY}lMW)HyeJ~SWE%0AVla~nlm|lJnuvLd-9x9?Rr~N1o6Y~Q7k+3Vp5A5M z(v8?YlwVBc7g71=p!{u7<@JGh{BkK=%I=<-ptc8XF5&#@FxSp4vQ& zmiE{8>>A5&c16{KmQj}*+1%}I$Wt4>?e?CTV5o2)vf140XfUQW+^*bkM5&b8u*Rlc zp&i`z%$zIC<>3u@yy9w-uha+&@PQqkHepD}{o_@r3Kco(6h1#xL1N%&800**LnD=R z68}S_77J}fQ}1i1jb3d`JDI}8U)NfxdL0qiOobg#3d1-j0PjF&!;aPiRgx$N%YVLyL@`Ge)Ew~wPEE!Fvfrd^z)Vw6l~_q6Q8F(rVDJg19d*kbw0>MUQCiiP*0@0>u@l(> zlvbkw3}T|fB_wu@h|57O2xW#@mX_=m69mEcq~Rf-rfqAza$eNC^NKNNu zi9?A)B7-!WfFfDks&d)aL!g+@CBu{tYpJ3ys-oU8#aca+qq9~%(VHJ~C2J)FF}vfb zz(Uo|O`@F)dpkKIak!j)Xq0V9f#Sk_P%e9HA+0}(qaj%RT=ua@eoiD0pycf$d5K8Q zq~z6@TsRMshEmc4lr#+#8U_eGZoHji?%gGrdl?#0iQJ`XMAIm@H{=c#xrWNUjB=Cc z*YWuY=zj!BY%!78t+lOeAxe67k|cI_oDyWYU>jdX3ubIBE|>-U4||o-<%P^GS>^l$ z!L);KVc%@LJLUTfV9R~9HS?m?Q1rynNJQ1^&+Bc10)bvR=q^7I%H*IK0^jG;L|GKT z^yFoEUr}Ou@+Z!98oiXkZhv}4+4^Jdh0I-M+3%JOKnqc9Q!K)M8w zRCcM$wsKUVoyo9mFHe(O-K^ciY}L0H*==Tw5zd3ie- zh3Y{Emr=7%>Xq*1m0op0_ST0f`%XPO{{mW^M?}X~u0RsHE@^{^Gy@szgNKPIXguVxmu+(t}H}#}`76-=iM)kLqy?=rKv?EuG%P$weqh)sJ&JboAcI zCjzfd;Uui&u*(ekr(j=0u*<}fI3jfDre20Rbf=qYf?jr! z=y9IqMbMLmCeHcgE_#l%>8n@*af70Z^_2;$V<|r3H zvGv6$aE%uYhGxgJpFeJ4h(uH#;5l?9^wD!l6-YVWJ1;DQoFoK1_}*<`+aO|IHY6+@ zlWJY~-}Pa}jX#|}Otja(vtFW(X9&iXybEMZt;q74Uz?a;6K(#4qUoRWq6J_(Nwy_@ z4P4(tcG}l>-!^y?IcbO$khH;{2sVcWGeNE4h)6Eri@PyQ&OI--uqb39a6bY6ZHImL z-k*ZNAEp8If|bz=>mF&W><#nKKkXH=arBDCzd&FVRJm+|Sj9lUIi4LO__6}C6sc;R zO^@*7)rFel^3a+VlBmfPRKQ02=z#gFfd5Jq6mFlNXjRQf z%>9A@aVQ=kTkyl=S?rOcyb@t~4kVKU>CJ&4*M|cDTzG_K8WMyht4S{{tBEBmQO`=W z>QmX92IiS4H?JYuk##J6AQ8#@W#I@b$Szt9FHVKX{^eetWf+vaWxq6Sgk7n$25mZX zf#15Y!6wSPUr-#xGEb#*>W6dWjh@V;h}!C^f!gM1)Hb!15M0%dqGsQC9($8azfQVD z1S5Qa5>Kl;+=eFCNr@WT2b{3P&KA($o4|b7c|Qv5!azqsci|awdlJyF(OMSaLxuuI zVT!FlEylxyE>uW|riGp0kE<{Ue>h0^<0%{kf8q*9*3Tk=;S#YQ=MrsR9>&;LX}naL zv7k{UcqqL^^vy+KEAp&ANA<6+zZ_WpJdUrn_=T@9p}PPe|A-Ht@~JJwly&!=Wn;q?BW)3W_wU~4m^u9+nj!ZUPpam zeG~e18vmCXR{#i?;>S$(<|R^YfgQPpNNm_P`)=t`9a%sXFZ38@Al6KFUb}llXav7s#k@3x1I^lvDHT|1`rL-Y!%&z;yrhY!cc(&hB{a=NsJLFaiPir$AAoQIm-OiQZwZ-lh^!PI1v9bxo#635AY zJ|LlsQ!^qaQ9M?4Hf0sa9^ZgZZ_S;8*=VReYB} zU2w6b)n}5|)e8*tx=OkEDtB6RF2&BI0`Vm3jw-^mA|dA@=`OR#nJ&j7`C5@_&P7&h zMP6_!5{PF){Fu~PLr}^0E>-V@C~&Qc`x$?VPPm-TAQ*!cve_%{n(ch55Xv`Uefixo zDMKVp>LYJj@qUpMG?RiNX_%RmCz8%GlL|yqTQg~)NQyI)R*IybdaI<>BI({vs;g^7 z(p;6K#Z06k)uPC16HuK<8g2smR3!B?la7d_&Snx5Nr_r>C#=n#5=92f=J*ovdNjYB zUh!6}G8$rK{243bWUP!IV`UtUm9afm#sY_o1brL{_Kb#D8GpvgI2kMB$5h5fbS161*NZI1!%JkDv8)6$g3*CVIBo&RC|-ftmXX2I3OH}UlT?3D`0 zr2eSBfHsX8kzmlaXqKNZeZ#7ycB{E1$OU<=nIxC(i_wdFrI{qRfPp4>xpBPOCu)0n zWmUw!3H>!JV&AGlW|G{{J~or&wztzvlAGgNGf8ftkC{nwqrJyWlG}0HA!<7o+xV~T zqPFo@&xqRo7ql~?<40O!H8p1EzrYGgQduK{m-B-PRaiDw8lW$%=vM=zn z=l96iLcN8hfj{)5ps;}CQo8c4D{WjKyRxse4+6mmCi(lK>WtSY8bJx?s02=m^wyPf z(d$TxlldR^{Co246nfD`_2T)nL@(+)X?}p$4UzeG%lv4+cd@S4N$dTtA>bNlY}Upm zhElEvxL5x~4v1f&Tg_Lm!L8=7cB=_^$aV{XK=iI4NNAhA{~Fv|<2;UfrP1@HLTiC| zyBhzRJ}lsOr9T<2LGY&ABI)h|aDp)j?Q)tuU-(C)vaBR?X}l5e)I^Rb2#Qh`{pP~I z*EE(FuqWea{xYE%VAvwT0uDZhgU>M*fZlsfXg)6!YOD&1m44efLIbMs0@;va^5|!2 zadL8DM*&^GRZ-JD0PI>1hS139XS$5hY8Q=EwdLdSLa`3k3?uu5yeRFjJZ0{dOBc4o ze>Q-$BD+2(3ffWuZNe~TYX_cUhxXGS1dU@TJFe(zt-J1`%UJ12H(XhQF#RrB*Pe%%*S2S1xV>4(%w&62c-0{MoJBahu%lavq=#m7w9Qlm;^yD4je}b z%oyH_F*HwOLLa&5Rn^bebbRQY2HIdUb@9*S?>XX!y49TPZ{NT_i9=O0%uj&ej3{I|AgHhCt4R^toC`y;q&^A=Z9zQo6At6_Ht%8e8N~mekR(vS_&xjVR zZc1gmv2{tjvA&W%Rs6HRicj6W@y6NwlgmG4{IiyS4)Twemwl9f^7yBuis}sD-|hQv z4Er}wf0a7g!>Kl$GO3W@k@GFzR0+ zC0R09f}Kd|w1D9Fp$YWbT!wm$QdcC%YOXb(`|>F1-+B0ShZaF6>3BNuGVCeav_0j8 zzsS^cCE~YV|LPE`x9~av+;*jzIgQgV#_89`>7SJe^8bv$P5eMim6o1K8xorgsVQm? zgY}yzP=dHXy}SM;JgjAJTe!@=v|TH}|gw(nATL z-;36gg&9ykNxZjXeV3*gyTMZb9A3Y#s*l7?-;<-wLqoTPO$kzLePPo)l9U-uAD3cl z^$BLG*^e$E#6=-dQdx7s(`VY`E1Uu9M!GPYS0e*-fd667{YUcR{y@AuvNrudXZe$l zL`9_gGKbp9XcNrpeg92rGHDpG<)q%k3sx5RZ-=C1gQ_aSwo;*&}&j zi_1rS;tzL6MqVhUE;TzgXd)@8A%xGamE3W>VFHJx&dwLWS2+N`6M-9=)~f)H@FawJ z*OHo)b)GKnE~NC86&O^n9RBAU(KI(Pfql%S#BY!gQ8}sXG0O_$$mrK3$P8!6A%LQX$pfhNo{%2sIU9`1x z=vOzsfEq=Bu`#zwjExBz;3JKH(T4xzQM|Ve5ZD|bOjMoa8S;I@aVMuI#hMl>i6uF$?F4FJT#4{=m_mgUcC2k)JXTBRp|AC?uzXF zK0=TI8iK;|7%H-K71$M;?=RFsRYlTly?+dbk`?UBu&j;!tvmKbUy@W{o6xI%3ebPA zhW>s(yJ3slkGupw{1fMR=`X3x2t~d}pRzv$|M6d}_o(xVEj&-7%{aHc>#+X5)~ET_!!zT{FJf*NUEw2zsIu z#n4`pPu-xNy7loxi&&T@Ufz%cYS!UQVh7H4YYsRz2-*q+CixTTjl8rbS$LAIN*VNI zD>0qFFf?d}I;Ae>84J;jwS3Y(V}YOCpo?^wGaXxI%!2n-0`IGjI^*3kV{uM3Z+57O zTPu=qFok3rx$K)(((tJ=0o@^Ht!r&sSme+`4n~4O>hWM}LueR$5L7QhsA3cQ9o!}m2;G?l@wPjd>$8|6yx?M4>7+4MI$ z9(GycA0KvE;vfGgH9P6%^h&$%-^6Er_mBnNwnG+poo(=vkk>eEykevWDu#nRo&Iys zbYvQc)RJlyYUIqO_9J5`bXCBnfaR~RQN1`F{6HY-6``)>w^&L3P1CAH-4~1KkJhwp zJ>l#<_Lofs^?%pr`Mmf4Pk)35``Y~x>U(=>aqBioWeILG0wyL``dh=s$iJpbWz057 zccW-~5?J+DHG5zjz6eczUcCrmJnuvbXdgmH*O>HIl1Q4ak{)UylCD-sgIbEDi&avq z)*`9DN?O`gB>7a*R+-dHCEe3qB>iE?r008zq;FLc_%_6({VHjlOsY{y@AVT&|5izJ z2Z^LdR8sD_BB@*@y^%>t^~Lo2*KF}SpMDnw#BZVZ`gM-@O_awzx?KDg>-5=C;&-~p zm%mf#_wdo;w^zCnq>1E*+2BCy!c;PR*z8J{J0E8#emd9?Ar)QFv-QkWAQVK0IcP^dtY`k#~|2)P&Yx!p<|9s3pA^!P`f0CYyH@foAK>i8v&v^d1nSV<7 zXBq!I$v^A(r-pyt=btb6=O_MgJs&MUx6&&SW*YyRCe`;iVa5x>^l`%c>t9Hii7!aP ztcri=kyb{CB5=gN4ByG4N&J`aot%}#$3M!yzeV4>!1s-6D$l#jZG>K+tR9f{7=6DG zzEfbh_-o<&BKm$i|6V^2+J!H(_?M0Fm>C_!(D!1^+StzFf?| zjE66rJ&d0TUk30m74W4O|FR0cbmd>R!xw((jQ<$EB=ax7!k5PU%U}Ep&QelmBYAba z(Sd)``DXzC`1vP?f5!9A4g52Ue-`jhIsYuGyeIC zf9m=^16M_o&8&4UVA1j=>jf|%U2@9kbA))T z0E;Aqg%clCUKY(IZCDCgE>I+&eSbY(Bk)d?E*>Z|gvlS-HRUsguNsAIR1gMCbx*{% zj&|gn2isdB+UpYfKwMMPE!IZzEfZdoMPJsVo{V;t1kD;VvI6>>&(BjWbrmgj7cF%S zU#kwV@w zCM{b&5ipEbkKueODWowg8;@iS@{~v2QU)U=#F@=d-yq#>(KqNRqUb{HHsEn+18xTe zY}2(0)F`*voum!^P80z27IxS1cw9tNH+x!9r82Wp2VQCFW|zpgiPmS!shiy*|C?NK9{8j9Vkd^6>v{VP8ndk7B@)OR`8$Zk%HbFS@+@Lh?yd!3_*7| zQs0CM6oQ#Ul^F+RExnm4Yd}EZdpj5f9Aw?`e<_?d&(d!Ktg$ygV5j$>RoKQfia9Q!$P<+2f z=gorzxvVwUPi6qu4?X@v1vR1KS>Mk-@tZyt{H%iDQCRp1gqg4JwcD8QJP6N<@G_+d zsUP2q=D*PT#$E6imNw5pjVYSM5@t8?i!>gO7_bTap)q2)r} zCwN#b1eYGBx9OxsFWbw#0_8RY<7mKr7ppy_)LwECFNtSK4@>gRz%eQhm33tjV;5UW z1@!JnpU`wFsWQjjb*bZ#ZI}=SI!h#(OFBDuVmM-97{&FcmUa9bH-4C*_O2)#?WJ!tk?Dfg9-h=F^9yZxf2C^A^xbR_dOn8jhC1}7>hFX4r1VW7^4xgM5-dab z696h=9R;VDnY3Y49O`E-l2SQ(f69mi^UYQHsbPcs1#n^T)FXvL88{ca;&3id-sqQ{ zAM9uy+sZe>ODN%BXPGdFFoHY_c|Gt^_@~NRd0Q9TO$Pg>o#Y+?`NOp?)d~K%nRJ6y zQbYRhbH?d@Wj~q_uzjmM^%uC{= z46sKABp51sLx4>mAU#Oa4HBIXiyg#YFK-M}Eu|_-yrzb!dbL%!SDOAun&?z}wp2-B zwLj`xNozYUaW`>3o%Q+NC^=U6DvxaKsO z=q884u-WD_lbDTr2IAyRG5reFDYoQ}L6|o}xk3)IUNrrK=+(1Xe~#GFQ@%Mp6gdQ0 z=0H;sb0{T!GypkHB!E6d0ADS#6M*l-C<4e}e^w&_3~Wa1R!~H1S&VpN^MBRHoZJi; zPD{Qzab)b!oh^a-u$XFq_ZFSa(lkec3{C^u2_XcU)ewLd4VufP*C6u5Umi8BhoZpoZeER9(ql%lyJOWMUB8CuU1EhWYxe z?lfe$7_snn8v53)lme?+Be{%`^kx$24Zagd?@W#K&V4UR#WjPq+=8T+w829kMnCtyPy&)xC`jq0@8D*$Ia3Ay9M{)qU^(y+JtwtO)q2Lbz9g zso;t!?}(I*4j_WRo48(UD1~V%iVxOzd5qw(L=_#byb<6E@Gi=7(3k{Yjyvw^?q zWP@8v^Fb`XpqP4Jj zk3$O&Pz(D&&eGL9XQkrybHB+!RFv0QHa|)`;g*07-a$_Yb*3UBs&u0g>77t6X+s_R zCL$h53OkrQtQf?a11io$)C&D!J$5tQ6E(weYbCGV-s~Lqi|zSNo+E9{kx3r^Q5Puz z50$&kM})V@^8s%^Xn4D`)&XzVP={6swWXjMMOg>=41~Io2MKDQ`F7)@vujbCb*x*m z+;8^O?n+0gTHE<{R{7W64yCg`h=>R;6BMzMjiyO{n8+XQkcs# z3Wn4sm6g~}n0ssmdEdK1-|MOGbLTAfqK^gc?T z>hd$)%CV4(++>3AB1N?kS7}vc{AZ@wa!BWYE0fV+lPpTi2$h_c8}j2KW=m z+K8r`stmhGdCPC7UfKXY8~2HRMQrV+Ycakg6kFIqvbL_&Of_ElsX_YEQl^^dcY%VW z3tqfr2W^vk*#amp%&pMLWd*uQ#|Y{0K!EbG;r%hxW5DFBGT2zJ#7ifKw}VsbkkYPp zX$V$Zb5pI&TDElqnU6g?nV;t9sTLn5rgozzm%-8XD>t|dJ;neG^PUyXE%=KO#9wI%C7xh)XS_ABlK{)5EDelEOHU;h-fBW2 z?0dhI2Pu;4(?qV97Tf)ltl9?T+Qdo}b6-Moy&=Ku9UoLvwGu}r7+UwpWj6c9B8u&m zppOjQW|1ObASyK#T#P8OZK_>tTL}(Gj>G{UcY%xAU1&(ov8IbDW8Fk!ylakqNp9Kd zuq0Q!h+{1Ek}L#3vi==GiVt7sv`~p7JDJz=-PFWI*hIP3#MN7)n72$efST|@6A+-l zFtoOIcc9(mO4`z7azRIPK>Mk+UuWBO>g%^aYqpiVYYn#cYGV!Y!pjlvnYHW}&U=B5 zMClPXU~{9`PyK|-9yHGW=rpACl&9!S(&xd$McQVdJkhSueojpdK&7W<@{`p%R zu-|Ttl319XmHXG(p$3!yWKXRKGy>Unwj#1+Zh6JPM!%;tnxmWv591{d+_RZgd}85DC$ccyFJ}dGDS9n6S_sXdqk7mLjR}dO1A6@ zGm6Gy#2X7vCc|idl+;e40ew!X8)7YMsBh2~)(7#hebO|>Nj#h)JoGBEFYlVo4$J$| z)rg1kcFnc84s^Eec^v9vaYC0R z8-t&irn9U%EG@L0xX_tY{n|JuGF|iyk?8<%^Id;~SwOleOM96_6?_Z>*e8D&I_0L& zFgDGCVbo~DNWx+CvJT_B=dd3idx$cEY6aGG&~<2LrD_~XCu(8NEIXoJ-spg+a%!Q_ zV<^=!md5!8Ql1)jupy$Rwi&ciT8)BKuS+Yx`~Zmur8+A|K~`^y{h@5lrM8A1WCSb! zi&vJf0a7y(LHsa)Ky0#sNY=wnAc)o$5SmEwMFdfJyBfl!Z4=C(L^3f&B-ZLMt3RhP zRU1lHS!)6wJwQGJi{i_mI9k;sctzyx3cgq}Z8j$XNkJxmljJR!lDt z9D6`Gj3{t=a6sT>pF`DiDr@Cp6Cs#zSOqVHBjwp?S+Ll$BxZ&XW)g1&%*@m<<6aM# zu~}Jv@iby)vWc0?e$X&;?*YWj{4q3pp1;_$^$^WqI zg)&rXkoi(@NzFbF`k77rJXQexT%q-I+#3!{YT(ncS!+8>1%3!(N-Ru%kphU@?W5&y z+GXGS*dlK*sHH^3yp8s#b!3=!!wx*=nI7UXpEQ8SY+iJZYDq~FN27NYhenR0xT2XL zBh>_3wpJdE9n1`&ZvE^ooq>E&2rY79;z-V7yU^v+mFeELbT~xe|BWZfL}~`j+Q?h% z8{sprp&T+xoVPL-Rz7=i0@2Gt!(qh`2CCf}E6OpEd#oXb1UHkN{Uq+KpQ%OgV!nCU z&AUapp2Ok%*ao+h4tlV4L{FPv)BzGT)V-@{sB>wkr8nD$y6IICxw^zT2+r(sh4T@i z72#*EM;e5c^qMQ=RP=K=6(>+@0l5JEY#gn*Kud+g;_`^Da6~jS%nNRmjkKfw zV5Za!B%x8GDBZy3YV(jBTIre`6CDA zD^BGo>5T-@PqxeFMJ?6^j}dCN5^7f71gN<~L(R>vAZl(y0P|bP?juIvR!~1t^ey0J zU;Vd1mx1J&J7+2**ItW^3gWDI zSJDk3pD2`B94A{ z*o&;7bnb=Y6~f5M&p^OfuANSTAHMIXHq+{&G(i<7dF0|+DUZYn{EG&C$27Z`Bkm=f zz-KJun|Mr)@7ecpe2^XLA|z7iP`OH|61DD$N9hHd@%>H?Y@6F-h^Y4kqBPWEcco+L z7OMKQU6HZqwVRY7%q|=}D!b5?x&Yas$Gu3g6ENOFQdtFuxj94Csa3B)0PiIWyu}LM z2XzP!vO^=i8p$gTz3y>(>?l9tHg%Cox8Pr~AX>%~zmspjXS}`!@wwL0c#>-TDry|E zL+>{>8&BG>5Nr)3l(f|r`GCSD?bi>0rVzLI@?VITGtvP0b))7+nMX>*_2ooO>EM`byc{O~HN30{5pQ67IPK7qUaI#UmT(?}y55Zn}7;yL5V8DEGU?n89-2qD$yn z`{mIhNyECDB(3X5gz^)D?W4bj0))EvRU$NWKn+WXq!d4OA<8s$CN__)cmOxgTFY?V zYPf-Sak!8jdMi#t)sjvMRqt5p?m8^%CQ@C<4wc8*G(q(t(suZ6Ynvu$bdfc!qj6iy zj`8&u`qGfo%MZ%JBK|d38W@Z@s_!{wq)rB7vtWS{oFjeUab(8V4LCvdM= zAfSxN?%a#qcdm#=S(cWmy8HYwQp&*;j}%|}_1ed3X+(bW09$qj?E^aut=twUSG8Oy zD%7$qJ}UH>>iK!c2wMTw{AwW!3%GRAD5}YH#B-(adIf)i4gLg#Uzi0Wld+bCTn+&o z>|#ot5J3A{k~(4d_G=%OJm=Lg1!HIm7G7^ZU7vdfr{GgH1p?T}y$!;G;Fp7PKF-^V z^8sZ;>va@%*(;gI6}uZE%7o?&75fw@@1SLtA6HP1?LnxJP1ZwvH}Q$xBq$yyFz`K! zpW0N@4pQwT#wIZh#YLS4hK2l~Im%J(OO&4xX_l$<8rwly7U=^O|jK|4P43S4mwPCbB6=jJ zVQf`6qTHMo_JBX`!npcN>W70w#_Z$0{~?J?Xq{oAMy`|Wv@uOo$;ni5Oo%kd#WO4p zzoMh52vWe_+@bciOm_Y{xwoaNC>2;Lg}~+353%8c~P5m`X4SI}?> z{h(AVZC00sZ8l5Z7aM%+e77N5+w3-5Q?@FNl^fwwmjVgKJn2$@KGqbUy9kwaeHIm8W!D9ME2dDskX$!H zDayh#-MT5AUx5h3XG(rUMi(vYs^v<8K1b!o#fnK3YrNDMq3}DWL@NBYiPE4AJtYoI zKdWxg{Vd-j7MawUBCI`{)ez|pGsb~UrD_#nDfPZeM-pi?D~tWX7K>!GZumP>WqhNm zkB_aa^>^=Y8QOCM#&pw_A;q&nD?kL0YG|%U+eEdOu_)O;ywYV673#%U@PXJ5Xp>*G z7>0U?qh2Uhh2tAqP%Vy=f1nT|U1&p!)a(Aj)RrnC`IMJb45Ho?F20NId_x-HGTm^g z{Y_!uN_>MVb{pb}=&h3ot0-VcVc2}pX`&-cJwiBOwu@Vv8pUfrS?Q7vR)CByd`bE9 zlytlO4rL~=LT&j%&3_WFEv$~EAF{g6vkWq@VVx{|Q5XM}yz z>bb&gdqgYRE_k}JE}8VzJ+k|zcgRB!J41V}%w>OFU_Nh0Ihf59MzUIbtCx05&b+<# zneFFog1y@>Fahp>f)(Y3;+)Pz%PsO30}XqA^NIi(_;dfSUYw(X9E#jqEH2d@E~Hp! znY;adpK9#l{*sahV5YREz_^Q<xKCgQYSf_>iztB8OIB^2K?57( z{7M()&-K7SHlDV6iy$jSLrIGPN}2+8?iqVf_#x+v2DNJJy$k5Oywxu{2TW+NGGz$;P}Mz>Gjn_<{7MZOG6~| zz95sJx!^L&Vaa&RC6mb>>kxJXmLh;|p{nWhDXymqFnAQ#UKZ;0i?JS$@O zcaCx(j`kGW=|$B?l>DGz`S=};BQA)R5&@QRvkMUuaE$+K8rh`YW${vc+k-?C0SF0= zPJ9d7QqDxo0A@g$zY*TBM?W92J406VUJ3r}YS;z+_t34t(Z4T@^`u}&BYOhfYEXU` zu{!*<`2*f+%XY*+LhO&X5wHidSF!ciT3^ld)6UhPxGFsfTAM`f%-O4s6`cJ4K?B}E zH|ijKJ^6m1TMjSQv?i*P+2y=`Ij>$WrVPa8K?ofXfJhpCe9FwOtcKk@nvxDcOcomrT*{!m&fgo0`gD#6T|=Vgin)C?|->^;K& za*7Mx+*TWnbpOi}6QevUPfv>StQ=|?2fXXVESM0r+z^CGKf<(n?CdRBh>602wBKQFO*R(^4W)wA;4 zORb)jdtGMrto-)n|ID+pPh^1qwiTsz&&tii&YqQLlzI|O+GnF_)TszTKG*KI9m8ko-kVYO`bbi_)UIzwD6m}ZnW^5 zTsvC$O+GwY_)Y$8wD6mpa;5N_+~G>oZ*rcikujZriutFEf9~g>C-~<@{@K7k)%>%E ze?H)!1N?K0e~$CdZ~PPCA1_B%3;sESf4cF{*=}Q^1Z8JpvNXwf7yBf{7{ z>wtdiE}W?p??@D+59WW^s+i?mTZxiOf{ZFr@bnUL}eOvLqT@k#Iz5 zh@TjI5zLwSWhZW={pxq#JrvBkc$d+yUo{G?Lq9^t;Njd*9Luau=v?kv;N_u1NgMpB z!@zmsf~EoK+PJ=hB^MkjF2E2G8Kg}R-*psFMRi=DS2p<7%ceje*x8-TYt-LvsgE*w zf)3`x_2*e~-Da*s+b+npwEcJx+iv90_9e`#?oi3}W(~eSWRMfqc$ji6$uYD>UcjRK zAZ1$pAZ1$pApInGy-eu`DPvG1f=*E79G}x5uU(fA(mS0m{I~L6kZ0HVy z1C#lfHkmWZ)ntxUBBOkZyxLBx#d5tA%k>gqgTD0|EbFa?h`K0%zC3+9tGnd+ z>G9Mr6#fE6VH?Vd69-^AeZ>|5N?HXnzd`5?+$&DUpK9Yb?f5l2=K^%s3Mkf`)}}LG z15HIbyQ{0D(T=bM6dS6{!L5Y<;?ND^&7`1mAuPV`hT%8J6Qg{XsAb_|6_iva^5Iqp zJLYtDNaP3WdsD)@s)CXqY2aRw#o-lgvI<#+PktT8T>rR?)?xPz$BHGAxDOBKL-fm5Zo_O^@D;lbp&21 zQkY15(hyErv%NrH#A;Gz8#8lO6Z&2u?i!-KK3BWUkg#;#ze!fukWxQK^dZNw4@Cp1 znLY%XF6}|~d9F+IX9h4=3YhfHXN3~;tWUy~BNeu_(H3(f6zq{Re@gi^i5ifb%`%is)hF&QWod&9 z44W+DvfBe-2z05G{o)UKo4~CFAz|=)(B%fuCmQ}WN{XMwj0?4#v*(&WFi`|$Z(!}H zc~e;W%Zn~l5%1pb879f=XVh_|;GS(mx;V_XE*LzcSxri4ZBZ}4^j++6M9UVvbk zn;v*COaCC11-L z;6Y;`k5sB_D?{q6XKfy&ZLn3F??yYB?bok$aYoxbAAOKkdM!yl`M*2@rcyOvqiIC1 zuvPu@5n{)}i8W9S#}1g9M6?6(Hs1KDH4ySL2HY?5PXtj3LbOtYXymmn-tnCYqF+54 zRxs{5Ta;^|*IB>y?f^ad`VLPDbV~bzsGm0L=>JL@b&NyNq`MBmZ+4C1h^u}z@YTP_ zb#!wQK4TtEq@M;4FoQ)hM!Siv((EXP#2P{vT0F|B4eE9c>c+mcarNc3_=YHj%Lg4w zzdDde6tKNH1^0ssQLH*Zfp3mscm`?lNhg4DIJEaf(S>m=K`;;@08k|aw=H$?m|rv( zT^P?+&qPE=!<%v=Vq%66b=fr18Rsj1DSFCu)KoKG!*Y0Z$=qo_k4La~F_^J&(VWU- zrdhb4$nyQo4hN#-CjZ{tD2+SVbX&R+-M*QPJER+tpWKK7n8A<^p(C9hL!+)v z^6&MFE^~0#O!Hzs?tpPkOVk6zPGbXRG*W2)rtbQGQnEFTGSzrP-RXRKzk9Gz%gFsN zXLbF9G}W+FtQCo^57HdNS}tzC%Edj(gmM(6R3DWVO^^49Pj|x7B;h92dK|H`n6MHY z;mq58+*K}E@f0CeCU|5)*-6e|yKJRl1)lHfQNbOhjuXBdx?Q(4_%(USyCVJTY$FcN z>WeQUt~VI(Z9`w>188B2$vsEQHi{`JgkxK%dvM}vuJ|>f>`y|K%YV|A(_AU|X_Vk0 ztsRbQ^xG7MJT-na_W$PASEaV@h!wEXu&R|EMuz)b1V#zxS&A|vjJ zb_Z65Ld-yanfI`cdyX37*^UVrm*P7^7Cv}?Qi#8Kn(3=8%wEYPShX5jwLgYCga3`X z!sYtwGK*YE4K-rP-E6q{X1Xl;a;^$?i8GsV7zJbjcR(sZ!;Rm4e_v7{!^tD-qpitv zCn!#PLYgboc98r3GQ8Tz?wIRdiurMy8ui@MTBKY}Y!w63 zI?M_4HwPmzNI>NOw=tni4G|yhf21q{x*=vzoNyNzV#cmhuI_wn-`)fvfM=pBjQMO= zW|~ra{iIpHI~X(!$9EexAu&ZCpJ>!pwQLia#lfyjtbrDCKYT>{p;^Gm4xLcq67Qk; zC1et6O=H%bWD**CL1Wrjdp~h)qujC~{S>O1hzqji|{VEv(yLd+iGBVd87I~X>WC1mTq(Iwf70&R6S2p{2#Vu3au|%|#*t7m z&_PPdGIS?=*{)(o!!2z2F9P~5u#p(p}ZgQR;XSze6!v{|6^?Up9X z{Ge2Puti#tpthew#B(abAbm1^DqOHj0cVy86jW^BzH+Irsx312ph*`5?Kt0N45SR;Hr56yT zwTQfdv7}CPRgH;=&wJab$ZwaWwa7|p;c>L^CAIKU3kK|ID&3Qoxs4#HnC!D^&1rSq z(Zis<{KrC-CaQC~m9hZCjE(4}B2~Svrt^D_7NTn9Ky9Ja*=eTu@k+!7pC6Jpu`?YR zuXoBE*1;eXnjbz+kYkhK5YgyxXDM0XXMRiS;c%&Nh(zvhNF-t_7?2>3T?jwI{KI6K zJIx+SfE0?E49Nb!14u-i%Ygjk2FQNHsubOIKxCw#BKT3vKTKw@(|bb*kRtK%M+68x z?f(LhV)5EZG*(tpb7y*moBm5K{q{g!OE1(uYM$ow3{Mt2uom_ZdLgqG5`cfJN_pk42-c zusD30Mm=vc^hYEW-KZJ*$FH&3%$`Tl>@M^>r6j-3(>)rvr@PjYQ$ze5$MSu1wp_k; zTF50ZGpAV6T+oaztx+Dm(}~pSC{&#Q8uDHf=iZs+9;ZpZha@K!*55;@W&G$yqJ&`$ zqsLV%8uK69hasPM`0+*^Aw`6c9(@5Jk823|axNmINM8FPv1ScjxUsIyXa&%B^82e2 zn#J#YtNF#$e10sJVS$ho**9~=86Nq?zpCy2c7U$2PjrS2e#qM3_(DOJSnY9;P(a3E zj<)uopiQ}*cdT^B$I2VNp~?-0ju=Z&)ABZaej@|^{Dq`neE&ADEpbIym*0GNtMx%O zf3Olhs3O_2<+Fo<4X^svGIuY4K3Uhj?oQiNw^BYG(XEwuiFomdv~nLd`i{wO?bzr z^31yi=!5o8SEY!~7si!}j?%x?BJ;tw*Honl%h!>#Bd{f-7Mk9uMl?lZ=aJ*LoZq|2 zdm+H~(<`Kkx~;Awi|4wbdGLiD_z6S|K+kS2iY|OhbFxuj%|>uj&Go02-pEUOa=<(fL$$bftZr ze%Oz-XC4uN`kN@Sgp6=be+{djRh)c0%ZhbW#v6-74&M~YaNmsr^>6KrwPIe8wmH^fFM9&yua+1 zBfRe><(@}1UP6sgpp)=8Ne`*KxyvOVz=~NiY1urzBRN5qXLBa*~&; zX1GYK&ZVIXtf4zpux#~beIF@HppNGe>Am1q@=BNL=XENoFhX=8T}QnNs;mZ^4edE` z$LDID!q(uoUeDswC>?bcv&&>(_Pw%KDbHTGrZL6qOKM=99_pmiwgL>SQ!EFr*5hYd ze(DCQ^9OsPQ^{RD5T66>x9GF!Y0YM`HJkeD*f;$FINW)D>UAve391_7{=UZp%_?he_MsdKGCKCEjhG@RmfsG>v;dH zXJ|{sn;+5D@}MEvQ8Y&B`WY1i`FS#^ial5isqx^Q@@q%!91VhgYgE4vV!zRI#%}Sb z3jFBD`{e6q7^M`g;)VbHUy^O%d=r!Zp}mPdD7NvLM^TWpDLtJ>^YiI;FYiK8y@I{G zpHxMMm)p@69{WVwB(#MU*=2ZkE!W^VRlzeH;OR~9^i6{2tm(QlHgwZi&YsxE$|dHc z2|#DlRkQc0S&)77obeC-GT%;;0Fy+_)YNjjZdn7jV{CSs)l;%mRL|6-Qcc8Ua&M{M zDs|OwwNBGB^-vpe`xM<0B<59>91Gx~g?jxStk1X70=VVuv5W<&(2#Dbx zZQEJ#-z)TdKEuwU%k7v3WR5ad-l*?YB|*<#=P?{>2mize0L5*eN)RqY2x~P6f1m2Y zve>Fdj=u&jl+o0CH+ZTedBc3oGfkPt@3K)T^)H?toYixEwDvTD)2;SR23-sdok06 zd(w0Yqh(}~OqKK4KY|&mv-rMI&BWd3>L+_;TjEUwa{yhvj>Bz@t;q!@CzXO+U@5%h z_;EbbL*o{)6b1Sh_dknQ($T<2kT&itRTrx9S*ZeUrY+Z_oAXfoqtNA0TzCqli005f z7!~i&?LJ%P6pFw3GzkP@hz&Hvhh3eSQ;sQgu{%0E0f#srhS*KULk443%5!mqaW%+0 z;RyH82)A{0TGBsvz6N$PK;-r?}G7oKECJn8xjow#CbP1!Fvu0V<)zz(umk{ z73*Bmbk2LjX>^vpya73HqQDW6ce5TKP>*O95eHW{@oaxs-J~L#z0``Pim^q2NrF{a znn)4bk$O7%mL@B1CYDsK(mtp}A8HcN7gX&5zM5X(#G|Ky{@@g3sY}6<&J!I9e<1Wz zVf;LYNpwU671%=wY^aX4ZG{K~KaG~EX(!dR6*YzG*jFuO{j7Cn{cwUNis5#TV zsBA>sL5&a^v)yyl&g>W3^{KC_tAK|F3q+t%yn{5YNACyzCkt~ziF!Z;6fO3{`L9p| z<aO$iij5^+YEY9-%{wK{R0y469Phd-ea=RaxZzuw+SoEJ}WAj5}Ws z5gimyS2n4@&}*BaVaY&zFou|L0ytv)VWNY3orKdSy1a)1`6$t4n`;}b$i!9huL&`A zWH*R|N|P}oDvyr+1MPhFjbjk}M{VFAu4qzms3GB0N_a>B-|Bfw-0N{~0#Oy>GZZIM zf{1QG5xrKlb3P(lhFw_QZ$=x=@YO*PCP<+$mYabqFL|Oe8!N{yWHc~`$nKvs@Y=In zc!l{i@Kd&dKe!$ThQipt(%ra-SxYT0qF!_v@3FAQapNI2?bxRcPg00@eAEVT=}>2{ zLbqc`*B%6Yb(RzQcg}Qy{_#-=eT@zJZUj9P#(oA{-RTsC9y*5oANMJ$dRH+EXLGk1bk^$-E2x%P@OQ;JdjGd`W4)?AiViHt0 z;g}Ux6kipWxW)6NjVOgOXAVwDVphVbP0* z*6!9~QbYX2$Mn{EYILWXQAdWoP!0c&a_YO#qYI-k_V>X(NTOz_{)amHABz3kq6Eu$ z_SnHT>Y$hE;P-_TB{&FQ2gb?A_gS#y!)eK{YU^a4%L*_`uoV@YJIh@pnc+89G^xPg z59o-?SVRD4u|+$wa<$VF+KSiPR(u8ZgfX`-U?FZH>qsK38bhK6KSZ-4BA)w`xXz@L z_;INf515);yydb39*74_UWj}=IUex+r5FzwsyH67z1tO~91pmcBfPe`|I3?HQ0=vb z8+ACf)GZ1yUfcge6yW6NH}oJg)ldX&kAkWXLG>=r72mpfuZg8g(Xas?!>RqFQpLXu zs^2O{!F-kLz$InWG3Bz;csDDqdQ&OI0u ze+cvEDG;9>fjR4tdkUCHL?4Y+G|T!H&YX2;o|^PL z3W~l%&_`ox@_EQC^Jx2+J*+k;adEoJG|cBbXpePgC1OgQfXW^ZD_KsSqnxUg?bJg9 zr0rC5l$Tp^@GuP#in)#u@5&KRuj*Tm;f$x3X&gD89@)n-z1eRldgGVBli~@J)E=sA zamJ^5h_3i`uaqROvBAKK)r{w}Pi@3kW2JM1Wr_{`i^J#Zqy~w1H5+>+`vRjMe?WY@ z8Bffyx;iQ#_HMS$Q9wL-Kb<4oVY_IDUDw=sjt=A}og=^4JqUN$LQ7i;O8+l@;P}WX z%qS@gEcw!z@6(b-=eBc(SahNn4C0*I&Ncs6HpdQgS}|J*z}l%LqqD%_3r*nAY&%;m zS#w;n_+?{jUghqRXXfE{fPBa!_i3DNkC~?w~iF=aqpSB zc)A7j$`<&xkdG7l$@A4#_qU3-mcQC3e{=3dQeSu?)g)VlcOoFBJs{~l%q=Ont<=ZX z@ad|xPM4D4+oRcLuGq&sYp!TIFTueY6=cs$mynH5)*XFZyY|-l-5>MKj{C5b) zmWWlWm1kSq{X-lfScBq>S5(ZHT(S2u$%hV|+sKhQ4t%B(38?r&SkYlFJ_P)+)oM0n zLp12h-kr2`ScGPTkg%(?8s>qm90i90XY}|=W0Szu=5W`~4vkXJAxVsh){MmZe z$6_&uMD?RMxP^5c0HVak=g{qoBQOn^snoLBV$0cb8-p+IU^&;YxPG>r%h7Xi)`Mf~ zRpne&Y0OcfH|5^vsP6rgs};hqdq2?dXh&5r^i2H#o$S~WMaSEWZ)**8TaocC7od|N z$J&PNcm#i7xA>;vjPFK(&SY+gB?# zIM%1tq+gy#URJOrKp}bld`lr|64RgqUjYTm2yOp!V5Et~ zDAH%Re|E=Q*Kn&_=lzXnssFdYsPdW$3;$>HSz&Rm|30d35kfNtgZS#WFDNjS+g% z6Z?oXB6sW)T`8&ny*tY!ZC7L>4DKpY&Kwq1q&pfxJ&um>d1pKO=gw4Ii8`WKv93~P zdgzRCRp|3;XTa1T#LY-0owjZcP~}R9q!$lH9HEcL^2fc*>6h+wTc3ao72+vB)HQRZ z%*`3(G2omx)rQCb00yX44cOvf%pCN~&o)XrIX^5vlzlBeDb{(B7225rtWdhgVuijK znUvq&*>8AHUq|G2(%w z0Z;Es0Wj_(Fb5-;pKBc32op?<@aGzk3gfTZHu3TUUHRFE2@RF4)CzSk1XyugeYsCl zwuMCGEb?%HdYV?Z%$3>c#3tce7PAcsW7f^&nIaqPSU6#BY3tDU- zM)j;DKJ?^VPDN~x-+#K3dp&W47j{^LIn;~pp&b^9os-3kdQ}-5h2i-*rAsbysD}OG z^W&-i<<$S7H0b|WZA-i{+-qQ1z3Mi692fsB-3qFv-|D=s=+cT zZj%({Mpoh%YiCJLYl12pa1ZGNYBhE)b9*ogzoMl@M5rCLGU*m0%3tXKDT= zGme=u0TGDdbzW5mD3Ot6{RxU7V4l($NgHw7u>}4G0>2A_ zpQC~Q&tW?FnE<|*z`umS-+3p2e@SdRSR!b;z>v7@Zld!T-)QPAFTp#qE_y{0BquN= zn+TG}JWfcu4_np#xWzt(lkGIbgC^`t0D`_A%E4u2+r%zfZji2D zEPRgLLdIH1N`iXRlQ%d>$@90QA|=1|A-ruPynSjoNy%435FfJ`A06)?e9VgZTAQ`s zT&~5IES)M@r8nM|uY)N`;Ya?#BR)qX?lhd@@XKk$+5CE+F};y<<*naHW6zGQYQ?0) zevd?tv&gv{mkHkYLGr#2BF7P|0EeJzFpaF5NZJgvFJ8TlBUwViS=}}oYxl3WNVwtE zDsEeE;T>bKokIPTT#|(MrM(FYuMigAOSCt!uuQ|kyupZtOWAU>#|09%XJ07sQL%yW zacOL6ODXHNU!}&WRPZAg(gsT#=b-Uns`Cf5fuYT%lZP5rj225xEq4;WQ{@*%Th`Rh;h+ zwN-b1=C$8#uK`l4)*SL^UDQnX8oEPi=zL3LDmK0;?NO$nRXhVGt@g3aZRaX36|I|x z%_U{mAww>Ir{hQiddLFp#oHu8i<8dcAIH!-3R=fyjqM$0@tc9Tj&m&-I;@u)q;iTx z{|i$k`hUNT&_6d8YmRF*4*oaZz#wL`!4=Fya=dZqKf+gim?sO) zEz2ZR;KAO<4wCPOu}^yqQt_@~A^KPV8IGq|lb67JQ2^LjT>;=1zxTv`=2`R0yG_pT zp+Y&o&!@=wb>jKWi?wJ@+iM(1ZpM0D=}}2f7kXqxw88MFq?vBhqXAI=nTGngeqg&7 z*cWU9!8_lAxBONK@1N&Ocu!LB&bpQ0ogbUhjL>o!e2IQ_v|xPv*+^4F?xF;{w}BLom;r=CvnQhb3T*;~;3C;o};^(m40MjTlNh zok&y7Nr;ely5tVeN;*yt_P`l0v}XLxIyvKM=gAqbJzvgv_d1&K!q_wEw427mZt9X^ zH}!DYO}4$%gZ7g6)47*=p6Ifd{_Kv>7F*B`Q_wzku7vih^CYz07~0}k8&1}Y-t3!Z zOF<~Y(3@abiZI-w!LVU~i$uMIV7T0Z0sJEghQ11hixdpE+)OZB9;-}~d!$z{+9NIA zNV!MGOttM1guCAq+9N)fJ<=B1XUN^-v-U{FEpB@R=|hI0_DJT$)O)1-D4g*kYsPgq z$r%ruEN6V#xpKz$-9$59gpps>N2|=@SiwOVv-;?4!a>IC`vxu4I zdG2FDRh~MD!_R88)jlazn`KW9ThMmXk90T}`1B&yh_fxeQza zyJi&an&}6fjKs|TF1zOBZivIh%m*0f$SDk+EYUW9Eun2OWj#)U@C0o__2RRhgua(w zWv=OI12*-FoS+w9J_zG@g7NJ?+B=*EYWo3s#z}I_x6YPhHs>)*AoP&e;eTKgwd~^wYHn#k^&3^obaLLCAWsg)@;skV zzU=4r4lArP`hu{B6A9;&){uA}ySQdDX%~t`!wr^Zu#DNIV)5-NOEXw3)>mj6_6Xn0 zKRb8i+qo_p_QyJ-I4thb?GOT*9m2Dh9mx)10e$|cN5hg@yyfST9YO%?5J-4=VgZR~ z9dfiN2v$Q%u!;_=`BkW@=h&jiHX7|DEGIh1Q&4=}i3}-ON_uXj30HVvEQ>iXG~J-E zHXJoKB0eAAm?}A*zQiz@Xm=S{U5liH6m7%aMit*B$w_n#kD3j$E|N9`0X@87P?R2p zHUt+v-0+f-_I?n%3dxs4{%i~>a2)^k^0m^v#=Uqh8&x?07O9y+XG{~ic&20}G z+sz)pA#YTEbs6Q|lpo9*J``K5LlFR?186Z9^E}jRIe38XV{2}}mFV>3kL^KZ@Ui(v z{P9y3d%a^{8Jk(b5VE=tQjqnyYp?Jy_S3`I4|c&E^D(IQ4fG*OuToi#&CPn9VX*O! z90(GyP2a-Z) z^v9`ra$(P&)~8bPbF`_P<#e@RD0#pVeS2=Hp+mp z=+zG|wXsjMD!^0C;i>>EC|7ycyQvdgD88E5$S3$xpA7RD64jeNsB(As{ zAx6uH8p8Xd2b`q$?LPP<9K`*XbN_>4&3!szLp~9@RK7eZA$_pRhacmb4-ws{=qzjCctn2uxNA~n$2g%Q;!frsn;=YsYj{q$EE$mf% zzb2zYc^b|h%uv=5%O`vG-({iYWkSniaVJ`y>5XU^%xIY(qWKJt{nX4ti;SiyHU2*v z)?6br$D3QIT&eFE5HhVj0s;Vuy{9tEdY>p0lHDNl1 zrHY8}I#KQCXAx4M) zG+ud!9fA0A*wv9x^vRiCD|8Jbut+?>wRfFK2&}as5PBCk-|0Z$lebw2{E`rOW6X)b z%d!!HLm7cRPay;jjco$r0+DX;mPZGXQg#q2wVj$)4BD@?IA6j<$UTc5IZf6 zW{ZOSH^<3G)3M{ozFK-1%<3pf+NtQpLSQGpbAtU?PXv3o)n}{oo4C(OJm~P)dq64m zpf_k+z&r1T({=6GsP-N$jt&1+k+|ev)Pf<8H;SV@XyvH&nK@Q!$FSMTm82b$*7y-O z?U<~@FBYFBIrP(2S>{T5Id0Jy8qkixUcjZd}bh5f+yJd9K8Ad>mnUL~J2*u8y1aw4nWF-G?`8x1^hO zpP@%sd(O}Z*!DFydF9<%wUJ269fr1%uAA1Bpsqq<2OXhhfKk8tES*#abJQp2Oe2}@ ze!ZXP&!)HgeuMls$rVBN5|$`SWLbeF=AEqV`X2{KJED9*Xa_=Q={HV<{?Q!~I?@_> z@??q74PyzRBjdeZhERU0$w1ZQ3T`sWXq7)a~2WsP>W$Lo24I5!Lmay;Q`=GRVKy>b*K z7f0A|^wJ* zkHQ35upBs?U54OHSsHL7n9>l)`>&%tPSyv3^M7{}?6+@}O6zh!J!{GjwRqFJGFIZ$x{W zFyzbN?GbU!BnjWnG2~|+zcMLmkhs-~P*^IC8l?_YK-}3+9;n_BHprTIj_PK2=1#jB3na3Zoc>&$uU z^bDAfjnI(o?<$KtrxA;jMm&*5lv13}N6I(gU~|OyZBQB% z!+&l}nAa{I_!9FO)+zX9dyDW-OQYv~P?p%(0!&ZLAA3qn%Gs$ue~hoJp8Cg0jzRG# zf9b_9yYZK(BYn!NeVLSzDvr}Who;@_Q|GjYb<&=uVIfAyr|D>3$DsJ_9Evl}KF+gF zr^h#CDHB5jw2zB9&Uh(Hx!+i8A$=N+{>&%N(Vy<<)n!L>N5~iL3w&bHIUAi)RH$N= z%Bo&`bf%0+e=2qG(6C0#MC5EZcCBouvZL5yj zdo?Vc9nSAnJWWIRG(EtjLzJq_JWWqbz|+*#MzY-OLj#OLr%2XMa^!9$IFH`+tNUUy zmFR#+!z|Cof0Q5Mq)JPPEVcgBDUTi3AxR!vEROboKsA1@%ykm_ zVQ<-LTH4tz^4rVbQ%2Y-6>Cz&d8a*XM+8OL)3zCw(w>&u_qMnHbf}~K|DLb?;q#7^ zuN~=Ml8tE)Fi~EEp)I2Q0qy1V!uj-bK>Ldt+FxzwHEJL|K&-*Q^hzO5Ne}Z+Shq3V zjW-fLRnLbe-l!$N-1IrhFL#=M#L!Pbeo0@OEvl%+K5FqNwCMN=NjPmC(|W6-%{-PW zl9B4orJtM=s#xdi>2;~~KyBDrD;A(QJ#XsWHkNgh5?H(V^ z_PCiV?$)beTo5@b(4dgn%zw_ySoTEyI{6j{h)bxR` z;tGV~!gNjk)hZc^_!9d@Jj5IEUSPU1U>qf#35bHzD18j%g5{tQ@N0IZhw4AsDtB#j zN}$$_IgV#p#x2opZr$a()+e2_l;=xY9|uLZZ)vlKd9&BpH+xuZcBvA@&Ayp8dr-+V z@YEl8HF>jt#;am-A{}W}FDHDSywAC+7q;}Oy{uOCE(L1OiDddx(otp~JMbjn#{l*T zSLzDtx$y(&`BJUtG+WOn(Mp}FNUD_uIv@@m+vwux$|SU-Vau0UU^FHW^(jbPz6<`* z)u)Qp!L3?pq^%rh+6nMtlBJ0GN2I+gNn4rPQ9tpHI%_DAw%uQAq>XZ;bUK~-m=Art z@Q&=G0%vdN-PzcnN`w6E_c__1T|UVMot|d@et8Z2%dAHb90qbHAYk-URD(xg9ILnT4hZ)JxO$PDrMs&FQ91NU8 z7S91a&I>cQ8)+QeqtGPE0G$c(;Akh=MO7IbILdn07RG^5(@aFhkL6#ZLg;(y&zWG(8G%sMs`YJ2=07_iPn*=X5Al=>%GXKDc zFMk$4Hp~ruMJAbu+yr_9h?NRQ?mIeiJ>PRmhaXLsNA9gOZM1V=m)Ctus{MBwiGV;D zX9po;pCPTHmc^xLU|4b!XfSCpiY6%DKHxxU#G4wEL%#P&h4N-sghQh&zbiJ=LOuqxNOzD5 zil11(-*q?$Y&hR<5sQw-|3oTEH)xKZ+GMNSUACVoJ-p?o%mgL#j8|_&C z9vkQ(T`CXQXD8z*H0rpOsq(C5d|oRnRyY?(!CA2;`I~+CO+~8CR=HD(*6GQ>{1wH3 za1|VqW9Jd+ljTj1G`DCtPHEW~ER^&SatU@(&FI(-#|e@gMSfj494AM1k?ePK<6byq zOSD5a#Na~~Kb_tnlYc}l2*1Odp{U5O`jt!aUsfaPsS9b~GDMu+cRWen6{4|xzi8Pc z!(z_JMm?LP^3@NE#E?cSRb*HMxnZuw==<4x046T{A4_^+!STP#!#0dRk~}PF=^Wj4 z!0xQx?l{%nTap+p0JpsKJKN%kz0PQ*y6t5kZMpqvh@)`u&#D(mADA1G?kuDX$Yy0g zHo%hIJhS{dtrHmuK#R*Tf?}4KG&;NFL^_1Ob&_g3lL@V(*M03?WI|)+AMtEGFro3a z3N%`=>J{}tk0evZH}d*P-{Ofkk6mo;FQ=!qJ^oRs$|t^`BweC*6G^k--ojU^DO-3f z6CS#GvKRNV;GAc|nFUg)QkHyQ3ETb|)!=3P%+P+vY`=Gx9q>qEt1Ve0S@%>;3{W%% zRqhbk#iUHkX-ilMi$&%MsbHd2C}!r7ZEto`ybP`D zTt(%w#eF^D(yx8eL%rp6awV5Z`vdnB#rXHj%?5PksSi9fZ}ELE=b5q7D_8jiUpWyw z0V)~e7OsO*yOaK7GVm*~0E5#Em8jl`!hMk)ALi0!^k`dQ6J1<55j`AI$X&YWh7aw0 z`km(!@bOXhk@1mqlq{J9>X!~DWZHDjwsQMave=PC74mbWtJ?0f_*BgkQ#ra2 z*R)nO-%(9K)?}%g*Qc^}8)~Y8S(p)&PoZBdoR;DgTL0C2Z8-9^*j8OVW<^?-p&4lx z>M!aeT|8PFx{CP*fU{Q{H>|)_lI_jT+}5HZgd{S%Dws~!2Bm^*t-SclU-SW>eolZ& zOIt7QKF?>!4^8|*CF_{W>$+lF94zb;CD_tcK9h9)mCRT0mDhG<)Af2W?jDj_1(T#! z!Lc=+4Joxk*o&fa2}oIu)tyol>4i&SM#?V%?=tf%Mjmi0yBY^&QDX zQOF2tT+Ql5iWYN~{#eQxb3G(yj3_xHCNSUC!2{|6-)@xf(Fa|U-L z3t0GP=gbwGJoL{l(Tv!7-Y=YIUQ-wRw>Qbu7QZEBYACTBN^-?BUUo_>ANiePkOI(3 zV!0q)O0;LDryA%c9;l@psAQ<-ebw-ibD(i<17)O39H$Moi$){jtvi)y6nmaXJ@0tQ zDH;{O={=KZH1=0@F9S0+*jDFe zZs+H?wg)n|R|1(GSw}mJh3_>xovU`;FD|PF`|*gKF717U>hikgBkPuA#D+APTu!wN z>$#3I128fm7Y50NNaxNYzbT)7K+pkyAUD6B9t|CK;-g`(`A1y;tW?IE9h7DnB^j}o zQDg_=n=8z3s_C1P>6-|Aa}IqYErd>(ax#+eR6^M7!z5t|w8**#q6#_lsI>ZwFbNTQi4cQoTv&rm_DLVpN}1XkOh;$4EPj z4?5rSc%epfv_@5v(lMU>Wyppup#~wnm+dsxuKDcffh~?RThO*^n9n(=vAk_^*B?`Z z?$qE1^QxPWSIhE)nO6GKCkcsL>zqiu`Ue{lAAM3Hv3*xLp{^Y?B+g=z3?P)3rW={p z9jle#IyaXfjvSl57)RehqrXt+6izoDu#JAA8hu?CIr@h7+UVaI&aup%AUnswpR3q@ z6JH&-1ijKM0OMtKlY;EZP!<*4*9|wBNkq>C2x|$#AxZd|W`}Uq6B5E-I!g!-wX-0M zhp0Bc8_A?Gjdv(Zbo?QaP#h!n^h+Pv(`MDv<62Lw!#$GxuPZX_Xd+YDIwx~meN}qS zJNpy$2kFI)wlVK~S@lX{PoFxIj&yZ-!a3h>mfh4ELa};)AWVR;XfI`;Gn`feR5Z|P zI2A8BE{{nu%5SSX9ee>v@oUeRq-yHn$XW2fjWOsjk zN<}&*^Y)18jcN)g*C^Jfa8HUU$T-FXx1Ztm)0SeO{fo1mK$|ygx^Fb(e~NnJ3*tMe zJ%c{l`dS0o{K6w?gxT}}L_oX03zJa2WkS8Vfs$&OgkxmA*;y}ZDD&iQ80A&xA+$RP zL?Ol5)cF~ufk?${C}jW%w)J;^{m1h0X&#VVBifnP6N%LkLb$q0=#u3#s7Ef&+|Kz! z?Hq1)%F6zA<%gOt#e-h-vJp0qaD>?%Z0?+1K#8e{Thyi(`^ey!pQs$kT(OqfBEQ*@ zm$r?`R+vn@Nf|m4Gk48%NEz-j|A^&JNM_EDdA)F(wIC-q7HAfm+{f@xdejd%>PD}C zC!Yh-zFs5kW#4)Xp3h0RbUC9I%g!4;U>`u?YEz#Wk2knUToFmcu3Y!e8R6ALA zuJkB%S^d6~;+x@*n(q<``!^`^H}}UiJ(M9*T22#+6HZUah`!V1tw0GDJ58*?HGReW zBOdz~tSM&eAsP0rnk4`CuRZL|A&2lE-NX(ZFL)?y46H7Mk@=5yhgmii#1%dKs!~Ox z4Y0JIv~C4_Qj{C)c#;%Y!`>-$d44;LRfW@H>DT>7D~}&EZlqZ?OPW=~S045lk+YiG zLx%o{*=GTiBv-yV6j6hRhji_e4fX2GZI!6TnYYJM+9=m0@>bF^#$Q-NTQ~qWA28&$ zUd;_^!q6UIs2tmxouBA#stC1p*%D1QNGp~@890h~)PVE5(`*NU-HD6PI+>m#WfU{N zr!$#5d^z|zXO=(1D;E}jRx~7zHL_?Y+|P$PEDf0Fy>A#3^7fca@{U|yzC*uxEZO_g ze#@64T@5bF7EQ540*=@sr4k0jWLH+en4s$A!rf(1P@5sgVlYYV8Y-{L7QelR!y>KE zuR@r2^yBeMfAiSC3y765nqGjWx7a*23+U9e+D7sK^N+aVQKW=A=}##SC2d{C(sz|S zv`Fq6B~e370IrzFwt38p!35x10^lYB;NJkiACE{TO?Kk=I_BJZ!zrd9koINN-B6`bm#0ynlCZ6GBbOGcVYda_0qddNWH9uJ9^)@v&!E=p5@VoZSBV>CZ2pz{`D zvUiz?R%Xzb?QTNRD8=e?s^wOL^{rHe3NDJmS}a~2?MRo|atuYI!cDQ6HpTR<&P|d38E%Tl(=77R zMy~3jOcIkh7Wp4(61)dT`>ZfQ7&E9`v<~%$ZctL%Ik}vK6|63~)Y~W}ag?UmO7C|M z?=q~yE>!MW${EF7>@vv7jV$@(#aAy5oK5149FNxcHUlL0ymnyUJB=D8^d1+Pjm(ZF zbDQxOP@9kSHfKUd+t*Q+K=AM5jeea~w*OFMKBJmCK>n>|?%H@8cZmV?H#+n=kJ* zaZzg&OvA;LN?-1FpMu_(I8hnwEkxRU?G8on6UPPm8C`f&p1=$5ALbvi^?o!1>gDW! z(n=H)*A(;ooe?stKi47A8zkMrHq%s8Yr=Du+$Sh54J_1_gP89rP({Yj>_gE7{&@J$ zwR7N4+O=)skwLZS7!kh!GQs7O@TAl-GUX^aQsJzqQUwD|!WPXa6}^nW^4nXI9&aAc zi0wo92s)`^LZIwMj!&`$&fP=WxqEOQn!B{ORLanGrH^#SuJ0+C&&%O|&^!%?n7zg6 ze~@nuK?ax)@oGdw`1eEt5*6XgOg`(XkTo_3b8@43Y(cKrGSj}e2D z9~UX|MFM;xpFfsx#%5Q~fDO3(Dh^v9WnK}H(I~ZxEn^_OjcrxEGaudO zRT<;3iqo#%`yH~;WUasK6b%-WoK#fA^m&+rjH;|SpcVOdmjOotXIAdF`1ttPm`5R# z2Y&b!BL3}TkhE!87ncs=Em1l$*kj1o+WICYdzf+R@B?itBXBo3%_X4{(2~B{oe!r0@B({&hb+Rmz$T6 z6{5g^rwS!|RPKzUy@k7id8ZAvICzsc(v z5S>0GV}(Yz2C**f!$!9iaP|)$hX@R8l~nB)4}Fk;n<79y6q!4^8geI`+%C&-jN$n9 zG4i0W0C3cw0r1-E?Y`iEw))*REcV?;SOf}2BJ*e$s>#Hf$vJo~+Pk_$+tdXlPHgsa zdX6WaZvGzyMLC|@{mt33s1i~pmVL2|!_k^$70WS;qPsBZZ}u-c?~9!`>7Dm=c5dCb zvu&8cxT6?Pm9di7LC*u&7g&pIapEAU$R>ZqY%z*U3e6H{u0R?uh5a$~>NGx19Gzs! z{J+~n-_ZI0r#gl+j&jJvIEysf!lz-jziP8RWiK!o zD4;vypiMjF6suX991{Jmbo1tvI6$w?f+}ZhCCY(K`#sZwGHh}=mntodid?+Ap#dWo z`)5~%nx$L=ddT!2ok*md12yrDMjK#PpVE~3P4Ipj41Hqa-^)A86N_8vdqVCzC=zYy zv5*^kgSHX`I(ga1HSydXU(V9Ug>7jz4YD>W7EC%(d$|8hy;s*bH)_=zxODYikjK@M zZCY5Y9N@E_3ceHB0Bax5A)u-T_^lK)YxL(u$3d1D0fpo{2T)uUgr5>FZ z%}}DBqR+#|p7eqgkycISy2tMH7;MdU($5VlRJ8aKY|*!KF7cq~bHBlAkco4fq)_K0 zMiG86>8FOoj% zDPjz8SDwzwig=g`#$m^M6}I0_C$}wH1&@lY6%_mSYduz}9tU!d1CNj%`!$D`H{GAS zXZN=^q$UKyO3mJpXUO-N7(EK#E*`_>vP!y$BQ`8fT zP-`Oa`xZ^o35jicm)Aj^nB-x1BxxK8yWzD{IgG0uK0a32Y!6HB{4rmXWk{^(;r4?> z53_+D;v=xsx-XOcEoAigh!}GM$En9(KlP>4H~>!L>gISFS3T}L(*0f`X?`JIgYf{y z&68iyJ&Qe9G^tqyM?wpoc=$!^s1zh*yQ;=UkbCM%I6xtC6S8G`0Ro2z#0HH{?@^Yz~6)N);B4 z=G=HZjej;Llf-;KG#KiZchO+8lLpIi4i-Oyu6PbJ7r$G*b%La(ex16?!>%$l^8Vp& zy{nUWcen-DaGrN7sjzNWDPsqefLGbX**p3YgDl3_YMQIC7t$YcgB(q$X{t zHZsQHKH;)oQ>3od)6_B%dApw#h|n)q97h*ZCJkJ|KVhulR5XK%mgG{T3@lsS&&n$i z6hZBy^!Ogv0Aj9DepIv+2uhDZn!hPlVNc9`^P{bmiY^uY6|ROdcra-Z8o&whIoEDk z1F@%pZ8wv`)%a;vhrS?^pDh*~OLJbJ&3SHLwF*<{&UXo$VkO@QiiZ}sGN6)nM?}$O zaAfAYxO6CmoJl&@QURaPSKILU!B4~!g=vzZK0a1K;?HjUGeQ<77(e}|1{s67l5$oQu*+U0`=3Y9{~QzZ zvMpC2u?7|llP#0FF~@!jC`?ZW``@AO0(>xxzxo+*|t(nD*m;-G+@ofCo0axI5vZHvXd=d zBv?DC6LBweF&g-8gpL%$iseOR1K zUwIZo|35J#e&Msj&W7mi?WG~=;*(=+rC}=l)?QjjgYP|tPIQh!#_nLmU}DI>i{Q6O zRcDcZ5`MRjmN%QI?@`Ay?Thl=cO`Wj1qy#6wK>H;euPTH_R=E7C1_`0c5(n-X%c8| z$|6=Dg|Ww6v7o2Vpnv_=lYlr^-PpMzyO+g?1*oV^A9WcG0LC{vb33iJ`KIj(x0AV1 zmcm;~9&fy0?m(~hq}ZNG{ET!G@remh3LlpS;p5!@!d^RMy{%Y9^HH)?edm+$=MmxlN*LV`Fa+qD|F~RCkljgM*cpVP>e|>d^ zt>0FXoPR!HNDBgtpSxH?#|7#G6#{+96O0D7aX3yI<&k(|NL!L~-3H{lQM-MnKZoK8 zbH3)DjiI`xy3_JIwQ$7+tB8dTnUDq0A3`Oc8LE||s~h?n_5ZK|eJ?Bca!iU+Y(Xy< zuNs1O|GKY%!*@@SBSgp1>;sFmSc2vW>D1>@Ewb_VH_&6FRF*hyZU&{29h(Zpo?oP% z1%Do1OSMJuw(`>TDwd`Qo#W{JO!jcw4TzEM$5-EI*bUKc6y%=y6>-Dk5<%Yh;{=$z zoFn3lUzC&*eqPIC7t`25Id+iFDLx(sVhxWw7{`@Tq>l1hWMQDs+<^V!TVtT1gms@v z>t1?`bKPe@1MBW~hdT3XnU4$%7#yLM{jL6aMjevM{wH&RZ!bP!(3m_5$;_qvz1tucf_q#Ujg*z;R zU?NE=?GcL@h+>-K);_dQ*<&Fsma-has}CK1sQ^0>PK!+i?WX^ta~^>YE}JPgOD{(q ztHeHej}?#cQ=|wvDSl^lr$sllXBAfK3fidtcxSWx`?a$Cx9bh3YK!o1!#rTLXmnt# zVU%=dy{k_=y+AvDu%1iQ>I6j(6g9$*mG+Q+QS_e%D-4Za+`gKH;o4vSr-x$(1I zzXXm|JkMW7EY{ja`?$|!STv((SjBBWXe$;J->sq*3%jit+OXwju6U41qFOZ?PF`9e zaWQx5r=K@*r-jt1Uv}zeV>(a@PhvL#>@Y`dgB)`k3?|*~3G=nh*cw~levKk6+{d&q zn`mK+_${%K6jW#Vt{S)J!fOuK2_!MOAJOZnNc8-@QRZ_Gh)Lhe=LqD1%QmRA>PHIY zo*nxqa)2g6fZa%P(t+;Y{H-xTE;^Io2yy9^6$Tp)U>Wwh^=Xoo`m4|h@mon^F;hb? z9y*JM?qw3bwN&)_z6PmIkBEp_Z#M7EpxI!Q`BG+zRm87U7}~rdDTP5n72kK{Ya$NOl_MSIl{Ar}>RBri7S|q- zE3)JW2+3SIA|ZpO;1zFDBTBi~Y2}SbjUF)nh%2v^0iwg=_xF`D0ZGQn1KQGtMY98P zX`PNs$?su%Y)6?il%+E$_FaD>Ufsw#e%uQ0(${Zx+CKLAH?oe+GOXTqQ)DRKM1|Hd zDCGGU!u$(6FAj$0nLo@6z0Ieq+k=~+$7h%3H&tj#32;yxj`&r-I^ANM!G>2D>J?3dtGp?RA00ECiCbOMn*q@$sJSUttU@+lSu)3w-JO>_>_BL48H z%OYm3fIt#E()5MgRYwas*rry^gkOV)WMvtWTW0-bu^4|6WyP2cJRp_~JjuT9Z)w}U zeQQMTDQlhUe!>>SU#3^CJCg7v`5+O=T)EVvf%sO)(6Va62n+ks1D;CqeO~!!uWio$JY8kA`nCzSf7~k62B8e?NmYkO$-Uq-dA@F*ycCKsLqxj0T1w2lnKZ7d){Bj`O z4T;62mv&h6Et1=)D@KXda(lJi0L*2a!=q%kv+cK639gn1*9r|T@rcJrrBid2hY^bd zeMmSaw!I!Pz%M}Z5NBC%@%IrQ-PtQd)pyx+SF*T(5$2?0G}~9wgz#82|hOR2rdyFXVleCk2&VAmq>- zAj%9YjI6R=z%yTP1;yEm?cv)3QE ztZ@L-nv^iDNeR=Tg#=R^+<&cSnxh9RbD#C)m_>Tj!0xcR=FfZrz8wLdPg%HZz~fzd z@tuYqBjaq*m{3|Hzu5UXtB49@@(B`QRNbwFa=D`EbrQu@F zST6J2do+vU%21F>UVT+@%D7wSz1sk?umVE?yJxQi_Ram2%rWKbFiE~ZfgmzEZ&Y5E z*^GShosz~yGNJJ*c7<}L>MD{JP~w>uNZbl6BPp)f)Cw!|RKcX?L?wHXlqOt)p^U3} ztK>oLXgC;A^hlmVvcK1^cUmsYebD2~>s2Zy+yaCmcDl*UKMko@M9w!EavKE2!B+`m zL2*XD+y?HirsG2fp4&EFV=9oNi@6%gf~{Z zU_q5Z?TWZvUuyT^+qQP*@d)~;;mOnxCo0q9O)HVGPAZWTy?%+DXmtrqG%pr0?_bf@ zPzwJ7TM#OpR4NJ{S zpOmLH%zVfV#T@GJ0D_1MuyX5(FkD*>gkiZx z7!GU#>XT=uKD*b@Gq6;To!-P&nKE{|%2r-1E?q3A(q^$l!zGI(8m26g)4Fl7M8hqM z2o1yH^VC3ORXDaLXpne;*o}J8Vs8RuD*+D;c6gB-?4Ct(us<%BgB`eB4%S-@mdk^o z!(}%6BEIuZCio%-9KMJ)(G10#$f#MTZWlnka;?6HHwao=zoYU7gYtVLo6wM-H|h|O zfw}M&np39_J+{Dv$V@&$i!|{cN%_C;#aGrTV6}Pr6JF~g`0*u^G7q;(0^*aGB+(c& zMiSx@s_LZ*W>Uf4b%t)YQpZ0d#67q7He+EF{@j_!(20PFYoq|l>_JNbXTPM6GSLl) z^NZzDoKeiL=-8DW4#mXk8<{_9f_4>Uz9g@iS)=78GC*BpA#pMv8^D6h6I z9lWc4sWxlRd)eP-UH8&}m_teeW#ZnE2!h1;zIB<(-z;#LWU(Ue# zvkA1PUISNWL#uz0>IYH%I;`i5gBsjR4caNFiaX=`v!A*)Lx?{DAWhBq&joy0#=(C$ z&qFQysFoitkS*WjmU%D)k~i}pys{S$0_^p_6|an0uKU&|v0S!H=ky+%Y&|YBS8Y_- zJ@3Va#Ml}yuR{>PL9gpGuTN+)XldbH^S(?Ze6L6L(DBno-2`jpyA9}GA7U@UFmJ{m zCVDMyaJP|7bW0j~U-Z7r7aUUJz>DbWFpi3Q`6 z(7VM75^Qd+%id5__Q>(kZBOfL_>pa4`qFZhn85_?Lbf%L*5(uU=+h2{me;>$paLN_g+@7^^^mEY)blP zd!sGjPg?Y-_z=pKYN24t^(YiV5pc1zx5e}v#bj&?W=t-w^vUumicg8^+VmV<7o48C zJv|46^64Y%u4zGubB>C4)k?2zLFpya<6ew#Bt1kOrrK%;lQ#iMO@K+&T!soo;81fk z${nQ)1q;Q@!_AfLS~0(e8lVW-r8hjZ+)6(i-^!aG&dwH(yvlk$E4M!WJnJ<#4RGaN zsj|GnlN!=g`XTIM+xwv}J*OSf!P+UG)X^Pf1O&brZMp}6M1IHzu!R`~Fntb>e! zc!?Clg`(p@#1W_(Dco(R)2>BT<2>_^XgkBoMceQFMoTgi5F1~I&?s%vn-cN{)p)Nl zv$G5p&&VadV=l#ZJd-Qx72EjU2DqqU+fF7$E0(@MrRD&WN7kcawKq@dSe{hxc<1=- zW_cghX}96>KB)}Y9rjF|)B!n`pJx#p!I=%vp!c$!K-)L-74>FNTIU3v4fC~zEG!%N zBpGAi>gC#PIOduNZ?~j9&*rw7v`=WAldME|93(cMyr?O}noIMQRQ8P(kU?-8y{^s0 zKH1sZYU&NVvh}WVR988&s~oedDFkEHv-~vr3F8zHj98*4YDmGq!A*-#j_3f~_b(d@ z0OGSb?$VV&v_I3hkk{5}vT+xQwWI$no|O8Qq$(%E8b!ZR}y`D~HuFi!m%(9&85UABHZ5};I*270yzi#v{d-WG0BrR2b zDWBYoZYH)ap>_0|-w&G2GR3tt=c~=inR>?`NcK)u_J$!b{5LEbtjv-_VhOeCMZa>ooSmcDEhoA|p(yPMPn zvEUaZ$Fg0RoERzbjz5$ULM9M7sd~|S4_WrfE0~fNsVmt1`gO4<^r!UyB4ZftUPb*x z{_Z%LzUPE2r3A;{J+h9RGUtf;OqK2?b9=A{KEjs5stQBiwge`mv2r)k)?2-(@qr`F zdbqk?p$JR6ufnU#xEk^wOoOXy_6|=%N(x0`rgf$)zt$H?o?fUqu`Z*c2~Ob1eu&c7 zrh6d{PAyz`c!0aNaX74wXzsw6o?i7Y;%}?ht9c552pQ4iXc;C}n?7p4VeMog5`it%{DM%A_ zKZd%`gblukx+ga+mRgSDo=ADU}{1Ssg#l~=>E?a`KPVKsScS$pXrW_63i zuLsN?OfVwUM=R>%jH{hX-`~~;L~od-`nVDLsN__}u3Nxw>7rs!SrZQ==xy|( zg`h@-;CN${F!T5o(`kQzZ$An~Cu)DhU71P0HhoW95^W*I5GJ0#(zzu+xdHjTi}?QJ zZ_t;10)4aTN+_Pagk z?A0Ykk!yZR%}9oc?|AC*t}&W14@XgWSXjK)4nBZk6NN1mj}xv~-cr9OLK)d!$!#B4 zC`E6}9Q7DabSu7Jz*tr;MN9Sk^vOg6`iRW-j0Ed)+*Q}nt{OZ8c2#ptTy%DYiqLRU zYL9ofcu-ZGnRU%*=bFn{Q|7-pZ|th?nk#3&?vg*VP~#`mq*@*K^|#HG89Z$pOl)S~ z2E&^jVH-G}%B2oYC0qVho9>zpwBauqpsaxKp%mm>!7uvGAXf?Zvy{1V6EB%2fc^F| zCF)+G*FxPZ|Av?MPy8x6VW51XPq($2c@@{U(st-bTa)ElN37pku!)}V|p8Z(&`k16?m zW+Pgp--9g>rG+^>Lf@5U#}`ofll`p)GwInEM9=QO2ZY8MR1gK`5MGSu*-rjT5?Av>4+L5T--+sUJ zBxg(5%6Buzu4-^I9YJuCJ$pwRWw;>r&?28_e~#oO@UjeQ&yj2@8jw#9>1s|@pDQk{ zPbeD~`Tfqr&Yj|VzopzS?0s1;rq(qkv|Tesg$D_XW4=Rxj-_93eM!&i&RUed(YvGq z&E>B8OMGW1VRdOc+MFfzs$UbP1+#YyvhaB&9kX*-gTbX=H6pB|Nqm3;xxG}>4}A8j z*<3PP9JMN^KP`7x{=VWTFb6o{zD8}q z6(#VcNw&+R1U^?vkQzPYiBjD8 zC%b(uowZkoXcxv}IlmSb!HQ0o+A8I(J?t^GUM!-^Yt9`~OE5TDo}q*@cC@~CZ=<%~ z&rXR}xx+$YaT9K*qa@3C3H|>2p6`E!k0bEEC4J>Zic;mkqo~SSgQ~2yw5w)zKz$vv zA{6a4B%Y@;p!nR!8y&kk3C5J6u(a?zQi4*e#1KLGkP{7g`V+=vNssj&QF>zDIdy0P z*s1r$c;`$r|A-$$@+J(38`|T{%!V&g!%AxSFf@EAWXS>;J4ZcYVFv^hJL#+Y=&MiQ zs~bZ$2?9q#F~2=CS8fXw&D0nwbY*=yvau^K1~&F#?Qv9f4T@&QuqO)OL>&C2xePVH zB^jSKm+=L-vHjd{rsjw3#d1!5$e#u$}eu8>PZi4-t82LrpLn#oc^(@ za=na118a|V=cv(?QE14rAhBi*Su32S2a+iavDLS@x{%ESQbm!$c(!CUP%o}|%oak1 z5C~$~UUCCT>1!EG8448DaMdXNC4I%+f89XG!})rCIERgB2O@9fA6pF7e6E`Cw~yRQ zn{iW>=|}G8Om*bC>vT&IkYsA;IR7@$e6B8>)^txWou>^ct7UkGY}R~zb8+Q+Z|2a= z1yz+59_h9qr3JZ(HD$QYl<;2Zp_ke9(C0|nQ7ZmH3eb^k!2?DA`5lXL0WuvC%%A_14e9>udy7=K zwPrBGfBSyW13X)g?k@XW9^F3mc=2m2i}TTa#Pb3C!o~p_zi`SCF8qG{vBfqR|3tPS zLen0!PS-QOV8CTh8?IDbhx)C|6?fNflpgU$!KDyt=qSPDqD0iFfJjma)U$AkJzMhR5 zW!a&f5;O|v%T#2eeoCxgo6^@T8Q|xwmW-IjqbC}z*dp1iS%FFs zd?@Mc6pLs6W1SrY;-i@D>^N)|to#?28r*Up@IlCEm}ZZLRRi4bL3cvS#`t z2Q0tQU*{+LBWKS~M|kA?5B$NchU5>LH$IL%s+$i~&a0eb3h4J*+ zZu(5XXRStAjKX(&t^2Z%PILi_bZ2H;XtmAh@69LL@2`>DJ3}FNDDyA>cd-8Mdh0zp zm_NC^hkyKk#C>^ORMi*w8x{c-2P+NB1rrm~a@-QP9E@r-v~kI(EV0bUvQW{eEHSVk zq^zuL_uFc_S}B?tWGR-GW_xN{R+q@qv@}`f_s(+fyYI~#tl#gCpP$dj%w5mD_uO;O zJ@+j3<^A>drtYv|_!-z#xD?drj*E|x{28&6r9)#q){LeY^y-!(CMGq80?ehyqCr9{ z-09@+nLzJpclYtqua;(^WtR2euS#1Iz1t%9F0Tb?82%4ftDfwiN~Q$GWe?PcYMWE; z9bYQ{L|N>(_MnaSVyS}1CFj$N5>Eb7d1yoD4&XqO9iPm#PSWnRk4e`JbiNYNz%f1l zlY?kPvhQrl^jVv9Kb*gN)cN}+A7HuCut6N&ju4Dgs$RKb@28kFfPOu=l@7EtWeI}y z7H+p*@-g9hDo0Tco+H1j?*-rt%GVCk=cnUA+Jhfi3PiuHngYPfR4sl4RGXz$yKy=l zU0=(ii(}Bjq(2jsCPL*LI7|laNFo)|j2-nMD}_0m8y3mc&K8S4C)a;?9B~21n#-tn zP<;NQa@hw`Ldxf?SdEU|aCZ&Nh@ag~(h5onl$O8-GB>5K_HG#MS{iMiut_Ss!7*AF zJouOXjdA@CsP%usqYiMkOFsOmyqy9*zVLAzxiiH;5+tYMf{%Y{EjT9J`mwg)AJbUc zj&hFWt?O~YyGW}8acFdt%atP@*lu4g+&=H#E0-%r?ETEXTw$Ka^ySJCpFL794Ll!L ziPJw*(HdafuYV*JK=9{0_*tfzhOi9h$mTdh{kZib(%#PGt;5OBZc#h9_rJu4gxZ&D zI)^xxiK}*Lu2xalK!wa`cxg&fhd-1QAm#Q=}pdHmoKFZy+Kyi!b21}r(^f-RQf zNsj1VZLtaG(5$wtj^hQ!pTBG(yi76ik}lt0JQTuXv*qzL0GfRN@8Ed<4~q@eDr^EG zt6K0mKiJvmbS_i~4+mxblgy&2_M~Jurp=VpqCHWd1`EeCeZ}V!gPu=3%3=p!SD4Z9 zYK~8qQn0f?I6vdcDqD|uLmkigmlVk>P8Zt?{Ck(t72)pMddsmpKT3uEL+1g`J07E zU^}xA6G)DvvhDMcX3fV1ciQG7O%CCWO9}X!f7Rv#D9{yATC>!v#xS34YvknzV3rEE z@u5S2mm+42NF5kR#YQ_byQN%9eOU8RU7lOtP&-e!fe zLWn;Kh#Os9Rt4^gHj+`Y0Vj=zd|~vhTr&&~n3rwUS7!5lv4&Q8>>*lZKSi*hsIFq} zk8~w%rgLDJ^TYR)^P3@s3&?kU%*kJ{OhCFK2#6J&Ly_e*NOJGjG3k_N8!UIhX=wO( zfvvzTkI^+Y-8$uvav!9plSyUlgDfB>A@KR501oMCAXjw%Uh;{HRSajm`R>_G)LA*p zLSd?z3#G&Z7In^W;6}?RCr0ud-TN)J82*Y1CH!nd+5I83p@h9K>0h3omR1BRo=jbP zrgd0;v={pvXg@6b(b&3ZhvmUtbg^2=7R4ER)c{XZ^|pKyX$0;X5RuOp*ak0JA`dt@ zLLXWJCZE__B?U~F`5FI|M+C%tnMNg7SH&5cs!vH39WNl)-~i3lV%B|l!ni;V0UBeG z;(Aw=#^WRQ_v#TlsL7_?p8Q=sz1!Fs_Izv63vcleBTVD!t=lDcx8NH+7xI!XrXlS4 zsa~_LUg<2>+vF{aD(0dkNbzFM%?l$}v*+4s)s@3uD)dknSG17A-piC%VLO@sSS!Dc zNLR(&`iKEmK+Y|WJgRGxT%(xO`VVC7j9S(3HwgvFEy+?=+>FKM7T9ZR)WoDTznU!XYX7 zf6^xb@4FbE=Oo=#v?aNDc)d12dUtZXQ6@F0H3p% z(e@iDl0jCTZ;Hk@N+k0_jNS1KD((wdOlaAydM$7e(V{i+Jw30 zES_53e({r1Hy^KAOk@dK@e)J~UCpp%eTSVWx1WA7=sP99!{6Up&#MDc^#ME(3HW*b> zFMYzXy@G55XksXsB&EUFo(E2vGl~7 z8tvgxonT&m#5Qi$FZkQk#f3|VwI>)_pn;Q2@ov?(Rs{w-{+DXImSv5-`V&7%0xH{Em2C?v8J0;O?y1-G|4vhzx!*Rmnfudb z?oWQFOimJrq-*fXi2S6hekkbU&XdvBrmpQ_v6$E5c?|y2_)QU0x9_lA0HhI1dG8KI z^>>Zz3GAd@UD=F#H@LtY-nC>_!>V12D737I0%js^D zV$u76NdDX*qZ;su8eMXD)!KUo9TArAv{Rl4vDv>Rw3)il z76Idt9tK@y-Qp?uDFVg?n4^b2NTp+XI-jkjt$OM zPB((0<0_om0Ra0;1bb0Wr{kn{<88h!Z)FgA*IrgzO3gg-rgFaVdIBEaSM%H!JU@xU-&Ih<^V2umh=%+;$%~*9#H(bxXf`f~MR*e6>`V2hRQQN~!*d znLp9b+3Y9W3Os`GDjOmz{r?C3p4bvTQ*s;RQu5Ppw_oZ)O!3}FikECryxX=D#e3Ge zx^vFnt&~%Wd!^0D*~x5x6d?MLucaI^XgNK@plVJ)IG)!Af*2WnSa6K_gNs<><0bvGsKx?9___5o`gk@Z1yBM+na zF5563>Poo*mth1rU``nW$3sEsOo39D_KV(kX*#riF`SzWy7<)zQn;J}tjicaGp8yf zvQG}ak}a>vrl2@3FFN7xp*Rf=Gbt4xqO-O(#IJMQ%p3xbcRAh)LSiYbM6cx11Vxf2aA`(b9VkB z*27|=kavWAS%%4x{A!!*Y|uqXU37g<3)#{fvK5U~|I7EsFh6Lx{TDfj;f-|0&$hFE zUssdh+2WoTEn@HMX<{%{8Lsh#1=TEuuzFuqrTCk*A3I3fh!2_|ZLX&1fEr5}HEug1 zi9Lpr$g#8bML5FeIsDRn)b;i1sV`U5Z^9(EmS>bHCKeU_kYwW}@2fk*xY#r?cC#h^ z$rbq4*!K~Uafq? zsd=^M221lQX2rkJylR}SbUGe1BHs3=G< z30*H&ers-OGVRo;K9&Edzn%WnvgC6*OOO6{&~x7 zZe9)FoF}~+-ozGjIi%<#Lwivt>s~;(ym!e%K9$nTu!zz106Gk;H^XHimdw%@V&owB%TBmDv^S*FY4bnxEJ~8` z2D^}Wt|;BePqw(YS(_9DO$BI)d`3lUm$$&i&h2&6r za~yWN<;^jIh~jwu{cQ0}lQ+dMi71On4ON!Lq?&&rcG9e&@P(olU6w*oi(ui?%!MZ= ztgK~fX}ndazgdW!2Gz#YuC#e9Js!P4ZBpreis{lUR;f0-6RE{j4`Wl@MK0Nfry z4pJzB90+y#KzM`_L%ZnH=*{Ls!4#rdBT?J~c>q?a2ShcGQ{*LoDu`+xGBo|y;U2sy z^urKU>1Mw8_*|4c6`5Vu{0P8Dk{N+N-L~GMY$;BZLkiN6g3cjP)UEue+6rhg2bacN z(R5hRpF0rFVF2mbbEd8h;m$z4j*+XR+x#mtD(FS1jjuNayhiS1&C)p3rjhY5b8O&@ zCve99s~PUyM}ai110fklzceSjw=523x_KF)@gAP?aWKcLX^vyMz#KoP&2jS#k2TPS z_F71=1E-nueK{-v#!FL6W*+u&HTN~L;%0@>bo@#eqZed6P-W)KrF3eZN)P{ye>oJX zer85AN7Weo9A9tB;U)g|7v=R%iCq<@zZ`|WqSNK!mm}VLnzztOeGBD?U;bxpp+Z%9 z`?J;-vNb%$zJ&(X*|*RLnC?R~-CuNezVbglOh@UhoxyL5-(y&v+(@0k6LI97xYU8R z)M+DiS`yaj>9$S_q0=I_PH%Q}YV8$}nzSGlv2?Z73-ztO)Y|IFyww4wcyjmMzO!?0 zKX8r?(@CAtZjErwCyyP#HI!=A4CbvKnJRmWRxLZ!%^84O%(_@UXO10TAdcyJ@9_hZF{qz5R8);qU!3hOA#;sci z23O7E5O70H|oZm7WkVgzuz8 zq}9))(7QY=(W+)7+UMdI&4j2CIAQn*3k{2D_baD@{g4Zcmr);48A5j|$n&_sBL%EQ zl}Y;a@r441cBo4u$1=He65Y-V#6mkF$|VK`AHw@C ztpuXpEmLBEn)aZYEuiMr;9erS3W`qAi@rV;iq7sJx(>h)bvf1uf1HVc=G2Z1Tn!zV zO-wPHP$PWMD@&GMq26FnpQN2RM6<12L$&n?wT1kc!CI4tu1Bp&08c=$zvo+b#QjDr za_p9#YQHV#{dN@hTY-#Nm{D;zA=bc7h*)dQKjMlQiC9Tc`tLpUA%$fLd+S3Pi2h7| z3RJ7RJ2!GxMnw`6+gz1f*}2hbrD&q`7I8qeaN+0fh`97+O) zE;wZ>2H@jB6d+xt1;=O!7}3Wbf>6~CEdmWzxCxo`*k{~+ChQBsJce-|4;#T7YyKz$ z?V(>_6HUW9v!PCUlwRjTy-r7}lV0;C)HxgKL_rkN;?x7cAQ_(8(;&?Q+xszEg&iGn z6%Lz!#8*+Y3bwG2FqKNu!kwOdxzu%$J*Qp{7ejd$lu?Zk>EIL=kt8}?Fi_Gdf7;aW5=9fX}L3HE`&G#?sF{Edxi!*nrPZhK+}Hk zNMmq%6jZ@751wJ@h6`7{g)ussSHP0h+-LqThK-9M^B@g97jqmp?tDn6Mm#~X#ShoP ziIZ7YEm76dk;9TFbYg6U3HfUm%?@tZ_}wHK^8tNSPjq))CR1aBd_A4Fqd|(lh%^=B z^eU!WD?0vaCJMmG#l;#C^k&N0xV~J5r*oK7E$2+6{CjpuQDz56Z!dJ`=OpDj+sw@7 z%*~%Es2^geQ#_g&Lo52!RSN1{F-8k*7zXv8rl8)&nNUcap`bp+(c6JEhPt-Dwe9fz z`!a2HKH6Dtt=p43c>_XpI&srG9?8#xVOLQVB5i@H`o|^pAzMr^^Rlv;(W4B5#i3AT z*+vh&e@Mm2ucp;&N{!TX3WQrPk(2>izyA^~&Z0b&IP@T(DjCL9i-c@EWMGU#qSYAR zyi^*EbL;|A#j`Hh%vM8G=iugmSer2UDic zP|{UsXm_sCwUeZl4y6T?(7455KVd<1OGRgpJe_iw7eQpq<)ew)2nCbWxs;z=#({UJ z&s(P<@aeU~l#JCc`p3z!^|)AZ7kHb0XBu)BLkuA3AB+S_Y9 z&DP!|M|;DL+upmU+1h*E-k#Udp4Y9taDB9Zu8+ncYtYFYC70;Q{9q9fv!Xa`TmBhJ zYsM$8P^qn0)5U~la248?_NuMaO7@i{@g|q1ZgOezCYR=_v*k3cP*L5xgrp>@Ntcch z{Fk9#+ss&zxZi77MiOvqDvt^1XnmaeTYn3$U(!^45aW}^zT1BP|B+*`zElQT4>J$Qj=~ZeLm}1^#kYza76C|U zF+Xk1ME+ndw5Gqv=Xh87lO#T;42_F7KuBLAvs!~9En>NtpU?nWc(FCPAqvl%i|JEK zVNBqorro^m_{UrPo!LKpZIj<0{+ygY1^y&^i|>Q0;%Oer?cgo`$^1Rv z{0fTD-eTb8YhpYog->l`;NP^(EYU|Yo+jGE)lA;ghK-`x$*K9CO7}dO{`#I~fqhRR z-ap4^wGvdVm#B2VWcr)_ag-+JPZ!JeT0641R_z>M=h{&f{}!$?gm}B#^(sqMm2<^O zI^1`wj5EZ?(OSlhDx)=7vE_*Gek1lJ$6W`-VTvdfW3t7Da(ts`5isXBEfg}0Ic=%Q zpeW%cgEe;<^fHk~=-7#0x>Lk?A4s!=9+c2;yiIAO%us#=e4(I^4zdoigKobya67vO zH&R!%9W~YD)mO^=cV&Jg3&jZXHKwExgNrFDJ(}X2DI1#$bhZGw1&vOcRkF&_{~^W; zA5Ry79Qp9rl-t9AuJ5N<-DXIg+cY0=szD@VxkujZtvzHXH|4QA&;GyN!)My}Fwn<= z@$mY@)Y9D5BVve{$JDOtEz>C=spzdy7l2@JNyd=NX#bflX ztR0NX9@Id_DCs2&o~Dib*2MkX!?Ga1c^}fFe)*9=*!Am>4HkhWAi3d0CnRSOB$Egd zY+!Yk9g_YeaPpro1;$9JdY?~+Ss>H9+EbQ9(6($6TAnZ9p~W+EB13t47?j1Q>GOpR zw9ay!uT%d0e5I+)lg_LQ`h8h;PSkPli_HI(D{)?V;{21~l`!63b803aSQu056XUd+ z!%oy|Qb5u!XA`CL0d%DS-1I&;wEa^{11CksM~nX{PkiwN)o%O-SNOLwD~Rx)=mNjv zez)r++x_m7lWh09UR@pcStH_o(eko0?HrwjcjWVAOgIoC-_xR{b{7a6@9Ne#0yqCQ z`gq;>b?d&dB+=@_W#wKn+W_KMw8yi6oK>4&W2Citp{Q2!=QfxU06)%Xw{P%xN9b?90d1iY7?Qv!!^f z7pLYSS^z@gWAA}j(C>~mXQYh3~`Y9myM zgz<{3F62(C;$fgcf~00#y$nWP#g_}*U7Zu4q1or5^+Y0CPrv{}G{6VFFu)tM0cLd2 z?Ap@BGZ(X68`?mwT^B_%>7-`x9@Qa+OWiPzz|Z2WEYv}!8Q5kbh-h&*!R#l+;1``( z67Uw_d_0OQX{U^enK6_NHB|QhFdM@{_u<}JF-(1G&!U8_!<2Nj5b9iUG1|}dWd+0C zk<-jBiysDmrE z;3suv`z_Q0jvipelk;%OeZ+15aVxqlOd%aeh({mKOC?SJZoi``XQz?&pB6_97;;>S z4d>W)-IPmgyAGbwO{8pW>D+a%o?_c|iR0w18^X^aC{&+%soGg1RO0rawr>iJuzj;4 z=)P}iVy%5cL%@C0i9o!(rE}jToMPKI)5gku^QfVLxB=`;i$scgzrfTpgS`9c2~xqC zM*5%FV1g`DhMpBS^J7Jo^-fVOX8)PM-M=%$Zc%#SSk0m|U2GX+S(JhuPXzk;wLvs( zk0EJVwa8edbV4#JE~5Rgt2y}_Z(;6-H9va!q4)qTytuDll%bx7o*By4R^sT}4VnVF zH051+B*aVmdW?J;xwoal#cVP8Z#d)$rm{eim@y&NQ2NoC+28m+ds-s&r(4V zCqc?@M>YO_OD4p#gP;sW!1w|`6c3;!ng`_#;3r(Z`AAKzD! z#1{=2y&Q8oRYHzYbTr1hV4)!=xqAy@(h5|$lPB1n!eHW^qv|b%DunJpbHlK!zwd4? zUuNZdDl~yOt0`UCG~&x9$HH!#4qGgjj;5pW&Mo#rTT(V8#OebSp64p-7dcnMf01*w zZq>|q)qPUeiIYN)ZCBE7f;#GwRAS*`}ZH^JKO z{JzLqjdC)U8P?;n)F!}5n3D^WkbfD$wS=3lP@UbyDV3_TeD175wtrsV&3xCDTGQC_ z&WSj^8R%_?8f*8noHFmva>}mxNlw{qhvk%YK1@^Ak0O*C zLF%{tB;mj4Ckg-a`y~AD9Fp))K1A^Mug#$*mJU#)5!5V%ZxoWovptl(B1`sgWS{I| z|32BnxvGatRS%&*sfV*ob^<77BzkJHzGBu6?|+aVD-&EkKLeu31b%Ib!idzM?fhIx zJ7;~Y^ZdNwMD6?xh^-gO^K&C3JsuLLk0lkL+2^aI2S+HM+@RmdN2i~fnKMFnfr#>I zT({R7F&5)+3wk^{*AR3mvCe({$qCxg84$fLl&%z5`S17sN}r`b%)d}O$pRwkLd`cq zr1zKC&T@PIsQ~%1exHQwivQZVnm&_PXUCI(xmG+3RfJg(u(Z zWUrUUYuUr^wXt?)vX2?$%VEz%$eg|BE zDDJ7T0*~~U2j>3Q+*pAZ|B$S}g^E}g#TeFsJmGg@1(q@^Fm<%X3Jf$`7sV-2hN8dB zROhHueva&x!oTR_xm;ysP}$}&MCeW%8*3=$O*Siji>4DVTlf8V_cSD5I{y)~N0O9N z{*dIO@^{I9Oz}v_o`|L<`i;{VjBESVGi7@BRiz8|kvhP0qYXnbkP5P~oSI8D&*h(N z1lW(^&Y_b^O0>;W1lda7*Ll3JznrX}Pe-F9!+ZH z1)haJAjRmXu|KzbFKOO8LMtWb3>;W~Z8mE;{% zdh1rNlU~^ElVe+Wcyr^$0^ebHTf^P1;b^?tj>;za>w9n`MtnekI|%H%rNglEY`c6a z+&=|l>WY^&etbUdyiuXnu=5@^|A_8idpR&j9_*3f7b=WO6K(nsMx}`!tju@rMX~s| zMdv71{3nA%Ad8X$_o6qX(blz+15>Rk!HoVk__pI}S8C>powivaP7B1rB6Yt{F$kF5 zJJ|xb_9QSQ^-yXxr*?L6U=hJpC7em2ph?p!3HE@PwGcr(!{w%n`B#@Wy-0lZb(}9( zB!NWJCZDv!1L#*B3WYQ&X!j(Nf(AgCO2Ir%X8jk;!OMEJcwje*A{`g*a8@CkpYkzE zy5=Q13+`pnCI!CkW3NG*xzs}o1P0`Z%^|PDBywGw%Xw%3J3Fo~dD%b(gPQw!U#E%p zdP%Ig_$-qfF&K5?3)z+#vj&-2$l>xE=CPA7>>mq1+Mh)nUmictRf{v`BN`+tJx?r` z??a`&fg#>@F4g4$4bb^0< zp&R&b_i_jS$zDx^@8%y!dyquB_EK(~AsMW!vXY}Lex#rNc+J7d8+@$z#DW(jtvE#) zJW|dpy*X?|Z7wzcZ7x}!4~7w?M*bR9V+K6Yp@Im zd?KT_rMr-8$>(+Xn~DvZP7lMa8bY8`9#;YSF17RE?Ha6|=FRCtlK4+1qAF=hX96=0 z!8}g`b9%hTpvuF^x-P$G7CkJCr}!D!@We2{AZc4RY~^Tv^2x?60lnm!y|C~(>`gjy zJi<5>Sck_1xGc-{BB0uHmB-K`FHTL0lCwRogne*`m!6Yrdph0qLCaOVxut>gHv{II z2m;A9JzKLaNQ!dl5f8Hv!U8WfMk+MxLO zBMHUcGc_nOy|B)5LcUfD_QZA49T2DY zZ=G|h{)({ydAb5}hz{h_t?2TDfLt|&JelTtw8S$xF;7k5JH1gFnh9+~QI;rS0b*9K zs}F@#11-6M0@c8+y=e^~r8MAIBm3cMDtXkP{xR`2PBc{Y!PI{eG;!2W{To*QHFeZK zHn5Nzh*u39^HKlt*ndkrZkFoC_kMBYTAcM<-oHL{vA~B?Ip5C@(aL0^O1Pu`>T4y| zv$c(^w4N$?kt7BNlIV+;awsk3#DAPic}BEtDKD>v=r|HAKGo(-A)3>UT#8=`P@=&?_Elumjc{E+aP2(mgzJGQ8(bSUNw~f_ zLxXFGg6qy+xNuWoHIme7+(4<hp z?g)ZANz+tfk^3h}*+;}Q>85Ya;@Zb{klAqY7!G|ReV1v!|VyoLp z-DrIpIx7GN=m0)uR$ZHjI1F`fyN0|vw)H@>iD9_?-4u!Ul3FeDXc)eZvb#b8WQZ0y z{Kj>HDsyy_M(wB3^uGGHb9(z2w&}h4LkayYX&Ur96R30SYu+TIU}s_>?*g7l=jj4Duwnq^pliWg(? zJT9JM;XkJPNXas#7487!3Q`<9ps1a}3<6Z^kJSt;V~!{a#^=>hU_9Xz1KSLX$J>f0 z>BYMyhwYj_H>mT!n|}VsGh((z)EJ+dPc6KeYi*fgTGIoEoojk!qoJ?qeX5J+PuIGL zk4J^nd;^{7rjs+>bV>hv=<~ju?}zEBC_$s2359zbIiLr7&*4P~oq@!r8Nz>pKK|HK z3aJCh926}${8byS& znFwR09TPPoyqG}x^pF#zL_0|5D3HeMAgykO6Kb|Untan@qf4mK2f`YysJAs*QzaV> zrD}~{MU9%RwZOK$#qGi7^zn3hrb<&NUfLa3jCWs}wflktBnv%N6`{BGhm)nvBSQKV zL24kRx^%sj9nxV6(u;LS;ebF$y~TeT#K$MI*cZ<`(aj?zGdbe66ap$oY+oWJkyF5S z@M}gn8LS{jZ0l$@T+i!E{ARiu`n_q^(5r@eRC0J_ku_P);ZO?zoJ-hs~hlMf!DM3XWxN&9Rts$Pgng z(=hQ3g7Eo+PNXS1W`ppp_vBK1beaa&nKAbH#}0B&SM#4M>dZ-{Kn}U$zP569a0RyU z3i#D?*ys*w1<+Z<@4aaS0%`?XcGFj2{r-@0aKS5Z{UqB8tfKMlI^bM^XaBK{@ALIE zzFhIqB)I~6I%^|Ec`p`t$t+NFOJgXMv8D~vfQjhlLa`esUq!B=O(dOYpb{~CUxl_PNW%q)HbQ}-j#4al%m0X zlnfLK2<|}y_aKJ5hYjvZhI`_X>(qz#&OTDuxF|v2c#dN zAb3W}gf8|_7ys;cc5$=4i)HV~F4lLnx~NT6u7|KxpR;7CC0eP#{10IHBj|&&nK(;! z&=%Gq*~ttSz3}TMqy+}om35rK%Q}QSl z1wwmZ!y})|ZszfDe*lh^=8wj@m&V#HY^+@m;aDr1GXm+|uep(3!m82v zr)aBqjkW4N(J;&E>{sgSv!9%uRs4yajcabrNl^t5dn11`-?pvZ2UG7U77i6sJZ~P} zZ8NPN4^oc>VLhHtJ(32GTm|Xee8hd~&GixM@~n-YWS|qmYkv9@A;B*zyf#H6EXCmk zUipbwf+db+*`dK*M{s|=&k6Uv2NCY>3HnUI)!mpgiLdS!oUNxjQ65Ul{>II)`iu5p ze}!TFb)^2*#Ow2PfJb}(<4W)BNsT^u7u;tk34sMo(GB#L7~amKy@-igYM(9i?9p&K z3D1@~X6tzk`j!O!w?8_e&pCk5*TqHNiTP8d%s~PA?e#2CP=M;!dG5cf)_)u7e`Z+! zd;Y-wZ?pFgck%_5cg)JDXl3tsGIiX?QC6E0A0FY>?s{6=${h)cm@p_VBPjlArmrFV zPn72JFZopW7C&vMYrO^4$J6+sz*ZM(?)`zV{h#(8O(vm$g+Jn5F7tMKB1X#`>$vMC zYXoUDg3Eqzj$p~}ID)>m5yYzzJo<%W1WEQ0B!rD1o=4D$p&P&>SZ*Hy9>H~csm!5M zX#~q1BREwXK_42y^1aRxe7PS-Fe5e$6D1yww*(vv_E;9ht8?EcI%$0kqP`}E^)-$9 ziq-pC?QdqH+!ZW@u=@C&9!A>h1?9l{Ot(54PMyVtb=L43cD5wOd7RPGH11GKbEPNp zF7z(&dz=lR|E4`O_t%zF2tDte$+#6yw2vY!*Mc;QAf3I(3F$b3^mn)>nFZmV!+s$P z*e_&3H~@{`697LiBtL)!-lB(HPwp?UAi83!7R#zaHdllcudAKMQu36EPqibYs&F9H z!`g_((TLvp-Z`TAwKyV>g4Fa2k%E*qyH=pg{J9wwZR}I_*Hm2i!Io6j;-S8wyiV%e zSX5xUuTQgJyMkaF5C+>%zangfQDKW-DPm;Ei!s(Ej={Q;y5`u8K5^C!R$tdpUq9}4 z_H_&ObyAcK;p5GAuHY+{)j`8GVF({DCp|+O*mUab-muPEQfDt}_{uSH^C$5Y&FbyG zYn&(WT*a(z-QUeuX4XnooYOmIGG*qCnsDL%%C#iw^ zb~;CVJ2h~cx|QaNm(AO3ZA&u6&lOLYzevHwT?552ALL{wldlczU(%=U6uU37V0xWk zIx%ceH9v!I;M%^LV#?q5)i)J+2WjETlR+}#bfJU#5iX0S?DfT<3n^X%DKD)zfJH*J zM-}#a$w+!0wXVW`0&5;+czHjbN8$_|T_uff)OXI&wfGrl{8FO!xKha{c%=SNt~hlx z$`kSXuqY{3#x_5sHd}w^T*cKtVVkio3zb>osxO|M$xFgZlujc(XEYAzB!DYj80LT= zHdH}iaMc`y;4^~Y#c!RX>Q4}4QpHV~acqi>KbK^Z(wI-`a>CdS?UWjrk8*Zii6?vj zneOWs04J@vS!LCkW$caIM;bs`qb;`;r22z^?;*gOg#kWy9|Df%(AfZEX}ko&)t^Kp zct95}GaQ_EMy{|3+0NC2U06&HwljIvFTdGkYnjuAj@ z!T`FQ06GByLMJa%b&w+2FDSB+2WzYNZmhZo9?%`Th&!S@f#y=X2xIxNGMV6*()P=}8qbmG}apcP*@TYQSe zm^@_*-?_NdmqAh{B~7QK3oCc?g)PUNgG>}vR;ir9ElNw{6kwC0qcK7sF^~e`LwNB1MPr%pAs-y8k8_hE4(<}>@ zAJl5IU~WN7o>h{Wno)ZZmNjQ-JRU6A`Us!N0iXuWM~T$-%6l;=l{tUo2UlZ*LDgVT zHW-8k;Y)^j1X#m4zmv;wE{+EMi$p^?rcLo^OZsdzyS8LS^>gd%(1oY;jY_v$&2cJN zKyO3FM_#5`xvccdIzDo*uQY)yjsdu<_QbEDw9O{>yRapswWtCgU%1l)+DxQ-SC;}F z4X&EqU7P;&KrHDQ@S2-bbWyyN$PZEDxUeHBVdl~D*Pp}0InV?5rgKvK{4j>7@##5n zQqiANp5*EUR%B^>di-K7={)TTM`*IzbDc7GWcsE>z15bvUwvpz&p*=O2aXr%BoMI7 zOjvL~?65^V5ob%0-RuiQ^Y64(9oj*&Y{Utv9U7-(q!H$%r;ZnO?Hg=nklErTZlT1j z1(di#y$w)rsNpxa+1VIt!65m}S3sakGy=VL7ws~58s2EMMIHq`Q#HNz(Q&i`{6$CG z%)S(+9q-Ue3M?mw18?R3LllTE5ZZ!)%@HNCs+L6Kv9FkXWU9Vo8w=l)M6v`gbSCpt z5>LjBNKE@~5nEc3vArikh%OtYw}^VzI2zTH5aX|?GBF~7k5K=oW>mk!@#;rBAK(gZ zPC-0R0qksXA1m_bffkFv8%Z(gR+=T{RSb<~-#!Y8%kU_HF`bIB^vn(c`67aq_;e(d zLkgJOlaB4(I{+)*)v)5Z?=Zl|evZO~RLH~w@9IL6*+k!fK`q^9NmDQUG{hcq^=vTF zfMcm>9gfiOE+{fwTP`~4x&73PivEagZLZvb<|TKVf5b=6OLK2Wl~A=491~sB8BN!H z5)O`;-BX>DCPXk-iL-D%hR}SRkMnVhHXn<>#rfD3#a0SHTjj)hSCQ-Rl9N2{b8>SW z9VwGa8bSc;cp$ms&BIEOWLQvWswAd4=5+?8IiR<=#-`a2XtOJAvy%1+mE#H>rVaX*S~iNw4FzU7UkJCp6~a?;k(Ov9`Eu0Jj$ObTDo|9Ds2VMLKM?|a z;6O{~!kM^^v~$fLCQC#yY2gQkOZ4GF!E&aVhb$ogOays#E}&U@@?NP^ljb)6Ynw5K z%w8h3GQd9aRuT)E?0z;_IbKGaVedBF49&Njo8jGOq^(splAAJle0g9tmPVQl;2D-G zo->RZW)(h)#r_I1&O~5J2e1W{mU_3(b=dd+Z4z&kfO08ekY>;Cz|131LOINcI;06P# zHFGv-xn$ecjoNnpp`SDGfI?nV9W=ZRq3vl~5p6Ft|A=NwCEEJzWV+4fP^k7~nW2%+ z`f9s_EbJSEM0oA9s4tB{xaswx--pdah69_72N6 z6NC7ElVGS)IgrpwOTd^Fc< z?(#%~?0lKM!mIY5@TTOke!gYTt>B;JLdd+n&tE4qHx{lptncq>-z&B6RqDHURUvrP zOKrrKO6I@s%xH*|Q5PxWNTiHEB4vCZDdW9J8S`8+Ht6Hn;LK=^9+DiJGlY&1h{Mch(@_lxMk~gJJRvU7EwzQIta?(FPNr(4^ z7FGpfJ-F3jCk@l~S)pGmlHp#YP%ARuy-1~2q{_VrXje&$^l0CswC~Z{_n0WF$=E2_ zWDC8?B&~RB?fVH@)f3&TcGQY=Qbjygkw9!p6wT*L`_vStanjF+>SR(nC+)D30-Th$ zPc1}H>uyYx%lvQ*ZHQN(8roF2G)l*Vq7PCgx%+oi0z1{p=Wz{Aa3)V{a02~$T-eYa zx02)p6db8@;@B5|)j2Wow63b6#8+?EFr5EwRoKq>#!8a=XQP!QiRrmN$|R=sE2Ax1 zkEg-^Ai;U>vrOsB!5()giaX%j)z|BuC{E+l#AsqRPOEvCg?D-4maFREZWo7fy-^o5 z#Vo^t0n)`W4oTvp7FyJqGs80gvJBGty*RajrR8;9IE0?)Y_;2&+U<<(c7}oh@eH@? zr*_{wRu__!Z8>0R=pudCJllnA$yx66T(u0Qg0*L3?W0^VNE?N145H^``hr5V18 zX!cZ*H?=e)X(L!uQ>mFA83Ekg{|g0cX~`&2-~+e!r!q$q@QD-EHrizJ7KTI>XMJ{( z?Gq33`uoJCDVisE%XP$Q*x)uQ^Y#I`9~=mX{%x0`y;i5l{%xCm=zJ1-?qfsqZ;K7| zhCLVV-*(#H#N|fvZ>#*FZqm4jcyn^&t*_0kuQ-A6Ve`wb4fd}Og*1P!yVu)19Sr-l z-sbPM7fo{nf3K(?_-4cL{jUZLYh%0lM?I$d7xk`P7A05iqBt2nz|Eg5_TLcZ|J4~k z=H=ZM;Xywa*KKXe|FgTlT2%$?MOQe+>{?$ukQ^sT$TrcxbA#2uvZ)Q)W%Wc`Pxa=F z79BT6d>_e7O`hs-S}{H~2!hpsIBQS6bz|~ld+YQSMn^LveygKZMwu{2qv+QxY^kmB zpn#1|fY^85Q4cVcn8Na6vyE-nNNRGjp1cO|o{$%A_G!u@y^O60{C-SeVye~4Sl_emNm3_eB4ty_mn!;L;Kbt;)vki#3z2Nu9tBk2&9!l z4hi$071WU&%ArvnL;5xGi3MgZY^GG@*W|}B7^c?p&f^nHtLxc2Pf8gUC7&|NQ;$#F z!6iC3;oqd`_t;lgD{>G-9H)m66VlG8iV6~p?Deh5cnfyI`C;hI!>!yN zhFsqN?M625Al_mJ_?+U$@Hn^@!FVbnMtjtm7k+-9G16nGAFqvY5Rm__e{fH;ru7g0 zYkPx^-(C~Voruj@tqo~zZ_U-!BtA;Ti^@5~uH|Kw%|8%DWC_f!getQ43I|+;6nwt0pLwRuvn1Ex1kcs>2WY9^98=5T= z-Xf*n?U^(^Cd~7J0^s!UyT=#pm16UQ%*9|AJRRBUg_&oh>px`mL4{yam;>N>$&ur? zO-;6%nnl*sNXNQzcJMMB`yRlm3kSv?4PmfW z3jB<)nM7lJ7L&HVpY`1=?#RN;3s?knpbS(#aC*uUFI>~w5bN*oTAmE|(A=;qY;%Gz zVgCEn;YVWc5t>6)P8KN}n*t;7Q`nbM)S%?)Eget3pK{=vN!8kXUeOv;Q)KO|vXO|Q zxmjiJ1E+o4sXFdIrVpJLH%|e5rka+ETad*q*cLZWE$#t*afNMhQJ*|*uR{GG zp?;8*UvwsL+#iYwbl0DB4NrdN@>87Q$@k!S(YHRNBK78pjT!a$aGDE#pFofY&xCXE z;qhKP8xo6Gw|2PV`e7YDJhqKB*fM6Jg~v0g7dln~#9P7#lvIBz5KnBY*Tg(y8I(CEkT2P>H8;mdG0*UfdRq|T z0+I4jw5CwOe~}GP>bekXOnynD0JpI}z1^DkLm8Qx!Wv75UUce{M=0S{ep3LraSYry ztqHhsHQz={P~P55pp-BuCC33Jia?q9Z$Lpy_Tpq6l<^G8Xa;3`%|+4L>Rfw~TAeYp zI%BZq5}tlPvmO}B{N;xustS?U5g{HT1)u}Y5|Z*kn>pMj#XIIU!?am|vD(9PlMX)) z81kZ5&QIv_dYi{auQ)%Mpv*D`lQYqhb}>eXmn@UG=1LX#I7gI#+eVqw!~9X3&WKn@ z=LE3Stp6nHrfM}ub4|=S=#@?tzM7!RA}{92-|nT!zqkg}c8k^_M8@m_izBtXk>`jv zch*VQ1jr|kJ4lN3xZ5B$Bl`}gB9uzjD4|7#hQ9a{xWzHt|AZRzZerBVfquW{ek)bK zgROocQG9nGtl#SJeyg$H5?jBQNc?X_{a#Y@t)Y#3lA;~>KnXaOC5^Vt{wy;)0X@3v7%It9 zjCS`(;VVoo6!5MZky26;9VOHGJlOi9JP(Qw8Jy=`g%7OKm?_T!UtXTcfUtLzA6~oNh8*Tmht^PW@_QxKT z{g!`dTYvS`Uz@Q0-h3PTqo^y{Ve&}HwX5~Nd^xQDZ0YKCewB>O2WxW9a$1u-ZB6pT zwv!ao`lz9J`eh7co`neaEtiN;{wgBEbhE=_ZY8?u#!(DH^2)u|66S@Wj9SMi z+d3lK(~k75o`1u+jL5v(+f;mi+T{Vds}xVNo)C9V4q}HX8YTgF} z>RC^)Jn_KmCZ;_*lQ1nvs}|1YkfoiyEbc}1KZzicu|7W3XZg{81UX`8ZVW;`C z%?_V@#q97vp8b?4)HXKn>9Cj@qg+% zT*%j9^ceO+wzP8@(YKNu@RJX-Hd|gq?YjNuAF-?ewCiAUR?y^p|Dtnpw!Vgw)>8`Kmm2i?0yjLg7c1jip5mqa@YF05B|qc0lH#N2ffl2X=fEI_{lu>+`M0TJ zWflW~zgdPBvyVi^0iHj`F`p{OWG;=~p%Kklb2^Lflep)-tH&fk*ZZ2gJecD8nA6p5 zCr%ekLil^NHcJw9(Sqi*UTXU>{xWq6N<0vs3blhdW$2zd#nw!I!9=Vr*LhWwfvZQL ziamZWdOt~uJ4TqqEsM-ffRvZw0RtgtN|C<7CdH{?4k+OT#Q|+bAtt`)A$L+OUWIr0 z`5%l~8_$$n99{x%8sM>u+@rxX z2A+t>+T#F`$vw|F$>a;mkW5~x8>jvG3Q39Z$A=&DOFd)SLw$)*(fxuUB~f+xVitT< zi#OUdBADHN(I_bvB`lPFFvMM4@Q&Se(wL4r&ia^Wl`~?_lz#ZUdOeMljlfv5yhudb zt?J^q5wfb|r#jH@q()9x1$Tc$&$QW=xni{#h^1z?$0ra~^w7)n6zi|)VwT%TVlaHY z+>4{P^bLvLTY}CYzIu%NT%Na*WXqg^pYwUE=(f!1^m(1WX*CM%lfu2@kZXu!vN{-} zo=bk^?1Yj`K|@G@#IP|u9eD*Ppjk{1W%okstu(PjbG$htp5kBtF_C`u9L+H_=!>6`Oa_^NXJ)j-bv&uR!Rc0iS>*O5iI{ne{l7g4++V z@gtLYz|%uYc2~s8Be)&99p}RU=k4t{%RNXpTA`6oIag@Uv$#U!6NFEZ8=}p*9Bksl zY@bR?H`B7pq`Q2^+`j*yhiEwcGu7#zDNp~*l1wl}G5?dSeQ$zd=aT@%y&4pkJ*!yz zGTxIW#;1OP81Hjpe7Js?)GWs19UhxOzMT#rF8?N^>7fnM%0X__PL9zYD9uQb*ECWC z$|}o9A=RM{dqW|cx>^MOT!zNX6+Jf9vBvAE`U$6e_Vvl?Xr^(n-BpBDn*Vc%B%@hfG8e z`zCu0y1L%=6e*{mlaqHlqBZ^^ zujt55f3W(~ACNzR0Mz~A;W}QeE=*cs5?}qMp8X2{xv`EYY>f;R8xW1(E2#|pyy+{N zJZ8=4B}G08WV&%=mYFOtapJ5GBRt}l071q+PE`O+W`KrAVAYO`FJ-Z#AGGne;k;^4 z@v0Agt84xg>!tNaeVu0wenW(3o1yLiEU$7fuzL*cApBZC2rF^6M&RW=+X|J3Ovu6d zkn(*9TXx2F7p~PB|5Uf}KVTofc~5@aM9vE)X}hIq@!AuQYcGh>1bPpzZ6za0jySJg za{3pNvK8IuE!8PRl%qPL9N@-i%Z|4q*^273P^pfTiprS5rfKuo8Tl)>Ij6*JKH$8j z!y$I){c1vd_De8xx$+(y%;LvGya%u9p-waqI&B08xG>HcHaia63#@Fl#G{QlmtU3W zF>^fat2}W*Rf23D9L8^?cRf5XSA-v!L8(Hq4aY7Oav(+tB`cY7~g{vRY z7PQ1e+aM2CQ@*g`^I`rmxu91lewrjP99u^&upD0+K)c+~rZ6{)SstLK*-QrT-TQ(D z{os(%Ncm*C=M!90Kb=hfU7zaLJ*}G1r?RWU78N8Xl50X0&JES6{K+sPP}Md6rkeew zCVKA%odPl-oEWySM@hs@JFx+=5>Z>3^hoB<-6l$y_(Bl3%dUQVh~NiQXF z((5W|F$vkgKyrREMb2C6N#YF9Df?|vT?k?vdh(lpFB z2eR}H-TsU-2AQ8x<|oJe%rZZBnV*Nv&kN>fmHF9de!et6KbxPw%}+CPmYwUD3_YgM zLdBS0V@lBw+FYiGaG7FYm4>y%RmwgrSE)Sq@RHU>?OgNAiI19JW|?35z?W&}moe~V zviYR|zL*;<_7V7Uk@;m6d^z9zvJ1WpHNV6>W`5~sez_jL^f15N17A)vzq|xrOqMZr z8+~32avPDPO22nQ3nSN6UgrkA-8S$EiJBA7jinevlkc0ZzkZpYc zJeC!W016#EMulE5=%&7n3kOBFddgYIIVCD*DCXqPfmFkzF-lW1>LYn%g6_}zP?R9UO=R>hFw5V zpI8j|WDYyR-i<)}B{g7mQN z$3%PsvVx_0izSMri_13Dag+siaD-xl;-k2tC#Zxs|7v1Ue=ed$EvH4j;1TDde(?|v zKa=}Bong+b8SIg8eD;1F1-g^+CdAVr(5)06VM=pIq1Ul0l9kg@i&y!~SUS)(QDPv=l*igYo?RtiOkxe!|lO{Q~8Jc@R(ZGC66e3^d=nO$Aw z%#Ft36yzEC7f|ynOG36s0xDdUC%&nwvvgx&ER+`t=7IE^D#!N2Sn%gDs6Vfx`8&^K z6oBMeIx`k2B=2jn7v8NnzC}zND=jf5vH|J*It|8n$|o_r@eITnY_sSgeB)VT{t+Ks zDLp8pLlcq;z{^u8m`pXP>2cs8B#Jk&YF`{s zUap|j32_EnB$e}3*Jei`op0udfqxNo{K8Of%r)wnBf9@h`ETpg4s$-`oW~8@NcylY zU?oXTPl&d>k~0V2>@e=0smgO*j(C%j#wq`HIU-0&N{%r=w~NXV^Rf5*&d@5P3g4f! z1-+EtnSTz&sCh_C63<)g+DdiswpMf=~f-m5V6$kuA(m=0;KwxncBw+izKfFbQ!4V z$tANr+HsM?B^Pl?xF}Wh;F5V5&)$ zwctLkCz*m)m37*}FN&hU^?|`P@?`f@07gK$zgNhr1J)6W33w3B3ImjSSJyur&zIrBnGi>gs>k)gi5`VfUi{X3)AMf12!6tdeld zBb7b}yd$51q|y(ZzZ7pC2tmKWRr%9o3m&%?{yIb2BhG8G+}5?!QvoagV^0OPo?Hfc z?QJyB*Jz;I?sE=wLOBlfd!vbgl0bInv4&7Y2&u@LW2DEArvbn;wY20nkA7J3+T=a0 z?3DXf)?VW{@KUrBCEna7En{TGlKJt`76mg&ug8A=&`GMi+s8p54MTcQmQQbew+R1w zlq4+kxcJku>2KlvH%%-`X~NNiPqg;QfKlDLTh^uJtUbX&&wIY18P$Hep%+rUQo*rJl!SJOBHO{Dc`z9x^yYAUWI*!28UP;E}#a7 zfTwVC+(Hc678ag>a3wsk=>}^0iv`g1_gd3CO0em5@rKpfFhN(DbW|8Ds!CE(c0>$ZKQXF$^Kw@nYsVtV zJxwGC{v0Z~r->GiwO^IWp;B8^sT|2;C7=*AanJ%MkCjs5apkT0qoa6WWVU#YfZiFD|Ys_@@|*j%Bi=z;*iN1PJ#gx!zhj`$lTFz<j*JT7T)$pW3@D&&%W)CS;^2beco6q(}+>Kqp!y2vVx$Z@U6$FaSPu3Js2 z;awd_66=OYO}iWUS^#0qS)~hw1JeYt6@cl)ID1!*NHR}goZenEy-yaw^rmQB+PJ%1 zruQrYp<6Q^5qpyM7QexU3F)BWLdV|BXQc24J4&aHvaqAw+B*Ka06MzS-qDA%u%q>{ zG*5<_C#Eu@-#e0zahs~UsQSEE4Jv@?sg>|n>5mDUhD?HPrK#sHcNFC}(hl6kqfiOUN6}G`XX1UVlDcPnSh~Zbr zyW!?gHXdA;SBqnhlKSeMt@R*U_7$iU%_M2OCcNXX->_7VmbnbP;ti?rYYQMIN87GS zzl~mQ)~G!Lo}Neq+2)=*TdNku{z|=UFTTJGmng@PaK4%cC>Nz6b(=c@<)+(b^T16w zn>D&zdCT*4AtWW>HzCPG&uc#G4WHp?vs`ZX{w~}7k(=-^okIK@w;Z(=*|RA%x9I&% ztRzL&3ccv2)g)1?2%oH$@5_7r^|}ju%&UA*vb7hfw7DP4@`J5~6Z#@a{qwA{YOt}g zMlpYboa>I}U~+AP>AwaClUwtuo%3=C$u70}To$2EeuT2RZLYdK)$H~LTes^gu-hAI z)7>s3+vv6tO`LI_N3U5-HBm8YDc>*ltf1HDu=@q_%SU-+Nfvi3jT{V1p9s|R9o%tt zGzXou7ZV3>)45~hQAvY=wUEQP<1#kEy?ckmHdI)8nXo+JcEIxW8e#1<4|%k&q6~i5 zSif7lv2(zmU*|g6&!cyQ-PnC%#|^kS%A$yQK>U;!1h(@aFN93Qc05++;#5UVpduEp zyi&*EiIwU`!5eiEh^|j$5g0Mb(2$7I21CC>dW)~}Xp0pqxUXTa)UgXbo=HiOPJhLA zF!}=n$wR~fe0(ft?H8%}8a~{B*!O6in_rkP5Z1k{hD!$&=s}1HZ;|Mx5CL#_jmlHv zu6e^6rT1%Cz;|f@Gj4+gY@<=P0kS%=_{m!1-_FIx z@3i4FBj)J)bP2&#YJ1SONRiWFZps*KH-(D;x

v*>>>&1p{+?FNkLpND05qx}9Uw zf=PWE9A3fv)+Ncp;Gt8+6WUS$NM)oF^;_qC*RPO8x@ zn)LU-F~9+IV!qx_tZL2^jht>De!2sn?&Q~jM7H?yWNp?0WU{8J_!%s$E#V*8N+X!G z{-r2UqeFNU>K+*a;?~`C=U&g`6}hR4zePghU9gPNWLa4GdsRvr6`noMdP{^24<~J| zx_zLD{U>wb_K`nql}#u^iYiYFqe~i z*R^yF_K-GavjQir+;R)EmA*$$Wk(&}je zf1>hR)V8lZ1p<8Xk&!2XX$_MxU3zAdt&9G%Bx~7IrRE9mDBIXMbzo`O*wc9IX*~8k zokqI)gJl(`t%WYY$Je&2Jy{})U(#WOytPcyTX*w+_^8I6>IByZiNhWqL0 zWT>9j%JX>*{@v;JS~5Q&S`zciAC>yt2OpvPXF5yE{TVA_iKa7;h-+ig02*RPs`btz zO5kqNx4U%tHz{LcsWv8eiRyy4(eK#Dh{kE)S=WyI;cBGn+Ck%P;@a`xDt)R?k}a6C z+gzhNFpaMMscra_&KeplR+K!)?6z{jTB-wGddbK^I$2bi0(A54rKo!mFucW2xaD8F zRPnXsx*{IjR*B$JS^zV7OppmEUR%gv@t`oW{znd-c2Ce@5SRgn!5!LRuzMzYF&={r z+45+r7327H;9{fJud)kc3L)DQv<{wcy1XM@N zeXdiK|FXZ-<~pb$rFdDrfVS0;4j?t@^W7f3S;T-m5@$YJE~mVC1?>Nti5|IGsvnYD z^2)_wTk_*3w&VaHR=El?<5Nk+c&~aFYkT#9I~%lny0%xZdzkiW%||L}!ozh+^d;J5 zH){9b=kp7`RX@1RV+p8U~wwc5+}7xevp|+C<)w;9r=hq;lrYUK<&~PbvDn^H2@J zHPE%!Z6hOkDwX0$+z{wq!;%L{e-qX_9V+IX$*nDS>wgdT|26jyuNR_jL@1JCsO14J zMFw)plG-gwOO$7E@q366HHjQ=KHxX3iOe4*{pdm5mzlUm7ttDR$#eSAdu%$c(FZKS zK;u%J>+;O_DDCBW?EQ5ll%=1CUXqUQhCuTZsQCqXPJVsxbb1Gt`_CsHeV*QdYsRo? zp`3M-c=(hll}}85wk{+e>@0(-3!8;iN>V07KGE}8l;x&a6?3@a3C*+#ni`N8RV65L z-CDl_Fz^CBtLK))ugy@YeO`b8vp24IN?ZR7_BPzjlh-q`+H&>(L30Bd6lWi-)51nt zrXhGc&f^~i#IBxk0m)*vg(D2Jv<2etyX)LlZQQm2ps~omAmr{7kL<3|J208yYOaAX z@Z_KiBKXF0LwR&S5zjS}oHbyymH|(>e!(w{w- z&0)niztyoqgSNfW_3h=i-VK=QN~Y=d?u`)yzxEnQT*EIeG_@N*0bH{&~$k1g6Y+LMZ^U zUcZPI{r;L4~&Zggvg8Fx#tG7A9!NfE!DMjJ9Ssr}O@<^i+8y~kn2cuo+bzE|p zp7{CW?3aS|%-EX_T7DHM2Q{=p?5ja~WI!c^Mf?LzRHaj!6v^MBTPf$`D=eaTBCi?5 z8$>(x9K0EUu~?2_kbE=8X|QU|zkLrb;fqSON1+W^5K5wgTn{=6dR%#k0Pg=?e(q=Y z-HHO_0tL_(1N9O2w350dS;_~1%==)e%ootc5^!f8Sd7k?ybIpKr8DP8T}M!;HU}-6~XPW30~biRj=XVVzZmcUEa= zo{q_o3mQB8Hg;CPUc1!6r&C3YT4x#4!{PSsAlGO^l1RlK<)?eUBHx9$iAq6v$Hd(- zsfQw|TD+Kc4BcJky7Oml{1W?)KRCwWJ6>Kd2um_5e1?9pfl)s-$nu_S60qjc8o%x> z2GjbCinc^GufB>nr(AKJR>|_*;8*(@{7)@m@r$?E;Em){sEdqK@xTZ#=EOc|v;pM? ze+`Tvx6rw!Tz9UCcJBdg_i9{Ic389*Ygp(nh&`LjlR8a7A%n5tc^t+|Rg2S(=&DU{ zfvGtMCzM&ysL7grQ=5Slw>!wQ6wq(EK}%`D;v@sY)2T6%y@*v@vZ_JxS;xi*H&~5c zZHq{+TxT)G^Y3PH)kB)kc{z&*^jS1KG>b`md6mw!cK)Rc@?sf>El1tIl}B*Y-4uTb z?lYBwKTb~91B#COh>rU>r$=ibj1XAX1bb>JyQC^r!8p&|o4v(UNFYHI$)E1fQm z8ZJZULTT>RO`_OCk7Ml;;Of8g_syUFPyQWRd5ht)e?0#Vt-N`kmUdw-ADXofbNO&}hcK592U5daK1}awclq#In%(8Y zYu)WGAI|Jyclj`?r`_el?PuCuK8!lc?((6$*S~Z5kir>ofc!Js>GGk_=JH`tT)%kZ z6!UYY`SF{dapq^L`MJgXl$xJM&Cg5bXN~#UY<|8mKQ-p(pZ`PMyMRYoT#e)JCc9x1 zlI#Kj!bMyp=qjMAfG&c%(WoyPbycd-qQrt2g=!=TM2kR3s7aSt?5EYXUhCb~yV8KS zL@NYD1uw0a*5al5uB-JfQ7-$RbI#1X^WKus&*%4if1d}l?_AH!oH=vm+=l)0T3?U& z4{RB4hkkkdht0T(^9~Qn_zxv>25CGZS0FZr$AUO<4vz&fe-4iYv1$&F1##yb9t+~d zIXo7`Cv$i#h(Xmn7DT+7$AUP!n#Y1zR?TBUTwl#&K|Ef~V?n%D&0|5lUoB%nFf4nE z{d=GN%dmgnuz$PRzdQzai2W;N|HiU^2eE&X*}oa=-|_6?^d9l$3OXDmoT2blAFVlr?R-% z#rO$6tL#uIF=k@-}As)VuO8tzYZ z0)6j)y$IBapEs=LLBY(+ErRr~-{N7w3eKQ{%cz|h-tQ?{ss8-+6G%92-1$1%6HS*d zM^+hwwqa6~UstryUy@-4^skrt^sJDoI4=S2z|N{T1-ENa$!7rSJGs zPdx_)Prw=Aqjr#A$?|!@WP-+b^y7J&k^Wh(=K2&Qyr2nQ7qv&l%Nb8@d$^eWpB31T zBy|@S(1Sr`x^*Fb{y5or{SxCEk*yZ$U)SOo0Y13Zg6B`n2%jTfBW2_zs=*@yU`z7W zY?S6gnX>6!%ZE{F?s;nJRXLe!@EvD1kT6=1G&hb8HU!cQ*(^p57zZQ7jLg9zKheNb zM$-$s4FzO-0$gNzMm8%<^;N;5jWyOeBV27V!U^#$_4(3patIOK;154!$7>0Vyk%45 zsIvhc&`Xc^EA2@e*o#igW~G4dk`H%g#Uy760KzO_Y#fG=6Kelku3HP^PqamRi5kCo zM?3+of1$pa6w@em(Jv_GLM*+88*fC7F)Kurmi2zlhE6{-SK~Dg<)z}h=Ar4cWX%tk ztHCBHK&%W&p|>Jyl5oWBs4t<~ceC0XS3t}TElz@_cxwguX;vHgOSmPmachG#XdwT~ z@<;sQeKIkBxch_#JJd4OCm=kQ^^sabp;UxD?KUz+49i0MV4+5zQs`V>D8u>l?P?33 zZK2GUHK3k8z6m0NS2b>anerdxl{K~H?YuIEuvcWqRSHp=J#DvlT$`vd>3E9rA}$R@ z_Xu6~F;t+C72q-s5xEaTc0K+Q>X%2T{_6ijDX^+JBtLx4x{1$IZ8_UEPD>*)>&nv%qy#=Bl2%<;N28aq2mL-W0MXeA` zAc*!Mc@^hM_)u@{A&)D%IW~Qfqd?u<@desMV3#?h1UwtN>`*4*pQpHV*)g3QA2sfE zND=siJ0K3}!d5Y)Q7&e@OR#~=Li+kXXO{`38PEgOvt~e#+!pm>d#DtuV|N?o?6#Y) zKstB57shLVLr(pShrYIyghX48kwPMkFM7;88ym@K4Mi@C#+lj9qjD@`RS(qEr@t`{ z?`IMj3-B~c&&yS4)4%?d$72~u`unFmaYv<#>1rOCaAq~k8UD^asYajPRev}vY>XB* zF9{1fP{AdCB|#g<4R{D`myfx#%L-1IqoRaleBn-^iL$@Dq0)g`Q6Y6YCquwf>2r@DarO@0;V-#UZM z_Kq(}$ZbYjYBc^>xs!)`h%Z5J9~7TYqn%MVs=VW;K9Z}Qit3#S?2mDG395InR*q6j z>`sz~pfya>(z0sCQZ8Q%;LkF_A4uTO!ly?;g(Eu@CqPj2mwxNLnFrTlo5Pt5x`BtUA>gz?a2d}}qJ|6HtISN+W;(>x(UtJZ2neMMxbHFDfb>QpB>A?di_ zXFEMuWpn+7&t5x4ATj>A)YI5YL>%D~_76T2_O)Z}K+`2BM*ir}Ys$93`x)8i=Kf67 zcdqP`$+)t!%$0rP=bUPM=(kRi>Hi+H`oG77%lWyBc3KP%lodDj#Iy318^z^3W=#G2 z&TM8jUc6M+Bxd~MFa)j21no!yttvg!Z;q667C>7HWyvs=oSoj+Z=VwY^pVRIlq8Lo z@KkRX$E+}wa5`Obo_%4phb5k~zw*mXPg64l*1!f~P3~Nme;ta0uQmsN)IuCmb-FUw zoObkW`k)p}6FY!ZYxVV(tnONZ5r|H94U}NLovt`po~;9gTcuFAhT|MwS1}`bsOm-z z;5@isC;5;+og)f8*`vBf(VH@TZLLbQ@v&)rGtgcFt3Ko8ZR zm?JYT9ekOQBI16vq5G+!IKxmt!v9hplW>*CjP$8uLn>puHBF5Ru|t#g5mJt^@k$Gy zJciu04m{S*Cm(W@i^!&60`keXYT~H?i=bllztI-%IFwSbwL07pm}m1rjRG>W}Pr<0;2MEweYB}ouhg?(PbEkfiTq+ zK2O|)NRouBB}w3{iZ2s+M>z8S_#t2#xsoM}zn-VKlATzoQUZnj8dJ^LuU`Z# z(-X9Wk7zi|hFuDgP?)D}b2#&Ka6kd;r$Vk)f3%#&fC_l8UZdwN_26>m&EZ^5mtXcj z+lrnYh@L?=^z2a3v;Ig`)`T#hn6fzLC5*+hmD&9Mq}}eE)7JAj=QPPB$d<_rhF0BU z!Kp-$73pM26e-+d^MYif%-6X+PXHHM+Xkz%;=N}Jx)nAz+5Pk4Yc(+iv!2dDgDpB0=Xzfi$ZILk{3oci>O7ILizjn{uEuEUGxuRp}> zypo_g0XnErp?d5Q%$OuIB>lPC^feZR&w2ngIRH(g3C;GO5(Aq)N(wVhIY-$jTTfIs zio)UELlu{%{l4Be$~w(9%u}uqx65PU%=|r_nGYQaXXb;-nYn7JWRJ~mV?UMbar#&4 zem^2CgA1}sEtX0<&&5iRn5h!5@Mswu$(+Juz+a}qkeHdE6wafaW70iY>e*V^uz8TKkQZ0P>42?{V{ymVy7Rri@Z{E+#uqTTfTmj z*d$koGCUf?PmdfV8F=HCdd1P0r>$?Y92nTV^joIRdT~CUxLCFFY|63Xmpux@DO@2T zoN*XaICS$23Kzlfg^-|A()S! zyve*)@wqtvYIQV^S#C6JHm9-T6kY0V9#6;7-BRkV(NZ;Nys!c7D;Wa3PYCeg5r_aq ztN}yYU5ri`ym(V%1fF(U%nE-%g@?znZ~_XywhIcwVO4@A4W#FmSA+yu#Y|T9C`sw} znKWMhyH}Ji>kOkH?$8C&i1}L0K6_%y;ORup96pTnuPEzo*KF@ZD9*Z#O_vA+oFo@w z=t_|`f`Yhx07G)mB#$ko<;BIfi&R{& z=%Tp$yxnDj*~<-5!y6=8I+!=e8z!hXNMXIF8_!ls}B#x zeR-`4lyY@jL5R%{sTDS=XN@B{?Q#_bkL_zI3guGdEAs{trZ&A)*w0DpJ@5EPwa)k} zlc|staqjz5FhlB~;}pkPs|G1a3prb@gE>sog8xqL2OMsD4EsE7Y9PcwF&A(eny3e*4;j(T*7( zi<|mq_ZKkFa~0HAfakC%UTbvbios0f2q~@{wO+AcV#YfQRqC?XTOrUeP3h|T8N1~r z0ux@pQf=lCfe_$(lPMHm#eD5lnRoB#qVUJU#xG&CzM=LtL2$su8Yu}PoD!D#1tl!w zhf^dAR%;GFN$O?zs^yKp%?J*<|yf}&*0Euu@#(_`&AlzqL z`2C1&ghX6#?!eeG5F(Q&HFsc;6f9fiQ+98)kvoAvq^hXfnAQsQ7fp0cZa7pX!>KZ@ z6ze3>uCc zEbch}!CfK>Pqu+k)a}v`zPS2yQ9?rW{4*6kB6! z+3shb9mFkajk!<$AY)+c(vS=2i~$#LtilDnyFZ&x)_pc}GSzxXys3_;dpIS+DJ!TK z79KZjWYUZYyX^@()r1WRpC;77IrQ`|Tu;8fpAt8&0ND!s2kXnK&2MW0dt~!#utdxN z8R2yshtJ%ZH48*7#Saq2H>{ZGlLRU-36r7s0ZQ+IIB)T~X~Iq>kRYY>N|sXI0fpBG z`N@CUk+KjElQJm}_7N5e^FjxSLK{{Xvi}V$((?Dl6`%S{4vT$$Vg39@{bcjgW6c&Z zs`zJ6{bVZ_nd2IdFKUH5;Dq~C#|+$d>K^+HJYW_QGxx(4yRv=zuZjB-qgFV;Y^S&i z#oL;`@rf%E(n$EsS$qjyIiK7F&IDn!88}E0DnuP zO-oUYmRLT<`0GJ?vW$qu77Z~00T#OT3uoVji&ZDVD<=XK*q~5>H}^v)!5kO&KS#Vi za^6Z{ihI{ zCqYGf({92nCqA<_>>I{*zLE9Ow?AaFW;U$6IoTH6b>mNZcA!oW-u5oR6K(kbaIw^u z`6U*_?e{N!zMOJ~`Z9PeS-k69{2DbLd|G&>@UC{Bna$!Iit-^UF|t6Jsg&^2v%9zl zNLzWlWA`4uB=g(K!|1(4rT*@tYGd8_BRPL)pL!-Z%VRjqoXjXbLwek3FzHW%+?_mj zxXC-wdcuru+YXHfX)y&p2MJH$vzjp@i-*WWqG6Euyb2#eRXOCvao!Ju(Zwk^|{5GZB=Y_ckj;j-aj5aohDIa^X=HfR{T`zbE4eb86>K3h@UfR6o43W z@crt%(r=l7GkLOWcABevP=` zC-+A#_)Yf582&Ly`ZG%`;MU9(;8J5%g8!`F-%>6y(;QZ}@l}Z93XdBtto24>sZM^n zZwX0^avi5-|1;XL;UY(N^g9NN>{;rRB;sxWZXiFf*?Ld zAih%uAil1E*fs(|tTaI!Kp<9f$971n0KGtp>uzqSBFS5+Y7QPVww{jFl9|&v)J25Y zi~$j3I-!MD4bAd6g~n+!q* za4A+lnP%+MzwFA&Q|$Azhdn)4@~C@vjNavT!b#B9fs>Z8&q7U8ui`nG z<>bCa$7;lQst}euaNanE@Lk(;p^x+e!UBf46l3#WN6}|O^ zQjCY(i7sS-MLAHRO{V|{nL_IJjRknxId>&WZ8jNGvsw2`1s~tiG6oP_xdrVJG{??A z4CHeZ2l$VW!YXwDF&w~S10~0!T(1&g8}p;C9{^Dw5&*z5h8F=$g9SZA?Mqls`MZ^P zaW;D6)yQajBeS4hNjD^yI@dcYxwukuOqX3EPTmVI@i-*i=pUU_Ihnb-z~4*}m+N#& zs+i!=M(Pd|?0k^0rg=uURzJ#?m;%VzeJ;y3Oy*JAG6`B_3ktY}nqlxH>EyiEysU|P zMN*$uUK+akbSL z+RsY4YQ4R18~k7?X9R&(SD(~A>$xCIcbtd(Br@M^%nF+G8RU&pPu-3Mpujsxu~|g6 ztHRI?)d>9sLg|ZOUVbZiB;LOc+ej_}7S48)M*Sx|1q4l`=SSl6us`}ZgP+dLEP`vd@;R?^)t2!3)t zej__ZGyY763&r$^&r}1D)Dnt)=@VMJ=0y7+;ZRD6sOisF){XcUL>lQ4?_2#Fr76DQtv=_$Rxxh__R$UD(%3u#ggni<; zR=|nZonrwfE|4L~BWen304$?NvJ@7M9mY;j;}TRbXU?P;V6T{U6A@gb@^wN>UR#+G zdC8#O&i!r={GeD<9`UuP>uXJvC041cCs*O67N2A(1e{}c4{=ZJ$US+Dr9LOaHwuZ@ z6PX7cgG-3Pd1+wm=LMkZDD#Yj&q8tKPVRCZ!Y=nQoFFMcp8kWSl3RZVSo;$P7VR|7 z`nF+aLnKlnl4TMs(teDBIXAqWbaD|Q>}=w7 z63vZajiJhPF=9e!2)Y*GuP{aIN+ajFHbNM?z74WRDNp!U1PYaED?6cF9kou z5CH~%nKG->^hiC5I3miSA!nTZ@RwmIDQEkh@#s)k{O)e++5>ic#c52`q%l!WW4N}8 zXiPNJ{G161ZO$k%GYlm**^)UsQ=OefZ#GB7DZ}9RZ*yy7UTsr=zP38&0mG8abE{U)2`og$hWeyluMPp*Z=J|Ze4+ml0_gj2D=c$1~cv*)SLX=&kuT>jYo>XkcE-dBnP-LizU^Z3sw?Tv89)t8gmt<{p3#L=E3dN6$%b>4R(S58i zSY)I+!F649Qe`v+h0HO@&zkD=`B^0b(Bv(~XGau^=Of+mPn_g%rpVRD(3E3QOi`S| z$dpq7;FQ11acYb68K6+GCl22Nbz&IWtpUv9W6SG`kxsYXRIFiiQFelZa1-*z)|KGF z*kd^Q?Jw=R+3YCHtIGUOxu@99s_-byJS3-w*y&)I(&{DaY6EpP8h$2Uard z!vL6V@B;#1-U~k<0Onud2L!-;27XLHmbE=lepm?)ER!l4&wewMI{JVT4HQBsqr1FV zb6$EauQCMD(JIio!`y3rhIaZuoQ6sWs8hi&0sP&gYU8IAeIr5JA&`d1xaA#Y2)<{K{p8;*ezcojCB=BEvZ zE{xk}D<7{tUyNZ|N&}=N;pyzQ5)Qy^WqD2}kUmJDo1f5a*;GQefaa9fr7(D6(Thb8 zziE$$_z&r`F}%RevM-UBZP74LYn{dSy{J>k1{@nCF6&T)QrgQ^R!f8 zWZ7^G@tX>S*)O*8fgti2ECDhlpalc`g8W8>V5$YTVceBx#i&5~ZKAx1oQ6RppDoA^J$rIk6NE-A zZ6{arfIW$_$|@@Q5fve|VH~gO_oyn~z3&xM0RbQPi>(Z2A-}apA)0FS{5BoVkp!Ma zUE53H=GgQeoJ9k>I5LMl88Vy^m>OiHaJC2~1;c@x6shvq0)_AE-JZqSU@Gvp{6TCA zQk{%coqdK~i=)d9qp853SxMmiqIB2UN_WEFFk+SptjQoHs@R_ZIeiD5G};y zB=TPP?lIO-^<_b$Y>zakLXoN-l*TR@i`q3oq=+p^qAtk?;g_aljNSF|UCyIABk#{T z)Gg=r0#-OeEJAJp1nw#UG4l-^YFGPSl8+S%`B;!qDC3zO*=#xmi}SkA!9w&CuMV`~ z6$332A<<9nHa)c&pj!;BtT!vt0T(q%ALl4S4$5G+Z_|UueW!|0EJ84QkBeYd)^{4@ z+?%zml@wyeaayEqpLO-1i4mmwr7ETblfFZLnOOo-w|xm$fneJ+gU(VuVnNtU9b-04Hcv5uEelT;YJIaEX=BUyd@{VFR;JVJvuPhr z%w2W_PJ4un+xSCQF}Kf)8fWj~8w!~cT)`B1qc~))F7oh{)@6j)l>2G=@ylPv?$M0A z`0Mh?WnfsQgr_XdC(hcR$w98T)XA?o@DIk5s4HlK@vs5nO?@6-!MBeDeyHoc(7-N4 z%l5#z97QAL>-;F7;t#V_lLgE%-Nl+><%qig9a`ckZG3h)8#zu&Ek~=q-$RL$aE8by z)CoPKk!y1`c`IK&8fX1C?2j?NQy_biV=Q;-N*HHtDhAZY*G{yn3M6fjFMYVO{Spe} z)tweyf%TH69O{MKzGo(DdwPFvNk~b*(UpTff#;(;f1xZ}jrq6$$di(qKwVI@cN?ec z)*~yBp5QYdf>CKI>>D_%Ld~D@HX6|euno0@_`gJtqLO5^`8gWp?#H+#7s(J(<};*i zJgR*Rb?)@Q4!Cp-1(SzZU5IwZ?i1Ms?mFJ7&az7se>oTik9BoCA4`%OcN9|cGbd8i z26MT&)fqn$$i9qJ_}xbGTiS-m2glR9_K16rSz&`XEBqu;aOgy(0%;xaF2-jkSa~)y z3O-pw%eD6y^_51!;Dc$r@)fFGE-LRxD5vwqLlu&fKI^2Tnk<X4wIM=omS8h$Tfj%{Yu+}3(B(L@BkA>gxqq05#=$j<=p%6Mmg};67Ve}?BhM? z3$O8BPgO1+VIS{~&-ZG)(f^m@4Rhe{B;fyAY9H^BpL>n>UaE3mseQZy_HMjkukpsk zc;lYq^*j#+VNYs{7lKb@5`7}k?-l(Wg8&VjcNNHLnCIa7Jy|ov%yJ5bKUZqgm9t4C zCQS?8WM*|QvP5}4+3f${e4v=3ugP2c;okCEQI)IE25LgnJlJi!+vHDfhE>%jwF)WJ zD#jOiD=qRrhS~9Azt6-Xj}g^CXi^39nUr?C*<{Qz87`nmRUY`TT5B6eWx?Qix$9Md zx92)OIM-0M;L>J#kJC)jbAEktejjyR6MK0xy6#HX@=pI~c3&VlS+K3#X#(O}eq5K6 z{wt7ndl~y$%A1hh*stO06$$S^ZYRjD&*QpN?sKNL z7jEv(w->VI7xa=})cIXlv(7fpPX+E`S?57s>wjv-f{I_5XY7|4H`8IK0pPE##Q>0kk5%K40SQ zQ&#gkn}3cMedhUI_Y?E}%JoS|+A>o(#|frgV6eq_5qRSbJ`-RERhIdfzR0XADhOh! z9Y2+(Vbo*zO^h0$kBtc~xVBk_+5G|X#aR0AN9ODI^f$Kka~m$SyAbYjLVNA!BVl+w zgTu=927{HA_iGKMc&{%kgKZ|BWs77E%7h_(n!ilUU#T)qQzmtX<(6-QchW8?%G0Ap zmZ6j|m^@SQaHYbGK(&?IWpu7bPZF4!jy(x2pt&@c-+ z!JEfFyS1Y1P?6#jDKUH&u6g^u%{}`PJ2HwIeyMOce_c{E{6Ez2wYlUS6dMoF%lDDE z1$5xwUxewNoX+E}z_I4-%k>-aI?g+bVAMXct+^ZwO7P_ax;_W)Z2tg3em6%16?q+9FQMal+=IG*F}83RwGea+`W9;8CX~Mg zf&g1Pn>jRG;tzHRh`tV*5FJ77Z5?Qb=u~R2GDip;Qbmt_&Ki|mZ&Jj?mdXhblO#|~ zg0KI8^g^#Yg#Db-wLOS-O?c)MEs*Wt;xY)Q45Y9pQ!-C6{(K5RS%=_N5O^~O+K2nl z`v~3xJ^>#0O9-v)h7Mmd!~jpC7WQ?t@N;Tmfe*!5Q>0;hQWedxA;cGrcP0}iEEx#b zi8x*-u(Mgz+2#W1>=C6ijXK*SMH5!>4~BGG!%vzx$~E9Nuzo7x5#kdlk|>@gijJ|r z+0@^n0!4g+{jGTq`@4oq9g}<`^ZKcv@k*`*68`ow?<$qA=d#NT1wcaQ4UReM@(^>T zLf;qgeO+M3oRatGEjH0r#Z(fs$nY^WV6XD>;9^U3HEDk^>FW`iaoZJnTGLUl28U~K zE-XV4Bnod!Q-RK)uu51PB&&o6*ck2pq_^H=)xfk@DpImM~O z;BR*bCB{cmP0lEFn`yzWoA&H|$5+7fW{T~Srdt_&D8)YYIL=$|`Y5%0VlT^w_p*HH zUY3vUW%<}%mXEmdV-aTqZt~;L^ zGxT5Be~!}5T&=mhF-n7~{;uGcS zL*dYBxtf`~V)ek$JPPu&3SbW%f`GjP&~U+jQYe^S zNJ1CWe!dk0{b&js~U4L{%!`)l9_JYs)6ptX9x6hr}u$8x7&sAIVn4)^wLgWvvbAAeu@R#Ig=y~msY zZ$Oa0R96D!YX5h3HYh3}TY=_EUrlhb6|>D4O2Aeu&%?ccFW8N>7hMiFxxxY5nv7E= zycLu7Dq?&-0dam9&!8wIzq%|>^JYPu{_IPHrU+}V=C#oX>0ba3>5h-n5VX&HNS21> zFtU`sT^dS=+SjPHoALy?;JIrq%i~g%!zXynsN2=F`6*!2|q&!T~>A0$95g(^q=JWsPX^E+sU3Zo7%mi`UcphaL&_c_R_GakU{ zylPJrHSY1dTnV2$FprZ(umxgcgj}tj4AIDnV-Qsj9)x6iv}xNWZUbaA6y>ku$%*0* z(aWHeQSpW$UMj4k`vZ=67M+b^lJWvHthi}jiN*<)(g>{*o28J@X1(G7nagkm=_nEh zFuXZRL%f;K{uqzkie|inNSDkf{;vz16Q5w9(O%h?k3tU`JEP*j{xV9Vi<2rKYQ0Ff z?dLcxT20is6PKtf7BRl%7IeBXJt{X~d5~YmNAmpzYs9zWyeMwPnB^)!PN`ybQrw9? zBCKuPHmO~8h3}_L7RFZiIgNc*reyRR3O)7LUG7Q@yVy989%2TFv&@1F(S+;KatS_% zQOZlQiC=kukhC_kU+FD;NRsZHu^>;gG*OOCG)O74D0nd|m~Q91RDw@f&^Xp=1PV6b zorFC4o?%E2*&n0+W4Icn|?+Wg@J?lKV=^q^S`(J{;m;y z^80+5?Vg|R<8?3PU0wq6^?9ySV!LlYaK5dSE&r%Fj(m$hZ+p?GEZQ$taQj8x`*?dj z@Ac44E2Q?oOMAGH6l?iQ_8WtS$hgr-<7w8exK~sOxlqbW`RxYpf;|ZDb$cxEURdn` zuiZX2DNv7XnA-79gm;S{hhU~=hV^NBxSl(;M=You7ykUz@QJ2B2bv0%;`z9ri>c6Q z8p@mg_QNn!)&-Bd7bDqLV-3(G=LOue}fn zT27{fT94%mt5c4~QLBZqPbkVhcXM8?5 zol-OX({z7551?Gj2{-gMhZn-$*wy6#^IK67R-1R6JXE$!QmYAbKS#AlCV(~qz3AI} zg%|Y=*AWt=djJV8Qjp-*Eo>|o(zy-#PIiD{h2K$ytyp2PQsGjlFok5tIQS?RbK@wg z_$YMyD605;yXy1$Sihvp`Xzf?KdXP#Aztk!T-r@I+wFIIx=VYyS9>I{^1JElg%%9o zzs++pUZkr_d7r-?^nM(zFAoZO*F^#Tz>^I^mv{{8^E9zHHt0Z^5ppek2_mG$3L_u*#Gg@o8$NOIsSSY ze-7CaJpC<jwvNB4(GP`b3dYbBy!s7H z!6nKRM7zDGpnv0^zrWvO{7(B1({L$)Q-Hf@qNJ2bvV}$yzk+vso04wHLj}GL1`VrVi#Ejg`u$n7Az^ z1-VJ4w6=1;L!jlwZ&=T>X$n<~s#LAobP*wvE?yU@HV-Cm{zfq+6!Te*fjY`NfT3HT z6*19_3&-WVQ?$O@%W-ROs8@UTg#YcgU3X#s-rM`ZKZ@M_4b}7UNOgs3wEBbW=S~eJ zaAy(=LuWrUo$|{(lwa1@@82||3^#KL`|(6Bh*~mbj0L|xTB2q4VLuLqA0_NZ#{ule zK=vaGDORr#mbz(c*pwhzloBXAzG@H&idSeB)yk?!7VaeE?JqL{Ua)MDcrR12MCoQowaAKdMX&2?zKf!1*` zZnVaxxKvKJMTV~a=e*yPvMIXzl`t2YOd|u9L-{q|j8w%Cps-Algv97OUkR}07N5#a z&)L?uzRxr24O@IP*Yb6;eCBmad8fTJRRS@8V*Mgn?eW*$l~(TdV=HC7rM7ziMf|6{ zA36S0;78T}<$L%S8()R;9LygEj7=!68|-&b`CXdju9aMH)}}{1?4Y*T`%|PImvdgG zP=s^ClTCWAMr7nQU7X+%(`|$8jPV);C+BXfk_^Ggy*eaBN*fbM!25vDiC=L`r%|H! zcfElTN0I`JsNRGc!a8z|PC?k4$g+X_L*B(g*T4(Hs3+o)^P)m%q#M`5E!(tQXIqUM ze4q^4XCLhd)ylm*C`Hxmrl&N*_HV9ir84E?q5nu$dl~C76Xn9$xYn=_@1s-#O#DT{ z_{r-yUMK(K;=ceUM*=UB7J8Rdz~3Jz>H|bkug(BPtyd^&!!y}zx|y$H(m4Ee?%ITN z83QI_`2_Z3dPQ0FRXLgSQly-V<$_`FjWRU;uVL}$0{1c({P}V>>(jgHa~OLw7o<3t zK|AA&Ao(s=l2w0|5uk`N)!ZOzikCRY)qg`Bca#Ru@EQ$(kO2Z0RcE^y^kr8S(C;?k z$+vt69z4sC=vHk&sr53xWC?U(i+Q-bfwyGjsvBrUd(==XPbc3m$|~g9bbP93y$l7;F__MkaLePlohz>sVd6V<`7( z3ueYkd6yal`$1_$NlA1m@pcwE)^a2>LE$Cr21 zD)P%`WYF~H&M3YQb$XuQf9EMuOnO-Y_%*v&(=N{p$Y8usZ64}3lPJ~(v4V=`Bi1)D0yMyn@#3d zsqxV!+dLe9x=QtT^G8bjxpAX)I)e7uh#L(XZL<;Qs@Tyc;+?LRK}$@R z8FN2hs)QM{)Xf-QZ__M(`GNFDTx=9KHohu;o${K-<5c74(XT1yNy2IywXXMZLSIdc zkm2W^)76~wqVzzz$t(vGx3BTe3$h@cGdw4dK8xa6)cTequk#6!mKWa#(lSpWEp1OQ zUgtnwIH%#r^`)9nZoj-qqpe@SvZe1}!dNTPA-FI2%7aiMvlto(um%cb1MQ;VCQW&E zK4CP7^hbCD07+)5bTZS8GersDSf#UC!kA9!IhiZjSZjsut5LeECwPcJ)&@Ox&xlkF z%?m@N_OOp?&rolLjmCJ9UdA0txAH0sZ#H%F9^p8e!h>E+(HhWkVI+yR+TVj&%~fV~ z^5gWf7hyUq{(IX5R-C<+o0B_l%>v_U6vKHLk{~I5=;B90+`wyLbcZkuKPr5HrY*)5 zpFptXBW&{(*lu`?t@vFKadXlu!cd0d00h$4_(X%~%RSb(@_7Yk5my&b*n>Ep(Z#&? zM5z>D)RLR<2;-RN1@_bS4y55ZbF)fVMm4A2& z72ACEh;5hBik|i_JGQ;>2(f8=>I*PrrnWAL}uSn#NZDDTK}!w8bt7=QL5aphK0mr@*c zch_w<{zAR;)=J7xKIQbdb9>Q#zRw*#$Hbif+?Af9{cq?CIA{@Vdm**k+l!zqS@pYjmL>8XJUs!qo9IzAn1o+&aM)$zdD+^=gf33i#KuHY{MJe7eH{Isn@Dmd zLC5MP8eKTxVNhL24mn3V)EqRtda5Ut3ARc3xFOoI@?yt=4U^n=!ZZ%-w5d~F;x(Sc zTQ-7{5=FRapAfx4r(0yFJnJef>Acs!XS@d7^L}U0|?elrHm9fC2`+Ijo+u>c`=j$hSug!X6c1x z;O^M!FUvd{T8mEC9`O*|K}!XGyEY6i(Y-+xC#WX6K$R*in+)!j7#J<|P@sZZ%RHH~ z!eY~3A9ZoDWW!df@Ln%p8iqgTF3*R#Xzq;Q8R5L(db}^-@~#O<>Lnmx z-<~Bwr_9yW+(nd&!b+5+@C$oJmRIN;gmn*PEw_d3^BUGdLdX@w71kZtQnJF@E2*5t?iD-z&r z-loWgi=lM)9q7mX4nMl%C|@_V_%#*llN4&@keodl5mo*>g5kR5zSmOr36kBulZ!K_~<`U$O_8OB#PkQr$6!9*yUoNi;} zj689XaZ!Oe_-C&E0%=*oShkz`Pl`7ZO|SR|3Fbd?$)h^9V?ts>ryTxJqp0klk)BdzSxP zG{q1WC5rt2QdGu?xajnNn(?b$S^g;b^zKE(o?;>D7jFbtdpL{8{gE=?Pw#X8Rq?e^6UMtfF64WU%d^}c=z^kOLkW7!N*5Nw<4`>Y!9Ho=t|+r${@jn^ke#!#jnnLRYPj>r=qj4%we zMr>X@%*yb>| zPXeK-NIQ6B5{Ncsymm8^sXKF|KA>|T09bAIGX)>QT`5cOM~KV<6S2Sr_5$S{`2^D6 zUlAa!I>H3$2?A;JD~hQvf|MYTAeIkpVV5oo>Wh_0$;rHFei2<>Pp!9L>$OVj@85*2 zKjw2lTR_lWETR4T;U=`tQhT#sQ4F)Oz28uKb4`UxXMqW<6lOE}pkZX8XMp$VDPLN^ zf1SH5!hcPwAI5*hWbU+O{IyUCCnNIkxk1RLj~%O!ts1G?qAuB9Y7JVaP~ik#r7_+1 z@-Q=yCXVTuKj4^ttqCdMOE2fJNe|YYAf8QmlfnmF^)mXx9lLp$W|YqdUpQ#@Eoyg- zquu$~Zstg8w2?Ksj*JI3X(Y&OuPg{w@Hc!x;&j(&X?iSonX7@ID}UPbX$`&9^lrO+ zsVNyyUa1EcU-q5iubs4%)uMDFE@eYyIarzmIp8}BLez$k-X7QqdTQ1lq*Q|2YXF{? z>y$4o!3z=aYP7acj8fA9$-{Y)5%q>drysc6u=sE%q<_N}DZ5$9$ zG>OnDypRT2F=60=%#e z)f|orUy%EhQO_vm5lx1v>a3@8VEpv#Rs96kZUu>5R+lvfNK#%2BWt9Qu^Ktgt2GXU znW)rxaUMOo9Uz0Zs-XN~FHm{T#>-hDT$xk zNhvlPmej7hV}GmQuTlJQfYfbBT5 zFcTM2l%>WOm4gmv;=vmbGdBf=-QmTZdpNgLgV8I%)vYck6EN1m;DHcrXz_0cSj#)G zWczhx+xa|!;gj)?yO9&7TOcWxFQwiGWAB$Ly~kMZ=}AF)Ax>L5234z}XQ+ebp2+e- zK>F`{&`ggN&zEL@Nuv}Nh;2%o zovUbe$~N2AX7F#>?1To8mZsUr({B_cgF&^ra68_@JNc{VbZL5am{OWJA;zeJnOr82 zKAFKH5N^HZ6MFR2M+z`k6EF{LvIDc~dIrqOJWaD0h3V9M&BpwkG|H8 zu-1)ot&mC!aWDPYEp`9HYWkR8F(ck_DVWx`ozs z3azPMunvFuP#MDAG=x!(Aw2RwYzPnfZEKnkYuXF}8bhlt%<*O^Y5zCha(kp#(&Zk&V zAz^&}fijOz(LCe76s!mMiuQV~O%EWlA-IUwawadvY|B)AN4To=y3g5{bzBLMbj56H}X5JG? zSJuYEW2U5^ixMysB0|R7PzNta-ls_-%5uT$UQibNKL_E0W8|ZlAfNeOcg*?ly9s*f0y+aTrhk@<)uQ%v`OO4czqE5Hjy zGAK)b$x537RqX2n>g(CRDmGf!*T31lI{m(;@#Y0anS50>e*+X-zG$?gWrJG2%F%KP zT5ctBS#872DWE^R3|1(ZBgBHjBd4V-EMlN2X{GXIF*7YfzJXIHU}e%c=L5RN>&9sx z2)W7eA8<)mCPIZ~y(`$Alin5VPJ;acR&_NpTfnkC*oiC~aAVK+%CFsvI2X94N%Cum zzzTzw9H0>=TV=Ek&Bu)dayw(esfd6TlBrnVH4Jm{4dkb9fgrpC8;${sg9>1XJNPcd z>|0p&be7SuuSBuXGfMSkSUrz%5@T|xS|=p$I0gHpC0JHbfs%FBcBk6*vDEeyXxm3^ z|IbrO+xy)@@qq*B=UDbSEyB{SdCw|WE`osrtxMnHGh?&@regO`l|eLrEp*Lu?Eg*t ztAvNztb|{gfoy>xxea)2_7Ar-UV`Q#7F<_`cIfmn8{Hr_0lsq22b^!cB%jWe`u$|I zw749{xi{lDWA!9Gv=h)2y>tyQz$46U1P-P$0C=fHI)-66to_ywlfq=yx%5G z+^27bz{vt@Bu)Cuv&y{Re-q4mz86ZQZgQYRi&}lbO@tCTnPPRQnJL`Rr+@${*^Zn+ zq@4EkDXa-JdJ)_D@f51qL|iAfS5`iadj)pF^L^gGDH2uS{ADuabFFD^=&(c-f?JT1 z#rf3|)M@S93MU~)_rOlXYe%r}IVgV27UEGcaCrb@^s zyo;CKm^A7e`7Yf#NSVVH_P!@^VOuq~fT$kpdn3tLmMxX^a7DnBn6SSUj+L&)9}Lh= zkfqeuAhlvYjeN#Au`_e83|ty6)Yq^fKk0w|WbO%=;rbuhHBKedp;77CRX9%;obf+I?I<`Pk8?kvG4U0eP^ zlu#JHA~5_xgsOFR7H~cQbCrySjQ5gu4D@M-EFoto)bh5V=`iZ!o_~LW>rj%2inMZP}H@Tc@JKjYAXQY^91&OMlL>n~&6dPKgi}_JBoZCATe_0AG7+!B(In0E*2;Wq9$Z@{VIkOt`EA#f0~;KgP&2Kum}-734Gke~p{jjS@L- zd^>I~K_M7>o_&JAr@j6pX zvH};QKvOy%gV59506DJYx-OpmP=Lw;$uQ2iBL_Dco!xxILM)dMV!7*Z__ET^?}PcS zKb;@ZXziG!DEz3W7v=s<?eT2EyzZhrON6 zt`GARM8*iD+^8wjRSYQ>)Y$cwAgr&xiFC9AQILxO&tHqA^u|E%{ABZ|I(c9vjhog5 zEC$WwsL$i&{{=SQ0A=HiX+g>4RMHBHiKyXg}Mj4HFOFgRlDym0Ul2jMJNPv9?{x29<_r! zfk6H_Z~zZ;&LW`;wgy1stN~4_<@E^8TnJtSPIzgP(@47P(LJ~rXa!kiT>5Kr3s(&K z-NxtjzE^#ShJ+UHIz54NB(*$GTBIU#55`?+JTMBsh2JtUFRgC`JngUSmcx2MLoqw; z8Xg)(&1VjW#4(syD-*{;VhJWLkcoxt>*wODyo0v+(7K0)ac=Mt1_n=thlDBdCNr^p z=qOos6-)NEH+M05mTe4?ZIoO33>@q;v_6psHXVfxw zann5C7fqtY-4fGT8l0`Wdi>4XLm<~Ex@mP7JfHBjy2!EZCCBP>UqwTx&|1H*ywxAJ z))dt)v{svStj}H#jrTaPk#@h%N+!w)X_I2L_<`qLV^@Xc>w`5Yg#QQRY}*d!U~lz8?WJ7loIss;Z0Q*DW7_gT z^20#8I�K+;X+(32M^zb3M^_F=_`hUDK@;T?-^zOW$t zHF=bsZTE4OAuf&lk;Q||-%WUKTaSJtoDE&?<*p5)LS+e2f>jrKJO_qhJ|Wn%7b`O`2iMSA@p(*Cs2W%BwziJkZ) ziTciGi8so`yD4!d$W&zFwUju8C0-*Fmr~-sEU`@{UPOrna_?H+L{1+{z+EiMA3?A~ zBlQyE{VW68O>kc*>*ShsYGj@N9H`dWM0GBZbzV78)|oBqthdy;p6dKW*7*Zv!`oBU zvd%9lk%vW@r?i(aE-@>-bDya26HA53EN{B3FwLxR$^D{2nWchfI7P+&sC}r$n~WJB zMX`3i@xb*tT4rhHN|yV{{UWzpmF7{PWe)ylU{B3`x z@~u$*>pyvyzvlrgAA69>S3&va2fWK4@er2pe2B_Nq5P1CyvuKV7|SOgq4GUpR{oBM zz005dD3;&;D3$Mq@{=F+F8}dkSibggDt{xCfBrG=@;jcu@+;O-`5Gwy+7s^O1-_{9 zyko|{?eFX;?enPMXKZl|&G#qn5b5r&=J21>gT^^qM34_ORXjlaF2$JE7tjXuQy~4D zA&~xU$0~vJSux={k8XrKrbw1-7>&`{-XA1z#;@hUYR}}xbunYzi@n-M+x>m2Kz@Z> zskE(aB>p`yc(|!-*az=FOYl_r%BPvup1iJ{iYhr3SJ6~VE5WHaV&wMd@G2*9 zLqM8}Dx8XBP`yzgL`?+I(WS~8SP0ST#RyTgPsnztm3)0#Fx2uZ30_KTZk&ifwGvSC zMj}wFz8R(&ngUcxOC=^U94y;xm9!^#Jk_uT-dHva&|O2&J-m+{y8kpFbifBPPMhFM zn2uCQHP%$pShHIOJ>r9C3(D`fuma~WR-^>PNl}R5=23?#^PmP@q`55xzhq$*E*2~) z7wlG=o6>!8ZnlkO3-;kLrbxxzAIoAt`qD0eg=g+^MP#>l*(QXrXlulxi4}Wh!xdaz&I>5dxXV>GM?s9Y9R&^^=9}`o}p%Sa$Hf2u} za8siQ+%pHl`v*r0;FA31cX;#KV+!3|M@n~QvnV{0T<-?Y_n|%g+vwF-g%Qst-6D+i zQop&;p8!GI%QVEC$vDJOQ`it+KFXwMDNXGo!f&F3&CE&&3Far0-x%LuTTOC$P zuOI#((IbLr*1sdjL7yXAWb(Jo%KyN*Pgm{YMET~d#P&TFIveAHMB$B34QbP!;)IgO`%L=Z(f()Z%m*8jMF*=}YK-yOkXYV(laeb6_p{w65Ya|La>yV|3!uz&~<}VJ1)sMEe z6v9T$yrc{<>NaIxe}_{sNv_Z$jMTT9SL%v#qcmvE7}g_aMq!59)ILQ+MpP#3eHGy(AnjLW~4RmHQhT*Ekdi*eme zsL84F*$A=P9NV%0pEWJ!8=5OEfn>S4!y8NP~~-cW5I~ zKNE?Rk_Q|+v}5ShNYv7eG%&c18`yOl)mW}!urNL#VrzM27Yzt%{@`su;EezBiTB~P z#VsB|Tm;~W0an5QJHABtXNX)~>q#4*s4(<|ZwU(^Xs@!X=Y4f9UUfCLG8&do29M zeV)ww7K=QiT9IcAEz`6Ph=4BRdv5gmeB+M4NF@41V%wiy`dlRDh{QerSbw8P%oT}? zXB`$6iGGpz#e1(l-y6&9abK&Fm#u2)vlHfiXvtKf1ayP#{%&DL2}z*vs2j|`N*Tq1 zi&0y7DanhwjD0X6x4cx%fLNoEHob&bo2|qd3AJEwRRAGTH;TLd_$;DsmlK4+nv?e z=*Gls!6RV@<=7ng8XnAg&Kx7Jno%%u7%la+NO2a7m9siKmrK^!?98S)zlk@+GS(rK zJHAw=XmKUhl+ik{!S$$`` z_XNC=o1rs)#acA&m&PM zA7GXA1FVvMfK}38u-M$qmT&!Oq+4nD3oiZP?2mE5Bt>TESf$L$9%@F}XI1unwqcBo zoK~e64{OMT%Z-_JZ*Z@zHATSO9#i7hh2p%SwcLm z^|i7sS9KNag8>EC+yhd+-v`SXZ{KQEK~d6}I* zFH0x(_u7`+@(MYJ}1=vX-rF65U1m(bhP%~__l(2|Q`J04EW*^K} zSs%<+Ss%<+SwEPq^l9+Fn|pyj`2Pd=|0&k4xp84+F-W4{*4#%#^bVGC9UNynzJ!zI z|Mm!vkv#F@K0>y6@;hIj;KuLl@Z@)T?EKEkA1wUNko(;EonNf#gWu_kKVE-FZ~SpT z_{S@HYk>5pL4~sl>g$1#KJVryU)0-5d?L@s^MIc|=)m9qOwG&TdB9em>pSOKH7}Rv z0eAi8r-uwx^ZYyynCy%8pL~&;7vOooYhQiR4LX`jl*$ z1CuKx{%Ssfga>neJ<3%?5t(D**U@_i`Smv?{+@X^;ctv9gfy<$d2%Je3e(;NN- zH$C+=Fz0UgGyR^;Ex@C@;m?Gg>-GSv?uI{K9=^{U;Mm>pXWPn;9stJO4S)7sc=QC| z-`((M!PDhCfsJ>=pDV^L8-m>Ye~CYd{^E}=c)p~0+DkpUc^k+Vw*ftsNDwQ@E(D5c z8YH9kcc}Q*b$%;=80nKc!8v`DxE<8M)=UEFsxuFkMUFS91qBq__M>aNH>ztD=Zi zm|oUhh+6e`g=1e}kX{(sFG+-Q$X71WrRJPWW&s(!DgzNPI6C);T zPB$Tw}Mu}p!y@-8x7<>ZyVNZ=B65G>mKznhSGwiOa41)}<;F#BqsSFwU1Axlu8dVz z$GX_ws;-W;!7W#gwcagPj7$kIABf}bMnUU z6R4-)v}1Uwh$KQS2eI1e6LU0gV+z35yioHlfrK2-6e1Z1=#VP3@<+auc$ttH>n$_d zTV|xU%+lU6!@Xq&y)rRoW!x6CGM!=Pe|mpkACnaHnj|FpiQXnD(c2^?dYhy~ZFQ?^aPA>Ga0f5c3Fya8#j1ar}h?a8S3oVYOiLS zpdDMOtL!Fy9CkI))>U8dcN_P61Gk0zb!F|jP@A4D-fH~OwVr(`Q}T()abm)P`i^b- zxJp)RV3(s)hifwSn!vz%BAypa&vhP9LoHa%PaFm(cpqHJJf}9X>VQ7 zwY^38uI-H)>uk?DUq%fzILtm%wSWCC>oR}CckG>f!w7n`U+voduSWOYzI(g`y*)5C zD@<^CY(&X4dxk=&Rze}H47%~@6<+jLxGPcADG`iz6gk4^?L)sVL<8DyUKpks*?}Oz z1q(S}{eJxAvGR^%so?q|EI2njTr>V=7Hn`#S2>)6j_FESt=~N6CvG(7;UY`|CPK_c zwVaJA`-3r$?_z5+p`V3!M0R*t@Ok=!a4(P)k8vEByIXh|we#~Vs;(n;+$bLyKt8D2 zIKLbYQr;4S1Y$~A{XF{J{|CZ=n;VVpEhxtw*B2;#z!MmBn6!}9Ma<6~+3fNqA-8qcz9z9jM?K*oZ0Us|+x@#6{KZa>6{Z&?ezmG+HqSu4K%WCO=H6Qdew z<#Kz7e@5JEraM3*SS_1eORaHmk1N_`@wO971@^o!Si@MX5srP`gnb{qAh7Rvj#c!) zs6@YRRdN2yQ9e^u^8ZsTQZ`OScaLr7-0+=q=gh?mjt%!JxAmB{E?~9NTLAfP-{V3O z2CqT8bN)hTfxtt<#KXiwCmu4KxUX`Ac{Reb=wsHxIx@xBO`9v?u{) z{2aanUC{pcE!!OMW5l<;*ppihak!lrpHR3a(O|ngSx^7gg@so74_f=*?^`#=DJ$1t z+tD%Ovv2z2zs5buYqa_y@VAjiM1=WAmgA?yNvt^mIf-juWh}?)TvQs9oN@vWogd^8 znOK$CRONvavC5^du`1`}3eO~5*mxLB5zeD>7wTAUF_b&hQZ6NBKU)1f-XEDQ!z+oE z^ay2RXl-G^WW3!}weq)V=t!uts0{hVA9rgt9WRq#T?awTc6^z74)^`_*he_D5f_Eu25Kugn&6z8q3xAFX}%dbg3EM}Q0&Wuix zBZ6Q=wz-!exB8WXJM(~CM=&%BIv8Nj^Mmkk7(N~X;~f$?)cT zO`v2gN6BiAk~LW6PO9=YRe9$vR%L@GP;HHgl6O(LMf0)T2T<-7OSzO89Ju}>-rpQ9 zU&2VsAo^)V$||t3rj?6v2@z`?qN!Rw*R?5l*UyHPQ!>9XkG9T2z@%9_c@Q}0o?&ag zPZ%oIVolCX#YxRmf?k8b;VsS=b~zAe5aaXnK!f=10?2OSb)!?#u}A78;-)~~ja7}C ztZG(xK$#E=MasKBh+wKkGMwAF&oaDQz*-0^a#=h>WZ$(ivBp%W)X2nYGm)fuv?iez z&}^7bLs0-XUEU`y~=mL~lD=M(1A-Co>ik$!N;&gOkt)9-J0CNyus5-p>=V zU;LjA{NH%V{~t-`KVtmtGj@Pe(k57>!icIiW(A&53iz-9 zZ?>K5%Bqd8SP|nhbmI@TfTOIjb%&KL=rV>iZ!d4|DQ`GZ)~hjonB2odNwgO|IF

!91I?QLiI$o#&}U|nQheJ}eoYkr_r8|m`;X!mlZWF7T3H7vH|!#=70Oe;6?>k;J}0 z1ZYM)jBz}K$Y?}bcIqy*Sogo;$)di)y7pHabsMX$2{h8tHMFV%9&IOyJ;isr=tUlHF|Zb&}Teq zn2$e0aUyG+gTweM)!8+*{HDppBtTBDG)yX&i=9HB9@lrb_d;#>nPP~ju!PDM5eYRs zLARlB^HPSw_{3pejv|Y(7(#at9&7Z%{E{KWuMDBWxSUXbgU_*T@bqn)%t$flZH{YMQ!d% z@P*CLBb3PgwgQl3&GHTI)WGg11{3?u)iYu%#g_uxT}SGvcV2FddIp)Nq!=7Pm=7h)X-nuG+Zmk+K@J;!V#zQ<#bL5Mq#`y>9N-EZkO@La>a@87 ziFW;=5(->PUR+{`1kxn(3NfJ4Ko9=-AwBmlil^`pP*5JO0Q@J$fRWFKwERgUcDxh0 zOMYu6Yu3=(%7QT5QC4H<8Mvmj;4hbQxYBAOZ5S$#2*iI1vsX&$jkA*=smS={PJp zHm!i6b2LmwnhODU%Z$7aEHsy9hm(3;SP08afx0#2aR>BV@SpVIZGkuJnhQ?hJs)kb~ttgbM~nu&5*-pNUAHST0>WZEQhBWGg6 zZl2aDxH2i0Nq@#?5{pm52{PxqS{jl!KgS3Lan$AOpp+0*3k^zlc9ue~s3E?4(wAZ% zfu|lCtcmw`%rm5hq#mJ`%X}swVBFo1|7v}r+r8H(X0A_6tPfcgh3hWjjF^3WN;9xN zq{^ZtiaD2vU&NJFiaA$l4`XpN&HY~`K9jgmd>-HBA&+b1&Z9%Kmjzp+QJj$)bG_Cm z>t)j2PqqJMy4StyMNByO@(ni!O@+Q8ORKL>RhYgBrnt^dbY_TRMDi&NG?1GadqHlH zaG#&`q#nJvJ9xg2quGww+cf+6Pbt>(pQl;Qf1YMJ{|OuVfBtuhR6PGb&2t4#J^vxa za{k|@5Z>FYgja?KazFne3(x<5nCCyH+0Xw1^Zb93<3cCl;hz;er02Aw5-sVBrW`4l zCQopMHvnuU-k`&*1U~@-{y*m4J}}Or+8^IdHtn`e$pS3|L0D*ng@UZK=mO>P6cT-k zDR;$s*{g`F1zeWvB-1vSW9RD^&9wzb?YFG@<$Z&c9bMSWsW zqM}5F<~wuFnYZU{H^uw?{rH1spXZq~b3SM0%$YN1&hS|(r3z_AH9it%LP>SjQY6R# zIMp-&LU2O^VA$K=Cf_LyfOk|e0EA$f0T3|_fMIRS4%3(&2^_Nxzi^G&;fQGfj7klN z0yvhVeXK9qaHAjP38Yo2K>64zC|8^QL(rV8(4^%D=-=yc`iK4m)BlCW^dCT4%xc02 zA4mOY+6r?6`o^CH09|D+`kW-X4K8GDAO9p2}N*R6-B_c%W*#1MLYt1u5Z+#lg)ql zhotieyT8zwBGEvi`P)_|lQ~z@Gci{e)cTz(MLAd7=3H$XbM>fr3Ul=b8hjhVeJ5hB zhOIw+t{&7z(x9m&#cSOmOZzcmI>uXH<#If&EyqW(Q;Fp`$|WL{%3#Epa(VZ{DR;#G zJ7JN@hhm*P={>!kzpy|v8JA4n`n#t!v5GKrp<2oO>qlnF3Jy!B*^z2q)IjwBq?>Rxfo9C3C zbg7L&r8fMKSJSIfMFrUb8pK>;^a1(Q=mv7$;lCyi2UXDU!LJ>l)~#;$WYeaXB1&Du z)*J3P=Y5f&VAvoajEj!}i@SFRVL@|aPOHX^az~58j!LlAD^g4WoDfw|bg*AgZ1eSMhCDKRN1qojG^3I~=omG=EQTyJL z7t-|LGr$AyKK);xAM&q%ez@TOv-x46njge+jrx{YuAyxCYGI|14(DQAS<~F1%?16C zRhWVeu#m7HJkv-`6U3?igbCv3|BDku`!WAdCy3V`$6&vZ2_jI54@({7uS2WNCK)lC zWMD(`ipPUB%s*#>u)gd!L0A~zqE8T(K0y@1l(AI{5VQjpghPR|Pm>8k@!`jUClf1J z@K$hlu>$59HHiOU%DkotA_K}6+x)#pU)Kb2g?oZvTI6Ig`cJMh`nhU3*+;dAN1$F} z+Rb{C7uPyH_=E}>8@pMY1Zbx{W$Y-M@6t3jKRHlk}k`h|M?5D^@iM?z99 z!A5Dyx#$s!n8L8xf1{siciZ5wa% zHj%DAgzpgHjf(s6iLCC5`EVxsDt_J&x)g+AHB)pJ{78xmhT^>Q&?QZA==b&#Gs%bG z60%_@`>iE*&rJM^h4!vk3&gn}Bc8KfjS={~K+`>CtZYvOhIXvQ?uX;E&2A0@;+cv( z{%dcui2o`m&jX!b8>b&>xp~s3dxeF%9U+Yc-im>B=*#Tp6YH|x7{AbQ(|?nmE9nHs zbb`ZP{q09^$A58xcn3vEFF&T29}Fyi(r04g(nm{nON+Xtg<%XU9(Vq68bpQ_!_TvE z{GNUFe-ppwKjn|#Tv$E;Q9!Q0V*pq_&x8M9WbXK%SsQ*oIJFl1Y#IETqLT%2oS1kZ z_UTaf(zoOw2i=+PI4I(rzGrdhae{xrBEQ5U$GndfSqMeY&3_+Fd^~O+W^WmozZj29 zpvxcfsw5LvuChfeP}YcSNorY0c!bz?`7kHr!|swX^l+{-Hw(e^l%pr*jn7vR`DV2K zWII3JG_Ey#r_u2Ju#MzH6)iu{-Ez3k@kiolql*XU&L7cr3+lMM&JOpP{(yZl^|^l* zQ)zkFplQ>q)dgWAX-HlAdW3SbKlx)M$(%sTL$5bJwKJT1oDJ)yPMiY{GjPHQXRLJ; zEa~L8!z^Xj5dVbU;f4!ln$@Ky%(moxn@&Q2naPL7)i^ZvTn0nq!b{oEcI#I5KS&cG0vce2`ERZ)GXvDa6r%Oj8P z0MZQ~W2AFWh(VrCUXh?}8V4f(tg8}=a8u9Wb=uma4~{YLVgBm~n%KNfHehx~bnllq zb$|+!mIGD#U#sjO8F!x}a_(qR*BjRNzNKT8QJV$0sW2RIEdxY;L4hef<9|;kxk_k= z^3BVhnP}6dznlC{&4pio{qcTV>?$A+LWOK5gV&ZLBtt6X(nnB**oF$xG2mLHi+wdU z@XteH-mE@;q*u>*x9k%{G2U4+sH)o zsh{7@`T*n{y$d@LfBASmv?nd!C5}0|HVSF=v538&Jp1*6w}-a*KtDEZT3a(%z8&W} z*%p||g8R|2|J;6bEbB)bW)&xitKaV5z!Q8CrHTgi2C(Vy!Ku}+3)@@hEF;`)wIv;V zw^fM!m?Km0XtEv;*UtlXb4lP#23w_kCOZSCn}JJVy194*n{Fn{VkYBX>E}rO4(%ny z9`imP+VhDG|ExHi_fiC%RzI?_KskqVEl!Fe0er}3Pc;JX9PDF_E#tPXZ`Qo3s4J31-K z5Ll#%1^%W^S|nUc(pd;nKxbihBq#*ygQQZraVD9GXiG++FO+^9sosIz^T)y4$=VPy z?4WsKZ^<+^R6jLIP$!)&scaoZ*SqISvIEPc5D9tv34lZ$mrxRik7PIg+g=3&6U~>a z!yv2bqj*FY+_bKRli`we-kUvQKHe?n4FPQ!8 z(_;AHywt)yzAgHM0Aq}7-XiKj|6b*8Xbbem8hhFYMBeUAkLd8>B7h(9rr5QrxO)zK zF{a+cnR-(YQ)%t^eQ(sw#hYkU$@69Y?H6{42K;DL0nG-_Z}(~NTzQpL0&3Je`v8s_ z%YRVq;ICKU)^1W`jvG}!eL&3=<~Xt-A1ix3dG+fVD+_0hv2xSg#!4<=tmL#)TOG8C zaWt3e(SA~ueA9%W%kyY+w0zPOns!q3pJk80QgvYI+TX*RaP%K|{)>`skMv|9N!M@i zyS6{TcfaZ9`7iK0;eA7k&Z3S;QzgVAzt>itIcd^a+9@n)Mlx2?Vhd><#_D#{b8-Bg zj0Wc)*Tf=hB^P{NFu-3baX4Phm`AF3^n;W)Vv+P@aG;(-05&}b0IYq10g!%305zh3 z`U!rw?FIPmnNP9r;tvU+g2Hg6){^aQk6F57H%-osKp!pTBVgv( zYCMg-{u433_V=px=WA;J`PlyR_f%dP=f>4C+$b%G^gq14jjOMJwi`dckceUjS@X&? zqly$R+%v;_-Xbi&A)sxnGj3pGZ}NkfMy}}YRGw4om;Sn0JGCx607V4h@H=^uA^z8< z%_6}9os4CF@X$~^ik^z9qv(8(%I7G0+V3d(n;#;BS7nVTJ8SL~i&wUu*|y9)SnhKb zIQXs_ZhoOzUIk9xtPZ#cpwh(CTdDkfzbinGKJ5b5qu;tsIPUnjk)}_*6w!*pf{x#S z9bvyPkq7)qdtqB!xLt|x{N;9S6%CK&7IvXdTh{|!xrNPti6yD9^+x&o`?)69d55jNRYak_R6Cd<#b;={*0*Sj3ehqTu}aHt8RK+T`|QsGqW#tq$mr2U$K?Ct&Iiw;%HbW) z0@8@YZT4cS5x>J69o6sO%TPkfhRLnOhh=k3eH#2JYj2_{D(*agCadI)_;bD02)p=r zj2<K#j9xM za}ecaLVpQ9J%-F9-g%DZl=%)Z{`~Y_ZnjP}kmF3R$1jUAzlA~Mcct*c^;VoVuhUN7 zCZ0JH>O5P$W%$7HGSEy3Da6BWB_V>-x_?cacA~}a1nowr1M?l5x2`jh9f5z6P$2uh ztZwbCvctwBqQma85`mor60-ibFUmEW^`Gn?rVDbL8Y0dQ`Lm$>&s-iLxZi(LJJJM@ zoCs!H`O=crM@y91N9^^FkZ=yL`x_s14(dim@ zT0g(GgSf=Mf%`B#?~hXNOFPXWBgmWOH52LDAIo*mQo@d@b2Mrn`K0U~Z`0v7LQ~TG z26kBb_AQAEfkh(`&xM15`L5UdCv-MG$HWAOVSzJ`gm&#G+5`=6tkyxyc!-{2BGMU< zDclq40RQ=!=JKLB(}&Zw#QB%M%bih5*+_{H;EsXwqWEbc(sdBb`dTONm^>wle-+rG zg!&8Qid0jd>kY06wf(-)Zb0Y|N}sk6aXXu*KE>O&_+*~qDKYtm*$2=ZSbjQ~14}Gs z4jgm$V!D~F0|QOs)J)kk3vx|RAop2;+-F5p9os8$@}Vu@dC~hQOiH>jxfJ^FCNIi% z$%?l`x>nLeUzXf<*!>Zi=7IOKvIL&bgxu1cyhJ$+1J)1jC;^@sIrl1Cosz#AG^%K2 zQdXX%mLxI7X6FaoI}f@8HUf;%d_N9;fM^lDt{WoimWH4K1Ch^7 zQ?J0>1W9nRTCd4U@$G6YbC({NjW+2~cprBj{Q4|qunJ1}<;TCEE`*isx8i(yAJQ;@ zfa{}Wl}QY{`Umy1M&$PIp zx4mw5>OSeir>|+A<{e<h204FTYBb;_PLUg^Bar(LQxTOj0av_toJy7)>bZ27BD%7WSDLAA9^dw8xL$ zdXUQ=Z+QXi@pq7AQ+9ebMTPmz*Cu7>xlc!hd6U*mot4_(@;cMBpK37hR`z(xsO5H1 z3;*%2&<3#Y-&V!Ghf!wr6jw&N7SW$Ix3hjR?|rZsH(yY9P%%w|CY{(UosjzoksICkyT219=1KgR+z0>}e8u3u!pMX$^D6 z?bwjau)85wJqOY{62U3_m9$d7m6rD!r~T#N#Y+RJEJ1pY22$BMkAWjbC;KoBqVjJU?L0xbo9-U4*&=|Mgy=)pe{a*X6S^6|-#%!jPPtjQC%!N^b2- z#5~En+Q#?sJ64j04O1`g05@Mj*0Uws<-#&|)k2vIKQ7!fyi5u$naxz;d3uFiYs4=$ zoQ7y2nsp*q$u4k$_5qH!xThgG-Vpb^-3_4ae2;g)ji)46LpGt^mbBX<-s<81g zB8Sh3?oRvB*C2A8Gq_Q7TrM%};~&w+)5H7wk0+k4+C57A2gnNb<^G=W6qZl5>Ueo1 z>mD!jJ8K&+88SO*yp*ECUvkxfFkVj5+W+o9V){XZvPmvTEobtNxS0f3I+L1ZLRhJD zsS!H+Rh7q3zy-xu%j#dbf{AP<+V$Z8bKsOE@hjiyXP!hgi_ka>0j`0**#3H@$n;~{ zz?5BDuWI`hZ}xf4$K>)I{2HXBM*q>N)~VBL|5{T6eA5EjQ{=XlL7`{W_DI)JG#ABZ z%P?XOP`_ysi~IJZp!>YI*VT6Sq%$6I_oN3t7TA+~`x8n(F1ys%)Y@hV%J#NDka0S> zIU*c|>#aDor-TpsDf*KwK@Qz0Jl)vy)=I{esVMiF`;D79FJX z81~r~_^hoT%Zbl2+GkxU-q(MEc2>tvxSjP;&Cc5IXWY)3TsdEB$w!(%7eK+EW5I)e z$_rki75vEr6#U0LEXWTG;C%+-$5Vg7q#k1t?Td&0B5zQlCwKfs#GSJ0a%e+sLtV~0 z5PkJ&`YOj)Q=9NsJ=iveo+i%;G)$g9Ph;e3Q~a_ISS;`jd-!8H?p9@6QCn{%u(-x$x8 zPpM9J@Ql-WIT7n?AQJ2o?yv!4A`e<5uNCEY?QT4nE0R1A;2mIBY&)1AKh~?@FWErA z^>)S_R~hHE%c$A|M73gI=@hl1L{8Kome9~zIu6Kp9B?O|(6@GNYNx6x z*V03#*`E`#I&OYjjmx*QhfmXc?D3EI3z7mQ6%yyc-q9ngOuny-i- z5nt|7*uGPEpwPg7z-6l#odK4NJ|P!jHfE)!wKucIn;F8HOUOP2zvMz{Z7IN2KW~m| z@5NrY2tMN<+XED~Fu?cjzPm9RQHQVS7EZpaN7?(}_bboxgnc9Q$tbm;n_9r1v)SJD zr0Yo0{=KUGr(o%;8zx2QlybPrQxJL>rV5`?9P{5-)=k;6*+liq1JzOevVJ1FOk|f@C{) zA#SDYxtMd(`NZ`RzBSf)wuuS047{zjU$ITsV6g+S5B5jsZt|kt7nL0mq}dmrJo`1= zrp-8SyaZq}x=rb#~g{7c-+>T@rvH>#9813X-)tC5^(lLwON+yTRv`wyekeT&YT ziy{$yvRcW(TuH9$B0mz!M=NJ^CFOeOn+-R^Es&~)?@%}G0GpBP|A8J5+itq66>pNUw0lBaG$E75lI=V#7jNeI zAI6?2_{rFN9O>`RcvJTKgc^xm9`-Le=P06zuR_3V;W2p3Y#Ba_n=NcclV;1|<6ySj zS%>1~rJVo3iwh5y2O1;2;q zYvd!_dHW*3_^s`Ns9GL=;$`^yP_nw!jYL^z^d(rX1G7IxW}g9O>osQI`D4bch(Qp} zoF2GNVyWiS;bCT*OV`v*E$7`v#9rjIlYD)2g;E{_F!?H+Vo=SooEv~d=Q7@$-(hne zgyvkXHD}w8_Riv+710Rgcfq7FMz=YF0H7YX>sPEU#;LsPLUxD~reLj|_6wZ~Sl?UJ zP1q9bX!^YX?^G*JEv0_70QK32;o9pD(%Op&ikUK}*a^&*fUA1Cj_q_T@K#N5UynqT z(-gjiIM!DtxLa_FTQ*{2QFjWuZ&l^HK=Ehr>Ns`YG8efjAiV%rUP6NG-&J*B56Hc+ z-|uxDvGlutnJn70y?#xc=BNzJQM<`AA9qgvh;OulhaPJ}bA&om8tva0=-C)pSTSJ> zGYcxVaT!;9F=c@dl90*Wze->`7z!bWSLMb?*m?RV&T{vzYm_PBy_87e1q5Z2Af#zQ{(s*_E`RsLDe$*sJ;G4n5jhHXHR%f zSjO)x?ecNOg9lqfs?}r96h!3o#*`LO*m?HBF5BfpyTEq&m9QMaSj2!YvNyii^V}Kr zJGRfv1EY}}ajfUEuT~DU#-nmQ+Tk=`5D~l9H$-8rt4jA&&REXlSTbW^fwDi?k5$r0 z7hd@e`eBJ`>Uouf6?qA8K%OJM2r@$9g84M>`wZcg^}G8Y72#cASgh%fmB7=wYw{bU zXhWzqZ2`ken4q?>X?ccLNS20Bu)kKJSAWm%Xs4ZV65C-ff_vBaF3Ujs{!tlchQ8Yj z*{rNo#jm~mv&lK@NJ)b=iAX% z;Y%bH^;a4|=ie9No~T8N4@IeE8Fcnj?w z3v-JECw^l%C+r`+*LSZd(4L`S(p9QCS^3s4khDH60}(<3Njb?||2aB(qSiy9E1=wj+7@zsrvJIbq_X&C&urO(b20a;%@VpbNA` zZc_>u=pG{7@3)uPhhsT~XD*uVd2}_af6sHI7}BH|Weoi07FT6hFPdCnBGwzOtJFlUQq)ZTIXwVXu1Tyed18-gwWAycvgM*I<3A z6+6_`ib3ow<#@VI+Uc%TlsWD38(=^Z*(Y7G-U>7xRcfRy-Hc`AXE}bH2{$m;cAv7oMODV%M z?IrU4MB`&8J8_so`XF3zFAmhw)jMCy4Gtl!T@(%q_^ph{2AQk z1}H&b!aH#c|L3?FrzOig5J5*k4%LkCk5FjcvM1==`sC-ek?*~n9d-$?`6x~V+bc

{coT#}q2A%-VLC^wV@*%>QmdA&$lloK` zQz`AFL5?N}hz0#_k^79GRkA_IKS{FlRQeXzQ@AR@K{fk`NR>D~!$(!WiW=eZ9J^Q1-!B&rZloejHXZz~rZ8M9|hh!9x+NmXJHv7gm8@V)xv+ z61%hrBNDEJ6!|~lPz&dE{hGM!g9D8eof;`p9Gf2GOH2o83mGGHj5kJz=>_+kG_^fk zWH~jEP$P&55SD^zpJ@d}nGyPt7h%SNJcE(d^kmk#_Fde6`XhdCY)MAVWSv(gyos-* zJcIMc6{PPmF9UD0D;{x%Dl&}cY$Z>c9WcBd)`0uj0}UH~LE2}kk^Q=Sf_}+9gDsq0 z7ZlPN98X~e2`rhAZ90UvI4ixPY5d?o|ML|={5uE!MC_HLsrwex`~s1JyVw8ul&DDG z=W}sxA6+(`ao+b6(Df_lCJ!&aKd-{?vb}TTciB^Ow z%UzvUpAxea97O1)9q4!4BSHK~!sPCWJtO6pl%DXUlq87SVq@+2XUx!VpzmEosY5vJ zo=fGE`R_+z+co?O};I7yDvLSzH*@vJH$kRm7FB)aPu%;HuohnP;bHQz^ zR&-OW!v$~NIrmgBiEw@6?ED|;Soh|K@$zRoJ-fsEXREt^);Jy?+r3ZYQ5%Xsd&wM% zD$I+9;?Um%g3BNGjn{OB#*IGS30>0@;P~B)WYFJF--12-cjMBytT8_?-T$V?0;EN} zoXiCGxhe2hzTtl3=`)jf-}|lX$NhEsn;*iP&P&M(UFaOO)3i$)CQn$1n_HN+)%hbc z8RS|t?VMQ#n3H<0ue3PS)wL52bsfEswgM&(TZk)w_o7GP*NgZ@(G{9U;d9>ucizhj zu#HgTOVCDs9FQLD#YSHu8-3Xi-Hk5l1b#eeH2J@Fg*17`5249dE)Y$=geXCv$)+bT zmu4(fpqkbGydCOUTaumydbbf@kM$M)#6iq*8zS;B|+X=3zg445{ z+Btu|e|rk3_dfAAs_s|wLMUgo{o;?$0B#3r+Zu1*Lqxt5Mx-EN$T=&K$O(ZTTT(aY zocS|E{x<9jTKl3AXv+7h6 zdvMvfdY}u7%wF8yhFa1Mg8C)D#vOUH?PrO$&u`E#Fs11=>)gBqrq>RBymoeD!(=4} zAaqc$LIZA?zqZ7!2m9~bJ3IM3`Z(IPv|nC>z#%-hY0G!_!VM!GvzllzlXHAa?Q(ZKsm| z#vkUZGj3_a>GQ`B~TAfADb;x9O9JNC6rIZzT>sW`Eg5i-Y)9oyYo}qYG`NH8G%0A`QDmz@hARN zrE_Gp&T)hd|h2~qq^gIHo49T)Xs_dt+>;;~(Pme&^=j+rtg)S93_uw|+ z(cHm4xE?=Xn&P@^f$9xNH8a7fUaV356jB|igWYHd<}00{?+v8oQ3!K`V5KcaEdL{0 zk3v~_#>PYY;l4E$WmuumR10qtv1Tb%frWe8xMMsnIeGbCq_^)C89Xa;)`-a7xbw!5 zs1}J%@xS^hpk+IFZy$bt$UGa6~LS-%?lxX*SS!}igpdmg1dbg@;IQ+w#X&6zqU zfMT9gBzhS|FJFb8=(AlbwGa~>oG1rJI~b;y-e1U3dSfY5C0yeo_l{j*cmCru)ymi1 zt*{{_*X3_{)o0o5^{Qn=^gC8zc%^zXu=snY9T1@_(lBTNBHY$t z_=d&%?*0w(T+@;8P;*OL!qKZC;k#cL=#>cccbfe4Qy^PHUa7->@!NL%_Y=L))6$UK zze_o!d^eZx4C4OTY}PmOO7lN>h!%uVuuEKl!dI51FN2BO%450fHXg?{+KV_%{05fy zAJ#P5-8V~(HgyUV=sHQJNCag#jYa>_=q~zkEP5_0>MVZ9^oli19$JT1N5*;fArTe- z;gVQai-S;q;W|qy=tIOD*WqmYGA$SP@cI05*`yV^JIlOtMioABm76nsX)=nMF@iMKgQBd1lR)Tl-6B%MfON z7M|h7lWS*ZrAb2G$tBn^{_vpaA{Sx?Slb<8@9Xar{x|X}rC@KP^Pv0d^`|F2#h+dW zeK}nZefgmr@5@(feff@I=*wL(5aTF_@lZ)(_Is>RWv4I{jnJ_cG2S>_#YjZ6oc_E` zb4XqFfZ;akJWh83Wu2?Pi@T!ArI2<0yn{sj<9amnBiK(f_&WA0=@TLoe(E{p|9W6{ zR^_D?L*`Hlg;<9-MDmBuO~0U83iVpscWrZf%f7$O?Jc``n@m+jPj1*I@8qB-*U^&> z%0io@t>tFg-Am{PB0G5hdA1IH2jUnez+T=ThftADr8G?5kL#dSQPKTZ0S2qcie@8V zRnvc6N2w;Wpr|4P&jzqLYQ{MT8aAsud44>KW!Dn{up~8g5NUmLU1=ZwH zl&i8xo3XAX!U$E?tyh+ICYq_T=V^tOcD@gtwd+8+W=cCd#;NI>aov}b%;$jl9G5N+ zWY~D6Tt(QO?CWxRP(!}owtj+7swVK2rcZao4R81<6xS~ zDsR1OE>$iURsJR-NROd6?GTOTd+rhoMiC)kqZ(U9mAzF{ZDS8X)0X+kw}bqn^XZJ$ zm6~u9z~!|GQ@H%T5%IfD6ZTYwkEd1ApI_o#b@2>oxkD!}eofu^fjNXkN%Z^HCA1Ou z7rYw5@+`9g^c^0w!-H=~+&PIOfhz)j2-W%Qk3rv2*uVjF(imRNM}Uu(I}Y&lh9^;D zahiQmEcS~A*8S`CuP%KOz-?Bw^AGXsI~qW3PJ_DnARotY92QPt9S9S7XR7I5e)3i| z^H85!FKXbfVZP##=<;)w&Ru_%dva%DRTZV7f!h1;1P%Q^p9KwF0nLn4e1vo%9I)*= z(OG|w@-!;TgOmPk46%hbbl#Tx<%OQl3O&zXIE(JZ_QQ|Q+-n#%orgO&-a)|ygau5X z_bw(QV_i%`_G|3ZK6R2X=VX5lW=39;Kk;5{^Dwn}7=VU;&fn`>dUT!e;ssRG@9MQS z|CV?k(HVL29!;UclhQrfHVQnslM0Gm3`g%7*EdG!CINZ#*ZHLA*HQZMc6k#R-rC8g z!c*GCQD&iTiq8M3gT8XOyo_TaqPdXt%>X4Xf+>b z@VZg_0zs~)exfa=-*&E=yD4mqZ_=%&Y`{6%aT!pF;$fZ|{u?Z7FiVV_0UTWePDJ#( zla&pScK$3T0mvo1ev>54TjV;TD^F?iv&b;`Pri)?KPMG`96Hk0i00KyJAZnN4UH}E zl;^<4Y;l}Sh#o0UV6$5HuZLO>Uk0@v+`?+TKQFS*D>W>tK^t*+ zfi}E9_dV>>aK3WhcVW6|XVVRzNi+kyRqk>#w23v1FK2s|<7EN_lu4x<>d&eHdRhB-=}N~R5z#XpkglYF8ed}1B~ z?_ZEB12B2JTAdHwBDR+?lQS$tj>JXvRr|T~akp%Rg~!9l^;of8gq&n>(RIYns3UeNRe&bdPacIg%cXG(uy)|C zrhql?ytz4wX4Q(RW2=1n!zfu`=Hb(>15xNt-HdC{d?X_L2itE(4`^PGfeFN)S4YXg zfrAwzz`+}^Sv{p@dA#V7hXF2-$6A!f#RqYDT&l@q>t{h8e~zeiH}CQp=O2{p6KJxT z9)~u(ntl<3J1~)uE8d5|A0Y7a&e!3Lf9LzO29olZiT`POr!~s;5h!q+Kk2+i`W=`) z08Rq`J979kaHtNnqsQoUi*w9aZiBMPWb*5s(*P0E1Eh>yJmEC8xg17vTl1 zC8-+#>xao?frG`J-xO;C^Bzg(wjV0#U5ga$*A8+?@27)6@ydGr6h4_qXWxN}5q|vW z4s8t!PkwNR76SrLZoNY+=Eal$zQY{a4U;ok|9eBRf^>Svf1n6|ckk2?dr_y_(6I^5LaM_*+)0bGRGsns=*Pt_8}}F92qX8}H@1q&o3_fV$LOoy`LxG&R{OSx>3g5{ z(C*#^qCG=V>siIQl4k_1>tqhRiv=;A**MmE7thE~oFH?~jXW>sDf} z{yX$?F|YDecaX^4qU;~YjLK;Z*)>Wp`HirX7TrH+I}h!I=PT9Vp-DYCgFXAffAF5& zq4n%n2lmcR{%3?vFK&FJOiLhokL4V47ism2VJ3m;;?3qKpMt&jjm?sQyz~55vCq{_ zojCQTcIxRKk>s7JC$)&rmBOxpVUfH|h6>7E30LUeO-DqG>}kzra)zYy^bvZp=pS>= z{Vy``qi;K>6MYdHs2nOO=a1*nWOX5&E4b83T79h|jnj&9;@AXS1xZRfS48u*id;Lys`( zRZF~g&iT|g#T3x^Ei)E&i59z^xKRx0W~r_yZu=WKkx`fv`S7;_qP4^A?Hs67SQrEQQH`GycyfdC+$D1}kMNjhb$qxQR?u0`= zjU(hXcv;xTtV_dj?Y?$Q&I%)(Epcr)^Ph)(M%P)QxA*^*E48=tyZ-p4U)-o;%ddaM zd&gT!OC!!{@8-)d&fpJ_!J|IP8En-U%=f|}+%x^o(8w*@sbz9)<$rMCxLnjPQF;}5 z$m4ym2DtbZ)Fe5s3$hrK4%gUc@h*>o^VBWk+{r@ezDVP%`)<(|5NYz(!$M*vqInoH z&gd6~newZM@TorjCGcUT@YiP^q)IvP|?gcO`J zS}P+fX%X}D+i602e?R5z&pST_6TkBa32;X9F?{?sDscSNS5YYeNC(MJvd&M3#K?gR z%25gI>9@#Waeo1Y>4TgaB<*$3hYml}JxC7w6bzCP>N>Vs?B!4+J$4@$FU!s27^G9q zc}#}dU!nz%z#y-CfGkiIXh+F}k~Gtq@$)Ci&KmO3WoJcAB^DGf4}-qp%|&_X7$)c4 z(!vLYbMZE{OvocNXXV_>gy?2~=D!=z)aC^hpci7Pft(xA$2LK!4^pWSDupYHF=_;V z>U?ZWtt<*i?!laUWpNFX!}OC|T6iWeXX3x1&#z;{WL>1|4C(9v)~4Jry1!xv^6M`W0>@ujj79 zzD}_4ikuo z?RDsLFPEUeySTqQ(sdH=+E`VqS(U9VL(-y>Q|g5GrD5_HQIV?@W`i&7Qpu|vOgpoXf6A)5xzz zH~iycG-Yh=K8W?kHr$QdwHd-f;mp)@m`>Y{tp)N(V1fy|7#r_gRTd+;G%I+i6}S4@&hKwKzppLcvoWsE zF2LQu_tfl?n{1^6k@>7dZsoMv0rG6@JrFF1IY;>9{_05=L_kkgr z3f=&8@F`R6_QBXZ{x7%S+hU6Mc%o_Z3lY_PeDb8;;M)ghXHSo?*`;ao{haB>@B9^f zrgyL(_a`D+lMh%zTR~rJ)KjAk43r``RzMkIb6TTmx&MYw$jd;&Z10bYHi7w%4W?wY zN%%AF>`r0AUjBQR6;BW&_!GTvZCQYJK5NnS8{@Y!v0dA^Yco%7K93LS53`3SMe!l` z=QQw#Z)bA*cm9N@`{D`Trc&(5LHr5t=JlZ8@Ndxf@K+yv-4Flo16uv?|30X!AO7Fh z*4(qPx#60FcC|G(z+A*+v3_la=V@#JsVla zKl;+Pm<7_kj%l2~;U3l-?gx$YXV%}HzqaB1vHauJ{cV|^lEw93*;)F;Ue@*6itH5x z{CoorD)QfzGcZ*ET|u?BDE3BKrK~NK!D?;7IrD8y31PcA&9w>4Ro#{ZT>950>34ot z8|w-x%@QmjN@+lKzO4Sj9L?`rG77ae_V>0ND|b?~FJuWcpQjW|@cV*eUgn{63uTPW6j_lqG>Zf0@t%K#w@4T=Bd%j*Tsx~8yM#~1luBPN^Rr9 zKBXqS*c4;}=7t1A`rc>%%FM2|!fNQy>P_CciM`<>gt6Zyu4h|ov2WO#`aO9l zUibtQkFACTvpV4mSaH_Wm}n{+3T+IzNOI$u0aTevc~26`9)MKYuv;60-5O4IH@uq9 z$WDzi!u2$)B0Z%#q#SP=7=dmTA*0T}nuKip@;d{o8Ln{Vq0u?s{2ct%voTjo^WXg+ zKsV<@*RJs9Pqy(rU8C`drhyyg)cDKq2yFcGzY3I`YUG_CCk{aedH81}CeQb@GP&`c zwYTu`+XGv;nvtLSW`7$`>eCkA)z*m0A^yetxVC-eZ~Z&?;7DLyyock5tOb#44()$W zoOg3R<2d&BnPytsI1pnaqjBm9R$Ze{b#<&dHZjaz)6D&rFPKmGe+T%ts9| zUSb;JW((}qwpdSPKA%WWAbf`NbZ8DRBqJm1>LKDZ1&IAkY$@b~5HgN8FuI`JMS*1z`rp2_%W+pp;qwe(x@ zf!M>apT}k_Pz+7?#G(v%EWt#|=k2z}Ju~pEv8Qr`c-g?DS1XU-x(~V*>l2^DF3f26 zZQcr&1poi`nR?j(oL6+xT2bA6~rkud{t^S(vPu^_1m@+6MSLvK5csj7zKev_6=;h%cj0 z;>Wexnq`zpTcZCxF_^l4dU8~}QF98u%scn7muwzov*&ln1ca$ytTFk_)jq(cmjR|i zII*8OGy(Gy6OwA3K{e^DXH> zY;3MsS$%obHU33aje1oL;#B{u5ut8&MyjhDMve1(b{M}&s9@7?iC=!ML2utb?$X=$ z72o#dor_k|Uf&18^Qsu9!S&mPc>8j4i#Rz5h4#?R;7buu%hn4lOV6pe1J4 zvb7l4uwb2{osn?9`7pn@a&6JsxN|S->{(WduV(o=dC}Kloy=Fj@2@?-N`c2~SkC3$ z;3$%3I3d_;%{;uV>@8~1)JJz3VP=~Jn1c(#Mlxr=vNB3ZxQZAS2b5p#0Y)K&#EHR* z^@g494Udn0bnooc>H4Xclq)U&7J0ItuWERRiiIAzrzXK?n#S6GdYz0r@!YQv%Kvp5 z|1wBQgrlP0a+t6IF%t8cy=VeB2wXP#GwkqxR4X+J>W4D|8p7c|#RTDWd9&c`_hP~!)#^}x>eD)9c= zPj25kt8LZ8rozdqA?-4r2uwOp96Kv6J}x`YZ4^If_j&hxhdl;=`}8Sa9^ zJ{;_-bhIP1%tYJ1;t}WJ&%)O&_%&_(=lOp5qJ~I#9;FQKl=SpM3o@?IDpx!}XFz`V zK1%g&Z+GCn?gYlYCivfUEOEPY#)?POJty?Ehi8s#Y{6MwQt9Uh8dt!l%Vehh6)ImR zRh5`yRf&DCDn7@m;`?4zY>rjM_PwgQIaXD-uT|}K=2%nIsHq=P+WQ?rHS^{3+{Uq% zo>eW8Ym=;uNYa>;*ZD1kr|g@lpZEKw$=5G~nuv;XBSOI;`XZ!dt;br2LC4AJK|+VX;9fmcyg>WoPZSz1~Z|FGDJC`PNcb(C>*r1+TipbI~f-X%@Vb zaG&ZjO3TTel&iMw6+d`C=!=U0gIu58=#!7s=W1uB?9SksTJ0;Lb=P3)p1;)%%?%%h z)+J>jE&ZrC*?rkwSFEdYKe5EmXWUXU4#qY zSF&Yz#Tu=SGce{D*!(YK^Tj*dY<~Mf>?&W*zT$Cn9T@CwP#^h`Q@u}{~xdal^trGNY>2y zQ29uTGvhUhwCb+wf7SgS?)zk9SPR69$C2MS@r$%Tj{GK=E07UCUv;L8w8%TrSpXAy z&sj+Hf-kvy&juhm)g})#q@_NPmn&qQZze@#pBmtK8sJ&SIfC|VYCUgo{flrh9^s^J zB;_e3S|;C5K3@p5>Z!a6XPI`#jQeG_v|vhSy!ntDv&PGWd zyvI-&{yfh{3rf6Rn|atX=oCJ-5=M2woF`0UF+JHd6Xv`GNo+`QrKGr0P+VzJ@nLZt zBh$Ae%HzJ8Hb2c&l8j|^V>F~;6m?^i&@s{%9E8!Z!bq$P@&coTVksdoN&rV0$x%jd zlo1$>_+pgz5+wD56jw=#s|3YW{uo7dj1vAB)wwYm)iA1aW7Mo;B+h$}DvZVyMkT_i zATVlHEHw*^nt>xra%2gPEP+wU7o&ogAYEF56k8?5Rzb1VAEP=Qqh^1M>fIQXHH_-r z7$tR##C?=yh0(aeXo4{65EvyDOG$xI5;(FYN4DU|78p(VV$|U!$RA3I?UG`gt8K$tUxF$FrW6tyyzv!A4-brB*k@t;<_62TPhIq zxDV!W59WB`D?mN~r)ax6^V_Ss z+o0-h!l{-POr#YP8Rc~o3q>BQ223=PYNb-CKJ4ZxTX{1G(IERvxqT{QBU%}C%^tG z>d6G^$p*Kc)Ik|{fg+CKk1J3n6etz9C6ia66gZiR1_~G<1puX4g3>HNX|7?66#_wt z`+ySnfP#t)FnLiaGOn1+_%yG7QY5R%(P6v}sikg@x^8VOAe6=kGk{3}r~OM>xX+#8 z2s*!Sf8%^jo)K6GM-gr|$P}bWuJsy&dYYD5+^m)K(;EFclm4(ahBW#K@wmZ7KZEuq zn1qskMx!4KpkHs$Zx)Z|yXaeJn}Inc>0288c`o{ByTKTMPKUl+i@GI<3jKW@lV{Rl z{SGz?J`?em8=DJ)1ZUGy9nKhtrrE)wlTC{oO-pD^!-S6=lNaNGci$7Yf%_ju3X^Lo zb1+U?BA1EirET>tZi}Lq;?mt?im=6dtR;A?>OD|f4Kk7%C1W+S$d*gK1zOTk*tv>ntzy4O| zspeav`BqT#>-?KvKey($$mX|*=C`=jSFO#rRr77pd^@Q5^=9+YkR~Njl-3{3oZS(r z)$xlCHGps)<$q6pljczvbWNM)yET99u()nu<*=mobxdUyELj0dw#G3fV2Qh7iMwHm zo3P+a3aI>UFFLm3-ksZUf2mvxKtf2a!_J}uBsrY;g)zS{v zYNDUt#ckNFe(k8AQ#+}?h<;^J#nCT|`WBi4xyc(|LvtX<2J=q0 zIq?6aqq!Cb?Bx=Tp}4_N++ZlKF@#1$ZZZw``o`QRmAy;x6lBqg$;%+);t%xy>pCb? zuRMKaP+U#0HtsIL-Q8V+2WNp@Jh(0dcS&$}g1axv;_d_uPH+ei+#Nz7NIu?szq(cX zqo=p})YP2rXL_bjcQ2vHUqpnJ_fRs+2Pku*m$tk1AO0N1m$3R+9f9QnVt#hLitzocfyT0WN@mWq&Y6)YD_?H0O{hmVXxEqH zo7h_rN!=CaX{(qeQ){8US01a-t?Q>(^ti>wr@Sn#I4^ct!CKlJ0A2!~0vmfeMldTA z9flxPgrhvY6%y7Xt$-!%uH(!j?Jb1}ze6XKA#o!PrN#pJWD7LV?|~MH5h9|oC5F-M zfV{~Xl3Ag140&IA44H=1?X#@E5OixJ;QsSU`r=Ct7U-~!UU?CQ=}(1YccblJWz(7O zc%flQqndBwBM2z(G0q)Xz4nl8X3%jT3cyHkD{1>$(?*v9y8-!X) zUb>9^aMJdw0}0&7L}3n??kZBeV!t&ysWE&Ss6U>N*ARjE6jNOfb>s_8QeHLz>GlmJ zE}ZQ;+=vfL4LdLOOI|gmw!9w#^M`&Sw5)D~ilX3bT61?K^=XoG{2UH#2OuU3h zlL~OWgebhdKR<#(s(cXVwMqEhh|#lf5Grzm+1SlHiFt_;n6>)%fFqHL74-&vEhi&ILvS3{K?tO))k1!*}}b`OzY`%Qvyp#yN;YxAGjZAqx5KiwMa#~Hv$n;u7H6Eok!>m^=WhtA zZx|iy&!xQ@LGOOkrsDo;)+>5zS8qb!4fuDuZ5Di9s!9^brTOlC>Y*&H{q(&jJSEkA zC5#3{NO6l?%t_4d10z2L;dk+P1&}725p0#<30chjV`4Vp%>A*=Yf}PE7#`Y{E<9Du zhdx`ZlVr7D1Rf>YSKGWIj$zPL(a7~J;DI5|y8bfp3sHBqzEWe&)Y~e69bh01?EnK3 zpDZm7!C+V~7VZg_%N*He8DBSBs1Oqtt75k@z4`bR;a<-V*PhqH7Vb+M5^*sBs#_<^6Zu3|uT8J_wpJ}3T>FiIc}LFfn8 zVq0>oDI&Ca%G*)=0yRpCQ$fl{+z_j`qzlnTOfaXNSIQU=8YiyCTTnW+ER{LmMxHw@EEh z|JzSjsP7g9(k0gxk@GhAG7&3?8xBas8)px9$DnI?r-3iWW)`l4P|WLMFb>CD&f^^v ztU}iR5HJY7IX1t%5lJ{*|Gqz%%z@|bCs!Mm%)z=RPUP+&$F1?b-0C&H!YYNl+rj}+ za+^(g`7!C$^P_@5{xg+j3u!Rf|xBe<*Zm=rYV!w1j&& z6Qtj*(X!Ry_{k8-z9bol@>2&1iq3o*vge6H%a;8g)ztZp#bF&DHb`#`gR|yD%%=U4 zWkYLqtVu-kHX{3yX@EV!Ry;XKX-O{4@OHY1yLbQEPeFGzg4V5#yqSeC!+hn0FoTmt zex5aLpR6@0+)qf&`qM|Lc{u*TpURT_vb^3D#B8pLG${(NtJfdTVct0K=?$6?gU87G zksgN{QU1^D4}Oad32wes9Mq2Z&#);GDoTRghuBCdsSs28lae0l||HS z$mSe>?nH?b;nij_0FJIFr-MZj>oH(d(>0JN7+@ot3t5|kq;Mws z|N9T4AN;)^s!FLRsz+@jOy0P097zZV1ub*a5P8=svBUVnhQC-iF_aL2OIo~ zz4-=S=?5#bNSEQ$K;b!(6ahH74N6W+FX?@I^EWA%ft==#a$(aS0Ld z*}Aep0;xgFr>+z@m=YE&(rags{Cbn+Hvgz7@baW`cEC?qU>Lk;mVP~H^GAkVV zMuV6kV2~ToDwep}PwoIY%{hXqk$+QmNXpIyWh&vJ}q-dN2f`I`=4+V!7 z<)8(F2ZB%B@{31>BHy0Ij0I7k90=zD5{d(N#rNG3ZhSQWev@^HzR^`ij1X}&sBp{1fy9XaF|Q+_CGtYWx&^bWS_??&qtMHQC|EEG zI@B~299={X1Yom46ig!I0b=j~F%+QIDV9_X-<_cBJ}E#9DSa0+y;L6bgH2y>JUyo< zw!N4UT6luoDYKO%-Oxz^^?;KAyn7~&8SYd!Tg1X7Ui8XWD7sSFUFiVlzkf$Bp1bA;Xydx9;4V$5e^Ru zEmB32N#`w!JssL5zy9w%!ro%NbQSukMPO!#4iZR5HeClGL2^gxp*DZuArK!eafs3J)6JkpL1Xr>GGP8Xr2Y17JdjGRmgkMX*=N zwt8kQ}d_De~)sHx-0I&XMURhB#$y2!Udu?A1l%Z2!$6 zwn+6+E`6?8T>bEXHz9Wr4f+-SKPIwFI)9Ps$xs@(TOUJ-DdeE5c$^2#O_~S|Dk-}t z3m_%}Qei;fY>62%*9$23f*5%^8wolap+TGB-{z-^Dw8f;q&I~()gE=8`sIfRZ4As10ASqEf@6T#=gLJfNw*e@uo?PF+2ECfoA|1P* z-O+pz+B6Ssm{&Eh!G<-4gM{I#VIk=?gl%v5#ZWGEhO5De#&^sSzKCS8kEcf*kzQIq zj;|*+z)lhjn(l{ww5$ARRheHuPG+!=R&4^OCx%!J)0C{n|5LzUWAV$)U-rAIwh6JQ zhJoTc%Ue4ly{l-sRvw1i5D-&w^MlVD~{T zv}rK7DQs)#i>=Oa~kU=7{ ziy8ngRGGz3GK*CL3srmzDgZAEfEOWfqZip^NXjU)P-&~lI zT0-v$fE!4Qw6crU{{x`}z%Zmmb2wla3NY6*QrH3O6Fme48}f#-vSttzFNnX_c+Xo_ z&~3F4tUKymBrG@*9ZDAp4lg3t10=9QBGLXEvBE*=5TRV+^2UOZt$09r6reoPVlod< zUK|)EzAu|_W3B<;Gl+5q!w-0i#m-JgNta$ojlCk< zz*sI!tWnskQRw5_aPLRzw?|3mVsIUXqVA_8^l)_Tc9B5YMLD#P5#s;A^9|15=5ieY zR){)L%Fm?`a&~5fC}BaA(4Z-B{}KP+0eFKB`&ZQ!#$!An@qd`_PV^sgYKMZq7Loss z00X1m9DrP7L1$+=h!QqL2_5?5P5O=aUkBh#dWBHHqC(#qU-(mGQ7n%f%9R*O1Xn5` z0Uv0Ew3x*Mv=RrlovyFfx_L;*3famCDFG6YW&R_egfxrMNlR}Vn5qUR2H#OGI{PcW z!M`_j3}vq>V#fzEg+li*K`K!v&iT1rLe3Y=5LJ}_m_%@(v0$(|i*yA(O*J|+Vll9` zz&R6RkYB@~0&N1zB3sYEmMFt!pi$i*192(G#;T@6{%+fgG)xanO4mUeD^BiftM+i<`dG2U3TSktKi#y&7i#np@A#4(lY=BOZXt8GsP# zp`->< z)S^R2P9`+8Mk#RRJwj(GSp+%455nb|N@nASuBy@UpGLk&g;?DRMw%^=eKxduun8rw zhxYTuZpI-|)<%;?aoBq92#{ZA=ctP&iF*u;9KAU>ctC--X@-K8WQkcIJ4m2)5@qF-E(rqz*P0JeCvM|U$y8@r7s<}iPq+)k+9KwG60~?4+q6+qh{wpAZ-V-;S)z`m zNj$?OoJE!d=<2sg1xH&wu0mEWh8Zoc)o*XPNd}3v^^&gA#W5p%k7Z@qM1pCrCns2c zqTi&Q$zrUamD|vsAlb3{f%zLhBJ2Fk zUZXUNo(-x~I!}FNX04B(ttTz_;YgDz{PpF9oP-c8&& zoGjswHw%|`dJW~?{D^P4|85|n9qRr*`;mHz(^#6$WTe?t?V^%Am*i^*ntoR^TKQCeXH?AS;tZ-e%ZAKMQ4FXLz^vWJX zfsNuL>cj-94ya%9jwTENEPp^L)@u`S~ zKMP!7W&Tx*E6b%jUSs^Irm}or@#={>6F`594TIV;@OuR39Ipn_-(ek3(@|)jbJgY$ zvT{0BX$3x?PM~{!{dqKtoaM#hEi{Rp>YP!Q%W{11)$?}qx%|TQ%ZwocVl`AOWKyBR zHc_mwNKU?!LHrs&;UT)t)6vmwaZ=KvN9*?sed7)MYOq|dKsP{a>ZD^C}r}L9C)Z@F<#5yBueBU>N)Fr#0-fAi+XZEte1>- zDq3|hKxvm*$Bt9o&IJ!rQVy~zl_qnnnSn*no^K%Ctux);jUe@q`5kJk$Sxr#dJ4wX z`&6f>AznZul}=;2p_fW)OFk5mgD^T-?|4vri$r1D?hr>SZidMEJ(LBYZbC#NF>GA6zO#MmArfeMPCDO<3DG2DIc1e z!K*92Gi~W4ip=)YDV!g|%=**G#3onc!FrT=b;&ig3LHuxAk->xUdl4Eocu|LrC=eN z1<~4AKCHR|9erm1T!qN_yEn~Tcys%1{(wm+^)^KZ-O~aSF(DXK^-+gz+Ph<((VyND zo9*q?MBv>zVK)ZO`rt8*bTi5;m5D`x>Cx3DXZ^nL=OYxFEi-v`=%F?s8l_I#($)@^ zPFzl_+n9?nCU@hI>HH93$XYNVf`xmW$NdZMaqZBX5eAmV5iomNzfqc|qxtZkUK8*B zw$3W_(2Fl!@0`9$?rf@{pq>-kprz-ZoH-{lUWAd_M*0aPm1_+((rt)Y4?}u}K_0r@ z1$hsC`M#zM1$DBW*C*6ETBxky*q0p}U+L!#x@Hdi8e1qlv=`TjY}cN}tPk^aon!w~ zC3yvdf-hQ`^Ff$T$#nFdZmU$nd^6`5Bq;~Kg0g1QGB<;v;7BxqsehDnG1F}F3g_6h zIhAy0rU)~SdLLUi#P^*O+gx{CK(2~SGxqXVm^$b)opuEqM@0^9A;}2;0`)6d0 z5Ow0B;(3PGC!~Q~1EY zYrR*m8EC-pR&SoXuJ{{ho*;Z9WLw}le(^!NjOhwCf{p92!YC};NX4?MH>7GZ)>?v% zm|Ei(xZqJ4j00M1MmnG(wLJh9K3DvwXBkER=@|~5_gjdpUC6o%cMRThl^b^8aEsu>zUM?AKMZ3>iwe^emJ$kamvdau^UWOkH{=M{ zL;V$gb;k|m7cP0s4J2PsCLS$s?wp|RC%nsTt9)Q3lJE48X0226(f&PLS&2xh;`B81 z(a`uvOk5{Dez?lUM_gY^VRe0a;PSR=KUTA&M7tI#C+G`C?s2NcAf4hMoys7c2B3Vw zRha`a6eMsFHdwIh7}-^JV(5F@OC|t%B+OmrCiKA=x_{jPVSra9of2 z^r|jtU1q@eiWT#V`Px|Zxs{-=U4B#&>)a4<8bS)vD$6A~n&kR@o~K#1{ATZm1B1@b z*V;{y#Z9<}176oW1x>|PWuxc1e>#Z2GA~?s5Ps;}Ff^C?#ve)a`p@gm_?OPwjDSzH zh{k4!=Zog&USq14`+|Rw!EWa(J|@rGQvVX%s+%hk=?AY|-Zq)H%`b;FJc89Kg;YN2+MnUq2QCOn73ycjMXNEo>hRxBX;VwEB>%sDan!&l}6!ZsC zfqP7a`eN`GRJN5}Gha&M==?QEVkJmovZ0fJDs{LeGQ`s&!Yx7+ zX~J;a*HbrF1^Sb%lz-Qk;HaCotDvOF-&cH=ZeN4h@OiB;vQiOjvx2Me2-D%{^s(qr zso-dYgBu`Bo97=NfpEKD{{FQ42++L^o&J(LMZugA6|^r5o0$4{<($%Rdol__`|YTIj6fuAJ2QytrH zN>!Pz7XHo)y>aPUOnl&kk>4s@Jvb~dJU0(240&}dXN zdS^8Hqp!YWaCKq!-Va9jRGq}yr})=vIV7H{>rSRI*2XuWFs5RZf9M~EDDJ=Iu@d+y z&Y3z8SZO=dH4|8c_MTMeZ1g7koT`mNqx#=eu#0fN$n`9o@}m$XLS3t8-l5)oWac0x z*Yf3FoX7G`{>$>_=(0W-|JSW$MLsC+(uh$Qxspw3Bpm+PM{NQ%Y6opW}`VS z%&KBlCO~cEajYgUzxI(sHO==V9&59thmw88wCeEvZ8tGlK#`kIz!!(%v}u~nHlK{2 zS0wbmj#v4s={7vDn@+kmhu{r|>K$kdp6}mtC%~w?QiED{#-2`A-%oee^$K^D2>!;0 z#iY1glI&Vig56*5Ku#YD{!L!RW|OTN6*Od?&*sc6cLqdyAi;TG*&%bjiY@UOpssr2 z7hbl9lPZh7_j}p_ZRHdCZ}@ZI)W@KnzqqX~z{8dHFr5c}4&Pn&qjiDfv=-0f&^*9~ zQ^wu@hzscWJG*Gb;af;!;NNeS+4$vM-FqL0p;cr1co^%o7Jmx&{g~?W9{MxmKj-@~ zo;{Xh`{gRMpU$Jw$hq#I*wPnag#gRQ(jHm)ClL|1W|sFPipuk>Gj0@oaT2IO$Y%94 zywh`NwJ0J1O>o(S(UrvXut_}vYJ8Kwr=c9zJ zU4vmpQ#xzKa8|%u9hSNEY>gg_R9p8ZZRaji_zr9VG0_QWEce}C7mYH0gTL}@3#mw? znrQtifM_d|xdrPJ%0B#aE}=MWcZ+p!#}b;hWfLR}4zJr0zwl|O6KHBlBXN!O8gcue zLB2-?Jqs&VSA$7`*wT`Bg7@2?tKwGbPlsWN7Wjl=!3pmw+u_&Fs<#(>=c3XQtZrg9 zN>YCHcmAP>=&X~7SbX^tr@(MKO`Sp3o!VGlII&{^rWWKW|xm06-J-P zx3NMi9`zzmWu^=I_-_YKqdPGBG!F;OhwE_7Tmz~3TV|>W1(Dz<-s|CI+tDoly!>aY ze7*bcT}3T`d$WeKq~bFhqg4u<)yt+vlRuXcd=YUHcGpq?i%GJtrMR{P{_b2-4IR`? z3?H%6NMta-^9+wdZ)w^ZTnr>|4w8m*Hocd2t@NzLK$AZ#9%dABg=4JK$?OIR`-jjC zj*OmLk}-oh$XdE>v>Llf)TIfL!S5tZqH1H~Z-bIe^ckKebKVl)gnpYizW3!{+MdjE zlfEYkY%&PueOzUoZSiN<);ksp%=;-d>|5rT!j(~@>%xnYS%b};RpR0O_R;#OIrg9P zQR6h{PRFs|FA{(IB}RhJ*qG__?Pd!@T{oj#l^F6y4T%zBb{48?cOuw_F@;w!g3OI6 zD7}E&kT5|;5r*%YRSTGX%R; zpE;zBl`wfDd<8J4#0C%Ur-iTa4Y(mOoQKP!YF2xj(c5kc1WqL%r(rv=Xu8Q`38UYK>KdhBT{VX&Q2N`W1@ErHypXC_AvfWdZ!QGyz_WwF0ULYF zI42~0+Wst5BA(_nUnu@+Vj^^Gs;bf-=80ndE%L5@u^fzb?##qSu5hc}m7)I5+K>2E zadoW1_HiE-TE z++@Hd_u*#Ky2p&v;W|`?{0sUd8nhZpDFNLjU_%9`zzxmY4}G`)%hgHsNxh?><`;xc zZ4NDFw4r_n2=Ad!w=nj5_VbYBqrqTN#_w1TWe`D2RNc6?UovccgLZtdGWz4p5@0Yl zA=%Ikm3?Bc$XB_<_Cb93D$y#7F>k>7C&1=L3)x{UL9vS2;Y=-kBt(9ms<|AQ7J1rS zO(Mqt`Mz7_3kmCj!l329EZ8Pqw9jW(r|#yoR-^W+I~QfC87}|_5_UE?f-^yT*K;9? zKSC87BgOw?bog1FIME-GZzy7cQ=4}bZK_mdq{}&dk}W(NLCIM>dvL|%h4(>cedo`O$DPgT>7Oq~EPNx>h-$oCYNeO3jkffmyBC^MkVWT=4 zPbZ?Pc&vuKGx_-X`V9BAk7>ivx7Cp+3E7gCa}S@?-5trs5$SVG%3!lIzbvHf<~7H3 z$%TK|U5<#Lk_wcU`Tl~$-UHe7M+I)0<G@K8F=CNySgE_p(kN7lS2JT{Gzc4MsK?F=AX2 z?=j}P8BJ_7y`QiVbE4I_8b>Hcf=9Sy3~!lacF~S>5IlF&nsEbE;r<$GFc%3e>ipYt zXg`NT5UfSa8NLf6GU9o+!FS!i5cE}vgaKOv?8jH$vKT{LL9EqhQ5V}cNE;PlkX6Yk z=)s<@a}7UnEOUu$Gft*%4IT>5|H|Pne(_zf-u4|Hap`JO&AtvvlQcsdHcs zq8j%?F3&%>O1v5o=JO7mHH?7TxE3k>cI+AvzShl!<4=w?i#f+HL~=4>-;kw9Fe?5< z1Mm=0l#c7r>9Fs&3-mM5r*d14Q+$S*frI{9mZg3PBYa zfMx7s7s7)=c~wMj%GbE*kpb9zV4*U7GR(gJZ@p3p3PjjYU0Xw4+aAL>uwkROJh&zh zD>Ej``Siww#=MK8H3ts;A}|GQEiM@?Ckn?ai#f1H8L5lIB6~J`KiP5PlV0{ckvNdk z@96JF`~zh9w}+>w0^MXdU?Yb*gGI;Da&N7Ynk)8{TKxcHiFxfxbohC6Fj(0yPfXx9?Zf@01vr z#9xX(dWTBkGrU{3QJ1lb|2V?Nk=7^9M9sdg4%v)XQil(;bW-`aQaKZEc4?|lgqeJ^ z7o{1N&Ki{)>`Ki?fL&izczC23IhNR#BRz<{!)NUsZtSwk(2O+WiI%8H5(mF}xj2=v zf(o@TqfAF*8*JOKIXW}i2`|aLSu_YG72ugB4pOS1(I%;Z;yZu*vlf{|Q$kUcdK@A% z)5*6qPCWQj+iylEwT#-QB4i%C{%kEd*x8IppqdG8L2q#K(M_IX=TqCRl}tbxM2^h; zVfseIBHF_N^*c63!;e5WFyEY)+?>$rKWfI#f>LAagX=KbAL)DeYegL<+S`%eGOBTU z94KK%Qpd73!X_%mVdvPB6ZYl>Jt0)oo<{U-(lRb!1DdX{D;3+7S1t)ZTob{+jLdfr zLZ!6*UzNRVg5q-oj(0b?X&fix{#gH&GGSg#W`bc;Y3ldq(VSZKB;q^O*tjXi8j<)T zYTFk@QbenE8(}N7puC%Fl(+NlzA--Z`PN#x%I^-j_ z9)GRxATD{)TVm$_6f@Qsa((dS(R?j9Kb-+0e;%*W;?XXrrau3=>-i9+@s`kAiX`$L zAeauqY~iF`R42dNz;s0^<;DoQzx9ClUIn>tnzqjfyL>ZjqxXx^nhraX>XmD7)o6{O z(jW{Njj<*I15fslN-qUOzX03GRmAZKzNYDW#5%>=t2{nhVwKo#W;4N+`l!{C?oYMBVTaU z8CWM2b^FKYQ*OB3(8q)BdZQ%3Qc~m>$Arj~ZWN%nj~@DM%10^YOjsQ@i20sV2zjM@>eMqH#eaX>8ld|>%y|Q4O*25jVNtC z^Qlhwr1Q;p{^A+Wfpxbw*vjvLOS1#J;maXJ_uW4ah>bfWiT#*EmlR#keZjxhy04z5 zo!h5=7_6AkU<@N0>$9lGI5Eh}19~pZCQV`JJFNg`np|+n;xyka_1Mrvs;q9DN3LPtiEXc#n5xF z8=1e@F3a|rC^9lEGVYTftzG(~rn|~|WPAPoa#v+ei2T`LS_D77H4{2n$US|~=<)Yp z>#G8}Oz()`|32^y9JK4Pze1ZHz$W7U_stOW+@ciiJ#zNaPF0`<@X>6Dw};sr+)2-J?d0 zA4y*a&98b?*@5ToyubycyJfdqslbg{i0NXM%WQRFmHzJokGm7WVF#~+mpd@bALQ8j zqZ$!39PLNdLk<IwydMkHzTI0!t>bPQc zUDY|@6`p_FpU{p0)ZGsl4WjYC|S5@(&AGW1~;-P9bsyHz5m-cP8b4qgbv-}nnV zu2~0@w+)0f1g3K;9w)UG#z|9!OghV^Efld8ZA14x^Y%TF_B=OUEw_z1KHbkfef{?W z?g8#{eQ99jPKfRX2iKD80cj*wwhJz3zIhsXksoRj`d6CiRm?-vD5V(r1Nw){4KIr~ z8=lQ^DI&|~=OpsgQp>gm@HshOI1mmL)AuApHc!C(84Z?HAk!3qdMT4J`ZH=4OpCq^ ziG7jAoXM0$nU~Sue>*Dsq?^zM)l3@+H8j=Da!>EZ{48idp=CB1Dpk~)j=r3i+g~JE zm#?fa?mN|2PK#a%!z3eJgtT*4k=7o0sI24a7PBBystN8Zm2n(NtUx}&s-UIs(rwfZ5K%=6pJkPcO<%?- zmaoYaavXYw{=;52#b41hK@{_S37bZxkz9$>Jyixuy7#!vLC&AeUrMv`6G%{w#_o9Pn`jr?Vm zd-z+NPDa^M_BLL&uolQsC-=ADvd-HzvH$E&SC3!*`k3mOQz* zBpAQK>mJpY@1&T6v_|oMj|$WNZgVDDcz?q>@u&Gvo7306LRuul3HEROXyFj2?oy*N z@t}sBBcEhLMz0h`V&FDuQth=%2-Zx(e32TvwCy#uXyZRzD_qPTZBFSaoeu2Lkg&xI zfE2P#$S<7bs08PCmP88`Kn%Skppzedm`wEO!{sL<_SmSfSuIRi;p%AAqskdO(j;=C5FWII_cyR&V_my) zA01;lFjLAPyIK7eP2HC(#AYttq9bF(p@w9l+N?pylo&E!0cgQs_x@V-R7FLE^I68M zIi=b^J&HANc)bg5#?o9eVnrbD$wgR_VolipEyfgXo2@m@1!$bnS`75Q3M?T-DIDQz zhRudMZ=-*r(YA%SnuwDEsxeWZ>j7|7v)>U&A`P1$_?^UA=Tj%MXzD?&P(S#wrw|kpfHm`$|_( zD(h*$e5|DMW_=FZK zOkGyrY~dl~W0}G{GZgm|d6l{{eZfu)mMo`RP@aI>%QOsdgdaT^(oK>M=14S`)R3&7 zoOL|)T{{O!6gYRXAZk(Hsm08za(ByYtJ#)0u{T^Hk7c8oU147R)@z?+N9LX#>v@N#5B=D}h z6WB9CGax8m$2-*`4lDZCX2C6TD;Uyp!SDCfys6UISv{)mL{ic%-=kJX+zI3~#hRvB zlq1qFt>{i4sZSqq9ay%+LP?ZGivi!xzaT5>ff9*_M>Euwe#H|o^CPL6JNTJ^9nL@5 z?1?8y;`mKZW=8!fFRtYg?a~dT$F-h)T(f&Dc2cr|#0(Ox$m>*dZNC$N<66kcbor3f zmnB}g?TaS5rQ%bJ+SLDubfcbE?;wzs&RndMenBv0vWcG=RLbx3xaNCN`_rw)$mKH| zL_+8TvlHGqXnxxolseW~a7c4yUQkb6LsB_!IBS97$C$hiv4-|`JoA*cD0(VN4>E21 zdbm>_S@?7VR5>UDysGjNYv*CO&XdJS+qI~~lE#__2nQnP6C-?vqR9#eANGbRWu=r* zc7o)^at43o3aj~)9B|Y8LlP;10x7UvLM6RF&=R02Adsv2L{AfLEtq`2GVa3{=s|i4 z>nEn23yV!r{w0PQvR<)4Cqlvng)URhcK_X9tclKUndZz{2GgPR1+@aNV%Bqj_II#!-;Ok^hI zK9lQqf%kNnvSWOQtdH%D}@$01pI$4GLSY zT?%%zh(0zIw&KHXSazmtQ(CSaYuMGwYH!&ssMf`MhHK(=Q&gfF(92GKhx1F^mL`AW z{j>8s7hT$yTa~!HKv$jtIoH^+G$pLpvll+<0LZ2rfK7C zOP8U`@a@{zUjcen^zGyGu+0rqZvVYI_@iCV}RKe(Gi%n#STw9aUR zADV61Y|hXM){4i(_A*Om?im-JWOGZORi<#UZ4XbU=Hho~$1z7jRCmP&ZelhNWs6R# zA!Y4mz`G)8Wr`We%|m_0-nFSLkAPn3H~M0WV)zBMtI0Vv2Y;`|sMoQK9#JD{q?VNBMcy){0U-tZQCGJNTeT?DB6pc z#6$EUYEUIWN4sIQ(`42;0?-b(_wi(Sr@=Ice*OY$G8b+0SS2x(HJ~^1yaQ+%D8n0g z5(z{7nj;&6>rOa!9gq;CkK__gSibLZC3)Jh*^FEqg0qT8nUjv*??s5>-r)jie7s`|rzj1%kzCufE*kI?zJ!!cv3W-*6bQ3hl8*c4J>g8V^gc zvI7YitoD=l15NPucLC-UO1niRjp9%P%{T&AWZoPLuZ9n)-$;vemLUny7d;HRu;kAA z2jS3?L4`1t*thsVA4ED5+2}=S&|=DpBVpua>476Ga)=C9{W*&-3^fCT@g=>aAbFjq zbHbLT@7M%X@q%M-!Zb>tcZeS#Ie%&r-WdNL%%|iYxcb&T+IM$%T@BVzHEG|E9{Z7+ zgX7(;Zul)CP)j?V&d5$~N5h7|lD)rJfA)4vi_imaklXU?u77bdUHH zUajGTNpA`Zd>b}$5&<&B1cbGAOuv`+8A_ts`$aZ8cl1!73=fa8O}F_t%3$tp^WqxZe*>zBO|X7nQe?(Q>88* z)ymb~%tNMcTCKHFjwQ)!T&*n>pf$~k#CPc_l+#vjzz+`WED7I>3M}*e`IR#Haw##b ze1zUTwKPH4v7++S7~O|4(MhzJ+KP`#6idk(m+a={*s|0n5%Nj-kb|NMlwVuj6h*8j@ z1i3iFcM_IPo!F`xvl^|AIFSN9rnc80-fQ$4ZWjDj8y|k8w~@;A#-ooF4I4m>@)?Jw z5VOx2A5iVL+RRixaCJd6zX&l<#WIk`h>*#{*#J-%o7L-@DJYQ>iL`g=>C^Q$ex z*p_?yJ);**Pe=)CQb z@1db_u71F%RRGidJxt+i(G{uYcy-UTncSl6{e3~+j#MOEq6Kci+v)b^_?GC* zLn~nMh$tW8hc)HbvK)X6#9sR;Q1^#g9>fJ)A2q!`KD|9|B7UN@ty}8{k+?7V8-3xq z%)G!G0JL=xq4AV^|K1x50e!5DF4Wj!E_0iYt(u?2C zali9EBjMkTxKaS*$ec`4ac6w?{zUTg`E2MRv5E4$V&<*m_;c7;I1_&KDn*IYPa&Qs zrc8`r=5=;~#4HN#gTT+7*dn8k#2?dX!wy(9&vZXT9(*8dfg=jM?TTUVKpBk?goM*H z!;`_rxpN?vvidq}Cn!Iw41gRGHYLI|E#`9NdsO>@Sm1J(qdl2xzsZ$U*Y~|R6;>xd znseC@?_zu6ByD4J-a}Jj_$x^qc7yCounaqEm7BN{rAR)-mD*^H8HC>`MrTuDR~pQl zFEQNHUs-q#+GC3M%0W7sN^+idj~0F(xQHP$5ZO~zo(PI0Xkz01J8MEQWcs@lrQ?WV zcdxBJ9%sZyYqpA6=)Fu({Qm%hKzzS+T*k#`YsE#Q;^S!LhoKxCpPRv@YxuY$rp3n3 zV==bS5@YMBlk;UK`}*ip!an`qUc%pf(ifSVmF}N5oI;J=n=nr8I-cj6q>6u$^~KOq$2!=wwfNI`sak$9u|nam<5F$&=wqIt2bL>C}HI$lZJ z4#99|_0X9;K#!?{PDa;FpIqMeKT`_DmBqxCHO!Xi>ggi_siDRe^?+~I^pfs5qHJoo zRf1c@Ld~+!<;7ZTMKuqmIw@}fnk#Q@La5C*Z)EW>y4Zg46MbGh87F=ZHH#xBV1|9n z0MTs}qq(ojT-}TGg|Uym;4NDXtt~BzV=lps{&npb9VOkT=9WogSB-hyc8>gepa^a6 zam%f*mXyU>bV;jrOVBA`*}+=dUYFRcGa8P@`j9o1l4oSXiF7gd zb@ugo6X#~bw+!dkK3K)lmS18Lyt%qsIjs)PTjiNN&v1isasqupdVPN?emRHE7SKsz zi%)Vg&l2CTHJAV8HWLy6$P|R};Nthi&fM*f0PUP#8K+MzUdSxuuoD@(RXLgA4G%th5V zoJ#4c)Qiydh920D@V!6xz$y$<^O{ z@LlT^7Cm`u$Vc#3$Drl=PmtaBhymn{>^QG*$%jwZSFuiU66NHFlkjdBB7W4<`Xu12 zduC5B46|M`XPEFGM*NSI*>Y2RCovBroPelHvTEb)l(7#yR7VPAS5CbK7CZ5&(meOy zbRZwe65C&cvHeG@lv(JzF>wOjOHzxsBS4;>@o=IkdMRC{5F~3)Y}7l>depo)@S8oskfq0 zRAvv3O*&^&A5RWKAn%aLJN{_qCyQ9&>|l=UGGt1_X*5^vEvj1zkoiOmL=v#L3)M!jA1=pQw|+rQ_`;+vkY< zJmTB5mA69kz-&@}BA{m18n~8Bn~ZAq(+y%R`Ph)Bs7*X8SoNz7;Ku>^pQ|e;knyh- zuUDjUJPVmq^x8M1AyXr+<#3?)h;rQIT_}6oy@QC~vhs5hoOF02UZw>Iu zED_B@r2~SOuL&sq@39TFb(nqpL*}3_iL(r zs7_{E&9mF%tblI{VyIOwAofA7VG-$(1p7*OEbdVG9&yS>l3C=$UKsCj+IY7fhEul2 z=eS;%`V?T(4n;Y2%lvp5F4PEw_!Vp(%~mC4>p>rC@EfU95+$jwN9(GVX4qQuoVQh; z9NrN{rGKxi=Vv)oEJae>a?Z+K4FiAiJ{G#cb;r1sm6P3#beD6_wyveyYOJu4G|=f*BO#pdF2XeZJ5?@C{7`J)RHQ4Cl`+jZ6-6bZjHI;TDgPA+~iu0 zv5VFyzos)4YNBpwz+kE;O4sL!;{#;lq%b-3_N36_4Qgv}*Y<2F*s|%%Ry+q7TFztdI8dC+{^Ba5;k9cl=pGRr-s>6;B$e!WP$yZh8`yM`?x;-5|C$i z1i?4eD6g|Ptf>Uuv@xnjw_|olYVPsDTKWrGnl}Cj_hp;bh^1-4*QN3jHex8%rLr^T za?Otgn`bd%Li)BuF#)Zl8H#@rl!{UKlI{q>1OXxT{VeVzuQ;kd=vT|iH3!xNF^8yD z%po+9?ulvXvuGvJ&b2?Nv*-J(H*fuOpOx>gP9cQN=U0obTE0K)in4k}D%0`qYx??= z?~iS=wUfpYx3#ioX}$6Z!#M1EyAN2MQXW`hOU9`sNy2W0^+w~~)U19#O2Ry)Td2dP z&!Oasp_lob$q;|`j&6QLJhl1<5`^TlTF*U|bl;`5=#!Ovv=b|qx0*mNMOH87>0MR_ zK93F#*E&$acSRl@Y*}M!3Ji2SusL>Z)Q{!|&1hp<3&H zTk8VF9U;Xm3Hg*D0^z5+?nGav6s*cZk7$UTDx^g#NsG#hQChqqG&Zw-8%m3bn9i}w zNhc@&k(r1_iU3Bs?=3Op&C(#W+1!#)uWID08uiLD%20Z`_Bl>E72U$MzJfD|FT=>3 zHX=>n98v1`9aUT=U<9{%_1N^3lI}x%wwt}Z+V&OF_9nx2v9wp)MftZ~l=_u+)v;H@ z#f3DyuUzv(O8!`c)!hg=oL?)Zic-ixs=!B`II2+~DQ$@T^*!)pNU*vTSkp_oPjuqR zyZtP1%*fwaTJ~Fw!mD4q<>0dJ$PAyX9}`jGp>f8i+{|YeFm$c zs0_(vdZrBIU&b^z0-@pqW?}Ko;vaLAEFMtB1G4zJRJ>v~7XO#{$MnnM07szl2-$k$ zNLjIQzpUAKPBZ(`-Pa6ILn{xqgAFA~iQsr}FuU=2oE(SK;S_HWD#*mOJinA|2X2UV zc5v{vblPeS8YrfNx*uo0{p!8=JWOn1>#4o$^x`&3BL{~{oS6#Gl_1yB;=c&}^qCzH zW+*;?#HtNFu&dx?M6gq0@<_-otydG`6KARPKIJ&M{!X9|usM&wF;0MU4@w*~zI_N@ z0UK`RyBq~;tXOIh4eh=p9b8d^F29ft5H5 zEq%GMFRfanj97lW+#zNq&?*yyo~x2EZB}zi_kqjoCDO**W!^8^?EaPxi(`TR!iY}& zM~)F4`j7u)L@l-vweX1k@9$zn2l^DfZrDu6z6#)0KKJ(=207aZ85NN`Z)uXKqQg3C zl;LmFX|)&3Ry@ljeDY2DKbQnM%z%ABE2bVI4g~kda20c?x)&>E&DI>At*fwenX}EB z14{;UvDDeDvtYPOnhRa(%K2mYvf;z{#Ek$pyq92;8<=dnob8tL7^>Dc`|P(8viod~ z&h+`wJ6h=4(L(O1c8%;vg@Hcf(NVThRdXG?c;WoiB*oXFG@C${_sF~(P}bdvY;%c|L!ioo z^dzm=XWxJ;_N9LoyW=xH!rifZI=F25M-Rz5NIpvX2mC_)1AbWg$4k`TlAE!=lRgmr zefSU75YS*ZrJO5|nT0Q=h)OS0rOqT)dhlIQX|b#nc2){=rB|rZ6>V7QzCVddO;{;O zWs;P26*dXYJ?SaCxxMy*Qu8V`RNRgY{aO5D9wHlpKG8@V}eZJ^QOPrs{KBsz2pD$nYLgcmP6XyXf)>fAc1piaUro#=vq-ce0AJ);d< zkhPSmvGZ1TPq$Q!I)#=SF@rTXu7jl}F=AhTaWrrLABh+0$AOw>|E~|FKhE;RHg4H2 zRB@WLAq6zqhC0PWTv!N&r$O(nZOX4)2|cNCtk;+7Yd+3F8KkG;HzDDG*oxa2(6u_n z5M>PbouG_TvWCRulCLI6DvO>or;agP*6=o-0&8v$`%>Od?7r%^*=QqU_P_i>F8q+$ z(#Z)lu0uBUrdhCvt;6jkqr&twi&v`tVsB11?9I!>-W&^ia*=+)OsBi&k)j37 zfl-!OnOlJ2JB ziKgr?efIuDPH43TZ*4#Xn4kWz&z4&c>d?)d3$-_m`>G1n2iq<=ZY=4(rkD^=`YF{NCPEP3tmauRH95%^y6JUz#QZJ{DG4Ai89ZM*VMU+~s} z{L`4S3+wM)=2s7)Pg5+lGZRUd`FDPm#9TSjCAgkkg6q*G7`{Qm)7Z5Albp%~^wl$n zNba~BiKOcHf=H&nU&;|3IOp$;ln(rON+zbG&qr%N3}pIF{3{;3WtCyQpE6Lshhw$# zsFy+Wu$Q-?m))@!C;niD(Pyy6ww53{#rSw7xfWf>E`UFX0yD9PIBPx>_p5={M9i1_R(6O&cWx8K=+JV z*V{*Pod09EqnNlUtdiG)FIOwOO4O(WqJ%o;4UxmdnyN0eTH6lYLh&`2(bSNq!vW<9 zQ0tp~4mlI>+pw(3cE&XN_J`^4xl=i$gv^VcCgC4qJr!KCx%qOd`J&0!^2woDgqo>J z0m^x;@St$=8987GOZS6_*88YMTJI6?ceHn&nvYk!sDpq6k;L{FVX@SYC+f_a=gAv3 zYQNDjVZQTXhUZ3B|2%bft(6iEvi90MXKBy=HnI9~C!j0no+k=cKll%&oRu9)`y2OA zY0sov*YC~~%(o{JbtDsIrAhV;)yymZtfh{Xb0wNlUieWf^H3Mw z_i6uP8^}d{`o&QiKr?vQ)H-b6tv1%12lZ)=zBU-z*q(`dtQk7oOP*I3%x40xX35s2 ztubum&}BryF`ILgdwVl%bDER}bTNdJ+lQBiiqHO2{mmR}S~Z$4ZU(h|Te2$*6CJ%Fxsq<*dp3r zD&7}g(}@`e6t&d|DX~b@iA^7<()0(UV{dSY_|YPmcN{89oZomZR#2TC^Ai<_l>o7$ zEE(<5OGI6*QZ6PmeyGGz8k1)DppQz1=nMS zHW@QRBovymr+epVh)lXlOv6Rj(lkVZOZ{=k9oRI99QT4tXH_AO6vL9(J}7!E=jI45 zEjwLEX_-#0u~IIYMM@u_+~AW-x=#R)oHHReYLob1>Q`}jHADuJOK;UlXX6KF?<*!# z?{Qo2%{{(rb9E09x81)JFr;qdMVn`7M5XfT4XKwO?beq*ijdY&(tUHUbYTYR0rWQN zO>@){7@!DT;2sa3I~@MukQNSqz`nN2$c2^#rV=9b6Gn9;)m`Klp2zB|ymJjFMlNYq zJXvhiTJwtox$N%@EtN|xHT3&R{ z9+T9prmrQGB{)ATKI$|Fr6%-8X;RAbz+rtVA+6ayOuHc$-&{XT3sl25mko1naW}E8 zQ7$YA^NnM8K9@NV+eBeHH`eg?d5qXOI%1`AAc$Td7VdY#vRC*Bv2iV5#P!51k_Z-E%My(FMPm&TtS#vi{>%Rl-hzk3wFe_y;;J=L4T<;+aVn&FJ!$sImu~CMMEFU*6e2X~ zI5C{}uBPFHsr&Wx_34c*WO@mrJQke9a6z2Q|k|T`OxuAX)fJ8=PWjt~N{tgF)7EtF2ZV)likJPc$|CyzR|#6yq7c@j zzBde{`P_>0Ny!#qd7gVPF3&)G-tpnmURrYH=S-yN3BM98CXvaYP96+$ds)OF!A_e_ z%-^((h%I6fo9#9cJUBrW>ku9Y$FS)%7YQz@E{jQmi?mdg#ZoV34QYcph)gD(jL)HmZiF60UCz<0P5nN-)C zJ1y;~+LYCPA@rS+4?b1Ms8K6faQ01xwXhLx9y2W5 zz>3-qflHH?5Mn~s6l(I;x>zMdhzY5kp^lgk%x756XGqLvSRkgILtUEmPH|m%QoZds0H{){+iik3JV;{v;^uNg5%(NP+>>qgd=C9=-n<2$ zDAFr}7jUoh-o2S;H)m2gWljWofO7LX@@` zc@zSL(sShf!D2<|N~)-A^AXW*_bg@$;<}yKKbP!xS|sVDpW0!%Oos_A7_xT`m0?A2 z<|>3&pfzsUMBp&Mf({rob(0blEs7rXeElj1q9t45srxhxpJBR~e6AhH*ONMsJY~er z08IM?GM-srDEnMHTb2?Tl7uzX8Q9A3mw&K+_KxwA+bmS9P>YVQBD=u%)5du28`JOPH?hicuT{U&0i! z;&Vf(HDnJy^BKNFNHcuP+;x~(;6c3l1S+eb!4R8_wZO^T@px)IFedLy1_h))x-v7! z%LbYdqN#T|YsS9#udi{OJ*1AdTVsa!uEnU^4{>D}^wubdx`n(QGV1pHZQCL0wu_08 zdrVe!^=Pb5(UK8sTKIjZS(NRAKih{uwh!gmK7`ypP_!=JFvE#cS~JtwxCGde4@wB* z>z|T$-;kw2yq%~ndy)u@0SJ21!La!e*Ne&4P@#!9TPx4fTDiTodCt}_QRs(@9jyhj zv=*?pcCxdzvC!IFXKVf}t@-V(ZC~tY?HFk7L}zP7Sz0TyxAv5?wPT^Ro+llx`5dif zaFHfY;6A^*&q0fAtcDcZ=*`83Iq4kNJrI^ZsRqk$;qn0}KfESe`KbAoP>UdUjQQCs zy)q|bNSQN{tg?Y<*<*zYlG9!rToUtTi?8PIMIPUav&UzL$1SVjGZ@UjyEu!;%_g6g zLv3@P6MAJ1A7};dPn1SWXW53uepkC|DfX%186~}d18g)DDo4cbVN-w+CvV5 zG#riEXn*0q!cX zxf`gt9&B!s*4#&*iRL~wG%&bXc@GZ{O==`c@uKnLNq)WbNrKs`l~<8gOIuVzF8{a- zt>%8a?lV@s~QP=&L*{-M%B!b(EI9Wf^oF8Xvk#p09h zA*i#tDI_%a*@pHi*~YoNtI1~i3_rva##*b%cDj%sBIEILtKe+zjHN1<1Q)f_^jkFkiN|Z1dTIQh z!1xV&Hx5E8gKX&%}A zBCIF24`m*>tL^G(I8P0`>ARPN)QVF3SMxTR7J(+SKdX3iZwU6{#P7xZ~U2j;+mJeE& zKXnysDLg-}!%Le@rHPWTLK!a1_HUS*!HO(!DOBoDQ7P@OsLVMtk|pHpf1GLx?Q9_XL% zCQypaIGyy*mgz$Od_Iv~U%lycEFU?8%GW^oiKk~T|JVd9pE!%khoOA)gzV+dISb1h zXH)r}X`=k7vs}yP+P~I-l?`$tHlWJ&W5VG#djbxdG#r{AZD)TsrE>jBZRu*%mJaV8 z-A^5a)>(nv_&uEnpu1NXRzTa4X=X{tmmVSeewp5LG9wZ_`~3|-oV?TErmfw6E@Bg6u@Y}MN|0k87aXat-5S|?6 z-@OIP*WXIz-@6|8^_J}A2ef1Ph3!;+5tRR1TlVsIKaMNg0Q1yi!B5A$^ibEb)_4!e z{r!MQ?(YK*^Y|{|c_oJM>~>V{@?S-Be}8vO?(a);v~TTS=NZ=iwdDRR`&Zzt<)~d> zbA;{EZ#q~CnZMzrT^{+&to>WVFiikN;ARl}W{%BN$#}VWn&{FF=T8>&rgI&~I`5ll z7((?L`5K6b`R(Dhk!G4g@feVHyFxsnvG3<3c_XI}v5bAl(N}(e#y&rrUSsAU(GNka z?P~+~^`_fPYG@lX84!)O3S8VSo6vSK?t(Z^vh`gn^BXG=Mq2z6HX|Lsnj)k3zWkO=+C-R9fbLFUBj-O%7BnUI3^+gacvyd?J zml`K##{Ly4=AI&HLKn5Hw1@-*rc6YeMq*7Yqu`e8X(XA1pyfhfQ(WzMD+d~Rhog7uF)z@qe}nhvp^CHuCL)p?I}QuDL<9NB8-}P z`>u1&$3^(@3&bM)$s$^W0rsAq3}b&dc)GD8cw-s!Qo{5FF)R@EjFR9@pG*CsebBq7 zZA~X<&1`$NRBZn`)0?~Bm2#D{PMv!5`1y=_vwM)O1St#6t40ICMyu4x6&jrfiCT2+wF1r7+kLjS zMyuX_KTp&H)u@qUREFpz02;O+fdk~bs!T;YK^6dL;xO+1V#AuHE9_L|PvbdFqIF!o zW$~YLxKNbf??mzUBk}ip{>y`3qw<`{!{j%J5_@`{=S1@~A-?==e(lvsgA8(%AE)y| z2#1w!*7cB?xfSzecUgOKX&JyJO6(U~<$VcLcNA=KS}`4q@-5$@V>T6ZA6|_9uz2z~ zeUvOgTq@r;f)#wEvVxCfeR^bf4?Bj`H#c2uG)~8#$yhz_!6>&tW1SoD(X7BnX|ubx z0+t$+@ZJbgVOKkxVt`jlNRqLO<*5WsqQ8|BjfEftD@aOr`7e;wP6wrX`@@*90Ol$< z3p^k}Wf0I*LU7Sh1p2h|)ga^z3-$5#>VZ}4g^!!?m+qt$3t%YLhTJ;tVmrJv5=gkv zz1pz%hc=*z_`j_EoC*C*gi9uH&YC-`*gp=9r7;;>sZVMQClwD;t7wE_o>%Nt4vp`H zcR9S4-8ET0{!JP=Rkj8!@{)H6WDtWq;K}S-Qrfh$+=wNfm0035Exj`;f%D-oY(I z;_B|4R*H`ti$~i7k>eqp7w^tNCygOV&fr@cTq5YXWrD9N^!5j?}K=e*yNY1}3Qv&B@37@K&Mnl*v{M z>US(gPeW%$v24?_in!93t0}G45|;@VQIW8TiW*j+!ph2pqGG2aNk+9q$TQ9He%~1(%KZ2M7NBH?y&htbh`l2oq8XKGDp*0teys!RQVb#Zby*<*JMt^ zK3}0e|1#fsC{#^-ZZxb5A{}>g;Vqm~PoU0NL`AJGkAkZ>@$QG$sgINIarR+uz}@cE zsmBx^1e*(7jF9(jW$$+wg{@8Gn~F%G94C4<8e04VQ*TLh3#q}l*Kp#JbTixiN|uEa zS6YXeGn~zwk=x$c+_aa`0KxCISry{#Thr;(@YKFJ3tjq6I`gKeZ2Uw+!LBPD12HcY zf+ICFHN^S;lI^aW{WtTl@9W=PN?6~MvAsO&$Nbs-Ns_S&@A_rg>&KI^fZO=g)KQ^o zc#Nx(j`oK%Z|-oppIQHYY^J|9-dCI<^Rx~9&>}itecHL9U;H+*=WI6-#_ZrT9!!Yx`1bH$SIcHG{2%sWpGnC6BEdm1w9O4LFFZ zP;kYTQs^hpIxdtNV!buhTl>AC8SN=10)`wDuV~~msnf^=I9MJC;GD7>2$G zW4KZq!^Lmm7-?To9?`T7 zB!O~QH{MuSE1l4+Tj~)NwC;9LH(L>B#ex-BBuiW>8xVE=j_V%n+-02JL@4M}p z+dhBf{vP)IIecDl%K>;^ux9Vv|WW~?n3JmJ$(dkte#GC>FG5nf<1Mgx-H$k1epF1)zjJT zJ$-h;wv2|JxNsXbWCP$%cxeH!`fj|^@b%UEaw z)?{vJ-^zE*Ly`7DGsXjPw)F7g8jY`+fx#TSr7CLNMV36V8B4-f^a?b06(c0dI*W>) znxo(MntN^iHqGj__uD?`_a|;s9e}Tm+0Z9DxHL!0JDtDH*77SCY?Cb$SUU-Rxb7$0 z$O)NZ=3<8Gm(H-T*Ho^F+s!e6rOmyW-RyTGC$yz*;;p*7biw7>&4kMuN0?=l58<#IvgM%K z^(8I{gYN$p)UJy)wd>Q@Q0*#lHx_1>Z>%7*Jdgod6l>KmjANLh1Z@Cmv*8M0k4ci zA-a0Gc-45Z=E1RAwe47qryHZeex3bp#vJ37_C^|Cj220W|8wbMb4Q)S7TUaOH12Hi zRQ!}=D-^Frhd7)hbby0f>`rmoDC^A^ZWj1FC*OuKz%0Lpe|wXK>%#4=+W2j7jm9h) z2-7La83Yce_X)Fa-#RBG8#-Me$+8~X`;YLgT7&tplWDBYN5`dgMsU_=OmC5R3C#Mq zWi3AP3h6S(PoaZ@TDWQx`DA8i!bgV~5u_fq`>4a+hjlzFG&yodh_}vMSR@Z;8_7eZ zdD(WL1P`HRp<-X?Dd5JMh-Mz>lkGy)hk3`g8cHd>U%S;Up4?1$4KspWmYGz6JUsr~(qIX@e1*>11EUWfLlz z#WaoeN?0=SHknJVB~s!x=c+beD3)!8sLkE2Hqm(~+`J4*8&jc&+3r2CCUg3=EYG?x zQLtvi*D>NHp&mp%aDT9oFScJ-7^%DEnyUHB;=i=Y_;@=xKl^U~uS89AAz8L}f&sRkM z7W+$ffPV)RuJ22#9*7WhUnF7Vbzi<8&w4?v&_ZrzjiknCjA+`3?j$etk(<#5yuQn7{5c~IMf;SarNT2; z1>!tt(89J%w5H7#p{sx7691J_P>>;yzyP#;SDStz<8qi-ll9VeaB?vlHW8-en*Ef%%MoyBdE> zu{C~Qf#WaH$DeYH|6kwP_)pF`{;${A$3JFG9^?O3jsHP!{M*Lt)5iSv$M!M*<#V?& z+u-fb_{ZlQ{}{*k$F0s|{NJeYKkVy2{{1@syZ>wSAI~}d>sH&x|LkjdjQ?vj{^@zp zf0;J^MMc*5X9unEpRvV0{^xpl{QNxl3w(~q=MU*Q>M}nE@W7AFGC#Z7=jXkX#Qc2O zmu-ILzhR%B`K!F==aC*YKaXYMkBS22re?uPh zpO1Vz%%LCL{-s?$e({N2KCb%oyONK~j>{<@mv`FdXZg$C^K;{uYJPr`g+D?C%Euc% zw9n5CAAMK!v20v!{i(w~{?3(ojQL(>IL80a?`-_B-1?Ja{A*vz zWBmV8`}jXtp2zqJzyKP$%OUcXn`$A9k9JjVaYccQ<2_9w^qd;crrKPtEW{*rzC7cI$S{HgCmK7aPU zWBl9yE93uB_VMT9zdF3S6uEEal*-5qur-dtpI}&{P>Ej^qvyws6TtK162tQR`0@eE zbD2(U(UqjWT)rrujIJZi)W8so#BR_Ucn%v#u_WN!&(-AbW?3U^@x*w&S@~{SMt;u> zvjnW65Eqv$nfZ-BA+OJh&}_(4YR>i7uH5IB_NCK$+T=uV31qKsNfc!dM9}H$H4k`Z zenM%>N^ZGBC81xBZ)R0)L6f(FlU``ay<%2=bMZtUTk+#jos$rS&k_a(U*$}m7g>lX zysOCS$-Ut*voH@mHTNPl_qTdyuCNd8$L3Dryv!jymB1>?smh$Ip~_WSm8Yo65Oq8O zM<&X*027#}&S$GGk;u4Zbc^Wvcsr+mI_P}J3e|@FLHuL|9{@$E{)nr3G3zi4%b1Kg z!)SzaE8}<`1%-xQ(<{&BS|lsSo0o;r{C+Y$CTE1%{{ZusMfd^JTJ`v-@36h^bhB#q zP&gMjGT(|jG^mQm7Xp4jhY_5mJQF;?i|M@>JT*~!FSQnUa5r&kbiNfYozyG&;eu?G zBNOxE@f(MlVdXt;qjX(*rDE<^CgqnQW{`{w@GLVf&h0IvUfP{Ce`fIihS~^ z7`{%C&vlOJ1VU+_OGwAEpWH&!;wS^R^5Xc2K+J7hwu5&tt~lj>M8aX$Igt>%M|W1# zE@K$*)-LYovHq~yp!_ z>!=o3O8c8u?`#opt4<3lyRP=Vw7MUQ=a@Aufp z_rAmKy~lqvzWmpJxWy#GZad5q;8`gX>DiQmF7x4k_fI9rCm$|Dg?D6jh!@3~#-5+zDXZr@Q<{X57Li(oK?0#x-GkkVnj7CG-paRP zzQ+%7byIgMU&GMF4^eeBxzvP~8Jfg;#?Sw&AX%tqx;3v8sLOia2ke)gaqzjPlb#C@UQ=_;ksNJ|a>96eEJQyk(#+z?CDvIKP&hEjZxCK9y6#v+Za+6kB zQ348YFrn46#DDJKY}lLuO&3682C$E`kv5!taMJ<8GPlL!=x>kImm!fc^8?YZQQFDB z#Y1Ozxp*wAgePN5XSctU;1FGjnGcG#g-C^y0Lr9MhM6?Vga9rp54N|0)SM`u1xM5@ z8e}w&ooO^2fWs^jZFYXJN2!`z@kVq?uf%g|xG1*4#8P{zQsM;`+A{+R8VURsKQLqm zFX?F?wE6x)Y%vmTHGgnya_tKGy3tqNUb0GjrHG;`(b%|7bBA%wn~SSN(Q%#K1I9If zF!qGf&J&Uy$0di{ko32du3Ao&*A~UwcfApt22({W&|f9T^XyNW71h${RzR6Isy3t~(|ls| zVm3r;187v$@X#H|F{~W~j8834FisGY{c*7Q!((j&86XO^ItuNIgQwnx>Z6y$NE9|? z<_Bq{F!8!vye{uF_Ytql+x;!zpI#=fM_;#sB7rvwS+}z&H>A^q>H#Fmv|fD=sNVU( z4#l1B4fsR@-9NME6}93;{FO+$0Ws->x4&@vw0xhgg^No}6QOI47@8urfE3$qk z;IFd*|33=X8UYtDgqlG4!frUooIoSYIZy&sOUOOqJT6m+Gys)@L&i6Lr&k2=@7e=qm= zUXk^?yEKV0!$)$p8}WcE;sIAAKe)OhxY}~Q#NaG{G3}ja7;>ww0>A5J244{NL~C5G z6*tfDKe~!Bj=6*e1aHFJ-R&ja+rC8mf!GJPln*TfbtwG-lSY@5d^ECgesd!~XBjs# zwwKAHkqV8K+<^o*~*mX zQKlkmnF3UVNB{R!gbZ2pKc+Keh&uG^v913jbU9ZIc}V>+wDOGBE00yLBmW~p@P9<+ ziMJzLDSa!f?_;}uSI^w?%=5tp{Ew851l|uW)Um;&j-B@j zsbdMDjzx5Jtcfe+pqVZYn(OkQS?hbQk6HMp%Y*N_Jowi7u74zKJ=&Ma08S+!nrrop zdaS-tkJUGtw-PWmS2JThz!>WR#@PJ8(5(@i#&IjTV2qiNCLQDKp*`e@n%m(_WYRd{*+}|D?T61E(?v zIF;N?wld{;l&Q#CCWrY@7dVyhg05%U(|L~|+U{N-Y8a{AT|XVTi;G#$X-|Vwfd2)1 zns{58tC>mudGe)pf+@0R}W={f&nYE9B%r$}Zo zODMH0Ll4S&l`kqqb%R|j7H+13@3nVpTmA#Pw!Tw8EqU&$BU|!@eQ2qURR?`<$;;c4 z_g!!7tcI2Ek?L0?RKJT4`rc}&TMca-<5V-gw;C2|HJtOOeaeib;_nXecdht4Tl^g+ z{&o_7U+i6GtP7VJPl&&p#osT)-znnnhvILL`1|yLvV870mUJ%_kS1j?tW&16dITTS zj&n*&c{i4M$!FP2Hco$V6$8F$Lm~0?${2@J8cX_pt-29SdkKcLN^q$Ldm8&xhe7xN zDwzENM`7}+uH;wbjr>}Y3~8g#3YNzCkd;+Qs$8zOrz(ScF!nr3{FlAwqHH|}sAqDa z&eC(hz2^-kcFbPXt{tNvwY}drh#c4SX$DZ5%?XvE)*(K}6i=wE6fgCAC1d9NI7HWy zqs0We@;zqG5-;snV_g&12s|$=+D8PN#*vN@jdUB4%S_6;`b>_@I+G3@x^|ochc@cT z(sN9G1yEeU(lru72o@l?1$TER5Fj`Nm&M)PEogw?8rG z`{~^YAwy#74RD~|x{141bkTgwMLFs7y$21hmPwr>uU+9UH{rN0hOywkgALXjv_Br~ zUG@YzW(D-8BhgEJKKG#n`47Ipp1O-Z|NGTC+wisNP~g1&5bdd#%RNz}E63;B{`lcN zlhM1q=7L=}GT+s{MUc2R|A|Sfios;x3ObeuZ9)gfeunJ7d)l`~?TQaP z@aeRyzz}2=2mZ3?nVtXgw4e?qPq~(ezDCwCcXBs;AdAuy7Psb?l&{WlC-ikO#fl&2 z4L<$IX4lqR+H#s`LbD$u6TMkZ-(8-8Nm|6gtQ7cyx3qj*dRHfSoxM?gWNCuqHG7l2 zy7xKxB}@Ny_4v3I%ID>EaMyceIe6N7JhI}qb0_Bo2fg(Y@bhuqc%;+H`O86$>^F+s zAETm&?l$4Nr0B^679TyOrmObG57d^u=hnJz8yQreX=#h_Rg}v7K~+uBkR7_V62vnR z2I`8=qs_Mn4V^8?WVzyxroE!tNRQBQFdI?tGL`ShP}1_3xe8T@KQ$P4^uQVS3N=t) ztLuz7b`(-BNp0Nsmnm)^D&p`=y6S#bbMsohiWl623Pq5Jc}@q#g^;*_{oBPMmrE>u z2%gS`6wv9Z&Y}8lt}Nf1SP}5g!?f!2UA}z%TTs&HRm1crd@|Fn@gl$LF;UDw9pz1? zgeT_Eq6l)+iuL3kj^lP%KWwE@Cw$9Q*iy@}(K1T4rp97=#IHjv3H*M^S&?;jL^)ZO z?s0{O0p3$B<#Y+{1WVD58pf9zYh@U3yu}OM{Bf6%rqf|Mk+ysXdbHN=KM;{lvy$^= zT_SVG`w`3dSJer3G9pfkBnGFth)AYis-aZheYh?q4AgFMHFu9=+G#6k>)i8E?j>;) za?rgwFs^)es!ZRzN@~C9O37`3pol&Q{6IaJE7%a-;p1wb)-cU`-xlGLp#)sMG0|}_ z!#z8Erm4)AU;1lfCNOWvHWSWGKME=kA=(QP-`m4RX{^s+JLHX3Rw%NhQr8U>*jSWt?xnN`e2HBY+A66OKd*_xv zR0dAp-RpSfoL)6wi!N0(cg(0gC-bxocI`)1MKnM4fd&v9Qgl{&TgxHxEH0T(W}lA; z?#)P!o(`IUNe|t3l1ue~(RjA+oN-;Tf9?Z8;9ABiZMN4JS|^b~57i9sKzFCW(lJZ> z->XO~ZX`8k`0ZU@O)GiY)GN->-IG544^NME=X7IX`6m}wH(;c9J20mav~k2=KSl@& zbMeeCZzva8Zxn}J-&b#PxJYXmLf zE^s5dB%8T$)Yd*n2uk1w$k8_UZ?QJ`AAGNhK4cKwpxCsln-s|re**=&OOmTc{+Q1K zA*j0hB&sT;`aY~(26>CQxh}+6V~RoV=&6qZA@{Ua0|1U%z4yH;yJIJ+ime(5VM(K7 z=B_#g;NJ9qXslnx^p5zq_5-IE>e{s-sj zPFC;8%B%E~OO!tyx{F#@B4XPt1F&i__QsKRTQ4EVz2XSn{ah8sy)}s`qy5_$kSjOO z&qq&p*MidvErRGSRLS1?)_YF&E(pNG>E#$8ou2_>ximLy`avwI}^tB%aXN(L3*`QXWYb&IxkOk00GA_VU|3AaV0uU5~ z3xslc*g4a91%@yh@ZCnCT#V-01Gz>Nx(uoMeFG31GXHw#E}v+!Y5)mZk8}SOQ3%QB_U(<6NM<;%xgdlst1=5RA z!|>6qDx%~PxEAZn;*RON^Btf%4|W8n@0J1M+Gk|%0%}<%1c_*$MOEcS0tx3k>%d=6 z>YLPoaR-45aNEkV8Q+1tMSygB`2mf9`*n~IqzDDf$figB>FnlhSi4h3o(TX|<>lv1k32T*@YWkX))8`sZbmmZODX2;UTTT@BKL+a|1| z`k_q^1c=11lNQ~Di}_k|={OSLxK9EoCP!ue7KSiHQGl?&Fth>f%wFTjpdn!({iCD+ zy`@tBG$7SEbGMZb5O!a}(>CedAOb|uO&!$k^tc41WlCr#q>3*F$Ydl5(8`QS0n;u7 z=$(ZW=w$-vfWmmtAAndqs!bg<&@LO{TFM^{SzLVoDt$iky2u4I2~i?BD)SyD1gW8W z<}=*J^0YOHECU>G6Q=cm_#AQ*R{x@lz41TwPNQg8?`B^EH&TVnPuzAoKBKl^hTRfX{I4_80e zKaN{M0ggA3flRg|036350HX&3flNZ+>|Kake-nb*83CdXAMSuDUr>QiqDyjtDFx3R z(~VGHV9MqLpf1#ZhUOCJvSXbq=(i13?b_hlj5H)X!0oT08J`L=6u-GKShhd(A=4Sdv2h3 zpLRfoHT2zFTEoDt`QSkY;YmE>D0UjE`RS*jn ziT|}V2Co#{w$Kf=4{Z=na}wYcg@9?hQ~;{Ny%4cL!1(DgZR9|J$uLkqf4bf2|F>shG<)V+53K>a#IrL{Kau}T1|N_kqiCMC9p{c2 zHNN8if75a*$8DQzZv@(G3m3Y}xq~&Zk#6DGyL9?Q^0Xan0s_6QkH>!AfV)P=^F`DQue%e#kf6$hD;?PmyRD?NGG3|-+ge&V{=wb!gz;WT)nOoonD->~Z8;xxGR;7`i3uG#|-^|mhnYNrc` znzs{tH1y)v;6BmD6W2<;xklv-aN?4`6#Rl+}0;EkDXeKY^h1&8u~XYhMlkvp5poyWHd zvz!OF637CPtP}#ADq0#jgFtD^y2U9Z(e4Ak4x{|WFQvSf@~Ohdxw;Bu5BS>)fXvka zW|fPowl%GYl6z^LE__0bZQr4TVUk|v*4R&EwS7@h!K6we0bogr#=TdUNAV)Kbuv9; zS$9ZI!KTXo5-F{wi)>7F60`^j@11H6k;MS(~q-EXe4Zr~VXKicIx-0JnwzTVl+{{zPT{klUnz-EEdUYWJSn1_Unc)*u)YRNd*Wv$hQ7!Qw7v<3YF9%PC zVu1cGf`K%%U;(@yV*%RM0F3oxaR0#x&-TBtH>Urax<8&$Bsg&^rXfz-`=sDmg zZ4FSYelrdClD(axO_h$l2+?Ik{SE&UTQ})Bs2Z1weJLXkkjG@6LMnsiJd!giJy1h` zZGnW3{4=Ar|3FQ0{Worg^Cke)^&&9gP6tR^u^o^=89RBKsuvwVw4zDX&5-+uuH`N;fo0IOR~L#82yiM&)}{(^kcxZhZntd*G->QtRQh+}kCi+6 z;V+Rn;rAaZbhGf!J9}zF?>4purd)U@+bbQ~b%U6@Cj2(!{VE_s-uk66(kHvQ^0FuM zLne2QC#f8B3s^A+Sb=O=?94eS3=CtwA(C_=QNOZGrd0-WVTi673g8{|?vO|*vu4Bm zO@j%^)?QBH@Nd}<*c}D^zJ{XXkm*CtE76La>)^cD)XdUN3)HBBqC|s#CmK(LMI)CE zzGXmvGfVQ6kchl#iIO$C=i{~8zWt30Ivdn__r5towQs6Uo+^pJzvn!;(-_Ym8n@G%BSi{vu%^( zgt&lJx=)!?hU)P(c!lfG#-hAZ&s6pp6XQ@q78mLIF_!OD$!{w`?3ycuHT;?e$)n6J zJajI%H2L|eRNM@U)cyw7jueEepG<8akPbx+X8 z6%DoMtX=l0a=up16&&lol(YMCGTfoG3+{T|-xTa0zg|dBWJ4_AS&06A&@|XA>W32pg@LGrh-^_f z;Wx>=6q%z_@cQzZU`dcXtMuUMBvtqH&CRA{k!-UQ#h|RsN}X2=g7={J8h&+<(GD7; z2MxE*HkOyo)9McBmtKg>6ph~wWVE*K3-lZ^+86`<_=*j6^mWYkCS9>`orfmAV#l%= z`tP3(%-&u-Ge`+Sx?hu^GXDEI?5``wv$s{|yLTGQ*2}mG%lfUZ2Y??)bN4k=&3`{{ zA6f_*9V{SS(qCy&;@91Qybn9cky5=x3nR5-Oq%pX7`O|hI$CG2Po6wakD4S-&^Lin zAg86lMCM;V*8dt>NF3@~VvE@&G$n~r9Gaw%@eIoKSvQZP$oiAtq9>)Rs1vPQol4uf zfstfg!)+?ABO+UW{jg~JWIrxrwyR~>qWiH*kOA((rSD}ezc*FNj0+muK<*x}^}zg; zS?U+_Zr7frc;daLPiSH}zZdiBrFqZtf|5UdqkXO0;{?P4nm#Olvjx_96)?Yq%A6&${ zcP51DFWWypqAm6Y6@-K;DkDcm7T_E}wrOhhOMgjGJ#U()4i&GNl6>d&>~GvGSo^pn zwc=Mfd{=}yau>NdRI_Ph*t~hm#CBGEyqbBXU!%XRXxzORE`#jJZiD6}_ohNXU6jznkk<C5uh~Bq;oX-nogKbrTX!c>$sjX)Mc&b@+w7c%2tnqSt5x@} znD^NZJjQoBunJy({s&nQ^%DAzQ#%%?&Qi?P?#+Q3{fG1D$5yt`*Q~j-LumM!M(&FJ zD*#Nn2&X9)C+%%MPMbJWmcR|bTz4sb%oJ^mW3eGgbtdyU?vDg1-Dgx<6Eq&3^w0@BTT#tot{V^*PtBFd058C+*9vSUgByG^MJ5)VIWj2${5+ zu>95~_VzY~Lt1g=+q1p!px0=)+4k3G$uU0kFpt^2&%+9_DDZOlXM1b%mg}6qlpmb6 z-3iGGN1_>`@`b->h;gV!=XhA+$~XoVA%XccTN#OqBkZeBi}QXw5E5lLu-B(O-n;-8d=a791OF(|y~dEwoKacNM~Unc!@Sak}SC;LIE zJvdkF^Z4-#-99O6pLxSwsGPIM$87kyAAA>dt*nY^2|>F)e5;{tLV|s2US;*Y9w3zCJ4})0y*lTS@iG{;sN6sPk>la|9y`8BEb7}s&0y1r zl0pVC*8aFr>qrOt5UtSno9|j%R-Dz@<|t)BTDDRunq{HkT2&>V1_;QrGBTTuil+l}A?!9zp%CPp;CT6}{572t+uhzs3Y5rQ5(*d#H!r7H}DG zS4Hh|ump3Co!!5REWV%NR-+2w6=5eD;uFsRN4l31t`UC5XK5P6R#Rd_Z3w^e*AUHq zY}wJCE!DWGz`LS)hn4kxR@L(6#sddtffOZblz?QMew!)@3p?sf>=7p+Ki&xm(S@}4 zqpY|u0&1a}?`U_>%d^laj|CNaz(<+`A5=&3^KE~NA4DO;u>HP2OBO$X-^ay7(w1Kp zYdW-%p{hF}=YBhBQFt3Tm(sJze`^?6$NcHH+Uz!tR+aMg5PM8Ci7w31EXkJGQ6fzCW9G_el?ZSfInP{x5KlBRc3; z%M^b~!{>PXm9A7-k=gF+sM&s`V?bM|qjTksI&!jja+U>JR4TcGxucNYDw-ooS00!xt>g|!F2 zK&6z=O!PVT`|cmX&{L-m5H$HmYdwkWsd;0B(BLN6QA8o&vsGxPg~BzJaih-YNn2u= z?m)tYi&ZNm$wHqpsWyvSM*0@F`fr%y)7iVHy9u(?IXhWL$rgnaQbF(-X+>{!Q7?3Q zkI|a_)M|-Bxk9QH^;Vpbt6!&iVtRBY7 zPRp! zee_QLHYfh!MCktAV&J!$(~^Es)CC96Ht`<#4|}hk%QJ6w>GN(c9Q>%8G9o-jq>GsC zklu*)G#dXlamPVKfrsxk7xxlLIUSxi*f?LD#{}Rqt#rWPbBkA|`(8Uqeg_Uz)^N-UzBt z3HXNcxir8c7p-DwX|{R;vDYg>$*BTo&9E~oe>d7075e%6SM3125|b2gG!8Jc~S%uh`YNeXRid#FOxR4uPbOg zPoVpW<_}_t<{wZ&WzJeCzJkad7rzKN;Pxd`ihmM2g&Oix{-o|td9Ki|i!4}ul>G7{ zRjQr7t}1Qb+uX|FD0qrc#5)uqlx~*LWrsLD7I(GO|2S&;+aqw0_l2E3Yp*DdH2TM0 zH<*{*%i^;)-V0L4AT2RDwZCX*6kJ*90uj92o7+Lf_>4Woqd%j$ct7buW!@3HvNb>C zG>0zUB1xbV#r}<1H5;WIdU5ivKiMl`X}(DjyBTwRDI9)rV)IA(TeJUhIw3+!Z*0Jz zq`xRtPS%dD5|>70Pt9t=xCn32JlCsqQ)MQ)bI}jx0?W^%OzoxbN0V*rx|1(C8l)Jt3?F(;>ysrvpFt=oY5u57Isni7k~a z^?W$rq8B#ZUHFFl%$=f9ncMmEs2FE4?f8SRgpVTfr9#IbieVKrm?COtuh!n0i>O;E z#oU~#Bwv|@2!%5~Dg^1fCz*vKL2b~3R5G{4inX7T=Sm^WuSMry0#GQvx}Qx@TgOUK z6}WQfuzwq>tgj5^2)rQ#T@-Ho#^+W3hR7ds z3d|yd<)lkB}}xLO2F0uucy0>?aPOqu1nJ$7yFkA1+47fslG^qcX2)Y&F??!8m$ z@o1FUCUXn!4@drm~c;HYQpF<;kx!28P_>8qz= z^xEj|p~i{A_QKrd6Y3&@V<%z8{&xZy<;+3@9}D*|B@cLG%iXkygLUZolIZZ!Kw0AW z8+*M6gzZ~q-gfD0s3qh^?Ybve{GI;Z)|IQ9=|yz&pfOyP-7p z(w+?K)TE@%Mzq&jwIZq+SPDxilK#G}A3xU|-kF|wEANQYgJ%qjn3ZG7Mr3RNEja(G z0vtqeAkeW~CHh$!O=$;a+JaylG^|I)3_Ef-G^zH*e~Mu01nC;3ig?~EYrP$0vhk}i z8~NPpobC`CYe`7ie{FyLhfx3Wod!vZ1AQ8YXs-K6(iVZc_<*J}Vo3a-_YiDAro^z$M!?FhZ{(TFVO(-vyt*%K~S*#jYXp_NMY;4tm;ERz< zIoBbEZy>ECxOPtS$K2|Wqa6JWUZEvXE?Ga?gAx-IJ87%(G?S|3+P);2A)(|CBhGQP zyD=t7err#kklc^n%oHZ_dG)$gVBjjC&hhGpk_-&RJyPdE1uq26y$ zDPUr&sPR8=T>N>B9;yp5TOt~!5Qe=DTCUlY{Z;+asIs?aa!{$!_H|3Fwu_0VDcSI6 z{M@H@@{`N)+{c{6_jAp(G6^l7NUpZuv{ksI9lYEJz=(b8C2wAht{hKNDCZzso20U1 zaI-;DPf49GA4v4LY&uG8`#rDHdBjGFB#_d7ggvPU8Mi-at-$ybP~}tnI-aBPx5@MS zkh`apr8Jj9Dvl{y#Q0o7aklXO0bT z@^9O#6~`Rp;{3UAGa&-t=0+Woe%|M$8a)`S+pW_NBe;6MjQ+mxdHx_3D%g8xZF?>e zJgme1#tJaqZ_l;?E z;%$C%tyC%RL^`843?U!OWaeEf@jBHSQd?d(&1I&&+}sS`xy-ArEdFQ=B@*YN8~R+f z3>GLpQPI}t5ui_#n>`f~ zY7d#GBATD|$r_t7j?c8BAz124Ig$4KXK%lQQ_MBt!8zOU`M2S=-2oWoKMPaXVUnX` z%--16#GQMyi@jy|QQ<3I(skL#UGU2wD4`~O5T_#xM@(U76~P-`($f<;=;^#1?I};2 z*T@%5{wXqplEMp*OMhnH6qnV-|4D~H9a3F4KqWiv8?qJ_rF8OsJ;E?}y^HtMzCk$6 z0qQ?Jdv3a{PrAE{rC+y*vTk)Z&yxV#{Ztaexh0+kYkfKszzQkCJ|AL?{N5Bp7=BK^ zetGRks_A$h3;of6WHpd0<_x_!r zRyoqZ|A^u*Onge~7O_T`y)zLz1+Og}mZh9&gvTfyO5@~peC+#ve4Nb%VJ;sv3ah$$ zh|VAV1jegLA`yzu=pWTX@s*41x2d;jJ=bpq3KF%oOf}n2*B0$^Ny8LyL`$k)X+N|# z|H3vd7%J1}yyqG(+m6uYBHBks=umzcq8B#0_Bm!}@7L^paAoggA@3wINX?Y@eKWob z5-j%QeBd;lxGY5dJpDmjI^6#3Z1-JD_T*=A)pQ{Xl%&Tnsd;Ro)vE$)U3or}~u%^=A( zkQOQ%!cr9T2p5nr|5t{CvIqC8nxwMl2_iGnaKGS$kd)X@H%hUb^05;2YkjsFzyHQX z#3Kr5mRuQg?|#)7K43ShCsSCZt!M^hBuA8b8Kbi0j*3m3rCyAgD_3a&S`3IM3b!5#z6h>sZp+%sTrhAv+SK*cQ ztF*7CtqggL;@OM17t@Vlm6)5IYs7FG+w#hTr*SISt2rapL|^ZHV(QbPT^>~p5}s48-@~34n!VOfiSDJ+krbUc=kaxo;ri!QB1<_y3qa|< zi)D%%#b$-!{Q1X6{xcA6rZ3Rs_goM{^W2THJ{jjaS?fH#<*(1IF3mTz-voR4jMww+Afkh>PX*awugK zP1Ol6oij^F4AJZTf864{sJNpKwW4;L!IOhCR%S|i5(%9`qvuTDi4k(j#D@yKOVRy`%buvarjCNOe8;T4KmJ|XXjp4Ldk({{ork}Y@{?tE zgu+W%19-w?IM>$0x9lf`_PcLfGle~ap9c{2LyQrj z3!Ke@m-Ce3l=|4Nl2j*;oI!ADj&O3i8|7N}`ieP{8mE0qz-?4i;6tQItv}CnNI!LP zrvfmcPUhZwUx16My1;J^s|GPrJI)W{P-8VGmIR#;6n!d6V3@QynCF-7Xovb%roFQ- zw~moM@KwmWqc7vT+fjNwk4&gs3uYh+FQ$RK8$F=rg@BwL^Ad7 z-}S-14tx)4@|AXi@qUkLaM?}bj)OjNWc{PZ*bv5g@=LE;=!QdnTIe`B0vsy_`1eSU z&R>;1n83Z`GbcTsXzJz=@W--Ih);{uYE4_IL z8bTLfpz1eTFfA2>>xhcS*J3?H>WEEjqy?n!%Zd57a(Ft!(cb@}StcF6&V+6zU6xav zUw76riU~`pr8JH1R2iwJBP33A6iozLRDD>#m{ZBb9%nrD#OE%GCKolu>s^VADoKS54nAwF) zoXiV0q@f*--ejyI2=pdG*D2))IRszO|fl=v}?KuLjM$;%bvG96rb{ zcNd^0X&mTYAGmZ(nRXH`RUNT!v& zaWP9ony{p&r9m|Otx^7-&to&~U^K31|DTIM*upHg?BIKT^g_m}!2l# znA;LY^evmN1E;b(`8KmRSDeDpXx;b)x``Dra_Xag^!LHd#XoLR8Jp{y_Mfd;W%vZ} zk2y!^hMZv?*S`lDIv3>a%rr(C59ND2cPg|`b#2O;9T(0}@}Fubx(AD|N_gNZB)v|e zC)93&uhEw;Z=>R0CvmG96dF*AE2TM|Z`c;%)>d<)f6ko@`;+u2%T^gD{93h^FsHP6 zPH8ay*x+9p)+#~2nJ8egXK4#zT=gH&NgmcLwaS|nJdP#3nPyIsf^eDA4zb|vO(TSI zstH8i%Wm)QT1Gnk;hAt~?6k6dY+KC`r&)wGBWBe?EGkh`C$z4x^ow#$BD{EtT_>!N zw{vM(tU$KXStzXjvZB5)^5h_<_4C!AM(6p-1iK|Plmz?F(MF|e3l#jGAFfKtT#Gvw zcF{-x)*GX(~uFGg9f zS#VX5`T6ytU@ONI&u_;=0{;bO=xh1QM##;HUOtf|v$$ae975}_0>SJ8!9!lUeS7A^ zo>Ij-5yw)#%9A$(Mrk8?FT4a3vN*YrVWUNuksVAESkv#Qa6xHeVozyzm;(In#_@5wto@qf&>-5Bu7$;8(TA>H?U2xfv=J{x;f_z~EQ2^JSO z)8`d|vt2G_Vt>CNUz?|-Z*WbbxY3L~6G6 z6uI!rA8GOdd1B14w{KSIn+;1A^G6R6zf{_s#~5rPBE1rC{~aD6>DyYNltypi^s4DHx?v8N;;XK#ZJMtWW+KQ!rGkiTa!skZ#A$T!I_SuU zZda-2tl`h<#D_OuB#*D~4xXG+Ps^`_N_novq`f;wQDtJIxkv8FH11EW7r0FtqwwQ5 znv+&iZh4S?>vSA7CHJqs6-#CM`3u@irww5`j(*vOKa5$e*@ueD{(PA_5`z@7Ll>#S zY9eG6Udds0A6fC$l~&LyyuYu0?}~Q&^aKX$3z|(aQBCKwj*^!giWuVCEK~;Z!G2M zZ{K7og$9$caFUfru*OFgp?WQgS{up4o=xgX~{|6G!dFy}ayikvQ?MR;%o6qMxr@E#_CS zWAeXJi6054V{Sl_znLF@`uclK`u@l}=p14gy;E?$DY&-Yv4S^`KhoW#f)wvNCA z`=hZ;d0Xtl$~$(+lR)MLCL6&$4?FL=-lc1BeV1{iv=4LDz$OxSuT3}<@KR4)RH=~5 z8;iNsBg2|EjlHs1dEt1NU36*nH5c5CWHyx}9OCu%bX%YA$R-(@j20ZQks$`#Nf%<} zng62k^Ozd`3I7i*D)}2Gt5MGY|C&Xk% zoTE5m7oG-nO>e^&;B;g(@b(b3cj-?3ytP;{P4DeSWdw0sxO6XOzSzrAv&GSV<^T~` zj2$|ZHty~c3wQ{T#9y+}@{v>4Y4V7TDNmm4#tu(guggkVQU`sLVT8d;uQn#-i|_zz zjeMPNOL@nC4f1xGWqMxEAg9#1oxQ*`e+#E5N;%lr%^FXR!8|EW3a=}Yx0{m2m z_C{X(Wo-4^RtYkeC`AZG{I+Pu_}WflyhrdurOWq0BG0Dy$t}^(eZp z6xSRVx4}96Agd)=W6b>N<$lK&@-tw<=DFy{8@}a2gGc0SOP}03iV}X>=riKH#8W z@lM%*lLxt|DvQzeO_;58nDgz8-tXxjli#4IdOaIBJi<3THpGPXDuu6kMacjDh6*D| zZ^q1#z{MP7n?o=j=;(dBzG1P=0HJV1)+Mlm;*_}nH@vbK@Ki1Rsx(ycjVCtQy;Vdk z7CL*!)b=PL75{kC5{*^6Qoutr;uvlWmAfAk6Dp46Ef$+BoyJCR5krZsG}yGKW|}ZI z&m3AUQ>=ShN9l|N=aO3+;#SQ4zTwfKZ>FdH<*lV;4ZINT;Rw?~yuWR3{#v)KkUNqK zvLhU1W+Z@;c8pBzQZ!=X~xI+r9B6se0yzj9z{C*+G<;fE5uO7J=6??DVXwQ{nT~w6CBws`ZgGGnth~v=q z>dO?d)$Givpibg6qVk!rLfvEv^F8}YvJLq>gU%l1+*Z;)Lc764ROvz^k%T2zLkBwC z5dEp5^Bg*e9*oXs(|6wSF|59$MS+rRl_LS3&_!2 z?c@U;MQTHO@Wc+vI7PNQE_rq|n`K{uD8bzUjmHyDID1R2WwF>&jNL}~B;uU?#U-`x z%uXuSXnWQOdp&5*2$@F-J5xsA_Ng5+iY757I*)=zhoqiF|Qr^V`v>o?-LQcb9ZUZ!q-0FKe z-EPi{;q8V{lGtSZuW=^EOk9?Z>2^$!-=#=5!x_BF6p?gHy|q{_*64e78U1 z-etK#6YT_fp14KK@r^LJk`%L?gK`P-mIlQ0=81_jGMj&X=PHj)#nHZyMewZ$JLVK} zd{k#O@wf>qqOOydH>szt)A5zoW~&rUG=+wK|G}Kj)Hyi596>DD0TN9jE15L=8_=Yj zJ*u5Uo|1MDs70pvb$6HOFV>y1R}fep{o9?J@Mo$ht^(WnVzb*bxN(uNI%`!C&o2=P z2NjLQOWPkAmJmA3P?7x$>;KB%gOND|sQ4$P2NvN#9|~f%xBZV9eN-C45rx_%Ioewy za)<~UiSm>EnTVMSP9wMVEZi*r?u?nwNYZLlJPZ3>zb51cA?{vIHFRFW{t37s?aTMp zWrn!TN-fs=XJmPjX%2hAIP~rQF&2S2;?2I_=YG#naD$EQ!Ju}VCAWU}lQxXe@|V$DhmSPXDlC7s zwf=Sue$Kn|yNQ-O^GIb_dO<7V?TU<~KKd@4x&VTklx3)_vYJU6<+=vg2=_WmFq zNo4sQHRFPu$4gWTCU-A?--p?$k3{^_xykL(BYj=%p1of)JY5>BAB~=FZop{%>JNP< zPLRkwAD42K$yxo}j6=zX{rP;Z(8brOcL6P*CO7Pv27+YWpKTdaCNsC^&COQHOP@^< zzxB#Lr&BG2bYDGvT9(7MOWVFLWD0~Hf^~x&F5^XPXYjJEp=qBlZ*>Rf7Z7K0?ylAB zT&d=8M~dqZ+}S+AjruuLf4{89z#d)+XCY8sEL~H6td};qaK_)+*NW>=-M+LG^%*eg zjqJqqcNMpJ*~h1lAi|B2_ zz1E=Wt-@36$2y;uYPu6saP)(sOv1Rgi6FUe-koZ=6s7#0v6s;x^32qtOg7 zf2IE!PK;!3t_lU-oPbN~0YBM#pxAa!JGLdTF|ujbH$7=C8H2D}^n&^D`{T_|GD>WH zp5;q4#kp&xp)k-g@!9L2=hb)tc3lr4cB0qL|YA5C1xSI zlpXXZ&0-<8l$pdaqqDZ9M|Q{S=FybN`U@8yoM#FtVS`re@}4)-?Tr0ko0Ov9 zw2$lMTuKOz=tF5@8~ocCmp*+0=6W1*)CZuU>dHLn5wVD(#uGo;oYyBIJqA{f(lLo| zhw0zCub35^+dc4;FmPgaH&x6}lPn@4W=H7VPk4NGFT)p`rAT+}=_&=#L$;rc*7xM@ znpql$Rkrx)7U^CZF=U6Id|XhAAWuHQFyobsG%5TFym z^YTn4m9J3pbcrY7j&OJisv5>;d(+t1eY_3;#T7iUvi9^5b(($PwN+adjD5sRI)MzV z?z3r@c~|!|iIf~#)q1EI_dMiHL@JH1Eb&JFHa?HVKHH*bsAS{2_n^1(&98T-yVN&Y zY(`iw?a^W+vAN3J)snChw-=2X_*Q7rBIFBQT9w?~RugV%cCxZZlfRtm(O|foqasK^F4ep|dX)2gw{i9$~I*!QC!T=27t^j={6%!UJ%h2+32@a%; zEq`~@LUqaJ_AjP@LUnauf3~^g>HXDKZ<~%oMjuc9g%R(&PQ@S4YLShjKS<@4N9jGI&6}=d6`6 zWvuT6HCpGRzUR;h!szT`Es_`YR?43IPC3_!x3K9DmNveAzkXVBKyT2BJDQO=``ubE zL^?Eg#HlQP-%|Yi-S-K$BLB>G@}Kxzj*>Y0qs^rff5m#h@3gvvDCKlM+cWQ&uuqY5 zr=!3zc+MS&K0gHRbLv3g8YOszS=Um0YOh(&4?trUYFhG>Wd%{l(vw!0GaP-wdq^M8 zywm^G(U_OD11DbUD{tOi8gH8VZ_aQr@AyiMw8Ngo6w3!pyl+M(f!6sag3cZ;E~rjl z@lu8qJO!4LvM^zpH5UdT32RlgPZ02-UG2x97s4kIqH>)mXvDmb}}D z!yHJhg$Qv>3|lWT;RSpMcvF9)weBFsH*P70nG*3fp;+ngI&l+svmWFE95c_=tBgT@ zjoF^XzFklGUtg6gAhh`+7Lss~gw4U;?wu)#6I6VDjh_!^uSucA49|QlzreiJLhDb| z)!OsMIs|{q7V&}{Xk$ZAQTo%7X451ss3x9_uzap+29@f0cMs4=FlO`);9$ZYkJ3T*gKY8CuQ@Kt z2sy(F(L`|5rz)rS^;~e-%)UlqcG8e6EIIOQ&83W>kREJhQ+Pqt*T(QoEcNsIrz9yM z*pz70Gpm@i+7rFhEX=mipx^HGV-&B=`-^FLDD(G0TEv( z0jx)=Xj7HVR4r)5&b+9Tg1oDxudPlO&9yYASAi>XE)Iafle3 zgWyy~B>vKzzJw70RE<&Qv6|g+=x&uQM;}y;Cfl#66#E)U*1Yme7->K4HN}hq zJFu~}b3%0ONMoFzyGl3?>N?yvKNR|7>+1e^N+;gGuGasC_}coA=?D&#by(&xGZD`< z1VPknLB$G7B-#(rVSm|A;Miu)ajUWHbfk{O)L})bw&m{*@h8$-t?kSOYbt zkB$wFvw$L2!WEqft*JJq9JUyIQ$hgC-N9Z5_!p+5puau(At8)@7f1lQUBN+Xo(z}N zifPaP+j`+NNpHd24gcPsZwi)>d^w5^#?pn0CyZhqq?hZwZyHnh~xVyVcaVW63LyNn+zU6!G{g_PT zCNsG?_au|ux#OZ_XSHYpq0-V@HL~W9pE^9hXY++N^1 z$~!#91y)(tv6@(q;%gsl+2c7W>lXtECdD56>>v)xk(MRl;e%eR)wR;|C9%O1)OB}* zHE)E|)>=;g?YC_L-G8n1SpKD%y;nm)MkUW1>mj|}$L?1!k9@Lk3WkxrZ^!Q9Wbacf zN6ChLa*Dmd$L?frQ(Sv5i2gWbTL_#x7b2>HUG#ifW+hafQDjRe$TJ=%m7-`n)nylj z3R?@pS9Yt+Ha+8grL6rH(iE20hW7Awn#xwX0j>H9vm$GUV4*GQr{?l{?)ZV6+MQl^ zye4%}c!%}%Tr9Bl_?#+b{f)y826pq5&YY>fOpgZd1sYLCtU!cG##b36fNw4DfNdi&~tdC_uSgky6v z)XtY^lDA@MCsQ7Y!DFav@rBERjVY`R(1B_VKjt|ie4rYBGCg8Hd{+|h!I4bN_@}*~ zVQ2_|Ta(*2A#0@*N-1yI=G}BxsuknR!_Jn`623hZjhNO>t}^ZT+F+7#6uIabz2K*D zP8)1*`_0`&^0IfnTK%GaR>>F|>$*xcYk1i^hk%w|S_SlxpfkDT#j_xtFXpL3DgG97 zxnc7?aYc!0k0bhCkZ(6^UFBJf!~R*Ux7qM`o8{(@sAk+U-r(x$ga0&nOXkW+u~9Gl z_LD^e)e&Zj5rU4*W&b?ljX#FYW#PGpMLSmi_UUx1OIJ4%q>X~$KR9Wjsh<9k1l>Et z(U4nP!Z;Z>x#S))*_n7Xak6A1`c6`dcL34qbKmz%EpBoW=L=BySp7IL;2UG9U&i;W zMCItTQ4{G8c?yAn3;Lq~w1y%+#12K$^4o*`c;eyO_C!U_gArPz&#PV!?0o%D85%S%CcCeyH@h@&Iq17*a^>@Nk!sOc0k?#}rA9p3~Ig__wK zNZfu?zKmP*vsTQ@pWxf8CqJZENv~IwZv-8h@gY~g<3dBt8N{n2oc~PzFr@6l)vM6| zHiAK=(FH@vMo+1Y8=dIb8?N~-+hwG-F##h_>FnAgjggeQWdey}+y^{Wvnna5?G|D>an+mWW$DI_mpLs+6S=?lMH3})^{Aa5`r}aF+w0m!#FTbTDXzynTD|SVlSkiC!jpyuLU0koZH4DiO(@Br zqjX^%J60W8&_NCge#vTYZegIs>HJxW@Qmk{m7{n@&6}P(0_35fEup_*=1p^4CJ;&g zf!d}q;YL$b@BA5o3$|TeAcPhM?iedg@MSQcbUk)3_e(u;BUEmPev9zOuVsyBKSi_k$8E8 z#9mCu+hG!Lto^Kziz(oHG%ed~83&7?`Dmoc7@Ct3IaCU>cNe85&WbQ&=FO;F=cFo6 zi_~J~<%+0x-j}bqCw9U|cyhw)mL}hQL*z|Pyko257awdh4XQ5uGHI(XMD12vdn{KF zEqotVc2PV?!40@x9>6nS7av$SpQNEky%NK(&7WO#?o z?T~^4f|^buIHSJbgky`XdWvNZG(m|?h^|J6Wll6f7wnNS{It6HwJ6oZB4zUyOAz{9 zUW`t7)muz=yh&XQKdK2QqGes2O2V!EZz5Y_rsC(CRKNFO7MA z09XuZXGR@A)Vf8=J_>XRWu&sg0M*_6c1Zmy@gb5Zrw~MImzwcB*UDbmsNkm+l7MmdPo#>ZVI~f+`K|O-h_M@*wua|k|@le{YJFfzvL9@kmJzm zsK7S->z(fKp2!(Z){b?tPCO*otJdyinK%e3tO@2&vrBU22aBvSQlJOe0y8W?Mz zH|XimIX)}f`H5@U~B+af(op_Wf#Xp zErawPM9cu>eT7a5H-@Bz0#ZUxk^||XQzFS=T~Nw?(2}&WH6W|Q=6Q>qXda%S;IF6J zTw!6-RrLO@XbY%3;-`9ZE0{KZ_M!5{oH=0a9+D;&xxLd{z-b6~E@Qqk`?FqKM{XXV zH6Gp`l0GuTfB^yP!W587RbD5Md-M=_X%tbIP~k93v0QR=sz~v#x1o9YD^0+%A0dr#K z_UtN9C|`*n^?-vM&ium>C8n^v*}6qOQ>_7Oq99W`eMiqnVQC7;Kan1kM}%tdh{{Vr zWH7Tr0JO4Ujuyn`sY7!R-#Q>^)a{xlGDq30GSXS0U8R zfL|3AUt=sfGtS^!L0lCbf)V5EYcd$K020v}P-gT$nUiM%yvrUTX87e!a&4i{^0)m8 zirFSJ|BF%C@OCRP=dm#QL_$&ECt*FpJwu8fu(;3z5J0jCB0wtby+-aaa0OPzUm3w)#lqus(cbu9 zjt0%>OoAB(1*)~CCstr&vTjep4F+#nson?*aM*l%lKj|b4jPEn>ypLQ>LP6Y(XSu$ zlJO8}DV5;T-XE;|gWd!4e;DF%Ael#lM=sN%a;)_c2~?3PedMl%JDKpOu7>}F)Zp?s z-C3uDRU7gNq_2Q`#uVklN0#V3=jDP%9%-glYm8%D z0hlwzLrdH@NeRpJL!M|q=(AponQ7Jvp(vJAm~K$@bVZ<7=rb;F8hOqLh|G>4ePi=Q z@pyCd5I~JNG5|gXx8Eal;}6skuF;qNgqk%%T}4&%l~MfFSxC2SA@RE+E?_XzQEgZW zdK}scZ%O_>se!PK(=4Vt)pYk6bH}j5n$xMm9%QQdw#r)qR|%^BPi$?lN&mBS1jCMR z=@p{l0l;gWmyz2D(ae^H^}UWX4ZSP@3(jmNGtb|mEG&NlZgmiJb9T98w@irgmWQ-K zoFJTUV+FmZM*G52im z&i9~qRo7(P<8L7U`QX}eq=&s~5TMB_$buPjlr$}miMc5}{}<%hG!UomLlOvum_4e* z1n@;13Xbm-5|=hguv*A zHwq4T5xLy_Se3qzytOSk;62*1FEr+uo5;FFi7+_b7eto*B@YGR*!RAsH2Sl`ec+RV ze;nZyyum#?EOa_nVw0yS3!_b=TeCEi)K*{@1?$F>urL;3o3RGsu#_b@Pm?JL-hdzz zl2^;z6p?QVtJ4Qk>pN&dwrYgm$l?GQUe9JEkivFmD9fySg)-Xo7QIO*KaKTSU7|Jp zkW_+6`~~BgLjn%FFkUA<;uC$rd9`dVD56o)+vp(O=m`V4)F%D_S%XqR14J5}7ejAP z)=Dy=hEPz0S?QF?Al2g0@>2)AMxDdH5o!!OMXb_jmA-W)$Si6toHO{l^ST8>A_F7Y zs$9vW>!&n#ww0?~<^`v2yzNYy)t#K~;b*i9rDe6lwZ7l3g=A$LqoDkSA+0f%R!UM` z?WT7vXw@?bt349H2CFs@3EAM0mgm6Yl6{VggG-!WWAR(>X~jwHzJLBv9F=#Sfx zoqq?JJps~$EZj4vs5^@b5#RuH`zBxj%`yc%Q5;m4Z=jm@b`n+?OL3HY66B zlk-8*JJ@D4SJvbL?@?`4YJ|m@_6BhCgLvTDYUsCw3(6+{htz_Ayo-uhbkQX=7XeU9 zeiA_SiZ5n|ltC?@2#UzNU}4Y9UxTB?eI~m95EMUrL7X%`WDGhtK73jJAUlLXL}GR! z2B0wCLVytD!cp1vD+!WEI&38fOhixqkm0oS9Fy)hh=MldwWirrJZvwDYAHJUZ(XR~ z%nQ8Jb54qc0Jms9D1MZ(aIbkoir-!wM7}MRZ6XEpLxTUx z4c2n1$o4f?kPO*^5>f?wEY-~aMWw~?2bu~nLFhu;h~P~9K12h8RPaN;%qn+DN^v@a zRIHYKQ-oZl6pL;=!IeWn3Uf8>e-%~5=ngW3l3bG|dB;mLdVfGzm;S`k9?4^Yq%{%; zjVOi-o#+W3Z))DpPuwTut>N-2;$`QFgHpH>z-io$fP4%%oj*z>2q>UwxG6B70+Io= zS{y1pL3?gLPG)J!%KKy@kBB_JHosG)Y_`w2411UBq#TN78`})}s^Xh~0y!g}I*5s; zL93T3)KL?1z|Ui9mZsP|=TA1q1n7ba{~xR^acIv!t-I-NFj8hHrqDW%QAo=Z&W`3e<<(7yyBY7qqyxLKA&Jy9EQ;$T|6;yX}kv=!Jf+1xgp@XFd=Q zp)h+16;8;pt4RwJ3xkNrTSC1KSA0IG4b=U5Q$i7cqeH~dprbrOOOVgyJ{#O>|F+u?1Sh3tl5P)4(GPnI1XdA()>xe&d$AkPJAv_Nq+O|tIO&{CnJYoj zsJUp96HTZ8300BkV9!aoe2Q!ni`}p>wEid2OQC~2cir)eVHB3-r`4B|J@@}4cnhG~ z_v6e7%@bqu+L+(Zf&m zf)%K?{sB-z5d{&V2$1!2awD^H53Mn=;sMZoB8~&#&D``WUmY_0^GSz3=-%wmD|8Xn zg%Wi0$ed(NT$NooRNw!RH+Ga`L!xd&h*Z8*?pu9=h>~NjhW$RUUEvu^tk(1yw8C~N zSYq={O3IPAbv>0t&>H8ZT_$J|g7bvLDGmhOL1s)83xS_~d=Z($U0s%MCb_{)34O_h zT}X4|yi?`N3z2XD=t_R&1yaVkEYNh`f;rTl)CvLAVF`>|E$j!a%~>ccblrH>HVgPlC>l)s z&B9cY6f5s4>>{NuiKa8X9b_#AseF%EtIl+bqKwaIy46eizBGq!yoe*;3~zdBj&%#k z&uyoIcJ0(ZYK@>aEK#d*=AKJGg%2g0qSc5PD)n89$N(H$A4JxrLUgm!dHRs61Cgh3 zFI%u){G!gd9dUUupaH6P#1c_705EMq>ywF80S^p2Ij`#pY4I~m#2+#%q$n;=? zkS(mTGjyn|KKB7)$*r(O5183=^GrYQc!w|q+4xW|6dTor9#m>Taadk;t;#*-qo9&r zp@S+Mt|NkCDa0VtOv=A3@_I^Gbzpu9F{I$-P!MP86_ND5C-ZfO%{M3MooMD{C2m}M z+)=HLwB|a5SccM%E_Pq zt77+?ULrT*o(eRHh0&C<%(zBRNq~5WeqqFnB@zV9Pa4aF^Wqepz|~PipZ?SZ52Qd? zJ_Ta7B=8AjwujWNx{cCvLc#A^uz9^p>2KG6t zTPnj$8Ihn4y+9J=^O9KHm79P%!Y`6NQdUe^5dbn4mGL5(uXQzsI73)VZ}NjtHoxf`cjQML^_ zTL^YTjXgNa>?Xd`tu!;w_ChFTz9`Fb=DySQV_PPExmHPA&w^8wy?4G<@_r{TxSv~K zFP@Yj=kT9N*w1Y-TkcS85g7Q9M<=r~z7vLPb&}*6ZGa&75!MJQ5=v(3oV2i@mS+L? zqeVJO=Ip*M?=hA4u*;?kTLO%+_ir=)0U1e!Es_QlU24iqDsA6_oKY%c-4U}J<`ZcX z-UP*Ajij-C&&8ynNiiZsWS(@9l{$`DjE{Fpfq*~(28I2!F`1&ToJ_A7l;Zre)y$$8QM!7Sr&X;%p|T$5P> zGFC*E^38BWQMn8ovCwanlWe2|s7d?y2_O*!M6A%JwTLst?>wT4qGncy79Tm5Nz>0n zO;}^kIW}`acLGMLS$Cja$)&{yf=the3WZdHyHaP>n z&qB%y=nS=VFq1b}J2Zv`S@fHJ0v!%SSE1MPv5J57P0d)C{PcM!N2q!uNwrNzPsZtZ z1i#qPx0_uoF{D*7&tIL1T#pFUYQJllqmLx8)%Z?24dYSPhx0qm*d*j=4dieSb0w@_ zy4Pykmu)8*4wcRg?k*7sh9Bgb6FBjb_%k_s5&JW_cWiiOe%DaU9P?(pcgO3f#c z*7Blg$-T^?k9^HRpfTvZs4q-yrx3?x_M>m8o5#50>1aWjhJMJcCNxf9tomP@YF49s zI%NGzY#vw-eyY3qG+lcUw?U8d*-o0UW9nPwA`gWj;H3RB6m^&_w)Jo*-A!vLi(s5h zTqzA(XIp|VW))hGyd5#xH6Kp6u1ipsw^vKjtr$D=%wsAWo5bNzl>AdtHe0$N2oEli zOrk2Cl+;qDiSp+%6_MCN2+_evo*2uTAR@(i{#`(iSDTW0b``tF(m$GsZT`evuhmO@ zMw-aWs`fbgsJithnW5ssdD5Rsd2Qx1*q&+p7>))OnINl+=2;3pzZqOlLT${h+>>Ae zl%WyDEht&1AiwCK;wv(oP4omD?W2SABroH%bN753-9}kj=ZG}eOjB2-DpEU2q4%RTT z&+B0Pr*taxkbBG9$H>xVqv{R|2Q=W8guA4Xv$XsTX^-D3eP|B%?eOkp0pBe} z0(n=}D_nC*t@@uGg71y-MMI<^kE>uGUpf=7^}jk^*qh+|?FdW@ z--g1#ccV*EdN(S$LMS)6nvT0y-6ruaRY5N?Z0BZulSbfRgK@nL*-$%~uP(^{qHK>9 zvq;>qAjG|l^zhM=eut9FlHl`?z2Wel@NVZH1i|gl3#ZX-P%9)w0;%!`A-Z-uCJG7u zcN7P+Ne&jsrC?4UNo#eEuGk0zkH(;*##eE_?A?j2N zi`hs-=Mnp7GgSbI4h2iC1RKfU1NYv)BqkqS%+Cixe#W8veg`lEr(}j=F+J1oy3;C^ zg>5-~gV<>dlkJ9LZ>KLJ-2%|MxQD_Wxn?@AF%lJIDlO$tD_iMI=QFx;w8r^mKs|IkQgLghyb1=D39^P&XZ@Z;i_m^Bwl~+WA4ni zciPYD`F?)$a5UxSC((hk5zN=?DHY;Iv={ohy7HG}zRb{RM^~9jO6r>cWVC}J{WsuO zvUJ-wq@66*yV*h9Hai0QE##PdnAj`8s&g-Lm^yULl-b`|)*n2qqxF^nh4C@oRfvha zNr1#uKq6UifB-~S=B>dmzzP4?6HM+k@NrVAkdSQH!;!3eAg52tW!&MzqmeH+*k? zxS*p>r@p?Ha*C4JJSRWczW3`I!mi0nVUlf2dp%J_i)&< zF9M@zVa~uI#501|FB;SM&TaWU7*Uo~_5jAaBJ(GYxVdS5s3G8Sfua7N0g?fXsak}) z3BHjxLx;NEg|GWqz0l$dzdgaVNQ~^?Zw6B2?+lUv+uQ|(7qDdu;zy%K6raNoUs8EC zCpcg2B=(F+{%iH;>j*iHQXwDD6kLB&=K>UV@XBE>`{Id#cuX7PW`Ye7tV;u;#PbeL zTrbVjR0%s@F}nDF^zr$$1pHqi9s;%MsdZ*R|7XzmQgS3oaxt6Me}vuOb|I}GjJaQ} z&lm8bsHht-uvXNI0RQ@)f1jR09pX^Ke4ci8WrMQYd_h8-lzqn1tS+S^3`uMC)>+<-l~CYzrhAu=$X zRBoO^YnTOI{@n@PfPf&rDq3&lkWyEe^lIb5wi7cmf(}24185E9G{X5KG1ULuyLwY> zC=8$DareS>iCFy`D);dV#0cj`@;Z#qaqIxUi4`nCsat9!m^GTi9XDd1oAEI1iMcb$ddT>5pa^4(n8)S^}2 zPceqY_hm~!n&nCUoz@krg)l^dP&c~z5VYl}2oC$r*&8iDutHscS8G;;W6(kB=A z`>){$EW9E^o`F||Z}$*`AtKtUsOZz}*|ul0XwRxu;KLjkPM}^5N4pv)^e@%=<%XnT z4RWZs;MeUw!)C-txQ@5lWB`jAV4!%JfaoBH#N>SN&;%c=yK0Cf8ZQIcpV_ zv)YW}TDibUVOmkHLS6Fo>SphN*KkN)QX79n} zhfYVTLh`@p)}uw4nPundpXb&bOj!pf64WIHl>X9B2bXO)Tk<`QVfSc|lu_&vQLZA9 zURd6gFeCk^)JLb(k`E$u=HkDcqu3|;ol?3h<>vnEW^9%%T~R%9R7iMJOyE2mL2fWb z4#^9vWQ?*JI$h1mYDg)muQO4$qUS<4HjF3gzqj8fUv&l*rW=;D|<#I_+67r+?|@E{<-fere{xsbu4Vh z%Gj6ipmKl#8dhEna=L7HO-O59Vl2)z@O?b5`N+~RziI?Wn>7M@_&x15L+X28MXY`W zn%zTPCko+VJr63dzd(78=}P=uJsVt#c+5^n>+)2ya)_oRqE*AzsGcyseb!*Ub!5Kn zkxEkUx}y_ymAt_TM$6;-DqCE~Rga;CV;aR5$N|O&$e{f#r6o=o4S@fyR4#pOh%d$g zNywYH-U2DQ$84&E?5w)?_hgURYQ%EQ%-y2pc6gBrMa2}Vb2288{j{M}rHtO{goxb` zPkl<=sp?28BxmeUL`9Td~U>p zkQ7tXEr$|1R*%ySkpgfn6e2V@wM!3GMI!_vG-3Ozp-sCQUcs} zr+?4LFirKTNB^@dp|k$m5UvcKCjP1%c%rky+T7y`D+%>u`WJoCSG3<_E~im#?S>5g zQE`lt)bw`0lPPZtl1CWD2A22fGdZq1q^^l#x~8Q=UA=G|V8>Co0BoU(*2N25Gf0L3 zw25THN1+3tKL>=l;@~2mFftR64MT;R4#E=46+_$LSXA~DaN3pMKD=NPbnl}`y)di~ zz&32pli%-OX+SBjSm&zK3M+LJx|mk_D&+Nj?IY_^8TLjvb7+=F6btJ9%H6(@fdfo< zF4~pvRhyFm24P|bjht*MDgmu}Z9{&u3$W9GH8a7YVp4*|Uc>@p`cYe0!1Qe<6K<>>O?YM2g_zf;^k&cVMM{%Sdw{M9nW zdWHm)+az7cz3+&=U>yZ8i9I{zpI#fLiJdq!WWN3|J^FMqoqyQ;Hifr<<8u6@Pb|8A z9dv#8MtPoLD|dY_2^s=J4qVl!v4~50LMrWhtMcOx1Asftg5VwI<3c$Xs*b}e0U+4s57`|Gdd7u zwSAGLxtKDIHvM7ErhImAzS3@0zCOEsF_N@ewAcMlEyG`{dcmJSOFv9?SZFoIV6=|I zw*=GJN$3Kq)GrTDVe=_NjHRdJe#_^9HT1XAXZkCnPY_=5e0zJv7QM|soAZgvOWu&V ze#Hs)j;jh09n2$WNoQRfX9*A-2uNoyy4aM!xG)Op*(j>@N5mig6CBhQ&?+I zq48HTpXV|#dM>DN^UzM*@)ER)zYW#k^+VK^2w?kRHFI&p9}QL;cWx7IDD(xbPww{x z-_F+J4RALrVOn&W7;##(R9&Fzz)O}yRsP`;V`d(IE+_n6{=BY)ydKkLyZrEm$|UtY zR))U&ZPJRpzKZJOmFNU+J`nfiz(!Ta-E4U?&hI=a8uvm{$h-3F&8l$p-OpvGI`*Nk zx2~61f5yUJi1$r)_QU9-_&2Iv2&ee) zzECG*0z3xt38Z~lIMZbyZ+EJs^TaFsr@dp)qvH!IsVm? zR?b56W~b^U^GbRiIu1UkjgK<^@R(9E{(6-@EIn)R?>bX0199<-(QqiLnW%vbY z(0n?Ew&xg#a_!N4R6yb}BUB5!g-d`cSG67fHn`DaMdvIXO2#@ramXPndCQgF$r4o# zm}Rwv-$#uf0~?urlIr>>eGZgtW+Le5oG~V@@p*+>g!uCSY#7AH__9KY7;pN>evTA4 z33HCJK^Aozn~kCB^VAkb;c5yiig~1>merwc74NxnY%wTw;~wu+?=4>kY>~Iuo~_<@ zzjV)wZ13~-wf2LYvl9P0JYQ~6TdHs{Q|Ol{E%z0RCYJna1Y`e~;7PpDz>#95CQ~$s zAIfr2{(LyTaya_CjH}yzS^xa_W6*>wNQXNWhuOA=zFtwoWioY?dz=eHR4R&;Y{7Iz z5qA9;NNr2Db|7tg$ha}B`eFR)$z}{0M|ro1?as=Z8DLKzU{8$GAt7==g!uU5PjS04 zeKNHXimUMoM%^n^ctcEUAU}0LWL@QJE|X>q>qM`U^!s&6&9%FJ&IXOsD?&0#`*^OS z=ag||ctlzo<=6wGyFly@A!J7(Xr}25<;&;BD19aW-amJ3_YGA~NVRF52m=2YT^`Ig z?~2Ihe=5aR=aQ^dt)+{$j1~6^h%=h2DY@&ZY*@9viONXzhxzy0Ja@qA#*i@lgbEwv zieJxn@Mne<8!T|%{pKjb7a5Lj{VT{TDewFVH9rTstip{#N@O;%WzhqSe5w`p^3b2; zZuwIq%JpJr;IZ#@uGU3dr+K&Lu~!;)=JD=4O%*e+DEzr{l(}Wz^n~l^_W}3hC7u-9!Z$ki9r^K) z>#}#A^M03t>_URyF^II`Xbw=BXp|=WOS-UFv?_eu^}Df~9ADobrz1onEOJN`@v}YJ zb-SZ1hSu~k^S}04iTVVVR`a+?|CYj65@`l}sH>r*iH3l$x}25`1_~2**Oz)s zVFzB?l!k6X^<;%yh!Y$d`L6g9kSG~Ne5U6mUD@@=vsF_bC&?n&ESER@+4ZL~Bu!+$ zEtViU9CJenA5TTD-PU@qiij0crgN#`kdLqf2^@migxL2*9x0)SN3lrmH$z~qwQ?Rk zhAn^HRJ43Wu3Pw8`l^fMJW*1_Y5w-zX~9W2sFHb-`3bIM+0v0Tq(3s~VhDz${K}0O zz4b?7S{wg0oJyodJ@$`BltNOfq9TDZl(H7(+ARJEaJjx%3PUyBv3E0O@d1*I#hKY0 zkukX6j_-F^t1T7PsT(WOy)_+pNhEy*Oqocg6uNnXp%{*Wh}B!+CIgT1Dz@QFB=vy^ zYf}b(mndFrw?r}OuBc;c-+hRUeB$QZXTS539PJx4rd^La2uk{`k?gg`5ieY4#_e)t zF^-{+_1Aqw?x7qCJ~pZN!j~(4J6##eXGxmKo3T_WWMUGjHY(aOYG*v4OF-$Opon{> zu@O&u<|oA;Eg{RiiL@*#Zt}LHAI-|vs@Oh#)hK=?Tz^-ffps<>n_r69(!t$If(M<3l0Eb;^POqUe?)2U8@z*R-&-eS|xRaLWkVdH#p#Q>3NPU&AA#*E_^^P&eS`SnfTgKF*^l?R-8E=lB_g z$-`Fk*eG?PvN}@0I8rKL|u z-}!CgFC8(ffFf|cr;_0cvA1^H0||Gd0KQrJ^~Ql>vRAMvfb!*miA5TZzA|r20^>eq=O4W{Ap;FhX+FoK z)7*t%+Pi<`R>)b?gW*L+KKUa(nblq&TPL0LVtvL z64o*WPFK)(bSJ{cOi&H7!}ZzEI&B5_bUQ=bGq`APy~-2H|20SPh_*62VojyAeO1S~ zps00;bZ!} z^W(?Q-kDlh#(SsFp8E%S`u?0?+#6SaZnEoxR;JZ#V^ED8PRi`*6WkBhK%#jcO@ zABSIB(wC>DB~3{JI+|nrd%hMh+Fl&>gb3Q>JQ4i?!xC&^^<8l+X5}FQwKB`Lq5o2U z&f}3>eFHVsAJ{5m`P6rAajn$|ZwqLrYM#LhkE`nG?ltsnHSGA*3uFH=t^w3f#j(a~ zo(<>Vn|ODtI|i$G|Ea}1hxY7dqFvM#( z9_p^`+OCbN91?bNTosI(WdVVJP#6Ze2%J)EusJv-@^=jwp$<2JrSdGyy-*z7`cu5A zn-g~O;@v;8UBfIt%umQCY_{|3SN#vw1Zf7m#UFUY9=0kH&NU@vIy#w__R%L2Sf?ra zm?TL5ZrvvZ&~fB7X43M+-NEsTwTd6tp^^Q_1C@U^!~u}^{3A}zk|Hf(~X$T#ydYgRr{MKwyoRCP1W1)vuTWv zaZqsgcvH20cz;lD4U&jyw)f8|{tOPyT7{-VYuDfTTYb*%4W$Ou0I+|x_2RS zZ*f+(3jf+}-C}fOqF+xmxHDK9D%7FiUqVsesfpFFKwt8JB}$%&R~51UKDx^aU~>2B z{vp05%AIrx1?@u-XIaPS`KP$aEyUxgGnH)8Se~u;6o4v3lt|*wj`(VP-7>K{mqblZ zQ!|jbE_L?H6&~iB`fI%7AAa(rqngQh6ymOg?H?5Ec6*K8@dGqp%3+P6Pf!D9ton)M zd;gu`KD8Dgc|h)YKOgJsi`!!Kuw8&wglUo6vjyWmsw z=y%~dLlQ@^V39i}D&k(u{+9!T=&ka0TIyLA*CRue;-M|5WE>iM&2TLxR;Gbjq7NB!9E;|Q_AyQQHPxqU zSu4H?f3jIfN=+c-`&wP|_&toRh}x9O`|^Z5M8BYFH5(gMb^l*8eK7KNR!I*5cmjNM zDgqV%iN||MdtfJLyU)PIb*g+VfLNBL$BV)=B1zDbq`^nk;FJugy4Pwj%VRl=c0EXnT!sr^@hYMp6dP`vVS3W7*$?J5MQc+pGKAiN%kmFq)(>UuT_{`> z4nj>l7FN1pI6JOCtP71Xkoqqj3+ZeQ+cWU>hhrFnr;gT@;A?q1!%PY%-fKC%m|V6} z2S#ETEAtO@>|Mg~34=fq%(w=wZfT~4q92<00*uocn5Q7sjCy{Xj$@F7X%T3kMDCmrQpgngFI)j|;&i(t?;~<=DEZG57 zEt_Xwg`@G9|e{|?js5s(_@B8(I^O^nK?)Lj#>r5%kXT}pK z;~A832Fh^VlE_^J9h7e6ih*PUQ+Ah=FtZo>8g&;2oLrF0U)tj;vVN~iQp`(s;``n< zwBz(C{PwL+Q|11UxNe|v>7z8c^rPOHwfy7$u_w^ zEcIboU|?-kIGij>#_Iz4oziXX-9J;KTK(5*TM(g%fd1&@bW>V|QyukfFYr2Ra~g|s z0sfo3o&Hr7IQ)1QI&}7o^5H=USUIEP;^Ss?PcmxNt#UM*Wbi%illx-eWo*r2i0Bvg zRRboVrw`_OPP@cHx`B9){vo0g%)7z3vy~lm&`XMDfW6F60ULPgjwM6~T3a=a2y1+@51NIB);15ehWDj{AX8|STri)v^SVEWeCZ`$8Cn1mQ0c+zoia)P! zzvsN$#=m)<6n6;Ox!t!=S0E6Pc&_1QIPdNrm%>3~s%hvUwDQyR+E8^X$R!gpen+?x z`$18gD5|9i-4p*V<6uAsdyz7c81FBx4Vo5luOq}c(t|2*e(`84Na_Fyi^{STa-+h6j zSS2T<)P~f4{r3_V2@pmturk2Dl4;dIrXey9(6okV`N)+RF)|`Y_R})`V}fsSH9X>o z&DC|9-P^2vqsdtOw`WD`v~40;H+Ww)~J8MXdNYazPf^LU>g*{D4$YB&2k+G+nOE5$w@C6(w6vIgXI z%Z~5Q!|)P<{qeidJ;g-T)xC#8-%U{3UiyZ&YUp>9pNK?xQyRicsFP?S3a1JuYyJl9 zL35z$D!C!N5@BN|lrj`xR}=6oxgeW=_p~Jn+xJYQ^w6dQJ=sRMuSvG)|3p2-wDe2L z@+>JCDqw#h(E5G@=`R3kdXV-RJP`~#;b)(RK#>lz60On(jmUMdtwNvmQ#F^eQO)!C zdvBRyb%v^9(t?Zxao3GVx4j9wT$d%+Y9$N!%5_im-Oo$cQICy~_7fwKHgxpkCFP&w zVp7yQ3*2zGJdvcC#5LM0EemOKN=3SB(1w#=y_02|9{Y#_BTi`}2Hfnq;Rz|uTsPdl zmu{Am-t!s<3qnK$>V|O$O38uuQxZ49RJccq{tp0HK&QWUwcYtaSh6uhMCP6bxhtrK z1io=Ys`}KBDr(5ooDIQQu(OubNuaEzoN6@1|Bagx?o(63g_{!UOH)FHo8rsY6i-`W zTBh{1v?LAlnW)i*L}oba{%L(a75;2m9-+Gcl*_In8M6(S&*MS!m$0l{nn z3O2CcE;g`VHvXbQ_=~gf7Zt=`oQ*&9jXJPo7dx3E_Pu5UF^WpUF^WpTr;o$ z{<19mfr9wUvhfEC;V;j^A1sK!JR5(%Lih({;qO-v|A1`##f9)!WZ^F^h`++Y4_{*- zSR8Rb3KsUc=j6!u#0igFo2NXhmifej2nDEwT{|d!J70ctN#SwZoBKD;31Nr$jC?|> zKHF7aBKKykUo$tC40;7g?db` zxfaNZs46F39vsLu>meWhLU%mvFRsaor&X!ubhLULZe;CGE4ux~p6onu-twT(t2|S8 zePZG=h{z&SQL~s)t`8JUvHp*CP;^+BNc)vUDBC)K zKwIQTSukj4&uU!Ua`*%OujS#S;8_)jek_k*cX!I zL59HOAX;&Y1_dl8*2kxbnAK7)&N{;KyibM1pAc9+td1vqypFNuxGl(8JF%W|ti7miE<@)y)QWsrMpJee04MjZ2U?R{CH;>uR&$|R+i zs%D9km-pxqjpkd;=K7;m7wmr3NBH8h0?azHt=EE?66}6Tf5BE%S;h~eCv2sZ~d9B~jgM?TmgF`%VaKUE6Lc&ZYgC!MFT_}F-gh(8)nQPE{Q z#l#BZDK4Heo*Kn7e{!Ch#1iAFS^UU&Y86SR+~Nf=*QOgUU~>^C882`Xo|x>-nQ|O~ zQVU?OL~ZLZ#hzu!cHC7{Wu2ntC|TCYYi%fr(evRV*ZOl}bBU8U7h#>*AB z{SRyv;H`(~Rn8k4r9B;TQqV@AVJgLgCQKajRAA7t15+jDnJ_lwuGm?=6NB_{4$>oT z(vS0!9u}ou(!*ZTBjTUU9?~PZCvPA6H|h|n-^KV{+#o#8gxh)gJSt9V_7EQR5*`x= zdkK$u`#diCH+%X#p0{zX_PNsG=F^J@Dmohcl5_mz%$|&B-kQl6(j|(iXS*o3GpgLuV&d9M$i2)hJLTagt+p~pqQ$?$c&ucp6|XSs zD1WUI+INf5zL{sD)OvamorO`lcw``6#29z3j>el{<$v;T9TOV|gF1*2v#^3vs-R~u zRB(e)L5M2Y>NhGle;NvPu;e!YLs&GM`ZmFdw1eIE`)QgaM8lQ68T$49l_~`aKEB0| zO(@ZHiJPAxX}y7SZC6sNh9q*+2CHR1AoDv!X4_D&^nTtMJ(=zU{AekKISO8VvRth% zIT-#FCFjGRK=R=3JGuRJI$0^HBghooATB%-OWWPItBz9Y6RptDlUpC$@*#U-1R+b; z6{<2hiJ`EK2U8o*-3{7!v(d)6r^_}@_Te$nw&n-B$M{rJ!2Zt2wp=Fr6-Dknk=-gw zcYo< zDvdW1+vb9)sg2}Xf~7eDje@2t`s!(ZG>JpjWBx6F`c&1#CbHn7e0<^+_;m!d!lqVy zhOMYIT2a@~lS%JRLN9wnw1baL9qoyDq#HuB@q-pMO;1bGuLg14Q$4VV>vFlS7Fy5f zz>=VIj^s3P^~k+|UN)#9lcX_6ry(u(L3->FTI&l`bMwgAL6igzFDlS zjMlk!^HO|Sb86*ErrqIi5=E?Jo%u$k-=n79#4fvaoax$vd{%DfV(z3&BGUb+rM*eC z|IlK7kiEs;ULBkuuYnRh4<3Qr-oz9UA9eh_+^ZRTn;lsw*DkmGK_BiI{`782`wL^$ z4!dV9AR}S7Kh52D9>aB4o3-xp`{S`+NBYB_jM~>I*Y=)%9L{|^2Fl@_YYb`C8VKJM8QtH2^Qs zee+RNcZ?x2=`b^yaJ*rA4>My4#~wv=UUP31$5N>+Q=wF|Q7TBKKJu%Y&!3>rKX2ms zXC@tG;;fy556}5=Ku0U)U|W__uTH**B+R&Y?ILw>!p9f+v4Le%*U{%3B5&(7@6l(* zaZ*+v>PMCXGo`4-IYHv7Vgj^Pp0K|AfSZ*d+9pD~sm_bG0?@t$(h#wG&xIDZd|w?i z?qaP@vpQoW#6t_5GsYEE!N%jfXN+g+WCdsI!g#=dy@O8|ahP(=>S%FdC$fuLu1K9L z$Sd9v`r^)^q819%i6TL)^lz`?nIYIMXk(1xZA~geXXXO@iZst5n*Vx&_e9Y%P15{0 z?eR-!(ZoQX;7gXkpQ7X;>7%i?(FmkgY%SR=TJ-^X{Q+2Xbb1U-5q7y$o2W01tM*2s zVev`cW8iE_LmF~Cj<#Bc1LVyF`O?V%x!8cLo7Te4_ut9iw{IyVfAi^g-3wm%d;YQC@Vo#C%>NsD4=;GWIfdx$?DEq4 z*3_NS>$Q(qsQ3meqszP(CS!I~95l>h4^{rVYST30ocIr(R%O5Bb z2C57C126nH++Cst8`)uKp}iU15t5vhdr^?~DKs@B)j5Pu10X^G<21lHi)Q^eh>#%$ zd@s%8DG$Itb-mG*i)>`TmF_x~Y{!pk19_0JG(X^#?IsYG-v?Y+s@zz@mdZQ+U;yG} zruSx>ke{I%&Uw(gr~maBtf2{N=%ACBup7X802CS%@E3D{L-_#?3Zj}Hy5M1Ve%JsH z{rLfWUBvd&AWoo;JZQqmnd1>ndnysul@`79bp{mf&%nd=XY}IrVlF@+=-w3sg`!Yn z7F0wNn?%6mbgbOGxTZ-rVfd>E5~ryPvV>NRF^EBQl~GG2&{8i>JPfx~w4Y--z4Ds9vbM}$ArLT z06f}NpIs$RJrg&I;VvRffa_YF-ETdgKOY9%x0&+aWQqG-pKC5gX_3_GMN-R)q!ty| z4q%J`Z4GEMAaL|p%P0dhPM~!fR4IV#0F-*`&qUfS1bzql)Hc$ zVV^}lS53z7M!mz^Ag-t(3(z7fHv**})Hq^Z9A}Ax5eJC??($AK$cL|S#NCq_l%N0l z9!@jdU_YT&pY}8FG#gqgH`q9eUdaLS&skQo7~dei`1fbM?DA>CI2$jW0I_kelPUh)b!-K5^X5}}Gi_TdE3SXq`{h0PWjt6%DsH&d z{VgOqh#|~z6=8MDMJK;!==){C$u(I8ZPB#^ zL1Dt!{^S8r;Be}Iy_szRB7H2@5Ma(^3xWyPa;5OCQ6pceIqR zrdK;aw2Z3PbCHqzi8U>%1!43?qYow>B^mklA{xDoYJuD%XIrMLZ9_3BF>b;yV#b#^ zQDR02HRYt|8++{)V9Q_-VYKfs2_OHW z1qnOQBtZz?M+EZ?1bYF3KLI&OFHA2*J43VHK`%#8xRC`27b;lNI9FT*I1 zIK*C3U}Y4aAsq2Vav(WE4>dZtkJ4k@1DBI?*%9s}A5cCQEkps(4ZRLdqA5(Xg7o>* ze{_Fur<+2!ENrZarNu!=4kaX$kMS;t z8bI=y{Dmx>ufD+LIz}#65L6ag`WCT{p`%|d=g${)!RO=*trOX*7y1eGX_`rmoPxT) z`Q`_GS5lMs4wcvgy@=Vn9zWFLhZcYOc&-O^tE_n0@)edEWvG!yqoT`HoSrlmc^)z( zlKPYaH7;h~+{5z7*Q?mTDJS4s=7U==K|TmPu5bQk3;(^0xZhcd`_Z{kjt>Yh%=7U2 z2z_(sP53Ovtx*ao39(XfrG7tfam>(2F^d(zZ{qBs2C-yLX)-v`vKS0ItSO4Yzm=3DqP z#n%9}bG`@Lh8t=y;{y^ugNVQVEFk_0gZO91Na8Oj!Z{2booOl4W}ayJe15y5Wf;9h zy%xFiw+TN19mPP_T->;meBL?7JJY8^RXu(T!kE4Yi|9a_EX-peAOlIihVcF6JTK|n zViMmIn0^6-li8<`IQsTYZrwK`xt5TedV!byJqt)K@RL?KN-D#RF>;^p#!J?`)KNZ? z1uT9n;7Np|pqX(iED=1*KKt9A46kgyI*Hz&^)fDis!Aijp*?eTa0e&I+qKN z@#)&QMUgy>{JoK?kQ2Uf)?xG^rQ<|6hDTyAQ4u^|^yy#HT1-xG??1NOfSQdB=*QTo ztUZWcLq0C$;o;1OSL2x%fh8%fmk4taKbJ*0oZiHO zc)*KR(irP?Glo_@X)6;--x~nOsFjsg?6;tuMoKHsxG>G>S`%LKA{2{$t+Ty2MZZ)05Gh ztx@bYdb^_aF|9u_3GEWM-9Yo9iG1GZFgBCAd`qR5eYVtKm!C;mTJrGALh2DIqozpN zB;MYK*#-hB1(NqNGcQsw7tETVnRAnCrQ1gL8jIe1z1Ww&Zzq&?>1#NnblX~FAGU6u zr$q}QGr?pgG#5$`7fL|7j%H#-C^>maGG9xQeyP|qbKp`gm^jHC+pwdLxMcVot<6{hSoDGD$3A^bg1Nm%1 zzMi&Gh@`Dwl*32j6DlA2t#|nxPUUZ-@}v2FKz2)bFU?jkoVq$wqe{(l3or3srYU%) znc(CAF^_N$k53IxI?_FJ2i}<%{(|UAAYDH+=(_Ka9`b>1%A!6tWwVXr>S^-!Td1T zY1#385K8#M7!=($j-ipLlES&T4IXmDk7maol}ze|gTT(>A+qg{YDZ=6cn+pvN3C%eO+qKZYp;NWK< z8S7rceS*86I#M*&(sh{17J%x7#pI(}-+hZ&MC&wH=l6v#h_;iQhU(MfNTXr8W7i{x z#OXGYI;Lodce*V+kewdt+38^vL5>;fb$U2WLaKoz--brfHV>y*y~jmt*W)p&cyqa* zYM(aQ>!tMG0XUo94;WpPr5jPRRZN@gHZKReX9QR@7W+(5ww_0(DEA_KyqzCy4iO8P z?mWLF!Uj$8&c?VTtUDAScb<5bJ5T2X<<9eQk$K1_kDj<#ca~$*e39xHaE#ZZ=dTg4 z&lc%(Jx|x?_H=B!ooo-ay`D8oplp*-x7;GyHiY0F@Vv+8m^)JDyA;`j^}J5?T#fZy zYt$2idaNRC(p+RwmqcOnJWJEuxi`&dmqLS$xu+cKCH|xRCDJGTN)B(YrCVYbAa1+S z8{xq2UVkg%W&8ub(dA&(saUkqa=nVkS5LcNRmxX)xnD))s~g;}WPg~`Sx%Rk#^5$N zzwL-9{}I~oZ3Cd$00@hJ---bH^MS%~LynX1bkJ~a&=N3NOml^SDEGpw-U%7K(`q0) z;ZgS1I@y=i?%OT3O4^U^HBU~-(|5hnO6`Z;Bvz31Zm_1KN$h`y;}^D#Ccuzs-U;yG zNKlJeJgbuXX*Uh7qtOsCXZU$foa>?$zNO;fF%v_{VH1~?6rq%J^>L_^9DKqcDYtG6 zSv4!Qc1kT@R3A+`$_a-lB}3TdCwhq-Ka%a!jkx)eb#Vyw;|G%zw>fR1PIs_-Eow}N zbY2+m?v2Z@F;`Fwre3H5s8y}*)C(A-2UAlLHsf}$b@Ab94AQ%B;<75s$-D5`hMtVL zb9ArD3t^;M@TU(E^PMjfLq^22KRpv#%v3NzM%LwH#K)*6?LIX!$B&z-Ckt{ie z#M03^vQ~u|`faRQRgvH-;vu7sg?IL5rvS){4>q7ulCw^rLH0d4(4#2&?Y_7(zG^A; zZks3tskwnPQ}tAC43?X1l)Hh-!AZ%L)Hx2?C088JjgLG2vA6xx|I+IjzQ zZ0A~^r=3k~N88=iR(uQT5y{J(kd8y77aK^AC8XD|mIvYy1uDBty-kQZM`hO0^lKeiC+O3M|C_)N^eF zzx%HM{z3!(0)oF?t#-3zQ938nFprfhYc{f?6yX|$z6hobc@M;uJ}#@QQXn?D?0BeJ ze&jDbnOyL4iTTt_$;ogE&6fsVPmd9$>)!CLr_1-jR=>fmZcyu~iPqE8RIcRBfgVlo z5mfFc-u0BoT2JuCuW;jfYSR6hKwFo}EAvr*+|qy}#dS|4lq%kD=>?Z?mQ|CE2x$Zq zjemBQm0j5XSf@B+VSE)e_02yH^cV^2_Qs}ukF$kC5$@I3s7U%x-d^2;J%kYBzfQp}nxz+j%a>wEvGzVwN!ToF`}zZzXh}{K<>1 z9?_-8Yg|tMu|a*UmS@DD$OfP?C&=S*NXb&{yMm8l4UVG@DL(lX8xg^DV;E(X(T1fP=X4MQD<=%6IJ1}! z2;i($$m|d!TfyI%g13l*4%i*Ge3zpXBD4#S{yqmyN#<*ivUr6x-+ zL0~V`9Sz(a4Z1rTD%zIoeFs;+>JLmMDJvVko+kR@++fH?x4%s=>gEx(1e+aR*2E#8 zT_V1oxhaNc059XC&;ISEp< z)8B=P-lvMjVMW&&6`ip=vtx0~#b{07=nXJp?L8HZSir3c@2@dhbts!7qYY4X-+9L? zOQd&0MtiJNU=|iAp#rbJ;}vpOP=P(%ho89k{@b~P>caV#QmoJzt{%0UM4W997(Ek* z+Oj{0OgiH|xD|?j#(*<8MfiAjB;ZtdHD->fSOhKwX52D6v$Xoo7pi zQYOHha%{nP@l?E$Jpx&a9BsENVG9bkd^cx^_dLqW;7?zTCFJ=XIr0U$`^(B|&zsnu zWKG`wCBQ_;>EpX^^|O?+M`-l_s3^Nl6>opRd z6{io*F_X?&`+HwI`Hu%{{3YpC=|^|v{7L?KQa(AjpO#NxFVwpc-a)ePt{zO53!960 zm;*&~U-lx!7BE6t71vKALq)h#RB6AQ#kFPC^sZ9hDCWEi=_|wFoxUv$t{Vw2t_xJN z*VYD2*W>r13)K;I%GpKxWx?0K+(#ai6DirUTtLdhiDzy=?eD>~@u~IIq>mi2nl<8y8^2Z_Z+Y@#Vccr6mQap8hO_Rp`^XdQAd*=( zkHmF3Sl#L)Uqz2b14JpUP|IRjp;hdEaIREasy}{lfIENip(C>L2dCifhmP|3qe?B@ zZ{_3acxWIGIBvk9)YMSBUB=%km5?NV5Ahpgf%fSeI92ccnXLTBMlanaMc08~B+g*xxBE zKKvO@C{9@X@!q{a<{&;_L(*jD&=)iP6>?hzDPQW2M;yC^&Yh=@swUvjpP^kKVX5HR zV*|*3W9l$q0(a%YDwkCGYQD0w`wP=R*Vfu-wTyAht=d`>5PgHu?48N)85Cj$h=$q}DBM-Z-icK}_KCE42fI&oQwO%Jz1*+BmQ~_@)s(vp zVR4tc4NdMc!EWJqwV_G1!E9`EZsZX$-i^H3T_)I_awBh6$W1E1=Y_}96{)^y4rMtDT(DkWmrlR zal$yy!I&rM5~Li2)UODBx$G3+jLs-U?e@jD7}fA{@6vZTQ@TyH@V%Vk{RhYaE^m6e z9Qkz{IQl`d;R}7ph9CPQvf-IOlx+A9>q8jzfCmm>5)!P5z#e!P!Th&Mu%`)3?d-3D zkf=^6NLkp(aolgBfamzuw%Ykg542t z)QwH@Lv?#7?N8szgE-7Z(Jz;<_=cD8;irYmfDTKt;w$h??rWiTxloo{#S57t|S=Y}HqwD`;( zQx=m8?NFZ!7y!}wfL!jDB|Mp8VsFlduHaBhhYO&6NT;VZSog&*Wl6Rr}&4sxFm|e8U+AL|j5VW_kJuv9~Y0 z!}4_IAMfLShRZ6TBYvV$F~qa?@q!b3e#m)dvvP6dmNz1h>;ndpR&h+c_dzC;KAcVh zjD5HJI8P{rq!b!WtY;&H!g0gU$f=Jtf!R0c%5+5q^x8 zZwYx6K@PSNa>!B%nR&|tS&5Kwf?PfoArJhOgnY*fS%r`j3G&=&2)X203HhWKGK`Ru z2y$p0LSFy8gk0!_j3DIE1o`Cg2>If233-|qGK!G31lf54LVoWV3HeDizvYVGb5z`%gFurQEibmROioV;Ho$9HlJg#6DumSGc!+~T01ZJ84Z+1Q z1gE~C2Fpl(=Hw$|6<&km`vU6`^a-F?o6Ahn=5nsIi=5`JNo)Ta#inaXAvc+&7*~~! zZ$#Xa{0gd^!d3R?D&x7UOz^F_^)z45%kCPk^)1U?t1IePd6{#W-4{`omb-J6(IZ)x zTA&hsx=JsS69e7u5PIQHC0un}Xmh{Q4a__)iB-+FHk7^iEcxh18i}ua1tr zdnwF5-wjIn_0OzyXcSjn{Iyn6?njPIyN&ibHl1(bvGZuHk7zJ-bNkQ69{7&?W1K#& zXes1MlYQ!JDHp|+oJxy5lIS$rhufa=<=bzPdrT{=usMUFgYRtn@Tj6yZ5xgnwMymW zc!?U`G2Pqni!u`N_RdQY?UC;A- zE7iV3$^s^Z^Oc-@4U_USOQ{~FxVW5}qY4Lyt~OILrY{2UVa$iFueW8=`$B(5(J?}I`3VhNNw9kw-uJ{~dfJ}7DjdL@P%_v0 zS@>_jCff!|(Y6f>+UeKA7x527;82A0F+^bZY;OSL{$JySUDA(eTM6&lDj?rXn3QWL zQOSL-9OO}tecFp9XY_MCx|xnUN3B}773QUwxx4GoFNfqmK|Z64@m0i0Z1FPuYB$dB z;ulrD%Bl&E5AtIxJ>FKO5=G+UqAKMCi;pdgP05i9aO$4M!L}#B-&QMm3?-LL9>t!s zM>w+_?g5W8g8&!U(J`EHLy2#)-F2TB`{(Y>Y1t?!7B2Sa67rN_*t_&MR2BUUzmZ!q zzpsS%Kcx4K#``e5pH1&=<9!6)pF{7X#(Pv~OzBZKHt6@Lxj48-X)Xe^K3AH=fnxNm zGUWt+EIP^uHxDfZX7~mCLEo94+qlsO0FH};>*SYsd#%baIjy`m1KtAQ!8;GLzzyxX z5*g{ht0W^$zQWu$4nXq`fM2ljL#hZ4Lu(*JR~*Lm01r=Na2L1)%{!7!LI5*h4;+3-Ox+ zP+I-rr`t1_chEL)M%=B@r9$F{OOd4WO_J*9G{I%7jIz<1HKL{YYs#+WD$$y!1elqQ zZf4pabkiIXdvs8GQ~hFDNpEH>ab?Pe{0g!2wbyusslC6FJJlSP962vr&p+8FJx;&+ z(^DDkYX3Ijs-FY6<`}r%{!-$KDpz|t6k^!YpQ*%;u!Na(Ziy_xis2!|#*ZJYwue^+ z;+EJb1!>sq4EZ#NS42nz`?)TYN|mr#D`vCZEZ_ZWdGP(N>A%I-5iA+%w!d|Ua@^k; zp&Ln+#VQNG;Yu_|RxFOW#>oj@ zaot9Spi|azlwh`O%Jxre6#u!3Ea+&RZm%O>u`YhFwMDtTj`q-H{1DSDgi_MFu6AID z4}Gr#@(Aa2hD86<(#fb_wk)MobJ6GWfOJ?G`hHTw#CccsWK3_qwaktaCT~Utu#z8; zFD`SqHm^CQ_Rp8VTs1^ht>&XpD1o*rA`AT3}WcwMU*iLljV0fegz59u^7D* zTA*k%7j4qX4l8cJucu*G@Yh!Tb<_3ubqvl@{<>IyT~A+!LF#?<30&e+pSE7l&J1|j znv0(8HxQ-2w#aY*!h);poa+fdIl$o^QMRmg_HN+5kkH-rs~ZZn?6B(#wQTo-E!%dT z)3VX%?Wn2STeYl*=pPOK{3yuG#3l}%`#Rbv-fWi#-I8Cks3oDe4W%ji0jIk>nG#0R zmDDLp;S~iJ(+#{kg1a`QP)l)kHFyI_)P9rV9B%N!S!3{|zCg8(!-&oP=_bxCJ(|2j@C>#hZV?awi#4^B%qt=OzQCBzZ!a+B^Y2uEH8=x8-8HX^fy2oe z&~NHxj0F84o|Jz_A{pL4?=rG%~gKI1RU!~kTqpd%uC(a z*5zkic?p`Op{|8{=~Q4Gt{0~di{%HLsIV<}!1=eIU-2ALx*kk`N9{$fkG9Kc9KKBE zF9~~WS8@OyqHpzqgBM(xz`^@Ck$z4S=@0W!BHk|lh?UEfO$@eOJ zy}WDm|8a3P{rmQf{<{~u=%2IAL%%oI1)gVLy*A(dw|Vk+koU1xO&h5@I?4|y8zRq% z$Mh+EM}hnu5m)|>q`eFIJ0hyogsIv2JF=qwtYj%G{OP7vt#(}?rP}UmwNw`umj{sb z;!UE~{C=PvONIh8iJJ!mESqv^Ff#5VGJTTU)7~_3fAw`n#qgF9(bO9rm)6`~)vM#2HmzPK zkqe1|U+3^p%W>6lh%YWix4A0gMrPL?$W?rY^G@=$fTbZM_FI&1{pVg^tU=&0Opz}J zP845qKloEru?P}p(3Q4}9-#wTI7mVF-B0so6|q;xY4qV?bv$PLj%GNB0Dx@$K|Umm z2einT9~;e&cep;XPV2}iJ&ZSqGP8Vj<$Ot0l_n~MyNUQ^VBSsj}N#TE(YmZWxJj@GH)=hcYfQf z@2qew@pOq1mQzc~rlSnGA3?T}dD%yfFo{;&^skbC#KTKz)7LKSPj16}G{lwWg*bNv z@wx=Vp;$s4_I3XNFtKV0R&C84#-1zXdU;1Fu~ao?AK#iA}`7GY*j${N zr*CX?uUYSU*Bjl@Z*I%&f9i&pr`1e7?nQad`*1!#&0A|;`i|=}eGlcvnIkmB8|CqM zermGld805r9Dp_SlN}k3vH7c~KwkPkTA!zXJ@Vh(3qk%{npghw!f#$6;lKKMVfYm0 z+p5f&?<%YRln&+rG;+@e!{`K-vxh3KT*-PM$)4MN?z_(UV;q;4&np$N{5Kw`-k5`G zQ~s-3pyj_Dk|O<=TmE~0eh&GsvrqEhL-XD8-+yw-uU|?i5#0&>Yv(Ka58p-lZ_h#h z_gD6b{xjY5zxbI~e$86XqLl>}`)3&DoXW>_aSz%yUz7f})yrYNcZ>pBf^0Ja2R_5dTkr0DD|T+Z`PMKA&Th?H-q-4jI=+cFqohG6|_J0!dIaCTr)UNepGg+NjVNz5lpmp zILe~paKCZ1n@hserSvIXl>O->mImXMDg4IGIf9Rts7L2Xabie!`H9d>1!3DoKJFu5 zp#!7Gm0BqgmxK5y-a5n`XYWtX5$UfKb2WJ=b6S7;OmWwNTA8kL4%;w8)cB5}_p9So zNlnK-IYs>Yi;TxTNpRUXZN6z<_!XqoxJ2T?8(|4gypoJ5I!gMUFEi|) z03+LdQifJ0v}@vW_`ehZT4$FtYmJH5|A~BA-CnYm30D1&9Ora?34$94bmW8O##ueP z+|sv!$3%-nD+g~tKVlIv@dXcKGxHr#dxN%SXga7s)@>zPParPx`NiHTxkL^Os(B0@ zscSqVuJP3N88xs@JQdw1Tg4#&%{YUuJxJy2HI@>NUDIaj)7`iMh?J~^zXpBwV_Pz; zm?B+Nu)C!%^JU;>JzxHkHD88o&OKjV{MNLhp7|nf=UK<0 za1x8M3u2h`?#)FwCSmqp)FIJfw3mZGfK%*;Om>*+3U5|>RIBd$ZcBDNofsnGs3IpL ztu@!5J`tNMKaEDe2x$8BIO_TyIZ$P!%mvp>no zMi`YvIkimGsO&L!n^fEnKtPaBMaQQ~x8k%W>1>GdZOP&gBX{z4na$$pq3-()M3S%@ zFGa#||Bt?Eh5LfT+pB{Od`BX_mfxk1G+tgciW6!q_VtOzARm~hLvTyh(rIER>A<}5 zg%Z@`+6h)MIpR7_jwM{7J3b|r6PtCWK3c>@=riI5b{1FPaXB3pn@A47{{!4Cq9_)i zY!yxKmLf~&Xx2N8PT!ECQ0tuWGoJI+y`N>g=PSd$vr&l{w`;Jx{u|1@QVZhFjoM1K z*V$i`1AvzJ<1Qfi5NQYCpv`_Gh+XQJHH=r<%LD3 zo;4(A9XG=ejY|OfwRPLDk&VK)A6qBImXSR?L;8AOJ!kGph}ZkMZL8=N{|BmkBi^!8 zY*c>yI*`lLxsq)}#jQKEWQzCqyuXc2mgd~d<8N^X`cDSme_8umEC`q*VGdHOd$ho1 zBw!5E(tt5YO9IXym7Gt5l+c6JDn6~%mwvpg^)wT%JjdI%cTThkIQu3AQg{=U8&D?+AW$9C_y~4Uj_$^dWdt zVu)b}Is9dQjGx<8-06JG-f=n2j?cdjv!ltF9nrVc)co0blJ3ixyjpNWSfI{`qUXYZ z3I>A5Ya}S_Ewvy_a=l_obx(%FN`u|oxoESfIDc2Vb)uEKTbsrH(fK_XDCPHq%_P)0fF|BXqf$`ODn_;$Dd^cepIqd!8zHD3u$Y zqa5Tn66-FvRBO@C8~Flfnf%>(UcT_FH>mRyqI-A`C*k<}aAmm!56%F&Xk1~&~@ zaecq9K=X@x{5kjq9|{xadHKcOZxrAcn|fXR;=w=nnO~r!<#6DrUbe7pHY+Q5RyLrG zw+H%~Z>WLV+bEv$xd#E;`wX>rL8G_52fvQ(O_-YnG9?uO7h$p$jjcMOiyiIDGHC9J z)6);%+m=bWZhDt1ciYVJJpP)t(b$I`3Yh~{aot07vI$Qo;(sNHN3$I|Ad2K`viCcJ zGv!-v%vwkJ{qikEt#u5&TmD`igd*q3#)=m&3R!J2Kl!d+X!grx)Gv_qMC>N9r*Zzy zB#@RyPeNfak!3-buEwj6r8|08T&#)w^}^#YMW-*TeXTV24H7eIZLJ!ors2D8Jx$0gFT)|BeJkOBbtzqtn&Jb#!z%I{tLW*~ ztn$d`;*%r)jdn?aDT?iYPK|N48w^;T8m5>BPJUyHF;9hzlO8IaJF*fwkGX|kU>S!^3PdR&c*B^eQKUB*P z-QZW^l&_|r0T-(Cb&1dSkf>(of~nOoZvxCoxk#$kFi!x?!I&bFX5GLuVAh>)%(^3f zw@qJ%-F>i+3^}9O-f(ZoG8Its0n^8X>F=kzX7|Qda1VO=bhQU{=$pVh%3nL3_Mlct z-0IWS9`rY=Is6GBeExLrg1F}ugS6Y68An=i*AS+!l5^hj`TNWNiUR$E{lC@!`W1(Aye)zM`FF}6HGiSgE%25} z$Z8-Jk#kqUu)9L2J|5!AAZ|m}i}U~jS$i!p9h1xK^ge=I0sr#AG<0kOI^BR@n)uTq zUv6)oZ-7628sdk+7yq2llYR3+j{UMB`%)yGKEhO2V|N-NZpY9Ls%-$&w(SV4EvGu= z|E>DIOZB}8OKkas0`=vPhrEsm?SIIbsjkoErxq%o*FJu7?o#>f_4(l4_W#HFZ2y0J zXZ||?HFs$iKAm3>JU`>P{T-d}RXh0rKS030p1*#w<&|n%9wL}%c7a&!9;x^ zJu`hFOyJ|mMs$7sASZh;+KA@7Bu^%_0CFOMwAC9|P$9_aFR8;zJ%KE!H!>6;$l))M z@iWGzgqV2cKie{BwwT8vsXzB6F?sCSwqB4R6OFX2B4svs)kp78^-RW&z0_g#X7(;1 zJ2F;V;^d4^DR<$Jm~wBxYTN2h?gp)j@hYc1vPVm*o0J<5=q9RHWY}XMCSs+A$vip0 zDGS-2vaSLF!v6R*>5-$e<`eD<{(#5ah;gUw+g~f<4MOj|hY0Pj2u+_&G@Sn#sXH3P zu`V)G>^2<}Kk9E%=}%X8Pv;gih)>pox*EXO67U5K>>k9-r#OhR_zCc5Y_b^sjB6Qc zx>afe27Ma@ntQl2BZ&^(<1NeW>kZ2q-6V22TfUxi|AbO%2EdXUr48Wh&D?NW5qr}B zem1Y!;?@$cG22+*tjh1`Ri;SA=Ce`^gZ*o_UK+pftUa{-E+WFcbh?hcCC?jc$b6Aq zU)E8HP4`hf&T1EpD6?S{&4#`M1I7T7XU6Udu1djGi8Vz9?H@YTW^lpNtX&3~PaS$X zfX)7KBB=@&;N;it%vIvje-sgYVtSk}fL}{RrcdV3v3D$GC5%d$&+%vmpdU_gCU0QXHoNr^g!JBDxk+~Q>*4y3Q)4$*~dvyd(J%vER#HwG@un;Mn}lkE16Adu&3&^yT^=^I1m0fDcY5IDUgx-=oO`r3 zF`nRB_%YYQk7Y0XSoXq?sfADXdxInf%YE!Wak*#v&!mFU$8>ud6hBq%d3JpUgGeLd zYS}0;a;(WcBVuHn!h%&CQ?qjOZYIwcM&mmoWt)LpaYKl#MJ@2%6}@ML1N_gJZF?#<=a-;grKd{cGmK>Iaye?DHXG; zrh>QqOq)^@KJz@+_9EU0L)uD;Fh`dW<=5$4bb}H2>Q)&$zl3lTA{3iGS3<(!EF|!U zt)D9)1LBeOPWZtV%zA--+>vx3{W#q%g4D`>U-dECsh~M8=jpFlA*m%U8Z|*N!|UiG zO$}0!hDUOx>JsyM-6TDMLc6)7^ZTR2D$H_5-@d)rJ!o0+l`&&v(Ag319*Hk(4S?~G zyh-9H53Q@*;c^@6zibQ$-S_JbZT@0Aa<+w%YrQ6Sn3ki#W|c_}lrQXIl07AhmyLza zx}1H3q*Uq0bHy^yOr;z6b;XK}mF!%So&?C?YNwV%4P<*7$o6%R`3z*G>EdNaItL%s zhqk`Y^YcKC`#f#Z<4rim@Z|&Rf%wBxG!O0WzR&Z%eyq6Psr;wx1^%>_97A$HQdiqN z7szwvW-k93P=wfH?8+-R^@oZ@Z;~swNQ<^jbl|~=&-18R^QU;5wf*N9=KeCbNfua+ zX@$0IGQ7DuH~4Wm5=0D>y8y&k`7vo`&p#py|GkJmJwr!cHe$mM54aQcB9hApN&hS) z4TNO4OLZF3Pcd*WRwfg#7ASxG6=iqFy-Ry;#q2SmSqD3!61*|*N}L+v3$XgjJ-Fs= zai7YE%yfj5O>pP&5z}+ukC5-`86c;~yxFdih8KIToqA5x8d(gm=k|UEd#+9&a}(mp z?YiBExxBg(c9GAtSa*qLzKbnEt#=}2jH#Z>9=1ER@PSK1%3u=}Nh`O+>1t|~y*fY{ zwx!?{s77uYAO|;lOrUz!p7<7wrF_TCOLUi~KfM;ew7ue0H|2C%5Ie+bSwf_2r`@m#W&mNhVldLGEV0 z*=WMBaVV22i~{e%V;@T6CP|N1vfggfkFtt`3cSkxHe#QPLzXdJatZwf>D|*!uZtM0 za5kE1;zWFfo2HjJ&#q1%@Bs%|7NF+8i8ovES#h4ToW@rG-C!fD0>*cJ?#5US0w;1C63Td@_TYT{Hobs{;B~D^StCi@F$QQ34cnH zd&B4J0dNSsXik>Hp9_=yYF5hW;pkzr<115m@lF*StUbO5x%lg7?Qf8*b{q{z83@O+ zFU|<{)t#V_y{y)lzc5E6=I`B37BHI7xBQ0Hlk<1SAzp%8Dk(4HV$g54W%7)-yS^Ot z{A-C>PsCGC%vI0tU(8jHGyikh|6?lG1R(Qvq? z8d4Dw12`sd!b|rex3P!V%-QWysGC zG;2!lrJ3X))t5QaM9;Y8+$Vd?)M@1pDzdh%yVB8<#g08Dz-es`si?k`vL};U&?|D8~~3}{ww+z z*oK$R%-hdpZVp$x1XrN<_wFUnxuMim3^r{iqm1nY{PQO5j9+hAx^sR#y+?+p0Lsc&RHre5ChdKOh5D6U9A#-L|X6USUOI%Eje>>In)ql$I zKOzrHprBqPe-^Nm!c8vaH71jiwg2N!B?g1KVVRpj{dAkjph}h%U{E>e5nCTASoe9Z zx=THE4|CQ1^b3WUW8e8Br1J<>ECS^T{4A+h^br3ntyshvxvQI!W$sW?92_ z^qpm0;jpYj`pmL$fjCU-U>DO0yO>rXR;ByHJsv5}`%|gz(SwI7IDhs*A4ivzgVvA- zYe0opN9(eO=Z){D?^(v{9P~O$z>sMCLU}Z+%LR=nQJM2j*hf@`}yd)C5Vezz^}KpRdd9k584mD+cuIziV_t^uhk9B4D7w;O*1bU;)QfAm8&JKWQ zrQA7#y!iZXe4je_2GKd}1&MD)&Z`5n@kPir*f@;bTjrSrUig3;{?Y>QIO(-N=n;Am zK*dMYcPmmBY5zw=QZ*6@4##GG!tImVMc%XOd3EY*FohaOk9Y5%P^Qy@80-|&|d zz<+OlXZPCq{D?SG1^4G0r90_=zh}I52Jac)OTF+p{G;fu3a^*c(G@wjB7z`h`v;#NDAtZu~Yo|_-m_v%K(jW`@h%tr-& z69rbReeOAZ;16{jnBLfZ_^-G0ppQ4k`wiT!TSmeO5i)WeL$Ox3AByg6tK}at=mH2g z$bFkje*ZwzYp%gd%rT{TCs-kTzI^z$^v{dWt-p5oPmD(DH>8aOh1^iB#pWFf1M~m7 zRoiah@zbr@xd{KNB|esqBSr*pQtA4#A@F)?K;xUekNWEKmd6pBTj&DpC`tJBM0?7z zJ4FmO87Nrmyxc$`b-bOC2aCQbn?6aFR;rZE0GPflTdVEX?>YNJgw&Xq(#@BZG_`-b zh1Nlm6oB|HCEv*!;T}M^5D;$ntHdAUGoZOZ2OsqTRzkI23Drt&Xh`ABggFC$^W{g; zD~UC(aWRQVt*%YEFE@IkS1YK)YSwu@n8JBCsC@oxpjB!D3wGG(trIf3s`+GUKan-2 zaw)E(rPy{bF2!f%A92HZuoRt5tbw|8E8Wgf!S>>>FslIeTH>yj@iyuXoK z(&%nUPK9pHd`aA%MsIuOJ2s7p)Siez<`2F$$$aQbhs+^qCqZ9RZau%Nf3xXn0D79y zrA^b*DE{ND?nn8IU{x#vRhlz#V?@rgbZps&mU+J~Y2vmpv0r`R5X;w*zbs$r8`|Z9 z4H+}K1)I}Ed?fuHXL{=oIH^(UN#js7NqSqIhMVin*<14LBW$>z^c5Yvu(r0X#q)6U zQ53O|jrm@OQ-OhKe@+eqkl}b;JcjA4|1kCY%$gCM++5A|b8J4ZyW`E+eRZlctFMAl z?G#0H%{@G{Z74?7{wRumk038nj*r}mbidyE>y zXe%ml3)qwO7$U*`AGWxlm@%va%1@y)LKw3i;avy}HUYtsHNQNVo{t}h&m*$nSLq_9 zgxR0InrI#SIZfRo4f?P26B>nf3(+~41Ot5aNk8wWsk8#v-BQXsfB)h z3`cut6$|@5qad6DMs?;n1?1Qo98rlADDYZ1&sj%}+DU>qrB0KfEwPbVaKltsZu3em zh!|!&NW39eP${Q6P5)YLqdQ~&eK}7Y;Lr)lEy8?Z5^txam{Z0$ALsV{aL`@7Nu!%V z?dTL^6!WcT-wzfUKt@8yaSXYk7#UgH2lMUeYwS|Y(h@;fB~SfCA6p~2Zm9RzFBMq{ z@zF+2)@(w!K^E=YP^D{V+pv#PRqfOAJ!2q4*g7*#d^ zAd5#SqQ)pvXpC~n%aXnEjklQXfjHR`gOlltaLWOr34EU(<3X_L=7TSH=$-RRA3057 zvO`EAM!5>%_#oGUVk!x->OZ>3DiozTG2x0&Gkl>df<5^uarj2(PJx(ffDXYYSX?o- zbn?A(l-&TYMAzpZ?21GBBYYoiBrqG=`Db3W5l%G^?Hl&B?gqFN1cn?MHVdz`W&u{h2Yx3{5>gQE zg3(_uxJ0|f(QwM%Om6CIuq}d;a6!i`;t6Vz$N0Me>h8E<*`VsS93QhwPpeOngUNS{ zy0`s2@|Uvo_h)B1IdkC{8X*bM$p5s7gcM*~aBQJd3MLRq#%vY5&>#*=yU(Q;kuW@A z1Pa47@{hRsRPVVo7YFmYvIMW0r=2!qbeQZ2o-6UKsA6d#ESf&vmZ4wgOe2qWhYgGi zsR>i_y4dG8#ol@i)deth<*4=Cz&r{BuWAoC;4i^;HP2i&{lE?8klSxP*VZx;W zQAY_n%HbfWgMv;l;2?2#RMdFK#Utw(CBb-bZ6YQPQM~cK@B89#QG+PN8`1SX@Z6Uj zbyZXn7c#%Dqx!A)gc`bFiNK8~(~cH)hn%7>KfS!wLkvU%kL zO8SPi<-Wkt_&_}!bws+f(7OQ^Vin~6ni9jA4kKiIXZcQ=!lCVVx-83{EkA_K^^SwQ zpqCV&Hkd68t2ua26*+tN<06rK{}Mo7Ch59O6=OSQ6wSdf|l+EQ-&7z;kXXJyFJRRAZ%+uwMaN5cjX1yg}wh;NA@9ta>q zd?fyuEr$U^pszY=JF|MVzbtAeYG6$binn5JtQPYmSNvzE#>CP5+Thndc-@egM6V6` zItO0=BQXL+`doVFg?IB4+rn=TtSv#YvD-;0D!jY$)EHYY8r;C3nm`qrP7-yweYl#eX63m zUr3^_Y+`j}uqP!hs#AUdcMBHTf!2e(9g#)$5r53nW4g>x446}fx_gM2e5F%LrGY;z z6@G6EKGMeq9Y%7$viw=1YAID6K~;}~s&D<(jvAd>6>B>MLT^M0dQN-z;~q^M*Yr)5 zy3X=mt$ZYl@;O@hVwdu+{s^%89R5G0_dr?)*dweoOHfH}qP>4?IOx7gMLutDnoDgv z`j@OF80UB`Q+S-;ozYk2;(GwtWhboFN0d<2!d_Kmfgz6>`FbEQq`WRu>{a6ZBz<{~ zn%nd?EQf}Owatt05nxn4k<<)gKwfH`( z=;X~@k)Qd%ChhuuSEtpUT;q8;&9~aibuLM1;T@hHUQ_*+4@=Svp2_KgbMwsb&dqxOSoVWok0gAJa~3GbMU@SI1bUJ3%@ zI4BiE(_Ch`5on7+zFZ94_@Po-^sI9mzARm5NgLShr8SShLXqI>^^WAGSX4-d-aF9>G=B z7h2zYQ&)!kmP{W$!p&YJW7McUNi~U%T{Q1`+`TM9B2jw2nEB+ds^ttjCdYA^?Rclk z?4n7&FKyWHpc3e39aJhx1kF1)S*=%@<6K&I>$2qy%=7-^H!&I+bbsRE)Wm% zXb%m4lr97y;xB(hIVmER5U}~}2QVMy%sl`;U9{xGPwKy-(5Npsq5UraXIVu{(LZ~E z_$fB%#Es5P%~KBs4`+&TNXj7fiu6JL4O)Z{dNPnjN`PU;Dm-&KU=`M}0s{QwBce%7 z^o$Dp3Y?ttb~OH~dC*5RxHUYulBTz9 z_+#!l8I)5iAQ|%On-pClXx{uLI`Dd990Si8`yT1D8{%leB>kDDRLq zi}*y^QpG>Y=|eDCQ$C1Mv2O>-oLGf6M)Hh|5M5HuhviM__J7~2rQ>0@O&DlXv2tZg z<*m&v7VR#OC1_I?Fy{*e!oE#AXH!zf^X>UI4SF1Fik7GsFm1Xy|FL@urPnCj{9j4?R(-RM-je@9m2EH^xklFwr1u-&AuT7jeUW7X z3tfC9{~ABH0z0ket=*S?F28~P85|v-fBr7??hKi%9ms|UbZ1K*L3UD)H?chn_$7Ehs>g*4J= zII~1mm{#>>87$O%aGkoz34m`}M|SiWNgoJbA@rRNEd~e2*Vxl)3Ws%Pt!Z>}K=*PR zU%7;3a4a>Tg_%vO?Mn^J(VK8r6;hKcU{1{oW zZ_N47;q29T=ul>>S3y&{oU@^P0$94MZG3^na|+E=aI_Dc;-BX;qRvlg3~ z2U!al_95h^Y034X|%1g{5@ng0UjgS zaL{TvU^UEhCo6x^O6sq)3+UM;;wnnGwf%pOqp(Lvn1jIC-CA)f$mOxwqra+AdAw>U z{=#{yI#aSV9znF?;`jhdIM0Nzu8c+Gv9Yp{ODlXcg}pU@@>!<2%2iqPzg~5?20C0V z!`hjil<@{+k5U6IN14L_(1i%@ilKyD*nI0nIwvD^+wAC0^2v_rJM;AVVlH1{$KcN; z$(d-0i^U@)M2A>&SxJ`D@-Nv>%blFl5)vlTtNi5j(#pgeiEG_ZYEvlT3_3Dx$g~Xc z1P&dg#}ioPej&K0`-Rt^aqbt=$ph{w$ep0*YPA&TmAvkd8T-IH-Tk1KSd}w4cD4ma1@VyW@E!Ezm7DTCARe z2*b^SXO9{JLCA#GsPd@^)nY_}P%Xwsi)U_Ew(LKseh#|dht`OeT za~v8$z}%T!F93RA?kVu3ZB(I&Cn(01eF9iwprvS8^8myASca}>2q$OKS{*k?WQK$& zZYjD7N_86_5W>J$L&-S9i@>swYfcE}%SQ$u5#Uk$MR-Akf@$uUG}CQ!KLEbUqm)eB zeL=@n8H&o*GU#ef78gnmiriFyXW!;yyQZWfwx2yD2Yq*Afj!6qAoK(Ji%d#}CK zQ%E;^Z@GQPVyk!wZe6S}SON2WX`PK1HHwCkgWkLzIdEJ0{K0Zk(is5U$SN9#*TiI0ZkO$-3X)d?X?Alec^4u>P5%n=f|!qU=9JlGVzMs z-mAJ0_INCFktn0o)80skR=Fk5nK_E|hl^hlJBuB0oy9$G1*x23@)f>_^iNKIDV%tb zmilv-@?H>J1AW)JM@=Mp-g+Hfp^q26+W^RHBl`o*BnRzrEyWga(O4uI>+wt?K&jj~ zdan83)o8ydv%aCzZNz+!KFIr&EqrbgF`rv)-N}iV{uk*~2xGQ!rlJ{L{RAyf#o0!A zz^48UW40A~49;`W0^5hL=)wS2J&USFvFbdnYG11QwjmF5(Pg%26(Ugp=A@8pzLsE= zrZoTRNeH6_et&kAZ4X!5KhAkK!sgf;JLxRsSl$Q;t)QX4*q}^J^9lO5o1GY*DWE5U zs*1{&hiXqa87sCfI)oF2)YJ!8by7lSmzcn=_s<0ySjTR!zl5|tL`&58A@RVzELirms#Ws+4yrTQI(X*%mUHm~ zjrpB~`n#K)<23|PZ*SAbywZHyXR*t@RCW0#XRkj$(HiOvi&BT0_Pb_uvD;7wd8j#N z+CP7HLZ5$P%1~FUG`V7^gDyk;{UT1x?s^!O^?1v`XD<_O2c+Qk%?TOCJCyD|MU?lu zKG2n6|C(1j0RGh0?gW155KyyrWa0q!$Cfs%@uP(Vj5%JGp6N3Z`#E0jc%a8f>;i}% ztwM$~o<3aB|#_lp8Nt1&rCq4IM<2Gdi7&^!D+{4c`DGi50n?!w}cN z?hoaA>xyJD&N44-PEuAf+R2}9hK}h>y4PH=f)2*R=88@B3%+>YgS2!Z<;F*SwR-@P z1=Rmd{RUX1_`zx2L zj}4F~l4!RFp>?f!gabO`VM@=dcv z+Zf6YM9e4t$u0-71L$XAK3hptui|L(xUug579Fi{! zEiy6A`3ciRJ_w%%J3nEsuh^X8TGPYx>WTL`lr!gj@;nx65i+8k^@8mb%uT@{&7#BS zZ)jqrN$4b_elIjo#ct-S<`MHB&k{L<{v}h3qWi6^LwlciiWFHU>1e}BA7B%vI-|Kx zsDiG8g!#N7k4_8+DYgs73#2);S9=wt9Cte2@;^4!%{)79x0%X&^I*JGGRNNtNrOr5 zOjmLzxBUgBvDcW_j`rwTvB+75t&~S!iQ=3SvR|}ha}pDOp&Z70yaZz1Nh%pCo?8x3 zA~I}lvj{;H(h*GXQm;WLrQyWsRC!RTq30Z?$}y_^;pJB4n0dx8U2>I3jDqp}j^xur zzrY-utIe@7M|X09kTsXFLXa9X+s;20!;%c>S_(&=a{1+$1B0JaM}1Q)*B|t;I$;?d z%w&n+6=FWHhH>Isr2)-_vM4q)Lo_Q(kYP_w$JOplmzsXd|sv$nNB{ylvE>R0%F zV%$pRhN7ymbUk8v4FZQV4}=wu7MmOIhp^P>Ul&q)ER`w^=O2Q;;xly42`46lLxcp7 zZ|?I=XHq*Jfkwenlc~u@(Iij^6LpCFOL-GUA8&K2IqCRC4)Q|08coWin3O~9IcOv1 zCzk)?T;~{vz()6XK3oBn$Iw)zbu?1bH#kS?*&~o+#@Oo7$wq}c*Mjf8U5o0ze=r|B z-Oq<2XA%S#P%0T_#5}=kO^M5?g9CnccJRaL?o(pS7cNsGbw1D)$J~a4)O@&IykR63 z;$=zmJ&!ddo+Lb8`^kyNp@;`15yaKz_K#@}w5m66e&Uz|y#HLi-enF{RVW6$rnHS2 zvQ;wVf%j!I2O?(tW10i<>LM`jrI#S{E_2b*)19xVL5Z1=dS0{~wWlVInPVvI=g!Ww z%FP@O)E!e|=$OUOGTiu=MCZGHbke!~FpHsM8bjA8hIVFlCWelNGTZ*>-sSq3#phwl7y6t%G3$|H!w^w@kE(IJ1Tc38iOMSZ8 zFl0=Z?V>75Dwf(V-f2az;IWUG&(Ii8<+D#5z%yt7cm4qe@O*6mgNNb(?rV=)E#%uL z3iJ*vAN%nJkEK%Tjdb_x@^K1_7${rrDTaMMY`C7yQ(|S~Bi^jyAz7xtpwuhMmV4bG z-jfbuclhNLOJl7D?WjM3pU(|q?v~=0i_m;_XiFdd1Nh~+K|CcL#O~qsLU^rqc!Rba zUcMW=mmk_HHMMT zu3~E<#Zvhazk^*#Ev75@{07q=VYs=fvi~dww_vDXV+*P?pZ>Jta>ua#I6jH$(zQym`3PlZsfCAK#K=@J*D^)|*|qhZs38yQ|iu~Q&(TG4@q%S*A}1K97Mj`=+)pF-x$TRW2$rl;J> zeuCz)w|1tD|H6lupybQ&^n0^CR;%l>#q5FN=9F&~YrU_Vo5Qx~3GizJj(l{oExd!`rknPFFWr*j2uLnT118ZYwKg*7~q~Dq)$+ zP0(a&#CCWt5j`s6`JnkXF&>gcRc6hGu4Lk`VoaxL9sVN#=l;@_bR5fM%MVdEeuznX zJ`BujQgjfW5fFnh_B38}M{n7Q{xV7l7TYHY`g@vhsE6=q5;7y>E*zGn4JxO=qC_WDfrtNGt8 z<{sUy4QWJsp@y{DdMWyS>jhT7Rgk11GUO4@ES};;UTVVnBBywgrug8moKxIC8K?L= z(q)HE{%G@|5wle%<}#hzAGf&uejZWTe6{(-uVO^cSMEUd#U2a|a*ApV7;)%3oqW?S zZsnpA`YBx@J-{mX)H)l-wiNC8bs;fqdAtqw_m#@VRc}t|uU6FG3^aR3J@l6^m8I*H z;*bb&h{bEwX1`yNNygJr&3IzZy4`tO4Ete>AjKJH>m}+TGkgQ`3v5q8u`tL<60h6V zo3CAu@#;Cy{xmhrwKA=+KqH5z!U)k4TV3#y9TYTg#EXm8xX09{ZT?HZb}s&psq%(z zB3PWMQb1~wrUrSk_xX}1(b8Sd45{X%V#rF5K8xXtWXJ>PSm^fkPKKN_2^n$+jcI)T zOV%I&7+mCysd#;v3{Gyh}GR=exu>$^D7>YbZanLK@px5 z@zu2Ujx}-iD(hE3F?vjLOjtfe6#v8|#{}u9N-ksqdNjn#gIwWh*)*RWrYy=8H0qQ2 z7`z+xfepBBio>jbrzj5mPs&xjh)Q692_kdIHgo}v!U>DY(XS?AebLJ>856Ppx-u_EXFnG_vc%VhakN=pAH_rd%a6cs{ z!jrZU6hf+;M*rNmEe(v^$=?dta>%Ry7BVua9BH}dHPzS?9fku zvS(?OeR3l5_k}r#cugKAs~2E~P^-Aq_BCH$)hSa-P*fzi;87zJIJQsVl!NtH2KH+_ z;fC>tu$VS^+S{KgW54!-%By<5RP<4Aswh3^6 zH`N^6uuk;ru7?4_C}E^f^1p<}x(}U{{Bd7IL+-Z1xRc)Q3ffoKEBVPWA>sCGG_1dX zYd^fbC!ZB+6a~`%sL{HVMlZrfDFJx60%S2kj8RX_Gj8CxbP4pP@{-MJGQv4UgEMyD zPU%GORLpO@&j0RzsNc9m{QE5bd){I6uTKbx3&g*v_6PU)ur5{^G_vevzB<*_{8GE^ zvmV1EjW2@Ne?CMd3qRV-eaD_l7~UQ@dF(4#yaKOZfU8C^d3Nsl5h@ma>FMZqyOMOW z&bJGq=DlZkrb)}8Oo!SfZVb+{!3st-OhF2m4_kM=;BosF8QCyXJZm@j!+N>GADL@| z|J=eX;a8gL-tJ0CeekuRdl`ml@%;PEDBC%o58T0_=o(>j4~ojNoBUF&z6_!-etNp} zdh$cv12_(J81M&sUsGtnx^AxAN4i?AS%)KXG{FU3>YKBFr;xkqP0qC6lMNl-nR{qQ zH~S#WY7ORD3g%3LnJcN2CO%JQe~!`DNx!nEyX;<7+|ygG(|TG@Sgm}|+0*y;mRNoI zb~ji#{gdR_@WsiAPvY22Anghg9-%ckk3!ZAm#>TI^?xb+IEU~#_YsQ8s`+p1ji_Fr zp*nJJyME(D(Cz9YwzcZPWKsr6`uBr~B_S26wmsG6`~O&T#I!cX8y!Fr)zcI7$@pPkT903AtL2xV`f>t&`HTD#QeVpH%Q*QZtiJ4?YX!xJFA?>nIG5wd zpH9#7uBOR>3(pE7N%^}@GCn)D6H`UP@Lv8O6x3K@B~=*vu5*qzLj^v*D5QD|-t0=_ z0hf7x|K%zqu%8V(uPXNCtz=)`ptw`8 zKc_3wkK4wx;{zv`hWL!HbtRQmEx`V~LGz0%v_nVIt5S6YJ8DF*fVs;(ED~jeu7Ej% zGEMcPr*A|7I^pzd0qgcS(C56Cy+!@MT=swT?^*T#Rrmg1zUBSDJEi{zZnE|NZrA=# zO6k8_YDEZluwB`*slYNVgVw##4{kp7d9$Bkp4!>V7~5(t-T(JAc@z7yc3({5=7a-$ zDeXARfxD*XouUq^)B0SMnZ z5($r^R7Iim(YGt4yN9)zH*f~j7{t=4Zfc5ot+*5}{Or8{>WH7e-;V0;UMPcmd;ZMr z;TJSn*;L^R(hTJg(Upm}%9i1+dMkI-SJQn4o$-!+-CTWqN?`p=^ql`)JsK=>Z{0yT zIaYZ=RSO`wNS*jXEYa3}vaxiLynBi#aH_bv85J=Ffb>-6> zf>XnAnk+cu8%rY!&PxR6-B-06J_zSvf^#i;`mQLAi5i7u*lD9GZ@;Ro_gH0Hs`58( zx{CMsHPL0QVhK3FpaamF^AO6r1m&M89e6v4P=@Acgs`FfA=P{$1?B6h=9gYV8MOFLrhC?L4 zS28<;le&4D0N)NMSYGQc1W#Mt+}EdJ!w2UfehBLvjHfF>Pj^U0JuZ~m^LL_woQNSc zJQb3=)p2l}`R+z4?K1tvQy5$2aF^m4g@uqebB;oJ1JWe3ylIka0xpZRD%V&MUgyQU zbTm|>d&@72GUdq`V5wM0Rd7&cW<{SUSy<1?LVT_H#;?Q%t=h225eay- z5h?L#BU0?qMkL>(jfkPMc!wGhIQPVf+rozTpgyoT6d&+V*e)>0zV3qao5l@`y**P5d?MydidC4?*r2z6$&2dz7+m|&fKGr;o zYCgm@CHKbcD^-kEst7^xyzE@5b|om_d15| zgItRhi`E(_9EcxzAU@!_Ygq;WvCWC1b#lU7!@;3dpgg^QA$@kHZ=yA&2WS0CI;GRg z1*<)t3A2X1NR=~KHm+ANFPMeyE2#U&h{SKMat_o>0o|w}{mBHY#ScSPyWdqBWZ|#C zQ(w6zXu_{b%^@kfqI#bry6g(;vJ1(3zfRuUbvi9(IxS*4;j#W#o04uv@R{e!t^L(} zkse@=+d9P)h~?QCMBUq8bWWgCcSHvM%piGIWeq->Dr?PW_O=VNYO~=dhcIh(7iQI# zFf)F#Jc%@67Ss7Js;@N>eXR-WA}OSEb5Q5zK-qGbRTSL-whYW?wdVZ)k&s%%Gyed-JL{d=?H`4z`=L96|Fs^huY@%)P8xnQNEzT>&s_S~%EYpyg882busU37bf z&DD70TH0eKUx^OGDC;XIRDtHNo@#HQU1=M~Wq;67!~TyJ)F24fsc>`#28~&3-sxIz z3+x{!ZB_fnDQW!m>BN;M4xhM+-ftSeiQT+&X@<3#%O7#Sn#=g4@qd#ZPT+IMNeAaW zC;*Koe;k^ilMRYzfC|Tj+5~-hJDi~Vv!6l@nDL(2ZBtUZ2~zcgsd{X+77hWcFQDp4 zRwl7^?}O^nEc=~S{bf@!j(dx!|9KK%WYp8X!`!nj{E^>UT;*Xz&dI?YDq^)=*|I}% z{tf!%WwhFDFHkcd|E3qGhpLeTm*@Iz=62yt?4qFve*Z(An8!O3fqYX(62OMl2yg`e zJW&A@wG0o1B#f|^WSiNvnG8Z>T7_fxcOvy-8oRf} zAM=;5B&n4(52Z%X8m6>LxgguBG^t z5)M)=k|tyJsr zT;9vFQ7pzLFu=qd3C2$$6hdBUCE*xLV_$Y!NzNs3C*^7#5n|=S;UrcfUMW_tBMAKv z!Z{j*JqW^%xumMsh#GyU%1um|)=4XbS02CIDMqec<;m1e%mEZ zsF{kD>cvofZ;{^WZRR~RLE%jOC+S%S??0V_Gox-I$)o4zfglnEM{x*;l!`^brb-33v zLdtoVVn4B&tdaur`WwMPCQiYF#^tn$g00|GUI_~V0D?KZn&xoMDxAX=%fuYM?QUaq)a2dB zk`AZ>a_PRCR4#Kt!*R%m2_nR6L^U4}V%txF5NT!W+78ObikXGJqv~H%_2E$cgHMR) zjio`}L8{9N0OLnC(tUU!?mP0mJ+NPqy)ySvpIUz+y#AnRlGh)+MG=GbhotcDB!xSc zI@h0B{ZI|-A6jYJzSOz$ zY%Io_L-MpPh?U3KkyakCc~S@Z5QOWOI#-@&2*L-v^4#xHD^D?1Xf4U8a7{Flg#q#92xz6TUSX~l)Z&s`2crH^%4x^ZQ@qJY>_;6>hi zd{yC7KJoj~XhoJdS@PDtI9e;k+Lo9WT|-upWKxsY)ehV?-Q9_Ex*FnhPoi$-Q#Z?? zn?9|Zs?wUfJCoXK2ypAmmh~}6X57wJf4n_a_*0VInKdn^Pg7&z{Yd8JbG1if!29-RVh60ZO-QB9F&}y<^^za_OXiWJfQwu6SHhBkdp?YzCW#VK2To{8 zsa9ti+zViy8g){oxA$qx8kOT{RL*R&DX zmbFS4wYRi>mr_%0(A0MCI|tfL zzdZg{)b-BO%2mjb67;#B`!(whOLJ7k_@b4Z&4*E$UG*~Z2uzW>%{=1{r8z?Q__%dQ zR^uS%m^(FMo=ZJEi+Xr3^lXT6|%j-TS(6@zUxVQ15|qt4Bs+(p4+4La#VZcw~(Hm9<{gXa@pQ1rCZkC zm5Cm;*P{4y*p{`oQ;*s^T(#G^^OlX@6aR5-FI{LYr{&hWk2EaNr5Dfnd^ZFwO|86( zAcsNpgbb!eqsq{~nG2PCiTI2y>EGM!F8y=s8iYfOada@a*;BF#L&c$Nol4sIuTOz& zZ6?_Yiiml2F3MI4n4l;Vsfg$E--v_=e$how-%)6oJ09;d9G2}&q4d)NJOAjuhPm#z z6kuBn?|1?4AFlAyx*EofHId^ymcAlS^eUvCkOEU+G`RxQ)k#l=sATj5ZdZY0c+uk9 z1#Rv+KE+ox{dj3jN4V*zYA~WV7?BM|v<7#D2G^wyM*99%DK)NAnPg)scTk)z6c)HDS&dE<^sSWu6SoPUgdv$yZ{aJ7=gxyV|z4S+**lA-cKguOF^H${#;qE+{ahX8=6zQz+wzb+aBx`1g)Yrd*fZNx=`A*A>vZ zPEQ~7!H}$h?&GmZ_*2Ga#{XVB{^$4UA^zX)nHm2jHvCI^jQ@f?GvmM7j{nl$J;Z;P zZt#E4hJR)6p5niHOlJJQwd23BpojRMIwmvzzuE9#T+mbe^SZ%5ueY85dy4fO2LB^%_*eGo zG5?RwjQ^>2{5R(Hl>bL(#($O#|HXMd=KpT+U*O>Xp5lM+sBYxHo&R&Q#UB+*Gyn1& z!xcKIn;5Utf`0nK93vg{wERCZGyeD5`9Ev?75|USO#dYg{`d8i|GU9|wVnTcJ?8%r znel(m!T&kg;cwSxuDnXCy`G>Q|=ZmSL{F&Lq ze*(%^WP%?pTW&1!;Bi6_PIqVvXIA@FobWx#JOf*n@;H~G{h#^eOG(bff@88F+R1)m zwv9=;cq%O_cFDdkYq{w0TJ3Id2o~VDdOQc87IQ+BDkTnE5Je6v?PmPb^slLJHvRLi zvh{DkmiF(MRoV3KU7y~+dk^{}{ky*?oBmC@&ueJ?8+*_e_V4t2y+-%_`z1&3-|8uU zqOZ=6@} zU)~?;Uw&5o+u1&TSC?;L{|?S-{HA;K{_Rr!NBS4+olXCKe#_AN_tb$~*uNXs8d>zO z*3kPm<-kAEzi2@={X1;FjX&R+w58)WKO6qMCfT9&@6<_uq<_`<+4S$Kf7trBp==BL zciBI(>EF{`djA%bWz|2{Z-Wh)_1j9DetX&c?DbpM^ z=c26m|9>|A-#;BayR`?iXn4IaD`*+?-1XR6Uuf;`Cw8WuYi{NCPNDW78-TgnsTtZ!ji(bV zm$rGn5;VT6EBythEb4b|&QRZZze?5Gu*tPn+VZR1%WpA%>__}@@iu~jYaZD8{I`Kd z=s1A?{)|7H|8AkyMnh}u$Nl;IH<{Y|VUTF=ff;{(d*5G&?L7+Z75pFc?^ZCuL{~o!P_+cQ(zhnOV@i~IpdlcH+?W8|r ze3o8A@&Q`7@1CtcKIahp;{g8B6aVb-8BDE>hSpxG{PV|W<<-Ow10eolru8r#{j6-8 z(q{Q&v9wuQcFO(z&Vk7uo_Am>ydU~z39nL5E)da^3xqAVrx5v?N+zh$U3Q5~ORhjN zZR`*6QI2vW)1TY^6l(uIX#dOOd(eLO{QbYnY5oA{)9&2*^XDOge;mL+|F}PU{v1NB zjfU2mkN7`W?^xm-wg2IKjzOKpYIpadJV0$ zAO7c$PYbm-8rpjx(|VV^JX+T$RUUoTd&}k9&i+(*muCxaE6JBxaU|0|#ez(S9rk}O z-}0#afBWIS|Iu4jzWwtunm+*g(p$Fv{5gZ*-wg1tKkCn(KSk8q+0fbpu|I$QJkm__ z2ip6)>%#X(>31x(u&FQP*Su-#k53%zBn+bg2> zMnik;2mc?)??%#3`@ni}=$|6Lsf88L!mii-3G$oZ-wg2giT&yFyMg!tTAMoc|3H3I zd!wPff~?nbncxg_tC5tQkS`NB=X*GgOd zt5H2G|5OjlH@KJ2qW>3+{bTeSpQ(OYd=GT5m7d=23N|AzIj{MEO*hTThBeujJbEXKF_<_rP;vYDTm>Zj*__gd-s-@SYm_`NdE z`-k|yIaB@g=XG;kYo#CW9+sc^QBIG_ALw5G|L^q62EPwoDU)^mshR4h=YRKF>G{8h zj7} z&~KP~t@QkKL007}O2f(Nn93^+KBAO){$m~KRty`ckU9HtM@f5o;_P{M{aNu?mi<}vjC+3ozg}76xA~Eb`04tOJ4^?>=bWX| z!plj6e-Fna%~m+dcdLW;i`M9a2Fb6I5c1J&lj5Y$Ln-~S_xIs!`#Zo%X;*l&GmnSl zm-o69|2rzU#9OXtDT4COILrN?a6>5l_&=2m{*OHEC}jgj(CC(V(EQ>8D^HUhe_N%= zV_$K^bj}~_x^%hM52F!PnoJ|+O-bvzM3Qrc&A|QcS}L$_S2BILXNP|y`7?Gfbt{$k zVvO_WVHhWwBDu05?5W*>e|c*M(mfh1y?S#p*}jU)g^+O*MRWrx*{Uuk#mqUIDI#Ie z-*7dSR_Uc0H%seqi%Pe)b{h;}41GLZB2ZUKmLiBSfID@#?|)hWA&YMk`zQ92NV=BZ ztw=9joZuh5+NULxlx5Q9pQ+8tNL+bhdTI;ckGHmeVh0YxAV9zQZda1>Ow;rJQJ>+S zcX$iozhj5LL#gwUKEV?O^5d@g1-QM7OY;p1Z=4UK3M4MTr)WH4%Z~=AEeFH5`Q?C4 z+Z{Aujs6}(C7UCQ{wD*82?CVew9S0zD98>`V{Lc8AeFt!o?eHo)y=8p+Tp1R&@rYzHMR;iCqm%`)F1P({ z?n}(7eEku~!MW|vsBhe{PmfH>REd!QO+0CDqzM_|3CYgvk8p_4z|7rYGV$OhJ+n<4grao?fue5=g|4r$%!uV ztJ*yNH+#SkV9JMOmK#_$nJ%XsL3qKt3Rplfyxp_JCj#%f0oo-If6L=WDu}-c5KuL$ zSB3E9M;Br=&3;6p#IGPk+IPixcHqA~WRumjg!SI^%;2mp;O|2~_ zOLHlX^FUF$6cc}y!hox^e?c+MrL*gKwSZhn@d8{mrskWuJmL|SEfEB&?Y^k*7?ZdC2AFI1a{{F*wxGcY`#8VIi% zh_$`-f!LEsvzHo(D}PT#;*i5#2f`}{;`n|U2jVY>%Yk_57i?`!)&t>h!DP;EgHium zn!x~iCdzz)d(EEgT-HZ1C& z4EoU9=eVdf8OoQ{Y=!c!`kt&Y_v|$CmE}uj`zaNwfX8Tw%YH`x9$MuFKa@56uxtIy z_lxxU5!d>Wtm{Wz>$`~PH1%Vy^Q9>?nY}156$?~=oOUC@a7BV3h<7|=1O@7j4b*we+>+ZqC293n(%^@Rq`b%FejIEY zUpEeL?)Nm+Fr?0;UG{1r^Qs?suQqd(+NX*2<_G1)gZ~kIhM?PWH+4+n87Nn2{^BzX zeYcP&7MzKR{kRrH#S=o6RZdb71LmS1a4oAXAXt49g``zS1+zK#W$yih({@s-A-w~# zCnUzfq`Br!y*l}Vqxc%;lqW|~tlu|IhIo`^h82%8=cx`@(0^q%39nI`3S4U?v-j_ZQZ!&V9qJwuA%pU-cRJZ(EB1n=(!nAgsy(V zNpEjvdS5kFcDC}z6nYQ(uZ`Z<{V2<%g2d;0M-^zJh?BfTwu8HbEUkz&~hgKGSuWX!3FPbVovlI!m8)s{{xe0emPB+a_E z7i2W|)?J{si(1Zb2_|69{!cOq+r`x-ky!3ZFS4$Xv7y&82(X`+WGUCURUKtL z$2pK^pgb;}7?u4k84D(Q$+V8H@q4(%OA5pQQo;GT2X)*)FTuqNVDzp)X?>w)N+ zNU`;4O{7FVP+w|yBvS~yNXGJIMy-Ikz)RUD*yS%~+CfgpB|2Fx!0!C7jc0AKK9I~zxX?u! z>U{>9w#KU>6>cLMx}kO8Uf+X?`iRgq{pY+4WG!Lj8C+O_(Xe(PgeXGaN);5I)3a~{>FtH z^3unbVm6imeB%Y;f5wQF%HU9&hl@qWqY2u%30FA zd{*b#?_ZJadGQ<0lIi^o-0^cepN!iNCU59)Gndv3ZOTSg;rN=GG@_c>f)mhP3 zl^=UuHs_-gou!=P-_7?!F3*wFb)5UvNJ4ZYcIyI?|t#GTCuO6zbeIJ@tnxm{gr-s37tG{WUhYL*;oYEy2OFE8(e`uGr6T}%mqMz6go(Kr9wGcUw5g{_? zwjbPw$3U)Sw2NI9r=D>`4xVPMj_{@uRK|bz~dRj%)*F&7p9* zZ;6SR`TTZxycHN&fMV$L!<=I1joUixE);3(-9dV4DOxxl@pPPDSo-=hND?pdM|pgyR7;Fe6V!UfO%U?Rg46*TFY+7M&$0d2_DU>Cizq=x7%A;wiwF? zh2)>`WLHvF!whr#b;$FxHS-qRs_E3nc(sCh=5d=llG^98W%*3*HuKOSuIF8u&&M6| zR^NZss7qCq<-{d~$I*btFP$Ce6hp=lU)l|o%l=32oZ9=enLq#Twk)KpDr>FypHsh4 z{w(w0RNje@oxOm&nw#`;di{r8>f6pomZcW3eQ)jNds{c(Kj`NB;&k7&`G7oyTiAX! zzFW1vyZBuC{L6RIANB9WuIEw3sr=U6d8yMC4md9?wT=UvyK6^zFad-)mm!q*r(j2weCdn{ePnuQZAt2KJp^3ELZ$!g1WT-?W zE?l7E`jtcSbYHUWU_9~|lL~Y$0sU(V(0d3dBNQ@+OQ2wdHR=ixc0A$}ud4A??RY$Q z8+D3LYXeonIJMCS+i1|*D8n`qhhpc!RLJ@b*x%}u6(#@#5d%i8=%t1_N>w*Uhf0G; zGzD5lfSulT(CyP5xyKsV=0%7`S_NkTb#uoUsY zuL8S3T6H+Z{sY_e`;V}nmBHj5pw7l3*hcBW>FmumGn;npbfjE)V#G&iDA|?_B_*{( zDAQUjUbORsLSuY>Eu7tK23=uCe&64E{|&r{AD<9tTsxfy1hKf0Y(wM3F3PI~6zF{a z5}!v%_HXy-C6A1+h3YF+rRZW?*@li|&}w@g`lOCsH})0tIzt+0=qSmXI0)7;@{NUc zY#bOoLT1+(Oz-MV1W}E?xF2TEz8EZqL0s6tVi@$0#V{xUj-=Cx1$+A7&B)XxPYu14 zLGdozy{=S#2E-yZF;|i*wzT=+CcnbUYmc=-KW|-#PZGV!LH5KS==etH_s1fuz}97b zL{OjtABAw*wr)F+qA@@ys$BdcW_iG8Bo2_JO45{iGbl<8cKtMLPuKiV4td{dZK&7T z38(izb$?i|I2nG#3^tmfhKp8gzlynlWOJ}YmWh>7U@V0{1sr$ z8LWT6^u7Q4#cJdkYGe^Ma)#E(snp0sFns?0TaGRpY6mDcm9W{j1m!_)`)3}-It(pQM7mgv|mfTS$xF$D(XvQu|Ekh2obH@RK%KfeMwnS#k;hIr(z5v3tpTKb+VL z=)QsYW!@C$wsh|+JCe*Ey{Z1!g3WNFTJI+B=YO}1MNbv4rj=3zwc=m!pdqyM01q)+ zAv|lE2zb3{AYN+^1iUWP@VXVOxm~O0l9308twDWkNotz@VfZGRz+wcROL-5oUPTcy5 zf7}1oTSqIJZ^rNH(#l<9NN0XU)MBIdNUcV(c|kj|W{h3t5s$A3kLM;i@fhKb$43P^ z9tIv#`x`eIv;qsL2b)*^+ohfFsGYi$c2-^P+Rk~<&KeR5I0XkNaSO9hN?r?*c(laa z2U;Fa-)5d+`+5ym5T(XIDRNGWnFCF2P&N{hKbAR>+`%2mH^i2r{bZCY_Mv|CTXeh| zC#Kvu(%aTLi#+tL3L{SwM9h_Da0L-nAlduu-1fhF?cp@2E7d^Wt!@srlla(h>@y`YQ+>0YVQa_n-SBPC`Gk(2dYicC;VYizpH?7vA2L z)Wq)g-&%tZ$Js#VcktXi{~E(!ZxA3j4hpOuh2$y{QX*i^icScHsTh*Ep2b zaqv@gAIkM#YZ(E;<`bnb9{$G5Y-2@5uZ5xw9UlKBn=OKNGh%ZFsE&rsyWc^TX_lQx z{cV#0H&;y{nHgzC2(at`Oix{JBI7-`c7eN9CCLs#Wb7+RT0wL(`{@@#Ko)uV{v8R{j-_jW(}7msB$I zhVi_5J?qhMoj))6c>V}V&9^ZdpOo2=y6r2P3kTC61a84W2+XHh@x(;uthlW?!_3=(-DfEshV#*OI*BDejJ94WTWAxge?Qx{BRP|L>6 z*~8gP<(KNd2o!#R3Ntq));No|ueaz(yaxnhFXTRg$fD_5!+*)E|HGd^LXc~?5%pg( z#6!Zf%BbCmGcO}2umYhv1bh%vEMFm*vOQjlg&-nI6ze~D0_ng3{BbNNoSKSS&5SCJ+zHM{w(#`KVGPRe80f{@p$c{MZuU} zWM2oPs^ark0)pl7s3L3y8Is4YGY@D(@1A-Aq~55l)<{!NR&oV#1{%j*Soo9AY}(YxQVb?p?Zun)mAsaRS`LVnq8&=k@sQL28 zd`|?(6=dD9f^Y26wGSj{FyTu)XulEcKEBlw3Ar~Hf6EQ-WesiEP>X{nKH9A9FA7cP z!pG%74<^duM9YZ41B{mSEJRhf7kLd;@SS0ucBbFuAg|08g!PYbb zzpQ0IydV5Xb2YbcDHh|3R?3j*%y1C$0=a{+3{9<_Wp4!`OPiY>wmJ_-zjDl{m(~cu zBRI<9qulmhv|vs*^cH+6Ed;l`g?qe6wy=0NLA;w_)E2`ZPi;B;@zxe5&XlHQxG)4$ z=iJl4&QUv9MSipp87_x|DvYNJ5v)+>R-wdRAzv1ugk}3}VW-wwH>K%V$bWe=0SyaH zS}uCn8rna=23#pX9Y^@Ko{F;>Z0$i$?O`rx57}~pk0X_%)HCNoIiwBi1A;tf;G`>W=g zG4qRU)E&!L7&|etVZ5rMLNyHC77EQB^6dVq-UC*>E3JBP>0;l>)R{m9YLJKtnYn}J z;U|LKc<#2(9-shK5g_REGzleWhZ2%df&`^SLWw9SLo6t{>FHG&)x}4ZT<{|FGy~V2 zbk{qpB;WYoRo6T3=Has!==-jAy%QPBRs#00 z*rf_JXFhiC0bqYaM}1#8TP|^%ftq_VM4`7lVFOV_q?rzaHt~HP-Bwl8uTSn z(ALabYkz!Kf1JMz9_P-3>jO7?6aw#Zw+JG7L2l2{vNG=_5B6xh52CWonrl{@Rr*@`twBZu!GF_~40kQL!eBSD zlw#F-v1{c5h-)52N-91o<7KvKx^P|;Gf)9y}t-GA#kk{TuY;i}rcJ4wh;(tpX_ zl888GQBsFlQipBxw%<5Y}vDzS=IaZxy6Nt9XdVQ07_$jyy-U{oESI&N57d?nKh zb{XVeTuuHJ@NxjRR?TiGILm6x-Jj;bKhgN1c&Yfq6==%WkO(kX2xXP%1MJ_>7LbaN zC^D3{gLQ?d3O#G)!$+|+N96_Lm^s#uEB+$u4^9<4&D*0xAZ?>R$NfWH>al_@1`LAWrS^pxzSZO}> zIvL>v&=ajLk%hKZ!v&^#bJJS;oATzSclpg?t{1~wBhto>S4Ak?NWv@lX#Ptu0vYeu zyWY&?X%#TYr=rbJy3D*~+)!nF>QEu0`mB_*FSq>;9#fpbE>@)~^OV=5Rq0Hn%vYJ` zyr$`Im86i})DQVI!m}YrD$Q_=IYv9MSlno6^CAtJJydD+#vCc(Q9aF1^lP8R+GaG$ zSI=E-Qfc$w=<0bjd6N`da$TuI0TqZP!s}Br%JkYc)bL1h@7GqVI}d7GO|=G`+@@*j z)za4{Xx?0jHj**$x?Xe~cb>!KQyXgX^*{#RS3~F9^T>D;aJyNlwYhc;->$sS6{WY= zD3kz`!yfCBQ778WZh&YhhUEiXnvmXg?)oKOh-uvd6Hr6!#rawAd_O z!`q*kwffdaeA&j1`t^@$sP8UOf53tI7!7rWsp^N1`_cXYtgRT}Ei-}!Iw_pFTEkJk zr|3H+<~gfvX#V(!hGxSOM)NKQnwK4kXa+K)Db%ym(8b81c3J+HEG99`~%-ddco{|Lhof({x_=zJy zPP8x9u+Z!wwle3jImVc~x2CNRvTEw)7~TF-+WsNT&3!bKS&x&p0=FJ~x>C1z`WLpS z?YGTp7(|g)H`tDSg#2Y*U_~Kl9`HbyEh5rHtqXZsbOG$)u>18xPbxKI=&ZlNw3s_t zgV#%k7s>?BPdn8oZ|F#JKv{UaOFU}4q3i|6_Orq+04@zqyj#;Yj2^M}w!Mid)_@?5 zFXl@Y$og(Wa&RjTNqcpd6+jKtc(*;Z?u-6xQeD%|{R=5bWWz`GuC{>YjgMze7WoOH zmgN9-+5_%W;o)ManI&VK4vdzHx*~n9z8^7z8T_!R-;+~_)UIJ zM$5oA=vBHMe---Dl;vdTs8<1)DtDIjkI86 z7^q$e{OPcQcuDO9qO89KNq>7yTRrRcAvf7xG6*lgBH85UfzuA_*+Nh##rqts_C~{2 zyHDEMZ|YdVEiHfJZ7zrg%+;^zh>o+GT4-;o*tIFJ5S)rtXCFu!A)0TIW_4o-7?hc$ zK`KlOX()|HG11zx61s;N%Vb{=Dhf1()}oN*KOpzkv|~l1f_}NCO8%&pKcW_+N14&1 z$|L8Y6(nJz{)Vr(Pu9mrSEyS8jit*ZE)i1A$qWhcw_4apU8T$ezka72i$mb#c5yGf zxh2*Y*tW{Ib<^00d?2f>v{psq1wv^osfd@A0MzNAi^a^J|Cd9Xr*Y)c7Bl7+qL0Ab z-1ePVtF6>5lg;D|3(m437z(hpw0CqRlkxzv)zB;sAnuAS`$N8Y^-DQ6dqkM+5%(LG zJwlBToFU~a8i73{pX6$hNlvmREk;eW0 zn@rBXxHQx%80+F*X>22@@*+xCOymudn7P{!>zK^Acmf*Mtgn5<*M^Q{t(=c?1&f(a z41o*vZRT#f`1mN#T9@o?RGKdj*4r3wYopSTctLiU*mk#qMgo3=Ew{bE@zi1>x;S{8X;*S!}EvfdKtXB(^}UQ^QwDex7~+( zTxf2#yc4E$yuVRK4)|ClZ+fYZldX>j%B5Oe#f_Nn?T5Fm;B#rp=dihyKHD}9+1y{X zACf6LS201W4S5FmZ>csz*szOfzQLM{BzaZZmEmKVG&=LgM=Yartk^PW)ZL4kPa%Pa zV4iqCb*^?kQml**)#|97A^~DUdpH^@*Q#3Q^3@g~;*^=0b+^s$Oa1FWO?T_{fqtOT z5cbu26EE`G8q22q;+Vz1RA&v`kzr>{sWZ8i%M54Kmy^eMU$?jlbt*{xX+V>A-OkV(*>kR~Ey zVrKfV#T2-t*gXF|8M?TfC$s!A`@N*)n--#xc-9s*KHnUCx4w<`C2mX?dpOg6!0pgD zrX3o`HFuZFr{w83W7@C`L^|u!F%CRIdmg?jK)D=%TBV}*i=7TMyFA6+Oq>RLsG#$E zp8Q@B=N&~z-?V`5_GtV~%F9kHi8Nmr{V?_zuavL;#wlKD=mII|zeHNi$V~{8%8C3q zf%GN4one{V`PrlB2>KQ~{n{Xm%nadiI|Q`P)k#;KMXkJ4$IL0;<9=F)ymw~EM>6C} zbwHy@@kB4(FYA1t&XUe}m=m&!@41UIOkO;}8UHF>e%(n~#wTUL2 zH2pnrd=KdPoAPY?WPFzN9OmTwbo7Dzo~vE*d+s^0NBKQx9+1ubsW&1{`ar0D(9})u zT^%W*`cGXRflvRwvm>e7@j1$V;4TcXRaUkP0LTo8yBC2uBck2}&5|Nohyr{>5)Dzw z*1qz+HS($PR(EFp@2d<4(U5r$V1(~)y3^Jpx1~yMDo@Ojx;Z$ za2vp2`3O;2fEpl`d-`(8)tbIfM>j$1v1QPj-ehn5Yo`j!E8bffE2zq$6)m~9^u{x# zS-ll==9?FA`py^ye)ZTILX_XLH4kho=qbpl`5<#QfG_r=J``(kfH`mv9g;U0Lh>d< z%$%C5E*#XjKn%J=3~CVNbqz`F^DT%wSfFGV;p#?HSvU*(AnEgBo+&>Mj5Ul-TM>ve zY?)WaZ9=8EluE&2bF?^4XPoKx3qfmeiqvpicxwkIsj0cS%jM86v(+y(zNTW{eB$=v z;q!XorfMJ+i3|isNm&4ayTx~en&f+$FJ=VzUbey*MIZZ_qSN%puA!0q@c*JWWxap+D;8|EGyMC2Z zpHXJ<1r6MwZ-!iqrLZN10P*J9{#na#=i$)PdHZfOmCuIJ%0As*eLp%wY5T$y~7Z+fkcggxDjMmO+zJ0+Wl zZ9x-G_QboA#Z~}V`?u5>uj%{&NVf@_n?R=!NNG3R8d?ML&|W;OEV23{wu0t(qlS@s*#2ruRZy0sMqAf+I=0QHT4KR^%dyBW? zqZdB{lJ#%{zi7kB{SEWYU!)$6JCk_Pr|>rV3#Jhn!;`Y84PM^0!RD)3{wqbHgYWml z_pdf2Wgw%X>*2+uUtB~~YCKbcItYne{66tGQ4>8giVg=DeF5kUaGA(yi=tFdZ){KLC{*H`r6d6kQE(yM9JW4q$mzGzVTDEnX&00i78T&UC+R=0xl* z_2I>D`xsXIuR@PG|HnMw?`^d*w|=Yo=}SE38t)D#)Wl!@2G|b}LjNKhx1z7b3-iB_ zPjF4{yYMm16iFLT0)yB(f=Qg%lIH}{(|8mUE_24Cu+bL;&*l-W8;bUm!xt>jk*zJdISwi2!KJg>~RnQm2Bhp@_oj^)&yx}u22{h3a{j{cROrNOIxM7i=5K@5 zn~(!dw~1?`SOg9f4wAk;+J+&br>!|#Y0kJr_3COhfRqPB4v+qQWt#YsGN-=I6U~y=X8uN;) z6ia}ZCANxki70J;`C?5PdzDk44YQ|12t6Id9;d*3KcOaZn*LNrWy}fXkYo$6xmZbF z>E1;1!IH`73{wxcW~_Wzw>{Azy(EsO{(!Gixm@rnWq8%(baGdcksfyfp?rjaw;~&U zE;h=R6{2}(Ih=o(A5NeRR@C%gsZwc3Up$wgV)LuI%hS0Wy*_rPV3lj+aF z(}a}KTphuO@R4<$u#KD`YD`$UKBuMJ$bmJk%^bL>!xqdE3f)0 z81f~n1Rh=4;S+ReDY|5RAC$&X2t?lGwIK`T8rJ)<-0>O8L8ov_bST_W7$q-ItXBIg z-A^;c6uRBJd14H5?=Gk>Q}gzL>E7~=xftD*`Y)Vd^(l~PCfv^_%FeOX|W+R zn|;I9Tt?qEoYu)cBp^_dqTaKrUN5Xymp)Qa6^y^qia6364F7=Hl-{6D%Be}TCyOq} z)4z>BMdL5Rw!pt+E%$$^?7xQ$w0c=UFL5hF+OBNbwjQ)@HT}0({3%)~{+M^?^nyFj zg&d0zNfQ1GvG|jtsEw#>V=2xjz6)7NANxGUA8#~_v~Iq9;@Ithc+#(Zr4}6RV z#laUpIK{!g_vzq+qZX)v{d($=t|U4W+LM^1@b~9$A%DM{Z{u%_x9ZN{sYfYtb&vX= zHyS!d)!98&X>B^A6?sb(frJ8~y1&TR9_@8y?Y3^O>*+F?;M7(xpI%1H!#61pyJ?AS zHA~%)r22s8{{hCz(z=3YA$$Azg(1`lkCDc~H{}*Mrg5;Y8K4nch}@mnJEUdM>hFr# zYo&Ec2Oj#0DQVEusx%)y6M54rzEl;zQWamNiqC$%GnqCn5QmB^ogf8qX@jXUOBff= zRB0b7SqO&(l}cF%hgk><3EE7tI7X5P7MjsGa#Ct3%KKRESETa`pB;ZQnQ-+ch5$`a zyj}$2^{6D?0SWOPkd&lFcK#uiJz~x}y~}o>qfBbN^v>!cmjrscf0FV&=`Q~UA{!Qy zB5rxt3EixrPA6wUsOS7@)& z#Vfza$(Ey}Pf$o7`5^vWR>~WWEz;NcI$u% z2n~9s9xE0G@xX&Ral1K)W&j7`oHz*+mt#ms(Pio5hf44`r5cQrvJkG}jpLM%Rmcq` z1m~G@kYmI!Rp$BUFkY!c-c$R-iVN)ahgAR8%J~UT%z;_dj?___V)ml+}}XCzU} zpH5PRPF01Dl7;@tpUPCBDXP#uve52IaUI(#IxQ5hYiO-H=l`+yHt=y4)#C8(W|M52 zlnszV3WY9qiwg~~(gKTC*<#QIgREM0)e4K{BbBOwrm$BFrX)bZHrJ|EqvACxVnseG zVhI&6Thyiq+?1jge3cYXo*3m8xY86#_nq@G^UOTYY+AJUzW={3{C*|-%sF%BoS8Fc zX3or<`4djOZWe!>xv$i~gAF(?mqx>xI*vGz8Jh(*ZwZF}#k-|b$dLComfX1_RE3mP ze7BF?wU_VO7t{3^RaumJ6P0>8m3jq~`qSOAR8}>$^tDFsvEH{*r}IFkzkV+S8{W0M zxCwtSKRCi3x38Z%+* zj&vp#?Die?13NV?-|2M2<0PG*rFWNA>>0YZDLBdFwawEACHLTNKFqQA6Ij()39wNU zSI^zjJ|Oh*ge(S_nM(O;-oCqRKc(JMDV~SsDoLnqIewt8z)@k~#ksjx?lF!QgBo1S z=YsgEV%|CRhq%gIXi`Na>555v2|u<%5Ek>3)Fc-g{hiFX^#)hv77hw47*oM!;QX|6 z!xxa9(;&5-ZiB7;A|~Nfs6Du1CQXGheQU{;gg9BuMuXt{$!uTzEST+1GTU%%(HTD; z&GzmpZM|*Wrz6ul{k)MnCi`zlHroAC03xVXpNar}qX`kV|2vj@ER=g8l{?K>?s8G? z*b*qUkKX|LaU>8hSK=-B*71FD46Gp68@=85J$?p9kV!)|L5j2HlY5{}u-s)-?oXd- za9=9Ka$h}8l#9gPWG^ z;SIUoh0=#Gi)Z77NPrAsh6>L1a>4%(jM>vKe^j>c*ZQc1rw6fxbB|IBuTw4j@@hi3 z6%`gE$l>+JJ`VTvTZZ4r5Ol-jo#>6Iv{ywpi0&Xt{xl^=dZ+;|pwHbm>@BO8V=!ei zq_{Uv=#8e<;ugGLOt-%xss^6C0=3`}#moG7?P0&} z4;0d;1QL2VEKj{!$NyD-#n3OJ4@`iR5RIi9f1=+#>VCoPC$u~dk{G`p1u@>KiE;U{ zXnEcp(v2Rzys#6%q3@!yw9}!1K9Jx%yd~H1acS%pLGJ*-Z@+_p7RU_6JEvT~M;@}E z6mZiA+Moi4o#E4QJqsgEYq3;BPg=ow(^y$q%SVal(~##?8qfF7L7u-19i}@D(Qzoi z`)5QqIpNoJh5=4FXAoP*t6G7uFWj5E=;p>jgn8XfuBrYV75?N?-on#V_+fhCA@EO9 zI9*lqkwCP20~zOUJw!YWDxQ8cfIM}$-8Kh20fO@dg5}7$nsd(k9)4%EzDI?$6f<%8 zyA|#i>HMLm3G7oAy4MK_$?ka=5a1zm5AZlJq^8N&MQwS<*vrDJ4**!v=&y7y1P zGam<$1&KJ*F5FWdPmHY~uTktHBuDv4kmGZj9M!Y=9dmC;1z;Lt+_qaqor6^pGTTf{m<)Lq$9$8_w8o$jq z_AS&(yabhoiff@^AdSNS)?dxH5#>mgXU^C6mhH=KSW}4_WWQVNGbH6}Pk5z#@EESa zg5~@KP2D!45tEa)_g|0tb%;v1{IzmHG=Ns*)G<|&k9jBOm?(Sc#wT}n7K8Q9T z!wVp(uSV2y+3SLE|M8c5I8K4wXrOA(a<$X0n5cFT9;!X7``y*@=-BSzIC@#SJAR^P%gW+OcU z%TYxdRARF2b7J+w;D8eQd_&15Z&i@yO9x!Mlg~9tfdG5+1`L_X z3yxUx+f920YXk!MjYCd57AMBfE0n>v?2)d{91fI_G&Rt9_q+{cr+9Yc-*H9T`N=&F z{#p*Gnj5Z@M4d+?0gs9TeC}F~FopthzrucaOHjt7e>9?nC`9Zu{U*$aDn(Awsu!D5 z6(k_m=AT!Hx@Qshc&`d?ttvX}*b)%pW_n zBDTgPi%5Rw37kbNt2?^y)P|z?kj6J2tLX&+-CSz@SI&w=Tmc13Ga3Ge#%w2Gp79_d zC!3y`2hjXMy;tj)(IT-W^6PfN_@Hd%3bJGox7;jql=wXx=snzo6DFK1IoxxC)t2eQ zW*CeoN2EP{;8`svdk2y0^!j%s?6K1@lu`MjkBe<=@DewVJ0kgi3XT0aI_s=^dHxNGTc@f1U-YlTmh?BT5jFd^9ov z&)N8;@+MP{;F9)sR@-0b@`7qiVHT%Q9T(4P%?^&lH++V?IB@Qa7WR~Q4e~}Qv*(Au zLv?6HKP%byK_GUL z5Uty92wl+~*EY+&UvWNsz_NCK@l=MgCF#BD;1#p3dX8v)Vn_#`m&LjkBJ&1n(RtBl zGdVc`RX@@-3Y1DL38mLvN08!9#1ELT;%1QIt{CY!Oizk_#Q;0dpaN7P0X;P-Wpl1n<;L*=wBUPkCr;Uq%iOC23qHa)HaQ(- z0SXrg-WYENcp5oj(se}HUFa@#I_+tsG-Q?hKU(Y4c%NL{UP&YC#`};QFA=YAbmvQb z4#$RvsXRV@yhigBE-TxuRKz-}Li<%H-Y zEdTUqKU-J}ex(juVtI8}p+mI~UeNhPPhxJO4|M;y9yP1a-dh87X*6WvdepfBKgPa) zRkD>y5S8{KqG30@s3SCqsU=F@+I#_~j$ld90I>wj##e1y=HCtSz>?-GP-^*LymZQGpf9rgzy6bW0SoH<(Mv_9s{LUpr6%irbiw!AL}0f=F@)hi2UTheg@?&#+-YkfIgl9?#=!^sROYKl# zxMr3ti8ZwP)=H{lA?TWLH9kDWSHud3^;6i~Y*Z+{!lTG}BU5n5qup zg1v?WRQ5A!I!V4H;jbGAC{w}R%#!`woc620cAmJRz6|c^2m+kMq)2|aPg-G?(6D@P zZP3jXjU45RgAL5RFz!~5CDbAP=q35Cqsw-nD!~&)5jSE-kBNaXsuD=x7$o8mIb`va zzj4VSBM3M#7Ac&)`H{Xb*qBb2H*d~G@f$>D=fF6L3j#5`{G3T;gnAkfpU=SZ`AyHj zVM6k#p26Y`*ZXBBxgtBuR|)H4mPi3f&=@VgIUOKm@cafDnhW0bp!CQ4QltkGIx|v9 zA{$6LQO+=uf1SZ8NyyiX_E*Y#Sm$!z`VE`&AgJD0+w;Lk#nxeKICYZ#IO`YbH zugkJiDGuZ84xPwq9hMvyuU1gZ!8aD0HW&lPmJfLMc3E`3T9l6nD&J_}rLCarEPzk* z+7-m%K0j}rWASV#%uERXzz-iik6%1x^Lbh^qPC{N&its70#NCB9N8`$c~qDadra_MI8 z2igtw9fXHRiN|9QWBx5e9n`WEC%C+zVK}sHS-|i;@2zra$eRXvj+PXVO~u-)6z3GW zncd}Yt~8g5wlnljenVi(BvbI*d}!2|>r9Vs0hS8;OlPSX4gk}ZEGg6x^@tilNY*MQ z?P3-IZT_;yQI_a(5W?<>Yii?P8`3NqQ*1x#1FaH())PFP8g7u(wnP#~|GQoX zqjWh!NcX$kiO@67S1Rv@OL141bC%g_tF9^a>vjL$$8v}v`c)2Zr|XT;>+JQT#J&D{ zzxVNgAq?g5K-bk)2=7lckG5NUt)||^5VjdloLBkYjN~Piuc~_jz9FA$tZELC0`1xy z`?P0ip`ohi8vpmTI@C7DjU!m*1OX`p@kFF8Ih&&u3tANC*%h4cswWcGlVxWC=7{Hf zTcSVX9r8~Dk_)xvRl7=O!E-0Ym=Q0;gt(qmbnRfG4Ah{4>ezMl&wsviEqH!!np$xo zMWZKXE1p4H@6et84lE(;P?cz2svMo2&Rd4DaJC#59>9-|Z~X(xLOe>NO8Ch8+FTYb zGEPzbG=V19er%ve&2z_~1}|VS*8N8l&rgeUe_Bv<8RD ziF`;iL}Sno+EC_Ex|Vz#Oz=TJXbfWO1?pi_Jj)CdAjq}Zfv+Rx>ct~xL5yf~%E9Ls zYGS~;h>s&EZ|o`~AI$;b3u!^`QAWGBweLt93(?2bEF9*1z)(a27R zbMXgPS~quvM0%E+Ie?4@cq=a9(C6mR!QCq=%sH|DbU&*w=OfF>k{RDVS?6l#B^?1k zjW_LpH_fo($wB#*C3^(+sDdIQ*B$-wuVT>ETFA+A=(wqIvn;r0Gr}}2MRc;q#(Hv;O z3Plh(cdTPq*r!5;6+@bDbgWk?;4eDx6pTsKu8-mRhUd7*0HhD=sWHkC!EYED39S9Mpy{Ek73T(+tBv$Lq zk2zr+?r?rJzfY&}8|M8mkGF(2mXUP@K6VYSNs1KHUB^^Qx7T%vgaO23fu_9Mlp z-)_!gdCH*UWYMCDu@Uptd5a+9Gf+j#BIXP7#USG{LC7u&4p`&N{C~X@pSzaq85prd zf70_x3tMO z(yLEW9|z$#mB}dT=zC?ANnuW$Son;Ss&Y~kOJKbV?)VDHUVDR<#82EcZWh098Vw^p zoxTj&tduVUI$A=llx~)wH{(tZI1#|D~XIr?XGb*L0 z!pqb}UsQUT&+4fZr-+s$ggR;?&9Ugm#&f|2N{^etrA-$q(o*=84C9mFP2rj-ih_y8(#uYaK#!E2Z#?^~6jmL?= zjXB6oSjN#NBDb&u&poDFv(FafIn$>1$h2<~4&0f=;TXyYNR5y4q)iorKljWz0vBCK zNo)aF;KIdB6hcLZBKCnA(G|2KDyEq5#EBKt28l*a{DBVH6o9l_KLPb=(_1X#3I2w} zbh&hx3uIN%{=&BFbYwpX$6Xc@poF9TV`bQHX^90)I`wBR@cUM!w*p6*UVtmf_7l(O zxPvoiciqC8abvFy(~mW*FvEYV z@t?Y8%G5_Z!-F+*K^x@)mGBQR_kG_N3T`s(C#J$9a?g zvr@^G3^%A{%SHh*HC_Hq;@jQg|F{b^)&Cu;e+y{Oa*G0!%9e2l<7c#jCIX6IXKh(z z1rt}DGC=|wE14sGt=eYcMPVL~RbWj5_hih9n!y@)9$k4^lL_ET+mR(TYYdb&U0=Vv z{VVHP30?-2A@+J^j`h8yk)$7A16GB4@eZkp1BI~G@h8&dJMomWBtn6UUf4@P2>!l}Av0i#7P5W&nIp(fF^> z>3AN*U|M(lSB+Et)v-bU`7>-J z8m&`}!2y&-=q-Lptu+N}QTZjiGE-%91!eDEP8#+~1UK?BH*+y@s{FM;$d6)C5KHRb zK=BbNE(;dQCfvd1$z>?0h-bZ`FbZV$ArS6VPX6z$k4nEB0?9+=H^>=vuLo3R3lMgh zr`2X&68%&lgtqLDb)Hd>NW~}ei_*DLVV=-gfFSa!9wLl0( z*(T*@NVtQI)zxAeeh><}tV_xy_A9Kx z3Q3*=_C#Kuk%lx8w90rhi*Zx-_JW+Ufnf<0!Mo-|#oxw(?%dnV8xI0$Im={56s#u7hX!Y-;lv znqfWaxM5$w9x4fQN`*#TO4sQW83Xsp)>Y5IcdT*lD~~#y^dKTbJ42Gs9hPTS3U=CK z?l_Vo6q16q~(9*Mr z*j1r3$sa-3)$g3!=@J7w{)Ma|k;h6i2R8t+TyK3pbbK>FCyHIX{pz5-GjuH0EQ*4l zjm%^IObMwKCDvMu;Ef8ZCikKt)cf}i53$wb-6OOVr|UADk=NKXOx@bkSGhYl-KXl@ zQ*qrjF}vSPipkO`NM7!)nO#bkCS~`Yl0TUys8vn?SRKfP{PNuW zhrc7xcOB!=WbmC|0OE0XY9O`pOd25W<-ful+psHL6dO>c(?IW2(-B~DKweEjLZ7Of zUM@6!F`&;HJicf-{VyC|>j(wzC&Upqw);fol`L?P7o~?6nD* z@HNT~>?RaskdXVpbX3jkI#%wDV=k8d7?7~u4={oJyMSSld>y*&C@@)J;aoT=VgAkx zq;;@8M)GrEz!gfUXZ+xg{0l#>(bfx)7En7m2NT52LfICe)+YuVft_lwjlw&c{5P}| z+6xh&SH%2*DIgFcwX-8a|F8akv_wnHD3y`K*xzcQo#Y&l`6boe@5@+lS z`%Ueu-KOb5Ah|Pc`T0uwW zv$}x~o|ygK-=*07guAK1V{;zfz!9}DCQzp7E`}QpmZcb#Quc)fYtgUPkTyc8t!a3J z{5_j_w7rh+moE=c^H=#4>*TY^AHC(RU{F2Dy1{O&i?~I_vt0BQ`z3xSO4uoQSt3mD zSQT!@dMj2Iao5xys|O^C(3JF}BErvmL8t}|ES`r#S2@5`-Izcr zpf&kaRr%#E*Tz9ZE-X1kKv%iIWbI8Uil{HHIa&MwYvQe{@UtsTpHRqSvB`VbqShbJ z&jene#3v8w75Mfz=LJU^P(B1+p{nB9c&exC6Qhcw2rOW(0vFol976SwxAlu9QHVnk ziwa+ur2ifoGNj_9QmzdC&ZO*td}$XIF;S64g|ob4b1iskCa{+)JrxGpx{MVxw2zb$Y%h|3xQ?n`Fr(@-&+I$xK`6Cw>vQe%Y4b- zfGf$z=4+IGN9Zow7Z{?+Nog2B7|Gr)CO)jb+~ zW3Wp$v^6S&*gnCjEo~oQ0&LIZ%Hps9yrK1Q)y!d(EnNi5x*K&?d!u5s=Ft_ND!U^a zFznPneD7u8(LLd`E5}>BEg+pEDea4!y0vI}lWBzO<^4!w*kEg(;Vt#|Tyx%J8=3VD zt>JGAX@ zH0PQR35O0BS~u$!JLON|&b5Banl!)o32Y0N2FE&?s#s-@qBNs2C)cf-92l>>D$)|% z(SKijw+&dF7;VqGsW-pUz1N!uRygSLrrYitRynSaeKR@OKCf0=8s0awBsaVgTHVoF zUo@`16dtR57f0?D4PMa-16**x=Y+_MfC?-c{!=pr(* z7hnUkvzM#_vu$2oH}4xw2=4C_O^EOB3r)y2oHC)3GKqQ(#x{v=Jd1b_`8^3glQJ>& zE_Oi8z0sCaCLq`8x}04gBh#DcqeNN~xmHr_ z3X^XDaYia&4Zjl2Z^!VFEC&rrM=6cKL~Wi+4?X=4k|w|VV=@B6x%fF-LOG>RZj&T5 z`|t8Iqo{}&ox;Vec61ApMF;*m3|>Qe6M#z++`DhQ*NTq4jsm(D7TW-R`Ny2+jA* zqPcuKPa2HJ#eUnCW4ZqS$qZ1R8j`@}AMZL+;G1aIo-~+)(Q6W0Tul2-7bCN=?jP_I zdE0O2Yn=6#Up^#jP1|97(~CA44AZl9p?jfs#cEHMwc=NnwbuyRGCrR+B%hb~pDw_@ zIp_7O+Pc!#QtvR(H}locHe%iDyLlO96KX$it~XDgg(0O=4Z1dGI5(3)F3W6NLA0(5 z<@~tB{njP?QUv_)bYMUtz0p`5OY7|Q0+v2mtkbdBz1*ftx!o0_mN zJH>SL;IwSs-TDq2_a+09Z723ZI^~Xr{L!f=?m{|jJf@>(#TnRjHXg&VD>8D}x9uks z|IFs!W-?g!&HGd*e4~8VJ@-}NP{O6aZJ_y`!hqC3;P*Uf)O=<{NrN*pdvT!%sK%E_ z@M;=4S~vX=9OOF=h+O6l4!>TIC#FLJdP%T&EdQWMV9!x{1~XnR4}f^_X`VD5ZLps~ zgcQR?9{ahXR_{-xVnirDV1$9HOVeQ?AMe?$5i?j%2j&bF(V~Yx=nnRGQ+sG_IqGuu zA2#+hT3TzBbA>%lF_pNtxa#N~IUI7kSpu$v#VW>BY_i~3roe2`&$7Kw22Wh3=ADH( zcyh*C2R=tU?~QHmjf{Ml4KzwVx7p>lp=V{3a=Y2(8b?0O?!U32+fWp$%DIbZ=%wyv z7bK;qIK<8Ci&G`|D(+ZJ2nB{F2~9{XEhN@vGD^^R4tSc5ceUua5uE^kHHc_TiUf+x zoRh#jpBbMzWYj2C(;IVg(5eQZTupjXr6$unVsB8cm4n6bv)*N`S#qHc#`5MS%I8aO zZn2nd89`7wL!pkQ9!FnK&cH;QL_$cf8LRhLyYK{2$1{jBO)&xH3l<_7w@Zz_sVPgaR1S!f&rEe6WI`X~CK{Kvd}+ssR89QRcNQa8U%nmHrwL)&+|I z4ci6?g9AMM#Rb-d4@%%2!hf2x08NWu6_Z1?{#pUn#RK>-J^;_?pPHAOtl1kJnVBWM zK4*r2jX|VnQ+#38*aFeCfuHbKQQEUa)N`@d&d8^@OR1oXsq_#$HI8Yfk&Jl_1zgmR z;h1hj-FOX&r^Ro#Wjrm3Barq29JC;eAw8AVfbk+VyJIx(QF)qK=zD5G{&7Z;#0$}8 zjY=Liz*3kT*sa6YHNWExy`m7S`v@d-lMQp>Y4HuihlSHaM`0r&1ccL5L$yi_-(gAP znL_OB>BC+moG6*{dIGbpNhwnIAVtE(-ki*L1}afA)Yo65IhtCYuQ8&2Hxu%}v1V z&!4kO^2ZvObg?D5xs0vZ0RdA=^o86%gn~F zMa#^=mwxJ0j!A2GM{a}}#WJ%@t#$@Qj7qhRs?4hS9Bwm`{kUkuKy^ABvirPYIwm$;#6>~6OJz(yOAk;4)lm9b@g=d|(Yul2C;*q~3e zX}>=kJebVfq+`ggqw+=YRM}&UTB(8J_`A(n?k)*Lm8Z{=@%v00J!UX`pXwz0fq{Em zCrGc77a$^dM>fRrGsZJ)!mvcBVMOh4EYX2)U#mVuHbaW6Y;F$i-991uvdjQWeI=!E zLHtIL3LK`YvBM>*qKV3bSEIMgxq8&aWl6la=F{6^`1r}4C8Lr!d^nsd+IvN>amdIC zt_|Sd*-D*wQ!F-J`S^K9IKzaWc+zEl4QoO#W`fxWl!ZaL|& z{o@h2vcU~%S-lI)!MM$*rwiR{Z#?7LG4jjk_t zNeX|ZuwBi>fzJOjDNYCEB#Ghs>aI#pW6~$&V;#&Bl2{$g1*uuRn{nw&lG!fC??-w@ zihkTXq?55R+#>%6BJmg#42D^Oe;FI*&q#iP&LBy|A7uC`@-#O?t??$%#WlU5$o<@f zq5ZN^A;#@|efO9c1cIp$kC@VC+)m0a7#KGKsSqw0U;(rUf5x{!h}a?Tdba|^XAkv_ zjn(|8?i8*)c{$0PQCGR~czkBWo(ii$G5`hJS1+-=7#$V8M=u!P0M!JQnZAAxe$C&W z-ng|BX3t#5T4M&+Kkf`TVH{mKZ|30nmU#FKT|%)DBKFYTS7tM}z+ueTL+7zHH4Lhg=r;E>;@x5}@t(W^F!8-&Ljeb5Ikh#Z5EoP?U)}~D zt0=O1mG1WQZ-cz=oj-bpT0Sk)*_hR&nf1;4T^~k>hUkA?LXLEx4VKXM{FH53+4m8J zg;ym(Lp$>f-?apc6==~*z|AC!b@bb}0aN+Zq_;?zT)=IQ)&`*`Bi7y+{CP%NdqwuL z5cNbTHb z+p^Jo7v9;1d=uW;`g{%FTf)9rueTvT;YMcOy%G25`7Ou7h+#Vtm1pi2Aep0CSvF;1 z;vnS&psH|Hv7;`D+ximBLz$rDi57k&dZdBg%{bk$x!=|lyu)dHhE5WEvi1q)>TG`r z1>8i|z0W~a+e1wo1;`XbBJlwyXj|iCT+7oTK9ELv4CWMURo1jNzo9R<_sw$yDDLNP zZOmW$y^AHblv>3U5!{s&XB1$&&ZXsXki#q=Rwv;CJKmT!hinEL-g0hPeleWsk5od> zFu-PI6;4E7OhP*X@-EBBV;Zx=Z-3gTCIZC&OsmU{0+!8g(ODO$CxzscFjgQ0p%a*k ztgao`N*M-V&VA3p;2kdsuF8ZF03N%YwA(ccT=lk=o?n>SMT~FG?WUtNH@Vnlw&52C?S8Uo%2kzzEwjpMDu;-Zwc(6H$Of!ZW^k2!=*nPhh0v?q4hB=#8 zVd`wS1-`kl7cQD?xRbQl>?m9!)8W0p^dBf6iZ&6ZLLQ`rriQ&%6cPXYRi?ar!w%E* z)&fPRgZ^_DTolr`_ct~h>f|p5gTr{1*XF-LL^-}n|96mG4345N{wJ%{&oHQIKK&aj zu=@?W;ojebSbgg5S%3MZ1~1meeUYjw+~dr%^{=1lOvH5~k9r^_zy5Fo^sN`6$rQw7ce<0YKGajb{wZ3s;Pij)jXooY3R7h6bd@E(M1m=d{sZvxPx&Gx zen$LK8kIZhVHfluXZaFIy4P)F`13gSjy@_>a?%s~qybdbE6n+Q0Y0{+^mqSJ&#^wC zCFz~+u^5g478#GxJi7l`o~|R08#lVxjy3vndG)I+VbE02~@rP1=jJl~Zw zPo=9ivdBeu#IDD0TYDb6ZkL)f{jO~ig{mU-&gzf{pA_LL$)`4A!tc#DJ~iKem4ra` z>Fm7X?JMgImx%A4gH(2Sgu3KwgyT5XP4?YB+U@J$iKlw=h0l4LXcu71d-DWzqxOc| zw{5!J!uP~8SuqGcks z6%Xx__s*B~(AhwbFY>#J?{}F6XWtNy@ifv}%NIL3n%|jy3?AbiKq#8~>VOu*dpqaZ z{EEB^gkT7Vh9LKh-d|samUy^O@lL?>Z%@}+T*Or4S{q$*=|d!$QX`OTc}SnP3HOM$ z$?_VcyQS&ul88r0SYnJUXN{=`pO?y{Y)kl0mLqbq2fhKLyQY0@#>fs)!qxn#RH$u1 zk}~Wq#|OyMAx{|{DAXIqSqOakkl1NK8)M*Ye7vS8tOn4YSB^9mxGY1%YJP2R12a84@4K? z9$fb=EaeD1ce$z+>LoYR4(_{puzd?`z8%q?hfwP-Nj2D~spS^rgdxNnpN*hmcf2G^^CTNBU8cc#~Ye$m}ZEd&Y9jU9awn{8<1vrZB|2T zqy?v$LsNLZ+q&)>wMbWMwwERv-#78fV*}A;W#Jo)>%K;MUTqy*dqzrNiZy7N-3--X zlR%5#G;uCym%@IR-@q=691b*oSvVV!05rB80^)L6N zC%eC*qtCJ=9$pti0WQ`2c!67IBIUC-pHpFw)R8%v&!|16p{Y1{_EcBiC-cQ3W2e(< zk|&DhG!9vVIJBLFKKkwsHdMOffaGE6k0sx#H818ZZr<1vpK2ZVvzQxXK^kJOgOX$a z1nqKBfuS_cdsd|A)THPLNRjgkS|5VE-}9Xn^(;#90gAplxO5;sh&t~}+)kdpe~c&Y zf2O}k1u61j%d|=4;Q+{R@kxmCtR)XUQ}tGNhs{Njx*?G-jIf-J>r1%xcZ zio4tDm}=9!qIp}#wW?X8tzyRuWvuYcUn0GdrVmGCJ3MzETzBa7Brxl6a|hgr zHQvriUtcKCT@mB*&|gS1C^TSos7Gyr)MhyS6*0VByGV2mdLTiN@z?Lh?VR`{{@|3? z)4Az4Ub{vCXchpC{d35V5aW~Da6EP*#15df_cAA0Hd^r!rd|C`7^YitcDX6!DolfDODI|@AIFexR-4-(jj+JQD1R(1|edj?C7%iVI%hRzFsecPdrU1>x#r(JSO}< z!VL^NEi7;HCgW*FE$;na%LGq7R+~d@{FJmFu5ZOu$}97J<^fb7!d} z-Irf8JyQ`Jt}gh0997TJlp;Z=Q?+^$w7>GD3LZwv*bI{f4zwk)|5U;4JB`tM5L2y+ z6stXyo2Ev}(fFt~vr`J<@3X(-qVKlnX4c_xsjuxb?C_>@+I@5Tl+ZSSj+PYjm5!2x zQ!*ooOBd86fNRR%?nXZj8;q2W3>{n$*~6zL)njU|;gW6%UmjjiGV@OuQ=)yp47AQw z(j}CN10%GXojmED6$B$yGRO-eo1@?6;>|k*-QR7rcnK)K=FLtD>-Y8c|IsJ*EQjs^ z)syIY_s1%zrnNU=_$tmlAsy6a6=$R}kO@l5P<)#5yQsx&{9kk}vHE*@Zc{ z+&YMkXZs&~T~C3H!b z>AV+YyPKnD9A83xFM7bP(A`FLF9^P%=-g9WDfQ0E!+&JH1}q}}Jwm5JJG%Mx7?fLu zvJiW$W`_kbeBC#8aY&D+xK8`Y`A0{fl5krDnkPqh)PG#`||NLj1bV_7>`bsB@+W2;NMF;Tji9gzn?8me(-3ibX^u5 zE11>kZss;Hj5l$YH~roi8J0zX(L7Jb?z9?J$|%r?uTJ5oUu9se>RfdJRpr3{X3w>`Rjy(B z?W=3;qB#}TD3p}mqvXt==}Kkd7#>{`Ve825vzir?jU76rWk>E^`;6#Ez zSRY2c6aw2g0?Rl8>o@}QI0BQnygtG}QnhJ9wG5H!AcBR6Km#iO`C+4TZER&gqCZBmh*&`lZ2LmBC0Z< z0V1Ec5%2CzJXI5X`f%l`FuP;Q{l<_}$C&wzF{6%Q+d!{VOTV>mtkv-=(8?WuKrrWflTSn1!K+!d>?e3b)VrEM& ztCYYGDFL3XfX0d%Yn}WbFmt$cb2@CdYJF0#Ol6r;lwx@jaEiC|KR*6^r+`&f{5DofQA(}LfPYh|SOyD`Y0H8n zO%*9peu7KOWE6^~E%e8CJqN^a*6# z+zx@av_LogwR^d6X`$93X=+T~IxY{SXiZ9Sr^x$rzXjeLquJ zPg_>`{hcsVl!G6`;1n5+UhrLW5~V1M=F^ntNsWawbT&79!u@ z7MkPz?j=B@KZ*}8b0ST0mCz5lx_|h2wGPT7DyQIxEyKncl}+gRiG^p38Ky9d^^2Z- z0vBz8yoP~%;5xUbpGSCsljKPWh`4^D{ zMYYEVCjGIBux5%FhAadB2!;^M6rITMeH(TD zE}Or58f%u^*skISBYk}?YnIsCn0HMD5pd|%*xN|AVvuq?NjB;1Z%{3HL^&Et4$_ac zwG1s&P4mz+|D(bQM41|z8T{j}&E)>$uKjCY>-}T6Nd|s1+^DQcfBTv4*iJfocb)6n zRKE=$0KPiMW)v-W&1oYKDF*((lswaG(CRjNk6Dv|GGK-Mzp0~4-&DmUQ~v$$QT&D* z@~mOMIdwAo|5S|(C{lNwf*UKhpBVoLjt!#!Kv+?x1MrOh_>DO%|CeVcw*U9HC&NEa zCEqHQT+DDo9p}HP|G#9szjiavj z=7TRSv+h6pxZz)`oCZ3lUJubfOM)P*D}@8emBx;!&EkzeY4sVE{V6KvTTt>9@#KU` z2>lalE8;s~7NG8l6{xZOeNICy(20$^w@UrLB)P0IC3`JFaN37RN~p?upfh56D_~qn zN}V;N`S8+5e8HZwxLz!nYyA0}`Dl$O$ts9Cng(aVenQ^g<_6(lvUNZR#qjcLNXwF` zrvgDbsL8#YFts`$XEeQ{r?{G~W+c*gI|3;c-*h}RipkrJWdXb-1@j@%mU z`e$Eyh$|Bm5UZ{n15gA1CvZ%UV1~N)eK74+p_NJmUJ?)&u?t!)Pt&JsX%4?I;#z{?o?o#~;9>J{rY*OkYTb`L70)*3bt=k)aY4v) zj1=gmM(=MS@YLSEJOsP!+oWYON;tP4w-i>BI1!8OAn-dXGaE+Mg5&D-=TL#&EQ;Xs?*^bfefr8Ixov(#4;X!#tQuil^iBSywrHq6C@w%* zQ{w8hGi*lSw2&pL82PS6CLjBD$6`igC$*+1$#81QXRfOHmva_QXnam*B($shWc89e zn0s`0n&K~QsA$a!6=~{H6~`CkD#j4yszwHEOcTQ_${U=OlnKKOT_u@Q^K*5}k0|XH zRqfB9nY0$oH3X)fRhdphgSh+)u5FFUC;mK$Nzc`^NBd6TR45flTN`&>&N+<(xA9I}MV1JkdYAMHr{A0}3P-U&Dy<*n#;Efl-+!g_KFd@*=SnP|x?vY8XGu}8Yg#~T zoW2P<&^DDbYwWKjA^~d47VD4@I9-}@Ybk{mY+!kRP}q3`YHZ`eq|JB|N;yM|C6pIm z*kOj)5zNVw?1>`JkY2yvOGYlyxoP-*&F4QgHWt!hmk02AN!UtRdc3nGdQXJ?gdwYX z27<@N{~Qe~M7#zLJK>CTo7M~)C>#eJ>-BsU4%X{J`EBQunZbQt$t^Li1_yyBa8W9r zGI@yt`%@hWydD2772SF@%^8^mlQnBL`|F+D0-^E7=t&Hs1&{Wi{ZX(C>^u3zehZH5 zPgjuPQI7I4krwVuI1#uFTiysP2$aoXbsRiDEKAM^Ffipwhgxy1PaBgQx)Bi4Bfst& zhdW-4pOp;V?KoYg&Q!AzTML^Y>L&tuQXU@7R^PVArel26%UVFyENW_6Sbf+y2$c- zfw*cLP%ioViJ0^JSho-*p=GZi!CIV4CeY=_$X|438_xb#GZ#$Co|oLw<@r$b^QKCk z#f*%yU(d|*rMHMp4HzUSb%#((1ny<1o3l_K9hQKy(oBN`+%y_SzEq^sHpk}kXKXpR zSdHc-Fm{adJI!41c`c^cuz5D_#tK+_CX^AS&B5rD*bLk?l0f@#3JEQ{kHXT2NH0A< z3|Ynj2|ad9O&?~`etvinr;B%D7xA$pB3xEyB_2UkzT1;=V6lmy9w6?r^ZB3Up z0cPx63prp?wapL^fxw#7-Eb0Vo}jYUJ!?159*^^Nf>S7pOQydeSWY z#(os*cBr3WO6<6bm*!pk0C`S2Ne`uw>!V9u31AG!ZWd+>5}JdvJR{&6w3g%&yIoYv z=ZDSdu!m%iwwg0V&!YjsZm4-uwg-u~f7L9VGx&vI4t6%d;pT(Ck=!k@{lI&Z>n=0*;)59 zri)FLR;qC~Ipj_}Xa8vgpR4uMqz$T>GYKA2q;JuWT`7T`GU`%U@Z}qG%nwNzpC!#* z3TP@m{6zY_liqNR%D5ZRE2l?nbbX`Sv4f1DjXd$`A{V6$i7B+1#xF#s>@s?uoxh;k z9g3;hbOkI~-6ytY9>YU;fWdD<7-rM1O{Q< z{{cflyuYh*o0Ea?U81yIBWSFlL78{wyS0J9m{^ULy-Y!LnJLScn1!df@RY6atERl( z!sm#>$9vRyBVmSKI?oy<*6;7=slrCw)}R`(9b?h)HE{IX_wl(TdGe``{n_YB@8duM z()-_EMISl!(KVD)?%Dx{X4D5mv)HOI9JGo`$nXl4A2=vD+olWD$)+`R(XNLBY9-PA z3%soa$|SdaFtwlD@KrP<)fxLjh%g#(v!zrjUL)mfDK{JX)7Ml5*jeA#GMvK`U6_Jv z-1@E5HG45qu`~b(Y%Q*SoR0tDNB84&?S8~*J49&Gk!4!x0xjpT1{zzjWklm1_Fj5F zp~@03+!ytxdg35A^eeiuXwA5mVz0)L0`)w7MR9 z*=#YMmGrp{tk0*CsCj!K>-MLR?7j-QVXEwVMUHCNCUsICUQ`DBV-uuDn5?E?xz-{( zfU-eN2M&tDER-Puo-1UWT?eTdxG}WnBtiE;B>i=<dloExD>#W;asR=~Yp4!*)xj>qf1au#K5>wyZmSQHCW6d!Qq#TFVZ=Xf z3Rybnn?~cI!>(=$bw%vC$`E8hmTaJ7Hjk!4rYA_I_mE5-;*WFMGdOx-yUgAt+J&Yw z`qUV779WwKSzy+=4DevFsTgg=ZKL8^W1$)gS)nh9 zdttzK=DrNkXyiQ6+?C^%N2(4dmO{p7NXDn0Lm49=;|HE585e08o{Iz|Ll}LXlSgX< zYIDw+S4SV~*|RtAYzlMi@$9K-i(xa8aypw+)7e9JS*Lma`BYOzM&PUy^N%=I34{h- z5U*Q)ratQwR=ZJ5`-MeUQ^5ozwbCF_zx8a9PIFsA?G0}ozs(b4*k67QRHi+B#o@rW zsH1yvUA^mtOmP#>gFZ_0poLFkEq_w{asD<+wG5hny08Jz7#}0Fy9n*~fc7gy^Ad)> zH;64fS0vASOb4VZ`Z{o$DauMLKhEp#psk!DM*kKZom~tMq3{oALj>3;nBJj5P&Qgl zm4?*(Ed^!7mY+wmQC)_I#n}6HYUnyyV2p-S43{V35{X4LROX81n(NowEfP%fgKq!A8lp1J+g-^-0Y6qi(d%dJV)V%*j zSze?pM?#i^p7J;EX9;Z{HSc0Td+v|MG@9CRZY%XOsO}IAGkuE5+ncUyzeVzUPN?xQ<2w0)r;rKbl#!aYO^EP>yh#1Hb z4M~{;?l^vogv1yt&}vZHjMvBSX+{A_EMWE~(|FC(IuIMFdn|#_z4kUE-y)r)_zQWy z48>#bI|J&a!A#vcO7*5yNz$y6x=lWe2CU%mKA%Om^EXxFZ`kFp&Ey1@Qx^UlU+EEH ze$H@Hp%zs?u(5f4<@Gl^pfzocWH!25Eez!BO>W1)D(MYReb6UwlR+bv*%+YzMwy1N zEi0yont*xqcr77 ztr=Rj=EibmGg!FCRU;N2Dd`89kE^ZexBdYQ_%`v!Ira%@=aW2T7xn+VQU9FU{k**q ztnp5vk}LMoXxE8Bs=}Pod1<{kg)6W@ji#FStHDzra@>xPbh(r7uyzQ*N(jRA}M*_D&KG z-rIjl&H`wyKkup3Wi;0xgs9uCjU$EJqd4 zRmv&w4feH&-TvO_wZH$iZ~Lp~|BE;O6O-kyU_TOU4e+%~V}g_QG-ZzV8{DRZNc{>; zwPXX?2xutLVKwX88$<&|Eoj)8I=5f;1}h0r+5S~<70NkvGTPP0nk~kNG@GyD6dE6j z?~kA#;7cSQ(pp>$sf|zKk8_IQ?aDa8{L3|vLR;!L z;g54+eMCw$a(3|{rvRb=-XofjB*3k7b_7JrO3||aE24etwaTJZY`+hE z7>?n=Ak}pr2BZo`uX~QYe;J?mFH1k%n|r!Phr>*ZRt-ncpjd!VF$B#U&~`m5xq~cl znOg?B5MADFn>4pg8r?$aXxDe;Id$hj@Utj)9ZZ3K%qjhN0UAlV>_DWC0-jY?^$$QWPEa*Tjr|ft3tN>*m1j5+LD)L`ZY|?u} zu45+2-Ht=F=_l@GfL6H+0VA|~_?}>7;1p0EsxMeF4jM2|#v!aN88v?*Q@k4mxwk=l zzjQNX%ey2~GM9`+kk?2mAf3j*6Jg!z0eN`@1Ilp@e+3?AW{g3`DCmbF|Grxedcyvt zqTTWa|Mq}%N6SYRiz{)-_KK)(+*=#fF-B|{>vW7k8^(GaW5|yItpv+nj{Ya~r7w(L zbBDXS2^KoR@SR4znaw0a|L1MZl;pmaWWsr~+$Jm7EjJdFd|{X2T5gr?-p3D?GRMwo zugK|-L?2}IVI94XLV7xpS7c+!4q3p{FYbw4Ngeodo=|PORc@8apvqYK6BDv@f{Xl3 zI{aP(J`ClwaXAUrA^7Y37-ng!egOl|1?TVMJqgmU>ceK&1v00T0z|w1$gPFyASxEw zf!<73*XbNB%6ILxX0&8n__8GZm(Y6q1GHJ!FEO$GzEfE+)KHoni9;vPpx28 zj&w-=%lgJW}q&l*+JEi8if18 z3vmZWFC&e$gFjFWNqHs3g3RlD_%+EV5nLk)YFS%}1Gxn=?7ZnUZ!aVz=~k6mP@4Pl z9@5;5+dvs^po}V5)KO;n2X+24RO#&9h28M?xq4TBPL(zVSp>-S@0&YZG!vl_$GPb#?~D&6&jV->7yYjT=brng(339e*{lBPVamKJv2WQ zf1K{k^1w;nl|9e0=ZP=D3pnYQpQ{{BNw@u_Bzf5t&0fd8YJac$8tm@}EazFxqpa(u z!~WWQd6ae9e>KRh{x#Fd8VyY+TR6xEDDZqDs!~avs>V|%2ftdbd~kG?7|`%wV>71X zVR4ic&NXhVhJ(du0H%|)No=?K%wuN^_MJRy`#m3Z0O-7jozw}q^#`K)mJIuGp8tux z`>|eblB~%?`7zNZ!AoHE3XWt*r7DQbT&+An{d(*(=mDhe!DLSYjXay7j z1ZmP2f)Br6y5}SAhP+}Meclu46?4fyk667C`sf;pkZ(e9L0sV`jL66vYQuP>dzhHm z<|tQ)OU>1`OJw~;e=gKy>Cgv17zmUm8ZH5OFe&-~shV4Y#*6+O{?&_rjq$Co z%zmi6Wp>G^K+jer8W`z*%gduv88vvS{wvS9s@+_u*96vULey&l>NVQ6BY+M?^x3sR zb(Th}`b!g4(R+rSlSLg|;_k{)<%>Fp_tJ#*gi3#h?lpA>YeG#js@6V97d2V*9=}bm zcpv*j(c=@5{?Le$H^UyI2z@Zf^@k|!N=fO*YRh=F%+iu6ZpoA#oMdp)$CJy2Z*$Sx zUIlgXMb6vsgIg`9oCUsC!|{d>Go@7XrLewIa!(=Jb!HHoFC~2e(1Orp38?x+*Ju^C zAGX8RiRe9iIZsb45APa5KwT+%&(PObJq`W2Y7=(rxtjnp1DF}WNfbw%Z#@pC+I946 zgCJH{+OHq}Q+EH9ez0S)mz|NU1X<;;l=_jApqyVn(n>#W*;`3J(yRdK{m>7CCxZ*o zMSoFvu#$cx)S@XO?H?JLVn8>NejEHTwqK(`xz#JDHVs_64XznQdkOV7n}~D0M9iY3 z{atGgo!;~2p#L&+Mrv}CBDHHoIeVL!bWD=2Bu%vIE*2am5_(u3X0g&BmK<*~T|0u9 z2*X&WuB77_qSFXXA2CZkEcDkyllcXGYq>Q3Ntfk0;eS9Hv z`-Fb-vNhiBeCRRV3AMQ;3=89cemWCQw>l?1<~yAU3xDVE$2_MqleW{Dm^k?n%l$Y| zd4JG!i-u98KkXnpq3r-4d$Iw{!{MmVSR6S z)Oy%(Z0_F52La#oXvKqoDC8AS!4@U3^>4)1*~Hcrz}C4N-BW;w{fqykYrpMy1q|rF z_zKPXu}g?q|Cz_oy&3bspLsPO&j?m0o>u)SS+|dSSTUNdua4*HXnfqYy~y{k)(ia| ze;7vPY+b;;xS6YCiJj!0vh}_Y+>H+}Z^a6ShF{MY9R8&kAyZ1s9^1 zO2$-Z0-UwOp{9k=4PlHu5mMSH=A>eQQJRQ!jn0x11YuVkmM>j9YD+t+-M9^Cmtz`% zpi4jnng~sTR3|1YvXqcp${AvL@0L>KB@l86)Id0ekiwG$f~6g`E`hL@K$RrmhMB0Y zh*O%F(lr{Dg@oNgqNN>Sw~!hyfv`(J1xBfz1gTyOg0hrqx0KIs-QgAz@#e2~^VfEb zo+0zsxcRF}J8E14Q@jLfTmmXA%G61argn|i%2I0GQf8HQRJo=u(6TCxdKFTybEyY?)T=b= zA&q)iBlCGAQ{$2e`N)JdGPN3+YDGrxur(U-C=#FI60hNP&QpGf7{fjz8+x z@Hw!{dEHP4TvWbPt%Jpn2G`y+u)S#jM|1-gHB;@jG{80kcr~ji^iX*G;dw)I#b084 z=@#;t5(`u19cR0L82xwPnTW9agg(Ik5!t$Xp6|l1-#^zzC+uL)w+uJv<_0PQ%4_+G5lOIiy=3blxR>b=uG*VaGEpu8lepb`Zl7 z+F&7JFYYB+vo=MMu-`I_d^k`!aiJbnC3F)BLqflrFWYh?{YqZ8h4h$ zcTT0=SwpMpoPGYU`#)2Y9<&kT(YFGyUIta7bEsl^KHs@>6D@bzM3G_Kbrc`-=X_B7 zCs>=mb!^oR_+14`l(czHA>K!1`CCBo#8A3*G`eB9SiL)37uYshnmD@12OCr9wGcQZ zCk&LvACPi3keug~Vcj0mnTtDz2X%j2tx!j+(>E zPhnr0Wa$!(rDnwvB3&YwLTck1bse3z1smp)-ujTy0$%^mBJzKxoh3pA<_$@{tUl(S zK{tJi^e=iSs0vYJGp_l-W7w>U#F`Yv$f`hp(fH;HH||Y+GrAf$$b*KVZG%Z#l{rG>(ZfsF0Z%bD!Ug6&_^$0(ByWSa zzAs$mX#^d;B@^;s&qeIG#Ey^mNKN_~TRW5nNzRody#`UCxOB9YGw-401yX3z3P#lE zJ&OdHhg6Qa=CEqy2w|BP;%@b#Q>-d5$tLIr+?`&ipjT7{C82`$82kN;&fH6^X(s@x z9-6D|{^CaASG*Mj>;nyVp3D>GcBMKfKaJKgVn-5=WD{1wjI(5a2;6GzTkS`3|lL1M0^smD1Op;xEmxRhX2K{9=K*igpAm1n`|)QeLURMA$d zX^J5aV$6Hkx#S?)(GKiad@3C2`ncGn8uJqRZW5m8D7eQ#)~bV~?M`aht}@*JnFF4& zc_4V-w8q<`VmzD~0RQp5vw-0jfB&*D;Nxkzztx6Eu_1rkCwG-8<@6v-SWd$_n?80M zPFRi*l$=l9f;SI&Dy3;nh4!PvYSo{9RJC6NAJKxK2hPh&ca>@QmWNE#hGvmp3Z~;Q zEs^rP$1=M*sBUt+ImeF~)0vC6QHCq(pu6V4FMeeLD&lOah;QBo6>*VP5o1q6MT~7? z_if%8yo@4f#R8}_#3NL7mc92Gh5Cr7Y7A2K-XQWkU-i@Z*iYxfugu%a?YB!XWh%H&8>`CxmPyvPAJ#B8HpOxld46OgPcxR4m$-d z!=MaX0As({OIw~n#n{6_7tuT4aSYVdr>GI1ztyYF@h4s`%dOA`_9*Y1ioFhWuVTE{ zeaF)|=O{e_PWrW* zgi++v9TzWK0k?Zvg8{zX(+UJdKHzYX+8GwKTQbFoc|-TYT=u0~Ay{+N2ObCKdkRY!XSxKA~B3+GcbY@C5j&9u92)_jjivk^QW$@7qPR-dF zEl1zpUQ-?DXmG!?HIcxIdZ<~E{Ro^5*E2Dh6CicHd@Vf3wUC@6{zinZ*3aAGTp9`l zRDaKjfuknVHBr;?PR=}&_lx0oz4jZ4x)sq9PS1I&{Kqry4fAZGx1~N!6zBXIlHLyg z?>O4XA;>fs86g}vE&mwQm%VmREZRUyn#jU=3`ri~FE2%5WuRq7Q_FT@%-=sBmi!=juz~f3DesewPr{ zdCDzOG>y}O9Q5$3A``}7?#LVRCm{G3tYYAD|Ll^|H%!yDgpf;PF?)uf%rJhEK_5bY zC3-2?INZ{Y`7)Rbr%Ye+}$ zrWy`I=TbqUNFGnVFZPyQGC^I2GDac6MfKR-8l5cO|6BSRjm_WCAUQkh})4meUYfrmhY3B=PO{lMovr2ujfN2yE>fz6GcJ1T7Yr+ue zLv62>?S43)>a2`61Ue4Zj@uQ_j^W{}mGLG9G)X{00QPHsZ6O+|M6`hcF$TmWpvhla z#rvW{@37FWw=JSORPEz=vrLTUCN56G86VYf#4Ih7nAD*;+5Jvn zN{r4CW<;emaGEH%QKmI9^BH3Paxe1{=l!D|<|AI_8=Mop%r_8C7pKXYs^eq?PSa!p zy7KAE4Z25DT`sBRMvms^d7M<~&sOSJ3EL!eJwvrO3}u`#IWrS~TV%czCoF~~{Hpu# zI!hoT)*w-*`GdR4@;27^H0Sm+7~|1B9KbD3flt{%I#mR;MznDDd#3mS))>JSHk*Wp$eETx=F8HP_g zByo9~q~DMV*D98ufOF;z6q#=CP?6$6Pf&phAm-zt@zD6sQEk*gF?kOMx}sRd%4=ep zJjZ?n*PW?-<^y&q9}4w+j+do}RbvOEs|m#akiq|(Ar1X;8&=l?6<`^-m{($1;A8xk z#$z3jB$hDhF_{3PQC$^>Bu>{bteWPRw-mQ%7zvW$h?2Fhp+H1J1kpbl8W%`c5sMeh`hw-nu(W5#)V z5Nr2nI&vs4Q_FgYKF1~8AW9hfZ#X+F>laK4V~y1!jg+1puW1cEN#!3+9udBjq`*J^ zGgCi~w<`<8(-R@2w?rKlra5z~e4~o*t%`N>1Zqu3MU8@*U0FBE1S4`95aJlc`@s9Ybv;y(=6VLVBMc36H#{)$Iguwju zj;gNi=}E$`ejjGKyN*|{UcGwt>K)~{TzZAv(bP7Pt(9Chdg&%U~wOe(*H^Qy$So;`(v(5)+40YDY^w;+Ev|n0fpUJg$!n z8drpXeQ50VjLNqrxG?VbiDFT8lH41{e8yFm@K%jwrAFB5u!hme(e2S2`H*r1-J3pF zy(-?djb80%F8@RgB~E?)<7@sn8X!PKO-d&jDYixa#Xj6JoGd}{XW${;6UQ6!24HxsRCB-|cS$YAmz?$PcRi&I7^q?3~`2!_V z^XdOcX^bTIf>c)z9U%p3q9I+J>udh#pK+BmCJ`~3tfdePAu2jG2eL8%=+YrBvn3uj!vl%mYi7kXr z@Y%jUI6s1vFc^Y)MW1b1^fZtrcAa5x@8WmVXFX)kxKlsiO%LV6koo2b|N9XpnQBQparq-`BP)5!Y!O@&@Q&foZ4>8g8u%XZn3aZ&8}Os#2K>m} zAaWjXgNT>-7k-0IM4eZB{c-OAKDMr%~Ne_Bz|OUVsMKB=d0romEHSl#&>kyCb^arqb^fA*#J;-t@lOikn_{ z`uYJr331^hAyt`6TQfGsQZ6i{Dm;cXZ z@;EUcdq=L5d-zw>9QN6^3{67oI~qCv#09F&|Gh&?E#|z%%^80x$(CS26{EU}c6@Xc zfExrc_|6#6G@b8I3bv=ep+&1S5Gj;y^HDz6y@%LEqWaM+62Ewqj)C>HGxe(N=+}jR=aY=rl(WWm)M*3 znQw30mKh(|S6Yd~1S36d3zA}){%*{B_%3L{7Z2#)8#be83{0s4_L9``wnoKZJk;vJwwIs3i!b+a zZ&nl-?rgW&L*a2^vPvZ`epw~%qjBHA?{;Q4ud%?r%FH7M1}k!U59<}{F(|`=idaMH z5%cnQCEJ?CvO-KP?Ts2Qs8u=rs?1yRm`5GkP89`pfPg+rR_x)?DR(7aBHYMr zk(7`h=9>S~X`ZD8%TF<^ee$vM5=XJp!Q-@()|`|m5_<8EEKyvXYhWB(3(z6XV4rPb zS-#vc+A1@S$1}VhimU6PxW0seK{fj1)WZpi@4Z`SCERs2tb{ymC5%|n%%aJ;A8q5@ zp6g$ii{X{mwZ-uF*HLzoN*?P;_E)jpBuXMG%u5aJUCVdiZCDv?b%z$pBh$TG(f7H`VehG7 zJdXVSwl_%FVdSH)U~;|rY$tYht+#()zb&Ii9i*W)#*hj^(B3A)YENvnBVzf3c1$R;8fXb-fgLWhHg75;YPBS*MxAa9Js2ez;EQ>A4c# zv?y^4jbVc6LYkyQT4B3&J@d4&UaeJHRe{7rc1T6%d=xOui9L`PwkrU+9zSZ7Z4t`fd6J+Tm zYBilittK6wn}-|E_$!^#;d9TS2==qbQp{8M#eS~}nd`wdOv&bwHQISBBFG&)>;|{Y~!0E*mV8#f90i8D|H67QXr@Y4Ax3%pm|>^|EgR0n5}#O%CF_}H>&cw zo}pgxt6J1EG^ya1tY?Urbz|_Xc3O7v0IMa$j0kDqt&j0obD-=bW+^JwxM|-&=6uZ& zc<(a=Fn(dz=-6Q;NFfOVbaJn&HQ#xAo7~}?_7){z9U26LP3w%pNczAShs&o815W%_ z;B@4Yt4tIjgXu3Un`Qe_FJC6jSa>3`zP#RL)u zvG#|KbBTa@guXNJ66@mq!7rON3y@pBY~~~I0r?IF=-B_E7e9*vbcVzowwJ)uazMCw zN1j~^17Eiq;CmA`EIXN-3u&E&=>VVX!e@B-4AvJh?M-q*5R4xP1mH_SOB~tX8Vc17 z;C9DlOSG~j=I%|7vL(37$?-^eR^TeIVC9@csi-_h;m2aAy+mo0}s z<`YjfD~+w&4yNTt5KFzSc<{7e??=iM!23W}CmW=TEYlvPxzeWu)&C?5{oZ^`);~c>J$L$D==0M>84}Y@z zp49!ItnLdQx8pBL`0GREJ7?kV!^idvfAQalzbYQUKmzER+0Fq}JZ8t=+}{;{vilHq zzi(FeuRV(0Pr^F$>alx5;RvcyINOQB>mS`S6jrr)-__0lVmyE{g68H~&H>~Y zy}RJgAqA2_)-jeaF-{XEM5hC_$2hBz;*ut9l1B|6NKPdPi?bjcj1Z=t+ET15*rP^m;?F+Qp`%vw_GjLVr z=1-d8G%=0^Y$-sE{SUsz`M#VQ8-|V5XpMdFAkOy%mMnGdzetwiSksyug?O=9D>zZp zgNm+;Vh9n@uAd-0OSCDe$X&m)$4gOC^0hRwDKngO(((X~>^RBFkVvq~4OC^q3@0tD zqbmJmH(ATCI%odw4niGdsFx53|Gq4X|36?~fB8B1*Pfs%$n^LC(_>br@7#|RwVvq_ z7#9=aBUI)5%bcjXgR0!b=cpK+2&7k83}C1mZ{GX?Ypv1eP`*bDJVz~;UgngvkHD7G zhv_U);J*h?K~RwN-qe;j=)UklGvC)KopeDCd2P@1D_5uO`jrac$ra8<+CjYJWvfZt ztEtnJD>&9u??WuUVIVh&WfC%76uNVb_0c4(rRwE%&Ux!g)vwm(jj{U%RT+j=F4wAj zeJ^eBz_2q3%p(}ZlUOu3WR>I=8>zN|wJ*_XKSZ?$S_?#++TnCtEzkC1hHBgvi9C*H zMHImk_V&DKA^6c$G(W$a7)w4Fa|Xxy>H_rkB7U3BPp?A-WtJCmMYd`J6vdgk{dsRH z)VMQ{>KKp4e3PJ+uSPBU$r<}C0K}LLo<|5dVHzN0u73Wxr#XY}RZT_5PJ_Jlh#)jW zqjU3=Uz=^`W}MKe)Ldwq=6!bLi82$sc2tK09D|(`WNg z;z7C{dE;ph+@$#FX_nr}B+$B2cqih@EW}gpvA0v%<*c|$1hBW5c`T*?~ zpR$w4G$3?A8PVopu>2h`z$}D*ml$m*zM|#J#L+YeD#aSOL)`@6F57DtYCGRY@{!+T zWbm+rKQ6_OFg!5wK%*ek176PG!6D7Rv|v9^Hv>(~d-oB);BFC30&nUEr4k8Zd%Z{l zhV^$og82Ot=`f+~MTWE;8iZ{p4x^B!cuDzre|$BZMhmkfzg$lf)k6*I#Nu5_j;W`q zIvQ8bpS7tfypxEOnE$+20p)GychiDtBi;S!VOMfIxs@no7<7Lnb^q-pPD*LIqdAjm zhF>>4VNqnrT=0YoeTJADF|8J)UN)54EFK~P9q^=1u5a(u$n^#S;)*OFd;*Ac!o2q% zb`@+?Gp?2vt=JnVz4=EpB>Pv&#GqL}fUm^qipn&hVQVpTc~35eKJlu0={fJ8m|; zu1r+g&O4eLzNlC8ONuX{M7O zx7c>Qlj&;k#$Q`4N5xw6Jb09tMB=SXZeS-3CYdY#18qy$2D|e&x6nqoKNT7DK?~oR zz6QNYbDe6l`3<=xk~N6A^?g}#$;T~xyXyJ5CELm(+p4q*7eU#U7Ryv5j<3$y+G1Iq zP^shjRR}z$k`*%VE_S^7(-w4+C!X1bz$;GoglA7(Co%V(#q95PSZ!l7 z)$}o+D?DgpGjo|ERGVKH(f0+!X1vXk&HS{L*bEdkcVjEyItDNrKLe?4qz3Z-hFl?L zi6QrJ<56=VHl7$^-A_Y{@ZJ3WFBUVwk-g!wmGLtK>YflbogWpeB?agf< zK4!EfXDu8`y`|K}fD+Sq?B zN{*`y=a6nhtnb*qhbaVGKmPcvjnWr0F^0{Su9o;MD1FfuN$GW4h|(qtUEfU~4wY#)&MuPTx0E5Ib#?EmUG_cng93}}Pb9MMa=w%7 zuf3+-J=A6YltWP7ii-q)1GiB+BYgS;DN=a*e1jfIgI!IK+H+SWPA*Xgh z@;X18dgEqGXZ&`o#i*^d@o)Mz(oV3vB+5^B&}=ZzqS-k#y)R65((Gr+Hfgpa^!o8n zI=$|nw$W?X(vY{d(CY-?>^xw=`&vUDyskMyA7 zJUgP+y=dunZ7a!Y|GgEt+aH$QeuJ(3-nrV}L;KsJ4;{v{dv1UGm$%xozjf1(d@fbz z(sxs-}8f0weKwUNw5`u?NW+7b}(qFt>{RlvDtIk$xM3vx^E+2_-^rg_10sC2x- z^QVA`2%%w>1)SNbHPiM^p7Z=U?|X8(E$CIRVm;wOhGYJkkAF>cbC}#ed85bSk4)iC zHCVlv^|hoj++Q#HTBkC;o;&AECzU-s9}zT?D(xjIVIFdInABokp82=(~molu+QjAw~g@+9n#rTo@@hXyf%1uexBTcCsuQrexFLcv{=VSH6 z3D!59XSpK&w;Ml)nfE9nXD7x^hDGN%wxKN+ohKPTXE^D6(-n3)H?zfc+7FP|wXt6; zGgOobyVr&8T;@=bS+0N;rS)XAVE%NcGbBSfZ?O?`&0W$yf5P2PkN5tn)B$(fLSu<< z8Lx7Ys0Vk9h2ShT?51t@-us7 zl~13YZ`dLuLBK*U$509D!4>Lq7J#61(9^r&2sI^#&M#YH`3qS-|)ExQT zHl|Ci6P0Lv%~0`gBu+0D++9_ezp`_ zJ!VB$z16KNovor@Ej;8U_QJoN-CDO^<>#6YpBed1pAK(&4$Xx{EFRh1nHFVIwpXGu zn1TPkDE(^n9^lI-DSPul=Tv*Sd02hY!NZCn^Vl77AqAHxcafUh@+ex;)$$5G?sogy z16zi19!a!=mSEg0PCg57?p$KOgYBh-RatdC)7XDmlH%jkj%RQ!;&^Hmx@w8gsu@8M z^W4Q0T}p5A-}i8ni|tK5giX%0n)KS51pFteYzsH4mPj=(+-h356J->>BewGd?Mu&t zpSRXWt)h@xoQAMRVPyCc>0Na1Z7#7@a)egkfcOfz=#%N6{|hgLP@ew^OqGNEaA?~q zOoFs2V9WMYMQNIRRthAqdfg53z+eK&Rc25B()H#)i+J%<=9)MshKN&N!#sb8$0?Sdz-u5%TqY|q&d1I?gs)?G_sP*0|B>-1V z4^pq>cjTD?dHoaB( zwIqa{O&8dOv`3B=%MQbczQFQ|iVuX6I%S^;%88=wl>EKx`0^i8N^nX56pur+lP)S$ zsWZ+bGRG5w$_o%yFXPrrqPjeaP%E|!Gavgxg!N5b%qL=%j62t%Q*+&m7?xe#R$um2 zeB1c7E)P{81c0y>WUfUa=&w*@pT-fLw zbCl*>2r#SiF&5^#q;F~1E+c%%k#NX*MoLgkK-PC^muFWEKoI@gMD(>MAkp6|{+Ojv zi|FN$?Dp)g>+_U?JzbyLvczrZmFPj>LW~Z(sreoUN27dZMMT^gEBFO6w2;4{(38C! z{7a2R0u}zH;(y{t>D-y6rCnz31%JBE4#1zHL>KrI#Q3K7OE}6%Qwn=n^dJubw8dJe zxFY>aAzAB0Pd>kd5FEI`d2Q#{)oTQ`QYquB%;BApxjH+zUUuMEgif}by)r>?&70PR z+bi>={smhwBsHOFRvl?5rzj8mNXiY9WwIQ+?W(dJdKT46ERcOPjdhgt6U^*6phSUl zUu;XB#K_PUFh!2wyUp_-GrSErxgHYc!%u2AS-8qGlVA3+A;E|(00 zKV7E-ZG?$7!ay5-dt=WDVtJa9b>ObNVH@tsqcOc3;QZ2kl}K#4heZ?(#LaK9h77hb znfCvPl5=SPAFQT3Zd=Hb(?-(kRFANj-|^sLe(_jb%-@SY=4s4T2LHXWgb3e z18o{>9N(3`opRQPW0?)HA*+BtzLc>WC%WFZLya*=v7*LfnKsNBv+ApBf&1O7xzJ;P zWU(RCA^hC-Zfr=UDAp}2UFe$?V)jcGiRz1(I|I8wp*RL>JPJ(*1s?`BjC^&y*_d%t zROh7U_U=1oJpX&f8=1Mx-kNi06jcL$#q7I9h{XjlKyz2_0Xp%8O+^4-VMr~w2;)XW z&O7lYGJI8LrGM#0{Aym}I)OIW8a5ri2-m!i!o3mpg7bzVc`O^lUjbvz%bsr~H{2&z?iC9->!+%bywhY?xl%K(B@n zGKLc}{@sc%OX=l-^l}5e97-?8)63`S<*>B^stlv6oPg&U*~c_&V>OSleI9In*=j>Y zzU}i6b2Wda*E9>I<`ed}hr`>U=FPUZW8Qr*vj-In2*J{_fDKR6ze)+G?>`_S(a_&u0@7sBr$_?uj3JaE8<1NWw$ zh-XaxjVQ&`(1<-$Wt2Kc0$bU5JhzWfw=S4z(m5`xpu_FNIZCTwnJQSO7tG5l7*PdL zBQE&Eh*g{NfVp7$@FXPW{j%GtZj@1NocKPjkyG&+iN2_MNK7G}&wOx@N-bmI0FJwP z)MDu>Q>lGOt-C0{E4}yG)IspxHYSw3N*5J-(lxS8@BgDsun?u%o;GTyvn8P%@jv;} zYv7r-OD{PqpornWblt#VS(8y;Bj=H%N63~2f!ae+*wIfQ@qBCr?&vebA9Fz&?C7Gz zhyav`Q;A!s#K%x#GM0dN-EePY)O7)=ICZMmU|YiU=H?6BzoH8=knaeu za$bV;5S-V6w4;LnQ5CkUv?sPUbKFM{Wh~i*wb~VDKtL^r#w%fbfZgs^e9Y$bRBOG0Y1d(6|b&vyK6M8V}^jSv8 z82nRnW<`{-S#93`QZp&%wOdpFZCSUb?pElI0O?k!z-1nJo{k0^HhJVxqB?aqT#L!QA)ZMwO0{|K`FOp0o=0jTt#e9P5A$6en;b0$xu=>fXsqkK zf2$(s=Ql_dM^khy(s{(3+(kB>q%eUc>Zv+8p|~+ttUm|X!INVZ?#+$GeCCpunr$~2 zUJCM2jZVu0xSY>Sa4{<)^FgZpm>$h3DxFXfrR$4OOq5P;=|5^F+V5VjnU)|1&pxl2 z?*eXGF2kB8mrJwldCR3C*QLvI#v^pKF91|PtG|Toe-nz6(c}kqRIhNIis$qiDYc0mFP1+z}hbmx|v;qKt1Yi~7K)(?OT1SNy_W^q1@@0R;V9ULieAAULKno70Jdu)oVNB$unuP5$ zBpN)cu$$lmoRe8a_O>Q^Vrf z;_Yql_Qsuv#aTWF7MBthZ(7$CO;n&S^QE$O5!t(&LS((Gy*i912pzxJg5w#JJ)W2x zPYIMevQzeW{&NM)e~V*08{qA$EjXTCo!!Ut;D_3Hz8}->c#i7Y&Un=A?r_fiikh~N z`FSg+cEqRIbG>c5kIDD~HK=u+dBi;NXQcpnR)o~BdqJq(S_bNwF5Sdl>j?mNo2Gxb zMSLKgNP4z-TsBW2rOk2;$10$xdP7;}drdk4IbmcXnsj@51$?PR%UW z*Kfx0Zuu&EBVf2JO@r8nIrLepvQi6btT#x3w_#$B`W%aNrwSK*)SBfb>7IVW22X^x zs)LA-yi7SXwC?osW&LHM8>pbo4MHMG_oRH>XumOLgVZ-?kH136=XM(bA}{3lwZQS~ z`hzn~jaVv=%?U6QGX!2zpOS!AZ`7TJwX)J0Mu z`fdR=<*=#M&|E9g95Hvl)S97R6JKw&*?>a#HOB0o_9Nrkd=Kk#hpE2Ul3J{JgD!bt zhyL*ID6N%dO_a<11gI?IDvOffx<#i!evhy6nAI>HvE}g2PU^CdNE>1#Su>LzPkfK; ziC=!?H`^2UyZ86huMK$68PC@xm8Xm&o|%aE-yxekVK3@lDtCDWNUxNuRl2=A#HA9{ zQQ1^5-yR2b!FM@q+S`o3&{R+)IhtsB6ts-;sHNH7JB=H^G&QBDCrS5EoCGiC4ldez zrr^0Nx3p%G1?NZn1|H@LCW&9-oKi3`g2&q(my0s3P^OI?#^shBgT@5uI!2zfcq2JJ zPi`dkd+o3gPshfJXLX^)by#{Y=cn215^(%sW@{5}j-p0M)Fpw|Mp*gZw zfx+4B7O7HWXj8#6b(AOs-(UC+?i~lh=DL_R*UeXH@s@CN{idopL&aZuyOjbRLM~4{ z?iOXp?-C}=Qhdc^_C30EA(xv9#R`zpbvx?|{))P_=R#!84NPUt<+S~*qJxUvC>uJpyE9QFkJ&kog;~K8kVNXLg zE#i173J=e|?07TlJ_k>}$_ui@yz|LsXZ+#c_x#qn!+y8m^yl!IGu#=Q8$9xUDWEEU zk?)BA?&2@9S6dypwjcLdwJ*2#U`GvVEq*~!29DT*uU#s5miZ%!ra|%Br~{GX7!VZ1 z`2bQY^+ywof_(8&d}*RDAYYgs`tsFu{F0Z!FK@z^Z_$jR;A8qSfxf`~hxrzLSu4KK zK1fnt(SzcJ8ElcVI>sO?He{+ORBjBmRhSADeiE(0CO?GD)3|iRS@;AfyxP4m#!a%7 z-VaLO=q%k-^mCnfKj23@f6tDe7c-`x4ND3Z4VHxb9n)Zw!L=_PZ=>d!SLTDI8z}J1 zp~VGsFhTW*GK^8TUr}oph^BAT2o$NHsrOJi+Ft`8?nuKnb-DQILPcW#5+521CW-&A zc~j3n=Y6C(OPKGAHvDWw5j3y$NL!Yi=O>%!5H@!`>{ze4Qj3=_#}HkbY_#62_GBAv z=&Yn;_w~H88dmzsmq)VtXf^D8=y?xt;~ z)W+bjTN}=*1?g*1V|8wb}5Wdtis)Ut)*BYB{y3_v)qE3LvZyWzQSi9Kt=8yPxh!UIFE5B?lzV8NdMHJ-<{;=dV zQdInzl(;Xg(IS9$1HNQ~l*oZYq?MiJ)rDA9iJI zo-5#ij@h6%7EP351s(k)o!evPq;#tdn|^xz!joEHxVs38C$4s4@u%Yi7N-eNMgk~* z|Hd|yVPKaE?|0yPv5ZYE5M0hHeD)?)XN5eV zhXt~pcF`xEPS}|;uUSV{0gS~$R+&1_%fwl6##|ip-F&mt@+sCamwn`K1Ajecy5Mgx z9h}blrxSl`j?Kc~rnxr!ow!TK-lx077u=zO`BI#exE2XG>Ju-j8nwr0pkdQy z8eDRMkMowRPvqPuKRJUEy33b%z1fSkPzD)6!i0F>lh{l4vWzCy2UnTI`ygHdQ*w9T6#dds4eRHl1ls45TWUE_prH~X)?UhUZV|lU; zN^No}rOT7?E2TX7&E9(^PxiSoTb}GWv_tabXA5ld3$pvWue4+GtG&P0 zlU(KR7XV++^lTenixs|>4%jn%o$7+GuZDC8{{C0m;2-CN-$@^7F<$fF9=YJlwf?o( z*DulyP&AV`fRDCfiPv>E=dl+z#0d6jU!cxH<_%#Z!TI1pOH!oR$a#~!HtP@{A0kN( zuEg%@_Olu#e-zR*}> z?}2p&YA8pk6Djiz$s$UOL~q-d66Ppw=lkqXA~o+eXBswnIfEQ3b|~OP0iPIR=1@^2 zx!!!`%S`5|F&iqVFAf~CpKIWd9Q`5x2`HKG%rErIJyKzgu{2lYq7$Rv`4qT>RGL1i z()5{KW_WXkug$R@mR^y)9^MOfXgyqhg>5~IIH4o_W4g=wJGW!~9co{HFCE{J{tkBO z@0URx>Tjo=UsWFO%C8E5Um4xo=2x2*NPg9+WY753LoWPwbjRSo&<_9A$7REhjvnUq zPr0{e)9;Oc(ov}6`_UCH+LZS9qt7c{-~|g?^D8GPZtyle(II$mcERs^u6uA6yl2?q z9pMHqrsW*n5qQ6NyhHG=o#Uea>KMGqId=WmA6?)D(;f2)C+ghbJ)mRo9_<3}C9ZpR z7QEf;@VH&yxr{ZPMzQaFIcgfmpD=92JhY-gLmyL7yjR`WB8qA=l?NReteTCW!AblY3uyB zc9!JF(_JLZw%|Y51^)Z?>ezlV(=PuWRHvAhem+&7Q z*faQVzTAcX_3haHVu%0Dqg>%X58$8c;`FS;A64*Q>6%M8H~gJk;D5GHhv0u>h7JBg z7xidJcT$B5yu|?Txes;#-lC4d+sOsqlX`ay zzn9tIed0(Lc>4gn&pJVIgZIq)JBHuOT=-vc$M9>1xAh1Yc>5#&>j=D0-q$hwy1@I3 ztIj4Hzx8(h=LTfGSnb#KS;Tb~W@5gmi~k9K(PD0hJu0*0CQIZ@{Z?{OW2 zx04IJ*Sqei+4#NG2JiRLHtsVuryVpu#Ut=#cALK%^U)|7W9e#V&Ik6;W189-rJBMq zsCF)+Yi}^O0L^PPMYeAuZC6o2vIOJnr=WSb0m|FicvT@`mPqQHu|tzUC4iy%Ax*aDexx7q8)|DB=9YQPzV{1@OUX`o+IVlNdIrP@5r#}1jP zr}&;$6dP)w>X{OaRkQOFm0Dz#YS`2XLaCc&j&yj#0UiBG`CtF2a~7@~)vN-crN>eS zte8?hrGph5`38oHOse!5+dMSmNoRC0xBH1I(vbayZCk3y+GrGO^a(W)A_u(Z>}#3R z;oD3+Kq2*@g8vnxVS53Ewn9BoA(?na(G^_$ES1`v*2KB+$CChm%3wj_F!<9cQILX2 zKZ3}ny~7c4WzZ&-RugX#svzY(>A6!+s}JNZN^fry|0Bzu(EMhbH8`x2*`!R@+{{9x zMYG*gq>oI3)AYi-48sbh#bfNRycCk;{K3%)!+b#?1Nc(tju-{oz?t*9X1b&Q_SBwb zM7=Y=W|nEFmR4pySu*gf#>iZYyw+kgyR1-C_tjmr>~2(B3}1XH6IIJP`4`T}qYL*k z@CUi)hfH$*y!hmvMl5etW|Ol&4ys9N+c@b__fh5*>+0`zBhXYFS@{M z=oZXOf)?{n@o{f6U?AI;gqZQQ~q)WT!;Y1ZJ_ z!pHk7tDwbO&+ciQeojy0qUxT;!{YZO@$Zk-^v;dnx#<6i!Qdt#7|i1aF&MBwJ>d>y zFjZ3&gCSm3qI>lO`EnU{v0`P^f}49N6EC41Da!kG`UL9&f$m6RR#@F?DB!o6Fd;`- zdT;H;H3p?HwHibT#y`mL2pB80e=iHBIEH`cmS1o~6}tKCI?x<3FS4UF|vYg$O>j5i`pxa^Cb8BptaY>Xs^fV^H4NXB=Hz-*e=NG zkUWQh!UK72o#zoJKLpyckJtTkj_lzoQ##L6&Qfj3o3-u9o6_Hfgu)ptB^W9;YY?$q zL|1hxTpZY7^#+EFaT7+wk}7r4AQmdaM^Q+)D(R^fHoq2}A-$A}g_Kys^EjB{utY~$ z7X&Wz5wbgW6;Ryd$HZ8huqYSe5N#nkghp71seO5u5CUjKM+F5bx4eSTcG-%1%3MAL z136X~$ofe_)Qdr~FmigR(g`yW&tUci3kRM z6ibNUMt053$iSbO>XZ=Wow|!4wnFUI>E2np(ljBjq*88T*Xi)(9hSBP9gc9@N4>O< zh9pIJ#ekQ%wYFrES8Yee%oFWfdvI2i_NEIVO#``G#q!E!q5)2wfbINulD6ry59oiC z$zx)G8<2Snq>_XN>0->imuhyr9HsJOPxiD+F@KP@)%DUE!q-CLPAze%EM@jmDYFMK zCAzE39!Q<;alzmxXL=2k4G;C~`J8liYw@(Ryt;%Ju%RC^UNu#CHsKs7rpnPi8cdYc z;AG4pR-6-B$voYfxy#O=nd38ukh`8^ZpH|3A5})@4mD3RtuM z+|xMe!k)(G;_v%^>1hPS@74Sc_yI5hG#{DS8U75M4ToJn5UIdQ(T^8&Gm`o9G**(s z-ljg5Ecm$>2TMZrhBE4wETkxgCh&~}<5~zB18*>FU1Tk=AjuabUk0~7@Bz#zX@Y#U zVCC#03lVf!hQ$K$Yfd;RpRka7|Mzux|946XuI(M?ER_RaRr(oQ`Z-4YOKG4Gb&sbX zSSF9Iv4X-+)`N1bR#P`s(M_Z_qXztz8>k$cBtu&3;2e(^=dk$YoBo>mBDPqBXnGzw6vxoFZD zOUO+Wi$$8`^d~5Ye>l-=wbID(MWZA*&|97tsTRXXOL+e5;$L{ahqfrRPrDYksM=|~ zMX7N&If(U=!$p_K*S5j&{tfMLPPus-^6(W@BpL&$e^zdn5T^YAa%2veJ3pCcR zZiBJbXk)D(AjW#VReOL|J5SbLPPJdf+AnCe{{XcodK{#9A_HkYewAShzJiUdq{eQ! z&3RbQ(gp3)!zOayL_nKjO~&m z>@f}6L~3WPnEF3uFG^6zZ@&s{qQ}8V-ldWJaI149L;6c+lFJs^C=vcs-WPotMT0o<2AT7e+rzFk@iI~*B+**=ygw(^i_`2GjMz4!|8sCb)e-nR?6My&N z-|`=PxA^Uz#*=vcthVAw*qr=eclKgzsgv0}QXo8E5BpeH3ZWMTGQ(PkNAAnIsy4#HM-7=RlLPwkPW7{k$$D0r|XeM*qvm$CEVw=Ehd6) zz7}r<5unC?psj482y}vHlW*M=^Wc0M92Du6j}&_0%|M}dX%srI1mnZqii<`fBvFXa zgXmT*P6dK#PeFmp5n;cUUXmm6WLkvu#DhzHAdX@8{&atfMK1fZHOsd;X893ffs1(eRkC22{3uD_9wvcXEfTm@Cjmcg9rw`G zzsOvfCNF8_s;)72qEb3hu6FOul)+6~T~WkWFnHJ+kyOs*_tAEvxxeU(br6fg%v z_=L$;xOYmrBMCZ}#v_Nnc9nH*)dWTscVEIOH4;-}JiD|9pD7jI0=t2WM3Nf#u?$54 z^%%hjmFjU^W2}%hy8sNw3cs3&xx84GTx)n>*Bm;J=<-k2sEVNrnD0Xg@Y#pKBtJ%z z{M2mKRRO_av75O>#Msoc)KmjDb%EB@ZEZC*-lJ`4SI;-F z;cT`&)UaxPKsLyPZ!0z!;G!OcfnX9lDZey4E^X z;(Duxj}4mwkiqg>7LW11z7eDK6SG}J0X=GIeR9*d_97PB#f_rAosFV-C%T2;R$nhq zcw_>zKg{f*UYyx~T;<$64hZ1Pz6{#&9#A1u$E5O@wV%)i6s0z9z&7f&Hd20U<1$+0 z8ztDLk$hPB7_RdVku?&=LMrD!v(uU#b=Vrh9WH zJth6eT;tbfJZPbTIi?Uth;kSi6QJ9X7D2(-WCVh~PAMCN64AHBGLMKzycWLW&5eJ4Z65Lf@#quC9-!JmVy?IJJ2TU; zRn9aFSzJ0%)%P$shvmQee0=U{PTsmTBh&MWO*?&x`VQ!lJdwqq zQ@_uE;4nR`-rNXpRPJG!G8%5Lc%EBUe2({~$Gg=0c|!yrr9H{ztDa##=eOCKI-mBnJ^$F9zyMYqsWVZ)CBYrx zUO{z^z0%2)=0cszq0ZSJZN2&DKiLK%wbc{07GlW0M1#38D>F{N-w7-7!a)|jngCt3 zz=`00brFERXJjp#kHo9g{f5qH@XD3L&No()Q3%0a39w&HuvcPy|4{pMAfX*)x8;f}(R=%adO`i{go39P7(uX{3SH6XGz8)?~5@7(E*3nNKx!+D2 zh{9I?f?^Upp_oC(%B|GrVC=I|>+|Ez*k@xtDJo!U1#M%U9aN_~)>)|4d4TGilJD4O ztgUA|Q*=C+%Ej6UEPbuD7r>Ph?ht}bri|HOuK5Kwf~jJ7QyGZ{=?1TBa8gKT0&ZCz zWlE_Pqk*kz$}Z6{<0MlDNlT|R5+c{p0yK>FM+56~v%$L?Rb!j zU5S<5p_?2XC}c+gW2=!8WRPIE7-4u;gW=%PGx7+e~%WG(=x#gFSqEP(&k!h#VW z;JyYXrAk!7Jx)}5(|YA!vAJs_{RF7df*r;1FZ}cd^@?D_D>b2DXwXNyy9}tQDAf0b zc`ublJ;=!qpWRKomk**Mgoe+oY@{Ee3-~i|Hc|v&R#3#C89)&$HHs+m3X0g!-9iA^ zvan*Dklo3@a2|Y@YWb$3<$p%*HGO!KvP!;8wBX-rSS7Fg5-tE z0Fp%-Bn=(`$>H4*gS8&(j8)^&&REdWA{J*bl29_{0|6n*Rud5qQ4rwU^lTPP?yo#H z*-S)hCd`w?deFE>-cetTjpuCGC(anDV$pEwq9SUYKGU&D?+y1dkIP;b_2fL%;6}y^ zO0a}G40I3pHBA}-Izb~Uza>;&Ri7pQ8UmHSVqR@(2bili5zPNjDk46#nc6uQ+qpw) zXLDw2CiO%i%5U)Ay!PK@CJopbQVwAuU7FolNX>4iW`|?5cWTYv2F;#Q=$IGVfk1q? z!|32_%xD&!URjvEhBq6(`_#u+3eUuYsa`H!V#Xtd5&!)6jcoc*Yol1!20@IZz;t&B zjCE49(~<>u)Vgelv`bMaJP0y+GL=O}y*)Tj_fL0{(a*cYJU!W!6irHM7|_jH-icb< z;N;s%yBa}VRb_Q`C3JN}S3bRpnyeuN>~t&7m;1#cPg@hS9VZ6)rs2_m z^ji>BmQDpt;!9Y-NU-t!gsw!fhZ1<7)H%WXwpD<4e;4B1$cYmnHgSqEFaQL`O~}v| zha2|^T?jIS5EE=u+{+ue8dGrSAT+t7+sT<1TVw$iTGg|v>DGX4zOW{n+|`5rwO+MT zdyp4rfsVBN1+^mQe?ZO-P+2)UV4d&Fyhi%)ERb&6U@6fm2`yhGoM_p&Q=sK)Up9%u z-K=9*C3w>%)@>%~GTE7+-Z5x@)5H285T@(Otl{O_BtQ}AVIw1^6nSn$MEs!9e@4vE zUo&_sT4T)irXxzb%BQ#B*)(!r*2q6<5hK68v!I<3d|cIFuRBp=ii5IH``CLKim;w~ zw=;EfA$7BA8gz5N*3I9b8%WSB=Pn7U;ABIi#(3K!$N~pJzS2pbEbZW%I;wN_G)k^Q zl9}RLeby0=0!7|BVn=ea6|d4*ZOqQHnAPTBSz_axXBdV$^-NLPJt%Dc__fZyPy}xv z!oXT<5#C6R?Yh`G%s)b7 zoeTDuH$9h6HWa^YozA)#KU?0Rlh~ZpTDJR-qGgLG1C<8So4!+5wP49-!c@}7aLbSI zY~KHln7i{csCN6!%;j|tBPN5-4#!rE1V%fz8JOnX1GR?oi)!I_JkD%YZ|0>K+R zbjZ92?4IOI(sAX#RPh_E_=r~VgI`5O$QMGE))uYlH0Yo@xD9fnI^Relq-U7wo?)sR z#e&ThfOO+qTijM&QUU(?-^^MT~4I z%+(Oy+vJ{L&4nc=ue1sT6iA&%p#{7*eWJybhPJ;yc_ApX+vbFbUzxg>LGMgT#_Rs- zbj&+kfG#?G6YM;5OWUh zaPsUKg`g^j!e3f2KOc$_TP7Yy0wG$g)Crt+pgC@9kU%bPBcL1M_~t05!0PR5RlJLRBJ^% zR)NW9C;!5wP#SIrv@We|l~#$SCW$#w5>N0b|{@T>e!lbk8t#8bRjNWCbr@WR6h6Ju|;5q^>U?Fno;bQt&HrWG$ zJHcz%JkuGkDEY-`501$_7!9Q@&vcCj{Y~JPVC}&%MQ!tkd0Bl&`U>dtGY^%rt{uA% zBbvO&ohC(f@O*zxXaFuECT1|9`oz?M9#>KYwR;({-m9^ixup8{K;*3HZb&?QC?T<) ztiTY7HyfI`gQYf<5)vQDg~WP&VL>VLXkbz2&{;j=tt(Qa`Rh4~ib~sA>0pxz#?%wj zoy0G%b*H=df;)Q}d)*`U<-2+sQ}6C+>@EJK#5>PD**^Ta^3mDChTN(az2$y1R=OXB zjl==cGpsV%rjt=P)i2L#5%(}iOv?KC`g{+#CTdO|OjQ5#+A6_i@CeJ4O7}U&8{m*S z15}oV8hz<1t%ugSagLgC%WT$Bkv%3Y7o>D|>vVyN)(Z4>gnaLu`CXu9L>K?UCJ*hT zBg9U6#u>Pie(;dClWN^p*<7!q?C4x9ewQ*Gz_KV+xcCGP2@iM-o1a)I{7qfM;x)G3 zC_PvjB1%(mr^mu0;384MB@Dtly$>34G5?iMeo&&$RzkM;M_GcDqe0MoP%sUE3T55x zhJKXygGOW#16WPI$JGWescH*1KJ)#3y+$exEiBF5LJhT0BU`A^Tj=q!s0@0F^rhL< zLSvaH#v=BOy74Lo5STzFMq1qsAiq(`vW*#BTp<6H6XLGf>xpqm{J^fdnd0R5K#;Jcf=xu^MGNB{Qx zufyB&*Kw798@|`r3tW-ox-G*Kw@H2zNo_%DT zog2O;j*k~NGe1)dv-8cUxiN2Mc2gy}bG=C@`t>Z{WsBE5vcprP{&d+W~l{{d8`0KzkB1Qin& zz8l5o^a?`3tb@tRvC5owu$99^%bJzrd@ZdX?~$8znC$dR1a}|J%JPi7;XNQk!h^nzSYu6hsn|G^NKwgN8zZHNLS%QxU&lv1h!Dp3z@;g5RM=SG6j)wl3 z1U1?$|HW&a$gCrl)a~;5|J*!T2P~EM1{W`@nS4GU#Z@11W{43O8I}>-UtLd_6wy3z zdSE@1())*s0wHONHbqOZ8~rG@V4ehT41$F z%Xy2|l$;eoAthuiR%<@?Bkl^?KnEWGFkoSdo#6_*l@FA|4Uh{M!G!l8w^ zTEtx9&skWivZ3IaliC5q8asnC`+GTjYnWYmnCY=e<$YDrjnm{}&uun291P#xJoDO= zcdE$U$3Oo*W|lR0f4<-bTAcOSr*1HjG+#+;=@I92L8S*oi(|23y;(HMo|BBNfzKhf z+h}DnL`oW`&H`uKbiq}{vl*YhiuJJ9h<7LIwK2W7jC6uNoIl(tJ z=9|@>g>UfMTC9@IrY!e-f-=MVj1vh$nlV~>CRt-Y2@eo2-tr||i9Fo*hBMnY>Z z4o@|g_O<$eA!AC)9INUhq&BO(E0eLkTo;Ayh+@d5CzEh;8;KlI9p<+#pE)g_v0zML}X1H_}#qi)_6;@#(P0t8{;j( z@!sw>-jcS*TVkFSbBw_(gbw~RvX}iCY zT>X{g?oZ{@T2z~{#Uu8nClcO&@<_ZBt_4HxXRg>uydcOK0O`Fq9R%D;{QJ!R?j-*l zZ54Lso-3u-#7NtlrrtMf^`I7KNlPYU`=o+txiah={T*rV4=%C7jt&Q7 zhma30t49tp?bfAjm9v)R2-^y;+t)m#Mdsy9ax_hW>qP{LTA+n?M!>jq=FfS#qxYFZ zUUA(|vQ5XyI;CLEIzL4VV?$GQIVq&H5;O%l*}Qv5t4%>rka}F4T31N7qnLToP%_&u zNL}BFc@o88LEq{LzP0&2FtG}X4^S!HLo^N##vBk8lWfTp!0u$QQkm2;ih9@*kL>H* zkPv+)Q)3_+H1)1N*^hD>lrc{ZwW4eY52;a|P;!K#ne~nILmq!%r1jQ`rJheLXk%R$?YQEG3XWE?al*1+z1$CH$THbOM_!h z(%hLaTkVJMZOx=I`C4_U&#Yw~W`F;Jg>){cx~<`>yu8&7oAQRQ7UzQnU|zE`c-MVi6d}sr?)2piG z<@95NCwX-tU4mjy-B!sD-fr{pgJ|m{=U6;0WAV6*#p5y-gd_f2FyRJ`)#C33qr^Ah z5f0a`ct1&Dx#6Yx#M`{-f5AB$3r@68OT+fy%wEn{vuAgM@(nBVu$SBU7z@7JlSHYB zO%*;b=*alc9-jyL21UNN5&1rHf|GpvY!u`h<&dF*XZ!)$l|DttEj|cJ&a|B`XKEBz zvNTrsE#I~9g)2M2kq{M-037bXJs`p z-qu9+B5Z3t_?zA{DXhjtfvvx>@cO>LB;*trp~VvjEJ**>NEa2Q|#hmkt3h2oOM*r+8Y(Zcdh)!^n# zx=1p6bsVlY>U^>d{F^=iy?}{`8jf>j?0)yHtr==?-@RMG@`>GiA#&wSnLLQc1AC=? zL*4WHlJ9!x_a!plSZ{H@TYP)l=ez3O@_cvm1ABJ98`j$;|KH;sI^U&>?fL&cwbx?! zPXYKh-qtq!VFmw3_wO0}&BZSHuQ~?*i+1=Yed2&$7o8w}WU?6YU*DPwKD)fF9K30p zFJUSTC^;{d>@wCT^~_)L_8g_#LC{(F{0l*z#cM&)`N;8jS$yjndw_gB8}3do#WPM_ zYe==b<2z}`(wDe=?EVM-c#6x%FMG9Ou3mB+aC|Jg&_gx*yL`O#Jj(J*X9ppYhgbHT3V_;&+7q!t*KO-z95uUo0b0u zjQB&1`K1lxN`05s#U|&&IzM_WuJf{W+B!!qOc*Q-(#JcV4|6<^bUY7qJa@N0;}hGo zKupVz*7h{M;eTtzzh?dm&z%HFeb!lP(}qvh50N=3o=AhqE{)v2AqROn5AvJ{4)RyW zAfrCYh($j~T0e(aKTE8iMb=LllXIzPaTWi29AM?Y@cb|F@2hof*&|2Hs#bX%w@%-O zTUw627Le=_Bmt4fC$DaOu%8FXKp|=tiG!cKm2_3mIQs`}6R10&&%;RSotea9}>WUr47=(Fwmgjr8^RG;v^v!v_#Op^Zld`tiRRNMORJ%snICpxVE z`2WHC(v!a{ynA;T-f7N~9fP;99e7pyy`Jkp`yIRw+4u1_?nCV3cb3dOe)l%r$A7dn z{wF#*{%6~~Unvs%UEtbw?pKQBe%HrE(xv}$+^-b58su8zqY zcK8Rl!+#XOzwzpJ@UP(i$VIYh6aU>@&OaT4|M712^Us@Y$V)PmY*)OuZHRcJM|l@E zu8PZ(H`%nAKYqGXCV7-e6Y;7Cw7Fn5pi;}wHXh>hJu`=33u#qD(-M5q0B`qi(K2m@ z$k-%^ZGf=}YK!5PK3KktuKy>N%Irh2fH}T9dTI@_-t_-`r%awiH-iw`Hg%Dx3VXOL zb)G1-VC5{W-hV&Ku2+y9+BS#jbkyL{N8}GwrkgLfIP)jG!M40{3r%rGQ(PGx9KD?| z756vX$VU9u#|hhQ%4e7W|AH$GrOTyUJZNZo{1|&GlmfQ#~djb!!Ld1aJE z5cB;V!nMLA1ubl$dm%r~;G&)4unQr{Q(qb6xK*8%DkOg}L1JAcKvZXQ*U$Des|6{{ zA14!=Ma%&^GV+GOhS*;jv0;YMo37IU{Dx~45`f=Aa+g!gxjU&&jWQU3*vp)X*GoBc z!|W&+QKrA*1z$nQ=}jHOX`!NNj8PFMSHyG1YZ@j@vHiCfrH1K$KeA<@IUGjZ)O%tJ zy@;A!cVuKXzAWyL+nyG>uObArx#lJ@C58MA=!4eOd+9hojgdyvUUsy=&4n^2FCe528}CGF<+80DO#%WT zlrQrH6uU!v(7Qc`^43Qqf-z z?=tI|IwIzM;u*=5;uR3UH;n^;XJ`N~`N!5w>K<5bu<5(|WEObZA7mBqI0_)wqO1BtxH%xb3%--g z=FTmo9Es3}kss`2w)DX-9M!{aZtIf29PcIM9*s=YJmg0EIs(l2J#s_BOv(`BQ72(y zR@xl|EQ}M_9y{W1c#gLL{&EUhR%;#?)og9iT1P{xW^7h7HfP703O40;Bj5itD=^qB zImA!Ez7A#2416oZfbS>c0vasc!@%@1pl*4OZ}E_$_n@>l{kYYPcvF-bPL{wZu?XCb zzDln5^_w#p)*NNo(B-lpgf-8Wy!DW*q^anVkNg;QG%QCTV=ES&{t@xideQ5R2TPv% z>@K)aF8C~ukPssz#3T}8`~*m-#NWi=Yt7Sl!XOg|5&d+P+y|N1y9deU?_aw$qiM?E z)nf4~eLGd_B?^})Sf;}lsJ~p*pDbRv1Uz@}bGf&f`CIHZE)-R;Epnf69drJ zE3F(K@Wk|yqUyf3s`IGo{`@A+o+bP5Ac_u289PKWwl`gv>dV(;p>(m7ma(W=@CyqO z+53r&zAVEbi-Cqfl940!B)Y(#yhM-me$+XO!|8q&EJ`6L7R=)tcebSw%R%1^fQz(I zrAr%eb0*s;1c6Xz?=7gk_I;xEDuK{{Gdac0S6e)?Tp^X4pq2YbmAk_#S3>0mYvtZn zTV8YVvVaZXc^j!Joh&QBbiLRXlgJtoXs1_KE{k#eTA4 zAyoX7n8cl`;u1&25>@fb`CJjA>|t;aQ8A$^o=X+?kre~%@VMTr+0KDgZ-+^`5HGpu z2y*mj5IQc`gpTi@Qf!)V9~OJU;n0cPM${a^R`Td5;!v^5BtSDz6hu#U^VO#X2R|T{ z-bJ(?r@QsnsayYO=gFt;DLVPY&Ge6L9dBIiXO#jVJF{}kOn#T)%*rRC(qg3-BJ#cC z$5<{+y=xC7{6)=Yn>~igy&N;Ap$xQ<&xhT-FuZB1@Q42t!@JIfkGZ+x;hrp(@OaVl zf8O?EB%!%9eM6bjYRxgCd-VY~XWl1kO&?rFUCb|8zrrPHt53d7TYZRUWv0E8jbYiz z+gm+G`SL+|S3t5ZF!N`Zs$u(K!N*6+f?HqN>gG+ArN2G%rjF`wpL3RUZLg@q?M=?n z9_XJRzOkeF=R2JxbH{&iZ!>54HA~tnFTK#15q_24DNU|nhn}`gkmId3MjA^_s4?Hz zEd^N1|IYHk_nG}ON@>9<#F7(yW&tb2wLEHU-|Byw6Dd_uzdM_;U+|W?WW3(+YPN9< zUowou_CznzkX6ODB}7AqiW+s@V$z^+x(qhS2ALuA-e%xm{%c;N^FSt0#)W@nfF^3zK&goDGei1o|8cXTz(+UafUmp5Sb)A1Rne(C0-p3q?>s<-|f#G1MNQ*+t^EyZfQ7nbeEC4#P^^G9O~PjLGvU#HOrrrnqKlz-Qi4 zNJ(bBWKPwpnE*(Mg4|)Bw!Wt3Ar;ej>_6p9b@LDM{eYn5uV$KhALJ^DCXoz&KE0GV@86P4rkjGs_X@qa}qBQlOYG>h6 zL563zs%AwR{D= zd(T^$oP_U+dzHUP?>-YtK)gG`zjQ!<^-cgf6W@&+1glL9Gb+X#0UnZ?9K)1ilh1*Czyz$O;eMG@tdchUX3(>egeDo)Z#m7GS$2-Nx zR5i%6@ab&*)9AEy9IZL~j5RLuRZPaR$$f``6yY;_H>+!m`!La62V3qF-Gxq%+@vr`DBhkcE&-b|6m^p?6KBJ zx-0{ZlJ1i(qJfZj(m8c%#!aQyn;8uotI&JgA0+f z=_MlOSX+q+%=<*$eZSa5b{oa8izzL*>hvg2|XGj37oSc$p186)R%X|io|^DoQR zk}+!x@qqcXj#NZ3wLtW`jK8uC^6caE=cF+l^TL87nmRD378KRWl<1Z5mi$D@UQad{y@D8ntl15l>Atj@9r_=we(~S z{<>4Salr?1_~Z2FT*FsmI)Cd{nIRa`%h0ODL|}9>x`dW2?5&B@S@#K&8>elpq{bo~ zVf)Zk5<8PwnF%*D9C&u6*6rU80eMgsK3LzRsXy28AF<&-yy6dx z>#+IwraVJat|vxn$fglaZm_|;&DCT9=11`DNwDKj_ zK3a(qE^(p{irgcM>^4;X%;1XC4GB^;gCtF!`bd!8mS^Xb$3xMof@}r?)oyFm3;^cP zx$YCahqU+qxO*4)sEX@x{O)Eq*@Pq;1SBZJDgoDmxJuMTqi$@}MWe2Y8u1myM|?-w zV6=eHL?tfqiPmcSm0GoEeNZ0=_)0_}ShT3MTK(2WYxTxeijbLPyMGiT0`Eo`s#*sK}{Q48MmRU|UUso3e2UhBYhgExJW0`-*w z^$dfO@~#3^ksc|#U&f29%Y3+x4GZ!ob-;W0Tr?VC8jYAM_a%EyOSN{#DVb~w4B%1} zE#OFxVYuyQ&i!DL#`pKY-_oyYQ>jJu4}IP%`Tytr^M4n9sr=bUFHgRC4;>C*`!?Z3mH( z_;T>tW1jHuUYpe_8XV50OVs_whNs0nU-&XZ<+*i#}T$(-qksG$=iA}>IjNvd15AyQmB&IF}Tvk;Anh~ zh;R)tz|Vqjl1rGA1%#NMh9>FL{SyrI>9iaJ8%X3HIEfbKtTnis9@EniSNqJ?t(UUDNGIAAV#i| zof1~=06GNaoj0qZQt%*Sj=$%0SQ5Y9d8ptGxhfkvxI9RYPmxJj)ecbkh@_+%qbWEK zTsn$Rfzj)RWfv?ksDc-jtkkcqsA&tpZAye3<+sBgiN14(_&BiF%>*+pju*h$#pK{d zHzSS}rW+7P>2?-4#SHHPfD*OiU!5Wr?08VV%t6x>@Y*U?=4ZRW%KVeIGCxf9W>8nK z!Yq)}-pC6)0Gtli2MikBwLML6Ufe~qtU)+;3OLK=8V-{Ryzjw9>EalXy^-Tv3>0R= zAh6*(moQ6$Xz~FVAr->*B3)I&GlkK!!*Jpg1SxZn70&nQs-p;Mp@HePbd7e)mj?y_ z1()l1yAZ7Wjv}zQjBJ+I!vI1ju zg_5H!4Gi2-z`#3;Hc(>?>HEnwf~KvJBj$x@T;lgCW3H_?=~A>mYlkgiVxHpoc&G_v z?Sj_RW|I2EjQB)zz6x|2B7Oo?_Y^E&Tmey?yfp3QIo@+=mX_tKeHDC`2phB6aDF}| zL`KgVi7RQ&Fz65sK}0kurovZ|{tR{LO@l-N_|Fk3lVl!1yJ1cR(~D1h`Ur@xb8`Sv zEFjWfZg{eq8V_ld<2pAEy<1Ye?98T>^|efTnb=Fh~ub#@1s+yK0o zC1e1cBH^cBN*5pyU&XvCu$;MJCIFVv3*hBAy3d zlQP=biaEv(hheP^ZhiH6vYCTm^j) zgH|k6I(Qu{&-*ea05bFty^};4vYF3CWOUdL>X$JT+$xF^@NAIHqv^p}d!N%haA4!L zH#`zLzsPO8to_g~0dFGESge*rt`0JWz`e8>)P7y``i2OwM)V}YJ~JPMLZA#2Z;?(@JmI))^o?*K%l=h%D5 z8S7UW9)32|&ewGgoWMey^iE!n3ggv2Y+ZoObnT@f@M-_>UIaZB$Ra&YTX^LQylSvJ zHK{H!ax(+Ak^AWHFmicrBgc#G63k7X&qH-qQzZ{oP(Th>lkn3%Y@7pYu00;}(-k=| z=Tb~;codzq)*SbAUas{yKURX_7eJw9sITi5A9ARwt}R|rkLSOJDOxRFrW4}T2qMXf-;)(E!dlWj|_ zC)nCe5@)S9*m@8D^dv>+@OQ4j~H_7+iECoKm+Q( zv+Ux*uqycr)T9H6ZT6wR%HO_OWJm)+D01WE98-dBp+n{Rwwvd!s2h` zq_SsQj-Kl>*zJb5yPaIl{e51Y+8esPu6u(?vUmMh!;5N@7)yP)5j9gx)~O`gHJA{# ztW7FwC9OZgI)Y3`&;533HA<2r&>h94h?*B2LSYy9aNcF!_?F|2iMl{X4% z_^D7#EY8b8BV|56QoS7jVRYGKp8*RiPDBKRTO)-$dW0?}qY(o7sA)RGL({}La+(gN z*fMck#Zo0EHG!Z2s=)M8jKZ&vuM~_byx9f8~ ziRF^_qw2YwrpAL~3+Ucvhg4&)Zl}iLT0GH$^xA&C+2xG4wV4F4IhQCW@*rU-Qe+qM zvj?pvAHBTp$~`i=t90zXu!7vtFQ@=|HAm6C8qe@oKIO%Ite7@~j~Y>MVV#^xzr05B z2Ktr9y8(Uq3?bgE#h&}uZY!miLO_Ky8vYkKtwE$s0C^W9M~aL;xD@`CUKsU7R=3hg zQQ&&L6738AOPwv~m5vYvj(4eaSbvpxSl-Lihh;9Xk`bb6K31hA3A_%4_~9bs+pYEt zkD=GEr`Swl2Ee(!xwZozT-7RTWgx0B5*4VPYtcTr{RA5dWq<-IBL#WYg4C4hg4;6& z`TeVOvt5_ol?*UA{S%pvtSQ(kMZh|W(Oc6r85Y+Be)hTmDJ8S8f*BhRMi+6|37`*W zE)YsL)ui}Ee`C&yw{h4ny<$$fylJaWms7opn)TWecVu`V5|~H)nR?hvp9%bw3<4g8 zeN##=p5gKYrXtwdM5kZz#i3whMpGrdO+c2df(GDIrgMg(wF)UO+pPmoefrcE0Mb+r zxp1sT?Zs2Ou;-}sLDWWwON@{u-c%)qT5$crh4c=lo=g?)JpeCWD^Y?~ufJVKc^}$8 zD#UzN&YotGi~-^5j)nRl7XZny3w7JD2mAD4{|7OARl`1W{FZ#KLG@j+xt}0FFECoQ z4uOVxHw{G(@5Kp#X@$=N)+!o+Qg9HF8>$=rCvLJr*V%;Psd~#j_ibhWARGf7>f zN4a>jQ!ZBaK<0m!+HDCy%MA>#dRdv6Sc>_D)3wigV85>7pR6N&;0!OtB;?y-dP0vm zXOKti%DO(VlS=LjFr$>#`!|GHOfF;K*fZ0cu8=p%(YAPq%9U_kZ<^Kr`@(KD z!@z@Z*XB}zE%;j!2VpR}_4SD~;&l!*)&LJ4@LFgS zQb97#siYE(nBA0&cD)S{<~WIBdqXM~qOpjWFZ~DMl@jzxaju^rduVEVf7vp&*bfNL$#)?(^Y^Rm>g zZuXR5+P8c4vZF{&Id)?y*eA{qf6ULLK4VgcdGH*H3L0h1u?lwH_7W{(emp1JzQ-jr z7fxV;CUE>ln82TF6S({eoWP)__T6}cl#%#7?ZdAW8Roh}`hQP5_Z9h`_HQ*i=l}6& zQ8UZ`W9@+aKMul1E&q?#_v8O@Aol0z|Mk=X`+s$o%x(Yd4%2Dx-9J2OzdcUbZ+~^F z(|+A%~!cWg8hMvx~X9|2hY3djtB}!kInb zB_5hId%@iErQYm^(={&or+HtY_VUFFh}o5C8V#|lI$-F)&Xjb4=EAU#4l3yqP_}om z;E94dD5uP4wn;CgxWA!zas(Tdt2kVEd%0pRi?s09JoFzp(6f|M4^*ch6}7P^h7Y)@ z2z;#Lk1%cRZ8fB!>@Y|}37SeVy-V~4-Z%_g#wK@%8OG44bt-E#l0;KbCEO~IAMMy%~`bZ9@!^S7$Xv(5AWpfRAS)$vKhvkomT>I=& z=C1E`B_f0v)E+=ZXjR|#dS8ZEa*AWN$IbK+HnV@p05-fX1;Zd||4!K|R{|1ias_og zY`*cU{+76_{olN=>X6dtsAbT(x5nV3K?8{9H89$3L?N~9GLrj3qApPgf}EKTB+$EX z7-sMsFRkxyVFoYIX7H8^VOjM&=%<^SdTbdRtzH&V{HD%4a+oEXFQk*2ATwysym<}M z{CJh{x7ToV#0<7u)Qcy+S${?nMborP38nqBP}+1sZ>Hw~@SG-S0jlTCzzSDUg%DP_ zP^+*CDqQK)5j6EyKku?yO%jm4_0Cr30g!!sTD2f|1M6K!_5OwR8nt>;px(E6dgF`q z#xKztpF==i&T8BTKpsaR`8<_j4*@}TRDnQ}qpM-U7eRJZ&-QC=+)Qm;pVh{q>7tEz zp5}k`@aKyV1~g^tpwvVgmH}q<1B#TLl{2Bvx_?m2${21TR`sO`OQLPH~@LY zs}WoqdroNJk9zoM_DH5I$N)Z=fx}Pp-Mxq#G^YzyL`0?UY_z6gGh+u+Vt%~J!fp{^ z_soBs*j;zNzz*$eMK!2y{%VrlC9J>4>(xgf;L!?Sze>)O%r5t6D4eOG@JB)+k%hv> z^8^b2!@Et~qj%vKok){2k7e6aPGo+hcmurJEt6oSTdkZ;p#t7OzaCe1f z&B22ikh+sl&c<(g7O#G4jHYRdq{Tt@FL(5296VJq@D^|gdMGjQnH4JetPRB1JGdfrG*z_J1jEH#|N`{e1?y6e3%%Y@?%i%(-Nu|Q>Xlt%R7r4 z{(F^lZ@%zEU)F0econq*49aK+p8r%Iza9KLGYNkFE>Ikg7RX@=t0>Y?4fZw(ErvKq zTOz|SzHR*5oqbt)nK4R# z5K{4C!}M8jSuZ1%b?@T#K4)HY08bofL^36h?9so3m3Nf3^1NqDBa$xW6>+1fgls@# zfu3i0>f%V5moVkPd*LPQq@GEa3ZTq0mD*~=wyL$Z?m3IwN}r2>=)xHxmNE4POi!`& zBdcBL^Ck{=(2R^r_H`Glcc{w?piAh-u8=;+L*HaSNOXH2Mz)z{WO@hu3PMAq z4}$MZazFKIs*aqh2_c%bsx;emW>`pXaQ1*%o|erL)eSE=NsGeJM3e<;!rYzoo`vQC zry*A#K_pj^k_w6sFYn78t{~TW#l(jT{)x5fK6PCl9>L2&2x`K6RPhTs3L&&rBK*3W z62mGHe+4ixq7vbkhe}2<@n{KsR+R)f>i}70Rv4zuz=B7?&j9{B1TXJ=Aa8#~kEf9i zukn#S|8tB=^ zYn0%|nUg>6&A`+p;9OUR1G`R@di=Dl($mYe`I-^}PH8MHEME(Nf~{K&{YVmGAry-o z{Vo-GUwo{{WAPG7mpfbLLauzDp`TVdxCj(AKYs`N@(Xp)c<(VfXh115 z@K?+|+d9j1>y;m+%7hQ$Hk=~DCXj+j z0X(20K?fl|bmFDkr6aIVgK|W+&toWDHh}1HE+2}Kh=Q$Pb!?5pT#p83`r9;1Ydrca z8GNjPqGpVXJmS%ggySEjnSzq$Iqy)3RxBZBE2IPITSrCC^XMm{7#D$}<_BEla2vMu zTm*`m&vB96ZAH3ynCtO90~gtBBfGf_1&W$iagq112-yq|D_Tq)f|6!CcljAZ@6aM0 z$j=xEz0qr>Map_Sp1`P$>4xR9Z&dUG^Lo^z4l%XmJhjk-8T=Wl?j{4(p>eZZ-wTP{ zBnt22NdqEW@MPxynTh&b08{>9FRLac(SQC1;ruPE=KznI0W50kDVxnwZY3 z&VGvq=5IE3wup;BQS*?uERrB&0Kgk^Ix3)d!PZ|JI%#(>v0#yHZz40gUd-s_HH6{q zJd;q)+~Y5pKh3J%?bX_TnW`Tu{jDyfF$Ow-+W$n z!%hFsTJWYHB!B_=-x<#L_yIHVu)Hll`r%vxul3#$PbM=(ZU6k=7vYI2bZsHOPD~fb zCv?6F`-8x&__q3Uz#;gS&$>+7?>AXb`F8$2;EIL)y$8&R4})`}^y0mCXgT6x01R(a zy*iFadmf0tP;MlT;RAZCyd~;uQsHdN>6QIH);EaQ(+xV2r>2zSu^6O}DFO48s1ojY1M6`VV~umqw7jE}|1N z1uz;+@TO(KTO{D6ucB+ON~$qaGr_2*oE1;w(m+t<1Zux5P^SW@3Kjf8RIJaCQXJQ0 zQTo@){0@cc3m>GHURiE`vqkB}yzgMRGkHc~g^;B~T7vuC6WEDhtH-MkQCt=H=1(|E z)z(v^2D-j@k21p5;1n{%dM>>0{F61VakG>fppa6vG#E#|?;MQy@x7Uz8+9RK>7pWj z6;S&mao1@ctygeBKe&3?D8?yD7Yx+wF0(T6tJt&eM}_|piJ5B}!FeYTWa7k=I7eX8cGb zW;6|&PLO9KDk6d&vZnMvf!}aJU|M_+_Z!~+_f7y`>D<*YRTvT>j8s>Ptn#Sg($c$lX=aU zM~j=C!#wexbE<|Mhl_h-o(A-R$7qs=3fYMcG=Q18`B4}!NvaDg7VC%%B1Cpk3DXEj z5}Uu7nm-|{`Gv<~^VjBSl4AF7Nm4|W?p=Q*!hMs+j4Xyp&4HkABhW(-^wk>Z$pm_2 zo>j3Hd8-7HD5JT%eT>)U-w6(NpL-=C-1#W=;{a&vK5A^$U!DC}J4Kopoc{}K_F=Cb zUsO!-3+kTvUah+xy}J)R#$voz7nEekfoAieC!`=F$w6NGKpFGKC!`=_uuQ#7&%Wph zNi9BJ^C)FO8S~gDENb~9bu;s>vzz~p(^h~?hTdTgyZSeTKPu-xd$f;CEwP^@;DMA_ zyI73kVW@t@7M4;AFJ-mRLM>dRYX0{RGH4)v<_SP3;gR9PXJ_`=tQQgZO9Xsw7VvQd zyiftZn}Ijd=OcgU6T^-$o&8CZqMX_n2kQt3EnbF37VEffC_fef* z%!T`V)eT`WU54%RwzKzF9VL2ygTAqs{F-`?E4T3xYeyYSF#K6C#siE)`M`y&X8A*X zd{U7qH$S8-J8Q|x2vsn60S=s#q*+R+O~uP<jXwxq`j z8k526vhKfYg!K3`fzGX)?dafL{-YZR9b9-ZDMKdAJ?}*sGA0&IxhKbS!(94VdKKWm z^&q96H9VGkepnQ6%WRguBM*@&zRxfIeV+{0%qbr}2$Mw>lnMbE3aSv)G^XgF3TJxM zBjBl>5AveR5f8PCrU&R_9b5!1Tg~~uryc{}X!F3J!KA2j@?6|Me}$8i zkhb%1GA>Kv$=EhoIpzmh0w^m1a?`N5e1Rl4w4;y(4c6DYG}6ERr9P<<9_`hj1Y3uB z?56^`SH`hM@~W&;!G4r+nxgz~7D_pO>dSJ2mqV|I zS$mz_XITf#x=fi9qF0J`K8&FJ97cj6)7D0_j`H>9G27UCY$*L` ztqKWe(NfGD_6$x=guZSTJ!Rr|KGe6y%f^mryF7P4wOyJXO|y>oP!CpZmXz^^SF_NR z3fgPV*qqVsW+cl7z3%GgiPEpJ-OY5{_7g60qAqQUeUjG^ z{p2-BAvCb9bPp(R&dR-pbsbm6AEBBQ-73Lb$AFjGjj48D8igAKsU_yBogDK{TFRFw zMY{U`g-bVQU=vz^Z&)qcodmRsNf3vP>`$xdo_KB->@G94-R0;*gv|JiL6WC+0E4#8 z4|%9Sq-2nC5rw^UWQ*~eBSr}Vq{1GeoOZB1b}f4A((V9`xC9GM;x~aY^oMsZm%zrK zz{ZJ(AnCohgs%-OBQ^; zSY8)Ci@Lm6v~+DBc)oXivi*bMzpxtN*8=>TpX<#9KijG$X6Sq>`(jU7Q>0W3wf12b zQfs52wO>Ekn=AUjkJerZwfC{uRLwJ<>!-aRJbo)zVGEB#3-5pb+b61U`J=i;F#PP{6y`umk0}m*L?0M{3PR_)%wvnTF)t;HQ?>8 z|C@12;7IrkBbyva{%iz0mQQQ8+FcI90nnpSEZD7~?F*P#`=0K_JNUV1$s$f1KD1C+ zfz}Nm2xk^zUYmR#cLS;)A)1uf@wRtrFSTGdH>XaL_F`|m4n}o74;kbVDndd#W{qB} zy@`pMi$qC$Y{!CJ!n|@m#mnUT*y*Inj&`XK?C{yPT87ZXLMWCtqY>3ad?kQmEsYQH zD-|%OUO+b=^o=%|(k^)cjjh<-uDSqsHyHdJ_p3h{eYXC8N}u`t)8|jipW5)r7w^ai5R`L6(#7HCxZWCU#-_tki{rRctTt7bSIF%LMrR~b&n`#&H5*!=QZ zZ_W{M_3t_FDjrfr*U~6?Uxrb-R2!xF`=M@RwTGC- zB4iq+;@7?6S0>onz!{6t=H4{NRiE%reDie)9zJnPtppI(*Uzq+&I2Qx4f} z1OuLUL3Ls{^9aeNmjKDDG$j8uksBJep;tBd$g&ktqvZ`#kbBdH)58)r=A$|b zvA+($`LY_>enudV|&Jt;X15@^Fi$v$kSJ7e6?Z}bJ*~iubWTxhmM{Ffk z?ymOqnVQBqVb1SB{~57#JbknsQe4%jiRqB6^tBsx-Ss)I5+um3XC`;cxya<2eXc6w z9R6b)XuUO>FP`70n-$S|`%8u z@7~G)@Vl<#|JJ|B{ngKj@6X+9<&JkZfb!s5asn)4xKJ>(huIHHx1*_;4?V|8?pu$M zzkv;)8vR2!Kj&~AOql=s)DhaEPl@3Tg4DiIDtuy9iBJ+YKr_uh1n5L zn4MCCr?_y>ogV6HgM5mCu?~_WaEk-iZ=v}Chf0oz3!i93S5bA#!zqj)4ZvQEYz;Yj zsh7cvqUJf`Jgi*R*Rv%Vc0%w|kCX+ma;A$Sn(7Wot*OEBhy|B|V*fxGZ_k{G4z#*d zl^B1-a5%f5dioU!;6Ok}^Z}Zh?-@|tBI`GWlZe=W?GTh`e0q(%f6DXlJ(MR!52kkH zPp83>`tj{#au5j*Ff|jF!yU4D@jW)L3lt%ZLr26`$9(9)xP+hc8*0>>)vKK8`a2di z{&mc5Wn`iHU-aq@0&pEgzznJ9JQ;lkg$(&XhC-uB1bY6#6d1Ll#{^ri@@Qyk1rCFe zZQ7ebc_B0h;&7e5l!WULehb;+hBJw0xv$7O>|SP?u$i1bxbfS^f?r&O>Q+eD{d zp*1ibyT8it=wq3H7)NRl8CT1(uSSaT`5s$qwl=J$krefqHAIS0ofP{8m3}3ChCV4; z?*}^8vV(f8*o2u3@H}dPnqm|m@H*-5Af>$rjj`n?1&(Uu*qe>gJc-E@?~bW_`Z&#& z^P*=<>2N7MPQO-vl9>n~xEZwMax#6YoIr8#T);%*djfDod*a+0w4Dldk0l}C)Y%1deBjI8HB23_yT zrTg%;HfwNPx|O(eUpMS{(i))f(?l4fAjw1SKwM%c>F#n^47-?veMM->bsahmlA@3} zQb^skL?LmWaDFiiVt@;I#C7sW$YVT_2OEN(`!tb9r6iA~d-bx)z=b^Giag@x*S9&y z!{Hz4x3_2d@wXJ^SDs4d2vJ8k0fPq($NRx#Aw4(UKW)WMRo=P6caV2tDv7(#rVlvq zT!k|ppDsS3;%C-S82Tu`9*&sD0{BUoHffEF@XdJ>4HaCF!dNmF&K+OHehSB@AZEO& zH~TH;l{6J!EzxXMaV~zgC!Q|fFyxj2ByehzGrSy8(l&l|RVQCW;AvgfI`_gnf+4w? z6wJ~b$(KVap>CjHhV&>GL*Nj*DS#eZyy>?I`glpQ^j{g(SVG(awm6M50eOs zt3E4!4YgxT_EpSA|A1xj&Tf1^UKXG3o8Tf#?Ke2`qUW$72dNmLoyOtR0%Jpe)* zh4~2(iTXBu00}9K5R}6|TOCWJgNbb}X=kw|T||kw7u$C|C^z_kjh6Q>9CZwEmmUH`_3M^T3-Vs= z+?%OMAD>A=uDvB{p1-e{Q#}MUZc(odeXTdMu^U&r>QM6 zuC_=iN7tLD9Y_uXWBiaD_MSa$rZ!ZJ}J_V|H_uD6&FJ zMu|mTP)JcTGcxW|4CS-q{Z^b_6;y0&3Y?If&+l@>%77i&Db9E}lPGh)VAfb2NZMb2 z4ET1LW1row?F}Fw9UO1>j~>nn$!ZKKt!&!%zPyNSw0`$x9=c*+q`k;x<2mHDD0b<3 z(0OmpkS6ta26E8P9|I10BXQ87z(F?*M@?!ONtm@2aVrR3Zhsa^AlO&W!_1Xq<|hSwpfukvkcZ)x=7f=~;1!ifP{H|z7CbpgVUo{d z$u~L{)cjzA&a!c|r5k)@m-A;8pJ&8e?hPZhPkOAq;I6V@OScb;g{*;folh6n8X#V2iq}0udzY zYbv8tI|B_1srG4hQMeZ&>^Y5Izroa-&z^&mUVzBD&}r6suEw>tBBOSaBe#+#A|bs- zbML3Q>wC5OG}T4?S(WyaA8r?@x#2zCr^EAU@M8VJ!wvp0-$6kGqP`TO*d0%mY&bAz zn4@8kTRIB0irP^tLQbv^hM$GA%jv*-SqB6VJnf-UHXTcIdW%_zV_$MV0RH^@Tq^W_ zqhOfe{RzWh?29b#s*G~OG?i3(;#WLmZtBc~B}FT-ApNrJy~}!$N)9*_duD-xyRqj?rpfm>4!=lVRm1ETT3@J4iIh{@!RrJX$+(=%K zovWQz&E~=xqwyDUmaPv+&H`J`C;=w@W_-238V1Kj9N1yJd)**h-MRV~86*bXiti99 z758#dYt?0!d`OfeM-%>@NWcx&`7W^Np@OqyvnbhM5 z46Cwo3k;k840sJ2exB^U{>rQMwUqnHd{f4(u^>PwtfnaMwWsRCbjs^6eX~b$-f4Q? z!I(E|xBj|X!+{cZV+Im1-~LMPW2N55O4-M#na;QMQNS(A?&K*tHv6nH3{6!LZ2gU; zNi*iu+tetU5=_FZ;-a28lVnkkttcqcxO8et6$P8?HICes+M(ChoQ&0Dtn9e*ZA+ZXAV0zFwuQ&I7i#^!fMBi`H zjylp_uTZO(_V=8Q%vvuHaGNgwB7^Xm|A3freu}kg%Mhbd{A(2dP8R=O6aR*_TK{fs zvHrS6_R?Xi zbf{%>2rqs@>S1ii;Qh=R9GgJxGCuO&J8tfmdt}1)kr^gOW^9Kw9nP`YVSBG^{}E>* zCwxhi>Uy?n54OS%| z2DgFFmgZOtUE?0hqACee7~f-srnsCnR%+Mc=0PvWGXTofd#}k}GCAXm+WLp3`==lK z_-^%D4pDY+|4xTcgRW~+h2sCl_|BaAGUacthP7e=3g>;G^{KQ95!5Plp4yv{Um^3E zQ>A~IM{`Hda_C>nN%PZ2*UuP7Szt`p%>lyK?wv17e`{Bmk|y;g^$l1HQD-mpwVY(0 zH3odZ{XQya+p5d8@}4>SXO;KJ@+ixTDyb>uB*<&kQ?c@~H_Rp4fke7fu3H@|r&Ot1 zE@AM7XiJ=i;hkr04KJ3Amzl}Wq?WgmwY)Ot`p+_a(@N(M*7QbnP48pu=Q~J}+VEJP zr56kq+@3>A$1D?E3sCV7#kCq9R~lAcSZYKl|AyVAtaBp$$p(g_VZp|Ny`Sk z;DJLry`7Jn57Ue9S^y^YZnVOLD@TTiQ!hr;Mi#wsPLJ4y|7cSbf6 zTeoe<(0waPm+O#X&HT&;Pzru;XwSP+9Dh2RE|oU{VVAQ#B+P3r0~uweH~n7qvblzQ zMuz@uK9Y*Y+Eh#`_YxtMeL0^q7cregQ&Y-VJy?^971K_TV0~fSml-zW0KXE1!Fqwl zP^^>LA?Wq7)a&PNxAi(-^g10>9xMDkKjZU%EHmt90$|Gh@%~zpC)7qskw7rYdyHiC zIT!in3XwB2Z0-JjgS}e0Z2r0F)0rwc72t{vBc|7oOUej(ngJT9tX@`7W*93n85DOj z!*0j6XEU+FK3tj8erV_I5sr=hnEpmmo|#;H-OHZ3Sn&obWv6R_&=4;`xv7H}n&n)i%CQkINq4*WAW;&DrG5 zMvi1ijpS{Gh~U}TW86eJKbJXWD(B2X`@EE^oa#b-UY@Db=4B4`_u6*nynMY`&dZE( z{m#n(=jZq&_VrRZ6Cs+3T?w^Q921lYU~-jA=IJy8_USBVe=PxhI#0REn$C9W-m(SG z=`7v6gXz@f^V5p#`8>v>&u5HU{{Bo3c58(@&OgWU)b=9pI}4m+J#W*7OizzNCW}I@ z!P1>ndiDbC0Wg+61xmkQC^JMY0CQad4#c%i;)t1t7uX4^T;)_0*a<46a*7J<1Qk{} z>+>ywikau1ruhK=Gok%?%Gt@^Y!o{U^FEF%Iig~FF6+1*($lu9$+^efN;Zg?Q)W`y zJRTg4pq?a*FLe;quk!8W#{@Mu-%fr^te525$uCUFC+F+r_t(oU^7|(Z#`m{4$?vw_ z9g&|#f9LO&O@EKM@ssP$pzsdaT;m9TF0L`wpNne@^6Ok<_Ee2){15eOUYm20pWh(4 z#$kKyfNShve)nj^zBn1f=RQ})@E4yeV_4?XXXcR0teGL7ohARA@0^+5?{dt{4~f6M z>|_~#d(P&8#NQ6@zuhhV_9rL*;P~6)8ki-eRQ&B{HtX@X|J~pge|x`t!)a;#gZh7x z?El9D>VF{Q-v5(+c>gcX>i;R<+xma8YybT}PX80K|4)8Dp#G0`?|i;PB{-5~6`+s3p|6BTO{lCz)|9M&c?r^2j$-2xk<* z8IN$T(BRA?ILi$QXZ`UGI0BVZDXUjx#alLeW*$Y~uWLm;T9Y(XgaZCR^CS30CTx_bZ&7@uNB~WMV+4n26f6DuRW*C%E`0ml#3i6Lu;!kc5AQ3 z<8x`Sw#b*OMa*(<60@{6y*b+qxV9O{-DV(nn*m#!dC(?INi#|B7>uQj-LplsXt_9t zBY*P^N7J0N);w|dJm(6_K_6ifu?g*4go8(a(bZ`#B;#=^wTR-^PO#a^ zHjDIP`6|_QKJ#UpMLFg(|3~X*p#OfB{}lZ!=)a#Ma`ZD#rWQEc%+renRjTWJ24$Oh zIp#B{0~@4Loox=%i>V$Ba-Git+2)`e^I7nt^)t|aKg)lLeiroK&k;HLsrHpDK4iG| z(|!L8?r8rEy3%b{KZ{)Xko)@d?`VDcf3$vP&u6av^T+FFHowiaKHc|~EIwr0SJWuF z@1Mmx+CPh3={8GF805-_+}CIRj@D=XkJis@c_`QZ`Q!C7o8RVIpYHog79X(fXM!59QiFf4qKXpI36px7g2~^S-rC zvxe<7YrV-jXT9^pD-{%FDn3Wv+ws&Js;W7DeTL2dC#^LrV>NI!b zmw6SP-T9z`8$`Yq@v+|6JXN|HeKmEOL!nL3&1fP$m;_ z>>{gU6$Z`<{tj!uCuqMT+V4v3cU=CiqnlkYiE8PrwPyP5?AtsyeGX6Zi~_^*;1Pz0 z`@vS}zB7y=ug8m$YH&e`R6i4jwH}gkysxZ4gT=j5{MbI6ngVW~La8-qIEh|ws)I{? z#{8P1X!><*Dis4gESj)}+p6O!<(YH5beL2x)9N%=iW2LxChC9~lc=^{;6CY3`P>^h zc5snZcFj?wK5gkT!`K3x@6W!g@WKI638}*y|<3fS*`j+6>FFX=6WDR=G-9u2LV1HuiI6 zm5a)9QN8ctyGndkS>>u^xhmNaAkp?#xC+|ZMov{@o#tqC9EeP9(>h>baVEP7qC;ak zaUo8vh>UqV^#(mgw>pev!m3QAtxQ-eGeMP!+REr{MO2w8TbYPfrc#xO*~(NZ3hUkt zlTuTVxu?)BxIS8cO<7m>H7V-`vg#(F?z_BqBxKz> z7o3Z;>efNsUqQ;e*X}j4U8wK;eZ+l3`@4@&SC<@*4j<>DF~FS}^zH7jRkpQkg;(ab z&dKtEBJHJ9;2{8RW`a-I-jHZdO}3`Ky^Oj}+=tN5}R z{+@RVXj{N|o4d0=53mf!5TY!wC& zzg2OiC{-$TR*E<)MTqYPf-B5&3;#;101g{gE>pka6=vyn6-k9*pA^zMWPYjwU8*OVXUT@G$V%daG z@7F=wGFD_*OdPCtYdAyI_->=?2u#3u0uIjF>^cGwvy+P|-YFeFy9`35dDm9*ZeVdo z**Q;k=K^}?&cL`N98p98_vsxxh=I@7Sg(8(ojo8E%>1pEs|Bn4y|xBIdSBn`qYC%h zDlD>97^!#nA+E3}tGnLp?iTCay|jGr`=ID=e{9Se9da zBIf5?sQfZp`Ho!UQ)xcO6*{uIo0r{P+mign7V2)74S2EM-8u%|Wy5A&uJM^*?#&g} z*(!XHYkVT+#_y@Z2et~`ImV~bT*c+Pv%2e(-6fB@V9n9$gKry46AV`6$GHHqSct5t zRraXNJj%#gGpeL}GvRDVe1?}9UOal_Euc&nGPMf2wlG2E=V~D@M+ z_g;D>$B>Nl%J^!+1&`b93oy*G*|w1-lJ?dL@*3D>Hf>~SM5Ki{?YxaFiHNicPCK5` zs{Vf=UARjYJ-^#Tiyy_hNo&o?o4g*qbG#Q1(;;}`GwKjr1}W`UP&3?K*6~NZI`rci7G}m7wrVRQN(F{0J02 zi3*GD8w$r}ic&XIsi&yadMGuXN{N|mFc=Yp?Kd=ZX71@o7uy8x^-4KzMN`2rSu)Z+ zg_nz%U$ax#2BH@wLhP1BN3QZjG}!X4jvR;>*Ocqy8MaUfOB-3*1@_{%Ak9c@@j>Dj z5hdF*N6kfBkWTkxu17nfTssEF&>GW;RJ!TyF(kzh@5d{y;L<|rQbJFrXg$r8taw`7 zioX!=G1w-w&ZCB+F8+46((=dSZ=1Zxt$x=9TRu0WdJJ5BX1WCKdjXt;!5MWFeSu%P zaAX%+q{>5_1u-$>D*ZENAJ~{RurUIGi$)1_39ZjHI9!8asP-cN#BS zj=mV%kyS-9yk>pR*XZk9B7$!c7T{9|Gr`B8G8R;B{{{t>ut!tdIoyu_lHKbZ#19&PC!X~-is9> zP~912#x>zd9rRRTcjO?$?wg8>@{3c%qP*uOSd@aF_XTLNtrt+ua_U3^-Z90JBLwP+ zMFRC?4g7*mB(LgumL*;gX3f>7^k#(M)`<70d@gSH!~10%Vt7yjZd7}g>XrU`4pzFq z*j8z+UMU2X=INF8ZNN%Lm)I(;)+pmB(RQO` zdgzZ}HA?kjZ2nn*uH+|_0cVR*y|Gm9%BjHru$~OZ2ZgGs&=M;2JrtUNg<#>M^1F>R z8^NVr>kLa@8`f@bR5;g{Ck}9$f<9`Wz$C>!MsCe08bC z`>rWZ`w}-xdxc^%6_BbE?EYlJaAriMNS%w?CGTZK;=!k^aH28QYG-#I^34=p;pfrq zhpzgE3?UD~izTsliN#``A?7hw zKpnlr$D1wDsF@jUGoma=sCmXHOb;$?^+BbU^#`F;BF@6_nSLYQz7V0wC6$?q7tRw} zOkb))A#Ol>i+YKL)`_D_Ib3J8^!QS11SG>N`G2QbMj3fo83D@hstjlarWe?D-TVbU zER4cx@TS)ccotTIGqSKEAbSnm=#33{Pf^igmWVMCsb}Y@;VBaZ%||zYv&)58TJIIm zAxKNeZHFkgfte1e+r%+rJrxKL?4cVzft)8(sTt!HIY@TGGjs8@%~^FO7}KWgyl3=D#RWpd_+h&v=hA z_EFLjkxN{P^O9WR!Il<}jDITjm!cxtn`ewv!x2!+6;vwV8}4vXazkMj8^L;eiM$Um z&)U*?#8?;)*#ra-mcn8P9^$F*VtFcVUh{1x6I|LMDns5FA^M2Qm9o6l?qc~&Ctv!< z%_>!Eu~qB98kEVNo(rT`X@+TVLh4j-Ajn%XWuHy!AT2zolO!?%0Y(lG9JOLRUDCqX zU2~u#5d|;oa}C2@h*S=M_A&qTt#wR^Q^eE-zI9#*f4Q9~c%5TiW~D8p9Zw3;G^q_d zmJP_j;hEfwNtg~iV4ZB91WJ*%pcn{Kijoatal&`{xt@#Rj3Zjw3iKr{XLhl&ffWnvpctM} zhE|9r?{yRc0W5gqS=v)J&%ki>3=Bulz;N{J9FE?B;piP0j^2Uc=-oLS^9F`v-oS9o z8yJpxJBOohU^w~)hNEv_IQj;LqtlgS?)6nu5y%RV*iuyy2)2OVo|JS@>6@3Mo3;|J zJ+zf@?HrdtRRFl2{H1HZ8sUlg8M!*Q^Zgs|vHBNJ`QSbv+8_R0CNh9B;Q^Eh4WLZF zBhF>|9dj--pizI-b<{r;`+3jG?QWy)VxLO7t2*;{pKP%?L_)U(5Er4gz2xpVxGg9! ztdNLf9!IwYXk=N>dJC{~9nVJ=*7rWXd;74Nx;d0PcnJXJn`pVu{@K*(?_sN{PlgA zbi$E%WNK?BeW)X`eOV@xMn`Q4>d364p(8J79eGTgw`4%5_{v=O2Q-O|AjP)HAkQGf zoV9I3_N2zCt?<#>ph0Ln^YBM;(9RiHM)*Q_V$B% zwQ=5Sn;PdG>Ert&fP=gG2kTKcGLB13Yx}gwbkg_T@DlBEM3xvX<@w>DPt%(CamtUl)XP zKf!^}8{}r@%Za_HPo=~1R5AXT{I5qm;J?P-vzQ05kq41OVy%4EHje0ZZ#p&FSmUIN zaiY=m`Ak%76E(dMt;1oAqZrypo~y~pTFg8TeE|{VO9=8o2svMay!H_R8C>90dJL)A z-we(?5yYH{y>fmUcHrLxy*hlnZ*NA|HisbC1CNdLY4hRneH3mOzeY~PIF0C5Bckpn z=F0leyJn}4jdGlJcMq2R(NE0aLt9k-h|3@RC_~zG#mmewo3>?0zb_`J*?_o9Q^~Xq z{rcAYo~QR(578i?Y$QincO+bJcUkZRk46VEX-#In@U+-G?&a$U;C7e(eFG<;mAc>| z<=<;OL`F{&G_#-XR5o-mP4SE)oy61g(1uJ8SwBG|a#+#BuRLrqvR9sfb2GsP_zD7i zz>!Ycxm^HGU$5pHLpcprpTJ(9ME9dIK}IT9Gr&113S2M0U>&ZP>jdBDSQ!*%MP)GB z?~f<{Tru-7_IDaJCv8?cpB_UGPMPNE3pV7oAJDGaA%NJSn+px|pvLXF;txX6_L6^Z z5~Vs^N?GvQhjoZTB&DpnJ{LR}fA?bSZxS_YQucfD|ITxR;jHRE4jZ-n6ECsXV*F19 z{EZ#pA5r+{?kH)i91z`*c-VWSQI1y9*)Io7-GxGFBg7Q?FXG>tM1jtVS9IexPs zXqLm(=899`h|hc2Ee^`bX`gYIv!ulLCylrK()=v^Z27a>28#c?&XT$Chxr|s$&R1A z*R5^E+)*pDR+6|cfGx$RM}=H_FswrJu`L=|G#~kjsL5zdsY_y_kU}{f20$C$NALa))a3<5PsNb8Ya4UGF*q9 z&Bhu+w5<+X1z=_HITWJBgw20%p|GrYJe=o2fosGePDyU`{dhdYSU!=}u^M0na^G@4 z-n-i0X^XVnNQTrm3hTB(%OuJji#5mo6|>+SO=9fI54SMu!_S5Ai99bg1<$OwV1*$1 z2I~gmoLd9I=gV^jEO+qxCw@6lexKKpgWucB3(+w%6{W@;@p?dh9PVs2Cq3agzJ1s| zD(ZHv1C5;cp5ng$y4sV{Dv75G(GhK-IXo~h`&bn;8>cW8i(v3bF%2vqp-4XmA34*@ zXRDmgQLZ>gxqzeG$QW-9e6aObwl2l&Mc^1AgCOCT zYY?OuKUwAS90bW$-WdcbmYPy2uNh_a7(7mvL6B4;&RC+1AqLEgKiHnjg4Rt`R3lfL z3h>_;yN$%df5|cb5#06ZZ`{msbDod2Q@5blk_pR74fDkP`Z9F02?z@|bt{?cLyXcv zP~g`6Fp9B|Yz`Mmo1Yg+$2sdv+h_$KQ;B0Cu629KDyVUtR-+cZYqnz)CTOph0;5mr z=y}F)c^RQW9MVhil)}Ll8SskoB&=O}l}W2SqT@%vY?uM9V9c`wy9-#Tj;xL5uPHFV zL{ zmb7I=2suxFM`Il3AfKTXd3iL9Ve^`kC3TVZVm$OAldQ1xZzE+)&$)QMABo-QY4;d1 zqQ?13FIuP2Z)vVKn)Ws?*rGibgaBN&d?28s7}coN^LV{Ei#J{1Vt&?zIlIu^=g)Vzx24d#Bi ztXoJecG9;rE%6zrs13E=Jo8)i)KmNi8TtWaMIk8v%8A=Ehrtwy6T%br9u#UlNPE+_ z^?bL@BCO_>{nK+Dw?OoqVy7Yy#MP7B9scA6TkrIunPG)L*a~5bdPe5S(bs&Zz=ztQ z#Gs7Sen+(5l`^E97_5>+q*OEmL&}6poe+#=axg5bX~`l;fIc?)dm80>Py|ZV)Bc`QRY>=d zeQL*8r{h_FA04gVJzWGItqh1iXORm8d+}szQ7y-d}XX1jfQVb%Ts=>yFZ8X@0tCd9~(bc=idu=?)-Sr z=Pu{p%LjCR?EAO<{Cl74`PUEl-#)E>{Ja0H&c7Gz9R82|+vWW0qU~_O|1vxNBc1r$ zq(^c3hnlQ!>Lsm26c*SPIW?`Y+ zfpOFP@;A0?^W;#4eWN)MFcK|su-GFW*pf*XrSFzO48<5%E!)7H|Wp)ecW>+M#Ff;fN*=u7>Cvi!d@IWHg#p-@#H!j;e-J%OVWw2-WH-cNFFd-a5!b zo3YUpRF&(^S9-Q&=+{wSZUHeVcoVn~(<7!~Zoz*T%xi|c6&(r{4#OA^{+=gP@;`c{ z%?EydBRH9^vGcQEN&5KTJ9E+p-(p6jfUbn3X1ddBFUf>4DyW*7xOw(AWyKPu zu%1%e(hem2@fR?`93Q*U9K98fcQwR?zx`jlL94MHVRj0iYqRiqb8El&oM6Z2pEi8X zbHm5Qbb&2m^aw^9Ki|R;jJ7iKutzY$tm-eeSn-LRF^wu%EVN=8@xqVy!%xLZElGD5 z8g$c2r&?Gq@BdDX!dc2cU6@w#ag1t|_>a_}h0XIn0)^153L%;7MaUC+D9GQ)N0$aC zdd^F7H`U#|<^V3#@j(8QN%{ENI8Z(wWuWqL2vGThH?bW|PnRVdl(3Q`fai18!3aU~ z5|$a_0dv}i8`wtJVbR|=r~u(*=7SGO6$GkHWq8(D zIp_=T1yT%{hvz9Z1*U&&f}AS`^+KP@5Hi&GP{_b`^HmT$P_MH`#-O*pIe1Gwv zm8H%w_=!)Znu^3Pcm@(9vzsf4B5tYxia3%eVi%x@idHncfyZ$xi~wyXb!O@JxVueM z5kWAu8AIbG%at-ILg}$BTa@KVo%yt=D;D4Z#Dh~@DsH~n1vnfre8@&V|N<^FTD(oa8rdH?$90rh9or33BH*UpkT`}1@6 zF>>Aq^glkbzvEPYU%YI9`uLSs52XD!ouz)z_&tBgK*#S1XUW|6AMSJFfOkdz^N~G% zvo0Or__Z_+r2Tehsh#&9as6xFAJKortS)bEd8}Etb&FKPTct(7Z{q4*+8%vdq&nhm zT};Z-WJ6osTUmuY<6IV4A@$3xBos%<#58L5+_^=)xi)4^1TiX9o6@DM)35sYJ0 z>DfCW38Op)(=7_WuY0#-Z2G)@`{&8(-(g#JqJOb%Zv8uUo1=f1e45!o|E}%J>R;uj z8MpqGWcN>W?X=TDI9m?KEAxf7?Is$~mA5=_$O#$J&qBz2T_Cx9S16wvxm_6%m7yPH zv-#}}Dv512Q2XNVxenfN_iT}7N2w`-ls#toR$GD)O4^ZxH^)=SibjO_-9uy$=}1G^ zAPPXp%A>STP;MP)ebcxE%oEn*PPn_3L*5yL1kv(shR$s78Pc8v^8@A?tPsr2#DwPQN1ztZ!tbY?kTDdnKkDH6Z*#Iq-Yxi`@8y%+AlV@eA!d zes-%1y8N8EO^u!()m@gj=oOc*@o;@0ddd! zl5AFIemW`7kUnieU?8Mc$9%)JKBV`gxDVxO62absDw)P7+d-6&4l5?DHLrXa-_9>K za>QE!zdXZb*Ufr?f#k?_v{c$bL@D2fQko2;9xjen^jmDRDnKeV9@P?W!pjbHj7lT?flBZGslB!bIv`d7OH@b1)qmi zP+vyg;L#u)f^x?kzQ~(^U$YBHvo{Lw8)RM?e#t!OQ3`y^i7_sE$ZHHxpN>4Wxuw}R zJPBg_v8*I(PQkEOk!qS3Lqzf$(QehC+5Vp0eYni&^a8;?!bqNBU{v|R_6>AF681Np zi3bv7FTt06o|etilH;67KK(IY;;MBBdc2Ora4G_}%M!)1E38H(o3zei- z>r{gFRE5*mn(R4W^7Bmfp)~Y}OkVlc9qT1w$!$O9xgkk8S!KZ5A z>2?Rd)HScU4mlNY)e1?@I3cxaLTY&eYMI40th{~e9z%_BqlOv>iGMZX-^t=%z4+%M zf0DfNk@qK*SGJ$)F0WW#3zaNVY<5OTKuf>@fwyE$5CM*PZI-xF+)Uz%lnMMjRRb4S z>dca67FY76xFQ*_jIb1>R4Iy_L88dm4iZJU6p11mNEBI9>=Z=`E*GN6LxyTW2p*Lk zq^!sLx_T*f)S=XYsHLV1o!5DJ0SWnasa@Kpk`}Qpb+WH*D%}W(-+{KN96l;_asty8RU z-|tYqC|84RjCYqW8g?LGG=G>aUzG2>d?C&L#Qs~3`|m$_|82niH_G>u+52y_MC`vy zX#cHH`)>sB8pQ2TF1{{LD`!8U2!M?O+ zAgWpQ(Da!&f~nY08KetSkHYP!i@$mQt%tLp+JE_r_h0_v{g=Mn_uq+4m%EO_)#Pw5 zB=;HnVDFPP*f;DdE8kFT0ZhLJzhMs&ENhfEd*1BZ|AzzK{(qBm`>#>EO$~4V!0l_r zcU7+KKPvh=kGB7c0+;1dxKrEz!8x{njoJnX>1(53>Xr1kXNL=sR(9w_`g=nXDQ^E2 z1-Si(dHb)z?Y|;`+kY6g|0>@8tFpKMD%$?5VEeCC+kd?|<7&12ACeww?~Y)W-t=CQ zK%7zSIiul-7J?&M9LmUeHE6U1{YOCvHk?Q9$#uqJd&QStE;P*U z{rk69y!T~kulV_)J7=$W&db^Mio5)LK=z7bUbfjQUVe2y_KH7L-ol$O{~f2P$*3n^ z9_{SQNGn%s(#jdr(Ubw0ud;|Jk_HjUJ!{yyPaez>&)2V1 z)OsN*BvI>wd^N&apV@oYun1A3ypb$aL6GJ`%bBml*?Q~_i_PzqL8Sy~q!xO0i+{VS z6t?=NQbN%yYNdB42?8>PlYL#poctA^=-mh++v|+-A(Jp~%P3>+bNLlW9q-z8c>6HI z{AypD)={wST|80+UT1q$kGP`NTbu%W zim)y1pJyxUGpbC;?+FVmL9KWt}8*yO#e>o(0ot+PwfF6_f1)c?LUt{0Q=8bmi?!r*@4>2RorK{ zmvdvKr8X6A0#dUx$(NS_v(?qR(GaYd`P331%;G-Jv8ie3CJ3oIUR9^mQKwYZ@u@md zN1Z78s7j*$bQZ}+8ci=s2Pg;pW<~G=E8jHK<9UmMd-E=`qo^#yYiE6mSGUp}H%|Ul z@O&F*mL(Pjc-#qDlBtYFO2EjOfsCL|4E$yz9`c}!Aq}gC=V-)BS5C-SG)^{d+!h3f zK9a&HZM}Fhq{xf_xxUe{f`z&CD_Q`R5`9x7F!$7f7Bu3m?@rfH4n}=T{GLQCk;J{? zk9BaFn~NUVX$#>Bvsl8EFf848@p7Im=rrW}dt|4KhvMN%eo5uSu}#r@{YR1hBWV2q zCbv7&CyX+h$n|RDR1m+;RvTklIWGL&ou~QYEjdsx*AL<7dVfLXF+eNYucegD&2aQe1l_ z-{s^)QNS|oBIyoW)l^o5GLQNpHe9IdOzeScgQN@Il;RvDyPR(Sy}lD zoBK7ml2h0CJ&k5izG8xsMRj*dl&r#syVwVM-ns0UD8GEnXCH6rDsmlBiQ6uljb3^o z4zv@|WbiLtOeh8Ko(ak-W(jRuSYu0y(Gy+#q@W+FgGQ|bw}+G4{3GqzjKy$13wBF` zv0=)Qm%d!RTUm>-EpODkw@tn6Iwv1>vlPn|FO8K@-POCDU(%Jb079ANmy@?xr8uW2 zY&X&zxz1FqQ5*%3;I3RoG1WwKidhqE^1z9%uAqMT-5E7csG;>kX-3c-xdWKq^ss=L z)2C9Ugh-O2G8P4vEmQAK^slFTChg8yP;K2XQ1?We z$30_Ume`h2N@fLW(aj_k0t2fuiWhJGaR1Jfy|0C$ujCDUpP780Ie>!HU-J7qZjSAp1h}lDw56gI$>};mXG@C#j$G{yWV2tq)U$_NHyn@8Yu+@&Bd<7*RSt+VIXOhJGk3Sp z%O#?*I7Ka}-(@*h&~CLlriHBXZ5Wv0{ZK_RgPCG0kzWH5aJ&m-e%huX5U%bps*3i# zxH(hufxkP{lGyf8E~#j-(amZgug7Znh*kGpj{SLG%@6uO^WBDgn{y0`sFEivRiBhB zV88cwPesbg9QSu0CTB6eO27tjbj)AhO9=!cX2c={;p9DuSW4#Lgp8UJ0eVsqts%I1 zxMf3iqi0BMYRrSa#eKVy$@5xXO;{3vsdv~mC|>T#rx|7>e$eHHq%$Z7l)jcsSxy)s z^X#u%Q)7WGu0NE=3vmp#YDf={z(7`vs88q_IHwD*BOA2U;vIc4O=`Zz`2{$8-aL$h|U8N6WY|lB*R4Mr%t(sR-n#-`Sa2p7{vR&#INuv?_!Ks;$o|jp zDQsSlp@)v*Q`nsA+CxXj|1!h)?cqn*ytLDS;8WPVH?4mA_!PF|)AODF*YPQAJ~Hz^ z!>4d&eERK$`^6{093GLC37K5?Q`<+`L$Ay|>_mcarn$ZYMyAa8C2W~9%D_d&p@$5G8PRF+jSB#UP)_y$ zpd#vN?{Cc*&f3jC8vBoN|55Xiebj#p`UiCD1HH%8l#k^p2V-tzBpqL>Tv7>XM(Pj- zp`tnOmQV5=38-d_-I+@K&5(uUMVi7M@%-#;gV8$Z)LWNn4j;(ZN#KNkX&{V zY$q6M)+Ph!DY4Km3+SEI{pZ${{-*LK&xM)Sa>%VHb-S0n_Pxn|7GRd~K!go^} z-J>+S5C56x2u;7Nl%i9K<nkD#;*7O8{ z&l3c72*FYf0&gh>#~%Qsy{0^F&k|T@$j4&bo|w-X-yw-3%cQO>Toh|kYHr_YhXv}^Ru&v!or-KM-82y2rmg4yjpavbo;OV9k%~E?7x!# z;D0%PQER{U^ZPs-?6CcVb7kD>OZ$hPmA%Oy`@7gbq}h-B>|#GMyySrFN9G;gp?FfE z4C^6|fA$j}<%xCn71PDll)V~$$Hi~$Or*0;;EC9{D@D}w4@A92{XNezQby|a=TS&a zQE(Q8c9#4y{eH9!0;D1(wMy3+Xq+Hu$4J&-Mb}-Q1^M$jS%!02C|n9At7VlDubS^e zaKBR!V4#Om4wb6!saQM&D^>W&);uFw0@{7^Jf`v?s}4)2%Q4uq+j&N|WNi7F-3(v> z!59H1oi|bxn2V9|*x=W8KyDA-02prQ82Hn_u!yoV&fWm#@&g4-6q_{3<&BVES8yD8 z|2RW*JRu`TBbf}~C_D?`Vz2fuhKs#AwOOnOMN$WdZ#sh@My7R?==f^r_*v1hp2;?E z4wN24dNr=xg&7^3moq-3{k4*NX5Nl%Ym z8*X7uOv@spt8WO!cKPS^!@uZQ1m2g&0v^a4p~;(q`mRN$IB(-=0PGsqLg!CH%+BX#2#m6o6@2e5+&*-Jp#d9Xuf1eUii!15P3jm@V! z6omU7->vJ5Q1!Vs%c*R1?+?-#NE5k;*}E4pc@Z>%3!)Jm+xm2N>C@`Zp+l$H6R~6~ zHfWpp&{7w|%AC7@dx55((@kLL%MCxPlrpe(!Pi?;W5mTHRe+a5P7P^@Jbga{`OW@n zoUlHQ-wfG@S@)V6hL`1Wxh%04|Bvx=Lwk0}-q(shI;5y0{MQ7Re3MW1J|O<9<jIBT{CuVo*-vd{^yZTv$!JpQ~)<2&z@Ggy-f)JokRe)F`&GMKdGtw8t6XzPn~ zfIv%-{)^>((`#`?d`uUWlL)7Xj$3o#*0=K52I)pGQhC>~Ztm;J_9ba193gT7DStbU zMV7NZ@&etzSQH+k3a3!vR8jbww`5@{6&@!FA5?{hQ{f;{_=GC#L51MD@mE#Yx)ln^ zfTw1Y0ip(axPYyBf&wEIKBKC?wN=6OxGG$?Rl#+?D*WA61=o$L@P@4lt|h8){#FIo zWvcK(f~yC^Wk|S^v*8vw%7YwR>@DEf8i-?qupeeyh_+s>aX8kTT?V~MT3#&;wIDwK zQ8->Xn7Y9{L*nN;F!eS+JYr{>q^_O)tjrfLwwtmHn+Zon8XzCNnF)|r-n0So{F@Sx zNeD=i)`8h^cS9N-fA{NprX;v*(p?a8uR0;?9UIfKQd3BV)p>{ zCNBYyYmp&4#Qb?T(T`ddkkxXi(ik01%Ly*bGFN4%t{xz4rBn+idEy=^vt<_GEWnCd z;^T8(*Z6qE3UeUIX2XV*|MmR!PKRvP_iLy8_Q&~dIM4xo(N%4k#r6BAFBiQg>B}(D zPqL>hKU0+-yx;y^okV%5PC#7@*bh1QLwL&%RHiX((EKRtPRLN^)2lE8g*`r@(|_54 zY7{mHI4g;~)I?F7pNRSZ9Ooi!oFj0YWRM0S{~z(546Fx{5=iDq>=1LMi2n|Lnuptn znZ(a22fN@WOJ2Tv#fhKu4wLwK@Q8Nt6F{?4A5r>A8KmOUI-<6<6bp#S8#uxvRL#s37>7%R#l28j3{Fu1-`^nyPccjX0^#jRgC zfn^KH^#{^%R-?484q_NqdjrcLWM<+ltMak+(*Ytc0}^|M^d^<&4FUU#;`FL%dP!W- zh1&G`_ZQP!&w?uncV+St9%V@VOndLcRPTCyeW*D!)ataM)&#{+r{GtyB{DGk>RqW6 zUUzE5E7eNdM*d4T8o7qa3=Xz{20QH)Cz&}B2K(IL!IaJ`Vup87RAaS;Y7&Vu3hom1 z*0a-8xnsxyFk~|gX`~JLp?KCM=Q_vR@Qb)`PNx2L=GnIf`@5a`+w!t=y{>`&4zn%R zykGdrHy@tI#^cFDso)MMXfEu&GsU4vC;o_vQMUjr>K5aCkEa__t$+1LN^cE6=bg7F z#bSfa=7etYZfh?5)u|k?-VCcliwqr+1zDq1RV7+$)PuxwqH+=X*rYqh?$`COa(mxpz z`!3#U&msnf!Oct!+yyWI-H0Eh^yPGAKN2#>{-{JR(w=@@YijJ1q8CUC#v!H~Bb~@v zngdQ(`YD5xc)F0%)|+gm{_-v-k8@XA2>)l<8)aZ((9Q)-mnsR>V&+HL1_!#pkAsQ5 zZep8aAUl?Ik`~uuM?V+&YRAT(XdA!EKK>vT#YHNBU5B6RHhc%iw-3=d(-495A>8aa z;@NAusn9+#CLlc0DT;B3nCRJhht1!$J#cEtfk*~-s5Y1MWraD!)fq9#>-L7xp$b}1 zMalaV!(r_9XfY@7<}Nd{|JiE8098P$zj-JfOmY62m7Vvhzot^s!!2a~_%YZ3B(qt} z2oDVb*4-6$uq?0bc=oD11y7C52nMIV{wTh=mjFeqI@gBE)(FWyA3bD{%M}M^p+maUKkPfwB|W9Xbm&m-}KWLBXG>8ZS=Pr8`q@PL zV;M?u>PT6=x*{ISx58^-K0`+PK6BKP=Oo^4b4#gh`@f9w7oZ1c@P)@`q@_N}oIeSz zO`LD{dz?KSc>bk5)Bckc^NA78xF7H0vk z#Mg!4m#UDjQ=pSptl9cwF0{7U)d%-%q%v{!r{Bkf@rb6G%I zluu|;M*I~PWr4OR4;SE~RCtNhRGW9aFZSde6P9;j%}$5NY@sqB@X_bbFb2J7^v7uaQk=B%Aa-nx2mou{mG9-C`>poC&5RbR)|^QrprSp5pEdR321(&aS! zJ>)M#?|LyP7KRE}31jiv9QsGEvJEoWN9{-oDGuN9^needc(6hkj2!#}u7pfRehv$N z`QQoCeQ_B5y~Yrz|JWlaw-4Cf63KV1+Bn91dJJ9oju#IPupl1G!a7J9RIu9+b$h1P zZ77!7jCHBo5WY|9df3EcA$iQTL9=5HDP<=9fw3T8y>7mH&Oy90h|=Y4hb-p$upvo9 z>p2?hwP1{VNL8CV&z0H%+_&*H^fN|#iG1yShPoOrNj}tGD!Cdn`%9ThZmg2l)|i1W zwcs0(#)wBSZZGg0D>;qW7;BWy$p__oG>zY7ecpF7+tcTfWTmme*HgxC_(0tpsIMdb z0%ZL%q7Z+Na{QL&yxHPVny+k2r5n)Vg|e8|e`aRV9~fPjRLm(n+*u3Y)$jF-Tfh%7OJ101bcdpJ@RH?jd=a6DDc>h43>`6O1-KyqLpT;(o$90e!k}#@kp9&s7F%O zCXN;?{;gmhk^C~W#mSrR!RQ`O9t*=Yn1hd|1S(=#AQiKiV@0YSB#W_pTq`gHPhN_@ zu~Gq5s?1u{Mf`#=`$=>I$)ZNaBB=BO;KgVt_SPQ)T^e=W0ZGIOs=!`BjEhl~E9lvS zU$Fl_)814Vft5G#JF0@SF_Lwg`L~}0=K-&a!GLymfUiXbMQ_Y{Cl-GV!!en@pE0&| zh3Gd{Hxd2<(mq%`oMemUirh$S=2Ymtytwwn=BfGl72v3G;@~Z?$YCV>?=Yp>Y|ezJ zoQ0T*-q3)l_(`%sfMxKMdTp%N9zvvjel()q9YpvW8}ezPb3D?NbrOFlNRfgTNn9{~ zVm{;^IoMV*I#z(|`z9abU_4qx2?&BsWknbX68iKlx>ydDiyL+iFEw3rteDX0K@dZ_ z;Elotb!hCm*_^#kIdh-QBjz`5;LtZ^`N=Z3*5z>~#n?r}B9TQ_7hc&#yYwLay;vl$ z-X|Z`KAhc8^nLV{`LtbNBk)m9bYl9R(q6J)hxY}E$S*YjnyS+d%hwra20d{r z;#CrcuJFOp@s$8El3zUXi>tSSP2L07zobFF-%vdTVbgu}|HuZ@;lOE!^5C zh7_85UIj!KWXrl{KbdZ4&yih*J4n!-LmqXqaU1+nW%KiNWS6ft9GW{7xqi4*%WnB& z9t$AeoxX>~yQvNQCfq__YR=h{N)D1Oo>MJWix%qj$l-)a&)LH;%$>edsg0_%_o%CU zu!v8cvWE|`%^KiWw8U7#g4oNbM?r=BSBwK&d3=%7jeuH>+w#dk6H9?f^d=8MdwRe= z>ekO3nLoM4>Zx9y`P^a|a2clH$JBg|4bK2Ss-^Hk07j-F&LPIDfM=;RJL?ffs*n!6 zY)IW*VA-Gf#8hg}scL>T;k$tElN;qb0UrrtHrAc0-Z4G-uW5^O>8}kzRD{XLM^FBX zzYnAD!)(35=@@2{me4zpepIt$2$!!a5zP=r;@?`IZ(42Z1ArW<;(Jw-R-om3q>DM} zy6Lz>CGkc__KRjC1pwN{Bza@PM&O!18q%tRrY*Gy5P1vfF5+2~e7Uf^S}(8vl`5~Y zUTI;P6yB|PaB!v-gI_o?z6B-Dutb%UvNlhOZ>(VXMG)p$5#L+^U6hmfb8f2^&MKRt z*brICu_kA=t)PLbHK)2tduXHf)(cpnk<1DyVM+suJL&!!P4U6z!5V%Es#TJnX?t$P zg{~oT_{2UHZ0Jc9Y={MImbf^|L$R$rucNb8%rRaAav>@}IKRRGR6x1@~Zu+4>sKayt&%V(XvQx3ja zJqX0P0&a?IFHm^2Ymj_r+^=c@2GA4ACO|?kBXte<0o#Tb|kZ6j;8Q4u7z5n@2PoLe0cU#g!S5-YN?DXSDFMu#F*)p>o)TS(Mp>I`zYjkzfL| zA(6d6ZR0i{nEQzjLr8SV7y0wVUk)+qyIF;gc+$a_lExr;VxrDRM2Jzo^=^ zyevcgX^X3YMUCT!=p26!XW_psUQp1N&Jp1O3bT(9oKkD~@_<}xFA)!@0JmyOS=rEC#L*7Ye*CwqU}m$E6M26B~j(!~u{fL+7+ zOdc&X8jM6_GszuN_=3Qghf%sq`x86?8ekYgN`XH-C+!-Y;o49>+yTDZN)+u^dt#z&yVaSXaJSnmins4l}3%XZymH zZI;=YOmunyTAyWroNV8#`E4XX2w5N;%4TcKVt;jtZEUumnS$nuHs;1vzs~stmHE{s z+IgnA<@|3Y|BGnL$^HWCw+EW&PXXE%@s9`X!pg%`x1uK=>g=c4g{3-igH_dAe>qg8 z7#);R&~wEQ?I2JNLYerhhZ@tAW6BZui*&JS&c2mnUREb_#jh*($zKJ~f(T9ajU4T> z+Ggv*xqgeEPU*Rj1IvzNJ!g$|;V64Yl4xE2HpwJw0*x=}C$`GG^6eCxvBBPoAz=2S z{dL~tT3A<)zIrH&t(98PhGa-Bmh($D`)}Z2J~E^wj`*>`kVZh_$6EdogzP_?4T`;5 zEkLXwvD4IjXy_UxrL;1F@Lp%pE~ECX+NIY=DHv5{zE;L|4_we_Ga$; z*VASH-u=>j|1!lt_$C8A3F61ebxl+r4$!l-o`4by@YlHh(kOgZ%9G585oE>Ux*4oSH$3nzdxu z6;Ads$BRbtJI5(HmA}&C0FAF9Str=<-VKtxdc*SE{&{ChDp^#n@ix#yKKBrJ=)Bza zE3ys*>*hVNowhhz!0S#juRURpuEUITrZF<);sEMT#mH)%s%U8*q9k{2in$7s9r{;Z zVD3dF?&+B$rQpDLnJoSZS`KePP-a!0u!n6?2gV#}c(na?wQpL1OjbZfEhp6~nn`S? zH7~yl*8Ea!&98200h#F}FWo&d*m;!_rbOwTPR2_4^2hQGf#uuz0av92Wl_t7iUTo4 z`SH|bWo*t){Dqm4n8lQ6W=`_uAeN|pAMqx!d6QXtu6=CSr>jk2<<>mu#eB?_^@l2( z%4+p>?Mkm^)~aNJ0YBpMelPxB z2Z=#U?*zMS8yJl(jAIeL!@-ZLIq#0n;4ia>J8m4%a-wEf*r9f%6yKsz`dMJy&NI;Uy22lX$#2?m%rWWY8&Pj z&v5CtMTm6SKYWoV(}5f022mr47BTkC*>ieyGbGmqh#m&>U|hSAosf?XiGn7|4lX{! zFHe10ta({+jfa@44a{6YOJugtc~kON%vv>P`?P_QmvdKM>MD|{?L+2OwAU5eKBs2&cKro!Vqe=729>z+Pw_z`eYM0L0nCsJj*Dr zs=y`y(Of)wPih^$9%jEP7AbeN{&D!R)<+M26k}e1M-vo77ohmz0s+ni-azx?oJ0D8 z__AeC6;HmX-(Vf(21W^F?f*yO(<7XFq#@8a%SZQzy~4vX{Sfm)e)F5hP&3qBvc@wd zi@b#x{NEq92T>H)Xe_PK>$f>i;C@WLBdeyw3W;q#5UUjwm{x#*kob_Kc_ zX4h-?q$-+anfA`kH;rG=Y+<17`JGSk7MiRq8X$lvPj**yPCiM+2)DrqUBn25smtX{V2OMQ#3`H(-Y%6DaeqZ~_P4j@F)5B$=JOw7O&=7*CRu^K zi4_ATAz~4-ieec~cbHy`V?qRFGw@hWMjL0Knm04c1?5gE%9A98{n#`z+MX$n;F%Vh#3HqkX^7`Ag! zgwVd&Vs8k0{Dif~AP=Fm1c%w1%C84u)(823O9$Wq0f2>Q!lp;(&Sf63Iu`@<@mjEqrVJ;Uw9|bXtv&IE(8?d!Fw-4r#hU>`TUnJu*jLU3K6?ewl8STG_!O=0bol-A?g6}i ztoRh)I+}^=bM%ec``tyqqI<H}E2*deJ}lc__OQ~!k^NX%=i=SW8u$6npFJ_PWt*ABleF!T~16L0REi(gA4wALU7%Fy%T@F`_ct}%;D|g&)3x1 zuda9E&+0Fo_%k_{a^TO99}g6NY9zX_Zo^|&dNq(;UpZVt6Oaoi{B|W-5H*;Si}s}S z7co8|_YwTBH9Jvd+ZXtRG(2_;sWf{X=aJUP@Uvz|i)1j!gOdFVnw`6T19kYg(Vi|E zR^?~Tq8FKWxgzl~)7T>wd2tytH^^cSDeD)z_1aN;Q##6ERS<;q=eiWhmMZ5B(<>(~ zPoWR)FQt@PDrP>rJ0(>@?go+jk$ zTf(jwX$)J?`~Bb7bQNFHM(ior&ilgBv3VAbYdt#*#^53uYG0oiOqh2LJMxgg-q zeejt$7nh>ywO)g*mc}^K8lzN=QOaYKB5cF$!nJDifnBOiRJDn6n`lO{8z@65X6Ept z?YUGjD8#okfuG~rV+_u>S@0R49=NB1l}@z4V0O`-H`Sg{)zB;XwMs#c`rx#Ka>k>| zRwnG>cr-}13@5wsT2GC%crAP6syo#ed-7KcJnR_^Wk$#zinJbz%rmtf^vM?3d+=Df zhDt_ehOoZ&PJGQ( zYtW#7GR1-w=VSbco%+$^miPNQv~*yP;yLqA1|bAw^&AUR09wA3nFr@#}>zyd+y zQ2TtrX>A-CYG$+mC07)=sfeUf*_~+4QJ!;@94`-6O-Mp!nM)~af>^A{?~2(O`2|HP z;bt=yl`R$j2s*WR@iDxDN9hyed#>&fpmCV_EY6NSC2&PRI#rlD2Qt27h2{f>E0+8x zu2?xGF9sXJup~6-#y1j43>GBNv`cQo1$9SQxARBr<90ZS%9!VWHi7l3?+q z!EzqK0?m`l7{v}{6pKJ88rl>?VsCwK^J7PX0Kmcqk4uUGeZ@#1NjX4c^h+KTBF7Z$ z7ET6afkq*7L>?d8b^cwL@+>0Kt_s_q!g;WefSrlE3@LYwf~^iB3uH{2SYm}mw1fHY4nE+wb+ z5tGO~1YnbxjmD}NOmzZ4*og}2px|hjM5*#PuQhMpm(p3GI+h^v2}Mal@-SG4J7^&W zET@GC%Y}I1?=3Vx8)IOz5+N~5_KG)^m>Ib=^zA`)kYM3J49S`klt?1&?5C`|RxKm5 z#?>(7^ZycDGgvQIaP9S#SPLlji-zS%cFLl0ad0?epI-lwJMEzZNDgo*)scM>!w@p( z6Qv0ni~A7w2Kpl%iqQg!P%$~|Cx z-1PH~aJR2Ie4Ou?-THDj7UlaL<>LYzdKVWxQ_;bP0<#SNNkE^l== z*ZMc~`#Fc|`uzj-@9xvzZXIChA5r>t=#X^%V+l#9hnLd6Iyi$uvORnAN|~)zT?kc1 zeQ%;pKFIus#(hoXs+L@o98I4Ro#m&|o_WYsE+P`JVJTct7tjTT;TE6!5V^_|i1s$I zA77YU)#4msg-id(dFi^Fl8fxyuUh7Bi<5?MgLn z=qx${bp2%q+=I$OkM`&i*(J%;7mwN{xk4+KH_~+74nLw!U?shRtHdiwJ$%p{wUF;i z`*3cLz6|+yB)WwpF{zRRcrd+E*c({(C=u3hR$$pWU;_SX5T4bTn`Z1xfnURX`ngv8 z9J^XTP`=DSH71Tk&{ZN1y{0wc^k&9uwF4MMa z-+PF`rz;+#Mjll1e5;{r-8#R?LG%yGQC_ERe^>F?6{2Ec0MR#SijOf?qw3-KC=Wgw5X_wJSyG zW7s{T$~-@3XS$1x1uuU3_vsJ$vIL%sf_e=n+m9yu`9hfN)!Jlh-o?otLu}P1Bg$-M zl?uw)2RlQ9V%>DP%4~gpuieGu0N?{=E?QsAPsj~H?!HPr+H1_mHttPH@P`rjCm{HZ z8u$;tgW$WSZ+F6DMN3lbkJ!|ymVP!0ui21MS7s{`2NnM&BVb}bfR7=D>lW75lI|u| z8A6vpDUzvB9og%dORv~vdpaBhLqgBQUnd@Bb|7ZnKAx=2UEu%)Q8P?4 z9;B!yV~gNy{+e*RTxb+A>UeTkxH@-7C~!7HYnD<*t+^BffBe zjAU}xJhcRF?`>PaG>-B!R|*XUDk_#HjG3&-^^AVQK&eVfHJuzu<37sdN$0-?5DRlt z{w|c@Py1Z(!&YpDkomATBlo5GyP7lR+9)5sY?};_ujCs>Pz#%O@o>bwQkMVZlR%^8 zlQuRUz)~4Gnd znoW#1WKW9zDoEL5{4zkTLo|RMBY-YW1Lzn9(4&{{2Ov8<*4hy>X5JISwHw1%Up~F9 z4*wJvsf0^qXpDR+?nX`DJzoi+v@s1nH`f;&Y(Uwb6j6<^Ik60( z98BnsMa3yn)Z14$Y%Vw+p*l)4#k(pEs$X6w0uwghgi+dq&vAi}Nf?^dfhs_7zkaXA zjbg+RI0Gp148`8rrXB_qoCQA~NeAR|?ge7L_&w^*#F#JPc|Ltxtt-VP6(I^GkgdBE z*t%+#-c|8-hRmWtgmoeF-hHXG<2srJ0*CMw={Kt`+r?4M=*8rx#$|)Z5TS|<;ez{( zCmo{=44T&*hZ}g1a(4-uGs(TQJN+6T-IJnMdVqqzVd)M3Y;!u09$>t(?+Vl6)kuf% zrD6quEYD*ejbFMOB< zt!8^6YWtB?|F@&s(xOu0i4IxyUvG1F1b4$X6mqB7Q8@~i7YQr#)jGt$FrL6WPadzE6_@@&DcH|eol@HQy-aez1GUkqz%TDPdd59DJg2x=Z5o`2zX@18j zN+-!@D3U>*#!(AGv6)c{&k@V=5Hk9T<1V2h-1n1x$8CL-Gs=p2bJzmFwI;v9>UQCY z8zlzXmerxmCYa>nsNV^t3yiMuRUucL=oHx}IzcsTAB@DE1l181Ap1y+K=U9QKvbUe zqbD73&w@XD%20CTXejYE|H5HB2>U_2F@Ce@{8kDA-(3~m_8jGYk@pp(kiz7$-i5fsfe{y_sg9H^tK3(M~!Q>!RdDNk9vXHdOUtA8i*LuU! z5{!PiC1ZHBH2IZ^g!Uq*-;kl6aMa)?!y%)}*uK%xK9?@BkI&wdYA*y-#{M2~^mh`` zSWC7a)UBv zVpBoW+~t`W@*`XDM$L5|8@$rY=WW}f*%$h@p1jcF+dHQC%83;rWK!d2Z4hv0e~T=8 zl>zuz(UBa-3C%Z~pB2)ZDQNLC5xRqp!8%qXL53=Z!H~ow zjLTJ4((!T%)e~IdCzgSf-C-#?V~Msn8O3}u7-9d`iS(~%Y#g@quVGsG;Yu40`@SC7 z)%QN<&@^J1y#qGNSXk&B6JFut_%duGlbvSul;~$A({uq=_$D8S8$m=rd_7r zKHHK?^cDLgtqc}n{Z&-|A*}zXR{t5O|Gh_9fOIAP7>+`49qg<*Xh>U{M&9gzMjp%v zERI5R_UTL@-NSMs1gTjA=?DPnY#HMNPr`QAu;;@@N`+}a|#uS?`cAB5u`-}i(i)?S#>_6!v3kx`Bu(dW!h!G}$raJX% zZ04(22ADu_*!B9!F3OaYrYu^?e7cq~Q3QGIihM0sZ%k#{-4yj@8EO;vYauYWLlywW z5p8LVnM*!XXG+zB2e)O;SCyuE!CcFCHh$hY>xGTGvRH*%BZ%KRpa2js5swpipaj-z z2I$}0ChZ1(*&_ZbMIN(WnbUEMXF$_Bk2G9TNL0;8L_rCwM|&3JGscUB4lMsWA2W5U z50hQcbC?5RlU}~I8yhtx`&z|YcqzqRFR=uoVYf8y;8hs01g(kdeIPD z0g?6U*vu}3jat;nOT7|W+U}TNYbm$_X;o4!?}Pzj{kp=#V5SPR6+hL_B&%9bfkj8s z0vhr2xKl$5(qVy3-KUjt+1R} zU`=lEfQ=m6EniJ7pQn}&&*8+CWLE8W9>fFZ-wxU5`7g$|?_)UaA7oqX3EA)1(r+8^ z*kyNnL|NWIny6Rc!5V93A56G)SUpWH7z7jM69#TL-uT24A!E? z%t85%rCA-b zn5vkJDpbkV?widY#(Onm0W`lxxhzsfd68KBn8SZ*>^&XzU%J^@Gvj%49`ZLJ|zBU*1ZcjXw$pS6nap3rE|qFnk3ZzPgIh3KgTF9taa@fhpE zpOQl{M-B@ULL|}zdBm$Z2B*(~pAc?7?jqP!^K)Ig)2BF>4ihI~`gwiT@|>&Yq7PxX zc}bQLF{?Hf0@1IwGPbCIFd$|>n)7(V{fu`}HQr17H_S&+fa--uqk<(SO$Yn6s_b7= zDHH=@O+!U1s5T%XP<66%@-MV??oSxEo3k%6n+tlW4sQpw%}ii6$85Z^lfckkWWf_* z_6aN9w9hRknp-}UzGy~G{Z~WNW4Qks=pPutJ%-|o_;olFd15v+x!no~0Y7(9ASBd* zRd0&o^$sWkv93s*8%g{RZ9|P19r#t^j<~yzM0>$;A|iH^j~GHs`?!y*ibLpGJ4XDq z@Z>l-odj{qg1BWt9zVEnD5EBKF$eurH{@H-F3FBX1v~aUKO!s+zT92DRb#zCi2cQQ zo?c}Grsp#F+a6pV^zRr=rT~lbkeYJHf1LJ%2(wuHb>C5AZnhzhgO#ACok4uT`*P!l^f{R^bl$)mYo}S)kaPBgJ1$n40XRo z&42CIha!83uV343du5U%911Jh`~h+tb5HgRAFuCbe`(+ba!6)=<#cCkg6zjyQpq`u zkPx8mTxxjyZr&E26Po!W3bDz5Tw3`L)nn;J%_DYOC{)QJDq^%N0fk7LSYWr4|4K|S z-FW(qjPlc1T#sj59@lIe=R0gsP@`PVrhLllC_=P(EYYBIBr z7;LkLg?Y^dAf^~q!<)VGT>nYcyw3*fmU|n@mYr2QlaDsIR!n#z0b?zS!s{ZXl z)oeHvp9hXFziG33h_t(7d3*I8J8E+~_>!J~G0NgF%1+YT+_Gu6{Zbb+a-|_9VNn%@ zO&;c;35LfQf?_=IOc-snO4_jo&F%8*;Yz=&Mw4B#!2(MhC%sp*u?tOO?)IX z>tC6bsPnVv6gTut+R&CWVM9;QHuS!SaYLibUii&zn`MN=S;te7(@It~R8r0trBGC6 zY-#5sLwr;QEWpB>ZtPkn7MLY6RmLKa6jhZN`6wekSwZ!iatTZ0Mn9}SvZv*7X-8w8 zSm41-fjJT=dnokeuW_)2YzFj)8hkI*SRL?4F~50SkrvVv5$`wvEma4}TdLaX`&cn( zr*`)kO8VYZ=1b-)xif&*6!WTiy^rzk zdadecyt_PS&CKr?_dfrvczjph&{6mI9P@`&kXcRs=CxB%8gP~P$Gm%?3Iks$S|?7(0&1bEUPx6hg{qfZRZF01k-h2+syZ90 zPOz%(5m#E`JFf*X;Ie9lwt9KJq0QVys?96VW?lPjHk~HgjI-L@1Z|eL-{!B-CdX=X zF0`5A1shEgEupolY%U1G^>4SLdE=qhJkg>r?^fukdV%Vx0D9`|7+or%VtQd?#8+Q$ zOUVSf1dr4yd0D%Vo3oxu-t_>I#mWL}3i8RZd(!L;=r<9Y$XdTJ$VUXq1jolWS2SmP z1Z+xFR%@=`*OE%)dF-GyM)Cv8-!t@dC{4hIECos|ispHf+2+a*DHB`OE9Jgg6)726 z&M2~+7)sV3XkKAB@~YsJHg5CVvc}-7x^65-AZtz)KxP|Ko>#c9g))J~ATbZDnYAU= zG>;#D*^Ngg1DJhLb({@n+qqaDg{vGA1+m{ zt&~-EBt(_9E>#w&DsMrRTWRldRNrewY1z7dSyw||F>Wk{Ik|_niG9~QwN$^d1diMs z+mVZ;9l1!_k&Cz-xrjV+A!IA#uL(HkE9mn9`ac;OEO^XTk_`4&s-9_L>a?&rO>CP5 zT)2PXB-=@fxSgbkJV_fwzek=T`(0(KHvk=?Nb-JBp+r@9{!ijnd5|djUKLFjZIT0% z#iDSBDxAn2Y3uybOS)O(tW|TqEifJv|GgvryITC$>$?Ku4)O0!@!yH!?-k#>D&to4md+(BY?2l4%wU}ALF#$tLOc|ob?lUr2VDc$xoM9G0zJP4zV;xey45b||`DYfSR{4ArV#1unNYd$H%1R}lH(jDB% zmS0lM=uH+WHFem$3nup)R>p{3Jo%kgW<{p_;^N?)L0(%$suu~il@c4L{4z~Bgi~(P zrhM)_IOSKoN;S<9CJ_0KHSgWhYJ1CJZ*Ng=dnP*l8dba3V^Ou&x*=7& zrG=>)s#p^Dd&q;9!>KFxb*%myt_ia#P1alCEk~& z3QFmkfCw)tDqWEp^|wo{n5=&A{NeInmT2Xc-k+fQmY^CuS@W7es62M422Pex`2nhN zqFLgP207Z)>h3=g^GwQVarLs*~ofgDeN;<6YXU$e5bN1Bs#rS?f2=Ym}1< z*Ahc*2JmM&;Nd~CG(U|#dy*5a*8N6My&lr?qCwh76!2S=jUf(tt5u$sDtG#lvJVzUFiIPWJp)A;&OiCHT&df8@h62J}G))^D4S2YPQq;5e+o@kTy+~UoPD)FRbRm4`URo zm4X!2Eyo2LFO3}Xjk?jI(mJhDzWMxk41^s-Hj6K&8uMT;l7j5Sy|9tZw2>=pz4xTv zkJox1B`VFf^?p9}eyAf(e2odeIr1y!VGL_4e)m3GBvVnTx01g_%zMRr6Mt`GwukP& z*IL=LaLRU<`>);m{)Qi1?)y%(Utdgc?$`OZ;eLIy-}`>;_k7<68dvKxuQi|hUPer7 zF6=ge7AuM?`cfmVXemn!VMSq6uhJ^zn}Kp%(OyabTVpQp+Sf9wt!3rj%xf7oPZg6- zjFaI*aV=AuB`=ZvgIlneH!{YHLFCog33L-c)7RR@h)3GjJm*p0ibxTKuI9t#-$iV{ zSI7JZZo!S(7JPDxxL9U*q#BE(pwgig>vPlxtfooILa^qA2QYet;>OFY-(i*YJ7#wN zoTSQ#1%Z{IU6d;vGxvP1!chZbeDb#`bQ^8-(p?r++NZ;;HUqyP2B6Yh`6m)lQ;Jv? z`1UaCH`y5_ywRFy+Ed0mG4Gq3A=;arC$U9^h_Ii8#Iy#p?mJEGw2C?|O6&M~=(ter zTjC4p%mmB~oT=|#A&iKcFB)xb8X^ZOWZ3vg*2b&I6@K3A5CM1B37Xv~0=MHPvDw`{ zAgzH-LgnWk-JR0dhfg~x zjZT%)NBl?eq5pYhN9FM&8vd>L&~G%fFK^#Jyl(xkQT@+x^q)ze39`W&^SMrJV~D>h zJ8Kzk>W1JQyl-f$bV{yZCR3T;)c|()?h3(M=nP0cbfl#qrCa3d&MvAY3qH4aA_!4m z8+=c6(%}n{#7FVII#>Yt)j1$wXNpm6&KyJ&sZkRdSx#qwWz{*kqhB(OhsYlUEkuW=HCDO<0^CV-=1RgG8t}7(4EzMeuEY< zx5=SG=73fyhnlP8P$6?@tDb76LNFct2aq?l6c@GG`AUk+MLN(z%TkV6xU(ipk-bp9*?#< z*25X!zd7&Sul)u@X=6n##Pe^8g>C-xZlS?2dP#5jcgWKEM?-iGq3(Vqu>!r0gX40u zc3i${!l&ynTN@Q^IT~`zLtJ^`R6a8bne7L3(E4bg#kW7lv)J}26GY$y%sm5 z2r#_5Jg(WOgv|w=?1aS`z|K%4Flg3J1l#^yUFP;yfhWHkLlT*wIj=vwMbVM-cp{&T z8@i}yBEi5KLtJhr1+h}VMqiVk*lc0ii*~5NXA2-PGLAh=IaPT zJ%=i2+~C88(kU`Ik!mfcF0!Eu(oz=oR9&oabdjyPK>tZt0t~MZqDi0L&zf|ICmqrz z9U`YecMIeU_MBB@#fgJ1rG1|<8utBOZQnO6M*+oiUJO1}Yi@hMjy2et6Iqmxc4E!d z)OraUiGo-sW(1ZEU`;Z-anX4<3Xu$0N)7z}7UB=){gGK*5WTV-_tP1+OwX_+Kx8-n zgs-0~=L0-95IpZk0G_Kfcz#^glEP`MD{j@Yu2mqn@Ny&%?JS!MH$^0`@Y(W~lpRt* z^_vTSbykX>u&aLpp+@3f4++sNQ3*MhrYd86wWswlQ1vnWr&LON3_(&ud^pJ^6I6MG z9#G5Hf5ho`c@Q*zPHQf1s#;=;kSOQMzeBuxt7)}9IR;iMuC3NfjktR+7!**=b@(dh z@F3ey9uNIKMEyR5{m#?+ozuv>FhVdywBq+1BcCLfEki<6iKE#7`l$Mfth)9%;|M8DHW$UlOxD%Tyd=dJ)!|O9r&sS z^8&ioM&isb*JeKVYI=p$iYX6QQ(hp}16{aT9ptI+slrt?nCD)?>|t^sfA`k+sXzuv zMYn@`6C5_O_#W&>ki0tzAgR+JF_#h~4FZxaqa-9h3iy)q3H8s|ZdZSyT%m;edt{VT zu5gFwkIVq0)&oCIu|ULKq?Q)8HLB;bh>dvFjE~pEVKMXW)$(NA|9z_rM9r_w+7DV% zQVT1k74G(_HI`7QZj>!YX)W0dNXIVZH?D8hBtO~cVVRst%D@$}S*kMB#C`!`cxK)5 zev9LYE=3O!noaXma&yvqpvB$&y?t@L$tfAv&bDM?#Tvwl{p|#&7`8jDSnYQtH;Ga3 z&aHF?Pa5gmFLSY5Wb4fQrsB{ozV@dbhAqYg#vz6|hv+sRe;gZKlAI)HzBG{I6_HhyR7MIs|`ws}24>?(p9#YMEOmwhw>pRtf)y zQw|LNQ7-U%I|l#W4{h*Izt$E0Y=A$1Kk#q+P{QBy0O5b=!*uwMuj&x|v+eNTeT^&p z^&bNMPiP3Ke@~qw^ho7M*odooyH{JL|i_=_s1XAGm!Z zzX^L-xEg4@ni&=TQn5!G`ZecWeFrThcuOr%6Z*~bDdtS#40+MrxQ>&C$SRvwv`FR! zI7*1ERAtRsa}Dk*7`()EP2lPte=w5*h z(wei#QW)~hn(I$Xr6+p+E%k6c_E4bp@NvC(e;kp8@19Ee%Bisyx`r1TAAVKlCBM)@ z*900r_G(YUDs$d1_T+7#h|iUO`;GcbwP)esL$sLU2*&C4Ei!2Z87YMYZU%?sS!q0? z(wGSl^Myfd#!wLl(MoEtc*J0A@}Xiu64Sg&tV2!U$@TUi+Ra+?-~lLp=>aoFEic3- zPHd7^0Q5{J>vnoZuvCKMf#Iel%z~XC|)O!DlqW{KVGGg$nyoQDLwf){L2%1mL}9X zNOR-E3H@Q7%Y=R&lxae>_6dD3NKU9xEKC9scK0v#_YCBt3cg6=KRd{Y@`G`F76@H> zQ;H24VwOb5uih!u=A%E8Vtgo3gU5bB%W0X{U~yXSZ_X7@*+UcFR-^!nCrXF5mv<78 zs76cYi&XieJF(&!zfeK7nfo&uy13@Pai{lH?HK{on0M_|#y>w9HnXSucG^x{+;2Xf zM=#l>;wAe#QMoP`{(2Dp8kMZI_D0*E%Wfu`2uGxW30}2%&3bVnSS~nOt1(N}xL`eM zQfe(S7?otO+B_2`GoC+-E^=>{q<^gHV8D8nrY4_C?z;`w^810ZN+rlf5^Xk|$RSED6J^b<+MF4#&G-eAl?ka|K+Q7Y`%^ zkKv^`?xs9T78R5I9%1kIYSnLK9VIjkTbpc)Hb>*l*g+c|4m2+3jn=+iutQ5{6*f?c z_+^P4Vc^x2w5t-b8Z;MnJwkrS1ECITVJkm^OtZyK5V8|hKDwBytQD-{ z6|7QnoOjr)u`1Bml0^$>eSCkrWhwVR19eL|$k()dLtyzmHsY9X-X9<_O${j~i7b9a zXX2WHPO^B*r7mP~@xV-E@y-$(Sv+^3B#Uz{6H9*`Cxwf#g4~}tGz*vW{sB7V+s+14 zjllAxA*JOEmv^vj3aovTLJDGVR)c zV%x42mB?LNbBSi>Pn{{I0WYc#KDV=V2$r9~N)$VphKX)qeO{aitZxU{z}i?W!RiNK zjYXzm8B2@Nih+TGysXNS%ql+ki&RRXO3H~_$?r7Wav;%h-%BL=J)Bca5#5s}SzfGV z!NEz+nC&vju^A^h(LTvOYLdr_#Y&!?F!!dU2cWcnJd}p+frnCJIucBY#Y+rSRI@>Y(*$(`T&-Np5wkOlse(f-Q$T__% z>a5Wq)3@FJ%E?a5k^yv)GlS5_dHj=nQw4v!8rH-z-PjcM0g*}uw1Ptn9rWiy>7Z8u zr*-;A4Ec88t8XDEiJ%SgCwmg)xgI;@CoyW&awMJX#2-Bxa1(+&8#-{m?k^AAt2F`_4`#=K z26`gFg!_BY;s$Qm38h3N8qAdfG-5Wg(VCWs!fb-^xc*L}Fv$(ZNJbbx?`MOtmx8fG zz?f`EwC$78ufV(|s#evCno2(<#h6EMeHsL~uG8RpdX}UZ)kHDoQ_T;8PKptq)xu8! z&$s*(SfM=yGPnR#K&!tFq;ZA7;2S1$g!3-b44ZOHJKJCB7o1&5ncY(aXA@`pZ-jBN zPX8_xv$1T7tIbj05CVCVxrupFe32YVYPYKfw&J7dR_gAsv?yH$@enYQ%NSCBOC=WmrUN3Vyfo)ap13QeZWWn z_3&l&IE7tG(9>SNMf0eXo&QGR&gH~PjY(V9vI|hSbEQX9Jxj_!cv7kE?B#8QDwTo8 zJ%(cL_~VK;i=gl9r_-ozJ;-5)QHdi$hQ_3rxQW(@2Z@t^&(`pR?%1$D3wSp8RpaEb z-1sSV+85FcAMOV;{Ixd2^Jco7_WS!~I_<0Mr@g74JnhGbl}b)XeAXtPlz0!Z0hv17 zjReMwep*UP1mmL_qB({;i6(RD1@vHR%3VeJZ|R$UIjJ3elbe3%Hl-gLQt`iQpZvzj zE&Gv2%gkv3evN^lM{oAOh@s6OY4_UMWa8n=m!xFw_eLwlb8r(x05Xy?-v|63x69`fOvZEm zw<~?RLib4o%tr8gi$Jm;Y^I%AM^rk2V*k!=A{32?rq_f+)#VJcF?-OUG zQ0&P2Unz0ExaTLd;oJL$#a3aF`A)0cw2S_%HZ8OMxAK2O|2!$yQF{J>vu1nvXz8Cz zPwZ&?n^X4Re-*REQLMWR1B~nP{th z?k|qS*ndHa)&ef{n|ZQK8@7i>+G~*{EMgx1a;xnykx@8Ovh2i#obM@Q1eRZKC~sy1 zbv%|DCSPy`N3rvx>_@!1wFYz$kn&Ovk&x(uLgt3(Wr77hRWOxkG6|$+w$bwC(qSwR z$eBEuQVL-+kJe9E=7S0$BF9=Kx4~wze3TN`wUKz1{Y-+XdR)2ccr+4NZ+1mqrKt=M zrX(S^@ei%aq#I;}u=7h76#L=HUzY=aL5#$Di2dphtl#c$f^20F9cuw3{POo*)J0r5 z{d(sa$$YTwfxG?UXMv6IZG4gFp}W6^WZWcii*r0WaAc@)x?A)zGPzs0_rLVpUARx# z7@LEFHkpPd0uyf;2<(mPvl8Qrn+Nsr8lqZ0nUdiW)Hwd?_`jyYQClOT500`&$Ve@qZeIhX|86%v5W(nik!Jjf?`kz>_{G;*$?*0Gu8(g4Ime$nfg-;hFaQ8EUei}+B$YpTXDJ%=NPs| z56jW7z|q&y=+E?WW=6Tcibf|d&+Lh5qc63O{u?#=3bj~UQLLMNIG>POS~w8ZpmH+b zfhKTgjD$ei=rn1=3j74p3`{$2Lk0W&U)D$SMK11vus@HgG7ZXHD`# z)fD~h>xMwnJ~+ATLi-FhZu6>Hz!^y%0d;eE7=wm6f0Am$`T~-+KU0j($C$WhWa7^p z6b%Dt!7KiB3N<5(^o%m}uE!L(Q7dB^35iGiL5dr-1ZJ46t|{hAo`F$d;Fme=WxIJM zB7TCf;OjzXR*t*RkbBux2Q*IERN$>Pnh0S$B#_L@TJl96_h05C_mCDtJ6jq zrAB$sLm2}ScQe;e!;|dA#@@%%kpD zI6-Jb+sp>fvclCk(Yk|U(sFukx7bpS37twUz5>1S;0HIEU^A~f{)>GhD85K^Gg~!6LFcprLg(dJSSXYRqFwJ>y z9F%96@7+$3PCM&<-*yv~bkh_oFS}ozRw$szkz$djG9OgM|4!$BYxv&__}^*+(UFa# zF|YAdvrkW?CDr!wD$_^hF?;!R^V>o!58KOY%=f9h$X;$O&Gdx7{j1af@{>j-g{3TJe8nz;LVRm9o7S5war0OHB};v2F84zMpawaRPakOZh8)=<+t~uLoYfNc)_MfI_j|2ao1HRr(%Ie# z?UZcf&(Nrgv(YlGQBHOki$^<)f(wT`i*oAvIg9e@axq`XE9JiMk;z{4d1eI#sSiPt!WHGYkjaD7 zi~v(nx!~|}l(>+e8s7<7(_3I0frC+kw#29uYX5@9qJAz-ww$e+aKIG#LKHmHfQ862 zId2#@{Y`UWr`$Ydr~90PeM)ac)*6ikRnn4MQpyBj(L#wxnmLaGoEr5_&ARusO+@ zAj-uaPnt8Xoh<%mu!@Zr=9F8Y6~3a*?l}qDwT2%wvA}b%(gmbfSa44ZX(JQ2$?uj( zmSgRqW-f{bKSu?YU#C%46hcM`)yo#lb1R;2pQ&m|je+EA-sDxf0J_rZsIG6>S}&2O7rIFEx@DqW z<1Vjgc6{6l4`vPSP4=`DR)L}5dC91_C6F%sIH3>#~zs|O=7m)g(wu! z)npmunJef@n@S8Fvw-mC4vHV8va}lhh|^6}A}%sxtYP?!`aGxzENUq{-J~4{GI6E( zOh$?Hl2s-XOv?E@p=@m)E!N-sbd}(fZ|a$NDzg)okHl7{#T)yEL+J z8WrRG$9OOrXeD3*zJe-`<%>3ATQ!(kB^fin)JYroG&%4eeWHUK)UNu)3LnOC$JvHR z;@56&G2f?QLB6Q}Ym_k1AW!Loe7};LO!R3uP_t||N+RJ{06i&X%R_83cmv3n1mj=e z1I+ReW_8zljZJ0D+L5IJ5(ISorvZ(+Q=_Qf=pfnXnJlPML{*N~t871wt6&6b{m4Yo zz(S4xGn))tj$Iv0UFGOqJt(`H?A8@1m$>4-pSZ7*EN!}{W@~+&CHwN~(~UaT{SSJR zE8@KFZ+o?M&u?sJ-Jg1n*Zpp<2FU4Nt=BE5G9XuZ6(D>l1IvHJmG4i@2I|f3l+BJu z5}myUlXDgh((Al@9Ru9gGR2FAalx#04`0NUcB9y@@Jo-@!+Ua-KK5u+9((m}n_c4| zG5#?${%d;c@z#5UmdZ$Zer++t`ohqTA zxl(rev|;Tb3CQcwP|T0Wv6A_GG+v2B``h}>K7PM_*9Tk9x)*!oK0+Q|SUR zrO+$|l6~L8r9EQtQ^Sy_8=vj%%u3qfUio?p#e6(3^R$j|PV2fh4an24VO%^k`HJ|q zzE%4+`Wk6aSYVImvHC`QQlrtUc<~LpjPje<(9SJC6>&P=1yOsZ3$Yx)b_-k>^8q~8 z>Vh_YU;Ezs1NE4@(6tY{$j8G7-a|@#5Z75f5H}i4ucj}EP)s=KNb8YLz6|-X?NPt- zLBVy14J5w=Y1cFij)l|Dmq|SHX;425N{ozOj`^ase){qip(zMHkT2{+Bng8VQ=GBD z8hHcNhh#zHw#7}5J00rK>j5*R%RNEiHyF=jE*32VE{APSA&u z=4WdCmw%JJS)9%`l@5%^5Vha?aQJ zXwF$M=gWNBoIwdSc00})0tCUFeR9rIefybnX?t_FRwS#P71^GpuSn;O+j_LSBDwot zk^gkr#BD8??KEHfRgeT0cB;o+T+?YoPDd}A?h??vKZEnQGtgw(t*RVI;|0I#CB zXaxm(OdQAckOD$z!+dq`6lW2KD;h#R+@g(UCvmOhZ7}H7;a}mgO$@4tX$RJuo%u70 z+7jR;g(Cp5m=7bUi{b^Z8sSVDAy-iPylw4_fI#Sj><rzmT!`8*lu zP&}VM++>UAlRM7dDlOiw6*yYH!^|Rf1JYd}z_Jh(K4aeAC*2@$O zIwMdlR&O$9Mt`vTV0g*%Kg11U`R2GbWqFNpwWV?>WKNw$t_H#X$KIEQM^$7E-|lpi z20C=J1cGcqg0=#Z0n~tijYMrUXsf6hafy2rbRu*hDys=XXo9GW;DX{ZjygKyxFDe8 zByNB!F5^B9E~7WL<7gs#NPl&fs$2JVcM@ja_j%qQpEE$;TUDn{)vY>p>a2;w2rHR3 zI^LRe_+9M%v4u3Dk(cfEOB4q1bU9$WL$clwF{s|tefCKFHKQJElEPvt!GeBlFIQ!* z_C6%uC#Si52~XlLP7@`2d|?bhM1ikic)x+O=LnX6hkKP+4&`}Zev@yu)`3YI`~ns) zU3?xDg#cw$`6^N*96{#4ZzGelNw{T5j8F$R*$=UR^ZOa7IJvReZ)J%a0Ijb(3jMiI ze_IFolSQ1E&lZLG32J^BHorn^J_?&BDyYX->d+o<&iOLFFnmcYIfalMNOn~BP`%0k z71q&X47=J)T@6mbbjt|r3R4~y+FhP)@*EI4S@RTDA&8EB)HzR>gEedpeAJq!*Qob5 zn*5e=BKAH7dv841WeM;BGU)ux`Oi0l*k^o3$4QvU>_9apSD7JRfE(YQTg5sncKOGO zhLtna>o3LYz=mR1=g5o_e3Sc*coWHbHxhpx{V3igE3rp4q~9T=C#E31dN?9|n?XpU zxWn66M*455xBZw}hnk$<_8e@TI9Gm5HtoVW`~LShncdawPI(aVL5N=v#MYmk_;?5+ zYn&Iq@`#b%VnTOl$hd1Ms+P7xZ(a#Ru-Ju8dm*jh8ZO9FUqjH1o=U$NGVl2WMKk?4 zj7dlp6`G5&$VLeUu4GD(-3Ts1it1SZWzJs?v?wKc`CD^bhC@E-q|?p-V?YhXin^W8 z^{6>zx!6Nu$>hV5x#t#GL?vB1M~-W#yav`5yv|v2i(Fesk7ey^Z;=ZO`Yj}QpVaO} ziXSS8HxO9|+1yrha1vD+c?xKuXdN$c`OT&K1f9!noR(%vta$P^!xX%Sc>mGgGEAY} zr6L@c%O5HT7@Bi}T`O?1<}WAhN`fH^NMnu4U&GJs_|p&edGOQ6`xJhgmGhhRC!)8M25f`}ITm({N97Zh% z_TK7}G+aE?^Fs(8Hu3`+Kfy>Kj`ou-+TJ^OK+vaI>T{vveBy%t20WOTKavjo-mc`A zFWvA5yR`4>FWr@lqu{Vq8lWjFenBrQW}_YVZWof7`~D4NW)6`V@Ix@y9p0R5+>Xb9 zhbdRR-uDQkRe_oZ4Vh?EnvX*@v)k{exQ56}DU1iSyXEi+izCvk95Frg99T24^Fa<> zjAkT+FjOsL3wW$}Jyjfy3{FWwahABig8k=q)$Ml=StJ#|_O5k%lS(TRaq(6!H(PIL z=Dcgaz%SXuxy8FcNkTElU^Yj;Y0fSsdnH%f_Daq}vrLio+m<6z#nyt9XF!=bhsr{7 zmX!tb1ym5Ar#~&B(;bhhKhQUO|6lOU%C&FSI24bn)eODDtXi^H`iH~glqGv*gqIS| z#Dc5O;00I0hcQ^@FJg3}bXI}I4-3|dR0@$W%o}O~7S@nQ0X{tS%3hA(f#wIdU*F6* zqiu&(+WZ)ce+Usl>z1boT961@u_RxtOTqYCC+tdEtO&-0g3vc+ey!yyF^gtv#1GEd zdcHuc@Xl_-3J9N0M?#F!9?$Ia@YjrbxWxQaONT%=Beb zjONREs3yQ?Bj9T^z)uh1K=m+vWiWlW&b5a?$M&3aa7035NU;5EXy1U$$P3sWoiwn` zN@~*`+nlJiIerM|{S|0K6FHPp7uoXj$&MfS{Sw^uhp|EslrM@rL~-+@{r%~?XrqsV z`6=WBsfb;5U?ECsu>zSmaS4M(<+3d*R9<~Or#Z>{C5+Ud7}^Ax8eBdNsX@r)=hceUxaVvpf8s5AGR)^c+Keue zK#k?r2+JQ8_CsHE7Y1opY*+~l4E8sjiK`1DZ1(`=rkTw;@3X!17IJ);Cp<~Zp)jyP zd@VQ8`$D3+;5KGnmrbj()J=0(7^y4vB_qetn^~A|u0k(R$In_C`Lgev3uNG6u|VGO zq=P&9y~yquu~>zXz@K#6YDY&IYLdcU{>wf5}s6V18#0?m*| zazCZ9X3N7&Fh@t}Nh5zpK;Hb;xq(Ijkgv0hHX^iI7?sZ+MX~J4BfFpBnRyA%G@{7F zgNhFu6-8*~>qM# z$SNrq52pvpCo(pgRxo~9FPA7D-(W&Cg7JwZjmLpZ`HL1b!!48CXV_d+LAD<2@%*eH z1{BxRt_RUd2bo7WDB#Fxnuze$;XKA>^u_hF zfl1jio}ESf{ff-&z+SHiGX;{OIS|BhGIk=D6<2jSPWar&F&ZvDa#? z*PVII$;8;i@4>+s8hNpiAlHf{O7CJQJ^U8F4zfFv-x{AY1Z3KLuC%`dKX+|c#n09A z>Vc&NSp|vA$mcvVsswUkPB~Rx$;$v-SjsHm0h0?Aie|8LVcER~;ZjPN^DL2TP>jZ4 zRvng~c{5tY#xA4|d?_7p8rHPX-R(jTkA)ICj}{XK>M$5x4pNC@y)rFNSyGj9}+`pw00tH;GJNfzt?R*Nys zmYR=T%41rULS&>>*%l8uuKD5~@LTf~h=1`yJRN?}op#3!sPD-Kea{Kj>uut-AWE(v zcHIr~TAc7CR^w*OYeybj4vR{rRI4!W&O&klIycZjurGh*Sj&{P ziYnfO9~mpIe7iav(X0K@=!U@l@!~R%MQyy;o(LNT0qveLfAc#8pX%gYRjRrRs!pM*EY5n`h4U<`ennLe`8!rURs3TPq^hfJ zmn$<^h12${2;Urds*BYnY_(!vv3}Xyy~)P?`kOz_QeUCebS*=01>8Ka*&KkJJ8Sf? z!GO6e=9TGoX zIt<7FW%bLr>?peKP}UN*v_I>*y-76@aV@neokpdmd)eO9@S+E+5BbXl4(37bXHbCz zHvcq$c5=0uC4N+oFX%USC~U^MoSN$IlY)zPJu)v>ETqNj#Xpg(hU5-CvP9m?SFBnk z3f71}hT)?B=wJM>^pDH1uh*3Q8~2_(*JtoOV-D=czAm}#{!+15D#hV(Wd(hLLhBPm z%+B~y+*n_e=ljwM3J-NN<$-`tR>$6{MO5Q&bJXU4T&;{KG&@dw@5)i6*_v}C74+e5 zLxBy&z9iX)1-MHx#!XxW6Ds%5nW$O6wKbWPd0nlFVN%&6+*^Kw%WV+v`(!<5NUx&e z?FaxrY?sDEwX8GcY4x=-l%uG>2XgLhgbe)4+x$tSS@3J=pQ*Yfx4 z>{ehw5DA|$tah1okY`t_ueq0{pX{;wcQJ*57s2TcrU_*kdjDnR|(nF}S1HQLwg{a-5E#dkwpYnEPuL zu2T_L=8+=AS-g1-t-)~R4BFRoE~SFouc0*<*4CgD+mkkT(w|+$bd2zY4d>=mT7}&e z0^ipt1RySFz;~n%$g6d{TCM0`3ODXF{^X9-TXmc`Dr%l}ts3X) zE?(@_dloM2P0Jn_X*RS+rz3o8S-@8gq6Rw#)q+Pos0^Sgm;k|-NnAxe{A5it( zTptvmiDRk0GTv78vsL{pSzl1p1fVSwOQp{(!vc6Q1LEtFFcvkiM0XQS_lFb%!alUI33EZ{M_lHA*RqPiCWlvd8Yc@ z-PI@1{43I@`)&_jEEsqW3>~(b4`wOVRd;;%AHlZ*27FEMF~TMI2v5$Uc}bHy71okB8e|x4J5pHU14Z8^l&q6b}I%LB6@>TY2n;n zAedszrwGQP5#pOeBuMOO8jPXA?1W~sfeqWK-eBUd4ue9rC{$%$D$6k1#{LpIpgo(Q z`_W`pL3nv7LO7ATc6RX4(#)MNTdF!QP@NZW=LL-*rl-z>MqueJ^jh@M18rp$loJTb zWsIG(lOb*6=V=2!^Jax#NZ}`=h=$UJ<+9d;{ylLNqS@R5+UHw-b6ZpIgq#WwKFN`| zQdS``jSUDqr-08tDBx+=-#CKW-`Xl$4+5x+?+PDpE+j4V4Q&+|Pk_H-At>2?z%XWE zrVhrJp8R7|T6<&8XA1KXey{EM*fB%S$4Q?b+rfH@l~? zA9gn4@D4?ol;Fd+1)Kb;U|g;DoRpv*O1gg0Dudv6EdN2dGHfvasuBf|sR0VBD)5e! zGvjSdl8a&U@6SSE)i{^Yetv>nD?b592<`a^@>;FjT={SN4K~`1_{?kn>{9s&BFqkk z&5a+oCRK`o#s}yUk$6jQcr-N}NDZz01Yz^kPh08IeLW>7m`B5V<}JS?Il)WrH2I01 zfL?0Tl)ri8!o8%|CTp%eoVb5`0E1F5O)D$sKKp|e_2Kh4R6 zdlBM3)3Bmjc5>sVxq$D<5%bk!AUsT`%CW!!jj_N^{j=h73yB#R(+NWFC@EOWvygR5 z!I>}MOn4TeBn{mJ|ClS*wMQwXErs~p6J9dABRpG(-k_6A_TQNw-=V&526l8@#w%}LGY_@BJxP!4G{V;g6wpv9D#+*Cp83GQ)}tw2Jz=$B-r# zd<|5ApUTVwkM2z-766nwf-*j3q^BU1#INHJjCb8T_KYd`iC0t8!PxXUt!aRo4z%@- zA}`th{73id*(B}!iI@WK3qSj1Yyu3qRIw%O=^_VIUF4mAHUYOX2vPE*56)~(Zi01W z-f(8K6_KZcRGW}mg3?L!CKGSccT>~89ywo_wgl(+*EGZnU)P8(4sm!Z&hcx+997k_ zf4(ulH!NF>)KjSNCX=`^NSNV5HU@9zD1AWL{Nd7fWFTJSft7?=UsjAhc7+XlKtWxY zcvZVYkHMf(MI2EFf1~x+yu&F}&ZQ-w{6_1tGkp*(AhOH@b;o1Z;}csEIoMkACJ37v zG&Z9~y3*_?bqWd`C~Py~3>51F{w{^_u&35juy}j{#`=dX?L$5?SS?<~*k?7Xd{kiQ;GkO#k=Cr<8?>GNNdaw^L1n_x+rPhQQDsuNvkk7^qQ zZHd;?Dk|IxtLR>B6@9zEDcP9d*)KDX?fy#=vxCD{;f0g%t0z@&P~exOT5&kfw6=P} z+uVT!#GSsB9EpBl6T0!&EfI`DsNTV^EX1w%%ui}vLb{jWJYoD zm!$5yg9!hX5dP^kK=>*R;W_&R!kBI*y0{9bw=%F7mq=^|lwV6UuR*q5)kB8FZ;1C_C-T@29;~pveua|WQM#%dRm6GcJKeP^NEn-(&YyGuTKjY+PW#S_ zd~8_#*HX)XdK|5N|0|j^Sr0r9cMDnr;WZ|~DZgGfY>%4ju!q=m-HS|W`4>~!@yhxm zY_7xw5*v;o$?|Dwh0Ri`*qeaJwJ#dIldQP*tTqEiUX`7Z;SmuYCeze{Yp=Br$2`%ph!;A5rox3j3YHSK38GjHbRUl@Ad`RY?#!R0SvIcbQj4VrJB z#N}|tJmrFBP-Z4))9a-MN0H07EF+;=0Gq{TSS}Css;s#J%TBiqmHjt9qO#Fyq`S_S zBW)p&c|btOc2odMu11>^7VqY$DNW80A9nF8K39hPjQW0E6LqIEvr4Vc@l~Hj?=6&^ zd+Cy%S-xODrVxcM0}fOK}1UA$C5ocjDQj9m^(KGyBWsrye_c?ALF_+g~4 zD2&q;ub7|5;5oivAl{PZ_lc|cs4yz;qBGD%iz>~hN(-ox^#2$$LRO;Kc(JZ>HVbQg z#JJI`a^SHoJ&1i`)Oyeq@u6z7+nvZrhlM2UQ zb)y>&$LjC9#b3IUqSj5?y=O+fAI5oA{cl9C3YNcs?NvS0vRk~t(N957sD2r~no*yn zIupen00^V2tMNU16PgF*rIR<=zOQax;Qgelm@l+1XxQPtbJL&4xkWFcs_!cC2cD|+ z9{mNwC=v;>O{GyTHHX+=ix_*#oG{aL6y>rtooeiq(5w$(=bmyk1 zoC~Ood$$(Mb3J$BrB>7yJp!j*@1Ef=@DC#wO|0UpHh(pv{<195aTeVHAm5pN8Z7%j z8)FG8*9F z${L`}>kA4G3L_3PE>~!NVj!|YxXcT1nPa(J<|uEA#>2g23XC!QB1o&9pZwzYaG+1a z<3xMLm=g{p;f4S%!MGY3qY(0K^MM5QlUpEE>yvlE{LETGJxmNVr{Fa4urF1=g;%Lu z3?MVFo2(}I{op)Q>cRaqhpUg*rb{#x>~UE z)5SBetaMIg`7H!DNTU>yrDC)LtQ*u2IP5TTdltu*&TmiO6XLrI^f^)9enp@C$)Bq$ z+tr7IiG1s}&k=FAbD`*Y8-2Mp^GB-r@$(!Q5$hXvw0^Jy_0=4w1O8?Te?9+i9p%^G zch*b`Z$yXTRi}rSF+XwKk~_=9;Q7jyER>vv0{>K{p##4s-cP~LR({O~9mW4;XU$Cb zgTCZ2!3;S~&w6~44nW5K&ibq)^Z&%>PW&OB&0OcGJJ8?PpSGo6S^~1~^{vU2D(Svo zrpN+n@6KiINdD|o*$7aU%x{$$JmdHm2s_3`e3W=Ogxg{oOC*|VT2PdaB?(XB_Oxo` zRQ*B~TNg3E6(TO{p;pRz9a1I}6=uF=Zx&!Y;q@nE8=T$%H3NTFv2Ya*5usCp>{FziCtoJpC7J4tyhFq!JW(U_s}9zPsJF3R(akuTQbvM_6D>Mg+088A z?89OXCi%1yZ{re!^i|qdIaehxok7_lV<(Wq5A-I0E#*wTr82WBm$(zhv4seVlOjQL zzJfHEodJ7TM=g$-A>@pf^S?^__pPA@irt|LL@<;zKpJ#u6v!mnh`F01!BlA)BR{5wOcQi|XzV%LX zzKu#d;TiFpo&G#^e(zNNLh+9i{_4#Ugs(&#Pjh^NxW0>LhxaiEWM4hu0W2JezI`ME zyY)$@M*p3!o04q5rI;Ram>g_ zb$z*ta#mPfl#KP_@0V1EPj=#ij8T>4CG(tf@tOtF<0GULrLGEB$Zn<#mAQOuRrb5 zBb~*%@a@4!Oc6^n8%BbdiQ_S6pIeO4dw|%FC_ByMd;@P6wx~7GU9QYGpr8{ZI$!+C z)>H|r77E)rKd?mKYPE;r1JLJNRv$7i7oB8y+!v2^j8I2u(*vN%zra)eEQw8cnr5_lzMqF3Vc;60Ek2k5HJ_Sn`1lJ5T2 zeG(FY;n#5nQ$u@;eS7G*E}T6_i8rIBXKym8EQyO?osGJ&1u_?vR-VG$_mwyOLR$6c zT^LFfCI%@rj(r5JID*7wR^`T-D%pK(DA(>zTAufR*P3JzBfU6OG*5&&I!~DlkaZ~) zjr9ALWGV@$B@(dm51I!bl7JOoAPKlaQzqx~VkgyPsrkvXVvU4lg*^ODCG2yajOHb< zgWITssVN-{qYk_o1SZ94*AwLA8iR8oZM{?+5DpqamglJ1b%ARJ8p5Cz5Sl6Y%<6TN z9tcIB^UjBgp2{v{0RJLcmx#&90ef?>7()YMc;~G=3KrNH78xL{b9d80!dm!+F6mEK z35w@_&@3yKO|20k5a`Ab`=EkGg8dUw&fE9wG*)ZuzkbbQS{xgkzr^QgK+#ni#LPQ4 z1-;O&B%3SY0yyFh!8JmilDM=&0!6Zj{3mG_^E51}H+l7%OD*-=)Q&P*Nz4q0D}g?a$Rofsy1dp=uQzK8|=g;#rc z*CTc3yGLY<2b}IcS(p3NBYuO3-3vKs@)v$bx#dWUHI|0P3ck>X22)qezh&PO?hPoe~#t=GdbGV>dc}E!3$)%kEfX*N|r@x>X0cz z_m`?=U&h9!)70(AXI`GwPVaIeEPcZ=r{Pc;lg1XB{Ud*`slajew||7B@mLnXlttLO zs{%q40nMQFgNM@Z4Mu#cxv2ufc?RgnX6K^Bd*D@FPS%n|<^}zT`HN5`h_ORK+zB4o zAQDj@{od$p$!TQN)>zHVPlWlH2*+j#C;AfDLm`IzrGrNVFlp} zW?EDxMuE9HS+>hdtz}%duoEmd645}mN+QQS7%dufZX&i}!-Lw@1i$Ssf5V;CM=d_w3Ih#g_LjjP>tOAV&L7ye{Sgz* za3;yY!O9mh%R4UF+kyLZ&ZT?XIuFEDLTsLrI`3sk@`7aOzjL;;%~vFam*;4N-hava z2E@B#p+OnDqywRwa2ckxDrA|Q3Qd>s1EmwK)Uv!ZOqUKDaY`rp!}{vNAx*>BB|7&m zmW4#k=;b(Q8y>0m%|ou(n{3$O){vOuGri&^oLQqm$EwrHbmu?HedC=YV|}s1SXdNP z;HNJp6(t2J?}Vb!-)Jha!>I-S?#Hd8(mF8WGy945iHV8vdRJ3?C{%|O)CGw{HR&(9 zkReCqlg?yu(8O9Ju=7W7uEuiI3HsN=nx$PIpmU_9Gy9!Z5@bW-LX-%{MM>h2^kpNR z+0|0@H%f3d`LGP}_?=7BZU-orf%zaa!G&h+{-kAnCs+ud3NL>ho$-+nMM(`YlM-U~ z;!Jwr0QO3Qfnn=Q;;dUJQfFKP-R|9D^woNru)awQjD=cII{ijiVg$_Ry~`P*{-mLl zP%UJ2oMHcDh{yPdVZ~CEX0Fo5#augRukFHtrYPBDLaB+w#N9NN8 zQ(PEt;|&}P5$cO$5N)q9?fUewJ|pm)XA)+l4Acy9+3MBGxWG`ILb2h`bdb6$NIpoV z_@uoam_1VZjkJwFD3XoAvwLhu`&?I~^%-*~Ci3MNKXU4F@#Im zhHquhbQS;R&45ghkzrj%pyoT{Kc382j@1TMS>;6R2Q1xKu-Ku6LMKcC*&1R zhCw8c(LQRRra>EQy76x?mT{CbM-2>`V}X6-82zJXH%@^JxF=wx>OHe6vmt$RPU2Vm z;#zvqC6UiBE}<8`M1Wt!D!uTcvwf<-#kJ5jNsXUnWf*908%A9kpUZ+vxxIGvo`S%V zGLEN-VK22zQnZn+aG$6tEyq?j;;B#s9yf_W*6R?xKEhyA4LtFQj-`g_1QSd>jAFTp zx?Q1MN5Zxd6(7xOIh%~IB}Yryyc2KRQ#@0`z;S^kD~$BX(8pC)a&HzAVGj05UDp@wvbZ@+tFM%rfoDe=+p4Gg3hs7X~r;^q_`+di=l1t7?NG=Sqoh4l^Rz& z@QtQq<4CuJ!QHS=6C1cn^XFMjNsc!y`l#YOQdN*gs%)80#!iLDY0G@1)g{B`ld1FD z*J?)n*!lh9#^3l9J7krbw=JjpNFI5N|!+qkV~kKld+Gh zwLT_5AE#j-G9nk8BRBV>a|9blsqvv}ojkt*jl1)0u9Plsqnz|-8U(O(5+4kOSx=EM zUs41>yDwb|+IX47KB~T;Bp=#TCdU5f6z?+-J=g!h|>p<<#oNB zDD1G-N0as|QL5!3jWD+mF^lpzl2V!^8JVx{`R0zMWK!dx>sXhK%*->*$rN5W{ry1d zL&?dM1z!R}(?E_rk6e{XpmF2BNJIz{fDetCSUv^AyC&Q)?eHQlFKh+kx#HX*-{3JioG!=jtS_W$v^x zClzK|1K3K0@-0K0Lq>V&pZ5 z^~Z$NUjOkC+McLn?4(dAlDIdu#SO9tIGt=g9GbR=Q%2ahSV*O!gpv!S!+v5-YQyd$ zw3`GG0d9u&OVF(G0{-mwmLCn*m)E(rO&76my2pp>Ys)tJd{XI0B;$}z@=EOYpRMWo zy~qa4ljueJR!wK)A6uOYfYhrShhA-d)-OY^!daqZFxst7<-7^g4=diaif3Rx-rb(? z9#7o9BthM}3QpW??ZjRAk~nc8JiZvrOQO^q(bw+zj?D*A^G8=X5A4&S`J>r}wCdwP z#F3xib1|b<{Q_Z0poS8tIjb}^A%gM%s0*{Lfp3u+eb9b4%iBC0;!xndw9MSlL*pk# z5X^BYqkFPJ!0eJOVL(efU=^24MjOb|d2a{!$*ml*u24mEO>77A5N~Wa$KIgUuZ5GW zYdDqUJL3hU@ysuYsT*%p`iUp=HlEkX8VwdLz{<(3;8Q9<5Qm!+J0#89E@D-6OOQh{ zm@|)0{H22s* zPA3ut=VaMf_Cou%I{OHH2Nom51;b>EWSSO3^cIyp;>uuK4ulL9y^Z@1B(;FB$mN5T zgN!!6n*Zgs_a>O+IbVLXo9GAKSZrJPo{+wUy^VDTXrou)C24wK$xN42jnbyBU~h2- zJ#ROy^;H@t3D4^px%E08f88xLzi~YVdC_hyQ(_@@d_gQKA22swoI}?GIHFpGvX{hK z*aWj^6HKks+!=8b{NY8h3B2sG4`f>g&P8(-5Uev65Bw`h&XXT!IXCq0cc$CWhuABf z`9sECKcDH(3g@ox2FR7eWjD7anQR=ECFfe^vyfL^`=rlhSzpAqBqQ>wcwy=R0IyG$ zGhn0g5j`+s6$_+g09XAhxp$@gcsK4$`FIoP7Pyi|{e4`^o`<8(`->R$EVuPZ=zA(F z7A8LkzW+{reNhkA>iuG93InU)B-rJFlt_howvEDAVJfWKhU(mMz}7g?;6RbcraACi zyLEuK@oqNfsWK|N(B*=rm0eKZ>d~CmY#Ar)VVbaampjjy@17SEHpyjO?1O3%%K)Qt zzCQn+S8)%QTE$>=@xfK7Ses?9{dN=0?pj{6YpsQCRl2@0T_wc$DZ=>J<<5nDG++#& z+7dw(1z9D;;DFUKHZXRwx(V?{pWjUcb#JY69{#bpDVZ3r=yv0dJ<3|B3muFDy6jD| zVRjc%f+V`}Ewb?ygFcN_``?|Nwg1ws8Pn$dgeOe52**|jyTn>waU3(9d8-6sn*pIn z9p`sAsb_N%((j<0s4jQ9r~rd7%&E^eCE1eTBu(kj&C+k2+D(Qnva|O$K<6*8355|2}I;ZB^c zE&@en>l>C`MZ~OVY)ZK}BZbX_*@V$471nMdmlXT2o?aq@7Q2&jT9( zb3ly0J`fGWQOqzU7$FDQJTAzS1t7c0tiL7aG#>~(Hk_F;pLB|U0GOR_3Hx1qAaN*E zuy^U%nhK;1`x!PvC5U2&h0H%)nSzGA8K(x6!LTto^KSMVanIsfiUs2_Pzjlv=q|u| zz(+zF8@Lm%rN4j2y<-Ikhdp>4Os{IagY**ncuH*+re$oUScWpSE|Ka9UoY_+q}gi1 ziSoX`gd(36T87#l-B8;I)GRi@_&3Q{%0h}#UuFX3k{a8jSUB}k8dYWvWV05sB1KXb zNZf{9r$wB|6#se%(li_>A>=xa$E0%9QGx$K7?A<=L-SDD4A{K2i zg1qPR5!dQ^fuZ+kfg$hla&v8iOc5@&NUh78daDg={C}kaP9Q?iRJhR27?k$7mUtI+zDCRZ$Bm*N0h&hQADtpHQ zs|YQeB8BF;s#{*|y-8RZxJe2tm&|HOOI{rNd(y_ot!<6|I!+h86jFc)e7M!fU=o(% zx~&>wNI2?AKzJqtgf#IZZ2%$)!b@o(q;09%Ox)&d({}x9Q|hNjO{}^?#%!y3Co}@V zp&r{`f=HYH#^uLI1DuU)Np_ z8BBqsO#yLX>9gmKK)gbinY}mn+^XYV<*ue)R)Y(y^WtRDC9@qq=)>)#F^Z* z5IB>V#+m%|rzQ%62DNIa);z2=TdVat)LO(FzjSex2U6M+Bh8Wj%CSvAYjcNa1B``e zP+jHePqjJ1-+2o)W*%#oL+!y(h3?Hc=VFybT9p(3L{3CS=4U(5*mDdkQx4ib4E=xC zbd-O+##uA7|8shn>HW!0Qf4{lueCS(kH+)+J34Id?5vsD-Z?$Y_QtbfP21y9@NZw& zA$}h>L6DlPdapG}!G^4M?0Vw1oiZuAYx4c9653}Px+<|{ad3*J}erKw) zpA5;}$s0Dw^Od4r358EOKt4v{K=nwMRGZ7fDXgt1<0MGPJ%C9cU-dhCHN`$ki2YLM zt%z!vh{;hc%CiIE8JnYc!v?nIl6PAm z)m+}%WV>N`F=H_nmr%N)3c6l3wC+h-uXC5*;Ad@qX;Leo<0Z;E>j_X4mq7=A16{@0 zF$JR@Ha?CF4htA61I*lIn;yXyCr+a)+Q~p_Ifril8=OvypW;PWLg>r3ciq8%Z$T>) z;luVPSx`{Vf`XdQj12kJGQ}ID5YyxLr*YQ#1L3y*dM9oZA4QAw5pJW+5Yy|S?x=k# z#PrvetcnYdX1=u4e15-8#1`wNmPf=qo-Ps0#J<`O-OeIA8?9K2Yct`=H)+d?6{+)HT&3TS+kd{Qo#(xC^=I9Rb`uae^ZI@E$wkBblW624s+{7Y z=?|k}{u#u2ZwcqgJ6GYBaH0Aw3W~G`Loe{@4|*iqbQZ79*gx(>g_3A!mdiso?a?il zl?}4boYA>8gXQw#Evo0LBCVvzd~UD&j!*1O7oX*$x{mTQZ#io^%Kx&mqx??RZ#&{| zpX02R!CrVH9fAwis6%i)lf6PI8BlJQD(iX~-vs8#7kEYjE#8W>c)a;Vt6d)u(kS_% zbF`%daS+YmLr5)S9wL%#PgSd8nLgQSX_U;#2kdUG-o_U}7$beYF`8Z(=(H+Sf?psI z=5bvymG!aa19#yCrAmC|ss=L;6(7}mI#)t$x2mDZ06bk=*~L6$z1XgP&`Dju#+Ic# zZgz^ut*VqK;esq4!k3leMDi}krU8{Ul7o8g<*L?XtJYRki*o~SSYF#t1A1VSf#h9@*7zNkcBXmiD{zh5vuAtR@Gv?DqOk}M~JH3RMm&5 zYA;c>uU>Vni)|hQHGekfK3HL%j7fO+bP3$N&f3ke+oUoJzUH9^Bz5;f`tRBMo!|3E z2yr?5WrOwX97u*CNe_eiJQgcem}g{hlJaL=vQs>fY9k~g1!f2YIdbJT?lnkySSpbJ z7tXY0qgY|O5nK!vWgqiG#4MxR4OW&?l$SbA_IyGo?*14Tu}@LZj});+stgAqu12N6 zn1O8CU3t1T8@*cOY(Rbq^YVSp+1O!p2&zH4#vf>pT{s)|m|xptkHczS&4Y}XfB%Ij zTZQ>P|0*-L!LQ64zHx0!GO1{8Wr0OzEAy-mRO#19=^$;O!}d#AS$ExgW1bc1xUU$y z7|J}O@)T;a5hmj}&Y#$e=L^6HPT&V|UM4y884|%U*rHF_4 zY39GTGbyhb_~K9R?o5g?Z=7h)r9qZ7WJSB;I{4z(Xq*E#L{?Cs5Pm zvb};GT&=wVfoNdmbuJAx>Sz>oH2FHso)|ki13KDngIb?Y45fS#4;LvLa(EFBL#B#i zt2~ltzP=p?`Tm47gFJ!y{;t9~$iIo(dE+#f{LZgkfT1YE?|vEICXNJOL1Y40pUvqv zY2(QlR5$mP3e4GfK>TAy%OPi@Wjla(vBf|aM4Q>}gx9=jJM+1(6?-a-%@|Q0uUZrO zHzz@?R;F&N5i0ngKjw5UCZd6QQap-so1tUSR>KFqup4Fu0sEL@W1$!0`9u5;HC;he zroq_-&$T6F39ykLLSP#%ev@!-MZqH;j-8E->57T{T=6_8y%xF?W2|o@_T@mUr=Y64 z{L?*D7rX$#Hrv*pqwsv1@}?3J-^<6eO+kjemMh@TcAW;<%me0NVOvl%7o zwc{Gif(t+Mk9Q-pIf7<1&Q8{)L^lTSR#HgJXY?rT>mnN?nKqxWl9$4ya9X-?Bq#;m zVe~fB`=>prcLEA-3A>kKldyZ8&6WK>?C)<+{_pe$%rJL2-qvAF>w$EJ7#-=-_FsGb z2UY)9nbGGBsb85Cv?@rIVFXUPDC`@tT0nGd{^Pj4O5ZF%Ps)?3W1B==WL| z@xLn+{a(a`Y@>@QX5}z|c{VN03D;;hKnQ7Po^@vv1+Qp)OK*v|2_{9-ad>GSzGoLL zdd%2eQ}CpVw;-(B8$Q_udiK1{qks@s6tld$9 zT+wYe<)@ZRatM`VCm3`SFT&cZG1DhC7}Oma(7-t_(dwC+^%4&JJcY^!e2&$_#K+Dm z5uDK@9P?zoORBZROXwiE8mmQQ;-DD>{(_mBQ$K>=zlOmlsuJkN2Ouf)C#M9htx?@z znHs_h?AC-c3S3s85R;hbnTmR0hbL+sE>|69ONfx@%SFVlWE&DYST7xnm*8+XPf1~Y z*Jrwb*k+Etg>%M;f8qQ!P(8X3vn|TNK>0>xmyJCYHw()>umPO~x6L4 ze;*zz7Jr9g6UE<0#EQhc zX!qkH%3~xB>eHU<$$id2k|xWh2D#c{f46-)CuE5}>a zu^6yi{avj83G-65{cTR^E$mR;jhh7$AZD zkOAaig-)U+S(h6ye4Mep_HvgV!!O3hN6|?2gm<6ubM&|qBJrG8#FM!ui*A?^89nZg z@V0vv89P<0pw}vh+)60F)lGD}4CvHDui$sCO4aA<`e@0Gfx9XYi;C(UtthUoo-hv4 z>M3~SUqh_mUI4t-J5;1Z?2(cxD*lN_CJ8%lF_8EAG9HJ@rt+N{v3NOIcs z+a}v<1iF{cy(^qR|8*+@y`N}X1qUH4V*a5BulN+KZA;iEEqTOm5B!Evx)Ob;CM-o) zSB_By06-9sD-xd$Yo#-Fg;(hrk7$h#8XZR4#H@|o6SSLOEKNZh3r>evGPp|)v;+cN#Xx8y~ zrkBh=sCcFiF6@YW^}wRc@?4BJ7PQ41yY_1*(ElplSRS3*wbEOy79mh72OqU}8r}Uz z`ug2-I%uCNztOK^9JA{Gd%$XMRzbMz*LQVxhZN<;k)=x;P z>7oinZiA_ubdRBULO*-p=!8m74F{IH48+1eJdJ6HFd9Xbc(KKRz_>(x5*!Vse z>%zafSI&hCNICRSskM2G+T2KOD#W`{^R(h>Dn6eBEL6D}Dc~QVGF&an-K3&i4@b`m z?9B}LsbVxSxR|w=q6Xy;WyS6AZd`>O=zuN!#f@m-L{;_@zi>v_KcewN=tyeaqvrFc zv?QepBx+6&PZlc{=XeWM5?Y{U9i5C}no~-D1(x_V20<-`eNtuqwgsU=Hp5cMa(f#` zxg}81S(W7~A68o3Fq%jqN$UaM+^wyFL{}=z2d=1?-AlweEFQhheAYOC)?f)ABUh*_ z$S$Q7A^7cHVnujX%YBysMU3XrFE*Fi3rphELfiqz%pc3AxgXX#d$JYEL%I;mMd5cb z+7K2BM21U59#94lwSOlLq@1%;dK-^q1p!7AC;n;X;MC zRS>>#J{s0U(K&5g+orubM7xCWbAQ;-t&6h?EV5lS54R6;g{AW6bDUPOJqdLHyyt@Sxf^;zRqx>3q6TUZj`S`u#%vsprB zOYcww2maX8ME%B7+{CSv!u{}ktLj6xsz2tQGp!#c^$*la`z02!T(^e4*U`U*@3leuUN=_j;!PzrsxKvpOObiQ09M?` z(dGpma~dDSx*n?AqUxSDz@?IsKwT7s-!N%o7FC?BDh_p2gbIl>NEsq3zNRX+^tV@p zE)#f%UPU8%u+%xCM{kk>PDPu_Uw+-!+ZfZix3NGhXMdO8MzQ#Fhj@Nh{Fx_yOL}FO zAJ_shMq%-fBn}-h7)XALwTQuD4THmP!dv`TOtqrb8$rz4i&_n#RcXj04i7x?)$J@==M#u~cTES^S7gb|$bqR*{crymPHjdz#`A_kOI$!PoI8 z$q$IG7NT&#Cx4c4{X!B7sP`4M5q}!epFUo`jBg8v+@mi2kPFj^bK+4hLkda49EmY3 zoxv|exT-K{y4Vu!;&lVcO$pJ?m|_~-1HJ24IqB{B$WyoXzg z=P*34u%7E-T`%MZI35$T93jItbY52CLF_k{X}$GufU-X3yk%3`c*o zVy>L^Gl*2^Rs%bw7?Mp&i_J*TeEMgn74zE$NnDWu3z_bpq=n0~@NMUSSf8Q-bQf1O z$IA{AU{h@_pp2&S&CVkb)pdyd=8s9~sN6%Dg5$SY@QX4& zOcl-Y`mvD)PLHh_%8Z9)XXcK#j~@5;YNDK-&vvq+&&|XAJ+2cipY^zW0+BlJg<$l~ z#(r^%pn+Ey6?_8*JRq?2L9f9*ihs8pighnX7Mw9B>y8JLp&z}NdyWl zH|`_c>ZSp1OEld2%>#oZ0l)ELFJ6n`isg#w0RXD7(fwMZZ*LThF7g^$KnNw8keREs z$bf~U7~M`b3}eq|Raw8EpdOCF9_DL3)Ibm4h|{$3b&nIRXQ|O(Y!ufToeqt*dMrVK zxF(%?POA(PAsv(FSjhk|6Y z6ylD4EtcN^opcfj8Pw+$g7-MWyIF&G1Hk(f>>um1oJhP$jn-hJC$&Z=LL)H9G;jQ_ zh3SIqx>R;8BNT2YDA%3kT(`;VVf5^v*WeMGK)^o| z;P1`^zza3Nxd0dfgve-*&zwutL@eC~pS-;TF#A-P{riJ_Y#~Dbh2t`vnX><@1oS?D z_5o0coBwQ!J|d_jfm)CPY9@gCr%Me!t8xLeDZJEXT-s`rrXaXZbV)w_OwCUY!3_X# z^Ih7vL79P*2!}N)ZGAk{$EuGoIMX7zN~}P17Urs+%5w4RNkslat6!Ev-uUliQkO#F zpsu(tsTM~s8q0%cz*yF3V>x547|WlTO5;(B^$(%?w`2WzTK&OL|1QFdu$h42pkl{Z zV+H-ePqYreS6Z6Da?w_wUX&OD-5pNd6=QcVXx%+Erzx4pl}F!`*3p+3S|Vv?UGDHR zB-8W2Txs;a_2?GzNq|Xw`HEG69=(l+#h=-^y^Ytq_cn(0>}|ZztGDr__;;!}4h)yu zo#n~S&UR&cv*ACl*PHF_)QSGJpLaG*6wt&ZWd{kH?qgh1zk2dF*`R#fxW=#Uc!~a? z`k>5Ek+^pq7_L{>3sBzM#7|U{!mO_{m*jKGq`wB`CHrSs1%trRJ~ns6&WO}jWF zZa>aaw#wv6RS5n+r}-`2ZCIyer$b2`MZ3^ZL_BT+HvscB8YB2!Q!I)w`RDcIgTZioS%eDd1HCy(Z5YAQap!S?V;Lj*0c0q48}i8t|LT2w-KG_P9Llw^aq_uZOc;me26 zA?8;l9A{GOw`))FTkX0(2ORW5Xa`B5sGD1L2+76GNpbX6@9_kd{@&0E=2RnSaBCWJ z3Xz0J!Y`iQy2WqhUc$^U(T!jdNn)QBEPm*`nclsJnp$`_HEzT`E+q@ZT{kK23D&YX zepDO8rYYJWN;arLd<$!}&Tf!cVA;)Ca{L7T|SWq$6RDcMoDBw(OzfS3iHg-&ieUgzVlVc%yGUdH21YSUKN?&IbVg%4>;g4 zn7(!at@&9m+U@r6uCtC#+xt$^PFp;+kG@*mb$-nXuPQnndRT+Kc|I=bLPd}k8Kk?Z zFh8qqBDcDJbmOMcpD5u7Dms=m)4Ac3{M_i~r*L6-8^6V4&pYuXNk;0tr||D+;#k@6 zWYuu0Z1@BSigIs#5x-R1ZsXXGPi(KSbtS5{TTqLHd+LNjFQJHEB#wB;8s-3%+b1i}L0mBE8Og85Q6>TSdKF|#3b?C&= z+t`H*9f)LHfDtXpH2%|R+FROHCy4L&rzE}7{_6<6aL@_ehY)Wkyky*@1r$3f9HMK= z78zxl*VF=4A)h$sI`0pYd4X#{>b&2|w~5~T-c<=C`iseGJJ^WThJ7i->OtqLJad)x z3JfyM@8tbGm&^Yf-O6lNb$n9CX`x=|x`Y8qN{%fcpft+5c#xxP7B z3|e;_Q|@7t^`Bw?3^?##@1IYdTd+g> zc@he-(!pEaCcL!!k@Bh8bPW;Sn_;;}yK_OAtf3r!l?yQIkY2#Bxah*p;d1-Dw2R}B zlXXuH0GupJ8ovb_$h8ZaZ6Z}HhPB_gh0a6_w-8uK9zYB;>y=%p;kN@df79y>dUaQl zZ@VShtrWjyeZ;nP9(sJ2Z8KUpOm@o!6W-8dOBP_&8RXV_ee@MuucUP2@O06Xj)!IJ z_;nT=7fJfH73o)r>S#y9E?q(pBkhBEeI+qT{as#58IFgH8SoE=81nRkuv=@q;)$1q zB=J98)Wn%Q4|_&>@2{w!5X9wXk9qW~%Je;hge#mbBho?AuYWlY?}MjbkF|OJA3Xg! zuT6N$Ge8FH?TQWoltw5L_YR*RKGs-I!R%p?86I%%$k{yJ52lUzV|%$Od&!r-6EErJ zneZh3f>9w;WB0a+KP^{2-3!fLRb=c3r|gsC{Vd1*;^9{1l12kc1%+4t(m1TU0+!E_ zn6OYu(Yjc=0P(8;WM_doh@ArO@7&bGP+nIlXb4Y zrYTvy#})IWvmVXBFIq92aOck(K;w{s*FGF~KwiUNFbodJADd&|JFvxOae}fNCW=Uz ztE46_ger6?Pzl+LC!*CwF7h}8>p=3o_VA7m#jUJjf(3lywpNUnPyv%&&H`2{44RAj zpf)#;LsfGG5wZ=+WJ#M|-h~dLy?7L18K~P(`CSDIR9)n$EXF5}Mn#dmN2&@^85yMY z=1M8Z6Km3yI7fpRR1jVN1w{Dn16f+wouq|vJSZ{HxFZ9%Gydn~4#)p=)|BfjLujZ= zr`kh9sVGk@PrO*mP8q>)+7N>VUGBtNh?xTkHtKScH~J}d1q=Fry3vpRncHa_{8B`R z37i<_Ums{qD#j_n?%vW)S!MO}nY&v_VqP@+a`wrCm`1tvI;@vxG$j+5m($=`u-bnl z+KU6ABwnv1=TY;$efyK_JkKB&2@s8QQG$|NuEV|(U=I<4DXo=Tddc{=dPSMwZY*@rV-uk`kM+Z>7AqahIv>I$O2TyFz2TnDq%4kk}bHSkBG z)iQglPI{{|L@Ozs!z0C+*-`A&IQeyyirqQJK$-75#}M@o<5_UT^?n(#%xVM%lVbpP zaV+@wwh7{lwZ~|^zkgqAGO=782Q_-xl~*?LF{y01&u~5#so8Zgf}3Rqu$RDjT<{V%_Q3U9_4L_vves$m6^EC?lwMLa(!($ z_s<2Bxm_+PZPqA9DN#t2KZmqZ6w^%caT0~n(c8>RM-gijPUNe4RxyG}M)P$;+u_o0 zp$*Lb4t8|stBWfH*@{iK0{k-ALfu*+M{;(J?-=sB$~5rB7*R96zs(z^Oo2MVql7#)nT2s+Yo>O3zi5`## zaoGIf<~=qJ7?`hN#i?r9kU8oAMvNe4%G4AOZ6CX{HOcR*>pfD6k&y@BZdY@3;i5qGM7C713fIW27g9+a z8zwNg#E!vX3WLy{N@cm5_i5OCx_7HIF;$D?mP-1xVe_A&v{uHhz;pBKk-DCD_jUXv z?}*z+%HB4wRlU_3C@k=E0HVmMCx&w8bubjTiY-!bFIn3v4N{0wR+#^G zYZtQfR88|DbrbB;xK4|rl19alX}!T>8fq8;l}W~ko&!vGeFxmJy34 zZUi$I66C;9@nHoW6(DkX?I=lp?^z>`iUFc{qgMP^RXl$UU$7G_w*qQ+DHcZHLV+tC z&Ed}AJvIG0464H;Orhv(ueGy2Tkwnh+P2r)UN7yn9v*sd@!#*L&rXg1KDB_XBIZ@Pfl1RD0Hmfq>a016|6W4;{@&$|CpgXh!beJ1G9Bdm3bux zY_(}B4iLxLZzO0cz%*S(Og>_&I5FiWXnuy)*h#Hbs|)gC>{iP4%8fzjg{s5~}8Ey7ECkpKk(3A8WmO+ac= zfSf6SB(B!ISM%|{U}=kBFTJn}q6@}~Hg{5#W9)u>HPng`x`p2KRHP(qu0F6Qt-@~w zTYQz8vpTh;QT;hSbsZBeEq9(ZcC%&!hBk=m3e8 zZ!(lHh_mSo^P+vpA(l{xKGIfmv6oY~MWqMTJ1-Cc2${c)83sjt9ldE4#uE~I4Fo}j zlQxn|>2}&m^Sl(`K)SmW$WI+I@82&k5hp)~pSmCo3CJvAeVEwflY8+d&zGB=)dn|lt}i*Uhl_@-!=%vevgu8vFTB%jIj}aaz*r8svyYI~ zCSbJ%Bb0cQg^fEeX-ab7=+_m@poDp%TYd$I0x7?f!65^q=Hb_}va-LZm^cLP5(>y0 z=P?k2a2cd<`SW?{akyfzP$uU{9h<$|P*RJuoQq=~K;$;GA* zPw=Mg0xfP4N3n_?zWt%hKKaaDUy4n2i`XEQGGAxdCv(@0Q*ZSAbs^i^I0vXGrkGP9 zasPVXXi0aaOt-78bOKQ=+<#O@;tKUfN21X7)`0{>3%PK2GUaw4zlS)n|H0x_ql1LZ@kul#=m#VnOo$#|B1!s7*GiHeF+XhGpC)C7J8{$|SQO^r zUlXy<*N8o@oE0#13pt44bk(1wBjtSv)(Vu1(`C<5mJ%k!?DD$cL&E%JeP+CuJ(4=+ z?RtoKvy%&)OibP6Cgq@+c1JkYA77l!-)<8e)DpD{MZaHvA-9C|b;EEWoRwwg5NFRl zC!;-ju{x;U>uE-)9(L10L-h%pI4APCCt#r>7Csjeg}D8Qv@0@dB`4S7AQ0iON%3{R-~2|U+1tATr^cy4(zCAlc`6ui45@SMM(VQ-*o0}cb= z`V<_G1viM#9fsSm!%cv@5nvwx_A5X>3$O>D4+3x&0nQ=7egGU)fczFvHB0rRLp}TEQnP;QaYgEBIvvoCkkug`lhuY}_g?HP{c%0)k)sCH!poGT60Y z6D`_pB#x`~bU}csYA{Vzm#msX)(&$2cnY-b;OW5j@oDw_nYPdIj_s3|abMw38+LTX z8Ex3>?b@Jv3iA5$7pQeR%(iZaspU1yYUStkWxCaF^)>9s!8vc(o0SH>zT=<+>d%t_ zzAqKNr(s7o#H(SiJ1u;%gnAWN`1|2819*=UJZz5uAocFvw6LwcH_W#8(m+>xkAbIb zv#n{j*>fG;W=Ldw)n6VTxpW&Q2oY$XqJJ7i;yjrv`Ix@?3o_U7z_j2@W~$>UXnX+o zoTotTBTsu<&ZnySJHF*IoR1KQuL8w2%KUqsj}+=~Z-6M>V&;@^h5R`1r&Xh2hfjRM z;aw>RUb~fp2U9t(oyw!@m(&NAOhq-Hj&i6YD2EtuyOo0nQ#r4l%Bd)emF|I6GbCof|i3mw?$fc0p`8mI>}HqiCQu7rsIabKv-hrK(rQrbfdFm8 zp$E*`1(d6N8D2cf8If`*Y%qQ8ln2`>&&ya&1HC?J;pJkyLNL;{;D*}4E7T5Np^o9z z2uVp{x$X(n>~iypuJ;kwQD&W~W%Bj~CpCX5s8~w@10zV?QSh7{SPDKV=t3W<%bD~D z2Ee=QGA#g}SO{C4i$z-`&a!li%~>biqP5rQ0FXRw;LdB z2sZ!!U2w3wK${*?c0$dRv%o%2?N$tSv|?WS6$#a|3Dr=C0Eaq?YN(^Ah7J&uT~uYwC&@|BmUi&OZ(|wokRrLMb#ftrwg9$Gfu) z2@{c^YSRil%X+yIkHsPXLw$C`o-BZf8t-9{iw*lJ-TiU(j_geUs}_Ws8@7s<{`dEY zszD8w$29~*`q0~=wt5pI5bt}zK(>kw#BBC5ViRFJaeo1b(zPiV4Qd~NcvPGCKW>vp zYhN*6d#0C$BiYktGVTGM})`2wH-ZJN71^7`bG*{f4C8i1tM#}>?-E7by z9gS4ik0CuZcNmSI*}rxgWgf~eiEk^3d*8T=?C&V1mg!Fb3+sLME>atF+-Rhq;`P1! zS{Fx%*Gu`eCiyRIEa!}eOCYI(0H0-7++5`;ApxuNS{Z#ZmQQFu|;xIB6pq>xl+G zaRY#B-cJn%S;dBniUF>96BbjjJus^&sms}t4Qgv}j352pCdiRz4k*AtbS_N&(;30! z_PsM3p+IM5|0NcbFA2*1*;*7)g!0i4gz|hAbq{EmJ=rEx8n+_^lOQb2*5a=sgx?Z` z>$9xcgJmO!x?w2}b&`7Ft){sdDTDm4rs!#qnZaDPl+70fjBP69Nvs>X%LpaQ#OhWjoyM}{{tT;dHv5W0tlsW?EDZzE`gM39g;Ck6jE2O||*NJdH>MChi2_sgs zdm3+~vD8%$^t&C?d`;W!2IO@DmDTvi>$05lKhm&^2* zO$+xVm9I`5U4u&~Oc%DTEg^@3W&BXcR$es)r5yGUQEnwM(OQ6IRpGgleX7 z57lU}ZI_J-!g_{qzpZ>NeTCe zIXbj*i25==MCF%(nm?|inzz_ay^r-7nSsc}DsZ`pmmlN)cA*jTbPM}Z#tn;)r8eVL zuXkXt=s~g?7X(y=CT0oF1^8+6GMKR%=?fU|RjT&qVEZjHHAOA@f7TErDFOVL9}3}N zJ3rtC+Ny7$dKWpCZRAOPlGi=tH!=Un54agdziHlkn&HS+Z8HtbhrR5KK?>ZBclK$K z8R8)i;zQgDJae(Db7V#dzR7(=yoqE%UMP&#@-DGKv*z=uCmSvn)soX(!GtHV&StZx z|Cc`KmD7s}+$y|*YRnBMz!w4dLj`z{e2ksu@>f>L#jYF;X0JhX&QSH6R%zpaH@Q1j zImUr9lpEG!!#o0W&u#eBMLH!~u4^D5NrW2DN>ZTN0%-I&WC|u@zH}Y8Q%-1DO=y@W6ccZ4 z4l~7zniu{Cz3hrrs86w`Uq2PsG)HzxNwtz8@`<>w^_YD=gg5bvwQ$8P*(&DAiJZfu zk1U*W3*BG@+wbz!y4hG=6(7mPg`(J3>n85rlejH?kd&0GMO*ltCHr`7CE^znYui(K zoP7Hvy)6`PeU(EwLPT#_dfQ6EDwz| znT}~BM!{$o&}e_%>KyIiePke``85z)zY+Fs!rNGyR3Y%dk$Wx%Ln0t!5Xe0mkgs|- zaWrZzp!Nt+jn54#QSlOPY3WL%q=4FfQIVo5ZC1$qR1L@^x-`~su+L~J)oMdDQAR?I zZY<=Wna<}YFv}l~ziFl-=1b?dq6Y+6mPaYzcs|Z<=N}Iw6PsoKMeW+tw;wP6*w&vr z7%zW>vu5V_2kBw97cakm@@Gfnd;B!ZcYay#h-KXN}E6D`Gec@=d3b)92CJudKS|@nzrA$ z8Sm!2X4dI z`e%-ke({fREc-ed!k_nbb;Q>Hs7IRE`j73?NI_U91#9;aDd5=px9*dq;7>ReiM}dk zT+}=t0^k2zqX$)O(*wo2jP^QMmm1;&YTJ!}f0c@VKT(T+KU>DXhdouOMYzAZh$7tA zOa5>4HuJScK|&siFhiqr4=X0?+=E*Gxhr7(f6?~(|0%B4zj<)$e?vUY`v0jz>;J~M zefc=8Rvr%8&|0MyzDgMnByE-&ZyE=F6Z9agHGJ z-HWOX(D6Q#NKjsvdzWhTO1>>p!;zCVcBLwDRpovQt_B&H47=*4Hu1Q`gr6|LV~C+Fxs1 zUo8%we#YMm_&@ns+xYLOmH2P?=fUCsaBV97U;WS_{AbwlAK$M-`@@QthwP&u84sEV zQr)fYCyJZB3n=bPjpAOOLzEKYCzO|(l4g%0Ijy5mM$dcRl? zsl+HWyCzU3JQC3RBcZo!r}IpE%`bB|^}nGdGopc1MBAS)c}M4%1VIiD>f@se58DPp zEP#m3(-0fs$HR6d1z{@ZWKU&g?wfdrEQgoj1qfpc!ASh_|JZvI_$aDuVZ3@vIvdr2 z1i~g!0!9NG2ud(Px{_|GCLIk5BaSkn(TK}PB;72E60BrQr|cQWWpv)?=#2B;(;3In zQCStciDY4s018A#SyZg1Ttbi}Y^m>@d#kFevjFqvz3=yZ|NqC2N>|)e#@u;Q zyL(KzO!8)zl9rK{B&h*TI|%u&4!FHG(0rapHCdp%TP5OIH5YMB31Qm|rU&o7oU#zh zfV%H$rwe(dls)@gMh8i`0RQ9Nn#DoCj?FZtuG=7^!ygAAQqAGBM&?Mp1MOV$022VF=A7!Ob|-u$8WLh_Z3fU#)ZQ2eU!^>zlFM~CNw%?vU8qzN>+ z(ye|nf0F6f*~MoRS?y_J-ewuCaQ+Xy+g)W-w{g#Lf0Bq=`Jy9A(MSJqwzvK0K2wq0 z7?WPDIuo;Qkmlc}uA_aR`PxdF&+GO{Dy5De3^OR7YOttYJ)I0)ee$K0#e}d?2+3r( zYgBArzX&y;e0NhKUAlKU5j%FiNu2g_r(G?D#Uo`GzWM=gXrEu?`xmI~&yonxI{PR{P67)t)hTDDq>{ zRRMJo=gSkKMjf<`b|=|GBPTH$;@$OB2W2ISC|?~^Eh*(nKV3{F0I62>jyLtY-W^vYLZQ@^UI_C)%g(CDecqhyhyFHpN-N1=Pz+!hASTYnH1}? zVxIOA>*j)QqWS?^;KJw{sjGYI^ZK-EXm}wjoKP9XXk^b>jX4gczbVsQD2lJHH_4Xz z6k-lp+ejc5c!KF`k495Eq!BbFu~Tf0JV|NKOGk}`b^VnjnS?FuUDzWrg>C<vc=Ryj~crIlKIeHvDEFZzWGS$;Cu`x@1K+_%lS<#qb zS`499(z}z+C=pJz9i_U_Vh9LN8bJ8I=NMKr+)T+3CB>ASRH|m3*&5s?q`r{X-^eg3 z&-AClQHtOhIcqV^I?)|`#%dg%VO25YJuk~o?U*#c6&&~5hT8YR8gR6Zesp#IJP zYB2@%K@+-xhu($7Kot>vi6j7vx{tze&$ZnOaw4~n=+=CNV{on-oc+KVgDVrl(In|E zO(@xZn}RWjfbnkzFy23VOo=q&+l{%g2U51k-=gLDq-RbmMi(Bw@zNnRgKCHs?a8ag zK$l6isz64lsA^mKZK*dEodK20UNXg-7@}z)s%GEjX?FF?R8<=NOH-$zc6dqj0@X8V z844u0T^v6WJ4QY6y5_z!sxOQHML@d0-204vng^nH*}AdJT;-+aLAdS5C z5?EN!zVviG{RT`QOwZN>P9=3HQ2NIsqot40BZGSBMvHXuV@Mf49?_DmPUrsJ5#{`N(k*yO9yPoxX>o$65Xdk=%Vk$z44m zcQpobSJr*3?=h4h4Rj3kkQ8$ab)TdiLuH=?n*D)#Gt2-ATZHy0wa}YHA~)fc|Jiq zzUv5f{I3ihKe`jL>{7WDUV8J_r*WmJT(u&r5`75!67da9YWlBGS87t;X|z4}8>%yK{0p!WQ?BfVGZ=OCY4s;;=y84HiAZdBpb5JMO5=)6>y>4jYMyk_8X7fEiz-v2%A_y9rv*~7 z5lnxk<*cGl>?0?&{oUR!um-sQu@FjfsU?{X)A6k&YxZCMhzP>g4pFz;ml3*MY1o}O zbO?L%IYV#G{E<$lN+UvjcF2HGl}3bWKBQJi5h{F0L#Qz)2oJd=Ufai4?d?8Gtqm?y ztql^8`~;*sC`i9N)cc71-Q)d$^u&N5`C~zPPzUMB9|=fb!bjw9aiqWQIArXvNB%(j zE5Ss0bjq}9UAB{cD{lRzTfbflug>R@1-?zsJe8q8e%cYvp1b`u1h)G`p z#m^}e?FSKx-x#3yWk)|yL=FxF#omK4P_!P@ps1iwd|-oK3fM?5eR$B=OLJeLz4U<1 zxaMir+|hkdEy5+8Ma2kRtK4KvL7Sp(5&isN?-|{2te>V-4A_)8u}vw}oASF3(v%J> zS$SNz@@QJa>3N1WIi!a~b>!8Ri`FcJtg2ROvvk8h&c@IrvM4=HTX6gUwqS{&1*ZO5 z@a=(tTJYI{m=^3lptYb4%84Xe4gD)JBHj*m23>X&?g6X)D<+V2)MJ;fc6S&>A5BM2 z&#ux(pn{sKZoL3)85+!&55}^1%N8%)fGv|&o}l&&bRaCTxeWs>GR9KN6R@R?Ryz}U zQN`#o>0(!hu{6HPX=h@=q=#GB$Wo+K93o_{Li(~Tm3zZ-Z!{x-;edFMq0B;eM=4XJ=kA;7Y|q;8(W`U zukSjjPkvbCkVuuPVb>G?=-tHA-}Y1G_x~8E%76SLrpizMp;fsXK$le$x@f@9MZ|Qi zv+dG5Z=Q~hso_+(m;PrT$uXB6Dy7o{9+9DfBcCDNx#j%+8q|Z@B=vMiJ)jz{l?4+b zQ#U;S_r&i1hVruY^6vayzxpL8;KzS*q57g{cVDRdepCNt4y^yVf8hP_dwSP8Abj&L z7`}JP-#~$eIT-_w-WO`H2B7 zy=OJen=#SCwPL9D5ZyPMi4Z`&vS|NHU%RIk>) z!gTK1lVfZD?%i|N{y)`Ef#XG`;ed(?aZr>Q=wRju)CDP--t(>Tz0A*R0~obIdW zagd_ynNJN8GeOzMze2%BzPSIMRy$LrpV5vR(EkeEyhw^pqC%&9b{C}YEJo#zw zDb(>L5ybE~>>T=fCe;Ux8xHf>%16ukohX#uP*=#sAp=t z7mWitj&Ts>lKXhtX1lcHQL=BC>C$>)8TLdaP=g7jvFexv$8uW$HL5vTQbMW8jfSiCzC<*-0wqOU6 zpNdrdSK#zeXL1FGT@i(8M3iIGV0#*ba5*X3EC+!IJ8vDbqtf3e$Ex*_4R8;ou6tU0 z57@I`fOz;5ZfzMu_gWDcVlU(L3%YL_&aU-@9kDH&YHl6%p3xIHqFsfUqMj(mvNF|0-d&buGrr+!|K=JJ zWlLKoXK@fe50S&pnK` zON2yD719IbE~nAMEJ>S0O&biNUmsWq5ilHswtW-E#kQ|b%gU~H zWjVs#R{jM(m~zn_@yvQs{r$U$gIIKhZQ|;z4w6hb$dY7T3!=$PG0Ai&>(rv{LU-nh zMD^{l$Et|hVRNuJUDklr7P zs?z)w45kcf)6hPH2SKOzq1~zPPoCWM6U)^;JV)XXInBG(3t+?cM4Mm`xA1 zh#4++wSy`%XBf&hlhTm1SeR%Z#rrd~1QtC3QVaP?EeY$-icO+@S@CNHX0WNtYGy2$ zw;a{M+{p9?rd`^jdWBpddWcfUf;v~CosN}r6do5tRDc^5?P4_yNLZ!@pu>NQ1`<&t zSkUM)*;i;msd`W%Eky{~xwV4{JgqvYZ$=2HIQ@pI%larpwaC-DbvD9A!F)WP$n@1# zH~O~{Z3nmNSW7*ZL1Q$TgVFqVeaUDC6zem|2&1Gh{!Dbluz#~x$b+>KXw%Eo?kK~S zSIvg*&(a38j0`ApcUfczhiHK;@#vhT4O13z-KMh2cIoeu-u4^ir$tqdzKrxDekma_ zq=yjSmgwY%kS8H*cj-IMq_DfAg+=LDF814ogcu1VmY+rNG$xX#Oyp7(vtih+;}}XT zFVmg=9&6F`ijITCj*^!x;xrjOQKBt;VEQKMcOW8M35MPBjZ!1Ot>s+$8H{ z2HToVHiqaeeWE9o%?5W-elMGvG0gt$I=phybyC~}1bl*C-OYM+v-RqJ&%{)Ryh667 zPfAEA&|yMjqr^x|KiZsxwh+yAuH?+VIbxCnlb^o9kX!>s7hW5WP*BYBP-C-)sMdc6 zGJe;4ym)oIRRw7;Icvut)$w{hF!WpBjTNAXKeu}*Tk~==vNb=aYz^}D(g&X&JGVc# z0q-ABQFQ?s?Ag+LMQ8eo5!caLTbc~a6sdLQr$mg{tBWtb6rJDC*YBGxQ03QrRW3sY zXC!?`2B+vU?YgXfG(wVv>{KG8`)g&=tiwe1$)y4dQRz2`_j4$WZUSSa0mfH9Azbm* z(aLFtc{Ib6K8;#TGuUFK$iDnSj1Q??SeZ;@G>NF48+Fx~+vKzoIj^^s(^ZAUfM1nE zRBdaDX<31^Rq_FV&Z*f`DFY6d4-s{_`0k+_e-D$x8)+Ny}J_wMw7`W95& z5SajVvXn~&O{v(#R5&}Q4QVWR4u zOlz%G>kBnevYuy7Rx-3B8a*e=PQR05%AkZhgNlg6fD*ep0l%QY$MeE^Bu^vjOp!vB zSIQNWv(()cNMsH0)(h#cRo`Y4s%`t?BmL5egHmO6=UuhN^|J+3=9$-|nrp)4Z-Mmu zogGmplU_A3^3M6H5iLwhlfF^jLB;kvkmVEJr@ChBFRoQNzGTOZp=>@(xA0ghK-(D;%TG9}56(q(OqiUm3HST5H zEYvT8tD7vpI70|mzx)6HtiJG;!J0&Rki6r?1Clb8uF<0I8ifLAAhXwB@xZ>BVL4gk_Hm^^xGPtLZ zc`L7sTZmyA3%G6=E?y;_CS>&D$p-57%Rtu5YfoaNE!svOAFz!$2DBxpF0t_aXL8@v zX2IZq&G&<<^a$_EAg0N7J%nJbO*S6ha5S52H>z*c@+hu>!IW8;v&m{Q$ApV1gF{%b z*P5vX0+*Fy(*hx*>Be|QVhnr}cK$j{ED>9Gq>r^hq0PMO1Zk$7(AQ?t&Jw6zTCB@S z0o&wu+RkjXotsF@Y$MxAJD>a>+qU?7t!)o{ueEK-_kG*;AcT?q!$6t|K5v)}5MSS; zErNXDwyeyX0XjgBHYNQbGE^J7Rr&+yz5`~rl70tb^yMVVMmK}+{kXLLb`0Ev$ z$V9=9?7S9Itdee4@K)(KZY5hzG>0= z=tcZ!eT^7c8C&1eJtjtJ5)|`_w@HmZyHy*L+FibB!cmhi8~+RyPMLg((JKd1j}>iu zNjt3Q`h{*AdVqI>-w4`YD6I{#&xOe+@kS4o!KnCfzJXZB{e+@^hgUoX%Z0+@c!%_k97_^U^6lCEJ#YWN=Jc0qHcdL|?h^tH^F=6b1VcM>Tuy-5sw>~5>D^SVlqfG8+6u(GcW#F)9I_r;#wh53YZ@+pWq)ch1^IB?wVx9Hf z&{O%sbQZzXy6GKk@g8VZV01T|FQHb0hE2NL2S?gaUu+ebi4@M z$ULCP&Wu2r168bkax)n;r$=lTPrlQnB zF~ZmG_uGQSCfsV+D&^R*aZTbLUfiziRy*_KGcBCu8-BJAFeq5I9TA z%dB1dys++`J4j~*&Azm}P&?1QostXG z+5i1yN^{;a$z59BMKo``^m~}1@^Ium|D}=^KIrjd(t>UD-?c7qVZ}4>ConE-^70j4Me&J_k^%tnZQtK1AM`2=H7- z$(l{N7+#ERZM%|wM_x<=azOYGcC~F7WUg)5KqLZJL!B)izNMc6P}0dEdM~6&Wml43 zNPu#tLw5OSeFI@l^fu4s#cz>Bh;7126Z-_3rk3`{5g2n|(p(B~uE2}FB@^`lc^Lg+ zVAxa=#y%U8@8QMt?Q4=4kI<}$o7N#8EF7GX)BKjnvm4NJlQ-Z#2#>@%$ZvZG>nLq! zQ*9SRF@57wQ-(vhmL{FNr>2blHi=MQykP8{->?q%`lM@K^_jTE&VCOR7|r? zx;moxm-FSeyWj=Zhl|NivSZb>GB>cjx35hyV>&LwKgXN1iCx2`@xj>{yrmUh;~-(R zH<-KyIEgGz!A@U>I4jv(5MG=p7TGOfZ$W4=;Qyv<`2Y+)P*l>;oVz!;go_l2ewKjIVMJCd7x9llr<19H9|mM zOKpf}U0YE)pyd&~{7+==y7r9>b?sA{>8ywxuo`*Zplu|N_O06ZfT?p6Fc#}^LqOU? zgc1#_2!o>HR3yVsjQ0GzbjVveLoLcp52*z1#qipc}qjcVgTen{(j9Z2opb_nP zySTNjR=lsBNCTAg&q-&zM9dZkpW8>2$pn!AWiAew==my>G?6VDI;{hujhft$M2qk?mkp=vTo%=_IF1Go|WxE%%T5UwG z#NL{3puyUj%Lhl#O0h!`R~5q#9$FOez&Io#^}QEC*vktIcD~??#b5VYgd6%RYY;6f zzE4@B&s&x>piXxXJeQ-zlrR>R35f>T;n~ z?_J!gV|PF3yO&$laqok7d$?8Y5BlzYz@60U68F1ATlqIh47chy=l3miC$)*YT}f@L zI%eMepg*rg>~v!vbR#dMJPK7RoX~6?*FHmG zT>)DvlsG*UXJUNW(-*{<8Q;azEAeN9?X8n%fa%LzI+BO=!$bh(*;gB?k|4VYa_&8n z#JGX@4_|eNVcXeNCmD7DvlwHo5Q}$A*CM|fn0pWa`Kq9MMOI!50SzH4?FOI%^Cfou zmkFdqGvukoTW;+7LupkHHrblrrhV09+XQbYKVsC#s+;g%OxU@= z`M4cfafNaI$)fF9$RRY_n`{ES;HgljTgnXB9)&O85TQA<$#x&S;?fYXEqt3!{CqSO zvVtzgh-i7CX|`9ok{Dng8Y2IB6QgYlY+_ z6Q-Ye3!1Z`9Rw(j92mQTx3X*ycX*ua>doxUyTs=3A$wAD1^ZyQgRN|4D%i?4;7miF zu?;|_mR@$8V;4c-B1qwDfM-huE5rNVFe=`9gpfnt#l9c%T0&fPKEu}Fy&vwHZ2!F= zx_QNqJz`ibh~vdg>g&~5X2NlHkr~Ph(=vUVpj>$08^WmwE11J0gdOsSP&SYd%TBMQ zg{wCKS5@;lz@BgR+dY_(`f zYM-fsy(SZi+2yl&EL-YROLMmPzlyf^7KF_XY0AUNOKAHJS1qRBe0dn!NgEbz&D3Zc zCqTuCSU@Uf*qQv$&}Ea+*#aXr%NJw+__DeDY~Kd}viu}zj&CqDrQHLC!8&Pag;vV{ z4!{t-89_rg45*N?M( z3FL?s{$fITw!teN;E}a$2L9C}b{zi`aJe4wbOY-!bF9Pb0H({tv8i5rJ0N+3L4FX! zsIN02q7;iyu9~`gDC4oTD2J-yytBcGKW?w0^bfhNoygnx%mq!)WEK(aM1;u%PI8>hVz7je`t{ zTy9n%1cLiaftHV5F%0eCZN5^=%k19uL-$M2>lbljP@7epN*6-%9AX1sG9=5l-5_Wf17Zz+4Q>gpgba$;1!`>qt)YzlkNP-K%xz2xEVm z5LAQ`R~77nkWNg{ z<{c1|wO;@+sk%hm#EW|=9XbAIUOa`BRgixM_*C@89*knD?`o~TOxVCE76QU?D((TM zafopTD-fgopI|R&T`+_lQ?0k3qSB9~?QzgC9zpxeo|L+xBfW_lEScOmxML68fi7lMaqOdo&aJvsx=!^W`5H~jLPs5en7zl#H zx3I?Zu!y5{FHA3a4Hy6{YyaFIdCjW6y*FJt^2;{_u{^kR$q=5eR#g}x3<9%rrPt}{Qu_pQ_U~Z)u z!u;P;47B*iV9_j%Pp0wyV|-qyVPr_y2bchf%)Y6-XyOYX#jU)^fo$PQwEMmA>cCfr ze>%Ko;A;ljlL%)b8gBo6-r8#Fx#bX|^+bbTGPW`AToo;IdTYC_>a&ZZd z#pI&QlV6a>3J>E^Vh?XoMAvDnx1gJS{LjdGxlY^hL%?fK!iv)lU{ar^ugNCgFo;ej z%6cBlM;`H0rL{J+?9s=3s~%;_tA2ja54_iO+y){h(NZ#SU+J0sL}ngD^)JQkO1cSx zMek_`RgUslwn0HsvKiT#>zrP3vpDCp!y|5YiKP(v2{~kh80vjfN#A|M#7Ki6wbB^5 zO2L~=k@v{f{*p{ie2n+x{20L-{^Aba(jvOH(w03=TDBEiR<+zE&b4yF;Y^6-M$#Y(TB~a8^LNWfbNX z{z!o0iKj-fRr-c%UbAA#y$}Pi;u#Za>u=@7FL?17LOBMY6n7};w;#g6Y3#iVqT1fy z-lHfIdL0bdE`p+jZiV31NVGq&2S9F?9-(!`OQ_Q;f2ko4<`+|r{(*L3vh)U&A*8gy zx1!@0?d!b#=pzCOBL}6)zk_GX`d2`@mg-2+zEs6_7^0O1;k0S-?zc=bt!r{f>ttGv zIQo4^C69*T5Ha~ywXkL-T{viBl$4hs6h`uZzOQ-?x|!+DQ(WS)Qt?n8NT(981v|z> z6d;-HH?VKNe=M~65(fPZzk*wfI}{E{Yb)-?=C$I6Y{6c(hN!^+f`2I0|$VChupzYw+GIm6iiw>K#LAC*O za+Z>Q{2*@0-RvJi^jbVPoZVzN_sO3_5>7k-fDZ-05rb2BvB~p94XINe2lG+;;K}g> zsx`a=nIXql7ttK^d112^!LIcvc%Wz7|8f!#(F2I}vw(P!JnR7zc0~xH$U!>v1#zMV zy^*uCehaD!LXQMfeg%bk@>*v-@hs^*c_L&IA(Q+zIX&RS5II2Lckh72cap^2v!0+` zO(pj-l-%bbjxHdlusRVW{`H(IQVwB1Lw>kF<^`dw28X)&mq7mK6nhWv^_< zz8QPYGB-m3#6AL)`8F-{?^x#Dy~_M~Oqox7-nUG{zNh#Zlq^6o;z=m)VXPvaR#Aav zCy=t+dFnQ@SxO}?vUfu#>_MM!_>w+Rufeb>+TVE@M<&JJ3n)#6}eAiClP=ZGZc>{Jn=!6A5>_3LI7}m0B>e0sNS9T%EOv0rAq!l1h zfCBG+frNW3L$-#4wwHDhqVC_P(HVfH?ZqU9fFW)o=n9y*yP411#0PAvA$fy(CJ#Ne z5IU+dgAXiI^(aJJ%5GE-_z*WSkzlSx5m&as#P>Gw<<0BuWDiqt(k)ubwV1MmrhEfa+Jt6B_It(SeDEsqMO3?)_yGAS&4tdS z1a|60*zL9zSXeOqub*O>Tak=~55;)`6XI~u4tK+k6=XlJ9XPkJ89K|nG_ENQZ5r?( z^9Y;gd4xvbO&jO&!Vf0@@9AVUj-*j<0;KC;%9p=OLM@(oyd@kxFOwdf4J;%DmKestdCBHq@v_y<$qod`FAoQn(LR%~Z=;c&*^#Rp4F($A+OlS**9 zm8b{ZWJ1(_FpZw)!eT0~$Aghgra8k*jK0TA6zS9>=Qd!)c`;E!U!G6lkPd?R zia|*dZoxwN(aCoBQI(gw>4=;3^t61sb>lmhq zNhYhUd@FD1%v1Pj>+C>VVL4gVgLY+?T46$CBf`__1UiqOVk=c`=B|rMCt{ zSl}gmcdjwck;thB7j_#B}7cVQNqat-!M+-PVuExA1&|P^aJ~B zI4@Mw!af^nc+1o!IvbZso?CRHn~wqQqDnK00#D#HAI9Q7!DV^u>)W^7a&6kAZtFB zj!!c&>+qzxq9EKCn((=yQRkW)saYF#kJ~E_fljo0rWHB-gSqO5GZ>$ZQ%v$joRFZ% z8Su)B7G6Pn>=s_M*QJtAh7+bKC{Q-I3-+zpK)M`o9SU5iPUaO^4rl!-tR-*%Ky_r1 zIxeDhIN6PRqxDn-NJRo;j;ZNAQWLhw(Ga5t%LkJ9Knc`a0zF%j2@mvg3CP|hxq*`W zia?1oP*PS`GKUX{^T=_sxPY8qoB4pS(5l|+;seEYRj0ie#5{`^N(+dUNGfban5o#I zD;ZpH-E}2SsJ`f5tNs^Uuj;o&`L;$bgRnTj*V|Z!Sxyyg&%@#gWPcFru>JT#5_6ng z$=T%_BNQS>nT48tPf}p>wn>Y2p_SPev#>cH z@zxBFSOwGdOYLmGPkV28X5jX+{j_&#&%e^%<$8Nj56sZseMuM8-qQTu?OnO^obA=* z8EDcec4LC{;%qd=ayJMrN&~w(NqTlF;UbT*8@n+-sq)}(r4xXdnI0YF3nfVEz-d5sVZq4$^&0y{zg z+=+Q|2tkOe{my3&TtPGhuy&oxi7nV!2oZlj_obK^IYBBOPd3-= zz&l0$jIzG)2+-O+ihx8!*noe{Akw48zz#s2iGQ0I&j{sZ@JwJL*JB2yI74_On_-`v z3t#ocoOsiO37LgIUCQ$8#$t|L?ae+Y9Aag2UQ5urgRE>Fjl@>KY1g3?QJGvyrkYnI0e zVGoA$}$9;oTkivv|RL@6@OH* z;(q+|vtC&7xQZ3-R6vAb>PNR7J`(Y0Sa_t!fEHz^6xAJ7KWIU29tW!OBcC!_Mv57m z`(Oqe_zS|MVF$6Ihi&8v8hyD!_gLRJp*zPnLg=38V};9%x$wc(un|uvF;1mRgN#E9tc?p(GnbS?o3itv;#Uo9e1FYafb< zRJtKguk>9@v{Fr;M4M7=?CRNEBR7Xx$5dk}_`jkyxy*N0A8IjhlNaUmYBF7lcqIg@ z5y+AmH7))$X<`(b@dO&#tRF?ub4dcxP8YkH>jP3zZL-t1kQeJ?a7pWxBJ7n}q*sd6 zUU^)_B3GSB`%vJ!NQG+_z*R)ydTz1~*DuWmxb%BA^&D4toqu?z$mjDl3kw$h&S7g%A-!;+ zZ*k#$zB>ya@LgT#^NlZD>Kj`4@LNDBke)Pgg?L&LIY8fWS2D7bz9Tp)Gs9(6=Re>d z0&RqF!mrIZwGgWo$ zO;z!bEWvO9f8M^2AE56Wa`nD{pZR}%|6lF<3SoJX*7r~J?t43~Gs}y#?!U*-{jMwZ z?w_vU)_J5v?|ya-S0r@980^5+q#Fj|1M&i+fXvW(bkb3<$+tr2zR!36`UmLOG@;vv z>(Wx+Frk}_$tlB_d~X<&gN8BrMD5>-X%18p3z?0Bvc@nde=!Wo%l%~1Muj-H(FSI@ zHZYaiz}$la!>%y{+mPqas1P1D)4{PC2d4;Pucon6^jLj-vMK~=UIkuN7{RVN-w>U5 zj9du=kI^d=^)dSG+5b02|8rwR_wF7*Lk_;I%$rsHid=Rh2a9_f_o0b&?OiM9vo+1+ zm1Ap&HLC7nU!KtIVt=`ve9dHkxq-agfDstZaAURD!3Xcp_6FZssNNvU^orYwsRfVtu}f@{ za)v=WOGwXI5U`AJ#LU&~@H@Py#@G1wT)h6h`fEi&UT~{a2}2+}$}lTNip3ylm@$i% zG!v7Gkd&hbhx3rbU4R@e+GEH?#Ss0h^M5z)ok`Yk@9Fse67D_Y_!n^RVYUzMHKh>T zv)cc^h7 z;DZrY@UhnsDQ{(K(06RXX0~Rhij$KGPHxk1a=nU^KPzZF7fOy!(ok}x?Z1SQPYwDP zQ1YAPJ}5aQiJ)XKUPs9XbdV*xnDy0#^y}kW@yr9U6Yogjfc*Ox4A0b{xCQeZ-=XRn@W+9tN#SC z(RN5fT$xo7cT^N{^|YU2+O~r#=Ax0Nylv9S)p!a;HH`B;;{EVj#m8zGpBI-mFa6(# z%d_noF27^`FX8e>N&f;ak4)@?%M0xUm&4Zo-{SJ(0~dtLa_WWP^0)&UE+-8-A1j8&H*8l#>&mtPxNKD3_WCU98?I`@qX2?tLKp_BlZFAw%R zFnlJg>cH3&Q+sIQCb)c_Z9dtp-NBc)O0T!UI1r5^K(LE>;X|r~WaGpTsvtd~svy~3 zMejt#`4B>Nme49i6CWt%(IPu;(yBR${Q=yu?O+|w?hZB;KK-^*u~|;^ioe3rJUb)2 z_$yWO$)Dg2K2P+WQoQ2xH2NqS(-BTGKFIL|4U(Wl++Cxl3u;?<(ZX&l;Q)rcgmQ_+ z`8fadqkp1fS1N9i{yfCQ5EQh%X7UXY9-;m^Zi2E(cxg7!)B0lCML*Y4`O9ftH}Sj( zKdb0byxmoAlI`3zA-tcosoo5o)lqMTSD2EM<%9|D*A7Huj`89aUfEK8O|Gxom2dW2 z;G^CIvAhzlBUzyezB+%RKS?ObrM3r)d91SDJi${qZfP374s9u2Oa-%Wm9?>$tKx+g z1q!GJQP1BMC@DkCphnEf40;wTvNFhVg8@UhjYx3-Pevhu2LX_);Knvqbj4uHFn_ul zI}<2Qls^xUVkDQdaUsaJZ%5<6M!oOM1Zx3&c&sYzIuM>KiM zopy;kNv4m3K%J-pQW2OGg^irg)V63a8{pLKlF)MtZUbdV+C)#GyS~{a?s5l<&j=@N zEARGY)VBD%sQG#m{+Wq?rVA%c{@XmLXupZqkL2*p_bB`|`+n*v2>E76zZ#)$h}9h> z8)D;$^?+8H+QpMUB)9YAwL&D$_ZF7?7gV(U5BcBncFC^-3VThrW&&^Xnh8gg7x`}a zmfoShIKCXhIgBjq6B*C5A40@f9Bp}Yf6*Hu!pg4ZyIl|qw1PzX9MEmcllZPSPH1+z z0>!yF;T}zBDyFt!Q-$)}E`nauit`_{AUOdFV1>{FZgG>RU^mXqoyc;$_=%#(iS+%) z-FSco^D-W47%e03Q5*Efh)aAaVKHRUSBWxekcX=}SWPA7TOEtzzE2jsae zm1v0%lg13!`#U2-`WsaP#TyFO)cmaQepQ?UygvKt^RG zu?Qxo;i#~fAQ4B!ChtbAEkQQCS!e299Gy2S5%a_m0<^yqZx;Qf%eg?5?W@cBw zi>>~K?7BgXGo2>7F7$W*qd&eyd?;V&QFLBuSvMM7U&#lS0}{96+@G=wz4Yy=L3Qw2 zg`9~7Q{ofi0n6P4%cY7_(1u|Ui8dYWhKOU`*o6Q_ptwwfs^5DrMtGLz3yKn*wUrt` z4*@{k2+;1G$V>Ug0+Yp;?}a)ahB^sctovh9@24>#}-$EAg(WT5{`rL%)f zjJG@lASn+d;a#kZKP53-&}kOBO-qKsZ1jjB;qVM{DW8N>{wytF zI=Sl`A~!iO;UP$v9PZ?%n|x`~e+<vW@$th(Nvs-ZW?mkkRH3tfpUZ@ zL{JBfm|u#sFpSbv@JN~eFY=}2nD=6u*aJdGaT&iVOiwjog~UwK9(wsy^J9_onxfG4 z2)ejZV{5kcZP(!chISQoUU<7ETmQLsnXNJHy6Hd9)h>|BRkh7rH_;GridP)MdP=E|Y0g zm)VkNP?w1#4J60lmJAljaKQ{Op;O>L!374@b%qvz^Ss;Z+gobuKj?R4iw|u?3#1k{%Dp?#G3%0H~Vcu zk`nEQtFa%h8K57A>HVD}lXf&4=boqaQipSU+R2%t9YN*gS{)WEjM-OaHv_n>) z)##IW1FT$fiBoZW(JOnbY6xe#Om>sIA2awI#NQK`s9-bC7PWiDM(Kt36vZRP^QK(# zx6%PW)#$p+YRZw5`GONZ+5%IaI)p@vdus0?1YmU7x!W50!C%AOYC^@l;vuxRilARh zRJ!{Gy_A4H6G8WCpnC!IlHjPTy<&sZ_AY{N&EicN_~RuFux6<(xDw<$(gGIfk;^ny zRYWIPc8hu=-+p;{gU$@Z?!& zE93|tbVIVa&5(7UH(1&X-!Oo4TX{5o8g%c2?@V}?_#lJ@u~>II-pzz}5CCUDn7_%~ z-O`X|V#<8M{0oL<`yd^b#xMR`VVT*a!=m$DnR9Pekums2Ry(zW=^%F_Z13KouCBEx z%enKu(rzhs0KS$2^PM4G9EEiR!n#$7yk;VDoX>{0Yrf#c=g74k;`Hm0ZDdyvmn~PY zC{xN?qbQ|f7f)8O0HB^Xa97p>PB>!rC%WqHf^`zOHd~`BYaumP=@qvk&8@`6xRMuZ z2_X)aRuj7+rFeytnBhmNoVFE(yV7du^iE`2_hCseNrMN0Gm+#rSo{M2j_cg zxnF+26z1I++#I|eq)3~qZag}QR@3p&QA7E@oB9x?XKN|QpqEIwyT=cagHHtvYkSyR~>^h?Jpn>;hKkr2sizJorPBrK!G{sHo=gepdry4YcrkJFD zj?)v2QA38%1P=Pi)}n(W(jLneK9AUoTb&^**m1q3zdEaFohO-Ik6YTpKmm~ z+H{RI)jyvai1Xc80|4?g(*LDrL$&(PvtFe#Jwa>gI0ZP!>D};WX^7f?OZBfI_TP{q zvDXKa9eh`-XKdJ0+u+-g7ivIzc0#j>$Ooz=bY&$u@vu{`wb{7!LrkL+AQ@*dUl3k= zmG;?=iOmqSB%y9X5MG0|cd!ohXRBwL0U4?;8Ymr>D0`vWb$M7? zSekNqGxdJ4`yj70lLipySeSlJmc4?#ViV*U{BW9oFne>Tg1wSJo!=sH=aqL`HJZ; zK_VaK=cOA#N|QcM)XY_Yw6;T$FU7mrQy6mIL_-Gg0o%$T@lmt@L1~c=?Vy)c+Tgd( z-Ryqk_T0QquXs${BabMY?z7-|4BFgKLeBgtCVv+o3~y;hy@@hn84RT(A}8qzPHrUPz|Rq~ zD=pil`edkW@f8>^FVid8CuL%7>CKOkfm#WxZ9I1Dw^;SReN5}=&pyddsCvm&og!M6 z>SSwi#l<^Lzar-sXc!+YRq|wWE^in7WJ5E&D2n5vproJujfrk(`C9Z2x@?Eur?>Vk z12X5e1kgM*wIjIUH^eUCB&w0{TngISk&bM~>e1rOwG3NZMa&BtWJ#bGKWo6`8zCLB zqOIQT0MRw&8fc5pChgYUR2kxi(zrFzxX9@|ZeD5Wa^9D@enu z2^W~MvtD&EQVHJ?%UO~?FXRoTD97T^z^N!)-ZC}d9?Wpz zklANng*w;OOJ$H3;2kOB(gSE2X4^Jn+DkDl8zp9ZLF>}5APWR(1QmmJV$P_aC*l)p zuD*{!y{PcQHH^FdkQ*hH&-~*hG?b>wC`AO-O6)i*;?TfGz?2t?`~enU+$>Jug3cbH z`;w*G@bD>EdQke!2WVq02X}DiO5VWF^SSy*8Bfq{as?j*2rRE}q9ul1S zVf4F2ws7J{9I^d{K)6Akak+!b&GNNSQN4o?@+Llz6fAn}wFADH5bI`LCwT?UderRl zjmS^+4VBQcl-h*~Od(oW-YzV+tIH*Zf;Y5QpG;VKG1Rz22)D`w&iZM@B0Lo5!ULWl z2X$EFZBa-eT@0k~O(5OI6>RfO4(aBzI=E#BZfB$2Kf3zSZjgd<*qYPC-kSTMCalTm_G%9&ek{#6 zOU|I~lZKK4E+H~tqX~5&ew1&6SF`U@ECxqX&)*B5$c=4z_IZ;bM$4CzGR<9IXvs?t z>NYBh?KoBA&>&jCE;br$j-tKMM~qrj^9{BV`sxFgx%0l`>W`86aLPOFa+nfVpV2$$ z%58|QH)ycM(##M4 z^iE@xBZ zT@#7g&kE1sTR|K9_7fDT3iI6TQ90`&zC?EhY>rTayq%?9`4gkA3`pjIB(d zv9yX%WH!nB1^aj@#-~i)U-0uVd)JVKlTr3w?&j_Zn-P?m|3m@?Bdo^ z1x=)PfMYDsy9y4HT|-G!5AgRr5L{8a%jc??mg<`!rJmF$Nn#5{LL>gkquDjXdl5NA zUfH%$qmE&qj@$doJ(^`sx5c+8}^p{%7upY)fz68-v4S(Gb{xf87DEjSF(NB$W={W%X)UaF< zb{&S@qlJya5-9rZ1N2iv<1mz>-#$P;HMAYkj~kC+m#Y0UCbGX5`ayqg1@voCgp&iH zUkgD$NQnV}evwRA!?=PLUoI|RIZ5BUVf9KPs~3zAd^%H&A{S4(l3j517-0}_hRwJ{;2ofsE?4w zVPPKNJWI>>dc=?Nc6quwykfJ;>0RQ2iaCVr+5uR-k0`{6YQop$bPBQ-&>~wo1RAaP zUE?ljs=A6xI$m0EY%%M%b7F`u*aVXXIKC8*xW&W%?PIst?u}*qB9&+X*)%qm_q+AD z-i|p9i1RVL-#5K@zrAO1n|2TIev6Ki`QnpC9FNWyBaM;pV=#Y@tJd{AJz+=87K7gd?3{u;Ow}EXYm9a===y}(tOBEY>{L3eJJ0z zs09G~Aj!v~POOSBNyb^{{#;th&xBYGZ-4nb2*R1XTeOMYyHly=G+yS36taI1p$(LNe_xCcLCY#$o!OCOW&{y{gReKTHDxDidD^GZ{7;kBq4j82wkAPUhP zuIfS)h88p}Nv|%PHr>yn$y{-d)I{}o6R8#70!H3KtX+n9OOxiRE5;uE0JfNNONrA~ zp13=v4cLAbnw7U-Y$&bmq@6QFdK+NgfPE9lkY4`~7^(+7fs3d|z}ilKBA&Y?<6Au0 zwv^3c18y5(0(0FMkGMm^Q#LP<3JA_qTgK&&!j>r^PaqEaVI*JBxFmi9&Su>4+C%I! zzfU$O2T{nL0La7!Ih{DoG@M#Y{g4cVZg^N+f#;7tZO08!EfP=t3`dTu{)VI5Y~{or zQQRGA2MTg~+J|CPb0lutM2>pUax+Pw$Lt{Pc>QTdyqOdjpAjlLYdw9hQZJfEnWpW5gi%*ex}&hLP$$JR74my-xbqdS!> zhTdSQBQMnC%5q{f)R4)oML6r+mxx8MfYa_HYtIls*r~c|>cd z)8)uF`4S|&n=I~=2T7lNi!&du-3wb4k(K^oXBf?2xHi7TRR+?w@}0t|9)G&D5;6&y zjP$&oS@zYoESp!5{_l?_F;!)}xCN^I?Q-g676_`QNlGBF0|XO^EzQ83tTK4!d5NFV z;2bominqUlMETL1#Y-u8N<2~;q*gdyuTUEEKH1UL$Y~5t!{o_ASE)pqEN{OIoR}>M zpdr|S{>KxXHEVGV^d}aa^pEazE;IRtNynhvb#$|LpY#>_G%aZGg`uo{#J_2?lD={! z*&q0;lKvjNLBH>i7!tP^n)~Nvw5P=#eE1~QuoaV=8(pdS46w<`XcO%yw4zywlBfav zkK(nNv@A=Z**^mOXk;1K3ls`o_4k`S&Xg)sMMbLzgYVc_^7Ju;hD@N8Ir&Ym`K>8L>r9O zNKg|7>b&HQyazLg6_r+0EEutBi{uTUq6BAxnYoU&68K#49??7ixPG8>1up;*Pmm{lSLr*f+{|3zJYn&27?a82>{nmq(u z5x_0l8`aBnsh}WQ06aM#O0{WFXgJ*q#GvN03-Y0iC`h}3^z!j_{);G<8*R}V4%_H% z%;r;&5A6yPw1kM87jvI=@KX$f=ed%CtpXC$H4_r-NG<$(il`3w!lm4ePaA(9i>Thm zFMgnS?B3rS@z_V2V)59Q|9;?j?0wSMnD?26HZ+dE%{qA#1;(8yLm4jniPoW@I!>%gU&}XK=TpcIgl~xWXZW zSqhasjr>UR$Nz{O74joVM^gAsjBM9gaQnv3vU*Vm4u4fsBY;JVO_Fm8D zxT|=iC0m_;WxOWAweg>NJegc#gmBVMDxlfR?K9{}Aiwkwnb(*OQZ{eFgg&-p@ zEZU=H+V^-yZ}m z(iyY~%C+>fG(~4HUz!gy;CWM5z`A#YKTHGi@gw_~zt!LgvGgg-y14Ybg_U|ps}^N1#Y>M@(>MGpN8cnlInF&b@^i+bg-o(A1nPUD3o)`IFu zg6(W;kWxy`_~7!ZDTNa2il*!JkwV;Or{NAcj%h3!jALpvEYN#AcAk$8+x_tsqX!^{ z#r82j1J+XYL!f0KhE+cXVmJ)Ma8eHU0^ScJ1f88yDL_9BXri=GSFh7P{esiPw~}aL z)BrS5(yr$gc!(rAWv>e=6HS;uh||QbL;YwX7D+XcX909?Zx1T9|HsQ}b%)}Nawd#m zqJ3~HEV-)7fRoC>!JB`Clv9^N zcQD2CF#0Qp{=RJ=JP-T3lvC>LADu&~%vvX`1@iD7bVP?~7bN<~dJV0mL<~9-N z2jfC9y2fOp@v zof1BnHj`!Tt`OF`%%#%;bIoOJ@5&F6{8yeb0XtV+@GtySCkG!lfj;yXUGLS^5u$yU zN2+eFuBe4-)0uU5d4pssGe?Whd6yC$U43W;y^9V@e9gkI<}%E~mcAZ4h>mdj=-Gj{ zvBm1sxC?AHfoj)~$UlL$aGsIt@WT`251?jjAaoP}Gwe)#ZBnKwMV+t#H!`&S3(0FE zok)NC1fD{2W3I{M5QB1x_S?UKiGY@%w*Cdvx$dOf`sBZsyY|X|A@44s^63CZDfG3g zcfOqNIjO9V`HCw4in)h^Qs!n&vNvRNSz6?-cBQ0E8Q-CTjXbLx>fPw;M1RjC+3;Kj z&sQ4H7s2yz{+QwG^lSiGZPx4YZ!i5lOMlPM-_!K>6#ea?zuhFO@@~EblOu$9a)gN! zrj1E}@G^39kY z>f3E;#_Q$C?=_2h4PGKnhgGgxq6OP9xxpI4wfDtE$nAM@19|0UgzC;aZm-uScH{FsrNDjVIfJ!$EXd&H_O16C-znP31rmdt!u= zvyAbY(g+hDCopm1B>&QuN?%5`#=q<_zPw0I+YPWFDxGDQO8_fSJ;ExidWPOhi|1{o)%v9c* z=6r;JX^!`V&^QNa=^*O> zcZKFS*)fUs?b!)EckM#%o*k4y@$NahbTRF#6yA&Xj8i-d@EUs|EV2VlX|Mu!(h){n zg_)==s?bawP=@f)w8+k;yF;^8JWSGRVWj06xd=*BX;(NBi z)V+;qrgMDI`Nkr;HM8TSa*;lGO4}AC8%-?%4JAXHe3<{5L_yZ-iDB&+C@*8=$uBXy zhEYPh5Ouo5Pve}5_84Y?A>zn$`Q>~d1fFK|lpI`?hdSD;L!x@jjhv1IJ7i|lhHl}f zv%vl8x1n;EM|*KM-AwgM@uV+!f59>gZo+!B`=AX&b=|=^g3ExzrajRJ_FFT|yddMT zODX7|co_hdB2&M8*Oi}|h#jY%zBH~jAH!3t{g^Ae24mzu*rDseJ1+F+Y4`mBB>GsC z6@TkC^km%G)=EC-oc%KNdh*xS)>87ZuKNp$Ewc-X!>3X)M&CF~Yo-5|46X8I><`t% z^?%;9JGJh&*&oYYhY&mAqhst-N8O%jQg0xoP|xkX+ej{eTcKl+=JPAdeqFE+jgkxD zLBof^WaRi$H+(D86t%H>|Cm?5mC3vCPu(^bUZ%0sU&E_P|Mgk&`WfF_Rk~}-1Cy}R z^-DMXZse8k6{NXXixQ4xDqV*ZP!Unb)MQJYywOx0zU-ozKj@Xi+927|=DIhe;i_H# zR{6sp|9kb@yWB-ha@Plyl7FTR44UEc{1(|igeN;gETVk-*#dsm&t&eG?2IU2Zh~)I z@nY=$R%|;yNBgoZZv1}!dpg*DzEcK9#k7a5<@^~1Fz#k=h{5Z!gcCq2U7BKP zmBXio)BdP-oeaO$>pylpL*c^htth(*jI37`sQp{R?nc zdoMW$(BcIduFas$Ej)=p7p`s$Bh!XSvMtaKg@y7MNYECt;5&?0~|)aM=$py+jx%;qM)K?#@bNkFa=li25$P`*Oqk zvgJz|^q+$+wgnRh+7GBja?NMzrE<{$VVuJhZ%ccf#_N8bBXZGM z#jY>?8G47!GBy~E%l>`C699k5Z8cav4$ju!93Njkc=iB8!qKopIE^Fmj5=4F&>El1 zi^2Miw5IS2yq~yhn$<(H_f(4T_?yVQ{l6|Kr5Gx&7ZlUKZx$3{=n?-b1^M*vn+19J zZ}4H=CA@mpW$6J88FQdFDn*;+t}lVjn86IPExpXmb^sz2wp(G_w1;rB)%%%QsQotd zGLtR%G=C<0^V4h$J{FeSK1V%Pc=b!g@`ax3xf?ON3#t)jADck<+~u-=qKVy?LxceQr+sSdy0_QO;ijEk?V-nD?L$L961%Zx$T zyVP;O?x6b9=db^+!t~0gW&6n`<7LZxvi<$4QPk**3us6n;&`JUK#`?mp@?EXu=GMI zvf`W~MQ3Gu?-Ks?uGrU|^mPR^aC)(W5VIQzt-uAkchSyDqR;ce2gE7V+(~QZWaL)- z*8*L1RgE{g;;?qk@6(kEU3D!UOVoMBfTp8=5d{$W9cjxI}1jv`T+9(d5*xA0<#kQF_dr~mEnu< zYOieR)pMWy0b&mbbh{2}4_**cM?3@u=K-bY{b>61;G}C6b`)u74IN1~er3!L7#Xmb zUDTtVJNXC3(lMHrSub77>xr)WsvOog_|7>qm3`;UVVyW@=l9HtQT@o-j1!Sb?k||j z(-G)KXSkTwG)P+^kh^9!(oK=s5!fkr?HRRSdu6i`Xsn;(1f5dS8L@FxISaYj2yNch z_>6q-SA1JI(u}Mf*xU!K+z99eR(y*#p&me)-y{cfi!QS#`IEByglY6d2SX~ zSTkcr}0WhT#RF(}5OdbNRM?i>EAuCsiqS*W?6R+BrsuhW&^ z37WB+A9nIX(1Sb{2pxj%Ns9FWNTf1&Sh`2MjUR~4x;;7fts#$dwYyv5L%-vf{Dk&K z?@pbfnBiIeWbNqt2+da3t^M(j{GQ|`b>woei>$t;U3eY_tiF$qZ6O-G+Kyo_6hLe)A01je|Mwa>XH5eU;KJjh zO{5vB=rPf$Ue3#W`xrS!%|g909Z#c1M2FR;8wZVrQ!-Sm-Voi2o^JW^_XuT2f zzF)BW$T&Ujla??!ibZtGLp?odwuWopYKjjppAYYGau9=s!^iYqNvJf%7+y8d|6uKl z##ohTE8jo;?Ps{U;Rz#y4AwfRKpqx&mD>Th6N^gGPVMkO< zv`<#VhZc`V#ba=Z>+xVaatYnxjPtCUZ%MxJRP4r49sw-}IrT{7%DFg~-_pJlhg!S# z7be3QVp06ejkb7D9{jq1vO|NqIHaT<*RRsm>9RL1DU!Wsb#)lX){=HjT!ixHNxD0P zMSq`Mbm=p!BIhq?m8z+1Mt(GP10FUitk?e}wGAA<$IZ^r6)Kr^x!7lNjV}0rN1HaH zQ*f-d;QI%id{$)R*5#j2L#$nhmXZCK#JVM+yw`0%D?BlgjHOJqt- zrZmK->N;H8xo-Q>#lUOZXlJfTn{XcnnsMWp@aGSIjEn|I&ZKUJl2zH7ts%a7Wd^R` zGJ{zzvc?G&*-BBD%WP+SNbaJexN(^uc@oz~7$96cN4B4kSg#~ZJB3UI@5D5A9hZtu z!M}=q%hJbi$^YgplgY^%SrgR;eM);Km+#bKxV&YL8qLek$wkLpmTp&(7A?ejA0IbS zO8UFpAmt{BxK?%msujIa(YG6DcF`6B-cP%y~vOD^g~+%&?}Mp}%QwrB*cUn9P76S_BhTXc=$i^lvi&gUPiu;Xri8xJ0U zMCT!i-p1XbYW90B2Ax~Sn8$TMW&0lhpjK^;)9ngCO7uz`{~ODZzo4atx+m?jVhHw6 za9f8n)88$}$GGqA`8-+s@p8Ts(TAO@PP?iC{hpzQaf}VvOeZnhf72Cg*g9(}bF)*o%_=WplEHC)d-Wkn@0w zUBSUm%raaA!!arBW?5meQyN-YEz}@6iluXhmI*a0O>_gq<0@&Ff)!?MUl#H#^dMe# zFEuZ*tvHhTa<;1|%s^#rg=HxU&xMh_KNptl&1xoc#pmiJ`&Q4@0Dd%;C%49(AK4}s zwaE6JD@J1rvb|@;1#&QJvh26&oh9?Hhd*GU+O=h)_YC%S1FN=-hXL!@X+B*Hxx zXw`MGSwPL^B|?ca`#m7A@qrSpDf4NY5MiyF1iK5 zA@YnvZ31ZuH% zl_@SkO694@FdrMD&5Kv^Zq<3Y-#kPVnDCqDv=__VN^0YTL|+&G)KD9z;%1U{1D8j| z^87tz4`x4V@jsYd($2O@!E&?qiwq>4CM2Eh2~rJtayU5<1vMk1IYC-)y^D3=NxvK% zv)2u(rg7juomo4afw`u#I1v@ivQbMGwGvO>J!7#5kg{oN3i?XX8F8C-5oX%Jc#^TN zrPb;!MjNut#h~*x*=}B!BTnMy=8;Dl?-P5_%?{$$ z=d5%$iqnl(HQ;&R&=it)&XHz-3ZaF>g89JnH|FFEwa8vDqEQC z0V}C75+%L>eH8)Uwd8e5R{;m`y=mh2F?YbsrD*TjP<%FeBx*-yvuvECQ^Hm%pdmUBFQu^EczJx4&G3v)l`})pyEoFh zuwqTxwORZy@34<*u_bh7jJ& zAB)1WWq@o)$?+1&2Z7s33H@mOEYL$V519CJ5uiQwS3~0)L=JL?I-zB)A*Cr^#*IBFvcr69$+O(e&38w)(x$ z#*1N0vnLj7cc!EKeUjd|*um*`>8fIN5G}f&6Fs}GDs=I%zT}=2I|Fd&xqD{O0YC@g zd>1sX4NE7-lVu^Ie7m+{9Y2h>aGey*-%{D$QdLr2HW*fBuw0kzO;sbSA2MgaPdywJ zgtg}B!X{tg3y)t646+H;Z5>pXvz{h??qAfT1g%+!4Z=>zHtqX2a6gI{vdNDE(V46~ z;(d^3PV;*q<4z~;g$!g{pH!1o7Th7 zx~9AVY!{l#&98-`IW_pz$KDmEyoE1HX}BDAGP;_}EvmT;x^>A&L4^J5m%sLvb1 z6V=acNosOv=_WBI_0=>dyDLAMCcbHC%Ag_x#UCZT7&omcvJ$%<9rv%5=U<cu9NTssoEY!WbY8l_h={o!EXlCqmtU0V}N82t3z3ny`#_$**L%7bK-jo}JL$D=h&j+jP!q3d zUZ&c|yf;15RCPaku(Fd%xHksDBiF(llK)Bzjo~UN`n_|9;15)W%j@6Yk z)*b74+n=nDkM$Wo)}{6_#}ydskNb@Eedl?`irV_WJJ2HOH`piREo*o2&Z5~cScL}r zHfI~~^yoWW(;uzb4Ule1RCLuEK9XNC-emqVf3Jv*f+V;G6-AV3*_tKZ9x7wkf~tp` zyiM9}RNPyUnh&FoM=6t*VcC8}O9 zwDjybkZs%MV(%*c+iZ%zihwKjE-08|LaEj*&p!d0=UjKN0;JnbxcemqWwV}pJ%?kM zp8M%^g5&nJ{A$uoR2N~ML&6INfDO>O_E4&%Zk1OqFPKBB>_v2N#XTvg$mcLBZROgy z9sddpY)+C%$9G2}xANFj5j&4D75N=M_jTOVcYmbRH9z7C=H6J532gSPo1I0)3f+gp zhtq(|bJw65%z6rRywXA$Olj0Kk;@FOlt6_{uXdG|TBE7e3rb5*&kPonlOy9k#r}@q zeUUuxeweZmBUcIQ(ki{(BL&}&DDp|2np5ew$lm%C+1mjBZAppz60LFjr&VL2VYkUM z3_e&ZC4VM-v|_>N@=E^&QNy*w734;}xSkI2gR2Xwi6VGgdhUAJOxp$@A(6gGMZuQb zh5C0ecYZl&-SLs3YqYH1Ev)lV8=}n06Wl27pCib=5FWo4rX;czwOfGj&}rJ<84irlG`Y2fOr?_Hwe`z5SOctuo{plqoiC(Ve^kNRAO#wQ=>`GGxtJ$?& z<8A=mKC;&!kx|u;nle1uvFUEGWJE9lY=dQRw7oxlm<{^in?2ikOM31vuBVmuXpA`2 zDi}_nM$=<(bF#Q7z_*F2don!M+RW(pp^sPzn9|8nrTR@Okl`5wGb;En5CP*eW%P|( zL65>}j1`1_E{1+oA({@4!{HIygfS9@(cQLjGuV^VL%&%|52O(`B%=eFy+X7p7QOF5 zJKt_bisn_TmzR~9Jn5`HatWe!e!UXBLzgXkIxW?pzw7l8yeGj^1_|C%)}?s>{o)H` zeXCw4thJO^znP6{uu1(;f}a(j7SnIKtZn@VGM~$gy$4JjsfWOROb3>3*%Q6F6F`+! zJHiQ2PlhG1cxV8@j)4Ym?cP%s&pB_WKnq2Cq8D`SGf+1l47Fy>0je|e6}-SliuZ#h zwiUxC;xTz|uQVC5YwkLv&rf@oRUjRHXUIo}2OvC+p9YvP>$PEc82 z*$S_>!T?jBno6eP!Sa~8aC;mYZQ`1QZlth(L%k^|s`tRd4cl;61n>VE9bB5IHm*8J zH%d|y?ugryctx-eUzmsTgta)?LD74**E2#}_o}X!boz_69k0fbGsub;(GWf8sUq30 zXm=of_LJA@XHxKbv-jic@Ek@4VCC?~Ae~Qv$~5M$Q!vKoY;?u8c1S*{uvGB<2`Y&z z&mgLn4A&Is1p4~HtMP4%=PbC*_mSXx2p&42N{UePBMXYKX0;WBT4ABg-j{?;vkS}K z=Besj1*IZuk!n{P6xXDVgg>4Pk!|j36@8#zJJ}b&>mu~oc>pJ!!>{AZ?t+E#FbvAE*gTMz1ZSN8CxK zeJI?y*iL&+Mcv$|#Q}JE#-3@C0tURaazK^okp0qxV#E~d#jWpIT$~^ZWcAU2t8lV7 zxmi^)W5kceT0zuzU|F}N8+00wR)v$DzIOFyd05XTj6u6|8`@feug2KTb-)_kIvIVU@TyN?X|?(z?tz7A+w@#h z1+nYz0~5tjrg(0qugKcrWfA2UjT0plMe25)}}h_#FOP-lQL~$w-@ws+6^Z z58|m9z-l3$IJ``AvsrEfU2(T-J{T(7d5rqi-FWI5nQKIkmx0V4b@7cLSg}o1q1zqI zak%YuRktWXN6MxQG7{GfsuaCFBh@neXRv3K0^cvJEGzb4oN#NU|617_0`;jeC7KP& zh5#>9@E1m$QtT~@H$?{_u+=VD7)nvM+@@Rf$gW;aTeAQ>08(kww9Q`vE4>X(+}tq^ zba+bzIz9c@IIC!-$5f=#XQ8&Y`vIY*2JdC}Mrv^@IusqLz4BLDAX^f(Ha%i6m|qvH zAyK(`UcdblvI&w|4A{CWKjp$NHQJlO_p6-8Knn*R7%0tzRn+Tbhe#3O)|HzbbAhc3 zin-fz5I55R!Nqv6-m2$rfU%Uc`<>9w1$ZELlg$lM*t-B|Lke8!3SM1Wiu%;ep&2Zv zH{hB$72}kS4M-EeQ+?=kE&Pt@pEn<@bsm&zA2<|sYTbWC-x^^f&o!1Ks}mIa8Nqur zNnLg0PN8-|fNX!kM|){HSg3~FW4p14cG7Qre}|BR;c&qs*LXNwu-!wjldPBj3$O&I2_!SNffsu`NA!VXXS)-bb6vi@&W`*z^i3j5o3t zrah1jx@T{#bk%o22kxzQ6;HLo*Qia;-7^*MSku0f`xtJe2~+r1TF69OX{0|TpTovu zTB6JH`dK+ZJ#UuBHZDInlM&yz%%K~_uKDc>s}os-4A0=;Mc1IH{op#qqIt)|5x4 zM5%K*F0KOm4b^S1;Xv{BPP3&LCJ@;WF;fl@^BqFWp2aOmhykUJFhh0*V8*hal1|s_ zxf3QQp=X-aUeD1}i|Ba+Q0Gsl{m%q?w6gP~rw?xELh<@Vh@aJt7=AXMisNTq4xTWy z%eVns=VZqX{K%6ye%zL03Y{Hx)BdbAJd-WRE-VQF@yPxPtJ~fncy|(lfJZC#-l{8v zbpkThN-tOnSsOWWvXRYJ`fU{752Q+W8X_mWct>n8<2`?4H6yPt3gbe)c`&y7WPu5H zwA;F63dz8lNYBOlxSIhn4C7`iphQu;BaZ3SRDJ zwZH-R-Od`tqDDV1)ZR!5arJ4FlZbJif$fJ&Ikp$QA@XuenqkUQ$XZZ9x-f9&sn zPAp_5#Nm@LZFq zcBSWQ_X6|#$s*8rOLQFR{6k>L;(0O}V!uiuPczRrZT(9i^Gdb0m*Prci^}b$rF7@( z3y@mxT*b`*H~UlxpG`w+z~z77*mRbql?UctRTiw!rK^MoCT@7C(mX5k4u@4x8LRxsbaV(5UqeLKRq8ccmz-+pb;u4(+jQ@st3^fD)j$fS4I2zBQQC>Fru-tiUk6 z9=_|13M?zP{Y+J6bfcd8yK8|?ovkkRz_G|{q@?WxSqZ-zMD~uhy-U|m2-zmK%*Ipf z1yh$4bMOgk)v8!gEkP%qxmbEjWTYUm(6SG$J5W{>Ux@SLqDgrUxZ7==p?Yxz)OAw0tR zyn^dhe|RxYlFMF<>s5^yo7#O)%Y|2wjd)%jOnv^!q}g?-gQ~wXQt&+j-?Tfa;%IDR z(u+jtkyY4$^P{3hl(Xy4qi^vDG8A~vD(FFXJ4w zn9Mf}!0Cn&U8cSF`#8<-dJqd&kSJnq&FZ~G2fD8I745Y7tX5_aDMxn!s+Y_=XZvD zEu2@LAI|>zt@l%hi)EDPZgex=M}9#VDzfb1JkLn&i5KF~Rxbk3@OngKIZ6svj-I1) zC?F<0mOX5PO7?Qw(h&+1vlY8&Qw3pDHU-2Jv%RNukHrxHMbjzE{@V}oBOj;^%5?3` zvat39=BJvVurwaE&dnNll9(Nu{42o5Sd3&ee+LFS^&YfEhiNJ3X3Lu3#UgkSxEXPH zwX9cUW0p<1>=A;d!nSlY`lbOuauPtY0njP{D7v!x)D`aFvo^qxusS5xrUc9X{qkAQ zHUD8-I5-$cuza2P&jX^bUhrKH(l=QC4|*=d5X=F+`c$z{vyLWnMz%uXO8azhpNm-( z<^b$DZ0a4tCL3YTk?rl#Raa1a8QSt(hk%Le%>OxTF28$DbPAv^IgjF}$Fy_hGLXA%5(m>RZ z7^Ki=kl5Hx8mG(tzQ>J&l?_aZc&PecAuAlCu>Tx6j&qUI4~Rb`W9H|5F%yW^IDO5Y zizxo%rMZ~sOz@omAtiPm7!~p!JnapoM~Km!OHU(v>#9$gmt2rUPw56dQOOtw`A;$H z2sBprIMtl$o>^)}bo@qpXe;USTbTKGI|6QvCQYmr0fv6M~ zuv)2rPV6Wx_1Ii&k8E%79L2Y@0F95Ouct6S4{Tf$)pg0<{%HDo6kE}`#M4GKdke@ z&H4W@kkqdn2tg)8D>fWRv*MSt*ISG^xvic@i#{b78dEgFUqs*qPzCVJGr%*W9kFwM zoU>b#=M2v%aoSi9PFYF2D>(S}do?|fUOw5Ekxc`|ryLq+Q%pX2AF-i6nH^gT$=XCe z?4?X>mi4+?LB^A@muX4#tNj&gf^%+T#;BS<(dkz#K$)-Sdw9*rF4W0^^rSJ)pjKvps@fZxGVl=BlMTeZT&N+u zs0$5U`yV%yhV%cBxMqS>ny%hoeVqvvI{~<;w|mf%5x_n%rNdAj7kJ4aZ_k}V&HKFu zZ~IRNElzIo{090FYJP<&CFp>OoZ^5e6L;k(n)#T)_h`BkZx8Gh0~S~{BMOnMGVtkg zc=D7d)=qiS+8L(6DE*>li^=Z|N zt^ULtNmFMcP*d?wK5iS7_A7|p16)BdC3;2QPwlUZc zjVcK#F$~2Mz;%O^G{y1s)p5NOLSB$ggj}>q4C_c3(E81Hn`vTtBWlpW0d)Qx|noa(YF|jD^9~9m3k3 zWKa(=l80!g7>@yuq1xMIW&;EIMxHw4+Jrg2jk-7RMCAlOx?RVnj`b1WeoZz&# zlV0$&BzRfcSpzNLDUbLzo?u4aNHXRKERNF3WY+q+8^D@Hao`kolr2Cax}3vXjae=xp6=(^|cj=3))?qV;q8uRiKZ@C+$fLK^muplP5MSdEno*|guWT>MacI@Da z9Xo8-+8&3A%cEn5?Re}E9i%<=IOWV)=sjq|TpH0o(`C>56H!8vQyTzZ5p5;yQUJG+ zWnBgtD1&M>3uK_nE`tmddqZM%H*F=DxS4K>EsbofTMIQ{tV%Y1l*`Pj4Odfbb~NQ5 z=0M#!_ZKkK{@$_&fe3+cU!Y6*n3W-?(*IzdqxQj(vj4$+|AU3_c)9<P2yEZ{hz;*}32O&w8C2TyX@uYZpJDsQ zC8a}!nzf|Qi{{d79vU4QLM!xb9!9IeR$Y3TClfQB*FG{5epa1<0tn++S53X~87J6*-m|qycj+-e_xp5v|4) zRI(+EAZWpA#L0}jgs;`Lh^>Il*X$XOeV~^zsLTU*6U~5u*UlJ$O@SObC&vKTWYrJZ zOsWkD2F;I{Z+Ys2zBH4#;dit@lKYMiBQFoWq9)CxW!-G#S~u!?&aHbj)%_!{n^!}1 zAH=$?vyHmwTYIi+ZE?wdQexoiB4?mY3}_OC?~v_HONP^ZcP(Fdt4X!$;b;om>J8^d zGljRBW4~SOb?y<*PKtqBY=OGYJ%KvssLw?2$74kAmOQB_A{Kqgk9}4itVXw5Z)k;R zKla#n6qe>|S4TU1pCybt;n{jQ{2`cidu4xKbSQj-H{lP*t=x*1us@G1J&Ir(s8imA zcW61_=2G*@L)D-Sdc%*~j|)Kw&>#k_7>vXjbnBp#S(+qtcoA8%v!+?_kAfDu_QIP2 zZ;wOpQfz=Qk5w!Se`pk0quwn1Oaj)^aZs*l6rK&&a?gKsUj@Q$k^ )49$90sCD z1>;D)O*{KfiYj}_RE&71UMMkgv6)-2Y${&W^-OjIwnVSRF|7_ohnMmYAnHh{>q#pG z^s9mxXiQX!W+k2j_3&6AcN|5|Zs}Ib5ZMU?All_o&C2*~^2#u1qbAwjBdjVj^LZ&c z4pSf&bwvs(5MP>DdI#8(9JhpRKy_vu)qrH=36Xoyy#jc*AALK-uJ=UmS;u0ayVK+X z0k(|@m?dNp+5lz3&wM9pMXywc_cK_p6u?RG`=8nT+(HZpe4}X*zoN-X;IPA0>k}Myv#tJt3msYk_ z9^Wc=d;ld~mQ!qZSGzL)-*U$h_yIk8GNhWLo>cKQGmLIx^Ep6>dVLHCk-ej^ol;;6 zMjEU(wom7wj9(bb7L}NYNZl!l!)nCaPYzfA^p_Oic2OpIgZ!i2@CK=whdIO9P*-rw zQ#WdQpd97{{q5$UyXzvEwK2tt+u>uJC7s^T2+9A(ELv>spn%P0VOMav%_+#c#NZv5 zO7=DasM+!EzhMfSm;r3zf=ctO=F3g){7(B$ zPY0@h%Mh2>c;7-mtEw(y6p>%@`)aVH#OS@KH%fkbie3Of8^emG8=vvs^Fr(%z_>zW zt)dvvam3%z69rH zir&P!q0J|~-@SY+YT2)QZ-NK57r1DOi`56ML*!s^Ga%hx6G}s0noH@HkpBivm&2c= z2s9EOfKLiD55W`Ru}c8pu_IS6c?0+Fec@W(c@$Y-S6-W@aP5#1zMINW?%x zB!KbD3EsaUyuX=EQPCTf@WB*`>AV`=)N`!gM%qb`3fM!K@ab6`oHpnpOugn}p^|pZ zeTlAX-H*iMK^>nFhXU=$F1!YC8b&@W#TfaZl*Gt8zWK-_W=c7y!$2rLb&x(g*)HK{ zAy<%)?h6?+BnwiDQ9u!-R$R*S&>;_2pezZ_Mp1C;7`j@*rHUB6m?Uow%|CE*bZ$ZTJ^e>LxwQLl11ns0B!JvSIZ zbal;sbtd))WQ0&cpSjRytUnPnbAQb-$|%x|-E=Tt&}iPjI@qV#Bl&D!O}pnG5?FH* zZ<4NHkO7IEiF%#W-pbFuoCcA64T)d}+9zWZBy!n!Ad!DR6ep1#N}NR2n898ZF~S@0 z-eE3HFLD)3!7$@$#=kB_t>y|)5&^c-ptaf(l8Q9JhpFJjpi>u_&#S@Wgr%01#J#F; zSL#Q|v+n)x)|Yx&Y;FDJ>$qkzr(BZYluZVwTpi?`60x4t0AgKYAH+H36d3yO8JbQ5 z{svq#FKM({R0$f`HC@vIjk7>m_NBzQWY?h>lbj;4gGlgn8QOxNL!B3)j%5GP$v%~Y zX;H9=F>MN=b^w;iy3fZnXIw-~bN2La$TYW(?!z?yem%K9= zeBm61`E*LWT;%cpnR#v-l~@_s`LHtnwIseWUcbH%^E?`3o>?*GY56aiXHRcm<{9M7 z^Dr^bzr~nmjHS8+DFS@*F!IR+OGU+oSSVej9L-s#$kZBBoN1mm4a_t@?87u0`ZCRh z38pzCnQ0D9Vwxl3OcPlqYXa8UpKXr#I=1<<1lz3n|HL+VYuB3jy+6(@-w|p)`VX1q zh7k#pI6DSN;$<~X62F(?Bq6Z{1(JjtTgW!KXTG!OV`2f4AHE4SmvVMF1z7>H%%Ud2 zhq|QRjPbLs2PWaGHMnN-TEnYlwALC-2Bp=R{_C~&*f&-?DJbP4yx}-UDqVy*O2t4e z9to?Xea(qo22psnK?#3yoh$RW%U>ML@ipTcG5pCmapexQ%_M&@-)Z>W(0Dg63NS9% zj_5csc!Q1!C}iO+A0c10>Y=DbWDVg%sg@?R+KWNvNOo7jY{7fgP^=4Vv}h=|GnRp3 ziW}OX3x5Lm%I8c-wjTglMEZI+=)ZFUINstqdx2#@ogH6uylqE85K?dQzJ-kw z-tYnOhJ_^dH~a}ovE-+xXtA@T89;g3hP=OKwGGP>U>gk*!)n?O$H(i^k`2%*gem$= zoxoavR@p~D;#kyWISBG5nA92S@}m9Up4}*+W`j*aQfUiHrT+z_-4`c7LXE!x!wQhX zAHqlhpztGJYhK2^W)xebofRas{(~`rCX?1b3qMz42CeW$Tm|+PVOkxj{-wcXcr)Vd z3QY%LwjPBU$g=fct^ThZNs=s3@eL)J`NH$7|6>M31|TW&P>QkEYEulFS&?#{wMHb8 zyw;Lwh8qJ@$QU5H5xl2U5Q49yoyagsk3?V^vjVegUb=++hA4sURKkZ+Xwv-DtTz54 zE{!nfv(X$0rREsPhs-hXrM^PxtHF;I0-G%u#F zqv{~BCYWffiBsyYIcGI09!~NB1OAV9gmyI21fZ$dz%4 z&2=OR6>dHdb?-JABZ`v3oA%N9iQChhB|#b2`L}SL|NkFHtr-)qOM;Bf)}^xIM=es3jkqRmq!)`_hxo3e z+(;CGkE~Ae{}D+lc_W2b>M-j7ELG+#_5SaCElWjM{WS=<_U&9^sn>qzd@L2qCb87N zXCq6+s>D){5KF}}z*47hgoM)K49w7HV+bK-N*#@upmD7+zl~oa`P`CFk0R=i49IC| zl_b+5Up7dK+qhQqS1&1lB%P00IURDPkopq!%?383wF6DuIdEgk1xQsu*|>W;)DV2+CnOnUx#O_D6R)46u)uvH;jJfY43NUu5tux6 zMPd6fP=N|X;TK_Vca>D@Dwrhvq7EKC>93jb!kR@4?C`YA zCAJs4wf8K^5MML1RthR06&-NeKV4c;Wqr*Ybs&X^_BM4An3|*}wv=&7@z_L58+`G6 zf}eAZ&&EgABK#aD4d$YmW<59!22n3KyD1fk=>2dUPCv{j(uY`7AESX!^c}1rK?AG_ z$gQRiVM;})1Q$B(`+x{G6A>)vO9UUqiD0uq1aA@%2<3bHa|@=3_Pv6SpOr2EORobX zhtteYftl}KJlB}{)$y6%HQ>xkrF~cSOo;XufWiOj3sA{PJ*ui zNR)OgVG)3O1$2vvIBMpgG;h(1z}9bfvQ}bSg0oc#e~>R?nEJWE%7D`Zs*#H+9Ur*x zR{e}%)=(+1RlE5;U5^$yYj8^@%-T553sTNpUXv0B^19l)2a#(B@wgRl!(a48FOT+< zNw?wgShnxIxp-b9n2zZ-x|i(PgsugOy=iH#ur|dTwaVc`R*chWX@WYiT3}TT#Vd|O z11a#@7g9fSPHe_cN1OpK4uo~ky#sB|ySmPUuEw&_7`dn@&AUvh*C0xFejG65$tnrK zF!A-Bchs{Kpsdd^g&yWX74>G>|4KnA8t49(3X199*9-Dfj4XO+TnXcRN&ahy4%Gq&QJ=PODCQ)Jq>{*O_LRj6Oa7}crz2H~7q$<@z%r~} z?CPp%_upbgaReMb(--y5+)wMQe-3zYeFAgtx0txQ^FEt7s?t+Ieo7YpWs`CjR4yEN+mWuYCrD^Q`O;&{I^WZd{xmH<&(|TbHtE`kC$bWyqqtg<(1_Ouusu`RMaTs70lyQy$t=nk4R=%=3gc^YQa9lXs- z$P8w!0)})(L|y`FXc+#wSd0HgtAezR4Y$E~jJ758Fh*qPTN$SkeWwCz2NYMArQ4Wc zm|njf4{uPRZI&DR!1eq%P!Pgq%!sAR!^G$LNFDMDy$Kc zA{J7zbc3ZZ&SEfhY-tL=x$H=Q|=+v3jSw-)h`OMfKKt%nLto(r_=ZlefNMwC$Q(*=??8$DU?w2(LT zU^zG=U%T`jys+XqEPMY&fr8mT(dpSU1zoAp=^Xv2t!!s>NbHVMMV@`9CxvyzeXWP` zGWMNuPwP&LSslS{D+beWnD)l^F-)RAPg`>t2BwUrU~aVj0mu`1{@9JPV5Httslrd5 zK(EBdN_1Gk*LjcN6}%&OZxa3t#tghGg16ctsAF%$i&i|eU<*z+9xQwZIw z)`_9$9VNP>^r6XWD(INe8)bH59^N@Ey3wYcwowi;>~{58quUGUueC&vzJwV=fZn1j zV-HVgW3Z#a8}Y&yywn~uoWc6#oixbd#$`wj-mpzu(gx^liciKcnv7JuxDoez$7vP_ zh;FNz1AlD<{4gAA<2D#Nc}3@L<08K}c~1EK1_~2O_i8}!1u;n{@1N}y1IM&4S~&#h z^PY|XOtU2eFfH4~DjY6~1;=(bNCmTuOV^9m@*6lZYn1{!B(_ECZP4}V|mE%F(dRSZn;}(Uut1^~*;f0+qFi z!djEJdx7v1JbxE_5!!;kX~j&zhsiIz-J;-IP7f98E#7YFv0>isX~I+Y(kn&q;k_tW zkBy??i!bKYiX¥C}-0YsJg`VsT_9m7OE_wo}`;A)CKT@Lfu;=X=6Xy-dva*r1Vo zGwDZ4q3G>N@mQb%F{qsF3d3l1PX@e9SFH`@mz$NKZ1$IfiZb06!GK>RDpEYDC865# z%Y)+CzfgriN#z~p#`0o*g|Qg9yyWs(Y3qV19{$7VBP*z1p zzyrk}$%Y3KZRzlUffKMUVAi(AT%X9F&JgVv#krW|WT#tZg_x=E=1yjKATxIqdHZ=ptE?@S~utm3d=jUJ^*L;(A zej)rIVY^=yLnHnla!he^Wj7u4@9BDvf)ulitC^M={U4riwdYH`v%2t;yRDW1q z`a|`W>eBnw%d1OOHLtq#VW9>C-NM2I?s`7<#T!4NMfT6app@_g-!(`X#y)toy-VRgL|tbBft4R7b*>)YwZ4&C0Fab3QP zfiYY?`F6qgJgtxlTOixl&QH*)PhH`eserDg1Y~n{&+V!hn59o{Tap&23$z8Adc)Dn zBw=lx64os6k5gFd%A3^pqM9iDHSDZ~*S&R<+dS>jZwYH#K$FJeP*WOlZt;**3$JB8 z4y{mg5AE4j*zl6m_^-M8gL~LGx1Z&ZR= zn9<8~0{e>JL`=#jlgOuNNWJ9+a};*mQM(Mv?QL*eWzt-(WDz*#ydtpHRq#C*%3@e- zbJkyH;(<|$E5XbvR+!l}-(l0vd>hf9M(OSx!ls=V_?1FM{GU{+*PDfc^SsPQ@Zu!CxCi!vIEk(_QzgEvq*>8_# z!GE?&_+xR@+WX40zBx$^U-^Z5UOl>%r+fXh3kFuxse&bXx++TxVtw=si*8zfVO;qg zD1TDk&9VEo6cuAp2`s)7#-xqYE}2OoX}>cwk^gusw~z3gOnkeTh}+%jWQ_pCJpcqo zEETfd2Zf<&@OVrZIx|}sdVM|yh10j5|XCb$Iv z0lMZ+4jmX`0{|%Fozg*+HL36r+E#d#e^KHpe=Lyk?e*H+RBAeF0|EUH7wBNmD{R*W zv^F#g?Sq+#XI-Br2X8I~s;aaBg#JfMwV_si@39h3lYn5+sen04oout-gz36P_KyGh z>e?6EE*B3@^(qs2l=+0mNIQNzpV#}}_15hD-HM!X_c0$h$ z9l$hn{9>1|@wgz`cM3IU5mL$jhz+C<*n@Yk!Eo%y{WV_*?5E6jj*`&Ft zclCPIUc8TPHep&-cW7k~26I?y^Uuhk4CUI7W+0*APDpfkQ0x`$jY}?ubth;y&xl3S z1^#)vo_jl?OxrOX=J+$v_XtyzVA5Zez0m}{`AaGVsSXDQQV4XcLJYgVhNu?<{Gu65 z!QF3rpCC&EO!_NwM7%-j?na(b{vO}nrXahCVD?6gU5AmeZxKvw;cpXD>vy+ex^ ziltb67~}Ng%;>pKy+@thiaGlL5_qrYF6QqlBw^jnAHkp7KOh2xC-*`4<9r|9?}HzE z@La8swj#B_<0hV)bgX8d9!J?=J=gg@;bpCY@&vDn^RX;>7sletJ2#8Dr}|bU6wLgwSY0}^fiM*Y{kqL zg0rcz>?(CcrMGOZng)Nb{C~8)3wRS{);OMPn_j}C&`1%G3RSBWsRXnoR1!#F0;!^a z;3|rX+fPMzDSGxO%1Gw=1h=YE1IWyKE|0;%udxvZGqak*eQM?X*9isM+; z-Cj0f-&QnZDa+VbmVrTh13%e*6=9NQcK3cAWzU2{IA`p!xhWWwmTg;4162$~|DvK+ z_7`4sDHI*kihjk5MnKU~t*C;Ef~K<2ZpyD#B-AyO0ku>XI$Bhg(ZQdf)kjmyGWzGx zjMMms9PQx|LY=-W#N0FdB!2zm9%8IkyaKwgs%)-wx>_nM++imb?>mt^BL3*cP~& zLfRK2`_1c(9l9RO;e22Mf!&!L86Yb`{DyIvN%&~ae}~C)nNxN#;!f zWuBwiJ;G||0Dqm&LU3_!HR(AKmWw&`!i8+xy2R9{v1L#toUi1u#c&fU8XvromCiwX zFU+{-XbI=DTB_bbZ>WLEwB%t3J;hVz1iM6(WoF_6z`S`?cu3=hpLR&|M|U=@F}iE} zkU{U7j>0fXQb0X`Ei>yNCmFDhaN6V#%Hfb$8%P(VF)yZm=}@li$IVfGAY$QQ?WMC3 zBPflvl8x~uFQCLf@RQWRP7g;iBEAr$s7H!kVoZ?ZXMr5wHm+K^5rB||OE-(2HcPvr zv2n(>743Isu7t120q6gJT_R_mjbbxMs?<~Ij#=XxsdvknJ1J8ou>tIwu^_X!16`k^2lah+&oU+eUoVEUxTWtYozU3{R z@I0HlAiFlaAXDs~ggi5f9-_Tg?r0Cdo45Wd&-X+qYU+*|+oGop+OVytPc3K`x0=^~ zfA8nFw29E9@Vp#oRr=sob)J9~y1*iC4cGAhpkYwk2fxECC2Oy!pB`Rd@)SfoVIxne z`$PFMF|Ni*_ja6daqG77;k!?XSJ#N$m*Yw^juNA1awrieLhW!%^`VB@vV|moU+;#8XiqA%z%4aHmty0fR~}=Olp3Cb`rm{;H_DlYXsXAUnjHm2wa zzYfSm{NabO5YJ!vA7TP^;fgHiC3$aCs$2R_j2<@vU+5F&^x?`ICrS<^qcdm=&p4>l zsUM4>2H@;~us$CA>_0@X8A8v1LO%cw+?M1-=mH=K)gJ2rs0U!5We{ZN5#kgR`*w0HB}+wrwczQ4at$hjH~nY21|#6n*uZ#FCL+j?}ct{AVbwJ{%N9spPv~rR$^vq0eW?^IQeHV4>`979DU5 zS*xLr2+Qn3(b0=~+*TT!IUmz4?oVxvOUi23%x*v*!E#~pqoG!QEX{s?_0QX7?13l$!eF*_Dz>3}wHwUr^Z6?1Nw zziWmnL*lS_SS+BX*XJ!Ol|J)f)*M7KKFrV5q$hD>b$cMyA2#?xjTYDBZ!4VRgBTRV zwPn0Z*t}Cz85BeQ`bX9dxNSEO2u8O|gKnna# z3;qc!r8Uk`b&63`ce^HcRXBA*RXd6jow)&Xg)D&J1Bo~}TuFMJ&~&ZnC4 zNU`XMC|FI6R5))7bYX+9uBs1?!f$3&Jft7bfuuRwp#=KS*v1r_+KeAhX-vHhWqWEW z1~aNP!dmb^rJ!YcZooM`EpRQ2MFCbcYaPvCUp$oTquS8Fo4h1;4~ic-Y#lF+T>?ee za+5D~sSca1Wl+8ka{vZV6Sux!&db~4^n5&u;5a{;JhZ_!TZs~$>0+n_NpJ~H@J_$8 zd6nXeer_xkUTgLh?D9EVRv$*ADXwyzogPp9I6V!QQbUrCkLLnONNx&cHyp|*LOuG` z7cdzJ%@2rPDq_{J=3|r4c4v|ccjegU8`E_fNi6NhB|c*x-cGf(rRyB37#=$kZtxT| zyNa$08b#@Qp7>(}P*kO^IA06TB6edhn|}^to^Ei(6VfdbeLxI%QDQ(kkh0i0r>e>i zWrm9!Eioj}ua_~k-hP6M(gO`FGoQoQV}1Hyw&Zo87HsT_ed)Tec`5u=8c>=7pDjXC z`Wej3*a=fP3YgC#HU=%?rQP@|v%$U8r_6wsK-E?xkBXk}Q*fbK$^ zO+rxVi0Jr{Mclue(%Fde98^)C6E>j}j$ir;E9LuPD#K_d`JRhTi9POuX6-a+6hn%I zzv}#SWu{nc3>p*#?pMvlCWy{6M82@MQm84<%5b7ILPI4srip5fg|3jdi#cX)LY7(o~O}=pxGU9O=+S-+tHvGcDN0{mrG$ z4})f(voVkh2oT2+IHq2_7W$=5vSh+Vhl?pThBOI0ld|KmB(VK+hy_A}L)^G)2ZZ4N zA8@WH=^b5uB%3kxVG!v8O4sU_%#S?+`{`X5t^EQwM#r9Kp-lF-+I$XH>VP@g<8yRi zH}>!LJ9i7!7|&TO_+%@}I)T;(3KV$DGomo3#|`JtQPBwZo*o$UmE*in(2N%D(jK;a z4Wa|+C!1&{WE#vD<%dk}tz6wnb!H^$r13gJ)s1NAC7EMVx~^2tJmQujK6y+ZjIdk! z5cm)2W1KtSWD24_$37qrQ+-GmGow#xf!0E+8u6IB0JC9-vw-^nnzb`{0w6vG5HY%t z^dSH%X!c2SfZ(*bVHo%C0#K#0+v0;aB>PXHqO=Rfq##<`jB~wzkLdWCmc`|mi%hcn zu|~ZHCe^pU+3%mSRaIA!CqaPuG@wyA>Nn$Eq=$Y(y*m8EaE?#gW{<#L=xT)2wV$A4R{c< zsW3owr1uvE%@P zb!QY1+CrYqx()9|xF<1CKE)5uXRD?r3gR*bds8^kddFdEe|(sCvq#N@9htP6Ul;rM zb>td;9oMbF!vL3al-*<#!d z;jztFE?Ilwm|`9h{a8=Iz2h+a_5~JB;)U~(cnyy|h(%*~k(U=;0~dy_OVFb5*g`DS z8U_TMyO_f0%`HmN#SR9L)Wq>PK9yi7+@WYL6p3oR zR$_z#A1dn7(R$D9akd0@dkTO(jy5=WeEfXL{Q1swlsaC6>+wRic}hZ4Bb#(Oq(L_F z4gf9mnUCiQU(f~!qVm`l(oVuvd)fDWB+YV&1>M4y%IR5j3*;;{V8}C`G666_A}b@eXMzE| zb^#}zqe!+`rvaU<+OwNUW;>GIN;%g$RQtV6B-;MfWL3pXrP_K<+F^uxkFJ>x;5e}( zcsdk2`*DDW2*i!BauwYJSz?ptHF2)*ABvTAPL9WxrAYPk0S+G{1Ghl0 zkcP*EbE9iUvHHnLxfxnO#)&{bVCRD9tXh-BU8DD@(JC}&41KK+euH$wL3p!fa$E_x zLxb-}t6h}GVM0hzo_SR4{@&+|teoW&UhCi;sy?js>6g48`Pir5t>h8?pif5LCknKx z+V?c_ZnZ(TMh(%eemOrEp)6N^CC_$-qN{ZhJ~1z>jz?EJN>0r^(n;wc8bO)aE&KYA zPL}<7dm@;ZSZI3@+et=fH!csjprA<-|-u?u9Y zYA@E>d`&X3>N1{?8{@5EkTt0h3zv}9GXX_%4GHCscVMHdE!g&OHHXZfLJ>?Zi`A>QU{xcXCOzsM@#gq_t3nEiN5irAdcLmu$7Z3(VpV$-4^Or~vE~rLX$)jwsVU+VoJbo;p4Y)^6gR>-iY5BvUKbfn=b?9=}3=jce4F`xuEC`Q(f!DVO zqPp|KR>I~(R%Jd4nfu$q_CnwOITgBs_7Kp!=Bd=XtPwDZJ!cagvr)%Z)pNn;ekDb< z-yZp4duR;uZR#(6&s z$j3?szp?8QU+EXu`G<4~{npp(Q{wCEcT=cK&%tFI8Wyq#&LpNGhratwB#vtdGs1LO zx2>h4^bhapbO}AckP3zYcF^$MXYn%v@8u)>%QANM?CHc{^0{9_(~YCa5yc(v4U$Wq zuUBvbYLu7S+mojgV``_**r;ziDHbdaE5ET{oLCftMCkvU zy_=mFTLbT*5j#!9rTz4vz5Cb)|Nrb=kM4iBcVB<+f3bJ_+k`{^{0=aJMao z&*oKd&!+Y!rjf7N*I|8X_|DgX$(gU(J!x10Y-pMqur{S_~G!E>$ zJ&A#xN*})V{UHg2q1P#(Xm*i^g6#yJ--Mtq)urx%WL} z#6-*^humVsThP%lr9iVhdjsFhp$)VQQ{{v8AK|B^ZTEhIhQEenSG*&I=BgVx?L z30iC6t(`ZCTI+`ko0V*BK8Ef>uQ&VlcgWruEojvPp`HhWR$1Kix_*;&+nYd*SV=^( z$@~gDOdg=|j<3I}mf$2&9;>{LLWKsIb~I+~it33W zJVHNa_OeqyQdP8OGpR%*n|JabdQhdR+MlZilUC#a2bJrS z9?B;!sy|M~9h}4Qq(enZf#siyvh^-nHVvU;Gx!IO!3W1gMG37DBL)$*Eol|=iYh2i zVB@T!O3K!A0ryMn)o=SZP~b}Dwo`a9Z|g#Rkp1Q`qFVi~^$7Vx3e{890X{*fOO4YLe4P;DWOo&!j2!r}*6EZQB$IUkY0N$WXm9Cxl_v{_!~jx2 zt-rs3z~z#X*J!CaDXBekPDh?D&&fAdUOj-rT4Lg1Vj=uReR{gp}n3cp+6vz{~$$6n>axOK*FlsgiZ+aa$ri49rnJ2EDTIR##hGc|N-5%rviDPKE zILi@bWQp9FBu+^`l)liu(L5eJbl72f(i&;1%ohmgsnoT!|i`$@Mb6QE`(VgJnXt zI2efu_g75{^014VjN}1a3K)4UHES&qbbgt)gWJY&D=!#RM>}f1$m>$Izx{VEoD0Ev z(FxV}lCt-3zz{Y%4#)1M-Ffd@$dNxNR8#Pv^Lz61X6VVpahhlihUt>ysv5hJ#29h; z2D`sclWMpYaTlNa07r+uJSv%`l_)=JLA%RG9wA~fx)tdHcj)LiUqQsz-BCKa-9KTU z&)M!5X14oeQ@UcQD8|GxYE*PK2fz0fwD_F6gz6LYcD>KoQd&^&lOh1XUl8#dqdUJ@ z+^*Pmej7;JdG0_IR#O@}fB4{C1;AyU4{m4Aniz^P@Q8i{4NgLz36v~REQpHT2Yph@ zfHuv8Hu;^+0qYRW3f`V*lCZdiS|#vSDcNPh&!S~QKwl;th-j^{VXMAMwrXq~nleHh zvZ-j$1PCN<8$EDrCp~~;+YUoY!;3abc=8qis2fGnzzbC`@~BL3bfpVb&ybB%y+L&D z3w|4u(NFQSaWoyg=cS_$`zLhx+7zF2KlJ<%yy%nmLI0QdoxrIgkU&}-zzJW$KC$gI zR28a4asb)ole6dh3U))IVRFQRW}F$nJZ3J=PB1+=JM}O-`-2Jqp0A`22^FX4`}h-u->@8?bF|FL7iRmo1l^isbB=oj@#o8O=zIluKbeCewn>X&kHvIBwP zTI2gw5ewSjyQsgXDgPne`-_*8?w!=kQn*ILYZvv?HVp^rZtf~>>XgDdzf#r5qzty> z@_|x?AA&D(_@vwvB|(&c9ZO>m1L28K9v%echiwFO{o6@SS)Hm5AWZ*Xwj+{H?+b{B z(paCKf4?t(@b}|B1M|gjX@@9#J2cf|E*X`OUJ*qwVK1r_wcmjn@g7Cb0?Iob_U;RN z{{e$0zl{1c@6qs$s8t*}2ANf?59G47_knbJ3O|PEo0r#MujIKi=}BIcUly);SI2*Z z+j?-!VF%UO@dP%}I9B@#;Tvg9jgdy)@JSS0zd zjJ=h7xSRbY`5?02Bp>FpN0JX~*~Y{}7{vwbXUXRX20XgsOB#Y)Z3yyd2!@@b!lO{w z1BG8u;b-UK%HKgT40_RUR|-F-E&eOnqf2p^9;E`Ob>~aKp^StH-SJbt3o=6MY`UPC z-F+*HFP`J?9S)FKq4`GxSCnRNXI%^LBIk4kyY!ocgkY>>T?XZjc6QG_7&rDydYS`R$AIw0HN{qXmO(z0{H-Iu##^M@&u1k@ujrZ{x%VcBV{gCx- zcutrBB=|)03ol@+4~fMF7`RH@&DG}AK)NnyVV$%-&ewyQCzPN6&i+;Wolyhd5v8nc zlu?W9)Tf9O;53htWpg|$xj?^7hz3(gy5A5F>aMF0tJ?zh_@f04{-KVE>O{YGbhn2- z*N1i)r1~F{d&TW{$~6xw%4vPtQ!^gPS~6mmPE-R2BQ}?!$0~CR;y>Ae{e7xDD??nq zPjr60<^nduuBw=MYdlbU*TR~Ny+hUpDZE#SE(O;}Re8+-vsBva58s6f#hu(B=>SR! zdCHW%80M}JR$e~0czXd=x*aGW@zh)v`!PIi{|QxWWm`W$cL?C9eV7^Vk-P{V`v)&< zulW3EMSXdV%~P|?7@LSq*_s@#6trBND5_1K+IQ)V_f(2u+~nSl&YE2vzIPI1uD#Z| z5i^P%4SfxKLQ3#LPYpUuL-%|W8Zd~?=x-~(?8ESN0QhS7#Sg1hZK^&TG<$0|5SefZ(40Is{X**77I(D$=4x3Rt~)Kk3L zSZ1uRNmX~F&1E!ZDszV-XV>j5^HxXJdIrA}>anbzB+56awfZY(zxZhohAvA~Ta-?p zb8lcot$uoOFtwt7T1}Rc))%SKE5<&z3f~^-mm1u)=sM5ddl!b*c!rOOx9C!z;OTRB zeR^Re@2IJ~22SWnd^#oIZ|G#VfRpQo+1AitcK;ae@f@CutRyedbNT5=IM%IjYt_6x12M2ctU z3(snn7wMVj7BWdg{t@^t*F=x()r-={!se(WQvI#)Tto!=5=gxe%!%On7#b@YTiNwX zh*dK;rs|Xu{LJ$@QZIOFHMza?v1abqN*>+Z1Be9kRV3x7?UkcDUnZ}KDTl-t)G0^A zx4Z*Zp0~Caqti*f!VL|fNSgR-pi0+V#R{{M%wPUzs>-F|Ly99%(6O2897rayp|ww~ zc$eC*!&HAbV1dbw_Rw_`h0XPjh-f@eeHcarjV*kbE?$kyte9@l1-r!M&F~-^cd<9# z?o**YV3lAhZgqLF9v1iL+L`(q3`EEU3jOwDO*-jVR|7Rv#D@;E0b*vqgJQuMw0I4j zQ$;y@5)A%FY}F?y{}HPHL~WjD@K&h-W24b4O0z8$MTLRI9%;789b)=*sqXHM###D_ z?oiKN>ql=zS0c@8jC(if78qc`dW;dMkPa1asZs?)qJk0DCRFg~W|>0o(mFB(vnz@V zgMujEsCQGg4i^1;9|kfKHFKXr&D_TkiB%JoILM%2%ctZY{R8qP7g~Byyd_!e0F>{R zdKlZ&r@E!u4Yc+f9+K9z|*iZ9S$jA=}v78wTxpE~A6(4O=8Ui@|{jmrlOrHC>o= z5>`B}Yux@!T!fo8kmsT8@6j}9jut`$&`4N{Zgns1eq4lOuEs@()Z+~cY5Q9mQ*@M? z6ocKW%^qi`@GNlmXV-h39Mp#CRV{(cS|bc{RV3J5GpjF-D+YWKJ|%K-+g?Bvxq5aO z2u5XQye|Bl#4CFfFA;u;BNzR7$mN|gsyYO6A=&C8luNuJR-(ce4V_jji#u=6D3#hU zQ+p_4DwUf3&V8#casx-dmFrnbXP{1H;&ZSLDtF+!OB<@%1H*i36e>;hsk_{bdR^Dj zjlkdBHj)%(x4=aVFZvXy`@UIDq98~0Og?|U&aSDG#AID;T3tP>69lsq1 zx3l=|Bmk2N|1OhnR7bb?!`|a!^m9XZTj(Z$0cXF3v zbZ^n4%{Y>AuC69a@CsIPic)J|?33WM-veal4A{ICpX)s}fxKK`9J2rm#a1C}8$E6c zeWeez8bXR&sKPi0A;lzAVIJa;g5mDq&&ZH+p-^Rw|F)YNp$|;QmAH>+$30yCC+&n1 zzIZ+k!>+~Drtn47(&Zy{nah>|ArV9KehiLF3my%YCBYHF$g^;-!CJzrzjq^UMNi&B z3Ll4VB_F{qoDT>t>M=INw;*mm<6gFJescRgMUCLi;-ov$VIZc8|4rOj;A*5O1qyQ37s$g0 zO0=@P14)zUnG{BXyQ4G2=+P9P(R4&Hiqg(nS8-qSbiMG4Uuo3-UXt!<(4xA2&`^BW z4;n_MlvwCXfcGW9yYfhvx2Pg;gGZ>_i;9dqH_1GBSipN?s9{G_x~-}TB)qTqw!GYj zU5K0Vg!fC)X@G6eKKu!sY3fcVR9OFoGBz~F*0;_IzXMqI=Cv61L6lOVb=R-sH+VnzA*)=re zH5O4d2wS3Lvu~YxYdTqzw&=xT>(O7Q>&Vx#za0$wZx!R{G3TWmq5NJKXVI4;`{e8h zripUN-{4i{0`6+))yf})^4~(aYQEv|be%LS`=tnG-&x*SIWPO?#NeeOcU`;R0z{fPDHx3O^2k7=4^BQt$;CkSUW&BD;By4Ur}7#@TRu&5ELAQ1QsFUseW zBmLzoaDYvTyZOU%jIuPYsNKa@j^=vWjc{0zUwLS`MblVoyfyL5o6Mg)m9A@WY2n05 zXnhx#?1REU0hw{Js%(-SqBrL@C}uT&Y_a)oaL+5$Wpk^#LGczTX*|qfIyR@f9k%Kc zM7OEzWxSs&hE5+{Q%nwSQDrpy(VV2s@V&FS&9EO;Ug&Gu@-&KxRQtN8lQzR&n)}%d z4~Un^&o%UMI59d!*Kf|fj&{Fv2;(JPDe^zkIX&LA$%wWUJ8vdtXjLdB0UI?q8URg=lhp zTp#=j1v^Gh&2Tio+(ZC|&8}HE3UepSkFCOF1vOc$$E5KMvBf-w5Bd)rL#Y`?FaU}# zVUC3I$lJeDi4=qfo#wZZP*Y=!tK3Bc;{IeTiPIOI9_ggGyw@k~6Xnuomfp~(%5(IT zjD3wwl)U>y$s*L5)=?-N;N|5-`=Wgom#}3o+dEvR+peiVhr4Qm5sy4q@2c4lZS>ab zlnl6wc%)s8-Y8607}`v7WpAV`?EMA-OJXJpSv9lieO$m%dnai$vx~e((XHOwoS@&8 z9HGw3+!GorHXPOAZ>7!2cv_u^TQ2=?ZvMmVJpIEg1M)S~pOPM>YFO_|Uvg)|uJj0d!5x#!CF>X2k!GDnF7iMWbe-a5crDF=!s--?!9X5%effNo|l?HSzOu-SU+ zpxNnE&$rN{b?cH2a;v6Xb~hYIp-W1hNJmnzFF3r(d==bNY;^ndCt(rVLubwfvzyE# z6U8M@qSR0s89H+|IGh;9CUfuO{8wp7W`fHD@CofY9?iEl3_(}(Q{xusx!GK6p}; ze~DAJ9h*~YDz4F^0vDIS$)W8)IgJK*Z4o+%RP>UANCioZ&5i|e2a$5}JncnfIbZUb zRC~^5o$ej{2A=!Z@bI+gEmEV$Jdsgn!^%^&p1hm!g`0s+EzecPdxR}B^}?2B#p>L> zYM2yl^kkOj0&C@LUi}@a9cvB1L>`jnPy&QUwyOHK3v?$8e7=c|2?!{ia4QA}F zJQq>L@v*^pzAe{ zY3MqY?5zl0b5x!1SJyOkuG*78x_yQE9XLZ6CX)e@;n-QO7sIdyw0qo6mx;oeAg{I! z!?vm$003eHt3USntrEO)YTpB)rI7=s}L*3CwIZ89I7_eGUol{t zCl|)CjT2yT!x!qPm!Uf(uLciY6}ZU_!El$k^hbaOf)$#!v(qn^7Rj^Bs9Fqri_n)e z>_vCdLUsw+7n_H}C!kfL;5FZHGkKHhpoG$HYBYRB$=_taO~Daw>bIfi4VwfUZ=?Cw zP^eG%lqPIGSr%7~N!A^JDywU-VHmA_D+xW(Rpr;%YGoDq0Wc+v4$vfo9NV-A z2v;BKBggZhG2+l%0OL`O*no#C#zDEXnY!@QB5e02J{aaOJpthq$G!YRAE%LuGnm3E zMx*kNOKtndN6JB*n0xc$gwhCk#ka0c@QQ%54zU@)=Q_Hi9-uVPxs$yH&#@B9mU_u$ z@MY;cpR|KVVJR-+p|#z{<8YW3hR&H+ej{(nFISM;VbG|Sy z7p>-XP*5*FsCUcRsTKZlIVQJJ^%WclelLci23OJ5L7)_8UAV0efK*s4K4X)w zpfz|(nrEtv{RwApQYdB$TpWs710&qd)__@*8kO-dHk)!%bjm2WS&cWaH_Si5n^g4c z%RwPfI@d!+xD+=AkmzKokHi`-z@YGNg_w*zdi&5Z3P`M>+JFBD4}zy5$G^~S z=q&RT7Y4@ki&!^1*d@wSW{aUl13X?6`~sPYBC+6mycS}y*w*Kl8a&coOvaNB>=tqx z=>E7yMnMDvg4vJqWVRz=u04r>U?5Ng0&l}k%12qA<{D8Vb8(U=b^D|~#psi|kZ>*o z2>b!|I`>W4FL4@rzNQRcrpcv~B5A%?(T9G~&HZJ9g|ICT`8m6q&7|%)GNwqw7-n>P z2@Odv+y4vWf<$Av~^@d77g~H~Ye&^KN@V4f@yz=01bOBjMX-_%< zIToPn8^w3U*b3E{T^-s!Uo4(F3VsR9VZ*z)*0!2lR!ta}H?&u4iu9dy3jbl>?`?p? zp%Ej05(lS68Hs1L!eBdX5dyD}A%*!5C`3tn* zDJTgRB;K;b-`a?8ovw%3DxSIu?gM+^^&j)s1JP3HOP_S4wBRdW@il=dqBH_i@jwGl z7`S^X(6jD}>86r^;4YeCLi4N!i&*SX_WKHsVDJhhUAqyx(Z_J@VuW*(Ps%*PegWUa zp4dyn(~rcSq;`3ZSbSmA5g-V=wI28kmKiH3vLBo+(62xEjomqiQ}=>Wm2QCbe{Y&6mKwPI+7 zE^rf!h`*p4Mao z#%Qu2#Ngkd$xlY`GjaD}4A})6cRu$VZeXd|c<>IE_M5SrOc_C)jg32-ko|Ldf&`aB z-JvpPGkNjM^d2dy+7E6b z9P}B7LS>Yd0L0QhNtfj{a zxz}-UBz8+AQQF6deXoC?f+sj7S2p>RBZ1CY2$9I@tmA^OiUk_t`~TfBPGubP6^oXFdb}c5Q!NfHrS!A|*+dYJR#p zU8gpL&RAFOe_gMjZ4$=zoUDZe5SPR1%mnJxd~XQ(>C9$Pz2uH63dW|{nT=fn5R|b{ zWe!wPcWWPNR_&Xho@yXofnl;z14>w z&L+&(!hW_8Q&-AM^y>{$heryo^}@OUav{|tm1lPU7S90SGy$!+rM;uuc_3-pDWT6T zYQ1Xzz8X%TZELOt z+2uX>@{f_E-m0T)g)DHBXy_ht~5Bk&xi8J7^b9=GB0)zj$F@Av&&l%hC2p z+6!2q49fEH2O!J-5<>133Sni>>#Qus4+DGJx*bqI`95U9QIIUJ+s^sfQj6%=!@?D~ znP4wS7-JIlg8UO(oP=50z?JRVC@Y)$!GD4M4B#lw53;{32XyK~`DmLPXi8KwvKwB3 zlTyYvwXx_ZZ1N{4l~ZV0<4&E+JA zFdd!&hbkxr<6yaOuawT5XH*sV)v?dGFaidJGaK6k)y4h0B|-D|9vGGu?WT~8CTo^b zd7iCQo@K$7`{V^V*d-t4mwJ@P!N@0BiS;K4F>Ed4m;d6IyQx)l7A?>yUu+oQ8L#78 z&&7A=xVGqBHGe01GSq7js(wi~Cg7rbQ?yJ+y_P_Bs5c`ZgnEU*1F~B|54L+04?F%3 zRilY{Y_stsl|q-+&%QJ3#OUWH(b%S>6|Xa4!Vn(k^`xT(&VlL{;YqajAI?U8lQ_d{ zfF_hWQNG3En%os{lrjTTcR7%o91)781#&Tm-*8OuXL1|{j#3`G0V%yvaC0Bt*x+Ur z-T=ih_9i6MLVqIb3mpZAq__v{ue~rK_&gHYYM!A58WSBO#QnSc(rLx$mz+L~W4k$0 z!_7LXBAjk7g`+wL-%NAMHyGL50EmNJ@d~i$@7mH}ywr>}F1PTl;LYqJR57@0@atpo^_BreO)eX{_+XvM=)d_b4&+gj&*(+>&fn1EXC}0F zqut_+V#M@isNsv)6a~6-&2?JbGx8)(#OkXT!$u{Aw-DQ(&YS&jU?eMexI0|{Nu@jS z%NMZ5^YfEhtCs zYXh2*e2zK3mz9ML?%F7(m~0Wd4?-*bfc7)l=y;=)otQD1Zk-KlUd7br%iPj#ue6IG z#SZmpy1ybv#2xBQ52jSNtTVi>ufbJwPdpk9B8=*mwYjj5AGV_N=LM3vd_8Q1izV~j za1C33j87xjA6a70J-d};DE(rQIS%`!mLb$*KV`{YmTgU{$I6OF1=3L{M50g10h0nF zI7->$FNnHHYsQ5aNu^l~D}8{KSPqL1!HXXb(AQ#g4(Z@vm2K>+WFtPS`=3;{fedVY zN|`5|r2u={TjnXw>etpz-d1mTgz^{Czm6HA#nzU*%iOd7yc`|A$@pNHrB*rd4R?1P8`z73>*fKsv8 zOSlfI-lr_2*{c0vc%!>xwA%e?sL2?6M91si&SQZhm*khtH$j7WS|yU*AP)*$5vE~= zQjcn{rSt+FlzgQW=>E_ymHd?-e2~~zlM~h-% zH@uGssM#a!#n4*Pe)gmHkmw@8IMNenEqk5?9oYaZnw73=GC#B~T~|>~3bv|sXb{q5 zzHc2@b~TyH;11*CH;~Q1x^%J`SfpLObd}7dbk*!pMzWuNnb*XfS+s@M+Sn|W z+vK*xnOyuXq2yq$Bhuga$p!R_IKcQx`e`;7Lpvvqo0+Z)W;!Qc23Mgf1)-G8TZ{KW zWTlHs{sj-gyM#JxY%PXg-6_=BKf&MY%!jZ59zW#Qu@NdB`woA!zJ?w0^Aq}kcQ z2{!2uz@hS7B4VkMtEmF%n4ke83*9v!Zhi3@gXLlcyjC}igu?uN`|L3uXUm2>uQMt<^1E|@km2UhCs8ovj6Qq`JZ*qC zn_|`cr}f`M@y74q?%_0gWd)S4xZX0|c)e*w+jQdv_tfLC!%x91r}x%)%q?h>qg3jVThPIE!$J;1XdRLOb-ijB=%b+=t=4qMyc zVO_ehS+zElV7>rn^ZH>0&8qorgkXIO|Can2e|hvg5uoAqcuq5lX%1LWi)oJMU&(2e z;&I5OMV@dF_`OcJxY#dGbEK$pr&1Ris}&q17T)?}^!J5x{!)z9D>e8F8iXyj*}|4g zox7kdh`AcQfU6Za+bOjzJM=P=UZ}Wj1)|PVG$kY04L8j2iY7(R0)XuAFOz+UnFAN9 z#^A@)CaCU&>hfYe@Y!tAE4(Gss3pTZ^4&(Sv`3!rA}0JCo5YQut5MkCDRRJ3j+PTZ zbDXSjPIL*In_TF0frZj;cFnha>OK2x!+=g&R8(u6UJyv3EU*+6|9FLSTF|f6%!HcK zq9UC9HmIwlPA}ewpy8#$4c}wU@mkH#zoDAOedkxhPn)|B!4J?fHxF~+b-^irp(tH% zDV6qM@K5AM*gYL2Y)DhCMBflkO`){3h}8fO63&s9y4X+YE*U}{jwMuVVDRMrLhjU8 zVd@7Lp8VSog`W&n*gRC>TYuv;*rEk-g9DLuccb1G7D|3Nm=E+!I9LdaiHSXYDjwyl zsE{4#<^}igf+E(X70lwxD&qlI1+TtDXVXaZfDTXXDaS~%VnC6s-s>mEXba=d;=_ zKKq5NT)Qh`Kczcl6oEyg`5&O`W1ZdDFI)tVPnicSky>{Cih>p4PM zfLuk_0_P&NvWznr6^Jc7#~r4ZV0}ytrsw7~`S)w7!SwO)MF|m=7W;YT&nWiu<+7N< z{=hJ*fq4=y%4F@pXLhRQaqvtUclHre`}2l?T6y_Bj3ipS*?p%_qreMaWJ#{DU+m+H zwFp>K=^b)udsxQ>MPZ7qH7azcO-DgG?nD9h#qjS=i!S8LQTdDW85fWx{lfiu7QggQ zTyt4I2?J5hx8OBA--efvn4|K?G5nINRm#Wfruo?oKVZ)$l{~Nlogza; zr6pMDp~q3N+hkGR+`dGIF%Mp1zy3?yMEsY3j+=-F3b6;2n@J!|2}by0wXfpuDQ%EOz*QM zav&47@WjLh3Y`LcN32dw`qvO;C7I`w<9aWu5f4>N%O(7Al2YuR6tYsoH(0`ROn7KE zU}~413g?}Hiz}Qf16dW$H9|G!vTI7!qmOXb#X{2@6UFDTHl?Pa3So|kOia}V@8lNm zg2E)X6?l3A!iCLt8d6{@=FCL`d(SX$FVv7Epqp;&rg zWT@99RHMO|w+h(uomfk-DT{^elTu^oHX)_H=n|^`hULw!7+&44>th(X+*@^6sNPIf z+;8b$Gzit3u-4JOP_rSND$Oy4j-CxQo1mx4LjXK@Ur5Mxzo~cEwk!9CVhaNcL$P52 z7dGc|Y*-FTRja3<0a)rwx(M)OPGLJ6rj&TAT7reb_BEyqB^~ecgzY~wWw_q}NNvi5 zc;#VGSx{_Js<8cN<|@v6I#XfP_q;}q3ay4J$m=EngMWEI@*>xE4;V{}gOD@YvO!!Oz45u{&=6J|#9t|u# zJYjGJof99ik7tk~^itJ)+a2lXQaGhEZB8~^%folTckalCfAGz5fUv`Jax|Gl+`E@2 z44EH~U6zp7h+#3F$RQW^Wj+&2~Rp3xBwXut5ksY8-ltc?gXOWiLa1I{)5C?XV4P`Gg z0Vm#MVT&iA1jG`u+zVZ=JAnzI{cqZXE8jb6FW zExQe&ijHKt>m3;qwQ=c*hS8MDu0ZK-47%V##Q2ApSUbAPoHn;_(6;2*VHq zaenM*Eq2Z>=5IkEG#vV!?yg8mFo(@)>(jD9cZTL=+fXe=uEPd^%}r;-0vPj_q@(9^d!C3_nDF4@!4>!_zfT`S>w?CDa}tJj^^QJ7OS{Gn_< zz@l|dL~^gU4~XFV35N9uFHwV<8{{SN0JkXGOKc&RYJdAKVsQ@Qa&>8G3AmruMu2pQ z&ZtZHVh!L!c!J_h;|C-tMggurZ#Hz>URM@cpErJ5@B!KV9M9<-U1JQfTzPFoSt^(7F%5_gM(eU?6IKHPa3N@6 z@$Q|PsgJa`PxGu(#+Ma417piV4;iKfFA(aC@s!eV8wZQyeeh|n-B0Eweqq^w+{FDE zkfeUchk21hP2e}{&xikE`?h|Ytnt?bmE!~*(~(kL}y+=FJ-o_#0A z*cP3;R~^O(nyUR66|jZyg0fv4kVtJPdf%Z^J5Hjmrs@eDO0}o)ppiTzDQap+0TP!w zQS!0hEu+zb&jtABzWfkAw?d*sD>~8ni73ob(AesegNcyK*{l%SX zY4b5!10L(+L79LH+^u|uvaJR*lQ^T;c`hDjhcL52+Q+(SW%eZEQw{Qo@k{kQ9$s1i zIGsqDx7ZS}yM;PinKLUe98odeSGNa`qwY$9PxsFWw09{6Ld2|tUJNK3{0UDEwC2Oa z+?2!88I&BzU(h~FsLM{5XHquyHK>V2C0Ux5o*fG?9}%j`+5_e?XIr3^Oz-LoqFT-* zU^|iTvVHCfmG=r#V8_5ptPPDH58Lf57dL9j<_(h7~It4QY z)s4AMo|R}(L#|LK3}_(+=P3iOuDHy3Fwmw<7V3ua%tEh}pdlg8EF{!*i1KO!`*jKK z{f#i1&G-)Vsw1$Mdeu-6brr#+3^EHFI=#^Ou?pfg$tt%J<((NdQ)P69cmw zTi8ucC6Gc18E|ValMn|F(pJ^}mpkI?em#fqZ3{pcMzrpyVmRdk&Ci7G{O`C2%fL?a zjED`Ij4h$KlJCpw1$3x6Vyqn-cS$`i|7LTKs+KN~iOWv|@g6!eTX^I>a*}TlOZ;#w z2%BDoM|lpM+8M&8KcHY#UZZ+}efi~+DE?T!7ifiz%nU?(t*O9N>v2XNz8o4jCB#n# zbcn*lkwSGDN`U&t3Xd)&37&l}yyOm@8!2q^qtn##4L@^-RHv}%I(qua8h7X{JfY0R z7}$OmKz{af$}*iAQVkCq)JFFBcVrDEm%bL>0RPv8H@wB(IL3v7H*qOp9owfBT*(XG zV#l=tN~64dLmf|_%`W64=s#}T-p?=U54|1jf#wyxtfUVDSX}4f5 z?R%Q$B{D>L(fqO|@BAxfxN0`s#3oN8@hjqX8^&d%#xcx$uVnf$cqSnAC(zvm zvyuiiD|x5#cfa%#CQ8AhF`h;{c+`)X6tL`LA0o*O^TmQjI0*exk1whkeQFb+$5b&I zGy0tWhV!S@hxXZUh+X2BhWn&O46;%*-4N{dksF`IZ^Xb*erGiJsXtsgpOZ5_bY^d5 zE5iv~ux75D-Q$<~#PA|IufvO$0Q|ts$fOnQg%z={R7x$CzVoR&XBAD)4|aKkes}>FlM=b7L)7*k{pl)1T&#PhVsb}Z}f zBQbcHy!)hPFaICn02bm(^+_k-=%wsWpQFQw+Eg*UIBxvzD~O23)1tFy^>}$^#$eQ^ z`JohQQPE0)8uJ^w(a+%NC$zEtHemNld&r4`C<6Bfv~NoN1*c$N7C?ANN zICd3fWN3+rg5lll`o~p5$FrdmIbvwG#p1valGi?t*qGu}pQl3mVDnJ6^)GGKWahc? zT)~y!aAAn~(8vU_ipaAR*|ihmJ9?D#piB~u_wlucD<@5#xF%f`-j9gF9~)NMu&vG4 z>PYDAYz!W$O|R&k7Q9Hrs8h~;0edMWS>uti0~i>m62Mi&COOoK-id3h04R!pJ|-cusMfo;qL{m@qja?DpKuy>)`j=?q5VBKRY7?UH&P+Gt!r_@CM7tx|0nf* zWLSdP)VO`jj+=QF-bbjDh@RRPAwBkNUNb&d?!eKAr}yRU`ohfHvd6Q&~Z8+Jd#=L4Gj{6{4_5(I__b7hA5x~;&jy_y!BM9j-Q5Trd7I9{cV*m+lW^D^B;1 z;iU1PF6odP+0ZsOrawUK%6%2wBEL}Bvfov;-Y_b7Ay9-iEe0A{=uc?W(ju4%wGCyf z&o?R8U2}R2K0$F!Xe-24C*p0W_P;J7L!t>cqtb3Ef4J!8bX_ubYJ!FxVlF7?)FIp{ zK(vkRpG0w$z(z*nJQuJ%ZY>wWRHE~t3#ur*_KAqubM~{}Y)@FxUjXQCRhC%rDU56p zWs79v?!`fA9C~Eq{W!FcW(Znez9?On%v*ZNGI-P_a?6~zmMF1;a~GwKxs3u+6d=w5 z26B&TFNN)*epD49FE5+uJQ6$#e**u&gmx352)5QkJvz+yZO1m-VW*lerV=;sl zyR!g)Hw6#GL);AE>pvhS#tCRk;8Ick5wJo|inLEjqjoGJmEzV*vfujw%Z{Q5K?ZJXM~S}{ zOTK`nt`gu~QMRUh0tCM)N^_;z#-l9ewLkwr%$zG;&|>CDu1x+yr|O1JS03f)EMO#` z@XF4Ck8I$hPT%iQb{Dy~rf}89@obE%G`|yL)nImZtN}x_mJvRh-;*qgP~P6)Zcz&7 zqFc#2J?0A@az9q`IFASMi|e4zPh5yD|Ixtxq7;fZ`%ZM`gUu>+Y66W5LO`!%u}g`* zlMn)%I}*W2Y+wyjzZz3=s0IRM-_#__kJ5c8-t9T~v6=8A)>Wt}mjaHwK_L;&$2PH)8p8YV>KOQ9O{1BD zDcA@BbPCbR+fW1-X&_NU1Jx_!x^CHgD^`YCf%+pQsz_qBzmMdM@hh8>!ObZ?J-m{y z!5_wh7I5gzap;M7XhPM0BWna)d8bG3>%TfpP+;k|Jmb%7&~tpI_;ddiS~^@?p_$qW z#VYB1iCW`JR4dcgXp6Q++0?iPN)lf&?;o^S4m~3iu^a&OueEsb!E1Q%FpR_Sm?-mT z^OAk>ymTnzIG9(srJ1~WgDK2-s4o$Gp}#y&+YP6&fj%5jk0^~2orjP|tyhXM^Src+ zO+3rJg2&SwV9bDrY`S1J489^z@-801{s{^s(lj)V$9Pe}9Cq)vr1|-e!^8}3)`T&~ zyw#DKQBs3jxFMoOc-Y@eR=h24-o6Lz$KUr!H{#&y8})_CC{OKhZxvuipkx)J`^`d6 zjSU$!wHZKd{JbV-ZFa@9sp|#B*r)?)x4ELIIB=>@S5bU*FvYF9P0EGwXun0b>GfI= z*&Fq2=TeHm!x!2hS<5SV`8-li2alAKmKl$f6G`DXv8WV|S4?k=PE5eY_4vb{ymILz z`}|4@sJhJS_%?KO-1Kq5R8P%)d3P%U)n=0>>U52syu0;WZJ7+lE;W>f*I2{}(GPNQ zsJc6R?+y`6ye0IR)!6C|!>;ZKUg@j;XYevndeW6j`Q9!|PeHeAZ}?{>mRikY_MONA z6q!4mdwCVH4@q+zu(&l#0$8O&)oWSW&8GOx9a%gbO4YR(b}K>Cx_MOg2_0%9S+bKA zL(%_Z?@8dBEVBP{6w($*xDQ3bB}FI}3RtXQLK8@(g-VNp0wts=?b0^cbmQ?{#RXBmt6&cRcOl5au-nKP$=M$5)TCB5N-bN&CHj4Nt4ndpu0Z;U*9+L z-psst^XB|!-jEN(Q}Hv^)HI}$C68_uTo{UG(+{IQVT*B) zJEh_dl|K1`W4s}`%Gr9HkX2UdRTvZZ>ML|U@M%RnXSF~QNomB?@B6E&{4gx(+D+AEx7fHZIIJ4AlmBs7MRq- zywjjI;*QW_Zw!zch3^gJg~ionSr8VR9)kdKr~AMSuH**2`uA?sy5w!nXu)j|3^ePN75ZQu5-Qy%Z*tek6Khc`c9v&hUUt%)Zq%PW zXsRHI8gDp@2h5Hb6lVvW5gIUi?r)-mqbBzWId0W`XL^IL>S94~=Fk&H6k zk=DQDA27QmKRDIT;a{wJRk&jas@`T40=(vxR=RRTJ=qU>avE#g7(K|UX$iSc_JZQ) z!pUbdY628EBo(gA_0S7b=?%J=xK}vm{yIuHA_}bJeh}CfAh1V3V7ooq zb$B`!g}o1AUV{S)qwu96xz3;-??I6o^c6;*di4{JP9A&{r(XTeqnQAOgTi^3ms55t zlVCo3GT7w)D!w5t={K0ChvBA1OoFgntQN=4I4H^@j-dF8i5=eSj zW&JX_A6Z{4M_zrvD{g)hUd$q^dmFK2jKW1j`HM(}=NWZtg7mj?emhZjCr)Ybd3&S@ zxWfJWX;6ftSNe-^WU;>p@jUwvI0ys5lt#;0wmC7|`&yFidZiB!njB^97=m|VbSR=eV6>hwnE`!x^j^IG@GZYS z-28}NAF!|08DI`(SSiDga^%9DL7zdjo>BaT4@W>cb=h#}M9{Pev^T;-{Sqpa#fnN{ z_n;beX|E_~EcHI2^idB~Nv!wy{hrh*WfW7ci%Qb77=@h%p;j;K0Xcwb2~Q9O320b3 z^8f`yG497oHBqcoBO_HRu_`~SB(1(t_z4CHv`&#^{Z5xMP~GxLf$TVU@Z2#NWgAE< zRGhynXT9(cQ^P1(sz+|;c;mj*2b847JGpDU2-Pu z&7>xuRhPaO+<>K3l!jDe@bTB-U9L}1%Kb|oE_OQG3&vtYIT5@6BALv7CeGSNhQTSZiBuy!$?Gf~#U+C4whM@N7ENhRZ z8p>Ds+oSG{+9Mb$oeye{95NQ5zD{h9Q0q4lr8%TOcbK$CxyT!Bk5c{Z5p?!F#P$f@ zWBu(Bwnmbs2%0BFEfNH^N2|p4D7l(8M^n8|2b!aMNprNT>E>vo@N}c*XjQo8D4I1# zQ^n@!3(_2Iq0NyiLdlbcXw}3($zPZZCI8Cn{*r%jAuV}-D`cXr(DVLQ=%WYytq_l` z5L8!eg`k#_RtVc5-1R1HgZSB`O5l~ejK^}&es0UJ7J zd`O)j!>%E~04`W%Ac;vhg5$6H$H~$O+To!d-h~*1Qcz#yC5Q4t84(Q6l%f$NU?{CZ zX@YbDe2{uMlGMw#A`d=evA3@g50bu#%|coQX$~;4b)w)s8Pjl?^_dLnF7}wl&+lk6idZ*mw0Ndvr2 z(ouO{pX&2@HF%x9Z?{*aOhBfVw8CXzJCtx*`2uB`Z$ZrB3%EEiU#{3~63TEb!sM&L z4@q;v@4~%hgEponZ%I{8g8twD=CiCfu}DA2`(R*Q=8f}F#9W@!Nfuws z^<>%Bk1l2CEQ%gm|3VOY21Ada=)Uz&2BAkXbQdSxLO`taCKdF{<74QG({Fnqc||2N zBs?UEY$3sgtQ zwv8f& z>SF?*TI!>xK6&7Cg!Uq{qJeN?pBM7vV$CVqE{On2-do_`SHo6(#Hkx7|bbTT}>m zJ&my6b02l#n7U%hMb88uksfYd@H}TD?|AvMN1*RmnLUeaB*-jyhse9r2v$&6EQ?%5 zBLAs@MlL6=Z>VdzoF-LAE>;y@!fnKZY4oq*yOzGM#P?409e2ZfR5p4&Rf7Kn+02cN z?b#)m%4f_nmAmpxewAR1xEK z%Si*dJ6+fm-gez>w~2ceTPOPJ@!s-Op~0kpyiEc{xxh@Uxg^ej(Yo=P`M2ZY$tJC= z9l70aWi0_e4=fB=SYshQ$wf_BSTnKf-M4%GdRGICMRt>&lj-=ObviR(hhMsjMw0E} zFaW72CTok8CNx^J)7SZ-5x9SceC3De+6S3aD=zf;iXZMquDCFA%+MmSqozeWYWhTC zM{Sl#fHDxE3@WS%0Xu5q{&VLmxWxm;Gf%t4b9}+y1K74pVxdro`tgidig6~(i!YIN z)KAldH}P;wx^TfzZX~w!%crQy4&x~nVxw@of_^r)tXfHm_>mjQCbgZOe-p$hZ~Kyq zzK81)E9=kVT%p1D(04GzTSLc2eq)AYtrhPQM*FzfO(Y+2#gO9}^?2Dt^0`M^3)S`S z;Oo&xV#uMj`o9rN$x}4MN}z5yk@5=QY*5L=8oA@JA<5dsRAKDG?=d9JlJVR1OVzil z98tHMWb^)pd!Y?UTJH(qH-V7<#ZUemA%9Z8G@S$bXrNa=<{+QBXqY1sQ-$PGsddpD zGgVkJ+;4BO4VwPB;`SDANn$#S22ybhwM2%eC_^m+mdy1VS@!E~wT3Dh1j>|!*js!^ zYHx9UBDJ>|FXN~|MT4-5*35Kab0DwJC1PHQaYY4LEUHddRHjyBXws6akd3l!{KC`< z6FA+{9Hv(2H1o&6_`K=LslP461R`!A6f3AdwCDNXG*v;f40LDou5{eVJYAZ}gTsir z3~OX0v6>o56sB-!vXDeT%_LUa@N^QjS$UZ6EUB!~o_+Z+nj{z>R^v&5YP>d|8f;FF z!&)ocic@fFQqcG$bidQAP@9H2&x7|9n}+0&S-@=TcK`jR)zpgNe$n=tXvolSSgsW{ zWH^=WfznC#K-tAZinGXmk;KX*8Q%^sIFB~@DvR$>ajw%rHrg_bLM@D4@gpg-LJh@3 zZ25KbO!#Bq^Jq!|`8W_@Imwz)|3i1#+<~y|Yr2T0Xa1YPnPE-;1AS`8M%zy0Blrq`SW4_?@PT zwvR#7sW|aG-U<^-CHH}nD_@oFv1bjMq+)d`Uhbvez`;n?SXS-ogT@*`ei@!y*3r|o zv2yY)HACOG+I6RT-8b!o-TJ<}_2gci#TPgo$MIt-7Q>vzx6X$fKCAei=xo3@Ex!E- zZ?2wOTFJcKV$XZsYw;YlP~F#^?8eWsW4!VuH=V2IxYnGyv=p~U8abH1K7iu|qjDW2 zQrWi(^yeZPH`noM1jT$}a+SL59zvtSYCQoLxRR^qwJSNVoHxDXe3ZHj4^+^m8b(HV z`5ZkzjCmthtU%rJj6I2Ye54hw8W}F)!A~Q^!rxnRUO^8}FNVhJoBwR!Tn6C3ehC=| zRO8o~U7pn^h#hqP0Wz>gpI6Waw`{iYeDw#Y1zi1V>{rv3?)qM2iyof3${5|?<8U9y zwm0!Jc@)PS0QIx!$W_%2KF9zry2vqi=U+l;7m9^r@e*kYC(Q}E-ETP<3}?Bv^5QI&k9;qSsfWAM)~{Bv{3S316f=iGDt2v5y{X7=X9vx^_Zz{NWC(_6e< z*JBJ`&3Y1h^m?{U?`(;s6Q^5MT<&aBoua{JN>}<5uPBbytVi3;Weu)gP(Pr4x`-j{ zsh;Jl$sxi;qxgPPd4*0jXf3%U=mOtWh$Mx0lhrRl*q)0PUCUv>sD&)&|o>|zGPL2`y1&t;l|THAy1nt$(`t=Q za~j@u?Rqn3uZvQ?rbkZ zx$6yN0XPv8g7Hx{rsV6~djO7J{873RO%`=p#pn*bQPJI`N2dqAQw3!;>D7ImzZyV} zqs@G(!B_ur0}i`kZlxMPK2+T8>%5Dgg#(_^Kwz?Th3GbFF}iJ}!dz&Dl_phav%5H_ zeNqDneE$IQow95h#O9gz6pFJg&NvHN(V-Pt{)HwD2W>iEXIVdT_F(Kc(Ri}S#P&lAZHJ;qgg7J%CUfTSKx~ zzez)v(C|DF9mPaFc_Lc8AuBEs)D&;!7H|!GVOLMoa(|ECkWCJ}o~P4=-ky1MOKmTY z`*%pAMw$l;P#v7$dtB~N@v=jQADY9G)!c7*o9zC{1Ng!aKZJ((W-R*(N4 z#(niK_|JUz^4v%`BS&L~{!ViIyI&cE>t}#@VFl6lTF)?!ot(h~;76gfWM$+4I635v zH=`!*@J#G2-kFTnZf4Gq-nmO=Acu_Pr1_R*e(`OXNeOY~G`dc58sjir*ZH?^FQ@httavvQFlw8S$9`ky6MlAD}#a+w9vhbwPw1+1i z#;bEHQe@tNxg~B^B-6QEQqeL0SU5NPI^WqF2NQDjQyVd9mgoDlIG{jV+SpgK)rcxO zNP-D+;#NJqwizvt(hi!~bYPZsGKk@Z+~gW{>0imzIla!4kk&wM(MuDSCy^}-dtDvX z?@rN(2iWuRID^-sem7mSj@}uw%{h>)8+cy)0~wUGLUOMjBf&S5ums10>eZdvdk0f? zOch2gnYgA^7&WOdZzW@rhPHT;e2ND5U<1?^KYX5EU|Yv#fDc^g!2Elj_loII{)cWN z@}IgfRjBmVd(QQ0z`-bPg*7@z70WaM>>jXb9Z{Lq9!YrNR0>)lwHh+WtG7!jMM)0p(DP^^ zaV23~0PkR%ndg(EE#jf0KExyr!IZ$$3%U-Ei%__Zc(8d9d4qBxTb5P$pp!sv zRi>|7+>)(JF(udP@$6T!+tF3@J+%m&^B#dALj6GK0Q}x}g12)J&dF!vC-$!5o@cHY z^-@JPN~^r%07ms31I*q5iLFhgcV-B)Mw)tJuJn)+5rLZy@I5RS&qW_cA;y!d z&WcYV@3+P$15}rm(J-}V)SsWpRp0v+U^c5CyOsDJ?mP!*POGQ)|AvMcG5-?`a{#Np zI;;OyaHR<~roK^=0H4;k?Nk7%dyb_l+>mk+jN)(YZ^aOg9;JC6oU8VBew)PQZTkj> zsc75pYm&sa(|Yx^Hf~^L4?J!_ct1gerso;LgQo3&2%L5Zo^9|{VJ<;7JvS2nsci*{ zuVXYdhr{se6@@B8LE-*N_0yqbdoUS_>=!8#A|IWbe^z#D74G;uJ*uEzoSm z9K$<4$Kg2%e*KFn+wm(Xt0HbMdq0V3-AL2;4AKJzH6wmJ6vYbva8DY3`bNNmBE)Tk zX9tS|GB#}&JV#ij-0*CaW(rD3MzhiR!Xd))Nz{ZH7(J*H-V%_XDLLwOKtm-;j#`c{ zyG*fB?;a$n7FFghm++|WWx%i6m79+_Ylm+-cHt3tyV&yupxyk#=VvxwI2@mi?-y>3 zzxN>Os~)s@^PX?M0w2)r6QI37(^){8i_`meJVa?Qtp2Z`kCi0@v)=rnNbSzEK|__4J+AcD3( z9-x7TlQ3J~2SH#4xpL^98>QmqXajDk_2~4MOkK+RYbdZUg`R z8pc>0fz!U-d}iuaNORAo&4;HQ2e_BK@0)gV^WmwVZ9Y8x za}?iMn!-l#s{^{W_aN7Yr}dwS@WiyJdnoKVhU8}ICU`(JQ#VRDMXA~l?Va9v4#GXo zy~8)xAMU*EBP@its?B?Ti2wQkDg5v2bubGh`r#S^IcPiXBcgGOfDO+NDlFQN>g(4( zq)wvLQ;>Q`F!gd!%VO9{?8Iv{Pq8K49;IY81Fszg>@H%h?cY1uBGNG&01rBkpRAUl$xi(SYP=6m zEqG?#?H_1sJo`$-Gt~9C`*R&G@xfsH%aF~-_VLy2GXUwQe!Pl24T81k?+vhwPacJohuIEbg z@}9Gg`lH+aC=wj-b^ZYe$`$qRHAwCa@ucH7fCHXc3LFuf-FPRTxO96(hNk$KOl_>9 zbF>;2owL;_OAjHCpWUcMD8s3nQoYX8QJj>j*PZTIKY=4-BK2EPwm77KNnq8T{D`Gh zFbUN3laV=K6`T82(Yc_$v=*WKY!yblRd%+DL|ordaBmE(VNx4KqJ0gma7g%e@d=oH zDm+WbM$ei=pJ8dG{y(^&CsbEV(ir<}R2$v;_sBl+vK=HVdiA zPx^JJC&sdEBxm~d9NEaZi{=ma9d6)efj0Hs&G%b;fzxsv-<^GPX`@fx?NYbJHEyp; zAG^t`(8q3BrvokweYfdL>UI2$`o;B?iwJ4S1)ZxsMA^+7i-n``V*K5O?dSINt?7Fv zb{9@N9)qNpy4Pzk8%K40wt&PAgAjCQ-73DdKK5h!uH;qfb>0~2kR##%eV5_;A$*6d zc65T{2s#8eF)GLR(y)l#otqCpN*soY^Y|xi|427{9vUvhaY&URsA{Jld@X`ifWcsor zSVG@)6gKMnR`Y#F(~=oY3Z|_-x!R>!yjv`pAF*T_7N5YPDe>wcUtUrwu_YIEt_~1O zH*XuDW~K5?ce(=1>a6e(vH757=2&)w{3QK#$(JhiTQ~L-l>LBSwGpS5>bIaCY$P>e zZ6#FC-hRJa^^mBT5283{Cwx=E+q{_5CUtxb{JF&)&fXVJQBy4kQ-$+n{o7c!mEHVU zzmyZH;eA%3A64kY{C{J1A0&zKsX4m^sDT; zN*`X+Rbyf|cEN-~2jk85CJ+3XdLXS_SL@l)ieqNb@1fRcT0?2iX+;J=)C6M2{gV9p zZAl(938za=^b#(h`gbhdyHHJM@8=ueGfFh6MRu2Y_?P1LJO>NG?3ssJL|U8i6ORS+V^ zT_i ztn0E?MmH0QMU{) zwLtfe#31$OJ~hX=KaX0h?ZuH5M@@7BHMZ?aDlutgB@gR3J`sJ^9m8%U1MOHR2(-jZ z8p_-(Vz;|@#w=pXx?Q;`!;5uNqhE6ZmzVTfecwv7L$4=+O*FJADEJhOFm@Ir8#{|~ z(8W}qk>UBA47$@Q(lx?v7GYj{=B*JyZE($^SxmDAT) zc$LxDHiE3APQfQ&h3bWW+F_-NGcArKgMwa2(-!^=YUk~U@9prtEl6!~U*Dg7J|d9T zf#3%6R~m~QdbMa4cJjO;GP?#7Kt#iKv@5C%ki z|Fdo#uk{=QVsFXT8l~i_1 zRa6?w4)Zq&8Ew*%_c~$(lS;8IEqRZl9Uk^6sf;rwZ+9F|tPC0rNo{N#G*yh=Vfq4Z zI`RzX$mY2|XnzWKAMEx#>LUhhd9|SH=l1kER|^$1#E=H*Q{8LR??W@YFjs)IOcoQ` zmrSf&JSxKo!`C+Ja9>pJsJpvCPfYe?xee-fw~j28f*&oX&=84R7bijQNB93yf0`16FyW?rlF(;^rA(}i zW$V*3>JLD%1~?EA2{Y<9_z^6!JkLvmY4TcWGGv@a<9Z@ANCNfG3rZYKKtd}GJgnvLZlQFY=lCCpAuf2+gpww(NN5he9TXCIg6GJfFp&%C8I+qPzY7i^ z6I7mrVZJbVmizO(VsUVu$q^PYdk+R7*S7YDnjMUY@*`41q6>)X)*wvr`#7fO#Ji3= zEDpzID}S`7i-Xb&6xqQbMB-NJyD^wt=_miY9|7&#Nz$B;E_v9%@i3SvBhwO#DzHIQ z=ZkK%L6gyeWP2>5WuXgJS{YSmzRlvKo(O|KsLTkR%1J{8RRu=MJc!0qg2H#=uj|@J>AkW}^3{GTl zEQ1LQ_F=FagDM8k{)45<;1LFE8C=cevyH(|MSXjN!kZYpoxv;yf5YG-3@&Hz6$by! z;5G(NGsv-YJ1`i>;1~ucGMK~Q0}L)@@J|e`WU!J!)my>!q$35z8U|w+R4~|;!RJ|i zpJ0$@{!RwlG1!_xe*(2#(84M=YfU!-#b3rOIdN+&6mqLoL3r_(^(hVi@e?%s2jeN6 z^{;MW>A73fVf~c@DE+2CQn=)^F5$yp|6TuG|6TuG|6TuG|6TuG|6TuG|NZKJod!^- z-ABjw-I+UyxqC9ViMelL?pe%zGjrRSJC3=RFn1zzzr@_bnR^X$k6`Wt%srC1&olRJ z%-!t+>P`~*WA0JRox$9On0o?q=P`F0 zb1!G^bmo4IxhIP9VD3p``plid+}uaheK&J!n0vCw4|8XV{4jSGb5CXNDI$N&eGhXN zGxxpBy@I*tiu^Hm0dvIJJnkIvg%LZ_Uv< z7dm-sq1Iw9DB#ALbF{hn1y*g5t8li}!DUz-h51frzO6``V=c61?tiN^T zJ80I@!NWYqs^x+20$b4>F&`NY8*j7N3bgYb;H%{q+O67Ln?ozci}J9GG&VcN9~`d8 z#%rA}yWQsCF(t+)i6;aVlFuM4W=BjiIR0F7VSd5FMsev`9wVUrc!@M!PAk-JbI|}^ zYlf1aQ$z#=a?~Z*$Dm<~y}Hw)sU;x2p)EvgDa(7g!0M)?zCvvRX)HATVV; z)0WH6H#@ALb&y_;l*T+8xD(}`5)xKBd!g2DwK_!nM4NWDD>v5)C1E#PtXlItbAAC$ zn&SfckRQk)ONmK3fJe~;a5Vo;YZ2r*m$!-qt##(Pc#`2z{dQP^32TwX>KqJtB7yA% z`4(&AeDL{&SdcEhQT}pK6d~~zSm#*_oLXBhRMW!QHi%MNkY6-ctPkTzT{RbmYOKlT zY`QkxZKhIFWw?D87ZaIjCw_3+?>E;PgSMAx}=LGlVZ-)#1eOPK)-pE*o#w zS{GQX)*Ml;ycrmqI4+a6CVA$f98xA>^5w7=+JG^v$7ViHELWyatZgD{B`w=~^NX-V zwDW)qQq5gFiOXCxhlu>%Og*sR;9Yi-v4yCH(7yTW*LVkLORlTHU!WoF2$3I#%(c0S za)|s;Y}y<%ld6%QeO8I9c zNf2UaEtCteW@Xg09UMaS1@*Lf#FReAuS(4bG!c~*%C80S+ie9z5%+VG?H2~RxrdwPNXi0czt^*ox$eaWM!_FCepg^-g zmz;L1B|jJXWNm7OzcYjIOc6*5!E{BeRu!6^bAu@ZdYhXM+@h)mT7BB`u>3+RQDk$? z$pgSUZ1#oNqCmR|y{gOVS6gnH&E){^9IN&>E+^I^$6Ra9U?PcyHkX#P{QhiO7g#MW zh+qy@LQL9{mtT-0D(n@A&u&JHG{RuHup-3v(#k{M<*!x|=}=Nq+F-gM{%JD|#qVe9 z()1+y&@f^;STV7^3xvm3!MeaM4r1V!0c~Htxd1y&R{`%N0;0{sY?ogUL_(+q7F(gc zz{*?cI0pJ|XdOwX=T8BeFCZ#Jc|-SO<YJ@-VWG|CELbR(W0UkQk`?61*#deq zt@5OU!g2u#lakZIz}n^*48i~>$lL`ofl(A@2g(84>3l4j{0AT=LQ3@7U%n71*HuJ^ z1X!-LV}KT4$^q$3fYM%I=CPOI>@ZY=s-1(XMuHTYc}pJEmpS?Kth93_}s@e8kAxl17G%(E6Cc@dveP%s1@MmG#1yPHkC@e4Vc z;hGN51MuLTIk>fYKRn;V(@{c~49|V=EPw}1YpjCj19;BB(;n#1%;-dT`Y#AAuTXdM z^q}!iPX34fWHPx-L4I->{5YBa$AOPrE{ptA$Q25i;u82@`d7f;F^3-mDdSgNUoGnK5*_MibcTK{-A56i&hqFwRCQm%#d;n;Un$hRD8AE+$ zwt~;+R`fZUkAJ_m;ot9E`1c~8XLA%aNYb0*Fwe7A>di2Eu)6g;w3611DHdx^enGX7 z;A)Hnw>6cxcBB&5&WVI_&m`ixcM|d3H;H(DILTRHcIF+JLU0GC5Zs|@1bkv9o^@v8 zI%XyTPFP5KUswswVd!+IJn={>dj7XJl|(hd@Zpz4JBmyJ{j=h^|MwcICP@f$!S2 zed7eL@h?)n5B(*4%GR41G%>iGLGCjOhrqw-oD=>P_+d6Y`P^*4SU7BJ^=WAg6VHtS z9~)Q1DFpu|WzYl^>X z6#nYws|9@gTz;=~IxUue4mXrQ2hdqLJCue6>V;Tm;=)kQhS+$hUuK{lLf`^Yo2=Y# z0Tw}>Duj?@;GdQo&Lxrh6bJERtqb-SQ%Uf*N0H=YxqUjWRFFko=ky{g&o7MUEk$Hz zGJUoUdqfBmk4tO$mg&6NF~`bF{4IG7TOstxNC0DVl14ro(hteI9{(>Me@pB0uZ^Go z8vmc$K3`)#B8BfQ;n2w?5@?4OrU+W8eE2&DV#D^y4loDaczA{Yz7TwPZXvWs1FtII zC^bwATRo)G0>{|WVe41`|Bwp@xF3RlS~5;)UWiMzlh^&PjsK;_*A@;f-_3_Ib#QZ` zH0_{&7E-d)p+vDKL@kVmzZP(#e&%!Yz<)aE5r&~UEb6$E=r7V(fCAqgY!n-w9(!;H z(WiXU8;dc_2CP-0zo^Z(0<{BpLH)9U3uEGe%0c)l019jm3n8BDCUo)|w#OE$gZCFi zIRAte(*N4{|DW&=+fG`DUh$Ha@>j?Af4hG(<`G&av;#IWC(eU+ z+RKWw>pS4df|+zC%m=ce@6G1!;f&m5aHn#UxQV}5{-^X*bK*~ecUsAQB^or6|NqeV zxmo!QuP076_Ml}iU5^OX0{Vt}Lbau3@!_v`2<~#rt)=N^rKU_UW@)n~P12_6CyzIZ z=_(aBF6z2Sp^z(Nik}*M4OwZK^M(#d<*qxvl!9p^!Bj`X3Snrb&9#~+c;rv3Q>XEuKD za^$d@{9o@MBsU)ZIzes@HlBCD$X=>_9L%H4gvQDR|8UeLu7=pis-!qK=nbPb7lg}Y zWsyx{$cMj9Xq_?KV4y;nO-6UrP2&i~3=Gt>7V`abxX5O+8k|EeApW9$Nz)6a5@QLh z$qu|Mz9W^}&&!XKg$?8<;0~7cmEyN?eKtcJW+J!wWc<&Qx@`ftfZ&Y)?IceYf}Kd} zFpdP$m}JAQzqOR#E0jOFHjiV8$uPFa=1jo9o->efMK)lJ!2fu_n1I_s5)Hz<*`aN- zlJ!e*Wj|Q2oJa^Of??%uLgo3t;5OHl~YPC9YOGeXYY|$(a3f@dXmy z#qsy(Fy)dCr7{&@2Kee>Bt9{ubgoc7u8@4gwV0R68RjJg{H8%3v%qBlze#NFA!_C2 zwoA><=T-I>f0Hxy@fn$!96rVrL#95MFC1Ul9r^R-S{KrQY}~1}yJq8-aNLPTb|Vky z?;OxnII_cAYo!~>w74%hHy5|n_kX~JJVY25+DZF3kDCkfS{Nirv4_lO^F;bJ2)`^9 zkkwZ!(P;m+x=ZEKwWt#Lu85}lZB!P)Z!wDQQbXfD^&oevo z0{iR4m?MpcuU)?F;B$L_vXe9K_WsU@^iJ1owz*8)?;lc#mq;&?f1^Ej+JpuD6NW8d zv0I%i_Mams>1dbs!};6;5F$=GPHiFI*Zr@ozr$(Hw$HUVhh`V%jL3H8nQ8Yf@uz-4 zZc-b=(2?31#@}CCp*^zX049(Qb6YKTh}02?J>UIlS6(sM#%>oi-X zEQb1vEQb1vELwAhjTkX>RDe$wk>AZC@Vi++w85o=M<%E5cuBWRN#Fm1?@{#qWqgO1 z1J_#Bymk@WAJz`F zSZ`!Lnji6((K$Nk+@UB3@i(he2woYCuMEXU1>;YJ;#&pdwIS)FT*JjL(nP7k$1mbr zslvxE;>9wFls~bY{wxchKe6n#%fsgn%M^T=bZ%Jyvr`+JSLTtiK_U4j7`i(x6UMT7GK)%J4~g$;uOGJ{e=U_qOX=e# zPMT;O$TczkyfW!zWKO`bwaJ)GN7xvix+jzIcEF94E$0VAO_?}h;-q^gUKRZ@laW(i zBMmac{3w%~z+Fx~8E>3uoSd5SGt(bW=4v=EONAL*OZh-KUyj_ctQaKahw>NmBd*Jb zx)Tk?%($T=lLoRjM~}{7#L3Mig-`iGf9UF&KTQea2mkRY=`;wui*7;+^AIj2&kX;E zCB$dg3KkAcOu#b^6KzFgnrUW?O1~S3a~i!-PbRaX|5$pc83*~!LNh&9Am$U&g!F0o z^^YHx=+sVFFuH$yl2fa90UYks8jCFOGRmn49X>R()j8ap6fxOLPDpL#HVHWBnG1^$k$k*pbnN zmRlXepJ%X>giact_6CaAF*rz)u2e3048MfI!4f*DTvjrCErYj8(v`+{p5eRQNMVA6 zP8y$%;b$>8g3*QQ)e?q(jlq!;I%$4u8UB<=zax-tP`UMFa$)fG9fABx>Ff-@oWXzW z2&5~e!>Sz5sxtnd*^f^do02H{1=V}(6tay5V>VE*`cK=~Kny|jfXoK0gdFLi4`_lO zuu^*HOX67?;2Rtca1qmC;V`YH!-b^B(t^G!UR*wloG+ltW>hRK7A`QM51T(04ymv| ziw~11h6|I7pFT7(kv=4PN}poV!+-2-C#6lp&Nh%Y3@4(qMRax`>!hoXCp~Jsw5&0l zWQ-WG974kT%RXZGkn*76Cx@gi4d0Zug@l)AUWD)=sfW#fkZv?72T8$&kwcKKG${{B zfi{vyh^96n7pWRGmP?4{HX$FWda-yS)lz9ZP1V+rcpBB(i19=yFU(gXrH_=arqn;E z+``2ew%nSCF_6zo@#oKHll=ATK-+g;$WN5ptlr)I>o4K(!#nuLZDH|gm%!)sZW?cA zcx43qg5ClCr96uGrzQE5hrxev34F06eq|W^l1uQv{1W&(It1h!j{mFPELoe+l1dyDuzO=ne4v`N+0((6k7Q_ zTj>4<@mIN_HECB~a#L72HHyv;XHd9&8iksYRs^nGMB(yM3hfV5sGUXWRx((};7e8( zPcDV_Iwg(o7=?8V79XeZB?ebASjpfi2J0B)PEe?2P{&{fgLVds8GMPsl?+xgSjQmu zDdUSl@wdD`ZFlVhyNC7H-ptlN6yfRR5&UHl{KM;q*j_H+MsNw-s1V)2AEgjif+uoQ zxHR}D?qQXlCKPwmiE#yMofyN-k~TVUIw?CssFw3f`G~~t)#5U!UM?7sFe+Sqy{7(T z0kk0BZ=uymHrPAni54|m8lU*nw5IhfL&fSLgS8@T#rzr6Ml;xs)qe(y)vP{uVEB$K zJcG*_)OKR9EAwZtt_y|D)sL3hCAfy+Bb7rl;}uT;Mb_z-#rJdb*=&CP$MSu(!sC9} zOk-wd>ZFO<4Eg=2t5xH>>oD2fgRYi1rFqU9y#-x z??1zeb|%FA?Bf2>Y=66)6yU3c=`@a2jFawFwUSWb(!Mrw5Pwa}BcHq#a4vGPJcs#& z?()7I`CO}V=nXC#+2uxg+PNg zi^NlV!F%e4ksbrNPXSjd;aSWJu3JoMqz~7;yncopIIem2obpJW5Un}2EdEQ!p34=G zxl|rGpJF3pDr5$NfRd;o4{Xdxy!hq_JiJ=(s+N| z)$tG89!k$-hwST(5DynNzqm4uW$PqwcnV33Em+T_7A%6~iT9H@^UQPo3+GqNznOeC z)>oV@0INV$zlb)R@<|9QJKZd`bs_3%sD-BHrW=WG(f)^Y)q#$7tYoy43+-06-{z`l zCraEAd=J|TJdT^@AM+Y7OMWS7sm6&}*^~7X4U^J;b>-E3`f&1WijO%qcD+phoWa`p zW@7Z8xe5Ln*_Zj*@@#H>%_i#?`Rv?er0)&c{%=s?i9<6xwNFnT#)Xyhz%cx=Mexw| z?f+f;6_I~)?8^k!D?&qH&zn!I+L`@j*_aDz6|sQy!~v@*zqtIag2x+B9NAX0qhMh+UdPqa_-ROob)MCcpSzHK{UTPy@PZFByVoQ@k5~R= z<24VqY!-ywj1-QJ#YV`Rz=x#Pm`*~Fjg7d=*@&+B{5L@vfxo%dh1u2xmOSw~wI=zK zh@(k)WaqGhFd^k_Ey`iV)uddcbWQOWAwE`kO{brmUzG2}E1e{G>X)5ckUu9coIbLA z5+h~V(A$#B86|jj%}h4smK{II#(QBR%c}`K7^i+sZ^RFgvcxSODQaFlz07{1vAwv! z3K_0^2ag+V{;v~GVmIM>JO8V}UuOB!^Mbg-)zs<4UqSwUA^C{X$Y*OBbO)C728v%d z|8}66%Y|C*p(QJ|MMT|nmF-tiUYEdM>enbdZJ;B^-%|V~%R{(oAQNQK} z)wXNE|8@Lz{B``V7k`=U3$@Y|`h|t|?2W&@Tz)!{_8CXAT>Fd5^Jn65Tu!~%JlAZo zSna&jpU$t?C|^2{6vJ5yU}79HRceg)-^}W46uv0@JT(kH?0hT?UD)|pr1V1PfnoC9 zRJv?78=f*kd`GFF#UT1{leu8JOV0bG@fBGYK#@6Yye)EiMb?7s{G2A`7Bp`R zBmZ!CHm?**sPVj#(fQ|<%6O1vTOQi>(GyFozrOPMrTnoy96BU%u-GW31Uis=Li-Ow#}?%-NtZ$!d~K>BOu5tV z`*s*vg?_6EoKG?P*Vu5!QRqrTbK?_#;l^o!-AoRy@xHj`?Py|hbtY!p=SpYgznFhB z&^8(-7<65if=b2R441 zoJ01ShU#Ya zj4M=*()O0#%pBUz=0RwO1pjNR57$T^es+FkQh$DE@HL@f{d;mey&pfBd*glGklz1e&QWQHnMkDym>7k$KdaKlgPO;9Q$XJ(SHgz0cHbPjqY}8 zE+3KPaB2P;-xGORd|svU3EVefAuR-sd7ApIyg==W)c(Nii(F*(Mt-EQYY4qvN$r=^ zGI%PALVGI;WxingQ!E~B1BJzMX8%M%VN5VNr@M{CpG4vE+Zkkh#)f8~KgH`9JSDns zrm$}ac`f79u40hItBs~^O$dELR~BElV0wEG>W&JbuZv~;wxKYgErlx?Y#TzqQcRb{ z`%+ABeust7Yh#1yMSetjJ~5cSvK^(LrDpPJ&*a~M!VV$vS9Wa8>`YKt+?m2U1~b@# zT=5Mo9tL$cGQN7U{4&^1*I}1-ip}SIMMn~HjzZ3{|sFr*}oWW-XQ*@l56T*aqwt1n?QQTP0yuEtUA!UG(|fQ z;{M1~a7VEF@H6FzdP%>-1kSGxZ9(|P^G#Y8tYow=j`|zlAbYLK|CefyoSc>ETn;C_ z#e&X6#g-t{ANh@>vB;~PtnW#8+6K2d^u#A>V(1C|UrW9H)z+75-F{uNfB8AuD%Q5R zF!9;hxr%(EdCo@ql*>{KoE#SWir%aSH>pooy#89r{nWG{4dy4*X5`P!;gw5Iy0+7F zdRJOr|1J9SOY+;i(rKo?U*mS;iuY$%iO;LSuS~9x<35+b7MH-azhZimtxi{=9WUvn z?u__h!|+}Pax>_#;f|ry&wr1J-H~5t1t%H`&xjA)Q{uldl+ceDHdNwIZVZL@sq~6Z zZUKWS#*~b^hXo-1{@d*(UG4a1eQ*<9cPV1)DG#!Bm_=;;Yz|vbdw{LyID+8;7T(Un z*Rt@Z9tsX$%d>TwV&?vB>n7~IQmX*jOe3-V2-pP)Z$Ut`)YUXLu#dx8K=u|p^UMxn z^I+@Gtfq1il)bef9`%vbSbs;;kteEGYy^7-m z@%uksY~Zk3Yz}s@!7rBX72-#_x03g-dj_v2inc4ood-?vb=Ai&QX2?X7-LG$u2^## zjb{V<;;qCMr_=@_blCoJG1<>;EwW%P^5@|`f2VeKeo;<{J^UcLE9nmjfrN4*;;jqp z+CsCF7x}SjgMBPotCn||b93`4y+xbnlzKQr+!n3b9*hrmGkaS`{p7T1*;$kI*vCL6OZD}O9LlYXKh)1;qZ42?6>IQbsq;y5LM*haB8IvYw0p>E( zW1`c#>~uXD^+#G>k;3^Em;D?e^BpK+ zb{-^*Eh2}NcLYokh0w7xe->L&$WhX8@r9lRjg&5^uVbN|x5AtF71G%heK;FIO{Cw* zF43jw0%tTE<-ZZBXa*@VzoBO&ujDUW`QT&A4Yp(zjxO9u*XGehl#j($XfLp$Z7HfF ze&a?VbpDgNA^!fepI5scwCr^Fudv=XQw~ELjZ}hv;`#%AyR?xnFZ8UAg^c0EFQK6) z^}~e{>4N1oRN6WH3jJxH3VLpFpm{C24Guvv1%C-m#eb#zTgd0tjz97~kZ+uO^-9Awg;LTXEbGV zA}n8FZBI1r$C|Qj5GLL*d^Wb{C!rT@H-zh=BgWrUKiyng6JhBkwjY{^uc`bKITcyw zhqY5Ep=;FNhuV6$Qte6TeqnK4q3Q2bIbpw3B{UIU{9W7o9w?tqvbJImT6@TxoRyJH zuN2Rw_l*zDP6EV=6@mMf$?p$=`7J5tvdu5vxHw_xu*BgbM&6b*Np}gU9SJG|E~Y8|Nduw(zehi`8B{J`m?523s8$8G$fP^+%he8tQzcGRe*1@35YM}s>W z+|iI8;NbxX=>c8^cz6JUJ6ekjCUAr?NKXasXmCe^I~v@8hp-9o0EF}a4`CDF0SNAB zEfawpGXRgAqc3vI2|RM_j^L3i!3htzwcrMSz{4ZwD1n^o2e2PNN(gR%{eTYe@W_>b zhX>pc4)99A!y{+n?*P$i;OW4M0o(w!+71w015XDCrhy0CT5y9u;Nj^2X=~u=0BLLB z0XKwGf`31N{eTYe@N|H*HSmBN!T}!A2Di3DKu44S>1ahpyp#Zy)jx zKZJ+JUjvBBGfhN|Br2|{JOE_? zWeK4I3e7O!Ae;v1+5>D4;WQ9V1F${7_6amN{)6;2kUqHK0SM`Xo74&oq_2VWHIP2I z;QtUzMb)#X#C}cw(^Tzyo+W;N^e^w-Vd{;faB?B{1 zF9*CF@IVjY-~l|i0R}n=NL{9)E*Z%Z{3SJj77f-4nTjY1xUr@H#O!OSYH)D?<4|1y z3lA3uFiu8_4~q`)8o+}a!odT0aFh0fHZYL3f{0rNZWI|nNCVYhfz<%g20XZB;0Axd zDNwwSwgMhGq^*EQ4rwdkkwe@HNCWU1z-s`n0X(=N96W#rx7a-@=}b+jl+yxH%4tC; zS-+qpTCW8>CIxOexZzP!G`9MrXHfP5H#`890F_GGjbK*+c%bhCZg>C!9^9xJCa?9Ysx86E*=&)hKhu?NvQc`mA=+R@w=yWM5M&r0~si}9}m6kSX(%pAYo}86+ z@4Zu}-hcm$88c^^&9i6c8P4?VPKQE_oeNm19ft5&UEy>{)ob??2mVT0RURkdkTb#+b6_U$`&?%GvbyKmozAMW3O@Zh0CAAfxG zXkFcj6Q6zd`RDcZCr^I)S3jd&=D3nmXOr^+0J%Oz%fu`1;m8^WV-m z^<33c_aB}sFGydzf7@M8)x4E;=I;64O=a(m%uD>g!^^5?E{eP1z5|9n!{#1vtQ>Ok z_CYf{wEw^VY;H0g z!%c@C{PWCP5^o(btU>m%QdT|kQQHl}cdxm7b=-&V9m*NoX3SRa;FtD&_fxx{4y?Xw z)Z)9g>6Vwj^vcO0Q;SN}9d7+Tz2?l&6w58!KdygQ`0HyQZb_W@(5;U5V#X~sF6r{* zd;i+@VB&l2XZTt=&wuXPbw(;dP zw<}A2ysPsc5=+Yd(e+){faQFA348;WB`}~JDU+wy*(lcwfRsCq`e|IRa;ba^s zU(KNZ@$I3vzIuGj7oVxHP>mi?>xrZd_%`J$f5vxQe*STXIKzV6YiTau>8PtVveWnjVW zSu>Ly^J3l|+4aa>&)z)PHsen#HvIng9}hhLUC;O9p6~Qf{h^^9SDji@88ho&yLvvd z_{9fOdOo!LWV`H0j8|LI7k_tK{L!gB+WycK9PE8kE4 z*M`<3oJW_89z6WpZ;ZV@n457Y$89NF*6LlQvVP~E#%#~+`^-CI|CqCQRMefTdY_N` z=d8Kuqu2dq!p!;Kd@(Jqp{IMm%f8o+B%0oKm;Gk!3l|=FWN6iA@8`<8jT-oy1P~JS+_Z3t?bS>TboDRVSnsp zN0n~L`lsjpc~5S}CFStL-H$z9vfCf=i5!&ox-kHb)(Z~%`Iv@ar3@)XI|{)n5Ecu+uv^C z_q_c5GFPh+%LXl0HnJJ=-}Jkq)9c+nNcdu*Z1q>!KR&b0rRhI={vhYYg``ONW|MtfAPt#668$bTL{XhJ+bavMvdq4T?$jE}ZmNoaKjY|7h>G$U8o8DS_ z!}^Ca*FODx&X>#ERL!gX`>GqCuIU?99`$VNt(r;ezPRg!1n;0F-p{_t+w9Ei^}r*K zzpfk6rG8KRf9Ad9yeoJ9nA^sW(vA7LVE82dttoH*wd?&oo{jm?IkI=wvEKa5uao4Ow=UwU-t(}ZIQ{OPdgW&PBlgc_W(T769UpJ z2)iQ;wW>b!13nVN8Bw|uO`TgmFwjRR-Fv28^72Awl|jjO+&h!+X9y&271{0TkEfZG z(M<6tRWzG##kDF%Yy2$mD)5kx4I^c;9*JYOcS$W%siU3s6}q0!eKO-A@Vj8XgnK*^ zcHahf@)GP;=parek8a6zL8zx$?Z4)$0PDy^1f0MuYvIK3fG}I&6f<9>dUv-T<3(N0 z?R&3Q8ZWVkngMPrn_q+d%~@s$$R5I-(c2g-If2g)iiY`^Ned-N)+~KCQ(&d|XWi5SQwMskdRGM*k1ms0x3MPH(;NO34l?UXVwHmCA z6{PbYbC8^5@sMynh)$F%VTE=Y$YFowo|6wHJ!GvOiK-0@j<_Y^BLI_VXm@GSry92e zGg>st_1@mAUN|@Ioze*;R^fxoci-VoLtDEX*%GV_H&%(t_RG zw=K0N!Z#v=o>nQo+vjB}J{jcbyBn%5+y*BVTLoaex7fULOMc>3?jy_taFc=LBy=MCJ7UCDAmZ-0Ua=oHv zJPS{cuUslix_g8hf08C9O*sMu*~f2e=$d7vx)Q1WZlU2Nx;i#!tU)YwcOrzi{KJr#doRuN+bqY!3dS(n_gyxdsz}~CGH^iF?uu7DUp|o+$)9;kA@am zx`WbP*_nU@PWxN_w(im5T>yl5K^*UbuFL=b7wpucW?rd^KgD5@(BS^X;RNAc2(mT7 z>`X3Jee{DzWwPOPFgrh5Y)~p~{9dwX+T(`qF;j0Lcb>$eCjN( zsf9ohG4N@2ihFa)Ud3HmpNBWttQn)E zuxD%Sp0vSluk-OTGntj+E-FK1>q2LiC}Nhh{>hC?P&xEN5@XDk?lHH?S%?M_Xqop` z!;O@C2!J2$GOgrcICZhudM3%{`}Ti@>&3?!5c~2~sH#hu7=R=l;S|q=2*rwGi-|c} z^+c&(d+GfY$67AYFn|~BwEp6k^eQX_6LY_=*FtL3*Zofj>&endYRC63OQnjRa9)WC zy0-w5So^o#Z^qC&yG-kN2+@}r>&+z%B@v zm)olHYHNS#p*^V4J~=%dXevv9$N8tG9=8lFNQ-Dncv4tEU(B?rGT~;+YP4NmfLM|Z zI+nii;<*2$Vl z6dvLWf5iqHz0i@yZdX^*5Xy)G4hofUBw(C57ibN}+P}m@EKH9D4EJj!r<-9)jsE7b!Lt(R6?!aSB%EK&* z&PeGG7cnDIJM%6gh5ci6Y&~Fn+{f^(NTSu%IVdrF7{?EVY)0j(?T&s?)#$EBtn zL;m_(&WW64dL{qqeP&qEu23xV5tcQZ&lF&g3$g%9J$<~1(_EY(ygj7c`yLeI$B0Dx`Zjke_1fzj@8o%qHX>7;bEo3zRYdSbs-X;EEm_crt=* zG<{F$8lo$Bmdi)Sr86t{X+$N(feiS^-u5ncwdUC@{>wVo*^ur@v9)zTx}I*JnloH@ z;3fF0TMYxD+%YMn!-U4pV{TZ%$zrQQD?hYaW*tLCU4!>kz_Q3!W%#$jIHNrSmtd#L z6LmZ&206$!sZU}G$MRZn?5`ke{fnpTbeWb*4%Q*r%vA>>6Uwf85Kfev)RXO9QJsMV zC;i=RZuq~O62`4}FhMkHpaa;uDP>8gy#YwotwRGx0y#j)@!wh(W}o#rx5e1E z``0E5<8_@<&h>Su!dx;z>BsMd)2#L%LJO5432EF@gaqgRCt-Fdvd~QbE8DkE2lM|+qljR5Vzu*iGZHJmYhVS=6c@mMmiMCYjL+j6O3ZRltI(i$z(q}9PErx zKi^&+y1)zK)EGlAh3g&KL!>`k4ZaT2qaGx4r%nquWnUUz98vLMVjO6KJ>sRAEll;I zgX1yR1X)CbZu62R!1+b5X~YgLwb+c}DV*#6O&0*{sN@?Tapvury{cjWIgxWU?H? zL4Ov2@)IFaZ!fq#m*KD6(!5o%gHlFAS=yodQ_m8zj$1;YUwl>gE>SK|Y}Ldi`EMw} zQOG9l>JN&$eh}8@(r}N$s%2v5M8x2Ju{IeUq<8W?>y?%2KWG^z%&dKLO|^ z6nc$*KtaThYV%f=%JUz)L;zqEg+qjDW3EwjN(?``D)X8jTf)lUOCj>HhTGr)td_N; z!dse72rO&@fp(IED9Xxu=Q7AXDO)(lSL*&7IIkZ4;}U`f%#6=!FqksIS)Qya+wuvy zDtKYG4F*V80eYLqGA66VCUyY;#>Pb2@Pg{6=m2c;ubL_34amxw;fvRPw&QQs%>s>N zUvghC8?rxQ#2f81tr=r_K2vb|KKoz3Emo@+K}Tm+lu0)Rj;cmn)=t!E(PPl4)71bW zHhj^d>L-`Ww^QAU8>=f!1ZP4O4C#cR>gzYC)BK8e_;=)d0ryd6i+cM;m`P%w8E%U0 zC}v6 zfL1(5qPs-eW`t$7^V=(jfX5pRkuR2^b@4Ujr}bFvVc^$pGopzgz@&N?6SO(yI2XRR3bExwi6twVkcAq`OHs7pB!lEFUqgZ9D(Mhk$yw|1@|u^XhN3 zII-4sZ6>Pth=Bm?Ak#Sd{6=YM`j2f$ZKcb(*v9)%6Eu9rTm?U!nXtmss$tUA6U-zQ(jGx>q(tC&6C1zZAl6^^-*w+r@cLM3KU7WjW~V- z_S;5Turf>T(P-n!sr|&U^Rlr0u%`h~Zp1n+bd&XMR_3!g@^$t~mXee?5P{=FwXq#G zdxB$I^{;Lq?ITk@kU}5A(r_b`p@rGtF0ab+ZG<^;OU5i2_!48TM~@492mxZ2ksro= z>3!P5Tl+A$zSH;*-$UlmL6(Xbx_9m|0@e1ss-ucd4-rY4GG+VuAhOmi_7{WETI$16 zLG3(#N}N}H6ly6A7dK4pKBd+*qtXsN&p?8or8-pQ1x*c%w6rtb`@7Xej?#Y()}jB^ za%9~OYzv8rV|dX%8g8Pksys+z6(?^e{dZt5*-Vn=JZ?q`nEs~)A1#ovu$0H~ECi^6 zh&CbRn1UzGug@{rK&O}W;0l@sd=Odd^RwFuxQ?n`9+f_rc#gCYwWwp}#qtA+4a_kX z;&I{;9>I5YkeW6KTsX2b__Pigo5heRfUJ(jMPZtO6rjW|Vr_VOjXJC#zODK*FfF=T zffu4NW%Z=nLMNRdyAP^W=Zkcp$+;eOzA!{DxHMB=bI`vi3<4(-igR4K%f+|E@t9aM+4zEn2yBg zigc`iVkmTn@&;=dB0$3N3Iudt!_+OTSIO%9aqF5uBkaUgg4eoF>~?#KVeMJ7U`lW8 zCP{P><1LX(e~ljESz2)XSz1@(4!kYx-qi7OxF&h)xg;w+301z$vyAw^3uN8oubK1E zMeTMkFyUW3J^N%g^kOw{Ew)iH81q_t^W~h^#|ypH4~b^%AZxGv3{+!a>}OnakIVVx z!dbFVtHzl>noo1Mg}kSu)f3G4S#}}C1+&3`R4Tzv)MQis{zVgLUId4|e~B+xvHp&K zSu>K=x4+)4)hsxfnE3ULBf@(V!;YqMpnhdyCDe@ zm(Mee!XPzq08xzTTw12IS`XbV9f#~}i|3-7e+wTaUn)AzG=xebZ^ZQ_VSj=qz+O~H z+`b}e++4e{pkJ;fkGZ-0FKCn2)7RyZ=zjt-cy^$37kx|dVcp-rvzLVUNKcF^!0y8L zA?8*$^mA3JNVz2#K%x`Ca)pOuMY97Ogh4uG>MKW}fSSe%CoUrN_EaAAC2a{bk){8S z%SfgGe^Rz-+^mOAw9JL#H{f*zQqSE`dgS-to-e0XaC`M33Je(&raYtz35y?;g=um!-qs5dZA4pvcK?Nd-VC%+_x1 zbB#|Mu6wPuxvC1_9r3hb*YfYwo7OUW54MOxac8cGBj=$vv|*GLjIIh-iAOyj+HshF zz!;H+c;Yqoc-s3ltQ}P0K{k(bU*y`rItYlCI{Xm7rDL2I@adsl_j!Z2L z1X_5ZfS)?jSp#KKD3!&St*oJR0j&x8T)$NdDOxQEgn*ktXB-ctaEHO%E_GZwrNbqN zFCVW_p2}@#$+%$!Bt}>9uRXd;Kko z$Hj7{@^Wyv;>A-9$PvrxGFe0aox<88JZT6-ow>a?55k!!wlC)PB+hM{g*8aWYP~t`SgP8CKMz(AgGRNEZ@-=RKhIsI}Nq6CKM^Rx@?65IqvlY zt|tA5y5znB%$ByC)(?UP1Q-%ih3OIkX~N&737QWU`hRa`7naSuGtC3VqcC^6=f={lJ}+-?Ma4e#$ynV6h5DZ{`}4R?6ao;!I%3i z@xm~1Je#D>h-BI6PXsMS!JvYHF4a>@>FJRJ)7`d&2^w^_ILtFleyt%%FH8ke12&GFbv0+Gq{9()%(D z!KOj;+D4JrplI(KIiU-#I!kwrW2Lyp*Q5)Gy$ACg5jB7r$GtPazQc%!JA3)5UeLyc zysl_H>lry2or<`_;|`0&NCz-2!A7SCe7pvA^H*(dd_ze58ma2fhZWabB~rY>2n6k< zZ6X;b+eW$2LoJPUV$Gfqn{`nqT1d!sdKcuJNnf|HJq*El9vY@k*ZgOp0MlZ?s1FZy zOo#JL$?e~R`7Ev}8e@lzG;u@t(#R=v5f@jla*}NnXIRk~x0`lp&pri%f~$GNR=8fp zvh=Va9C`O%B^60Mx>i}Lx(?!>!j^hK(^@L!yJ&IpiJcw4vkJCDdU6hN?@?UjAwru8 zc2dgUS)o@Yd_cyHl~@CjHG{1VGTc?1SGk=EcD!NV(>4d>+l+TVz!|0!kD_kHq#@aD zv`3~dGbE*UUEEu&kr-Jc0o=#BavDniQ-t4QVd4duUnV=|-$lSWr`uDl8#F(PT4H;H z%xS`oi4o&Ef+rf+AYtiMqxrOt&vlspBlt>yn<_+IS5G1%c_zl}o;yt(vEApO`y$$r zVWtFZO)_w!7i~9Qw$l)oSK~x5@R*G49B?9JzUySe9M&^;QC)vD!zQTCxkOggNL8X$ zt>DIhryxRv9b~5pUP`0o4CL5}KL+NmVgbECB*uCS(`n`f=Uo?$wQ0m3S zp})_Bfd-amO!2zYLJ8Fk1B5vyW2P2IoP)RIz-`{=l=tz~&ZjD3)b!9??Eb7-@$;2Z z2L?WsQZ`;HP&;KI9~S0u3Rt|PSNwD}Qxd%4-4&3)s@HVUvPXMM1YM!?=-6^V?FahS zf7G|`v9G=D`JXFqivD0X6&A%96Krhozvh;DanPa;5CnQn&ZXiDn@*9T&&8wISK}sE z6Dzwa!|az4-H5I+e}4^Z{yw|?8V#})IY3x&CaMO_N0;2EX~}mY71&jHhn1$hDzk`D zIHI13og55yiz3-K+8A7$5xeil#oP87$QuoRTae}lOE`5%hRvuustd)(;9Zcy`#{dH zQW+z9a|VQPCf-BOVxkhM!FF5g%-uMgvR4UWkiN5fRi>vFZ+H<^|W%gHd0=*SiuVMaVRb*=`ct1M78 zn`o*>Pqv#)t`v~?dmqO!oN6y1T>ft8uLJpV|M%GuA3bU}Cp+QzAHS7)%KHIFrO}j`Kx@2uY;>zztl0KYASyLy3cabYY%w#1>y;t`A}Oy# ztmM5tY43gL{+{rHWB4=EkNwkc zW1N)SDk~F(d$Dd&tD^>fD>ui|Z@|8DrfrG3RZsWi544_+`2I?}ux1_R$i8(g3ot3~ zw4}WPTB>UT-l)2ZKtz9_{k~NH^D2729gCa2a!R*Psmf98mzIZPOL~*J)rThRAf1|g z6!7(KPvmnK0BH>A67;-^9u`7@BG7v{D*v|pRW~_MWAAP4R{ay0Y!Fj@f@IHA!FnG< zy)_z{uV3b4-E`vdfOyPAJo-QOfoUatYeE`9;2@-v{kfR7{`ga_z}@|cs*5=?A_s1E zF`)chLhfH6@|*&CK~tZk4T&Q_xM+;1sH!(}K~N{W^n|kxtz^@lFR803g>Ql2h>GI` zK^dMd)zYs$O7!km*qH5?<@(VzvH1rh35;S;#-+>Fu1eSmk*2DM#~TM{1Iy;GsaNVP z0jzjEv>V~yQL5K}j6sHSS{e6u33tExkAv&2XJN>WEdfuvsD%>hKdU~P12u@&!UT{o z_#|EKUg`f6c1v!lq&_0O68_%|Q<`(O9SDOIUDT>lZ4Hd-EhuoZ&EB&Ff-SeuO0xZjZVYlQ~1VpiFl1YTgrg3msq?l#a3 zvsN*~tZ@OQE*F4G>Li`;?~dGd#&RD{?nFIa+^P)bj~$zrruc3ECXINP9Ey-&!0vA* zTlolXPRZyZ!Y71bWm*(jIWu8BH)BBhXv3S$pi+u0;~UX}kj0^wCwqtXCELsOn}K?R zNdgJp*wIfaUk;gc=Y1+-8&O4hprsFlEreKQzI3@TrqJ^jWQ^Z%ZRA3AAT@w|BgpLJ zfBsQ&nyi{$%QTVYaa{!PGzq&AP94kJwb|uUk^J701*w;f`O`*uCzB}tb0S9vB)u}> zhOV|A3{mUuXHZ%2#RzwIo&uK=&B)>yUNHH)TnS+;+17vXvOkJ;dUhU#DbWEl3_OVb)4)G@#}NB-DFA_!l3uAlVP45akO zk+KillHCn+!P#jphiTVIR1i*ehudOy9g!r8b{3~#5E%Sn(TZyJsL@jB9wQfIWxXpG zAF-=Dz2|KW_($W&h;QC+!<^ZhM|D^nP-zD=&`#xex`S?cH1qJ6dWIxP&f(kS7fHBa zuu)&Vd77|1Kmb!W*h3fp_~1X=+o^9=*pztfzb`=}X(1;1k&=75+u%D^G_s!zz__Sv z0yuWPscJy&p*=8VpoSE}#4hH?9x1M2J~Nr$k_0MZ;EsB`oebE2+pYm@ywfh+>~y{e z5?aGGA7`IN8p}Y{$M%HJPVH>bLk|fpocm#gw4Ebwc|B=cWDa&viP<#(>i;SZCZOl!DxT5Q*LWZ|Cn&|jn-#Ul( zjvNdJal_NQDM3LEk1=@r3p?3roTDKo_ZR7@!gQUNCi9nmih0cfO&AP<;K&yV2 zjR<6skh5SXw#D>OQ`HSLk0UtJ$uS+HkEHZfPoV%wud_oLISr&0-Ej?g7p~UbPIVI! zbPsNEAP_^0!j-`PF35RiGAXRk^sTclc6z%3({RZwW)2T@FCWKn%p+?nYhhee*NY0Y z%g-o9B6qSXGnRw@n#o!M*(Z?%JL6+3pwtx88AaGA=Mcllx`e#5gNe%z7)WC{VIsvW z3ocBw^b{CSqubw9s)6D6jB#1#@Q>Bcs0X#v04_k$zZBaH`PkYE#g0e4SE~8tP~w-t zDOJb|-S^mf`J>jpn!V#q zl0Tin<^ZTbq4bi@jR&tc6CO?Ug;Gq`35&o}SxJ}Q^`GRmp?zO-Qn-(5Giztvn%lNA zv!ohPw4jKKRaMN@+fs*k3B6Ly*ql4G7Hesu!l1>TdgVo32v0y=e=?p*S1C7YMzo(SUyQQ}q6|7cUO6e{{wl*RB)!Rzgzr$)TZywC zm}mTcHq}rCIw%=_?Of+Pt7~^IHY3F(9|+@+=yvbS{(`n)xsq+Vi}ewXi^O#!D@@ER zkNRx*;!=8l4Kr5$o(j{H{WSSeRBI%SEdXjxB;&?3B2bL%-3?nS-dk^Xzgj%yECqWt zU~xclZ+bt!aThmPvY$!}L>y4khd@)Xhv8BML^4cbA;^5cx7*G7BF?5&LGL%1h|4HK zT5;A-{nplp?+&5b!hUEoqL%Bx14JTXb>^G#i-gfKcJyC!KazQr@Y^`&oF;kq2BmR4I?5VzjS{jxYo zlyQX3{7l+{qT#`0G4uaY85nT)2$cYD{Kznx5UQR+nXbXTVnfw^1I zg5-sminP-5PG5$n8s#f99#R?&DHvA`Hm);=Dnjw0;QH$pJ{RxPw1V4i)$G$dgZ|- z$dOu#+`OvrZsvcBNo>xj^mwexaHloS99TKB_M47KgAuT7*KnXg16U>-t_GcK+ER>U z-7xzBx6z;bQY@fMnT9Y{Zx=C>2CHj@7XX1PJ#M69S;$DA=*MB6sdc!DWwaNCONVbM zZdS${{M`J+Ht%haqoF5K-bIN)Vpm`3pWbZQ{ed3IAl`sP+J5BQTDIbCIS)4ppuNZJ z@Jl^dm~sXksFyZBtgw0ipJ!Fvhb$cnO## zrQ)tNQr#y44z+}0!3~2r6Bc>(rOH1N(rb!KZTLpT`HH4)BQ}j-`g3U0!4qvmjg{nf z&98v4k0HX@)U`jY=qMLlX5ezwvrWDp&mdk*SZZ{rr!{cgxuH&eCYy{@xz80kS@ROm zOEn>GUT%|*jB3m=-`qjMwFFK0iiZ1MG;a!-W6NT>xnWN4ZwMxGB?*r<_Qv%b6Wd@C z6Rh{N`a3o2Kq=_%H-L!8g--Il!~cO5OsK`f^~-htTR?A4p+CszHOk>GGdypb=nEtRtts^*$lIm9nEsl#g5KpXmR2Z_R`Zd?VA~2=3|OcrN)$@R zA51?WvzR1iJ6d)*Yz@!jkqgQ^tj#*xqEI`P%Lp%zC>qa0&n#FJB&g}`V7!a$jY6QB zasr>nRMqt|W$e`)*k9p-&pI%2reg)1aj36qH=oB}yxh`?&C?8Z%b(jnFpjVoL@x#Z z6d+|SKF@?u_X*Wb6!`GHewbVG)AH~;^f%a4qt9`zhH^FZCX)UqxL8MmU0bWu-@%lS zQ##UJF=#zvW_shmh2WaudvbYPSG6_`-b%Bt4UAZm3+Zv;t8!Auz@&x2xL^`B+AA5;XGJA)TWoM zZWyzC;YRG-bVHKEY(KZcR4u^IGo0CM@c}jP?fYx`M)(&ErOup$fnpzi8xfFtwAF`TuzLe1d05q^fb>GpISBR zz7YPAYz`hVKarTLk~jd;!l~(uKEd33CGI=Lh>4di8#)xa&6?P?3TyN2rNV5b!Q0VgY2zYMWjC78i7w{apYG559NK#IR*Z>lR>ebXp`3i*j<{s2` zK$m zTq9g$rhCq#zOKbg^tWgpizkC(_tPmqFnX;q$ICv@SE-y@{->9`K@8QvmLz{qo9CZq z#P4UBJ=lTQ{f!_jUBkzcc(Ytou6$4SixTO3Ml6OOz|1f`$LU_k^v6E9YQFTQANf2! zP8>XXptHUJ3`OtwhdxXV0JFm4Fk8O}{*()#8dC2hlBzSoGF@opGP@SvI7Id4SZUy? zXvXWs&G)ZbNnaC-H$|qD_|Jag2?{k9^PLy>SRNql?7Cri={zUwE-t>4)~+nvR#4R7k%7!!C9v4*Q3zhNpZXVAjQ)(`_!tkOf!mwU??5) zEDxkgqVIEPJ2$%#EV48LRHJlE?cHVrEnV`(?%HUn`9d+aXJ`;$Hq^G)Xe@2b2SO=? z3<*#7sP3_7cLWr5;^eMk{)v8HOC}8miFa|Q#>a~s-A<13p%)*XMg~t`>fm_W`zj>- zc=3n@i;~61$sUq575$*Mwl%Khy^-5n%w!n<5?)9ww5ph^+Y>$!p(GG{p_@jX-a0Kt zz&DHZBaH`Kd^hD%H@Qw!72Rs6<9VX{fTyk#@U;OT@@%(Ti>r|+@D#ci=F=Hslcs`^ z(;3TAu9WfNTui42znF!O$Gq{&BJx;qZ5Yk~;~{H=J6$IcEik70HJh6wpf+6MUqIOf z-(9o5*bOp%AFwO1P3j<{ZDhki2BV1+f?fKRZ9R_7k4Tqkl0(6^&KK`*F)Ma6*+IDc zXZRP35jB2;yCF!@pD$ICOOvOpMh?eot{X9V<>avN92WNiSiUjFjZN;`y>Hx`2?dL< z46-XV%qt1GMkUocks_ZtJOja(u1=Yz+Z%TpYIT;`S6{(y;C=@K>7V06a3r@xq`PE@R0>_ z7>FwIY7z1y$N@yb4fB8-rQ`c8`*)8d-cjeT4CO>x?;+CM)SP^ob0k28-qD8rGP{JE zpxipO)zR4P+!1<>0E8^FbjZyzbDPG;czGt$SEB5?V zM6L2I$E8C}3zt);NWSK9>c8_KSZQIcDv$Ozp54+A^nm+U%=GI&N9$|A4Dhe(_06I%X+hDdfSrkuia!v2Z{5?dr;}o-e6>P zxlqV}#aDr(eWb$fuCs^5f`3j6KU19XspxGSBTnOVX>+|7SE-Ls@fOnI*(9Ob+F>IR zpGb2%f-zncfN4Nutkv+?O<2O0BF8V-<1uvZobJzWC#kN$D_$G6jxm%n-qnw0Pky-M zII$|D0}Joaa~DN7;e*LKh35B08dr+vXfE`(14@GZwR$13kQvA^9E5O)~|(x#Qnen}h}I-pd&Df3Wl(#0s4t5MR> zQG5u%GA7#`4CXwnn+2kLRY%#B&CUI>otV`d8%eR)qX+@o4$LnVRuEGIhX6I_Xh7ULKVxC&%;|WdI6|oh^pX;?DxDiV(Ol1T z9LIj`leEF4@krc|)cuVid324cvp#MB0B(H+m=4|AxXM9am z=RUu->xa;bJ$Aae)_@YmO)WV7Ms>Ztwdkigz5XcjxG@trs{lhB&U~i{2dvIF^HzQ& zYQBhKV;k}`j|Ic0$aKp6odlkd>Kpy+ZTOKHj*V#Y&3>l%&_v${<`B?W3rpXH+4T4% z48=6~SFQoSpjPj8>2o8)p{$i-QxghIbE+p0HN4epUe>ko((UU#A|#$uD{&pF4(v>s z)Md{dW*s<#DnQ3ndoS%8U+$&d)}=q{3p8QOHiWiMo!vvE!1KGgjw*dSI4%xQ{8v(U zTO4Aodcb^-3Uo~T)__D}~-Xlv(H z-vzLC@uzK3?8n_Ldb?9qS9AJ0Rik{!H&Mb>P5eJ!F*RL2uPQ`F=qUe{WJwXZVZ&`Q z`qP!nBkC;t7$4g79a(*&(A=88Bpz-JGE)F?w$Vg`VSZ-$maj!*phDn$%sRY{))a*; zpj%oCDC7yDPQPIZw5+E=SNdzEYByQhO3`Jv0!5MQe}WaQ=DwF6R`r58+P0;{yJ+-A zwe49Z3?MCGS*qsexZvm+tRVh9*o-+fxL?FDg~CCKv%jbYVcXpGHKzIKA?=vt$~SRv zIRKv|*fco8aG*COtBd2atmlK1dwBw`@1BIBKkD~h(k;C|{U&0XA zjpD!=&Z0QJW+G7jOq?ew)VSd!4luQQS%1n?!MBFqJiTLmaI+87s2#$tgBR- zd$#}Qm@sN`>G*vV*E2QFum2X}wLt9gFPXPcwf$4+p@%zRYYp$aGA zF4}?hF5Keu0+lNG@8mFL0g3L`kNrKl%m0srMuyqb7T!$;sxm>+O5x}%4Z3cW8T`vfim&svVrJ|9M>IFP3SqVS&aqtc84l*1 zM7a)s6%^a}Io7{nJX_4J{{v?sK^@ejrNSIeBJHb>e^c2ShlYV#4XLRu()V-+@eP}b z%y_1_MgJ_ry&~tr*`)$Pjr2(C1WgAs!rF$l}V6%-y4j)gM#mtP71g&y3e?sQ@$ zUA9?Z48V0zxz6gF>?nBo9oMQo7k7A*sI?r0zdUbb0hGmquu{{;P|q-}JLmc1+A0r+I^!(|$bKHbr7 zanX@-R?`MlUu>L7l|SOBTRGbOan5cD)ebW2&lcTg$}6dRzI|Hz#%t3EM&QsbZCau3 z<7%UqkL#nbbspY>9LyA$^*_Qh9I(a7RyIMZZ=@Y;sZnJy-gF?pMvrWDAA4?wJ8SKg z6xb(sa_8KFwnIWWvhxszbxuXNillOVg@n#S!VfpU+i?vtd&>9VcsKf*|36ZD>fq5C z0YZI@1q)F`DQOh)z;$%DCYVmHkCa07P2;=*MBC$PKy6qK{ zTFBiiknBsAhO^AU2LXd6w$JMHcq0h;R!KVFOM7=6bbwF~x?uN<^TTl+2?ASUn-A49 zmSNt>xI?$>10YA|T~rcvw6Yw|bjk0XpYJlg0BTQ4cOs93bN7X#td3Rq^A^|IQ~7bk zldv>L)VDR)k|+&768c@O4X91>h2_fWM@1e zl+CLzjPhfL7&XYQyR3%~s8o%e^5B*%V_vBMOb({7Df;OV;6Q)zE+cZc7180q_A*Ye z6k;sws*L4?J=lm=)7x={l}risDyD)4b{T&EPkca{<9BefOd;d&Pe;XB5C6zC8>Td$ z7N;A$QkD?^D?~L|gSR78qJp=`3f0wBU6RPNE$kv@|3bElYeDDyNZjMsX^ffHG0?8u z$^?L~+dKsw*g^ptfW{7ES^SByl#VQRuQZts5(K2i{iK?rQ%ISRp&sDa8G)AQ*YT_T zJU1;9P7S3YGZA5^>0@X~&}F=%24UU|jPH)0{7GzvuR&3N{|+@~Aidvh<%vd$Pdb<~ z{s#;j>dUx1hhlUwE1x2hE!^#mWmxZI4^6M{WvIstbA2d7{&0N+J#q}ZuE{1X**T@} zk|t%flvjl31}y+#2@kkjK@YLqc4fuoiT+uV|AJ||i)bdinp=73PqB|@9yImTwYj2w zd%p7lW(8N!-WmDAAE zY#ajZ@7BcZlA1GjYOwjK2};I@aTROIhggw`L};195buI+?J1&JSWd6&W#c&W7OrQl zUWXIY473aHR4CGO9LLC|bTLu2OlqxZe{dP>l`oG-(O`stIdkaV8Z`py< z`-pD9Dm0U%HC%*B%>vE4@l@W4`shsUbtPwG9W*N-IMVx9wy3%gHLxFg(4Go-M#93MRBaU$1w6`u zcVIC;mBJHL+1P0;UX|m5(tF(c0JEPM>4#8z{(&djE%aPB7uy6`3eT3<_af#<{ z<)DB(4)oIz5T?Ziw~Zl_7_r0N{!pGU*-jl?ZA}nL@!UcaX;oV$Yz4=i1d#-V;OGkC zy2?!HutJy8(OPe;9lLJ#9H~fZl#s!UOeU3~Iq+k$XYP;Z7!q&82NGXvUNEZ0T>rzfr!wOt|k4k3Y3 za5UvOoD7e8@xP&MM-onf^mu@2!7~$P;lavFO+s%tTb|xPSICD>Crl<%FKmstskW$m z4OK+B(G&?6#2@^H3|JY}##D2l9x z>mm?g*?;qhvYvjsK8ELx%4%V^YHnBqwAVQc(U%SDUmkbUhZcIGWrE~TKFAZZRLTCS zb&p{m4KYWLF3jN2S7pV4y7dQQuRaWD`6UYoR6_}7T=^XS@_VJfB~tpBeO%%Mz{Oy% zngJZYVKsCV6}lInZ(ho7Pp~xk!rmUsnkJ!W<7AVh-3;W}G7|;fBj0Jy)?lDHlK`T9 z(Pkq#X$pnO%Aaj&eC_;K@VqQ$weZEr#ls{|y4=`}F(vF#`%{#Uk zD-01!NQ~m+@~U5f%Nn%tpSU$ykjCM3Zdjdft+Ip!xM!_{h)CN#oHoazighvlnxAP- zw~Dd8Azmmg{1Ev$91hzebpc<>1bg!|Ovhey3pG zVS|-r-p)lenW6R~eA2pc)+9E}0jrL}X`y#0`eSy3ur37oUXn5ks#DLSFFSgj#Bp2e zd$NDx1<^DZ4z>+fXitcj2-d8^`_|x02_sSUUxA@FfODJ??`ylUbK1HF_3kjl_S{Mk z@8f&^Wuv+c;7`oYDMsB7mraNk>o~wKLOR_Q!w&OyU16|ddO?(DLYEGo4)P9~@ zY-Y^Y+m6}XKl)SG%*KbiU`9{2<{=;m>6$z+wtge_MrP^so)McV*Hon@dVBIlKR}J}H_OMs4jM!JxRfsxGQ;B(g}Ztx z4d^p0?elCBIkvBWAdTKSRpVg3w)Y7uejJ~wP&q5`5M)^Se}u3xif8qFdxs#lm^rQi z1J#zZ55UD&LUX2a<^-S^`}t9bb?EEZi~mHrnO(-@WFEJZ{e9oAj!TSd=LDE4FS;`{ z1?ajAd3}YIEIptl=g95#*v)pt#aJEwsqa#`2GOY#{|;MsTs$zjSKzurBqY5Ix5?C4 z^y!+o&7(*Zw`oGxE)Moa?m?4u20%p+KUW09RGt4+;jsdBI*n>W&?+H;8bd9rO786u zZ-}e2IL)^XN&}{z)K^$u==Z&Ix6%|JT${w2RZ1)0R6ZKMM8uI8r0oRPsuPIEwO;Id zlFc-VjN}%DJnLy@R7(=bdK>KcO<9)s6&PiBB6*yr#)rM#R-5pm04 zorA||wemfr>HcKsxv)+KyLUm#_)lY>B>fU7S`?aSZ}_afw>n;5(*Y;!ubYvoWkO|4 z1t>!PUIPS!v82b|IhN+IhiIj)Qa!Eqxbzc>8kfwIlKTfulT2Nn6+bDwtO>E{P%J6Y zRB`5I>j*wvw=jwI?@yB)NtEI*xC^^tuZem$tk|FAg_e*MwbvPcNd$YE%Bw%odKwDt zH5=A$$UY$+U~BPq49>ayk=o%WtJix_D4u&;lw(z;8GI+0{HmewLu-*+r8>P&J4BDj zVrgYAE0)Tz4bIf4ofA)NjT_wsKe;5B3mEs`oOid!SFYVZqPrp`lN<6vQ#7MWZB_n& zoc&?DC1tt?IiNF%A^+hJusC}WQmRz8X29y^@{}MM6ppg;G7ZXoqGlI2iv*MT$7vj{ zc(Iz8TEA-@PgYov`bkdSAJY|X>?i!^c5-tA&P)Qo)`@v-2xNKL%}hq{a#}G~cSmM5 zoRB4P-F0ic2ONz;pof>d#-|#Beo!)0$+8k%K~(#~eNOHT_q@aCC}wT}D>ou_3btp~paixEhgzz#;vJ=jh? zOKxub(wS{BrbA2w&sK?vovF89KCVZz? zj*^2X%P}4Ab{&Aogr%Mvgo)nwjFpcq5aek4gY1*B`PgU`&RJ-S|#BwPG5HM zYU%Q7&0&|kXway8G2o7vOD3ypeUcus59#0skD;Obr@MeF{Q$LR&6OvAI3V{2m(%B@ ziCfMGM2SDGgO4&{B5agf0-!s=2KmQhmB%HpdrmQrR9_R(#_Er2Oe=OgfexHMex|wf z;XizFf;kQa*d`fRhFBnP@b?U4^O?fyW-1l}VcZBFdUkce@=_aznOPbRW(AW*JZ)kZ z4{Pds?KdaZ=kRSJTertp7sWeaksm0As6gP1v%9Cdorr<0+`VN>K;R!|$%{J8y#94l zAMRJ9`TLkN^C+-s!Chq!>2e*xez36~tNKbif3AvC7hhf(aM?iSfdRf;uu?rSLndOq z-t~eX^IcSd6n4cEo;ou-;3FRl-cL!1b^jpjw&lJWioa~z;fMQpB5;po?KOI>N5t~X z&;vNC)JOw-nt?d8tbokU7jB-&$D4v3z5|PEb4HdgL%?8rUDhu|wcexW>)oPh|9uBj zqIn`Q&jOTfr8f!tBaZqdjodz!xOGt#zcITWX6aZ=fa6Vw1{LTuw0|%h}$m5 zhKBJAOkFpV4KiImc|0Yx^d@b8)`c66`{mUj}249 z756$J6Vylwe~;VjOTcY_H6Zneg=IJczHt?E3alM+_C}7Ot9v5@bfw9*G1$W5P~#mghkOI%3?A7)V#Uk#bZK|6-uWmu9F~W=P<D7UOM)l1gpnBvHr06&EyREkDV8B>D8~6`kDq! z_V%I@Ae~;n&;Y>Ai!~oGKj!#HoeG(rXa_pfLr*WBXdeU#l8_cER80q`pq$l{qON6t z#)?J$VbF{KkN_LMGZ40IiojTvbSX5@)xlZ(wRGSW;%D~h68Gi+0*1W|QvHY}qCT{& zDFQCY@c&HV(VTOh##v%ZXorcp%$6MHh1!#3(4zvoT`fEpYv?-DD)Qs-pQoiBEdz*<^D&--DR|*=KYLiN0!6fu_6!wa{Jy=7W$zSItFz^9&80^p9bbEm zY#=%%!c4kFMtKOHI9UJK7uASzgrDvWqE8BUm)F>Gpz9`{Lcbl!U~_qbuYj|o0CMBp z+JaD#9BPz@UBjo-<__h$Rs&dNE(HA|Gh~fvS)XSarcf2Gd6OQP$^2rjxn#yPDQz*< z)2{1fPF@1DR3s<_mH9{Z39HPCBq9StNF4?#W7>;LS>1R4~46dg50ZUEP1&w~miZeK>Y# zO7-L|cE<^nN1S*Evzc96O}>IQd2Ut=I3?pi$J+BZEnu$XW);5b4HVBR)30aHfke+7 z_uG_F%oIA3I7T)Rks z2;;Ao4sTXOh<6nNYQLurF=>owwX@#R`UI6i$rnv+i5KPYAmCq6b_uJX3v@sF z<-3hSq|9f%CNW?Ahk+(o5J)X-Q00UEN3S z5%za5htf$pFME`-4lqT16_k_|oEC0nDl4gaq| zM`733-xA#Rvfx%MnmcD*_}40izD5+Xf)Cx;umi@+1M&GZ3`QBA*BAp_F7x;oL8!Wj zh>Le7a=_d-JdY<2931Hz(h$a=_uy!ReAM8;?5z_b@SB;7v_cYnstcAtLP%X^%NzS# z`|H+xgo_Q))S6bV!iOej;|(IQ%kd$--#=(P1m|GXc}{AA&`Zmr(My=+^$~7+{|YrNiuFj5o2jjB0;^Uv?oC+tl%2bwF5U9)525&nsq>|`%~Db?&b4+@_$%y%4n8{98v`uZ_Yat)w$%p4*gTj zP%Rekqs7)0A;=+;2fCgAKA9Y-T}xgp+T?}(Gl1LMiJFlpi7UUJfne376kQ=JkW(R~ z{nt@Y!)k;w$YF)JYRnh^O!qUP>lE@6xaHI|IHFq(I#$*yfnT}+X2t<4YVUZn|-u#>6CMG zGg%?|oMsfgSHENs8VujAU4|u*Cxg=qw?3=d>@=TSJ-n}6bzgq%R~iUJ`Y%TK`_g&sTaZ_yZy6gnlFE$Z*{1_MWP`jYbuc<@Jf^?zGVIc zSSrS4yFNVgUa>CV4m8;!ttAPaHg zdXU}Cn+!`+G2esEtvOV%KLl!oCfGv|1DTWaM6uSusbMFCaYsggwAzaggKnGu7vDz{ zn&jwyjX9S5bP1^SqFbZ8ebZ(D^lD>nEL1E$*Tqwt*(N@t zJZyap1_ny+Uk1YQzRly3Tm5|x=66toa3=6c#n-xjWtY>T*nNre)On9{z+c1I(icr6eRzZhgK$M-{O{#D_vGD{+g+teB^R+2`;L09 zMI!D#w{AZ15$YZWq`X}purM7$ceN}PE_fu1XsWsbHNA0;U}vCp-n1ki^iJg^Vf|j- zjj%sd38W=RNZyqq%w?F386TqG4sLNbibBU<^R1$nmQTmS%J+FZqyd^`(+^p2U9g%h zf4dCxCyCR9`HZa5BS1xZi2~DJ5;6rsK|QT4SNo}g`s4Br0CRy=3i0V^2u!6s3u|Zm zVOtDDu6$dKCBZRuD?mYuz#$o_!X~>LfI`9Bb+vwyQiuc=xC1;=blD z-YM}bv^#IR12Cb6?aTJff!Az~k<;-5QJQfw?n?%&4)wxKTyC=>gIC5hUN zVbAT;RKafYx;h(RPsIKG;VTP$^?UAr6Co|VGpMMMwJkQUQli zhaIrW$yJkI`+9Cz;E9j*yO9Yugbf(La(vvwwVC^>rKzy4L^vG}ill<%XSUt9cyXGg zBW~%1IYroAvH@VoR7#3`iTg-@(+$}SsSXgPieC+b)fQ+hiO8i)QSZR-OY7#IEa!Q) zjz?b&#M{~+%9~2|^nga(W_`SUx?N;I`}L@0^7dsoN-{Y_cD6^(b_k7Z1Aw4K~$+%paV=ko4C##X`RyCy3tR;3_SjybH!4w5*o@~nmLqFSe4G_im)Q53K-a*d zI$--UaR3ZG)wnp4gr@f@(P(*+UBBlC2ENX8+NxCmrJBo(e!9zf5a?e2^rg{p3}KUS zpjxNlE}nd(U8hR`gvtR_=RHw5O$U+{F}qgxIQbzoqyQLp8J5U+ya!@aqB(XF7YYh- zk0%I^8jb2kgtB3#w&TBnM;P~c(&in@QFmgtp2gOc|M;ig(^$8VZ5UMdwBNzDOAg!t zC@X9ATwG07R3B30VZ{@F-SA+H`?SZH|1&U_g!r6x7ti|t8I}sz{TucR3HW3FEJ8`4 z*%a#eZEs-1RI}v_nxeIO=kn^ma2Jt4D!{3#0{Osv6&9T%VA!|R@^ zffO*6d6D`Q;gaE!8wGz;we0DVkKcoI3@OaAu|yWI-YQx}5vME+p@CLtOO_)xz|NVS z6;JAMGVdiKQw2##?!}|NF|q^klyFc_rB_n3n3WwuEk3j=L1&z#W#|&A!yF}wnbKX> zbPNb630--VRhT#ANE(VA@>uy{vW4!X^-lJ`H^y;>No+ z%v%nG#Vl!0r+r>Tl9H&CQBEP#=!+9%@Qo=p!an-mG_i%Q>Bn`{vK!E`b6^d?V<=u> zt+~WXUs63uDV|TynojB+E5nVr4W-BdA3ki6+63xwU9diYW!MT0OL1PyAesD=)Jv?R z$CCH4L|ZkI?7Wk5r=nAZH_eA`;9dt7V%l?o|IU!o^_fgQuq&)I_oiwPMJX~zT+#$) z1-0(4dEg9^MOhz%iD#e}ZfNCNO1J2~W?4p!B_5)4rZ|-x6oc}?9KD9*mJUzPE<1Sq z<#d*)I?a0ef+P&VUO|8}P=FakBNdXf6w`WwVek}d#FmJ#aJl;g+ zx70Wv9NS7Z&RumHDgNVIj|Ds`vbT(*{x<@{(kvIg7cP31Fezmm1`OT_@P|b0#_g<1 z0n6*LuRF9Hy-A2REht=zy@9V68t{I&+8jFu5a+~t^C6Rg{6cnCY+76ardDu1b@W6E zTcKWg4+2=JSc%ujH;=V-wxpZrw`1xW!w(D$`?M|}|ED?TmF@c!mSLmyfS9XMag+l8 zw%*&^XsC=4#apG$AWT&t+Sr=b#1w9K!pfUjEn>kU%gj)_sc2KS-K4ZZNgGk|Pb`R~ z`);%u|KZ?8gT!M)12x3oDIPs8R7LH2pEQxnm+>Hk%8vkAD?8Zssw3!L7ksuduQeg` z+*BS`)+Yx;p$p=%bcbn}xa3i>aqEOT%H z7GZG0=(3}_e?!cOC&R3x?WT@_%{d1yE1o$*>EbHoU;>-dXK5ufWsuB;CDc@T${_{T zn+>y9?SXd)qEEgHO5^zN!C++X_Jj-suvUYFudyU6z9RqAR5;AH)@DdjxMb;ndfhKSHv12Q7q0({Jfz6+9&CW_v*`JmNPEGe6l^ z3`pyZ5P6}?BXHEHM7a@?0Sh*>3xtxU@+5ouX{OveHGBZW$=>|dAlesC zI-xJVUt%b8!P^?Pv7#YXQK`RWi2z3bAjUZfX-ORE7l(b)Ef|+MZ0a>o&RXvaVZy4K zB~&@xfIL6D81Fgy!(Usu5NYn3;6o7g;{0V+XQ!WR)y`qta@ee8;t*-hpRO)}f@zou zfX>DdzA4#Y1G!QcyK<1#N?MmO87#ot!JkO=qATF*kmS4Wpb|H_1L=X! zmvPu_9Nwr(ea}p2)j#xWuacn(Hh|CWCM$w!Q~)p!mc9Pin)O3GdqBE^oFx9`%7#pI zVuf}mKLdm==!?;?ULNl+Sb7eBmWlcy>890(_Pn|Ydeiev z?%!p^leA@N@RTsXHFg}Di({W&@MdIqW_Bf~O~43fugv_+k9Lr@F3MuFHMo7#0`$)O z{o>HT`VNrUIJWQ2$?uDrX3=sh@hZK{bXUAK#WN0vH2Vn|vM|b8@_qNBkHhuzfxj#8 z#dLv1ey^^jB?YcjJVUyw(ul=YYJl5ym+gJC?ZAq2Ak$GF%x3Pk{Uvq{U=SA_W~_GY z(Y^D*9+UAy?ipeo!DW}7f?z&;)m-T2OPHgPY|_0`42ubqjqvK^7QkDcyp$_F)Dbyo zk<(QXaC6sMY={*HuplOOH{e;ifU4}t0e?}BQ)n7NgD|fsf#~sY1D8v|-jUnGTw(_? ztO0Ch`ToOt1=V-esIvE6zLJ?@@~|X1H2~$vdG&hjK@*cb`nXA*wWlm6BSNIaxPwO- z%RS@*6YT*!K~By@e&HvPnQi;6vom$ShVIIiswL(xkoJA#s^#QBHMfivN*j9}fHG;L z;wPTTapjls*$}{X%RsKcXEBMw2*dJNdF#`i@20VN2r&F@BBwS6-+p4V&}{8~!+UJb z`%WQ{X^jOc`E}!FC3ksDZCEzyD&ZPUkN1u?|hqV(eF+uz3VrBl$@$Xsck*4-c`~`YPFw8yu4+~)= zW1&`AHCsDx@u4K_Btlb-6k93*clmoxgWSBG#dDIIeCnT5c?EAO)QLX&$-iI(tjj(qa19w7WaA4;E{NAm=iDsV!l_}~nr z>$A2*{p!%*!NDn?bNOCSonabBkpMm>e4Un@U%@lJuY-khOA&<{CuAdqc4&@{D>cmdQKFP{zd1eE!|X z!XrqXk{g}KCHXTvcm_-NU_@F+kflkDEUcir2e=2cK>Iq6|7k5Y9-X9Q@5_OfKBKa< zFEje(>}TT;pL3h30BI#zQvazOG{ZsA;;AO481CH-SSf`zsHM&0eVu^}NAsM|VjzYk z@a?ee5WxB(HT15rq!TqkF)&NHBHJq9lE+)5>Y?e1J(*)19?jEit%{0h&c&ykQi)Td za^38u;aW_M?}BiEBa>U;TkvrTg!jdO}R{om9MM6{Hj4BYbOWXn|q6--CJUV`Rd=a{G;;vcUfCYECugE z(7y1Ibp~;N=%2?<@*>J}oX8uWO6H#WH+?52RuvkA5GORSL&6iL0W1lI#w%7_6CL*P z1B(?G%EE#jxIQgxmdQTxZI4R7D!@Cma*x+bvv;t93^tD;_OH^pRgG+Nr3(AM>6z)< z3fwB^x$m4*myi)L`<)V3br$(YX!3;li`};jv)_l~xV)V2aZ#r-{lPyrn>fcmf0v-W zy7VirZV`?=5%;}ltNteS-nV4%4WH7? zQTA;yi3#N0j}LkgGHkq?YNEwlzM66NBbWPN_0K$6J%25svLTiRx+K~d%jASJRKp-5 zZWt*nF^J$~FLF|M+!vWDD3porg<8LRMq;3L0P#@9HS`OmvqL!*|H-`XQyL5tGI6)X zu=iu&zcI{|sZM>G7qDAJU&ym$&=O(*ZjUIFB}duTb+`*zPWH#VlC33T-6Czh!?}(F zShmO~y8G6!%>Z4*Vh}eIXbgQoGcAl;QP|I3yNsS`kIUHC2*&tFLyX%1KD!Eq5iA*< zwCw%bs31Tu?nQtUc)ibHYeBD6<8{QSNsXac3*I)BHTvR|4&-*ttYV*~PrWxNc8qF9 z&jouYj6|8*tF=Hku}HrGo?4BqYBHF2MV;sOmcvv4FF?@0N$4aR-=BXIe_o!t8q8m8w_aBGzqMINC1dLP z)3&OIz=u^Eoa1cjIku8q6FZfH=LQGwXi`EhkqC2XbPfr}lAc+0Uc;MG54nT0bx9=9 zvrtEJU6v78L!|Rj$BmqHXPg)CpFPt4%^3ej;wT_9I03|Yyub6}fZ~>f3W(6X7X6}~nh9Vz(2z-@%v&!2e94*Fc0TT|EVzMw_%n%0&FpN{O@wwhVE z#iuc!;Nto3MQzhdmG0c}p7cDM|7-~JOyg@0HlsO_)Z2(1y}u32xRxGLC4S4l_vOKu zoq;G6vPT8Wo$7j=-EhaqXdMK{)bOsTo68dzm@CW&#)kfOyvKi)nkrB?lY47pt4+}- z+(;R|N8s9+(VZj$qJgP|Wi{xBGl@DF-7z%@a$LxeAK@EXE2ZKd{?e0|Yi;b}za@g+ zr3$_D*7lrVpLbNtSrn1;v95;O%}G{T&L%&3H4`!(Ll0E~p^3t8Ot^E}&z)7*#5(R7 zIG2QiYGCr_RUWlh`|^-sa9`qlWbLD&(Yo13vvX(jkC95p= zZSjs`m`LrlSU2yCQ&u1WFNvY9RYIfd~ zx`$&*#~xD1i(n(v#E53sBBd6+IyVP`LTjC)`YyDr2QQ|80lDM%06zCV970L#Ul%rb zCeJ;aPvpd(@E~$)L*I7Kwx~M$RVNqBIblRsow&M>AmCpDZVC#S)8!H3W4p}@f0w8d zs_ZjlCdBt4i&)DPS@=*aJ=U@Rcm&@E z&2bgIXn>2&aF*9ks>3jU-Wrrk{pckT0C_Up>m5DtpFcXi+_q($<~@0&HYX_4I~~-+ zplR>kYoZ6qxHGk%-!9vi9f0ESdWYd@f=b-J4Ou-=SR8CulTPkyFJpvsGqZWOHf_`9 zqP&~qGibDi1^o9DIf*wJN55wG6;n9eRT#6AV#RX)VUUDQZ9&l!as*L$tFykA$w2U7 zZhu4!9gAZPF>TN;Qqj6UL5kvrKLCQuc3Nnoyv+;u2Mu9W#H5@dVD@k94x!G6oAZ4b zvMu74UnRLjOrr?v*~W}^lAnc!`%ld^bI?$-UZIb*BrMNGI91=e!}(I5YW9h<&Zp$L zwIYv(MiUMR`2MfL`nZn9;Zr`2$*w5tO(r4dO!?F+Q)hGOW&8b|qeTg^=o?!*uZt_3 zc%LF&@kwt!AN1Ig2Y-+#=!|D)=%)OF&YH%F%pK6KQDhL(GW;3v_p}?*QVg*CJ!51= zq@JysR1=Q{H!}NBzoVJFO-)mxC6)}M{-VOs@wi1A{4k}VA?$WE*IS+{oUBWSHBNhK z*Opo1*@6-7%|!q`@dpyQB<>nPS~QoJE=6Y043(c$u;mtY(EZo3FUOo?mLz89Vcl-a z%tP=^s!ZM&tjAS)^|296;)NPSy_|RFfnel@4+D5Z_8^WjU`qWD40Q^KWO~elj#N(L z&$>AU4-pJ@j6pq!74fl5s!cxsY5ZTc=QAfKF$I#mNAX6K-y+vEBwL7lYy9SAPef{{ zZt=rfMjCn?(ebh!Mp_Yap=fNC-DgLbMmo{!@{%*h%_il|oTfqCmAmj*$U1aQgqZQ{&X}9;W#xhP-Vq8i~eyMGn;E67TFwgk2t#vU4pF{Vz(Sy;KEjZepYdjzrYR< z$;4rXZwKeu1iZRi2z%CQsgqt=1i7D@B;5Pz!6tRL3*GEz(?kuG*1(+?nK_jh4J4lF z4mM$CM7gxA+piZm&(p(k@SP0IeAO=ENw!+&_8lj z<5eZ%MDfH1ZEmSy4v2@{sAxykm22DSFT95Yuu7eE+?M{5Y|EN%d58-wOiBHGj_}ZOj z2?%}61deIp_#Lsx1&|g4;(LdAVLVTao^&z={8M-eoxCkK*;JI1Pkk604uG_s&VuIX zXkHfBni8p7cz?b%m*Nk%0-jq8^aE4q1nxDGmVBctW7aev%tl{ZRIHY=a_?1Qu#tNo zT^e>blSGG z$_v4{2AH+{(gHh2Z{DGaUKoYc#I_C}ljb=DTl&NOZC|Dd2V!@K37XN$A4}>P)q$rKhSl1*q94# z!yH_pEk_^mqdQUEcd@G`uh4m9)sc8;wJP{?AtQ3BXlE-KW*>~OJG<{#gVCnj5uCC* zZIQ5-C!SG|;)s#(7~Ir<%CMN@hK^*lI6Rm|F`b2ryn1777f-R~d;iLv`;#YHYNj<` zrRD2c+=rjd9qfEy_1pW?Tv>cqGV3NHJ5*c!HOa(A1_){6?dgF7L@s{{-}cIsI6Nc~ zI|m%GTVQ-*N@}`6r3TIA;?zk=i54kEpFQuB4h%Tc-5b@&_$cB5uL`{a840lSyxOM-FFmn!guVn2qL_k z^=RAIw8fRTGUY;J{KUABQsbH;q-!|5>;ck3+FkqBkK>?=+*2cX>Gx>rg$tS=(bg)0 zR_vB-i1y`dohM#Rk+D3@1OC%@e5QV+A5{Jff$HP0(kqp8(*ccE?;m+?GZ7CV#-XEm zZS2RX)zDf(!zD7WIB@?PU}?OGW;_6Wbm&FLn*}AmZwT_TWl8#YV3CHV4utk>9SCqB zcC6P$hBCD=+5qN&iknhnZ;=5$Z+$ITNDkdo`O5XvjZi;nt-L`c@JX>;7^GoVzD3OOO z9|RD3DGBeDzOk_ihV!Y#Bd!@s&s|YD`0&JPF!qQZfCY4`}+Q{k@#4ew~EMcDS8!GABGscH-;m+z9gHS2JdC8(UU0*jtPZ z{5P2Ow8P8~@SLZp8)Tc1Ip5zsBq(&Th6e(~l%<0LMQlM+x@Z?%UL_M8aWog*W{(Yk7gbY-%fuf9P2DP}O3z0ULiDK6V=t zUK<KKQZ%_~-pz;r93WT6uZEgYBI(_z&NaQ2I0WMjmsN$~!3cI~Fv*s|^~RwC9u_nFZ7+t8H-7wnD}G7#69cjUvnH z9DNXEK~5s^*!9q}+MnyqL2eC(qbtBhaUYw-a%yJ_uxw43-PZqnjV|8G#re)Qc^qWd z^}C@icaQEcvXvXLC9LQ!sb%-Zw-`G7Ot8*oiOdS`Oq3@C5KKc?$+OEm`9GwgahaT{ zSmU7=+rnFdyxFKOhO|Xe-~l50`|EI#6^D|?WZ$nD0xh)W%3JC@@YpIevto~EMo_u1RWSb~TYHct@r0T+UYRcYyqkV$&)SFMmTlA`#TSFgrdl{&K)Yp+mQ|;4wjBtZPy-w+ezIL~$tjo~*L646&{H|Lj!Eq=xuz zM}db)jW64^hb&87DMSh&%~$9|u*O&+eSL6zb(Zz{A%(6Kx3Xsgk3@%!)x2Isyr~nx z(;|`C&b)a-W(S@}LS#IBcJwWHC-MxoMAUKQu3=ssN(jZjHWDHB>eEk}ycK@+D`KlO zEZ=B0Prq3YPiLNUP4m~AW`p=0dr2|4!HUE?a0XFbk~=a8AP96qZxtfTD@e#1QvC0E zG#3)TF=$+nlyk(7QliRDJR6TH5YmJV?3*pY-(=74Mx}VI>pw&B&U1Ui9%3IAhBG%O z*6eEb?brdUf*=-VYv>snfjWcRwVBdUq9b65OYplo47>G2nslyd>osRP-V;Z~Sm4mj zOg$_JxwJ|=7k+qF2GRa<)Qg5k+taH3-W}sp7 zTdIOWtO&mlSanJK?Hq~K-yG5qzW8T*wTzJ_r~g)b8eKM%-ZEtb#Wf@X&2%OZUi`p9 zvAdl3{84U_2F0&hZbKa+uDn10|7&)*(^dWb!bmatA~GY%{l`*k`x)et5dVpHNhKwj z+txg%Eb$y+p@lBV3xV8xRMM>?d?lTaML+7BvDPQPA1q|1fdmw_<_GQg;L?Ud9olNu zpNbA50g~lO8;o$THpv42`b#VB5A$GQYL0~ltv6=Sbkjd)cTCRv-iUE6OVv4MYijjS z(#fdY;ZL$&5Byk6@^fqMov)q+2W2yhSPiKu$1bnMVq4yBv%=G^&vIPaHADl|8H|QS z;eO$hUtox_)ungt#9*p %wpzxBLHFdgeW@lLPFl{n%2L_aqon7j-xIGghJuJ3bG zxcnPHZ^@{ftX9t?9vONG@vYawjf%fSUjvwg40Rs4aSHTl#Kv6=dw<2j8N6?t=b%Zr zy^f@bSPyHTPJEz*&_7)i-Y2UO0fWQLjo>r>DM=2abGtw!t)=wJmk&w5!iL@$&Y@?< z36Cg^M?@HgxJx!bu??i+rRWA9j>xVjM^pE)4vqU#dN#ZsCPU z5J{3VVA#yp0@UpVF)~Va3FK5!emxeqsEgEQ2Iv2$ntd++Rc)7lu?cUGD0 zhsUZ*7B>Wo>H$bzdH!=CtuXX+>*bD36=bmve2);=?1TuI22x*^z-|uNxc5Z_cv>=Fa8ouJULQ>EvVX zGFix@Pa`Hx#E?vjK8Tj~P>P{A=ea50AOt@uE5^U6A3kl6eJawUxCQzvVmo%3aaJf*T%#w!@__gcMxPG`c>1m;RV@h+YlC79;(ALJhr|gN@{2 z$vew(v3f{TuL2KWI%@1C++myg)>N8>x)M(i*27R|J0JB)Un{hr^ zoy0vo-REdnUOE9lSAvZjJf1v97cn8LwwZjVqOc6|i1@;@qexuyBo~>HK9R?U*2_7Z zdD>nc{M2K|m9^be?N6)=xMOP;wO@W|{Ge`R$??#-2zPU(EU6J+bSRd3mWG;GdGld% zP-dzuoAa9QB-8JtFEV&By(hzzGZj6s+9@${^3{R$CxG-q3@*nP=Qf4B%|NOToi;6c zWm@#+bbso^zdNw)jdKkc#HdCoSKi>TTN-b57O0E4Faq(^Gb<4c^Tsk_`~TRT5?OTx zzge_L8z^daXFK{jy}@@nmnnFFvk3a%>U zLxIj_wrYO*!7h_zVkjao($Z}0_%>($Mw&&th02})4Zl$AA$o^WSb__bdQ%Ul6a+A$ zrl}!7x*R_MJd}3sPJ&Qfx;*&o{M(?w^R$ylfm8X{P4Y-$L3Cl=3H#(t)*@5$C6hgb zb3|8!%W?D2^1i9XHXf#n7}9rg-vY+hr`Ed*ykU<5sXDr$w$N_vg^=`^f^M$XCB};# z00Th$zrg9D9SbXi(a!MkN5nUyE8UWmlgA8z`i`K!9Tk&H9&`%c`11^c6MYXH+=3bZ z%I4jKU>TC{<8Jvya@p4!$DrORzKxtQo4JB}aX9MJB(@GmO)ZjjX4I8CRP1JtM&seW z%yZ^unz-_{#Ap6$(_9WI=E3Pb+A0b$wu>i>J2Kk~grg5auo<}yJJ#U}AeKv~9pz5f z3@LP`&{$MC5TzhS50gwX)F-U_8BHH8tnTfb%IJnbxSD618zJ2c#Qu@ z$?_mHw2O=~Ix}ryO8)UzOg3eGI>+Tzcj!ZtjOTFaYZsMQCpJ@Hzs_u<8~n`)fOCBO zCd2--{wkWYia;b{p&S3VUi^F{l5T~}lf;X`=K}EoC&!T>bgL6aE59JboaiO0!hSQ5 zb}^*@$6o?{L!Z9V+~aRj2J+G=G-nUcIZ=XnnL(a&Tc{46tPG#O>E-Z7jr>dQMS z)hL^B#^q|AE68;2=8p2ZUeG5AD3EcTEYHq@i2}3TJ~a&sDK6XncezXcp8X4CkhxpC z2qE@@cj6|}R^9&sg$eLWwXvZrigi|Zk^Y5Jqhc$=K+&9xYE-=K2(VnQ6QD;P^5{GV zAm^vNt{kZsxGD?bV}IfZWviOF`Q4-2r*trTaw!Zm3C)P5sS2ML~U zEBcmzN7^ATpkv7Hw4CAd3SuXf{dbSi_#Avd9LNQggu-qcPTQE7P1Hxi<%u(XfIUw? zL2*zL$EJ8n5COW>z94sYpmZoPa6^=JrdiP5-3u%`kUbLC;|Re~dvR-}=_3XQ4wSc> zYzC}^SuIX^plOWel1i2J%$?a!SR}|= z2q;fXitA{{F4J0uws89s8#4l|C!1N7!9;rlEC{_|#1DoV0nLEPMOWgYJ|>_YsE0G| z*7Fc8mHQi$GD`BHaB^r{hD%AxUFJaXdaUuXk>5>EC+-h+yQ-e~(-eY@WlKUATHlGR&VLa85;JkUL}Zp+M_9`!@G+1-JKo;=rq z%U_dJtCwC1dFcmjt7|&X3bV&e5$+2Y^RapqX(Y2L@)qFAc=WyD+A}9>Zj-etG;}L~&4bs~y|_40)y; zhe8tezww*qft_IK$xqS%m4%7h7qlPcGXr&;-n;kykqU!RTv%U}$bIGvQ0rqy)DCl+ zW(}*sA>yDU^GJtelVdw%kzzb;q1u8ONmaay_o)DHPB=Dy;e!`?P=A-+BV#n^Do{nT zjcBWpX59;IwYC~FDH6A2F7s{*FExDr4izX6(>Z(2P($D+f(!@2&5ArK@BYbv(=;Ia zDL0>RA(nD2Nkz+IzmW2cCPOa#GoSeaGVtAAZ3i$rwJ&81&-fv`d(V;Wp0rOfPzP`? zrR)=+bww(1BRrlfu=_BtS5-CTap!k@7B$)$a8?t0vh4OnE+G7T-x*!#pT5yp$50y& zMB~!x3M{`~$(g~cPW~|!uCTgRk|hJjRO}yYNG_`uhyf@BsQrfzbV{CJ{6?SLk$+~o zL!m?}HZ<6@m5V*RSSh{7&`TbmkUU6pGAS5)n6<~Jp=z9`e!!~g#)CP(=G5Sp9vB#p zEA|=%eA82fmPGjr>zJwOp_Ls0Il?kD^U0uQk+bDlx21^YvZ(I~(fDykFz_Z;wXGNFfveshp_ z5QH&i2{Tb=UU}Pnqv#o!vH;QA^^VK_YLX!^B1-pJ988;UNs7X^tI#TuKXV%GoMoi} zuMnw1tBlrr;Ko8Y$O_kXIQmyB>#9>cs0U|?7~8aE5XHe_$DPo!`@!OkJJM?=67Ao^ zwtceIn*)84+CwrB@$j-QnRC8?d!SYLs3)Hv&L@-Y_uiXw$CI0B}ThtNjXv8J+(| z!PK(-rPjSQ>6~1Cu#_d}ZZxkTQ=T^uO08wQ<1ve1CP;iH5^D@`@l2I*`DV#c6`~3a zlgG}ns3X0GiCGmNhAu_%3klBq;l&JkTPn-@?I+2qHB8tbN!}poCf>9nwx9-#^WEpKi`7lj zTjl1#?H`3z=avSR7FIdYUvMH{vIGN^V5p+7VfaIDfysOhf)WzXmP)n?5@HwmjvnVU z$KmmMCA}&!rSwxF>&>s@nqDIS*MwZ1Z-Y#X^wS6ffr6k?)P1x)H-{>6wNv`bM_ zY(j&6XGPZSB@0HD?90~GUJ=}A&d@KR>OgpD(muS!F&;CfsCJl^fb}m&7h=O;lM~cR z1(zc9l8ZZ_@SfRA_Vh%-7jqm?MM*{&vk8zDzQ)CLOj%xSU^W^u5pY;7BG8yPv4r8# z1b%8vQb(X$WeZhBD<1ksrK!M3@Y)%aHZml9QzgTT3Hbij@`e z>^&vgpx;iQZeZYN5gT>yaPYgU&Ta7ab@5X3wDvh@ueQYBj)Ym6sQKMfupkw}dBaKC zw$atw@K1)2_s2AjYS>H$2r^&{oGXh-Dqx6MnXf z5!qm^#>nmt-+RQ?AsC7W!Tu*oB$Wxn6?7j6V1Uhn3*Y~a_$`EG9PDu00jU|&ShFFB zDYZi0?pE${hC*&5l9jVtKyh#V7Bd*p_koVRTig2iV%Y*(3|<{@Bokm8zfJD_4r13e z2XxWC5&(?!!@}bPnnYDJr=8-S#5Smr^xiC1mk@AAUn*#i!qABS*d=jc-G;kYB-)RZ zhH{@MkGqOAy-V4dwdxS-V9|s|j-ngY8uZ989OdAvN>3#dMT=NwrI*n90l$6}U4af^ zs6L2ER^Abcx?oM@ZF+iwZr*UI8T=PsBoyQ;S3n^FSH?5gKXN`cJAri*n=Ym1h`S(Y zYSnevaA|*jzxMl4tf`>Z5* z_II9zpcnFc^%EVACk643C$8>v1MdW*TKb*QM_zCIE10c+<9z_z^%XZ^(OMuVo>GZ; zDQ6xfc#i2Bgi&yEGOQdYeN!0t^$_&8ep{i)lOy(lZ#8tpYcWE?THM=`^{@2XtYF}) z#*Uhh{y2+J?gC!dCr4U3z2Hf=EGJ8mzJlh z0sg#Tu^l;9DLbbGphcS)N3D(B*|^n6&<(7=!Y)01^*XCGuhV-VE&QTy?G06Ck^uCd z6S>tNhd!He$NLv;mZWo~PO{Q-3a8NwH)&_G%EjU|A)jWy1`7q}Mv&efwn*vwSte$e zQ2R5kbWCSqbH6HvOla=4NGei+h9hu zi^qBe8FnYcey}C@2p_2sB=+Cf+`$2O(?sb=m>)qV@Tb+%I)!k z>=thT$i`i1H!MD(hk6A{@K(1fn2)rk09AIVL>y&k&iYb90$z6E3sftmyV9T*#V?Xi znKWV=Y{6U&i!8=hI?q9pe`Xu~VQF43mlvS*3&#~)l3|r#O|&3AQrjN~Hye=zdk=4o z7Wqi__k0WHudKmsY1pZRPzr`6#EQD-32*#Xp3+ER&|8dxU#?Z$UPEiF5p)s2Z1pFl zEHotjUIf*Y4Scq6H402DnjOaaxX640ivGsqV&Elx1NFb)u;9&%GgOgr1$g&ebQuf|IgDiIH!B)9P zE^h2jqz__cz7*J_P`uc*D@N6ta90_r77H90XevQ?9RAenjP&2Yyg>u5X6OTO&qchp z8>A$P>TMeF!X!-48WjgrXnR}QebUKd_V3JyHA0pyY3&D#PsNml4Gm4mJm+{7Xed+v zRS-^lTAA#%)xb!jRGKIhRh>(}ch~z^rd6Sv9Z7^5sMl^700~TCFe{nVOTqxXA<>xH zp})j_$W2p!yfoZ5boO6L*P(tLxR$|a2-2p2lPJ$PZoSLn3C&jhwxtC^e&TZM_jgaE z^=Rr9yiz>0{rRPO3Fh#WZ)g98D?=?VW9iMLuW;n;$X-8+4t*1ZF%qNt5d^gB z&wPJxKfyVo3)8Qt8CQlf&TWE~{9(`xnW8*Xd6@E;#X5nE{r@VXVg zi8E~1KE#&3mD`R1SV4J^^s2t{Cu#@7p0F(yT~rq2j*x<0E^lo$auUR%;dK2mLYAD| z751A^jkwNcPA8%TcFmsRpf6s(#B&5XXHF<&akg%2-ByRVGI_b^VLpEGQie|Lq{QJq znH1f^4vc?U7w)RS^NJR1x$VRib?BP2zsF-`HngoEZ`=B>$B^`ULI*dAJ! zl*@{xW~AQ`f&vDt2R;Z(Kiv1vV{>FlIEhGlWgkk$5y&J2+{w!;1I00b`sCd54brSE&+UKq&vektwNTqAzM0VP_1}E-_oZ7nmE9F-rAv3;+STL8ak|V!Ntg)_RucDC#cj)OI+9811_nPPH&kE&0PAcEeW!3?&*NUm|AHFddzd*L{ zpsneEFTLn*k-V>Lp##B`jNk}^S%(i7Pyc|ItJ30K@&U3%yk=x(yhF3~5FoEu{zCEc z3P3Io-%DSp-oJx1sp`K2lhCp70~QF!l^dWgmk_!=$>c-nrt>(ot&+Y+HfF!&LU+3W zw3kAHgK%pTUqysffK?8ip4P|l+|Ed=c5Wpqf#j)E6C~?p;IIM&rw7E3guC0?l2m_< zq}5!el;XN{lS0%X#2jBc=6sQ6VlIcy>Qb&ZMDhhGa(f87PxopbhHwWG>g`w%Nls)- z9m+Vi;}CU%99cME1h?{xV^CxMdbG&@V_r=g<{;uaOB^{dlV-T zDm8FI4#m%8=KmLcLd!ND8sOegnqS;CB6f-fEcR31Hq{84-uLBeR}|Myo;)VP_y#c9 zHp|M&OlfoAs~F)rw59hiHWG);@l(z`Qf;cjougLqZP%Ewaq`8SIXyUmZ(Cu!YwkYY z=r~2CkqwcnICaNZy?D}CyuIZyuB>CZ-%8#*oE8F8(jSPii|vW2pbw^u?x2?cSd^Ke zJt%Cf10NK@)BgHxjX-yHh!tizQx>2zpl@;u>=m3m+E$xvll=?{^$NtJu1inj@hK6~ z+v;gF=zJg^N@r2b`5rsSBs(b{vY43=!z$5f<&I&X@r+)o?8;;O`)3=rZxWzj35y zVu>2&y$D!>g*t6{0_oO2w>;cY(HUZqigr9?GJUP0g?7z0q#`M@1)baDV3wlWe#n$$ z$LrF(-;u2FjeihDpjYcnb1KopSsy1t9mT?>d}wiB%X@hR#!w&2uujv~*$(X~>%xOZ zM_c~oMtnq!IyWZVt%YA?tpGm z?258jCg$eh@czlOuy|o(zK~yS;8J(5&r#0q7_aPG?B)agX7|DlM*TG z7Q;eX;JRRH8IpI$(V2B?M&nJQK}lV=s&vyy@E=euh6RwA&vbd~4%8N(6~Wle3x+cg z!_Jf-$ePBs40k&~i2HV#4Ov-8Yy#>X{n28DM!T%Gg`HMJVZ_#a9f|OlHVPTTeWBGz zl`-|k$g4@$W|@(FAc`E_H%Jg=88m1w$PiS1E&`Oks6337!rDb)moZ#mR6S?`0d>GX zG!k1`#V2tce2SP@eg*EC_%r` zDrHRt1rVZ=)KL)%-wzQpaXEjj4-jA(jFOd%Vl(i2CCHI%2A?p_qO!=yOCTr8S$nem zF5^fDrRpQn9$B=D|Vnb7PQU z<$XJ~m6e(6-nxy2He8;ivPZj&Lg@6|vls(rdP}+z%^ViJDk#YKtJ<0K4HL5NI%9_w zecd?P8Zl+0n|IS>$nJGtalvG*b_88KC`6GeaRWhzDg5G)k*Jn{!t1jmovzZ;*3W>Y z+t$61Y#*@2^S9el+@b>E)ZUKf@NMgvcUyQDdAEjhQOv_8j-(f4h}m8=G4Iv~ z6ZXN9)n=D8&-#1pJIfsi>f?=q!rzvxw<#*``cDxFJ~pU7UXX`J3lN@t0!t!rgTZV6~~U`<1JmBo+JRZ z$*4T>Ah;5VuakF&6oGlj>>WSsT@vOQ+ISI>K=82#IA=TA)0t{jO{>)RX1$J1OOO z6aAPCH>pB+4Y8JAmcNwd1jdke2}+GW(fSSlDC$x2~vn z*}7+Y#qJ0#cey?t9j>d(rbobEyJSk`Sa=MAGkvDt9D6e&5-@D ze%>ojlJNbfE*Jc6A1@-3o3;ixaIW`k%2^~@?iW(1oc4)ltcBd;UMnMC7Uk(k#Xb&Z zG3%ExM@~x;Bg*`$6U(3wPa+e7q&wDoPk#z2%-8y_VYG$yeTNr7ei_oAw}l&n-qPv^3Kq^-b(uqK(fj;V|8Td&b0ebf2m zUIClB`PM%P|HCw@Hh!BT1W)@02|FIwv zU7zCRO`km@HM@dKb;ivYGh+u%?fmAW$MsVU>%Cv})xFE;aGEm{-vM0sf$J<3fEf1= zdP8u%bcaq%YT66BktnlYZqkeK((BtCBGst_@g|iogx188Q83Ox&F>O+CD#OtAg-Km zHrm{|Y=v0xBu(570C_@%zvl8}>fZ_6l@jPZZ1S5oheR9HsWtI!Z7jA>XZ|h6wBz{H z-=E=-_OKSjHN6u|jS0CJp3AruiV+d4K_~|B7^9f=^j7nUFJR!m40;zZeq-Z$H)RuH zM|TM{DuV^kUi-2QPc8(JJN;f?%J^{Y#{Ex>%&9H43teR79~}V6i}jY31#%Hiyzt0U zgrJz?(x6KH-bqgoI{sUMUtWkKKSbr1t2#sInU62`1m@J_RIvcP|8wExYF@L@EplX$ zNT9M;$R2ixTV^~yGB~I`mlNG3YH*Jo-tD9kZLjg3?#B*k5rLh;Y1GQPx$$kpqE3oHGxr+_-T=rinF zmeV?$vp;lsVU32f`eL;G{Atn8ucqLf*O|1__&+V=@f_S$vf;zVY@YiBbfjr-9hT-t zd$UO8%sT5-Nd?q|LUoR_Pa8CQQK~(R`u7WI>?mn3L2S;3BeRwS&8@8vRUEzn`TDg> z=Vwm9g_&I8c=ozHM?%U_MIq6QsxaN>dZZe_DcT&&&&MTW!c%K=f?G7}=8eoN+nHH|W!@WQ!rd(g=I7-H?~g zy2~4Hj+>ES?bd`!>-oWokq402@7mQ3szFKk#bW(u3gj=cvn_uWqFdpYoXRMUkNl#& zRP?zwqzwgKF4uV@SL;_jm+4;nqU*Kk$~YC_qKsq#ba^uN_IA56w@W|&xD<~wEt;#> zY9dWiSn2wG4G938!q`!VIrXIrEhe}+-MsvYI|QBSDJMBsI+VPwJ0?-@H7SN5lg{d& zdx31Q1N@$2dcL$~vsB<89(tpM-l-)&*<9Cnt?{1owZURgNsqPHHZ)fMsrhto=TWcd zDgPMv?0@MM6Vo(O3LGyfvErzA^1O4b99lGD&;f*1Z# zfGmSN+ccZewBw>}T+G~H%uEmpF4kQn{UHlt;T@}|BhiyM3$By09OquKgIBlri$w?FBM(nov+pxQDk#%VFXfhTEDPXm%YnF zotv4}TRMZVnB#Km@DwI+`02x08t|houG4rq z(1uqdSG9t*2g?-~%CfUE9VCE+qUJqJ%^p|F?fTiT zA~(1kxhJ&Jkpv6=Y^HuMhQuxaT{N>Ov)Zet|;VNqztjhs|$JJEVtT`$XDRv2Fxk|c|nOST2fz|n$r{2jX zs{Z=IsIhYdr3EEBRj+9jBQvDd*A{&u`IkMdJogqeRdf>{wL8C3d`^`K>#shQ5hQcn z?cX7%c|{!*A()c1HPkz*95N9!6=3=os#UYE4lm>wO7G{i01!7>e91~+7@T!~aE!#^ zWoz4|#q44~l@y{lbFp{|lToq^$N05qJ~_5|E;3WT(#v{xlYz#+Y0-fB!#OH_K0 zPVxY15TZRX%MDVeOPS;mb(@p^a9t<7rfZ9llITlh{NUx<s=77MOZ){u5mBXH^tn zN~JL2s_Pg?XZO-rt<$O~bI0e`{by3#n9Q%kh|C#DOU<|~uxc)|4PrO25471^YjHxXY}2b986f%4=b4{35ha$x@g&j;o<+>F_ZsVse@31;>6 z9vw%&psi%`BWC!#wp^a7hXxF{Y#cxr@+5)DH58`QUt@`ZrjFUEueguW_))?@DtxB7 zzC_xhhh5ExBUGECMyWU~pKAMB^LYJPWDOLMu))@fJAsBmggxj*;a_Ei%VtwlI}L`?JBb0n8H5h4yj*h5-x5 z9XVJVBJLHrB-zU?z7JX``Mnt)!hh~H3?rh)0^j{O3KH)1!Eo4yhZ7V?otB_!}AiDxCZ zLqwN)lre_vQp`d7g6Ir<@&{bCO?w0V{LjAGC4>=rj47646LZV>@! z#Oo)Ves-Z9p$@uUS4;p$QWO>bw{s(m_HE8OtT((}{@=rO{3XfVb1K6<3+`nvLCATw z>%SaDl`>j!DVlEgXqSlDN79P2f6gH6c}5;&-=)v~Ityx@8%6QBEA=T5<;$6|W%gT(-<_E|R< zESBpzkEqFp)S>GhUTgv1dc(E~Ne7Ce+ee|IWsyQ1H{8ei*v?Y(?5#yqs9FCh`BI?v;GwVMYt zI_lMHjKDA8U^%+w*Z3P0ki*h#@t7MBfZ29(wPp`5xc!3&cJuGUOLgAYi9oBuT|)-8 z1FY^_;NMej&CMk6R@!S;&qC7w> zgFs8#pF&hZzTJ*1mzAwE@wDj>sTYh!SN_K;`%*%z={??eKc(-K{^O2OFD3u%#K zL9;zj$%+hS_pl{SE!V{);z|n?CjC zRaDVz%kvA&1b~p}l}YbgS;3l05U5#{{xZAi%*u%^t%ZW#_u*yg-Albuvo(zEVy77n z3^xBzK}SwsGy%xthHNG}AaNh^I};yUh13w?dKV=o2WqcsUGXSAf#DJl9wVjKdL7{a zV0)oN$M4?Q#{gQu6i5r19@Tc~lkVF6Xk!gwM$tiHX^7fAiNogvvPNk-YGwM&ixCvf z#k3`qwXs+>|3W|;aFfr3N6KsE<6w>Ej}>3uCmBE0_|DI{N)2-gq!9zL-HHnCPSHH4 zZ=G>TQb>nX{qQ~#s1{?^NjOyb=d>-ub5wYq4GGdNE(Zmlohg%w`3jg82MPpKQVp4z-hI*;!s-aU7o1!)kd9_)k}Xv2qc96{ZA$S zJmhf@@*#spm6movaz3ss5V?qgUH@H{@jX5iEl1dRpmw9l05OLGCNbK;Z=!=xfsdd*Ii*{y^)$Ly#Bq(P9g zo8o=3f9N2vtHoqXX~*uw8HB+YJ>`=}9Pe{RoaT~X1+}iY$7{j(@I%U>I`*=8*6h_z zBFP1!Rj=+=4T}PfOS%=|zy9DR?j&ut?*Y+G$bK| zA6xd16b;Xm&(w)71hZHSQiSZAF%7C=QzmvMDy@gSj{>7_$khgmT&vQ9{6?%!{-TAa zv3YlIguE4#xdo-3?-tlG&xPFA6>9#GBQt9Ov;->?FUPbT7~_9bDWO#XWv19>+iz*X zt>M>ZYtVAd3X|pQm^E}8%uq%0W$dN7w38Df&iQ=?+teaTBLJKK%9Wtqn8C>aVW4%q z8IaflZ3LEo`xT;QD|Ek6{$4Ur2$TRuUY=J^oFy;Xb}G4tTr=%vxzcm9Yz7Rn)#WmW|tsu<;xVLC^b;#!T0C|^Hb@0-x4^O%1ICbOw z1F!R;{`?(?b=aX1Uhnea!7i|-v&E(b#=9#$87&K%Gw%LhGDe6Do+PKXC(38T@f;5Y z zLEYH%Z|{s0*r=1}5fFRN zWuXR%n1zH5#dvYHf!?zyR znL79LC6JLUySLS;gcxa`8FT}8ekCKme&X*q2M6Io^wNTm&sIO5&j^QM3E^p z!DNG;%&#G8HkCRc>>m`125nvJYUu<|nTq!ez+vPeN_4!29`u^h-8P+p>%Fx8-3S2M4sc4m!a~>VaseVSM z8ZkTsn8sQg;#}u4=sglLQtEn3G24c&WsH0<2<>P_s$3T520850w2MV>v4**u!Nxu= zP3;5&u0}}n#UhuVY@n5PkqR8EOc!&1Y>O^!oeq?hVwZfMRKd6y(}g&3mjp6JBK|<6 zs7r&5Ms9h!zp~{Y*cS0|ZQEoc6;p1uwOM?OyON62co$NnyB-p85phLF2648@=dpf!tp{}vb3nY$ z+R=Nh$Hv%<7dUdJvH{@>zw#8EuV>R%HFY3nr7?I*5DMnmdAnK4SUaa_rIJ&`OUd?_ zR_IcCQB~KiuBAO>g@hYeD%A|_8gdh_fQ0|FxgTA+@e?sjBr%Qo^}eiKDk{8nL+B$j zEZwn`RZ6R)7k+uP&ijj!p&4q3VNzFGHG%aj`gCTL0H~HPRy!h~ZEc}UP9>_- zD4&5x{)*4v_;oPqo+X%Om8$TBqS!xuhESsXvmR-kCSQAXAl{|8lORCC@}_OT>sd4T z(=(iNvOXzz5#@%~ym05E(liy6&Gbocrw$y)<7+@S4deHhJC~C!7v9i+y*c6+?4oN) zZ&3v)qct?2T?9DwG*jj+bL<2BK4qxrzvCI;tcQEquo33_KBKxGF2qxF;u#x0*7%j_sTFsz@O_DYHOp+aH$1xP7NC zQ=NwG!Q)ZU>3{7WtCb z)KCkZ;%yNFn;g(zq@-mR$Td;}g?zTOw6dw<9|Q&h=Xi(rK{5K77aL8>TRo^nu%Efwp> zO#Gk~>0J+Pj|Hpm>5-eB(WytP+e{U90!7#Uj1AuYEUs5W74M|=8Q(AlQV%wQ<9wfW zC0AI#h{*-pwrtc`Y#r7YDSFEw{gLGO6hH&Q5qffpwzkWBuW$H<10b5<{>;XUTs7)` zyGs9deR{qX+vn@Orvd|?*i?X^XJj)O!Kn_vR=$3NVrN7o$hbk+Ad8N@GEb$r+eig^ z5;g*|W2Q{qgZhy)AQ(q+hlo;&s;bJXP067nuZj%JkqizZKWZ$;G)fdQ65k8Lyc^yQ zqS`_)ZWRTfbPVu8#7m4zbGD^Mzof}ldjpj~S4CY|NCu*LQl&@KMJVX*BhihZ2zW*DAnC6IkDj`*w?T+K4Rch@t&{pnuo($`!F3KbC`v->P?E$EEMh1=`k zwing`9z4#YBdjxXR*qpoFl*a>6_5SUow#y%wE{=?C*faxWlwAJE=?``wPK)b1_Mz?)a=$U&|<$rVfh%(A2G zI$f^&MzNY656?9cXTgQSo;0cZt$@iE@)OYBh`|e<_qb22$Ty;w`^JfzqUYEmnSi&K zwzWmM-8sfWF{#6p;Kv4gM38JIazZxC%|=i-sR*q)V|QeHwQG}XqsF{V>12x^hkK@x zrDFTgDD@`U!GYjFomxxS^7iTxF+{dyrTLSIB8wPb}@@( zxEBwAQhzN28by@9dF9F-1ykapRPMN@>ylPT)UBM0T)ydsi4LT_gh;dvFQgW2(TG7! zp-=$Gj;eb4mv5{@vnSW{EUX?%hOH3GTcjS7?h5!*TU(x?ja65e`z)X58>8JG6i%ki*dd< zH=dJ$gsGQJ8X5};wH2vb@S|mH5voY!s?$1#A=Blmr+Fq$(!{~#4Lt#Na4EJ0ySn&L zExYe#ZsM!KbN7l!@*=d%$p`=;V#W@ocX(7K`z)t7 z3ipx&zj}`0OPrynps(PuB^K(~kXJvI?~|f}by`Nny#-CCkQWD0;+MpE${%FQ7<{`8 zR{N`FONKIz9RLXfS>;l1(i)V6RYUgU9#~nQRxmd^ZKd0ZM=z%GpSQ`iauvAzkq(-V zY<FDA{7A=ZFR3CNhV#;Zg8fcdZ=m2KRaY`a`ZM z+TiIfWfG)8!O%-z8E=0Q9{4mb2T!ZUDH4H1Fn1Judsc>{h)Tn=0mLZB+D!HA~XLRyF&8eGAgVi#z)^uR7xB`A-NoHgVUgOF5~fz9{_feaC* zZ=>^XQ6E;5`>`JTbSqLre9h?et?jyD zH1I-lVHzb?9rV6c$qCYc3rDc62GYs`-|kPxUXjb!@I<;15p^?A(l*Oe-(GcbmK#-Q zGZMDkRHP-K$gG;yW<~UV=f+0qYn0%XsnJH6ao7YN<@QMEWfL(TY5+4Y<^$y&P{GGR z72f)+^!mxFO>qm&M3-1t4tLE_3TfC+RBZt<0PP_BU}tD2r0fK=Sd&TfG|a>HM!n@W z#s{E$M@NfAeC72RfRAQ-N;HCrkHTdwusa*V@6IiqhE zSL$bBc-(sfSf ze`|qVN_!4A2~VUkJjM(Gt%8J&d&Xqq3I%W1~HhZG$d=`{8iRq7WamXQEBMvcuyL*58zR^`)P-lvj5)lZgpzj}rO==K+)(+SmL#Yu!xY=wl zbIN>t>oB*o(J9^fAk(uq{#%QMgIpaYwv*9Jzf##!6Vo5c98tjgkfc{(#x{nzi8!W^ zdEvm`hbn2RpF=8barp^pTjZf@avKToAcft28^?I^t(3SiHdM%tu77iS=srdwgb`k=NYyV6hkM zn!K&b-Jg4D?Dc0fMnUtK3ojFAsrT`Ru^8)0Asu@DkoPhabUd#H6~v&C<@se*yTcn# zw09cR5SHFcsN`jxpd4xn^4MVTLg$=+^8Gt1WuJ3BJ6yKCwx6zl?6%xp>soHM?-VCr znNnRuX^vg#-XM132}Uh!8y43ofiWCnGOEE?3GoRzuw$?E8u^Y!jP}i?`=3HA7IBk1 zbweX)FV~zOUs|-vDVEzsva2H&SDBgDKMC6h#WHLIuTu6dPF#)w1$fNxhEU^+tJHG} z!6##aewVWuW%oS=obO)C=x^!(vo~Afjr-3@$ddX4+q-E&=%+(2B#xDD-SkoVbwyBlJ09NWDOslZ*< z%@v+GUQoAsc~{F#LulxzENZ$ntzf}6Gv45_{Yw>)v)JpI@4M{l5f}58J8Z=Ln9d&$ zq%9vov{%z~m$|lEwiWcf$cHWQ?!x1VI@tNJZinlYYiUCJ3hRTg+Re8u%S7k`!QMW-r&)Zy_nM^bo3oO z8VB@-hz~m>8K~$%eLtAFZqLFahH!rIsGiC@8{tP4w<>=u`q#kR9V+_PJt_xx zds;BGtO-|6on$Ub-AZm!7a6dkQ_aLmv>-4eCdD1YqPl8yC_qddDP)j?F;NNGTJBNt zNWXzr7<*ZN<;+6VR7kMFI(%`#lfc7$SY>4^^duz9dalcsS(V$0Gg=$@QDAje=E$+o zxG1_Gab}PT!L~O+*A;*&BRcEX&2S{`kzs%E;>o~>mG`zC4XDg$SG8&UQu>NRRDZ7V zO$$$IwE3setqgD70~oWW4zx9{>Q$vrjz|_xE;vs9?(q_yVoWtZ8 zY1S!G?&ilR=3<1>&L>y;N_utEmOoIzgyh2%3f?1`y*TS6N;V>N7v-Rh@g19r z)7XNKP8X3A-sb?umoD9P!J$x?aK$xHt}>uOkd|u*qINq|{mcI7!q*$8yVhs>oeAcS z)w0=zN(L=7i#p?*FH%W%z&of(AKzOC3~55_R5<9a;ef}TXct7keJXfBuRjj_1i&nY z5#OFJ@UpYFp`Z_53h{vKHi;vqtT37l^a2IHJortDF)V1rfkt#DoDLXPSC}pdM;J!X zdA{HQy?6>p9%lz5lk4kHK`QRnz8k~{p^ z9%nPLDNLZi*yO~SZaX;=>xa6&lZQ-~^RJuV7%T;pbOSkY0FjfT!^M>gdgD(ccWP_N zVYVTf!UAE>9u(H`s>4$+vOAtom62By)glCRSt(%cds8Af2pjN;a^N*?ea-)F*lmo& zlpp$>59@*-yU2aHE*s{z;GQu~*EBBvQd+i}_d5{7QlpRS{;OJS|Ca@G$@TA#zP|`Y zkf-Jx2-ZoW{KPshfgb?k<++-?B%A0i0XSzY0+FE;r!ZZdJ?N7U?}YH_8&q`oVBt}Zn$eb$siU(DC%O&CJq zHbq<;RkaGHO4Aaui7wSq0QKr`sT2#l5)E|kAx?g0xnLmhBE=nZ_bq4S5{ofyGJApp z_&JnH0yzw0xZhHgF_);8%c^{aqwQfiiq^{i#CNdbSPc3ZsZC>DR6U6zUh$f?dG8Rb zi-0{nyi_+V_(caWtH_FFN_@6BC&7Xt=Ri4j#IsN2n1@EbcROGBrx)^l1l%pzSzg7| zIV6ou=IC{#+|Cx8%7+cjB&YrjRHa${1#Nd(1Q=(-%q(at0zX(4p17*?}Eqwyn_>+Yti$*6`9tw;$CS(SIv;GcZ~C zP^((6@B15Zek#8!?>~4lImf>F_6wiVD5{z&kEE+g)lp*wL3m-55lBEfSiOp*z-a zIHL-*)8_c&PaJR$VvfHIrPHOMyY}F&z|ZBqcuKucjKD_ReuT2gW38M+qPoi}{17Oo zw9Y*P+9nR<8Ki?I-~*kA!)zt@%>7w~(i{lV6OHlU*ALYB4IevmxdHR!wW9OI!i>~U zFgp4>1w*GAp9)-NhmK{*J=g3he+b(nkUmx9tQE8<&MOJya3f0Sv9>}qa``5oEw?Ua z?!F0eld;f=GU0Rjv4t{@C)t5UZ zd_FL}7SF-${YM&0gUl|gIydFUVsrnH^~GM;P9|^|)MmihMtTXlhexp>45c+j)ZOX&FTN5H4v+2C z2}oWEdfKB(K|9^5^(OJGNsvmo=xa@C)OfSGGPh$HYt~hjZr-56DqL`fy|tLoQqLiY z(hsIzp%raYCyO4q%vQL|OrN-l4G$?7BHPkLU+oOhz5s|B6hn(!a~-5eL&)xB14i_xfD2Gyc1Kqq-0fki#TSR*^ zr-KF#eM?F5?ElmjG#($fqGtaNgd}ake6tFr+0gfn;%`ua8RCCTG9wo6#XMtDn|4(+ z4SSujYaw(%_w_>#4RjxNfqNk8Cg{x&~3|%5+~}sZw_Jt?>lQA8m

AJj6P}2i zTo!rde2>xcbad=k&S#H7T=he?PIG+ z$}g7ReWyfR5~FfJG$e+K-33+Q^^Ax zQZ%VG1*!j5hlgxZobx>J?TYMjF*r29;O|}ss30O_0%T&GFYxG(Tz|k?>j1V zOq2_-qKGa~xm6dKpoX**o~3a>yGN|koue;V;5%l26IEM3=E>Br{0AnoZZz&vHkP-n zFYJY|`h6zd8fm!lJ10bJwDLej*Dz(!!ty@n=mp8@F4O>uoBtx6AeRNw%P%bI6yEv| zbn6)PuBa3FAIGB4^#8kb{Ch}Xv$hOq78miF_nyuy8xqsA9-ubJ^S^rsd0%QA>rl_T z1fbS3jHDg*WPKKsrAHPZ5R-otXkd0F(Ues7#S)Qgyve6=uc_2w9%EHsZO_y(txE*vq^ms$sp=9yTBgH4y-fP5z>hXv5};Jg;l z#2LbsUB2#;B%HBa9g?$eO+djd8A8*D`7+VmgiM8wd!ZlJCf*2id=s|ZX2jZNqBgeI zIGP*3zJmtK^?u_N&=0@M&E=qFS@vx=sxV&Zv+Kf${$L<)?q*>N{WGBOK|kBvCy0>7 zRtzMaioYFXx81E>CO|7@YdXILD^i<`N3s4LZAP0$gcM?FVXB{x-vUVmwhd9gxI6;N zd&B4)Hz7vj|6VdI5&vXT_UN)7mwRn@uZu@Ifz;;Km5W;>so$yY#sjH z8rn?*+w=UdHC0;fZ`b`%BDrv^QwY^5DvOlW<2Tz4;7wQb@hFQjEcCU(gMd57ljM&b z@7TRz+(BSs`<99??kAm!PWN4Elq`T+V0`_|IXGpF)b4slDLPLyl$dS09>q!Nf!^}M z<)v!|>e{o%$Osgs$UmRn&<2nA-+BtpbvyMO!MoLmy!*_0yPx9yt$QYpMWI&0IpAnE zv>aXi7krERCSNNCs(G&_6vf()V90knZZer=K^rK(b~ z+jBDrL+P3^B)&1)E&1oDmEeog_XC9?c$h%wFQvRq!YTgRZ71twdL5Qj&45C_9e;@k zHssaPID~hofiE_Y7KNx%O@3W*Ssk&};~74fh}UAL>c>d(@vE>fr{#nFQ`TXKoUZ3?m{ip717{}A0cBs>QZG?+->iL2Mi)eWe|OXQ^$>ltpC)Tzj?U#R=6wV> z_o`Dt-A{E9p{tv;a5uVHS5aX)LaO-d35$~ZLCv8>B3U+xOq;=ux*QI1Hm~E6eyHQ3 zG(F@@*dJkCJG9POH}1!VkztI-TZvk7xyuPSobew^bl$2VzQAh*_+YRi(x(yrKC)L~ zzI0Q9)(lT8t8XRO@k?LoGp!GmY)|5j9h?TQccY5uPE_F&p_IFc^i=4#FPSvf5sybh zq@t>)X#oHZ0+p}SNgWFkj^H9!l$C-a)1-Jfeu&ITY*avHEH z=>#96khHB~-C}*@d*N{t;A@6YT;w zF{Z`*89HU|Hk7u=q~|HyE6>oD1|?g!8_FIpQLw1GD8ukrz(WFp<(YZ0&oa!sQLJ<8NH!}^$POXTJ>S` zMRrVK!$u=wXKp`DcTRjI%Oe{t5Udl0qk^Ss5_^c<&70&t{qBpI7UzR?7W;A4IYNq$ zY|(fe0o@sOqG^>V94Bf^*3!45;G8&T?A3{KmyB!gK!}>3w4)g`MViSJjDXdm@C;ii z09MSOV2JGrCxpR`Z1VD(ze-!j&foxGmVs~K+3JgD1#@lU~* zKnuu&B+%33Mfu223Nz|Jcbaw*%Br^;W+1toRU9a1e`ZQtzhU@eP zFPG817g+1Od(tkRSLLSV5nmsAgoa4)JbYEsp0iah_M#VL2jQZE&l``knrrzf>v|c( zw@XHPA{V25(e?n0mJhw%Q>zPz3;^t>J)lj}nrbALBYV(b>;VA;EMf6opAeG-(Qr_8 zuYfQ>+WWJqxyF@0xiA=%eE@OnbPOMr+iql47nFj+B7bqhqrvrD5CQ{ zS-cu#^J>&vjK|D9n>`C40(b;7jKk^0N}SZdsV2h|mj^VpkhB$Rb?X!1Y4MH|`S`+r-dSCbO%`KvB`qv$9{1vuA)X|!P%7L~9p#=C0!GzGe zKc|&!vZ?59k4?+qB!5GoB_20fv`GFvZv)Ctr}&ma_{oK3)E=;S!aMt@x_{8iPi)j| z>8M`{wsQg!3Y5sMq*Wl{sxu}JMq>Rxm&mjsm#$TC6 zp5>FspGvQp;ylgpf4d-y^R2V#TN#hrjnjvml4`k88nT?p{8f`h8gfiqX)D62j&>EA z9_ zzsGzBrr(bv_(#iI7V=cpWQAzYH~yIWfeOSW_~ma+?f+*_%tNp=3blBZV7&4AK~VYp zsEW387iClMSynsNL}0^$b?BfB+YAYm2&VSDvcC^LnT-5gm_-?43BpKP0IRbS5`Kymgtt#|;^C7(}z4%E(3N&3AXh_*CkR9>1 z{pInyCp{dWPZEf*T6zPXp!Wu=X2c`G@-cTAiZ54Y8wAH)HQIbC&W;idgv)g%vwfef z0Ja-%pa(Od{&{a#Qp2`0sso`}sVLNzdrLIZ^7Tw6RFapD-lF1R2M>R7QezXm6KkPED-e%x|5vnZ2LryBHq|q@d z*P+hy1i)xU4Mcj1{zW@2Y7J$B68qv(4uT0AXgi>!EQWiK&4h z6MTnEeEEL6J-{T#9iQa8i~(gg1g=xjX9qSa3;XN3oFvxt;;3k$UfD-dDi=z2a4FjQ zSQlx5mmN_$fUFzfcit}koGB-XsjwFwDrJ>K0U*)gCN65Vb#b4s;ChvxOV)rY;=&3B z3fcSl4^4%D^Saw7YDv#)ciI6|FfyJXUvl=&&A7S?YC|^5teL55EH(c_50X_%PTm*5bJj`}UG+_}pX@&t}vtAUb8af2Udo6WY+;*7~S{Gar$UpMn0O@sz84-;3 zo>sIIp<`p)5|CzBgP2x7z2oXFL3UC~$)#LgbE*j{o~`L+i%?9MT0qgD@omwefgQb2O@Ujd?T~6qDOQX3Snw9@AHM;>r>3!iTm)%mn(M0xoryAB`+}%;a(g6~Ys+ zXpP{(13UAW^6hVd)~RGffTAh2`x_-8vR$q(rIk*$91dV1vsM1de}48TNC1=jup5F) zITIkM@ii?vDYn_tMbRPeHQocJkGr!aV!kaU&oXO|`~brEd=chSj-qWIH%yzz9i1o- z?>2P?zRrO!DWnNq;9Y|-J~fjyFsQq!H+~9RaiPPmk1HPbR*V9%$ulu@P*U(QM5O%A zFai5qhP|Ml16J1^U%Ytb14|@wzPjY{?LhX389HrAQpaoSOIABASJ4Q@p*axIu?;2x z_7*f5jaDq%DlhH5D<+Hb zKg8ruF;03;W@;y9M|Q$xrMC;@kUlZ^tid{&h=nPLlaJN`wbx92-9GLoW|_w{&5&YB z)HjVjgY%hmGN@An^-Fz`7%ux|XH{!*-GyeW)sxm`{#5=YT#@ap+dYLY@)D%knWwz= z25o0ZqT)9e%0pJ}@$?Gg>|u+7qWz}l-l9GljCF(}OLIIg-rTLGt4`8Iq^qak&UR?_ zsx_Jfs^zH!3x0$?5Rt8eK0QIke7~TYDXswsoAAs_Fem0eb+_1Ov zp$smc9iIqB-Y2Jm-MTry^BPDbhGltbHF;8?`SnF3V8Ho8niGi+iyHhuO@u=3KDB6L zTTbP_u#=p4B)|gQG*tY1wL$8s+>qaxGbG}esEWIkN{KYc2ljTG$-r;%EC6#puOttxl%Dba#((T#}U!(uvN4 zg$ZT}9(vcO@{^R??xIRfFie*qC>t#(cfi`?LI`^;o;#y*+2sEig2X2a#(a^T6ofbk z$B)b&MlWwP1YD_5Mq%>(>Fca!_Q5x`HC}_fx`@#eR%AZO;l(=6%7`N_={17s20Xb z0>hx1i-=X@mO4%Y?9-aQ5A(RNWx9GcA^%X0Oa^YW1Pi2&P#jPsT3bB)8e!;9aaSX*zl5)YW&mA(OvGY$zFz4aJb zkzVCG7G-Os_=efmSYe#`>+U$UW`yr|q`dZFhspuQyPm>HG_NnN!{9*=JMU6A=3RYP zY^qTzXk-Cq7U>?U75u)JFj_?iOSMQP(LfSkno$Nvty|{aIiwf&1MQi_0!nPVln2sr z{?9!KCHJq~Iu99jtt=3eazh;IXn%Lv|F_4;N9Q6Mc7QcZ5bT5dKU%Nrm4mlEw&e6- zGW$g3gVQZ&b%WbzUT+He%@_V4;K`q-C<((L;FH7z~{6Of5iG#dCB_L~sa!LZCeR{hD6! z%7FfX|i8MYc^tCyy+fZ*nEbt7*8C;L&v#K zDV2~3w4{QA?j{{re4LJBoi)YW7tcMeD7UL~g6>D~A^0XK5gl|INlUq<$iRB`iOF&XJVci)*Sh_&dp6TTDxSY)=^&Hw3M`Ha803gu;!Ft|1r^hz&E2 zjP!yJ&-vRxzoB)|AED(lC*0n!>fG0Bn2xomV8xfvDLLXF-AM6>@sg0Tg*dFuZ`u>) zDi3U=VzxdoRFEnJI0r-?hqg&%uf2Dv=9nAb$*>rnaB+e&^&*g;;y+x;$7@!) zRE`fF`S*%#fSf2-DyZ?WVLMc3*OV;-Kt{$i*hk*AJ||O6PJ8#Oh$(j$K;FM*&bPC1 zP;oMAx}TUDI;|u;L4uR5=-|Z2e_M@)t6%P#XSMT2IV<|o`LPPMg;v~`h(+_t(1|d~ zm>!ZYWO!dpRyi22y$zajcN^w{o>w9V7x|@MQREi`3}_E|4Eu!tT=-=NE^VtSsK+uZ z$Tg|FMs!aYV{NJj=3S-(C?=>z8hkZ?zOFqn7}EGrJjeY%5t3}==Msb9I7`)Q5>|Sn z&}jsIz;jD7Kl^8L+K*>1!k~|REKN}WERW^N;>**{MZ9ZQY64Zrow_QUj zJlxE~f!m|m*ag5}Wt}E~DO7p|T|bmY#fWY7R}nr}37e&>YqHV*ts<}kt1N@_Y2C;Y zq+mHvvQ&%dcI{1ZerVHY?$ngE0DDN5bBCL7@5yzSLC!!rARL8n_dt=4{p!|I(=gJF zSCu_&8LvG*%y-wbOX@3lj=2N9r9usoBnCi?m8{78R{Ts8IJjOeSBcL|?1tcbIVSKkwv8E!^t)73q(@zs<8kM?EU zRe~_m5Qok5o!p2Sj1v#SQ#1kZris9f`I?2_SD5g_yiFw88e9QZfHX!hNzkQ@L_p$0 z<4$doDfR)Q=#ahfLt{T%fk2To$?yASkKQAsg7{aT+voJv5;`YLI6R7@`+c!s#&7*g zka`@lb0@Q1Z0kwj-yCM{P{Sz?R7B3es++B9?4_%=SG=kld*)7u%`tGc=DwzE zGN(ff3C$dZpFM~{Nl<%p5Cq{2X<<^301r2so+&N~Ch+iBhb0f( z{QdJ)qMy>d;lLmJ$=g5}Wh8Fh`8B+n#ifK=q}(`3%It4BJrF5Bd$|hQGDW|pnHi;C z6U6TFYhGGVsXCM3TR#TxeD^LDL#gn>2ABBUVFWHqP8Q;@eRIe|h%bO9CJFZDMR&Z5NmQ>S zo}&75lr^D;s&G`q!~D)VR{-@(Ed6B}-6&2`Teg$qlOfONYUGzqMD^jM>0s?R{{WUw zS?2-|4hk<#*h3+TIqpHRZHvCeyyLD`in!`vG&e7$8VZ}S3l>%y+8~4jzh#}7RQnDX zT3sYfp>YIzb0K@%M7)xY-2oWOPuO|1MEZM^-hfa2m|U4uP*W#|@F=Si+a6Qz^QD5d zSWHIpCxh5FvcDkneNC#rqr>tfV7Xar+3Yg}PJdH6Df|#_&sr-rK~uw0=CA!j1I2%O zR~RjM^GZE}7Oxk^q~Qu(LNm?KCYyi6dz00^6*iy(|x`bvcI5`_;}=hK>b^#bN(uFH~? zOHuFV>)!vE!sVC`FfboBcs9Oa^%oYJ%a{`LFZGZBF+k40)M8bcJr$p)yA2R0*8L{KBii0kKr7c7OA z>4Pa9h>evgpW=|?7?cbE7 zNL~DLH}y@_73y|3>TrCsRejX9SZ)Xt0v4g zYURB^A5%5ZDHdA@VfvaYEFrArdubK(YoIW}$73wqMFnrKt<8KwxQt!96>zMZH!jsa z!eaDCVfDnYB)mqLLQ2$6ZH3}G(sR7uyfO%k-OgdxLGqbaHI-nUG#UWc+e+?{A9Q_YJes+Z|FJezU%TmzI4exxm z^88TeEQ;GldrxfCMpHPRA7e4^#fH8qdZhy)+L6B@_#=Ck_p!rNvkiCB0DTYIdH&ct zZ2XUw$tBf5j9$ztWB<<`nLtJ;B@t(ge%WLm&G7W9wv0+{A=j7)C5 z-Txnu0`C&yrp_Bq#~zfsEV(=+K(){XosWLhxu1ZpDw*P2ME~7zFl~B-`<}J#;)Ne!6Ev!m*?y_5G4d!lwnu7~dBGRu| zcF(G_mgB0}z9zL~L{>P$yTQzJyhV&h`9Gd`iWpd;u;lBp?ca>oUM=i&8X^97z%`}C zMA8wP5O|h$nkl~fi+u6K`w{ytK~c74KdY32R14}ug7*xS{omNsVFS)ro^dW)+)Xfr z@CT6UqN`19j9v+;D4jpF%`;j9Feu{KT)BlaM|8Ov$kx1rSAYCC^~EXVIFQx)vPojej*m!Ah~w#MtCO%T=&-=D59QQG1trj*p3BxG%U-R>tff@Q7^`NK{s- zbvh=Mj*_t2{304}gSO9n(>p)g6dh$`Myq2=K`7NA=rm`LQU73KCaoOSZg?iGgk@#3XX z)!wV;n^ANSM7PMeSknWk*$@s3u>eiCQral7u##g0$#n~uqC{tG`S+-D7^GElD@=Snl zqi^sM9Af-#yK4%@4!A&MOJZH~R;7C-BnL~VAT2&r!=Qgbog(8g`XTAm8&i~0#wn=@ zg43oaU=@VDm)mP|pVsp+HiH(L(FTd*8`T3YOrvPn98nHCSXs-cGQs`*YtETtw9Wsj z+N?hai=#9eyo+Ux2dVM+0jLt*Rk&*Tyk`}gHF7oJn1tYq=2?d2Qv=U@fFpg#>4v+a z=$IjmatOTEtvp4%x8jZX!3(^n068G9)2pUOZrHs0?*fGqtQ?VPRF>>Q0#z-30pKZ; zcApM~n3rXb2R~!)QTI$q)Q9_;K_&c44te4IJpW4raz-{k=(Z(>W828mZ>?-2pT=yt zi^gkOYhs>&;nt)#+DoEVMH%@tgq^*8K1;K zT)5PAh*QtxC|oDtR4(0}8bvjWVFd=Yt&wRY%{F_@ON(0;a`RSU`|EF3e8hN?H6AS+qi`C8AA^?`&(yCXIT-Dnd<`46us;WEv9*7M_ zBTJepUNT0&7GzM=7+eE(9ypP(7kKi%XuGpF`4&;XxW5%ci6+5@rDS}_q8Wt4T;g$K z^fiOTu-2ar4({kYy6{W-A&e|)T^ zm(Se93fO_K*}#ByC_e+(ad+t3!(4sFZWyo~yvaL&yXaUB_HJ*?l@m>%L}D1$Vmzl) zTM7{)UyhNhej9Ykcu4J0KH%FL*EldnGCZlb1OkN3w7H;ZV5=b*Le{Y>Zu<3+Vb${v zt>zuaDH;EWstQ-NhYn(V>K$Z3JE#`S3s&d+ zcxS54Bq6E%X$}84@626#<>Gust??a+$HIyIMuQ0=X`q95!7ti!U2!cP z&$fjf*KK>SocePLZ3@Pz^@}ntnbYlpj4XrD-GIYTzo4QJ)Pm95D5cprbe(;aIrYlv zpt0zDm>CZK<<{Mu1q&Gi-?*%w5&(lTI&f3{JH%*Wc4Hq=K;-}v8NIy4sP%hd%rAl3 zh5IRAot8i7Sx0FK;Pi60CI@*^ZfimbvfDd{ynA66T+;1rYmWx=`?CHo&Rsbt}e)8D@`2|HFf)|t$H2*MFbjr zruNlfg41Phvd@tS`(kd^k$qgH4<7H$a@#Yy$LB|~bbR^=ynF#XPzv>s6_Y$ds%v3j zPsPd)pEk09(`=hjVaFp4r_-0lsbZM?)}3EbFEJ9bN=yXYElPv^pk4!Qf_uzjxaLWa zI(0QSJ`)C<>(0h=MretG`P>pFER>|2|q>^aHY^ZD8dt(DL-XuQ_^ z(xlo^-K&`8ix!PB3WV7_q9r5X5PZ+C&u)I8D!`=}-@moy=VBxhKg^(uiY==tDrdZ1 zNZn}DEN@v-mjiN*JONg@CSc})8VZ37YyuQON)(TU54Cn}I(59H)LdlfA>71x z107Q8Ehfcp903Imjw^ub8B7bioS_@+s_^TzWZNN~FEnuJPB|YGO2ixP3Zav#3;jMD z)9$BbP8A(af_!f7=VqCK`XlZQlD7q@O6+9Rs*o)$Mno;udMDqN(M|NBwnn@0ab%5< zP>B5#(`6pCcc~Ng0H`zX;g<&?*Qu-*YXAJ8g<6{Ke99j4qi~tyOHj#c4WGN>PSUNu z1jXR=!u)qu*$IBcU%O=|QX227hLVf1PhAv?0!e0F4E<&tSo9m~e1({1v8Swt;Wg&0 zB9~<%-Hj&)fVLe>JdmIQ6!J)kMQ%2ty=u7>oU}2JF+ku6OPn(PL!Z*4d>NP6gGnbI z-E)ACd~sFYI~v;2`C$X1O+Ee2;eAYj&@vMc4sEKV_kNp798dLgF54GpRntta|2?m9 zPH*J~^rEWAKhAodb!Lgi*r-L%>XelZPJ@w)z2l7`1dLQu%H*%cf$rv3=-2%^8i{M` z8)uIxIOh5*kEQpYtP!01hOwGJQnbtiz9!H7{_q_q z(EPlnS9h(-Wih*TY17>55$$8C}C>vxo4-o<3 zIK3%18H;2>Z^C)>WRaa2<=SL(!es&wBU;$8;s;Az-E(mvqMnV*#6-~)s|69HPO#|l z*ns{*rQMR|+yl?3FPQ24&mtOSsa}i3h^;?=>h$+qsatA3ciUoTMe>vT0uR#4S5h&W z#zy^zxyp;jOveiKqU>A&9P>t=A&ZDv+pa(Y&HUjv%ns#jfgh2roy{8zu(^%w!8}+I zzk{S^F3t04!LqUSrU?2d`sqtW;&t;M0c~6C|EHPSVy?v7y*_GP?@BkW0P1YK1x=le zxnHYN@?g~?6uNxgkc1#1%(6*&DEqv_Zl)Y>v2PoEHh?J~EtG8`dL?IaF;@{fdRiVe|7;!W+CLkGbpiAWI6+RoP#K4*paYpSHLFt2Tq zu4}Z|dhh&lga`+hUbEB)78PNJnd03hkN&)QOrzDu#Dwb<5>6Ie#qPlayGNmI=o^VO zXsHVjO{<@uuEC|a=>fmM>xQv8;Z1AG8@YuhOvlw!IN8FSsfJLz->Ij2D%S3bVYhZ~ z?RbY`V73FLQ1`#vhEcX6zPC&hZ92^G`<*r+qmglJvf9UvaqU3)!`hyM(a+E}D-^tw z7MXnZg0ILb3(l9=k+hf3z=u~vKBC#8I%n|>X%J46 zso#hQ5&0`N8|@rPhV(kh&XbYFk8>@3GBo>a?q8QTu$N|~c|gFRT}Usiq3SO+SKln2 zWmUZ(M3i<633fv(<8yf$SvwZtPrHW7@Rq^t8blBk*;)ijA45B_tgoauv=MCByZv$7 z$FMNXTBhwElR~yNvXY$W=Bi7flR0Y+y*sBd&^>AFU~SMGh*ohMxyl1eSF;^2h85Nt z2D@a10b5809vGmANkb=B$H2kV8rCcW^!YDZ?f*V0kRxJ?jmeD-w-+BC0tO-#08zin zz`n2g4HJN5aqXY=+yaWV!fi0X53&9rX7ET+&sSR2;n8>y_bwS)jPI@aF?2giy|0L# z3?*a-bRT{P?2EOHF)?mVoA@NS#83SyYQso~-KAxxx+L)b%pOlrMybp49HJukRHZ`h zozj)M#4Z4(gNcGYx2=o!p%Q^P7<{6ai0XNR{bkUnrD=)0pklI-_nP!4Q7f;usNVd% zElgJ76X_69?;*Stog-0_a^+0xJRIda%)3Wu@Lzf65q`ZMZnTE<9t40Rx~2{;G-6@} z8z%swK7x>M8Ja|C?MDI zvv9^y;5E?>kp&{>D+SvQOVQpr>S)C$J$q^G`9J%bn^`VgQT7+B;KC?Lq)kWtr-{!LJMG~fgjY{h*a+tZMBz@*!|?-hu4;wV?$^CfLgmPCLaTG{Rb@_}6cP0n zoll&@d9yR43GvLASn{EWdS2())U}}>md1Kq?ehq82jS!UEr|;3REi)F2WEH~Zi2qZ z@_i$h3ik}9O>pf%bBaOzGs+05x$dKu%9|E5V%bUis-`W}hSCI-`of^>2A06H5C0?8 z&}oQe4HeMkH<_u)6bvK-B`&E+EshXH`Dp7o+Sb1D@e00Th$zo|5YSUobd#RDN? zla8;zn!k;c?-+Gv^3AD(b1;OVfU8GTs+0;fVLe*MC8Oo#o*F@De^9z~#c)qq!5>%< zsU+)waE0Pv{S1ysP5bS8xXoF@*Sj8asgPs5Rj3UMa7;1Y+XPK3M(8qZydS^Kp8R65 zt3HD>w>sUhi?lhJjGPS1r(9MppO^2 zMhpt?jsmuJQIlVYR~OUicESv!Ra?^VCUa$e27RVyjwb|BT)tSbdI}wCGt;|x!=Q= zv5G6iRDl-Hut}v=k_NTG=SXL*%ge=Y4|sJa$k)l?xPYrpR?XW#;o518&gyBI`?Tgq zTFV8eg%B?QRq8)&M9L#hXa#}vhMcaneez-|8@aznjKaFv$+X^JOvC1Vl-QD}eOQ!4 zS%T(HTZ%s#PT@|Gg#&OXsy-3B$+Q(AxdY09(aH#kI6W48nim+V7@-o$fiP}}P}iQ| z$`MB`Ls;It9FZ3qHuWq%cnaG{*7IW`3EakuehKhmy zBu*Y%N4_}44dOAKbcxnJEtBga#D`9N2)s+=@0#8XZU`YkMMF;$|D0=%xegm~A1sNM zRJjUx$npib+mZ*O31i}{CgFqzv&o9=MUr^!5Z&-^AUww#bdDzGeXxx8HRe6BK{?2We?iO?Tt2rWQolDa7(@Fj*Coez zwQU}@*X6UjU_kwjj!3NA0Y?EYNu&xkn$LBt2u)*o1^o=S?-l$1ZBvn@Y!H0Vf_1~P z`6UAN+&?L_11sUMTo*3IWWT7;y2DBK#^Z|X?~ht^!y7VKqvh8yNZ-T@=PPruSWxQe zlX)Q@Mb}S0-W;d*yIV|&K2ywwf-u-=*~b-WM8XwV1ud$k8_a9l^$AB_A)EG1syPoWP_l@jM_H&9yBHy2`t;!a4^qD_IpttT zDW49c4}Y~H>rrZZN@Re3xigxC{*~|?Z_(vY0`OJrPPJ6mm`ZOWBD?!sy8vhg=~)a5 zba$FskDA6xNXfp|1o$&6X2j+)m6CF^bDUXD4{eob%rRUQpCo#uPAJad4G-8P0k4ag z`yGPKM`p3xeHt_7V7W{QGp)EDrMw%-ILbU(R&-V-jjT#IkYr212?P2-_Qb>(TvWjoDa&e_kNLEh)v%LyMU|5Wl2v5BZ@Lel3j+258 z&1g|R&|TlK&cW?^7A14#*~YtTrm9_FhSa~uo@|=Cw{rbPiTfm>v44c*y#Lsuy_}VG z?uezxWuI3^%UX-$vVfSIJTfxS_gER3uvCskPu5Q`kIYv zd97fBGvbvirbL)7sbHmaX~Bl_W)y`88JAl=M3oiP?Kf zL*LZ`l@LRv%!N}>KJ`QWedd_msWc2wY#JWcVm*pV#zsq8(nEJ7(8OtkGvxP(J)yHJ zNm#xC5IVAAOd36=w3apKK0g?J>VHl}9I&q!Va`8fI5tci=H+EhTf}tlpB4300 zuE$Ys*~I+7v7D!A;m1r_4K;uDd8<$}W6F6jN?HG!6t-_7E2dd(0_?9fZY=Ev^hWNR zSDKT3H5b}getFT40tCfG%BId-d_INPU(H{mjZkGk>&je7jNfI6VyWR0P^c{r#2nAT zlp9Cxb^8bk$eZ!bku~0d^;k)8h)?f2viN5xUz(rlgp+iQ^huD6H&r+rw%C$;^GNAi zYrW#OU2<>PO+}2Aydt7EVZRB8yBfyljqyekynpaU91VQvdU{s-deHNJ$YG>}n2jgd z=Gug|h*51R=vy!_= zlK6aFgjBa>AMn-H$$HK9X&pe8Sqj6#co>rHP=+Rvk5!Df6Zs1`soJl|>$<>0(`oTk z)nR`(3-VvC;|JRKs1B*9oy2ywA)BIuEC0cPcGB1?td}x4g87NMFA@PTNaPr2zLeMD z6K+0yf)s2d@3kgf_-Hyfv5?%Mz7xAp;pP)ecRilA*)y>E z<_|g2s4ouj0rH|^B2c;SurlsYQ0iGtsEMa-D0hF@#FCWv&!U*m!b1siIdWI|(7BK~ z^-Ki1N{I56Bj~7Fub_AW?dDB`61^o~n(pDqak6@Es?J%Gb_>vVqL*!cDuXwq>TV2N z9k>Qo55h81aXL2=nsSgcbUc>)!LD> zCTD*zxj!VX3>M1hd&r8&^i|_b?aXvheWQ!zW?7?{T{9PWqK(Y`zsLK7Fer6fglWmy z)EVSxfCHog^f{V9ym#pzuMd=nrrwT^OQ}G{eN++c7tJCb;=})0&S-_==j7hORU4{c zW3zmN$d`1+@+(>pibMMbAdU@C#C|m}5ZiDDT?yG5{d_ye1)5lYC~XahePBfO1Mv=` zv6H$Ndsn7wFp-6=pcJBS_8iG&r45-UY@F2BomRUbjqKU%Gra%q0wqhmX9VqQTFO=+ zjB;aJ_n{VfX7!{~eM$M$5NNfRWJYs|A*PfBblTzgXdD5JlWJ6gV!;~58Ou5!J`Js_ zocj1uL+ZD*BV{!eC4peBVXYQggwd`~f4ID`U-T(Z9cv?B{ISLh2X-vfWZkl3g26AvwqEH+Q4L-{W~QLveMCi&P$6P?Z>j|a&o9EjcO30&rtmNjIoAsx6`%4kg}&bA$BB6@ zn0{xDat;Sv)gR~v(|dsmm8|lxFxU{mPUax1TA9#Dn$0P9;Q&4lMrl>uAPiS3O(BL( z{k$!am{eiU;!XiAZr3n1=CR^c(bf9wrr@qxLRRW~zrP3)H!O!|JF3R@Br^F!gWx9DSQZ}!rP@>&v(gOT z@ofRX-cUV*KJuf&7cjlM(t);fz(d12#BBsGA!KodeSUezQsjHpBErFZU!%sg11NJiu=j}-bpUP%D)lv7t8iXpT=OE=wI>{Yay1nE~;r zc)rslF!dD5Sok%CQ7)KUWAW#vV!z=w5`}z(7=%Ovj>V_;wn{ho12;cdML||>m61<| zP;F8oNI$$YmZV}5fIKpvnrZ|;uBVr(ngfd8J{a0?rVFZ49QSBjEu2LmQU2hYwp$p+~@!WJm8g1`wxI-uRW*FP+ za28wASZ@pkj2=nE_1F7{tFq!N?D!GUom>t+VCTfQjX%S@Q7SpTk(w`XE$K4Bm8YbA z8x^q&;|dEJQ6l34$h!Vic%Dvaf>1i@xIq5o!Tyisp3mUX{E}7p=(6X)4PU3;PFOX; z&lN(zDhXFFC&y10>I;+WcX%h~S}L$!%dAXLn;Rtc_Xz&^RHrNgPtJ5bIW+p#X=zUJH&vinToX)~d?@XI?7V-uolz5q~rvta(Yw zTUfwrg)J6|<8VBzuT)~)X||WL;IvJj?3)=ipZ8Hc5_=olj7;f;myL z_Fe;JxtvXJ!k|+H300`RRt>Ri5!yncw^!LnXmTrO-P5ENYpDW-d}Dq+J4Ow2D+9+T z6lxRX7%RZNzcP+zotDqK13aAFqlt~lPt3j}3PO^5TvkoScptRXm4fmR0G*bf8U;PE zc7K2Dg{3X29$u6PZ$d^`eQP#V9YlR3iJs8sNWX7$O+X7#eZ7R|&;y`{aT_uV~{iG_ zr`uje5rwKL1oJSmh|!|WCWJUsNoHnCHCKp`_Eaxrn*RcC@GGuglp@Gwoh;H?&T zaBao>L*LunvI|pGqr0=8>S+keckko9V-Z6OMv%OTHJX9s($@n`f~U4ao=>hbhNg&| z2eWM<+3o)T;E3IkKuEn3pW1-3LFY9Vip!@mrvkosiU1V;FyML{G_$Wi=?vXk=Lpq* zQLP8o88XOnN|r#43Y;PjqK$_fHpAi@hO4sbLTAa~*Njs>k%o^&Q>_S&Y##up{}o-o z=Ek^E-4zl6q%?w5ML=6UEuC8bkU@}2w(kG1i+^B9q^zS=DrcZDTxPwDXfRkp;57J$ zgXZ@0>nhp;NFu=nT|Rbe{H~~Fk2f%K%H`;BwSc{oBqg@990aRVE>)at69tAFW=>Mo zEI>`@H(7u4b2WdXrTpa4eQXcw{IH6wRHbDptae5xZ1wvWAlLV-*=sK@W=A3rh>!e1 z?>p6nvlMz!(WiEOr6=RsXM%ymWuVY%!MnjXn;5Dyk}5eTQVW-4>NOQD9GYn%{ioN$ zoOdA2@;u0ySzs9B$GE_93q)y~O%9%W*OwHi+=5>piCkS#o(ExL*Jpp4FjJ!up4f)3}tAnXrPsgZ`}+n(#1a0WBAZv?&TD0-pcM zWxXJanlh&79%iSwAnE+mO)wxu@ApS5AKFE(K*j%Ch*E#xbogX632>SBI3N0^&B8Lx#c?O#RsBqTTMtIn-!`Snw4e9oz&nQa#vm1QJk8wgyvHjOE<=AN1z zxLZ(DuyDHaVV-Y4Q`Gv3$rgUlYD(+pyOb86j=Zou+0N_Vi;RX-+}zWWrUzj-q7g$E zWKNrCeBcQ|w%qeAe^}%&va=BgG3pwG>O;;99n?f&19+M$fkjM4Fo_jDA2&VF3pw+# z^4E68auT+sj2n8NsVCvW$-ll%zn9rv3{V0_4-6Dt!T&szN{0*MY!@#Oa+DmK33O5cqNsq$DP_# z3r*-I@;ySeLLi5m>rS?)-}aDMf>$7{ddKu3>z4LUuXD1h?v&=&}&|eeM`2fN8P7FB%-l^#2+G7(^1SCefyqUnwr4^$P=Y;^WB_^9*8%8;r4WpCsmsj%J3#noStRYhTV7Vts6So?26 zG(>MS1j5@CHk4I>f18$C{vh4+nxmQMR_R%tOV8Ab9G9^1xR3iUV2DHmqzg1 zSl55DmIaMKuw=uSq>jhuwTd1ErNpYF0+3xuVAbKdRYZh{ic!s)z*cY2k7&x|a(M#Z z4fl$qonV??1P>o+4kuym&4S&j(-`ldW6A@Gy`yYBHSV{c1AX+XoS?L)j^1Y+0ovM3 zM50xbNjCsesZk+Mw$gv+%kQ*i_H12gCH^eh&&r_1Wf+0)T>efFJ{}AY)vN^Y5K< zQ!EXs)Q{h{(|jIi?67p3v4L2=fe z2ymIzplK8_Vf;g0gtm#XSF6+Dbt5SAs2v!wXG^0`xKi$!vd|u4b_SWP;f)uYWkA(d zd*&V00m3+S#3w8&To0r;$Zmx*-rQ`y75pM4iEj*_Ge2M+rZ7Gb!|jxLK7|Y-+WZh#tBhor{W49=GxqsshsO8)ohtMSr(OXI zqmuGfJ*y$jvS{RqlGBsJBJc#|`8olm)Zrnk%ZkFnct0rPpZRMd_BD(1X&xyKbV4=g zEfx&26*^~OzP(Sdb=(2L7uH)P!n7lURFNudla`ijhm&VwU|8nHBaXskbXml^At5JYDN!tj@F)&` zprj!)sda>8#MVV(;h0isntSFArm~-#m8+iiAt8zQ5aFm29|ZvHSOm4A0VmNT9P(rMeen? z_ZnE4y!(R+QLyf0K)iM^62smCjL8vIc*~6=nCJ~B$q1ZNoX_+uZm`ohUbYOUy?f=) z4#6(5wZfn^R<_52y=NR2&DF=<%rZKzg^4cXDODQLM3lOE8u}+B0_|k!3U{F-EjcI^ z%`e9iEY4D)Np`@Fq*Fz4Q`DVBkixVW5V2%?6t^;R_PX zDn5n%@jNhY`j2>Yw)Qr%fJo>}R_LMMDt?G7sNJv67d&2g~|a z9Scaix!~&leEijyeb3A*n*HO)7yp}y7i9YXA2&# zY2g^lnMRr{AiZlIGY-uT(t%#D4mKNCMI9UvSn8k6QUr(?j(AaKM%0ZTH71v-(1CK} zBGA4>T~gXJEh|e2*iRG_j%Snc9R|Vlq+6@F^x(_f`n5wt6DNUVaU*c3lO19AEp5L? z*Wn?(jGJ;sedj6G;M9b4AFSJ8>TV8$_o4>cgv8h*TT{wVuoP#Bn|j&nL^h4ww09w~ z`uT1UoLx;*Wb~W&3pT5!NuxI7c)M?N9z zc78iH1a0H867!U3rOD9|{V=wK240M33dS|usr!bXPn3(m z4u)XHKUm|CnVA#ZeSvx~JI9BMBwr%D_FNW#kQj)dz>@r+tS5bJ| zP%k#F*Y7WI!?5oUT}MDNcKl? z7pwb0kc#9~qT6d%C<~6--2E!YyzU3+zjr0{Z8OscB?;ocM8c?IXXxCI!T_|UqH}iWWs7%H2!MnmKD4?89z^&MzV9JM68;hTdtGROE_Ztb zcBwSKfqn8L??_*f;s|!a!_^_B`a=slgmTJbCi@e1H@Hn5v?*7AYf?K{m`JgQ(bKKF zrZTqxmg7huCiDW1^GKcy`7(f;cfQ%L)~z8h(q zU7&bkYyTHky`IVA#WWivY88YRE+l09-)CKEFGvPL5x^(z4$fz}T?@z|Q8(LVpv)el zJXU%xexNu~PW*4>RM^fkW+`kvg}8998hxGVS!8G}*}B88Te`>$M!$HRqgs_18I?m? z6yxMys28>PZ;-hB&@=+(BO31L5XJ+j7K9Jq+Z~* zR>1JP{*)E4Q%l-{2!`AGk!Mo(_RVyAErXr}Gzz)G%z-te46$I7xSnHk8>i&qA2Yg7 z8ToR6lWYBNGLUCw94Aw5&T>NWL}VamG*~mTqcr!xU}*1%oTc{nfl-B0Sq~>G=8brWDXH(n;)3BhEgS z*{B7U#U&tynv(b#MMQ@=YDd2kW&gGTk$ukz*kp=X%2vLUNq6km6*jZWg_nsCt>f+j z{gR-#a8eX7H9c#nb+!_P#+W@a547uIg~y9a-kd){R39*XWLB#yPy6JhBkaE%EhQ@* z4H9?ZyLQ$5G#Yb*K7B$2j&6hPjsm$We998G_|IyyI>*`5@8yrUvrg*m#%n6RckGTx zGcmbC;RbQBX7wNRGWt$sa*>`!qd&qy1P@5GE}EFZur6`5r<|gb54y`dj6xaK6kH%~ zh8ASAS9#lzPK&HX9ceNIPyT{0N3;jP0m!Bl|rHMI_UJ* z`P7B7T$OK;y4wGl;IMn;Qqi#LG%%zn_|B=gqW?Au8U^gRen1yha^`62mSB0D`*%G1 z`?WoV8l`0DsY^#8iyMk=nRX^@C<+pkCN_e6+ogkpNSYqzZ6So815np}h0Q~)2Mq$N zS;i9DNUk@Otx{Cx>%vK7!PogL987%7tL2sMYKR~c3>U7xR*-wUV%EUhP_m`S10x;5 zG8EWOpLPQu5^1J=D40`({iVk`0^Q6zl-k-9(XjS@A;W}=B0rOSdPk{Jf(u0bigNZ| zUcKaOS=c*VJ}oTr^?-|!#caE`Kv0b~9HEsap+aG{aOxTBI#{e=HzcbLn7e#2$W$g? zRgA!le=ykQAZ0E7lZirC^GEUN@ofFl6n&guu***^WYFxR_d#=x@cx6XToGeBpd&wI zL$_Myt9XjjoXG7pv2}dEX!Q+KE_K*Fg2o8rc1m3MH_wXZvQ?Q(g|9NJJ>Nv7lk{Ao z^}~)!>n~y&WG9bJNxaeTeQ1y`BCM4<)SQ~Ps573JiFfmims&`LSX=iCXmgtrSzq?)JF5a4)XQQ3knD>!_-e1-!)Xne-jqiH6ONrDyXZsG05>#dLzMxrNk~oU<@} zMbs2*f$zb*3#tCP?V?$vdNE&oCqnd2br2x|mh>RpZtZ)a%~P0xB;?-==PVxgm@;`2 zdoaG+crYJ4Hks~r4#LrGAgVko`P{q9wadSY}i{Z;5HZ z4zCr&J)~Niu-!hv_PCzZPap((a28M3{JquCSOxr6hT@kt-nz~KadRLLE2u{XkmiS9 zPX$1a)DDi212~0w)HC*}QnDyYnmo~tWDiCkcJ?klHe=8>n+uauo0r6otaETQmsQPP z4h}ekIXwQvYb>(k-|Ks9Q-MmlSu|-Sx3)U+5i61ua5ubD+hF`!tH?!0qw={RRU-nb z2gMK0OqHTi=fdH9yD=w>oim^qs|c3!Y(q6Sjb(bQPb-bUb6)S!HGRNy(eM63 zlaw-w3ZO9Mu)kjM{Lu^_&)7dZp(NI5vMU2~EQeSO`)qAC2rXwW`Dm$rpu+&jstv#P%n zQm^g}sm2XK6&z$t0{a87#014#jF*HHcWO$8hXiCd+IK1<#v7Rh5$i`@R|+O#EJGXe z6K1c+=~GFo$*l7tRnb9?B1w}{?^(b)Xhi}|5k!bhRTm(zj~AMy&sBiVNkt&`uhDnS zS!V{vJ_?|qDZ+9B?{N@Eu(rh0YtNeicTq4Rs5mAnzguKM-&q~iz@w+Y{X!9_?qhIF zZOVS3?A+O4?T5GvRV1w-QI68qx&m>g2xIN*2&Dgu6frCWFpj2*#hMBIK2d`dd_8J< z4390kb!*K3RJnK_me1N$iLa@oZ)rvL?3kNrv?U-8`9#h>Oz~n2Y3!6#=nAZSV&Z^NO8sz zx41qdcF{{NnyDOtcse480pX3o_aR<9Rfe*AvQgxMb}8vm4zo7UK2Nip>>DA6^vLfd z40l2a4Wu{sz6AYA11Q~0QPQ0pZ6TG~F#k2dDQ^pl_XtI9>O2&TXVP777iAveOBjR{ z2*fzw=Uh_vC#128bO{c(A2US$UZ7CAobp=t6{g2Vc&I+-URNx9Dw;$JKcr^mLOuMD ztvgIdMOV;ai#@(abdtbMI`JHxKIW!?^Hbyl7Q#0an=#~u3kqQ=@A7j&diPg7@TV!K zkA5MXY;JEDJ|vZqk$Es3zOhd|6>uu1l5sDL)-*otGPkI)o__*`*XS>@#dqsBypf^S zx$rb=Mi%%5ny-n901ClEefub|8MyYdNiWIhf8(v7rNDfUSbQ3tRZ&kBv(^rI*|zUy zP&;|5ro0S#r!E9OB0OJ_8cT=8X$0?G9NNLQb9exQ4tiDC{L+QbRk8D61a^FU4`YDk zM$0M%hi0yRkbo;835gtXP2gy-JRGb=-L64-sJjiS4Gmn4-fk{Xf|5y2NpPu1>V^DUt%PnHLLI3s@w&8&Foa4 z9qsr+)XKHfvKZFmi{dP6;Lr{t;=rU08(*4ff~Uj4arFlAKMWW&4O$Oak9E`6A`TL} zPIWd>>CA#xUT`YNnG9HUB20|x6~CV~=x|;^d$^6XQxsqf5I`c&svR$7jSWPT&+kkVW7#Uo4#EhNPic~Z=?Yvd+f z>yj*x>3|bTatT3*M+BPwGOK9=inx7}j9`rh>Oog!l+QO`zv{=f1fl}e zN6T6AYL&uKMT_ot?dNfeX~)$G^U!+G%UgaV)R}qOxX7XtIntsLzHrq9R5to zlC0!1<$9^7(14F~iCg`ErHG6O>`|W1Ld+g3VJh8Uvi(4qk2(0~0l;|LJb+Z->^ePb zR2B)-fTX@uS_a_Cxq0;DCGa5h-Vkz{H1B(C^c$O+&anm zdSM8m4hRnV17Snb9_N(SqmruXmUO&O?@c@Oz_d*fWQ?V1&eRV8eCp^Jf<%a482ep; ztoy5170Zx*&EM?5uT^HzBTB4X2p=c&fJZOp(5ITif8|IDM$sA7h#nA+(!`ooSr+J; z%IVpGc>{vOnk;2FT;+ddSAY2Gh89#G;&Q}~R<)Pq=bZEZDyvRyB?r6PI*F@=f|7<= zBL1!%1S%2uHH%CJrQ-tMmGesg$8I%aP2R_Y+w=$uBe$mshj6kd$}PQ`%fw6^4e0(* zb#BHBihd;boX6ChHs`msl7uc+b$hP@SfCB<=NSKG=TCG_S z@LzcnF_)iQap7)Gm^VZu=*?@k>{v+4Rqp$*t2Jj6e2&AuU6mDPR8!H4I4pR;Q7@I? z#yZ8Mo2cI*8!C9_C&Js(PCbw>XoD>~e8^e44mFd0oK47`j@Kr#4zMAcU#@#5uawg} zVr9kq!q9V#?C6aQ z!jJ7fB1tI`8eGWRkkZ5VNvy&29+$h{sH?)qWlT-8uY~Hw;^z=X z9koh4mm6(AiNvK_l6vHO>TtUOKBT6;aGBuWhIv6Y& zs~?b0DNhzKaQ@3*?51lZ;s_)$&=m3a#{NZ5snnXK@<=6p!L*! zR51^hW3p14?O3y=uKY}PJ*p2EdcvD;##?a%Dh%4uunE#%wQ^0MV+hV$)eLL*my zdlQ9tRgl;2j!JrJK-h5Ise{HxX33Ug7#0=+{1qn7P zY$WL<@KRoNrpm9houmi);E)%CW_`}X71prJrcm=vTWMJ!w#2VWEKh~U_5wdvLP$H9 z9o5ci0acfjnb=md{@0tM(Sa-~jOj{~<_|PPkJ&2y6eg)r4__ESb4IHPbo&60 zFS3c^faZ991<~b*s4N2I1LEZEof;8`JPSq%cpMI!TgR)(aZ<=p=gA$+$0$yeiwOEW z9r_<2PtGA{K3tIe0!(MK(Asb<)uau97|P|sU76i!TywI7sia)qgeN7rW5nK4$a0t* zFA3h`!jm4LDeLr@WI={j7SVCZFeDOE0rQSFt zN-Uzw)Bq-=o|U;AA&HmbtM961{82T+Amc)SEa5a?u;vhkI@MCLD5F&9_ zQ^6})e|N2j_C^xC_UYXf4+Tsi1%fMI4XAc{q+%=RVlYi5yY5xVKHy)V5jLAroo@86 ze)c_{j{y^_RhV`$nQ`M|#H~l#%s3P!{pyw0BS$ry%C(8*7H(qBNQTgdvBzXvYe;z@ zvXmDt&>az&agXU(i_F9M+y!IGkg$nB^K$-?R{wx}{P7x;VC7NkBBH>XpqlWUszh?T zzy_ZcfPV+5HV#(Y-JY0Si$fnSQ5N&k8DjqktyIP80!|Ck2TZss0paA-))nE=H>eqV zs~SE;wnfeO-hKv+@Y(}jXzBB6nmen!g>jlM30nuca-3#7j1`bZx>bw!iKvU8KHjPZSJ!?mvt`wbZPo~1fIH^&CAGo4 z!wn!ve@ECQuxJ^8PN?9^hXkyzbdc6#ek&~PL%7W&0`R-G@BY(-D6Ty0PoyPY*XGP# z262%0Y0i`)Y-WZ_1YeGt4Ue6+7dY|>Q1{y3#s2syJj7_O>yJ`rkRiIoFfvv+TkIO$ z9~totRCr9f&s%~w&r}q#zJU4fcV@7`V3DbzK*ynp_<)H4=-^AjcIhg`x%3WYMb5nY z1;?7AiVzO~u5iQzPjYcfh$URR8RGrad}#=T=qj>zWQGR4Us;HTsJh=45iYxJ zOD@Zo`AfBPN@k1AK12Q+nu^qCDhVht9O9`T10H8;%}S#~4k|IjzSBqVRXJ9Z9qM!H zkBKCpUb3$>pk%X0f=sn#Ql^K;+>adCAjM%qJOHZ>uv0Wk{bDd~(}s`Kb^`xH zp?6YjVlMP{uikWlQ>@MhWQM9|BGq!`a#3T~`s*-ry7wA}t5=4NLC46Upi-JBAj=_$ zn5`Jh%pxUoc_-ke@r^&SYLNW8cp_i&hlF%6Ra7U58NR9#q>Q0IK z+w&qZ;TTIN&h~kmzTc`?%H^G;g;8@z5mnb>$225K?>Vq(L>U;m>?xIXlB1hyU}8(x=D%F2+gYo>BmX)5f1Qs?S<(~ zMLU37wVqt7tZl*LG3sbEuoAy>(mf$0T!(mAaqBo0AuZ}1b1zDc87h12bn58F+d zW&c?XsQY15=pztdW*SHZ|NFb1{CYxV#!~%R`;iVXua# z6WKW?vXIXnHXve-8e?QoCEz0;0C_813SFD7nNv6lc8iWv-4n^t`4vc zb^ZICn_FR#TK4xv@4s9Dm)}uQzoo=twDK5nutChNAuOan1Y1-L)~n%b=~SvA{DA6F zPA|ht=#5>QCGbVAJzg7+!|TU$yM1A<=t`#r#^T9@fK zBGW9u*CLre3wfu`jSj&-`k9rdg|&&Lom>vI=99fPh%cM@^t zs6cCd8DKH=F#r^n&F;)01!z}O_7QI+XEQMj?^?_+a64%heqF?TBuTd^$4O2|{SPu2 zvVHF?QnQn*Sq1cN<#7-gpAouWBG3*q6_sVZGxsvv9o1>R#C!-y1ZZ|lCax_*aX*kS ztVh530A{jLI0J3nY!TBAKqMsn`2Klb1dFL{`bwAm{W@nJ3_g-QFGnSRmzBeU>PDs>W*Xn$%TZJaxsk=p#}giQfns?Vf?SA} zk#AX7+Hk%9C`srSyl>NSCP4~r9`*cEK`e)^VN~aLp0I};L<#>*qt%^x&~dS*h(u0n zc9O^O5fY^a+&*06OOvhw>Yhdn>1|EQ9GC8RKeV&+y*QIiFg>Q*5$;fWl9S=X`=jQ;VQ$iNKd1QR*7 zb#>DvMvWeU&SK>zE&M|c1E4mIU`q?L`xt>zNG$kIV-Z;>&4u7^d?=4KnZPh8e+g_} zW?rdaF27xPn>~F%2ltwQ#lg$Qs&?&@;ttIXkip>EBCvnQ3NcI=449pcJP_4 zFraO`n0hhUX25D$%ky{m^cnF)OfB|buoiT;={EIMMLX86j9JjY$EmVd!_K$T45MF^ zMtdkTKaq!g)*xGW_z1l>mSK8I#n*Yh7>rreRN;JW#LHTqh<8{Bi?>@*nov-B3q7F0 z`ZZvZ@``glee1AnD$c?jhI!n2k5wryZ+2q=Gf~vUVUUp3T7WG5*`YSLvN$)W7+y7|WX3{r|ybWS+TnX+Of{M!tM;qKnU`by(&r20y$mO_-b@Uq?hs z30liQx$TAWN)Ru|e3I7&XZj`u18jH5$Uu?WpmuML`tal)Su~yMcVbscV8{!`-hDM9P*JG^+96T`0Mtfd_Oh7bLnfGh+NVUC>8>e%EjFQ8GTh%p$lEf1C z2!I1KdGYWZ2TBFns3a{+$6RZ~LjiIQ%k%HIzg3aGJKvIhcjq#v#X~_sdXw}Yry7-&sP=r*yOIo;{{QaJ-eg|AMd?FKq^ZIUkxlif z%qBlT%iHz?ll#4kl6>awsUDR#Ji}p`_fi_xJ9(O1Ip#Q`2`x42VJ*r)X)#uZ!CqV* zebFEr9xQYj9&RrDBM8|uMoE#C2&pM8}UB5XHtY+zC{Y*%LQ36OTBQ##N8?Naz- zja6?7JOu61w4R$$65gmf+nQDi9>nQ5UE)VKFuy1jhmFCt^?c_Dq9gfh%7~m2ou7jO zzkeCOyp%CACou4vo1LYcGmBaUJ0O*24-zA5v?O1CNgtnh837pZs%Vr+oSfBZi;xlS za(Te?td*71i%8H6j?Nz8J{WJ-9_)tR>qE{T)@zVmB2Cx6=++0TT(??W{%J4 zvLOI+BPIIpydfE@H!mASKOS=NY9?t$M`nmibS|FC-j9+KO;A8 z1_O|E6%qGxXGTxA}lP&;%i7jiZ;KX{veb)foZI5 zrs;0evxmuyGK#?6Z((FIIxNjQ3cA}6*qy`!am1u}ODG!H`9WFLOzFkbrF z-kXc@PPzFm1p3>{rQH~cf8EwTsQiSG@ zxc~z|{J#tb9Z*;lN)npB57s|xypqUaSES1)Z~j^( zqy?K@+8##4{^rmzciM4S)R2=lM?$j{g0cHPQ1dP?0OTqf%YBO$ewm zI4sGrFy7rQp#%)pALG300(h(4f1@XJa5HnMullR<)@2&`<`KBXB{d6EMqkOWu6A{% zlP~sPf#`XL^aX^L?52e1iZxcX%WZQwp5by7_6ok1_N8(|GjeeIB5&$fUdE8>Y?KV6 zjV3A^Xk~|rY7^8%pAX!DUaeYKMl4`yn((pFN5o{2WYMw6orlOaOFK*>Ydd=v)F9B= zFc$?SY6%@vfbY1u5e8jJyJR~{kh*#lqm$Ixk`YB!R+`+UFrN}~@NUcbv5*t|hYd!B zl_|QpeZl`eGSjF;7R!uO7|Xo>Nl(-UqZovSE==)Q)+lj|pQ+7Yx3&*97>4%iOm&_=RK?$xQBcZfUC)iH5 zhg^LCH?d+#JjUT4Q-CGgADD?AR8VUxO#Pve65Ps9-=-HjX}|;W7XeOS#In%1nF?bW zUnE2V5j){Kp-qn=t~0XBrBbAkxvXaPl-g^0aMshP5yyo_({O`ohPY6_1RHtxFtW3(K>l+YO9g*&VgVPuf+&rOwwp79C8C zN+$+=a4^p40w@UJ*?9%k-R8fZq5>N23$=7k1~*rIGN)y(Yd^doIr#DQMmU=r-(Zn+ z!-t=u9W9k8`%FA^(N4;Eq!7kO-99j23_0F-3XBp{K96wUg&x`~(9D8m8>RzX$JWv7 z4*VRFVh6wwP}m~~P17IGL;%aGI@gNlwWGUPLHj>%g5~$ehav$Q3s^$;>PmS#LSS7% zfbc}2-lxqXCgE}<)>)QG>PZthA6M^@DYS}Ovh@`55C)H)w=jjptw*9|pa=Fn4*4|T z;zR3Q3SFusKtwcJ`%{0P3~OkNVyIR&R;--DX`Hr4&bsR#BUZ`W0b5P z+6Ah3o={ROftHdM6<2Xr`_n`mj@kGjC+p#xIWmV}P{6p@dD#(FTH1hT^~PP*eslDv z#|;E94dMeDl=T5~&H_64Fzppm?~z~Fl0dIZFt7gvX$Ns6ocW6)TxJll9vK^<*zzQB zZPS$WltrN>%gu}JEl7n(-P+ixSjk7V((!&r1aMIG3M%jRMG8@4jW5mTT7~a#2Nl7H0fCn?&8x}KQNrs8f%IpIOs+8P&KT!y6us@d zhRJ&H)cFV=`h(Su*;@P2tG+(dfOnP=JCHn4Fyq9?d!(M;5xQj1!UEbfLufnVjie1W z&wkj6kP39Pp;t_EHvc)KkMgr{Me56fr1I&`>!pjM zxxNQ8e&c9oT%QNItLYCG@=?{ymP4j(Ge?Yw03}1tZ<5)bKR7Tc+62eZoq#8@Op2EE z_AoeWrE`A_a1CXkWUDjkB^ZF;I1@FuxvGElW19^S78Nw6rYBlCbS2Fi3UIa$R<(ha znf-J9Bj~!}v}_+;k=mmGzHS%+0}ain{E5l1Pjm|o+WFf74VQ1%F?nRg*a~X7S@yp- zSFz`Ww5aGancT*y z)JD+8b`^XKMeON`mLDABiM1FPc(?*4&)Pz3%gDwzlBj|6Zn=~ECKDxi@U?Ah)m1Ix zNp$Kix~u>&JY#x@;B;}%%hz(zY|Fm!Y{TAxo-ms^u3&A7p$C5D+@5(RY_mZf0N?bI zxKhUQp%z`PgxCr<%^Ts%5@^(RWCgTFj3FdrC*_hcMSK2V@HvR~S=&xmf^FbSPmdcdDeeqK=k@vHa&a3UE2pOC3H$~?{5cM8H5~wsf3+%kk_4=B#rc8(Z4|77(?*i>|`*7Qi zT-W(EN9$3lc*0j9AHl!=JHIvdS;QY;*=w}$A3Af(AjBgm(aEC~DLmrynm#}@9JF#Q z$lza^1LNg6QWv9@hgFJ82;pE{9?%^MjWPs^c+)OlNeZ7D!Dbf!*d^AI)}94l74_s_ zLQe;wPnDUfis|yJYSD`Lu$G{U&5&vWRKbErf0fc!uTnGz?o@gdv_({&_`Ga^Qko$z z5IuJ;wmE)L2x!=v7HJ_=;*wZ}V7qbHA=jJ#)b_6H{IaGe-N~Og1Q%;?ZWbTl){tvwN`fA*&z59y$JF1rFiul={HPS>KpCIB) zRPzJfXsl`FQT<2Sc=MM4$Sh#ktKm9sJ2crC$|J5D3bg2#2S{f=KT{XS@X`Fu<_2(S ziA0Sqs8UrBqKw=OMtFty%CdNI1LkFryzfyqa9xh?EXAdP znAxCvh=LvLAi3I|?0h?oKJ3Hj*U(3Ys7#f^TSPTA)t}&^>g{!O$ZV4x8%Di@BW>bR zD%nOgkk?tuy!sNc6g^?55n&Sc)wBMD6ic>?L7yuZD6jVzuJAIB=>b5)OCzAgpt;@6cP9-+7Dpe}O(wnHh%iYlTMaI8Yxo z)wFJ}Lj1*23UogjyDSyEI4&tPs2Bd3>%(-rX5I12Ps;r0<;~WF2Z+9)=d>J5$lWie z6}=5nfQR8Q?18WGyhJ_+VG5{F9u%&JzDF2Wl?`%3YWEiurfAc{2gZkfe)Tf*Qd0Q( zIt0?BQWYWL6wI_rO}moj-nOI`IH1UMtF|dhId9||0E9*RmDfar?OUZ)pSOK(GFSZH z%u3)qQ^^GSlR2+EM^ni3J)x8926=Df%a`_1KBxH19>EO7IP>C?jK6)Ob& z_oQviJMM*StU@Y+0G+q;`5mUtuq{{~-a@O=%GY7LZ!IR(ZZrw6FB+98ma{HlC47LT zs4tLY^?cd{t}tOa>JA7gW_3ph2tCF12{!iBK~gy#2+Ly{mUj?1`?UOKyDEfP48le7 z0{MKZKs_F7Q%*J~v7Ap6vzgi?+Fq{_&|GPpy{~wpvqqK<9r{ z2^*_!+)vl}A|p-SFmx^Q^1!L7jOLamI2mYIj*;>2@fQR_Z&)?9wc|lUygz{XZWK=K zUR!`HpEVuif^<%FHJ`#Hvh(c2nwqXq{E_~qVUc|L6UJ-C%!yj@GRG8)CW!;>XSCw< z)_e}MR?cad_xqUMEx=3(KKa)y?(L|<;O|2HXtKcn^N{TtBnVdhr-A6Nu>jb5tCdm> zF+Nss5XzXO#y5Jln_%~odNPRx0J1G=^{R!CY zuP%N-+X10p`<%rZZu3_QUO9dH$~LX3j{w`m2vcz1Y>#&&?UgrrQc)2T z(pxa@4Z7EmoRA_L)Na$3hJ0AJ_QilY=sc+ExD9VhOR_iqm(mR%EmG*OSpM1Dn2a`u zHJ2x)RFujl!fVjNQ_fB}E1}v3MqYi1op-Pj>~pY3KBcYkR#&>aEk?FC{>t(R{%IF3 z&g{P)JtzGK_B=S%!HOje;S&G%HGm*hY<|ZyO@&zMx!mvheZp;KhSq5#UqFSb7X-il z(JB54N0Xj`Mwp&|qT*?mYCZ&I9^;}2&?E|Ix17(;r`tnrjWa>@uk6g#K}B=j_g2b? zyq4Z3)V0H+)1GTj*+`rx_%6GnRAcgI7zdcw(Q)K9PcmH!S-7*zNv|Z|*+xmanC#KiuUvXTTjgu&^<;Ya_5X@QEp(m5&Z`|n_{2%gZ7 zjbqg^V`4f|Pnkr!VO2AT!+WA$ggvm-UI2>lp~B%lozo3#XXn%XVPzL!aQpu?4q~YJ zWb#i!&NlafCfc7t!^G&ezG*N0YWo8XaBY*hP&!YcgAA>*K*;Y~2voiYGFMxE{t4rP z_*&toKs%&(c9r5*FjU=3Ef)$r@eQ;l8=MCJ1@s(-FdU15Zb-d`tn2(UX=8Dzk>^M0 z?s~q$=TK+YkouLN)WdtLaOQjKV6JTidpTq}V;8CtP!Ka8qLV|0xT^~{JZ6~{;v&~J z-;r?^c8fKrI~eghk8&OA9yn}nmN~ldvbmB}Dd#9kMtpy3vIz?J)eqkZ;QRTZtEsJr zn>HdJe1%q}{4&|)06b=7u#l@v*BbJw4^@UE5vTRln{AlkHFQ5l!&x|f%q_Cid&)@n zecUlG?41Zq%@YuJBjfm8I}Nb+(#vkLtqm&(s^qI?S)Xo}4mIA$Ze|ujQts6r{TsHh z+CiJelg+x;hS4>Iie;4%ofXnFeH&}^$|7LoHd#;pmYR> z^W0ttD;CN=-QY3HuDqoaFXg;{BgU7)x+eo`E6hsu7rDjC;VMtAMR?Q)NJrpuS`=aPzN4+0IQZamKttRtfC2FyC$@mPleu1DnyiknR@1QdV=@{)ca4 zIO`LpH33KXGG2o`lg|>f0|(|ue;RJLThJ9GoRkbtUV`}6P;w$xo>o2qXk|L<>AUdB%IhPfCI-A5zyw7NzO9A|rTc~>f=;M2yZ zwh$b&H8Att8T*x3#F;-dHq+%-$W%B<{$bTuPOiQN)x>o_!@JAK*gSp5Vv>sRfpQ}# z8u6x1KL|4}x}P2dm5AStL!3$K-~2?n?UBRWcy{nmy5fgBd+-~Kltb@`IKe`%KoQhh z6T+v_Zj$o^QI20}_K3?cMDifPb8UNL;0%u*WB%p=+p?%3;>$_;i^%oNMDNWT8+mhR^uYZm;_@dl^_JAI(Ia7(RO+dFzeb4*){>_we9?U z&z(#p)pTgGTx6m;jx>4hC!xC?UsqGiibx(7=75EM1C`KFCMkoG^0K4l}llknJ`BQ*YIe?g(kY|JT*GCnD8t&9#%rRZr-aC#DfzB7!X87|YS> zY+M?p^@I$rfkViouYc1MSv0_^kpzN(kv&W%X+9r^N0ZI)^t3$PslMmatv+YarM=j^ za|O7;GXQKzqFs5DSS9DatA<@b> zP`1HG?8P31+i|H+BRf8iw?57W0sJ)&#<0DuPIn=3LIqEEMVLM|=U{+sHn$E%L08|~H%Q79 z?&NPV)M=aRWv3-L^p47nR0QjY=vquppPZSj=~hv@I&&$&YPujit=f*`qbt+Xjhh|mYJ{09bFj3m!>Sorgck`Al%4}#&ZiNf%az0l0I+{gJ#Sx& z@{cEMIk;VymmEW`sj=1m+2yaOd!4q0^faIFj58J`5@gKU4ZQww)CCgSP^8J&jbcz@ zI!rfMDbWMb$7wI?2RKkZiK3Adw7uTp;np>R6JA?%Fvj5;W=Nh8lE`!>E9&d#7mc;n zD8q*j>;RtsdGJMsg>9`~5g9%t-)zrMtV=#Q;T&7?DV)#_mMjH|IGv77zF8~x-q38| zMh!?OG~iv^Be^|FZIjKa)AybU3k%E{g;npDaDw>js6JiwRM6W~lO>7JE^WM;$b@`q z365!tR9sz|OPLn-ht&aAgr)W|6;1HjA4MLsDtXHGt!1)%wa9a=)f~$YL}*XiL}BJ& z$ho+_VuY%OvMPUhM(_WRYs_(^$RMN*f~>E9|97DUu4#ALwZAz(7P)$>b1abM1HGON zl(JqchJIxAnoEW~W;{Iu^GDpf2E2||@dh5sC#d87ahAJf*g zhzdq)@}eaOwD=fuz47g_DeP{@BuhKRkj$3e%6g8RI4452A2mz|Nm zIb^N-qC>E>^GiVJwSwQ;<~dBgmtzQIaZk(a9)D=%eGkFE|I(qwWmr*U%S`$aLiboD zv%rXoHfAn?*P6neQ(N}-IxoHSbs4I1uO79@KH=h7 z(XcP>Ht;ZF#dK<#cdq7&gR98+zgh6+IGdTugU!?KJ>&JhC%dSU7yxf?HRe9O-+IjE zJHMT*(K8&#%dXsE=;jHz{^SAp^;4urqL5c^G$+dn#Q_`48b&T8DbA6KnJ-Eq-2{8` zy411h9w#Da!syW_Fn;CSAQ)U^(IK370d*`SZwLv14*CZa2Jk6y7+`ngJo3<+-VfW} zy9INE*(EYaFK8!(Wn6{+2lAo+W3%^QpQ&@P1~Gsks^?jIK3r1d7AK6H{?s> zpYzF9`VxU|m9QGAD`8l8my=egghd zGfvUussxZ5S#vKg}M1Qzm}`hlBM5K-@DbmEdhNOY~R3R zd8N-O%7zpjQ3EE0q<|2;1;v8 zLfetw%Q0AQ)8Nkk^!VW8#Z1j{%+28ADWUp&T-EW*vJaqZ+sw=3yu<|nx@yGFGSV>H z98m8(iq$Uh+?`30;D?;g(gIgvmn_GAUEJN*Pz!FDM`S7ye}MP8{fPGI3a~1I$yneg zJ#A3AziLmPLW+k(r_c#zQQIqVNe17wY3EnECb;sd#~<=tGlG@blU!%;i6iEFXJ~ph ztXUhC*&+Xhd8f`C0fqZki32xIbGyq(j$m)HlmLk-Gan0Z#`tfBvGxII2r55gSEArkh}DK%ooBlW8-$Jmn(|)ZSxYMnxr# zkW@_{Up+BAJV0XpzMdqeat5`aPA7329YvXqf&Y2Wd5=RT=z#S%&T|vrWug5cI{r`Z zM5mS$RnPV3@pEfujE0Vh7REL#M2yEM9A|wo3SL;{<RSGRwP!T~)Vt~9=#rqh<#h&p=7izn`Lnlez^ZsV~84tz>Rl zxFM!BJD5q#Pmgj&b#h=g3Epk~zxT+@6fZp{lwJR8Rwpp*^DV~dw6(gXZa&r5@syih z%BmWLx0jbYaaJLvic$7q3KhrbS2JlVEbQXvLU>4}XlP}x|BZqCXyAgqW7#uzk!6cq z+GHqmBzymmaFRj7n9`@oOCkMyfL6Mfw&UCCWJLma9PhNV^6(`IeTi!5|%$0&qhGyv*h1e`8v3wfcD7ZS8M!qI?ji9pD2Z(u!YL z*R`{EkxicgKks0dgU?09h=Q}lFIdrQ5fhtEjxX;ZAu}%<9{D5P7b+x&R?ixpjmm`) zk4VzqhuM9{@~tFc#9;6_FkkeQ;Bj>e*X0lAU}8{))q(}lQ1_=^*EwXvx^=P8^NcAI z&>>A$r}L-MQZLN}FBn=@-{P7S-cR?kpQr3z47iub-6DG(NvnS>F~%5d*O^xuq-s51 z4a%TnQ_&@JAaTM{vc6@aD(J!}G3)?GtCUGUIF4f2c5Le;Fu+i;Hr#2f+EaIwL;E9^ z$BjyMX#BGKK(t+%hFcBp#mv8XXx7t}G9;HpcL#z2 zV+Le#`j^AL{nkhi$rJmX$sl9$ZfJ8-tk&yJ_Ws-prN%P+{f2?Eht2#x%|AY*O0t@c z2)H!BCf({rJ-x|I@1<{QOo~{Y`J!y39WPbWtKHGf`Wnvo!*X~9)ROK_4B3=BV#*)1 zi_#I?UUU%${7_hR$US|n*_~t_owWNgY0ml3i^-y(cWkkIuWKo$xnd$M_so%B2G4Pj zV|nS!19C5O%lU;!t_kxLgYN7*d|rf7y#1|00HfPC?R(+FEhFyrWJwF51PB9ijB z|E~9ouf+B^|CNk=#UC{-<52KKv=>soHD&;L7o~Pm(F9V5h4x3g9Zqa{6~-b~M_a#V z7-@$C`lisvL?~8Wf9-uoq63@HGse~-H!VP1zV_y5{pq(ZF|Xl#LFdwvCnWXf4wrGf zLjNA5XIyRE^dDF%h#-5LaCa zf_E%YDI-aVmc)9h&CVu~N*8-16AKQ0jQqsGgf!9+!6`fZ9fMbg>-bW12EY;mF9K|m zn|tnQZ$I?fLhqn00}12}_=DaG@OhVPBY|<-5Mm#A;67Vu_jGa-%MT|^!`={Xr7}Z=n zU&*N4nX);}OsfF(>PHLG13r@oxYxmDG5S*FVBu7YUH~7_8IZbiN!)r$nhY+NrXkl4 zgg1{RETw^+>B01$RrtUc#-rA!;RvT|j3h(-n6s2ODAAGK{%;7Ug;;Ge(=LWjrO$fWNaM%!Kh2 z_O8VTojkZ?kDq7)nuC8}*v4Y(>BZ!gscQ9!mUS0{w7Y3WMsR&W0_4AR6q1ZgH7Z*Z zzRI zT7=?$DA6Ck9ouw>AAQz{44I2qM(PuTHiVjFx?rwD)nLxc{nfBq4S_>#w2;i@*97_4 zIcGC`GYwc890EXm=4fIOrT`xbpEo-}@0FFS;^_ru68@YSUoVA=S7#GRxGX6H!1dUD zS2r+ZmG_AL#VPbExPvmmz0D@-8=Rgq1e}JTpIG0SH?cUCJ_9t+mC_-L`i08eoxMNz z7vD5vw%sDpEAvbQi-o6%?^T!MsanEm4If!m*g(Ua{Z+Evg?Dn_bjKVnz0>yT%w5p_ zU8o@1Fzdt#?bEyU*IvEBse)XiLVYbKkeHR}JCv5%INN0J_Dx2*A*y2sYvKF&lcIRhvBtluAk_l4^$&VE zIm5ki)|C2^S5MEFeSBe3ArWT{UVPN1#iuc*8JkTfMS5ZyQOKDMrllNW77WWsUeAcD z`}YX$-S?tw9;-ZmhUd-*y@Mw{!77T5T zx%={j~Y7ru6J!BXn|A0avmOan)dNcS?QWJ1de9} zOwoP)Uq@W&L^!d+qP3Lo65U7k%7p+u2JxRN%^ndmgd-{9(j7!{(Q}>P@Hp{P#*apK z-aZ`SAl&s?M6T+Yg=+dQ+W-aG&oDVGXc49o!KHN2G)a~Ii&n*B@Vy#DYyk>Hns?m$ z)|VLaBMhu+ci@N3rPJH!64!En679*;;0rAoaP#m0|+`L7D(VP?NRr|+x&Z9Bnwe>v! zM?kp0%kt7EMQh$)Z?c;+qxKNCg-ryG+l3xfei@VqdTGO`PRpmjjNxT&{&A#GPmH6S zLlZB0bTROYF$ci9ro>Y{%j_NHKH8|U7J^kAvsDGc=6Np+fyaK3_cm}{A-Xwov?nhP%F;%+ggW=`M<&D_k4R)i zb-YB?h0pW|Cz7gT-{e!(0UC1cY_wfb44hGp-gtkEC4R8o$fJ!%$+Zxk#7dR3H~YcUXadFV>hU}CGLrK+$sGc-aaZ|a`86VF4KP|K}#rT z&#er|AC;#FFG6FP;RguxBr8^sM6Dvx<(&Ssv9b*i8}-5s9aQB16E^ERfq*6jN-VwB z2pd*irFv3WRUoGhr1+`q3q2%nrS9XiSpT3GIE8yB=4l#bKYg~=wP~oiyLJohBWF~C z>#w7dNL4;qcXT$FHD?QLqydinBI~$7%E+8vLvvZ|ZO(jQNX5@E-IX=?-4Yvq3LecBI z=!UDOOA%N22R^Fu^bV_u7ROqz**O7(qx^{QN@RA3KXYX!X$o-&s_H{E`)Zn#~XU!0IasJ4YM1 z{id);{OAvFOd{i3-+t&0&t4=wVIJ z$fcnvC~rmJz7WjSD|XseT-l1K9ogv=A}$qlJhFUp4qKPsW(=utc3u5b{9huixV5#e z(%$-W|81+CzD#d{XVvT1OIK=1ve*v#fvAE0M#4dGoZG_%NLG;vD27`IE5X-$jfGY5 z(X1aT=?SlVbdz>pLYf%5+-W=SWpkFFj^Yc+^#SOz4lq|QHG|b^thkf6vIWhFAJ3x9 zK+H{an+$yWBc-iHjN|@I%$X4EO`5dEY4B-(H(WCR$!tf?SR~!YDQ)nhD$|H5Xr&FXeMc={FHkFEbFPFYA59P?uNi}T`Yvtheg!K<7ju$& zafQ?jhBof#LTrgG!V^LLP4Lj1=dFbSklv^mUudk|Ja7-SEzMuiC%GoMiPaXe1BYMM1nlx?}Y8*;N z5**K9GB_z_Yo6J0RBc~oLp|!7$c+C4O_w%+o#@Y@No>M#gPhuJ0xY!BX3*L?0c^_d z^Q2ZNa*z94lTg^~d9?_$;W5@sr@=Hdg&C}t#u zu4H=7#pzgs(QGB9lF;6Q&YYdcx07ypS#HcHp1U!UaOB~jDW0l>FblUt=7l$P8W_L# z39XK0EI;*4(SVoL;WZPyW*g1mAk1mVzu>z!3TK<`zV9f3+*8iwrY3`Y=X4BXnvS^A z-~={EBHRV*C9@y3WgY1+{3LkN z`=tkWkSG%vQfD+tgT&y~pY~gIc&fJ3-4mu}*iclQ!FYdxq=b*0K=ejA;{1>(86BUw zM9#8kL2LndSwqM0BjCCj4&xw*$WpPdZ~Eg!79^ZtK7xe7m0fceQ5^Ul%I(MTcz|-+-lF& zvg0-0VizV+dx8WKFvLtk6avN*p(4#`m3$-`)-TS=zEaV138h)o7N=toT7zuQ?R^;S zRb;~FiOxyjHuWd942jG+g^(DbDm>V$35hV%JGUAu6w~a4tM#*4zlCbXq`vZK3Nkz& zRHSV<#f`0o5yZE!PV?%-mwcBg0eBqxxqij#zWHBx#3INf1x=zA)Eq`C?2i46MJMo1 z8Q%L9;Ts(pQ~$K|)@DfAbogV*&>FH8P{%dWNvuZJ$+f1bdJ}^Se8)mKS|Vut0MH}{ z>ESQP&tngjA4J9*u+buqM2_qk_c(pJv4419`ZJr~SOFQK{oIrZN^M^`vwXxRxKi$1 zHBV~_lNyN_0I0L~k z7?Er9CIP>R<5Ugq2J%W^)JSl-eCc=gh-`Q+fMl%US~t`j!9LaD&znXtA_hkdG@Jaa zqqN0WyF=%!MekXvv;h8y@$Fe7@1!Z4XoIx=E+jx=!a7m*({&|s+olZ2HvseEONN+& z&iu(OHc0{h5}|}~=}H`eb>)?8_*|5zv7{d_N&*WsrzJ&TgENm}i)`mB^X({;^*>1L z8jC3k>F8%6nr(A9v#Oj4Cy8eL(g|Ea9a^>Aw3Z~4LrUTr`}LE-_NZ@3P{#;g*agYL z(dx=qt}_Ra_t)u^-F)y|rpY8RS+%5`HHmZgMSeE735WIEdVb4iFd!coD7VM|OGJV* zfULSTJ1_^Keml|ROn_F`gTz^*5Jo^>>32#zG?SBOdBq)Il@ucXD{VV^(?=q7ysx`f!cTOP= zr1}(dZHY35@#O*RXLxo#fuR@i_^rv67j#s3+HA|JQJpkx8&bFFxX0t3T5I*+mY_1p zgpbUWV$*yyRy`IYafH^p5GCR>a(y5~T@>F^PBBC9tvv(zJn+SLpf}q)PVpy4S3a zB~U&~hL$PYOz7OCUq~N`aOR>~YD`nMD~qx@olD7U?58o>AiUOQcOe~+5q5^`warRn z5O8xSV{&Ml!YCnp?j%iU*q@=(>vF(5-HqEENG@Sg*kl@1*Wt9Q#Ky#|mG1yXYznIF z(l!;4M|VSpKq+Am+CMbtC9hQS)+POo+4-L}h>J#iuSN%_&JJ_SF?yGQO3%W8YYC-# z!&qc9Xqc)roxP^>oAPQ@*u^uO{$grgI}^7A!{$&(Xl{L30k#CBzIj#OmeNe|+U{J) zTH_G0#k}MF#l!^pEJ2Bi?)jAu>$ZaKS(Gf@s=}3JSA;J9{3p|{gi3dI)Mx`DH_6yL ziaz*ASG^7Ml_ZUps(GcGZb`17mRVE6y!p53zpn?A(tBHN+`&3MG9qyQELl5tJ_?+e z>kYcgdKpWfj^`NfHf$*n-W(x!+Qfo#mgs)kHh86Q(igT1R=1onoXzl%{#E4n%l!fK zG;@BtJheaUqNbvtaa}z;@*}`?n%U)o^PS zv1j`MJCvM`9S5)$5A z{UAfK?V{w*Fi*p{WUE%AF|n&AU7Xp>@OOWd9hkp2YK3NQ*NbxrZRZ&|k0X}Difrd* zvoNT>f~3Hiv`!2E>bSRMNc&5?gUya_ck`mREE|{Kin#Qob|JJzq#wTZePOxM8w1S< z)iR5~7`9V=)N33gc_M0T^d!V&*^=Gf{{wUKtF>`>C~@f>Ru%X%rbv&`Rlw8rkMxC4 zXeD(g6e=Ra3^r$=LD{BX&HuPoT5*wibSehMm)DGe-tEYI?;6a*x2H_3rBTz73KSwZ^eVi7Z;5yyS`L55T zy473*NTg*8MAuy2Wb;RAmJ{5;SA#3)D|-hjeAVg2w6P36Hva`O>r8s@sWr+?`VjNR zXn*-qv$xw=&jJfjJ9iA(mLEJJNG|FX7~t-(SF`^)fb!tYB+0qt`=dr$$u+&2s|}|D6K8+?jKy)O621JA%izAxn(xblwZZtgpgmVm)j;9DYxO%j+s91$&cUvW{SvnSJX=9xn7L zI;FV8nWpPDwUOX)9FiZ^yCz$Ep5n}areQnn^^SUe93Q$vfbZqnjTIOdg6bCe@Sj;L zigy8P`~-p5BdKWfPiflUMEYA2e`fdFnQ>2fDsSZWk8Qvm6Zq+^-n9sP&a|spno$Bi z>?*Xp5ck9yHl#YL6wyvg>@GL5hI55F(PW5+$zEBwD57uRJiY1@cMv-j5 z@PGdcgF5DrubKldG+>l-dh$4mUwJ=_6azqY7@vJ){l#%y*mC6x7s?B&) z5M)rUL~);VI52wstkgq4mJAo~)7@A}ffQb-+j5?Q<-Mm+ho*0f+Gz(#0xnlFUr0NwcS_Er^G8r)O+2w{D-Y(8cc>M-a=h2Dsr2bu+YnuCZ55am5)w{R8cL zi`>9Y8N4x1Gy~nWAY+WDQf#J^{kG8KEXa#N6JN6dHR1R|KBOhxRx#yVxUNlp@~zXg z-tD`2gB`eMSVJyuR0|tm-KFc4>mZPw%MgNL5!pA-@sV6HYgtSU1PcxisVK64iT_1{ zE%3S`Vfp~(d8{f|^)`+dh=L?m!#)u)9a6dY@$VHLhDe?j7u4Vq)S)W4G4WFaV)$_g zl+i!hs15d+osB2l7N<&(9L1>jF-18XwVn0ouwb>Vy)^og$&aGF_|%*f`Xunv(5WN$ zF!fwkn~dU)d;&=DSG7QVH-*z+Mrh4u!vWwYIvC<>?HC&=10iGB zIG2ZlNU6O`$A_kwvV^8c%)8MfH#>$B5pS>f>5MG2(L-spuZ#)1qeS4ZjMQKdBkdb*{DGj~uKZA;kz+bxg9uHbx!X z5%^a*nHp#&WkdM0t%01`#%mwcfPGuTiVuVue>n6Jm>P3{Xce{798i2`i00o1O3I}m zq0V3M*dy_t0|(Y|4!a*lMLNwD;&3aGKr&Mu@X|7yu?iRn|CYC^#8*6qde|~2n#8xS zob4)@4P!B^UbCO0AU<5*jE*zg=*?z=Vex=A{cGtrlB`$uNu&nRyV(9t2*XX3>>OJ3D{1HHWV=3&To2F(B@~KSC`63 zj39jWmo=h+dOiN)Y6*SIt6|?*}-lH_u6L1Z@i6sx z=PKQcom(J!*);^zR%Zod@VIUcA^5h7is;qq*`Mq!@eT)Shs-+#m4r>>OhBhNQ>PMM z+o8DONh{K=rN0zWARp{DkAk*&>$qmJ?O->f$c+!Nc~=USo7HrP3xTxVoDa2 zb84*Hdy>Wj51ya*jlxB#;n(~gW@sq!NTdJI==6NxIE6Wc!qjUShMFjmpcp1|v2kW# zZh7xA#*mQZMown?_+}o|Vt}JgCvycLN_1?L6DO(UXM4~f4%}<9BN92M1q$o&lP$hM z(KbI*E#1Bn7mtgktoh+^Zes{~{eF?@vF7IC2Z`|%_p=kG%q`F-!!~I9!n|mDf(g)CEsz(%gJk4Z z@SS_cR%3HxAiG+nQoK>cn81^(OXgp(`AI7{M~(kYts%mm*7SWt752(`31vmQWZV;9 z6bBy}*@?yE2@cdhtEjdosXj4xE`(L&iWhKf$Mvo1|Q3 zn7Qw>S=xLTx(-G-S?O=y7&ychHvpf zA_dgAzo!%4*mVVVfuVYz7Ebpz9PJ79&ptX9k)%L8AJ*z>s{H8R{FRIqZ=}r=ZX21S z(5^2u6(mS`a9L|+-dno!-L`77yyj9JJ>3X~=b?*shGhrpD%&tg`WoEl6jxW-7l_68 zeZR*aZ0YXV&8$ajH|&h%`*)%YD#QZk6XZV;QNDb{?qnC;IoCdW-h3Vez~Sj!S#--# z60#*J6NOea90(f##FhViRJ4eIy-e<7iVU8b_cDLm%SR+@7}R&On;~%1s)Fg`lcl1< zELGabK+k;zcna|v&9;%+U1Tg|UXad-V7i*fwr zp$7>4edp-8J#eR)L=Rqn^EQrQ4wG_ezI%WpxOaRjQ2!^(1thrTVvskz7X#4Q7=D{6?Ij;=K*Z7omMzG})k}Cma;(CcDU3J!^2#WT++WV0S z9OOPg=oh~`J~TdaWj&i`4VcH$1ry=YGGYZYZ+gJ-n2&tz^hK*{pPB7h8wVp8W1hui3C>1P7ID+Iig@si;pSj0E7G_{5BXhAH4 zy^_(O9Vhm^jGTy3B0mhFlUgu?=vyfJ! zbYD?DtmS#yPtQtIUm-AtIy<;%ZgqvwE{k+rvV7C!SaLD2#KO$Q8iX}xX;7U#Ysr0m zy;K+d5adUp)!muXlx`_yyrqx4(6Y7mdnXJ~XRTg}U5cRJ<|RfUNAR4L-2VMhGRmJQ z3q#3kKir$0&+u>K-VW+m%ZK4FKMRmrAg^Za*7T`gi zwq9WFB~6XG%@sLXax@XOxZ%J=+(2iG=p?Yq3{M^IQ+f!Aa^u7wT%eDr@;ZyJ%a!ws zi&nVap~-O`GCli%psrn`f&2I@X7aY^YH5&^=&w<289IPbZmhn6fRG`Ee@oKiJVtNd zo&9`r=uRkd%4KMb@AN~reT&F4&K=R`n~;+?=DyF54r@9JGm`A++AJRd5VnWG=H#br^*NE)vlhSIyBo@Bv-Gk2jHd-^D{Ei zY4Hu>(ox*3P>Tls?3oTGGiGdfPz{Snn!zB z*J3z4?aGp^_OH{z7^G7CyT+sazSbj0eL*`MsoJJ01# z!^rb}xif1s{Rn8c<6n@yYQaUSfoVgJVtdj(`deHjO`F}rzg(7ahl&RLzwIN=n*v*O zB}a|7z1jWF69-{*Qom*rU-uSqwwvS_B4-2w;~8%~pWj2@FUy7I(8n9^GQ@ar!6)0z zx9xwooEw(ma0MyCTrb=nhSKv!N{Sd~MV}z3fYE* zKX$i{cWx;vR3tP%HF|?WgNAX)F~3%8PmUl2V0 z6k*z7r2RMf*J>tAQRWjo=r~!0DX3%Pa!ci%o?YuMx%$i?4m-KNB^u@>U%UT3JqgSj zLtbTnVa{YZJ7=^NBu5MXplv$83O1L@DcNICUy*g zQ}4D)jqexw|2yQ9CH+UQG6v)eKmWB{qaEgs261Rq5B-X;ZW&~Ej|wU9?Xwz#hr>t9 zW*O+pPy0?7&feFm4iQO5vs3)s`7*XlJo1=kXMi8M)*33r3K)c99jF$uZ!!Vt`1S4U zMBw9>pG+t6$)C^>AcX-!BW{k-7hk4UJZLL5DjudTl&W4=yphr;EmHruA;){3Z%y#0 zzi4t_NEf85KgVcTt?uA^+rovPHIZ@uLU9Gb{F}C_9gW?$sRcs*X&+zz) zYQR%9w46AJ1b|=mm+jj*9hnHlH?ScF#Ej>bHRe(rsI&$QvdD!~1AXYQ+uu#tt*=Xy zX8WDQatHTWaN%)qYqcFZggOU0knxH;14&5(@E|GE<1L7NK#9+?3O{3!QIC{wc(Jf@ z$WtbGFx)6D1v-%1Ntl9&yV4nI3|$Gh%YyfY41buINTGO)Jg=$4;L`&v(GP8_Kll!9bY~^0X5s zu^xQ{xrB^6N&B*w;*R0R>*;yW5;ETsUhrb$GI z9TBC?(Y`bP^aeZzw01oWny$L zLX@_FycV{M?mS$z0B1-+c%qy1fiekk5C^4XoNbk@fqsf(i}GQW!`*;c-&_?`dL(bs zmP*^Q0MUm>PEL27(l>?&E^eM0YTyc9Ej~6JRHM)7K1h>vjk$Dxvpj@$M`^ayk!8xd zj0#2sn9fqPRgZQP{-|d4FSVPK05;J1ClXyz!9)^fm~HJ*?AmEkNanqh1`U zJx9#dAtV_atW3zAn3)S)xOHxvEl8-a;v5+|&|#TlFnNP*Wym|)k8f@TQwEvJr!&!H zK^%sJK)z~pb}~SV1LOGX8r8J7T_p-45i+y|Zk|B`EESi@>MsmqwnzVV-r651cZN|= z3oDS%y10WfJ22ZfwN6Woyb(}!E%b@Zo6!l~yL3IYP@~(5$!m0FCm9^IjIT~M>~gIT zmSx8iAPm&_AyEV{o0og)x0x+%%{%+%bj^6p{`=2{o?7G)_9Sp)^c<#ml_F3*a>#4Y zh_Z}DRA@KxhJ`OQp(fk8p5X3F&;LI2I{OT>&BWl>$h4ho!MF)+-`U6qLK?b2ZN^d@ z_m$Ld+GJ8EpB4`LVuO9lGA_t9^J6tYPo<~Y zt2itIt%&fQ3e!KkX0XrYPBrlF+G&wOY~v&~Quv&J9lliovrlgm5~sT#0k0-Vp`mL~ zwD9n5KBumCF^iJfP9&;c@@l{0OSU&c5>E(?2mS2hnZAx_*j6oR5$0Cnb?6s2#2eR zyI;uu3tPIS?hQPenNrX~&K9BOLM^cA{&knGwp>)908+4W>a?RVeQ!{_GMl}Ncrc0O zmnyO3H($KSakg_x1`Gge(5GK^BEPj0a8qg>@LPacnA?Cto#Lfu;~Z;uZ;?|^CidMk zANEgi1JQ+`jw1#_Yr-HbIX(yBYVEa)gJRF z{`jU^Hjl{ko=K`oA9An{YbyKW)u*#)9fguQeF=!C6G?CBihm9UypuW_bH46^8r&M9 zW%Rp3{SE`~q;CtsyO*$s$E12B+II~0GRBRIK2jsXYQp<(&v`{O5&!n2_D}+tJY^+V zxuP1pehJ;OIy0a*laq)CINP8%abEQt6BaCae@I9sVe-yDHWSd=Dej@qD#YBZhhm8D)7c*;V4QG0UCbqm}9zrwmwcy?+$T+61Y%~zF3ux!o)6Ue@ zmq2ge3!@TsQWTa6nx%>5{b$(ofWgpGPbG6MtaLmxqu67Ryyq0tmBps*?U{H+OGf7| z)72gdJVNgr6L!pL>UMr17pcwXAI{7t7C(uKYXR62Yfs0Ygx=P?VNMkD;ul!@Ozt5F zWI;6_k{QMewCaNupNDcC_R5fSvgc~|^Vy1-_g4|5P#G8=emdId=Hb_=U%4iP>x4I! z5GpQIZtdg_dmeSH#Y~;Iy%9qFa%B0F#!ueJ($jK`Wbj1a^~-5{;grAhq711$;*==y z_$5;s3)UBTY=1)vUCYUqh6LxE1zgfp z?6=V0Pz$k~A$&<{OD530mTXjv6g3I_D)ifV`bVi)*VfEQPK~s{fK|PSEU=INvUAi2 z=`~Wv^NJXv#Tu(mz;0KUyrT~TFMWOs=tjxt09G(?%*mxdv=gw%sbGR_T&TNnPcUc$ z5!0Zgp$!SG!R{3#aOiMxN9tjO{x!P4I2q}|ngMxYUqSpD^Eh^-4-UG#yIjULNGwhR z`wkWYDs{@PJ^-xA}2;$e`dy7gA^ z4p|I{aVyqXqkQ!T_XXK= zRTYk`v7ZkJF{P)QgpLS&e-xVShE>xCHp57-t41hbc@EDEGW{TP|Iv%FmO1u5OH z0OP_^&1XVG)}wy%Ut3@mZPAvCYj8zNNf5Jp_$57bj*Io@b^!B-TjaIrT0I3_)<~4m z1PUS3xs1#nX;S;zW7NlMpGd=J3b9!}8y;k3jCg4mIJj)REx1#PB42GI=GJC#Zc_+p zVH~Oh37D>C4cc+ZtS_3xYpAmUdgqcbz#xP>R<&<^0CNQbhYIw+p|U|~D*spp2xAKr zHmf8wV?%(4nc`7bQN2?FIzF64cFkMIU5AfXBb` z0=TAPOF8GpszIEm5bsy5gKAqu4D_aM>vvKNWiw-lYw4dc%VS?SAI1+0C(*qkgbXD? z!b0f@@l(-fU9BO&2n~{8rGZUyMpvIAnKHUtlDPl3GB82jp;3%cOxZ6ycmA6y{q&JB zIpWRBp{>@_7Cs3>29|bM^La%WTw*rTv#69Tj(tO1z_}u6$n)9QBjidwqL6Sd!7(iN zy;W-0^Yz0@k@;$C{0Jp9z3W^X0C7?c*9}D7&_GAH{bB)lLHLez`})S;4*uJyCl4_9 z1>fK5$xgcdUsVQum-;21gKxzWG>n_m148(B+wCPW<{OLc#hv5!K_tIhif3JF=57zL znd;D2&u&G!Lu}v|SP!n;s!LlBCq7pWp~Vo5E?rVOpb3yWT(oS`NOYavz|F9>CVVsDcA1=% z6mmwB>*EII<+5?wVwk1l;FD*uj2YRicbZhJK)`>So6~hJ|4B%lftr$XUX@NhRI7 zpJW^aZalA*`wHU%jFM`zxQL1XtyWW;ot_=f@ug==&e+(cM=n&x%(mZp=pP1iL4jwO z({F;zT2X*Ju~_E-gHT98M>76ub&9UGxd_V-;Ze^@jX2M^*12J6U{y*q4E>jCxOit8 zdB7I3A`l=CLEevL#b(LQ z9bhMZeZ11B=L;kM;p5D|wf{2!YyXrpugNlj3B{1uXm}nr>#)SCsueBh)83R(eQT~G zF1=7i)R|N2jxpLlfQw_1`oD?UEul)G&?KQ=M7d`+nYghZYE8sMX?g6GH(}^kN)+Fb zM69iIAl6@z0XV2xEpq{nGp3+AC7oC7oYmIJd|uuL5P<0fu_A{K811Kdi|Gy-$C$( z`WJW-2re^=+6=*lIrRc z$c$nF^84gTLgLx-xA=tPoBjQiYm7o;RABXGr3ti=Rix!6nq0 zAcBx*mR{tP_SGz7J=m{J8aIO12kgDC}=|>T0FcQc3^7U-iSNo5XrW zuJ3=RM4dO7d@YD6irk)1`@I0rWaMw=d0F|!;Jsn`6=6zH8l(R2ak#TqR2stYZQ?yG zzf(tpG5^~khMgv9NzZ=fkmes1QfU`@U)w5w1Xxi6jViX$6-#;U=2YaW?9K%6tZz8* zeCn47QK$UT68~a5>;!rn-+@6D>_=GpO5g|}a@X5Ijuc1HR3?-nbDu#g_e90%(=eZA zNdZ*IBf_Fm0{cy46z-{A^J#eW@J|O{%sP{!jfd`Z(O%Y1jp{1ag_^tR{(!o zE;A;PZN1Zuw88rUB@EqmWrzjLKNNONBAFy^@>&Xe zQ`1mtK$P!_df@P6%u62O`jASYI+=1r)KLt&4B`fNEoU+nFDV1`f|G(F`a4%rgWFp> zskVD+7$c{oz-8J9Uk2-Asv?cWh1L}tY*`lD?lW18d1is^GD@DIWrUJXR$e70T8o_l4lsC@4$F)*s znFL%<{Z%)6I4e`NSY@GZ_ipK;iI~RGNRRDdIy`rrygD+QKsseBk=Nny;RR;mdKHIC zWDwLv7UsOmw|JV5-R%d&sze^oPnKJw*wNjAnK=mko-;m20%&=T-o2D;h0#TY+Db55 z%eaek>O`)B4&jI2<28z6yAZ3`;owms9K~U2mYfpaXb+Krq(*Wd@{uK9?nJjPS8-xqKhsA% zXub0y$zp9yR``v}OQs-U-v?cLz{Z+7gO~GuSd}ZOPn$ZcG@&Tr1urGB2Z+Hz9Gkj; ziO_!h-KjFg-o}lr31}o$@0^0a59m%?Pk0*;0~YbNyG5D69)Qw8zMZgFs5Et{FCeA+ z6qb9zU#{4HHH0vDW~K|JO^k-qeD`vdu&UQB{d?~W|9N~Y2#%HKiDonJj-hwILlQX(`ul(?GTe1DH5cp%Q4#a_kZ_aLDlvS zH4()5@F$U$lS!s%$zUDwT!%sB3|($zGs4rMPWa&QU@ZU!S)gxSO}seM(^Q$au(7`v&N)=(tCX5l<8}RFI#PEA{PiS^*~^PN@9r z^Uk-Nz|c{U$nI?^SpMjSbPWMPh=H;*Uql-hgOdjVOsz42;~~$R=!ZGU1>)a~kBlzK zQb9lFJ=Co*sbZ9JTJV6$L$T{(Xz2z!aME>iGwTOwy@n$Cy}rm+Mk`~)tkM(5LIT#WdP3S4Ms-}tw`DLf z-lG@4n3yB@AOnL;KbE)i;{!5~pn5EX`+73x3d$m{wh4BYk}%;o@mO_rV*9vr!Km!_ zG3diQCl)jIbVL{AGy=rK%87ZW;DMk-p%?w-YAFu4efyW!oN*sWFe^tZQjBl@m?~oq zgu?K}7SYU!Vtkk_uZR%!&*m*m+RQ5|`S5x6{+y;Azk|AAsuN8b-eqX zM|Zuj0E_qf+7E-$S5HGyoFt6*t8EFGMRgxbv%vU|lyARtOcPE;J@BqH#eS@}^ik3H z)u#>RTDi{FxDGnpu#&W)V<_FTX?qoQVm(2{PRGRJ$HZT}_n$Jp+MW7f>MB};hs#BD zt@Ri2dT$o6^cRD!=5*}N_v6$1{Y!`7tQNKXqw|5r8DNAxupkzET<#JJ0mWl0-Lz=B z9>r*8GR*_5TgwYh_jzI}5M1bLuxY&)rqlf$d1;h#?2891XfmMLQi|5USf<870678< zBoOM&O%x5WA;Ipt+_;JbbPixz-OP!*A+IwN@u90a;OZsc&qQ%MZlSF#yqK$Ted2y) zZpm+YOp6Ga)Z0llpup7ytixFGkKfBeh1Ttdu(P&5&}8t2_8bQybOQvEOX>xF3$5WkmKPPK66?+W9^2~t2y>9u7;AbQCXlOW?C&PGNM z$*2c8_D{N zvWzD`>ma#Mea2PoaRqjcEt<>m3AdF93DHn|X+sSe2hRsW;eQgufqnuzX2t0`t-1KA zrMthZQ3$bG^;VTuNY+QE_j{=;Tq_sMBs2QFefQuMohOENm$yZu(*uCxfHZPisLfxG zpA(M#{afz6#8~C?cb`Aw^tx=)DOXw%s(Zc2vmOVHB>84rNJQ)@7 zEGE)pC2*TRV$ggsF-msXGJ~HPj-7Wf3mdT(20;hd*siX4pQ{&njvf0bzMOitGumzw zV>K}@Y^ZUgX>bbxx8_7=q7T_bL*ld6X9yW+NQ?wooz3Q-I2Ci1D6jyzPvFWjF9s5z zg3y_hyvPTE;u7CMO2z0$b3gC47g4$t3okZ->I)_!Aj}vV9wzFLHr62{pP7;&ptcd? z6w_qy^`TTdOddejx}v;qgP8#tOZ>l&lQOMFK);4J_R~V4{5f#8CcIqhdDJT3V1dhp0f6yspi>mAed*uaTgpLfTk+< z!T!Nm$z2{`H?}EkxdEQu96UUpu;9-tdlJa+4fPT#0&x&Em$P|Dv24w3Aup)-1o)Eo z0%WwQ1W27wPzVpfUHIMYst96B+z^D4+!7EuMw}w{wz(&F9>k}`LP$iaOHQ6B>PM;R z*`b>Ryxz!zf8_Z@j*A~*{-JmJC-fNgYC$gj!@5opaQgqndufSX>BLw56ZWdHtW(ha zMOs+u4V`Amdk9*Wer_fX*=5QPxsF$WxJ9oP>WL|SPFlyxEg?8`xHj)kIm*X4 zYG(q}l*CYmNPH~!W?HKzPDVP;CbVo$LHKejmYDhY$XqiTk%Tdf+<<87Z^J zVGWWSttBL&wzyh=hF}1V+7iVo{lN9SYJ{Y0*Ek`p*iq`QL)1((MRjD$K#+aCkM1c1 zmYkb`-lAsiMAxU@CG7cByCD~6LZ<#N>O?~k(c2kMi@Xi}HbpQ5k$>zds!&S-O#ueG z{v!Of;DY8Jsbkq-C&z{Uz^h2#Eqrh;6S}Po|E@)*I*nl7gV?X_i^A~*AKsL`-Yzyg zK|Wa%onp|9uWoI2Z*yJ>(fN$_5hJO;R&{y7hwu5M3BmzZA6nNCVRNtvOg9{5t`NcIgIX+o@>ytvKZ3_m_2bOBWC-^|YD zfe{FmUW5(3)m4*eAbE<+sCBc$Z4Vp~+~86DYrK{HI#P&eCTb8$ZJ+_m)+-cEm$SL8 ziPFD$6gh_Jl>6HYruJYKDWcdZEd(4@*1T$e=>NkC>zxA=`y++yDCQfWD8r!)13{y1 zj*e|#PUbvA@%g5@R34bdfK>w-X>XnM!bO5FCd{b z)gSVN*z=uEfiWH(G>xk{8iu4CN~#%K?B1P#W|J~aelPb~hfr-eK|lGTPu6GMoO8cG zSRrs`1wu)*c;T^gzGXbaQ(?+Dd}Yneb{Toxno;zZIUjD#DSv#-GC4ec`<&i5g+Fd2^hNHaK16aL`ED_P~_5(F4F|OEfk**(VztwBUXl0*-_2%h^b7vVn-oC7xwLA}V0H zf{}2aE$j!bESt?piwts1zRA`IZv(vLY(T#dua~LlHWL33J|QhagOnt}yzA2@d)Asp z$wZ8~N4VMtp4S}Q|6|7kk9a=FBT4eyy4`std7?)}PYEuiAZx|u^}9Bej#^3GY9st*INH2l=oy|EnNJ8h!G(R`pfB>YdIn*udZ zvZ>Qt`KyyEeFnUF<7kb_)ae-fJ}N=Gfb3W`2xzHAmO!}!*-r(go3`+oq`sOoxI9;j zARPR(Fy*tVJCI3I%er{!hky>>%uX33HA2iweI*xhV!8EPib6Ib0EJDC-( zUe%k9w|LS6o<_Hn2bW?F*9C?bfK_V^vv72x^D9dAtoOx+Vt*f=oi>NnRtD5BPd%?T!d8FvBZ z`bERLZr=V9P8$OcxN8`q2g1-*-df$Zxp(cOl{`xKUU zK;9)OC@LgM=llN;{^e0&kbmYmKS>7RRbPLi&bRk`!Qs@Q+$}cZFA{As4D(&neKV~0 z1>f72!#Q?43>@3H1FZED38~bcTyRV3ojLa7s#9)WN34q{{Cp0&`d>GYlp*5z77FdB zrAC*(uGZBW^gplpMI@5xvOHqM1FBZfvs}Byg42FhG#MB%WUPtgixKiy>`M@#xm}4N zCK@57-A8%HdAx4YcM`qEWdAT zDn8AI!3KHxOZpZ?rvYi5Crgh@TSGTjE3{Wm47P^jwkfjxbtkOenf;Szv?(TW&+13) z?%lG(%f$0#=~-o5LWBF!joNSU3g!;Lf8SQwGbNIY>7vNJPc`!%52B8nw?$FADv0vG zy+tYgK(}2O*ThPJYMNk#z;B&0#9ibY?NWQX$#g8uHWAjMkO#e2`{nnh zraZ1WLlBDVFDlI+oHBIk$EnOKm-cpmFiesQ8>LKJDoTN}fGIogVO#ZhFHb>=!Af*E z_=XBSkgnDi*xjlzFo66x!qR<~-aWlNVrmTcox2jW0MZX^{FrE#MViKN!5;FC zfzc4xE+7nQiw2&4;zVm29Q)iT9^6-~)?~{%h;aYL!i{n@hczPH#V-=pdHHwZ83WMu zBgc&G-6c-wehP@qW3aENkzlWMHO8ktKSF=r0)*WB3{s&QvoCqjV$KIxibGBtj((aD z6L z`{qR8)y6mJx1ZZI&Unh`bkGYSKEXDNPpDp_G^Azm9rhALP50m!G{ZKCQLp&%T4fmy z6HV{VLRT3&R13;rRU@%%h~;57!!Yfb{vS(XW__8E5WnMn-KVi zaq6@fNAX81gHrEe_znLJNJb(#hJP|0&zXA%RKbhc?`@iE(vgWY|OLEW|o#nu@wN0W_+IZf*)48A6=euTKI`_n$P z7f-GMD;okQ{DOu^&xbBXh=(m+A?%jmzMJAg(wgco>nQvoxL;eCUD7HZ+?V0fiDJ4i zkho^GRmQPFPqjh)<6&*7xh||YxAa4lHp*v(e?X)I_$^|XhTQbxk2YdhE6!WFR>uI$ zyMU%`VFo$CUQ$9Ka^@};dD4=ohGQ!eL%(A(FY-U#OG2U0oIG>0)yrH#M1ra=S?Y9D zjUXz^ktMlu@4$bl zPIcct>RZ8|4AYJ@q|KLNTGT>(&NZtz4G^pTg71^mpG6xHf^BOBM#IJLbK%+%+WV(P zRd@n)+~x<>KpbGQy;%BYaO?gZ5-Nhdg;x`q0VQc<>RYya;b}zoy`>%J+^-T^ROVU_0A0_ zqDKK_&%veJRv6KPUOg^a>Jh~x+HGsi1QgO#=y6659D!3-X{ooSky{_n-aulO+zHV2 z{2bCE8`aw(kU{nZDuX03)MnKVjBg=6WL>FwQIkXj7wlgc%pn??R5>#SvnHp30Nd;V z*t?skPsXrG0lBu3Ls|yuucPxb)`lm^?Guh_u1dlPQ_;^2jEgX8dgIyI*3)MvOfXnC zVz$1F^Tvxxho1em%o;yU*+}vsx?6LplCba5Y~|1@CxkMv1)R1wbS1(_X#z*^=j`mF z%Symjia|c7<8Trf3?(<|1D#H@Y^fn-NfU>B_mH1|`h$^Hl&uGsj7+NM@}tSYCc4Se z&IffFK$iXqksSW5>8WVwn!7npF+D(pIgdB~@*r1-wJ4@Uy5Ir%GmJx0HvT0M>I?GG zo#w=zuDyT1zMc(q9-QafLfk#QZFXnV?d&QDVmXk7>3GBv_GNG{0|By=q!F_DPx9E* z>oUv4s;ZzPKLtST6}PQA>SKPQucUtQtZddF!5Ul=R^0fMH>5;kCN^9=NIH_r+@KJh zKfh~hE%UwEfy%`ovy}?9NB<}eVsSI@&H+0_;> zPL(%=Q!qxy#!M~jW!F6!c(RE+&CDj>xQ#zZTAAb9m^v}jeOIa2a&^-StCVsGWfq+! zd`Obm{DGStZq5pw>4^Av}AP2b=KKNXeTJ*UQaE0_U7rFuUaWKkjFJs-j&5I6W%mGHge?yA{m(^x`>e!#@1f{rHa+*|y zM-ttrnBh_dgnXJs4xW&(OVITE&8LlZQ?hNSG~t)Y>VvyO=6+iYQ`ZYXmQsrxw1=;k zpZVkN`yhz?H2oS0Nsu(zdg@Eqy&B zoch9ROqD;4Wb@Ol;i!Yd+(@HjPihP-^xNAo0B;w)DqbOMeWL;8O}u29OCNj?N?Fa8 zB-09ir~BWCc4Dk}`;L8+XUd`_aOd=>(b2N#{d|&Cb4S{<^)*)@*`o%Ah zl~?nL(+>u0DrO7x`{>Qj=HU!9##a0Sj=Hq1bCq|Mn2&p3$tA2-*c5Ox^vX?A ztlilgMD>{jUxhsjWP+jn)v1KfIiQuQvQ@C>1Yj874GKTwDMM>NLbY@*2oEc&3c22X zAyw+jgEc?6Sa4k!yiM+f#FA-W9y{Y#09zR&)b*)`x^E zRfYCenk^u6quF)wMF;uk(-7Ez&|fhxJm?^md)?_U5w$n)3gY>Az)TRF9>*@tIFkNy z#Arik5AWwkV(qo@(;@zCIjJ)(v%;d9O7xudMnQfL|7cOoY1%Y0OAp#vKfaz#r#)Yzew*pj10;4i3*kqS|fI>BdbCyRck%H z-}Yf36IdkjJ{Dzu$=aJv*Piq+!eEnkh)L>n9^)PCrlLi*KjqFHSQI}0Cgl_ilaQSn2I{Nu-rz8swSDXW0;Zn99V-*g3;Cn;w%{i| z@sQrtxNqUDAMjr8x;W3q{wZxxs}(FXoZtHw~(?}z@+4LN=Pb;nsxovpUsQL$4c zut|4^v80(Jf#C*V${+WT&UbDru%=@p!%fCE+`dmJ0%5M{Po2lWY!oqhT<|d)n@V*AxcTZE4e1y{U?tB1u_S)+p?iMMsay z+~!ZK2EheGffhgXBxskz$TllKQZ`?t;=7X#l9OaynJZ)Qkhj{e_*7Xb9L>J#c}Oo= zdD}g^$C&AeUwdjHXFeGLyo(7R+5Cgsv85>4_W<^h=@*3KRWzxADLxUAD&9M>21wS) z%S8WG2duUcuoK2!1Q@kUZwiz3!Z9r^Qr=}QIIrQ#T5p!TCucBZSMZZ@99?h{UXF$6 zJ4Y|`@WnN-L>->}2C&kDiNR~`7>hu*nsO_EPjRY|BdLYkadJukM$?L!IMPT2#R3-+C zhcgRYsXW|IUXEP@!pb*F<%|pRfT_)gkmRiygCiXFHu(LF-g+o+b_F+YG)NhONTT%Q z7E`#VN&T(uT&MW6xfQ6gjTIXk_XeS<*68tp((3ZCN$&7I#65>sQ zAWbNh44=;vazKI=+#7VQ;?Fz;RNtoK4zm(O*qcZH7VwJ6-{4Ni_WRQOQlA!zr|B@s znMQ}M*rgZYNm%GJ9iMe1X0-f)bsfO%TTf(A-d+{+$^c#LzF+5aysTy|XWIb_C&t*h zhIw1{P=oJD(&LYpbJW%`VW9v^bwWi>%<0eHR|Ve%PVj5*QQe%fl=AJzoBs?`_<7W=!ENO;ip;V5hqVz^jQe+Sbsb5Luj;*Ijx8oJ83cq0{rx z1kHm9igP6LpS#L~$T(Z#D3yt&N2S|n>EG*4cLlY8EbBX!N^v`O?)K+G7C#~`P$K8m zVL`U!Zx&PU+pxtqoyQ9^_;BqN0f{H4b>>i*6T-~S|LpqLC-(Q*PhhAPFiiH?Ol0hm z#8G>lCy>{oSXNTUAH^T-3>Pp%kL`+OKI>W91um3f3nDk%4WnQ^x`%%*bkQ6!d~O@C z_U`|4WJ1L%X-{ASgVT|FBH{me@V)t}6=Od38T&-@@nz+n`I(U+)*ogaiX7yO)W$oA zj0D+qJdY7&5F3lH5AKOLalW&mfsa2cm!M=P&PCxoPxtl2D9rKD;#-*W#|;eb-oquH z)4;9teSn~6#2}bqKv-JOVUxTLL;%(*LZ&HqQl@LSu{Q~9%G@R(O@*s28Vau285Le} zu_mCzx|xx!@eX_H2e{v}HHqulK5t3&?W+iFE&t^*i7_X=B~FJF^GWn`KXq!d?|J-0 zjrf!t9ZlT8JH`9(NzkB}HVf)Uw;m~tBL2e|Qr0QKL$w6BH5SFJ(YRU2`e^3Xm3?Ut zY)<`l2#-UK!-MnEg9pR#Mawh^vFyS;Anrz;EIA~PYJ1+yWgA0l@o#G9Cm7A*LDz&d zPLI#Xxk@AzgZ*woWTW1c{6vll9I&9)+P}@nH19kS1VA{Pk@^ow6{Pzz{kP9udG?Ue zSg(s%g997z{=mT>%3~dEp2-@d@RmOpQ#)0ZbS~| zQ7uIm9=RI=F~`)I?oT8~JDuvm#rn2L%1if}>bn&6^>8{UOqZoqA9dR=Mavo`Fm6x= z^H61J?lIX&HIoSCv#?w5u=G!wl(@a~h&(2Piy(=a9osN`6srT@)I|LiW$neVF6VVG zp%>d}&o*s&SvwdEXIAO%84`{S2YUBGGyV+uC!-4!aGMT};R<3Tqc%9Ez(ymcAt2PS zF2)yF8zqw>c1+*AzlUTU*3AXAiuYHD!N~kmdW;oy6c%sJQ>N8Yed?t;lAIrh7+lK< zN5XW>vvbJ4Q+>qL(f$8QCRu`|4J3Y%RaFjJk5t;b=*r={zdO3>5E;X-cXoxASFT(O z|8wzS>PaFwd{1~(Wq9w7#vsi%)0MfJci-jV@o1=N&$Y0Tv=oW{<0#;}YS5~ z;pae6ercucM}HrZRJ1n%MNfr+&6R{-V%vk|vdy6jzI5bmUfq2CT3)wZ1b<4d`9>!IgFi*5|TrIDkZ*qJkP(B2 zXq&o%5hE0RVkL)die;)$wQePIoX@AK1r*6hhg>ccN73Nm?Nd*FV}C5I{}gs7%J`9I z`?|fckLCQ6kG~^~6&F=TN(ytc6@PNdLfF!+)HwxX**=j&9;Xz(0B63Ouv_;0zSvZq z)@aW=7&m8AU!?cna5KBNQgK^S62Gc@h|kUA(bnC2tmlxe{egH-ktM74EDctVW6$MH z1IR1DEIpf@#Q*I0Qo$L>;lMxnizL&_9MZ)r97Y7>X$bac$hF^B@nX; z#aXjyJTlorU^GbtMl5YwaT^P6;A(c%9i(64LYIVx6F=f2!gZjpwEkyh-9ElLv;8SCvq(>D!ad1fmd)mA~fPeZyb zN!RCoOXZUk0$G};Gpappz2sC-{7KMJwORxZR0*cjLP>>?%!)fQC@nAW8Y5%Dk_oJg zMU%8Uj)-xN?;ngNjJ~CfS0@1M1?ZCJi_BFx9LBDDCMi4p!C0L_|LeGu3yi>q*dkMj z+byyHUdp%F{=*jmGtc$`oPqkrn$%Ri4?Ml5+HKFGNq}fDbdiDY@~{{m z8Ahajn9M{nVdYRdvUbcOzzfbHYznQDCrb6h%87u7GB{RkqRSg^7b)vmZ%>C|DF%*Z z+>5Ea>4pLX<5$zfsVut*YEc%lI8<78a&|lhY6`!(M+Icgqc=0Av6#r2z(QeZ?gb5i ztF_jwr4d;_mhoc{AVybV?z##gmZeFi**1(5ryFBK(5`kXsH7ZyfzRn{fb$NPhE=s| z+zE+%wtDO5PZ6N!tWt2ByvrDNn9bp(Oki6522V5Ly=}@AuaIOiij*Ql{u*p7OT?xu z4X6Hn_zsz8a;tY^X)fGk4>6!7flO#Fv`Kccpw7cYqz;0SCd>ETs8h?Gy1VOi=JPY1 zX4t5BC=H{w8PL@=xdcWYZ$OUtT)Pp$S6db>k)e3~>lq>Z1DL5~cfg(;BTrS(Tet3p zGTGkA=8kw+&#uU;tW{$Npwwx%#8~8_1Fu~<_p(f(s?GjIK zS_WduEHisZzGuyTq{AHTM-PXw9&{9oS?5@8=9v=z&|}I(1#e%V5VnNJ@8w5(t$eW; zDfhs@%JLxU_CL3{9r7D_Bg|zr+1IgftbMK8OKBb2ENE>}BM^+ci0d2c+uyV_#>DDP z?zD2s^&%liQs!u*MG&$WD+ND_p6#0SVY#s^f17ZY0DHkM?tg81P$E=^4K*EKwM`M% z8&F8xN-ezd>Y$ppEWa2iN3c^iQ!9%fJJoV%iq~dbMmfDIqR6d zz!B2r>ON(GPJt2a=CK^kE9Ja~5EutZglm z4NP2Xyos8jT?u#1I5hjcNR2tEBrmsNWp+k>w3#Y0&<^GRW^-j2j$^TWqy_+B6U!|e ze3EJ3#SAwiR4J)R)(HZ9vzaH`?`k{*@Wm%w=^KZc-E7hz1UWGYg=wVkg9)raoGGoY zIj?x<1v-;aq(cZqCxoC8r;)i2&1)B{QlH3@6-L%>JleDZleA}W4Kv5xAH^}%P};V> zx|G_vr=|^r8&ZkfOF$kK1yf0M(9i+m%^RAfb~}rwC-fT{c?ZrxNcbG>k-xqa)9XVi z7r#u?VUCixK~+)2T^&@Oo%})WhOrG{ztvtepPiq|toB`|dp?GgHmG`d+MQFnwT`#i z`3b`tNY*nT`&tAcrmry0^meso@bu3B`l!OM9Y#Y@tBIJIIHT8|GeiSPA-wTo_%GvD zRj`Hmh%6omXEcrlmd)xn#_e`NYJJ?Sr8WsYbm1m__?&uP79~1gz?z8oOCBq_@J-+>w9OkdVo?2DU6UZJ ztb#K$Fp3Vam{G(yltTJ5odn4{2p#`tZxsQ(5m* ztV&7rqs$N5exQe3+!4qCWc|;)IKgt-KFPp!3r$zCJ^fn;hGRnO z7MB(*dx2bFIR*jpcTp#R~7OGF_DRC70IJ zQ=5sp5Cz6|Dg#6n>+7SD1BAmm*FKE6w?wnJY$bZGOVSd{l;au|>wKu@!CTN~YrVe* z4++c8aO{b7lYQ+(1}FZ^9bY{fp6=3^TXcj9)UZJ9RwW^tLfW~E6ou{VcJ?-j(cTQy zB3jO*=y2t^#5;KduAW;&(Z7Dw3mOb8Kk_kEgyAu}z^Q9LObTF4+?_=46)btJ`n}9b zNPjk>Rn8S3TLq^)WeB5lh*Kno`Yz|k;jCSrDG9nV_^uLrkr-IDhG;hP+>SXWtH zo1{QbWyq5#g$qZNH&XQWKDG>LUJxmLmDI3VWn^PRNPUiqcu71bsPw5+&NR4eK<=B` zZ}<-;93Fv6YR@R*P~mciNE@TJqaRzrsl^cSZyrag){2ok6UKO{|D9ab#ftJ}IbR(d z9l9^IGI!wgY54~#7p1k$?)>$*@(^DN60l=0OkCe=I>-~k`S_8fM@`kc@zdqZ;n{NI zr>lItkqz6T(cj2)-2CMjMmGSc?*(3xrxj^wArW010`?=A$kZc5@$ZCQ4TzQ#&23t| zBV~)1JQ4i;tkFPdw!7$L!LV6`wQ?RX^!M!OIHso5n!=few?<)!tMyPvl@4@yy`9F= z`DBRRAptHUh1o)TSuS@MMN`GYwpIJNoKZ*RQ@agdD1@H@H1QgFf#7yAM_LFB`!!=C zJ}6d5nEvMuq zmx;|oYltHMN!W-MWh*m9^8$IBM_@?7cMKFvHC}jpQ>cM;yR=_P;qTW&fEXrF9gIRR zp_{9!)RQKP3@%W_SE`G9y1nnOy_?pW3=|j-tR+IKXCU+Y(F9v5vG*iMu%r=@f)tTn zy)%;W<1BI;E{|0mdYnU|Dlmp3lm1kdzSOj*d`WJ4RN%NF=(yl>N!IpCqF_B9|SlK`N>uSMOZsqCRoUG7Bsw~saNO9}vAGYtu5H3y6m;c%|*q?`! z9ta7cZKEhylZ@AZm-`j4K;<$6l-2rX^xzw{*U54A+15eLt@;43w;YV!BjWqx9yg@#AIGPv-aoVXfU3&KF z3NDA4cUp(hDJ@Mghsrb8hsHkK4*HwmF=`<;^PM#gHNj2fiQZDrpiFT{At^YehzoL- zH{To9ktGUID#?39TUghJs(E*RDCGpUefEHQ{$9Vxmlns08OupXi5z7{CJ#NSgs6B1 zAe--GsV(L-m;i?xH9SkS#NwatvabwzCNoED4_Hbj0t+DpNVwvZp+e&Mri(a7wU1wy zhy2s(!zYLb7ByN3O1IJ$L4%g*FQ#*v$x|D}fm`5PSckV#2)?MVTau?4O{o3)q;}K> zQdojnq&W^euwJ%mn@Qe3ymC}|oAJ_sI&Pd>Yp9=rLTw0jH{_$}KD$shX2D+vGO_#= z>vi@t*m?%Z%&7e@>E?7KYU)^LjSrRma-H1=j6eb5gv%i^;-&pd5xFdMPU+%bi#2r) z^p)&)1svA%b3IX174T-88@*@ERh`FUjjwoa=wtDfz|*TV**S4-x8%NIvIK-_KId?0 zE*%leUtT-1pNEiC*WYq%P!YlLykUjVmpTVi$0^`6K_0{9fQKGjBR#=}Q{JV$MRzPkvxbd#)LrI%anZ ztT;B21WWE9P0JHTYBdtSV~Z$(;BfY?ArcmA%SusJdVFJ0y!BP5g z)4yAc01h+Ccmf>VX4tV)s^zA3Z)VS;6JU9_&*e54QlqUJIvs_i`g1WQ-OhMur@&8a z?#O*+5h}^gyxm%9FC3|kK^k#6_fe9)5!;{GF4}@TOFkI`KIioFM4GQLG%yd|TZbFYfK3*xWCN&uii4|z9KItRLod@!Dfy-nXytbeOPe3iNVHrq z=mcS4uJb;|*px-q>MI9}{e(?Jg$m`~srNsadsBtIRqq&I%ZVgUdBRFO!f#;YqZb@goVjRd@7e5SUc!N#duNR5o1Wk+v zsD%L*s4|6`vXbHCSlc^_4abICrsz9BpqNf?4qtF1?P-LT5(p;laR|LI+%yGSoP0>> zTsnsgsY8H!NPT&64&67pi9it;1hkO>_(i3AbeLEWJ{*$e8g4q_#F(C+Ea>JoM$|(P zI>UUp9dQNhimGE0LvO02L?;ar|3jO1cdjpqz0-o9+>FT#{_wo9#qs8UU&?Ztl0<^0 zW6YZ2+X)Q1>c_%!Tx+d`H7_s>Iwvk){*L9g-5nkv!-^;i9ffJBrMJtU$^D+o%PVlQ;*U>w|tZm@WlItwLTa3$1;Wt z+80bel&@QPjO)@-eD<|J;!a7Kh-h_n81IV~i375|w59`nSL#^O?8JgFbX6;OV>-$iHC4^P0@OZEJJn3VH=rp#-Z!ceSHKpL)Eh*bVfsa2@R1Vk$Y6CX39^?POlOF%g00s(my-1p8pM-;mH1_ zX|+F;%*b^uG}P3P0IdL^?8-|E34-eo)Ej?jIm&mPm1Xx0yXTonmYf)Z*EvyCL9*y# zf7l3N*S>VO@8!_&BIu+Mpg#Bnhg#8kWV{c8#!xj|F}v(*=eIrVJ%-1>7w|e={{bn2 zYnM4h1t2>Oy$d(We+CadUVmiXJ@)C8Oe(YQ+aii5_?G_gzpp)c12wUru7D}c245+F zBeKf=!;V)zaAi|)J~$ktq2Jc!qk(?Gn>v^TFMP!6uwB=|;o~Z&cW~g>6@ze8MU0%hpVY$VQUep z$~nVWpZ|L}cQM|a!%F#gK8M0rWnV!ss5S;bP-koz?{&(S9GgISjIiE=Lh}~GC=o=K zpnm_IqiVIY5Ved=j-R4Ev{|1^OrHHQ&Zq%AFu^9iAP&h#?&yMX7#IP8xqaJwXxa8p z;h{|qQhIXLHi)_im_lQ=L`e;%e52(W`J<`#0r|1xHlA+h*(uK%qM8yVY6W%T ze>7a#W!~9C)}-yjpJ|rUX_mfgjL+iUn6<{`somwlDrlAbsmgLZ4NgSrD|ggxpH-a58&r!zJx<`cOx0x<-zQ<&(wHH*Aau3=%O%IilKzZp&jrzQZ`s zCUG`>K`i=?Z;QqGa9cKoaW|?j8fPoXIkm>q&eOsJ5~JB0Z{_RA^Pj|S35)6+NkcN^ z>MfKvlo#v9Wh6srSRiJ8CSA~3ajJ)IC@iNSxS2d;S)mb-^d$dA^I9~?=Z05BOS z6XPPUjEeuXI`VuO`p<{fT_8j{5LlvT?G3}0w^!GiccE-LoD0PXd^IlqJfnW4v^^D* zqAo7AzK)fam63VKM&+itjS<$!m16X5nT~6K>{WyWncX!ITo%fa@+2uD4n(^nrOX6P z?{lJaMlw0<5k55EIl;9dJ?($^a-I$d?BwfBl4TP2`qp0ZSD_lkalGqo*X*HH5VRGMUPN?~HnL79Aoawjaw&@Dc=k>+2yE?Fu$k~+K~9>s%u=`!{ycK` z$RRp{(Jxi-m@+R1f8TCO$Nu?d1&l$^v|AbAN##)32$H%mIeaUSqN@Bsap2RM_T-t@ znp6FsKY^4#MW*+{TQ^8Njm4Hu#$By%GP6|UK(eXIuR5vcp{xT58y%py5fW#Y>r9S? zD&t8$sO+tKkghI?GE9`Txh}RUwACA~s~+M$-hWp~^~wpWwG-9V_2s@Ou)o0K10YZa zijqTUM`x@$d61wG7sz=olwQ=W%qKQVepL)@&r4RkqH+wl*Cs<>?g<$$Db(rPaBCQA z@na@T_f{&j`!)BlJ;Fc$;cE812DYqoMFdSQ9Guj*JW1BXW5v>xDjEM9{>PsJne z1YPuvfd{lfJ!ig=Wm)Q`?he>z^^e zFt`{0%xVzxhusidRiCTt10!W{ekheTV3dqCgD%4ngZOv>`|h;ajZjjCx-AAbqI~76 zM*yS6s$#RJ9*dyTVVZ>$i>_mj{>JG^A3e@$LyDS-$O|DsR_fab41V01xf8zpf(Pp2)HkUL}{d z0|Rp4g}VPBSMVi%(Z#mwC{b>WwE`32RM#yefGo=4`m0~cpOKqIAsVdudXv(mW*;1> z44yF)>?8BXOc{_Cp^7>bC9MdawF!-_K)wowQ0JQs);CN;zs3l#Q*40iRuf%8a<$mr zc$K4_zqpIDk6t#8Wg;TW_*x>)aGQXEnu3Y}0DP6w9=MaAGK7QbJYS+dG<-H?SZ7~l z_&68EJ{{1YVWzGjBT-UDbeNbWsqB%ZObF{g%N@y`yQx6Jw0KoR#T4AH3}HJe)?RCb zI2$)FQ3qjx3)7twF}@p_Cva0v zE=G}ld1zV~PHPI{KSBq$*+PEC@`4+I_e^LP$R%C5TLfMJQ@#CYj-9?q_T?Z$1RFC2 z58IeC!{D>&{r0ZU$Yz@$^pZ=Ra~jOA(dpi}ilSeqt_jfb#_F|SHx#9Mk8ohMAY?(@ zRAQV-uAY8=gL))6DSoFU1qqD6u5|KMZ;cs}EBM9E0u(NmJW1y;FZvhC7oW>{xasJF z*Sud8Z(0C@n{hhZv$)K>W>n8nW*WVW6b z2qb9efp=3L_zb}j?f|Hc*D!V6%w5=9^9(vCCT!Bq%8Np0Q?(t>F5-&@o*I|f zJpsiDd5X&s^cM!z9fp{!S691QlO7Ep-i3POO%=2d{ob|foHml<*|@$W6Ly>L)l?>a z@pQUDV@>4U!Fb5ypuqhY*@uC3*QOIX$^sQ&hi33tX7ijrF~@!d_;DQXGsIehY8ysk zGKEYzhJ2V)@W`RWgX!OlIsY-aw*z8(xJXS5qG?If055;X0RVr>BvbiAw*wZ$ZzUN)7$}Wu6k>O_WHU!h2?Dg*@4k8oc7) zm~I7WJSVuO`pp>R{Ar)YVNfI3YhWi{_iCV()IuLRhiwJb(~fFevT8vnRj(sO8Y{>D zz{Uff$sv1!7^@p;G>}M!4hER>$CD5F2T}{s_=3xpN}uET&^md|aozP)I{eZ#^n`zS zGmcxu150y8IHq;^LhtaoQBd~;bN|AJ9CX~FtQdg)OD(4$@m4HzJ_S8Hg8dfCH+AIC zyb)`eS{rnnY~T9BPmsqlbM9(maiC~Q&v3S1iz%*)1GKz)z~^#Y;os!VI8D*x+jkRp=dx1$mBit6AGCS0d@069i|~Q z^Lx8xvk;CU{|a^L*AE030)Zu|v!sgsqXp))J)2+WDY*OPQ%$9H+-u_(C)EPnr3At* zr0FkEuZDu-FedxxXNa4K$-Zt}guji+N6Ye&^i&cm40*aA38RZ}55jo;_SY9)-mEaJ zX{Fb+QfSbY`Os9Z2}ZR`9pz!rt_>G`qd!6WxHN=@sABPC`<}h5qM7pHNMv3bT(@bU zwL%(|nfm7@J1>cDm2F2-huUD6aP*_yf+TrS(K_&geF5VTLDu9H)cP!qk`yE>&@NbC z6)W)@g3B8e8c5ppflqM9;}U^`|DWYuoaB_qzFRUf1rfxEO-0O6(4ZSrjvt)3ZBIlx zR?5~t{xrFY+E^7LTz{ePbd+rYMT829eam4@(-#>)M;s72osRqNEY&F{|H>;YV!<*# z&-n!!2kAUsxzvn9locPQy!+qYssyIywjMx9;b`now#!1H z(--o9hcN0yS>5FIML1#SjohQxBzAqe>T4eHO)&kMa2{3!7Xo%eHkD*c`N%p;AirqJ zfL463Q%-bs#^6c^CI$8?+3fhv*p*sz(QJD9x#r}6#sBS06JPE}j1K zk?1u&2CP2`Fw`fb6IBwMrR|ULHdXU9A>4)nAzUo}QL2n(s~qmCXg`v2HTesmBUrw=ST`OrpA@=&8*&u)?6E2_qcNE9ec^%C#MSK9FWa-B&W~@I0{u_% zf_OfrpW@ug#s@Xr_Q}w`g1B!j@7|~mGMHfL1c!%X|)-a$()B zu>8MW*Y}>vqI_ZxWN)$TkM%|w^*JlKbmBy>h++i%O@HMn!|E#YkqR-os@6*%C!CqOQR6Ca5^f&yG_|6eHX14rkA2h za+r=)w*6a(@GzE|iN}hf_xTdREr-a8uF_e=!*j|)2Kv^>-{*m8DHTe&wI5JK&1X0L z-45{_#}3Ecj+(RX<;N&c*?~QdtO8x*uyAtmhyddsh~cXr1njDRp4qr@&tk$N^5Cc5 zLr_9d=5nk;oiH%X7Lwha{*>(ghYx+6Q`XnVDnopw#RCx9e=_haX>#6}1}I<1DD1I= z!hS~kF(WhPXDBlIOVbs|oC<=PaCoIJUWa69{RKZ%)j{&0uxIYnqO!bvVLQ)lVe( zfZPXk$7&TcOKgD#LM4yt<)K)l2Syb78&vuG*Oq#Iyj@2MSuq=hYGn9r?iIR)o&&%? zwAV4BpV0&knknduyo5UN4e~ZAohx$G2!?_~1tcmUNs$D)wsyDYa5gbG9I_DZ{nlA9 z+1+uDNO;qFq@5-E{Gk0E%$ZbaX=D)It3v*{>KFVK%oK77=hNh4=+-O|GGttKrzzNU z2U`IEW{9qi2rOC`DL2%(FoLeUuM>GgUy)IPc0>{=(%2e9hVXQ`n8H;VMmGc=arp4H zUyvA`mi{`wdU#GrYHz}4AvxL+#V9Qedi*UWYPQ_RC-Lz8WBrx8#Q8?=6FG@x0gw$~ zsu4V=(sWX*wV%CtQ!i)$w|}Hlp5&*gNGZZY>FIq_x|&n;eZ#?o--gkU{V7m%GqHkbAxM41<91hY!Z4 z$Znd^VMwY0FCd}Xpwrd+TJm%j5wO`H`x?*I0Byw^XFvc@X%n03g(TdtI+3hl@16$j zD<2@9d2JVh429mpaWXewL6ZBQesifG%Wsp6wsNid5h`B*(L0Y*>o zuzsx#JHBm`Rzy&8q)zA4Khv0puZ##cL+go(%^a;z{fFq||DNB(DQi{=B)+|V4$7GQ zPwZ}CNas3uhV_}Md&iFru%(<#^Ry}2$_lI_=MR0>+BP0+!#Sl>jvYbjOCK#X-Q?ip z$pAw@yua%;Uj1YRh>%y1AMlplfYe?+>bbShE&V>iU2HZd`kihC<1g*vmyvFg=ygS1 zDp0EloN*C9As^`MdrT%*#PwA{k{fjN^rcB?d#reUTg0EDa2@$o^YySqRj@hPF!}Dz zYw!`_1&sq;(uCVl9n8Arj)MdX;EtY#&ZC_l+HS*FAVI_Q!r5dKxqzhBe}S`QlF44q{!ttEW%&xL2Zyb(ahk|BN)nTrzWu*?(T*h!_aI8?~k zh*f|m%O$)mGk{j~{Yx5_ywTidWp8jO^q+Tnm|5_c>As8SZD`E$jL!D5{XCUQJgxXP z+65xnCe+bEqJhUh@xZgFVpG=+HiO6@$ z-h$C;7^KKP#ZuNYj%}3Nc<8e{W;b8JQ-lbMbO*Gm+Sy!kUTs)^8TFxA%0tBH3s6TS zBxT~VHSc(RcGqoh(!~jTKTW5ar*&~zq+ZXB{U`)&bkU`Jxg1e=5*_C`IW{)*3pEf` z`7O_a=c-^ibb+%8h-g0Kz(}VCprzNp9|D+|dNvLWr*@FSw_*SxO1mXN24m;$1C0<$s zz}<P3giDl}`)pD)P^@u+VDVh$XPq+|UuM(Pz1A zdw0*qYpjh~8Es6xyiY?E=9oUp7yX?|PL>Gywn6et&EqdahJ$=OW4yG>I`iNkpA$Xo?5VU#A4=TK zY0d?&iuv^x8=ax=;3IF=al1v+{mn+rpLjFMK;X`JDUO>cXbP@IvT2;E)p&3uJyW{d z`?Pz&LN0n-I5lvZhqTXIaDg%l(>;uV;7>qo95#O6PGCQrWFdP;{x9V}YQJ5-*;Gp- zd>h2s!x1%s=5!NSx3a+&zhfAdYMvekh3AlT@rGTE`Ae(hZ|Q~83r^QW)5Xf4 z?n;bpKDqM>!^U$FAoLdVK@ypFq*vtXjuoH<+->gLSD4Xp7Q`4NQYN!l4Jx5(KTq6-m;zd{(!KenYlhEsivNr)x;jBc?;fck3>48Jp)A-=Hq{U(po9S8k z$qxPZV$zVw6Fw-(RLJ{3vL1{Ki8EBEQ= zV-zl>>^U1jgr@&5e*=6lg*<24k$-dmrqD1IE3 z;3GMAwQE0+Hz~Q&)+@wneOBA9mLI!lWcY24vO`E3TmB9}vUy_#yBn5kqNZ^0^~$z6 zZjLeyY-n)P$X|)i2HJqh{bREbh)sZS7Jndnha$d_QejSr(G8ow!lRem=vrdvhMPn8 zPFM*4nAqHw5vamh1Z7L9PM>EW^4|gwA>7ChLQ`bx&#=)2T z&IvzA#J1D;v}0NouO-*)ARfR^(v=J!66czfbkpDLw|DGv+iiWWq*DVsod^EX$Z9SM z=VmYkTy79UT5Ft)o8|5G#?w-+$=M2+ zG%ba=8WRnX5^#ln;H*NZVL%z#K6#Xdr1aO;KIkw_|L*x|a_pW3fSn9kx~!1Vxp^K~ zr0icFYET^~RKAa=&p9NHEGwf<3*B>b!EqMGRG>r@f}%I^Q;SX;3U+~>JNfw}Lcpdx z9%s)qOL!QRxv9<4?7M+TIIFUG8G|@2JmB)T{#5V!h=BX2d>PNhmfTH8azX)iad$8Z zRnnpz_Y6&#m_}2^swz6TH|cDfjq_T=9r@{*D8I;qt(YYiY*V$#YH*I2+=%v-blgDv zN=y>h3`r(jcNiap>`~@Qul=;cran0hjx&ElGanwc=Tl%SgZnwv6rN$)0_R5H_U_Hm ztek~?0AfVvoVjDS^|ef;NUON#-Cp(5bH<+9@A`~%e^$}#yoU&D8#qS7^LXLTmqcmH zW?~-WC1qX{WPNES-Z5E|)L0ID-}r|TKet%WUaqlC!=T=T_h`eQk(z}dq0pOXJci7- zhz+o1O&>iT!5li1Rj3N*;5h2&uEc$qKJhoLdz8FqhSLx(?lvmU2aa>B{CCC8a5Tvc zM}j97w&C6qJ$Wu?UZOVIhVkI8Y^v7xdfv~mTrPcu!-wK`2lM-7E?mJir231V+jH1piv65*%#Le`gT`~LeqUY!dCOFJvxp5KaeRO zNt8YMmz6-IR!YDBrPoYiwAzYvWC`W4Cp26pQG2Pq$%4{cKmlY$Tr48j_$fB+3f1Tq zEWokJTVvUE?V`RfRU~f5kY^%h(D7xX@-sXx%=~RWMnfuLH%Qw$MZ=6MXVqneV9e)- z{65~x)_y0iP9te(^YR%BfD2p~L$8|RrfrLfd>$WO$45q2w~Q$z-bY2%^ZS2aN7NIp}YhNe$sLIv?pjnt%_;V=sUlZ3$sL4Ru#y6rL#R z{)w1=76$DJIho+%z{%CdicnVAp!`OIjvG%$V1kiRCrX-nhb&L6{^rVZ0KIWR8L9@% zjo&z^>b_%4O9xu*$j-quS=QFQ1H7u9wCtu8go@t6k>0A{HW6vx@37C6G^-=-MXwP( zMt2K{-!h@}23>(%!6S(_q+t#9v(CfXv5)8-DN?U_A)y^~c?Ieq_x{q1A=j`)h#ii} znFwsx+AVbibrqPGil(7Y*9+E}`&jm=qepP$5rtpYhFu?sZCXo0yrj5&`o0=7EbD0y z#mN)sIlq-dtBTYLmT~pQBTjJ7`;2~Drm83+eH+JvJYFQdvO%E#R2b4Uh`DL({yUbV zC(?<_r+wb4dX$;5sc7!o>7E^E!;WbK@CXWrB=2~XKSltwgqNFoP?H)0RVx(F20V?2 z5zVwfKkSie=WyS`Q*}z>LWL!&QdKfD9ToWZA}Fc#EqI1eKDij-c9}fVuqkA zd0n+S6)NpS>Bhfm!8LI692WpISiH8j0?~uBQgPx)xJCv?T50|_ZElg+3A?3U0T&Hq zohGz}A{`2$R)LLjZPG?jZ14W3G*nE;SFD{ep3_;r<$?ljNE~v3@&D~kf>^u|uNyEi z09OF2rZ$0S-rXFG_o`J=ImTIWlY%C%Tf&QU(mOiCE#JEkEB#BuvX3Y(9GSgmp5 zMMfhO?Ft+uG6+3JEtGmbeG2iu$<;va-VuD5?S#7j=en}pjn%3-ap{wzA8kLDCx<-~ z|7nGR$~s7L4`7=6rHPA%D{sKVC%jAxoULOM2NNPdD6B{xi#4@(l)CJc5-}JT+7_^%Cx|lFh+L%FD!Ng`8{c99_!B49QsW!hnObRbJc+Xf%W_;CxpRgA3b_>kT5(CHViV?}?sRX|>(FrsewETii)J3a> zvx*v{gL`2ja-jQC8|YZwP+zeBTFxO;wT&Hfs6^w`xR)?K)i(ZZ5siIN=XH2kFS2dw zMBzhozU`~GGD;#=9y^U^yTT*BdZ^1y~DlcV6StxiYvED$m}fOda{iihueGD z764#aZ^YLVOQb?MeW$}Gmyx)LVEx*?yT;Es4bp2y0dyL25X6|hdUTATj`qD}=aEc! znt=$)<&}IPd05FQw@A-@%)8!Sd>jc$QQ+_dbSQ7^M9n6`T08g-CF;+nGOmcl>joWq z1jTBB_cYYBQ02$cdvaXz9IIil75N;otxGN-Ii2?GtxX&L9Zd1292@2z)oZ=Xd9;u) z>_vb9*PrmlrB$L?aU19K1QH>K#0S8XJMcb^A3$}zfyEHEzlISYZPfGOSp(}}wC^@jQ zfWfhJ0i~UX+>;h&t8i<9bOq6f%`=@N>gy%=3T&!~+pe6JZ_e48e%w=9 zvEWht`Ik73cwLV8u^jd8kksL*d+ZuK%UI0dky$1&t;3&;ZZMy$pg>S=0PSBcZ%6w> zP@T#ndg{LD2ArW|yV?!`SccM?MrUb1DU#dRj*5*r*pw(DamTv`qN6VH#&L#I znBLdc_$}neyz;R|<2)|UP@ReV9bnpzSsVM_6Un~_c^~GVki>y0d%T-ZFfJK5vjo97 zhwZ~*T|an7p1$p12D2@`=>`mree1wKVLsY!2`)01KKj^4;x^Ohk-h#&T4mqF%oC3g z0PCZIeUn1M3!2-dWUQhhG zTSeo}(V9zJm^N#Z_qSU%g@LISq_IfQ;U#S6zJWYb-GLJzc1n6-4WrUbl(;!2DM83O z!a2D+*yU?vwccQRE{b@i3}pi{5Ou&iuQc9b#!z9Gns^*<#> z5pSR|l`ybP<-!H^y}MZ=-RLH*Cyy0V9)3Cde_PBA<-vbj_5-f@@RgjsPI#$t2V^9& z-q~KnVO0b7N+dDfSLl8^FEfXT4W28WZs7?@V4zm?5ZO{lFIlN~)P7KS)@37mk;{eA zn2Ai298vg>r4DoPqlO`{odU8ZmKac4x6T6DrZB5Ji8i~Kn zaSw%!J?wRxxSn*P%1mCd0ii$pOW~dJF!Y<9i7{oyTo0jr;f(701pryLOjINX04~l8>x8bJ%MLAazew- zVar|FG-<`~d^}%~6&d6E;CaBOts1bsmI&lu(vk#`XGV>7fp-#slC8*vshucahqQU-GVDwI`iNz10$6+ zCopz0T2RCxr{RB}K$iD{K5lzTv(PobdqLx;z$b%iI15D4Vgsx#Cby46`OV-=5kV}m zpp~uh&ZR42L)B-fH=Dr8hU5X#&DJKDlK(=#dPg9Kk}4^)+ytP#4)E>zuiejbOj8iBqRon8?OYJP~%Pwc?ksj^2O-6S-r|A`pIT2Fz>)w%R)Y`R%eq zvCA{k7592^WEaGiN)ynO_4ndd$sVNda~LcH6M}d4wCChXH`R!j|H$QAJ%WH^UXd%} zkM6Qn$7||QCK2aMVpnF$TF%-5@p=_!qNWk@y#2%phMdGe^WEBIN?_?ypX8U-Dt?oY z=jfa97j7~LC2oY=z`F~2060I^79O!bU<1$9@j`Uq0QlCcdmJ(e6IKhz|dZj0dzfF zda!M?ZR1}m00Th$znosaMo{T?deynr@*uj7=Q6BRQlq(XkIE76Sf-f&J~v)4XSP0q z0HNjo(lQt25qn2h4*WPJDC^eW#qK#f%&6o==Tz!&#HGd%BwSoiJqD)!2?)1P8N`R^Fyz2w0mhYA?0cr&!6DltJzes!Set ztD4=-z9GDL{au!W^#rxEBQvVZ*rmdYzH+R zJsz+3y9nM~sS7S9V& zUH`(Ob<1}uv=5diD_n`3E2QnPb?FS#Cd^_QlT)buQsBT)mYc>S^E1Z372hYKNPVsw zNQlQOJqLXl`FMCpeSm+K8R;u*k8N%<*kM?9LyA()OqaNbQP5ZacM(zvVHs-sH-Pgm z)N0qGbh--s7wo4c14wp!$eV{qU)M5q!E4S~z^s1i7Ho-*w00rL?GJSblj%I5i^n}|U0s#R8xa~IB zj2&K-=X^6;6x>z7cU%Y3-<;TiCHfC4ugm>!>J{jWXNY9`c!)xy`&CAW&zW{7K|W13 z)>~Wpw;Ucs#(?$#XMGB{W**i*X3u!c+@fn{cmLaObb$f0Xhw||+t5voDmtqj&{dQr z?5yL>#bkx`cCxJ#B^eZVWj`>AU2=U=dyn(x$+q5A`*8ZPSGHUI06GWQ5A_H7041Qj zr#de84Rm#xU5StOYAUW=U?34NLP&=K7;?P<(|hE6kvKv<_C%FyV;alllpJ?8GJ_msxF$$1ROrsg z`;@2vKusAoo1?k30VZL0#iv*3STD~`(w!jmhcp9PZRWXx-MQ%GG->0(*LNeNoh&!f zJLEbJ$CTVrZ%;=A9(Gqu7^7Ga3eDQwnmPq|_4AxI;*G{JP~OAFpKVJ#cBNBBmt0P& zTHS~ibUdrKp$h0*gTiIJeE!aGk#L3v6JPmPJ5Uuw;uMji=@+|9#`llFX35^yO=tfe zV{bQ8uaHJ{y@y_;V7yVM7tYU1e|sa+N|m{-nKx(-tnL- z=oaDGRgOb>$6n;i*HZVEoqTwaFRUr?D^q`oEaEm=$hQa6^SGaMbq17?DBFJ((3MZo zLr~>vDbP{=;eDt?kLfXq;{N0b#vFg;k}U^LXFCQB7DEp0xtOVNM#^e1YjaBH$Cuhq z^84`YlZ_WX%ViIwg+kvJ1U4lW;%ddvySmA+!_URf87l*GRv>i=!{wLcmfA}ev}*|x zp-s1-86@N$x+XY4Lt?{^VCD(jX#kH5$y0K|f$3+4MQXE>`>v)1M!UD1HcKp~djj_n zb@P;i51kNSoX2}~3dW3Ec8b`_XrAy`Yo^73tbP*qQPflC^~Mfcp>#{cxUz{-r9E%iPmGvYepcf~Y0?9pJP zuLn4}q;~rFu~RCykwnVFuuPHIA||vyl!Ov4)dvYstNxi#qxSF z|6)_APkM{tw>{ko8{lAqKR#!ik<6h~p>BP?B`X-MR#B+IZ>&} zC@XKGAatM%-TB5#=a47M!2imvSi4&^q9+f&jH6!0R?8n`d`-e9#)kQ*gv7j5}tTb-!U?FDfu(j@7>GieV^Y(s9h$@z=_BK(l$e{5Ut_-ehO;* z_6-q@+$E%Km>WagzN$1BtFFY&{I@xTB(dIvm{}@3Y4Mha4H-vyv`3Pg6mVG)rP;Yv z?q(Hx4cDE1WM0k7_js+8GQiZE7pz$Wt|G#GueuRC66kokgyN>AprOtrKLwJ5krZMx zO3r)9>7F??G~2O$(HiLN?|45)4Lv>f;4jxf{9>!GfVd#$gM$^5$!iIcY;TE=R}-Q^ ztNh%@`r=+22jE7P-p>kU2QIL&NwPyjTp0ax%uX~pmy7BEY? zQJUfVgP)>WE_EHZDAb-C^%+R(PUU76TeldV?cZ!+3wz=|zSS-fP2R^u8H4isgbn84 zh3tjac<#3yFn$AQZcRJ@&Anw?YMSDcIyUDsY<18GVLDrxOB1gzT=*x~BX~0%_wB~| zw75A0P^!6^b>Ot(1hFo1CP{olm2#L#Ex4&hrmFgydS=fAAzpBl*EKnv8-XhPd?pEPfC39vJLQ1)8PT5c zX<~Z@349{(nf0z*GlhGWyzcL4Ly}t+=DL@}^)zkGaI?;A)797=`FoA0dBy4~VtTf4 zeF0@%XFtUkPvppO2~DGhPzn6e2T2p>R-*ji71RBIoT5+St(kP#1I_? zQ5xkE_TkI9mq-;}1UR4CXyckRoHS&i{s#|7rMhB;$xJWR^9?C0hMr2tBpOFU zHL9W05k(oezDATIP1M|h0NujAEyyaw4}U$%AVshJg|Iyz^4dv7fIe-zLR_sGik+7A z5f%O0J+V@7V2~?l8x4sOEJZY|gE$?hv;{bLTLn0zKC&f8cM>E+?YjX8l)<&i1#kSxmW)9UIA-M9-RGls7JyT~&fOCJ^$i%$9@;1Z;iM% z_5Ua%Q5|!?jupB|qOOc}h=G#=O#irAZN=?1R$$>Xc@30)Nz-Xs%WZ}O0nkru@6D4) zSC6A0TcN}HkmvLJYmQHPF6!5AhRQm;PwQi*_friBr5*+%KX^v<=HGxM9%Kqy+Qy61 zWr^M{(L4$J&sDUnv`uA+?ycb_n!h|j)RZzeWH%ov{)2#IS8{m(LBot7(**S;L++r% zUXEI0|CpF(%Im6dRz1dDNA>BR08O+1W_g&-3Na>L8Eg3YsEjH&gj7SG8T=5_P%3m$u-Uo=!U%@LlKu7AKvP{pv)`vt_bR;u;?1&3zs+may34 zri?m{m`Pw#ux#Xwrz!_I*qRBExR$Du9QhX}6E4bsYjMaABZo4TUT+YmB4sWn3&hFM zl)$$_hW@&_A;3cwokMdNUH!gX-le+X|B4<`Ui@RDtmJ1=iW6!$F&T$D)fbFZ=+! zD5__CFY9Ira%p&47Pa>ipt~=MS9SY4AeJjyBTpO?zDBQaC++jUBO@1n(Q`zbMq-~c zc!(ZCG8UvIdRV9rlWbd~Bte0|+qmml^{E&uoU-k9S2Io~;!%9a&0Hj}6k;AwTLH5c z&Vo1!V8t=J1RlZul)GVRpriNijaz~{CUf7hI5)$-A4*`Ed~~g7nbbR)G|5!a2C2s! zZ%{L=j}BIBl2f~LL^(btzWgX;G;}x|)hHOj{}M|+;Y-jwd}fF(`Pr(&&me&$_}neU zGL<+gE{R&0pkKH<8?va5~^Y_L!JA5EMvZbF4;N3qV>$QPSsR8CnYWjx=jQ$xO-7c+4K|%z65hWjNb9=e z)dn*1N89x)QUN9%AT(_+n)r`DQ+Bu)Sfr0P48vDZHn~qN#l;tO+>BA?toI`CWO{_Z z=7Y4}iTdD@SVai+)C62AI+6TorN_>nlcTX zR&Kj}Ee#J!PKuJ>UXBCCRMA?GA(lePk}W^-Vl*LY8-Al_^jV+!4Cyi`r@ z46?~oFTg6Fs@T#zuq7LNAExjo&I#c33SR1=twB;O&0S)zzF>?3|y zESaF!f%5Ik2Nae#RSJ^^A--F_>kTpqe}R!CwWa^3U{MAM*|f;!snYiUk~vSTy}{`^ zSfQtSK;nGkm~F4%v^c9M*+N6*%9tDq2T-@@@>nC_uZGU=C=!zDR=Bt^{AU~r5*aE5 zS*@<)q@D(}vpk@G;?^=Uh<)7;TaJ_4UHr=|&C{5?u7s6y}3?wIC*0CSQEM zXnxXPud3KYEgi=!R|jq5&5)2vf|H`zvDYH^-;J~cOt63ZLvP{hr_zdI zKwu&uwZA{j#I^)cvFtQ#h`|HNPIbUAu*xGX2x~L!ISxANDG5uUxAM2Mrh4vDhL8hpA$g-Wy3C&%fGu)t2UTxLKeYv8Ssr&o;8S-Tcl?T1oHFo!Ev}?dDnP`MB9c%Dyj#?-H65m8 zQK&UA{+94KC&1pJbc;sEy4=T4=Mc~6Q)8g?K>s!BX5Dv-`VCAA8C-teQiYp$Y$d?K z35K%zJ#^Ho?vC|jJIQQ&IQWiEJ-@L`CJkloVc*_!37sEI?F_fS*U9&;9^OF~eo(+3 zc~Iod$6S8Rw_1=){73uD>G2E|2DbxnQrd#G1r=zy9H zzeX0*zN+)iE8RLpok0ht2@#aZfXZvTVjidZ^D)E+6O0u!dc-T4yuM0VpE-Ezk*5A_ zq{~7-HT$Ya@C1AZuVu~c8cBrcW}_(@zw#j*y$;1vHGtEGnX0|^f2SLc8%8L!GBuE``V_5iIRQqwX&U5S^<9OUsVax4P}(*tPB$Y=?9&ecGF@ObImy@uYV9CeR2PkdA* zo+|rg{e|YSoENzsZ!>AmLPSs!Cmda|YoHOiTt?rw&45`kqXgKv_v<_QlEgb{1jSP# z?yh(>qI4w9nB0)%Wk2RxgxgeKV;$7goQa%0BpB1U4h0{R3zWp23)syV8TY!|14 zKVzZl9ab|SbKf&XhR~@+ypX^>9V}GfXTe#j@0Q7eP5kE$=r#A_dD}I7lam*F{oBR( zC7waH1bZ?c9(tWksQis7n{S_&YBuSO@!!SA-7ClhjaYgc{hcxRG*tj-fR*dmwN;AV zfy!Hr+J>lJO$#a|0SZ&-p`hwD{BD-n1RK-y zd~VKO-|3p7Rr{$*)5n0zR5C zLG#+&IXaL7?-#MtG11Xw6WW#FKG1r@XJhe(-PkNfo;WvO=3si(1o{KP@OZTJL_@<@ zjma0`*zJ+dF2Nrouy+>#%e)7&@<*-_LZbK}jn9MG*cvYK9b5l;v;M??&=LJ;;luH; zkgii*0~iGC1$p~Wh)6&bcgf?0kBwri?fPd9EK_J^P*H|1u%>5fq^hhHLF`jNi^Rx0PE#%v&%5BpQk(Vo6ntQQOKaYZ)?I2#z* zUS+a$4qcOps1EqZHkne9Lh0GS<=})KL11VZlT4$o(}dflGN*HJ8N1c_z}L@fR{qks z`hx58HeYu|Sd0Jt%nY)O5W#eVU$$8mns^K|BBZpqiy-K4>ciu8{&;RE`WMJM1bb#6J~4Blx)YFFgjvO!)*eSPt=MMq_90 zbgu6)e7UJWZEV}RR$^vPV=QnKM}e4H&gKc$8=dB-E^Z` z8rKa=R5*5;>_@rYf6|;qk}I4=VS(bN^^Wd!s1E`!5M!_~E$lL}Fhy#-nONb&8bv>< z{!?GvtgP9>BY!+Pi{&S4lK8?dpLs&p=Wa%56K0|x)J-y&W1Ygit8Lt(SJo2Os$XaM z6^;ENzLYtEQ*}@MBDc$2AZw%Zi~luI^_^8|uPDAmFuy^a?OV1$AmubSr1Fc|y4;%} zFDgKX9_!iE3J6VMygqJJ6%w!*{wjm^|*}(GX zTn694ba8!Q`gmnutF`X0s8l;9DHHz|T@g8)F>`ZGJFC4{3 z7^Z&7LypjJD~hodT%iq<6a`@~c*eFfl$Wdr_~c^7dCLa6Q1Hwi$mU>O!)7;74b;5DRV~{Qm6KU#=ZU{P@4Z*iQ%t zmPIn^8lXUcHbCCgfN5k*!@pYAPOG?{#~xG@AGsXh5>RRV+nNH8>}hp8gM{Lmp@40s zo=jn-5&89XL_P{=!B4u*O|b_I!ErF5TPPffJkFkZ>wLz_Cw|~TsSvp-BQfOi3ADXt zzU;|*H?jBzkZ@Eu{rZ7fT|wi8pNp#*CmZ9(l))(cLc6?gs^B|>Y=hgTL+!Ei1ze;7 zA+))%WKUL(KhxTQh=ZHTK9Q3kPAfvO5`nyFk}o||$Fk?^z89cB#+UKR zN_=**pRU`khyNwXR0@bTxel*=aSlzK_1%P$^^9*Zlh1(_QuM((hp6m6{_kr0>HHV8 z!;SmyMu3r}0F)pCtQgaC<&%bT4?qCu?I)QtOM!jp5Si*%J5A}6Y5PSYwl#|XGz}ns z*HbtI;XGvzXdVwod1Ihiy#=g+DGZYX!cst?mnXa?wfb^vacA+F77b?;(MUcXQUDhK zc=oNg0pyRD)Lk6ZlKbpcrG2u;N8i3d$u5vc~p@U-3cdZ7NnVr?-P{uD*!skFLNqn`dRiE3a6OWXv z!(~LUf|g8e6}@O=`eM-IevL6-tSJ|zmHf7z_I=17QudJrs7BFasv(wG_qbuGj5IOM zTJiN++NKRekuzqGqyLe$tazu-&gOk6t6iXtA$SnpHganL zs*62D))02;rDHo5d8r)Eg+YGgGNHPFWs794WLpgm;yv~2%E8FfwER1b39|%U^G=Zs zmDA@v6=d&K=uo&OZ)V}W8LLUuZpz`?aweMSBC`UCQ+mtVeoJSu|05;!Zm*w?i+F{a z+BKzX)FuYKt^*&^WZZM;M~t5de#wpjO;)JQjK-4QTzZW@#5^){p*7%meE)#>bz<^9 zs_i}}|2^)c_!v&F2Q`9NDD!d#waI>#hB2L0*%|Gfoa5YV-m@xR-zLHlO>V8L6fkm! zLqB-=8CQrJ&!9<~zn$8US{e-etdbp9N$k3de^HV~{)L`aucS_)6tFXct;OXMFQ)P` zg`3z0I?lqF2zvJ~szC?2qLudn{N6AnnZdJ!3%cdjHp%1?QG||td*KVceV;+D$txxz zjQoS8z2d)-$0~$tCuj{s>GaMs6#Iei-t=i-lDkhM_V#S?SmEc{!#}QeGYfR-!uE+y zq1IwaUC4W96%Em7e~N8v&9PSkByyTxVOiKp7P*#iN~>4q_eYFhejP^l(AQVq;$F^x zO$oMsM+F%){T&_)P|;oRLr*!1=0x9@O|qT}8CPeAzXr;9BJBQeVl)eH``FDk>b#Mi zN4^E2JT`-@;}Dky!FIZV#)8Hu%Oo2;TUQc>C{pq#m~Q6idB=yf{;^&_=>rUV=SZ~` zFwpS*_iD2^;C8;KKnyWYO0e2Y?8<)hJ|c%=Ad@+ZRbY;1Mn)dMra%5+UK%C?Ylecv zN+d5_xJA$3LrZ8cQJSldW(p$5RK}L~kEO5mesGW--r(C$mmNYZFT5_Vqs{chB>qmz zJ%q2Iu@KFi!B@Cb1U^lX>(R;G>m7GuyU5;qfvUxks&|rtu8|{y){XIc%7hxn7V|(C zNsO=Vn&~KsDAy!zJ*Rr^Y!xf#!Us@H9g0Bh#aGW?1a<|sIi7K!zl(;#0X#H)Ej|!_ zOC`!IqP{WlsW|KAv|{IrPhvi^6Euzm(d3>g`UTi$X=9F-*T1-Zo0LeH7lb;}<<|&P zrJdsQ9Xl_HA4l#zo)cWjiQre)O<6I9%#D%WJSxcGOV9l#lv#T&EgQ=+Ndm=!2d-Ck zojPIa_sRY7Y+vibg65xxxajJ(f4m%7;Ql=&=8zIT6Q>OT6;=r!>f(bCGmPdGcdaN2 zz8Hu-QUd&I%}`kLIuV7np^D!y3ty9ctmC&ze4yQ}%K$G`nL@*oY~Kkf-qAk8W_Rd- zT^Yh6z)Ihp*yfp@G)d=WPA+5#LGmB}qW=&kS%DmN)JGjnW0N|sg-Ls9GkiyfM#>li z^Y1~FWwi{c%OvnNNI1J#Xkh_1cOSlm@dtj~d{|j*0*@W9ao`&nT;-+ArJokRtWMDP zJi25!+Xa1%e>Pr`U7l2?ZHJ1qqCgHq0%yl8-OjO~>1$&h)ZMSxOtNk$r8*T+Bg zeLh)uo36#a*Jay5<4&wBo71BNB^WUpR_u7?F`6HR8x>(Ww4QL~ zU2&%^@T=cb7chT@S(4+pcp!~(Seadc&7gH$;X7a!re{d_99!dXJ+WoTEIzOqXhcB* zAu%7U+CgCCe2M~Ztrw7db0|m(>21lY)Kq@AvDRfm=ZJl!d5*(q<1Of%X4hfViR7nb z#h!YB^7Y=J&2?7HPDx#uh35d8N>KwFKZ7R54%W?sT9D_fgaH?!wJ%^$Dr6(D&4!q0 z0Q=#lkkDw=e<4`{$08Y9=?;;rp@Y7d0z^;5c(ndC7f>+epw->CPS@xT9{!RzT`5 zG%I(yr)bOAK_yOKLDO#G+Z5~%ruH(MzL*v{_jcp%0!(Lv7pkY}Dgr}TmoI~551FJ} z+Nkp>7BY_eI-I3>7P}d}E=}B=hAk$u(9g%lM+=;m&=k(pC}HIjw^E}Y)3!aAxK%wk zzqjS+P-ug3A6L8PmPK*L(k)vpAM>3hZNWT|C5ew#T5qD^ZCkt;u9@`pz7wS1gs$+z z>|KNSunMGQ*bC;iem6@QIb>O7X-Wu^C05wH;O!$%xijQkLs8ahPw?pI_DbQ_Cm2x? z$H0|>5DGax7P=l{y{6to$75w zlN5pLY0s_16dXxxJQ2(hh(Fzch-Q}s?K0UZ z=h?LNZHvRA*s6l*@Z8{wM^ln-@>9xXu7Ch=f2TY|RuL1}g!xhILTN4OuIb-}_==#? z%7!XUrP(FeAn@NgjhIs`K->&NS`&-&3l*ng;EIt9n+`M#-q;`WF7UfeuIHeyIbynA zD*H!Ri>%As+(#B`$c6L)sKzjC7megt+(hEE`C`Yx=BVcxGZX<{^#07{&BAZKnmYF=g2zz;i&A z;f4D*XQi`cPyXt>f~7*f_ulO9=4%9HfMER7soUrY$P=&0|B)1AN0@9L_c099B}rF1 zA|WcQx^nBc=UE8vtPN^+LJ5!|6vO@oNg%@b5WHk0X}?~F=gj+?Yce)NDvqev`mDlR z93}w}n|6!Ka=7s{BQhwfE zleu={DJ2G-D-uwCu#`tKrF`&8i%)djd&xi62rr>9oo!-v8V_YTcZ8!K_eUELPZHeJ z8DD`tinVUCm9A+OhZy^nh~dq{NSagmd-Lg3rNIbv1`ZIgy9`+B5ttoY-sN_+4t}_&7ZvdT{S;XgZ|`)J5(aNc-+4Hh^e6M&r)2X(TVi z6;nJ$Vu3e-E=|}OpEOMz#lzPk#aTTTL|B$(uFk8tR*V_X)mepN8~d?uO7U$@efY1s^3q}!l%;b_BO&5E* zEx*(lW~)m)dD4sPfXVQD+9Mw zSM$l}O4ULny0bRuq@bF_I+kY(XfR~s^Uis7$TGhp^0EL^ah5vkWJ@j*mdsQ4K zcEpR}q!-q{vWoqE0aPSrs&FVeu8DiDMdha>Q(sjZOnwEvN8*uZO`}gTtu_IMD6YOt zM-&n2nKIqJa6&o)G+e{Me>z+}(h>xXXWRC$nrqzr;w2{8f~hD@%v`RLe$JyuHXJ4D zzmW!cGKpoKt~R#@i3MTpZadb%1TQRck4Qbn)o<`zL5^72tUorFekGGDV)vUL&dPdb z#Q)K$&mIN}s-g58&=?rpNs8n^uo_~cIsZcjIrg683}k3mRA|@)>Y2bF{V2`s!?`e# zt5ocM(#u2Fwso}SZxky9AAJgtf|`v5aw{g0k|N+`1K$q2`6T(4LTDiV$XWVaAg8Zh z1aJfPmfhDFo@p>&l_aanf`=YBN$HE-Mw@1l@93%~^#%sCq)})5Ik*xi?*OR#6k-_` zbtoE4%GH=dNrOFQ?f?vOd;lQWu%2ncC0LuIVCD{pglQi~ltyWAnyKC6sBY2sBbrgb zvUS!d<5Fo){*=F-ib^S>XSa*;_45k;O)F7_4QjEOsEd^tr$}|W3`4?|73OQ95E~7% zOt-&Y}d_SY+0ovN8t#D4A2YMmKd|SZ@7^oT6RLFo1DYFLZ zx##tz&Jjo#^ksZ_5X0YQRI3&QW6nXXHt(wOR-UaI&)bK*Danhd}d>sHSExPO$@qm*R%xwZfnIvqd)M zvR*&62Qos)gh%C9zR%SZau&jQ*UCDY0?lc8VW1aqip^+t(dd4gzM9drcXk&RoRbdy zSaOP%)$jDgsCVA8XUuV6DFt~?V~z_vuE_v#io!tTXF{7x>l-|byMr|w#p>vBZ9?WH zm9O=Yb@K`J6MHW@_nWL4R;q`DVTy5Ko%iK(I*ihib<8_K0M9i%meUcNeeFgHxz9Ud zcTg(b+&d@l-fn~e2AdyrN-62a4pE-CGtDA@i=Qz9%OtK#2n|r-yOSn8G}q_~FA@0O zeDbybtWUW=CJF-#s@_8MpkWQ~ZqqrjCl?ql`6#^D3b)?o8Zbdoz}z^Vs3Xe#(q*{r zftC1<%n0rkS`=200ScjycAMc2i0`=d?+%I_%e%_J@cB5%xXtMvF2EE@in!Q^1?8Cb zFP3=n_Euu4?ofuX>T$HvRtw_qV@bE7uD-v09{RgAFre8vH@J2@z1P~mlT`JET&2_ zQmq`EA5`B*awY{JV}ftVx{kZ~x^HWHi-;V7_!#?vz|1ERWfbyyptXZEqFhZ3&;^cvAV{f>0944SIk zgBIU!T^R5=f6$o1BkJ=hI%}Ij}Dqc(T3T(4u>pEstY`6`9U2o^E7IoPULY9)S>| zXezZsHbhR<21B@x4P$MM-U@0ILSq#&a3VIUUh*cWi-(63hP_yNA~zi<5In%PLpk0} zwY#q-V_z+A+lpRJbm3U>&zh;SNRN?+@1T!7sa%W&-d6d7A$p!~lhQR}A!i=r(%Zn` ztmnfz^1n%B$-R>X7j!6^el=rWr2Q0*qC|5-R5hgcmn0F!<_bLo%b40St3=dDaSEr0aEoXB7z5QcG&k@ei zx~?4e*q!YYn5NnvmZowt^fwl5y%_=jzIYinFp*EZ{{P-VFO_<;0=J1f=9;BM4pI3( z6x>%bXY{&K*99WL8Isnw#Az_XMyvuA{lk)DE=C9<8=|pSU}IYw3`Y4MpNet;!lYNp$Au^Mr;4ACF4Z->+g!&otFT&W*h;&kx0fgq%;zp;c zXMF}P`VpUBK7T6NwvIc?);bmbn!Nz{-PdgzO&Nai%D_Cq_U^-9GLFUe6|%z}qX1M1 zap%*cuLpdN7A>yFl%W@dKMf;0fpSM9>?cGt7hexeV`{~J*RhIJ5qGZ!&QKSte}hka zPp1$pw%KP2BE;>|r=STEl5jyl9PnAr`k)n<3&TK_F`HOIDEQWcqFE0~7|Wlo+Jql- zqM$oy!(iFAJ!ueO6uV)Af&H$A^#6?5VvYhkVr9a-dt5WN(Xnihj3c%h!^@W5-8 z8#tUB!;!m%d$m{ZNDHZ23~#GDuW7N~-i0I_}8O7+YIMoYhgy2OkXxVQ+g4q(>vVsS<{lQkty0g>5{@8o4k>I=@)FfcIM%% zuV2{-b>muvzDNF3HgyhY4=4*5b8N!4?i>B?bv8n6wx9CNq}U-?YnQ2TEjC<4AP3?; zdUSDzS%R$5ap`lPl=nHs4}mZ)nAh`M&?#{odkf-Nhw@>bRv^r+|1C^ZSuZ+Wuv1?D zpTYGOzBPf>s<>s(bH1f|0~sMKVxZP8+cVu#lI zu8z|%y(>7SKV)=WSijFFc4MSWPVWyhvSZ95nWGCOB6Ni*#UMG%8Q;2YLkepy+@8fCl5B;ZlJb zJt9)th}c8VGUopRUtg(-zv|O!fTX#eJznL=dAk=Z0jwR!nRFV>{g_}?xr-M=aE3S> zfZw5zKoYrUoBsfqb;uwlBLo9nKFaRga7#R$OO9k87!^D%yIeFgZa3J0`Fbhx44Wt> zw$Ht;i&@5McI5%zbqTI08rXJNH`;>|w*Z)XU~Q>oNU}D0Bh4G#C6shQU`tOgGe;o` zH)uVQ&;ufb-1}XBNe^DWGQCX4oplqS7q`51bg^>SP|0_MLp|2v|9N=E&Tx!Z*c4y@ zn~Q6^c&M4J$p$$45sm`Wj!)2Vq2lG zzL-i!*w6qRT}>4{;|0xa%{ZXXMiIp27Y<5r68S)qZyN z8wyNu177$4SMKQ}3NAuCqOXGwR4r>?-KR$CRer+4!Uz&7XGTJ6>v~UPuJ@TW(QYiW z*IY}H&lDK;>OiKYVY?fOAdBC_jis8`w$^`Q@5pP&#QZNKME*Gy1{)*!<};Oxm0I!y z9fio~Ni_Kk8274#n3a`iiKa-i2jiQSv4?yVJXa1SmiS=|-8%aYu3j>X%3gE;?QlQ; z?3NahJRQ`UF=f20bMMEo@F1W8dp}5_W0SHn>XwfrdU3w$iRD)@G}AF+oNZJ*-{p0q zlWG1cYD1H5=Z?r`;Q*2N#FHS0PYwN!Bw^~{kLt4yu5JW*oeg=3uXYLUY%LFEP0Ahz zLz-Qaqy7s!D9O_NX7L4PX|-Nv4vo`Hr3YPp=g&~tLkb1%khN(p)7tCZh81+*{XI(A zYFdqMKM521_jP}j=^Y(|#7A_2)d7X#eQWC~o_KH_Z^Bc0_Vk}Cy2J0C-)A?LHY@@- z4b$$ON9ZwzI-9ud(ndESdfrv&jS*iAu(xalvgC!EEYSO|-Ytnx(T$q(1BH2uA%;e{ zHyU3zx7ckdHjasox3SbLN9kSg=~4FI6{(BqENM^#)Jy_6o35v3gzvSc)LO_0xnw3MumfI-rA9uzxn!~6aZqT3oDT_$76~7`M8dQ=)ALwI&t8j`6l8$BN z2T8KgKbwQ>ww(9G@*9n#O3=f*8V*s}wOq3eJHXyxK6tED7c-j)>enwPQwQOw2Tw_` zU;2@OYB%ADN<`I4JodK}p?itBYzgV2h=e~%wBudX7m&ur7ew>2tdJuk5NLr2GWR=k zY#*lR@=|s!oVKcTr@Y0L?c)bi`w;PgNy2knpvhR2qa5R^8|Px`%Uc-l9a%8;>aL!w z=p2P1nf;8430p6|I`P9<+2so4ASIRYq@hZp2jhtcGQ6^qb0=w@C5vVl8E3}!Sc@%P z#k;Z{&Q{9y^tX%EK@@g}kASvNcI`>S^tu*akgTj>_CQfV?XPem%kYGUJGv}DimZUb zI~tNV2}W^0)q@^r<>ih%z)GI1_61=QVmKOWlRu>cmA&jQc+}!WJ(#ZOb~(hQX%4Wo zzw0hLwTWyvauP>p@#MT{LZURvFfbQ1cx>1ne@NmbwYa|ZjyS(hxBZF4yMnWzkq#=U zIL|yoB0su!dZ`7UqSj)A-u#*vVd^kye@_wbSp`))X`z=F&A@9WP%W>w^nSD#>OM&M zJzN6B*2x%qqo19-jL#&5O`QG9kp?v!tMN#1+oJ}80v}|rp3e5YCHeD)`P>LhBDF&y zCXo!afwT#`D^JL%tO?g{!Ke&{vYibfrq)!;33~R8ZKN3qP~i6Nkyj`a4}lZ*kc$G< z_($H(n!|CokZ~JLguW(Zy5w;?R{sLHy@M|LE|VFRndUrkNhHkbgLBMvgHkq&9G<-;p^MCTC&uD zq?|g&onJ}eOFO*Ocr6lHlej(WCEUvi2wT94KQC~6)4PX3zon)Y6ltK&gdB+PLwO8_ zk6nrbIEo)JL1!Rg{J<`Z=YciXT7fQ2NtZxiSpc5Qw7M)|^X!UM|DMA(o_g=THMv30 z+t0^E7{HgQt}12&SJ?If)pbCpSP{Tz~ zU*!JgD0Bac?;C^#Ed;Hb9^xTt zHQ(d#LVjZ1O3Mo;qY>vK(^!F;uu2f_51}UGhF$gpH~5HmtusixM=(@&emaSpD=S#; zY00)0+ z{1U9ZjVB1~3|Jjr)-B3*zCl6F(mL~!=v>d0c+xp>HVq?fmBGXk&_2ByslPVshl5D^ zG_;RPkpgk>#b3#}t$F3}kd=XmD9fmB{)CeH?PXA#du5G`E7A*5Z2hX}%kS&B*9Y3M zmYA=z?IzTfVx4#YD0{7=KgViM3Ee zbqOd>Ql_jmzUtdf{V5WNwqyH_P>&4F=*RE8YRI*Ymv|k@$^&%^3w?@F`d1)Vd z<1BfkbYLq~1Itp<%hL4MwpEoHW3#3=Ceeo%m(i2R|hK+ zY><}l73bW7x5*FuEfSlHP{yCwWlO6SkPo}n*053?nOf}|ZfX=~xNLeTSg+AY<-b4cHg>!|Wxuf9Er=yM=8hnPZXBlyWJ0L|iod^wHjpOhz*&fW?Qm?xp zzsJtI?+~qH0;~MbnTn6m{B52Y*cS%z=|6YkWJpMG2;t~?0D~elil3Ni5#|ywsGND1 zfB}$dipyr{?EJ?n2k)=5le*QjEh;*R=G;%eiDJY(CUa&v;K|=oh_+($I+X~=dMh+9 zTusiLyRLdIwR(_%ztFs4Avf3RwY^9E|#TO&V2J6R8{{BiFt&q5f zw;F)n4w9P9+#lLl+uGg6n(pSxxG%xGB11}Cf$hBb#>aqU`7Eggfl-}>dh^@E4Qm-e zFy?&hdu zQ*71-KPka=#0~A`dk`z=F&R9b%S>zk$yX>n#$L3clExSzxyrkRauh`D3h$#S9_N}E z{4Ow)Oz;Nj%MRhH&J=kN(D)gSwlqJYx1ABv@4|K1>Bz~g{BDdH0~2wfP4*jEWjHWI zJ@~uuJXuF_E4N!nt~$olvf)!xlq_$3-Upo(f(fz3PV=suI27+?+f%!aN<-e6-G0l$ zEnh(LgYBblfujgQ!MU44b0=;du#nXlw%oFS;ms_51gWI11htv6(WC>UpB|npK~i-< zc{YmDh~v02^ELs+Ng3W~=FD#!I+rUFb-YoC7fkwv`%*XIvFVIFIL=oR)~-$W(r~6d z)Vphhzdxc{ql5e*L|r3vUOPu)!{CnJ_E@1xJSz3qG%nJ{o2y#TQ1pdIrZwkQj0epr zUwdO#yk}73K7hNjQ*e@z+z(nKP-Uvb^4PW-89M@&gE3e~!@33d@TChtfDibe?DvfQX6(i$ zgph6<&~-#9&eJ@ zmG_{L>V%%Yl%E#Z8X+{r^f7GTx#UEXs-3mM{Y_>E93o%Dzu`{#JbI{pQ)CA3K1)|N zE`qYs;Co}yO&jC|1YdcPtyCvvblBt77cE~Ewesx`SMbGnGo5Z&vTJL$p1yAPvf*ca zuUtkO8fwl+|5&51U$$-3cDF(mt>m?lpf2O!ipjDLk;O(-p+y--83nGN;=;S{iOcT~ zIpv`Cx%(DhJNgs_r8C}?Q-zOv#GO!SwQIlkK^5ca6kdAElRuo$YHvR_ z!uGmTRpa~ao^p4D`5^+_*V%JB`ix6xgUKBp=luu6722_EZoz!|-G?R)?b~r`>FT@a zEA@c1vzp;EZJ~&jiQjGxgXeAcj>)L)z9N^x3@zlf)4HmgaBx3&Li_G@kDkjIM;RaF zT)p^o-1X;=`n06blnadWrr8ExUTLMh@L4t~?-P(xC7)A1y7t+wMo}B>7vXp1-i_|o zmD`X@mUbgC;ZbV&&!L;PXA)Pvr6tDJh=TFg4PE1CQD7x+zFt%{Z? z&h)1pu^(;ty9edAK8SnBsEBDYY5?4_+Z7d4a=f2~?H)YAJBBFjbc=bH@?tM# zH!nRBSMnsr&f1{Njm_fgj+3eJ~^ZW}sDUG*b0$?f*XfShzKyC@7=dF!?D^}`K! zypB&5@N&!O|In~+S*KZl|E5fp&MKSyu64@G8xpjN7m3jAHFleBZs%CDC>t7t^p3U` zaK4hRxv0^0^hu*Y4azAju~IkuhbJw9M0obtN?^)3k|9*qq zOK6HcGIb%8``PyUDxm}CW&#$44r|P8>|z9UY(4($yZ6p9)2#`JOj*kLQV|8u z^qh&WQ?y7z%*&n>*<%22>#{K^L|#p;>9m0E=I2M4(WZviDPcNC2gOq@Q!r<>_D`5t z91G9M<8^{Xb=-EIH)di=UwHNJDTH;)&e_Q}CTnr8enQ0tRPORrH%yndvRN96yLf*^ zkFT$IrY}bJV$wl{ae9^Ri&F(6YbCz(I<5Y;F1LQ@slmskc_;RE-PT@$40R3G$Jf_1 zpRYKbv*wfRi3sD6Dk*D=YvWUD`x(|>1NJYGEI0jf%Qu06I5&Q6YfXMvb*?h!B#w)Z z^Ng&A$D`-Pox%=@{gfhAlP;4e^4izJGok`LO|3TN1Fs03I;Jb-vJdIm*(XtZwVJ-u zojUv@OC>Rj;FamoS1~&rnejpyO$9$x!)xFu1ZTNoG*R>V~yfIUUWbRXUis-QoC@~dR7_Mm9 zSeDdSHC*PT4 z^6C2MjVZ^Wevp}z$cgJ)_eLg&7OWK#+y-&Gd=IbCedVIANz{+HJA75PZd={pi}rFW zjWte`RNpYUPHVz@?V=%XxH$v=raSS9#zQu}F)_Ba?P@{qhmbqSH#Yvv!pxNF1U(o3 zm(gOY51QRoB#KO4kf$}7OdEZaVvlQjKRCl4r~Gam;INlpG=;rv{jJGjV8)}WW81Nu z0^RRdedNpWb_LD19CWwIjC;~Lm@ne?YmhXfwDL~w<9PYXL*VUlLua|qlpVrKmi}Ji z!>e*Qdykr1Im$3?s&n|=T$4%zeyr>hy){9BCPmIv$mAIj->i z;T$Dzlrc@9h!TB1WdxP?UMUIpduqGG^xAW3A8%6%sNx9Ss6p0{9}l1#-CGshx}DF@ z@~*bV=732d@MPbRf4k~~T{QXcU0+J1@3~4TPCiw09{%_^>3U{sq|4&tLjBeGO*`}6 zUpkUN(_6=}F2&$$(k-)0%Ss5-MYq-upQyxfb8Jtuz^Nv%%Q%WJySknC-iI|4$NNV3 zI1jI^OD|ZIX6@auf0sS6rc@?s_v9)J&xo(-h04K)CrXqb1j+a|e_C?mlsHe#mIod@ zF<&^DJiTwvP*2eZpKjc~@3kM!Wa`*b*Uq9mn=c=@#YVArwrFLIe6zY?Nr|XW|MJq{ zLkV9u=9{Gb!y|zA3zg7`vsTX4`Pu!Z)Z@m^H>jl>z-E{rX;etzd zia+XQ?HLxIXf8Rt98Z*p$>qz4xBmD7VJpZZw5{DWd7LvUC@A9^QAICWZg)zw^9@d$ zJ#c)n2BU6K)0*NZot}0(bmA_2-7NU+jZDCzj+>7fmiVrc>Jq-yy>~yqA^!(U+0|D0 z!+`S<(Utryfk)x0)3@~u>c+~R?F=a>z2(#i{=Mu8hn~IrwC)_wj(y$JOShN>E#VqD zDsP!TMBX4MH`BHGzD;AcG~VXCs73#2nXRrfeeYK6KlRzIcz1+)x$-7V%Q3tr%Cs&n}u zyZ-xT&ilS9#Jjvlc5pbWRkHD}{iLTAV{u$ZA^5b(;qkreGsIG!U4Kk|*1t^bqKE&d zPiL1%Y+GMB)GL;LWan#9!A++zt0fM-S@u}xevz`wM#V8R-fd-3alFGxAzZJXH2=mrd4k+!TAHDoM6BxP{oObk_0e10fTAuIb5x z+kdF1`M5i$UgqB5f^6bBSaS57-m^%4FT$PvE~`c;pJHA_Qm&jg>Ik5Mx(hrA^yXB)5zhAvtw?8mEBZApv zb=@=S?#ASVn&+vuQkrrSr7QclY}#MAsryFt&DamMw%l|{&Nn@Ws<+WzTdEye{mwITSv4Laz5- z({$pGhnp|2J;QkQCRO5CZsB>(c71XYkMh?Njj)J`R|CoK9cP}t%9sD<#&z`V)_3DX znP%Dgbw?$;?zz|xWVlA=RTXs-ORc`!<3_@^QYV*k)Ogl)zOF3L`4NL0irQiuu{uF9 zS5sq<$9tFG2KlOn=RDdKWN-2gQr;oeiLKPcKj znDM~rFS6U?!dH##1FSuPi)cA;|t(6lJ+gAJNinaHJ^`Z)!y_LGlOLM6T z`XXn!&fmp{1s?H?X%pCz*NSDXe$g*u^vv5a*+%3Y=?F#rLhNUqo-~QkSE|Axmm?kX zHwm5GbZ93wO$Sm`xf|}}@{#H^bt+z_D}4Z^(Y0guuid-x`LM^Q?d=?U1CKd~b{&-0 z>c72)YUQ1CuW9wyJBMB`4*GU;*BCQ4W3w&!bgJLVV+A9OC*eC6tv!@lX|h4N+A2{| zNERdHnXdEDZ{J@2DXM4I`qA^qk#h|1R^iq)n-{ImPN@hg^T)m!B9XkCMzXE7J?k>U z(-TIGM2h!wbe-h%xYb!6&Hj+T{~MKG$u~ZVFg8eP-uQ4cj#NdVv=M z0ncI`jV6~oftsAo!83!_pP!no?X>%r)S{~XBqn9^vAv2J-J15=KCVpLqx|`OuQCNP z4OY)+=v;~s3CLQ>P*=-q?MPsFE!ydGDy_RANy3=8#vP>_9=Kee8_nAklPGC!aD`cv zVbZ>sE_y*)fhTW~u;uO){wDF)PvpvU z4|HE@+JgWTU;z#g0SRye3J?HPAP!`J0#E{KKpQZC5ikc$~XW$OJfiLg_0U!_r zgGdkq;z0sP1L+_O z)P%ZF9~wbh=mb|n59kg3U=R$2;V=@$!bF%1Ghi;vgC+11EQb}a8aBX2*bJY*mv8_M z!!bAkry&6$B2+{QQ9(2jeT0E95nIFwaYft_PsA7TM*@)TNHh|I#36}DDw2leAbCg; zQi@a}wMaA4fpjCi$N(~ej3bi>M6oCZrJ@R`5~_{rq72jswL)D`cXTD{i3Xy< zV}o(PxL`amJ{W&YBqkb@fJwzkMQ4mc;A3(ghijSIwu z;KFbTxI|nQE*n>dtH9OZ8gPxcX51587p@04jC+rpz+v%ZJRe>HFNK%GE8sQpCU_>^ z3Ga+|$9v#C@&5Q=d<;GlpN-GM7vn4ORrnhG1AG_08{dN;!jIs`@soHg0Y{(`qzEzu zC4w43n_y0`A=ncf2rdM7f+rz>5KP!kh$X}mk_efETtXqCm{3pXB=i!72_uAY!USRV zI~}_?y9B#ByFR-SyA8W5yEnTJyB~WHdnkK2dn9`_dn|hbdn$W6dj@+pdm(!fdl`Er zdo_DAdnbDz`vCiU_EGjRc9h6Xwnh5pf(u4sH$^4mA#S4kpJ+4o?nmjzEqOj_n*V9Pu2< z9BCX`9Hks(9M?H2IO;iiIr=z;IYv1qIHout2_=z96p}bef}})JAsLZONDd?yl0PYo z6itdH#gP(7iKJXo5viP1NvbC`kQzx(NNuDpQXgr6G)9^x&5#J30-RJ%2~IgqHBN0# z2B$5jE2lfB2WJ3hFlQ)d1ZNy)5@#A`Cg&y2a?b0VHJmM+ZJb@4?>R>}CpmFsBAG-M zAj^;y$m(QsvK85$>_B!VyOURv{m232NOBrEhn!2!Cl`}T$z|jUatrwhxsyCZ9wASW zr^zT6nM<5Yf=h-=iA$f$h|7e_iOZiWge#0IoGX?qjw_ifl`D%Ymn)B}gzFMl1y?m! z1J?l8DAyR*G#8ee$j!$s$F0Jx&uz(V#qG}R$?e1K&mF`a%^ky?$eqcZ!=1;S&t1e_ z!(GeW!rj3=z&*k}&OOCV;-T`0^GNY1@F?-9^XT$0d8~Mxc|3W7c|v%?cp`X`c+z=F zcuIN7c&_tQ@l^9P^0e^0 zf0CadKo*b^kQ2}pU6LaP$keH@IatVphsXxU`$|K zU`7BZND-t8Y76QLG6c;9Z3OKFT?G9Eg9Wz>MhHd<#tJ40W(sBr<_i`JmI_`MtQM>n zY!U1b>=b+{*e5tFI4(FX$R{Ksq%Nc>WGQ4L3?C{-w3C_^Y$s8Fa#=#o&G zP`yyIP?ykqp=lvh7%R*zEFdf)EG4WWtR}22Y$9wg>?G_j93UJn93dPh950+IoG)A? zTr7M^__}a|aI4F=O@v0J$Rz$l*yH2a2)zW%t1GM+F5!w`OhDMU0NN7tiC2S-dB-|xd zN_a|yNJL1)N~B4oOXNtDNR&#HNmNVJOLRzdOY}($NsLO2OH4_ik_1Vzq==-1q>QAJ zq`IWOq^+cbq>rSZWQb&_WTa%QWSnHOWR_&MWUgenWTj-6WRGN@*1 zRPMT5wOp-Si(IeVh#V@9lPAejsZnV_X;^7YX;NugiJ(kY7EqQ_R#Vni)>mdKyDGaY`zvo(j#iFSPF2oU zE>q*GvQ%Q$boyi|dzs47mCq$;E;t}3IdqN=H?tE#VRqUx%;QZ-05Of^z9Ry9>MT{TNJ zS2a)dl4`kXwQ7xOqv{jY0o7qutQtX$sK%`(r>3CBRI^mGRkK%fQgc-cQVUg!P>WWJ zQOi>+QY%%fQEOIfQR`CcRvS=zuQskWsRrpJx)fc3u1jaoUFh!gm2^LPFg=VOL64;; z)6?j=^g{Y|dKJB%{(#;>@1Vb=_t8h_Q}k&%s!mWBQkPJdQ&&;fS2tI;RJT?4Q1@2% zQx8-RQIAwlR?kq+Rxeb)q+X+5uimEKrT$)hLVZS^puwjhph4A8(9qU!*YMEr(Fo88 z(g@Yqt`V-0sF9(Ot&y)$tWl~_uF;_JK%-fsMPo?gy~dcvqy|n?MpH>sO;cNwscEIT zQqxn@PcuL>L^Dh?QZq&~M>AKmSo6AOwdMoOCz`{Wqngv2P>ZZZ(Gt=U(bCnj)N;{s z)$-Bu*9z2%)=JPy)=JmP(#q2+(W=mD(0ZcPqcy5Eqeajb(57muXlrUSw3*sY+P>OB z+QHh}wZpX|w4=2XwUe|nv@^AHw9Bd9d#XT9bFxU zj=he9j`}FZFC)U zJ#@WwLv#x)g z&=1p((2vni&`;J+)6dk;(a+Z}(J$4n(67>O)bG~s(;wC!(Vx;s4JZbD2I2;C21*9% z2DS!H1}hDG4EziN4Z;lK4Dt*L4ayA~4cZKP4EhX)4aN+n4NwL*Lxds0kYcDX^cj{6 zXNEh&lM&1aXGAmN8JUc1Mjj)dQOqc3TxZlU9xz%M9gH4EFJqiB!+?e)LvBL>Lvcd| zLv2HILkB}=!ZT^9=BAdW9;V)=ex`w@ai+b?iy<~`;E=ELUi%}32A%?TDn3jqr$3k3^?g^7iw zg{_5?g|mgbg|9`BMT|v~MUF+DMUh3ZMWscRMT13$MVG~-#gqlslG~DEsbr~c$+Wbw zbhdP{^tRk?8DSY|nPi!3S!`KiSz%dk3GjFX06duegCO(m`FghhmkqFdX4|uddGq`= z@572+sOIAS;X7wzVE`;}&W9lL_W4}%Wj%0E`1!o7oGkr6 zwf#&E7O+|0h1j1u@PI3d%XHo?ifi@{m+9FwgfBZU&tKSPfhp*y$c*%ra$xkJ*J;A ze{X9%V9g*61MxHf@w06_ZDDrjG1_d4Hs?2Mqs?LSJ`m4Z#d%3IFpq)1V;5rPV;5}m zK45{L<-5Sc;`|r+SPz)_ybJkQ+zT8FIq|Gr@(=y)fKZLXfZXi>A+oX2cE=Rc1{nbH=Nn$Fv1eOQAw{hKRlK6U|%GW{C^ z^RcuA+h4r%HrhY*%-`8j3!JnCJ@fe%^vtj7U)p}^_@}nN^hC{%ho$AGj(N_x#==8! z;chgWz2p*CK>spBWXFl+k-hb*uXe=8mX3x*{Wa(JYtrU#`lCws|eb!AzD8$Y6 zC1(3)S%qybg6nTZ|BEFG{=oq|dzs8$xY=uF_WC%pP=5CRPZvYpLcv!0Kp4FG-t&9f zjZiL`HQSX!ro!r6bn~`(9u{O@p4Q~*{Hc4@8=)I-t}RZG8I$PVzi;qytjB&C!I|LL ze~xeFALYmFHD{}Me2TwTEyW>2YmM;6>Q$r!g0;k4_^?a*R zywkkwuh5K^c8Sj(Op-C- zFrvgFw^kHI*xlY0b1ntPA!}}zZRD!^uz$xIzEqBwkf4B-TPlwHiC>2By2|2C!5Y88 z=SelktCM{;>vgx~9bNH7`iVqqT$}gXkl@_YA=?Ih6Sh0!qxMP}U%7!5v-rw&YeZrZ z&kyczf8w9czND-q9xp0om~v@l;thD{e(eE2qeoYt-=c`yUEJhGct0rDSEtSK`n`~# zVQ#e8x21KG-{4+Fx7y0I6T!#f`ByKiafx>~X$LM} zEd3|`4VS)t=45_dzCkwp?hj$<^y;`1frkQ2*+q)X8WCfYDF1g|m;wSC>G|jeVE#I)W0^5gzk@Xt|+%42;baBt*HjR=| z$I4I-_0XD(n|^8&H)eLlm~82=7Cb{?OAu{+O&@(E^(TI%6}Q@CUZ=jV8`E-+$ruRw z9<)YMmcVPiLE>;&=amXadMd)!Zt$JSIlKi+a~%p#cM!c9FfHGK70QRB9G z$;q9^nO<4EPPK3P|HS`8;$74a+aAw7w_NUQ;BNZj=doA~x8mlp5}(~&*1?azm12t& z&R@NKxL;UiiD5yZZp4docS7$)urS{Ikx}u$=wyk`~G-;8TDUO z|1jWCi_1P?cy_gV&U%Cjxg|j^T z&_Q$=66k+%_)q*9&*B-kPu(bzG?l-0vg6a)L9FPtH9Pjo`Plbi-EDVjE)JWJAUXOO zdPfFElw&v32HTr4(YhL7pI+(j zb!=D3($}2P$`$UxJ!DJYaP5T9Ten_LY!0}jx0wu9_=`6M zM8s{`bKxr?QO-hR*#+yg6V-QmU$|_iSqhB=b$VCE)yS({8ZnF(Jh%4cmiCsL*`$lh zg8NoKxBe6VGWFx1Pc9;l99LD^l$IB3d}49RIk9)=EQPF2p1Svv=NvWri@n^IPqHDa zXHF~Md-KYoyRbZ@>FIS?c<@cgCUoPU_&+)apXrf!)6*b`NeN?YytZYV$L6@E%GB-0 z%kQTStoX!{wJ$LDh`RXNJNs<*+;hoVePk0MclDHwF?$!qx&NB}pZNdV|DW+s{Ez+E zZ!$%A$OokDE!elUuF2Wqf<5@o}FL_CBlmaDG#k&XYY7XYcGeVv+Lv zGy=b3Q1XfJ!%<~`Zw z@7n@C_r_+GZB*fr6x%4WYCP0*xvzj}^v;zOLu3Gb3%&KB>eVY4 zE5F>;hfSsnIJ}iaT+9rXrm!`V^-mO(iy5CyL2o@C9nj?HvR`}p`O!b|TOZTpm@QTnJz&muwj)r zQABta;AQ2_MP|<)6uJt1Howw9FF?@0E;@^yn#E`oy81scIE&F{v7A|q zPN6gZi6OHXeHJ@Ci2)-{Kru zb@Fd^E^&$ZcMKO>slU8y`1$r<@ZaF9*RnvKg?lysiVL!8Ef!8w7X1aMv1(Hm{_)|j z@$x@)|C@L^t2X?Zzi}Zx9xTkE9t(G4;Zd!B<3C@M&6V}P;b~xDj;&evj-UD(f7J(B zW4xHh`5)iW9Am|ARuui0^IPz5=I`hHUi?iTWMLQO7x>s&eur88^{oDWR@_g2E>`}Z z`{$#=Lcf63zsd6dec!yIE!Yc`{?S&%sajG*ZDtuVWIDMiSz%qUUT#Cik0X0asJ$|`+v*xd)dLl4?p?W_e&HDfAxs^ z*Z8@9Cz&mqe;Y526^F1U%v6e%r*2^!k7|D%&&T_$em!d*Sh(YFei-aRttZ3c>tDcW z{lEH|4zcb8Canpb1 zljf4<{bzr33~T;_7i!a83-#^Lf6a4o>#ue#@GIY|{~@1O9Lw)FJ~|hj_D?>?z5k`Y z-^P3UzvQ#{ZM^i@|8Bgdzm50Q|B&zB>2u^VrT-J>+V|h_DdRPKzo(DH>XZM4&wo>PnE&U`tF^!NO*L50(|;3(VO=x7ogaV3?f>75 zJN9qlets7JySRdX6E_!w{)afax`w8fwvMizzV{lRwZ7|E@Tpns)cn8I!3Ea$PsUdX z`|!B6pO`rPV)Df=YYqPONB1u!cZWN=cgqt{xrq_IJj9QO?sq_2eqZC4O3H-F&VauQ-g)&R@}L*$(9Cit|B8>QSl z2U1R~LzcH+fSx-u;gIq<5Q!gw?eE@#5^NVx-(Uo>W{*&~VH0$GOF`QMZbN$d2S8EW z2;aB_Lk){K$hCn0H2ffRfCW(Mm>47%T!(t@gV6W!ccAdi3OqdLfRa>4po9GgsBE`| zT#1uF*3k}Dx`cw|Z_a`Er;lJw^BI_CW`|zr;erPp*Fd|jPoVDf7NGj<7|2u_MUJ0Z z53>p15uZ)1fD@Y!hDLIsNO~6ZO<{-ACl?`$Cf@)zF%4j0a0*rlu7}ZP(XdLx1;v#f zfHL12K#1ch5YrzFBHU8J3X6I$ZFdq7X10OgZ$E&Utt#k$rUd$S+<-?cl)(O7A7Dj5 zEp*oYf@*C20W~Y~0K#1Y%$A*k69!^Him8oQysH8#`^I5zcsHPJ;zczD^1zLDD=>7? z3w#>Uf%@(TfHPQz#+_OVRvp@b`0cs{$}SnBlL{^%c8?rUKMy zasUl6DM(?EAiN}@ft-_l0PQEXf|l+K5F2$F8Yl~a1~~?-qJUEv*YnV~p(j)Vix#@&!9DMO&!ehS^|t2cXIq97@z zZ-H>~X}D!u1o-;>8}MCy7uF`eMl=+!f;bKWvhn<5c=z2MILbT@s3+9WW9GWBGfxt+ zxl#(AnOQ-%oj#zotP5SKH~>F@<6zvr9Ikdc2YK3_f+7zqptbrd;7QLx?zWi0;?HCx zhh7fUjXok}9yOr;4I8|N<%auJ-lG9WFkpqldsNoM9%?OZhKKmK!L@@taFwt*z}|C1 zX$1%1ksfP+w1_~Z@dEh#JvXF2@kb7R=mt@a)e3m;;-O!eG>BEv@dM-u7u<3c|iXcF_iy^CD`Pq zh4hrU!$)c>kl-9gNZa}qlpb@17KblEw-3TlpB43SRqt8ibCqf!;1=3O2L@@N9B6S44kYzsU-zz$58$by}dYoQp$8yc;g zgkNLrL9Iyv62iU|TzeOcJb7>j8m|40NMY~8Z7Va-CqcdIUVy=wD@f?lSg6->2__82f|D0jk?Pq!?KaT|Mg6Kk_lpwb$gnaz{&b1>UEO~TClN$`qX5I zBZNR^dkO3;ZUK&wXTd}t202^43%ZLwL@qEsLUx6(aHG5?OuAHqqA*>u1`4_|EMh_ER!=^nDOr(RLLc!hA!%b}R-dVnMKo@DlWeOTtxq zgrQxg3am*Gg$A#dfc@MPU}G2!IDD&w_PyT`nNee4d?5loKKsld^fy88T6s7_*GEjh zkpR!sQM4=YF~nXtizr1`f?=stRC*#2S|wu8bpbPAVg&|hD=Pv->?6Q1On~JQX%L)$ z1>CZ#M(a+Vf%^FKz+Qp_FilX9VJ`uTeyoFUx-rl~><3IySPtVKjvxVv-0;SyIPmyU zB(PB7M6A7xfs>&iTF531KYkSkQf+~NKUxRnmFj^*MtZ1zCmTrGXt13Ge8!nl!Juny#1s$=6$3p^Cqd-kGw@pU0T>cVN4F|r;V7pXO5cy+{x^YHBVlqaBFb14W6eS@rf`2a5dPKL*=oZDv!P~d3 z$OV&=K<=X!JXt9XTjgCK?aBn?Stksa33@{dz8By~!cN%LHVg+J%z$(L+MKfu*x1$6szpxw?syzYegK@~Zt4qOj)J2$ac_$eE#tz;IUIyDX z>;>{v1)w2z3=Q{}1^rdFAW7*QP>$h6l}G~c-ApFT4qOX8$^=or=Np0UlNjK`(FW7g zlhOCsY~Y=V2ND~d07U(OaiAppmU#uxtZJalqGXuuISwrQeBpl4kHF!T5LmJ)3{Dj6 z0u@mcU?gQ2Swh?k78%Tb7UFZ^lK5Cu`Wz7+9CJZS20Gw_qi4_$9dc0B^#p>;b%RBc zgTPFA9Te74K!$x>p?r8D)Y_s1Lifpo%zeV3wRt5XxmR{}ej^a6-dGrl2BV^+UErA4 zV(6CA1kYAIM{;=F;MKJPKs;0nIH_)c90(3rIyS(TFXsUVJrTZo;s&<9&IUWz>;bCp z-ol>O1laUC9J;qNz~(qNxZ?8zNGHm|=#4jk`(`rqRecB}AF3crX1bw<4G+3o(HDN? zyAMwF;())#JK&%-`yKWEBY1_(1(wz>LA~uC0EO$LP^x+tkjfN5o3w8OVni;SJb4|W zF?7J^5d;;(2B76VX?Tr(9o_VS56Wr?p=&yF;qq27G&dCsBiW?j#+RGm9lOh5vF{Zy zrbY)Bgp{CD@if%gavRF&t^^$$FG6-b9C$Tk2X3vokBkaM0Oe{AlYf zNmDb@8qEMhQ~c0N_ZS?WA)|NQm%&wOvEag~VDQ$b5KZ6e1|rTlqpPWPP(qE2uGz5< zV%}AQ$%)-Cr9up7^oPQL&Mx${(^+UOREQi8bATe5?g-m%Zjj29fNF22fv_eI@OQ3< zr|(SQ8`#6+guHpY z7rtk^g0lBrfUmo6LvD3ZklS$=!EeJrjgM9!I64s8eH;KcFGYd~D`hBLG6q%`3L(P% zR6tJH4hzoRfJL=-=#=eE5Wnj#u;%xI%ITBvdGcCddOI84KJpqwUrw1l>_&j$18+fj zRthk!zm1lkw}#d|QNW(<8@$dm2kXBygPXJiXs6&OV1B>}c^4f9d7FDdw(oFM_% ztbYhfwuHj`CR=d-t}`?+H3tXOg5ceWD7gRCD2JVvy0CiCg=2e6 zguyum#Je^lqMb}|kF*upX?PdDWL!f=-S2|+T(-y&t|nmlgANC_zJ!lXG2uPOH9+Rh zH6+zU5+rdvfa`4l znG8VXJ}m-W_k@5@773Uybwq0>1z<978qly#0juqfK*1>{5Wg(|uC+G6okOSLM6oqE z?b?Mz>~jYPi5y6tKNZ-Byn(?x${{m15Vj@};mLslI1{x3=yMt%Zd$YFs@Z2FNPY|n z(JoN6Oa%DtKZQixY=II|L7<~c5I!NcBWtnSz+i1G`XWCNzH?lTS|+RpC#$PqCRr15 z-UF$6r%laRu;94Pzl0J_@46;?)hf(UXGID0f3^pa1)#*;fxg~TYZ z^zsgPA}|I#<5Whz6g0snn?=E9?l;h#O9s}y*a&_!JEQm&V{q*Ac_8c-0v~SO399<5 z!IiZ{*eT-$GnSkH!*9i5bE%Xh2J0Sst!cq<1|>neGUMxO>ip#4>iK& z;q6KvAlZ8g&>a^ck2T)`m7U{=(gi#ymKj2`m_=ZxWe4i+h`>D?ypY#h;=m2{?MR8S zDRk20MnCe7!Dtd5iR>JMIU*2o-!=-q)O|s9xVFK2*;yc$8-jxYVZbG48s>WhVd8CeY^^kf4&^bjO{xCV&L+VJY_*=GRl0=#9{4JXcZ0ip96$Wy`) zc(B_7y}F7HBJ*e0O#KJoGKqvg_|DCKuZu+Z=q6Bj$sR+I1JVo7J%ky zG3X~Ri+H?o15I2}=o8(Y@CE-{@Uk`%OqZ3xU{-K@8}B z?gIf|E1KFc0D8g+$jrINK+`}51v8v*tUMW+89fX57w-jOPZfY?%@$;@U^+bUdI0@` zwTI(JLXcOhoM5-(7o;%U9op0<0xB;a#_wN*uCiSPx>MB=&E2m--!1{5N+<)3rdlxO zhbTxt*$QGzmw+arA^`dm!1?J-$hwJqU@9*U74t@+f$bY4e5?}Ith$UwA6yO(y6eHz zIxKwf^(2yXIuQ`hzXp30`5?$oLX)JWp_&uVu*Rw#tbF7E4%#)qvx@D|Vg{ht zs0dgI4}uiuXR!T)HQX7g4Jx;$QNIsW#l#}wzLDOSWJO$^6wy9h0LJo zTRo(e{{%>9cp%E6BJld|GpIP*2#DO`1G}^kxW?-&VsvXa40urhZC_si{jckRVdyhB z^_3U4-+Tx-?(9Q$THb~Q#nOn*nw8+C;yPqMTnDmN2!WBErf`aF2;g*hp~=<@AWlXH za^8ply_-v*&y^lj;r(YgR{R>7;E@1DK1WfTVGS6%l!o{hya&yJkKh}^4G?Eoh+JNk z1U{czkG@se1ucDUfYA7I&?w#ks5Vq+j&6k-gad;k)EydHYIe2tzc zk_EY!gb?LzC?vfQfGxM2U}*RtB(?d%^}6AR@RvNG;bVs;w9}v$(;OAx5Cs>OFM*~L zUf@)mIWW^5gQ*X_P$4!6=&k7tGuj?OcufZtd`1N7=3fx*G;7EqWDOJ@+JOK@7K$Vs zhDRy{0MD7dV0_s%WP9Cu81c*t=!L$89MpEiT#X&(AK#0l%hmz2%kF5sY7yk{aYkDz zreUFqB0SKt3$)MfO^*H(z*%}Tf~1-Qt&VVXut*1b#~Xu*XL7Lr**h>%6$xrmmVvyB zt01LyHOjTP7r4o1z`9fj8_ma&5%d`}Ol(8F$~S-g&cf3 zy$W3NZ$ytY)Bu4VA}V(}1Q2jeAn#chxWW$*(Uk8nxw!{kiSq*DxtqZyVi%0%s74f* zXaHt#7-V#^!NApmaP#>ffFpYWX_s(db(9Znwz&yz6Z2u)xFjI-uL7qJ$-(v&B4CZT zJ6P@T1gyyV2t1$fhU+fA2Haw=5J~Y!FfbSbU7YuT$2Q}rYSI>H&a4FtYh!pdwisCz zwhb~aIiW+A#qeR&CGh;SCJ_GI@_$iqo^c_*Z5TeU`?e_!Dj_sPi8d)KBmAWjN=ix@ z(J~qmiUtyul(M2lk&3j8&=d`YXwi_QQi_Vaz2Bb?&+mG!>pYL+%R_5w_36f_?JP9( z4XzHpB+X-I5LK^@VfJEFqoBk~f_Kulz$~b~>Y>qVx3GBzfv8>Q&8NmLLDe_`uAKf8 zSvHkj%>WvF#cD*qk#8K08IkkyTauyabFjfum_Cm@CU z^aR4n%#&xl%%EC*asI^I4&(C<@b}@rk&xoX8+n^Sk)6IE6ku`~-3F8C3kh5L)TKN%=nUEglv$6F0c!Z*?-( z4S@8qM#?!ol42dQF;O^@A|B__TSILwJn|e?PyI~!ReflQ(xcGFg{VrhWKJQ=>D0M= zezce{HO-h_H%C%Nu@1&~ZlW*SC-K28zo=p`iCS`uFh%h)8@qB8j!XQ8NS+FOb!6bu zD?-YqgREg9-B{}Y;Up>pGD-qa;Pg^f=Ju?BY!W@ zLw){ZuyOD+3a5Eer`iBXo$%&#T9w|uOQyPU?KBX&fO~9RL3&4TGnaohaP({80tPAQ zKaq>md8I?0c80l#$kLRY!w^zXqKeLM)cp7s<;gTM`yq~cTl0g@zu(~Xr8`VKu!^>g9>G5wkAStdH(Mk%1EUVSp&M4mX+*j> zJ1`}lexDBGRzEwD>Eez3zrWG$I|X$8h#pzUMpM$V|L^{7R+yKIGQHDmPOm-Xs9%Lo z);$>iiD0IWYUtLQKe({a6S6DR*zA?b6u)jWm$NA*ISm;kiCW=P`(JKoI^eS9CX&r?q>FYk6cV-$`%V1l z>a(R7cX%3o8EMNBU9AzJSIr)4X;56tbzV1p2jyRmLD;36s1mq9xz1Yc^jR)Q_QzMyh5hu|1jGbX1Hd>@!&%wb?kSA-CGTcetdw!w|*iMyE(jMkuy%J zAHkE!m&snUjRd7)Y38vKYRaF2?8a=iw{P|`5WJL-T=|f7Oc|m zG9t3NSk{CSBy4{d*PMP}xI-w(#(Y5RR8dNqy9O4D)7h^_-?20DED9~2(~mQMxuByp z?Ok@0MHoD$)pv7P&n#uKGabtctv1o0ViE4trHifCpRp2=Oc+hnLH6!A@`$x!Wj=ve z&~=K zs4{kX$b%9z?=Y2LC-L@CCC;bsLEYeQ{0r$MvFz0tsiuirlkT&R_dy%3Pr%Jb-K16a zjBj3}13L*7^z>Cj>`ok9#y%&}qu=3hO^z0Bn1}mWn~{B?lx*aDXoOP^ys|Hov7Hw` z`KbtkZH`=if-MpgRhX*gTI$b8=jM_1l-gHM%|q_JT1%59FHVQetJTQp_)9wDKj6jk z1@v`#0?qx{h|+znTy#kbMQh}fPvdY>f3+8D+XE={@<-O5m_q%J>e#fvH&iwLH2rvC zh$*txY+`FZ6uwvU9xXWp3J3DbizT7hZpqkCzgA4@B>C#Qbm^5c6A^eyKczL<8KFpY zJ{Zm-a&!=Td>EGrT0o(lAK|7JgbkMzD4;DEd(D5-qcI#y{JZc*UIL;;QFLc=CY2R; zuvuj{=)vGMo^Vhd&v(_b;+5La5SvXOl%K({ybI;oCCG}3V(*6h#s8Bt`qP)7PRg3C zi@yPFjUVVP zJdZ{gr_uO5H<*8g5v8X4l6-A6Rqq*~4ku468EJ(jyBp}j!f`OusVC!#KQMR~kKZ-< zbikb;o9Kq=H?nBKUKJj!FcEfnilnvH1V>*Q@#B8a5il(oHD{iI*S^Dv60h z@ZPZ}5!Gx?%P521sK(IQ;H7k>>jNf6d!q8pVOp+df|plQD9l_DK90lrOx{87|Ml~v zC3>``N1yudiqH&|2HrDwfG)`>ljoI1WFotsZ*RK<|13A|l zgcFIp{erz$9wv($g;E37vA zkdWa@oIG(22Q6%%boLCw6R(h-;#TNgnhSX$C;r~wiazE|V)ryuP;2VTkB!$s$ z$-y$RTmOa%Dj(8_afeyWOC6f-kk9(<4x#bWFmhWslB#nyGDn}ucvMixPy1a&>->9M zM|By(_ub+jHxW7dHQ*xQ zX^)uRl_Dz9Tf<*Vti~825h^!YM<%6Dxo7uY`hM1p=ef6%|2Sti>uo7G`bT-MOs|^t}?dhN0P$2cla6h72?W9=s2d%K6}g2uY(qJ z~x_;gMB2_P5WW zS=qPPoXiLqnTz4%^&GgDTJv#6bLi#e4bWRSkxKSIVJb625b^#wY$u68r~e&qIr)ut zCIym^pg9gtb70?pC{cJ-2U*3>qu-l^XiCyzS}>@I+Al{Dt8|D7KiW=>>r_$ja~tx) z-DrjMN*cTDIEwd4*GnGXh{7soj*AXTngfhovtjJfMO!ld8Z%I9>p8SsO z+G9adZPS=9-caB%2dY@pgtyD%*fT1n6T&rIwaN_3zp3)t@^eVl@TRYJd8oQm%^jO0 z=yHSwTlKP+=IQ!y=HEU=7 zFC@`GL0XyigWquJ!DxFsy8Yl6>CbED#^v{@V(%Hq&m4=@6E9-5_e#9dk7Q?kCPL3K zk$qCorWCxR8OML1^^X9h8;pX3)EeG%_7QHEe4w=vE9q3P3Xhm>1JAX4_?AtJNV+MR zFY?Tx@Xgt@#;pR6J$jjM*-W~)a1NQTd`kl3O!>g2So%}q#I@HO)8``&xWD}mNgtZQ zZdPxgSu~l;MaR<39k-ay`Bn5JRs?(db&wSe7sXo!T$T3 zXhzFqO01lO#rHheG#?47f3g@Bt1jVz!fq0KQcw8>0sMEX9K5y8;`xG7`s=-lPrK7e zQ>ydfU~_;@H1(nJYaPkHFJsBcWfZLx!|JT|tSel ztj;$4wjjlK`jnKEO4n3%;jUbYCAZ4>zp?_nb-T@H=Wjw1=h2uykW z7-zG+(DLmi{|bB3SauCMrY+?5_aYnkdV=yCW|OhL9_5Q^vps*iXu`M>-lqEqW_d;Q zDe>44uM}ZH^GVR=hp>OFfbJD{$-XfH1<8j==K4drr@V=8`EZMVg|A1^paZ(q6cK2> zj1~upVU?W)T2AjJhtx$>?)8DX%FH2SRz+#&?8wTtiaYn$P-5{2=v=&sHHmY0yMrOM zt-Hjo2lOGdr3pUMW>I9cE1cEd!mE+7CDPs4+wha$u`3|sORGp#MhN|D-?MLzbm>x` z0Gd;VQT6P0xHPzvg-HPW>uQcQ9lh+J$#ICj3F7zW{z22$URrtl7ZvJj(x|bcDSUe@ zs@~oqpQnsa0zA6AmjZlka3-gl(sm}`TZ9?{hkKBA{1&PbG#|>o;q)eM5OYQpF;|f` z`s0+&AJ(10tYlaAz{8A^V(#IW{xeK&b;B3DrG@fR@Z1qZCswSWv$el*$)lXx9}B0( z8E?t2CU@w2GmJY|Oop?walpWv{^KVRo|Hu%XUA~Qr>Qh%R~|w~DBA-%M4 z@Vs=gcd93=sAdfAn8~%S$muV+cl_hwN`#vPPc9HJFh z_L1%I5Tu7~(?`%D?i>mJ&bU|0Bl zyGg@-_|en3Ic(>B1F9dE&VCFzaj@S#YIT&tO5I8}Yr6;Bw<+-CC(lXATLFhQj-nQo zx9t02dyLTALTzt6@KZpUxm*y${wbUJ+V7L-e3KA6Zlggy7u{Ig6;+6@-p%V&1kvUp z#xG`s(JjMt@=cq7**3OxX@(HiML5#t*RAyAdNhmnupoLL&2)v6D1NLqzcPCftLo;9@brX)Dz~6gtljtJ1w6Yjb_qp?Fnez-HdPxLF(xWW|g-( zsbl9pb}g@<}y~>L1g@n zrC!^Eg$JG?^Pn9y{rbp{yX2tg?n8==T8+ZJA*{`RExtC}(Vn4fUs^9k)Wl&Nmc(<~ z8|iO<1+u=oWBI^#%9R?6hnB^7wdFLH*2vMnIw?v~e#1Kp%qX^OJ73^kgSq<)xK6q* z=BG(8`?MKIc-+FBw@rgc%_z+KGKhap#&8+`juI!$Vo!3W(f5hL{C18D*|cgf?+Pbu zdAgTIl}^Tks`0!zv;_feM{$01K8-baz=G3?NW?ykD_4BN+M6$t^z{NHoz#$%Gnxeb z`f#zq8^!oY5?!y*UV4a6{OJVACTYkSW|N%KYt~`*iKg&*e0KdhWOm14`pNM$OwAK# z{3Osl-cO$7E{LX4LP0k zGUo)h7{3|$v#vlbrwr&y#?FjixN3h8GyS%rE-4h%xKt_G6M( z7G2*Sjd?=d79(fRERiTAjo41t64jaby@!;Vvx$zlpQ4MZ`}xNq|9IUooG^1Q zX|``;E5o;;vRa?qgjdq0=*PUM)d(|HVwBt8#l(APJ^`MH<16n(FT4f`U2)WffaJl28~-5DOPtL!E2Gf* z4Ybi$3bilhVwZC(DcQO+`6K=0tJK9t^~KYfeJ6O}%>=04$>%1ufmC356@gE0(uhNT za6BT0wsFhY@|nx%^W8!G?3sy9)lzn1x7q<4scprbk}I#yyLW{A@JE zY}rhj##!i_b%^id??^p(4lS9Nif8XR6WHhv|6~EK*PTfRg*|!s!Z&1YPzN*99y(p- z$XC|R!#;m0%26@i}9xp@N(BS3h4RG z$=9En&Mc(s(St*?JIj@0)9Bo)jkGCkB<5;ZV03I0$qxC-ob+r2PnpBw%~ar)@`D6D zj?su8`SeIvm2%J7vg~d@IxHc<)>@gsWgvpSYhFe|a0&N*(Mx_yohhtE3!NVhKxNTn zTI#ii{@j{O@+ZfTSz!(hxOY)BAuvqF%~LlnO^WC<6Sp|9x$+vTx_*6rTTyzY;} zZVxR~7d4S>-y3WeY@rA1%h2&D8YTkMx$7EX@>%{B#)A*=u+52-m!6=`_uYJH&S$*- zVZ(O##?tqqT+A0R!t^6MxqM?XTpmoLFV}s^C94QS@<_*KKH@Qj7m@8INt)@27}r(I zPU<#8dgintrZdEJwHWredp;Jf31$&-ugGS1IDau%LW!ck&_1RJ9bsMArY%n$Q+M*w z<`LAm@gu+PWr$LOm#*v#B8`-Fq_!`fjBEGejjTOf7B0Z3sbctkJs)9ZS8z&i z9?or^j=V003T1yXPs~Jf*dE&cqzR`DpRkVi-ejOSg2la1!C#TBbbgIH76r@GlHC&Y zFvf{_ezS&Qr8PgSV?rAOdZ^Cr4hiP@(rcX#d}%f3)7wQU+BuN!xz5CO#aPz9bq`Dz zJ)=p(R$%-YHAJ7ZqZtS1vN+2jF0K&A>dOmp%4sDF-4j7=g;yY65kY<%8rUc88rl~6 z6h35NVfrkCBBfQSK(m!6@p4~r?%bQG1RTa###2e&nDLiMtPbg_RJZ4IpE zGk)n(#-~(v(I^ahv)cHnC5th_pp&O%Jb;VpWGrcT3XdHNNwq$bgg@1>kM;T_Fvo+= z9#6vWiUM5Jen`PqQ*p8Q0shMAA!w5;A__9Wj`z_9i6am+Ifr<)9y${Cg-SYe>B!>@ zC@<{Ags&~|6tU*x2B*=_WN*GVPzDK}UF^&_X^M`zf!D8W@oSPa*#v1pew7wnieJ(X z+(67vDdZjejN^`%=!|9>=^P$U5ev0QQ#B9aZ>Dm)({YrzH=pl|%E6Y^UHsL7Ez~y{ z$nF(wBf~AK7`aE6CJp{$MN7qLeeFIzv+qCrIHHP8{QE{q&nuGSEn;7N4N?OGh&cWoHPVR%K8BiJ?&>mzK}p4WU<-e4gA8FI`?j zOE;{gd<{L;zT6X%^|h!wWkF->i}#hrkzOe*q zbL+`c*?^Bq`ixqh!{=Uap+Sp1OhBO-lBYLNsp3yemY>P3N{`Xf&@?`*zYJMAQ<(Ck z4-|j!6?YMF!{vh^{Dg)*jT^cJ2je(g`FM&C=ia0eC&jg|&74pVshq9zUt- zLk7>x?4W_f>-?Fz7|!2)z%qw?zI^>XE;%v>A(gxNmclV);Gs)mGcrLVcd)6i$DpZo zI5ecnae7oWTQxrpt3Pcfm&e%{%(10}zC^OP`jDBkeUxOej<4QfKx>wIv7Wihu;a%u z))Tgq6g5iO!kw*%NQ&krUDM$>lkhL?G3~!Mp2h2UVb}K`Fk80-i`Tkw_va399hQRH zu5x(Kn9Cw(%|=|X6RF&EB=>vYu&zXq*rYs~{jCz4WZ#mCxjp4eNU^xK=LnKNLT%4O zVJ4f-bvNB6IT=ajePji-N6K)G!f3n@JH(a8zk)>lR66Q>9)~8T;pMfnsPoguz2|6HjxAM_Wi)IfjKP{le{iwJ#)`v6o_Al=^2eE2FIxiJv;H2>dpTmVeN}Q|JHaV0Q^EON@k()MYYQJ%b5A+T>%P;F7xiG?D&KH&V-UQXFVLj5Ss31MYucJ-cIXIWE2@s$%^F$Ubu7|w(zij`$ z`*ePxGW(D$L6QG-C?cc_e|=4$@m~{^pVxA!Bvm?9?#4@GYDxHaDzP!PbotqDex=x! zb`9q8^^1ogCDE3)2M^HPF_*zk#n6FaU(ofAASyGJD?}MUJG7Subnl^H2ImG2f%H5| zm>L)8(&2eQ?8LBWEWewGNq_gl#L9&Ft2bi%)&ef;sSVL(E{Ib9L2Zd6c+!)9gk|2W z_Hi?8|9;^?GWO`7s!u8q#cs1m65BkTsw$Vk(c&0%EN=0U$I>92Daihu>ZOWF%Iv>( z&~Niq+`9e@(ntTJkU-sZ#mttDzgYau=}fxU33gv5aW_BPoC+tqdH(ADn{ zT$;v}mJD%#+h7@j?Y-C(>~!Ig6$UPNRD}4WYU*jE+px zKKq&-@0Y=9gEp#enn1IP?r`gM?UXB4&-dm%Cz-R` zsqLyZ?XfX|!)J!l)>=eZxY8P@Xs)}J(S)EG-1hA$y4$>q-RO;l#jk@bwtfl~GA&AY zy%8Rb!d%kL6|Q4L=>EJg6~`CoOpq#PhJ@3?Z&nhsA12Sa;`P{0x87TaF6eS$a|y8lzMg{`u9eJuPUYc zhmx_%AOqcNrz7-o336XVQIAFk##{8V)14~x+*FoF&GdkM!48Ea#)@r1Fdyl_tPX|JZ~bQeXG{Rz9x-Rq zF5j?9S&PIpVv&A+5l{K|3~zrgVhbI5;Xcub#qdVhEtTXeT70RiT7xN7Z-R5^FTOo> z7d<*Q7Jg-EuvGV=+vi`CdqFXevkAuR^Py~|&MX{OPv$*uZ0XMTQ5dP_2GMDEaJ1tx z1wML-7spNMt{Eq@jW+$g$*JbT1FH1bVm0?q(Q8vpK6%CfEq|*{<)t<- zy1flSvouie_=QPN`AMJhpR@16&8Vu1X7N{QkU#eZ<(J%~y$5cP;Pxy!q_Ym`*TPZu zMUTyn&ZLY>YOFdTjcx?q!MAxi7hj_-$}@hQHJU)q0=ZiIJ{#Pr+nw0353qU$htxtc|&4~M8@6klcbhY}q+ zF(%KQ{`~9Y%RI_a^XwgSyD3bsNAAF5l{6d}?L#e+r>SDaNz8FQOexlC2sN{Vo@z56 zeaVHcu9}1xlUJC3x)tqGR&b6if%T(1)KV*x|*-a@GJxr5QQd9>`?s7tr^8R?u2J2Zz$flhTia2&Za(IVKzZ<4=&z98X$WJ;2whrbEN* zIBzz5Mt)w$Xrf3K%~?1!?+|I+?HkLa&rU_hie$duVIkTdCzDA}EIm^B#ekz&baC7DUpUk{}X#P=!0wP57Y91Gv^jQ9<1{S~((VDz~yR;f1xqDg2)#22B?lq2HIz=YoxopIW zIQ+R5!`s(ulK7JOT&{K-zCRqtUlo@F^1W>8+Ar|iHn z8{|Io@1Gl~_^lU<^01+F_f{zF`vToj3-}5D<4CO6Wp}S+qq*)g`|)ul(gj^ewVdMI{usxqhVgBFg$uoaA8+uas zu(OD~sKK8f>_Nx=HB4lKFdgx1f%Q-Z>)fM zo(@~&Q$U5Ywc*)rPpLQCDd6x1j21KHFRsj^@NHT!l}f|Ls7*)-K7x?gk$ggF9WJMJ zA<{hq#+UE1OF}M`VcbG>3*BjoxG7{jkJ0Vnq3q^3Z;E&^6;a#$F;ZqK-H;mqpWDU% zz0Zc$-Ou#;E5pnze|GrMXeu~#jAzIeBzmQq-ZTg?+!Qfmh0o7l`)QWIFF=ZqlMU{t@~h~_Y$K* zX2Hw1k$Ys$B4_K9Eas#*3|G#<=&p8psG*IMUOP!`dN&KMD;VP5ROWlWkOJfuP?$~w znXS1@FB7w17&ebvoccmCPqcY`*#lbhu96LRNu_Yj98$#>?7p&^0vaOd!Dk_6`N^F2 zPq@v7_pitA|L$=0EhFe(stZ=%At;S{O?Q3jkg?c@SB|wt?zB=Q#Qmg+UmnoK4;lD% z%Zg&o{Gustp|tC_8gy1pVG^k?aAf2MrnT1y>0dANfwi)5$zG1LmKyYOTMScLq=DDk z2l(8!ep)kpI(bF3)1K*{;hwdKSnLm`dpnWFyief(OF*>0VugS3*P$B)o?5Vfx}B>X zXva+VZeFlp2A;Wi`l$Fkx*Q3+au>74~Mw zF{-i_Vy6>35MSuc19NWC_pls#w|h1^7R4~)T@ZVj%b(bT_Wpgxcmj+XbPXrH%Tu4!FEo`7N614&f*rs{{skfaFU>ru8t{W*prh~$j zmQ!1jJT$|0@hb|;D7nLrZPIv&P^)?Dos2N8@_~*XBQ5EdIvN-_?fO?@+#R{wM4&*I;?Y zt`PjSmX-{7VfpSb^7A%?!It^_VDlA{*nSR`LBp}~hZ}`4A zcSM%&c*tV5c_hw@yabmVB7xIw2!83Uju{;CJc-%KrGC$uAZ_ci>qx zHN2*TA{l=0-X}70m`Msff+T7@66rX_W1M2iFnb11?++x6tUD-mDnzehBkfK%r8nv?xay&OG~TxDm-1)Hx9 zamJh36ihbE>EJplKemr{jaNpF&`w&ZW=VF(ui#X|SC}0+Pj*phNLtrM1!oiK#=_nB zdS9J7{tEIa=RuO|IL*}N#-i4C2J_Cm49C*N++*8yl1dze)7=G>wPP~ZJv{)=g~#b~ z_+3gF_lpl~yowFEf-IrHm}KSWl3ht0BpylQM9O)*t{jG_&*jii+rzgjWs!b*9Z#A% z0E6qB`2o2UY|yWy4I9tWC))~2GR+^l!(Wu>Voxy{k8$FG8%zc*xKHOAn$~lUpT4nc zh$}_d`(h^?uXE$-W4aL=a1ow{1tehqk(+%W8u&Ah?Yn%C_NyGDVv{%IRaC_%ZR#U| z9$#9w;gloha%3l(|<_HMTrKgM; zu2#U@>lsYZX#|4bg_7jG-B>ZjgxXXuQOfat_A2KI#h9OB>a#lVgo;_*9T}YMRw7wd zdlKw=#y?*>Oi4*aY^By}%&J>}(i;mY+S&r+9mhdx)H;6RSTD`#+J%)5#?$CBDezQx z{Al?=9{Hxk%8Mx5+lVY|ONa7N3F~qs$WrnLMc>GR?EU~0=?>?z3qs&CwwsP^YN0;A z3d%@!hFSH0e8ulzs5+{`T+9$ZWe>0$HXCR}&ImfTNEUyWOR=SGx9Pg@JGM9}0P~Lx zgMrgm*k7N8bH*cSu4FrZe=C6Ww>{$jf*(_`UMAb4e~enLd?i1V?^GhtMv?Zi6lL=P z_iu{evBPK*UgAM1{`YybhXEXD>1N1;*rdZ!Q!x z*3s1`5!hO0jynQEG<@}EdN~*h!}Jo^j6Tc`*(T7yOR#Gtnvh%FM+*W3X!O#(%*L2{3GyhJaIY<~^7l^}^e(;K0VYFk%XqGIS3nb06(`T1$6Ue8i1(Ssu9a6kQ(j zP){Xme6#q(<~v=dl?S`&Qr~29+EoO}kCj;LDMRPjKHfCLm<*NPQ~hFL zJg@Q`^4+0(xM#<_El$(E#j-rveLA`RS%vD~aws<4MaQcLsH$Nk*{S4__JY0GvA-SF z#oK627GX<-G}c$ekihdkKC1aIW%~6%^3O;zel?kVnG@u$PREz6-|6w~VjecKc8K#- zkP{nBEus(j?m%_gDKwe>7Q_s_$1tw;=sX>cyv{azYm&a(A}+9CHYUyXhyGQhRMHO#ZcnD3^-IU9RLC#H+@yos> zvnYW@``gf)7D1TZ9fRDwWqd4Oii6Gv_^(}~@WoSvUz?H(sdO2tThm8}C(LEO3*;z5 zpov_gr0M6PHF)Legj2hHQPnF&ZGq>o?AlJ+8~qp`58WYYIX9NS(h$>*d|+!PU#E=B-^**Le)5EZ>^*;Mi2S%=ELOtGKgI7>yqB-`=}4KgX@Of zyd*#o)}zySv8^u&+s=e#ZY&l32*KjgAUfGQ0Cr0d0>9D`IC2oO1>)S~RUNi^sq+b$ zQaIOGPQO}zQ`5?6EGg5EzQ0nZGX{znOsZnjx|gHN@hj%YjHb`~jcAzOIFj4*1S1cr zlA(bRmaJGtS7aUdx*wcADGgBIq}OosD5EKZ?KJH8QEpcIj-tjt!RF32IGAC?6o1@A z#m92)cq0-ewl0WYy?~w%SI6qaF7gz+!h^PJ(+&4fbh#u-iGzAEikB)I)aGF<86Tj;RmR zq%A2IiMx4F_wX&Wzkd-8UfMwVYMCod<+Dm^V!sY#hAN#E5)we zN%;!kInE8VOuQ6L?`=rDLlR@YE~dj{Qboe;)=0J<%7UE=QAckKx z<;T}tC5tV=Xx0qID5X0v81mi&nZK}c=?N^a|I1%LT8fA%Mx>OkNt-MF@6T)K_>6C) zu4_pBcOyAZl|yT^1v5|ji@cR4yv}Yec1$hD;_UsT*g6Wgl75kx{VCp=+sKwHg)kHzIl=?ReSkph6;dAfPLcgb_?`>F^gy5pQx?Uc zDtr|C=&eNg&HrfrG7~a?V#S;XhH`a}IZ2kBg+z$}o1m&p85a)na>Y-OlfJ=jmMhS5 z@$LM<+fmeW=oDY}TowO~GiC+>(HI>gON|Z<_#tdYlhi9PqHY{N-K~W_r~jz;;S;KO zvXVEA4+K4b!A~X3qa{o1Na)sEdOhSyrfYoYrcVHp1t(E(x&|NJ5Jj%id6&eX?FT05_su^;E=yCv9+f;8+@qyq6W7JZ=r=2^4uh(n)HnXX!^WAbX{~m zH{UByhlJixhHDvp{&}jJY2XQVF>p*60HGEm?K{$&3WTU@IV*0}ojGlDi!(|ON zOJ^0uwK>8)<0yR(Ea93*)luzL$_$F5Nmu+ltD3csjuu7ZVfDNK@n+=E^+c-zvPowwJQ}_Iz5H6wQMazu;MwI%jp`a9T5k zZTwb(mJ(^GsH;+p^A}1+tfmV5NL3RDu?xQ7DEqYx-Dn`BoSQe`YI!GhtD_n~k4z z2VXL0;?dy%Y7ubayB#bru=+Nhir0}iFQQKtjkpw&&-UFG!*APOo_u5+T^roVG>0vN z=iB>uxIF|)hs4>5Cw(|~?q8mIk;Hyw%y-gpl!}p^o=Fy#I+>qvI4b7VtA^q3@yLg z3&pjL3l!47sYyMIPq{5ZNclQX0?1`ct=*sPma@8m7q z_--lYE-VZK4*?Wxdzu+NIvaYt`p9N1zK+dmm1ZWt-Um)j`=LGFhsw1J156CThY=T; zT8V8?cjFv(b2DR+g8SLG>w|F5%7&Poo{oL@OlC%`sS0e=cjk&)aadQ;6@RH`V6QQQ z;bhw{kUWWFYK0YoIRkG)R^)Kl-%-yFYSIpaoEu=;#`(}NeJ{BDstrXvt72e42kd&IC);q^c<^$hnri$I)|@NbzLCTTU`YPb~Lhs_D{y0YmYG3`}o0* z;n(2H0x$5|+?GA&)Ea#~mNDbzo<#qry}|2ZOZ*Ot(W&`17&Xxy+YS2wF}qh|dYdBv zj@_V&WgXml?1i5`y}*gEBMq%dAjTYyHc|q4}bek@kI?7*LfTpzG4@Q zPa4Ti>ee1#Z(qT_>t@EpYxmjYX|G`Sq?u^&OTn0x%h~<9@_4hvX4a+Xe&{gi5}R^& z7BtRXPdr$UB|B!~FGVoUz2M8f&1LaRu^F(S#dVy2Hj(-Iv=TI&_l;fUw;f-5K8GpE zb;C-pWEovJ(|_8dI3{iTd?j=hQgRJcQNRr8Gbap z$-dTBhsGHtnMbwM5Smz)NuApcs?NX2T&iCM-#+-neA`eIYeQA$zuaHA(xWi*{q`5Q z-G4m%YE%fbmt>>4-7IW*-pD-kT8Wol0$%ZZh>1gHv3-v32h$>Nrr<0klw3@CYx~wP zuAmOL#CzjFO9=aN&<@x;Y9c#z$TQgH-yRzs--XNm8G;_2MnRblg_vnGEjYwyHk|L! z4W{;~!fea<0xzyLgwH23(9cp9-rZXQS3W&u!&aBVF1d|aQ`fc_Iy4uJcZ)(jeFOIM zndNxJs|>TZ8pV@+0-m;jm0OJ z*?ULY;<5In*&%~#Vn6v4HtW$m+*iRF`qvqNr!OVK;p>ZVz?gQ-k*hZ`^3E6hF>nh$ zx!R3=F!m!Z8`KPn6?B9*m4onA^9eX2Z4z^G#T{r!xnE}YvheSvEVhT*6$jtg!v1GI zf@6kVWK$gHU=IJSM9WR^%SFZvjqQ#DeV($_M|H;?q08W@S2A8mcZTvl4&Yw=DkFdC z0ee(qut)kmsMOYjd68NiO9W-XzXSHc%jvF6?-eWIZa@e-YUyq;K99n?3libHsyAL= zb{#zLEyG>=r($u37Od0h6d1H`9+vQ{4HZYk;p(ft*zU(An47c$A8eY#%uz+c)1X+U z+xR9>U3(OBHjG8bZsnQCBVF(*_Jugg=Pi%JAmzXW^jol$nO?|%D-N~Cm~CSrY{6}8 z^6EKS%2!~YtZ$D+hJI$6*E|3>H-Bd|jt}7A0}XEPorSVzOEJN_5g3aNW%um~fXeFz zvE#~BgSv$iv8-o#d{blt+si)<22Fi~^E(elZ?hIoZ!d|B3PiBcePd|#DPj6C<(Ltg%dQ^BOQ8jbc7E>5Ct(ZbGMR7IB-raK)uOAzV!P$=NWW^cSbhbJ3<$FWy+OjwP$XS3>uPk5}e(*uBZEdj3 zu>&}|dOtjmzu-lW2JAAgr!fD~E%2;T9=_hHh>G;)s2hF=6o>Ah*1ZJAyj8)8ZC_aK z(j|1Po{8a246yz{6E-En2|6vxg~#Xopl7?;vpJ2YBmp6i*La!0Z{>8t*ha3qxx?!TU}}*kW_uVEohTOk`gpSe*SC`IYXN z@o#%P+QALxop)xOE_uQA&A!aVRZsD1X)`kolD{s-|pH!#i} zwD@wwTlPuXZaf-$7fw~J2j`9zg);YIaoxN_@afzrXs|t(!PQM6X21gQ;XJTpa(_q) zKMfBWrQ?k!O89ZU666+Ni|>mKW{S@|huh1_*_gg(F|yxP3`uK%?7dRV{c^!r^w$-} z-SIT44$orF8?R#S=TGeD&tY)yQ*&q)9|o5{w1J`%?n9$31+mSN?pUbG1m?-6Abd2# zl`YYD0X*y!#0+@<4Yg5iu<6Zoj9T)ONxA$HF4vmP#JYEaowi@PZ=T3O-dRz8K@-9r>>FJbKCLc?uQr}yJts}a{agFP z-RCo)aG$Ptd`tm&Jw*pzh3aCj%l|M<&i;S~SJe>P9#H0S0^*03#pjb{c*LUyZjo2S zaTi*nhfe~mAKo30r0N)D?iB>aceuvoR3-fhdb4(7I2Ci`xv3i%1SflK0P(B*W zG@jZKrmdWTgR9JjO^!p+abZbl)%h`-Pe{qzlUF) zinC37121SY1)d+gZE0Y9h`F{nWqoNaE0 zIe}l9*Mpzp<%zYhpSCBq>DCFSohgUCW@h5Ut~=mHV>y!;JsG#74j?N7V=F&n&t%NN zg)0DBC#=STl}a&%`+S1ZyJoVJp3ZYlzms&U3(a|NbcU(Day)O#ZeJIPaFN3i@HwPcA{RyvzjmOa~cj9nk z0`twh4q9bz#P?U4W0%0i?Afu!u<+Q)aAwR+Y3oI-eVO)RJv^@anr8#5IQmtpY8{ty-xip8dA*s;TQ;JeatX3VR) zaA9Ez9-gH@^Bg4{9MuY3Ucbb_ySgCbI)UA(ehhwx+u&t&JJcsX!ZQw!;dYOb3LF$O)|c<7RxYu`dSX#Nqyvub9&34?=q3n%HdqeWaVoq^{|CEm zO~K{kopHEoG~8=m2fxKR!V!v5h9;fG)n&s#t{RA)*4Je}b$U8|%nv7R9e`6}(_q5HhtR&{Oz{1tgp@7?;6&G1 zP^Hvoly5%^CyulP-^2Hq6_zikP^j2l&qu+sJDsran_L)K?f?YT*$TPtuFN`BHqIKH zi5nJ=h3oxKF>6NYu-fe^Y)z&$GE?7!(~GOXuAYUiS3ltLMoXFh($0c<%w|x%?+4So zBbcuq#h_g86`)e-u~(RdS=)LVRy$*1C)RVolDp#*~A|Iyo?~@f%os`2sY|io*pnYr&!T9ay77A$In`V2phn&UOycLWN5&*(L2= z;7F4+SW@l~__+qbmJa*jTlyomeU2;Hr8Dd^<44%mwKnz&J_UvUy9;S$2VrVvJg6_# zgmWDSjgGwC65yi_vrhX1wX?K|2?wgK5KXuF~;~ePSC4}v1%))NFE}#MiL?&<^Nu|1yqRn2Yz<2g0IfQ*iV9vnZe4603GD1JlYcWs>)vht}nX zLbddQP-o|GHh92eY_t3bJLU4fm_4r)>#7_ER~B7o(IXhU>(dzdq>k9E(-U^x^rzTq zJF;t?MuM?c7P|FM#YVTMu(>-kq2Rh(OmP3XnCQC*4ONq1)Yoi$dBX#+(sXo5*T9#Y z@mL~fAKspFgbgoU9KU$KVu}?h2^BB)V=ndd!@22ctYwr8%I)ukwHCL7qw~KoFM1wZa>$@F@z z$F!jLPb{;d> z!xINRb79`Lx&rMM-b9xFJI2~JjNa!lynFD3@p8R`efG6oGF1!T4w+nE`fKsf( zqjey22khT}8NQFW&z@?T3zHs1;peq}5E=iG8O#pJYvCW0CCY%}hw`HU(5s2bk-euQ$la6C#ecVd@4(C2k~=26|p@ZdDZ z9Cf}2%gWz|erivgvd|TlZ;yn+cLy@_&pm_v14pqJi%h^W0V#OO%@dzgX#g+6wqS@~ z8ndFL8P8%-_NaUVdS3p(EI;2K_J;dH4c$C=IxPTqMCQQGDKnT5^=O=OGM!yNYcEWE za*|oRbrNKa`o=u1rUcblB~+-otiE*w481iI%b&W8_0~{KG(U}v8(SK}dmdxob;$ud zIGyomF$f&bJb=Bue_&ifX=Xx36uY)f+Ze zxeIf*Br&%pdclW+p3I3xI+)%08feOmgrBk1@x=uV)<53@9me#6Vci}wE8m@jJExM^ zV?}4abRcxvFrKY{brf!x zFqWOK&w$Xowb1!Y84Rk?8((!>kNWX_*wl3qV1Bfj>6V=W+~?x#(_|O;5IUT>KJ*d> zTW;Xx36pTjRwLV}QW3aEw)y;eEza9Alil%%#U=Cm;FFxmFyzD@_IeKvLtlh2mI`Zd zSEC|$wd6N6m|KD>AOYI-qr7CyIgBlCVjL^&hBr=2A@KBU*tetuyRBp(T;DnwdNqkh zcHcKxdEq~FThoR~zxV++DZJoE(E+ey-5Kn^dj`I2wH6v2-G|NN`hizXEhx)pu>Tft zz?pCRvpMIVf?j?UX3e~eB_iF>yRHM2N>Z@p*QjCLs~G5LsRyi|3)ES(5s&q_!E9HV zvDMBXMmaSVopYD59r_=EqrI1~`%3PF5pOvzXuk7t8J!Eun< zEfX)Xg<#yn#ki?cC44eG26q*^gFj*)L+#l!u)`$sbEjGM-t25FIAIbjP^Vzu#hGmO z`!(p8Gk~4a@L#-?c#j=fSq-}zOu~0zPSEPqFZS`ObSURnit!yeA1i0yV0Bx6V697g zvEGXj5Id_9BxUS@(_iW^p0N}0qWeTvd43#zA6t&Skx&{Bj~NVcor}Pxf@v7+&yr=`;;*w0nfZD)xm{ z-FvYi=OXdzXkTRR9|3pmd6+dj8qb$&h{so}vBAYBu&35tyniT|U9sghR=-#RhP*wD z4{OfGGU~dx@qQL$WbcQY2YzDRGvp`Qe8tc?tDsEATUezwVtmObZ0r6kY!7}6_l`Hm z>9LF1&F}t&wYzTN{oqyTJL)~9A{!L zx}x#QGRSmu!cUDxL#vx9xafLq=3;sXD$E&pmPNRj*b+zUhT)yL$TpoA37h7fV|SEq zjJMbe_-N)HG%xmI>zY$hGu+IyDKQgu$&4GDsy14`jULAsD~98@upkJ7{h! z#yqZH5dN7wnu*cXhOvQ4=0p__^f)w(Vxq;EG_@;p#wFp{ z@KbEf`$YgMy<_htU&fCOuQ9poHi-KkiDxRv@RZL-_QDZ2jISvNuWKi;*5onFTMgNf zC#h_`MyEhGusjyHw-*bhKZJl58)4W#kJxXf<9KIzENRFI3&(q$g5-o=%+o=M_-@Zy=D(N!fpPRprp}!#?6vhCJ8az>NP0IN zJb8#mW-oKw2dVtpVx(l0iV>QHG&%y<(YQTm@b8*YVBIqy= zz$L0Joayij12SvEmv`G4vJ8;s&FCsZ6%%%3$m_bpXCOB*(R~ z{cIXz#uzHtQ$ znP)PFii2?B!r6?%m2{_YAf_gDft;L~I4HXfj4wNp9dfi1lo_LEwm2!^#*&rrG5rYk zs67pJ>rdmsf?t`2Eh^)ZMyJ^NzJ+jX;#p=urz;Sf5`{hfodyT1Rl+q*3*zQ%FV@X` z5?1wH2Sp0B26b*l95k*CRG+<`ojj)}eobn?G)pdkN1mrM@w3f%s`Ff&?^GTldX!*# z&H>#0XeaD<+<vSkGfQTAa5 z%&revbt9rv^m1 zpTfmnhe5adE#p@@6>`%W;h9dJFtTR`Q{iAV%>K~|9i~o0&stTX#*o@r=odM;F#^h6ssN`~x57`dYnWQ=F%CZo%={Hq@Zje%(07*uMoxIcY974DFX>@$ zC4Mn1cgkkVEGP*#qR(UdUQO{{^=#J9%b#@R0{c1qJf^OE4GVJS;M$*EnfBF-L8|sK z+jp`T77w@tE3O{LPp7Wp9n~WA%iP8^d_wbBHVK+eOoo3Nc*2v(LEvv`!VKzA9%qKU z#CChW;))&pvEbJLjO^YXuii<7ejS%!LU3O&^hk##2Ma^>(_7(6v2~bG`w8o{^$v8X zzmWY{v@Yzvun`ND0sIw|!A$fRgDolyWJcER58vePnE1p5Jo#lgv-HJPII;0POh^re z|HkgY-s^t>lupB17lvY?I%8qVS2Ly@I}1*hZdf>X9V9*-1sOP;DZYOOK6&;AFNgKT zZvz)GZ*TvLr(zy}!;SzL{3!$nMsLL65676`lC`jh_fw|vt&6burUQfR8o<`zt;|B* zF-%NZ1uwQgg!SIZj6NV5XB}rS z#~p0Y|2xEu^2cL!i?X#JRm9qF%Hgb`o;Z8vXDIx99S;6jfI0f1HmI}Sq06S3(9dry ztghM}Dzu1WW%YZ&x0Omzx19m#{}Yy$?*pxNw_;BgI1F_!%kfg|cx;}$nR$>l0=Iqq z1d2h;@JE*m%#WdNFlhkGHgUZMi6tvTll$#q!<2AvIG+VMW=|&bYXn?2zhK>O--Xv1 zQ`wdEy)k1tm>W;9yqkXs5OOPrpE(xaSOx7j1!x@zbs7M_Jp~vmcvWG zEWA9fJ|s_fXObS(#VR2+*}&-^u+u(0>$S(P%CZ@8&XRR&?1;=kueSTsP5w1I^Iz4b2 zWia@I2ZUap!7gwvi944*z(wT?g0t=*HaL6_WK-%f2j>Rj?V}}e`owwgeb7o~+Wpt? zrd&_xwPiOx>f4x!SoI1YZ+pP%|1Av%dXC1?r#nMKufup@wHr2PTd}^gQXzKNFSefU z82k(i!`zW8;FKyEM_dhqF3Mw!i`PRK^zT&W#OLAQS8^S@W7blXKRt(w9yY+wmCK`Q zPssnbzyJIH>+kUX|N48l|G)k&ZKIAWn3ebEVCU>>a6fez4pq)!wtPs02M^2R75^7_ zb*&$}`N;+7lrW!7uuQ~Wqdb9GxeW`A?ZBuuw8H@f9^iF~)sC;NirujeR$Q-!hm8X8 z%3U>c#!rpKe@n9F~HUZa6oF;NDuhumZz zue}UsO8#UVb~@llCkO0PKLYxAy<{vQsgS;RG;>O~4~zDAg}ZKQVg37EOt+S6A*Osw zT>L%*n8D|nY73koen~^NO;}xst3QS5{e22%-kAoTkCtP^_qlA_fzCK<(G7O)>sq+z z`C0Z5WaFHrsc`R5Hz-`QJag{SJB4Rd*#fSR_4+CnOsx*ySBqiUE$7(BljlSB$y@Bt9sMBZ{u;Kg@(Lag_Fy}6 zju^epg&99c^?&{8cDsM?Ld3ENx2pe<%KX)UZteSb?o_R3=b_TyJ(qe0ILe*P6Xnhw z9X*`~O!Raf>*(b?ccPc`az}6HLleE7A3FLtznkde%sBZv7oX(oT-C|Xx&9`<35SCvca*(#Slu4n1iJ=K4|eTbBE)sn^bpsX zB{&B(X>(UzQ zj!yBc8|&;zNn$6`=)iC?nfGWxnEo7<^H~*w|l|$-tMlA zeB3?8`MB3=Lc-q*cNBR}^6>;2rvH1cDf z-5;d|x_@gF|T3AuzRp9#J$;u5cdQb=YDQF=RRGga9_GX;Xbp3(tXPY zrF#vBQ1|QWL){}AsoXnmP`LxhWn~iNvZ~-I^GooQMS_>CQ-YVQCwR-oC3wp+z(=+w z!AG_Od}YTHd}YtUPxdLnPgWTHWwplm%K|V!rW+F=>wryU1I9FwrC^|J<(NR(HVl%T z91|qFiNUhmF~PC|Oo*)9*brH5hLg$1aPg6HGEiL9?vY1hZ^F!5G=J2{E#{1)IsTCp42CEZAIjX+m?^ zlY%W|KPI%06)MzHRxz=qtYM*6vY^CPvbKd<%i+-qhLPp_4eJ-xOU@$z~**~{y45pOTjhrpF7 z^1uVmo`JVgJOjTvdj-0sdIg3Q^$zTu>K!<$s88U`RG+}*MSTOer1}OPDC!q@CDku5 z&e=ckOR9fh(P9CC#U}*>RxZ{g&~HkUz{p~Ofg`5`1|}5?3YL!A~3d?GVpn-GBB*XoNF^j&Lx-kC^{f!UrDO(h^D8vr z)@C;0Izb>eE+LR>Trr3XpBuzAs~F5pnH$Wls2IYXn;XJCs>pF)=5k!wN(!#=JOvk7 zNy)XHr{wxo3gsrv3*{D7QgLe%RNR(IYVPPfHTR-Y7*}L|7+1b>ILFNo=QdW3;2zG8 z;J#JXaAg)~IQJ@%T;zgCu0xe5Zs>w2ZfX@Rm$g949jFq`eOnOC6|bt}DlOD;nyPxP z|3W=?ysCk_wa~zQuG*AytY+lg7a2KKH51o-k%^0|X6B|YGIR5*#c($k#c)rmHRB+w z8RuHPIoBYoImcCR!FA1Q!Og7Rk~^5ylDk^H755~o75A%pYtChHYp!OEHe83rZMZ%) z+Hx6-+j19bwBxdq+i~w}SU6Z>;fmFa0AbH|so=kC_*z`b44fh+6Qk!!HD zBd2uh#PwdHX3wM8M7p`cnuAK3ou3ToVZrrYax^b^+b>}KB>&^w% z?!mQL)`NRkrzhvIq9<3mZXCC7RUG%GelM=v>Rz0>L2oW?b#Lx;gFalrHGMc&_r9F> zn!X(4*pCZe(~oQA-k+Pcra!mBeE@ehV*t0?eIR#b%|Pyn`ylSynn9eC$6zi!Z7|o+ zV+dzhJA~`$5zj4M8_(_X7|NYrJCyt3F^qFrH;k**a5(3?ZaCMi;RvqZx)I!!S|ho! z3rBJZ9;3J+S)(|Q%A>i9nWMRuRTH?F`3YQwYGXLhg=08bgR$I>%(0xY`Z&(CbR5^c z@_252)_BgV#ssd#k_p`A8j0Mzg^AqY21(qqB}rU`iW9lc|4ih%SDeIs%$&r<)J^7| z%ueRC4JUK=S5M|tHBvZsP6~ITaw<1>ek%8{$`o$rqAA?aT2r}!OQv!SYfa-47f<7Y z>Q3kC&6&<^t(eAbT%E=Za8KvfE==b}R?gshugKsSw;9~6wKKTlb#fJ1b-pMvYX4Mh zT=7$}xZW>C-8!r?vW}xNak-;1zP6Kc-*PAA+1iDb>z5T)zNuYA*=>0d<(_)Z%26wv zm9y&>Ri0f@RQarKN#%rk<&+IpmQzO8tE6o-suS2a+k)OS~w zSmmz#ubzjp-zpEKZ~cbKwyPT|Jsjk!z$CdU!ogG3EXh;V-N8#WEXhkX#=%=PImuhK zz`;kgImt(L#KBi}G09hT-@#AyJjqYRIQpwxCi<&tItHk6O3EY7rpY57mGq4GndTW$ zvy@!xnJ(9cmGabfN%z!}kJgee*7h&utxZVx*3K*Cqdk!Bqdi~BS9>SjSDRbPPYW4- z+G?f!wUHVA+K#0Iv?DSCv_A?q(ay+dqFr7(P`foFP8yxrhwDBRe~;I`4rAaCq# zh}h_DXw}%q(08McVNzpX!@P~YhRu!r42L)R8E!Q8H@x2HZzv`YFt}|BFa*k*7<8MO z7<$M94P!P18ZzZUh7Fs73_0>(!^KU(hDY)c!|-t-hR<@&;F!%BYIrIPjk6VoC{LxK zTei{=?-^>Cm>p`E<*70>9B{XUXg~mne)?R+^ZYyLAM*l&@rTn>D(n}ZloX=W<5fu^i(5vo?J_%=?iB^NUg@IPXS80yIdQIx4F?82&O&Xe9IITAyQDlWZ%_1!~AIKc{cI;pJ`G5b{b*@~Ga~{vhIrsg#U$6UIPtF-KhnSR4e;}a2>`9(_ zwzZOA+b0DNpWe2K`Ao+jHABr_`;e<{+%}87>66UU5CL`3dd>XOW^VRUX))YWz08!u zc`k;(EG=H9&_(Zq1jbh%1Gs#WYE+;koLT{Q{!DL50@Vb89iopKc7tF0-&Yi z%@-*i|KXk+k9ayG^1&6->onbvtYV)Cy0M7O@}9g~@Q1lt^6c}rl@uH}Q%9I>uenHcU`o97;KDGE2Y%O>l@o@evc9hc|+pqS%s(x|U$ zV4we}Yx~-4>a~JJ>7@^?%pYk;qRx=(+aLEBh426R8IqY&G95oxXIc@cGJm($Iz4%U zkt0}gj@zd^$#Zl#UXXI-6+(aBsv_}mY%FWNWerV9d96@lQ_e6XFj=@d;4BwUm2C9% zp~V8f2=JOPPSpXnmYBIokJt(t7HB+6(>xN}d?|4zy}Sn==qFdLZDLP^e43rcw}I!` z4L+K<+6B0FnZVPeO_g%_2W->7{FoSUE4O9vS@SLC%gptfdew&VSH%C(p5cySDFvk6 z(6qjKROLfe@Z}S-D#b(EvuyB|qLub*vrlfHjN1_xVDAXLq8_MnN)hvz$Gujs#uQAx zk^R%n>)VlDiZcJ)bXNrcBRV3ZcjPn|{5)x5_H$Q_qMP>kQN%UY!MeoeQV(dgx~cO*xp_dUaHK5Dcur?`%Bqaa%K{1k7()^^M1 z2h=t`Nl08od+@zXo*t1Juv&Wk-Fc5$!1Vi*A66f47|og1l&{2^O6LVG7GIUe%VY36 z?Rf+9tUj+3?L*|k_3hrLge=PM*A?B49{eahrDqs(OQn8-%~?l6;ll%&(oeU29Mbn@7z-u#t)Ia-sz_dk@B%U4Tbxug#n+>~$WJ zHKjzF;iO0XkB{Rb70l+;B=+vJ(EQH>kb&;E7iORAG&RJv(pVD8DtWDX`X3%|F{}Aj zYxN&kzGo@tso|md`PbCb@Zyfq{;PJgxnI)h&C$wCx%|+z^Xls#Zbc~N-S_0aw!Opr z)hUVYg{J4;J`A@U{e>WseWxk`S2to_^UK^6j2-FJei%T%+iL61epie2ohF!U{}^);u@{jSZdpC;NXk~}#+U^oU*;{v9q9M&&Wbh$hehBzKr2<&s0 z&^9ADj_yiclgSE2b2Tg6lGyKD6kc>aSY`k6H$q)gTS02>p6tjg>)XO63dga3zlnG^ z{uah!3`Wy}j6+gJFZ4+}t&G|p?WOTr4BXuJT($9Gelgi2*!F00)NQRd-PNBN_M$Kh zq_MOD(*Y*QWa$6r^q)N^miTtu6$`l^M|beSxMi8a^c#wdxPgP3?%%qk0<+4QZ&7i3 zbM)m+XLvQyPeWPwCw8XxCDoqY0$aU?g2@hL)=#I`4KkT;0_1nqKh`wRis|xCBdn#F zEU9jG+F50&WNXZ^QyEP)+gXRJ7J^zPxb+V-?m)6%k{E?QceTSk{jYphLc$64o<(_R z_pydbuX86*LM+j3xZw3$_#SbCBja4wiJ5120xt+DVhF83H{{7WY}PdRk&7|3IN>TF zxb4}j0XQ@ErlFj=thuF6=sk0WwWQYTiN7A|rxp@CS%0{0B>1Xz-aUvajfNNBeO1hV zYRhMC5j#|?+Jy-y(=&`VMzj}Y?KH^Uh!!|IH=~Zp6}TJi0?HK%{1oyBKJiv0;@aj- zKIo=#s?^x+`3sS;J2LVAr5YqNtUfZ-nCu_s=PojMnr}2BMYwh9Ak(odP7yo!qcBUK z>hRvnu6C^hcColmOvM@Uir3GT68is&EVAM%vrG~kE5epZHAQKr4oC{+zcJi2x(b)s zOsCD*0!f+i^BA({2Z?l7lxvxy6+i-NrX~1p?T%W*ry`I|Y18T@$bhGpi%u!u$@^vj zDz(Nq-%oR_MYG3QeJ@|!Z9^S3GS&;`h*f5@Grw=h{Yx#^@`nNiZ-(bZ-%aIp2X}p8 zJss$8k%Q3+NC|`A&={(e<q)m^s#;U$Ke?S}^CB7b zWe=0Uu?jV>R`aBa0uUT7!HUK}ACp!1Q=v6a!4=p`DXuSn4L5=%Z+VJ%i1cCv|EP2+ z0{V3ocBu3d^)xsdz}6XQS)&la7oSFFeV^I(@}woD@>wFHSGdS`b5Cji{0Vu{W)l61 zvC62hz)^_m)~Q|sL#2VuG3JNO^7FHKfw14V?CRaD&fPATd5kJ|bAw%5Eg#u!8KLg(4rmBD8^ z#tJQDSCJ#^_8&dw`hCD_^&q0 zcu#UVzlp28i0%&*gf>+7gTx@M(4er!2gB#qWz|s*H~7_h7oV=uxL^_wHGytZROnvD z4_QA9s`ox*=P1mboVx^m@+`9Tnn*~G)ZE#2F&~~{scB6ATnW<3ry2q=efJ2ynz^CX z%${L;Ush-F5z-s;WI8`M{VlEB=?2@+^zJW>>Jqy`T-)vjNha;M6}YPg6oUCKx6gd&x|tH>by3^C2IPAC z;F+O=cTx?QZj`6sgT(RGEpM4Ko(tv}%fW#^5}JZuP*|D7U}8y!v$zyv=YlR9d!oqm z{A|v+hQk4(T zv~d%+bIgP_vGb3(#U%2y=yKNt=hv1^q&q17HMD{)#y*y)-6qRA5$*n{b!2;9&Z+VJ zdUGQ6YZbOEvRzOzE}Fci{m*ctY3ZMf)+x7&*<|C{(eL9s{<(2olg45&^S=>O7fM;&|Ap0A^)bPA${nb7;BHd2d2 zJ?FxTPsJvTHWQZHw^x<@upSTp43ZD3@N7Q+=m+ebD#yy*tD(NE?b3q1eWQ3~mP30( z2UZn4<3_uk0M7s~h~D4tC`pIvx+*#eYffArIpi36whWr0U+9>vRT(~ayt2`8UE5Am z+_Jk9wt^c&hK>`^1Lp7H_q*CoaB@`2v+a;3TqOiuXNASvf-%uaeu7 z?z$wppIKz`jkRqwi`i8RzAzoJs!(t8t5P#Lysu!`;+GB1ybhf3GL#%DX;6ey$Rnc+ zeJ$I&GMuJkxy#-U2bnz;7LVSUTzs4-7#yq-Vv{je`dQs+u^=eUH6>ptru$sT4*pzP z*RUVpcC}WJYR(i}DD5|Uib_E9#f{LLvp8~59ZOq37s~yir?D<@(4Wh!qpr>XZPdbhxhF%rV(0@ak0ugt@P+kWnm9{8~m zTHK1A(E>fsk5!IC3mk5{V_oZCJ!HFd+SUt@3;S_BAz8~G(S}E*yfu3ojE;kl`dux3 zNB8r{7@{9;8n!Q}hO|vz6bn0LF06y5aeNU6PBe@LeV7on=ju%BgVUrYS7-H^Ef>1C z5726Z*y0oAL{`7mq93*?92?p$=E)MT|7?C|H*T^)?7!guZdSz*I%m7j#eOx7UndRs zyBrR?U`tF_#9x$t30`!^UO-mR7Y-&_&pfr{SGvqSZb$H**`>i*exf(%s=TlsP1n=4 zjec2oqafmdslNs5UQI=bV*8nsl>si@Zej+*Ut_f&s@k|4<|yN7xsPa^ku<{{E2XQ zQZR$m*^sF4=k8uNogCi5o{FSLxMVF9yx5oW$ z3*Po3LkuF*nb3@XdrFNg1%SF|-yYq4aB&#JVt(h{FMWo}`)j&Z?R<>BjjBwaLd&14 z(^=s@>dRZ!iBjbpHOJ~*NcTFkm^Zw(wM#attsLBAYWuCQ{EasIl9IOtv~*EJ+tK*! zYF9=_{HT9;8@}6ey+1VX{d)OTNT#UXYp-7SS+wJre!64pjE8N9mYP3^v#8HQz+kk< z(%{MtRG>fozTm+3-dUhY^>DT$0@2{GEwl9VIzjoFoEzbs=d7Z;e{PA zf0ajSwkhFwfPMd0MJX$FNCNz(+GvW|m>=&vX4x&ti@n(T6^sSIG2+ z=49XdA$~%=HDyvG1;_<+@)`7t){e$$plX7=M1Y$v^`1W2M{K__gkpc5@^7HkF!o8P zwvYA3ewHwD2vBt^kY{@C{`Hpo8n@5F%gMHfd_#TwRxzWyzh*`0c=jIm zgUWZ+CgNzd3m5tBM21fKE`9OdCr|i$@6s`ceh$>ZFpTEuwLE@!N3mTzeK-7Z-J`VZq34!^Vf4eV zrY+gs6_UoNoqN`gF7HfUXU>gX#Du=Q7O565U#>;bk{K0$#eF37=Jl|q zT5vI=UP(33NDHFH%9mw9*W0mefcgF&`e$8>)$ov=xWMBtaCh}tWJ8K{RG*n3Q$@Hk zVjIDQ@IkEnSd3yVc5!h zPQ9fFhavxNUM*32YgS7Sv()x2MifIF{a+vxf&>wJcMOI&MUW#N-(9>L4J@VVO1H?h zpzFbYBW~A!hu@Wf1sHg6APD9EjKAP3e3RuO2cdu{eo(dgQq&M5o#9{L@e?-7TftvH zUlD;&M-(H19fhf^JYKIK7G6Ow-{2GQQ4Q&R9=*HN&5G;K(mk>Pud0g;bB%NLbIl*! zdt~rPXJ92DtpU-Hg)w#{TlN<8 z=J$TqgIr!P8{pd#IxLsr8K0TG+t1jZI(g)cx{9BB^r3b2B+7_K%zf|7oOO7UTGI>< z9^r$oJ8M{5Z*}J_Rk!xz_kmP<_sqYD zPF<Fz^><1!SLgRE9sO{%BOKS3O#vvty_KXl$5vS!P*f`Q}p*Ke_jJ%qn@hko5OF zwD5vt3^xH?WKz*mrJ{V%C44%JWmL9Wa#~8nEl>qSsEo8kG<7Y2-Bf%CkLc7AcNkLd z{QEFTSAj)5y>_*+H1IDlX%97U5H}t*o+zC?o;M!bKoX2JXo<1NmyVN8m40gx*J_8* zDPBkXhY-}_L-`3pAY$LBJa1kX6o7#ev2Tq0--QHe(P=4aSuNJ3U#N^Xr;aDl8%on3 zk^#F65_mlbR|u91N5*_$89)Z%Mkc3eOeL(tVASZ$xnG1&Y3)G7AEH%eDS$7jgwF)51AI8yA%#O1Ae8dQ zPJu1nEwHyRr_gE%uTev$MS4TBMLy9)#THQ(sTgLh$K7|d1iGQ9UGjyQ)Z(eP(lK>5 zK-5G)kYhiU-1{+3E$~yxr`%(9-8Z|nx>+4D&p&qk2x8$0vQ#gyh$YTLp!5{05DV-fFhM5;F{5w!57#!`KaT$OS9jR#Xu|!v#a?YG0VOln)4G(wO!;`JCddy&Q_8Uta3>4> z^ca(8A$b6N<=qabls(ROF}^LS7+f&1;M*M!hWlzI1DtZly}R|naPMwmu(!gp6;LBn zL3*7f51;&NICW+<-Sp3ftbPuxZDeW>S zx6sco%n3T{57^uP!S2JG$HujT-whs&Xn*grfsbTykVuGjhVW#Wo4Z?k1xF8#Frs34(;&Bw+Qk~EVum$M)7A?37l4a>ms z;5_)_cd;?yMw{MFpik3KQ~z1y>gh8dn?2RUOO!<01Mlf;8c2F`Xs^?uxwq41J%RYS zXPk&M-g4SAH+|(CW<3pNf&6UQa)zZQJKSk$XpSo1cFH47pt&>l_DibQx!!sE%53NM zY}*RYyklsqx}D9|uj?Ou&BxmmhGid{z zFTYRgF`rv|TWxZ%Kt7GE_0mZMz9*FRD)@}?aQW0E`Pptg@Ty@~SzTv_G!sH<>uup3 zD<{mzfT-VdO|l^7HZ5=4Js17?rfEm!0;`Le6%5O~)y3@e6;s5gnUa-wMpbKXy3OA# z1wYQd^tMW9_-yX7Kw1f{y}J@uI}x63S5<*tYM}vkEgxp8w`Z$f6<+@WNh#D4SbBt| z&_O=fB&5b%U-Q+;_0+a~qjK-)&s{*zp9M!doB7s>!0qeir`MgWPs_I-o?q!NvTd(@ z-i=yaNN=akIMLtNFj(XpRON%E3GU{W`8dE71zYM`eAYVdhqlxv2)N;$ev9`CE>HPkdaU)OfA0}0+E&WFA-nXw?6npiUIi}9KUDeX} zw6@Y*=WO8s=bGf}d)_XM5xp!d9FAKkYE~!eVA`bb;^9e5JMq>#>9qGxqJOD6u=1c@ zb_TA8r-y3y0afD;my6$?KkDOTGj`=0OvZ({XLDDW!Tsf|4886I7X11T{!cIFqX71@%fO<8?a0#}zHmQa1C(nKFq+bZqK*s#o%0Fq1$3@0r_uvM|Zy z&K?cQwLEuZ`mwV3*h5&~(1CSv=5d|}vh?iBbL8+D%p2Kz_E>BT0&^QWFe?t-=ge|1 zQKMdaEb1~h-PefQGwZvXr=8?ya^bup4n*b>#(N^bSMqzciPt7@&lU4bzK3x66~Bzr zk`d}>b!9#-K9FPy_U)jz%2BT|&T>gy?|mSR=RH@zP|NuuXTPKcQH83R6uTzz-W~5T zZu!}Gc=*aSuoF46;v6#!H@{$Vu1egNQ20DEzf#2*YTzx<7%Ikz2?wegZkaTKdg~Yq zjJ6&$ruVckg&KNWH=gwD03pbu=7j6sGn<}u=PU{3WWA({=@T|ONmucJSzbnzrWX6b0LV3^n zr{>e9W1oZ$(Ux~Jg~V?zAx^}~v);bumX9+X&SObo;>UF}=oPD-F?;)qgcWVhp7vn< zKYc59OvoyNer>1qABHN`^j*=Obd+(se z;~Mdz;IX&Mo^b+1>w<9wNe_uS5>FV@gj1|Q=#kL`{+?^zaH>7!1UfA^ z!-`A86~!LpkrRR-(bES)Hq0D5W0J%Cw!`)+6U=olM5lB3!>HY%o+2TV{8t-B9(bsLSQFGlhbzu z&R!|eSGs$pgR10v>B;r>7w*p5$tmKmj2i*HV^ZN^dzwi|8#w36IXyC-pwUCgkEYuT zO2BF_`XqEZ!cX=WwJ%OqhC$aI?cd&fsh;Uu@w=`HciuBj>a@JTuJC}aTM2!=!r&|a zz3gN^6Zql^5QJ|;G7!P&$lbJ+7Z`|eRIMpd;RtlHPgECo8GB%=94pK_`)8ZvQkdp`Ti(7 ziRS^ldL@}D)EZ8`GEdJFPq0`Eiz3|U>F3h`G{*F{`h*>>^fO_j2zPoweBt-^tP<#S z;ruHupw9nhIGx=SI{1j@QcjL=g14@mGx4+%Kt1)my8=D^;tF2~0c)38Gv_PiEXX>7 zQ*SB*1VI4z@bHH7?MWrleO!DIW;Ui&CuQgjX#81IG|(EhLdg)3aJe&vw#o=HDGf4A zJ+B)Khw!4|d-KU@8gs4B!|tyHF%`>%8Yk;R1lvM@g$)jB$Jc)&^%kf)e%@Uh+;4qur9UKbNCdQOC| zRQG6Xd5Q=-J?6sUs+-?{ZN?Xc#;ou_B>k71MLF^D_JoV;HV7=ehYr zfNKp;dkhUl3wP-;`t*1Sh$^f!rKc)P4!R+;Fp&C)_4X@={j>9fv!L&rSc&?GllCk1 z{WHq_vt8B?Ej9aU02sQ*8JM)+E^yHPj!l2&c5BvWl>g^co}HB zdiZJJ^m#hjWVdL=aYK5s!ld{wKxk~}(_MVZGr!4-(O<5I7u}6`-Lr4x$ML)J z5d^mbeMZC73P$DG#KLVIvmpL;)pWALY9>qm34^LboynvBzoej01}+Rh=>}feEv8k9C7j#)9n6D-m_+9<2tSLIvjp z7`NFHkCSvl^~*NWJ&*>U`~*?jf^$-g+dLaA>3@x1|Go3?)7r_o;*3$?XAbLo4L`xg zifRwRh1G$H)#~!}Ul{^lg!GThUK&3sdHI>kVd+T*^`NBx`kB2!0N?jmT1lv&%#K?8 zDyN=qznfk+t*ENt_L~;%725Zzg1a+^s!>HT?8yQ;CH zFR*~APCWM5c;i+eXnymcX|Ack<1~hYz8Ls**fa8e{b28qzJ}|PwGu}TN5mt zEgvyY#6R{!n9}xJ@^8es6n$%yTz-V{|7fQ)t1R z|5R;xAJ)BgK;RExOpV?AI>pDi$(kB)tH0Z372uREZ0~0Nd8F7NFErj|WBC!*-#gy_ zqm$A!gp-3lmF9MTT%60d?#AWiM|w&M{G1$YsVTSm`w%WJ-x^;mKXOx2(BPaG@{}T6Coj@P z5A@GQ7+wv?^cRu=Z!ZU%N`K7sj!su9GnD81PRxU$cS|z?MW*bu!MA0fHUL!frF#&i;M;OfKLgmnucbAOG=>lp z;MX$5oc_1Tz}=_q0M&mv`oHwNTM!s1!#9j@FrE31STN1??q&iWOFuOS{%0zEv|=Lv zH2?og{n@}(Wd8IUKg5V+-e$_LizMeY(m&bl`YJi#lilDgIfw_!?lxM%(uxK`RpeH{Gn(pMw0{AC7K%+1_p3tJ z)AAuh8a9j)add(+RCLVo@hRWF^)>dxW|Y*tIk^?gZsAC+y6!fR&M<6|1yZy{hBt`M z&}`8jOXoPdef}e?8-6xu9&u`z&4awS{RSDe4}62v+|L9vba`|1fX}=64asPKVQt;I9Et2VU zZ_xb1Ma7^w<>^K)2C}qz#FK@Im>iG`135Nz!mbM3;Y#_ei6KBgc4QXL@F-Hfi=kv~ zs%zp2{|pCbHU=VJAJN%b{Sm7%+Q7g!t^#4+uCG;XUW67XGT@ePI{zMq*}FJwd+R)hJ^6SvO0fg_0aD?%#Vt$MR?eLG|Q zXdTwJ25Vb~g|5Ov|G+}mV4>@za6x*!4_Y@7Dv=}32M7$utflDF%h;{bkLs3-xu~_+i*}J!&yLs zW5Y{eIdx`CuWxQZiDU3KHZU>0Kt|@J9603QjE<(h|LLVnF&@Z!mv?9ROalm)P39#E zphSh0=5#?0!w*e*&0d2=9i z5|rgT%%8K{IHL<2liI!S^Z>Pi*DV1p*ck<}Tu3}&M4e>U<-e4Vd`YvfacouT6&1Aj zN``!))hGvmq&^$JEQ2!R<)4Ukv9Z>8MF%N-$#rWsQe@WvW$hl4wTUh~#6r09z}NWg(2yStSOV3DGO(qZ@L zj&ak6sG9`d8IFZTBO5v_#k+DqP(I%ib}TP~6*N+>zrW@M=32J`>6f*cwsOEVl*G)?#5pa23xJRQdsvB~ z!rn^d!%{-q!t~mrTz;>HGF=QVDec!uiF?7=D2K-B<_^NQ zW0HJ%^D6+iICa{6Zfjqd;M?R&N4V+l4#nc&%bQ(>xbW#qCJlUSa$Cu{@y5$#@vb{i zbR@g*vfaJY@cYFZ4%w1;q2J6R*cijMN^#=6(R@}#MNZ0hsXBd8ntdKY{nb6jY6WMs z6tl$r0@{pESt@=3wOa~>n_i{P@5=okzLF1$3|013PrHxeg-a-9AA&bRqWnm!O6MIm zWYJGKZ&qdcwM-V>j^O&NhzbjUMEa4qzuCQ0c@SY7ar1<)<6Bl8nFiIL67liV?$*fj zGP@G!1~#4cGY_f~b=`%DJTF%@#K^Tz{arDEx{Er1;Tpj=P|48qZcG$qS)1jC?J_C0 zEtnp~kzIF#_#BmOzqD_D6l;DKuH1Izldr;`aFVb2$-7H3r04S*qhu* zbq`zT(+)w>bPJ@Y&aXDnO%v)B2eE!6blN8gqe0ZC(Wf|W%xSkxRD5};(FXf+5z=1m z=7OA&BEyx11zI5cILvWlXu0NVdEs$u0G{J(`EMK%aFDiO6V3&C!bZu3#Osk=C#yB_ z*m?@;@md=c1%ahO+6;|$!bL0%_SG6Nw2GTHp9uW6q39hW7&1sdNtv$-{WELp{Mw@r9B+_ zKFX6oVS)brHhk_wzc}F#Yj4j~<@n#S@z>rLP)&l0zagiCT>Ev<8R^>jgKQ}dG+9>X z-kVe7Zr-2V_~xZarBj72)rij`5g@_sw;v|ORia=dDB1n(k5D1JroZyW5$EC9nG@H> z-I)DW<&~O^n#qIff^(Xz&TFdg-?Aim&^PVbgt1}-TmJ`~4y16o;7Q2mzT9o|;+zsV z$DKfzI8-Vp!P4R?+~Vw#tg?+ZoJ%BPUGPZJa3gr`8^{dh$xDa zeZNdT&bz9S=>TTV1X*b1YF zFQ)6%#B;J&ba6WtC?bUM&d{I@__Da>X{a_E-OdBz%<4%G9MxH&6bR6ZzMQg9%N#ll zj`rIQF_RCzVxn`(?dlI=$G+}mmFf}UkDB!DNRBM^4@kU|gCywBPsjIgDBG6=feTJc#`jL=`JcDiR*IKvc#bdVhNOuth~o=1 z9}C~>YjzJf9TMRQrb7{#M)sn#;_fDgB`(S5%)J><`u@xM&6r7uw z13j7>4qA9>grPvbKqc>%7L*gCho(OerE7$_v0J#ja*i9ymZU~AUEEm6GqNGY-VRUL zajrCmWFFrrm~g0tX5){R^l`q#p(0^#lX=S9Y&MLRYP!tF)jvQ@@mEcG$dXXUjh9Q} zU(F}v#fZ?QCa|GSpdE$3_fMDBaT-=AF#@=OeBs1sT{S0!r1RcMJQ}+s-X%3rdHcZ$ zN4wR=iVY4W>YxBuCdi|%jBKc}x58s2wKrtP%CDoaWd$WuF@7YS+O6@E5>!#{S)*SN>tX~- zludRWCz|9!5e2S1pcLgH2&2rQr%MH2JIBLnplD-J9$c{fLcLKdeP`F>l8=J|2M)W^ z*-{%nqA!{}&Gain$;0Jx6YAO~FG;Y!WkILz%Z2EFB8wDU%25H;@5+f9#K~s)%3r8R zlP@S$dLghk&&EwHQQSFg7dq0+3rv;6u~H;cSat6*Z6ilaoT;J=tG;bMmzJv zZv>Ww(b*8Ckx_#jN~^La*6(Ip36WFL;sc<=a#c;Dv0|RsK*h7`Xwht2aXZkW-RxEB zK>eU%B0b*PirTTP^osN&4=ro6#T-14MA95CiML`a#G^@(--Y zE)5*_{eNg`kU~{u|3lL+5~4C$xBsq0d#rsDH~H)vDDrZ`=f_%1Jq2aSxxqg;aVG3g zR~mQ5g^-Zq&^9O|LWn;-$|<{!Z84}eIc5SDR|C5P2YePlSyxhbV`8+&10JBl-iji!%KJf(%o~u6S1sRH{XMpG%oQb|t4pKx< z6@i|M6}=)o;ZR~<=rXdQb)rsf{EeJ!{z~Z!eesd&8hS~D`tL15)j!|$1j%g^uf7n4 zU$E-cM)(b6V_$F4zs5=;J(8(ji4y(mkEvQGv1*eJ4fr*i8es(boEJ`K_8;+~Kf7C_ z+Gh65Pjc)G=tllm^NmVHcM;@CAe8kTTq9ky{|t}m?RY@(ndW9O;Z@X zqh2|TE?SrROo{qVXod`V5($hfusmtB)I?xF5ImDXmrDb9bh8laDJ8DO$l3MLv_)2U;&Pen1S_Mhye0nZ$VJM8`7!!jdRB?7Coqh1M$9-wwt7(OkmWMESyW zr}1~iKgG~RPHaNy5ft!Bs%ByiC8@_Z(prg4+1->~vHzM>2oEi$)w0*8msPO(10eg? zzNwSUMAA1N=c*@)JQL>fB5n0DK%X$pT*4v```&wjQV1#xXZNHLq<$Ud3s6j^wMmv|ziGByx#{L53I z=wk?ItcnL}v5`&G)gTM0qU<^fGZ3u4cVnvUC9&M)kIcl zUJ}CTi8=t`9IUG$!Xgf5a=$v4D*us~GEuj%5rJJtf+lb0Tv&~m@!c4k^j%^;>{Ndc zLe#z`AE$Wscf}l-iN3;&y3DSl(Kt{B)r^0){GWvMocrznFPKDQO2q$J^&#FCN}V-& zt+qy5uN@RE$paS+R!3zM6Tapp8k`wsEyj=BPba8Y5=)&$_wn78m9SO&W?d#^2KPPxtx|by6h&@>!4_gKT=CRxMIu+16(3l6@@wc zOu`d9*vk&x#f8)HknP^XeB2~oy#6masO(&;l@d;GpS(l)58zIci?KUFMci=uc8;Q; zto-^e#i$c$U#O+ZI&*Q(Qpcfq=j7Vgoakm;h&XYuT){jj5~EhO#q$rz`@6~j+6Sn9 z7}|$~XGtYa)PW8i;Yd{{fMqaJ%)4-lrhD1>Ow54F5a_JSJGUxOTj_FfQ0&w4yvqJcK)__^dhxG&!8s zp#zO&|1#FKKGE?JN!mNrKR1aE1j+BO4MPGLG?P5n|lpA~2$OvV4Ijv*I{M zy_0xiQra4QO_a^HDJOmwQcL2v3nNhE*bOILj15uXG<9LdY3>s(eiI)6b;$5=`9$$% zK7RQTI!cjaPN`V_s;~O$CXyyRNz#;i8y`= ziz^4UJWxFbL%nNn{$fF@e4xNgL($l-*30ZAOn{8wuzz$Lq53>EYE2|LR5f2oveiLn)mk?^uX05xi4 z18|}a)WVqlj}%66A)*c{o2+h84&z5e;L}BiLrMRo3glZ!z73-}|Bt3?k7w%t|1Kef zJ_+Tv`gD=RlyYa?RYIj)a+wm6%UtF@rc^3LK5{9Sg`{#P#;{Fd$z>r%%!b$)Gsb3S z+wb^39>0Isd7pFM=Y8Jq*ZcK)KA*31&U;P14i5-bx?2LzLTc~_AnKEntbV-$P|CVf zwGJl#O&@XN$zCroJmY^xz3?j7qy)GlsloomLcncEbIzFD;zEYOJ<|LlV8FJhyCR)4 z2w)@`ej%V?-KGw@5~?8>R}7^+J4y0auM;UKvXXvv> z0G2Ipx(Aq`>>BGkhXK?y4YvZ$I`>O|cx?w%M)3mrCI!a|0x*-#d@OMGu@FO5FQk%b#Yi%^Ep$KtVFeKyH4*T0Zc|F^F(DTM2JGKnJ+WL;F zsVX@W5TUx;Lg|(LK$E46`pNJ4DnKK&$#xh-IW$Ls`Iu=8#I7XjHh9!g4=seU=|5yE z44Eof?$~IA;YT#LWhZ9*vig9s0Z57gU3+Lvg5pc^InVu|O>!X^s*S4v8Vdl_6qrw$ zQu$y2yjx(hBjT!4?n!lPnh21Dq!;8grnwYW~ zmd8;ps1T5a$PpukNWEYIJZU8lm0||akkdC?@g_<9Q66f5=ZZ6wAE)$*0No0^E6<;X z++s=FwY`8JfH(EhYL{HxbO0uAH-MEk9k!U@>>uDfv#mC12hxy$O(p?YNi?{&1CXKa z0;gs7nLiz+7Dft7;WMluuY!Eh7ISMie_FRy6QJL;wF5QHeVNq2Beojnv<@FWZaLm3 zf&-oT$Yz9x0xNGOAhrY8Lr?&Wb{h>S0&GGq1iN*b zKH`82a0h@;S6XL1DFK`&(6$Cy_HFf!1XkU0V(SMGw%zXo2jQv>02}?WRkRx`=MHp5 zEaD+dI(w`A*%2~E_h#T1mUzapE zrC2W!ECWs}cC!%-7rW(CH&SHEA_E?BmxV$(__@qr<+-(DVdYio5+EGDfj^?&8V60k zsoZOwza~-kYq1ER5Hr*vI_DJKIzB~P#7dZ|IOOnw1?2J+!2+Qu$d-RA18 z%Qg3D3#h&&Wl6AYOVn7>Q4ft4RD{IRd9*#SH!wsWqXWl1suEi&LV)S9Phnz97rF1W zmnH?kcqw7ffRUf=BTA$!Uk@@EU)d>I{TOde|eEbTn+S<$9g@_0$81VT< zd*o1T0YH1eH5k#aWLByIz)O`J1^_Q^ojo<2?gT(j+9P7Ig&iO}ZU{)$Z`N;;sTflCZPd{;Cx-icHczU`?#w#d8lHgjdRt~fpuvBF%f~YwkC@L z-E$c5j1mB1itr!;EPIS@3}hQ5%XQWjF|PpsriVWa2W<_8v*r+}Y^R7NJgO`O{HjilRPaRr0$u^6UWa~#Ewln3R@vM^-U(3ugbnt-yW23I@cp@2{x zQ7yGXY~6=Y-R6JCs@jmYEqPnDQKsp&R9ie`iv9(l7f3?0;;=@KwCIDUf%Do8K+F(6 zVcrIkbxPID_v4!T`F3+FlVxWdisC>Ow}2>dby(}_>8Lb-;ArJMV_I7YgJ>L?xtG4J z_z&kvpk^lMBf8a1oR|5?dVcA;l_H>oe5M!lPwOI-bDlDNJc!1SVXcM4;MN4~Hou2j z{jFt$H~0J$oZrMpz{jY9pWi6`JFwX&wzZE_wo|mhAWW5)wN%v!-RI*_u z`AfG9Xj}4O47;ozfJmbZLp%*w|#29;KA06m# z{q#S*^$5J`D8OFz0HG=O3aP&mx;^y_`Z^%^+z1IQ8f2 zd;WEw{BvxkZyeggIGTb zWb@PjfGSz;ZK?rC%+A}8s?jcaw5tk;135&?Dd)@r!2ZNpAqy3c^KZog)!SUd9xN#X z!baM`R{JfL3BY1Dg={@Qf0m6tiD7P$8OgLWBwK*00l=K)0*~Lq^_s^{lN^AI$^n;W z!SDZv?WFq7n7I7U2Q|^3&NC9Z03!bfvt!G;WKy=Q9MGskv@taR6gF}x4Y9^7UvUR*3U8j{Bm!5pF}XmK$L)20X5fa_U?}(|ru?!S6m(RE zZ#TGTat;u-Pc?2MW497OcS+aJn>YfF);34BAdv}}K{8k-yC}D0Ygnl#;4#3kdVJea zy~6`rsn(pZzfqH%4uk?})_FLA&ha9*i8V@i6lzydZ5J@f^cR3)0hO;jZvq7YV38%8mkcTGQ*kjVG>De@iv{A%PTD zv;1^a<-ia?m`jW0=QjY=o_IQ{exM!*yHTvGa1&i1!DSGwim3d-l&qc>jvgO?> zz!Mb#8&SXjDL7fEjAsgvqqq%No^rGnFt*tWP%+Q6E$)|CB@H`mV)s7}r~$|i=E-Z~ zl3u4UmOu|mgeYLV;k-a|svUinvC?LeWKyg*G~*qQ7XwmtTapxyT0!soHB-mXe1tA+ zKD{NOw`h?ccvu60Kq3H$?C_E-nWal`!40&ohY>Jy)kr7spSk(d!{yDyt9=Ly4ECak|usF~@C z?`lCk8KQpT8W}KP4*7&x&^UnIJb1Zgc4x!S9nCAI98~1el8A{Y^`?%g}qhMJQIzs zgM=UObrjvsMeUA5&hsm7EZ25LU&kddrZ4ydi&TuXD8%1-BUCD;{5jO+1d0%JjqS>< zFusDT58J#jA^fm7=w?uzAKUD6aZccN3*+W}lQ;b~wcqXBZ^2HQV@*!Q$JS+XnLY5) zABhL3GIadq|5&a^#~#aZcNe9LgVMpH^2f>(Ne*)HFkZ*$1K5eXOa1v@dBl+le% zy?l1Lb(F?NUf&`FiSW3~FP%6+Ny#Td?-L~@g(GZY)ZH<#GS>GdE=PES&m894D$qQ> zq|(A$rpO$=-}4!h^GpJrUvV>&Gz?{Qa+XA5E^RRl33|qz!%=OWF8m_|NEaW|g$>$0ijyoUn$%EUMlg#d7N00=PJ$+C6zb2t(*o zTM%d$v~!-#fWz*h;nB1t4o&av3x|EK`xSq?l( z$i)d%>YFUI(+CR47}1NbwH3LKW-HK0$0`oZQK7YgS@mE+g^y4|ci7YjvktGI*?vga z3ZGsYNtPXrgFDYWkN@v6O{w+nm>bkNAa5m^R-ff33>7CxI$bi8tT2GAA9q znGFp%QH$Esih2+)7KtnUvo!O3)M5Giif*LlC;@mi^@q`&AY(lhSP~ucjt-c;TdP-_P19{Bl@e02exI8*D|xeqfA|%|8o5_zAnlu)UyWzUu-y z^@SGl3*@<0)zJ~9YSeM_2GMB3ya*!*8|4J-8A{-f!8L3;`e2K`{5#5swz8TDzA7{4Q<1} zGhMsGchEv`{0xv%gdox9G)HKIXd4oSLylEEW7|RkL>sy2P=S8vhQsVEho;2AT)9Y? z!1dA>EkC9T+&LPV)_isa(cuc=`+PnTEkyiOA51_hzI@(_T=Zq=cwa4#|AN-+CVZxh zA9t&wJUU*p#)!=t*T+ESf-0x~?32_YL7FWJ4>Vf&kH8yrDz4!w(opv6Rs*Ko*r{HN zFGsGjldQ%QM6XkZyja)Oa!1i_f_AGv706bpd3O)y0R!t#=GQphZVsiV*=aB+HnLkX zX=$^L2VR+LeZE$@x6b`7yhSp z`ifUJAcGynSck`!W4$ysBuc6#o?1T>hHx0TP1LAkWWd7AFf3?kBz8C_H|ejv>wWQFr;*}p+9Y)l;DXv z>Gc&@2pJ|Y=kQ?Y!1WhZe~`F5C6pG8cjHTRl{xNi?84^kja{+{eBy@f>~=JCn@u3! z`i&hQm&9*?L6Ze-FVNg`Otc#gG;dZF3|Y8PEgrZnx6P?q0nUj!uB9pA`)Jh5z(p+U zk&4$&L>|0sv`UK6cCmVIpw!rD%GIBezKO10ia8LY$cb9s=_ustnKv4FinEDEZSJgV zh9FD1V`3?aFSJneZeuVdejX8GIt=n1(+vq(5R{J{cUWRXEQ)m6T)~1gNPYEtiK-`bc?#+cc~17`GAW#7B5J@fYthz{=KoeY_@3 z@U@l-Tv)2}(TQQ`DMEk>j8XM)+U^Bu#U|bp|tg5uG%WVd&_mW}01s=W2 z5}PY3$R?N5!nGDi&poW0o$w34TV+6|fmW9N`sn^K!Ht7w?kXPVqX;}c(*DEPp&t;# zb(F}jl43QM5p=Oav;eO~c$k;FW#~ zt&y1j0xB9^zp-(gnASUHf^}cuEzOpqwFzH%nr`cHveEnm2M2B#HCz|hg53T}Zn8t#8_tC=Ug57t%M+DtAuf@^?|lw_}aMO{t#=r-8MrWW*V& z&9Q=bXI;V~Kif9n1$py=1!9#Od@IG9M*L<=sY!UQ-l20L)72PdvhgZ)J!3pFU>qml zjJXII;qenlR_;r%ur&-f|B(STjmS2#$&dcg5$>yc%N|GN=Eg$9*BxlqA=kLyTGc$> z#BS(XScP8ZIMXZ;zJJ8~AzGK8yRf;%4Z z^J!164&-EDmt#I9zh>V|^(15!$5|d4+%<;5u7|Ult*jjqI2EA^Z0VNi0NzT{3s;IT zY&hR`=?^b)`l9D(lt9*IbsQZi($Gj-{zzF_GwO}gXNIwGo_oXQTAKnI-Y{d=x|;J_ zy~&tc$b(nS!|qSqnkHxp+=g^$&`5`Y*HS3Lm+&;f5B>Y@2y{XkQZWf7{(8{kp z%&d<52CYw^-+RvQd5{l$&gx4m_rm2MfB#Z-svMO`UM4t~xd9aP@W1!6keT9lp!)@A~~N(w~`1 z2>yWZQEu&RuN+uQn#g^IrwgDNZNvfRlO8r-g3& zr{Z2WZpIxN`aVNB=z7jwNKVgvhQc@b9PZg?-5W|0X2Ab8mM=NmwfvcW@Ye$hI=O2p z<>FLr!Ib(`t?!iG_oNn>Tt$bCJFY!LKmKhTMZr~dug<;hd&req7tvohfG<*BifU)- zptZ+_Mb1AH%x2SGBccriir(4PbsP-I?pmovpYPY z()>}M8d7$5!e!TmwOiGoV=1lJ2OP%FyH!;W1s`ii7y-Vx`eQ4a3PIh=8U~b&h$Zjt zx=GZiM!nuRzbqk!KkeUY!Z8c~A?oJc(&?CuI_fgTPbTs+#u2s7K@>LjH=EMs07)*( z62*J&4=Sj8753X8ZH+(b=VzRd-$=ea%( z4qJ1#S&I?%2ra*bF3pXbBEj_)6{}}EOHvxZNIuso*75~f%2rsn)Vk`6D;7Gi^%_@E zLw;B`afM%+D~NkWjpSrgkzcc>bZj8>J)PXWt$OzZ&TO3`&1#f(stOva`+fdefWiFRw2EABfpOLn4kW0^@$? zYSV(%3Gd0;R#gM{6X^VDqbyZ5o24a3cjM-e$(6em76&(?B?w%PmiK{YCV@p~UzF;f zs%d&6ucTYx*5K7O)!Lm;ufUoty=ns!Q45EVG z56VD}(j_e(B~;7gC~_nvsXGlNdw{y#x9Zlz?{kqfPSkI3kG~A+5w~sI=OSbLuut)g za?duM|KUApr^GYW>g6)IXP}S)DR$eHYU#jjWB8XsdfSSyMo|5q;6b%^pu`y7J1yU_ zvApp}i$re?;az_K^K4SdlkOf56*Fvk`Huzgj-~9(V=k&@AqzEZx$MjfOQ`!y!Xewm ze=?Y7o5L4sI-Z29x_J1H4qR4NFwpmOYJj(mpNK>7?uN$^p}^|}f6(Q-PgXZZKhbx7 z|0()TsB4W;$9L{Xd+1?@$hS{)?oR;Hcue#}6cbw19x0JAq8J%3R>;f<{K)jfXSu9*?@PL3Wl z?D_6sJei{#Y$j))=j>$acV{_-z$9Mh-?-eK|EM+-xXSEkyZytx=j7F-voclQwHX*9 zVc%w%?1g7#OZ{ki=v1{Roxeq2kCON2D%=5s|F6DW-j=M2TGZnd9I{zfUj3(sr% zfWPWEL6`Tnjf932l<4(ja9UP79gNvNN2VQ69qNc(lyLv2-nRFb~WIY)U z&^@ASm>W}_nU&j{eK+-^sWqx5y_n!65FiQ`mUQJ z&n%5*nq~QmF9em}E8F%*dg^d7=NfL)Df6N3ag_%M+XkuT=R;dfovTxnV_?S!f88Dj zZ!@!gHf>RdQ{xYg5NGmSH6hz9U#=?YEIrYL9t?VL_VaXtN$s23!^KtadrFi1@F-cN z>f8^0p#MXS5$L&?;QhEUGoj&3oPtnmef7=Ann=-5waeY{-J#B1u6N7do^bp*Iy-V= z@XwQvtK#7u#XU&z@RDvj&=I5Yf4yIA%X_=O+wb#r{Zqy-$($2@2J)lFirnw^ZO^$W zZ-*!}yZv{;my>LstlmrCIjx;$-C%g{KojA`w@7ZofYxIJ6SzM6+|2FWKYPi(@4wpk z{+B|gjOm&L9WZ|TcY1cGnMG~ycGuvCzed?-moFSHz4UgRvMWh%N9D;z%MajJy3Y9D zF?TNMHd?}c#8Y<*7q1L{bH-TSJp1|3^&e(FhpVOftLDdk_yc(7BArjP&2TJ4^;duhk7Pt8^@ZxktF>D7}{t3jYmh;&yHI~EwC!C*5FOm^i9iMAjqxCf}kP4c?dCY=o z%ARAMRH^*pJ-mNg47H*+cH>E2}Ye$*au*G>ry+;L_mVPloC%>@{}4Ii1}6 z*aR-6tdM_YOhnhOHY=rp8VT7X5W~E|7%XIuI;Sz#*IV+XndZjF?<^g7+E4+x!$}}o z@++Glg6x+S%XZD8z6`!1#*|;rMlP`ag!?Hhv4CYhE45>}*b}FHxbY(LIT&K*^S5@B z3XizoDO#mEf|EGXG^_{1out^f!!NhZrx#jEM_7t#CeZGPr3H)Nah;d}0Utz0EPQ||%U&;*iE28qL`KatV*G+gAqP*m)l#a}~w z5bHU~EWW1>2lLmo)H5mWZifhe396Y^AA`+j=pGa-hBz4{s5J0{pFEHrRA75HepFG-+y*^tfShc#h_$R;o>t0^)GcU2cE0fJg@9vc z2mEDl!U2_b_He=`hedo)*}RB-eLcMuI!$uBtrumv$h+Xrvu$y(i{sFb2bp0jDIANI z4fM-l{VHY*W5jw$r$w;dG^mz0<+@oW+8B*@Ho(p7BJk)!o5SEL%jp%*IBe`@Fr$x8 z9oIk@ajf5em=0PJ@CUk+5hDV|e?60fBoD!&XDMWLp<9+m0lC*Me=O7~2vO)+XmSk#s_*ll`v^KwjpBZ(H(vIuRL^Fzh?;-(8G!k8<{ZXeSU;!*Rh z!IxHAzuK%659upZAq$*EV>Vm|F2XtPf!B}PLR*TdfQ0Gg9$dTHRZz$Ol9Y;_sh5q5 zNBkkdm#?z{;CH<6;$?AJ$%Dll1Z<|1)(g}s#c-~42qS-Vf4+|k|I z#f$>XYEtvY`iq7@Q&uA`l8G2!;8|(MfcZ0Io2+d2=D}qR6pmX;P}}`{t}&*&Mg>RV zUz{*G^-xe5WU=t2-XA$JZ8q1?&)099`Ye86!-_kV&!9XoU+%aJgZIR7SEy@XPe*o@ zAo(L|~VwSqxhe^v|8qRvl3$oRi~UFQ1jBZnRO} z9?V9XRQGpzu1%GO&)-fh3tU0oUVG5aj-m0`+U6~x?HM>)8l>2uhB#rQ92Gy8XdPun zRN8DznEiQ51-vNi%;972YqZdF0{XhmKYu3vBxMz>pXI$uLts|$--mLom@aj*DGE0F z?qNS?10a+}epNtLk83FFyS(RpGsJHh9_VU!2Np@H>BR;niQ<+rY*Pj-tgYt8DWoj! zp3tQ>k6Kkn~DO#dGbT%P&z&obsZ>l_gU6EZ(aJ|u@cHmTO+?8PinU?DE zP+<``BcRf%aWVNjEqLR;(^{~_2Bpb+baW6z9$8%bz8Y>@MGhJ)&vNA=W2sB6j-BL4 zdD-gRnR>Sws$8nb^87VDSSnl) z5}=nooci)bOP6827RR~8HnTrIG*(9@P)-szHq!o6kAe;_wptM;qS*uz1-iccvFQMa ztLPTL=vMN?SyAx!txZ-G{rG*j=%ExLqN>@SyGGNunBhNCqQvl?Tn8CknwS~K?57-r zS}&D#@Hwt46Bn|Utj<}}HDf21)>cnt!aS^E|8!ze z&f_N|e-a4}u1!mO2bt@LnFN#|j&-U8W`g8ybYET?59?Q!&o6S&9)VLUQsPeq1ds6l zt9^P7ES#=Z`{_B#3MBvT2shE@{7hG3Zg!x^YO30s?i8+e>r@*O~N}v7RdF?(?`$R6m#5GVfopIxezx?A?9|a%1xDk@IgIZR@ z*PUY$K2869we!FK?)$+fs@H8VRK(r7Ss7dJbiw_z{O#6HD*uF5Su|e!do*@PgHyYa zJJXX9^HDbzhg#yU353h?orB!BZ;kx=dh%S^mRfgDT)TJG;KfUU#*3HX?^8onK6z)W z|E>wpnjcvd-#xVGzuO0)av?&rGh$x*bbhSnh5XpaljpfFP0n+*{hl#1g6ZRaO@-mm zw8C&i#s0(CiJcj&|Mq{R=se94;xz8^qf(z~8Cw1uxo`QvJ)IRp?WUK}>2Amh@sbY_ z=Z2z!e26>ZXxbb1Gs1v%8XPV-^{>glZLR!fbcVw{s$TZ&X-$S|C)AFYaLI1=XW3l& zZ(BcfX7sG<(O$0eG}5wSPq?bk`O#qWoleF#qrsu*)cL2HPJ@F7Pk5sT8}AjgzUw3C zc@1~ZSp1RYLDG>&q(nP-$_hJvoJKwhdQZCf1UD%;E@Hw=6=!Rso$luKT!`+NJ=%8C z2vuIv)u8O^V{paevAeO_&vxYNTS-ciXN}Z)wPr$8($v%Ht9D;8)OM@VRFQAI<}6(P z`s9s#v_$Z!?Vq376@^0lQ&kp@zjnXAo8_?!yVrZ!%dg6$eJ{5iZj_T1e+ZVQW7HuPxI_Lod_UrHmh zCN4)A2EwE6Bvm)>E65TprPDeK9xgmI^Y>povKh09>hf#eeW5z(Y~-cwsBwc|6s4S1 zwHaU?waw<7iOuXm=YGMl2DhQ%2gs~~&>x?a4Y&{o=w(~4!DHrg;5mmC`L2D7jS`Wc zqx0@!#s-~ASt|&*^tChka|G4N$o;o7y}NJz8kK;_V7PY-2H zimD2Iyqed#kWX{QF#WroLp!XMdBW3NB?pa^>>cADTrT z=Vn@aHs%P}ksrU$tg||1F=lmlgy$i0%52=ir|QY&OaA zSg^yB>VE>%OLye*3ztoJ_mL+vKbuY?{bZ^=f7a&P`C)J41L_*7|~Hg zSg$V~w9ft))Z;J}epK6j*6GrA9rm$54V^i?8^cctbE6>+lJ^h&dUd+&VX*XV`PIYk z-4cf-$%Dur0PvfN3#bUS9@4UxJC`ec-epR8Hy}z$GUMH@u@O%47ZMxxmZyJ5Z zLEKV??*3l&z0>=W(|Z4_=&K@NT8G~w4C<6Z%RW4*!M(9 zvJ@QjOYGNul@-Z%&XZ@&8c{Uok^U2L$A5w=PP1DT-}tdJs_$xeKBk6T>5?I^|4E$2 zj6m_=3Rc*&!2od=`zLDGRGCTa#V1Dyvll*VgFoInB2PoznmqqmCE!{CIZ0KG*Z(iq z{3NNba((ioiZM*(GOYwjbx<@r)497l@mAgr+(m0?ua7=1P~lIr{t;%#TP?)@+F=B+c=E_Rl$thnBj66XyZ z&&o5w==PTCIfi*YC@rf)SUP^$*HGkr@;TAT|CrX>CnuVY#cwwcuw9;KC|$3U2h;ca9?$oC zO*z!KET=X0w(n}=rrhGm%$aEwQ!nI;l&|q}SvX0vQ&xKS=h74NF>Kc&C5x1E?>qyw zzcXvjHouR>oYEe2PVHDDu^lH#TB?iA+UJ)vTWSqDMTfnD&Ti~(hlG86D=E6FUMeR{ zrzrTmcQsWD`^Y$VyRL}+IrRQ>jzt@0H(hi(H<2k5yXQXq=xW@1-P7`k{~LT6V7xnZ z0?GbT>S75yk6e*NO8?&Wd&fyBU}-BiQ@|kKbO&u4-vNjGYCnD5{I3kKGp{l`WB=q0 z{LpE%gx$uYxuJs{(7(q0xf+zMHlDp5BkkR2$P5ZF+mt&V3)u!1aFHQ<~Ilt|0ohX@m-{mcJ z_TEMRTIZ=<)jN#3Ya~W`8n0E>8qfIgMz#g`ZNPWVcpJSLa|b?||NVmPypfsvVNiZ| zQT3!3GB^3}Ld3{sq~q-4j=y1<6h`E`@fN9H(Kn$F0&=5X<+pdPb#Ul;u{4+&pd1B>EWbL!}MJEC;#J}pJ1-&^LC(|@>p1JNnb?xJz zKh!(?9o=m&g42C;!Ql3s_b-v?AIL{z(SKi&LG6Ep-Rl{Ihe3*L2atCL3p_qZ{w7c! zByAT7=h+j{w+BT*Y!Os+6ZWMkmVJuk=^Q6eFqwd}_0hxRNWPXEvxUK|K;mc9zqOSLXas$Qlto8yYX@MN_1(koERLq+oq;5{2Y(t8fA#ikeO?n6y6JliY-Js0ewQJjD!(peCR9?P73EEEi- z^+~Fq75yum!o8N+-7RqbsV7QNbez?`iFIvnWkLE5eM){0yt;kjja!=NLCm?CEVBh! zQvYZ*aob5*`i}Pj%NUrZXYX?HroyUP(%he=v2Hq3cy^IHdCU?uVbceFcg$uHq@0n-S_c zngywveHoM&8CKoR%TYSQgcmph3vH1a&IDHiPHAuJ zp#+S|MepR%BnlK86%ChQZsNpk?n*fnrwK(D zsqyWmQX$IJoA}4u~mwNk6s`>k}In4-uzc(gtnb zx51a})Rt=86{3T$Pb?OPNsaF{q<~YUTC~1Rij2dW=ri_m zp^TTc_3Gcr>_vcOCrgHdKK8Hi;kD{{K3!&F|LAXtRDbM)PUWA zlWPj5*}x$sEm(2z^qNAnR22*#IWZxgy}TkBxw0ZT@x5DVL`?a0eBuFdb_`Sp&)?r- zL~T#>7DG#Od>q#Pt%+xp@1>*#qGr{HGt7NfiFo^Fg#PXO;ff} z6&NkIChJ6Wc)XZNXuR0c^oH03aZMbNLsL?2aVmgN>34sCLM*+6Q=7ieyuP{g&PY~HuZWYvd5IcL2z;-*HC5nlef5YEP~Zj>^hdsyG1dNGhe|8A z>n2)AHi)H)2hPli$3@5Hcsa3G6B*I5_%%x_>cs->f`wl_gOb6MbK)}5h#W8J7=-oY zv4`u4uM!nP+P-aTH1aY^Q!;%*KuOHdmc`v-5O{5E(BV%*mYN}2N+`)0+GL_eOtjn^ zR5DP>p2Hu+Qzyj}&!qAsX~SYqWk38A^7rBq2sHudkD!oQmuDlXR!w6z)`<8#ah;?K zA-*m-S}sw|5bw!aM+ju-ZwH zM}U9mJwYHo?V$UdCS9LTR)#{pmx;|MHutK$uH3o0>OXUrXLcr4IYgG;toYMa&ac)u z&@3x;?v`R$(5G!b{gPILFVPaCt+#_EZ%f6N%Lt4Ug$dP?8FkVt%aWWVq0cqY%sY?3 zM32{fvdiNqZNgA#-w#Gj5APo~=$ynjT-d&Fmg(@(pMm-pgF4}0VPa6kogS!PyL4Ou zRaSv~F{h7hGL@3xzjkYo|0>AZ70U?0HWVB!dw=<| z#X*kSr^#+Z`=P%Xk(a+US#4HR-W1*ecer~>b*S2D`=dV~!s}TlVpBn1z!pO&oWr0^ zcXsJ0$-djfp__7ft!sEV*IvD#VzlbP*IPi#&^NwBef$_@ zrtg>PWDlx$`{Yp_hcbJo)%^)8UmxjtvAgveL(Mw2EV3{QDOKP8pK&B>5OHp?^mUfS zhisXJ-evXM%$*$D$pkv6P;fQ-rhVz-R`Z37znO!pX6gn-Gwz9QyGex4Wj3=mXLsd5 z_Uos*rt(z&O|P%+#-H%_@{bHKy_q)iwLsNBvf&{%Tg+E@(1QV9(6+qd+&> z-{IrIY%ki2QJ;cg;YeftejfRz9n81%W(E1U#gqB;QC*KRnVP(yHKoi9hR?%Y$6IUb z6WUjQ4zAvd^^uQlg!4|cs%{q&%F3#4d?LOsn2A>W=gaXdk$Z)pdNcg%#$R6YW}z)< zV!^QBNvBSgJ>~4Ds^7+I#MK{zqx)@+s4=qHaGuLo)~;tn?xFbGW@EG70ge-h3c<)B z$j_CxWtq@7WmV3HovTE5d#j9^TqQXDpEFQV^T+Lupwj;MeOC0_zKFMRyZrGW6xnxSF3F_9sT7Q?E9h_R9rlM*4*dn>)yy%R6u z(?SJexNRdlqLl#m4+UJ->>5mJQ`<|4N@}zM+~nkd*%htF6!~Zi>WRuUc;DcT$xZQ+ zP4~31MyYQ+OGB}BIUq8D!*~N`cUEJc#_)T9`8&NokH2^jPYUHLt`JzJYIPb&^O>4AJ)z2A@snYvU7~w&VV(zGY4rSDf6;`*_ow6!?ghEVd-M z2{_qA)HcK()>xvOI7QJE^WS84la)BGYf*MR28f=h@Hp|{NsdGhfqMhr3q($c_(FQ0 zf586D1HoWCC4?^~d^9G>vKZG+j0%erOUMT{N`Vc~Ea%hRb=_bkJS8+v44d?SG+lW- zl;8JHD#})ro%x7D%9eebLWCG0vM0%I?EB25MD}FOG83{dL-uuKE&D$9G1jbOH@lzj zAHTn!=XIZR-sj$X&U5d1pEF16`QhRbuu=6AtbLaTA9C@;_QHEnQ!hL#&4p(Tp(%X6 zQKElyXn5v9zRxJ+<`8ay{rN^2NgxXYL?#anvS=s(fcQpX*qEhQ_z|gba!fJg_*z8$ zrFc3r{gO~Sh!#$hp0L67!FGvAyV$+<6t=Qak$=;MLE*pb6;pKuU}Ra!9?gWh0sy&{ z@3UNfa|q@IeXW#+Va6(7#1-pDpOiM?zt!l`Q=h&9VC_3ldRvBcMD>vmp_2#S#`RBl z!P4ooCp6wtV!{4PTNYON0;VIsLH|v-25b1yL6d=!8G;5IIq6KMg9flBI*dcKwvLjaw3E7TLRy+j$fE=nYgHm^sJLA@2kE4@hlp z{TIi0PWwL(d+7p5#G=SiWH^oz+q2m1`xP@+~kY()3AM}4+Pbe+a411sIYCZ1yb?neV2B! z4HJs4=1#|jNzaP#t);s+p`uIo+r`_-@H?oqT9yv83k*IKTwY|s@&pC_46%q5Xt^ng zx@Z#>RIA>QvO$M6ZlIb;g#zEOE%nnJg_x1yQ;hVHy(xK&?H(6F6G64zsGW|+>~aOI z6$Ae1MdSqRzgR3@7YdQ#6Kd`IKAzKvg&%QgZmO>1>mWuBmm>ilvO4S&^IYvgm|bMu z;=90dQ=5o>ne+Zi<5HIRQr`xvF^OYMo(PGg4v2fD4cyk%meK6`+QO555Y<*_r};~0 zVdZe==>dZ*VTZTCu$pd8-rtE(1u0fr$l6VYEAbhlHyoXUAhzh*xzZifsn^_?z^Yi2Zij@`Zz z1TBHloBiwS9LnRhfbNhhHOJGkNy$#9qBKBzG}e%WZ7_+{yF#$BCOx7-{oWlM(oiQ> z9c37z-fg7iV7RJ%M5N{!D5hQc7Sx@VYLk9McKuAvY5x!{->gb_cBGOFWI9N@D3npT z=CpmU7dc6mQ`WZ3n~BK|NpYs+y*N}RlOuhmAw+3?n7@H(0pXFbN_RIB>v#L;^!E@< zosyW%mg-9VlI{5U(6PB^?eeKEwhXoD)i_1%!-`5+0sPf=#%5WR0XJ#^5!M!vrgm8P zGRc?PPP;pqZzsG9Fkm7Tha#CnWFUCLjWbT(BmQA2T;fj4Z0Tm_P-&|L=|G2Ad6Z>% zZl_?v9|5r-%n<58r7Zq!J8WasK3BxU?8U0I#BU3c%93`f_^UQJy{z&xx$yTwNWTH$o}zrVy0T>e5Lptr?=p=n z1^H8LbGjYRuBup2DyLqKzF!nL>^}T;MDfdj?(nMpWKov?kJ=?{hhlE_JokK_nx`FqqgWMrEI*nJg-QqopcQz?@c~-4+heN^MCf22akp*YR*&j#pX)=& zQA6<0tWUg~J&2VMR@cpBn|H@NLk=fOtx zL}3ws+A}r%lZT6X&MV{S-5=iHB1VZU@zXDy7kYt)F8rBP#zB}aZT=K2&#|~=elq!* zu5Hf5+r4w%?m010S=b?0{)zH^RnUpCNB%_3CMG!pE%9TwQ)yK|*SJJNvdMwKV3L4R z1v#(CTigCz+g$rG;S%`C=%|iHKN(}km%ol@5!EMnt?HSaAG+vnBNoqDr#Bp;&Co_$ z&V=}}9-nclPAAL4i(&#%Be{Sq?aC3y0pZE-(Eu7*z`C={oKzwHJo*370MYV`o6S;_ z#WAIeQ4}W?9Lmy$jDob)j)aE6bh9*T$Pr z2IJksxiwAsU&oqi<&9j(n~J)FN2vD=a^CXg3BtWbvxlK-`M>Gpj4XD8)5o;NLz1Uj zJ>BFdR(NXQ9y<0l-A|oQ{x->{Q!+)EuAuTK$nSeqYk1A#^J^TQ@OPfPiuJE3ynSaU ze|Dmn+%AqbYf;>EmE>L6@3A@dlQKi{Jm_SSk;3H~-+QG`X&8Mzq|Z zDQr|qS%m4Bn*8~ELbrJ1cNq93)Pg%-6qjOXfe01MUvCaDbWz#Ao}bEGo}58sobDw$ z-pC&mjuv6sX^S^9Iq6b!f+bI(0r?Z-c`PTrachr7%c1^vj)pO|&x+T4Y4hRL;9r42 zz5*G%?vAnyHDyJ#Hohr*jA#Z)4o+?sVe$ayPWWQ=bO-x_wD>+@^4FRlq!@wxZskAS zkWH2wf+S-Nz{a!W4vFLpiKvi-t&?sAt;Zb38HYuYMn{Qrocsjd{K@YKBbSE?PeuE- zkJKgurhK)U*$t!B%6?8%AK;Rw&Vs&=rNTmWoGVG@x`S<0z!PBZgg5kF{?siVQKo^$ zgwbT{WK$3^V=Q%VgY$fj3NRv{%_L71mjA3FwqMg#O+%Q!9SBTn z&z+Bc{-;-H^M){dL1{$x)IgK3y}$p`g;#Z#t)Ng6xPq|MAHKgA9VAB>7he%pZcFCG0%l(;ZinfjneftZ6zw zBsnFmLKdFIpa1aTeBjTGS^mDVjx84WDH)UH^J52 z2_{#fj3{$)wDm~-GF~OgEnkBvNe4ZvW$5@6zwQR$F7D44zH+NdaL8Ndp*Q*ezBym_ z^U_Fm7EH4(+L_8A4}E}iy86WRS-=Q)f``wm%7Q4AO$`GwdgA5NPI6YDD^AG883 zrI!bOxAnP3!lAq9HS6Tai7^S`c?HAoYLF=cI3q$dQ59ZiZ9Z-s^kZ)Ldm2W@l&gWf zxA@yuOLSZu@o)At@Z&F`zPX^{fR=CPs}Vj)bg_osJZ^w^QLg-S?Y+%5300f#r+d-U zF!s#kNG3t~Uj3+-YRD$Pjc=Aj(KGVO7%3u_q!_eIC~YFY{`sIaT3m7DWbr_Eexf71 zDJL&kKh1Qg$nT;Ch+D|rklntSz7Kx&p{;UK>10ckIUDLvM&5DjO#OkU7x-S=YJ?za zQOR7?yq?-csTUU0Kar<%bR00zx~)PDL4)6mNq`bwQwQPS`z&!}<^7{7XkU`G*3>FQ z@>LKae>8~N{Wyi}>Wbr4%uiyq!fQOVxJ7rOVC3AoS-%mv@(M)h)dn^k2wJs8r#Bqv zXe0;lTdNV$3uB~J972Z0klcBUP0&9);z#egjl2Z8)KZqiZ0BWaU%n(;R_M&vd!!zA z)?(K8b~DO$L>lDw7dzRKN%k2)k7*T4g45bwUR*+YG74e!-HD=N{+++&EW=}1Qw)Q$ ztoLy$&&_Ia9jV$=EfhWTb$#u(uyoKTC^0y2yr^iZ`1*%iL-h)ygKi`pl*{Yg$FZQz zNByAHK@;iIhwgqTHGM@7<=ICE7lr3FYz`d@aD@-_5^TWD4+2uhcH2Pb3iZAbhie=< zvMVbCx=YQCmKMSMI*WDWw2EgvSlu&;`+lXoKy^apvXWbN7vSXTLfFF!r> z>@y4E#Fbw97vGt!EG9fP1S%|#Jz$O*(o6B9)h_R2O$O@Eb}i;0@!kUUPG~A1s*iF5(YMk5UTUV}_LX zGRsvC_P0s_=E_=xfAt=(z%6MNSJ(@8{(A7|lkH*M5ue*`=W8R{;eQ$_DUY}SJ9Vh< zxh=qxCoahP`(iQg(;hWa?7-ecbTOY2LQ1rkg*-CjvmNNL$do7#-$5Cg6Q{r9ykTa^ zW;>CF8^3~<0N%H^e{Z7Y8l;ooXv`<_of4wY(Xo6R%^<@?;y{_<*& zmMpa$*kdtDq1)4sGN(XGmaAf?>wRZcx--&c{LWLxQxa@-rEpuyFBFN0lKyiaz39WE zUr5I$OyA2?zJ!qD_&;v=Z zsRcol%&RRuA50(f>g$O_m8gz6r+SZ(hz+8J#>uCrjxWM~<1ql2%5lZcSA(!3)gexP z!2S}Tq}XqKX-Xefq>L(ojlDjO3^uozqMSBz_A+LNc;f*s$4ha)(aMw@t$AtWNjXBI zXb{B>-#%WvdV+~alQ#iEo?0Kh-H(c@w>D%k--G@{oPXRpPs8q-_lp;`L2b*f!oq#jN-}cfH=TiRH5*iB}Q_-{q}qvxpQaN?Gua+3(VWzNQd`gZ8e^*~29| z%KY7yq4;?&E-cpTi1h@Ddwe~ zxz4;70p`75Fq{?jo8hX(bQa@Lk6kDqHm zv^OR71hNz}?$Y%sk{wb*sv+ik{gvp1tCjzvRE=)|2J3yldk#8J&fP`_|M9Va1XEb5 z&aM0`n8;*8V^8e(R)RgQ)diP4zE$+;(RWAv!pY$+MpgCz#d3HQ3O9uY)t77?3-Wlw z++G2rpRpI{1wy8A;{Tx=x z4J&6lN0*#Nm(0hp*Oy>gh&vI@8#N^-ENISOS}UGJ=T~{l<&zObrm|}fItlT;pki7M zWWziBZ{w|lc(b1s9*X%MGO;L1qZz0eOy>S#5W1BdyfVPi$B#NoZo*;v^k?L=jGA{^ z3qyP-MNhu>7Vr{-2SsaJIx0!o`Gpe%$ocxY)1v6fIzut|T49z^07Ut`kyfZ-xtks^>)jy)?@d_^dY+P*(RiNsTUzjzmm2#yY!~0F3QDf$+D!3?ndFs zWr3>xv;1rmlzUr_^X6)ji*Ak`<$I3s}dvyEDg98xmbFZ3{+h!4?7n5KHEQ_9~Y z5mqb>il*YPM&wTlPnbKTDx$3Y)m77*57wGv42s-<$Kk;_>j)sTt*1IApniob+YxTa z;XmJYoG>apu~<$A+mzW4C$7WyFv9+w({J0)vkh^B)Pnsf``*d#e8MW+c(j*n$POA= zfCE-E`7@#_HnUf@&gwl#T0v_KO^3Df#Nhy5vuef*)=3Grp{ZbP2Jn6{$f|ruQpmqd zwZdj`d-Ey=tWc5qIt@hlL2d{hjuc)DMh9C#92YFW6?8>tt$p&}!Mr zmft5JkB#(RL-5~2J?$MCKC_Tf76UMj_)=e(7v-4eq7d?8w`L`=#u5fB&{FYtLQj9{ zooKz2fbn0dw%-DQtf>3hV1RL2EO&4_u}`WNz2g`>1YJ+hXaIM{+9mKB$5ccEl$UV5 z6PCGpYC0`kk;BsyK0O8YTezb|s*^*vlx*I3eXoZwB0h%u*JcPR#InF~y>hP%J?vZG1dA=$&Xv7>EcL?=^Al zf{&6Idnd-Ov$XvptiJTaYIE>g9pm!07A6@oBkAHxPEx%h(uXDanngEuE*PqN<7V^T z#K}CDKF&_##9_cITfDNO?3D;&=vT{khOlaFd<5=ES;#{d}R2u&)gf(m~<@}Fw$)eiBi&8%(*3P>jXEA** zPnxaIOvJOjP52h_)J7$VMHWH+e+oG00Iy(H-(Ffzl8S%%1B`AX^+i#5GwwLU*h*%wNVjg9y(vkvdNx1Ne66oFy>)SNVN#oQq%uwZ*D zR&cBGs1DBB*zHb8vqM`bXkA^Q?v21MgH|u52KWBo8U5F+y3KbPyMDS#%V(og`y#< zH+s2$k+~!-e=%@3Z}6!f1{iTpT#0$=w}D(~W%Iu>sU4Rum05T7)%K)v9poh8M+Rm_ z(zx|;5r2)D%2}8GQ@?7dN5OyI$HJVsPuka>`r(SNN(qKOJ#kAYA^BG@m2B`UA9@+s z8MfASSe}y(e{d_V#TWS*>Y7@VlVra3CUV1aKnB52l1%r zWFtE1ql@pmu7cafhS_b{Z_b^|UZKS6C6j+{-r^b@wo~7bGC0eKT2+Yw`YDQrl)GoU zf9%J?znZSq7rshPo^Gwy9z}@Zd3kC!4wgOB7I4UnrTh|&cUE%d0GTYEbFK6hU*-{4m(U+tMIzzemm(Rffsdk-|h#^Ql8_9cg-0{ zwJ^0?|OryEaZNB@Q0lL^(U+5BJhmXi>F%P%;DdL?=iflpM>^vEd34 zXvr?Mr3Y`6ON7_^95G;bL-C*Y{pH#?7l+n4I@9{^9+bCBq+G~rrgtIlh4_B$;AH&C z|AgiVOUU>MrTd?>9VcmcnhI1G%c1|hgG3hcw=Vh1sAtQWA21IN2HO*0I=>$853*t| zlNqDHSE0pSXO~bG%VxB&(k>(|G_MP}vGxrg*Uq9#es5Eec8^30i|AsxgnF_}*GOdr z5OQq)r`?%ODBC{m|Fk;>{iU{pzTw%IEQQ+c;Q6C)G|=oWhX|{dIA17u{Ni%tAY?r99 zs^1n$xPUx(f|c5S`G@Q$+j38vFSl?xG3;+&0@Jxv(4ejavY^@Uk3x=5ooT3cQQ#JC za!NAtI(`(qMVP!WiLr$6C9=>Gh{k1}k+QAd3ioRtzC?ZSV-RhXW&7aoQ}om3lw+yO zC~myp6XOpk>t!@8O6S`r7rU%)`1eb=wAI&PfMon&yLJfI-qi_TRs5@V;}3F+m;C2S z(~JWIk8PJYE-_vHpZ(>-AeX>bwrHS_mL5IH46_|R2@CxsuMv`jFR=Up+?$;1y-ZGR zCHmVikts&x_e*p_7h0FMCqNkvTkq^#n;szejX)R2<{o>}#cG2$*U;`Xq)RZt5%wiB z+Ql0(m&jTd>rM`Horuauvn^aD(#q|9@}|KsD$t5F@oaHa(mCDOufM#=JaL1(4k1Ec zUJQx?VfeQ`0KP6;;==sbT5*&%{=;l%T2WHK-39sbTQM+E*94kS~v7qrp7f2aha@TuC{Oa zj?m8X=P=+ywgaI(;rz)rmjB;+0J0w;LeLxg3>R;l{W%P?73+sxBYu7e!0Z1CU8?s~ zJ6>7x88P?(V|Lo#-CudNqKxe=T~ZD{=O75+s{?)mLq!N?OPOjD+s@b82pX+f7keEk z-L6S*x2AFGMY@^!yE7*olHq{kl|nYV`~sXkcSL`CG8CzFoW&B^j}4g4if8hFtjk?{ zc}8WCcg~_dpm;f8@ND*3{vEJf4pELX$NheLrE)R;gcRKJqF)bf_Ed8(BE9 zH&`7~<@eAg z`P^RQy$)nq@wI$e4Dr&5Vt(AMaX{YmNMlHQ-C6wX=fa_$)S2kv2b{eGBRlWifTJpRS zGg~v$?cEHm@zR<7pv9|E>cP&TI<|4Pl9MWr=XS*fah~W^Q^LPF4MiKfvnHQi>J4P6 zd+={JrM5R}7EsIlZf8MC#Tddpp!^`sCDiI&ef?|uWzfAT(bfqY6FH2tWzv=vxb*O4 zy4h3A6(kE{dq2^|eBG11aIU2WvE9MZ7!>C$<%?CkV}KHzGj$4xzAK|T^~mOrh&b0N zLxZt`F7JLx-%W(mS6ol4`1nR?Wytuh4Gh2Ui>!VGt7!dWxpt6b5)>~hewgYSqCilA zql;tM#rZuvUfj`pmNWjcX2j-R*SLcWfOJ(9C`BvqIt`jQhnnBB)m_vmS+S2)MvkrQ zv$9=3?D}`tHn!rxS+LLdshsmEtNc`_T>yK`$=H1xm+Pn2Z)@ZZT87**hf!U|Zb!Bz z{H7f8 z;@s}dKat|Tx;fpBdb)2}>xnpj$4Hz05{BVgq)(d8F4v+&rbp7U<^IifqA1tI?kvNb zeBaa?$ll%R?IZK$!w|7hsI_V}89nBul}t;?n)$C6D^^d~RGtkAborjAzykaHUpBu` z*1aAZ{xkCFGS;T5%hA%lSA*`D9@5Z;k&Ytn8!4%96xwfWwNJ_*$G`!fdk~U@O8Dp1 zP$5Ny3=!R568zU*Pdc3}`mh*6R|bV45d5h? zN}ap1r~R<;y{=_!%zVS}iex0>ne%u8@4Il{m)d5lr_*bmcF+PH1ej%m#oB*t6;FzB zP-Lg}?B+AqKR)JO{|YR6=xutt{yO_yF~+riL%}MLlWh(WnH*p6!}>Orxe&;&27yX& zaUPY{e6*3Vu}SF%m%2&tf8TbNx&qX*pgUQv?!36b%s76=O`DMSdPP>K}7x1UT zBXz4-E5x9Y>1w}N?QdGj?0!zDJHCNzM=%yPV8tIrN1rHuA*+NW(#4z^FglFi=XKQR z6lj*i8Vl?H#C|O=N*Inj2h+`9@tW@y)DpXoMBy@iTV3D16@}MtgBM0B(?KU2p@j;M%b;uGOz`WNG{Q<=qE{&#Rfdo-1HO z!kXsAf1gO+JK8Uql@XrN-1~0#%#6*w;n$Uw+D(^lu0>Y@AGnpR! zZ+VPPY_B^tC`K~CPw@^yrf)Sr<`ysdRalznM-{{ZxT9b5#ow*?wxq=&ZF+Y?yzYJt z#_>V#_$}icKFe#(cZ{97ZLyu(^2QmNWjixBO6k{~n*zVg8)G5zgV?E*{>{2OnbGRj z7?bzN8*9T;fYG6Mec?xZI#Seo_SLFWv{`8H{j1H_c7SR!$$s*4YISzW( zC`K+Id{0$@(>f0`%V$>6Lmr5_-iDEVL_!lQ>;$J1uqJci*LMZ0j&YR>w&ZU82 zd&{Ryt{E~z(J%FM0-G*2_h_j~Rs4#A4nEZJGtwmWgK|dq-h(OsuBoJfqlM6rONkUD z!6rb_@lERqiQ)47wI#09rN2x%pN_2G*&>Hiy82m{Z@ChmeEG^UcQ0f3^ns^&hsOjq zuw&;kI!|IeC2IRyTJv=FZ^*qgJ$9$Srmf!Sa<3KLLcWUw+-!ahoA*`suJ*7utJW~S z4NkE58?YYk`H%jgRmkx52?_8U&JE?hT_FSZaJE=Wxn=(e$qM5^H8l%0xl{XQ=W9LW zZeQ2iJ!yB12$vn>Ztvwa2m5!Ri^{#fqu0%2o7Ic!j`C&}dgstZDOg6_%E7-YeUU4* zakXzJZp_+Dw^_WG`ZGMjWb=Fk5Uj~6E>soChQv|cEkAo-@V)|+9^aU})oCBGUNZ3B zj&1q@Ip04GFn<|PYu>2tcf%m{q}tzmYDerRLy%Keye~+zCD^G=D|kW$%^j&{X%>DU zZ}+QToj()p$a4J#IN{+GGTiB*H4Q-Jb;G47sscYea_p6an8fs}r9x%CX{Vt0<*$q8 z<~NfdJHEV@v8|y+*>Arx@B?}sGZ$Qipi!MGf=q16ipjUV1#2h`yrHLqmG(^V4N}cM z?+$p^rcVFaoWv_>j(X_;MOWCnCoI^|OyrA0;l?}SC1*P^gQOzY8-rkV!E4|B%M6Ak z=;I3L%f;)`rXOTcC#wry2#|)B8$Vps-73KTq;Cu=exto(G0WvM;vyvyl>KM(7i0NO1_U5%wL0$r_BeiC09R^?xYUQXG_#Px+Ttced^sf zvG;HE4D?+C{^n0(;ZSAUu$#8VDW5)p`rg?yIAdCDn>MGUj)1|aJ9U_EJ}gsUv&wyw zzk)Z<2M}x^vPq+($2H8SseT6JneEn=nD|ye;HePm89xW#P^cl!JmBK^0E%Ct86ZPe|>%~vARAhQTk&pz^cQ|Zn3K_KA*mYpPE6Id+Pwq z)oGm$?C6n|cE}+CbGe*a!+TWeQ4P?*+c)CyZd!TcAZBv);oG2=XJm5~rJrK~xXRCh z@Q<&iH=AXm=hyfPRVQFnWs-E`68cYIDh2ZwyQ?RBaq~?wTaBm!KNq(dTOPGhE;*N2 z`}#=aeoTDn%(;88!UC<|m-4`{iz0CiP6xl9eF2x;CWui=DR~FMXrRTcR_?*vOehvv zJla7~5R1Exs0U-7+6q5`GCW@lIau8IG!ha*x7qQBWcf!~@S!cEdTRZ#-Kw=1m3bPy zRQXo4Ol%BR_v8FAV5LCQAEZrmgM4ZoF5`E~Oz7d&>STI?nftr?40mgsD^aBLb=nsf zAYXVHf?0hR;c82o{g& zXFt+L)FM!Hg~dJCs|ahA^K$h-du2k%>|Ye|E&(jO*6Z$jj#vepHyTohJ2z0Rt%tBh zD3!(t%4@Sv=-!5u-fXju=5Z=qEd3nV802%5`sk$#4oJlfl&0;yUbNHOalR`LnJRJc z0Dm907yCsr-3(YVy)(9;JU@9+q{-my0ctRYj`HeZAn#0;*m_H!vvE(*3{%Q018wXB zibMGU{Ozd>1;DhUr-oUDBW#?#WdA7pz_Pan;|rQ+gA`99`g= zZ@HawwD$dQnMKKJt{_Qd1?>j*yN-{_tghq?b0t$Aa=D$7u6E+rGGk@(>)ITf{o(6{ zgUv=l=Z&$=w`I4EmNis}ec2FVpf7bHcg10>hJ?A)%9+qcMZN+g3@)5~X>rStvlz~E zjtl13=RvNRJ6VkLi*T%rqLYV`73*UZ99{bmM0$arKY0Fp}wQIJ#L{)z>ARE%=)Wi&As( zLF&ypbS9#6*YYHkomKpcCKWijXd8B=4<_j7q8{2D&#{i$@%n@cJnXk*ed!ON<;I0w z`JHo2aa=(sD?~#&|Cuzg{||`QCoFl;H!T>QHO?z4Zdb*WrJ3_^5nQ;FpqPm7o8PnU z9R2x0t3q`tkAX5S)NY^513-{yxoYi=5;bM4W1vg8x;}*YLv_{%ifhcFj-ID?(^O>c5#m&Zt=G zt!aP3zu>5mJbd$=u^0t`TlHx2-#<>AF-mkwLAhm18VywA``<3ce~fPijBQIfYZDc& z(+XCp-v9CEO?c!VUF=~y&GDUne0tE~@Wmd?S)dXP z&y-efZG+l#3Mac}-qi*~!IGMiB1-Hc0WkUmG!HJtB|Jf`AP+*ENDbw;<_R?O7)6$i zXZttF*}sDtz-%fvVAUu`LjUWuYxdbz;T@|B*JY>d0iRKpc+UTp!f$0)!QkP+Tv~bJ zzo(yUtQ+z?zcu@zP~ftD2t8V2Rq$;m?>w~eJa^?DKl*T{p@(FmE^_=| z4{{mT`QsDwS-fWJ-`8fm&!9fxb!xwNKU268MGx4n`Kntp`!oCB@IC%Qf$y~Pyw}S4 zKF#3=pOAX4#kK`z?#@kDm6||YL3Hl!2azc>vyYkCg04Brc=K<4jBWIKBsDk+H!z*A z`vO#0WFnO*n4lhC?nd1V%K}zo)8Z{gHBgTH_R24L zJ6o1bO*hYHf@W&5k;ZRSejcR_O(1^N-Hvp=UYh8ROPk z_ffQP?kL+Y>O$VjGf`E;vpKJRkDNzy(%}Q0%zoOV-CBp4z<0kf8v-o1s%x|+7Q}wu zQ#-j?@oBkYHQ?i)Xw=i^VsACjA`^Ge#_0L<4jFv+UxdDTvYV{36bQf$?gkcOwu8VIxwtTC+Or2#Y+)jyc z4+@ZJvaEA3W$Zib%?VQQkmj-flo&?OfWxeV?iy|hhsJ6uxpCf(yhdpL@w7XAS&1in zoTYDKx&~nu|_;ZaMS%qI{!~ z((1{x7kAf-`tFeo4J;1$lxGwxG9M6(CGN&;8v*3)Odd|bZf?1@lmCMs`AEF6Q!9X% zF|gh>6xNXLe$)jzYV_@87I{!YJCLuVJbF|FYo|5}SK#1DMTn75Uq~c#5%?Wl4a@PQ z9k=j};YUh89Pm76kJFGFT&%e-FMG&e4>ZXB(8+rLhzWB1`~ z4$Yy{%|q9lTdyWKhH-SL1Fpg14d5dSk@7Zla2W z*`V!+Mp?h7celSaXF;ftRWe5fzv7zQM$j!Q`a1BXIY3v9E!_k%qNYm{Cf|-RQr6}C za0L8AemEn@I0t&(67~>0h^sfVxfOY&-7?tkpRoJkt?P8Ko?hupIv&3`(5ua3=zDjJ zgpkp(6;JDmksg6O*LZQ|dts1FChJ^OU5Yz@JbP{(G{Jb(RyKT8yABkxTME0fmC*0< zk3X!5O}8K3-e*9D(c)p;$ik6|yDve!3E?hkN%cX}5<*9&kHD<;@-bplXpf00?Y~$V z%pS4oIsf)7PfS~}uye0ko+Z-gM=_xT>Qw7+4dY_VWUBEpQ z)V`Z@^wf^he))ph5L0tJWBFC<@@jwBol3|nE@z{67&VG5`*w)He zl=i|Q`^LQ)?>VeHg2X$T7UTY9YFxn-l+_EV`M1$scONZrcCcj2s#V>hV;$60cQ-+B zcMv5m2Xnb80x(usI}x8ky$;BIShX;MBK<-=gK3=a@f;dsLjm=7>bF18x9oNMZ?xIg z1mqePUcZ*y?zg_lmV@Ja2a&K7IVjaA;3*m6t=tFLDj;V4M5L;~|ViETYN)*CGm=x_{CfP?4oU zwIh#*frrY_*L%~$$7?`w z)Yw`6S+lQKJ?PQIBKP6&Sbt5culG5}hCghw?XyTrw#)4G4Z@*+?}{w_f%yAKZUMkn zKXH5GrG=F=c$*(;t3HnC;u=`~LxG=;Msa|Z!^a?rQXF7orrmvq;L7YKwJ)5N zaTf<;chD`)MZ262NIG6N!`oN35t3oCiQ}7`I%nsMMZ0Rlwwn?S{4@wNO zvG>Sl&{g^1=qGpz^*Hsnp5>pY_uX7_xVf#q!_fi=-qYyH?Q_v;u0 z1##6dvAKLNXN7A@dE7t`*Z23b&fJntc6#NjH;lwuT80`>e|I6=) zbM`bOqGy%+DO|4ZYt!C!NH$qjb(N=P|Kj%+1Zfaod?B#dZrHMr6Uz2NXvc+HYvU|Y zW%njKBGvwRdQhy;<1E`db6beGJMZ?MIt9>f*?gI;sGqBT40g>uzi1-=|C%N0Q+$uj2L{m@kNhA1|ujkX8Ps0tibL2b!-_R%-8bf zN(vW+EMdK0?MBvl{`PcfL*ik=XLV(}wKov^|CGf9AP&AxYC3aKPfKj8;5DZ-YB9uO z_yFiS^l=T~P8cx}+J~OzC_VCg%`4~C9NQfVJ00pViPRLZLxcdstXKOWE+G_+8VO3s zf27QWxp}R(*`6U$AuKh6$G6F*mtFN{Gk(x+jX&92*ID_np77Z#`_Fw>36Hde=a_Sz zuZtjW{(9NL$4`N0MYj&~95(xVma&>c&%1Es#1K`tM;wPoWh}9b3$96&)FS)b)ug|{ zZuOPtCACp-Vk@vQmhN-i*dkuEBIg-Kq=M9u{faLcpBuMIdEQvjC8ZNvZ8=tIIObaa zHBF&#rOxTGmL8#=i-UB7L~}hyHUHD@Kf$>uG|xeGr_vAph*6O#wx*=<0!n{d>HS#u z;&=@W_ro9mMIt|6MU56PuS)5-Dd)Cao1iKhHBf)|%4D-_m6oH|2qFiQvGT?%i3c%+ z7-q+h=ruLONE{-fHHVr)a zhJ3L?AxY7;uFQSLnW;Ui%4fK-%jWmx37^U>fphgK3e(5o)M3_cwd5Z6yog!4+Os7? zr7EZnoQbvkmdMN}R`*LmmBHMFlg42*wbsH6R0AHzRoMp#dP9%@Si5-b+;&5YuZ5s| zhsJC(t7ZC0!`QKCe^)2V`!P=|okLs&yvlvQlHBa})YMp++IPQaS-2cYdBu0l4L>a& zkZpmyDERef!9RlJa;AxINRlj}-E!qjh+Wby6WYTwD`0LZ*kWyJDj>)1f=H$U>2ESz z?_Wl=Xls)wo^k^;YVc69xD*nfJH zaM+gCV9HvoVsj9>aO^P!~Yl(Zuk9OtJT&Cu9~0c@M-==@nuaY3(!=Zvpn1PDrpU7U%B%! z^nx=%)4!3gDe+q(!riuo-(@!EbCJ91k$2P?OUGZz+vtoUBmfIx0|!bY3Sg{=n$OXf{B;iQWnN`{%gWw&imY(iBqSvBifrzTGPCzMXPkNNj>8={e}4adK99%y@p-&H zug~lAdcWS!A=U^}^WnDPt#zs`L}%cAe!IJej^q_q6MP>m+JC~HRwqQb4B7lC;w`a; z-2yyEpKp1dCX369s}sP>x&Rvuw@BgG2LIfViFCr`Ia-D~>pft_!+M7tT9vqrEn%lE`E|aB6mOyYKxarSrW-4ED_4I^xDYL2YzNO@Z?F z<4Nkq=je$+q@wmPff3zxx(&aas=fcA%TdC7f>NJPU1QkO z=2cNoP2yy~NaF9%%Ih$R$4#@KKZM2^vEwC5A<{Q}Bw^vBQiK24()j0`6a;rsV0%*F z(Fi4BHx5m`s<-{4?dnQ!r<5(Cm?p%&AIez8LQ%PE;Syg44n%W&`VQjn9dLkCuA_!7 zV1LB*o_ZzSWslYmSDDA>Dcy?|i2aqn`)EFoWx&g*>f?ic!-RWj|Nq1C znSlxrg1|-5k}u|;#cez(uhgmsi;A?rzxDt5`PRCZkAf5#{lFN7EYg{GtGT&^5?E+Z zBV}N3jK7Vn(sqdNBIzpY%s?9>Z{NGnx<`Q#oNo0l0zQSbrcMUOsEGJhTSU(vGb0@p ze~EStF`)^4ejofA=P*w-zdq>hj9=QzOC6eZW<~6vr|J0s7o~wNv z-cI;M6s+~MCC=KVV1L~lxhXJSb@wVPt5_C45wd&tvN>jIIE3`{D!xX$p_c*?&P`3t z4Gd6i8T+94RYOh^OpQO@U+aa6Wry9<&y;xOa4|x0Z=lv_a#Q0q3fwiUbnLXj!Va84 zkpDv=A9aPB!Q`oOM@m^{;|o^Wj^iwfa%)GG6-(C!FHh-Y-GODV^j;(<-H3qXKbSAC zUaFZlaMiS_45-8g-G;;$Kl-u@uj%8KFg)AlVE)vnvvG%N9GGnnybGEeR(;roEh99` zjnv%mTRF#GIGB1;dMRJE+eQ>Ve`q~l*gv=Y<<}d-_^IZ#y>dTr{j#egG5#4iv(Ob% zrt$pQ$!eEDi-*GMTG!gL6+%l%Tk@}xdjU$J?Xi1(gpfL1?;H7BykYB(DLa&-&}GvS zTT58Ig0Xp?RlA5;VsNshH-S$t})rZA7^nZIUwupo6N)@tq!fUn9YXz&VhR3unJfnjR zei=T-^%@%8!oP8Mes;<%R+^=rd^1OV75xVs+E9-CGov)O6!&tGa_juk!MBFLYm*4o zOB2B_n5qBmpe9t#1CHBG#o8XlFwCxNE&IOqX|D+H^C&fBE=VW60dpQ`CxpAg^<&yM zUiTeXy@VP6Z2WmgU-$FHN9n|U5z<=Fpzkk?`VM^TYvC=%0T&!*0J%stSRa6T7ZJ!S z*N5~Q)RYw+&2wN*7P}j&nJ1Xk4lC>*dUHf57oYR{-?6pV=a!vE#G$MqTcI16G%v?0 zCfVc<#zAhmY_*^z3vrkBuc>>WbcdG+%fQ*DJ&f}9lyYt%`+d8m^m#nWmaNQ|sk9RL zA`!VAMhFoe>qg&3W;jf_Et!egVC51Qjeb-5!XPrbI!?(~#4Kq{6DaAIlx4vL+}+Hh zOC|i;0&m4_bp{TY5Kiv4?PQ}DP?OKH0hNG{dnLs5qp?c%?X%iwq6KCK+Y+)kAH#(h z50k|LjO5C867MpN4R^tBVg_eqY;3cknHQd++_8$=7RE%ucQ!hxsS3F9q16o2y2_p} z?MV&nAH(>h;xMVMvj50}BKgnt2$yZpu%? zR&;Q$cJZ~0c}w62o*OfoDQ0?x+FUC$G9s1*WlI@blJmOjy0$4GsTCsP;d@$bmlDj- zGN$cgjY?1V->s)RSAd~kcK)Ce@^>pNL74@9n$||Y9j&x2MQ31-R!%K}yk)m42&KD; zA#<5CNc+FNQ;&%9#7v`hXY^1MYzA)ubJfD18v$Kc3X>KJuMuC$)PHO< z3$j|az$%Qxn@R$Icsqvv`?T=u5+rcV$F(pM8AP`IvS7++%3t)xgjyLRmcUb{@3XsY zDpVUE!CJ7(t;tXfussQ}K(#v!5Vqp<68=ua;0JsFeThf$h7P-vBd=$+Vn0uDfzKYk z!05wpG{t*SC$mzR-hysuEi~B@@%7_!3@v2g|IK<&jUxW$i;fw@4JQyf_A^$}A(xJRj#_S%Ne zxAvMx)bBg(w;>Mc4!XQkW)%bOEyIT)Y6uULlNuB3&4Yu$@-I6&!MWZuMr{QOTehjI z>)@$G#(aOA_qt^nLB`mr0$VZgQBC`M=9eei1T=KM1iJy#1W_ywYq1WkFdRH! z&owuodc&7KPeLpa|3Qux)3hK>a{Ef_3-lK>%V88gnD#VO{@^7aD8=~bGR0;e9GmDT zP@R)L5!lP$sSa7M3Zgo^EvUZhX{=T()mY!eo3kN>WZe^;4jzMC&%D1@$&6@;ttOj~ zYAQLk+b?rZ>piEU#*TxD=jIE}O*h6=*7(X+hYo5WbN(AN)N{919PnS1w#o0V1b0fk zSIn?+N@$#`_q$ObutuR@xB!uO40(!3Jew`9FW5e}w6Da^*IZY^kZ|)S%w+K@U%?(^ zbBGPfrVafNadaLyx|eXH>@)OLcjD}CwB{=qAF8K&c>Iya*8#z!cOS-2vaKsAntlJ&6s`kBiY z7<|S0giZqq zA12n#R6(2X<#H_Ea*=Xd% zX{)&EcDX5iM<}QqG$mlASyurW41ez;y434pHE>+3*C)6u5dTzUyaw(k|h@zPqa z(N_kCstd!q=Rq#q_n;~-QGU+}+HY8eN;u*Q%tuU-9UUTF^Vce*WmgYhk3qe!jsDX6 z7La_vBFxo@#zM~%j0IIkwE2!1_Qy02C>tFSjVd%!VRQ_}IA}cXQ_{L}EEbvg$%mJf zPQE1pp*sF!sPBQmDWAshY--1=y}TnA2q!8r4x0Qi3$Q&hX`c$o_~fi1Q8>-QKsbt{ zT|b^6@M|$tX)66vKV!{GPshrg&Px~c_n}Hk0I8>U=;mkX&4tn~D`o1!zfqgZ!3XZ= zP7VrXTt=M54!Z$h9>NE_Aw>C{EJzydqhrvAAe-t{Ag*MHbt3>5zbS!5_aFry%FIrw zh6EVyS~O~2w}9T`h)z)Qku?Md3x%Rp?E8z%y(LJIRnW!Ob6(6x^Bjaiss%d=B8i3} z{=gxGOE)0rMi?G?^x$J0rTgvi;S788CiL7%Iglfckq4AQ8~U6ltTPmZ;Q}Gsg%fohXz<09Js^Z71Z&8wgQBdSNHSV$=q{ z`Qki1()@ehiqs84_ueJ?WcV|5GU+VcryE~RVxu#dz(8`yGUM)X`!U>dmA>{o-$)en z6Daa>J+wYc(x$*Hm9clHo56F8Dl|^DVS5&UHO~<0W~j%sD+_Z>~O7kMB^$^k;!# z_pZ`?#CM}nm$(@_6btkm@7a~|6+qvuy)obAV&@J+JgZI^*YTF_axu#(F8>`gh(!#$ z9PwKi&X0wjFl;t0N;EPhQBMYrWX>%dF^{$>347n4^@%w9UwJGdO|sOcI0SsI3w~Sd zum6l*V#JTjJqJ6aJtDpN;z=pqG!;erZ<%4fM`6%)|6_>QKSx3X%;gKkAle3SYWc;` zkunQ7g9)g8%Gf~p!rv+TV+xu6wZ3CyGn^CK zW@k1j3}G+`)k}rJ5l||Wbd)*>1<+wyQ-%eAU@PKwf-kn^j#(e^PN;+Trhq^D5xpj8 zeXC({XVE|eu&{!^hZs5@I?g24d5j-_EAaZX39~VNuGCh(*EoRpgB#P;Xmq!a{=2@k z-K*@FY(Yh$*uJWSB!)@SaVrHHok$76geC|pj+d{0+FX!CVWh)AF^>VwNXy7p{!pvb z#gfohi&~Gnx;HGwoApyi-e5It4u9GYpe_wJ8XCPQdphHqwh>r)snBxCY-L(s(TJuT z0M?5EWuv19dVNXn62VVX#~agmv0L>}RApg+w}u%k3nS&=D|Q?;E+8*G@TlV7P6oho zLC+zAPod`T;at-pGtQ24zQ2LFxu7=rP14y~6Q}N8xYV@is*7+-g{gT%=Q%Z9ipfQD z#@cR*&9B&52$xsJfTQ#K-Wua|^r6*A%LmUJGbLda*qQRjZ+i3c$(pH|-=N|ni7F4i z;kEt{jbdHq?(%I+sL`#6CMsc(8P&m_Pmj44)VCvfPPmw|%x7aiV3S+$#Ak=`q8d0IJehxQ?2eIp~}byCFv)c384AEgVfOcKZAW7 z?64t_%u{uV!L1;_QX#l$nlo}hWn~~(5G1o7-T~_=v7kUBrc3phg|62eI7z)rb`Ach z!g>0^TE#tESqm%u=U~$XOCJs_FbmX~x;OqB;xjX1(A1VS%>h;jTxcT8Bp+1Iw@NnGE=lh#Nsryl zp8C-nm(3WKyA&ulGjV<8T!#BX52@jNoB78u$K;iCX%Rc5&y6i{T!ob@0sFqpQS-D*3&UA} zN%wJa?@tHSe9e(DkxB_-HWoJ8l9`WmHk0@DNEbBoElYU zvc3bN>3fEKMxEX_!u|X@_D~q^Mkb~E0^b%PB2x8L*)x&|XBa19OX(FT>&>R+dO#X2 zq?yoQp_74PN!{*Sh@NBi`{y<~^NrCUS2h5hDUC?Sy?(Mhrvu!kOU)54^lsh;_8leG zg(eW<>E~yL3CMN1c1v)^pc47fqJw$HLHr*`tuWuZz`mh0q64<0>8tV9pd6nQV;BTW z3u*REKI^Ab1iIaReHOFyYta%r;RkFp=~_$O0?lLwJzX&N0+yo;msp(E{r~e!5nikl z_*VHN`=;!JCON}Ny5xum`Z$PY9%$eHtAt)e$V}qh-229;pgs0h@@1sGY2E%!MK+AQ z>8vJx2`=6L28t)9;*r@WGA^7;0zlDg>oyWE;pr=BThrccJ2603gabEBWH~>QPB(s8 zJl{*4rqIkp#L(bCs~fV@k8+B(_Joo~t(Bu1BiN zD!*kyRb=XyL9YDYo$)v1O6eTT4-NA#HKEcSvWRZ#x|+T1xM20La$1NjoQ3-NEUc^i z&_p9zKE!&;(RFSnF*v(2uPTkF2p*0d6b)3I#(CEN43u-fmc!_sdtR}W|J*o8DmZQs zD?V@Dp%2*s_TgelUZtz1b|+kaOkAVR0rk>O%!6R^v}@PItj)JmSd}*yMsQa-k#ld8 z&-$CU#Npsu_O+Gen5Cp<_hzeBWCr@Zma34&V#%JevA6W!k92vMI|n$i$REx1OjE>? z+s13ym~Ixp*~vSTN$Bb4w1qomuC5Vne(b+DpCmTReE|N!ZDmsCdeL>URK=FQD@H{u zoT@1>So+U4d@OHvaotITTxOl4Mkg?TzQ(%Z@|uXLy}@@Kyp?Qq zK$!&dmh^|S1Cy%y22C!SB(HUw(qJ{K_Q!kCT8%=Fnpm7xWy3^`{*@~uhpSyS`?Ouk z>X2sGS(6|&d#nPC$i1uiGg^2LcMXu#+V?IMi|!bO3xVisfdA$XKoL<9EN~)T^xH8J zJ7-1|)a(w647mj55Vtd5;1# z=!$@Odqz+)3m3c@LA9s_T_SYRfXIuLdlvRWWMJVdkNv*Fwhe23MA-iAV|rY0dR%fl zdW9fL*z0RYzuX*ceNq`rM6}PCA*T#+CbXN;{HWHKOKQKpqDrHELIp||j73G$#-C~| zal4+e8{Vyb{kf5xNYiiad$~5#w~{*Ysk_|6h0z-Bvq&L&hYLXbqdfW75raO_laZQz zk?Z+rov)pd`r+P-&UoOA;t7xez|g>%5$hg->$qRYH6-hp(< za<51ip(e_$$15}#?SutwzD)5#Kp(tZia9P!e{sEVIiJ}K6@||p|8;MPm7ebct;1OL zJR<4lll1e8mVDR(FK?41;6-qz)XfHxm~{9`E9Icg;dgSdu23|m*F0|kzEFKn@pL9m zEj+VZ$}21l^*@$5rup`p4{A=>ltgk2)Iwau<`{RkEEYE|+29%EXh6SMCrK>Q>HCnQ zL=GuMOuBc4(h^nV6|D4o!P^f}M{-18f7(;o9E~2_qn(Kx>c%rz;Q|I%c!{aeK3IkC z5$b;liSqWWgsQGat5)s;bt5o3$r4;r)F`wC2#9IWB_o5KQ47rA!8o2YpDxl4exLAU zu0=}P{xygNjEN&0Wz*n6Qh|)1AF7ASd(J>-zfjo{);TV(z!z?)R!On7MO^|aKb_D? z2K#f)Z7b`PQh!5)lAxq{Qd>)FJY|d`DU=OH&bs+T1`gf_DQ~6I;+_^zwdT^v z(sM6SM>|2HB=w}hs9h7&wD2Wl;2JI%7iM4beeLQU4h#m*3J&2tJo=5A*%yAqa9-%s zE{CRKRU6<61gWTly|H}s76=S9hR^_g5_mwr)0$YCh!pUF&hdaQp>B^x$b+VKe{v$0 z@p#Juip+(DTkyz8RpM+M?-^~xS?4pO?n0xz;1lsv{^t^d|7P{BkA(k{F%~aHK`mM( zc`ypcdD0+NnT%Ooi&?`BO*C4|SGEPj=!cvphAD4K8vnP)JZH zv@kp_)7?EaIsa>8CjQ+oe(=ayXLrV#3||-JqvsU`92W0qoU@nZmv1c3wd=QfeYam^ zX<lm-|eF#{L%MRVAHq=tE_j=m5ZeW8T*R|>s3YL!@ywD0%6KH}{j zXS$9y;&xeo&(Im|xI6|9@7I-e&}f15&MAMIZ@n~7pS~Lyv_LcOP5ZyePqK{tKLmsg@*_7_O2Z>0+I%dQuwpx4Y|5WZ5Xmfc{-=3Tq-2e+VlG0gJ zpYXo>#TT-7W)#z5Pr1)_aL0i@e$tz_Y&f(1F|!fT!Z?4T6+ycf|87j2ZX%R(@egmq zyAz-Uoss_HxmYYYLl+>*Byy}!&I@c~MgNtMJp4Gq9xCL;t5e8?tYKDl=2SRo`F zMMQ`NL{p*w#B@V=&!*Hx-V#9W0%${{kOiC4a%EZlOvtIj42uu0bqpf7sY`k<8o;}! zf{A=(HnK1ZL#>$$OQKj&?x8{?F{G|2b%TlUH5eFrKEUJCApGW%?EXh1h7#K;8!C-j zC73Q|cv8np-ghEI@%M=_9-}ORnQ(}+xeT%X=T_!z$2yN=nrBW%@f|EiM)@es9n?DpK{g6Fp}Hbs?=?ivYM^XdNY(jxT+=UF z{(WExE=)Vr#Sw7G;l8BS?zTv%{IYMh^Cqw9pnQ;%jP2ve6MEE&rZ1QxN3 z5Yz2PVmC!3#optCA80iiBpkPj<1XCLsSg*Li@ZBpC=(jQGSJs!UBL@vjo6dk>Nbpw z3=x+(79_i!(9elgx*mw5JWB8L+#=)H4{D@=KfqjA-}BV3Qr!*^&8?zay#V09125~E zgC{k-y3NC^>pT2SOtO2&w2rla$Hfirk?8X=BJXb%4HAJ&k>d4AK|1$ImR~&|&K2)y z{4PYY(6}5rPR=byxWpdS(R=Z18U@!#rEEsGZhv|KxwI|?(Gf11vZS3?UZyzdR@RqrE@6Q6*{VAA=M*ETyJ1K2phWhehf^3lEmJ(7-|0&-Wi;xjTf+${@P>^caAAWzD z)h^}-zmW5TeDF#yZ>myP<00$oq_=Nzik^2hS?q_HG+<@HS4IDlfxkr#)5803x%UM+ z_9nF>(FLmS5cp^v$vLGfyf~MjzxlTSj$6h1NswiGV(2Hr2$Jxe<3My$46S~c3OTKv ztSAtj+5h#)NhIjTBmlLjG0M>CXg+iM+R^o93fr5+1-1siuy(1PR)kMffr0JpLtG1w zMwF+2y33z9@aFkYdhm1b){;oKR4ybazKsJV1bU$ehEq3u;7Pbv#IfLqxd8--?R39I zQ8-SUp=<>!F}$0Jg3SVO!xc>qkmY-cj7ifG`_D&*EgQDkLu(J~O?2*9<>c3x-?^ht zRu^JJD+}yQokgu?Ob^vx#CPNYq<{W?x)k){?5iES-h$KicRm5a2?!R%!d(TgIh zIQ12R-_$TUN#Xxrg>#X%0O{KIBb_dH zI~ek@71)b}^f^LXPao-GaBq`l2U2QL`wvoNkp_X}Zw?|EnXUM@q+qp3co))#15CNQ zp)-JVZ?@mSc`VWZi?p-Bkc3trUy}SHjVG9WBp4VB^&_z?=C9UtB2}9x-HWtP95|)b z$3M8&Cx+4=`R@eYzKHBcDlBUEAq!WIXcur{T8rzr_Qm#(t;`HPlGr{GXUXivpw z3%j9ze1131X_rkUSLA|XMg?vK==ry|t=$n8)O_ZC)y(wZn(55chBDGOrf){c-mKV3 z1>@B3Uqx=CNJ3()7wvJA^R@HJM+1yg?5Lh?$MnCRUt=;(<0hH!Y zzd|5WhmQsM5I?a7Lxis-GJUW&W;RYLfzXId zw6Mj2;FHU+N@sB5St_xic6;*X`9!y|*IQ5Y_2BJTgI>b*d?VjSMc$!V*7Zxb{LjDH zwyQrIU;Qv&_|ZD1VEacWdcywNW31oVFS`a6>z`9tY!Ci%ur-4?6qN3-aGjF|Mjb}Q z3;cdFAq|X&r|lu@Sfw8KvR2K$K*R{1e>r_i*e)8_@hQ#ytz;VAtMti7Z}VgpY@5-A zxYw%inyycVh8*A`O#UU(2esE;+R)Q0~R~elVFJB(pGCt^TO+ zzx%Nl^~~9q{iLE;-TsnSMtD_s+=_GzbD_`Pp20{=VKjV9{~Sm*+skb>t>4eTI?T6+ zrre5HeD@UA=ECA|f#&{s>dn)G%Rgn$JQf*vrh0udRV%so8Swg}XG3))Hdfmzwrj#gZ;^?I(Pq;iI;jc9w8B7!A z?%h6*D38UeJDZ!PhRes|UNEHYp*_HAhAcy43C1IzJK(C=^iW{-@iW z4n$r9X9_lpl+LBAhoa&iAGr=(8T7r;zBNmx-SOnmU}kge zx5<2}{0MAi9dfmCx&+!2t2RLe13!(w;Ggd7}M@`J3F8AB-gy zki$f-o+&@Q{c*tRCz)5#4GU)q*>!q@q53zw>`Tlb7v6xdS|UAjV08^sn|e7zF&|MW zadYIue0qr(|D`H-F)_9Jt6AE-RI?mt0 zWbY))2VO99(6~%-diHArGh{da`i}Oe%Zj--Tv>P{da9e%2mgIhskAzOLpK9iWvQSY z+3&#dF8#ADi_8tjqR+_}r@rdlfFj{kJTL=guljK{@At%&Dat!{s@=A(w0C!>5n8d={zHOH84 z)@EF`lQ^qkox^kTanrxlz2y91VH#E7mxdBo<=h1TI;c1+_Fnm!i2RiV1_+b-S$BpG#90 zPxe*(wpazea{2R&Uh!)h=#-2!?S6SryUf;VwEo-$<8ho`vF+XQqu6YxDB~wBuRb5$ z`Jra@{qOu|j2_R`CslT)-J?7vY&)-*A@eRO89lC}GZo45l8D|))7>Y>s3?SHD;9Ql zH7%5I0XLi`_h-6aUfOhtmobiJj}nVQAXlr?uPuLBf60B^5`|Si-stC|=E+!~nRCx( zpOmfGn{70`>^*xvFtSRdY36GtR0x&EGYpx|BV?zu!=68u^F4AIkW<()T=L zQEV$Bganpq3e!M8i@b^I^!?JyVOn^AxR2NQuHm|EGCMal^V+|r@#L1h1*@U%(9Avm zP-Ul_mlD?jzX!!-o|Inel&y)lNA1G*5Y6uJ?KfKjDSbl7$m_+4II3EAjC#oU&v>E@_e zA2sah9q8G?Oe}0F9n>bH&Y_T^D|UgRYNPf)O2x|742*yY7}>frOs^N>ggEyYO`=7; z4^MIa?%YEY-vZiYR#wvs2Zq`#Z*KNBLux+_qOTF=t{k_gQ_9M&IdwIat!QS6jAVf1 zHpa{Y6}k=szHGU@4WfC5on7fU*i=zdfZQ9}JG=gK=qlJ<_b9s7rOe4hz@0($00BT<(@Fo`=(oPJ7yljs`Jv%OAg9p2v*JV>Uy9`7|nQ!#F{4h|Y*h%cz zUDt(v8T(Vnv_Fa4quc-FQURM8YBa&?MJCV%DZjdEt-AXbz$S_&d03 z(p@R{(M|JwEl%xYj(lAZ^PDR`ch}?NquWNR{9!EsZ5)=&OlECCZbr<_EBe=h)M ze$In@F#gKzzyD$yLHTbK;!fH$O_uL97F&>4l{Zd?v z+3DYgoVm!IcbYaOmWV5X^Wj`}yvZ1dartx*_4;a3hflDzSad9J;Z=$Dzj(In^=q~z z^DR9KK|{SGcfC+!MY>BVaYd+mv_s^bgIjzFNeKMQ*77JEFS*a zJrAc3$fH*$eVkO?h73kJb${59*R(pW`qF;&VpQ28^$#0Wx87Jr6CVp3AOz^8{Ngb( z#hc^_yOEh`4McsGfQKWZU;^r~7k<&<_Y39rmjzj*E~G?^P^|UGqbb zs6vNIH=Cimt7Y{YsQLm_Il(&}5?&b@Zl+Ux5aH%eU7V#AoiY&IX!j0Ui)U1GcppVD zFUvpRojVAJgFUJtJ|ST(QTp}`@asW$+fX3o7uH-2vQqs{@!`vDM!UZP1yKug`>J5d z?Z{uuZ$%YUF$#lDg|mBySsASBYfOmG6&F|as!()~7|Kc31^`o2U8utUtHH+f_ zD(BJSHw}L}LngpYx<^GOv=~auTJ-|8ZH8d)6WK7cHHV=&I^r?)M0)_m+kC>m4xC3^ zy;Nlg2-kaThkG#`q|WCrJL=5c|Pn@i1Y@s~t z5P^LoK~+~(L8SW($IYWC&7=aM;UyRmCh8;hZ{&M%Vw2MQF{hYc(`iMcOJX|a@@3}> zi;FHKVui(|iwCx$rZi_)Y=V1~d@9j0{K5e{x9-%roWi;N@VC@g#38$4+>q*<@7+UR zgh*OEGd?-|yZSB%UDq-i*j#}odvM9*tTwZPSUvzao!H46e_(^LTq@Tq|B}V?MfiI= z?~%Rl#K-P%>*zO4v-qA}||_0mTuvIfKEt<`L7Ahm_i# z5Czltu@uV>7qiRjji)(JLNH>J`DG{HuDdvtb#EZuX*% zl_!60;^y!P6#m&rQ=><}xn9nCq8qLXL(%!HyKd0pGqQGV<5|Fs|K@%?mF9pG)x0(BkFdxbaFX~Jy7JE_ACzl8!LQ1`p@Fn^nc zZU+A&lVb6xf3@N7TkF6;P%+Ke!wVWij5M7Py5xZ+Ho{(m*=fo@QFZFchCUBLaWvuC zr3SaX4JXQJ`GRndJa;YB5^c;zbY2Baz;eu{O32T?UZ~+l43J*N6XxLmA|boy#6CP| z@6OuDAv}lV;IjDU*&7b+hJiC=_)n)6Pz<)vwXt|w{6X8uMMU~hWTI*XJC1*;J0bbPWAyh@ zf|5rFrW(%SDRx1tJ!PBMgL%e5|8@kQ15)<)ee!pcwP(PckwbprrfI&nnbW@BUr!Ji zzxl@UVZ4Y^m39GIGp4Jx@!o=S{k${suu(G&u1CMaFZ1LZd`{}HS?`hR=EU`15Ym5< zBA-3*(@Iq!XVZ<+@wNQj$2Gji2meAi+n;mp4$qJDWREHK2>f={GMg1rdu8fwiUyT} zC0}_;?7#y(WhdK|P6*Pk>5^Mlm%KiD{aNyL4tO)(7~ZAn6W)rb*93^$r$9fT$NX20 z+Uvq6ZJkpW|KVm(BaBy?h(FFGb}3UAI*Jw%RlJ zxN~mhEM8Nri5ngMrM*;fzbyYr`q8dWvHX^g2&~Nen0^6B_nbWtWnX)$Nq4NFkA)S+ zZqcg0TIMbof4=X(Yel!eOLOm@D68B5xcx|f4<;mUr8FvA01244e=pF^8u{>@%DwH0 zR}e#3HSe_~Kqpmi`urz=#sqH{tvXydEfqr8) z^$F!5cC&Z*P$jTO((|rtfBYS~N_e%Y*H`_~Q!U|7-k0qH&d{rE!^=ryN1x#hiks!r zZzHw?Bg3nYrjFF`jXj4+nqmUehOIM(?;{_Gn%=O!69cX8A^k_Gb_HX`#KLTyV1cw3 z5jiTAZq@EM2=*~(#yG_N^>ZgnJ;FE1K~B)l4^`V+H!dz-==hmH;hn9ts0%>-xoOnM zd~)m05p%meOo>wtEqyg9HQVS)mr1~q8f%feq~_fBcW*-Djt;8@Y8A9n152f>}k z;c(`n9BDD4%`u#47s*eoBzi`Mduud)6D6`lv04*|cHT1NdWY$5Nk<;>$`^%Quke^dT7f?Dd}Z7QKwApM^df zD0#vys?&G{0#>)OJ+Tg+)U1Z*kpewE$u&pEp@6F3qu=zpC2Zpg1p!b@sm~VoRD2bU zJt1Q1>1Sx|MF1hk87%}iO6>BMliu&#kD!(CmupVCThx1L1s;r}hDH*V%snx}WmmJs zG1mhFVdxFhG*&#z`G$uL?jcDRjdC;kp7fbCFO$f{!Z_yk>{p3jSeusJOdr)+EnPnC zgRoDvN#JuB#_F}z(8QsV?P=F`YLExY32(05JF-uT_8#{?B0*Yw$z9hXRU0C!HaG0^ z4jo61GLk_E92#XZ1O3yUC-7C;v$tYyby61Rp9ujpb{q{yKJhxPoY@u8c4r_R%ol}y z!W_hq5fHaJ#A3rCauHu!4qAM;S*EImSd3j8xmswhS~Z9XM$Dc@ZW{b9u(ph}3ywDp zZTWt934hyj-Wj)h60t}=HIzIi%;R}+<#dS9cq3qQrcohMmGa0uIa|9Im`9>{Z3G)G z@Z$IOi#O+0Jn@I|bsNOtr4~>zx3(%KIBy1jZ=x)$2J%}SH0nWYE)Y@*qeU2Hlerbr z^=%$bHHsGd`ns#$raLC(QJXGf>&Gw)$m5G`{7o8{G0EnL1y6b{&)QR^9sWksaN?Ip z-*k9mx)!Pg&z9%&V5_)Wb!MFJaj@Q)_bu-n_~_ROhaBrrVZV0glhE#&)7;~? zCXZ-UoAaAQHE`clhU-W9w4KHO;lx&KA$m1*Ib8Sxs`;J*dAvBQH-mq-j4kulR)>PMPn7|~X~CxmgQ~j_;v%(i z=H%dBodA&>T6&}-78tiLZx0iG52*~r81s?g6z` zG%OKko3lr{_afz9lX=gge~OaX+Taw6B?h!cFF)i}X1wqtU;YQYrv>?Ki=V4Pc{w38 zO$)plHr>L#r}4ycP&WA_7KV3YWSlfIgFRnQo;)%CH`?eiE0ShdI!k)=>v1&Ks%RYk z2d@g>cx_ldSO@_Qwu5XP9i+7n!-D6*Wt!EHh7DBlUiiINQ)<`pM}KY)CWQnQn+ytl zaI^@2I5LL{bA&IxChvJQpb*KCIyEHD$a=>Sfwu=<%VCv~*>xyq!H}|DHuIfcS3ynuwJS`&8P`epN4P2!g}vT%U7uS{-b45@W9O58 zdQ)4^O>33}%}4c&x1LcF9c|Y5j|=+6vML|T%!Uhu6SU%6_7@s^NqNFRFf%=zGt2qk z{O2`={2UhTj}6-!C%7pT!>SCk9Y@P!jd9KVFo;kI7^3Ce$rfp1*-Bq&<-Jyp(|+V; zRDfP4aUCWzhcZRf9nQpU!*1N>R!;v~WOim)g25dWJ?;3!=D6UXeD)Dx+bplJ<`TnF zk7{AOS~zsTR_UQMzX(OVhG4!S>Xy89D{SG+2@S$G$DYDTUVTtq!NtG#?jNCFwBPOy zZse|ti)7x=|NPPuZa67_urNzYEbaj53r&CkZtUMfbtY*k#+F{$RS#39sL+$2c}(cEQje~FQ9yLgJo7AUFk~A3)^hKauefNv zclN74M?yD67FH68kh$9n@`ue6#0jJux%nF+b|!sxK(t`P9vAkEN@;p3QamMmry^3? zng6O$ng4w9h&cvAXPD)nkiUqP+#pbpxfJE&_~5tOURl$Bib{dH7H={{UkUSNoF`zo z6t3WAHBDp^3`rMir8_ zLkL;&Kf)Xl?J-A(fzJ`JyOka_z$#lB?mT#VjwVBezZ{xOs@H^G=&Kw2zGj!K;cMDE zUSTn0_c!3yV*Q51#CPW|pTWR9L5_qz*XlP{vKpm(JQ{67hbFao{EKdh@{m~m8H`jp zn<*ZEghkY}^W-1rU!aW#wAohb%$(n@KC!LV5R(IZzi=W7RNQGeSEQsg{LSB?>_E_p z*EZ(L;2CvO$PiiQd-%CRtjE4&%;toQPTU*wrw+)~U!{EI!Rk8w&TD4MBT43&5o9x0 z*@L>1a>8Q?=)2Q{s*8u5tDF|=rza0VdhF)k{6#*A=eKhC+`6YrfBiX?(Egy-LaFql z$_Ihk^$2nVT}G18V-A*6J$Ux(ihKFcKZd>pN3cG**`HQ(ls5>RYIpTeif--$bN4Nk%-usl<)aNf&t ztL}C4Uyp`wqFtBjg&mYd)(AGeoyqSIOjvD0hZ>h1g#2kv%RRpZ@eRKc`weN zx;U+Idn^TLPu8EMByk9lh`TIj#;&%Eef;3O~W& zu45W%i`EoM)SPY+3${-92?QRAy}ZXXz*qj6&hOSFgR*Iz1QS zIc)ul)Ey_m{X?t4Z%LzvIs&d%`^M`d(fiEe;abJT-*>j|-I`T1XVGHM;$62ot5+?v?gf6= zbS|@91~PuJ52E4WmN#j=y3WULy-|l`USz*Oe0=OWUtJm3@icRtuCL;W2zd>ACFwKp zwR~L?Q?Xl~*)7m3$B!RzTXaNUTq)gFE#B0H&ZSFhRGaae@tg6tE?#_?e2G8z^|F8N z3nfewLQ%q?RT}R*JRO-k;%AP^u*>8ld=1d$GHDiKOge{sg-X0MT^k&v*q($r;~<1R z2ksC((|I&8_51i7^4KjZ|NI6o|3b&*ABOb93@6jk40DQ*NXbL zru;)2xcqFF#9DFBCjayr{TJ4-?6;!)oAQ67jK8;M`8(l%EN$~PDeuOo_r~t88=d}} z^Ofb4KWtCKI}I>x@QamnCn0n<0`Bvf_xFJec8%xpfuTiF-@{PWIWj=Rn-+CyxqE|_ zyWy{|9wvCk08b}WY2FQfQh?Rsdp^=#{%L~e2^(}j9p1ww&Wr@FW?%~z;|j6U4I zP~rK#lKx27`JqM96LLZ;}wzpp4($X7nPvn#xW;tkebJ#GV4*Tlt|GoHXWO4$9c zCf#viQjU$ofi?X|m+gw5;Lak(9b=?G%A^Fj6!|WO7wH>B<9^!)DIRYFFVm`9`itGx zhy6WOz=uY6cwX5l*YCNu^*cfLEOy_|!gM^`c3(p)JVzx=VmCtg9T;^7!t(NoqwmV< z{T$yHzdSsT4-B5HRu}o*Lnu#0yU1--gs=(nlHVIY0RQAREzoGy zkW0`mbJDNqh%e58s4_d)NrgM8f?ag-eW}M0ONX&iFb`$0lh7oFJ%T#4`p&^#%TJQi@>=?4Srij#MPLE?U+~-T#Lf7!h88SU{R^&s=R63W1BEsCpnI`}`V`^>#+s9qdX!!W_xX4Gq z51$$V+0-yZK9~bbN11`uBR`%W7L~E;;>NRas4=1VOv4-&g)jK=X$l~x$4pBXfpUfM zP&yj!IgR>!I*&VZ>LzgiIOGk4FOKCiP|m^3Ol|}jg0T9~a@(%U`$E<*9Mi?SIP8k~ zBB=YABwIHi!Ijdy+bSvH=H?dE|ne2 zu9~!A#&Vp_qk}^nuhUpu%9)N=V0XLu_=Xt1BF3i?z?#E$Hq!{`)HKhcZ5hsQC^kQ! z56urRn`3=Aqt%xtwEBEgn>UdUhvfPX3A!Tx42>W^4v}nqDP8qbB++$Ygk36|Znh%i(@G z4AVB@5AL7MO%b}!_Ec%XX*3PfIL|s{j`C{&h@CPH4!8Kvl(HGozorShpAX|$-@>8; z1Q#jTQX-zGqx?F3LGW%Ap2N|0gWtq8b;qO_kVhu7Ip|!VbMdx~rjLUMe15sH9Uv`n zwK=4t2laJD`G;c4qr%NO7Er0WB<{>p{ndu9Re@gEZ_iGJ{Uz|@255<^BYxL#tyX>u zTL?U+kpc0h)vY&JkDz#*YTI~h4@+;ui*d!es@VJYJ@Q1hig5!>5s+)45VHM2}YCoS$^9KyP$)6XJ8( zbO^wY=X6BelG+;^FYCpWJ{UhD(JRhB%KSAGZ>r5y8pcv=Jm~Q0NeYu5Ll{QCXNHh| z5SO`7lIhRFMY;>6P|^TycwhwKE}qVeigRb-@Z(ZA)OUE4()r?Py+&ff@k7(3bCc=s zmhqjW?TlmdAZZs4_(p~Y9UeZ~CLG31dvI_FgPClL3qKJ=h;Xs42fRypcGCz@-R&yj-~R_?Yq;q?`Ci`CR+;j^j@cMa!pqR;Tfx@I>RG zeKLAYbnkBAqV+{MA*{NE8$B7HZ^JT{0|yFce7-ezKEnl2H*ljvg9-KUU{Y5SH&ka% zomv*WH#j)l5&Y?AmJc7-KYe(nJcY{B75s3U`f_-`Jj;NOlt=e6j;WiL?aPVI;Z6?6 z^_L8ey)cQ_mH`(hzcO~SQnlO#RO`Eji>pVuR4#}c(z}8?9iMM`eED#3`RZ)`L}9yu z8yb$M^Y{z764(9&xt9wcTQ3Ih+Ay6q<@W3z{&dXvp$WbVDX`0eKNVA7fQPsys(=$( z#T57cE)zalUx2Sw^NY4tH_pLCt>G*!8-8RUzP_9f&RKP@Te$fA-FXlm3 z_|azuW6NRIbyMQ{i+TxFt`k+2zJ1v)+O(%T0G$X$Md7gII(OPc~tov11IAhiNTBZ zW;%EXoFnn&mErJ{%TDGeCJI@$SSqZw(R7Yq9yvAIuWDwsYH4gywW4>MKFI&{P^UHf&WFyQi^V$WD4oCD7^YhzhL01{h3FTTpYB;S zo$WLlbJKIvjx}%96*|V{myj+*&kpjN;Wn}mJv+!xq~q`Xjav@l^bXKM5i4ip^ z{kU0$;lT0rAg7`n8NFRdW#6d%4z2pZbB<+9{3&=>LbvUc?<;T{RcFHeD?Ny~WB zGNN{P^gtgg@aGg;rpwy!117@?T|ygWy2W$8P|CKecg``z)*Eukazd4s7N z8WT_+8FBfq7`|k~L?3q~@$%7?L#M_pUCV*v+qj4fC;GuM;YLq};VzjPN-})ikTt-{ z;OVxvBzQ`&mBG{Pa^ZT~arI1w6Z^P?h~x3E^Gz=Xogi{vTnW5rlz8~l!>a*5deVp2 zgmysTmj_21gTr}ZevGZ*HS|U`nt!+Ov{5*GLwD_t;_+M$w(2Y=5Rb>HX7}+xh=3o4 z1_lP&$6r~noT}aM>r=5BH4|v4^A$lgeoVxt82~$E0z~2Quj?@F_e?ZsvVBS&#p?~R zfd)oZt!Y?ZzV6XisN2Ouea<#qZ)SL8a4>Uvg!K>+GLV5v4vKW_F>Ssd>Fji=oSsP! zj~u7z(wHtS1&X9%Svh^wE#a}SKG6l7(|{i7gx0ZQ zNZk@%5W#~%KiKuD-~j+R9@CuS4}DYHWWH|jV-YOf(5b3I1U? zGP;-uP(0;kz0Oxy)p&*{0Kj-$VDssZ>2dF>+-{;ALZS2V1c0_PmqA>P)M&U~ozVX2 z;>;Q13yCn_GJ11M2Ga+1tq5W*U)P2CJlZ~KmLLf}g#Yx2ZztjUrZr#3burCy&_?G* z+AS>-B*An1f>IfFG6hP3Ck7@>cOmGXklw88;pD5HeyUM7<)kbfsI_yZtP|DtBm71hYau5b zb3W%e)reeOrF0QF0O|bgf43lAq`&iWq8n`NxOM3x{avKTZJwxM^3$j7>akjQF6s+X z(@A*jOI8tgin8VE=(v3A@Qfwl$+FelX*xi-ZR@pKcyh6L8dZZHG#y{{^r+t|JzOq7 zpYh|)Cs3PW+ILmb$M@`@pJkff+$kKTqV!wk^kebq(V`-s^Sfo~bK&&3QfZj($iS^i z7w$=Z`7nLG;cvHZU+z`JGnRnISireaCRlyAC3v7eX%Tq4VXNit25!s}e5cnSPY-C} z2E)gS;VVHn!i(dxVw>B_;lt%&@ZM4pSIJ__(;2+4-{^X%JTNoUW=vogv91%g6Xb+d zEFGf<#inQW6J1x}@OA=M)9FBA`1$y=azCF$cwz5R%J0%;QJCz8>@<=!A;8=^ojR|K z*xNYeczVawEgN5%5-2*cyUB$~!qcW4wC7<-czVq+{1dJ&>8KpBWJ$yq?e~Hc7(sYC zQ(OvCoGy}3IYf>$ndzBM8qy}_AHs7kFjM*bW=io3vb`%N9zPTOJiTCD4p((FfOygJ z>ii5@aL$;fNJEme%f|Is_&LXP7hoh7rUec+Z+gsB#n;)-aIka|oP}$u8QXMfT3y|P zufS6bMFE)(gQ{?@i96)ZHZ8-H@!&SaOQvKJ=oGHfbSz9h9mDcW$;`_?-Y*R z8xq_<;>R=NfG>f&ft#KK!D@~*76Ujazq3??&MwXmcMP|%L~bc=&Sg8|q1 zaCTMGLx7#aS#=P7bvo*6RjV{U6|^71&-3j)9m7FKvRnW)4LNr|f4$MHd&qq`%d_49 zdA?w=b-?AO0|nQ!>U^eW1anX|-Hhp?4kCutWvBBk)uJO!n%+Xgp!%P#$#dag?zzHr zpc8mBSZPn1j$VjR;M&zP;C;Jr*0#lznI7p{^yGASr^!6XAV1}D)3JLE{P=mN(WDKT z6yCT}XZ**)okrF4!-x`2`3QbY>w44H>cUR(y6)1w9na@VOF)^$4Bk!9DV%GX*{a*1 zatF8Wi2ERt!kez?hz(F2FUA+o7nscGBtCpkj))K2r#pqyv!UZrL^EdnwR{{?)cEXp z8ZC$Ek^tZz|K5(^fCOIs2Jo|V87?VaGtzV!PM6PXgdo@lZ^+T{(u{EN+or?YYVtpBDl?)a#QS*Y{chvDA2TCJ`1UVc zU#Fv2rD}@eyDj0fK76>+LS7np=}>*XX@PoY1j9S2H*yQ6ekn1Y*E`LifWPV%=pK`% z7?b$9hFsHY%+1Bb)AZcTmG+?sS!e*KN~0;hc?*gMJ%EK0SB+=)rZAwi4<*YfaRD@| zv5B(+1L#jD$jNYd4#%h_pb5LbPvI z#F<7zu}Apv>{dkvkdx*a{=L`8yVCqZ4#!Um`#az7cQ!Xk=-p!>i;TFRah_l0qQ(d5 zb2L57BL*~y#Qyz`JiXy>Z00TQrZZ_Nffd?+n6br$L2bfM5ApJgdmu%5gLqcHLUt-k zBseUWrPbyAzS4fhARoWnZQ#aZ;c&kqD_?EC=860cB?|EOnC>O$G2|KPn=w+&CmJ!luA5(a1J3|mebFavZYcsQ!I&lrK&LC zX@Oo0^1^ksqOZ5B@q*7TmP-O(2JcT5{I$ro!{Obj(aQ@Zz7gp)JwvbF6e^0~AAv`W z$7@05z3==yhjI%Uy_+&4?}Zl!HR5j}lkU46%VpHP@;W^dj|6ptepJyj>fU)7&hJkO zKB3_H{Yt?j;{E=mq-W=W<(oB0h;Ayr@HyaUI&7d)W;&baH>|Nuf~vMc`NDLkHB%gf z66cw7>9V}7kQfW}mKeRxXTkLb47%CNz9hJK`uNC@ze~!;0~$*(IHxH*>PDbXVrg;T zW|7@qrPW<4bMbWMB$q>Ke0Hi-NJA#E@!lZH)0&w96{Tu)8-E&a9gDdXG5}sFyhqnB zOr0%z(lxduJ)m8>AfDZBMe7Bi;y$;o;yo!oLf5wOqbFtj!^?;l`1yT+-Ng?N4LDMK zqeU5FXW;)LRBmk^SQ#ALY_~sB>77HcIb2kk>x6vBP9?iRTJmf8P_w$&rykX$(YNy z`hrE85K=|)e+*b+&p$T3^v(cJkKCC7 zKo-Bys2T-Su9`YCp>q3?3U?0R(h?xQ&L81KDJZal54T$jd?C09J`Q=2K9dqHmsYL% z=Wzv0e7OKpI=Y)5XTS=+8Iw<>^10&kLwL^CH4F3ghGP~K2;#%g z_H6;D)F;_rq|0`(eod!KJ-%x+^k!h4AtArhU~8hnmE%jGspD)}^*d*Br-p{;?fFH` z@gja>sx&T5_oII5nK7&Q9*I=1q;PSMe~v9+Mf@&qSvG|gCrFqS&gd!Mz%SE1Av|F3 zh@T#OYFp(H`jduHYYa0lHmy z+486XAj$s%uspU>lAqnTP7?jWjcMV0)$oNzEp|yeDV)7?q4{=H)lcs=3z{$D2hw_z z4TH`I_?CClY53Su)UV;GaL4f7s2m@9#^>vp@Y+S5P_(b&ua{c;-RW#7K=tcW*~z>V zqWXc;8oGr-AtXjR2xoIR)sO8~RQ9@v_KXV`J0t}Ess37{61(XkDcr_h5yLo1;flQT zC#Ff^^5Y^}HCPu)3V#h>Gm`jE^`qP=I^xF7Y2SXC6kcgsRovVo3Z(jtQucW!8X0|6 zze}~z3G=d9?JT@m!55Y$t-4if)`F;Z;o4$n;iA2j?Y@cOMxi`ovPGAO9}haCToo>j z=W?gW?v(cT0VIVljt&f5plg+3$(a<+?&@Cf^(XoY%H%o!oT-SP0$4_t(vKclQeC@Yt5=>VPp`)KJDV?&ARDOt`5Hgk`!*cX47cYmxRJ2 z`ynq^?7XF#gPG3bDuUCfTY9=2Y?Z}zS=^ZEN%z{Z`y(h&X0QvL`Ks7f>2%vSAiNB5?;lR+byzFT!XnVHJ(2c6W**F=A2bG zBkB0j9hfp0t`!Z3xduMCrlo~S_B70n^y2~CbiGRZ+a-3!Sq)D}pDvzfXQmnabRoFs zJI-G$kZ`#peic5>f1yrqL8T;s`i2&Ex`R76Jzl8WO;4;J4MoChM$BVcetJ5KGw}he zdEO@&p3%hknlaiI-{l$eb&*88ha_$;Oo9u&izL7G7KV4jct?})x)Hpiad>kYyGyu4 zkD=fiez-lu@%V{Z$3J@UI3zq4=7YNBK`B`4*c4 z861-%(1#(-v)@*yK6ZA5nxeY ze%wK+@3P=Sv>cA+-!5KmZelE)H@v&hp_{73h1Yxv|7}`Yx(oQR4j~i}zj= zn_Hd}qem!7+TneV_`QUlockbpK;K9K|Qy3(3c+96JLV5Jo&@ zm=uriHU{m-|BY#$zK}I)XgOq=(ENS=i^kLZ=i<`4e){Ry^sb-2 z5%4bxb*>ArGPVw%G_$XV!}RP3tF{m*5^(6(8IFf1N~A| zRBm6{61ie{USYAnTskTcx~X|(>B4$0mQuV!uh2CSE}er3QnSua;JDy*?Bh!NU6xp_ zP7gi!W|l;_0Ka4)@sHrf>7fEY3YB2JGYQVePmL5U;i>HMPlxneUXJ3}`SCMD(R|y*%l(eiYfw}n2`-@9V1zDmgR%au_OB>hsJ}=0W1!y# z{TwgpzTgD!?8U?Rco96U!iV%tnxyUq`*{&M3C_nWZU2pjcRAji7RH=Z4mXUSUi=-( zLQ3cBiIRMsG3(|Pt;!*j;H2_X@N@{?*YkY-#`3ex*bFcUPA)g!yiF6R@E3i&CAW&_ ztxSiiM@eu#-co5I&EV5feafGH$mOP&Y@vG`^styhnj2OqBN-DgK&;LoN6(i9;&{_{H%{Xg zAc0ppu@Wk;ZPnR|kqGDKmBbmCA{3Ush>!#?&L^;BK|X1oGDw%L^Yu!sV740AKSS{f z9RQ9UcfV|=MZZ9J&(ZWtD8ene!_7w*iI?8hfSwXTC*bwVygu~&YxI}}(G0_4w%asx zx!{H#j-MTWjY;SF>s}cL4-@K6HFUv%8ZX}8n3+hca5*gKf`$i1gM*9Gx)~KSmg%;H zk9prCHk}?Ct)z3u3P&fwi}NKSnaa^39pzg-{~bFM^ys+w%3RQc{d~EIY+4RoumT*^ zDwGrtIH)y9#C!CHWOn)m^A|JpxOmK8%*;hRxKxRUv{Z9c!yygT7^QGr*rmHkC&z<# zuHxcrh8~G8o_+Bpkegb?sscZT&V^;-@Fl?`ZmvIu;OX>9Y9~Ceg*TTa#p5}xP(1B; zVY}MMK(t*|IJ8dzT%>*3bS_}5r0{v@(((e0fMXp$7okAg_$J5+CU0=Qm=PCDGx1uL z`*%n2ydALrN8SiRU|UM<1Omf1orKSaWB3|L_(~SPi6XSVox?=_d~8|pXSw1u?=XdET$u;+eKDr0I7#!b zFx`h>8cI;cj)w5~D_m)+BBVvVf&Vkg{G8*VCuluks1ft?jMx`VV?mNA_@w!{$`6HH z&J|=K;7K=}w1;exmaKV3bl(L{58BtbdD$l`nJEmNNULN;}*17Fh1o*#rjHYyrYkb^%WkUt)Zf<)$@zC zw678I@O74!aEOnuw{W-*(Y{H`ab_$MPV=9W;a{Myk|F$b2%d!_d?N%uso}jzXdcA; zKRi(w3)3SU|3=lSq{RbN(!SjQeyVY~>H7A|^NPN&!twDT?ua`5(DljjN!%JmisuU< zyO?gUN0H=k$JOvqalnJ6d>O-G&U4A`Aihw(kSWu0;uMulS}d=b5fjStL@E8ESrJp% zZ0O!4i2u-ZVXB-dl%_KblU}v>ZfKVNH5~UcmfOfMtO8$-!bS0g?#;@`yD=EY;Li0D zv(>YHDZ{6yBk*VfPD=@DZm9->K!@g?Xv_!udQGSaRGImkg)eymZ!-&c1W{=H0e;*cd` z6OS~&XOcRZCl>rQ72k2{#D5=se#WU3lMIAPY1UK z{~7=>OH)2cjzOw06y|n}WXK2Lm&@Ta_*#Dc%PqH@Pb>WOhBs!;HT(%M__K3fa90m&HdzeiKXqYZLZ-v?*oX4Da^Me+qcMLe|NMpgR4y<4e!XFQDU^y6*_X=DgcN?f zFaxNjXI_%&G&Vgwk>Yy2f8M`>ye&2C0xJDk+3KR`C1esAqs;~gh3J&i+qJJ$X6(qLUjATKIc5M zJ3DRV_Qm)2x4e73&vTyhoagzR=UnDo*^v*EAFV@%+KeQ5472Hc%;QxV*ZB6O-#G~dwF19AuyNqPH@K{_@IJVFxC!z`}@+Ot3 z7RtPiX{0wMtBxmCs^nz3vhA)YFD*9SP@A0GWKi;StnAZ5vKr;|O&jdY9&x5vRbEyy zIVIUluPDr`lydv z7cv@`AIH_%xlTD5Vun=S8|#*HA9{Rq^=LYsIn4NGuRD;Q+n6_NkMgBW6ZF?V#V02> zrdO7i7KY|uTxRyG(02@EeCT<=F4CSp=5!fZS;ORLKC*t{^BGo#3%zf&nR3JDyT3Sh zF;=`ja$Z#0ZPa2ySo!;#o{lhprDU8*>C?g(&eTgurx{&E(&dRi<4dXGS63@rG?ZhE zPf2cWtT8@AH&rh3G^867Wzk?;rfzZ5@)}3hj(2SnwoUuZOBGnp+Wm=e^?jQ z{jF&i-QV=Tcf6d(3|~Xp?lxIrQzF*ql!)~?dCZt%b|K~~afJTm6{J^H*D{+hV{?ROiPpJuS<)ostOCT z%m=r1-%*9t`6Uwynj{3E6jaauOL z{A?%NR2O)%GWTCHe1@j2pH1}()w`MVmIbOv|C_7wRHZ6YJsQqfo-AbFd8lgM!6Y71`Lg;r#!WgWd55gu{C4*&-j1F|1OMY-;A zNsRh$r>6}`$sFO#PD{1PHa|H#&1PQd`lHgD4*mM!$m4`ue)2Tk5B|C7QPe3~HU2rT zI>aMs!DEgySg#rSle{ru!}U79vYGj;wx;=GBJz7{%=|W;FDW8_tXAmHf3=xEUFXY* z$R8Jx-*%0ezewlvMC6ZGW}6%8qw{-pK3_!s1hkvK&~V+gX8r{_-;Rj*SR%@=*ZG&| z{3|2MPt?;i4R0W$}yPAzga3nov+gV(D?q<^Z%p!OVf6h z=)23RjB_@Ttqip;9cJ-wuIf(qL5JO4;rgsLrAKMgGnx=vkaQrs43XncpEj3x3;p<2 z@ig6D?=$@GkC)@s=IX7}L+3a3bBoHn?QzfZo5p$3%(QbuXErMe$LAI|em+*-1ClG} zM6%Efkry3sy4SzQ#$MB))hYuZSeyPQ~;b`M`N<+Q%d_xxp zooTsQ&VlmP#Y*=$<0FfDe&r3%Lq^Oue7cOitePy_Eg3 zyJ1_C!nQAqoCU_QRUYfz2}ao^VQF%da&lN+bKPw|r+Gg7v@kY3k8_&&?_SUI{GRc8 zeV$O2Usc+8%-PZ2v!FShzqjqdSbpm^nZDOPr#J8S%xJjpVUP0U2PDPG_bhtao4@~J z;wjA?$NzW6TQ6+Wr#(`_&Yv{C&Oi_A^BSS=2bf<%FrK(DvKZ$nDvXcr8Xr8@!{aBIuWeGawm~RT$+j{GNgEKW}eyJL0<4+^z&~ZPV=b)>W(1=Qck+^f#zHufQfZ zk5$^nRFs$6#uS$nR@#coYPe)JY0JfRq+Wn@7j#K`q@;X*_v?rBRcY{Wt zV19T?;MC^#M|;F;;lAV4Bwi9EVpLUb%g@U%YUX|)sh?yr7k|Zm#_w6@Tu1fI7&1J0P)6#RzhJt**Z9JT?vd`?&@I1zdU}EJowL2yYwzRbBVhWJt?9$h z+Wz0m*LZ(+^V@g5oJSW|RaRB}ea1I?-n>}<{xi>-SE(@#zxB+s$dPAYJL!C#BhI#% z=d?RFsfS!2a#ygG9AjMH{hjkSe{ah9pYe<_&-E9Eom<>JYGz{ zUsz7x31yWN-0t#KlvQlr z^B#V@Aoo4?G3V2#VjV^jKhSsnPW`zM9l;r-+9T+fCd?aKIWiLu@`eQIUj)9-!z zvu8c@@3&C)X7lHlmyHQc(8bmm#r@4Ee7fH|zDrm||ET%?k@XKfw=ef9r>12ZzlY_V zU7uf2{x7Y^{`&798E<(sEMq`MN^*9ZbLgOql#H}2mEz25Hr;q$$asNNvH6^lJ~trg z<3nHJl6Pz|Y)mx5JP9uMbL;1^V7|L@LPfDmU!2#wyke|5eL**Qs->));2K@uZ4_9b z%Q<%#$M8m3k;CQ`6@_CW>Qh+lmbVS5VOeRJLxyH&r05f>GauNlE8F zr$4gZ$XhO}vJ1^Oc{RVjnAhDZe`9-$XJT6X_IUgmaoFE@+F-s*&p0AAhS_ZPv4tAO zuhYO_$c}^`X)C~oZZU(z!r7c{IE)u(+(KcywrZaGpE#UL|vT9Qovx99J9V z*cvLUOUf|I3BBvNp`_;G?GAg+>etq5f93s_^V{LmrwvU}@+z;;n^;1x#_6AtZ3}wO=kqr*>+ z>~nm_l7ing{+G)xl|S;lZ^`(x98 zp%+^l-^a~2--=u|rrg+&8r!=bJuAb;C5K*XC7Zv{_+e#vqe}{HRpqu4IZNM|U7k-) zs?_hgHovONuYYs*W&(-64@z@}f6FgUM5CA%x&WOdGjGE}0HCyZ5Dg>E%8zl!bt zM3q{YuWV-zHl85wSZV9nv%n^|9Q0}Ur(?I1NG_1GE|KN$b2>eZ9N$My7jZtUDs+p* z-qORK%I*`2tGxAC2D+NdMSlXGJzmKKTPJ&U$CB#K=04Kiv$|t`#PTri57O@jXRFR` z(wMAcW#fcK{!T**#~M#+n_Gq!(s5{1s2u$Oc*ju34W$p4Unj*?lOy}B{ISosnJ>T7 zJ7Xy~GK29pJX=GChFwVd!K3!3mHBo$sBKx;od!Z8p8%7-ya+Kp4iZL zRU*Hi68Zkx$oJG5XWqL`BFW4a`u^?SzH__BcVYJO{TH3jXsgbL-!o?@pTr=PulK2Y z&1b|$=fm$cpAj>i55H%=#`VO!F;HKK2p=#urvLxu3ww^Q$#&&WPT$A>5T#w68mN?SIGi{zd7bzhC~m$7tjJ;>df5k8OD0s+lj>xYKYf(LN#Uo~LgrRJesfIBmcbum-+&%oq)W^KGW!??_TWj-M>v62UhFA6%&A;*evEfT(vhfbw z=4uv-ym?+}P+r;C3C7Et-TDWiGjnLph+OkmsgW<=nO9XbsIY8oRZ-(-N94Un<<*lL zenp0+GhUP>Cl+dhPDXpZiO!ZIB%1DX$*(>if3XN~>_v2{&`|+3NlfPg0 zJ9T8k>s^~a|NDP=y#Ae0ST>=w$#&$AN)NkV_!p%&F3;xQyYhd1yj*1%*>I`huSzc{ z9FsSpr0Q>1uRkhX&NY-2mlfLZ6{CMoxsCVRRuqok%kuvFjZgZ?e2&A`)wb9EDbJVe z>w4JVWqjzmUuj`gQMta&v!uM?)%?a^a!N{CR#tB4d||^5D)Rk-jqg`9c4y{fq<1f@ z%I{uTS<+p9z${d5C_QqxyKygQ#BeuvVd#D1Sy_Y3y=H1gW|QFo8S=X8J(gdSU^E#Y zdM`%P4Eq}&cAe4~mfyX$$V6q_L!BjW$7nu(<8wIt3TyHPjns(Yw3MbV*!aEm|J~!8 z{ryp>N*ic?4PXpontPtkJn{-XgHz7Y*+n`n^qfvlbxMP?sgsOnd?$weijgrrj68cj zcW2Xgg6$dC#&+=wzo_vxyP~k)dQIE^_sl0RFdLklH86BQR#tAD=v3};#nm!Ap=wN$ zyo)iEeb4=oKHgxWvf_ef=D)x3=5dj{zin(`*)JMjl{eNnPn?{Uk|EE4rKjcWGrzoi za$n<)`d>JY{ecZd_~FcfP^SeU8sQ zwa31vA3D_7PMn=K#F$@J+PMwae?r%FLf3)hI*km6U$2qtMSH#ekv7DjalJ@ixk$@# z>b9BJF*2xsHLVo_;%!Gw|tl}3H#&^c6b^Ls^U z)$a1L!m_G9%DlX{$K+~o`1BJ-H(d8=oKlx(&c8{rvyaVYYaCx%*rSiho_*wg9V27t z2E@>CPd%JpUN~lqe707XCD&V=p>G6LHfgWfuNvwfxm=pYNp438Z{NAPj%RD`uiUhh z$eJ{bYh?L*t4G6gx+ffF{0{VMhW5h|hK0REkGyA*g>AIVA>*1Kje%EmB z$rg6odg%IJDKg;}ZZ86E>Z$NtSLpJy|{(5O)8N`Q>4Hrct*_^NMt+eNw6@{AJlP50+EM(6w;m zzOKf+#(R&$zjGBfukdpF5}V3GTUtd$c}3{<4I^DHlV*iK?7&}-Ubas97}_`$9J-F1 zU6EH@QdnU#zT?uzCXe(Q>HSlh=9kOu$zk8jD$T1L$1zHHk}+LnUP+aC*4P}bGRrrL zmpmMTS@C;fn-jox50dIQJ5nOU862T!NJMcl?={K<9c@SZuC ze%M?-UAihAJEn)(k0Pfu|FY<>HnYPTy?>AEXuWgYqYEeY-19utKjZ(7#Q>GLFE?}0FWc$JU7$av#q=$jE0g~rqc4f9Gi-o2TU9O_TzdnFo%jj|bP zTCO>svK#WH7LP5i$}4HShe@XwnCVTD!;c%~{?yDu_k;rTTLY@Byexds_`X4!e66#n zu%fgf8pd}Hnk36TL(OF@AicTijhn~DRumTMM9K>vZ1R1Ttj5=g8sAr7R6~-yH#B^F ze&!%!Ff>owQ1c{Ep&r(Qa&K0X@dbrV$2WU@P#)#n$8r0w8NavVapU@DYFeuC7W!dX z$!DkSJH5I26joP_o3M|1gpc3fbUaW>&uNv>$ z4!f_cz`U<4r?Zm7sPj3JmSdJK#@9;B37Tt1Wp!Dh#Iv}d`S*B)#_wfdA;{}9r5>$uhbt}m18{h3-bLLOD35&=PaL+CC52A8_SRl9vX z)y}>jZ^iX*o%(yKz3GS$Jo?f*g0u%G@k#Cq2(i30(R+uOvEN306nx+Nxmwm<9peA} z8Nr{a=C=*$v^m9WszmOrVwunHaQC>zY^k>ZExm!ABJt(sl^j9BJ4uNM_AgVTZiW6! z+oxt}X9scTZIO>j-`Wc&){mc^4^fW2G+X`4*{dDhY?t;~b7!DqJ28klOxO+ijbBH5*Xb`ekfX4Z*uq$6jfi!1~+cVC!;u!5_1g zN%{dsFSMnn@y`6g9+UN?X6Ou+$-(|eFbrNgwQ{<2WbH4gJ4n)h7I&;RchIoQRG!wb z^0DFe>4E+7Z{zcjP}u^-aK^X2$V)FLS=KG`(jFab^_gwQyS3EM0B4^?MV+{rPNMuq z$8BT+SMQ&yZ2Id~XEYpjc5hGDAo!$CzkQz3BVf*dZ|(K6I!&(6r0LY1ADIzWP6Bbl zR^gvucl33P^tRZ|v<<~9=CarRFeS}*pLUP0GB00ceS~b+TCPqs9LI;I#f#3J_{JZP zo`f7}zk2_Ux}?)>+d2DP&gO<)46qZ>zdRLud6~F=R2Mao|O);o$7b- zD(I#8hKK&8OAHJCzJyRO<-Fb{hh)ztLT8#sJh3N2fn-WeeX&^Y+3zEJ&~-ji0||oj z4Ozj`{lLVEW`Y0C_c_9RxWp7wMxo!v$h zoO`Ue*1=B1Jq9OkxlCQF#(ZKW0MmBsW$&hdhWFysPjs5ck@U;llyPC(v3HpDAK?u? z(y+&VS=JOUV>7d4toLDN`Q95b4Ax~Mo#(+oTK?|8@H~}$-{Zwb3cAaC{YYhT@vPu? zi+7F0;R5Y9gN6whe*V?Y{CqX}X(ia_L7fZf_&VDbTVKu!q zq$cK08@dUx{&JheCB9wCFW#UR>9lsRL%IGoz9-FA}@7m8gt0^&Dqi&OFvUn``|U`_gFYbjO(carV@D^=``IdIx6A zV@0F0Zc=FV_nzSL+!ll0513)e&hWZ2aB|8T&t!z}?hm2P(MS^N{BlKiNN$T+xs{K_ z?pI$?t1a)Nz22x~|K{A^PRtDFQ>PfVM04M_lN04G7~2$IgZNsH&&M{#e<*eJUfadL z5i4+!b@AB4qs%j>l>W`Iu~aCJqmY4{wqH+l3~?DQecqE<4zIs)-2c7TaoaNre$vGa z&UH9G*@-pWnKMHj=6cvBAJ1*pV)ToDN@4SA?`iGsG(l1hkf(weJ?M-TaZIgfUTHV^ zrMa5aQcf_lr$8@qA*-9oY{!yJb6hh0ICuJZuYit6pl)kzaC=ngpw@iFmH>TFVc4*B zJ!yJ>uyV&1sbmYO%3SDP?v*)7)RZa4puA>ttS~ai-PvNJqz4<5-rWt3Y|;C}KeB9l zyN7x5>{@l=U{SbcV`-&E4RJU z7$mSYMDcwv_>(_H6gBP;1s@=($I$LKn`bM-c&np5A3yEf?-^(rRD%5!?KHXp)szRpcp8bzmX__%j2q7*4p)|{ zzv-jAM)zGsINJdoJhD7!%OK6oVeMv0BOm0#$j;s z&73p0P@VPWgT+3gK#usQ<+ffmR~y?g;kRT-RoOvL>x0xf4-x;)8ACC^EnE1uJYG{z12Ldy)wb_VbiJ zmOcq^i>Wyhnz*~_$-lRcSEc2e?)|l{vla#O(a49U*)|BdBQ&HDIdb?vzg6?I^arDw zG1jN3B&k{GLY2>ETNhIUv)|rj;hX0=Z-HY%d z!h>rsCDad(Yri%3j(LEKZxUnv0hM{EstXCzkvkC>^FYQJ1(L3$)o`c|tE6gcKN!80 zBR;1_Y-lT0isaC9M)Ga8Kz@GI8IOO)C+#;9!&{Deu+!py!M{Fme{MKd`|Z0IF&$00 zW^6e|-dV3M5-yq5H>p=n-OlN$8CHj)JY&|pAe1Id^PimIe><}1JVWd-r?sYdn{6U*QHo|kNzR{3acJ-hGmvsb znBFA89%Di_e(jZZ%k5p+_s~e3k9lsU-7uK+LG%j^DWqr4PV7GzmbscNM`e6~0}n1Vad>f9TWZ`B|4K*<(;S5Kj^Q~$jF?YP(H1yvMi6v@XM^&@M6UVc)6Ck4X%lA?!t51>+*Bl2|aerQB>5A{% z^49t~MEB#xy-;Mvo=zMLfs{51K4>}+5?hA`=lWz0i{!(L$O(%+v*q!Mg;E1`69h7G^k%_6^MRnujycMD9@f!}*pWi``z|fLg1x70E1ADgr8`IQgGpz9L!vZSROq9(x&#USay$;{Phdd6X5us zA@>hvy?5`cuVW*veN!TNQ0m7QibfH3wTL!oYLLgn;=UkD_&X6d*Eb|vF<+as79pNS#c1pD7M3wj&k zkF^9>aXa?;q6mrW#W7NiY5h`I^0e`)0+Oil`f|l%WRpXG0ULp4M$wdvW2_uxqTnv_ zvcRx3v9lqdo(xAMvEPm!F?L-L2WGQNjDeWR7gBIT)%|Z}5eZ=xbEXfM=;QX<1nx;} z(*K5e&5pLK);y$WjNHSL4Pt_sJd}dSIJK`}*a@R9F+^ML2!yyDWiub5oOy_3LG{@> z(wsVn^cDq?PrUly3VQ`-^{Zl8QoSF$w1Ri~#j(ySvhTjZGtTj}E4*0fmQo5Cr+uK~ zk1e@LhdZ8SIr!c=;SF_f=-ueWLHnO?{|7-lfj3Rgz5(xg(DaND{NSfoyxQ+!!Cn-c zZ9fX=H^@K7HQ7_RI~!OgSV`nF9HQ=*biyFrt( zO-G6~WUGZHaD#F=a_@E71CyuzIB3C- zk|mCWqnO-r;3)6Vs@V`zZ7BrxmMbx&w(lnP6sQJIwW|FD!I=iwE#eE8$Jw)5ki;KM z24-Yj6;nP7;fiS~$wUq%^^Sj6Zo z@8&5PII>Jc7ju2IvTI&gTHUX6%JV;%(kAcADmwS3!t(wHPQ}%TDk7azQkzqS|6ru2 zUG31tqtSlqb(SoMJ!`5XB+$;9gyUx zIB`)qcXHL{uK$m6JlHeAoxahZ^m)^?^^Azx+XO(DTG03(k2%GEd{M$ z{mgNy6f*vkaOF;kL(HlJS!QTEeE$+g@zV*eYvuOvB^Xx8=N3cOR2~DuAUwNAZcJQU zsW`g_*VgZiCEt1#g$W|HCLRVol#^?H;5jh@CyJ9IcX(E591(!LxuXTL(ha!3)KK%kSZ6s}{VmlIHuOvEtM4+$yh z64z%Tc2^K=8nXRY|IlFM#Fe_6eTe_O3O~S)eua>vBPCdSCH~OF--XLvXn3MV-Xo{e z3qSZ!+4Q?ll@H38A%$g;3mQ@Jo$Y4>8sUTn_bc%+AbkEgRb2wNr94+Bvm;;f9FgO* z@uOd8p~1ycZM^)1OEXV5x!8V6@F}@Bjy4GbT}ffhdQj5`BC}I1_2ViR0-QXfwDUy- zW&W^dT_UEC#bY8Mjm5H7L(lnu?LOSaA6)Z)``P~mmdzY#9DLkjIpXV#IE5ek!- z-y~muWW0@U9<&-_#oiK9*x`mxMKG#(L+8R6)qH1t?U)G=hMtewo!k%^x?*4!nUv{m zeH7wbSJHq}RHu}t)m{7Nl;itG|I^5<>G#GpS#G}H^9 zNzklFoG~#ZkA^@82cUOPL@p#@KE;`FE&QM<;gzM(ATHMKT0Q;NlX0Yv(dX|UmF8s0 z^%t8>)-FirGPq%x9F=sDvDdSNl-X6#y_tzR_cxsBh2uqtc8`=h0vtdC;#j(oOhui? zR9Pc|Lr_Z|f1k)37}oZoaQqu$hQh~cb?{)nkqUY~1^CLVp6PT0t#0Ffncn%f_kC|{ zUQ4@`bt!Y~MKd$IRI{*0sMa`SDOdL#`@^4(zvw|RZunhWVtQR z^9v0$SdvK#{)c8V>h<{iDOvf^4JT-e__bTJvd)|jnn333;4neLC8l&f5y7cYn|7?L z)exKAk8C`W$)(wtINFZ(T(}<9uO5X#@9cyXBauoMf3-*xH|?rpXaqx=^?m^1usj+{ zcpO^F1v$Cx@a6rOhrp@f+U04x!ZKR9P><)vvrN1mqK6q-A0_5f$f83^0~Xf8r?*G} zap5bEkp~hTN~Yw3;8XBgh-c%vmKiGjb3*zn6&MfO-)NG=%K{Ddoo+T+0yAa&lgOwP z#j}*Oh-LouOc!YH#~;t9XPIvSgGH0}RMgrC`Gnh-#VN?`nFvgx^pk;_A5s!vGPvfj zX0r*Au(Oc3KV=Uu3&!uR-5#bXj(fw?mUpL&k%Z1QQ$L{y*);@H)qFDCuLwuGn%|#z z;AOs^-)uHObl}~$pP-FOP&%QAk3q|uq%CNru8;!Uv_V+tJ{H*8h3#RJ!U;t$A>?Sz z)cL+@tW11M-f=%iBID0gjT4Ie*p};ltcV%%;sabQ@w8p=kZ2(~7OKWhy|z6|n?8BYc_@M}v+&3`GOkR@}@$LZ>WU)5NkV z^>U{o`B_7~eeW#`8Pt1T)8u2=um6Vpgm%i5r*dp;-Ae^R#S8E?(|)WfM#PXQzde3l2|3xzBxL&^Bspv-02-Jk@(_$#0Fp_j75}tIZU$NSVpwJ|7Yv-iKAJ&i^|=*7p}^^9eI&JPDtGP z5S6Le{P4nzzxr~(WVwF0)zF1k%3UdBGk_ucmtoMka*p00td0-tq;g*#iDURti4U9b znZ2^PK5L=*op7lQXs)K68A#J4bN(`BfB=S%*0ZP9y*l7VrtSMBkYf;$&PDr<)UKB{ zQ~#7q5AO0~X=+l%=Q)Z-2sFzX)}@HX$K?=Qie{2^fprn7^ZiWdX_;rE_C)RMt4&$w zp8&jn-oHo<*q5(*>9Yrvk7sqF|Nk50T-!@SmF4UKzWv092{OdY)hs#khGe_-ZkQ6C^&bSv|0db=rGqUQrO$@IPOb5QQT$~TzQ+>>SZ!-pz9pss!b zZDILAqtT`>*|*Ffj!9;lc5u>s09?Z=v8x!lXaGtf4=}l3CF%fdiDz@zwIZ?8JuSFp z;+UvxSKkG8H2%4Mexrd2F>q*cq*l{I{uRGl0cJ17?TQ=Z*MaJDe6OW$m#(sr?{E4# zhws1Tp8OR1!?URk2$Kqk{zbehK*R($*NOR_(4L_Oh}1aP{bDj-B&Qz}N}D!X(sCvv_L98|pyTzNEQHI`c?^Gh z?}2q^X}ij+vVDFtD^igj%U1tkfa31F+j}AhKvFF)h&0KbMNg1V;g)*{xn+5| zkQH759FNAsb&7W{0FrqJ>&fIocyXM=D#6&@bZk+WXCP)aU?FA1HHeGvv&?r>FOs4a z#7mnC8wu?NUo12F5fP!AmRYU?8sikwD~=b(-7S;f#)sH3031E5_(mGlAU43G9YvfM zwA3wn2TDKwU@)vS;!AZbG;oi3XKap>|5X4cT!ECf79dKCKiGd&*lv7N{3hz%IZ+hU zR=|p4liB)Wuo!(tqnvqHQZE2@4_}&lftByCrZs{T+mUzD;(s&~;F(ciRoP=SB6O8g z-+Hs4820%rpP$P-WXFxmNz|s8Osv6oT{f zwbJFmGU(lv27o0!ThR6FlUyWgG_+zvxdAtsP9te;RCQ}{b~sUB^7luUdit}4^e7c? z*P@#Ls&nWcR=_)L!jl_5=|0YFe$dM(`B>3Z$7)zTUljXacbu!a`yKb_nx^GGlnOfK zQb~`hAC=eWGo+EB=RSQjePO?ABBNiZ9uTIaS9clSFzqHQoa23nmSx%<0b(?UTha6w zR@TCbFm%6yW$c9NDe=kcrdsn|ElZ(Q^4%eNQfO`@#}X2xE|IpUzuov-sq}0vg`5L6 zj06b0myk=V5Agz%OPtKZ-ybA8vF685kC+Mc#*tyV+(;I6!AM=*XsF)JB;PR_=&0O< z$4p75CG0k)%~C?vxZeJ*VppJSrtP{v>cWJ9SqvLN#=hL8W z)4n#MnlKB;do+_%b7R|Zl* zuI9ZSBnaE0Fh!P2@MS(_CI!Y0uJ%Slg%pNoI_sYND%f{D`nG_c)Q*UO-_t98iU5$H zHX5blPuX@}8kEy&UbBDcNXn(wvUI?Ju&U_L?>Qqa%PMsdn#*N2@ z{$!!szvJWrDskgZzu{lpI36yt9hD`s1n`kcVc&K*$#KpoQqv&lQ%Mn6mD)g#BlK6Xl47YNPV}qs+9lYt zVOJqq4F&dm8Z6ClJ4FV?=#?V}U^D+B-n5-(5yS@-{IYE!yseh_kYaU|y%2!O`cTFe z=l3)%X3X)UZjgAwo+r-V;o)%@({8itZVY1=zktr+HQmO?C9UuvSxiD) zx!|7(>OS1?lbg4Q>qJ_!-$fM1DrVjJ;m1YC?LnS6X|0H%i(5|-e7UKcj0LfWxV?5B zYaIJbX%U6K>gFLgJ^5wXvuzZA1B=0kj-~{t7|_Mr@-(&^hXk9=)-Ld5rFVgq9JtSP zKna0s+z^uw6)KX()=-TT|IAJ2g@a}&Db$mbp8%Qn=Qu<1t1F3P83^H0D(qC+(s@9m zg@(Zimp~lo1@vh8dn{9ZK(dHgl_d*>2_k*+?8HBMCidOD_I2PeWu>jOg~v8UYdfNp zHtE@*_8>?j-(*-r48;El0y}ZN&~aMjgd&G*7n*~)2VE?C(h;2E!Qh_=V~vahWRJ2> zFE55&HVM~Ll_iq%B-Wm?OF)XhN&Kg+wcWVY@A5Wlhyf9~mg7jKCPr3!shVHaKin#QNG(g&Ul|c4riBu*LhOEF zrIfvrlXMBO8>E1B`pB2GfO@&{MMC%PAu%@Z3Mu5`BgGDM@kDbzb2O;l%OLqX-yjY@ z(V9OVO`n?~DI$ns?grA+^3A!SyZNH5ZpdV#&z^XnJ`pVR>~q}w@O789_holTABrO_ znV%mz|6>H(L#y>)3gblm^i7`N+6k2I(+isvb}4{OaJmHD;`NXo#Chd$6CfDbxC9`J z{Xp;1QDM5(_VHz}=jWHE08_&<%)f$R!AII=Kn`^awrpYkj%U!=f?K5G8~NqtdPcch zvRCoHsYP{c8qZq2e5#%vOCiTp0F_vHqL#=o0WzQb)_o{tKKuvZuMpVRX^R32uu)aMh1m-ck z8QA72YGsLz2QI^F5)V(wEL_JFZ@bl>^OrP9Ki4n2y{`e+hv`&y*73TmJ@-kz6I_XR z;WdtvHuaM@x)Y&1UZ*Z*p^1QxVy&F7A$*U$Jc#1Q%1dh+h+6>m^qWe)V|sCfma4v0!W z!Fg!0sC6_>-w`~+F>i9Q%l$in)@-z%p5P+n8+jsU5ckIWv5aWvF8xZ!fN-#A>F#zQ z*u4ho7c%kH%(^XK^a~v+SU*bx3EPnSycr#HxiX#W4?KA>kd?}T;zDoUkh=hbHo1O% zL*&UyV5(V3OT@g_TA(IjWFuy1JVC8r-u$bDl1_#Y9T;BuH^LVdy8AX zY8?S1Va-t3#$#JQylHfd|*#jvoxhHiac^Tr`cq{A`G2C zim^qkylfX*jw5=+aE(gAvY>a-F_)pWpAJ>nB~I2P7Mp9$7sZO;I`mT!CL){V&4Vnt zpU8-nhOgAlGV-=jtBMr%Y(zpg=%6XeiRaAq^yYp;kh}Z@?YFJhM=7psG8(?YanIQ$ zMwL%;1WPteoArA3n%qbOr`HIs@zo}%l4Y#KJSrV5l#p8qA@6zFoB{K3+FSba-(2zc z*2pQdV-0ZPp7RE3u-2j*^_vxD5;CIJMI;`S_MCs%5=CkOV5_IjYpB9*DH$dX_i6(J4ay#c?gA;DZe_2RIEv3y)`l->Zb!i&15MB=JI^2^Y= ze?4(#UJJnlQEC(77XoOs9}sAr6oazW$L>EsSd-%HuTToar4SJ@?~zZ`u@AnyuevLQ zh<%<&#swpFLpSxKHgwv_h>juqlY7*gaUb~lj27fFGd_oM;uA-k=R3w!Hl`$v6x6*MH70dt9X zO3n;ue$f#uh`t8K(zQy5C8>jtIjwxaOq|>f!3Sw-GC8bOlHZVXx{P}Q9H~0@yVzJT zZ=3POip^TvH`@x=J|V@Rq<{kr4o4k&_2iqFxIZZ3$MCS&GO&&5)1qd)VSK_K;FAEU z@}}K4b!l!y^$!tvA}ra5PdH}VVrsGZ;U}3s(R-3A(}q7A zrCFL-_`IiXsEOt$Peu0R?0*YiPRKG&SXCwn=BWlB)Mok`#+8c=5o8iguUYdIN!)b4 zG6YGNloV9+@SH;nUX)J>nf@T1MFL#&i{J+WDxFOeg929eT`k?X8)UYr(Q)l$F_N(Rq(LxQVs-jIU z8#38qm(U0F4k+`1O{0UFyQHS@d9J!sP;9$I54upR;Dqr*N~%rWm*Z=*4O$g3m(L@V zRN0XWA|v2GG~wD|W!HcD#+|{u%*Vk)0KIbZ{p#>BhNjm@BWdwAOC+KRSOUbyb^)xD z4PD5E_!DwOY!O)JescW-L~)E91cQn=3`0+w*?OsT5v>4T#9UFBh3Tlc^Aq{aV0PWS zveH0FcmaKJ=!O|+@~5?FDk1YzFuxP_SCpyRfnFH%=Rn# zJrc)g8B&Rlsu-vQqDwOAP0~WZe>(Dwuu?FW1+JSwXYL-0P&uNsf8=ggL*+A@`aW-p#Gz!01u(f<#LD6VxuxV9JyoaxaJ#t$8HR3R znb_UT1HT6VW=KDastotQ@36gKP4?vUlW35TwGHIY1&bz_MB z4~;QvO1kB|0fHuWYXgy%kQWeg;UN83pSJ({8}Lyy)GXVC4O_!!L*Al(Uo^=H99NbS8F&kSR`nTLjX2ZiM(1@R1E~WWVG*n|9?~3!W z2p7Gv^0Et@I`vrI;oFI`b;G|o3!dDlv#x)wuS2AzS1MhdxISy%58bZ4TpuOXP7yC! zO-rwtts{5cJbFO540--SD<8>%l@*ky0|qyQ8NdxZTRXRP>bLYB7WJ!(ow(1d#lQ-4Sr4%UB53OOH<`%kl73}9D71R9?jGSI+X4y&|(;%mANFU@LB z6k`8%f1kSQP?EM!FQMwglw42;>KP`0;ujJDpUxfNTH>3-b|Y>503+ktWP8X7Z>mBM z(w@gU#W4r>)BdQ$89w(j_Od{6o~p*NVg1wtC%8Ik<07qPTYP4Tie>t7Y%(Cj1*1zS zHba2^?pUsIxrDYMV$72}OBLYJzBOc=i-6DJ>3>1>q^xhpX+XW0lFV)L6#Uu7`vIkShDioa${i zcJkloT7UnrW?+Lxk7{&l9UwBBFqDULrp9LniZUzC%z8lOwb3U>b@X6ipk2eNtzze830eZ66>xNYC0eZ zyGxyby3?lPnB@#K8XGFTqN7 zhHidc{D(&{oqo<@;Plf10(P!QByI?A(&11MwYGHEaU`W`907I=Ucg<2#< zdE)qZ*+?OsUtV?uGr{G%fI^XehEfc7Xx}l{u~s;1`29TlxRTy4Z!G3?Isn^?u6F>! z1u;2co#aPd7?JB02h0y;yT~Di3Y0uddCb@cbQNp9@3vcMkBAB*^4Oz~?jSFUJGo!ez((i(3H9X>yq^6h;YnJ&>ug-Fp@zt&Qsl+@*XR6R zvx;Wdp;(rlDRmTC4%s={tSp0?Ww9IxtY}ngux9yq(3q>ugkmcie>zD)tW2v;0*T(a zW|`@a5R8&x%*$=OLqDVL2Yq2icOCxl2Tha2+G4?uAP5cuDR(w`E=$bB3luOyKY%BU zBs8+sZ_h+pU~f6f`|<5Khj+-~GXExsJzp7beK~o!%o}dRxBBpRo#Jp9nu*m(d!6hk zQI(|#SZ!2+f_N!Bg-6HfjWp^`3;Po(b<#U+LBRLXejNTb$P?e)9;!brSm3J}z7|-C z-*_p|5#ZT0sVzl-$kv5H2w>k_9eowF@_oY?wI3msnRl^}icKqRM=YJ3_YXzgd72XU zQpIk-(i4riJ;&xuFz<*JvGC^+j+Mf#r=_bL^3q>h}+w1@!$iatyBuYHf$vw# zUq%xx;|zWfeHdr>3ce6`h#PJnTgH*f^%n(TT$6PdK6@H}-b}sa;hN;N;aoELE}j7q z{@GW^Bz=cUJkB#(gTQWUR_pjD=vZir%@OY|#2(?<@)(_p%kswCdG>;Rd9;KG7iHVW zcyONg6Wss9vG9~eKKJ7^x=TU(-v1Q|B;f@%Y+tvD-GKgp-wrq2D&3eJk1v^`XjHK} z0JgzWzJO;K@lp6%LFn6+Uzn#W53sPmzPJ{4C%C&BLhK#HX- zMgl!&MMne9`6E|=RLm_39MCK#-(F{R1^hscj;|s_51`Yc0&6m_xOPHDISv9`=_78Z zzvW}7s18h+b8voG@b!f3vCwi%jLOqn^Wt8AN7t#ib{YJ|Y(nVlt)&n z#8^5TGc~|jq)J_x3nFJEHXCj?rsN^k;GLO~3X`u9C0FzH1YlIb(%E76A*dnXfG&E^ zefDh3Zx21dDb)RaJNOG5DfA$V3kzj;qj71SB-XHmx$#Rj*lJf!JxjK(CyT}0TYcyz z?$TO2JDK-4rcxIAf@cZHvXP3;P8Uo`jzq_*vrG6 zWhH~A&1L`|JZEbzb7n3D1W3l$Erni;?lNzAyk<>aOF#d~Z8&si=rS}|`YW4LU6KP# z6(_xHJoI4Uiwt?!=OY|&&Au>@0@bWSF?+%w493a=^F1G_LQ9iHzLGQZZ(E9~|`u+1N;s77pes3pm&2U0s51rG_zZ2;93V2H6gaTIP zA*;2@^sH4oP&WoxBpsLg?%xE)M5jz=?Lw`L<_R~OA5a}~YF3i>Tk9GrxH3HgSj>_I z@giog>+Jv%s()|TC~{M9P383)!QMW@S2)+Xs7lbgQ6?9kFBqNZN$>wu&Nl?%JHTP(l^NPHuuiHRGLDDsUEQI}pozgZr>#!cy zkc|(6-tY&6?nqa%U)7cBY*xGTk5o0^bf#*cGUHMK;A?O`|64E+@HShJ%(>74nw3wx z=KqHVeGdxAH&4K1_r6m;O8ilymtEKpXn*p_37|Y*G2%P4K=%SOlOFSI{WXGoa()fy zs1cm!Y`{Z;eOT>~eEIKE*nH0RbXwdM8zF!Mz0dMM5-+`O83nZB`-cr2`**QyHh^73 zYn#c1g$T*hqb~Rb4p7$AMg}p!7hmojdv?--M}GlBgSE{7rIP7g)RYJ;A+mrQUi5?= z3mVD1lL$jy1R|(&)@K0Ajq5qjqCs0HkyN!qV}Z4L5}=$ydS_mA`@&+jM$R*CO4uCO1c$I%Re9Mbe5CXad1TV0L zvLlddd@7(^q+Z^BorQ26V@Im#`jq>5UVc9+8du304gjl1wA2cikg!kRx zgBIW!5+N?3xf!uQAaMQnSL`-1#}0tT`psTC;^Z3lquKYayw6WYqxVSGDwmZ3u# zF_%r-13eXBrE|W{VsFv)w{h$vPa`Xc4k=(f1L z8jf6bFt8EW*>0M@|?h4@Z-Cf7>H-O5q2;5+3>-?VPcd3 zXeKSXqde-_aG|IR5a14KuG{i4#Ch=a=zyxO4Fh&%v4#O}h>d zkEg#O*;`9L*!x`9L|yaqm7+8k%7wjUeCwGY$nx`}{$e`=A&;l5~ z1ueD2&32wcqOsjYLfqH99!0-=xCuJ$`6==1@->Bg8dkL4oUAin0Wf12X%dx|z`e@y zSg_DD7NB0s{2C?3tEsbJ{0(eFG-0eFCzWgnOpc~9&SpkZuzMbx+<(U@YG$$TsXC0m zV&5Z?;*@v%qxLt3IbtF`o5tJ24w_&957pR#FZeAYNoA&wpq z7E90O4SGrUy-(jvS5j2t zznAmEf=dAReC+v>h9iqhRE#(-?={~Ym5NErS5x6b&r7eLrea8>sO9;?we4!T(Pi(= z4d5ZhgyJ8vzls2%%lu5vEJdY@Hmf=tlEz(%89b3;m^2Z&t1pt)5ALO&b0#IX`i_=H zF4c_Er91(<{u70&PAs?od2oJsqyTbdWvavNXuFx(;%d?pl5vA0V!!3P-MdvpYtMA| z^;?}9CA!0S&T9z3B&IP7BF=X@##{oU%8mY3x= zhLfqchYInK@8i4@+4FPyvnBg2V2{3r$dJ*IfrkASf7G+{<=>>X?sp8YzcEyuYaH=_ zD8scYX{Sb0)@KIsW4OHJm+Vq2Y}gc2)Ff^o07ecQuW*rFeg! zp>@9Hp~B>^b@0*XFEVR_Pd&nE2L%pq?~#-Gz`eY_Cyr66&-Qn{usqri+_uo1TQc%I zRBXZ%4qvxOi44g?1GmIa-&rx+nXN7{ZELYNk6&u?^;;N|GLr$Vxg)4BiR?rFw>Gw? zPtQB3QD&=Vve0>3$9t!RkFhl05smW~ZPWfOg}`WFlA%fGisuuNP`SU)@KW{lbl+d3 z&+6+M`}5~dg(X9LUx+w-shC%I7@ge8-`iE@vLJCrcmG@Qn;(Lq%nQ6dQqqGR?k3o+ zWc4I9joI&?)s3@1H#l2Q++YKryIK4Y-R#m!WYlliCkn~pg%C{*-FqF&)1 z@V5hNBb~MupL7i?vKN!Jk*a;80<5Pj6uc`He8^EOHoL-|IFhG9_6kX~5xgA*4ZP}l zbJ_I!o5+g!`|;LB7ss2{Pu@S%nDvUcHL?ItXK@UkZ6&_(i{R@h^ZX=Q{4u#%0VoXQ z08y%bCF%`49i6nD80@t*`ofCutnVq)v6fkjR|mhj>POr0UI2CWRUK?ETQ$?KUf7q6 zJjWz5a^{lnwTTq#?94JWxTL<8PJMIlti9|(cNoL9!3pPY-xBqT#pc&JH0bB!&$3~z ztZ9M4ttYW>U<$<*^BMkw4I>})PZfY+#pkY?rOYnHS<@Mb)-Qh+P%AW+S?Ln6AF8JY z{V=p)TC4dXu^FZ0wq>CV8{JLH*+S8X_H&Sx4#PQl+r>XW|;br~^ji6DQPTSk!7`Oebz z5O0)?q1U=V;j7lkbYPf|Ctp*JQY0y|o23GN8SI6I``bB&aedy-qx*(&uUe%!a(pRy z^t(THaFF+zkGq39r3aHDL)zRkD&syk!4;I$PE+_hl|E!`BHvaD@N=LkcgW>KGuYC1 zdYV%2@_o!+kNY=iY7pxc-#@3H`*w}P7$e|wNwr;AX7`HJ&}|v3hiq^Cl0RIN4C`fo zihd31MVCZ|fI|m3$IyOl z27>bPLi4HNeoN4M4GrQt`wv^!+a#~x)N;0&U0zvxX#}nHf&_uaBBIF+nrrL&ZyG|W z^7V3cupIQ*Y^>)>DmHnrmFS6f~@#M>1@uaP`{;zI${a zSA_k7MKP#1;7TzwCiFka$b%<2BSA^nu2IBtVC~aek{@!Pd}eL+lrJ9%)t5Kfnf1(2 zT5nS3kgw23o&^{cgFFMXmzwqEQ+k+`IdUW^K7Hw4&Ta0y7z*{i-OwHKi2iDWm=#&L zonz8o29DJ@FLv)$Xnh(`fysrFzXpvR-#`84i#)i0#FOj1T)EOvPUr>Bo7$Z*BK6vi z^YL;@lbpA%y(>N-{@yU=j7(m`e^x8^81R~sKd~lYhjhv(fAP~*9zJzm;ORA49HC&dDXac3bva0VW zMRR5gQ(y27-bP)(-Few7qm=<%ezdzPcOmx9j(eO#U|z+%ua=?)@%~JoKg+aFfUs7v zFcR7*f0Y1=xD%Ab zntopKxykJ5CCL!Y4+<4Qh9j4*w9B7xQy*PQ`|vbEviUaI{_5F{Pk%2zw4!=%^Y-7y zG6dcU(tMKNdeV6Nzc;|E9e-afX+4P&{@YH-M>`*dxD;^gYbg5#?T9AQQ(faaQg`nW z6o_I?n)2?(bq(D+7l5|*9)$sogVSD)Xfn>wvmzIztpbgEsg)~?9^MT zXPj4fvn5%>egL0uvprE--h5vtJD>3(xf%W;YJ;`6bUEV-YnyZU&cZLU)y(;wx!3g;+>v8w<43j?W)ty;Tl#9nf<}H79PnR-;}jap~~g~ zvk-D^__hpIr9$_O*$HcNnaxR~;ke$5$hq0-amLZ_E-pLo3j2}Rh0=Fwz;AJ-+{o?t zojr{T8&jK`qR=G(fF|pdz{zDWsfNdvV@zxo@ zM@t><1-EBxqW;IxmB%yv|M5h~U5Z>oDx|377-Nn~rH~FnMk=)2*IaYVnV6#ED`%xt zu9Ev0k&$CUo7qqKPfZCb zyFnD+R}v*aOT1j7s1;X4#Ny(Qmnb4Q%0@^wlkWzSU0JL+?EptZ9)3f!43B){VC~_9 z582G`*m);pbD2p_l^fRzLd{ali{|gaX^cWP<2X9vf49`x;7%)-TdqQnRV}^eTCQqm z;^p*bPN8LmI!-r;N8H_6r_(e-o3CSE#5#4=I@lM=SE-?@Jk;9_#YhZH~8 zd97jZr*FF13{|5Hg|UOv4swt+>G5agOvj5JR%5dV6b>%9^Nq8v5=NxU;tl-e#kUi^ zgPd5pXBR+hZeTG$eR~?OjFE3$I)Rt>!dIK~Kx1*diqKT|+^25wGpD=mÍUn;k?xudxhbD{*U&&`=-kh zIO|dWveLw1dzzYmZF~ClzkqJdx8Qm4De_%P+w{<-xvMe=(fAn&jviBMVDfCh;)bPv zanhrB=z4H?UEP)C#+jk}3Ui5z_!XtV|BMr#yX6@6i$kMZZ3>o9S5*yuWiu^`K$pCk zGjDQ{~(4$I7=ao7OZM+vU2IT%H3r6C+ z7qs|}_RxQE_$;E{YRl*-ev9>!lp^0ox?fwIB;OuYdq+yi5b`qhXTa^f(6|*s&~5eM zMC!qs;wvi@pZwVDb~~Mr+vg~L{nW2Nt+whDx)Q2`hc7D#OIP|avd!Qq_2PjMdi;`8 zRm>t@?|X2G03!5hSFop5!=Z&rw2at%4eH+yO~YaaYUrq@VWHd9!M<-59I@%?pt!ED zhO~f>n)PJcm!XC<&JE3g>X6&QU&X~^gJ+a96UAmw0x!o0f88JNbd9*6`to21k$MxH z={b0tkf>Q-meO_M*;`bE-OE&3@kY^q^#ZhkFLdWurdCtoVg4ei_&eaZ$CvWWUE#Hl zo)Vv6ouYo*CyT8UfA0VuCwvKcOuDA{0yTbR+4r{D_OxF9i|pny&FOFxT8ray;O)!Eny&jK6JBVX zw7J6yMPT{8rCC;%5Kf=KRjW_9FJmE)aKsMxOa5d_JQgwgYBvzp^z@PUbc%`&v)8gk^pM}wv!TXuhy4ZZOaNf{ZS!W+&xo(n zGn;WH2MHMyHp>#Q#(XUe!{&G^gB18iKDd0zWjhiH)wD1zJ}Gq-CLU;e6_ zer%h;=P{VYxLfYsjS*!#O8v8aMie-ndFUo{JAGjbv0B0ua+$X8M+hxl=#a zU^qT~KHW5Jx|N*1Bnctd^xfGoe{W3>A_Y-k2?gc##wiP5jmUr&sT}Z}wBB?uk6y$d zHKQ=ZB?kRMR#x;bQGX6qC-&+6uMDcd~nbMF*a>!VAqZ_EomRDU39zn5YZ3-H=;#+SwYnO-eTB$ z4EizB%jVYoo19Ss`ZGs*i#`y#)uC70rD|4;~K3_2v1^c z`pik$D|ORqYyixlYKsK~l@$~(e7IjzZrAx4rPAo=*_PinX2Iw`+rqI1nrMN`;d3ip z#bA+B4+WUZ3}Y>ouYiqg^UIv{t>%&5xo;0(TY6ihLk;3DkcQr&*`)9+0YZX~gLo-B zhl)^;MnuJu!X3no1swyMn7`quMv{4yh3{?L1?T1Y2H#(o8g&Z=Sn7i`duA^jp!FXz zAHb?fNo-bMN>NOBZ?x7VflqwozuTSDN;~C+M;4s7z5_S6{;+v_^g4-NzlhQHFEO`{ z3^pV^(Ce&xlUrwDQ_9{IlN;uvO3Ky_IC*h>DzV_(hl$b7%C`~=ACceum&DWtnrf)G zYo(3}_+peS;%Fix!=~^wlQ+rJi<#H$^qE+D?~7Fuc!zF9 zzYay9gG-NovtsEFK7_DDj|6Gf1Xz{+2JK1tUB|TsN!hT4iiKLDu^y>k7UR}eG~sKh zN-9n|T~laCI+N*l2AkHQ?nI71JdRW&9gD%rx0ji?J~h6cj#O>2u8J^-KJ0%0=dTGA zm*C#6%KbeUc!Cl%nIzQ*ho-gm?2ybq)B)U?++ZH=Rs4@1Cjs+r!z;akYIRpyR+B(Q_BxD2An2Ra%2xRk10MzlV&0+Hn^-lHs&Vwwp2=i5TP9ZY>uQ z4$O?~R{M=&$;JhqV5w$4UHa-|_|}M>p#YOOY#?Pk64R^m-c1un=&uE==OBa3iAZ3`m>P4j0oinDB9CH4(GqZ$N5d$`q;KeR$!oYp zA{*Jb-M4fwtW1maH#WMRawIeTdd+=A-O+8oLzi{L(b8sia}XQ(3dQkfk@|eDB|P|o z*zD$r3WvBwveFG8NNS(c;YX12ax1mad+v^;S({Z7VbK9VZ;8&6yGU5zaTj~z-h=Jh zkptM~`ieQyx%OrR15J(>fCjz$YO9b(Kf$~ft zuM*uQ{g*DQDJ9ZBlT}pdy-1+qevM=vhC9IgHTWA_q+r(m8-VVdJ=fVJK}fVI ztemQp7zn+CAIW#1wXom+!Vxiqyq#JKzj1*uqt^82Lfuh|a(AimhaycY54M1O>7uJ3 zr7b=YD6H^I#WhX1>F#)*GN~!{`~ScG^Dfl;DsCP7=I;3C7={FB6o)Q%=&F*Ovn_o_ z(k8Pld}jFmHY81S@W7Uw>-u3Rt#cjfFps@#4=)t&xk@eX5Jh%68wG9aI6bzC-clz< zruutnwgqW2B?dj%s4?Ea^I}t209o3`HRi9gKIEWB%F~Yp1|{M9JvS95N12$69~I`0Q4MKEcJ9`OAhx#^rYMG#q*icC^6m z$UwI8q%^i)dl;)~QjVQ~PlZA*>p*-G(utU;xHCQ2*YFEez$y6;&M2XI4$tTtt6b(f zsOE}YE0M&B-<&z(y{*qiiZUTZL_dZps3SGpGdtGDo{Jvnh<3@S_d@#f&T( zVAU-7@B9$r(Z^ycz0-mJ^yLDtoI}WLS$}=U@OFI=Dp`OZ5i%f z0Z8$&cr31rlV)Ofb9K@#$|7GwFdc*9$xK&4o#X>$q)Li$>L?wWZ`=ptVh!0F>4QP}=$rn$pJ!-xQ$N zu8}v>`~4V+efc_+n6e#rYc&7bs9zUpRoi*_Hp5}}ncZP|kE!>3JJ$1@UW@Dj;)-+x zxW6(VqLxx##=`Rnw!e4}>

hf_#<$yVTNVe_zli7@A01i{ng=!EP1Y7-hgE`kQ=a zMh>VA_%SpEGp^U5c-a{k)WLJ#v^WCCE0-^9C8|MVX$6k~otxY-yx_Zj8z3TPk8Ob( z=UntF7<8R)T7MnQxj;t{xa7aPdguz*K)T_8?E+6;EPv0Y6m_>}jxbtU_P6q1x-NTz z6dqDrwf{_qUqp-4)^gdl2z8{wCLb`}kev!}Pwhw|<1)~MCV0=!9Ps|930}ZHje2CB zoEHDV)Mk77xy`3YKcl#vXHGWoqp+N9+LjvnulXzawh;64QFj!7&@_`kB|dcH*~u1y zY|_IVl*o-WQqXtSqu5vLKOFOcOmSU;WH9NTvjfjP)B6kLrqdm;_FpctB}M!ngGwYkftLekoq7a1)YTg;fW*ly`mk=VaZ8d-w1La*c?4n|SIh z1ZZ*JdA3G1XnfpVjf75uH&Es1w*);95ki;0EepUBl=V%PL%&+79krl86nsiSiD!!T z>2yaZgzVh(A2tP%jKg5Tpp0&a^|9HgMxI|hb$?T+FT}xGZOu^v=8A0*adTQgxipj> zR`d&21KAI>1Kvsp^OIE3OKSF;hLhcTK$X- z6YvDD8h2J4E%_4&EHr%pkWu_MJRmHQbKi0&H*>^}S1RDdt}UgkaCVFnJCiFb5T3_TlAe9O&O=19P5Bj@xcP>Zxf%QK%6q?OA!YWQMmgx7n@=T&y8{vGwvv9o3gQoJh~C%BD(; z+hqjA&?Opi)FF52ien{`)irHj1c$NO!U~@Uvr$s#D)}3vKb?gvHw}Jyb;fyM;>&{a z*4W|gVE+@Pk$xM!t3?5A(o6+Q(5SB3ZIMQNx4l=(PNUq!w^UW>kw)`&MlK;*TZ(Ux zzvYqsptYdo6E7%8FYlK7gzQo<+#_@EL$%w<7UgZfy$8}UH6jt_LF`&LjeOuWI|3(k zdhG6&>bCD`z8!Pw=!mqPnQzfw(dnTuHHA;%82xo1p?faQw=&L(YbKP>wp=Uf^Yar0 zWwI1=IY-#L|D~US^qBA`02xH@0y&@OQ>R~p!1Rc;o|)+I7IaBhCfHx*yj{GLxm~lF&La}l*zGNy={g62~^&e`j%ho`|&S;%A0Vf zb*Dl`+FZ1n4o?_>LLb8uqN^yuDs$P3cpTWQDhjmSg)_UF6wC}XCq(Yg@ReeGlP3n; z8bNI-3tW8sdCqBj`uE$pYd@V*zaMWnn_d^iM$i3P?NuQM z-vsyzMg=S6WL($RP3`UU8*@LFgM=B=jv1&L*7Tf!tlWd=UbEuPoN|V}*6g?jTb9~V zwV;~{J{w>ToMO}LQGZLo9c6Aqckb)3)0QgUSK)>SOf%1dD~m1r2n+y4J%kD3o+4-0 zt!EiE6a^S5ofpM0!xI5H+V$wy7vC+bT;YWIi4Uvte#Opi^x1HE!z)%~967T$@{5~F z{=9Emik*orG(mcj_UfhlkEh$AEOp8o0hF_f09{$_*at?{WjwzqX*+0@o9?F4sm>a= z#UBHX^UUeq+bfjtm(Z$<0n>!Mz(+uqw9xZylEvhtW@`;6>NVF*)sfz~^-Ro6wgyF= z#~*8!DpBOhRROJ z{=6U&BbjbwM!H644x7SNR_iftn1UaI?7w9DMC@wgr)|_qg}*9LP8qd%{iA}7Y4_Qe z!T^D!IEM|OA_&r}x@LLFWPL8B_7a70;}y)TtW17ey!%iA=>}^NqDyI za}&?2tcE48#2-~zmF@h}q&jz-*0$mTg~lS{p37n}QGLLsI@udMYEYs$8J0Gx>jC3b z{Wn_mq>si^9n|SR8eD=%xU(Rj_uW}RZ`3Q^rYzJ~UA~`%ax5JcL&BQ+{6^JNTWHGR zN2p2K7{k`x?qfatNk>22uGW29ECgy?{3R0odg4;k8S zFoH)XZOU}2{T=P}fpiV^(npjCU>|jDK7?`ThpTlOZKJFMM?1UpH#F9U^}=5NtVMjJ zDUYv{&Q@2d7!E*v=B&iX&ZHHe`(yWxtvnP#2TB>GyP>o)0W5l#9e}eX_;`m)OU>6X zkDU1B=M(H0ZhtwAe6rlh&6oJ7x&#O)eD>yIQ>;^1NiRI!`@Uk2Hr@*{)=M$P|H$18 zd~=$quK%ub>aXzJIT}7xcs$t2jgI%djQ`On{flt3&S%%Q>H5vBD7~7cDZ}yEqaWz! zLw+3uFyFlXb70Q9QK_*Wx7p+4pMQEynuaW;sE(Wa0LUl8|J04Cdg$Y66umLx4nWpI z$1*52kOv+j-e2N&j3b@y-@y%Q#&t(k837vuCF|g@)B?wz)st)LG|Zc6(bAe-=Zchqn@#Y(rTo@Q>nq(;f`QTB12=0m>|=%dWPDSxWQ3H3e7AAAWq-YZXeYg!#3FK13oFuslgKgyC=TaqN-mg5}c zXhQMh>T|D-GRy>@y=8i-9aW_-kjABC3pY-U>qZ$N4_o-QT&TYJq<;mD&?-8T==-kS z>sk+{-uu0|7hZeP-?4HXWEQ!c$+DW?qg?nly=zCh9Cw_%wL%sJdh0TsEnN9_m6JQ# z6f>jN7=m|ORN0};`(+2kH776SjY#{%UkHi~2>#`6L+Yw6c~m`m2ACocps0D{Uap8N zY)*QWVYG@i4iByGF+w7ozF~89_(|s1Z1NQHD-6Pj+4?+3vn2Dr5GH$|@rInkHF(bfRb0FgZlk!G25?&q|19yVsb&-k z`Fuoxo+<-LS!kf6i~42hRSzI3slRkwAaXf>zUV_AC;ka5zJUK;nTFpliy`zcNh|;}d^8kd>3JmvA8dkI|KVA%)C2vynf;m=5rcnGMSqogzlUkjq4a^V zN4g(6jK!O@*x#}K^CgV#DdIi(tQWrk`4qSt?azk;Jm*L+@;ucoNL-D`e-wPVi4A(# zQ&*|Qd3SQBd4`_OL3ST4u1?Q=uJs4WchEr-htEr0megH|uTqyb>Fj252;MoE)zz8HyM4|3r`Yp@tTpSih->k zTL`d)gbUrBIV2`B{l!9TIucYC-+QMJa{dRb<-faG)9~X7{Uv~kn%N@TH%xQdDO{7s z+08gp-r>uzVD)#Z3o8YX*J7!_y5IHs-9{#?YTucTy{wb^z$G$#f`@=&9*vd%9->zk%4#HpN9?vF_)3wu=0XU_f z6147jw=)Vhy5WIf9XbbAA@s&W1n}{;NMXK?Mg3cTd)v;6H%Wg2cAw0&K?n;v=m8lM z+*=ZLfR^Y08oA&lK-7eTvIOkRFH>X1%`gUSJ($(9fR^$6Yb#{i^3K)dJixX@qWHw1 z$>L@_&=Rh37N=7TA{alTm zz3SsHznpjq8XuJ?)T8}^dL_vN0#i6%spZUbt%L{1bJL;K-pHaKH`@Xp0D%1Z_f9uS zdKwT+$V0$(L;$Oe5~~Ebxju{Yo3}}D;Tu0p$aX;z&Lax^Ha^d_WL)Bfwd8=Z0xt44 zFGN^|w7|ZAH4+qqK1M|E_Gif&Rf7OdGC|%Nh*KLN-V=>CdpavqMm@CyHRX0vcZ$jv ztti@-^B|jRCis*Mh}TmwK3WU&vA^g9PK{(^tnBzTfxniet zq$l`)0~cifG-;t?w3}Ba%l%ER#XN!6AOVG6 z1UjmC#ymnsE6a>~9YFi03+bj-)Bomoys%pBIfRUlKwdlf2WjBt`VTokMaTwfhk@ZI z2UKjbnVRx881W#d2cq(f+ZVQsX#J4w?a8qF@Ix)VwMDa z0QnnFDgV3zteuHL8+3?4aGd z9o<{01>wI|B7+dW3Ct>yyAIkM`A3g3*Gf?||rtn=oGnF+VpgwzV5< zf>&ndK+}}vmTDpc&?)%UWz*2bE;e9xH9dAakc8S|okp?5y^r$3J2n-9m%G28Lj+8> zOoIKJ)Rhm!$uoja-D)myy|POGn+lNu>J8Lf-o@aPfHH$DfP75?KN=D}0j2}()8 zrk+&ZB4vRb-0Sz-QvVwGeve3tCxCFNPmHYpxr8`kbb#z!R930a)TZJy1BBCx3LCIG znmZuiE9j|B%N24;8fvj7Ek48dGOgvNg%L5ZZUXUdC=?fV;`L%H4IA2Rc!%_Yp3Q%< z16b{^@gq-xYG#nm%e(+s1EtWW80sg)?}^Cv%clA*rPP1dUrOl~5B?b3cv%Z-CU z(rDDiiHd2)?tsqqdi?w5!>M>f*W`AxjH-TB73I(g5{zkW|1pwFc9J!!0*PKp&cC6$ z`MSvigwArnn+#o3X4}lI86$V=jNLkxd&_}bW)&6wsEml6j0yAK@1DIrq@1IY%a{24 zB;ixm%&=cjY`tDj#dTnk@PmYi0wUm6?)?n&8li@fG+eCI%vrJQ4ELoD(=k;Qf(ja`j-9IZjwO4Du!|DOSK(M^5 za-Y*`Y9?~rQ!*I>xiH7eR@3bbFh^taWt~66PXksS75)ni)!j=3;!wl++>(KH;gv7?gEqnH57cF_Yxo{_n!*J5@08){8Edgm zNwf-dXw|;?^;iM618vvNXBU)AAm9u zb^QvMBye0S2v86v#8DO`xpGz?%~{%K8)Mwo3ZpujOi90IZl6T?^tlf+mw60urg_)9 zD%|<|pwI*(^LZBk9ojQ@l^ta#PR;@pX!-x#pWFRf5#1Vx5lY4SE85B17X~V-DAo91 z+R&Igb^|4|Z56KjfI9EmDY|tByIBf&9R`8pw{nx*k>_ z{Nu452dh6=g)}r4s{a;wH@D=~xoQK%Vs1ADs2g~4c@AP&->Ek@oRk`+?DIOdTD)%#|CyiPm9vvl0Mtw=yC*_jBU~XdX-S2 zd+Ihmr?l+kc< zzpXp!gmuD$@t^wp-@ErDuXq=_FQrI-=l23TC54zQ-WpsIV3m8YSu;lkvMW3&4aGB< zp;99x=9vRdMh3i!C+)MwVr^=#4v^^H-|xn}O~3PglB91z2kwxV1Mp8ffuv+uBORzD zK=3&QSSSa$d)Kn4EwJ!?AVb?*KOm0E?(H;Xop^FY)pz~OPUJKL zyz3{V?@#2`YO7$|a*sRL&*e7qfaQR72A@O@!Z^a#aGSiIEjNs{GwLw_XshWl_-sZR zJ4CKAw*a_1QT((H^)brr9Sc_G(XZe!H(fX%Nr{y#5XMb)YriGdc?D6!(_!4C9p_1e(q&fHb9A{ZMGMwAse%Rhd#) z{Ueper|g>fV+OcOd6S(LArJNWp~;g`iM`MLi1o#o#a8nI5SCtXElrCLA@yrRjaC@1NCFIcE-*}rDD!SBh zE6=^fXC4G*6t07HPnK?aYwa%|V$e*vJ<6b+@_)f;?}>t6te#QgNF5{evww%C8J(-H zaA4;pqOGfCF}T=TS&TFX3*IvIEd5Y!;iMso#X3hYkZKZof%IZ}+h0nXmUc}gf}`IJ zz)DAh^^Qqwkpc-7@zLLV(IUYgC0Wo#AS?{Kh8;6R8OQ@{wKoySJD`t@ePNqXWwF8q zo=~ij)9&FJrCi<#e>S>phg3X~2_Dv~+n_ zMX`2~2U7^G_j)pVD|%01WkizaVZREVo`2OO+p|SDs{3Xxq`Gn=bGsgWnO(v z`jkgyUcXq(y+fs;zNCPL2HpO(UC#9et;q0BWtJKkp45CaTWS6i<@uf@ra?LB3v%z1 z&FTu$?)O4V<{z=qcviZRBk5+?(gk`iaXGA;_hiRumh3kbRbPcbB>>|el?8zyio|1g z+K6S@)o9a-prt~RRO?0m19q+YfGWjyW(g?9$lU8a-Rg6UhJ@P z5G~v6P-aOUcs8J4(TqfY#iZ~-qw}DXNS+?7qZ7FjFgt0leD9P+ecH4kvp|t;@NdX;MESGmL_%)0}Pw0On&G>i3 z1j_;|ATWAZ=gkOOnScnVjfKG=iz!}yY9;V)YP#u*&&|rfKUGswWvJ6VBiyS@!Qh!d zGzb{ow>dxzl67|E=jp+?-D$XGmPW}cUA<&Q4pTe&;R{ZVCQQ@TajqBo*`e&j<60tG zM#Slem&PX%6Rg1?5^)>}&RSTf%-a;Mk?%_Y4^Q+4QnUsF2xxCjHdl&haFvXRFsi|qsKo@4u`V%^w&%35=D&G zK1BITM4gH*1*1Wbb3hJLl)J(O*>qvFzffMn;jh|lNVal*D813f|MOpEO67)f5tN~5 zbGfR(xt`DL9=u5aXo$;k<+FI{mwmW)@->Otn;UN!BNQOHkqOG57(+%+I6l9r_ZxH% z+y%2E;=gAEIU>`tSgk`=+$_aOF)-qzQSd?@KWPHSL7Vo0u`iU+yAIa|dD(A}28Dhw zQsoaK957AZ*Kd)^sNR?-(59~QN2-4F<6jnw9&U9ou@~NqCvna z@o*1D+%!6ySoL|xJaNi@z3b%0P2r94Z#1^lOPIJ}}p4Q4sscwJdV?z7j=j{c}zwP=(HcCjAOz zX?9KUp7z4)OPFXCsf1HOO6=|I zN4f)uGaFjxVacZ((!D#?Vd#BeK4mCM6~wDF!Kld-)v@!S$7wXft(W(NF7?*W2j~7( z=vfJk^M3BP{3Fm(Y}(>)p|!t=kE2=MN}#nTF-Q@wG|FzcfH?)Gt&Eyaf76B4J~OW5 zXU$+%KZeqC6X&nH(HAU0zr}ySo5*a0W#eCf_-R$PccpH2oa|D{YuF{iJpu7JhZ32W zrl5{bCK!~+C?;eQSqG>>qO#a7rIMiO`+Y@0)7WNjQe$#M@a(`&T-wgJgU3l|&y0jK zU(;w1p#6yj^kBdw>Y9Owa$4=M<3Yk52_D z7yw7Q!)^|wi$_VP!Q{8Lb76r0Apz&N0VR@UTjDvi8fMd4U@4lu=Ccz6_$@aFydYBg z>AdBAu!kx}1d_WEN-So#zsRzHB9Pr1FA-LP2r^|xd(f#Zm(=?0)zFTJSi$@hS-Mc}F--=B-x5ubJBeZ>k;!5OHuL7@0yTfLNV=cK*s{op&43 zy}_Eg5)_$4;_UgdSw|8=4qc`?@Rnj~h);aRF90!Q8537UiCqdTb(?Zj`nd9=1nv^` zRWV9fJt#@!DB1G zohprcs32XI2-v*1d8)ytR@tzU={kjB<5;k*s||Su&N&@GolFVrM&(|YPN}b|ItScZ=_S7i~(so;nGIRu(4k!A8gyjVygahFcWmjPx zByfJ%pG2lZI&Y2CW2rgqGYtu>+_Ly8F1a$^=Br6wiffO=5xlQxreI9?ZoTeZcFAG>7X{q^R>x*#5dS%lJ}?xJ)!S(d6A|h(brJ*GZdv@{o=}J7 z`+pow+zzxWtM&0BAE z$a>1AR^(PNJX^a-d^Fl1I9xS8*!BMJCJ9-<^f3zZI)Wl#%=2Ioas=8 z$;tck*_nboM0KoBfw5tR4?aKrk$3Mt78v`Be-mFfDabaBmT&LMKes~0?zlmx)qcZr zBf1hL0A@7(Z?#~|NeO?foP}muA6}9A8Gr>&w>D8 zz`UaMe~Ig%)IyWmChOCA?gHPWXr%-11plR_6l=2_Vjf{jxp0kw1}QyqxN=D(+ zbY6231xCWW1sH^1t9!`F`v5~z`?Ee3iOtI>fRP828TfN&)th%w=Vu6=Ah*A zTaEW#!9xDq3&mxokKgy19UssQSdY#H^G5F(zQhAXcFcngM1ihCdZ+U5yEJv7@ zwH0Uay=cLHPqn`_!mG6{^5uE(chxTq*}_~-Mcb`XVM5A&$68I;)}y;Aim~D4{|f(k z@oqf6IinxJ_P?v)htbPRtSZc*<4@lWm)Q{9?p3W~Lg&DY!`dVONKTHK$7>+dFgKU5^y1( z?`AK%44yJ%kS$s*wWNGkb@AoK7tAg{)SsZX%ca5?lB5PpBI4Bf;+Z=7lw1;)Nyq5z zQ}oo1kkI()hiW@Et+KI)G;yU~>*bnCklR5|@dmliKZD}!!FvusC>i_X=-(WZtz);Q z9EvrSai>RJj-3`q^Q57lGLyusBa@j&X2Q|!-92rp%99IvuN=KqjHg=#yr~oKSPE?? z%_Hc%<#NbAc88rv z&Wqh}QHM$!Cl)fUDsIHwe+NN{ym%f)9<~>Qi|M(9eiZ1Xfs?u(45c0{^zvQa#jx&J zZrjgWeBwSi#r}M$&Y;Ug>ditl^IGw-rMR4-%rULSAKeF6Luw>Hx0da#7+X2`Z~Ker zEUG(EG1lD;&C*|N&!TIW$L^VNve&!vWX1^vlX?~~b-DO_Z+xV^eW|cw^I_&e{rK~Y z9>JB}QmtCeycd#j;J~}@28FN)wmoM48HIkMcw1=mWITVs5DGIKrx+b__ci#G)Aum* zFVE0*YWcN~QNI(*&b~Tn*MY-%oU(Is1h0jR9$VTx9w1evFe#21606lh=~i9;D?E|3 z^z7Be&66`xwvf{Hb+Rg%C*&V%rmkG(^_SfBX76HjWAU(DcMAe}N}JT>v(|^lS(R2f zqc@%h&&oJ5-qLS9D#%RS04iNq$T25Lq_Y-Rlb7w1Lh8bGDXrUob`m~Tew3-sJX>|+ zXVlvk`S_DB|Iy8gy&74Xd|_<{bYc24AKh0${_@7ypw3#yS}Ed5>i4&}^DKrrIM6Q{ z_f2Nqg-!dTcixi#4hvj}vx|5M9{1+bphnSOYpssNGA(oH@yn7?wM22*?*9yvz_qe- ztA7W(;#Pn37~T63wvEp?ap`%0;&iQxc4_8hlE2%3(*YAd}S$v60W+m5JzCez3OUg-U6ovjtRr~ z=MI2vL{K+;wIP%nT zn^b45?@{6t#l$=HNpQ4KZ|JjEmN~xO&tFjlTp-}hN5k7j>Sn4>%S4rnedMaQ!yYL# zDdxroO(nXD&|V;0n%t9Wze!GTPx!qEb6JGm)woY?9l8Y-v)W$}4^Ht&3Po7DI#H9Ho-LgfN|pZ}!G74ee*~E&g~hO?BA^ z#tVs@)$7HgaJnt#c1%!5rHfvb)yQ7ea?M)Jh|3)CPjPMM3QtVLz_bq4_4OYorV>%h z3v+$wP!qHnIqT{;<3Nvi^!b};FDi%hC}X{Kd`L{48&=i{nbt8Z?4fstl?@*y#gB5* z{QJ>0B#KV>%$RxOZ0!Y(WAB=urY_}pZ~XaU&hgEwdw;rUmpz-zq-Wi}%j7}r*Hwgz zxDA=H^c(g$#^uFKwq_MVnDSNj3!xdLvo0k}H0n&%xrLa?SATRY!RtQKv$`LZ-PT-d zz8%o;-n>*8Umj%3uS;0V(u7JopCKUY%APH6HqLGj9Rqc2OLzK&7v^>`;_p7rez}bG z6J+MKw%ci~$Eyy4O3w(s3rY(&+6!;p@&_*w@KN%?mmGgh)={u}U__)v^x$}YL}FlA7BRflkVh&Bht)jnjz(EbQGY>(sdwdU=~E;iW?SVw~zr1*`y3ShDJ zUTSlS7z#LZ8c%2m|Kbf5i;DMphKz5R%^7-7D_ygnhR3#A=a`*%herLpdsovdnuJb& zWw|}oAaz&sbd31j(L%5C&9iqk|4eNgpX)9}qvWbAM|k0E*Snge*vYo_i=*UyC2dYW zThp-nR*v{#taGip8OZV|=+1vg%C_jv`1jg#_4*@Yw7m0vbE?1Q80O@6J&nd$`tNUA z>bRI$oxJ*Jre!Eor0U$y;t7q)XqhuRFyr|Idi{! z)0Q7XyxPrR(+@q`hZf9D)d7%=7StvW7lKFfa!(pJ$U4U6#R`mH1ZC7$r^=~R7#CkT zB~7*}2K$2MeR$Ru0s);N2((DYQelfPSN%03?n$10888)Ey+1&mcRLKcb1~yIucv3| zIgQ|vqH%V2N8jbpgCEsrCZ2zk`2X_XaDiC9y62HJdRK1TQoDF7*bk+`K5gqijlRQl zDGnohqY{22G`Mi}wyL~W2#sau#$!o6b+^zOSafO+*jR4-NjUjLl4}ZGo33ZC-6XRr zyDY(Y8Ae{S{~rLlKt;c+l!0%^5`Q@kug8|karpjW;*TGOMXU|K1n(Om{^G-nMu|Uu z0zQd#Nt@yG@t6AH@3FCKWsT`I1K1Y)9DEAfD&^t& zFB5;cU-&(2$F;H!B|di&b7?>P(AS8w)Cr&cI#;@n8Tbg+jBg5F^A@ZHe?E+0n;w~?JnEmkru4!bvk;z) zML8aVk6^8o-w98h`@EtvnFJ(igp@P+q3Y|EG*d(C&TBz_UzvlGqZZ-dY8wC$OK z^^aO#ILG;Z7;?Ta?0n%RkJ@`o!7W&pIt#FjoI%p!ZRPUWtPIroS%UAJ!Z=n;Y-dp75e^9>34FEaMsUw?i?4!oiE(xeBnDD zx7S(-`yRJCslj7*(f6ol32gs4efEaACJ7(^xxLn2c)=51Q=v=;{4%!n`nhHcd=^XN z?}3>oZJjf#UNecMu9tuRPTzXU`oa>n1Yh`>U(k=Pmw%r^F82#tW-EOAFBz}!7sBi> zo%Y~@-PUh{`Q7%ugo97h@BVSF$--Nnv3?r9_E)w{9lRD>$nkD?NDXb^*TXkIN9^!t z!8v=Z?}s-(PaLHlc<75ZCNtrcFWNSA!71kpulcR@yWv}YXP@1Cc*kDrXW*Ov&HA(8 zw_mb;0iJz8qx#F?qpxW+p?{S3zm4XM*I9olJo2DMGmDrs!DkP#{vOy~->A>D1MYFY z@bBJ;&$%A>@k1NUX8a}ab%!%G?Rf}Td( zX5r7V8MLPgzvZ)j0gfGkab4l9Z{k`U-wbbfbEDaonQJz|+h#SI2xT(x)3Y1RPPt$B z<~fb3?OAXLtKnziJLg(|5j^iLv34ErMmN>GrwG)8E!;c5!?O zEIMDN&dm!vI{p)ae0Ujm^AiuJVMl8qIusVKWxGN&bHp8qJrmBJ~JA6-A@? z!VYW(WrPN6%J0RjZnwksmGlLUcfg-uo%j`4`yAs8zVONO z8_ja@;hAmJiN6#+xQb_uzXN`F4gC$j75-BkJQ7ub6g9^P)ptC{d- ztd25i*ww*v!54O1h~W!=b)h}C1~*=0{cd=|=ZOzxmcT16M)$XHF1%tL?UZ`pmo9BI z{rH>V0qYx034aEx?;>uL7aqEy(IoI^!oXG57xrFFf57jDsbr(^-7;5s!L^O11%DHK zUpMh0p3U%Q&ff*^-o$;S$)jNVEse&H-vNJ`W{kz(1<$;-QRnfc@ImMAfWQ2reMY0!K;@dv)({3%%Ivwcul7+{R%xbP!`wlCyhB5PwPyen(#-wJ=@ z{3$p(MtS@(`0j~D(<i89Nli`iyS=Shs2zi2wVS!d#8->GHe$9tOpv+ z!4KN&*2ANBSZxR3?at4@k3NKcR1jV76%tK7QF z&F}EYpWEwhhYvkL{Zbwt^Q1lA4d*^ZJ8qSC3XSH#U$}dR>wjUNQ5P(^Fmhu4inWLD>I{y_XW~uGt(w_;k&dsljufW6bB=80^FL;`hT-_n<@Cvlz~K z-kuwPpTqW0W=x3f#;<#U+~x&4wk(9r&KF+yg1uH6zHQ3-%`iM=%P5`S0lS%E*yoqJ*5?|EkU z{qXMJTRm@s2mhPBRy`bh$&Nq5-@Ih&nSxImpUH7<6`pv2PvabdH#&lK^y;h}Xt)!0n<$vS&(1Ri{l&r~^H4_gni$A#}Xm^z7V5We#epU%sR;2Ny{ zi}LShY2zXGy27b?+h+3ypXvHLp9xb&n8qZw!e`&;Gf|H3fx8a1zVOt;s2_hZ+=%t# zcf%iG3HHPbF>pajK%SH!o8SiK=|xTpXrqP;h7D#ZA;;84feh=a?ERw3m^2_ z*zSODX!N<<+h>k<{vx=d(ViQFw>p1@&u5ms*=O3mG}kPL$Iapy<1d15oZ~a0FUkLh z8pr1POcGysz29dt_$m0nTZjq%4%jf?XFT{mc*BwC0Dlww#*wxSyWuzAX4||K{sxnE z<%|VB^I@z+`BwPD1-48Dex=F!Tj1kOww_(^*GKtGjq-co8E?1uvQ&tz+dNk}y3odC z3?6l~PjmH!aO=@lpTgUZ<@s)we?y9vjw25E5qSA=_F4&e^6}_ov;5|g&pf%vX9DhP2B_{B~eZ~~LOE(c4Ct%il3 z_dA`y>)&tNkb>8Jz@D3eU;BVPo`?VPLEA6pxA@E_TI{(?;FZqrf?sw17FhZ)?Z0iV z@}7_QbbYrC9v0^QIX)9sKTf~G7e0MD&*nCH7egCAf&TG@OR)&P@J?(G{$BXR68jvh z@ExCYI*0dv%BOR53AROiW+uMyuuuCmubTz^XL7ELz3`}|KC>8qA)I-Z)r|1>&NrVy zW0>p>E`@78W3Ls5t3T^AOE@?n=P{& z)~&L~XTt0%dtCU!DtC{oc|NP%_zycUKXq<~KXu1_KP9LKl7C)U_HtFSR{{eJkS zwYL7n@RGH5OzVfO?Y4|?D;A+1;rtF;zwq%6tG8Y7V@dk*9r8V)&wP3#@xfmTiyLj7 z!a3L2_WNPewT!ov5kA~aU&G%Cf8A}by9ZvC;yH0V2@k!2zKOpO9(*I3%Mf#T`zD|6 zt7PEtEwq#4V{p|Me7b&&!ucd9DS_I#^nP)-y`EbK#doM|N4;Fud{tkC- zcH<1w-X2IEWo4gu(4VQ$FMeO8+`B%+lC!5o#9-L3*X&C z-0(y2N7xvC86Mi}^Z*z2+G`0D&KH)PpXp0$=eELoM|`?JQ-p6CwPoV)brZys z<9-;xJUzq;zA#~9H3iR_^cf$=qwt8FeRi|psb5BKavYYh5WcYXWqV)3tG?nhEgVn6 z$*gCJY)~sG};)07; zv@baMq&FPgysW)_#j3O0F7{Kw>h=>CT)1}i``XWmtys0J{peLI&sn>sea*QYN1wB1 z)%%vUuR7+U;}-Z=EnB_v+!gH||7ZWJQI+4{+_Cn;_Kwq5pS#A@Ht&zr_Re{F9%r_% zIOoE(D?2WB|ILxLD=xT@tE^ZaUc2(5m2E4|ThYGnFZciJ_jht?2MuXmanXu4f1CX8 zi3^sspSJp`nHjd$Yay-ePaLx7s_^Ti3@$D&&xItzHkM{ge;%`iA^NO+$g9 z;E*)<2>tuSXlAr9S{XHC{;|+lbSyEJ87quc#*F6()6L(9`dj*A{qg=(f4V>4U+Ay& zSNrP*JOhD&;6P*`I*=Gh4rB*%1EqoTz|?>l^ba--whV>`A}okVX!z@9jpy{ zhJ0K%G!z|*4JC(CL%E^+Pg4#9(qTrLD{k=4okZu&ix0Lv=$EtN3u|aAG()oElCKXNI%Gx#9e9 zVYoP48ZHl4hO5K1;i+L0J;H32K0Oi~35~RjL`PyH@sZAv)JS?HGm;%Cj1)&oBju6W z$kd1#tsC`^HjM^GgQMZm$Y^vlHkuesj;2P_qq)&M&!NbZsE*b~r$#+vKAuPu&!lB6 zJQf*?jm5`0c{<6lG*2iymK)2DmBz|Esp^;+uN(J_`^E#~!ST>|%XoA=HXa}E98Zm> z$1~&E@xpjR;>pG&{G!d8xPMG)+rir%4Cp(FCaxyiUCgR!2++?1(7bi=T zWujl5GznX-lgKACDb$)luQ?Q3K(i%OTS2$Aj7c3~qLj;J3)xb(lKsENZKQ6*GvXWZ zk2H-;$1Xe)*%!aWNRk-lHGbuh%1D(68viV_Te@1dNbS|E8Lk`l4>t{mhFgZC!!dnU zay_}0T*qkb(@&+78q++HJT*#vQzJfo5-mK1PM(8(3Z;F~4-)wZJtF4fo^nwyy0}ki z-2LNC#67Yv<_Q<`+<2a-mr-|>sCP~zC(@|9I8mOcPShs+lTDL>$>3yoays^D)w+#& zZBhg^%fx_(vBJ-2A-a!c;yPNS_QhWMKwXcA@gmR@X0(VhUL<-lj21Z^FG@W%#tYM1 z#|V(<%`rw)duzS^zNWrNU$ieJ<3w$iJ~KHp-jiaav1b?b>~c?q5l7Dd!zdE$4KebB z8GGW4Jqbpi6yr~Z5vbT((fX%)O`oUF*B6xf1GCK%=Cj)1)S&dK0AoPQP=rw+K9pb_ zNDpNh2?|3c#)9h5)KFd4ll5l<*i z=|dSoM!zuQUkq(XugqwVENX95ZG>iYMe zc2{)VuG2AFMr#?T6FNpukI;2P9#v6LRTNbfB~?W^RZ&@0WYAC(3JRm2PSsCF^;1MW zHC2yK)e}NJGQMYYj4JB*BqNiINiqhdbPUhy7+%paT(sg>t+c3C;%?-~>iAvK@msXw zK`ViMV@TYM+%kd`bo{95_~Fsa3=to9$8CgU> zHWP`eerzt1Q~lU%#HfC3J`z^_*o-8j`ms4l&5a16pAhPap`Ik;kIj{enk$KZ>Qp}g z)lWqAlTiI+RX-)w&s4wY$FKTnK|is9I0{M)q|uP%041IOMMWM|B=dfY&iQej??p+u zeRKU3bG=8iBGFS=_0*|)%BY@-s;3%y@??D|DwqwSsc1H)s!FM<@~EoJd{ooGvJDI?-D~^_Eq=l~iw2eWEwN>a9if z7FWHcRc{5=TUGVuQN0CKZ&B4-QuUToy_HpOM)lUDdJC)GI#q8O)mw32ZYh~?ljg!< z&4xQQAI@k-T-2Oc^j4>O3#i^As<(vdEvtGfsotitqBptI5vg*yK-kMZzVbxow>Mf&sE2`dVsyCnNEu?yjsoqkmx4i1DqI#=S zy#-Wn5!G7)y-6Ng(L7X?=2xY)sM6x9w6rR%ph~N%(mblPpeik@N=vHJa;mhlRa$(u zS+4V;M{_07Sy**u*H#(TnO$AgRA+X56;hqq6;?`hX4hC1)tOyo1ypBtot03X*_BpF zbtY>qAM>K-wcr*Dj*1ylShmZ{5Ds;#hU%jVe`)mBlpRa0&GR9hj{R!p^( zQf=i`TNTw-ooXwf+KRZ$JEiL#n|F)0rp82De$`fsYAdeVl6))AyjgNNcb#Tjvd$5O zMO9%*Raj0HR#t@>RalcMEUXIablG@L^Kns_=3EIomWjsf>OG_yi@A(Eqd9p|*FrVT z%0*)V)tFtyCsboumzP&GGq2OEI-nV~Xe_Q8v#a@nYRsrIixG*n68#n zZlx@_yj>yxwP*8z{qp&QX7t(pa{4LF>iwG6x4bN~FT5b96$MI%JO5H=chE&&%Z3!4`{X@d4+txqFLIj&H6o>_XjoekG?GTFTXwjfUE!{|Cj7va(~JE|Ezg` zhIRO>%=@SQVwr#VZ@K@UHTRdy|4+~R%ewbkry0NG`<=Q2`^R&Ao8`+!pUu1ew`BJJ z8}s^ClhtQ5^ZEGhpX$~(Le5bxo%4+UDHD=#XOe-hpW;&a#`*d$qW;w~yWfZcR)6|4y zO?Izb#-AjkPgTaAlHFJKp@&Y!n=*4&kmsG|DNpq@@l=zH8j{)AXDRbfgn1{=Q}Zz< zbn=|a{V|?}WF+z|Waf~(S9*GcJ}y~^JPA8n)agti*{<|$n|5a1fksc_SgRFdHWS4AI-Wkp|n~7)8gGxz zruN-WNLQQ@js45t!0rG3P4lH1{uR&#UO{I;G>FXUM+6j%da%bD?BZO}ZAGe%C7bQ{3$u9=n@hGFcq_{65nO>hyXSjC8$nM(*nqDVkObkz0ggzMMS?|b)u zXFL7gH>dM|QD>rxu79U=t>V!Pvq^J-knX!gbS?a|V+zqIcizWoZq zqpN27B}7Q`Qu{T;A0HRUOznc#dg1;J@R0AD`Q`iMZ8u z+O3AlE=TpbeZh#^tx3C8b=j?gd@e(cxc$Ji+odVH)wIvO*NE&l3+LV1$L<9-xxJZ~ zTSI5vs;BC5(y$&QQXw;0tceWn5+)ayyU-wuHCDpyZWr8+qj9@HA-7upOV3aVx3gGqJ3uBr$K=iQIx66Huj6h{G3WMvYHqC_ zaOa&EV}=JpT+Zm%cq*5*04Vyd}a>VVrjjJrLaoLiIE z+=V zBbqJ7b=M`S>*I{(=Krsm<137inK`Cz#;Y42|Ld>2_MeGth8g-3=A&2ojwtH#&i%h7 z%IO;rn|=P_t0J3$`gQ+j|7;+pD~^QjE2dqBn%6z+(!TF7r*x0eqx(Wlx?&IMo05pG zN#eTSnAAO@|0%geLH7sB`YxuX`{Q-Gm+99XsG#l@gf%ye>8drc-|FcvlQ+DYZ~p$Y z%;7IT4*b=R0>|_Rj1* z*}F;Y_dO&h`sW#se1AGt(EBlgd1jHj54nR^`u5HCP3f1iGNNpFo~hS1$ag98w4L(w zV{YeMp1VoRGhvR$GPOS0&+F80o~y3k+x@%rkSOn!6!)L?Mn$Q4W~P3_86>V*`C9XD zfBuL5`FBuD2MGEReX}XL4FCYpBLGlK0|XQR2mlBGhpvlG000000000000000GXMYp zO+{2eElEyBFK%IUX?A5VY-wU{Wps3TUvy(`VRUJBWnXV|cwcxpHeWV0E^2ORY-wWb zy?c06MbbEYCU+po86bgh5rRYn6_OxEo*5rQyj(s=vC@j}zTv7MWZ z0}3Hp#fg8DCb_09b{Syd`26ww_hDzU9*+@Be^Ek-IXY?50+)O0f)?=&y_4e^A8Eub zrt)zrKBIkzZ_ywLLBq)F~Mi)PNbchbx`)8{sWZ}Rng9M@cJ#KYTU z{zE>gB|gcsjklRHH2xTC361CSIPwX$I8JZZoLQ3=JUDym+*y<6x#qWs&uWTCnzC^` z9}ns0eREwQ@lbqL72jl2JRDy|NcmY)ey@G=RDAQIjJO#e_V>jQe6#1`Ak1`4n(m%c z)CAx0@f=^ADQ?EsN(lXH$|s@tPOJDT+FJB@l+g0uq)GQqaZNGDL-CbN??S(eyBhJ| z$5%9W_UyTHTEZuIIKEs{k{O?Q4)gL;7c2;ghvLgs@zo~&XZVnhCBox$+;X!$&!NqU z)f|s7xi$Z-;PKvf@_1|*E|jCI;a^HvdV0ZvY4aDP&zU=a_LMp4d2@^0v!~5*El6K9 zcm4wl=1nP@mOf{i%k^MiSJ9j)u9*v`rB9uUQe?q`S$!9{Af+hXHD&&N(_HC1yYE!@ z%vtwpX|rZd#ealhl7^vg?gabo;~cl%lH2e8IWr-f+5ywLBtz}T>2|;ef9(>&=OwQ} zIxyF~ce>Erh*0~oc25@}}y$@<+5qhb3>agE1Rr;v$jM^?j$?plk|Ni~= z@4tWl{rm6V|LR|XB!4gYlBX3U3z9FPo77pbRgQvmb^s)~=Ypk^|9o9S3<&W0Kam+^(*Qt{k4bX1Q$Br{?ViHm zaTEL>ADnVSutEYf(EkbUE`sFA5nOg9c`Af` zV=QvQoe+Q$cV-@iwu)t6!m{a5;?sChN!}5Bax9b%d|<(51Cq8X689GDTl;2Ux8Po5 z8~ztq4y7d7UdP(+NEFyt6CwoGCj$Qaz zx?uYC-4Q_z>*;}1W!QT>)efmFl_$N*lm3lKFY%@5%|td}Z0}o%g44JDcw;aq*Fam@-~UaVjlr>?bV^D>6@j;shU7DMyi~di5h2U( zWFJe(g3(xGz6KV)6*7NtK8zpdkb_8=c{X{UBp;BJzr?X_2@%wnc@BA(gai5&d!Ptu zz7BO;B+&a1-Hr+>>;Es2VDoFJHpqn8$7Uh8@=zRdEHcU6Ya=krUe@5sWw$0p2=zBX z*54*1Lu!ycK1&enncvl42WihhTD^#M9BChcO!k5b^Ke$4UE`bz#g;) zSt9QXOu05f6=zxRJpytha~78)W{T+YDoOJ0c6En>lDscZ-ccCnIX3|-JqCi1oqvFr zg!nLM1p7gatL2e?zyp9KU6MaUL>YE@pB?FSnAI;uwn)j77k7nDiz{eTSpm-j)IP+TMvVkquPi0jDgt7=hXPiy` z*t>g4p+i0hGQD(87l9p|!KK*kB%fGH9gtT$M>nRY(!GMS@?>dvF*`RS8bT)v(y4pnKBIubJ z1eywo4)5H0b^Y1n&5-P8C{<*Qc&OgsA1O910qecEv^$RH3M^YkH zC4=T;ZUl+^nh!1E+6JHit=8 z#-jD78VazH_chxe9sWRDqyGrszIk|B-z3k;6rBP-HOCr!iK@zlg^TZ2HJ4+o|7r`L+%@dkl~$bGHf1@Q z8Z&o;9xSe?vi14M?)^!8{3}zMBCV4b-H_V7U`DfHOf)$f#gzdF?GJ_xBUH4j5 zUMu!W$u}ScnZJTpgEfAz41?9Gj*^EuN=Y;|4U;6Lq$Gwn8l^`$0IRBNW1f7*skqlV zWPh-rPAv{ptCc@@$`^wb^AL%q_%sG9Dv^bPJEfdLzv!J83XX@S3d zuS(9|&1yi+%9B~h%x7P=ki?jyN|4)ESSfxF7g?{ZuyRV*up^}y+~rVWXilHteT8C} zqO8HLqPTmtly~zy$eBF(Yv`ga_$jsZ(VMj=1~g$>+A?D#g0J$#ueM zz{@tpF4$xk;vf>V3`*t8zuM%lfP)qH!94jaTYV3zmdT&>pz(&O7rLBHNhkWThlw%6 zGRIG)T~TI!bD15dqOd73uMe6&a5q<5o_-kh0dZ;*eIOj($>EC#{@9e}^an0qM;hzP zKmcazrnD%JRekl(Mu{jaQQAxi1HZ?BKJ*akKh5LU+K+XxrB6f(#00Nni5|`2Z+`q` zfiF;v*GFhF(NjUfiX`BJf+|&%N)&YLa5BzAT>{EGY{A1Y*X{C86@2=#H=s0P@$Ir` zn`c<{f=}W~n6Jwx`EyCGvT>;eQp%C@rF(mQJ81uRf8oOEp2@fs0EW5{KI5B}PcZA_ zaJLSf<5o1Z;i0yT!8g#P1*AT4CBH=0Y=Y$N#L_U8p(CVt~1;iB?s% zjTT$U6^qMG3p(tmK4f**AZxgr+U{F~kycO}4t`0~`2*USCK6s&MMC|x zx=5J!n3a#SAqhTei@Fgs8Mg)~2E0d?ES5h5HK{i9C}=@-AVNaMmlr0D7;wR&G?E4ANvY{YW$#A1x@uel}9)sFZDIc zBUcZD1BW`4SY*JUi2--fhN|^wZW5Vr7fl-_dVODG<}VC9B9NtoZ_U`(6S%bvMdHBsM{y1eq8LiP3pwyRS3775Z-`U=MrXj`My>8-oV-m$ z3CkO3(vNU!_Vnh&ChHB*Bb&6c11M~i=pSy-Nx~M$F#I*%N@y_cduqIXBUq~iQJ=p+ zf*ks@ek~+}MpsKeZ>)iz3Y5ma^@v6Ctwan?`AoiioRxhll59VVSd^^@z$#uAL#kA^ zVgdFbPxc$hSb?Oh#|kp{GkF15<9Rw8!@;#C;{HF0Ncb~Le}YtE(K{VAZ(r75>_~Db zhz)12f{MiAU!cLrN+8wfWw({a`vee!i}798n??N;)GzH^l#aMl6ZNYN824~oD@%7} zhYip7zz!ft_g@nfh2XF`g3!v{nN>WG=F;dXrYfq~Ljyp@4qT~AEtPwa!Tc{%TaV?L zdV%8W$V5R1^oK5L^U0SrEgt(5z0ZYn8n<_afp*ZR=@sGp&|8<#pI;ZcUgH{_k2>i` zWSse!omp;0>WXC_3+g8BC2WDC2tq`D$v5-^90IodMUk)7ACVB-Y4--j$1N6=E9*H= zyn%ymmw&ToHiYh1L0hH$0Ms=`jymTER_EvZ;tl`?r$U7!#fK=w$NxqAxLfopXs+=^ zpUvtl1eOuQL?>RSlT`Gc;0uR!zP51m4jG)lgLhDV>yccR-G2w6_^^+GqU=dP5qO#7 zag9`04mO!XzG^_a9;koFj$^Zuv#dHqU$KJ6EinE|2e7-JCVO*;Txeh?KqRozRW!Vr z8h$6J>{m#p9And75V5L(Mpf>1O?tAE-PAwSAf2o6!{rfN{=PGz1^KJmb9$2Y{rx-? zx>F`-`#F=Dj!XxCcE}$)?-qR z28tf-IM2fr1u4hgdeQgvMWsRMe+>8zG{Q?@4>=CM!_m&aO zK`ks0!p^fiC#|Z&b&phIKhEWxozbt)eO&U{nQ9jq+>=xl8XXbK=VJ$F}FRRI31t1%2Nb!%yH@iR5>7I&P`@<}|2iFc$xh;$DE(3pU zMU?0>mz!x`8*e@fY2Kc|^^HcF?ujG_4gwKsRU*`4U60)m!Kd;Sfp0^#IsV!zkvfCY z0rxxu}WQz`a4==_oIf8uD}Ma3tmDn5r0$-U4(P=IXCqH~yil=Pig zNy)F1lmVXsaIhkqZzEhe`RYRZbJ!-xnRD*pJ^Av<`i?-%FJ|+Rn=p0AU%z4iuB;)r!m5h&YiI~}dbE1a*^k@V(cGs}mpXGCABDZ*xk@6e1mjiIM zAk=olbrx={$(K)ahQroo1E%_`q4`(M=IYC{w+mCYY zri1wB%{|8M$6;U5jQot++&Qy2wb^ZIbEn?K6E@x! z-rT19&t}`(I6!ajxs}be_saqQ=k5J$rP|(Y5Afz?Ywi8_FrEo`hubIGI`y^%9iGnVGFzB+f zsD7}R)I@($py%z^^H^WX+iK)()ARQ7ytjW=~8Iv<2a;=}7LlC1gCQz%N)JmKR__zz@Th=|@$$lik~} zB*|8?G*v)q9%KKyBSEOdt$#Ozi0|y!cF0;o0B@Sq9-(=NNDepkR9mukn zf8+QsA1q>PwtH%VVB3b+=|3b*10?AY%&(6G%`1-|f;2gw)XrOju3QJsPaPjNBm=iN z%HlZ8PGBEBh06EHIJJ7V^(j0u!&x%YREx>j5UNjbo^Z%lSQ>z59)(S~SauM1Ro({i z@egUq*+u86H#(G|X;|lco06A?qdC*&c{nWvW z^x5Oe&dWIq!++LM^l(~|Wcj$y1xMt=X>ob-xjs9WT<0uez>v5XTY6nGEE|xMZ#g9u z9jI@In#sKd3Bnd+uKH|8(IK=kOaw5y#g&!<2!moMnR$ppPr9YrbVb zTAt-x6SH|!doT=pp0X$f*j2~Y=q$R=DitkCa`gGdE?=?t`F=@HNf{zVE{H<`w!aUE zoJdGn5a%fR6v^O+l===j$R+kqNXSzjNU~e5>En-G(c{g=ZD9E-pa5)Q8UENNKCZt-j;FxNGE^z z`JdUo*nsnAI8Nycy_5Oz!g=-+27M1lTA2IVWbQ3ggY3N%(`$Ah%d2NJKU@Qh)g8-h zqB4suY=o&u1FImcA*RK!{R5qGz}Df4ge$=zb`N0(Ic>*{A8O5GGt{&HoLTkDnL6}-t z=KTYc$=r5Vqe_BUuFMey$j@OAAm(l1J`LE)zt&a{J9e z++3e$zojJ$ft%>KF}igZ+%v{OlWVe)g>9Kob~6fdb~#PMA85}(k9F?MJ-@Q{W${hu zCH9;F^%HH~>3Hc^2hQf8OIG92zXgL37t0>kn%2NF2NH}}R$-)FIi$ipYNQ3MDqI;+ zwz!f#`)QIe(_YDdf)C`sf3Kw~GT&c%aV>OPd}0qa{Sg2FtsNTrqS{a(-5&Na=u_(~ zwyoH`8v8-~skh-+#Ee11P=n|_23-hT1OGub4rJz0d*)B|SE;de7L18{jAKFsA2|4L z{^ek>?5Mk)-TRYLc+lM$Hz>;hU41I%&dtJ=3v*m2Fg_f5ix87J;n<`OYBMHcGhV(D z4AP)x{W5^mavXX^q;0|xK|BAWW1BS;jlR{{MKBnJ1PtwOvIaWQO)z=u0Ft^}Rdf|` zWbBJh#azOw)~IAf^8(ND0xyIWh}8j+@W-8bI8l zm+Dnp`X`h99i`h>Y-LX_LD5)z62;tIWUTk)rCo96T$pbIy~*k8`%Hf%`nMteHNJp(Z{Fx2viK53f#1X8Pd?P{n?flJ|a~TF^!4cSZ*VPE5u*BMGVlaR>o2>KbMRmf_ z=L^+4>de*2D5{`EV%aCKRNm~1>gANrI6S{u#bs+Fu-b1V&o`$bO|pEF*)Uz48$GjE z2SMN8oU@5@;|ER`UA5E2!AT#3{!n8Py)hOXPl#K1yp0x*;PJLvJd(#dYVjx@zee2(w0JC!r)%*z9?#O^t$2Kh7H`etHZ9(U$46`NcplH!;t4!HPKzh<_?=o@ zlyZt^0Jz;$XU1_RDW{9xyna@7E7E&>D=v9aoVK6ZrHu zG^gjP@@Jg|WXzgTAYd|MGfLQ zh_1jC*Ri#~qA>n}EgqyEtv|_E9n~Qc_Y|P#MnHdp6?k#23_#{Zs3(Sc{*QLdPlbFN z8hiCDRm_?;hpWr@=I{sB0cx~)uE0zwizLtRm2>H&hI(S#ihGhgF^>fvh~!(ln7>#A zz8F(G$5XO<**ru&@%Eh&~TCWz!CtbF$rsJLBN>qo(&((d$+(yH9y^*y%sCH{45@YM}X=>!=7~`@qEx0JV*__#=^aA75f@VnU_s_CQS2G ze!=ZOSWkDc3?-GOgV}>c~C}eHntZ7ke z4Un^N(O1YH?C>lrI%aXkCBQs!R6T;#KP)L@b5TJxXjE=7PZgLtv1*~A7I(Z+L%j`W z2!`B)w2J05TTy*S?AeG?ktZ2MQLE5*qaj>_2}D`X@|{H z=m##fk}OI)qce3LZfo?Jw3NoRxKicB%>AT6!9F|R3KU%vd>Ut$&9hBsSEteuXR}!5 zp@ghQZN#z){N#xn?Ry={sKt^T1-uKDKK7H`@JFkiDqc;@ewO{UW3t8u zo2YN-Ab1JndkpuLb;=nAm#pPn(sUc#@_AD4eEH*i-0SAAmE;1D%x*uL_q-VfKWs6v zL%Aw`kOg|2GO<9Ti3N%*+Q^F7FPPTV+$v)WKFLS_{WSCN17p>?7Q)cKGMt8fHT>i& z&a}le4;}ImW4N8lG^bAUmNHx>Nu}mWLL3(_OYLA0LC!;)nVR} z96HQ&)Y{T8*YaT|%T4d!vrppqsz<-_In*lN?PmRC;1aP)gv(>Px5NBd3JEL|ox z@P$(HA4 zwaKbUh_yP2+N@_H8K|P~BIUy~If^fj-?SriH($%qeOa%eb8&PtqBPSItEwkF^EjSJ zGPimbt9YL0c!piW@vPMG*g2jO1J8pTPdUd^&)ZX>;<=sU>7}+OKOE0015YN$vzp`C z$=kC=#nYPO`9f_^w2r4e$FtVJ6ZnzvRB}8I#StFAif2b##FJbHJ8N6zje_c@Z~~Mv zs-M>FEl66n3L%v(NFMeW4`UZu2@hj4nTv<9DmJ4nKbW5kzqakPA@}|ydizl`vb;Tc z!<|1;!?~h^lN${#i8OI8N2)fV^Jb%qXKN~;{W;Ax+-9YL z{O)Rwc4&aMX0YFi!uH73>^fc%k}_HEq&J7!{CpWSS`KS|WuavN{UgsW-XA4?4#_aRz)iY+wmL)3P>@I#C37}IP! zE)8n79galmUKbT@-VYl6hN$%G|AS7ytMoR%Gl<%}t9ujry)dYY;e&=1umq~$Tn}Es zwW z1R%Bn2`DSj>RI0k2ZOYPyr$mxq36fkUx@6y(+1*FNgHu8fff!ln?Nb)Pfe5VXp&UI zLs|Jehg&s*$ESy!w~><)-#)>rf`e*_4edP-Rx0#X|>&Z#(|5H?K#QO+NlJ znRVsr`tC4dPj#Ytbr16S=_mk~#-T+=Pwe}fMKJlu8269R-D7iS?o>B1oo0ls6Z&vn zpKkWyYK}guNc|mlo24^qORrK7NT)(-y=CfOBc~P@&pO#IbGb}EO~_WRk!|6`7RhGl zvJ>;=>60DG61+WrAO%#<0q7)~5=xgp^`qX6=^2{bvmcUcNhrCx&9ETsz@Dl%-sGOi z{ESqGxS^k!Jk9ct&)^?r{tWC!lDQk8-_5|XsLegxEMp~b3dJ`=A%4@o3IEIh{+Wk^ z4{hE9!@|73S%?{cTJ;&|PunrH_x8Tjka>cjF%7$2Et1)wLruaY70Pf0YW95A>|ZIC zZ4YINk6K`geS*5Fz+R8%a{KnHRacdtO>FV5pjvqla3Tl1vpK-WIY!*G{s;WCIA8u7 z?rZbqcZu;hC(Uf#S*Y&`fs78#`C_)dL1Ub-jDPha=hImrH|wnsc(RAeMH_plTohK| z`1MWw{Zq)evd<f@P3qBvn?u;Yui^!BIqVa4rqp( zGRO`j`N+?l%)DQ{l0;{{UM~Fc{e>F@v-f%Z5cZe*!fu@0Igk8$|Jt3ORGz|}#5E|9 zRL6FxE}A91nnXL6uC*3^)8OVSoeUS|_i1fFzyFj_k6#tXHr&p#)32hLcqc`$GwouM zxySx83$7Zkt)j(a-HX)thhQhgy|(2@Tf;7kSXP3hYuFV5iVqJ(iDfH!C{8SUjfbpg z<9;Y^8{mjz=|5QL@d@DmG$Yu_2qqfANF$hH1bu378~Q@CC(h^@@2SCUx!I60@PP=} zOl>XR6^Tnz;}#R6Z|cyJ}S zm)MxH)}=yLzgKuIlCmI`*&u?>$F78J+OcoH-}uFIPfc%0p?eNYS&^iVoG*DR-5W?a z^e$5VD|XA@F*Ev!%f{H1z=D_BKzC6TO9|bSry`r(4CcoACf@e4~Cp0M8d-3F}R-kPL=i zoR23%5IgmcBs3&T$`Drea+09=hoSs~?G+o_+qfX#(*0y|d0njC^W*m#4>Y-7V|w$1 z&5B7@@3+$MZh;|GhZL_K!DPXKEtns}kGf0nN{oI(Cfo@GTDqkzZG^)_#mZ~dliZLi zjHdgW@%SDKUO>I;J6ghU=T;cF=UO4mpPu{518iXaclv_>qXC%pzqaXh&)lZ?yKwwT zguk&@IR2LJQx3jpNHSa_4hzP8$_>Kr-KSh7{N8=aS80s&`;?<-KJW{blg=Xh^9z-) zO%1zHx$FTFYh&7mUZ}je7ouz7Lgm5>`owSzWfd0fLghWCbbg`o7m@4JUtFWzvKef^ zw0xmb83-O@Yr@Aw%@0~B|K;ZM@l6=K7}npjnchBiW#H%mQP?}(iKMUi#SPqB z*9q-pg9r^Jh5i`ty`=j}1t49OzoX|u?4sf5?TmCUih4w{oCd~4{JyNjZUS({j0jo? z&mtOp@kuy}XCbr&%{S*1? zuIx!m-|GP6*+OHAU2wGrm>=muug?iA@fYgD`BWpR3naCq&mRCae%Y^`VCJsyLvSt+ z?ugsrz8YWD>IDsS`W~{v%C&|Tlt%Q}ypFf=DqSIj3S+39s@52|n%er0u(slTge5)H zhN+;9zo5=J$fhKv>1mIivov!bRpj!A-TkA5e34PViU-zx_-gIxpHTP&dY|eR1G)&I zZ@Idq`J&-}tVn-VArnh<{ayTz8_Lt=nIzlJr^#&M+9{Fy&Qq(H>o>R-v68F@OL2mh-TqK`!@$w4^F?HSSGRA z{+Z})z@KXk@L#3=K72m|eka1e`v>re2K>_qzvK_#8)J0%B80!?58$^M@HZlSw?BY? z*MN^i_@BG~KK$bb{DD>gzwHm;=NRy>A^hWi0Dp@CzX0Ly`2+ZV2K-2b@7E%Hd#Q8> z9l^?n*+eNx@O~Bk!e|}-9fV)i0{ku&zSe** zM)(q2&MNGW|=qzISGr#})+izcox1*}qa7Wc|J#Cg<#&w#)I{jo4 z-2sYrNma+AB;U|ncH2~Zr0a8==Ss%Id8)yF*UGKnCB6 zU7R9KBan> za_|&UX!#l??xfXwl>a(v5zuORiVcDyp34?kmFWKAniziaP|;OYQ-0^YMgA5G$@0MI z&vc15Ee7@fuHoFx;kkT__*{n`gwR(vfj0AB1hHFU45~O6@Iq7AeBt>&M`QjGQ$FTJ zOUj72ooZbl0tnmCMXgKMx2&1|l8Z_`2Lp+r%U75%1Vpj zy5xCivEaIzJhqDK=)m$c_i zZ{_*>dA^nBuQBp_v(b;>Hu#|o4)_)a%;11)4Zu{Me``Y$=BHw+>8+&Z{RknO6!hDw6UeD#+A4eIcb?+Cg7~;d3A0z@ z!(q0qCA=lF-Cv>3NyWV?4@~QX+SQOmvmEwXNr9&Y{XHehS7z7Q z%iV&*S8P%579IUHNx0QFD+}VY>mYv(w6Ib$(5Ww%*n+$AhD_TlK9)@mV<33FNAc}4 z0AGz4YF(ooO0mFYUNlh0+2^uDE(Fnc6KuHZ2hy&quug?>e^Gx-r$wb+2ja8U&__x5 z!0^L~=6yw)^Cyt(=s;#)MdNr_lFz|TZ|slyPM0|oRTJmp-mJuvB-j)$auD7RE5oY* zIGP>EGt3T~G^sbfd;-6zNN4y>!3X2u*Gdj4N*rFK1iWh?x*B@cn>_|Nw^;>l?g@Sp zjR+q3F80V-v@5y>I+Xm3kS7r;&WUp^@UIQ` zs6PBj=_NEYy6#7pZ&=tIIo0#OppNWZmV|ZfQtKK3b=9|`oVOqco0V{Z>22ozDGtv5cc2fwN0GpX28i1ig4P}L4pwMdEsXoR!^ngKni+Wt^2`>Ci| z!kl;`R$#WPVg!wvNLi%lmm)Xd^@XnZ)vnAOp2Qv1pm|wF^Te_oyp)2EK>#r7N`Yq~ z+}!(;CVrcUZY$^8sCU_m>)7A-a3|OW@q+Q3UWkLMrHk1mC98=8Y@oB@09$R!YU%*{ z%AJ}!N;1?IutlKn(pKE!S=@0zp}<57O{qG`=dM7oeJ9ZXv(tNYNhVjc=v@KTfYvQv zNpvdq3RST}_mgV*Gu0=mK86HXEly#?^xZA=+9K?!-`$5mwJZn_-Dom?hs2BUv+u~dm(c6bpB2HuX&m3BYf_b*lR~dK?o2%Gs?U}oQ*r8sn&rl?5bSLr?lRDuEuYDqV2SSg~q@}yG)}RpPo?>s^ zp}Mq|3S5Wfx7YN85!I*F)q(t3>1m)|tzo3M(QS-dkrAE&6!msAazewxTjjk!h_`Os z6CFyyWVI2@qH`Vh9fS6j`EBBgYJPQdVhaK{^>uh?lhs=bAwU60i6kDiF7Gpfgb=_J@H@%vV6T@BtG`;YFAR^sP) zO!?vVhwwNyF3JRtacfvSE-aoJ63-R>kLp80gp{|2#ZyD$mN59R@=fsne+*w485WNU zi${mWW5VLGVez=|{)f`9GI!bU_Wuv*8~1;Ce_MscTZhHlgvH~-;t65##IU#+7Pp4Q z+lIy4g~i*4#goF~$zkyhVeyV(@szN5r?7bEuy~iSc-OG_RblaNVe#%^@zk(*kFfaF zVey_}@wBk`HDU2yVexCj;=RM-eZu0`g~hKAi{B6y?;95H7Zy(si)VzzGsEKj!{RrF z#Rr7Nv%=y7!{XudD?2Qn6BZvF79SE89~u@P78buLEIvFep8E&w3!fk1^UD$u7LN*x zM~B5@!sJO<`N**HkzwT{!^%g+g~^|=cuZJ4HY{!lhTIP;4E$}5CFFjsXV~@UItxKp zFVC=E6u8SXtOo@adWLnyfOnoF}81oJMz{GDJPML(nvm^^}coM04$5xsNyqka#Bz;z|K`vIcK z(^qSf|!k;*U@W;P5 z!5`4zH#Y;nnc&x&;osEY9}R&As=|hSD4tFZBHeDnI2*lfQ5`Fk+EGGt{z~#? z*5&&It__nGE`JdG&~rIbT>h9q-qoA%rzn1vWT^^%E%}ChdAhT(4X0y$Z-|M|$89AT z>kP3{jG-S$eLTb#oW&S4D04r%HY$p&Dxjqff~<+ZbjCAb^3s6ZR*;KXSAznJd!bWw z=v>97oJbJzpEUSQDpy|SI$0GB6r@mB<-$h{?<+j9;OJmd*n_75{SgU`+ zIDE0Rfqgu{DwG}7(9(OxX0($5t@;4eJt6mLL*KjmMUsnanP&ihA78z_g4aG&tUs_p zZ>l>ydQMW_37NY68JZopj_wfRLmw=DURyL7mE&nGZP1EN152LhX4USdu(uteen$p< z7N@DtUOY9JUra1Oly8)`6>Px|j3&JIH&IX@lYI=()FOS4=UN2z!bo~aw6k1Q`ZYc# zN*yk}NKcC9vtgqW@zzQGNDz#Fjv}wuJ*oUn({@=_;`_}IukQf)xAFY9)%>#qtkk>C zrxkV zh;b1~L7Z2rCLeZw&n;LnO6h7ykt5W{cMN$Pa^FcQw0e(9asjMn?poKR`#uKIHC_ zpVPRg1HLuLtpjn#lLT>t`!L|`0a^1uRP3 z-M)Pp=#$Rr^KeIvKIhH!VPBc)b6%s*8iPK2kzDhi`H$%Hu}Ys>ojxGEF4K2HlzhL5 zKH2{%eb#XL{G&zsSpR3~Q~TxrUG!=49+-SuQbb8{%#)O?E-_t5uL*u_tUu=b$fo(U z{z#=KkEEi8fyz#q-<2))@#Y?bMY)e}Ed8vUOg}Hpqn|fQ=;xgZ`dPo4ezvYXhQ?lB zHvM=rj-mIfuQ&ayOg%<#ARYKbJA63q^ld#3iQCW|gA#Jb=lJl0*Gll-I{fj$)Reuo z7^Gc9292rLPkBDSsWhJ2bZ@OaMUx!eMNrRb#zL>73qR4E+m5>vrP5v4RdqpRGojih z{71flXg^jTHjC~O+U_e@a|Ca9Ye29WBYAk$q>=Srt~KNFsL*B{stauf)}S@xo6nMj zknbzeyFizNNdi4tg*T&dd7Ei=t(WBJ>#+*M;htYZkA#w+2I)}?C@d*$a3LI_KYXg+ zxoL=OQn(V0f84(d9n;b8+*tE5>B-Qf7T?pQ`!T-Akoz&(HJ2rkA=g~O-j`F$FO6gS zY$87qk zLOkYkAC`OttDW-heEIa)ajeZXv~9D>JMkrhk6>r{sQvKJB(ZEYY&wT_aODo|D0=Ct zt1=o!q6D8n-oh{cXA7eD#Tc%^HYN#V{myLI{56V=%7FkAc$EzC?$we~utxH|G?_mZ z#|F*d?{&_dq3#WqlhdxdT!k^M$Mjyj+b{W6qPESh#PCy=BTyhuaqr4gX79D<{3XU-;Q>&n1gH zetOPnx!RQ&cnQF!BmAwA2vjF-Z163K%{eSCyNRv=6qd+``Zhq81p=wW);M;_af9o&RNu_yfi$v ztaSR;k5!*|i@YyEJvKSdV((MG>xyc^7~jKv;_<*5AFp0_I0@+3#+j2}Aui92=J(UB z^vXy7XsltkScb>fp5hXybU43iC{>+`bSi}v`SRzEsvje5CEv$x=4ZQs1K&7-PTrwt z@h|x{wvs1*cHsE=x*m0sr7`d~a-x7G(Dzb?$_<-FCkWfo1jy#zr0UhNc!(DMh^f5u zQMfdl1>}!bNe}W%NBz=Zu}^%w|D-s3b1wCdqfJ#hLTTf_vn~^M~>*hw^ZrINzFZWncAs*1E=11H(35!}FV3QeBJ@xs=`18Gd_Ip!)3kCyaCTiG-8vlm>>$5`BcF?FM z%t}fY*e9^l!AH0ALP6f-AnPwC3G7_}p{H6|9<~TO!Uhi&$#c9v{Ni)n1|!ZJm4H$% z8)u26yjaVhjCq}Y(OXBy1aTih z;2dI+ybhkiGEx6mGOHsH; zhI}j_;aOAhRaCqau2ki4cAPPrm8IO;KcpS$2^Y|c$$76gdk6$Z!c3Sxp zjqFyQl%%v*Z4zHm0_?Hz(b{H>GUoO*NWe7W#&TR>XMUnBk^7ooBKL_K_i2+3CtIxK zdB`fb#LS~1FUCTDAC0t?kFq#?OXA4E!c!fujEiF5PJ+qnTatol5clkeXLXQ-53%4Z zS!Zb|LE&L_uXQN5q~t53Qeo&js2n`7%tBDpJ>>*cuY~)TL#B#Avt%~@zv zQcW49(ZHc}d2x^`U7jAqrHe8M1=pzT>A{lg$d73hv~o4JY^DD8?x?hNeQX#haAIL&o`NMQSP3w@(@1@y~c5m2ead$-> z(sK^qJa%_27C?RU+1rz((7Ehf3tZylGSoq$ zU~O*;FSPMyN>1fKcXFV1>NDAc^Gq_d+3#zppSz|Pu)6|5%4lpKifgmzY8c5cPA9=4 ztu0=8w0K!TpI}vk6NUQruupwdYEv$Oo-DPRE=!hm7FB@i5 z30ueGx;L6SYQBQw=#X}v;So3a_#9^!`-_748#TXkUmZVKdXBCqj?QinBKVtYsC?Pm zqTy>+8jSu{KKgdK@L=Xq+qO^!sJK~qI7?uUK$SPwiW|IG)V;MKUmYjD9d;R@=$$Ortl5`kjG<@5;wj*m|Rmic!(>@Gl}`dj{D9B5KP<1)me4_k=da_r}(gFgZXe^)jP{zowFYfsU% z-|#a{`}R9P`^ZLs?zOu9K<6CXR&X3gnC$Uv1m4`_Biu1l9pR?mi#E+~R0t#3-xCDG z6cWIb@s_`5DwUoj-4Yg4PbI%O*&7K1|{-ODg^!!t$w6$barXw zX=v@Yhgql5EsO9L`bfc&LgTqKbsM@=(3Sj$&cq86It1h{8aeDc_?Le;rQ-US9O$4e!?~KhDgW3|H zwkAjy{++E%C-3sLv_E|)iXK3#%~$Trwmn*w7Qo%|;IcFvcF}Vn3M~MSr6Kpq`71C; zxa9oRnfuG_hb5(OZDz2(gXKwtu+c5Iz`{uFd$w|+KTq7iY~seUw05+C2_Qp)07`?j z$KsLEeDmXHd z6_>E>ST5&?=sig3_iE{HYU%Hj^DlbU=4LBvPg1Rre}`&5zy?abt=J5irXeJ~+a>$&mC* zoEw)wwm@u*l(R$h{sT7VHKqERu?-i%`W{duRN7Y27$!^zBnj-+8xyJjgMjq*vIgSq zvNN>eG+JOYZt$kf(-;8>mYt41yPPsi3Y_}l20r&2{kC!*%H3$u1;MgRXi$AJj#~-M zJpplXy%psSzQ4(?1mclcPyFjL3U3^hNAnMtmm@AEe}!Crk_)+v8nRr|^O1K#>sPU) zFk_1~l0Wa0P;AQLJzf)`nh&Nn$?~FDbo5u{G^`zK+0pAnzUf9uZ(6*t$?WC&cy2uQ z`j>tZ)~@%V2EkOrTU5gpto+6B@4t-Bt}eko3Ns86M z7o=c~RO5+L3)Fb5YKT8B5>=H1X&KsPtj5#Z2&Njr6eGCQ2rf2)^NipOBRJUzPBemJ zji6)%bB$oO5q!c3mK(uWjNmFG_)jCa#t3dVf|W+_VW~WZn*sfc5!8l8*FMeX9e+3;uI>LFkN=(V;OBKt-^RtX+HA)P*o|j#aod7TVAt|6 zHj;JWVeAGIc^Es&B6%1|!OoxIXAr0GS6PQ6U>XQK1+-$e@I41x(@O+Nzix&lWRP^Q z&%Wber5tRsC1e0~uz#yyf8t;hEg^%egDEQ5jT~&OC1l`ruzOXocn&66LYP7a8>WJN ze+_}0X-C-o8udJ71u2j_OHwbOWf`Yki9YR0LZ{u8E&rW%|1;ChA>We?`_oa@D~;nAr+iYLq^OgO1lR&n_9!_OJw$5tT|HfHu zTu)KIPaNj&rtaSsWlNrI7qD^eb{=vD5ZtYlkz?gYuuD>D^#uzzXH1-hdiKa{rGheM zo;{F1)hbo}+)Gl*3C40pk|{K^0l&%A=j_b=_#0V)NM-uJ^iu8!QJCKWPpdd==??$a z(?OmaYMeOs$8-3vr}%;(zN|)6e;AW}L;XhMm}e}4 zT$Sm^qfAsbCFN%5_$J=*1kWbwIR6kFec2Z}eBT2W4yKe+|E{+mU(nW}+SeW2zFwd2 z7L|WtVI`mC>|1!ezLg&JJ%gRi4Cd_16`!gRM;_|4N9uFI-8yr3>7`YJVGG(?{zO9k zro^4>kYjA}Epgg=`c}<|d+Tx9gbt#gguYhT$N!9MaR+Rhn)s{Xo-|75!xDoJOEf-I zMvs+f+Z#u|=F`LN+k8W>lp}nwN~T}W#mMu%rZo9h(V za7_B48cjZIsD-_Le+(MIssEA)l7So$pUl8j70J~){0<5z}yIfAPF4XZ7OYBi5W zPXA25r^g> z@88C@vhsDS4PUqL+gDCs-$h?0;Zx?0nodUfZFn~XEsEDxCTTw4cRA(L&OXN_&xK@H zA^hJ#^nNes8%X>_AiJs(pHplE;J;8ZOt=ih4p|YEH1W<=cS4T3i(c(FCLC7v`9`#$ zUr2VpwmF%XZKIa0-biI@c-ih+c7&S!CeN;Nnuf>a4Y*>& zVa;dbtj&Da6QlrLPEDnOi{?+#;JC5d(LTuK=t@6Mk)8bgqoAvUR;Vj}hmo4=cSRut zWX!-TyfFIedmO_aJxVlqj(!pf+JIccPOzoxbo->lPqkM^g!<;1{LxMP#5Uuzh`WJX zR#eHgP2gMZ&$26ee^&TaU;ME5_odR4t2wIHx+*do=kGM9=|;x6`-n)oLp zv37GD$t%^glB?&CxrYg(czSPcW}>wCkqID;Y&NtkS|hY~5GuLHz;JYd+U3fW?Q7V9 zooMZ|uW{WNK>bF$Q2*T;(}nta$Ox=}oNI)jAPf^4(d3FBi*$H`2{yVGl`oE{0lq_3 zlBU`jCn7&I-6Dt^@@wUNRVU)a4F$Vf9fF2riyJFZ`OIk)y`=nCeH|yY`UHOfW{=cr zk5u&y1oJ;MQui9E9~r6p^i=o9QmaEC(@<5F`kWCxcm~Kj&D|tqdxf~cUL)IAW8Y=_ z>rR<0T=az_*~@KxoOXU&0*MqgT4h567Fm=4ONmqQarZh5t|TKFHx)X4XaFp(fElk8 ztXA@0A8WhKHqLfCTf2*v_|!7LYY~ncn~N2o`|J?z!8Od1i-sS%b?!^}=7=$m={ql# zjX3(BrzVosvs&oDZ;L1i!{0O>Wf&7UM`L?&_@a)`XVlXr@wM9dtpoT0BQ~@{BJm76 zm#fvSJvnjlZ^lr2@kL;%bXU0$V#LYEL4$Er$-dhdj6KF+d}yTZHBvt^Qupbp#V}eQ zJJivVFG-*$DehH^a-=>Nu?Gt$Kbl`taxvLG*P$e(k@2vK-G3fmFHE@E)Y`J!aeSuy z6UFkS;R~ZcwT^cvtFZ5&XdU?0Aye2PSMrK3s`wYJrZ1gJ2flH%sz3-{o$ECma)+;10K=p1@c^Y9OM`zS0p&~WiQ;EiR!IdOM({4}6W@oB! zT>J^7-W47uG1z#B#9%jwZBEAR*W!1Rn!JDgAL9R)--Q1PA9}DI0b$5R_Dp~gRUe8ez4~wsvl4Cun9fPO%Tg^(w9~c3IV7+PQ`7-HeKZ>+w2%$ z{1ANArsvH*XOZ<&5-)+HaLX@cH|Y~59fLVykv@k4B7FI(`zo4tSno98I4 zxdE#MEDJYFGJe8s_F9n4iv3`DB!3#lhGG({W1F@KLV1zJCVx;qG9phtUS1SgJ~GNC zZX6vmT^`xWri_k|M@FM~#;ZB*HS~U1ft5|!&iCVF759yc#OoF1540)2IbQNDNFXOO zPt`d|F07Dh?5h%J)}D+*$4Z>uc_5_@0u4Trd>){#QKo*4 z07w$CEk@wSVS$r$JXuT!5ku_dqSr5Q&Oq-iNw#ovI*Mp6v-uZnK4TeEECTFIk*w1X zn#sDA$chwWcXtIXBMl((LLh1h!if-r5h6l$+nE@0d^zmCgT$3_^pnv0pvINU{!X4L zF+4kPfU;X*Had2&BA)#NrDyHo*#c(&3A6JcJLLP`d_N-(su@kT4av7O$?02%ito11 zu|OxpSK|?ko%;o2yM9^?;wJi@BzhXU+yE>wf{TsdJR>;62u?PF6OG_lBPbccTqBrm z1T&1F5NR~lNIh)?nGrm01dkZOIwM$X1pP*^(g?0Kf@_T6Y9qKx4VJzZM-3|pLSm-T z#v6=ani1@31e1(lycYBf5nR#Q26)VV8tvQYCt*H=(MBzcO?WFQ{Jp$C8t*=D{fFZ{ zr`>W=)}=gk>kR@t*!>3*(I!G;(L)t=Hh#v68X{$;9rqiY-x3*}@`1 zDMwG6eQrU0V$1P}P*PIfHlBARB9yk2cl-6Ihi$2g2qiY<4di)9Yqm!eTN(mo%i4%g z@vk zeAb`&9(KovBm(cDpM<4fYa+1ducZI{iDwVz*$A~6y;51;Uz3EU{@3D$QI>U@llw$EfM}cpF>PE)g$&F0!EQ>O*`TFi$?q+IC13LVU4x ze?PLvrh-h?oXYU(U%~O<*oizvaE8z7(ikkeP}H}4PCkC;aH=cc8lnq8-nJ#`~bnq@&Rp?qlWk<9%S z(}%>lx=6ttu-!;;=6o~%RDD+H{T~>jo{}QPmE;JbcP5mrwpj3G3c5V36x2%cSm;3< zy48%$P+g=Jp%Wl_fAdxrACi1_zY{%6tJTI#%jAtgCRUq#R&DZL-cwhR-bi!*u)T>P z?H$6u4y`p2&G9Wa@tt(Fi7tG*uMp4kY7A-c6-_BTX=Q0*qOf*O{ZxGZQlm53Pq|a@ z8yVyy+><|9$OiW37cY<<$M7oLpVBbWnc*-~9owjm^Ehq&iBLy>yxCoD$=og1hTXx2 zf>EY|7TW*e!?tWW7EG`IpozaGy@&CeQyGNU4kV={6}cH-jYPr@LIe$u(5w)>GG2`& z*SAXgrrNh>ME>|;SsJNUYVux8&eLhir^owQab!?MsVAL1t71fYwUEoyF=}xYDflC$==+}H;3~yT+-d;GoYvL0HF8%S4 zs=g2glhkI1@;*oHD>#fVIgXV8vlm~Dtg)l<8c#lKo*)eF^Bn&NyA1rr#CQ1Bb|wLS zaYMmjsslPve*?fiw=29Yw3Y7~(iXHlu+Lt@$AOW#1N|-PWmX<9p_^`kJC1oRcz6*E zBL==phAs6@z^|p6sN&^5ih!u0qzZx&>gRB%nzW-&9TsHMLCNNyta` zSkH~Ei7B|+LFY}cYU`%gzaPo~5;V z( zuTXXOhgSDERfmR3_G2s|y~{xA`e*$jjgk&!NrpQO=Tk?}w%U<21G>tevm>j}TYpJL z`73Q9wf<@iCdvf!o({v$#%NnOHkC^ssPu%-=t{7m;P&^_f^^ zI2vW68t8pxl_l)rhUG7kbUkiS`Ji0ogQJnO)hKkzXC==LtE1?ac$foee*XEtLI)|i z!=1nt4o_pk{JVg5{kAu~6oSr4?4L12x>TbVt}X%eg{LnFwla0jd%s1!D~p{_2g=s4 zT&3~S4EHs-b2iftA1rY_%Rh{fSH2MmSl?3vYG(qv)c|Ts$uEsn{qVQPCeT;gd@Jz^ z_9o*#Kyd6>3b79z?y2SUIUEcgz*{dg-B>q-MF0XVJBpuiDXw6#ki5=P--jZXuHXq;!ltcF ziGGRf-_e-Am7c5U7MFkn@QPw$qi66KZvBd+koU_#+Ds-NOgtGr&;Evwrzu1DKB{6< z8zM}DN|j|ar`;qG`hJVD$2DJ4y6AhaNXT4^`!9Mu1*tK{*n$nclGySVjK6HoQm!Ow zJ1~_5$63@Bodn0(`QL~H8{Th90*MgGO(8(jX`w(xgbGC|n=R^6pUoJl5W{oe2U`MmG%pPwH~X3jQedCs$+ z=L~o!D)PQN8{XjJx|kr(L-cAN7dtPOG+ zS&Db|*2nc0jcXu5u65&@L`fSJ9VgYcWcgD&U=uAlN>M&Qv&~-q+#WzMJ$KU;1TW)7 zhXu3tKwrDlvHe*>y zHt~Dxh>xVqACQ>M2Fb@?9(*?Wzoeq{1#J;vdjFoVy6BqVUlW#u#wGuXup|scjNiqx zaQ;PMS!m3zofUe=D9W{yLhsE4^QymN6ScXacO+o714M82vDwy=)O1Uz7;Yw8!E&6} z^|ft6>9JC1MtJJ7)I*Um?iK`Kt_$FA-c|m?Ycu1aY-jjG0&zF|VZa8Yj)m~I5dPkU zKlCqqOw_BPSdopz3*+-~yd3~f#tS_8Mik=7Y)KrES;c^GI)o@UMAONwQq)`t#z5oCXm(IJ4a9?b<=q#tcS;E(+^~Mu2lXSMfW!> zrdwFbIp|Nlw(|%|vzy$b5}g=&-#t1d$(c##Z9z=TxAhy;VR|~g=8M<%Q##>m)0QSl zpO`DK0$-XGHFq(7@JW_-b27*U#`#)Z_F|LsCY~_t`gC-woWD9G0enH&G7fP>dcBqL z4K7Dr0p5vh7Ly^3M0_OHI>r# zn-Q1fa4}??^36a`O(5E1d^2SHMm)xxG~#6uHH$Y`Nj$R>%LpNlDiU$|fZasBaui-Q z&f%;OpuU4W0Oa*as>n_5zQyCK15`9$ zF?oEKu41~?kLd}XMHevrZj)!xWqins@+|s`3bd}l0_{9YE@JuvCeM;9_%JHkvm}5I z_eXh_T%{DVXGv5&ra%^k!kzyBWYtqvjAuyGZlv&2GKP^JlWb$WY6 zEs^Ns(>uUpS9&ZXovK>0fZz7>PJrSV%Pza`PQ zpkrq2MrKowy9GYjnt0sJ@ZruFkGmP+mcoP3{`Xr+B7Um=L2n;hIk8HvKpS~p{>@l5=8ao$9x8{7 zGp|%kTLOpd-u5YU`4PKG0VVqRc5)-uLy)Y;2$No3BJUzx9&O?!lf+Hq-m%_{n7)6C zwkh9)7%wNKL;CK*ON8}a-B5Tb3Z-;<8@$uByKvjpdOCQUwJYDIzLrAd%sX%tDjss` z3h+L!cxQ%OHrb3=YvC0NjgX4Bo4ICZO3vo!3yoiPw61#*3!9y>*y%n$A`x*m=Wq=B zchQBQ$bhA-BSP49e2nNa`CInJk4_y$Q9tG-P!DthD7&XYf_G3RVdDw#j3+md63V4D zc=6!^Ebpqyf-R09s12W=DEVh5;3Tw9;f7G@WOCeqBUnE-$}m@l)AdMht{IahKxu6x z5!^74H$GHs09HV$zjeAe5REO&vPBuXzBC=9%DL92kPS@s>};d=o<1tOKWod?2AvHA zLM@I@55h&Ylt8d)j)2X(p@{$CbPU>XOA6jzf0a3ttC#S+dy`dsLl)^ueT$YsCNL%d z#@KqGJ;L7UihOyFJ034%@)o6_nJ~*cKGCZrrB~Edn&qO=r1=|-gP)lRgFo3CqG#BV12R=M#SDEPoPDQG@iRfvOJ#ZIahpvPDy z$LGe&s{_L`aVobZA;7!SAY}k*%4OHOH0|UNE888M?H`~i$ zsEFmeY~afM&$MGeKm{@`rS^~Jv=wg?w|)9bXeRTLY~{Lg%zP8-w01#qoa{bDyQ&-! zxYqrQDFl46jUvV$dcil3^wPf>`H3j;iFcvIYN3J_`P#`Ns2CWVw0Pp?)0Jx3=Xu?gMfQam}2UhdtjV^4iDg$zw6OTasXF;%HeY?Zwt; zrn{Id&pU0^3%`RIpS2NeO)wcxt^j|=$(2^Lt+}x)%qhC>?!t)pmewM4W@Z<@yj?6V z%o`2CE5_y9ca@{j613_27QRE^vXh2@@#cQfjwFf}N2i3XyGrXmnk@iJi&6glLnki1 zX{E~X!&=Li!dI6~~i{#;`tzgNycU4?Sl#Alb`j)^5hxhE@*}O=?6{d_`Xrh!mdocD92*v&f<80Z8voq_BID0752_4cu ze3rT8+gdSZc;dW?EaV81Ilj-Z?v%E285KsUtxRz7J1{5+;0QFZXh)jH@%6K?2^kEl zv|fO&;FDN(5#SVM9CE-lA;gHR`dcP$%z*_|-k@H}&4R6gpF~WHJmGoy8K5+iEAApP zfqn7%2+MliE2w8MphdjFf6TMpt&Ig9fx|+a#APUI*^U>+U2%%qL#E%`v}~INtLZA zsf(C0?fL{f>VZUUdIGLeLW0;FZ?$e@D?~H5xbsuk(DTk%BX&==CxZT`BK=QA%}J2E zINroCOO{44cRQ@sH8nI{<)eAJikcegMmJjbAkZQI?_EV*yx3NDXPI~--SC`|aaz&U z&{#_`f?ppoiAJG_>h8*01?MG$``~`ID?{_8zf6+At`E`Ym_KpNNQAAK(-ohR;pxt< z9DR?Osd(9)P6>C7uPgLJ@hE2!Pl_Exk|}Jv0Wn=$SXJEB)*h_`ll>gxiwNCQDn}pkKrj^u+Ky@Vn@xAcO^RqET$jgf(jrXhl=omK6fIz zVlsa)$iv!uL9yzrl*Bd>-T3JS=*Ec~B6y8$CdO*dCF&NNAZ|Mtw&@l^&DAY}NKQMv zKS)-l-5}{#FAn{Q$;!AJ{_C!+BD~cbxkzN}$ndY2s?e&OI z@MHM##@RZ38MmeAxHDFEKW)|t6R~$MhnDocl+f1oj!N|AB{jC125zWsgIUlnbrG}0 zQ(`fuA$0z|miPcn@u{|9OTb1@B(Q$jEbfHCNw{!x6k)ZFDTYBOHo6cN9CRCFx#o?Q zJJcDvh1TR{tySZ&rBbA7LBckc}@ z-my->{kUcy@5k>u{nz~%5|VT|>L6(Y9VvngV^VEwh{X*w6I;tI8`(u90{0d+vf26p zh4Z&-8a}>8aZQ&_I*_)J-N!F7vjL+y85(2M|H}ap*&oxKdVhNXiFYRFt%i|3!XxVx zr2jlR9%`aK)RPH~4D3+^;pa?@4iLJ4oGmf#SaYL5IrMs{cK`K~q@%#q(Zm*&(`Gc4 z7|DT;rm31_Y7&2S0^tz^LN02J2<5@uDF2m5md1Q!g|&W=@2}&&p@K;+lUP;X)mFu- zmt@l7=o*&?M;etmzGQ~mMaPT^kLk&Vp_&?1c#MT2{-=`-nD5amT)A~)IN?1qhFA;U z18yqgxYgiNQQN)2?Dt~UYd98;J#LMF#BH;~d$Ok^^yFwqW+N2u&{5)Cnq2m*S-U!~ zQ616i9fb~S+{jqPRTYm@@F=WvsDz^)A03?O~usR9g1VsSt=7VuzYN_>jHH&ug=Qv4M z2H=2L5ES)AXFrdC!E4eDR*RW=!CsI(wg7Ms}q_Xs310EBl#2Qj;Ch>U%5A5N! zoZ2VDV{{Q{Ycr!_h!jlLldRAA5VO2X6rpze$5%}yb%W4C>B#TdKe+5nD(hzX$(<~! zL}Qi8999qH2=##LcS}g20Xz#9o?z12ZfHt^k-H+OwCYKXayg;=dRKg}WvJU)u7+ts z^CY`JM+ABfD9&nY1N48-t{`eW-ad%_*Y0NY|3mx#1-8_OPw4NfCM+k;;^DK1DRd%B z_K`IBSsF=;vn1`-pU^o!TLg5;7SM747y=xI1JSH@_yBs(_{c_}nrOAV#xhuw2O-Aj zFQN89N}b?Hq}`iyG$a}ebUx(<(rcX3#)=-6wh&5-As-UIMR_$acpKBnW=V%ZiWqr|bjT8Yq$P3!5M|hCi^+DD(7-Vi#5U`Fu z!GP5_DyUYwTYU6J;geVhAM~F(PP;H%$-6a%rv9sc!PHMYWDRz47S(Hb(&FG zVx>j4L@aVEEO-4ix<+5@GMZ7r@Ep`SALp4$F$Z@7Vq4Up4i5^;Xin1eg75|ekrUl`G1PO-4fl1 z`u_)pjKx^>?t^?$YR2JksOSO=lK6Q*7IR`%Y7Mr>mXYHCj;WT|bp@Pp?KVFUV-fU8$-bS4p8#nXj z4)No5b+eZ6ORQw`In7FRNs>^d_WFev+ z@QNKoMGSK1C8<=m{0PKh74@DD)}u+NyN0Fm`@}(m<vjmcjst z7xhlIBYc8t$M~NBt7C*BxpWbBmEf>;WCNgU$w1e6G1U3#0MuE=E&K7x>2NByiub|A zH=yX32O?SPXQom;BkaupG2Q=SBK2Xi-Ulo@aB6S^LzUXJ1|zwbd*#DV4LZOVn)Nyu zoOc~Qw|^f;S*;F+?~*_(_^>q*LuE|ugS1TO&W8d87JLi7=HM${-DM9{QD6zdacs~o zPD~6EsvEEr2CP#H$)PH=@mQ#D^8>n{)9TM`#f-n}N2Uf7TQv~-B4wd_v8#`^Y6Kn3 zd$5O~u79={yR+{~L~y_m+<_hn{b(bL(> zcb1b7L5vGsek;3QG&j<(;mCJBasipW@S`q)yfpmE2ft(1zlfKQP{yU#X`XUfu%kH7 zeK!aTrrF>j9-OhJjbgM;E#doVElGCJ*(+Sl*_U`V+cGynV>)q(w{(SB(#M9qT|p95 z5&e=h{OCz2JZo1Rsi=y^oQ$yN5wP_V=3fkVRzUC;nD#pczpIi1^qML&-Eiw|NOfvlsLv-Awvh?qTK0Q-a33rBB*ZP>OMcYvd#zDeHQ zWspz}3Up!hx$Un=&`y4)kq%&RfB&&R!nmq1)K~mrqU1k~%Hji!NXoK%H+0^Q|3qS> zFP|3Jt5L=sTlc>shkO(m^ov&-`Pt=k)}2o$xk9Its+Y?~cLfVuEgv=>iSV~X4NAfL zq&%(YF-?;9$gN*iHIfS9l(=7bX7<%NZfHLGgR>NU1|NWQo6KpTYYz(5-rWA`^ zQ7rTdap}8()xObTlC*nP%aXp`0b+qA4)3MWx1g~ZEP5mkBRE2H`=&y3(W~XK;S7UZ z|Cl(**x?nI7>(e(p*yC)+C*@F{m^6(L$|Q}N$O|s8VW1))wgkUc%J!7Iy^6hc+EQ% zAH56R0wRyQza@E8G03Cw#3+73mz0n=O5agWXa^oolvo9zx*P;|q9@hb2{iwzn{}9| z{R2tX@GK6^uYGzXMyei|naqos#ypebnu7O)l4bTNyEwmBsq)LDMNqs-j>*$LSD+0sY$aZn&!Q9OCEm25+@mZ!W%1_wQs3^}e{B=8M z&N2;imS^D&=ZP{-8p`m?A47_|WS?N}D$nyo!DH2VB+480IV;20)QtiG0tUs}k799M zS=i+`oy(Y=U+o@CBf<@w_Co`+xjgqZDY!p%50b@y2;pTJ2`|xjx`8I4w_LVpEc0}T zwljM$PQFBFn?9s-ZTYIrQv0BbJTuI)dv|Ff7;`v|w3?z8xJ?XohXHtRpOmQ^VJww5|JlyxZsB(C(65R^4HR_WY#guvq)nn8LRD zW02^d(mc3GIHJVwW5d*i2o=S6wTy3aCg!Jn&m{CLaS7AN+n#j-ka?dF6MhI~#s@R| zip(+^zTj=DwSi}KkRu|S1+VP#k32~eQO9N06%$1MPg_|Gc#6m@6KU_F8KgD$;VPme zjf4z%dV|mM-HP!=u=xz zj?GDqw2;t5PF%`ao-O4-NC5leG()KS<^wo8RFL%HyU3I69yDagWeZTptnP9hj#78- zI#qch#&yJ$pRq@F=QP7Dw+L(HySF-r=iAhg@daewj^eZOq@%aVWz*udQIkQe>1TjT z%37JhW`GNtNPohcW zFsURd>QESi0<8`OxWQxa7*_?8O*h(K1!tpFb=BzcqR;6x1rD7kB$EJ7m04cb4O!^Na(oFJ2De5L>xv= z)`B!8k&(pFkMG8$O_(W{m6L;I>>(jC;03ORkNrlSd>cQ8QEhE<>1$+6#KxZx4Fd`# zlj6$Xsr@`d^$yfm+?#&9TC-7tQ;o{F9^0*vDu);(U9HS~?a>8o$a^M<-3TTxW#_29 zb%YJa3QAzFjSVvO>cC8W)SYUBiUK{UV=ar9Z4}qWEHc~$wFTQAOqBFFxp+|USHDx{ zM!f3|Bb>UP%ar8TDk={}xd$f7Zj3DA9+)J%OC;7RNk@>e(3xEAk?kfaKZ7kj^A?IS zvEgL~gvv~L78Cm0O>~2HHLy%DN8bXUr1kky#`(QFDEE_2sBDlKk0=2W!5|Y##yd46 zl{_83j%#SfzlN!)y#e;tO+i-R@&AMY*EuTpA*0+M-LNvRZVeMY3Xg=9pF-vPF%W-* zbmFvx|T5$HW}8}ibx(K{Z+5;Ey$ z8yMwn`a}3HvgrDH87pr-gJipF_9)4j5M(_+{wds{V;eM4Q2r-T#)k|eJVhh?5BAMp zA2#;2l=C^}d?(1_2OqM}oIMa$@^LDODjNR>^?DC!K2iqIG4}3wq6y!NGzQIE8gPTZ zV(pWM!rJAaHa}s`+rJSipfe!Jco`mPApHrh!zQ_uj)b@lnedjn7k4nukjn^Zd?W#b z<+u)=|2~BrU3a+(Z=1S)i(K>u6&I`v&7oZMAUB~q?mBu0Cw0UD;^(iAZ~+ddyUt#< zcZB&Na)I#pN5RsLR(LRGohe=z+O@(KCduZyYLbiY+kwa@W_+%GY`n&KdzPvxg`1Fh z3%gXSe~R#?Lw1te$!LM>Y)rRMZN~;JYb1r#&|DN!W8Yj3sR3`RFPbs7A7?+v*{eAF zDa^*WKH8BHlzo`9-{R~)F?$LyAlgRGUP9@&&gAS*;b-+(JP7zcXW!1XM{{-#*0$ld z-JI>?+52MGYS^ryQ6fdxMV0x>m#z}Y>4z}3BhfCtj&{1^xf zgS?;Mcj0$|z?xlwKmt5_z6}JLeIE!^z?5Eratfrqwm%SP3iYPJUmF1E$aoIkVTp?C zQk&_?iZ5o>Z0t^JilZ?ucD`cT6*vsZYakgzn8eI16c3yA6uI_zC`Cw+2qZ z-vv73oEa8RB?+Unc&BG7-uo2AHwJ?aXg!XCT;rQ{OkV-IZ{{fDigqFzmWQk&ju!b@ z8O~$QpNmc6_VJ_!w~t50vo8F-4=IeU!hUEsB=S7uN+O-}786ExlWxX0j}I5RFVfrK z_PXA|H@HLa6h%zg!>zd~@+*f*d)CDVbOdQ^a+#fj;9Mj-5N`d9#+p_E-Aqd z;TWxWi|M);#*Zrr;{iM0t~dIF_coeC{R5hknjFV0N&4X(eA8-MBjn?7GS*+3g}uw+@eHx@R9fPFW)}eo#|FVUys1#0d0lZUj3yh@5|*3Ye=9bhEkkK| z2}{lHzZIIdmGQkRz9n5FKyw8EUz!EgR^brHyyr^{fthpA|7D$#_w;w~hNxhA8muSJ zH*HkKyJRt;igoD9qTPettA0@r*$R(c4PCs`?xT0HacMlIq$}lD)|DP(JF8G6|rC|0(Ppi4trI*Fe*uv z{@V5jD-?5GY$CtL2Zk6Qq$Na2zM)aDL|NN$?R|Hfv(!4*PP5{gp-6JpE_KnFtLTT9 zRi~Ex!8Tazh<4w*0&1RUjs}i$>?E*zc%vHwXn@#SAWloLyuU-UFqEV_t z2o~D0M0v|BfQ9{Qt&;y8>|w?d?4j&-$Aq@|=#r3IBPOt0W1A9kftbvr5TGa*mB!LI zXVkNIBc5=@b=KN;kD;FCQ^T}1@U{~KE}yLh-j?gS#~KE*^QoIZGB)zP0NuQKZMe7; z(9RA=xi^MP?F)i(+Rx3x8e$0=4Zr?`PJbg!lP*V|+ae)wg`l_t}>hlRUK+#ng?{pVAZDYfYSu<+aFsA11qPiH?3= zR$=NXNq8K+Tw~>_(Y4ROmV^^cpiSleu#35z`ea zehS`d$LnQzl5)HyGr|gO#X4EUXdRAEXHu6(#7&NgTB(LaM+7p{Xg;XsJ_*tt)90-X-SIgf9t zG3&cC03HXgv`XU3#C!OarUY?1LY${Jv(TbrMEcYhKk|Csfx;o|%hZr$NWm(lEPhj=2$bSh!$)jpgqT zFaOU!*#d)x{W^eC24KfXj;e=)+VXigO;Xe#)|>UYkub?cL&(rn!r8om09ikoov2GA zWD*lxPqj9?ELQSOMUT80hK;efjKOLy!@IyB2#i{JIRW=Xw2m=$78NP?i;a|<8&S!_ zmHy#fR2(X-^XfCX&NUIJ>alZ@^tGv|!r3|Ua@P!=%JEK^%D56M6E5!fY3O*-fgpLd z{UMm^gW3o01qHS4sPprd7`*ok7zz!FZ(c>v^%TGtD zpaUTO_2&r{F{-CUQ#bh8eT0bWLyL*1=85I_WxJ86+DeG1%JV)VM9PUhN(5oWI@-bQ z5*K+9iUf&ko*=G{AaTtT#P!hz#5Iq}i5nP5?aW&;M;-H$40>ll|1>;-8-u=-xTRc| z@gM_k@^RbrQDtlq1C!@ok3pDv1&iWshTM)^YECPv5apb){sI~gPY0DoN)Et$=5;~B z4;pr~_YE1v1N~wI!_Hbs;NWGI^!-x-90Fg3JL3Yp4ha{ae*~V$f4mctxN~AIv>#um zxYhax+u;Y*C1;|T(SKP~XXifu3rTp+8Ltf>yy^c)YJ~j~d{z?|`hS0A4eP;u*n`JA zMfwz9&rtp96sXcCIE<-c7vx9Qk z)OhWidkFEwGjo(gLBN<}Li&YYz<8y*1Re89Ivu0Uetb9KfaigR;rjp#Z|2-d_K&Yj zBy_?$<@Ym99i@+a35`68=XuPtFlo_W>NIrZYN#&NHU&0$XMNEBE1-R;2(~t~7~7Iz z>h>iQH-tno-=@0hypgN?BD_M0B}Am~K26^lg37y-6g5wQUl307G8JzE3vNmucum6~ zP84fyx8pW|r(}$>R;SIiC6Yca1$!~^%a9#fo{RCt;bU0I#D^>l%gZ;6*%-iQOol;r zC=Ro^w7VJx4oqrpHx7b}5J5C?XBGr>oVH3p0 z6ujHT|2}aPgy1d)9M>z0b!8)9trtJN65_knD*uuzBrf2qf8iAzK!xHubV%_{j*?3c zC2FHbq%F<+h>_!FKEOZlhZ}DPj~lr0NAk=Y>kC;`X+7!T8Gg`0^5|-b2$LDruy^>*ey+zz*eR5(j9fXlD8{?3MFHM z)6Zf$(!X4cyPtQ4sA^8zfGC^19bOaLf(TcF=qpPutx_saS(NlWumhCJ`a2-|-nJ%t zdW}7>#a?;w4qNgWTk^gvbwk~5yXi6_MIT~bn_SwycUJO7pxEijHI{uIb2Pp_rvriT_`*7N@g}oY%I^9rQJ$AiA+=(z#Y2b`4{w=>hc`=bMxm&9+P2ep^dUP@ z*aCyN3J<^e`(KObR~N=a6t*cXf+acLn*Ufl?O1dJIIVqjp4|Wrw=)!zi5-haz27Xs zuIZu!A4OlMA4UHds;%aDY;>^d#2Z1@@s)PuV#YoJhC`JG;T=RAilCq*dNY$&n1w=J zE-KN0*=bx`t&+OS(V>nQbdXWr6k&oEY!l?t(<)MW{3i=W<0b$uZuYTJZ|zUGgnnxO zWssY%5kd}i0X32vFOb5S;=x=KroO8pX&ruWBvMG92ojPMVirXBH|@dsCR9a|mb$oY z$Q*-y3DaAHh(nPDf!NJxiJnew#PC2CCbSI0xg7S9eXp&vo`g|${wU0dkc**_T;+5f zmRxkUxeg{ACQHcZ^qmU=6 zJa;2UD(lMP3vo`Zg=X{hRzVfIT()Qw^Fz8mw!*{;gjKDG0b%g*>F%fgd2tM_Lh=Ak zGhTAEg%Sv0Hq>xQuICb?Ks9B8JlO_zTK^XSzpAY+5-Z7i+>5pt^8!U0 zih1!BhEelOliA^N>q1F~l-X@*4bfT6ivGM0;=N!J<2h`{1M5zF1W=SWl#@Q$@HltTA< z|E5@imt_nu?>6UoF+}y6x&pm~ny!is?tFl6)-30V(HzoR ztsjN}ZpyWQ=9x|B4|z)YLoN72WCHkRjg_up5m2dfCE8z~aoK$+nNqm01tG9SJTS3D zxYLs8O37J5S9sngKjU6Z?yrrRO{x1VOmlp&%q%vbqA3Ev^wIOkHo?xn_dIGmoY(3~ z1R=dI6kqOpo`eV9!i0kNzRK=851b&q;65^k+^!WbK?}_w;knKi#OaG}2?^mTC;k@* z@tZ_JkHjbcSQtbOZzl5ou+eoWY%q(L-E$KoQE0(AUdyMgx<#%)6DMwp{)!9g4tY;? zJVH#Q>^>3~hWXrt`Pw13+2t&ba|}WE`)^^uDu!-LB8g+8(zjyMuT{FvwgL_t$E`Le zZQR~wQutBms%E@Ldi8`oBCJP%p3$RtO{)-vmCH!JyHJUehoc?9Is$FoorzISYvJKQ zJt8J5A%e;Z1&^am!OF=vcTOuwiG>u~Hc*e-YV>ckD3ZXemHs5fJ3P%ZB%a^m`3;1q zp=q8+TGEfLl?X4JxUM}i2wM}-nj(<8Ty&vH(53aX_U-#T<)1WT#B6=vfSBDUEI|^J zN#k?RpqcTZUx9yWvtVw#qcxsQW~8dPW?-zef44M%6Gequ7azf*c=4Fpn=N%*!iT)O zy^n~5xBEk-WT)#*Tm9@z2DQ7IaNli9f*{rrX<>nm<4(4a^cJ$b(-XZ9Cuv=%A!ias zRVum|FA%nCMjt?Cet2D1AqC+}+xH%Cq$&!PH9h1@uNP=q%mqFO6^IWjaO?{%P?W%d zoQm$#Kk1#(haCT#LxPrYOCoAc`~jI^C`fn%S_cFB`c<%yv5d$!o?rez^J{lNSG%FbC3dUI^YWibHX?K;zdxc8&RNMmk_av zn<3l(eLN3A00h)}pNw}Rj`25!law?=_f9fNIUII}zm$u!@^pM~9G$14mk8NU{jZyK zf~~_F&_UuwosuCu91jX8WtQEWYpE+F%RDL)I8s`5!2;HS8BT6Zy4@sci;F@hw}Ku3 znZa;#_lt0I*U$eS!p+G=PV_{xLK*+W7{k=#VFdq+G1C9cTR8y&RtF<1)N3U#1!4OE zIe6vaGs#TV28WVj#d6;w>t8@)K!-n&szk4{{tdi*Dj7L0q8^=>GsvPN?K*MZfij-| zl-TPR)@2$JALqm;a9(~~YM`w*g_zZ*lF%p)jTZ2hm?anMb~%||D5EsC$0pf(0C z1QD>jihx(fTPOzWA^&s^#JdQ+zjAmkbJa6i)O7Ca7Z#R==9UX-Nd7r&LRlahhDy-p zO(NLc#>S5EURDnbjlKE&;Ml*24UXO6<+1-O_P>r@^x5Ee^~8DtniNhVc)53d$SSyU ztr*FS#zwLUO=+3NEl4-RL`OByUdCdqnRc;Z?8E2O#&|r%TS8F4h7JJpSD!Wo5vOj4 zHpWMpvV`NKnDsqZ7aZ6**ml((kn#{j40p zV2E`80Q0oQt~9}}BsOP`54wylm3};oXn+ciH#C~X>0S>2cKPAWFe7BmF*1sY1B)cW zewDUKVJ7C(c6I)OalZ&yN`2ucqmkYCIDd&~EsUGnYs@k4~KqEahah zmf5@`Gwpl`Mu*ljjO!=r+*{hGbd~@c?D7#GVL=U3K)aC(-YIC9Le~m9ZZ^Pkmlx@5 zd@=(S`m(QT5&=~H+nY9GC?3gfD)UFd8l?f^^_!80U1y0{^aON!>u8CTKk#;q|vIghV0^MYDsM(OyHBu<-T^?%)gBjIl;j9P?oWu)oZZ6B11;G+ zP;(#jeA*k(^Fc?$TG~)1pycQe!9(7sR`P<}wI3ZA*ogerC(HzG5^8*uTB{0(!*dpg ze-ba@@~7-?1n8^`>I1Pe{@26wW54~%X5O!+v6}Ef)TRWDyLh3XmmoqeeT7a(95!Wa zk=^}KARA8hf*{5`Q)&z^bU|At?B-AZ5Hg)`c%iqLu75KUy7R#6VK$sc>3l}hRc)lO z#nh$L{RH}sY3l_4p?$(A^>Kq9sVjjsp1GvuP6j4@yms>w;{ISm+D(`y=q(?Fy~?e% z4SiXg;9|bz`SXqNEi;j@j1K2$`1J8~Hel`=0YjX=DD3p{T1IC2NpUX)tI_7Z7KGK? zVi;BnR$^PaFOIA}{Zk~6>ipZSC9#shz?femU~BaLfbOvNs?dtuNw7r{=m5M#%31Ai z^HU=rGm!@nj{_)uEi$0Dqeu7#T!8$>JLYg_JSga2^edgICbLiBuVlxjUK-pXffP*E z-kVKvJ6Sgxa6BxRk^->uP`u4c^k8)$P#?H{SF=i+O7DgiU@MQZt8?f~j|CkEM$R^o zl~O;wr|)&vJ~*GMoe9lL|6}t)eIC#*Pc(&y=XK+GlN>X#N~5kI1zm%kaf04e+cKHf z>ECiBglRcuVm;R!l&4>mf`Xesu!_2pj&T9E=v`PLg|^|5Caxoyu0!jaZLq0}Fr}^# zv(1h~yLVVxT_I^T98GO171u5<$Eq|6kdd3a4zCwus;kU&wruEhR7F%?KblD z(Y`pv+JqMc8Cm~NoYqC5GiAhY-~SS^{!;{}IHZtJ{W#kNmUg9hi4KoZS>H>hkaI|o zp`$~`f{-@?9kLre()`y)aa1K8x@wz;`z#(#6&ztBO*n`9kC+;=@?8rf zy&zvn{kt9&y@m&oPR}%jZ34Wg*#D-9?nC8O*JYtaZ}R3F7<9Lq+((zJfKI(A2r&Ya zt*$D{nF;@z1;Z{EZD0=>g;jmpsGZEE)rE+l2pq%;zQdA!ik}FG{V1>E8&c4#Sik@V z#92#bLZyoNfr(IqxV(gK=Ln%s0iGQ=)NOfi1b7gVpVi@y}S}{nMf4!1HjkVCN?5hP7ug0=26i z&%+(XA;gSo6Luc%j8eD>kN1R~hYK3$kd@+gQh&giifPxzcpmPSj&$hk>M%NVc7T4Y zCK$o$Dy(z117$bOqwKYuoru}RxMO#Bz%yxQ0X)iF9K;VUlzgG;; zYa;RvHB$cn^xvibgx_b$=jRgX{M?zLbbf9#g1+JW+}WYx6xv;PiHE<2@@ctGj?rnk z7C9olqyBKS{*bFbjMpCu^@q3hhcf+PxBhTmf3W6?hi>{qfBoSu{o!%_;RXHSb$ZBO za4+5q3V(Rx{KN3~IQ%^gf79Ww0R8|`(mK8kfA7QJXYf}Ff9v6I3;gYZKOpD<0G$AU zLg3Fv&L7fwxjdWton76dsOMSn13S0kWny+Z+2m5Zf?XT?7QMF=48!3?v68LS1oSc$ z{cUm+8*=9;U2AW?#KGzcUx}4)7vXj6ynV#|yb3b6d&c;eZH>5JFwd?Aa=g71)mf9{ zod!q|sI+V4A?&$aG{}Vg&qZzH>r8Faa=goG2)7)Si{2yb6q{m`%kFEbWbBgX(vfj= z2cR=t)de+zyxG0?rdR|Aa7bdR+#l7v-qMwfQv!Jw#J=-Q92a8C9FVfTPo&}T>Kd-# zROF7x*h=YJYctmg-cby2TW6J~yHr$9lzUfi6GXQc2xz&p2q8 z2OT7=+DEV^9kmm)Iqnai=+cUz!z8r zS0)i>@qMkLc*oQz-WECDfff$&txwG@-|637c~J}i@A0P7klvdW?{JG&`6BTR=CASm z7Ve`3b z62sS{;eFBGhe1|eL%<@tufV^qH4K4nD^Wh&H8%ARJoM>WgCRU5XLl`o7G2cFq3~S$ zf=lF^g-etTEv}@Zjv2@`?wuGVWDlGV%oyUkrJoXgoBV1;j_Jyj`(QGMYlX8|WhAa^ zrbfM!ElPA+j@myZTQ1$C_(s$am-S<=dRDVNN@8{!#hYCNJR{UmsgB76Gn$fqpce3@ zwbE{!!fe{Li*aUt^@yvqlfveFz!^Kuq)I>53-Cj}=h{t4M8S4(_5{v8huL@#LyJ#J zl!E7TM-EqL=_x7!pb+#MoeE$AyVEhzurm=1l<!`e24I%qFzx5rxY_zsdG3mI6QRmsW9X*61q6h*u^wd zuW%Ph(MGsqp2T{;mu6DbgBkZMYs|+*Z*Ag4NMM)Yq)RP-y*nx|!B?fv(q9<-4An05e4Rb_7n_ zFg)k_@>{eVg^WPryw7WSpCpt)2~!9}zv-lDbWc=^rbq&wiLIn5tCz4K;*K zsAr*(>D#m}w+GAH6z?i3QB4Jw}pXL7Zytr-`Jp5`D4OT{AcRJ z9qJ_Gem##6=^JrKdqN_uyMnC?8$Do(y2^NGpMrjpvy%8G`_cWq*GQgOw<* z#NfZKt?~J)r8XK6OE}{a>VjcE0JTWAt9yA*6l~Ff9Lw3&I=Un|o)ETg%obduci)5Y z`=*)j`M388w?C-(W^GY?&Y!fb4=7;>6tH+l{DkRR4X5SUeY2A=kW%n7&?uB~Z7^%O z*d}Jpr;}I}T6BuZ=4n!Fs%>VkZi-rYie!)+1SN0wPp%5HU2R%>EbM+~98DkSHex}a zb0tkH>XnM<0vPN3fsx>fmqKZu&$UX$d9{6Wyt5ABuy#&nGmxv&Tx_yud^fEN)V<44 z_jO6vK5A32?oH5H>R}%-(+Vjta4hl8*61Xy^2Ba+3|2Bs@*5Q!vB%U~?kY^*h%?&b z_0-iCFmH#n8UKdOfFT^bL({0~F`;IhtI{hP!~cZe_Sn@6v~I)#Y+)3vJ?SLcP7SE8 zEkStx(kbR1Xtn|<->hD&s7uDv&FY^`CM~ObJpC-8H(!aux49Ps^9idsJWu;8 zFN%D+)Q!k;qs4!|8w+8~I$A+zdjGc0p89syB<8`LFFtfV&B;&Ul z%2ZRCiAbKukcDMNRBp!UX_g$duRom*-IqP|fa1Dhl3$!eQ7-SwgQSt8a(quEL@BO~ z^-9J@MIO8_N1e1lQO7J^kCKHwn(pfs0aPD(p7mZBqc6QKeK*}?;m5o4-q?VEG;0B% zlP|O(LihHL6ii&MJuVW}P~r+LkL?OGM8e0~9ZQAR1Sfo|_2Pt)a@mL#oLGYAqP03s zza7#)7U^g~*A8;}4Uqn+NVj15Do&5XH0{kL9KI$48SVjz5uVR4AzRR9mQ|t^Lnf^f zX2NyFEEhFH%O^&MDwy?!Np^SRFtL^Z+k5WBW?dJP*CgW@Si^>Vl9jmvys=A|6V)SI;szoY+a2lz2^WyanJ7#maS%fnvjrcchrpg(Qg)(5) z(Uzi}DH4}m({XU;a>%CiJ}+^4I#*%4i?f)ltJ-$aUScRSy-sT_QtKr{l@dHsrOkJ7 z)~}pJQ&{>oFEKE&OE3YJ=>__sDk7?S!jh1Z*z3mpk3dM0oUO>@_>J>9Z#XHEKHPo^hdf|=vSTYc{X~4z%J6)o)ng{zv z@+wGSYL`OR#=eo;Wi4hELDq@}wKgL%%Eyj)n~|*Bryt3>v-^^)dk4I&UWgq}X@;_H z1b-;>34M^D18@M5SfQ=|lbvR|>cA?vPvlhl7R)M^oSlL6YylEf6J%A7YnRHyOB;FB zFp$T^ASuk7lmm zE&tF{h=Y(BG^!vT%%FL1LO<5I55v6ALE`jj(?r%MRu9gnHmV1Y6~80JZ=U#7#BZkf z?IV8E#BW#en=F2l#BaR#72#)_^3S{#datYJEeky^4m~aiJ$gcq1;NL`GsN#?{H>mc z0*{Fctqzs{Mm**heu%RvRVankIAQA!A=rK0gjB``=}7SN|Ci3kdoh1AA1K|Ra%cSl zgqpXWrnvZvbvLAH*Mgu&(i(*d4Ekrpb^ zHB=&*+fNQ_f91pIEBp0DF0BXbM-3F2$ovnpJH_EiXjuFD3>P~SS?pDrHBPI!n8wAl z$YK*kv42jA3u-K@UEz}Y-tk0{YdEhaEN{*D5Vfv$g>T=boVSegmWAa_or(8+%^a(B zy92lXM!Yc=m2v)dcNkUVxL#xYgHY@bmrwKTYW;e=tVH|hsW`@UPZ43;s8nYPaL*ho zeg(K^3UJR9;GUT&eg(K^3UJR9;GUT*eg(K^3UCjLJ`G4K1nvdPLXRPE4>=-N7lgpQ zpdk2|DZo9G!Mz{^?gb%mF9?DAG6C*tbr{5d8O6xB`WGZy?utx#Q58wRuUm;EA5yBb z4`Iz5?;3&!mEZwA21lV1nG%6CAcw^tR z_orATKhm|}C!t?AMfR&cwn4wx--^H)KYKHNt~!=KdwN2{ezn7l<)Xk&C{SzBv+)zi z6K)xeB3js2_y!BwTiD1$Sb5=ScEwj4`3kwvO%?^lLjmAdX)8Hyg(up3j_&Y8o7Hgu zc|eB0yC_XwGb?w0piAv7x$bUV1g`IJ(f#yQ?jmGp#fUlqrDS)(cywlTg z>k>syyTXWI-;;C|904!svloQ^Yhcf?sXG<*dpjB|0$Tm&>?v?_lx~gjuoWmnl8XkQ zqgRzVMbcHRsLp)3cceNK`KH>;Ebj^g=WO>$+0z*vU4qb^?XH_)v5%|D$B`?3%h>3YxBP+Otzn*?q4R(Zc4|)D<+N zqZCCS;tE13ySEri$z|*C5|i3yHq&OF3N~hW2aoa=yg-b7AoXI#mZ{CvO8f{0v$19D zw8?{ZW~nC>)BbgBMK|zTGt{Q8ZuA5Brs{l^p0d=lsg)p*P{;P8L98y?rKQ$Yu)|f5X>K{h?tu1wvBNv#&nmUuoIZWM{WvUg;?0{x= z0N8I)e2GcHrtlb2vFY?bEwu7sFO~A`P`DD5#j7>9)eFQl=zLx8~biFQiNW`29Db0P9aT zMKbco5n?A~XMCE1eVH@3&V|2G&gS-v*HREwJ8#L(Set^F+Sv_v=!F5i)Z@uI89K3b zMO+Vx7B1N3KI>$|x2DmzFjl_y#w3FlKa!tSM^xZ9e#LX1RHmrHv5$R|J#0AK>S(8U zn+dx~YNeuXfo%przQq||J1l)`=-pw%sLiM(ga+phi;_@!P@=PIroc?))6ue>l``HQos}{c9$}djQ?<4f7TZ|2y1>AFI9t~y zVc?97#Mzd)( zD(p~(_U2Y-vDxg!J)Hb~K~95KwKr}HU-zrD?&=EKxKK+`JqR{gDn*Hf3xhzMWv;xq`GQeCb zKyuv+vsFRz-(l22=Bj6g=V@*FM~MrZ9dZPG>-gLGQ2N&S*?ar9bv)U>jqFCW97OF! zn+0~7J<)I+ao!iafv@cm8h)vyM^@49oPesxCXcN|6+x|v)tBz0`CC*;A9B8d*so}- zT=XxAjbDmaWfZ+I4nD9>>LXEXpNuop92jh@d+0l3%;a3I>x~z8kJN?^!!E3UlDg2G zy5NtdJ>aZBxTp9z#{&Vil1#AB3HMR7Jy$i!08`5SwpD&noJkgn|xMuA$IR_yP zD#$Q72c#c>NeWq?aV7?T2J<)+`jgi!avq<>&P^ZB^H__fr5=VdPf(fZSY|kvxrsDF z#4L4fz%1$hLTrOSMkq<2e@{(X-vZ1YGT^sR-ScYvk zf5OTp@6|Mn=;3MWTEm-ujzpV(5l@@Xo}>5>C0nWsb?K&H6;_CHCMw>U$=bl=SZNiv zFf^^UsZv;pFTxp;JP=ybRO`(S{Dg6+T1$p;;bL-!TXEBdkCA-=oC&C>svw=Ml;VW0Iwy8T{e$LJ4xPDdE?<{K5@U>3ev1sEQX#^pw9 zpjQ0{@->TuoiL9E{z6OvvP4~Bc=1?ECH=nQWi|gIsv(lNqN^N-E1|QpQ&LQOAsrL6 z30M>x@#PjMDllZ+5mm7;1ikj4P>QrBr^P;sZw645ax8>*OaCVdfX`O#7QmgYmLPCv zdpo2?85|R|Q zQ5-k?uUN9vD{{OajG_rUp5^r+XlW;-Id;LMH1}N{P&U z80ql$bOBj3&&qMFMKt!IoK}3;1E@obN!CKJQ?v<7`6+;9FfN~0} z$CCW*)XQiuool5JOeA?;$EC7)2e)65>FgkvzD&c-*e1`r6`Jb0%R>n(lAPDncHmK( zvGA7`J;TwIlTF51nUXdCMNF)s*|Vfvjx~EIAq$bw9<6@7N$*F+A+tH&LYZHxvUI#5TJp2E$Jb0Tx~9wW!EFOL`DlQ zcXTkfRjcWF@kcelXI64fiokP!x12PS)u`i-5Tym8x>#KK>u66w!y*Zu%Sg z@Y_Q=eF!?Mg!mCO);OXdMWrz}@Sh~#Nx5tuX68eWQ}Hm6Z&sZa7)=v1KrYLz%U>BC zU3du@{pp?MG8dMuZKt^UNzMTUMBSuSCdgcgQB>rtN({D*l#1r^4qlfl$uyTskLz>! z$w^waG|v8sv+->;Dpa+XARGHvH;*=g)0|!bM6Yz-Bpw8fkH(g$Q)%s_Ifngq5ZA{Q zn>HHt&wGV`6jr%j@y!Gxt4&C62rOfAL;u|W5Dbp7N=CnE1HUmmPaom6=?AEH$uyWV zh^o|TZRKBPsaBRtmomz{LvkKUuTV_;!~{3sr-#p9ip|w+Om_CSsk_s6)4{O1jq%Pt zcH{t>WO+tfJWn_C+|`^vY;ktVQMY*VTX+V=uI88&XK3Ye0rQ&_@7?$ril@LCjlsW= zU&kxHF~A7K>^Kr)H*iGhx$0&cu0TD;ZAKzA|3mz=t!|S=&0XwFfd`Z9M&(@k0sk7p zztO-l0};Fn{+jsz9gWWVP%2q=w%emacn>KW~=Z__^#N;7&48zV?-lWy}Dg^yM6N#A5VG z$yI;WmjB)gc&pkR@1zAe-Vuv))l1arqE={VMrFTEE`5hEt#7tP>m9|bcloke!o{L_ zf$^kPPQ|T!>N+E!OeR2Sjt!yTO7{HaVset@qLzXV=Xl}&QI@c-7i9-1sv#kG3?q_S z<(SP~5B6J8ogQt_iB`aO<$4VL$Id$*Kt9R)Ah!BBwUJsYyO14(E)H#^i(N?mdjS2f zVzfgB3X+64uioDZ76w?g{HV0Gt!WVn97Vq(L+~akS}$^Z zQ`rCKl*$t^N?-%t{c55JItV)KQJlI*@j221>AR6Xx5@EEZ4jQ|;iWc+C*}LHS}&9D z%V}LKr{Ys8JW;6vz0?9xssc;ZV5tf$1siEYrsArKa(;|D#vA^3O#_~CbPR1+fR&e; zV5PL2m}N?!!k$s>thKwUEIHnO2nQJ9j67Acm(eZKuw8$&4Y%Dh+;*R~0PqkxqoU^% zUPpj%$zEHzYX20wIy%MjHDHUFnu7*jTw@?g^^H{j;>L>nsF5pC!OZ>zf_2u?muq$82D-{tI-_PYAy?&GbbM)lD%veevyEH&F> zSI;0a^2T&J7{pXM8ji_MKU#}hVmV=4bvqX83b#0rj|F!1KZ?3t zQ7b{{(w?a|1A@2+QwCCffAsopXWT*T+=!l@+O7A|4qEK%*m*k7Gux2I=w?~#^!03T+^o9j31^a*fYMJmJ5AcpqRE` z5gU5_USs@xIemUOe!g@heg4MK=beSLNHYwJl;eH84-NJ{*hulV`Rj0UGV7}GU@YxO z3NHc)@v_@Z7{duD*n}9^o9*#r0}d`=91LYFe9fh2)^^XB9TbJ6_FnWxOUS`9Z2Msj zwOm|M^+yDu5qOEKTy#H)NZ;kC$Fwxe^u{=UGN7Cx2q#sfX3I`O>CBsH0r)1)8$M|U zX^&u3b%My1?Qy0Q?H2CcjN&srvz8*R1jY?luNtPmwzG~*mDCN62lWyPD&DA2{ZYcB z@tN~yErq2ft?G!b@UG$A~bUEQG|$}6^H$)y*ekjZ(!UHzG8Mfz5( zauJ3ufzt2=^vYWXohZeow9hu0$#zC-H=jBt76LyD9nc$ge3(go#J!^sk8YlpI@D)w zig19mW^UVta8QV}#(|z$>d$y^g$Di<5k5x@?EPwFjK2A;6@5s2GuQMN&8v4zCg9aR za#0HtVHB!o&)6ipcbiyIjn}E#U4M0)-UAT#PbP9a^Tw3zH>}=Eh*vG}&q@lxHIRZ% z98&%k#X6pdK(GJ*pjP0~2n!Y6N@M%qe1b-PfkupY6t}j0c8Eca=_Xf^x!rAw6_Klc52eX@U_VG59 z-G{T+w9yr#+Tm6Jkr5oMV>&bCA2}J?v^i+_PBg;&a|H7% z&_0l(RA<+)HbAdFwGyN3bA=8@T>U;)Kr?z~Vnal}6vzM6-PWLX+u-wt_z3!L`U62B z2Nfuphy${BWPvQT#qNu3PbQfo6Ql{!@PM;xp6#B1GhVyv64}!;LFkfmqMbEpW5`j{ zK(=^0{Xp#-c)llG-ud~0S(kUxD>gRpA1M4C?I(8qq%fX&^v$XPY^FVSEygaLRU--i zQuKd?(IwOQG(P-LL_R{kS;e|P?SdJcj>4JZB3Dxr-xe-Pv8GFunom5un{+57CQYA7 zawX{=wfFBL(Q>?8#wEPDHTkN|EE>Yg3F!QRwssnMPIrNN7YWll|E(>4mSZ!CG))=@Az;$ZCTq_Tyn7I#*}2yp1+V0UmV3Rr@K!B-O?T^Mjy+ zm+j)*8dhB{oAqyP?KmPTdwO?KG%Z+k zty7%g_LXCzHkym3PeJx#eXGb3^@0)kQ;Bu5zH7)m$;>i>ptVrLDf##NJv9uc;GNe~%ODECdT|eakSe~esk&B1K)BhCPkdG!r z=AwI#|1Qq0YLGh^bJHky4EZN?o)lV(8$}Nj?7kOL9@kf5@6Wgr&@BsO+dNq*kJ0IG zF#tT~vn+k9)=z9_IvIo}Pj|(PIg0998JZ;YX@~JP!SVji@H*4j6vz~)tl?+_uAtu^c|Uf^LQSz@cAD|*TqjUvnn}!#4o!$uRhNU4YiJ=tkkK3`;+%lR z94@mW$G7Z^KF2DZIf4&v!5h4_4+ml(=&TeG3EG^(9XsIYjjMY<=BnltG9Mf!GMWnu z{3iR%RqL93Ae_G)pcOA(XT=VrPvH(UHHB{?%FKE;h@yA{!I%jlb?1x}TC$xJ7;!mU z>pOGp1QDrf8n0q|IdZNi;7YuBnO34cE;9EduEcoEt175sL zs!QDOE4%N<=m_@{iLidypxMu{*#!Qm8W(9NSc19dFEL)U6}+EE`gg!kTq`^DaS=}FvsMJX6xdL7!U z8O_Zq#z_9l(Bw9u3A@jYg8r?u7)!*95{fAC4>~_Voi13NzySJ_BXriqHLE}ZRsrF7 zzL#$JI$Uq5^w8t~!`_=fM^$A1`(Mfio&FKqv1)T5ml zaq}OP<2=ZNI4>u55Qp>pKgiRjm2sQP>db16s_CrRs;f47%oWE-kjRLBa##mhxAC9%?FTr($Kin!(88I##LpRF)8!-DB&}|&P6km788{TiO<08WSGB(-6@@4WnNph%utPlkJH&A!MMi~>?ZKe8>Iubl z((204bl}y*=$It3oYT&(^!VM`60+LkWI3J!<*g#fT|-;4XXgPk&YheEHL%@T$oeJ5 zza$-RVLGmfog7PXaDEC+d zMd*51NTI^V13wSSqTbk|Q@Q9mNlX!J?F1CN?M_h)n5EkMJJGlyXP zDU-_&8;z9L?sA=t`_%1&vHGO(KzjJ@`nfubTzK4zX^Ugr`5&?B7z=NaBN3FMwmo{zE0G#2^)sLqlzurT*cV9Dwd;)wH(JGb5I-@hfOuKK5?u?@iL$PkVxwI3M&0Ei@hl|zTzMr0 z4W+d~cdA{d?U%NetG?xGktbE=u{kDOm6yRu4w>|QN<^!?$J%nKNNHt@>^Jk|hQHpJ zex(eO5?O>dS5L?yPeS36iZqx-ie-@+uInAv%QiYrV;1k42*qz+SKZf{iO@SVhU6g_ z<@!(?yv-UY+nAdooOG();Oez>KAxR;ruJ~}13{|yWg_qODbXk;`Pw1PYf}V9Cfcpb zucAE;fW2spNJ~eW_nkbxAa(T*Re-&Zu~b%YqQp+U+10UE$c>6D&2p;C9xBcIoN2Q3 zVfIq~M4pgWzc$|eFk5s$IH-L^lElZ!aAWXJ9(D|okYJx-oGI(toGg&;vPeiaOUi&N zbdZD!+7bS_8eErzD&hUcPF3W6FBMMi9VK2rmK2_H%)D?U7aBopEdqEO({ioYtRz|Qtx+PT3dD7zWrXx*8aa1|%K}rc zVGbK=fP}xfxnkC8EygREcx=JcT$(9R!k30?jeF%Rz6>)n*c7Vw`#x5fG*5laK2Jcn zINe(N3%+gE+EGanut5kj`Y~*-Y}8B);~s(N+B3xV@?9mK-%m6TZ5Jj;g#MGU;u3ji zJ2~3v#hnQXd{6%aA5h{IO6;n-14(aC(!~kNL4_>v2&@4TB*HD7-z|>m4hOU2(vz{5 z86(m^PmrF0^Z_D$WrB1l5&CqcdY)mMEalv z>4T8|h)6#nK|1f%+?+1QnkD>UC6bv-MgaH4p_9;4W#UkaV1g|B`-CFjR=Iq_jaZ1R zY&54}ZGkzJ9yzBVo*N$*iy~HwoKxvMr|xlz*Wila$M?kWV~FX-W3ueeiU8#vlW`4| zr9bC{8eSdkm(=8~i>`#l^($m0KgmkgnSA(=>D%QNQ&FZ z&K<=mC1>gi{~J{e#uc^6| zMG-2leFdbwq~HvHRz1UzG62sP#sLs*zu^7BCPdvf+JGAd=K3n!TvELgRDFzL`ocSl zHYsgJr}MMGjn6-l9b>F^3cMk!7m;XxF3eziLx;9I$!=|kZ+CSI?(|Cy_}Fq-I&n?! zoR_8O;|zw)7US$O(jO(^M%6x6wY{o#$mw%$LUJ5l>wt-*z2j@ zDnoIp^lF=teYMz>3?;Z{9L|p0lneyq_YZ)cM3q6iFg66n#^V%YLmvxe59H7%gSM^V zRP)=!EI}XAuT*It{rviD82*Y#eo}Q^>hLA>KQdaaK8z!Ssjvk6_E2`b!`=c=nQl?i7Gbfw z23lS;e%2MEG6=>lu!}6*H#N<&w?ZrH!e}yfHDxQDR=9VnwZbwZ{VYzbIBQ0R$k5c3 zt;G0;sPNA2qPQ&4cpkFuSa^=UA02J_`MW8Hqb9Ijx@#9!2W3$;-uC7 zZaMX=+*nB%Z;x;ik*vsJ5JON~02QVHd+>Kso@z4>fgYKy3$m@FY;p9vna?DCY3`(eiqQ%0 z;Uw3o))gC^h5M}c!tk`B#4>Gc$ywanijuRY-9S4<*Z2zd1I~UtV2682skYuoC16W6 zL!3KB4UF9zOqp@39p1Z?673+3(rhG7NU7$)*I=;E9%VbU^V!@y&`;zIoMwL)LsnYT z;L4#*UMu_lH#%(y1_SA(1wRIo?b>>n_QtZ3AzgpPK6$_%wwuW;hdxRhiq2S*tr#om zU&Va*OMjF;^I+~kpXcclwSLadRa_GsT8B~b1Z@6i<(K;=!L)ePZ8Qs}^0uY|sy^^v zYhCf7L!Xrk9yr;NJY&4E_PefZMBd0lHaO4NO^xCfj`3pN!ui|83@H00cBa6Am{ERy zro!u{zg{%vx(d#^Cx`ew+O)3N>qx#Zk03D0sZkhf&YGo8Xktj!!ba=89MFl&S+)`P zrEmLLH~qEZtk3AjsN38t8NRYa@Np*tL+x4%B6$fSOfq}aGf{~W?MNUiIjm*O`b_9c=m*zOY$nshbiX6WbNXETx=Y(!-t*3p~pibPUZDeuDZPiCu%)%xKoxnPwzv|6B`Nr`Asl z*fmrQA+|3y-sYJT9pELh||kaF!A{h5B^O;n>~D5bPM) zuS>x$rw(gz=&mA0Em=xQEH8=?c`e@Gwu4$&_WXjCV*`?DNW|)D=&0=yO?< zMe}MrOmnNnuWJ5Pc9%mRb+@XGq)+i(4(;T?NNO{pGWW8;a`-ej6pMHEozUs5Oa%a% zQNU^M*Cf3W>8E5V!Ynw_3s2Pi;HhWqBv@6=G#ZNCZ?G)PGjyn`_cwrZKF?G*v@mSQ zqvxb+aAug%^krJ3_h%Kf8@FXx+3w{Z2N_%G4!xHIn-cG!MB1JN{%#a^jVICB%}k<; zGeRWlx~yCu8tQcmu7Mu!W)(3A8r&C#VrQN%8;j&B*;M1X%_8#GN5+*@X#&sSasIPu zFKHa$+Df;ndU+e16M=rEr|){>(llWL&8IBorJ207x|`&7y)wo4alO!g=CF9xvs0k5 zWJy7zHtkkdestI-w>v79IgS3eeyzLt5k_t zS}Ii{*0WAl)dpivy3ly2W#7blX}xepQ@#MVTks9+6bGJsUXj9N)*WYeyNBaKxS1HB zyKmsUctW5-{Mk=p@z;5opms3IbgG31(38RCf)A*%+RANa0IN6sWaZY zAeA=yze*Jj-!F2-Y!W+HbOQk^duB=9jiucDIL3BEiM-$6P0MlMdZ>FeD{vupwLhLf z$AW;*BY^#>j2`424HF-K{K+2l1B@i({Y)%6kY3^NM63~lB!;hLlDIvcHDNreKG3Q4 zrNm_xYt6H$pz&g=U;$SEWvzN)Kg!k}Ul|w5E0$jJXziC`Ie4iSId~VO34%^7*M1AQ z{$TV@PY#dXS$2%xdZHm3y$ceK-T;F}@08OpdOw1vWnkKMyBLH#uAYWdobz(XIQJ6c z;C^9Qd_ly3)MN4s8t_~LU-^E-xNPo3 z%-CjDhdk~tq-kvAm5gaWaJ4)O?NAY081|s$aGTWzhk;N_D@c~7g zOP#SF^3r-d_5*SFs+O?7-?hfk^Y|qDR>0Ak$Is)`U=y0XIYnH=J;BFE%P29>1tBsI z7%c@N`au(qh#Gyyr+piVQgL9!W+*r3W{R7#NeObWE1O}+@p61T5OaLnRXt7BN5zdV zYg-sgM}%}r+Kj7;#PMyWBpWe=7=Njn(QZ_akf-BcK>$E(c39#Vm-3q6+Q@#7k`!?} zGDg(>CciXwJBq~1r=woV`^b4wFMG)g)VqT4tQ~qCER}flkLBafD1WFj{t0DU&_7cI zX0z6Mlu+FEXX&lCA<26%dB#jaDlTzD=Pghl-zAP(y9z9O7|Z)`^*IRY)A#&%{<>qKIU^uad_P>OS5OTrJDK9|cg9EZJ+4q> zeBPkp6djy zdiB0BC&5EXQIdDBwW9L|YfY_W3HvXR<%dxxcrkgPEq*b{!+1|Rk;l7w9Y!;PTQ0K% z3~W3Tl!Kv8pc&Wa2@(e(@kJI*V3&n$m$A$q;(iN_s|9Q>Ky~Qr_zK|v+CMQU)WMAf zEZevS;0H~$&&n`j=$Y{Fler?QhWSj#SD}W}MEYY1T7aSkd(cbduRGd*{`-70)uC!| zwKrn2Q(ugGIb-;5ShHG~a+ZkI5@A@3>xPJNe2>SWY6DhG?#9i7M5o6(k(1DX`y@$J zr^`aeTSD~kkzb0g{p4Yfl$5SZmP>Y5ix}w6qF(o#!lPf&WT=Ldhypt7GUTS2U&YEzlhC;_$A=C|kp29?(B@=zrW=u+wRGxXDr}q)l^v!< z1E#HjI157l=Wpf?KhjU5sPE{P;rfJJJqPt|&51fM00^%HR*51TBZ|!AA~4kmbkhnc=t=!o#^<{LjFTM| zrFCAgNi7_=fkbZiYmlY_X$1{|OaOeti9!Uxws&llXq!0RL=NwIe`S$#m~nEB@UNM| z9M5pfK)DjD6F3)G9 zrUCoeq9$`N{aU#Ur7n9uJ5K*2pZ5@9QQp47!g2|g{c`;bVvesIl3e3)DbQc?X870! z7kPp^Xa|C$r@*!oLz^r<3s<+YrNmSC!hFdvGS0$Ls(i8CZhfQ$n1C-5!bE;#gu#8E9b>Dru?MP%soMQ~ z;B`}iKW;%TsC!=?H8(S@_Q3n>uP8da^T0h@j>cE=CwZI=Lk-455EDEiB` zsG{91rj}e_j`3N0mheM_<3zaRF4!JsIpz9=U*Pb$3ns&wU0H(ARNf%B8|VEdcG`Wg zWhy)}^+Uj0fEn@0nM9Lk!_)DZ#Pv`Ehhus28;t*e83S&OrC!x z&ex`i(^Iw*EKWD0QW8;aX8pJUw=Ly!jfW0oDgIS4;S|mBTLagve%h)m-xjO(l4>;W%u*utauU=}vcBjxv~ik^ zKC+EMc6?O3(M4UoJ2%vlhgvGJF5er|lsh$$4hS!&_5{V)qW9;jK370IMOC%YqvOsa zO?uAYi5bcjDsMbSvL`ZhR&@^bfMqDE{*aS;GuM(NGgqQ`Ehhf0iV~}z61gAB;e~A6 zHn5ee3i$NRxZz*SD@$la+6LbFSANg?m%HeZ4eWr*Prk~+UGaE_HiEAwIvW@4z?%`{ zL&{|k27mZZBooNwX)pi>o$GAj@W8S9W-;^MflX}G{D(i=R%V?~qobkm9rl&D5Fx^q z7a~u#0jT>q7cF^rPlmme96fH2H#&$08LpUp!{3z~s@4A`cXcv#Rn}StI>Lp3uI9#6 zn^1nZ`bzUMqbxl*Uu>~fEnsnPnJJ3kzQOt$I>S*+F8n1+iBmU=@#iaU-U|4fqL|l# z;ONN$uFBC)ZOUk~-W9Kg>iyzpS#Ko1eW|}L$x)z7swC8J3+Z>1YU>4)1@g^kJyI_N zM;D5{`XKw>et0cCv6S#|1L4x!VQ~!uZ|tu-dV3EQx$d%i_E}1DHDnc)*>5g7EdAEYS$uRu84{q6Ln+j|#m4{q=t>|>q#ogxmIm54J4yLH9-A#nNU zo@KmnGwV4zTGDX);zv(cWuVfNF3nK<5KLuTt}^3uYA0}axOACuaD0Xmu!Y|YHhP8X z10@Ig8)N82;4GRb7aD^$Q~5%A%r~~+@jN33i5bQ-s3z6eG+vg4b4iY(+|QWvUJ8=> zyr05JS|4icgJqWLLqOTouHZJsuOcliE^Il`6i!dCqR$W5?#S=|w?6jr<`%HCd^Oe9C^f?=W#F6_lTXX@Tg{-^{W6p zUmR(|(Z9uf|ygv(@lF_W1 zO4#cM8FgMpPR&MlZgWXaPzf9#9&)5vmpgMTbs0Hqr7_fS{6z2uH{fV^hXdEav{Zk) zH%m#NksBYsps%!_d)o$S^8mZXd({SowojOcNo&TIQjR=DSbZ-kOIZV*H{-!1fv1_Y z5rd&|&(@c7XcdX(kKTK_^+l*! zV_&uUPy=|&8gZ|;#5LT}QYy57FpEYMZ1nFGOfLU(h>|}hpaV~5ri0@NPX89G67mKE zk!QOfdoYKuJlCO>+f<(#RK16itoK6sqr_L#VfVIK`0y!XwYd9jaH|36e~x$f#wMv} zHSz0;`Ks@lg{pSVHu`R(?^gP5pznjKHk%}&mLk=6RsSS-s#3j&EpC!W8(cYR<9DeJ z-&F(J9NJX_a`2RkC!5;XM(;~fse(;v<4(dNZOo|j$n?VVqiR_2xs4c_A$ZD+>EyL3-Tx}LR55$pUAen&Vb0a98)Y86j zB5i|@NS83!-Tjq?fql$MHbv5NRDZm)a+E*p&stYSs6RJwe~O)jJ6!|+On+#Y@@!7+ zFQE=Ov`@?qt(6_Ro;$=Fc{goJCT`%tSfj{Oj^#WmZROx8jex(Ks@rqCfA66X{C{Wo zr}n1CdwDwm{}s8lA4S6NEZpb18^VUv9(QFm3zN z8TI;IIp_?*f3?k_jmmLqciYq!yO?s?mWmd8x~TS4K0U>P88|Km{8~FUO*o9G;&;aL zEh*fchu7jb>EXuo58+K8u|V>9q}q^EjAqbU`vFV;jSlUr5*;Ipq;iool`XSVIfO?| zUUGcd#^hs_tyL2{LhN zWt1?5#(*-kjF4o%x1wbPQwi0+i6fGHWUU>AL%4Ui&xhbjk0{%uXYVYA*Q94SWF+x{ zDB3D0`nWJX>~xI}(eu?&^YbtPdg%XbexlmWOwJX2LmnQGn*JRe#Yhf_wdU(2iG14h zZ=L_=@SPKZ?`FWyxCZT+fS-^&DFnZynlZ_eYMjXtR1@Zp ziQzB$J$gB4iJ%wU%l)s?%i|VFFEB9l-)H%A^iuSH3%v|G8ci>amOn!;>Bq!(v%Fw4 z@$HJh_x}RfSZhu*@qGjE-Ba+cD5mu;Tw!K%!iK26^1E88P7vy3^m>)Jze&H>-?x>B z{vyKM|K>EgDMVX=tD-yVzC5OXwK&}KuSJO4Ka_Ed(S<*Vaj$L5k? z?AQq1Q63*3ckCdzgw~vBd`86fk9IA+(7kS=dr|V%X7sPwlviZ`wwnDrDEikT`&Sg( zzp2!}t0VfiKRiA{Ub$BGFPa`N<=Z7u!%*ZWkgN{J&I~i4XwxM zQ55vG!+Dk+3TuxH@HA#CE7KHl{ zIa67Kei$$OoGFki3<6u={0P?$gvY_r@}_4dA*_$8_s1>f)4Sy56fVcC%Zob~*QH$S z2sW$w^)}qVe8ihpLp5tZz$j~q34lxjcfG^gk@Ge{aaf0MhF*;-96pcx33n^FE3c>B zExSb#rzyLrJEb+yl3!x2fj7}|{S1e8GIZnn!747@NcuUGt%}0_2~qBqud-PAV%L78 zo(#J`obR&6h#M!?AFqi#b-`xSj(8V}(fx3S(D$H{fr+IB9Lo2}{ZKI6;S_5v?pZ-ORKT84OBup`rDd|JM&hq7B{>8AK`}XaI$-xsQp!D3EN$R zs0nv6RqfE$rS zR#tec-celVR;*4*_8v-i9p^oq?&@B!(K|cOVtu61ccG(tGY2XkHbS6t$L$6Xib zOh2qORqWt&Ouj?=P}SXi9ohk>h+^4FV3ccnb?PWPko$=U)O~{g#^0byR-6OUHOaQd zK`*_BE!Nt9(gRai?Z^&}O7Y*&Z@Oh(d2Zdv&fpIDvFmx^&qNQl{n}&-=q<;s9oV2@OUIRju>Vu(s?bc$QkU@+aW)ynL~`in)%GqC~F`?WKz?u0ezE z5nF`847C6GhLVE>maH&mCWT8rOIZoiO!p{-cy3BrkF%ZLMeR*Pg?D ztaBHoDhl)gX?6LC7R`BSxbW%msA}0C+{qCq+e@FYxbysvQyp+{EPnU!&%^Ij`P*7E zvlEVDOIx)qg9o_=dx%8b7Y)8-#zofU4JtMtip8d+tJdclr=8~1?`U-d*W2fn@;M|7 zU=T#mzbBRNBdGkB-e9V=7CIep7T+zp|BfKLl^kweo|4PGroQa_+Ou|z=kN(rccS-S zrhGj3f*GCBwDq~Bshb_%qv_V#o&+HrbqObXz5%~9WGlv*bF$gFP<|)2NLZ-qIdWC* zZ??(nac`Pg1?@C?D7y|XgK&kr-$Zrfap|6}*vXrWdA`#`3GbJUj!UDmpvsawto zM;wj2lXyJ~$fdeHRgoGt&_7ymZ#LU%Y2%hSp!Cn60FFM~zD+bZo*Gma*n=r^%{F&9 zEfZTyiPhI3fT&}_S0og9qV9L>-oZ~R*8ADPB30`|)QnmVGott9gT+eZjDDeUgn@-`SN2rvX###PY&D)|!8C{@qxeKlj&~^TqC&O#ol&yrgP_ zYgl>V0e4EdK7%G9$*L^z>7#67R?>IGv?MvKlBQ)@jzik$LKD#5Tt-Vv;U3r7)CjG6 z9&7DiApJ#zx0w%c16dLBr;a#}2xen;lwN*hqD zfF9uMM`+lA1roI}mMU0Q0 ztd%Z_U}n}kOFb;{a2x>>wnraf*bwi41g?m-YUA4KFWSnbwoaP*;S!50Sub4|(|~)B zR=N&D=qmEBrHS#g_0Hn-3hR`H!V2s5Qjc{-agJWz0PFm7DheCJghD zK*Sg1@E}3=!qK+{gRGftto@Z^LF7kr#;n7{8Q~Xd@b%W(XAXM(~9E42{45$%2!m{tG}uuEVn72-bD+F8^o z|AHgn7I(Bp0_LWM0>eTxj91*0=AzC@G=TQy7O|Y6j2*yX)uOllm&G@hGog$K!^CA# zN?mwsum3x-(Eto09#52dHB!e>>OIk^i#at5skkn^F*>!4Q}@zP22KJ0f+pJ~iK;&r z8H^9>!P7(0Q>x9n>P21OzQ%RULR}A`uEmv+3OEE$ zSx(~m`bXC{OVsy>sP8<~_blog8&_XmXnu6X{ODf4FbUN14F8`R(cNNE<#18u_AON9 z>!>odGFyrACs5GPUe-I;-I)+KJ2#1EW>k7bJ#7YO_rPv&c22sh8}ea0*x*Yk%EWGv zI7Z}rKwfuRJ#?`!?NNxfv#H6+vFdTAz9o)cR6Pw$H+2z#8`D12{-WaFQvSiTk;TMH zn{hi8`rCczW6$l8vx_hfBo#xmY5;UXZl;Xaa_#i+#pONd0id}j)6bRso_?mdE9s}l z%yyFbaJGCH(vQ(O;D=WjtF@a6tA~Nr*Qaw_tqhU;fk>`E@{dTKFOq+Jm6Kl+$*6rL zb_&K_B6+h&en2FDO21a_NAeZZq5JM3T0c+fklif5&CeAT50PVyfbEeAH_{s=cYapibMtGaT_dc-n`WNLRvcQ?kH(ZUo6>&{)@8h{xUaug5@_;{LJJ8K| zbY1u|M>>RPHSERGkdUK^37!2R?`J8np+~qSqW83fWDBd$==UZ%s3g zTrP0d)WHtmQRCH=3#4S*C|VdHn*>CIMXT<|jQyWv2tPlFq7ul5Q{-3U6^uW`D4qFr zh7y3I9AgNzg8aCP3K~y8BnPFrmrY^NzBBr5$_g2B{NDNURae@zPlQggam^b7i%@85r*=l_jShXpWL2L(nWt18{Qrw~+xus6yPZY5?mDVw_sfyh z*y~R)y%}tdN?7SuOb_VBmM`G}0(h4-Z-9vVB^AqCh%%uWfcRm=x*Jz2a z8@rGj|Z6B^!sqxODakrOhc_XqGdwr=z`h0pks&;`gL@+m=_(I%{ z=26!p+&;$pCyvC|f!77ThRqM*t3u-IoF$Ae7h|dhF&2S2fwNDaf10+i`45J`lFNC$oT&7Xxx?`Js0nzDz0A~Jr^G}=i>0i3FhL1KvMDN z=3G3~P0Yo{`=jULvGMiTs{&tN`{L%~`>)2&#|ZxQpXm8`x1^*_q50VRb()Wl3AEMz zJD!qWM9)-KowUW>OJ`;4cz!yCi z6WYs1;%m|?0$;D+7sA(HB)*<_h4CdQi|>KOd|b@)F}@>5`zD@?ztox-yzPhZTr7Se z9)lm6bMcH9;xV{2VlH03D}FBSe?AIlyG@*Z{(L;nb^>Q#d@68u%YA>qzry~%e>2KH z$WV+6UKU9EyB-!bUJi>I6J8eXfR{6(9ubK0F`~8$_-{T>3T#VRs%T~@q1ov28?FV{ z8K-W7Cx8-b={Kdgd1p-sk~{j*IvH_45>qu-Y@8Dz9kQUw6?d~(Af0@GIrV&&O`Ll- zE=n-3sowX+g(XftPPh7cGJ7?)&;*_*6m_Qpzw#uL=b(@8uG z4*8p+barnt0`HM*(;R(1qpd(6L;DR?pBw#KTQMajJ^J^b**`?epU^!#FfQ0;`YZOO zrryy*IG;I2d;vC;_KgTwS8VVeN}o|~UC}@Yy2%n@|0p!MzW6|1;0nE79Q|B7JBOZ< zO;?mH#kP&OH*6{H(NpnfS#bQ-JY=99{uOXsvSuI+lJT%v&j*TG&r+(#k9!)?(o{3Y z%b^^5X2rIInB-{bIK{sxRC1sw>2ixwZ^xB-Pr{d@1nv?=Qe~0hqR5=MBJay0$18!M zYxpeuqxVG-t0?lED8g3BUoMIld|4OkyiYQu7dB}ltGv_^4DP2LKo9y{MxW}S_-%gP zOt62Au&+KKe1)(n^^PN$u^cdVn+Ux)?E;QKUe#vwD3ppJfqYX##7yZE^Fq`Tv|&27zn2+y%6e>nAEV2)&E;APyHX(DeWckkTGi8n zqjLxfhDtB$%aDEN)V_CWhst9e-v`^IKg~p2fr$Njbq9uc_9@jJAQ0D}>JE^k>k{$a zJ%+}s-)%jVKyh^k2G-TDy5mjU%$!r*u?ALi7gB~Bk>ON&`1aNwd~KE5fCn2rbVX7c zJ)BGr))hr56=wdNiR5V)aS;tx3CgM4b}}x zxfNOKBC?JRXPqIlz96z*fvn>qvf9I0oib~U$a+4qUJ#M>B{bE8m>#oV6tnbOJ86xXf$htHl>j&Yi3uV^tc~Kp*3R&kwWPK@| z)h)BG5m^@@>-7;?9|~u^RAv=2;*e@&&6inw(TgeJtbI7EYdXFC8?tqd$TlvV?ciTH z+gN%#3fXr4C7f+&INOIJTR}lXU?|bZyh%hX{pjJzNs^$JOp*lEG)WTF)=83}{F5X> z*|knfEK$a=;rZknMSQx^doX>*X;dVii@<&qM!uoreM-4t63*~pFAeVN?o6*9;8!@% zq95n+4|=eh4_EO6@rvLM`mCqVxAgf*{x&}cf;dY*_Ia1l^(LHUht?zG$c@pz12Mn1 zNB@pJZ;LyBGV=RLp)#>9Hq1{_^qDqeNm?r8B4hH@q$t)@^E-=^6>$V}z)(_CdBt(yL`}akodmIs zr8o(O8;NplhY=i{ip(dnrkarBh>@0SZN^(Gq-REmU`D4@CD5s0d*EaoyBshEcBHWN zHKg13A4=jQvpnioobfl&`7QtA>x&8emFc3dmFHt+YrP>m-k$^eM1)E@qpjwNBxp>n zW5x(Mv^i5^7^MN*ppKT#_2{H;9}YITtFdM}@8nQbX(#kpI#vm89tPcJKN$d*hGE!au+R>oS#6 zqLdyO#VgijhR$dYE9NmO>N6gzPNK?0Pxxv>AGa8q=%dzQnLf>_-JWiL%R+Vf;phTk zNiK$y=UTYS-Qd18XeIpf6#lUO3Q_>vS6%4A@)UdV`lMp=~q zmF1twEdRWh!D@K$f~!|G7&lS_)SCMrfd$xS>F!ozqprbzE+L` z;QOx4R$}+-9Ft2>_5(cm+xf1_X4EqgGAWPyIfXY_rNm=sQ)FE+tp^a>=}Q*XhJ*qL z<7L4o{IYL}_8rIbLfo<+dAsl8LYB>4#Tq@lk|_9DIyBvx2`e zzof5~)#yy#wc-2xD0z~5Hrv~@yqmX3de9|d9&SEx3#PETc>9#T%@ylNx4Cb zCc7OYVcqCPteSyQfK3!abfRz|` zRCZ&I4L=l1DRa3t)X_2`9fj@7X@&*^<@*1ArjVYw5I?Dhkwo_akA=dN@q;Uv$;)Q zNK8t9MW*wt@c0Q@VpmL|;FTP%uZ#tLbF4r~Xe0I^ znqK1Q=k}|k=?CIb{CS*O{X=f1Ij@P5{7#dSwh2nYiV!qja&=RjzUNzy27{}=ND|!p zy9;UB`=<3(eNPr0;J6hC3_pJ&+~~gBus_I0Q+K_PshEg^60}{+|3cOG7P73Inj-kZ zS)YVSbE4BE&28>e)#`NsFf`>0kfS)9!}XOhZcGAyH_A_Iohr=H%5!9hS*}7lE3HkXWaaFdFD|`#37^6u0RHnTuig2H4dCY&8mq+aS z(Zyw0Ap_&>&86%De#$>0H}zry!qXo+P7XYL3|~5f*YW)8|Bu-D2j5c0$?GzhUY5TD zRqWKcmkL~8QrhayVF$bEnNbdQNoH#`v@2uulOp+1vqRg$fg;_l4(+JmM`PbH2kqo{ zGUauw>mEmS2L{M}1%idA^KIK<)SHpXJ?Q(Ka00pPxU7+gMN8)7KhFF}4 z!ZO}hONO{RxiQgwz#qcXzVVpw$AD*FaP+`qz;osA z!n3~l_uvW1t2UZ2dLLCEN#ngV7lwqhib#CDis-g|4#x<#)|O!hqEDM**YD4xM{Uz7 zYYW#}YvGq>o)?<-2O1TQ%l_~>d+s{*l4)aI*O%7p5<7;f*ZDprpEp_?U84{($*I?) zDF~}pESR^x7|tV(AwNXgX|)aRK1jco(y2oCiGC~{i!JWX?Q^>jIa=Hc=(V-x1@zss zI*&Dt>wutg-z*zSj(RyuDb-)dr4+lbB+pW?(O!M@H)OkPgI&8n*CITnPo`{?0GptD zZV;{w+2N&dS_YMhV6YVy7;0_%-Tg^M-OlQh0~+`%bLsOWqb>q3+Fp(qRjmc>#?Vm+ zOUgwmdjb>LZz|G*SXdx`NYqm)>bVv5G@zb`%A@3w5bx57LtG=lgESGg%M4ESpH9koVHn1>Cl?`}rUL;id zrckF??^6Oq#TOgHBs|z2g2aavG*BBEDaUL+Bjs4FI|^Ie2=}oBX9w+udn0P5Wq6@* zb1zNx7Z|wN`X#hlv5)n_DSS=G9EfU zHc624ErKAE#|7JB?~^D?5z=l;B$(K>PwWV_bfoqpS0>Fste5Vt1?_>J941w$1pR#j zF`Z_Dzlivhs%jZ|CEA`6ZKt>oaA=?LjaT9;02yC0G=Jv)?%YYynDG5|O!%KaW6qtF zOG9&K(#YSRJB?g+zzd>^$DT3gPNO5@yrA*BuJt%HJ#DT*;B?+wq47h+eV_10+iPB3 zBXy#v;ky*ijy*-``|9|2oWEZ)e%)4i1_?h0>=JYKsRYCcb&(=WyJ80TFqXcZfjd*; z$n74;G`VhM56q3sj>xd~`cBAxFS4I!U0#R6YHAVB^fZSy3f{!?>Ak}5MG)ss4(&3M zp%VLVG}0Uyid3rP-v`vF->m0owT(!(c1vrSyG;>)YRlX<`Peeo2)|?@OFSM7zug;t zOh<;%D3u$|niGDsnU8FLliM4dXf9ThmbvF3>A582Dhd}E7=FyRm=LKJA?_hgYB;?z z{5U22I63?{KKwW~{5U#P?c(suh2h5q;m7%*tO#bA&QDA5juKktwuUoq2tPK2AM52~ z^W1|H`n};qBmCGFe%vM>D_Z6r#$%D{TZ6G*jX^QK9FeWmD{VyGUJ!LAkW!)Xv+A2^ zGiE-QrKr_Sb%No`$as&Qkr`nh_5Y9m|Ch(VO-uu0i#etiSzlQ-D~Ur<^+VZPvLlaDQ)FkkS5`GP0R7d&CU z;0f^sPna)w!hFG#h%b1VFL=Xz!5iia-Y{SAhWLUv%on_2zTge>1#g%yc*A_b8|Dk% zFkkS7_<}dg7rbG<5FRX|(-8E0wMjn-dawwm@8x>#M3^^t!@R*8<_#eVq5Y@qL6|pq z!@R*8<_+F3Z}6JDVO5wnG#33nZ@B46M2}d7`EGb~GZhiF`tBDc3EaV=MTcgX-G&Q=d_#C&;Tr(`OvC;(qKtT%U95j7D@5a(*Yt@G z`aW_Q(|6t@S&GA3g*}Z2UGp-VQC!pMiLvbZtI2LgS0bnm!OfGwH{j&yB=sSS@%8f@ z*swaVqDK0XV zibSB_T3bbv+F#cZub+g-*WQ+rasJn5CX5fEUsJ@ScFEO6?jE!i$Wpz{C8ET+McL8z zcq|`Rw*b`w$1@ipTt}2WrHQHB+ALG7jEx74Sj~Lnu!eJ9RLeN+#ruszj8nNXo%8QZ zGv<0+{K6v$$El^MEoJF~=HaTtx?;{@ChM~9W}JKa&X?bP<#(?9X0L8m+Z@a~YqzqZ zma@?XF*$MTgez^sNT)d|y)oE0$@K@qD&;o`M~pUI#v* z=^$af?iGHn;pZj%Tqk*jVzQM5@@bxqlO1i7XJ<#d{NBfh*>5by8>TRkt@yBH3ukQ^UHcbu%rxqT>tX<9fJpT*P>w$)BkTH3IPM|V)HgJYlPs-{p?Q_QMz9Riz1 zKd+cxsPfJOVnsYt6hH?i#}znyZ@9ovS)iSzk(pzuz}RrR^B={UJ~Nlz=7!&{91kOx znK|?}C;WC1y^ZZIudIPo?k=ya4=l=z4cZ~_xjgauD&iMOzp7Yb7h(n-zew`elPspn z1ukv6{}v3cftPyY;Bzv=smv~$oZ3KBVTN*hTWqWO^6;BM*@zdP?$B;Z2S4rsK0G*8 z+<@(R4*u_MN@thqV^VPNvSJQ&5h}FeE_mK1p3@!rau!g0xZj^042oc}B6p)ENt-LvyR`r11F4ct3@ zw~wcC^Z6RBw6$D+VF4cm%^Yu(J_{jKP&a*#7bLgKRcNh4q{ndSw6iL}Z*BQn;PxEXZsA|On#Z3Nw zp`?88%z=u~sw7bJ8Pwt;dK{P(8b>vL9E~L6t~-x}$MLC!A{Lr!Gzi-HPKFXtiRA9C zOz1=9{eKfxeycE5UR{~U1B>w@vVb}Zwu{MDIYrgRP7%=Gw<<%40II=A&a()}XlXEk zii5h<$V>9<{4JEO$D}tJX?Zr!&ON`AAtkTKvvW9WrA6wht&c^z;@=%anWlnt>;lzVZjue;JSTG{q%g?HIv0TT)9Ie|$*^KElDFM)O z6m;EN*Uu0L<)yiDwx22!t-y)K^>|u zb=JQ-D0^Ao==)XA14y^P4@X;~bP+=o&Wv-U2J53^yDH)Rvq(o|w_KS%Jt{qP9xOv_ zqprdO?oQ=}Gjd(yK!Dwxg+IG9z(2mAk`Bs1b1GLY{MvOItxP%2!o8GQ*g}L_8+7*$ zoF!DZY@D|&w_KZ%t8&;+EdL=fZ{Iatm5AfssQc=inBTck{!coRXB?;P1&6}OPUums zS~k@yMKrB<7Bpxi+qchUjJdP*xga}B!w{rSKN)E_EpPWU;I%7Lq;u+wom_<`{3gNg z4gCJVpT^c0ePvw9>Yg_h^ZZ%-u2&+?qs1i`)(ORRHPQ>0Aiw)gRrhRQazvbJCSgUj zY_Q%7ctpQQPEY*_ENnvWB~DAzYVOxJ1?^h9TDaFb7q)l)HwA;fb4sMxFq~QRH3$vV z=~-gY(zx-`s7wg%Yp=fnnRYxwOTo;725Ns=O<=f*v+nu|##wuH2P5sIK(4CKNN=Ab zP?jZZ~ZIG(Q3y*>-L{!6@dq6$=Y-$FIj8v;3aF> z99ptqmuP%>UbZs7fbuud=T7<*NdGOWcXnR|Y$^yZwISws^Rtrq`FH;Y0`Y|AqnUpT z@`sqEeY=@|MRa~szo&0JN!{@xVHt)i`9R5K32bjU9n=-GZvW+`1)RdEdK{G9$V1Z{*)35p**#Rw1S=U zghfH*QMOyPo1DVwnQ2j|i_otmO4|2sSK&Z{FJo6;Ac^$S0UQ*)>Gmk3{f&|KGLUxd zlf)X}LU3VpmgSsf5wcuFSwyY3$Xb87FqW7FF6~}E1Wb1A41wt)VUVVd`>$-D`=sa| zNPWh24k#?UuC(iOP4c$RJ)3uvnZ=d}5_hl?4wJb1bcgSz{0eX7ZHoJZfE7QGDqeK< z*0V23*C|xPhqoo@4uaGB&w;YIob@hH)C44&VyZ;^zxI5(g+lBtetwR6fUUPo`b~erLgOsP4J|wsV@f<;Kk!vKF$}_jd zuV#bbKyu)}GL)#BVbe0s=-_EG>xSCxv%YfZBl6V=@6mq0QZ0N>wU&Jb$;GPbQ>F<2 zGDfd+vK7ktW+>0WYYuO=I zdJ*_DUQ@F zJl(p2l?z|A<$~_OM!Fmf21z=q!|A=d)j!jsDACN%b#Y83ll3}N-Eds>j^Nk9pvEu6 z;tKeDegyJpF8%ZZRi8#E9x>XwV#L@I>x#>+GtW`Hs|7|zs`_fiND|3VR6?=83pF`E zpScU>yx}lISks}HpfQ7%jah4Y137S$?65Sb-sUMH47anb1l)nH%^!YXl7mHsoJWzdqsc& z=f%w|=qd#pYc{%1jYzv2$4F)LZ>fc(pPG%XZvx5mAUJA_IZo$_Fd=-(gb)N1+y5K1 zo}<>FJx|5yr84A`o-%1B@FYw(Co#iVI*}O$eyuzOv*i6VvSaS|?5v{#e@D?I-FzRB z^yTw`tg(f@^Ef8eE&q}w}0|3&cVij%A^zhXQqPjsC`d8HR4 zn))SB?kw_DeX6)u5le!HpDDM*4h)><`r+m(TEF}1EA3`{=SVDibi!iS2TaD$pM_o% znU}&3>MileoQdAQ@CdC+Y*+8g{C+}YCV`^q$W)C?bL5_PRD}3ND!Y|}cZDbED9CeNc?sN3QOCPt<;|*izKM3z3l2licIHZwtBOE-!m>RD_kL$tbY%djR`8 z>@I_$gPqLh{t`=PkDz;lNg$2N$qnn8Tb+Q2usv7|PRl;MRfU5WTF0YX$^??!Xq; z;(eQ6Vna1Dzo0?>v_9M*tC#WT)2z$od4y1RX1uxVp8AuMfKqGGz$JZPc;5*YMB`QA zlp=G=&9Sp~D^S+?A=!m-O-0t_TUl=OuRA;JzcWIAK-@66{^G!be84>3nxr4=c*`!M z(e84Mar$1yTU=J8>7gTFS8u2GoM3>6OaQZ;r%%gP0%t(@hg4{pa4frp%m0y)z#<`$ zKUbljFV5vV$zOaluO%h&&r#4oJJm-T8rw474(-~Aq4A8eAILK82fBu=2V(piQhwrt7(z2s?EPzW zjb+|STbA2Sv$my(wJtmhJ5viZe)X&y(83puN@)Eln<}wFa#GlHKEFK;1z$0AX`1kN5yb}3$@na?) zKjX(NJZ|PkD;}Hp5p%&`&yU^k_$ohk$K$hd?{G3T^(1{3)8`-b!TV+Md~PFsKA{f@ ze!=hQ^J~ocQP&lGt1bTTji>hmTTeS?4%Rndt-A1h=(++^K@bzD~G7BCJH`PTTvgWZyf0l>H~-NskCsE_A0d-U0r#% z@{?gXDVygVZHnaj@45vG%8dIXR`)A}G(U1IO6Igze;nFz_IJcF-$%lNq~PT21b&uc`HA=NDN`XRIUcGERSdw5gHZmm9(kf8BM*Xy|%0baKJ= zzzM2enH%5I%~37QH*2N^wJ6uR92?gx)!T5F>6HAb>sYoPbDg*!y3TZ;J>|sk{g7El z7qmFeuib>DGU`0-2U|<&QI-Pq**U9O18Ui&;&i8{>T@53I@R#&x+;3T_2r8CvSaIe_D|O*_`O{mDO@sHxHWrjsHpFBRcnGs)lx!<=TV)) zOd0114c`TWJX&n}w{vW~FSlHH=DvGg%<)A+e@xVVJ72!s#DR3$%z}?%r{3&oBXW`FI>+qj( zHTaQQ-3+!AIDyi-MW-#NG;lDs*57aZ^LUa1pPm1Q@Lc~tf~Q+NJl#!rZ1M16<9muKp^Nn(}gGLLzEo&}-dLb6?tc^(++Az%sO?u0JBEdro>#xOv4{EYy55CA>a zD-xjjf5!h$tbgv<-+Z4}R2gAk!>fR*Uu=|5mfC~i^DKeCo+4vqE^?h|<|=4NB)``~ zH#o9AU*_{yg|{+dpZz8}H*`*hl1N@2W$r}_SjD85+=#j}tt+a97@aEPqlE7#Us25a zNu3q`Bi8q7cx>W5lVTm-tc8+IeM`P1{hj=EG5D+QzgARt0e54=C3B;aMi`D00Op7FFPy#^Lf>#o*y^BTTBcBlJ76T= z1vjy=XA*L}*an5fu5?SH{h3S;;X`NZ6KzGFC9!`-ksf>BkFXOnNIuDTS4uaio*Q^i zz{eG)No;R|-l5N7`v5&%=#fB;xohYpLMj*K&|CV9w1r>i3U7SH^ck6ps=85CV!7lWBJPEkqrGACjFD(;IWu?TyQECy>f=|3%yqM3_%^R)`|e_=)@Xmdl=JaN)5z! zkq`poslZp3XN=EQgsvGc1gH_p@xi;R%M#ls8UB*k1O21Yj z@TZYs%MR&ZQMFf^)!sz2AqN4c>NRI?XwO<)w}ZS+r#Do!&=! z6}s#e;W3{eR3hEsM1K)ThZvRtHwuv6y37RWZfgSnEm9x;|6%VF2?BDMAW;HFvXCXSvH|1}UNMRS5jh0IE}#%d*o0&lSE8b#uLp_(iboV75fl=x z1P=`HLQw%lm~~MQCEVGss=8-(cC$g>-}`^Q|Mxx5r%%|K>F(<4?&|95>guYLdgVcf z&lp}^&)E32;p3M8xkQI=IaAZytLe#hKR++@$#_|_SnK**rCrP?LXy$Z-70^W#Nq*Y?Hv^7$XL2B%Rln> z2v4jQc^Rc&gU%axL9_LJGqbt+BDMNj`|69N{nUR}AL;_m(iGet)P}yzJd>t8_1YKX zdhj#Qrs%B|)6ok~UKgO-mxlZ4M!zCf*C7)4poj&^?9m3*MqhH4ao90Rr2~(z`G+b8 zYFWn(529tgIuiZAEYfLNgWGok|1?~Yf5O6irF6gZ*lekW?t>NY6^afUP!##Zg3nv1 z75TNIe_*>@OvZ`nWz**qjvQrFfE}D=ZFvS6fnkWAvISDHdUK{}z@3e>rYM zw;6Yvy}O6I`rjbR`KwXe+{;AXL$fT#Gv>4(@;>(@_{X#L?&c|*MA=fFX*?`vf03SiX|(w-EXoL^}R&0o)s8F-qfgC(U(#b>9mxl;#=D>K1pPOHs3X?2Y> zijrVnE|kUwoF(e?4>*^5`1IFu0opgC_L}!^2BkeC2GvhP`Y4t@A!xe({P0V4W!O%a zCtY)!yF3Z9q^0<%{SKRJ1AekdM^UriXRc4P-`*yzA-iKU}~20gxQqvSr1qDXk zAqEwxr0ct(ltCt%fNTR8tyB_SgaW}(f);;?P9$Vv%hiV?oJY0?J>3X4KVj$n6>cAj z>m4mcoR<>_A@s?IQFLFTJG-zlPq~Y&#C?E;ViaI)y<{c2Uq5BX_N$3X{Rc1=4MKBK*;XF!m1CAw8@TI5F1OG9O)RKjGTx+D$ulf3X6@sl^UWSWw)sYaqCGl}7y)})` zrCZO#0_9KI-{EFMgnit>$75(nhT}-|UBcXT=2nLp($RIeqd!jMJFr*km!rADw$(I} z{89$4jCaC!bHQWO1fMo>Wu{)}FQ9Jz3OlOTJWc4w`xgQ59Oa&k!Td>)XHNS>AKFPA z?duTT`HLCa?^LO13y5|yqTLbD^Yj3WLV{p)6ft_%0Xcf79Q5=+goXuN2w5m$vxi0U zzSJMZv-94O$qj`&i7kYpeCfQ9K|w`DVQ3xdHaanhwVH8!5Ha>CBY;*Yk3Jre-+68y zbSPAK!3dS*^+bEsp?=~3yUcPCkas(Y^*6Y&J?sRSt}jfT)!g+3-wE(2SSs)<^4;)R z{_8}U%6I7m_=Rv-Cr~U2vov@3JJufb@K@ufn%cqC^n8@kHy2blA%9Mht%31YF>YK-og53Kg>fhGA;@_y=AXp!PivCj?|g!yTJx@!HAlYxHI3jfh* zbnZXWA2xm0@hj(%%dRl9=Awt`U+`HCS=@Adbm!@Ys)8ezeaGqiJ~WM9@bV-*mEn-i z;N$cBqdHoxX_!)5WH9ew@U~?Xj@GqCeSB)fA!RQ+m^J{1O{r(LRZ#m5SJ-Vq^jlfe z0{x?_zR`<*GpMQ@+AOD?5{kYzu&ui|XlSHR%-%NgOP`08fff{6Er)4^U5C?H9ueuY zjD0v!l+1~m@%^PtW-?#&0N(fCN;h(QWW^e?<;Byi(%sWAW1^K`^B$R{Ue)y~Pi|!e zhd+R;=-ty~azkKV{@C*d?N0N>$ZB&beSoE0XKveoRv~AVf=qVS9tj{kQEGCaiXXcD zm%-HSWw%nd7mkN+pO4+X!5bg!KD;Q+pA>LkCh%P^kHuCRl5uX3%Ge%W8Om2TZGHvH zd&TvOBlkqWsGNfN(m+1}{6|QsDo8WXu3oCZR;g>T8Xp`sF{ec7JhaUdca?!1(S0mR z)yk*e#~5foAFUqM1v~GnS&-v?<%bwmzNGijKlSOVCP=A|B7YENOs|-LX>4lIfZvart z_*?W#bt$bfhme4Np9M?xSgU-i8jA%@#gzrQCJfx-_cdI(555)*GT0MQ3TQhO&*g5l zf~1ZJ~p|2{v%Kfc6#(f_?-y?^~B>jU7I2jCaf-YSe13{85~=67@X zqpo0|^&jQ0X|Wpf)DRC_SUXr*Sl)$NTtqqo+ZL)}ytJ=E+^TYi8y`u^7b)Nj8N({#V1!HfF9 zd-0#a3(_AE=q})j`+xuZ;RQb(`cLw=DEMhf@ce%AE;+21E?QE&s&=d^y9QLwoE|sW zTO*UQk0KM+0jRNw^|Z#jeeBKNA`>PzP0LW#jSYK`_hVgWlKv&-Y9J|Go#>L;Sb)B7ZB-{g?Yu zc3U+L1-_v^4bG04Vcaz>tXG?6G7ZjeZ-*Hq%L!}Y9EiOxpc@UjVL*ozKD`26D|)(pQ?CtiCMP+zW1 zaUKrw)@o;u>WH<6R8(lgc)2nSUT$2d{WRr=bhlRNHyUqI*xO)tMNBmLUEE0VyAPqB zr;$-h$J<%0H(R9|?dT?k`eeHF+2ar6qmra;FIw5#4{7UT$=HBkb{oUX%;%QH9=37&|cpM~Cz4k%<_* zj_)X)5BD$tyZ5(WQT6vdEYfZ=LH>(efQ^f@$v4w5qWTF9Q1IBZ!HFhiVM6hvI2Yt7 zQ63#BN^O!XLd9+tpVuZ`@Wzqn{1E$Mif?R{1ksgWsYFi*V~#=zV4PW~_>9L)b2%y% zi~wS$8zvNj(q#ee8)-vf2Fmh)7a!!&)$ige5VH1Ij12^K3V`+8?yGzhuib%XGbfXM zXEW%F++6uNPM_pSXbom2Du>lC{sS*8e_dr{A1$jT`qi#^U+MmcPhV=0YNSuB{E+Nx zkQ--3P)kM&+jpiL;CaccbVC>veGCQNbVE2iuE5JzLdDK>BPJBJHlp)rd@Sya)`hM< zQgj0B4YT9$IHedFo;#IvHKv4Q{ggjLQ~q^q*!X*tonvz)Otgh#+sVYXZQD-X*!ILZ zv2EM7jfp+6ZJyY7?)?Y1YFDkQ?&?qdp|SUVmI=bmdBVhTj~kYmJ_6}4-X8-j@n@-G z#VV*fM{(;tf|u@owZwZ;Rvax>g0Fv-DOf}BMA0moTo_yRbZe_$O}flru2gCczVw81 zF5&OG#)f;_$cuHBjK^bw)Ll`)LsMsuH2-2G?_5h9U$4&6+bK>Z|5ws8V9$9cR-fC7 zpVurpv?1qNUrO0huzUn4?6s^|E{7N7S32`iY*SJi!~iJOBZ0(^$9C~b)6}FEf(DW zCEpQCakWBaHynb2Wv}q@$_p6DZt67e-Mh!>Cn8*y@6omIvy|(e#<8~2BVglh=%fbZ zvwGR#@X~VKk@4`~PN!_iPMdFB=v@kNOtCCcj?TRsbH>|BFLJ%N{Fu=rhhq7nz0DB` zhYb<>iAX_@|DL~^k=HASu4INDrcB9*H^qm)wNBn*P9V;e;Y(bPdFIbxxJEx_KNj#o z;G;+>%*tvgQMQqdrT&8X3{0I&`&Q=FB{f15=}YB{g-#51Di{!fA!SKu_km_jXqG0T zq9O%%HduNPorVx}KvS}srmh<%uuJ7Sya3H6 zU{9b6f+7qV!VAU~MX?TNz`U z^6nbz<`bfclo{evq%y~q5bf0GJYH}g%dC0a+*g#IkWGR2lw;a0a)@t;NBv`9_^rh9 zL(?iV!ehW{m2;?(_G4@a(A(M8xO(v0-#9h-{N(-kjC@~DJOyO=@fP@Xh0iwc{Z+>- z8$QW|zcmc4ylwZS9R6V1@Nt~O@l;wCb^{`WzBpYAjJlFtDV(kE{H1jqZd7+M-iSy> z&geayXm*&PM$k&LhC%&{G%faVi_*W=Y{eMk3+gvKZOusdM18Ud%6YOW#K^By!U6}d zzSpr4{-q~LTwcjV0$-dEC&@ZTfp#uK>Uf$Kz-Mz&4@wS=7^-0yaO;pFtUrZY_!*d& zb|osoffA*qM6K-Yi9`4G5ZuE)s^QMWTGi^kEPkTUsC`^OmQD)d5u8p6W8{F^qc1bPTcj45fVgX7E~VNv zTsaM=p7LPpIK7?k^kIap+4Epd+J-(t_Y^G8Q&?du*mWC;2DN9%-UiO0jNe0aavCKK zSd|rI0K<=)kTj;2){HSmI`?$dAa?0Nquy80?Rwc4t`u{d`>Qu zm2*mp-wTHYR_p%Y=D`@lzE(fjBnC9Gk^^oD*x9Q7xZn&#xrL{xdEsJS(QQ$E(f7RdN3Hlshi=^zlt0(p5AdAK-a z|GEQg2Cv-V=JO@pP6?K^qKPDQ*%Prjd8{K2{*IUW`Hti`^pdQFTSIv+&!I1rCd>vm z;%nXnzs@y9W_gTbt#wq5wx5ZY~K(U z$hk%wy|~N-PxeTYam+g%6h_dUJ%h#@7E0Nh@zMUA2Y6h)45NI+K3_hocSRO@|Fy#5 zgprU;M^PL+Z%7A-mG*i~iU$sLbC&WyKGmDW#lgA{vd`~Oz!%J<@sFqqyS66xQ~H>OhHLZMFt7P@yo&-&&b7gx1yq&m=fctjO6c zFLrsj?}YMAeH^S6yB%U|vf#{B-U#l9a<`H^10X!2j;(cjQ6z?S~aQ05&sT!A6FL7JvXxgtMe z_BajL;!<;{$_zc&txFq;t@L)ZIMb37e*1V#EOKC-Pu`VuOs(?BvG#%L z_F;k+PT)!+CNv3x3?2Uvj=ap2Ux^$~xJqUcPezVknLtVjr2mM&WErIQv}7k#MO&h$ zqk-SXa|(?wOlN)kW z)J?|MxLPfHn zQ@C53B-NHIQ{e(^nKJ%y&?MJ1^4;;J#_2I8XnCgWC13D=!v=>d9 zDha<}-*jsa^*_R*Q6iL17;!+t|KCMzz9W(Fjioh(>hj-&lah$b0n^cTPIWLXvxplnqz{whx~;oUo&IBeZa z*Lw$Vx}-nz1KRF5R~FHBU=5aY)oi6|L9A}6>J8oj??!hCy1r$J#&RZ$4dBDu7CEGX zwYuqe+DJXM##UNV0* zPsvwh2VBxEOVu(!;z;;mF080Ut%Z9do-^8JS=wshb2@{9DeX`=*$2JAPzqz@2Y~{% z$RxLZN1S{$o^5(o81X?8e_+BdrBr~9@2MaPx267mRw;zpwEUe~aSt!q0v`NM7HiWE zDMWU8d}Z*f_Uv?u>m&@X_|V-{1~}7gE53H=c{O-na`x zHn$^5e5y0?WJc0WWoyB2dfcw#zJ$0m$`32*PSI%6U}5>$FnmFl+>ha_L~|e>@d)dk z<@j@*5Wwpg$Tg?!x__LW$g+)qT#sToG)26=h~y!+R0v$MZCMHwN4Dn^Gv=BM*F10e zg-!G7{-$g@O2*)FD>Vbuj}OB%&M${G>_m2cU@)}#k9FimC5 z$W3R6RT0?e-ogdU& z2)3yn+qe>V)skXdN(yRi8U8S-!=@Z;oAF|CtJBEwW&Y3Mp;o7&gN;?|0)vLPTFlP+ zG_>^E`7Ik^Bx$LR=fk}lk`oH|KU5i|*A-T*=aNwW=bMsBhv?)1lR=!DbNx+Y?sEBa zk8{3bDweey5l3VRq!_iH$#*yO?R;--YQG#_hDtKl4WGYn966^}HP3r?v8A1glJBQ| zQBpaSa8Sc(C|Ttt_uBWr|G|8XF>$=V<_{> z_84M=-D|;$pjPXFKp@eoCN0Kx35^Axo`}JWq+CYqHnM%-Z1oGk9^*Ub` z#R;fsf_}2}znWtcEBttYy2s8Qm{H!C54$MJlt_+L+8TqBE(nL79QMR4thCZR_yl^n9$j14kcZi3 zVf9L4ICpi^YhF-XU5+_Prg>XeIArlm zGkdXJ>(ic3Tjhd}b}TY^4=SQLyn9tCdVaajjX`Wlql9EcM{0oER#UX#8Ej%X6Cri?-F~)btsE@R_Zgf~DwNIVxIKNuvRJ2~Lde+T3vcUrd??BvCFlG@JzH@W2+Y=l%BI6$C&*4Z*`+@GS z6u$&0?mbZuvg{$%?^v2!F*x6gA;Df$(0qe3CDD6+?Vf^pTHR9-K?8;P+8u#)>#>m1 z4s+LCT#LI0C2{GMkn;rb^#FA`yrwxLg7ccVqA^i*eyu+PrWk~|`>-u!16UC=!=SuG*BvwX7G*QU!9Q7~QeX$73(Z!xUMyhlDf_+}ShUYrd6t(z z8Quvv|MTn>s8cT#2q;tIG)-T@m_<$DEQV_jV{1qL?MVcUTX3GaR`jTO2^OeBKJ^Au zpTSThFW9@L;<`OibpyM>H=ip8_}LfKD5605(zZL77^J!G2Glff4nV#+AQ3FND){C)DBrij|k1HdJZvf=b)Ns;hVEE|u4^ zOsAEXIu~;MPt0ofel|LtyeA*Y@dA!M`g2QTt=9TrpB~mF)O8d8|6y41#)i_Or_C`rJDJq7`KS#OM!Np^2np@9T5vwn zk2f-vviDWP@hdG3F;;S{m2rg8l|DmM{t}N?yA_KE^pfR={iM3nKFhIOImTg$pVVy?U`WXTej_{F|g0!lhqC7{5c_=508g=ff>tw3BilB*o> zF8+g}d16S5GA(74+8>Xl8%vtZZbw0Ce&mjk#NR1f;KYOs4qHi>j01(RFWtD~3qBTS ztgA8MiEkP4CQz>59LHM|&_>-trdHEI1r78ELl#n9UkZ~Z?aJS`pYqAL4aUOzm z{YvYBk+ruTI6Acn+V*%22l`E}TO5D1mj4v`=I4FR9eo%&yg~!NdUXBYzV8r!*xmqx zh(F?e@ioUlKjUa(pdZ3$*BU@Az*p!X%;M<7fT50jH(QsTYWH-@jp7Y}k!g`fLe~z5 z7S}1?Tkhy8a-k8srSj1OS;=XgMk%LsQ(cJ*yRLjHbq za_iG#$ykgi-}q?Xc=vDZ2B=&ILJQEL1>{B(eRNalds+$R_o5^5JvVj%k{aB0*ZN%? z@RX%LCmt1qPCu}b1uSmgPeqSXvBylJf%43HJpRTro*CbbO?KETo&)HVUG)AFVU*$9 zB5#?Qab1C|c0k=LV@cvaz1)GmvS4qs^42Tq0C5vnh3Ge-6lKky@(|J_a1~PIiAZ*` zt~OR$=l6<{b0vz0svjx!gAG}7m+dKW(wCb>>5-nEkev2WhV=XPy>XeC)7W^6LQ*<< zEj&C6gFKE>w=<092Cq*Hvj@s*+i&%SR;Q|i4bY;L98A+RdZ^2zdlnno2vhjOlyyF3 zFXbSGz{$HYTF3J7Ln|{g*IE)}f>}aOvbQwywko5J2@juxT(iE>qfk|rWtlar$JJGE zpDKHGB{Z_{t3Ywooe3OU2G$NKF!=6S(ps!!5 zFn;xB0e4jY0ic=8L8>vMZpi;U$&@ZhFH0<5LOl^mb*Neh3qjoCEPZO``3I>y5W~G8 z>VC^WJ$|Sy274AYL(}}0U0as7B=w0d&c}p(RIw0J{H%yGjNk>NKOJI6CbrJFz@lN3 zv3RaaFm+q;Q)huvg+fiGlZc|?^$lcLNQ<#e178t-OS2hEv!$L^AHX_548(#2d0HTG z6d5dks*@+<_ovpZS(u(2Q)K*MBlt4(K138*n{VYBsYw5C%W)BQz)^u$gW4dxX@^%; z>wBdHIn)G!S2z(`wMsVEwE~uc8=5@p-$@Z*fYp#L@z^B2>L=Oyfwz1!m9~mGKgVYl zMDO^AtJK~@w?WR2tu{dDfYdixAxjrNM1KIf=YTi2r6nh01S&|<_BTV_a=U%}SRB{J zZ7`e?YN35Oyev@DSIwX|j#fQbY3Pt9Sho;cKg^!neDD6P?xC};6H1%{OQtT&(BRGb zzn;PXJS-@FqIeyJ;9uZl!B{_3?^cI_lP`)^jF`Vh@vZhA&NO&|G??ur^M12HzQ*yy@bs~q^ufb z*O#4Cf8;RpQ=eaIu>M|z)XbH}hM*OuXRa{!V?rloK1-D`AH~a`OI~B;+ya=0snn;# z&Dr`@66?!<@u`T0mL~Fa1bhu$Np5kLVEv7!DdxDs9`|*6UorZCjPf4=<)jA39e5U% z00dJzihozz-4wxVr0hhA+|CyKVAPAogPtj@*k=1ScN*i;qKJl=ENNtx_SYm~7pDy< zk|r_i`g8%0`6!Nl@ZKE0e&Wo1KA?YQngl*{blY}cmH%sF_#?HjilT5d zpb2+Tn8OL)6on7B9)5c*qq1m`*SiAkyhDni--n~|!;dg!$y&GzMz&}bpi)`$9m9v) z4jkNvGK@j)_-ewh_Aiw}fN%6pz(rH(0k4l-ioCshkqrE5>mD_8I&3sm>FsSk+j>Cx z15uw@#Vr(ZnrcDLIF}I|c23Od95xl)Vrm*NaTqs4+dIPPAhl3D?|ghMiSD87rspMU zR5!jOO%5%Ao&ZawV;iQx>|_S#W#OAQi8`gEzTIR2WjvAT?PHkTV;015O~2vvIk7#5 zAd1p&z*+cG0;_p`1KZKIm)X0Rw!Dm(KR320THKw!0cNAXGY-!!xN1)pl;=nyL7Qyp z-MhY3>Mdi7=9@8W-*uFi7=w&@KrDCNLzs{d|-W9=#g3R6Tu%B*;zrhMYg+kcH4RL07f;| z#l*ZK&_Ef9`^bl4%{CZHZiyhDx>+`erZiF2z`nv=rai8K>3mJD~8BV>MvST-A zE}6OEJ?N*12Mu3}em}>7{T0nRP`!eY%{oqj(NgiQB8M82?#?t>Xo%G+3heO<93gmp z6A5uR78~-kA2(Q*|JUx3a3dY9HC%GQ!;vYUc1f1gF=8%S6HGI^!wAJ=F{r1;!y{GU zD?jvw+Vd8XLj#sK{5*uqIlB5k%7uNy?C5-$P4L0Qx5%e3?vDZgOF| zFsR|s+E(dP3?+SPTP;+Z5c_1VMmJis;i-*x7g>NGwTMAff{wv>N^BF*`Ix!h-G)qT zah^>#_BkOc-mZn(a~{bY2)^I5lO4fbImCeLTPiTTxjI-6slN5JWXOFYR>)sz#^lor zEbCtrrsetaEq**S=fA~1j!!5^rQ$14W%r%*CPKtw>XN^ggZJ2{2WnaEJg4VXZ6=(& zL-AW*g&7-`Vk5aWHRun>q9m@TCyMYzh!C;;F>!TniSVbfpVID6R91WQqfUsA){bxH zP4r-yhdGf-Z}Ul6nq%qrvuXRpV}U@iOV!jLUJ@ja`iB=Bo1geEx^fNsIXe*R3rqup z%BWwaj$4s~4uv7p$pc&YwnM06B$F{L--+ciS_(Yq9h#)Q@v$xc@p20msGXp+T=TO_25)^TiWM`c zk@vxTO=vUpEQ-2S2^vdYcnc;CED@nL*>r6QH_2dYwAdqX4LN9+&N)Luw317Lp*6#s zTnFVtnscFo4VTU6G*m$|kk)r}qaTtxlQv}9k`py1*o8b+u;9aATb|AZK{K;Z)nnXQ zZ64eZ@9T$3w7TpFwbe?QEo_9M^eftSk5Qiri7!F<(D?0u5t5dC7)wjR-q}{jI1!;G zm0@=9Ah-PZ9j9oJG0I5bnocN>Y?f?f{FbwM0d$(fDb>okt zjqVZ@-z->fprTdYK#a4%uj6C+a|M=dM}v6~MhPzveTve*C&WZwkaM zhA()HKJURYjCTCNscc?3+^_|V?QDU}X3*}Jgtgh{=o+Hvxs$X1=Y^YJc6f7Qzn~Dv z-R9i~hKG^+)(TZ`(WV0o=2kLZ)tW+eSg|4$CN04mk+F12G6|(rRuZ zsk!>C{j5gs^BMa(;Hgjl?V z;pI>Ew}X!CS8nJ2N|XHm=CfE#qa%54 zFhB?&KN-ii=cP%`{oXYm7bevG^I?8o2(JIJy?Hlk0oVQf|C|~#q#vH1zNNa~#3EkstX@jlDE*RCin@dq@`N{3L+Gw zr6flvM^g%i$ogn`hC4G5y|g#8w5O6%rt>Z zr*(MGn)4Xh&$SrvJy0Qc#*Q?hbc&(3n_pWBcV$)yHlcqCP$Kw&F?L zr@Ff>{$MBl=TJv}@!BQ2e2EWRJ{P54e0z8E%R|{jQemZHDz=ilkh_NX%q*J{Ws>-v^I_SrXx{R{YpnM zD)wJ<=(&3Mxcg7K$EeyOLd${WJy9A?D5iL|ywmyhq1($4*^kiCH>Xy)F7tm4=xfR* zwFmll6O7Y{*sQl!90BKF>q2h^K_aJf-*2D(M{|UhvT;7o2PdEF&2Gjr^KJrPU!hqV z1zPB;$=DqVZEjoWaM{Ef%YX0DZu$jHio!$FScxkf%v{iSxX^aj1XdQHbZN^5+$-W3 z&-{WOv)s1?hiU7H`|T=yM7cZT~$ z9+_;snmaD9Jr5XAULsk5zrM@LBwMwI^=xNA7l!6Pc>#Rt!%AayEH2$80%=ZW(|!xOg2hvX}+igRNgT(D*Vvu zQEkkT8I;A5jTuvR8VH-ALh#vHAv^qqR~>I)p81LBIL9UtHs&y<`~)wX|t?=GX`Qp*Q-?Cd|nMk%c|o3)X_s zhMWkBd#5CO!ui*}+}-OrQuwUEE%+>?Jwn`sCI|l)1k1IO4<+ILRzwTe8BEfU0D4pD zo13_|fXl%a8RgS#qRXZ=nrZ5Xs7Omxphn`UoA33*+7z#J=BM7BC%y0n>>os;f)pn0 zXBKz{YYAjiwfiRd8KManNQHD-?~mI0X+a8|6d>~25uC;1k#a+*CI%)sw0ur(35@Pm zz(G)Hl=Jc3;47JqjYRq}2tiV(YJRzChO^mr&&Dq?o8m<%wC!YYam$}qGWx0mW*6*) zf!2XSd5>+hSiLkqg`;+EB$!?S4U?iLWX?Wl#>7Krl{5`Nr*ulb`uZ{c!f}bE7UX(0 zOQNv+2YBaks71#ys{qJ25%-`vd&PqtZlJ_&p%tG3NiGDd6#61O>SLcq9Nzn|XGI zXL{J)&L2UOB!fnYtO7~&3IZ;a{LM=E>BZ!$-5*C5E-5CjC2n^xa_Wap8Un%D9B16z z28!WPjM8wfNs!vH!pKWyu@l8}nMCf;wc!oLJ^|W>vxUwr#$iRp0$D+1q5PGUuKH5x)@61sM_^t zI5?)BqQ52<8hbX(_k1)X)b=0$%QF!-q0WClsC2z5a3oH~?jK3AjD2lW_=Q)U^W1oj z7d))Wl|Mj|i|L^q-=Q~ov-^WgM4H$=!09u|Kd`2KpasmjuP2lK`=k?#b%XOPK6|&q zwFD{B^PZiZje5oJ_^zMAr{6UzKQ@B4%Kmw+U1%ux z3M`xPL=cyr@D9sP|&_slyNe!NYtE-YG30TnFJbm!~y z`ulNw8w=~=Tpl+47YRw*9lyMCu-LyLvVgGw9@|H66gdI+IxN|}55#N&o>uCv0m-dYeOUJrHlQ?|> ze99@!WdeC~0J6{Ya0l~{5iP&mPABBgrS}b_BiQmKB(!k$jG-=)izU3|OO#n45TVK< z711ORME}^UY>qpyKEw|_JTgDV4@r`Zre{eIpoqFBR_M=eqoHLrX;pA*tVt849|-Vk z%H$}=AbCeWI}~F~oqNPCT2IP!)s#Xb+RRFy@Ou9OFm1r|5RhbG=h`h}<#DOrcpzGR zyr_|MxxyzuYjyKsd>Yf&?TS{W)pH=6F#cxZNMK?XZwDXVWnuD=!`VAO&s|h*StWd{ zTVP}KkgJH|**_F7dekQAgsiOK=TgPUGVQ=E`BXFb%N{3UFVf_aR60 zr~ygngcf-}5;q0+`X|4an}9k#f$zqYaeU@Je;;C%@0~jn(MNEa>Ej?X?B~y@K4>CX<=8GJw^Wj6gi4yIg%tf<^#+#)*l+&^Tgi1q^ZIakxnSuBiCb~ z?gTJ)|2C4WS@_s0!3nBW-uU)YsDB17Fr;V|FBC8e{Lla^=QXOC_^v}59s$H)%iz6j ztflH+09!mIQ3#nrxbdR26i8t#=PACDrQn_pm+%=(z_l7RRf|2c(E1{_Fc!33T}HX6 zmGKxiW3`>?v$u+9aB7aM68mF(>hL%kt@j)QuT}Czt*C|mzXrmmXRN#1M--_(owe^& zSp;gb41FaQn~J#lq(!q7U-ywjBu6K_D z0{RUT|MttyMX~hfJNnb~3O((nOP&yX!@tOI`3_@Dj|W>T|Op- zJ71+J=EZc>iosM!34>q#*g{so2@}CdvGA`UhBt0;E#o7ab!|;l&GOW0g)Mr>wa*%> zSN)~6Vx4$3Ht0A|lRDw|_oXx1_*VSi#YfmEQ&j3W9+$tkt;IsTtE`$`lU_)&#i<5$ z>VfxM8T#MlP>M(5qPDoO>xR=3y~KoFSS7g5elx!b16(Nn5t#;Y#5;MY@7JgHXt4hX ze6V_UMVs{&L5{5AlS!}94PED3!Z5O`FZ9}7D^ApsWRMinW97BE55C9y9Yq}C4tYZH zE9CcR31RdvDi>_3mllxdo(^YYA{;};U%vZKQ`fpuo&PZ(^h%MI-TUfpih0+fg}>UB z*L(&1^y!bjKL@NxWmh_C@c0lUH(&pIR%|p2L>D+>Dc|-&D$Y$Y0bc+S;LdJnCv| zeD=vTNM+^27<}gVn#Ck%kJS?qc!^cA${LM=i!T(v-hGK-KNWB6+YI+4a9dTz=EU-@ z{9?lPu7uC00E%<9L4wD25)pd=;LRlrun=FPkxcSWLf46c%W0_R)}r^$6)DH2$)GM9 zSa6}ZT=NiirMG81#Qj`sS&NnwEzYmsG^4tfL(bm?x4ZvwvIL&XiOem;E5)9w{eJ}a zF1ISb|K_P!`0Vs!>n-(D=?8w<9+z6azA%u@@Fdpw)9UQ@U2;7c?yLRqkSk?d3cAhd zT@1AEor7KsaNM~eyBB=vc;bcUS{m+8Jpb1FZ^s#YAxrwY!_F7B*M&*)!e^ngJJUeV zk5|{xwjg`1L@8JNzd!pnSNkOND2{>OTWg(c@)Z$v;&oF;mOiYo@*&DxeT?pDHp^r- zr?cEYpTx6Ju|n`qr;YE;%kOO2mHs~Lzltfol-G&?|0T>~Im84*^VKumZ}cO9w~ie6 zo!R1|aX4f73F?7wf)<40YP-8kyDW6<`atN9Ij#mIn9}qMDlIUkhY3|wPh$eQcQ*|; z9~~)DH~w>2k7l;)3+d0_bMy)5w=n(~XO44*OW|u~13Apb8C3&kW(?Rs(b=KlVzWpO z%X{=_N*S#Q?})!WZKX&m>qHNQV)RX>ok2eIerX!;i~-y)i_`SWX1+k?L8?DE*{ru# z92=U*@MQf5HN>^krM&Y4hA78;5eu|WjeNAJSo{RdQ#lUZtRa}((+(^tBhj?x+Qv(u zPtCB^$Hb}|9OJ5FHmDF?-VD|zW++F~C%b7XRjB=4LyYe^kViOpj9r$Wo=#5vQHP#@ zyu7D0qwM~oXn+I#-xbrxP>4Pd@=YV=a2GulTW3`KeSYan+Jg8K8u#-fd#Ii4vl@JV zwFAl_<}ld=(fenU{yU<_6HE=SDSVe~;X;*Cji^UjC8;h)2|EySCB4GzsAHHA(U zH6kt^Mp%}ueNwS1wWuWIk&H^B>&4>(>?CPt7}*655tocn_=R80l>w-4HyqRiW$w^} zT1<|1w9Q?mp_&>&?KV=sfT9j3RFp_=E9;T1Lx=ZzfFO0EUNydXgV5cUV{-Y53by+F zX>b5;a(Sofy2{wygv$M3b!GsvWO}@$Par8yrQK3YabY0vNvnsE9Uiln&p1wWvj``cu|r2Y7sY7gvrsbOkNQ2CpCBRxl$$ zw%N#`BgLke**>$6B+1gKEv$_}^0ajZ4=SsgFYB~~pR?v0Mt(H%#(#rb+`k3z@(Zf` z)=#Z^a@3@>6bXxKHEq70O2Wq2h$voM)&3U5gTCZvu9Bz#oH>dG`3Nk0k*~uT^%V3w za`_Vn@f}JZiNaaD(i@RAp2?-@SJ5_`M_lsOMCT5DN3#XL3-W#j#Y;qZ0kHvN*x~hq z>sNaAMnBH&*1}+8K0p$|TK=wh1gO;ypw*$jZf5V??qHUo)9v1i{lbNt(l~YY=d5>@ zE8))wu^_@$XDB@dWkXpQ z3k%}z) zt9R|DL;9^yGSB1J0oq!Pq;<)vZp_*AwkOnvkrIDd3^m-M;ta}X+RxAKD34OJ(Rp&u z0Stw55@2_DB(+US@^a(gu+c|8Q}yL4gzyPtSPWz$%}>ZmH|z%MwB{-_h>e5mF<&Ea zUmFnD=PUeaFL`ZNNcUi?h3JM$qDoDua7H#FUFxAh#-q}M*>7!Jge|`4=L2`;@ik_p z;5tj4;s!GwT>gSM`53>h%G?ES2xnj>7dZ2@!2_uF_}tJgvziUqp%WUTBpihhjDsGv zzBKtQXt!>^TVA4_0F6YtI>V&q@SUS^uV@)a)^&qO_=TGL)-5-zAr=<8*LL0mOWJMR~_wQ}%@J!x4)vN2hT z6d?387(TN;CusDH1pTl6>MbHKGA`%8&dT4d(p1BdtwR>EagV^c=Vf*RwR*G&2+^T} z-_6fx0;X`5@i1zg4t!o)yyd82-9N9|{{B3Io<63qJ3fUV^{ya2CnwPK;O7k!d13d2 zQyan4bdtPHDZlzaq?rkOm0^qUfvsvSz;<)|x)UDq^KJItvA{G!!h}6J8?L?I5 zCm%a!UVj30p;U`uN`2H*p!U#S>;#4{bd8&1a+bVCs3#>_FY7}P_Ly_`8@$-M`78FF z=$QoP0T9>{22Tr^Jr~cD96dW}4c31qmz<2woe_Zg2rfZTZz=&X4e|pQ`BdCGiB_+@ z1Y=9N?QCiP#g}`Sg7C6}(7@~=LNN2!M+)Qt^2DwO76WwI9B#uW;90uRDB;4Rx^h0dWwoKnqyV8f9w2{Uoh^dVMk-z-HS6=lI;o zD;~QQ!yZF^)r37L$7*jn2-dlG#&c*s9r?I1e06H-=2rC6$RTA;!&+E}1so2~T~iRR#!sNaPfSPO9;X4U##V48b>hN1 zmm=RQK52KmVrALBtGE^y7)=yKMAkrDi#e|Xmj8UmggCv?L;E`A4#BWvPy1OH05l9Q( ztcHkQsv@9X@dm;8`?edR^;Mv?es&HxHjn(>d5jOR{dp^}i(Z~rFbOzi9_Zy5sJ482 z+0CBv@_|&TBGys&wg{?Q|H8>3M3}9;t(?i*3;%}^slGEQjpBt;+>I7O&slPug58?% z&~Rc1ZpGgJLywHVn_$23D)^^h0%2ncucekPn_w52f`DJIeEYr#YHnx`(d5oyb#KOs zaA%tGpDn>&E{Skuo(D{xO7LnUP0zsFs^ozwKKLfIl=Q4(>8*U0{xI&eCC)U<@x{Q( zw3UmH0kWf3rEGH4J^Jpuw#=nzX(j;V-4TM5BUVJ{Ar9^<5O*eKZd!HQcksEI%r8%b z_l>g~S*xz?>jD=~=xIIJ)kq7yp)fHHf>E5tvPorr(=)DwFDf^?FAEg#FCGXm@AA&Q zF`n$}8DQH0!>oGv2W-`2aIKXemi3Lg1kjRod$O`m z$6!0rJrR%3L61z7OUo%Y$-ZY5)1x z=J$q(4jBC10hU zYXw_ZFPeVFQaBP0DT)l~@00PYDxrgN zGM~sd1wvkRm6t)*v_-PU;nNwyE~fo@eL*VddRhG z%;PQ8ddigci6QsgSJ`Iy@+Gn%_8fs|6WoUBY?Co=s^k)Uz6P-|hu8ud?hs#9;6Mad z0i|x5!!Y_hKp5XY`#s;@{?|>pE5PJD9j1NhuOmuSqAq-0uc2f9=o##Nm*O9~sql25 zlaFW>_O9N}N*-srJ}AraOD!l2#JUu{5jX$16ZJ;H!JUai56OT*vxw3^W;u`kZnRr0 z*Pp7ZVS5Y|f>h~ArE3I!Wx3NR_a!a!MzHvI$rsG_1BpTk=g;5NeJf_DRU6ALhDr>i zkE*tYmfC%Vi#MH>-Kw%^gOGAi6aQMKyWuT|j(fiZIoA*&xad=)_UJoBU{}=lD^Xy#o*F_SO zNr9Zf=icpkC3JN^e06`n<_}CQ;O84hAE!s?z4h!4JC;*bCv+v$^)x6hyt6~tF1kHzKOkhc%xdWN+DhoN ztz9y!Vu{9++B*bmNQB}heTC)ClA*A}Ay-zBfC^gQ5bUg5V-WM~L;RP}`_11hC^oP= z^=B=1x!p#DLOxv4Yxa<|-Bmq)3?c;v{UV@Zwq77On5_@^_G$!`x&u@gh?7cuAGz$n zUwJbS>=cp+B$EjPN4uz8|8k95^Y{2wm@%c*szdLq9pSHVA%GBr15`f-5damTUyR}R zYj4v!az7Q<4$e+LgRTo;hfThh*WBF3wcireHYtRXL_i&_>8^0)h+RC>$L z*#n1{dTBt^6p$8;HzViebDwFAvmdi~52`64_v<%&?;}CtPh3&=IJ0d@wmJH4$y}_0 z1!}g8i9-%0)@9iA{s%!ozQ4|hW0-`cRH38V2rTf>N2Vs7c#!Z(-M@y=I{vEKiN;Cs zLC#Z0i6~+SMc*3eS`^pNJD0>6?D0*VELi15NhFCJRgP^W9dbzCx5}W6OeC32SA`*m zvR6$U%_0u^$@i26Zx@HY&vd|^7VJ!ZFx}SnqO7~%0~0@_^6q9J=K)A-+U~sSY-v$a zoEtBaV1V5T{GQhR2*Z_H#zUOeWAAhY`0ksT->^GbuWWsVdZion3d(t&yMyzKAmjD@ z!ocx*Z>w*-);5xD%azBgdTZ11>a~U$+%wE>UZalJ?RUo;g4}OqBtH~Me)vL8@^|rQ zNcR4?`A|ACRIc(V_A}xD0UxL5KJk4F|J=)j!wd9dYWV`-Z~%Mfvm6o&`@O>SUo3gx zE1Lc*CqUj|_q!Unx52s(42C#^a^f+%Y&9K&Iq~>P3-0BYzwjhE=JzOMu=$U)rz>*# z2k;o-m~v)OqgQn1A28T^;yJirm2XvARg%xV1L#x+Z-5!GsGM^^-7TT-=3br@5QF;w zqWf0scQ%={v$>4TdgQZc+9{E@KGU8UE+;4djXy_fUm~HeUgf?C+EI(=Az8msBPt*E zZ$z@zh!I#JG-9w~t56%!0y5Srw?I*BkggqMUImU@WFxKQb=ZQGnNV+Vr z862nZP2@W($;ogxsO>NehQdP%_b7@on58{svLLEc$5ErA;-Ph&)-(?^2^ln51)+*EI-e$JoiQ7JQ0aA zh?N$hv>CMyzybC$-z3uw_`mWL({xLY5TUx=Z*#4XFBSX{R*BFE|`e-C&bC8 zfFP@sx1Lc|8ge&)mXkkaaHlOZsSZdsH5?+wiUq=PYgdw1xq(ia&2d!Bl0j9k&lPtqxM;m%b)$$UMzNls1qlSZY3;v1F4Z<&Jcf7lgYCfwZXV3Z&f@1q*# zN0O6~SJ+zJ!(dP5-en!#$;CN4kwsn7LQ&pUPU!4Gtf|@$z53TcmLMTe^_!$>i1OdbClfndA0j*7$5J)pEY|yJm_vg_8$#4 z`BrkbbCFQIsH}K{6EJ!4L4!Y7s(>=uZ_pWY71Esljq^Ji!rzr~8#MF6M^^dF3m31_ zn-?DbC6)+uo>kh#%?ryhm&i(xsi6&@LoJnJnx))fp{@ThrKFrub}9%wwzOGQ4hc!y zCJpdp;t|br@lE5~jldp!XHPVlI~`Yc|3dC=$;{~4mdwYESGjtFiA85W7hp=%g#YLy zozr=I+*8$YuYE7rxRc9qHtxsY^Nl+U*&Q@iskUcn$nJYi9kNB^;sfeancd<%&g{k^ zVOdd+ff{JNq(R&R=j zAX6Cw+)VY!<4DI*II_}&0YvP5Av5>Dx7EOlbOMM+?nAdaDfLPu0Q5v*k+)Z3;0ux_ zuHetwPj~sn98Mm$i!MeIjO8ginBbWLl)?=WGJdDQ>G?AZ&dT+Eh6v3mLpsvMbs z?zj5st6q7K>kqo9`YM>ZioHLsf^#SM_9akZXI5crR$2Z;UngRaQ>5q^?~_m3{b>TG`j5n&=zu2-M~7o5lH6FIC^rk((<`P@cOn zuw}lb@EBE=+K<)w`%1M(?;EApUu*WA`YI=?Rc5GFUZ+*rovK_oK_vw*y2tY>r~1X} z<#^q;^KOR68&47*xf&kdjSQx*5?xv7-Q7GNyL-c(tqpQ!<)xgRm3<9(x)yJ;pRvBBhaY4vN^A{oogr0dH}%p&n(VYoLa~FqX@Eo`F97PD7O6$CxdDbPA%~>gj>rX0JZ7SheHa|lki!@}+N&l3 z{`&K~E7@0aeHzmRUCoUN)$>B!L9*Rr^%+{^9%Dv``G;AvF3Rc}cUalJnXhqG1~zht zaTgq(*VgGSa1`%lQ6JoBS*ToC&D!1-z1>mAE?CFM657=JE^I5=+rGk`-7kb5VW8t7 zLo{Vj`oDq)XbU5e_`CBHm1O@)9k(DK*y>^#k-gRoCD&W3;8ZnHt?Ss!*sGj#NKnGh z6dESbpO4c7dZ3IZ&=<%#Z^GUAv$lZq>}Kaf>+fR?{Jx7#&W^*~r`IvS>qz!GdUzd) zUPpp5DJg=yp`fwm%Bmp#2F1b?WMM63S7pPTkv_3X-%vO^*!Vk2Mb}d-R800Nt8_rA z4@L12Z9~^m42oIhqVxib62SuV$g@hV#xu6iS*x;*zgZcRu}K#*rKz6^MQw41nWhNE ze~{DALD|wNv%Czq9prRYlygcm(+)dq*0djN(yyYB^@YvVzhky^GP~zlQO>RsmS$oQ z0OOyc@u0Qm55m$^e3dP$tj0!b+F6^B)hHS-TGMvJlY)aa$uk!L_54m)iZ`ppwC_03 zyBSbn>7Qf{9CIT=fnU+u^D|*79$q6L9n|wGRP;;r2_l2nhuT`D&j9OkvYam6bz zl9{y2EL3E~;buU}2k9MH&A3YCSa+a|y=cx^ZId>U#~bM*kL38ET$vK}&Kf+en#zJ_&SdSiMz8wa9j8Zoa z^+~&uX>WmyXBUZ*F-drffaDKZD%DFLl4U&*!g>%3y!%nXBj>D#wDn9VJjh}*930z7 zI@HjOF?1!tENpJj0BC@vL$CrW@9@0eycdJ&26ql6ISPOtl%4QRolx8#8Sk87>P3;k7;se}#2W_se7>51mi(@jd`HQKzQ(HDwI zW|MQuQ0{^?B~zaIDTrAGNbp4hUC&4#`WVglQS@WF6MiW6pR!IwHGov$-|wrT-&gDU zeKn-5R=czqeM!1kkXFD{CO=SgE*b189=>|DE`Q=zjIgtM=Co3RFC_rwyN(J|q zqPaj)R<6b1Sd(IGu4!~JVwa3_YGNQU^dHk&^DliG+LkRH_R)diYDNbsdx^tV^LwrO z8#g|GgDWV0kn5Rq6ey>Pw1PD`?wJ>w-j~#m#{{di54f^bI>Y>a+0!+@U(97hh+~3H z?h5R2j#V>F<$`nz2~dzH*GCO5p#zJs5bybc5GXY92eiol@Ufn>CL1 zm6TI9y>^SwDOVfMHRO@&ao;zG03{~qiT8O+)BQG=11~D?3?YAJOl9rLK1FP&N=dj( z7{kuTqKKg&kAzxIiAFUBm15lx&DLjpYvNBOTiORq9XW_L49dDcNe@o^Z=n^pQH+bUt;5QL|@g1^_PC%>=+i`KTNDR$NjvEg~k z9_4suJOye-qS&K6`3k9@aPHTZATJmiBf5suWMN^ctMIr9`n?i~7|7ucucKBmb!0>g z?EyMA3oC2rQs^b8#)JX3o)6nL;_VRAze)qS^c{5nX=5~9)7rBkhFd9!SMm>y`pInR zQ)1Ek@NOArX5+7+@yf&T8cxp+8oSxoFxq^Yx)~^e6I>`GxGEYf3@A0L67RF2D_t9k zH>f<6$CaaZ7p6PMu22c>&SQ+wgv|#$lkg6{>UxZVX3X5k$eqQ=oxzA^Vu_xCH&FW@ zGLmCk;QxM!9f_b&W>8wa7v1y{qwjob5~`ELgQ9l(Ei zjMHH}#_6y`{>x*W4&yOSxpQ*qR@LES?S4FS8%EEcvgbIfJTFpo$03TLUdDt>1&Wk3_G>Iz!K*9ikBC8V( zV-y>E?g44Y6qMfVJsF>q{}+329^XXu1&$_NpmbqcKz6i9QBc&DMQ9Odpn-`dP+1fg z6h$g7Al3}vQfy5FO~+teKwMDV5x0ItDIiPO+JaxTECLF$sW369xRtfdJIkG9k`}+e z&wKCl{(5{qB{TQlx$C*-o^$S5_<0ase%{lKK;%-U?- z5<94bRQi~pEVaXR)TtQhseARLuOMleS#saR3HV{9CW{X(oLH&7haY0fHIy=mJQ6=u zN4?7>dscpyF}fT1ujcoBe7>KbCt~&5B(BMMlW5GRT`3ydP2^CKPrKDbvbEC9oN!p{ zEn0VzziuzE@=$9W*E$)W6Zm;rAk`*-$tHlQ((IqxXgr{soX>;sa0etu!4!dyLOFfb ze!Ev$W%o}1+U_k{hHeJJ-{Wpy{0$G{DpF+pO{=Y=rD)H3joK{$FSg(ofVLRe$X^4h z@n@MyLMN5(#9b297QL5D?`1V6#@G|y6BzXXZFRKZOcU;)e8_Y2Gsqx}%BJ^WD;ariJF%DXy39TX77e9YZOWZ?T2hHLY zZ)U9^YYw7a+_)p?H#xh=E+3SuQGU0%w!~!l?1QKJl;;8$XZZr1*&iR&8i+5jtRG~-tyAaHS1THlC^J=2+-MCD*Hg-wnyJg4vBQv5`HT3*}NlzNUqpw zYJwywnGXDC0QfDcAMY^+ViWY0GYJ@Zd+(!x)~rvUC0M(Yep1TeXAL?j(WVZH=I3u3M=s{*6PYJe;X)YQ8sBW017J>I(q+H z5mnxq0tlv2d-pb?mO7&Xp{v2HeCPjG;1BuKJ`_16+fM!9@SL!Dekh-`7d{~$4~MJh z#B`%WSpQN2oKe%ti_tTcPixuVZ1)bCG@`A^vNjR&aGWAuNmqv8S{k2oCXB6<-Yr}3 zzX6+Ze_1>~e8J=Xk?>2X|Jd!{Z>4T0p+^b&&U*{8(`2tLM)`AvIWQL|p?DQef@#hi z0IFMuw>7O!q*}H)uVaucT zQ;YSXcCo1yG*r)o!$<^3RnL*PF;8{iGTD2P$I>=B%d^AhZ(00@xy*5LJvNDcs1xdU z6dy6e`KFaQ{@~_(1teUpd`le=? zO#N~BaxdT?nEf5?fsVn-l^64zOQj1<6kdqLQZuJjn(OCbpK~SmT!Pu*bEp*$5Rs^1 z_{$b1aqf0_3xYsE)}odz(6`!MHAKW*Z_ieI%0L9(dgm}8R8|@BgS4`1nkDD943c3zd z3xQZ0i^Li_Db=*iOsu7E5p2k2B3dUc12TOvjL6`^v_hy{4cN-W?@K13!qcd2s!j~K zrI4zui(RKvfmwrUw{7O`ZwbsG@eOj&9!D8WVzH(8omS|I4ufu^BwZ__N}Sy&3U`J0 z5Lv!u+Ebf}yE4ztK{NX~c#{021NM3X?BnqO*yr91`9tN1QWGOd#bxBORd6CitM}$+ zoY}~gz_1q!waqug80*A2Yu+rTXM>3YKAhad{rg6Uwx^Hq%E7@uNda;iE?a z+ki#b9_4Q-#IHI$UpYM0K%3h^Q=J&z)j#Sdy-2E$2 zI~$LX&p5vep@eY7{_H@8fLeG7P&KVR5!L`W0+B{{4L2KC2L$v*%`p61(g}a(X09-^04~=#62NVJZ?SpDeL^G1)FM z$(!oXu0`w11=28c<(o8D3palYqV(C)>5h1}a2@7V4$1&Jt$aF8V{OQnHijYn zqxB37{b7*rKGJlM+u@YgdcB6T6_?RBN-f={zIK|p&sd5lomw#1deC{fOZyyBH3xSNOu4rc3b;}dtV_R;gnCaIUTQc8=}JEd9|SDVZ$%y4Yh$~9Jsu#GL?Y7~@wqb-)6TuYX?XhONmWJ1fay6!f?>;YHYLIrlDOAVQRgkP zx`LKLeSRXX27trX_b6*9*7y>wsC0ZmtWqO@WXEXl<)nqIi?PlCfH35X+(P6J3|^MZz;#j{naI;jU9 zGr!L>+cuU!7SrGu2`uB(h8WbBp8mAm^G-hNx3CHe2XFHBbXhx?ND56K78vP&?X^S_ z)exGt1lwicLKC40nO{Pty|?B}UBepLf9yKi>Uo?3Td>;Z+LCE=HMTEY4TN)?MS7vy zuCB``R6I%_x8!z-n(m6@+c3O{x2uDT-%5aiG)Pq=E+(mn^we_lp77*EI$D+>sdvW% zVX2AqR5>KoyYQ#1C~AQowCG2e8$YjfV?DbEcr1hi+uHEMW` zvwCutiL)qP*HMv`$XR@!`}tCITUR5pe#f(o##PBmdKIS`#ZjsMyv?Py#e^!z^ZcgW zDgfZq7-N$@{Nf=|?~9z}BVyu@7SB16vvHqp*qDE(1!x?^+@Z2dB4 zjf@G6TwDGsVQyJnf{>=rt=zn2%<&z3NzgU0vvRuX!R6|t(rl<8Afqw&1IEqpJZ*-0 z*bLh93zIO!hBM|&_tcZ8D`JxM0F<^HBS>sQ=a|1o3KDzUTf zQyYSwjU|7irsA<>eOj{CViXuLoLAnVeSWpR;t2KHJWlK+5?P_Ogo!qHI%C>V%{1O7 z85+j`!XYRxyaYgB0J*cRRy553;4Zwq=Lm1qmHbx_^YcD7L#lI7jJ330TDS>Go)tmhAr9~gq55TJ;L6C=Xl8v05&FvD zdKZcnVw`>PECU_}pkc~e{9)@jI>JP%L!_RL4OJK898~Wj%^_3vo6?W3feS*p@K%Np z{=O*=>%Wm66Y%i?dQ3%8lACBoG26&mo2Al+&7vpy3hEmR0%Lx-C&xvSGt}1mrgW?g zJ(l^i_0P>gw7=zZyeFsN(B9{iwxY7-j5OVAkXBoZ$FV<$$PO2tF!j0ZVwA-C+>S9) z=_+yMy&k=hYIk!r!v%R$to^Z=60(HwX{mPj#SjY!WV&W~O>7t>jy`*VTfK)`jl)*A znDuC3{?%qCuyA4%`}4$JX$}t-C$#hv#^F>*$>Bncrf?yg>e7XVLNl-9LViObTm<*y zjR*xWgW=_TVK|rFYA6dx#@Q}nu@WRVs@OwTg5leQG5qZh#&1*cTT!6HST68iL07(4 zVYn=5PxF(vh`9x_`o0_>4}YJ0UK}s}Q;0RzA`$5K(qlY6lByy}`vZ?z)}lpKpwO}w zS1WM8Wo-)n-c1X51uozluZI^9mghNs?MKaDL@hZl3w1vMrU``${w!o2*J6~*KzqyD zw)orHvKEn9poL{^J3^=_SKbqr7x5WafC%oB1xcnGym9x)8xP0f&D7uya8UF|E9Qas z(1ADfInrJ_F{LSnRIq{>3xtZ!WTRQo)~W0%WC|UM&%xW8aj4wBwi{by7OAwsa&qAqdP@`g{qmJ&R zb{)2~D$T&IR9pA^^Y*D!; zmVbG^v+Vj=dTU?dowDm(j52(EkU7m|+Gn*5$T(&al}vJNG$a{d?%md;X&)wBW8i1iZgnVUR|MLvdx7-zm=(hE`reHh^Ek-G?nAA2Ur zR6ePWSf3t-ueuR0dRYqUV8d6$_^g?M`zF79(q42kH91^(AvN&|Ht`q~=Jg+v#w~Pw zhWt|dq?Nm<7VYvLjlrz_flPHmB_-h8wC#b$j45wSHg2U-s`g8)wudUVlKijtAA@AP z+6^Mb0F{pZfI+Aew(y6FSgTeKT`4{F8e?s`1BnBmx2)}JLJL@qF)^KwrDa<0{mfe5 zPWZmBiSI^$#-#gBygykkHq7m;9}0Eu*4lo^+aW!a=F?vPh|`jJi|nQ?()mVH*v!eI zzKBq$I_emp37oe}E}UQ9EO`(9o^E+m3EA=Tkk!z5U!s~x$8kSic2sCV4^m}+B~xX|`01O*$_7G8;OZvj8@c>} zJD_}_vHXF%Ate!T^)>EhB=d zz5)?{@U4S|?W(Q5zGPWs_Z+hXCv+^uV;i^VR@$P4-&L9eE1+W|?ugzM7a@Gtl+mtG z5c^KLgJAZar+K1${;9zW%Nt8h$f$XSyty}%L|c6#XiC;WW@v+T=sltSKT-ao9DY=eyYV& zjTBD{mJ*VwQ}#Y-J`z8G&7qadV$`GgD3vX8Iuhgcs~=;H+FT9kkEHOpviAFLjtg}W z`kl`G{*pHSN4-e=!4^`mNe^5d3+VZy#f==B`=jj`9Q|{eTklP&qU%=l|MG{!qXjFl zGPDe*(8#4}u_7L3Dm+xjnqfRxzdwy=+rj>{4pvlWBDJpmb1J$C+@I~l(hk-fGh76wi2BhS-g2nJ$bhvU>vT}970NbSpQ zaVDGh4icfiUWk_Hi&6>s@03-7DXA36q;hjkjPgXe#hf$TtT@XpF*(Cyln2W#u{pzI zm3zu9aXG`|l-tWK@j1iemGR}47CFOPDE4wo%bejYmArCGLeB66<+^f9tDNDjl>X(G z#GK)YN}qB|QqJ%sC8OMuoHIOG=~`|{$r+xav@5r?&KcfXNiMfYIm0EXv<*pq7Iln4 z%YWLYsHXOtnfoQVf5Io~o8;a{&)7f7{a<>$JWfB=>nb#V%Z%<{WK8 zpGAR-xGDho1aWhcGczRwyz9S=3jy`rSH>E4hEf1S>B#tsbT8q{$&N_ z0bX*qXLOf|6q&0g4`mg%w`*gvdgkWZ)ax=Fp8graV?`Tn`gAxilC+g`l5{|XdY3(p zdoupfRv|frauG+DkRyc<+T1p93@vFxWcJOs6PcYkk;v?O;BC#zxZkgBa{>Ew3JkA( z;0jR@$gLOl#EpOKO};^DFPCg)!7OSJklPpJW`x#PbNaI#Npq;BP>-$?`YW~C3vaUG zIFoab;KC;GZfmP;$FFQIQu&xrkCa_MHj+KVW69**hXjX&uoNH3l+lgWIdOkxo>Av(hVXi1WZ z)csnFZC;a)q)Mx9YRWUckDuq1aFnN=k2bI7SA8q3+nk$d-D=@&&ByrN+*E@N2&wEsuWlU_h|`q1gQg8D<@GM*p_`j zsR|Sbz29Cd%z?*fh_Ny7_z=>6ABF*~)HQI9%%N&1we26}i)k|b`WP+@YMiW@J zFlL)$?>hcb=4r#1GVxx(vUN@;%0#OX@~s=fS=z4G#3s5nhHlg64MFLcx_SR(hB{u-~6F>y{Du6!m67u1z zRFAbxZ|GXOL8_)(SkeYMtM!A^j||XBel3+=gN=FTrgIr}LYbDvC8W}JoRu58D^BI> z+>=*{0)G+XOeWS)Zn9ma4%pP261ZTEUa&uIlivRjVZ;}AziYRC2Cps>Rl~-Kk6{rz zmcE!xLCFNe&DYb_G8=o#s-sNv{S-^n{tGrQtiM?naqJZ5e8r*!IEwId?e)`~ypkEe z3OMul7V5Z$V?-wJ6*}$datd%^1zEM8(MCSQAHG4k7*=&5Tc=+S{AO7hi65`egvVlj z>;;c=_%R(GXYyl5c$~(M0B2>B=`m4O#{%lFpsQ6+8D5=Qs?7fO)WwwNZj3M|WGs6h z!hYK)be_DgI0kpLZA8AKFvcK`#~8b4=h4K<8S1_^vhL5Q?wMG3)#D`C?-j1exh8O* z02N*xB{k5Vj83d&V!4sX9SYp8xDkQ#T1g0CY(prf^-MR19X>36I+keZlOG_(5WmrE zxGoxl>$Eqq4IQUwX{g-~@F7|Xyh0D_Xfekk-0-Svv;M|sYngvG>l59UA2BrItXD0@ zp&fWkpSFHt+Fovh(>5Y3ItwhhpgmK`iCV%Rdq~3vu}GXb0*gR><&PT-TCZouZ89SZ z{8_^SkKbt0|u3$p3xU|nOTYY4rv~tl*v!LI7*m6mACV?6&qj;g1tz-*} zoId9}(1jhv9;s01kWc{nJ5tV)$>IA`9QqjentaS2)Lz&VAMQWQ(}7_75GRp)x*~8i|Pfe6m=_(@%2paZ_mF2~T5)$PaJhyf2&N zoz*Ht-X{dHZH?KQQ{)l7s*X#D$h?^{osVJT4?K>~7HQ#@z;x7qu}7$p^6GtntJiln zt=?M=;nlk|aBnCO=PqE?P&-kLFA0}WMzPMXQTYmez;(ou@X^JvtI%W9Pb6Q48p!`l z7erd|IfjN-YvWspcsoFwyXL^IiWLz@n7H(Az%8<#wodGd!?ax{oYZC%(hliVZUst4 z#pJRQ+-}X$noay|1AS>(2?giOG&x&Gxs)Q_gFIEK#bgH$(|I85Lr6Isth1{{hmoD? zBCCV(e)rEH8sWo4SL0xR@&Hl39O@6e)2#p9kgaAEBZ40IwBvkJ)|$A3vkRHmk%T7X zppe&|qJ7#ABJ)Rj`v%_7p@q-+tsUyIldde;a4zH;*J9V+7vFq>i}5PID0>5MHfw(; zV|Z$I!(*^HT#Sp`eIegp+#>25NxIzL>3dKgI}~4r4Gfh<27AMLt;OPyz2Q`0Z`gLB z(cV!0S0!>X1J`!EfJ9bHeBARwA}LtP9y7<;SR3ij4KTt&-^Rl&S3e1Rq{lO;h|_9g zfrlQN+KYwCC#;4w{om!Ks+L*X3I?~3Ufj4Uh{WB9jZNaq8tfS_v8xXyl#fa+v+gP% zl~!imEvxPA>fD6#k&v{rd?X}&8`4Jt&ZWSk?2$^_!nlfycHv>@yuY|;H_&d?IKN~i z_Lc0}C+tx?o29DU#vx=p)J>{7ZSx%PY}Te;j;QEQx6iZB?)ec8K{|s*;HNfMLr|$j z5201c9zWt5`*3TBi2k?2)P_HcD0Jv>iQpwtmLK-jSi|55N>r3m9`5 zTLCmqjUe*zlOQZqC(O?#sY-4vIl~iF8P(Opfw@vuteD>1Dl=9m=E{v0>VZ3@s=lJS zN;9;g%=M#yyV9J8auOPy@_@gHPb*OYtfF2=1=i68()ENV8zcS4&_J+~%L0Qd&7SW# zj`W|NuX#|46v%+&ui??z1@XUHw51^X6gsbIEUlH?4d;VFve&#g0I?U#x zSn99_^Jxw0^_U5}iup`LPh$8!We&XqHY+MaO>u23C)|ukmlV?fQ$z`}K+DUtOfE6N zroJxp`n4xx+R&vHNgX$77tdwY32flcM|2(b7AwplJ@#q0MfM+Uq}#C*wY{2D!8|NF zN2@e%=J<0v`KBt$F+Ce|(9PTBcv*>`S3XCrk3t^@kakG>rXu9&p^`4d7CaYH_Ndg} zc!n`<2hwK{FNwuPhfL1)mg*KW2@!|g$U}$d>i>Q&3CAJ6-HvZV@Y;9wmz0o1Z7SDDic(<$oC_|6A}L; z%GN;1te?*+NS7Pd{17ikI}v|mE-wfR2ZbRXlNeIfn6A2nadscs^A{;0;k>-cRkTNI z;X~R2y^1CQm79+z1vmgDS+K^qV0ic*m|?IECvEu73wS=`QP4NZQv7rVDMQ)F^>uMN z^QTzAVo{&YK=s3o{xsJSv$V1pMfR@4=Io8QqA7cZM=)V6={|WLYBJnBoy|XhGd0Bi zjX0l>%p#A#>M(5(^bsi|ApJ5zeeH~Wo#74jwLXvl3q@Uxq@9|Je_Lr5&Y_$UlRK=) zIJsFx7nEMApLH((1lW$#dCVEZWk z!WK6?LG6#>4Fvp@lXaTKeyP)jKEle6Hg#4eaB_4}R8k$snpSn%HIE4wDd=x*rs%y; zv(D9hq;9s5{B)3o^JUe{1PFX3qXoa=&2`Owa-plCFFf`!B8Zq@mEtn~?@>D6Pn57-G*3DBLlk@OCPE4Cm=(C`=4FP5FuPM3X)Huri9? zzAk4Wy*fL3c|0|m`JQAG+h?VGF|29j9b$!K{IzB>ldzlV+|6Er@!Y^XY5?%^%0+41 zRGSzEq(xJbg~?G$elxZ8z@{p3M;DY3XchyR9_6JPASEuBAf@#%c3%u{YtG;dv^$fA z7brgu<4BQ1*_U*#9?$Nd)0AD!*>#+K54^28kJ)=T`#sM74`-+2*NvE61=-EXUl<3mU&_Xau5Sz!-`%{_(UV z;U>j6F|g48l=0`6qnqI&5ufuETk-RUrilgnf;t_VZ1m@{4CY`i&%sASXbwC$MOpaW zic@Juf6T}m4l^i~DmMYr&@R0%;v#BxP6&<6VY=XQwjIa#J#&nB?LOrKcnx}V1^1{6 z_vqA6T{W#c^{6?1j87HdBFK;%&$TnNSrqUSv!>+I$NiAOmez}wP)emXQzU*tGJ_Gi zMS`JQ>{VgP##reOCdbr8YG#BkZ1GWlUSZ zD_6abKT@9d1-ob1Pd8#kmCT=vkgDjytFphy8;Dd zE2*Bz*#9u{v9J9IYd_+=8r5}$nfJ)eXd0O$7I4rk>V-upooThxLyEF|3{^cC zWh>eKk+7{~`F+v0l6>?;`QPSvtUH}Eg&8oM^Z5k^T4X%?{s(%5gjmize4guOs&F9i z+>|-Vok%yH@OU@pIXfVA6-4Scu|5r5p;n?xO(l_zDuY7@TB?ZnVS9Ah-!N_K)Nog3 zr3mqjE{kDn$v?ws{>y^K_&L$`Sx}$C^})vH(cN!US8jpo%FdfZ^S_FZ{SmUTBbOfx zX}T^W)?nZRlz`64YLTI-*A_k;CiGiP6B4!>6m@E;wYP(eY8;nX|zz7O9_e%Dwt?z z(;2(E6blDhKpAV+Po%?+n{6EfQeQT^4r-3@NLP^!`%&$B9;$@d7vCEVYrQVyA8`CQ zIv93hxEtJZaFQT~IWWZlb6qjz`rTsG%4G;vNiHGo;;9TB1D4_iq?aG~1<+<<4x!DT z1`^ur1aE7OAlm%pkF;j|y+`lIFg7r({@vY7FN8ke#{UM~Urh~_+5a>!_yNYz>Zp0F)0!rI(Z|pjjc=J9sMG0Yv zXF1_Y7L?DNBdZm~I@!eM0WRVx?Ut#{QK}b^SIM}9x7j=m$XgrjENo<(WlfM>tBRT7 zzKO0+|1Pk&)*+W-n$rtbo<#Y8Y1@?=?0-vmiHBzG5A+7!xTkKE-0oT`*4FWxve2eF&|3^mi0Yoz6NA}PmX`3T^981~pbkS06MzPR5eTs6)ZMAc>-8(HP{2kIcyYDIL&s|g?+qz2W zhOSbxwn;17C0KIWwN_GW*DykOs{!Vr$`qbq4P)QVi47wqw1 zc!!<3OW$EFLMI@*%rx{b9WNkIfPtVGV70z|Oemiw!4W^YQ>}!eY4BQBEOtxQwMNS7!N&i%wy102aKHvE|0Y39UAI=rMGy z1C&@Z(t)V+CqyY@&+nNONf1o4)R&Sfzu7rSP`A11vTM#XlhYA)BAct^+SIifG}IU# z>NO~#p|OTmYYs1c8x}L&|6nvZV34^)(koKcofiW?agG$*3WyLGxIRT3&6jem4yH)Sh-{6WTfAyvEqGJDokNTg*WJ zFPX+2%WCFhar|v=OELvw1mDV6YI#QbRwu314X9|LxyB)8nKvzjS>{a^kH{@fWD5L{ zcD^WlE~Gxrs5=feO*ZPmeR5kE01JB(k$&PaA$-;+F4s zb3-5;lD`sztjk~1mCkfo>ROdgs5Tj+E0k;{ujDO-q;}=wXq?2)5rjsjQ1$qGw(xyLUAC)b6NQx$y0tKDCmfBrazJX>Yzo3h5$OFW{AYhP5Cw0qy`6~o#t?lE}e$7Nv8d)hr8 zWb$z@tB7rW(Yk7By4b#Wh1nnA!++N0&8q)zyD7=!zk@VchFp3L%Rl=@4e>1xXUfwN;UdoyPL_;Z9jN8kS>;1&=w{^pR3+YvhB zMvy+yEQ5?Y%LqeC4;@|@INDKsvP}q1N`gLoiqRK^?+HI) z0pUmiCxS#&Ko63N-y&~qmFB?6(C$ep(3w--;jc^*H|weZUqGP0a#)>kf@t6ur-W&d z2X2W({+YUPz{n3VD^g#Aq==bi^lbza^vYBt6Xex-hySHt!Tx2LrjSbCGU-x7x)C96 zcjp9yQR`efgP9gGOQmkn_}2#sF-=BRSU-_(wW*~UqyYl7EM&m+Rjv|p*cllq!rSy} z6Y1xa&<|ZLJ(q#T9^zBiD$Fn`DY#1lap?NXNVpjFpLh0Q{pa2UWC!7GO%76?bAL<= z)0-=ZcIt}fACNPnocckwA8$#BG%^b`c%gVz>>H5-F%!k|>G%kFqcndW@f zbIJx}%cGM`&iiS}aLKhP`b{R&0&`d$XcWajq7d41CTp&Y6c(Ku(w@XLyzS)M%>28; zV3culBpCHF$T!4gTu7m>xpdfJ|YkSW%izaIDPTZM*ew-mfK65?W$*+{Yh0JJCcGGO-Q?C16h0D$2w z004@P5E43d1!Yg<>_jNMW&~#6am0AP1AAd5b#Ew6h2E1aycJZ3;Tb%RJ4#F6ve1Bn zQVjP7*tdwgHeLT7S3u6hExb9rg>gEux_x&UxA6A6F1UsBbpPGahRhfC=P8S&e7jEr z-vq#!g@hCOi1E~D-u@z{9Lk8-w-x|8in~At&n$U5rR|zXoph4JKd-V_+r$Y|aIr>m zHO~GRQ*A%nM(h51$iBi21*EE(35Zzylj*}N&>%7U5OT>yt5|AxXi`{eH*S3MQafYF zo@V6ly$GMBngu>PC0u3TuFqmw+9iV4+M^Xe98%-*Y5ivMn&M4upPoi)!bwNt+Xyos zkAb7{bCWpP5#O@V=SzJ4;~^#i){}VtM}DI_rvn+r`W~VBmZRgy2_&DqvLPszqp`0Z za@w;M=8@;l)(_{-u?M3u0(*%JKm~Ewg*4qR%+;CW}gDo@y<#Zwg0M{#~0o z13fJKj@qX8)8lb6;M}Bj$c9w(L;}&OZ3jCH#@{ zJ^5)3`sBe@pB??QL4PE7A&^LcG{AR+o$Q`fhFP9?0RxOk`JyiWpl$3_P{N>Z!7aQM zCg~D}@e_1zv#5^c9}|Rlk;EN@32=iE?9HDPbssVBR!X=CYdV`B(tClv>kMOVwEso{ z57Uq%TcI;#?c<0c<3ljyR;}YCMi=#DRr>i%VY>RpcuqdV4y4$f9rWBI($V#srG7s7 z$5O5}TzE-t^q$QppFE9R5RV{fPVQu>s$MY$0mhes9KA|hTlKJIImLP2ssY^vF zE#x{vx{~`4Msy=Pi%|1WR1rmSNG|dDT_OzyX`w!d$%f zGg1WcFXNAtw{bO6{vr61+~1*_!7m0`WY?J(>8URp&<%0bv=~cC1Lv>hIDPvrCGz6T z#YA3c@V4eP^q&~qmlh7thke~=3NOwvCi=;7MQCvzzL66@p~>99PDzc6vq+TI20FMT zlHvLelH4h@!Ps|+drTr9@rSfEV}uUI2HYB&Y9WcT9F{1DoG0Uw)v_!(R}k!1S;C^^ zt8AXvWnGMX4C9S&{VvVqU(|?EV0Gm=qd`skP8Sd^-zz7X0$&OL?PD(1Jx~UcrU-SgTuD32nlCU#^Wp5?e~VRWWN#RyXj~EPben< zO4c?6k%b#K9*KuuZnLQKa7|r@54Ebbc;>NaHL$eVcr$>Afv&4}AymgnrNhj^p6_vX zOLz^7(6lX`jh+cR(Z;`FqHFaN*b}Lg9Bl&gg-tC3SRy}u>Puxh_FGOc=|nN?qGFyl zsdSf#cb}|2Ijw%axR;?^4+a)_i38M)oWr7(3AKFxQy?TK3r&1Q!?!a;l2vL$SoZWp z=xU~G_-ZDnT~hdFrk!#kkV+Lk#uX}G1V-v3NI|zy77xFWvm`_X58s6Tn~|aE#*EA4 zo5K9lPc|J&Vwu$J)$9pw;1`&syhEsi0i66Xl~*?bxm7q?M<8P$blvJV<|>z2Sc3a? z=cW?eTtpf9R@fFPCEL*RRe^`T(_^Rn&-+^zr4Q4(jx?yF5wmiu=vFJiLFc^)bpa7vbutVbL#Vh^sA|| z8~Nr6zq@^uxF#yu%*6(4-wkf!FE`{YHdj}CX{yhX9=g9?iW$x+&|RE;fosR(g1!^v z-rq!Yfpe%%Z63Qmbe9G@zH%{b)rzGi(c!)M;f?db$jFIHZz{hs$eTW6c@xXgg(@YI zLsLf>uCkJGqgbw_W$WH=V$-RFw=L6Va`vx&@lJrHaAQWpFTVa=K}`l z67a7>P`eu=A?R)SUmbeP=5- zxlt{sKn(p@6Q}qQwgCr-zn-jXi!g5Le|L!At|eQ6wH=G|o&-g|c#h^&{zOWP2U6&{{cR6$J6 z=mfi3QbC)>=iG$a5ZJ@jcrzlat8u_K^Xc+SvbOPfa=Ny5w!tVu+1r)BU;^yfHOi4d zxlL`2ia%v9Z6un@YZlK7?j?+;$cA35pdj0uVT(uu{#U86&&fT!Da2%TYN9U`e} zMFpP@wb~jkGbjY3Hrg*3*Z|0jm6aQH;C$oyFmPUEY5q8C>MxP>tMIVLXf0&|q0CB8 z2@`3=M@*`-tTbosuMpXcBIhiAu)r*TnChjN;6ijv?nFgVYwl_4WqmCCh@ZY z1*3^hbM*2cmhp zm)l2ghi^llHOiOtvgd@Snz!hx1cU&@|uNaDHRc4i#so5=^q9*n*OTf$f1 ziMvq_Zo+Hwi14EY6*jenVU93P%9@K&94V(340h~MTF92!m%hn1<#gnJAPRh8wnfrvk<2 z*h52x?Gmtgz7pcQl?F>}99IEG+&iw_iOpinKtPq>+SRErPbq<|y3YX>qZt1#Q*`Wu z+IwH7nQX4}afO!vJX%d6_$Y>tbeJ&w=vYDu-xUV%j7KB6>~oJ?k4g~Ba|{&sAfX1C z9yxE=z*F6D-f~j`HO+et(gMi{OfAv)ntP(1%t|)--)v|SKQ>xFGRr4z_`2lGBmGRK zC$qgb-6mCSFFAw0iDusn1I#(r9@^O{dj}BzHKpAm+5BhlDCAW7+nZLQ$5;OywEj>B z-deH|eaRX2$0D<%js6;xML#?KzIs_yrdihO&A*GZZvyt-Ke9KCxQk^Ec{j3W2W--aAIO8-_u5B~$C>Qj=@sZ) zlwwh@#RSBnLhOH3yW(oJwW5|J_c1)kTs5io-shyJ&{LIMat>zyNkDnUvr@jmLdfWS z&cARbp6HV-#KLfFe9|YG&P+H_A90u)WbcHh?Yc`CX#JSo zn~y;;vLJr=7$sjNPq992pQ4o6%+>bnzX~6dRT?(3)^q!y$Ic!YofVtkAGk()pm&mq z`bDv$$Hg_PBJ*xPD8w!>k89a|$T6FmlCTDsqLn<@QKv{zUN^M1o6}5MU-+*g;vcRr z;V;AeZuI^9|I_#A`u|sZS-!~g=6}`q|91P)@3J@kqMzEE!t|XtK0fMwN!0s0QSTo` zy?-9{u0*{TMZLSD-mOvZj;Qwu(eFP-eINBcAnLuR{vI}SOI3ORHBoC~eHkFUhMAn^ z&2mjKg=E4LFtul*_>u);0tK!{{cmhAPHKV3|T;juvZFvJ8KORu^sXueUdl_FR20 z48MM|i2_b;RR3OkuYm)XQB%1I&9-%t+d3I;>mss;5^a@3Tc}b{uQ%9aKaR_ny(5MF zlAL|=k(9=`ma;cD=*0N6=`c6zS~ZP3kG8zpf@_jY&FbF*x1puIieA1+WPlgOu$VX< z+Vwl2U*5cZ@gaNnkbd-e?T1$ygLrpCCi$yyNLuF|cv)Tz!)j}DRi`)*6NbgO4)5G} zV{rnLgW7;^8i+_WqkQcf)UC$5BeQdhC|k_&dj@Itk)-nTVRZVI;f+Cu_lXvXfoK0! z|8tJb^Iy5f%u6vQlL$c9Y83yNrP3cw$dJ_N@{b0)r&@d91B`A%bYbZ>;e*RFX)*gi zMb#&Czk|_bE0J~L7^Nst-K#x2hKc@SQ|J<{PWzDWf*ukHyR}z1;SRe;S&C|EV%l+D z@__NIb57QJ!Q>QJva6M;Qq_dra>-eX1^d}pI2YD>N6YXD@Eoo58C-?8uOd^{_SEq} z3SK!PVM)hc6zaE>95cc8v6M8LU^bkuAk}K9ltMwQLboB>7^<}8NUl_jl+HWtGcnk` zRR6hn3(>K5Y4~QS!er(^c&KCQu`bi99!B5sz);@-T)5nOv(3{rBdnqq<9q_SjX$q4 z@GbXais=Db`#Xw@_O??d0vEI!17VK|#q)y7iU*mMp;%RU83ssg+*n4xgwc!7beEpu za84<&vdHpyDFbyRbjN-IrVU1{~O+?DEB<9;<%^s;IiMO^~DX zyRL(B4Vtn_RVv!g6__^`c<4#0+9qe$O7qaiazTv?lgzpP{qR<1-35zT=b5mD&1%?R zB6=$FSc~;YKmb_Um3DFksSmP`9rZ;}sl{4hR~==j6<}8v;@5U{Qn9ph!Y+=~hY=I% zRmT>aXJ5YMM$0(MSS=mk2*p!2)sZPxJ&R+*YulIMx8!JJ5HEEo5$A7pOQIgm+0~+@ zvg=GsX+B0ZX^cydy|$q9*CmLWyX8k6!(<-Ya+|(eGbC5PIJqfC7M<63rmGPAC zDDaeprhL0J?;NJq6pJZmk7_Az8^|S<8odG=oqYwtbvthL2ad5c$57UQLT{iB=5x_c zp(w!?aG$v-Du08&z0UA$YqF_AUrU-q?X*bbj}<7p2B#*=r%%rO9~RKfEKc6yX@K&lvc*2mRa}8((vsz&*%gw`#r6Ndz;f)UA&&o*+n|U>%B_iUcDRJ zY_A^eO`{jRI&_Z8K4vI;jz*7+x8~@LqwmW_KZBy>6|~PbmC(N2sdlWdGF* zJ0kjgCHJ{HCVF+zf3Nlu7464GP3KUd9q_}1z177S7}{hh>yYL1YiFC`XY%n_NA(K# z%O-IlbRO!yi0URx>iJIl<-tY_0DUF=J#NnP{Ic*I2n`hEqnEv|+b=cAV5 z2D9XT*CcENtffJz6a@oM*me}BD<(WMZAQF0Cg{Mj5HU;%Yv2nuQ*5!uyEve9{M zV7$fqqQ(yO-wv0stU~X%+7p+fFa04)%@{=?52%1HR}zex0+$NYg_(w0e>8>6c}Bp8 zG^J6uGBw3?m?8+0Z}Y=6*z&Rs0v&G%qvQC(&A1ydV1=QL1Jp(y_Wb2$;)U2fUs^HV z^FMT}qNLiqc`3j$*3P;bn8JScY;6|m^?hM7Qb6mdO7!cYJ$+e{$$yNP%L#yM9jh$G zjUA*He6p+2Y>{5vVoBd(DL!vTE%8wiG>T1ow%S0Webq@t%j~9>RnYjW$g@Hg}&KCA1j*!Js`1bRA-M}*BCstPwupnN}9JD zTcE3eKeTZ*4dhxAR~t^7WpEf(!rI8{1o&WAVJkY;%=;d}@ZMJi(+sOoORItYleNZ- zBvarvd9ZcWEG!ovQSMPH*A~m|!E*f^X1~pwoF;qPWx!^$?q=t~cF%Tb+oxSh$C)!p zj-6C%^IUB3nPn7%H_=aVc>YW9^cB$W1Pc`S0FJ#>n=2S2J@u*CBwPrGCYXb1qSY@N zV`-0WX$bOl&4fCA(xv$r_@LfJ^Dbi3up*>B#je4Pp|6l|x<0ZUZ5J#d$)guj1{Cz( z6fb*6w~)~}IZcglgXHiW(Y|fLXL&|cZ@tPYveL@hroFk97wlddp0X;sKLZ1=e=YAE=v^Z84yj1zJ%F=H8q&yN&3XJu>C>2IT2qhszUQZDCabiv zI@HNXZ5jXj@p$A0`Wt6^>K3sN!)++9s#f1{PhM7wa_bf}3Mn&_abT}V^Nw@W{DAAZ z7xB=iHAwAgum6X%`wwvTm7MMT02p1wK3XwnujA}w&OQx4YfxQOyX!v}@ZS=OcmEBU zDWJyLIqmqJY?8K=mqr`4fZug5bMIYhG?fvyiq+P=NJ%cUd!|=%P-knU9;=9^?uf&4 z8c0WN^`A)UZb|jZLD!kMN2I77<@PDp0~MSc15|JasKiH(G}Q^vv+IF3ih-&ADnwoc zHo>4rouff_pr4e<%VAFLz+L`eCCy1M&OUvXvMreXC}uk#+Ym3n)fiKFM-<389a_L= zKz&1XP#X{hQ}Kfc)Gm%np4*>6P4~=TfRGWJ7(y=W54NO{s4dqTn)Scv0xuFV-=H~s z0+R=IOkRVL&kAz{#~mu#NxRe+iNohH4#yBB=FNdt=vf=Se5Bg=B^kwjj~>uJ+ej2t zJ}k&#c?bqmd1Q(~Rf|6gjl2F@#`nI!TWBj!{-OrVrk|8m@U!MAT!8<6OzZPJXHVqp zHUM1b3&P`y9{d#k#2yb7sZ`K>8^l=?F3ZwDO0D_I%qLT0K%dp|d zKCVmzIzt&}_ZFo()Q1viw;a>{0B*&Wy;T`(QzeJm+V0872w_V=;C1LKLVMy&Fc`=- z0@g1Z7_gvT=Q5Zv)9Ex*NhV!|(};g&l6$TJ+fUJSw|x&*V-c3WL_Cc8zOw~i*#=BjByaO{vqor*5i zPn*2XFx?sYba#oG*T?$P@-EzH+jYzyVV%_6{lYYd=Nok7OZN+SdQM8s^H|sBm097v zXff_MySfzXwz;m$Fgbf5LA`e^btG$>HS1S>-))t>@QE~XV;~-HyYbF;sqCe=17z0% zyU4H2SWmm?PK-YqPMyQ4Hb?d;yQc^dXR33$&D*Y}?Ac+@{u6IsfjFp~HPz;gYhlkm zJ?nN@rvNR`fJ^IXold~pnx~2P{D2bB_fa&7gxamz{w_jfXH;H>iZO&_C0+EYMT=1< zVr&`Dqjj&K)<}By^g1L?9F=nRcNkYu_Iy37Q`R=%e#&;p-sCT3Z(c&YoPAoF-wJov zV8u79ovgAjA{J0N<{+!+aQ~`W*E(Y=JCxDQ^+K4@vMJ`W5iwACaoGd0WjDmh-bdnr zE{mnOWJQZ@>YbUgYcq<+%FqfRo&e)Sf?b_ZW>+UwPk; zVfL>iMQ}fdbxnEc@3dxRr2caDCo-58YCz%Nv^)>P+ZuEZtkt|vSgtE)kK^oxn2iSF z+Oqc}Z%~@ZA1$8Ux=L95?4<^m^-Eafb4YusOQLE0QE_yy?ZR?3bVQ7nYadax`SAU_ zaQ6N?@MBz%oH_?L!`U!w)=`V=e5~X`9e>-DEzr-o|HTITZ-htJ$~N%WZlf+(dP{uo z!@?ysB8X)LUseURyo*RMk%f^)41ub(2kEm=ByZHCx#cFDZXgQXFVLBkvizU4CVKYw zATdyg!D5=p9DkoRmRa|C7%!n?0#GRK*OQ^sqX$-E6WgxV1qULNy9UC`}!{K?nC z4(ZcquSML9op}!qt_#Hz#7!m~pT;kHEwr3ynHG*vL$PN(ZVW+EKGJ+$C&~RP&CEIq zN+Kkv&t=n0=7y6ou#NTwC-)8|D@(PT2eY}9NYgqD5<2T>7Z(mPf|raBeuC2bl*<85 z4Js@gm*jDrkHOg`9VGWJtw_7PWDCx4#E};&u+cre$9 z!CcORIr?YBB=#gEgQ6_$+V)8%l{y5dp$>{=`Lvy#LK7>w-$Hw+Pchc|q%xqrrrFhH zWx_SWa=nk}k4W}xj!6hF^*s)^r@-QZ$+(R0aM-qJgSdSOy2f8Z+(lpy24-kMhDORX< z0fJb9S$6L`0)Jt5R}NxiatQ^;T*njhImO1iL1e|nv#{$ny~@Ah5Wi0c~0D` zXrB}<)i=qyKO2H}b?Q<$P>|Vy4~i=Ay0NFWq8A0N4d*~=2W?3&9Oq9;d;idusGZJv z=%%*xs+Oh@BDnSx^TuN#t#};)2CBM-Q1ZL|$pHs?+t8N7vrT;w-2kj6J6`j4>V>CT zyYo^Elryyg!{|({5Z~i+8(a(|{gljxp6{G)#DzS}s}5BZV--&~2x#!dV+7eH@b{Xj z0Q!v)WUNcjdkfTE2`Q`a>eTugtJk?Hz>CLPTWx*4?`l?CfMg0s2q@!qK;uib;+O_< z82@@a`k7xsv&~Y+@YX=@8Mcsxd5lzA8OKJ_Uga5WLw{boSE+($p!jp~BA3G;uD_NG zJkIM*X^-_23tS*p{63N3o)FJdvaCKC&2`Pn7@PX~VY-3|PnoD?_fbRTn!KKT&!_`W~uZSHvhy4)Lwg)~|4?W?O;6tK8L?2Oh>(UDKEA}z5#Sqv*T z0qVl?!Te0OPn}dImA*}v55`KgvwsT+tqtdum!T_hbs_DXoyA8Q5_XhP(h4(SnGRaDG zI^78WMIHv7ml10H_C&;Ca2Xv2&*B92PQ^gVtm@<)^34f44Ysrur-7*aIjsDRQ04lu zx$Zrxe9nn55BMZu?d`pPD_6tE{;LTXQ}RwUJ!-F``wuB8iYT}lTA_7$ z5_|C!{8zePDs638>zd+KCjKW@Z~~T`*rdNH+~3+j#AIa9WT5oV|6619C*QwIpDYZ7 zhsfpM#37z&jg!ZSS1eQS!?7{JqNX!Z6fD%=qc!<;}nti*BYv46E@s| z=oc%cOOvdwI8BAEjX~4>o0xX}?HA*$`j`U6;v+)Eonm0-cg3vs#(wlU#oPO~gUL~6 z^LOvi1ibJ|T9oP*F>Ha9BvT+wS~=nMP^k4~_yI0CA9l*%z)<*pyy1HeC4MRP>w}c& z)xM8D=W{Y%Q@baz$z*ChPSk+?A2eb=v-Sfj2>y}GRKdNF=fA&RUpnl0;IqhD+wv&B z_J@1HFo&Fq))RV>MKR?T1fGrDKi|LIKpZy0FKR6M@jvZ_SBb6K*gRiTF^<#(*0 za4MfQNRdcY))OKCRe{HnASARFY2_k3wOmKd>?7om_R$|Apc$Pe%2Ggv%cLsD3A^gT zxsZBU@ykk*<*Jj=G0M1nZQ{p`B)~D^Lr7 z-B&;vjN80gc%YU&-v{&aVR1Xcx_l3#UT+cvOi>a+y2h@1F6qGJ*^6{ye6IV3Tiiw{ zDiMM6L`H&Ef0XIr+bE?dA*)`S@++U69HA7xK8XRPCy7tpg1VjleX?#$Uidwot|uun zd?TK&xuSV?J*U@0(N`}@<1D<-4%vB|S$E3jDw=F^J}LsWl9wvY+KJ0w!BvT%ua(+Y zshr-1J3d)E@Le){7l7x(_)H!e=&=7!doD}2!6FK5QL+%Xn8W>eA5RVulgU*y&BV@# z^zNXLNn6>GKU-yv?9Ncpouy`z!E?sqi#WC198wIc(AtU_Q0NpcnxiIP7GVXbW z==mv8fz8nK%}Ja#fWJ@B#s-LXjgg*0GepYmpq}iS!qN~hOYZG7haqPy zZ~uKh=-EsN_3?8B4ZwMgqa$~EBRfKfpu#k`g zM;bjvn8EpgMXLIYsuE%3wHN|_=U7!)tS7zRUN<)tk>&v=9&arSA=L>$cRCge19D5S z152PnD|qzJ^Cb3DZbgffBTXj@Z}LfS_i{6tXUsxySJX^q3%tZm&{~>5%q) zI|gb}KogO}TXarB^9AE_j}w>!Eeae{g1U2Yv~aF%%UKwslL4j9qSDEJYA2uV6z*h) zpZmLx&vsuPPzA@}M6bgW9n~wy%aa%VjUTdm&fwMTf;ahO_iRS`_Z5?_DFCzdmuU0@ z-e>t@m9G2ae6YV+TDi}>KQM#r2>6ujB~HofUV-!eeprWp4uuIjUIkPp((JLi`9Zef zYn=jUd>M(NW-@+$nP|{D?Zb0PA(VA8UhLt;w1;W`a^1TNqWuMwoLEYuiyz_9sRvC~ zFOgQNL?Io)StQCpJKLYgf}8C6WzH#WR9qtY`SC797GiZflh9zO=e_YbQV=0A70`z- zrn2ieaUX<>6L*sE)7bE^NIvi~(G=SbHl-=X;cQ(TLo~%RWjfKIDSc?fOL@dZRCqZ) z@=@w!{FHE6=Wea&+2-`&b^Bcld!z7@*D>v`STV{8%T{;PhsaRv909asJ(+5Dk#z4g zY}kC|@8?mle`zTAOk!V}zch}q%x(i?Seu6IeRJXxO&2gHkLuq<=v?_|Gv3J+pXtaK zy5(g)6Dxw@Z}FLQY5qN|vx1>g4`Uw?l=CSqM9=TzD-v=oc3c~Dsh}hH{H)VDbQcn( zk!BS79FLJ)$oX2c>Y45tn8ive=wivT(hMcclG`CvW!QsAjO4zK@B|)lK##=eb6F8R zmr`kWZnpxcqOVd6C#LYZ_ua4OX&;|834dCc_*o`*;b#-K8k$+sgFXh$`BBlYx)`Sg z<|R()!aj4%N*lhY6=bOY`);Nv^M!@ZO)q(Za8g(2ee>*YQbPP z=A)zDFWCQBPpMqG%2X;f7N_g+HW4{2)ELgwwfrmiUc&DaJCo!uUr7Ft_3q7XdNRLl4BFJsGK8RY zrdrX*%sp#S*BJEw)!ulj_^)h>E4j}uCaK1SKll&5kEGhvmavb5skoU3{gWh0(!|xc z@Tb|5t^Yyk6ZNgY<2u1)u+HnG1cs9DW#c`y8nh_nb zH}9jXB+CXgza>Fc9*}YVR`79XKUmePEu@BQb2X$$^BX$iY{fs-30vNH{gk+6vbmON z_uyfN+CKvocR~uoT|(Ueh7kJ*v|}*8qh$kX-eN!B=x8$OfeKQUc6yhmiD5hi*&@FE zDI_~T(Ly7ZBsdEEGNe8_=ekK5c=Yw(Eetc5t2l zWm%=v$aOn)EVOQaOaC|4t%Fc9cq?+fSv*s_J+E-_3LL;%*>FRFp-jCTeSfP?MsP zgKR;(|qf{22<3FZL`2`CV#JT2Tc zC<)LcymEgtbIyJ2CRpwF`~7`>zdw?D_q^^oGjrz5nKLtIJqWn5&3QPM{)1TfN;F`K zBcT`a7>S-`Q=aIvB9GH&O&ZZ>T0DKuz-IkBE zb0Ic6_d9Vn46@hdnXq=}0Uc`ta=T+~kHWB)oNHk1)q@Oc$L#p2347QZ;LXl zAN$UBJDWE0K=!-F9*^OScnrS=*|GQSnu53QXJo=yfGvFRt*Wn5AXGeV=gXCl%xqUa zx6@S2j&E=>HaNqs!9v?d*@O^ok945}m@1kN0*i34387h@%3+l2m5)$D(Pn+fjbh{} zA+b>_+u31fj~+8~i7f(e8=$%YF+d07&^GQ8lzZ}7sg+tODq`vZ?jDq%@lsC;>saeU(BNoOMl+KU3*ET zkb|vcL*HT}yO`=tg(h)5#Qx-Cf3E0dtg?_{Uc@>9@UgUBW?0(dN3;V=e3hdUr`9sw zfED``{;d%X5sW_QoLwiAqyi0f+Cb9MhCycgbrjQl{34Tk&Jla^lv5 z4a7+BhHGtyrRj_IE{Wqf3#s5A8OIs36*$f}$1+*Fr(i^}bR68r8QQo0iD@UYX}}D- zHZ5xtX6?#0$vdyGi%Q7Wx{6)B7p|h4{MNEaMwOBc{(eOk+}xtb#S zPPg(imJ_EijTsj0>BV(Ndm)i5r<}A^gZxGkmPvRO!xJl%GUy~$e-BDltf$3IsGMsx zOo5)|ZB;%GC7z}Q54jq~u8A!=g{EN7@Xc-IAcdLKplz-=>;rrRrURx3n!}z(6a^-t zS8t(Lt|0KsPZJi^hGG?j?z|G`&+NQVtA3&pA`5S;X4pds$PeFo2aZGsV$<| z3E1G9bNW)wdfL0k=fv&Zzg-WMIsThWOFS{>hR$g-*1Sj#2hxoBGEt*J%Q|yorMO(Q zx9~Dx;c0Dn=#3IAX<{#imep3XW(!y7w<_c23sY%fn4Z`jorr^R3Hz} zYcIP~6WS^H!2N8h|Et4sz`=cRA>ekdaf~>D1o&BbaDa_ff3B(H#n~a6UTYkOJ5GK01@?hroXU3>A$wSqg<4ZFUzElcz!NqpV;y>sjFFgZ*Sd2b z@6J$uuf_{W7)XTTLAAg~OKav{j4i&YCm&+TLhv_9J#;05QEJ*Ipw!pC&NOMjVz;*< z8ww$P)>)GDCB#g!&%i1j`+Yjz$@5uuPx$p;Kxlg#IGnP3OB~t)W!&k9UD}SYzoBu`hLhUkaMq{c{x7Y&z&mX* zQIfgrb6ECIX*k+H(mj<}ZtmBphL~fOL+qe)2yK%N zp}c<`%Jd7lybrJYNsxDwynhYmEimN$MwYh{Z!)|Z^B!Y)YvHwoy+&$7s8bzGS0FIM z)_lg+7iqltm7Rl*~TAM|BX;uW zWa0f$DZIVIv{F8WOQc*j$T@8EAR**ch_x%{=#EIC@_`r`S{Ka-(y>YzHi1)?;dIiw z=xfb3poZqzD;>b8zMoI^{TeHu7DM}j#x6}~AE&HF6T0P%I6br)z_@RO8nJufA6Ndz zo<1A5nmTBm#j6ftlg{X+^l4eYfTm>(X$CdnN0i&wv0>%Vu-KQp5(x_^a%uB)>)|d{ zc)t^6?)P?~E~rlV$3K!V)QWb=n-F@^!ofZCohR`=&}!v(=|sl}?ONzfENF?N>x-zf zbXiX5!GRLNW#OG!q@VLUQYaxz(|A44fav3^Cu{+3ms{p~+8ScS9y zL`){~?_`~S$M*a)j$|n6$66PX%%D@MPQA>D2&66=puB!9d2S3>CjJAZ_|g1&nRF(+ zf8`&7Bg^;)9zuud08C;xtldt7ewl3xP;{qOrGv_E{J9p!II+{}VUm;go7m;i6kWkW z^Y)L)oIBnS-+#s(nW0q9Bh^3^y$>bo&Kn;`7t??6<%q>IgGI^T2QTf|V6}5-DwD6o z5Gtw__8f%d)ZzCV4qJWkI*i2|@pRDawB?#jmU-FLWSRGElq~Z?t8*>uO0h{R%eJhb z?5!+&EiBmUs_}sH;IGhEc`1BtBh0{gz-<1|qN*p3s_NbF7X^a85z;RDR#mTn1w9|~ zzJT{9;QhC%**%Z>JB#I`^Tl9i&fFZdK3;}>Pl@6QY3gwJ@OQ-P*KAtkRNqX)Q&1jV z7g^+vgOq1i33=VP+MkTVCks~45rXowwT)fz4Uk zT|msX4k(=Ni9unu3#ohcuX44|leYb1-rXnofxfcE!Z(BWQ->C;kloLeE{Q-RkETiV z6uFz1yzCxq;j=z$dauGY&;1sz;H$1e74uts8J?*v!|t~S8!mir!@|3YLO8R{rt;4% zYz>}71`Kam{JprxN{SGAh}kbOV-{7p$if{XRAm6Z$`ekXhD$wK}Ce(X0q$*n9G1$~AY-*}aq~<1gyH^n0<6WzS z5D7jN=WuyssdUJl`pR^J6e|PgP=0!tCf93kCz`ucq|&o{6S=~=N6DID*oNHh($<5? zc>nL-%x>MOY(IR|cc9>D{~=&IYLW3)c70`!M!CtB2l)MxfAe9ejFeV*IHcXfsVW{VE6c z;^=-ui+}iK8PiA9oqK{UIpqdZtzGE zuXJ45ygP|_2%;E(P8#(kJ80Anmpt(qa(NG@`XB3Uaq^&g$6!QW>FZmaw>fWD&Sxk>%kgS^(&*XKGvhqXl#UD*u%*;2y0cL)EVZM_8iDymH;wd(!fi9t$fak4r4_F5v8Ud5!+Q5D~g3&OkqTS zm}1pn^eWlCQpxsZ8Mmg8E5qq+S=4pK34`qB9iF&ecUpJ|QF8j0mQ6veF$OC`6Q?Ej z=;Wmwsz++CMVY<+O?X|bqT%Um5tzrE54wjYK%+usYcrV$mfd4o?x!VlKk3f+)t*IV z#mmO`<2+@o-Xs4Or=jrv2|D~19_Nll=nrI}bH0>8!Sz#^9DY6RyDI1;TzZi^T{A)P zh}y_bz(uw{64itg@MUw@(e(PJEQ`Am1dErFm^%5wk8zlu?&@IPRC0Us{yU&QdHS2OYnV zjKEs^Vy@m=!~3}2tw-^|`9TtIP})sP_9@oN65a~TRBY%Tw(%a7B86_?`<9TaOT`hw z0DWxh%MLL@S_jcIS0 zx%YaEjw-K4{L9O*E90HFVert~o#U1A*%Xfhu=XOB!Yen)JKM!PFOh^o>tVf*k;}GR zQNhTjjNzp$o@~k(*_0)c&E3CZWb@QkA{+lBjAY7sAem#oFi6JANCqkVT`NqG0aHx^ zz&`Y+y}^lEZkSbcRj9~1%)a^cEcVUb@J({@IWWak8v}YO9n+ClR&-nMFdhwCINqFM z3jJ`cS6<}M)(<^~#u1E-ajuE!%3LI#zDY7&_<@;-^4n135FTHE3kN5q(Q|Oo$kG=L zQ6igl=Oi^y1xrWSX@eL~5Y(Y&c<9*Gbo|1NpOG2%%V+9@E!5IO6+ZdVBCq^}10&_@ zo%>d~hj)%VT^83y_>t=eTiI&HJGEgpb}HM#t&aemrcO;%UO%OE%Kw;G9tnHh^(?gQ zXuwoOGVzqrnyoM1Q5f-$Z(qXhJe8TfY@ocj0lj?Db|irJTm1YzGef+J6clLs*m5r+ ztmrOd{p+WUxi6RD@j@dx5bG8~rx;@!vQKP>PpI#9|HQwL=ic=>@!U`EG`{g}%kR%) z&#KeIi+6rMdiZ851Noh1dU$;$XLQ52bR!AhuMFt-KoURhG)N-Ns*}V77#&9h(nIC;{(XEdj?wM;~x15udz`u-s^|7)m%&=1@R-xQ~+_+T(#dT#g z?;F}W`{ad1)G;ku5ycPrl=Z5fI9Xi<@6@qP@tqn%onnWLYC@-Aya z&@yy;k=D6~sdF95+TJWKxru_`Wdyu5o`4M_6r^A`MKX}X2wEb+=|F;W-|o(b(9N%9 zB=I!&?FJ2xQl?Eqlr3ZYl}`oqpc~vnBedK8^;RNdb)rGW>$-`wJ;%?VFy6_-R*fza z!&POZGFgtoT=BM0yYND_D7czk!kSm?D8MNx8@fmzeZ$LqD-%v_dQe-<*nX76s{_sn zBog^g=zzYILr8T=$3>~xC+*j64~YF5&`}&Q-3i;neh&8k6HSwC8X)^j@oYD$-rnl& zTJ{KnW$qRNi!#9^SEJvQ=KgEMU6amv<@ZsG?Ue4$)>--jgL8H%7j2@A#$>U45A0&? z$TjEI)2AjrUTCw*qLmd~7ys&+U9@{eP*UPp4%RlP4)}w9>E&dcMz)`Fq3Wkl9hL zCId$~^^x0jjQF&ZIG?o9BYgq1RD3UJTO3n+(T5js(3`xFMNGc%D4$S0Lkg1nwtcJB2`ZR%?5**L~- zALebKZ>5|%`Bt-Dqkt(GN8Z{^_n+VbtcZ7#a)007NV)$bdgSbZ} z6N^D*FKT8MZ>+Z@XzJMxNeZ#~*y5FFo9Si1$F)BXzKn4zw~@4PSV^g6xI09<{HJJE z%XU$orD04t$V+UX5$*%I@*M2MXu1~e=7nILrSE|F@h&g^ED?j|4-7Q645yy*9%cLe zOcPWAJ5XFX;c*XpuyZ)t;feA%nBmoS7oWA7f4pxgbQ~3p=wX4zsLs`N8_wk-@ZFAG zK5Li9T5Gtj;`Lr9@5rUQalX+8wLXIC?$n>$Vq%A{;klG+a;~L&{LsZmOK56+=H>2T z5*>?CDCHBPM0Uy0+`gN2q`QmTy;_IEpLUt1%xRo5pK&OaTDfa4jTjz9?BQS!JD&}1 z@|9<>T)l$Kv>Mnl=#vy+irv`e36Z%*%R*f#Ll z#u(IIyoYS!IN(mbjv8VXg%Dv?@1NeApNlNR>Keah^@v-4(V|c8M>)5Cql;4n3W8tr z14ge*aP`Vz1m!s8n+$_%--x#z$*E8J%(HqJg`-b>$Vdznpd;bpjV19XdaRt=RC$*& z?e>2gUt=v%`=_F`9oQ`rJDb+$ygG(qr)%RXqPHVBv+ID*=cBk7Ih=vc`DC96uVOHg z%a6>;vSRdEic%p7J3i&Mi?a=L>l8rgWme_4*<6+4sZbgW{*a*I&;z{PZ! zwie~+8H-n-m4@0uKbYx?mHas!TJS6H1{B0F&aY58gsQqDW-~U@yE)K_^zw`8v<@}u zu;(C+N{W9Gs=9+~@j%PspY3c9k->t0CPvhCaVHOFM+EIg1LH+`X&J2>zIt)kb7+Yg z7>?HwUoFb3O0*351KLfgn~L`cYqBr>S@wXyMPli0j7#B93LJGVu16!^87_(1_5|0X zv~2NT=8;!3GuY9(kEg0!_S-HsX3vk09H1(%-^q+)uDi%$kte=TSj|@zY3Wy7 z?GRo@2LVZpluJY|d`Rwkv5sCXr1=Ag+L+o=OQWN=FD^h6l?(9rcIS1mfA+?>FChv) zQx+IO3P$E`Sq4R|JhaNrKQPtu7u5A*%9U%Gy#z&)Aq1geUhFL^)2M%tW6?~WSWvi$ z%ehfMTD;b4wGeea_PDo8x)I2QA&c zS;DjM?mccFqkr~|eD>MbEgZJ;O7ct@zu{WbcW>w44aI$z;)(95CwBEPQ36Y^i^QLYjYS3uOlCZ>A+fEBq;!7sS8ATHMxGWa|$fIYFw_xS@k0NY%-V? zA&fTmW^MeQd>+Uquf5tTgcjM!ybsIFz!@d4-bI{^#T>tTUxr2bpd!ga0Zi8_H$~8P zcq3*7_wh6G8>t0%{f#9~v;(gZEEYU&dJp}T!c&)fwE&EyP73d5h<-blESJejqjWnr z73O=N`*)mEXShoD&&BgQMdrU|Ul4qvzl~1z1UFv9S1*Jf$z|Wbawqmue*(Xay+Qs( zQ{gXUj%#_@OFSk|Z{6(!GQ}Om?P92s47laV=*bF6QSoDz_Q68Oj?tH`7Qm=mmT|WPn$GRQok9Bv3Hrnj&)mCA(4SF@^ zCr1-64xH$3s(WMciFwW6bVrbT^DGvALd#kF)=UP3RhgW2@shiP%JRMzX2z4Hku;UQ z_jSFg3-U}pC9K;i-Mf7XT{F{pw~+@58<-1Jl11wYkJ1WX1rE*|e>NRBb8<3cn{Rx| z2>9`I&E0`)FDFj!ZBfcLvAnyPAFMY#eK~I3F05h6qtnR+ei@sYM?|(4Q!=R*XEH)% z0r@DeM=KCzINh4rsxhM?{_Rv|SV?;g%D9p<;G=1AbH1?`11DY{q{nm*vDsLwJjuKD zKICq=nlWCQj}4ny#($^rwOY$q#WO-^dj9?#yOzG3r$2-ht}|6w#4{!&wlsrh45OB) zw_4U%p5;i>aH7my$xJzO(hMxmB%@2T%dQ9`<=c?LnYkt119u=bk*Hw?9>)e!abCq7 z=^7avQ|7&I_Bq@{{+|;a1jwOYMC_|ynhU&^%eX|_5p(Kqa@p)o^g$%E(lz_cMQ7$( z;`_DBrcJhVeFQqyjzp)_TdDQw7E{dX+dj--(nk`fLTNAKg}V*X9ePStiFC0-8>bQn zc`D&)Yp~8v_X{ZPSF#;QDrNaGtJbbX3Q%U&v5H4JRTfp?#UiHOf%Smpd8JDu8p%~EiYCep%dOrS6o!T@Z>`~roDs|KiG&#$zNf+yQwOEHiq(A+LrG3$% z*&Ne6{P-`7P&ag-^)PW{Tc9^z?m5lA)WI~cFLQ?fJT@y&jS%gaER2CagvrJEaqjkg ze+Tvr@NGMvv^knJjcikK4dOCGd=D?-J?u}7nT6lOCoyHBO@14;ESltQxqV2%(62_@2k zwK~T|mv*{RMWT%d;vdC}b%px8QhsY=(EMo@Bc(@oWRQ-HnQ6|0U!rvvAdJ+W$zi&L zwiI@BM}Z4zf@T&S(=bnqoQ2Cl(A>W|6MJ@T?2a_^Qkp20l;#*7=Lp8wOZvKZX^ti> zEG22->#X2GD~I;a|Hk@!I@=IC%r4Zp*Htdr!jTTMl`Fk8RMII zSXF7ygNc9OVv%GsoI2x|-6zp^bt$iV&8j$O^r_k086B&jVeS%L$};42y`>UtX%x1^ za7X^}?`IeV`eA_|Woz=E+XmftJ-s4gF%}LTu>R>r@P!bgn_4COXU!XHbn!N4`^F^U zg=&O?_PxW(oWiq zG6Q*9^xn=l9OIKM3BfmhF{Xz=4|&OWC4Ib4SCDd!ktC9H`URMSQ~n-rpL`_tHxkdz zbgRYVf^EcFSH#td`ICG2%B5^e`2v=2e!{yaonnqZhC_*oJ%klQtGpP7BVEUY8FmnXb8TqpEg(|4-sFZc=o8gImxpr=kG4cot8ftTChdReFZwU%$;F}9&n z&wh0Z!FA;7z;p{Ewz3WTpFMRUlnfa>rmi)S(pn;=BsTHJK9`Gk5XQ{`X#9au=hQ?V zYsWEN8Xv@m>7uu?NvSS{w(fl^)1dM%Q!&0@D!=uno(rPduy=6f9opc+5Vesns%9e} z&NUZrf5HYj_a_wZmH(RUkVp0pcG~>;!A_8h*09M{4;4khRl~lZkBfSa|XT_TolJ zStN$T$TEbd{-vm@`HMX%w*9F}+p;b-cqUm`@}Y%HS;9G&Bf~6DS&VBps+i@e_3D0H z^7kr3=Ha=Kd%1Rk0NXL<7#q_4V*hjm{r#3*xx|QYvP27Tf-2G*eBoC$1F^j?p$l2cAd(~U=;-KAMtqYpu0efx zSgtN%0_*fJd)8JfBXpRd82aW2i+q;KPd5g=jx@cR(R8Ph`XY1J{qdjCUH2BvT{qxm zT7~w}%vJZ>%_^h$z(hJ&j6WJ#>SXTx3ET|}#>ixv9kXZ-w+P)AwFbGOo`X+#pAKd^ zqAaKpn*3`@O6J)TY_QIds7%X(^MwhidkkjMHLt~m2|=ZBtn8fHGB;jiqHvEClLRV0 zCqu;(D-+|3yo2?IW3QKNL6$AB?B`kbwU~Vfv+rZsFH`!^IV}4Xc&xh%$=S`aXS3`H zEIaTxWTUO7a>e!NySf&pUQ4U0-q@n5PkpVbC%#eD_m0Cz;5i8XmZP^iqz{1SE%0oD zHT_K}e>2edw`X8ea8^|xfWMO+=k;gpMgFWG-f8k@T_v5*pOx-f7o|pKQstGeuT8U{ zUk+W*fiq$JDKeWI!2_h%x{qLp>cb$>G=h+H1camrqVS^(S|@I)Sa8_lpTe)z-Gw=q zZ9Y7{ZwbnK z`;cGF;s^YtucBfwonl+3Fjhiau5r$ftbEJ$U%XLyRfIMe3zjM)-)ADud_8NPm1h<3 ztb2I^u4=eqm%Wi;zNUhk5ba|V?7DYZ<~2qo(g|k0$js^q&J#w zFnNkE8v86-V}Vsllz)Dc#W)ZTJhFlBB$dDa{w!BCu2S0HV@3bO>aQwpR}S%{XDEr^ za-b?_pTqe1E@937mVgszc$GEw8*Kk0#%!R!b?!!O@t@T)hi?723cLS2HHeoh;Ef;5 zoBmSjA|9|KKmVQ%Dw6${;%N&?MTW|l3rDt@bt<)WiwvKhDt*aI=+WOo~Thn<0~l&_#x6O>+(f1+rgzic?Tatf7-fyi*BdN)TDGv z%SDOzy|TQ4>(VT)q5yQUHLBQ(+_m&#FLHlEFZo68jba$Ze;8r1Q?13zwms_Bz%$P9 zltHK`U*!$Yw0U8XHT6f+UPFi4Ci4zmOrALH^wCeP(4h;B4w^$R7G)mJdw*xwohO+;IN>SpUQR-1j~E(=@4v{=8v1Z-0KZ z_CM&)h;aYEaCKT-mwLdLv#0*_KYxD?y#JrU=ao44zz%1ni9SvJdFS>X`tyvMcE0}H z`h)xPX<#YJ=X8I)WOWAXY$%A!Im0hEO!@^7%Sz1F-(#*Z8-*tF2hk(#K>N?j%&b9) zXAOlEH@rsLt#Jm7K#8s|p>`Pd(<5MSCb(owCIJU;$s>DOQ)<)dG&BKRYcL(W$(59-Y_;+f ze#$84!TQ$l6ug)w5cv*-n@DMf)(bnM7yR5dWR1Oy-?!oC4H^Lo=~Oqv$OC`W)=KwU z1RQ$uhaZf#hWm%k;=&4@FwyV#>(7qy-TN~lp+D8{{lNZADLWV68MF`ke*GEMLx28m zjqlIkAKV{H*MHidHMe!|&t(bydFkEr^#_S4>_iLA<&0kZ^{vHwyy5A<`Tmt)&y6~% za7upId!bt7tCOP9n61cHi%)xz?-Q+KDSLUxF3~#HkG!!CV@D)pk$t)Uwvn6^-#^Y32^Y_Nf zTPt@_{x+djmNh;)b76mSc6kolVWua20W>sIIEI8YBfj{ z&fsAf&uOa|zD*5$;g#L6?LTc7o}}nV;SsF>^i^VTKghYW?4pD>lmdx`T}gKGEt4Kv zDN4Q-@!#`G{Qj~n9%-j2ZK7}^z`Yv1^2B;Ti%;6`liGweBLy{5HBtad6xNLGWsO#i z?FFcZ^c-t_)yN!{o{j0*EIkX;vsij2rf0JB3{1~p={8KavGjCIPiN_An4ZScQ!zc2 zrKeze3QJGM^kkNvgy~5v-Ad=6E2>6XAgH(-(_u^IrQTcaL6ijR+(D~{Ke4hjO z5OC7L|9^o?2UYN_V-C<>n^%Sr$x|E2;B+>rOd65wgwU^1mH95CHWf-*qqDcdlIqZE z2%+oAhpNpJPRSC(d6}N@kUpO9{2U9crhP45YlGLiNi5yv6rPWWr5(caHDYNay|)R^ zN1c_sVEKXu9Q8uz=cK6J59GWb$a%jgwZR&lDymUw6W-B>!O`$iGIpU?;TAFMAFLKP zxTKDF`j*Z(1-t7K3;g$9Vu$~E7YdUaG3`RTFe&O3a`P^Rf3Az|;c1TVTQpc#7L1mO z!P=z2pU5EJWo^T9IaD`?>Spk|8Cu;etea)`ly3CpcApM8ev@dN3?p*2C&hu zU(-3{2vyqRe7b+JbMTa*7H3ue>(nUXz+aLuzD>X>`sYHazF4YH_*Sct5R4{ani!09 z1$MZthuzZk3G5KDEghuK_QOsrDQ+MM>uNm1x|MdLH$ERC#pa*sk@H~)^`5xj^<74K z&p9mnNtV4Dv!`J8PgwQ@mR-)W-@@#vm@QnU`^O>(UAVm+4rulNEO;tu&II7e2gdfZ z$Ym+PQ`SHZhSpCC^qCs_=wzCO{m7mg8-6+scAnB3_`l{2Tt066b$~wiU-ZcN03M_i zS5^$9-aNvx=dtY9F?$4NFCJ*R&l>l?V~zdoV=A`)n|=EJ7T1A%ue8}ITP62ND7^%o zOr?}T-mveaauKvAyH83WXKu{DE8d|s(fhcXItD59NZDyXd%$Ho?XwSif*V`W0oxO-v%8DGf$s}*-|@=RS9z~S z6Zr)z#N-7L4C_U_WWh=hW&^pgm=)Cm9VppWen|&FU2$C?rUe5&Mm~FO?NKXZA)-_- z0_6xHGy{qK(aMGhJS%36;)3?%Pq7fNE+KRcv8))dEWs1b%LbO!&lgU~@Pu#36|K#p zb+=gBStdMR=P7Mve9PciuF5(gbkxE)7W%)ya_Iq%`f*Pj>zH}JD|G_XfR!Kk zh}8~kBpKKUaFJw&??B%K!4 zWcP6n&dzmKW%mo;Vinc^CmfuY5v&8=lqV2x+U>Uf6VdLL3KRGfXGf!u9o^5^(Q}Jw z3a_9?&L1jZ3e!=-hD&n!`NsHhFgaZ^(5E&*4tTLor6c-3q4^0M1KzjFKD%2k29Zu!R)gbM7IEwf zHTZAzN(&+w!2Ur08u)VupiL%WA*C9e3WeG7DJa3`2qHY7=r$iouYGwe*i}S&alyr z{`??oL7#-VuYwS^5`?f-oHMM&45<$IMI%Yy!;;xaj`{OY5qN=DI_1KEiMxpxtfN!t zO^g?y(E-_D!C;yA0>y+Umn3PdU;zcLYKJiog)-UKd%_b?bh%-OC)_{VYu(|sHtEvK zPIyGoY7~F$5T36VOSe+WUJz^CmB0`%c&h_sB6O&9Gz*~tBs6w@mq>F%%6E%2OG%zd zxd^B4MJQ)XxWvK*jZ63po`4cet_vlY+`Rty+MmldKe(ESdn`fjaXpiBQZlifOll`b zYbQr*Cx_a}p?0$IH5+7}I-&9@66j8eg;8fXC5_leIeKH}YXW8`P~xk!C0q~z3T&mA zhrt6tjkf{;P6TT70X2HFku)3maG;v@urM`LOrqTGW! z2qMhSY1msCVYcHUbv3nx#7n>#bxu{KTIbU!{tfOw7@(j!tC%b}M2!%O{VA*$QJnZW zSR@m&b1*wQELx4!U?fS;fc;Ka;8VA?3kMNfPphI+n+bb_C$G^L#mi_>Y-eXLDT84$ z6f=2#{!oiA43e(by4P!M_LfH2G{D(V-F-HAOKV~2taVp5387h5HXYEDx6IK7@DV~E z(wbR+{xuWfbv{X-DbU8y8^r;^~E4F9I!I#?}C+6?rUk_7*}Nl1?quw3qxRK9%v z;A-ai5Zj=mRl{lJHMAT)2556XopPP?=?OzEuA*vK<_5c~fTM7%jfz|Et5IC*WLtvilVL10cg%ZSG?g z#sr)MODPrArgYraB@023WdcPef+Ewd&+aKXARk19%E#!StO3M>k=m3l#OBGYK73>S zJX)!D(j%vBo+gXuU4X)N&+BczC~arKKIBpMVI%B9Xg`#a+@Cw;(Y9eDZ64`MpWr>_ z6})FW<)dw$@~Jjv33rc`R9%OaJ2RLa%EeZ+{hMJ!=f+2^WK$2^;ChdI6Ixg`i-nPv zq!~*^Ic0~ZPYO)nl$~PuHmj&M1X{!kBf|QOawBX_*2ddSE>!0Q$yCDZ4%8qipf} zFTB1m6j_*!QmZF8e-jMms8hPxMja0|7$cH>pF_HxHgU%MT57T{@o3JP70wS-vI-vH zE&qkhEC1A6zR>0_J?lc|G-4yb^+9iWV{xRo0nfAYHhbh7wusUWv9L=l-YM0HAdHB` zJH(7GZ*iSCuv3h74#(fouHn-0j%~mP2iAxg$Gye%;=o2RV;68oZ)wx))$oNm7x|62CwTCzC;a$X z5jIrOZ=8vFU1ix z>*H_M$KOojZ>I4#$Bn-^Zv4&q_?r{Quj2U6z=YcaL0c%E7s!xm$H+~#O@fslz%PA4dN1Em+pPzy#_rWPDvMCyq z#%eRvZ|8e99#DL0P2EVEu-&$D-Szk5U}cd<*@lg%OaTUHsVm z-yp}mk2vn6?i?4^zyGiB+l1)w=Zk0Go`}EWunWM?CgN|r)r`MiB;fC-9(nlx0Dl(% z3col0b|>Jk`MdGAAQ68;BK{^c=Mxfvc_99p5r4Z8f6W^Hb|e0pHT>-c{0(UqgF8(4 z8?qZ%1K@9$8Gp^}rp1If{XOe1L0T&xoy}})#L`wF_%6-f6XN17SP?C=F)Gt<@4@gw z?c_OD+`euG3JPmcu^&TAD?nWP=)9!BO=9?lFR^{)m_Li|`Wgc}C{Ot$8w#j`7(JK_ z6ON*A=b*LEaHRvv`J^kI(zNWB;qaZG#DC|P^qpT=d5|M}d@i$*#)VQ5=4YY;C6Pbj zQZWSY<^tLf67V@jPN^N!C#8Cflv3vlmzn7USyGSuM=Za^X7D#APl<9rvpgk@264)t z1k7wS%?9jCMI8N47|&siYJ#DpB0h=w*z##M3KzeS49wHT;2A5s!^A6$t^vA9nUhi6 zP`oXcOu}9iHq>Wfivr8{X)*dWEZ^1A4j^4x_VZ~)QrXhoE`;zp)ELegX1QGm6Jo+m zs!<5(0hW!tPHpk&v}?;%rvsO(&LUi{I?K$Q(Eu8IPv^CD-RYa*swOwK8fTt#u;2>xA^Bc zIpBHtzGcZ6rn43o2HNLuB=Q;uk?+p66NYjAQ2Nvw z9W%25Gp%A&0n9|CI>Jm>_MAKdVLQi6;EC8v7F~N$D~4Uyt40Bo_ww2_Xtn9^Z6o-$ z5q#U?;M*1l-!>h-Z5+O}0N;oKUuZKCzT-6b+F6are*j-^>89ByfkR@X;Yc_A`R(H| zvMoTr02XXEa%rCuJw=3gl_syr$zG|5K-!YQ(~2X~NG2NG<-8mDDLs0;j|=$W$2vd! zDQUCYF}AlhW7$RE*@JqRvKfTXq<=rh^5ZZmgm7t&y%INLrVKD)6Xx^(2|jB~^QAjJ z|5xYBRnFl2LBJ62z?+^qN2CDsb57xP+mJ^5tKOpXyCaIyL5RJS1$`*OQpDwq;=d(s z!@MG?8cYPKn)G)az=anI;ki~=3Y$7DXxifc8m7$vQO@<29-q_CQ@Fz`73*#BkYJ-l zx!7CSE=t971{H5JpEt~JvWeljAaeQP1x5yN`?@8mc$-cxfh@JYv@wwClqL*914a`= z=zczH=%a!Q?G|~29XJZ!1M)(|KhGzfZOO)8fk*J#JRo_dx!HA$Vn0Hus%uHhOWK>; zl9)jHu$5_V*!j&5)ZfHTLhY)k0 zGIi81NVvWePi_;{QHY=a&Hd5AMD?1kzsttKc-UZ$CJx3e-EG$-^Zd9yQdzhC(M}Qz zyW|lzm;5l0!@o*f7hd(~)!uU07>%;}N*jH`sA@o*r?3@yp%Cn2WqzHYt#; z)deRR&8)_+qYWDh_nz|4Kz{JZd4pj8RIK_6_q14-dgcD7y%+cI^h*7^@Zxi#7vF_@ zoWVm@w+xKHUHXX-M)yKrd81FhVUSP0!c%_EB}H9@F^_c0)6wL!Hi1+MRI`z&2IP}n zW_>`7M;@Y555>aWKt5;jh(WDG9nC>^@gA{suTyv)B;8ZVLa<5`f{ORV(vy(q?9n^% z%oB)kq<&;xurJs>o}AM7w%|Ubnzd($xtUjnF=r7)acewkx_)ozrs)!JV}oV|*q7N8 zVHk}5RhVCeQIC`jJ+Jo2F4b1tAQm=yB-dG+@O%_g>{z58Q|wg6tLEeOqsb$A&*tNJ zgLNt1(w+0)aS3a7A!u{eNHIWXK=Z z6P~cEIbPrNedUjOTCYUw@i>2gtQ01HY{T^mNYsZ{9)@#rZtIRr4E?HH{ctAp73_IW z%jCalOz7}Q&7Q(G;i;>U0OWp%$GPdn4Y72qc1m^D6D~W8I*n*72~VlE5`#5lJZ|Pnp|26a=|NYzj^Tj`G|BMpeEr=2(rwf(j z;M{|(&P2Ni;j2d>-6>T&g9n+I&M+21JfL-M2*DG3)g-EW@SxEsG(+DHMK$0@b@?tz z@?DhVyJ(zz7mbteqPlz+yG?}wldB(F#9o{ruBMqc-*)#sfr-Q>FzU0(ly2|y8l zs&tDT&kE_$UBB}^^cCIo6XVG&mX-;aC=zh=(<$sY-xt31tPd?sj-2&{r={E+_Nxwz zvVMZwk3__d%SCGxmrv0u&_imXb)YOtz+|Y6p2DO0}uISbkq%!{i%ACYc&T}qxV_?hV|J=SV{ z{COZUyDmlXZxzd3#XI4VK)~}3uwgcu@~maqYt5A>46tK11?6Vh5p4)iHD7}xG({`z z5)5+UD?H>6036%Bk_xP|#RffWxzCL^d8M()C)iFs|BP4a$7);sWuiQaDQl@jqh4Z? zjTK4qUjjw2ue(^0T}BZ*6!A)py40Q^Pa#lu5aDu&KFg_eyf^GTE3QQ!6pY^C9T@R} zY!N&JFdm!Y4NtJxNS})E?br|E?=(@GFwMq5iV3c1c1+1GmV7@s8yX7z?NVhmxVnK0(CvlXHxD;$XKQ1Mo4abz?V8bz` z6wq+A1h0hjFsg`#6Azn$Zjf+vgH}H4LIV68-C$f`PQlR?#X+=)C6**Y6ss0>gJ?lF zs5|0P9EmWdp~T~%s2i?I7_}LAC}D~B+7K=bUfZ>*_PCS+Ga__IIpR|C85B$@b_NDh z3J$l#NCDiG@p2_d3;?k0CPWv-r8smbZ#NLlF`=at>DbazN_32FXVfrB9dUA znZBy?yLd?1blT-9X*zjpU`gb6+q?0*?cMm@_C$WS-Nf(mjShEXZ#CW6o1P-UlEWEQ z0y|4!Rh$pS@smcz8ek^g6(_&3waASID$pam0*507S-rrPr7H;J9$aFYEXEry*?iq1 z&_qp)3o022#+#R*ISLlcE3{8-r%xGgRC$K2$2{qle7+_^UunGh)WAAMN?e|UI@3MN z%K%!y2N>nhGu~KTY`4ce{HDF9JQ!sG%ir(j)Y{}R+Or!ead`#oTRr9#(HOdCF@8U) zgaFF;JsY<1MwK~0yXO^ffwiYR>}-CId5obubj%aQY}eY$H&WvAiVT2t%PTPe9iIol zEaUlhPKVodI@CE7`)ZGQ*ds?zd7MEPXlSa512@*G(cx{%D`DU2p66x!81o(9jf{fx z0Ch6U-)|;I-eSRd2<;x8Uld1aaRf4x71^(SN@KUT%AKe$ZoY41N$# z_V)9WufY$_2QmA3$=Bcq2@uOa-?a6TNXVw{`5~sw-Z+RALEVz>>(jW8^HJ4QG=Z5f zT#n%!>&nhcN(-ups`PcDTdUNG-vc>4(WaW8Y*uUK>@S0dtZ^s{L+Yilw`FSg7(jHq>5E{*un4hKuFn$%JH*N4^QINJRNM zWkib|z+h{^`$s!&JP!go?#Q&X+~|}?A@s7Lwi1Hg&~IztVh=`N1sq;f96?RJJkkyi zyH`5Vl8Ilx^ESG-T-rrvmwbHa&sW;z{}~L=Q~sz;S^W={?vwZb*i(Kxb+Qj2iJ7|m zRC)1hJACbCn>5x2AGJ#(?Jeopev;9yS>HzXbR1Q^3I3sWe+D7pZN+Y>8%aQmHE!I(CcE)5FE$Q{up;;6X=5Bfdg<^lLj>!glOMYgcs97jHsi9op#4AW>MQD z4vYr(CuOwaYg3F`jSaWfcbm_6hRjbXrRwOxv=7Sgy@~m=9e1KeEVA$duPlUTTb*IC z%oCg+#gof2)xWG*8BPRO@ z~{X&vwgw6 z&Hsz`1*ZFE0a;2aZ+lEQ?_bn7%vfZrct_H|4%;m*X^xG5gMBfH_x?NL69yl*Tl_x) zKJ-YbnGZcgeCP(WPkUryrX}{Gm3it(UEcDa+vu3GRe0)7=9555?@T-$Ym4QYExc|$ z%jS>i*y^;V^EQsGkZY2`s3$z(dmR|jdBtx1ykfV0Ua_0bD|XX)#qPNCirsPN6}$EG zirxPQ=SB5+UeuOwUeq9}QBUcofs^F48S!5RAU}J(8RX0`aU_EL?08`PKSth(^hL|b z1BdJuPxPx~0*^2BmA@C6BX;1S{)w;b7Uam9E^;a3e`ImA4Cc^MG1wKIeFfv9IZwe? z60E?f2`{vsny#&qqD%{A9vjoqqP}^s|@fXD`vu-Z=W%8%IBTb^6(B&`;E` zZwLDEHvfnA?R0+PO+0VRX7(|Sfs=Tyg~apm(C)&0`4s0DGPdR0Snqo%H7-@=X^91XX ze5IX%O)(4&IwqV_HzwR4#&*Uj{8tUUMq!xnS_>(VAePp{k80?XyYhq(Dpyt2<=8HS z?nKqUREGyT0Jeu~5s0oC9^qjGXT;?TPde)Z-WmNi$)nb~r4#JDdYe#*HirZlZy}&* z%EcV4r~sOV#g(G7j!`Gc5YPnYtmZOlmHDkP%C zc&@QX2tIFR0W+nh;DL0vw9v4xmjm@uYjD4nKH?<1q-wG7c(EGC-^mQ_h)q~?v6yiN z=v`17S}uu;H`fbD=~Smj-TFZ~aR4!T2#PPXiW&P@c`4ZJMMqboh=te_w3sYrbW&~CsZKgcXZwQ9qu9P+8eVb#o%;gVI3OHDcn>l_ zfl9QL`$FIN?{dm+6zy~9a*EeRp=|F)+5TsqIyW`ZQUddQXK@dPx=pih%CRpl#g#BtUIYa)99yvAeST_xuO8?TH7>~-Z=h2MM&Z0vy zh#YPhkbfQ$0}=%zhzAE*$?oi1<(@|3`ZHb91F#7nHl{A+2*-RQ2G|(^d@Bkh_7x%sXs z!BRt)_fCKSq}+W{6GT}UbofvvkTHe^yQ<|0F+5HMib3Hzco=;iilwH=f#iwPe#3%X+t0E*xAZ7Jrj)kg&Jnayxwo+3EiY z4X3a4T47pyZ%RS47O31rrTaJGx(k1sVJx1~rof^2C0Cdg{U=LV2=ZCP-`i{LTq>-| zOUr;2HzJnS1dh9lzw%1Jh605cGd_#oI=^s_-J(S}+y~N_buZ_L(Posy_jmU<^UA%w zfCLX9!3Rh{GarzlOEl9rhKF2T5CS}m=w!Poz(dq2H84DM=9?vG!o%~3hxbT=b`9>Y z$v_^AkJqDrAHgTzz(>A;k9>xYe1?zwczoo?<0GHpBj3OW9yUicd@wosqLktWs$9?T zQE%V_0N(H&_=un=)JL=)U+NUrWcSXfCk$>6v=#3GaU~Q9+>f7`=n0-0@aP~<=?Ske zaxX|N9_xuQuy?SI4QFS9^f1ROhEuNhlvd9fSq!VfC{--17pkT@&XI9zKB}1+E zmNv}(mP-g*@Wzatc8jhD^GXdc7x#-{=YF>LPo*i+=%h6bmJqsHK0Knx8=i7~E62ok zm=XCZV&Re0=MLwRf2D|m3HUvd|eHTM4XdiG5G|8~9`na=mO+F~}BXIwjUc(6pgPeMWcx%6?}5%Hjy%GYdm) z^ai(7!aZ(^yH{y?%xmnKM2VybI_|4wJ?Y`no=%o`L zp?Zr^KK68M^A2o8br1X&qi2Ro&E$pZ99S!6G<#vP53KiQw0hJUZ|Uy9yHRBT*8{}q z!({G4KsrTP5U(r7yYHSs>whsk5Bja9%t{6iCcHy}3Oy#LVc^I`vSK7R`kBORg?*e}+(I|;{OH%t+? z!pbj;Nbc4;_es*SxX;oBJb=Vp0xdZ|DDSoH+o|GTl7|A zkV4ZHAgE$7*ls0A1o}g`hKYv@fUj_qCwv=qxq-UeFoXQH>-GND^ZrItJf)j@=x>7?rddLNvm1%I z^8Pj;SaE)3aV-LNpU*^1z6dU|Sr%qOuVssFHW^0`FyE=`h1E-?CY|J= z%N3PJp!ri)|8V?9ptPeI`9XObaGFafPJv;69lT+aC6G@pb8Yk6L^=C7Mxw#4Zb|^H z;ctEWcrSFpTYj_6U0T23La%(OPuk=wtfm+Q7vVMc(HPqI7jJWyMulfEDq8Foihk47 zU3&!++({marxR>8+fDkbvEeB-nw4!e@cawRhMS4{s~TPZP^0S~YDoW3L;8oBIQ>IS zoc^Ik*FV%4_M6pYzuAuZhX|~@HFl3459B=do9FRY+=TjvXuSR*Z!-maXqbNzffc@z zY?BB0q-rEb1gH&G##o9~2OrXB_4#3l?r9Ch5rW@*gx|{yew(|&4@xclzlK-+{~TTo zJ;2LTS|0#-)w>06JQZ2DQlmwcXq^mdW2%a<4@P zWkaTzav_7ywxzh@=WWCPs4IZSecK%PcyJrq0uKQZ_}FpiFiI2?P=9a?A=8AAX>!0X z6y9nPo~W)k_8B+fCWS=c6_l|G(BK%$Pidm&2chBkL4v6hRUcZDo4SL?3o>;dQSKl0 zT-U!^%-9Wd07>(|kw@9D8*LZM@lYb82_`X9g41~5lYwx{yYc*T{J4+HaJM$@dbVR6 z2^+$?vJ$>wEaJ}s{&etXK7aPKv840Q&oAq~D_n%%nRnCq&p+v)@KcK=exn7utVIrG z|4*|lxMRa27<9{BtS$vnbdebD{~3MUdd7bn9t(u;+nR?15uEKQgOUQ5;(bRiLuLGF z4P2=mZYo)@*xNiXo6DAToO9%+6R2Pln9GfX>5TMaZ5Km)e-`f4IF4Fu>FKE^l?_|ZpgH#BZtJM zsjAxZdJtSR#1hCrFSHv;0lHi|MNi`^VMkAnA|?=1a0 zXs(Pu6KQlZe~#l%kw0hg=S=>*hdnE*EEfK$iVFR;Y#vWqVdO41UY8oLYmL`c#_LMs zwb6L3H(n#gt75$FH(qxcuXK3EUOSD{R^v6_C}1~UZN_Vn@mgTKI*eD*cr7ztON`fh zjMvG=>p0_ep7A=%c%8{#J!I(}q&xrWg8Hq1!J91pyG3a^n@@M!J>gNdmdQZN7f98? zgMvrC*%sOs0Leze5-rIdxtB+}*;c#_m%CBIXr#h-A_QW~7 z`VyLz?zW-APRe%%NAF`RLe!w(4qq zaKF$Vwfb!;8VHoD_0I4(Yv~Su0Qn!J^oJ7BE_nHgS`X~{o0chY@k>Cql`Gfnm9oW`W7Sw;UrGHO3?4yBaUDD{Hy3sU|wDT=Pf4%r95<;P6@x82is^4?955L**+*C_ZRuKxTxkoftg?l@58Ot>`E! zH3R2d*vBVdikpft;r;_X!TB{%{2;mxd*q20Pqg!rF~J*>EdF88axc6QD0=D=kX>yS ze<7~_m3!#^$zDCU4E=;$h6Y&))pz$dqxum-^%schudWAFUv4uUm5n%ytSQ<8l0uEN zPxAxIGvrQm{$EJ(GuY0maF-aKl5P@AwfGtT>;6P0e{pE@KlT6rxBf(;e&3xGSW735 z+m#zmvAeTUPGd-HcLlz8o}%}a_~3wA9d_#Q6)8 zhb>fH8A#{nBm5!l4px_WrEjHW^YId{H*KuVo2ktOD=EAHBWiISmM6?wH_b5XSELe*wR*BX2S`$X~CUhw9BC^TZ^iEy#7n5 z+?Imhp1Kl^CCbOz#JUt(D_r5ntfK6zkFni$&dV2?N%wvt-Q7sKg+qaKXW{^Ykt9*7 zRZ2{a2eih^N7}G8m;9K`8LUfkm3A)pFkyi){aKJtaivJ$VruP z8&Q1l)ni*O=?R-yeyhzTEr>V^cNVvc^2MSIUGz3~V(8Wn7&bIyAfzXRirIx4w!OMq%bs`-LC(Kn^uo?XpL8!(%)hcdJo zxROGm8tG;99&1>RA*WxEM++6@L<6mDqPziL&ULRK&w_Pe$;QLG6t0Gn5YT(O_%$H2nP`(M16v3QjAHD~l-5!eR2j#pn zeq3HDFTrQbKeK&Wd%4kdJxl~O7y_=2ARADUufpaj_vqT!^CR=~39 z*1g4bju06XvS^R6LwpR|1k-Y?5vAOSU+`3P;66>J1o0dtmJgyJmg~oj?{B3aR9AW) zTj_Z}bfvF~uC)HX^Ly8uX4!kqS9b#K9SXYV!y*1if~4_-}nY{FQu+M8*MzuI+tr#iDV~~ca1ceSjn>0-g6`-=nDEkD_^IdjZh?2~Hi`^2V zs)S`np2~>H6-Z2EM&_>is}WA|ZZc&=(QvlehK9V_40fX?fz99w-ZvL#fG2hjo55pd z9CRGnm2_kX@(>ESLe8clH>(nkTodORO6s0misu`=f-*(FfMi~tfxOaS;FUmr2oM0x z$dG3vyEJFiUl=A(3idk6bGDf<4OqOU6D3Fqh%w5buh zW9Md7P>Pv zK|!X|DNlgrjuLwYavXsi3z;yRlKv!yyJz~q-Kw*|-T2Vm{li^8tH~f~eCS90f*G*> z2#GbJG5vts+c9usZy=pPi!r;QuYC!yQEM>9hfUhUEXdHH7*1E|Gh5)Bfi~Y_qde1W zY{HW^V+$h{cp+``9GU6_ep%*_=d#Hw;B0T4?lWD^Zv+Me5!K-E9+4J*!j&AvIYqd2 znCE;OGG+GcW}8&%kPbVfuiHzFuB!#vUJDet+zvFj{zSe{JU@ceTf}=C;~^kNHGE25 zy&c%X`er%-B-2jdvnk#8=R^}6HFJDf%|Hm%uF9yZkdTtThcy!<`<@Ob6t;+3AruJE zUrL!EErK$r&;k(F!%J-|-wk>h(=`8_zNVLCnegq!z%-Q+I+LXv2p|jKb?`!CAWk@R zA^y$xr5+Z1rZm9^kZEn@Jb$EaQ_Jvn&x2-i`9AyMjyU@n5$BIe|7u`ySDwU#E$2-4 z<$a^_AvH?(cC`LpVU``A7(0L-oB55l@MpH{Gbk_Er64hvaPrxG_Zb{mTnY9V6X1&CYLBKTl685z&6b^7KH ziSQMHo=ANDan53@4GT?# z&LXPCRXT61#USKW3I-vs>OgsMxVWhX~@+Z=Zv6O|W6#>opa%q;vO9JZHh z1~3-_0F4|OXCuTQ=)Vzy-pC$o2qivyngYk+E86;0VlGUnm2ANx7&PesPuzqc z`IV9udY4I_$1DcdB{mF3#rp<)_(uwUGHWY*aP~@;8Ap`bV&g>)vba~6sIxzNWEk5T z=1Q)^z|e^Q823?-(D4V&iL?Eo!Lj=JvZ-PAI8!ZfGmXyp_)t6uW4`GCk{wBfPU`40 zQr*-t^G4Is5&1q-IdAP)HU~`)d27oubO1yPeLUohoEJ?|$utpTheSJD`X=wY_HZhh zPkl%qLqqVf{U4|+tbE6!89kE*zn3wh==N@~E*N{K~3;KlfKy;IFF!Ps`$poIW;y+)Avrl2w~uPKRo_-U<;9yEgbl zQ;u$^NnNdkt}qXUfGYph7zM(I)Gt#HvDFWKS zt%7};JcRaX6nzYx0w3G|fcup3wuSWfg0vIuC001l0E&QXXdFHReC^I3XR4kJM}X5O z#KTuP@0+QIudnpxvjAVSeWo&muNx4)L~GMB0YjVBMB!^8!q+cU_=6MTLFo16yKS1#I3S3&c4X0U#+K`Z?4R2wv? z4S>U1IUhTP7CDp?*W*v(ypIsot#eIAIK$!9{<3IV_wPI_q~Z85UC~32(*$ z-df6XS(u(#m%REqwrG&@S~rLZVqRI zA9Z}uT7&STEtYGnAm{n{%w0f-D#XnFe99@HLlwNI65yv!4?lvO2KB08o1oqSJ_QcU zyb3{{?>Ey^SA|)73||r61#79h8VOY$+3TU+CAcCI=U;K-JPm-}(oVkp1VKoihV{CC z#N4(cP5XdFjlq)|9~>Lh^`U2JU(Ozvws*Fa2FFaaK4x{`vjM_*AVHl;g>%V3p3$>1 zw4?(lLwW{$qP26`Jm50{!UD@?Aw8?*!?g^;bo2}Z$VIzb`Z`dv&;X)lYX`7Qhyyfg z_S_bvW_21h^Y?*By$=X8ePj?hCkBzb1fK{H*?teZ|85}|XdxH~BMgKI4#Nb8;j_}S zFjA*5gU1#Ik1c1VXJMjeVSvYSI*;`52vfn~)&8<*TKBh{6&`VXn67B8LwMAd)-@LC z0H0Yd$mumvcvR_GZ9F{#@|76?@U0=h+Xe6l^bCm?K6TZYwZ|@vp49<8QxDH}f=N5V z$R*p@NbN2QxgH-)R#yKVB~OPLq#aep zp+2C>a6z7uMRgWK9f$m|*&(^I%+VmhLV*J<0eNY04Yw|tGL_!FNs!Q}7jJ%S?_r9@ z&9zx0I`UqlB_bM z&SgZM%ZNIcos~M5A+4qA^GWPkRB=|?Tt>9HOqFLoCV9rsa>7PBuVX8T?s2eJ=;fByC}y^aDW&;R8l0R+qnll#GkZ_I5BSP_Uva%TcfZ=6 zm*;=k!DqGsiEio;Gh2D&R7H3sr+&%9PdTj#`r=c75=b@+ z-isnlbPCqfE?y`AY*nVJL!%QAP#W@}D43sRCg>SPo@0wEqj3oGvXnG72OGxFrIZ0` zE1Nts4-q*`9NEkan>=S?q{=4ELDi{EUPii8%m@C&D^&X?(x2L-6{JA@wLzyqy@nL1 z8JYt1C!r#h25Mtq?a=Q8`6f*fDo8i!wW@qV6@H@nstP|*5vuTQ(##yR`oO`1Z*{0E zS_vzP?~iM$PN^gX6<3u|KM&q9D0NV_yT?wWO6JOR`WPy}2jjOcd`0izrL*s1W)qkS zDZCzy+>ZdRI90U|$SbW*sYCG{Ahpj`r1tr-ruL~=PimieFZELUXvSfYgUmf9-WO(7 z-1us|=v`XK=rhd5?|zOMjNSV$+xSsF%%h^zrE0KlFi^$`uiwUPD97_Fw3`5!FUznW*XM_TP zWxKTlj;cz|^9Pz}SXr<)Le6|7LgJw!=)~O@mz;tGJvI{f&Ky(rdiYW-C z`>7T|telz^t5!G!gKfHNf&+}Y2*rAlQsvls`f_Xt4ixV;-{ z$sQ%=H-ual@CA>Vh3trHq-yLSzz3EueaS-GEl8iEBKL?$)X`nz07w!FzTgXHnypQo z=M5ExhBvA(bQ7r|D$sgDI-pcP$3SCRQY;&cL;9r|={zOIZ6kL3Za=Vse8Cbkn!`M& zws|8CoZh1SpBe?HWiPAXgj#=sCz~`E0SXQzKljA%jKtvpv2BUS9J|E2sBjQ_6h z#eauPxp*X_-_R^#mto4=I1gGjIgrf)dR{VvQ17%5-edDb7&qR{TgwYG;Y$y)XkgxQ!e}^7DJ0f$Qu}NwogBUWe!Zhix zCVd|f;5^6sw!I$UB=}G=8hhZ)9`XM8((Em=a4GJuKP&H#?+i=iw07Z^x?Hi{a*dxO zr#A?h-N5nJi?^tI#ql{e}02Gw;kliMQy-3zjnK<79mGedOUVs;PlCCkcm#NI`t#S9q{qePR{qVJ~hQ{^F zJiFKHnO=5BI8n!G&Gyaj>W`ZLM=AMFL}QOe$$R^6{d4pjc!OT~O1onG0c^=Q;@*db zP9hI_BtcM@<3k36fCaVAcC`IR7vlomvG#92&tB|g?|(u&w-PMDvB9tePf*2cpFK%p z&Xx2r^uJDOV$RgxS@ieuaq`V8&-EwYyjq(`zB%ThIW2kaUzBf{JW$bB9(e8(O&<78 zl?Ph%@<2p!e5&$tV|DUCyVA3kovF!5edU4o^zy)GYQ=qV44l+U9@zT3e&m5)>g0h| z#=7P8l?O`n@<6M~zSRX&{&`sdp(FS=CJVIoCky=N;y3B!fm8pQJb?Hfm#+lmfmQ+Y zm9&sNfQG~%6U^>MCTLgEWxY($qmv1aCS&<+zeyw$IAbz_O(zpXlrz;T9Nj>SNx3Q$ zy!|T%9#fKHsl6Q1gQ`qW&Xt@{WdbG;gh4b=9#`A^BMzjt5DZ)w1*!kq7?lZTYBIr` zm`v~-$poYHGQnsj6P(gyf;R~EhWC;QO#i%0fD(N0&)*{x{J$a(JaC4{;(5o3EEd8q zCJ#*9kVqbwsh0<4s`5Y>A>;Wqnmn*VFArR&$pZ(TwGgC;k~@Js5K-1vBB^aBn7c7f z9+Z`R51m|7{rQO4}{e_NUi7GYf0q-B<`&(RdoY@V)_ctKlUvH-1 z8ydU-|BoC#^8d*26aU{r{6C5V!2hFkK>U9T;)p%zoPb1JrQ6NqX*kxn>eIy3V z$_q9S=+6Q%U@nOPbD0?Muqp=pjSzjIDh8}1G2l0S#en-YF3<@`V-q8_vdjJit$r}Y?D@(NA(kSGO}-;Fks z3*kF--#0sP{?7#V#$LaBkQTNWgSui|3cP!i_Btg+aqVEI`JH5|4`LHnvI$ZDF%qEu zyo@&Sa)}906dnybh}5b^%$ygaMp6(KQ97<%cpjVM$UbxYpU!Vj8Jiz^3b6Tk(K+iw zc6;ccn>`9VEIWX5I*D>R8RcA@Bke`X`5skWpi<7|L^)sSOF4@eXb{I>dl`cFLZID`fMg%SpEo90!r#2knam}ed4z(e@w?c za7{u_O#S+6@w!Ip<8?{1Sr}_?mCMpQTy}h*_XPIlY`mm%{&$V1#V_7X_U+xgRIO|s zM}yzk8zFuuyy=w)F}wEQZ2`KqjTyT4!RHD=`qU=v!IY3_X5X%MRjT|bzyS7uibbf6 zSWd|F(?UX~L*968#PLWiiDL%3)}+d=bCED)zo^Q5cyq1NGKvT8^^2TCVSWS-?~xVL zk-S3Nc>jYhwRa1mf+YPg$paQ!q}*XWTHMT9lV1TD3P9QAGN0BSNOBM7QyKHfI>sJ1l28mqgG)tw50l~#*e zciezFj7vbsyu#s|9&xZJsW!|YJ|4|>kHb4+%m=1`K%2ze{lOtd9XL4^W2;HsHfy!}5OR0J zW>?rhruZzj2)uRfemq@|5;H+6(?{Q@HM!R2Db50tlcOn3-*}M7)a2FLnSYF$wZ-pU z!MpQ~108XQPlxlF6R zgH^ukiOvUg(r3*nHQ+W>#WwKu=e4;V7VRQrH1xe_0Si=75lnS~Z7M?Z_^r zByjDNUwcYD?@NXC1g0=`4|x#42JMCoD&AujfKgRfFyL3o-Rb{guH>l9;ZC^1>kVon43B|S!QCi-F>@Rv= zxtpMvm+WmF0{0qSJ!F^hkzFerZ1OU5XkhOCkTrKdU$9GSA9Y zj&OYp$uZWDf5iwv9_{<7-n;gg=3T3@!S1D+El=oP?VD!j!FZJGoxm+kL7i;nQZ?Dr z713l*Q8S*alr5`RvL}qVpiL~43cF%)N?|;RY0oI1Qtu`0#a{zCi73DCVe7I974`*l z)x`EFJFTayg3$@Hrnl!-1&t&Omh+yYr+MFs(=2-vY5aD24kz=TNUA;xIhv@);zz0h z$anKTak4G|In#f3XF*mz+p9f z5fN`AW`i0Jh%r25qM|@|Mob=TLR!LME8opXp8||N%~I);8o$VQZQE0nLTu-+BN64o zeM8{;IbZ&H6w64{qm$7F-_x5Pzj2BR6VxB#jWvL9bwapBlt)WtHs9<>R39)~wpVuqj#?=6(#f0@40z9TuQlQ_NIlsN zwI_@X^(SoHYc1JfyS0J^8BmugaLmHJQxHD-rvPa4DZC!tA4c9-2+ zT6hola;fEe>P!zDg1%bNkFE*omE$2UwPlyq9ygZOABTE1*$sAUdEvQ2gyP^og+5Fm zmbJtFX^8-R*RDB4mtyO7)1}zs@N4^zkwA}p6dzoBCq2LNm#la#d~D~5_jm+N2@;$a zEMLkeytJTFOU<=gss4K;B3ip$DT4F`KY4K_`qms%N?t|ZnvZ#Du^)fBUu7}Yzu`&- zCbOGZB$9Z6fn@q8%zq}VjDLVp&x3efTdeWi}V@RxLnM0|V5o2WV-6Uj8 z*5-ZO%0Ns5FdK=7Yzpw) zOHGB>kskL1unD4cP+6YD26>v=IHU2}EEv^KH-Lh3M<=RC+2qB0xU~xq+(v4PraqiT z(UaVMr}aB^WzR7(cPZT3>Azr0#RQWD?E3tuL3e!)V)&09VPbf2H*&=L@rIUEl?SMv zJu3sR)Ja(hAEd2xWvLfE$a~o2`B|#IvN($rme@~JVVQ~wOXhx|UiUElQG7h2sQxmC z=`VvaZI0&uiZ}n7))BtzG?+Ds3!#s{gz_~gbr6)k$Mmr5?Mlv9wVLoVba~=V*lfLv zNwI0q%0bUBrrPWsTqIFGNN;=5cpku0Y7GPdBx(yein(s8#6wA;Am$0}cUFJBhD>G# zd~yFd2Fgm{s3AYdd>|MFJL9w@%n?$=OZTgfreLGZYNN@H8qRxFDw1J^U%dmn)Xi6m zUimH2lMS0O*C9yM7?JUNX>bejKp+jg?8AXMuaE33&WOGb?!cf#K;M4u!%^J#Me`g! zaxVj9Q_KjdPaZ|4#(_0>`4_P@C^?M*)Gi$oHv$_N2Qtz?uHW#|3tQ+BWlq9yjPwyg zdb9WycKLC^W+JZwzS)<`=*{>v2rX)QP}KZ}ZjyR%1~^g7m?-#`0cqTTT~$n1h1=<> za64TUZl`CcpQ>=H#^$S-t_ru9y1?MM4i>>zNmGLT^7?VvNZb^yty(zBt()K}$!qXl z!q3=xy<(TNn)_g0a)&`}nj z1$EH9!76w-1&L`l1)Ji$Gfn+RHoI;>?+8nGWOTzIih7~b~~I!dwJPwLhO|x2^$cT^Cm2Vm8kHpfiF103hyfQn_~JE zyoh%_f~WUc^&w%rD1YFmc0CO71*u61Z)Q`z8xL-j53p4J0I|k(nMMT8zm$@E|hxAr9BfX~N>-vXea8MDXYnwBp?5YprJs$h(YQ%_N7Q9Un=j@D|vkO)3vAFvdKDqCO);$QczW+1`!kv@|16_PU_W<|GV%&9LZ5 z<6wjO)aV|~R}ki52eNl9RdtOCO@zuwfR%Ts^4eGzMfmg&v8P6N7Iaahb^+97@!dM= zvb6)K%Tc6gKGyPQ&=M+5(pOrKl>$We`iqB`*mLt=3cjc<3|Dd(&$<>oj-;b1+Z|D^ zyOWM5T+6@UHZlmZ0q>(OI|%4e?t2zHNkKL(9&48fYt>Dcx8j)@JcH=3KB1FOJu+d_ zu^ipQ*I{3tIEMs8T-Uh)?{;FNYayzTA`CMJtgmkw0!$Ttyl+;fIv zDx(-JZ`c+OPE*u6`_5oe&SrQO9I0bM)XnK zf7G+fDA8Bj`$^HhOx)v`<&gS1!>vK`5bbMXw6F7Qo^O-wQlhSsqg@T)9)FTH(}#=f z37jr^Ea9|F{Z7ji?Q7uDsnLGAi>}c3W5Rwa3om_lfZ7kO4iK42qvU1%V2OI5z+$?F zg{H=Sxvy||j}%>I`!EvGz#<-OvUvs)7=22;Hf~~L)OPS2dHLCoYkHyp6>HWui!~d8 z&BA+B+;AA1;{a+a11Qe>K5ewW=;!DL39CLclZm$YX z+EQ?yL*yi%hgtq`Wo_IuJ7xgUH8S>21C=K~LuMxVd%!numI}UGHSn$M>KA%8 zHk=6Tl!08yZ!l!JJYeY^Ix1Fu3dq}<`$*7GS-HocS}>B{ToWxT$g0o+CMhB#{+Bd#}D=H{P&XlatAK_b()^Y* zUlTL~EHwYQ;uhw4)1L(So+>~VEY_hz7owtn0Hytv2k`o$FR|!PP~_>EyJ7*+$j~jh z`*ES~fTtLFD7HW+4bdH3nMHL0Xo*;M&0RS%x60EsntL(e6H|=pb6CXCD0tq}>p65! z`_lu^PJklS;*Dp68nz#q6kyplQ4d5JJRfW$s`B_o1`!*n=mcKe%Y-N*o0jrxwTtIX zQBD(xNsJ#XzC|o(VX-wu`AUel?WZI>pM1UI#hmsN|uLWrJ(WNUWDvmc9&x}vt`y-8^c3rbeQJ-y>4h4=6 z1YW%GJZhR&15FPV4iDL&O?22JkJQ6=67z6hx)gp8dqwmOh2rS!sdM*^;Min5zqdY1 z`&suP;}d9`VDE4t>=Nx0_c(@<#CfjUNJ_j~o3A)4BK4^6b3VQd^ZJQpHRmP8!*!FLULKd(i3UQ66lex7Ah%q?`j7iaj(azGNos9s>?P*Qc`bv(@3mB1? z=*Jf7aqV+#_qV8A$wkq=#_J|#ikp}z+S$ORx3G=|6ki|hq`Sxw?PM~rBrU{}q&~6x9=ZEf$FB7CLJ6cp}HQY9bx2|Z4?R^ULD>cM}MeYCw z76P)-ksVH{&0*aW-(hhzd1;qi3RUcrKR^iKd{eQ0ZA2)j5aqXsmr;jPH^OzjIGMBz zT*=GC_Y=yIzJragT)lv?2)m6@eTVA?krnE0W=9n;bw_AwY~rZ5W>bXsQ64mr*GN3+ zz!1Sfmk~82YM(4V!%WrpU;lz4qX+Qfc9{@>^X8q7L_#SJbf$0_qbC%cKWqloq7+cx zwjc?HSJ+S{GL8o|+%2O?HX-#0c`Ba=!s5(4pw$1pP)Dh+1j$T!>kd?<%*-I)WoZt} zOf)k`3fQAT&zbA8>bbnipwnYtNQSqmk!l+XavkpTLU-{o7h_&?F=nOxig|q-M8&+m zMJVRkrI|UZt1`S(^-a2Wx-$%?&qb6-H+Y;Rq-!78O1Deb(#OzW!pHU<$lW}5J6%4_ zVa4aM;#;x!V=R7{6<^MZBX3dhjab}(#W&xskLDAz7svbeaMX(M!pgyhs0k>`ul}zX z^r&8g{w72imxGMDO2^pbWf_5^gXqSlQ>ouig%cg}iVUIPOM!xuEc;2EvtQxD+$st^ zMoH?!%17rIV)^Q|duY`WZR8EaA2g|sXk5unxE=A%XxHCnF$AwO{?nd;7<`?Qm!Hjr zB9=e*|6>0@*N*^to_gJ4NZ>{B^rtC?pc}DYT44@8+!Kxe$n00icy*{je>;>QKIBA_ z0a6610Ru(4u7KfYzJ=mSBwl9t8;&^P)06~G{(|%gvw#&PN2OTsNQU5vBo*K4ur4>d z7dWgd%W2muv(jiYVDWU!;Es%+8snhx$#yDRNZ#IaDLN0hOTcY>!p)53oJTFf0(mud7 z<)_~eCS0PrMPN=fXr1$Bg4QS^D=D9(XbQFfQaMUR@)_gT*i9ub?Zu93zM_unsblxw zoCWu1WuVBi6pjF=@f$=r@}OOwjCNFmVFR7n3aW59&p*&3t_rx!U;##T)LKsJ)sn(E z#kai5sI{C(dX`A=8afUfH7bo@>#7Q;QzQLnvcQy>@%Ic*8v4&>gM*rr=-{=^C`S!P+1F)Pe_c@G_$=gsDED ztxA-7`l!lR246H_OlMb6^MBt71gV$4{NL~T9`)%GjUgaKd;eLVE)0)7gm4K~@!XJA zG#wDhl$EzOitrRV2AVxG5*hd~(7AcHCKo$_5- zIdJ;W_04d^^3 zxHg&mUSiKU&m+ki_Tzn>wHq1mQXyQ2v7^)J96o3gjeBvqJjapMMKj8SOeKp^Ps6no zMfz)W*)^E{F9m*11%A#u7U)JOJuw1s-tPWke4@aBB;0v67=O~BhjI5~48Hw+;d>i1 z$LSZot8V#M;JbqzXLtTR4wh4W zg`IMW_buxOp9*a;yt#=qcq-l$&T>j+*juZL9VIWIGyG#oy^wF|M<652-AppVkLhFR zLik`gN%C&Ap#7Bzx)O~5g7usS4|Z`fkUmd0Z5Zu;Y>-Jf$SCr-KY$k1el@s$Cj~_B zwCI-lY_9X_GGph0bq}I}Q#!?!Ob8?BtN8933nb?tpIMPFrymtF>-m)Kd@22?;Q7SJ z8*5#&1*yiF-JI_$K57<>dw?VyHlwmnJkY^s1}J*2m{~{hapzOi+ynGD?Mv;c7B#mC zz33NLs(i^S?I52OgefUr4O$o{0!?Oh+!alRP!omB7f9rD)&OYk)IhUQ1x<{4+aGv# z;+oO&xjG3vI}?IuC$3xP4+YOo0?$qzc!rYLrcI68G<9OyB+j;R>I4WXtPNaADRG}F zc(}$RP{WQvv7eXI+t{wLfMvUk)W}%NmH0>lQzK-zP!26$aVyX&%pGy+f4qZ1E0d+i zFi8LU6@v5(gmg4!Rr2SNusrimRGdj4LubJUOZsy0Jd2?pf6%D>!t6O~$Y+eqkh&gc zs?<_XERxCFBu0M>QkCl`=t_nrNK0qLf*EUZlyBEeA9?u-W#RP}14d33(Nz3ld4|R7 zch5zmAq-AZuml+Jc9+{}Y{%H;c54gw#1x|1P6M&uEsXuXTV=o7f_PELY$(a8%>muH zUWeCh{IjKR1}G9NwtSSL+1RCT7>qtFAN%V z_JwB*-)mpE&+sqX7Y3g*s2=+5%3d4Wn3w3lTy93+@a!Iu(#<{|Wx8;mjwUO^=2#5D z`@f%Cf3O#?bqFN|jC6HYbH!nO%oY2_v_R)w&CGKl-qN52!E9X|tlq3p@Awn{Ilc(v zVQGBP|3E*1nbW)x+L&%!e&T~L3A{sc0|)eQ2Fl$RPs81^eMs+9;>4`z1q(AkG>c{c zm__W*bB1^p{|N)TG?FkA>|)r4hvPW>YJ#P?IlScD1{{zhH=@PYAHs95Ik&;k0>H`# z*E3!SVT9PB4wW4;sq7Gu0BfaD(*@IC*g~-Rq5ILo=VEt_AYoU8kKlf4`4(hm} zhmq|wwy7&X`k6Ssu%xAj#lL$ot_PiPbmeKHLmEeqM$=B~Zmy&tiSZA3ors2^z<=yV z{v*AUAD7-ENa@h+N{#>6?>TG~vVpalqw?+nmHQZ=#vSUzwF@wY2Yoc0)^5)8Iu26B zOS|}RgPJdAN|l%y7AS8Q9tVP&*$YTMSMnMm0~`gF!0zpaJ*$OdV5fErVA%Tv#{fM} zd<@)s~B_;87bkU))ZY{OOIYxF*Uc!(?M6@i&BiDW6xsZkWjnN*7)_ZN-M7^rvlIc&qvp9T6Wi;bQqdFhF~e zbrba>7TJy{*O|#wLA?csDYsBRrx@^}y}#4tsh_}^$GsGo z#ywNcOIxwE&vliNHBK(f2p%J8xNpbF+~c1h5%HXXAxaI@b5FGA3$bVHw=kV5aWj+J&Eje zD=jsW-c>_+{X8la63D+>#CU~<4>_!D1+oX>=sp6*4|e<{Cu2a;T!My9$xT~8zW z^iig4m~JsBZ%rrZ?v9Og{CxWxI)3hjU)wRmyy9fVHdg!@D;|W!KL#LBE@Q>#v*Nc| zF^|O;V=>2ylUea^Sn&ib{uVoLu~G36dVXaQEB*vNwp+1vt<921o?QL)W@1YRygI4Z z`ExUQ{lVe$neObFK@WTKed#RwGRTS9uv+Lx_U?KD!hOn07%AWP{x{R;-s*s|*k++q z8aolKLC#Cru5yF?_6Ym8lFzK>yx))s*w#sC_y9887!-((7?=|oFvZ>+VxL~XUV%4u zW@aRkFQjDvBy))+=fzH${th}?j~hnbG|esBt$Bh zvT%pfdUENTb9`&ryyX{gB7$|V>k+a9d&YC z?u42oR1mRh^E-*b>^z$$shc!O-8xNDV%48I_}Mf_>cFD)ge$w(ezx+NrMx`ll$hDb z17CuQ(MY*q{my+@2z+Hy;T;xKX?}%h!$VibXu|}c4Tw@Y+CaQ7h|0(auUGEB4hV&O z9c|I48)Em|F%eAp%Rf>K%DvZF3~}*KRD7L8AOvUZ(5^fjHihEtdCsIfd<|Om7VsH> zWbVr_nupX4&y$v36uoaIzq(Sobq*??KhdqTmxxQ(wKyTNdg86+uJIHUD1dS4#7|X~ zK4#;~3+5v7`nIwaI1a>G8CPc@=n5P);oZO|e;ypI&v}z5qHdrsx(6vt_zRoKfJjC6 zuXN~6wmUxZt0Q!8kHsG%d}p>jfbYiz>09NM={TLE4CPzmQNAID@-1;FUydlh1u>6a zTV|61lwU@;x>k?!Dfh(mJ!v4X{yXqJepL6b0D~`knlLzpK89ZYw1)2^rzFC61FOA% zJ=Hcpt>ODulmA70Z|?d3_^!wIEzIUsX?cn8eRXa;zE3iq1>cLWq1`$3bV5}B@UMXC z%b`kKd_$=gm_7vC)Z>v>Oy@!^Ykkb{<9KnyR- z=PQ1^hWA|$*f$@(c&kA4wKid7>7_?_pL4w`%b;aRlM~28YAZPjICHc$YjO_-(Rf}o zCXaFp468h1O~{{kj`)G*8OTm=K#KOFC0YJaAfA(x00(a>%2ua*Ik^c?IMa1;SAEP+ z;3d4Y{k1_2uIV}tl`rcOSh&0bzd{d^!HURB?`KsKTNCCbSJo2ZE=sbtpzw%V_-C^= z;ku2z0AF!_))HP`OHX+DEvm%Jti9J?G#kb{?28+ZIS~8}$?rRx$$?^9I%`1@||Xf_lPUi!89w`x&F{h)s~-IJDpt3G<*L2$*L3>JkP`uZxuE5 zny?`G>Dpi|Hk9mH_uAxX8Te(M=TWmogfBO9UJMN9dBkktye}G9KZbcIhC<=BcpJ`c ze3gqE1{&xgFF&uoz3ZV5Ee7WM_sx%)uKQ4BhZK zyssq7K*leoEamoex(mS0RBbD}mIQVX#k-O$G$jt=lJW(`S@X!rz~(tUY-I+S)5BNO z%5z9ug0IGvgSlt@+}nPi-C(p!FK79Q_fN$O0HC&>4ihh}&BD}*JU6SBEPc6|m8vLj zlTJe+p7geQfcx~R)-o6#eSA1)ncdjTZxnr(lu{)5;;bT1#IkZIzq%)I4wQR7Iv1Dk z?=)GUFd1mY#mbicjF;ZZB1`*^a2C^Wp^Yc`KqNy+eo-}%tfHgy`K)FX>`Og|rlPV$ z3gC9#!Tao%Y50v-w}ABqhY>F?!iO*!_obmpTKm}QLK9p<=p(MOYbfs8Z?CklJrxRe z@Dh}P+*XE_bMk$r4qSNT#Vbm())?5o*zLjlS!?ODIBN<|33Qp{T$07)1$)*spG*z zrkiiHY#Te~dpu#nF;AHA_j|&S3rvO11;%xG)cqyzB8Ir@4E+aD2n~yO2V%$xgl+DR z;|W{IMIc`X|1z2|m+U=?v+h;yJdTbYgiUN`>y zJlaWV13vP-PpFI0jQ}}r=A~LuW+$9d$};8QroVm;9A*TVYYL0*_-tt5iOOFNg3 zkG2B5h9*tXCKhUfl+6c6gCxS4EHmOW*Ov#l!jJKEo0h`iZrY1!VLU~s=k}Q2E$E@E_D(v938Pp#<|C?FuHY~<` z(1u;D4qG^qV2{SaKuOu^a1A3#z0D^-Pp7QYdV&UBx&)brU)$0w^Gpk*sX0+NHLD^6T_^fGol&O_JPb)z|W ztq;|Ma238%&KzHQz#&!GJjYH8zFDX7*js|s#SY?}&2!L*l6&}br&KLumkN<;QBr_T z*`?#T`}l%3+{s)xQg-o~4H#|>)20`W#v9ZwK3r#O_ZvAc#-$JbgmkjUK|G$HRg~|W zl7w^O0hl^?YiHpFf-kK`wC*cR&n2d9p1%EIykCs3k#6(YV))Is3F;S{h8^ndy z+h2(Z6;%l?gd(8WN#QC;v_lE-uFf1FQt4=_Ba)aVc2J~@jvUrHRIR3B{G4h~Dn^}= z(9@wo>bOognQ{{kZtB$R2eiRgn$!VBDdMzNxMs#))0HO5rVb&qMNBbuvL*2BHj36m zu1sD|qa^1kAl%FIJIwTbekT!T!P?>e5--t{OK7|8lNm~IYi}V}JD^>-?=Sj+G=5Hm z_=2Hu7G5RoBlC<$bw`O*jv)&@kyOru_87So2i7a%<`u#p>`GKwHjD|dX1E1+RDA%e z;d5o>#KFHBe5{QSCyIFH-QGu8ZVGIE$&1v%n5< z!|mPcx#42>CeC8-%)n}ROx_5axu8kw@BQVIHeXO?(=NYUS;$%YE`;N zqFavRO5HlGJ-PRZ(m`l~wS!Mzjx#V;yV=)I?eKc7c51Bl8~xR;iq;+zt6k#j`wphw@4!Xh zVL+^9et+ZK869VEtaf^TwXcZQPK(t(v9|B52KJuSBa5__gJLb;THDV!J2kL-cUY|U z6aCeGAzC{(5ea9Kxdz{Cj;|z(ld|U2k zoKH1?dv{c<_L>Car1l5gliEtmTRXEl2cjnG$U|GS=L<7<`D$KHZ9wZ7(YGYx-M?~du*<-7 zkXWBR5O%` zRf(EhoiC@h!=Tp!7n@QGP0o0_lex94e!wVGv{6oj%vL-XMrgP#7G_&r~WvuO(`si z-ywZ#^X%v$d7>x_tJ|f6(Eo?5_fG8n3SQ3JMG@~$z@nry7~5@;Pl~YrGt_@I0ZAA3 zj}xh3vq))ydE{cyjG`=?r~EY9gxjM5M~A~)M0rXzJQ3pBiEH7M0thHelw$>h5GIOdKrg4I%0LHo@{Vbc)<@#@)L2A~i1Mcb1lIEIM?PR| zX&t?yu3B`lZ+&o(L0XhGiJ~y0gPVkmHd3i)+2w_`#&t6pz-mttS-FFpfX-*$ zBSn`Zg=Q4!Ccubefs=p{jnq-$7+xMF%6VOs_u_;NCL0<+CAO6ENEzUgCULEem$<$g z9#y8@m1Kws6`Bg+>hmmyRqwNf5n-V_=<|7cC$6O4$S4kL0`MXFOqB%jj@&A{RAtX? z_qIog)=b&QWg5s>g##P362V)K#rKW`LxBoDrmn!{{!;y&c z<~Ynzvdb7ebCis%%O4`^gJJYBv<^PDcj8^<&&H7<+D;D@pIpKW(css1G$~VfR=kF_ zu3^O=V(SrTZkNr9?_KtM^#W)A!pYfBLT4LA&fZ07F?K2A-WrITfPQ`o zN8iJkuEnH%ZKufZotR!4!(XLt9*CJD9P@@pi#zf8x3AFi2d*D((Cpyo`RA_;H0UD8 z)3Zu=hMAoyZb|pnfh-?0b$3Du!EY8&m-;zDrlALwJ->%)OesLkcIc~7v~H^QDpoVa zs=bi7+7npK9ILi0akbmAnk80kE>w%A6I5TO`_E8kc&yIFTh)oG*By@~Xj6f0l4EVc zZzUYQ`FBulV60kA;%e_=wbWR(za*~q3|1QxtM>R?eJ6GqPHb$f&TVh0%c;K1KN7U* z`V-U{5Nq?}#KVuo;Rna6B`2=-^>3kCTCCdVZ}uH=4UTwDtj@MK)pb&5X-+W9`PgPq ztWC+A35S0GhaVQJwm5OMnOH47R_&?;vpn=N)EOVEGvZAZ=G0mKSAsVGz&0tdHV58F zIDExMs5T;2?VUIJc2$5~jf~a#^&9H?tHU2o(B^V%GbGmL_Qb>UIQ*zswMhxOI`R_K z86B&W_J#_i>hO0ZXtM>|42`vEem&vvRlk91=f2=O6G`rDyQh3OHO|63a~;+JVY_$u<na=&}QoTJSFu<74jhC z4Jtd2_bu3>l~KXoH!ut!1xG{^6j+TK<+JigM0V_8-j2(+C|iqgb#}n24~DneLaUyF zb5MR?q^=XMw}IUcQ1G^VF%4(;749*+0*e24uO zJmuu;sg4k$G)cdInA|;;U&ALXp46nvHklxP!@aDn=Ml4E#Sk*&H)5tMv_~C(H`<3D ziMLyQ1P8JU(jE*~=bpfVyp3_<3Sj^UGPgaUJlEBOQK~VTUE^JGy)Y}&6}ZONufS+_ z&9GJX>b8oPUMPw%YP@9{`z*4dalKro0c$Y}ojFob=S?5snKEQwl-a6MfK(Cr;@|nCW<gtQKz}ZLAuiAp;#8jwDPdiTmVAjb@QNu#`m^cy2JN@b+U2f{+RG zvGD~hb~Qm-OW|-eL7K?D)WkjhEJ%CuY_qpYlooG_=8iK1Z3%8bZ_&w4Gdd`8A|T9{ zPEI@{6wHFsFNA{I04f|?MBg1o;>?aJb9#CRK)%GJMbh17U;rjN<$;2HH3mvSE$m)C z+aDUN^QV?g4Z=thkZV&T@4G?s2k`^@fcbVrc`<+sY`T2+7Q6NE@|&GFb0<6>ZkLW` z9}uJ_Azb4$Hj@$BY(9KIkW=@faZf34te(a_Qz1xc6d+UTK!b&Y=oV1TJp&y8Y{BXX zn8rIZwwYpq3R0C2 zfo(nJatZQJSgvcMr!&btiYz7F^FoV|y_Tw>_R0rB4?D91ywsMfOK-YfknQ`-g6!Id zX{}3*@TbLyODah49F>Nc%m$#d5jn46f;=Cdaxb?{r8>3uS9jps>GSK*VEBH>m=@RJ z`$;#Ny>*;-!I?-zeZin`1|xg#cijT-PLV<*N?SL3*9%fu0^WkT0DV1}G+Ak72>@fc z0!PQ^3dG!@WI?xBmqHZKBQ!`$Ckm8~FIZxZp~yJ^Wt-7GEQ%(WHBpd0m*=XO!max~ zsg(y&%z&rjFa-|gh=wW7f_u!A>J1ng-x4D)xmPI{s_*g+r37%S0>bO?9)T%xB^U~s zp$v45yxl>J^I?wROFH07I^YZZkKqf(`+$zN*sMF2-v&5>i9XBi*+&s>!!?4@Pxx^? z;Kyt^6_Ehg8YAMtY$@%K5ZQs;6QIW-IDrv8%Hq&tV(3w1YpD0)KSI}0EV(}pOXiDE z$C=$c*SF}LnIK#UucO`}AL39`lE4xNZ&EUE17R31GXP{8X)*`9=gtXn8IGo zG*6ykX2)F*idURi7Jci!ue>lEePztiq`mS7hF-5+sek1GYJ8%8-*+<57~d*`{$*j` zhQr4uX;5d9`V z^iBW4^Kb!8N03Zin953pl3xBCD#Br;W%$ za3yn-;*et`M)>1lL1PH=6+V(|cw7LViU`7aG3Zxt^8c<^cj9YT6RZ^Q311y|gQiHY zUm1P<<(uh_dqItIvq5`zuH;ri7B`%3b96)@E|)G#rK=+ZDg&-9p0*UYh?J1%+ejod z31hlK1sg^FXJNmBh~xT)D4GXLjI77#L>+59lMI~q*Xp?oBatNJVUW%a-CW+&Zp5r7 zzhGIwyG@0INZS&cWA{yq0F6!-F~Q)CM!Y)RrMm0@^W19IE<{^(4m+pZhx>5PyBuEP zobzsmM-SFR(?~yCPh*%`N z^1cC+f;|{QEC5UdW_r{De;kk0;-X)P@Ol@;c{rkN#GM!vL7tgmqx1_lX%36^w@eN7 zhfYa0HUEN*5-_+{^cSale5gQ(dh+oMoJ^< zsFbH+0#Sn6_gO?Nwq%!~tqZ2M=$l_y%fHZ$5hM$inU#g?#3U#X)!6fpEXjqQx80aXdhPYLKEqv6;kpfQTr2uFDY7 zfBY}X&%RrbK4a;WDST(dHQ44U&axQX^FvcVLKT-X=x$XzW+qfC>nH6+#WYOoi|RAA zgORFw=wXF%M5$!zaZGtf~MIx9^<&XGTn=t+jL_uZrr;$nHw%C*jEO9 z!M@N52nvZ+Y^gRaFU|k?KEHF4lQylo&3o_vbMHQrCg=5+=lA@c-}Cx?DtYPl#pFwb zn?z$mmp6!~Nfocz(reiN2Cw0h33Ck}W^Ws0{o8*eh>|6o(e9m!Gh8lk}lD0%)aiWVO&^`2YqLe@@7@<<|`L`6?jT@~^Sa zxBjzfpRYIzSrwD%Z2bGik3$L^5A)lfL&g4S^d958!jLnANBRJ-*uZr7!b@qf!KYZD z=vT(RQ+A4tzYq2h-d_gKppPb&u3unWKPPr8U%$Y(evEm2GNzQlqd2%t_%Qvx(O%AQ zDZ)W`F1PpFg>(a8J`Eu8OWPS+b%_0gmk~`~xi$m1QTPWz&zSS8`5em{(z8z0w#tmJ z^HdCCN?^y5rNz*S@5k(JQ5+#HsN?P_`OaErZ~PP^zlbxvK6e5-PV}7|5&+r& zSHm1P`<)~OSZS}N--&@JdRz`KJvW)Y$YPJf&D3Z5srpi$Ewj`Jo~^zN&sJZCXR9ye z+3GVqTh)eV%P1vTLqa04H~m7o^hVRab*6C#my!&{6Wyt}YW-&q!tH9eS*g%SD<>kna2TEEvJLTGkA19!W-+H3CUL9wG(fn+YxC<%|?uJ5Im zyI#FI%U!?;j^_?=duYUOc4a6p(xb;P#fAbS1<`Arm6&)7&*}3v7MLMsjxe;na2uX= zkE3&(c{WTB)+~QL{Zg`as$C4MpT&%{&cP}!1(1&9& z{v4W?6gRYsuhWgMb>izC#@9cK8=uDK-(6%tB9k$*+kxm>QmUwT7}{nCO`@jpPuPgN zrHakmtgu(A_ys-$x8fBf_rAx6iQ_l$;qs^HSV;6Wlq%NX16f2`XeN(f>$L*MiI;_o zjuZKkN&i(y4F+6uXI$*H9^7XxCCr1WZ@np*>ZgSbzbYXSmHTPW$9UsEvT-O$3gJUM z572`5C4u*g_3_Urk|ddmQub@6J}HI%gf^r`h{bJ$SOk1weAn@0k}WUNdHGk8N}|sf z>IjoToyCH~e=E_WjYY>L5O6WE9aXr;fCS^A?@Td%Hsj9nO^SYb?B!m`#?D`QMQ5RS zRr(6{_Ah|5Ux9>v4sUo#Y=9?Y6~;SW9nEXUXjbgCrMeY+Ird;Q3ViAgfgcw08MpnM zZqt1`)Ux<+Q43Xnil?h=zQa|sYLZVMo9;rh@&?g?hO@woEUJ!J_hIqug>agcJz(_K zK#y`kEvW)1hXU<+7K1&PyepaSVeIA<=JUc6fVLH2uv&sKpZm>-fY~-G)uBaAHL2(! zIeMO@WfXHq3XHis1;*UXV9ebN#@w9&#@w9&#@w9(WA0v_Y8JIE-K63OS()AcK$8lC zAbv@;zd58a#hZCNHEwN@1MW#de91!md4l*CGr<_RXZ0aL`397Om#sA@FTlVBz2`8! z|Kq4+`#(aPgnlU3AoMl70A+*NY6h{d5yTd?$HeBHSbSsN5cIw!S0I|hsZmN6UQE$m zl`76Ns4nNuWU9~Vme`PvkN#go`D`Qy5v0&5IZSqq_EY^S#x-OSgXAU<-zsK&g7|;6 zo5`B7@IgWOLv|r!l+u%AK5mvO&Nhjz3n<0hnM(AHT-@@PcZlgv*gibzpZ@P8{RMJv zpy3~+y)Ys`7v^)=qiR{F?%hG#*mSXt5$UHHr1!XDs`e~=Dyz=PAiosEoni{sq^IDN z17ZrU$Pn{VMIgoYLb!X55Z!OV!CXXxmu(&bwf<#RpiW&kZy2cc*xE0q8YYl+HcGf~ z&@|!FcazEg-7ATwPYOMeC*04pS;+cU-sQ83F=Hi(T8JaDtM-dIyk9gk7WdsZWZy7P z8G>xZTQs9N|I3A#6q>?l8HgE#?=V-B_i2P4W_fq4IuAZ#RnI5ls~`6^F!irSZOM{` z_-ELnM2>x+Ay&Cp%)xtHFM|i%=iqKQn1S zn4{MIe<#rVg^(dbCgS^c-bp|s-c8s-!8{QEe~e~_4VKKA_@Xfr3wb8C@(q}bOJ5ye zmpscssp4D5+3=fbtc%pNpzpA;FlJg7#^>TRGpFbL-^`ejeH7c(r~Z8QaFYGmoAPZ2 zwNot)&N~LHXvCg3aB2~KX=>EwG-;Ot;joa>HE>|8lM$%*ZIDGfZaF*;4!A%i0$!n*>vOZ%UV3hKSC1x1?S~>!C zaTqFIXRTMHiD1`R%cwTIfyw$L`Xn4hr=$CG9G&w{3Gq6b zhD#I}M~{ec19PW=D%{bTE1ie_H^&sw|EAB{|3=kwWc^B?wns@cy~tF9!ubbCKL#%Y z+9|r)%xF+c_R^%Za#tG~Msb52tRs?zm>Kl|>Hfc9+|N$Av{??XX(duEZg9Yv;;-!E zo7HOi+bHJ~PpXw@ZBwb15R1&HRza$rL@JWnksP1`*2O>1jV}xKE3Vp=pF>va4DOBG20dW5e%}LKnl+fo0Jn zJ?Is{#rU+vK%^>PM|`=2x27cZfd!? zjJOmV;6jlxs3xw#huN4^9Q7`F{ZTn=t66rRMbtjj!wwd-tx>$(;D zQ>*BO)4+pn5QAq6FwYX*rDsp2qeBnKEf(^HccA%2Jux_M0-{oN(!CuEI2(#YER z@a#78=;7p zNe%sTCznb4xq5m{O;*fp)HdH)^FbM$_?zph*VI_p`+f`x;MOOZN3P&;G1CeT zoC;>Xvvp(+VU8sxr%g$0s+B6)*AP>oDMG1aH@`j+Xm~`g;6UDraYWac;;LUcnLc!a zrd-D^=n1j$BLEEiXRcU%(P3BLI5tc+^6DEwc+@ke*Nb&nFVadqdDY>;}epY5}GvdnAl=W1@*`)-#YN7T*CWbdVRHaQo;+iVE*zulZ;b zIxRClKt;dI@x@&}>;CSW)ZykN#*Jc6%MrVJp%H76#h%7Wf;1@Jh5b#=vcGYEw+?6q z`e&@WoMkzSZDITSY;)=5ycoR+w1dNJbE@vI?(>zjDEaiu{n+MaRPr+ms?E)>z_QJa z$z}N8!?bH*Ib)jxglIJ_=O(e7&EXAN&O?PiFXu0?)iwUCV zj3}2IST38BSY^uP+gL7}lH;Eh%h!>}0&VzTpn}io=!w3{^#7}Bu3O6!^8?#cl5fpCx5EO1?3O5=Q zt~M!LV-pk>%VgFN3L6tXl5936o0vx|k)x{Xh*ZI@vJ@L~mr4?JL~#;t&|8knDM81^ zrBu<4EPfV;&_o(!Qxj=Wbbo_Gs4Wc+^DsWr7NfXcA(8gS94ns;{S^jLNBR{8K{L6; z|Ba!t8iQ+0sY#6%Sq%mOwo0{J+6olqL2{aITpH~&l{We)N+Y_+;A=&0||IEk~=-RZ-+NjD-?v;SZ$4QqV1D5e?Zo+b;!{JbL9LwMJvQX zyGm-iXm`iyEWxet!~ib`F#z6dGU8RHNhcN9QI$0W6nBV4zIM+QfPC)l_8PMfz^3;y*cy=Zkhu z&?#!8i|Xj2#blO*)viK5l)4$yr5MHuN;v8}R}-~_2dtnG8|y~+is1O{Le?Zb0GdP> zuuTiTxhglX={5f|2ACgOVW}0-1VbUEnOu8hu z`C*f;sGSePq!UL?lWs2w%mQPdN;l)J&bm1NyDc=!;`|5<2oP5a-8En;0@G#vG~q?` z>EqqB@|46LCBM#Ba?}fF^Dd@}$O+R64Z9%HIVRPwFK<8|pLa~r@~rkiMAp1pd|D%! z^?yy<0l#27;8=Ha#0^oVqXSh z4PY_{n6UqWF}*oW3r58B*rqvA*6nF zO1qpam}(0uN0`ECfiJwqg5(up;l(yGP2n_(`$}JD|EW)(+N_SB+Thcsru{{}Nv{iD zVmQ2DGH>Nt(zqf%}G%5-1!u*udV?8zQsPxfSYU-YonFD_l&CHSLjWcNjna9{Kghg2CHa*_G;wf&KD(Rt(Q0hM4aSxR{m;lh(9URP(9QjcHuneO{5896J||rY2(NR z48X8AeE?>2kr$v&xllAhBMmV&jPB|0;}?7S`m(6U-yf_VYevKeg=m5`x20A~7pTIU zWHn>!h}9yTFUuN)SKgjh76> zVO9Y#TsLG=H_OLvmXF<8OmTmDEF;NBV;n+0 zR)M0bt4#_84EZQW_s?0F?`fx5+)TfJKWa2fb{pc7bMLt)DV=(RS)ae7)j}nL1z#zjFEp&w+---Zx zS>rGc0Ut9;)A%>R_zqJ3Sm)iGCp`@Q>I@*mfa9Iud?+QPqJi^SOIcd;{+wUKoKI>> zFiM=i6P$lkc5yUSVv3N~88fubSfA$AEh{!y)OqY%4|s_V^c-n;DjxA5Niw761QD~H#ZIyWu-rH!=Ydx$w&*{;vuOV)5J zH#FmC!Iyhn*z>KarLRO^pL2EoZ8Uv)Ac6|D#9leWij%cKM5=g)#ee8pll_N^eTz*Y znBVuL$@vc$oWEma&L1rHKREZxu3dw;KP~pb{0;wd<_~0ueYLNTU##|Jv$=jY*yj39 zHk<1#%(>wip-E2+Z**a%q>5!M{|_h`u{2cE^>bq#eQ6&vLw>VS;#btqhl^83MU|i8 zI!5xpFvBZiF3JCmh8GhzblX5Gf}`p)6RR*m)DYWjS#V)&dkX<$>(=M})X+*8t9L-?ctZ0aBX7F0dHe>Qv zJe18gt4-PaDvYA>1!SKUSnab~!CS&UJD%M>Bk7PJwvzL=k#tC8*k|if_8DzCiDB%s z_N?|<`w;u=5_>8qW#u_WrpgDg&$eY%4o0-kXi^f81__o-36@NWtdc1KiIDgZnrVV1 zQ^I1Gr8QHla$wa=jj@a*nT&A^vdOMyo9wkrn`~_EAe+n-SlP`ovO}tem}Lshfz>P{ zR%kWMGBVc_+0C-w3bRbnJ&8U=FG{4%GP*sX=!+BC%(4d*eT>B{Tg&)qX|s%|l-G)X z)m8`3f(X|xZT;}3+3zzIW6pbEBCF7Y~DfZ~RE_sSK0@Cmv z4NpcQ!#2bI?2CvX2B*vQDB-1vZm}5Lmeo+5;ia(3%M34tIi?Zir8%0Et!Bu}+bp))qI}O` z!&ZBbZM8;2e8ROgjICy*whXb=koRJ>)krUEHZ)e6_WGf=nph1{V*7gH{~7gZZI#wW zy`uZ;v)O928pc{}H8|os?@!xmSPq;Fpw^4!Ad+5|)_-`F_4ihL4I1krUG2uL0`Fz52+V4netsY(kX1}tWivp0UgTBP>*69}+nR1h2=Qd;wk_!G7iia?g4 zfuinI1waZqfc-^FyBRv5I|DkPI|DkPI|UukZ9oTDiv~U>hy7;3zzt@>z!}DM!xRkc zM8QBB(Wi2SsK9j=}Ini~e;r=Yz3}q|lV45oDSb0(gLbxru8i>sxUk_S;_#aQh4gBZSD*E#xLyFw{M+uPWKNP3$6{laqX%|jkN2ix(#RHyZ z0q?j-(RKvypJoe`DB3Fm-`5xRPo2mx(7h_@RI%rN%0M1Ydis$9EBJ}P@$lhgU%4QE zC~`G0?gt@&18R3NV_o0)OHpd|l-Rfl&MoEtKm_P>?jmv^0C4v0Da>BhH*a# zG+Fc32#D^a=%A<478v{1O^p5YS#EIx+O- zH`xOo6GS^+n&fK8#8~Uk&gGwtsBdh=`S}dsofD{GK<}J$O+fFz@Q*xt&MCI|>w^H1 zta#K6n({7V*spNJ=5HDhy>$GaR`e@nfw)NIc%Z%z5$0I#cX_B2Ol$-o4Qf|8M9+JV z)|Y~T&W^vt^M9?%K;L9G?`F7`Bmg2VI?Tvgv7_`C*cB1s@IZ?@*M1rd0FnVK^V452 zGS>wueB=u;P7@pXk>@1?8+ja>{+#9P)#np82d?hC!5QI!0w|e26uxA1b%`}ThX=Ar z=ASGo=S5EeMQcMyKQ?zEESpedE-ey%n*Wq!jW&NqyqWoC{$zd8v}CQeXqrdQTd&Wb zu0=g4N`m`Lzn)8yv zOWqYYjIsSa14%9FjL=Ws7~w~uZG`b*7(Av;Jk@|+(&st!dmXMW$=9U?b^}%Bmp0g7 zWJ@jCB5NcNwd&Xv2M3aXL>WbbNzu0B?wYz9^1$|~KrTP=%chhJGNs=04T z!LRseHdh#YDcPe^&yZ6+^{I@PZf68%VE_}LHWw%z^_`oH&wa1H&wa1mn&EIa^>pYjLOx$ z8I`MhQt!#?JS{$UaZNIXfXCZ11wZRh!_{Y@N6^* zbX?4+A-e(3A{&#F30iTrn4GdSVnW-~VrBy4tcVF9zEttqL8u9*J!hGg86SRHV06d- z&ns z5m}Y8HB&RDX<2IZppgR2)AC=7d5X_}|2Znkn4`d21^^ssem>GkxzI8xygolvz4zHP z0am=nLG*{_d+P57B#R1bsrvriS-|sHz;GU`Bmw8KU`G6M62PR%a?1%wd$QP~`$w~= zR2*C_z49D->~UM+kjGvPo9!3qh=qOIUO-b9UXmnf3FuH&FI2*T0eZA3mIh*HDQ!;z zhb-(hiluKtcLX_jnC7nhNfznL)&&pp!Nfjv1LDGIu47)Zndey(skz!y0vYal)H838 zODpMur)BN=TlnZzb_2D|>+EonG4L!LmRfiXTqBd5kzVgQyA-O-r{#JgKi%I+3yE$e zjEtk`c7!JqZ9YpCbG4%9O*P^WL+g=zdU2!g>|#1yY>h{7eG@98!evQc$qD#GRDHUt zpDqCC-A1k9?mQ_NVs)z~RfD@oFRD{Z4lL4V?^H{w<^0w~+U%VQ`&YbMq>3NdMRU*6 zU5mntcRCh@J#`LnajM=?MM~;xvhLqQf2+9OmOjdQK^q^QvBx|*V~#su1muRZN!X|Ch~jMX zrA?VmvKDqowFoC!i*S;)WOtIauvfM z`}$Dzluw$ghYQ+?F~BDKQaOkF(y7aWKTzt_gq*{oHxd3U%B*?%DpuME!wict*=;Hj0vvUks zLNQCXv7Uq+kM&aM0l2;I=SChjC0VZ|cd<-*IGAImm`33;&x<33c_tiOl_{6`U8vQ1 zC}(hqdX;s?9}`8#Pdg(S^BrN*Ml$C+$|R18`HqVDj%J_lC=*&-x;P@{J2LWoN5yM_`Hv_i3ES zlxlqc!j1+}*wHXiL@pZx{~_zobJ*YIBkud!NyKVAaYQZ`X`Wc0g$0Ag9_KJw)E?J|G zVn3wAv-g@uXYWl_-$NH-V4V%vQn_EG*c~HM5Y5vSq89CtQn1QyRDkt*cXKeUfXl&X zP&CX+nysC47h(ADb{PH zSg)1Y*J~v&l*$jYUMt0Vt^9!N)f~)->(v~Fb-n%@?6*@z5|F3>%k-~uppu`iR*FES z_!&zp-#<_pVeXHREf68@uSB4dtGhxw1mnM%VtjKZgO?3t=lXty%N7p_uCGjS{S7Iu zuQa&6Hgo*M_cfd;tuNh4e-CmED19_@FZJS`=F#GvBH6GmRR-xPZBw*uq5(uQqO&gT zA~l~?Yw6(#8eimcTeT$-rEDJJKHMo%_ti)h^SQG0lwqT9hio4(`qnXA^vyrWhAV0} z4YKxB^sSwvZ|xj?YtM+jwP!@%+EdZD_IuJcT$LsI*8UGf-$0F_5x9zJZ1TEAu<~qP z(!)n7|M$`6{^NIYBp{-=n(TT+v0G4kw^To9NS>vi=mq%n3Krzdgnd^-s(CEP-v>ON%Y`tshD- zp0)W71Id}`Y*^W$?{@+7-`Qb)f20dm<$}p>{TfDCyG8)4tv2x0e!{ojXuzwjWoLNU z>I@&$Y^0=fg`(e>Mnk*Nimi4d)_XSp_>~#`>xSNL_{X!If`8ntEtu}EULaiHZSq6g z;f2iT_<1zVoF*8*TkA7h9^YxA?knM?6O*FFMxRuuaH!vsmGbi>F zC$25(mrkb>;Jd!bCZ{5`v9q4edT`}hwLg`_7Zfln-a&bJH+Wbar!Qt{tKPX z(vM>Y_v1*&T6qKFIp41zU|{)McMJ$Hd6EIhL_kU_cZaCvC}te}#*Z8poii$J91uiY z{UUwI4jKLyj;YJ}EeNYmHp1$;tBZL_ATzAf&j9BQ@pALP-NVJ&2l*#IRM;I)_D@)z z4ey=aPpkOxh&Dz5yc-d4e}M@rdl0L4WCX(hv}Hpe{Gn|~!02vcjYt)1VQ87-v95>d z#(MZt#(EeYdaz+w=)qrho-+ck&{@Yn6 zXxu28pvbaq@&0oy{5nK_5KCxF8o0OhNwCBhss&3-so@BpGs4llCxBk)ddGj6QnuQm zYVX8$e~UM-*-F`sq9sHpueEZUhzMUE`_o3z3n9MSW;C+6K9~0~hc6s(#CKx65q;4N z`1n2SxVhdsZI9?HX@&B%eKB|Nw5hq4X)D0ANfi_MnA=#R7v&oEZkG9rpPHNY&;A4R z7ylc8Nh0_eZq4hUt+&vv`3mK-F7D^?O%P@as{<9cAe&Mj?Zp~d4(m)?R^!nk(fy+} z?`|}4MgwQBGub&2bS6q5d=wAa)8o^<4Q{QJ+F=r7uT6bi|a(XYi*#;ovbFEkhq{Z1~GS-)QBx&A6`}?)=4ZN)N$ognBtv;K? z&vy@}#gFI#kCs-Q=b;0iA0322YoN`nu;ts_e;r`uxNnf4;{+8FL#?qLl|q|& zd>h136+UY8QFJgwy9VuLAL*u_7IG;On;jIf*)MY|2bN9#^HPZ_m&ZQ#Z$d0}7LZH| zjE~RB#ab2#DhoJ2MRC3ApM&l<5OZ%oEFK5xVx$82CzxUmNqbEjQcdn44D|W*3^ZB4 zy#Y;4acf@=7S_88{X&s!#k$vfinfy^2Df+#4x}x!D#J3XbvDIq>|>9nJ5lYCqi@pk ztJGRe%WQDC7~Q@Yf!ffCo8|Cgt98cfvCY(0+fwFP8?UZ5HqY8J%(J!(^Q2yYd`^U)P2|xc?q0J-x zZuZ0GTLRz58ehkIEWb(I@;zr$NuyhgH8uH@t+<*3i+?|0!R1xx*Z{ThE(H&w|uhKvJi3zO$BJErrg4hZi0j zv4~CO+hr=>7o5N;*30o$rdna=y~O&d@BT7$vlBN4s^3RbKay2bZ?kG}53mMzvuVw) zjK9eDc-gRdTu?6EENl#$@@BIzbZHdL!eBLs--I!y(rS_U9~T|3{(JDpUpbacew5kF z_W=F)(n0Yg$p632g}nZBPd++``Q38$T(MPD&z)}koo4) z)`n(p%)$0lGwoBcF$~Ae{iqM+aN3(c&l%}lEC|=%oiAFiAtdw9(T;3*ufBS+*|U@W zivDH^w9_L^4*I6g-s^to5P-mZ;ji08UZt#g8)7AEAPYo|>*WxFWeXf`SJj$gfZK_# zQXVWdIXvTUcx>#t^^(o2-R)5HF?2!>7S!04@HG?DjdPE|P^i^i zm#2y{Qu$W0eXkN$kN$)CRz6o0`90eP<@a!Za#x$k@0migv1?DpY*-R}cJ?}Kou{N} z%J3^%b?k;M1EjU>A7Y-)wdSXzM=TaGJkm>9xD6oHWmt8`{^8Y~HJGoT4X^Gj|7i7{ zi-o?E*q707^Ez{W#)&&BhnXMNc82!b%+PoI7le8r90gTdrioJjZcW4z$~Ky9c0v#T zRL04#@JaOOjOD-QuxV|n{C?3L*N+UW6CKuDgl-PV{?2+|_`!)j*J1xBxc}j5Rw6#0 z%KUbxr09&Pv4#+&s5;frd9f1ame-RzFQJG3IOE|SemM5y_}93)jjz?No{TM>bmkkT z$z_eu_h^-dnjVx$SC_+|h5EIVd!{>RLnNi-DwdWQ)ForUQCCe@jIPxp&$lQ!T)hO# zYY6Gy`@V)!l>s*?BGI=;dt7%r*3R~Z%k8WM4aAPE;$M4}Qbsm`>t&00ywan^E#mQ# zAjIQ?j^M#L%+6Z7+`M1WPxEMR%K3SXV#N$yB_q^VpuR>6?3P8SOR8AKnG?I%8agSD z5bMh5M3+5=Gi<)^Fmar;y!q(DITxGZoQ(DJp~}y)^sCK`g?*ZxBHR#~b5n?}k6l53 zd+m0q0-leQrd#cG2EK6mt;(eChSPBzp6k=S#7@gUK?`SvYtZLd3cXpU0mCEluDb54r4%866@EzH%j zj`uS*Dz^wm>Sp%x(WhhJFWtMlv<<%o>&dNcr@`ZamElwY3<md(hQ)G--ZB z-HqFD)i>!XQw_9icSe02S#4{i^(zhrbktVBjjc=NjXlS>9M+6;co%+K#7@39u~w=M7{t>!U8V zTyp)OOV&R+kW|CZi=5&X(qmMuJNC2Fk#T-7Per6|Z*1|Qe2ngmmFdS6Se~&h*0V;k z#m1En=66L5J1~uYj3aQmf`7--TS)rHBcTED!a7o;h^gXp3Isu7b!#%}Id8aLExX16 zhjYbBMy8VJImXW7lNLl%J4+<9KOJ_KHp2z1Rb_TV??`n+Pw}U*vyd?HoSU^;LFmu_ z`u!!6?uYJaAJPv!8-Hf&hhCF?f5{WE{W|O~5wX9B<&PZ#Kt_-8ir4MzT(5B28Scl; zzJ)){#j@qX1p41PCXln&izREk#2HbgXB$M2wfR7AbgnFUD`DtI6v^9&4|U6>JBk0h zl*E2H(T0flq*ffS#y_-@@1+0yjmxFh$}5sdn=KH~+C__)>>ufKDqH7xujXP zABWGm@rayXr6jB5B*~?LxzfX{nRCrIOlroHx8a`}yqnpCheR(TlC?(oMN+7OS5_DE zFA4Fc11!Y%HL?(Ir*ALb4q?&x_~t&L>VkB~JfDoPGnRFU08= z#OVjc>4(MX-{bTJIQ^tJeS>&+nK=Cd-i>16*!}lsE~p{;*GvwPqPJSaqr(HSq(-0T z@RF`r`mz>vOIPpIe49Oz+6CXlr+y=1Wr^RwIbDqyPEx9fF&UO^@o0fRC}n$T+ecS; zEczecDYSuzEoI+6Dh_Ro%KEY`M$`xrYQ3*jfP?~DHUbo54cKNVf|v~Lo74?eJK)g+ zn@KV7YO&ZQU&bymB^uXqH`#D=*<3{hgeIU1^s>sIrNFp7T$aZ7~CQ?3Cr; zL;ODWb~h?y#2GA-0Zpic?>~W_bwWFJYh5CVgQ)g6{pi06fss(cYi1^$y__Eh&;!T< z2FIM*(<88A1yrnw6q~*Ch_tqdE?>KpFJElJBrG4U?})T^F&;_Rr%+e|4*`J%2_GCo z!pp_eJ_tr8-c3XGxf-R4H9X-a75(Qd1QJR~HB($Ks#Qx!qdT@xE2>8whH7u|U+pWc z6Z7G(bu0`ou6H<-M*i;7V*=$u9pF6s{B~JWk0Iamm|PMSP}9fd{CZg{K9)kLd&Jx7 ztX&@kM|rfH9JGJZ?3uw=27(zb_=;7N8GO}kLt^GPn}JvVz5liG@_~$aIq7FJv*~9U z>zfoqwe0J(`MoU)TxG4vZC#fwOzE@YGT)J!dCC2ePE{_!6n7DH!uJ&)E~S>OC{(KLErA>@v> zWcQlSxP!^UH8aApJ=mYK`eJ(F`c2G(aXA{F3WmZDXqCYOlj$S7QsgpJ z8OM6QEXJwj*o=I%^e`DWNGFoK35%jV*BA`OTY=!Xh@K^4zcZfU&9N5JN)=CV2FfS^wa1_NT|$F9Ol@W2?}|GCD9vcGk!xUBN?za()XwYf-x0 z!LTxW%L4jD2>wCDY&r z&hng-F+qjy(pN=eW1?-|RIQyh2UYiNiXHw+t|-u3K+|+Q&1(<^tnpzikM0&Dlpd|; zSHEf*RLkvbMBZZ=Dj+QWdb?#r+nKYmcg-9!DtGC=!2|PZ{jq(^cw9gLLlfb4qLWp(aToZ^;wo(mnQ4N4?vPEiDTb&HVMm3 zyTPF=g>LQYLgS78$HaSP#DQ%#ObiGQ@|RidnQZxC^Zxn#Ft@hU5*ivL_(=aba}Vo3 zM`~FAnLyuOybei0_us?nQC~Zs-X%`wz<#oPp_Am9)rnqLZW09_&?=YTtV}PI7uu$`qkd=2lN2=Ip zOJ7@vYYUBQKW@3Ui>|f%?=r8Oi0dXA*X3HS>qA86D)X|*xNNd<+28%~9V*NZR(uj!Bt+OF`;m(N0 zeiY!;2mfr+^dMvb?oRsWf?IhxPLcIopuU7ZYo}dO)n?apjyIBsX{W&-WYhP0%)GK8 z?#LnghgpU@}SPX0bPdJ4PA7A;2 zA^Vs=ki^GA@Dkq9!$53X6!w3@#(MJez=eyg{F0D+tJ!~d(&6{gzj*O-5%PG zdCP!PIwD68fzHv)WgTDBK86nu?41ji2U;5 zbvgeH!{D!6`OoQ(Rt-@*!VwrIwN_d0DY!{$d$M+6_m%fVn!MhG&dJu}yS; zo?Pu2)5p)Q9wTJ&F}tYiQZ-;km2V+yhNd{Y(t%QTSsxo~yB9f4w>K=(^4?OlyK0o+ z$=nqxM4}qLcL0J=4c{?Q1vok6UWPm-gJ4SGLS3EWV9)N+*ca{@NYaBYRDv%TBF9Kc z?hNcf+i8+Is$RZDbfLbh#=nMPh#OG4-Yu6lb&fLnT^4Ou#?yaV(RLZlxJA8ilL>|O zX6%s_7;QHhk_~`k9@KwR(erjImlyYsWN_n&YZ*zMYow8{83`1`v8{+gi~C`BAIETxi$c79 zF2w5}(eg$v=$*$46;}%hoZ0XOHL<$0|F5j>RKi)b6Y9>_zKkGCUNbk>DQHZAEjP=_ zO5A;r=K3IOFW_C|RK=E_OfpmHfd=EqLKNGDXCLPJ3mr?I4_|T;ND&dyF&er}X>VuF za?kHw(+To+c zqlMSKgT4DbL;qQt(toN$f1xvoY(F&c)6;CWwPoT+9`>?_PR<=ghprWeMu|fm^zyY= zh(n{rAtYF=y+|AyBM!l@w)RYM$RQ3PLMc6}o{mg*X=7*?oe^0@Kl_`JRkW3LuxJ*iIcBzqf_Gv`(kG>X+rCbuVbD-Y1QjYq`9ZrIB8T5;kg1uEPM}N zxGVPC8w@9pfXc5WWRcFiL8Fh$J z1^?iH)J6jjbaVW6xB6Mstf9$KY0U7>7h>3)$5lXo^4Oux0wo-NChv!av(cdTz1dxzaUl zBC;Veo0A^xNKBr?VfKm(=~ug@7d&4afMF-7>C@hF1HQm!$hY~@3p2hBt+@FO(gfyd z6?Zj&G|870*Q+ZI(5_1=x?2hhb1npKQ_GMgM2F`FuA>iw<1-r2rI0^zrW}@$KT?VO z5ex^;tze&B3jKt2_c7q7L_-`2FX@L2d%yo1bZ_8(abM?ssd!B<|MJxSExl0T!!MBS zUG&puUrn+zk$_IPY67SAw6AzIKFw0!!T9d8o(E}GJs_&mWG z=^T^(F6V}@0A)SQ?CGkJrFpbN@i!DvWyDj5|2^m9_spf=qv{U+4Ln}O)xX*s|EBm! z9u8e)o}AWsXU6_e9Zi}AAkZJo>sDQ5p@bj;QezDow0B-;?r-VeFMSi0vYnqCl)sRS zwJjE_sCBGJ{h3KBoX17>8EmK_+*dc7xi9n`6kI+tb`byan%!hit!`(N_1?#3i|R6| z;#-&rw)$9N(E{P!011!n-xb5TRb(lmFiqcx%VIzNf;hw2eELQRPW|6wsF0CxUNVLh+k3ruU_8zK!I^`WpU5*ZeTct={#BV&=s@w$H!PZEoG?SMN; z#x&@4C(c5dV&U%z-(pWm_72|2p5>(@C@Pmt2ZS;|;&ANvwSwfOV<9B+rYdFE2q}FR zdsxpjr8IEj21#jdC7P1bQQL=-(mgho(jQVjZ?z5LZ&Eqs@YV+YF$4O>zlcp))t(c= z_iwSzu`C5A)s7A0Dv!3GM6#+sXW1;Jik3cFJ0Cj*t7HI3X(;DN#n1EhAl|-*`#|mC zK2Uo`#;TCk7PP@EXk)x_06zT*nckuO>~mOL*C#i=DQn!U%AEQ8So2b+I^6n=g>G%} z)Iw8UXmyZh1#);HgjoX#v$t^G0fvIV#wUuLx*+o7%s+DhyOjs&6QbhUNV|* zmXh0MJ%tgkSr`B1=s~KMF^Tc-0PMwNpXDC~{~YTb#6NB>1GtVD=K7KO<$7Wlq>~Eg zC=%MmHX!R;q2uY#Lz#`OyG+dBvOKXDJSBeRUYe8jLf{as?qP-vG0FbA!H^w!1{CEX zS=$rSmWZR{^H{ctU7QUN)_Mdy(8BY zL#wS~s78U{$%mi{`-vSHmaXvIfa3|(k9#SjD!i5kcpMMKb8pV5cD zXb$6$2rmmU)NM;`T8Cq$KK>CQIL+ec=4P+4HEOJ7YrV`A!{2ww@J_y0hjH?4u{imV zgY;TIap3Y%76AsvYvV+2X1q3LytelT+2Mbe?n$xU3(R`IVbSHOF`DAk^%)bZEkp^R zHm^|Db{c$lby5kgISRhhRyabvficzd#L8v{iFM2sQ~n+j0bM8jb5pKfM&2ef8@u`K z3&mBtFt?~AL?XSu5!;uR<0Vez%dEBzlw=?UciL zb(F3_?=ra2>de0nskF4dLB1hku!+lA#<8D`7;5FF#;}%tW zwma-;9VwaCpnhl&?zoSTy>7^V zDJ0Q05`R1=r5;%oFz8<^jWRjRQ>Oa$kZpLtm&u*Wv#luMk6$GG@&A(bnO>gOAN3iM z;PJn5hFovzcco|{g+A3a{11I^$PYbe8&V7SCG&@_jwALf;%l9cll5qqzI}xCwNnrO zI>Y(!Wn8!AEeNj%)7zObEYsBx+K4UgOUdlD)gJ;c=O2h?Si%|FzF2()=cE2mNjpeF z`;Lg5?uWL+on>*S(eJ)&%h@oPJ_h~ani?&PZ9L^#qVqXY=m&4}p^AThn>$!k|0)Z8 zxV(tCpa$-pFUhL$dcRv!WSA0l6d9bq~51)RH(Ax#bcn;d+r(5FQ?s}~UW19G_h zgjFy?{2Y(|arJfw^+< zXbw}inqO)(?YU0WR{VlRqkk@!i|K{9R^)<*evQ*Vz%9X}xqSbb_VoQy@L5w4iNl_9 zMmoza6|*btM*b{KQKYJUpXF(rob3WMJ{SoL+C~Ta6P|Y)$ZEt)7}X74PHHB$*SS=@AVhj zHq4h^D5h)Q(e}oiO}u)fiodX*pvvbOSV3F(R`8OIjmWz;`-?fI^b)(O^o=-Wm^ui! z1j~2Z0(XOL0?P-I)!y9@DOa!u{d4pU?(-|ydA^9g!E=6v;Wzj0hSU5CcA0;QzJXP* zU~hQ|eS;(W3ZxRO9Utr)B;S0yIqbQ?7kb;TI6LCsvl$iP+FnKz3^UYV#ZPj~%6g~l z+{=vhSWbG`OTja8%<}NB(kJ19pZrI|1s}2h3taGu_A*LMI@Tg;`A+O->kKdae2zU= z(Zamv7G9XjZL~0-`PC`(dBU>QScVBGl1+<*rgH%gW?2MeKJ9i|o!$!M;$43RvYM*# zXxV)9CxjnNVA9d4UXQl*Hm(N(Kv=Bd1>wW@ZD0+G-Cgg*PFZL4`*f5Ahfh`Y`Lg}7J=3G~wO?V`93GF3 z(ec=L7}leC7H>a+-zIn?zH2-y0#lw&1C>}p5Ve1HkR*xEwBGSOJTVfV3hizxSQw^P!-RaAp1Uxd*CdDUb;zzNDfn$u z%6hbhQnn=%t*X6MCMx*PK*pwC+>JyoGAbd9l99mMuu)Y#utj>fkol-pE=|Z`&(Vo; zSnZw&WMF?|Oir}On!h22FI~o$E_~@TzKCXi_&lRQNX6E`%kh_NBD)%p{^18tAo6Fz z6h(~ua{tXnth-v|mUk?dRz?3!4+c+W&WKaJ1N%s4-d&(9>2 zP^5|f5jX(|S6{^C=)rnCG;b2_pHxV1YN9(U=|AFuwRvaZ5oZ;WwaxSh-98S6)r}4_ ze@WKMY2_SJwPUg50x@|PqIRgvyPMW304S=xXt5{SN^g+E-gWT$-?drQw~CS@ZdLiB zXtTsEaJBB(?(xEx@%rhUx?jW90We5ra(jZ;d`4>M#gecU{~}(wj$SF6B3yCw3oL!= zZzkz;!Ou^zsYvK8n{aT&1e@E{q-Y)G=g9h{JLs#THTX&!6fz=LpG7lG?3o8|dacLR zBt3c>6vuNoCAONIG=vwN5c#?&&39|HFhIHP&4N1WBfWPml0qt>l0RqNKaJfqD9P^2zW?ajondkdB5Yona}E=LO2 z#$i_C1FdN&u1&4t>hnujA!7>$;9YSY3%m<6mCLzZ}&ecQPKpjJGbVy1~nW2>k4c_9)_5Zi2nk-WG`N_my1S-aPJ&2 zsJOVzd~n@RiV%MQ<)pYpzNEMT6&~0Cj;(pytIsd==%?Oy_XIM?q9o3@sAXStcwKwS zC&~I5K5d_`v{fNNyZRg~yGZ?!`Jd)!2rzg{;N-yY(x*GCDoo9QY3nr@bfE3S95@K=rE zN(PEZKLx!6E!JEZN9Rdekj7zoY~C?uz2kv*4RVD)&W_e4HL^a=r?vV@_c3MBsf@*0 znuuKIG(06!EA(|^g%TC_456a2LO)6hcKlNm%r{8Yd4_7=H<)a+V%2;i+a9l^?(v2? z9xMUnRQ)qx%d%!SkYmpxios1#UK2;YBz@!0Nfvc2*t`uwe$i58ngSZ^@CVW)TWSb3 z;Lmas;-YHCOt0n~vD>N{0shWvUd@%()m)jenkzF_bERc9M+DyQGo7cT-$Mh2sDnOG z1gb3+Kz;2?bX9C>l)S7!pr)oa!GLl{>)?*9a{4?RNcq7p+MhriMvoKtC91g zZt3wV_82bNgf7XtcXRBugF+klXq=OM)yRI}^6rFvdjA zy#Pf;kPB^1H_=9q@}9?yhwaYi@}5gV(EHW08qtqON!BVQQMGKoL)Di(snAaEb~TX7 zP(Fz^D@AMal~yVc!{@3RE&n)z2F?y8+CRtRs+1n(IQGps0xU7OpQMa@0`UqHx%s#| z2ra39kd*02Ng1{3o;0TOnzIK5E>dX=A)w@YXScR z{Z28-)HzkQH<(`r`8Sw;iYr2rI=`RTlJtZI=m`^;h+~LB)wGWAHcOY3j<8K79YM9| z2zBn@0lROIj&Rn0nU2te-}eve3H2)J2_=L3F%*#2P|$!1P?V{Y)&P83QXHH$@{?XN zwDQ|Xwp4nwnnZ)Uq!ML`_Zm(CAi&y1Tfb4 zo zkMy5o$zRlq8->-d*IfU;d=jPJ^`iCCC32x$7IoIj`CZcEQI-Oss1*87JF5Z7h)>(4 zd3Vq9RV2m?ETrHp@3xtm`J4~G;wA3?ZT&)>wI zVLrzX}N=-CgZX zh5P<~2f}^c4e?(dMpjTH@aYsCY?*XG&spE5c+UWwfp=~0d|M9gw8s~|#m=1Z-{z$H zw;to9`ZMX~X%N-`c)CzsUWYOOK&TA^qcQkdT@m*7sh zWRIMG47Cfp55f0BH#qWl$|d{w{mtCDXb-(%cD6UtI?FqwK0%c^&cKA0D-$3Es)x z`OEDF^x+d(PNL;W?6hYb0=G-Hgx;JTot^Yko|N^O3&S(sW2%%Mk;4-Ees{^t)_C$M>+Ho?ck8bblWg@`xG`fM=e~740NmieWr8arK?GDp|RZre$XzHBu6!;NF z8A7}zm+VsV%Wd)$?~@p; zzU1kJQ#=RyoV^P*`UB>=Sk-T@AtejF0^Khr8IF2cga6|wnWuG;`T_Cpa{f#Nm#43b zZ-x*SH3RNMgC|rag(Qw>Uh#;=!u?kMbLtQ1&x=S{k1d;y7TCTmhybM@zq-IEafp|& zIa(|5S=}cXy6_!h;stj0L=RzN4L4z1()tD>(jVJQCO)3NaQTM+BL1CEnoe+kK`?S6 z63a=ACeg7$4(A#CS|j*%RGME$S@?BZ7Jl8SxF7?9wU#EE%2@y3h+m_MYX|t1*)U1l zNfjggqfTf^M0PeRS_BS%XGF=b!M`P43NaNB9F;$rlsD>gJ+FcvJ0L#5k5Mflhwn{- z`@xU3a>-7CU{cBa*pO%-g%%wr8?gDH74ra%UUB`M`0>bj%uIqED!NaS*5Kxt)BLdK zO1tKE2oTk}W2__G`Yq;@&+wm9ze0argyj;u^pjF%KN;lj$eGzY#E*&=jXn4(5{In` zJVWa3paP+*#Fc1a#a$)*mxx7<0G3?seKOZnX~JlG>s=?al4B@3>)ilf<31DTeKP~b zcr|YW)1kJE`H_QxSRdMs*IBj#pZ2n>)w{J8Kz@Y->ku(LHVJ#$gTM8Xv3ds%)Otyc zd?}r(aJ~Dbbf&^p?w8Vu3NH6c={$vt-7lrn6wY(Ml+IEZXJ#q5UqYIKN4kd5#9q5u z6r%m|77owL+Oo}3#e<^(fYL{MrTY{{xw$C$9kP__uB1&Ptae~dnI4?Xz^%#Gz$dT z6*Hm;sE(@v-wYnG)7#t0UfUe&y4lDps2wFTn;{~!5X_F)Gv|qy{+9c9^Dzd7 zxdbWB9_`K8zfWKcB{C~;4jHs`LR;uyKtoZ(B6&;2#hhHkx1#?j9>U= zrX)9OXL|8h@6-6KiqBi{d9L`3SSZqWG0acUA*aYu7|kQ~JAadW8#|V;2^P;O_b{K>9F9;oYRqbuesG z`P1NS#-Z$IPjcdZ5#+szuGg2WBeH=ufTMfrF7d)E$B4DZ!%MHmVOJrVy)YS=Y&6Zgomo;ZuZf*` zj(FWos5UCQ-myi@T@xS}NSD)HMt2t=Q*~b?>P!KcHs3)q68YV3HZ6=~6G79T{lF;I z;%-&w(YW-AtIe z+ByZyW)@bw^ZSBqf6E(=Y+o(PN;dS={U1oSskv6NodO#GeEiYAHS`KMdD-|lhs7y<{a6%D~BJsLXB1#xLtg2-;~9OZ;yiuqsVCAXOy z?fFj;&Hgs^4WFJ3bOCF6_K1#@;ghKUWd$PXtm$}n0y=V_Ms@XY&f*OwUR`pcV8*#= zygL6hT$uu^yCnA71(MAW2A$(LcfY(L{wlXs#+B$K#Z92DD&!O!(Tk_wr^R9qJt3Zc z09nls05SGNePeJY&lhg8v2EM7Hyc|UZEPnSJ8x{;wrz7`+qTX3=J&t1>VBB&dd}%S zeQKs_rl*gVF}FT%^|z7;O#Y-X(h&V!aq25&`e*;zXjcI&~h3m4+ z!gI)1=fN|~NtTGF6wv4-8dq7v3hXc@aZ?umRqubD-z==GvCY4f{-wfJAg@|Hx!_je zPrb?u*>>|Oe@wn*xA6l@E)c}xjtuU0 zBk|kzD=QQvKN)^?eVuP2WkpgZSS@eJKI>%I=a_@}e;w9YZnVS>y_XS9-`#}R&nnoX z^nd?b%>NZ{<-5p2yX0fS+1S zf6;ulcPCW#c+_7`_ebeCz^-T#b|iLNnzaRhJ^j4m4G|DSv#DaPU!mh*0IPD+8Ow%M zzz*6K7stb+;(MQaTB?5`0o62l1Tu7Qjlbr7-To!aq=S0D^efXfQ~7|D|KX2`=K*5S zAwN-&YvGlXRd>pB>iB5k)TE{1YxbAATo;CrKQFK+HZDdfY>po8%2gEn+Jb8M1xKzg zzSLfg)f*)Si?npj05YYVDt))q@<5foDm#o@#(8aFWYI!~$aTYDP<+91U@8Wh4|vQD z*&`!$frE~OfYmdxmL)kx&pRC|&mcH03Y=1jPx5P)3a*w*hjj%X^Iwc?p4ae58(I&+=V za9c8J(Vw`ICta~W;}nu;ZIg(63dnuON-Ua|Wwf>XhFncd)OBt*om0HUL=VVlW{P{0 zG0~|jXMXe}2(%8(VP{2-fWCDZa zt%p|le!1h&M&EXB&TsT%AW*tebNCy-OF+g5mX4WsxS>04T9}`cY0NdWs@D`bOg)yK zlzyjY>as^cs3p0Z!ge*jg2r6n&huO1rUysU+X1Sydc|HUe zTs=kmXu6gE$$2$gi>>}3RJez0von-5aS?=4eWERDNjIzcYbzpCMhuXVqowX=nuu*m z*^!7vRV_XMkD}pegn+JTviXt9_~g=sU}vT7AckPP{sT@7$H_ZNwvS}Ag~oSbq)q@# z;i;c7V+ZEEz-S6GlWwfQCMv}b>*tS5XC#`Hi4gy;$M&bMFAmzyDn!_TN7AW{lKmNx z5ltcjNLMqo%D!&WY&vmMd~K1rKu3=8nPDTbp{1wNZX3_v+s#?0>&rWd0@A@Wp zm=eArTy2w}SAVrSn%*<%qyhL+eX2Zv_bdspLKrCWE<(vn6}dT$YF$KVZ~peK&DUFx zdTYX6@SPonab*7^_0=$8B00M%*oTiCKfSBRX~D_H9@1{{;2ggXML=s^X9&TnK%7;F(W^$c?A^}JUmcc%s#Sf=oqm(d8O8bm1B9FLsx zHm#XOq(r@pN)(Y-)8fqX5L6%X2uBV^QvMm8*8imXp?HIw(WHN%AZW5Hvj#>3Cc8Oy5^@;=#b#8-Rw>Cp@%LVk?g*KTGHS^DT2wh z+6~P{Vld7s3>(2H_6hfDUPutnZzhig|3_XP^JZ%$8UG^Y5#!JU3Agg+(-HofO5*tQ z6S(|=x9Q>$KBv6HelB;GI}-0$2^^ESBYv6D41UZX($%eX0U$zvd>Dx5FCVzIK{#4M zf({NMq^h9GPS=yN%qtM!1tV{W)4ATTOUb!wCNYMIiSj)RkOTAbCd78g5YBY|k-vq5 zI+qxnnWvMepOR9x*09OW_-{LOrjgNZX7a$!MaVkTEgKZ-%3hS_Nwy@^+!!1uMQGTa zcfmy8GoHpC`CEg)dXDPPx}Ncf`kB*GUH9TYq1=Ceg0L4)A7ODB;hdX(obLn!rv0y| z_a4>&k{D8w-aLS65Ub*LXOE z$iYJpQ?C8Q>Y5GxcUkQyueWC5>7Lc}EL|wD56BgG?6#qpP}p#NQJ{iN*2#o2O7e_G z+z;>M{=F3!t9dz5eW)u|H{VWzMMIo{kH@LT_weX8OJpo}wW#;)s-+Y#Fvq@5?uO7I z;U1FNt1a^i$28ZjvCcY{ksT(;@cCjkLVewdhEsQ?fbM=58d<-O!an4FA&K^J>1xc| z(gORnEPvcB(NGj{gBQ3xe=?D`jPmjRfHG7x0Q#KLIW*00w09e}$VaYp6K%aebhZ)f z&~+L96gs{)9WMgBqe--G2Add!gA-q8H`!(WW3DN z_347P9@#ccA;&!<+LYz*fuv|c^?LB>&wZUisVgm%do;&1gcMB@;uRQ>Syz?@i+SxWCu~`brvc2Uvx|1b zO~s|*FB>hmyhZ;KrLgoWY>p}tnrKMRC{2-Fkcn4?l6%q~R2(TH!|+pJF^?!we7q*w z^ps-QdWdxUi>j3EPg=knaKiRMs3C(Ke+%_50JO<*s3lImzZogZ_a@&4kU*-J;XQ=l zD3NcIrYA4jvc)NL>hBroAT*4Qc^=f|Lr7NsM&PadL%MSSonpnn_aiXUWOqP+8d<$o zb(P0*PxI&HbNJ&IafcSaTc!P%@pPC$b&l4M?w-k>m;ESlCPt zT@&TdT01UTq64F?U9cu$i~r83tL}6Cc@j`BBm@?Ap{_8@ZSLl$wh5(FE2U+<0HJpx zRi}Q7X-vBX$#u3-5Wqzq?_dqLC7DdsQ!2@;gSw)#KV?Uqa$HPj7Fvm0ePRU>)S1|a z-|8S%^jl2BzIToqNRwS=oINKYlB(Y;zyb!A7TcW1yRSS;cYd0#^^R?XA!`Zy)xt>W zQS?NcyR^Lk5d5Nx(9;yYrDOTBnp7CBc>v%#?P04D|;kR~N<8 zG^@EbTPhA_Yn?q@ty$0TM0jL-PI%wCbP3_-->tY>fIL3Jt&S*jto-2cDzNP6H~ z$o1c|cgYo#{fLjpjjR}E9y17HcTBFD{mcNs@=gh1yYJ980>sr zg=iy#KbGE4$Hm#}v3)&T8Bd|Lb&FMTWz>d@(o}Ep52>vw1YvISIw zbG@c!)Sx`2)T%=Q^9sSIx8k#Vg41i5UwKEE$3B4*O-Oc?Y`+Eav@oY;$ht@a0?GIpOS?q^OvoZM1pbt|g+YA_|mrYJxktnr&`4qbCXrm2~t*VfKOuH#0 z=_mI71IZ)*n#eR^M89LIOkke^??1^n=}IcDG2~G}D_JNZaJe8IOGj`HBXR~Z-K5}1c$FYm+rpB??Q3_g$;A$G5K`M!n{uvZc(}s_m^m!f{Z3jiiylqFvZqBlN^+EV%wpz_y|9z-^=W;y z;V5N;TSZDt7&xv!aeb^CCX#W#K6-D~l-;@b7;HbDiRqlsZ!G7;hpjz41+~xKCd(5V zGSIi`VY-75Ai@QPU=y0kxfa+oOIaDkX<*+9Ga&T%T$WM$z96T8Hk6zT7jh!J3m2Q# zwx_2{OqH7?ht8Ge9><_zl0=z4cuR@5Cgi*an#cnp7Y;6wpA+i00T1Ecp)v~OSZii! zc2P92qQcP1>G`(Lftl|Im2TW|7QD5bcJcLMMYHJkyD=>Noy3p6n@9<+9UG~WR-3~bZ$ow{6&eGY9=)#wsE{ymUXkXL?HC&WjToRVO;TE>@PnEYM-7!tUWLArDQ z%OAX{Z6FapaSr0iOa~xz9dwv!Yapow6`1`O+q@h5*jw8L!lw#9a9PQ-tNa@pdLZQ*)cKj~oJ+Sb>OrPPg8tYEn0#COaL zIheUBnc*S&)YoUrpQEP9RqPf#j!6_G##CBzs%jP;JgGR?HR|h={w;-*B~~VhZO@$j zxYBRtoi^It6Pc{NS|*<-={(R5@I%1B!>FKX#bM>B-udFnC-^s&m3mpZ7rsropRhnh zXg%G0+sN`F*Dx%{7H>cKtkfbYI?R!BPX26CFWsO@C-3mgZjj4wkN*S+DkLKtv)s~4 z87%;#BEWI)1(%3lZ|D_G8SJFt@i*HN%WbneUB`-=85ivAp-Cw8WkRPRCU#@QHhmYFkzpsRWCw*bXVYAyPZW$1DX_qZ zVWtanizpn;>gJ}Ekf}G2?)73Iy<_|&a!>yt^&ywWq1_`kQ(0CZRivZ>s=xs(JCHCE z7V%-V@!#oLpGMPwr&ESO>ks^h_ShD!n|8!>erFm&Km-$ZNUhG#a7%s%7#mUdOE2JeE>4I`>Va8AB8WIL|t?xHsZWS-#g7Whm$uniuzlm&9?{InX7vjLMbTd1Y7CptEx=Uds-3KJ(UfNx!@DGAOz|KV z&6Dkb=4jc07#DPNbcvTJTdjj~77fj#`~b9MHS(eyQzvEd7 zck(3c4JFtQobgk%EZXvaX+L<>Zizzc9T^UKv^JsI#Oaj0p~wPG)|p8KrZ#o5xfnE| zVmyALZ}s^7rbcU~p)nVy?I|&CkFl9jdI8tmM-494%F7 zU7~KZ-R26x6(lLSp1kb?7XnJ?A23xwK2)fs&MF9D_W`Oq%Tc(8b?%3G#O&FHc$+>e z(g;`OpYv~d?UG+4X`e!ghkLUWZp_JAB<~7VqJDKI$lBp)wrsgbZF(U`SAcD;%GV8o z>1A1_28`gtOs`EfFQ=XM_{MW`B#CfoN$+O7KJB8?y1hnTlsg9JbpfdK1lQ?pl+j z(@hGEZt?6boX8kyJWznqa3jGeBPH(4p6ry$v+%a6!Su3QCF6QoETLilS+s&M9a`u& zDohkWRAt}?A!AZS#G^ngb@)@>O6mmTopyWsYktNI%i_DdGXz%L7ULY^8R=y%<5!`# zpE!!|)@MnWQky-d>DPi0+i=^Qhlxb|2 zK{hR(Jx!@juXu;S2H$A6o3HnxEB;3W0XoTq{PcIJH6!0!T+dCJOem)hb_c>4mWV8< z$O*r`f_o6wxdq5Pv*=d(=^X^hTs|k9{l-j}Rhl0N^)YB8e@@53&#~g%4h6t1x*1Vx zz03r{ktNgfRS?u{x$Vrjrkb4^-`IgX2o{rS~CL&qkVd!5i`_J%}?{3moAT}pW` zb6XkFeG2$FLbpoo8Bw&a$rB9*!(wAj;AF|KjNOk&>X%CWzx@ZsL)-Wm=QywODygcg zVLiE0`|h`y#LeGQcNvndw7^;7Qi^vlIwhYKIq|T!;8eHoVjm684BT}~HeYz0L|Dv! zUUMrhF+%pgQexL$=Aos(vz!$FM0U-S45)^LWd(7uB_J9(w=yLOcmbrpZT=Q&My355 z&X<|LO|t~M3RSw}o$gPMJLW~t-AjUznwf0695}m`xQ?HkzFGibH->;X9<(|z8xem* z=feDJxj37aIX=uz-LQ=6z)rpd)H(M(aO;GH#KBSIrg$4%fcZ|Bs5+e_$zMnYwix{c>db?9%KI?r7NUx40+JlWdanLtF2hzv(KwV0RiC_J7qUf`I3tW1 ztczteaJ5nE8SBiyGs4Rvbit`P@Qh|D+GD74%m^z_s>#!D*Z5(`6*nRj$B#cJnxM~f zozA{yg}qn%d(s6~ci<6G=F%F^*R-xh;)e%F?6xW!Ew`V~W-&|0fAU{azbqE{)xR+!N4)s$1Rt_)Bwmgk6C~+h?m3XromoQu(vy*bu=PYsAvf z8ql@}LMXHsw!bYQnvo_vnK@wUpuwgcw;;ZIuh{sa!yF|ngp`t|zVLOBk5{(Lv^3A6w)r4#bG zaE9z3L6P>^C&Nz!`MT`M^;$kK$pnO2T<8nNv$NUVESJUVrSgel$X7NVxe`W_%MU-{ zin?bQm~dfeF)!g(OoX3CS7^WNAnN?M-Gk_)(k*;VDf_V6Uc|VD<3ZKz>5S+;+nldS zP*}Gy-m)5OhqXusq}r8NA`~IJtI6)vS6*gz2P~KwH7uCHC2e|BLGE@FEW*8S zP1*GmdU(sKkN59eW->TlN5^zWlB7-ka(se^mswd*ovWvZ1HrZbmAieve>;Dig?Rzt zy{0|p?Aw01V;|jx0*dBP)}+~>shz7X;508U&GL9RM&edOUj(k(kPeLDJpdFqZlZj-s`JWuIu}%N z%}4|O>ZY6gg82uY!&B(yg4eqiyS{mVD?ep9;TakiaFZYeeR3Zix~R{+03~iK!I&-{ zv`Av6o*w)BkB&00gWxji$DzuLrRIgYthmGyA#7s`fHCY0%!s}8>tMan+;FkQ;*8s6 zZ2NMLu|xw>JDX9VG2eObZ|CcI_XwlFR)jV?=sNDNV;3T3-x!2>R8BtbccK@(hvBm4 z>r2A}t!2(^2>3|&O-=mvSIXC9mbg4VIxYeQBfT&P-V+qW-@)3KgMRfHE4sQ8(X`Ha z)xX-ZY|010=FAgo*eVa7WWB(QFpje+ZXu^3(4KMMcGi(gmd{(CUH=kB#!=j!M)~~i zaU9y;t!Xuyoi&JHr2Jd6tAn7h$-K2rfH3{ipSbMq?c9oZCYC(F4sX6faab~g!r!ij z*?A8IvQ4u63{H}dRoE;%=(kP&P}?D9HsCpGeBdI$meqWwsgC|rFKc2i?Bh{z4~+0l z>J#0EGnb{J^BOxG#wF_)bI`NJ?evKW#-31;4Yb4tmPx9$6Z3DC*dPp{^duC;DFCW;>BV@Qe!oKsVcqD3`*0txtbq}|Q zQOc#Ddp3iU_t_H)#`KKpjEK}BJfD@L`32KzKm2&skghzXBaytgR#K#>U+>_;>rl`1 zA8rMhWc$IEEO+Qh@8B#cJX@vbwG|+;>mjJco^uLk5;qN?b_8PzL~`?+Pyv3x{T-l~ z27^i>aP1NFh%6(rM=>Q1WtY~T4!-_fxO7HH6ixx%b}b&jT)6+x2b$0DYQ>T^dmBf! zNpvbjFQK-~M#HtKa<1~yl}=bJS3(S@_=Hy&MGKv6g&4wO!=;W69>d{U&U7iGbbG_g zZXx1_v%SEa-_a+qsEe=5EJ2O`iXDT=^t1=T)<)`Zpv}G3>&f`A+gT8{r4z$9IO45l zpTgmpv6Gcz-k|jdQx+ZenQxHw6|A>ar)P)dcMV%W8gY6RY*0tq7T7`l%}aMFE(|fN zsEArDBOn>Cz7#m_wZW^0ym)1*XYMc32o>0G^R%~rQZ&DqARDdkjV^Q8C$U;i&Exm} zAP#2P$2U5Rn}?~bu-2f!OYYhGZMK%WRQ31i5c$=Mps9Aco7OBixpYAW#|lBi@*@k4 zEHSBH4NS0SvzjRW5Zy%Iee<=y%iq;cj+!3pLGa?JxSjnw6UazvC$hv`_tLU($X7e+ z@#c3sKW@9 zxBIFt7VW|}jSsv0TF4nEs|M6rqBF>W9{I^XY_Mo3<2m?Yh||}DnCrrDOvf)Nli0uN zf>FF9ukzH!)f;LW!l>$>Z@1i+*KnU#9wc7>mTBw4y*x*^)X@?c5>zZY*QjnpGpr9z z%vc&%*%}6y-7~b(`;^EBQCZOHN20nvXcREMptixjFuwdS$|8d%alOZ+{3UD|fH3~` zbVG5t`p?~$V7W(!dNsU?SxFy(rFCRA0uPhl&Zbr*<>-gqo`9S$9s2e}NM-h`0-KD7 zyBBCsK7euVAZg6gfc(S_MhE0%JP8>+@B0x+52G6J9U{m4m<-jL6x5&ztcvoy zO>a!aPFEqsA~^%gqxy~c$E+j)Ey{-A>IUrTIbj;tmZ8f_{*l^ z{ybPy<*iA6$aJk=pL{OV@3etxRP$t0GIdtWAQToBZtW^C*J*E~Gkx|)v&w>5Q0E8I zBEeCUzUqh7d0urE-yb45f$m#M2DY2vQ$fbzo?2Eju5 z9NGq}@BJ3xOi_eGmxdS%DX;x<1)a_0Si|w2&I!?15R`XYb&U0k18@Lp3$=L&r5>=C zlw*y~^FQ6U$SqSYSrvmbVACr7TARlwN7S!hYK}L{I)>+YBO$9d$RS9MJ0yfmsxZm4 z&*~SvzFd!Qo{cN|83&VvKmvpnNuwa2)()dQ8x3PP&l!>$3VwCC`^RA23R z)3gb5jR}l`0RmX(UQVo4&^|s@B2M+clz7K`vNGI*M6OXJQ60l^rhBqt_}Mc?!%&F^ zGUJ1p_!F8H2i7VM4lGnej@`-*(R4iX%x09=FpQ)!`+5aMx0An&yARMmuF^q8&+VUc3xl~&U76(?dM@C!WW=wtgSwom2mN=- zi~qI~zo61LDtXKv9qSH$1Tb9Q^R4zsuf4}8>((H?wgsPfFddpSj!&}(b$GX}zCxoHNtc_np;59S?1CcGkJnp~z+KZKxdXTEX*5 z4(>j|F-ay&A=a?abmL5o9Y5#A#>Re*EY%WY76|<~n(Q~TsWqV(vL9+(zN1|xWgM#z z%ewBB)E4 zKyp#5yZ>WYzHmvKRlGi_hgRz9z2e@oIc?tlF=`pW_>g!s)M=4&;kyh&AV8tFQM9ay zXU0?pjEP-CfKVm+u=wM$LqT0mw!O4Uf-SF(h*?z zZXQwiYlI%q$_W%n25iIp!_xd>YI+^l{K88Gf{z2lGV~_ykc_%}?|`|Jt_>f23cq|f z$C(z7gsOXEUPtMHrJR8LIlu(cQSH}xyTtt^eNRi|r`ao))%~^U>$KtjWK#+mxXbXq zjfOcs_dlj%_^?9<9oO}bmoXnSW#)DsYRs}Ho8@(4107*pY`oN;!c7|_N&^Ap~D%$F9h^tT^i`nZ5E5Z?Czf3`d5Yp~bv>N;I{m$;mG=#qmcZV1#a zywHgbVg6-EZ7?cGMhFHw#f{KD+IP_m85=Iq!T=WY4rTpOKmry<=$rhCz6O=?G|$kI z|NMcH4OAEK+6Rj61nKTbMn& z$fIsSG?~{cS=orM_xOtwb{ovKxuoPS@VFqBAKx1v-+ST$9(W_|w&^%*0}z zp?StWZe>#Tj_no2uG)Cx?a2?jjpd5$Rg!;S#-C?~9zS@p&qAHg54j-0@Fi$iRsEIy z=Tkt20ABgbJK2HxMbNlxJ1cL&lVB&K)USpr;ClJb%ZHNW7piw+eheGgjl~UDauaV# zA93wM7ku|D{o*y0G>>NaC7<-6v#;HI9kmzJ6WPnvV*j%!dc4%^$k*&hF z{!4{f88DvaQ9y>lpcdGxjKuab)!Y=o^te9kl*K-!|HP4;-+FL%#)GTweRgDjyV0&( z#ohdy28ff$^^RM4>ye^Vk6Uh{P(^4JIWsTAS=Nx>fQ}zZ8OiscYvoe`2c6 zfCf&0dIz=P3XHxf;8YzEEDcEvwAW5SUVljJij`rPCfg`dVFnj&nIZn1cP<`s=B1`e z{xw+(cQ;OHn>8H_+5{h=K-xO`p5RKAVnmH^cLvpj-)D#R%qSRqM2PnXGxu(I+E6W2 z)kbi(hpX}A1HrJ#`C6%Bh5oI|Q^fFbE5R_T#5UEfmx;>5a(^h`OGPTXKqOw~a&Cb;&kjLy z_h6qWDB)%wEuhlhafkF^?N#|9K+&s0DnQY%f+|2^u>|HqSdB}`A(P3z3gSe8H{quh zw(#-w2W)nc2JP8B!M&$#6fkGJ74?8^)?M3LE%d>5{j@v;D!}<_Lgp2M9<+@!DYqAW zreaEro8LwINtiqAX5VQ5mVqK++z3>iYF{^Y(yBSyc_m7plUkN8K#)v?m7^txDX1{|`!>F>#PIHmvTyPbONnEBbg!0c+|!e?>9xgp1pyld28(la2Oq zCuaHOK{H2OC}U3hu-ythx*C0rM%@EmHbW`Lx;BYbBt^q{FNRS`Et>*10Rm!L^<&}> zL~0W(vlWK^5^e8yu%c$g52GKMk_mGh+;7{U_dWfxkOsxei+*Xc{jy8nO?P0Lk2;j8 z12h|0HR1$3=>&WmFLQ!SYMl;2>MjghbOjjOYhW78cb8&h(_&qM)HLx__lvXM3Hbhr z`ERNzs;TyObM(X1O<}cj2b@esCgJEa)Y!gx7x=S2RjS~)omborGHPpHedI)rY=Ym# zS>F`a!1U~UZm<4{z;F5d2E+Ew`jjcmMjgD&Y@y6-2WP#qhz6#;|6e^;7&guCSy-Xj zy1uncg=RbbMv;MOi4{NC1XA@4Vz27b*8j1!*QiI*$Gv^w$eRtAcKmFpV_u|V&Z06| zX9lJbW~0TDk@;`=MKh@(KScZIa$$gn!TN9gMt0HImq6m zxCWao^8!07sZKb;9&{`nEHjkORQ|n}iT35e%tTeOJ03}so$9@#}Y$0!tef4(I8drp;Q1XXDzpEbiCqE3z;-Nsdk4@Dl{yr z1dX8fOu$UIxYiyKN<$jb>693jbqTO`L5?Eoe%%Mp#6VA~7CI9K8>K*!1DWpq#bC=) zKB(OLVnFT-I)8$i8jS+tWprsKw3puOiCTh-Crn21)nVCED57+}8OfO*|1nTM z{$pI)RHK_RS}nW&M|$6-3rZWGL)ZSV7viC`!BKsq%=bV?P&R@^`;e?p7yyOmShD(8Lw8_P@8_8&ki{2$Qj_z$Rz{13!*{Ra}x{s#tnz5)Gt z@PDBGz4bp3kxCbsMkrfL?5r(Pa@P7FQW_obe^*qNivF1>0oMV+Noy&wq|%7Z?*)52 zEIMfA?*_wpLtKnN-GxkLRHqbiZQ^m__l#xrg8Iq?mi3Bxq-M&b7vg2#-b`%-JGW}- z#9H2o`&0&98f!%*o5|8omTRg>Z(}L+O9KxUk5nQVQZUFxq64vJs)Ol4l+lN(Jf(evFtx}{pH~4K9yOG z+K0NHSb5vxk7|Ois(i1Kem*Dj=E^;I%WV#JxCtX|IihX~-j#>fSyUP(wa#Clbw*b* zU$@2n>w3cK>?{@<_3S7}(eD2H@4VBer|m~qW}w2KHfMJOlFoHKEB}$Y2Mse>#L}<> z-;ML&Wf3wN%iEWDz5UnvFFmm|;&e7|hPE-!G+r)dQY zk>N^(?+L9h6-s7yQxNKA61k5-Gu+C7gCOji+ih@NbbQEUt}4`(zXmEgzqDJEl?-%K zk**otqrakudf+?xAC(wbLcZjK39Ddm2D`8)XWFQX^}bYaDE9Y^Iu4Lb68U|{cACcX zx#m4?vTan@x#lN4{m<9?4ctG+*f(=_K?UTxL4jBUK%N_*)C&;$0{|guO!xsH{89)3 z&hG#NZUEM)uTZlm4VUAjZ`PfM^lK(!DU+S(`CxhE?la4~f2oD51!49N5|!dMn+|9M z(khZ8WA35Yq{T<{ezkmFYGs`?XKaaWx*R_%nQ_ks#pxlt&I~Q6^=8AIX}+ojvbi6b zofKM;_wD2=hM}eW$g0^1{}{|%(>DZ2qj{1~opYqSM=|;?SKNDW+!eq2oyz!OZ9>bA z+7}3CVk2jD{Ci&!gnH_hCw#(?GfFi>Gl=F4d1I|`_P{|((-BKW?_A7vy z!IoyyAn(eWjG^1gq&EzrC}c}6bKAPsj8qg4xy7(+;Fq}U{44|;a-1a*Z$zR1c_U>2 zU=Zo;Bj;$=+^RF%bHoaL9pnPD91Y;{cPih4oNjyFjY^_kIa zW|u>>tO|XBI%x=OIV5h?URqnM%ggjkFMRhU$>DlJf2Ci^hq{Z~T@i-Ur+DsmXPCqF znD=kK8h|!gxK@Kei!{S_arknbkM~Uu=xkfga%`!(FggyU!y@Q0m*e2 z{;EXf-i~=8GQYOV?Gp0uu6t$~zh_-RKp*;S*dxtkQs~b?d~Rzt-fK!O;KfY6r|jle zf9;$<$t2j1X}AXdUEQYpI>f|1ZOeO_mj$6^JIet&bt&4=sLKCAB%C2ZKq-k;-8j{QnC<&lcK2C&!lH^>`|!jdWG7If$x$zJy~;31-YVj z%poRF3lY^g5EkSl#8V1+aSY2 zaF{mF6`x6?@X}E74xNfWJf?0dIG;-DK7{o9at}eU`}=dV25clLw)(EycZpsHbFnEX zuTP;QPR}Sk@^d;iUe+*cx`PMUU&)ixFxSK7UwKHPnQ?m z4r&4YUMtr^a_yn*a8BORRR8!E9;Y+5aH6fY`3O6`)+=2>AoC8U7^J?j>17c=c4g){ z3WUyv3?#@-kmyr_XgU^?H7}zgg>wyi_ym1Tnti_Cz(3ltJYCyqsC{tceHkQw(vE+D zd@D#s`8ujM!V5qCxqOp@elCPVzW>)poNCKq;q#&JR&kJRSwG7;#5dp3ky{AuAEu5O zxpZe=xGkG5r!?F9s0gUu@3=Ehb0|Nr=jv|!ieKGFla1Jby-J^{eG0;x0-FVG za#EHt875u-KjJD@t_r46?L8`GCN~*C%%)OrNn>y3`m=WNx4#}1f8cxWGp8bXb1^g0 zJY#5`ZvEt?XX>`_st#`GW3;Q|_TvW*bQr+&ko~c0^AQ$2(MblRpLrtF12%=+`u_R5 z2JTISPJ*Lgm6237-?f5~nu=P{y7QhF0p07%9`IJ00 zU#gb_^sLGJGWp}^_M#t_r*hDW-iM2uo=4Z;?}Y(4M3z@QfE3whGa0gHv*q($IJED4 zkR*WiYJQRvb;el_G(;ca6oZ)8G$CJv#JAw0;o1cs*{bQFI~7feBp;VP0SAEVJ5>7n zc1hVM@K$1->xA?Yn7js{TJynL8rG^Da;HBuDTAPY{PX^V)o2IEFaE^ReD7IkjL9i( zP>@i0|JC^EsfkCX5dknQ%cFn9O2);u0N_>}Og&u!-n)x|B$hoH#h=dxOv@48YZn7g zOfRIDLZfRhTM`V-wF|EFi^IxP-j?rdniL`4mus$8Yk;^Wy|aks9@w>M$oy@3tw(eIizxR~`^iGxQix~mOQtf)ZE24&*>dw& znRK${#$5F$o7P$rprZH_PS_GKooxPo0>v5O{dS0H4UAxjdje)O-O^ayZbnUPhtCPh zcsGg-0maouuMO8?&SWrirg9Y>VD9z8P9)q;LD4)co0-k>h*fzDt##YsbX2+wh|S{I zeC-+HW=mFX88YSf1#Ur&GrEYMH7TAUvxb#++%7KdG7@V064gny1gl@3oen#a^*Ryy zW<9bWtS6E6NvN)8x2AtonXiuW*TC%g{q>37$r6%0+O|W8&=Sx}z~tJ0&OB~%J5s_S zR_xyE8lUz<9^PO&Hv38Bu{&g6FO3gBVQbGuyKS~<`2?~5kLLGTjzaP2@0${|m!KOi zZt!H6x-k{4aZj_QJ45fm0fXMNvc-hyJ2-SxOEk2%txH|F9(q(`m!ci#LlETowqw-2 z?LN2{Y^2fIM||_h1D=RRz~;JeMHgEiN|+@Ju%OT zLv71Pk!f z`3oG-jYVURj$0k%pxgPhOdYZfkX3Z;@AREavm!OGe~aEDcW8tdU&gP|_;-!3nnVJ- z0<}P)3Sd>1?mYmlh(;;GmqXZepa93jZP~-+z3~a_p;3ZwoW}_usH}Ds1BIXs2Ahdu9IUK5q8@yY(O)Z-w zHPYu#)i;HkVy3dq(GBFm`bebQnql0I@4er$f9pglb~V}Ey(DLALR)7_?wy5w_{)pM1$58_kx6}a1RQ}=L6K{a8OUpdRi zcbg!=z-1vG7w&klF1EQ*NxP!m;u|M-!%8(Cpy$|Gaz+0;{)vOWmxp})g@%p`9t1fm zp=+*gUe<84s_i6d9>DslNU3)Nz>CSgTl#g3qNJeofqaw;xK%qU0WxE{!NHmvdf#YF zso09zN_OY1GmCCy1vn}Gy9huX_wJ)t70N*}(mzaoWntZ@ulf#?Up+oE9bY|_FY1z% zcfv~YigZze#Uc_8XoHnFekY3Ob!6G{`P8Ty5WzOsI3!*SqRM!XR@6vVY#fJCsW>5Me^{T|oJrD_79XJ{8*wm)R}y!c-8wGXew# znQpqKNO8Tc5*MoA zzf|uaRy2vv6pGNXrP2LvMvz7C2!@X??%{C&bFj`nVF;m7F{bl1j_F6IC#MnEX3tAg za*k1BXZ6j<(&9laxVE?3r&@?cpV@nwAli#7?f+rx8e>F@n&jBFZQGtVwr$(CZQFih z+qP%k*tWfsFT0!lvB{})`u@D>licoGT{R*~8*wtI^=2RP0Yg|tUO9S)F;y(6ibMb{ z>{F$_KH>=RHSu$@CL>(2+f?TP(uX=shtyHTPzZ4`e^!%grx;}9mWSQe-boWpm znRkNv+4W!wicpNivT?JR|Qjlg4!bC>urB=Y{uNoHx`}# zR`Es2DFM*!U^t>-e+=onHsGgf_earJ*tt8~-JiZyjk0Pk&29mUF{v6Oiz38~kJ?P} z)4hu^D0V!Ko7hU=S>5ETpLG|%%)U13c1kSy)` z^p%-=z|Xt_?b|L@7wG#h#iv~KeBVw~|Ic_fet_Rs;mmGr3Dz^yYr&bV6mb-7@qz|b zQi*4sGMto29ZDqJUoGnkrux)(8KS_!4@}iR2*{s$F;Pl7r%Ri)fBP|-Of3<1$%(2| z_O8l*u28nK{7ORDFVr;M{hq05yL|kX;kGyPLfNUI>{`&a`6qvny0q-s>75!1jkhJ5 z6k9-5{@jNoNrSsXkC>i*Mi=t1^EMc2yp*~?7*guTiB_dflHRSeHoVozB2r+ zd^yuLjkaNC04cOPXZNC;A7M93B*JfF6c5@&)!mrKedH{i0TMK9Sg=6D1Ab9tmn2?L z8#Qo0>A&v=z$AOR#u33J6NksHU(%PUcoeU|PEjn-)k^5glEYYr-|Q3?^(M^|$gv=9 z!T%hEGy?x=cbAVBAAN`zqfeUKVW%Gv`2?nVXGO@3&+Dz6GNNerfA;%zAsp~2G$JoG zbvYeQN5=IEK-dw^(Z?_EZo}N>6oJXyn_!}P?}uX(m*O77ZxaG8=do5)n454BKM)At zIKUF`f)F-#b3%z_DxCFb$(GUqnFS@)Y)qPbP-;51juu4!1l@SQswkN6)h%1^-1o3T zC<0^z|D^4DIYQI5-v4sM!GG2mu2%gWg>NIP9&j4582i9x$W zpLh$#%y<(b&AtH*JM{?MWz!E0IrShQ&%Ozfb)|B+^)#(bAp3R2K)wWOMygtbyq_kv-E#i5ov7({lDEW)D~tPvSeyyO2UMCQ%GWF{$6^4QPd0A0gQ5B{1BBY*ky> zODsJ16d)hw!nAi@P}k2sgY-vMXv9&>2QT^43ZK{Op<~z^#m-3~u?M`Z+Ao*27wsm) zi$D7Af?=d`OhcMD+NxsHlM0eD-?BFqC|$fG^7yBML zON7CX7|kE5|H%k)5Qh-0M~IOZAw=vQoOEhRfWeP2)h`ty!9jl7h<(=>3^$;5F0kTJqG*5dZObS%0`NOVn`;QK` zs@ZBk)bHk@hO8;6u@J^g9$?@IRb*D0@R47E^jwmKNVDcp=K$kT3*&xRDuU(1F zp81Ru-*9|y1UqeMH%=h)u@>n8hOC3W$a&gV3~3KO z`q2|__8GtA*zdm_JTGvkK^A$gon;wbQF3I%xf}I5=VvqfT6c=q(c(NYuFvIu;1%IQ zScB(TxBbkS@gUGLYfrho13DEuXSiu3uX4lOv}M_bW8<*=6v-P6junfy4B|dx3i$7p z@EPaAq3`;N_{qfPXeE#u*Bt$e-iwzPVji&U95l_FRX2kl^SZ+H+m8NTPc{R9de>IM zp5uJvEZ2xwy`uhYT}rXmn9sJG9qwz9FaS9BdpU5Ed+0L|-$norGwVKob32M**YaW6 zibIT;K;$&QqKpaT{6Jdpz`O~?S2>}P9m=AaCL}zzQ&pRmp7^R2es4B;PWX!Q92uUj zcOe@L@CB|$73^W*WsJdsqCOq{?kM198)9>V_kr)WW5UTj-ThICS%nn3$CseV)&{LR zZqR_~f;K6d#Y@o3g759Gt(7j{HMqw=_-(0=elMe!ptm-B79jM%FMZBS@UJF(p1_YI zz&1K|RnAB7`2OKl1Khi(HssHgvO!sUh%1n=bdbpx?K|kUATM+`dxP?^FAoX|dIy(i zu|D);6OQ689Nm~v7ziqeZEra(_2#@c$gj!#@^6v9*X4d5(ClH2v3iO>1E+qbTG8EQ6pJ{_X@@2i7K)#j=f(nHeh_K zHY*vyB!-7oVn_V5zFxA)0MPolkS3*iC09Z#n6gn9xfEqh89~hs7&gM&)x(r^$>}p) zIA6TH#gM@3yERrdOCKs9b%D!Om++@CA}{C>ExJSVyD=+Dy)E1#Q(Hu}FFsufsqK-a zX0#nLRM~zx{(j_&TfYWByeMw?+#(L{YjtG2lEF5zG!f_etk=1IS8muH*X)N}zkAC5 zG#nJja%t=u9@(L?{czM0q7t0cGo)@BCk`#j9{QOI)pHfmfr`j~4l1**3gdwaDrPN} zNta~GozVD|>^6771j*|K(iCi@FJ@Jn6mH#P=Mo|^&C~S(uBdgGK4hEc$X_+@)klDc zTviTUueL(YaSq+OT=_g~qqyVb~tq&MVNL_%3Kng{-jGGk1T;Z@U@nF4E+6SDQogopu? zTMvW5ic&~SjKXN4qW3j_Efo$sl=eO?Kwy~Ka2B|5Q+F=G3-OX!)$*Zj-gmXN9)oZ^ zpVm2(mxhAwX%94nbknVP9ykMs*rSfYIuzvJ6xYdQ99bu zO6R4gS&*u@Gyb%#p%_Is?Xpdi>>H6Io|mQh#4u1X7oJ+-=BUlHXx=wR;G*?>xR80h z?q1-eR?n1~=$$m6dl91^HllBo2U^e|=BV50&s}x`GIAxhEWX@LJVxFQ&K#rfoPAaT zRE)bAb-pzla~JSj9?<@h`s5=3qiWz4I>17mm}`v`Qubd84C^p3GleID$q((>EJQ=) zUt^H4d7ZGb%-qONouBt_1f)kZbylGk;%V@T>5^9rjFEGPcZcN6FsmyMPvXP%kh=Gh zrw~bybwJFMl&-B8yTFB}^%HxARxwm5-NDfm(o~euDHgGg=x!z~E?+$nY>+D7E_n-T zuzdTa&UTiENyULJgR#MIc83AQ z+;4=Hd4^uBZ?oqf)=xZcD1JHd2*0eOerB-8cUXZxJC3#^%}}&a7E{x_78M?F`OJx`;J8UX*6N!6l)pL7}j4( z836dUR~(QI1jxdida3gdXia#}0j)_*z}0I&(;Ya?;LG}~)Xlem@P-+A-sV))O#{pt zrTR?!3-TUy5Iy-d>p6=u#No-8HNwUh(QH!L&Nf)F`lOUD8$TZF$V~JQ5F0bYE5j7b zs>^w@zb6w)sN3=Q^I~9{xbJSsylBr{B-fnp|61O*Kjhjwi`_yoC{qL8w_;CFan;g4}3n`?O?eD)mo z0KOMBD}W!*0;J!c2nE58s*Ojudm^PxPs#o#~96Z_! zWrgQuU*%*QArQop@f$TlzpY@K$rvW<0=NR$=!t+MqA63DldF4!RJ6RCPK?EbKB!*&JC`-&LUT1(m>5H{F7kvL2*o8*8v;p%bZ`ua3OiH za{7Ax$pKKlXNHD9(9gv!_yr>vg>&Tov7#`BfP4!po28K zp8bTLU1WgF%$lzu-5CEqpa(>bJYBI3yCoyQp%}aN;7BO7btc!jsOwPCSJxE;OamY? z-qyIFZQR0Yd-YL{(7WJ>n<^IMFc%~ zGXui5dp;x@;O~@0q~Yk*Iy_b!Cw+CLH^{B`;2d>7tTJpHtci3Py#Z6*m^Odrg1rs9v#V_%`zxKyD;<;bny&HJ_i*KazEbC@6T>6#x3N6%4xyd^sm_Rz`b?3 z8=Y^M!~R44zBYz^`ts+K%jZH)3Eehli(LX{+?bMvLBEh zu&WnPcH8{+?&Y}53|N)p8Fa3B(~`8OqOFmaEs_eSOtzsT#B9;_wb0C*(fpJGV2$WYh%kP?HdXY{tIpZy}q1>uK!TAzk(s{EcMV@b4!I(p6ib9q+xU8 zI4XpMTc4SUoi>rWmw~t=R*~R(%5PWtU}pun_~BG&CF%DKV=w3-37X+x_)7&% zY7MIf%;bxA4jWUwj*)_`4lZV(baO3B#2o{e?i8-}^o7Wp=gF_n26)c^q}Z4+@^-3q z?rrpPmuSGj3;nSGOX%7<^Fb=&U;WB}8Iv5;g)%GEPU1=?khDkDwIt+5edu!JPM6cn zHIoyFu6@Ygz;ffW-lTaWt4entjAU%bsS_}skmh0sEmnIp9lE;6{S=lEK+xW~H|;bl zR5W7~+;6Xb?VJHfO5MA(pGQ~%KnHMtTNeTNb1!na=##(Rs<**kT z*Anwrjo-y)A#qzg+jXOcjX`^w;>L-mDt1+vm+;8Hl0Sl2<}Z90EzG>7%ZoU1rNDs@ z_XCL9`%!wUl$z2HgjFm|J}x*;d5GC^h5CYhbl1&hH!N!D3IeGM)VR9B?1XRr!Ym-} z7M3LmC*R)o81%9b7#Uq+I>NfDE9Ou-1$tL*`t?Q20O9G)zG;@!xQ}+nd9Ls#_gBI2Jr;2V~O~0 zkrwN!3#u$GNADC;`Y)g>1@7e6AT?yuwMfMV_c65P$dHp;t(Yz_el=dgE&IYXN4Gxx z@3F=|L^F9j<${zsNaqEH+c`wyFXP;;RIF!&)7PP3X^PY__j-i%r~Sc4cU1beNdDJ3 zR)4HS55SPHYjU8Zki2r@Y9H1wQo|P3V(NyE%p0g_wnO=`(`(LxKVglzyW_I5Q_RI>2Tdp~^OFTFo)=JwLxB_ul zDr>1DTya_t!g1TB(u=^k-r5{cLWW+w9qz-0}+2lJ~H?hhyD)0fb>5p z1JgnX*#Q{b!7uWTrYEijd#$@ak zv^(&~;w)jTNWuuUqC2EE8a-rZ>mtQlxAMr|PTb13%HLQt%o<^MsY)+4AzdHt=Cc6l;xM zd3G0#8&c`vu$?<-#jEa%vQVM?GHH%Z{8VzQ=4@+(I@_VHj;WfwAo-C`*~4g$;O_2_ z_A2~f9Nq1@(K?tDwb*Y`rx+72QhQt<+}sDk7>H>(Ynw2OXlt?|!Hn%u^GBs19F z9>fEdmHURFX!P4}{y3&7UVpixOJur9#d<$`;XpYL`0JXq5h1Swk1Vz)dco4mXixtt zYsxCXon+lkI(eEZZ>~x2(|1hnM_Plqj*24m2C$)XJ;wZ9&#GTH%-eSkqlOwn*7Ik| z(g5V8?nC_|#beyl6&wYyecdMjc1!R!;~(ix%!e&owQWET$w(_S1JYKY>dpqX>j_)N zzj~REm*%0F9TVn*)Cr_lf-%->2wB}wTO0C}(?PPI@aG^E#w6)<2P8r7-tr#P*dz!C zZzt*oELh(g`?WvhCi$RxJQL2WOcJ%#vEysU1c4N8BU+M)k1iUjSe6lTA4R9YA1u7# z2QxW7e>)_WFW15#fsN(i9tf2u^I?*Ekl-jCg4#Za3W}}G2iEUuM)zjV{S)$p zzen;-r>d(JuKYHq+AxRRN=7ABw;30I1s9^-7>C2Mw^Q=y7V{>5%bE#Tv06Ltn4NYu zNOA}Z_R4@Em$kTr|BY3hRYX+2nRFqUQ3+f5WWzuFzM}DoOJ6;-vgsC#W8B-_U;edyWi-+@x#!YUR~4ty zVANbeAa#1!4ibncQ)9v=?%c^sNRRP$K30PnU5)4)h=;mrp^4yHyBDaG_ z3kKE_!@3icSr%B|ZTCgA@S$LNBglnA8y*&5m74?SwXE>#e-)Ej>Klb=Yy0=~9WC5f zP%r&gdoh0$gy!*8FEIktR}mjBjKkaW2O4PTE2D#YnTs$mpyulj=BY?MaEvX$5tAO$ z`c*$lGXCFDr{Om}#LDa^=YQlI1+Hw{;(+~l&d-S*he^-DEs$1O7e>Kon}dJ9@owa4 z{JA;j2}1=&h;yydpqT1iex+~)7a+DxId?r-D3~&TGLD_V9_Xw~vf!)jj==+q;$O`T z_a8@nzMcs9o&~@Fx{N?uWk<=`=cQXz*hR4IcLXhqnaJ()k%P1%+!zY#38`2j@e(n; zLPq+}vlXuuNJRlQi=cu4hOipkX;RRmO_6r+A@AuGFb<*3j)2T`iPQM{|Mh*UxlBc> zI@YG?VTE!(AR7Mzp+21y9mx66=vM@|$P1I?GYK}~5!*Y)rx*JI&;snDB_X3*7jV>{ z%oB(EyOtjrAs1VKKKiqS7X;4OYRP=Y6gWQBErnNg=%hN{23vXQMXkcrL_ew9-ymiu zZ!$f^s2O9V>3OsO(Fl4&x|#b=2;@^}{F(Z2S;SIlv@IwHohf8SPbaemZi z(i2$iEKuLv-Mxl_k_-K#SJU|V8hA|;9Jjn^nC=2=t>7cL#C24&2%cAjT2)gd2(KG* zJ%rRhiuwbn30=rgQU7ObUY)4?nN{_=-Q+{QrGyw#9Q8_TSkYIScD+@U^68o{@BXpB zhe|aN*HRv~`Eq*Ooc3fb%jPdyaT7*ouo|LOEA+O>63&i0fuWvruV>G^%o{-x{0A5J zj=lQ1cIzNsC>;o0g|;Nf!d+KUZmjWa=MQDM;v!XO+t81hS z(5}X9td&6;(!p(qPRR+#?wcVu54-*FPmH^$cexv~M{QBqiZvA)-oVN?hWAZe|I0T- z&x_b?cQ0_yi(B8ze;y9E=G|`jzy6Y|_de`Fb`F%i^#=KFfqN#o8pE8JyjM9Js71Zf zuE@Oc&&~1O5bW4AEdh3poxIx?=F4Cp8$h~GG}sXn=PJ*DajD#Ohb1qEe_vN@ReRwM z@`VTdQ(;lp*$xw+tD(PHq3VTsM@#2akKV1VhNQ=DpCxtYul?~;|3Y`GpuBqUYKs>S zVNg`eiluB=M9P;GnMq-;>N*Z{X^4WLEN_|oE>KceRW1OOr9Wc8(u+oA(_Dh4^|=CG ztxsHNi!=uPtBEM*5nkRnBqlTkEY$GhG}Fz@{UIJ6zH`(1Q|n8@>~@pG?RL%klI=8$ zOqJ(wJ@h_NsJS{Q;-v$>HeYe?z{8P>9}CBET~p+{i5{6tCHSZe?HLmnSdr@IVj5_8kt9zB#wNTKdy*LQF!0#qL3M$es`aN!9_-!mezp>R)ig|me z%-^b$(KVt6e12R!<ulk=NJr0>yI)jlTSllxQ0&Op&)a40v9A~ z*uVaPN8X52Km*Ojdc@?naWLXA{GpD5%)i)O<@~vE&$XR_kE2RWU%qTF^5?L%^%VCn`m#M{^7w;)nFMxdU%#tLJsO=i>(a&5c-JB*M|exdMM4#JZD zRz@a=y5FR6k-SFO?RLs6Wa?HOPelW_#@0nFTOAv$A3B_>)dcptBy#3Pnj&-v$`pq80PQzDL#%0w(Z|TY2im!fH8&pEa zVU~c8T3ADk_NHKU)*~-(NbhSAZsB=E#BZsmb3%V#)Uk>SkBTl2I&u*>06^6z!?G28 zmQ>~B1i@12)Gk1p2CeM)8{9a0pHa9LOe7&m5k~K&QPxD z6LsVgRK`PIqle!Rmkql?L|54(B`C8}vkTR%VVY(<4|R{57q$-?nw!=6n2gF zlO{5+c^WEjhh@Vbd&JC?Iy@3T>s+90HOIq zDuh^icaMQ-WX%BF#wO??q`rzL%f(cY@oOo!WGl0(NoNTL_=Rv3?=`~;;O3yqw?I+f zTL~_|D8ENjEgR+x{S^nLv#&+%o?F#Fw+x}*yprN~!Q+Dzs*!mrL4V-G zq^4)iT)h~7@1OgV2xItGjWIg6lD-HhXoafB%qh*ju%OW81V8bZ4=tVzw41cMJeLz2_l+$cYgfQ0iNYHMQd}|RB_>BkKsFI_osY@pRVY);x z20ZN*VpQ0Dq@lQ026o@>&s5_q8W1NJ<(=*4j`jDDBHxu4{`+F|}u;;U4NB-Yi=&UO+=0_3?Nm1HASb;iwqGf$}Yp#FoJlqs+Od)+5)Zx`UWUgK(Uu=EgZKiRz$l7}@ zh=P6!DP`lH>7@{HkX-PtB|wB-RvLBY${aO)nu~>2FCHfz;Bvg{jHm(>lF94YuzBQ)~)Rf zRTl3M{!Xp=C=q?ZziiHOChQ_xFAsI9TjlZnp;JWYwp@FqRm+OI1HRD5ZnZ=B{QbYR z5>(oVeOC%*%lBtd!`m#NXaoCJub+nA98{|bF?IMw%vnQ;>=1GyDrZU}7?^sO(O(qK z51=@;u-qLMAkS9L z2B6ys%weu`KU#mzi!65cWW(K2@1AH1i(NE7YM9+1dcekPw`B|IR;vLNC++o~4hHIR zfVT20wJ`WKKb`JyW9Bo3i}fUrga1WM^4ZC+1kwAc=R~vi=!RnLS0rsb68-K4ou=fa z(dA{JLr(wVx+6t*GOSw}`;kAp{E9yeA zuh&rl8&JWLw}Z|gjE5??DL_TvIUMap7EYxm3oMsAZ&c2 zL7#E`Un_eKqCch}-a2V%^d(Ypf~a@kcPx(Xnv#Ea@ME{koUeKrhx3|)0+wlTTKAui zVl)`e(;cC7)B5GVQbtclX7TbRf&)q20t55u4B`;4Sz*$|{exdvJX6pi-~;b*Gvt;+ z>0b)qMPy}BeD1)H{j8}gI&*z7@N(qln^E<!&er@259lF7Fur2dyYSUI0{H;mgPjOaqKR_c%Jm#i@^JOI$b^;tMJDIB5i)` z8a<4=bh(SLWF!1S$B`+h1-!L~azJI@u#c>~RS=Q+S;IWxYFSHPD=02|LcF@*^D+#- z28B~{KcdH(JPd-!5BMIGsaS33G5yqxhxUMxL;hD4!n4}k5~%_1kj$UmNU5-FDtSJO z*r}Uu`jF8@7_GB^o;fHzZIYV&Z}=}t*MN*aY>J*beL&e=lLXT`l|7CX{jz10sb}Em zHreQZ?@zM8k(J*}6OscFR9pZQSV0J>Fv96kU3Ka2$(HzU=A_GO6;`{06=_>Vl4}KR zMs~`>@9dG5NDG8&YQ=xyt+*S`9N&m#BvoY+BEnRY7S7X11IYyf7bM2IPji?)LA zIiW~q#(IXI5jAnCW{}#p1_s7o9})|UFory%!%Uhd7M&<)X3;SY6~CxLSTg;nwl5^f zOiFp$bE|K-E)}+N_=b#AN-+b8@)z#GI=%H`(|;J@^}D!2G`eL%$uDmc3ktn~p=);O z`p2Lq7Z@7`q)&s+aQ`+z`=ARyEXe=xqkY!o6t?IVUF%6agawR*qK830AQ!e@#VcuG zhzS;YLXf^Mi6WD}kAb-h3}4--p5NA0Q54QO9XtR7BkC+>-ZQ_jq1NG1^3%Ue+VdBO zMKnWAY;x3G-BhI4%{5CEb2O*fGL$AiFWG$iBY!Plm3We!GNL9Ks8`cJJ6@o#zWFg3`GvC|MVbURBw30ZPPyrylv{4(_ zI7budlVsn&*w!AojRoWd!Va*^A{6gbk|&aU_RZK>_Z3BcrD?jT76JE-d}FEAYw&@R z`P9X4@G1I+ICHVeHSJmC5@vq2$ca&|Z;r1|uhNu~>9@>UNh#HDmbt)E60li3XAKE| z$U|5EgiyPHAtdn~a*ID%ZPOQSr+T1B)NmM`f#g;i|c6UQkc5-lG-F(%gW; z3K{i|Xf(I);wcDk51rah?XsL3XQ31_oJ$EM!|y&-NzE^GSXH!cDnBT<2Y9qED-+{~ zYcvrX)>NusqjXeC>08pyPQU#Fx!2(4de`c?QMQSOp(*2QGr&{hP8J`YH(ml%@s~mi4?@$iY(R48# zEP(I$6b(EP^6hi%eOTci^-$*C;i&ZAB%qs~a+r9`eT3gIv$64RAIfAckmt~hH&W}% zjgK^riaE4f2(pEy1d8kHob$(9Go$@8Xi+mgAQl3#SkGQM{@t~J?qS%H@#O@`@tc=Q zbFdQScDPxjJjxu#&`vnb1cN?3}!PNs)wZGb^v`ZP`ZGXbaPty(p%Wc z@<{0-KAqb=bTG=Y#^#D3RJdmjAWpCNy`C&O(4e_%Fc?8@;b$RV+sJN9(F?s~e3$kH zZ+w96v$H6Eg7HT!eTHa`Mjz+eAMfNuBZio9A5lgCD&0?Yue5%Vj2zxy_mk@%#VK_i z7YQAsVH7x`N3D7teWaOL{ucYm zZBcm4br^w2J~8%_z4DeB2ARa1du~*yt1X#(PHd2k{$cfZVJ)F&R%VlI$xXgAazu(>!*Ypi3GYA2QEN_xjK1&xfDR2!=I?Sdza=IY2) zExQ-3{FeU9WrD}aHO`7|*N6f>==~nAt$jdGP5a`z@@$6S>(r*s%vVWV%2f;J4O_wL zr?^gL%-AA9@aoYF_FE&uPqLYXv95se zDAZBU!jzT>K=LjU;f^$xfS+CC)_DX~9jC!ase<@J=_P~jZ=KnSP`{#c{9K$>2L*pF zhG1@%vyiw%pp$%}f}a09Hlt5gc{qt(ju+_z(TLkj5v}4RXp@>*%Lho|P3W;x3!tEc zPx2cEuMh|+uH=Ot`A?ox*lD0v!mt!fY57@kHk84eao-V$c96ar3$=pwmbR zT`G5;vJ0VLp?{OGyid=D(eCaM4t`Mg#Vv|rmN^7*mZTcc2qJApg=@l7UtDth26b&) zAu=WmJ3rp($6k~=NZ+7oPk#Mx-#U)vy(;G3HUgN$vrNac!?!-D7HaVa&t@09xBQ=s_Z6;Ni#j+Z#9DS{GC!Z0IiZ*+_NTj)kmFL^J_ zWV;bUGa&ihIE;WPL_76Ft-&bmq{em*@j{8UN}o`vm@~EC2ArPbj6Yo*Ftv|fp8=wi z_FL&J9Dyc;!>^^d5%gA`X#~&_?sR$4^jcr2s z==r#G^RLr2x2HgH`{3mKyz(zai?M-^ic*Xj-kZNlI6MC94U)VWzE3`QU}r$-MA*?U zqVx@H^XYK!Ox|xPuT;imcQa}Z;FmKuRBkHdxf4GU4($7$a#!7tl)&sqr zm}JFxWWa|>sI*6{MPmC~*gaMSxNtDvu9{dlZFr^D% z0JnKDyiy<>I(#8t*=2v`N8%fR6B@n`0G|(J$o_8D+P zJ!S3$lom19Kt&#Qu)F>l+{gKUUBtYis^xz@>7g37mGxVpfp?oScAN6|m|7ur9g9vV z-Si}A%WWy`9+cGxpE-3}nTt!uA9GWXL9}mcrK+(FJ2o2A!oNQ`soP6-kZb~^J)II) z;o(Wb6%VREwf2^tSJh)x&n(FZQ>sfa@K`4^LP-Cf+u33(Rg z0+7`}A|Fx+(jLo4G1m>_Stsnp=M@%bL|`UEZ10hChcu?2nmhWQ-Am2*HDCZezn6b5I{oY@my-djtlY&s$PC3iLTm9rbm0wu^1n-(RJZ;$a2cTAj1hV;oBWDh7hDwZu8M( zV#uI1KShO^8><|f+FmJo0d^(Z2+Ni=14i`J`d)ug1h>DHkTK(&WSTz zEQ-?4sqO!SjS~$Qa|7?^XTzkLfJR>S${V`Wi6dd^5r>a5jPC|}8zja0a?qa;D~DCu zm*G+HN%!G9{wHuwcw*dl$iR*rYPLqQjxnTVcZ zgb#RsXY;(UoNpwUH>R{?R(Pb(54+&tT|%tU5YXc|COkm@&O9GIKu@L92MR)D zcl}6+hVjcuNH`n?;4CmC29i z8-|UD4me#ewjzuz&+8g3jEeHDsU;GvWA_0W@gzaQwu)KVhrwTBSjN2K#59$LuAzc9WT8dI*5Pcw0rxq3oA-GbF4NT5tz*n-Xfl(4N%zG+pGRB>;XbF=^G%5?v zVvj42j~@(}5p;J#a$k`($SlSeZb>4L{Q}Y!PkEl{S6}Oi)GOJ82iJ|l*(B2atkjFY zEu1Ndrgd=+2Xx5n@NCOJV^y7cqyj;n^P_LSz=OU^g4%PF^byn(SD?b8CQt7N)yy#c z4v2Wr=(`y84Gqo-)m?XlozOV$Y$;fzA}v^r@`!Z)0p&O?YlQ7G@wNB*#fdXkNp8SS zA9YTO*QPA7PVCjR}*qP@q z(S?cl3iD@oTW>T&or8Y0%f4)2@$f3VZ5W!H7Yh!5yl+m@2uv-G&{jT~kL+|ZFJjJ( z1IX`jghx4YC;$5cc4|u@#?XU3aK$>B1WHRf{sAe(n{d5gh%BZq-MuoCereQ)7vhVE zCyL4g$;fK$4Z*-{w=H%ic2Kz~Li2g;4-eF}*7T+Vwiz*ZOZ?Ga+YW(~r5&B=mYI%q zhu{p|te8=gqF7Z--bX$zGjAvQ9OjvJIVYbyzW7P*KNWa_YPpHw32W^KUQf<>YFBp` zHQnb@Q&*oc`MmRiF4yl>16kxNC06wz=51jD%B_iA_I;B2wL!fy-<$j#( za|%zyU+3mW#14d?b;uf^EaQkJY=h}}+XyVC!8IIY_bVt;=)40SRn<)jmA&|ONAYM& zjZNw23v4jgVznbae$|9PE>TM7$_n2$DU|^ln6zUzbLMWn$Sj&(rFktj-oEkYxx+34 z`D>PCQ5XKXouwPK7q>C*IZn1_v!T@PY<-O~SwVY~7&{KZ$g;H<>mJ~%_W;mJOn*0S z&Z2Xj$TEF?olZxLQY&1O)?);VitY!4T=}EzI+lm8FhxN?V{&^!&lc7KlP#bdtkLx1 zU7858dTs)Fv_nV7AXlB0+#Boiohh_L8`;qI-~$7GB(!QX(_zA;W^}cVW6nnE!muvAAsl z!f!;Nr}9p-TEY_lm+9D_a7>~I~Gi5J+YCSb?5p3m-dWWfL> zmtApo;XCr=q9)=QOL_JzQW+RF0ieqCciFWjOiRABCu9BU#pZNBi3%PiInP)-@ZrUd z1urOK*$JLM#@Eq?7w|*oQQCFv4`Pr~zOMW>?parG3##B%*~h=GtpM*~=S-prA+mgu zh-tsy5se#uu5jli8KEDKvmi&dW7S@b;;I07biYIp>r}4oCWmcOZX@^Cd;kWdsol<9 zFd#d0;3p0-O>+%*lf&V0AD}fUe8me50$z zps?JzYgRx#)#AYDJ6EhrKk+|xzTi_rj=TbS0r{Tr$fuy`=>vCDD0&C@4p$+K`&O*DoN*k5aG_<{R1y1F zm9iW2bAXYmSUy|9Rct4mp?}h50{Ic$)JMiOyge=2ULz-QU$L-#n!O$q z2SC4zgwEjP1n@r=hu86+v4=PRu9JIMIAS>uSM8x_a;0XBS74&1MctU4-EqzMOh%bF zdV{%A6Oc!srgvPLQ)la?Cr`oyK7~H9rN(+4K-(SvlBvZu=z43bne)|DdA;u)Ol)h^ zC2wQi(sl-Y#Bz+un1mc3OoVYogfdrq4NG)ushMwAS9y?)N;Q4PQ2%1_pCr&Ywi-wx z`?Z7kg=6?mhOS+{H(baGj0w+?!c$CZz?JDc5Zq#5`~~NQ80Q`!COVCTf?FPF7xuqCc*A}w;l8@^B{w2*xUaCnm}d0G6{jC z4&8hh+;j=?4WaPD0EGiWr#Tc55ceP;M9u7L{q?mEhL;~?{p)ocu{bGc{geMg*Wdd; zzWzLPqL5#sqlxS!>D41 zAQN^{GuG+o2dLQ!dI=*ZMX7iyl_zFpZU@5F+0AS)$~C$O1S^1FM(*eIYF7xoLh{C8 z(r;)#YL_^a;MYjs{JL+x0Uj&*a?QV^7n-%#e6oDRItd_uM}o)h5P0nF?B@QCaU34I zlQlvzqE~q2PZhAIP9CE{qhBL>rC&B0s)L5QOdHe-diQ-tXXw_VGY~vhXz)0{u>@@v zkDo`{_^Q6pHiC09l!MuL9%*9SB9XhrFR9n z&y@C8KMv=S4}|P;KC7-|tpmrPRUfjIqL_2dfG!Yu@U0FTbS32+Sl7w2jw6y$g@1

6O*f6(h*YaaAmfClKD-h2B=Z z(XFY5%+glrs1?IH;Mji_9Y`E4Dz0;3>{`C&eS{zO5q{W5_+eiNe%P1C>D9g&96#)v z&GEy&!d~$Mf2x2zbu#?Gdn{l*qLBcAijE()GyFh9b1|Wwm9dl%E6a8Do)U7#1?UrJhkYOi1G3_(;L4CCN zi{08nb3#R--r@TNB-jgUF{1kR!>jNTqk9-n2mN+ zUeSXL%0i?+D57-}3iyN;^#d{W12MD0nR-a@*Emw^Ez%TRU75sZ>aU@D--W}lc?%;? zw;Qw0HwTAcZx5EkY%v-0%uh@1!#{IbTV9WW-pnx4o7oz@Db(mqX(+u}8A@-~80n2S zh~88Ny)MUmuRk?2h?`7l~s7li5ii#}vKzg*1T;%=tdFJ9b(p6=}|+k!)$!UQ0B z2p{b@yUM#rf1K%i!{Tc=*FF z+&L=0!{+~8KfxVqg&hW-P(Td{yzI%<2Pq?_h-L%Yx}*scXQZf}-(f#% z{3br>$ZikFOR&a~dLTqzlGn|Hfwj9~bsNKEB`?!s=o&_1JkRP-CPwYjIUQnR4ykts zw*fGmNBM!)We}rtNWCGnlb4%`;aQhQa+xm#;Bd}eZ4k;7#|EFpfm$U5?RTJ89|OJG z5A^Eee#&D$KG}KPp)AL7JEbPzB`u3@KXu_uQfwc>8BJc>pYwUMc0M=p_%Dr#JpN1L z@Lprq@uv#dQzt#2C-Rte)MJ0-GtJuhTA)otk{k8RXN(U@OFPg(ZK<9UVi1n{Jb@^PT`EWx_a?o4SQ4X`^@uPo2gHAw30KZkNNQyELd#l|X$ZN5;2KN~ojR7)7@3e_(G{`&ZI z>X#hyi*Gq~qK5X&Aw%cUx6azpSFi+a4hq;DII?S9 zmj~iqFZ&Q{9jV)av~O{welEoC5TvyqHfutdTK8V3q>3H&xMp+C8m|oA>d4;x*l@Vn z?8rX$*x4{X7Gt!%+-xbfqqEd>b0E~Uh!Rb0RX6cfMQ4^4Em!GQ*zA9;nn-={MSr1fIPc@$j^V76jJ`U#w{!xnMAtW%U7 zI|N7SCqjI=AT8DSGYo0Ihqm4@Q5hqQR~%afA-iVoMdKCQX1FUxJ^I{`O^C*tzDG{B z35{#UHzBOpTZR>bO*k-U#Y%gbgbp?d9lcKi?InQ${8Z{?s@BW>dN230Ugq`P%l)jE zd4YABY1oltPtOo(PZkC5$qTgAf}JDEuAKtgI-=xi5VT!6DX=T^a98HxuEf}`nYD~w ze((MG!1vjYAM|5CE^9~QMdri3lJPjJ+E#K2a@(42zAL@lW}&UV`y{(^B^TAQXXy>P zm1sL+h(7nGS9`j?+qgGxHSEnirpe%9Z~Zl^^bdKJKBUhb$_L3l5O?V1Y`uyaNyUx( zb15Yr4{Xtu-TD^YNn7+HQK5uH<9S9Y+^8eQD@B`Gfy@X(_v(D|Exd; zO-zSAF&%wQ4D1CK+Y|3zZROCQt;wToO<0d>^d7w@?QxCXBeo5!=>)?*zV-AEo%VET z@SeU%TVdMMYj?60X4`sg12xv@Ek)>B+d$bjl?^+|tpqmr>Tb5V3xnzHw5@P$!gz1e zv@L=CJ#4G?>#!|sf7iO}!mR2mNsmGDn(9zJMi19xw7U2BMt>UYaULEoo6PT7i0mHE z`v<96Q6gpnQ`V+2BjpJ>PI(D8y(nR(UxKub%*v#r>XXlP`$ff-CImHnjYYiw1{$v6 z4R67Qfi{BjZdnhviaw%_gi1Xweq}V;PT$1@OlT6LXJSG9+X*pcq+gUqc829&nOvHw zg~JcX17bSGp)p;e6m#6D?+?noLTZg$4O{Fxre8@(iI865JLS9Jc0U`sM`-q4S(+kS z=-s|6S<|m;Heh|qm$Eedx}fXVy66$EV5u!=Im9$YDI%yNyuUCZSpTk59 zwEPBrZOm~VueZH@*zwqfY%3D~z zw9x$$I$sZt;wi1HXX)sY@}-p}(u~ENg%{XZd|8Xr>7*Kx*W?*I!(K4zl=GVW>+J9p z65;gi2g-k>WuYk8peKLmbN}71$#>`Zd)t4fo(oC)$O86}m|=qNzapN_p`K@W(2`#X zem(is9vzroHW`44gaILYFw}k`kPq_o!q*wsarpX+*$7{hu;_ksgd8QlUD>A$Mo;~u zFZ{i%+oZuyq6R<1HTcQ+KJa7G;3u&^@H1S4pNu}>$E3ke;&+7~lLkMDq3~mh2tTHi z!VgQfx!iOT`0@S_@Kbl~cZHu7*Pb%`j33%J{QQW2PJ(}e$mYvA$YXM%$IuVY{9!2$ zrk*_1(jm3INGFJ{Cy8@J_`|aY13YCNlG$GJD^&?Yld8T8`~g1^#!m+FKPoSi;hPZ{ zM_u_zcA(YSNG`IN!#aAGg{AL}*yu7jF=SF>i{WeT_F>_5EKYxZ3S`(xkLtl4*& zdgV8Zw4pB5hT4C4tJmzi4t9sY8_5B};H|zp0=`;q#8&}$3Ceej_{n>^@zW(&eOLU{ z`HNEql+8o>#!o-OOMl@d5`Po~Hve7x(UFYr3O{8Trwl)T9Q-4H^ga2b+j7uuy=XG# zTbCy=$p$e-?G9MHfV~#RB^OghHB(e_%^q2|cPQn}gYwa|ibG>s0A896?Y##7l;bg$ z#TOBk*&YyX1XSArgP~)JE~)+xmyR;=|Xn`0&~w3@`61K0JT1BYS&je0WV>F|t8O z?O}rU29i^3cOTO6!0*jGNw61Jgm%7`KVJ8*pA?io9(TMh#xm}AS2yNwJ#G|eMatiL z>@t(7$ZIKnE?(R3BJTGyVZT2Qlwpn8o>}V~chY~A^bg+y?lwUNGc)3c;U~X=Cl^%Qon?uuNNKbR{kFon=g_Wjiy- z;Fa0C(iAt6e7KS1!;K;GVa+UPgJu>~)TreN0ImFX-L(X&!flV1c%!$Whn z`fL{k>6EG>5S89^N)?gLY&riq^u$u+xCH3&fFZiI`wO1|nf>%L-DiHV56TmLdU;e6 zls!#)dDd4*u)osFA0U|?+N_tqY=QDsNA>csZ=sz3E%MZ4F!y;IlvCPuAJPAp&V58n zv@_(2?|OzT(aw*=4Yg4?7R;x$wK52UlNs$_&p7gZe%X1}P2+KC?7`V&VJ96HdFn zGbD`0cY=iweW!unVS3I`ekZvn!0$Mecfyo>>ykBX;ZgPC*#-sQ@o0|ownscajP#2s zc!rQh`a{kS#P5XUhX;D+2YPorG=FK3KHRd{9;yrv=)(I(TmLVhEYB9D{5nJsy+Sh78Wd}*XtNEulP%WuRd z`tq~qg5up&i$h0M0DokZNBIrw-=)h7PGSB3GVqkv|K2W~_E}wM`RD4wT(a}K=0%ud zYFEJ8^)I8`{U|Z$nip8-C;2*$3R!1;2z{)u{(4;7v|#&&`3-`$xOmG}_8_h-EGD34 zw=?wM>AC1UMB1?A!=eI68>Zibgai=x0p={OK}PZC=)_K)x!z<_NL}t|@P9#l|3;uM z8~a0F03V#-^2W|So(lW<;bHGkSVEA2YKO?RFG`VDm%4i)Y-q-<2OpKKGP?%Msl-Kk_4dodmvg5c%-DFkU0=(!MatgX1-_xa+A42!zf9XMXimyi+FvpMCb86R^eF-gkO*tOj;=a^1oP)qBOO5c zH2N-sa_gDsFy$8712l*!qdpSQu`^ho5v0yC*aI~5L!WU`s6Hdq9-udE20CZ=s?Fg1 zmZr}z+51ndE}YW4u_Trc70_5gjXA8#3~kK*gMy^r+^@XK5J zyMBBR!}^U6)C`zz&YU_zP{%D%YB#04*gD%WS|m15Vh zd6;NF)@V;!9q~LJ!QN{;;P>$wAC;ApPdC^#Yn ze_beo58(H{(Yq~z=&rPHVQ54-ZLfZwnt1euNb=fV{W~>z?F)U#YqhQ))w{lz20bfq z=dDi&~kSHUyuuUn9t4bV1Cory6kit>K0WF-R|%eZr`n5%kj$@p(V>)Am!v z=Xb}R5uw+LogA<}fn^ zeZ5_cCUGeKk>HQEgldQhf`PxEg!X^I1_(#O3=rmA?CXp%m>KYq1eqx8jA14UC(J=6 z3Ij8@`!=6(%6147`Vz3Q{90PJofjzc4e|fGH2JFfedMcUntZjUKl0UGntWA_Bww`@ zeGzXm`JORp_JP618kWW7Xdn37XkB`ypt-?E-{>-~*U;SH+knjLwN@;*>N2mVQQEd4 zAZ@!3gZr%x1oxYa(zO-3bggJ=JeP@SnMJhO>;I9YOcVLylOz*66lX9;kivx(2ybxq zWCu|Aqp>Jz*a=jJXIr-^^ZB&!bNHB$^e@ss!R~{b#@< zKaZZK#Sc1IT@KBY@Y8QJx}MPGD&xT^5~r~0D) z*(cyhrYS13R+mnZyDrP!r2culK}Yhh@MWg_Uq6Wb(J8NQpX|#2!SxjuZNv~ShR89T zJz0qC$)eth6pcRGh4U-OLTLs0z z)1dl8({GTyZc@)omO8`MlQ@$l-f;Aw3~gFZ2CjQg&dfl*jY&OuGXu#sY&|n(vP>I} zp4l^5_R>VbQy$~~JEqyV3|`%n7?fiJn%<)2*Eo|sQQmugjS5#vSbmL|9Wa4VHUC?d zhDPw$Abgc~-Y9g`2d6u>@qEmlFY_FjtrXG3b0$hGi%B?iGgyQDnIMm<5QmO}{XVM7 zXn%$gtCY!QVe+wYa?F0Te;p^q>_pRUjh(0|N-!dxK;X0rj(Elakko`C9(XspUcJpAqFhuQaqvzpf77Ni zT=AY}Vx^b+D|F<>j=#jWq^( zlD4CjqORgP@1S+w5m8>g1E+Zhlb6%_HmZG5lN}^4-@)bOqh9?{U}p(0^x~GHk~1gPI~)XE4~+TmCK24lxI-HN-qmg>+L;2EpQBk5@u;A0fXp0K@7-V*Y)3HEKU?Q zU`QvPQXaSbUia%wUl2V|V%z4SiZgcM+M$46W2ZbprFpF6W2HH)^v5-3I0DAbVx@-{ zQQb^dy7`|}n!!q`7XyIemWBJ+6m}1LZWe%iL4u97d98q+7|TjqSt*B=K4GQnSZQJ* zl}57CrnB)N8hZsR{XLmVBUtHyH)(_yLTP%X4)1xMClgKX3q;B9DG*HVp9)fgD7A~q z<5_~VQ@!Y+6aHfFyi0+$oU<>%^uEP}2EFU=gNwBc#%@wS{dxc}Ab>FA_!NrrKtayQ zBi6ee!;I@y{)2%XO7lxCr{eFW0^nAlKMNC);H4n&0?|90?1r}^CpHYDM?OwlvKC?3 zsoZkH2}3P!iW2>mc4-%2x@ggJT;p@Rlm!#{9NgZ%zngdfNk=-M*GVRSppl+mJ*h(D zRw{JjB_$T?ieu0UMLKR|BsH z3A}b;zeQgTFg49~7Pq@{9MXL?f;7F%@>aA}+GzD0?y(l0h;a>ozfta(QL*kjprcoz zBYWmyt*3{nCzK8~4TIWegKO`lT4aRPVelTL$zQ1ddhLWCQ(tZr42KQLzDiW?hAnYG zJ^I@devfxxxf-2WYxSInkt!_qZNR`LnjLTk7$!*e)iBBxnSiwy)RBsM zP$gmTp1>|*gZmS{{>7#NI50#h{`Yh6m#$9-nke=uhQY1Q$s(q+Iwyl)NAT-#eof}r zM1GB9S6U6^e_;sl4_Fp=jYH~CUxSattNV4jic*#OAl7_z>Q?%f(6yyc)o%_{qA4Y^>DVl@N$&*W^ZJtkHm)pJ4 z)HpsmyVR_+C9ff<0DhA*GLEAWV4Z?OUxk0@jSu5X%% zP5FE^jfEAOyoJF?G+QJqEm8p@5tgSBzN!p2^|{5hfGugTQ83t`LE>b{?iK3i_YwNM z2)Z4*%Lw6Y>DS>xzvLTW@}V9KfGNstlbx$PpO^`IehshkI%Io|dcy#dso47tZA|&T zdRHtX$nxYG`KFOL-N{h)*GM;w^kw6DRJO0iU&-57nE_pWip%6s=9*z>YcR)&V-{Pd zI_zf=9GCqC7EdlM8o;mFUS^X%p_N;(#wI-;SDi~s#}5k5;=?o+OOe9Qo#_>s?Km=A zLlFb`eh$-;EX23+H9Sj@E#stVEuEr&3tNo6Liwk0R?Azzh6UUYtrGpUnR`XW8ih;a zey8btdctLRqTNvcM-jM2hvY0H6n_>Hho9pKX)hauHZR2_a+GfxE{_}Gua(A)u*oqc z@|+B*#g<-fS@I4|qP<4WOO|$~ZRnzso zJw(miyL+pKhu2`8P0GpO*ui&07sX+L;TYia^Eg}++EC3jrM8w%giTTEQFHSNJVyd} zx*Eg5vjV|0`ilVuKr4P8#VP;Q!$oThLZuc}nczooD!v!1@1-2cqd%-8Qc+Kr1Rq@1 z-;F(9{#b%3j6TeBN}GT_Sf!m5!uNzA$C@8L%S3w^P!~{(OTHm$kqdoQe}r4KJ(ye^ z!oDYej-!pOn#CTy-d=AhUPte+&@kase@c3ig$zr}OjdF5di8;A5Gi!nO z7)4UVglWOAdX!#uE6(-@B;fa5X*QiKFLzMl)9o;a`1mH1E_n$}vaR_?w_@aF&S4a_ zu4V&jr2g-Na8fT`0A-^^z*T#IO^rE2T z+I{3Z4a=`B0KWnZgDVAk+Dwt7(`$gOC=~`Vej>ZZ{jE@DV*eEine~FdPAIOJcb`Lg zZ8loOEA!K`)MuBT@LQ!#R?nt^V)<9mnO=bPTP;Pphl0?hJ^hRa289`^9 zVjF>yoKI4%Ygpl)h>y6>9BX-N!{xxe`>mcYpqH&t zyi;3A7QFaO{ z(=^5#9oWNpqi;_WZ&Zp%#$N2VT9#K+PD0@Go>_GQty|SwU&bpkbqCb3?0zNQUkCT1 zv=MKwWIwOK`=8-Gj%{Kr_Ff6J(shogOwJQ!JABnQ4(W4$-c)Sx3v2+G7_;6$i{%t= zoi*Yu5;04vbYBUe^p!ZjvbzirZoT?kR1XpF<``szY$ctNeXW@9NQ*;}9v-4ZIVnw0 z1i#>!U(soDsW6W_tYR}B2b0yDD6_OSk1VD=VFM~ zt%e53_CO1)Z9c?me;tEN%ew~|Dl+z#g|lfdQcf&w#!6`5hGG1IkN(Z* zUfxt*%NqO!8&D;5u|li;E7ih-HI|~;yan*aNC+_-m^yTN53axKJV*ZcIA?LK>|DE0 z5aejt1dJ14zp*|{I;noyFu-(?-(vEQe#B&ocHgdE-#7q2>&p)#Y#Pip>FXd#e>S|f z*vw57YLNEGgGA3Uvz1P_PD^gNh&1dHQVv8j8nz<1&uHrND(v&GMzw)4bedG{`2vVn z&)~U9M&jjnKj}m2rA1PMbM3VT!i9(E7VSvfq93AL=n&mPhv=4|Lv%~fA-Y9BM7PWi z&?YY_3H*3Wgg%9i0}+qTg1#cU&8K;4{g1zbBf{990?r6M}6Zsp8y zg_$wcLjFy0$iHR=O)^f+rT_5zlv!hOtHs%&W6lmYA1yg6u{X3;vK8LYZ+A2DIt69eg$ z0j^eKYLTa1NsQ?9dU+z_X`dpV_8B8jn@&9KAb%ZWX;YgX#mw-6JSD(oPGMYT-;App zp1CdX%xxizYvgLixH{`4Yeuz|Fn3cfMLQ~20(aA=A zG#;B`%v{hqz0zSk6=^mOQiXOZN;2xBx4Y;2{u($DRRgZS&Zvtn{Xz84jI?Yg`7G+6 z_nbsN+sk>!f#;nt{WG0+plbfNz`Vk-59clf-{73fH)rTmJ>6%&sN1jWqlsT9Zaoi< z07s{5$KhQ<{@uxfrz_s_^goE{`c;NCds^CXOYsZHLP|SCsRNkise+QU&qPe6D=k@2 z7L>j%AY5;nnScbV6d1%FL2{OQy38;Go=)@Q7neopK*V}^$GYB(6^KPe+J3_ z>3eURC}N*oP~p#;A>>bl=^3BFv+`b&*6kL4Sd=Wf5nu1*OXQQ*vmw7g{{W zW9FV|$GE1xR5L{(xtg>G^`htC2~nAP0{J?AQtu#Ur%OAj@8!&nrT9hUCpVjP1z4+~ zxGwMq8U?}2vfZVI^ZM?UhV%MvZe#w!ZZAKt@2(FyukSt>a$euvY&fs)HtFZ}N&ws{ zI^5XxbXD+^ZQvVD4=0edvXXL_4rStc(MuoGzD zcti6!?IKA2K#;aM;>#^X2-pq@j)?WBoRY}!=r+Vx_ln52cq-yFfGk)4pNSGT>Goe= z18*nE<_c7$Ca@o~KF1&5EY%)S+u)NO%DaW|`#TJynA$7N?n_a@yiFAz@b021 zTNaBi_wbH zOVQ`R;o#9%nJ5U-^40iB%5PaRwB>7{WF;D7m8t}(xQsp=@sU;f5YGSZpD-~7;5GFR zzeT>0n%E@Y`VT+&-~++(Mx9XZyHFV{S+hTMrC1fG-!6TfT|UR_n+dNq`oH7@&b2BN zqNSR0b%2oFJa;IJkd|AHsT3MibbN;0z<-36QBqy`5omdI?qESqY7iQZcn+DHqB6Hb zomtvi{teW1%>B6_kLnT{zVUovwn;7JKKQX^&IN$yV2r$UNoIS~RhfRhdhRCpx{sc9=n#6J{!hluj=ovsQjEhR{bmDuSYf_?=F(SB&oz?&p*Nq5gn8&vVy+-`LLK9Q6R}zEG*{j*LHls-;xnFlk2b|fhu1mFe=xB$sN066N zb0M`{h~Fkiyo6>cJ?_0U;-A2XF^)$tK0oRBk>wGH8KUg6-N}df@JAdz$?#dr0@hL+ z{u_Mwm+SU!jvhfsg=5_^dO)+O;U)F~d*}m_-~*J959rVMm-T4Qk3B0nmF%9OeZ}mP zeZ>szD`p$M;uZcCmz2W9_b|UG`h*!lpD-iv2|Go9xwxo;aLl4AqC%cy$XRs*tO3P! z5{Bc*24z10jj)1h)ZGatUiB`+IqPs@)FXEVeqY0W$G555;rFItB%}Vg1!hE56Opb|Nxd?O&7lDr9BG56z`IU=6$8Zto7%l>hW*1Kh zC-87vsNas6$FH;b^?rVx!LL*KHIH8f-bH<&x0=AMH*i}PxLp&tT^+bx8Ms{$xGmLg z<=D?|)#c&tvw$QcIW^F5^{YD*OkwhGsotx`q6)Kp*FSF@vCTHTyr*V(eO!%HUhhjY zn`*3ufE-H(mJFC)8K$2aVMjsn_7V(i-JpFZLc<4 zin`!#Y{{sLrOn&AfU8a(uMEDJld}4`78H^JQK`@=3ViiJn^bPzP2)uv68#&U*&kaL zUj<*`g&GvW7Y3r=O{l`6;?frRbRxak7Kn5 zs~=(2-DOlgfK{K5)pua^&8&JqtNwy633tB%)pufb2CIIbRo9dS-i<~o>Y2A2&PPUm zv5cbN6vusys6y%d3Yo=^w9``Tj++CjQl%hKo$%`u{?TUwR9Zai&OpehcE}IKiJnby zfPXNEHGZhrms<%w}dfXG;Zq#GQd(^8~zpc@h(2$8##c{H;vZ>c3pkz_5jYr zZs0xlWAA@5{2s@CUxU9dxt%OFME_RD_0+3n$-Y=PxRu1AxDSEvB@}jFJwPk|48F&= zZvc+0E?_dRD{h-Llle(T*|*nHydU1}8)81$a~97VQFvMldiEE9Ykl`b2Pu8C46^e` z`HbQUqd(%L8?*yu(hXdOGF-^+bRP~Ey9?QMu5>}#LXN4t$E2M`dM^x$ zISwRO0FwJHqRLBm>R|L-{#{Wv-le`3-X(`ZF)Tvn@Qt^`!~SBCWy6<4so=F;x_ z*b2SIakBvQ3z@B0LHdRsXKan9}_-lIa zRKQq!IVkc>-^RBPtiq1WcJjALje!}df>VI|TVG1=U}@yf{x8Af`%&Kq9#dn3;qli| zKMXv6Vfs&lM|7b+DLmd}*1gvUp+V6vH1{p{b4_ov^OU{#FpE=vljXgl(MZxzBlI^D zg$^jW0Xi@U8rH@lVS78CNX$2_!przmRtNuyLj?c1(D3sb_A|b#c(a+)OHF$8%tYkT z57YaRN}@HPFE&b(pK49jTRnciagfD#0!LJ8c#K6|!eJwe7&L4ouOY*p`gO!%ulzdl zuvc|*imna2;Z_>9kyv8nAK#tkAo)edu5F#diqkcNLjHK%3zlaehT%|p<;9&Wjk0gY zF&M%Hrm**YuTR(CH&}b$IPHDyUGzR9zcKcf$o$3?w?yVQetHY>8^$Fp9^K;)iYH;5 z-#j`y0=#~Th@*OJj)`5f_%(xHNAT-#eof}rM1GB9*8zmDWa6-d>Ky7#_?}iUGUF^y zS8uU)d+_-T*;nKF%sGMB;Ll!V-fMJ8IQ*(+O@0Qee(QCre(5!;ejiHf-^S`DXYlDR z0>8}U$IQ5?hbf!(5KpTce?kx-}Xfa7ShC z<-e;R62pd8_p^14QP}xis2<3mESE!RE{D=w4yCyqN^@uME0<8@atTE)mr&$#D9+_j zoXeq@S@hy0g~3ytnd>7OL^PN00T}<~O@Tuf(0O^F!(uxq6t}yFh}j$6@lIvo2;W&K z8IWyhBZScyi_PM>6*+f13MLNAo_R4=i|4i| zb|53$+i(1N!$o;)oG33$l&8kXvoBMxTRYISsH+g(;=ZbBoUo{iz0gFK{Cizz!&Kw5 zZx<5aV)yDh;}ZBIu5+Ya@8OM*Ugb&)v9sgklS;JSlAieE$fTkUXa6fw_W_{^Wmbq@r?{z!EvD!)c_hiP|UyK1!&D zQ97lIMNd^6Lnx~4%^p8jk1h8Ka>C#o1gd!9dS|^m4Fy}~V5fI%qu!{uaeb^tiG(X_ zULjn$Jtax`{5cP9PP10BIH=rJ@v7GkW@5b60SQ&|>xt<+6H|{fhOK~jlNl|{ln`1_tl&n6LrzWwUrb16Mu&068 z)4(Qspc~=j-6X0XzCqo<+>IHI-K1eRY4B+!FVUwhqf5fS;IjUg_-XUTCK!eqcanF! zcfW}VjL`RQy6?~Mwg~k4eQc$k^#&R9x_+gN@8nDfNAEctz2|WBp2N|5-%JYAH{(YS z(-%o~_AogX8}A#h4>VN<>tDFNf}9f*tgoJ!$Axf80U)EtMyPY&=t0pve9#q)&3H^u z?|y?I_dNI^4t|JJC$Jv|i}CDNxSj5CAq zBU!7=CVhW1hUk!ga^7`_Ud|Gv$!Y|)tJ=Pc4UDl|K;P^WfxZDV=1|;)#O-ZC3oVS| z!47%qw`3X`yD3}azg72Oow2$LIPZp=5(4}bpLRC4y)W9y7Cqvk!f2nr6-ImJz%XJF z%3tM?FW~uRjWWP`D>04OQocDUIyZ4{tiN0up9HL=P=4e*bDcCXNg9(VkC{q__RCD@ z6F)e^5#J(sj^$Vumzz+`oS$rc7Y8n;Z*l?7njf7^+~^T?%9sQm3W1U9TQK&e`5R~Y z*BoHPYFk>Cc9WsqjL>d|Yd6W-O`>+gPiQe|cxA&K;7Vq6N%fhzAJRduXs#C1tB^)r zW|QVl6+D%^5ef|c4J*`?6d3VQ^=0_QW}Z7$Gn|PFV(1HX_@ezHPnb&6;JKM37Of8D zvE(Y-kmN!^v6|7%`&ibTi{HBudrIE|lnCFW{yE1uC++HJGB$rN%fvWB-s%ZxC69-R zPbwLoSMty;Xt9u3GV#`OUvkO#i6vtumE=#B%FBI;CF5@^SCdP|OfmL}fdup!{$7V) zIocr|x0VbnxyMYevzFW(1@E(#V3Q7J*4t7$GwZG9Emm`twY+MOJYlfptN}!2 z%RFdHZHB+*I&(#N#Za4kTOzzsdzan(kxiPs2H!@nN|Kx(^zy1?$+%44%6`v!Vh55CMUb=xyPv!}MfUwP~($-dfLVKX0rw^^k|_}MBo z*yCI6sa^K?CR_A33S-kGax7|j|~$J(VX zd+Ps3-Mhy}Rh|38von*MOtJ?E7ZoAVsH9k%2_P8|$iNKj4SSGQQau)|r&heQ778-6~rf5C@T z#wXSKjb{dk0(Cil{e7z3r|h|gM^$}kI)%3AfIylAnXGvEtzVH`Z@agpNSzo~sF7VbA?YMp94GYlDk1ks5@eu6svth*L1|E6`{4}4M+H}3w( zZ&YS-iaoT>TNBNuE#-6X^|`4FKQBf`O>G~aea?EH9;L11Zkl5Jd@zz}R=qW~GWHH- zQ1UbRoLp09_ey(xdJ}C-_g?CwO|-WK-Sp`PeD0k-_bKD&{jIva(iYyBdr?QFOVkmS zsYwgRxT8~yAB!eyYV6d_zff0DdE+GG8PUVKYQ-TSF{ zKQSJYs;82PItgc99hLCEHcg3?SL3P9u7vB$EbN;#SMIDiB`>upu`P1d(z9>?lSEPl z|I5T{PK%pzS6zgEn^XMX(e~o`jFTNQ6*5C8L$G`1S{=dk-J?%;5-+nXC{=dS% zd8nO#g@0==?ase_So;4q|90r6RQ}EQfxQ?0?UkGUC;Xd->iqvd|MtzNnSc8?{>dqQ zI?lg6J-P?~X6=9PgZ7^O3T|h^uX;pB01UtG;Nu3EsADVq80u^s(DHRF*tWiklKNIv z-=>DoyENUuT3+!Vm=VKQ*aojE!iPW`EwT$%V2U1Cs~EmqyfhO3xE^=3_0Spu2+RLy zmO3&(OA}b8=zY{k8NK?ms4%@w&Whh@J?o9_R>K?H)bJ*kN?64BmW!H~7uRSJ{~AJ! zYgGv@JJu;hO>*Vud?B`Cy+S0fHIo`^)bwUtbfvvGUomc_u9UBG??`l}9^Gy&vg}*K z_1p?7g%`c>Mwb%yuZp+Ijf6WT;aL#lkf&~iqtx)3Hl?AV4cbw8$~J0Ytx|Kufy#?^ z;{&mR@{~5E{UE)!(PO)+pM+05+_@yKSuq?uV`}*9F!`~!v2i1TD4gUKwX;i^2+9C( zK##xVxn>wc`qB2i#TexZc=pLp`u8ZFvjP2RKtJ!5pKJ(tPN;gl{A8`_=_sRjUuDIf z050^u7kaoX|7>Vc;ub{5n}GRl`Yf;P@{Mnz|JwLw`ai3*NxluQzS3s-wi>zbtZ_|p zU*EW9F4xB8Y+SBQlyj6?#oFY)j&Vx3p^W{Utfes%7(u(JZ-Fa)X^Yan-@SR2SANNn zM;Z3Uc^^xFFPw{LC9l-q^_Egu;`*^&}L2Hn<4rx)OtwUNAX-!CLMp_fnnvvEf zyPN}&MDyz$q=Y|SriSYrp|#Yv-~`NSB(zE~Mh#SXgf&OfyZPItI>w@ou~f$d)G-0( zCaULb=tI?0BUe5GFqLW&`W}62`$4|J3dgG9WA@-s0^lV4LQ9sHg~gzih}DOPs>JLI z#?s2!t@{XA(hIbj7x<}@o;vv{Mo%$beOh?9P^E?JA8Jb8@idk-H1~bw@Z9sb3h!SH zm3d@(mZ~fCXY1y7fsj_~e!`54pJdGV*puQ6fPOZN2WG4&W~KM-BKuyE{VcL4Ap6eZ zp7&eApDbbg_&^4*-X=zocV@8DOq`w3ocCjk|NAp6^mrFCJ*OAusD@)8+!(HW5Gmo# zVe->a)tJ$&cuvSatDz#A9$2mEe7Y#F*hd}6Z( zMpzs@R3lS~hWOw_0ttAaK5NE8+&3dUeu%sMJg%}ck}ukh5oiQ!s#LEK9nrCgmbDUX zH-h7w_~*9x=Txk3p3Gmj@z(^pHD$s}c#v1>90QyMdvfKk(ceg58JtrSNo!OZ zYGZ6Pz;4qTA%78u(*{T-+JL{))9itgw^|!S|7H)6kTWn1sRPnVW)JuWB@RlX>;Y+T z!pH8x2qnfsuOsY2+39*IIK=z7mToK2($^6g%If!aZ*~Tyt+FxU30m^q$KFuQ5Hv{XP&P$zvwM|dsM>`(vxVKG<;R?Og&7~I+sT+UH zM5;EzMCv{IVGmE{D@9DhWRd+lk-ZPuxa4B~Qe?Mr`shlL-GS^1WItTgyShkHgm(^4|PuZeo9$<*NwElfbP-%3WGy&Tv9f#Y!Y+<1leYm-ZkXSUa0y~J zc=as_p0TgreVUoZmDmtKcb3DMt*Y@z2^@U9@i&^L^?);|U1&e(b>X8=5MW-V%Y7@a zf*Vca&a&{)u`p4)cc>88vL)Z$PJ^7s%UVSV$69>GyJPH1JV&()2!0;!!vI+50j|gO4d1p9Zs#tSA`NxwX|5OrNvS$9hosbW;kBN zt+{LQFS`-eCsnavnzPJXoe7SVq>rVNrf)UJT%TiujJ#4H!8&>uU&t}b) z=UsJtbgpf#Po6jaG#=!6U+uIKMyKB-9BLN3-V1)Zk6N3_9MS`##~e{?`)1& zOXD4zdsL?WqjOeSjC0;5tnGqPypjhtq1axz>nneJFx%;IXA?KpBqO2LnoX}f4?$W` z&Gxw{BF{Hxa0T*CuC5l<4ai3e@&Sa9IdP*$+E{twh6yM)HQcOqGy4_2qmZJCpp0T%()geL$AdJAgW^1^J zpL5WfEn1qL7DOCE`OhiuqpoH9BtDbYOxI)Cj&*^}d%S>@@lkwzDgO}X(_TR(^3GPy z+8bYbQPDmu+2$Et;KYDNUqw;vF)YX@SikvOXT+O0jKn(G>g|4HZb9Y=WS-!4^E!s*O1T1qqeO`8z|+t~2z!hkmgYcVVu?6AgM7mI?X0_JN2B@BZV?F}^uvx6q5c z@C&*>u_la;L`@*%ZqO;*6$5}5ilRc~8 zVmi&}4yuNr5b!kr08e;h95d|2${b=i*uo&PZH85NJ0^dp%w>Q)LD_TdM*-GiaUw% zp}#Y^q0OqX*u`FH;Nq`*T@0?eKhX4w)xqJ+%Qmaa&&N7#w#6k%jhO`zuQ-#P9O;q;j2EuK2Ss+1lFTD8*S zAgfw>9K8NA)XTSHA|Wa_OLmEFC-jftXzo`GXC~h&FJGe>55(LDy5^Ypbf_SQ7Ux~A z#OeZ`3&HG3MqVi~Hw!)x4ib6xFkUsbQI+5WbB$88pS28BJoKOO8CrCC>onX>6CY6F zc*f$9#6ru|Wn$_ZB~{wE;Oc~uB471Txxi|EU+i2~APyN@(}=5Ez7fn}5juFw&9)Lo-)%iy`+)V3{zFmZ+K1?uT@docvh4G*Sj(&nasMwp?U(ga zk;N{sp7nASZ0`g5)_~r?Cm(@TeAOY63Dahr=g2FTFhBd?8o-)nJnZT^*MSmJ_pgsD zAJ@j?$7QII8Cg_UBUF_nSZB(&;kR$52wJD=@5aiTH2q_t1;ZK6d3kXoHM~w6aSuG2 zuGS*dUIaSq59G=@toVEw=R3o+NT6v^|LKO~I>pnm=%Hzb?;4ltsb4tH9aZ(H+TIk9 znsIRjdg(zV=L`AVnNh6ST`Zp0hmNZxfygV)*%Id)R=yC=>8Y=f&grS25YFkTKV);< zerRgpLtMQ%emIaU=vh&TPRo@!QnF|lERMmO%QB%4a5-{WedUr}v--GTg75P6$;T;) zOY2iCyJq((llvm4FD0{j0-5|tv3%QGNG_K9YJJLtGO9i;^k-aIye6gQJXusMyZHEp zUi;EBjX59xTDN->s#v`BcaRiMnZW?Y_sBNThK~Ua}=iR zEt!NFm6`hY~>PQUbgwCo2KGkwq?VbTF`uU9c7J3>9tXI3 zCRu>*%la*qN}VCe2R>zc72t6`CB*j;?$fNvhEMz!99^*`eTkf~q?aq3fK>MS)SCzMOc z_ezp{kC<|rv{?jM>n7jtzqovV){^hfkbM7&knd|C-+L|j-fPMCUQ51jjLY}gB-__m z@;$-R;1n5u3d{IIQ)PVXh8T(XorFPKho*}7R<1X8(@gWm<3hrZtA6-$%2`6tgy{U$ z0lmdL%iMpK0mYSkTjkE2u3Hpxw__#q{((8Ri^6Xy!&$S$`MEp|Eo>Dt6Q8_t3c4srp`DB;YbDnHQ+QDNPiWUFVbk zt4)5Xd{6kuGN0bE^W*H=N!5AR!O5>0YtU$Ut7sldyJ;ntuwL=lV%96#=xxI;=oPOS zAhePqyFp}Mj%*mun2rH3^|{FW`So{*9C_jS(2AF^y-(u0jT#}Cn3=`L>AIhYQUlH_ z^ewS+m)Dq)2}dq=f7PeKA7OTj?S#&v%g?|Ym%I`RI`?KJ&btlqK0ZqJmW*;Lh&s2; zRaPAwzox8uR3Am~adPDe_`2ZpOe&Y>te%y#t~w7#(z9gATQbWjQOe(|XT^`D8$-KV z-blm!Ke_UywB(i%OQc)NDXmg3!s{(;4?>UuA<_^$8J!viy^B)lqr?E+V4#5Fx*_ zO3lZF{A%^x;bY=T{?;yE3VR5tP2eN!GNxyFpgMU(L0x%+EbD+lOm~WA4Ok}^`34ZV z$m>r?r$5U0^76TSF%DY><;OoLwmxXW2hBtQ`{&~egwi7_*N_%^3M%}YF4Z0sRpG5J zs%7*Vs%7-LM6-&gMy|5qI{1~+e|L%|$ERrQmrW6T0(P6CRZ0EDz-T(O0a5Nb{hK|A z>Hlr}L`P?*&JW5rp?tHLDgK~1{sGU=>`Tp0Vmf@74sGCYQDpy?l0M5m=&fEX@vBw( zkpaJpmtC5MjNc_PexJy&BjfjpjOP*=4rDx+$oNAdBOMukNMt;p$jCs(^NEbLiHuBS ztW9M6F_Dplj6Wta{*=hbM#i5K8GlY>T!xH4Co=w$$moNNza%nVNMtyX@j@cwuZfHt zWc)Re@wY@qE;9aRWza^VjE}QVdaG}f2`9du(WR*_>4um|oQi1aYm|sDeXXib%~ExI z5$XCOxYc}4&at_7@ZsE_ATS&_4>tn*z5$*oS}(+67m*(#>17!Sgv``#lw?u65t2pi zu9iTlHmV%9L9XnNS1|LJNpXTVOtJ_be1Jv#t}3G32v!HHQC5{Qz{&kb9edDQ{848Ibs!lbqF#u>; z#p=-7);kffvg}`}zgE-RlIkByb$d%3h2Co4RaAeP5}DRjKf9Klvl?*|(l1Z%S|hpp zQ9@xZlgQj3GKu{CeBm#Pel~mv5_vd>vyXH7=qi!D2ieDvU7M3*>s`Jp6-=?;#U`k~ zftg;oqRA##{(?>VXfX`FlD?BoC@xqgE1$}hPdThLf{kA!i>;{Eu$A9|U`ZF#SG74< zNL(__0}(;LO^t2xR%2Q>#t1XYFhNj^Z^c4;sT;JS#BUg7PKb&A$P%YlCZdNkhifET z{5~mLBIpOzn$H~S@=i){L3DO$^5nLu#?2D~o=z5}Re5RyxFwrUo_alY%5C zZT9JVQuSpd&k;$)>A6cwXOf>`k^Br7i^YBO95}dlxQwa!|7y{2Ls?vQ{xx>M2%oea z<#PDj9R4=f`ZhQItrOomT>)6dobWED~dqTvZ35(G(gu7gnJKfxYCHCKF^dDz6Jqzhs zB0Ur7nIb&{=@}wD9qH*J-GOw6NVg;1F4EJGo+i>Iqyq?T_7va{^eY&2l zzB(|B;8*l5uy`2gEtx%lhBqY7_`1}UK#VBTNCNE)?e@d?(C_l;u5R?nEuw{0gB|-T zhsGHH)9*8$yODo#6#Cf!uleTwEJlM1M7AulCm|a)LguR?`&Nb0eax9Des8AV?}#nKcbeN0Jz z+iBY3!9I~P$J#VnT&NNnCRv*^jTMy$oTG~nDXRYDGS*E#4u_=rTZ9}}BMfo-frR4l zuu@Q`(#ESfJW<)+qGoSawtuX+2}|Em7SN|D;lmQFiVJF#nj;fo1HApXl6@A=Rkwep zxKF71w0u?n#&ASwZ$G3IG$``#cTYsrr0u5S?ojnfE>&MZ5@Cbl-lpn5DpvJ2)lGyBNs{U@kO+wiN_i1`)4bf)ReK4Re zn6K#-Ye`fKxVHuLA1zTvB0}PdRRY3SEADz#{|5(3Lv$x}@$ofybJx;ra40r+yQ*)1 zpH$Ty4d^S@a~jEfzl?HPO+bIVk-sNVn7?o0^tyolOZI}}{yd=nwwa$c1@u3e{B$~? z|D%nc8Up$T_9knj-5Z6rOMaZY-O(T_!k-hlXoLTG;EK#zQm4N zzCOxMOPEPd_wrLlgPnD|8{*|$MO^Ow5%F@vZs2mQt1s?P;&v-8v3_2Qym=RJeZxw` zm0=J{y$AE97hl3>oS|X3PQXTv5jz+XZ@sTjme^Z)%OX2q@Ez87t+20ui7Vj}(SPu-i{ryKll^Lxy? z6y%Qj5kvm89}kFn>~|k%dCJz^|FfTvMQ zINe`O{VJK9K|Ek#M#SG#GC32EnPnvpXL%_vD^l?`J!j)NyR2m4WnTL7vXbe2yi{i& zU&;5J{%Y!D$*nnFYAq+?-&Ha>7d7NY{Ea1(WjxAOoG&d>vCJFW!#5aU(-+>{$;QuX zv=RPwFgn#D53WWqGgc|V1W^rl3|n-?G{Zq+ZL|E?FDPlc;UIChK@RJ@G>WC=s}Vu6 z;?`_F5TqfQ-r^w+%bKkX3O#v+ON!gK%P%=8-$`PbmMM9QHHU+;Ffk>hW?yW^I_%ZEKX?9{|GMN+Dp})sE>vvSH0G04MVuqr{z&=KD4Xy zI|29i3$qf5-pK4ir~Fo|4!&OUS+Y3kjs3iEKKS3ay!vc_X^L{uqKyk+E&NoE_z^P^ zkbG$=iBI1Ycc|E5(=Ek{w4GS8dgV=vnB>Fib|udC$|^~sD*`JPdF(F!`Rjdrooz<3 zH}aE8i5dDbjQ(r!^|ccNQ|!B`j6@mD_)d)Z`e3#aJ}YVZkL z$rie5#aU3L9V7W^=OxZ#mMHpLeEIPN0a__x4Ef1gTAeqVVepLM5$C`1HIfr632gt# zm%%o181$9%8ASbro>nqli>$nlQ$)B?{g%@3o0UR<2~-D%sK!#7=f!uCEXPyb>}^++TiRAcqk{c&at&FRiyvH{kav%BY|Kb`b*y|93#?w5h@~K6v^_->58+;2W8R<5H6CAS6@$p_kR=8@VB(KIu_Gmz!P@DHRsG zx8I-C{A5ktLd#w&Bw?3VBL<%D<{O_Atn!l{>zoK(|E^{ zz*4bZUQy5OK&X$;2OkF4Dm5QFQZJ@hS4H?@gLOj87CDIEC+&yBdBQIGgKH6QDQQ0R zy_k-vG#yi2>c|WXUR*<&I#?Y!CChrx!m%+SRX#x8ByRq_%mcD>tX&RSK7vEn;(`~u zpKl_5R*SqS`hXD#aMSdUm7*pJ1P#w_Xz9w!WeCP z5}bV!aF}*2{d?s4GSzcI2*>-p^3yfbjZxRBo-gNm$OH zuYLKngDxaZu6R(q;z4DMS43E&BT7*%F2hy63^v)V>hCARR@Lu0o@oUvS%fs`j<()UU-2h= zK4nac>Gj^nXOE%nJ>({D^=uNUi)W81!|jH%QqV*6K(ED(i9skZ5Z5SBU{IMcQ|c`_ z98DvH*%{j6BmAN>LW$%2B`FujxDQp2Fbr=;TcM|0--3RWh~JgKnWs80q946a#bwyf z6Uefz2eKTNPRR0rTs0`&LYC*ckY#muWQhy22}~LMnrb{uV7rjOHq^uypbp@y4ET~{ zyaUjKMnX*u;V#7aFd@#cK%9i#mId!+nsbKqaqql@*`xR_A()gIAUj zHthw6HUWozoi7DnhnqHw)adAT2?28dak#3B|5Sy8E|YOrtNI zaWQSJCZBYK<}d1vZ6wqV7zyZK(4r*U8)zKX7dPFOL3Y;<(liA~=+`@FPhg^^&_qpf zDI@LHoLJWY2P-2dXLTjfEJfR>9Y|LK2Y1Mee+TH(>FqN!1Y*%qZPd|qa>+ue6OzQBFSRHuiI};fBUB<}M7$FDr zPPuBLz{&l~2mueA4h<#;!S(FMQJ+A?HHv2s-(!1+wkjiJLd#!(nP$8Ubez9muKI%U z*^aXO6LM9M@p99p@bW!9@p9t+tc9O_jFGi*%%zPJTv}g#09tiU;3ABl$K`X8neies zEzCQ?)hmAXF-F$-C~;Q*c*XtW6%SNBwQ}W;#dzZQ`A@*lumQvtW;yKGD7Mhhh)qFM_Z_4bT!!~E+W6(hX>u5~UW-g0|){fFRRbBRz5QezFwzvfMB8^P zM7>}ML^HwUP62{mm)hSAjO;Ath--|MH+h$DLb6Xc1qMmxL)U>1MWpgTld`-H8S3%| z@v{*)6o#vWE*Jm7;&p-wI?LFuJZd$>j#jxbZ*05I{$MoGlENd|rWU=OBzMBT(>|w( z`MF|73X5Fqlm?NT%NxD1Ux3_tW1DE$e*Fhdzx#l|AeW``bp0fD;Bw7$BM8~_6ti^1 z3X;$&*t(mqS3KvKt=lHpx&~(J+JQev+GiAkQ!8fepy^};@;s|Vj;i6KZG_``MIaY! zy&#Zf$@s%D5xaZ>*f84F#C-Ze@8h#>BJj-{Q=K=Gp!cRSZ>)~b z3ho2E`1JkIyA7lgjzQH9f^XlzGOsZi_+VBBzRRr6Bbj_wMwvHUC)l&MTEEI0J6M2$ z!eT)iXbrSKMlmox(1B$E&wHVv%*{QPj6H^@V2_WqqC?`0rUZz>cPabBQg||fynE^r z_+r0>FLuQ7#n~SCqA>|yFu^B!9lTC8enddJ;4T<|>F+VhII8M|+1XwB-_OCeUm@SMgik>S5kutsn7g4Hx6rhB3|%Ei=w2r@uK-@97nB zyivkZR?kj?U=7{qk35isI^-3&RH_=2fIf)3 znZ)RWB);`Y_dVmNVJ%Pte5jW_>8{;64U1x_O)N$BJb9QQC_%_;3>I@gE6ka2zm7 z9t$7Kdc!A!2?U#j>#$SuyjzOr$_r1}waClB7U4Jfq$iXeepSeKOBE8SG~;^&tDnu4 zmj;NF-#u5J7vKZQoUdO%)P}pusGiAXbLHdk^ESDRwaTASLD@AqUv^C|V?;T<%pDC> z#ezc#R-16Ob8(&(kM_^ynO$*pib88*WS;jzwl}r+!+F+3>|GsvjQqznx zSEZ&SGZDz>uu@R3xVI|uRvN6S$XjR`X^*{`&*p>wHJpELbt%U5e9iEGpcw(v{OMsh zv`!5{iEdruzWee|vN(d$P9>85adY3eRn_S_S+4paTSy&{E4?f*;mBGu?#=J6Z(M%P z#!(vaUy~=b61524@d`&rFkWwhiUlEZJ~p3kaoPgXXIc>nhBVd|hg`|0tH#%3unC95 z2HZ8f_FEk}wuOa&?r7-I7M!D}z=NoQeogH{V;^z$&%hqdRh$v_)X2l_)cz%8S@zicbM zdntA5^MmnKZ^q%ynPPb*lMbjwr;zaC-%G#wf&(TY9E=6Tb0~%Y&5K~Tz={CIMr=kN zJ~M1_Ch>58XV$}VW3{ANMA}O47HJF8Y_~{T-TXs2#xh}$Z}W{`C*P)xkIF@OD#DY? zH#Q%+#mH?!ZZmRWwK~38IQt@c@G9=7Ty@Es6i;I?kNtnVG$gHfcKSLT?|5oLZSrvc zI(c{?D&Lm=?;Qx*QQpL!C~wR84ql6hf*T=fdBENLO32PgU+wB|i`-72Q?c$18dlb*auJ+0YdOvN`3Qenyq!Hx#X{0rr?KZHmeN19bKBn z*SIJJbz1kFRQ(ATFV{*Aq|&ewzQZt`@pue*@AtKC5;TXd6qH3A^ocNFAHfJFhL9xv zk2w?gmZ76eL}S_cUK~J7p?;M3c#I*Xm|4#M9+-sBwgJ8 zLK*O1gxemWWSO)Zylf~I%%pj@jpgnlwc)YuYqIP>d`(PVlQQ-|%9nZW>BIr_-00(~OE}f~SucWyxX-XlnR^z!@*_?3rDsnkVB{ z?g@_alb*6^5`Cjp?9b{aTEzitXcff`02rXtlgrm1@X#OdFd&NDCUHN2a+`!DJKzD# z<5iceN%NR-eBrbg!`eAUjP#}u(dfV$_i69?d?Be4`?K0C?bAt{y|Pg3+f0>kx2j|B zX2c=l2m9j7-V_?r{puPsdQ}hNNWR+q{t>6GyT6-M`&(N7eO>E+Kv{kk&?m3_vxU|z zi!X^pNyZlP%Ac~rvtHHTQT5NU7ltx5`wZq|aI?A^^es7@s{zYYrcZaHNLy z2pE3@L=pbLnI5&6&n!VcvjiXGg3$6~Z4)tiMqWf+QFrh=YR#t(czf-~T-RxH{$Xdl zj5qQGL(%3~jl9{b^Gn<|8gaNtY>hQ}3`e&Wn=9lX7nH=jqGyH~LyL8lty6?s)aMf&Y~x)_GO?5j9gE-?SS z*(vzBH^3FnY5A#|X-kUl!JFl0wuGIAF@D{de&8(NYRmk5d=~C`t%7;=4#({tsyt(xQuG$R zEd3R=J;Ik`GrvvCYi>Q{OmQ6>Nnf%XNhz zLNThYc+-Vg+yzv(cp&DJ^i@@K$-R1AD9;@;!}O{X(KCO5^aGFHq8X$2wfAF8t>2tw z)2})(pg+I8N^jJY$_}2ZIp{_sB}LNTpaS8J3!y&xA>iwlLaNELH#CruH+}KNBjCzkIw@5YC9p0eI>Cx(T-nGFY%=`D3}q+ za-VhskD9sE5LV(!bJ9Vv5+A}|pSVTy4648HAO>7@U`dM@|H1h95AyhLWen<$>W9qL z2f3!;2)!n}S;7@EPS>}Xj~=wzpqfvi=3k@cTuL?Hc^{3=2o2G%`a@oSTYL^8DD@CZ zneQ_*yBZ1iAz{z^$>V(!&VA-ZH-rxmcTMxHc=CnYDt@T?}WVl-t zu_qT%QRI3mVo6}FMO3o0_+lk5Xc=AcacD7e7+JLOdNn?g=7$ z(?Wfe$T;&3f2IwuQmEK!Q;po2g;L{^N{vdCx+>nYA$4f+O3JW42yJVHhhsB~k_g5a z%Fp{UFG+6($HN8r9aB)bF2_XINWS? zi$YzMH&<|>Z*sHA84V(1yb|djCs~~SH%fPJrv5xtqc^r*4;Y%5V~fL>kHG?N4b5ZI zGHez=y1Oe$dWoM!Y_z^ziF`AQ@6|`=v`nFfdbyu7Qweuow)i?~ch`gz@P3q=z6DKx zkD6|&wpmdbPbtQfFNpbFcny8@>(q}H@kg`p(M|MG>o@tF>%!cIOq-+_w_ebSjs$a6 zqay7Mu<@qvfRmMl?M;fb*IVyPr>}Xr163acui%`$s`S1pZ4z-B5zx{>-}vAfKE)=k zH+Z~trW)>$7GDMbg5z^+?gQ~&2Dz7W0q|d6=COs`%N~-X&x`B= z*sv%T%vGcsMc)b^34hz?BzA8Ca~UlYyZJZkd436yA>lOVH#a9YAs_~J^rlB4h+*4! zeQO+C@WK3)4;JtT;P1^We89d+_3}o5OBytKnlbkRyPP$?okkS?XYDyQ?3|+0KuP}B zFAzr>S|2F-JUG_x{*=h!0w_CHOa17h7 zV6T?lrnbMU7JM;DekoWWDY30eP3J@eLnvqs7{NkGEjXyO?^FuvwCq!`e=Yb-4R1^f zc#ekt0B?=gC-?#jaqYNqFQi5F(#kJ zsYUy4C^gYu`);-BMDVLhtWGUD1Q7Spta!&J&x+2o0^Zr&=O^Uir4qw>DS^8d=F2EC z%mVX$7MM5gcCvM%rkC^#?&%4#4zSDRXu-1(W&Xp{6TQGy zt?5RY0Rpf{fb4_&&%{NQ?RLi7o>SLD{|ZdYxM-`QKP8M?PU2owM&hb6KW64M>hFvN5F8=2ETC^S3re!6suNlSuSbZW+A{k?0jbKqz23-7hhq5p zrD>j~ML;||OM3$9G{{ZgRdjZbOG4f<#c=e4htl?UryJT=Vw&f~LVWvlY0p`Lx3AXW z0pmL{c1q;gxX?o-AMB+SoCulez6`L)i?$>r9J%scVoS^)?*x?)j;JI4W%bw#Pq9&@ zG0P>n54bn;=7xjTYV(#!B6RIaj`jSUh**0xptoy9r-+njkr(F+gVO4tbGmW!CnGcHc#WG~ zYB<`d7PYBG#7TT)4_<{ET9G-`34i!`nZKdg#PMj2<*<#>-F-JD1A${Qjy?>kR z8qyy|M+gnkpRJt+#eEnJvV0izhkxe&3Fiebw5B1R!sg)hNp%yosJ47OBwR}LrPsdn z%pMDV$+rh9R>4fyT1&#E=5R^4;lTKXhDX?~@pW3k=!aun`YyfWqS~(Dx4UV&*uY9c z_XhOEk>&uHD9j$)?X$wzJN-BHU7NaU#Q9i^u;22H zKy;Uimu}%hzOknp3Qr|_wWJzvsZXFljs{gFB1#L3Zup{ys^dnco9DcdGKaOV30D{| zq&je21s5%YtNlO5udCW`R6qC=ufDB1@B}fP0ez#U*PE}56IkU*<`B3Mwwf2QrXx?G zZyX+(vo<&wgMWr1M(D?!;a{eV;~p$1w@Qf^&2}jsOP+%%TNdfB18xvUxoHfhkv}A4 zZ-pIkMw;1J5Jjr&VQGG(#Y(cF_jz!5Ci_12MSkQUL^@H@JW)Bk zf}!!L;S6UCltgBV`RBz_{y9hN5{bZ?#KQD{ju!fc?;%)0qzWHQGiH@aKoT4q{;4vq zYF|J%P3Z!Veef>$38?h(Q3}Hr{OfyxAQrhdw+xZskLd%Nqyl%Bs942F^dM;N5X#D6 zO~ePODE8N&v(Mg~0p+7PnmWb@434^lrdh5U4F3wtH#VlO5dmJp$xLsDGXIfsWcNsE z%zZj($y~k?i4aK%As^ppAQIsVXB9laz>|JS0dj$`M&poWbX0iCq1-5+z5*L>h#$xl zQ`uryZ#8M2Gr|6RA1g1bcvj9TC03()wn6`1zDf{bmbp#ySLXytJc1`La5-T~npMU;K^j7hl zA$t2EFFzj_MRBgj=gwd~L0*z0BqUq~?g%cKP9)VW+;kOfQ4P)2{X7NHeux@^jOOK8 zx4H7-7arMH(Cj?E@CaIg92bHCff73hS+~XV_`YMFysC$p<}zgi<%Ql1?IjCjD_){;0Hf~Ffe)~ZOpqH!EF$y z7u$@R>|%eCd}SjsOD$+rA=9)s1w0oP{YZ^%oo+ZMszqBZkLm8sRy;0!_hk8{n{NiQ z51)E1@Y>_m8=tBFm7Aw|I{nsTk6gT<(QN_LIXzC(eSv_~n}qh@!gu?DI<^fE02 zft#A$q(xo;4x`;jq_TAw4p2ms>^e2MKs$jlRO;iN1qxa;ly<)Jt6fMzXa#nFvvQng z?;XJbut|`%(lnJa-q~;dW3!X+-W)~>`R>fbndB#Lbm4_clNvJ98l64#0n0yGZ;PK5 zu|Z4vKi{|nX|CD~lp2}j@P^NH2Ho)MekO*{ew*px!cypzrr(gv96ruNNF(mIrIbB^ zfGM6ScmaWgUo7Sy+*G+67hTO)51os7aS3IcH~dj12V^SC2M)dr9wVC+y}s3jazCST zdVRP)v*t59F})cZpp>HByz@v^Z7`pDQZpjCCqEK$9eVOtpa^5G%5Ek4H25u0mI76LS)L5Gi-pdY3)oh`lCP@maZUt;36edA+f{Nvfrh zF~Pt4EP~R{`I;pyh@g-4s*yJYUK3!1EQkZO3ADEHv_NYgLi7Ke?G_ zFX|0<1w{QyjQO|x$hxLf3HN@#H9+GPX)H)73 z9eKrj7)=ogAhUAqgfyIXq5&3vINMG5ss#ATP57z=_{vTAss#ATT|A_Ousq?bHo{j= z3w4O{m7cn-e$@ZgQpy~WlDU*K2O{%I%FO*rPsswVWz0W}&VZ&8^4GW?m7S(HKf~L;&rA4PSqE$64jz3%7FUbji?hMT=Ehw}K$bBJv60o9{xM~NQI-y>ckk*d z)Aa#3?}ZP=yFgrbuPnFihD^a@`(fuW??PNyd{HoxADhEAI$2Ph#aTHMuTLKI{XA%F zl=@HaaKiJ-oK3eE{2w38d(VHQ~s0X_gt@e=Zh| zX4aVYOUA>wosc~dYQp0ueDLh0moL*I^RfbWmuDf8Or^NZzfIN8`y&75KrlBod{*)@ zSqaEfYt?XvV_~*?2g$k;OF!`Ua9%O)IIrsa)S8wwHCE%(w}n5lE8%m_g@fnH$D`0? z9^VXJv+7~0rscOb>r7lD;%E4V#kvY1=5XG=xpHe0J^1#y=&6RDw0-&XfA+p&`v2X1 zWt4iD{*KVwtbH@-X=mb#yXa{ne=(n)wumrtQAg-`?9>~mwS6w7{ZQ8gQPG)gRI-*H zT;hvpJ~SuC&@|$E%%F$Hxr+Q~qv~l2UfJDVvX5gsAp^~`W5MTGkP(_bXB8B%`f;;u zFVWq`aC4^ae;a<`;)`}Y(XUuj>y5O~L$2+qI{r6+$aSnSI&S`DZ+uxJfhMAN6>me! zy((BlQbn5Lsb4sjYS~@bQ-L=ZwKQ9ri?NhdZ@0mlTx_fL<$`)e-vjzRjEH$iRvH>Q z0q%sxr1sW6Rh#AN2bkiWfoHd(UAUXwsdzdTg0|k%OM?%i!FfX2U9ISIn#7C~5xjl4 zMFekR;MznITGb(fbI`#6WFsf001FMCJH3u}eUc-fc-M!GSC9BY{m^6<6#udEF z@8qAHN3Mudq)Bf$nM7UF6NLujDD=eDJ&EHApiuor5XVmeEkmgTteZ~u0b-FlHTHJu z%xY}!%x11S$vObreK^(hd?r|tc8stgDxhL792{#>l;hJCHF}qg%9Pms-`{`*NY)9T z$zSkQm83A@K(rzR`O~Dc(~TgF)N@h}v+-!rK2177+)9qU>pmnE?@RoOOTq*3E3Py=5WnKG6Ti}zejP9j zRz2H8yIM|(HN@YJ9X+_(VqC=V#6UvvOQ;ISB z3X-5tc?lbh=BtyVUBFXgny3lr9mhbe2@Ex+js6cAb4p%HnMAI~G>eb&*>;^Cwb8}& z_|_@JkfE|monwSVU6&SGL!1DdhX-%88WQEXrD6#!0UH+gwanlt4Yd@LD7-l|JmPHh{R4yf>YY@Prl2#q5cn2RR;&N@z3!Sl%$U{zwN;>{7fMt*&}6&K6zHa zc=UEu6dsgCq^@2;E6@KDrx_%9C@!c|Z-eBqotVlQRM+%fsOuokBUk-0gZp68r?AN= z=Dr*gYW4M9Jg5oZr%$ta zQ0Z6Dpyt#i+lSG#{K_Vb0*$3JO-agPM6=&jY#q*lYCiQ=SI=ZmD$9Xv|&!yCA-BVrI@eAERfSYnbPLz|Rn8&ExbO zh`fv#u08rxm!D{>EYYWz%dDc)lZ&qIDmtR8s3MAnZfI4}pN09ENfm_N6uq1yQkC9% z;LYy^1Ye$M6-|M{f%!e4&>Md9c2>X$w|E4Nx1zyMhGizz5{JUwl-lisU?F5A+UGYP zGRN#Eq!_F7MpoV~B?^$xwBWG$%rKs2u+io!JR3`IHyiqAv5{y3`ddPj)wRFF4KvOD z7cAvXPkR{f_j6i)yY-xP9~}}PNdQxE6$CT@&n|e9q}`3@!HO=i31z%~t9*V^`QcQa z%B0j+oRnXhR9{)Q`nv2TG(r``dMJ9Emfa>-9%JmYhu!Q`kFuMDJ7$<2Rh@<-;ovwd zmD^e>7nbUKLo=BPvz9A64#!Tc!wf4^)n^xG!5(0O8kt?_f-3B5!Wd2l;=SG~CJf@T z;|6g~Amct(UO<1O(B*wSewxn}4enmi!K*G_QSDXnzG#JmeXwcoW_`f&b2s~}$hY&M z?bq9s?7ed3V>~~*`7#czKz37bv8sQ8=`Ti4u1~V>n{`#(zAw=Ww&lB-(gXUud=bV7 zvpGtQe0zBJ&j(!O^9w`p`QqU|-17YL{?qyvC#~=2$?IFn>wB1Yh_A3r$!`1?R=Z-z zC06^cAziCooLKGB?u+CZa`8pFYDm{=mv&q2vhEeNU3u|}c3s)E+GRag`=a-~2m49B zfs2S;;)X-sCMz&oWtbP5ySBvTI-TU@8wwSJW7BK;5zCVCRtU*4wjT)I$X)^v zCT1C{Hb)NX(h~cbrx*x)v+U{&TWgwL$81IKCWbxl7qD^4(kw8vGt_W}%NCr<5u7&i zDB2MdOO5z#@j=WXF(cNbM-|WBMR!wI8V07=5U%9j^obv2{V_Fn73acK)ZA5&VOavE z*ucNOIWaA0V)3)Mas(Q`4Zf8__b(AB_lEWL(4Kyn@I%}o7rnA29QBM7Aua%Gr5-3RD; z7*M+?;&+8R9Ku2*++i1C!sgIsDU^^P$yHt!lLPBn{6gqpbJkuKx}I18b$^QuhlHnF z4lwhb)9UEO+!(4;^C$Tb{bDVjI#}~Mc0eyi&R1c&@5u*8JBL7h6vWYWC?A=eADKQp zwppJ(JmRR1EG*Vf6xGO&4IsWga_eAyPti90;lU8`?35DQ5gRWq-KNkbFdUXZ$s^uQ&(mbaZ zLZ#T*KV>{!_FFB7>^xvKFQvc}TmXgxEpJePMFmu3Luwse_FXN%vWw0FN#zPb7+n2b z>*6;$uxfBOVy@hF4z6bb|2NahHwGg2WbtM_X5R2C*gr3Uanx(6EiL4hZpQ3?G?WAd z4vv#pfg7pumWOD414aeQAJ_CD`6Kddz9i(DS@cQ}#c)&gswM0ULz(dFC}RsQMM{8D zTApEf=GH7F^5imJfLpUTejBe$5EQ*HEniJ1JzNvpQnfksY3s<|%5#y>`@Ln+N9*;; z#SN2rf3<|t#BC`3pgHb8&tqq(dR4sZuUyvE^~zHCiiniE0udvhPb<{HTI8Pf!WS)~ z2bpIl!?+sLBUk1ieq1E4P`w)w6lxCJRedvDOQ@c=XzQ(!!#4^;ra;kNoCfSy!yQ@j zif0mv$PUF=&;e?TV0y8rh&+s7TZ5A|k)LXsSX|o?r8!NO_d92}qz+j{{ji--pl} z&oYa!IXF{=h1m@>i>f^3E%wyAPceM5G~IAsrg$3VFucl4GaSR<%_$6LE$+>#F*&9b zH7ZgE&WtXr9?a^OM$NN%!Dku?z(rew+2#{}hV|VXxUX2N>4SvEtNCRQQ<~?@+jb9&->?~y*J%? znYNboXOh%3KWr9jpU!I^O@z;5pR&ds#nX&g!`dthta_UAVwgI}^U_~_B?pS&=}?;7 z%6G{dp>8R`QxnuDB~aB&$G`sZT&<+JE4ztS6gF5)1+MhAOgkHku{}U=PwVe3 zeodSO6bmg@vydh}aS;X59fc-4m?3m~DSkzSYgFJwzPt&Z(U`kS959ryuG*__Qa$f4 z8V_A_Zn8#0(EDvzuCyyMY%Ogp4?R+tWuB^uQ*!a~;yRG=K)+1B7(_KsW-@6H-pv#d z(5r?oTqdvlBQu!?si{~_Pis@Y)A01=)KrBOFE=^2r;-KE+st_=2m923=L+M`P?WqT z=OR|eucG8{=zT7+qmt54djSNo=%gln9L!aWhppQe^?wAFf}N`-ne`4kivoq5vKORD zI<80!t*pn%Wm44%+E)2Tr_DNvfMX0SLjmF8CU&#f5+iQ+ASD&TF^;AWXQbFVxT}D9 z@M@l#s%Ypv`1o%(UvOCp=ib@PUhWqq@yG4LbD>+sRdZ_h+sq(qk&${0B{59TwRq#k zIB(nm$6v9gIA{DI`*9SU@wX8a&e@$a2E{Pv4q}sYFy7yKvy1VzS_%EK*H+A?^RbxK zQ0VF28S7PItCapv_TM`J=s4dJo+{yoGnTslV%UEhRnL2iZl)1_vro_c*NHK%7)XFq z%lmI>mKl7bYy08b#Ty9|Hxhqu55jYcxiL4s5#>tw=n8l~3C&lz-Gq-fL_y*LFqMim z`63lAyumiv8;%i!`=)rZ%av&oKhZ{(*>Zvd0diDUSURwI+>={lF})7R3`JW0E3q3Z zFH0WLd>#=Z$e8N}kc>c$auZdDKv@vu1ZT~c_1y7rzhPDkU_*fx^w=!Ux7n;tS50TL z!h`MuqB7Wma`XU1a`eOSsArGCu@cWZZI1~2iN1$!!(FTF7$w5N6D8Oiu?cfh|`hS>p5^skuU&9uF9*>IZSUqkB}aA8hC*P9uOr z)pgck%KR}xrXB+W&@$|pzg}j^(>E~l50~1j|K~FGeY=EAUHmR&>P>jOmLAz##0NHU zLdthT*&!{?PIbRPRRe&kE=Q-}fds{Fk*w=ra2sHLRs+E-#yS1j^U<6pTz(Z^d(9^d zzMNXUMPQ99Gg78c2x#z)YfkAa#A7WI5*%0P(Uu*;n(0`2%0~h{O~YCDO5({Nd{+u7 z-T#G?3-*xGTkaGEo=L4q6u1-3H@Gso#NbL8awV8uZ3i*C<3j7@stmL_m`^1iTo*8Y za69;^PtQZ${205D(!qj1$l8*D7cvp2Z7sVI4&BRYiz6r+enSY4H*k_n#adPBn$%m@ z?n8sZBp*J0o6MQ3EKyIihZ^2FpDCP5y@3c8k#CZm_meI;k6n%k*9f#OB;tA=Q4UB z@%8E3Os5?>p-#Mh($4~VZ4gA{E+{Rl+Y{%kq1BlvYCe3nF4&2w0lC%?_2 z>tSjwS5ggUKNelDkT@L8bu7B(NJ4b=6+(8^q_z~#^o7$TwR z;4qR$Lem2JJvJ}Ts0BCk5Ye)Zou*W*j|YS%F}B&9Q723#*#E2EV3nVU@;?(cV9!e_ zJ4=xL$Fb>mIOF1NRW3gG4hzjkdJ&k#lvI8y&`5b|jY4341T`+}s*#9tjKy&FFnB51 zn9poTHlA)ga$Rq-@vr{Wt86?bA^eW%MK&JOB^!_FT{a%sQ#L*Z+4zP(#bskJ3CA~C z!ZAH-Hyu_yTjZ*INW!IJDyG>|rb4h{PhcuSeG*ggcaoZ0($B|5=MAk@y`8U_lmF|Y z_H$sz>C3DL4Wn6tXQ$NrPT=@4xwv~We#AduzxOhPl2hKxcgB%;f49OLzv$JSBn)A93F zzDwVFojcl!Te8tkXA<}N6Q1eMq2|9|SezTqsAk-t;hWbcUoIb_u9>q#vhsCc zv??0S4+jz<_LY((-(U7ek1b7<0{FjdkJ!%p}*hJ-=FBOXS{d) z9U9X5K%@Z2YWeJZC(x)@?DFcnA-WM`a|(j`UK&^+a*wU`3N=!Zg=>n-*MYs-3*-54 zL#DUpxLs-RM+GnD$IX756-o^i569$O8L8N%>GeJid)lbczyr_%ki^*feZPJ}F>rf7 zv{o_7n<9Z-=6o0uyFZ{0C;Id=8m21}SZkdQvAC`@I0^+gz>C;%IK_=MQMw#q=l$z- zNt|0EglsUIIf(DsGHv2CGWj5=XFcYmPg!tyk*Lb0{2OQx7$Z|1nx`>zElbwtgnv;; z*MVq~Xnli_A4bul9XZ!?(XId<`D!K!)ipNZio_iDwqYv5N(c8R8xT&(Rbbn_bX#hCa)2FbSLi{V?@sD)r|Z=XwHJ0s8Z728kr2 z+y`2RU#b>!;3qIJhGP=ex@E?C**F)8_v_WqtC5FTe)&p3-=>BiA`vE^eTnU6yH@`? zwQCNqXZ1wu2!8e)XS>0PZ5MSpo501Q)aplp_71rhd9l>Hak$}?q=3l_Xo4feJndB}C^X^-*NF5Gg z7q*Hz7p~Z43yy$#!CX$SY!nte({ptv;lno_);g75(mzZJyhpqaI4ms`-ro2UPkeQJ zdVsdKKLye|I(p9QJ^zQjH-V3;$Qs4Fvp@$DZUX_r7&JIR6PGbj&_qDf&`s_o-4JCF zl_-)?P|;zaTNqij)7aO`)r>3ijSEk42Dfi;1{58^4#6yd5Kw^tvbf-FS|KcwK$d>z zoT_`fI}7@~`M&S{|L?sYKhjmVs!naEPMuSA>Kv?lG}O)g$pWSidnbbF)p+Ga{b^y? zevbtBt+>dbfsTeu*eB=K=d|w|Xdv%o7XZc!p7GQSIUwa%dv0g*uJ~>w`JQd= zWI@EP2YMaVK*-k282~He3eOgZX2JJm9J@jl$^4J#U65rGp5K(W(~Xyo&om}SGbYQZ z1urA|{t1hD7g$U}*&G(MLU0BPrg6SgD4Q3@#o~wKblJR5bGOJRn1O#yE{{C!f_1o* zzGOSfA-s}@*XZjLveApJYI+XAzcjSb7gOYmmF3BZe92*73iuaMnVW&~dCJbUm8cBFCUOmfF#q}OmjZ!)oCq{lWR=HuF6s;O$qjBmR*-OzfN zXqH|Dps}m<-|^V$9Z_Wy<85L?9w+#G$iln1tp)9YCN|KJ@H8eYW(h#An~Z^lv{y0B z&mIcVxL@033?wA~4U<=}WE$tbO~$}M@+X+QnkCb?o@q1&3X*qV@_Lp`W=CEARSnu$CE7)s5 zs7E7NpPh<9lQNprT11lpZHa8%31wv{x;_Peh^4y)NOGEY2`uvd$UyW4c%EdeAn{RB zlb8-*ABhF~EP?$T!3J6luva748(P@GT5N%NRA4);|44xr6nMMQa7@TSpQq1*ytyC` zXmUkP1*{MKO96h#tDz45a54t)0s_1b0bd^h{5k@Dx0%@qeS`*EUG#;hy2|;ZC>Utu zNnn3?m`_C>z6nQj`I?vLIIhr8&PIQ^9Yu%fV)H!H)Z44I?I%SYU%&>l?0zpdU_)eN z*F82gv5v-3BxQi8+$ee*&E9iy3+ITcEiB@zwqA7l5t>AU9lFB~-)-K*@lG_FGuwQh zC_KD_j@H8X$l|;OPr@jah9_O@)qYd^V=hrxyu$&FO%{8OI3 zZOjG}9zR!jh$hNMkXV1y#{Ba|^K0H8rZF4mCzj}s*!ObQtp{V;Zss>1{rPKar*ZSK z_9W-7iV5R3|7+Zp{+h^Os3hesD=v<&RwB20VkEkvp7%SKvKZFJ5{qf^Z$YLyYcjj1 zB5SyKGBC{;!}litU>n7At!9%Z9L0?y#qtT;T9|Zv9uFdYd+1<&kTPivr2xsjb}J5- zyN7qtO2DaR6K(zHeu(VSv?$hJk5;{S#`ED&XYt8(w7b0OlK=$#QtcCut&^G zgPOAThiLYifk^`}q2k>AF#5k=tUsvU>z=PEPteo0%MVbQj_kwM#zah?_CtI2;Zkha zyO^pZUK|B-Na$_i8?J3&rckc{$Xwsdo}fwI#c43#NG>3l0l_2WkC`DPd*c>$0ok1{AVY!#sHoli ziXlrm-jS=dSC2D+Zw8WJL^jp&&L=B@Sbv0zK$I(RR^` zWbTigfb`I}td|*gb#M85zw;6-q(5HL-*ybE_L5k)mwIOE`O`&;^WlwAtCOhD9 zv=#?h7)_7%eG5gd+8eXwN7jC5E9(En0%WOwM}Y60hC1VMm*4Nm{nn-w5^wN9U*rvJ za^X-Mg{#*!lVOx)w#Qmb0K$`k-FopaFyA=(l6uz^{BZOTU6bOg9J_p^D#1dxYSzQu zO+kq!2^|?|1p|o%%XcUbPo%4R&qU~u_prs|gob|geN00ir-p9BhFYMZ!Q0sJ?+3BD z7=l({uCGr)u3WU7aQ0J+XRLk|E0hfxKrR@k;PYo~#36jS^a{Q!f^OZ!8=bpL-nn#y z#pE6i#3X;;oRfC>p>62Y-mxZhJNwPu&(%Hz-*G%{UKo#k)pOR#({%n^Q&*}KBm z+=sOJ2eA(u5k$CJtoS}&n|zEGhf_Z8zTT$1Foj+2p&LwK3M;;jM|0E4`>bWv+QG+@ zP2?8Mv7Uno^`XP1(^|L!v^*q>}9@>|#qWNq5u9NvA>EzOri9mT^RjuKu=<^_1xx6=;k*Z3F zH}eX}i>|T}JEJ{CwK&#GG7UR?riWqk;IkyZf?->DnK9H$zrv=3?8-yi@G4zmSLtZA z5Jtl#H)-QnF}7u;6=AX2E_;>$PtHOSrwnJ1P**5hz=Am@c0CC}kcPEVlfu!m{6Rd_Ygzs$_QM*SmxhGZ1Zc(Pkjp4o91TXlvkRAd=hIg>^N0lOrn-``trC^A1U! zc-|^D?uRV5>gFH%u&kRh#BgON<{!4W@+;he(_A6-Ox=Rkp7QI_jSQp~XU)@t zxYn{ib;t31A&-W2oW*5KWF0WFew@i64kp0w?q$qZMgmix@9}p|sO#fd@hG(leCbR# zR*rIn@=0n7xq~LMj=m#HMZB}}J4=f#j{KdA28xQ+CD*yKYtRhnyg?`~Z4aW)=-jX) ze~Yku2<=t-gCprkaleyn2DOOlxE8dmRiVK&2IOio=5bD+M%|bPpT@DObIWX9eZHMF z533EXDZjzf+XE2eLWX%e|HdUXDgTt`3=WdIpI9~+$QoC69Sx+#Den%RHARmZvWeAQ4IEk* zs0AJs25mv`&uq|kbr>|r@n1b)pZ~88ST`J*&_JgdeZ))c2I?dZ)J})s-01+K4)`ZL zMc%6vKt`)nddlWKA5Yd%4-t3LAOfl2<29uP346r}7_VxZ+{CmJ_PZbAW790D3)3QF zlUh&1V5MO|i!iguBFxZzkmD4z7nKku?!(c>5!s0&vJb5(pw>zij^jQwsPGvd8VlJ} zS{>@f;G=TDWmw5RtYjZnk_n@D;%h#n6YF)rTRM)BnBRay;_eH(>L&5J^a?TQ5Lr}h zSokB3h*Ds+T~}ZgSBObXFoJXxviwjdTg^G#YCbrYXo}uwk{e;8+1LZpk#)G#*cedC z2AU$bAu+gg5F;tx9+)(#PAna3;i_T7#SFpsRWDSA>Jy`LB`iZ-z$ksIb!Lv#|H5mU z|0C3q{zCH0ey)wQo)8zf8{jLZl19>XF&atSsEG_C{_LO zm#7F;n3#$X@8#GoG=+Y@?7u`)D6b1mp}1vSQ%F>3Fv-A4EHS2>WFI7UXakuY;Hme2$dA(K=AXxh2VDVz#ETOl~xN z_)>jxv{hfW3NOR6i|TI^O}c!0HeQ!+gRAJ3u4|>J+R~sFGDnULE-lDF?9HzbmJP)N zO;rSxtp${=6_w}7)<1IS6@*`%{AqPY^-b)9YLj9u@HZPD5F z0I=(uXe+KEc3trOzKv;eA}@}vw^3u^jZ4|)`G}rI9=!oKcH-7r-H)YCp+|YH0BJX) zT9Riy>59m%)tId~pJmH!V-tllFZkkEY$Z(_6xOFMv;EXfS{*fS7nZF4Z<(s9mYAtp ziBokxgN^dmmguQk$)_qkm8HJRQtS8}Efp1YCSg84MOmr6aRbY?OwUG6q#~KLGrQRZ z?7b~Wc3vpfz8+Lf zVTMf^pB1jP$>Z^U$EM8V7d)hIsoX|4JT`gB&*`U2%5D1nizMIn^ZMzX<>&h8oh6}u zIuNKFTXi8FY&v}3B4-lW9j-WuchEmZv#`<+7wq869XcqKp+lIiI++!Exa&?%h`PV} zujnuCWouW`!K2{whJdIhgbXyIbqr|wO%p?p%=0WsDYV#zJ1l4pT5ui`s#qU>3zeaSYFvZLo1av{>mqJ<2PqADk8RiQRxZ@l^xm3(}T85Fb6 zhI+;7i;2q6ePZ9CdqsKZA++f#hHE_`ao}0+QL}ycL+cmaM>`5iYK}T!c-ZTkP3Y@V z%dT|En$%b)nSn9;Cf;oaK4Pqr5X=oDB{FX+5T23y}L|UcY*MJqbR3VBF7`;2Rs|_1KQ-J1Gk}@_*QLKre#Q_2WS*wsPpR| z=_|>#Qg*$Bn$vcmv!jf~eNdY^jc(2EW!dLjKn4}cev26@faK!G_&l)!_&vz`%&cTn zkW5wSM7p>Iodi`Jk3&0wjYOo+C(pf#T^|`nQ1Pmt-akf;wS7rVfJWz=@yOb~Z#K8z zehP^9fN-QVlH&%0csCftyMc&z0}=0r7~w_OS(5w{xu{-3R!&A%bvN0bd?|CbnUT?lFix^0-8PTnXQ0;` zpKWK$C^IAuNxdqAlab_O7 zZy^WkkeERh6W_zE-i4XCWxs)EsLDH;C|S6CKY9kC5g6_EB!vDcR*V@CjN;>MHWr##-e1oQt-9K z={{83>be=PxR;k}@$f3twOQ9>p_&P4GvVlA*n~ zh#x$^heps#>YSw^v2=>Xo*#OoKgz;#y(>FN%xO_>Z7`mmTUL`>=}K+Loov6l1izLp zut?dhqP!RS7{@7Re4bSCT|Ccy`wYC`*&7@f*N%3gLkpmt72JUI`DDS1mYv$s&yje) zRl-ERIyMkL?cleseW+Pod7kC|VQ~Op6x=Wh3ZGOwSa$`qtJkt!yAVfQMw?+kk|Go1 zn?Wu(9CbFOBm<{BM_aIvj~d>$$R*og)SNJCj{K0YY&Bvzo*e5VCwS}0HCWJ^fJbE$ zkIm-iDBDHVc|$u&f6AcrrwmGeiYWalqV%U?DE+AzN`K0r^rs?}z9MoEd|XnUI;Fow zr}Vv9)`~7DeJxV@i(KYUqxFZ1dL|M$~yDBp)aK;FpPHmM%_| z>pj4_HTO1Kn=o++lEn09T4vKSaxfD@iVaQX>C^Smw#am0GlVh;&)DX*MV%{jx|ZkO+io3i z!?yoGdyL?7Fz*dQ*>$|>xIPQ8d2D?)14Mjsq}^yVj+qd`1qw)lv@!T?D?W(*|KYL{ zU|I2{)_guRFJy(q+?-6CvBe%&v*}UH^waeO0O%-3N2zIMD-W`vy!WE*d>Mr=g zRieBO4?{jQbwexRvTH4R=~0)OfDvaWo@fyds8p=Hsr~s=q;y)Qt})Wslk_QVkG0|( zge8~6GMP1L7F7CJUwjVAr`i!3llfhBj)JHd50yV@UaX+|UpVBT9OZ%MgW zI@*F|@+%e&L0*U!{BZn|T_LJNKf~^7ua)2<*+~HgoSp;8SS((AEE%8lSs`I*{#eVB zTSdS&h0sUq8CBai-I2VEvQWN?g@cWnZj7pBM5LCbB6GqiW05$~n_Oc?h8b@h@G2R@ zc;(Q~%)+QeQN+Wj`1qJP!rFgejxgarnj=j3uMq*rrZAp-04(& zcu|M&`Y(cD8weAmnfq z@qJ~?#nU>3qF)3dF9xxEe{E$X(TBQn!95A>(L`cs@<09^5MkYxM3Ynv(@C4e+g;`L zU|yN$B@z`}(p6pw^Oo?u)bikjuJR7TytzCtk-*^PUFALbF62$*d5LTWn>#E4p7&GC zTVKoa5~&UD=qm4@Fz>rOFEcy(dOYgd0nJftR(Z5_BlZ|~MeQ$6woV_XWeJ>(GW??e zff;tuqbVlsnLCm}2)1B;T|$P{UHtpz@JQU=1mD9+P%~^%utx|_qdfxeqO~T$p9=6I zwG{AL(j&EqrAT1glu2u~Ye%()G1~6@RjvUxVR2t}Oxs&T2Pxi#o0?4S@9>jsvyfFW z;OlgB!kXTn-5kz$_}#_9GZt!CT(ZfuFo!Wrm|=*EHRX?S#=V3>?#8v@e`YE3c#8G_ zgC+RhBwFvNK>@@{^3;c{LXY1`N6aPVAmbO?Zqsi2%uIIce0U*RBa~gqUZc7P@rlBn z1`@)>7ClE&923#9IB+KjYaA|z86O365|r3O$0NJk0|_SY8HdS86|Np8E#Ymp9W&po zj7DL}Q$XsV%#S!zN|1dEvG*^J#@&VzNNU=EUzyd4_97Q) zW8|GiU(_rFi|b6pwxnb9W9Y7X1;HII!vs0FI3dRi_1`w{%S;n>tFmgJ;NbU;8>5mwU9DtEyY{vj2 zuLF#lkGg>IHx45YVB{IwG{7k60Hdrc7!Pt7=!~fVVHB`U>7V4Oro@8YY52*$>?PB3 z)hx=hkrSKic{X-b&t}4$qduvpJQ2=&o;K`~P8A{r|8V-2&t> zczGONUIbo2XYl5C1@CSSkEuL)3pl)j2t2Vfcv)S+6F5AkH02dJJTU^VxHEWr*LQ{g z>lwUa2CtaIE9UTQPqMq3O!Jcp{w_VzW~uNi?5^i~?Uv`mVY&(nen+pDM>;;Ci=?u( zbdltO*hhf}*a%xMTH0CiT31Ke zi!lpHuBjjB{xvKe?5m2^iS=cD*vhC*?C&#z@sVIdB*Ryc;Fd`6gGg{yB>2Zja4`>7 zCC-JAx&8eoKcDiGmR#aUyWw_PR5z?>35PdMK^>WET*=&2$x<=~FerImAJf^<-Qxrw zTB!#t>G?bKIrn_pc_^|J$KZ;~;z)gv(v`%}XoPG*Cv|^MHf>CYuXMYv9dOYoI&x7b z5g(BHR*;wNfV&@W0y>MVrmwVG@V@mKavZBIyS|M!$#L6(=T{P!jE-_4+?K>$2xHe` z?jT0q{cyXs-jzyi^-YAzuvebnKrV@Lty@ZqM8(WbmAd~Cl13Z35!W)Wh%<$*g4`GV zlHC1!#Nk+7LwQqHyqByOp-fs~Z>DjH76v1uO%1vhimiUAU9+FK9GmUF7P`KCls>$- z1JvM4%w6Ck?Mo!~#S)(F1QT$02V7)=UO!Kt5syYSfXh+|Bs;KeUFm*H$AxKPdjuKFq=?YH%+d z8TvyXlU4A&LHQF8{sEl*4bO95L??7T(Lq ztf0vir>f$J+SUi4-~;-<0^8)`@PsTNL`FukOWWE{nQ4@>~gxQXpXvL4(Q#?VN zS<*@ty-8CbkzjuL0M&l|-`cH=t9jKmnC+jVM!W2e3DRUJ;C>K&&#s%_7$$XzVT)T4 z@9(3p8=4xKS(QuJ*D3V%jrmRZ^{AGFU*(t6sCDnAvAu92HQ;5VW-`Ej5_o)*M>WuDQ{5rSZBp!cwgEId%I^)nrbG2S-GLLnFbfBEif^Fg+4XjRcb;!35o? zR951w+fsDZM(ia~#ZDz+dFYnX7&0G3Hn`iUnMSr6b&jfWkRZ=}sf*=ugbD_FI0+@(&}+yO^bq!8fiN9mO|BxJJQOg~vO);g9y>D7+3zoNa)3 z5c$g5IB~o+2aD63jFJckOxyAH;Y4=`_-qzg$Gqtb1 z@EeSJdg2^Ed1V{3xxe3iRg^gM`dhj<(~FBUs6i)$Mj|Cx28mcrj!ki3i4;j$WYT88 zg>t3?%Lr06I$pp+-$Eg>ZGQ_D`WAi5+B30KFG(4ZsohAW@Qc}#B~>S6vSMLR7K~== z4f<$Gt6?;eo|DlyWY*0DFYMmSM8QpO>ZU^ze}O0{$%Sc#IC#TQe)^smxlbT@5D|&X zgV2P!(iAr8PWim{>^593o2kadiFgK>o{2-Vbqd*B@g8m(iOWy$-9sD_j43|)*(6pR z%kw{9Kw1JA$?-pLGa_03=SLZl4FB_^jYyjRdAkv@`kzNN1OFRi@UO2B6BL_Olq>-*>AXPWJmI=;5Az-$Xs!!|$7CvKsZ9UQfZNwccGxHuYQl-o(@vGX&6==_#tTvd5 zOZ>|{(ahTVd!pI1f4j$E-6DoU#r497t8>OZXu1)&&NIXex4_>C*AKom(l+AN@K{&- z%WbSxBk11RX9dQc(WsOI(=ODThi_-)5?F7pVZAAQyXa*adnS8?Ocx2UCvI#=FbNbn9y|8+_$(RFIEu!4E$|3tE1pVt+P@ z6=Sj9xL;Ry;h(8kw;wH(?KSBZqqrX66s~gR|KvF;$p?^1sUufIsf{T{_r4`q zBUd8u{Z7SG7js{RTdWNLUW@?Sv;~k^3|C6|UwOWhl%biQ^gs{~Vl}uI-*h7aAzN#f z3;0Kxj4uIZlL2w^lA!c+AE5@<3+ELD=!;+mjq6=yCDh{1$Ruk|#)ufD1i!V%_n;5Q;i_)C*aCDG)i##Ds z9-XR;o=RK@4%b*UImelFQuLlH5FW3f;Tr>~D5le5}LXlrg@V;<$I z(sSJpxqvrTiqhmQxb;t%h`(0{F!$11YXt>{f z8D}y^c?=)qF?@{2@F5<8We8=$NrJEcjo@xV@)aDncLYsfQ**`tb z<}Zv_Mki1zd`i&M68+P=L0WfZv_(&|=xIs*={+E=r!qQOPfOO*QvB0zE}Vr|3w)`@?g#(>$o6Pva3`TDQ{8&(K(UGuRYw(3_b$<@hh4 zHy?((pf@vt-eg7S%?(3!dXvS!3>ixFX67%TH!zyYxS5RL97HE)>S6$W8VF7nCpc|L zaI$rR1G~Aqhomk{$kHylmX8b9+(-&{q&ZpIPq|2QGL)iBPIIPlnsbl;!Rh{@QciPz z%W2NN{?hyWqi00MyG?tq4QUP^=|Voxg?yX~`7jsqQ7+_zT*$|`kPk6oyizpFdoEol z!;Rj1E(5k-!G}(}IW6LJ2+7clxF{;56CtEQGrC2QAe{ms0h-YxivH;22dU4DUQxtH zr#(n|X7mBd1LT+|r1FG*{u%xKg#(n)X*?l~Ck*t@_?5r#B4uD$3)TAaXM`s$7t)ADB3zsr>)}{Z5r)5j;9s)hdzw~Yk}YWv>E=#`6oUdFNQ0W3A4ybMgGn?pGUtMUGgH)^+Nhd z(aLqYUU&_#Rk8dcr|-EfoW7R?FN0u&9syZfcau)w{&^E4a1aczUh?2533K6_xR# z7++X0w>ylvB##qa>V3tcybs2Hq-fqE$&*DqPqrx=sa3wPd$MiAQX;dhPV@KZ%{vtq zIa-GO9n65u{0im^{SHs*dU66h8563Pkug(S&*$otI_DfU?vR zy<728(c3h8PUglXoc>vy=0>gYs#v~k2s9@*%&t+g+gP5@ufkK>q5@GFE&7!mkmP@R zI~CbNXEM)zW<=8b&u%j!8UAOt8j&pjvs;Wvj{n(OjNqY-f-fTcUOYtfzg|UI@R-Kx zS3Wf&X?|t15y|i?l}04XuWT|RIew*rz-oDZZ@^@V=~>a0(6fLGU(Wp$Pe+Z4@&&Eg z9FX_pFi;@)_QkU!N`9i$LkB$8tk*X-PH)E8c)fLF6ZA%pMM~u#)g8vlq5~OgF`y(F zPpaJDq z29%2oDCq{2LH_9%`$t_uK74d283vTW29!$;D8DwKWExP0_@`gyA9XoE$ugj18Bnqf zC|4Lzt~8)rWk9*wKm9lUQP%*J90N*@0VUUfa;*X7Is?j31IjS}^y~ejZU88G29!Jl zO1=T*Mgz)l1Ih>k%1Hn8oBX404w65?|A#XFA4i$?5>)j5d3k(CdklHBr6?x6=-Vve zqu z#R`Di58;yHNxv<(_c7^Ub(mpKP1q9%d)9|NYr>w@+I?(La(n@Ik;~kO`bG>`2?DDr zGl0GO3xEw|z-ETJSv-{$K)eX>37c{{j96buFOtg_o7i61p2)^ zKmX149cDju%*U4Qy$SD4Swar`$zVTL_G4l{4XsQAq0K0?h9#_IKjrLaDf^ktex|XS z0{Y>yx`DpP+tCiFT%irv*Tx(_3T02yk@|)v((_t}S8LjvPc!cYzhg;hq%W`zoh@lk z@uZ35#`#@TqHwD3?q!8%(!JaX?S7u=8hWuZ$%;=4sd_d7if=E=C@Sj^0BYq$ee*zU zQbzlaS@1CjKdKuD2>DKW9?fBDj~<}A%qxRvld)VhSE9$L*4R9g4;UT(let5O*FVJ? zyujR{!!PrsY3%fy_flKNKTAYYqr0b2g;8yXRzSO5`Hhd0<_L|Volzdzh_~%{3FtXv48H%(YRUG*LwUJ%NoKrpTB>!Xk>E8Ia_Tlu1 zMex0ELKET*(4VG(;E&o%y;MDB$ar^KGhYDpI<5(k; zc_Y7o-;GEe&_tS_)H89EUn2B7R&lIbez9fA9qJr2?PoaTo;5H>+N@Mw4p4FqJ=H9o zZOK0k(|`}58@c5Jd1v-X{ed!;T9JVUZ1!A_KmKa^qVl2qKu^A)*qF8m^GQ^$}e8t?rRh zDH>WH9v1-?y|lpD12!RY&-&=Oh*LINw{SrlAB#+2-t z88*e4Nzbm}$4(Eg?3p=og+03h+afH($Er4bdc@veEzG0`SZFO*e}Kgvu;C*trgRDDQTY6l16HF#9V-TRL zySIkfV2-ikC!0LZ!s$_DefN)D-(%QY%{p|tjO(0T4v1=T8hKVb0=$ZL&1&`>?KC1_ z=>!Wt>SFnqQ6&ehYM zBeVU=`*Q%XY6SmJSV=`&2Ki^PvYtLlm_OX9cmk4gy;M5R0*hDM+pQHfvuw&3eD#Z8 z<)WxpZhd23xs8T?LiyF`Cjn|@+BwFf-+jENj=!>=CQ(lj+P(8kR3yyze-1T2H+t{dmZ(+304tU^gju*T$<9StQ(UQA%+eNf* zV1p0iOjKe`_qC(v7pzZzIb5TKtnHfadlao>_)VxvA21f7&-AnFugTQe`uo}x^)z!n zJ(yM>x52wN@7=M)M9 z5;}UkF?Sd2$FLZJeVIzn?GH!Lk=dG=1PQ@et@MVKUhYF?#+jgFgceb~qXpl2$5{Go z?qH7j(I7q=;g*kYwCFaz6!*PezaP^TvJ}z>|$3;PmtB>LsS0B^3l_O{U?1&}i zDsJZJQmCw++c=u6+c@ga%^`JR<7m5-PaTc&TDJ-9FrUQ{aCgHEClrS}`53EWv)ZmJ zw92pr#ZJLEG)uG@HFDeO!%Qef(`D#v&{awy(F3-a5I2cdsylH4pc?ian)Y1BGL z?kx}6 zh-z}0*jNwwVHvO^BEen>Wy3<*U(M)NbkjI>>XwYLs(VL9ZkSCXjgO?J-zX{0W=&L6 z&0-_$iWA%T00q3w*ngtA&gO=w3=Kwx7UAhao}no!10SJ)O-;VhTah5hmr|zWW<3u} z7Uk47^yD#0O&;Jq2h!uixHX}pow4fB?YI<;A)t*AXZ=GM@2CqTQJJ{C324TaCdk&t z6sv4%ae}Cf*e=NVc#0Ad0ZlDwNTV6v$!y)B8+jF1+248q zJ+_5?HG*$d4#;yLwfU_gr_o>;f^WB(TJ#f&gDyXE)b6ODl+V*{Nl`-a@9I6weyFtYz0_ zHwb0lBq5*lnWVxBDcuElx>GFOEvh5nr$MvzO*RE9l90@z#r8>7Q6_`zVrc~)J37@7 zAccBr9CDSd@t{NA>=0Z>?ee>5<3lP%W7_2aX%iYzl;oeB@_D)5E|>VV`J^OU3AD}X zlozd_+pTw*bH-z6KEok5XsC5;N7q@9$sT`iK$7=Kjr-6ab{UCd@?KZz0TCDvZ%e|Q zi-3J9AFC!`4Bgn_%s=h8n1W$eN5tUVZZ??G29Blg@U?1w#Me-Qqo)x0m@cQ5>Yg+5m>`;w`Yed2D+p1vEB zPs0CmJo|2*{Sak`EKAb&h~7;WSRQen$M6KhhOSvs_u${Dd(a*UHX#$eFU{Hs+uHt} z@E_6) zJoQu)tp8K=IsK&mHT@JUyErNTv}X%?_W)4UXoKCU+`H7JUbPvPNaKE3)ONwujD~d3 zgr@wposjt?AgEu3uPz*xk~pCT7?tlWC9kYf^cMoWw8~oY5m-6G_?kZ)|-E+n>!%47I&7$UAD8E z0KL$XV{ig7Cg{Y6(5tE)dWq`qqkKo$i zvir&rF+L~!`utzTug?j;KL5q|RR#EkK6`99xTgulIEHx$!**j!c6W_ogkXeU&itK1 zncu{&Y1z=Z2g5+G7nPw~orBLZM-d}-h=Z%eq()I5u~j5I+vT1tWp5d)rb1>9%*w>w z7BSuh!{yu#xW?o!W#LJ{w?@FXQxSYSiTHMsJ_DBN_;!-v+bPc$SMGMF^6pI9?prOM zh1^qx6>U3ew-0Cr{8+>e$CzI!Wzte@6nrjHM|=;c6Oc;3P)gP~ODmZcn05EB#c0byb{8i^^U*Q{E7~3K>;d_t*!gw423(l1t2qn^d zx|D>lvu{F^Q%(TpWC`OxT>}GS)N~ppS7~URnj9j(&4`Gu>|LRr(e^8q)Jo??sZ}mH zwZhJcg6R7O-f`_h}Mo7hbBpgC&&qM&jLe$n4sGh`%4OGY#SPiO%~8TT9SqBSP>&9vj3 zqaEiQ8rUOx&_pe`AU$ZJ+JPQ?jk9i7N&;>>^b^=Ib>TM_<}YJv9jxcmqT=2I>v_AF zbWoHrs0^(mN&(bq@K!PDG^}T&Q?Q=D@N6N9s^G=xB%ltdb)(gUyulT&>^j%r1}Ui~ zmk!u)EdidT#OSw(O2R3zZvqe<_*Ea#<(&WkbeaR)A&ojhvgaQXR6P3-OLsILlH?kx zvCd`IM36k)H-el88%zzhUsBR**wJR_p6XcmhwOI)MJEp6%a-O z*viK?BfX6}BFGKX+zE7v8GIDYL+v7=XCr!lINykI7Z&ez9^zw}yN9pm5X<|-J;d@J zgV;wXO>2FU=zgY>CC_BZ8YZLTJngiFl2@^0i6wu9$+*91U$A5!OTL07@4{q6cWoU@ zzKA> z-pjb;?b=^oYzYImA;?3hqJ+D}6+rqo+W>J&8;Sbw=?>JtJEU^wWwL8EIun=o?$pKr z?mA@^$vVhk7Fk`${GF|Fs#jGx^}jyj1k+%*y&A_LfCop9WX_Gfn%$6=GQ(lxCq6Fd{<3-?uZ-(Op4-;I8Ouw5lCa@&u~Yt z;Ss3%2VlRa)^}ixqWSOtjr>t4j#drl!FLix+=(1g%`f4QkO$vMJorw=BbCMSNP{G0 zXe~0|uwx>BIFB*1#kK^>P7{Jha-SF8maBOmg=K;vZll%CUm$fJp2TDlOU>!vh?+ z_Slcn?0JBhz+VayAh77?Ap@YPR`R34qT9`F2U^UIkx2U3+~T5PZKV>bbdWD zlE|+w3XCPW@N2~Hh+H_>R zyKW$u2Mu=gF8CPSfCt~el#q`*z+(X9xe*{=Puc?vqN8VXI2zipo}EC;fRCX$qgDIW zz|4aPTkVvu;_Nzy14Y)z^;+RMJo|{G8E_%(N)lQbWXz*OM?|NtGLzg+g?0$7r3qbD zB^-1BH@yolxU*|x_+gilyxZ9~xy~gg?-A8ob^aFQ2hUQwnsB|VbT0@shjP)Og9yB| z(C&UwQc|m>iza>wWGU4+5I!aj+y(qJS~fQdcDpZ0BYx|5rLx~Aj)-&!EO-qtLIvvi$QuoD=%7&hhUt; z-fV0rpYqi_r=4?Hww=69%(ZJhu$uKbu!eW$K+AO>;7-*&5Qr6?Y1HHfqQ(ywVl}Pj z>KJ~ziq>@%t?MdU*HxYI+f~GGQ?`lKw4SRk#BWy-zYT0%0Ql`HcD@5Rx!RqNYq^@5 zL2DVWDrQw&a4lD0%@tPX@JV%u+RT8K6Na=AManwbUtkgMaSlF;y&kKK*y$X60``}& z@(3W6Tg;-l&V3)ON_4nfJjs?zH6YZ+{=HOgz*?B*J+ReuHAt zPEm%Ss3ae94hAMX^`LX`F)<0YnAC$J$P{A!UgY18pf3~HY!cc=tI7Gc!Jj#=bOl_4 zkGPV8xo6oG5D2`0XC3_Or8bCtQ$ZYn-w6MX=;<^gDOzeq(N$3CfVs2+#8On@M6c2H zlJ{`Dd!*F3H|G8>y$bpFJ)-##`jiuXY5BNR=wrL8@YX{N{2N6tWW6_QE&7i5uXHxE>91#1U(g_%VJ^ z6Nj+aMi#rAoJJ+Chgj_Mg94NOep%`6ROVTgDXHFXZiW)R%lep<;sl`51;FEbo6Pg_ z8c|_CPAJ1y%1U8c#o>5SZbY@IQ(E`UX6(?kC(J;Wp2B-`yle8|xm{=^E9bVgD>Isv z`C*w+Myg(h1}IE=@nt5uyu1-ufgtMRR2)n4HYc+AlgbcxR#m&)&8A*?D;%2=hfITfVJ)!hhU!i z!feW|g=$2#-!lL!l-Vv)F*Q`^d(&d@u zq(T{{Yl~CbKTfuVZPjI&=qR18^da2D!=ZUL@5dA~xo?notq~AZelC;`-GKR8DBmY8 zWo@RP)#oV>t&vKfqwHwL-zq5|QyM;CYJf(!uXsfqqjNv^JglID{%w|1`JL7FahQ(Q zH&By>GCaD5SPP3Kpxz(Odo#9WoP>5E8i+;fgWXa$U^Bc6(oHT!T^YxGr^)95M_h7) zQ1&b>`C6fD0Uic@WhMhDzIPM2n+_}|l*u^l-by>l<7y5)?wflu-lc4q7Dp{viv9ES zf>2o;wdvM4s+r71tLvEdI0(^8TtG!YVEdg5B*i>a$}8Pd^lS*)S3saoRGmPrL=`WiT4AksnRpG}2r2%_JX7RLZ8P*kK&LS!RZc zH#+6hQT&fhcz=Ktu9mmka-JK8Ee%(Aj|if-*(|DehTXX19>C&w6laqOS}-S`PYF7@ z*B(t^Q}QeHZJCBGE~Vb#*`(is@8D!dAa-IT*4EA@<|T8~#B5*_(AXXdj%nwe@{w?z%ly4F`x{cJp}pMO zoH1vJq%wR_lP?j`Ydt=bl+?08^fA9$@NUH268O_9v2nL(K8T8x-ml}`X=39(n9Jnu zww&dO)G=UU!<&FN_O`_fkCTbR=wygEyPGj)(_?WM8>PqMF*cgkRUkpkuk-}yx`R(j zZ~-|Yu*okIMqC`3HQq^dK^Cra%7^(N2);(-YI`xIR?&O-u&B;HEXpUvieMt%!fzq= z7>EM0SrJcl?rX!jwJ>i90{n3bR`~l|3Y$Zm*0Pjh3RggQdCD|4$1kPKra*6&69Oz7 zB-w=wmf0!f2)AveBvz{Taw-KJdjTEwDWSOBS|nJ*eo^i1>mm7KQ-pDO39 zz%@bOL0FSeQ7+M*-?XTwO|Hg)9E?a@+5laY;@MPX@1|EO4Z!o1QW;5k3KQDw=CI(y zS5gpa6MVQUK&VylHLy^N;5*7f&4TX$3!P`KR3UUu@YS$TQ}7d0yUyT;NM}%&m(9Rm zP3`;RU@q0SNTtpPcbRB|*D ztc=v>X7#GXnVv2HBg|PCj{@!TK|URmNf)&M-NO z#lob}lsKQoa?ew2PFj2U1nT=!q*Kcx!O}=8*0xeBMz){75e)Vl#|idLEcRD6n|WRL z@BT<7lOw_FBK1vV^`*3*zhqX!FjikL78`JqVwSG!%a2rYStJ-Asqah+)%VqacJp^& z0M+m*-9059WU+ayzP$swtS>mgfczGNTsffK{B2^e_A^+US*(>|;zky`xrt)yS?u>L z_V)pu%-@>>j81Kj1pgTcuF!+)(GGx>b?$fYu{!Y~_)8zC-)5HdTUO_1KTw_bv6#Tp zrm=qW^V#l70q}1%^Dd&MfLLx<3ezKU_(coMrV^6-Dx>Ek(z3fE!J~~e_4ANF^*4I5F%o<`67=iA^?!-2ai{#gX8Hk)YUF{g+VvgF34}mDRZ67}eOH z#s0?Xw|1<5V5Fv-BEc&oL5orUTao$?^~3tVg1;H<>;Ix3>p!c1H;XM{r(kvcI@Dj^ z�*_5)4IxHT^oN|5d90<<9DVj@7s!NHxB|Vt-@xm$$2*`0YALZXo+Usm#95W9OH` zk2?th2-7nZpAk^Kf}w>bMc%tFQ!QzYSMcst@lIbCO_~9X6hT0X$EClhnX6XS5iDydSzTgjMG+8_5{$r z0xV3tq%3f%Gc0IiRo##UD96N?Z@>Vfvc} z_n_DWG-oH*+rnQ+=7Uo9SJVq~vFL3Dd0td~n72qx{te0Vc=<3^ExQ^&O8HfS7kOHg z=h+=q5LM(+o*%&E`Sfl%FC|iCOq~A#FQA0S@8IHmH;~aW6(#m=W{BLcN8%vzfF6m5 z$b-xSOoEjEsb_O=AoVZh;&IWF{yUoV4-r}4n2^YvDj_~a-%`~EkC!z=)MAfd-GhqSr!!Bd+CCEFBIG{g-U?fzJhwj z#r9`)u|4p%wi| zhd^aRXqh;2P#a@Bl*#I}OeS7!Y6GZ@Co6~o5C8-6;$sFv@c0W#2v(m+Q=+nws}XurNbqC>EIMJrgM0BX(V!_ro7QGVh|}jsda^V6 z<7`=L$z+PrJuB*mDTrQ>~VY$|(idIpJHU|b^NR_>S9h&Q>SII^Dtly7eb%J(g?8W%Xc?3D8HqeG-Y zVuS`wiBP_oobt`*luzU|X$q%&u}7AeGLuuj`CU`K3)kAN7CNjP4gt9ykwA7WcPYy= zIrYMYgQTlJDO;m$hkQ4-n0Ij@pbV_%uT+7=w2J?!kaWKaJ@GiL7Ke18A9zKle{O zx+{W8W$Vx$PRstDruO%-G5lxs6M6jd?LWwhP9M_+D;g}~tZ2cQE?Cj(qOMrc&*Cq{ zibkI|u1`I2vPGwSoQz^;AEe80yEJjA}O(Xprst2u#*>1H>m;epB%{K9YWbKDD^o+A0 z^kFCbK3+ja#=GhXd^>FJa@dE#u-i5giU^LAlDCbG`!RbL0*0IWvJl?sQ9rX($2N`{Ch-B@GpYZ;W z2CKASM}(PR$0M>UAm-NyPdrT{wg)&1G20QqtL<2Yw*oRHk#18GE|DEIHpPBPg6!A^ z1cT}0AJ;}^qUn^g3MBdyF82TSVly}Q>%2IwLQAvx?H9)@j&1mx0I7*aYB!$RT~ZP< zu}S7iad=7Kk*A@(<=YLa;z`wyHPWK(XT7kerzr;CCA4I6Hyg!+TK5!K)v(&pgo5|@ zQmrwW;#TpoO2k6U@@+C7y>OHi+isUAd6 zakab{D|Eoi?2lnkZl1+=#_T7iwLfl0-!)1Q2?e-=zS znf{&#a~@2C$&HVhZa&+cO!ni@iPyJvGTG6WhT5B7_Kbah4(q|v!HX6*X z_5}s}mz%UN;3n-0xJmnh4F1ba<})ioLRMnwcgZH9cjxD+$`VUSaMLqwVSc`b?VA$1B|M>Q2+Pmw|owYal5U~?gOD<+J8nrp#7)5ruIXqi9!X|>H_l7&|&?u{`7gZV}CT)O0o^7JfcLMMte{S5MQ=;QxO^f4@GW_g6b&^f#mH{$Bb@C;csXHLAbe zSbyIgLH$j{{{Bzu-afpl^ST$*_QCNXc5-wq+Hk7$UQX+Khcil^D>_a}Qcn%7mF{i7 zJCLSIoHlBmrnqS*j@{NI>AjI7igl~qlha4-6B^xls$Si>@Qj|BVPHC?mF>trM*`^} z0XjmU4>0;bHb~%rkT4Pm#H`<1@4G()Zs-0ne~cfa^R>@D`(wRp{npoS{S(gjhlBT? zFa2!VKY;&E9{@f5gggaj`X+f=ZG)3TbFw#;q9@5GeXsq+@t*%*?sL57{%h!X|BMgx zQ+H&#)Gm7UI?7dM-YMPpcsEU&w#`KJ7)yR$zF1AcdQt7U?cx5+iXZF>uL;qn9a-EMD!xuRAc`IGe#uBb%gf1zKR z#og6D`#AaRzZl|+68($9vL8->@N6<=@cS8maD+Q%b6=pn|4^Pc9^JW?P7d8he>|>o`*@Uc z&TWwD9pzney&25aO}vV(tyccd*PkJmXE%;o<#sm6Km~BOR{kYz^Yt%ov9sCDmSF*$ zsr?Ilvy;z>?!(7IU$=9$4Tu?>&!OpTAI=w3Q-68YFW9*~8<_Hj_jiakc6RsL;f;7c z14?;Y=)*U31YvJi0RNtn2z*&24n>;lZif9!J*oCPW$>!Ncr|nNmpGvHT(FOqe?0T= z`yVcw`A_{1m))CRMSQg^sC2%)`>l8Qnty%UmX|kd{U!C-`kg88ARc|?`Q10yBz8jr zXw{vZ?d57MeDG2FfS2OQ*`H&FcJ|XO4nu`RTJTpREu#fqjIbar+S6ZYHqYn)qRU_Vxq+;vHGP>W;wV%wo{S2i%Hq|Lgy+??)BJ6lQw- zTXOdv{kV=)VXys}m7Uj}PXGRRCADeB+@il_t?+;2`)>Bt$3#hV^3Zu&-LHGJ^yyA* z!GV9=+$C0nxK#tasHanpZ9(`p_)(59 zi;&9kV3LO|BpKJ;;l-S*qg@@6%_nBwD?ZEvFwgV4AKP?tE0=hJ9K1o)^qLb+_Q%I& z@KvwJ&aBv*dc7e#_vO3xrXG9kirRG5PT#T%x1Pl7!pf80++u0>q?I1B*UqU;U3R+5 zUV8>Zay~P#^Z#l*`P~Oe5HEiZES=>DcimLpfStJ!^p8G1Y*##9a=ga}{eD$Db%QTS za5?x=DDT9~H=9CNbA!2>`UGb_%4a86V&|V<{*XKaf99^kqftG2Uq10&?|oPDO+VY| zgAYa8&?QY*z7?u2NoTVuKl^W46V86tLZf-np_S$E%33zCV|4dqcx7qWfm(g6tWvPD zCTL}C4#SS9{9%cVhu7D5czunV^)+tR*ZAu7HNJX%jfdCQcocWY`pRQ{{dio~*XTX# z>&s?+eN4XCOY19tpY>IAq3_&G;pyCudw6l3g}z%e!2?dmNZ-8?tfyXk6TM%w8+iPtW+D3bEnuC0Yp%fRVZLhYOZgd;>%F?TvCVHpEDO@4=x@0es+U+do zD(HfZg|P0^7*@A<-a$zD2!H9KvSyetFG)uLbU>d^Io@g8D=2SNd99fAU@uP6SFon6 z6K--wd51bUb1~;&U#A_fi=STrTM$9-U5Cp^Km(KDBzsMUu>nL4ZMIdi-}=5>mhanH zI)~&fMjOHqRk&JCKMbM;H?eoT+Ock^WUsSp8Y3s1!t_SvT_Gz_4y2UWTdD?Q$F8ZB zyRaiu8&#&nS0r`G@oqVVOXZGthc!`hZ zsLZuyRdNWI;@{lmy-selV>NBu$bGWR&0KfTfIX%Xd;8;<$)aL4?RLe^2G<(3cMq32 z^pHGZzhUmXlO7Y+kT}|Wy=(Nn0^AFF?5U=c-Rz`CNuOzvf}o_wZL0^jXLqU0$sD7f z!^n}u-i!Qmj{R+F$(WrV`gU>B2iU@|0Y0KBdhZ+Z-giTLoTIm&mAALc9#wV^M2+OkhEI2x z_lR$Amd&=9J>pj;WoX3=%jOb&2Sgg6pzJjVQrqB=Lgga>Ws+CZVpr5Qh~Q(#v?}af zHEmJxwUZcQ!^1ad70_*%sdi|`h1qfy>*OD+;Gd>fJhh~P56-oE-DIEZ%{sfLL~y4z zvfCh2d5}ftV=vH~EjEcnHNCY(za?K`aVB$}*t{u(VwO!H4S3cIdL0;kZ zIKE&js~1V}O0l$7zrA~;41wC@i0j>PcD*ad^DeEG`Zn+xs#FQHmJ@p?v^?+QK^+x` za5L!DcjeVLLotOV~}`ybBnyWnOD(ml7Gk7dx{jE$iHo!5&56e@_&Kl-%THm z3V*h)&Cc}OsU=*C#NJVTRD>ie>YD7gj4E-gl|YR+$z@fwvqe?b(K@{1&@v{yArC!H zlB9kNJE6O`5O=I0d-q@&{r(s&mmb6gNE5T9JIHCBcGEr2|& zDisM3zuCfKV@=t)$Db7SX9t@3f|0vCzdx+3B`i0U92UjRAJY;ab+kh7cC7_BIbvja z$iZ>ZmQzs9(AT>PtCi{z^8!k>w#l}-><(Y4CSeq~Uc>depaz+@GO;&bO#C?S*LKB)a3+Sq{V8?GfghA5?s++s?0pNXR}=41V7#%Fo{*d+Vl7LlSYjq3)D*S0%^nHv?6AUB>6p zC+ERzR+!lgDvZS5p$4QrfWMj}x4=TD1t5s=>n?9{bM|v|xZ;2{(=y2QK`*%Z{4tYRe1>3tNYLqpVT_M^ub|@~daBrN~0yWyNGB;FeSy@+r3&<$j z$srh(h)QoG$f2?XyHdAMKm-j#zd`z{QM71Iw5YJBRpdmAiY=_ji54AYeH3eylV&r( zC}J=`5zvbw`nENXd0VvVNQ(|AHXQ~iH_bDKex8s0F zC}=yYaOk8ocm*&5#GZW1#ro2SGiBDmGxzNy6&vD*(EJP>Oq5& zdVp_|BLTLP3~Yy4v}btX46}`wb&YO%R%Ejh^(%9{qQ9dGq6Z*^Gk zHdH-Cdsp{r1>6-@z_JP>IylQ1)=m-*+W#zzV3FhlO;pV~yEW^<)%T2JM)rLuKzG{ZP5oCLYE)mXy)$y{EnpbQ<&ctn2$0zM%56@D%3Ii}qYQBfT3)C?RB-8z3 zGv%fXJL$e{xyK&mn>M{#j82df^=US!(;*R3*1)#h$H|PB*m)^|?HkHUv{^52Pw$^( zi_Q1LO`hQ`-Z5K<7@vY#wpQHa+boY$to{lHMs#bh}J z-d%_eU6YAwx@m6mB1jKcOAfmy-g~ql$u3`}%i2$p)9oftLH~y&r%$D3UGF9!tV>Q( zV(+bL`^`XR=Ip9gwi=Z}tru$*v8llbJcsU_vWDDb5vLlm;w5+_^CEn3hh?UJM32XxykOR)Wqo0lOA%sqp00= zGD~jij^gRifMY`>Hq$COZ@(4*z&;Tcy2HaFFXXe3jVoNnjXy&gryqFgwLo(ENOGX- z06H^EPU@UXt{&k=s9+Ml({!s`x@Y9pX-GKdkOBs%CCM34=@KjfFfS}oZHo`xCc)Sx zV&a=8pu(9D`p1yac5zrE4z$=O5{n_%8V@2wPCC!-#f8;)kbY}tle{k?Da3l(872uM z6O#uUJxLtGPqNLkg6HhT9l)$@J{AjUiutPgIZ13U=cP|q3ed}RJBFwh8k3k?TA1!26!DA<#UGE6! zdS{~y{nN;=hHQ3CgA0SI=Y~1}UqGP0yC!8z(sguDiM?~xiXBkGeVyVkGoGPWAru(F zeRdV&Py#--tbGAKy@dZ9?^+1|sqB87m6LfNW?iM`1OhH=2q??5g0(rldRAV2S6&G` zblXVC1^=~*7wwWCads8>oAANkEdhVy8vKoE@ORcBr4fH24GPf^FR*aJnnqDTQLPGmgucGj5Zy;o9zc+oCr{>>-Z&tgxhkTc#BQ7!)qvhVpFg>@fl9l;+ zc1LK%6qVQ6!cihpbwd)^X(FA-w)|b@@99MxMSL$4vm_Z^a!I>#G>Fx77k2CACXM4& z+iXDJYk@jGVO~HT*SFc$fc=`Uj+1V?u(Ouiuwja9R&v(yj)Y?_u@{;+$S4TU^#Lf>j)^|;d;w2v^Lq}fU#&ou+r=Or%Q_>$2-5|1pf0AA3t)x;3+0>>45*nGdc z+LtD!ILW3y&!3-X1&Lsw$Z=N(3x`GW8-jM^5DR!ie1Oe6o#apuGL(H+ux4-&i|LoU ze%IV}roT3Btu)?n=!W$ht9BpCx!yTed@lLptlrq4v@7bH!`f`Xt`hv88nLYca1pxo z5DY9tG}#MnZsv%YLXsMVuM8wFnvCoZNtTKo3IzilGF}|;Ua^zIVQe?Z;bk53(=qRB zz;!otz;8Gb@FTG5n0TKu@xBk#Pq#@wwX4b&d?b(~eAb5~0wduV4nellz2SIF;=m}!!smGJ zgBaAzVH$}{`l4^^wZJ!&@VF0QOMI^@-6H;Mv`Xx9dxTHL&ZveC7N`gfkeFVAv@fdA@z*eDRH~;7TvY zccffvQDyRO>MCvUtZX2TUdiEj{2Xjh@-F7s3p#fIwzC}9ax?D|bZm1|gNhxrUS5Ut zgSgS8#G4Qn^@B*;TAwQ*6G7rnax>Z?@<F%!CcwI-r)$`xT&i4v8+|PfE*1ZbL|}8 zCr8e@u1LzlkV8Qlhd8~EIqjwvMdUVf7D!nImFx?rO5gU~y9)w32I68l0Xq5!bc{I! zuBj7mW1j++{Q#E`JiNnRxjFyi3aFR+-vFgVr?jXM^|H1#v?IPIoK$QE=w9N-Sr z3lOAuk|D;s5E@3BCB~BgqtE0X`Sdfn?|rh&ICIg+yr5PI2uTtNo+FY3MMkM3CS-6= z{N4{Q^q45bU|2%Cw3Ou0ARN_&3^+xIJ-11qdkAwhn5<+J zb12SyUn9&h3IMBO;@nj5m^|n8K>M9m0y~`4X~!xsFtm=k$s+)UArCFHvt5yU?INW# z%b6L;_SH8Nj%gu>MF2;q9Q%Z1k*?awfdFv0035d@aYT}H6tr>h!RXK3YK|qONA$H~ zndt?jb~5KkFznP2cpMUYFN+!@CHqEzJKnb62v*y;sv=ph+}VUEG{c_Tg0hd%r@12|?>mUAGICenaoz1eVzl^9!bR%#dTK^sd3ZMtq||$*E=!@lPZUtgz_Qu{Rc1LVKxiH_M=x zcGquPgJcA?>eLA1hQok_zD&TS8*l^l80eo*&f&|61LNlWI3*>xVw`+zgDW=P2UnaS z6)lHGzK)_>N$`S>9jBp^H*IUgh6V2AgRRKnV8d2i@=?&kvZU<0K8(*-19b$TLgz25 zVSo|;$O_gHjZ%<^zkI4=e_mxPV;jly+9P}&uHBrbB)NuCTag^i$YNixGo_7h&R z5u+u%_DHqhWs`V7(+nQOSlCR5cPM+$QlrjUY(aQKsa0Xk*^6SOX?T=EH| z3<1jK6dmRu9&Ch{`<>)<=&f|Y^`<#F<*+R{g+k?z{KVdm?#RE=w|=*0>|`v5I^+x zmGPo}hgLr4s^-|q88u*{N2;_+ydtJKrdZqjn%IbeaNOc37hVOi3;M7YRz#Yba5Y*u@a)DF{j|bL?Uv61$k=E0f*P?xs#5Xu+jeC!Ez_VntQ$ z7chbN#S1!qaZWVr`&=+_PyB+Mz08P{Iw~{B24KI) z?_32X#<)~K=cZ_Vk+2*?a??fb%Ym#P##Fw<-ebBB>y$1dnId7o#!!NA#Gso#)$y@V z2*X4$wy_W_IYzsa@&cf^$XW8!+j46jdqz^C_sR52&D1a_tBQf%Z<4^R!4t*%;E73- zxGG3(2}{jkBg=Uc!ng-Z48jA{F=XHXN%XWTprvjix5!;&+B5Dwg;6E>o1*Z5Z*kxpHV z!g%6_{br!M-d9y5@|7>NAiK-xXoX`F)(IzhoUo9Nu)dEZg(OM(>Gx+fp18@iP2L0@ ziR)c+q1oO;A(ep?e|bktfA$b7AddKp4c4ZKK9cY^q-j|jTpu$OgFsn}Y+EMVg80F& zEvYu!lIEEZgupHx@@f|XNo3<9b`8N$si}gcNF3<2jeK9=N*A{N{QYPoLykvBGKO8N zSFvU9VbAQU$%Hhv@lTvqbfrB3y@>-k`H4vaNM!~fHMR@LI0%LiYu4r(AGsO?5eTA= z`@mv}vmB(#5(oBio`RM=uO#J-6C|0lX`@(;tmotfDRjWzN~U6{*Ve~C;6;E(zX2Xw zBEUms_u&q)ni=DACP{_SqT7!XbCV~yU}>0Mw{uwI&_ObC1m(Xm~p^*irG z;*fNp98d|D9oku0H_2Oc{8CL?wX@x>tZh>Wj^a}A1N*ZJ75?1i* zf<;dG98-ivhJtc%j_DA{e?7n=wQa5>rBOeCpoVSDyU8mbltvjq-rMzxTnp}6Gfr}1 zbsB|y83eae$4OI43}jw%C~pQhfv4&6q#!G^0z)}4-zu+u6G(k^L^Q*LTO zhcmETC7f|9C|9)8i;^*kBNTSuaK^P&<%;t`or9CP?WX!M3|H7OFDVUZZ|^EU4i>_PVp3Npdcw5|E{f42BI4Dv=@KBqTfvYKfMh2ugg{X7IuIa`P7_ECV!7n{ z*i{@Kw^M_N!m((OCphR5rVWR08CJNN0VO3oj6(WYIs}F6Xp&Ze!vt~U$lsG-!m(93 zlo6m8h7fT$JzOt+F|LcI5z;sn4r!o5loT>Llu{$8+DFXTL5snlEfuk1kKwc1eBOqUsCWEta4$hLOA%U>>T%0Rrt#=WxMFN1O&hf?&in*I( znQP46Fv>y7AqNH-hXjRMd;Ftv+C16LkQPk)ZZE@7G27{Q!HN<)n2WHJ-Vu|$-M5Om_ zy4KZyE?08P_1>qYBtN`&zFI=1sD^E4c+qV`r7zHHW(Tccs}NaHCPXM8@hCH>CH7)I zA%{RBxsowE6egq-Y7;ndP|AF9`@-vM(XJBgm%3}}AS*igzFQ^fUnrsr$h#3;ZQS^f zSyZ&FU8F2;+LA0G1uNwis)GBR?O0tCJ!L^O^=vzum35h81hH0*HdX8s`%docu~m4Y zA7i-%Phey^Di)P@0ilFyuB%9&)vJtM&)Z;igNsUTio-@j7hMYZ11h_pBU#q_ zVYgr&Ua%q{p{kJg?;|gm>-wNdaH7RkzRAraJZxEP}!RcEy6^nHpkPIB;dUePV_VFa#!1V&&! zY6H4*E9;wCV^p9m3P*aawXJcbnDy*wEZ&f6`~m5}`;& zL1k_*jOY;ST}8U0@&$5z=C~5rTEc^ZGk97u^rq_a zxFkr%CGi3UiUi3prZ{texR=8km)&%?AJE9%kvKpopg~66JH{@|QV2|725gpl@I-{f z)x}L5y$_mLl8tm|L~;+saJZvnTx$JwL@vmAJ#gbbrAABl$*kl`Auyq9!uPQIZjqBJ zCzF+u%fx{TLCwaBuG!evE|6kanKX-zT`<(J$f4BDkVTFqb|IHSYuiuCFqbpM@%2Nsi4 ztmtgg5egE&aFD{*7*JM~B(zB5(%Iy6)}>gDm}MAHKx~JrVOpj5;-CNuMhz8`U<#EZ zkJ_1{o0_vt!BwLM6I}cZ4cg-*mt3?f`^(PFoR)UTC6x9lZ&s}LoKBKR!RGy7#l|SC zpr;%u;w(fzea`2Hs)kdMWr;`wSHNGpKc?u8=yN2;GlKx6@H)@f(taFSlVpDU^=J?m zl3+dJ)skhZV!>9aaUd>^pdf(kyX}L)!(G{ZL9%%9(s~Fugjo9k`eGHwSqwzQ&h~ zG*1#(c}bgtaG<;Ug96=S9O&-;K*JC!)MLYNze2s^fsg4n)9q+l72nSJ@$G`#n^~0{ z1~v($%d`90t;`(PblhC1H*ZTT1Y3rd`-QnDLSgP`q27ug=I*0K%KhZpGF;)jdDJpo z=EmGP)0pc>TkQwNxUZ|Kov77oa}nb{B{A+((vusi)w>?l>W%7Jy%{kHB{2zupa)1} zFqR6X&HqA7fF+Hy6rr$(MH;SRM;7OlT_R0 zph{LMptFZ~@0>Iz7k8g-DHSUvrYuIV)5E_}K8tSJWs%%SDPiH|l{jz(RfyJUslU?~ zA&6+V@K(tyVj3pcD#0|+p1Mf8n#57VYTH2Yy;Ba4O+>eth*r_kZX3t2#^?=p+$*%J zy&Ne<;e&Cc07C5ZCm2X zwxaHsl#closoB|!}?zLiz;%CD9!FJB6Q=BmAb-2GS-x6w;(92J)A zM~;ynxqwMk_w>*&>84FLRTNGEXl^nAR?$gz2@gP*uG2iJTd0QPNSLRmxu!bTyrQd# zVWxYyM!K`y!0bEl%*oiNizcTndR#Wko7Qy`_LDxUJl}b)-vQ>Z#+`H@tGKCvn?z$v zS`Zy3T-G0QIiY(PX6Ukv#eLEfl7TN>vK5~r!x@(pb@Hsd%k(E+w^p}3tJCyCkD}}Rgsq4B;843! z!9NuBEN!n_Ye?2 zr>f+*D);ljqK6b|ZjL=9k2AT2k8d!lGh5f@rU;f@;y^R&TITK{WFw_XXmFsX(px1B zO{C|b!DBhi8&EEqjo?m*fC+#aoIoVAXQaEKFOK`qs}JfEu4f@qDafNkM{R&C_`=#1 zxPOTt`#n1vR7HI|EnmK(m7NmfKg?^>x-Acf9dKYQYepXgUPg4_Wm$?+Z^E>tPk_iL zeYE3i_e0$1hgtJjsKB{PS_h??1j-q5&K=Mqzk+xq0mS0&7nmBnEiS6=e3u}51aFU; zF6q1}=#S#o`Ic>9)gLrFr1(}g}ujEGMB{VUtz_U$4~+rb1P~7#k`|arN4uZB29>V@+XAYkeuecT^7g1yl}1&K?BOuP z#5kEkRW4g4)O^txn8uD3e<$Fz@!eg&PE`_G=QZ3OL3<1@9D>mOlG9<3%@~J4HvP(a zV-UL}3^NhbsfPQ^1Z$t6m8MnxNmHb%bpLL0ireA|YBO6KNv5d!kx)UO63VGyRf)mq z(-ak5)vL?I`#5jpa9!Lz{V!~k!zCg`yQ(qIfP0iSipxfPI{}9ghCztYO-@F**0^{p zbB%MAfZPSx9$ggPDOme`9rz?`!^uQ(cIlNYzPn;zM(7Q>IfUzvhv9ag106Ig0ntHe zBDzIb15M_pcP5JOV{Hx&Lx6qj3MdaLXo<*XL9Q*8O=+x>|w50B_+ybS`2ZfRr0dZC57B0 zk04)t1IwNOW5lBm%V= zr6U~UlS+}Bj(0Q2F@nex-M09RGb+_1^#Q|iSP>hECR)^eZo1FK@1h36PbxuZ6nb~B zqjMIUBR|O%$Seuir%Df~Y$I4K-A1}?^-YOae*&J~EOS)a*?*fc| zAGJ+BU9>G1APK7Qc@y*oAwC3OD@)8osHZ}E?oxUujV;nSz!#t7H2b-A$cITIFXy(A zqL;cgZ%;ED1cXVJ_4qEf8`FF%w8^0#-lt)SR_MI5cY;`uPml1fR*$CdlMJuNbcYHp zY6lif8(25Mm9;3ogd8c_&s^)AlaxKZV{68@%eYM>?PL#bnFi8f&S&KP@16uIK}nX- zbtLCO1%W7y-!H*I4{QEStR@A;ZJEM0UQma`K{+iB?J&nuXoV0~!2K6n;IJ*wO_HTh zj-S?$Aqqik6cM1ndLX-smR{xUXwkGYv~i0><%Jkgn(1Rw%~2s9V5miCkPHM@g(4fa z#7Yj?;3<{e(GpGdJDE8GW=c5R$-|ZQg@RzPEISq|wqLjr#oLJO7gs>sk{DY8dc`$} z)*bPr&)HJr;9AFRRHX$Lq9k;9gp;~$PE&zWmHUJ^Wfm6CBVmW3+sVv3sgg@h9IO^= z@DSCH&^AW9_zFHN1=r2kRg%-I)NPxKDoaw$jUBoot(Pm(in^!>fzq+(?ZmfwpLY`9 z>SLRb*juEfKJ$Fyo5kl7XFu}S4Ud#zLfyge84mJ@A)XSn^B^4=R{P9MC002vFiq>; z$RxJlUH;lbU<`gAi-;_=U@M`eHI4Sn;lh4{uUti%EC4$HCfbhRsL!4C^-OE4D7FI6@b!i4LydSS15% zYnok%@BwZm61|i-Fb2$*%|XdJ*)P};$f85f^!_$^6_;0E;#ajTPS*J<0~${b4diYJ zaVNP4D{)}nkV?V!cELu-K^=9hYo=(3RbhZJvz>f@;k+Tu$SPEj=27Mwi*db&o1CL( z(yb&F*4Ak!Sw!iwaUA;4)f&oLAsh71k)Ua*GP#*iW>px=KbDqy}iC zN!dsu3&+C#b-2zVD|DrqD=O9NfWI)H)3vjEL}l%f_(fg2E$oBzw2huJYOc_TjfspD zQk-WMqL2My6MkDrMrq|uDenCnigS%Pumqg8^erxwxEZb#TE^6{%!J~so};;A9FQzU zfu%ShVQi@+-0dV6Z3BiS9oE zFYofOsnzU6tJ`Tc0kGLke>BZ52eZK(j8Qb2??_VoTq?%&l|muHCU{6`t*c`p{Tu-4 za;-B;i18M8wD9WkI0A|skY#lQtwIg%5DjG`}TcG9F(o$pd1E^oMyQy}Fhl zC@1+3^6vPygJ!2XVyQ>!q<~=Cw&SrfoyG?o#{dZBh}B&GA%AF7XaH(IKQqL{!*y1Z2%d1Ti6Ep zK3CSaC~H|N3Kt~mGejEB=91Y7ZBGYaPrLEQ*wZCi)st)zjZys{|MU^U$-QD%z51{z zg~^<0UB`6%e_0nyH2H>qzgp0?4`uj-B4?d;zRt37=p)lS*J|B?TOluP~n-B9XhIo>X#c(Q->BmWJy-#@?W?} z0djxKh{Nd4auNpIXn*?X%$I);({UOg|B?Bs@4Y0#koc|p3d4u(G%Y<6-0ag2$x{-B zr#~uBNf@49Cs4vAVR+h>w@DbD-X>2;7@q!;JSAaxTH_9vgyApb;ubLdfUn+P{K$WU zh2iJ_OfGH>Ny486SC-7jzjOs>`k(QKY;H*gm+kRn|XBeS|LfOTjh! z`LE-e-8)a{A*>+OWR)%9$R8Z~ia$xi@y$GDxbMlk-eK~L6MG693P6pted5(M50sTP zBGxz1*Kc=Ndr(W?H@@A6!BH~Y&@`Yxj2^og7O{6kLqCc@VI>PsJW(&Pc|Y2a7C zy$+s-@>-kt<+p4neMfoww=V(j{WjeKLUgUq@~h9$T>ww%bNuS_1^|A8U;d5Ui(6PE z{Kh}Zy@2DonRopkaW9{%U@8XfyAkaSOldwCUjujLgEA`ds67*-iP9@NYbw z`|+o?(-u$X{^3(E@#7Ca)l82!x~-kR3L7vuW9QavA$<3so%_LqxZ*1J>VpUQ@dpo< z=y9gSp%cE_?dS3kvvnakmA7P{H( zTfeyV*J;_lST4&py0b*%m!Sd(dC(XBg75w1Br_6+Pi9m8Rl-*1RpO~q;>-C&rUw5G z(!VzP*G&H!5*cx!axXNyjVC+ahD~_0vW&+!&&ebhS{6b}Bl#|J29}})SYXVn%93zhuJMjI-Y6nw-k|F(y58{2pWATa=D~jDP3e8s5QOp0xg^-HK7iRwyH_d` zf4*d4RsH2*UaW{Z#nZ!m0J$x}YN{u=CS!d|xihCYz(e`w8o?V}2Q%0cmD7NdgLkZ+COuRYX2+GmzVU%&6!XC4&K#g zbOm*S=2YnTahhsvI)ex3 z*Z&$vmMQE^(&4YMcio(zqxkmYoAQ@`)yaMCu9hgb@diT4au|um-@SA9?rfeQPDEZq zV)X!>-_Xosme-x1evoz?Gdq5G)5*(~+P;OZ;-lF9Qp0&Ge|}B;41M7N^Mwy@x^cNu z)_3UI{sOkYvZVpnk6S&qWjb|(1(ZW^=G*YMZNaadzhpPin^UmYOY)Qx^&`s;Ba#O_Yq z+_Ig|nsvvc>{dvdALN0KcX$adA!04dgLO%{UW=kr%4RU)go@ai67zbCpOBiV5l>>21ZMWHnGu5 zl-IN*l8w*GyNJgFjU+b0;x$&wQ!;=IIPBb^mw3BYhFspn%#u34Qe;KcL z@qjzpxLR+u^&!T1zpy>cngvNxXx33FIMoRfP0u7L>VW?v#7-*scr@s^)d@O&t%E5#PcGk!q}hdm%p_no zn|)17>Ro~`QYC6Cu!O?N6p%;{!D;FE5zbv(7v@b1aSaoKj!CA`u-`jArMW^&6swxv zYH5!yS6r6(LK8jG$1k)y)=ejQZguimk4GBN-M|WCYPiUiSq|Lxa3w#jJEYA(K_I-4 z;*I1&MH+rcL&~m0*}ehr>83^yxL~%iQ#^j8FI*Oqaa+Lb=-pBhM`RKu_Ag^Oi6cq} zW-J%VjngX~Cgq!i-tAHu3Vu}uEfhLu5EYfexrM_L4Mm-v!Zj`kcb{LI=aN$`ocCjK z2Z`f6*UO?X^sY2YY<8p*xL|^HF)7lZCC4Yv<}l1K7~usPv#TUwUnzwG=*8qDjXW&b z$(x!mQfD3q5j4po6LC}%@DCvAXo4p3Zy;z=&_NUAAL@&CuI8YgdHn~RWPI>9Rdij? zl&y+?)JEbTwc_!9m%h0{eehxyS65|!ofc`AO$!)}`6~C=S~E`s{0ZHrWCfAC;cdAm z9yMJ(C|{MhV(y72P5EkVlTYy{q|(B+CWA*<$e1(TU$F0(X33JF6E2WlbCXA8IZH>R zA#E7pz@t!5E3d9OG;|Q5=^#ZOt8I7uA|N-p=uf@_cV(#tQvuJyfN-qDcF^C%$TExmQ}LYU90ElwR2QD zyQA67TvuS<8&qP$5oFvSuIY=t&?a3Ar%{sYH$uALfFx@gBK)I0T&>m@M3Up0diqiv z9kG%q$JFA?h{70yQJoM+x!sUTE}2F*K|V!4`ZvG~9;j=~{kvfS{x84&aGA_||N5_Ok?+;bsuylW>ia_R92sc-RQC6`IA4ET zPbNQCa|`p~|NEN8cV2$f$v#OsG6wDz@661JFFssW$P8tTOXl-)xOLl7(UfKBm?l<-X9ooi5sU^D5o% z8O57EeDc+u~_bw#C!+vh4-z^`ZFYU)lI+lA(GnarfnY~X{&DfGSpCY)9+OF{@TZJuYT(T z=es`p03iNO5Or@LEx$fnSKf#!o>8vKJ2c4;oij|j&^X7ha!dRww}EuL*SelvL_JSe z=WID;{_<}*SrhMcvRmT1$B>?K%d0%4i_$37AC{@S%WlP{+`*CET5d`avRl{dTJL0e`R}hO-}aPP`00TvY#tgxiw|d#4A+)AivAA;+(J>hQ(NU^&$FAn$+kq1SVv}EgX6eIqCQ%M@h%)KrLqsEp^-d0-6 z@T;#$a)%6--$-C5-J;40Cy1YcaKhsN927U7;<5<8?qb+UzU@RSOm!BAD>3`4rQK^q zmqUX{M0mMf3@(|HegRMzE|4PDOiA?z1Aa(l-E}fO3|@K!yo{^lfF0Gre0{k!H(Qf; zGIJ_hPyX0NK{JcG;+pJDg=4MgrYLFa)LLPY1v_QB4Ai zAuNlECwW6f_r}p>X$>e){9NbGWcZaRQISbv*W+M}^@dzvj~7+dR-z|m1AbLDaX z7FuuEWFRjh*BZ3@vh8A`z1fY=itVi??o!cQ7wQQVV)AOSg>?k;i$R zV9p9}2r}PU9JG+RIi@DJQL3MI9OTXSNkS2DgYX}JRDlsHlQLYxq%NBysmqdI-ys+s z41VNgt7O0HT?}OmUG&L@2C`_ELdfG2dyiH}dsVo$5sV?MnGL0Ch1`T#eT$x^8gAUW zq%s#(r5p^h+F;chaFX*_T_G}uEIF~Kf=RzUUQo zO`HrxHBeDWv5<>~_5R586=m4OZYr(<>OyAU<&r2gyrf50<@B%GnGriR=P<5ViaV=# zKXNRot9rB0amt6p%UV@Q6sK}$yDP0(F8l=~Z3`~CLg#e{&WSh|d7nfYA0%fhJV7-F z1E8`Q(;cOw0(vxUm&$%6&PChew3(K2R`uT74l2)O4L}Moc;GALx=01nsG|!;{6QXO zl)hkGP9Ezc^*YXaR^?M%`W|YlDVo`tgEGv(@5|JSNBf{1;)9C!ydo(5h$(&>;cl5( zuxyCC$xH2s3!1!Tr47cg%zrVJss-{cQM1Af@JwwmZ0cY5B8Ap~7G;JLGHq&vbof=3 z97l0!cp7E66Wq7}^JDlNRNeDQ+#oHgVE zye^>ZfN7d~nV)GX+&iOVKba-Ne~_A&qBS#)NISfsZP)DzZj)wk?{KAO0m*zR7w2L< zdNVwT$nggeDKm)Zf*wRPr3Vqcy=s=VOradMQ!5VlDff^wx`6I`tceh}p2%Qi_+++~ z2dRS;kmL}RIaLBdD(l+dVugoMj-up0Ii%7jbpNRCfs@IUWW_w$8H^e89%nll9Ay)v zH}XD#VcESk13vp~L6ZbpAy=WMx9U1`E?HUq6J1h4~x zVHY2rF?{OTL(=IW`;}rh-OZc*{`$}mR;kMJYV%?M=S)RuuYJ!V%I-X94pbHrX~&|O z)mpIicQNlf4Bf?6!fU+D(_P;u3nB-9ilaL>F!V61X+49{xWIA2h%r=fGWA<5CvCBH zMRSFYnDrLz67tuZZ+Un&M-LCRRG0#P9wFoK95C!t7tPernes33+(Y`P#U+FitWoS% z_~T5!5o@su@`Gb?v85R~Li*-rX6#B{33^B?*y@r;oyaU&J0lu9yI&UFY3Y+Fn1+0` z9_9Up{CZk(^|a#ZX~kVc{FSAW+*9e?Kd9+j{C-WBTqEP?7MRFoMf&q2ZE8|jzC!Nq zqKg7exwKe$S3oG>ue=-4js+*on`9>!RIHX4nyQk5jLN#AYn>5=b4zMw=WVRAyuVgP zHQN-z?)|!2W;8D2Go?E|@1lg8C7FqJS<5YKO}v;QANv(@Vd~oH$!C9h%tAEA&3>VL zTW-yxzIVWtHO4zY=~>Bl4GhvU;TTLY5T{^WrIY>2dV`=e^|r(;^BkZl$~^8%;$Vy# zhV3K5P$gh)l5IypS+j0Zm|28AIqy$0=Vnf*Y+c?3H+#_cru~D1%DQ|%|GV_7YCWIH?sucIZiGjeZT3@#x#e(Cz})1J<{T zqtM}(E0TP$&L-6y_7YZth6NxE7kw&jltgHwE@(DD9cYF-mInb1tufcT9QKZe{62@x z@5$~QCAa`c2n2Q3;$8^;VJ`+d zJ!opF^R6}HCMSeF+<>Gbt}y>-`hV$i)-Ct2nAUZv3=FjS+J3T z(>_fqv3ZpPF{`Sinpg%QyJ*B6ltSU7ZnO&`L!y#>p@XzIDq@-2ojr&{;x*N_D8r7( zcZFVY6<7t|3Ur`z;B#E_Av2kGU(f-{J(VZ@i2k5712K>OJhvbtpzkYoj#JZx&2Y0? zY7Bk*uqzm$2x#+0NWHWn&hbW|%ExILH|UZ~LWFkZA-=|>mYL8!Q zQipxw9~b%(V-s}CHB-~q^=Lqo)rT_d-r&m?%=X3b~e+~IeFSpq{7=UI70L@)3Xr(G^B2|C1stJ08=+c+mQB!qp6GO z(TZDF;#``HcF;7FJn*4_3fMMwZg$Z%tP)@RHNWhI&m5fsX}=?gMZ?FKEPBARfe8%FSn z?%>+KqC+zJCg^2Z(}MdY+{B#d0@ZT*BH1wJ0s*$WMZ4n~gb>LHr^D+(8ger(#S_g^ zPEk_X9c>{^=hE#8fUdc6>{zhyH-=&n^Z_rb%A8C%s6S{|G0PRM9pB7M(Z;~UGkWSg z8fmn_vSgM+seZbi7)NMp9;jFy-Rt_IT~*h?7&*q>Z_-AyAmQ{$d39M{je?05 zRM`UyPbFU2anSW{>xnVQxv$v4=t7=R%E@su9>|Z=ZD4m7?+rcwRN|YH&nLdw_q?0< z)?Ml5qqmlxPrOLlQj?9@acH@pE0obkFMjg4{RrXn_1M)kX+d_->9Mt$AHPZOy+~ib zd>b2#s_ZvlUcYskJ{eNk?dum3uW+~lZ=Rty$7%C<+I)fjEz-z zP4P}XqhICe^TT-Or_AScOqb6HBj^+lbEVg#!bY_c$1cy(8|Qct@>uH~%}{&nMxM1` zI3`qVr+^%Sm1{?FP;Wn0hjpTPg-`m>rc0kup*I%xv2agNxrP)^RkT++?CM5uER(S2HEA=y&s2> z-AVq$PSjYC=d({7hgLyc47PW7A{$ty#IrHj!rKoW^zy$ZyP(7=sM&O5)7hF$Kfxd; zv?|YTh@lvP^sd`36%a-7$aOtLho4y-#pZ&t!~*)$?82-Zx_L{s6Rf6(*$NsP(xW#+ zU0v|eX8b8ijs5_cN$pZqo1Z_l-DtT%69K*$bb$)Nyb!oY3$TjZO+LbISiwxc8{*;) znPPBCW{kJH$+!7qA>A1`2^AflxQW2ikkldxoYOODr5uM|Nn>)sO`cgbAko{= zgf<8n-h!@4;m=_hd%2*40n(mW;kIaEJ%rV44s!{Mrc6wr54nmcImLXV z;d~<9VuTFB@dEboc{tup&CV1V;#h#L&0Oc1OXCj51TMs*Zjl*!M?dC!M(ErWIu1?; z@hEBJfEIb45*i}$u;?PoHRc)?4%Zrj$LR6_ZkF*I7{PRse6&-vMx6ABgPGE!yjXZ8 zPgro;5(frMMUhx9uxw(@jLAm`K!9L@^D4^_ZvHZPPnPNUU9}qol6+JQcXUCx4Iv zGz^;FLGIQmS{Lmc1&vmCK`n-8HI(+?8|1|91p2tAVXbP?%f zZeySWAtssWAW=hIlNhGAx03)`C&WMGbPHB9mnLI8qd@y)SCqfOFVh-U9tm_}?|3zi zCcs)a@xUG5qhu_K zPU@7Xh;CCIDK3Kxu%$~QML!8$C)jM5z)c(s%!DSNxfkOb31Wqr8Q3Uj%GxgJ+bnY4 zQ)t}KXg%hT*LDx3dABCN1HESInlLDgVJ0BEMWJSE@+=7M=_cqxJoy7xhH!h!#B?Tq#RMKUCmotBAd^)Ecb~gjgswDh`GNlXe0mKaDOLwYNa_WDvd5xBF-j*^-!Yf5HQG^{Le4yY%t|q zCg-^6e#1R~b2W0qK00I{;`VDoNfgSHbT>mvt6CWuj|)P zbHqaP30ivV%lb`bs4aI1#0`rJDvQIq?DOtYkxRaR~B+tHSDp zL|sm!(E~^1|E?{Q&<_2^fU8H%PbWW3zZ(Klw1})nn56C7WWzE3;r#3 zJcmgAg}V8Bmc<5KCx!tt_0R~1*f4I5Tv>IH;}FVpe<`H`9;6FAiv zps>MaM2~6Swbf`1jDL&=kYYMHH+*W~}fhNi=F@@}l8fCv+IB(pghLDlaz43WWD zvvM-ORLFKmL?1WU12`f%wiOrsNEaT!XQz5}5UjiSomb9rF}1+)OM^3GA@?)s)k|D_M!$7r-d&6(w%@V^D+6kwfyBQ7%Lnp88H0J{n z??>|TUnEhrVQn^{R=8l@II|zZqen?wi`lhTg)rD;U?T{%Go-z+|HbDSqb)B>=xWs| z+TN8!WyYp63yu~FoHLmE7Brvt#bfQSR#qhv3g21S1{!F6?p4U)XOVZ&A2C!9$jx%> zF|Z{XCzo8g2(%twa!z5zv%9_e141*~uZkFK%4{DLxS)5kE3^-OBW% zFky*gnU-`aCZq*23MZIST+kId zemm5>R_kr$q?-}g!By;rBN8L|^Ah4PG>EHz;@J6eH{0%>M+lM#JETZtO7>aulTKDZ zB4tOqZ@O-OWs=q0ov*CsY_MRQY?v~~*voo(U&BOvIxO&@_BPEFE)9D9^njsVV20K;gcq4|UZSxVi@r^Z3ki9r9L9T5{5Yh$rt1?;s|yRSo7guH zy^OEjEIYs-E?M?jH)EFuRi%*32l(`*{}+8kO-QCkaUWHkNFOFSZS8%>Sj( zLNFba@IoD1O5T#tb zjzZXkVxihmz~|N9*r2WmdAFci)JC5+L*O-8P;nYBT1ve`13AtpRA4JyG(pSwDQ3D> z$(aS_9&+^dAn#+JtbAQuA^SQ-%RuNFK+7l7oM&E!2)c*045Vu~VymH8q7-aX;7J%q z)B$IxV1{b(3~}cyAMrjC*F!?rTWFW3tggU%TWTwPMyXQT0TKEiaRza-T^mPo+fYt8 zh42vIJkvh{Rm~a+m^R+dF)|-Y)mL{g?Ef^lFFHTrBfU@6t+d6$Royp`%nQ}9n;5$) ze*0%-^yMMSPChaR!-~zHy;1J5wRK_Dv$VW(V8p<*LM>6K`2-D3v|gyuv>&)ulpWZ) zhEVLLMbp&A*QrZGY^t(|{=QoHw$LHm+L|(WM4_HJG*4sxaq+s~4u9e!rh&*d>%m$J zB5z)zSVBqwY%D}768bp=kct%%dy(0=(6UTx0@it%OHK#I;z3J1S{p(B8ErxAhjcp+iUzCk1t^;g-9!s4CZq|~3h8Kh5kOLA{X zk1f6BsC9HZN^eAUjCg(P;(B#8m*{1`w79M@4inm-Xt^qONE<{nXOHh z@W%_sLLwVqOT4Qs;_Fj@V=bO;{rHzpfc1Ek7u#4%sriOLA7Yl>-f`lpSHP35x1?ea zYWJTWiS;{d8;;VQBA)E*;uYo=cJ9xB*u9?&uKfU6gV1msTb_=v|7HnU=kaL7mpcFS^de+qdOv>gF6!+RUT)e*;pcKj*1-)K`poi3}>L2 zM8}++()*9Er@Ip?lpWk%t8P|C#*kd@*|VdesF(yt-l!X54b7nk!*-HGmO1o0T}?l^ zjD}7R7e6E9VsfUM`SFs?%iIx@fwGZpLoE&StxJL=RO{q@PpnSEf-;*8$KL00CL=zX zDAmq%&Wf-HI7sjTMJyWTiFpLlnOA|7bsk=cpnt}$ICEYZ8?qythZahU!=~i;cj)39 zM%SrT%{tx>UG_ov*F{a+VGSym@|??SgjyoMs-`7!1faGG0SVThsO zxkB3hvO>^VkstE=Q62WvX3lkntDf2bT)4HNR%b2;#7V44q zqY!!m#sn|N<%hI2iH8*C^Yhz#Mdc=!#~iIvygcEdWeH1}iIxPijh7V`M)~k2CBc)s zkNoE2^9{1#Ay>F?BGpDZ_=^YW9y>T<9}CSh?c&i@DB9*xSn!1nS$g~Nk=5j*BE~Dz zT`^6$K`)-40`eL8w>ytD_u~HIhCJd9L6glBwB-|;Ao@Xyxn7GhIdKeNL23RoM|sNK zE3NJ6m7FCvCEH@{iS}sxhQllp^acw16va?pf4~y()EmxyV)18>-Hl@HGhEe}LC@{e zYW2rb@+vaZ`&P9M$=sbJ_z!GlDDh=^Fj56urwQf40*)*frg6bl#KR)sh(c~88fLT< zEgV~w!pG|f@q^9V&cHjoC7&6iA>adPVApEn=|wr&-o-8yimaWhhCu!zwNavD<{76B z)RFa~fa0~ZVDaXcVq>J&t=JeHc|=wMk#1K?(aRv0CE23s~cz zQ8M{G^uhUid~qPDRiA9gCDYJQY4HN}qgM(>*yDXzDN>Z={+OMEg6^-^F*x1ZqAw$C zcr*3p-^47*8p;Y7?;+pynv9?>>w)flyn#%WvZwbX8JvV6$mhnzUx`6wtFr<+hc2;U zjxrAiyq6dd)c*|g)`VJDf`vl1X*}aAkni+gwkUPfxikl|`s^yHYj<6Ta*cGkh8@%==c$~ z*s|yo5VsVHsN}6xo(i+TOPB>Yzd=-X3|hj{0Oc{8OY zjcLn_@>q3}6l!(^wV-7S9+*hi8lQE~?E_Ka>G2GsjY>R5()h4+1R z8?N@62t-9;ZjcwLF6`T7@8{6=kT3XK=(r__5m&CU8Y534K{_vmb7U@5zA^+-yrQEfrkPcS&B}!6cZFx`gaf*UtVx}>j|%j+aWoLJfS~g(OeAcBY+L@`FLwxR|3RzDX2h^ zGl?GP*8Ls`$1?Y`^B|)ys3h4C^{Cc(ZMyNI>x0KRNK0yr-xR!ijgb(5Af|C2JNJq? z7n_nDR{HaDOgS$K{t`{pLkN25kT|KXD;wh)FVmj&XkwU(2|m?9(UHALNc^TjHUoBb z5E3c_rRES4JS~zu4OKyBm8tS~vWnex-AmjmG_iJ;G|`r4ifqCZlqGUnKDx{LT|-)7 zRTraxJTjoJ6E(H6Bo&kR1hsj<00_{RzE3Q>Uj1}Uk2XdGZADH8776SC%Rncj&Kc&y zNUgurXHEB?_T!0PQW`1ti=SN4V@LmPk&gz#PM9z9^3?S008^9-88O775-`l+gsr0S zf!)&X-&-_D*h^4s>d9jSYf%)tb&A7H?S69TD6B}auVJ}HdN4k@Es9LNeSddWKFS=DD;G{r1qz-{!?8DGx zyBJZQtDE6&5jzW5m)hR1#8@S0?WPcx0ioTtpEhKx^UO4~93pD{-pWmura`nLl{&x7 z(7jcf7`OrvVJtPOpazsFUxF*I-mP!A1e@^a8%ZHlP+ahqqV7bPT*szVQh6`xH7$T^A>=?BCoYUjWc@Z7 z9>zWycN^^=Y;G6NnKO7Y4qEzWg#ytD)0M>;ctnz|Jg(a*X$k$;X%cQd+1urEDo>IY z=4cC=i1-q!nn*YAkN|wkcDzex*g{`LV#XHTh{iTxZ~%CwlN9kDf%~Y7_8An z9<#CXOGuj?P`TN`CkALbSDU=5psx|b2&RzP?q z4>5Po`pD~&=$|*!b&*$Sw5mj%bXM*fZ{G@dnI zD(oa{bg?ll@aUiV(IQ6{tzrRGg0Xo2jAGes%UGHnj_uBZX#6u7By?Y)1a8FOhiDw? z9Fmx?!*niw*#!I%m1g!KQX~LnEKzc2H^nS5RA*%*=~e|g&vnm5|Jn!p9!n;4A0p}= z;9~UsMo}wVYI$v{OwptcLDz`A|HCb0k`#O!;q>wg?js6SHgdBJT+6-aYAAr&+f9BJ z&e|Y`C}OZL1v=UpAyn4|Y(-7*5BF4(HG<`4)n0ta@V~{8_!F_*0{dX+;(fNmYfH0R zn=SenWff960E#_j4fsWV23dMjbk_O?0u-)Sp^;2Gm}shOA};0aak^muZ*t$*8dTkJ zxe`;WO(ep8)_o{MJbZn44?y81Aa*ycYF|6gKZv0UcGZAiAo z08xbvLU8IRql$#(l)X@NT1+E6!CI2agOt;9xbWYx&~7cPrm+d zE(W|Qk43NiJKIaiAfFC7zCOZXY)FWW#Z=QNsuF?h053t4&Q@hp#o`WYM}8%reGr%7 zc0uM@#_}{W|KF&3L{EQ2|C9+zcMu2+DQojB8k2V_CyVP9=M=EfD;Kl^fKX5*$R)}z zCEC{Z>fJ~~8A(}MO6jpeq%&ShZEmDUpmK%qK|Dg`@%LZf?}eUieNt95ZkDg!=9lBN z9u7=D)vF2E&qv|k9J+mJR^l8yjJ-_Rx5c~O0>K4Gqvt609qrJuGcQ-f7wxwCTwvF# zan|lm&?~`trgbz8dO{_T$4W+&91g|W|yrwvG!KNwzrx{9R9Hqo)}l8 zKrYAHKEh~0nzh!@T5s`~aJa>-ujyH+ofR~u0HNSb*WwmoPGla>U9S%=nBB4p7TYbF z;XPncjs(}D^a(khMB(99ptEiO?(4Vx^5?aii{| zR@I`RKdMS630s3h%PO9@|B^wswSetzTGF!9vTJWm`yH4K4NW3I2n`d!FFtS)3HfZX_hH{j<*>SGoYnNI?S9g<>b*k3DVQaqJodIkmuieH4$xgJ zq(su1G!&x2V0kaJK-%attW>8_PZMJ%Y)KzpZcqo*;e(g@RY)rLF?+F~3TC5$_&hd1 z1~gj0&IAKXV_G(6iY$gwf1P?2S|C}|8r@qf3EJ1M26g*J z_O_I<2p+FB9~d3MZQ|_sI$VFN7f4FOqAmc`F_f8nqr+X?)p{;2T^oK}J_6lVEUCX@ zawPFc4p>QeA{VS24u)li<*;~+x*{EzX6-=quIb#c@$hZL#2mV2oXse0QZ-)aSRk`! zlqS1`7|Mi;2yNm}HsmnLX-}=O#Yg_1cxVc9JITFK zMZg-XqCGKJ))58m`L){Uwne?LQ1v+j)coF~MGq1mFM-@bDVP{T{~njSBg|D#P>qRx z(BiLh)ola8cp<2=`x3uLYSh?gnS2x_bKt?;!UMnMXuhDmS(ZJwO#4k+n9puX=W5C% zW0>r0b2vQq5CrKWZm<_ezb-RL7i3{p)i^JtyWZm%h0#pts%&c2BPeDa?Q_7`z||}3mG- z{84G5TuPQ$+DyTF z>HJdZ{CAmri;_9KlDSst{M(W_D5cY`$zSL8`85-hnxK5V3+m0%M&Ox{4;fc#t^6Ic z@?5jDOMGASnA?)g&49LF-UZ5VzJk2E@z2W$Wj6;_++WSmi1&1iAsA_b^l5@|>4LmU z=8=?#Od&l6Ffq&_p_zg{SR?YClubKm>jp3(GWmQa<~G416}>+`>K!zy|4(pYt_gj% zU#8^w=M%QkIGr*Xogdg_2%Mz#(zopxLwLJo8WFf3?PPBJOkpTn_nP%SK7K_D2Y4p` z<}1Rt0qn85hw|l(536_eMIK%N|7O0fsrpOf$Uj~5Tv2=2CjXJI!rZ9w&=`uNUif@U zc_t?7ctHxb+o5mGy0zFmV zV8iI@)`vOTXAE?Jl?I{@0=ft!ZLAOBKREXZ2{-7#{+rt-fJ1__q|#ZM_-D9C7dlH9 za*zSLk@hX(rya~O>QozH7aCz#8ex|jb%YvqV65!*F>^4Ob;X`>;w?UMmOyis;5bXO zJ4&-he8rqbYXp8@HMDrPBHI_Qnr2V;$ZJi1bm z+ZnJ#?g&fk=FifN3&5JUgqsRq{>jR3LIGtiaQT%cpz~PcH{*s8Y&AKyE%oe zF@^2^N-_D@U0L4N`)>Y*mRY8q9ePk91U`W`=w+h2nz0t$7SKBno>)IbbKX6b)I$%% zGx3{Ft;#>Wq z%{?}hwE#$^$TyknF3lJLF~8eyOEBk;;g$S%K9Y#vBnl)+Hf;!a-|AE%WK<@?A)R+D zGv^iG+^WE)1z%Eikd3UGof=M5Jz+7<4diT8o#;2dMlqJCt}LssB1aBYVoSM-Z7B-I zJ$&$)2nFI-D@K@MzthH{O+}$Sji5|NP(DzJ$2dV*ZkWOe7pPm5{4AjoD6@=Vh}#!r zf0`37C8<{sL*4fK>}>57ay#WUf8xz;&%QWvamFQdR1QuPogR zFc2+_uf_?n-l8zR5GB%LhgsXC*yPHh-1U|~6GIN%g zxkx*4l$-!5{pA7}CEfJZD!8+H-*lCR03C7NSP}KT{Uq;#h3{z8gaAEreFqW=ej9a& ztBu0|wFOdqObPFukiqmaD5zT$)J+KQrE-S&nV?AmYs;YdniK%6P(yaoD7^Ix0QnIE zy=Q2?lN41z0OA6`&5sbX!b^9=0D&|bFPp*(V)g(*x`c8@;juI=Zuyq}=OJq5zuaaW z6UMk+UccNMQ@*c-EV%LkKYgH-%*~APog@tWcAe73gw;%SZ}bQuQ5Qn z8C3Lo^HDil)rfw6aZu9;s6!m+*$?a9uNl|L+20R)PJN!El+ulCM~1IE z-9j27J4ugFlTal9&bl9pUV0MGoL$^=-FsxPI_5iWd^jmZ{6*l70sBD)EB~x^<)Ma*j&_j zWH{?j!SU66-f2r-6|Xq&!cb_PaopNiw|6X^?q{1<@Sm{h-4NMFNtZl>5|~$1W^ZGc zY$^TBafJ@-nl&#p8qjh|!3cTj$nwQ0MLd=)^~1!rUy#}6iX7?wRuP+>@@IItc?W`f z8a%zi4($d*4gmy^5{A}l{$gm`R?)piEvPfa-6HH(@tV;CqxPt0vIrtC{h zdxa5bF}6T{5eNZ}xc=#WESPZXNzuguh+svziW;~dC8)9Ny%1~w$I~E*e0DMekel+J zPUIvZ#%{%ZM6yEIHfbH9Lctm_zCWc5pY z#l=aA#}T$u_bXpvm%ruow`+(X=~M-V=@M)sc8Eq;u*RsukS#FJ1JI&S;mjuI!tWx| z)nJx!Q_KbYhGHm-xDBS@CNt}F!S$#X-221a_@1o4>cddJV`F|(qA)dWY_4*30IfxZ zd8lDI^1Nj0xFEPM!Hi9sByRr>;Fr|COx*mw0gt2}M14}UkXbPTPcVLThU8sw>V>FzG({OkZvrX9p za#bIR5;eBrF+u9Oez2zItn3afm4BWb-mgr3SXq2QOVlQTT8lGLqwNC+iQ-mtPrWuM zIX1h+wl)5-lA2ubt4LN?3((c_95-oN){2{7yV|2;Kz$)Eb$+M*I|#6yP-oDuj>t0@ zsm?!E;5kAi*|B@7;4@;}H5W!Kz|GUKN0#{h43`I6+x4L0C@)4t5zf2z}h= zbmbB=TSD9&ENt5BVCeXzZsI}#<^vCopIfSo-BMm=TREh5_nbW-TkM^w%7j{x;-}4x zcdAEoT$;wmgI5sV`46hTv1zFojCV`vwU3rjEFzrW$+=&G(S~X!d*`AsSK-%@3W1et zRxRJWFWez5_legIX2Aem4>#!L@@D-CdDR)&w$Wj&?SGcFE^(NM-S0z1)2Ylh_=n`I zLSUxTqVU``alR_qwy0A8Dj z*j7X)-o1dVHi*jLddQV2x`B|EXuP)P@;aCbj*1gvJ^w*uWXY1}xdchsIOIUtKihHD zQ`E^xW@@{(iW0yFh5ok+eJ{Ha4eqiz4ah!#K0u$7>LSlbs8u9N54l~PbCvH~atzlY&V7#y9@=(s|1}l+*su8Xysz_L)Ym~R#TW^oTF(l&?*oSxD2;FKhe7CP@+~v?p%sY$xm~<{ zPw<@`NEG_&_TNsrKRfZ6|8Uo108M@88ytTgH)=gnc2o|NqOUTtZO`C;qr2uoKISH( zB3nq<@NuPz>2A--Ay|jw<5U9jqu-8}Pn~hu)22_xD3%8XmtHt7q!avhM}VhJHzyS~ zm#^$XdJaP!(v*|dP`mpt3ht$*4N*>?4O9AhBjmcWWGJ<%!KFB7MLKKUVW4VQI7_GB zgir2y)DSs(|9Qm9JUKH3#_8d4^$J5v-%Fu5W6z2~moNnW%118xa=cx_%+PDdH6yu= z_&ll%OVcZ0DS0KOd~rDAxa{|B2*t;_Kf5Jj3RdOJ>YE;LjM8*QeJWMkyJawp&pt-g zohvcI%(W^1gYq-zNi+$=#b>>uNNVEH33iE?z;aW>| zWsAvS)X!@V_0h2UxWmpS+j2oE^6Gd2>k>^wXo6WFg4WFMf(@Vkb z{b&{{l+YE8zijTQil47C(R0P__0}!4U|Nhu!}=uTz*vX;9^$kA22V`~Dc)@SJ5~zS zxk4)MbO>+8(L&vv58zHM&uX0NH;#jQ5;s8P@>1X3``W}#zM*;e{+7O0J0#m1^#abI z^6q)SFH%>eMWc^yRPPv6%~!{co$-?sa%kR##~D>tB`Z+ocP8YE9g#v&sDrxU$7RcF z`Xj3e0UsRvFA_Xk4EnjDCL+fx1ZKgjXDh$spnLq0IAxUuc z;toQ;rnMt%M*U*j$VL(H90c^7M-F;DNOyi}u!Fo%wfsigwE?2TPhrLF3K|4myuCZ6 z{nHOKct9?1)=clZ*=|LwrcUwe10=QbgJm=qI=8AxqI`9RY$Z!Z8a zqWMeWCSE1)KFkjoln&JFzk3I`u3==rwp0*<^B$skvl~;alXsa9HaQvq1F9HLIrSLs z(}j$C(3o!izUNti6F>nu_=2I{X|%4+Rdz>2vSPG7w80l`8BGo;?!rR{-F?oTnqxL? zP}n)gW;^)q;H&{ZLh>F20Sc~?NL{-F%A%3Hp61#DPIq9*D*YnWYLac~WG_h{Cky1m zw!4Vt9d+XdbSBy9Wty_tOV&w!F*0NS2OavUGyp+gW>mF_~#DGrFBlZF`EGa%OaZl zyW5Yzk8}|7`>5#U&#iOPa~B)GrY1_hdP;p$(*yJGlSgig0|ejggNH}7_+t=i@~!cY zNK+NI&RPDG8tp5#*uj}U#%42YQ4TcL(>izZ8B7Fej9v6oZp?K{d%#tT!@O-Kk6Ob9 zMEZ8iuw(xgo3|hAb;=pzRqL6C4dZ3&+5FSSL-3ymf!VxIHlFE}v>#mZ1DM-(LWf@y zpKMfGI2j z5SbMC)AhwjuWJgbucY4WmCEF^;DK)z*V3;$8xlhjP6&f}FS04_w`|wvEw;B)Ep91^ zdmpH%-O9UgvbPD_TRM)E4DEe|9aSPEkZi>^2=LPa_v~-bJfvoAzZELE9+T3VL<)#0 zii>mzmG%!i2KWMtQyt_o)I5d|s_Dg*m}|n*wpZ~H?{-zlvQS5*Pce_6K@IKeRic$K zTcQqdrkBd^F-xV82jf*Dm5R;)cXt;~e~%R5;J1RRNNO;K1b~wiV&SWpeN*Es=#@D{ zuULB$n$+5(npT`1em$adFL&m}s-rs6lKi?W*Ec ztrh${;J%Agjb*x#O9~3NPECUa6=L+f&F5hgyR`fWIMBL@e>J%tXss3lI(6 z3eXjAXc6eNK*G~WAMncbSq4pDvI93o5eH?*eM}RneFOr_?245^8nBY^I4dt|cD~YE z@}xDy@XoLc908mB~~8X5hQd-Bvh=kd$o$729#WVogn^%rS!1ea%69!&H|V6EtZk` z?X!QG#V}tfCV|MziFp_hzgf3CF}Aoa6i9Z`g-(2{ByI{TT5oLRP8DXZ4#)zo(g(5U zZKZc?xAF%Sg)e(V`vIDyXUawVN(Z#LGSJ{Hzl|M(LDR}DDPYs88fjvp{9~grxWUF& zY*pnZk=^SvvsykH%C&^V9B4J?{MCX)ICF;mxb`{xgFzGqFaN#goc2P!W142Go^s(3 zyO!aT<-;!IyPMa*vy|jJk%AAr!jQ~74tyMQC$|e^l;p5l@xlBFZ@vF2x6Wvjy#cf( zR|eGVe4Wo~CP~ds|8`+lYgZ$w2;iY@uLD2apJ!jyc=|GN&?-e)t7Y^h)-8kzx zgFXFc)TMs}-V&|dP*7*czMrU9F;8HA@_W2|9c5T6_$8>if0J-_(;TCOjdqQljcYbs4dwnk3yG+yfP z^>=vDfpvoFP(=$mMr2w`2o$NQ_X1tBdC+z~k$u7AyiMaZ?Tv!&=5MA?QXql&a2R}i*;8%Ti_RP4jnWnUgqFQMtXhDKfRk_ zW@kVdI1I8M5g2^wo72b*{!NPH+eAPki&9tOV4vFImF!Cxb`FeYRwMJ6-3?QHqK;Bf z&%P*UFuwMPVFscpJJObPqyHQ2(e8RUOy?;CJ@1YQvfZ}$ecHN|s+Ky0_jn0wbLxlW zr0i1yo)EeZWp09^oJV!GW9zB@iq7sDTHYf7;|8!sSfrcvrh0!mP@#0OjBZpEpnL@@(=Hj#@jBJ5=4Dl3#sK?Cka z##=W&8Ee(+tx^5UtAn$d|J@!5#`vrjsfE%@yS6l0av48ny*yK{^p9#1$I#e*Fkq~5 zC0fM-h)yhI_X{*`6K9Q23cn1bqEl7$lzIR+4$Ur63842r)4R~CD-3EYD!y6LHD`(# z-P{wH)$^#%?Dun2z6Dt*FaD~vw^_sfd=Q)CyEI>kJ2U^Up@39@ypo#TIhm@MTQ_B{ zx3eyFIrhplZ{b4-WgB$O0UWdmFX}|W{l`b8Jy>I$n&#ZALTCPdYOkrM1iOm(Wfw;L zsQ02&2byj}jtRXQkDWU-?WVquV*lo&XC5%8EA#vij=jiU(3NVJneNMOl?n>)s^Hrq z^>d?;KR}3Ej8xSRyBc$Qb3%HnqbwQcCx6$#61C~LI)TBZwT5^N{%+EY8G5D|veRL` z7kN5zIE?+srpR&~MKtSr7SOsYBJPhduE+ox2PMTxO4=+bFGA`tKLFIJR1m2X9+O;xLB23H+{U^-79!7$v%j8G4BchAD@~t~j^B}o`3DSO zKn-9HWj+deFzS+Q`D4r zOAlE)Ip3yQSmLcNDmZEy)1iK=vS-qeOCwz^*Q}neXjR1B6x@g%PyN;z6Qjo-zKvrJ z1bM!+3jDf$7aWqX;X?SBDvI4NVJgno$PPeN^HTzzWccA2y6YjKTO--joA?G``_Ed) z67ogHG3GR{(Lv-ih~iQd*93iQEg!5NtmonC829#`J^9Ev*Tb{>E0Bq}nEjNIOMME1 z2Xv$OAF6l@wjJH?22lPz`se{*aWI((9rCErK?}xini7FHEPckz=3hv?|BQ4mfY@kU zV}5OKYK<)i^7qc9`YtFV(NjY`b#$G5deSh?M6C#l7*(iCSL;SnQK~#@RvuZ7D~PFu zbM|!(R~mU!lXQ$=$&CeH(^no0lc#@~+|>nAl2=(iYIqkKCre)9!{~c3^Qg`jk#=(6 zOWK&m-2q^_grXz$YDeExt)eTJjdEYYZ8?TkWjHG#@-Y0%Me)Fl0qQb#FPmuNo=LHa zay!+dO?7FOJjYN`Yq&AD#rP>z)A;Qw&JWBR?x6%}9Uy9_;&)wfdBSBh-lUHqMsr}$ zPXa$5#31!9_vrjga&3K^yb}?j<^iMi3j8=xXC)`Pl`*V6-8Kp(JZA2SE?Y zHR&e#TdG_Bl}hQLr04bB2uR&Ep9#c-X%7k-M_5-_y=-`WGNu}+d*!>|?S-levG)u4 z%1Xd7E7$ObwFaTtcTDJYq`q%l_hkT>`z3tK7dbTzh^?EyaGIcAs&{Nxg#4~iv0Z<& zQB(e$xV=F!SaMtjfUTLNu$~v7QrycWElfR;I|t*+j%F+7a^z#VNbm+&-UXnp-U3M9 zlgRV#1;jpH`&geT@2%YvvCbu-IaANJ$mBv;xL+Haq^W(}9zrnRJx>d=Vn$vL>DzV! zM-jaid*`&#d=_NrFqNJS{Y1XNvbmg&39|- z?$|oH0%;L$o=rKWpFz8pYeeF>`4soG#J!f^s8O_{5U=c z0zCUG0adHLMX{LK=+TSeD4gh?kW=K(?$uh-*$~_v%bHkM=;4ltC;O$*cL*u6+4~hw zvUHFGpu6l%p!2|cpZ2cy`_Deh#Jr;z+-KYMIHMHacM`RH)<=;ov23s)v*>m_q#{|>9?D|eej;y15MVW-q?i{q;F24bEg<{*|bCcUt~?u~6}QKO1)fZF>g< zd_AZJ7wtR^Or2w1h1Ph@ z+liFf?wI^X2LRF{JC_Dr6p%&Zlib&c-W)bQgWtH`+mf|e#wTOP4{GM^Tq%+L8Qxu8 z=|1C`)vTj8(fu~(G#q@=hm|H>-m1lAZ+zmv-L0M1c3HOyr`ntw?scRvh=}2@Kb!K$r z%Mc;qK8uQX0cH89kjY}o#=YfgAx-viLIQhJuViU52c%^A**O1H#soG`R@rnDLH=`*OslH_ z#`G%x{pYwNQU~UzEMeDc$Lci;&Yb@A$1xV0c2{-pB_2ah)4_txr%CCV2~*3;sT2oI zvFdvtixxYh)bJ4MCZ((9WsL3v<)Io;)SCQbPaGggTV9jeUZoSzuX}07rYVq^j176+ zg;VeoEzQ%@5CWeO$rw~JZ@^W;2C$UpwXBF)6^Po3To=4>D0o*H+obnU_@_1kTnHJL z_wRA{H@D(xsWSTr1)`sDDi2i;y1N6XYr<`r&IHSLm%d08`8cFNpR~qsrB;jvxJ8-C zXP+)@;@0-~V|=>|3IcVYY{-6q=hmfCO*%5JS1o(@7jx*}I3UW}^@2gMvyI19cK#v+ z{i>uMp*SBB4$kl-UHq8g9IJEuS{Hf((e-KPan`Do?L1A8u!-Bsv^6iQsgIQ#!{;5G z4_2I5z=Ro)%(Ue+ePT{d`Wbb(@8I>*GwGDu?pKZc2D^O<(?7Os+%Brx-q$SlmkbBf zo=HFZ@bp*Fe6SUUCL8A2sU4H7boQ#;g0)BHlGoS&f%T6MEF0gisk|RL==fwd?FZmk z`F6zb@6A>}ybnZ9PT1iY`Sgx5wN>ny-x|*{Z(wVxiYVX2r#!Uw!-E?Qjidf}^$i6ArKbM zZUi-=?z(vQhUJ>9RBx{l@GQ%Aw-cmZJ0QM8wPcRd;Qhv`E{$60-{ytq-WDrOW@P6c zfPQopn_W=-zjD5GA@Jr0A)f9!<8^64>Ttfu;C1OjUU8TR zcuRsfIMa%QcINvxoaf4e4qWV;5JoQaZ?NFGYl5md+q58i=lbgqQqFd7T=vMd- zpX}t$j#d`Cs)C$c0E~E3^Zhh7%o8`a?)TFl-i^G27tt7eZ(ITdhsqtN zIR|eoKTGb|8-S%IGYdoHGThRhGXAmL+4H(<#E(^FU9C*Ixv@Ib z!8;u(_E8Yg&XNw^n^MWFF>$*z62J?&9kwa~Xc|H7lLb;^mIsfY>bcv7iS%g0GQ<%c zrlNY@c&S!chv#tb;_l2A%{KwE1`KB7bn~b=%*JjH7q$hwd~&Qe<6mO1m_QRhp^8S| zG>=}zO1X!E0Nq)q1d8>5fF+n1@IMCViEL?lXfo0M`dhkpmx0c5PO}5qUW{`?$cZV3 z9e84`=}@q%`}s~ty(r9m1vwxu%ID;ogl=7=9Fn#KcL^5n*oXKwT8!4=7gZ|7Ip8*STuJ9n-3nZOca zji$kIO`5wQ`u$~iMmw6Odh?N>@xyZaXUh`AC$HpmN)wh_SW<7BwDGyeNY?}`(dZHwG?HkQ8XwMqE_uQ1OkF< zgty<Q-m=XpgDyXTtjW!R!sw zL*d1;%b*~-`~f|YcZS7@E?F$D=E%dcT<`kniH}9$^fY~06qhj8QLUO?Qfyk6Wi_pE zn3AFiI~UH9A5(A&Ykgpn8k67@RHWK{Xk_~5xi!C+cyD=XQqqTDNDZWMy<7V8wgh}( zNA#yl<7q4Y9@c3xfS`4K$4H-Yd`8T9WhJT?KKx-;uYCBUBrM`sVsW)_kTs26GMYZ9 z`GX=Pgu877WsIio>#I@0M6Gi0Ppsyj%1LIvBiez6mF(*U#%ITSF7aibj!ZnGEdxg1 zM(28@O=>$=FCbT~u=IA~YI|?*D+}JG!Ypt8H6;)AC|1`iZ49z~dgC^G=D(Ex$;yX; z9*)zGrU6!ZD6wW|IbyMFLF8C0xcU>)dSJX9tKds3@9`4TYf|5=Sp!o!7{u{6C-h$i zDS0`3SfZu8q1cT20u#_U41T#)lycNwioq_vw%z{&en5f0p_4@%LYm~jlSLeOvWO!? zNPpa!FQh-@%oWllXO55#RUxD$9-ledBBU2PvFs2$!Gl;8?cbt3g)E?@f$#G1Cl`Mh z8c|+|%iu^{Kq@}67K~!L1N8U2y&yJ!83tc|hB7P#|NiyJ2zb2c%#?ym6`I#)duWq5 zwB)4;cHWX5yd_t$&1cw{tA=Dsom}H@1-9JJ+479W7V922@MI+XNF(j=`_RY~_8H7L)jJ=i)s*q&MSHnh%5Het!!&^pB>XDvB# z(5C*08tE<6NSO#Dhm-fJtGAg+MN_!CtnWu9J4&e$2Mf z(yEH}>{WNup9m4041VDa6A#ta;kNWgpG~q&>`GUvE1>l zC}sJSI-@}w66- z6eF8N|JmycYoNOCTWTJ~RAWXlA6%~Q*MH5(T&KzzpOcilv9IG&r zlPf-r_vqc1!Fvzidaysz{*V6GlM@Q>5tV6nV%aWHE?3Wwv0A`-q=--BBGqXbUd7J8 z`OPE!5t`DW^Y&aJdVomx$B{^XZY7bvIvzwCH=(HSE*s)J0q{d{#lPI;mCr#LpHoa1 z4dJSF^IbzXvE5J8uKcbc+_u(MZMn!@LriCN330xI;aUk5P9+X#JDesy}6~CHA_Sy|xU% zYjmL0pO;=0;}2-=O^&qs+;^z=0>~8^<=(@RaCq-^gvm+oH=}(S+_xRTN^BT&$HHI z=%_ArR`JhRcLc`(czd96fMV*==wkt_ePVnw#hb?jig^QA>a>y1%*O_zE;Jt>C?;oF z0>3PUo@OtbPOyX_iY4Q|m`7zM@;)WuJf%K&g6AZW z=j4Psk>?~OflVesYNV)6;N)SV=YS12wlUYa)&WOSRS?I82Z!8oVZC@t_`ch z!p9Q*t#Q_Jt=Vp_86p>Ry7aACxT4H@br}Bck5brj7wP=(v|v?`r#- zr?}Kz|eD_~p@Kfsfc2XU;zNz&$P<0IMZ``4fp5E0#b*(pK2=ixMzX6O&!_R1M=7nQLm#o#O8tL_p? zYKGxK^{xK$l|o=Y!1Au}iDBa8&4Pai9Tl5!QWSnuEBM0@!UHNJk@$rYaPyBRuEHI9 ze?hD7>Ryj!bgw&z&3tcPyrpeAHT+SsIhBV}eU7QtVyaX`s!X?m*DTSc&^T(os7fgQ zHJLI&R9vFJJKof;MK444oTjs&iSm5Rt)~bhCb{D2*uwf!-sckFAC7o{E|jt8J?-@U$nv^EDsCI z%5CL=hj&AXtm8vHwVyYv6*$5*3G9E`*7bQbB z;;k{~5dpVd5f2{C@NjbE0UIX&D4rHCV-^Y?5*1t&k8M!|sZ;M6%}^Fon&X&8C=QTC zT(r)kEU;(YRcJ4qjTYhW3i@zeHYm}aJ$Sy1r>dkHnf1owiti6ai*{QI(>%DoN-137 zx$9MX;jPGw8?UN>$2*VouDHh|Ct?iVAV>BF%tG1dz!~*CIOY%we!>{{^y?5iN;i6JY!EID9 z+`}APv<%8sg|z^RE?NEEL{<|HO1cb`(=FIlgA6H!Ya^TQDqJHh#dBKPuD0^%X*L{< z2A}(fc-oTUl3YhXB`o}b2u&Y4&k6H~lR9Y6_5?E{Zhvz;ZafLjGAe@$tAU=O)Ib&x z8yVB|3+{{NZ_(@6y1?I+3)dFh_!k_sfu@F8m6T^!mxM@;OEN)Ow%@XjDJnfW9izw1 z%_=!06fXn|?tk5GDg3bQW!j8)2$0y7X%m6`v`CdaZDQL1d_Fapa2PAM?Oi&%Y1Hj) zCaoGIt(pcQ^~uxm77az=Mp2oTqTH7jsh00cQxXEo{0#YIdRU|8h@+M9kv$yeWV)t_#+}U!Pl(N@4lt|6-Naa(8-%^@DAJ?*gP>Wr%+M{ z%xe=^cjh0v$mk0y8U5QKv_6am=Pz%PPcVkP?mSAbZ^hT;_r<9& zg2yo7P1NCt>(S;)`CLtfJH2Cfg*&z50K_e}RUCAC+U*tY@f}Aid+qm^Vz|rda~<+c z?x~pL+E-KApXYH-s&G3xI`G|AOU0b@ef1UYq>di9@O}lxopn^qN!_=XXmome_EpRo zzpu5jy5mg6!5@0sEtQiyDk|nU_8qG{*>TwGKa^Yfn*E2X@!eK?#hj#lrx4s}of(|? z&5Eh6eLEn%$C-?gjSiwNR!mLbx7#O7>U?Q-#njY&2N0Z_Q!#b?KIpf+=uE4a>evT; zk!PG4n1>ha6;qS;^&piskc*RuX`6+iuI?aeU)c#|Ww_%k2KQSDErQ1x$=QUQ@B;H& z5Un>wLmP_XsngJQVQ`OKH=&tqhpRnK!KMpQEr&ujUgq~$*oh_xcI`!h9SwNdPQ zHDbirV+LZAfLI?!)>w&qujwQ+FuLBM5fe3HIY8_nRa%ZlteIh-j55KAoy6c)t4=Hp zi1lOSPK!<^rQ+$yo=rNL*&3N#LP-Pw84~HNteZbzix%}UMyVp~?&xK;vZv!P5lYt4 z+ba$}DpY{9KI%DWNBmLlJ>$%0wbW}*A|fcX)kwvTf?8KuJ-^9@;<*QMJ%Ezg0{EK= zzYV3b4Z&}s9G(Om-W31&5yG2RX!R^tm?0({nn37ehyo!e#Doq>XK3g$h2l>ve-o}8 z4OedaJd^*0%!{vRec959&<`>y+}?o)Gb_CV`*~|mA2MFlc|2(VSP9Xw0-Z7>1oU%U_+nbm41` zN7hvSFs~MXCqn@22Vib307n3LsTqI{0AB3`pbMJfLT6eV9xlrC4r;5eO>};et6uc? z=L&D$M2<*;^FionL1*g`_pHj^1ymrg7DaAl_DCss!*)8)^5r7$mMukKSt{H=%7Z@C zp(3g!F(_k+d_ixf@bCR(G|OJm`lzKDpqnaM@3*LA1~MvIy_QY@Uvy?x zw7vlK-@mxPnN!jFeM7!sdA?ava#c}zzEf0Y_lm(3Nv`dR?axB4zF)*@TMEvLpfHz8!SAoGe7WO` zsBOZ*GtH=_cYH~PEwy7G8MpCZ^}xVYIy$~a%T<}wak~8~oya8`mHRdO-A-Td8K>PH z*;ZJO8JXZ@GTqYgEv9K&I`ZsHj9WT@t0czX@0gy!#JHtn7FWY9Ai+#VUHiX4(ftag zwFy4h-;6?ej0xmxj?u_UJF+qbS($;X%tTh^pfnpDE*fR2098*700aK`0%%s3xIIgQ#Yx;q!n1TM)JwXx}5f zTgq`DiCT_&u$!91LT3t7flS+l=k35xCNo6<)T==8FlE3Pit^1kibBrViIUEEN#ah- z{Lx~`X$SlT%$;y|R#wm3MShZ{vaMqm_joqF?Ac)f^&~;g4|+Y#%*C>!@u~>B=CxsD zr=uc#e@7eGy`G9qFL-v_F@rzM04#NYUxMNvba~Fwm+lm>Pv)|)BT{#cq`lH^gRFA+<&Nr#O^7>S>@% z*K@L9F85K^Id6&Cm9|+uW?bpf%;ZpJkku7Wr$^b0CU_^6z{2uA8A-!$uY*W$a(Q~} z=nM6uj|rSP4A0Y*Cy{OBEHRED>yT@kCJVJTa=q+O&E=+W8`ye``zv*LsW5uf$Prg1 z*@h2GOo+#oXXd~1n6Nc2HXP1`LE<<=H);8(){@^-kSHybKf;9 zZcY&FUORjx%xVzq?i4X7SyOOvp+97&jNs26!9Egvg-{9xD1VUFW9=;5uP4D=;;a1) zh`WfD?~?eN@Z%;U{wB)50Y6WIrNCDQE`onn^M#9mKP`jHfj`ZR*v^>5dWrC-8OUB1 zJw3loErZL2KL&cn#Ci$w=K}O*sYa4Cs6D8Z_|tZQqT-Lr=vNQ*SxlkvM}y>mUJ!H}Z>RcU1$YtXP4XSxS|!}MEy zd38x?L-bq4%Dw9HNnE^+G0lELy}#;8`O#$JN7F_5aj{AQ;!TvoaI8G;@1So}Fb3o_Xd z0WGYxFU}E)kMN95&&?8MRjUi|5)j8Vn3Ey1(N+_0v1s}hg;NZAO>RmOL`2;bz|{7$ zc;3HOf6=4iZu7^RT_AjaSFee7+f$}ArRv$ka^H^26@ZOgwg5bY)Zlp>iNUiQ zX~8o>q~>OUG9gL`RmV!*7ON(8!kbLqQ1UjIByYPWd8bX1ca1eFc@jV#!9G0~1c_vC z#968z%ycxzm!DnAY1=JUqdy%ZpG`6Hd7zF{Ydb^o_1Kv~^xGOOntZ7vx-Xc&wm(@0 z%X8y}(?`qG{)XWrkl*Cq%Vq%^F)D0m2pJ`QGsF?wpH*b`5C#^rc z$9X5NAxdzwb<@G|m|{G<*hNMN_-N_zrkA%{Q23xv230y7_zj`^>~7qrFslLJX(_s` zyqxZXTln&%7^%_CrWjmo_XxAfkucQyFr>R%48CeFd@I_S!GgO5$9OC_lL;tkm$qMy zl@Icg!AgK-eEtqr#>_i#lf^DDCSX#iNSC4A+~L-Qb;uMfGRrY!QcADAFDQ*k@yg}q z`7LAaG2^TR{$tjHLe4tknTL1=Fy~wQa4vtB!9Xn|Cdb6U!L|i>ToAvnP4RPv#A}3o z!N*eEOU}n3C8QaSX5KpNH!qkao~9)CgVwU?rPH+-IO zLF&zyqAp>>xghnefpUe-T^c1&P~D#F-?!+E5Iy)#xwu_l~$=;lVko zzQS&(HzS7P+>3QV-I)bHxFYO1SA;#ceUxebmRchka2f&rVfzXts#uqQjoaROi;c}@TvCf5mo6XZJq(8HXk zyxzh6GXF7~;6HtqG=@zm4$(xcw+PJ}=Ra9A^=L-BbjN769w)5eR z7VO7^VZT;qng<o^4v zc03lpiVpfP;mr}uEq^Kp{D8AqRH3++`8dyE7qNnM0}l`OcPaZ=)JRoKO@oCmY~j`& z+z2cT6#+vO-$sS;V1EIqg7 z?v)9YnJC4AQIr`79-?02tdQ$?)afCI2Qq}NbkqkI(i5EI1mTWbJF859QV^;~CUh-bEtsdSK(p46drmD z|DxR&%rv;D5M_fH?y}W^8WiPi(D6fO6)z$2w|0^NrG}FD+CKmb0qBRH{FVIHNWiF*PGU)-MqPEJzlE9oTzu8d8$ebHx-J;gi%II3^2@>v0v_~P zsUbt&NLX5!o(8B5-;yJWIvXNRVg!0Py8eb()SZjTWz+|rvryA5W}=qAzU>5{`T=&m*8spEh-Pf=mg^W-5Hp5ej#0qUq=;op1yPxMtOpT0(7&DjH22JdcxQ1E8C1g~6r;U(IQoidiVdAuH$tljgsS z&ON8J6RiaF*4xg~E9V{x+HHZ&Qr@Z8F>6Xr9dW zx5;FGn@sk%DaQUbneDHMTL$~vWVXLeX8YS@w!eC=H2bSnk#2tv46?rmn3xYfUyNMjY0b%6yxzfOshp}j@s3Mwpp?J^)6ctj2s904v|pGTlb?&)>lOP3+iR}rFJZK;rHtS2;G~UQ z6E(}&@!ffO>_koqWz+3K*=REL!vfj%Ax-@?S5QSXp8vmY=-GUPVLh>^%(jWj^LB3A zaUB5|XHb?&%rbWYlP6>Qsm7nabvUd}vpO_W0yYfn7Y?wD!JjOPw+DxMEQNRYgbj2H zp*MUep=+v;clS52y`dPZ`xBU*k8p<)*(em};~C?@T&pkme7rB1p9rqRFhFt${*S|d zHzM+e55xbF1ll-|+kWlPiwG}1MUB=^{gIVsp@*6Pj}Cax|C9}OH*kYY10)X<+Jct| zF&oHQ%--Yt?lJ~0$RscQ8s5GKhMIJ`y~VIbGbu zpvReZfwih2gQcYSyY3b8CVT^P)r#OPHzRn83@(Q6z8%56G`SIN_ZLU1#4eLUz30TiK$M5CC2j1D6|$S7+=4iJ!w6djW|$$TR6gRn=6 zt$~_j-^E7_kbY)Vl-sWvay@M6kmQB`f%SIOhOt4h@nveqXsy}kvT;b+Fo+B_c%l01 z*nMS{rqvgl7bkaAXESkli2O5N4^etzF6jHAKKd({eKIY_SyDfkF{X_nhW6Y+#;T8T z0{Eg&{IlG~ZUO2|7NEdj0WLc$F@qztK9%glk$5v(+uqaX8@PS=jln)(5?V$l%-Yoc z&oS}^X%`smg)uE=4W1S4cWV4L*^f&VNs*cV+(=?Y3WD7q~nGjc^1WN;VP zXhE9H7Np5+K`zqoHCvFeLt2pkt@X?%`*FZzKMs(ZJV0vl0NIZNWIqmEj{Ue`rCiEZ z49aWNRtzdB)4Ss?G=KKrwM^e=KMvFnX3VARhrx6-nN3F%Hyu|G-akW$O&QkhN0ZTh z9N10k-RM-dSxMRe9~g-&Jn-QIHrR(NaY57BUzrP<(GK=iu-}A=o_BPFSq-eO!}f{9 zj%vckwD2f7gsK(*Qn$4A#Uxdn$ksvX+u@Z1u)PJFAU>w~$apW>P_F}=iXT> zQlQz1S-}&<_K2?fwt@)TFNv8e{2o~a(3Huz+-ZnTm|I_9)iKw&^#?v$oT9I?qIHm7 zo1s3U{N@ zQgGCif5z2MUa)ul`NEHfC*ToEEw~5@c$MQ`(ub>@gyL_qWlzv^_U0S=J*59Vf&+)1 z?QCJ}T)r$}W04}V?_3|RSy&b9O`yvdhjRIb6g> zSSB@9i`Mfra&}R5GS{?3-d#In8;d11m#@M1HkV!z_RcH9o^)l%?N^rd6t*zt^6_T~ zZ+>w}9Uem@^U8{lQ!wv}(C1zm`uz6M7u;W?4ci&W(vUk1ryDa6T`yzE3a*IB(M`>) zT$&7@%@m*49f0at5*$n`;s9#kZ zh75;fn`!+|wS0hmX$QbDyYS-=ENlk= z)Y)DZQ*U8A0U+iei(!j@^D-djYZh~(mF)_Em^1J{fUDoHtJW)g_g_&1IPi%ptfO8w zQliWJxV=c*dzqt@K!0c3v6%4*ERFh4m3kJx)Uyap1T#lt&i}>RKbBS9ZAkysg^DEP zeTjv^@#gjgn*Bx2s6XATf4}SUSR&+o#hH=Oo^JB*F4hmp0)IsGw+e;H+!<(h#n@XR z9jD1U>cuT#a!}+R@KMCzL?`+zwD6jDdWM`}MStaMoBA9Cz;iAM72Yof7yhH2_9_d4 zW4Pd;{mJOf8=Sk=#;%aOQbd8r_yaO`yodZ=?^&eOQSf?oYZSaw#gl6VhegiMafRC6j_3W`yl+>bBSlBxL+J1KCa~kS zK{n!O{?Ss2&S}M5;g!=?F_>v*%jDQ7}RM_b6yn zo1&myeLo7Os8vxgQ(Y7Vi`4&$uMV!j{9Y{RU;soozX1=l2QOuQUjhSYKy4@02G+!{<@^-L%e$uiPQZ>) zG!T|)*F^K*m}m-)2ke|RBY43;Qpf9DqV+sXXybK;HCQ$IJ#N)_S8mlrdAC(#;~%V= zREMn^NB_mD3Ha}=8gq-RoTGN+C@qsNMSj}i7(Y|+m}Gp zBFbbCm{$NsBEL4m^x>raAZd9P%mLkynSxnqFM)i!LdI@1>9;MR0Wy~Q zCK;p3<=V-x{>MQzC2Ven&SV8VmeQj-BFqZod_nYm+A&Omt%WCpoSjVo7)nw7FmEd1 zsLbrt_YbEv2%A;#veJajd$R?{Y$`zef9d-X?UzM=b1S_65*|zGQT;n%R-IY5^JmQY z8+f%$^dJ9X&)0bC0`bx4QKN;C#{a}&!xN1ER{m}D|N5AJch-Y+Vu$yof@fb$VW)Zg zcb?l}8o~hk=)Qfp`}U_2+_%rPxo@8}GE4a*EgSM5w=#VolywSaXN9u1EM;tWmNH== zOL3kM%6f#d^FrCNtlJ;zj?cRN@qzd(B>|$l!){w=!#_P!I|1p) z(NI6r36HBk+y?K?0WN#E-3GYe^Q{INgaNWO8h%FJNUs51vfHJGkHYV%a4Y#rk(Qj(GJGIdx@AAk8+~H%xJ;4NvJKPS>k=J|DcX`qe zxR*2!z@K|b3qF<{xgP$O9K~O^T<(?|saE?uu8;>ZxdCy8L&+QIxhdJ5`bFwj?43tG z;tto?BD5Pk_BuH6R&^X+WTZ%dQ7LdHyJf3e zel)=?du?v{{*jXMMy;g0+8`+}HA~7|Ra9Q;6qP4?#o#vw<^QkLk4&Kr(vLl)FpdwY zAKNaXADOz1)sH}isYp_|R#K9nA3+v$71=_Hz@Q&}!7@B>+NdVS-U&Ywz6(FjTS-Tv zdV)}b)_88YHGI?tO_Hle=nAtouiEuC22B(bX{yms9#caETu1EClA$G{BzdbOw|c{E!zECZUeI7ta6*!lzEw(Z^``cEQ_qU2?PBUNudBi9+Ub=~ zdF3;{)N)_yeo5Xb$;Z5)VrK?N!k5S4>AYS_-zlXZ^QQKBQ_n@e+~ak98U3=xmwH(L z((4N0hhEoK_T9|j*rb`kTby23*z5Yjo7(S9Js-ug*X#O9a@9(%HlM53=W55BPURY( zt4<8Jr}$i7i{WGNC=Z8hKG**6|Ht0Dz%^O+|Kl5=PDMBJgn4M%EG#X^Ls(i&LAKls z^H_PnRMb)-MYB>t(eiFmlvZ}8va++Ct*kcK6jYR>f&(2O%0UDbMO5JTd0*GPv5jH% ze0%=S^LqWCyFr#6r%@8Ir%u>=ph+%5v`UW&btZID&+CjeYUD~d1;FB)w z9l88Ys=|-0e&1eWm?qN*9W=sp8Js65y{a9V57C+e?937INR~_egYww{3l`_a19nAb zzf!6Q${)5dF`g_bge!)H`v~FPhlR_8aQ|W9zCw841d|j$@Fg^KTJzq%@Q6+@2;H3; zq@onFWh9A$Niv57nywI~=@69!dAk3fWr=)KAzq(h*u`FM71aDdqVG>SVXroE^ya7}ho$5i!*rR$ zP+c_wxfp(nNQ|q+iw}-!65B252Fu1!e^?!;F)Rw!3;WawXQfFPsI-O-TI#+3i_rg< z2#xTJ(2SVe5{!8iRtp`}0tH`zg0Da^k_<23v;k`|1Q~UPH54Q%>?w2%0#9(tY*KbwHL;2SjSui$MW z@(qGaN4LB&g&E3m2||5i>_(cmo!ri?fO#w zbrN6tQH9H8`oV?z!KbTw=?CYEr?u`*(rUZR-sL(RA}o{YS}Ac zso2V0XwqOIOd`zk5%e-)s&B&R4d-q|L@LRBHdW~xGm=~S9wQ&c*=!C!BPltHZ60FVm7n*%g$2*xC@ z+J;vd2kV5JbizWJTgjw^J=}Fz4rADU?j_@|7*#g>!$+#a4c-<^6hP@78WMjIXDo z6;ic$e`OYp9y;MLe&rH>`qB-~((pg$P-xTcOW`jR&esXW6%$(4SHipaw9>ug5_rpv ztfd*DR~vreq}P|{Tj_NjH(4GQd-I<d7h;(EFybCWx zoA`hN#&G|?gN8@s`zZ0G=)8<}qx>oHq|iJ&K$931_>RlJzFPnPYwx^y9|4!wtl0>h zed?(s7^28Ir0mq}GzX4vhSf&7xfq@Z#msIKv99p>WQ()Tdx32|G zH#PMJemHvcF+l(M=cj-(t5&@UytIFR7;w{u4etO#dU^<+89|6|gf4>^ISYPi2Bpf{WAh2lo@R`7*k3Omc zvdYSa0}bcS$$*k4pWF(3o}a%0cq}MrGH_du9`^$>A0J=f)qVRO0>-H`R1EH09`wD=m@B8xM3vFCqDiy;6zEu7$Bpn>K0()t+y@) z;=cVh1E^fN@;%^&jve~|{Sp%11!fEx5(^B!@y0uVk$w6M1qzof`xf~Akw?w~k0vKy z1$I32&?R8iz4z*YpN<@v3#|R_yF z*s+zs*Z~9P0ZN4;5GZSIz7c4=aG@*kM|Sp7V0}i04)FK&?GD`1t=kA7^~{+&f#2uM zDF#No{`v>N=FrfCz#YAMjRLls&C`LYpL}v2xVCQH2H=@7W5R*7`SZU3ewj7vG;mk% z-nRp%R;+jxFh)iu1HIR;-wwR{;DcGfhlPc4K&XU`4>p4h!R2MFlieKhdN@#9YeduGor0eW`s+z72rZ$T?gQBW#u4X|AGZy19i_l^CA%b)mIgOLMHPEqJQ}% z52#(edNc6Aph3~V%scK_4g~k@s{*Ef{P8(p#rNOu176>|cM@=*y!>Y1{i32ZKzVHJ z+rWJ{-82ArZNh{iVD4|f6#`uhhUWnJ3omQ}Y%wuOz?%E+8w))7!w+WQU{us+K$1q2 z54898ZU@v~x+DeqcI|otkTQ4fhrr__M$7@enm)Y&Sp4(PnLw}So_iU1!(^HP+|#e$ zt-!7CzWXy!P+S}gR9(K@4Y+;#_V0j!etyG%?8S?J1wJh(hygmh_@V#|+qUf!VCY+K z{RoWi-+v&`B8Haj()ys|AzFnR~`}s2Uw?4;M`h=p%n$0}v+w3q|JT~R+2MxxlB!$3UD+{V$c=rLJ(B#;y+^(q zTzcom$yt(W<+X1fef+@hqaM)BIGkDY+0}9SVVlAlyy7B8zqfPJhaoS$@=ox@JwF~B zplH63o#ET<%$(Oldzn93H)j5<-YX*4KUg^MwxrGD&+hJie0FE6?XG(|Y1^x2rsRC_ z*vqe;eEM|S-AfbmCVskPT3zLWXTFmCvU<=ReLwzwZ+TJdO%r}IybyEW4^bNLOI_!V znEvx~rhe}hU*7JwxZuTYZ}oq^;?UH96I>~9WL*=`s{Mz$>`u|V=|3B-G z`%}D=#fN}@QsJ&jDZzsP5(#do@Sl#6*cU%d;-`=JnN;^l1h7uMU@~xHuN|qtJ-5z& z4Y>F7Ck_BNy%Tm0_~Ocp&w&Z0-dh1>;FYPsm#dau0yZC9^)hhLOE(C3uXn{e$^jQdmRd(GD%xW?g1F3he^a9rFYHt7%$G`p|@I+3- zUcmdMIj4ahue|*#@W;|;s(@D?diWTS`+LGrU`FhK96)PZJQ#Q}JuMVinIb$6)V=uh zYT%6-_dNmJ`ty@T!0*Km{s`1={<0U4uKXejSbON6?}2$I9`FasZVcK0ygq4IB=Fee zxG>3Q z=Z2ZU3y1sO4t(-5ud z;QO<$WdnWQdih---e`Fcm|Lj)0C?o735$WgH{940DB1c;C%~9|TNyBVpluv*L!Vy# zfUX_8bOz2F_I(Kq{c-2dK==NirUBE>_1*-0y|RN2IQvG?ufT_Kk&QsT^p^L4naeNk z0YanhwE&M^T{j#sj2pZPIB@gcLg2^hYm0#0C(q3Tjs_L(2gc6ZqyW0TJ}?B>xYth! zY<_`pRUfW^@l56k35tz1iNIo!dSW*q}ymsO~pddKQ1Z>&=K@+e&xAREg zR( zXMv1cR(AxJJ?vixoO$ynDUk5sn>&G1uP$2veAaww9WY?eVHr@zE4+-nHfk5VYgluYud{zc~ol`_Z;QV8jPeMj&?Lp1Xl-8}@$;*pf1L0mDAY z?*W|i+5P~aK69)dh|7rR3iNvU(w)F6{gFgqT+J>$aPTwz6yW4*J|}>8vwkQAbdPWD z1DJ-~v>O<|U{U~GK%&1K@XqW?A3(Zg#4O;-^RX*{tDV(%0N*X__XzNcHTxl;LNRR^ zu=GfHH{i{qbGHC@zwi4D@bKmCF9KEFMhn29Pj5X7{4_Ux1W??!tQvSG{F~9h+u0`` z2Bw~0aX;YS{m|3EihXzg0ZgvXs08LTe)cGEVoY>*V0EbVIbi?S7uEwmFEZZ;1gnM* z0a88;nFI9fm%1F-Q!>aG2pFw30L@)qRRg1TN}mK|zTX`O^1na%HZXshDG7M?z{a^i z-d#V&0GcH;V}Z=w)5ii2b$mQ8aO6k9e}sJa?)nQyudXQ_wc)Kho1Y*4;Y29~Drb`gYZ$A83~vvIZ@`d};o3i?;X7-7)jvtOY;5 zvLr?m(OCWSvDrPwPyZzGosCKPpG&sv$>~&oaz?w|--Wf$9Mb=#{rAhgOke-zE$#Ng z6ZgFAr+#o@Vdd-R4rq$TAI^&VLn~HW0+uUJa&^I?I9=(Y$+mm-U)^?bS6bDtqZ1~-F}K(4KI?vY>c}m{i@VR8I>2{rTX~T^{{PQD|Fh5k z?DOxlkNyAV|H>O8@%xD`vESf6e6b7m$9p_4_$*u43d`5k7_q@~pX^c6U2xkj8YQk%x#B3w?dQ+1%B_cK$BL&C zvZ&NKRudAhsAs!u?Taf+P4YXAKVP!rWHh2YE9bKx!r78&H1v}T&3Fk>?W~62v+k16T+uO2zB%R zs)v(MBbW-}F|8U6p6!bWbZ&Wh%?vzg8X4i#^75KRcsw{J##75{665394{kMO;v>XZ z<&n1~6|1Qf6|EDWDJkTpsyQUeAAN*tM;mrnj7<^5%yBHGlLECXv4>SGq8r%Se)cs; zs<5(Y6)Rkiwy45Yc&`KH7JF6PobqU>5q*fO zL1oGB*g_!IQ`l6CM||}Ubk8f;slx^Mr*6K1osJjhwH?yTmN3mhEN zTkAq3iO>^7|2Mr3)m2n}^o_!al> zO018uMJi>>kHj=@rLr+bY-{j@e9VEZJoM6T9@|Cp#~fq3@bZ{bY!|x5RIuGT?AEc} z3~Ww81qgPHEf_>cG0hX%suNMfiS3ox*-_*bxsiLzW3I5PaH*KR6L~iyG1wl7A5+Kn zNQsz=iJ~QA4ou`O#-9JSFWy5g!&i(*}% zIT6{hjlWs^$vdc7{_*Jz?)gXN|MHLjGglft__ zivn02oR8!z6$spqO$%b>5fABG$Wg0wJVCh8$+htiv4%=n6YeX%$SxkXFTd1kw@fnHAMBZ#_adp^q z?$By6=_qm}ETU#=XkgDEPI^p^GFp?_LYPLEfh^xKkkxn^$PVxR4&%W zZAgb2xM^$!ODULwp$KZB2#<6%0-h6l-ky5I&A?b6dzG~y(=-$}5}|y(>9|Pta*IV+ zdnzsl3s1$ZjuD4<1hh?(X0gLWv^;O7PH2$GH$@%7YtO|uYd?$w!8Ubr%VTu&Rl7Ap z8GnDOR=B1WE^37%TH&xxNY$Fo`Uoi+ldXd$p*&JE*i2P*LW5e!E)Qv93vY~8QG)~$ zMys%<*)VOwSs9l4r9(B|wWq+BPNl@x6cu|9@Jv+*zUHeUh#K~ms3M6X_F`2r&}^l$ zytuBJg`l;pCQ%Dx)R^~#F^Vkb#XgZ`PKT6*8&g{p2FQ5r3UuYzse+JSskO{(rtmKkwXi>BqLvccWP~$8CK?PH&S3 zda(|A4jNShw29=*i7S=b#9rm`W?pHf5@lAhWt!`;{PHlY0v^Sd;K6!RaUpEc8_g)o zQ)e}aVXJk9)tRo#DbJ|Y7#6i&PMJU7+d42oMoH`FM4ZBu{8i1bQZEuxJw&Nvp3@=;Z_bNCZ4grPqggIyCFpfET#@lgty zOcNifa2wRb$14oZP4oCFjo5tx?%cE@lFNWKruof&?LU6APx?*8b91Ep#?y#R6s0z# ziowI-JEC8TT|n%8Vnh(*r5Gyh@zI_jbF0-D;UjA$+?koXupEMUF)NV6T#$Kt9Saw% zb7k-@%Q8{cpCU=#K|Bh!gjO<2)3Y)1O>1b3skk6<^fv@rt_DjbxU=#&n<3%Rmm8Lg z7e*oX2l2ev2@AM@Jo>&NUK4{rC6}15E$0-Fa=7@2$;%fzMa?>{OYBsq0VabhRizf+ zf3j9d{U;FpXV#2b?k)t@=om}T|4@Ccae4Q^6lJS+PQtSI<^CUV&{#8rl@#l zwzgFVZ{PM_T{=m-c5Eldb*GQ}8dhFz$KGzoK5obD+>YD39d~d$#{VPT*Uyr=eZ1U0 zK5ieG+lS2W-L(^GNDTn}=ed2xE{+|#V#(rp3+5|bf9kw*^5n^3!-gosmM&hhc&>6n z>W&pL3ziR2Ml6V3uv|HL-lC`GKDJ>k+I!<4;=Y zV%XkU+3{(Yby7_ zik5m|{|i2Ht*OFCH!QTjOlv9`tsCase{-sSm~Rf%B69kmOigq8>Y+|m#PaR!<-tO% zK&7ibaG`(w!mSIoNP#D?si0P#rj@7bgm9TokZTgZS83Ry1NMbW>yV~G#Z+YY{)yQ_2^GdCa8n5um2G6`X1 z%<;)6C4_;(PF8Y?N~-Ia<4R1>lgTIUVnt_dCZfB=qFx@TDu}9)Q%xeOX<|`t4^;h$ zY9^<;ov7xEMSVO_RT5Pjr%EXc# zJP-{gq6$uQHxcz0i^@C@4IxpLoT`zi28cyFdZ4Nys)3wp8c_`vi+1urHG-&yaH^R^ zRU;Pd?1AbGq8h=e<`dN!Vo_fYR3nLMB&TX4s*A*;T|7`-L{wuqRSNgBSBpivdZ5aS zh1H5hJgIq5~vaq|CL}Gv6sD4;itznu>pSVh)3ra!15ElTU zSYS$#ZO)Ycc#Qfk@uDL~B*5wYbee*PsxJwk4>B`R%4K@_k0(ha8TG%ScoHU9H%wO~ zL0$A*^xDmt)Eys`|D>A7NfV7BzC>SZIvJ=Bvgol$La|7yS|pWWCzaum)Q~4!#nGC| z0=Y3ED0zJq){Gho#4G zNB4}9$7D~WO-Zx3dNP|%uEs3g{ZaC$Ggul{7KNX>MrZ$m4pU9DRpl(PDY2jCL0) z|4T)WF8?nTET+g|(;_lpbfDd+AuUFAnx7^A>VP$vW94kWCjl$Uv*o(4qhH~FOh1y0 zROn#Ok1-w4}?m46iq?}-F`>6V;8Z^Q@h{26%qqpH`5R(-!PW$&j#V` zGh_IBXYRt{Z3z{Dn%xqcQ5hc@l8ou(RqRC>>+r(0N7(KooH~A^#a13&MOG@E$bZ(j zC-fMzQYEv}X*NZL-1(!fRJtIndYZVnI$=Qq4SP@tNGV<5$uzF}XCM%PYnMciDZsTy_2%rYTB%ReYL5Us%9CacBB>bv#XiDMY>zE9J8e#=BN`FaFvY zuL}-~rih(GZEyUx`vw_7nM8z?^5^gvXp+bGA4WO;dnyJs8!GQ*gbr~+3a(vE21My( z_y1tYelAt0*j(v+4wH+=C~^a`u*K}h<5pFa{r|2;Zba_KTzc4&s|I6=Ww;;PeNP@! zHqlP6%+>#x+q?J{YQPSTBBv$X+GLgICHyTce_4OKmvMb<*HiM}<0G5W<(oc|h(6KM znT*#RN>2Qzh7vYLh+|2c>$_VnKV`WR-P6)QDHv@^nk~gEvhTN4P!49vs;RbQMV+BD zS7nxpsvJv}b?9gr)v2h-vs5fSZ)u25v830W*X~Qf)L+Hx9_X7(AC_0iELY;9Yb^~+ z^DV_S`R3B6$u!ue(B-)lTrB5mY;>j2QUFfHq^G5!#!P1q&_V4eI$e98rJ$x1NB4_6 z*JO0M5Cv1I;6Xabl+sQbt68ujw;kJQz}bcx6Ny^>pXE<^3vFx@iBA!k<4z)DqwjF6 z&KRi0D^)jTQmiC_CU4H69YQ??v&dX^#PcX*g3+?wzbT3_Ej}O{oHhfhG9S(ht z_RV7lOY$^)RZ{teIt>2nD{w#smd8+|uQHm)c9dMCy><y5rJAbm5sub7@EFer9!0 z%^~wxsiYCdhx|}H)2x;*-G|x{=F*_hqf}^A%|1rMTq-3qG9uVL#md(*A|M)KE*%wG zqW~ubBg&frjK~bdjIM*vNbVj3S`{4 z04^*Q*JT;ufn|)HmBx=?yk6ztj;57zyUcqu?>76(?kz z$`jY2lIcpN)}0C*gi{%G9V$i_DkIw##U^Q+$Szk=lpaJ0;Z)wd4wYnADuHdu5<8!BFJDDJ z@dY}{zWh5Rn14sA_;-8^|4v@Qzl|FHZJQ!iEEelGiuDV``WCVN%VPaRvHquG{U0F7 zM(^=nvJU&(Z}FMfPSG`6zOBnNU;mEBWN&wx-C5bUq~DI4qWTB)sl4$E|7E=cZ#duW zd%snR;2u}H@9DX&S4i-NQRaKw-=n$zh5HVU?m8x7g>Y=|3xpv2WcgDwzebQ&e-sPL0dSI1r?4gxO z>mG{hzV^tP?N2|ly6-c`pG{u!*iu#WsbxE#c>M7pi_R|I_1N-x!{=U#+`ZtL`J%K`(;dc`1K;kiOBIeJ@@>!=8xZh57($CYyRaY zV55@QpKb8#rqHM{m9*C+j_&Ea9;{|mlYHX`GW?mus1^#;!XdSAG}O>jE##Te)&x zv@MAM(Z-It(hLR3GvTgqbsxBLJ|tRy>B;BeQsn6HRrq;l_;CsZ894Wyr7$y9Lq~y= zE8xXh;O`q;*2ug$i1iY5Gz$CRYg?W+G z9(0PEBK6@%7dXq}+gg@a4*I_2EK5LZS=QFoG&{@U(^{6SSsOPw%c5*8%Y*B7Mmx(g zy0t8}Q$w1ZW$|wsXflJ;`_lplCz(sWgTqm`C|NgO&?^`2c_$Svh_h$eNeUT_V&6y z6J@$S(-brYGLyd&} zI-vu7W$AaG>Ki(Ps=45gI2b_SR$3NR}RS} z@{Qx#OC(PJ^fq{DgdAPs2w%L4!Z2nK-XATPH1)S=1*1kt(+XLd`Y4%JYN7*8LbgOJ zWYYQyL*lKR893kT20Pn(3?@ zNSABU!*rsm#&giN^v<) zLRRtfsp5H^kWR85=9fvaREtu z!Xa@vmpEH1HR^qD3RrNb+j#uNQjM1CDQ{D)W3!*Z(nU;hu{ z8tORpHmzWa(#V&Q+_lv4a6+1*6D~(-<;#xnPL{(?riv#|IC|bGE#6r|wT@`&%Xk5P z?I=4ovsRH_V?KX`+J}OWQ7=u``(^9>ta`tbdcSJD-z9CrQHcg(7)gwEX{=F+U4_;! z1t&5cjzA}?w0>rs%ok5mYv|-(k5{sb)mp#38bf=n-%*m6f&w)??84eUMMJ@lsbLQY zBnt%uoj_Oe&}oUzM5Rk$Xs_*^36`wC7W4kSs1u4JdG+NFSAkQ@MTN_{(33hJodt+Z1dQhOIB)Z9Gs*^Hm6iw+Nbra z)cR4R%_U!~Hm6*;#9m++rIQC=GF`wngPq)LKbg)>=CPAW_LEucWDz^L9i7N9S;j6D zunT2TdfM79U|Wvy#9ZRc$4;`unU9@hi8CLO96Jd<E z@mujq@x;qXCq*kh6^xIG@o~|~S^UkO#CPx!$vNqi-J(+kyJ%s&OpI4LdWx$~&f>S? zmEwt)lTQAv_@ppC`xze>ot(wr>`8nFACa7sPRCkw65WdN+Ru1pqqDl|w`%B&HKeow@0RO2+U2W0=E-FcXF_8oyc(UZsS}e|YfX`Jr3??aAL`v<^=&V3caC zYi_RUbDiIp&11j2bOb$^M*2()J%`Y(DYT!bsEgyHTE8l%?p)(;&T^e_#o3yyKN7uK zjgW%+`7|amQ}M1+!S=J6+3L+Dzh0`uShJs+d{RK9M0j;;v}oHKK{=7u>DMQ z95-2=<&Zhc;UufGKK{=7aO>-(G>}#3gc3fS>4ifY3Sc_qb4Oc-vwS|B(M^MVTrLe` z=LlB_scbll#jl-0FB-97Bw_tSNfX+$xNyOBC|5YyD&}rQ7i(1YrT<$(v z&il9I_$_x`6!)~I;uik<5kCCb4V74e?t1L}Q{4G`OmJ+(S{ur^TD8 z?fi3a%jw5%kHivm4`b(_>dxOIA4LlvcmA$fA0GvZw5HM){)Kb{sS}E|enyrSRk_O8Lb zHr*T82o!y9rLXg%&otn60Jj2v7Jiai(s}K^y-MUeG|nTjr}z_mhT@N(BfqCeVonk2 z>2CgjImJ+)hOB_yL+)pZ0g`lPiZf!A`3JLdo?w7`x;McgJ7X7yf=`AxF+=g^AIyXKs1IQd*l({4^;Pc4u$E0^ zF<~f?rO`%TiKb1%dzkqIOppwCJop4T@qbOVeVt(9h15s zT|2<Z;l>Mu{-N17EyL%bdrNwe{SjzS!f2^zKQk1yRg)vGV zlP=NhGvc*K_}4#H;BCTxyjNkZ!|1NqOk*pN&1Q>$MPUJ_=w(=_DSA}|jYg4tWxOz~ zN2Zonv2_BmUPu*NV-U-QRN)Z@D^?4sVhi+Pv5=~Z*feRH(^d+2FpDrp!5+tAj}+17 zsSw)4Mf_=eg#MW>`s3LX_OQo*(`^_)?WA~!yW&nda~s$d$A5eOg0gb|7xCi-Qy2Pu z;7Pw!NB_hBha02Yy?Uy-*-~ATZ7HjH$@reSD%~=4)j3OEQ@LemQ>G^IM`Kbl=HzKHtel{x9x7(HlH8Pl!ZJN$dl4`r@8_TW0+L0APHCql<*(D5r##~+!F zMXMStV^$Sd`q6h$lg%=w$!O`MquLk|vL& zISwVWpyV;^a?+t9>Cm=PSgc<+*rw}EH!fyfZxVFBmJ!z^80lUpP3Lz}g8@^>()(Ru z#KVYqST(_vu{h6H-_H0pm zjc|d!CgMg>sWU7KptL$rXP6jFqwdiXHjQd4LEwlBRYlTPr;5=fPFLy@=Y-%Vf_<1X zVG}Pon#9qn^1k>X&oDlOl5`Cv=~zNx1|kLCi5^nEt`1jyc&4sN98NoHkgbPOwvO7f z+4XTG#{{AV$JF!>2n4zZ&-2RnR5Pr}$+T z?nM4mN#Ez_`!anW($O7q))|e-c0We6tZHpXGiBWOBPGr{tM8bnF%{Fnt=g;_v@K=p z_iH;I(sw*;AJjA?Oo%AF{T=0aML6xvR0Tr6V6N_5y8YH9?opwimE-@fsHEiAHeG|B z)Pgui_V87Kj;QfW`wbJCppP?^iH*FP685okT?4M6s$dvsU-{?fQT}aKUN6+@NiE@$ zR;VdoevO+JcVIFNiLL-k&g4f?93>M`FPwsfRVp&CPN33cx_rE;T><&%pcrp#_ty*O zSmmc{$Q_CI0p3ELKr8WkRlda2qusM91zW}+?Y93uJvjX@^$!hy_O3na9JHB6V^#F* zKs>OEhf|kDP{+{-N2p6|RK>F#BW{Z-iSKMvCG(xPRlE4k4waGbe4)x@J5^^u9M2Db z)53vHS1~T`U5Ih9v5GNe{TZuTDU)wJgqaw@nBpG@jH$wlv*1H~RT}H5tQ!g&RZ1^p z;@_(R*!Lb)Ap72^3ig6|)~F)OkJh5YucGQP#otiq25p`3p-PR?&CNNwVZP zl5_=0`YcI0ZU$H#;QbYjw*~HG;7s8rj5#?&B%EOpJSq|n7y}1f4ZpL&?_R6swO}*U zGD>-^HoSk23jeDpj}&Vqi?xzjtu2hC&ah2Y&gLJ!7mwdo)w1s$s>|%W$hwb6L?Q|- z@!~`zQcgrB&L=juC}^&JnAA@dM6 z3&AX2lsI1cC)U9rU%vVb7ttoxvi+HLF!q;^6=h0hGEqOuy39o4;@r1#0ZFqCQ|jvq z^qY(;r|CDP#|E)Jjy1{Hb9-1S02^d&;eqTScWfdm&U)L&M^re(YH8M+(Et_O8=M~) zW6=!JL<2cP5KSTwO=fVpX{6Ys zF=CUh7MqlD@fNvg68oyIYit%c^D0GGl!&e<`72HIUL^(_BnYhO|7Mnec{55BDBj_Q z*Q7B_RIp6JFwMU_tiB~mgPjs8WnL1E&D!M)RT}%=qRJHeOTJh*gy~<8xh9uif|%!u z3dT$RUSDI~!gc! zo$_3z&CbVS#suwA$mn^EcQ&cZmRBxB4-fl5pQI>Z<>3YM^ra{5|bgAQJ zLyYVzYdwYIw!&hMQ@Q9B)_!N%_x~X%dJ4uii4)gaiBnYc7~`(XH^g2e;!lII#93K| zb1?Qel3bT(<8n3;r#cQ24LSLfGRw==bq31;c*(0sY)o z(=b~3FQvp_95EQj8H^JdjB{b&m_O?6J_W;xr)v^2TTR1=#6{^zbco&73>CKz|H00^ zFaKYR<1|0@cc);cWU(ojSy_L53T70Wf|=$%1!LrWwa;l>?QYMOV^#5Pb1+xfxG>q% z9E`DL4kni3L;2x9pM%-*=W{UWjcrTBZ}AtN`p-|oOz}7glk7g&%`A1AKKa^p5+rcV3Os=i@7m>oB&A-?)GtcucC;ra-%eL#xzt}v@zntJAwmtun z>}mex#J_L;rTRMaFK@Zczf5sTyN&MiFO42sy{$kcPK_)&e!U?DF8Hy{lMA+6do@n|}GPo__H>zW%i-77xSVf9Dj7INpsabeUr5_IIXO zlI>G0+X~#LSO#9;y`}9bmR8B&g9V6P7{rdP&YZ+hi9 zPtz;g{$zTEO@(=yTM0hudLPiHCBeG$QTMr(H~!w-$_uG{ZsnEK|IMxZyXIC}o-d!O z5z6clcJee$!Wp_p8LiUpgIp5FQ`wpz^6r!RplW^4C4Eq{CaBtSCAvLfqQY{8?(zd^ zOI|@Iya*5QlL~fHsiEqfsrzFSj7p#iR*fdBShK!?>d97quaWOb z*HA7x7}J5l_)~R=hQ$>5mn0}{A%^_RSUR0^RwLkwix3JYAxa^{KwY=TNr>en#54YQ z?2uTW0so8GG(m_O5~8|IA?*L`#?3HPf0V^U<(!OQ@&Qi$$lng}4V3Z6&NsWfx3kxVGpd8%{2Z(=uX8&E(_%&9{X~JV6 zs4JkZfVxypBR)aNo;T6>)l}S#a>%4L2~|?GpFcKAYywuU)#33DYA6FWlz|!wPsz@w zfJArY{yOm132R73It_^pe)U(yI)H7!}`!}x2_np)D(S58O z|F_(9DVFM{;7@qCu%bJi*GP|Q1|L~jBhtK(O0`5Hc*j-CJ9aFWpTYwrLLMa}A>Np} zsL>?U#N%x{+Jx$O?fTQra*0H{ejj3k)>PwbR*$Gjmv6-KXs9B^|M%8KhsA5wpOoNX zPdd;iV>{J|?JOI%3s|A0N^Dn#V7oTbtX>zri48@GU#N`Uj80rwJhqHL93v3N2*fc0 zag0D5BM`?3#4!TcNn}F_$#AW?+0qm}F_|{gjI^0;qs=XZWlGwt3Zc!qNXxqD=j>xC zhBi7Lv!f5$-YY6em$*e`M7T-h(-oGBx#{M|);>Hf-cp^LZ8oiq)5crMa;?~#lx!)= zJ&C=Cjh4LJYV6ImSu%1jVQ;X-lG}`(CzWRPm|P=vpAKOLG^^LfsUt04){Tz`?b*qe zb#W6S&8C%g`=O0aMRr(Pm(BOd8Vl-<@qHpPtFDCalVzsXRbt;)Wj5*S&hyh`q_Dax ze4n_DsY_wv8~K??tx&K2ovqHh(Iqc!`TzCWhvy_S(JYO*2_emB)S-2x^u=65q^&Vq26>ijF|mNB{G zs7c}&8~SDpEQ9Kzh~CoNT(f#m-IGv1_c(SJL;Kt^?9PSqxo5B&M7oo~aUH7XHnQu` zJlAYC4H9|Hv5_|8B8kW3c-lOgY#Ca&%ubI)H;VK~V4Fygv=HghbtR{V>*RlHbN4c% zFw(uK-EyhPi0nLaxwt26(11~?V;a~%rU|=)LB+xZRDz0`T@L|)Ly&b6msJVvr>Mw@ z_HXv^%ixD<9f zhj2TS6mz*0k+9Bu73+Y^BiF@2rlECFtZ1QVjFhzxKhL@$l|BXji#h+zu;>sKs?Aif zzD%_AiE$7iDw3O8y)GwXj^uzHkt$V*Q?u@D@Lry7`cN)3iqN zlXcNN5urZ4g{5D#(~|BQ8LciWGW+3w#rrHDt;p&}idzqx$2CeWZRP!tcCA~@=E2o&pjqtL1i>V9=olO2pm8KRYfvn!2ucF^t!PR+6{ zBt{GEe6^;^erRJWBZ3Fq^wJzOa0#nV!(Tp0f;K$MYKJhP(2y5cjYu@#73}^H&taUaLJ0e37 zqZYhSE0~#FvvNz}oq0I@&J2=3ly7QoWebJXj=;gHC*t16(y+`Z3KW)m)j?xrlDTB< z!$$5L>JfDrOvqhi7wjzE45D?6XeRR-OOLv85H7^IA8HZQdgN9xUjofsR$Z~sJhx$8 zxrMv?g4|;*R04TtvQtVR{^Q^qqbfzE9y~=Lkq%EgYPGvP6&c{fk zEhI&9_lqsEwr=H4#<|?yBFx1h@geH3`AiC#lcO$#r?MUZ(Q_y&WHPb=PTmV8j#;|Y z)iTN&(T!6lB>MP4pHmLPi?l)tiOPtOVpXi!Hn%ieSnhd{(RkDSpxHDgw~5yr0mT}Y z9n=cBEGS6vQSmjsw)Jx!wI(cV;R4T*BZA&TBw zPof-kh!SQ+&I9?mK#09|d6;aajS{0nHbfLrHpCK9EQFD9Yneo0x!2g3`TwaLr1`B* zve03)(n%mXekz1Z)f5*;*<>6W05w9Bx;ah0>6eSt-LFwzUEcbQv5V-Ib;jR&W32o8 znqBgDO5TYT^7n)@Aqy$~)wfRBaZ(<3Q7;dxvA;+&p&}k>zH3>kPNn%<&&AxHhAuTr z?VDIyt;NVT2_xIX7}*987Gv0Z5^43*NUPWLw0a9qtDEA~ltk7bDWa+usV_FgQFT0a z(-cP}@a#=f+yM4`QBxd|%B!iS9U_-ebBT6{q@{+&`8Y<_vNjsYjKJheIBC=xX0dE_ zDj4~w=-z&zeaW?x$mp|4H9k&Kt(Q;>`Mu=_>Ngr8UnBo|e7sJYrWKB1QJ~E_3NA7V znQHmhMizFof)$JVBNuQb;RpyhBoN zDx#w8DJiz0l#=B8nHHWfqjw_*Wu2R@k-c-vEi73^u8P`` zhySLC^*kX%MY@5pTa9EO`$laSk4dIfI$pM$@)J})bPYBHmttl}4Xmh@79t%e zVP+|HzCp#$`J&OM@z4R%u%Emlo|e{XLwE!3F18_5rhn#7En10Jc&sC0EPYjZFv zMcvC{`rMQGv_h)GRbk=HurM>ET_0mGvkmM}X2{wmaUF@0G+MeaE_LRBWGr+~38drNRa<%8?>)2Dt zMARl&5Ng$%MT44=_(9>Fu7-q|Mp0XeijeRi>Xkbz#Kg>VSQBFAJFLmXJn1OQ#H@B$ z6JqLxVv>|~wo;uprXI%dj#h?DM>uypK|D>P`llk20RaoV2Fyy{42z7*4$lVUpa zk(ka@+tZnTUWZsBlf6|Oip7C_E#-CNhO)635rhWtY;PVoXk52?^Gq6d0RI|W&d|=> zYG?j8*doDVSrtpAcoy}JBYSz(k-dB=W-lkj?Byddd#QHMUb0)V7pq-1F?->%QKYrY z1|Jh;gMU%dV)Yhp#^^razeWy9uvk{bQYn`#ip@y=3z?JS{}sq@f{b`WxXv(fv(7M$ z?q+S22o?9!3prX-$vqT&+Uul78vAw9O!hE5CSr-z`0Z39nM(I!?@}%vP9uRYFUc0q zWN$Ggv1ae2vb(gVQa?Rg`{NE8hHs#_Ohx%Zn)rb#jX3Y7vPNRrSuf;M@c^xkOr;-XO?)2P3Wy$QTj!4(=x3UzZ)D3~jx(DB!Rcj5xxN(%LM5if*` zr=+q0R5p-!`cU}+y!?Gs-pgGh4X+YRRYItEAFfd_X(V&eD1vK**M~=tj6JwUcacV2 zT{N1(+h_*f@=ZMDT%{qTQg;`XBDqS7sB8=|?8rsDg+#p3MZQH`zSTOul7!CiE}|f@ z_Qtv`yw(`1UDHcZ#Aih=aZ3_>kRF8WBehuGQB`q zUX?^UAXO*KiD$1Suk5ECkfIZ&Z^n;D_h<(gbwYR&T1PLWks!VJneF&hX}ETPNhds* zjCRvwrI`vper6|rZn}fh*@c(4>xE-Pxidero35q?aGp>;fJ@PipGo6V4CG}qA;tZi zCpFNtKrTgJe#QnVMsc3ht7ivuDSGlVwUA;Mr*j!Ed*@Op*qL4sVkl=RBOx?gh)(>B z0&H*PO#F#$1ZV5R&yYF?sO$hR89K_OvPxE##Xbl!hOvE} zjkG~VF3OGkToNymdNM~o z?PgPn9qjLqYl?2KNnE8gmrNaLv|Nenph^4#|DI`1TY0kW)fHO)UrlASLa8QkhYHJO z(|wvvm)PSI0GU8$zxKDYP*uF8VOa?cDA8WnJb*dy=*L6MC2MD#v@|y5SsGWFEf<@L zEEiWDv^2)Wv+s*sM8*iELrUSr&L8;+gov&rG5E%n(P z@q(86>=t(n)gQy_JM!yvkawrgPN)>gQ$GJri+$-t4>;aG&Nd!tU6=q zvZ~I~Zq+rY@C?>>fUv)?yUoUE{C0vLp&IccR3mLcs<8ZEeUY&!sw%O3X?@Me z&sSMKvc7ARi1psFKF4UH{HxX%l~S>MqV=^9saSrK^=!uA z0F(F-T4+jwO=&7`cb(@05EpUK);B&_2Od9hwU_YsM>#_xeoMt~FY(*k>ALerD!Kmp z^X}`*`}_&0Hw=D3E(tSq3NwuL2{Sw-Q`Z|oDRz*0J=#a1USCXKAz>ZU)a&tg!3g=r zh5aRxih&9;2HjHG?9)G5tde#{qZQ7v=@20sKYWL%4S{OIYN9+t7G{X_4KplKRCQ`G zqT1jm->)_-_6ZeiLU)yLPM#4Oly5S%8*q7i;up8Q@$`mA)KWTSGwtiImRiCLPs&1t zd|`=-DpaWDhk`DnZ0-2O9fNq;gQ&ij%9_=NNxs76P$5=DRf^^NgKDX0=z#L@#JBtN zvIl}Lt4$Uvn@SQWgj!*W$~ROffC%NRtSxcRJt!(Y98{||nfpukkpSI8g>n)=Av6$w zNQRR6iQkW4CFxAL+LYQ~x|jI-lU5L*JL$FGu2*>C7k9YpMg03huPG|3QX$W1QEQ;P zS|ou|xJ&|2g<@DhRBL?V+x=RrHAyAZLINf9+V9Xyow(;dSH(zz04O#^r4kzC8BTiL z<)#-2Fa)i&L=_;MVzOg5PPZ?uO$757U`-3JjA+*5sp z1|0c|y3=l8i^D>7q9r{Qa&$Mf zxlqAkZyQdEcUOseQ&fSwyDJWvMzbCl%pBfFY-|cGaQ9Y?O(G3fv2Hh7^meg}U{?ue zchj1SDkN0)>`0zG;+@Je+chSd{4j&J0+sFHgKJL$b zTxE|6quX+FC3o_^%*g}9#vY<-%g0sRo2XVPvknCT?(aTu^1?lRUJ4De5()0%1FqUl zP{bk=_bZ}dg{1~u=58;8v!iOzRaTQceZU2~Eeg^1n1clfS7ELJwcPi8;qt831y&30 zKA_5B4=d}0%+rPl7ns#3Fp8Z(L9Wkbt)jZj??VTaigr1{dLoba)N86>J&KQHAc|1v z1H5J_YElOX9bgqrbR0dB`yJIU#ZUn2AiSY3xELqP;$$1vFS!F!eI~gNGY*S%=oHS@ zfnA-Y%92V4zzkg` zSUN>&%h`ZVbDRcNNOv4H9KbYmogl+;t0Y4lnLx5ZxRFRy&q|OzYli9S zj%AV^%Sat-n&Mcxt2tHzaV!&YtVG4J^w;26(}-gwP{*1PaIC1SIaZ?VSktIuB`J<& zxSC_hVI6U&S^h~T79Fe3xw<@blEF|-4(?h}h=EBa4*dz644P#8K_2xWoifWzLRTU! zf@c|js7F0VuS_yVa3M-l$Rs0&dDLB^0&@(Y?ftYt$Q+}Dc*IQ%{y8QA9cB>MMe6vj zmBx4S95W3aC4^^6k`ms@juGddV-nFtLYbyi#matcpdK+vnPX<4YlJdRqC4uovBVLK z{y-`TTq1;RIk+QU8%HM=Qy`F=cFO9m*?-=YZ8tGH)m*ZMd>Y&p;+y z>1^S*=h!RSbCS$stVyxtxGPvPiN>0#8S4}?o;0KxBWqA1TP)I`GFok-Sqpi(7V>m0 zj_eevOa)c zT`#0d1HFROiCClcvgmonlv^?XM(eSC;4!K z0Z+@|6s!&_~E)J>O z+#i+34xE$fwW3K6*O}E*IJdJXoasd7%j4TQH z&A8ac5m6pPX53O0NWuurbaCgA`7Wg5q%OW5B*L~|q!IvTYPoi)5%@++Xun7pM`l0} zOmBQ8_E*W(D6FT~aVWygeaKBWC?6(IqJ$}m{((LxQ9trHnMwh;xhNaJG%W8R+;jrV zXvNJa2DAbw>rkwK&A8OY5m_Qj1C_)Iz_2%5R&0P88X5uHeu;_zxN*xSFavw*C7e@3 zGZO7XfAxy?;k<-v^BbCvXg^6dG+F<|&`|x<(1;X(n_k%fW(5qbA6Q01Yeq4k89*6+ z6FQ@^O)&67gC2>ry4VYjMGwOgyBSDxAPN@lvk!?T9ba3@Z%uQ5p17!?) zOM5jCZUVz9rVf~1#Qck}fnWw?Gj1R@Aq~BUYn0lle-WSKnq;%ACKOLYYRcNcbn1v_ zam7F^gFfE`v?daW*ibH2p+MR8Q%o1vPV_}>Zr})m*4};sD4zj_(MUZg{SeBS>dQz( z4b&z9HCrIMm`|9qN9AOKPX4+=ik>YHBF>4OCED_JELMW$_YxL>VJBQ6?g3AQr9Wt1zxj zs*?4Dkj1sj$nGHo6RW9c0|KQ+pp#fJ0Vf1BEv`+%6}8t5hous*1-8A%4@*Tr8jy61 zU{iZ1k+lh-N(Smmsg2YCeHp0WVGhmg4ns(y+5#>T;)@AVdpm(}sU9qaaFlVZ4HVIi zSPZ;OLs$aJrAlCcn0Fnz9|9F{kr)qG)Q~rT?7>C=sr+#Bm=XLA$SfwH)80?CFat42 zMqof5QUf$;q$+6yGJC=hWvCv3ucY`!!lt($n3LMUgowLl6r+YpkR#w_7S1}zMjd!3 zvHLb;0C7q?$%Fty4SYihL>_I3RCFFe{enLNkyRW~Zz0hRnObCEEkp&$GrX)P0#-y> zM8Ck59@T|J76NOiF6zb^Rc+uCO~?;$5+!R0l_P6~CE2b((;KM2i1K3jUu`@Lx8&fpUuf z$AO<2D*QjF;6Kc;$^;;ToIC+khfDy+i5g~wLcdmLLL{Qn9iWI$=Q;&Db)g`H+kz3; z1CUOJj|#fRjiw7NMSm(1Gcc1k;4rIK=n> zx)Wk2g+e+IYb0XAC_P9@1JqbeC?y)42V_OUbV5RcEHN3i$-vWwkWwO<{xJ9tn*9c9 z2njP#UNr0vNR1=_goY%PLO=jy;92rC$Mlk6&039Fo1rsnAzRU=M42VH#FQyRCv}-6 zUN2RdrIaYH-V9k&{6Op9x_my&ClnEfbRO&PTn@2p8j;9%lX4}1`mP(kMoUqT$1QIyPf)h>+B{?rR(f^O`YrPDorCa9cXbmHw}1Q z&h-Ob?!Fw82@ViI!ZFKaR6oyV-)n&A!qu{gZPt@K(H%c6%P$rIflW!6qX^k?6@5Xm*V|7-K*DEA2^ zMdaM<+egA(P7WXuntPj>&Ui8-yPQ)2FR{ln?0ZKdl_U`hAU$N?E+d&Cf+YK1StQry zYFW>=>S;0vBv)`X?Ar}U@+lqD5ntu5ft(L;A4on3EJiBHpvyTQGBnEGc8F{+3jm6b zO+o$$Z7F?vSnl{W=!q~d)t-d4oI$_63`-$?2B?cmAsBJsE};-_M$40L4{;0JipU@kp1Pea(Yrg9{7ot$rq@YxH=w2z`9Ym*YA;pNHbUq} z5ZX`^1?WUxnDg#V2z)SkLjXnk5J8avbe~P$Qq0*9N2#}PZAdY5$d$T@E!4YPm8?x_ zB39H&Gd3Wh4DEEXOs{i&AY-E-0J@M(4ltKe>uv>YVBUtvOAH2}m(mCwc&jm8LjWc; z0s_l4rmkn5diNP6VUwzeZJeRW8W2)06YN|CA&i`-QLoNCI$5?MzhvEKAb!DYO%W;z zR1xMXFUCN#6k!HL=t^D17U>btNSa2)2$v*9L+De)3CN~oXVgJ6Yh?ooNY;GJs*#9s!CbXHxXfd+N%&>zu7QLM zGc!e;C`v_)09KfgAw?KY%9crzF<0t*)}=?ZQBpD_4z7DJ#z$bHBBi*a#Q7#f9N8KW zOOQ~h&0U5GSd%R1VmWbvB#fXA_#IG+VZUs#K?xX=iUGcCvIO8=sRHZPBdjQi7;*sD zy%^df0uf*}Qi)TTQG)y#2r#m(Aao#CQnA~KxmT5}henwj$iE24$b3M|FPm~u?nS7F z=w7yE0tEq(k#jI)13?C(dW0CN2?{a_I_Oj&gOEYhLLeqsH#F;_5JMRNmBmoJ)Hvz2>xCvGb&9w;F$P9o+WXliAv#8vIMXmxO zIp0DC5QtFr2t-u=1cUraFsTqIpbvsT!AhYS7Qr8x15k$S5<%Hjh(FY!H~5i>WLYvw zpP?BRMJ5?o0?Cz((tczLz%)T7NU?^3X+$PDvkHc3$gN1|<%be#DGHz{xm7ThDA`po z1}phhF#M7X>kaa5_q9isH2f#=CB+vKTLwUe#Fk1v5IprM89^{MD>*?hEYqxjM6gt3 zBzg_VjYh9Zo)989Ia3G*U?o==BrnJoNRUj0Mgru3ay+i&TZ2X!6$yvh<7i2A2n+m15`?Vj8>V7IZ-7qL;1XB>p1JnHAOUDZ z$8D|4aR2l;t9^maYG10C_e-0fKdTAT7_4K;&CmD4SqhtPxE(Uoo+8{`%@)j##aHy3 zRqxK@`ww|vtmGZBzr+-mmv-RC!tncP(n3=TZl-?H9%)~pv(MAp7e`s_a|{;yyjY8U zah%1zVv;wUJa3hnc*i|=PS9w$DvSLNtK>FI#b&m`>^vN8m8z_q+w80yW98~ClF4M| zimmY_a4X!(oPsjAH4bW6nagIW#G+lJfeL1>2`WUGr3Op94=RLNnJ>)TB@1)b%Czu~ z=X6%4%VJN{nWbY^NVn-o7OBL{9mA0daNOe}qlT|vQ=BdxZv{;!lbf``k3LN6JRjQPS0+8WO<47pGXG~F6t}?!fj^m=D z!~4ik6&mWa0}NTXKG2Va%FROL7H5@?nf-FJN=;U33udaKI5SsirN&@pPMN8S zpb9a&Dmql9naWN&s^aGoXvad0&O(fCl2tlvRvBHrmD+-Zax}@zHCm`Gm>HKvVd*fk z)I`T=q;iu=)eQB5dMs2!7NQ~JaBlRB$vBFBWE$R&Orz#U3B&tQ!YKODwBh_{+Nk+a z;_!ZyIC_3GV>myWF=~F4G`t@rjhY|L8s3j)eRDtZ>Xj+eV#n>L_8lhR?h9x$4qVJC z4Vax(VOH)_v-1lr6h>OP9<#G;ER;^La-i%Uix!OEq6G(#s9zOdzZnfl)B}aHWD3DE z&C)*Kr0#8Vw*r zou9_eYm#9i9&q9%GOWB#6E9L1i$cptzM=K^lA`^hdNh3r;J%Q`tk#ICZ8KiF-3eh-XD2yVA zJvybq_jdO&C^S8UxND* z+?U|Kv|#s9N$-YUzxqz(ab{`W4F?K1)4`oli$brnmLJF*_@FWO4-uzc%j$g18G7wp z|Ju19&pFUu{DzaQ_-&La^g4Z7>pkz}XU#qJU+*tFxuV(yQncWU1+ z$yxGA=ylh)=HfX&|HHBQspS!Cr%sIxy)GF0-*+a(PrUc?bJJ`wS#*8B}6q1U%~UaB!g{Hm_t{^&0=-u!_f z^!g^}+qdX9{M%%4D}U#Q|9F2!==EQtCe57t&hf|QmA}5UEB)Seq1SD@f0mV=`ulCM z+e-8GVI@Baz3%rN*z4Vp@^;T#eFrXX+f*ESeYN#N4HKr-j~P48C;u0EeWh~cdI4{~ z@BZ(O+KQuk?GeAQF}h2LVQG8Q*!1sv27JEYpV`BGR3-FR@ErG)UB>nGBuy36U4y^0 z@f^8dpT-U}!&P=NUNynl&XrKq3jZ#`ze{Os4#ot<@E70eg9`X;fP-Fg2zI)?T6T(t z&%2O|Km*OsSR|N=D&XuAos`{Hu9g4|Zh8D3+pM$3yR5h){CjLa1Vxje_YzX2avU^* zS-1)25IFmi0C+qQoms2Jz=-j3F#G^QX_$pM3FUQ>X66zEEVu_BW}Ze$LgrV1$6Z#* zmBzMzMlt!AmBTDK9NPfS;kL$4&G9g^9mDG5EL!s3auYTnH3=y_TnWqbtW~v7(wS_Y}bG*w;ZQ+s*Hv(8O;8YX< z6K<6SH4tnI(AdJ9x1bFUU}O?zW{whLVEU~nhK$~RGgIfc1q-UdZwnYFaHssXU?zK- zKefg~w0)Y`0`4_HML>0|M=b%u35Xk1ra;<(ye&+tm6$>WC8V54-oo_ZDiBktn1V%u z;Nq0u6fjl@m;wnEYKbYp4cx|H#?72y31(^u5GO!LAX`Ee1`7}(GG1uWmqG$a7#M#Y zr-!)&(>|fgtT+QwgU(Q^*nt(z@Dv@lkx~;6QSxE3X&vfdBy-Ad1*iaG1V*(HVD!Lv z%xVo3T8I&J;-V`tf-K0lsTH*N5utBLL^!KLgkC^|WsnDr3=-;xgoM6t3JIpK0SUFE z2?+_qK|;M668c9Q5~d9c3CUN2g!&*zxafxjm@{N+X!c`5|B#r_qQZne#Dv6=VnWkU zn9%22pA|N1cwI{2V}J??k~GO^?KRisjmn7Ie02{Z)LKVy_$G*z?-Qf%TDa~-(cUR zZX0k})dW7sQTVgJA>a3f_;FRf57wuGvOeE_Zz$4 z&WVl7=1nbokW~jJIlnOzJN8p4TW`Q}snuSCi3ZJAqi3V0>tUk_*TqJsT@M>gybd-x z<2u-A(&*dhtdZF$U4W4~w2@>9=IbxOeAszy3os+kU){Z1he=`G+tb5^4xbPHj$Z|T zy==($8r^;A-`8V>!0#(UehY`E$tJxL$ZuOO_#ly5P*BL zDPWM{ali983L9j2xc!+QJD@GZAc0Aav_%>SinyJd>n9KrD3$=tE{Z2)c-;H!2NkRv z2FOB40BFIR0560Dz!O)Ipy>cGAtV5RKv%`8*${Z@O`&i@h9`7U&>`a!O8n4628Nqd z28YPdeh|W&G7*SGIB=&-1>zA7EE9pC1ecJr2F;#?018b3-eqi(hYWxdn8FMbzFBid zCl8811#G87e6KfyDieF|!R z?Nek6XrJPnU;BQLBU1Ly#(v#XxC>G~Gy#$;DDe4}??*pb1C_D^bP^p%r)2X#MwD-s zl@Cw{U8$yRV5l#J;$lGe6di+fPcag^mh}KYNd{KX(yw_6nE}mH;3T6`3J^WvGTW%0 z=BeEyh6cEnK~x?tfT9Em4Cv%%gS~zeWTnG-ed|EIDh?Q-+xLyS+ncV*?OO+GN6YP7 z2h;%u@YMf5n_cN}^ z@6QaJ8ZEy+Gtd|ag>WAGAMtw}Rps|*2C`SGL#Gk@eIPGodVM~X`>PyZ zCs!O?<=ZN^R=Kpwn}R3`z7JNi9Q<%KE- zR5{-_m~K_Rrt&hCg9W9mieIT*%I{7APf|IN%7IkAGdOWoxr@qER9>Ra3l!&2Iflw7 zLep2-K>S{yN)8m`SL|M8^D1*!nflkA-LB1~{i*EruxZovv+0EEXwzS3F1bdVPP}e5 zJ>$CBbkg;*>5(J%sO7zHD0wfWsl%qe%I~L(j@$n0u4{aMWbe&GJ@0+~3;cNnNSW(s z%KQ&gE_p8eH`5;S_Z8D#@;BdvztF+kBYFD@J#U|9;2~!g?Q@KxeO`iSUz{k~S0oL} zmwEC+$uB3s8CNCR@37&|vlk0&h2T7VvrVe9ac;p`8E)h1Mag6mxMEv;3Ea}zm{U*& zw<4g1jkzpHC8BnX1}X?#6I6)C-!qK&L4`>+<_m$lBr<1hWVedm#&m(W^n!HECN;_< ziBgHc9mA1CAOp9l+PFqOzMc+M3xzloR$(rSQWFk^6}S>QRFU8x%EtAJ%o!WmSrlhu zF2Ya-K{{-c>g1sWsb1g?Q>IFAD1(iw6XL7rP!%@p6NfqsT-l_0I#ivFa%GcFsJZGB znHC$_CInn{!ccL7blN6W%0ux|mB5|Gp?H+vGp0BjSINgW(V<*C6(CkYuBzxzl_D17 zP}OR#dPSz$MmE#{R~;~vQIOm=zp4bONua6{r~r*NPzBW${sNvNKsVv4gbr0~!{Woy zqEsH>is$-4Q6dqbUL*ocut}!{zW{lu#HMi7ghM3=T%Aa{5||1eC52{zT=7(ZSb^$F zl$-&sM6M4MB@zL~i9~>jHtCq)7obgQvQe`TsIC$PuF^)$LSRk_R98@iSYQ<$s#2f= zpNu1vH5ShqFlog0t!_8~3T; z{324QIc(#41ZP_W6vo&%(0LF3t}ZWXO(yV5!ea6Do9>W80|p@5WD3Dg1!;f}qEMt1 z@{~f6Qka4VoF@get;Xv(Pa=(Qm}T;lc=6hV_#__4{E8&jUB~$qNi?Du^?n*VuT6|A z$*(8_fDkcOUdM?UX-q(4H27(}Mq^xu2Qs50jm6h-Ydfq?k%=r}sj4$}A}$}xGQ@EAv6e0X>qoiGV$ zT#0hzM+%Qdg5$%(-4fpUCur0|$XaC~@p9Hod3 zXxxT!TscyBOd>cwJUot4C}^K%LNTrz89D;25EvgG9!DpH%-tEOVGI6F-b-&+a2p8kOC3w)R59}Uv;_&!VM zb<&4?7rxVSOLh1@e6J<+`n7qt{2AYMiF^LJllZ<%==H;4+PCnXmjme;R($Uz^!g`% zIrkX83v>2wzGL`4Oz3sG^rMUTPRs|tyy*daFDCT*{u@fZjql35a>x2Ad|xK?`pDeQ z-{3nlE#ID(gzwFSUVnJ(Cw6?7X3gvGZN>L#La(#K?p}iL)a0G}&yDzAP3U#)CG$V= zU7HUVOvuCcZ9=c#ZAxpycWw;#{p}un?!1I7Z#BLPlz3oa1ilXxdL8$8dmX+LWcaT&i}1am(CfR5n-=4{Li=m}ejmOs z6nb4+ePlnrGqm7c^L%`7DD?VQ-k&z%yF`EFUps^E6NO%?QOF|ulGIt;GqMx zZt<>NYc9V&HD^ia^%wnh-+MRx(2AR%oHO?)w^%lWUMt^q3jNNLa=m~z-_PCmKl%RE z$kyZWd&XB;l8;gizt_Dz{WiUi?qA~__w4K^r}8#7SJ8IOLAsZ%Xp`m;oTVkZiq>n6 z!r8_YSJ5g>1DtJH=_*>HX@Rp{8Lp!F8n1hMtu+@PjCZ*=)!N>46`3`c-0N!>6}nf| zE^l^k&(WFOOKPne`q4o5{92n4P8XVtZlQKjGMp|mCA!VE%U8mw1>%0GnAA*xxt7#h z2~#bp1(Ai@J3i?9S^V>E{2gQZEkhgo-UXP=JMe3ory!57fP(IDmvidoY6de0uGZz; zUV9L$=^%M zkA|=Wn{-BEmcn@PMiInY*kY3Vx0~xDMCPQxTp|r{#gxG>FZXB(g*FL4MOj;DlU#&A z_XnHnV<2m=aTPf7NkSha4mk3~yM%ZMHjm+r6C!?J5E2UytciYZQ6}voe(v+MUXW^4!ozfdbGe+G zY7fc8)luOI@fC1S5uS((6ZNw6Tq^0=PL|7?55yNbiuq^vTjYZDa5& z&>f2O@MD>^rGl(pkY05R<_e%)mGmkV=|PBjT9%##5+5PKi)|pafVeG0ddMt< zs*DG$deNJZfXdP{F_)Z0h&nLE-~uG1*{R2^835;}e*2 zk?F(5V$dl80s8%2nYK>+UTPD#u3CJUKcMxks%?;|t5jUK7VatH0~CP-LKdIe`XHGQ zui(3tqThGx74aPb@txph-zA9x`tC(Vd|gP$ahvQPAig6YK5&m9@ii*qgM|CAEIx}V zi4XX$N__ZabddD{vzUc|^{uL1)GWx_6_`Si>BXl#vh^j>F9!#ZZxeg`3dpylwq-cv zTZYIx3eb0SNa*WQLEn|05{4jsV0C2YeL zu+Ln3X?X0*7!~ZR8xs4vRoG{)?Nwo4-8Ez1wh>?-Fe_u9xpvXYD`H*V-O!8eL*id-n1DM&^^jN+F9T?%SZz*4Y^qLdPwDbPewN#TiN z(NHj?K#u}C6mArxp%|k?Rtm6CNKxpfI3oJ#M)aV-p$Qjb-fRq<2mqKbD?zf{5z#Tls| zDs)rt8_Mw%XH(paI+)^H)T@+OLUASPL;*WN&rzI)`U`ax#Y+?ip}wKG1$77|4p6*+ zIzWJKH2a|}{@To&PU6?gzWM8D-znG8zMs5~_Pz2t+V{HaXx|yv(Z08R4g2=5kxL!8 zIIb*{kHQ-HnD)RL`L%AVyypJ|sYWO{Xn?NmyE`=W=hDj3*z_A?^l-s8mEupNO-*An zYw%}$8_3TISHt~qExDhhfs*!PLY~tAe~U!Njjutucf0@l^Vr{~cyM9mvv@d&Z~9|V zJO1im+H5vmj48w$=keSWgSYDN`Y4Qpzg2gPjH8T!7pH{0gLrXVzSEM(=Xn??I7sP- zgBOVprrU$%C-FB8@#+L#;TI@Q<5e?W9U+vFk>#4zWHea@?I+lRX+ED1Z?1k5j}N~u zb48mSq|77}zL2u6gCO4!!Jd=_D@X?kd8Bi}*%Bh(gHTP#>q-QXdK6J3qf{R0_uL+d zg%B=*SZvJE)_EA*xV4 zeUH6>-@7b&OBK6ZG&uco6qXh)MIzprAV$&nh=tReDG2w7?au^CZy%&_Ah1+~VS$0Ghn-k`Z zK}|;yA)bJOTZO!i#ODVJp~J1uWlcOV7JH6opoOoJ%}2-eXrU{wcQ1@ENAD?SNWY+= z_H*nFUmnn5jZU-@5L~g|QwR4wPAsgzvw98bDeH01Kk@R9aOth4`@66|gJ%mAABe>I zugg7q-jdr)_DG~CqY$KkE|)$)I)IaZd8^6(>u}21O#z;qP@IXyc|}I}hjfNNjsh9a z;E*|CPyjC54q9zTr)fY&;NJ!K*A4%C_-->%?f`C*);qOWlAnwf{N4ea~(b%EvFZHdeQ#G;KuduC{<(5HiV z5y`&YrKL^|1E!ynPuc?~t#A2%Z{Zdd&lUb*&-jO}#j~Y`0K;%{8p{?K>20>1_|nqE zA*v(?S4kP7%F5s>8U8B7QUZP<_1zKPFwgX=q+b-ZI?nBgM^+esbf8jm@z*=Ww|FnaDAvORjV=fAQ48~X68ZaEP_ z&yx3&`-q*M$NPu6&%aQ^=d~I#GZ1eQ`Kbe*AK=h=eVVKp_`Nua812QLzxE=`q<26U zx(9hjjscr(ZuZ_cm>#qpPfH)xZ!g9#%LKPUXv;}NiQ)cD$Bo;O5G$MhcrzZKBEJ|s z77ss2#N%r6;20gMm~!Z6XI}fj^3`3j z8-LjJd(G`<`){m``)>P*d&fH~dLGc2HjOKO`G;o@Pk;3vi(3EtY0cC#W0UuI*U!Cu z_q(O3zb(>~{4@RL=Z;iw`(v27O4x57^fdImW{%@(T|+F!s@Pl; zK9<#uiPq}Jhebv)5#uyt!p7;wN3eQktTrk7fy25!@=Kc9E%!|$%xI{A$+-z_pHXy%-IYx%7o4fGwj?Wwn0k|Q;F z-~I5*twrX^ZzQ~Xe*LKxnqw&iQ_AQ4`d4Dvq#rM+(`x2k-uH(KODB}>p0O+a+b^Hc z)Rj#spQJtZzZ!qg_w-@SA5QElO`ovu^3rebp7HX9 zxtj09-nn}I7Hj3dPPV@HAN%c^6${7dmZq?`S|6J***y0~&6@C)pXIgu_?6C|f0=pn z)*Cc+KmPRs?WBS-@mKRwrW`BR2=!k`H$PCg=J%KW@|$T-XKH@;!Dmesi+9en+_fj> zwT)kC4i%ev`p?cf+TM8M_&9E==Hb8n@}+0KRjsSN_^qC@ zdoaq^rKzsHpyiLU-JB;Urt( z1q&8fzcbBfTmS4DBfsjo)!CVAjR^~~R&9hj&lneNT)$;a=H~TT+oxWW|5>FnUTQY? zIC1vQo&fOv>#w3U_#RZN6(SCpFE(y4=>v~gg`N|F)bg0)l?wbyC0;#$Rs+aqn|f4u zzD3jF^F8V?-74BAnZ%j3Tf@8)c*ol&ELMuz0a0GKPotSDSt|6R<9E4oY#K(x!p9s7 zEB+|torNhn;{pZ=XW(B=8an_J+!ps*IyntM!Is0&AO~VBGBXaq{IKO1mb{otgA16M z24J?_f^$aBOM7Wyt1J{q}`Me_Nq5Fy4fK zQ|%QP_guSf0B_rv(>7=VXOBr6E_Xm}?2b7I_xs3wXe&rZpikS(s_bHDDYufFRCJJ6 zg^u#v5mjODTq(A~=Gek`bvU2T1y3aJ6%v<`gJ2Mdg{o3itCA>@ZFm^s;fA^3*uU6VOEA2bf?R6&FT%|3dB(@<8TN{+t zMx}LP@aW03c?xa5k~Yu4(UY-#iqd|i(muoAp7lb+n&Cp_x)kMtQmKB4T19)TiRc1V zOEfu@dPVy}@^%N&@wespF-X719<#?@9!9~;N$6C_J4#e7>-m6R&)B*`t?PiI;pQNH z+h$g>yT=3Qf_3tCor!nMFxjN4ko_EUC+a%qO1hr8c1yP&n+d+E?DKryaSdN`5I>B} za|hA9rK-91D5w?tB_Dr+k1u*mn(-w+rW^+AA+7S-ne5YZ?O7FizN8SN-4XAOr1Q&K zoL!!8(KC9D`T0xuhs{59qec^$QP!HBXSDMiu>wpck2zi!3xP#m`c#;D2__UDtI^nI zp3ICBr=Fhcc+ZA6z0*|w_Lw8qRu~P_iF_01p^_rJWn&*c39b0NPYYqj5~h0P`AKk? z?j{B>^NZm5i59g38Tg^A1dkcr%ARFP__`86{}h~O!eI_wr{^8vd+*k09&@}m77a;o zwnBM2;yRC$UX(|f3zd0q?b`pH`Tlq2`yZS6L}`O5j(qTq&Vh3sGory2LMK7dk%cdH z$WRF=6HTy^m+C3@@piL3H{!&`GbLEXK;V>VvN^UJd52)66&?i0^G+E)NZqTzw7COd zNp4672OPFM*sDTIo3zoRrlU864oGn+8MJ(tAF%zifVyXWaL>KNa`>RaVK8?>UWt({ zn2D%+Wj0{C84(xJgSnXNh#JFlKGH)yq1GFz%vX+IQd|As(BKN80kQsh_Sk6I)I);; zP~|-Eg{v97P5bn<8vZfw82|H$fML*1o-kb1&OzzTgDeDwAJ*Q{tU~SGroCxcvq$T? zCO!w5KboN5{%;Ag|0}W?;s?Q8N-*25zapd2gBeBIL)=1pui_oz1J9V^*xk*SF&rus zr(Up4@QGu5{M6n@9h0I(dpz%$DnerH>8ljc#_A!;YWD(T|C=_XVdU(EuMV~&z9p^+!*R*HF!A7OFg_D`V}28 zg6C8>{b0*TgDmAlaQwiCQx&L7v}fJhJlC;9OoIOJ_3(}z)0=twvU>p`ON5@!a1N7> zgSm;)!s#9?K)X6lOh;Nay#oyjYWE5~p9_+UR6WK!Y#LHiZc1W#<}Ch!Uf5nQ%#@R& zu?5&K)~FToYBhnr?aQJegE%4(v-aX2QWtv{rAaJtkz;`ou)ZO97=Pb5o_?qS;;uNr zGS(oEfPOg~AADY)hAl|+;ff9!&+(i=m5hqgbQqbJ7V8InpXMq;ha3uIO_?Jd;U$b8 zBz-NFzMyEr#+~zPg0MnsJu|&`kiZCyXJwAC_g++)8efOdkEor-;@`DbR}^ZAngWgs zG_VlJ3x?r|pz_SlUn>thlpy&$w}CAKeeK?*b^WYy(2{N zD>_026{_>ay)td~M>>Y4ucrk`gj7+SdKj~2rkPHAbM$cAAWAuLsPGOT*px)TafN=K z!}NsUJpa&Rj%iB-r}rV-m}8=SvB4%a@;%K$PZQLM6DGK@U9TwJDNHy9aQlQPrR#0C z_-I3&+z>Eeu~FSHfu8_VknUbyT44|-RHK24(wsPG$UAb9y>VB{zlNr%L)~?6fTp$0 z0irxZ5p}b=p*}!U2hfB=0ZqOBc2|{x{_9D>a<4%agGvOTEtqS^g9{@aYp;ptN3PB? zZ2`lg(1A~65MlfnOq_xvYU*6aLax!<8Zx!WzM{l2R-+uhukelHygEXz)Fm(_$O64JfNzvT;NdG{3+}~G$njh<2#>der1%Fm znDlI3E{36^Hd2fMA;o}@Vn9eSAfy-&QVa+w2847=H=vX$gUAJYr|W-&kU|uZC`DdT zM2;N&l_WHzaITMF?8_<=P&8C5$b(8W-%&*4m&@kagkip?5vz(ZR}&ABOsf~STd49! zi0`yv$T`7Cgi)M85gUOj9GoW-YrIo=8j^p1-j6Da3on8ahSC@duU}mos zdX5P_bwW=)Zmf%U3-RPt`<|0x%#v%uA6;Oa*};hnr=sgZ)s5ulR-xg*EO`<&Tt z(a=v#>0bHC5St$Z$I!>y^7>wIggfu*5$;{;2=~D&EFRL6>mT3Rq(eM}qPVfC&}#@? zzwBbuqwy!$@ORjrHl{ty=IHcrnwE&C7QKOY>`NwVjp)!bUh;3)-J#7ZuKM+gO_(DRg_f8pKJz4!epJ(5{ z$isa3-Z-`WBLD&gNMW)84@gh4Oi!yZyYZW??|vQik5{Wde)KgQ(5>hDns2yWu|p7$Ggt{ zy*t9S`mOFXJ)t9%VL{CbpNy#ZJSw%jZ2D8b6W^(H|x%DaD+8&3WaFLeM4x791_%F;t zMJctzJ67dsU{+Vm_%Rv3wwLr6)R3sgzG%|EO z4n0#vW)q=BucGdAQUcT7g5#7lAhMa3+6@_f@e4k7Ar+x0v+8gm&IB zBZ5RSD8mUL&dt5v=-~WAbi8cv5pX_-4|H&cV)`7k%6S2++$}o{HOv=f|TE%35an95l>%0mIy#%%}|W#T0cvE7kpE1owj*!sZ1Gb9XMIGRznEs=Ikp-Oc+U z<}16oZCC3ZLzzmP9VT^1BE626utD$2Ggd zkX{h%BWQ23S)nVSX?Z6O^L+#A8Kiz!sDH$Csl9~otnX4ei{dJ&R3Ox*pQWPkJC4l! z*L5H2I_N|JhN+texscy|rm5ZM?gxnL;Dh!!+k^phC)tfe44VrtnSCBlS-2@t& z#_$rL-YhLwXyYJIxM-W;vyJJ+SXrD(GAdlE#qih$VZX9ed&{_?*G|87{BpNCetBs4 z1={Nszmg5mL#)iIVxQ+RJUoS~uv_u$2%bHMXQgR%n&)bKv^WT*tI%5=ik79~r!NtK!qNw7|uK5SaoN<;=l2bq>aBZ;J>txJGo38K=j zB;Ni6^_vda0b+8JMQILX>GKTyEoDF(QuefgP!mVm7m=EWkQ&G|u;h?JF_4m=kWxzu z8<3RWBr1ic1Af0RiNw557*tcQmsDz}o=-gkYOA|M#3O65(8wAab2rl z>;4mt_#g}!N4jpH0;JP63jDlb4?u@3#UW)`9wNQhl0XJ!X#|T) zAnmAn!1{J2%ADi%r{vzJFj^9fmh39}Yj?3|ej*;VvzcN1lTs@PTZR(GK&U8JI< z8U~Y3>UO(|>;QWYzt-&$rGESvJ)Kub#TJ!BFRWebdid|%9pttWi#(8JGO+n$!tk-) z3BJVHiwo*fyJ!a!#v1c@@@H`h`Lp6lF=p*bF(!ST7_&743}hScxL3Aq{uY3fE0ry%Xb5*szY!Y(QSC|N`;qBJmvEsqPG!+vj6Vlve<_FRGr z>)Ws-JCT}y~nV&Af|8g1)3cp_gtWnXy|ZlBJG|Oo`wN{&^fr8Jk&r7qvOb2ZeN@nG7Jx` zV3t{tFld~XdyznmLYaS{wJ9*zlh6z-X9dyT-c}<9#^k=boGh>5`Y2flUWl`YIKGgl z(+5dbKxqE;<JCR8x4^2fwgO?BPFBv zNFDMbat_(%xDdY~8L4tF*^emjT+x*fZ$LWeO2OZ|`u_@Fh7g7Qm5N-YwJZ-rA)t_o zg~&vd5k)x zlNp28uhkpBW|MtikCJU)3xNNIWKysccYg88rmqxb)7O(ho4#g~JztNKEnf?;UNR|I zio3peWy4pBvf=B=pbcMI(v)oX!dCzr$i78<`>%mlHhZNgo4uY4+Uzx(?Dcw-Z1uwT z{u+`=0lxFsz$+WQQk0EePX=xDvN>L(>nieMLK<5~EE4RHE%+2)UU7V@>!3#dkCFd{ zh5WOnZ_HYo{@j|(?M7SnhOG4&8`h`N*K)S5&st~P3>R5zGN-Rym9=h7<_JG)i0{Aq z-@nnqVA-5GuTKy8rQ^2PBKkalpLvlj*a?*QAH*DnKdsM($^J9=&76f59anoTXRP-Y ze|!}BdJcIn2R}Ap*9cHH^0y)G3fhnU&EAKwM+CnTarr8*L@;|FwNLu;Yj5&5rtKY_ z{rw8vgFCqT7XR=0{9F8SmwsSB|Cb}%&mT1^`}t-21RJ7Hr8Hz;+xp0%EwB0Oybwk# zZmzVW!tiy>?#laep+;c;WsZ(ko@E+p|0PJbuOoCX)?`)9ABF8t=sHS3Mp8^Yvc+J%?D7PpUSmtFoE8hlJk{duIC#NkG$(i;Y= z^qX@_u;7{%t({g8tZ>Aqu`qScwS%|X9NRCG7m)~rI@`m@8%cm%cQXJa1^|>ee&T#R zK`&`xkACjNWt|3mFUdI9zOXxC?!*n9adV|*J%+i`hMu^&6D^%^V_A3NT>FOZq`4Cp zb|wI|I>ftwC~q_wX&A_8*O%dXWQ)r>Z%o?W zYuf%>6IgA=g=PrUvp)XTo{xf07<^5viQ+W`TsnJ8S60clrA# z57dG=(G4)7x|RR;qiz@N&qBJg8Pb&>#gW}B6Cs6}2r0}&NMR<@Eij^VCvIv9-O=JC zw&9pGj&4I^H5hh&t|T>)OShnO_{Tm^ZE$fn=fg%4@5J!pM)Gafon|1r(v2axFQ-mfL6`t0fE{W@}L$Tpexhm%ud_ATc95#%&EJDNL^ zUuNEqO9mEJC^AdMR!16REAVCC8c{ZjKX z^K$dk<`v5i;`|XLKR)k^1is+NuJ7>D(fvSKE?#K-4EuP{?(E{C_m~ zRVLh%<$1k_(X_50tApcj4X*VkQp=9DwqUK94ABezj1e%Tx7qKo6`aey;S-dZ=NS=i zy7+>^tnb_G)6ML@MVh>ReP$B$Thyc1D4jY;r*B}Bc<2-ql@)HAc}Es#@pI3AfLQ+Y1)qGwlL_T=!*zDu;E`{yLQ@{@6zv*!CYi zYg`yNATXyzsnsmaFbRkhz;X5s-gvH1U6M6HkPh?GK~GN`42my?v?7IibBV>IL#ZHl zTAOr-AVK(r+DlMr4+QgH-2uUviD2GE!RWh*VAz5d0HW5GpeJv^{$iH8%{_JI+|MG- z|3HJbNS)@R=L(KkSj$=BOGPBaxg>Qsd#5-(w=o^LYYlL&6;nNynbW~X>_5_@c}DP# z*?RAyf+KAHD@6EK$T5DTCz49-AYI}3p67s50i=i$#%~+*sSgl|P8|Td)r*YJb2uMl z3Z01(G`HKTrEwqmh?D%ib2hs#c6xldWz5;xdt#^aspHHCkZ8W3Y}X3ju5rxwrR~k# zq4#OBQ+QvQ6yqz0TR9iJ4_moz@AqkC@BO^g<9XtOpNxX-G}ybTH|o=!*_3Oz4Y_e*|L4!FT_pV$UD` zrqRHtQ^BqwS+zEUu^;Ch+06uMIz6w7XcPx|`%aH%))!$KZx}x1Jq`jgE;T>BJm~vq zioM~8aRMa1_{G@)e1cQrd9xVVEX=ZEXV|@CQMF7)iRX4og9tRhxYpx9Lp4+aC-ET- zu>>!Qfre(!elO5)2k7vxT|qjGBszQob@Y(D`rIVifHxg$D%Pz~hlTQUmuz?h_J)Do1PI$IHstBmVGZBe1 zaSdLGR>n^Q0=b4@8%sg@76E-!gmFuQ6c(f}@)L0O@He-Ohryw~)l)J@hu@OSdq|&? zDA;!qR0;R|;e43plQG2K&ZD6}q9+db0hAOcgL{<_7L~f!kpxt{dzVZB&XT4sF_|H*!+j_{-QJVI*!dZlk2ne+Q{Z7lj|0G zognwO=p4OHWb+>+*Pp`Nnj}|V{@KU)_`Kjq_MBGy!X-#;{KdgiBb%b8bb{3E;d{4I zc|2hCu=zN@w}wMc#2pKcWaYdl-q^TD&*$|sS(}h&xIVpyXP_V~Yo%OpY7ZX<1=_56 zazVo$UJC`ftOwoZsd-zDH#w z*`L)A$`|Xwuo2up#M^N4)AtC<&-Gb>!Mrmdsy^-JJGe5=XDcYnUfYU4<-`H>&^Sr% zZP&im-8idFEys%T`ik>&J-ctlSbod^1YvNxV=Tju>7m70W5ZPCS}f0V>V%o)*`+W{ z)*UBcqr6660OT=Q;oD9v@c*CBJYw&*<~Y#!8Rsu>7T8_V@s>Zk3!e@WWqi zo-=&rINrWlzk~JO`4K9TUhcdVW&s&(mwZk_j;t7hEjaV}JjZ_9K*#E~0I}-{6`zp!52=tk zh_MHMUpgRvt|4!iE{-jj8;%C^U31>Rc=jjX$B)d~x9Hge(NcMCQ8kGqaFcUx{}}d# zDR7rP@B|}O>#AqLH4E1co#gn*!yW(f`SSWw*%vN@%%62W6aGrC&nL}^We+?SCY93K zXLO{sR%uS6*S2 zG{*q#Bb4^f()N)``=uf6C$lf`>h{&roG56oSK8lA+mBJ&PYj{@X7&X=6iM?yW{-_k zT79J@t;Q*>KGg=ddVqc5h!$Jvpj8x#`dDuNboPaMe2%%Vh$o<W!B&(Q)v^w zMsCptYh;YSiC-h#VWf#)37>@pE1@ndxCtoXAZ_B8`(ML?<^EP!Fkv9~y|js6=(%CR zLeB_k0zzL-syq@U6)6%&z@$QD_mv09y`0T^jq%Gm`+n=pPS!CgjFjY+GEyOT#8+;f zS-gYA@G{p~&HD;-`^ILT&+7-Lc?0{MS|g-7UyO~&?Vpwtoqr^2Vs8K4S$DM76NTNA zH7U3MDYl>tYx3WT>_U3k17b8|A?`~G*E1-FT(jd9Gm%4o6#K#=rDB*xksgK?>;X%(R&oL_xqVPH z9g1cXXW{)4oCsy>%MH=s=7qDEQZ|1MOlj6V)w3O<#>f8I3BeiHz;%04C5$K8f&{qT z`WXmS8Rw*iXlcBnrE<+6*TC-s10F@QKf%mbV$B~#tBrm$rde;nAwX#X>GZMj3fEh= z0>_SgvrP8Zb+il8spBr(r~U+lh>v4U8+%|!G$ZxP4Id;8=g@{xTc@)J7Dk6rTDN{H zNIqL*98oW+MDyeCBS~tkm-7pl3o%YDUf#N^U!5BUtp_Yr#hfM9ia3%8_IU37p&s*e zi?jwZ+{>L^%K1Fsfv#Y3_%`OOw~9EkHCm*zo{E_yjE}~+68kWJLi5aC8vHB^ zorG|F?g&@5yfrK+e^AyZctNT#Cq^QeSzL$pJ-nS(c%|+-f?5@=X%Y7 zSP~D3(&y#{3(SvuBF;gFAOKvnIJHf$F-sAOfV}s@H2R&-L+AW2k&fw~VPfyg*5jO8 zvNfMnV&$>78e9rw&?g!IOYxz}d_l4;^)L6UcT3W?WLxc~BJL*&=ZI* zOuTOVY;X!e20wL<(a1hRBuCd5??_UwFD^*gk9&{oD|GP(Z+}EL^I-PPp4jifAkLl> zkR(}3rBb;A$mts(<#t#!KDi_i-(b}JGz6I+V_&x~*72zcb@7SIX{tYJm5ZLo=+c%Wuf~w7X$X9LxqkEh8$M zJ794v_SqayF+$!YW9BpwACVvB8OZ!Ief-QWHm?}#l@tSL`NXMpJX6QdEN1iGCv5Zr zTv>32WA{y((0z2zvYUK3+AKqXFXs#I56s4DE4!~ymIhmZ&*r^PL5JRXIBi|Z&0dPD z!WyLQBuNYvCUR#PG;G0pxMGjXCgnTYpaIwI@2M2A*&vmptJG}<>`N^Xm>jc}Twj{zfo{xnHZ3?O{(&&&(s6N_6hc#xTwJv+zNGCO<`m~VAa`Bc$@%TdJJd$ z3&Qu}f^fU%r}v{XTxcB|7NEz{wLMFPvH1@{eR^`HJt( zuI7Aui**lZ%`NZNg{KEpLud~qFFj5BErHPew;gElP;qzV0^&t z6L0o)*X$9g14bDN%5uEkt)f(FmddEnA2LtvGBeP=m>Rv{m~|h|Gzty@tlr5TAy%*0 zEBrQa(`Fzb5g+Mj>Q9AbT(#y|KqmB1ql!9E$RU2&cdz9T!& z4Gak_Wakm=k!0DMMj*#|$D#_}VJo#s&7StVHJbcFcF#Ld?c>deM>{LR*n&S047MKU zoy{so!wo>_Xz$cfN6UfWqhvO9DO``m<(PNKMRqk=+=_iMBEK*@n)elBXaw2;*FTO9 zmKBB0g|S9#x}J{2;1i4=5!sFLY{L%UGzDWYs={6R+=nz-cW~X_O~7*J>A}&%M&$fAry#odL2eGC$%>?p7aq>d z3BwEQ#(RG*WWaEN>v`UU+#DTV;3ANBVs1`2USN9Xjpn*@b0Rd!>?_55US-67B$q51 zPVIoeFl#KhC^O)B`Z>KOB$}7wlR@iwVe)!j*#EBQ`PcJ^%MMHrtmp01(Dl4qUm)vt zQFwk}H93zYk957^`%#e6Y_h!f*WF)pdGCiusoj}W%X{4pT;3}v%ufBkw7m;>6h+oB zJUuf>21w|D0i$vmG%Ar*h>8;Bk|r~xTV@1B5Y#NLQFiez%n%ep1}CFwngLvOmv?Wn ztLw&fUGc^XxWWv9TtFdQUs< zopb8cDN!l5fSgiqqS1r)zN+Tr#ceYVwc8HH+xuoVceVFn*Ql8XT6G-%n))Q#|G4!g)h>FLUG<67W`G@X65Zj zo7;2UmDu{XgL5Y0mP}j=)=&$OFR7dNbB52Ehd|nhnY!7&iTCY>DGLN$ycsczTS6SN zbCoDptPsl@M1PuZDHM^ND@DZ&da}J*D%&gi|M(~!2s`XRw)Ez7cu^}l45a&f)YWS*KP##%)$(Rk3ISU@l*?1M~HzePGuR#NeKWV-I2lcyGN2XTe7=H)G zu~M-bb;iuakw_<=PnZDQNNC$)boKB?D(6*7-lLakL2Nk%?-D;@3nXTPu(Jiv@_Ub5 ztTDqBYk<<&OZ)^_RKJoIAC0s}BJDwg1<7j*L23k_CWJt^BjbWE-porlYsx7T)7~VAlFa4uiz-bby$R+hNQ83#G5Z!~82Nn8(AZeKE+) zDFOvT^;HJ>#J*IJdosuev8Yi?IIBLek28dn%aKT8->yjP&pig}&sg`p`2R=uy=h?G z*OC$KzJ6_5_dSiY{vBGTq2pfBJMN|Aj+@=JX;am5JiF4})jn>!>EcT`s@In_Ye z-|USOGSG!2^yDSlrmu)4tY1VFe-Bd~kh-4BrbBA8Eoe`$ulgKAq9Li=uFw0k$84x! zg}eOo9QJBMBqMLK~PS za`NkBbnvdQ&=+{JSb3%ftHWE|`i4H!zo4F7126JaU=BUd0x(NdUL;BE>UyLO$gp3k zN53=z8b_phTOM*pgFM3wi+JHPE8P!?pn52uA)BzlxOw^pX4af~k;wD&uag&#vxv{l zSm9wcGE|dTog}eh2tpWOmem|WcD)NjL$4cze@032@yOIw_^nx6SN_RhxVMY- zb`lQ}lY%z2u7x3TK>Gw!!Xx_MoQ(B9xtFDVL6VJcG7Tp&A}&6Ut^~XnH-sago!C*v z^`J@sEWLC_jHOrIj3L_QK8AiLSp)YZ6A(2i^|M`^!Ber)z|_h+Z)9kw%YZXNHB=DKqi?sHdo)a_f1l%4mvaq7%R zO^A!EH?o1hDoq>sgOHrsVDr5WOeJ&&U;%o(5k~RWG&YJcgLhk|KgVatCMO!+_bbfW z3$k`=KSX;ks-epb^}*~8J={KkHxA(cPe zbmnq#^Mg^DLqv006}A7I=v|a+aJ>XLdp4jSM?oVxhll_R8t?6x%r8Z6D7$<~h-{Y1 zE~$qaH1D!^iDFwYf4xmUhNb%M$0-jJ<)jUidKcQQ{V-k&x3~&$%Wqq662J zA1;N3S}> zX)y>JH7V$(xTFumtx)qgtQp-*!c|@6Z}KISn>6iN0oBgg0(62u&`teQzX6^2OGyr> z%>z&;p8vD;E|dl;zDalApt<_AV1pb6##Qbje;RuWYNS@uNO8I}#Yw@!TDnKI*ACs2 zW~gqSYYw0M3Hum!=Qq?HUyJ_n5fOJyD*0C%>C?YOQu+BR^ceMwHW~Qp)s)a__x-r& zx`F_F^SH0TF5QMX7MdakUJf975#ZPA!oxTPc&9QJz2q{4wh}AnF4~B}V$_{=+FWi|qUkm#s%la!KNslJ5eMhLh<+6$k{;r+<1g!+;kN|Ni< zKR_kna-h}R9SL*c*M_LO;DN7DSno(OiJ;eKsv;;lg||WKwL4Nwt{*V<104aW>@tYN zP3rM2Oa*$uA&1bJQt(W&TBw&{`Lm++5Y1Vle29#d@LhfC=V++F7f9oFOL9rBU4AYH z8cigT*ygk5A30W%2jqncX+b{E+%CcVCp3Ik9u+^>D_21#{Hphu zPMsRgy)yi#6#Ced$!vhE-pt50jJ9U}H-m%rI^?gqS}U+$)YIGasiB!nqV*x7eSx1d zD)2ys(SY3}T24{9n>BY5BBYs((o_N8>Weir(8gg`g*CuZt7Uu!fZ%ukq>BY<7(ccAN`>^?15sPfbw>g^p_Z28}>{-%-k&qlEQmD13> zLGk-F@NP|fpD@s1h^=R9ub_DY%h>Diu;j|BU2F#V+5}ShZ2Y7{1&sp9suH=1qr_U! z2J2CnXBLp~otphhDBR{6BR84f8h!Pg>u6iH=bURi7p$)S@Ujs;_V!#DSH6{Bvq|FL z-ZVE@wM*c|GyIwdcutJKn|s6B9ID#gb1e#~xItqqttAvucV_#wV@h4U4YB=OCCwTC z&bVzhIVq0(bQ>^bG%Km{BNnuqO@ohrg^wu4PiLQI!>8YApX}_@O!%}^`*bV&^bmYH zRn&t%4Ss}n??5!0wiZbg9`9L@W^mmqdTZ0r?-92XcVWg+kKn|0s-8lFdsuh?nxytY zc04d}5zF;NU);WH4Y;h!X|K%fZD4-6(T%@u{6`tLE)wMJg1is;vc6U61L>-CzsTDU z+hltUpnOliOb}Mn=8}e1QL`Q?xsK*j-E<4hr}3g9TD?b%qW3)Kyec5IH#vHxU7XX~ z;JB_R*P)2Onu>s$<~-8XkHPY2lh3G+ox%<7MK6eoDg3OcjCpB!Z*tNdqt5@t1Z^HO z8$bB7dYWj55*N7|^@9&hNXn}q`5sJG+)e6>|1dFK3S~HllC~IX78bXlsUo4mohPxv z%H?F_EK0FguFESpUu3j4kGwaB+Fs7)NmE?gw?4?)p3bHX;In=i!~ZUI?Z<@wd&>y_ zdq}f!cc3?}`r|(d|3Q$vWEtV#p{{%1q)k4;Ebdnuk4na{^mzUfzut~=$hDHZU7}7D zPxG@K^7FZ5W32*x>}#Z#GN#54RmyehuU4B3 z%=c6o^M+sVhGaiTok zYLkZ|KPLeT_&NT5#38o4?_QMeoy;};a(!2Zeh!(?HIUenBJn)dW~|e8wnxSL1~C4S zfmew29>y{L321+?lr?j(SfBRCr4%0ujTFz)Pd~Jmm4*apQA+zE2{pj9N<+6m3sa8I zY_!*s5O`WtdcqG3uZq(+%py(9^cS}ufO+Ni(nyPLFZ5@XzL*6V3?9@aJ|s);H-^&R zD(WVE%s@)tcz-wPBQMX#^h!s{bn-hv?h0#>hW1+3^aK-32P#E5-oSQrLO{P2n8DtT zvhsZOTuGe-iS)J;lfQGwXBe^*bY$0T z=?2+PYXR9z^_g!7*_g+F9AApkODQpXJFax~@^<99GfP8qb0kt4x~&J>K}cnPj-E#W zC6c5ND*rH)*UoFf!cyFf+>jZkuz40HQ@gX&_Z~KAhdErcybH{^?h$B$SFS)A&e@Mz zejk?jh2$M;a6b(4Vt5)t{Idnq+#iSS$`#&CMs%f#r~_C5tE22X$-CL)RGzb-iQY46 z$s=32DZ)t?*K zIUO`}hBqbH<758*paiXZz+ylZSuSmXbx$Qp%Kpujx-_&ugBp?p-*WYDIS0udYXIVz z&HQ=V)l*kc<1&~CV|Yi-UDb``HbZ{tTz0DtWS7QrjdU$fCXF&(dlZGpxE*V zekDM6{|Z{N3f4pQMk6(_2>LBn*QVW(2-USEJkH^N0v(prz!a>=y$YWnORNPqGDQEI zRNC)cMaM|m)PawquMpYuv8{2&16FhCZX~0-F8ocx{zaw5pz!X@f`IpY+By`r+(x+} zSss(*z38@7p;CSq4=~k&HuWwi4gwHq2YWAa?!I(Hk&ZZ_Bhe&F7?!7UOH>w$-_P17 z(Dd+jaxM|o$zo`P;O$IzKd1UOXu4#LfjPqAQ1LF%s%?R4Yw^&I0$=^bmUmwE$GITukd&Gc3%gwnJX8_>$=8!KHYsaZA~fjETy4?&{UKBdc_>dD|LU>=O7u9rbIoKe~#YOx6A&e zbD}=s6UjS!4)e~w-FyC4-Wilja-i76s2B5%8l0#bbN{Yh;-+LTafA?cYDNcbgb?*= z-ec6|*eo8AB^>luytFL$o^A*Q( zJ!8(|xG9ptS^PyV>MTyDDs1w3VM$Ie&xK*U51o8N@9N|~ZA>EnVPhBX?=M-N7_C1y zCeixu#xDNfztsvw$@_{iiM;Nv9^j8@g`yO_&zMBfqOLyRH*1BW-Z zD-=h|$2gr_DJW{^;;HDGjlSEDdoDcWHVfVzT-0m(&s@yQ`At^El&CX$71yO#U*O0i zIg?-dOOD)}H)bYum;1LdA8^52&wRkqu}kzeasd4HUh-LeN}+N_eL8?u{6Md0r}m_p z*Z4nWHA8)`wxj0DZ*cUAxqsuH1C+D%GLc_7!NS`civ2l2yU9`7{R_U$-mCe4+x#z@ zxhB5qd}m4@@R<*@&60!YHs>|y)G1#ON15_x6FR4SCZht;#s4b0_c_rmTa0g0`CZr^ zy-a5LFiOF-{D2T+cN<6?i;9_*0(VtJlYQdPkV2_kcCJ zr*}j>dr#EU8{1Ew-ev2Pt&`s(o_1I(?YuL)r%6Bd1r}w-{2fYb z=VvMQfBOQJ;r9JkHHVLq!7O|z@%*f>ZBZ`2W^h&W{Qln7N*Y1(A^BFhgIu=YKY72m zrDaS21*fsJ`g=Q#3yojAek2a(Z;VS;cRkUk>{8WtJC9t1u8JgXFNv&p7S2whu#L> zRzBF8DR^7P3Bi_hp~rrx$kwBdViIUI(e|)+Eb8NU9uq6<7=u2Ryl)BSVwIZtrFmpJ zq>6)32h~dz^`a3FrvkBk*`6-QFWem~G!^ zj>F8r0cP(YGpp%jm|YsvL72T4!z_(ox}>uk%wFj1idiZ9_Mpio@6^9Z3|n&y+dBzc zQ?NBx!?te>+f2Y#@O}r6-vQ!V3Guxf)8Y}g0^(uZZN$}9KHxplxweyB*DewI2jHuQ zemcK&+vRT1|M+rO^#63ZEBZ^=x0z;}d_ey;?=m$hE!L#RktMfs!PY@qlX7EC>Kkhk zv~H{5ZH32HXx(mV-Jy-?@vSqMd2bfBuFvIE{2Cd~H84jW^IoVdAE2N1=c_QJO7)ec zwHpu^q|N(&@r4KoPUS_QXmoP?TLHhV2&W3a6qj=@sN*s{)g_z`)<^+P z7`(xT^7{~;iSY8UIdEBT3!Ya&2z~Q1GEXyLnHsP-z(yUgiNd9n)~+K5Mt~kMmCvBK zDL8-~L4N7mI^^GJkaP1)nH>yr-#ExxEylQ7OnNP5f(TS00>jX91Q*_my&+-lL3v!} zB?d4z4q)FnfEsX99B{J^JdL1scfOU`H=MZiY@(&7)1mNoe$5cO{2dIefK6@%PUrN` z;GF(nMZ6aWyNjc~IPj$t4-24uK;YNhjh2^SONJDw1!b_cQtVM%R26yg5yrvvPIXU|6?4~j|eL2wCZgLbumR47kYeyMw~x;&Rh6b zk&8=(U-8SHf`_U`RPu$0ybXAh=+aoK|U+V=S5|3LBQ9>zZnR&qzQ{#0@R>@_xzyBq242dg&qwo%#Yx`VC1VN zL65M`s+d%L1$+)w(J6%n388$frIXb-^oV$|FKT7ick2nCaH^JU0ZwsEZbOssBv%qf~Jr0n}szJVaK2l@i ztG19za?yL-Bw}c*GZ@}IZoj4)gyP@@ix_DXBlYex$R4~HN+*Q-l;{(|xt?~&$0qq* zqXE-eXZ}T}za&!7rbCdo71q0YdbW8mL>!EFz+b2yaEH7jcq&6GJm=zzJUcw$tf+hp z(gI{yct<37zPI2XJwo))>y-R6JEh1DNj_Y;+99_}{zuFlk<<4PbWfeh19S|JW4Hr? zbB1GNdJSH6POoZLuZX-icrv>Po;vX<7oRQ%FANsF!Aq6j;5}D>{8cH$VcotytDa09 z0i!5qXskG9AN94(zlFHI_rld6rE}RsOJY92=e8Yu!Q~(Pyf|Ox~SoMrz;4#?B) z+gbJ07*Lr2r}1^>+YDh7)MBFpBk_DJ(Ox=jN_1rc#=a4auf>W>l2WJT9 zku^SG5tUK6p7m-g&dzqHseeKw0OD`pmn`-v_gAM(W+)eFC|~#9AVuwC7PYNC1lW!Bgnqxo!z`3>%C*q(-Om#aT^ zF}hEvuii}?>}ga&DK-nkCYV{ztYG`6INf67OTIJlT`4gq?w2VgmU#Zc-edBIeBRq= zkvp!Bbn3xgcsp?7LGyZu{xlbKV`;6p5*j$-=jh>O4kpJ#((MZvFG}ACusD?K&;zfZ zy8bd6Zv(DH6uzz5Wpc>Z0n5#^M8lkAs{Q|B)LldzxOa63=h>M&^i?zG(b)*^-3E6l zdQ|@-A%+~I@^tJ1&|kB5CZ;}5sSjZ406c=6s_X>p<+d7hacb$tet|>rG)4lX?a8t$ z@q95FT!QL9IsLqd8&5mj|Bskx@(e*oM>KV1P5glWDfy51;8}wn2M<_F+k`X@74ebx zueKN%Pg<{^vUq-`ft^8sRg?FfB3_Y!66B{imY1P3IC+bwsqb50*oGx}%L?2^DFV4` z-PgjvJtEq^f~wUfAIT5y3~|9=DDA1>w>GZDJ%C~kF@Hlf11j=%!20+KE=sjV2?o%3 zeodxT42I02zn=@>!)u>v=U(`1wt8E5_{pxe@6(|G$@Z#&pA#5&F@ivcM_92p^Gnee zJbZgWa~Kxqk#_+oy^y;6=V(~_$_Rz1odR$BHrx|h5>BIc-8L3?v>oZ`jSQM+Cewde zJ@m?qwqDw>P1t`Kp>LHj7ZDX*MC$Af24<|WH`vi-FhYLzxNG~QVe*p(b|0~Kv(*<^ zWO69CF_X&(eTz(+U;GBVx%~&rlZJ>pce2_;D-)|f-hWh-r`F&_RDeyuKs^TRpkgg8 z7}g%>9J)o6w@LCo=sCU$-S?arwU9Ig^x9JFR%tPc+(rD-Q^b<&HGuRu^2;0tvsjc% z8+bpS^Wr!3@}5uUUu{IUe#M%k7F)wji5#lfS&aALU-ECB_g*oazvEvD;%BzD$4FvqSCuym*Bk&y0Yyc%!R<`$OHx6{w=oF#&6h%UKgU@x!U2k?{)Z{0XkGo z9#YWa`_ba{p!#HgBF6|puJd+4BZ1d}g3-qNjxv||vF|0U-0bX<7shOi5;CfdOQ0oCq$eV$?Sd%f=r&=;QuA$ao^5c{|F`FFvPn&Q+ zpF`SV^}KH&Kmw(V_vM%kG~(=Z{=*?(kmLiC)&9LfBEAM8$moIZUWF}#F2};IYZ)4i z7?)H&h=-pdO(3j`N@)WQiYt3yUG|l@o6T2+O>F+jhtwlQbTSk(RpV#i>4oES`DKU6 zA!&d|41!K%dMI1Vem}*ws>jFn;ReHoi0<7#wLyJX0~NweBxnbs$5$P~4rJp5M~7WL zfem(>(GZGl#1T;tK~J!PMhrnP?MY6U0>k1#by`m1n7A>W#soGAx@Q&d^RlsV?cY+2 z4U9?<8bVen7=cQ~TDla})gqchonJKi{@+@ZFcMyQsmOQiCf!?f&b0q_&o*t=Tj(r&G?!mqfX3YianBaUGhcNWSA7KvBBY@>!IZJ~ zC3h|{L6t*B zUK*&8Cd3Vcoe?D!JRHd}y)u-KULtJ;%>Wiaxj#b}yoZb$AKRB}G@>jcjQNz;X<$2w(>|OrsofMUC(omoU$L zx$NZHghU|k5s>AmKTQ8qTvy)%5JUG7ir;%GR@409eGORISfh3>+~LO>k?!Mhe)7xE zOeA@?z>t2TM(gjnV_1L3^>wEMUB05-{aOa=ZkX^mZ~iGpw#B-8c^?X5p0(o7F_D+G zo0ROo{xJ^a-*y1Xf9jz8kN!CYseXpo&!h}#@Bh%taF1e6=)x*IbAb&7Q4Bb;K|N++ zs0kSL<#f`~6{zLHsuZ<2==^g$Vx8@PSQR9`FBSO5FC`qs5UeStP)S*=px^xQ>`c+UT`yEV@L328N3 z+6)|D^{<#}@xxu(;5(_lb|Vhb`k^r1XmJ5pm7g-sH3B4ofP66jER_nXvkGT(LADm@BMnCSrk9+?^ za8~{)Htt^}Aniz*58{Emh56rQ`G1vI@26A*!{(_k#TR)Ri_C@pw!mN2zaxqVt#C_W5u$j50jADze(5A{S^3IdY% zMaw1LM}A^>j{)`_27j;>j&NO5uuDR9q~3;-hP^?(hu3imI1zMfss|VjzV9FW?PRlvS`KtBgD=G z+~e`l-IAF531iIg5}IzL2tHc1^ZcO`s3FN))KB#2E7y?o0A9wXLK}GBC|o$ueOmBF zKy6+^PDv&-G*D%sbdq8d5N%%-iF_Vq`bUvlA;Yz!j#kaF?6@=WhuD~Shp8vW9+pp2_$4JX4@JNdoBV=V0Dj?mNisQ8I({#xMbd&j zgxy-(?}{t)B$ZL14EkG8@-=bEzoO*NF!@%5gSE#-P1RPI_Tf9A%<8ro{J-T9cXQJ;?*0*Mo{=-v??^L!4eemHb#HtFNas^+>KI@o*r( z@in_~!|s{?>KcpWZsfS0H$hLvjbKm3K7(r^o#m75Yw_v?O%iP=XePz@I73yjXHQmN z($;=pWlxDrkXkF|2SpR;4WMn2Wd}SKl!RZvB7v`Wo#6%1@vCPB>X5O z)vb(2b3P@a1D|>T=4W^^m5c--bf(3M_To41YW&37t2P^`}OM%JF$6owf z0sxr+{09N7M*y2(E}@)(IV%!!zDhaIW6l+rb0um&a>Xi&ej>ZqGDW-rI~>#rzk4&D zn87jDMHRmv6me9=A5d>J>#DeGfTyB{lwz2~t6T;3_RZ-1K`tN^^8fU2lKaen(>@JR z$ugn&Y_kZuIJR8J3S>+#8WU`11m0fHaHLs8;xfPCTNYt}X2aKsv%x9v6YifPOq&cy zzHCaE5UY&Q3Gv8px=silv3l;;fK|DXu=0ziVGPe5?y2}1ou${h??(GaLdlP*q!mk! z#FA*0@OE6r+`JD3Jalg}3w0R);I<`G_u@ImH)t_puJ%glYKmHd@#9Df0QI1ueZPQX z=#WEh#qlShH~Ed8yAAGZJqtKIhUKj_B7OKO6!e8iz2vRytlS^Mu($$74=r5*^npKj zjp%=qBHH-mFi0?g`PX<`ICqI20HdHy-c(f8=6WC$28ep&eo2k{*i6&* zrZ+7H^}g3QL(v)zlxds%9TvDQw4QWM=L$*wR?7d%$^FM6KU1T@<9#3%8}9Yzfp{{!vW?cJ~ZlRnSZfw~weWM?l&ed%fDo7QFd!JPDW$2kdw?zrndz^i%AiC|6b& zr_-+RCl2MWn81QFvP|k{K4wOmC@MazV|<<23T^ysJ{xyyLlQf>cHiq=r$R3z^5e^O zB2Pnk?J<`9+Sj{ovDs<-N3-9E+3_ZXZq65yF@3KjUv?-JBVgKRJCz|wW1K4^1dQtT zf}oU`CAQrY5R^$dvU3moWXnzv;U?uexFg66LhaPG7-iAD8iw6k+?fXP4v$3wbJzpY zms4)SNR?N8gF-S?B%gN5=c!uxBn*CFI_}k>)g)Y%f?950>yWnryIGA;6G1hpdzwLD zr5$-^K$0g}l&O0J`N14zY7_iqqb}sg{|+8=VY+iQi(UO-jI5UA10bvc`vUNQK)6IK z+XaF+K069`CFNLzQFw!|#S@sh*Zdv(V~?LRASVf7Cjmucw3oorn%d-mA1=9nY*ECOdO!k_F-D;d_f!01*WFlmc;piMP}(M{3b^xQtk9qJ z&5HQ)8>l>n-BV93O>rc$8-MDk{3TCai@R(nhTUOG5W^xirH4)4#>6RY4H4Rc>Db3E zD?C|v-^(Pa9s!nmo)gyhp|^v8UHExPxmU~cEi&50M-19lfj|P8yMgUa(SGSBl#GO_ zQ`A2mMZJHsm6n_f923NsG2{8s<;hDSDrceb#xAnnS{QZ7a^BhmY45L>sGbp2&#$Q- zH!in}mnUqovT9z%#s75$I9)$d^afMJjK^suI;`IG936yp-Q@5;5x{s>YOdQvrb3Po#$8%*7{KRB*JAVs#T`&5xwJFA^N=4ZcmZKW zKz*-2k$={*WCi3iB!4Z5+V|?@N7wcS)~9xinR_wW1WFlrWEzGk}2cAUdi=J0m3{0MaHS7nBcxEaEHLDgCG_(Ps_=JH^LN5cwBKFNr z`8ZTA+iQ~jc~a1iGsfr#YjJlZJhF#`#euYl$j%z~6cj~>gbzjit5Op0;u0X__Wopoo4P_>wovNdd%nrO95W4|minrPS!MO+Gj+R5V(n?9|Sdv-1 zRqiU%;t58z&4=NxaQ&CN%41Gi|1rn=7*i=B$6J)+0nG6+=8$3K@^u??i%vNKB!arq^=P901PUMGu?kaDL5>U9< z4Zi;DN}gnq|S zt<23GjkrpwA+(GbG@&UU1Oy|KhlcQ6qDEOQAv6OwVCBH|I5NB)u*eumvib%-qCNyF zy}x1))rXt2>fRyPAVhf>WOUuEIQN8x7&Y?)-H&huylu=#Zg=Pshez$v(+YI<206I+ z$T+ercSd-4bvCWlJY?!{Rq`SONu*w6>RLvX z@XK$83|yV;-UCYgP+FuT0qVZx7>fq<2co(9%XGZlB82R&0EDq|K*-4CTscTx6(UM^ z;yyWwHa-5m!~}xAWwWjNy3kh!Lck^;SEg=Ovv;E}#E-5|QKb10_1LMxx*fYn|ol34{q<0`sQImOAX+*Q<8pX(aTc#ZcwEa%^%3j)ZBcz?v= zBPh3q`{F3aCwoC7b( zweySAo!?$S7cjht*2KEIGCwF3?%y%gDkb=5rvB5>iBIe%6v0yeo*nYvPa zy-6#HK9#_?f!D+EYul9V%Argr(a-5w$V8epo4n{@P=#Dmz!*GL_renvxB~a8LH@VN zCjhA$b?A09a%i6}D9)AewMy-~9osU&N=_K{cGUYp==_I)0k|s9IioNH)QRs1i@lHDp=wHPKbZ7x{H3t(KrQ^k`DrPiF~0iS*ORbbZW<$4(JT~ zJ3jj!zG^oyzf++)>^vGAp@2rWgO5TRgDob}Ke|ZuyNx;#`Kp(gw=M=2b-T55z6SbM ziW?6r;SX3$#R@!&`^KX>^#iV{cq|!INrsDh;88|v;vcdpN^+E6Vf`Xp?RcGFv4vcW zVmh{%<;!yk;Kiq;J{8`6gAg!m+EU2)ulMQ$FwqT4_&@Jr-LJYD;W@3n` ziq$x6CW-!sE%t&on^N%)vFr+zGy90j+zxeemc^i$FdId1St)WISYC?Yg-k2d&vh7t zN8Oy-a1W?4JRRXkas~8D#E3+27h;Y6nQW5&3<%Kxd@~#QeSdA4KaGhvup;Uc@CeDd zIJgdButv4mg|PrDR;qWw1JYly@A28+!R_aBAmj?r#M98xLR(!n3r7Flm*_&!k~sWo zyVzS$zayD$51LVw@fO*>hpm#JJM3Z3vmk$~yO}r!&DtaQOHAI59Cqv4I?j-bw%NH0 z3YwWa*K4$3UG8*SK4T zf$wnxc-XF#Q>ezs;9QIeQ>|C9q(ayB>Z$& zP@_CC1eGhr#V-!UWzC)5HaswKtW2raN;-qE9Sjcqu zZg2%2>dt^aj9&g6I#I*la_w3F%ZMcI-%gpVOtL5wEb_MeTK?W3MTRSD5@94M<5Z;w zo&CtrP75R-8;HeAtH2Ox9R<5||MKjV{^g^{Fxif47KPNKV7n8n{D$5@ou14`_<)HC zP(4ItI>xbg8w;ArQ5<*3*TS-g_F`{fKRKsj;A$KU&HWj& zi%9S#H`1E^2=U6cO(51RWD~2#p^)kDtA`l>8fnKPgsAVcJ=lH^6Ay2YlyWNzOj(Oj zTqAW@XyrpFE!54~jW2g&PFR+-K&osxej)%I-C6iFG(xej;+IdOZC2VZ1AvRaBU7zh zuRcpa<8%IuHbCse!g&fK5T#g?w_=Zh_n|Q+bSJHg$DGRJ=<5GHEdC^}<>XN#H`8|i zf1Ju4$Y(+`SR2QKXzZ_ONVHdL4x!gSq_@|VXGMBO)LoY~v7lZ3+YBZaAf45Fu=Sb; zLM zm7E`_O?|l7NO3A-5yNb)VWTGbpZ>v$t$U(DTY9F>(A(oSU~`VECcQZkb^mnMoZa}c zUD*5thV(?X(CNv3t*hs4FOA6D9me3#I*)WhAfXs+9(v5XpJj z*RTn@At5O=g*p^^gN>H18Eys5VWQ&tA(lZMd?^yq8-cx&G=O&7eGxYg7^`(p8}&vf z=_l_4nTi3-Z7?#ghOtb3_>aj3^1?ukt;My$BBP#U3Y zi=EfH$S$Nrd7Ev679qcJ^@^SzjI?;hrn#r!!7|swqH?$Tay`0J&>6JwlcI7Bi&<6# zqLjDR;thcA)3baZ+VK&yGV@}QYlyUZGTPhVU)G+3^^2Upli^r^iSJ zbgoq<43$l>pvmf6??$WCzSf(x)fivihQ}w9{=Q6lKMH@}#u>%*_Qzd@S#i3c8F^ft zI^00fUd8f}sfp?!bZw zY19t+L}WM4pvR2tgjv=7CUzRAs=p?f+sGPG0W#b2`_WIXjpmIz8wr|6Ba!2MHYnb{ z2|YXbDwi=DYdd5%>DD6kbu-%xLo-!q8`}WripN1t!(*#a!vpKv5Y_TdLy;sFTz1}zWa!NvkQJrC;;dwfE_;tSTxm9~jwD~|{FOJ* z#S6+6KwtMwjJB@Px}zRUiJ`DqIt$&om0yKt#hq{0<~y|{E>lfdK$bB?{looii;V6J zA&O0Tm`X$&agx$guAIEVReKTqK~Vwqc~WK@H5~QuhH7EF)?v#EVc`+sM4k z&rR5%Wd6T+ZAfT0YrTqQ$Iv!|MrX$_V|1o_Zy25VD)lm%meetw8HP|ZykPr^)PLQ_ znBl!ph&gunvwlA(dAAKWBGkptYCC2pm8pBw;x=+rROUsSA5kkWQS$}5N^5FrD0e@L zcL`#UM= zH{^-2T?b?;NDW^7=VV5WfuJXc55*c6JH!14b}!BR7rO2jQ1c#nhBJiUOFWM&%BQ3I zKm31N+Josh@qJY^MxM9wYkKn=f^;Mo56HvAL7i9K53`FNkpD*-yd7!f(}6!tVJ!V+ zk!ynGUCTR4Uo(c&n&0NSQBTR#q4dL5ns%%`-r9k_=6w+8;U zY0s(be;9v()QtzUu&!EN*Jwh9t1pw}V&~Q)E#%$jxjW5$i)s(DL)Ja;6{P9xB1ujs zhyj6rvp(@YT4jO7Uqb5{9*=z}em&cW_G~o#!T)GO6#fm#@c%>QpX(O>*OY$h`_%C1@`KQQG5B>~B*FiR%JbdA|A5k$cLP6% z;Gg*-3V&8I{J&E94};yo_fh)rZs7MP_^-v_rzgX&VD#-4ekrBzXiW|O8aDo8@ZbGc z68x!*zJc!Gk7WJR4SW{wedoVo<3Abx&5XX?!p~y;(+&JwHvVJqkA0p5KaJ71Tlhx@ zQU83G8vfO6{Kw$Gk_`V_D!*f6cksWU^hw>oAHc@{=dtmh4F5AK|6I55-=y?YM^eKd z#KwONe%<;c_oN_nMZRZFfzfet;oW; zc(JS~Q#56`((CEQZ9V=@H2)Aie=gTa*8)&=4a60buQAZyF1$?hYNXQ7MMWaI zPbIofVszi;n&4fKZEzRqm%6vPt|#ZtDHd&m?KbuF89Yx>Y=Q5z!N%1~@;K^e@?KUZ zS_HXwH{qq2J*8~|XLe`cK8I-$UVM`3etEi|TvXEEV@8d43~0dL>rfV0VKht1gBDR4 z-3Pn#%J*<=x{K7lRk#~92^&>8K0I4eW{yCs%rkbtY8FhXJx|1|Sh%y#LT9MwXM&0~ zLGAmf5ly!Y!lHaApNp2Sf?WyoMJz1ta9xEl4ZXD^#KN7GCn4idtjHXGGx;uQZ9bNU zmY&Yg?%NXOo#Ofj^Y7cfs==O%0Up~=h;ow{RL6Wl^JsyqwV4JBi!Wlfg}owm!s1%bKRY8Z9DL0$_K0l9cSxeiq0G#- zDJ41n;&b9kady-DfB*YUu^NYPn*_f(dv@TnG=srkJ)rbW16|)e$X{ilmky&p^Zwuw zWB#_;+n&R-alzNng|@6woPB_SIv@r^0P5lpZkym=n!Vy`RHq=hkp+m}$^t~6znl*ux}f;xHesQ=%p*zrx>Ms0i37m9_NDor9GcNih;)7 z9tKXJ-<$x9bAsb6W9N87W)?u^+X!CDSx*- zDJN1VmuJf}Go>R?{I9TKIyk5f68u%^+Mq4=52y^b8io93VRkc(+gdhmn;o;88N_BW z7zT(J0b+Bp{||+1;5Li1n{nVu@&SpB>+5aYHk&{5VVk@Sa1DkB!F%wc(KJ5O2BUXB zjb2AtLqPy~k*~%*pV{|mWSD_u*ljairB78}o2W4|yw1qbOk`*VGVCTY?6xT{0vTQa zGCX)rEZgOneS%?q0;iJ{ybM@>UF^@f#WwppMu#`@sOdDj;4O!}0U;iF#z2%fEX+Qf zM2U+G=0!#cr0rp##4X8`I3xv6YLqx<^WP4XIHyzM98%&wpu~mzgNrxOL@wLv;C9Y` zz>(i*&!?^8w*t^Ie&r@Bu$tdvbMcNpTQlC+61Hk%2Wk-+iv9R8T^_6 zj8a|;|J>#>%KZ}}oyFuGyr-x0a{1k&x6UZaXK40&&l}5c64yZk=^qBLYd<2=rZ##> zi9?!zK^Qm;Kgwi^_;`W!B_o(vP%>2ewQ9fG9g`(m^urSEa!1YJm16%nm+hQWxH7!z z0ajHIe7qeac<%t(#oQvvUkV$sJ}JN6Sy(?WSMo1RH#;zX{|VK5BEw+kZV93PH7KRF{V^?U;syCZ@ z8d#jxNS)-pJYvphwmv!KcD)qtYyL5s+39zsbI!ufIg?m1@0KRXyLkkZ9bUebLWh+$ z;0-qmbkrL{88EPi6$GTkNcBn^Uve6c&)eh?;eHrz6l$HjBD@*DX|i6jqw$!#@S0Qq zNE`iRJqSPR@pPiP>3Bxm{go77bQfQB9U9#ubrdRF=TA0%&Ceqkc_?@!8v~2*ub7SW zwZ9Pj&SLMe6CNjU_8yz0VA$w@#mswVEC%O=RYcy_w%c=Ln5T8PWt-%mxCy`6ULBm_g5 z7~v20+ON+ID(chk1?+w;YiCGd9J*Uy6a3{L+^(gi7KNf=VT= zQoD7V(6&>sHrg!*Y*6oIA$TI)F7FgPN5)~ac89#X+j<|;>IEcgq26XzZ?ojTN2?bT zN@C5Dyqloj3s7$m>fI@{?Gmh8?Ut>!>IX79F<`ty4t7giQ6p{;N<(Fh3}&O`pRN%X z6QHsK42x|9ODApsh}*zQ1fh`__r(mXz$AJ%UFLlkDelR_j7#wO62+n@2!vikkuj5B z(>r*?V(ZZ;$c=c|D5&;z$aSK(4UpUplEparOxw;P&oLxP5q9UMFbNRd3I(laY?hq_uyD~1>Fhq5xAfFaIE#th$ zFzRv~l>~V!5EAGlc)lAhShkA(do@bcvTgzD+pIf*{0{%LFOmE#rNnOCt@pseuJzlj zd$sy6e>8>;h|Pl$nGU!>Yqb%%RrH<(G`6A`B=3Zg@g0qfos$1P5{(Gu0HDy+w#&NJ zX4#~VNS08-`UFr5>LaqqQzzK0+u4W|EN5(TBXn11oh>6WM)Z%)sLL=Uh2M?ir?~gN zSGf|k5J?Mb>n}IsCea1F^F^pPXY|B7r8R{PH}*WPA;3cH4-pID=qIZLe%7OtgId{| z0V6?>(-UGiv;E&dQCpn}ubCBtCygM8&WkM*|F74Fp=^C1hb>R#NCWS?K)kwEk}uJs zaS&Gq`77l7lHZnRMz0U#o*NA}7)UDzp*c<;F6KAbMjE^w*7AN5|91Uc9{GFWp*hFH zE*2l(voP;=sDV90qhCpkETp*a`nLyBRSUUa34V4 zCW98Kq4h=}$?K2hW>@Nd-*rD;u4n@O4T5sOyb&Inoc=j5)H=uEhqvHEgFg1~LleKI zL-f|SiNT1*jD~5`%ppU)r-zFXkik2~If3}zb7Q2)Hjoa-32l4rma`zJdTQ6%h@T>N z1AdBM%th9VMQ!_R)*VHb!x&6{EYAD$mIIL3IM21kYK$>G`>XkZL%9S(@SPdJD_( z%`ARS>)=`Z^UeQ(uQwanR~7SyFz=7>_5`wC26={S5p9?C@FqzMOsE2j9=bZzp`e0JWbA z!hg;6h`|`dK;WAJvr+k3!wH|eavBUZ9J@=B-Yod`srFV!|8K(U7C?Pivo+Y-H+Uqo z#{t39F@}GmHqdGkJm=vl5NI)4_7{0N@vEWPa{+!gSvFY?dM=n;qXB`ta%w?QNS7OW z?Dt&YFvNoE?y6nxNn$;2FB^g(JZ0Zh-IKBgE!B^4Jq}dgV+`k$UjnlbQYL#-wFNJ< z#z0cBVvwI*bkq@lww&a5#te}AxiYj!{sjR%&GkkAWb{ZEdtZWl?UXlbw{!p(Y1?e# zeT#`>?r_LQKobD=ohgZhJIkNN`?c^8;xt%0kdt0wob*Rvrg88m`g88o!e^>0E>aAM zA$U69Q3&)%$15}3eMGKS;a3=;7yd;K}$`HOpDK>pw7RFC#$0*!eAE>`wuq0?wC5D3ZStp^$%pVJtxiS0*CZ08ABSML`KYh45J zI_Ns{AatcqO5E>0PG4$iHP3_9{CD@$YJMC2Wc>kt*6%~fY%Nb z;j+Kb#b*lAQD2ma6t6ErB@PeIk3w7O zNEkVhoxXtkXQ*@uHoR1TLgT03Pqs%q929l$n26|9OpJxbPmbTOh1ra_CP{MCf7*eA zw{n2GX9!)XT_7qZhq7R(SYOOgW!F>`72!h~qSJ(^tSL64uILWpbq#Ha7gwade+ht; z4Ali3c9Cdo|G`^7K)qr}3hVx?6;XXG;wlEBuW-nqsw^6y<}rMS6TWl*Cw$Wh-!^wJ z^?mm#;BU76pTK{2{7=DW`w1h->4!EcZqi+0hLBIB-M4GeBLvGASd@Xc;c*j3=x&Yv ztg9wl40>13rLJCtUHxl`Gcc$6XwFlUVmaqfP7H?;-a{7;lWSu4_zeAD<0DT>#s~K< zRAX0sW_H2n&Hpt%**}fXs8}uM?@h)Bt^4X6u?s#s?oGz0CKWz$_Xo(kX>(>BEdh8J zgTsHld@rt|db|tVuc2F+=@zKU^~CYU@0M6gM(8cMQJ_nSXR8GWp{+H48!*k~b5W!npQ3deaeL-Eyv)JLLS}X) zRSAF(r?btd6nC$rxX*)KvVj*~`VVA~>*q*O>4iM&W>JwaNV_YK`fdU=w3LiLZSp4M zWzi0#se-WyohiHTlL=ar^?ZGuDLZd(K|ten9(_H;(KJXKN>}SgdHQVVg=KdYG)9`V zi;VqQ8yv}XwrTJO>G1oBerE@87N8cM8t!1HQ7|@1>bhQqx@Zh%r@ZeS>yw?AMB4mgRSndu z%7Lmb(J7~Fq)N1=cP6H+X<)2XEbC9uo9jyNds9QLw?*mw6Yz{(;dS1V5?*;CydLG* z3FG6s@@)BZ-1?)GWP4jU{7@8oHzvJXlHSen?>PzZ+xHG(HEQrZld^ln-)8~uJ>%aA z9v_3}`DOO$`jQ+&(qb5Q-%N&LNXGxPT#~I75|v3qmHV>2-|;;*$q!~LLXONZw6XDa zheq38KwA+XjFA>?Fn=H=6Z94ohi)k545cvS8Gl3=*J&^Y>;2PzL(gdc!2A5XXeZ6+ z0+Tvv*LZZa9WzNH3G8U#$3!|SBgNs09h*6O=PudlCIws!*_MN1g)S*h!K zMTidgl;fpjMACglqY~b6+gp^+hW(;X$$WQ%m=aKyL;u9rc5b*nH6LLn_=WpXN(8A@m2y zr?-7q`|iSne91vrl#lhckh~eGb^St8(h=qz`r3Q5;9Ueu(ZjJg+Q%YzGc&N)yJ-!+ zcN88l*&MzZGUH15w+XmXMpCRw1B+;-)Guj{UT_43<|G!1TcR41=_hxh>_Y<`hT|rT096nq87Y6W`$K+>eHTVF8tgqk znD^10cmAwnwuI>5?Cz**jM}YM4Yrg5wJ?@E|Nov?_Aiv74~FEjj^#A*C@HX3xhu>;_LBk@_pYrgV2z^ZeTxsW4zK8WO6074sKJr3 zqZq%(5K{l`+%-$KU`D7H?><0eLKq8K6K?re$p-5mJwPnn$G`eGJ9`OTKT36t#N+AC z22th+;8PSM&%KMBjlNTRZO<_1&Q2gOuc7+~>3mQq&Eoct5 zTKF}NtUY3nZE`RYa1@5-{IK{U3KG0`2#Yd!O)8Y}d{*e`%aKTmQTHTn>gOL6=~mxL z-rJjwIMGZSqqV_xJL8B!#Qkq{+UZV_I|MC=O6b58>1H08#kz4{AD-KF$^d!`1&cw= zxI3zzz|3?O9-TV|Bb?>$7st?j%EKkU&j91lQP?8!C7Y$iTkuVix6r_(csFY8ZOo98 z2}4pU{Fe8P#G$5)Q}4k9f7TU3ifM__g+hDgE)_*F8*~P6(5_bI7S9hKgA3b)o+w4!Uzp{ zFY1#mmrbP53-D#UKl8_q2;SVbTg5Y4TUf2f#-$iE_s~IT$}r~qIFd>>&`(we{H(`& zT-0?98HO(FImoMN@%*0|ZxYrk|CrlavkYqQ8%zcz?KC`6pwKZG(WtVQU*zPhX80)M z9(+&Ccs*oHabR**%)ZtpU)(quNp*Dr8yNS;JUB(|w@CZ_pR8=UHIA%pa;`&u-anOX3wyQfZ&ISyI3uEGKi_aONw#6YJCx^%5x#|CE z%z)WjkN%lkp+{Gq%`~X3C)kOp!F1`PrU{eT?8|u&TGbcMAOkk%74C4p|}9Z`X@#T z_w$YhU*jSHqheUZvc`CqTd{C!1uy#NaGufGJw^Gf`=oqUi+gs4t^r<@1FN4} zVhh@u5CgoLV*+hiU=Ij4lIJ=xO)Tn=7g)nrBjA2c`Oa`Q3K99I$v=A$RC5Q-x+q~_ zt`36X@htqk2!H=h`dfnM5#jGQ@OP?9{1zah`q&N3ynem74gs@>J`*j#IA|Y6F1G3$ zVYp_*)RGkU6*GJO-2jzsdUz|L^}|Vb$`{e)`~M^EP2i)bvW4-kbSLSAglhI6TY!iJ zH^QQ{Y$@nYs-PQCKyk^)8x(iN&6eE;9ypP{6?z!ijd+s@qn$89wWK*F` z9}($&=v0GthuHfNFwd&$yak(4)&vrDUZVz8t2u9Z4{tl|q zs5t3pk1n9oCJ>V*|LGi%&kxbjG)2C3^pX}G-1;PS|0sF~G-KSDE@-B9zcc>vb5Zn? zkuR(Psn^w*OC=%t_f!=93$~+wIDYaq@5(f&M!A4kDOcbr$HH^W6{GWJ3Lfn; z-Zqh!129loe%~zB)tbD#kLc@DtBZ2|09YEF?W$PgY8%9yaiLGzXqtnH`2n={`w;dU zy)UOJ69MdSEc8TqdYVOEngSolgm^MrC6Eu@iZlgH^xhxen>L5nA9E5i*pDVpL7u2h z64tUYHb*%X?u{fQ%{v{hKcopDa4ymZ>%>Yw8R6{1kN9Qh;!+F?SqXouwP*An?wDJ( z-`n619oK#gf4}|z=vr zb6+%+9v@0C>pVRvl>Y0^(-TAK%VN^!&Y`o!!Q!IBqIY(>Hu2<_DiJ0N5Mx#?GTNO8 zSRwg}DtG}0LcWGUETEdL8vZE1`I1p&VA1B_ZmHb&C06Q~F8T)b8WV#|?@^IMO2VyN zmOG62;~y(uorCj%1i1+I!;yV?fuuP-J$nLU6a!zfS6 zdXWpL#{^okL)oEiPh8EZig`2gXUsHH(AWmSb&|8nPo&sEgrk6ONHtF>&m-K6Y8#NI zD$i}EU8Ug~BEL>cIgcxM(W5C1Kj_da^Ajlz6ovNOCLpPGFoSQGlg||LmAr*k`IH8V zTGt6+Yd3WS4E7sWJtxSyDMHhBqw^;C*WegVU*qV8B`p6}MjLg(36K)!7zVFT;%g>m zbPmLitYJua5ED%BL&6`4@WVP92}i+qo6$60F%BbL-*?DFPT9m}Y^WsXrXw(jRm>$J zp`$6xFxq^Rcic&$W(1#3(nj2>l#L|%wE0e=PhZ|i^l2kKtlx!c)FtK3%=UP9N~^&E zH3Z*)!vAq+v^e`1WjcaT-v>Dwo!soY2QoWc^dM*0MA1B zyGQ+h_m|#DePi0#eiIgNWBJ>0vHQzN+F$yH_}f%P{vt8V(RQ)hly-+&vn{!t@d0)Pr0B}wo{~#;vDj;i(akJf!X9ycaSF?O&@_S;wVIq zx1iMP+=z-+Rq1z%9Q>A|9k3h-G^DwX@$$C7nSs;zvflT2=5L2*irHh|fr%z9vA63u zuccRL>6Msn)Y4NT=~Y_#PE0pIy7pe}s>h#(2_EaN#g*3G>FJx33FV#Jl||o^F`DNA z7+`PSU#9bAD4@#$5o*wiZ;uq;QDLn}cU|KhLugZQT}yPPS6D04X@La$L}>^r;p;1G zXYf*3ukA|9cdV(i@R7A6Z!6qZ|JOSTul{U#oE-34|KhR5uK_63ayjoWqH105x@K}_ zLS5@#Nych6_hewNQU0RUg|ky>#XzIzQXCoAEBvaJDB;$z+pm?@?dkAeg|+GxfO~na zKwe&K0FPW7wO?fIbhYsO+X3IaB)zL8QQ$2Y7ONEHWr3PnVFvJ2zij=BtHqRGBV#C? zH5HclHLv)lbND0xUFC7|VTFeha)8j;z4Il!?CVXP{@;jfN#aC%jWYuXgc!vlGj`IO zIwX?E@ttD+JW&^OKRtrKMQ0VF=g2t}`yCVbOgAnpKGQdv9;>tI@!lMIe6)}rpDdxr zhC+I5DaDfj*_TESw+T-wWS@~9t95j45&zboBMZdmZY0lDR=?=8nke2PIjLuto~-Ld z-E2e>( zljTooL)h^@wvCMf6H^($Hf-_*m^EzI1npv&<#%axx2&U{o`?B);V}7>#>1g+2jF>5 z{6~LcJ)LN+af}E)A^J8@XJCImtOzVOjb4^HWR%Rc;^|Z1V zFefK3kZmu<8&T4ZMD%W5^S7w<5kjGb+mv5vtzZCkK#IRqTVUkg{Cl{ShVQABe$dLR ztd$h}{9o`nipz_(PYvB~fepgdmi+3yuC{KDhd`cmwLRr{(AD;&^F;?Ogbl4l@L{muSOVnh( z=n;zM2Q?NZa{GmCVeAr2cr%RES3s+`$L@ziBOR00x9?b&8lLw1;e>GffJ$^btWu~@ z>st83=X#*PM5BE3*TNFS;JtW3OAaU{>xpNcf^o(ux6A(a-D> z?fInkT&O)4XwNy?Q`DZ>+H->T9IZV^YR`1-nWjBW+S8~#X%@V5O(BQr=}0{WLYKVL zP3X@-kQ2Nw8@x-6-nl8tXuvQ)3g%+a?SBdV@a2tYKR7dRaQ;463nbk4vqa6`-h>$U z-5vF5xZbnsG}V8WUwQLWoP!l3ntuY-hx@)`_|a^zEt)2D56EA(OWy;%GrTYZV{MAZ`Os^nYfQq)tx zSJx_irZ8pb!}QiXS*I(UpyNH{B^ST^z`i*1=*dKuzKM2_8+kqRXMN+dI>u1F175a%D^-<^l{MSt|;;ul32~!3LI1Na*N+70gT5#1xQv(Zc?R z;nD3bqjo3a$P5^$#WYMO+7W@@LJ6?cy|}1FN$DM@3FmzAKr)SZCBpZQISf@B&L7q~1TKuu=F5QfhJvGHzZg)EKQ`z*dREks z=$Wn+HDJ-I=%OQ`=r?jzQVkTX`oAlh{;wBJ``3$_{`I29f2F9T?u24{sMyfhVyCEB zf>vxZ^O&=fe=8F>v}pnfg5Dh#5z z%F$Ec%Lek^N8X(@6c9H6d)#rT$&#tiDcMvsXTsb4R(8a83*NQDKyS*<0IZ4T?}efp zP+xk%kYKh__EnRL?saih7|=W?ocszNl8#T0jKX3X1tfLKoHW>GciQ!K zBa8!Wb)Vzk#2Czyd>A@v&DzQIZ-VPa4}q9$D{tE&_(V=zv{zd6qnLR>RI6=VTULyR zEuF}3?T%M2#me#;7?nxM`NQx^CcOR$Ula0s;hjx*-AS*!3nPJx+%7S2+8`z!#S7Ca ztoD5?%Lp&zNmY^>ux9Sy-^wSRe!q31EhFXC+k;DyoJ`a?ld&gv!FXsVWYLGHJ1O%@ z(JI^U%=WbM{%@O9%Kv-pxzp&iek;h943kYB?zPqgw*vQk+YsWxFAXLh{PDI#-R8G( z@ZS2ESoBnuyq_f}!`tTHVe*oX6Fc>HJ%Mgo7p&b%j!{2NV=)h^*uJ*bnsp8+wwS@b zZqPAr6KmdQ^YVk1Yb$O5%CHQIErJPeXc_;pjc8s^%5*zsx-p!n@bmZu1IA$pU}ar@ z`uL#ewb!6ykNhcF{;YkPKP9iT3ed8z$X(>iY(~|S(wEHI^!6IhBSsY+HDM`vkGS_d z-tRSE%HNF~&-TqtWYHEJM*$a7w+(8)@GyX?@_iqUZ0}ZMc7RFwDow{s_oJB`(sq8? z#B}`Z?#T{O&hukQ?|U>D?i41{iLksB*)Y7aMES$8W>v({U++^WhYQL=$SRkk#vJcz zbq9{u?rs8>((fn>KF@ftQuKtg>FeF>Ke723hG3K%14^8$zRhPSw8*WQfx?}m`Lc@y z?kB_8)S#<-DR5p7oaozr$)zUo?*s(b^`6V$Ggb-ecDvkM?00?;d z%Ae?wx)UCo`{9t}d>CzV)ZsoOib?6fftMs?)gkqHk*JXh1gM9k?qH%vQdS*BsLS$W zsPz`HavL2Pd*hG1o1)u<4w57G#=dd=Y`&j!Vo@8A$(4>CK!oP?;k=I0NjCZT&^^v0 zHtsaaFqxuU00RyKX|1w*@5ZC^LC7e~+=5G{pvLMxxpGtkYEtBD6vSD{+K5DmZOyxg zFQHugn=6UL>=vK8c=@L4XGHlDb79psoIBv_-Q??if9Bq;=z=wK50G!mMQaxDR1tVn zemj%jI&U&w&zw=CR=C=FIWy5JHZt*P<602ybF|nLNo!~o+zrfh9gr@(FS!XJ|LZ#l zd9xz@mMwNfS3%A(v1_KI@y4ZG4w70Oeh^*2U6M3==8GmZ#M^a!|CIczA^WCQWcdpx z-i}xAdF$oEN&3;w!6i9dO8TS~&OYc-2kZPb#k6wvfUW|U2_NFg_QG+2Syn@NXJ3rF z@cz35YO>@_(46(fdR}gCZ^I;Z*Q+KS`f3>e{7pt($mm=7&z&l9U#?b;cYnnk*(;N& z0m@v`ni$Facy#8sw9M&$q^0-fFYtYlg4)fS2QKP{Vc%wWEefzu705Wr7n2S-M%cJh z;oAD`#Mo$1(Ac*?F3RC)E|tsZyj(^XV)u*#krY=ft4wb_2kR3{;bWy4}RO(Ni zl`@7(WyF@6#Y#DscPL@#qQpS0gf*swtDXmW;gDk>h>_eW`V=m4W(OnY+J;JBxDI1bDqc) zbEC>&BbE;XkLz2ICON+QTsd=InC3}lKTMt(oaYyD zo`=Z!rlDe*0k7nXK1ZAZJNh31+~MnB3-ace(AT_8J|+~}G7N>bsGB8S*oV;W*rs2O zLE`W4!;acx#CSC8R-R7<4!4!o#WlVRps!Ek%Vu*TITv+UC*@yE}b zRNoTXZf3|WYH$UB`C0Ey!2ArstW4o_!TCz#dRFTw*hLw8f_#e`EBWV!CN($~tsrSn zy8o|DYRJ%FL2R{*+4=&jU6K+jhO);MZ>C4?~Nko*jo5yQi#A2DDfy z`ey3I&5;gPiOt6~mwb9E|3N3^>6l;%mY*8wAzEN!4;ScJ564Jyj?pf^YQmsY=A%n{ zU_6M-Hb;`Af7Pgb{u?wm7#w>%0L_i-q&XT04x6j#WWDA!G_^(BZ$!!07%;Y6~KUGf$fk$defa+TWh zEqE8ZmKPIZB>Jr8e;DvFois=Tt{S1i&(H>g`&M$DypBzyImBr$E)=!|_;V|a`Qs*x z!->`n?#F>uZ+M<$gw$G77i0So)ZB2NW`7krKT#d`qJ>nq-=Yo|>+iR?R4AM@LOpP3@!!|>oE+$KB?8{9&UMq z$Q=>6qp-a=jlm@LQh}!+&B|n$3y}*)X^h;5>}Lfjy{>So&e5A{suazKosGdc=ugHU zl+g2T<{|X>ixU0aA(%-;1^}c_^B58(08`xw8Mx)-2qv?}#sLH3cg6>k=vO5GVZ(jA zS=q2A8jjB79THv8t+EB`c-N@I}donJ3Dqd$s5yi#1CDBS=q=I?9)GCek< z<{U04AyJrsq>rxu!Uzu+y-r10_-pjXZLNotU@!QgNOFQ$X;o-DG-?jIr%>IW-QaYP zGpxo2DSuQdN%>>wkva<=n+IVB7rxDoY-8=UMu%wHb?C`a46CO6w~lETFq&y$!BL&# zKTtsxe>16m)??JQmlCM2lhBuj*U(LgM5f`#sIp%Cq$)3G zkhLOYQnC}#I3hR{+r>~I_FQ+nao54Ep_uZ&g;9do?a z-6#m8JI3A&xN+3gGI)9NrZm$lKjS|LEOIhl45@oY(UIkXg93v-Hk?80 zeg<)ibP5$Zj*0VDJZx|5aDZ1By`IAH+GYE8bQdVkUqfh|&bt@Hk)jVZ!9r0B8`?2( zjHlgd-7Z^yr=pIypp@NTKdRIY(?*VqV~15f=Q@dyQn_gdr*Tt?0uLH7WbL;ecawcfJ^D1VM9SC{5d1oXp2?G9RVS2ei*Lo6ec6f18*( zb!aP^H8Dw`tLE(MG3DJ`7MUh>FJfzzB9@TOC3Z zQi@b?DaYmneXA9rx-p%_OTPfS&MhOfP{M2i*DDo6*m3gxHs5ndKnfQ}=m*BTx5&qp zOE(iZPvZ)DZHO*(DZ{I^X~PX?n_^3)Towdp39HrlCBekB>wX2VnIfA)XaLIviEaYs zq}!~hJLj0zwi^~xzT`9o_=4DJ$xNhanM#k;-@;?_ah#TZZ_>F#GD}Wp$>T7&9+O*q zlpIW8URqhQ8Ix(BJ>-kB9ShqJ{ZD>YJG(D8FXRL1rwb!%u%L;<;EzC;am$UzgRF`F zNJns{G%=-=9<%;b7p>`{>(=>aZxX@Wr=l-BHQFkgb5(5igzDeiEC|sGQL;o^U6)!gA zov66*855forl}X3rZjq_-Ug4&_aSs+-iYzX&BhzXuW%()zKnWGd+PhdEI1w$y?SJg z94X|G5pT0524~~%MA6HlVtOay*u!hCk@y)lIRNkg0{9LBI5|sI>aH+&_g@La+nKwM z$qV8M^gMc`z8D_@eYrRKIHLpo7ztb)un4q0^^Ze4js7~`&9m(cMQ}R&Y~{$O_!DUq zt2zj8HvJPO_QcYVj29}ig)!Tb2I-sQZcCr`9a;Xp2GTKGEB9*&p97#$O|RuD9J^pte7;V zVVahtV@a#X<`t8MH{8OKkT&zHw&LKeFCr@q)Z!I`K&taiQ^iFsBGflkw--8Y(a$kiJd=7z zfeS|ZINivX%}Eo@d-IQ=N-YThgw@6c`-CsHYnQUNGIVR~;Amv>Ks_Oy5=Tgv25+0u z;9ZH6!|`D7Vt}(Jg={j4`q?JFEEfYHY2L~v$FmwfW;N)rhQX*S^LrxawXBZ5eA(Q}Q6W!+8yoQ03rt zRn@pwL>ZfRIN?2>!`r>jW}r(F)iGc4**Uz@ONcTXJv;-ZfUMgaFZE(17s8Tl_>jDl zGSEM(^4!I+RE9*rUL(o243p%YmthorGq@m+4xwQVX5i_%O+JIk#tMT;CspQX7H>J4 z#aoVM@y0@F19Hg;HhHPhEHdX5i0g6?I^*}l{l6OSe^pff zzl!Sr^>F_`X8qUyU-$pQPxW6P+y4rSDNgsV4uZaGOz5~Dge-usD|=gU6i6odj`oAR zjUYJ*f%9n=`C8Elpu|If1#n#(!mmPu2jvf~Y&T&3d>*f|dowa<+}@|tH0+lP-EHqD zE^tP8?eMSRB~mtMnvn$?I7vQeR$Y8L+n~j#F>5p|^tijjO!X zYNl@Ruc60=AnXQu!F8(Mn$(6j>TysA*S#x|GBn==9Lf9Zg`ip<=y zJesw8!%!cvv86^2oV6yUHOOY|Hk!4i;aLl2uqh*1zP%SMoWh_2fFguQfMJDh6e|Ud zlNB^hR?s+Ej#eOcUgjib&GZs*%|aT-F(b6#?>g|00wXq0H{miniSY+S?*Q!Z(_ti) zqPsecx_Vcqi-Wq_rgYc3+Jn0Ka-^$C)Sn(&e{jULFza%Cl*fb)=K&H0_d_VR{}?6) zs|gJU(2^qSFz-yGC|B9ImZb^Q%yJR!ugx{fW)Z|e(U;N_Iv~lr0n~b6IOu^8pHdz< z!yvApo5BF}A`{j(!)Gw#c@G;rJa+3HOveC+(ckb0;TsAacvZP)RbpqZVms__q5fRC zqV>nITpRTlkRyt8Vj+M5ov2ziCe(@ENif)w{4MK5byO$bX$W_MdVs7w<FFA_pDs z9yDmRM>^0E+79+<0PjCFcr917EeVASg7*=d{u!eN>>R`q$-E;1Y>LiiZh#TD^V5ES zv8M+E_KXq>EZZ9pR%kt0qT&0EWuc*m{Xc~Dlc-0he`L@ z=ef<)QFQ9k-+@5FtFr5o8ZweDelrU5SA{U&)p~t-zUaGG&mi8WLHsDr0pPkM?*ULR zL09O(AYOc$L3~dbVyFnJ&>)@*5MTAoGsbxwM$RI)1$k)M$DAkGydcNP$E5w|T?OCj7|%zRhUJZ_n$?6A6_arL+**<#m($)l zU)x$|g|}8FKHf{>BVT+M7v5OWizYwq5aR|9G12jeGXDoor%B<-q?5DafvAaUM*5u+ zf)S4?QDme|Q>~O-+gl`;YUEc<$79QXcIc0_xQU%zKyOKIgm?K?I^Bd$oxc$3)MvV= zPW_iI(kVwlbcY^h^6*nX*%{ACtuqdxLq`T{qaFkM>!>Ro80s4Cwhf*(ge*9KMDk#7 zoD7PCEP>~Dl-g4AjcoQXgHiJ7){R#hD$X`dFFs*0z)mH#*r%OL1 zVUdD#{&|M9AG2v$Pt)=nnif0TY#T%B3KyF!+HmKD)D>*UMYCXjS|hAqd}wNl8bvwI z*6lyo@i+36HK?J$x?$tZq9bv4lamh)z%tsaw%gKy{qA2R;z^J4={XH@?Yhm#Sd+}o z{IfEv3!if%8CWT=T-LZBzPJ?Lo9nD;=U@ajDs#^f7x*g7`bOL&oc}UY)OzeEnWBMV zrsy^k+VUJet3f32tGBa@UL!jdo;&T|1WA*hZYaMz%jWP!1KwkPd_6lFLuTc5Bi)WZ zh(ow5iRs2?vI7YlD$RWb2eazf`0-`dgOl!X(w(4EBhzo^<3~AA7^o?1P2ykgVW4a5 zs{_dFzr9V|`WyCNf607>fBmvvr%SKq-9bIZVLdt1m+~QB=HGlLVp`@lKn?!&8nhqv zJ;v3r6MSVtQuyu>?|vCMb|dZ>f6{Abo&dShTLb%TJst2N*k~ji18;pIgLi2xy#JvO z!>hJ~x0!!^x(4q=f>#B2|Ir@abb|L4f|r27@eBj}i!i_~adg?-bv=%M{X9C9U5%#U zxA5--;PDy%`eFE%Ud6ljk-+-gx|u%1hhm^!FL9NQ9wbn;2B~x+vTVXuxz9?mvU}VDW|3g+YDcagfU9`(Za(ha>UY8HNf^#-Uht7kSY@ z+fdP{HjT8Ggz-=C1}T#`#HLJ4|w?a zLFPqPxjB(dH%he39z1HETjD##%a^#Hn&*c(HqUdpF6Vgy7nx@SVr@dFc-ezv(!fto zS&BAg4X!{dUNS%l`lM{%9VJAEr|X~+O^|VtLX-AKBDSFk`>QT?!US!?exv*8347vu zMv)x<%gM6u2B2xOrr}{MovSq~MTU?k6Sp6yig(Y(eKb+1$N|=n-NTOF9w~27q2%DP zF8C)AO-v7-X6U*F@rgVOyOYS`I{|$J3VhkDG|@MyyXc)`6f>{#tG-4A)sGgrJ)$qK zl>DLT*CFkKU(T0&ZJQBD-)M@`g#FQu=D0l3}!pg-%JJ%bcJAo2yIVXhG+ ze6Mr8@>V_?^yIA-)pE(T!(=m`;8*oWHj2$59nv6*N%~pj<5AP~KcnH%M5~`}KzZ|O z99BGo2%?sM{9|YEk(v?23t{N7_|+xo{7Yq$tQd2fsXUm1%v-1$QT|489O4LJpM_^| z7gXz^`o^cSYFEL7of_P-gbo$#Ectu`C5xC$=N3kmJcA{F!;&*GnT{kHUg*T1akRYu zBWhlW%HWOW3(D{4XQgBYalKsPryW6_VL5%ICGspStvW|f;w~!C#w+thyk_p~k+n0+ zZ#TE`zoIB}yd}_1qK}+6%Xf?N1ICCyGVg4(tIE8XpCWR1<1~f^`cS-&cKQHx0Ng?T z3~@r&Meh`)cWwCiQeualCHwSba+eBhcf>bBSN2Zf_&>~e zEMEEeARU^D-laz6rh~Z76JS{)pBj*>wD3G3ed+^cfsBU7`QloKzI{{yrt#Ea2r zj32SLgRkZXjLyDVc4ZRuOH&WL?GO1+*5ucLs3`H>%ZYM7yqYKH8cR4&f;N`SX?65? zqn+STTsOWsaahC^Xv4ARztAlX_>GI@P|1$*Ekfn>`FF7LdRG1hl`k^}j^R-smsA}% z4nMNvl4`=0#gsSX4`SsFto(H>@73{D;p{Q}>ka)Yv2dy-o(xF^1WP+|y{(Du81j&9^&u0omie4;MO;yrpi z+&eMcRX>|=8fSy{)SYZJdSbQxx`d+BcqQTananDT!wR+evG8Td5{>YEoGq#Pr)w{5 zH(xVsUf%8#^sGLbFDqZxLxPC{8h9^MxKG45-h7!9CnfE*W$w)H2W+Y_Qn?M-h4%V3 z%{~(^|JxN1B;&`?aAGC-}9W^H(Q0Cjj+fkTqlOOt_^WM zizc-(_>DKni|?k*?T8pSZ|2>jkOk$-@{sH~lZC=-g?w>ew!=y|$B^w*qD8KAomeR* zUA7k2t@si)?;6oJ^O#uFTCb`@mL-vwJNzoRPSy*7lW9WIcZgMbi#&{4EGHGkPHG1@ z0~mL#=pKpNMb=R;kT7sYoJevxv|S*?D9iwZh|WW0LO|}qCzwJ zTTXY3g$kW_gldBlMjS1wP0IWxKNnC4IDkZ(yb^1Y0$=Tbr3!_EJEis8on_6ZSdcSvZoK7MY@}Sr>qirD;kxa1ca_- z2#svXTQ``&r^Q@w(utT|8^Y`XonZFJ@STat&z>OMPNzp|x*u>GPXLt}PbTU*+LtrE z#eM<>n&2&s0Lm4%w){#m0RbC@IqwSpb|5{#mj$S`Fg>!t9%F|QdKnfwU=(MOD_y6D zxYQxed0Bq4ip&c8`;^Sp7*FS_7&v2gijaF?5{4j3Ks1Y5?)F zTJ+5hA@}lLf~y{Ix*l-4CCe|E8);)=+f6imNN3JLYN0P@p)WNWotSu?#&o5t;B1`p zCA3mT<7?4elfMG^rCMMjVcnpqI&h%%+K%qHx1>k$z7;dc=Sy<&K4(H92!Ah4v=VpU z6un=IFr|SD6P!bktch|jzHCyKm~<6>pL1zVE03_4w^N~eoYPEr>b2J+%G@pLVRB{sgAcA&&UsS_Z0&7|W%3x}XOrGF61}Jk3Q09<8>(pw#=*%V3{Wy~DQ!t3%g$EIZmEIH^*D=~+ zc7Ep^MH)n}ErZtHtrBO*phHl$u)ho)*XgZtVrbwZhHLdlY4F~rN9x;Og$8fyW6}1m zA>AoMTD0susjO$0v>$L4oYXn)M=p3Jh+ddh&*+4z(TOL>8o)8vRZwqm+N0^ja~hox zT?I8d=a}$Wm~&|GX1olZ9x{T8F}*`=XoA;zQV1QCMUj@%N2ohHj<9VL9-H@KcP=a< z!Qc{0{+K0y{AH-OHH%{XQ)vE}zL9Gf8rNWt<$&_l4W^oK+(NzXkG_LYQ=P$wg|)4+iv`l&*5jl~^s4wo+m8uS={+FZS?Z}S1ZEQQZ5 zJe8QumqC;K2YdPd^hf&x4&5gpF(v-{1yLRw3p`uX0UTxYw@jADXid@)$f8OUoN_?&B>nv z!~4{CIJ^ZOc0y*XXYJxZi>~tZs9+L(eda-?t!X;u+ZLGZLk-B`@MYysRei0)$mgkl z8!1;c9rsv2gWMw^Hx`VPP}0BQ=<@B=MXWW=WcxQqS##G&?*N^@1A#bkL=8Jt8I|`L zUj{;WpS^X_AOw3yf#d7*d0is8bl$XwHS??3X2xjEysc56`$#`c+Now_OAwR;^5CP`W9)?PUA}0^>;f=(q!4VZjuNzSS$xjcm<+`ra27NulUXZMZ+; zHFQvL6xC;kEaY}{cO=otx4RUe9DFp!zhsO*lJLc=gG%8=sB9nh887VfW%h%`J6EK_ zmPJP`4d}^Jt^(#@vL9ZZX*SteRcWn3W^UsUT$IdEWd`PHoLnEB2QRs7Pa-hvaf-73w9e(JqMCU3c6;_BAqM!<5FZWB#I61jYG~1yG>IxRn??hK1dD} z9cxo{x}sM67j-<2B$;by!2`v8a!otOAt8Hl+*O*DDGa8b08_K9|KG;<2$gSA?r01d z3pmE|U)Y=E>Z0Si7J%idt`?vj;`=2*do?}3w_NQy-d1!KVRR(Q-J;v-Bwn|CS-FqF zX7TDL>vTPI{C@`IZAHhar&J5SdRuyRXgG%O?*!8K-mC*+JsFz@_LSTMwC@eo&&;Ej zCTaw-NX6-f*SOFt&MJ8qbk%mIf6)AsHd1pi+pNW6^j0Nb_PGe6mUMm$6?_=SguZ6{+V|Yzmgd@OJg}!uphb-^WxIegDLDb`I=O=KDf> zR0}^ce(9iYI`l9qdUMiflgBd;+`dplJIl!QL3O*p_UYg};!&T2HEX`2iSI9u4VHCoa#13CK-D z|Ic$m&Wc-{34o<@V9)sA!ALw`6!yIFPt6I>r)X|NuVjXPqPcNd68z{Znp>7nvcPN! zUY!h_tvA~sTNQX1W$RfROsbOq>v(jGJiJ|7>Co2Q4-#)X@>}BF#?T|x3XjdRaBIJ^ zAacp6y?^YDBB|li}Dr`56Zkw7#(y(?Os8LoLffXwruvdsG@5NHV6CGmN|64 z@sow9k0?&2qgKjSY4y+o-zRzrCWYJ4LJY-Wa=%3PB3%U@V0tJ!@IIxEzTWw7Xwkd& zYq`srvnHE-Ui7ZUwvp- zAKsN${pNu4crYnTWj_yQH31toKKN8tQ>V2;Vp`CaSpiQ|&=P4w-KF{`THkPFp-eVR z4q*7wQo++*QY&27IA?vZZ##RatS0W?h`6I#v zH{ptoMbkJP8q@8>M$gRtH9=gwc9?d^CmWL5w9T&zW zJ6`{xk~9H-*Gj*}{V(QQz8p2z4{C7PIGSr@x3syQs?|A$(=Ot&xfZq01Gvas|4}>N z(dOGOUvbQ5Q_dDlXLIDu^OS1yJ?H;wzJIbx$kshLjb=U}c+bB--!z%pl=JRUrJtfN{>Q@T_(8t-+l!*! z(HF!{@(C*P)_~}0=|OH%u->e?f1uCfu)8Cagp}HHVClz`kBoHZ?V@is>@WAjCVwwb z&-;OT-fI%QAjB*)qHV7z_wc+vCr%gRtR>d}b$kC;d>IuGU6sZ;sxw=4PEehrRp&_6 znXWq1RHsR~i!s01j31B64_g6jpgFlhJpyh-2JsjW$^oiS!9@${U+PwBc0S#w$l3C?f^@m&y zKj5j2TqC&Z7fL>{UL@DS@jwd%IUu->akPU|Fgc&yKg&08sOa@!!!vv-X4p6v(9hoY zu;b#)3dv{5QnQ*Cm*QL|UFJ=eF*Ur&Hm068%^Ot8KL9V5L2LO30(_5oSv7nQOIAJR zY2-~QV+6=Ewh{8c55rh3Pb1&MkR`e*vPG9a&iN-hNN{&ZKEpU0*M`~df`4pxnV@96 zR(m|wEo$3)jozi6(K$P9%=D#9!5dJnN81?GIC+w~J<4B= z<>t4hk1J^qb)pEI^xgHKXYB(`{aB zHgK~OMC8nT@!zaS)iq@&7r&ae*6!^l;WZA~x&|hoR0}!;%_4AuR{-YtE6K}2`sX(JIlm|uRM~uX zX!4*keRsSL4Km#hR28{9_a|F)RwR2S-cZ~R)m8#2FGRB=EyU?W*I$XXzTlKdSSm@L3F@~Z(GL|!_6vf?N1 zgd`skT_+;79)?fflgC=kex-S=yV$REk99ZuHPU1Kn*AE> zvF<_Fq#pY>qVGL|%U4iE%rkRaqXej*7ekddhAVFjSKb(|yfIvPV{GNoHNNmP^OWb^ zfRmw^UZof`L#Icg%k%_#i&Ge_t;@6}>N@ymiri1|#@A1aD1Tk8Db7hCq@+Q0qn-EJ zeIO?NaY{!|APsmE(~dUa z=o5&-EL%sy+JcQSew>pnP3J^y?X4yJIb)YwpVmFCo2!gP^n|q*TD7N{hbfoxYg;Db8|?@bzdZxFv{B2(L=uvC^Ec!sL+wSDWZVo4A7YHXz49IwX0w^5J=uDA7H$PGrPooxqp9 zhI|fm_(PT(r2`=5Fb!)uVU(UuQMxzNpl4%9^Z4InM*tra!~uDJM@K}s2?wNU6Ls~8 zNC{^T8jxtaIAw{H*eSk_&W^^{H0LtAyeEt{q zrWCfZL1#|^aktACW4XJpuTuv{QwKLrZQnuP_KqEV20Pe?b?~*R?K>DfLx9!WG`a-^XtQ7>jG*NnHh^ZCm<#DiiX*iFhk z&$qx)2+KYN?x zYDr(3n58!0&0op4+yJR8U;{sUpX8g=S2Q;->mm8(^)uMamzH5p4yngK>e(^VO54y% zn8$kN4ICJ=ZGyaCaGlVLuC|3sO~}2}@ov<^G@u$+g=5v|6aFY>3E85%4e+n+ z?%wdla#uQiA5Gsg==(VOo=xB9(02=cpGV&p()T6w{W1FP5MAdBMb|ci^BsW4F-B@` ze;(dN`@q?yOKQ2ZUNW~h21VJY!6NKEMexj zYTJnK&4&t)C2mG#g1G;TnAsxAtzz>bfZ`1v>l|7QSUwUODe0XIP$e3G)ZMp^!|<4o zOi+a}^M)AsvVB^j9^i7dt#o8bKF3tOg` z+6W+#d%+hI)KNM-N#jiT&a57ea8>YY7sGk#=7(Xbn>#mK)D zXsDzE&wI%jfl2wE=##-LTB0o#Dl z8VCCeGMJK1i{jP6BV3d{;-e;4o8HljPJ-P{&K`>E=@wPq7C5e#dqtn!bnwqb;ui}7 z8-Mzqhz#m`x<&9B1@9b#)x8rv%&|Zwc+t!A7ur#Q!^11W^(~@r3GA89HME8LEQ!dE z`q7O3@hUX&j=@J^Y|0Q}ET9RDuRMP@ZKO@gVl<@pIyZWOH%b}od3G=-S0Q&~861Z{ z(%^^QJAb!l-e7~rF*shXVTlR&0~0Yqno)A?XnE=Pz|pU>-TF9=@fz5u*4n%Ue)2Ma z+9|tyNs#vU$Vt<9VV zX0DXY9IO;?K_QG6OrWzr+KlXw_si7{zoX8nEWvRWHUf+z*9f!fbd#J_}BGOTzL60B)6hXsjI&@RE%?4wMruD#b;H*F>(-B41PM83ImIMYY=V zH2pgR|D~a3pK18D9KWD{A5y*ZH+nJdNQ2;+uZv8dH7;-lCTPB1@Z?3!oHZd3gwJ`2 zsQXaK zy4!r?N-Xll=DkA$h>AlGL$*f^7H$`}A8XsivZdO5Db*HvC)RezB3E06Uc!6{KwXVo zIm=ou2Li|914=ho#%eY1#u2juJn*+uP^)-%a~sIgfm0@H(h-Y%HxK~}tm0flC*IxK z>-#KtJEfq;;(Y?f`I-<2nyio-IA^l??tfL-go90P*nqWr(># zqVcyIo{8pf*?kPyGnKqWEx-UA#k(w*6kTB)+h{`}*F*O8AcF%ZjMuB=fb#UpxKL2d z2tU30nOJ`M>vr$)&&Iw_ZujnN_x?${_aEB54|=X0|K9H1_gp)Ep55O;pRkhr4Q=>8 zqVt8XpJ>4tJj(n}XdFnyJ8qL((RC;YKTqrCD4SD=c}VN@j&a!j1B9%VMy2~asHUiV zEmKiw9{i;%t!fwghN82zOvrt1OqlaLdx>KUvw18k=+dz*v;pcVvZ|i9ktr% zlPLHwT>&~X1NvyLb&e5T1tYsT7mCdk>$8wA{xwr)d|uX39fB7CD1ziWF^MlwVmFi! zFaMba;<2oVOgUb8^&UEE`FTts7Sz#1HmI!7osmM$U8G~L)y3$1?7nBw<Be|8;kFX=VM{!;jx#=_42mq$>?gMemJL}#D7%UG_Jg`TBT?L_!{Iq#}*b;Dq#Tg>X#YNjGiY+N4 z$WCu0jey+`6EjDcr#jP=|L(nd>15p8MJathw3h}8FCMxqiDs>f!uXX^lC!`6$0${{{bYe zH5kmOc`sgN8(Jj>TBeF_ntg`|(?FC#9vM=q-MBqUy0NY$I*BF!k|nRh74$02!-O$ z-4;uKZw0EBj#SMt4XQf+S2|#hC26EyV^#*p)dc%_}9{d6p9eyDl zYFkH>1N2HVi`fG_E|78?>>?53y#n$nhXF$>Fup)?EZB_H;{t6sPpbf2Ee}GCtJEr0{xS~z z!?euseLkKV$*iF3$=j4)QNlI5d|Iy6`Y!p#4H12|L8TV28%G&f(a_*Pr)?S-EhIj& z=D2df)fUIE!VA|zQ5)(pdDmoZl9>^r=QsZ%wB`5;6}sK<2(A_?F@JhoEgj zt7v7(m_!|egBlinzEak{6{cEm0c;`lD)Sq=BED{u#SD{}&pV z@$g3UQ2A^$(O`}xuV={yOnwTJOIh-dpD_B*l5fD$|_!s69GOMgLNYGNlE?hkD-KN`eQf%T#6QuSs%3r(Z~SPeKm` zxzf%xM|o6(@87|9*>B@or~<0eS71bU-O3+wpa$O-w$~N-l^2hw?FP6dr}`f^RvLwzJT~ z#YXau;jBUi|0NM81NEYg7wNW~%IZdjUwPzsE3KfcCgjSW2YHS;;tbx80k{5yWZ-0o z46xA9GjQ0)zYkf)LY6OLv;2axU}!F-CuT8Wt%|jU_VLlDTGfsYtg=F*RlJX_IM<(O zs`KcPif^0m#R@0iPMGi*OP<1#zry7EF?sOq9pX`RBL6PX#=rXNzfr&am`e4>cU}L}9qW%epO#cuG?IBI|N1^I9G%p{ zmsyfkJ9?_!WzXCtW$l!*cG;8Q$1W*wZK~9~Uo2`xMoA!c$;MCHNnR43KyvG9)h{<% zv-UL%CKJDEq3HN^vp^UHCs3g4lDUp|KaDodS}m!2?B?qHT7fSs-`u8SrdolkQw7y846U(c21L2LDEEs}tNxUK!lJWl82gns zI4qA!<|=*_`OuPgBbJ#j@~cLXORXEQRZfl5OoaAJJ~vIi*$)V4M}Lx;6?XSI?XCsg zL@v#0+LTRu_!*mAr`&W4n%vc)4LUu^BksdG~OVS^=%+zYBcP>f zY#BDGztz}m@l2Xx=XOea&xmrP-Q2*hnu0cy?sMpTn%#hr&nW4)pu4U3X9h+jh*2Ou ztkJa-ANJ88x$1~1aF$~Dluf}3<>8x&3pn`!aRHa;k(%-$Z~<=Q0@7IW4wk&1CF?L5 z)e}l<1|@&Yl0Rq3m*KG)MJMHOMrZyuln36x&7E-C_sfMh@Wy0Zexj>2+YuM*UZ_3) zS#(w*G0;7lDME}So!U*<{{ayrCfR4g1fXd;6Nru<8WA_Y0TG&Utja-ei#qua%tE^5 z28z;XQ#~2at{szIE!n)ATygrgw4mYWOTnu!V)4iElkA>A;}<_KiQZ5yu5`Zw|Mo^E zZ%n9+^8%Hbk`!G=m)x!lYKVlda7Asx*bZN!X<~3%sE(p5RL9|%Iv(lPu8z@EMO=l=QB)@e3rNrTc#hS>r22BgNQF?Uiy+05q$1pD1w>@iT`Y$C~rt3k+9L?b*4*v z&R$VoJ*SN+o?Y8gVa`xQv;lR4HNIbgKGc%M=|Pb@DDJ-iWTu8RoOD+N=u9E?WA#X? zVxRKqutd}ap#2o@0(lJ5J`y2qnr|T9DA!rtO^!6QSOxK4tAwghUKoZ9|0j)P=(;7l z4eiVaUc-xuFOD(d_CoYRyW_WdA^Tmkm|v-VU#We2flQILeDe!Knb^mb^9w-~M|cER zTef2yWU-fq@~+kL;+K{9_JVyqOS@pjOM&>d5^q-0w2QtC?6wT)Tmhjp}r)|uN+$hYNdgvfXQ;;6lz5-W;q?XCSC8lk`!V{U@+lbVZ#Z7F-h$tCy!!zl9u05AwfE00 z!Jaqvy;{P)Ln0A+_^(jR3P|cMX(;hucqx(;b4|9f8)_))Mar;88oEacv`w|jhlQqn zR{5gPwAU)vT8&k>6xr0tM`Qgxxzv>9H$kUp9(}zh1c$|@PuzmTu(Y2x6R?d=xXhRJ z3>q1~g%h{Omf0pkv)N4tEa^vP;AIVfQf|yBgfH3Y2eOS77UTZx^vj}LZOc5fw3{Wp zJlHqn&$ze`Vis(Bmh@dljGOEET`J*^8o;xbkltu6?b2>S8@UpGQ68X0tDj`Z7KKpi zYNUQHqFqCl4@A}Q>V=V+ryyXVc*cBL#bV-2C+Ol=-eH6Rq3PGf7U7F&e!Vo2Sy3Tc zV0FF`?Q!M3e z&QpL8giu^99`j9P%l|T2eUDKuBCBNWPn(L;vjngCZ76IDeMR(8uFLp!bw=nbZKTR#$m?}yun|R$ zL6kYvPQ4045jqk((=)<=jiy6nC9d5C!7T`hB0(X@zd4fMQKNi2fEcO$?-C>R3_Vhx zSPG03p4%zEWXYeiWE)EsF?k{;=dNsBsedWdzi!mOWa?iM_0LHC>qq_T zOa1FZ{p(Hr>qY(hKfJvOd{jm9I6jkefP@)2BqB(VfB^vsX95H=B$K>>3809I5uQ;`pw^@$qr@ zD9yR4Bs;M*r%y?C&(fUUCD}KY<|LJ5_bSclRg#@lnsZ}GcJI=ho+a6RN^=rRvTrKQ zxuqmKxin{BNp|1ToB<`-{YrECmt@~unsakWcK_0xekIuhN^|;_WDhLONiNC0r8H+m zN%o-9oZ%(eDWy5XO0ox+<_s;#9#Wb!q$GQ2Y0lu1>|v!jDJ9v%OLGR5WRED#8C#N_ zTADMaBs;A%C%q(lWNFUmlI&5XIipIlN0;V|EXhtU%}FcC9#fi=T9Q4swfX5&cR|(l z{imR{a~zl$Yv!<9fK_Lw3dqm*5!f+(>50hC3iyyXyC9!F`RBNob4GR1XhOd zkA~d|fMU^$FX9_M=r07L%0Gp?_Db?orw z8tv6|MXSj?H|S4eyQ4|;iTWmc7&PJQ{A4^n1bXq0hq11Y6V}>FN`{RbW}y3>L|vcE zSRd3QPpzZ622x!kw7S0b{6<{`L3+^Txe#6g#^G7L*@V}Am34no zRF;pKc?J9$2)3Hw2N0CC^UENrH+Z@{O#7=-nPHje8-4S*s;HYy8&!O)g!tag?EX>w z_E=SPvr6`DDjU*;UoqQMS&P6u5(>Z}TTRTQ64)v*R%p0KJFA5%fQyt8}lKzMo6CIhOHT%$qic* zt;NfEM#EN%wYWXtORey~Yzp>I{uj8sE}dpB+V8$DwbFAk)gIwlfbMR1x^+Edc#fqs z1ZEJtZvgL`AosZ>7KR)Pw;g^lyS>X$G$!jgQrne@G@B0}wfKsDXEek$PmPEglQaH?N; zn{4i5oZ^ahS&NQibc3&4GAp=nkINT5{kyg%PX~W=WUzgkotK@=JJ3*(=&?0cfCm;& zLg0GBZ^06?i7$U|c|>40w9mQoz&{HvjPyLhb4GgZtAZ)0Ky=I1T4fmxRf+E5dikzU z-k9lOfi7bH)bfywZyKwx1IKc~y!fj?^amsP^0BT5MZKYt_h^Cy7vN@_=Q{0vcMTSX z9JPXHxK9*ZNVgV`h&Gv^Z|V+|IyvQUGSP^0@Wj!&-Ph^Ogx znmu(e&ETO;@{L!m>lec9#NH4!pU)4U2|A{pctV6jIgHMMwq4<UDR`MmNQPkCFLi>~$Xa`B6OUz<HG@uu9T0_w-DbSwu1B@Xk|W<^^j0` zO2Lz~KDI^#VrhMB%_tolQpB@g(DW~o?@6^q`-^$|K#Sexdw)srUgYh=f-kx)0wR_- z=ee(u(wZc>$y&Ssv{pv(WKMbToZrY9k`lIG1pWAh6xd2yAq0Ldtpjuu9gL4gI+()$ zu#@9rbz0C8xeBgPF7)t%MnbKa~k z2f45+1caEg9ssf{pKAVcfjBPL@}m-h-V_6R{7+F68+m!k)v9&N1$dJ0Inx8w8ksBi;rM!fImY#n ze0iF$QKaHS>>p3UCNV69x3?uiWcJW^TCd=t#ZRv>@mVckNH^s79EKl#zW81(01Rw*(^5ooAQ|Sr zG`wqXL)X4rYU~w*O8V30I1}H1GH{bM zO6$r>Ba6yCodc52pJnp(U0GD_9#&Mopl~sZWB(s>VFER=Z!} zwIcfPg#1Kcp=X{KFdDtFLs`#du&JnMmvOW4Csl=Y)zdS2w{IUO&xtdLir$t-L^kL+3l(X8imI>@I% zL;J1}FYGRKSu=GtBJwgD@F=(^M>JeU?rS@E28LSdISES@wTTkMcQWCAX|9qgniFBa zfABGJZ7WBfY7W1t&E9~!$2ym9Sn8K;`H=d;z&_0kd^6t=zcEF1GI2g~t=jLTgc?Ctg+>>%= zoe0}cvQmHWT<=Joo)qC(8KWC(DAwOQ9=b)L$UT>*Qz z|AW%$NoM(nf~~I=T^#aq|!;s;|synD^+^rB=wqyD_k;$LrqgY5@wHT4JZ z@*gsgqLZkceLFUol>}I`7Hxv(JI)(rLj($rp7-FL%s41X-rG=_=)MNaL_pnqs~y6Z z2w-hoNmM@wgyy@`3_L&9_-L@k15Wvp_o&7Fx&O{)I9NCj>(AxC*0l|^U)LtkwrD>m zDl=>*P_0v0%xB>{J_{vL*M`nO4lrhe@PcMS)E8Vxw&SC;!?3Cud83}Aqept~E?CO%Shra>*Km?wR~5-AaY{re z{|VzF{&8rwDEVs@$Y1iSd1LND*3EVHw6oUYPf!Kk*X8+Rfq9{#8|!1^T@jf_&j-_0glSH|AI{#;eNS;-7RjJpr2BPwD*2Be*|8 zB%{D2o^`xX&-xu^fzelBD9FDG!&xP8E)=*9Ob`uOWYq%xl-M&1F}xvfzxYA$F&SoN zzKZuh=knG6HzMrYDgKV)|AqV0oif=0@!ss{7CmK$_0NSDb7s%=csdkbcx3Kur>AY< z`MK`dkD--0zA}58Vi#J`S35x4h znm#$kVsgi(&yR^Vd14BW@#(tQ2nV9Y(;=likc^-AQb|t@Sf_OJGeOHK_2Wk+$7!#MA!zVGalPb$CO10Ewu7O_;m=Ug8@UaxwewB63xBj2(TN8 z#c+eN*$}_V5HAxVlsZxlgA0K+P43BHu&y{PQk!ab7R8^$*Dm()H4Um$&ci?;9i7T1 zMCBTum@DINGQz-qgPedBW0uE4<`b4yb+uc+uR7(g?DB4ma?YL^&_D+#aFd)Dmh*OV z=C&Hh@bJI}w9)w6Ezq$q;O86sCtSa7m z@mDJG=SN$)*YU1IyNB|BBRn%vIs>BB4d?2?=A~e>W|jF4CUD*l4*SnBqR5QRZ3T3x z$ehSCH%E#R9+8-aC7?5b0h=R4iMvE*Hz*+%Gs_f-AZ5v&b(mkN@BxNjhF!8B9I7h0 z)$HW+jTCXGS+oV--t(*3+amo-V1fDl3%H;&XY18q^yz}qgdSoj()u&%` zlgYHO7ssIGNf1~yXecDf4Y+AzGkyp9ob2J78|c#H9s;V>N`3-Hsez3mSD9@?FQTu( zB+h6Lv%U@Geh}sh1Z{0d4vP}8_*b83Iaza|rTRxep*!DZawqaX2_|}hBS;5_?)7#8!-z}Kuc9?99wFD2V4sF&zn(f#NmcoXbiPM;^)sZ0eUDmLi} zLL+2z9(0ORc>xzTU*y+{*;859ECBKh_m4)Os-DE+22W?S3(Me(5Q_KW8FH2SH91jG z*}m_n+IG;YlFRu%k0n>})tQ7J$_rp@J7o(MK8l)ryq|J@0(Kh5Wi7r0GuT57_gCMF z{OMZ~5=hEO_IH~$u^>ymWr@DjXjZ{J6q6t3KR1JYBs%D1a$m6v{1JD7FJSo}>DPdM z9NyzIK(G{f9e2s4e14X2roUS9;ejCe7I8MEBd1$-c`*(BT}+<3Tf7AfkHp}f_bQIv z9F=!UN;^rJlHgJlEKVkL=y8#;QJVoT?uo+G`LXMPq^bhbsTY9qdtb1Go z1`X#}9^g;JBRoF&@hh+i1)M1S(Pj!3&d>8f?EnR%G=WYB_D#EBZ5ID7lK|ugehfk` zwcz5ySft8ktrH#CYGbjNwPM5`>}(4q@i#96LxM6^_JCd{I`BIxfIP&@Q~)6bEAjBX z5Ube>zu|w|y*BZS*Ju$nrZ{Jk>Tl$rbExyWpx~zOKBT;}XcJT8-aE6PD*+@okIuq>%g` zyG^b6|J3(yZLv3Rhuz7&8`9B>2njEk|+6J;64E4HCo-`0NT#Bo}dHh zXG2mJQ%Qc=X-K#%+Pqn;P>kwIYX!@tepw8q~w zIOR**V-P=oPZIdS0KA;M__*8lp%!-aX+RMjWOs-Vsv0W$tdA(tj0-W3kMXS2`*{*- zzvMzpaBEQwqR>&!1|o5y>dKUE4QN&CsabxlQyH4%RF)-KSURR!OZM^-1??>0_@&82 zQ85|-DHqs;ZMb8WJOiQ_jtfp{dDTv(6R%(u!eY>qvm{Dy037=J(#esBWBLgW$~`y% zu#rL)iO`H#TDkH`Yti>uO#^%XdMddCdz0a>psx5K^t(9k8rk?8aEcxR*B8hlG0TDm zaGM7n_-puqg=auhJ8KaLaRY9r5|@Or(w)ll+Td`)8=pY{#k%7-Dg&{cZXg|?4fwpc74W$v)14Ze zHP0ZJ5NLhOS!|(C5WW$LR8tX~$HECX0K`vj0Dv4`5i9B3jc_cm7KV#Lzm4M6US!?R zh?BAgS6da|DQdyjcp~gG94&e%e|L zwhhc2t@G3qHNYRoA>bJJX4XPdpSlrqeV%>ny%NmsfiXqp3aP@M=;^?3^N9$5uz=u! zS?DxhKOX5N42&e$^qDko+fIW(yTKJ2_O|+K)X3d84>fY(Ya`lv|M^sm$?#qljOUBK zr#&@@5!Ul}*tZ3eG6XI`6ChM5>qY~gE-8y3G`D*grp8cewz4Qu!n+$JzOfJ!Q=ofa z%u+Sx*OL*r0iGo1>Ep@7@nEcyRWtBR>@&}JCX&pp0P-K35fVH%B))3`-_@36;s&FF z9-}XBA581C736y5H0o93)hH3}aqXO()m~Kxh7mn@hxf6WE8uqw1QnG+!=8rO*Xr?s z;d-85lhVLGnu&wG0Y2eA{$?|Nx7XsUMor_)5q_~h-lwYQ{z4x`=y_78!sf6d*{epm zr+C#2_bjiv&{{0>_EF#_Rn$g zNAS;T@N}7`#AU(62tDy3EwNAq=O`eV6FIqB4#}L55aV8wUu z7Ij38s7w*%jcDnvr5Ui&MoTjwrM|mJGtBQHkybW}{ku)uzrp z5Te@O3!1NiWSqZ1Po%0qSDcYRMjOp`PNISTBZ>T zyreMNH=ih^V+6^>m(L*H^hT(J&A=S@LU4M^#x;whsMJKy#2l2JApKK5WGc81J-)I& zRInRvQ#%O8(;iEFCQ3j=pxEExuG!sWxL<)lA3Yl$f23vRJ5c#`cWctf)O+C6h%XO; zq?;+k+YIbB#($m}eqNlOMKLpR3-o!`(U|vB>QI=BEt8wn^w&AAmrsrsTb6aDg>FH> z$imm~DZGi#bS2xj7BdYhu$eZ5T??wixmpHnfo&EqrNlP0q(O=d@& zOg^<{n%W3`YCHbDE$_xqvvvLLmNpoTAmt(K`2;bZGMb8ZG@j0Is-2py>LQ(aq>kf# zxMhbA@PGZ|DSuWg`QJc(Kg!R1xFz!8?cd)P_18cgk-~q0@4c}q$&ZA8&-8CgVo?_S z!SNWB6#LOnSbDigepw{%qU3KEP;!w--Xfqy@*I)8mXbg0LCJTBllaxGPB!5TA%}-GBG?BcSk}YEJ<3#e`Dfzscl5ZBt zFH!Q#qLNM`c_k(LMDk@u$@3}s<3CXHQIY(l==Xd|{zfEEr{wV>d4ouHP_iU?@`gwr zP01rga;ZorzXs`@-lpV5BDu2|rAVGGlF^BTFFpS)N_L3kla#zbBo7qHdntLLNbVw% zw^QyG?nekh!Aoke0@1zDc+CneE_$Dw#Ka@ z5B8vkZoiuMqhgcxeuRRSt^MHr2o9g;5@=?d)HuZF>Piz@v@q-WdV2SXp&tqCaufbY zWasEdG7I1jlKnjMC;k1%`zL%MN@ee67+;8{y9_Txx;zDYRM)q4`J48BlxTQAN;bS7 zY55jIzRmF3lYp;1Vfug|&$}ckD-uC=MXx=#;1d+S@U&&$Hr8|Ie|l0PDPzJ@juq~{ z$?Q{}lPkNElC$KdpaiN_f1P9}w8El4C{4j|KbbbEO^rJTRGlC|80S<*fu|IFS-OYU z?vfw1z>Opxk?+Kmr8dKx63#^mrY}nPEs0ADe-^1`j1RUBZEK4+pc(j1r~Dlsw!CRA z8k|O#?Q5r5)f6^^(Za%5c6dsY8i-~;-3ESwikr|}bvwJkiSCICFM=AeeGB}XWzD$= zuTI$j$K#U&(@XMd`T_3md}n|-tLB2_3J)OG(lHR2D;=HABNOU##9I`mUJk_ z#9rp8{s_?_=isL7oEn!=Q{edh|f3ZlkB_?vOmygvdHz)XSK+6(`SwN zoZX3DM)=RWJBbc#g9Yc^KGyC%R-Pqq0w2jv-kBfxG^jpuq>SDYT(X*_U^c(-XUP{> zNiUMAcyVZNrIOQJ@TcOC&)I$1(69+?*yW~AzuLKTr(}J6O^v z&o}6&c(DRmU%hx~X%_<;C=lqx0h*jVOa6iF6PYJLn_PaCA=QwraXjBBuoE`y1P&`? zKgvq0TNE!>sMXCy@nCT6J_)j&9pLN3w~4|@LJ-+9mx4sbx=n0S?dZ`5TKq8vS!f78 z>r&9&G|Ge2dYB=X@?JTb+OOkK$_wYPTdzV#tnYt|sz=T2a!;BHUyO+&!e9X1m#Rs$ zx|pZ>GR>o2U4xDthLM?uf)2CU)_>BdZ-YK4uL{BeacXLPJyFPr7!vr)2P6f(CbM)= zbN3i6CNqY7T|bSw5fekQMaUr1`@tojcTxOUwytx7VeGY_&_9Pa6&$%&?SR559SS{7bD@?nU4v& z1&=;YD~O9{qN0e`9TSa+E4O&-b;xKCrAn|OVsa6&WbLxCc7Z~XHhg%WxaD%GzJgv3pssxj^r&*l+r3Z z--2HS?UPGTFm#$9-7d9m?`qj z^Vp=P8;W@Go|U$-(SS#|w7>**Q*ej#NoH*V+xjGx&!Akr@VJl@iua!vD$;mzp@!u&4(xwGdyLqq zy*O>45=Hpx`B@$Chns|7d)k+_vX8%`-H2W@VR*EuiG7n}nN|&Lgzwml8{uB#mh>?K zyttzQ;|N3cT{xd#V3$P0?CD+9+$)r3p`z5%3)HSJRc|m?_Or}+`1CO6vk1KmyCI79 z?KgnyK6FYS83dQy$|h~ar?xb;@+uoSFc+WH)&yek5k}RK>$U;!@pXG*bo#WV$L@j8 z0D~QU>g#s8JrVE*U{6%I4R|{P7l_CCM%uvhZS2N{Fe~m?T*}95g&GPw`D;D=u(2cb zBZ2++Ykkm_f5C}*dcYtWV)NtHyY{lraX*f8xzly7g^fV9tR zK-Lrd;qX7QMHoDJHvwPDtAewe-)9u$Q^EI%f#|?e)_)3k1KZvh-Csd7-LZC#r90M> z#yi%IJuU87?ja#}tQXds&@h6ng7M3l7R1U5`miBp7Scz8De>zER8p{kM6j_O>I*)-qeIUgm8T?xG%7GV^MITV7CLAHl-k0aZWDdx%9mU%^PG@y|dd)k#qAgSm))zuE#4b~JGH z*Y9JXTaavY+Q2UO>q#l%H2er&Oe=YHl`Z{GKa5ZK7=(4Zqnes=)ZkxVehKqI9dwki zLjD`&s6-(*1l>T822Oic&M9bZta~Bq(IM-780;BuH_(cG1H=emFJ{pjKg;#k<8!71 z{wi~-L7D2h7`R#U#^)Ove*ghbvFSLdePPyx%)@x!BI(?47s0h9Gg@408q8$X?3-FG z`EqNJf30FeBWSbTk93e%76jd1um z!DhT-Ci@!%E#JrcIEz8wfo-4)%(-x(>S4Lv-tDlPMTs`?#d!u7ehC)cf;^1^WXFHh%+~_9Y<9 z69;sX`z0xRkbagnr|dyII$erqE$`bJYw=*deWJDo(uDu<%j9Wm2Co-WU~vAKBGt)L z1AA!E%^69HZadt>6~_iNk63E)KLB1WThm`nmb@$AaB$!(@MXTj#=l0umv;p?Uhi)f zo?^d)_fzEmj@GMe+YzASO7=PabyNdQ%Xu~EIS@QlN#6WwUK;jvr*fwa3gTACuJ1<7 zM*|?(w*~ewr*B3o*p1+o;52J{Se%fnJVRUNR0wcb z13O~YOEgu_5SR_lZm_O71DTC@sd7IK)#fVVMFWE)LZC>-uE*{6PE<<#D-2L^R6Avd z38R<%i4tnSG#?)m&~B{;|E&gZo$+@}Hl8l@a&vzqyeDOtOo37GN)iG=+H+3D;&|+Fc zYC4Yt80t;l3L&K?kwUB(F1?Zb9ZZ?KfjV7iE}`qsu*(h_3!GD3xFxon2QfCQ5l!#? zQcrvBo4@FL?P~s;MVcCf_JFnMZJ~OEnvu#Vw!NxexL)Dw(g`Wi0bHZ${5R;{*x2rk zknlu*Jc(Ya!rUJC@xyWa=s-S}0Cp|HF6qasukqhQ&ttbu3>g(^X8#35JiU?oTExrV zo&@Wn1d2)W_iVwx#NN}0yO?851&icFyH-l8ldR+FfVyD|dGchvK_y0UVI^><;yZgY zT$Ij_)m4OmawTPu2F5`y0LUHz;=?6I%1vNvm3k5{$^hhma9r8NaVlR_N6X4nL`SfKGg zxE}c$WY!i7u*LdI&6RiZjycgCF*#1U9OPgGM?+itABGBZzu)8CKaZb-v)8roG~uWL zZa_SNg0VQe&=wWPVW9U<@I{S~(j4WUyI^mDhi*2Z9jt;SInruZTAe2Y#Md1mvIU-R zXV}i2@OI-#2FpE)d9fDHV8yYHt-Gl56K`9foz`J!2DYj$&{YLb0}$+tVtr}_D!bPb zD(i=sO~iJFMqn2DzbkAk^Dy7;D!_*j>1-5U0Q@dY2)EDY7wd}S7Ehz(yWJeY=OgXQ zg~KIZTn8!bUkeA&?W*uR7LDh7WElkNu3!KUcl~O!WUfisC8bqc$89zG=hF34y+`AW z`l)eiuS!2P0rgK0uDz0eD&?>EORM=nukaxBD)J!BEa9JOa4RdsMNcN%-C6seIg&CH z@iMgVAfVRUB$&v1G_~V+b{FP=vAWv5?qMPwth*)eC=+=dVRL>8 zb~RH|$#Y(hU_+2?_snF@pY)F3paCUaAJpHfWp_@ag$)icev)EM+Fn9>X)~aq-Kn&f zEITWP5nRZ8&h77@8s48JiVMfTHfAC3QXO-p`D#TU@7Zyh>K*Fi4M%0NRn*TMO&>22 zQy;%xSG^lQI$ZUxt6>PFdN&Drb4@CqG?#LWyad^_J_h-xM}(-}h3gO6FuupnVt$%T zE%hsB-mB`YmUnp~pCmZ*S@nLDDa8Oa(yG@_i`q(*F9OrsZQA@WSf| zJsgm;oWtX(s9X|idasG#s9_F&2 z4SJYC0?2p^3E+)vBmnp#0_a*5(~AH4E6slfVS_{gq-l9SPWMXc>>`Z_T$s9mQuBDK zO(&Rpt_u=Of=)1w>vV#-Ze%#Y_#LehOzkivn14E25)3c`tbAAq!L&v%U)2hF+1?7h z44U5h^soNDmFZv9rRDU``@WUwAKxW(`p19Y%Jlc{VwnEBrs~swAS-J*gWIr7sR4xdg>#V!QrmSg3KX(r%+{iC}`%Lc1pgEPP_m4E7 zLH7`ChosE#SpjciZ0r!4m1755C`Su=f_TMD+8d-NYtfg6{ijFgkPQpy3RLr1m%Tx{ z3f-_yb_$_buoHuWiXGjf(^!yVD>?-!Hqqo+#47d#w?RH}0_YV28o_~*O`eJD6#`1u z)yO%}Vgl-b)cWpiVoAl5LRtTD;kfxH|JQ#I-7KZQy#`4$b{vu>d=Y6DRK$euN1DE~ zMvvyvI(CE>^xv~638fwAFSfzG3`y^BCw=qcIW~Rs`jR@oh^Hp#Wc?AP=JC`-ovdG@ z)Eu6gtdn&KrKSKKO&=Cc*8T0Rl69{XBd;EsJxqmY06xbVE z(KnwS(DlvhCjamC&9erF>YHzC4AnP(zF$z^yzh{3ee>4hqg{DC*NO35)6|;&{C<5r ziT}lT+$o{sIoA|Ao^5@D7+i>-|3^B?r_OrG?=AJ3bELdWybrO@%5 zOAd}lPWkV~^Tb4<|I*t0{QA8Y1rNRo`M8ePf)^aC(H7tF%ItWJ_iRv>@FVum;28;3Fk;z?}BZ{C~HAS_Gg%iU^R z&iBA}_XTcJ@u`q|y~!XMnhLq^;C@||AbBe-A_gr!hD!4HAm96hs(+Nc<@g?k?oo)> zlwI}><4p7ndYYgd5Q!Jw`}hLTpw?|Wfdo6GJ8bi|#YY+(Wes;;Hok)O^-dVMlvO#NML{3xi;R~$TUA{T%0K+{75Tze8iN~s=@c6J}fPUzb z=*m+@C)A8#NTjzql@a)yKBGN;+rjq}CQPIR5+3y=67MC%UqJN}2V&B|EM;~LxdWO6 zHuDlkk!=)5QCH)!3Lc`^&VD$pXGW5n%n*QS0Pxp-LL&Qz0dP}v$QXe2yHQPQI7Wj0 zD&WRES5k5iox?afkwxgQ)=0j|=1+~%jJ9PE1x1rzSR$mft&39Od5rBI+=PtR2v{yT zQ<_kn@0M@jZ6~~zMV|$>LKz|TScJAdXzyc`eptaNau0*N1pFs|X_m}eQD`DrGq(oi z!>#kP6+;lYgW*04`$k2a;P`;Qg@kpKa1F3zK<)4D3YShxjy-EOs?6-@ijA zs**_>(3}RLOi=DVr6;)v4e%rqLMMmB?jQjhP`@8X34z<@lzY(8;&6Ctz!$gUO!*W9 zXh5X^YCx60@xIiM_OpTum1Xnx10@utnO7-+uZmqGAp&lde{P71o^I?U1XO71gi-9a zvK|s)VY@j53yft;x||(Be?il$9E$hv3vYn% z>$bkM%nN9lbF%fUAwpV0yn_p@NO{g0wzw@Opy=I#DYGf1FBD{{HwUe0%JGfx44d6g z)gs0f@9;R0aU3|J)fIlc53X=qi`;5l|9z-;?8CmOp6T6|VbB>C>)AKS9I}=S9td$g z`EmVb#q#yrv?xlfMU#|PK_4MyyYPi)IKQucJ)%5q&T!fm=7h`AK5`+BZlT3l%Q{&! zLseXywJg$NxFHYE*6E>fSU5dE9C`L(Z=#35@TM>US*!NqzrpXw4x-8wwKaYeIRoN% z${ha*zmpNd?;P%f!{bHO!-InN7thR;)cg>{S+N@_hZ6lfLtnb}X4 zHNd2yqwstC!|ePR)MfL<|2jlx*Pb$IvPS=9%Kdr>?(!nP0P}By?#}L|SwI*a!=NoUx5CKGF8ow(j%Ec?yrhX{}y#{;9cB& zcN5Bq(YelpW-;76GO7e+MI~G z&hahkn-9P}c^&vTcOoUEzJ&<#I5g?)37Y6=tZ|SX{S!qbA6NxiFO4kbHrPb`7cSG& zCn!cq-YSda-I@T}o-kq-aWk^xAcw^Dfzo=dyMT;EHoB<%SwsArGe+NApL$JfFF?R+)fcfgd7?=qWlxx)Dh zl{^uJ3IpXqY1r_b(ACH@OIRU7yC3IkxWw$4N9lOg5b_u-0uQ8d<I@P|GAL55bb*uVIUK`mD!1Z`$}d+=kE zFTJDDsNKo-x?rV%@7>wuVds7BqxGraN;KQLe%YYJ@yhf;;6bsBb119PQ@8BE$vZNZ zaNRF3EL1)&^xuxL=f;w@%4Y=Wj)#xe_(wsi$^E^fNCm#}LRrBmlc%kuR!ZuZp0gBF zj^AD%7ozm9(aFAJ-FROX?}zNzASnqh^F@9kA|+biAUtFAe&vdO-53}^?;pYbM2i00 zZ0t|_tL)FwvDBY2ygw6s-NvG-O=JS^2Wh?fGQ#@t#h8DGpG9M+KQE6l^k?~)tLx7l zqCa5K$osQG?~g?NnH<)iVgI&2zob)t+8O(EGX3iMQ#FSA1JpL&*KHGWO2IzeMSYqX z)~7epsZV_ZDSDrR{h1>Avozh%pL?&eKf}l9ooKt+5x^i(VuKZVti6 zx1*{3i=z$g9~f| z%lK9Ojcax(uGtr1&GzcXkJ%9sr1oF94fZAdB=x@)LknY^GI|hucq%OCN`D(PsTG?G zzhi@@G*iYADt-iM8r-S-jTT41ms*O2r<9tC%B>}z@FdTTm>4-4cc6%c9r5$<0_Ln?siK!f`?Qp6M`(yIL=KOfO)0;paFkgqx`fghft1E>MG6jTYnHG@LUUXmmTwtR6f znvvbaVhPQ_p8Y9xvssf1VTewe-Yik%4U|8QVnfG~NTl0VP;W&;!24cm(&9x6Nz=hg z%~#4*mCVF`L`g&bz zb2lfB=9`SfQHaEG0vN(n@OKiDNl!K4yAHiqY-^hrP;=}=ClOQO)2|<5stt1CX6u? zy6w({Rq(fKc@uN&Y15NZJ+UNG><(Y^`4^h4TkUDDB(3GYJa>ME3f#aK#t3}=J`s0@ z0n($DsZPZm1tHpQkFp_q`c#rGExV&DF#i10Zhc`p1)oU_iXnLa9`Dmi|K@>p&pm1@ z{$-5*+gx(FUH%5;H1+6Zo(!zYNqUImRDUAIz@I1WKa;mahpRWRy<%U+AJ%;s!X*KLYU}b}-AsG|@K{afk z8r+Xa%7^O&)IZ$6WDht&^nsPcDLBvO4-sM3j)& z)5I%I<(OiFOPjPI4ij!%$NF$v>Vy0?C;Z^Z-&VAd*RUrClT@q&KyAY1agY@&AyT>N zA^MY={GQjA1{m;;HMs|o_CBEE(~iQ$osi@2xIm948UotHaRl(Mc+-vgrJJ_*V=$$E zyaqI2vFP$n;Rw?Gg+uexB&7!^f!-?OHaIazX`|4H@pNK&Qd3ZuZ!vH{pnu&3B?70} z%p{b2jB_fn*7X+ep=NnPlG9w3D~~`UuKRknJb#*?gH1L~*rd51=g>n!psffU0Xpb4 zu>Xf-l06vss|sPw)hKE6k(IskBG3-{I*F0*7xc z5X~O=70*uh65ewc*b*sV=;7hF|NL_IQpuNIoQPU6MLkd}ri6cOgpeofcdx~m zwv+))+#V~mr-zYfadVlOw%**=z%Ov;tn0}39qiKC-b)xeu^8?&{wNNpmV0ZhU_}+y zc$UmuzM&c8eQ{^Am1Sp9E4L3Qy7ZJ3GYqSpB*%`kev);zQy@zSIW9{3~2GRjm`QT`5qVIvT7N>V~k_lw_pOs**`N8{9j2U@6rvd>Y zy^!wQ9+d=W^XBg~xqAifLKO#5tlB(+G2jO?@`QWH}C^(Wd z1D4(6TA=o1WGko*{0Y1Kb7uo;J?F8@?`o)R-JiFP-;bMuJxLGg2}L1Y^dF)&JF*qj zW>P;#b~d2)t0wH{Dn#wDZdXEW==*Fae(`dPD6YMr`G4ah`lV4Dk+tYk6l*TEumcg` zi91t-pUV`c9g+A4ygF^B{A7`zDE=l0fK>J>0c41bRPlF^0GP~{5x`v{W3u=w34n*$ zJp?dcWIQbX&Ln`(fVE>I5XOt~yfNEb5d4|BL_p$0L_9&R(IHpsKg+bA%4fg`vsyC( zZ_pvu=|9)%!1!?cApzIu5Y_t6aviu9fS)E{rbE=~Klkat=u&+e0pmVQ5KTI8oeqpX zvHBBmf(~rce`+4pIWYQK06(!jd5{i~tp7~ZfinPj7XeE;M27w|RR_KcfZrnEnL5N> z`p?NaFnFtDO9*(04l!T<`LGTQhYFiOz$H3Ff%cQ!gwcvRr{epX`?zsMoqiEl6yppd zEvM!6bg)`4lT0ZCgQ!OPk6xmc!B3R6UGPNt6#v9~PR5uv-VF`B{$h+t-=AE*VOu|p z!#j4@b7qs$ec4BG;NeD&Ui1+qXYyoWoDzQj%=15f$rxXWP2zQH#_6rxCxDmb%yajG zui{LS)p6j8%i~Jx>PidpN-Meb9bhA@EWl!dN#}2W`~Ft=E85*C(Rr(MeLc9nrsJdU zZacrW$mjyYE?>469#O$P(8Y>CLLo5qiH}F~(>jpBoASl?+R@JBQlcun7b83aQS@Q= zUS4ip8HudHsa&~=HM|sK%9ZONrU^_kIKJ@T8i9a`i@SlhZ(sb6)lkjQ3WNYJ`~F)9 zV%FlZcAsO5^^(^q03Rk*EyjxhQ(E0 zr5B-tC_Xu#%WpSl%%otnlM|oQdnwxb6l`%$b*C$>)>@1bIiEZ7QZNqL(;N#nQ>`Oq665XcM;N=7as#4)LAh!@d5)C@0ti`{8S^GFXd5?4wlLv8i z0)u$UaxG;Gq}(9TS^N+9vhn;$TK?UjI6wV8`!nCDTs}t`hOmAL zyk(L6m;;vRfP4^q@?CNzTWEp{***+D$!7l{##AiYC_DyB$J-p?Y3o&&FHZ0u=Q#4JiHon{T(d9!=obiI zQJGR+=~ZVf?o(I+w=9Tv2gRDA7d>J9q(Z8w9V%5cLEM!ucmsj2QpyX@BRy~DT>x_4 zqMMgSFT5Y=x#DQoieuI`jdK1ai1~RZlJamXlJX@Y<;~9`DNDWwl5BbI4vMKaP_h&m zJ+Gvl3A)7pAL7r^7wzp3YtrQjhW%tMJT3Dr<01$pz}HP=NpNT3TuuBbvx%Iecmzwh z|2z>=Z=M`H81|Ln!SGExI2b_YF#8npSs*`j`QAbizVE%syb;0wOTHHd;lqwkx(Ip2 zsrbj|fA=?k zpvmapBf2c)KE!Dvy#p?}syzQ6jsB`2zhLwTlgB2>dsqoCR>69B>&$rDx~3DJjj!79 zY)s%^8)w3$-un#HUZ9oN{na>7v)3+v$2WRDJM&KHJm(*92UjB?#EB=~QGCK zEG5|?pHmb#<$ZQDzqZJD;*Uap+Y{%li0~e=$_jkR<#_wWJ7h1;B#qymbrw%9_(4Y( zzc7!Nqs~5?dwa#KQ9czKe#hn`y$a<`@To6!SKz-+81<< zd=`9ALU&)1>?600FVx|mkIA=UbaRn@y!wi3G0(vloWq7n3h1_W5YD$_-IiNnO>S$z zueCTA8}I;8z4>pNtVP|-T5yGw{STH&vvSu--t)=Um4C+a)l#IR+~>5&x$9g?B0BGT z!-Ox?Z~_;U;R)>W^-su6tG7OlF>1U`0jt-ASahs*`aGS{&j@FcY8p&*!~uf^r?$`0 zNm8b+2GK+(2*YBgAX3}d&fi15wcupNuJQ#I$QqUZruC^Afcb?X_ji0*_emBI_lOP_ z&p>tdpNu8J2|^$h;8FW7GHV#L66TFeNhajLjnjrT)r8m5X)hRIs{mG!N5vSGuoK}e}eMFWo5Lv!T zWI4OQ+ILQwwXbIlC((7BL{o`GN&Ce|E>zK-A16`rY!-dbQyPiZZUK+$Y}`39`*4{@ z^FDG4iZqObTOC5UoSbvZHS??u@rmf*4^G68kcpV)@{NXxxM*R` zXHBMwvil-T!k2{_E^6Hq19#ITymz*hNqDyuC*h4~Z4yTKaT5LrnEmTn#OyM?Hc$KD zLpN={C8e-E8srO_d}9U+cDZ*e@(l^33I(ppYt7^=M3Zt@L?=U}9MROtOHl?X$1@E3 z`KB&F(&rmw00)nmO`6@r3fPa8mCy-$+Dhn!wFnqO`(_Zw2G0jj|L>Lo(Y6m>3)%%B zBXlau4qDis&!Ft?peDpSs0s1FJdRiW3|6hYRD>J5TD-$<&8o7?*kamN&)b1?8eP9L zG`fMjXTj0kB;)9A&_)O9lm9qI`)@X^Xm$Er2d_CQB70O(5=|uBz{!ip&N{K`(rM2P!YZo} z)_0dJt=gKoukSy-7j0JWSiD6<{e(m1%Se*JezMLKd`f^AA)v}VjFc3}%+iU-L3d;lz0S*HDIo~wO zf_&5LSmc{V^RJC5;B9Hb)2-Wc>;C>gbr_A!1X+>!FrV|J!}`+2IFICGvD!phu!pru zbCrRF-N;kFg<-pw$y=~d8u%J2iEOwqrIJI+}b{9Bxs@t4Owgpj}wr$~y zYIWeYP;ScqOn@`ch~udz3CH7%i~6==#Ie&vTzE4(_zNNU7MPtb_^>B72>W*0l1xJE zbH(>p#Mp-sLBR<*SZC2EuPD|)!RC<@zab6*!REFTzad8f!RG&*_zeLJ2sWQ0fZr5s z-it6-BiI~DkZW|v-w{i7 zmJtEdRrt3gd1Fk#WQgAeW=0n#d)gN)k21MAr#vDU;9S@nDX}5`q!2#UV@3EjlDyaw zEFj=>ESi6pf)yObTq%U9ZDVlbYma~8d-6XU?|{DFR;b03gRLeew9rs0^nElI8UlsT zWlncq2*-~vzOZ(~&qIFS{K_emjp&(t1w4s*y13AdA{*O<4k_VY=mE(W9krhGZE|LU z-Fw;W8P9yzqb=QdFnzUwQ%czP?`n^U!^qR5iQiS-110a0RFiukz@^jG0d#C9p1|0` zT>@>`qOPdLww^Nqfzcc~+2q-!hl=P2b%9m|7&Y7R%nfx=f^uCgtp*(C^YDf_HW`HBDBU|ii$O9P2^(}U;xV|)PdbuaK82OsZc&& zFaLRq@*w$S?Lx~N^%V+#Ex;P1XFRU){`-vm6W$#_CF2~0cDELRVUvrrc|mL8^3g_g zWRFpE8c9dZooY}jmV8rJvBqD~SzIDaSeD{g#cv820zf{)T=cF^nr)hz@SVpR;Yw2N z(RlWK?jeEYe2n`dLuP`8iTiERDZItJpcnDvl2>IIqptX%jADI{Heys1Nr`{ZquE>U z^V~oYD$h7M3msa@{|-NA*!|06jQVp%{Ppm(g1=M~DRKyv7Wtyf_ZFe;7^^zcC^WxW zJ0|@8wnSTbpf$9iedCb?e1t@_ZPU<}hy#m$)~(|p3!JEheR7D49BA$a@%@c82i z#AB{rhx>E(P<{*f5yUnZpH{lpO5A~8E6JQ zB0N{Eg}KT@whLc;r8%7NK>?R_I3Wx+Tih?M*nhsH`s!WpNnIZt>^f5~ zgQhg+fL^E!jO9JgfSzC5vgg8!bU!ny%Y=08XXx7EUsW|k|0vY{e#+!67zeieFr6UX z;10SyLoi3d!vwFK(9TaLV>@3u<|%wc`E#f1RRxz-FKCoDEk&T zNJnKyd0aRzIW{|p`%%|vBpexvG!vfeZDf9^{;Rlrc@Nt4$%ei&WfjAIDHyasV>bS`iP)fUn7|;ESA68-Oy5K&|*242PF~8=fO0jK3&U zA?VqPj$Sp!eUn#pdxr9(`H8=`GX?qv=}T`#b}lAR5>4AF=#L zd;UW)8SItC`OiC+PTscrTqOniqW`=OIRcwvk~5=R@*)eI*B#(uH+k!M70FJ7gZvHu zRsTi%H-@*LQVvf_FO-MrH}d_(PWuaQWx#*bie2Els~r$N)D9T`9dA1JJ>GQje$VJY zrd{B@w_PB9JkQ@Q^1B6k!Th{WU+n{(`Iqu{`~v>a*!TU|_sVc>$kIzOhApK4Ng0{* z&yS&-spXJgp2c~#d-xqPP#*F=F=euMMO2I@Mp9;2aDy5hQrn|g8^k1KbB~2?P#MPZ z9icpQ8NKr_DnG9Vl6YmOE?%Lsp1d;MHk>N_|6soV@YgHRo9Q|A|A5{)gty;<-Vk4I z1HN}&Frw%&L{S9HUh|PF;@cgD?_PgU3EwNORM|g{{QrXQh%4fI-K?hn1AIUA?!U)( z;fg3+bUMnXoi}2M92KE0t#a6i^0LtKp(wApzll&Dccse491cf0?a!h2$?$L5kL*E| z{HL2yGj2w`?B3=WcP9VW?U~E*^yJ!hCK308Z!)*~WrD?BDTV*Q0XjHf%3q9t$$?os z?WcE*X`KR-c$(h`2e&pc2Z6yHCFdJsjGVt=N9y+r?nu8j(vH-?b{sWa%3AXep-hUx z8~dnQ1Nxzsj5Btf17_c!7-B%x(tZlMwSA`A$;!>QZ4 zJ}+KM+vgdK0Xul+CZ1We-&%CW1lQ^TRUF9a^F*Ozi=WxN1!On0+a&6Gjp{o1R%>-F z5Ouvxb!~hrq^@3~t`e=T+o-M&-!#?c)Q{NXX>=*X~`yp`ODI{@pgQVLU)@HYON4>#GVVE zKe%qQt_Mlrq07_Ij~NEvM#}$7bchp_ePC(E&c)~!;F98CXr)}JLsV1QIan4v$6@LQ zGCsYNQm4YOdFnZ~hp2ci8*oOa^3_P?=YX~zykKx~MwFfmlzum~#3_A)M(H9TYM6Q`aXN6<(dF`2<+yWd(MO->|?0J{}U1Xq0<=u=>4$wdL93RLh-+{ zMRnkmkNxSVaQuhbixuy%mH=X%APa;mxErA!b@q&TRpWPeG^zf# zuw4AM(=8E;qVVs2R8{{)Q8{~Qz1b9A^@nHms#l4stsih6H5ZHQUQ4~{VklBquNCp+ z0eE5wsIq|W@tt=A81S;mKhnfV92@MHe?iQz;lhs=cn-VuQZr^>cbS^X<%r5k zn;GS~A!sMozE1NNS&&j(7tXw;IKJ}b}j)C{d4eZm4(6Lzm=QJmS>RQgpEzmnV z@16f8K+N76z{LpO5~+p!%pA~|wo0MZZDxH*vWFnI7`*UquT)WYY)-Yf-;$XUXmE{F zo6%h${2mSkR9Lp21CsY*w0ju;H_{Wk1*k3%y@l*|8(i`Q2>X+8S=CJ@jqwV$z;P4U zW0VK;yLCEL;_m2ea=W{Dn_~EtSy^H6SAGfxJN(DHJ3uPNf4uRort}=&?UX)Wg@}GGFsf67zMGx%le$gL(uixuX125ssAs1zAcWuX~=#(Is(u4E^Q@DLOUV$83wu=sq5pm~&)~8} zKgjeptL~_@L7wSKP9j)`a+0E_p|uUHzp@m%k>75p|=-Cy+l6 zGP&Dt&q;)t`Yb5jO$)@dkRty8^{4C!N|8hFdr*5($uy9Rkb$i6w;TvOe6!WN4FhQ{ zzDwA6CTru#(8lBNhmWTNR6KTfa6G+Qj0e1fujO23LF1z3x2-EcqAH6V#s(*t&C?V)Y!D^ zKbs#(o1a8&FmD|TAIuokFW40vjQ3dhV8#JN{;sQo4P`>2_k4!C6FDm76MUXRyM1Lc zMxV#i;j~)AOZ^_WY`WZenEvl4RZCaFTou$N$a0Utfttk5Qrf-{F?@_%iN) z-&`dTbt?o=`AaJVuvR00w?cD?rTGvEpe*!% zA*!c)h0APCn7dEXl_VATq)%3>Zw~rk+EvpW;<5bGLq_r^x4*Cxus`lZ^4C0Akpwz~ zp8>T@c(GO{@xKcaK^9O{9E@}*SGQ*Gj`ufy&(=w(@I_Swh9;^h{FSB{LX37h6hfdh z%NRrH;y+1q=Q$q*9DaGRt|noyN16}xFg&F@(KsO7WtKATiNjeJX54U-TAk6}kcts<6 zTw%%c|M~YEk(&Mir^_Wg?dt{gYAf#>K)2o}_yX_9|L_GK2L$v)g30>4=FPmNzL}uk zMyGtqrOdO?8dzds`{&l9hcG_S2l+*olZ1tJnF~ahXUSC_gW#gr8yow_-1?vcG7e^A zb#oBRma9Av1WRD00xSW+mL(;yYJ{7fln`ch(2ByIzMoCjR1^sp)Zh8MzlHTA;`fX| zKh3;nPRg^fErl5R^#+lVATkoz+gc&J_4(gWN749CoR0{%s%rW`ydzuSR>$7YVefz1dnncMh^V87p$?}qF&Px$WXo?puxj(a0;4|0ULv7*> z_`^`q?rGFAyM3${0>nL%B;I`Ptu;b>u!`M24qaW{0P9_H;qMN(=9^jHpM}%GK!3=6 zHv?Q-AqR5nk48FQW(w+lUAsL@!K(HBl-_UlfV{z_JdDw;(lOHNS(ogn&QTQq<-Si3*Qs5kcjwutcp->l+{<}iTo%HabyawwP2u^aW~C1q+gMxODrSO0`7 zKTdJ%=8OzEc%nF`t@|u1BYz>gz(tO7C-3EW3z!apewrj#Yl2)TDNA`me(HhbYfsFy#wMt=P1cmQq2ivbdl#2_zw5$qES(2y5ODAz{k~$^6f9-^`mN zF8%l0-{F!G2J zZnkTP0O$Ku(RAIir{8`4AVcbCybj`co$zj?mREJGSJ>^n!{2NeV8lfBQt#d--nwg6 zo6b{rQJ?GSpetYtCv*-lbf3IkT#~=c5PqWmgnI6$ouSL=9SYv$)~p7*I|ze0bR&MN z`Xj_hDMsfQaPr0L;>i)kueR;wEcmS<>YGnb_F#XW+Fur6Ew!7mVr?0u|8mhL2| z+r>)MXijt!dg+8c5fWDm)cNmdmwGm<3rivu%9Z>259#=tl`0oSjM@)|L|Wv0iWNi8 z0+YJAoQQ)j{HhfVOf$Sb=reGOJ73%JB<-Fb0t4-EcN#(K^g7~t^L^A#lFk3s`z3Y! zt((#uE{xPi&m3m{#s}>`*&ESo!WyCaYd>A`s79Ub2=*VD3D?u=-Z!c1AY9{_jv(&p zr&-NXQN)e7181Nj{(wc8{LGwL<2r*ae-QJTn#Q1bc zL7vofU3d>5g(R9a!(tzlQIVjolW>dRN!=XFaTvy6dw6 z<}!WnV`F{g#~@%d{Sp1&S`pOw)$7Yx){MkK;x&~9iLBAq@7ypG$Zkh^!4b_B%=t5{EgHtsh+h1SGgfDG7P$QoH{6XkCu7a$)pb3WT3OCVYAZq>i+A$l)ZIpAEgdu zW_ABXyz4NhD*gN;JRejpI`A3O&~$EiDR;@YxJ!nnbQIZE!jZfUM7v76fct9=MP@$% z{dEkU?}rH+4+iDj{icd*zs1I?w;4QDOfX{@=mxvH1T77fAH~zK+^#O+h>`TZwlA`m z{-f$k28bVi(sx?d$KHT*m?@@nQf68_{K(1hb5FcW%B}>5H5nX!m(*C~lJ=GHl`;mN z%IWTl)#yh&;fcBV4(-H?iX0OAdD!$2Q`NuyXkB&T$8@yhtW`T&qDuLn8^mqjACdOQkQ+GBJHY+iMDIca+b{Bdn`V#}wHf+YE$-dM%=wWaWVJ8| zE$43r&1%6H@V`d_6uU(4(cV@KZ~?iR25U`c)o23 zdK86z%%ML(_9w1JgU6=QuqES_t&x_Dy50S`8b?U7WNg{pUr0WW#f}Q%8e0o<^d&LY~K~O&V~h=dgm5iQ@?#@ zcyG>^>~9U@yS^eAdUz9cU&0v`r{gujkZ5DP@7=(7PZQk(@J(UiJt>)m_oUHmDVP+` z(p%dQpluhRdWCuiP&au4^6bN=7JO*^^v=H3qKynsZDt$aFDhtC;E1q|6zJ_Acv4$k z_by}kYt42M-ru0@e@GKwaRfX~T&34abSsRy&@NWVJb9Ef5*o^# zn_|fRA)S_gZ9VSG$%V$9m!|G`7iWEU(yICh7^~C3lNK_S4KUvz{cALC(oQ1wed50^ z;J%M!b>eyhpT_^mdi`Os(d$FBj9d?Cr+jFs;5PL|g)g;D`@*TFPExr`vK^3 zciTNS+72|$>X~kKcgJRD0<^o^m}9|AfAbHRGka}2=?t{&!EMeBcmd$Te^9N5du%)H zN^~OM)q%GpMjDH(9-A2uH!*Qws_s-e@$H2SHShObc=7t^mMWWP`sjgRtUt2`bK*19 z``#hQc&oC#W!1p%WoYzqqJCLnM@8TQ=>oZCr)L3B=y8)>*qQFKf>sLElosKJP>T<0 z$|F5!s7__C6Yr;A+Y!Wr-Ul7ZS%(q;beziJ$chDw>$*T5%q)|YnUxNuMm6rhZAkYg z%lXwg*W;cgE9mWCltGI;)~PQ42WDTCj@L^PNcL0mV`bO}u--UTrz@~jsC8L}dg6~% zU$f%3_yFNM<O}Mq~M%gpek$QN}PjSy?X@rT0 znGD~vP)Gk0m@LkrT(93qt$#-my80+01#SUoH;@4c$+{9R73(zS>{2iPdk0y5JoC#y zP9z(0t|1Hi(o&}GPu8L&rw5C9H#+91`qm)h#&k^{B8)0@Zn;+R+hGE7^sil^RTXTt z(&$8zsW2PHo6#N0VFxH+&>S1o8E+AzUjw|qM!tP|LC$56Jv2j3p`5f|J-L=n5|JpWK1>P@)z$Dgat!5IPYS zEx1lAegCSiV4$LxEmELy3(irwALG8*+LwFWAI-<)@HoH5crF=IxMKWHZ za1AbYnAOJZ++=uf2uK&vYT3U94o-1;rl#Sf3_2=|;f_Q=4{wJ*NCs+;)2HtK6}7|^ z#1axrc9ek#twR3#0?>>Y6a+QX(jqp2(zI zPNbWbNaEx=;`YLAOgi~*Buj9VWsXOkStEs)ZuUnR9jfTc*NJM6+dYoNWaKL+IACl| zPfWuSM&Yx?K=15CP?V-I(KQ>~e$|uJrJX4cIQ6tdd!fFRA44gDo0*(AwdiaAvn#vN zfbM-M*cyahJ#^k!r3N>Op_Z!Mi)(e~nq#GWd?agOZ?qYA@7#^o>v|@SwkolKql3^O zZ{#;u(Gve_$Y5IW0QPZw8AIK$E*K1Kr0aQe%|OyWQ?@3^Hv7i`q(AX#*&uf(l=%Tg z4xuu48fRczo_gD7AWx*tL$F2>IL>6ZV(PWPAkIb$%&eKAVPHx3fpUOVY@ z&G0QnVWn~oA@&<&f-Be20iQRDA~1iN09GJC`#ZS3Utowe$Ia3UFQPP+UY){uKNy)h zi&AgI)LSq$j>fdd`9-P>sJoBcb0O$d=9vO-8bc%*!jUOz;%eS?Yn4V$P6<38h8Ri^ z^MGCMPs8caZ+G?v1OLS+H25P>$R-r9ZPc-d!Z=1D`(Df{D#rmS{uY@+N0TT_Azp=D zI1p3`LLK-AK;az)P?1rA8e;h2KZ^u^mcS1o_<96?pTXk;ZyQJnc3*E5 z-vHjEgFjvMmGF3U=fgN1{HNaleuxf!>A>)Kp|XPK6Q_4NyiPrGLpiZJy^7vAGJnN; zQ!x8)hty5F)WbyTD`P1O4H< zvUg?^txuHo@)KBx)VXD}kf(c;eSrOFjFYyZ2n|ay2DWIt%iG?2Y;>bFovYcC z{Ucm`-`d~MHw*RAm46e4TC;8wS+n}c0_{pX-Z4~F+8a7v8Z$s%_^Uep#`b{^Ug!$8 ztQD({w|%PZ2v`|uFVmj%$|+!@*Ddd{(d%6jKEg~zF_(#$6^2N|FWJI6c;nm)vUhCC zGhtg;9LBYLKykD5FHj=g)FMtaS(GE}zVB=Z?Jz?V-Ed9~`UYB|fx2HTpbJ>Y-kY^H z(%#$laevDgT@HNPUk5OLMeO&`76W;cja}Qka?QgRf@01Sqg_W5xrOD@LmC9kd%k4< z$Jg@D-a|)Sx>KLE_vS&GP3?*GY1tK`fd&L3*eUqVyU+&GW%&@spG%y!%{2% z*{dxoN3v8VC+f3L?C0wvc&I=%Ee!L}O=}`~=$#My^AH{}RL}jTFAw!u@A~t~u88%# z>zUjfa zqWRD974P-EygI!7L?;gMw2hy z4-Y-M>r8ESN|P@sWqx(kV%sA8nit#h7eD=y;?0Km#m<*Q@0Ou5_UQ@E|Mfw9tFdPz zzn8rNE23@)aO`geiLnqtyg7^1*F^+gzRoL*(0J;fMCe_EP;Cp+#uSXeeF2_ZV`0X* zc#!nsP>$u7Z13_$-+q_ZF`>Zjo!2D$Psex{80FLoX_+4;+68iIk+jSgU$)rxD%nk& zU>L~9y$kMUqpa|=z~0Gt$1_>`lVAX&C(GE9)RKrs3xWcb9Cf}xm%-5YzS9VDynCE$ zu-K^8wE$>AuNl9saumLG5w7xj5sIf3?e1VapgY9&YW5I|FYp%wpD%XPQiMTT^bAqf ziwIi6_sK}+=qSnTh%!s&X;JZ#`N61U>HhSX0_lE7OrbOqB1dB6{nKJLNcTS&r|9--HM6l)mZe0Byt z#%DvL!A@=GQm?wUkaio9)#Ri@Ic!rpL01aqn1eZ$>hMdX_vkHO{rA7)xl@=hU-V3h zhN)ooHGBYeth+NiCr$QV{^a9`?;U8h&iNKTEP45D;Sk!*m97-A{TNR=T>f(%{1G~g zN+2a_)N=`(2K$6S-R@^W1tiPAUT8}nuHgDC##FN??WGGh9b=Nt5B)wlwOE$!sJG>xV!M%cjCO?jJAFI^$8+%W zeS$}G@MC>~lR4PYCpd+J$My+Mkq{#KA+ff9PWXZV=+`mdJbwGZk~{{TRI=lR&_ zk%j&DTrMm@U)W?W%nkrzK%KuIyXQV!19EuqdEo#EF?FPoUvfW1t#qGRv+Yylc>_n{ zd10(3pR@+BQIKuad-&K(GbkCjuNT3$3-CWN5NKXwTQ@Ke1H3H&@)%$=0PKwba1#I? zcDgSaBK;AOE{>c8k)RBYeK#V~&XGwFnTD^wL`2>Mk$5fuA>vr)#RJ!#`DNY{`Te0QmL>~}WA8pKZ0 zZuN`oUN(3Bu{e+=V4u6=LeSl5n)fq0k7C1wvNvXJ6x}a;*PoyXrSvkp#|A1Mz(*%S zo)zzEjgPt7l^yDTXLI|1&nV8Q1+aasCG;$hYSVI^#<|7}8%()ER?8!@QtuY|pzX^E z>K5snz3Mc_g`iE^^o-HA=#*2Ed#(OcgRRExZd+}%{ju%Xo$SF~xKH=?oU-{Zz^MD7 zvUu@gS$fwm`_=0ezyEM_QdzpOEhh=v#yfV};K2rU>OD-11y70|*6hitRsViJ>G#v` zlr~*vFQCqC$Q`k4P7#Fz_|A*=fJu?BVd988p(hxdt$?Z7*kjhv%8ZYovi z)#hF_9e<#@s? zy=y%+TN|WvcqiNqK8IxM#nT;Dm@=XaU5GMtej3Zr`9g+1ofaX-<_pnzpFS8bPzmiM z9IuaY=C?bPoGrZ1ia3qjiGY?y-vr`-<{12qtdi*Z2meS8>Yi_=8R|NHmg#v{`C^tcK9(; z=tvCt2tq01CAL4SKG$oBz@cLO}rVUjoYcFHv{VL!cyVycI26G1Spesu09IH4zyFVja}F$lyHNsvvVz}MIEEyi zx>P&8F#1b4sy{zjl{$Kunr@SBz>HC(&^?)uW6!D%-n?E(Y6*4?QbR@2sto$`tF z+TnKOCx=&*avg1G6X>G{3XWjA=t^X~@u8N)a)Dk|&9FvM$EI?>~KAn&4)h-!eT@Z%4L~1$A zo{HLli<+p0(4!NGV9rFW()4c;>e)X=g9=pp!uSig9-n$qmhQYLyGw_tcP=49+BX7DX zh;M-_+B-$KQuKqr^D4^-eHTYgQ5${-T^!E|M-2q>27+w*9dAaP@(Opm_65cPj@9+s zFNGK4k{Ue`D>G8f%@&n`F&&WqxjKSs2jju`#PrmQx!(y^SlU%wEyL)mF*sNZ5ibPn zk9FXM+z$afD>b;2@ZL3~Hb$U}yVz+b;ZFvLT#t1*S%8MH0iJ5s+wb$+pZg(g8br*q zuW~%ru09IL)618#=XQtEjwZCgdI%H8=5+gy4(Z1#yuX$SR&aPn-wT7ET3+X;EA zN$+}$KEre2^U-aX=H;vPZy4#$x6s+gePr?Oq}qFPj>*ax-kLylEv)3arg&x^vnzBo z^VFgaacV|Rt;gHYCz!b0mba`LTh-<+bl!+S7x?XJ6$*GfdS-&upg_$gJhX= zuvM4@C>}eK72cBXiTUHV!_O-!`&4;R2b~u3ju~YS?nWzHo4w;a4BWD^itYwAHq8E& zJOOtF&v_IQ<;mVv{A}YP^}$M17Wj(2)52iqP`cC)DuwYD*W6Rj&y{8GR!pNd){0As zWzX;{zpXtdQ7J9L+d=|Q@`C%Wzd*+nB;d^|IB5%9XVmUTp?6K!b@ujKZ%to3|5d9y zXv|L5Sr%K>?+bK37G0Fg$Vb#`YB>095$vI3ZrHkNl!(G>dd1uTlf9~*tR~xIHqwbQ zQad^1(Yw9TKaf34vef!{mpPQ9&|~xX#@?IRe&|R#C&IG7Cwua-LGJR7PbPCG49M?a zg#)lN&u%=Ju5?<1pknx?m+{?s`);h^%kJOM@e>Tb1@gk)PeH?z+%?fCd4mqm)C`AG29vc3QvShcY_)uR zuLVSywCEK0KgOjRR{=A>(qOMSVcohFZS_eP?EX`+*@@WM{imZG#@@TUmz^S+_f^(AHvi}4m zDX|;t?ovKDMQ_pg15x;5sAqbC(jeEIlofhZ(tQLby*p%3QaTRG!E%=@JyYb8vefi* zRw?sJ{CU`nzmLb`@4RG})L4f=B>+^G?vgTza1iQuNi9VHlRFqZ2OwgFf%YvmRzXA! zM%-@(-*JdpW#*U$i1{329*zg!1&CQ4&oPG~<^;w(o=nB2aeSkfE`X3)c4OSsG&!M4 zPAIV}XY6r1oXXS;Iaq-gI7%`w9W?ztaK`M;5-0)=}?@Dol2F*`L?|VI6DxfRLM1} z{9%y)T`MP)IpT=Bm@p@yQk}G(w+)is(Bo%r zp@7|vxO4V|S~+2lJ?;`tgXDyLa>8yot^-XIvL|PaoUqrPP~nI>NSn%YHh>@C@*ssA z*K5afaGeR-5bTJpS)jo>J2*t?PPhTv0ik#q=kK@^)GCbvh592zOdAM10gHxHD?1WCQOighrr=3b>8ttT>C)<%-(Y6tU zG&^W8b9cr`Fa92{7wsqPVCa42EgE`vt)h{33HUyutzOG7Cyn3_6h3x6M{kACM|9y+ z4oAPh(XVp!8jN0!(f4umw>a}Gj=mf-e*mLzyS%?XLF?zqQR1|0jCol2aF|y-I-Bpq zl_D=$8NgS@J|EqjnqSF21N-VP5hpg$(o)F`d`0;Zzo|?iMG&&0lTvKuOgD1+6_mp3 zMKJ4@p*V8v#^Wv?F4VKJl0NhEfQ=VBq<8jBz!$)IJr~!{K!om{AS-7a7QaKOkS!M- z2_>@SpaWVRgk(phoY02bYdZE4>j8(%odJ2%)4HCS1y1Rm{S$yu7IRrmoZ(r>um-bI z{0<99rf%6%?o7ZFnsNtttL(}YliX1)C+v5`Roj*OW2eE@T`#!#JL!TKgOppj)qO zDWO`|Yh;`hPdFZQ#O(zfpMxA{XaY{)xQ1~g3G|I{9IeCgH8$quv-acMqM^Bg(KH3W zDVoVTG&e2gOob@+X_#(dOwHdI(-b|X1)M2svd%IMQy*hG>KkL4rpI(8jYg|T*n>m? zO#PT?8SR9i=gl2d4vg`ZUd+W(M!Z=hCp6%xV$8G>GL;F_vy5pWWjYfNLDoB@w?W!7;G&2azUO5hC7@O7@*t#{?M5ia~2Zv5i<_rUg zhZ2S6(&`EGfa}%b{dT*%6NZxcTcmt^9a8ofWslwD@W5;=N%jnqJ$IS3$pJ(uwl$z^ zTR%=Cts8XxYEY?dmK|cgo~|5rA2!0Ml&-YmHj0izd@i(DPN?9sIfTCDASoK9vSkl# z0?wzJ0F}<9gX&RpI2{$L(=e9@uUvDKysg;lEihd8Q2bjuVQ6z$ik%5vc-*LCpFQD_ z+)*RrC{`gWU9tsu=B&d~O{MH}zyJsNcfdH=O^gIX5%l#2yVBV*QYVk?N*AuqRDEHu zI%N-**n<_IXr@W|iJ~U}2^>m_L&*S|SYfga|IKw~I6S{G-R&I) zGSpcJK?VxCmadReysYE!48tP82lwh1|_-*lF5bUKs#yybcZKB`s#L$_) z1NSe0cE3Sd6r={}lohe5V+HG2TUq0x9Y{{4113|RoHBaTo;L_NJWuDLWK~WGesFja ziyfZm5~pWaiNkYQkyEL0Vx!@Oz7ES_JBste4ie`M6z7iZ252N_!hUBQ5aO~@JY^*Z zefCs;&IeeUt%Wj-E1pm#cyL%?ma zr5hBT(D(rwe~C7J>~|_KC|$Nq6t~fl>cj%qfIhSswXp2lP#w9;6ODG4yt|a>9e6l~ z{a&m*2How62E{b`ES`eKD7zBfh8iGtX@65R44)gYrHt4bB%Gp9Sy8Z=tx`R75xD@mR}<*dw^ksg0I=iapV(wr$4XiuwTj_!t^VV&@E z*+uxds!JMuRJ#ALH2S1;zbZ{@knTSqO{CSw8L4!)cg$%Z4NSM^FNXPLGzmWMRVi}R`~rLGZYEWmm56kZ zemj^Rz3?gKX-1AuPHkh#)mw>Xdms)dj3k53yg^1%`qCFo-;(omdi;@`C#>=3avrnB zHvsv7WTLgxGv|zmWRR_<04Yu+$eKP?aKJQOiwuuMzd(>+C`KPPYF=B5+^Q9Q~>b-ne3^T zn25WXkr~q$?az7ox|x+oz#=4I7ZPwc5U@SMUV!xLK>D3W`n4nfwS5f%JHiA! z4+=OC@B;B)JMtgUs{>@zd7xE0&<&=hsM6xX1=0(|(;){C2j_Jlo?s{%*1>9z9EZd^ zFT;=o*?kD3duQZ)MrSW~cwR=$!0uPalcl7i9)^Mp*&i6sL-RQ)^ZpqyO@GGclCpg1 z;Jpq09yi0!!|~_RrOe66E-ACuCAD^s{(f$gmt^5Lp|rD z+==jo_?FY)&YXcIJOQy^0Hg}vvjBYt+!sX3CeG~;L7Tw=04?XiGw%sZ`W!?b1<)p4 zpOhQXB{9$Kyq*Ett=9ko0xbF8Xm zt$?Nj%WlwZ9H00{pY$#83qh1 zV1Q?4K!PVA0nRfBj6w@>FT(788|J9kw^k1lU==Xz&xoHzNBE>9iYEOO6f+%ARP89MhYojXhC&eP>t zpmP`M+#7W6cwM>4I(M4R9j~ib6Q?}}gyo-V@U@4is>>P83p9WuHGO3yoDUFs(`+kXbIhp)=;i<<2EQ6`Y#BI3J6`t zRBYS@MZ?gRSpx3u7+C|3S_rHJPb2WjoHHQjW*@#5ipZ*(j)5SO=A6aABNSMUftfo{ zq?EfT{lyjB#B~)Xq`phwPyjPs>mEHY$7A-HBqIduM!{IS^{^bzmFD^KwD>OKGZ_88ppn&zV_G^6W>y|h4};^-am-m{a0szzX~DF+H5725q_OlcVyH16 z$0HKVQt3F_L*Zwc?*stw1cY_4N|3pNxww@O#6^fBn9L6#hf1(;WY?DS+(AU)%@Kub ze97@xIP)Ilf-kwJb-{z!%OFb1Jdc6Q=dFZiuZBYd;&2FY=t3MWA)w&Iaz+L6 zJQl`91n(d*CAc&55(ae(rY8iLiK(-N%CFSQ*ASB%9DeEwdI}u1#CW*?D@(@(%5Tt9_#iDG^IgqP! zs*S9D%{gE~4GWoF^g&JZ&Z#o8p2#2p1mRNWTr_dNp0gJ~TFwapj)=`^MA(BkL@+t# zbR+EN;J~#jBm$L>h|SrLu(inIOwKu7CT!Qd0b^Ts;t*b`!vi>B z&OXfhVm~|%As$U3JRt8z%)5s2-rolm$h!t{Y18Af58?L_d{=*5_8>0Y3lJA8H|(5B zf-mla4#3wVKHNzKJ_iwgFTtPfkI!DB%_R+=bZrgO$_y(C@@oyoi7GLX`H~K)T?kue zWYNP&i3;q(3h+3X>18bJ6_u*(eaR9dqbV5T+9jlI)$!+y10-(-PG&ExZ8;m2gLy+qy=|Vf{U88galb4 zrVSII5ll9GXoL!MrhXmV0YG^u2Rup?YqGVEs&JZyu_#07z zwQdXPun>HN5{NqNr;$*^)MJ9rF+p)i0fu0NS;1--V=*%?AfHYy0(ll9*L={ya zs#c1eaAD7_)FBaA{isi|RB^N9)hQ60lJF|v&) zo?&c|4xJ$7fqUkmHWrDX--xbyB80?&en?QXz~VxGED8t0BI~QL$m)wlp}^u;2#dyn zusEh+ajriWc>`gQ@l{x4^u>bbcbUgSNHh%49>+Bt&h*Ejav&Viz6yu5zBusQGgD(| z8eI3Y+{xT1P&vUijYbj})mY$O%yhqw>8ayZ3Ux!9+_o_&YXe9^G`-}~ax3Z{H7I*) zX|Bb&WP;$DQwf5&mQmrcj0Y?shgXYUgJWk6LLL)k0)BHYAv9H!q2|<^n4xPyZ)qvR z;TDGy=wov_5!4@=8j7(95U^q4={#;CDyZG_u*U-%&pGFOZenc{z}s1Y1l3?7HVkTl zbAl#_Y3ZfGNsK!6CT4a>aSBtYq6Qk~P{46g*m>mPu_zPgj!;K2>i0p2Mtz>i3j8Y& zyb-}0LkMAa#{vMMqZ&=1&jA)XB(S5 z$q}F?gVAhV0#lh>7r^S^(j8&!i0AdWUNg_SCn~ zN+!LG*Fuq8q$+Q(H*CQjcXCp_?C+fTeRdbi3or6cz6Zo1edVAUTOZ_yMGmSL_pvZy z1JnW^jeG3&yPJ%1M?D*r?VbfQ?4E0!coBwj5Vv?J-MC0^Icrz0kQ0j8xNTPu>b_Zc z(KdqYqt#j5bYZe5_~Zn%lHxWFJWpu%JoE%X?4b-xWDD*YF=M6)6|$wqt~~Sv9w-F3 ztp0GL^l({trV!ws>kk*NhkFi>7TP^;<&ojGFX(6;D1qQvLx5g9Aat@GdND(%0Cd5C z&?$Q80)|cl=#>LPr|F?rGPDfPg#$v%dgwxi&aitnv3a@!n;@Z5#+??pyCOqQI3rv9 zc4euyy-aC>Pela=)ZQQiY+7r`pVn}G8vb$80-OX*Xvh}$@ zFlcv|NOt#$HrexHEgd|1sWTQAkY3!$zBcxKLMwK!Y@@)XYAnSy#nM*O4fKrrUUhAv z!Qjb9pG$g%1p5fINPhSovOBj{w@lLSUpyyee&8%TKo;vIPELi2z&L5!2%e*kllFqE z4#Mid*8@6nPmS#M$5YBYPWiU8fK#4E_`nDrO~*+GF(ricz@ShEAwB!$g>7kE(51$>nI91Yuqhu z_Fh5mD&tw{-95%CY4eKjQb4zHkH>qJi0g(-t_f!nuSrN~WrtHaV`;RsTY?4B=0((M{-!9GzcWhu ztn$g~>C+7c|4Fm=vY#bYf)rCqT~)z^(%>HGCsik9>78DqyLpi8Z~mzqw=Y=izL=K% z9XaTSmcb%&*0tOiv7aQ>?)IDPO0wPUGg*RAXrm4|??c+&O1oQXP9rb?dOzJ@;M?p+ zoB-BdSo#p{g5R2a6s2;nYPt?NH2LV4a%g%O0pc=jG~!QA^DpqH;i%wG^U?nde=bu) zthHQ4{#>bs8SFK+4}ZR)hWPVMHO!xHg)=GCCI7P;;?K8L;?KXTU(cUNqe%W_sxdz$ zA7vgJqYE1&!cuf$DI#pVE^NFAOVfp+ThE_cM)A61%=KsE3@w%da4xaMQ-V6QpJ~|C zq4h+I$>mVUC!XL9dU&%Q-k`x_s5}Adne(P6zW_<^#Rl+3r@0jJOPlkTg#y@R2iJ0r zdMy-%E>Hf-kbfluyb%g`g8<$P`QK!Kw?YAL5kO(cU&sJ|4h8&~0NxJyxi$0u8VdL; z0jvr6(e>eTF$RND)aN{iwE2VKw?N&Mj--7q<&)u}J%`_fVf&IgFzglz`(XGDI=GuC z?33Z6F|0f3EVLcPV_KAY%HE`2m+}F{L+@$NdChercTt5urTLhx!BCoi1AU>DOY_&O z9JE&DAn4?k=TYxlWj;w8d1z4bdNBvEievE&O?o(E&dITU#Z2v$?#yz>6*oc+*qMw<7O{!yA`? z_nfz2D@bpbObiy`TQx&vQN`wnGe7AQwN z##GUU%3I~q{P+uk+e-6Cg`MWG(-d|Z!p_d~A!l3IsfL|RVP}2Vi5y1fr92yvsbMtn zj^gjS(fr+*!r!g(N#Y4!{Uvv1%#Ekwco#r`W_@!T?WKNP!?%8h${fD{XtPv9^3^jS zi6v;=iV>D}duefy=D1?vI}mmk=-h>YaG7m7ccsqVq;qf3xziY@)>B$Qh6o5j%nAe} zPNW&M(@P!o&V+i0(&E5P$=!~mlMYL_9Je2T{oNBCsIb@AEeE7Uzc*7q-$u*cK9#4w zWR=)TsHCHn^du?hl1iORhm*iRUuYha=lF|&Au1xfN|8B-wDg|4&cLX)Zp>o}HJ%X$v)qkl5yh=yqhn8KZ_V8r&!k>WO~gxXx%tTynAr&t zTYVyeR{E;RYQ=2Rx}r@)RgwLsQk1c-Nt6-LEK6i>-Iylg4`Ljy0cPUj+qw)9+=Ici zjNoIUpx%_ZYL1g{*NAsf%ZHhLu5Ye^^j%@U?;T&=Bi6F$Nb&oxv6A$rQ(E7fb zt4Rx$mw0^;*8_9CqS~vQLh+4dV!b@qE9kXb3#`CEtt+4zuTe#$T3u|u2Trl!ii<-U zcSX4o6nK}|paEl{iCNf6OHE`{6Dwomq}}1Twu!uWC6= zyB69+kQSm??M^n{j+2^A!pe$Bb$C|eDBLoX#`mfKs6m62-R=`c&=+l}R`Jk% zx$L>y#A7#4269HUBo`n&`LYv3X*Pt?yr)Y8xwfovcsH&+KsSHhfF?Ze)*E(_?z#Se z(B2M&bkAGrg*sZ_NB7()bWc**yk0Dgbx-xPAN5r{|NrZog{4B@+)z4zzUj`#;6W~H zsVy2t`BEF3Tdbv%$vb#Jgb`CjxHN?Dh`tE_8~9dh_*M^queEd%Ar>pp)MNb(2?e}le%z48AheNW%;|BSwRJV7q6B`x}!pc|h1 zs}uX8i)R!gW*8CplX0&S5ne73&Yc#8YtoCfid7;+6+?)cTo#4M3&h`Zc;9~>$Kz7> z0zAEMD0M$i-+_a^vOW<;jQ#4Ir2Z_E9?38N1NcWWwq{(r{KUaHL(30vJO*F-oo)Pj zXXdawd52Ff1?3LAQ?z6Hos|N@ac5Hq1TAXd7gmZ*d4G8Xio7=OFOTu(m;8C0KTq=K z*Zg^!Kbib_mOpd&lf$2R{P_)j`gO=vH;_r`P6#ZG`bu6u5MrJ|qwy^O(Cc(x(pBrpNE8BWA`pk#FS3&2Z{S*i8mG{9;8-4*Z$ZL z%M2L&6;;#M>6RZx)+y~jR-ZnMR3;c{)#qrX)%8ETTO%4L+nJ8^HfEJZ)QWS#i#KdU zr(>=Nm7d9QLH((YmQRQu|soIXpUW)V~^&j z(i}CKqfT?|(;N-xkUcT4RA|e6d|3>x#OW`KSyoGHddp|hn%?3EX-%&hJ|8^+IjB=P zI)|g52eHTand?sGOfYBpsV+OiUbKw{AP2v*OT3Ok+5?B84?x~mq z<)M{9yL+2tcOPqmUQ$pK)@x^>>K>O^{fIvE+ zx6h74PPcK|%q2E6IeHe^@(z+?iLHPfxf)C^!-)JB*_w`ulx;^v%2`^aJon_x__MK;6Hv8JyHVxaA+*|R#O9A@BcKd5$B`No?0^JN^n02 zQ$#tne(q3u{O0q(4{-a0cXd-e-M9gohg{A9#K0-IR@$as#KQks z++ofqe7cz^&sG}A*v5PNaNpyzxF@_7_ea)29zD44zOEJ=m`~kyTG#{!0A9j) zE@|%)>buaBaDtv8PoUFn`B(LVsCX~R`p-D)``@!}y2mPQj!twpMJc6Kt(H>oIT8oC z4@W6wRjQ>7yweh+-AAI7s;Ypc3cT5gvF;O5%Gs)umb2h}Fwx{b8l@bpI&C>(nR9f4QDn*8JrOn!Y!ccj~;zfj#J?H}&ImEuOa|xmJ8?=(Uw=ORhpK>QEN0-CPTfOo^|UHFQ_`!#WfkA@ofL8iSFM#KF_OGH=^ zhhdJPu=No3(d=mQ;7*OV5XQ50Wq|dILo3l6lUx+3S)op&*Q`=1NH=<78tF#2+)KLA zQ{el^05dD~LV1iK;(kuq`x0K3D5rML{f_K?ISUWErnb!;j^k3dU15!n zl!13??w}64w3>V6)hocUWtxHZi;N$^9FaMEDn`?S%xpZkf=eL1ogCSV05|`X0e~51 zw_OQL&|3t&{eTa8Q%V}1uAJR;4TI0FjOq{9L~yvRe)h2t+~EFjWdw)(H~Vl1ZfJiv zmEiEu+U(;92Mjs8O5$^!v(FG*`8nwU+;1*Pb zIs}fIEBPM2+2;sqKR`7SR3n5Py#}cSqjqBtI64TT8z4#vqSz&9(c+RGQK5(TB(jrr|^DNh~&$4)At3`=+t_b)$*M{TnFoUf1a^<%yjl?vj2L|^Gj>h1j z{eojLSn3yS!r*WB3m$^OSN01Y=2yR$$YF{<6gt8lEVjFg%+AyXDgOt!i+3~iLav;_ zK$SBX2B@4?u}>-FgiCoHd#u0rnm$!?DVt|Q1c5;^zdAG#Kqx^rCjf}KWSx2WIhvD* z_u=nNmp0!Q?Mjcj!j-OE37#o~z%g|YxF$jHjj`aH9BuJgKgJ2RzJXi?cSxIuJ(N@i zAIC${M*L2DXi!pxu}0clDQ&I*fBHkQNk#M>V#M!*51Equ#s<5)I9A$RByDc6FTD6M z?q&MGb>nMAKM|sT+ST;ZHB% zFuB{MI41YHZk&c;(!%sJaUL{u(bnxfoSVud9ZK~b00P$&=Bvi zdmz|^!9$?;x(`pp;9>qFv6MCrBZm0{g8*oVa1U`G9*jUEyu+#>cqj&s@CRbRJq*~{ zYwJtVGo2JDY-AArPkU{>#F>#o{$i6@nxCaUMP|%q$&uH(k-$8H`21ok? zvF^jU7#!#K|~$0x?&<5rkOY5oz)3&qmXwBLmuBW{gAo%P!iI zjjs>>;zQk*>=H?KpJYmi1S#humC{bf=TE!xY<#1--VfL$C zw1H?`KJ|UEZ-7s$2)EeSqiyiXj5$~pYY1;X606~W$rXT!9)%M0#~VU=-X_FUS*jJQg>5mvADNf1eJfuVlmat>qTUQZFbqn zJ_fLV&;Hd`F6>{1iUI9k3o04I=PMaQhd1AF7&quSyxtk;2=DjC9R73ry;G2;cEy5J z{@VTCDG2CYp16VSi5m_RO%ZWCcyZ|OxQ3r^Izrd9QKV-h`G`4!$PX3nqsCx`(gti1 zSMO35P0s@Y1WVn)Z^{o-fC)+>_#vq02WV--UfN2486&g*z9Sf9>*9D?|6gRwY7|z& zJaTj`vbEKbV~GvZl9e#yh_DjQI-;{LW@ssAX(=nUludOaWwMqs?MQ@uv8jnRRvv+m z>&AS_;70GG_)+qG&1Bj;#&N%(KyMKRdQVWG*TsouNbndnc~Vh z!53WmX-@YPm;MYx?qrZQj?3aacXOU+InUV)S;>&+88Vw8s~NJLA?Gq=EkkxPzlH)C}C@FnZf9;#rZUiTp}An%WNa$cFsowLZ%1v0*i_PdLiT#mt04FsR8!Kd|w zPuAdx*Mz?I2Uij?6nu*KvyDF`{3+v4Ie&KYXE%Q;`BTlGTK??i57sKQ-~ZmC@P2>B zx{>$qbDbJ_|GriqeJ@%fdWe8e)5B}^(f6G<^H%$zh4aP$yT{wET$m z6}w_H*xlPh2X5N2d|w2xls_*hHbti=(Ut2ci6NPi93IwClBtI-kY%*9jx3|Y2lMj+ zb-W6xhApGx1N6{=u#ENuenj?`btuY&WpuhvSVlh)j+={xV{Ex_OsMjOmI8GB%O+(` zCj02M_%oHvqq$@ro%f$Mk5+#6Z0;C_@+fTYB4 zdY})uKVHZE@2JTAO@jN62Y$lbzo9wA{r7Jb-2bU?+`L^l#_kl32{l{$b3f=e$p7hy z)B&y{26QA+7kD9&`apK#f4UP?eDZH}g10{T2Ax3pIB2z;YFVQ&WYB965Z_e$O2kN z`1QxSb%dq4v}S-kEwpCvJcq^W!(6%gb%Yb@eFU@Bpz=Dx>v>)H^iw;w4L%!#r>Ig4 zbVq|h<zh< z?jw-L5D`BN;)nXxD2N#5KN2HCN7&Ia?e-5(mo^z83XLiLBc^nUzD&Xp|H&9gI>;Lh zxZD#3?BqW(RHPmR5m9~>KKB5h<)TrCBLIJEv|k+z5i#EA4EV&lk4XL_aUwp}zOajp zD#L=>Wa(C-2Q)zX;m;ZTIUI$Sb&Sp(HktMbA#^4DIwm)LDLBOE zLcGxK-jX)IJPXakJi&D-KMBuMD@TrKv zr#SGLh`?t!Fe@UEO`I>!jtFD}=F8a;foz_9d2U1?`YgV@PtkfMNiP?YUft^3av3{z zX!~zcZk;w6`^BvfB29Zi(DgQ;*D+be&o3so;U|4^CqzJ(>McSqwpK3;2)z~1ETiNJ zc&$Lr`{2ieQ~oIonli{oH)KRnRp2w5KiT|21Uq1gPNq&eARz4Zn&E6_;F*?SF!DSg zzh93}C%TIP=x7^?hrjU?wbu@?2UGvfyv5h=!TEmgdDR<)J+Brz{p+iByQn&&q(wWT z=yixeQhvfvzE^n|4S?w|2*bea>q)1ME?jp!#)VX8(<8~YDDmJ0Conz#YIAOkWbxyJXDTgDjtB#s-xoZ z;>0?EdM-lX zMXJF`tp+e|)&Z;Z97KiX02hI)0RVk+z+%p{Leh59Z}uWM9!Be_<&hIB)PrlEfVNb3 z79CLKo(?J!(_N%=&06g;ThXNwdQwkMzw%JQXdq&*4WWl8*?RU1%<;OM9z3J}xrX#9 z@SMhUO?W9>R_$6O9bWKQ7eE)@j9J&lL6UIENA^2P zy}6d#)Jhjj)EyALt&WOm!=~9jLc4+j8VKO#92|i;b|62UptA9z+dACe(1VvXg8Zn% z+n;Oc4g_c$kO)2dNGrv37fF)rGlZlDGGY&&1MQ((+PVpKE8UTW!U5$GbX1#Ex0e_X z^Ehx^;Lq37kvwVUL!{{<_2@Ye7Ruq@V}{VPM0HNHj@`eAm;g4Hk53Dr6${(7QG+&TGXi zk=GG}iy4y6c785qGDI@$2qezyc72l?pA$7k`Fw6Ik&TBEuBuk$bUZ8xdE)KO(6BbV zS4HbgIFs%v3Q_yaInl^8?5<^_Yuy1RAql~B-7v@u{dr_tiz@FsCt4YEYG11rNUht; z<%!1a(mFLYb0>ECucS z|C%aqEzO>Okyz_|FzopFj`+;-xcv% zZ&@p(Y*k2*NVzM-F!{O0oSH=E)>E0$sf8f2Sz~IrXYkp&surhGWKY_Ucd+?SL_59X$2h&& zV-5Jo@7)f?M-M?IwaLa#oEYzqw0Gh8kzrVYXo<{S!!f957|=TxLSSNOqRm#p@|;>( zA+cdk-JktEt28?LHYq9kHp<{9q7+OtSXMUjLs5GHn;3@?A5Nv`q?9Vze>hq;c7+~I z-6MNeu!|qLQ;MBRr9&Z6q599E4nlY1!bGGO-_XM3;*~)0oRxRHN%n~ByYWs6e(TVw zl&T9cy?7&313BVQ0uB##3#ZbiKExpW0xQ403^bc6dqJ)7PBA-_vog%>_4ZL1FUKaTG&=LfSN(<9j{d!X?k+aKF3^gi1Z(_NnNT?jV{ z_+9oqY*Hsv>4)%P>{h#HlG%>0Y?aQgv3nMn)EnaXTFOvCfl&~mfA2{4432pJC^Wy# zpp!yCrXJaoJ33fx_c%?8&r)sg$N@F^sw!lzhv+Dgv(oOF532O7z_r%kUO;!Owx0C4 z?0LY1kYoCQK`EzaYk?O1qouu4#q-ZN7UX!s>_G~?Dxi(nobvBkF?o|Kl6Q@v01 z7(`IcDM*j+`}IMtkA1-X#VDszA}cj`%1X&8lX*cFsVOUagH?9VRFh(>Wtz@2JG?hl zIXsC~&irFQS5I!d-7_W;Bpg%phxGoHa@^_pMWP8oZtDw@b6Ah{RoF8;&N5jsQjju= z-VSfM1A#t6o>p;H8$jtfpX?g|!sDz}WV6QVw5sQk&K+gQ9;viJ#YgEX z?(_h`-My0Z623SO4WN|SJKAM5g1q zkLZoU39G-)uPoDN_-F9>=p;6+^d?%64gL7huI!}uo`YJKo@&zAPwUdt>@d_R&N7EG zzY`hM(o!)sP&(#`@mxVE&*=*DafjvB%7t((T{ELjXj`e8)nLY z+0aQRo9s%g!*iROi4Sxk|Lp+&Bi76L*y(}#yDtH!Jsjq=^UP@l#A!DrI#Z9&UFJ|! z;xxtXKL(uEte%Myjg_r&+A+~sC)DOQ`fyqsahldzH;2v(i1xbh4cerZ9imt|$JQLZ9IeU$61kyT9+xUm4JV(xOLlogk<^lT1@QV^r)+R{&QyJ#$SCr8igu0XV+BkO%|Z zc6(}%Q#mnvkJU4YOF*oGWmtlge~^s0L12ok3a2jeyVn)*8N7m9>yLq>dcA}~FjNG@U1eBcYb ztIk&(q2J96TzWmm5Z-^w_J41#NnHlbhP566Lmi(i>|^LJ!1BgukIE|-9iDkG#us;@ zBnwIQk6w!*!Gj)rA3E?d?jY@39xH!`)Fl8dve);QY4?I=g{3`6#XR0=M+DEj&ImqY zS74}2!D5}Mmz>fam-uFJ(4Evn0ORodwo~>#0C<&}LTz_l7_VOS9Pmra)s*R*BGX~v zOk1K@9wQ40%Lzzi(GE@AXlyySlo%y1J@5vsEnqSX!YqK+1D5v3!Y2T8cHM zE#D0$>u-V@8}aQdzHNoK&1^1S*23VxH38{;IOdIxp{6{7EuxEBIz?0K13tZ(8Xkjx zlUwm!5^1`__zHV!&jt<$Hl^wwzNBGF*>9XOr_`Lj)0*T*!cfDiw%vx_*7kbCs_M8Z zv$x!mzTb8Nt;ZR0r>k)}a!i8ec|Mo)gQRu6;u`KXQZU4p$ke8FD%rI3y8M(KP~7&g z=Vf}!F1Pfqkm;I#$f~wir<6Ti_O@Q9H>|Q;tq*)@5%|(ldr|BgqLlRAwvC9N8x{SY zlpO(sxN5-FDHTswP(i$mw#Ifc+&=(Y+?mEZZ%`{85UdyP>79C3QO^WR^tzy}j~E3M zjcnI7GfwPY#8i9(a4gv103l=GF63Pucq-!DD2GCx7NI+b!?UCx_iCrZm5Qz`f<2nUE3Iw>)!Q7*S(N-sf*DtY`fW8n!Y`^I=fsRXyP;r zly5cs?XZps{~F*EfLdC48LC->T8*hOlxlK4NU0{nuzi{7`(AE8w=*;^4#p|Vm7$*s zU!&gmzEAX(f3k+8{~ZOT)}GoW3)*$3WAY5WvlkkAPb@U_e!j@i+g57mom*p3U3|ki zsVfxKrH~FHx?sL`BnteBE`ET=Gm6WU0KY<^qy2ug0-I_ruB=+%UmTw_ORW*}-b7_l zdc+`690Q;;D;l&OYHHhgKK|zr6kQv2TEN?3x%5r@R`twghe}f}0QEJlpY$ z&VkqxgLnsTGKXfga-%6H`tP)YeMPKuSdzUe9vYqG`8ZHNkn=!C=RF&fB( z4G`wkE3T+K_}mMXt#!`KG6v~e|D7?2F~!w34?vE9B1WgRsv~oavoCj}nU;QbZYMyb z!3$uNB3zO5HZKi6O12-BwwkduEAqevR&t;SM#*|4b=uRl>LO=onn<%hdr zVVGm^d<>pd6HwgXD_=mTpY#y^sx~qHQUHIMtqFfw^kb_L=YMAec0!GqYw#wlHU+EM z%M(1Akn#K7CCbR^N`*7z|fj<}di~f6V%}AWrGJ;P?zHBv>=Wey@Y;jRbq&2bVqJ<0#^MVs=`e4 z42J~o=aBIE32Jl8PpHj?ZsKvPd51~Uz6u=;!%c$1)JJWj<%0pVZ6EEqnnUX z4JgZY&s`80e z1~jdcAVbF!SmsqJ-jp3F)hI{TqTWA~Tqp)V@fmbwn;^8Ll`~of$zfhWhU4*Hw~}ZX z=V7#5$7tz|Rd+#E=j;UBDd8T+b2pAZW_&LQJstl^=*a@9I2b)fC!^;YA9~&^2lTvL zMzH=#JH^2jD*#dox9;N&+`55s&{}iruF_h!<^;FyT5g>atKP1)&UU~13h%3@VR|=i z+ynon#^r@IZrN+myalaboImQg)vfy1@Um0Kld#!_N?EMRGDz_rIwYBVga%p?V9uR#5^k6)>m z^S$e#FEg%p9558PXca9-o(L%WJ9NPQ@{b8R+e@C&fQu)W11@$PBlM0aWAye@vumN~ zm(hgX;%Ld29&s~v+cS1=!J2!are_GeU5{@+9-|t2I>hKw4rr_46SNv)P%`n-#YV5E zJy1R)^NTX%N7t3H#MtZM2>e&vgCrx@v_Sb*w37L-=w-xe%b7o4lRszynU5R$gjA1P{Uql#TQVIe7S`ZV-JF2>0D7$z|KCxW?3oR#&KeTT}<#X7uh- zD?PVBSB|z1?D9nF@Z~(yr7Xq-A01`aVDGvnHj;6# ztw$pZTqj82+L*G#sZD)(bCMANDdmh~l!p_UL4PL2l0W6N~rB$xoTvok>k>(jtv8>cv@fE-t6 z>x!wJIMq;aEs44PAj42#Kv^oFBf{YOvh7tsu;*I(02G+h-dGW%^T5Wv9Ob-uAgvJ2 z69NRf!S^l$YX0E}&5Ew;cvfVGOfWf< zlc6UD$8Ct9$;^2!egY+y9U;y$_zmJb!|Q?b1j#)&zbZM;9f6!@&bro6{E0F@;C+zo z?u=Ki1)m?G4g^O054^?9HTWH5@~Bk)5yTfEuAux{N8l$>PI=yhq_a?NJT}dJn7TZ4 zEp^#88Tl_Q5~!zxoFvZq;|I!VL1;>CgBN7QaLz%_iRGO3 zm~-+7<@|&>&cR#pr_xlbW0LzDMZ1emqi@)1G3@4b$8f2uw9f`A@2}WrC+e(yp<-@u z#X22T^y>T-kNgG|yH_gqfn4MM;2Imb#!mq?zSe5Yj^!%7f~&mARo)M%GDlW%4et0c zVDjK0;PgN>QJKzZ37MpQ_a8O}@Tal&TU@E1Lk%R_XP}a^>k}{%u3>sL`r%JF`E5v6 z_lB%6%nw5M`wA=zYJIl1R+IPX9mXG$NWQo$Az0qHE1}Qu1i!q1TL&d)21q@X+3q%{ zgZw3%-H&`<5WXLA4#{?Z<3SqL-WBKV+3x+0X!$-DB%HU7aZla4;@VTw$+SRfA49l-8$_yP?`BNN-lJla}7&e@Sv4E_$kd^utW?9~d zRaRkCgN>?26;;1IVA%=y~0 z9&_kS#eT_Za12KBYaLkR4bO*E85JHhrdp0f;B5HwwH|1@5my$gF~9MjXr$krD!q-B zqJt_8#7e_)Jxi%JrIZ#_+fqHViu8ya1Ylcl{92EFNTfmY*K3LO+EG2IWF2n5G|W)& zsZO^0Ta9}C74YQ4{a-x(sMcC%aE!;5da1EU4_}8*uPFc-i2!_w0OEoGyomrlY{W0_ zz_uPJsLoXP*&cT<(8(3gUv?sv#fwRe|3T1HoEnNX`gtbM4Dc7f9g7V2 zytZt2MfVl~=R1+>zo6CceN9;XSA*&+urjI_L7a_++I_foKd5b_$w=N;D7QSL!~ct$ zZ>G&wl7GF*c8_fqUhTN1;N8(X99~Sb@Mbm%@3jcu?qtHbKw9#lV4e};}}sz1*}=ub?u@Xn5I3f@aMMZkAM^ZlzE zEBkk`UpReRbj{XZrPVig4yW&+`Vr}8Y*YQYJ3@bYj%*HIuO{I=)H593?t22^Y3Dt4 zCXDwY%y;NWBhyc6m!58cYV%6mm8@%M^wNnB+6E+;op*}s)?e^usdwUerr#cmcdbpq z?>#seew-iv_%QfCtZN>Amb%%bZU`DB#?AGTl1I}>d2I>C`;GSrcMm^8BtAM<2jZhS zc>RLmT}%#xS5ehG{q(+1($B10!{}#tW%Kpj_saU!SB2G&scep3zPPt3digOq0{t}$ z@8Krl?Y}Vs{Z%yIzeKHn58e{qzs}9`zrCYm{m(jt&+ksn@#FVLHPxTbBlM?p=jPzu z-6Xs4-B!s&fgqxXM@X9$M3 zx_cPB=|S)!kMD(h`cV4#a6$9*5gC8>EQ!C>y~FU=xw!f98>ZEdGlbRO z{cT|VNcf7*Y6@SiBjKxAc*T~c;B~n^9Nxcz;Dz8nu7Ak-=Ewh>x={T8@^JnC_xvo% z3Yt%G*N4rgm~H=;`8gpN-ZkCA;H}!){P>O=A;e`uXJET0}kBwSsT7sQ}-xcx~Lq6)rhO*AqUVK~1hC6tsFuWz(xMq~m*^;Qbr027Y zo|G|hK0qwAsJ1-CWm}@S#;j0WSw$8Xx`orH#h$htT=XqED9A1e&_7$8&o8m4rvc+9 zpv&Uj$Z;7lMYdm!SS#jL81P~)6TdK)pVX1<2ozi z*O|LQ?0A7bxDSoc?HAe@URgjG|F7cAE>!HZ6Lc1*EzjbdIN#bs#*AIJNynHTjb|&P zE=s&-(ot2p*cROay`&N-F{K56MsZC`6sD`VJJ(U7sR&A?p`X5*U8oN0r;WCIxh~iM zE`uS!m!l+rWvC-}+KRB5>OmM>7*sM`vp5|{XX4pTSP;8lDELSgR-8~wJ(#KfVf5~@ zUua>wUH0eHM%f=5o-gHHko}p+hgSZom0T5Z+MDvSfxbM@wk6^Vyh91G`7u8fiTCiK z33`?k7)qGJ)S9|SYCCZOR=*xzgU&nn1)VO=;6VFyF|4N1%L6meH!ikLba-TIN26tH z1wUyzYiHTH_FCrZDpNZF5HjQ)5+fGa23 zBe^S>fC|O7dcChLI3fSN7Kk;ycE+8a2eGN18vy?1VSdiRHftvl_^y48UNq#?szd8M z9jK9>HVEp8azt~bcn&`S@1rH?Bj}h&Ci-XBvY7)8LeCv?5W4P=gV2#4qJMidz=4q0 zx0!(sgug!(tqZru8l|YVGbMDmV;`ckwvy)EWMytUlj0TA6Oho$6tyK9vQ--vX$IyC z|F`<@boig^&y9@)pp!?>0EWeeKxo2$j$qbmFo9Tx8r5y&2Xt8^w(w0`Yfm7{+LXFX z6)#K||F}d3X^ImvRV7jTlqcFBM#m>CgxEvK_&I4&=o0XgNv$yB_jW1jIamOyF4O4T zHQv7fpoxJ_&u%d@-l&7mI!Z6InPeKg0x8`4t2~pZb26u7GYRxEB8`mq$dn-G{(cDU z4W#od$5BJUEi`2pms}>p)gt=#>_l;!appK;j9DdQnvK5%xnvV{rOcQKt@L zyyDBmwXj)Y%r!Xw=MMMe0FJrmS`QuF@F=9O;>=1=oEeE^rl7V>_Je|NnA9^_s#}cw z9pfI&xn6Ph$3wgXSw>X$Md)){eNX;@<5LcJ9)g*fl0oJqa&EpzbF&Ao6={&!F(~tC z%DfFTuZ7I|A>lQLA_r~!84Xd^5T()n3qC?M-;cbvDyqhP*#!ZEf&|(>GAJy4gtb6dC@d zPSaouMFUK?-5nEY3^-dWfXD60TN}vc_ns%CbwgckXXan@;)(MEmYh!o$r?s=iV7ABhEoie_|)fh@p6zTZxu`?Mgvkw>OjDScxWQJg&)V9KUs=*VDd0XHmaMTMOSm-JWRbs!sx^71Cni+ z+?kTKk}qLWOIcFyD>(#{PdjO)QOX>=A;3b)IxqO0~_JYhT*Z5ATid)^621A6&94;T)8DMd_V(rGDMWc zX-W|0HdxF$2?Ir0)P6Fk6wsKSLPa422lYD1jlUvpqZG$XDRm@~+3tdtd^Ql}rn>dG zq9V`UFZ4S|UYh!VkJnjm^*llLU)3D)a&EPxuj$onA?1wD^q01!u?f#p0rLcFn`Lby z{2Iwi8|t+U_)3-NIUzbkAE2$jDE#vbhZT?zrWPycql!NN(G_ET6ROhjdLzd}7kkUe zUQpD~uLI3i#YT80?}!72f~V>3*R`0gGZf$^BMZ&r?jw5qbX+6OowcLrTT7p!x8j-b zQ(<}o=0Nd6w1REqZH;*TRkU<8yzw2ryHbze5rdaMV-eVx{Dx68mjv$z@_vZM+@!hf z6qkZrN>Quko=~fxTxCHtS^~7@aH1BJVCuyA$7ucxg;nVT?8mQb|gum?KD}R zxT0DrWfy>nF?4AzdrYi)jAsGL6MXce$Mv-aX!3JOq>BZ-s1ce z@niwHpQv>$B1G)o&_I&SR{Hgd3it)#*BPBU#y_n0ijm{V8&WKXY^X{D@xcaOINruI zSA6UAz@?a__lfxNXaS*~hd$yRVSqoN%4fyZk4AIhp%qbo*R3PrPGSQ%7)kh2#EqAL?brNb&|JC6XgHWWk? z4VSkV>dNdNr{tW$CXLdXREonL$3%p>sWf0ZUp&gBuz`{<4tFK@xA2gli#2q;i#ex4 z(ZfvJs^EY7#nt24f!os(H7_|EE23Hj5K*0Y(T+rvrPdo3+>9dLO!S`kyPuMHFmus} z{z(!c&IhQ95tt=SK1ZlQcwvYr?kG?9<3vqWgtlwx_xL|!F zit&bm@6a@t#>SP|vJDU^UT4n>uUJ+pM`j_d2eV_994d+Cpou_+(qHoJhodQ%!DQ+$ zC`*tGGxclnojzC48=7C?z*dLNzpZ)P!&~sHOa2$$>ux~<6F+r8C~|*5rBb6xZU{8(9j2jcBLo)R6gK(*~~?bIdJg0Bz-hfVTJv zWQ;9l2NrQ*n#75suTtp&@ips`j}0r%k+F(%u|g5+LAs&msa2ojeykI;%2Nb+d9&LLur1G<-L|jopNeaCJV3h!YVkCm z!JxVUsKU4z&AL9MNQ6>qs=6=vH007!|!kfso% zH2}#yiB{v;*rQ`9&<5O*i*pZB8@fZyUsHl=K1($bG48&kn*POPSk*Tl&0)38rHS#sBX?HuUUt86XjiQJ+&LYu3uM8uQ5{% z`1;Yh&*(LJ#$EcYYFYjEyhBmyo{SwB5yBX2y)#zvZm^d|Wu~8ad8NAzPeV_?pe|3P zF7LoDr*nZHg9|)K1e3l5Xq zv^I?>Z?PcWvXmS4tD^2RskLIijf;Lqe2I_&GLywui9X00+rQ1I^BS2v7QuHV5?GUB zU+bj=XvMk=BJOH zi?ond+T@j??(Y)&`*ed<3ZF&qC_c}P$KsF7BIYvVW@v7^TY|96dN`Dm-N)solz>^( zl_fk?ig-$4WDqYOXUu;UO4>`&6+?WzMk+$dnNQA2)lo<*SaBhfDSD$he!<=$>0M?K zMpVp1!rch8zk*s1o5UNS9huJD1S7>q8p~dwd3}kQjG>=+2mGudz2VS%QTrD^xNqoL zK~JQ=>mcifzCnFukpivZ#cO4axOb9dNmTg28X`+1{dKr7is77qM7)m9Hc~mEpHESl zyoTkfUsjXCa^+qAS>kxP1umUPqiMl3TH!zdP;e>X%2`~*!Udw+l2ZzkyC3PC_X%4| zosRJaoj~Wxk2bl@bPAe<0jd)f*HS79do5QquKN~)sgzo_M$fB=(OW~>9m5-?$N>?n z_RmqKgUt|kRV4kbj=>uWFw&5yDXsHn1S74!>!*ai#w4X4=%_t5kn_MUT~_5;1q>wN!nDza-WZzrR3b{^q12`aodf(4;yqgLnVYKzPD$ zl6k#WWE~W7-%APcqk0EX^kkxFbn6f$>l%Hv(cE6#bq^`WhxVq~b2a_MKZdv)iRGj2 z!KK)*PnGeIQr`f?i=Ri4{4)mLUxjy)@Hi<8RgW_1-<)KEq**fQLrxkDNmFH#%1Nod zcuF!z)1J9?UM*fzp6}5(%lEj-|JdiHx=?>FKYur+)H&*A_D8`0MA$y4wQ^KjZ&c5r zzU*bJ8qM+kB1?m8l*U zmD3P~a_}3k98#EHMLk%-%k$G%Vx(2zlIZe1Nia@hz{xUG zU(m_&+jmEZpQnigyFh$NsS}1bSa7*Xv%)#W$9d?qJ*>eOb0KZAUqtEBBUD6*!d;?c zO+*nTD4%3xwS&~AI`c{6Y>`(>}~;Vj2d+gKBZczBUcu;pp)De+RdUgv&O zD*So`YImQZKFz>BEiNGsXV{kXGuGLt?^%~ZCqagS*K`4GIwppfL7P6Ow5%ix$SW9ytQCkVgL(b) z9rzL#GV)Fhg+q{NU5hbSYYRYA%c&VV`um%ab4~&g9~R*kPm2MvC64J743SLX7RC^_ zJAFf-DV%dLH}a4)hkF+9r-adB&-;}p^|HTTz%hs;4vla1k8$aSDLIJqz{O~T3cwtkLO*Z2l=K2 zTdQ8XP8wYuAo!aw$aO2l)%8eyFlFJ^`;d1h5*A*Fr{WHpXqQf8)D2=oKe7hIA^PqF zz|1*^0;J_D+^QV44n=D4-bzM<__*sxqA;1BH*G`PgULFg4(fG8NH2fVf#VY$r1Bw6|IPx)DSX;LUFKZcFI zljD2Y5i-8IzZ~D^E(SF+KA@4c-6A&f zp@@x)VBZpGFFeuFguQUvD;zZscY~AswjfW0efvCq``51{*$YRuzmDyy`G3@Vy&?A3 z(~mXpe_3^%=Kt~Eu==+jYtA3^wd zrV|DxamF7?%Lp%&9XIeoiCTf1Xel)~u0gQpbUv{7Wgq17!*ZRc5ICx1#bsQW ze@7|umip5X)_SA&XI$*j%E`ajJ6EmH+3g_M{%z9`>Y#V3BO*uL?UBC16LEv0 zL;Y9Y`7`W1F__=m<~G6aU-$F-e@^i8`_~gg`8`ezaq5fa{6i8^rEiideYpf)n8P3M zPd4}%Nj5533jvJUKa;dW#G8lZy4Puff8EoCwFUJL;_)x!x)*psfzO`QezG6%Na7in zL-(cdD_DL(RcHL_tC2b63f!bBt}%PfD*RU_ZrelK)cc^t0;Ao^ZE7v*WL_Q7=ndGY zj=wr|qgstEEd87|s(Zx)pOBrH5cj}?Kiu(O#s3Q#zvA4c)@!eC{0@!xkKdNgp*TZ> zL2>KnLF4zA@jTXa($?EEVe858L(Q*eFKP83{5h(A&W`*TNx{;X;iUR0Ct z?mik0Z_;6X$a;-4Zk#sVTG8xs*%m6OLD?_%d`zYs1qNE}au-5Y>6!$cE=go<rWMGA8M`V)LGv6~Z3K4<8`?jnmB)+?iQMDA_ z?73TX82u`@NQ|l=CS>dCc>&*3>QZ)yE1McPRAqn8wBB!VO)Rpwo?poHM(88=iBsat z7DD3P=-KA9m5M@}<^yQGRo;Sd%K}!bb;D)6b`yW^d*GclqQgH>nM*jm3laXYjl7m> z#KonoU)QT79HH{7rw>1vA>Lr$qbW9i-#Z_DkIvYRu>Fh@wth%|fc%rVT#$CPo?UhF zm>OFB8_gK&G@n1{f1aOkPkhD5i(%33s{BC0{qKJ`)2-90(fx@`yG63*7RuyX;e0dY zZJGIRAIZ}#a%<3DSxlFieDuE-Gsv)5{PZ=UJXhAhh|f&qnN-92gv`K*@%)`WoG}8x zOyCB{es$oC+wj~ZFR)K>)FD%B3QBDRlk#{UB_}H8CJwJ(Aqo|>7eyICYcPpn`R-MBApUr=JAjZKA_j}Ld|yR zQ1{2zx%@3=FP=GxAMqpUo2G8UJs3r%?sV5QKCN5;P}s+240fkzzaB?)9~Zho7J6nL z3#G5Tie7n0eZ2zxPa)^2*CqY!*Hg)HuS)vUEbm@X)00DO+^OBD>`{zcS?xYc!Y1j;rT$CygDAI<9Xm0 z)$jQj$(^{$r7F-LMKXkmcA6GwKU68I zcI#mmJ#&zfQzKWN+nIOW%w|MlITyHA@zT;=1+=52mrwnMsG!Ry4PF^(JuR|*P)Pr( zFaD4Hqgl6kMR5N{yz;;F56Rotzu2!s`!{xFQ2&k>1@%wl2lnrdkDKeCCeJ1Em79w5 zb|8VhNCNHFHP8&XU8mF7>UAh4i5~0Foun1rr-xxEJ|*AFZoJP)d^p1SC1EECZ}d#U z*%9@QG*)^*Dk}W9cuG^)pc_8ccg9LXa9^bkOIS}vL1l~SEs?;)q9Mh|JQNW|B%+z` zFyAv?>=lFea^jtN$bS5#bp1#fe$qZOR%a+!#WHQFFkfVTOLvb^*_pAZ zJ}^o??DD0jiRbfZbZu8_32pObro5>{))$$w;5ELK!+Bas?m4hpBA6e4KvZwUZRmP@ z>Lnu?5Cgx79G z@Mj(?X`v4AE6`yXmahz*nD7Wu^^j0TH2-_1!XiBKl5FVw+hoFK?TB701|&hc8B*=$ z-)<*`ZMOry@4PEk#vJbjobf;_5WBC^1;iXrM|wK_exsMJ1tCb-UihK~@FtZ|*C48l z=&cJ&nCOG1`or)-Mc4$ukaT|=8x*53Z@oVjgYOQZ!j$|*q$cMkseBOsRyte5W z>*GbE{X&NASw-E6mlLeh0WphLKcJDF33THjYx&$N+$YXGPb<%3^zO0&mmMR-j~`N5 z3sAuspaOvbK5+D3SZ?( zT-lPPKR-3W;#`S<%=(`U_A4y8%#UJQo}ej8hM<;r8@+Dc;>SN8#VS@=DxXTB9$oZ& zm&`5R?HZs&N3(19p(v`^D;Pdcx{+A(<5Tn=Z4)UH zQ@yxvfzMGz3}4_M^!E`R&geD)ZE-vl#m*i18%u({`|yl?2VQSmf6m-{7d{Gl{25i~ z`s34jLd%|Q8Opp+02jAG%b+!GHb--@Tg=*VbRWqZsN$h#owW~+Fv7X9WdtPPq9=|o z1|;_&IOHUj3Q%bo9bb-c+qKdQ3=l8=3> z9y%tA;Dw}AiW+FqAA?f2Ljzl2L(TS0jkYgkL%aIn`YOE^)!D4yGHiZSPu_cEVStRI z4LH$=70}@dQ3TX-+LFYMh3se0i$m;y;Nn1+C3P?M3!~*Y$nUrJCX1^$NqSbfiupFT z{bJFZT>FHcje{l9*#SySm$_SHuB$kKjbd@;ipkY6)lbDyKR4ew&dTW{-k2=Guf=&o%enC7l zETbMtATS-h-)LOZf3j7<+)!w4bk-oe76bsXQiYes!kALn_$)`YbtLiZmxI~wk zXkjnjhVf^AZe6emQRMhP+IXPtRF?V^8-l8fU#96_%sd{Apug?x6GXfZplzP_{zQ3; z&GzASUbU3GYg}U%@@orv(a*m9=sztwE+C1( z;jbYB+sqFh4&m$;XyX?q|5ed=Y{^nL`L2$FRW^YG4U9go1jJ zKv`SyIfVj6J_#*yk5&W+&(i~ntOzJV+TT9^<~fByBueOt54w7&YfCCP2=*`1_wK9( z&9N?cN0;O1W+F-9xV3qeZQ1X|Yhvp$6j`33;Gf*5e_o0X7xUi0*)q`uVwe#_3I!Ko zmsY#xWi%l;mqi#{u|%?qyo^AoUE}ckbDSDLve?|K0%S4tKV}Vfl`iBEUvniHK-2A4 zI?6sHRu8V@(*bLtlH-thL9Qowt*YoSgjYchp6DPFQ4_TqIWD?;RbK{KiC%X3qveCH z@IaHbi$PsuPJ(tLZX6qN^xEjq^JJKc6DUI1Isd7`PprGY!*}*Zso!yGvhQe(800@W zwH>C4%rALc(GQw(=|bS7V#@mhk-Xm7C=IkH_*4!sn~RXh&wU})7+!JV3*5;?B}=6? z-kDaYsQa=o2-YF-{uSC3_635e0p1L`*$ZWar@3G77Vxew8~GTnUYxYU4$zr$z%ws$ z{1k<}nSQdU4js$EvFtg5E{|&Qb)+-6qBye?fDC4~4+u}>K(xwTSugETlFf|UXv9GX zH5BwD&DX({0~YmD98-|o9Hqt|aNC<{7+DKdiqN|fH#=hP>pXYpN+?*%Wv2M<%SK9n zg}+h6q7R=3EyjJ4w2GeDv1KlfL|4oc&j?{&jeR#dsDLFbRqgOosNJuNgg6j4Q8IO zgn}Iu6DMfN)qDk!8;IXz!O0vIfmuc|F$!WdwEtWvzkWA;mWgA~>ykK(hQ$@`epqbv z)gksoLHcV8uw~5<1%nK^Y$Z}ks}u`Ztv-|kGl7&vD4E5nePwEusNz-RT1*gaR|AOR*6S-6!~c|2wbXZ7fELyF6X-u@wsq$ z1Dlo|bB@`?ZZKx_kQ^mev6z8E*{AU7^q^0(o&Z-(kDc!q^KQc{e7(9ue?}YL3cB%^ z^I`Qp)>@bCUVre4H-fyTxH^rY{2?tGBIgfIrTj-bHj$s&iSn;+7crl7^}efwU7f^l zBu$?Y+w_R|nG;BLmIJ&syB4vo{EOLU!g9@uiI{q=nEWpuj)7l+6Z!LqES{%TM*UTkEvKt9 zv*oIT+P_+{Wj+;BNdN2DF>?rPh0oA6$*y=E_fw>Jh3VHT)bdB9CXjrK^cR$Lld!%B z`1|>2Yt&$QOtg4iJ)dGNEk5h-B1c5TYqLb0Lw2n&*Wa}??3ydp1#mFlXIla=`fMnU z@H%kMS}_S~dMZNib4{eW0W^v?^(U|GPH$4hNHS^0ay zbnZfniqXy|7Lv8}Mjda&zwbaH^4xrY8UNAmp`&X>q;%!@G5GljTjtorxM%?oev(`DjO14tr zrdC)JEKXCY*_Gi{277D&jGjIXde#4AXs^D-UVY60%Q}VS;;oeKw>h`o6Po)4=GrJX z7o;5*wpUZz98=W&o~tOsX)6Sdw?Y;GrU@u;VBbe6E>qqpfwsw-c-;32?of7tJT^{o zX01>h)wYpn8jxFv#Q1=S#^^n;-r5Q?(F(`^U2ZWGTfZO|^$cyXyT<~%NLx7K<6im{eD8T$7W$n>xjWjb95+&CIFR+o?P zN<^2eTW7Ia)S;1R}HaW*^MSF zzV({yL~BIU3?llG(#BS*6XtyEQxO!2?jJTtnamY`vpZ12W^!(i!u^M{8`&zWzZFC# z)R1~bc+u8a^u^hc_5mo*efL3w4A};s``Y4~SZZ-SUl?Y8T1%|ddOf?F7`?lZG$}s$ z2fQcQ;eL^oE3-xD+)JeM|Cdl{$mhesFEnlD zWWm$#Yk`k|{*Q;F;SX%YGcOUws)T-;mo>&Ep{rx+#<66ena4}We5v-H?l%MEJ)GED z$T@>J2exa)sCHtb(ywLa^}b9^KlHB5iPme@!^~T$moLdoskko;5$Gs@feO48T9T*W2?E%`Go!xNI<0?()hNbg#5Z^w)s}+XuDblN^ zU1sOs&CUlvGQc_S70VVi_>6fjVT7!me~=pd77MU`x#_|NwjIZCXoJ5X8_AojFbCqy zvcTp)FL#&DY9@bfDC!vk_*{A@!l}q?9N>1=MlWt+Qym>ileF#&ws?MZ99}$xyoV#> z^lH{0-mmXyL?Y>HzvQ*vs^o0I5Tk=y>TEB-;CG{l5jdBDTvkR!lL6!yPb-{wJ8YIv zw>`3uK>dGKttq8cS|Yzpjx5$Lo{S6=1MQY3-qB*;Q@o9gq~ZvZ9|c~Xq%NLM z)87rwYaUJd>G7YA)jst5@(Qh#A zbd(aSHkkH04jATlWF2^B+=U)E$J&^B860SK54+X*6sK2pp~luW0tN8l7Io!Q^!)}4 z-c(e~dYj!@>PFFZEUr>iI7_XysAn^sl#L>dlWO@k@)nNVoT;uIO{P8^2-kr|KNHAK z2Bj164@5fyG^-mc9Up22XR>8{R6v)pd_yF+O>q`8cD4z_9@4+wAXZJqQ%o&L=ZB3l zx?6>apM@GM+iVJk-s?h~?>IKyT@_2)(S(D6i&O#8;#61-#Vaq;iR`)hIR9C~BCJC) z4~|oiUDVMCZFgj%fJ`y#DWYIP1KF&_PGjkz6SaU+QSfQdgp^(!dAQ&l1(R8e8S8Kl zAhy{#W4}KG1(+*Z#nUgzy*lSlp`A?enai4Jv`yx}&rxbxXOUP#`6LyrpTqet1=UZu zqUj12$oda)egfx@YS6Z=lV$!LoIi@&cgEMTkutvv<~tYDVFk=X;P_58T*WX8H$yMA?Ft7#N zCvuUZE836@l9J}Nl%MJxK|@0MG$ehccI~o0uzrniNUqG3Zi<{=!$VT!Yr-!w|9k1V z53rdC?Hp{p;A5^>#DEjD!#wectoRD&Coq1`_!@4P`H%DI{<^!FmV6DL_A=)W4XTgm z%2dzd-jR^*n84OjI0Vnc8b^z^zxa;-3R(qKt!dys=ZRSP>Ek+{1~>%EZmjdF8=n;9dMWx8WV_i)XKjs~gm@s8CUHKGL~=@#p8+0I;3r6E zK3SNS+(1DbA69aTDYs=v6$=*nh6~-U9R`9)X!1WP<(gv{^UGBUyTk@iGEmLR2J)+K8Jep}>0vuvl#j8wFc zdH>~sMGY;nC^2gv^QVo{O8Vksn(}C|BtR^?I+i8?S;#nP1;)^lY+QOjDw6)Q-QVt6 z64Y0Zp%$sa!c9QA@Z=Pq*+lmjeafU!{@D#Yw~%=>ncN&=eNyyFpFRnAojz#!@(q1I ztK@Y&C8tPs>Ra9ttbs1sYcT^wFt>rN6*dMWLtA_lw{Mz%Gw+m5UE7aKVN*rb3@%^~ z`tB$nfF%UL`Xr9T6~B6&kpKD&>8_e1jn1jM=t11iI)JyX%OCS<2c2DDqovq4#A@q4 zwEyYnuq?aDtb9z03W!Qpr|o}?sRzOSihDa}^rNT!GD*R6^^jq^5EWT}HqN4MCgZ%H z$QaC4rKq_zAg$m5HLL=;&jeV2kVT?2f)t-j<(xbn2R7@1cO9WQqeX3mk;MSFXXt(; z>tnc0jI!u7FW93N00O&Kc^?+;A%w|-w!10Sn&4a^zTwKr(paZqu~u&KVD2lR4ms=th97nX-AJF`uUyj^g2`8?<29i#O59Uh2+hiijNKC9wtf zng5!ITQU)@yIg4Zwy^b~4SWVUo?UNGi?YG)E+ZD-+Vl2)W~`FaPZ-C0z1ej95lMd) zbe>8sH`3UbF!r;P8vYo~H=R_)mO7=U5j;O31EfvEJL3pZnEDT=Us*d;mKYB`9kC-^ z(>6~pZ&$oELS`N=FojP@li#r8&D4#R23o8nNeX08_TaaVA5Sx7_k!DPiz!nSXCk zelM@Kj!%>MDSF@8oZm!ftjz4lbG0|OjDVBf>Fxq4m2=Bhh@b7!b^Izy$zDPJe|B;o zr&I86?R4huvXy|44U=-&`!YOL3$n+nusEMDu>d4oIA_{@Q9E++QNP!mz6m5AnjP_T zYr=>Qn(h}aWv?v}gnRxLFj{rTcLZa4<#j_(lx z^7{z+T?u)?bP$YBu=4i8w(!DeRO#N-fM1SEwy6DB8@!q;R7R+<3M(w){N>w1&oq!L zyQgEWcDmy`vdCQ`9)CsZMxP**XDzX?&|`6}yx=oA#48=Qp`l*Ed>*map|EDiEDaUJ zQ!;M~<}uU9Jhc557MV9%&$iDe^QF7~FulC9%V)um+V;*ko%sHhfb(#S#$s@c!=l}| zC>xZY66fayo@`LESg66iNzaA}b7cZ~k6^+Jm>@0fV2wQZI0G3hvq$DjeIeb=ZQCgB zBI2S7sURg5xDzhAsc!q5cbD^)U>cIQ*zQxtldEoPx0V`S0q1Al+Rk(pr44 zpF)+mk6+?amZVr=lr(*%CSBUrGVqULr=Fj%> zvA^%03p7vg`@Ya;U!%&_mlbw*3G!iif%7Jy(HC9{BGxV@3$l@9{l~r+Xpz~I`ejYS z7cBM;7we@EhdgCrM;38?9Vu7aBek}CJiu_2?Ri6%t%&g(SJ2dP@tGUl=x#)7QvV$) zR~?P>pU3YJ*L7sW)|M^MlB^PET{EE1mE|+ekN!ImGzT+@bhK;_WA&Mk-WT^7#@@Pf6$gQTXSmS zTmANn`=0SrsFP7Cr8@Bf=TP{6ySIm}8xF(A9e5bH%tl#;U8%IrXZRX-bRtpMOk=(m zHC%=_&@&8PKEaoQ5BOy{y}ZLOxVRY{zo1XxI&8n2rPcW4>Y&pKzWncB*ue=8lZJTv zA-`HVQFu;7>G&$v)&6vwV2WK4Y=MrQv)%xI`x+cMdhLkAbCq!&`Mb0+lG@kPlI}WS z0}?#Cr_pOUkr99~%|^JQrsE9*YtSBbK$&6H=(0EvHZXXH@H`cyEJGmTyYW8a9L;!V z*t8wy!CNL5iveC4KfOKa*Ia80r2&|DO9Jk|)+6Ha@%-LG)@adQUOshN)F;&O?G+q} z=d{?JfO%yWbx?AsOV*eKvT$)AL5Oy1cx%rlmRSoeE>m904x{wrl@3{_L}kl5KX33n z#^}GJHei#S3$1of;FI$ED}x4fE)6KUY`bsf0=ESh;CtPW(A;s2e9x}W)@AU0Wb3l^ zV8FN9DzeeWwVezVJ05PDwJ_a;u7OxT(SC5~+Ywg=RuwMR{Na5`C zlv2eNm0|RbX7!Q-`HefBHzZGP3Pe;MA?gt3?_jHl#UXH|zTA3U3EMOU!hH|x9pWX^ z7Eq7cx{T{3vYTN*J=;Y0O0M%@Xq_osr<3Nk=C70Df#o$dBecR0t}ua&R01nlv*_{# zIRsjmxGAhrZZ9)L!VIRM3_#318^bB+T!1uxYfwJV5wxgrzeo8LIZpVvMGSvwxy-=L zn7EHd*ZQ`?H>!x+wy~kh*DO@z5swoea?QRz)4XdXQSj{$*%ZKRd0TSjOfNS$zLB-X zu<25E=8@*w-0zvJeCxx;Eo7t11otRsbRYmNGyr+b=LeOCV+5Tr+{@V@@VfGFCk>Q9 z-19j9%7*az7;M=6Bajoi#JS*MRZ1J zyprU=*wsP6ULjy8 zyt-c@OG*cA!&L_hhD(J!tea=0i@!%no2L2vI*tozcWyn-sT1UDj@4%kS<+e#zwKds zeXlhw9*WDf5G||Nnoe;pCh$%d?ZBMW1hMrSa_M*i*A9v=MrM@9@Z1Tr7c~6IUfzq# zJwwqCr!25{0epWM-@1BDS$OZ!1k7DWD`@Sm%81?QOEQT*j`zouw-cm@zlrnRfNK{G;SLj`5>0YY7 zOF~fDi6r%1j6I<)AhWeHcapl`OMIXAExn5*bsiq3JBKHz^ET2$M|Iv7dPr2~?V^Y4 z)Oo+?qy_&x+%Ut>1pXPtKNhq;-!#N*|E<()UyDF>)(K`H z7QNi+p#*8RH)c$|)}o$M>_A>db{pG1WIJw=psu_Ps!mQpK zv(nFJ8HOFhU=Uq`K73f@!G~pKi@w&Z(o(^X#b~CiNv3VuVzd$gxKdh+R z&1zYe{-UM*9~QMnsXyY&)|v5T2P}M9`YT!O&sx;Ilv0j9#`*&Q3`$${do7Sp&<6sPsM^I!Bd>1WYMvW`AF^QfA?&KUCkxF70oXC$t=*t*E<4gXZl|n?BLF4Jb3j)UK{Uc0e+EhVP)`W0)RkuS%vGEU}6hV!SeD#{NC=w}3UH9qaerw0-y8KxY> zr$>|EsT7~?>IYBN_%t#@`^p2Ov`;)b0iPO4NB@@>7jm2hdrS)&Z69m&r?jO2|?Pndc01h9|(EcJOa(BK#R2JIU}|75v)| zc!)OlMm$X3KFM&P22qhj9me-*JGy9z{U~wT_5qNy6CTdCg`7^^sH$f>mdN00o3T_o z$gILXjpAxMdqJtLIx0Q)qY3&ZBa*PaW&TNYTmfL*mQ2Is1^wj1@(lU#9fO{{d;;;v zOBse&Hnknuv38a;T_{a+@K0y}m7ooUo3MGcr7{w(Wa`XhY z2uOX}&qx+0I>M(LGP1E7=V46e z!V{DKdHVB*kI>ZD^p>y!BSWYlg4Ht%T68%+- zVJ%hSd@o{#2}WiREg5!|$?s4-$CSVl3%LZ0&#nprfzwXTo@sv~Mt7~G*#!@X7*naQ zs?Jw+E~Wm7sVmPg^z-=gDIw0V>o5X4N5GaAF_G;;qC`OO$Rlqg1i~xRFFp{vFd263 z#LRliTq{W!un2Egus<|UwMHc{s9hUyloBMn(FK-Bk|ip!gq*U-q5vE-EhDuaUMJ!_ z*oA|RV*&}q8H{KLO=At9oZx^z_yhFHkg3 zcajAV^#@Qeo_vXTbQgTTfOI_oXIM2d(zaN>E0xE!-b1Vj32P6o{+A(l%tCEW5va41 z@Sl-bBNW|@88sTI;>_LkGft91nxa$**_UL_o2_N93~=yTWdYR8SIcK@6t*P5VaP2M<3w)n^- z;^G=6pvLxII3OQhABWF!QjMG}qD{Tb11*Zc*q9A)7NHG(iXeIK^Zd^$JV0K7LEz;ikDosiN zNp=@5S&Jok?I&r02Wl@7F08om!S(~wmqx=%iPGs==HIhS0bV=l&--ZAy{miaRTLv0& z_TgF}%A5!qj0yaw9{EZ?_;XJZZZPmI4fhxg9_b21g-ADl2qlRx8T15tPdTk|_)Jzt}3ISwNpam&%;3##rpgp(iFQ^f4# zAGJ*f$dS9U5c9jXAh*J8M<fzeq0&GC-)Mp$hi6r+qOKx{382se(+FR`1ke?)Rbh8MSX~8IK}0Aig|;AK zg$fmvON4I>N9AY>w9Wq=^Ucu~-TnXev(n7>&GBaD&6_uG-n_R9Nd@Vkq7&1>3|!?0 zfQk?7!yh%0dy7wUmt?{rW~6E1*XK#5(@s8l+Q}zRJNe{^lTRL>d=_lQvr-d(`;q<} zCE2wstSt#^Cl|7MX)|f?Bsu2|Q^75AO#9p7(~t)3$=63ww<(OaCJkOg=@4j+pvy2B*hu8 zYe$kF#M#2azajtTB)!OAzmr8;D3s$vnmf|P@guiuMj3}2oGqxjD`{O3F@(S^(Oq$gOKPqamA_w_gU+rmZsjSg;c*LD=z z0yAj_Dv+6zpm~Hz96sa5{43Ksuii@JM!0&roJywEiz7s=sf-t{nM8;R2e1NIFR(e_ z!i_7(%w`q_e-jLXc!+!MV z%OSKB|k}jofN9xrKr*Ik2K&@7Np)80gJK%^Rs$ZDmt8EPpGsg zud`})w^VdAMd2zE&RUc&u@+m=@f3wBvL);?vy#msuXuy+3_2AZZ<|oE;5dGjzg)!{ zIpcz(lzyEoG>U$UDLT^9lJGq%-{hrbRO{&^i~J>87<&!tKxh zCRj4j%b99iJq@^4bBKyJhZsQvEjZfWOG%)KP+9I$@OdK?d8GxkTX{as@1Yc1T;x|&w~m%Qo)OF%(v#%WCb z##=p{mszl@K+%$Ay)EMGf`asl!>n?|qcMtwGmeNQTv5ebL}LJZaIxZHX*!M($%`Zl z&nV>uO;S8`iTHsT*#rEl;U2=TOIQfk;`yDSReuWW_?7w(w2GRR3d~l4J*}Yg3No+? zV2tNk7m7T91yNoCxZd{hcE?*LErQfEnUj7Jz{<0zJ36>+WA38+tD@| zh}>k6LtfCfUpxa}=4FW(nhxyb{U2MH`NKGFcINSM#Ar=3{rCgqgqh!rW8rLO={P>% z)84kbjSc$KW z=~7>5E6J1dDfaOq*u#@dFDPV8iq8!&aLrZp;9f3sbXOhvTFLb9qU-E**#ZbB+vxQa zt|VJRkwgEx#J8}>q$FiIxqbEyje)N-!(Zpp*LLvLn$MpmFS2{Di@D8fSk6}u0P&LQfQ1Dd(@`m0}y^J8HwInniXlV|wpkJKI;9)MW4ejYI#FbTu0U$3D7J zx;kf#O@Cf;@58&46YU(`cS?BCvc;}f?QJfhvp~F8`AVBYyZ1AEO$;*$HE-zvoqNTm zKO`v*ko}61$akFER|S(`a>(CG@;9c1u7)5-+9)x^>I+SDdsyOZDS79|>eZXJ8eq<^ zvMaZT=f`gP_jWNqwjzHrOv(Br?Pd<9)PEm661#+#JC5#W?E0fPd_||B$ZvLiwL}+F z1JK>_IIm%t!)tF5XP5C7<~m9LJwDVfI&+;w&f|1w;gC=TeZ-5YqTfc^f;;Rmj@`FN zMYSVs-F^1%XKdXsIueQ`_|*M72^K;3DqF%9NpZcQyhztPukmdV>|aZMqF+Z+lyw!f zlkJj-viN@ddnuvFuHViJ7BH!W4gw*bMXV+L7G90lK{t}^`n|M_;lttUq=YgU+hvlh zhwiqcwFqq-PnHsZVxBQJb;KQUe?M_N$rP$@AWwi9XsnRpaLc9YrY-nbC&NO-nwEw6 zR&@_*o=aMjCy`R4I;k571oQ;#qk&GfISm-t6MsBTUnhbxFyUz;~CS$&`dKRGYyf+Ic8 zsJ_#mtznk;$05HQScvs#E*ZUv=%&!_je9JH2L){cKE`oz=d9PcI*|3{3H3Rs%<8E& zdBy@!2UfuZmAp$6ltX_Kcfg+w^Kgu=@)@q=j|*)%YC_S{_Vx=3OS1oBFoWucm|cuy z8yv}2>ghMGHz99S@a!=>UPM!^ejlre%&7en?Aj#|;(ozfs@FKNi!s z6X{#I8E0p#&6{gfQ##|Y%}ZAM#?xEGyLpll*mSV1o+s%yUINbBr1VPHBS8POUVA~= zKq8g)$5H9_G+`(E(uC2PINVoT6I_Ts2Cl{0Gp4>nQ*h~Rnu2MN+JK^}x~P!Y#V7_J zVDMxF7a{nG!sr>&3HzaBwI^U90%JAyH-r2la=k-3sZrX9JVT zuo=K7OuK_YSMQRzvI{MBdhFWIXttQv1M&R?dh5>k4?AKQ9hz>h`Ge!D7?80~Te{3ZqqDZZ)m* zjZyAjXtc=puP~00CoP1pd*JJyQKoft~;nd@t`hrt)Ikisok5lVK|2VZl^p8^;MgKTe zDEh}KpXeW_wut_5YOClUr%FWsVklK6`p2mqqJNy)Df-8$U7~-SDi{6Z)Nau~PF0Bh zacZyVAEzos|2VZ@^p8_NiT-iwpy(f`4vYSA>Zs@+r;dsKaq76}Uo53m(LYZ8Ci=&z zlcIl|IxYIgsVdPwPMsC~!2GKuG>8|4baVmyW zDyIybs*NSp2CQoQ=fJdOW0XltVfp@OTGy(jMPIHjgC%Va7Ms)0SxuHo#lFxuO15JT zSlh6SZ3`~qfx}g%32b3kZr+Tt%9c9y;|{=0V%@btry=MAq0F1{8GIJr-F5-68UIq~ zzpDXqF2oA{(gBZ8@u>%}j@GadbV)rBEHlgTrdLrI%Emk|c0dz++6&av>myA-!|^AR zaCaM2U&BVp@9M-GPS;`a((#bG@(wotwz+eSI+KU?i&o#BgY9q(y-psBPSo+;;E9%$ z@n*=F1Q}n)jKA!pjASj|Rm)h&8QJ^#@4_2d8tgb#uf^%mIxzg^4*yl3?#B;B_X3($ zeBp)8xRTf#o06NPo<4St>Z63Y{9MpotDo~D)VDICJ~lx{>)ODlFhK9UMq6Dj{Yk{s z28=PU=6-1q`G8e1= zXJ7WJeHC;i(trGwk12gO^1iPhs!5iM*fB;~^ajUz6}_3VnT)zoz2g zr4JO|LO~G944|Lr7nX-nq<2c|Cn1VDMA4i0$a1ne7Vzf??m_tBKEfd>M;48?*VN+M zR*(d={Q;NCSHqYx zCg_Gtozka)T@YkI2@35S5#zPF^~2+NS#&iB5qPe;^lkYW`BgmHrPpqj@=&QPZdw5NiYfI&? zEvS8;SX1v}#--cCYwANS<6k)A>yR-SGP*G1G|tGOfuAS4cg~#Af^Gb#p}N7Hn}R;W zus|9YNTC9@hytxdfiy1A4+`Lubk)EG`cZ-8hyn-VxqyKS41oeiaV++fQGp>;;6y{X zz(=CMo(3u~OvfRJ)PHb+VN_sUM1ki-fq!s;Oem0#1)kvonN+|NQ9u#}o)O>FITl{7 zxE5x=(s=4n#pNe=Q80D?PN~JRWQB!ZPhV-wS5Qp$wM>JXwai^!g&cplL`8o2Hh&4|72(ivab0A z`6w>zyaOQp5b`iVmJ!7HEPNfpo;dOA^ZYdnz77ks#Ygz-SooS5W{czaYbp*@k3Nmr z;*-moWsCmq#1{`hduX#RL-LlgeEx>sZVxPED4 z{)mob)SJ2s{@A70f5RU`{wqDZ=Z+Tmg@d3cn3*bzYC=?*f+y>%(7El1>Ka&d-6RZ( zo@B6zT0BNvuad3=S5%{iZHG|%LDvqL~wDcYT!s7jbAQA@f@Fe=g2V6nc!_! zrunWb^PHQ{*Khz3RlWb3U@(A`+11;0xzOJIR#!or3%b{f7$HL%$rDZb+aX2k?}(J9 z{q1+Tnf|)FH{ajkw~GGOH0tlew}$&Wb#b#i_D~n@@7Qyp{yx<~^mp*$rv0V0_)!

F5+B0{AO5UZaJ^YqZdK;X_WLZ~VMH>%6e!pg+}- zJ)44+BC@Ome!qRl9d%Rsn>e0v8z08)$^QwrYa(z<{S(|GD-Poq*S?XGI!ZqRi#JYE zTA<6~=J*?&zaYmz@~3?>Zw5ujWAR4<>Cj&K*r4I7^@3Z`^g-zDTS~uqc5BvL8LGG? z{7-|oMR+@c%k5xiF9-1M$UkEL$Gj;K{=M^zYX5e@2nXNSwu^G`jrkvqa#CIgJ*8t4 zYqBHVbpyUO#oh~09Qq@6uO&z@=HNbZn-nZYcV~ym`H8yjYI;Mj?Fs%5AYB~tKRzK+ z2^tj5bw?M{2bS5CJD~azKVrUkb8xQ>pj`E@Htfv(SFu!``JS!<#rg(E%yGGOfQn;P;@jF)%NXg^Ul9$yv;lGqBqv0yiQ_rvz|!>%NhvWkuyQhfk_wte71yLYNy z@(%qL$&WtHvP<&sQjtH_o_@rY$@dXJ?N<{O!un-zD#;pp;(a?1Ku=nthV~lGf*Tl6M4v zoxdB^f{(b*!7^x&yftWU&ZFw@jSLj(hr*rkEUDxgK@>>u=Inr|x=W2xdiks~;Z-kr zZ-cgrTrlKCxb2;gYRd5jOCVs9ymO4I@gsZ!*jfs1x5=#oTO0X5A-Q&P8T$Lf$z83d z;1KB8Ss4p^Eb#4OqUqhHg1-{%JF`0njyDBAA{i)?$tOK9%QEsFsajVV_p7DLonp~@&Wp5d zpna>D5r=#pYqSM--KJO$N7o_mrz#vK>v@xBvyQ58?eecBT&;}RP8~baPu(v3_ouhp zWh%h;>r$`?MswEbz)YJv2mOBhoy!?Rz84rxyIe}mIGxx%l1R}VupQ)bT+hR^bk7b( zl(?v&Qlc=@$3Y&1VzBl=eBCbvzp*PWAL5r5xCw?oa5Kj*$IFN7aHjeKmGCEgu7woc z-YyRbB&8m}EU`AZD?{1iVe%jPJ12fiI{hXW0Fm^UCg>YzX8^7xJ1}3l9H{bBrRsAZ7<6WQT0J_+Y)TrFY-%$)IdC=ctja{b#4AsX z+%&`fD|tU`OVe{l3XWR5dtQ>x*+Oi47^48{VXpw2p&jV6KD;Y7GaqA9kZH9j9m_qH z=cF_q?+|~mV~8r*QBEhJ3+hu1!in5Unu$_e9GagU+)#iVqT0Jn8Cru|TW8q5%)uXU zJGxCa{Ek^~bC0OeITK+=vIR?Q=_RxF1STMHkKi?>z?xE~mrIKCP%V1t>=5Wg`(>UH zH!$(l887NhKj=C}8=BIJ9nrd5p(_>Yz97YJgLtoY@s{3+mI^H{gX(o-iYHR=%bLM)1a&jWz~j%P=}tdVs&xx zkZqjYs}`V7BJ?Cc$H_x4bLevWW@IdCQ9@AY&*@UUSz%hq_#AH+F$6U#J1=VN<`jJK(3O zKW=RqUtq2@>?m3eLBk-R4-dlXvsPj%9vKWJ4{OOJNCIxxCXg$Oldoy6EUH{B$n*g) z46bP8&7!gYHNYGAi;n}6RAA@|5MZsTr||5!54X!~(f}+p^2iONNHJE7^t7JV{XTZt zfsSs;IibU62lUlBt6qwywR#PGcg`abOThp$zp0{N>r7;fpQe=qx0fJ zhuS-^Pq-b9ab%aUXhsgR`=!tTOhge2rD&3W)LHRpO!I&d2*fPM+h8k}+=r_qcbO4Y z^r;5|wr~)+#7E{grk2NWC3la=VvfiXPgyXyp!#HD1MNTAan&3K?NS`??q}mL++cYl zN@IJ8GW5A{Rzi2wK4%gv$GAA$U42>p&a^+HF@?M%%il1m+55(b_ZfJZ?~u2eky7au ztRzzeGn^$QZp>dpuEbIgc(z+i(&gO2c5hnp0d0TTz#^w`omLH7D^6H%ka~i}mh{WB z4uQbg8=BThacGH%t|ugSZLDcw3}h+w+|R~|!c!fL(yj9_xTlw$2qg~Y6nm3POoAd( zwqb#!FSE!OgNL<3aVE+p0+7copG_$&l)9apAhz0@-QYovBhSa_0*RsI56?#wkl4AS zMgC>MFFIX5dhdoTUp-GM|4;Fc$pR~jUPMbGbo^sAutX?8BXoWVI^vg#0@rvg*Xm`@ zBpR5*cKLmDOaw#m5M(9C_A17-K4`d^o{F6nluGt&A*3Nmc zk{UQMb1pL%&z%-FLq z7FPX@<}|q9v>m%%(2A2kDsjiIxB7%G`I>4X6kN~pti_mMQWR( zqb<5`krH;G8)1b5ASzz+>F3oWr=FW^*25vV1XisjohaF~v`0Ol(S@1;M)UKMNS~KA zuN#28FT(e2OCo$;%4O1mt;nz<_+cF!6EW!PaQJTmMXwGzy9l9jk9zCox}aQ?>P!F8 z^r|KqvWNAi`!#S?Gn#Xs!)rUb8TMv8K-_efB=5K2#_Wqwdwq^Bx~ehcs((MnYVSs7 zgz*1^7Wu5|z5};S4?@&TA= zW_b~2I0f8b(T>>3Ya}JhCn>J=)`C-W?_iD&4X2m@Qb4W0Jm|~LJC1Ko+!wpdxeD0u zxD!U5sn8G)yiJ5}Gg+(5HCu7D;7?Nl?zYJJJf$>-lR@@Ni;KV_xCdLu_}^Fzt(`nM z{CuIzo{iT;DDyap-RL2P-!RIwu2ug< zjvw5IO=$nuZ;^E`MIJ>i;dT9B4whrvr?I(Uu`T^4Y|2##8>B^l*tBRbyK6FB)Q^XvH~68AB$WCa){Ma9UcY55+eyHL!?qeFGS92T6fCs zweACIw8(y&{G&P&u+gSy(UVY8-eS=oeaW$}VEY*FBPlT!ljSJhluawxj`xyjJD!ag zxXC`U6A**NO`*e8YpRc1u!A_Q{zMDjv?vk&=I89wQ9Um-t>+Y?E&t;GmDrb~Ofef| zE68k)l^iWVEq)hPO|d<@iuQv#5d@op!pmFbEPP$D3bUefZ1pwBOlIDI{9W#xH9F@6 znyuE*u-%*ULwLq;zB>QOe4!X-mm3IEXDZFrL05^AwFbR_olx%qWIPAv_D8J(&eYs) z`r<$e{to&;Q0_n@j2VYe#^?6Y2a=kl=#ZC9*6Zc)dYxDo(tq)J&1*%~U=?!_5Qe8o z+2ah7`(oEQ_iqTE={Z0${d32x33P^w>vIg(Nh|$=?I0-V!f^*x@s|%H${3^40miV( zydHK$Iz>+JUkAi7w5Hv7!jsHp1e~o@6}>NjI-Ye zgKKbRbxAB;xV`oa2*0gZ_+3uIuMht9AB5qCJbIJoYxsoo&s#y<#Uni&z^qHYzO!ONhiVwvDO;>w`S>9J4c$1w`z}q&sz>U%g!<`0Vs(C$GZ`g80a7 z*6otF{o_yLF(m7ULOMs%efKxOWXQ`tZ^f|7x@+|veeNKwMpX!7t)e%ja5qeKBMgb>CQqR)ObERw5w8CFeom92_rjhboQc0lok=}l1l;WD5Y z8;K%m^p+QS4PF*06JII_AqwI&lFT^boV%T|B{BL?yFxM%r7a0q^p41)+i+D~4Mp=W z-tQ{Q%s*~&Zh`)sdkXp!)3gE`R?s@K0u0Yb{-M_k%&OcG&}j($0{*#zzDv^}IJE7L zynXLt^}0*cT&zaP#$03T>NzyW!SiHmu|`b; z?3+B9iX1=feUGH%C8;0wBJsib1WV}ZHEkQmEX41zvloJVc9`CVI($q%C~r}9Ey)h%UIa{PLtR*p&YMZT zKx_4!V``l)j15fdsVWfyCE$fYmI1B1n?VbYIsQjj$ucYoK7-(oA{=ASHq!5n4~0~h zlv~xJOAV++74cyzpdWX%AcTi0Srv%$%A|&1da-K^%p}NZGu6X?6CSf)q{al=TTmyt zEukS8cpA|8*AtQGkbi*nb95pS)#51NCIX@(Y_-Fs@DPWiBP^)LS#u79AIoqSCkgM( zKYN$)nCpp1Qe8xu@ZUfI053e=6lbqC#25&R zdg^Z%6!Bn~`{Q#-li`ldf1xt!qRzYp{LthppAJ#v@s?eClp>n3nat zXz=RE4BO7FKBoy1cK zsCc1Vh1cnX`cd7-e%QO2OIzT}T@rf^OAEGQV-WqZnH3sEZf`?*DQOz&rt?8Lp(FjB-ZjdYj= zz6wQDOHF;yfrrv=cpTnD!{lnsZ`*MRT}2hR7U8*L(aUsnf?tI%BMY}K!l&q>kf~rC zzS_?&=D>NbWs-L}2Rrjph>ZIzs7Eig76eU;>M-1>b5FnYRGrD9TFdBp&~q3Vv~PDA}<7l z@TT}v5ZTK8_?QYt7aLHfFSI4^s)z`q9 zCH363Bwa5-3EwJT*Na-X8e15|5fnB&5|%GHe!^)ajDZ(mS05f9UPR{2F_cMPtSrYF zPYtTCj{$;+L@#%wyl;QIh|sSN*~>`Tx7SJ8RgxY~p*JcMtDy8fP}&?U0rXYN zaMJG0|504Dbv-4{2*R0I0H7gy=%D5`4 zGUhy1;vWt5yL0PwP7~R#cqKjF7L|~?*yb)twhSkSm-CAiFLua%G3XqC5nbT7T!u@z zD#?=IBOf>xlciRfybSFzEpmgp1gci7Z+Lt#|7A{~q8uMajsh`W?dU=6)pQK=ke^Bc zgi@E(1xY2Fvk2(I)h5u6h~<>{0EF`3Ng@Wd6)>l z$G{9p$$7)<4z?l6^SV|-X_E)zu&JGv(Zv;vqKs*Wd>luMiPPkH5IX6{e~6KI98e{m zfg98t>+~?EtJAX`JNDU_$Ocg( zD5$;^@h`U2akM9@UsT4{i)DiePPION?|kx}Z)^b^*6C+ASl{=I9)sAlfb*r2LUOQ$YvLcl4of zJ`d$vf$MtW@35}3LnGQUCcbwfDtVDA*@}%B**`#D1au=o`@-P) zf8&SgQzM`XLDwPzyQ=_!qnzc|h%9R;%TCO)8nbMsjbM72O+FZi)u=bCJVe3oT!I07 zhXYk$gvLZ9I-%4KMn5nE4^0m=9*=UsHZTc5-!uVLASfNPM{&yw#mGq7M%!NFIuSE|f6LACA@}8=s})_mXXoRU zT&W}b1TcNOH_be*rEZyZcy6t$olVKDf?cBw<8zG4CqtX5Agb8qv`K4Q>R^vb0=h_@ zwHpTi>a6hKe?^0jks8&4Q#AOdtkB?7Kx0}Q3q2l;JkFYxF?IL`_@jTJ{>L=~ z_|+QV!DMhFo0BkZ7@WOJB@pElM_K*|z0SD-5P0SOFai{pZau~#Rbv5xz^k;mLc@5` zxL^>Em@UMBD_Lty3)`~OlpA9sBwY7!n1<__ z(R7Gto5OfMh;1)C39Ii~5;E~Z#<39(^Dv%1HBkhuZ-54=q9I(-xA%oBT0j+b#btae zRz#t;vfU_COYYisrh=~_gL0L3yk7n*ee@bKCGVu-WDpd9%R)K1rET(yvs3rw$J>F=)Q!FODu`@)Yy2=HR{I?mkW3?TOvJ zBLLm~1-Et_FC_QVlt1lT$WL*bY_K7GmP7Vv1wEjL&e(0}JUHu#2HyZG};^0vzh^B+LiMnySL8Ai{ztD!8 zX$^TNx})%`yhHl#Y1}%-JXce(^=);Hhyf$dDvtHabedIDQhvJ5d6g}DpQPU}rGG!i3hZHSH?xeE?Ax6!`hR25Uv`Z$ zm&VhcK-S;h>U1LMf^f zb$$%N^)CGPAsH#~-~Z)5wV7|{gOTp8rd>HcFUUFYT)EL@Au0XUwI@xhM$5%6^!2X$Dpjx8P9*wP@X2V zHB$SXs27(DNg->`sCZ!BIP2>zb^3;e4W+ zPHky#qk;z;8preC`cesSn?&S)<{5>k@TT z3smU=2$ivAQzG?sE*iUq*!h#chBSqwh$X{0fsU?4Q-8+j1VvKf20ayyv5Bu;63PB< zCs{bw%WKgwKZxz}+MVb+%5yDhD!ddeSdu+J_8@Vg;)Q-<#1H(Gf*IK5O*84w`*W!w3OK-L ztoO=e#7G9N8K^W((m5Z44m?DM)Qp7XEp_T6-!mpjl6UPY`o5gOiJgGpRL?i8*223+ z&ZbZ{)sA@MQ!R?;U{M}2`aA2<`KrJlSPj|mtt2$`Ky<%Oii4%Lc0JBS0Y*v(?3}Y4 z@@c#B2?bk-vX8VW4tlPE?l#QpuV+ck+kPcf;*ftgPcq-7-gBe@&3Kr=y=#5I)z#S9 zXm_>M<9n{q9Ng)UFZiz}3pD%)#it?;%32={e``shiD=x6L-tes7Y{eB|DP_}j~CI# zsXS{mD^sBCCL{^fmOz&<%SE;RH^ngc#2~?tzxG!c^3-y)-P#!%n4J?t1LJJ#KTaEv z!1uF>(fBrx==f$3z2r2I!B=A>zaW=ru9Z+LHfZP1HYG1va+fBlzwW}s*km*-QHOmS zPK3#0Jd?-3c=CTi7YjW?xde3#E(rz$tt0t4jl!bHrZ((C1u70h3}7*n7ASdu^k&R> z1%~OjT{ty7a?r-rmh^{=CNEw$TV$ZN_la{eEi3Q%cpj6~f}Q+Bo7=`zZHYs9sf^I> z4ApbNxgolCZez!YFEdZx_M=d&eTS!OxgRzi(HyByOVsanh()}Cta1R2dSb2ug^U$1 zLx1k2{wya&EwC!jXgz62?Fumk<_52AB|WL>g?c#31qMav@9oOJzpZ30wJenVv8~*$ zrH3b#6ZW9Gs|$G8b#VMVUG+v@ioLyiE9yAvmPaoE;Gm-U;Hwd zn4+DedJXJ?4;Plghc!FdZm4!&14V`Ac?^8o^t}vSY>wnDbjQ91raZCN-a@q8z9*8= zyTy@Dk5@v@=H#0!|BC>Lv`1=2{?X)7d2$7L`uH5lNIw5kd;!qk2b8F{%SQ@)&OVX& zj<^NY>J#)J44DehTiZ)9lKg#v&vhKX+T`yX{Z9zDA@bxhdL^Lb7}dBf^t25l5jW<~ zIYzYEw`@TsG6Z+Nk7pWm_Vij;+wgVXyEEyG-Ue;Au)1|cFb?U(vx+FSic`;y#*;9< zBT*ba#E;wu`}gP~jd}n~AS;5_>zdg6*xWfrovW?-g%7t&D=QfBAi-~V>VIh9ewYgW zp=0B0YlRto0eLepn6LWN<{-KJu?0)<-1Y4bsN9Q3e^;SrayYTik;;ntb_*OMup78Z z?<&Fd$DKeWx5MY06g^&q{2~nhNx~b-LK_{CvJf9P@i~Ti_$MI?MN0e7aKMO;EQ#2% z5I;hWiS1ij`R&&s3rj6xNhBTe1ir+;=#HlM`9~0o<&FFw;bBE78RZup0c zTTNrOH2Xf4oSW5R0ohR{DJ|hoE!wexAYL297ZXWQ^5}n+KU!WM z-oSg#NRa!isi)AMp0+Kd_lc}1*`lLs5o9vIx7F*yWqaPBy>oGWqfYq_9my=f!6tio zU4qGj#7kzatKmm)bZ+dfecI$fJ*xU*c|BcodGMXKpLfH=uW#p*o0z{CUC2?`m-|I_ zy1PVY*?gSz|3io#)risW6*{`?`wK=BrEZj{Id|gNZA0{L0Q9Z$Nqo~hE$|eycYAbu z)2KacflNpSMMD#G+P%EmpRsYJ1o}X}xZA_eY+6w!L`pU6uP5SMyrfPXX^nsQe(Q?! zefuXo-+%QsJKys^`5)$cBwwF3H{ZP`HJR`3Cv_3?9eb)FJm1s4{om&MwO!53_cyj? z=DP^9msb8a^SxkC)A`Pt7@F^g1G?z>zR%Xod>_~5JHyt@e3M)3^vcNjZf|Qm-v=>M zyGow#w-^5}^9^TW_a&IAwKO7exqbkz!2YO*@@4xH*za*+`F*!fQSh*pSD z+35|uk9~#ya7nY(K5(Af`5bBtOhzh<>Z*S@Ul5%h%i4pe$wkd1$w=kVP#$bl$f-N` z)`t)JR~WyE^Bs zeX>57fqr70o!qrfXKyqL8$!MeYn_F@_Hn@O_QCbs!ta8Nwec)TIOX;~DMvacey}(*_yGUEAQih8|+< zCScBb^|Nm-1Z8VQ;5rm^hW=L1FI?BFZ^CzXaYO%Nna#v(01B3^YT=3vIx0d>MZFml zhJlRS>S1KNrFbomEob3*KBuCcvyXb`69^g0a+8%5?JkVep#H1+GGUoYh-9rNe?!=2 zZKkucI&eFRWi6KFC%sSeDUX?@frDKA1>Jc^k<`@5$dWkL}4tsl^6`XW%E+h>fL}xK#=G* zB!wxy>vpp9AdJv44B83!)b)Mq=-iNx^|wa=WjLf`L|dtw(eO>(S4mZu zQPsk7vAB$?rolFE(u)R8K}A2_)ROK5p1HSzaE5-7I|#vyr8jFcExrcSA4( zQ80Oq5Fu$d^3a@nSdz)Z&*(GZtSf<-?^q>e1EnS2brW{QwJJ5JUN?|?@)+r+?lz)1 zmh4(1=FUCO2*(TxHKYtzhhS?HhL?KZ#qQ_&WBOCrhMPP z7DlMH5!j;nIEdWp=4b0!*+oz6UBdyHrK93>e0Ld*585fJ-2fkQ1_ha~3CxAI)RW&| z2(r@~`s~ccHXnPoUc?D--3N6JAB7z}6^3;iijX{TeQU@}&4m70JRxNG&UCo#$hO@^xsZ-UW0LgchH7m5%maqz-s2^k(LKO9aszg&`t*09^-zCpnT&EOtQBIK2P|?FFrfb-y#I(p%Lq*z-J$x#ae^I9{wri_L3Qlp6M z7T|*dyUd$j{O0@wug^t;9XQ)tHYHx7b3SSUPN;X(Bd5>n7+3&#&yVE1zrDy3IlJ;o zJ1~Mj?-8FT!sp1=271{pX*g%!Bx-q2WbYM3;??|PbF-Cow81Pfa*@A@BF~5-ziWi= zCc3>DVeeCsyw3g#qNTLiqD+P;vxz?s7RdI2&yh&JxR{ZAjk8S=q5^r#VG>~e{9_r~R#dk}R5ye_|9v=re$x|;%teKfN#JZp zMBTGQ>9eFZB>%9^HVC-bvyS^tdPQ&A7E!<~3Ka3@8$@#h;4^PX{ttOcRb(# z%?sxDGTTER)wgEF`M{;;F+c+Sq9__Lwn)p z#kI^!Z1inJFZIdoVheC}h3s@RqrDA%#HocB`I_un15HfuW~@J_T&-Tf*qHIY2=xd{ zF2*8C$kn}RJx7Eln{ctnm()q`@f3a^E2c2JpC)x$!LPgMRW(qJ{WsJdH zo9yi1uD! zR<3xkzliLS(#FG(ZD=hjQgiEq{#9(%uHGkFe1kL2zbRr=Hs^*%1&z@0Wn=FH=D4{H(nFVcs=F4^r3-X(j(dirn*&wFjcoGESZFyNt? zIb?zgJ$%ZBz9bK+x#y73aToS>!uR_K)fxrC5f#WqZ^0rlAPbwSNPaHI1s zzlV!!d0W^-#GCfT&7@&V^{KzT%jB~fO-As@q|EI{+C(+;C@L-JVa2wuad}u*ajXL} zF4?AKCXWa0Ag^lLdVwr=ypz|YZnxrS9JI+ke3Oy|I5PRI zQ6Co=^k(2^089-5Zx=u#1OEfSb|K(k0Sp)j_&ES;Zqk5l1aKDvzX0I=5b*pv4E&6N zYf-cg0rv^us|;L+OeqB1Ab@ii_$2_{A>ca#crOFd+sCvJFkb*O7>G(!TL_pffGG^z z0Kn@)z%c?C%fPSk=n(?;7r^7O1l$O~vuPTzl>lyM;3fe65CWcgTU-t>un>UjL%`hv zxRimL@gxxft`)#p4D?}un-K5~0i4XhA^=Ve0p|(eFb3k&WlIS7fB;_0Kv>dgeM3OA z05-&M6vU}t;ak+St3nx4AcJ|cCSe?k(dZesicBNOJVq%yS^Y~T z3V2*hh-MQa;VAY4qJYT@A`VbY{BH<4ScyE1(ko!l%GX*?OAEM=iNO+2XqYoEB&i)%B`!C?z zbRmk~9tl7Ers%&eMAZL^g!=^CqYKfs90^}Z@Z2W^Sx=9I6)?;psCq&q>~V&rqU?U1 zM$8l6L}Dg!N{Jh13A>}}6c~WipS=!srg1~V5La8i7>QKxsUsXs{hLPa4|hWyRih(} z2dhm5_@EcI4do_CEO~(v4G!h0UgU8oCqH#}gguVgw9ei|w`Oqu#XRRV`4_w8CCO#^ z7h5`G@Kd5QMlO=er0gAwEQ$Hv33~U%JLb11vxo-u1}F7;q!~^cAS%F_Cc6ZVHS<65 z?9N5y4mvYa6FE~S4)>}gT1RWH3d)_=s1Mo=B%CD<4u5v}VF>R2Lx0jy`Yk2iT^nOs z{1r(>r2jM+PxLG$>_GqGkZ z^AnUg-n95322C}Xy_0$|%)y`Ti8E_W4X4~j)8ZQ#G$G#Xy^}#H{KeeD>>bZvT2tnh zrp0j#vKh_Z2@E<%=k~(q63pJa_zPaadfWZY`|6Y`v<;eEG1@!n6P?xj1*I+C$Ezjp zgqCpd_u(__$z^7Dea!ruC1tYDRM^S;Y`ME3VNO5j*;U-JIi16mVRpCgyyPxwg(^9- zyv?0cal`s@kWs#V3jQ(KE6whPR(VscAO+n&+G}{v>|H#%ie5~1%e&pP-8mLMzGC$j zWLD9G!rE){(%_41^)4REUoH;M>*>xZ8N(}S1(4Ib+M<-1KkQytg5BGG_VsZ( z42O|6@_J1Y+W$-)hoFhn>Ot1udd2Btvnp)pYP0uD&d{V>x@vY0WL1N-5r2GUGb3(!>He7D6WcM|q4mk?N6h+N-d5|(-d5v)2#jm4 zNAhu$kM_2bA$T|@HqB4Zf4=o#*Nt}X!>!p8&Rv@{e}I`HhmSI?>wJSKXJq%?^8;pg$=N^x z=^pV>wmJXuy|!%NcY2Ucs8WzmIk0VtSq$`3Zk`0>ahLka zUx|u_gs8~BzabcGWX~a2+2lI%fV&&72FKm1<5#ZiC%zx|EblD=dch*hf_ntH>yg8$&hVSgH*5xqbC1D{asVNb%6 z`&RV5IhQ;M;Dc6+cXSM(|H(j7#$HGf`>DJim30j-ldYOgO?yjNs=iD*@EpKYU4Y-b zS^4Brla6xe7x~&kim8W}rbA|k(lu|QLl+#Th!7`R449&KfwsIRiF;!2ezg_!Dwr*IJ&IZh;W7aN2YzIf>ZiXC(jO`P9{Zn zvLkoWsE&VAbn=5bkY{ZlT_)gO}Ceuq0~^ z&%J3KIa7VtaHa&zq*#;P!*sb*?B0wQdeRcU+@G#d8`&c`Jjd+vCA2o{N-O+(JUXCb zz7(ns`}Y-E)&;j>l~T6RmcDa7Ds`{!8S*DpqfU8}>}_AaB`lmh$|8(=>*JBejDT=| zLf9UG@ZC%6#AK7h1<2HmGxd$gbmP)GZG=o7{7VcfJ9I*?#lvWpSU1o<}dC%O>^0Dp<0iNOG-FFVe!s&!~ip{N@F=P`r(n zHYGEOm_Q4MA|;!Zbaybx**?|h4jS+g8Rf??3#|h^A*JV(^BS9iFI0hk(1QP&f`^d-Lx zlZwv3$# zU8^^}THM5vHbfs;{yxit!DjkD-i%(of05Jd0%^mZeVFSLF3o!{#|?cLY$57~HRRpA2RX9)W&XiHyt%L;^x!zkL6Uqh z;Qu3gq4DA^FkmwdH-sA8Ont#(lT@ir1Z>iab8eCk*t6AeCBoiq`Ktn-26X`A(~|HB z06wv1q}#^GMA+||r%VK*g+{xXipM`I z|IbF{N56LE@>)C@=WVR8v@18G0b(1q@ReZ52-K_Nr?Ufmtn_}2EFj3XrY~;{&cTv< zxXyKtUA~c?_*eCcdSbwC(Q;A3o3gq-7)Zs(3%wfqmB$B^>P@Tb=}P_G&CLH^{A(~A zfHve-%$v9*x`l8Jfwvp=kLTBl4hVOe%UvY9(`A9?1lcG}Qbpz`Qtk}Fa^ zY0w@DrqV6OX}f&XB41QLY(XmM&aCV|Nb!_ytUx|lh!|G|24Lb4Bq-Rz@t%#lc?P?A z$EPB?<(1e~qx0o>i)^jXI#7es3e1_&rgW7&2V9Bj_(Gi)mm>>T)F^cSlz}~pryd2> zasLkWsCTGGilajPfu1iplW+{oHjsf_!)e^o7%op zpXPlM_LOxceSFjT0HT8d@Ai3v=7*|;T%F`@;^}z`yQK_MZ#}{@^Ww*$emoO3Gl|@f z`?Z((9;QzG%g}2BE`m!)7-dx;Kkk&wn?%ns0tQx7y@yH8ABS$7O9^*_z3kz2se1p%^eB2}->{zu5I)A>zz%^@n^Kw`_&y(N1Hbrd=0ZaO zgJve6+4b_BSvZqyKo)XFE&H@VM9ReHr!}F;uqf6FRjCVgr~DueOI6?|K&)5u-SK`z z-T8Z;2%jUrsdGMVZe2Go4-n@z74$_XF{saDmS1zdy`a!77cfMaU(9tnQH~EiE&5DqV_+$LR;<)wKssiwRTlog+EFapH2&l3wMCN7FRouIsvlsg2zoO-E|$ zwyM6VVkCOMAoJL(FjHO<;?Dt9&`IbkN-!9{&*T065i`E=$1R`ArDC~xRL+uKn>&Wa z;yOvWF9~s_8_Sc&8P{9jt1-}rtfGfO&5%E@JkX9FZ)Vr19wjuH+Z%^)LqC9>BX^I8 z9%g(93%$H)JU*6h`s20@J5qOkAE;h`fVYD=?>Dd=X)Na34(JOJr}6$tOAQ_In&odL zd5@XyGHXik(j9{di70L`k3W1sJ_FEk8pv+Q|ID&aeembYd;tE6d8b90Yf#^kgIZ8~ zQ^9_$yL1ehQq$uUv{6uNGR5PHuc=%Sy5^^QemrKF-St4`Xvktiq`tGr$1y&UdiGZ` zLhztR5I>qNqu1--MFD4)1(rj%%Y=_}K3jbWCxWU(Ha5-)R>)GfD2$!wVHE0b!Ts;r z#*a!c8l}4V{raGP2`vg}O!Y;U?Gr>1>{P&vRM6+rWpSAhco=Wy4+I8i$TuKAKYk3j z^8tKFhaADUG^kxYcziAt`}THKT zzAMjDrSJbXPn9=><|!JVevD7QX7G9B5Ac!xkN8Me!Y7IGNooe4ip^KXXZ_~?8$Rn( zqw$g4rIO?>kv5>=ewP2uRD*6>llQ-w$xZy>V~oPBeL{Zl#hk-=mb=gf55Lf+lzm@{ zYWO?4GoS1eCI&%aV(`wpM4Em6kThLDn#X}OOQ5|;eKcY`>}qY-e6leqK6JgI5q`MM z#&r*)NVutU3nLJFa%HoK8U9zqzV3Zx#7YO_TY~(k@tm8S<;R+vK=uHl1f=hrR-)xIm3g9MR-wEP=kJ_$DiNl7C1fG*k3{tbCUHGtAFjc&k!H`M(t z)X#G0=l9`$?vCi^9<85P(LJvbTiDU7Nr#7N9o{=AGQKI(PrqBTAA~bDDgA5V#)i2BcU_-Fk_j=lXMA|nP)O`!|I`JFV zYZ)XZ@7Ji;v3b|wA6jYZkiithbC9H5<$mrVSmk0-+LW8zxe;kenQ*s~dp9ieTCoMw zIS4}~{4Oc8$4km{<7r_Z8f3TNr6R=q3zyHn1kb+i37~!e10#>jyu1sA#=Z5#H-lKJigc1L$T7K^`$++_W=Fu z_LLO!@JYQ=TpeKa`(V_CWO@hu$aUn=q8)WPsQ#ME zUCvMS1!C=F-ucJ!Ut1(HQoAa1hh+J;mEu`kI~*r#gU$!tmm5qTw~n9DC52_`e4XZ(k`$r(W$h{OXyrQr_9+3#Fjv(RG%MT8nU8fa}N2)V%eQ---fBrZ=&aDF(wbb=rb3l z>*$sXuh;o$qIf4~DnMtuG>|u!@IXGpxfjee;G1NV=Lfc|(V-{$KJ}9q<-%cnW+I2XuChQp z|AsUCi1isddZ1EbOaIAivXoFjZnL{&8~Lz*r3ha$E#H!|!?bRPxj&p~Zf#^;%ryl% zqum2yq1U-OnF~imw1+NCOVt^u)_MU)vi*G0D&bOiz5;J~YD{jl#F4EV0gP*Y;(Xzc zj2^MH->3~TM{mn1S9|;$U8#!E#)zBBqcB+A$>*ANjTU#j4f!&KF)!XK^n5TV!O)23 z>LW&c=1MW*&+@xqb;Z#OK^kl4Xc}i8eZ{fSN_=CTg&L$^iW$3x{jIl5w1)B+Ip== z{-47?chzf!{H630>+^36|NNxB8GpfKUv5o3X09p0)odzwkCwqXAsiqPCB4=VG;gE< zM3orwsRi?B6kX4oQC%?4s6O>BkKPkB|A)2FyV1NcJbDiEf*_8bvz2)x_ca)F4Kr`# zewN^WuIeYe$|5_=tk$gjK1b?dZ}Iu&?p*jetvj=MfNgOePKPi4dPg2WM|D=C;L`FmU+@>fgg8ycmzG=>j}PXDJ-`e-&V#^xRdgTf}!s!%zjwRwH&9PK>3Q_}otif6=A|WTJ6K zG!MBNI85++IJC5+`?5vkJq=gs0}=6Z2S+%;W3L&lCA3yS4I-8=54*hk1Zrp!pI$4~ z3bhXSqeWKmFp9inrP4H>ePsTW|1p0*61pKj`GE-+3EdAt-c+9hGMOA8e&dgLcb-A# z)UzrwSRS%xBz|I7?x6hJot+$tZ5aM&!x_Ji$QW_|6qP^of6OoaX?`)^o5!QnZCWV2 zR<5GGwzzP-5S)x%(Y5|Tbd--<)0fZFJ8m*%I50D)Mdol0Ese$DQ6GB-zqt?Vc$?{> z9-oHwxK;6?RjFIj^F>`?6jm5N52d=V8IFGHT#uvp@%7=*heu`!hpUvDGK=;yb;*m- z1g5P|laeke9pu`9G0vE*Vs|ixEuZwRvv3v$*k@{S`Y}ubHcO;WFW9-aOc9@g=-?x~r)_dg7Kf26%n z+A~5x%nYpuqSQE$sDtGo5Ia*sZb!n4g@EKAz|#;ECrf1}iwz%OVlY;#^#HS~ z2?Y(BYng2!RyDU7R>iA30R(f5=V?@1a~-oO zn{nXPJ=5uC*@bUrK0&m2{skCmpv}{Wq(uA7FuP(VW_!JRu0iMO4@}B}<47ZFm;D^8 zdnvw5ioP!+TUu^$DzqrQflp}O!UguInG^stH5eZW%{=A*$J?8LH(6}|<4M;PC<#cR zAVR^a1zB2IY=uH24J4Yfh-^XyEg%XAVge|Xf@zS~*C4JJ@4X0$3wm8}VUe|^K#Pi0 zHc=D`B2J7TvXrIE`#qEZR_FG}kzG!E1U<^B$t>oBG{67AX?MaT&=&Y9llfcdxDX6C4l9CUf;LOucX0Ra>yt+ zo@)**T6fz(kjwrb+)x|q<*+fg{X7d>02xZl(Wly#VY z@K=!z4Z07CrLB!x#obuNAf5n5K)Js$9+m@7A2sU_GySiN^VAPU;|KZtgTrW8H%zqe zbv`w|RfO)FiLrsLdz4y-;X&Am#r(|d^(QG-v3LIF#jF;^jOE2_s}0jhbi(p%2(c#) zI!~=mFpcJXuMFI9Uh}ISy^N36O*cUE)(PMCZtn0G>E5|w(PD`beNp%nWf?j^Y@?8 zd;2(*zPI3ay?yBqGajkrX*3#W=flre)1!Xn=?|d&uJotp20TA(D%$3JRN)9&!8i9l zdDdfnEl_^*-EnB=N59%G57amtlIMkm{_-3ZB+s*+2rtiVwgt)afoow{jUIUHli~n* zuAJtR=a_oF5QmlLJGc1DbHd|)E6;ITeDd7pUtAFP&nH3bbuElMFTvI|E9UaNXJ$Bg zKJXeZM&`wwy&9&IuJH0~mNp!r$wWM(Wkv9Ldt*m`Eo@>ySKz1V86`gbGH`xAnehMX z{CspejlI-_aPxE1#{%YOm~(RzeDiT5-+avOZypn1f2;P1K$r0PuCcdURnoelqFX)G zZt0zb*XPB2AI;lYRp;Lao!YgAGVzWYP+QgTu32={H8bzpP+HIx*5gq=yCAeH;q6h- z-f|2Lrjb`*W}*L1B0Ki!<^pm5MEdn1cq{ArxUe+)Fvry>+pUhPT6-@Md;nX<{yI(* zNe54cpZ?~bVL}znD9w#Z9M;qU%7p0)^#|E$=HxyU?+e&|vr29*wrce7&wN>IKeKNP zq5j5P>LS=va;tq7Gf`Y=8MOlTux~2u`^QdK6oE7WYJc9rZQQrv8f#vUt)Kg_qwpXh z6D_#MT5tt)*g*a0URTfVwC1kH3gpo`G2x{?#^-~HgqQgx2RE`PyK?(XKREv@s<{Ky z>|UqgN%Tff49zPMZTOAi)m`L#MAKi{RDv@_FPtf|2%7Te4KQN6c?IIcgq`Kn`!t ztf@SB>aZ7yEBDZ!UXB>f91nc&*%rb7chMhdG{^9$<8?62ydC4aXqLWbAN|aa$z3ko zr>Js+)=i(Er_njkEU(C?x z7>!KZBk$_nobcGzsX>z_Ec0X4g-2CZjnYRQRP4C zYI5xFQGUTOmZL-6msCIOnbn=7)+n1a-VGGV|Ci*+I2&pDG5s7n5q_=?vM1V(M-%z( z?&Bhy&r&*DHqN^(2><<%2EWgD_y4=}u@TarW&ZjD=o9Axp5n<^eAVt8=fLJgeFosTV6!_1bZus+>#uL(P5d+-`}etUnR+z$)o17{*7bk ztUg!nV8nRG294geD~=Fh&9YHvtlk-|cg}~o((zQxje6&-C=AW@XEij70*9NNrzY!X zWZ(IKPBS@IKV#oylYVvyzUd|2e2)*A@StBhG@9Zj==9GViowIs`%@K)KW27Zi`GB; zKlI9$PB>0n$=DCyP^aN<;y1~$8qd>y8j)GQ;8=Mv)lQ_+-__!9OiuM~t-iRumeTKP zc}yFeL&tL@@*E3J2d6iobSI+9xlx7m&L5NYnHgOSla2bLZv1(_A!f3E_E~uTlR@*2 zV8V}B!dVTLU^9ek-&ty(t$Db%%{YNYoTO|zUlG9#Jln*th|>W@z~|pVn+{`}DycIX z-kI;{(9BC7=Lz+h6T^8yeU4bR*|7s8@L)Oo+h@-DPVYR6%U0G2+c2~AmHG015DzE^ zqd!@-CvANxK22Yp#%7Sda-57VFMd@mXyJUAplr=CL{MOY*-;v0$@<1#AycLDtLTR- zyU=|-;e5hZ`r^*Sj2W>zM}a`WU=UIVYiABZ7A!@eSd&y`bW~|ED2Kg$Ms>!XJlvu?uU8W~DnyV_YM##oi3* z`t{?|wcZ?EAcxtpJIb7O*}l%xLn#f+ZvSqXn))wKl2WQZJm+f?o>@-EM)>M@@VS`u zzA3YGeT($Z9789i(wG6R+-yy*%chMoE~>{-I)AA9Cp25yld(UenkeNwOzfNy`{T&4 zlzM^Emu2h;DNFBqk!)^ghQht>?MBx?Egj?4un!*N1ti#z$ z?^=!eT)|$eYno2GJ4V+XlvTdb>9E75N55-%9nL-Mu~?XR3ierCxrtii?%YI8HoA{2 z<#_R~9NHRLba)?=vEM4w)Ms_Y_WM01==@eX>^KdiTgOS)Q@_4RE5vfCT39oFRuBF9 zJ(J5dXq@J1&Yb+AR%-M&(YQyzkM%SkXHV z)?(0NQ2&(DP=-7|gS$pfcgJ{*;hv=v3^YM=Be)4$Jy|dY&0Mxs&|y$HtOe}aBO>87 z+IkJt?Z)sf6{IR&#s$ea<0|KKf>oZOV-HNNS1_untJhyFhP>X_aP)=NS9Lyx4jJRd zXvMf@$+2};ylFPO8!x>mFds?~#`W+MrfF2krN?kdI%>~yze){RH-&@H%Pc0y{}BV^ zuv=$gSPm+KwV6!lvQX&~U#0fRc+0YACoaeSFhw%0uUOMQPd7wQXTcPF8vg5fL(M37 z3i($WcbjOh(9y4J|7Lx)4dFp>$V=S+?ROEN!qrqeK1@+d-%-J6WD z<#3Dg^lpA+r8__yC{)AGOd_Y#3HZ9%VZK!x&hJk!`RpW&fr^PS&N>Bs)X?uao14_3 zMam_Y zYilX;HpRTG^9tG!lP#|}Jovu2PH~h(Ij+FM{|4zNi4=xM4M8psh=EtqY5yZU$Cb9W z(G=Uo{@^A(^@iLO$Kk#yAg8b!#U6P2L8>t@l^dKQk31>Dr|YIoQ|KiMxu>x(S*6?T z6*Ksm@hPh?%QHIbX) zE9ZXu`B3`-{(bD;lt%vgaQ4=eka)jz-TWh|qaaoB+}sOvNt!`#dLBL?D>w?K-UGGM zk2Typbra6VZ1`5%MB48#JIZ1xU@Ym=JnLeT;CNr|TOc=P#2cQUEV3RZ=5&z9ZrjN1 z(J>chxt`=e0BpBlw1Hf|XaiL|l7Z#N1iT)o%1~JE@*31zJsrs7j1A+0j)6xF8PzQF zK7Qf4Ap`kf0I6GTx2u;rHq!E!$Ga?|mM_sd&avuS212oUo|&jO3lQs&13$*x2$( z5hjot4xF{OV0YlhKxxUCj`@=Wc<=w<`!ChQ_0}0j)p5lhF64;C|BkO!GWJLnk?}V+ zeMq}mjAIZ;&+eNWrswv*Ngp4HAKX4{AtZ;lpW^8`TN_SayRG5$s_hZd5iQ3o4}wi! z96CY7lR=LZ<&&pDXg`|o-ZwRDylN=Fe2{?#s=Mz>O~`rx*$60O9-HyQO=4(6MOeM+LBWK$K$Iga-Z&XT#kp^617;FG@i38;pS z-B;MM!L*6=yhfnSSXa_bShvryCqr@14hYEZj~8u~-$lD3zUysq9`++F?&qBOSjY-j3H2HpC=GHHEn#xQHY2N*;2tdVrUS zBAS`+{s?5lHC}Pl=w|hkFhouEq4Y+Y%_=reXI4F&__aNq__dLazwPHR8Wg*IDDf*! z&OhfU(~;wO=2+kTl~b9U>`>y1TAB;Wg?vbDu#}k6vJBN*K6Ww|TyPn=ibU98MbkEJ zpC@Nq(z)0)#muY^q=8_-%7wF+>R$^zNT;Y!LK+(xrFIX|V{ep6nwA=aF!4qS4^T~OEmCf( zjzY%frY0j0C%#DG0%a+3`L z)q^vSuHX`<{u~nu67)M#ZQtZN-pn=Pv}8?`zW5J~2+7Wdj;~i}bAf?Mr2K`@jy&ED z+f?@CFk)NZZp5};F~qhtz_x{KO&+mrkZf8bri{KEG9Rgm)YUK9NfTVWe(AsHUM%}k zSKl;ZxksU30HEf&B}m?$hHZWML4Mi&0K8T?RGt%qgfec>Vt(XJe!d7DJ(2I2X8aVU z?}SNkU{aj|6t>-t+Vf;69Fz}NV*haYv9WCRq&fkqH9F7}2BWk=&A*HjHDh!;Pcb6E z91T+zVkxsxlYDfi;Ju!Ag8XGET<+~!ZZE*GDjK0Z-5~hH9xvt#khcU3e!=-ikk6prO6rpKs_Lx=}&kvjC%Ff2$8u;zL)t4a!>SCU8lpzLxABW$tcJd|^KLpdA z$k`54OL@*H{JoBH-YjyOd^y`<>c4nS4gQuXX9tn9lP_l*Or6ei)+fR5hbU)zku%Yk zGZ}LZ;5lnC=be=ECXo~MCXurx=4{P#Uc;P;l=DWB^N=rR3(WcFSju??bN(_GbG8=b z|B?@jW1Vqy^@^u~oP<~5?0EC3vVVO3 z9`1Y}^ewW*poQo&6E0G)hBaWH%ISjKBAeNm22B?t19|7yP{gxeej( zC8_PHXCF+sZv4G4^^Zn$m;n4gUVlzd{U)bk)W5o0G-D}bLaEWOEHP&8h7BBV!;ie& z2rUWaS2#Ss0|Nr_{BF*@K0LsegTC)Eut7XPFJ6xAF{8MSuD8nF^PFew&DeW)b8lc= zbwAc{XviJ|XzPos$@2Mp96g0gRaQ-|bVWb73^Ab0W#)oPI-EJpKF{SUXg5Ne8 zBUjlzhhzMM4sy&uJR@Tr#|zBBO+#qaVTjOVMXbLzFV)a-*2Ckx7J6H0$6HL>9-^<+Lo|k=?ihK2f=wNQwtnm{&0{8U00TRKC1i4Ng1s_j5nkPB zcAjL8<)N!GS!F+d4%W#tg{Afr_C;ODU}h(MW{EaY#`pROqaL)%z*m%t6ZDyWWEz7a zF)LS;FjcGJX7ApdY;r%o$t|@?lPWDT@rey10`3%#=F+MZ^EPZNEg9O~=F<8U=Au01 zMtNYGWZIIORi%F(4O2ykSR?JOe(^idIMYw7;l~-yVl-5fO(~NdbCc2Cxbi8WVqu9D zu`RC|l@U1@8toM=dU3x=8?XEdobEPfi*CG(bUrE;qS$`I-~3Oq0@?xJBjuV4&ydC0 zyno+$CRinV2@G{Pe+!Q>EV_qBYeNT!qU(qOg(d9ROZdZHf@R#71L}c7QeuTPgc{Cu zP0(hQK@asWBtsY77*IkD=L|ina9`x};*82LqH!5%oINH$G$xA@PO$qS-ij-MRGYC!$$rw2E>$KO_k>Eu=0jLv&}!RKfe_X~Hh`!4HU-s<@r( zLkp^SRKB(rTtKGW-j^iItAv)b_xq|;DE`Fn`?v6WG%0#8M2wcU(IWJ~TFy#^&EbLf ziY;0i_}Go+IQ*ooc%NjHR$${Dg=>XH0Cj8l3(!xGUwwBN6$;X#CKlxgah^cI6!X^_UAd{+TEo<2Vu`Z^VG)Yc6U6BwGF9|ren0m>g&XB|TgnHAZ&0DJ;*z@*Ma-}GKR*h% zaMxv$e#?_5Wr}Ys1uHNkJCKD#_K1ElC|7<&^6Ou-yJ*bgD3E8Sa_Ah_@7!Fk5eh_nMZpNVF8J7?|Z2s!ZPZT{xTE=YyjmU2%!SFh<^X(##G^XT7{ zg&Uu0?1!zrJ26VlW8rybVV>+Pv>L@Vz?5=1Ct0)5py8fc=7RhRv!krujFA&4;?J{z z+;C*#0gMn%oPTp*jzIbB7Ec%&VqBP&uQ8m>MttKMt)-!LMfR3t%sO7E1Icv%SOa_C zec3d`2#4SRWHPxPQG6LeZi;?#e+n5Lh08P?ITeP$8oKuST*EQrW&2t_R*2>+0^2-3 z8>jSgleCriDXs96kQ=!hga;645L@G7e^iF>zv!=q}TAzwoOtQ|BT)#Tz_4 zV7m7fqsL;J!+%U!)Rc_`c7UKydTWJIa?$N59UnTWiON1`bI=IomqJqBG6L+RV1%Yj zgh~Oui*yXz{pF@D_@mskDvstRs!LNe!9CMobnzbfP;cB{7OrCBp7ZKPeE0>HY5$Oh zpE+61NdZAGb;mhQXr;U@YT5dwNm1C!QyF{8O&CT)ld)eKQ6YWpogq#$OK+MR`8Z29 z*n8mQ9%bty2-Ok0DZTTdZ+7x^dMzEx#^7YVMZ-Ug@qU>0cOMqOhte_FM~rjSYi(`N z{jfXEV$7t~e0hroZ_=e>_9Ra>zB~XgA#;hn`@*rA71#Z+qMPU3))=dEDZtjgs>!X5~R&b5iUBE%KmrGab1~d&~tR z{%dySMrqAi6@_Z4t^cmnd@NNSpF{F7)~$b6Rx>C|G5z;T z6yH)$?BxTFU$oKr6&UTKfMiJe5hf}9^H(W`c)kv4W0fal8W+w_&D7Ysh#|;vn~gLk zo&I7D<2aM7=%ctoC~kN-4ij6|gQknMc`jS3fk)5)HNFX(Ypz#BunbU4*J^Cjh4nH_ z0pIi>;lmlG$H{1z+O4TNw8Q6WWS(@6eK17}f>4x&5!U+Tubs6EO`63(%ms7TrM`u8nFf zq&4V^yYW)faW=*{w&;m#Z|-h|^C8xVc^df?arR>d=mZt|C1@OKEx6*HDC$|t9mL-X zJiTj*3rKZN4sIm;*5VZzv}bYUDCUAA7FbLkBMr; z7u`%>JnkB)rhIhMB#Tw~cZ`$Ic88^4CoE^M34oQieEz>NstL%VbPp=SJYW{AOSaZx zmbZ8H^uVin@zmaP3(|U_y7_6u88H(U8r%k=5k4!T#ul#OPWj&zE8f>=F#b+!EkQ+Ti@F-ii&L?U zNi=zUk*ZSjG=YZ&xRxkYP>9j7tKKAi7|Z%%5F~_ zYsKU16t|i74?KSy<=+yTUtioo%Plj8!Z-eOf-QAFbLo}-h^TJvP)61t7{5G|?zgo( zLdAkR)fzAivZsmqlo+NyWDB1qG{SxdicEeQS8jUXdH+rR~-~yZ0jo4+Bes)_3+~DUtJXkHP;8UclUhl-?f{hgMUA|X` z)7pM2U%$R@Ge>!)LuvG+&8+M&XYH8v$HKna2ket!BVVNv(?o(*)l{_{G9}KTZ1ZH))iScIax7}fuqhLrfJp1ffJ;0D-?0X%CHIOgb8k#$fhu0x~QrM$EmP27wvsm-5I%@ad@lyvWe@V8Q9UVB4+K zm=w{NJFqb~1eCaIDU{eO;8j$_>e&RWesaipw1DpSF=mgghgsGV_A4}^7d4{0XhbeH zq9byNqI%$gHS3S9NFf%^{*ja9>%bh-9n>f>)>Vao@u@BlkvUM#n)Q#J_X+o>dN15q#fb#%x$o> z^{~|r_8T+`@J2M;|&@z*{%je!3D* zZw?TIi6jW`q9DwwBm0S6ff=Szh7T~qFrMMHx{Juf)aT_ z_Tlgb3VBcz^6qJMt;2h>gi?ERPew+G{~KEeC=9QlTBG2!5P`;;&aL5<81=90$_=^M z3-D(R+5cHOhdV53N4^{NhY0ik{lksYKD@-Qckb8c;;F&}qjV9p zz6ZMEHea!|06F*=f6t9kY`4Kq!Jg~M0x7?5qyP0HoyTsRtApWy8Vni-w-yD=g94_* zZ~Qa?L$SaQTbEo=BsP4nD4%Lk?m8m+azKF&Do}<3JtGkO9K8k_S>%CqCShw$Mta42 zZ2kDFRE@o7t~58*(@h$kBGT-q=o<8c4WaEUf@0ZIIcSR)79K3eN5B12X}_{*mS!KyMlf?LKG^q1EQ_*TsmnY3nHRQRrq*iJ8MyVN; z&ib~(B|0}Xh1R9X+yWBksVs}Up7ZE9i*JqV?^1MPwn*00O!%EcOZ)SwNFutnq$L{m zJ|4rudkxJju=DqdF{Htn>0QaEcandMD0f?k$`s9nPVH!D+lf zdF{hGvWiaOCjpNDQ;37O49@>18W+pPUxW^i772Ks(ie8BgdE_CU8>>jH^c8)+XR*1 z_i6Dy6Mld7Cx4#~zmJLcLoMadY0j+JAENoo3Gn-)c>a3|{C;S&=n(uqM4c;6UC2MV zRmZc~D$?{+2Z%8C+0vB=@dRNZHN#?%f;gi51ZPc*zW8en?XyN_Z?2ASy>opOUJqaZ zFdDurwLL$zdIK|K%(yifCFGyzpc5paSgM1a?u2%;A?>++f#jgGx^hQ}IwZ;+mEtGN z*#7?F9 zu~Twhg(O$fk8dRTAg6OB-Bw!V=^Is>0gM`EmOkemlgE|e6VP%CO^Q16y$*l#2!BBG z4xeI_b`#PVp1u_?@Ro7hn+hNFCe1w0$~P1ytB>iNu*R%%TQ{Rro?WyM<)*v{cFB*AE)}Y{z|`CZXtm1R$F#=m%F*hd zf2vMBk0d3QpE#Hs@b?Rd|3MtAMk}6zmHaT29~$;(i$JDZOK65x?2jnwm3LNIVH-Rv zhxKVjTRL3F^g$U%-Ggy#6r5wD+X}F>Kh@JjGS+A>PBA-Z_>GbONp#xr~$hn4A|)2>ChQ{`; z#p24Hs$ii;Q1l-!=o6X`0hxy;i{sP#S5ZYbswd|`g^ksEVf?OH=%6)MRILliZSUmn z;-dzht7aVaCL8Gn6N^M$z_^8uyz7YVua?jsJN(Ntm|a^LVszD3b6&3^0Gj+tiaIYj zH3X%Bg0fX7EMQ}qfx;rL(9tGudI`VlBqR#8w01wxIl(;^c?InhJBw1yZzFvTLjCN8ehb72W*`<_9slX}|%lmJ|uXpg^#a&MzqYqeKF!Y z_|>$XLxrg=bYjeI1c6*vAL5S(``t89(wb4f2jw(qpZh(DOJSu|emo7ApDr!=lyE18 zcgp297eG6EOZCD?wa4RNQting?@x9qi4&-jZhts9GpJLo@zE-!;|Aj#TC(I%Rqc%$ zh_7vyi&FC4FE#hWzlZ0jfM>IGINx2=JOX~xpy2a?%%JlI6f_?`=2-WW;yWJ-{?Tj+YdhPduR%!)h; zj%~r<*o3)Zl?x3<8D>8;UVN5E9WpJ)dHsGszfKuh5>D4=;?45F@N#98M6=s=QmMeF#$b(a= zjjav#cGWv3=$-38zCAsFzL4>tdu^OXjel3YBgdX-cFfHIooN|nYMLvZxcr^d+*4n? zklWoXM_=vY7f?lPY?hxe;Hg0@#U-{G&{-uNSn-2B(LD^ZZ%m)r%OFhJpBbtbbmN8>iD{0=s_@Nsj&(Lnv%Z@ygAoVp~U z+VogPN1tfmGxs^dmP3EXjr@rIGPB*RgTT(lbbYepJ(xSL!v{lqM z#amyTpLzAtX8yJ2_-b{AsdbU4_17p-YrI;?Zw7DOD}jgpjB^e`B)!+7X)Ef)cti5Q zo|5SmD0n%@-CUTC*&2fwzGYA{>Nt*_FgL$5h2w_Fc0V5vSl<@R1s!Lu01(@25OD_P zlC8``yuBKXHOhvm#p1OKAw&rOYXjQlzHC4+!5F(iy50!3Cl_X)^=I3jD_X zp4K;>L0I+0uhGQ}!NsV7yw20eYXUn%*w8um=!;LF$|$GP@y{VGw|_(Q!hU=N!43yM zvyy>9Kds(L2J!N210k{Xq7MpJvD#-jYI7=z_fnDB-V!9sTZZ#3Tuh_si@7@XfL?U_ zFO9GOT!>Y3;8*)X_HR91=Oax$M>zndwhlTjF3nlwfQ_h_v z3g?N0xUiTj!Wb(~fNakcZTLC>-v6&T9}1r(?5eT0&s2s5k=VyV4ytW75ljA zvK0G7&s>^@nt0;qZwyXHAlU&zJ*x(Ux)kbz&x|+UpPKAftfwPgh?b<0ZHOq__BNDl zYsa7ME#O?6*hgioS<~-s0|lV^N~)#5Abg^v!~{biHDSQeacf7eY@{ zqBo>Urh_utxb33&s)=+pr+?`r^+~@X$Vf1p)8LefX;0zArNDb|567 z5Xr95G%SH(D+p9ex*3vi!NmR&l52J|92B%9TjYM+yzPfRxKZQgZE_y0?M9q5VU*oP zBE1M#dO0V@*x1zeeG0=)1}WIs8BK(!6L^(BKJq$D!ByARkzYaI$SpKA`~o_j?w?l` z(}-_P!cs_(%{MjSBDK+tVch9vS!e?V9G3k6-71TBL-@PxFahVl#x0XQJ^&d}^fA21 zq!;=00+A>1=5%!UjeWeju|NJZEy@}E6|KoP;`{QSc|)H{#B!<^@@2V^z%E;;i^8U# z)T3=nKTo`%Wv=u~M#-WY+X2#}g7z5kyo^DjjzOZ1K~zWipfvIrl8e$g zLtMR{r9c5m6CS8h*w>Bll0)=X9-Ao!M;pkN!?V2)N(p{b_1Stdgr@%(gV8!c#5*Mk_M!} zL#i|&75=121JZC5c<#X%Q;GzH&NIcuWqFrbIP zqIIv%?5L^(8Ayg~F)%!YtYwr~8V^dw_ZE3hIyT9kj8io1(K=Isjm(aD=`g0N>}^nH zS_@kdygZ@Y?YK2@riKGciEre4;a!%-J{!;Q(dbRwfZfxGxc)hm!amj^xgnX$rwZtj z1=)J=Z2yfSQ-P#L7~(p10e+N4OD~izX=z-I#CBk?Bnbgcp*{EU_RLMz*qTvejJz>V z!HeGTK22k5jqmkN4w*%1Y0!EEP=Hw{(<1%+Tnmq$4d!?*DEX%g!sv<+b& z-&jvRMmz7MhZcIVdrrn*qxK9^`0gN;(`-#fnIdx;QqQk zqCM(RLfld~;+BXVDW-`VfW;FCmnbP~Buo=;$DxtSDcda8CKgrJK?I653y;>`kLAh` zmKK6vPbUy(*{`wO5Fe-ma;s}G5@ZZ>&SZ-aj)3|U*@^_`9FK_W9!kVF9|KXqzl}QXV7QSbb@dL zo(>_2(OX@dlA(VPEvqEc@1{v6nYk1o5B)10)Y_IP;q(5)LyUF$BIQ?L&TJ-t$ zfqsQOA2G`?eI=Yj59>vHP*J?Rdm%a{>2qO-4oUVw@Tu$h<-_Ig0cG-FEJA(i)?{ViltKOmUE9yMU}xRf%EKd><6&o{kB^y(zj@N zd6O9tBXG!K`QPJd@N=JV;ce{4ev|W*y`zQm4cGK(1RRufs20@j$|yXs4OLuMjW=}n%l796i~JT@K@yZaLD%|mB&cISuD8bA33Z6XM7CO#OC zgWoLx^|Eg`l|aTV(^*D5VcbOF3s>UHv~b%Q3eF?Zq@TkwoPFvSj3QuXtcQlr%58;@ zq~ns%ygq0_niqQu|H|2)s~_AKMa8d%XAdZu+32Uo^6D{j>{84Gg|!#*GdVV3D0$70 zGfF%c!j1QM)K75hn4a&x_))aqAu`mM@k@dqKH7~+z2Yc|%`e*+3&~{;C8nskxnQ5U z^hBK56s%WF>( zMbX<{z{0I%Eux5|tswculSH9&tO|*NSHFk}l6~>T7&^mBh5nG7xw2io_^W38)mV6S zSmdhzkY0_1S7Xmc(<={uH4I)27q5=MD-JsW?P<0!0q59KZ zp5e+*;pP84HMsl~c6FLse#+mM?=8BXqll)lM$tHH$IDx=9huz@pYW@H zUFF?LG|NMEo&?J4>4A_9d6jaMYmAarp|>7EdzE(9f(vMpqsOSibn11U&3HFjGhlbj zof9g!|c>3HS=u>&w zOP@(M1ktA{2X%`~r?B{8L$}o^N#)B-xYcnc)?Y|2j%)jd&j=x+f@1TipmW{$- zv#^34PUjcjg=8G8*oGjx7Y(c-H@7_oFEG~#a;B=bO26kyCsg;V z0x`x6=^7PdY!3F~ZrpKaV^`LjcvtH6-mVOwuAqUSoOh5V@vd}3!-Z@D6}uCs(mSBx zE0=_A_yA~lAmrGm-SH5@8`&DZhvmpq44yPd!#B3JYC4leno-&1 zNf1D`h(f0@ZFiNNWRs#VMp(OCX{y0eaL_7GuN0;t|Gvb>@_n=#<77D#Z#oWKs3*i6 z!mP=9?SGE>uV?v_i?@^1RRfJ3o~`vFY$^un!UXnzLPcE;iZSyrvod z#2%95|7)-KqwK?MG&+g*;9+R5lz)&_bmgPrX*yWTWOD~_=_yI{O3$c(f$%+@nA+|R z90<+GUzP^KYhC$3=nOp?vN#xI?+)zIg{E~14@>uG*RVq&g4V${K{vR4UQQRZMfaGYQ`d@poti5ah@!Z-L#6Kb&5*I*Da_I;ucfHBPdmDsPO33S2~v3!^2aCE|i#w4~dHQ^y!89C#8 zg_E^e$G;VDT;$J1_qKmK$k1cjcHgx^5G@=@D!IV3)Y9}vWjnSlJnia zcdZHbXA9cjrR2Mvo&Dmg1gEFvyVv#hPj@BudLjXo@#6(GVfP!Zq$Elozc@mA_NAGa zUbH+ydapy2-v6Zt>4~EJi1qXGk2}Ms!~WToJ!>X9HSNxKk7-pC$)A(U4+hr|V+dr& zaSd-r9P@Mv~7pYlJB9MR*=8&CpT3Vfv^q2;%+*|i$-mrD z73H|1HMOP*G6mjA<;>l*UO|6_sW2xRtIMP8Nft*T^}c8|?%+&Rc^++6sQXv?*5H94 z#WJT4+8Zw)HafnI&Dfu-nxCrObKhvLQ8*cfVQJ+ienA|G1|jG3a&MNbqn#95L4tBM zTZgmeRdd2$_E?%1=&0iqRI|)?UgCFDQlul}g4JYvfoKYetBZVdokUZO1^&ylx;pi= zge?D_gb#-sK$6ntCus6;;evYRH&T?4{XOs$L)grL4@Th!t>z@aWKoju9^5FLe&l)t59=O<+_-#@cxqb{g_ z^Jen?jlCSXe}3P3`!`^w=-+3T|F(bQ=cxU=RNZj@0{M5F%D<($u>4bfqknrE`|zz9 zU3mvIuTi|`eCn*OX``pqe!>G3mt>1AQQ|80if_hITU_e;Ma z%=hojXt@2q+5Gw+=KFHL^wsUcl<$}RvLQ_WZDHH*#cP34j)wEqZ1MN}=qV31WFC2@ z@=MJ!EqVMV?4pJGR@jsxXy_5VLgzjQm)YK5XjcFVT@`z?7Syn43es_$6>r2T{)#`A zidS9)R?G!)I9^3HvA1@5t2sxYD)NXbibMjXP5z~xFoNX5bL=jxCQ99yC65iiKS0~~ z%Na+ZZyy}45fQdQYg3_0ppUo%SnvzZjC0!(t_VVdUuHF%Y80xeidEL|n(*}(8)_7F zCm6gR=l161rgOY#Y(Aj{r1(ld2c=_2R;Z<)W3xo*UR$0(e3bzHoIcb5e;zH1%%A)d zKK^KH{P=Tf{q^!Esw^^ps;hkb`S-Q!<4>bQk@?f~L?w`RpB3FjY+vZkLs$Lyle_ME`SWmTWd8IPTYv1=wJX=h zpT`a~Fn+q@<}QFgn>Tp*v+RnGKejLa597yIy2_vY4PO4-apj-$=iU7c@Bm;ypTDP5 zNo4-~ysy?feo8O<@#pQ&ua`eNwnygA?0vP~@$*LQ?-$mxnQYbfwKR2%T^5i1Mdr`I zuDC*M_r*mZk{OKApe&&7Q<S5r|>f`f9|{R&-t@^Zv*_XY>v#It+)He&x!MX{Mq^O_44QBrpWvmdAo1? zJbnK9`1AWWk@@q+?IGi*&!=Ag7|#3nbMd3==TFY-%gwO;;w_E&@a87eU1qzmjy zJ~y4|NIokC>86Z5Rzk3|__xSC?DECHuq?j)QiWOi&MaL6mHvzAVrzAQ)%)~)UwY31 zk$xb2x^0R`|2%y9ACpA-%f9qbc^mn41M>F6#s=iAi%;I3IOivC_kVc3@-}N@1M=3z zCvT0;U7x%?{Z(XnYu6=2-oF0WD{rg*^vPTC|E^!&N#(*S?6 zHZ-tbnBm(me0SE5KVN)sz5Lm~zJdM1tr0YS@^N8r!eNeR#Z($0Luv@Wi9hBs`0se!jG4Jy`u0PjSU+JnZ@Bvuo?9DMhB`E&lW z2F7=L-~9XB@7KqlYdae_4@e1_fA4+Y%b%3ref)`e_y2Gn&^~1TJ+sEkpB?Uq{PEe> zLd|zg`;g`zR-+51AV2 zK5P=D&nS#^A7-NT?sFQr&+0fz+-FI(*TURqq5lU08XKirynh>+pkbOqE{RV8=@9r) za|vgiXmJz8I_$?qYyd@*@5Wx1c;-~P8J}lBg1vBri$VwQjX1M*t5MKwpx4VI69}ts z-fVc$3SK-TUOYCth7f8Vh60I8K5#Z4Jrk+FMS~*I^UO$0S00T-&%Ko1etsl+enjcV z?G4cLxsigNSv5iQg!cVR?K?g&Qv0@2`uK?LTfzG`FH-voDZPJ$_PG+fj;8V>)}Kh} z$EQT9{|uF{JRPb2y_CM))&PAzQ01-lwP1PU_iJ){;R0z-Mw4w6sc!q~g_}tlnP;LM z9HAmL=Isu>z8H?ygVgvnraRpXpwfY(mcX8zCTK$SE5704@JT#o_3>Zoh;%Snj@DVF2?nb)HyQ1q z%PBMi%vSyeGT!x}I$mp!$}fZY>X-2NyV9JQL3p(@ee{SU#|DQIYj2}^@HG=sGXrs@ z8Ua7^$sj7k9uDVqtK1L1wLR=mqV>ghLNX!Wp@wRA?{`${iP_|&?6jC z^pR83Mv-@48X5kiljr8Bbm=?C9<5Otl8&)v7c_Nz6>F46rE@IF?;LZRXci`B1~ zvpf{0*8Tco3^4E5lS75w0fj6j$OQRmoql~Qi)%2Y;n?4CVO&0je>sg>@ogh0`PAo`N$+q z`4`3`-&?P^rxM&8G(jHKbJ0m;g47I#n(B~$T+(Q^q0hZa&2}S+nkm~x`@KxyFF7i@ z`tsrkdYOYQZ(>34d3&sJ;gGfqp=_%>GaXfjoNxHdFrL6JRC`i#6m))5uLu*Z=U`pV z9uuwM1>9sXI_9SW*KTlJdBD~*8|$>iXCuo+bjHh!eO*tW0iS#5iztlC!K+QfosG3T%4Tm2 z#skyg1e^|&*zT`rX`MjTv>`k}f^q5h9lBEeMz0YdBu`q9ybZ&FA<-^V9h*n0;Y8Fr z(3ur2N6ChN+U0AYZVgPq-{)ZY^Rx`SpEi~cOreO>@?^3o`{^s{EaG9v7ANit>>_N#UBu|jJ?uZ#3OrJJIQEZJ`vs`AWI7~LAdP6M0{Kcsy$b(>{ZZJip6;r)9&Qu~}-J|gTt zsC^Hs^6|r&aQlaP$kHFOuuYIGd}+~b3<^)f*Edfg?=hdtU5IPCFJ8o;K>tbKepp~Y zxRMS5vFLQ>!?6SabPF!pO)Thh2TK%Mev|fx4it^}TNv9 z-wv-q{(bDe12x|5b*_qiB4Brfrv&s*04a|_nHn7?%;zesI)X|TmmhO5`?7*Z5x)PcVLG!`bnaAMi~ z?}dL1{1rGiA^xTQ>vbA=)G7LHCZK6P><(_VN|P$Q_~j~m_@ZlCjY0DzA_trGdEY@- zKff29kPGqTuVVO6_Qjn%x)TQdSw-ukU&9COR~x=Hni{?y;Q&f^_&2=lFmL$g!*yPe z09RoNh(xD)Le@gW@nV9f=CEQF)=sBc9XGtt7a!N4ZipadBOu|V4WlLzR2`~yXC6m9 zmcmN*cqw1!=3w+)3e%Cp5{5>e(!Y(U0aL|tNf4f8yBKn!+1qO;R z1@uR`*Crb`3Xem}d9=`ib8Jhc5Fqj{m2*=K z+;-cv5sKJn_R9E1|3X@g@>0G_EK&Xv3`llvJ2Eael{kl?A(won;gI`>H3X5g zMhTDH9g}KBNAaWfB#PI73BH%Mr(ncbrK{%B6DfHGm(imX6|eIwWf}@|AFCENVtqWr zLF@FiKzeza!hb701RF>6boF|HPZKaH5&YXwdYkAP9_%7D1iAXwkF^SC8Ao3B7Y(48 z#~l&>%wJwcqa&JGE}~K0jGftE$B~6peS&a@GTE5YoI|g?S$peFc8+cv5yD=siIwdKRjvr4JTI#bZ2S$Dhz9zC(qUyK&zDe+BYx$(m9aUW#p-`o7d9O;u@kgx&~@_ZT4S0hGoX*pgT{! z58o__hOc1YnzGK$cXN1Cj3vLZ6M|DMlahf!Ybh4``=x#`dhnqt9B*!>-IM+qUA>4S z_QGb76Fm69Kqg`K9i22uACgeyJB@sTH=pK!uxY^6Yo!OZ%z~5%6-_O zxy{s`JhF-RWCuU%ZH%O5QJV?+&(&)TH9qMMQ95f43A9*@j^JOI@7}%1AMXnNhicGw zFzH)R%_bpdm>i5>+_7-wJWu0#I2BP?Zv-+rh_w7?e7g4a6vx8Mp3Z9(x~|83Fiz3R3|>^E*1fct?@x<=Y> z#8P_qp^^3*zuryvXAei(Z|tIUefa%~dcU0Mi{uIO-vMIs1=9Zu*Mgp>Jxmvj^YIZy zy7VGm2im65v$%w2*6%6PD7tiZ?w>o@dkgJK%(#2wL#uBMEF|9GdA0{*xvphP(4ya!_@*^LnO{KJ9z_<;IZ%tsYo(DNYbB2* zb)bqVUkks>OmQj9&U}r5ocd#Y8_3MgK7gKCcXD^-??dC;`qPJd!74EF*FMYZLH)ua zk2m1&RMN7J+VliwJ@B_$B~PAX@Dw3_o-x6Dp`AaNM*cAMLYpE8U1o>Rb63x$=W10l zHAO{-)t%?Jj3(^=y}>kk13Ulp{#v@$ucj_H`MQqdYn`q|$4qNLcw0mrS`6JDg$;KU z88iq9mVE+W6>sN=Ze31`mK*YfK>7Grs9owuRtnaDcoV_u6#a)@=kp6U*76hwBD)U>jQ>BzegO{o%qe(p zl^UYT5E_B5n^yHBs-R2tu6tFhgev^_FeucX+tDzg8}vHO5@r`&gR)n@55%A`$(h3b z`lZ%~(-9v3A()=+`uWpyRaziDXV$9ptj^~2>^I7fo-g|7Is2Cg^h}&r`+uS5?vo+( zjAo(q{N!2yJ%?s{>1n?wfS!v-`qT5spfL2r;9>0JZ~f@`=E>`(=eK?R={fYaKzgRs zsPufdFQ@0(5q|Xi)JM--Pe!0;;h;)AgweWs7w)76 z;5D-@S4ua_W`o&r0!rTlyH&G+rZCvJg}ZO20xU8I93+?J8!WiVrHyVJ{**OR6oNd) zVJ=B4aI$cHawySue!WbvTLw0-s#d*T_R}wR^&$P>mPq==JCwdJl08YblhXej6G^|A zMCqRlk91yM$IDNRRR7PseG%*5O6fBq*1v+%yGN*>&QqW6quT!*@jFj-C|Y|vq2I~v zSl-8U@rWRwX4k}2(A8{loR}FU{`6c&^(i+;>d&i`UKGjxB->8uD@RA_&j?B%e_xHV zEbxB;{49!CKj)`nYM7txBi7IPd3;#I{8TzN%+I2T^>co%9Mv#C#}A9t|Jc4%e#H8J zA)jyrm+efTFkJ671X<$5c&Q|Te4u!X;6ZXz5>vg{CU$T^@_Uyl)Z|v<6;@F15 z?Vv@fW}$gv6fQznm#YX|g-5*4uEb4!>9FqzT_8(4*fS}7HG2?a@W_+M9B~_DwRGBv zKg*WWnXbLRHLLK5eOz8vy?r*>+u?iZ4Gh{vR|4&ge6cX_#ll33#eY$m$tZ`#;s&&@ z27>g(M(-P3<5Qx2_SD#?oESV%*iU~Fufd;9pI{k}j>KTF+5CO4&v~*Nld%Epo{`aE z;#U(MV)Tm0c!>V-#cSpT#25dg$3KWKPLL%#de(XU|M7WNFZJ19pE$_xA5@hBS1MD+ z8H=O8#vaGxr6UddP#Y@gMB^l5?fmo%R2bxz=1x1T+LN#$TtIL}i^|t%xG4ae8qL9( z61Zv+=jmty1q7oK?7538C1djVwjf?yJA}pLiXea3bXZRZLoclItuQcbP$;h63#hHt z?57-5|MPG|he0=-NA*j!bnv&GNA9X-zjOHfC3r}oEpz`u4NvL<<#09P-@Xzs3zopo zs;LA7+pK2qQW-|Q`52V&CP^B8I$r(Mmh3#bpoRDZ?e8i7+y}H}@ZM%Fs9`O((n-`2 zi$wNixoAe?N_sad!RsQ>+*?>1_^72Q3{%-{yttNFT+uC_7ND_7WuCU12`4ehDor}V z5hsst@m@+1h5cLx0__3&mk*l2d7pG|Ce9SE_D6I9b$&o*ndQ3;LYu+gsL6!jasF8r z*1m4|P-UM(iP1kxp=;%Kh)s*X3YvfxSE`)w3%-#f{-zTOW%+=)^z53K2EV6>vm5MWMP)qr+HtnvlX}JdB^5zYjalMaoO_0+!9aUyE)VkkZ~R2wj76}f z)WCTY*spZsFmDQ*){)-KWfvZ#A2ZpHc*rkrUWoa-Jwd-Q(l(1)OuuO!h$w?QfdH>*7|flWGSk^viZ3`{e%TS&VP6Eg^nrhP z?zkWRiS=VTajpN2k=BnqO5dIxY5nM$+0gpn>Xk_8D=iVvn^b(A8WfOegd=oRU;GNr zzGTB`mH&qnhRui_!K1g{3DfV)AvLsLk%)gd3?8AoMnY&mFp5xoEs}|DjPnR(>RJ$( z&c`>@C~5$Zqlh}>E$JpTvzI`NXHh?B~2aFb!~YyWwF&h&=QzMgB!MX2;{AH% zORdP<2#=_qxo}O%*b@+jJaB)?+w;Gz-u8?Q*Pb5Op3~Ow?V;0QU|t+GXpqjoMbE4W zZqd2(AuUq&>Zf0_5ow&Ma7tPhp zpg>FIIVpSv$w?M}4fxk04V;NqrQ2TA3L8`QpWFGN36Qe03hC8#QBe%q4eE<^cwKs= z>KbxC8S%YP9&Yq>=a#6yLmtn~;7Zo#B2S(?6Bn(lRpM6dO#+3H*cdw%1zZ#mcb+3|a~BNbBrwC>EWN%H{265L?@C;ZP1LTgr#sORl&KHedOY}!XwK_og}Dpw z;mMzOr7)A7KvCj)_2!1yf8NWN?N-p#a??SskpSg9Q7YxmbP|-?s}Z)*yO|q=P%<6F zd)-p?i$A(c*0yoc{8VC*Ri0iU%n%&qWdC{Y7;-BfBg#1uV%Ux51H|d# z4MfgU#)X58I!b>L(_f*Fcj_=?DD+|`@hhp5=*2MTBzog<(n5L)PnS?KdLpBEysccR z@StZ5uctpqdAcLrJS*r8HyQ8@#{8Jg7OQ5oktu@wGC$5DBiIGF!s6)R z_+o6qd-dWjEYokTpu_mz*k44vULWuYq~IR9jx1r%f#kXGggOKK2h(VJwI9J7G`8Vw zQg?Ld!$Mz^A6q5+a}`R2tp%lFgoJG?2zD%OEG{16M>ssOpm+fU?Wm&@v6;Gjck-8Y z;qN!EC%!;{eWthyWZlkOQ>vq5P~_(sfEMjI$G*1!$DUas%%@zvTJhH!$jzp%@Spvr ze;TL(`|n=}Q4t}L?FkgFOr{yU8kPa{Z`{BgZGl{~x2NC~cYZBRarPFTNf@r0-W}ub zY{SvI?uq}<*%mF&T{B8d&QtnVOG>L^q-fkU^s`Fyba<|7w#Wl@c;~8yWAn74{q)*8 z+J%vEaqmB_1^K2S>c2EVCw31|j(wvVLe2C2-`J-I-#60|5*FT^^P7X4-{7qN|CoF8 z_$G_4VL0hRQ=p`>Sx}%VJ1SJIf>3CrfkayYu?R>7tspL_KmtW66w(S8AyO3-6%=>e z@m8QNK+Dn=Z-t5oDk6&u6N2ot(CzxpGFNgXO?;l;`@Mg>f3!(v&g?U1&N*|IJoA9r zrg<%Gx#hO9!?CvLxX=<){Pi{CAVFdMrE`5PtB3cDGA}Q+xicz~LhMEPO{t>|-^k1> zwrS76@nX{}jjHiysjQ{F7hgrO2vBVwMY-xO-jh%yw_2QdMY>cEmH%>3gC=7R$xn=w zuk#1rL|zpV-{w4CsqXnt;x=lKvYEm<18YK^&=QUsq~mDR!gE#z>`N$qSGQNB0e_xJ z^x7aXzqxR|fl#?iq`Rr5YvL0T6W?l~P8`r(O|0y7MEjtQR#uH_5NF)P#UthA$ElY6E~w3n&(YHmYO z^-6{lYyPslOHxo1gVje{FsdZi@^D^nVU)J+d7hJ!mo+2Yq#!I@lT6O`EWBe? z&T8?|=M5^kIzf%Ue3Mx9605RQH1T(wsa*7(Xwv?7)CIIp*@B#kxbmZh0D(gOnEPic zPnslOhPT*AB)i4ipKIhtYr88y(cqk6S8@{JVsobYCTN5h+F82M2CsZYqhzj#L?4DD zI=V`RqQl#;7b_q_nbu;!ixL=>4~r=HSEKksLbsr&FpD}YkppzD`mjL_C!h+}$C(9C z!SOU`?lawktA}@pao%pJj*E6=;C2@GzQ_&T!0}dSi$ibU=1$?15BLvFc`8TFP7=Ld zn%|sFAe1u0sSedlb!2TRgI%)A>Z!1yx6Ey!g&Mz zvjffO+Lmk`_*hrd+-=$-5({z{H$U(qtpDFDuv>$=ihtfm{e@#8Z1w*4ktYc{pJ@78 zZ7{K2=*IVvpzxpfApOb=d|S;bmVr?&Kld+d5ZND9s#%OUJ1niR8HPN1XLHX7;;0!L>aa+;O472?>H|w-x z=@{1Qg-#^{mu>UhtD2W#Xg}X%xX+P?M9J3(RgPv@P{^NRuE?>`RwiOSnMRcZwJN)8 zx%*(<2UOXVvvZ@AAF!zX2RJFg&9p*=^-@&bUS8^nDZ-t{EA-5LXg}E**1_9=p(I`T zRR(8A{_eJ}VS8^%H5dLICl#2LG9b7z-Mw5Fqs&XU0dGZH8P1g5V@*aCbGSE>#UrQK z;Xjf(yNcNw6fRw>pCj){bn(5#ppGf1U|-$j{`|9;40P@8drTBE zqg&P_=bHl)1=l)L5mo6g z$VidMczb(Nx5p)+=6PmrULqc{tYt`cT;zxam@r7|>IwsE?@~YlQM63+V^xBtDEMC3 znk#qS%K>{ypcj(~t17#TQ#Mb-dtb!L0r`6@3C|cgyJ*KkeI6+LTzb=JIs%Jroak~ zT;ZKiVc>wug4SozzB$q0FmrL?>!Imq``h{KO@BL6{OxS9dnO(RqT@7PyS+)Lumv5q zfk{r6uO~hIpk1kIOB7GOiO)^L$<0iPDxu6p2^e)raC=if$2Gj0q+W30gB=#k%BiM3R)6|$MB2X zSuw4c$mjG-&$P94ssWd~o;F40-{+-x9@nAeBUz}K3yV=EhIAom_`mf$ZA9ml-aHa9 z9wL!ewQxT#H|YJ*NZd?Yn~8*cpl0*ZZ|P>chA6Qtx1(z!broevgo5UF?J34Am6a?e zoTJ01l#fgB!csgLPnw)#7z2@Dva85^0V$n)8eT07bpkqmShS36wy#C|Uteo6_MA%Q z5`KdZA}kw4)pN2{e7+>KgMInd&3LEC?%JO0ww9=+IFx}yP3EP8iG8xJ1@;O#5IDAO z*t*l!veJ~VxrJh&z%s7QbWcW8dXW#6>F`@7i2|c-xp$asWk+Ix6*r+Tqf!LD%gqS2 zA3As|9m0UqNgrBRi(1<6wCPfL(h9Bf6U&6{1L>5qrW)!*P~Eqo`L6#PTs>24_rPw!{C zllBg6Lpfa?mY}LI-C=j0VW$0tCIdXbICy}`a)6I9{&rj=SM4!{SM4orIOf8k*y3;! zbXEsaufx<7Hly%GBi6nD1euvsiK#Hd(l8RSYp`JiK@2;GbOiKgjcb4bX^D(`GKB$xLr~fzNmFnd)W2COkIUsn1^dQq)Itvw#Y(#WY zjw0DEs6GM^Fw^JXCtNMAU5)R+Gfg%QYM3qH-ZjM#djK`D^{l_DE*tqC_N3jctLpvt z`6@lKifqAz-`SX6|J^S?vSG|Wj4&crI0#=61GbR7V*zv+5I#QYL zsYLF0Qa|k+V-N#wriirzIq5>ANfC+MuRNZK_6zn(nuA5Tq zhe3^zo(S`2D#5HXwl(-n#Q^GSw$6whlh&Wj7s-1VLr)VefamKxorkd|B3{v zDEQxe7yOhg(jjpmk(@p(UPezbhhYACT_!JWHnjwNG?7*eY5<~Y8;L2Z%|&04P$|LP ziR@0=8c5yfRXm<0q)V{%&3e`Q2@zZ>&!pD5qFkh|v!U>vL4{wZ!uY5lilDIFeRo$A z%);`{{$NvL+FMZRIP@aa-{@=;EWz@wVXo%VmHJATQl+c0(tg;ZE13ke7OLhI6g75v znWX@*OaeY{;A@B>xzf=OXXZI0V~zo3{DJ~WyP`R1*adtksX)ofV<<#^NT9VSP1^G; z*MTgS6^d^Q8#TSI9U^B0^KPRU!QiZj0gTHLzc>a|AdhT7rCZ-4&`p~2LbwW5@9lIM zC0qfc)5i7+jSUeQkPn^&gD#7egwe1`40xlS-Zw^-5^@n=Lg(?QeRctb-o92O6-BGT zrxP=k8#On20SpsPU$9dW>Azm8ji7JQa5?F}UKvT>;vnTeaeX9xi%pc@w`W*=MNPiM z?zn1%NzU-Ke}STsDd8Ttrs~+hY8iJn<(ENm(V% zE!1o{4Rkt?HJ$I$-EJ)3lJNs~xa1X_hdX*xM>yB5*Ha+5JL$O*64zp4!xxk|4slK? z`cxsu%119wWy&7kRXY5# z8odl+ej(?3zB1CrBQ3J_iP#M%!n9~jVy5kxjGyPa@KLYYcNJ`cvW~237_{Y(ZWwf0y%x-js7XUR-5JtQKYkAF3hC^{HJ#ht&bF!nG3yyj#8)LEPmFH zk?v1KEDSNqMdDWub0Jc1o3k|}D1K?d7;ra@A6-MuH^CHN-BO~#4<#CWW4t`#uSo6? z(mdQ{$1NmM&IrUmUC^k?r5e9Tj^H|tmQ%%;1-xcI;EziPiw4!BINl^uVJjB+-y>j9 zA6^cFI{F{d+be92G6UWAKv_%sc#WYp`k#f5K7$SdNMv%Xfc*iy>21#_GOk0Im1~A8 z*K#%B$kR*?dAi*I4F|th$^CqxK=PyP@L(AA&DjBs!*Fxq+u$S~u*V_89D z66E<(Y$&kttgixyuMqEu_pg#0H-*7+u<7Rf{bs5zHAz?NQ)_r%MN`8?m1NwUF!w4P zV7zT$R+YJ^5&5MB+i~ncl&8O!)$IcHJ@N+i@(b^OqO_5WHMxO=Np#G1U5qgotw+vg zLyHXahM`9IcSjU_iN;u)o_1eRJ)3YWn+b=OiYrz&s9q~oH3phm?aw3q#+|k3o9pdB z^SC+JzD^@5x*kdDYoTa)5tf2RqstDPphaVRS*9oTd7v3HJs)qw3S$i%%kmU5&1+w3 zP&2s~G^2zRPYTNmb&%Zc4l&A zF|I5^e@-Yr^%Q)^sZqttGwO)NO4;*em*|D0hPDsL3GZ?ST#kk21vl}bzwiVo?0v+a ztDWq-=ww?^^J_yJJYjgcqwsV`qjy_>@lII04>xr3GA%Vu_6Wa!y`xt3#xul5Kg3_t zajE(EjyA?`h2uL!s}u>A9n*D(aCCJ1R)5Zi({<--bgVAtx#_y2GdfO}Gkd!3RE&<- z<>XA)ZTQhGbU8z(>-O^K1YJ(A>AGz?+N{elPsbgZnims8tMKHVb*k?&NT}(qb>w4}!`!XAo&t=A6zy2IZM@OLl#Jp_M6@V8y}-WZO4wJY`Frab9uQ_@-6^CXqB zh$V6X`6!yAdHiZwjw!RGajjb)z5a1OC*l? zPn51>d}`n!>NDLB<0kK6jv+rsbKd6*@f1^(9cA(ZGWifEtF1Ws4w<~0lXpqzn`H79 zPX1aZcaq7Ua`JmJxtUCUos(C}J5r_4RUvv5G`X!>_$DLX96cO-#P$&U{y8xKfl@)`j@CJMeD z&?2geV&Dgu20dxDr9HB^Q;ya8^^=oHs{(d(*Jzg3oK&`oqCaa%H?vC=)4#1k$x2dm zH9ud^Vtb%p2pR1fLgEXrkICk$1l-AL@NDkDo&3{tKsN7Ss1JgnxPDTH{S={aAe#sB z(Go`NBnjV{ixh(f<0p?~KE2WPM%NOac}6^j&!VxHdxXiRa0K8SYp;{`vcHpU#Gn6B@F%oPny%Hn<$iA& zm?%>IFR8onmL9sC;vk4gYygC71vgntdM&euO>@#-vg=tXL} zU$>F0%yFKPuZ)QX>2;=D?2tV$1m;D`XN>q+>rMPO)?X=uM|{6>i`>p{vofwNS) zj1J}(ErJf_2l}~q9ZbB1KaCIQ%cE=f4bC8%HC5;_nLhJVbX&dMG|ccKT7M zm+mOyv{6Ej2f9Wo4QiTB6=K0@l7X;R8X{rD0-O8MNjR1P1lyBOF(W?2h*lFTX?Hs5(gvyoAFhM2SANmS}3`eDd)PfGUpw`e4M>ib(dM~Y4jy8hlFJ9&-#yy6JE zvvi>N?>_F8;}OD9A>B#g?l@P$>P?A99P&>v4ngn1zS|+!7M%MWGftc zs8rNm@FaAI*9%0sE>gno;;E;pJ9aB$qt~IYP40w$(N7L5?{}kL<@WK&TR8>N;a|Rh zb2$_F#2OP`Y-h#37wP;4MDyX}m9uYX%;0prxFj*DDqblGuz4gmQMbG zu8z0A$e4(fnE9z^!4pjNEJfu^ZxJKVL{pf{0Db?6H{EbSKL8;A#05_3)0B@&)xZWi z%$m0QR^avRXoU)M{II=!8Pdd0C&OIao8F0J$Bu(BcX(wId z8>F9adULkJdLT$VfnZpS8vI#%1^sM^l_sOg`5xa)yK*SY18rE`H+3ZmExEgfe$IS> zD4IfihOxT_Ls#n@R-K8-xyMK?7zK;wh4|!%9?5Vvrs&!yTZc*y_1#{TJ+5kb{Gx%O@9ve!F zq30{%T1>dfw=MjB(Y@dO*RNu0Q+A8*J4Tb)>6(fs0O-0Ow^pNLjj5RM@!o+$4!J_2ht5^v; zf?%DGRap1pjqOFOfF_%XhIAgcv8>4LHj}EfquHn4ArAWp-sx+vKq$-uI8OjkL(8YUE8b$52 zpjV>UAe{6#(;7JGtrigqxjLFw^fFwT7)f%R#fnZYsts^lV2GQr z-?;fg@ImN!McDDSZSNm1+TZ{N&rpuKv-W0%w(UiIgoy7o#BUIfp=5tYuMBmXr(vL< zE?eTajGO{IMidj_@Xcs0`i|C?JOGGGFO!d0qAgI1g+2M5`&HtOmPV<_J=Fwsl@bm3 z0+DXDJNr8n%|G3P)FZqb_&#_uI_T=2PIKQDFXQKnXzt%N`sq9%*L``KOJpg;TB>Qz zY&6KXZ)Yiz8QZ(S-MEtF(HO6+j$9a%sa7UG9QF)u*+DIL!IqP_<(5Vrg;^`2o~b8i zy7X4q?#o)D6!L$mRsE(`1^EqOb^|oEDXJ;K(G6Srq!19yN6-uE*yqRyJP@o;8BAYd zxRmV%bW6M)(e?$q(x_zZRrZT_21)YnJ0=Bet~{H8*?LRCpswlWB4iQlfH{e`d8R>g z&&O|bn6~F3Ig_XSvkSZ2qTXYJ#imLe?NJuPqKUVc$oH?QjZ9^(=2FV-$l)iVZpWfgQPnm=(>SAz1BC zk%LsW!Ym}JDE>?^8T}X`OF1kK0T@qr;n$1|zh}qLvlR9KeCI-oJzUA`KOR)O(VA4???-8G?H1Hu(LVPgn>$3Xz&U7-S}a zpjM?Au#g6GbTR8d19FR zW@ek9`-blrcXkzDG9h^RCgdA~?}e-4w0j`~XaYIE|GWDjCeWV@*jAS>YT&S9@>%tb_%82?>L$o3I2;wa+PRy?l4r=^w z@b@9duLms6rw9ft7PUR0Ww&sWRJy>zuW2m2m_8;e92>gb*r z8#VW$I54|GbB2!LGGX8zXC*^$>G50|by&8sTKG^?hH?#hYik6q74M|k$t-tCkTmam zKh~=Bq94-`-IO%kkF}nkryY`{Ch}Olzlf|V7BJhCeIoIq#k$0fr(j&Hy+o>h<>KYeEC9IdRZ>pIDXqSr zJ>q{R?cwaGj8VYXD9N)^o7Vs41+E{rh1l_mstL0{xS!d&9pr?xsi%oB{4rp;1bjYn zmDzH- z;OK+WuI}h<718bdMAh#_(33ifu9J4PFRQB9aereI-+(k9q}MmU9~%|9(l1FJXnxHmQ*3s~RgC z-cxhl5r<9WsE4J034MN2xO`9Sw2LMfDnKZ+`!(?=0f{D?at~~T;mI~-L{f%g4)$mG zKj=s1cxPRx)7ih}c-vpl7~SCWVNrJ3SU3ix#Hfk<1mm=LhD9mXy%@KkSdw`wCX%cS zFQ`MXlxHtR8}RIU61M*pZ4mp%pjrD(6adWq0a0cElyV%huDuDf99>C)vIS%Sc_L~O z$ol|pk>Bnkws6Kvx>u!e{w+29H5K}6Q=IEW^nH?CdOcPi=0jrlB6*BIiLjX(tI!gRIr^^MXyDx^HhGS|M))Ei(<2jVywnJp(|@GE{v7xmW|?YOTG-B z1q@d(5U9HqF(h^&pY7;>v_Yyoyx)xDIVjFzn!^#+`Ef0HtWx~~k`LEF@@+D?mp{25Lige1qO&yA&KP(CT?PG$ zU2`^*U7JX{z1!^Q7c@Un>ZjXDF0+R=mk&_YSV-9_4Bw*2#(B5*5p0swqRE7} zvKnS2H&)I_7Im{(rn{fpy2_#qi?O?N$X4iFd^EBkSQ%_bsBhg)76Dxm&6YZHX)leJ z);j&jFo(3S+o5SAFi82h%kY7xa3)zJ@$uPMi}HiXm8^Dn z%F9s)*jvdvI}tePil?Me^AK7!3@I|Lgo%E#iN&AJ=Av{QZYp%$;z|4BJdF8?7&+#- zETG`_VR5?4mJ0C&WDAOW#pt<#O+Fj9K0W+R`>+4ukB(7cU5s3VS+oXQOxha2d=*sU zd|_wIh0RH$2o5%1@;k;!`-7I~lNl`wVVaB0g?(aqmT*r!2P7q+vg8ZV2Hyy6$1xfO zB2etOMZ@tUxMZDl|+J23r=-ByD_8>izLd?>r^> z`>`+^Kk$3+$R-s%MeiR_k@w9gHkkpiUQNJIty^52$?bVxquNrZo-q+uU0jFSX9?%d zhL-v-o{um2Mf4|KZUN;!0Cjq+nb+T_%YBA&F@%qJW@dz`{iB|yHunNe?O*kQsU0mR z#oK|0oTICGfx)i~er=?h6|Om+Yj*2uz63>l-BD88WT1!$#4j`INk(+$-C9*{WvNS) z@bj5KiG@_6y{5joNtg8mWua|??@5%AoeAywU)av{8QAsqS?*!b@nKrOf2gB=k2+8N z{;@95Z;Q5U$C#tL<4ovWQ+%q9VXJ+LtF1jp)y~ybdmP}&^ATiUJRPVqgQ@^!K$^cC zfQ$PUKj{8?I&}1pl78m&un#*XZgDZU*j(3Q-i*M0mwVlL%0vQ|giE!mhQ-gEC#3N0 zN#31jFUl;pvqb!Ko+QLT6s6wEBAOp(^D+4_8~tvi-o&ZXj7%V{$VS1hyq>0In=W?=<<2De=aYd^dOPU-E#WCTVZ!&L zosr(B)>Hl=UxfU$6_l?wM95EdQvQ~j2>A)lf2jV-NbP4+{=|swr&9j>i0!wd{2>wB zFXH+mwm+HscRW)2R?1iFBema+^0!n+YJUaSUlFN&o}Y;k+voY2AF+L&pCJ+3e~0Uj z*gntCA#bGid4ANoNbU3dY`G9N-|xR_pG~PFx4WtLpTJX^cJt5CBrU%rGVB`P1$TeV zD!dp{*ZfArhgjLYPPMqoqb%;2u4@y48y!NWWBcNVIRnpgrxVe{EFA(a539RiZ zhU}}QQc3J6X^+rfG@sU>rXkzEoqAFSVdvc0m0q;v(DB<^ zeovVX130q}n1vWy9fuom_a`flI3sgD6sJ2-Z3igk&e^N{n(4Xj%Y8@>d&SP-6wurx z3^*9qMPn9VEl=!aG`qt~wW8;I(FQ071y*VWc8`#w$Lj#l9omzVf0>h_?MQy(#TbZO z{qcAxAQ;2Yh`RC%gW zgL+~e!JJ0{=TE`5?#EiloSvm8+A*Z(Jg-7&0A_4SAPwftX>Yx&P({yW34 zt{d@Q82?JG$}k3OrS6(KiX(|$`k>3-+$pUIqmg_G4cHjLKV7IkNU*=l{dVPoO#!yH zG^^473iEgrzYt{JDVN~=C^o@GnP`3~sl?>0#W`Mr5F{U9^+=}1(F4NipCp@nfuG>S z{3N*?sINyRdlZV}=h3o_Mxsj6iUHY+b2ak)?wIy|M_Q~OX|NwN>_ z^E{h7ntg402N_AYF6~1}!{n4We?sREF*!WW}6vDoG&$=OU&B$`$5ea6GK?LMBK0uJX+bmQR`&qraj|#h2xNT$-*< z^1?QlR~LmZ6e*mPjY*GEl3h87zBZii+KHt)B|4({j0|D+oNpo~ldp&Wv@`_3%|$cO z#?w+PnVReqtbicR^M>8Urnc_s+M8lJd6>cR4jU~5pAZjF4Tc`{A4z| zJvH8vjfI@iOfBVS>NJ!4NAQOg5}z`wQnPWZ5jl6rrM@Nz-)4GJXYOMOS$=8AdXqdn zI~fCL;dUTWZ`K_YQlP}?}% zc=5J6)vx{!aL&cz$8l`+?n(6`!uS*4lgV@EXneIW7ooDAceB9|Vh^S@bgyhE)6?y$ z9}q7e5h=IfBlOdxNIUuVM)oQ&8OJ&rfF0Q=lh>>@sx!!s>fcq+^d*ZN>+j;(+x@2a zKkiax|LoYm4D26on)usUK>r2DW}QWPhU&QK!Uz1`+k;cPhfI-b%- z@Vk&BfUeK7P}dW&`VZ_ZX&M#iFF+RylE@;kJ<#9Z?f(8=Lzb%76VtB>F6k+TXG%Qv02_e-Ybnp!z39Z2utTcZt}3(VtX*#P%ms{+2@#>0`=2 zbS5HwO!==zqVE${@c6%pNFVd~BetJS`4c0ypGx^%!nd!-H#U~nuYo#^P)4*oQyYk# z{4fh>#r|Ou3*E(f)KxZ*4IL}5h0m7ggbhBggioMMZApm0Y#wxRgg4SZA?-_*xW9%; ze+Y2t23~OJ&kaGWoV-a_OuARx8G!B!_pQoe;IcWwa z9pR*0PP!eFc5%`JoYVu8zNRE|(N?AsNMzZm8R>71^DZ^8E~wkNR+y$yVutT2PFW|a zAET5Td{a54RQ$;)$-Xh1QYrROiep$X{zHvBn9(Aubi#Rp%qOi?zJ9PPd`CjBns4!{ zS$jX}+uZ+`i4`?%Ua59#3ud%G(iLaLQ1sg-9=<<<2&eowF$BZ@APGX{@qh&3<_GFk z-_mgQ*Zuj)b}#IiN*f93*>?oqCJ)NnWDN$t%<$Y61#K++gW5>@FEc*J@HVjxLTR0I zjsHeDCW=OrRN=MV(e(4h4s@GTqG~5z;pSzi`$GPu0jdkv>HSvLiOq>~1F^`x#836e zkS-J`maS)M6yV6AyQA5&grsHvKzw{0rAJVWSU<2%m2XG4XEdskE?-uOIQbY}s4*Ro z?8V0bv2!TXN?+)EI1l}AL?5M&jnq1ZO5gWF&90|g8VI1_2m$of1KdggNepm-0b1z+ zI?%{QlfNt^QjQ*pW{vVQ;v}q>>**`G>~CCl&rV(0gSTSX`+)}Mt=Q4|H(^iaUyser z?@3pi^C~PqAIpEk<)7D=Uwv!9o=i`^go`($J<|g>h8pH}MMVYwBGg36azA83a;Eb< zRBmJEz|+Y8(QUj^=>dCq(!btu8yc?6PfXc~J8XHGbU#uqmS&NcxGqwMI_);T8!HL8 zTIA$o_MEm9b+j0M*eiyU14Mp`4bS?_6WpTTbxJw!pC)`!_!sX&$M4Lo zJ0KZoMKb9mWHqfyRL@ma%Qvd#18^!%j3#+wx08x6X;M1VC91gQ zARH7s;>3riu%<5z@jDppi@!wcwYT}~U~DKcavYo;JfOpALE4?3$Op7C4R63esLm$# zAODm8uYdV%b2oFT(T+A4ck(JNybwp2)FKcy$1>u_*{T-DGravf;p($6V;LShT=22K zZpF1G!luUof1x~`PUaYMkUrIz(HwUd5%r@}@R{q5N$5ZyciXkL+{F4MigF!F9q}$+0=^f4;``B-B(&5X`Hf05? zFV%==2Qaxa7QN-<7i4B2ihFUqv>~ezX#*tv zqdok)W6vJomIqpYb|$iSt|$FRS=9d-72f_qix;?G*!y!QA0tzLRs7Ol-dsrd=$&t} zeH#ff+6~bBRgu&9SklY@dhFm>wxAs<#ijX!8#S@U!v5U#Os3b z3Tk(3FkS?qappxu45_+%u7V(3pcx*Wzt5rm;*@5tA*R{7Kr}(}6Qu9`spKIR7z=h{xzi(!_judZZzlEqOcJ-Y&XgbTG-HOgWC=qc=HhN zi~)OWD0>qcWakdXS?gs}E5sYa8FycW4@bFeu&M#1JL`& zFvxo%;^5Y=4jAZn!*IORwTBo|vxE-k{f!C<9-Na1|b3 zLt2r;u^XM=B8{q#_k?L2L!tVvgxqDpxpOJ^F>GhVorLSDM-l8r^LW*4%BD$ty}qzl zyV$GnM*k*7UXGoNGG5U7C&}97T&A*VYS8AS{bE;slPcRsQ{)r*ldw2cJKpWzd?sZB zDuVf;GbN&SG{WzLE@!=$PLb^NS&n0{j`pF%BJ3VcI- zST_w9Z}zqQNVq9U67xb6^6*c>l%Qx~s_^!e1CJg4qeiM!IVJ&wo5&$pvKxPhZekF) z5?5?>X1!{2ZNe!wI6I*~RkWSj$g+ceqxmV$1QV5V0O~)s$6vp*HIqVlI#x`aIuP}* z{(n$DZdZP?Db5m`l3yzN_LYl$gVFDB@@p;4pnH&>ri&NF{tx@|+IiC4fTqQ}zsOfz zvr&2`AnIshs;uiGEq=~AQf?>bFSNqHM+JOTpfEs1b@?@tuCcB~%>Z_q2!GinHc!?{ zaqULApKRzPqt`(UAHqArNIPPOy{sV_$MN}4xg#jTAMPVkPN_f_Bt0@TA210c0$Q~j z3@mPXA4_UaQ`6A4NW6PJKTV}Ez4PkD^4fdPFGN4Q8S%Y`@<;55_})YLPwbEQ z-b49)e~3h%tlv-dM{Iuutx*Af3;DT5n>%epV!VL@j6x65DoC8{-N=T1u{Tjxsl$fSw744@)vu)unX@lU z$I0)S;+L_s{mPd&wP<;UW}^pN`3)8H}0o_oFjGae4!i2hM#yKz1d#QiMq=VeyiV8Q)y43pD^*c z6)=?pI`~Ke4``q$=tlz@mp}tL8%qPi$ae)|{k1e8SpKwI6KFuE#e!b!A1L1U5=M9| zbrch=K~y5#CQ-nQ_*w&IeKm$byI!EUY_cntnuP>poMOwek?O+Ecj&NWi_!hV@sZBi@3ZmP4wv&G|@^7 zO|-CiNRA`yeQ2&b0>rKsG}l73>&Y|?G!^fA0mEe;a4g+IQr7W0=7eh);Zfd z8A`tbTZ|E(271=1sNGFc`?w1-vii;8M;Qa9C>lv3XAS?$TejcLibnosMP|MrH~3FknR#1uw(5FAPt0 zeF(i~?JoUQ_ZPQw{mzyY#Q^;}Wx=lGb~bn8c6K5Yd>y_HASDNDKT_oW)u<}bc!^x% zz-G>2zFs&KB?-ykAGJ|6t1zg6Kn&VXnpyp`f%u$jar|X>kI|N=N?V?*cmb+lc^-WQ zj`dcpx{VbE3|Qj()DHkF;4lby1)uNAv1(T3s1$inMlfuEWZzJ@8R&r* zCjPtMF(}{;bd!db0MLuBX0C2fj3dvo!8j<^Tr`I#3|i}N&%Gc7EEYZ#mzr0Uc_zfG zUKwh_v_pTGrty7^8WH!T10lJ>AbngMa3+ZCYgp#_5V?*cJ%BxoJ{@E1N@v20zA4lj z!W@I59VYAvs&AwmuvC{PI?TRdU!kKe{^=ZmZ>`&P9k7&4$F>g)0la=?$^rYp&JI#m z=-suExTb{@Sh}G$iJjTx{pEiXU*%uov4-f|`uQ7_L;V2UiRpWt4p5S`@c8j+$(tDc zIzZ{D>`|$g2&4R({l8wraT45Azw6?LV+O4mx77 z6rdTPx$qKzlu&uMv(ncV?gz|?H#oMIH0RVRyKy@Pbho4TK*hQa-#T#&8DwaQ2iU@b zkN{J(MqK^@pQTCCs1nxyD(o+rK3*py^X%!t>)%P9^*vWfUDFP7_xVPk!mTHv$F@*b zgILwGX~;Cxk!ll$SB7y8se+y@*-=Zc(D=wP><2c>nYBY|^z;wb==pAI(Aym|I{05K zPpc;$mz|)#<@-1U-;(XPUD=OhihD3rwR$V6E9^t1juP_RPT|L&qenCJ^B~@g)~pkC z&MLI{y{oH;{{J?je3J4%jmSS(cT;}E^4%!^iSXq^>=Dq_wRrF~W~c$B*n6rf2KjVv z!5I*}lgSi0%Xk4JF<+T1HD6H^7RD@EPy#8BBtk*kN+w-M8Oug{eR8QSI;SMlJ$W64 zNd2{oT=(9Tdl=R(OR0yw<>+QpezBWB{T>5bR@!RL*%rWlO*EJnk;|X~!dyfT>(b2j z%^u|a@mcI-g|#w= zYEAi5>(3DE%6iOJ$}AWkI%Qxp=h%`I+NpB@OB8MAujml~UF4u|F2^lav+)suV-*wVy_h!f- zA<)6LAstK(bkNy@I_D(8+qE$m!?v%J14KiGPs^4zG;ySsXok6t}zkLE<->IGGP-)t8Tff$G5D{E zOPit%qb>J9M`yhkZLo~C+$9>?pnlu-WU|U-`tX?ec@0o@E4U4-q3o~;+cL5rb*QJ) z&P~y@zf0THk$RseZOD6s21RG+CKig4g`%}W0R6m#2C$mb3>G87Vs&7T2EeK$SThM0 zrvu9ffW0rlOcE?!2i7S7wormKmtZY)VAZYtVE0O}mJ%#M2evx^c9R5aCBe)(u(b?^ zGyZdwLBvbYL?wK(SeN#fPF2&+Dfo?IADna zUNoG%hyd}|?&&Dq6CQ?jrBEsuIu@9IyL+N(bGpQXn=0_3U#U~;EU#E7;5Tg^q^O3!%36->FICY4>?k)@pUzC=Xg80f@P zrX}XWo#w&?l#)&nVBCg2QRZhUT--{VvAp2Y_2$CI;TKh1Fi(Y_6VU(|_0L&Fe4feT znrAXNTC*5>)@b&me1?$od@Vy2IWhy49qqaIBvXx+fn$=*%TN;zxj&Sx+ugI0C|3Nm z9o%&l61~40xQm(YDJFc6BWJCf(Z82a%A*Wlry8WW*bmI`6yzEE^6g_Om><=bZy&p> zBAUuzckU{uX?3>W+*hWgm&=Jpadl>|i};mny$c3kZ_jPpmGxx@Y|YQIt+X2}?7j9& zjlduKq2$bYvv1(nmmxDUlDgxwV?k#hi#rZYR{9v{wSy_eg))|tyq-!q;%FX_?eHa0 zjHepZiaXsGGbziQ@pF?nZj>=#tMhMLZk6oBolu?cI~>Q0a^fxJOQf`_gYL|0E!%n^4_|SRsoSP;L=pUF z<#=EGu2$zyNy;~Z-}BHOS97N1N&S5zU5@XrVq4h#NC1D9fVbE!egWBMN-c-i_l?EwU|TEhr~2?Ymcj$JzQ0zLg*KvAD&?@HW>>V- z%tFt;NJ?{6E%a5jPmpTZ{s-Vl_G3sN+kkkayIXS{CtZSK(8|ognx&&Kwv!aVz4GbO zgC1cgmk@$FDQ;^gMIa)}Gx2x;k{M)De%e(+ zJM7NSNcpL%B%u5>xL>_~5YB;_+D4Oz%tY$$+_)ep-u}3S5EO z#wWLi(OR><*HK~Ni%_l|1sh_}^}014fFeRm5Cay5{{EgY+j(MSCZg4V_j9%k7+=Z; z103Qlwf6^{JX9wCmy=_^;N)u|nFB>5Ud4z;S?553r^sH+u7C(5pfxC2 z=VA=KJZSkVRuVlWE^Xm<*iCHe{g@Xk^A5_qJ-k0VeIA&M*#_;w)w=`RLFY!AmVUzd z{{#snRuJq2oYU^8Gi@%cVz}>u;a)QMm5vWw8_s>ozzwvgzW#*vRN98Xp8D{*dNreo z(_raHYF4=}Wt>~7&UA{b#uXf_s`JhLf^UGG`h zRhKDl{VvUvekB#$D$5^9U1}%H&I3wPiBHa z0noDYm^g!X0=q8Szb=B&NbQ2$0X)X`N z=C2Dqs<@wgy=@hiCa|NMz|!-{U@I)jOgg6kmkV*POhC>^~PqdXKkK{*ZSf z$#-{rH2{%N@Wi0x-{|K5)1pN8@eZHVZfhVoxuR~t5dLrA=D3;^=b zr_ufZAmRpt764@Gr>ukY6D0-3=8lMuG5*I-n@t9hzCmi){A%#~PH5Ru5_V4p6?ORh8g=w(`Q3;6)&a%knc%3 zMr^~G>w67l)}Mr$ilQcq7~CVQtEtuN=$v$VpU%na>jLK_p(p9fOxYw&Q$PL$V?;NN z2_2;Ghx=y}?T;@a?jOpZ_(`PwV@KaQsQcfa`V@PxSQ7HgO8jkPvq!WNN`3faxcFKc zm{(njI+y6!q(&GYLq{PPJj9O9uK}{x3V@7~kPjdvnohR#^`d;NuM0qR$NE}Gs6m1H zK6|uLBhv2+et+>K@OXF6i*(+2yd&O-blxs}L-O&@BhA-Ol;8K`@bjgMM_WSsN*XCt ze5li9T+ve+DR@m%|KD$<(2pMbX*|+CpFqaMC^Yfr|J3GQ-IFyd2YeXN)@x^@$L|NJVA=Nyn5=AKAy?4Xb>@Y4W4 zNl@~!!S61=@C`bVXM_8;qZO${kPekz5G|sdp1^X!ekUi9WWQJqM7LQZgI5qC#8uDC%ujK>9x>E z6f`2*aplODQ~lBvIjT)}LIr@HTDh9F0DY{!R`n$Z?z3IKRsnwb7`j?d{gs_M!BOu0 zD+(gXpY5p_$urNwEr^QJnVz(F*5blmmw*`dKry~tCQxJ8mwENIs>BFwLb}zS^b#aw zqqrNp)Cu2*!}jMLRETBF@JI|}h83Z&;Njmtw1AM+6ce&WxPOsii7)6-?#NIk?~l+BvN0GkSx-{xzmDk-C29ZFl{|Or)KWxzM^=mYL zL|*atMe_9g>~r!)c(IgrAKp6dOh!f9N>PgqCTukjJz2^ryE)@03`Hb8$bXG59Iwl2 zvW(h8AT+QJ*T+dJea>q%@XubS2SId2_{c@117?v6>iZ@(svL!kptiW7(&l@)Qj)_5 zyim_HkQeGKrNO-LazsD4;h{rpFHPf=_J%ZmgY;)sA->w;AKRyQ@Ys%Rkz*@cj%yLS z)!7iDFF+9w7Rd!tp}a=yY@}!{zGZC0MM^O=m=x6;Y%Uw_ zyuxrYJsUOO7~eNTEHE*h=zSnFYPKPfWG;F*77ncE2}<7yFqUeo(dc|abh_faYWKW^ zfxR$dF}WO>a}}zr#i1@>V;V_`&pp(0-rwC?>CuZfCM@^cY~MG8_%?aFX^GC1`tEN;PwbmQiiq5>^- zkS#rcVM?p5hD7JJo3o)Oq(X-P{TJa}&uPDTj6U^hO3kBg71-PZu~%{8TI@)i>;Sae zAKSg}t9#3ZpWFYBh*gQ{H`_M%a4bsz{vK>psV5#x-QYMB}IkF_F z-$dim>*$By#fo~`Gh#Jdub##8VJxmoH_pD=p4;--Oye2auIiBzbT;723fezu7=*x) zlJ+dr>AblFcZaVJ87la?mQWz;P9-Q%^|dca#}9K!GJcp!3b?#Ie)veq&wp0ez=}dF z8cZ#tz`E_46T&?vW56rH22tPeNt?&(5@R`WDt@=mks3~_Xq@H8jjoxv{LbsK;$*vfHi6D-$)XCl-3@D_ z9C4QF;kFn@b6bKT85hB7ppB(9vZe?dSyVs-NcP^%uSCgHBA(%##e5$+Te$M~s(b(n z&=_j-L%LeV6YTIj9NfHGxwZ+!l>LeEF6ZYVg5O@t!S>_}S%2Jf=4VkQRPH-sEJ}g= zyE6&4pSckC5^h1*f$={mCf{_X7cCXTlI9m^;S)GCx~o5sQO_7S9D%64#QNEj$zLBP zk$wiW@9H1p7({wH0QTc0THP7H2gm2sy5YVvX~7X}NHIyJ!k2yB<=dIFonF6K6;qwK zD&=HQ z64tbRzFmWwHV(J73AKR>Y({eC^WPFe#QpL6ddXBrl&IA$1dc)fImS#KRl#w-)|B zg})8(SEc*Q3;Fzi`EOHwQ~P7OTRjt1S6%#!!Khd3PW$+P$?&yiliXWuz$?z3_R;~m zD95()TXK{NUoE}ZH*@Elc5RC%VzvLh0s5dw&AY;-n&x+~?18{jYqf_ZH2$ zufAWYuOB67#t_*hVuMQH$m>Fuju8FMa03QKt%l2x}FK|^bnH|sQEeBR4elP z@@O?2ywdobZKU_>*MC9%0o1(tXr~Du2;FJ-aw+Fkz6tvI$K&d%=!bSNvf4C}1XaQz zjR&G;z_$DW6oL|#2JYsxb~LJ)HR5iBapxtY=E0k9qfFQNg|E$&V(JAIoLdp3ogFTo z8OA@#Lbco7K)6|y3rZbNui^x15Lt+vlG(`=7}!rC~>p=sVC zv;+GkKx*B00Sy|hRe(CYbD^FssBkFO#!IaHgl6WNUs;iEB7M~LMy|C&%rMol8E^hN zF~ua~3<0BxIyv;1P7-!^R-(oAXT8Ne%mf!qM?j@XA`j|^FUjHwYzQInSw^`pDNC7` z>^f|;#BaB_zKgOdsD@)Lu_(>l*6kSC1~H3W8J1{Q?(9js)mr$ExzNko==Ps^Lv#U( zwL3oL9yu$c{>ApCEcpdXu~vD?%HfN5XUl z=TfXO%R@9Z$e+|d7%}q^eqp;lOxe~rkO$5FffU_>=z4?Rekj;p59MGg>cG^wheH8p zoHXfy!M6r_Hc4!ZBUcT{;uHJ=^abU6*T64nkF;qYZqw9HIK;|(xJ752Yf#;>=XaP4 zP-VV5wqrSdGZ&$2cz5g|`RjZ7b)WoofqoUpU#N27j(t=9veK{Z^4B!_^|$=Bf_|AQ z_!shBwu;zp`{=dMKm$pFPUb;hgs&N$D+hi*gx@`BHvIl$GwiODx@#`H7e*jU&A{CC zTQhPm^()KvUFlS>W4?+;O_hh?c2fl^a2*UqIL!0%~X z>Sp|&LZxzLsXrS1r6yr+8!j~gzsGT@e)zpE7E9&GQeOv3jmO-Bu~cdde*YnsO7+I? zJE+t>veXNKQg>tSdt7QHet(rqb;a+?sMIJ~>d`={J27`Um%1IlPv%mc@%uO`b%!i9 zG*F7xWgwTz!0&yzR1$u_2ESLtrtj4jIk3Ooal!nx%{@FR*nVz!auEFRnBeyt*NH?i z`ctNr2DU_6sUDCmO3_!JY=+NIHxnAkuMbd9V&|(sFeznddAZZPqLhttQsigw`>;E% zPcwb{Y>#*K8N31RNUtu7a+p#|tA_$*-d*{2!O`?&%}`@OEq-?-D&LkJk8vF}0#upN zY~AWN!3(^PXu9^I&on=GaQG2FcQE;(MpY>*>4X%|F23fuXC~qrZR|#nU3fGP zp;K|hx)X3XElx7IKLW{pzyGT2lT7Znmd>gE`iH6g!@eFA(+&`LXKdx;hy7D_!<0r{ zBasHEv#&g*s_vob{J$&Oh490cPV{rtx`a-j#e1uBbj-1Yx+I+6p^QR;}(ccVC5`;$Z4zgyq_@UZQt z{CoSY12qIDBH;hv&h*epzcp+tUBhoVE)?ajI0S!$+B4H{Xz&&uc%ae$CHf3KzsN74 z8Qm}(#X!Jp2|#fnK4#NbrEI|wtVsGlUT}$OeM%`S^TxQ27y)*zF~e*<<9i6dMmcX= z8{?ROW;U*;N2WXPA}N^kZ+uA1B{5B2d$#A=HIpx$_U{wZ-d%<#?hG|_v>}F2%4`dc z41u!F=G{sSEPZ)fGpzT||Np7IXe56e7@suv<{aqr=0{>|)Qj+JKuu3V?|I;17uplJKVtnK=syJL?)?37}S!TT>V~Y%Y8n zsoc)XqYOgPDw41N#Ef*oCpPp)T{sw-8lV6OB;2tUMi61O&G8dxRIoVL;CW+>; z=Yhz`USsoQt+Gm?@b7k|RJ;*&p0vYIao3%_3j2#!#O-6~eWPd}vlc6shO!0pq`j7N zXDT-?Kt z8C~cZ6?ML9#C8myq6{TrUoW>+DAB%ap+on$oANU_Jz{xOz3L-1{f_qpRE`q$uDrs; zQ%2=5P4^f!p!!z>jR)FvWP0+eSbv@Lzy5BZc(9r7sfj77=-z^#Ij{RtO4Dt~6(1L` zkxn`w8kPWm`WI6TU!h?#1xVkqBxnU6Gx#MUKN>aV2>l~>)A-MR8(y)X8$MB&WqQ($ zaEMkc;rHK1{qMidy=W@n`>mGT=!vWUVnXLJl&k9GEf`?sZ}H$?4XnV^4~Dlto(>At z16AL(M0@jk=23hbJS|@QjUU!FU#gdns;XFsZt(H_4%*uACy6d+yb&%+bh!8O z!SjF@r3IDP8(o)F$6XdzgJo_GG}e4^kT&m|)E?^DiX>>y`#x)q|G6j#ik<6~A=KGM zfN~jG7wnb8`$|>f7B*v zb~pJ7wo)K6D10;K4k(}T1N#0-%UcQj;LxS=7GU*e3`LhxEgF(J1oE47HFWmQAth~e zHsHXBrR{@9^C*qxM;uL891T_FNi)IXia*?p#75#9Q7+EW0!#i>$7ksD__3BRU(vjc&gUUf>xgcnMuL&BDlXvzA|9Ia1T^kQx zgPUDp2pQCF5#QC4yGe2Ph;SpparzGb+kH-#BNBK>P>J6uN;KJxOflO_ZUBmnWqcegEvA?;ttPXm^94=8hnggO=z6DK*#9cU1Ph6}cvZcrBGpa9&h{TZR zC%ybhsS|_OoWY^FE_Ru1)?L519H~{s1HUoCx5|JX%a=q^CtTa2Nq^~dJ!SWSj#>(a zT<5f+soVngOt1VZ{=jI2=r{`Pa((Hjx^@Tjy}lsK$-sL?{HLal-UCwa3{>*kX{iR< zh`$cgFIDtBts^BP(BDDxIlEKveAYB-Q6^|UrxrAw&jZ+{E;T%#`(ORv%x8u5Fx}uN z*V{oWc1|R`dWKxNWh@z(>6r-}x1~V6&b+)d(`-FgFetj8^I^-9VaArpgYujc?XL1D zbnAxK|Kx*&U+%T7v=j_#W^mpGbB0Bot!?i7!(!IFGl;5+N-gH)<(L9j{R3Cj%J-?m zqB}MVB*zn0n1=}kcf=ZfW3;N593g+g?+{?aD`)WHnlfKM15;=Q(D%IW3dpnvWzM5a z3>hvA)pgQHQCkOqz9IL7K)PIYB6BK^~ z{eUMDie)#pXxu$)9$Pcsw}=f4hZC8YdaFzxE|dFla!;AuPbPQZnX zj*`hf!n>(Hne2OpllO9RnM~d*llKKDZ<5JdIC-5+epeQiu42iKjPj!KC0?k9M9w( z9-dJ^Bcg*=@%fNh`|S7HYp=cb z+H0@9TpH5<`&YQ=dukKWO@Z{$Xj7Sog}nAJqWmY)ew*IB2rhG53xsdKA)1lM$NYDy z!4Wz>*A|>&*Z;RysQ!Dn{=dIMjTbGr1zHfb!~Q4tVeR|3*7r`?M60X_|oz1nFIEV_?#9R$QYkFX_hH}ca4W(P<|g)UfYc& zm-NRft5B^1kW2C7^y1nkeOp;I1b|8RD*jH4uZbTY<9R(VW>rWJ-W@7VicwbG9~Z5} zS`{iDqiha&g1P@XDR*2qFotimd1Ix$nXDcJ< zJoGxuq;D539awiR)xA1)@cXD&svo^gUsnG(QU55{p!%~0l&Kr-_%v4Ad;RRMGpZFu zH@Dq-S^csoWmQV7gAX-0{O8M(VuHCVMXUC=UrYSUY0uAmWHhnG-iF1 zB_rEydE-#wgHXvUH^$_wiWH~ey&o#L5(|c8P{yJ5(T+UNg-?XSIc|%mCilkJ7*Wg9 z$al;q$Ggz`#~Ia}U1uFSABnWpTtWjzN|G)b^2KXTL!#+xaC-ppAIX*+ZBHAtAK~`A zyW-02A^+9E<&F5rGB#!=G=|#8tPc1EK_oh74#pRxJb@A+q8%? zkpCqHa-|Qo9JN|x7dIT}3(p<1r@(klrt!QE=v^zc>Ae+F61`u|ZT2pQ{Pd}AN=@!$ zd{>aG@G5zin9HHAzilonazjf850wc>Z_8j)kl!sGJS6Cpr|p`4+;c>@w6o}%9dB1% z_8#KHyh(|$X(sE7gzrbAZ-)8G3YWF=3zm5A5IXjswjhZc-JX(Gmxk|q-3LPH40GfP zFK%t{-Lba65lV+AExm%!yB!Ip>dU@J8(+Aq*3)~@Gr&@memf-pyH5?DiB~ocyC;yW zY#v&2Q~@|Mpt@>uV((Qp&v2>Xws`&&he0Pnd`(Wmy=qA)5hV6lwWK<>=&z!2g*QV% zm<_7i0juBZkX)4gx>Ga?*sjK!oJh>qiQ0qod$?J;mDgbF86ZZr$wFCf#^)EKPbXk2 zQA3mKT;UZIX-9Cx&_>A_pHORja$_J7eXvB(AGNai;mE4ph4Ufg>%F(IL*E?kKdW3%C>&2Ae zKei9Y;1&!%?+e!(4efs1a;EF){hIMz+SGdrub`J23cSYZI)(d$?ty~vHHDHMX|1ko z!x7(piWLC!kF&RhS`0jT9Iq7*Z&@;+26h?Xh8o)6xtv~w*#K-$dS8i+(dT#b#WeIX z0FXd$zkB8t;srya8Z_ngVfSZ-R@E1_`+DvuJi!+lTBLoX(dU>xTN~yGgH38r0x{!T?X?K*xvo@ zQ#uN~g=Bf7#v1o29|rd9bsng_!$qbT4edH&E#nhbC*FrI5kRq-htNKmG&@C~pnZd@ zh^|?oo&ewM6$&xTdVUA1YZ_7Y_|G&-QTJ2mSg|$_?e#%Y{OHd%x?X&Y?b??KgXCZC zKsz_qGnvbq2frUiw+BDnVXJqogVx2!WOP4LG#am&CYNL-+RctwleyV$aaZtu(d@qi z>u$ym&;2G6agKs6d%Y{_zR1J*tZyxOHU3(~Ph!1sPDLMM$|}4gihGB;>l!ymq0c#dwfrWV>``iID=raGVcyH&`VxmlkRIx>`ze}5Wy0HTC6(G0x z61^9Ro_~f=Ww);3wX9`3+E3kG$6sCXOuzN;b>Qs1?ESO5zu+ua!Kb-xYav54eavn9 zMa#v=74B=&{`Q-WlWCf5X0N zURfXGBL@`gd|7uWxo3@{9XS_JJCSgahHv<%hz}(fZwhXJ4G?;IZ@<&P_cs&JULRj) zfzLR%+i&2{TiyKG^C6yzicJ&mj-rOM#k%bgaWAGeM%-neyB3FeO1(HQ`751lUj!P%!R4gr%h%=?p@yb#nx|xW{Ix|ne1L}z0fMvF zsX=cjH<-60GJAsuFUK#hj44dX^M9_q@d*qB0C~ox!t2{wqV`8leuxs2^p|zwUJSn4 zLmsh8me<87{+n@en?rBqE)T)9dnKy*3kJ2nqK@0Iv46f|vgf)hnC!1FygfI2UVPE5 z(B4pXQ_)zb0m%7+MeD5Bi)dq_v<-o(%4GkzaTjZugS_4Tyi@nU{U_Cbx2Qimy8fG~ z{u`zsn`^M;D9X)AXze;ZuiYEj?6)BA-#C;cy>W7R{D@_da;0P$I{zI-EMVki6A22zBMRl-xJiLo;jRepIvAu${?Bo^XZ#&U*@8@bs zadJ%IBPCfeg|oy9ME_{HM7h}S{_;(wwK zk^_;f!trij?zD7Z`~JFJieh-t3&%<2e;@wlu-ndaa9#nX_F*oXuIC*gZKJmdN8YJGO0d-xcmVVnvFlGKure zZA*njRA`jE!Md1W!skyrmNy>|M}@o@9(>abpUh~qbY4(zROdJ1`v6Xj_MCc>+I(^#o?ddC1fSx;}q_G|{=Mt&SU6 zG>V>o2zu)@V`ehQ6P_(@PCju#lwl2?T(J~jUx5XRx+=T6XkU9WK#keeg**Jai~mYq zAjo@Tm-Sm14ZumzyZ3!8GQGNJc>CDQdJzDDZ+?&N>Qd)-T6>ifC(MoFE>ipfRSvZ_3$bkylbhPN-1O-TaV;Y&54PWvUb|3{7LJJ8D zC$4pg-hW9qwptsPurPEZ8+bueHCO#ESxHn;g@8JeN z=|NMBe${mZw_9sf^OXePFQ~sU=yvy}eR%YS%w|$v_KBiLp>ac1od}F)lSmv!hP(-KjHo#U-o8ihJ3SEeJ(cUGWlD`Q$YCjBp8TOd))s_ z2w#f)-9DWp<%G1SAU_Hhtf#_g-=u=f`jAW6{QZcp@Aehr+pog=Q`L5Ow(_WAAwi6^f?HD7RL?)GKhro2@R;j>)_(F2Gva=eF(lRd&> zid>|uAf8`%13jvK#`1+OvESpTAqh1YZ@(IK|D*lT|LyzZAj1d0JGZSt(ChzKdq|*n zU=7(j6+fOb>Spepz&dj8iDcFT{*3J^$fT|V$|`h@n5%3amg@2Kn`{nKg6=836!S!WAx$TjMH;Vr(N(Ti^bPAtIhKvdBa zUjL!O3`jX_v=-S4g%s7dPZRC+_`+#U@+D0kUrid8_6Z*Td$hwSEbMrJJ7`Z3jPNgG&FlXi(4C5dKQoLB9iXMV*=lO zcNAUKb{9Ddfp6bJ_nPJl5pgnd$ZPBn*M4!?gH4ET%&p5MQTDk__(%)1cZT`Om(iK0t(D9V)jE{=0fM{+bi4Dv5#wx3S6(bBreohSaus8inud7WnnzgP*ud8l` zIV>$Rq-QrNr9UQ>=n*dpxp)MTx!d879%b{BqsU=T9fu~4$jLfac%yMZ%-}R*|+)v}KETL#jPve~_AccI(Y3zJPI zYlAw^}gyfJ9f!2zyM@yX+tLx zUSpp(y#_t--spSTbw|;4UjK#Q>_%nPk7!Zr51)#I`P6=2=5DQ|Hy%DL&H(yd_d~}E zZ;N%q9yHGAw@DTl?QW8HkpYZ>Qh=;(_PWlgrpKjaZ!L85sAiu~R`nwZUnj$-A?(Zk zma^^q>V21g~D3_Yv3Cwr%m`tNNe$S-X#n-_vs(lsT@fP3KExx|%mD0(0A`q!z zmYv;M)Y$gBeyn*=cFcA?{BD^vdhyjD0Td4NZT$@uaPiW>0(Q3ji#SgNMu&v?;(S;n zb=do4*Dsx_qoeh1B9Anrhr4N@cLEOudYCYf9$O9CGg*!MZ+}%by~nJVUPCB z%n%e`bQ6RMNAtS8Pd6H@KRy+;eu>5FGW$l_59#y2=gvF?Lj}78ycLT_8hDMYq(4Df z{U&jV-eJlc^lH!i8sDTO*n*=cUN1^l`|xeYS#ji^dUiAH6Ar^Zp%wNCpJ#@wxeI8M zp!m@vR&aK6Fz+yJ6uhm#xjr{`=BCuxTLq;Qz1L0&jJ$tJAoqUPlz@JJ>@XXpVy2gkI|7?pSfF{Z%tH;j`+q88i96TyhcM{%LnxsNL)M| z0>Am-j~GZ;<}n(>t?{mN#=(?onCPDDx}s{`sMC$6z{KP!fk&=(c}hI)*!HAc<9j0` z52i}H0()ndd8qx&VE>V17h!MfN*|a0BZE^Hp*jnGpTS3o;ddVVu7%%xH8^t)h*%&S z)UcJI zmUJg8r7xheiu4(g2s%8SnXG0XP)fg#pYc_`?1M_FmjY8$)$C@aG$$7Ck-VCwWw$7$ zxbAD@Fe)ad8)jdj`x;FYmW1=jJMMrQ?tegaoz-1Usi0 zsTrN#aR*UIg@}@)F3q(~bM5dNN1=Y*I4ZvCu8``g6SWv&=W8!EXWT*EC<5%yuG9^0 zh&`muZVZeKtZ;FK?!c^AhEfBw;uuN`%!*gT?Gpk&OwhuoGd$+73xQdQ+>V66tR#kp z2WBNRG%7GFg`pb*vxcbQ)3@XVmJQXyr*6+Rj)spW=B6|SW)0&O#|CB%XJ|-Z)>RB$ z6PTqiG&(TL&CoXjv#!>{Uw$h$@booWxb048geNd-1a~kgFl!`3N?_I~hOP_Dx|X4_ zfmzo9CEOaAma2uV@1RS0BOE>&pPLd6%(|Y_#0O@LW@uPo)(s4e49ptC(3rri8yOlO zm~|7B_3gkkBv{X*a`QKQ;LdO64`b2bJmLFfOZoWYWS|oZq4&|Zur>6D|HJ#`E5FPl zQRNa;b8xnttmY3V&`pC175DLYe&scn=WDfjXkQuwzt4!@dE$4D_(fX*n6MS==*mx* zYLw@Y-sXrE_a>bPT>Nt){8ZIZ`~}PWDSoX=rI}1GK5oAXr-YxsATnTX^^8_ZpGA)r zl{_6rLqjXluwd97mOg|rL3U4@U9FUQC~Rg>IINUrQkdCFcCAu64#L_;_Qc`|e3y4z z2a-Q*79gOHyP&$NQPWR>PSq907u+ZSd$(HL@KK;kHOGYD{~X=56}RQ^QFV3bTn|sfZz*n91~0t|YyW7HA|GGNHegh?07Imr20Xrpl zQi6xeq6>Lm7P3{Qsg&SG3I0xkZ%c5s1XoIMsRW;u;JmVEe0ByANz2!X6VfSV;wg`Q zD@%g6OYj>KyiS6{Bm)c#f-VVOFdeX6f~^w# zRDul>+#N-!Y7zA@FTtMG z4tQFE$0gV-!A~Su?SM!#j|5_D-wc0*h!5t+*!BWM+-u-lWP8zBx+LuIH6-*K0#1pG zWZ_Rq@cR;+CBa+?W=rr63EnKhR0$53V5|hY_(9q^0kM?|$6@zxBHPvIE}u{SeZ? zV!KW*w*TR$qU7?6MXjM^<#qOAe%d;2P{vf7Qjb_KP?{>HU24Fe5=b_}FWMGc6!K)p!?5&uC?al6o9X*BD&<-iPS1G-n!t^MOj_GftaMdCTN0idxao7nG<4F?QDdc~W zgc?+%i31=B)d^`I#6)#%{>N<#JU|8bIpcvArz6_kW_I+D;?RX8p)ygt<^3V^H$ zo4)01wGZt*lb(G-Dg6y@k<7U;aVv_2+k>J`a+&#q4T&-Dr_mO(#d_$Yi*(X=r%U3A zB_<*5Sl>x<@I_dsQZ zRh&+{s#y?v|3$P&tl;RoIr{S%6g?NC$;ZSmINHb2A9M8cP{{i~Lfgk>9Q`jG{RfWz z5Tk#L(Q_V(9dN!O&XbhQiOS}gE;W2KSxpHmo2SOA;a0brQU_n-;A<*>g$;CT8h=fI zuM^;FF2IRe_-KZfQV;OdBrV*U<;5gZlOZZa7CA%~IaCx086F*`g^#6aDGiWfIHZ`M zr!+!}s~|-NME^k1!pF0;l*N7q^;nERW_(ZCfQUjkyVl(xW4#+-A z4IhVQV%}@vYX(#Z+vsCiYD%@5au5=vLd^9L10^;?a5SWMQ~H@Va3c@E*D>%F`n?^# z-pKvNuds(c4h1e9tA$=>WnrSuTd=}b2rChVlZ)Knc~!W38v^J||V|KPpe?EOke}Q9oCKGfF=lR(_MRnjzAx`RgYRCLaI3uR)t{? z=`P^?4M1Uc>8@UVTo?~vVO7vwC-iaEs-dXHW66Mbw`%+_6-FY>)*39E;HZQEJMR!$ z?w&D?-)vBAzI90_C>& ze`nkQt@SCT|3-nOKx@Ff$s1d62(j&T1b-jal0HF%!NqQ3@&8JJ2Z7eRmn~O=-vc`D zT~?^0`bE)Q+o9jdx@#YzKzB7GE$XgLKuVfDsLP&K|LzQE_V#6S@TA~jJg%LoW;ZO; z23VX=-bX07`x{dU6WihE{fUT)y)y_=f8*#lj{Xv(?fEq?(Rc>epB+*;EXwH@ZouCaLF@j@a09C@hbdZ`iy2gI!`lZ z&e4pi`C3U$v0AdV7?$^b)SZFHi8A3*i6r`Sc*iDeI5?IbVpr|As+qJH4v_ zd$~adx-Y#Hb0z)zfb)et^q%G=-CR0>%dm1UfsAJ^pTLg;>5NfNNsTB`H zhpYJ9px<)SrzORj`N;TUJTI{NUqlRS%SVjpW=LT3qZnzEvpzk>5zA3ge*&t^6{ol*2!r5D9EfHa`>JrKI=T;fXpCjJRN zCSDwjkHuIlA%JjVUGpM|%>o%T{x1&3$LZpKf{&)+!T8wH_ci#q&>fABwS5jg=Js9* zAGO`l_$cW6zre=?a#gfnH7d%ekg84H^my3C`=hkm<)LnTZbKnm_4-X0BK8CCKfQoc za#S+j7kMjEL%G8_PFgzs&jhkae31pl(p*bfd~0L)AAn^N3`o%DfH`<|Ip*&V#0U|4fvP+o zTA-}(FY3mdV@tY|G2p{M8A(5JhWR0cN{%MM?h40A**r37&nabdVN&v*(SC*_@k{y2mny{iUSUWyXh9)6Ah}dc*_P9`T^HCnSC=`Pbh8!|38q-6E<( z{959-RU}RA=A;XhGoM3)|HTb|#z8!d5SgF7k9hnjhSDq)y<1p89aGNyV?myBrXK(7 z#y=r(pVc^L?8>Z->@Y9R5VNbZ7MWnl~aX=y&EEL=4zp9|Z+Kxx?vAKo1` zk3Y1)ha>Rey-}jNUGU)(`0!t&xZXpx3p`5uRsc>eP}+~dZx{TYgWogo+eSG&N=yAY zsBiTsL^f1qO*!jBGlG={Ip5l&Tw>T#IpGHTd2roI`5n5p`TY z{EdXwlImi0Wp8m3KA(~p!E7rC`|!IyhKgBH;pX3sXg^^7kBag)+}Eq;{LG2KAV9eFoX5h67hh!C>|p)pY`&Rm>8!v#Gi%U_k^iGy@~`B|SOHYfoP!9@(5d@MIi$ zY4pCh^}*Q7?AeMn<5RX~7mE)z;BV(~@p)yuSbgx*SDnZyE9N;X2ucQTEXEaNwpfz- ztrZk5%72MHuQRK0Kcs1v=sopi{E7aO`;!j+61)3u%G9F-yHFqC4Mu+FRe6?9#$GZ>R#ZJI3S)T4v44C6Tjks zcv`;50xM0PPpd7pml*)dBp8sO&jF{=GLw0T%>{LX^l}d>A3bt+8BC1UOGFcReuKJ9 zbK@La;o5J@hr$JlxxTTVCy}xUw;RjR=)^~#Ks?^KfuK`X=R#8cu}+B&Qd{ZEavw}3 zF1%M~<)BrwkR<0=d|aX*CKNhu(t@x-Gw2|j+=&`~cSQK#H2lc~CTT&mW-$3Ym`{g; z-hdkPU`{$Y4|*_X7K50Rjv>MfpMe|4bFf}<{Sp0D03cz zP=WajXGX1`z!&tao6%;SqZcHqI0#6qV5D_g2cM^iG|IJ}@pb~B4Faj}b3B!b)4bE# z@H}hC?ZPiuzZhoQ>Hw3&f~{4uRN$aK(u>z%anBAY&Q3o2dB;?J`ekKhMVD29Wof)P zjJ;x6DO{H5yVW~XsP>)mtmg!oa483#i{j9fpq3N(go7P?-<@6^#QWDDJ#u3ddFDcc z@#*Hv==0s)E70e{=l%(Oe($-#^qKSA*U{%UpCkD*))T2wOZtF7pCT$qIC0lNj^8~< z6MA~m72nf+6glB0_~Jne@xEN6Bh;Ogo7NE%$}`S-CWKYNQB=WcRKaOf!GYA;B$mmh z30amoj4C*cDmaWPIE*UTiYnNOD%grD*orFN3sdLUpi%r;5Epy+c0sPImgO$BAQU3D zFzUXt#28?{Ls1f|1xYNFyWl}(;Yi@chk)fS>;UoVEC^1W=adz9LAn`kjDEyz#{(O0 zIRNnoA^r(BCP?E1X(B-@Bsj{F*uME`P|Lm7&Y$3dZB;%d&NxSj>E{)9j>uy{9+1@9 zPeGn%GbqoqS*~+am4#2bi}B|zcYuV>L)} zDe>dN5XMxK$i^}AC z(g2ceqI*y|w91`NG3{HY2S0SvYp|L7)p5JvFZD7$x|dkpYWFiP-jLPv7z)%L#+r#_ zw)4OgbE^RNEhl;C81)~z12X0W#K2g$Uu2qGYsb)l5G_D~dl?5Cq9TuDWEdm2@t9T0 zGLP}twSRezdS1spZxW-~fTP(={p?l8HNao$XTsNxW-Hcrl(E< zspy1==~_T|h>4{bz$gG&T5 zYd<4mdUz~zD5kv@$1*>5@K`1e9?Mm3ybL{HEOFxXI>@c%0egm9g(%!B2CSEnTO8}4 z{Wv3;N9!GTfMX8eZjh3 z%Lq<4hnt1mtzE=^3b7BnIr^D62mLJ~dJjgYi2;v`9=fRb42};(#WzRC=WzU5S^VMX z=zNae5I1nLLez|T(}Szrm4tzAbzB|(HKr_5@nw_X4AnOChHwUiv-phL&w?v##SQ>a8Vq7_FpNw0rzl?QRm?4jd;%)lD-;c-ueTs;ofVjUC~Dv4N7S)$GqA=q3_!1 z8e9~{m(0}cIzE&4gia>`t?d+330kpecP^{7g_UZVq`eh0qOddtg%!YF3o`H~B)fSiu&9}hL|2VOS1Y%n97>0- zsyTV+9dss>|7|*x>G`&O=pK6~9oc+&J4I)5^t9XUL-*sEm){qT(y!Cw@fTITnPAP9(PCYsG|xhC3%xjNd0IUH@nTq|vuw@2GBZ;iHL z4yndmVTe-1bJAG8K^iUpEr$K;^b{F|&*P0|Rzxc7PFca@tSid5 z&j(Ifb*;|%z~p1BX*2OUL8tZKtwOQWZ{2*16g$OyQ{*JZmbG5#pa91G_1kFNv%f{- z?!V1G5pDktk;StdJ({E6yv;rl-SM5l`mU8!TlFm1_c|Pedl|-@qU$G1f7NN{qD^qZJipbo}KPxnPa( zH!_$5U>$?7Qusp`#VQd*#jRP%**0Aq(7;v-&VZ(y7K6|svxCx>xAL25&|9;cb2sDJ zoLQAm9Vw5&%llR2M95s91&M*)%ac2Wws%c84f#H6e;;cpI#@Z4iU9v6imSFhm`j;A zbg+ujTtDxu&^D9@&a!@XEhVXF?6Aok^~w-ut(Z#+=k5-+0W>SJ&axt4!Ccl!*yUnr zT>C+5dMln;t(YKA<$|A~W+_n%&P>*VkGi$s(o}qhQZrU8(u`%#XvSmn@cn4bn3fN@ zic#x<1(~D%;)4Oa^dwfaF{TqI*QF$=2Ai-fPY+fw8oF2wy-76elPB$C$^Jy=MYFdWsRd2ZCOXk!wXQTuY)6!TM&a=>O=&5nR-~#XYgUS8eEc! z?^$Zb7B}gC|CUTYZ^zKj8{M=>ly(po+>%PX`GlY#!0enAfZ5+afz}W)(Y-+r}ZR9iniK)p85ppA3u)OR@UOyQC+ze+X4~{QEW+d zF;wYRUi%JS(+pM+%c`9!R54S9ie{=%(M%Ppn?&XxQ&A|3Ar3TEXltelZOv4nt(hvc zHB-gO<5biKCvz5|xS1*xH&cb;W~xx!OeKAAX1zexz~e)t6zBLbjc{4MiP6DWsY0v_ z)KB5eCRCloH#Y9*7CJVX8fo7rq-j8E!E)*nd)`U@K1qK};Qff@3y+Hv+|yQjmt6!# zeD*k#v1kF_S9*MK=D*9#|2-h{Iy-auyt9r?Z}77GFUtJS(-Bt{6DpPQlx>4^?y2sP44(VK8DSZNd<*^1ciD8g?a>q(p5=3YyBMB zjO*b0^g&rDC@SS=5%%@0ml+d2PsRoVRJnsL9+H*%>|!JGCQ7h$ldX z{v7vbuoG6ND1F+)JV0uGEBS&uR;62 z?B^3(9BVwEUEgtrk8)!==(oz%w>)1YFy{%tzsP3Ig8A0~8^iJ~ zh1hO6@h9Tx$cz8iqclNcI_1Pi?p+lS6L9*Y11HcaD+|BiBqvr-5{_Tl6GJ??U3u*= zSy?6gW6U7Mw=z)ieg2l%E8UkY`O>XyPEt0{aw(f1i$(A025mtdi`Qv!;nw7o#_%!s z%40DxV`J15v}|4nIpV`d-IO8$vL=R)C8tz}TT@f&fK}o)6C$wOWGE;l9By@^T`BE_ zAtD9Cbr3rQvJSy;Gk_^iB@^@1D!zWWyb^%Sy;S_gL zb6(C&>G2qSp73$EC*>TZze}`f#@!r05#uj;!e2ly&O`in?f5K?zX#(ldcvPW{8tcv zFV~ly((f^PJ%D--T4&F=58HDV`ZQyb2TwqN0z4`85S+}T1xg&#&Z{>dienj znyaNmG*}Ki;kHE2%A=9U*ch##zNGAYgp4k|jp0*?E05u4Ptz$6Ls~&ECQx1bA!*Yn z*h1kA2igobS+`Ky@R#wPm7jCs9h^9<6;yL#fWi;gbVO)J>J53qry=nP&i9Wb&&n@2 z->2M)POV@Y=c|SiI3Gaar~6Xyn;Ohpt5OHrV?8U6bJlaIx5M;y)o2BMlq_y}omTKM zf2@TZWnn9U8tN@Lhb3y5^kYm)&^Am75Tx7`wqhxzYCyBMazmi7Pq-lf?cjz0)W8+g zp-aGUTM}baHMVO7+ZhE1w1UsL1b_~62>>1B5&$~DB^*$Lb2o(B6S+Iyf~_OIJtW3! zj!D(bcVH=W$Dq%dur*3^b*L+k$HcTowDd!|F?SP`ib+*-;#k5Bx)K!o`|;O8t~*68?k@Uq+b?oy>3iWjAtXrm@^o|9O{)3cO_&2d;nWGYCve|Y zqf-s{j#6C*)s-hWH#F}&<6~0Bbn`uKzj0m*|6`OM_gtNB9MTvk-t^C$OkKLMyiyIf zkMf%FH_bHPxeIgYutbFWN2zf|8_-YlE!?LX#GF$^vl{Lj1^B!kP&JgRs@1{~sA~Dz z=o}j~Y-TVoG}9d8*34D0oVG@T3cRipoK#ESszP3L4FG7$Qq9##)M-@9bdN%daO5(W zZU>qHG_~0zG`mbM*v&)OpceGf=)^7GqzclBn_H z3Oaarcd7+-Sb*+o(hEXN^WA!Z#q;1ZwV+!}ZW!A}*g@b56F!w>PyLHr3QYaOOswa; z1$()a2AB+3ijFgUFE`>8l#Gzq^@+D&CqkO5UMr~KIvUBFCSSOcQ?iz)A4V*q2%dpJ zXJXDner^RowLFcwwSt{u`Z-j01io>Oom#;z<|a*AK|kl{(F!6|N^tHv-5fbqH9s87 zM0D3Lw1aK6d8)NQO4bq*e<~!H%DB}Fv!xc6HMAev+t_3`b!5*Fxvtjxl5p!2hKd2WR!r{>4c%XorN6o{2 zV4vr79wt3*cEAf0du1!n`f9b{9F0+2(K>DA7hK$Fn)j}Ly`Yb1DQ>yXnXkKap2I}a z)rcY8br9XWL0i49JznE8EqnffiRi%^1KEPo*%=UcKNP&(wWlrBp@t zT?fUg3mD_s+Rh8IT5yPG>N&}zG*}6t;a=BS^y7r1i#IYlolX-aXs#nTvBV07>>0cN z*~*g~Zb!suu0GGoR&IU^6IU%V9b3$>kpLe=Ho}G1)hlERi4miglfe3c;&@z*$)vD#RZ2HzNkTn)lZ9Zi(a17F_;iXVPzcgz+to%*_(XilP7Ds= zi=pvY>QFU&YM3YR$oOHocKKmzq#3VbIm2@emY=ln>7+c!c~!VgF{;DI5>py5kD?oG z=*$KSa>If*Brouj@mGhfYaTS?o3Vgvbkg4^AUq=12#Lblhvz}xN5Xp0h*?Jkp1qba zqZ$q9qb4Dx3BO;*S6kx{Mjtjq^PrgPbB!juog>Wn02V!31F0|%8gxUr?W$a(JKUO- zQil;^&@DXU*n&%M`@() zidaAB>WsP=kVZPLgmH>=Jv@|M!to|9BlU9qeZVE|2M8BcpI_zZ?R9fK@(HR8yEPV>`2laKXeFqsjuey$vxY}q)>soYD7&mDe{tmz*~C{* ztS3*NN9SQ>?=#7qv9`kp+Zejjs-o!f4EqS`wDo~?o^DZ(=pg+bK-o&?mu><4#JvQI6lom%H|V{ zlprHMNF!mbJ3+|lkWLAHeUg!L;WX}NsV~ZA^DOFkeEeE82n=nATUH5r`eD~#sN zyZ=1rzkr{j)B?2-)y1j1DRpv6*ki!{uoX7Au$TAuP!*)JFnhU-K2Zj%J2V6K!TyUP z8DudpakBnF$zXHsk8o8ucA~0}VG9mbeF~ATh*Q;?SlZ0n8WKVkq==)hYvM`78K^2* z6XGNZl*CNrlu6NgA|jwGp{tQwl@gKY8p8R9Mh2qGzm97eE?NM&h9T%ERLo6W)>T}w zn=4j0d8$ff%;Pd%kYz05G5}h_n1QMn0|aY?v4?aRC||*{f_Jdj6jGzysDCh*+Ufm8 zm>v*X>j|Gs^`z{FgvH!?Wpf3-VVlxvbl4;LqUdT9qAID9*ytVNvM5KWyOzShYCmq^P9#J4q)GNZP?kbDTCkVE2OB z)Xi=3$mUl(2rvxcKD*g5jGQUX6vIB|QixCq6DygMOqKY{bMU7+HBY+|{(Rg3Eed}# zoE*~!;IEjIdYxJ-bgIR{V1WBNQ??C~&*J1_q{|>l_)G#zv>t$j{Hw#RE=c@TRsPg~ z+6(1RUF3wM!j0~VVWaHG(>)Nk3pYc`pU_W9v=+wO06bJcnAKgvd_c|VV-)?fC?une;qcq=^yr1#W<=PY}~K>X(w0lF(qUdFveLu zXg1`iSC2i#?&!fPWDuUFGf|j!x|PcE8nS)Z3D_zbK*%xOSf4`+_)#e6 z6WuuIH4f`Wc@~clZh%R^LC-K}v5^EFP>u2pPP5=cRV6jIhnv+vqgemIar>-g>0un)@(qbAy|Pc59}UbgVUtaFt!TJ1LZJUn!LuBxAglJ16%rw#91PLcdu|a<$ z7lKr&v)lnmcHuYe1hM8dbLR z0dVfjJuC|;(wS|P&|Dl&;U9P?y%vaR?AfShH4= zFnyraU8lKBQ#V*%W+THaj+)XzyDz3{SL_c^f2<_5b7F=f#_=qW=B7(bfjpJ7qWA%9 zIC#|Oaq3!{dk4hK1>#}maTbswCRT$aJwRG0ir+CG!Dd-Dvu)ze)}db%NvFu9g5}R} z;_XD@<*kfOhnb3Y(Uirkn<(h7lg)%yd5zOfOPPsC#u{9c?@XgQYQjEfk7NlvwKg)! z+q}fcsHwsAWlYk5b)V$#K-GuHDG0ZLhZ-8b$)1rxKeaVjE>OAEnQuJvnk1#v5k_g& zi;1Zlr=S(0SzbJtJ}S}L8okO;E%eYzkEpg%)(0KR7jrJio?LAN+r$QlYK)%XO&T?f zn^@yDI&@svDlCT~0l)L}?m3Jdm?QZeR}_T;9hR)%`gt7MOKVLNH=x1crwynCitc&? zFCe(BY33&45ewvkx%j2#`oxQh3CbVziyqZ!z}u-gyATR)Y<2j|PJH+Ro7W_Y*sF1s zn9JBo0gIFFqJY2)CfM$@x{Z5tgy+j5r}V8*;5N+_f`y)$CC@eGPt8m`2W&PIT%+2H zG*4ZceFlRL>El}HV}==m5t_qbqYwr}7K^_Eg%DHGV8O%)T;Yrg!I8GfS>H_jx{78z zs@QGy`W93qlUJ#6yU@9{8fT(Vr>1v9-hA#1O5cto7pz)KWlRK?Qo(woRavYs_q#d0 zMd_0`y^G{C_}W2V<5@!tREUp1niFC1uE1{36i1AxsJi4F=t6E5gZwCIhpj}Bd*cSm zU6=s1Rv<&0Zmyaj8qvrZl9}FrN?dL<;@?6 z>bYi4!d1b?wb@-3{0C)cWV2$a(@>*~r&z?T%!#pLV*h37%jxFGYjJ)MGFFL`9$Gr}IGU;|b7GaJIW=z|2m@d=BXenjkN)$plNK-&|zkx=03AyH>W zc&I^B^q_Mz(*grXc;N#SH60liVVn+^%v8H1-cic}dEpK+pQh8IHEEo7h#+7{d$mZ* z-7s-a=EzZ+`N(jHBZ8Q2u1*(8v-%w|3X6paq2Tt26J?7;%pKMc#h^+bs~7$zCtU?S zn!se7DNeGxDcu2)u2QBWvCDi%q+=P8%jEIh#mCH5$%D~EVlf`J7)NKB<40=Z z8+dwV;tt)z+}!S&mi9<2o1>qM(9C zI&+F!!@RP*Ihy1!6`{N!M8~uRyZP7`jYH_L&lS=N4)PK2c5gv79q}4fL?i{f_~1LN z7u51W7<6X8FeyEFe7@|ij z*hV%H1t^SoBVG7eJ*8B=y3e17Pcw6Fs-1L58#{CI6v=u%4s?oEzZz3tLCqHrj_$N?1Pf_=)e}E z4v9$+MXusCzkUC7^R0U@hqvGWAIX-}zyZ&*sr6H}fEWayl2{aX7--#!-CdBVV)C$?K~kWYWN+glpb5UjAy=D!fftVT^g2lZ6) zJR_owV=;-XiF8+uT2M_hp`eB74{IZcjW}TD3M&?!r=k}`UN&Y?3nGqD3auMjL3Djn zkSNizW!tuG+qQMuwolu(ZTqxs+qP}noWA$Hn6H_L+!E>2*(7P9(6SPTnyZfXgQ|;z>`bYA;^Y7mE-bw1_y^!U|}5b2!dx= zy#U-V&sXHvR1zDkrzroF8J#N*kU^281r%M-mKtn;XzmdbVrR2>K)Wzm{h}6p2V-6W z)Td_MlpY_L2Q{M99Q}#q^3)tNU0fxo+WHk}XjoJ6;pGeo^X_K(kaS?D?Bg1SyeBXQ zS2e5u~0Q%UzY#l%FxxcjCfh#Oq{fSax#{`R7? zHx_M!!Gr(+4n0WkG6hl}B{hj)3W%4Y648>$#9t$%zYRQ%Q85_K z>Nv|T{gU0RXa~fXV*FdIuHUw@teFe5VBO>iU3DoEuEoh1^)?}UXyRmk(9=q(kDnEYeJ4m18W!kqI{4#h1V|{pA8W7VTj0iD%fPd;h&e5YK6cC-p+!`tjE%kNl))*c)VaSGP9B`Br- zug+X)ElO;pB(+`Pd_fyAUl@J$v7O5R2LZX6G>W1A3_XO$O-`nA6Rq;^g;rs#y(`Xr zi`Wslh-G!85cD}~OuekURuU;J9y>Um95fAP31)7v{4bON`6q)nr+3pfE_+c#hBd_ov=X@--PI$_-}k@H(i`{ zdq4cJWZZ5iL`HUnI*EE5urm}zv~&)$U3K2P$*m~XMzt;F8)>PS;NNsT^edG-1|ST= z?Qtop#hcrlI@}tS-e0VY zoeTY!U&_wDUUx0exM*O5njbaGKh03AMI@0fe4+z9A0MxjpO7l0k^*M&5l=?qv{u#0esA^ItJQFRx;Vg6 zFp*Ew5Iaz=D*OSFmHJ`&O@)>+^248VDf&Gpq|+ z$W-H!!yoosxJ{GGt-z?yACwovO_3|1FoHX${wod-Yd9%+djBrkZdxl1o7)C0`!Irw zuriPMbf9_L&G^Pr>97#XpX@_bnpvx5%tQI_xm}gyujtP$17$r3)%DzIFurp5ifdX+T7U!U@z}J+{oOkqD%(a!Pnv))Pd-beZx5=Ul+ki|T{AKl4F>Lnpijk*PW$mVEy9v}^uuzYQ949>5I0Eq@IY|0OSq++#?Bu7 z*e3jCjA_=$dNnHT`QC(+SPkkA_xY0M+fm;R162nmX@E6`;8cJ$$0EgOairo{Lq*IO z*`Tq}oIGd?iNSiH0Iyin)39v8g3mdN4#*&zM)=J-ZU}vt-Ngty`(O)cxal+tYFZ zTV*wI-ZROly6>-H!NAC>y5TP$J94#6jIA+rn#c-04Z%dn?=>)?5z zBeoce!q(BOFD<|4^}){ZZn_O5d%Qlw26~>J(BszgOZ&*uN_Lt1-eG+$`~Fb1CqB@L zxjIY_oyr~SbqbNIJiEsWHfMCOJ;&lLN?OOMblTXFzeE=w6aCD=WKIQ( z|1ypLaC+G3`Htz0({Z6MOHY&uCf`qt{Ge^krAg1>Ug6Eo#ZWsh?#DDch$qr>;x5e! zK9sJH&8}F1y`W+nSA9q8m&xhL>)br5I5ob#14gB!g53N1c3p9ye7u>-i5>Tvy8p}8 zw0qFKFuPH>IG$zfL!@^&*0ma;FXZmF5g1v~jktDO`$z4>(+f7{G-#^&Hx4&cmt8hM zH`ZUQE-+*H;ZU3G80gE%t~B0PC@fyH$9emXBhkL^j+R+4zye7oIqw6=tl}~@9P<+| z@@62248g`QLRaV%NhlJ)vd9(`Vy*?hoHXBO)%p?rDk|tRAN?0;HUZL=tP#4S5xe^# zKqT$C z>*JJNgxX*j6W6DXAu#^-1>Iv+FkAG=L~_L=rq$#&dW-N#r}gw&Mm#Z7e|LJj3!L8)tAKOhS~ z9wsu>w$^9(BJ{UQVln1isYMtyLvX#U9aIeBb0bo71`Sz7;x3!t{iWuF;sh^}-xA&h zO0^dYVg{*PrP8ScO!f}E5F3EuMr~!s?#@w(5L+7#&2*iR(bjz!T7zwo<6p5dftflDvGiONknp< zND>j78&|nSL}7OdQ9e|_st|R|c=5^lD^F@?`egqZ5Mtny1!`F&$NtVzsOo+|FxD~n zh{)fE`fJNS1>m+2NssO>ZxDiqtOGB19>wZEviM%p9 zl|C#T;73(@#PL%$c4YEw1^cBMGm}2rnm!%0d*K7Kr-J#DJK7=#Y6+NShWufs9rTxv z2F~`cN+6{3l+pv+yt%5HjqqR1#`V7^is)Q%o)^rI#I7&?!mrGqO#^lPSNd^^A z)-H=a_-o!cxaU!?A!J>ACYP_^A#Xr`hNWyXGlN2>xc2Osi?Ft(cK&3x>&%z*vBqO-xTYHq!ziaB#(Vu+S$`#Sya=OJ*%PmTflj+Xc1?#P*|0>y8BSG9 zMGjiV7OzCDA?hMIyp+eWJCeb5I+FP-dUTS!W2Ntfz~u4FIjRsC{|TC)zxWeK z_fz=kdwKuC-y@Vd)5DlIL7q_1fumrYszzTz=lSyWKsRIahUHDq^#khCuWADs{&S;K zJV0TT`&2JkK%hH~u&!c4*;7;x&Q$5SNi(|L zFPBxs<$CeD=J~OWtEK;t-MP-QmrEu8S~8COU3L$&zarjWdZjt}4lp#|If($1kU4@# zO%|)rHr_9fwyK*ynz@MfDJ=ipI3w_57=Hzw6pX!4Dh@ql1f#-RzSmv(AD3e6u&zw-^`g}C_VBtRq~Jv%)sf(3CO2qkHFY2!|0I)p0y{- zwgOLq*>lm}Y)QuHSjbl1xS*>j(`u`|w=fG*SdcZYE&Y?r0K`{2{v?pZ047)@fy)V$ zRh~qUN?;{l*26TdoS;E=P=c)8Ph*6R*WCxF@m%eIaaR-roiAc|$$#4+Zwt8%#vk+P zV~GWM?eZ|{HrV9{`_Z137Vhv_HpzVg+fYRy)je$li*a(h!ri`Rgp53{-j?WQ;; zVcIiUu*X5Moo3N=K(Q&0Jt`!|Fz`IhJfg#>Bf*P`B@u?(Wci~Z+h`TPV{yGGRMNM&^~sw@tW)f3O|ev^=?mnGejlyPt>; zZMY}RqfNm5u#aJZ4#kjLmEmulzzfj2WAG!hlPdQ8tb2L`DB9yF+Lo{lN5PKX{rL^% zlFDCi_1`C6YUN%zm77{*pG~ELH|?jEZ`k6P#s-BldH)2`1z?0EODc#?pHf=wgWX0w z@nV1f39DOA99@Q=k}HPn4Y?t?tJbvPKVWNP+(k!qFz3!|1&# z5&Gg|l#>H?wTLiHaAyVR{Kl9yIM1!=2NfHdiACCG`ST{B3D^erHy zT#|k}h;70y1SFaCtx^QAOR>m=Uj-5l3yy?+Z#L2e*(x!-P8kQkt*YN z!I3se`ks(J*&(|L`&5v;6a!|Y?YcouBxb4mz;O{QOl>Q_DDsox5b-Bt4SqTTTE+WX zMxP5BYAwo9Xd)hS>*VZDkZ@-ccFmABY5O!GeJJ|)rElSpK9fNvNKR9CrIEBL`tYT1 z?IHi4L=FUIo*f`sq_1Tl*d(o@0XLGju}GgyAUjF=0s#}Gt?B{t(zlY3d?bBkNS*Ql zm`I(X0fnTc5)8Tlzg_)5JPbJaT?9}8nfM2l;mLP!|F8>!xed;2N-6&?oigk4Xc=R zv46QDBS1-3uu^_;`vIoIKeo@^Al+qDIy^0s>jAVa>Qv4lq?0KB3eAZ{DFYUuCEMmi z&LIlqBS)2Rz-dxK_92?%z&yg|VqzvTF>z0BXu3~0m>%#@o&Abr+iTI{>Xq~SW`h8# z2y_Tve!NdMD`ll4v8OY?Gdi+7Q*OLIT&LSkOydn;DxZ>W5*dNsG@*HRjB4dFfF6nF zV1wA6UNl(E2+h-LKy%b<<9l6^UK)bAw#l5c{5rUJK}dxD;Z3lf1@X3$VAk-jKz=v} z0DHvh-Oe`QGBcRL1ioBggu=zLSi9G={q?-cs+g zDa}hv{ewiV6fi+ah)k(*#4;>V1d{xxpInJe-_mUX4^zYu_0EHH>z2YESZDsPSEf8m zH(*(iVK{bJ&;)x_zAF`w2(N9qIDzKDoffwW2F8*A4upLNk@O79yHTGeCE6mh=j~UE zVyhKUJSAZU`gQD+Xb`oSp~$NOkVw!A`3F&dzqJBDoy@D5(0)ajmM<^o&<<2dY@}V$ zQeCv$*Gbml9jHUZOz#Ls{oy;ZcV{HucM{)p|Hx18h*dLVm)A^pk_brV>9+pM?2>Il zMjN*j+I?`^htb%314?6O%7@@o@05bWgA>ynC-9hTN(4pZ#BB2(I=C&3ro4ReW2FCe&9)-mu9IJFq<>Zt3$hRY|gLKOAnsS^q%C@yC09MK~WvxUoi^giho; zk76DER~88nKq|ytBe>Tgf1g!=Ru6852fSC=y+AN_mk-SMP9J61X$%cEg)i*>*&Eo% z!CA}Aa9OnJ*+SeP(l)97gxLPGgY3}+bW4a4v4s)-f>__^*gN37YI!T6k%gsZfS&}| zHtl;#>$f=bK3y*wkDIgo1DUfp4CdKM#~B=JFhgI$Q{MNQ2WZZX8nBU(XV2;X7kmgc|ze5Ws4A|UcVQy zJ147`Pw~z0xb!mFEsJ}fzHWqQQt5?IAcuh!d>KO%>oMKdDy3Q`-Np|~4$+f|Kq?bi zErC1)4o9m#WQ|MHA%$tImhHv(TBN)%Vg(TcKu&o=T~j-){Gcyol+^Y9^mV|vWMep6 z$V8QL45}ty^hV!^#$T^DRC7Q797tujVhK!({;p}$mvZyiL^G0t+MROJhFmC-nzjp9 z`AvBn+Yu<3cIcC~X*};MiPEnuw#SK%i5HBY11p2U^ZX}*a=G;sAr1v1VIF)Qhuu=V zG#T=GEh7`5Nc&Sg9X>~#g6E#V?Hdgf?;n27oR8aT!j5nRNa8&Bf(X8hR1oF*jt3*o z7F{|%oiF@Klt8P?c))Ib;@LbYRM5l^uU$~h?lQxwoTr$?(;J~SGxmii-)sVWNdFKL z9=ZgB_=o;8vkFVK!ki=v3w}FE1fFhy<V{MfF3dMP@i0wa`h|CJ#)jKSfd zfxjr5VgpwKt+eQuwqpS$sdPF{TDsr;p2j}k;8wu*8e;tkMXu9>qpi(QmoQek zuomzEYQQ;D!kaYPPRD!D*4qHDym5B^Mh^*&Yn^l4^oi*L-cmp~W<{mDuj*1*^p_8V z5}%!jkgt$Rx5OlSBND>VqKOH4b*Fzq%xRdF5?E74ltPXiE+29&Xd;#7SyKX%Ah9MQ zk|aB8DM%fW4P(%ul-xee4SDG+CB}~Lo=Q!V$*Edo239$m0vouISSH!zywe)E_s&sD zZ1ygjIbati%eXl@c&Bsem;YQ(ap z$D;W0`?W1ZzlXPx6QLAok}6tv<}6Y=v7MU6NVRq-?O zk_)5%dj$J*ns2jkB;)eLmG{zmYTtl*a2<>LqL5{(tr}pu`6bM9fG4#amB|Z(!65w) zGc~gq^Tny%bh92Z!+u77ziI-2g2`F3_11k?h)~Q;n@}be)pW=r$O00};2j~Ffm(x@ z2hJHq_K!tSumu(&y$&&o+$@Bsg191?Em!7P0FpkW@6lyU)Y}Tk2zVXrvDOO@p-Ax0hhvgqx!5=wW z_ii~`4Hp2fZ?P`Yl_9Mhn?2fw4fhLKbSlN2iy))Vayb-I7h0b-jKjDQ+hm_@%80E) z`_CP&F_hUu2M0Q(A}4e?ohDe)@kG%N@V3yAd8IV&*u!b!wl4q8#?q5_{Uhm_xG6ca zIP%e$-YH$Ke?dDW(>uH%aC*Z>^*L94YcU~a9f%D>v)0njo3VPcQ+~n&)FwL$S2g-q zm|S%stnx(6y1JJgsgGM1zXEi_lb>gPyP3N$PC#<41ESSXW(y3Yvit4wd#)JjRA4>} zbFsPq0tOB&lJez%g}dN@4#(ndapS4LnqWKsin8GBqkvhF6;`J!m4^eGu33V}TPz0W z5mv=Lh4|zWVvW8Sp;VG6CTQL%LLn~pl1mz{3Kc*3Z~5MYKH9f zSb_5eM8bRqp_-6vzY9UVbtkY|uxU83c@ffaF4Rj?-Cbx)H%|@L@L34J_D*g83}bss zv2C-&-3U%e^wq2i*QDrcB2zBNjPoc)t^6$-ote1J@DV)6{B%*BYq)TfbVJHI;L#*!(K!XFB<_=+s2b3CWcSI=R4xxk>h<__|)>r(1Cn>z@(umXw4Uw%@ zROJ85X5XraL^(V_fYkZ=kM0Jpy*6NAiK=&a$!!%>Mc_%7y>TN7pU8?Y5_Olo8{*_- z1-|cN!p)Zvj$au}QU2Q}fdkGq)3dxA3Z9!-%{N>bKah1R>;+x&iBVk-xmgm4eE*8` zR0PW5fvAA>*ogiCO^`mn&w~eN2<&Ay9}s)<26^^ZI$b|dZy#`P#%f4a6F9p+nC$Iu z(VwdPY^2`{ZfB>lKQ^B%E}42IOj{0(dWkK>e!vDml@T3bOWM{ zqfH;^3_b2uLGP$25JqGGen+n}?!2jeTkT)n^)SNq5>t>5Ior=ezqY9YU zs%?aa7{pYzhtR0TgI5CwM`}zzP295E5e3W_m^}ujeW;LIc*WiccB+ZI*V0;kWO-Ak zc3zS3&D&Esp2x>p^o*;12%jpMFO!sXjEy}>kq5yxh@6M) zpzALxkSPFFJ`F^OC7#k53(Dx7tjTQToo9hbwjP_ofOJb13DJKwBlrYaa|?A>&3@h@ zJZ?Acelh3{t;7@?3HX4(n(w!06)GT{;|l}G+(8sqy90AV4aKPBFRb359j!LR49A~N zN*LXH5Hs`4H@Y7G!T3ILEI8%pzUQ|Y>@Y*dL8bmN3YfV%uphv#@1;Ata~%BgL~?Yb z?mOu}XilF_7S(3dz5cRoUF?cZg(cGF(+?aVj-H-~Lxqcba6kk`kGpY*mjhLcyKEbpsHuwBGR;h(+?|!!#dcDBJcEnMSx zsZS*vs`@;uhsx1M8b;Nb_^?H>*Gom?p(m`b+$KoX67W6=g~QLquX=i<+Aon|gmbxL z+l=jl!a(aYw=J-ZX|K-*1ZwMMz-W|04bmE&Q z;%dllxn^L+{d)z|Np5UkIl?M0pT334@+UN-Ix5fxz*nQi5lQ5pn?lsTcx6hssUf#RC7thS4EYd z8bB@oCrRFwm$V+%4WJC;2DA+0;)*bY|;hH^~w3a8EuvsG@`mXLDbaF4X&OpG!hmA%xxr zXibIl#a%GV)lD!fTNmLZt1UGKt9{)+O2%bKuw5IuB~;;-7S8_-R+KYq_c>I zyf-Py8EscLYpJrouyi=*lKjj6!hN8jdEn``a%L%ybpB76MA>X$zyIRWh#)G5E6dJV z)@If}2VLuK{)M)hw=>R!GJB86YX?3(7FkgA1?Igtc5x=+>Afj-&cuvdupCPtiiI+3 zpKZP4w}Q+)@bj~`0*!P#iu=^?P?u>o{vwAQ>m1l@7)nRgdLvJ@@X&B>Zgt}q{QSAs@ z0x%l^p)FyQ`rcPpkbaRo2(5X+ZR00SDiL2W1wlK}# z@neA2uSgL;cm`;#dz!yH$F{x@&9;SC8|EL>d0!pX1z&YjHSt(UQ<>NwVDMRbEprbk zVn92BOpKZ_jsI@{5qk~h=3O%p_=5Gmcr2=g(s!pf0(*U9iIJ-T(r57}%@x!bp<0hc zqkPLieou%FAbiU}`H}_W-*x1^z3=g_eWF?Pt#!IjuPoqm{O~2_v`i9yP3Tknyq^ib ztI2$Xl9yX={jIm{_ASiZA)<}3^|`yUE$+Gfd>MNDL}F89z%&_=*8+{lU3bO)M@w;; z(`9+`PiS9VJdGbVHy5S{K6mCn?gyqPHTX-%!iV`2Pj9pp({r~T+gmG4s*rU!lC zX6V+F@&6ZsgfNt1!<^4J7eBi#f0o0X7=vBV#!@)D$k$h|3qDWOnvaDR*n<`z^E-#V zw3golO27Kw^?eW$qV*I)VWv@O;|Jk$_BS>ox0>*h?6w%fv?5rw2y*;Ipxb5Flg*ug z?KjCjJ0uOPUFiLGR!f?`j$vXbfeoZTmB}54kts2_W$SELfZQdmt@uIiON3u0=R<-X zn{HwalW+`g%6FPcc!K9Akck09jp@W0ez7)u3R)F`{;ZkwaupUIEvCbo zgN4%uwj__P+Fy;yngtU=@v}@C!2!;E9*{D1fOPvB6am*&FA8l(P$MBN27)#@r=m7% z0a61ju56EBA7K6Xp<`SOHqUXNW4>+HMiYc>_v$1;x8X%Z^KlurW-sx)fNk*{et7~l z${61?l(2AOZ=kZd)q^(U^qfD`8i$6;WlwD-SA^^=1)W~`2*k7sb`(a()KxHK9}fx% z)_2^F+5(}uBsxNu8=h~TElSP-3G~pI?zET)Gn)XNHT;HTd%F+dPO2t7BK)A5Fn2U& zMX~7q*Nxq$%_+37ZM#lh(v|$q=FuqINy-W^9at;MQ9=|uZq%&~GIis2$wix_09g}) zb1WJ?0gRTi(Od1a+R7&kn4OmJ9|zZ50bS1YV{S|Mu+bG*gK?Z^H*d#zuxj345DvEVxqtvxX!W>PzdyT? zrQVA%lu$TO+JZPjNpfSO_ftklaG_Q*Sb4)On#EbZPG2#clWi}~jdeRkS-$Pr`bbVw zD>`~Qjt=507R04AM({Bwm@$trms@DqegeH)GrF4+%m|-wsahk8?jrpPDry{edjnQoc%}&1olhQ*{ z&EiwwG8hq_`Jt4j#^x4lr7GucO>FO`=2=bZy->$iJHRiU@IS;>7WoA|62TG6Z13em zLQ0i4v3I;~4mHD9-jlG_V-Ky+@nzJBIvXMY0BO!sKa34f#!Jh)@g?K6Y z5l1xr*sEz>!g`@vS{@@AqNmh`pt!x7tpcns$;u*d7m@1RL%nIQv4vbcAjZC(1}|6l zyMJergpU}+?29N!`v{*d*NX2kH1PXwo6nQI&fukiH%voZYuWfYJ3YAl2Shz@D7(*U z8y-yfr#1*904X=n6Mwb=8Cbw1i0jHa&F4LX)|?Zn;*Uk=SYmkLGKc0>H*ro_Vsuzy zY0V9zTWiVpH$gHR;iDyw9 zF^DWP&sTprli{;Z_}AUSSN}ORV#5pFvw^v$L%W_jE-IrZhU&nkV}UteJ4%r`SpsOA z&S`mF#mHX0)|=Iw(n;r5AP$>MH}5`uRC0R@%|IVEE3# zc_`A&^&q?`eh$T*Mgr0eBhaZ0V6hQMoH&u+nGy(qjZ@tu{|vYfB=Bp};SJ4V0nCGC;P-7h6`njhqQMwb%C5Esl% zX8K~VON(^j_3iGfadLzw71SViqiQyH@@0~Z|ENHe<=k&Ae@mbbA)UoHr$^>d7>cwf zgTkjQl6FcZ4%G3mgiHQDp%y*zF|4*Xpc_;(`|yA{w>0B4hfCil+V%YT$M6At_SF6L z*@ze3yo!Ih0rKOi4o_y)apXG|)|~JS-8itUaqLst&`a7tMhEiEh#;@zE=xcEQ|}#`edFCj6)6nSb6qFyNC!askhDxZ z2u}13mrg;7hrq$-)X$>kF#(J>U-su75A2;ihIp~3O~NbJK{NW0uy4=d@N98S{2leF z_IE^5r|>wo>M71-{x|+S+Vc_RZ4sH=u2E-tw(F^VpEJ|rO!2yE@G)@K9K8IBApOI4 zfv)<(sx3+rzOetIQ(RaN3hUHqiO5SnRdPc1$~q9+=^TQ!90`S;z<{St&$=G+=gbqk zzhpjE5M80LzT)C85gF=~~MkJr?6M#+N@!ciHxoCRT!QrDQY`OQHlCg$p%p3fEAois$-$v zgUF39-fL>+>6d8=NYd30dciyQ(~pDl4_3)b_}|uRq-ajO?G?vKMhm?e#!yQ=ct-Kb zZV5*44TewA`#cKp=us&{5g#@A=d#ynu*K0{lRfxXOKwahXiR;_A97s!RLc%e}9c!)|34*#pTEKQqG8oeStNegnJ z+4czWUXFpL#3^6@MozC`#)UV8u3Yu^wANj6+q#xHxU4};mz#ZHR|kLB+!|6F@|Nd{ zp%o(28}G1F%>m9iB6K@gr1Lht`6u!5oh7k2pO;KvAl(CLIg6$(=B6%=2>&G){1vbc zFW)Ly=eKIs%AgtemB&c>oY=H3J>dG-zT&I=Dt%CSR@C>JaUQFuhwB6D z3Uv9kEvt`ki_omr%hvT7VO~dC23>HL-}N!_PB;3u)_|OwQ}lx`j^CYNs&f?QD}(se z)8xlYx0;kHG@hosUJvCvH=?FebDZj3)Zq8le5Qb+OHa{*UDrPNp%2weTeK6B*7#3r z_2!YEa|&Ury1+$UE4uI_uC*|FBUx<3^$^}>tYpK}H;vJvI-zAxrka&t@$0b&F%GTF zw`t5xKr~Z~^8Z*EZ$|}jwl4I@ari01SEmTf@A1V*i)sUMC_2O2bP@saV~v@{Ud6$a z_m|r@iNI8+a1d0b*fER6Zr=+*t=SFSPl5yNcupuVoK zL0G?TK$Ag(oiq_HUm*~%0hD6oK$_q5LvT`r5JP})DmEQHU4|4tLw5N1x6Q@r1N}JQ z1JqKexIbq4UM8cjU$z&Xa!m)SIf$v-m+YAv9Ayn- zWwbSyAj&uDDEO~+c;cv4UmnD!|DY={V+4CJ(gD`Lfu2JltSRy@b%_TU4GG5xj-(lH zBo=sHmpgI;XbU<~U5Ug;$(|_hr4hZgY|oPeAIuNT--T}?`;YG?y^-%{VNGY8>4f<2 zZk-Ue_HiVH$#;&f+J)Uy54}`dr>E{s_}e`+MQtX;u%Ny&Gm`w9tKNjLa76PMpwbp6Yk@%_z^_c{hUzjcmI6_ z8Ry*OexDCLd_v$NHOChW@gB9Yv`WKIjE+Rx#S2>Jw0m9H@NdGKp3n_-baN~;y0%+8 zWv_LxsbATvJz-(zk_U3-fIsoYJmJ~&A|g3>yp7Tu78sVd9gK=88}R4E`9ltD%OQ6N zU=@TSdt-&2b5Grv4`E3h6L;)_`10~dHIY~PP>b%tvsN+G1t1wqXteB4aWWRN2Z^8H zE#N4TKs%?%$y33I2>Gi$=)>8eN{`PZk^8!uXEkV@ars*BNoJKJ|Gt6~t#z|Urq0;9 zkTd>5M}SGBz9jmBGzh;%)RV@pAJ0SLgO|7OpbPss&C!~CSODjvw-2j}t zZaWe%(3J^s+`6~T#*;NLAgP}(%hZm%3Wft6`tZT#K^6u#LQs?$IrhMFquM+>P%mr@OR%*C`S5@9!%;}9rgY5;?pHUcmu1n`g!>K%|wNgf$_*x9X> zL$q;)t2=t%`|azKor{N>DVMmX<1t6FZaL$OMtk*w%;33O3BX&D9- zWXwu}bYNfa-VDX=1v;ByHWNQNdSEUeVHdeNGA9-1(c?G){y4#f5 z)1LpYs8Z7$GNy=Bs5NqmKH-b_h4qK_&w7H=dcecW{C5Jq20h%GNK` zFzqui)>R)J_3c+Pp7LoAZZ9#mc%X?cycNIP6Vgh&Yc}bd`-+*APYT}O(lt?swyiyp z(n-;=?Bdqf(j=6%wB~zNsFsP6Q+Jc&c;e}8eQ>jf>G_g1mV(xY-5M3QX};yj+ddaQ z=omwk!@O5IEJ9%VZ-aTY2PII4TU0Q7x^txg9twxyg|{E@=7VP{k&o^6ExGfSOi{cS zsbceR{90849|w5EjbP0QNTRg)>?r(wP@EL+Y_9Riw5AlXy?w4qT-beo`{$atEA2L_ z>j7llp>`*xO|rhOpZYJ`*=;3=iN#4!5+vu>1sl(|YKZvwd}EY4l6QvoeV zf5FeWk=ixP%qsOou)ym;w~NV4huUcbQ7N_k;1^)(2qU{L+qbY1$`_iGSYGL&l0RS@ zLc-nIZB!Lw9HMu^F?0gCU>DTB0Rkdbw~PxL?r4VAofJoJiLO9Olfv;30o?(?M*`-P z)~%}L4pZj-vc+|YVt?O(`fF+q58Qwr(S%Q|sRPLMM`gcvF0*~C+atBuKHt=-+F`yP zqqm%G2C)-0um#r07k7IH>>82J-{geyMBmT0SVG#PPkYFYLyLLJR0R{aN`_>v{~&Tei*X8&>Jz85Jxx3-5kuuA36B2XI%Wdk-2bq}!*7~P$HNUJu3fQ z&4TOWB>6U&xkF$4wRF?%?bd@fe73NAj0#32%I#JM^&G5vU+x30zJNNP!-1exnDZRn z4zc2gJM>R!F(u~_?&p=A7q+bXD>J9^GGph_*aZEnxP@@3#th&o6RYR5Sd6*`B~Q^x zt_Some{%kxq1sPcs~#SzU`*;|Vc~w2=qslP_^2PSY|B6cdTKi-_r2x4_^uMWnfL>N znVG&k#7KCuMswfm=9=zlvkl3%{|qX}fg`wOsXW6b_w(`($xg=P2Ml~F@sz2pcVTSz zY58rGH@@%KZ_M{}`y0cD7rd{>#_IN*_ocSz-Wc(lhL~NlvJo`gLw6Z-)B&BZK_R(9 z9p2#tX!t)_i;i3c1xnwufghiwcH$ZMY%|)Lrerg`GxonX1wCyUZ-3vhk4j1X(ywEV zN3P7bvv+;wg8-1dWdIbb5rPWxgF$i=7=V-Yif&s-l)CmD{j*Y06j)pGEk??1>Jb3xZID7=fju#InI?Gt=xxRSybsO%!$e31p zrzrG={~c7-ka)UoSIK-a!WjL`&wNoRQ32e#XT;T;^m>5F zW2jK2%SEXl`I@-DPh48zGY%PUjGkfWj(l@E>zMUoRrEVO3ClG)G#3V6>qUB?ReB#= zw1O8aWU++AV0?GHyT}G8%JtQ>1QUF1OZ)7Fdu8+7y8alEFEL%pVj^CTsV0XNlZd1n z>^{4Rw|Tu@CTlO$!|p%-ZY-i6SiwfZz)vD~=-YJ~edN!cSqFQ@xH~ z?NOhX;K0>fj}sBVkG*q>fM39ppKPU%!xhI4H?Kmqwz{d~q;Z|CD(%hw3E7s?=>tUd%6V{r( z?e>0dXZ%PO$lHCLXWa1@4Gc~@t8vQUn6;f7?CXCS=)Qc-<6{TilzqQuJblN^-=J^N z)qLg5Us=t5^f^ht!`@E&K}dIvEA( zlE4l>dLI7>Q{U}Sz0s-e&qx6b{&?d0mC#^(%k& zt?B%MqP{sx@YL&loWJ1 zxBb`vI+d#Y;Msn^nK5E$(3=@|OK)Z@e74s&Gisi_!kZb_5_@zLiLJ}~I{zP@u^;pQ z%lkV2AD*$-Ik{^IuOX;fR@g_Jjt$S)_qn_43;T%E+GYKjySp|;`(xi`T-;?k)Ay;d zCl~Z%MHp zu3;1feUyezmVR$FOz>p+kfnW`Ro|9>)hiIz=l79r3&S&hf1HO9-q`X?ADL)bc*edj zhj{dvJ`(Qw@Ql4Khwy94^IejreD)Gu3-RAedJ5yqmh_}Umo51hdAFk4E@Q$sA9ueV zN7Zpz-c=4Z@6WsH!R8Q!@~yuOR}}IVMcvci&?G`oC%>ur|HYx8_ckE>ZA=mMMzC*r z+D{sp?89(#-J-97{#P~T^X*JL1sfXduk9D#T|Zx;5^fqNi4W8XdRI~Vp5S7&pt zdi-EdZXa$JInjpkB{el3$Nzv zo7c}*a~$)@mYUb_R_vx~|5oflFAxR4Cvz=ijcR zF*06gcbX3DTUH;a3So5{_1kT;zf!wWO(c)?hs6TIBILB$;djLAW-Pn7AzDh}#| z^nx#oJ5&YDpD^fuY{J=I+~R`Y!oU@#>4lc!jvpD5Jd1+MimJ7V-^buZ8(SV*-0@gJ5?j6-S|~irv+wL*+T^Yv<=Jwc0inIQuwKW$vtwx;+ET?WDe&p}x_5tIwLvl+^O9wqhOhQf2%>aP68v|5XJS5yvg=ZNQ+y5#D}E&RbU2 zxqkNtTti@};RQzA#=aPXbL~Qo4Zmoh-KbTIdh6)HQ9=7N>xB#a$3BvH6lJ_v$(G9n zd$GN(xMM`&9JahxH!to5J#$Pv2G4ZD5pG@dq{obdrT!qjF`?jw5*u5x6DpbXBjahe z(+4jwLlzLE{pP}b&`sykT~NTYHP9)j5NaFZxmltww=Ro;ERzU*YbL1*&jES2E_Hd9 z<@Tw*U>M^)qsoHeE4X9Q&Bk+{TZ8%Sitve5dZCq1JwmX1&)bNxFW|a{w+y{kP;~OC z2Mg}uEiw066z#mZ0qAYj$AXC2mr?w%<%y*UC3^~f0E7eB_>7wTGr{dwA-PEDb(oFg ztvsV{4dgJl8i#F0+7x#_X1ty)AIyvPl08P3XDG!-X#!NNZuLLdRfRBD>+fEz?fAlYOqeVjW;>QJjy-B<#4mP$5GH_v5of6?$jW^_#ETXR- z%;trz&9}q7YiKkhvi#pM%w@m(P zhf>D4lGyppqM)=tulYBbvlZUYv`0h~EY8jQz0cZA6sTi&%9OS@J@ zF*38wxQ8npL3Nz3%==tbG%7O!TL7nJ^eNE zbqvr5o3K;t+P|XKXE(caCTAVAqev0#kb_nR`3i3fv1j*L_3z7gcC$@yUBuTBxIjNU zNEuVd!2F|xj%L*N^r7@*K;aT-q(x_{qJ^Tc_g21cv(yZ`_GKdP_gUl0Y4rt>s&Di> zJ?SULS-=2<`{xY=t2u^UYb7jM!^@{6Saq?&KsP3kswe$~HLoYU5?bP;@Dd*pM&6Ee zAe?v)xYE$y5IUD6gJey5rr_R>@Dn|g78%t#D|Ce`23A@m95Ptvr!aa})2OcH*v*O* zQ~UB|cesD62!4&!|KLXVYIp$l$PviRu6w}@h-k;uRcZ~Za_e%u;#r~Y(j9bRf(+x zZ~$rt&A%B8_bzy3kqOmq%oD7S^#T#}5*t*88UO?zH4|I%!5C~{1JPFIRtr%K1v);X zK7XP?Xfp^^I>89lH;b>LqJuWf>&M{7evI&&8H;EJ(fAmWXirGLiUBdoYWTZ>C;3LZfce^486Zhhl*XUQWYp)M3njqY(aHC@c zVxmWIK(i1hmDRMEHX$>}HbzVL-7Ax@uPL?W!AaOO2aj*Y5pmlm4{zpENI|OR;YUs_ zW?C1>L=I^~*)_e~U zWvUI&R2w0aIwmX=NeH(g0ah{Zg9mESy)!C8w)1QPX!qM}mRHeaTFgW-ta|}jXiN<| zBmPm473$ims?FFYv8y(stibl8^ysb1trrA2GKz~m6fq~so{NBJY6k|d$Fki`xK^U~L=4bGQ z$$7-707<5WZ{SUxP|tc-cHi;^jAzdXb2mppWix zL&M?pX~nfu!okG^Y;jXuw-(cVEoSx9BIAB1xd%@92=NH|4`|C?xDW;~CD`+Ovp(7{<}i6BZZcALz1|a z7k3tU+((oV+7GC6v$_))+9p&345Z=cap-k|X03L;--!UW_#MFz;)6=Q`Nuw$OgUa- zAvlDf${js2a;qVg5Zzm#@GC`2GP_N~mTvtb@+kO!w+@GtEKp+Y0nY}v7W z`*xoFpq;lj|JbV7!q_yz*3^pPinP)jpwr;^;;a=1t^+4@6xLKl-^P}=<(3(QI-%ub zscao@KMiGHW^6)RYISCDby{X=1~1m?#H}Xp0HMBccNKG+aKP?L0Jli$7=RpX?}VBN zmF-Ne$}Fx*<4V&21b{%;UAUjClHF$aBnt;VlJeK^b~oe~Apao(H&>dCAe*`d2-^zx zyO%N%vFFpCC3nh$#n`VSg4GB6;V+YUs}&Kb18%ljagQmYw!==el}=OAzQ{MZ`z zTc%dW_8b!fi#yP7wk=gO9( zfwVxB^Hbm(_Z@>HGXW(%JfuY}l8_R@`MKV*j zVM%~8T0m3jAILb0Bw+eBB48jeTA@>8A-SL$5UuSa{Q;Q!?uboWlP0w=G#s@1(dAYd zQ~mJ;c*e(AwmfbySIT2Q?JkAU;~ClGa~?CC4j>_t+`Io4DaP{C6rlQZGuRNxC&tej z1d}!#*0CY+!&ikjKs0kJ&uVHUyQa2DCT-CV!i-;Zi1AZaDt*Rp$`87mvFYKC$@B=0 z$t&GG9)4a0pHat_yrHC)tnDDr>bC+f{#1B%3~8CbkK=*;DMmd-+PN?HdjSLMmO`S}^FSLT4 zOWjT4JJh#ToY2XZ%ed0$yGa|y>$s{c<~E4bqC9e)ScL;(@Q}0ql8KyQs#QN2m?SGo ztTuHp#`k%6z?mwdz;D;0to#I^Scmm%?tYagu1pfb<*UOz5bnfX0;!wiGgxOwk;kUa zs1+kL)+t`l*PSZX6nhx%+1Z^1wvzB^B*z0nObX{kXoaV^12_P&I=JDlPorvDL;-SO!=zBl$mG1Tlt@% zy?SlQXZ&8PX@s zXx5E^82oZ(3E_+AOI9+d@7i2*&-?Wm4qJajwd1B|h%3|Tn%Rs! z#8yzyQU8GiN&`nBzJ+ETah3HJV)87+eaivPF701+i>TX39g0YkZMsxszo1WDhyx%X@F32B8r;M=YaI&yT4Ns zaLg-$XT=pyN|UiAPorHGtH1}#XwCOX!*1?Zl5j`~s&_HxNQGvT7mJ9wQhaI7ga#~T zqAYCeiyha4Y}E*6N@9X4UZ@1zXWtkZ_Y~VebmZYuIWJby%B;L7SZS?A$4q4CNM?+e z3@7`d44|!&-v-&ab)3hgYee0vcjTO3H=yS2= zhUid%jEIju`(y5g>mmr=uR%$<=f@KVRz;mj*F<1l&QvsPgfRt9yo@#5qe+S$F3a(9 zRZ%t|54ibV(xo?vN=dEcO_g9(j=KLEp`&TxL4I0H-!p|W4*Ovm_&CQ;F*1*_`@=1< zZcUV=V9R48E7%WWr3{4wBWGALFsR*ubQB+2NLmbyTGXHdQXb~5%AcMhGEpIsM%aLY zDtr+VpKPW4lHijEH^_tJE`ccNQ>!EyrJ6?4BW%gdIAU$hV))586T1;L7C2Q^d`1;( zx{e`w9E(L9wa|k%TV3TPEtpY7>r$(9tGpuCUv0yap3Q2x*Q&TvXI&wS;;SpX#Cbc< zr=H)OCZprwCr{H4t6R)CNnwpk)}(#`H#kATTNTOr)E3@!ET&)-_)_XU!Si`r`HbDh zb{$j4r%DENxC!ihMBBkrCzUs~#{_Fy)-;6-_7Pl2Mg?a)e5%{0-%{K^1~o>@M0n1Y zC=!UHRdR`yk&$AqG!tY!)0?I<*}S}on8GNv&HWi0f!vk>LJg`UyWKrfutro#K;J&= zBR|pY`c%6_bV&}`k{|U%ZaZs=4v?D~kzNL@0;I>a4Ly)u-Hq&9{bWDq>J!<2K*)~% zawK}ca(y_x4T6_iE(_@aZ?XF6E>Zl|fLf9$o>@NDw25dx*4zXd%5wwF8=Sri(uwQA znjP`zT>JbE_l9_cfm8(W6D*VY8k4P^H`!wOjF$Wh5{E@%9kUb7xuaVpf$}w2p1>On z&^=JX{S@Gp6Lxc^V~sj1*GNsK61)XYIM11m%c0+MfUcl~)S>}LVhdvND@3tKf@-9~ zMi=o<=zNrRu-|P1R^1pwCb>1G@1wKe2lp&$G|Ae()EGCLF_Mk%W)EXR?7bZ9{cj`Nd%tNb8Qp=tT$pLQs__E1LK##+H+Xmi=(Kwv0f->0 zDkdjjv8v>c@~PF*5CJb#xQ?GB{4ePz45)h^j|hoxfl$9=BM!w$4CF4nj?BPNqzvM% z4*)uD;4RP%#>at(49S6=dR?ShT2e1ik7mjhVm&{)QYIlbxOVp!1ADZnWk(^-c{9>L z2m7JS{kWVk9`%tjI#&*Zo7^(70wO^XohKSRt~iJHw}CDLulISlV6{~elj3ESO!oJ& z#U$hy%Uj<>4lTIp@3+7u<>ok|*N2#hu8^^>+HJ>_(!)}!GQdCi%}KvNNL2^43|sOm zoYlRJXp^I^2QaJIW*jPsZ*H|hej6^zk;+T70g6jiWUeC?2`&6I;}aTT+vWvCGm;r* zkWQEQIjONgUM154&EJjp86b(;dJDT=@8gG_ zku-a^M+MOw1Rlm1P}dmBgU5jqQp1ZrCB7&olE1-pclNK=zX%2?)YHg61N7*@(|G6% zAz2>?dzIcrS!17wI!y8|G(FIB+`tI_1*rE(v|Rcie&uyj5GCE`ls#MZfL9Q~C%6f^ zZ)w%7>RJi=5sZWT3CKmJuKCm!owXoJREumv z$z&WuV-xg~S8?oM%{@_Y)ZcSGo1mHeD@^?ZN&OK?y#rJKj;VH%I)$YE3{!Vw>M@c! zm!#(NrsiS*Rbl)s_#Fjfp}~8u=dIdRJX5W+A6KevP%Nf}O^AzuLeVYk;J9eOi%OSK zVLYkJ*jrb2BCQaa^v(OhYq991k7YME|*ELv`nf^2ChP`Kz3 z#kQj~^j<$UbWqUh@gPPkqv+Tl`@&dYN_UsXubV)8#wu{O6OG4h0MF9PzCn{8#`ZdB zf+HUVxEd^I{)G&oNW&>5He(A8)vzVw!PaR?@7B<<6^&r?jnS|r{|mGt!%mu1I}V@T zZ`WIMj_}r%JM>oLPL9kjp_lMMEty|R7noL|BK3T61!#bsr9*jfmrmS&ADTN=oih`>sawUf=Ik{56m4aL;;YvxaqTwoVkS>EK1 z!bOYKQ<+=Rs;0^jj|#w}+rW@f&QnUC{Uwt;J{A68HwKeN&i;ux_Ys{_qcb&vho1=^ zeg$Y}@bD`*VUN$lU-}dE9^&1=x<11 z2P+1H=nhVRTg?qBl9&dGzfpS!uU1{ z|1COZFK?YOIDrVhh(+#hfn(g=P+}sA{zS+OqTgveVaNb3AO24eJBd5Yli^y=)P|6W z7q*3&S7}5uwUQU!r0yCqa1b!jIFK!IxJg+3Oh7$nmb9|321R7N)r#Y~_-)3Lzd-^# zmsT&Of=~46yGofZqpo16FD{@q)<0iG0o0mBX@XXu083_#8o2@kXX<2()AV=0Pr?Ga za%$NLdc{ojS`y2u}XJv9&sJhtyi=dVj#@ zbAZqMhY<(~YY{>DJ1SpGv;+$3D~dIVw}T*J%Y#AEl*WSbLr``|3{|+!GsXvkI^+qv zc2U@Pl8;@?y9rhOSS>;Qp*8Q3*7He>#F}429?95%+1NGsEw%9X51_w~Vt=(-VV6Ou z61M3CZRk2bTw|`ow_hwXv8VMo%X^Tgy(AOkty-Z&YLIr$b6sw5e}Iq3N+@v`xIW}6 zqpfg_v=Gq!)L#FhK&cYYpRavP3(vb9eghgr@rL?9+Vlo` zey$<3{Wl=b6F_V3K;RdAzo|a-Sx-MK^F<%?ewE)1`vcc^?+<-8XMcqEHTYF$Tx5Xc z0vNYj*;I2TMjNTtY;t;a4mrI^M$}Rl6tk&|%h*)o+idE}^?JcN0mC}MoJL|z>lh+$ z%FEThw<22bt(xn2VWo|?YDS8lfqb>rMl}UItpS%Hft1uXg4db==L7u4WO$rL0Mgd+ z*2z4v`bMb0Q0Mx^PPCT|1>~)mDn^FRv&cdzi?UQ|^loj7F%F{2RSxXcqdAqX*Xn%l zk6?d^Qoy%nq%bJ^Q!mJONQ3q=p8-=g4)aoq@HbT8Qw#_uEI>&9yCY96} zQeBy?Rq}##q1x0W=i82|8?eR@d(M^*88oGI$RJ+aMqY<+wV;GV&7 zS(njj+~;`!=wpN4A6RA}{yIRdOn(bemXL&J#5bUH8=&%lSOXdN7PcUP)@#+^kd8U% z`60g02i-;3NS8Gh<93e{2MI*oyq)SjTJu>}Q^(Ic#53ELKf(!$6kW#AqD;W%^umb{ zU#>5Mx`*4|#3jsQDsfFm_AZ3uR_%I48Zc|!lu|_o6j>RMoTC$~p=5UcHr~{gz2XPF z1*I!{x+jNgJE9fRQ%HhP;W3a4rgGhN#dc-sb;TVMSKP%5hX9y@>?(*mGb)xPv*q`S zUVG`iVCA>qs)|x`q9TPYABd^Dp;%Rv41{4qkWmhKe-7fWj7qHUKCJIP zsBassubS2;C-q^Qa%@|yV9T$=3{{>#8-xxWnNoS6Q76>t>@Fp`d#27g1eZ$(t6RBN zo18WO2IH+dGURleHyv+;sIw8nGhwf{k}YpAosc89r0!L>B3p#Lf0LlN3&N(mFz&ho zqNY0%ATGK~g-IB7-GNcp9ch@BiD?*f-GMRJ9oe~Lh(k z^d)FZs8FL5_Jghjm=TO7*R<^fYJKVnaAb9?<|uDn${^p*&kaC-3ZYNkjsvZV5&kY9 z19odM4E~z|ejP6T&SzZL=TZa_d|(ImZGvtiEi%;V$>jb!LCjK zyDFo0HG$ey6}78rncLTM#o5)m?U~MPilDWl8Y^;l%UTq|&4F||eqDqS` z@E+pE*nj~o1`At9a%#);*2M{WVR0!CL1S4^tZv{;m&O$8J&8f%yh}IF!OOK} zz?HEYVL>r(nZTRHf^^qd#H&eE7VXo`+r^1MUCoZSJvILF7Huh=Br=rJGRExz9@{*h zQmID(32=V!uOU*gOB!*DuGBU~_@h{56MrAf%3$Fc3{o~J+ynQM(tdVUTprcZ4l zkAmQl=f}b?8Gd8nHvxV;{2qqilki(k$8>D5#(M1ja2#Oc+Zuw)-vMa}KoaAZXvgKe zpaw(fN|`5#Yo!6?V2obqqW+T3lAfZo%t(nvMC^lJ>Ew^4#Oh3r1c?2O$ExI@RNDih zRTk7=PS^?3%j3f8ORh^*GFi0`qaaWk5UA*_*rp=Ib+iiKUW*{0xk37iqoOHvxz=~D zDK>DBdxJGAE3 zo)`8O)&kL&D|ylGkykNeL69Vv zGJApN{G9I?9<#Vy4TR-ai8~_H{dRP{VGH1=xNb0-MWpo{U$= z`2hL}{1Gb!9VJ!Zo*7bcd|~vC(Bc|ee1~UCS;${NUcl;fL+Y&>b-TJwtP!S8*jy}Ca&b0fuFhOe zidSbYk;5nVr;E#YNCPk3pR@Nz(-?%>qhg@bClMtw}I74GvP zMs$6ab4?&dFD%WZNJ;RkRlhyrErk$h7si6eV;pOlmCRdWG?vVC@?4WqQ8=2nEQu9% zTYts~_%MB^l)2zI+2LSX0@+Sczj852T?PeDC{(y z!kHlX();j9D{C%8wS;f)Xx7Excu0p99L1YL))Q9Pk1Lr9hzM*vP?PPd`VW*2zrx`x)6)b# zO>3E!t`RcR;dX++PtaMW1JClAy!Bx)*bm2YmRadyh2A=&E*rOSsP)7Uf9+xLM{9ZO zux&a^5uiP87n1KD|NBU75cPw3(OHLd@nZR2T*#s_`9Xtj2 z=Z82X49?b8zFJP;0@P;Mm;E+KmxS+6g6e~8p=6nsiDT(c@>XLV&J)yef?GZ6&qFlejtqq78R6iBY7gJHV4y*9MnAWj_VwPqF2bVwpC<3wMl{EngAK zgxp2>@1UXiv2u?261)lppRv{WPxl|aI0051%x}K&G>~6IjSgoeE(MW=7mPNxq?pJZ zlBZg%6DoBUO$v}lpOQZ6px!#EMsG>4L1*`Fy=8Hof4_%}*|EkEhK#3^jme;QfM1MY zZ%?~E-9+C90fUO~2OBJRLQ&&xy~S8VUJrRkhQ6{Ug-4Pt;4QH_L7&2Ve$H7QOCfY) zOFqE4KwN#ilnfaL(GRyHk?1EJ1@bXRLEeYSm@bKUrwQ?F6vN}8w7Zf_MomiDjE2Fr zaF$>BgWTZ2C@MxA|ABL|URa{i2{D|gNlDXLV#uWBLNYIDs3H345-}RQN^4n>lCI5I zk&?!)$$|`8M%0;hG1!jQTFB_^r{wrvP{%`oz_`^l{1ZgT{FDT)T9ZP|lnyRbzKdik z?=w=;v4feij5>C01HqL(C03|{{#oL_KwKrG7$4&Vs3chb@Rq?Kw<#(RWX-5fFh+YZ zU=iZ>Vg)K+LE{08Q4%cq=|Xkr_(VoqVF7QsH`Y3hA$5cFTH-oLIq-|1$s?ZM5o)nm zAt2L3d~+G(yw0*p*}vumLFvH24ula-pmA9cmwB1u^pWNVUkt= zgp!fqfRI6bl#YvW?Mkh11eyLwpQVE>e=JtcT=2<;&3#kIj3%)^mT75d?fvpX;vZfp zN~oSD`4Q73KVljo4`?hT4#}TU!>(P6&nyp0GwQvM+H}<2ufmzbM zQjvlao8F!n-=*QyMN32hXLMfgYce7m|HO z2&FYsCulP{xJcs!Z5jvmNNXubN!MhwfcM^_6^!Y6VWkby(>OU~g7jqrrdbQ_WbhG| zn*ik|aAFn81_?%6U56g@4SeC9_d^tql^~Q_nMFf+4^P#6+eSBgXy! z1##mPE!3N?RX0fSH^>PFxR$ux$gU4+EruWjmRX*Gr@KbAX(0Nv4trT|9dZEdTLL>+ za{!3(0HA-y0hLg2AQoqH0l8ol(+}u{A)TC9DOtaN0{q&HDpVWG%G7)G!r~g><+pRC z{qDewao`Q=EegXVHR{Y}Z)~vC!6f9DUJSQtU*ner`K2E5oT;8iwJRZFP()~83292k1OAha###y;!I$N11Q)k(DgMM zOU|dfW#&eMMTR{x^Qcj8lM{c_vhq`if6#fc7~kftaaDSupbC1E4UXfCssy+|lvb%U z!ip+zExMqg^}Hpg6yyX%Y)BQTrY>xQ6M>%)#ersy;c7vg=n3Y3tz{)RKG8hR8{SV2rxe=RXKt_7h-FyTD9Nx{G6ul39-PXy03Srj4qWw}=sz z{DDqlzdPCTbe5?X&ZGJ|&z2XkOvqi7{|+i%!GitWPjyTU*x&o1zIEViw}U234b`vy z`k`0{6bkYW_|$ft5bYiuvO}EKVub!GQfP;N?d$LvVyXijzFFOxE5Uz(^!EbPpAP*! z5~;uFLHqlAp1^ayKm9Gh{?_~Z>v_W0*iWJYdko_|W|IZ2K{}|#S zoeJY&?%1qYl)pUN~+L zs$Dr;TJU=o&7gT|M$N@Npc#x6r72-U=-$A@g`4~Bx zH7DU@Sp}+$yB|nr*VK1m2VJ159}d};u>1a_$L}D(F+SeEs4fqoFywA&=>Z((x zs!pA%I&}&LpP2PJMC+fWiLuXZynmw97Uf4<-o%*kj5ZG;QmVRNO#9RL9Bz&~pOsqM z_`ghyrpN!eZMB*KqZc9w6obu20WYkcC}0eAep(E*pNhp$^@XfheL?Es&jnbw@F-`-sIV`K@(F3@{={XLo;PmG)ZQpJhvX-JhAv{v_*sc9Xyw16#bU)p;%{5zBu;9m=9`{Or221k-u!Ujm$oB1YbjgVa3XP7vy ztpO>J7(^f%S}-i(KfDL!Ce06-PWL7avZd{}rTNlYZE4ME zZE4N7#M{3ToKPL9LU6EjOOm*cn_LyZN-%&-1&0P z_@7@;BE!LKpb};k4p}rI9e?~R9CqN!i`8V6OL0O0GkKgvwBU#C%B2Rr{xJqx47}%W zq-lv|_^Iz&gz-}g4E$xt7fC1!;vxw0MVa~vG6k97IR}G=(hPE4&IEO-y1$lO8QnrGYsVzmu+3wcbm-{&MziJT6)t%(7OW8~Z6?T}S>kYWp@FDG;gtxFJ z(4nk9s@ihqyQ}gIu5wVX%kX`Sh3fQWSmmg0WXNoD*b9D1sjDj!cqpu&XuIFFhA@kpfp+lvMp#W4K2>_nRz2EkOD=KO zv%Agr)omr*+4U~bZP~8}eI4feU(z1oSISg!DOTF*;C6wiHwxU_j=cB8gX+xJ1=Cq` zH^Kj3%kR%a{;T;FI7oy*^VlQ=?9AsYOudFXyw04kO*-s``2y%X6@Ah1WeXlVofXjABO)nXW}%{aoivtVD9)iE>#Q_K=cY;XCI@`u^-x} zGqDijpi^S;wBRjmApKSno3WR)4o7qRB@MV}l2|H@MWq(4pGynB=j(^l8b+z-Yu6;y z*C&9U&@6aicyEv050VEv3O`vk7SuRL-X}2EXmIt#44s9?mVFjVArp*dOl2mFS1>zD z669MGjJTeoK0EV5&cw?TqWd-TT9HOj_66_kR^{@+cGS3S*mUn|Wjvq?emARU1-~O6fV3OnnO2LS zWKQ|64OpP(AnHc7;tRVO!E|rm+Wy3Q%-q4V=ZzBF0kR)JV+|L`K*|0w%DF1RD}Icd zTaf7)#hXOi4xn5r{XWHx*#|o&G=PrnIC6`#|81HhG0HP@-njK`BEvCYw}YRPNo#Hk{|r+a5_4pMqacM0T)^gxLLiO&(aK zccs13ZugNy^B$ahJle?ZY2VI~4#(RYt^dd{FxQ0Tz}j{^SyBy}W|PQEAGwd%+-EFe zX0S8?O{zhE@*hd|)*B7#BPgxzqbC2+l(e1FNB*G6eIzMucbebkMO$x<)u@fPpON-o zQVX9yAP#+fG<!5bUu%*Cp$P3&-x^rF-29q2`j&!{_jw4Qk)XbzNNWYHmTgG zcBGaAk&XnofMpz(3M(YleS`yLN9?6#9J)ev7adkUx8fMwhv_ycL@8j}ErSzWM&{37 zSZX(aj-4w|j7GBIKyd*LhJoT?jHqI@(8X%uimA|nV&Z|?p9iw8`8XYr%-}(7P_A|2 ztoLxYaV2K{0^Jn`ckh7P$Pszdyx{VgdV5&qVb(X&l$-v5{bJsBoEqrxPrsnzC&b0e z#YQ%p(AiUwy~{3XFG;g7RkI+yr@U$TH)*k`s0J*I)7YI1#CuTiFcpNYa(a#=7^RQu zT4>HAWcfp>?$0=BZoX@xTFo8G_tSJL>@*=z!m7Chs<~fOd+IQgfJ#6~a%<1}Ft@mC zhH(w6`f?yK{1>u!0sW8EKb3d;l)7pETX_eOSs$ruNw_fY{8RrK-o25p@$Luh>*Da| z_No7i`NRDO`;b+)bv}9GSBtHF%CjGCXpZm|!|NJ%otLWE0<1R?QT#`jg z5--xIe3$tks7|0C-HH|vHIcQ6r=MXe(p2=4+GUkj8O;Zy8h4F~&9>Ig9LhOpV1M&8 z4eaeDv|*w=RE(2pW!uxZ5F+jGVc-(6hl@{-s~bHTI(k~T?(}^7bcCKoL%SF~S1@{> zKh0c1cG-!ZD}R_Cfh~yEYW+>Q{~0t)G5gRcCSl%o<=dy28llY>Zo0v`6{&aEWOKZS zHU)6Qaz!p~$e}%Kd*u{&C+Dh$nFBm;MPm)~9w^Y{r^y3tOA;WqCTRs3trny>P&5up z9nj=eVhg#BWqyRV|fbo)_j|4o!4VMr$M7#gxN|-t{|nv z>POoAPI{YcSYs5TmnZ&Pgfg}KR3?p?&rD(159&pBnDbv2&4=i9$P<}5ntGC2i3M7R z@`e>Xa|LKw)6ngM3+O}Ic(sqLOwUCIto0NA=XoPYiD&^RC;oH1=_4^?ws>W|;P@j* zyXOTH4Ke2rqVs-LjhE}Y=0kSaPf+g zg7ilT&Smp9trOtEDd+{mxo^>R!IW!oCd*c5ULlQ_mQ&b^XQooakTX7cNIQ?bU(AVF zOHuYbfx;2uydI6?`dHcfMECY$#bv@%WRJE}d6f)LnqZceZquE0vAWN5u5Tbr;_Go! z1)xOHDab?JRUocD5CTQSAa~U`&J{p87?F-go`@$}-+qEg$GbGq`b#aXJ7UthF($3Y z;-*!jrZq}U%OPKcDe?8_O&~Y{-5NJNZZHHdN9Z4q>mt}O!T|DzbOC-iCBO39^Z(5( z+4Xq5Vl+Jdudw97xM|(<_>Z#WZ;xp#dGWEhEa|ylNNaJRKxcoBR}i$J$Qb@abVN>7 zEo_vzk5S8GPx02@X-`7#hYQk^$=vfit?*roMj5n-maN>9L)%sAk*Tp9+vZZyCZ!tC zp)^Mzg+%1R9(yHZQ-d5BZC#Vz-c>*H=?289=`q@30JXIrOuUO zp;jtD-GbWWv+&Ob1m4lcJ}CTQ=uYylNV{WrJVLqoS5y?4(nC_R?;0OxulnwG$QD&!qdMkAeg zJ3)FDktux&>7AhT8~_*MeVU1AenbP_cUbS7E2kWd3YpNC#n#9J@|cDQd_!Y|){>ZM z=9(12JD+oe--|RQJZ9$6Oi+M>NSd%{eJT9RMHk*vq68yNWy-oo)v>qMz)V5wu&pd? zE0y4j(iqar?T)^E$+}W?#q#vP-4O zgHAwjm?&)yjaM2ULG3fw?)e%^ERGGLHHq@_BiL6%eIkvEmXp{PhS)0Pof4$MHr}Sh zF>Q!@epBn;`$+fr9{V-3ShdU39CEeVyC2F%d0D*xa8vQLHeOyJl8Y9*pq&>hEfWfx zdCzCahZfrSPUP5E(0V%!Y_HcLixzaRIr33ztXfl*n%IsPFF=!>&06fUR6)&l^XH-K z7p-UQ6)R17alP7JpCm~+ThJ1Aa>tx`N0qmJMU6F^8tXCESSihl>#?o=J(N!SKdgPe z5TS~-&NaV^-#TYQas@|HwA9LL$V9jyhE$f9o(m-p3nLE4Ja*G zUGSb>=vAj57wh1~qB3_7PS)P~r9ti9Um&|4c$i75Pn4w(Ya(zFgT#dncQr`BTg~lg z|M#tuB9izUfHv)cR7^Yg0%c%u?7mW$#T}6>9>gpLMY70HvuITkRj5f#NICfs?E-{u zhGLdOjXTDOa~+8yR7{hrH>PdgqujV2d$N5l>(O?#i`y4qL&rw=NWTOr(B5O>`9$ak)jQdjuO_fP^Xkte?r! zn^?nKM?;t(bvmR&f_7dKw?c|&Y`9-BK1A#gYw!~-NhjCk%sZ>(K0@sa+GhczizlwB zeNxiPwuB~V(beX7X1&@RCuboU)EYZ9N*=62&Ri}mNo$24%g{;h({H|ue%kZA>N&;q5DI;E*c-$< zZK=NWsa`3K_CMCXI;r=C1-V9L#h4+apjcVwY$wsK_RJvl%o^wyD=f;F-gs)x9M-~K zh-x|WW91`jZ)kF)EMBX&w?C`?j@GvkgQ>|?dV3f}nR#~e+$X$s+L zRA{u;>c%*@70_!-sAT}g5#fkqFmCYax4M--OSB@T#t}ab9g4S}qT}iZS#)6oay*qr zW8&_4J$Jv=b5~mT1KfQy%3Zbp%E-^bQp|YVgP-HqU5KBKALnPM{G@%jDM+W!K-*R1{L`Q1m)?@)$fS{-5WMVKtEx?W9V zks_}qW*e2`zv#y7g#)`YJHGt|jPLHQ1b!WP21L88bHc(Vr}v)Ju=|0016o_~^{0s; zTFM^{h*W@w(^bF%PFDd>Nf<1_|6i!}ap(f=U%EhjsFxkk{3%VV zkmtKqNXMEVt&r=T-n*K+(T5gmg(2P@rg6vK%qF;+4~-FRc#2%?Q#+}b;Jt+aPe8E0 z6L$yq8OTs4_#V@`HMKTczk)okHI%Dd?jbGC?)e;FxCW{0=Ieittav;VG{#?((pI8Q z%h=cOOG6-1s-v%BwnHuv9Ft1sR-{5z6~O$zC$UEuv6u3j(MRv(UgGx92s$(BE#n;0 z9D(1Q#H~?L8$*duif9{1&O}#2sI3c==C#`S;$!hlCUo{gb;sLiWWr&s-sifK&+{-p zEneoQds(-(0rE_bzIZAHtr1unQxPi`FMi;Ry>hA@-07MqOwMB4R6n9FwkOEtJ zrvw^S_vLW)GkJwW@MaV}l!AwS1@CwVezY+2ak|MIFooU}L&yX`D*|ZRy8-|~nqg$_ zSHjmBp&cbVq!B$%6fau;5#-kb|33Kt7XJ6BaO{3a8=mEt9l#YwKH1$j&I=#uUf-tZ z{vthRb$ia%Kg$+NEWf=OeUa+IW)~J^whAYq(ksvbh}iZ8^;0k}e0Y7l{Y*hF&L=kr za&fjG&&(C1S-^@^Toz4(cbK=7ptFa%K>Ucl9j$*lVE2vzE$`m4O7E%WQ(ed%Pm6dH zH3^)bFCY5fwCRzHHfsSZeHgH2vyBaF0eg(L68E`eeznhl$6uj=G{-iO&7te&z;6V~i$sb4KitmT=v1{nwG=@fWORo3RvjkQx7PVSfdRp?{jNeMgCc%hD zoO)Yj9@p~oBq2M7N9fNc>KkeP{&1M8BZeL7=oe|<%zoA!CbNMpwIB(V-8E2|McK@F zAb@umw9}=Sud9{?f2<+XQEQ-x6^1PQ%g$0opdB@MejuBf^R{*O2L$vwz2m=pHKn`1V@O9`)Nfb%5s~)~ zRDQmaty3LU?phf!1P0PsVdD8?V(EK-IQ_q_qV#hWK9YX&N>VEuB&4}z|-N!>Ytq@JrLno^bCRm?^NnwW4)oc(@w z)&crZ^6vXIW#}-TGFeOx1Ur-+)lBz6Lljy~P!7LBzVboAX~2t`AK8Km#oEMrywVgK zU3mMJmQGYDb@xx$_*WYFtv)N~TfFDBrep(C9?^eFXqNKxWo%@9{ouIG`yTvsXpztn z#Or`HZ&hdd37lV8?6*wk4d}GqJ7Cvx4!yDZ$gd6BPVZN?Yz&7(Lm*Q?|2`J2JhKtE zgsXM*Y&GhGAeAqQBG?%W%sHX3NRVZ)fgH)Z0+gh!E!t+JNSRFW zw$Cn7x2`XaCMWi0hZ@+Tg!z8;JRIqNsyq?os(fX26*Y=1rM;3-7gUZ{>h>BV9YKBW z=FgxEN6Q~eR!{&XpviN0e#-R3ks|uWQ;+A5RqXxS7vq@R^p0;T{9!HcV zKy~O=7<0>AoI$x#_oX2BrZ4fM*WOh>65mG2D;E{W79)u3K#>JYtqEoCfUV4&>xq#} zBJ(3?&aDc(i2T>{8@f1Ih$-K$Q0wrj6^!R!D7hMLXCI`Rh|$^ zhrlT>%0fk|YztE!!;0f2c0Z0(pdNVG|6*)|B#XcnY*=0I&>V4Kk0wRRejJC5YJI?{ z;I7FsxN_=_;VE5r7dP0K)(m5h^dZRc2Sjy@_|g>_BUqEM^z1J`Qwm><(_d_Af4TJ? zt?6GwjJiPES6*JBX(d}uP#^f!2-XLB4~;Sv`kCQA=x2eA@%!1Vomyt^h92oYUc*3L z^Kv}>U~8Hxs81^ET2AYbD372zb%V&GOO0fu2crR8|`G5sh#X%<;XJD$+EDM^;as{r=c{hZ++O)r1PI? zzgw26?f24UG3~eKdA@9iw9l$e;gW;p5o11e#;{NMQ-(ITw~&bb7bgwSm#ytPIB^EF zSfjaJd7uJU@d9-H3ON;_mhYoMAmcxVvXBJ;khUR%Wq; zpvB*a&ul2LP_4K|0D>arRM4VPnl6b6Asdo{%X}zdzFJ1fpk+&1i&zC@4WUDN z%KCvg(I0KoIndS7ckI5{@*RzNRzA8k3Jr+J7lzMQ?q>M>`1k_lT81x(kGCk7F}x){UQl{5ybvE> zrhIiT;mhLV=PElHer|mH0_7!!Ul1Q(uB>JF^7#0r%H8)y_Lt+}E0t>*zLMdyV)1TN z)s#yq${Pq{tS>_S==?U~*C0NX;n&2$)Bft0O9;O{4jyHa5@7fzXcVx;rV(ao_M+lSF-B1?uelcGdP3W3|nZgfiKyOw^p7I_@c(h`sKVJ zYV2{&LzfBOUr|e^1p1X7>|CH@*UolIb1h?1UDN6x8KMj3xypR*(ihq z=ymC6y-&Oz&jLssZp=L9Gd7C35W7?7Pu*J#OfEEt8TeiRO#uX^7<5P}a*-)C15+^p z+%n#PL8;4R7}^|hxflJys?&&d2iu2G5mZD%-KiWi>E zG&$U>j0V0Q4RgBr=Y1+XK#-<7Bo`nGcdr;>Zo#wyq080$R^<|UIbxTa zgpNG|hhC(_KZJTgd0FiLD3wwi67VHblfUl(X2CRTR1f-x<)GSgpd8EkF}A|vxU^~{NC7$%KDO3;ci-zisS;VQh5`to-uhwMg&`z^?{i1sdwK!^!*cS!T#%b!u~ zu^9Uvh#krye&f4~wjpES-{qjxS}Jd|#V<*)u5yXg$5Un%5+c z0NBN`Nb;0ycv6N^lF6oLBKw%%KONn)jG2FzEJfOLy2(rAOYqK572Kb73WcXC27#m^kI<=9h2d7SFL(fJRbDk0 z25#nfF3&F%zPDOWoRK}-sc*lMKQvO{w!`S2MLx2Sb@Fv| z_qzs_83Wc##&<}H+b&2qXW_M;f`r&!PN~o#%>rd%CXfIGuE=(bS&=39yV9-FicI-| zOrfx=q5$Thy73Hz=WX!*(`DZYQePl6lAMeQ%`#G&)0|R`QKwtL7-1n-!f2b$%D+2^ zKD?sygLcKw#x{0?Df5+j_zo=R{+{a{G!R{&{XucG_6Iy??GGwZJ&m=B+TXxP`x}Ed zMzXi!Nr!iSHqeS~&NYPD?_hgOj5NpI*cwAonUJU^TARTXDJ9@cB1;Dy>=xZlr0D6xzP z)!v(;RFi$4)x9#uXmIg3184hPO7I5km3QWtZ~!?JN1kf)do}>oPTBV)2uQwuw?Wab*0hb5~zN?6quA*OMIg)Ol6jX@-AVIrYTR=lI$? z1y7~o=BHCS>y%C?w}bjc=(mVn_6ET#aF}uvJvQU;`8rmSGw+0u2pPJMbUL{cObruh zuQiRf(>j6gwblEf3TZAa7tO_UM3IrDOdC#c*p=|AtQimigym8g!Isi+A}!TMlf^Nk z3Hoo>k0+=-Xz9sl1cRwTqA&)P7(-V;)-gjU66MMylQr3X7;o-Cavi7e?1y?QUBNB! z&y&-92K0c5PWh*%(lM@_tQp><34*&boAzd5{?vxk^xB$igV-09bW%mefG$zEYxx~8 zyyPo=Pf&Ao732Nv7}13@aCJ8F(_Cv)Nj8Uj4tSy~3s9gwkXS z;bxes8_E6zO&21K1&12>2I8s)U1ua`-^{RTuu~dEKI8*MtQw@bbMd4#o<)OE{Kg#M z`&k&qZ_LS*@5{u?G~G*SwQg7U^|A@_D!`vb{`b~WP-|c&-BJKU`bdW~4yGxK;N9H0 z!ArZ_XVo3YMNYh$gwOLZi$^cBc${6!=gnKs=iRk|&s(yI&l7!o9y&zy&Sln*Wz70f zVAhWoHX%5Cowg9(DR|Y3W9@CKQKJ%L)Oc9%&OWTXK)&QNHeRRgJa>v|LfsA|YTkC` z^`GeW=a@dc%F=DUrQ36NdoFaCue%?XPCdXVlN^GPWde4TRA}4cDb?{+y!!*0e+P%= z-vJZ~S{mf(eE#ED!T_!t3W3Hjapsuv+;3R(x`LJwBk%E;_@Jk8GTH8$4^0TC&n`1l zyK_V1NpU)+T=EcU)!KIgdXzq+9D0hr6PrUhtd&FNGrq%QE1{CnZ8UUOZ08crOtSSm zcoF>%l&9G-ohW)Dh@4xXA25xYj7TnDkb{r~kV|FvTrNC*T!>!8&RQBHf1~m9 z@Svp{9|aI}ew4zwL=RBUk`Uhv9YyJm!a9QMX3vW55>@)dfwAlNECbh9lX;Wk+7nN&)v5A#52YC91yN1D(0Td@{oUm) zR)02|pJdSx!05jPml$1ru|YJD7-UN7w~@W?D1W=#Uv<0xa4RY_O2r9mRRuO_DHLKx z0bTea6&1ugKQS*@43WFdui6#!i5~N@vf1(GWAlQGPU^p(7yR(s*yjn~#H8O%KYYUb zWBTFUH(fwK+;*c*KdcS@|Dzx3`Lkl(kMZZ)*%#o?vp4AYvq1TOnm@7oiRk>gE_@Mu zUDH>+LHOm_wC_GEG^sAU?S0NrGse66&|gpq9=c+l;2r48P%{KxSTP!36K(8%HT-|-^t*a2+w8kQiPohei&g3ga3x`I0o-QIG4fa5FWkvN7;6EVzK7;Q>xS7GW8v*_YgKt216NB$S z`1cI1K=?@pKZNi)2Ky1NW$-z)^@R0?^@#O~^^En7?O-*R1&fNm=)!@A4I)&7j!WRC~fUR&p z|HO8S@tlpf@3-^zouNTtA4Z|`3cRg}tdy+0{Unl|;y+3Wrl#$as@J(LA*$S6VOPFB zy=S%b;D1+WK-dQ{wy@+Hw~(tg`-e!LQ6(=8$y@KbgvdkcE|}U*N=-*x5~W78^-^i; zjkG=bFVU79r7c~hEdyzL@?WAYB}!W#l{Ow}dkW_8I7+!jAax~}9{*?~OU}4ALfcp! zZT41LdxF<$?oGDB1C;wIhQvU}Q{lBb;7&(kDZFPt)~6kp`hYmr^KJ`8}EveU$ zRiMyM!zlH69CoAbiGMm1690{sczC_`uo%$QMU0ApsGkY zP&6^EuKH0J2@Ebr&+>JZLiyfUq0b?d*-id6U<2j$OO)+%K|H77a0k*$^S)rl$9Sni zr$Nv+0zayG$!%J&`*IWzba)+S+AonCr){}KCrEAE*AA*Z zyKwo{{N_e)daXC9*{`Ha{@RxOHGTeaXZw-dMt<|2#f^BmuwUs-2&cC=Va-e~G_3Y) zuVoj(cZ+>dv>(wNeA^4Z4JA6Hw%W#g>l%}_$-<3YJBP!}CiKY6x<*L2Kz4vibhv#< zp%Iwh$R7FOr+`mXb4&s_AtuX&|M@Hv|A(_QU`77GeKvC)(s$4@-kq!;toi!Oap^yt zzcA$P7P|tg2k|MxTT(=3nmZ6Yx=^BY+dxgNM-%X0DFZI8LD8Y>=OXv9b zJiE`n_AaEzUjI2?f1k=IYwh`K#6H&AYVI1|b1Qm{2)_l)LuwjIy&{JzeD+PC^WpE4 zPHD+TtMnbP?c*(d4J43X%uGfdN%~Z2u@nEqp&IJX5!9a<%2kX9?fplSYvlX-;KQwb!fS5r1%raEPZ%ih zcj5Nj0ldwX`PzeyC|!#-8IcB>pevUrom804ZWkls?WBTqR_NH{v zruq4}LG&<~xHuc)?#4YluQQ|7S$Lu%gSCO3{DY^+q{1(_j~o`f_Z`No;Qe1E3*kn* zG0%ON!%ODt-$8o<@A5WAVBS85_r`*-IgDM#aoM&t%Wu1CaS3|a+u0OmW53F1GtuvE{MqJbIuzRLTM74IBxvQ0b!$xJ-K zs$(DDcNWL@EjfWX_qAPp0)r#img{jwD!5y?;H3aS!v`tyS9}G{6Ix`F*4S1*4KGm| zLd9>itsO=ET%uV$3B&b5c)Pj9>Am8Z31#6%CJQ&<5A+P^)2B%k;;Ha1E9J{Ky67$5|{ z|6U{i6hQbbUmyIQr4&-1HanUq*&reWW$9x`o4ZdK4uR+o}9q$R94~K&fLO>iR zPXRu>QFcw2e#WS`CsvmdSSZyCp&u_${j218MkZDN=mtRF_5GtO>?ZhuU*Z|*{0 z$P1U}DzD-tJ5m6h5G<_7m8*cnihMz8L=tmV5}iciWR1kDiNq^VX_AWVp>)+f zU3ZTHigAj1nb%jL^~$-)>cc0Y9Z+Fe4(U)3l=XQ2iuL+Yt`j}m;7%dliPx+dh2pJk zEZ0Q=8t30r9-W2^xhFA<_sflPdMD#uW9y6f&CZEo?{%&2vuP_VGrZ1;U5@ZBq41*$ zwC7j;1XOv2u=P!(z3xmGbpI8`MOG;(Cvq)pu1;{%=40)dYeX}X*m-{TB}ir0?it?6 z6O~(8f@)H4Qc|1%R!W zL}Qe(7_K=EZ)un$Ciz%{>2Ak92Nw{$OF5yV)yXNKRbhyuQ;#SHD9h5+NV)a&^fmwr^KE$H$z~fw7>BNZn8a%;pX{j_W zhuxMreGBz<5H>enh1+rH_fV1U)o#c%vWsu9sWe%v96d21VNr#I0762*`*CC;R}ykU zg6buej*y2YQ!x@^iZLS%iZOzU;TnSY2uj;X3FfB7T(uD2UV+&gMr^G4`q5-|_&ysm z%m-m!g7b$hj2SSw{zfs~i*os(;GI$6;BD{W-k}7A5{+mfkzHE&oh!vLX}QtWi|(sk z0(xvJUNS)!To=&=vM~`&EaFde5pfICVuo!H$A!0<7B~*d0W1VcB zj(o>QX-2kmLoRwwL2IcQ`BG6%sl241)blm(!S(c>ZjO4|bD{^af+TDkY}88*jg5G@ zaLQA@hbJpzzFa(v`V!tSfn~YHT_qUA+3qT*L7Y`rRYt>Hag)b?E*wr+6!0{j!&wCm zin1kL^6Y@eCXWw7lV?u@TU>0!%Ne|H`p6N^W3BMB)kMRP;H|QNwC@&?V!X~Jk?Rb* zxkag?3l?4ZYS(sm_7;C)l~du>lH}U+z0j$|u(46668WL)(C9VPpN^US9aB^4e?~{B zjWk3+{NqUJP+)Nvv?zmSq_iAHgKqI@<1I$RUp03!JtFw;a(g);u2i}QNu zK#c!?Y+N=bPRKzG%J&Z@GQl|x`Z52S#YBU7hRC zGMZ3|l@^lqcL&7e$DG4C7ydNIxo}mEz3{x4iS}s@d3ip%t9x!DF>%$}8CL7sVk=Je zu9`eTT!?u0RGeC`0(Iubx>Zb(nL(!&-A(EE?IQYCoEWmyo*5zLPfidoNBsyt8%3WD z*FMW68)N$Hta2sGvbV1Dvvshx?&q(a@``*M%w~}P#5p;*47Fma;GJ?lIhovjd_Do) zThQ0g`9_l_Ho}SoWLb%VzE+KdRCtkYa3#cXIMeg~_gu998WUOU;5U!3{sXy<68P7N zD1qK(5(uX54O_2KCGoE(sw3oYu%qS}m=3?R6@8gylLWhOPmouYMcX8ob4^=gs)dMo>mz9^gJR)^*Z-w%4++>sxrw zv$&8k3XSIzqV%+W&4yVI?)GDz2{C!{^>c8gW!i`aRNdY{KEOPGKg=h6TY4FoQuMnC zXoWrqrfQi&;eP(%pOBDmqB-?Uj*Z%D4fQZ0qPGMgFkR(Z0&z{A2J?Yz{-Nt>208x_ zu5<;aXsPT78QvJ$MpcBXrKr*&-~AiusvtP7%(`)#a( z#0F)+ahM4H!#op(Csd%1ann65fh}?eU%}FOOETqInb1VScIku_6dzzgVnzB^R76Ul zO_hc2E0uPZ=AFcQbZVtfu{>!XN&EfEMcn?~ooUVfu8X9T{;x)G2mEIf(i)``{@`%# zBmbGiwB7!1hH;y4;T9>_Nh zbDvER2LU_82Y>^kX4**5Oq!J>-i@mx#rd)k%0HSWeA(JmkK$j8@r86PzN&6d zFPS`6x3Y!BTKL}p|LdjM^#S`Dx^IC*;~rNpJoPsaT{({m*lP_*h)Kfv`RC$SlD!rN zmhb3(+r>E0G=UmjD|kzrNuz34PUB^l2gQ+i6HEfAx8k7MyXww$i}YST4Z(apwrf;g z(Qz&6>ZPc*&7&@KSHmJZs3-Nfhh&qM?66D67g_x~j4+Goem_8Q5@;E&G|-)R&o|g} znT8iuM!v1O*V5G8=~p6R5-JTaXk}aFn@q}{d;_JSDrPom;dHrlho+H@#-yj}fhgxtm2HC5&duL*V5w_Hu0GJK=SfMp3>rBL`324aNg@ zq+Yn+DhTOL2(8E7hl)TtmMDA&Q$lc3gD3AC(kG?z@?7Mb2ba!*ld#L*J{En87+fQi zIb(5-8k`DmNht~9B|N~wV1M{+7>jNUFJ^Bd<7q5EKh^VdcS|>Zp1h(bKOekO&rjrG zHfptI)C6yctJPi>Ph|~d##40TDYC%5p)uSW-R*ppj(^*3zcBw2X>GE)SxFg-t^MoQ zkv(kN!RGI_-Wm6PaHc*9(nLYR6K1mtoKlG;P-4PyY%30RI52pdnI|5(g60nCg8x_& z+P%GSxgmNo6AxUW38cIg*k@%dCzi|h0F`7eP_clh zZa`XS)Rl_L)#H=33tf2a+v+>dyH)_K@+V&OX!^WD#H$>VaY z{N@r@PNKPGGRHsacgVNCC)gzMeTQ6A5JpGP1rE6c$~_18QI-X?hUXaZ?Up1eg!sPt zOc+zC{Byxs7;;@>qf2<=B>{8KZb`JN&xA2q!?SW6s-x*rxu%lt(77S3c|XuOc(%ma zpP$=`nv;^-#AGx2TjR0N0R->hRevsrlsP=LMz$j2l#VzgpK|s0SQTwL>&{w@|1ti` z1xakC%lt~R@T3oU5lv@8gvsL@bF#hHa)SS8lG8iRWS!*9F>PhtXqOcYDP^b0==82M z8NK&$g)UQi#bm*2Go_>UjECB|cBT9+5;P8Zf)!Lf=p9##q2t0LR(Hgzq%BothMDtn^&pqdWXm4_^J-fX ztZM*;NeP%#S)8Os=h1p>R*YT3lo^+!UF!H-=qFFTQHs$CxUlM#~jkf6vIw3)**=t1gXslGxEGsLg99SFWNuD zJ9(TVQJgE}HQ{_}U@qOQv_QziyOkE;wMvCu{DU|5loZhhr167O`Ej1se`5Jj`%~BV za!HO&BM+n<;2OD|f;ah*?~)AG#7xd6 zb^3iHrb|vf*Cw6u`}>>y^s1M+Svp~ZhmmKbQ?`-Y&2MwtZIaI>o#1wJdmV0nl3i-D zN^LgjtiyfA!qZ7M(D55=(!6Ht8#ti{)%pEceOa>auzHJ5xexPJ=`<+a4bV4#kt;R& z8wX1Pf1rYCd7(g3|2wZ6o&pqT@rGWN)^OY)bpkD($yR-*da5t3Gl% z$uK?9^&ibv?~HTqBfK@y_E0XsCpFl{!Q=~mrH}mn5nP*ez~9*4yc3^#nRi2}ZRVpR z57?x0wvnHi_i}H8-p={C?T#>T33%RRmjVv=xtY8NZzx>E-@6y<@#0ph)r&u%TD|yv zsP3-axG#H(uYo3UZwtMKb^&4O`w8^ZjFF(>)CeYhpNfB<2Z@kR4a@-$>K#W2?w~d4 z4+#DM6y$7RwEOTh*L3SYuq|8jc1S*lcV$X$Y2k5xwM;5^7cQNda}0**-lQ)`c5}tq zrFmbJa$h>U%X3i+8;;%p+}~T4-B9Yi%LntOW{3N8NC<}%7*Iiabkld#HT^Hn!*_MX z*{GL-*6f4Pzyj$^TX6_<<7TvGORR21-FlG2yDFDR`AT_y6h!uakyy&zo840mXIx5h zcrg8BTmnB^>Tx>KG_w1AS@S;*40)T2EGZ+euI1RqoYdt1CQ;y<-N&ZkK{uzjDi^fn zTyHVgWV<@I*4!K{*#HSw9v^ZJpW0qS}3*KAvHOX zZs!s!5~%S9%HO#qwZqD+J{0nWs&P=K=l&XS9U zAv_G=VG-3ldQVr3%NO`nxzn!#0ALR4F#|Qs zAi#XA$7E@k!GQThj~Sw2h63hOJ!Y7O84j33ddvt7a}i(;>oM6HW+Y&a=rI>-m{EW^ zs>kGLm`ecjnI1D*!(0lO&-IvG4U-3$FZ7tpG|U*l9MfYi*DzNA=1V`)kXeZMFCoEO zk}G)U7zN(GAO24Y4exc_@Yx@Jvm;yR7$Zn~*&PC@f?(Q^-w?e(xrqdD@dh~TS8{w_%y3i7ILX__AOcc|Z z%~Wy&-LmAZGC92qVM5azW_8envg+_G7Vp7Z9C3+58C1nyJZ?Z0T}=?TBf~~<8f3Q5 zN||Nbq&79PaUnY-nI(8DvMJdw@m@_|EuHbYfzsUm06i2(z7hM=ae%8Kz11we{LpAf zARirccyGytMs1T$Dpf3zK7u?+sbOg(==NJtBJl6+?S7j+Hybcf@&AF^v4Cu(Lbe#BOLNHWyt-E7*7RSBKJ1?=wVT0lFSqr z$b|KBUBZVUQk?Q^N5d!h3Gv|IUeb>+|1%3l??{41#Z`TFwTr7Qo6f3Ezw zJ_Y8d_mA|>rm$lstJ>F z%K$Q0G`WUJi4Du}E_9ZU@?fU!)(7RU5Dmr<>5!5T42K);i$;v`*IUjAu&DGbjiIN< zdKeoYXfTBeU*W?0y1e3XnmaZsulFOXj5KQ?1?6fe-Cb1~7E|QX91>U$2O{h2Uuc@g{tcgon`mXo~D0 z>#PzZ{N@T|7h}o7g(j=Ja;d>JofbQ>wf3ihN?Flgbx9=|*fOOh+>8sxsn*(R!Gh`8pRC-v1dZiZ=49(%mZ&fDLEnMbm(uHz zO8ye&Pc$uhiYhz!SG=C;nSSVqAJ@s!bjJ5jB%*%-x`0SG^M(WmE3-w+0iD)*2RIg! zyBVc!i;k4R%2jFUqvW$Fyc#7Rq9m$OCP5Ts^BagzFD}cMD~%96&mD%b z-5geC^uuAzmP4LFhKH^a z4$WZolBOn*t@KtC$S2=Ilt6;(5=vW^W?v$FZ(QCdH-kO@HRBA0BXup z)pc?5hod~!SFQ7d@QqSRty#2}%CF{N3q&E*FEl~x52f-1v@1fFmRVMLxe1r8T~^td zQ)vy~WCW5vor0!~u6;o~ih~RHm0{OVrGuKd*v|^tRw_U5&&1$qHHr(PyswMmQs%2s z&eKNWk`1(+j$4Gify3hoDbm9DTzuJQ8wv?!ID4Ou?-#K5beBsqdrzLcaGk|9Y#Sb1 zRK8|Cw~sQ_L|eqX=P$I|UcHf;K6YwQX?V}o%qPUWft+EHz~Acw*rCGBi}-uszov8( z^K8o3Z#K~8j|2DD45V;L_9q3pR1Ne&$g9ig#_gLNEbFMoCxq2gN-1GlPdy-Ch3?_=}QBz4BkgK0mKWvsKn_Ldoz(t8#b_I zx6oE>IX2o__ZY|$O^x?hOgzODlLw1TXONUFeXuRxnp2L7?r}wgM;W{8>P!wd>BDiXEIf6KUleSH>;5rI@1S3K=b z?_KXWdHXTJv&ZRO(hA^L?lasQsHD3O($+1d26L5#CU$$1(_0<0U~W+QXVBb5oMNR* z#USoPid*>wbeEC1a%X>E^4A`zsXbHU8d={W=AdO89U0&JPH1cc`u_~&EAOP!#OG47 zx(H>XU3iUoPrZ+SveBFL7#;AJ?C_tS(;%`jJS*=0O}z5{Xi2eD+IcXo6!-B|;{M82 z>5==d7>(EUC{`8}YpA|hm#|{}mrKLuU+%8N6=Oiz@Merg@ey%#;|>TR6NF`<#*wp_gk|09NB*}cV^Sbf0>g00y%cevXMKrN|>a{wIs8Q z>NOF%xsNvgyrB#d@J?|Khwtb1RtJx#4Rl`qV&$*BNpe4ox-MbpGKe)9Z1S5LTA{4O zOL64n77oX97@?<3?n?_Bw?7CGpYCl?U7yRzuS8-DVKJs)j4Cz8Scp*_iP34I7(+3} z&ArL27<#4dCSv90Ui$07!pgh7XjSDI_)tksCz8feaz=RTg|}PLmo}ZoQZ9lNnNWD5 z7cOhW%*R0!zOOgCEe`MK#Ova|PGi@_Y5q~N1xaiJ`HDy34eoHJ*cY!75Mkn(d>I=e zpt>OjdP7{Cj%f)2yhbGn(4FDVc8iy5K%9uBkyazis+Ec ze^XGm^yj)T|y$yS%d5^M!8jdSf*<_5lcB#kt zh8xPfv%+MbxA(C5pgWw+&)P28Pg7bPOKXgl)}66wt^cv_6WI5mUh4NbalRK8H_?re z?26x><)g_0`pa)uN}wGGi%Z)m{z7{vbY$j6rsw>^f)`Vs6ShR|ssstp@m1um%Cr3h z@8ksM!V^voI&ihZH7;(%RP&mot06N5M`Y!5pq9_IZ{qtFCBd%`elulzt5lR@lPYql zS}dCLH=Tb(_5M%YJ(-wyFedNx1l*U<J(eqJ{~Y+6!Ra+7FsHB68RgCq0h z0;#>qZ4>EiAsrBU`%F?~mtdq&w*#|d_k=N{`iz&N5nP9q9SOSe0?%3zb)dH-KX^tX z4WxmmnUU7UNK-V@*mA$}%S58BKhUPAw2hQ4Ci3_yXB0$oY!ffSGfPNPl%T;;wCjj+ zbpnn1QGP8j>iG3_r=DNyI~cz@7{5A@C}V`E0gR{-g8NNG8C>~-{C8gm^CWZmgec3N z?2Kg@<`~1W6`gvPp~DE>eGI@YWmf{OhAA}(v_uo<0FNQIJgUtu)W5zwe`n?K^eoRl zoL$-zs64|X<+)P!Yydz?ls$NDj#VvAOUI8CXJd!HIE@i`7Kh)}9sdNs8M|NRqV2OI zwta3J7iIRCAET~U40U)$DSXLTco)7TpTess97o{<3JWN_hQi4dPNmR7;dBaZ6c$q` zP(9nrt0)%!N2nw?) zyqLlq3P)3zOW|b{UQXdy3iBzvio$UePN1-W!fPm;OyN`tEfh|t&_-b~g#v_5@9_Ih zB^j(YgfF=re%IqZu_1U2RDyC`y$pVg&?toVAapfC%?RNgzCm<0SAGjZuOfst_raGC z!sYDX287TZTJUj%{)W&xgz$oe;DZQ#f)M(+EDs@cFG6P!LLZ#vMm*$(emu$tAcW5E z%kjK=u$X8;2z_RjTM@bnp)!QfZ)^Fj2#r8!F+zAov3vzWy%72ZLgu(G%!% z&?1C(BJ==4 z=(n|8LTEEWPa*U&La!k7971m)^dv$bA@mSJ#}Pumt>t*IEBF9HCWMwEG!mgj2#rPP zR)i)YbUi||5W-_{jYFskq010ji%>Q~|AEjzgkD4_9U@0UyN6wVVb`L0a_FoAN$kz+q6gDG80-K9Aole10KJ%*?(6Qs0((al ztE;MKYNxxZx2k$DbHOkblxa&egOpz0)3vm53ToGJ92G)(Luufz2- zaN*wk@h`&lb8!6%Tz?j>zX#V(!1WK{`cb%Er`i42{|H>a3$A|&uJ3^B`{1g>6~l$o zIlq1kS0Aq550?Yi5Uv?qAHwwmaN#`6@gIll4qQJ8*G;&77OroH>o3FgS-AcVT;B}W z{{q()xc(7bzx7+>dgD!U{a8(hBz*L!gNUAXST^*7<_!1Y()dJfkw!4<&u6}Zmf z`uE}bLvZ~mxc+sx{y1De1=kP5^%b~&0It6T*BM-Y3$75ZzXzB1Td+zXL=fEy{WLwl z6@;lDhtA|y>V{4locgy0gFP_{qCG!6N{_eCQ-6Ct9{aIa`Ea$9BMzdlIh#!amjXqh zIQ8QM-xZ=!?m-#{;SoXhK5 zKNLrPy5}TmJC38c;5nF2N0c<8Mygjn{8$o8!j%s@GQ}UoK}r#QsQm*!E#&LN&c~}y zfoM&9CoX20;k%Q0VirY`OnQFkA$u>tdl(`5*mu0$IGVyui5MKuQ!jcEio|duO;5!* z_I*{%2mXsfgdNHO(8zkCLMeu=V0P@syHPCjT2-z@^;FeFkaWTh6|?Z7$4=-a$Igil z{DFU#_Tng|n2#c=gw*!{Zcb+5z@PZ8^7ni{ZuxQAI(7mid-aQ7+&1CC7*+2xW;-i$ zRp;a#22)0Us6EjJMa#L?5qy%=PFeD9{eG&5E)W&BIitQMERi`r#yL zpQVqZBt3AZ{(2>2NvUa`Qa|uyFQP)Iq;}@%u^*=Ifv>@%XV#0aj0jT>vv@v12*?j ziiu{QNMjl(IA<~+2DCK^s8j}l$DLbJ_@OhJ_+2-daa*Eq-%n=LZb0+JZK60Tfycot zK|`WZL9BEn!5|D!Jt0*OdxN9qZ$#C z$SM3TbpEF2#nigQhlI9_VW34rN$OiP23TV!m{5;&(=*{ZVHl;BE0HwjSu_rnpvQ&w zY0VQqwy1GA;>Bn_*3peP7VQH@#P|~y#&nsM0gcht{Rh^%pw!riRHX=B%AyVK8n(+H zD~&8?GKtXCEIq$iGEyrVTM1a+@WgnLnkFHeGG(9gpkTXEI1Y~NR2*`dfmDL#v2V@% zcp9WMTv;?8TCVKx*2;}!)gY%fU=8;M^hM2}J}nym{181_(vBz2QDRN!NotLJ%H{zT zW-_16q8O~_3AJ5oJ3E<2=*N903j@%jla{ zGV@(>Q`}KZrFV+a!5T;L)Ja9B;FY|(z4ig6%${-~TCtzb*!}(+)fFj0rM19GQRc9h9)f8B|sA-|Xdf_D00)k1(vCsry z?RI(t9r;D#i0TrKYOQKW3MoY1VKBI$Rf*l^u(@~7Dnr7e&x_V17{yL}Zq1zZ_?~qmc}r%~*5L#Zh+7xsW>0dUR+5){v)8wj z%p|L&Ja&rvke42c6+M0-=TsD!dFlULn4;;1<}7Cx?>nDFv6ckL|Fj^I5#X_t438nO zBrg#KTM`0I#G2T`$6T#K{gOUfyq{p2LB$}O2sRGof(!D(I!UzUN2U@`-pmlRe- zz#ZR>&u4mCL-_unsf7zE$muG94DgEO+V?-5`!uV1W_2p23%Ifd8jr>_UDuzbMV}c`LMj424$>rJKQev$K|(I= zQ#s6Ok(XUepBaIB!ZI!CfeSDaQRh>?0vuKlL-sOCGJ_}U2DIbp@tlR>k6C4YD!d@c zq-GR+GHH>#Nx=>iADulhB)fdJf!t-chR@`rWqbbEp>fQiL#CaY!zJ;5+g4EI@g>D9BWI zZ0hvY%F4%8;bkxtVM1fC97Qn=S5SjM$K=U@Q;OUwFu1_UefXU^hDD*QAWMb|9$RV{4D-9+fOH>AdI=dG|C-Yg(o-i0nIf-IVfuxqA)?1+bDv5CnpdLxt!sC5k5((oj2*3Fys7qyR&|*Hi`m7za%jCf@}d1tbmY zxu5b9PXSKycl&HcL!D=`s-%u56~x>qI-!bU+b50C-wpi9GK%JyMv-cQc!K5B!x<^^ zOIT3CsG(I+bmwVzyzfslzz((0W3=gkKN+KBNOCi)v?+grP8WfP6p|bekVIo`^@!yZ;`k+k7rR*jPNi_<2hv2 zQf!~dMb)ebWcQO9BymB`LK@pV3A_RQDiJcwfET|O2&l~-CeD#hBd=<)OFEL|Vj$(* z>9uJ5!i>_^3nW#nWqV&9mLeU4B;Dk<_N9$+E~c}Q`inK_(fGEc^C*Dnt_A*(Ya+C}r& zHL{{tvz`XgM6L#B-SEtv%)KfMHKmj^lKmj0u|IDq`5K;7S>0kNBb_wOBx#J9WO!bk zlbm939;QYt1`RZ;5|#=FRhzGE%SG}+2aBSakrT|@s!H_2MJ2L1If|&Q9Z###tvN{( z)u^ofEMu`3RW%@qYwS$3QD6K=B7qGd`N`iANs~dt2CAXhAy%RSX96R+z5VEiUr2H)cH>J=AVB7T;c`|U| zyE9@L*Ndz(9C2ox5krRCY}6TNM1c2ZNJ(9K&sf33u*Mr#2JCzvPrG79z3v=Q)tjbYh8s5Y0|c* z)W~QMMc!)4 zArOQVnndv|7MbM4|5PXU*o)b(0v%6nkW4uNPJDv3UYoS$qX>(Yl4j1rhdoW_GACZ0 z3L%|a4oA;z`2l@#z`nY+pLrHwn){$mahbYl#^SZ>!!f{f2 zC>ypoic%}3nZdf@-LziNK!Y^^=@&*fnE8HeV|PJZm+dT5=_nryG+9%mnULg?AUUSy zq?R*~7i=%g(%48x?V(~aB{|)?{(x5=p0@j)-RJf}_n<9Wd!6>d(C$-CyZZtgJ}z&~ zSrkK@qMNcdWgvX?KHGQV$+>L6CXS(5%4KeBWB0@)#v;0X?9-&7OCy$RmJ39R57RC~ zUKELevngNcJTXaLF|n_nCq;7NpWFVKOHJhH;v~IFwrvj@wOK!;irdn?EHNg@vcxQ5 zqiSRsptp-M0rp=Z;)P}~ptUxY^U8OG{^$n78 z3C_cl5Nj6YzO9(>s1n)BTPeB{%>WpA)M&4$3(eJ-K2EtAlGPryhnDDd2lV$bNl4^ABxkF8 zKyx~WEy%E7B;_JIjr0(mr+X%kr5rh| zH0=GMtt5=c?Y+IOOq!CiW-}REp845Hss$_82_>BzTJy9*I8z*W)T0p?VvlNiR5BUm zi*P>0c-5iUX}pqgvr>>ntz$xFDkG9@a9KREf}IIuvk8s96P51`ySZoc`3T|piL2!< z5@cI@-9eiquOSIwS;mQxqvApKJvn-bGxJ|+?!W$r69iX^O~ zK^P!;7R=N>C@I{gmD*#_V#LY16G;?K?AtS5q_-)YIT=e5Y)McOoN-~S0;*gaD@yQK zU91+6Lz6O;+~5e(l9@2B6jm_XyUoMBp=j>yi5$ZWPWd#F=Pa-2Kdjsi7h+oUL5U7~(64bNIc4>{0uGbitO;-%?~F zY|kSpLQjk`of|vt!Akwk)~aa2(NGT3(Ahy+dIp4>;jLqb{%@?;deP*(zOlY3c#n|l z0OOL7+l*EkMOK27bK*v=v5{C$&K}&TZ6#K7PT$*!)ec>9+)b=~$7RS{f>$YS$$eNH z-m4K)08x_mN1fqg8&Fg?HMl{owcXlSBtf3+_D~w;1EnCRCBnQY|dau2C%&IWE=C(u^C`wUGD9lw8#O7D}Nq6&E!}$ze&y%7)O3 z>R?GHs|Z_N?+fjV&Mtqp%V^9AD#tRS8)J$ zktAxFyuyh7=Igf_& zk1IK$V!_Rv^xQrU!aR>^z3S7sK5!*h@nf}$Pcf}zK$1_U&TMV!oFRk_eedBz>-N@a z#!V`U8M~F3)K^I&wOAq6>y3@un|D6gy6cQw&mS8)p>cST*W)Q+!E`pMP@J_wnW^Wh z73m{rYsY50LP1k0qiy-v#6z=1W6wNj@|g$QaZ3EMj-BNA-VM*1QE>W-_pNa>4?Qc8 zD2uOYIH{!+^&4NF3r>+G<$k_L!TNcT1|wN-FcA3ZsLCNee z367`%Ce(LG=8xv79gvPd6=SPFNftq(Hl$J3P;3*;WFfxlB_}zIszG(FuJ-TLhlbZ|Wa70@GeT^l z^R``aO!9CjsSMPILGxkBP#3c(_`VOLc)YR(?Ew@-tJmx|_uE5K7xVCP38~eTt%f@w zgH)MwF-i4Qr`1-6J;{%Hf_#qwE?*2ch(PczGsStfs1QEqit)XGNzY8qUk_8Qd{apg z|6HXO_*3ALlhO#nYv^j5&kS#tq!*b>x3)VHyoa0J?UH+!s$$c+O`{&F6*Jyw5>Zx} zrEb&6M2_NnK-p$APouD>IFNH57eQ%%W%)KrK56inO`S1_=1F;Jnb%@yT3lb% zQK=D)b15Uct}iq(v2JiLxf$t&nAzMT}4G$d2WnL0yRm^ZFgziDTzH z@9a2$p=wO7(uY;8Pme;AL=B%T=~tf5F-Y?>BAAAFu^s}+h$erMps+41myUUvS2_p0 z9J@zdaHbE7>&;JI-R}?O2IyY1bD&qzjRoBjS^J>Xf8HCmckJ%db|1SnMpPm@x01*e z=EC`vstpf)xbgsN%aq0yRd6~=M9}W5WLfW9FIgg8v6jZ97tc8Y89;@`=(Lb<8hum= zpd;L5vGFE}rhrtkpu|-tx<+kv#{SyO6;9fZCq?SfpeS(9(0n12s!- z*xBZ)Azw|CqcvlPZG(5%+Wlc?x6^74+oh0&ZBdd8D~{ga8cp%!jX|e2FnJhQ*nBVZ zTi8V}6Ob^15rW1uW5TZ@Ty{8SNsQ&q@RyUBo85)%E)eQN6&EpHR?1v@_(kLLd8snh zl5PE$lP$@K9y4|28K-g#s%-n)fR(nuWqVq7BBt1Q3PF_Q8Jy}PX6C zN|rENmQyZ`(0@^NZp|PobfD@pB(jC&=SRFu7}V{gE~8gSgf^u65G^pjf_Yo_ZAGkOe1eT@ozbPatmvJBria9 zjgvJObM(9K@*GVcEF-Rfl@fn2VG+kw%-;;(X&2zt(Tu`dB$ckYOR=MnklP3LC)Q~=L<}L>5iZHM=PZ`-+JW}1 zP(ly)x(AO|YxPn{93UWJlUNX(B4H`n#T62+_zMz&$uxE-DKq4$erG8yM!676u6)Zu z78_zBIDJV`f5Ch1q)gSd^S-rGx9;D!UId{Ry&!2QMjJw-SE+y*hFUK(_}g~xt7vlWP299@L^y_YfUK4G@4oOSrlfwGaNLmhgN;Paa&w>pl=<4 z*W8`XR-BvG6vq?RW&G5WYIQWr74|-0FEL}^4QmsS+bXU1Ei5SW-nVuj{GHoNOZLdU zsEHCbt0PrxL5L+g-^l441xK<5>*=T3qYCuPejKfwF7G0rIBqlwE=U53^(dP9@p)eD z)A_{C;bL8;jfy{9e+53s0`a2PiCP5yf)r=gVy*sT z8%;hdN$2#eu{>Fc^JP#`!dB*_%Ffc5@{^=dOlmT<{5!X$_P8v3avY4)3ko{XeU55riu3^_ zj#OUthrvJMCJe>5_Ap-;<@Jkf9k?35^%6V{%^aURUxRGY;H{47Q84r1ojh?!Jsttt zavqkqh{b3EI#NWb77bz=Bn#BLK}h9A85)(@W|ZoJx8_@>8-qiEGUEx@zN{(O@eAb& z@nUhyBU9=_#m#(YRV(WNQlm(+I z<8IlH`rX5x+>fEr1Dj38c>OLf=G7iDW>xfR4Vp*k@8%u)i!I1rZL?7qzH85LF&su5t#DOtONsRsQ+`SoEHbp-KCsggwB|Fu9Lv?0;16i)Zhp3KTx6> z4q-AT_YsaKP@0=%$fbliodMM%aS@~`7cmH41LnTv;VwHOk&gR}Q$R?p)oConT`qQY zAXcEDv=XGv_ciw^bP#u+bwG_3s0~CZf%kq$y zg4L-O33}gBMJW+k@bTzXQ$8M5=&)Jx=}XFEiETY&{2mUUa0@F^pcP(WWFQ2^?nok-L4V>COm_ENoQf`<)X;B6cYY$%po;=k}i9i||{qeftE)dI0PA6n!9 zKq=M&2T1%c=WtOPpk*xQfU}C_1Td_`a`3oWt7k3}k23dWy}@p35!Ms`{1IYf?r$*& zCKEC|c5?Y-5)K(PR3SDb`(ZtUHR{-&-?=U1N*yoNt<+bsShceE$lmJ=hkI?iZ`G`T$F3EF zZf?D2HR$I|X2;Gdk(kI+LX7;>Srzqv!f+$p85&{tA(dq%+zCb*xo$&}3?EJWgzsWa zkV|%sCcq*I4rUa{}aV=OWVc|?R4$uKD^u^L zRD(%f#4gFIkS9V7THTGRX-QklY2mqmmIkey7M^9a2c5ytk~*MrLMc>EXo=d{z-d)= zOeBG>c_c?1JUJm%%nXp}Z;XI1LlG4#f!9^vT&lOsD8kyE+sn$Gfb@TQ9K?JeCOs#l zliq@Pom}k-gCqq+skVkDP$qDvGyBX?cy*S>Fz2?GO}wTqt)|`O>EFClJKMTra~^zG zM)qYymaEuHqJilo8iw|_2gQJ#jf54+m6@Fh(LC0a6+{|ZkzJNuc%hOC-!HP0T9eO< z#A23-?iH$LWg2u@<|#oRD~ATGtk4!YvLK@eW~{`X!j|GRD;%S=FDjgcD!k?Sr?-?S zX-JmLnbO4mu~>KpS4>S2^R)g?zG$}j_0$KW@33v zk{ixE;i}SGIIL7A*=7ezeKd3O*I#8zMi3meT#I%k4AC$vZMB3faXFa1Ugp)&bZ%Oq z+*1K_)p6x@3Z8th7TmD)1`97yQc!HU0k*>ERl!U1FQv-3Gr5|>XVHts3jQ#uU0+|< zvpe}Q^c0ad<@OHEAt1bwWHL@$9ksbBCuYc|^4bgB45qFcI83@vwHH zbTGb}Z_wi6?k$JYkro$&6vk@=_IOr_nOA~+DxWGi6;letjrH3eC4S-e@nVpm;Vp?f z2-7O?bR=_eW8#Tv5TfZlz|NR-ERaWRq><>k?cDDikfPYxc_{XqgD3Yy7%@9BI|Wb`6??gStKJ?6((bQn|%bzbhK+i`aSUs*+hOmFkkAKk|69Xm?v; zyW8DsHxESTVAy`t?u+ft&>9T;or6cB+ZwiW&-RDy)=+&cQXg9V4yni_mOmS|`v=WE z(QnhA_QBzPyWbqPcf@_<$*R{Mio@aVmSUrT8xOVXcIMiAC9JUWyZE_3+bLzBI zz#jb_Hn;cM%0IkQSFWd>opyKFex|&g=B9ET4qIHJeCwAg=i~N{^!J;+#~I_Nok1sO zt7tWcgAOiyXMUN|#@&Zvd%ve(E9xM`YDGcQ#O$76?J>K2)7)hT|eTaOlOiNwJSa&heyd`)v)8s}(psW$-WQID-xn`LEDHY??!OgT)ASprW1UC#@bxG)^JT|c z<%NZa4@ng{NFa`=&HUnINV*np*}@@hT~r}SzdvP57JSSnY|ua;pa3ha5EkNb4(S*> zgsk#n_m8G0lh=HVAJ#9ox7C>_%!4Kgwz2r`n4AdMUU>UlflV# z7S=%v;Rh$OZONdT>xeUdeT53okFmi`8ZNkSS7>z)sKXpm&)Hcenjj2cLPH{tnob@0(K~t#hqLu^;8PFEGX(Un%QjcMwTbu_>(yIIGA|(wMbY` zQ4OWeqPM-Sm4sR`r6 z1V^!I-LuxSdu(jY)!z*ZoPoShxDmMnzp%I=yX3)04W+K<|dDD+C%)N_%JfmQT z53W_A2_L+SuypnV`yvLNDlt9>a{WP;Wi~mm-`z11ZXSdYvb>Cq#+tIhX(UVKO?7}w z=`t?;6*nr`n!qb3_CZx*_1=_F2YKMhGN53Xn)7yUp50j$Wm;jP)%szc=4{!}fE9hl zG9nVo85z0qWC@kjvX-mpsO?5^aHQLVDo>Rd0;oeC;)25-W~|DPkh<7Hfr`h3)Zxe+ zjuTk2ov9;b>RFh4+n5{K%<)W78@Imr4>plTViJVDmAy;(8Y9<+8BisEx)!Q_pAvHagr{UA0T)9<1!Xg2-N!=kj(@a%VqBP`aXi- zY%Fl3G1>I(_6KY=nQ_u!S0b(lA*udNah-&O?7qEz{%>V{HM_j8U?Mpu`EieX#ms}( zP4h%QX$?Lodbf;(Bi&Coa)4TAR2Lxgg?gIpI|uEYuG<>)a^GPmcelFx21af=n%>8` z4-=wUc}KkWshTix;<45MJbtpXYj_X&4OSn^;hq6zEkrd)HfM=v;-<;b7nLFR@4+bg z`7f=1UhKDrkGnh6V@<8b=NEd_2L}=$AkV2uKqA0ZKzX-R?P5|FcyCobEk(U@mF3@r zUy#!5=OH4HBqrL3@^q&6fv9z_l~MGAc$8VexZy1lR18lFQNE=B_{^>z(HO-GDT~0R zqIb`-Eb#^Y`-uNaD^dneNJn zd*wJz68?kZaeUc=G(a({Jv{y5N8SF;!0wU6iPJ^3`uD(45AJx_xMbZN3~f+IA?3)`=K9Kbo$nk(A$AQR2VPIoU^21!6@*oR6m77|%KHMi=+2LiukdLpBfWB0U7bM- zLKDR|5#-eoeTBpjDP;_S)VWW6R{=_e0H7wKrrJ>aD-!nZNTv&KIwZ#qu4#B1vXv!Y zJV42Cw3A~ZI3M|XO(HMl3HC1JUkD<{wTHaDGRHy;UvD_s5AYb|wuBfVX?DbqLLKtb z-8@v|FSdNVB;e_I2M<>dutOfC=QpiR=&p&`iJL4#T9}l?{!W90-3Cr0(sxtzTen5* z-Uj3R5DuFnQj~ZqhP(?%4!(%&ZMPA!1`A}fkq-j$HbYO@d{hLRMS@L(fF?_7S^){> zDI^D6p$l9;u~snsljPDmP%}54?GY`(NAlux_U(xYUfZHPvv)8QEIW1a($Xo3L_UMN zq(~e&Ven;^3MA;T0COgXq#ZMc3WkgI1Npbcizm`4%|h(t0H4~`geEW(CW$D2?Fyvq zv;_`ak_OR+@Q!H|Y97Ttn{Hy`neEm#Y#P^mcO;Ez?n=Kq9?9>>m2{*9GM(pZ|Aq!^ zh|W&C$wf|Ln8RTtgqTSPabpOgV+i8ww;CKMb{iZ9eh|Q);diAU9N_ONe?)%jOWcHD zeU99VO-Ln_BT;GPh;+KM+;vs1Bc0B!PB|bbyt~53`$hVx_<1OW43gpb(I>$M;nd?% z%4mg@kAylXh6hlEdP_Sqm4#Q-Cs6jWR`!49nQbciYOWBvcU%@E9nr7;|gR$k@`Cya&w);zn%?gGZsU0U%3e zpiT2tHnI#$b^lCv%QNnVfmeflK=~vf4O5*=p<|`zbi-251wy8@4D5+|9Z!=1EdjIa zEim@?{5Xg_O=d`_lH5{wm4xO*j@qTj2@IVZw;^ST1pJ0ziCR7^xJW#5lEBR$l1nuG zHsF?taN~)3-C2>e`g@jw7edxXWJ1xxjZbQxQ%BQ+8Mob%+Z1pzqyY&!z!#BC6>~l4ns?egwst< zlItYtCZ|ZkPy!N$LQr(O8it~fux9zNF>UXGUgPA||;6mUEcwQBos^mA-1iYyx;7v6FZ>kA+Q%%5|Y66xS`*^Rj zExtSnMwVf)G@GrSlsaZx!Hpi;d(wG^o_{hE(!c>4BO74&$RFu}2QV~B^?0MS^iUvd zNmD0f!pQRna`F`|JeQ?CJd@+S0iRbOM7sunWk4X{4`7-L4#c8FJ0tvzbW2HD_I%OFu@^aux@ ztQHIXMAP0K7&L7y3R=5Zwi@sP9mag(l#lckj68pmI@FtV4er3t-3vK(d7+7XeO?hY zd!K}G2KAaD4KO=ceC%Xii~WOVl!VTX7^E@md$ysRhCY?)jyY@J#&Q4Bv`vuhAcfjd zAf!R1NOya^OtxXD42oz23i^)Nza=Ho1paYM9mJbRU4|7KOa(jg>1y+F2vP}6KjTS0 zpyUo3^mo76stfE+WCIp;-cQq-J4=KTC%!%qie}kN zKu!!vgZ<7v37burkiG@C)pBOeC?L%YQ=3*2i*lXR6V0;5xLT&!EYWa%ELOFyU+Pt7 zY2W|U>{yx6yuPyi<4cl71W#XHm}MO$LjoiHb-a9n?u6qAkDHUspHIsG##1TtDI`{N zX@ZerMnaR0rJca&v4YopwWl-alB^P88J%Q^+9>kQ0kihmZ*41o?Qp>0!$AS_>~3xI z1Nythf8D8cTCIKS_i@Xt^WRNbU8<&#xnxR%XKhMiRpxQi-0e7qdvatD83B^nr)KD& zz_;d3lOh1+QvgJPR|HIxquS|4DO5`ONA@mmZENd7^xY4t(XxRr4Xw~-Xg96JzUv3G zv;aekV8z`8@;Yd8%+H9M@Zn&HHd2J`EW(tIx&YsK+8U^Kx)7(tB{LFD{UXwFbjo!b zH&KaNG1_xR{-l7JPTFTPC-fc%IH-PJz^N2Ih5_+HnB<6xPbt(Uf|Lno)-rVO3T}o- zTj1%qPx(!n0>4k^q!y4yo^ryIDt>)`?8mIdlu|;ICJn1S>>d<~LiRv&TrHsCP*Ze^ zjRKq4OcEJJC)lJmW29D<=tkA?7I0YG3ess+$*7>AAMYt`atRvC3=61;#H^U=qu3C$ zSY>5svSP&U-4iCd;Te3%P`VWc;Bn&q~H5yDU$*6T8QxlvH!b zmUI@%+BHx7=1a4EXKI1Co>Bz#ion*&CPU;ynRUW#tc z$U%L3DFMIJhOV%`bDQ+9+dM^|1ZR5s`jpcaqzcZ(qb4nP%GE2o(ahsf(sB9osmtxq zb#=J2dd*iRU)4k*f%=S80hm$V-Ke0HK%NET={R%C#K3vtJ2myFvtnC@8Z2ashVxS; zrFh&ajF2Tn9@x>#k&MvL%))U^8@$(;$W5}?&_RquH1H1CD2S#$v2=N|+SWdng~X@s z0BiB``2okJ$&%EGHorXqZ|cNHehQh#aWFN+5sl-?=xA08__29;6qo@t%|>A~jphj{ zMF;b#gvNZP)0j43Fjz^c!Hj0McIYW3s?!_zE-6@^)~XmkoWoS1LMjgX>}26}8*ky* zMLg9hmcEaifF~FQ#bK`Alg_@f6-~g{KWe4%L^*Yq6%^lWHq|gEUpt6m6G2^0?%?v+ zP-aEgrkobK(uTQ-j3&HdPkNRaLiGg9_}CN9lUT#`85B=q0@0sT0b;oJ{Djm+UTpGxsDakZ6LTVuPt@=Jinvt;q1BaJR zq#VL=@Nmpltu=_`#bAb|ngZ1EymoFsSOK%`t6sO4MeHxaCb2_4QrAKy_X9}XUw0tw zm2F%m;*=`iXvh+HWR^y(8nX!_`uM!3ZU87473p3Op7iDI!XmJgCe?J2t|q^R?5HA$ z!W!&tVyjRK*wj%`%6@me9mMIe2C+TK66`3O^x#uFm^7D>di-T7(qlPI6>!v3QLi#~ znx)LPZ!~enu)5h%tw2NZ+6v9&M`gIY@2fmC>iTN_<7w*)t-*PiI%l=hx|$DZ($Z67 zjk>SSyz*w_2~Ljcc?3n@+_8!)a?Kr^$5F?#cqrvXcIdtt3%R_`>Tm{=O^{*I*@`FB zQyIW3TsuO(T8(3nIuxnp8LY@J5n9>GR*K5k%>JVPZ_+vPRO zz<2wPu)i&Kx;w%nxv0}TXo|oIojj!2$z0Q;X)5*!dn9+9VG|ukZ>S+ma84)psJ$9} zaz<$VQRt%wA_=Fn(}oxv?Dy!i?*zE_12QH%A>cH zD%N;ZKcbF`m=nPrYR1Z5#SW5MS~D?H4l+1rJ3cLIn8`?CL<(0E=Y2Jr+ODz7*vyUs z(-$@)?IL@IiB$W5riB-gLQr0OS5Bjt^W-tCyA2u& zGG`-qZm9EF8+SDUoQ>S62{!MX*5$)mHE2w7=Y+ZhFLN48?&y#JX|pld<8b_h-Pq^Q zZY(%z8tijuIQBX;yk)>$4ZqvSdERa0JnuGgo_8BL&%2GB=iNrm^KK*K38MmN(!Sk4;k&ZfBK=sN?^N$H;89=#UU!vq^T^A@BTIT6YCUPCJf-~XN zB4s&P-{&s65JbV{F&Uh>aA}wbCNz}s->g5f#$5!-sYU|j2+*34ged`!FilbaYHkkr z&W|E?N%-s>g;DHxN*IHUboY3^#{9q!B6*QJ=i!Z{q-o~EgM>JsyW84oGYV!XCduF;6CjK67M15m)WGFj04eDK<@ z3(8%RJvqk?NmbPCw4Xcbjzn2-0rlXRnd@>|vgZa&>vGX(4%*ST4OB%XK@AExLB45U zX;9+fWmLnz{j-zwgK}%HP=EdQH3b_n%>5T`cYwiCBkxGtDp?E31Ta{W*w`y(8_YCS zOusfnl*cVqjdxsm%E+M;eEXcG3u_$|)DutePn3v9k|2L?;9nS8Kq)YBc0P3IDUkTx zo+aSKHfEqvKnoI2#V=A|eU#LJ6CXqkTy=mcK^h#*sZlTq1wC1dWhHM4I}s_#RwB|jt7-Ky^16OTDNimI^5k#C7RI!VXq<%Snmc{`7h-C(|`u79*KD` zh4@mWtOWKQlbJ-dd5t_(gjYj2mX6fTVa(j8~NhrGrT&0QVVfhKN6(T~FxiP4Ns^J~`({iSP^Vu%%N@=oa$J{yS3xGt$ zmU5@`{@+pF;AZ5I_PJybz>;GF*=63nf@tu={>7#p4=j@CoegqS&`Od=m~DYSl8RjT zw5uyyNo=$oCm2$W8^7CAs_}dIl}%mMvBL?nmK1LcnjIPb?&L-XdixhtHWGwcAXQ(2 zj#p_gI|T{dUE>{dvCrAM=?B2KnL!CXl(u%?+zH-^C#@=O3w?0b1 zLQXQt%bhM%RouIhZWhxAK}e}4u$n1dTD4-{D~LFitwMvD9}j>I#zdbENkaQ;_6Me! zGYrNNX)e?JA&AbrR#}5lN75gCr|4#`h@?h{%K9%|{0L~NI0~@>Byk<~j&2^dxX5@^ zG5a*3?o5wl_iBXjFln%h7Ij=^Uu~lkTL4*opp~&BwRT(hn z?$vL{QfNusAOEdsmHB@(*#b5eYKDU0zBy{!FD;fAMK2YX*Uqq{=)l(AbG_qRA&WJY zXs?S|wu)FEKpx2}da)O_*A})XEv~aHt|1n-D=n@!NKbV2K=hKQgL4@rCGg#w{lKZf z;{12Im@{q+W$n+1e*L3Bi!?)_W7hU%czocws) z)GeVrLg-p$46+9&*Lt3&ykwO`vB2aj*yb24oQb#%Qr|a0Bt~;H9SAKF&truH)S^(_ zv+^_2w->4J)1ol{7Xwh-t8-IRZVYK2g3L~#Z|2d7%IOD`^+Ldk7p$~pX z60wvy5hUEWrD)G4ZIJy3T*e;6&1uHw%FeQ13S@l@!(Vj@fj8nCX=WT@UvbV1QWI7> z@EVqy*U#T$2WBU$wJ#6FYX|j-n1{%Kn#~FVA$gk{nt*7{jRp#**@~u^e<&pGH?z2y zP{}2-(>-d7dqR)SE%$5UUDe< zMzH>faMS8q{4m&G+R^voZY<4|ra*l`2FeQ^$^J7i$We9?$bl>((D<0H_&jJ`9laky z|D@Jo=+!ZGulLZ~vE(1-6gc31d~N?dIudx! ztt*{L8uOJ7rbImDx>>OWCHq>@%XW?j_XW?U~tt8lB2RXBd5r_%Xa+_%ot55 zjmSt3wS&lSjD2iSc&%D+-W@eROO5c9tdM-^^nPnepyQKj;6t{;zoKfrNTf@b;x?SA zfZElhVN`KCsl{qkQ6gi|WBnR| zRuH2gb`6k>BX!Ql$srZVc=wewU=G~#i7NyN3croaztICL7AvL*L1%mQm>#=1<>;2w zcH6is!z1V-LAY=k>JN+&gL=XL!O0C6%56*t+aE$(!5m2a3yVP}0-h=&vsN8kQpFa@ zwvHhekr1QG8QcRWsAydG0(T4CwqFc^a{9}(-;KC}DwwNs5At`>!5F=)&8DT|=8%&} z$=)}@gzW@%H~b0z`N`a$^=c|42_5N-O^L~6gCtEYE1J20?UWVVcl(=RnL{2}>w}`7 z#?qX&tKbKk$2nV0Pi;E9=T!c*GKRekHAU#J$vhI>$9ChLwZ6HX@w=mYx>l54Z~#~4 zz3ObrR5p~wr<+4!eRi-Ppgl|p#;3xM1LIwyw1t`&^7oztCq027yLBO&mKAz30wRk z9d?}qdyxP(Qme<nLIlD>{*ay|6J^PMZ zyOrb52oh9q^hf-QMKC_gWqcq60|Og|{y##AOguW=igj*mxVrVsB8g-R*dz29eT4$2 z_GEL!H74v6l7X&xuyR%;^#7rupNp{k2h`#x(@UEJ6L0 zlRQW1u z^NM{KH96pDTr+BgVJG$?FKY4Jyx~31QB(Lv`(loNuI_(2cKX02m(kCvCa7uwR(f&> z017n#HnWg4RXyAJ3JHru_zl|NSMk%>D`J`rzsbSCupi+e#8WV{tqJbFp}2T!z_Y|4 zglhsW+^i#sw|q!r7?)BV!#LEGm8}xdD$^{V&y6jiGVyhm-8u?c`cL!UBZeYz>;@H% z#LGvm)YW##%U^bcxn=$8FR8EEZiIPP#G`)619NfI4Vy^6kno%OXSauuqTl8Jw76~5 zgfAX16^WR4pxh_oM{X}Fk1CeHV-E9}dIlv_E6{_kHkcBBHN^2zcY3w7k3Z_Qd$69+ z7E35iNWX``rmqhEsIigMu}uH#oBtTgDN?8D8l{E&N-u2O#iiKekho`tcPO_dk>klb zATE=+63sS&Ii1OVv*N%N-Po^bgC>WEWD;n)xeOG^+n|W7L&*B@kdm*Qd-AgXhqPq^ zb>>Hk_rqS!v;8?b{Kh=a2p?$3x7$B8qJ1n~qNtSZ# z)+-*W@Tpht)QxT)bCbua86;~O<&j!W<fftCTkG}KS~0N)&Fryt%J=V3TI z*3#&(jb95js&ca!)9nwI;Kk>m(P(S&dM1Ju01h}!Qm7P*-gM@c#&6FQ|$$XZS=$cCij5$PJu z8zwsO#DpO=Lxs-UmuXWsQID4bBr-&ZMKKU)sIB6}-P77ZIfEM5=U~&F+EI^{9Bj)g z#N`6XTD%MR3UiWAA##-V&(Kv5n)LeAuA|+4H}P($uq81={xS03Kl&yQx+AH~7Xj$aF}GOfnke0AIuUBwJb}Wis2jWnLqg zL5 zA{HRmm&^AFFY?q348Q)7rUHjtDe>c=?J&`T?Tc0<;?_nM$WE7g3S*leTC87Ab(glR0bJ>jxJK3e9R3rp1t;E+x9-{8H?uyQRk{ zlIEoQ(mH44AI)k_DX-_R ze`$aat{MQKgwaQzBz~}dYH+moO*N6olS1J<;Us^QL~fFY zIqu0OksL!sA6c4V-5M{Hm$dO;01uSg7HV1Y(D2FhFF&zO(CqVFp%mp1_yK3f8S^r9 zhKzaIvT-ger~v#>D^*dBJ}EqCFZrG88MYx0j-HCV6^!YZqLEp0?nT41(Ch3>q;&sc zw%3ZNBvuoR*=7tJj9q>o7yp7E)o563W~3w#IM>$CE9=nvXtJSi0z0JAT9$MVUDC@l}*Y9Oe zOtZ{ zy1cCUX94|kl5otw8_TT@us`kuMN*WS#w`t)AU3Hu2Qbc#7F5o6@=f|04%x`LT3*ul zt#I4Ia`}x(XE&1#Bc24!iho%RwMPW)og(3iE)Yja-Q=S}Op*Pqn@FS`v3i1IbwwVh91U_3=KQp@kGMUbC+zQnH=X9K>|(Vw%bS?7mC`U*$lAL8nhvF zm+1J133GvW*)ReHd@3+mX+>8xhX`@?UnMA`iF4hph4~3R57kn-Vjw-beZv|&2&SGA za>Ow3HfKo3cMMj8Ae5jG`@BBms8?Z*>KR=4BWs(Mw|CH@#t2kWlnBpwB;9BA_zm-` zYln-S+?Ue7izC5uUZxxv{|QJiP?AqcJz7epJpW zRy=p{tta1|rjQXqP_1GC59U1UF?ZqWK6nPgfg`dxnIy=4JEk(vajDKil|`!zQ#o&c zGnQ_mcjvKYtM~cX_gZU`tYk&Y$aj^%nVoYm9f0X`zmq-;VZ zNIH7YblXIx7L_ni=22YZI-WQxPQqN+sO>ypEKZcDX~0YuK3ghl;%_xSvHHHQf`JoLV{PisDTN&C5EW?ea>W1LOe zIrqYCS5_Mv5eGUIWpZ=m15Od-f=*(#o4rUW^r$Lp4yB=&K#^;niNvBxmWdY@{ee>( zSALCnO69l#xqPu(_sW^UOZzb{1&^3!;p~#n0xl@BYFcU;vy`hBmLPXCX58e#lgG?7 zO_N_w)GjX)%ZQVoWb_@znP6;aBTSa`x@W7f#5|~SRbPM*0F8`8H|*b7Vw3i%{)tv~ z3x?V+KP7K8IVXa67&&(T{dHV4seF4Ut(;Rs`X(g?dP!jZ{u@)#V$;X>Ym8ba+nEyGCu>ZYA?00 zEwS;fx|#+3H?F6w+Zz@2I(>ZVyCfT;)QY1HPe8tLw>C@jNj#E12;HZbVwtliPr<#{DHEy1QExM$;#lSTFk zCobv5#A!~i|6Gl#A7-0MbaZ%R{1`~J=Gc-fsn?pK@gDjbScd%e zPs30w&)Ea13h%kRNZ>y+S8Za{iPR2zt6-~62+zQQjUo{k!c~bbj9hP`BbeYXQ8)vF zQ5B#e&Kw02uAV-m{)@#{r31^{=HPDOSYfPMTr%N$U@V)#bC6&3Q;#s`UGZ&6EK(+? znoD6A4sB|_eQ#bovD0dfAajD^a4MI&x#2pWYhg83Q=DvBi$tA6u}08UWzCY5DE*hN z@J(2d&IpLcCX&SJZOcs=?k_-6rkvQUag+!sU*w_K@V&&InRB!bt2!BU9|z0)5s9H= ze-p1CrMkrod6!FqqZi&J?O1*s{nv!cP1_Su+4)4-Lcu?+X zSl!hNC2PmHNE&i3e=-7U{P=`?y@sT&=wv3kP_EI=y%-)+)(T%ntx9@!15Ze2)(eSD z3yN_y8LE=wks<$rAv6^Ewwog(Gn;-*u>yDsxI%FPK_I3QCg)n7K&_)27A7NzdK^lC zegZGnxDlx5Y0-M-?RNQ-0*n1oR_s(G8Ti5CbTUOP!H5_5<88hh6GwNd-IV!}5>mC` z@klh4%!$&hl7d6y$jwVJPK5?AjtP@A2f(e&8b#Kd5|+T22rp2DpME5nPymeT`@POx zgM1K);tBJ-qN3b@562XwKw$&@PLmhZXi_+VNjhts{fbJ_`CHMCNmBqSjBy0hfYz7G zBTxboJPX}TIzSljElwiHM z@WQo8Wa?Pkgs@6axP@L7Dn?TjA_X?L8ki+&GPibrONT=CTiJ|dw3S|>s4-7+J_N%# z!-SGA9EQwwFfO_8(N~8~=<8}$qm01quWd$bs}M-z3IQeYnMyZ|%#AS#JD9(Zv0$R{ z#p3%BUL%j5nb90&8MRHDNx+tEP)1vR3!UxwH>2>Cs( z=u5`Q%#<^VIeLl1+Pab@G_U|G|j7ITvCi4Yd~2H2OVc4p+L+v>DCGr5d|)TiOQ zixiNQU-^dsn4;YG_o)3ltf(-diF#tfYe{>_rmE%v+G)PdASa{zco|=|H?qZW?xt?d zWTrbG=aX;n)y`t?6j(NIwPP1^j*3$sJ?c;7@ZmEM(>J;>+uSR**=B)m%YE+*?=3RB z&8zqxiG3_DwquttK_cuFdOh!p+oI}EKE03CqJYbp(`JZ}H_zRRYsW2+%`ZmM*CV>| zs~jc0kGH$~H@DXh|JTp_*V8Tf*{f#1J^9d3-(?i-A=ZO zs&8hQGEEvwHj5|OI7ki~aT0_u9gBS9R^n2ZpfS=g!`upddoNrbu5K+oiMmiAjH)Oi z8#IxRYIVGMhV3{^zGLEHJIe34wH$tS$2mOhdI_C@bH|oO|4$#En+nJ;PVpxC*bq; zY!3EyeBjQM9y;l8@E-Pm&?i{Vohv(ke<`#_rc{U{7(9bytY%Jrq>x-uSC3vbj^BDb zguiRf&vv;V++MwW+P-;Tx~6!5YO8fWMQ#PZE=YDiN%ApwytJFQY_T0LNp?Q~mY3eN zUN}$2Pp&WzxJ(?Kl7Tfc-XHgYD#HA-cQ?`a_$7>=UeTh7)8*aIPiYk?)!MD@Cu0Xo zdfo0P5978S_78K|DSGX~UwMs?-u>urMtOYS$Jk)|F5c09N|fX2U*YdP7ju?mpWSVr zrgL8}Vp+{TCyspYt9(~q>-ZlLe4meeZwGqMJ6m6A_#SsYuhe{FSnx|MzjE zkZ-5i+vu69I#8+eGsv`lxOz)9r!Tw!=GUwdlDMwB6#J*R#*ZX6O`{izO>ocwU4#MZ^<-uns=Vhb& zwnFcHjqmZ*=at9j7Tx=W`D^lN!_D?VOl<3sZ)%aR>;64v)9Z8c>)7_owz;eE6ZUI# zDSC5*-Th4~2ls0n-}^w!`&93vyVF+lZRl%t&()?apWS&vHvKI>Pk)lgGl7y4QpH zy|h=imKT1-RokmgS#?nt^=ZYpPRbKo#Z`0Z6|nk_mTGj+vhDk`PZ1w zi_+GI{+EYb-!#s*waU&qY9vC;bOD-zU6}@#oNC%zJx^G3N0WejCkN z`{UO@QU2E(Ece)9iSqL_)iS=S>&4fJl*ku6hHKy#ChDf*Grr=t2%Zl0gU@UmQ8!}Y zwc^b}K=y0(n`4^ptB>S*q1r@m@F@OMm#Sve#P!2V>jTa8bE=R{2JR*hNN)5Pnr2ly-tOEo&S9B@-U=U(ul0b4B2{>gZQ- z5NH*&EXJTq())kKx7`xneM-wtud^w;#Q)pew@y*`HVa-ZElk+CD=bIemSrmZ$uV<&Nw4KD*U4MfeNy0l$(0Kd?#_Q2eg9(KQKGNaAmzS+KySK*tg2_P za8+RUOR7yeg?j(X0Gy+|_nSxL?E3y);fB9qeQf5nRL_3bx&-ugCv*ZY~Nx^J`kpp7p+@t<$ z{J!YATB7rXrBA0A|Ajs-XB#EkyYyL#COLMoU$e#a4k?0M?bzc$2H>8w+b(%C`vmYB z%-AP?Mt#+FOZZQsQ}lc+wWC*XeNW8iMajKG(N%$h|6k&}%K*9VzYul_YP5*7@fpu7 zq~h92$mG4FZb|QIqdpH8mUY|mA`~4`OI3EFBhzVwd`nX-uU#@*%}}!50H2`m0k5KR zKz9>E-^_81Kz?k-r4e;J-@V@U9-jX^y0HHEXwL`hYn(h2}pXW_x1W#MO{+*eN6sk29t_bWdE_!`{&zlKiP;p`+%$CQd|5avfQwq$V zBQIne0ZK~Du1*1p??9e1t*>MTg#)yx(3riZ!uPipqUW14Zd?6=8AV%fqI7+yBSl&t zgwnGc-O0pu5&g)ROs%|QB3{!hTSng3D3m)=oYRoefqQ!7RCv6s`x$#-%|H9iu3?4{R6MWb z8hf!*bQ~C8p@QSw-QS{H@sdn9k$vZ?~g~uTA$-uCXLkQKJ0urHS#IqQLj) z?v%_Vixa;)x5>4`@3#TgTu##oA;ncCrlP|CFUk=NXqq07x3FNx2@}pcKE}Y9F+)RBRdsfTgQKI<)ullT zK}JDALPAAFMn+GGCqghbHa0;;K|xhfo}ZszkRYL@#l_Xp@$&lhigwX+)YRGP!j?Z^_adGDHc>Z$d=hxetmlqeuX1m#ii|_O0ZDHa5x;MPCGM&@y zsiCf}t*!a`>E-lrH2v@p6*Vcd{dLgL8IF-!SUcCCY*!lrL06UE!?65_3A^mIXj+_d zm>??Od;JH20V3$;=tGUrWY#{fh_p~a88RdSg1-uZ0z-1g@n`#0My;87ADJH) z)SIb!1Mnq>n<1##%W%ybUq(xiQ2AQH4y7%(WSnvaBs-V#BAU8S2B&E@gK0?o29~}kA4C4 zF5VFW{4!RdDv9ImEW1n%(HDI8V*gy$Fx%hG0@w6rKFO`m&yei;zNtKXMqn((dHp}$ z@R`mBe9hWFMesOlaUIXxhjQpecP-wO{8?T;s`b;A;90a}aB>Qwb#;R(O9tV9N%lB% zI~OgUU#Ek8WysysNoRipg`P`uFzhqNqj3{lmD?4GOt0O^_9yH@H*|(Pc-S&7H*F-m z6uM7V(FmQd5_yJSo-&^$>h+c$ykW^zvANx`+f~P{-swK-OHK7%^I&fp`_A;-&)XCu zBp#x}x{N@h_UI^NXpO9KnT7gvDeKjczV5$@?>UHdAr=jgZ-oo4i4B6(@?c^3GRhSb zuJn*fKX2;OVg*p2WsBvM?|UWQrr|xdoFGU#OPZ0zPK+Rmh<*BEoIcL&4-JPn4uBbA zp7^`ERPb&zx6?N&X1wOkuTRt?D+y=HCkuv%5M=Wyp=Z$fJMWw%l?l_wJHxH{IF8lx z+!NgDu|5~pN#U?NYwc++U>?c&J1o_5MI*Eq_RAFcV+9^Fa^0TFiURwJDrms_T7LA8SV zif-LT5b+t9_unKw%Nx?3b=u%U?{&fRnG;cZSQzF}oAsJHH%vCNhFEmlcJwPIoG$ae z)}A=}>6agr-dtlT57F;JDsjEN_t=PK%jxD7tiIX0d2c@^Lu20h{3}kF>NWc8JB(@xbPM{+*SD^ZLNgiR(?wfXCp6CHRSQ0EY&v|HH-?9RXDMCpQ7u$WJO?tX#0; zpRc}9`rzt6qkXBj;aq=i_D~X_gn}CSxf0-xfI9dI=3p>aUrAjJHQdZ4zI?KoS0XoJ#0Ecs^iW^H?4Gq}QVgJu7u z_f_k$)JFgX7Z7S8%tzoBhRgq<;8!YePjHAp9m+C*w+qVwb`t<9uuB*gD=q%B5JeHH z%n!)lM%WpP9&0edZj4n2sp0omh?QVo9yAee4DKKhAQ+T~BoAAPrVLRI&*GOZWSuu9 zFK0#p#KD4>2|5&n&MTU+Fk^0jU4^{}coc%jL!RL{2LA&M@FUb0#~_Xb3-+_`3pB`Q zpiHHlK>7{s7zEbaXAoAStjJYb%4K6svNzIgs{K6yTReskV?-n4A547t>? z?5^yr6uXS8)T8vR^sNk{RIH4=^r-ZxOs&ko&Yc}P6Fxcsx?f~RxCJ~DhAk+gw{pA5 z&ZC9=9fcby``2^`dN1|P5g4`2{;Etn|?N|=vW8!G%gO$Z9%rNV!RMG=%jIR-!s!D_>o`E$)(&7aL6 zfs#NA;43g07!8~PUIM#-KY_i#Jm3y+7I*_J1Ev8rfG@xz-~rGHI0>`{egKJq|A5TE zJ75HG9Edt|_W1ph>9-LYuajzL>ojx0taQ!y3vO&zj7d#+t>N(iGej-Q?F4 z*OcKL?wsx%?HuQvhcdnJM|nIo&muP3f2ttX);q$jo|{^jrO>mBHw?Cs~BgH=sY@T{ zccy{1zP5q3sW78G^9|EZs&Lx1G$)q3ME@S<7QuAUQLBJqmN2Mvz9xO6p4ZO2kUa3yR|pM-WFe zM_$LDjx3Hmj?9kqcLaARcQDM6V55B_{sucHh{lK}z4a6|rM0md4eIwA=Nj4?>lz*E zg6e`AL>fpM@fu7T4(g}sSQ@M9oEop{!5XR>r5cET@Bf;qqNzHo%Bl*e5>+5oU{%0X zKvi&5kXNu(&|Chnq_QNl?m6p|u)ri%R)ul;fpY|(7iY}ss^Y~5_eY|3n~Y?W+SJOujil)lOOz)>2svfK!t{%1?+ZMwX#ujjkW{YSGu*HcVh~FzWJv}--Og|EHM1Dg`pN!G1R!rY0uLlu#RB^X!g|_s@%~B5&s$aqusaEH~ohO zzf_NUk8Y1nkJR>G{!so>{s#VNen0*s{w)3i{u2Ib{+yngp3t7uo|c~Qp7NfyAL6l^ z2(%+~6g1)p%EJ1Ue1+Uo>RPyuOn@j=t8u!9JBfmOj`%>%Oc$`oNMv z^gu$Qc_IqL2FMb~YRDW&0bv2%`b}UBz98UCCWzdntQrd$}8e8@n5f8|53%8_6528}}RC8}A#18>kz~ z8;u*V8(Mn{C>tnUC{!p~C@&}@C^0CtFeRi~q*f#_q#`61BqyXSqz$BBNP9>G5}_oG zB$Dwu@oI|lih}VJWdvm;WrXIG=Cmi&ZmLYmOqxt8OiD~@b@FxMb((d$bux8Ab#e~; z4x$dSNMu2=f|87qP*P!%aYcU$VTvM)q>E^ZJc~dI1q=I%h>F4s$BHZpISXHlbj712 zqa-H9C#5RID<%9273SE@70kiRQOzCAsm(3TT~3tdOpAmH3yKno{uMzNNfluiY8AB? zc@*i1!%M|T$w|?RS4-K73qeAHaQVd$NXHWVH_@D+LjJV9aoe)jWY36#P@{n~g1mW& z@-oK+e~`hU3Ig5ws0<=}V-3D{9(LoQVUG zsuj@{yp^9TP%9!Uv>JgpqS0Eig3%NuL}JOzNsdW(NnE7pB;N~(2%Rp4AcZ1@G=*4= zK#hKhW{GNvkd1eRLd31 z6)co3R8Ez!l+snxl{Dow)%+<6k~9+Cl7fCj~sVJ++sz@yeFGw$FE~qR>JtSH* zSp2mpwy3qpvZ%0VK80RjT(DmdSx`|4#gdHDiBgJEi;|NOj4Cb8EzT=0I;=P>LCbYk zrB$cZq}8BRqE)NZu9U2ltyHL#sT8Z!v(|tuj+QS`GgUrP9WIL~mnuVE7(GosojR>L z6*%2FWjqBtg*}}-tvtm&%{t{d?LRFzMLuOdg+A3f6^Bg>_74sZP6yZMm+aT=SLxT; z`zsnM+96seS|nN_nlt1(lsfcxD03)(C>WePEQUn9F9DJSHXcnP#8}FKq&1;Kg0aw8 zk+3XLS*DzvB|%hTuaIMo$~-ENfOs_07>x-(O+1;nB9>)D)mW_#og)-ST9z~=;cRqr zU)6!$Ex3bUm#jL@AbmdlJiQ~GT}DBMQifKBS;kSuQbtUMZxZ2qJ)2FcPa023Pijwc zPRdT|PX3%^oTQu-k+Gn4r`4qupf#Y)PxeoaPYz2iN-k0DQ1w?WSB+CGQ%zFMQmt2Q zR_$I4T#Q`ITkp?PrZ+4-tUatiE>12?u1u~_B446dB3YtP z^5zphyulauc}DbwUh;MOg!q~D$a9$1mI7nY{60*JPb2hj1IE_d;P;XbKZx+dG&nA=Y0F3@r=vT*?z-hH z{P{y?R;~0jZ)T;7`$3mY+r9FTCB=?3RTOtiE{+O<-l!1*EZA89npu@7W8J!stx0kd z5q`cw1b=BlqutJLhIiu}+)xRL$D2&+fN-8{s!NqS&~g7Xys671x(TAV9FzF(C0X{% z!=L007ed9OJn&di(#C?!)6obv+2#bg#zL^eZac1qEB1G$P7LG~s-zv%SCl{cs0#Pm zMO8=M=PaFTQC$z*qv3R+7p*%GpDY&vt*sw7Uo{tBnFnuM;5^YWliLe5A`O zE!8hp!3k*u6W0Sm$Eij-Xp&I&@$?RZ`S{cDo`_l$5pI23NbpK6iW`&lk(KvPJF52K z^i!=CjgsE@_QKM$I331o7If}{FQHkxC;aK|ZWCMqFogDcYaJDpl$5q2KHf(*{|jY6 zn!gswW6NJzNE9P#@10wtZ-SKBj8rZguvIYM); z%*+{_r_W=>KORP5BC0#^R9no%-R{*AIk|N?JJct=f7z+GelX)=(LAQiB1cZNQ?P zkzZCOY>3f@o3B~H`BL2~ZrzA~{9~V1!1ZI)dM?_7;aSnZ1=)X9$9m)a{cq~%F;N{G z8!SNaf z0Ij}{~!<7=+1wM*r|m#RcvS49GVp-Fylw#H^({3)UG6B%SG z;QF0k$)fAY08)A4rtp^qpyi5vTv-+ru&r_nA~6I2hmN2eMMt6ls?~_w=Z3=p>&wJf zKUh8Y#qVO_RjEk-47`f4O9C9=K=tDnACzUrfWqj&77KhT&^jvkS#2#3_?EQnVC6;z zic2O(1HWtlVpQzk!qgA%i|;4yCp0D%0npoWnniM52X1nnmsQY^2W(D9+0-U?00Nam zcP9oLfiJ&uC;HaMfKW013vW_80PV^7q%-S0`{Eab@I6V&GXff|l8e7^z5ve0_p}oe zwgCN~h_o68A>g`kmzdlOZD74tS3J0}4~SI_$e})w0Wei#KYeiD+!tShup)8$@+9D6 zm(W8(^`)%2&?o0;EOWi8va4 zdS8457poMTClsioe_nFA)E{_X7%N+vdmrGYUH6FTAp_xWWY%GyrvY=1C+qpo)PRpx ztr`-WTtKm63<+XKw=aI^Cn64i_YLslrR_Piv^H?RMRB!3`8^bWO_zqkQ-E;A*l_~n(Z2W&^uA}M)9(X?2+x=J*-GF|)PeO#S`Hwp zw1~ge6$9kmv7N~i3j%!Gp1GfB!U0kTt4l_Ci~vux;|R-*$$jx_7V7d3@mKq3 z5H11@5?AkhpPJbhKbT9s{J_a-;B~=I-(|%LpzEbvTDT||FiUMDg+@OHc&j1in#m%- z=X^_PhBI)$KjUeUm*xu4?YOzxus{7RGEAeb%S=y`q9 z<=czEmJ7^5f_DrU1v9HqSH1&Mhpe@tZutX81v7Z{iogCJfB$x5Wz1X1H3N`XaEKp9 zX8@R+qeg|o&j7;j|7?PI4Q- z(FlOgt@H}>*#sbAengGyIu58~P0QztuL3^532LVueZ4RKH=ww>qcjjm{@9)KvSJnx zhInyE@Oc6)MCtKald|d#ha*1u8oB&+Hce>ra!V8$T2&T6D{;)4TYlZ63 zar+2B8612*8VLqmL5BVqRS#fs>CKJjTg<>3nn4DyTRC<^ z0tDQ`tqL?+asUgcz`&-TRQuvLoDW}(I%xo$$^NQHUETx;b&N3bw5bEPIq2zT>E8lL zk!wYH&i#P@y3`FCIUAr}j}$liD;v=HjTvh9N#7U$cuU)XAu27v>1I!SpI!>pekgpgo;?E)Msj3y9M1>txaZZRT?pR)`ZBrh6T(pm;*BY0TyW`-2bn4dMlt`Nl3?{PZ%~_L3mm)~V=!+W#LPuD0tMUtVYJlu9wll6B%Z z(Po2+7w@uE?kA^&x^ZqedfG{4Vy>?syIE{f9Mc_)Yq4sFM|r6gIFq5l^u7tKYX^Sp zhu_qwFrnvRk@p38JT6aH_JgzR@L5OUN7XEV8af)>a;!YW=;8UuvJ`4sAuXePHA{u3 zAI~_kq|!$E`FdHOZcN(`Ul1|bs@-1Vd z23a~i9BVH45vSsw+MZXOeg1-FwcFhe(~sVbHCBi3DNE1Jg!wB!AE>%MB>Y=7=S5^i zXHEl~ZMI{r!+!W&Wz!rJipQIe%FpQIUF+}9$Jl>m2GGd9t-0HGh3Rw6Y5dL+bW@%v zKW-e8vQEEtDjQVse!Pu3;gOG=w}pYge)z|$e}1PU|Fk$R6#QtDLm;y)=9>2vFA@!B zxn>_&T_eo%%P6!20SUE#{y=3lTBRhE{Ys_ZX+&3BrTZZ|?aYV!;mekv?O3un9`)iU zcT_yp*OCQ~6BsLP0#=&WK6y~4YVlBWfGr`QBdf_>YW2a^`ACh^!w1-IqN(D!UQJ3b zKF95ck3ZHjC^(rq-?c1j6`3{T``g!vUkJghb(}Bh>_~0nA@!~c(_P8o3U~Xs*+}A= zu0P((gL4oiBh zcd7gSSK{xd_Q=khBB{QADJJt!t19nYHrXMU*|hV+$Nlg(`M!j08cx_pw3xRaXMFa< z-PS;g`dHJ|D(4HM2Tr~GRRb#*t+@9v=_7~WLDiB{g^+h;?E#NNXregi@~1)+gZ9H8 zMx7|qf11bRe*&)i`gecPv)iy0IPjSbKa7eQiJdmYccm*(t6BpQc%>$kH!~g#}Wt-}e z!52la-%i&ERq$MW{&+6B_(}1r(%GZqqo_`^kwki{ji-*zxtN}h+Fnkxr~6&z_!%5W zLYFVy&AZU3esSOZrS|a-;p>b^xyf|tX_&+Jn{_jTvPAudgadl1sCAN_ns6NfdwbrQ+YdkMO{~hp>krELNh0_2hL>)Af^pq+2hy*#wNJe>55nrOFZhl)Hpg^`NK`MV zhO?EqjGrEQ(Ugz6bL7;t-P;5E;U6Jnf6vpUU&xlk`=l2|t6wunDdYK4ro*n6cm44= zQyF*u4`boeOG18jTQ|fXe|WDwR$A}(>~#aE^zsKkAIORQ@V}Xzx;er3VWNi(nsQd< z#QoEOwkKnrHKzt;*FQ;}IkrTbd(pcvjlk>Deo;T-v3agtnh&DTZe3rUW)yAq@xIc2 z_y#r4(mB3W+^`E zO~=;?pIrHY2)kja78b%(;?b6f> zz&yD-`trxt8=t3axdxc#%czY{XLQO^W9(u+ikDwQ7YCN^hd)}NUdvdn@GRw})#1{` zP@0R2QHgwD&S-SyuXmsE{KeYIH25!PDsFUI{aP|5fsZLZ({n8P@*+8@ORNbunAWl% zK0U{mvm3V)Br>nnJlhy*Qc36darC#n6B<^2Px^$M zlIpkBb3O9B(ZS=FBEiY8hd2M)+f{q)7n^$v@(v`9cx=+;ZvRpP8^ zOoKYl2)E50Owy>>S`QC_8~hZ$bG4~AW-O@E?SlW-J&oh3LUYBDJ-jwXcQ`g3;gx#o zN3YCEauDDVXQ-dXOp6}>Huy9lSr=7z$KS6F*EN6ZknR!5XhH?&@uJLDB7Kg(O? zy&sI0G9>iA5B)k7<%E%ZIX3-H{)CtF{bNQ-M%7U(jy8{O$8olVm^GwjyL?HaZ4Z22 zR>zh07zc}mWl;Apm^_#<$dR7IUz(d<@aXjO;5~OCrj#VdI{QMRG4Q&dfm@Kef}Rpd z{p$%?6aU-hm4z5ExcEw<@U%?r;GgmXZ0-iN#rIFN_w2&1G|ZJL|-By?%>-6Z@8wh{Czv<%xB!57L(~ zD=o}jVR?|(A0jiw_pQ;PB`R2zDR7)BXo=~I#V71Rp+uja0CGvO)qLy6a$xSUHuTy< zaj%I--@>G=-!l{KZ=MUP=b6&6eKsFlWx*j;eJLB{gU_*#+~~7%gFohJ&wVk+jeuT{=s3{`g?xTL9_o`${UNuGqauOj<}$Be(lwXGq?78G4yNH9qXRh}JBCi@rP2$k zxm5DRmZ4~`+!vg{0jVAyL7p#*IqIRrW3o?H-n}cgQ&jkx`w}5wF%j7tG(K@q%es+k zgsFm7!6Np_kC${9H}7|Gzb^BOJc+}ZrJr93MGQ+CItxB}iW=~Z_H>t{W4xMYuJNj= z$*MlxtRg7NFLZ+LVG=$-ZtA6^e^R8sN1HWoGr!C8^O^NCJ)SWS*A3kI?^qXV-k74! z3|&3o!{i_o*>$89gOTiJ+K}(&YPM?Jmcu>F!_qB;6VSYm8(=g$f*$dfwqLBg8Z~cdoS3;<2f^y*epA z(KzF`!C6sBa=G5ptv7RFzMRd++_{e(ushv+!%Y=wl2~OUX80gDiZe7tNXS?T3>)e8 zJ@F~qwdes;>84M?gx~{Zd5;RRct8+;)cxNbnIa-KM>f4Yjo!}8%1d+xm985IP#tZ9 zr%&bBxkU$CF?kDq#jX<05nqZXXU4t7U!+gG$ZGW<@}=!aRv7f)y7lJ0uP-KTfv7WN zu7!x(y<1zDFRyO6*_Fm=bUG$Kf4|u|0iCde-<50oVkBu1@;y?|_3nhfw2H>D6NkjS ze>=IieINP!rD6KwAP0D<#L+MV~E} zJcav}-=p^6=gVj+y7V2|gK>>|%u?nWwl%Vm#|4h5jR^L6puYrVscEQH^ZQn=u??q0*Y zhlWjyET#%B>Fa-H1O@b_625r`#s&L5vFo(8%8yZaUBO;>^NzdMd|qvvZ`Jam3^6Nx zN>WVAbxzYgL&PbgcC#nJ@Q~7XEhguu8D9zYJyw^fm^`vpEjkj?g5hDj^D>nm92Enq zU!N1|`MCZgGdR`Nn! zNsZe>%rLBbt@svCJ*~uncn^a!hmVr%pVpSjOt|$PueG%cJDP%0&S+VjY`{Q-Pa32h zP;)q7wdOXNbqQ2vsrvig;kwHr&zoqVTvC^x4jSaNQt_Au+#}yq&QW;dK6IMb!StC9 zN;h`P+}%@kb@C%dT5o1mbqwXOs+#Mua-}xrclFOl329j}MF+z-XGcUF=C0o0N|E>t zxk|e+|K*pCUa)cUr?}zIrf|(@-vgl!W{`RxzhB!;t+m8?x zwAI?*JUqS-7u&8V<0Bs(Ul`OoD)Ith?$4qTXySlad*m$D0ns{9J)nByJTtcZUT#!! zVOLS{le7f)fI>eiYZrCtu~%aeJ*U?0*&>19jTGM)U*6sujY4S_s)aIs-9P^gY(%n@2?Rl@2y zoEKC12Z5jGYU7H@EnhP9exT`4GZ;MK>M5suy)Ui# z_S`gj6VEyjaWJ*(y6NPDhcr4uMc!kKdB_8G@Mo7hgLMJ(1Wf2<9r=4yuY+#gj`#1J zODi0YDRdUTH>VmBr;RT-^cw4{{-$z9T=yu;bd&mzl1qS-mSfG)uMeG#pLB@FJv%T< zk0IbpRhA|Bqc@!WPpCPZf7|`7a@sX$Q$?bK4gpncwW=w5dW~RpWhknw*|;O@T3Gwd z5zRKM>zC4Am+Gb>x0q*+`MSujvxeXjsVXiK18?(4w?*gdBoB5Bq0CGT^441Z9a5FAI+V(FS!|AMZ^lrtOfwV61rD0u1l8+ zkR!5Oq4uqNKkj>q|CtgCcB;Q* znJOesU_)ccL0S7Zhfsbo zZr58wRp5pvv~f_3Z0=Bqz&Rzco56}n`A&1KXZr@93Vq;V^Er`*Op^hQhU!_qkK^Gy zBNO2*8)7ocaRYxQqgSuHh=^bH)E4Xvr~TRe?uTSh-NzB}&0gi#jXX7{=0`ud9n5erE$3D0 zysc?&I8=CJ%-iQ@YYtOaWso#k*6TXtb)%@OesMGGl6P?#Cr70gL=UV)db~J(4C9u+ zqEz4cx_*6F2sssyd-&N(KuNoe40*PDA$p+V2d^eFoe$yp)+`D-vnA{pD026?(H4HR z-UR+}wEPo-(M}}geB?1}m+!(M%tEiPgs-NooH3UFrMGl}I&FXhbNRsY_6GVhhdt0` z>+!|Me^n29Os>xtmtVk--hYN-sGrP^4a;lIxTbyc%@P zdt5YBxV~IIq%m&C^WY(J@)etwRP-G88?BofHmDZ5mxVk6e#Hlo3a<7Xc3h^5j!s-2 z5o(h~SrSGoHzKD9!%!>jNT1~tp@(ZaRi2ONB2xUOMmLa#S?o z+z%(YasJzq4Y<6HmK;NnuuTFz{WYPbr3(qhCypp)$Yq|@T}pYRam{|D;&I7*ykznn zSvTnfg7O!ayIdXQP}Z;|G~Z!gsrzb~6IE|tHknsP9@f>edQ-4&*3WeJinZo-`}5{M zB~2wwP7FR0T=NxaGkq-d^)r23;{}GHnU^OM+sj;?X0#u@zfdjD&V4gzNo_7Y{j-LT zIY*?0^vqggyx>Rc1?twKuY@+{$GQe|bEq-t`eOa4uQu=SY}Y%}Yd_cC63&as5BU0o zZ92^3cz^r%Wurw6&?iCWa|stn{mY{Pv-7-SuokD>ipsP#!&9{ip#h-u^OI6vT zA5f2=1o-67zqs*PIJ39jY%Gqm9B%}5*JPocjk^AMPN8JrcU;iV zpbzIktEO)8=hk(v>N{O$_lcVq@!)yeJrYZn$NeTYDRH?^Ci;R|vV5ERjA&my384Gg zC|6J=zE#P8-L#7-qteu*8A|g==F^8p1SkUf6lxU8G3PEHWkM?Db2GQ1se5+8>_^|J zUviN-x3mqiJagto*?;~TM#;VU8fKxYccW4Jja!uKz2#ASOjtK3Dp@L{AYf5$x_IU2 zyyN>}AL5Nz?8BSqFz8yPY2`ajs4G)4jR!T~PVtrNY?ecZjF#PJ4-WcM2h1^!Z zgV!|gT0UW9;Qss-({I3mYY?H{h-8_-j`ReJ-x0Dmov9xLsw)mc&!5Z;KNun1aJAw+ z3YVOCzBT1!PC5y6JSHhDJ_EL#;XT^mmk&oHXcR#_UG0&`;*f5|9E-Hb^>nGt`7kC| z)$E9>+ldWcrxSk~YM!}%lOgP+eiWTH)?HL=NbFpAZ>RH-EvJfxdrKGUaw}_YHtH*2 zZVl>+^Aak(Rkqq=!p_xi$)1s4{_I<;SUnWrEmZdNQaJDhJD~XMx2~t>uSD+y-bK9) zue;U{u^d}YQrA?wAf7SyFug$Se7x<3>YWY0k43+8FSXZ0smoxv72c!Iewf)AC(r5U zD)ptd`*`cqD6W{Eyi(k(=10S=zqrk<2;JgQ#4duQWLG3b!b zBV|ed)D>@wk2Ok50Y3~fQqq(d!l$(Bs=4KZBb=>Qs(i5XI5YJmP2TpGCvj;7(If9A ztDY=*M=6~)>=I}H$>e-nFA;ej4syy!=IsiZ4SCaUy)J#fW#A|MxYoz#Bfp*)9?3# zvL+hIaIDASrF6BP&yMrh&nNEQI#H`QZ+b^q>vPjvr;BA`wx?}GtGwU5J$ywf;N^p- zZ!k!{83ult2%bTvv!R7!%1^Z?!cUs?N3_HY6>JTjb{V5PfsBoHKJ!AI!{RrM1_NX3~e?`g8B z&NEoeE!T|F+`m$0$CIvp^~{x8#<~*ckDW`9IMh=D>*T3PMxM5yfTV1BxQ*gn;?G(- z_MFZn(XfguT{@20;mp?`NjiC-IhUYZfo-Y$MU<9!a4Z~igW$Syb5v6}7u!f*W-dgu$54i)UJ{R!?7f#r);el{1UbJU8kaik>A^!X|AGLCUkTg?jOD z5BU*Af2S>1E3qf9qY>w78fJ$y&9l`$onA9S-dAtmI6ZFCNaV7CJC^vrI%+_hWNY*_ zF%J4O%*gwbKu`CS1J7~TyV%Dfr0dLgCEX@1^&k@pSFb$!6z5+N<|N`Dvqa^^#!qTr zq=EX|cf69Qu;_3TBXXhFO@M;M2mcE+HKU-GifRkcBT#M$ACyNzFB@7i>`)!jqoU0E90CVuy$ zn~Dnu(NBL`1@d*mYTCk%h_xMj+S^3)Q1`_0i$4BshLE#$XQnTxinhiMM8e9lbuI_t+s~^r*3M zuvpZb?j32RDevhw!Py2;SAK7-qVtp&WCIIMI7cJ~)^L499DM(sPvmW1_w|AfV{)H; z+7X2`g~3I8^O^-8PL{}rO3vsZ&Ki*3y>avKEv0g3#^lRO#Pgca>RX0zcAf>jzAq1t z^kbygZ4IJ{p0PjH-sr94o_ku2B!?iQ_*0xJRKix^$y)ZWaum2nLXhE|&QF9tnGTsS zTs!n5z}5!dA;GKIZ6DHdBxP}9YZ689@-r_@V15j_*g)Em1rr+@eoJ|z4h;-XCw~eB%bW?k?uQ96Tyz{8e^`?l_49zqWwD1}+gJHMS}1(? zGcP;PAjn3#fqr;8I>uS22Er9ZR(Ac2*Sj8;-9;lT$^ZSw5uO!6Z-i`e(N<#bfm=Z6 z;d`EMD`F1TK8>DKcpm^e+&^~cLu?ou`zn1*Nl!+Z=TtZE>k~!(9H~P8gOqzn5ClO0 zAXv6-+qP}nwr$(CZQJOwZQHhX{;8$I6-F zG~w~7cSRq>BKbqm>P+L3YYEz$akg+sTVrk;^27aBv!*hm-F@`#!}J~?C&!LR{2u<| z8M;EPjyu8~-hxJY>F^%H-*scb_=KG#r~t+Es@<(~tUPv|cWrBD71TcNLyO_scI4}ODV}}0#Mq-%9A?vHh<`-)XuF8QRTfDRW z@#X+mV~ROCPSXyO`!{5 zE##+8zbZ7P7A-_bK(?Sc4t$A2_~163s}5J`Flhume!cXKawDM&|LPU7_-m6mA4gHv zf*2W;QUBQw|2Xiq$duZ{g|+;Ixl{c`tM!2Q4{h|=h*R04lS9Yhh%)Rr;?yORrtn|x z&?XU&C4@+BXD>VaP$oqi{pl;ote*(d3M=c8kNRJP_4e1JD=21fvYAs6t912AgOmAO z)$I$QK{Reasu+AqGdD?2_3ClZ^{`5L&Upf}lXU z6=ms*i$vh1owg82ESE+J@9F@5#wGG)RJdzAa}%sB-0#>V?VM$O-RWe;Viz{vmDGv4 z1OUiuK{^Hoi-w!&0uQ2uw23b!O(5J&3)qguFD975GQp=!vBb3ki#u%2Lg+d!Y4=#; z(vH!!66lCt@FyHZ)&R`0Ugm+m;fM%s?~h8o`O(mxIt$(_Ois+)sE*+yxafZp;0&~b zV&&jIJz%a+>b7=1!V=$Dm6vnjsB7A!N7MY z09|e4xQW)t>*tlqUBy3)M+|1V&?1HGmUYgKOcK zwpDY~J9DAjdJOlL*uamsV%+%$Fw2nRi|j^92sdn7#LMWJNlM*xx3y~$F*Jz*x5v7i zu%!N!L-vde4?#2a$+%DVLWN-=zYwsYGonH-AIZQ1qOmJrg_k3<;m zGh%LV9EeQ%u1tn;R+-%ub^Td}$tq?$MbuU#REtz=_eTTD34}s$P0FDRzycWquQ^bJ z1*CVuO+Ry;f%vA^QCkt3=GG~8%nY!5fAew@MH`l$@o|dwq;-Xsz};3b>reDU5F}LI zoZq?^_!?3yds#!oMr4eSaBDXv36252iJxh9(ijQ?Y!`1AA-77-pkLSCh``cv)%>3+ z$I_hE^pj3d$tTIkwjS>^&yYJ&=D*95cc0_K#5a`>EC_}@Z4FWASofw)R>K=Lp(k(^ z+UnoPu&aE9FB}3xpkN%S=YG;-!T3^@Sd(Lk)NR~H;3ch8VZ~%++(ujCMN)P zq}_wz&wq78MVv!{(2$KQEu|9cNLmZm0FIXH|Awwg#Xf!7c_)+w9lL@~Q2keht`B(MB+k zZEWuQPt&{{=ZoMFK)hsUOAn#5`MQ6w zV4F}m1vDJUs=#K%)3?hlC)|lp*Ht~jQ_Fg&nj;iBit-|JabVn{i8k0bF?6yKyYI$E zwq7yEv0?qS6HdXUIb0>eW-7R=phw2`brXiZ1ZEqQFp0gK!9pC!c7^ARijXP?-L$W! zcQ|rdpuWjMmF!*e$!Jy;xXP3W!$dPxb3hF8eCT_Zp8>-w(2=k9 z^F^|54*7f8c<+!VJ@JHgOa1t4hL7^z*#2f(GXjq(h&o2iKc4ITw!Hq$=N_Pb`qBXt zXIqL|0oCD0!&nH&_QR@5c(G{t!rt+4D=VD_3U^auc*}19yad4YDTN1rW#{BYkl4jp zO-`bvqq$COU@3E~OWK4T)G9i19->cx|6?~8v;(C57`_CUmjMKivCv|%3(QhB=ZN9EMx%ZD?WS+!gEWF43lhp0AHyJ;vLZ;CO$UGo8N2=ON z>8*aldJ>2?cKgPyctOS=kiV+AeF)SQg16TLG9YCecR+SsTQ!%C1U(1NXoDaF3?>8p zUL2T1N@5V;MsKf#@{4c&`?SFjCM1iv-I9*f=hK@)WZb>P+N+PdYGHC`gLEl~xr zpqpWZxmx=${hb`=`gzq^7Ps0uZZ!T%vat z;8H;wgCsn(;o1FV`*e2bq6cMEYgI-NEQpz9|Wp$;v@W!Sf=YB}Q z@;19}p@U2Lya-cvLRKefxJZ;8kEKT~#F@0WFSANI!^TEb#^xNP$kbm$)%&kIp z7Zj3OU}A?%ssWkp$o@)9hf>&F@klSCW#~ z=7GaLn&SF$l;?i*Z3+mVrz_c;>kqf@B}YL}bXS6pX;dS=+hrme7w7Pgit#jIFAJ~y zi&R}cU)qAVPF4Q`wIv0war#X>&@?3$JCw*7so=SnnQ`HMe>23?5?-;`)+V3Faj+a1 zi^KJp!~a5?g*hmcs04d*)As;+Y=q|>J|+=51tl8Q(wa``#GPvs!)B&?7-GUw?T2JT zA`ct?ty#K?O>UIkH&*<1Z%baQ;4K-jXWX@G4{b&*1g|GVap6-8BEA8iA)hYfquM8` zKDb)1f0?n^ipn3bYtCu$sGvHy2TG0H)tU<~oVg?AMIqn1sBV;W{`IF7*OE5?g_Zdf z(zP9UP_Wn#I1-&r$nXcoU>k(ZhLJh+DwPO`h_YU!=D3&F^3D%O_d6=GP+ook)FM)2 zl8}eMfDix_+WAg^0Uy@>ox})pStKSd7hIM-wYtQW_(8&VS`Wf=UF#K_QTAS~38<_W zr=Q@^lE&~u2>sUCdR`u4JRZPpAprRy{|-PP{csLR8k<1k#D$(R2kDmY+~s^i4)OOu z@Oval2Q)i!Zk&_L#7kUmf+x-6H}5{k_?>wv{%6VFI45PBiW)i8yU|Xux){ulntg21 zexT2}%r>%1RsQW5z9ld2{a32H!3_6wyw8;unt}4JrKA@~tI8T^x1uXrK*Yb`r%&bY ziOQ=FH#)o5Nvd7Ga#Ha+JuTc=+AFzpC7kSA5Eg4^1eL zXx^KH>aXovm7P<;_|Eo5?FTu-20__3$Yf4=&mt2;9z%vEW-r=vJTLF$c&?P&vN@Hj-GN z!y=}lqKe&FP=UPXIiwk`)?_JiJfEJ>w?*(}WcHP1>%qgpa5f>30uC zIm@~2gia?uBQR~Jw5mnNhD*>%2?zNTc!Mo9!D?AK&elL;>E5BKr&zhb#5G3oZZZt3 zT_p%&{;n0OC8n!cHw9D-EXpXP=mTexkS&ebH#u2cT4n08Xk|g*3E^gax z)CphiR)kk?y9(ws9ydF^?A;E4EY>3(Cn~{U;MSf@`vbyGZsMG1h&;s5R0~Culi84$ z-B{q$VmN!IKq-p$2zwzYVWcpf+&lc0bZhD*JJ2gw5(sj4eL47%$LdtTT#@r zpcFoYHbld;&!v;0Z2k$|BxcXS_E^Z3fHlB}c>Ma<|5wo|t0qftiY4*HQ5WQ%1+p7a z4tHv+b>l=i@q0H3v@$*RV@b@DJelIxnMfR*q?hR+tga0YM)B&_TtV}G1j5}t4=5dJ zX8aICkKvQ%TW6127p%ibE&1N9VLX4r1bKA5SEbSdUbiDF7mT0UdKC;9V zICwJTPn?*qnArEy{p+=K#*xb0m$Zg{NyN(Uy${0Dix)T2%=JB~r63njfnKGvuV9SRt95ZYjZ`#EtB_S1*4l++!tj*4W17QGOVN^Hs+Ix z#)YltTv?es;ill$PbfYTz+0TpFXfr+NHClsDr7xMj7n;agU7+9$h$DR5|mHxxUy5q z%>HxmdR34ki&jdU?1;;>mJ=ylxLc;2>a7PQcwm_T&Xhg8ei;st&GemDwUnpz%zk~5 zV`ZfpwjsvCB4i1xxHoXF%^dN5`e@GQ6ko<>UtE69vpbp+3*S!_MQfQsKArpHUwS<> zEv^*0azFkE12=#dX_LimB4onAk6Iyp+fg1Z;n)E0>Pn_>FzP zuuJ`46O7j_=_M?JWyS$^@!xe(A_#rYy8P!An3CtQ#3uNyq#ao2!3|5gaLYAG6$Clg z@GT=Z+(eQnH?(pC1dMOPg($1niUp-}JYsYc(_U(H{KjhT-ZNX+ck!d~2>V_8;hc?Z zaaRo-1xs+2dASphu3$S)mZ$r4uP_qH*@LZPbjc2b21Wf|&#VR>0RUz8bz$_s_kI4Y ztx9{<^%ReretJP-OG2{G#3b*ot$l7amZp4|fsP6rAP%?QN-KezLS6>bf-sb!e{`o~ zc$9U9e9W19Ngz~4`*B{~xiIVgTXldoJxg@k*Die!NLs_J_~!XySW^NkW3M4|ayK>$ z!tju^IbVjMExE+@p1hWRDX_in6d!Emw+fgvlp8JJ&4}A-!mj9>ap6{8Ro#1~Q4QC! zlv55k3O?}m(J@6|w$rkq9Yt3aLSgM}S?BNidtBkKaX2vG$BQ8*j>Ajcl!Ah=@r)i{ z(A*nVImLuzZ|E15LoT`L6d+%Q;%b=R%Ot0LQmG!YKVfwp`)LxL$k8|xFg508Oz%Cy zNT3+`?0|p;?eDc;aFDz^wl^zgbb9?)!IFwX;;}7?Jl~+wV^_=dx5YI_M>J6|&o9 z>0$uWs?72Y;!B=Yzjo!Iblms_tHp)M&1ZyLc~g%WCSv}KedY8+_824NY%&s zL`(|$j0?Tm9@|LXnzS#-tH(JOFOCR!(Cb!r{EHpmk} zxQ~pf3zkuqFh$oX&Jc#jyFz-JgCnQlF(iyR42ec)py^~4FqinsPussaz*VjMNk z?&DYHD!^NpP_|$`);FO?;>3H^t3FN?4$}uIRmY)^-*4MwRlvR%upY;0oO8(njVEx% zJ|z9YL7KjIaRi{_Bl)^nAdzEI^bGTd3 ztSyU%3Pzy)?+oz=myzsH^3H;}RKO2%P$&CsHk_{^_)R6XYbpOA?NT<+dY!P39!mml zxvxI1j$<^@A$qG80{otopMP61JZCgCmu6O0P#D>9TMGkN$<&iS6J_j*4b}+mWR+uz zwL@O$p^hUQ{Lwvndj+~r2r~UcE6Ty-b?+@M6!@l($Y?E2e%mc@ecs3F?BBqYsueLo z7(fvPvC3PQ2bHchvgTajGvg(mR_24c@KX_=t5YKk`&@6yQL3YCmkstWhh$s*o%as1 zB-zqPu{rth0~J7jsfOuDdM6S??i7}_Bb)Iu%)kBERun+FC@}qQbk2CHt=;L^h(}29 zA&wHx-R`D;gW3!`lWe=9Ul8M>|G5%V%S_Y6f7-ksDtYqO)r@7FIj+k8tgeRN!4+-}`B^X*rCkjGix3&P zo@PHphb%C;J?lFQT)fYNJ}^QKAlvbAVetQMTJ26Vf+L;GS+NB^CHbNmR1yu5#5_>C zzOQ~5p)@K352OzCna!Z0gMN0{C(!E*DTWL%>{-Wy z?6+P*5fn?>!CgXZ7w(yPNBu&y=ovc41GOK3asZiF3?HSBq_k_G10g;scFBK)P<2Vj z)n9Ij7;Wt$T@Pg8IFh>$7gcL513|6eoo~1wTvY~l3^s^lE>Iks=L&nDa^frkN=w0 zF1E&vr`-h;<}$qi2jKx_w|c!(2(xLrp&LGG5ZTbhV|yGZ?OlCqAo^smiUe28tDZN& zMCOJxBiqi4cOD{SH_hg&E_Ii@Odmi-%_luX2*9`fz-EEE_CH|@kDuA<-Egk1CF1$( zlZd1a>jdyB<^O4HQyEuB#1&vS#LtuYzAP6kQG6SbnYd#dj3PW_`LZDuQ;J!+;L%2{V| zG&q}Hv*RR!5gTmQ90~-1HOR2*z;bOim10bG7`_187xKT9XbNO9!x+@;(HWD$s;xuO z0RpLc?MjR_$0hR5#|-l-T|1&oThK$L!|f^U)JCztx4-|f-PtA-7m_ROibe_=sq0_l z@7iqq1maB)>;{Nz`W@EIDz4ws9VcFhhSQ(dNiy8BT2S3 zB=2mILwZ}FvaoPvR}x$+8k%&$`e0D}`EMm(??fM)k;MD~#H~#uemtU(0)vqr$m17Y zsTJ}7H9*S0lL*cbMU5T@H$BIml<<3SMr&pac|c~6mLArzD(#R1!L^1M4Z;RQRfnC&~Q zG|Tqk%@OFyXjr*al(RZ)cNWU|k!8oIbk3o2X+9y%OS2N%>DeX6$5>4>?rjSWwSvgL zM}>XqvDiap8c!K@b{fjv*+Y;ykwV6^y^g-%Aa5BUA=kXMd~&m12~eKjvIj(rg~~nl z4*vzB%2bRDzf8IQX$#oP74natvrg`(0>R_pcECGxJ&*hBe#2>2GxOMIpF@)X*HXR^ zk8hRsW&CEf2koBFr>RlIt34%-7;Hgl!e~@bNTEoL;mhz7G-r^QacjA8+Q7~oB|=Z~ z)XZ>g6)AA1PC?MeQ((;r&(Ub0kW^gU8uUb8#|jl>IRWL5sjR#(nO?8ptm_{H&2cd} zl^KKPI9AkKvFDHJ_iQgl%`CyVPUUa$GsHDuh|q)np%9qT^38=Pyg{zWp}gPs`ekT8 zEi5lO_F1tOm?G2_RD0409GFtE&nRy)zgzje@csqIO zsJF7g?xitX8T$P-l8rZC6)G=7p@gPr8$-hgZ*KDRCyj2ValwTGiCZz%pqAd_OdONO zR7=Y5ruqz;9zxb)Hw1EcLzt)>lT40$9`I=JEtjR&+F>;N9Ee?Sy9g%@+wiv!sn8D0 zG3RX9Jc@WE21W9_FQo2ZMvy-{LX=UdL$cu9znVV5#MQq%^hZ#Z`YXigon5Lo%jv#F zh5Z|!-?$tb7yZ*=T5GrtoKC|Ouf8F{Fnm(g{ktu2#}pK0>>K4Ea2oj+s*u_u{n5{= zOdC{iI9|3c70IDeTq{V-rz01W4HEfX`OGqhp0CAv)rat#Xak4G$e)-|ox}mKI8=EN z!#B9?O?t~c5)qkBi_L}7HIsGjlN8Xzt29Yd+z`yZtPqtSp<}<0F3}9ar*QuG0^wn| z72^_@gMN?GX=Q**D4`@5z7BvCdbzSJ@c|X`af-L>CUA+~D$YX&H>9U$!SEwd;#t55 z_Tgc#e^;@3aBM7rHkVfJ!pM92Z_V3O-lXpFE?I)IiMt68D$)j0E zrOrqGH5%!{TZ|?Qe_)z{ckDuM{CSM8!>aGO4F7|NPY#F2tDw0L07mrgJ)BPl7NB|P zkU_g2;x`4l088nPB&mvdkV)6#gsB^Ck3;0;RKs$=(qi;#Wah12OHv;>nq9Ok<$ccY z5E6(O z%*~D&Fs<(6$nB;@<%bYso4Eyo0o!uhngxyR47d)~;o(Ql-m zzBCyuc%-{y`RG_Q&W>E%V<9?zo*0M))y7Yx5q;SXmrxZ*aTh@E9y@{dmC%r zNpIqo_OuDcACjH~O-ofq^%gQ8Vj&5_e}O2LyURr@HozYJOdJcY zapYO_6;NJ>eBTO4@MzPn9bKJBagWl4e!7Hdlw1~+xP&>SSeNo}pd(Wb-k%W~Ki2aw zMf9lQXp1oma71Vw!mUe=NXt<6#hRT>T)?Jts4uVqx~JRRw+_qXhu=WmAbUwrY-=(s z2v!`49Hjf1YRelp6EBf&nIt^eHj92|&q(dYydl`}ZT^lPjmYX3+)XI4m`|^oB%NGd zBL+8SU5Cx+d2-xvkAwCGq|wJX8Y_Fd)oZ_<4G9`u57R`=I{nDI>ZGM0`v&p+FHzby zypuMYD9<_M07q)8Iyl``K@vmye7CI!K6KkW&%lnD)OdZAEqQ}+9Yl8->65$eGFI$R zOnmm8^ce5yg~W9bB%poRs5aiVu!iMBDbY%65Oxp=?nYOS_@TyEeDfNNXBZ6NvWY3Y z&kcDA28Pjte-4%mrA%2=i3gRoY4TmIoRPpQ$bA9f7MS2J1+z(We*xnt%Gi?IlnNoi zl3wF##X;)*1@etEot!P%Gn&V&ME8oFhd}n>=ZE%dt8lw=ohdth+5LV_pn63JdNH^e zV?aN|BmWMe;^RKIt5=!5Uf|;g7x{Y|x-+O9>pl_G2_u5)!3yz_cpOk<5cVkmyY%Rb z_RBqyB`7dpXA^)0spdbax@_hp9UxyI`BEb!B1wW*gBf79fPC*=2me zTA=ZK2lUTLM3=t#@O$Qwlul)6k5X@EpwGN1gZlMbRiyTjcC1ub7CK$IT%zyP;j;gU zK*Q3omMZ?0J#S|b;W@bG#JkJ)%i)^unoYFL+8F0hCB0YwOx~~Kn4^)Z7#R9)-Wgq#{UCUPEA;eD z3`;%g%!2N@cX+|Jk}eedvBn_8_@Pr0tnj8gKO8!r?a>k=cQdT+9$wVmO-*x6QR#dB zoO)nAMDiX2VX-WA+D{S(i>pATm-0!$W^sffqgrusUeO0)fQf7i2j-NgCL6TKM^$_y zWoG85F*joc8(Xq*y%+*;6KgZGtY8%u~-RaZdO8-xn*ET*j&a6*4BzVm%`;(d$Jw{a1AY`l|xCoQ3t7>K?%qqZGpBHG{rGwp7U z-t`zx)(-uD4534UQO*4oV1`@UFRW!B=Y%&I$CK=i4x9k_B=Rg^;Ucn%hy%?s1&J*E?h;C0}>OUh`pg{A1bXBiAuP8aXKC~M5Dv-N*y zhphtF5A>Ps3avlN7lq;6hSmsbw4PnD9pMLG358JP2Xvc(FLc|7&p@eE?{~)xrht*R ztMNa)ol}4DA!1=0OK5vuCAcJia0ZTE%TNU>>DU?E!w!39N%3|VEPHj~+f_`0i&6*Y zG_YNEDVX0=;!*Watwv^7zv3+H-B5I_sNjoq_*0-Ued% zlp*R^qJj~FaTy_|{-L7aA;pC z=gH;ajdQZXa(Z{d^sSRuOQrQv?*YD>+zp#z#qjMc*f}mFI;kyzsp#9}NT%{1R%ml> z{y5IsL9W0tUCp8Gm`zewdiVKieT`Z#L5S_o)7okk-X2*Mr^jCvH@M>M2IEYlWW4YX zG2=9hOsKI5R@s-}+EglhToq@t{oyE~n22ekwF?R&6)X8rLid0*}?U;+yH#(++}r+0=lcC%B>Xj7Eo z>y}7fZ!1o*q#c@Y?^-?Bb&!>(z(5>*1rs&$YINP8CTorFP$yhVr-e074}t>*leW!W zK6?;De5grs?MZvPucp^#WMQOSu!phd9557T3h7zCWR@Z7P2_{!4CP@va_|Ho>&qXL!L{(%kuL z%;!xl--OuP4-#IBm|l^#*FgiAz>r!=D!#lW2{19|AxN32rjL0v4#!{}uj{S}hp$kH z&3)WYqcPU21dxF%YoPqRK-?Gbf1o3FYDZl-74?omoZ_@fqK)d+4URw)W< zABS35QPoWvZ>GH_GW!#1i?$X#`d8;0ZT!VWiF{N~Ybu zi8a-@Gl7?_zcZ~EgK_qy5dL-W1>tpq>8YC_qut;vy-Ol9ZB0=RIRm2wFocBf&=G`h z+;%e^If?w$O!^D5?2fh|>&epgJeO~bH^*anS#Iqt^7Za}0yG1);4M7V+{{qQ@&qi4 zgT+*$wCxbQhAHx)`txu4Gh$%ReegVC!gdk&ZIufQgnhTZuR~jgqxFhty~b(KcZSr) z2OF)L6htFOK_!q6%Jc^V5>9HPupm$;Mm$(cPpG;^Ifi^J`lwM;P9)!)*xWk$#RF(( z0rxwMXj<%U{Zz@Br+xhgZ!E*@FoflWz#iJFK5o6Pk<_vUc_*r-yxA z?$)*@RX*U*`<}N;kNi}#_H6Txb7GW*f5IiFN_pzi{E`h0(9PZIzw4wdW_PQGk4i|X z(FjLW>&bA9#7Gf~%prukAiEpNB25js`YY2TjwiG_^IE;||Ipq7eWyaP=!`QqUh2ZA z*eYXHYw7PmIRohTZg#N)AX`8P>^c7(@K*f?=!m1M@H3Kx${h+cN`FSpL z<4S67jLSkzz_IwHesr~Q8B&QQ3vEzfC#)oruQQD8>QKxH^_#|FW$e!eE-y2+;s)2w zpOPOP-B!_sz}kTCHJ=CNAvQFWuVRae0?IQP=x)HspE`uBvaxPSqnCOVwCKJ43DBI6 zaS^WI{ToQWN&BqBj=lw=2{o6t{zh~vadf5Sb5an%1NXdyh)^~H+8#@ojL|r}`>T-0 zupx)5V=Idw^{_33Y^kbEW&=8w3z7&Dx_=IJ*fsG&iHzJbv%PWNJf7BaD)+_lt{-zh zeJ*&FrS(0ku>Wb&j_wjbAPTTHtPFX6w#N}f#!e`m+fI$@CwNIacjktJZ=yLp2Yso@ zfJfeYt?cyUD`OO9I4Ms`r%In=7bZI+_dbOn7W)e6p61*F=s9GJC|&pzogKN)mh=O) zIXZQn*dUa?xS(ZUgWGNQhEu7Gdm3W<1tWiWc7UJO z>EpRCg`;^DnFbvz@Qoukt4#b>y2cyg!!nA;(@pQstDBAlcD;Zb)$_qve2_vzs0c%v z>3rb)K6;n-lPY~?`05;j42&4mX942$8?w5fpmw3>+v`o*$v3ck4DI4gWswzH9GN7S z?7$pvFd>8P67N~gtr`??CIb}tE|?K>T0(_ROy=8K`P_V~-}lg%wcd}6j|`K@cWtl7 zGLrsTQbV{-of`ngIN{wk6Z-&*_(4KJ9iWPLcf9k9wQqoHvA-Y&TzM3Qo|rhOT-)=Jyz)lPGPk?~*%8V0AD?#_y`W){8@`7Cxhu9C1#16IckSr)oed>Xq2 zH_(B6=p`{hS1HdG({p>}{&Up6@^11!1YKakfNO(Qw~&uWhghu{`dZzWfh1OZ(GM(S z4{**wytD3Zbl&U&d%0!!_qr{GaChYWdkjFK1_niwwR6P>k`z81$Q25a*xEd6S4-qe zIm0=C+>yUg+6!uTFcz04jR5Q8$wGmEs_1i8gg6~we4D1MHJK(9UOcNBCIjmGDZ8R% zk_T~(gw&fJKsuh=g3^lM`&rNnmSAW) zikZvnnQP8966x%#gwu4+pJx~aXLy#Hsc$UXv(1nARm|+-IS(xj9(!3 zo_%U;01iv|yCa1jg=zQ*CA8a%3ih0t=H|%;iLGN8!#|{fG0ObK$2**$ErYWT5O^i68Gc|yUC6o2=@g^@1ee`V1t4ioT1P1@ycN#?v=6m4>%~^lbd*Wqro&--rh8Z{?j*C)h$I z@*kYGyABUSr}}=E5D7^yO#8&L#`8s1$4s#VO1q`d8XerL*sWl)3z&c?0>3)QuuATq z%0VNLD_5+Q@H`b^AeJy~mDKGGqJ2blGsjFjoD^_b-mow z<)h)fJ8IeYML=~=IP_69t(?fPf4RY9maUJxB^SRY=ba65VBPM5lkf7z`6NF`6j~@* z7WVHoeeGO&`b&W1*Y(+nRi;9wGN2Sfzk0wR!Hp$jyPRpK4dE80bxORouN}|GQCR8I z$w^<}vdJ>Kd8qu9JvESx7Ya0#3o4GMrdJSrI_(URFL(0EIFc!c^c~RMM)i?i>^1B8 z$DwHnD6MOl{*oZxS(DZL3tm`IH>}vJcH?}6cm~!FZZNZ*--(+CWR8LIvUJJ)l7DvwjLpDicje zrlsFtz320RjvJ_#V%?+YOR^j!ZAzR3mx3x^L%wpiux~xX=M-jkfYj_nu25~}s|DbP zax75TuAp<*yU~aeSp#rlv%KqaywY}d-;0^HjAgx$O2SRDRHN z$uF3-CR2>KcW!v{d05^#2DELtIWY$cWu9lYwb{ZV4@~=J3Auz;*G=agOl`Ox^o2jm z@UVZTi=B$ z@hbYr3!|6u)-u#?Jb`dI{C;Jf&j(gOSV<`dCeGd;hq%J{eH#UEc8%uf^zmV>&c@E1cXQaQlFxt)0cv z=!FLiymeRUMOt@@&#rcgEdPALm5V%y80UaeY)jdpU|An;qN<~_V7-Ce9*2?ZbVfyd zQcXr9u+D}e1zI9Z$QpEtPa?MH#=^oLpk;LVR)WfmvUx8Vm#Lti^$fD-Mbk!shPoWj z%&@Jv?okW>$;P8{snPt#lq1O&G^4(85h5>3K>5dST}uzx2Cx#mgoo;8N-bSGa@pp4 zGQ`NfR;0$7Q&C~V!XiHU=EjWhcg4@G1m5mi?D@b998s54>2WU$y^#@yW|!`nflGmk z{#YZi>A~j4N6B-neZ3b>e;cp71h_F$R0ce%Dojeud9;!@1w!wml%`N(Y*ZmAPxQ7M zdGBxRgbipv+Qa9&-|CtEO)o$`_}*!I>Whm`1HFZPQdIGS;{(BiLY^j#R>b>TP@Z1U zT(aF~2|D&>ZBiIaAK;-ozbFo2{VFLxr&sdZ5&ymG=em?mv@xy^9V#5*(uMl~AH(2n z2pEk_y|`GqH|%~;u;ZkKcPaQ{kaB5}rM#~S^D>NPx(CjsspT0uf=7Us^ z@iqQ7e9Rofe8lIXJ6^2Rr+sFD$-cf8Ara*2_0Izg%tW)|Gw`3ji|0aR=2?JqEep%j z=UL!`Kqe)iQK`s+mlx!$Bp21028>3D{u<8503-ll_cJ53*+va$q`FYD%&!b;ez&^p zqaK>SULd`l0sw{e!YKVjkQVW^)KCJ^jo<&t94zEG=Z$I_Nn3Ht)s6*IECcj!WPLxxu>tK zI~81!xhLJOY?a0PjJLM0r4P-&95nchxF_9AjB-ds!tVzzpX;cgGlI4%J zfWHwpVhciU!s!M79sI-qvD_ylxHvfBg=PICGrYY7gOxup<(3CR|IQvge<)TAG`1+e zs2Cm|Uj}ZZaRp$pZpKJL7V80$Qc!fea`hkmyrP;{iw;?(u7n>x1_^-Eh## zMhoV@{1=))5tIPdO`P)CS(^Tf)b~q|k=W@dX|Ne87J+W;Pv4UhN2r|W(3AF)O%{Z{ z6&ReVaFR~y+0Rm^tjM8nVxZG45dd*ip9W{ICKBO}3S`w^4##L2W6|2YyZ8x`Dl~yE zYa5Ava!;_Yuiyq*T>$MoLZ=6bY~iowSj}jRKmaD5PrWGUUODHq=}a6|t9Qa|p;hAj zyfR;bnhOM+6OY^V)T#PC{lF4mS$NVND^xOV-b==){}Ucarhy>AmCgoGVA$JXY?fUN zQDI7Zw<~0eX>FV$4`L}Pqo%tfevA0(&JeyR$wlv-(gepK>Wi9^f|7$~XG%p~=~c4i zwgWIw!begjFBeus7ZSN8ZsQo#o-b~=wzh|~{o1szMw7*DuKT`571k$)(i8;0v)%wa zItBdj!2%Jt(TyRe+4RyPz*W_|=q)HwA(*t^GX!2@z; zuoHYdqsCyZ!pDNiKWTi!x0NLjOwwb%;Gj=40X z0+dZYpENP4Gyu;0)L3zsG*xa@I>rtgK#HM7q(aNW4#R<*bho%-Mq86KY77HB;h+Vq z^l&V*LuxPx=Ck+Vm?emh(tVtWLZodw+matVJu3W7Ypt7g+|3+T9`bn((R-!URraO4UVeM><~{wMRsTzG}@h}>}vTm?mP zt8Hd4VA?(BTuWMVj4usQ_8K^krke|pK=XckF3-S9RFzk?A_n|;l!#LUp1%ntVi=X? zfp%Diht&-3^ERk*@D-@;=h3|?g@b>zPuFYs4T|V-QKp6V(yQKYf`A3HSEn0Bnn)hJ1l`V8y?M=& zug%+2uj8ukcXN~oWRJJ3l5Z)gU0A++;Gm9i7G87drSSxD$;>Feg+ajj&F}+)c_w2H z#}^I3ni1<=C=RPzsRr!zuh_*z(=9L8Ce7@xT;kG@MS>4r!(EVJs=g9^h_L&o**uHh zztG;bVorsD3O4m`C86?SJK zn|tOQ`jJFC8r;QfzY%uqfCmAzs8 z>OiF-d5EL9SU}5WG<>jK_8wi z<-1>AE6x@tgFb>uT2Pe-WsGz2efvgnu>w<%7BM|ky^Nm7E7*yum%cbN@0$x`#Y>TP zakuZ18NjoZu^o*H4d2>`a`pxpUonr3FDw|C)SW)q&_mzMQIhtLFZ`*yLa+nbz2wMR zKj8z*JQp?r$~Oye;n1Z1tCA=SWL{QO5QAy@QsKI9o_!X~>LfI`9Bb+vwyQiuc=xC1;=blD-YM}bv^#IR12Cb6?aTJff!Az~k<;-5 zQJQfw?n?%&4)wxKTyC=>gIC5hUNVbAT;RKafYx;h(RPsIKG;VTP$^?UAr z6Co|VGpMMMwJkQUQlihaIrW$yJkI`+9Cz;E9j*yO9Yugbf(L za(vvwwVC^>rKzy4L^vG}ill<%XSUt9cyXGgBW~%1IYroAvH@VoR7#3`iTg-@(+$}S zsSXgPieC+b)fQ+hiO8i)QSZR-OY7#IEa!Q)jz?b&#M{~+%9~2|^nga(W_`SUx?N;I`}L@ z0^7dsoN-{Y_cD6^(b_k7Z1Aw4K~$+%paV=ko4C##X`RyCy3tR;3 z_SjybH!4w5*o@~nmLqFSe4G_im)Q53K-a*dI$--UaR3ZG)wnp4gr@f@(P(*+UBBlC z2ENX8+NxCmrJBo(e!9zf5a?e2^rg{p3}KUSpjxNlE}nd(U8hR`gvtR_=RHw5O$U+{ zF}qgxIQbzoqyQLp8J5U+ya!@aqB(XF7YYh-k0%I^8jb2kgtB3#w&TBnM;P~c(&in@ zQFmgtp2gOc|M;ig(^$8VZ5UMdwBNzDOAg!tC@X9ATwG07R3B30VZ{@F-SA+H`?SZH z|1&U_g!r6x7ti|t8I}sz{TucR3HW3FEJ8`4*%a#eZEs-1RI}v_nx zeIO=kn^ma2Jt4D!{3#0{Osv6&9T%VA!|R@^ffO*6d6D`Q;gaE!8wGz;we0DVkKcoI z3@OaAu|yWI-YQx}5vME+p@CLtOO_)xz|NVS6;JAMGVdiKQw2##?!}|NF|q^klyFc_ zrB_n3n3WwuEk3j=L1&z#W#|&A!yF}wnbKX>bPNb630--VRhT#ANE(VA@>uy{vW4!X^-lJ`H^y;>No+%v%nG#Vl!0r+r>Tl9H&CQBEP#=!+9% z@Qo=p!an-mG_i%Q>Bn`{vK!E`b6^d?V<=u>t+~WXUs63uDV|TynojB+E5nVr4W-Bd zA3ki6+63xwU9diYW!MT0OL1PyAesD=)Jv?R$CCH4L|ZkI?7Wk5r=nAZH_eA`;9dt7 zV%l?o|IU!o^_fgQuq&)I_oiwPMJX~zT+#$)1-0(4dEg9^MOhz%iD#e}ZfNCNO1J2~ zW?4p!B_5)4rZ|-x6oc}?9KD9*mJUzPE<1Sq<#d*)I?a0ef+P&VUO|8}P=FakBNdXf z6w`WwVek}d#FmJ#aJl;g+x70Wv9NS7Z&RumHDgNVIj|Ds`vbT(* z{x<@{(kvIg7cP31Fezmm1`OT_@P|b0#_g<10n6*LuRF9Hy-A2REht=zy@9V68t{I& z+8jFu5a+~t^C6Rg{6cnCY+76ardDu1b@W6ETcKWg4+2=JSc%ujH;=V-wxpZrw`1xW z!w(D$`?M|}|ED?TmF@c!mSLmyfS9XMag+l8w%*&^XsC=4#apG$AWT&t+Sr=b#1w9K z!pfUjEn>kU%gj)_sc2KS-K4ZZNgGk|Pb`R~`);%u|KZ?8gT!M)12x3oDIPs8R7LH2 zpEQxnm+>Hk%8vkAD?8Zssw3!L7ksuduQeg`+*BS`)+Yx;p$p=%bcbn}xa3i>aqEOT%H7GZG0=(3}_e?!cOC&R3x?WT@_%{d1y zE1o$*>EbHoU;>-dXK5ufWsuB;CDc@T${_{Tn+>y9?SXd)qEEgHO5^zN!C++X_Jj-s zuvUYFudyU6z9RqAR5;AH)@Dd zjxMb;ndfhKSHv12Q7q0({Jfz6+9&CW_v*`JmNPEGe6l^3`pyZ5P6}?BXHEHM7a@? z0Sh*>3xtxU@+5ouX{OveHGBZW$=>|dAlesCI-xJVUt%b8!P^?Pv7#YXQK`RWi2z3b zAjUZfX-ORE7l(b)Ef|+MZ0a>o&RXvaVZy4KB~&@xfIL6D81Fgy!(Usu5NYn3;6o7g z;{0V+XQ!WR)y`qta@ee8;t*-hpRO)}f@zoufX>DdzA4#Y1G!QcyK<1#N?MmO87#ot z!JkO=qATF*kmS4Wpb|H_1L=X!mvPu_9Nwr(ea}p2)j#xWuacn(Hh|CW zCM$w!Q~)p!mc9Pin)O3GdqBE^oFx9`%7#pIVuf}mKLdm= z=!?;?ULNl+Sb7eBmWlcy>890(_Pn|Ydeiev?%!p^leA@N@RTsXHFg}Di({W&@MdIq zW_Bf~O~43fugv_+k9Lr@F3MuFHMo7#0`$)O{o>HT`VNrUIJWQ2$?uDrX3=sh@hZK{ zbXUAK#WN0vH2Vn|vM|b8@_qNBkHhuzfxj#8#dLv1ey^^jB?YcjJVUyw(ul=YYJl5y zm+gJC?ZAq2Ak$GF%x3Pk{Uvq{U=SA_W~_GY(Y^D*9+UAy?ipeo!DW}7f?z&;)m-T2 zOPHgPY|_0`42ubqjqvK^7QkDcyp$_F)Dbyok<(QXaC6sMY={*HuplOOH{e;ifU4}t z0e?}BQ)n7NgD|fsf#~sY1D8v|-jUnGTw(_?tO0Ch`ToOt1=V-esIvE6zLJ?@@~|X1 zH2~$vdG&hjK@*cb`nXA*wWlm6BSNIaxPwO-%RS@*6YT*!K~By@e&HvPnQi;6vom$S zhVIIiswL(xkoJA#s^#QBHMfivN*j9}fHG;L;wPTTapjls*$}{X%RsKcXEBMw2*dJN zdF#`i@20VN2r&F@BBwS6-+p4V&}{8~!+UJb`%WQ{X^jOc`E}!FC3ksDZCEzyD&ZPUkN1u?|hqV(e zF+uz3VrBl$@$Xsck*4-c`~`YPFw8yu4+~)=W1&`AHCsDx@u4K_Btlb-6k93*clmox zgWSBG#dDIIeCnT5c?EAO)QLX&$- ziI(tjj(qa19w7WaA4;E{NAm=iDsV!l_}~nr>$A2*{p!%*!NDn?bNOCSonabBkpMm> ze4Un@U%@lJuY-khOA&<{CuAdqc4&@{D>cmdQKFP{zd1eE!|X!XrqXk{g}KCHXTvcm_-NU_@F+kflkD zEUcir2e=2cK>Iq6|7k5Y9-X9Q@5_OfKBKa}TT;pL3h30BI#zQvazOG{ZsA z;;AO481CH-SSf`zsHM&0eVu^}NAsM|VjzYk@a?ee5WxB(HT15rq!TqkF)&NHBHJq9 zlE+)5>Y?e1J(*)19?jEit%{0h&c&ykQi)Tda^38u;aW_M?}BiEBa>U;Tk zvrTg!jdO}R{om9MM6{Hj4B zYbOWXn|q6--CJUV`Rd=a{G;;vcUfCYECugE(7y1Ibp~;N=%2?<@*>J}oX8uWO6H#W zH+?52RuvkA5GORSL&6iL0W1lI#w%7_6CL*P1B(?G%EE#jxIQgxmdQTxZI4R7D!@Cm za*x+bvv;t93^tD;_OH^pRgG+Nr3(AM>6z)<3fwB^x$m4*myi)L`<)V3br$(YX!3;l zi`};jv)_l~xV)V2aZ#r-{lPyrn>fcmf0v-Wy7VirZV`?=5%;}ltNteS-nV4%4WH7?QTA;yi3#N0j}LkgGHkq?YNEwlzM66N zBbWPN_0K$6J%25svLTiRx+K~d%jASJRKp-5ZWt*nF^J$~FLF|M+!vWDD3porg<8LR zMq;3L0P#@9HS`OmvqL!*|H-`XQyL5tGI6)Xu=iu&zcI{|sZM>G7qDAJU&ym$&=O(* zZjUIFB}duTb+`*zPWH#VlC33T-6Czh!?}(FShmO~y8G6!%>Z4*Vh}eIXbgQoGcAl; zQP|I3yNsS`kIUHC2*&tFLyX%1KD!Eq5iA*EJC%ax1_$qGQbI0~ z2y_HX!<$kMxr4KHNhHv-P)BlImJwJ(r1Me7jhu96oEPw)J<|To82?A& zC?GR90mON{zw_dN;+BL8h|s+k1olyZRw_N8JSGw7EG@GYzBVBpDfq&`ZG_v;pLoj- z`dpe@Q`hajphfYT)|LmKj_lmFnpwETr!k-4;`#4IZPQDY?%eU7^gNsYYzXsA<7*E# zqdAe(+lU>#zYWZ|mL5_ie#^i2<-wSpfhZKRM+M8B>Ux~raL34K9R$bJ@UEzv%M%!w zE6fMRhW>WE$A6WYDo{6*duwB>P0=UZNEyCI;M$ncog@OHfvJRLHRy*ki8>hFF*ONt zT*#0g;Tu{jrQ#m`(vz2KZS3N|C4%0i3cd8!_MBgzcT~$+6p{0>u7=yqNmg3UCO>&K z6EYq{4^;x8iNbG8xO3XiomJPwI_?=bmxO|9VDjcw9<^8d@{nP0U*de@q_HY&@2mzV z&t080PKFk9<$M@xo<}{hmO20>t1S0z@s47cNbR*)H}8y7Rv-Z}UGZLq->278wP89_ z#ekGT(N32cZZ_7uz_TB8AZ8d;#O=h{(IK5m%)y?8TLAE4#Gx?u0pG_UX!7%H&=p6% zgf#9Xf1fXq3`&+h^Dk4x3tCt7y#U)dq8hK?cq8&4XNBw_Bqxz_d6rw**(z8Sigv~< zB+;m9$XttmC@!k(44Ke#XnBA@U1xz`<*kIx@FAn_MJ57$;j|C>9(rNGd;=hah4vCY z)Yb3>E@%*kAt#h+!&&^&d_PUP%Y`b^JbEdYZ*rG|VoKcAjGyggF~Yezxfj$UC=$P2 zU;k}HN@ca=@#F0#P=bKSy_tEWven9>9Zy!Q1tF1SOcko z3UYP<&m97uSUrzf_)si8*0KM11m6eEaTUF2fQ!v=me)_J!!Uo|8k9@@=p_*Vc{1GV z9X;@$KRUhKwq=~=J$a-yCn(c99n{01Y46@^q6f*iGqs=JF58zKfa36ahv8|0O5DB; zSv^r$9BfyUPVQ?jV}x`wvw62RZPVtWyqn`QXtag}{Pz<%i8mQXzh?IpQ#jmJ7_*aN z#d7{(kc3WcLD3U(1W|aav%Z$eK=5I1e?$x&i(?HjZO|@K(YilDisFVp0D{YQT48XjAv))ru>4=n#PID z9nh{(WDwFa{2B1~v>Vb=46yt?V`N37o~@cx6ORQqGW$@!qnW%-O;e&JmJFl*qQcSf zxJ4TLFr}d(>~=KQTb?SMtV@SAPJ3$CmRaN3f)Vb`MF2hV2NJm??ixW_G?$kyMP|?p zm7i3wAU4-pJ@j6pq!74fl5s!cxsY5ZTc z=QAfKF$I#mNAX6K-y+vEBwL7lYy9SAPef{{Zt=rfMjCn?(ebh!Mp_Yap=fNC-DgLb zMmo{!@{%*h%_il|oTfqCmAmj*$U1aQgqZ zQ{&X}9;W#xhP-Vq8i~eyMGn;E6 z7TFwgk2t#vU4pF{Vz(Sy;KEjZepYdjzrYR<$;4rXZwKeu1iZRi2z%CQsgqt=1i7D@ zB;5Pz!6tRL3*GEz(?kuG*1(+?nK_jh4J4lF4mM$CM7gxA+piZm&(p(k@SP0IeAO=ENw!+&_8lj<5eZ%MDfH1ZEmSy4v2@{sAxykm22DSF zT95Yuu7eE+?M{5Y|EN%d58-wOiBHGj_}ZOj2?%}61deIp_#Lsx1&|g4;(LdAVLVTa zo^&z={8M-eoxCkK*;JI1Pkk604uG_s&VuIXXkHfBni8p7cz?b%m*Nk%0-jq8^aE4q z1nxDGmVBctW7aev%tl{ZRIHY=a_?1Qu#tNoT^e>blSGG$_v4{2AH+{(gHh2Z{DGaUKoYc#I_C} zljb=DTl&N zOZC|Dd2V!@K37XN$A4}>P)q$rKhSl1*q94#!yH_pEk_^mqdQUEcd@G`uh4m9)sc8; zwJP{?AtQ3BXlE-KW*>~OJG<{#gVCnj5uCC*ZIQ5-C!SG|;)s#(7~Ir<%CMN@hK^*l zI6Rm|F`b2ryn1777f-R~d;iLv`;#YHYNj<`rRD2c+=rjd9qfEy_1pW?Tv>cqGV3NH zJ5*c!HOa(A1_){6?dgF7L@s{{-}cIsI6Nc~I|m%GTVQ-*N@}`6r3TIA;?zk=i54k< zX!c46on$jLP^jsJ$iTl#4?+ABA?36?QKonXx1-!c8QY#JO6ro>EpFQuB4h%Tc-5b@ z&_$cB5uL`{a840lSyxOM-FFmn!guVn2qL_k^=RAIw8fRTGUY;J{KUABQsbH;q-!|5 z>;ck3+FkqBkK>?=+*2cX>Gx>rg$tS=(bg)0R_vB-i1y`dohM#Rk+D3@1OC%@e5QV+ zA5{Jff$HP0(kqp8(*ccE?;m+?GZ7CV#-XEmZS2RX)zDf(!zD7WIB@?PU}?OGW;_6W zbm&FLn*}AmZwT_TWl8#YV3CHV4utk>9SCqBcC6P$hBCD=+5qN&iknhnZ;=5$Z+$IT zNDkdo`O5XvjZi;nt-L`c@JX>;7^GoVzD3OOO9|RD3DGBeDzOk_ihV!Y#Bd!@s&s|YD z`0&JPF!qQZfCY4`}+Q{k@#4ew~EM zcDS8!GABGscH-;m+z9gHS2JdC8(UU0*jtPZ{5P2Ow8P8~@SLZp8)Tc1Ip5zsBq(&T zh6e(~l%<0LMQlM+x@Z?%UL_M8aWo zg*W{(Yk7gbY-%fuf9P2DP}O3z0ULiDK6V=tUK<KKQZ%_~-pz;r93WT6 zuZEgYBI(_z&NaQ2I0WMjmsN$~!3cI~Fv* zs|^~RwC9u_nFZ7+t8H-7wnD}G7#69cjUvnH9DNXEK~5s^*!9q}+MnyqL2eC(qbtBh zaUYw-a%yJ_uxw43-PZqnjV|8G#re)Qc^qWd^}C@icaQEcvXvXLC9LQ!sb%-Zw-`G7 zOt8*oiOdS`Oq3@C5KKc?$+OEm`9GwgahaT{SmU7=+rnFdyxFKOhO|Xe-~l50`|EI# z6^D|?WZ$nD0xh)W%3JC@@YpIevto~EMo_u1RWSb~TYHct@r0T+ zUYRcYyqkV$&)SFMmTlA`#TSFgrdl{&K)Yp+mQ|;4wjB ztZPy-w+ezIL~$tjo~*L646&{H|Lj!Eq=xuzM}db)jW64^hb&87DMSh&%~$9|u*O&+ zeSL6zb(Zz{A%(6Kx3Xsgk3@%!)x2Isyr~nx(;|`C&b)a-W(S@}LS#IBcJwWHC-Mxo zMAUKQu3=ssN(jZjHWDHB>eEk}ycK@+D`KlOEZ=B0Prq3YPiLNUP4m~AW`p=0dr2|4 z!HUE?a0XFbk~=a8AP96qZxtfTD@e#1QvC0EG#3)TF=$+nlyk(7QliRDJR6TH5YmJV z?3*pY-(=74Mx}VI>pw&B&U1Ui9%3IAhBG%O*6eEb?brdUf*=-VYv>snfjWcRwVBdU zq9b65OYplo47>G2nslyd>osRP-V;Z~Sm4mjOg$_JxwJ|=7k+qF2GRa<)Qg5k+taH3-W}sp7TdIOWtO&mlSanJK?Hq~K-yG5qzW8T* zwTzJ_r~g)b8eKM%-ZEtb#Wf@X&2%OZUi`p9vAdl3{84U_2F0&hZbKa+uDn10|7&)* z(^dWb!bmatA~GY%{l`*k`x)et5dVpHNhKwj+txg%Eb$y+p@lBV3xV8xRMM>?d?lTa zML+7BvDPQPA1q|1fdmw_<_GQg;L?Ud9olNupNbA50g~lO8;o$THpv42`b#VB5A$GQ zYL0~ltv6=Sbkjd)cTCRv-iUE6OVv4MYijjS(#fdY;ZL$&5Byk6@^fqMov)q+2W2yh zSPiKu$1bnMVq4yBv%=G^&vIPaHADl|8H|QS;eO$hUtox_)ungt#9*p %wpzxBLH zFdgeW@lLPFl{n%2L_aqon7j-xIGghJuJ3bGxcnPHZ^@{ftX9t?9vONG@vYawjf%fS zUjvwg40Rs4aSHTl#Kv6=dw<2j8N6?t=b%Zry^f@bSPyHTPJEz*&_7)i-Y2UO0fWQL zjo>r>DM=2abGtw!t)=wJmk&w5!iL@$&Y@?<36Cg^M?@HgxJx!bu??i+rRWA9j>xVjM^pE)4vqU#dN#ZsCPU5J{3VVA#yp0@UpVF)~Va3F zK5!emxeqsEgEQ2Iv2$ntd++Rc)7lu?cUGD0hsUZ*7B>Wo>H$bzdH!=CtuXX+>*bD3 z6=bmve2);=?1TuI22x*^z-|uNxc5Z_cv>=Fa8ouJULQ>EvVXGFix@Pa`Hx#E?vjK8Tj~P>P{A=ea50 zAOt@uE5^U6A3kl6eJawUxCQzvVmo%3aaJf*T%#w!@__ zgcMxPG`c>1m;RV@h+YlC79;(ALJhr|gN@{2$vew(v3f{TuL2KWI z%@1C++myg)>N8>x)M(i*27R|J0JB)Un{hr^oy0vo-REdnUOE9lSAvZjJf1v97cn8L zwwZjVqOc6|i1@;@qexuyBo~>HK9R?U*2_7ZdD>nc{M2K|m9^be?N6)=xMOP;wO@W| z{Ge`R$??#-2zPU(EU6J+bSRd3mWG;GdGld%P-dzuoAa9QB-8JtFEV&By(hzzGZj6s z+9@${^3{R$CxG-q3@*nP=Qf4B%|NOToi;6cWm@#+bbso^zdNw)jdKkc#HdCoSKi>T zTN-b57O0E4Faq(^Gb<4c^Tsk_`~TRT5?OTxzge_L8z^daXFK{jy}@@nmnnFFvk3a%>ULxIj_wrYO*!7h_zVkjao($Z}0_%>($ zMw&&th02})4Zl$AA$o^WSb__bdQ%Ul6a+A$rl}!7x*R_MJd}3sPJ&Qfx;*&o{M(?w z^R$ylfm8X{P4Y-$L3Cl=3H#(t)*@5$C6hgbb3|8!%W?D2^1i9XHXf#n7}9rg-vY+h zr`Ed*ykU<5sXDr$w$N_vg^=`^f^M$XCB};#!0DnL3oC=s&hYU^#5bcW-IA1(#|(h_ zj-b9B6_ZOIbPC@1^9+I$eGeSmf*JqH=G}x~8ItegZuv!W+1DD!px!CIjhr!?xq^Ff zIO@|Rwhl*4Es}L+)Rj9_>}HQfr65KRlT0$yC#?G!O&=|+?(Lh( z=!QVJnrE9EA>9nb{*Z8e7I0Z({PFt9@*p&{i;OZlGi_l?{_$5#Hf4P}$K_Rb=tGl? z=Wyw37nN5hHdA1~&TOL_{LKk~bA0?J!~V1WDw?y3KqO+J8~?Xn{Cp&mZiUQ~#EZe_ z0`UPS$B`g(s}n{mzaYe%=q0Mcelw7EF{J>h6s*vWH+kk6J!4!IBK#P1R7l4>+ZwH##3W>SmTzRqLBeO~ z--!50i}oawOD*9b-*KhbVr!bqgOnF-vb2HX`Vo~fmTzVsr8?E*vw_kUy*+}Pg zIp$zUlIMLT0p`Nj#xL;%Wbf%RT{|iWC{Ij^>uAR=(^`hMaQhP*GXkt9n^~2?M0*1) z2)$s$4~7~6&49^8SK^{RCZHXthcoWh^AIhS`x}!oO7fv_a%fwIOG(OI=0Nd!tnspu z-%U>^?hkgms-F4fD8fmK0Q6V86ZafINR@B~og`;n~n8lG0d!Z_HasvQpt(#O`o z0=pZ*ItHqEid9)6q&NvL+BJk}#zRxw5JUqQd-j$D2vLM2yWbW+RH25F)mvLa zsUMO&&^@zm%gmh~^+UMX-GPOkJlBECUz1d;mtG2a=?87AYdX&gv&T&l?h6<5v3eBb z-mjlMj;{)pi;7dr+q3#{IPq??^j$znT8#+H>Gbg(KeJ}jB+Z+^MdVx{xR*O&|C72& zWs>3+=C4i8LBrYew(k~hr!f0Gm0MuSxFp}7`~SlJD%FdY3!UNjU#z>${F+6F8^A+S zCPTMFQPgXW*R@qJMRSCGx>Ob0j|hQvlO`T4rJ$`4?8Pm_H}LrpyuXwO24=u74aCX2 zFs8sB!(^*|dHh~PaZq%t9ozp5d8QnPLK61B@tfv>onYz7PtpLDg^Al2v>)X&19hC< zyZ8Q)3WHHxSYMRLedY{M>tjdM4s)7j4XeQ+;-Dn+NQY#TV>@J#Vmxi3+JYHLRlJM$ zsQ_IY0P(`whXseNC-3x8Cwi+`j61QY7^KJ?+HGKXK6(|tX zIeX4fL*OTZ3gyTG$8vaH=l4JmU1mgMayEpkn)WtLoWO?pZNkZ@ZDZ* z2QWLeFJ%nR_#wM{&ynq(v`;Zm2XHT?>=U4MMJjP4Jf17C`!KIpRW;>t=XZP-HQE|* zRug-&?Dj=2ApCsa8C~d~zR_66P#X_K>q4M zE~^%Z0Vo8h{f7^9N}gc+MxWe~e`dNvp+qV+G}yG2i#@zpDZR(gOCF$*JVio&<6&?=EXa~kcOWu*bH5UE0|jMjVL#zHvA3fFcx`d2IKs#83u2WN^H z+q7j6#ld37ozSxT!QzcO(rYFX?cc+;eX`Y?1AUU(LoyKY@UkzNbH0FkpjG&&C!ZhA zCzI{>-kWmAnZ2Nh4J9aE5@##d&Txk4CQEp<(;LGd z!Rn?FzY2z|IxDIGa71>i{R)K{o&QF`)Uy4h*1a|9oLqjelqKkHG_N32o;MFlt!2F9 zF^ga(NPH#|YYcJmOqFr@X30?%q6!U@$Ih{+BfW-+Srs3KE=BpA`7f@b>I^hImEG7g zlP)fGQZ*)R@P>u?<tZdAB9!tmIjs{sG_i8 z_(N}j$$Soi5)#jrO126TVi*!UIElBKTOkPvd^k4+vmP;E=RWYxFN4Jy>i05Ok2Nq+ zHPw+cJRRVG>PXr_?xh^Tq)*R|9_KX2;qiJUy(%%K^iv`0&9CE{ULye4gj}6(gG`L{ z(+C6QEi;J(Yo@^OX61>Q(+y!Y7Oc@=r0)3Nz@i}gv&)3VNJ{63w68!=u_n`)fpaN= zK-Z>!bQdZVVxmc9@oc^)E*kV#8pQ6Vyrtmm>6%i#wq3p4m(G^hCiIa~x1bNk$m636K@O z#>I3@Szc{mHX1S!a9AxO(3m)}gyGQyeriloN1$6}3sprc9{NY6slZ6^+8LBKG9-Lc zDK=(^DzRL#L3PydX?c4Z|qeUKq(0YGba9I$x zO&#r*#<}JyN!_E>i?ePz?vhKl3(RMoV6wV`?r?62{=|Rdx8@UW#q#gUZ_+fLl^B~v zYn_N@s~06t-S55Nv}f@FgDF6FZImm>p8y6QyUr!jtelU82g zVQfVg_1tKPqp+w-l`gzhx2RyJFID5-46SpnEkpwlVzSOh=l?hPu6uasMKzc)-lK*! zg^Y{aM$P)m+abogZzfskLga!RN1Zb_Oi`njt<7R_w-GwIdJ;MLWp_AC1PWOnb-vqG}L^uBQb2 ziXBX{TC|@1!$I40F0#~HOCwu~l@;>rJtf+p-%g-zVBlvF8+Gq+@Vl$dZSeMW@lx}& z_Bm*;w#46#gjt!W`Q1~nAQi%S!%5n<(be1VPlk~9$4Or!T?H@LvX-Z6P9Ehz!<}4z zreVV%#;3eDJk?szR>v}kWgXfRezuDd*WnM{wGQ#l?lTYbRP&{ zfX#vn-~W#IErewp>~PxwsTtE)vmuBnwL;$RR_=0!LT)3Hm9twwac}(=GZ@kLfsVaf z+xq!p*#cS&UL9~G6JQ&^P44{;V%IeXbkV*N0F3m*!s7&*L{&7Wo#LLvHmH&G-Yiy^ z5O7CdDrk?w(1`!oC2?WhhPziJ+K-fma-S%VyNWcuOWB#V>JaN-(S$~hq8rs3^vEzA z<>0GIPbCvYi&$o*m(cnFzkU>5fevA)K8Q(H-VutrU`^z0dU}Fx-f*cI{1;v%6yz&c zKp_EF#xvMIay~XYfprs`E~Vy(yC7(4)pgi#X@7pd_WMz+si4*pJP4zZhq_c`>*s1c zvaaiTkeazVTxh5Z^lJi8Md$IJ%>u|LpmTdQM(n5}=~eE{3_6*pngS|BK%Qi*seXC5VZj_DeNQE+lHtQ;qOQyBR55cIcxTcOC4 zBldxBHFU&lF+##x+}o1%uk_ojVBo99j+&7EIE-$T%NAkf6y-ztgx3c+`gkB$v$gbq zs9_O3>a7!b&Vy15k!sB5!&V2EmZz!#{=8tZ9XVDhJEsJoMVlB$t&QE;xYbC|4XnSy zE|xz!(sKAUmJ`xkANq;sWCveI)3r_l^IX=k#^ z#o{y}pJu-X3kBy!klr4)Na_1oCT5pW08^mFBmwd5)XP}N25jBrMke0CIIWBbfiE-S ziq>n8ej7)lCyouWKde>_OKTI`U`Dm%js2z(9y${5nRlb=|Jy`-!D`NTxj;qltYELG zb5TixDjYdYD7&8M2!j=4NZq{3?eT-`7H-iF+aCuv8<7Ni4{wbY`AGKnd<*8Utif$**r|k23Wg=bin`|sZ~RuC z(nw*@Ta1EVu2tM#Lu;%NbP>R8^(Un)G$j3A1l5!ce70~k3QR1T9me{&$b15d{>I~C z;3a(n^}pr61&HAHGWKt59w!r}b}u8qrrcE~^d4|6!&ixTQ3|2Ui*oLBQ7pC*1g2gq ziMY^CmQMV1y8)AfcA=gohQ$1?#bsgwkv_XF_zdPTRo~Q?_pMjr@9vGESz9BqZZJPs z&#hoBNeMkuCtMJg`QEilk&r;=(LE!BoVFZxUOk)B!Rf&gV9)&RI5QupcvCM8UFoqQ z%>XaQ!t|_=;$Q1PLYQu@h=3%G18mmMa83D2CQX2TzZ7%t0=T^b?K4B-O|_5NHC8C# zVFYx8lU_E+vJ*j4_TE2(EOzR_R=GzmZtPB^4`OA$6xgFsyx6oWM%9{dR~e}m3mh0| zDnWQ0{?zM?^xwd|K?AO4=mT)iMZC5fq$G;!Z5r{yBuvm66$eykdt2Il(#c}>@63ob zLY6LR?FWlb#gv5&4Nb^A=Xe!pC{zDc5Kenqne4UIz(}K1nkW=iolC!W*ZWweRiT?5 zNrW1x*KQa92~1%yE1A?w!T`M?(U{tyzr=pXO;dloG~73I_FqcZp?)2>mceKU(x!lu zD9<=CL3C zaOCd5ps$R`UO$QseG`N+5~KPN1hnhVe1C60!8xJ})32x*SB5gqZGws@gL#JF`>4c` z=mg)>YW3?IZfm>nA0I~%TWJdLx)r{OGi=vB#FoC5+l~QPL3xn$s=o3kY6rufuq_o` zR2Jlpkb+(=Z*4Vl62zk6bp0_xmYm!b_M1_SxXxxyC!z&*&7R_*FJ8aIa|AkPPAFt? zwr*?PR)@GUdAaCeK7R31hEDCI#Nj@f6y3rOjDJ}d?yA7^iWY3S?Zg#z=$f;?$75wS zw5=d-+xo4*Dg>Iz=vA%tp>4O=9$J@_%ZjCDq~8#N0tT$(_t%Xc1u9o+<`ZZExePTP zuFSL|c*{^$)>F2JJFDLiM@T}@4`-V<<$lffi#t*-u}}&d43Upp>j{8}brwp3uQm0( zDk}S&UxN=~>-;J)DVy~_IGq+M-0(4yLJOq~M7ORKNJn+4QIGI&;R zTt9;aHZ6@XW;C@kgiX_tcsPTaLD{Vjvvb+#{Nt~g5X!v9stE`cgBJ)94-Zrt^0xuV zBbAUDlEP&{$DyN-WG|*#<;h@Ajai6#G*1p26zRN(x%oAOefgTUbp$eE+7JS4N0E^t zZU;Oxc>F}!+aPtl8>_+oENf~w31!^AWs2{_dtHD)RfTf>{Sob!lQ`OT2xfFPz168T z|6R0HeAW?^+V!MJ(f4XG>$|`|OpUuLv7zA6>!*ibsS9_F_)Z?n=_=e@G&76?gvZKZ zO7(IK00Fu|rRrwL(gUKios*d0-mCHgY+ybtY<=TH_8(Go6_WC!{NmTdvw`-{H z07GWa5ITLiI9jZovr6NQ7&DT&%W_cUS8(Uwc%HRF!0^0M-j^Ndk^D?b%HTe1qG3k7 z2Zj5=q##x-gr5Ye)I>`w=s*?K#jDO%@x~SX5N9xQ9ou6ov#`B<$NmzCL*}uy>fJ!&eZ1G^wJ5wZsAidE4foH^bZ)u zp)@Q4%$vo+Ml3@4)&n@As1#gJ_pcZzi=YsIzphn3Sv7H3eFVLF=NBSQyPT;g@Zl>FRvWM}*yQYb?}r0>!vWyWLqHPV+q962=3d#kK?$%DIH2 zk|V!Ntg)_RucDC#cj)OI+9811 z_nPPH&kE&0PAcEeW!3?&*NUm|AHFddzd*L{psneEFTLn*k-V>Lp##B`jNk}^S%(i7 zPyc|ItJ30K@&U3%yk=x(yhF3~5FoEu{zCEc3P3Io-%DSp-oJx1sp`K2lhCp70~QF! zl^dWgmk_!=$>c-nrt>(ot&+Y+HfF!&LU+3Ww3kAHgK%pTUqysffK?8ip4P|l+|Ed= zc5Wpqf#j)E6C~?p;IIM&rw7E3guC0?l2m_Qb&ZMDhhGa(f87PxopbhHwWG>g`w%Nls)-9m+Vi;}CU%99cME1h?{xV^CxMdbG&@ zV_r=g<{;uaOB^{dlV-TDm8FI4#m%8=KmLcLd!ND8sOegnqS;C zB6f-fEcR31Hq{84-uLBeR}|Myo;)VP_y#c9Hp|M&OlfoAs~F)rw59hiHWG);@l(z` zQf;cjougLqZP%Ewaq`8SIXyUmZ(Cu!YwkYY=r~2CkqwcnICaNZy?D}CyuIZyuB>CZ z-%8#*oE8F8(jSPii|vW2pbw^u?x2?cSd^KeJt%Cf10NK@)BgHxjX-yHh!tizQx>2z zpl@;u>=m3m+E$xvll=?{^$NtJu1inj@hK6~+v;gF=zJg^N@r2b`5rsSBs(b{vY4 z3=!z$5f<&I&X@r+)o?8;;O`)3=rZxWzj35yVu>2&y$D!>g*t6{0_oO2w>;cY(HUZq zigr9?GJUP0g?7z0q#`M@1)baDV3wlWe#n$$$LrF(-;u2FjeihDpjYcnb1KopSsy1t z9mT?>d}wiB%X@hR#!w&2uujv~*$(X~>%xOZM_c~oMtnq!IyWZVt%YA?tpGm?258jCg$eh@czlOuy|o(zK~yS;8J(5 z&r#0q7_aPG?B)agX7|DlM*TG7Q;eX;JRRH8IpI$(V2B?M&nJQK}lV= zs&vyy@E=euh6RwA&vbd~4%8N(6~Wle3x+cg!_Jf-$ePBs40k&~i2HV#4Ov-8Yy#>X z{n28DM!T%Gg`HMJVZ_#a9f|OlHVPTTeWBGzl`-|k$g4@$W|@(FAc`E_H%Jg=88m1w z$PiS1E&`Oks6337!rDb)moZ#mR6S?`0d>GXG!k1`#V2tce2SP@eg*EC_%r`DrHRt1rVZ=)KL)%-wzQpaXEjj4-jA( zjFOd%Vl(i2CCHI%2A?p_qO!=yOCTr8S$nemF5^fDrRpQn9$B=D|Vnb7PQU<$XJ~m6e(6-nxy2He8;ivPZj&Lg@6| zvls(rdP}+z%^ViJDk#YKtJ<0K4HL5NI%9_wecd?P8Zl+0n|IS>$nJGtalvG*b_88K zC`6GeaRWhzDg5G)k*Jn{!t1jmovzZ;*3W>Y+t$61Y#*@2^S9el+@b>E)ZUKf@NMgv zcUyQDdAEjhQOv_8j-(f4h}m8=G4Iv~6ZXN9)n=D8&-#1pJIfsi>f?=q!rzvxw<#*``cD zxFJ~pU7UXX`J3lN@t0!t!rgTZV6~~U`<1JmBo+JRZ$*4T>Ah;5VuakF&6oGlj>>WSsT@vOQ z+ISI>K=82#IA=TA)0t{jO{>)RX1$J1OOO6aAPCH>pB+4Y8JAmcNwd1jdke2}+GW(fSSlDC$x2~vn*}7+Y#qJ0#cey?t9j>d(rbobEyJSk` zSa=MAGkvDt9D6e&5-@De%>ojlJNbfE*Jc6A1@-3o3;ixaIW`k z%2^~@?iW(1oc4)ltcBd;UMnMC7Uk(k#Xb&ZG3%ExM@~x;Bg*`$6U(3wPa+e7q&wDo zPk#z2%-8y_VYG$yeTNr7ei_oAw} zl&n-qPv^3Kq^-b(uqK(fj;V|8Td&b0ebf2mUIClB`PM%P|HCw@Hh!BT1W)@02|FIwvU7zCRO`km@HM@dKb;ivYGh+u%?fmAW z$MsVU>%Cv})xFE;aGEm{-vM0sf$J<3fEf1=dP8u%bcaq%YT66BktnlYZqkeK((BtC zBGst_@g|iogx188Q83Ox&F>O+CD#OtAg-KmHrm{|Y=v0xBu(570C_@%zvl8}>fZ_6 zl@jPZZ1S5oheR9HsWtI!Z7jA>XZ|h6wBz{H-=E=-_OKSjHN6u|jS0CJp3AruiV+d4 zK_~|B7^9f=^j7nUFJR!m40;zZeq-Z$H)RuHM|TM{DuV^kUi-2QPc8(JJN;f?%J^{Y z#{Ex>%&9H43teR79~}V6i}jY31#%Hiyzt0UgrJz?(x6KH-bqgoI{sUMUtWkKKSbr1 zt2#sInU62`1m@J_RIvcP|8wExYF@L@EplX$NT9M;$R2ixTV^~yGB~I`mlNG3YH*Jo-tD9kZLjg3?#B*k5rLh;Y1 zGQPx$$kpqE3oHGxr+_-T=rinFmeV?$vp;lsVU32f`eL;G{Atn8ucqLf z*O|1__&+V=@f_S$vf;zVY@YiBbfjr-9hT-td$UO8%sT5-Nd?q|LUoR_Pa8CQQK~(R z`u7WI>?mn3L2S;3BeRwS&8@8vRUEzn`TDg>=Vwm9g_&I8c=ozHM?%U_MIq6QsxaN> zdZZe_DcT&&&&MTW!c% zK=f?G7}=8eoN+nHH|W!@WQ!rd(g=I7-H?~gy2~4Hj+>ES?bd`!>-oWokq402@7mQ3 zszFKk#bW(u3gj=cvn_uWqFdpYoXRMUkNl#&RP?zwqzwgKF4uV@SL;_jm+4;nqU*Kk z$~YC_qKsq#ba^uN_IA56w@W|&xD<~wEt;#>Y9dWiSn2wG4G938!q`!VIrXIrEhe}+ z-MsvYI|QBSDJMBsI+VPwJ0?-@H7SN5lg{d&dx31Q1N@$2dcL$~vsB<89(tpM-l-)& z*<9Cnt?{1owZURgNsqPHHZ)fMsrhto=TWcdDgPMv?0@MM6Vo(O3LGyfvErzA^1O4b99lGD&;f*1Z#fGmSN+ccZewBw>}T+G~H%uEmpF4kQn z{UHlt;T@}|BhiyM3$By09OquKgIBlri$w?FBM(nov+pxQDk#%VFXfhTEDPXm%YnFotv4}TRMZVnB#Km@DwI+`02x08t|houG4rq(1uqdSG9t*2g?-~%CfUE9VCE+qUJqJ z%^p|F?fTiTA~(1kxhJ&Jkpv6=Y^HuMhQuxaT{N>Ov)Zet|;VNqztjhs| z$JJEVtT`$XDRv2Fxk|c|nOST2fz|n$r{2jXs{Z=IsIhYdr3EEBRj+9jBQvDd*A{&u z`IkMdJogqeRdf>{wL8C3d`^`K>#shQ5hQcn?cX7%c|{!*A()c1HPkz*95N9!6=3=o zs#UYE4lm>wO7G{i01!7>e91~+7@T!~aE!#^Woz4|#q44~l@y{lbFp{|lToq^$N z05qJ~_5|E;3WT(#v{xlYz#+Y0-fB!#OH_K0PVxY15TZRX%MDVeOPS;mb(@p^a9t<7 zrfZ9llITlh{NUx<s=77MOZ){u5mBXH^tnN~JL2s_Pg?XZO-rt<$O~bI0e`{by3# zn9Q%kh|C#DOU<|~uxc)|4PrO25471^YjHxXY}2b986 zf%4=b4{35ha$x@g&j;o<+>F_ZsVse@31;>69vw%&psi%`BWC!#wp^a7hXxF{Y#cxr z@+5)DH58`QUt@`ZrjFUEueguW_))?@DtxB7zC_xhhh5ExBUGECMyWU~pKAMB^LYJPWDOLMu))@fJAsBmggx zj*;a_Ei%VtwlI}L`?JBb0n8H5h4yj*h5-x59XVJVBJLHrB-zU?z7JX``Mnt)!hh~H3? zrh)0^j{O3KH)1!Eo4yhZ7V?otB_!}AiDxCZLqwN)lre_vQp`d7g6Ir<@&{bCO?w0V{LjAGC4>=rj47646LZV>@!#Oo)Ves-Z9p$@uUS4;p$QWO>bw{s(m z_HE8OtT((}{@=rO{3XfVb1K6<3+`nvLCATw>%SaDl`>j!DVlEgXqSlDN79P2f6gH6 zc}5;&-= z)v~Ityx@8%6QBEA=T5<;$6|W%gT(-<_E|RGhUTgv1dc(E~Ne7Ce z+ee|IWsyVmhPNp0h zU+aGDy?ippJhY}SArYxM&*OKsn+G&H>eXwEz%SrnIlAT7_!|_E!_sZ>m>Up)*>-ZZ zW)Cm8{euX0^Y6n;b>7#BK&!%CLk6}3tnORj-&1bQ%_TFuV(9Vm!vjxuA$bqo8_ZHg zA#%!>a-Y#0{%Q?c>-&T8+!YC;JU}diKug-6LR3P&-HvlI8qe&t_8po=@Zq~CLjPJz zll%==f1NckD252|HR6k*snZUyx(~CZzWB{^bQ8ujmuJPRMqgYk;)*J`J^Ach@cU?~ zLB2hGJ%3ypgGD{k$Ek@v4bN}udD?Q6=hy*rd_l6Xb(;Wb6p z2CU-u>ZpW^U)`A`(`=LAS5D9lemw;eon0{Gb^zT=qVQ)QxhJJG>PGCG@JgEJ$D4~~ zp+{4(buX_qZ+X&dHN;Uska?A?U1d7MqupsHqe@YTE<8|OLETd~Hxr@Xcu#c;b4CR} zy(M8ucUBm`YyfpP$RhyznRo*_F9#3$SL#X*>$W07TGD2Cc3$2+Z>l#)_yp^=xXtk_ z8d()nJZMfWK~MW_Iv?&xq&|{<)vuxj_t^xNNJ;8IGfEx4FTG5(^zlVoes#=Bc7v~9 z_m9?NxSG$WOF$0aX)o4_->gElh1@l%4_B0V2$RF6<^*b z89&wd&d<3@4RZ>l5d*Q^iVE&d(LASbopDK0NQYGY@IDf#7Gu^)I8^!Pv@OGPRCt~Z z3DPbu2L+#< zNh}y(q2H-`%}$lst%Nwo?5G2zL6EbX;(f7y=peAG#birq$L_=#guxg+<&#Go?{h|+ z=8|9qwXV3wYr**NL&~8#_Of}_?A1;p$pxZSukKb2ivo^Ix)tHS{`4m~NeJeb#iyEe zl)oSOQ3~m6Y4=)~Be=|XczY+oQdy_xZe?3J^`(uH;CA--+;lc}R0OJpt%H zW*Re`$oxWI>0N&TXLaT1W_Slf7~A~$&Vl#Ibm0=4+8q-(Ds=;jqe|E%;tUK;37+6Z zrMte8ivVO5$AogIF|FwLs=McxY_!C;?DVXcv!^0HK(?q}0K_H)g_4#|rj;^Ya8{(I zS-h$uyh2%DOmQi>?Se7-y=Y;Yx>7*^DL~f05M`zoB4}fhwgZijM*C3`T6Ea;smCQJ z$EZhlmY(SUG37H{8D5iQ=?@{DU?(EsFQ{z&%AH0g1eRNyuRIu&R!rcYF7Iy?$TOpa z5ckp1t}~frmXI)3Q1h>DR?j&ut?*Y+G$bK|A6xd16b;Xm&(w)71hZHSQiSZAF%7C= zQzmvMDy@gSj{>7_$khgmT&vQ9{6?%!{-TAav3YlIguE4#xdo-3?-tlG&xPFA6>9#G zBQt9Ov;->?FUPbT7~_9bDWO#XWv19>+iz*Xt>M>ZYtVAd3X|pQm^E}8%uq%0W$dN7 zw38Df&iQ=?+teaTBLJKK%9Wtqn8C>aVW4%q8IaflZ3LEo`xT;QD|Ek6 z{$4Ur2$TRuUY=J^oFy;Xb}G4tTr=%vxzcm9Yz7Rn)#WmW| ztsu<;xVLC^b;#!T0C|^Hb@0-x4^O%1ICbOw1F!R;{`?(?b=aX1Uhnea!7i|-v&E(b z#=9#$87&K%Gw%LhGDe6Do+PKXC(38T@f;5YLEYH%Z|{s0*r=1}5fFRNWuXR%n1zH5#dvYHf!?zyRnL79LC6JLUySLS;gcxa`8FT}8ekCKm ze&X*q2M6Io^wNTm&sIO5&j^QM3E^p!DNG;%&#G8HkCRc>>m`125nvJYUu<| znTq!ez+vPeN_4!29 z`u^h-8P+p>%Fx8-3S2M4sc4m!a~>VaseVSM8ZkTsn8sQg;#}u4=sglLQtEn3G24c& zWsH0<2<>P_s$3T520850w2MV>v4**u!Nxu=P3;5&u0}}n#UhuVY@n5PkqR8EOc!&1 zY>O^!oeq?hVwZfMRKd6y(}g&3mjp6JBK|<6s7r&5Ms9h!zp~{Y*cS0|ZQEoc6;p1u zwOM?OyON62 zco$NnyB-p85phLF2648@=dpf!tp{}vb3nY$+R=Nh$Hv%<7dUdJvH{@>zw#8EuV>R% zHFY3nr7?I*5DMnmdAnK4SUaa_rIJ&`OUd?_R_IcCQB~KiuBAO>g@hYeD%A|_8gdh_ zfQ0|FxgTA+@e?sjBr%Qo^}eiKDk{8nL+B$jEZwn`RZ6R)7k+uP&ijj!p&4q3 zVNzFGHG%aj`gCTL0H~HPRy!h~ZEc}UP9>_-D4&5x{)*4v_;oPqo+X%Om8$TBqS!xu zhESsXvmR-kCSQAXAl{|8lORCC@}_OT>sd4T(=(iNvOXzz5#@%~ym05E(liy6&Gboc zrw$y)<7+@S4deHhJC~C!7v9i+y*c6+?4oN)Z&3v)qct?2T?9DwG*jj+bL<2BK4qxrzvC zI;tcQEquo33_KBKxGF2q zxF;u#x0*7%j_sTFsz@O_DYHOp+aH$1xP7NCQ=NwG!Q)ZU>3{7WtCb)KCkZ;%yNFn;g(zq@-mR$Td;}g?zTO zw6dw<9|Q&h=Xi(rK{5K77aL8>TRo^nu%Efwp>O#Gk~>0J+Pj|Hpm>5-eB(WytP+e{U9 z0!7#Uj1AuYEUs5W74M|=8Q(AlQV%wQ<9wfWC0AI#h{*-pwrtc`Y#r7YDSFEw{gLGO z6hH&Q5qffpwzkWBuW$H<10b5<{>;XUTs7)`yGs9deR{qX+vn@Orvd|?*i?X^XJj)O z!Kn_vR=$3NVrN7o$hbk+Ad8N@GEb$r+eig^5;g*|W2Q{qgZhy)AQ(q+hlo;&s;bJX zP067nuZj%JkqizZKWZ$;G)fdQ65k8Lyc^yQqS`_)ZWRTfbPVu8#7m4zbGD^Mzof}l zdjpj~S4CY|NCu*LQl&@KMJVX*BhihZ2zW*DAnC6IkDj`*w?T+K4R zch@t&{pnuo($`!F3KbC`v->P?E$EEMh1=`kwing`9z4#YBdjxXR*qpoFl*a>6_5SU zow#y%wE{=?C*faxWlwAJE=?``wPK)b1_Mz?)a=$U&|<$rVfh%(A2GI$f^&MzNY656?9cXTgQSo;0cZt$@iE z@)OYBh`|e<_qb22$Ty;w`^JfzqUYEmnSi&KwzWmM-8sfWF{#6p;Kv4gM38JIazZxC z%|=i-sR*q)V|QeHwQG}XqsF{V>12x^hkK@xrDFTgDD@`U!GYjFomxxS^7iTxF+{dy zrTLSIB8wPb}@@(xEBwAQhzN28by@9dF9F-1ykapRPMN@ z>ylPT)UBM0T)ydsi4LT_gh;dvFQgW2(TG7!p-=$Gj;eb4mv5{@vnSW{EUX?%hOH3GTcjS7?h5 z!*TU(x?ja65e`z)X58>8JG6i%ki*dd%$p`=;V#W@ocX(7K`z)t73ipx&zj}`0OPrynps(PuB^K(~kXJvI z?~|f}by`Nny#-CCkQWD0;+MpE${%FQ7<{`8R{N`FONKIz9RLXfS>;l1(i)V6RYUgU z9#~nQRxmd^ZKd0ZM=z%GpSQ`iauvAzkq(-VY<F zDA{7A=ZFR3CNhV#;Zg8fcdZ=m2KRaY`a`ZM+TiIfWfG)8!O%-z8E=0Q9{4mb2T!ZU zDH4H1Fn1Judsc>{h)Tn=0mLZB+D!HA~XLRyF&8eGAg zVi#z)^uR7xB`A-NoHgVUgOF5~fz9{_feaC*Z=>^XQ6E;5`>`JTbSqLre9h?et?jyDH1I-lVHzb?9rV6c$qCYc3rDc62GYs` z-|kPxUXjb!@I<;15p^?A(l*Oe-(GcbmK#-QGZMDkRHP-K$gG;yW<~UV=f+0qYn0%X zsnJH6ao7YN<@QMEWfL(TY5+4Y<^$y&P{GGR72f)+^!mxFO>qm&M3-1t4tLE_3TfC+ zRBZt<0PP_BU}tD2r0fK=Sd&TfG|a>HM!n@W#s{E$M@NfAeC72RfRAQ-N;HCrkHTdwusa*V@6IiqhESL$bBc-(sfSfe`|qVN_!4A2~VUkJjM(Gt z%8J&d&Xqq3I%W1~HhZG$d=`{8iRq7WamXQEBMvcuyL*58zR^`) zP-lvj5)lZgpzj}rO==K+)(+SmL#Yu!xY=wlbIN>t>oB*o(J9^fAk(uq{#%QMgIpaY zwv*9Jzf##!6Vo5c98tjgkfc{(#x{nzi8!W^dEvm`hbn2RpF=8barp^pTjZf@avKToAcft28^? zI^t(3SiHdM%tu77iS=srdwgb`k=NYyV6hkMn!K&b-Jg4D?Dc0fMnUtK3ojFAsrT`R zu^8)0Asu@DkoPhabUd#H6~v&C<@se*yTcn#w09cR5SHFcsN`jxpd4xn^4MVTLg$=+ z^8Gt1WuJ3BJ6yKCwx6zl?6%xp>soHM?-VCrnNnRuX^vg#-XM132}Uh!8y43ofiWCn zGOEE?3GoRzuw$?E8u^Y!jP}i?`=3HA7IBk1bweX)FV~zOUs|-vDVEzsva2H&SDBgD zKMC6h#WHLIuTu6dPF#)w1$fNxhEU^+tJHG}!6##aewVWuW%oS=obO)C=x^!(vo~Af zjr-3@$ddX4+q-E&=%+(2B#xDD-SkoVP`7${SIbR9Xy~XcYPvP8V8J#s-r%wQ zOBIl_*z1|^yX@-`7xR`oY{dPT&L0n?EgwO&SJQNtxwc!j74*Hxhb{5$!sCfL+VgRA zJe#|f-ju}>*jpkGU~8pyD_wk$R5cE^jqt@hsTnha`qn%?bjgDj_(5E)eSQ!$oZ08W za0dy#ra4O74%4^LhBa{1HemWICj;7cj1;5@59$X>3sc>}MJY|5X|?MeBbZp+%HRNh zr^ozB-ws+nr@wmzlhI6i0?mGlFB$<9#5omm8XSW@o=TAQ;eqO%WiSG!Q2rkPd z(g#7F)qSG(@yWRnf?>z6Q(2LL=4ZTN111y)e+ylPDrY4J=MWe?YQ4&94IaunKD?2I zK*1}pp*f=aFDt|2c*TCEzD*DzvDhGFaS}?S%30F>?WG+hGN^VjY8L*;L&BRKyATT2) z#T~<ygC! zi0;sx>@qHFmA}$iTTc1i++=9nb7gk!ydfiA{lo^EUhnv{r&aiXa@`oI`-sM_i(ta4 zyFw2@GY{~yqvYoDW+}QYJ#~{%n3Nj#w#*}>&L0CakNf5aNYIU&08EwJT;${kQ0#aGAiv z4lk2zCFA6gcs*W3tQkWlQGr9Jj^{EoRSL!FXKAYL=Eo@JVuaGpCs+DPdUey5KTyGh zj3FeN~ve|`71}!v;I^&x!Qb~5eJE%z?-&+R^X+rE& zIOwk7fXAI^7ev2(DtJJzKMwo^z$}Il-<~e;va`3LpbuUO@qp|$i6f@0Fq#eY0tLT3 z_)Us2ENH}mMsy~e4j5Kfm@WxN7)H=}zTg49cnU}!uelwia^zXlZMM3v;~_ZCVwo{> z7q;cj#k-&VZ`3f0;I|OQHkrm-ZqH)kobt~@^T8CkXf;kQ%EdO`7Qkd30V%lRLfsFs z;P|SAW`QY`c$eM|BM28!=lCmw+J<$bGpk8|Js*o-t0>G%o&9TDF?^ zI}pQCqmS$Ut6FUTmj!dl_3w|qzX(QhYignr~VC8rCI&tHL>=* za}j=v10)1{%i_Ta9-&*i;% zO1)5wz((ADgtEzFt(-%my2~p35Gbd#&OHO#CJy8oq=P2l1D%M&Y$f;1{aJ<590<}A zjq%{u57hY$A3Jlo0rTXwqVvVVjMPssI{G^WL#G;_3S4G~j%CR`*X${O2-_o&K2_wb z6|^YMD+%LpBTDG8wn8*=`6iz&w=QPxz6o*U@ZOe_vCxS!;dA=2g))xiRDSntLwE0x z>9aG)4ki2%8|~wBbIvlu4~N8+YOEC&6bBA+r}FY{nVB>{qCLwCN3>*L?^(sTl47YQr4I^p zX~3!!45h%mrsJtW4fL)^oCYD)mpdhVJ}|u&&%y2eM;c3m%r2`sH|54+bN`U_#a`G> zCU6+kX297-dI`FRN3kFbr8P#><57NI7~I(6F@WH+v)So+p;+b7zJX__ba_9jEl-9( zI`X=5L$zGzPgD$$M$lzv_Z}BtxdL^_mC`2HO1+tUP1V~m-k`!NTyTfIwV2RS&moD@52jzC6>U=|iypYlR=CSdpSX$*4=EQS+tNf| z?F`Yr0EihBLyKE;9i&J@$$5nDDa7gXc|U1PVdJC|OcXj4j#jV9GQ`E2;+c0*yl;hc zCz5&L1z+_WYz6yLZWf;LH7`$i3?DjEyo9 zgwLJt1G1m)MPq%FyCDQ{j;1$D%+>Ob-)MX7T(Z@`q*;$J$M3;6W$5@fJ0@z*CCOMfxDkD0_s&Ei7_L72`n} zSydj;ZOe=jC+fU!4q^iDJ8K>qt^?szk1R)w`s`A`a;U&Lb?J~aT4{5#*GWTYaq;0l z6qlRfGA?qoHvbBkkSCG_+;r9xo`{@W7qR_oZX#RWr$$TLzOuKaUPNw6(~$vJ)#mq4 zqAm^g{NWjRXDvBOVdyDZs(wu8dp}>uW6u6it2eXCMycYWtmAJg5{LtEf2ag1hy(X& z+gI&+&s~#uns3OFjeH;$-7|vmD{4pg=a9z$8TWh zYf@3J)*WR`;pWQQ36?Lzn2PU#^H11M1mba`g?1ixUeJ0%+X}Iz`Cq!(t@<2t`FlErf@;>M2 z1lpQ}s1x}g$D+^l|GRYjdq`ljwhU+%7x9|+ zp3W>A64SFDpfsq#gETeHN3YM;0IulYbOwV0I<6w2B(xR95K?D zS_hBjnOKK|O_$z)d?zP|1<>{2ycW>J8N!uazV4DFoUvRTlCy73K*22;Leq%(GSS?G zOofhnp&!;J-UxJj6SmxD#M)+}Hn!I|nj62qg9gj>e&ZC-55LRJ<)CF*_H8$+Fkb1i z>%xfsU?6YqW?>8cGobK6Kik|Vh>*rs3?!Y3za3<^-K||FKr3czI==-gQk#rNvHl%x zMw>>26k=&%s-KVF0!an74NUNS5Z|7243=&~P|du?-^iVX;L zdvKKTq?^S#h*~T^{K84=T7GwI9sb=K+D!x7^Zc(hRa)+E*ZolTxd9Nnq?gYP&*U$nW2jjO- z=r^LAOg$5V458Qow1Gi>ZIUIWs#3Arb2A7->6$SlzA@S@`RAyW;EU4t1BD@Ym_X<+ zrMykTDgN4RC+lQ-9hOwhfI_|`hJJH9A{CvfTBcQkv}(M6_9V)y1MPJ=jO)AKZb8_kQx|c^`yRCR}lcLXiDdNYfr<6^j_hm@BBQ61D_I>QDAU-2fBZzsMKC@wty#uOY(i<0|6 z&7nmiSvHAGo57B{91d|duj7$^sN*Q-A7NcPw9Z*K?#G6aVT{LHiCS{G%LzD~ z@gGZc-l`$Kz-tBgV6Y<6rxE@>vR7fgbW?)X3{NYoZzb39OJC|Utq+xKPvVXpoCdFV zql)KFRN)h$l)H)aROq)anKafBk4HnKqN=B90RRpHm9NxE9SahU;38L)m4YJEqv zw&w~0NSy+1<50J$H~U9e;2AicvMhmi`0y7x(vB0YgbtOn@PI^nZQy#=FZ8ac-}X8) zQY7j=jm%nIv2`E?4a8f4y`RU~#`0{V--!Lnjc|~vIO5BUkj}3)y2K)fV++4({jjJ# zTER8L$t)$#Et`3&W!~`Tgv)Z8xgwka0qUw`S`6Fy`hEm3i8-m^eh?<|YqZu?sn#mN5fYqY#3|lDxTz1S$9s3kSQt^nIMbeRWun!Rp_hT~jck}tH zx8S9%Wc5r`TTc1Vul^G!{S`W)wDfcgC2oD2xkXMgAagH&2?&Dvuh=(Bnf$6L+zT7S z!+&5pdQnEFJ=Rre{=vB6*n);Z$Q2lFahDy*eSb(DRDx-l#%V^NNG?#OgB7cs^Hs2o z=AZ#s0@XM(x>%Q1^Mv{;?yA7Zu+d^=`_qMKPkPZV6BQpHHU4Eoz_ic~QD*S5+}aNm(jf!SnIrd(k`A?<)-BkUmtpehDh){d{xq( zvsEtkq8DTb;i7`i8;`S^YxybbdKtsFOGbJk7o&aA_5h5Q553(}s|$z>0PLqdpiR=6 zY9y8;d(dF)0RaRoVews`5R(Maa8PxxfG|MX`?IOJ#+5(0Fc_450CDVe3?G%-Ze&## zl!C${e|VirUzQP#vy}*kw|vehqVqjjyc%TlYSded$ILyOJqsWLcmy-!i!wjQ4Vt(A zVSU(sgpv_wZBCb5_z(6L-U<*D9+!v1m0AtUMWihPo4Pxa7;fn2yP4=K@4shYA?^?r zNEKa09CatKKrZeV^hVG zIv-nABRT#K;2-Ky7arN|OLSM0wrW}1vOkMq=JjQBy^Ri}9^aWT)tO`-)>jKk^0N}S zZdsV2h|mj^VpkhB$Rb?X!1Y4MH|`S`+r-dSCbO z%`KvB`qv$9{1vuA)X|!P%7L~9p#=C0!GzGeKc|&!vZ?59k4?+qB!5GoB_20fv`GFv zZv)Ctr}&ma_{oK3)E=;S!aMt@x_{8iPi)j|>8M`{wsQg!3Y5sMq*Wl{sxu}JMq>Rxm&mjsm#$TC6p5>FspGvQp;ylgpf4d-y^R2V#TN#hr zjnjvml4`k88nT?p{8f`h8gfiqX)D62j&>EA9_zsGzBrr(bv_(#iI7V=cpWQAzYH~yIW zfeOSW_~ma+?f+*_%tNp=3blBZV7&4AK~VYpsEW387iClMSynsNL}0^$b?BfB+YAYm z2&VSDvcC^LnT-5gm_-?43BpKP0IRbS5 z`Kymgtt#|;^C7(}z4%E(3N&3AXh_*CkR9>1{pInyCp{dWPZEf*T6zPXp!Wu=X2c`G z@-cTAiZ54Y8wAH)HQIbC&W;idgv)g%vwfef0Ja-%pa(Od{&{a#Qp2`0sso`}sVLNzdrLIZ^7Tw6RFapD-lF1R2M>R7QezXm6KkPED-e%x|5vnZ2LryBHq|q@d*P+hy1i)xU4Mcj1{zW@2Y7J$B68qv( z4uT0AXgi>!EQWiK&4h6MTnEeEEL6J-{T#9iQa8i~(gg1g=xj zX9qSa3;XN3oFvxt;;3k$UfD-dDi=z2a4FjQSQlx5mmN_$fUFzfcit}koGB-XsjwFw zDrJ>K0U*)gCN65Vb#b4s;ChvxOV)rY;=&3B3fcSl4^4%D^Saw7YDv#)ciI6|FfyJX zUvl=&&A7S?YC|^5teL55EH(c_50X_%PTm*5bJj`}UG+_}pX@&t}vtAUb z8af2Udo6WY+;*7~S{Gar$UpMn0O@sz84-;3o>sIIp<`p)5|CzBgP2x7z2oXFL3UC~ z$)#LgbE*j{o~`L+i%?9MT0qgD@omwef< zCktG*-6dh#kj343f)^TS-us!Uc~oNIKLG_!=+GNSEn|}05x$=Y4*W$73*RL`6il!c z`vfPM$rHj~9i}zw;yvK$GoZA~+PGX@82Fu3M22PJh?@2vXH;UZ7xx2(rC%E))Jwo~ zcjgPE`v&PpC~DF(*}-ruo@n!|_4G0Qx<==)s*>gQb2O@Ujd?T~6qDOQX3Snw9@AHM z;>r>3!iTm)%mn(M0xoryAB`+}%;a(g6~Ys+XpP{(13UAW^6hVd)~RGffTAh2`x_-8 zvR$q(rIk*$91dV1vsM1de}48TNC1=jup5F)ITIkM@ii?vDYn_tMbRPeHQocJkGr!a zV!kaU&oXO|`~brEd=chSj-qWIH%yzz9i1o-?>2P?zRrO!DWnNq;9Y|-J~fjyFsQq! zH+~9RaiPPmk1HPbR*V9%$ulu@P*U(QM5O%AFai5qhP|Ml16J1^U%Ytb14|@wzPjY{ z?LhX389HrAQpaoSOIABASJ4Q@p*axIu?;2x_7*f5jaD zq%DlhH5D<+HbKg8ruF;03;W@;y9M|Q$xrMC;@kUlZ^ ztid{&h=nPLlaJN`wbx92-9GLoW|_w{&5&YB)HjVjgY%hmGN@An^-Fz`7%ux|XH{!* z-GyeW)sxm`{#5=YT#@ap+dYLY@)D%knWwz=25o0ZqT)9e%0pJ}@$?Gg>|u+7qWz}l z-l9GljCF(}OLIIg-rTLGt4`8Iq^qak&UR?_sx_Jfs^zH!3x0$?5Rt8eK0QIke7~TYDXswsoAAs_Fem0eb+_1Ovp$smc9iIqB-Y2Jm-MTry^BPDbhGltb zHF;8?`SnF3V8Ho8niGi+iyHhuO@u=3KDB6LTTbP_u#=p4B)|gQG*tY1wL$8s+>qax zGbG}esEWIkN{KYc2ljTG$-r;%EC6#puOttxl%Dba#((T#}U!(uvN4g$ZT}9(vcO@{^R??xIRfFie*qC>t#( zcfi`?LI`^;o;#y*+2sEig2X2a#(a^T6ofbk$B)b&MlWwP1YD_5Mq%>(>Fca!_Q5x` zHC}_fx`@#eR%AZO;l(=6%7`N_={17s20Xb0>hx1i-=X@mO4%Y?9-aQ5A(RNWx9Gc zA^%X0Oa^YW1Pi2&P#jPsT3 zbB)8e!;9aaSX*zl5)YW&mA(OvGY$zFz4aJbkzVCG7G-Os_=efmSYe#`>+U$UW`yr| zq`dZFhspuQyPm>HG_NnN!{9*=JMU6A=3RYPY^qTzXk-Cq7U>?U75u)JFj_?iOSMQP z(LfSkno$Nvty|{aIiwf&1MQi_0!nPVln2sr{?9!KCHJq~Iu99jtt=3eazh;IXn%Lv z|F_4;N9Q6Mc7QcZ5bT5dKU%Nrm4mlEw&e6-GW$g3gVQZ&b%WbzUT+He%@_V4;K`q-C<( z(L;FH7z~{6Of5iG#dCB_L~sa!LZCeR{hD6!%7FfX|i8MYc^tCyy+fZ*nEbt7*8C;L&v#KDV2~3w4{QA?j{{re4LJBoi)YW7tcMe zD7UL~g6>D~A^0XK5gl|INlUq<$iRB`iOF&XJVci)*Sh_&dp6 zTTDxSY)=^&Hw3M`Ha803gu;!Ft|1r^hz&E2jP!yJ&-vRxzoB)|AED(lC*0n!>fG0B zn2xomV8xfvDLLXF-AM6>@sg0Tg*dFuZ`u>)Di3U=VzxdoRFEnJI0r-?hqg&%uf2Dv z=9nAb$*>rnaB+e&^&*g;;y+x;$7@!)RE`fF`S*%#fSf2-DyZ?WVLMc3*OV;- zKt{$i*hk*AJ||O6PJ8#Oh$(j$K;FM*&bPC1P;oMAx}TUDI;|u;L4uR5=-|Z2e_M@) zt6%P#XSMT2IV<|o`LPPMg;v~`h(+_t(1|d~m>!ZYWO!dpRyi22y$zajcN^w{o>w9V z7x|@MQREi`3}_E|4Eu!tT=-=NE^VtSsK+uZ$Tg|FMs!aYV{NJj=3S-(C?=>z8hkZ? zzOFqn7}EGrJjeY%5t3}==Msb9I7`)Q5>|Sn&}jsIz;jD7Kl^8L+K*>1!k~|REKN}W zERW^N;>**{MZ9ZQY64Zrow_QUjJlxE~f!m|m*ag5}Wt}E~DO7p|T|bmY z#fWY7R}nr}37e&>YqHV*ts<}kt1N@_Y2C;Yq+mHvvQ&%dcI{1ZerVHY?$ngE0DDN5 zbBCL7@5yzSLC!!rARL8n_dt=4{p!|I(=gJFSCu_&8LvG*%y-wbOX@3lj=2N9r9uso zBnCi?m8{78R{Ts8IJjOeSBcL|?1tcbIVSKkwv8E!^t)73q(@zs<8kM?EURe~_m5Qok5o!p2Sj1v#SQ#1kZris9f z`I?2_SD5g_yiFw88e9QZfHX!hNzkQ@L_p$0<4$doDfR)Q=#ahfLt{T%fk2To$?yAS zkKQAsg7{aT+voJv5;`YLI6R7@`+c!s#&7*gka`@lb0@Q1Z0kwj-yCM{P{Sz?R7B3e zs++B9?4_%=SG=kld*)7u%`tGc=DwzEGN(ff3C$dZpFM~{ zNl<%p5Cq{2X<<^301r2so+&N~Ch+iBhb0f({QdJ)qMy>d;lLmJ$=g5}Wh8Fh`8B+n z#ifK=q}(`3%It4BJrF5Bd$|hQGDW|pnHi;C6U6TFYhGGVsXCM3TR#TxeD^LDL#gn>2ABBU zVFWHqP8Q;@eRIe|h%bO9CJFZDMR&Z5NmQ>So}&75lr^D;s&G`q!~D)VR{-@(Ed6B} z-6&2`Teg$qlOfONYUGzqMD^jM>0s?R{{WUwS?2-|4hk<#*h3+TIqpHRZHvCeyyLD` zin!`vG&e7$8VZ}S3l>%y+8~4jzh#}7RQnDXT3sYfp>YIzb0K@%M7)xY-2oWOPuO|1 zMEZM^-hfa2m|U4uP*W#|@F=Si+a6Qz^QD5dSWHIpCxh5FvcDkneNC#rqr>tfV7Xar z+3Yg}PJdH6Df|#_&sr-rK~uw0=CA!j1I2%OR~RjM^GZE}7Oxk^q~Qu(LNm?KCYyi6 zdz z00^6*iy(|x`bvcI5`_;}=hK>b^#bN(uFH~?OHuFV>)!vE!sVC`FfboBcs9Oa^%oYJ z%a{`LFZGbrVpY7uGh)l4*s?PEQOZogDD+|jg=|oO5HqrEG7f4YXZ?y3zn0SZ#w&I zX_Al+n9xxO^VN((A51W2Pbw$v-;|_CUHo!4^-a!TGLtV_E#0NZOw_#l&gAeJ?NZG{7&{Zxi%(0DnE&WRkiVn$e4*r((_0vTkw7 zVTT`-y3JimzN3p#P!$Zf&{y!{=+L7(6U{DRsP61)K}7IyMuA~acQC`YO+wK>Ar{d& zVA+pHqZp-s9>yV149m>D1*qK?#tf+Q*1h8QmtAL;00y@73$MWs+cA1#kyO+UXd|&7 zbbV$#nz@qyu{Kp-yXlQMD^Yiab17r;{7~mCirYtfPi)jiQ#hUEg z@d9(%GQy-1(jCj{htBr`nC5egOm4m1{~wS7?-Jss&Kpk09+bN*xjZC5wa^8fkABp- zpMb6^netg-@jE!Er}%mDK&@a_VFfX!t`VVcAE5E(QoqTr!abE~1n&_C2-wNKU3hli z-k%D_uHh!ZA^NqK=j<{kCN}}JJ;ZjixY{&BSB0feq5I_Wcp~6=4PW`p*imEDF!F0;;DhS`jz3a`|1a9 zNwSlATj%DST-SJww(l)9ST&YoKO&iC@}hdNY2?oyk?X=fE$4YxRAN67%Yx^9q>hcL zA5Mm_MDB{>$xrH-F zbh#PG*1UsPfBZQ0#VO=Ckk$IKNo*JpzB|F3G-%}xgCQLZNz9j`zv=G8`8Jdq-bj>k z{aD;Ak4~r)+p4-fF@d0}UQyUB6~bt=YsdK8Q>Jx|e>FqFO0m(z*yKRVRj_sDxV>jl zdzX8TkBMWrFSt=w#_S*Ph;0-|R92~VIwqBllCawRA{uXlw$FUiJ3reL9c5%jt+x9& zI325$PyaZzUwrboe{B-SLvn{wT&N7z6Ym?{xJ$>k&lnkkkaFg~`GTuNQJpJU#5FIp z^qeXo9BZIT!`FP*D|>8&{WbEcu=|k{Fds2uhB@X^mV- zTfP#-6mUFJ2OBoqfJgdd94yWmVlP?{EMTyKi^E{!$Xy^$Ppi^meb0i)(;!fRx+9(9 zHXoipc9QT5u1E)PIvs0%QhMWs8d#T*sI=va72FR}8Xs zq2=K`7NA=rm`LQU73KCaoOSZg?iGgk@#3XX)!wV;n^ANSM7PMeSknWk*$@s3u>eiC zQral7u##g0$#n~uqC{tG`S+-D7^GElD@=Snlqi^sM9Af-#yK4%@4!A&MOJZH~R;7C- zBnL~VAT2&r!=Qgbog(8g`XTAm8&i~0#wn=@g43oaU=@VDm)mP|pVsp+HiH(L(FTd* z8`T3YOrvPn98nHCSXs-cGQs`*YtETtw9Wsj+N?hai=#9eyo+Ux2dVM+0jLt*Rk&*T zyk`}gHF7oJn1tYq=2?d2Qv=U@fFpg#>4v+a=$IjmatOTEtvp4%x8jZX!3(^n068G9 z)2pUOZrHs0?*fGqtQ?VPRF>>Q0#z-30pKZ;cApM~n3rXb2R~!)QTI$q)Q9_;K_&c4 z4te4IJpW4raz-{k=(Z(>W828mZ>?-2pT=yti^gkOYhs>&;nt)#+DoEVMH%@tgq^*8K1;KT)5PAh*QtxC|oDtR4(0}8bvjWVFd=Y zt&wRY%{F_@ON(0;a`RSU`|EF3e8hN?H6AS+q zi`C8AA^?`&(yCXIT-Dnd<`46us;WEv9*7M_BTJepUNT0&7GzM=7+eE(9ypP(7kKi% zXuGpF`4&;XxW5%ci6+5@rDS}_q8Wt4T;g$K^fiOTu-2ar4({kYy6{W-A&e|)T^m(Se93fO_K*}#ByC_e+(ad+t3!(4sF zZWyo~yvaL&yXaUB_HJ*?l@m>%L}D1$Vmzl)TM7{)UyhNhej9Ykcu4J0KH%FL*Eldn zGCZlb1OkN3w7H;ZV5=b*Le{Y>Zu<3+Vb${vt>zuaDH;EWstQ-NhYn(V>K$Z3JE#`S3s&d+cxS54Bq6E%X$}84@626# z<>Gust??a+$HIyIMuQ0=X`q95!7ti!U2!cP&$fjf*KK>SocePLZ3@Pz^@}ntnbYlp zj4XrD-GIYTzo4QJ)Pm95D5cprbe(;aIrYlvpt0zDm>CZK<<{Mu1q&Gi-?*%w5&(lT zI&f3{JH%*Wc4Hq=K;-}v8NIy4sP%hd%rAl3h5IRAot8i7Sx0FK;Pi60CI@*^Zfimb zvfDd{ynA66T+;1rYmWx z=`?CHo&Rsbt}e)8D@`2|HFf)|t$H2*MFbjrruNlfg41Phvd@tS`(kd^k$qgH4<7H$ za@#Yy$LB|~bbR^=ynF#XPzv>s6_Y$ds%v3jPsPd)pEk09(`=hjVaFp4r_-0lsbZM? z)}3EbFEJ9bN=yXYElPv^pk4!Qf_uzjxaLWaI(0QSJ`)C<>(0h=Mr zetG`P>pFER>|2|q>^aHY^ZD8dt(DL-XuQ_^(xlo^-K&`8ix!PB3WV7_q9r5X5PZ+C z&u)I8D!`=}-@moy=VBxhKg^(uiY==tDrdZ1NZn}DEN@v-mjiN*JONg@CSc}) z8VZ37YyuQON)(TU54Cn}I(59H)LdlfA>71x107Q8Ehfcp903Imjw^ub8B7bioS_@+ zs_^TzWZNN~FEnuJPB|YGO2ixP3Zav#3;jMD)9$BbP8A(af_!f7=VqCK`XlZQlD7q@ zO6+9Rs*o)$Mno;udMDqN(M|NBwnn@0ab%5

B5#(`6pCcc~Ng0H`zX;g<&?*Qu-* zYXAJ8g<6{Ke99j4qi~tyOHj#c4WGN>PSUNu1jXR=!u)qu*$IBcU%O=|QX227hLVf1 zPhAv?0!e0F4E<&tSo9m~e1({1v8Swt;Wg&0B9~<%-Hj&)fVLe>JdmIQ6!J)kMQ%2t zy=u7>oU}2JF+ku6OPn(PL!Z*4d>NP6gGnbI-E)ACd~sFYI~v;2`C$X1O+Ee2;eAYj z&@vMc4sEKV_kNp798dLgF54GpRntta|2?m9PH*J~^rEWAKhAodb!Lgi*r-L%>XelZ zPJ@w)z2l7`1dLQu%H*%cf$rv3=-2%^8i{M`8)uIxIOh5*kEQpYtP!01hOwGJQnbtiz9!H7{_q_q(EPlnS9h(-Wih*TY17>55$$8C}C>vxo4-o<3IK3%18H;2>Z^C)>WRaa2<=SL(!es&w zBU;$8;s;Az-E(mvqMnV*#6-~)s|69HPO#|l*ns{*rQMR|+yl?3FPQ24&mtOSsa}i3 zh^;?=>h$+qsatA3ciUoTMe>vT0uR#4S5h&W#zy^zxyp;jOveiKqU>A&9P>t=A&ZDv z+pa(Y&HUjv%ns#jfgh2roy{8zu(^%w!8}+Izk{S^F3t04!LqUSrU?2d`sqtW;&t;M z0c~6C|EHPSVy?v7y*_GP?@BkW0P1YK1x=lexnHYN@?g~?6uNxgkc1#1%(6*&DEqv_ zZl)Y>v2PoEHh?J~EtG8`dL?IaF;@{fdRiVe|7 z;!W+CLkGbpiAWI6+RoP#K4*paYpSHLFt2Tqu4}Z|dhh&lga`+hUbEB)78PNJnd03h zkN&)QOrzDu#Dwb<5>6Ie#qPlayGNmI=o^VOXsHVjO{<@uuEC|a=>fmM>xQv8;Z1AG z8@YuhOvlw!IN8FSsfJLz->Ij2D%S3bVYhZ~?RbY`V73FLQ1`#vhEcX6zPC&hZ92^G z`<*r+qmglJvf9UvaqU3)!`hyM(a+E}D-^tw7MXnZg0ILb3(l9=k+hf3z=u~vKBC#8I%n|>X%J46so#hQ5&0`N8|@rPhV(kh&XbYFk8>@3 zGBo>a?q8QTu$N|~c|gFRT}Usiq3SO+SKln2WmUZ(M3i<633fv(<8yf$SvwZtPrHW7 z@Rq^t8blBk*;)ijA45B_tgoauv=MCByZv$7$FMNXTBhwElR~yNvXY$W=Bi7flR0Y+ zy*sBd&^>AFU~SMGh*ohMxyl1eSF;^2h85Nt2D@a10b5809vGmANkb=B$H2kV8rCcW z^!YDZ?f*V0kRxJ?jmeD-w-+BC0tO-#08zinz`n2g4HJN5aqXY=+yaWV!fi0X53&9r zX7ET+&sSR2;n8>y_bwS)jPI@aF?2giy|0L#3?*a-bRT{P?2EOHF)?mVoA@NS#83Sy zYQso~-KAxxx+L)b%pOlrMybp49HJukRHZ`hozj)M#4Z4(gNcGYx2=o!p%Q^P7<{6a zi0XNR{bkUnrD=)0pklI-_nP!4Q7f;usNVd%ElgJ76X_69?;*Stog-0_a^+0xJRIda z%)3Wu@Lzf65q`ZMZnTE<9t40Rx~2{;G-6@}8z%swK7x>M8Ja|C?MDIvv9^y;5E?>kp&{>D+SvQOVQpr>S)C$ zJ$q^G`9J%bn^`VgQT7+B;KC?Lq)kWtr-{!LJMG~fgjY{h*a+tZ zMBz@*!|?-hu4;wV?$^CfLgmPCLaTG{Rb@_}6cP0noll&@d9yR43GvLASn{EWdS2())U}}> zmd1Kq?ehq82jS!UEr|;3REi)F2WEH~Zi2qZ@_i$h3ik}9O>pf%bBaOzGs+05x$dKu z%9|E5V%bUis-`W}hSCI-`of^>2A06H5C0?8&}oQe4HeMkH<_u)6bvK-B`&E+EshXH z`Dp7o+Sb1D@esWgOGJulqvhqE8bN4(P`Y%*a8FsmA6O8nB0>kYl`6 zs0|BnOflZu1WhYO=rU})AHU6>{9>`IK7%v2I^D2~v^klKoD9pSn5tS_H^hN3Ul4T< z$?;xSQ0~sZ7dpaU1|G9o)V9=h(qvPXiR@yn^JCLn#*srg64JFt8?@_XM^=P*d=NLx zj?o}yDhyX^JO8Xj*wCBVb@?Koj~BW|3<~d#0=9NhmO&cS5{4vn#-7AguQ+q^#A;hM zTJC#dUK9qkMpx$tUFkEvcnrjCA08ay_%9;wGkBJ*fRVIn&+SW!OnN=R={IX&=ihC7 zh)5^ATRN&EQ3dyOj=Os$@=uky-@})&iYvraffmoONu^bi2DQQGNN26f%f)XGcy%Yp z*U90yfU8bc&D%fW+G&i=>S>w#wB|=z%LS)}5HA2#>OXBn$|Fu_1%dR2oUXKe@?t6* zxxYt@!n)bXwBBG$!{&XI*pjGySd>Irg62+Jia#1o;ZBi-18^v+J`uagv=t$_1ImHX z$_R-#Jr;bL7Z|D-p%Te~Fm8xY*Ph_Y7WF3j<94QzFLRFftUPtJs6B(~Dp^sgsJ=`u z^d`oyZndYG$er&NbMG`P<-h=jih=$lP99rFzBt7V;xV0ciPkxe9p5@&&ouk_Vy*W8$nP;e-aW$%^bpBnq^) z*lI{5`{SR%9}-|cJB}L9{3ceE^U-P|hE`6YrD_=su6o-0wLuVfjNjn57_%nXWfVo} zHnDnDZkxhmgcNa$v??riN=0PjjdQi+s{>k~`OH*~)h7g&0(>_goU<#r@bz=W*a+q< zKQ|NwxcdMU)4$yN+^>Tm`xxIpQnNxrf2D!@Q1*u{Q&{PIeF}@V$c?JLW&stq-*p=k z6n!np_vDb5^)#5Z*5YUA!w9JP^wdwm9w^n|H-fM3d}xH+U#levY_g@X*51WXcIwFN zT1t+8LCh3fKDQ%Mzc(-#L;EY&CC7NRZ63DQ<+HnBK>d!6NUYldM*%KLqzX2g&vmQ_ zO=Ecl{S3G775o2fQ<0@?5PZ;rb;GjxB?9%_KPj{WE8(zQ7cRwQzo^iQQ_O~fFxY9?#}#Qr z!WCErEvlyFEV4lY$P-Xi+}UUEi?I!R>k$C3EH3#=C2#s$F1))W64` zY?`~ba{We$`y`>Ue}v?`|Jb6voRxL%h^5G7pI1ViNej5q=5LdK zyK}0D(N|;*r$_UPIC_g~sO^RmV9g8$|IN3uKLyaowz0mS3M83_BLWMuiD7q7IJ+A< z8D@|k=aq(jtp5g=0JScvEPYDAsZ<7#qN=6Qvp;W-Zb$z?uw8@Xl_JCoiwjis6ygQR zI>)6rwxCv82XvYXXr9iH5KAulnvH9Dw53;E=D#NANXiWd!;o?uIerN)=V@jxMm%V& zStSM@z6JGK1Eb@iKE7Y54b`0X>oH{V2V@^yofiGL=KNrlii3^t1^{J$`YS53`@m4s z+SlKft@+HABt??>HCEY_^g9@d*?USu-_-(@5JRQRg;P*I^+WxA=9t~7Gz?H|8XneS zJ&H=kMoU`KLw6+5#A$>xAhRhnK%r4Su3NV6+laL`RksZE$N*#dfF?)TwFDUliXooJTFU zb*ti+|05c9n%@!-oi$yOi4(>mUxWIt$5C$C#QeaqoTq8w$4prbHGlPat57pz%6TwK zS^t_8wr?UUrde$Q?5{O$EbRvLM(&$env;Ds7ur~UdC`vo1jR(krp{b^K84s{&0nL9 zP-Q^t%3Mi|-(`tnso@e(s4Wk~9M8d&8%OSS`v?lioAJ((HQs^sSV?e*PwzUi_-80z znxE=~lXQ*rNsx>;RX7{A*phqmNaD}9YB^@3d6#97?SN! zh9;4ZRgAY2`3pFy+ONp#y1+xzY4KFmVShIZ@?Wmw2io|k4ymY}#CEnJo1%j&|G|NF z(%34jmohkl`H8wO5&XWQf1kazDwY=Ha{Q0NBbBUH8=&@vP^cpg{Un zqk(nR&B!vD#3OK$86tE6IqW#s<*%slgAS?7uaQ=)j*klis=PtfXeN#MUyI{q8Q6Ln zC=~}0&c*|L{Ky+N1Z9ij=Dy1anA;5K53mLm(?0%Y;MNxyPzz1QIGrx<%1wk;{fn!M z?K)9eN{K>x&`U9GJAe0rwL*@M%ig=@4>{7PFAnhm@}goQP`U50GVV}N>RC;wiKlHS zcYoN#l9cz)qL|OZLkV&@a##7#xsW;aOa!`0i1L*q=%`w+pm+l9=1qeVy(M3o?%~LB zvU+c-&RLRn3($9>mu-G3gEyq=ZVX%%XoceE(PqvwVZdmvqMRD_RhWL;D6Gjtx-6el;)<+i(V5 z3E3L`d^^Vlnpl4*Z4HQhU_|u;@eZP~le!msSEg$)k%g_G6rykT9LZ&+4VfowoYdEy zR=XgL?Ah!yy#MY3B}=_$1np~D%2ptZa${Tfp%!^&^`ujMN%_bJBbWi=Hgfncs-trl8@(XLN_xV*4m z^eIpsYa?I$vBnDrb}ZE6{H|zDGDOBkgk4-&9T|>!8Lks|fFQufoTWQe?#ivYfy<(6 zZS5fZ&+oPwx>5;zxWJcaE%9KyFshuy$Kp;-|NED>IC6H9T^mp#Imhcacca4J<9RYe zbs25QE5V;m+dv9$+?4vg%M4Q*w32frHG=9!A;^aejoAfEm@lPFm-b$BedQ|Mra~~k z_ji+IdSn^E2cWTMm&y#92dVnia*y^~qJce*SAqZdXCQTbL`9HLA!2xMss#nlFT%ig z9PMkS@GuZL*9#vNpYkze z1oPB7CuRV5bO`5@n2x*s<;E1-9l9-Se)Um?c%$3T@^V_H;I3LiR_c4dzX%dHEQe=1 zs>bysGWkP;;3n5t79R$s+Ef^`(hT46Z2`dEP(6b_@}t5RFul9dfwptNL&G}6Z3HhN zWO0RkeuSLV?iUG>JmKNSxuIjd?#VxyBH({=YJm?KGfyeP%r#@a#YIP-7a%xp1}~C= zw@2oR`^o~|NiM?7?85y2!ka0Af(~w??vI)UGKv0+Cnu(R+kxzzDZBimgKfE)Rq+$|)JTjk}Y6L&7rp;x z{*UFJ&*0Jgl2!QVvgg1JU#H$qST(`V6+*x&30E&C$4?jP3zO@2cqivtDzIM5tV~dw z8zlAj2>$t0sa%cpV*BR{62vK;G`RZ}+0eirIh3DKx)4;SPY1*iSqc^Q+iIpb%D*%` zRAj;Y%;C9Ub@JexAt2~e589b}nY6~3L?h@UR7{sFqj0EfO_3ywjG zwLJOOs>=XpUMkt%`y=TQe>5Mgc}dD!SiozAEf$I6a6GK9RASxa8E#P%#Up!@)x^a| z=;BuP9_L(heR7}0M4I=e1Y=t>&_erS^o+Fl81(0xSOee~)$smx!LQ#W%fas!nW(B04)1w-N!15{ea{NMM zT~*9H%j)~*;8y51Nsf`7Pf)djIZ?9qUIS&hoK0`Spi>12Rj9sJ4Y6zy+CrkYSJ_Bt zaw})u)1($_sRD(3V}3n5Mh$Z-1IH&6Y7^ubE5N(1wFBLe}C+Sr7fu*UX%!LLPl79Yc^FKM13TQp3vt= zzi)F*KnqcQy@cn`1E7DFmHih_VfQ85n9DVx;NVGWr-Lqgm1QjW;M>iO+CW_-tqTPO z#JKV!NUFkZLKumsWrL&Lv+634wvl`n7Xxx2L8)Mw)&Ps_uAb>#C5lJ=s$g$lYGuE( zckW$v%h%tXla-~ki}p7c0(2;OWK{zacCCbb4s?!RX+)4i$eY+r7^#I_+Vl5xu)v?Y zVG~C=F$dyz8@AthAm;|g+TVF+g?Tyg{mn8^DwfA(W1>Jgg8@4W@bw@SBQ}I zR4-+k{{nCDE3RLZBFJUq?)75etrmE2ZN>aU-`m`>3sY31yR)C_X$Z@A@8i8=5km_` zki3aCnt|lf*8@$0r?x|$Pp&hDrih#evuz>S?f(Gah~1GuNWBuD+JLe_=QS3J%cnA@ z0={^P02KZ(;CdT0v#&qt4BcAi2-Sa4tq0Z_GRSgDmOzaPoFWgRjfWgI!{Qr;tFr1s zXUX8#j8i_5hL1&4tq6{69{{KS6+y> zfW4C>CAPC11gldnRh(=S1%?}DPEyq@Kuzd3S%33$HGiX}{N&MnY!B=Fu!^iyrDZ9s zc19;`_4^kf*Y~X1YcDQlMM`axWIA?L}{B%4xW40mlUYn zf?pqrTwPI~2VrB^XMdV7Q=<{%f!e4Wz8FiykV)iOJ`k^!?bXL&lB|uVtLW_0xRjHb zu!6pW{;d|8@GxirEfHO18Fd#+m_eU!q+C{EF z#s6D~Qh(rd_+&H*aGCcwANr=v!bBpm+1Hf88pJ~ve=f*^-LG59587YB6b9d|H?F9s zL$GvAb8()buckXL4@I1QL)8_rLgHusgIG$!QyDD6y+ctTiC+9uiz`$J?wp#a`8zfU zZb-urUpE5Ma9)f%R5u{BKvrVRya{nTEXIo-1;NYeCO?4fUq$#NBsc4;&aG?t^;2tn z&Z(lAZ5JDrWh7)92wb@~jVZC_o|+xFTToN5aJuqgo^L->)cT6a7JkrbO6%vllop?k zys$jk&gKcUVL(U8x z)I?zec$zAKMNCF8i4{H{H$Bh`IrFjd*LKEo61Js`8+xCqC*i`$zrIetm)TtmPy$8| zBN0a&cL{&N2@e8%vyKIexq)}oEz1vM$f*lApxHPoT}1v!VT%t z_W%Px{J*L;JX*O(piZ8U+uJg4mISxvJ_`ejpok%7;up5hv*rKpo^XPE-bd+&r}c0b zIV(-1g6l*Mk=NAZDsgE)9-Zbw4uWc|*2EFNOd`ZW<@4vY9F(0z9O$&xc<4}67QJ#^ zRIDZLisE5vmCf-6D zt!T&szN{0*MY!@#Oa+DmK33O5cqNsq$DP_#3r*-I@;ySeLLi5m>rS?)-}aDMf>$7{ zddKu3>z4LUuXD1h?v&=&}&|eeM`2 zfN8P7FB%-l^#2+G7(^1SCefyqUnwr4^$P=Y;^WB_^9*8 z%8;r4WpCsmsj%J3#noStRYhTV7Vts6So?26G(>MS1j5@CHk4I>f18$C{vh4+nxmQM zR_R%tOV8Ab9G9^1xR3iUV2DHmqzg1Sl55DmIaMKuw=uSq>jhuwTd1ErNpYF z0+3xuVAbKdRYZh{ic!s)z*cY2k7&x|a(M#Z4fl$qonV??1P>o+4kuym&4S&j(-`ld zW6A@Gy`yYBHSV{c1AX+XoS?L)j^1Y+0ovM3M50xbNjCsesZk+Mw$gv+%kQ*i_H12g zCH^eh&&r_1Wf+0)T>efFJ{}AY)vN^Y5Kv4L2=fe2ymIzplK8_Vf;g0gtm#XSF6+Dbt5SA zs2v!wXG^0`xKi$!vd|u4b_SWP;f)uYWkA(dd*&V00m3+S#3w8&To0r;$Zmx*-rQ`y z75pM4iEj*_Ge2M+rZ7Gb!|jxLK7|Y-+WZh#tBhor{W49=GxqsshsO8)ohtMSr(OXIqmuGfJ*y$jvS{RqlGBsJBJc#|`8olm z)Zrnk%ZkFnct0rPpZRMd_BD(1X&xyKbV4=gEfx&26*^~OzP(Sdb=(2L7uH)P!n7lU zRFNudla`ijhm&VwU|8nHBaXskbXml^At5JYDN!tj@F)&`prj!)sda>8#MVV(;h0isntSFArm~-#m8+i ziAt8zQ5aFm29|ZvHSOm4A0VmNT9P(rMeen?_ZnE4y!(R+QLyf0K)iM^62smCjL8vI zc*~6=nCJ~B$q1ZNoX_+uZm`ohUbYOUy?f=)4#6(5wZfn^R<_52y=NR2&DF=<%rZKz zg^4cXDODQLM3lOE8u}+B0_|k!3U{F-EjcI^%`e9iEY4D)Np`@Fq*Fz4Q`DVBkixVW z5V2%?6t^;R_PXDn5n%@jNhY`j2 z>Yw)Qr%fJo>}R_LMMDt?G7sNJv67d&2g~|a9Sca zix!~&leEijyeb3A*n*HO)7yp}y7i9YXA2&#Y2g^lnMRr{AiZlIGY-uT(t%#D4mKNC zMI9UvSn8k6QUr(?j(AaKM%0ZTH71v-(1CK}BGA4>T~gXJEh|e2*iRG_j%Snc9R|Vl zq+6@F^x(_f`n5wt6DNUVaU*c3lO19AEp5L?*Wn?(jGJ;sedj6G;M9b4AFSJ8>TV8$ z_o4>cgv8h*TT{wVuoP#Bn|j&nL^h4ww09w~`uT1UoLx;*Wb~W&3pT5!NuxI7c)M?N9zc78iH1a0H867!U3rOD9|{V=wK240M33dS|usr!bXPn3(m4u)XHKUm|CnVA#Ze zSvx~JI9BMBwr%D_FNW#kQj)dz>@r+tS5bJ|P%k#F*Y7WI!?5oUT}MDNcKl?7pwb0kc#9~qT6d%C<~6--2E!YyzU3+ zzjr0{Z8OscB?;ocM8c?IXXxCI!T_|UqH}iWWs7%H z2!MnmKD4?89z^&MzV9JM68;hTdtGROE_ZtbcBwSKfqn8L??_*f;s|!a!_^_B`a=sl zgmTJbCi@e1H@Hn5v?*7AYf?K{m`JgQ(bKKFrZTqxmg7huCiDW1^GKcy`7(f;cfQ%L)~z8h(qU7&bkYyTHky`IVA#WWivY88YRE+l09 z-)CKEFGvPL5x^(z4$fz}T?@z|Q8(LVpv)elJXU%xexNu~PW*4>RM^fkW+`kvg}899 z8hxGVS!8G}*}B88Te`>$M!$HRqgs_18I?m?6yxMys28>PZ;-hB&@=+(BO31L5XJ+j7K9Jq+Z~*R>1JP{*)E4Q%l-{2!`AGk!Mo(_RVyA zErXr}Gzz)G%z-te46$I7xSnHk8>i&qA2Yg78ToR6lWYBNGLUCw94Aw5&T>NWL}Vam zG*~mTqcr!xU}*1%oTc{nfl-B0Sq~>G=8brWDXH(n;)3BhEgS*{B7U#U&tynv(b#MMQ@=YDd2kW&gGT zk$ukz*kp=X%2vLUNq6km6*jZWg_nsCt>f+j{gR-#a8eX7H9c#nb+!_P#+W@a547uI zg~y9a-kd){R39*XWLB#yPy6JhBkaE%EhQ@*4H9?ZyLQ$5G#Yb*K7B$2j&6hPjsm$W ze998G_|IyyI>*`5@8yrUvrg*m#%n6RckGTxGcmbC;RbQBX7wNRGWt$sa*>`!qd&qy z1P@5GE}EFZur6`5r<|gb54y`dj6xaK6kH%~h8ASAS9#lzPK&HX9ceNIPyT{0N3;jP0m!Bl|rHMI_UJ*`P7B7T$OK;y4wGl;IMn;Qqi#LG%%zn z_|B=gqW?Au8U^gRen1yha^`62mSB0D`*%G1`?WoV8l`0DsY^#8iyMk=nRX^@C<+pk zCN_e6+ogkpNSYqzZ6So815np}h0Q~)2Mq$NS;i9DNUk@Otx{Cx>%vK7!PogL987%7 ztL2sMYKR~c3>U7xR*-wUV%EUhP_m`S10x;5G8EWOpLPQu5^1J=D40`({iVk`0^Q6z zl-k-9(XjS@A;W}=B0rOSdPk{Jf(u0bigNZ|UcKaOS=c*VJ}oTr^?-|!#caE`Kv0b~ z9HEsap+aG{aOxTBI#{e=HzcbLn7e#2$W$g?RgA!le=ykQAZ0E7lZirC^GEUN@ofFl z6n&guu***^WYFxR_d#=x@cx6XToGeBpd&wIL$_Myt9XjjoXG7pv2}dEX!Q+KE_K*F zg2o8rc1m3MH_wXZvQ?Q(g|9NJJ>Nv7lk{Ao^}~)!>n~y&WG9bJNxaeTeQ1y z`BCM4<)SQ~Ps573JiFfmim zs&`LSX=iCXmgtrSzq?)JF5a4)XQQ3knD>!_-e1-!)X zne-jqiH6ONrDyXZsG05>#dLzMxrNk~oU<@}Mbs2*f$zb*3#tCP?V?$vdNE&oCqnd2 zbr2x|mh>RpZtZ)a%~P0xB;?-==PVxgm@;`2doaG+crYJ4Hks~r4#LrGAgVko`P{q9wadSY}i{Z;5HZ4zCr&J)~Niu-!hv_PCzZPap((a28M3 z{JquCSOxr6hT@kt-nz~KadRLLE2u{XkmiS9PX$1a)DDi212~0w)HC*}QnDyYnmo~t zWDiCkcJ?klHe=8>n+uauo0r6otaETQmsQPP4h}ekIXwQvYb>(k-|Ks9Q-MmlSu|-S zx3)U+5i61ua5ubD+hF`!tH?!0qw={RRU-nb2gMK0OqHTi=fdH9yD=w>oim^qs|c3! zY(q6Sjb(bQPb-bUb6)S!HGRNy(eM63law-w3ZO9Mu)kjM{Lu^_&)7dZp(NI5vM zU2~EQeSO`)qAC2rXwW`Dm$rpu+&jstv#P%nQm^g}sm2XK6&z$t0{a87#014#jF*HH zcWO$8hXiCd+IK1<#v7Rh5$i`@R|+O#EJGXe6K1c+=~GFo$*l7tRnb9?B1w}{?^(b) zXhi}|5k!bhRTm(zj~AMy&sBiVNkt&`uhDnSS!V{vJ_?|qDZ+9B?{N@Eu(rh0YtNei zcTq4Rs5mAnzguKM-&q~iz@w+Y{X!9_?qhIFZOVS3?A+O4?T5GvRV1w-QI68qx&m>g z2xIN*2&Dgu6frCWFpj2*#hMBIK2d`dd_8J<4390kb!*K3RJnK_me1N$iLa@oZ)rvL z?3kNrv?U-8`9#h>Oz~n2Y3!=&5 zrpHEjs6OXjS1f!gnnVgeq-NzpJ^YZZJ4{DKSI}XLJ-$bDlE6+n@f@8#=B9x2Q{)2{ z!Z#F~G315|3SlYl@^e9Y_g6jerzxk8ej%J}Zf_VqB$bhoc`zNmu}?h}a4M#faW9P4 zG(PPzx2Un6e*%Tq=r6Lxck4I2k)hVP@HA^i7Wf64uZfBP3c*5s`zWv(xc0M2FUjbC zWlqQBM`K)(&~uw(n+8J9(<6ybOA$E(AUzJYSI-ONYg21n*rO+QGJS zcmRVAdR5r`(uL1evGZUAc6@vfV}RvG%PIwjX0CpafGZ&hi5zlG;ApTs9IQp%u0eUI zyA7%h4P1@hZZ1&c+#f>PgrJE{0Ya3>`%A*`Ub+W!@8G>4>nkDJg61svOQJ36{Oo}?f62}%C*z77}n#9;w)?6&<-Nvz@!ZuUz%xx zr^CQ;^#<`j3>Y*GS`S!{b<@`(4idXgbv9Ay%z{^5a4N`|3|Ms{OpNLkzn?Yea9%-s zxQ(<^6krSxKqAko9WQ0%^~|cDvR7f%D&23rQ0!r1AN%yAQq5~etyNAW3o?BK1(O_z zKPG`;)q|-V?zoR9M4C=DWJzv054s%A)e?T4=c z1g|>!ql!k zHn~}PiT4I3K&q*L3fZRir5+5lg;-5d(`18`f>D@oV7x8{l?X924qMx%^};FRbs?c- zek1CT(p&PyBTNb{B*hhZQp~e!O!f3^Ngx?S9$yY-XlfD=k`2|c_SOq5{-M%USYjmBLX)i|%*r=W&Z^$JGh*(0kv#%7EuI zEW!a|^j;}U*!Kjh&ZrYZlya0D{!Ge}tmHD~da0+-fRA&DTm68gh>QsAQJ&61%pNOY zD&1eQ{Xm$HIr!%RzgX7PM2KG)`(1&o`>R(K%aDG}-|WAyRc6s6N~~N6A1CvGM=$2k zr<%imDhsK1A@bvEM+)c<$q;YfB5Q#7E~YNa>S2T zwU_1Rob&%Gt4?ht2fN!kiK~Tzl7?9#{;nJZDiQcKi%bTk;{xB6^Gg87ZZ%^~-p7O6 z^au(gx2Fk*aIz=LExnq{#7rCw=>AZ3ZpI6WekAvt$JCrQ=eMXV}?nhEfp9wHrGys$0u&sZ$~dE-0!ah%Gyq#P2(A1 zyly3ni`Rz-=@X<@P`-3p9R}`#^>3lx#DN#&qR79Q9>}oZEq+uBX-!XJ{v+4 zRqp$*t2Jj6e2&AuU6mDPR8!H4I4pR;Q7@I?#yZ8Mo2cI*8!C9_C&Js(PCbw>XoD>~ ze8^e44mFd0oK47`j@Kr#4zMAcU#@#5uawg}Vr9kq!q9V#?C6aQ!jJ7fB1tI`8eGWR zkkZ5VNvy&29+$h{sH?)qWlT-8uY~Hw;^z=X9koh4mm6(AiNvK_l6vHO>TtUOKBT6;aGBuWhIv6Y&s~?b0DNhzKaQ@3*?51lZ;s_)$&=m3a#{NZ5snnXK@<=6p!L*!R51^hW3p14?O3y=uKY}PJ*p2Edc zvD;##?a%Dh%4uunE#%wQ^0MV+hV$)eLL*mydlQ9tR zgl;2j!JrJK-h5Ise{HxX33Ug7#0=+{1qn7PY$WL<@KRoNrpm9houmi);E)%CW_`}X z71prJrcm=vTWMJ!w#2VWEKh~U_5wdvLP$H99o5ci0acfjnb=md{@0tM(Sa-~jOj{~ z<_|PPkJ&2y6eg)r4__ESb4IHPbo&60FS3c^faZ991<~b*s4N2I1LEZEof;8` zJPSq%cpMI!TgR)(aZ<=p=gA$+$0$yeiwOEW9r_<2PtGA{K3tIe0!(MK(Asb<)uau9 z7|P|sU76i!TywI7sia)qgeN7rW5nK4$a0t*FA3h`!jm4LDeLr@WI={j7SVCZFeDOE0rQSFtN-Uzw)Bq-=o|U;AA&Hmbt zM961{82T+Amc)SEa5a?u;vhkI@MCLD5F&9_Q^6})e|N2j_C^xC_UYXf4+Tsi1%fMI z4XAc{q+%=RVlYi5yY5xVKHy)V5jLAroo@86e)c_{j{y^_RhV`$nQ`M|#H~l#%s3P! z{pyw0BS$ry%C(8*7H(qBNQTgdvBzXvYe;z@vXmDt&>az&agXU(i_F9M+y!IGkg$nB z^K$-?R{wx}{P7x;VC7NkBBH>XpqlWUszh?Tzy_ZcfPV+5HV#(Y-JY0Si$fnSQ5N&k z8DjqktyIP80!|Ck2TZss0paA-))nE=H>eqVs~SE;wnfeO-hKv+@Y(}jXzBB6nmen!g>jlM30nuca-3#7j1`bZx>bw! ziKvU8KHjPZSJ!?mvt`wbZPo~1fIH^&CAGo4!wn!ve@ECQuxJ^8PN?9^hXkyzbdc6# zek&~PL%7W&0`R-G@BY(-D6Ty0PoyPY*XGP#262%0Y0i`)Y-WZ_1YeGt4Ue6+7dY|> zQ1{y3#s2syJj7_O>yJ`rkRiIoFfvv+TkIO$9~totRCr9f&s%~w&r}q#zJU4fcV@7` zV3DbzK*ynp_<)H4=-^AjcIhg`x%3WYMb5nY1;?7AiVzO~u5iQzPjYcfh$URR8RGra zd}#=T=qj>zWQGR4Us;HTsJh=45iYxJOD@Zo`AfBPN@k1AK12Q+nu^qCDhVht z9O9`T10H8;%}S#~4k|IjzSBqVRXJ9Z9qM!HkBKCpUb3$>pk%X0f=sn#Ql^K; z+>adCAjM%qJOHZ>uv0Wk{bDd~(}s`Kb^`xHp?6YjVlMP{uikWlQ>@MhWQM9|BGq!` za#3T~`s*-ry7wA}t5=4NLC46Upi-JBAj=_$n5`Jh%pxUoc_-ke@r^&SYLNW8cp_i&hlF%6Ra7U58NR9#q>Q0IK+w&qZ;TTIN&h~kmzTc`?%H^G;g;8@z z5mnb>$225K?>Vq(L>U;m>?xI zXlB1hyU}8(x=D%F2+gYo>BmX)5f1Qs?S<(~MLU37wVqt7tZl*LG3sbEuoAy>(mf$0 zT!(mAaqBo0AuZ}1b1zDc87h12bn58F+dW&c?XsQY15=pztdW*SHZ|NFb1{CYxV z#!~%R`;iVXua#6WKW?vXIXnHXve-8e?QoCEz0;0C_81 z3SFD7nNv6lc8iWv-4n^t`4vcb^ZICn_FR#TK4xv@4s9Dm)}uQzoo=t zwDK5nutChNAuOan1Y1-L)~n%b=~SvA{DA6FPA|ht=#5>QCGbVAJzg7+!|TU$yM1A< z=t`#r#^T9@fKBGW9u*CLre3w zfu`jSj&-`k9rdg|&&Lom>vI=99fPh%cM@^ts6cCd8DKH=F#r^n&F;)01!z}O_7QI+ zXEQMj?^?_+a64%heqF?TBuTd^$4O2|{SPu2vVHF?QnQn*Sq1cN<#7-gpAouWBG3*q z6_sVZGxsvv9o1>R#C!-y1ZZ|lCax_*aX*kStVh530A{jLI0J3nY!TBAKqMsn`2Klb z1dFL{`bwAm{W@nJ3_g-QFGnSRmzBeU z>PDs>W*Xn$%TZJaxsk=p#}giQfns?Vf?SA}k#AX7+Hk%9C`srSyl>NSCP4~r9`*cE zK`e)^VN~aLp0I};L<#>*qt%^x&~dS*h(u0nc9O^O5fY^a+&*06OOvhw>Yhdn>1|EQ z9GC8RKeV&+y* zQIiFg>Q*5$;fWl9S=X`=jQ;VQ$iNKd1QR*7b#>DvMvWeU&SK>zE&M|c1E4mIU`q?L z`xt>zNG$kIV-Z;>&4u7^d?=4KnZPh8e+g_}W?rdaF27xPn>~F%2ltwQ#lg$ zQs&?&@;ttIXkip>EBCvnQ3NcI=449pcJP_4FraO`n0hhUX25D$%ky{m^cnF)OfB|b zuoiT;={EIMMLX86j9JjY$EmVd!_K$T45MF^MtdkTKaq!g)*xGW_z1l>mSK8I#n*Yh z7>rreRN;JW#LHTqh<8{Bi?>@*nov-B3q7F0`ZZvZ@``glee1AnD$c?jhI!n2k5wry zZ+2q=Gf~vUVUUp3T7WG5*`YSLvN$)W7+y7|WX3{r|ybWS+Tn zX+Of{M!tM;qKnU`by(&r20y$mO_-b@Uq?hs30liQx$TAWN)Ru|e3I7&XZj`u18jH5 z$Uu?WpmuML`tal)Su~yMcVbscV8{!`-hDM9P*JG^+96T`0Mtfd_ zOh7bLnfGh+NVUC>8>e%EjFQ8GTh%p$lEf1C2!I1KdGYWZ2TBFns3a{+$6RZ~LjiIQ z%k%HIzg3aGJKvIhcjq#v#X~_sdXw}Yry7-& zsP=r*yOIo;{{QaJ-eg|AMd?FKq^ZIUkxlif%qBlT%iHz?ll#4kl6>awsUDR#Ji}p` z_fi_xJ9(O1Ip#Q`2`x42VJ*r)X)#uZ!CqV*ebFEr9xQYj9&RrDBM8|uMoE#C2&pM< zr=@-Egq)oQVNtETUC$A!3))E-mSpT6&?t;OuODBxlSJ;K9G_3TS6Bt$04*{mTJIyE zVOZ9P7;YrxxwNWs9l$8}UB5XHtY+zC{Y*%LQ36OTBQ##N8?Naz-ja6?7JOu61w4R$$65gmf+nQDi9>nQ5 zUE)VKFuy1jhmFCt^?c_Dq9gfh%7~m2ou7jOzkeCOyp%CACou4vo1LYcGmBaUJ0O*2 z4-zA5v?O1CNgtnh837pZs%Vr+oSfBZi;xlSa(Te?td*71i%8H6j?Nz8J{WJ-9_)tR z>qE{T)@zVmB2Cx6=++0TT(??W{%J4vLOI+BPIIpydfE z@H!mASKOS=NY9?t$M`nmibS|FC-j9+KO;A81_O| zE6%qGxXGTxA}lP&;%i7jiZ;KX{veb)foZI5rs;0evxmuyGK#?6Z((FIIxNjQ3cA}6 z*qy`!am1u}ODG!H`9WFLOzFkbrF-kXc@PPzFm1p3>{rQH~cf8EwTsQiSG@xeNy#P*@a75}Ljb)<0{!lE`6Kq{}C7 z{#Zyj^*eT+wfAuXOJ0szTd4*{J(7Z?1)E*k9!A9e=Fl;B+HqLakdrp!qm6AA20%b| z=>r6*iGnu6wKvxBa)Wu%OFG5U)t+Y-riVMv{n)(3UQTIhwi9-3uH`5E<97UDbndIl zm4cf_aN_}$h^0&d-euBm2L{~QYqIzbT5BV*4EgJF3BTQCKr*DPB6T?3@pk`8bkCQI zs3bj({{5OY(eo@&kt@2RQeC%A2&ghREXlDj-rX&s1Ps<6MJ9`(@Akf(`7X>A12^~{_@3^@U23<+JWIIccx_T6&lhoOg z5k*y2n%t!@pAvHLZp-`PW?{h+^fC%Vt}o@Rh|_jw9(@MaQY8Ao zOB>AxkE({k1HrO|sf_cuOg8^JjDWt(gLUO&Qql`r0MXo=CE9%rMr*5F`a)J5N{IaA zpSjz6E490q%qWgQ39ca{p|npY*iN>GTzvpHv0_O)#^E1RfF;`>n28=#P-`np{h^T( z+{#birWZPCzytCZ0Zw4Vve3Ah3S${xBt$GB#;PFPuBcKEQdUmwvEzGG_;vM2{+M6n zsm(KR3IGAc7^%}mOWHvlrfSWmvTHn(SM{{JVElg#E4_|Wr?w{6jE$?Ko?2o7HiV>e zf(Yek0Y#XFh`mawnX;R_jB%#0`jW>was0oYY#Cdbqf+5H5{_mO=k(vWyI3WOR%3bb z+pB&2i&GNbvs(UHTdw!1H0vNUe&p{_*K*J3RwxChaV3Bmgho7PyEtM$yOgwbaPRe+s5XMK{J}_VmIo@~* zj1p5mk8t0G9@;C=%z|YbrUP8Z*3s(@{2Y>E2fz?e*dqu{(;v@70L!X6*NW%0qq|u_ z`#*1j<@d*jA^{o;SVH&eN_jg%U|m6g@I;~Br_CZJ;c_I_S(ZuaNfS9ASMQN2w2E4? z^%U|D29KV%FongfN1|n*2lhP<`843-L+f1%U8*ENL^N9aQ-7ZfYiNvOs8%*stenDW zoVG{Ky6Yb!TI`~wo{0j^y*`Cwl&l}x1*&+SP*N;`mXa0~S8-SS(?lGO+4vzR>*1R@ zGKXMLz_{3X*%4J*+JI;E#$DBZbM&Xj4FoU^;sY9#^#OCv0y_9G?G;k*kzd%7K*4`< zC>rERT*f2Pa)C{1x}q@8r+wHkunA%ad1hBhwSeev}a+L&3=dS(p^6Ad& zrHj^O?$u0bxi$cem@-|6H|fyrQ*FGtz6UdY<7j7Gp9i_C=?@n2QPs?rL#AyrM~sL7 zB}2|{lG&a=I4~*N1jo^xfG4s{ik9{EFgR-$co;sd`5Y`g*?CT%Xcu2>ZMN}GvXM1F_{WS z0xgXxRW7q#plq*r3pQ3CU@@b1T%7Wx`2S((s!5HH)9q%T~$Y_sW&$xJV2`_<&rT)d;VYWIf(XI z+fG-4ZQx5!j~gv1?hHla%!bg+VG@c&s%J@&_p_1CtL>!-8JqAoMdv~g^&UbJs5CkY z?7YtP`kJ$*Oo#msb3)Va0_}49aNCYt*ZDO^>rtwB!dD?5!N2}Hzcuz*#2;YUYqanm zI&;e)#3Lxt$)go1JmT}3K0q`av~n!S;9r^paemG6afv z(=K003ZELmW)}b0CDxMGo&{eO_2gedPY0n-m6@uF>GG>;(Te!6mY|ExkZJ-{!GcGB zmC{zPQZxtdRC*M&MO2>nyljC|njtR`J$Ej)Iet+HXxN$-X(3eNl30acyK&ed*PH*; z_O9#XU_N&8K^X|43r8~UQNBek%3pf@E9`R7W-YPMbT5yJQrvL9+BhD6;XIg=pB8gn z9Wg#a-%1#=Q4kv2u90#<%el&1Vw1{LAn=%psbFY1RDR`3rNs7APMwZq?vS0)~uOjt`% z+bo^H?_9nUk?#2|UmPd-vyZ2}FI4=4kFSB6;6YLQie@;m23>~ZLk~#yIN3+v9jJHR za`#^#OmXR`^4N+9o!Cj#MsILk80r~jl45_|$Lj)qyL!P+9< zYn*!X7~g{2kS{~L?@=~zU5@W8#ifCm*`Rxff*tH2x!RrVd^?Rk?8E5S&_{=;OqIi1 zL^UY?B=uM!kb0ZQ@cY*+w;x*ICQF`Vz4eJz=L2VG{P$v;KtT3hXH8 zV2zumUNCS7RE3ELZ2logtOTo^WEA4ui_h)sC1G=~hH@`rwc@!raH=N~4tM$>taGyO z&|8<^d5idefj&@~8HVy}g+}Z+P#-nbv~I6L{KZlVbUzxqEET&rE-5sq7yg;+!*si5 z-SNv$%KYf%&DMkmh`ylbv>Z&x-7lvVy$w--hv6{nfv@qrL_P;$3aC&X6t0K9M;KR? z4RS+j_ZJkVXw$<7#)p1>^)mBPQuz8h1k$8Z6(Qji%(O~PyOQSKwxkv~pvZKqwkb+E zZ{!*Pghl(6*F=NuTcuT>w|#ChSNz}1O5i+G$pre7Ij=lNQ^@o^p_A(dd2i&)m-bOU zr})g|3gJ4d(Y!rdiU>tvB_|uF5n}o8VLAG@Yb!x<-|zF+E4Zc`vndT2X%)n|u;txM z0a8W5?z>BSB1>W{aPx@i)57Q#D+K-bq;1SQ?uBfuLMnm)owxG&9j4B(Em$7jLaWls z*I~PFEhg1&GzqUS8kH%Qvo2vJe1N5>FOX#QeA)!AFkw0B4hSh`bw>yYJ;n72Huls( zQaK$6%VQdrcMv%HwESkfDuh`K!bS1|`FyHCJsxXQPBtg8oKF+8nc5`UUat|*Txp!W zuYHUN#|q;Xnx8laG7wj`T2p#J=YLcQ8>?>IPuKY(BTe2gbS?7oz^SQ>=9VTn8E9CJ zk@4^G7X(6YST(k_<3U5bKY;me6i)44TYxN|H67%FbWU_NpTZ=v^X$W#nyyg%k^ZJ( zk$n0S#%spRiCXb8#}tbui39CtwBq#Ed=9i$&S{wU`z)T4~`PVG&?Wn`x??U}( zvcUiIknI{I2v+^4f#|QX0N8q~l~N5cK2~rNKtuf539;V))0Bl8(UDyE$hd9KF<8=c zF3{h#0;7DKf`&X4X}-^i1oZP6Q-{uc{dS`-)_Tfa3TSR@qR|xqjpUb9cF!vfcUlc@%@UrF(j5cIS|MeQ0#n z5Va6;lLP+$dhRU3C`&%?TCP?73E1qg?`mBt4Ki`+Ojl#u0ij>}oW&Y$^H&RAIeq)e zHm#|T0Nca}Q*hsGk9Q>Pl{b1)Q4tf;TQKeoy4R4LkRlt@Zqt^Ad|0>k#eh2KJgDlp z4R1?JvN!&h(hVOiQs}Q({@L4@j5dcgmnWrEl*%T;YtX_|&Q3Thq1pyUUVVw3cd!!d zbFfD~rLFN+SGv0`Mz%Np%JK>RX%{Ze?7tp8C;bQZJUG?CiX{x;694x#fFMtWAbMh3tL8Y z)lpF*mH)7r@7MrE`G}q2^B~qO<7hIdlA=j?VxP%Ak%EXfEc*S}?f#5!?mqLDucfFz z+~qfCz#TfUural3Bd|B{i7B7uGgB&|*{8Mn6A-iSQO@=a4Zih_m9#Ef%UO;UHYuJ( z);8m26uIK6f;OfaK{<+L`!lJ))Wz&laknP$Ns4t0@gFe>%a?zH^k2kN$HW5r-Tpwx z`L=<|80Mq+u$s_sguFd-=iMq6v9>yHBOtACa3*7=vh~`G_^!w`B7NcwS(8pVAnlYr8BvpbxMh{cku_+BeERh$MeQ^FQPZ|2(scq%yP+>1Fqw zGAN;AN%!V2zE;&8to);(B>50_Rr$k#==6KxIGL;s?XOR$YZ$riS;Hct)V4zP6TTgs z$5+%P-;lmV{xgG=SS)8dcMNvP-oYW`jw#6!+Wc6=6ma4u5AT-Ib=Fx z7pf9a5Hla5lS78Ms|ze1B`Q2@3bs58nyk`}v`(sjY{bHXoEwa>m%1HNp+%Yfgod`_L6A*VJ@W4Y2pp%Wkr*4J!z$ z`pKg{8HQvZ>W)?zH?$sXs8@908L7T;s&AQfx(KUpMWt9<~71A_)8*B8+B4Fe; zSx^3!nvdN@YEDOwxinOg_Z9C`01^a2LBE&YK@yG6arC}`yJ8UeclggLkV`r_aNiGX3JlU{4?feDA zZa^LHe=wyK-29dZC`Opk%t#ivn|3ky0MH;YWreh!)?&QMO!Req(XsqVw}OA5G*ngl zyNCBWEck=sV$jrXu%Hu?p`VB%SG33T++GMP7Ro-|;4#atyrmN_<-C6*#+SmnCj)CM z%u4kaxy8!iDo?IOc+>|-N8oZ<&XV27{<{@<9YH0%!^#Iq*~DG=X3Frqy@l3Cm0Y~^UUV}W7&l0l(2j)kA8g92+&=n+{ zlnhT^g80@@aw1lqRz3k}WjgEWyZl4)B0nI#jY@Muvy^tdorI~cM)(qKk15{j$)XU3 zxfl=KMbs!^#(UyAKojmIWd<08hG1Y;+a zAOxj4cQkg)H+3HUa3h?fiYuolGRvbZD|%WTHBbGrV(Buf-K<}%hBp=TpFeIgbc2ML&&7Bf726LG{CBn1cHE(JxnHP zJ|Blilg;q-v^?FZzUR}eK4;LSz1X~S1-QX80BlI2U3rsOCFj1YhFk85%Uv#0BJ-%= zk4*f1a_|p9w%aCWB3PZ9VaEUoK=!{aD|%^G!d?OEJ4^;HO|Mp^&U(-zA%%KKgZKJy zln(2&WL5;~r**pIm}fPQ*GiG#(8uNG6p{l3DY1yOAjC;mDf|ka1Y{qC={oUR?*pb{ zwy+t>@b>P`1HG?8P31+i|H+BRf8iw?57W0sJ)& z#<0DuPIn=3LIqEEMVLM|=U{+sHn$E%L08|~H%Q79?&NPV)M=aRWv3-L^p47nR0QjY=vqup zpPZSj=~hv@I&&$&YPujit=f*`qbt+Xjhh|mYJ{09bFj3m z!>Sorgck`Al%4}#&ZiNf%az0l0I+{gJ#Sx&@{cEMIk;VymmEW`sj=1m+2yaOd!4q0 z^faIFj58J`5@gKU4ZQww)CCgSP^8J&jbcz@I!rfMDbWMb$7wI?2RKkZiK3Adw7uTp z;np>R6JA?%Fvj5;W=Nh8lE`!>E9&d#7mc;nD8q*j>;RtsdGJMsg>9`~5g9%t-)zrM ztV=#Q;T&7?DV)#_mMjH|IGv77zF8~x-q38|Mh!?OG~iv^Be^|FZIjKa)AybU3k%E{ zg;npDaDw>js6JiwRM6W~lO>7JE^WM;$b@`q365!tR9sz|OPLn-ht&aAgr)W|6;1Hj zA4MLsDtXHGt!1)%wa9a=)f~$YL}*XiL}BJ&$ho+_VuY%OvMPUhM(_WRYs_(^$RMN* zf~>E9|97DUu4#ALwZAz(7P)$>b1abM1HGONl(JqchJIxAnoEW~W;{Iu^GDpf2E2||@dh5sC#d87ahAJf*ghzdq)@}eaOwD=fuz47g_DeP{@BuhKR zkj$3e%6g8RI4452A2mz|NmIb^N-qC>E>^GiVJwSwQ;<~dBgmtzQI zaZk(a9)D=%eGkFE|I(qwWmr*U%S`$aLiboDv%rXoHfAn?*P6neQ(N}-IxoHSbs4I1uO79@K zH=h7(XcP>Ht;ZF#dK<#cdq7&gR98+zgh6+ zIGdTugU!?KJ>&JhC%dSU7yxf?HRe9O-+IjEJHMT*(K8&#%dXsE=;jHz{^SAp^;4ur zqL5c^G$+dn#Q_`48b&T8DbA6KnJ-Eq-2{8`y411h9w#Da!syW_Fn;CSAQ)U^(IK37 z0d*`SZwLv14*CZa2Jk6y7+`ngJo3<+-VfW}y9INE*(EYaFK8!(Wn6{+2lAo+W3%^QpQ&@P1~Gsks^?jIK3r1d7AK6H{?s>pYzF9`VxU|m9QGAD`8l8my=egghdGfvUussxZ5S#vKg}M1Qzm}`hlBM5K-@DbmEdhNOY~R3Rd8N-O%7zpjQ3EE0q<|2;1;v8Lfetw%Q0AQ)8Nkk^!VW8#Z1j{%+28A zDWUp&T-EW*vJaqZ+sw=3yu<|nx@yGFGSV>H98m8(iq$Uh+?`30;D?;g(gIgvmn_GA zUEJN*Pz!FDM`S7ye}MP8{fPGI3a~1I$ynegJ#A3AziLmPLW+k(r_c#zQQIqVNe17w zY3EnECb;sd#~<=tGlG@blU!%;i6iEFXJ~phtXUhC*&+Xhd8f`C0fqZki32xIbGyq( zj$m)HlmLk-Gan0Z#`tfBvGxII2r5 z5gSEArkh}DK%ooBlW8-$Jmn(|)ZSxYMnxr#kW@_{Up+BAJV0XpzMdqeat5`aPA732 z9YvXqf&Y2Wd5=RT=z#S%&T|vrWug5cI{r`ZM5mS$RnPV3@pEfujE0Vh7REL#M2yEM z9A|wo3SL;{<RSGRwP!T~)Vt~9=#rqh< z#h&p=7izn`Lnlez^ZsV~84tz>RlxFM!BJD5q#Pmgj&b#h=g3Epk~zxT+@ z6fZp{lwJR8Rwpp*^DV~dw6(gXZa&r5@syih%BmWLx0jbYaaJLvic$7q3KhrbS2JlV zEbQXvLU>4}XlP}x|BZqCXyAgqW7#uzk!6cq+GHqmBzymmaFRj7n9`@oOCkMyfL6Mf zw&UCCWJLma9PhNV^6(`IeTi!5|%$0&qhGyv*h1e`8 zv3wfcD7ZS8M!qI?ji9pD2Z(u!YL*R`{EkxicgKks0dgU?09h=Q}lFIdrQ z5fhtEjxX;ZAu}%<9{D5P7b+x&R?ixpjmm`)k4VzqhuM9{@~tFc#9;6_FkkeQ;Bj>e z*X0lAU}8{))q(}lQ1_=^*EwXvx^=P8^NcAI&>>A$r}L-MQZLN}FBn=@-{P7S-cR?k zpQr3z47iub-6DG(NvnS>F~%5d*O^xuq-s514a%TnQ_&@JAaTM{vc6@aD(J!}G3)?G ztCUGUIF4f2c5Le;Fu+i;Hr#2f+EaIwL;E9^$BjyMX#BGKK(t+%hFcBp#mv8XXx7t< zs+B^CA_c*9Y96IP8nqYLrw(>}G9;HpcL#z2V+Le#`j^AL{nkhi$rJmX$sl9$ZfJ8- ztk&yJ_Ws-prN%P+{f2?Eht2#x%|AY*O0t@c2)H!BCf({rJ-x|I@1<{QOo~{Y`J!y3 z9WPbWtKHGf`Wnvo!*X~9)ROK_4B3=BV#*)1i_#I?UUU%${7_hR$US|n*_~t_owWNg zY0ml3i^-y(cWkkIuWKo$xnd$M_so%B2G4PjV|nS!19C5O%lU;!t_kxLgYN7*d|rf7y#1|00HfPC?R(+FEhFyrWJwF51PB9ijB|E~9ouf+B^|CNk=#UC{-<52KKv=>so zHD&;L7o~Pm(F9V5h4x3g9Zqa{6~-b~M_a#V7-@$C`lisvL?~8Wf9-uoq63@HGse~- zH!VP1zV_y5{pq(ZF|Xl#LFdwvCnWXf4wrGfLjNA5XIyRE^dDF%h#-5LaCaf_E%YDI-aVmc)9h&CVu~N*8-16AKQ0 zjQqsGgf!9+!6`fZ9fMbg>-bW12EY;mF9K|mn|tnQZ$I?fLhqn00}12}_=DaG@OhVP zBY|<-5Mm#A;67Vu_jGa-%MT|^!`={Xr7}Z=nU&*N4nX);}OsfF(>PHLG13r@oxYxmD zG5S*FVBu7YUH~7_8IZbiN!)r$nhY+NrXkl4gg1{RETw^+>B01$RrtUc#-rA!;RvT| zj3h(-n6s2ODAAGK{%;7Ug;;Ge(=L zWjrO$fWNaM%!Kh2_O8VTojkZ?kDq7)nuC8}*v4Y(>BZ!g zscQ9!mUS0{w7Y3WMsR&W0_4AR6q1ZgH7Z*ZzRIT7=?$DA6Ck9ouw>AAQz{44I2qM(PuT zHiVjFx?rwD)nLxc{nfBq4S_>#w2;i@*97_4IcGC`GYwc890EXm=4fIOrT`xbpEo-} z@0FFS;^_ru68@YSUoVA=S7#GRxGX6H!1dUDS2r+ZmG_AL#VPbExPvmmz0D@-8=Rgq z1e}JTpIG0SH?cUCJ_9t+mC_-L`i08eoxMNz7vD5vw%sDpEAvbQi-o6%?^T!MsanEm z4If!m*g(Ua{Z+Evg?Dn_bjKVnz0>yT%w5p_U8o@1Fzdt#?bEyU*IvEBse)XiLVYbK zkeHR}JCv5%INN0J z_Dx2*A*y2sYvKF&lcIRhvBtluAk_l4^$&VEIm5ki)|C2^S5MEFeSBe3ArWT{UVPN1 z#iuc*8JkTfMS5ZyQOKDMrllNW77WWsUeAcD`}YX$-S?tw9;-ZmhUd-*y@Mw{!77T5 zTx%={ zj~Y7ru6J!BXn|A0avmOan)dNcS?QWJ1de9}OwoP)Uq@W&L^!d+qP3Lo65U7k%7p+u z2JxRN%^ndmgd-{9(j7!{(Q}>P@Hp{P#*apK-aZ`SAl&s?M6T+Yg=+dQ+W-aG&oDVG zXc49o!KHN2G)a~Ii&n*B@Vy#DYyk>Hns?m$)|VLaBMhu+ci@N3rPJH!64!En679*;; z0rAoaP#m0|+`L7D(VP?NRr|+x&Z9Bnwe>yA^3o?oYu;aPvYRub_7Jv(O$3hHg&tIX z8I%ZmX~U>a%csDM;bm_AaimaBjH8@G6EAvnG4P8q2f(_f#8W-X>>cGk+NiM>f>j)| zRRzN4c`pos$B;bjj{Z%bW}7C#`rycpIdC%g|`UUnpdZ zp%zlRHx1p=fVemf5d~KblVy7s52p1sV)JK8qkXv+I2e54^+)tdIonrDX2*&5HgH`b zx;b*RCoc}l(nhz0I`{5JCd3_&NMuEIyhPQ7&-4fS1a(x!{c3VEt`D&ht&HDpSj3BjRF z<<7u~8~W%}+PnKzZg2HHY8HrZ7a$}`q#XaXx8I2UYLlSU5r#J;OydI~6oz%X|9(yU zG#?wf1ZN_#!4Qu|DtzgzYK#!3vo=uBpmnFf{HroP<2;vkQL z?umBXDg7eeJ}O{x@iy5m(|;yGODJg1tqjN?m8S?VLSvcX2MF{eD^`$1ts>Fooc^@2 zvJDU$^}-DuROJ5?HtRcqfF=b>EWOqU8&+MVdQwuC{;)x$y-&<>q!R_fmEhzw%?b9DQ}negc}Wd9gTqmS$S5aZ#o0M z`q?#Ip1N6_52SZ>B=*5^7t1R#rs>K@DpbYTEmdGOOQRM?c@5xA`TdkIVw|ZYVbrwf zuV`IEw^pH!4QxOK@+X*R7;_YUw-OIYd2~kI!`lT6JHSGi8kLs3A$hYq8;c4qAS*_* zbEQ^IB}ZLmu(d)eEhO4Jb5m)#JKC1Hc4y%b4$6Bx1IRS*D{D|;M zWOj%@b8<>g*JUdABJp}`Vx@N#lM|%X_;BmlwbdxHyVvL4S~oHa91Mnw=rO9>J{QDj z_0a;~SyY4kk`;cM%@6y)>MII6M;o~Prm#r-=nrp9BI8@%e&`O+3uyIx=O6-KI54u& zLQTM!tLb-bfz#5F(Q@V47YWMRWEFqyj~`j9PC1r1yMK+U38Ie4YQDMiWL57}HrE9m z(&}`L*Av94(P*G^No@!h;9)A$=0*TD+JhJYu4ttEJ!)y7EzrrNxuF7stLw-4FAFxL zOO(S0ub7_?UzsLznT5uSwQ=CBe?}`>V!c{Pc&!Wsg%uRHg41D_|rL9Gb&Gt^#Y&H8U?GjFu~5zegrTWMgJe6LdC4uLGYS#jYH zMP++6i*>%f-}iFrpVoAK1uI1tbCP;-h13j&Hty&`Y>6$x6G8n=@Y)kX5Jb1P+ZyVW z(;uN<{pC>wP8yV5BWO_p(HOqZa%Uq&mM#gPhuJ0xY!BX3*L?0c^_d^Q2ZNa*z94lTg^~d9?_$;W5@sr@=Hd zg&C}t#uu4H=7#pzgs(QGB9lF;6Q&YYdcx07yp zS#HcHp1U!UaOB~jDW0l>FblUt=7l$P8W_L#39XK0EI;*4(SVoL;WZPyW*g1mAk1mV zzu>z!3TK<`zV9f3+*8iwrY3`Y=X4BXnvS^A-~={EBHRV*C9@y3WgY1 z+{3LkN`=tkWkSG%vQfD+tgT&y~pY~gIc&fJ3 z-4mu}*iclQ!FYdxq=b*0K=ejA;{1>(86BUwM9#8kL2LndSwqM0BjCCj4&xw*$WpPd zZ~Eg!79^ZtK7xe7m0fceQ5^Ul%I(MTcz|-+-lF&vg0-0VizV+dx8WKFvLtk6avN*p(4#` zm3$-`)-TS=zEaV138h)o7N=toT7zuQ?R^;SRb;~FiOxyjHuWd942jG+g^(DbDm>V$ z35hV%JGUAu6w~a4tM#*4zlCbXq`vZK3Nkz&RHSV<#f`0o5yZE!PV?%-mwcBg0eBqx zxqij#zWHBx#3INf1x=zA)Eq`C?2i46MJMo18Q%L9;Ts(pQ~$K|)@DfAbogV*&>FH8 zP{%dWNvuZJ$+f1bdJ}^Se8)mKS|Vut0MH}{>ESQP&tngjA4J9*u+buqM2_qk_c(pJ zv4419`ZJr~SOFQK{oIrZN^M^`vwXxRxKi$1HBV~_lNyN_0I0L~k7?Er9CIP>R<5Ugq2J%W^)JSl-eCc=g zh-`Q+fMl%US~t`j!9LaD&znXtA_hkdG@JaaqqN0WyF=%!MekXvv;h8y@$Fe7@1!Z4 zXoIx=E+jx=!a7m*({&|s+olZ2HvseEONN+&&iu(OHc0{h5}|}~=}H`eb>)?8_*|5z zv7{d_N&*WsrzJ&TgENm}i)`mB^X({;^*>1L8jC3k>F8%6nr(A9v#Oj4Cy8eL(g|Ea z9a^>Aw3Z~4LrUTr`}LE-_NZ@3P{#;g*agYL(dx=qt}_Ra_t)u^-F)y|rpY8RS+%5` zHHmZgMSeE735WIEdVb4iFd!coD7VM|OGJV*fULSTJ1_^Keml|ROn_F`gTz^*5Jo^>>32#zG z?SBOdBq)Il@ucXD{VV^(?=q7ysx`f!cTOP=r1}(dZHY35@#O*RXLxo#fuR@i_^rv6 z7j#s3+HA|JQJpkx8&bFFxX0t3T5I*+mY_1pgpbUWV$*yyRy`IYafH^p5GCR>a(y z5~T@>F^PBBC9tvv(zJn+SLpf}q)PVpy4S3aB~U&~hL$PYOz7OCUq~N`aOR>~YD`nM zD~qx@olD7U?58o>AiUOQcOe~+5q5^`warRn5O8xSV{&Ml!YCnp?j%iU*q@=(>vF(5 z-HqEENG@Sg*kl@1*Wt9Q#Ky#|mG1yXYznIF(l!;4M|VSpKq+Am+CMbtC9hQS)+POo z+4-L}h>J#iuSN%_&JJ_SF?yGQO3%W8YYC-#!&qc9Xqc)roxP^>oAPQ@*u^uO{$grg zI}^7A!{$&(Xl{L30k#CBzIj#OmeNe|+U{J)TH_G0#k}MF#l!^pEJ2Bi?)jAu>$ZaK zS(Gf@s=}3JSA;J9{3p|{gi3dI)Mx`DH_6yLiaz*ASG^7Ml_ZUps(GcGZb`17mRVE6 zy!p53zpn?A(tBHN+`&3MG9qyQELl5tJ_?+e>kYcgdKpWfj^`NfHf$*n-W(x!+Qfo# zmgs)kHh86Q(igT1R=1onoXzl%{#E4n%l!fKG;@BtJheaUqNbvtaa}z;@*}`?n%U)o^PSv1j`MJCvM`9S5)$5A{UAfK?V{w*Fi*p{WUE%AF|n&AU7Xp> z@OOWd9hkp2YK3NQ*NbxrZRZ&|k0X}Difrd*voNT>f~3Hiv`!2E>bSRMNc&5?gUya_ zck`mREE|{Kin#Qob|JJzq#wTZePOxM8w1S<)iR5~7`9V=)N33gc_M0T^d!V&*^=Gf z{{wUKtF>`>C~@f>Ru%X%rbv&`Rlw8rkMxC4XeD(g6e=Ra3^r$=LD{BX&HuPoT5*wibSehMm)DGe-tEYI? z;6a*x2H_3rBTz73KSwZ^eVi7Z;5yyS`L55Ty473*NTg*8MAuy2Wb;RAmJ{5;SA#3) zD|-hjeAVg2w6P36Hva`O>r8s@sWr+?`VjNRXn*-qv$xw=&jJfjJ9iA(mLEJJNG|FX z7~t-(SF`^)fb!tYB+0qt`=dr$$u+&2s|}|D6K8+?jKy)O z621JA%izAxn(xblwZZtgpgmVm)j;9DYxO%j+s91$&cUvW{SvnSJX=9xn7LI;FV8nWpPDwUOX)9FiZ^yCz$Ep5n}a zreQnn^^SUe93Q$vfbZqnjTIOdg6bCe@Sj;Ligy8P`~-p5BdKWfPiflUMEYA2e`fdF znQ>2fDsSZWk8Qvm6Zq+^-n9sP&a|spno$Bi>?*Xp5ck9yHl#YL6wyvg> z@GL5hI55F(PW5+$zEBwD57uRJiY1@cMv-j5@PGdcgF5DrubKldG+>l-dh$4mUwJ=_ z6azqY7@vJ){l#%y*mC6x7s?B&)5M)rUL~);VI52wstkgq4mJAo~)7@A} zffQb-+j5?Q<-Mm+ho*0f+Gz(#0xnlFUr0NwcS_Er^G8r)O+2w{D-Y z(8cc>M-a=h2Dsr2bu+YnuCZ55am5)w{R8cLi`>9Y8N4x1Gy~nWAY+WDQf#J^{kG8K zEXa#N6JN6dHR1R|KBOhxRx#yVxUNlp@~zXg-tD`2gB`eMSVJyuR0|tm-KFc4>mZPw z%MgNL5!pA-@sV6HYgtSU1PcxisVK64iT_1{E%3S`Vfp~(d8{f|^)`+dh=L?m!#)u) z9a6dY@$VHLhDe?j7u4Vq)S)W4G4WFaV)$_gl+i!hs15d+osB2l7N<&(9L1>jF-18X zwVn0ouwb>Vy)^og$&aGF_|%*f`Xunv(5WN$F!fwkn~dU)d;&=D zSG7QVH-*z+Mrh4u!vWwYIvC<>?HC&=10iGBIG2ZlNU6O`$A_kwvV^8c%)8MfH#>$B z5pS>f>5MG2(L-spuZ#)1qeS4ZjMQKdBkdb*{DGj~uKZA;kz+bxg9uHbx!X5%^a*nHp#&WkdM0t%01`#%mwcfPGuT ziVuVue>n6Jm>P3{Xce{798i2`i00o1O3I}mq0V3M*dy_t0|(Y|4!a*lMLNwD;&3aG zKr&Mu@X|7yu?iRn|CYC^#8*6qde|~2n#8xSob4)@4P!B^UbCO0AU<5*jE*zg=*?z= zVex=A{cGtrlB`$uNu&nRyV(9 zt2*XX3>>OJ3D{1HHWV=3&To2F(B@~KSC`63j39jWmo=h+dOiN)Y6*S zIt6|?*}-lH_u6L1Z@i6sx=PKQcom(J!*);^zR%Zod@VIUcA^5h7 zis;qq*`Mq!@eT)Shs-+#m4r>>OhBhNQ>PMM+o8DONh{K=rN0zWARp{DkAk*&>$qmJ?O->f$c+!Nc~=USo7HrP3xTxVoDa2b84*Hdy>Wj51ya*jlxB#;n(~gW@sq! zNTdJI==6NxIE6Wc!qjUShMFjmpcp1|v2kW#Zh7xA#*mQZMown?_+}o|Vt}JgCvycL zN_1?L6DO(UXM4~f4%}<9BN92M1q$o&lP$hM(KbI*Ec zU)KhVZUa`Is7W%nd(SQ1=SAW@9o>Pa;@0a++<5h%=Pcg?i2_WluQ{BEV&kr>1Y{I3 ze1zN>cW{f3zt;8Jk4|;f*zuJ^@=<9Ej}GDpN_5Ojl9fMtieG z)UhqtpCjLgTSUh0BpnrXuNcQ!Vm$Pu#G$VD0={t9<-kDlGaGXBkVR_h1mFJq@gyPV z??sp1JmvV=M(|}a1mWlA$I#@CqWr@fJ0*vjGk%0a=0i6~Q(2qEFVRm63Tn&DEzl>! zHfa08yl8rY3D8&scluLvE2N! zGN*xOLX)K(7!;@N6cX$~vu9vC!J1$n-$kXDOiTZ32hubF1tg)D$_W%B4Ek5(DwG4mt-@l1WaL)xoqNVsV{>C5yIQ4EyivuNz>}*>=3lY-Nh>%< zjsH!pA;O;4^nF7W_R4t)WktJW+!J3E2Ok;PiN)jz4%9!ZsJ17mJ~4MLgjM8=m{lL* z(NT;1yDJ8B-4<8g^NUX95jRii>&13 z)hwDTXG11)k^CH+_wQ0a!KaX$q+Dj0x$m=C+KpO46E6iVl3R)}rTILoyPmGsq;2(k zZkRR96E*)_Yhyay_n0xd_zb&-Z}CAQ1=P5|rxV`Tbp>{Tp?aSdPWLt(?FseIJ~|eW zq(D3$*6M1i{OI5Om5dc{q|Fm<8=0fft}iqdBuII1S!-tATe|bzwraAx=29I!-3W%~ zp^J8gWe4gi+b~J`8rF(LhtVe4%?2P66ccKg`!~*9N)WEb5z*FJmRd>#bA;ptpibjwf@vLz`Kg;q5j2pa&zmH&KHw1|McOzvZf44#_z zGJo64M2MGOr=jgdTaHp9>4_<%sHjZHqlX7Xkdw?Uj zcYG^Q|0l}@B)H{bkT<>Ki;KFFO3p1EdP~+fE|s#ya|#5?t~k^#C7ue)JDtO8l|$WQ zEi1DC$}t)kdO=*cGjZ0)Li8?>M~={d30x)V8q1iUb4n|LW7tcfMM@swFLJ_**WlYKX1Shrt6ywsE(LhZbC` zML@@I{C%+P$^dx3U}G$!a>=>1TPY;EZJK|2ZpLYq3ozcUuy_4AArq)-K zP^Z;ZVlI{W{`idlikjN{e5O6k4Hhga_9IO1>X47)#pX0xU8boSRfaV^iqns-io^#e zg$7XCuIIOElaL%E?+GkSGI{8>{_HV0$qxH#ABUh_gH70I`|z>m!5xM^@< zu;mhxD*J)36@n8(ru6XDV_Vg)mA zdcgCN2`Xx5sA3^f~Zop4%@WkMN`QC z!>56!0=}bZ1nlsPoQP57Dd?qX69b(=ss|SQ##rZYgEFrH{POvbFYmCk#+$ ztzL>Hqmf!&AVxu?CW?dDMJ_V#Z{{;PNIrL z5pQ<1ykB7Wbl8*|02rm{@MWV9?*yH;USRGeO^v$E6**dRG!eD9;lM=PKxd2SB(Td2 zPaW=4dI*VfR=D1w$#EVsJ^O&5u3e*n`}izo^0w$|X^@oY zuTgCoI)G7btiFMOkRgVDOVZ;!MsMGp{d{uhPAGE9WoV4=^h39Oi^ww09nt5Tkdrs) z>+&xm`Iys|m*nL{%5W^PF%iotnN8n$j)*0g@+No+blCAM5jsta`U)quOgI3QfV4VW z5@gE;QP5&m!cjCuSe2xGS3TD=M4!JF3|Bd0^6ahCs^8ZF;db27*u(KQsK!)SN_Sp+ z)RI1Ow$$37me=qPnSQ`_fx&)9rxh*~1U0v9Dij3qmf3Rz( z$_MGyuAZYhG}~AtSFf!H;HD1qGcwX?@eSaAW6O7zCo(>4&!8O+e2E|3yVfz-^--pF z3?mvDuVLLvUaHM@p?SHqN8{V>IIqGs z3W(3@9H+hg5XP2cgsL*BcRBD1*@lHbcDIgqZYe5MBs4!YdV@lPhH=O-#|dhX5XXJn z7PwLSk9So4-UhGizcXtKw~~cl5Ip`AVcKD&{Wtm7Y9>rk<`X>VI9Y`$sAJ=DOXZ!O zUF$Bn`ph5>JGs6k8s;TmyZ=2s3CtQpUS)q_&SW_~%(e_*wCUxcgR*KZ=f|PNhu2jC zzpdU!{Vq*tkVUK+0_YL&5%X~-b_{`2@3u;f?-%<2JLHok{YS4d2ILDr|FvAB9p;V( zacEQz{fe+|8Dw{l3MugIvl@ek!$-?z8R*JS`%W0n-q)%Q5lKh0Q~ccdGPX=S@|b34 zfFHTm8Y;vJ7=&USs1~trG6CuM_3i6K;NzE{OegZmpU@E?g#kh%ZjR9xU#3<(Xe%`; z9;Pmos$N&Tkqz*9A}oH&UDfM52P?b|sWnFz%SIW1rgffI~rvyr2wv`NkoVp5v9%1zBB*y20R9|c0CQ6uex(+HpYLQ4Es{v zoC6Hq$%bfABeGSQ)hl2Vd<-@Ms|>J0l(vDq7PgG;JY2N^XGlSKqMP)AG6``I2c=}3 zZI!Koeu`s@@?n+3-GEu&ToqJ$ByZA|O53sk(T7J)PIsNsH--l;Zk`%y;0j(XJ~kXw zqtEF+NRxDpxpaTCJcM>fX|~jnWy-pY3PuE&&Qi2hk9I>w(lBV>690YJei`)l&<#Lx zv>|1aa2`vK%Jo^UI0Fb9@$YZOqnx7lG1>|h!lIP6PI)A>dZh)cLps?0iEk=8R7xho z$6noNI4Uf%**SzjIXBZ0AI%DKre1Lw%;x7c(!oMpSdh7vZq9+d#|hZr-{l0WP!Nv{ zShCk|7o^yDt&wefi7rm=R`m+3;2n~4iz*fXgr|NYoUBrGAnG0OFb#9z3NT{&l92q*$ zVVPqvd4p|b$UEAPZ*B!s2ARsIGtp#09EOBIzG`%KGC+$13&YGzV|?a{Ep!>Q%~Zp%vuWRNBtL03EgWK|PY2g1*z+mupJ9E+eH(|wDE zq6?#R${v{xsSDS+!eZwv6ix{xIiHt4cpbQ>iA=8w6a60g_Z$60AzV_PS|%F=X(>y# zW!t$lmX7U;omm4iF32_WV>LierKj4fI4lCKi13{X(?7dru+QaAHSq7+X^}!~<0Lgw z_?&*Lp=(gI@bGRvr>=J~i;~$+B&uHWYQN%3wl_i&PY8_% z{p{nJzK&?vU4-@wr?^)oqoiG&#gBkuSXiSgD+8&9lk)A=C1|#r!X880sHL^YjUGCf zIuyGbIIa@TlIO!BWje;+1kFQ$qve`L8zt=(jqICbUdqZaZCxJ+ z5AvTiJ9{*SvgG$P@MTRNW)blux|5y+>jkld)XwdAwIuYRo7|RZkUr?QW{-dEYMW30Q zp2y@-V2WkxpK8W)%aju%d%nii9`h&u_@-JmkI3|%NvcX8aF`ZwtY@m#~M&qLu+8$MR3-yUT&=2_5>yM1@9&jQoffGizxLXL^Jtw!CB> zLNoET;NB+4IHs;_G!U~3Xy?|`&eYYHKyTm+qY`ye6qX5^rHSSJXV~+A!O&7qC37yU zbUZVo*kh2q=M>YG#is4;nRrG^M&~Zm)gB5wLhl?CcFbw&c77ojsmVp-ZhjJbE%8+!j=W6%!*@~I> zR}rL885kaZI@;*w;n%5Oxh90`gg2HDDlSxR?c@%79(An6Or5vA5kmcPWcicEPu|GV z({hYt@I>GB%V~Szl)v<%45>Zhlqm7|B~uy;))#qfe?tmg%gL681m~LtT-@QmfY`KG z>%Q*VtvJ`C<`3V~83Auvuj*!1ZfIQWx6t2E3$dIbd`W6cCeXc>Y*dUCH3|GG^xJv* zN2yrX*33yxjkLgkRlSHTu#f+;bJPdvHB!g(iWs8B8mmvhZdaGQqYncweSQn*M#<;^ zRxohP$)!NF6R^msV1jL2sJn1aFlYl2)1aiG4GFEm?iD0(=x}jI>S2WbHM+kz8R@~A z0eNCyLHru?ICi8D4!XR%T*fv?EKURa4i*CB*<4|B5XI&OJS6c)z>Z0bJ$tbUh+01J zWyK6(p4nXf+EIAl661m5VUVY~^;Yo?SqzAAE7pLa`Z)@3bjiMYAMMD&shAz|Umd8m zJLCyUd7uqw5AfACs2XP3P#{4-esqvde4nU>HY&EeUchZrGmbDv=uO|bJGGy?kA_p- z_uS9J(g`cH%0F`B{HZ~seDw$S1=({|6^^X2pAQHzrKg*OjtG2z6q@aZRnrGH!$_~I zMkru;4$llS{UCGy(TlN|2Y9LlDc!FCU|NW*6eu~|MF9%N;V zcxe|nxNN;GxKoNEUu`4i)@E>SQwV8c9I68en673G+HuLOFPg+_sIvij=aMkMAcQ+s zwQqd@a|Hs23iQ9BvO#Go|5yeHV+#~Et0XjILx6{w;!#&oy;A}@KAc2$&0EJ^hmU`d zKWmfYkzaHIxTa!DIp@ZzL7b-$?^msZYFk7M^rmj>cTx;x zGh>Kr>7OynV_!HQ#t#c8(Y+#s3?)IrLg@+dQ_*K#ts%e&4U%A`fdB(Q{J%|dMpvIA znKHUtlDPl3GB82jp;3%cOxZ6ycmA6y{q&JBIpWRBp{>@_7Cs3>29|bM^La%WTw*rT zv#69Tj(tO1z_}u6$n)9QBjidwqL6Sd!7(iNy;W-0^Yz0@k@;$C{0Jp9z3W^X0C7?c z*9}D7&_GAH{bB)lLHLez`})S;4*uJyCl4_91>fK5$xgcdUsVQum-;21gKxzWG>n_m z148(B+wCPW<{OLc#hv5!K_tIhif3JF=57zLnd;D2&u&G!Lu}v|SP!n;s!LlBCq7pWp~Vo5 zE?rVOpb3yWT(oS`NOYavz|F9>CVVsDcA1=%6mmwB>*EII<+5?wVwk1l;FD*uj2Y zRicbZhJK)`>So6~hJ|4B%lftr$XUX@NhRI7pJW^aZalA*`wHU%jFM`zxQL1XtyWW; zot_=f@ug==&e+(cM=n&x%(mZp=pP1iL4jwO({F;zT2X*Ju~_E-gHT98M>76ub&9UG zxd_V-;Ze^@jX2M^*12J6U{y*q4E>jCxOit8dB7I3A`l=CLEevL#b(LQ9bhMZeZ11B=L;kM;p5D|wf{2!YyXrp zugNlj3B{1uXm}nr>#)SCsueBh)83R(eQT~GF1=7i)R|N2jxpLlfQw_1`oD?UEul)G z&?KQ=M7d`+nYghZYE8sMX?g6GH(}^kN)+FbM69iIAl6@z0XV2xEpq{nGp3+AC7oC7 zoYmIJd|uuL5P<0fu_A{K811Kdi|Gy-$C$(`WJW-2re^=+6=*lIrRc$c$nF^84gTLgLx-xA=tPoBjQiYm7o; zRABXGr3ti=Rix!6nq0AcBx*mR{tP_SGz7J=m z{J8aIO12kgDC}=|>T0FcQc3^7U-iSNo5XrWuJ3=RM4dO7d@YD6irk)1`@I0rWaMw= zd0F|!;Jsn`6=6zH8l(R2ak#TqR2stYZQ?yGzf(tpG5^~khMgv9NzZ=fkmes1QfU`@ zU)w5w1Xxi6jViX$6-#;U=2YaW?9K%6tZz8*eCn47QK$UT68~a5>;!rn-+@6D>_=Gp zO5g|}a@X5Ijuc1HR3?-nbDu#g_e90%(=eZANdZ*IBf_Fm0{cy46z-{A^J#eW@J|O{ z%sP{!jfd`Z(O%Y1jp{1ag_^tR{(!oE;A;PZN1Zuw88rUB@EqmWrzjLKNNONBAFy^@>&XeQ`1mtK$P!_df@P6%u62O`jASYI+=1r z)KLt&4B`fNEoU+nFDV1`f|G(F`a4%rgWFp>skVD+7$c{oz-8J9Uk2-Asv?cWh1L}t zY*`lD?lW18d z1is^GD@DIWrUJXR$e70T8o_l4lsC@4$F)*snFL%<{Z%)6I4e`NSY@GZ_ipK;iI~RG zNRRDdIy`rrygD+QKsseBk=Nny;RR;mdKHICWDwLv7UsOmw|JV5-R%d&sze^oPnKJw z*wNjAnK=mko-;m20%&=T-o2D;h0#TY+Db55%eaek>O`)B4&jI2<28z6yAZ3`;owms9K~U2 zmYfpaXb+Krq(*Wd@{uK9?nJjPS8-xqKhsA%Xub0y$zp9yR``v}OQs-U-v?cLz{Z+7 zgO~GuSd}ZOPn$ZcG@&Tr1urGB2Z+Hz9Gkj;iO_!h-KjFg-o}lr31}o$@0^0a59m%? zPk0*;0~YbNyG5D69)Qw8zMZgFs5Et{FCeA+6qb9zU#{4HHH0vDW~K|JO^k-qeD`vd zu&UQB{d?~W|9N~Y2#%HKiDonJj-hwILlQX(`ul(?GTe z1DH5cp%Q4#a_kZ_aLDlvSH4()5@F$U$lS!s%$zUDwT!%sB3|($z zGs4rMPWa&QU@ZU!S)gxSO}seM(^Q$au(7 z`v&N)=(tCX5l<8}RFI#PEA{PiS^*~^PN@9r^Uk-Nz|c{U$nI?^SpMjSbPWMPh=H;* zUql-hgOdjVOsz42;~~$R=!ZGU1>)a~kBlzKQb9lFJ=Co*sbZ9JTJV6$L$T{(Xz2z! zaME>iGwTOwy@n$C zy}rm+Mk`~)tkM(5LIT#WdP3S4Ms-}tw`DLf-lG@4n3yB@AOnL;KbE)i;{!5~pn5EX z`+73x3d$m{wh4BYk}%;o@mO_rV*9vr!Km!_G3diQCl)jIbVL{AGy=rK%87ZW;DMk- zp%?w-YAFu4efyW!oN*sWFe^tZQjBl@m?~oqgu?K}7SYU!Vtkk_uZR%!&*m*m+RQ5| z`S5x6{+y;Azk|AAsuN8b-eqXM|Zuj0E_qf+7E-$S5HGyoFt6*t8EFG zMRgxbv%vU|lyARtOcPE;J@BqH#eS@}^ik3H)u#>RTDi{FxDGnpu#&W)V<_FTX?qoQ zVm(2{PRGRJ$HZT}_n$Jp+MW7f>MB};hs#BDt@Ri2dT$o6^cRD!=5*}N_v6$1{Y!`7 ztQNKXqw|5r8DNAxupkzET<#JJ0mWl0-Lz=B9>r*8GR*_5TgwYh_jzI}5M1bLuxY&) zrqlf$d1;h#?2891XfmMLQi|5USf<870678bOQvEOX>xF3$5WkmK zPPK66?+W9^2~t2y>9u7;AbQCXlOW?C&PGNM$*2c8_D{NvWzD`>ma#Mea2PoaRqjcEt<>m3AdF9 z3DHn|X+sSe2hRsW;eQgufqnuzX2t0`t-1KArMthZQ3$bG^;VTuNY+QE_j{=;Tq_sM zBs2QFefQuMohOENm$yZu(*uCxfHZPisLfxGpA(M#{afz6#8~C?cb`Aw^tx=)DOXw%s(Zc2vmOVHB>84rNJQ)@7EGE)pC2*TRV$ggsF-msXGJ~HPj-7Wf z3mdT(20;hd*siX4pQ{&njvf0bzMOitGumzwV>K}@Y^ZUgX>bbxx8_7=q7T_bL*ld6 zX9yW+NQ?wooz3Q-I2Ci1D6jyzPvFWjF9s5zg3y_hyvPTE;u7CMO2z0$b3gC47g4$t z3okZ->I)_!Aj}vV9wzFLHr62{pP7;&ptcd?6w_qy^`TTdOddejx}v;qgP8#tOZ>l& zlQOMFK);4J_R~V4{5f#8CcIqhdDJT3V1dhp0f6yspi>mAed*uaTgpLfTk+k}`LP$iaOHQ6B>PM;R*`b>Ryxz!zf8_Z@j*A~*{-JmJC-fNg zYC$gj!@5opaQgqndufSX>BLw56ZWdHtW(haMOs+u4V`Amdk9*Wer_fX*=5QP zxsF$WxJ9oP>WL|SPFlyxEg?8`xHj)kIm*X4YG(q}l*CYmNPH~!W?HKzPDVP;CbVo$ zLHKejmYDhY$XqiTk%Tdf+<<87Z^JVGWWSttBL&wzyh=hF}1V+7iVo{lN9S zYJ{Y0*Ek`p*iq`QL)1((MRjD$K#+aCkM1c1mYkb`-lAsiMAxU@CG7cByCD~6LZ<#N z>O?~k(c2kMi@Xi}HbpQ5k$>zds!&S-O#ueG{v!Of;DY8Jsbkq-C&z{Uz^h2#Eqrh; z6S}Po|E@)*I*nl7gV?X_i^A~*AKsL`-YzygK|Wa%onp|9uWoI2Z*yJ>(fN$_5hJO; zR&{y7hwu5M3BmzZA6nNCVRNtvOg9{5t`NcIgIX+o@>ytvKZ3_m_2bOBWC-^|YDfe{FmUW5(3)m4*eAbE<+sCBc$Z4Vp~ z+~86DYrK{HI#P&eCTb8$ZJ+_m)+-cEm$SL8iPFD$6gh_Jl>6HYruJYKDWcdZEd(4@ z*1T$e=>NkC>zxA=`y++yDCQfWD8r!)13{y1j*e| z#PUbvA@%g5@R34bdfK>w-X>XnM!bO5FCd{b)gSVN*z=uEfiWH(G>xk{8iu4CN~#%K z?B1P#W|J~aelPb~hfr-eK|lGTPu6GMoO8cGSRrs`1wu)*c;T^gzGXbaQ(?+Dd}Yne zb{Toxno;zZIUjD#DSv#-GC4ec`<&i5g+Fd2^hNH zaK16aL`ED_P~_5(F4F|OEfk**(VztwBUXl0*-_ z2%h^b7v<6wao6Sgz4024q$<_#O1H9#IK)(>L zm#OGB68{iBAuU3KlqA8t>(eHC)|y7i(9%G%TcEEnO&g)ahN5fT&PrfQOork<0A@%S zzU7>ay4Gn}4f{*~#b}QWbI$-*3n8DjHNiJ+I%W1@NLv3|D?>g1X*#-cxx|4D@0l3- z>6(Y`{mN=BpyJ26nL}nAY(Lr7T@%Y&;f*LvzkI8ms~rP_@b$You>K?dh05*CW+WS* zj8h335Vs2yT@evpQRs@Sp7jH7Xa8dj`${8yQB_`b!H*z*PTRs^T}z7c&RLkM4+F0> z{M6RHu^iw#ZKA}{e4sic{7<2q0yR*wsncBftCK2y2E2LWXpPF$=@|SzDnYw|>{vAj zXsJb(K)D0iPX(r%w(yyxzM3?+JXeb#9Q?E}<+G|gkV#U@x_Ie_fDYfxP8lRMLd;8j zB^PmGx(NZfn>`BY;Nc?^SKS*snH8>H)tipDc+vx&Mz@p)mtqdr1%?-ZRd~sn_>vbF zTx+QH?vt|-Cr^mjf@SYWU%DB;dUHg*#$q6(Ox~D=RGiLXfa`UM^Kt&y$5#&3IL?>6 zkRdb`V|(yT-lDxxeYg%FuC5Zg_V~1^(lz0$oY~g;Wu0LJe3O)Ur-Nonn(&lwi;Q=A zi5sUNBZg3nWsxEr?J621nL7BYmE|in$hu+`f9}7Z;}<|&xmfn#p-u@C!8v_rxYZ#&j!q8RTTHV|vf8(#QK3hk$*Mwh>?*3}yHKd<>kB$DZ}JYvKHs#edlT)V}B z(|%So85l8Stcm1{5%O2;OAw*CU5Ozm2jFC2b;u22{?H&>KW|cw?pkD9cgk^Y{}2v< zpUWLR-u9drViC1AQ**IvFg{u=zi(_RKFx-~26_2Q`W8i}0co8lOOH!iLpN6|v{z0H zwua-jDYE@_C#>F?{gY?3DJF5x>PPJE-Lk{W#Pen8S!G;8gZt5q+Hdd*<_^Gr-&WZ( zC6bKkqR72ZHS-=1qK=!lMNzvdi1NR^MJfG2w_O<5#7co`nqY*$Z=Ev4UE~|>QhU0| zbS%x}4bmNnu{lA4e@abS5XW=47I(pB;KV~HLBU&IzqNk37f7tt$CV64RIm8J#Rh~i zpg$YQ_?Mll3N#Eh5!RxR2fbJO<@cthJgzxI5Q^(BD$O69GIZ+4smv>v_I7|UOp*&5 zrA%8YN`bO~DLe0BTlIJ^PeF>oN_05*h6+89uIdrSRX;^47QEV&d9v>m+T-%(Jlr6Y z7hrl@q_y^qFE}n~korI^;1ym!7}Gsuu7h~UbVpQEO36_tzCd zykmxJzgPYI9UG9=Jvh?d!+lt_wEIarVqanhji5yRVz=p?Th`=0`q5(zeG}zmpV+Aa zQoxU831dT0bV!LvVcMA0_%N>Nk*|w^(Gb@zAPj1Y2A+Q6L~9xx``joV+*hpDWXn2; zaR0`_jdC@IH6q-_FA~;y`FG+O1JLy&$BgaWB~Ivm3W&{Pu&=0*V6SvF#-~0%LVw-@ zgxvfLQlT2NFL}^n&IeeELrxoxewq;ElQ<}?+MU&}%gGGGlYosq#x`H}vN8>(o<=d> z$dQ3QOW%|3wccAb#TO6&M?kp0oG7XmhJ`U=(jgkd-uZ(puyfVfY9ot`m}TR8rLKeN zQ%-|#WKDBu>6L`{qR8)y6mJx1ZZI&Unh`bkGYSKEXDN zPpDp_G^Azm9rhALP50m!G{ZKCQLp&%T4fmy6HV{VLRT3&R13;rRU@%%h~;57!!Yfb z{vS(XW__8E5WnMn-KViaq6@fNAX81gHrEe_znLJNJb(#hJP|0&zXA%RKbhc?`@iE(vgWY|O zLEW|o#nu@wN0W_+IZf*)48A6=euTKI`_n$P7f-GMD;okQ{DOu^&xbBXh=(m+A?%jm zzMJAg(wgco>nQvoxL;eCUD7HZ+?V0fiDJ4ikho^GRmQPFPqjh)<6&*7xh||YxAa4l zHp*v(e?X)I_$^|XhTQbxk2YdhE6!WFR>uI$yMU%`VFo$CUQ$9Ka^@};dD4=ohGQ!e zL%(A(FY-U#OG2U0oIG>0)yrH#M1ra=S?Y9DjUXz^ktMlu@4$blPIcct>RZ8|4AYJ@q|KLNTGT>(&NZtz z4G^pTg71^mpG6xHf^BOBM#IJLbK%+%+WV(PRd@n)+~x<>KpbGQy;%BYaO?gZ5-Nhdg;x` zq0VQc<>RYya;b}zoy`>%J+^-T^ROVU_0A0_qDKK_&%veJRv6KPUOg^a>Jh~x+HGsi z1QgO#=y6659D!3-X{ooSky{_n-aulO+zHV2{2bCE8`aw(kU{nZDuX03)MnKVjBg=6 zWL>FwQIkXj7wlgc%pn??R5>#SvnHp30Nd;V*t?skPsXrG0lBu3Ls|yuucPxb)`lm^ z?Guh_u1dlPQ_;^2jEgX8dgIyI*3)MvOfXnCVz$1F^Tvxxho1em%o;yU*+}vsx?6Lp zlCba5Y~|1@CxkMv1)R1wbS1(_X#z*^=j`mF%Symjia|c7<8Trf3?(<|1D#H@Y^fn- zNfU>B_mH1|`h$^Hl&uGsj7+NM@}tSYCc4Se&IffFK$iXqksSW5>8WVwn!7npF+D(p zIgdB~@*r1-wJ4@Uy5Ir%GmJx0HvT0M>I?GGo#w=zuDyT1zMc(q9-QafLfk#QZFXnV z?d&QDVmXk7>3GBv_GNG{0|By=q!F_DPx9E*>oUv4s;ZzPKLtST6}PQA>SKPQucUtQ ztZddF!5Ul=R^0fMH>5;kCN^9=NIH_r+@KJhKfh~hE%UwEfy%`ovy}?9NB<}eVsSI@ z&H+0_;>PL(%=Q!qxy#!M~jW!F6!c(RE+&CDj> zxQ#zZTAAb9m^v}jeOIa2a&^-StCVsGWfq+!d`Obm{DGStZq5pw>4^ zAv}AP2b=KKNXeTJ*UQaE0_U7rFuUaWKkjFJs-j z&5I6W%mGHge?yA{m(^x`>e!#@1f{rHa+*|yM-ttrnBh_dgnXJs4xW&(OVITE&8LlZ zQ?hNSG~t)Y>VvyO=6+iYQ`ZYXmQsrxw1=;kpZVkN`yhz?H2oS0Nsu(zdg@Eqy&Boch9ROqD;4Wb@Ol;i!Yd+(@HjPihP- z^xNAo0B;w)DqbOMeWL;8O}u29OCNj?N?Fa8B-09ir~BWCc4Dk}`;L8+XUd`_aOd=> z(b2N#{d|&Cb4S{<^)*)@*`o%Ahl~?nL(+>u0DrO7x`{>Qj=HU!9##a0S zj=Hq1bCq|Mn2&p3$tA2-*c5Ox^vX?AtlilgMD>{jUxhsjWP+jn)v1KfIiQuQ zvQ@C>1Yj874GKTwDMM>NLbY@*2oEc&3c22XAyw+jgEc?6Sa4k!yiM+f#FA-W9y{Y#09zR&)b*)`x^ERfYCenk^u6quF)wMF;uk(-7Ez&|fhx zJm?^md)?_U5w$n)3gY>Az)TRF9>*@tIFkNy#Arik5AWwkV(qo@(;@zCIjJ)(v%;d9 zO7xudMnQfL|7cOoY1%Y0OAp#vKfaz z#r#)Yzew*pj10;4i3*kqS|fI>BdbCyRck%H-}Yf36IdkjJ{Dzu$=aJv*Piq+!eEnk zh)L>n9^)PCrlLi*KjqFHSQI}0Cgl_ilaQSn z2I{Nu-rz8swSDXW0;Zn99V-*g3;Cn;w%{i|@sQrtxNqUDAMjr8x;W3q{wZxxs} z(FXoZtHw~(?}z@+4LN=Pb;nsxovpUsQL$4cut|4^v80(Jf#C*V${+WT&UbDru%=@p z!%fCE+`dmJ0%5M{Po2lWY!oqhT<| zd)n@V*AxcTZE4e1y{U?tB1u_S)+p?iMMsay+~!ZK2EheGffhgXBxskz$TllKQZ`?t z;=7X#l9OaynJZ)Qkhj{e_*7Xb9L>J#c}Oo=dD}g^$C&AeUwdjHXFeGLyo(7R+5Cgs zv85>4_W<^h=@*3KRWzxADLxUAD&9M>21wS)%S8WG2duUcuoK2!1Q@kUZwiz3!Z9r^ zQr=}QIIrQ#T5p!TCucBZSMZZ@99?h{UXF$6J4Y|`@WnN-L>->}2C&kDiNR~`7>hu*n zsO_EPjRY|BdLYkadJukM$?L!IMPT2#R3-+ChcgRYsXW|IUXEP@!pb*F<%|pRfT_)g zkmRiygCiXFHu(LF-g+o+b_F+YG)NhONTT%Q7E`#VN&T(uT&MW6xfQ6gjTIXk_XeS<*68tp((3ZCN$&7I#65>sQAWbNh44=;vazKI=+#7VQ;?Fz;RNtoK z4zm(O*qcZH7VwJ6-{4Ni_WRQOQlA!zr|B@snMQ}M*rgZYNm%GJ9iMe1X0-f)bsfO% zTTf(A-d+{+$^c#LzF+5aysTy|XWIb_C&t*hhIw1{P=oJD(&LYpbJW%`VW9v^bwWi> z%<0eHR|Ve%PVj5*QQe%fl=AJzoBs?`_<7W=!ENO;ip; zV5hqVz^jQe+Sbsb5Luj;*Ijx8oJ83cq0{rx1kHm9igP6LpS#L~$T(Z#D3yt&N2S|n z>EG*4cLlY8EbBX!N^v`O?)K+G7C#~`P$K8mVL`U!Zx&PU+pxtqoyQ9^_;BqN0f{H4 zb>>i*6T-~S|LpqLC-(Q*PhhAPFiiH?Ol0hm#8G>lCy>{oSXNTUAH^T-3>Pp%kL`+O zKI>W91um3f3nDk%4WnQ^x`%%*bkQ6!d~O@C_U`|4WJ1L%X-{ASgVT|FBH{me@V)t} z6=Od38T&-@@nz+n`I(U+)*ogaiX7yO)W$oAj0D+qJdY7&5F3lH5AKOLalW&mfsa2c zm!M=P&PCxoPxtl2D9rKD;#-*W#|;eb-oquH)4;9teSn~6#2}bqKv-JOVUxTLL;%(* zLZ&HqQl@LSu{Q~9%G@R(O@*s28Vau285Le}u_mCzx|xx!@eX_H2e{v}HHqulK5t3& z?W+iFE&t^*i7_X=B~FJF^GWn`KXq!d?|J-0jrf!t9ZlT8JH`9(NzkB}HVf)Uw;m~t zBL2e|Qr0QKL$w6BH5SFJ(YRU2`e^3Xm3?UtY)<`l2#-UK!-MnEg9pR#Mawh^vFyS; zAnrz;EIA~PYJ1+yWgA0l@o#G9Cm7A*LDz&dPLI#Xxk@AzgZ*woWTW1c{6vll9I&9) z+P}@nH19kS1VA{Pk@^ow6{Pzz{kP9udG?UeSg(s%g997z{=mT>%3~dEp2-@d@RmOp zQ#o$A5G`nE{QOZS@U zyA<{Ha5^YVm!(x7b=xmR%NivxZcqmEP-SWEG1*8plL+Oruv_o2^iP_UxV`g;JSKyS zAc>hB+c10-s{`NEMEw}iJ_g9F)$ox}! zj1_eh7H`f|rqxn?>ZLl8oF9i6T+0bZ!gS2DbI86^eZK#u3QWMbMay7Ng_FXPk2;ic<+wJAk8<^mARUC-{s-) zXsBt=wXl)26p8-hDB!zndh5Wd#pM3s=Ri?@X{GH)e;<-mv^N1oPlbWam4shn+k@q@ z&7ljvbmVPb-F*Gzt~nPF=zT*`LS_SEte|pjXhf6Mww(LpOSyIHq31-4ZVbVtk)xgC zgb4(Xvyc?bH-g?OEJGQU0@$LE5rcpJ(y1lWF<@}S6zaxwl7ga_|3Ujj+e{#w~*wU@k zIR#_cK9NHnrxd;bXTF@UTlW0E*i@a?XwN$sH)m5{r1#%&GrPA^aa&Rnzp8tP&&}h} z*4=xo=a8-afp|}mC9C!<4OWk1&*e=8%s;>jxlm)U7^e{^lm8LDn=rEgFLX%+v~P+K zc4pF1sNlI9fWeT7zrLB!4TdS>tE#HSSSOs-P%7Q&4+V2Djr1Rtgm{}$hAH?XYXQJKX{l13ut1G}GRxT|HEM|8XOvnaxWcYq1KqIX zcVTdSQ3|qN5rg(EY%?_ecW3;@&BxtvXoKE|3y1|_vA*5J*~)}9G~!EjJseLKvy{l@ z!8+|R2TYJ_Pwag%CNj;nm)^!noz3p-wXR)*K>pX>+3iqc#sr9GdR)8iitOo*TemON zQ5c~-j@^5q*A77RE*eqSK=fu>hpdo^#7E+M0)Hw|HVW-A=kRzbT@L%JLasy%JJ$sB(jKGH2B2$UmEwTV!%D345!xsTF&-MYFf%?Xp)Kt9>JiVsc zZO@`ffM_suk%8~>uO66x)_|@UC^b!G!;+{f8$?jhfhdy+J-=tLEtkwaaE6}8=M~~J z;WwLa=3IK30*p6~%PXoGMx=h2%tSI_bWnh5`iRSJT9)EV~J6Q5Lc|R9bd&c02}Z z3ctBW1!T^nH#4TOn8=yHLSbp{1r30!wbrbq5m`T$@naAmMpt3(x(XqdrAemQHjERe z8)HMzu68S^q#S*L&*^J`^A47VRkdr}35k5Rdh6#;5uoR+QgEBR%NTZ;&Ecg?U|RhK zPcz}YZORm{kYqB7lp;g^8f+{}#HK6_r~ZBT4w+|it9N5*F5F}fF`y=aOlU5&Np`WI z&cj5c4uX*;%lF-=Q_G#YyX$o3^D~`h*r<3Y4WqUh(A71$1V$cjK#urayAi=xTNW*m zp?Ll486o@wn5krUz@8i+u1V+u)Sc=`o6w)+i zjIbh24c!AGkZwBXU%@3!yN5L4~MZHbQFtO=U8s$ znG*ldW6DGYZ(pDgwuHy;l^Fa-?TKw#Oh7%v~tSzA|Xgp=4hiu5V9C61wV?O?V9vqxv?yN zn{bu@d%-U5e{FhDB2!$NHi;$H5zG;2 ziWpZSMrD5d!_C_-$e%3Vw_6~4-BL~+I~3%iM|hN#wn!1F?~~W$1$CA#{YKP&w7=QO z1i6@S8b~mC8-XYuW?|$Wux4qYv2S!#DF_o%yineEB>bQW$?s9UeH^q4*ab-`Q|#wk zsF#nGgBV*zZT&!K`!MtN7RcrSG!(L~`uMc9%n#FpuS@K0COKD#BHw4kdwe#uQF_us z*yCbllBN{u4-{t0-`Pxa@~%xe>zKa45z^)AK4pPUff4QIu^i4T<-CRv8QKI|r^ZaT z6om-}jD``=>@Oqp>*CN3kKHaR#hwrN|Jz@+ zbu+Y@tq}|^q99Qzf99%j*_@RRZ+xU9aNs3{6X%9u?=Cr)m}88 zouA6A_FblXK8BPwsCszXom0BCj!$9&FVMC?RG(GecY_2HVHj+ z;U;&pe&TZdW;dXL6iz1k_j#)HMczBTpCUedc1Hmo?c%HFf_eENcK9>)MRMHk89Ki= zVO|xR=1%EOLCzDcpc-Wk93<7_lYS!30xE7AtIB?|N|zU}=Q&SAz1~JtBolVPRv{|l zS(sEO^v(}7*cny2wrYlMHc9y^vB|2Zg znuz#I9xJ-=P2eoF%^NggQ2kt8lOU|Df-^HPiVm=tQN%cuLi#hE1j#!H9sg%>l{!TD#p0S!z7L$T09t;Kk!x3o%$HD{ib4n!jOxxxskzcIrlR6 z=&Oe33z02KQuUz43h%8lU6l4Em)6u%n~Ay*1;%zN14I?;>!Xnagu^=5K8(1xM6z)(h|#*;~Ev~e5mHZThL}}y}t(!3CqrK?1^=geeFaBC;rSGUp*S0?$VfBbc71j zut4orB_WzZ+PRDrh3)Ki_BM&p-VD?tTF#{CaOJtgJ9z`Ho?ArGzkbvU8VoEy@-bC} z;W4|wscSz>3Sdp#okZ^yEP1W^z068Te>S33&J`b91*be^2%~d|QzVD_F6YPLtX-Zd z3A!@lt!AsQ2e)zElJ@=Kn;fxNS6N+~q(Dz)$df3A3rCbUQuOvdwhU=r5Gj3?)Ua7) zWMe}}eU6HFNjxX0^r=+NG`MU)?wi_g_zxx=9)U_~&nV(h;c|ya8>6c(_>rVXP1U>c)8)+J*>dBjt9-nX4cns8-^g^_{N)%%Hvp*b1zwV;6=`W9 z5nUVt_9K|c)FVXk?}T0rh?W!0ZCbn|Ws8?Q5&ZqE(LiXnyXa)Wuvvt)avm`B_w496 zrl!=I!kLD*Mq!Do^-xEZ4s?0FoyOAnWQg7&0WKqj*+P3+E_WA2Q^mu!Rr|S|QAg!d zyA5C{gr5O4@fvx7;C3-bS_lmLHDe+^FR*jca~Cj-vYPK9cXklc%sJdpwvxh31zrS* zVctd}X3@ZdYx+`4kVehvmuK=Vr{pDFs&0(qN9U`WAt3=~W? zUU+;{sDX96v|mZ#@7F_s7$#61j6yG=o2#nSlO~D`E>Of*s*8KNz3;ERo7S2P6c`Vz zB|@raAoKgt1Y0Sw_asQLq!E#V6p>!NGm`P+EOHwzk5wIdoI|22Foq(N{#2E|)U>C3 zNp5;n;J6{^xZrc;XTCTn`%A=xK)D@>1;xU_mU9y^kxN8a*+2U0YQa@*<>}mLdiLrHE{B-y7DEB??h0$$LawSl5TDd3S#(L> zSV|@W3n2waxZ;$dLgM+Ri#SKMk6)LE{L|{gCx{0YHChNtx6&3tgO=$prgNIfQyayB zTi{z*hqqD)zNoKTlBXF>sQvn+cGLz^Sb|xkISxFqUbbqRN!~xaa#VSn@zQ`gZk$_d zsGosCZ3uNYR4xu50(9Lo!tkF zKmp-|%ONu2rTt40xh!-}>Ed6DHFXa3mF#x~9MPk0=9YM#H)l%raixF#%Az+?vTC|PIGDY6@P2O5JVXMl#LA@pA!(?KRKeQc^A`wh z)30A>^dHyQ37SPL#lB|$j$R{I3+Jvc7%OoG0kJte?y%haz#lUQQMyzRrQErnW`cH| zG5Y~gvhFXK3Mnxuutc+A0=db-QTlPyzgvs|4l~Ml0vz3D*s)Wp<)(ITX3wG%V0pLC zIyZ1`jQ zqD|5&W(9 zQ-!@%?-*aq?TgRGA!4sVARfm+xlD&rjyg+U+o?4fA;TRTFp>l{lpuzcDQ#rZGXxZo z?@QLO7W7^`n(8GVop>xYO6vgfASkrZODvy~Ln&?<$7g0`rAWS!lkmcld2Y?%p1pjj zs=vv|Jq$6hQOhEq-Oq}DBZSq3wV{C>yNF{!jP-?GjZa7ZLRwmG4NhGnIA$k-=a@2X zNoT!c9LQT2KMZDggH7G97m48nO^gPpg#i|*GKHG5lHuf7+dGO4$A(*`=sQ55m`-mF zUvMMsX@r&%2qx}v2)!`eGzD9nd`RhBI)@CYLx6ioeR**X-8Z_4KoJ-Ow2=Y$MWuUm zm{<@#9FpZ4ZaU$_n4X_3=;k&?)I$(D!+f|MaRuy(s$&vEZ>pq3Ck+z+Lz{PZt}ltb z(}JJejL8iC@Vv3b@#cSD%5s{LM1rPc%$nib2@JaG$HH@5YpsPfFE9+{3QLl>in=h1 zV|vp37p2v+5Hw9eukp;9D$)nR2|{9Ol%K@f6++er*DBw`uHZ_EKev)3EY28Tx8?Jl z_0Y;wkJ3Q5e3TOK#QTM{J{R}LGKLJ=7fe5tuUmPH>(Wtt_O(CaPDz=FXmxcM?~4|R z1G2rerUQLf>R8h3#DXw%RV#R7I?V_#$4rlBZ`h6HWzP{cRn5Qx)ILr-)l9)Rpea7y zH>wg>z!t@}hHw9kvp(81DClcr>QXELzW6uYVT>a!DceGUk3UjW4%h{112(iCEK8yXTonmYf)Z*EvyCL9*y#f7l3N*S>VO@8!_&BIu+Mpg#Bnhg#8k zWV{c8#!xj|F}v(*=eIrVJ%-1>7w|e={{bn2YnM4h1t2>Oy$d(We+CadUVmiXJ@)C8 zOe(YQ+aii5_?G_gzpp)c12wUru7D}c245+FBeKf=!;V)zaAi|)J~$ktq2Jc!qk(?G zn>v^TFMP!6uwB=|;o~Z&cW~ zg>6@ze8MU0%hpVY$VQUep$~nVWpZ|L}cQM|a!%F#gK8M0rWnV!s zs5S;bP-koz?{&(S9GgISjIiE=Lh}~GC=o=Kpnm_IqiVIY5Ved=j-R4Ev{|1^OrHHQ z&Zq%AFu^9iAP&h#?&yMX7#IP8xqaJwXxa8p;h{|qQhIXLHi)_im_lQ=L`e;%e52(W z`J<`#0r|1xHlA+h*(uK%qM8yVY6W%Te>7a#W!~9C)}-yjpJ|rUX_mfgjL+iU zn6<{`somwlDrlAbsmgLZ4NgSrD|ggxpH-a58&r!zJx<`cOx0 zx<-zQ<&(wHH*Aau3=%O%IilKzZp&jrzQZ`sCUG`>K`i=?Z;QqGa9cKoaW|?j8fPoX zIkm>q&eOsJ5~JB0Z{_RA^Pj|S35)6+NkcN^>MfKvlo#v9Wh6srSRiJ8CSA~3aj zJ)IC@iNSxS2d;S)mb-^d$dA^I9~?=Z05BOS6XPPUjEeuXI`VuO`p<{fT_8j{5LlvT z?G3}0w^!GiccE-LoD0PXd^IlqJfnW4v^^D*qAo7AzK)fam63VKM&+itjS<$!m16X5 znT~6K>{WyWncX!ITo%fa@+2uD4n(^nrOX6P?{lJaMlw0<5k55EIl;9dJ?($^a- zI$d?BwfBl4TP2`qp0ZSD_lkalGqo*X*HH5VRGMUPN?~HnL79Aoawj zaw&@Dc=k>+2yE?Fu$k~+K~9>s%u=`!{ycK`$RRp{(Jxi-m@+R1f8TCO$Nu?d1&l$^ zv|AbAN##)32$H%mIeaUSqN@Bsap2RM_T-t@np6FsKY^4#MW*+{TQ^8Njm4Hu#$By% zGP6|UK(eXIuR5vcp{xT58y%py5fW#Y>r9S?D&t8$sO+tKkghI?GE9`Txh}RUwACA~ zs~+M$-hWp~^~wpWwG-9V_2s@Ou)o0K10YZaijqTUM`x@$d61wG7sz=olwQ=W%qKQV zepL)@&r4RkqH+wl*Cs<>?g<$$Db(rPaBCQA@na@T_f{&j`!)BlJ;Fc$;cE812DYqo zMFdSQ9Guj*JW1BXW5v>xDjEM9{>PsJne1YPuvfd{lfJ!ig=Wm)Q`?he>z^^eFt`{0%xVzxhusidRiCTt10!W{ekheT zV3dqCgD%4ngZOv>`|h;ajZjjCx-AAbqI~76M*yS6s$#RJ9*dyTVVZ>$i>_mj{>JG^A3e@$LyD zS-$O|DsR_fab41V01xf8zpf(Pp2)HkUL}{d0|Rp4g}VPBSMVi%(Z#mwC{b>WwE`32 zRM#yefGo=4`m0~cpOKqIAsVdudXv(mW*;1>44yF)>?8BXOc{_Cp^7>bC9MdawF!-_ zK)wowQ0JQs);CN;zs3l#Q*40iRuf%8a<$mrc$K4_zqpIDk6t#8Wg;TW_*x>)aGQXE znu3Y}0DP6w9=MaAGK7QbJYS+dG<-H?SZ7~l_&68EJ{{1YVWzGjBT-UDbeNbWsqB%Z zObF{g%N@y`yQx6Jw0KoR#T4AH3}HJe)?RCbI2$)FQ3qjx3)7twF}@p_Cva0vE=G}ld1zV~PHPI{KSBq$*+PEC@`4+I z_e^LP$R%C5TLfMJQ@#CYj-9?q_T?Z$1RFC258IeC!{D>&{r0ZU$Yz@$^pZ=Ra~jOA z(dpi}ilSeqt_jfb#_F|SHx#9Mk8ohMAY?(@RAQV-uAY8=gL))6DSoFU1qqD6u5|KM zZ;cs}EBM9E0u(NmJW1y;FZvhC7oW>{xasJF*Sud8Z(0C@n{hhZv$)K>W>n8nW*WVW6b2qb9efp=3L_zb}j?f|Hc*D!V6 z%w5=9^9(vCCT!Bq%8Np0Q?(t>F5-&@o*I|fJpsiDd5X&s^cM!z9fp{!S691QlO7Ep z-i3POO%=2d{ob|foHml<*|@$W6Ly>L)l?>a@pQUDV@>4U!Fb5ypuqhY*@uC3*QOIX z$^sQ&hi33tX7ijrF~@!d_;DQXGsIehY8yskGKEYzhJ2V)@W`RWgX!OlIsY-aw*z8< zUcXxexwl3<8Ix<|rleiJP9En^qC;RRl=&-3v;Q6$!6^maFOW$jW}hv?6KAZalE*BV zH@Db|*e~N?9-(xJXS5qG?If055;X0RVr>BvbiAw*w zZ$ZzUN)7$}Wu6k>O_WHU!h2?Dg*@4k8oc7)m~I7WJSVuO`pp>R{Ar)YVNfI3YhWi{ z_iCV()IuLRhiwJb(~fFevT8vnRj(sO8Y{>Dz{Uff$sv1!7^@p;G>}M!4hER>$CD5F z2T}{s_=3xpN}uET&^md|aozP)I{eZ#^n`zSGmcxu150y8IHq;^LhtaoQBd~;bN|AJ z9CX~FtQdg)OD(4$@m4HzJ_S8Hg8dfCH+AICyb)`eS{rnnY~T9BPmsqlbM9(maiC~Q&v3S1iz%*)1GKz)z~^#Y;os! zVI8D*x+jkRp=dx1$mBit6AGCS0d@069i|~Q^Lx8xvk;CU{|a^L*AE030)Zu|v!sgs zqXp))J)2+WDY*OPQ%$9H+-u_(C)EPnr3At*r0FkEuZDu-FedxxXNa4K$-Zt}guji+ zN6Ye&^i&cm40*aA38RZ}55jo;_SY9)-mEaJX{Fb+QfSbY`Os9Z2}ZR`9pz!rt_>G` zqd!6WxHN=@sABPC`<}h5qM7pHNMv3bT(@bUwL%(|nfm7@J1>cDm2F2-huUD6aP*_y zf+TrS(K_&geF5VTLDu9H)cP!qk`yE>&@NbC6)W)@g3B8e8c5ppflqM9;}U^`|DWYu zoaB_qzFRUf1rfxEO-0O6(4ZSrjvt)3ZBIlxR?5~t{xrFY+E^7LTz{ePbd+rYMT829 zeam4@(-#>)M;s72osRqNEY&F{|H>;YV!<*#&-n!!2kAUsxzvn9locPQy!+qYssyIy zwjMx9;b`now#!1H(--o9hcN0yS>5FIML1#SjohQxBzAqe z>T4eHO)&kMa2{3!7Xo%eHkD*c`N%p;AirqJfL463Q%-bs#^6c^CI$8?+3fhv*p*sz z(QJD9x#r}6#sBS06JPE}j1Kk?1u&2CP2`Fw`fb6IBwMrR|ULHdXU9 zA>4)nAzUo}QL2n(s~qmCXg`v2HTesmBUrw=ST`OrpA@=&8*&u) z?6E2_qcNE9ec^%C#MSK9FWa-B&W~@I0{u_%f_OfrpW@ug#s@Xr_Q}w`g1B!j@ z7|~mGMHfL1c!%X|)-a$()Bu>8MW*Y}>vqI_ZxWN)$TkM%|w^*JlK zbmBy>h++i%O@HMn!|E#YkqR-os@6*%C!CqOQ zR6Ca5^f&yG_|6eHX14rkA2ha+r=)w*6a(@GzE|iN}hf_xTdREr-a8 zuF_e=!*j|)2Kv^>-{*m8DHTe&wI5JK&1X0L-45{_#}3Ecj+(RX<;N&c*?~QdtO8x* zuyAtmhyddsh~cXr1njDRp4qr@&tk$N^5Cc5Lr_9d=5nk;oiH%X7Lwha{*>(ghYx+6 zQ`XnVDnopw#RCx9e=_haX>#6}1}I<1DD1I=!hS~kF(WhPXDBlIOVbs|oC<=PaCoIJ zUWa69{RKZ

%)j{&0uxIYnqO!bvVLQ)lVe(fZPXk$7&TcOKgD#LM4yt<)K)l2Syb7 z8&vuG*Oq#Iyj@2MSuq=hYGn9r?iIR)o&&%?wAV4BpV0&knknduyo5UN4e~ZAohx$G z2!?_~1tcmUNs$D)wsyDYa5gbG9I_DZ{nlA9+1+uDNO;qFq@5-E{Gk0E%$ZbaX=D)I zt3v*{>KFVK%oK77=hNh4=+-O|GGttKrzzNU2U`IEW{9qi2rOC`DL2%(FoLeUuM>Gg zUy)IPc0>{=(%2e9hVXQ`n8H;VMmGc=arp4HUyvA`mi{`wdU#GrYHz}4AvxL+#V9Qe zdi*UWYPQ_RC-Lz8WBrx8#Q8?=6FG@x0gw$~su4V=(sWX*wV%CtQ!i)$w|}Hlp5&*g zNGZZY>FIq_x|& zn;eZ#?o--gkU{V7m%GqHkbAxM41<91hY!Z4$Znd^VMwY0FCd}Xpwrd+TJm%j5wO`H z`x?*I0Byw^XFvc@X%n03g(TdtI+3hl@16$jD<2@9d2JVh429mpaWXewL6ZBQesifG%Wsp6wsNid5h`B*(L0Y*>ouzsx#JHBm`Rzy&8q)zA4Khv0puZ##c zL+go(%^a;z{fFq||DNB(DQi{=B)+|V4$7GQPwZ}CNas3uhV_}Md&iFru%(<#^Ry}2 z$_lI_=MR0>+BP0+!#Sl>jvYbjOCK#X-Q?ip$?G*<{bU7*kXMi&@Rr?x)LuR6xwX(O z{XW87Y&IwQoo)r=FYV%&k#3Ucbwyn&P^$@?aS=ZuAL#6ROeR;v^;JQV8+7#arAcUe ztayD}#Gj&Y9r;!B^{_-$usPW<`R>kZ@Dbq!jRRfMgxgUa%(~@{g9HoUj-H0jqn#hx zZo^j~LBsUI*<=&BfUf_j5TL`lXmmlHH8(c^(%;PL?-jUO4;tVMonixTLt%oE|*Nu{wkRLIzfRe&eUCA=*&fL8SVOB$BE(cETbZ*VB|pLcqg zS@4+YzKiE=Xw34A&i1nXJe5j3t@t+D1tQrd)X_nrfyY1bz_X}gQ`ZhQi9Uo8smNnk z(9HCwsPDJQ(3?vtk&07I?w9=-^}P5MUf$s0XTC1yQ*xe964NyGC%9&lWY&%fZgJ6pcb5n;I3g2g! z70g@VZ~r^G~@WGo{bqo7{Rp~{PWx84GG#T@(ZyyOhA-~VfDxk;(e>^+U$;4 zOm#yiG>Gl6q-^Xssr228@OH~QNs)=(g3)Riq{u$SQr0t$ZIs)1=(9X#H($V0gb0gt z2ehi%*<5m7ZCHO9^`TkHL&WF{P)8&rW#Y0m?|6N7*KKdo#R+>qO{bctb#YmwUeAsF zCB1A0 zPYdlT^3S!f&}!X?C9u`p&=IWBXSr;9chAOatc_V2ZA`trVgQBA^$xXh+EY*Q20(o* z0S66$N33G`7e;mM0>LX^HdE8dxJt|KBDBdduQY42Tp7;9@&zewk~(abR+!7)E4>wP zQiVFJ7F{vIhD4rVfu8~ZHQFp|x!5*+8~Tt&1--m^9q0AyklapAmI(Q_LGnz^<1a&o zgM2(=ytKA$v#uFXcaKzg@rCR7)d#8^qbe5jBD4bQ4&&vcVR=V;Gido*oB<=a6*q zhFy*MORMB>>4nn^PS-@!#mb-K{fbApk-fZal#q;r8va-#wDs7@3S!0Pss#&L&Y{>l z8IE%Ot4(nQagVI3EEq`LkB!eo2L$@=N{nqjx$_Fc#&Z!M^cM3$5}9|TSLEuB6`%#& zZSLDwn9*?-#26$}CbL)#EIfYv{Xg1#?j}@HIy$Js0^ex}8#KtjJ90w5BcPEcMHDIk z6+JNe?S)J-_g>ob)Q41MmnDV_wcNm4FEwnbGDxr|X^%|#@>ROwj?nr=OEj5pnQC1{ zs;G_2GvW8ajdkoH2VH0T<#`~fr)yqvs&MNcGvPPb^?misfKVYW_kSJW%H(~g5(MTx zB-tg9MBl7@{{}Hy?<39kb-t+g-VGvnmAq9P&aez4_lyU&zY7ITQ(HSW77i2f9`1sk znY4PU;3q*WHhC?q9nP?9yOEF#l4m5}BO+ssWF9DwF5~Z>eGfD^gZi5DLuq1#GNvLI zl_C)Qa6#2Q)`4v<%&w26U!Oagedi^IY_{O*gk$PlBN0mCMOYZYs0OhBL_oX0lhEsi zvNr)x;jBc?;fck3>48Jp)A-=Hq{U(po9S8k$qxPZV$zVw6Fw-(RLJ{3vL1{Ki8EBEQ=V-zl>>^U1jgr@&5e*=6lg*<24k$-dm zrqD1IE3;3GMAwQE0+Hz~Q&)+@wneOBA9mLI!l zWcY24vO`E3TmB9}vUy_#yBn5kqNZ^0^~$z6ZjLeyY-n)P$X|)i2HJqh{bREbh)sZS z7Jndnha$d_QejSr(G8ow!lRem=vrdvhMPn8PFM*4nAqHw5 zvamh1Z7L9PM>EW^4|gwA>7ChLQ`bx&#=)2T&IvzA#J1D;v}0NouO-*)ARfR^(v=J! z66czfbkpDLw|DGv+iiWWq*DVsod^EX$Z9SM=VmYkTy79UT5Ft)o8|5G#?w-+$=M2+G%ba=8WRnX5^#ln;H*NZVL%z#K6#Xd zr1aO;KIkw_|L*x|a_pW3fSn9kx~!1Vxp^K~r0icFYET^~RKAa=&p9NHEGwf<3*B>b z!EqMGRG>r@f}%I^Q;SX;3U+~>JNfw}Lcpdx9%s)qOL!QRxv9<4?7M+TIIFUG8G|@2 zJmB)T{#5V!h=BX2d>PNhmfTH8azX)iad$8ZRnnpz_Y6&#m_}2^swz6TH|cDfjq_T= z9r@{*D8I;qt(YYiY*V$#YH*I2+=%v-blgDvN=y>h3`r(jcNiap>`~@Qul=;cran0h zjx&ElGanwc=Tl%SgZnwv6rN$)0_R5H_U_Hmtek~?0AfVvoVjDS^|ef;NUON#-Cp(5 zbH<+9@A`~%e^$}#yoU&D8#qS7^LXLTmqcmHW?~-WC1qX{WPNES-Z5E|)L0ID-}r|T zKet%WUaqlC!=T=T_h`eQk(z}dq0pOXJci7-hz+o1O&>iT!5li1Rj3N*;5h2&uEc$q zKJhoLdz8FqhSLx(?lvmU2aa>B{CCC8a5TvcM}j97w&C6qJ$Wu?UZOVIhVkI8Y^v7x zdfv~mTrPcu!-wK`2lM-7E?mJir231V+jH1 zpiv65*%#Le`gT`~LeqUY!dCOFJvxp5KaeRONt8YMmz6-IR!YDBrPoYiwAzYvWC`W4 zCp26pQG2Pq$%4{cKmlY$Tr48j_$fB+3f1TqEWokJTVvUE?V`RfRU~f5kY^%h(D7xX z@-sXx%=~RWMnfuLH%Qw$MZ=6MXVqneV9e)-{65~x)_y0iP9te(^YR%BfD2p~L$8|R zrfrLfd>$WO$45q2w~Q$z-bY2%^ZS2aN7NIp}Yh zNe$sLIv?pjnt%_;V=sUlZ3$sL4Ru#y6rL#R{)w1=76$DJIho+%z{%CdicnVAp!`OI zjvG%$V1kiRCrX-nhb&L6{^rVZ0KIWR8L9@%jo&z^>b_%4O9xu*$j-quS=QFQ1H7u9 zwCtu8go@t6k>0A{HW6vx@37C6G^-=-MXwP(Mt2K{-!h@}23>(%!6S(_q+t#9v(CfX zv5)8-DN?U_A)y^~c?Ieq_x{q1A=j`)h#ii}nFwsx+AVbibrqPGil(7Y*9+E}`&jm= zqepP$5rtpYhFu?sZCXo0yrj5&`o0=7EbD0y#mN)sIlq-dtBTYLmT~pQBTjJ7`;2~D zrm83+eH+JvJYFQdvO%E#R2b4Uh`DL({yUbVC(?<_r+wb4dX$;5sc7!o>7E^E!;WbK z@CXWrB=2~XKSltwgqNFoP?H)0RVx(F20V?25zVwfKkSie=WyS`Q*}z>LWL! z&QdKfD9ToWZA}Fc#EqI1eKDij-c9}fVuqkAd0n+S6)NpS>Bhfm!8LI692WpISiH8j z0?~uBQgPx)xJCv?T50|_ZElg+3A?3U0T&HqohGz}A{`2$R)LLjZPG?jZ14W3G*nE; zSFD{ep3_;r<$?ljNE~v3@&D~kf>^u|uNyEi09OF2rZ$0S-rXFG_o`J=ImTIWlY%C% zTf&QU(mOiCE#JEkEB#BuvX3Y(9GSgmp5MMfhO?Ft+uG6+3JEtGmbeG2iu$<;va z-VuD5?S#7j=en}pjn%3-ap{wzA8kLDCx<-~|7nGR$~s7L4`7=6rHPA%D{sKVC%jAx zoULOM2NNPdD6B{xi#4@(l)CJc5-}JT+7_^%Cx|lFh+L%FD!Ng`8{c99_ z!B49QsW!hnObRbJc+Xf%W_;CxpRgA3 zb_>kT5(CHViV?}?sRX|>(FrsewETii)J3a>vx*v{gL`2ja-jQC8|YZwP+zeBTFxO; zwT&Hfs6^w`xR)?K)i(ZZ5siIN=XH2kFS2dwMBzhozU`~GGD;#=9y^U^yTT* zBdZ^1y~DlcV6StxiYvED$m}fOda{iihueGD764#aZ^YLVOQb?MeW$}Gmyx)LVEx*? zyT;Es4bp2y0dyL25X6|hdUTATj`qD}=aEc!nt=$)<&}IPd05FQw@A-@%)8!Sd>jc$ zQQ+_dbSQ7^M9n6`T08g-CF;+nGOmcl>joWq1jTBB_cYYBQ02$cdvaXz9IIil75N;o ztxGN-Ii2?GtxX&L9Zd1292@2z)oZ=Xd9;u)>_vb9*PrmlrB$L z?aU19K1QH>K#0S8XJ zMcb^A3$}zfyEHEzlISYZPfGOSp(}}wC^@jQfWfhJ0i~UX+>;h&t8i<9bOq6f%`=@N>gy%=3T&!~+pe6JZ_e48e%w=9vEWht`Ik73cwLV8u^jd8kksL*d+ZuK z%UI0dky$1&t;3&;ZZMy$pg>S=0PSBcZ%6w>P@T#ndg{LD2ArW|yV?!`SccM?MrUb1 zDU#dRj*5*r*pw(DamTv`qN6VH#&L#InBLdc_$}neyz;R|<2)|UP@ReV9bnpz zSsVM_6Un~_c^~GVki>y0d%T-ZFfJK5vjo97hwZ~*T|an7p1$p12D2@`=>`mree1wK zVLsY!2`)01KKj^4;x^Ohk-h#&T4mqF%oC3g0PCZIeUn1M3!2-dWUQhhGTSeo}(V9zJm^N#Z_qSU%g@LISq_IfQ z;U#S6zJWYb-GLJzc1n6-4WrUbl(;!2DM83O!a2D+*yU?vwc zcQRE{b@i3}pi{5Ou&iuQc9b#!z9Gns^*<#>5pSR|l`ybP<-!H^y}MZ=-RLH*Cyy0V z9)3Cde_PBA<-vbj_5-f@@RgjsPI#$t2V^9&-q~KnVO0b7N+dDfSLl8^FEfXT4W28W zZs7?@V4zm?5ZO{lFIlN~)P7KS)@37mk;{eAn2Ai298vg>r4DoPqlO`{odU8ZmKac4 zx6T6DrZB5Ji8i~KnaSw%!J?wRxxSn*P%1mCd0ii$pOW~dJF!Y<9i7{oy zTo0jr;f(701pryLOjINX04~l8>x8bJ%MLAazew-Var|FG-<`~d^}%~6&d6E;CaBOts1bs zmI&lu(vk#`XGV>7fp-#slC8*vshucahqQU-GVDwI`iNz10$6+Copz0T2RCxr{RB}K$iD{K5lzTv(Pob zdqLx;z$b%iI15D4Vgsx#Cby46`OV-=5kV}mpp~uh&ZR42L)B-fH=Dr8hU5X#&DJKD zlK(=#dPg9Kk}4^)+ytP#4)E>zuiejbOj8iBqRon8?OYJP~%P zwc?ksj^2O-6S-r|A`pIT2Fz>)w%R)Y`R%eqvCA{k7592^WEaGiN)ynO_4ndd$sVNd za~LcH6M}d4wCChXH`R!j|H$QAJ%WH^UXd%}kM6Qn$7||QCK2aMVpnF$TF%-5@p=_! zqNWk@y#2%phMdGe^WEBIN?_?ypX8U-Dt?oY=jfa97j7~LC2oY=z`F~2060I^79O!b zU<1$9@j`Uq0QlCcdmJ(e6IKhz|dZj0dzfFda!M?ZR1}moL;_0Q0aDh)w$L3Ai9p{ zGOScmqq%X9$`S8arkMXeH(oGjwmyOYq2>S5G8g3$dq-Ce{5T~j>(<}J?m0WmutQu| z%r?t-C-G{oCBz0Tg|Kj&8zNkcFZpepZGz68PmOHDS(CJTx|;Q!Fl5@**p6HT2eXk@ z-lky)Seq|uFSXXESjm`_LGBK!OdfZun%&L5A-s6~U6zCO1huo{{I3F$f*e*MdW0|q z+P_lqEbG#YxtC$-#N^;_a-sNa2Q?i%9_~j4nwZhp5rKjE zCi&~Cpii?(M@6Fb;$PcVr+hrSZVLF=U7)X2VELz>5wH9q&)+6lyJQ99@($v-0?T1g z%1!Tb2GeDVgQv}ger5eM)d-8omDNU-Lt9KXxLiUdhqXE5tLZ&i5T=g41KgZ}NJBu3 z&h5`d5+<9WG*ZvB7{hp(Oyd0(&kIpq|H7hm%XccY50)n@T#1}3r0uYE=?v2*%wigo zQ>gt?;J{Fpo5myaGseFa-zTC-eXbiwh{q~D2Ynd%cz8&CfPa=5=__oHZEiEzVOVxU zic-!@m$-;g&{zL=5mE_Z8EX4Cfb%cZYS*K5x(fUk?58CINOpY4n}AGvb|J{^4|OcE;4qq`IvqkTEPlg4s88s8KqmPdZR7EK_euNW@{nzgxekh0RaTK?Kaqq9bS~@d^1}V+*QDLTnEzMoY;XS`VT6v%l&Wa z73hp-h-CYCh(e?LRYr)1T#TYO|92uBHV>ySJP+ODv{)0{0Ph^OS=Roe*D~$9r@N#*ABbirC6%p72;}rp17) zeiHUk)Klm6#tvJdbW6mzvWZgUpO{B;{MJ~6gkN;4a2)n;DmLsvt*Eo7m<$I|+hcpc z52ADQHB9t#AuR36oy?k@6jqA#;KrJPc=4F+wG*VT-VZkYumGd3c$;9!D3K4+Yf%%N1}N$dMF zOHB3TpiJ4H1ppC8TYb8$;c`sx%m@uEfp!w>g9)vEGE3 zSt>kf@s@`T8Ao}vN0OTqa9I(h*|}BjW)*u4*PVW3Ud_w*c&(H&z|@-;tXTuDBEo#H zx)D1P=y;-;pcq0S^f1(Ji26k;<<&U?w}o;fu%+p&Jp8tClrct1xCJw5i|FV{i* zVymx!xFF_(gB6p>YYCEUZ;6jr6QV(@{M^U-;$9mE;6|0+&kAJ+F0iplw}HStpzQge zL7v*HR9Y7&r#k@;-NYIi3 zqYmQ!kxMD;+d>qN4r{yqN_rm{!-;5g?*JEK{PA$Z|GV|doX2Ul)G<2^H1?;fD!k|l z;U%ZEim#qb;!FiW~on&JC{pQ2hWbse`T)Sesl8A$6+Hs>>J zbXVtWP&d?N6f^{!kqg?pB~?(b+rl3Nw#x|hWD zG;Pgrv(9YO)z}^RdyS`g#p)_zdbV(V0cBlhKgAbM;b+5Wn8s!r9;mf&~NEKe=+K$+uY_@kioG|1@H-Gj8 zSXK4^0c4}PH~fWDnn~Hp;)dT_mAveuvT$NM(@K~1WUjCAEqnfVS=a<~Kd?&wvpp&b zbOMKphldL?7Y14ZKPGJf87iaBFjT@AP!-!ftLyiUx2a>3_J%qHIG@^RnMH#rhMwBB>)ZBpp-NL>t$STASe?7_| zMX&vZust90+DS!#K5e@~T&)?3otE_x75&>iu~KkgkSk~#4T%vfMKr8~I31_71vq$H z1vsQWvL#1%5+p5D~T3My2S*w%{mc z4%HnYxbn|boh~V07br(vRf0I<)+9IhI7kzr&D|p4dB*Z+S1N54i0(iwejXgsdm?{B z^iR+ep_D$~wGH>tm+( zQw<2E9tI*mct-W+-+&|@WC~i^#*5QsiQX>JJPG^HRkWGq`zdS+IlrlGD zHy#A^8J;q%}_354fO|$=Id6>=$ zF(zIaYxw!7j4C*UR70NRFEzq}%R41;wRA+V$m*o{jIF^Eb-BTpw&0nbPCFU!UFZQ8 zC!Lf1>O{@6WwKA=8Xo1%eI7EFu-N0Kj5>~(Nnld2Y~+onDhD~(nhBA(ma3B+`4=V= zF3NvvamWuNhccC3ZxE*SFi}GUo5xN5&#!oIw;Q)J9Hxa0U*Kj~7Z$gKW!ukq^^FI++jduX&hBu*B43 z08nrqDipQ_*4d@QL7RNXqKhvt`~bWts%LyJ>t+dZX?R)|wf7UCyDy4Yb^ALYmMdB# zPaG4zMz3!t?eo7QBNu+rb3~g)VxKd3h#o>R7NjM5Sf~$^Y+IuwL4m;Axa(QM;UyJ2adqxbKPTY@_#bKkKz zH^aUkN?@6MbggKa)H|9q$yCt>smB~|P&2HL4pwZEQ@e9SIX))7{3v8JbT}NX*2 z5=%beOVB)gW{55M*{Z|OAb}+K+%3g2l{hLciCUPTU${FPvb6hYV*HKZbyNB3tm1xb zuuuCRO`I@pLWO5XvCi1Y7n@~NRSLK6o@ARe#%rv#pnq#Gb8a2S;-A@~yE=_x=jJe( zx0k2q=KdH>MwZ46Hl17&-oGbE>$>CB1~T$T+x06_0VW+FG;J@M_>VtRcDNTncY z!&gx@xlb*{#TRwlj8W#S_ag6PdW65`gS6g>`rwjSMF{oO1asfmS(dAg4k7;=&l|U3 zy+7BBL!znPv=Ln&&6P!+g*8Z;G7X$oZo7Rg4G&6Aijv=6jswP2(OQoomO{yrEkE*N zG$E0AK)GPOajR?~-fS0R3f*hGR88&-vdL60z$%}r*wQ?(?QS3!=J*3NWC)WMj`qj7 zJ>8PUX%vH06PS-A-ywxrqKGx@BYs#cnV{E!^6kq96qYzu3X=vQzFWTQ4KfLTfsrJ& zrT?d3Q3eUww8-YE()Rz7IZv#;!Rb0!p{II4;(X(nZLi?8IIAexLPO=sm>dcRP`Bvv zSR>%ChR*LO5|ZjxxVSO=XB-L=87c)?t*+ywo(8nDJfMH#)-p1PeccaRj+5J6{L3uO z(!m4|SURr~T=dcu=7V^(ASPKRUwpo3e$rp#;b#@8MV=F{s@Oy=9mgzJ2W{ickdR7( zlcL$N*CO}djkE+zuz&kQZ{h2w(u!h0U?L#3zdy~ywggeJ>@;kM!2`)ob-*yN$|EfZ zYcuTSZXh0_Vm(JQrb&!|bmH;Dvi9k0m8*y92DV~I3xi=_fsW@!Xw7FT2m(x>6F~lD z!!DJ}zq?YaHkfNd7Qu8G@QqcTLs6e6D{-v4qzxk^S{PO18HrI)kGF)~{LW2UOns!X z580=l01mUinygiG((ZIw!!P}>kLue@f-Seo;uA%58gkAo3nVxahCvHZq`K8;nHdEo zj1DX9R~PHm#`--dKD=)AH=;-`9z(L36!uahUdt+LlWgFpqF7ZMJ+aFp`4iIqoBLZr zpLv+6L=kYw-RK(bl~qi(JBte;yUso?@2L)_`EfQ&Y|9rP9dB|^M(;;N9Jzp+dv4Q1|O z-`;ZxogYl?47b16$@i`v-a!_AP{1B}P~^?WTz<{u$!bxWJQvt|@gnCi?=-813 z*^fgDW7)?B#YItdO?~uxs8?j@fSL`zMi$e)s`Jh(-8w~`K?kM@5tPY*%4@q~9;f>A zF~kQGj1@F`#4DM+zDil2Ie6=lrv7cD%R)aj`>IIr1bhdtWzFpxNrdQTqbVA{@*y1M zKso~MdIhCCQJVQK2dlg#e*UA+E8{@n(rAUBo%Yk~ltqxS`mYBwml_!KFypp%jn?GB ze_F@;+NOtzl6~*BvYk{~0e=~&)PN+mlLW*IS=C`N19)#PwN*N>t)4Iyf$Kme#^sV) z-lyJLJ!;JGG1g>Z-RP5ionD>%KW!lOzL)aUi-3JI;MF6#^-R(dX-`Uus?7;I7TDyD zE(p`_=?xBG^O6B!(s!olLf+DckUSlB=@Ue|1ak+T`pS|A#5-vO#Zw~gu6Q+~bR^A~^)VBE7uR^RyVufg z_XrIglszpJe#qECfOOsXLiYQ zW5(+uMq;%B`UE^>Zn+GqIjW9q7pH?iW1;FDRx==T-!nyq(5Xedkib11EL7lU!C9*B zmdS!m{O1nnHTUCr+ckWXlNWpa+r{`Lo zE648`JZAZq8oc>6)Td`>YR-tOEw6GosykTj^V-}wI*TRtHG*LN3x3p0 zz}hM%?LCgxw|m3apwBGAr}^H%U2SZ&k`6FB$R()TyL_lPKeEf%K~PMTVdd4E87#Gu zxv9KDFl}wI+0VlAN)&-h?Rt{yso%J^!-txx*AlBnULU68Ae8Sypf~99@E16%wR4Y! z5YJ2tC=|ywv04?kcgcv3rN8@s`^%KjwKg~0fSBuR+Q;e&4pfO=`gigMhLHJzwzf!* z;B8^AiS7&*2b@e8VsExjfM7nf&1o2eA12xV@>zA3^kL|mX#|?K1dh7^CW}&~JSiHs z!cT>JSDKm2?pCWrgSxR3VS`}DLSNH0sdt3cNeC;2eHmpp5_wb8C>Xewt!ZHs(RAE z-keBU6sT~{$PK`zbo)KxsDZ!bSfGiy0wbl&?wo!p*0$|4pD7Ygl__H2w_p4ieeF~V z2Sf*NR7+2D$yi4(^0PBdeQy9$gs=QdNpG{Va;=}phnq9vh|DtnrJpv(O>pJJM${b1 zG#_tKhB4JBtG}zldNW^b8F8gt&+HW?>XV>5k;6~PN$!)dWkMU@vfv@&n?>OAI#=nTwPH=^OQ6Gj{BU z`jCiA-{@s;^SNO8rN2f~S+-mB4S2I~zRnZ%M*-tBX?^&nLz(xHfSMz1OentXIDqzD z=BLQa!KSDE3%JT1u4DM6v?T@|_K+BQvG1?+P&*5=0?_Gg1_%;e6hQoV^#xB$D%Q1Jhx)-omTV?joH$M#>frFr&hzblctb>un zKJpL2j?Q1@yIg@*0UO0aS9K4d+J%6J^hlI01${|F4;XSx*GP3hl5F>ox+)zAd6Bq-X-kwZTvC8-beve=^43p*q!123fXsuW$T{HKzqexi zX2{UVUg7SZKv_qJa~y~ikV6(G`}@Y_nNh8WFfa3y5fv?q8o6y=2f@Ied%|4=_`D89 zIHS9DqcjEe3E+!Uvd@)5nW;}Y;? zsRZi~{W>k;>3Ebz>Q64Z(Q^5QQ|`tkJ)P@wsyC`bzDdTEAwyZ)U~9bWU}Z8Nw*uoI zBY!a@%=n=w{_f)pDZS+Vy7~Ej_JP(o5cPg7>Kj)CkcbadiN4@)ln`tW#}9CNMV7`S z<>z$@pLVXsC3BWw@h(Ex%o6_141@2pqQnle$87>tisDr&2|9+`bRcAc#LidzRd%#GECBc{v7XAZO{6V7hpK+QU#Wr zmbG*!KTN0Gf5J~G;n`H7lCx2Y6q8svU?N=WpDb;S(q(S^W;lD`9mSwmNr6_B^wFIC zdVdH1NXUJN4~$7C8)@EzAw=~^-~eX%1wROSiP zP%JB41RC9~>>pKPAzKmNy6o5}cgses!mb6f_?==>eknFCdk)$0aWN7UP#vxsK*Tj- zY?BOY7~(Z2#(W`Bp_3a$;0nLZYHtZx9>F8L4DWniWWnAh%W5@B3z ztOWzR&+~XO-Z#3?_`wwg(Ey_JAR7mhv~a?MVtk`=D&I=_%XF^`qe5cQW_)kMo;3ZG zumLDc$$etoTgt5Nn~3DmrY0vk!Zj#;ou$Zfkrb?}8W?Zhc+=rjEHVeedn_7IMzwS* zayWil+dYbRkeksVk7d`@^V|2u$?%@H{ECKxd3=ue$q4G$2L7lT3)0WpLSVF)#~xIdSB^mS7!@gDU$?^1ANuaOi+$z{ZC=CdCP_pI-?$DC@^#8KH=6@Q*~-U#P;cW1{$l9 zsBOu%k`-#l&tj2!-Z-Mqvny)b`?}MAx7^*Oiu4f_w1PY`4AXsICv zlPlhnrP>^yx{ZW1ebwpmp8-RAoQOA}(9hm}KL2cB-*xUT<%edFldN5rz8v@d%}W?! zh@Zd_p`sscCMt^CTg>-mpo>WgY#Ho7C`nAG-6}G(BP3-`r;uI|V+u()qCA}Z98*;D z=xrN|v%R|^mx3cmP1i|VQ!w>=koZe>f+w&hw+?jKBcuw!)F z!>@cOuG}TwzdDX6U^(i}eLN6z1oeR*O(L@9I{l11jC_UIhNr~++6sm0q#s;f8Yi6p zYDjHH55yU?nIqdSrzaw~ACQ~bi-s58TZ&|yI=`FYu8jTSxY6(=Le7x40f{*pbnY;l z@(py=%)%%-d8Mb-D3a0}dBm7@pWQPd1YFSlF)Gii`$B@MKin zHaF$yuGGvYX`+S)Ee+NLqXf{9KX|p8a~7w;`SaN4ot552zyEPgNbBi&vri|s#_ksn z$#EYN}Aw6ywwf_#r7QUfT_!=uMS2(VXx#)*6MLuHnYTui7c%#gup%Ccm6j# zsfHi6)1nxTj8Ot-O^}3#1(T;(i0C*v@WYKjqDd2U*#z+|t2ldyy14-$s|#Lt&>gt< zrl-1w6;KRLjomI5hW1g>WKj-kPo?baGk3c&`v$02ln<+JoKCiKv2{pzV?IBuy6SM) zO{cl9edJZ*-B_$!p@^@dql74??#c+?qZhA+Yew}@_ln7ez=3Qfb;d#g2=PT<>CM9n zF!OSY5;p=kuQ<#gO|?;k7l)3d4VARHn^c*@{%uP})Kx9g4h&_g5$I0=A zYCeQR)+L9Irs}2Z5kop&5EhC7M`O0_k&ayn&27bw5*ocvutxkrt1Et#y?UCC-&C%X z_8yl$YO$I43%1sQEl-`vC^c^_hPki+5CH>T<+_0Wp=UVIuPU@yPX}^w$TRD)ibZAb zmbzsV!IMz0BDd%P^8=dGOw$#E#jwK?+-?R2N`*R&$pK^*S>=lqgfm5`^w#N+1gCD*_}#QjMlmc9476Eqeo zzg>0*Ikz#d@D1@<#j|N#g4P0?QC0qPT4pMj_7d}nRRI!NJ%I(Ay$$3`xZ)?qjP6ua zj%VkCYanT}0JQRwGfE&K^%S}Q6S%ZOt@Y9e3RK4Ech1ZrH`ES{E?Sw*%wQVQhS`~s zVKJ!8GKdk{~hqfVeCr(_y%LgP;7nP#KvfX$arCJ-$CMWfJ$il}4DAMQR*= z($C6AMykrTiGO}GFEAM=M?2+2jKMF54e&8-k;(&N_y}J&nz6WMlV4uNPg#<<>DhIx zM-2V0u&s_neLm+T7k0+m%J?D~t|;r=S~vmO@4{lHV#h`Fmh$uoX(R@WvpdXSSCB^$ zCK&0PQ&zwYyhk3azHi*x01AelfnaXg`gpq{?c3}#cM)c&d_jpAgj@zFYBAb+*o|Lk^iH$qG){@1ANfQp34H50kpt0q>D z#`R*#4r=*PR#<#D7v0K=uG6X3%pV4M#vS%iuuEH#ZW1GAEWJ8M}e4&yUnt;=Pt{ zl?JY+hAhVfQ9bBGzuBxyK=a6-r3BG8cgKA&O7~!1iPQNFK>=5K}>x0 zKNiY1P9a8-YR10fkd4Jst#F?ipchCa)VK%C7-qe72y3(u1 z!`mbFJOhdsou4N!>^%7H zwl~)1oJBttMXs^L->*qoea*Q=52hO|i0KAVl(8aAO7hDs7!2HZcENt9p+Ij za#TX3q^E)P?Pj|dWF zw((S(Zn^ofPeQupUq&uFsa+lKJzXHiicNCf69?wT7-o`H&DsJM25N9+!?oNCLPu>E zBhE9t^y^kzAXtC;8OnKpTBrh4Q8M{Dja-vs2^IFJvlEJTJ&t|MVzO zdn(enc*0#1;2qeGSK`-XL38}CQH?ik{E`}75L@7<tpg5D*$+N!>d$#?5Me#8*X3&hXA~R_6gyJuo>@Y+N9}i88Rtv~nzC>+ z$VIaapE7rzJN3@!PM>RZaYjSUOo07Vw+~NVy7@g=>ayKnHfg6%0qQ1a6b;2)JqBPG z-p_w%CYT|0fyEu!5x{~qbbe}QEqF|lRPz|;^Z@aQCX^?YQ{ z895*CG|43ZFvG%~ynvA5btMYfHp6A;B2T%s$whXvZ4(FxmVxIeNpUs|tB~EwG)3qW zk;4EqL0m2M14e$YBY~8MdF2$64&i0n^SqXq85%n6 zD|zQCWXtw3CcS_{?0cr^v*^?=%$cQci-DaaWU)WwpFidEQ|c*;9s_N1WrM*hFX+6BKD)dC0d0P$SD z6a!HK>Ui?3l$~;H^R8tro9u%(mdZp-NZ+|*aAUDkt|&Zm58{HoWZROP4=>kjovP19 zJj`(!?z=ScZ*!U$oSP)RwjS@+lJ-Nb>TMXjnYp(NY zWv|v=ixKBxLi4FNQOYEepTLF-9jBr`gbtnUNVIb+Vv)z)?$wrm3Tb)JTu!*G9OmtmB^g-AsB}y?9Jq$W3kx zcbRr1=(g&sKY@GpuY`FPLjp70gZYi#-O7^z&ift-Q?oic=bQ+451k(aIgEq)1ISOiiwt5g?4v}a0a za|hx2NoY8^HnoySAsRFL@2*cH(HVs4x%O)8=hdt$q}1YdyyRkg#@}qH)?FBY-`%bl ztn_hmonPPE0(4344L~hnT*v7}!mtXje8`(>jAp;v6<0tBdokjdTLvxY`$a2(P(BCZ zjc7%G5Mt%msp%VOVL^+&;vETTTd68c1lfq*rm$d)z0OFkB}zb_DJOWvixDs;6zo!6 z^uhGKkp;eCJka&8&V)KDCIbc@FYb$$l$&{h(Y_(%=<)hct~Ez!`1S4MED0h^XbtMCbs^OP%rr*2*|gCrchnlH(PK#aE);K#nzn10Je zNrf8VPh!YW7bJgK4#?MllR%%ysa_AnbI8x-G9S=iYv!?lH$d(%(jEov2?eiVO{|;O zcVyntP>!wh!rV*TeR5#)06##$zhP;PPVm_@EMKK^gPR@ta4Vy3+jJdG;{>``<2_~L z>Cs1V*e9|ysD{JQI1GmGLQYIa4Ip;S%RayDwM@cd&%_#t>j?5_Qo9j{D$|7iRl!wb zQMbJv#L7nFPz8UC65S58W6OSL_goU$e8mQA1x(#6_FQ@BC)?KSyhO0(PA5psVLZB# z8p?cSsATj$_n|dtF8U(?L6?E*U`%ZwQj1oLzsaGlunlOB!6MN+tq#uPB6bvw$`ork zK$Gxt!kPPev;geE29I)q2=$Iz^~}iJxMJy;R2x9MTRNPJtm8Pdh+=8;SKT6rVtLJA zlpB#7sia|Tg%7>ull4(hYVti5cbFd#u=DgvA;w2sj7oY{!Df+@OOL$FTf;QJML|ZV zzJ3i>f2gYl8@zWsxO71aBcq9SctEQqgdxVi|@zH=#29^L$m5|KHjUW(OZLXI?xR27W-0Sb%HgABZoqF zuHfZA6Nxi`pG#h3c-&X7O7pZ#9{5=q_>x#vq_J z%w7Iv%*Vy%Hsi_x)071S%a;3|$TqkWQJey97_$6c9lb!`Ma5pkXY8n>-ZMkyY*Z*M zdjrQ*Z!~Gj5@LVT2>{_bGd^!H=jE@g#WAFVG>p|lMhhBaWR<%EeoLGWzfQnpZLRKb z7DJ6f%B}_-;i#|AOeuE*?hf*td+PJ$3wDsZr|5^tL#VhOx+*loc>E2dr&ku%bH4D? zx3DdnNB3ydtpkl3uo7o9%@K3zGgV)uEdou*HkcjTcPB5FAF<6W;~OB=X%wH72n1N0 zcl>(OR*E}ID%uzi9fg~wLz|h&hHcgF)t%xVI)j|Fd8Vs6LSy)+^AX^V6GSyR=Bl$g z+j;kPXEHGi<=DUuM1e|U+W0&+u zmJ?DgT1%&sW1an~{FqCi5YO`7x0~zmv~8r?DzZL6MucUDYiU~57#RCILNq{{S;M&S z6btg{jBtY-oj=NT{zWmLxID9etAq=+2@fNUp;lgPcdDe=>MQ`ZjtmO-VWm4S?3&+_ z&&w7G3C6H66-P>S>jhJ;+2Q+VvR`B&{HKeoX+J zGL5C)R*C4jig$m1{|`h;Gf_dy3wJrA>gP0;IUDMR6>X`w8D#XQbN=EYgAS~4{D_@_ zsG{_h=4Btln^DWr!4Enj;V%vpFf5|iW2U5tB+YveoS@ND3FcR5^yl)x)TAWyuwwBB z@Pmw`x)4ufZZ%kv1F<^=}BOlxKCog=4$1dA`qpF#;XC()y(djN4Sg9-Q{ zaxkCK#0KZPG3O1i)ixk*Suk#4HO>&%Ry1%XGUK@5Lg_jQ1z%8Hu}v1V3+oqK!6=S9pFEOlKQ*Il_#lwpo}GM>&Tw!72E`ox zmH>nHy(}*%vt4a*e)TK4KmE?3Y9F}C+A(8GW~ByVhgrDJ5<6uG=FDilnipjwaNkxr zjY80au5Gabtv?LPpq_oN-lhqYpNmR=01k4CAP5$1Wy92CYgx6XV2uuoZmC`Z#Y-Qt}b@G_69GiZ^svvmfA-c zhxt7xCQWM-_LjB zN&Pc%S*PH^EZt~aF*cP(Zi{9%uBfRMx_9tzh#6E}W~1cQA|lIn_%vD8Lm!sxJv^6T zru2bQ9GlskPl^cfMwA{xJWSM% zZB`Ix`JsV)l8pyCOK7pf!)U?}^_8(iMDXT*KNz369IWt)Pwy3-X)>G39ZNcglCBQG zmEPg+{3QAvv~$PQ6AQ2p%CSxi&nL9dZ*Vl|pU$hZD^Ib52+}?y!y&Zjx;gI#G}ewF z5AjKo?h5lH^T7|oz!^L1;~mH>vPmLL(J{=1pVT90bcQAH9tCv=I)iu zu6zPDKs02>v`pYD0#53>q;p-|7vV{*K6U08EV6*DZS1+Aja9|LG%#aH9dL!ex<{f;QD_Nh(iM)AHN95$&8;{FO%=XmEK9-J>8Ut0QuTI8Y|@ zb0Q9m1-s%-+;*b?%pVBPxF&o0sf*^>zqBGc)~}q^J+Kw6gGph~TSbx&hUn6M(tt=Ye3e(~2+<}gxL3YVUh`=p zWk|Xmk3<*p1J$dD`n=gnX25QBJo=`u$ZGY_gRc-|S?kQhCQ1rr9H!% zEGwFRv@#MY{Q4RI*^`a!)CLci;VR$R1FYHZH{C340y7&kBO>U%DP`4^rod{ePe6xX zn(+~a1S>Mtb$PQ=*n_}M4PrThvvI%~{zSQu5nS>EBJz4i@V%*_q}+(tVC}4vy|RqZ zlk+?%4AXCTXG{;>55^6(jHl!d-{Ch{fuJ-XRT;SZ1V34J4sHd#4YwpGpvVdkP6!v& z{nBeC;jiKHNDqQ_o`Bh2LS3qu#5=|4m7Tpg!!oF-)UT~PG#^jR^5!#PFb>y-54vx% zXrUK_?5>%_n|up~#-sy5Y~oF)qWKux`{4az4gdkbeSUr*R(3{_{kViy$I)G;8F%g$ z5SW_he272Qo=I|@=dfdiA+k>D8$_Jt?ZPbUVZ8_m5PxHCjFE~1vb!D98*8g_JGzZc z@(U%OD7ERJS%*;Cl3%|UCfxVZzXnX%9M!LI096xw5IF?K`y6QT*=aXz9IWbff(9s z^d2oA13h&*a?|;9w5cguu0o(u56|$Ckm_fPXz#I`1)lo>wOgU1K9w@?gyQ_f0#tv) zl8K%x^l*iokimHr1*_S%DDu7t0S&h;Zuzs?i1}!$PsUlUGpccf^^>4%vMYGpW+1t~ zR_Iv&Ozd)Qh(bCS(|Y6AbqF^~YLcs7yLFwS*j7A2T^}*aaqrD`0xSEWO^L2(Y7`bL z0vf;D1PbQ;S>bT*uOK%etFxzNU-SEy1mE-sK3}r2 z>ozTEL6rblz%(29BzC}AIC-Kp%dv^?bf1s725U!agF8b_$niyijSGJ@R+#| z>uiiwG_bofK9_xCadF!pd#7#y0ffxqIjI&1$B6W$nQke*fU!TrCrr;SKSn6QZ)zG6 z+Lsz=C^^H9H|2W53l>RJ+qOxySnU%yt?O zOf5X5$T%88F#pi1U&rCJDvGWfLL(XjyBLxE`jW)1ifs*P!SvmRjZdH2#9LS2Sl=2> zzdjysrR(b;6q3~TyXt%y9`QAPr7i>YQ^*Q(f7lp+vBVIdKlZ#l7&C*1A(47oTaCiw zAwXihKlPSy&NR@DfMr7)8HS|N*)yM!YnH}OjW_EwMCjj!LV)KgzL|8e7{c7%O?f+b zjev&b;k9_zZTwiAKNGhH&Xya%9i#E}1qSnDVxe(aeQ1PMvRVe?qv&%QgtyeXi-JxNFZpruccz6Cp*fZZ#fMs|ar zeZh!sX6DOp$wHXF>33Q^8y&0qu!+O~MzYJ3dBV~#VW$`7dFi<2))2F&YS9v}<{VzE z-&0abr2gf#4W5PK4^cNH`%s<9j(lUXq0|vADYTuoZjui%GZUt5Nt z1DOqOZR`?y^Jcy8-W0VrNLgVuNK;z8*BJ{=s_#v11sQ=1je@c z@#HTo&J-#{r%r`XH0igXcSW|ny0b`b_GvZ5FDldLBlqoo4a}yuKavO1ANF zn{b*$@+Rej_p8mA7+gTf0SxM51MQ$Mw?$C=zHl$Sx>{gxR>2!GpA{##VN*Z1jFTgw!;n);(y;okJ$L6AqR`7fx$yiQ=MM%V%z*A zM?>G$hZCcHC1HgOV-Qu<_SHK7QUiG6wWX)$uJLrtVdr4mQodL}+Euw58yhGj{xam? zXmY-j@}}s)ui588XXA?0d~I66lH|d`5KT+wDpk7=@z@&0BM)_gS4N_at~;%v%6Q(k zHyXDOgAwoez0T*3s@k$h!}lASA&};v61N7l;tIN9Ww?Yf=?53I#7h`lDt*al-g!;{1onRb|Yv`2q}`Kd|dct z<%klUIPI_n2QytKB$H-QdU^Dk7DP|MqoG?5)dONtM&{uLY=4cG@xyzsMFU9S+6!Qu z)xX!TP{nDCuOCqoNQ}?S8=7-- zX*89D&byH;UcjjgS(>?-Eg~`$A!6Cvbas=3*OxaER#!4Vq-}Hdh?xjDzM89&KZo;a zmpN(MNcolA$TGf%Jg?=>c-?afTY_cVW036f&YO+wX8|0sT4^7ap^+)lCQFL>`=XLD zr+XFy#TZB7oo_DG#ETRe`p3nPtFI|rQ|-+xATF^4%zI`nC7ajqioE&QIcJUSL8B>S z^WfRn^P#b74EU?-=wECw33N$C^bFJqnhl=BJNTsO=PTz|GIF zSoVZSgCcE(J@q-kH5V!&ljBme=jI+if(Cd2ea_QFHmAvr-c!$u8i)diU=1YFT?T*+ zQ7^TlR2_1y)U3#Nf)6O(z*Z)AEO@S24HX>rabbui%<*K?&ycc6Y}{5UVP z(S!qJv4=n_wtf_y8@7NFOdgG9lz>>upqs!!C(cF7AE@C&xlrQP>5A@A<`ls3cl|IV zKAEpHwC3-E^oX4VepM$SPE;n8=(|inAN1jfdj}yiw5>4^?+FIv=5{5=XF=5jyddYy zqo&IeS`{IJ0%GXAhg@Qm=>!_BaKE{C)F;n0Okf*POrfK{aK3D_L{vXVE0BYMw?U)d zc0D|`fzQlujH2Uk8%Ev)8=zb%JO*{hSv z(GTavMv!siSovai-j0);D6wcqWpw}_YkY47LAf)+sT891UnW4>4{ak6E?Wq+ghHuk zuxg3iBie_Yj5i?&IO7vDeB!A4Od!2iJ0YyQjMF%slz&@_I$bb+TWxUtT5(KY|1fZW z;CGAwB58`RY`~(>OUZkFdB^vG2-w++Crt(TA*Es!<3LVFH=U2hX1kdv;K$G1hUCVY ze2SMK5)V?$9Qw>Boa&KRgscEqF^nLs*Zv*Qu&saWUj{g=TrN#I6~~OAyMO~~+2yIn zz@0)YCZXsAUOTS2XJUevVFE&08xu+KNZ-S@om+}!Oz?rMp!;T8Fmkrad8=5I8Mx8Q zA`5i3aXJz(E0aF5LR~P8Z50{$d#_OW4By)NK;M1*A0fva}d? z*;C$?mkkb&0$WI`qcgi%wT3Adb&OO&%eTSj3unBK`HhXXH@zs)a%fnmJ`h` zEfa%tz&!myudrg_(?AF!3fTC0w;SLpFA#>2HeLEf>n7l&#pbW2u$ZFVqao)-#u+ci zC|3!_3}@n%(YdD3$q_ym@-`d;KdDV0r0&l=eCVIUeWm?kEX;`<@E%Pvj^mES4t7E% zK&i>CIcL=rGu881DwaSvrEGIUyTL%0u=;{qS3OvRAk2K#cY=V6lD-KIszj<$TFdq0 zVy=B#z`>t2V5}YWx8KiXF&ZxZ(yhyWf}2( z5$q}!JASyjk%EJiP*|_kOJ};}*qlio_3&2&wnIW0VVHcGi?V~EcXoBq&Q-aU$bKWaVp54% z=;K{z=fNH%3yH~-2uSqhp{lFJG+2IV9btDr(^n^pmjSkkvwW=nfU7zB@yWF^+XOGK zY4WAyOU{$_34w$SWptnRtL31v#=~q?%9X!$6XytdIv$`0+R2I$j*14tBC`t|0oo-=7@Rf&l0+U)x!DDI^xE(_{pbb- zHdbpmrBpOY++Zx!QwD0FfyCGQRT~8H#)QAR>vS@+hI--`iOI~pYVLhO4+1)jy*7Y|ZiO!W&7jNPQVS(x6G+t&!Hd{pcEoP6Y|G!|gA z9JWg-_$i~)VKmvkc=(7hCAbKC053U|WBjp1+cD*^ed9qDgbZlu|f7LB}mj4Cb z^Gr>w>JKP_fC7}XF!9@MAh6ORh$Ums*K}ClU%%2?p=j9s$h_nIl-gk>`~e>&8M%f< zrwywX*7T%ZSwpH{Y|-BiNi0QII%mgZO|*$NFuS(`Ls7X=uSH08#Y!#!U}vG!;NBt- zDVi1D%(-x^y7oTe>s-y6Hw_$kJLU~kvWUGhDGWQIddQ&)Q;^qobbAA^L3aXfl@)UI zvaF7;xg!SJKR;kD4_a`ZQ3D|(qSr6)P0JBogSS2v?FE0y!rvs5isP=tysTrF4TfNHLi=lh>|>yL8gBf0+dSO*u`+J0j>Y|bu`GeJdDqziZc3v2J~p~d zK8P+s^r7-R>v#nRq?9=3yL6cPEAPW$2omtR(rXf!iWIoaEj$nNpiO zV;TCvT%uq*Z`4Zc2;N&WPf-&!D8R_oiK@p;mGnDG*bWzQ@c2l=ygqL`Jp3>lMu65% zTM%*vIc;j2OGUA9KCknif|O4Pgc1 zq_qLzfrC_Ea<|^(ev}{0&BLMA9%)_z(_(FUzJpeA5d)hEQbbH-1Pxlp_BNqqHe2m| zok7<_QV8^fu!nomeU9A*MR#|d#f$TF&SA(zNrgqn@=)?nxp)A{+K2L40s~0EYqb;UDo6u{Tz zto$5iZCYmw-!D6si1s+mBs&NT#ZCrHTB@%lK;#&utLsZ&dU*4xqOiR&&9JMyTTD>-UY8Ib`InNOow_O`4St9Y@29z&?HtsBwB>A{sI)3VJ6m&ofN8TfCj&zhw;|Y_TRBO z_b$uOf!3*V*c*}L%*VZkIAR8Qqtm)C{n7i9zur1M9|VHZY^C7x6U?+yOK-RqJlsW% zqZ7LLep2^|MNwjqf?(*Suuhr<2co{?=w+GTRRSfJa~UjRE*jd3n&-rK{A#iQJv@Vf zF_^x|5oggGPV-2KxZ}a93WeUthSlj_Yw?4eL>=ixS6r3Awg`&-Zu&j^lV#e{^s~_2 zolz=@d<1R##Bx1Ip669|0gwq+$?p49{P(J$@tX8o%p5JlLO{H2Y9M9W7LyE%k(z*1 zGHS`uZU$Ngr|vf+Yh3`AWtJ8f16|SLN(7g_sJl^a($-iE0WXxH1~5H$c>PF$AM$iS z6rm7jqd-L(-?Ah+4Jn0l%a1C;vq#d8ZP5ZL0o9jiD3HoE=a@p}T z-u?nv6I=LJ-LyX-!2ja=|H0Q^{GS>=3>X9`2nY};7$`Uh@c#f51OfsK3Z~6JNV7Zlt4eIXF@a8jEib#DEt-G?VUCbd zk=hechH>%^lYxF>I{J85#k%Ln+Wl6U0RnAvoR9km+3QH5Jlw+XWf$vp=F7{_#{d2v zk>HZ9Nh{{{*H$N+1_T~JWx+qYMR6Bc!c#oosgmx1nE;8aO)8xx`J6WtbGb&k6SvP^ zqdC?q9M0CPAI^Vb`T!@S&dsH+5-EEIjvyTfM$Bg!@27;IRZ5Z#%6)Os})ecE-XAHIj=0Q!9* zkXvH8$r_djWu*Zig4>R7xcH{8+@-d-a zTTP9W5i2r8Uo~f_g!2Mdq4%~EG8cVzDT7eFnl#B3Z=Z+}3%s(Rzd$ZDxMA}gy0g;g zKwp)kQgLAhp$F)RzK>?9mSm}Nt6LcZMIyMZZ{b<9e-4_BmwkXG&vvfQsi;J`5d8M% zV2fKyQFR^Vp3r^TR2OgXOr+p2HtF)hOeW3ydPs!PxeVHWX^@_H4T?a_%w=R`iPuaN z!?K-^1(5(;9XcEoYB^(V?{p}AuL-~ z4_}76a!m&U0O?l}IK`a(w9^u*#E@)8EST6E@zPFplr?g+jB}^pimwlv-NsnQ&0med zj(4aXiye8q90~&jbw!|YWEQSkdM^CRA(4TKKWOz@_$(Sv(l2;b5k#&RQOI!!6$+m- z$)0=mr=*OsRXw;FgQcWKsqsniGks_RC?lQ3?N8lWU;}e&&{ua$<10YHV>mzvm1a?U zb6Z)pzjjCpqZk1tGmk|XMKQR%Ka$=4GDX|Kc4saD7v8EjxVLEKsr;C)Ix6%9g|4SeJzv=tWJpbGW!ymqX{)?!8mp+R> zeEfgwpYJ#GkNN)rz>k0RRsENJtN%^jpM66B|26L)-=FU-5AeU6?>_?MkKjK6|6eNp ze^~$HfBetcKW+aR;LHN zANusS?;oZ5yZ(>T{h>U6XxkqO@`v*Np?rTRJp&^XGYcylJBNXxk+F#>z<&Vn-`T+b zA430|`v2@XNPiLeFQWWK)cILSs+0m?3*CF1OW;A7PheOtL*zGiXeg_Dheu~f*|mL2yUQ=Y{KuH z?y8=ifZuZOeV+e)p8pNLUDZ|9)z#Hi=bZOcO?&s-O2a>{+2Y9c&rL1Q8rEeVs5kK4 zunXCJb?xi_7PIV~`B~}*6+gT`=hs#@wa3pjJK3woR{=lVzFgw^<BvT-Jdr7 z;P;C;H{Mu2B4|TwpQmHrV`4;}&;82%1(feF7}$Cm{yTqAN+n3MMTId!W9%Q=%jN$O zrC3y{$bbHi&Q}%T1Q>n)`9InM6@A?QAOCAKId}4(p(-Bb76-3}o@?+N`>Ao5T7~lv z=&C3iyG+jIda8L97f8rzmG4{y=W7(_YB*o3IOpMfol30#vzDAsW@iK>gm|VVgme!` z3>i8#F=Tu|QpmEYNg*2pl0)8~njCUHASL9tsVO0it#?ez)-R08(QjXzqdyjvt3R_iSKlSf zst;OZ)t`-Mqt}Hbgh!MngvW&>hNqM!hHoQD;kQeY!Y7b~sD85(qPCI5sFK-h|oEs4#tMRNS1@s5E_ARPLO#sNq5BQ3Z3-qe}D{Q5)uD zMD-785p{7+i>ROVnNbhsWJU#4Y#9|jw`EkLimjqD=eCOKS+RB0>vLO2EvzU+otrE~ zeWW)=&7Wh8+N(E34VYt!I#V$r_VC<<*l#N)#4VVX5cf;P#JIfG6Y8_u<5aQ3w72@X_jPdJt8RHjLG{x`VWs0w@ z%ZmSIXI6ZhX6E?ryUg)~Bp4z}5)3s-q9Ls$(a?q@8G4i?8D1vIh6yFfhWRALu&pG; z@D@omoGeK-+$L#;KTFaKL0r0_?znVA2A5&5kIOK0<60Poj%#5kRr}R>MAZ8^g&7 zZ46h`d4}I7Kh|stV4JWw`MvceNVUou%RO2U^APTH3Y{U^UO~#!SylV`i5yQ|9fNrp)Z<1fk=S1Yvq~qA-6+qOdhON%&|< zlJHG*vhd@QWZ^+{ioh>T5u&T63XPYh3Mp07gxsZRLU@IAp%16u%c=U zVf)e+LJyKDOeo0|npbNngr};TZviNU)>^nxZEQASUp<^UXd+C$K(jYiX35gOs;T! zMXvBT#wtXtv zSi6%je{Cn>i`t!qz0*4jzt=7j$hsmStWFofu&#?Rw@z2#qjg<{D|NaFch_|js>F2{ z8n5pznBsZ}udMGOOpEI&tX|(!I2hMUxVFBR5L&ml;CiFCu(WO;;m{j>grDp76{>CM zD`eK|Cv@DW)!DpQfXQ!8=9^ z=j)CU#;+P9lr(u=7_sJcp-IeG;mf6Cg~FO8f_FuUP&Ia(khp4`U}!vEcx&l+!BuO5 z;9fsL=o>Rp*tuq+kW_n;P_S;2u&?%HVfm`b!mEv^2piT-5vo?3D)f9~s?fLEG~xcz zX@a-Gbm8XW>4L544B^_=8G^ZXsi0m`DtsI>Q&_fQrf|K+EaC0dvxJd#XA8sD%@&&0 zog++MJ4a~QV6M<`$z0(;wK8G%)-qve{5)a%s(HegnEApho8}9uxCO$c?F)no_3s(i z)PHE4U+<}L_ok=DwGE#c8`M{u+SCs)P2Lz_Dy|o3I=V5?bh=)UY3GI@)35b{O?@^7 zn~pRLF}=Pi#I(3UsOj{kP}8jjl}wWwMw%LLjx^;rtZoWuSkrW_VJ%aihV@MA8#Ok$ zwlp@CHi|b@+!AlP)3AwY@RlZ~)J9EBowhbLHPI!QGp8h&b9IU4c2g3~eRWCZ*QO+y z$LW&IGo~b)SL#yC`=+FrPv}z3UrtFiU(=&2klOAPHUd(T)sQixvzPe^Ml=K&I`@coxki(cZMZo zIOFzYI5QJkIPH5{IQu1JI>+tFbS_P3>D;xarSot?E9aMcS~+hdw04e~(AxPRL2w4_ z6`Zvbjn3wKjn2G8le5oWle0K6%Q>Kwk$>YS0(#<^@?8|R^!+AK_>AXAH={%F%*6FH{P!L`rp%2&)ea40)_NhHPu}?yU zq(1Nt`dkQ0=$}%W(EmnQV*jVJ68l#RN*K1SBw<)iNaC=Pz=UG3S&G*LCKgX9Ni5zK zm{k11w4~zefyu?%>B+^_f>Mf`PfsZ}2Bj8vo1R)cJSeSr%Jj72`9bN$d#9%tpA5<< zK0iI9_)1WV;(OCu6bA=q7RS!WEKUt>S!|opvUqJ!t76aeR>j*YCX7b=_RVts52M@O zyvSy;=Xi39@~!TCqbDnu7`+8W7MIK6A||7`D97R{5{o%ZZ7f-yqAa7$CT7s{T#M0c zfui|Nx5s6*=M-7pMeSV2P_VgvGA7T0G*Dd9cAAYrch83Y_pXcnWP4pVw_^>xHYzWm%zdIqeH%;lDo%3YCbDUjpVd|zvQ ziL`UrxgB1L#f(LAtcoVtz3`fqCq?ayHm^laqTMNaBt-`*#GQSQcJ~tHoCK{R{2?JhTO53L3`8=*ILckqeZk!v2)=?W_x^@ zY#zudFgC5e{CtPm3NOecey&o7nLTciBU`+^oA~vDV%9Ms@!V{dGs@3oIRl238)T~;B$yjZWLtBH zI~SxCh)X-v>~UF)`K)(+f{b>qJkJ&9;?)GjhxmHLv|MXWE)j>ybI&PPA&X08JLN>D z%bE{TxSfS~?UFQuw}d04++cgIf4%lRyQ96G-Hx5+Tik9V%qu3dZ8;8?)sveq7jRp1 z>_(5*WqCm!jwY{N?hTyYFj}&q!(i%3g5NLS={13vl9hsAo|quDxO_3+(4{d$Z?Q80nA!or}ea z{Of_yoa=CV>_&KdZs>Rv9A-?#2k=jEd$V$jvaL2c)KOOCTVT!*ccEKUzh1l1WWym* zlm+9;0~I>5@+|mPn2zlLH7UCXuFrLtp#wm^;&pHth$p+Pum~Rr9wed2=D`O_6njRN zSa{-sra4^>uoZ|2-X>F2IM_sysnGUtfSa|qqB?IU^{vA`ligVs-cY38+?6R1>S@0CBv&9_w%w@F0gtdbvz`@J}nm8+p+-S> zZep~!#TsF(c%4PrMtFf{u>--_aTPi}VjK(5DV61M*(AdVw4DV95|kGcVO&agd@dV1 zNleW5+B{ZBN2zAA!Ru$W%8x8nAPqiWw<>!?9><`OAVEG?v{6=U4m6(p&it>Dn@r$U z`}!0*9R^RXMVvu+X12P}gu=HE2@bCZ8wNEP-Ozi|J4Tu3tHBrZ6BV?{$~5UApGy{B zN%=&@6ZlCHxV4xlNeU1}Xj>_}SJb#!MmzSU$s$=lOpj*S9BvC8enlqGb2ukuH4Mo% zQ5=e$A{NCmmgi@)-O?U4wb+|%com7$#*G~(&ptUX*N6j`>T$`=w_E%T3=oRY5rj@i z|2M>@gLFNU&QNKtSXo;|#W;VkS?sA~*#!FBg8eEs9ml_Tjhjgfc+oggrD>bxaJWFa zz%AYW8Ri6QktNHS=c`qmMxZ^2C95dM<$!we4j@WIIm$A{oP-x%CtW0Ka%+P5@9tm) znV0W`F^oMbnGoQ%$^uayUuJQjt7!HoFgPtY}DB zPdcOOW(d;eDjv#T4*SeDc*psu!~9c$3jq0rnq||& ztw>WmnjhV?NIi=0%r6~mvGR#UW|}7RCRdT4ePlG_2$L)t1!Gb4N4Zqd6jF$N)aO2S zlat-NDBm|kUs%23vE(GTy9f+P58To>Qss95cag>w%Kv;zz5{fq4UJNt13*&a{-YTU zLnG)s+Hy>xx>n=~=BxzvCh*)Y%_ur>A;I@1=@kkAAS$94ziF`&Pp(x|>|$q01({~{ zv$mx1_s@(}p9xGApHB?SdK5cL;!vzcco=aUn?OgVBo#@n6KbZJC4PY?pm!yDgC|a- z+ua@}k7(!-s)BJs;|!HdS};#RntPrX9MH#>JQM~FO36g)g}xUw1St)3paI7ij%ZNg z^%Co$jwI#7OO5bLqE(|p5gVPCg2ZJSpbG@w3h ziUhGwnCS*j>$Q92yEIRbYEzuE+tR+GDwWbos~M)SMD@iMGj3q_JM&`-v>tJsxkcsT zLd!>fGd_jkqr}jEAcSO*hGPiNDZv*__;NCbiT->ekbGtmDqbh{2Yo_9A5<|wMgIs4 z$ii~9(vpn5151$jyr1Fh=fYEF_<85pAby_x$>TwJ`hs$F!k07YWLoNP=)s`FhKMrU zYnMzYWf|&e@~4%q>~iP6XHhoCR|2J*k49P)_EEOA0P?B(PHFc$aRL`Yt-gVX5?&!b zB{NfG>QwU@n!hM90iCerr4O}8VT3tgAWQ5(;xXc+%yLl$1Gu^O$nP z&ZoYP9G4f>EHtz^B>S0_z|~-g`^G(WxjJCdVK0i)P@JJ)U&AoBi@Hg47s)__}s1Xy^xe^1bfGbiWv)d+B>AzLjQpyxdHOv`8GTHrp+>H>jK zzFIeq5WiAl3LS!lEb%!3z|hGQdT>>P;_-MXIVy!3jpSn`A?~JiwyztO`*l zCV@HNp?n^q=(8$9hAb6u+e9yxEGQ-K6Fjo3kf4K%S%K1!mKCO1c0SVeB#S)H#3ce+ zQ0i7D`NZjDa(Hq@@hysOyX^L&Wk3SDaTpo1rH;!2i`xF2vsX%$Np=>DeCS6DOd7K^ z)QG|W5$~c@7`|#0g)@&F7F&^3vILw7$tlV- z-)v#RhW3_2G-+~yFBscF_2@PrSvD(-T4`iReZ%go$ne=tFSN-?)IWj<%kattLv|qw)-psnBv6MZmK}-dH}J zIsl~c4)(At4i$+=(&&{m#czRz&K0S<@f-$<t1S(z><4^7eJXYTLcy)~20Y(G2EW{e=AsPx3!Bdb}4dsUBz z|EjQHR#9OazA2i{l3hik(D$pSRej5!cUlXNlxyq7Fhx!HT?A64ut?MXw){`xFACaO zU47&T2mT~Su{QalBfe-_s_) zTlyg0$GG%ev`f|_{F)-lCjIZ~6J29tCc8&Hb(Z)qp5XXYpJJ8uWS?)3b+!yq-d`ew4X z?ENq2bXkdJkyJ#P1R3kc>iN`hf;znT1O7Q`Anbg3;c^2HBUA)yrruvE|P;v@gDE8Ye-VKEtk_5Wy+=a7${3^W_j_KGxSfwbUaBx`C4iWhv|kltG?_z z%>iqD>M;9c4}uK4%?ZCn-_7zFOC}~fe1FEGp`FHtgb~rn!0-5Uq*K4`txTn^>^U6E z8m>VU6m3K;aUA0|H#%xe-+w8*RTBRl$#5A?eEdg4vbC76WuY!qV=46u3@W6}nvrSN zw2OS)+b86QIK1-!NRueWS=D z{j^6Vu?rbbng?nr=0e9=Qq7PRqy^!QnYN$CiSg@Xn(7rutkEg0fm{QXaw-jKP^NnE zqDtl4;)fR*iN7f`GzQK2H)Vww{>N%h%wlU>tkv3rxq`X2=*JXhL05RdCeiF?%)%99 zcj@V;k1%|jl*N+_p#U|dD`qC1Ip!}pY=ccN?|IiB5Bvj;*P3-E3$A`nqWFo64;Ksx zh#gq|P8M+1e(!N@d9U%_ohSi4AVG2E{jSqC(Xm}l@#KjwVA4E9Z%EoZO5)2(2lH&f zS5TLFJyHc{MuXSg%Jg>%T0b0BnF&AAQtj#%cfF;`B5lt?L5CH?pCFypXTAO3s{dG< zox^6g&tSTVc$H7W{+xQu%9qmESl2cpIai)q5imm^v3um15PbO`>d`}&O4$-3mzaTs z8dZ!&sc6ClIHr;7u|u@U+wWqUT;5HICG+<3zgQ$7yczS?IeNK$nc03$dc=I4H+?}Z z`Ep6phlF&QUW2Zi<@Vb92(fXe&inF7#Yx3U%vgmz8wJtSP|iw>ilUzx5%(ol78qR1^oMhT-D+W1Q-%a$UTO9fp|)TVl76BL!|LM zSqWW-q3hrKP6amm|9Q5aMKN7!Oz}3JEpx*GcfDyT8*Q-=B?wadI&mM37@?S!7_XS8 z7`B+Q7@L@*7^Rqzm}t)^`_{ zTq}pXftdEl_1HpGA(N026$dOxa0|vrJCMRJ5aI#3fgtpJ?!i)enA3UZftRQSO73&L z4!e}-iNiSsZ0a-J3KWY!E5~ah?_Qo!;AjD702H}ED83YatF7Km`GUAXI+U5F9ZkDohn%?w}{A6tNT&6-5eA2A~0|Laf9z z#JI)W#V~V`a~Y@zR4`OXRZwtHN-;`NO0lf~R^Qjzfn3PHKPibn81u-@_&XcpKyD~kfGfnSzFNQ@ zXb)o##fx~CC>N0(iye8L+xWu`bw{@kB=}5(u;(!g(d+e|T@FY(ghAafI^^mDsv<;2 z>fx`lB#pl>11bC4K>=f)4cF2~jGlRjGB5`ii?9ar19(xMacn5`@Io*k(~ysldWb9J z5>gEq1PlTh0h9?aUcfrY0U-deg78ax-3o9G1Ovdp7(fj09Kryx0DyA&sl-`^#h(V|wR3!mGs>2%O58(#ZLUbW*QcoSY5h!m#3_bq5z$Rc5$OGVE`UrgOr~-V?J53$| zDC$$Ef3OkO5IVqoh+Zg8L{5ljtj1^!X77xNjrom{jOmT>sYsSo@m0VoiwcV!?;5kq(rVd< z=><8L^#BQV&`cD-fN&tL>n{dMBzNkB2M2K>fz~;=@FEAFx)}!z+UmiFfRY}AY%?}u;;vC5n&5SgCs*VA#(trF=8qJ1gHieBOn8j zL1BO}gfL(j$Pg(S@IhP;r3N#HyyibIfnGyWgIq(f%V8|M0q#NWVQ3g*$3DX=O!wyM+CAhmX zY+bD!Z0)YnB_aBMuQ-Gl5|{T^8BhYqK@b6DB4+~quCb)NE+W&?GB^nvM01puGWTR<%sci=O@ z4T=ry9{-+Yh$O@l5Dt_DoFe?ET6{(#1CgP0VBZ0p5YDJK=r(9Ou(DrJp7iZ2tt9Ly_IhH(#uUaP(5}aJe0E}XBzAmuFgy9y8AUNL0BHrdg7gRC`2{@{ z&5Lrx?^WG*I8L}{Z@Msg7$Iu^yE-Z1?d#>{fVu#FXr+km>d2X)cl3g?q>rGF?1$2V(1PBAe220juz|ec+oRZn4bg)vLnwiwzyRbYggbmE6eq$n(hbiZ z;T}rJBSaKvhBHxxf+K<`gdl`01bjmBLdOyPhV37>u4;&=jj4^GjjoMcs`&KsFn#~g zh-EBie%gIrPC)=DfFOVuaSQ)+9E<`PWX>6L3SUCU1}eY30d@H{kWv7jzbxP~1sVY{ z0hqryP_3)W>9Zp;G0c~MjdTT0N_mRW9)P7^u2h-jKkT-(Y5XyG66DsMDy8Y#f!!~< z$mTmg3mseYg^%fQPo*?_0CSE<(w^F-rOwgi8IzJs`sju>%evk~2u zHRYQQOnSt7^dP*mxS}#?6%C$YI1&AMBH{tzJcJ`Au<_D8x_U!U zuAa$TC?C>=QjyY;1YPVm4{nz?w9q@01Mwn5eJOYP`Zu7`tRXLq5$mEAna;2WFdSn~ zO^h|zh4|&>Cw&gm!@CKipiH!5zf4;u%gHbaUv&Et?Xqzdf_xUp6Y)L%g*j|Y@+~LP zHq+AAq27PK!-MkfG>P=mFVZ+Dh_u zE5hmaM#a1T81_&!em%@pVjIs>nNQi(8B&;N4tgfMYR*;OF(?xp{{r{UIbbfR;>}};4xoUI?}>^Fq8-y;_2!ZXi!jXW?#10h4s;rFCX_>--rUn& z;KLFWXK|aQur5725c!E8zQyMp7yFRe0~+2;bcYBK+YeuK;CRzrNWj8}z)(G!K&k92 z8m#XtX!q1E%Yg&bP_tKJ)KDlihelcYIl2RrpUZ^{Heq#2gDpi$YH#0AKiOr`#N#5- zFg770sRy>_YWO|DAi0cb&)Lv9fh?(qY0nNzE4_{9r7{v*gFK1_WQj9x5tw>uquLr*D0YUnpd(?}!Fa|49DrFhNs3(GZb{A@x z?2urHX+s>)0=8l@lvXSkPnh`7C9$Z~->@c; zr8}#`g0Z+J5f=c=#p<+RH7*vZ;c`Mr;t}z*|zbvdn{rHWAhjPH28v(vyw&%ElpxX3tf-M_f7`BAl=c6|uYOqznU>9;Kn-L9QtIJ+g}+tZV39d^pv-_>0hkM?5X9mdx`vAz&+~3npxAh&}p5+@m<$DF@A`2H?>iVD_Zft$e-s@y%see0r zM!&fc^##R@I3E%x2ct#IuQ&>F@Zn=ET|IMTPMtvi`Mn(J9G(Zk$()#HovX3@I0E>} z-mUtB_+)rgm;O~?0b9m>^JH_3h{Jz!GXH;j*p>lKR}otR2oEQd)p54q7J}ZVN4r(v zJi|nD+#$F{Cdd}+&v=|Ib_*877H_X$@EY-Idn+B`;d0V5PQ zFwgJA5>m;wk&aTOy5WOZrCh^}(4-V(i)3dn-nx^gZ>drhxRDO3vfX&VsPf%t!>pR# zAOTm+)?B~QFYj$6YBKDf6L*S1p0{-BM412VME7(OT-q-^onW;%*2mmUhsSov|&tY zzIIY#x(1H8!yun|3)aN4f;cH!$wz!)HEM6`$hU=DBp9}&b0S8Jig%e{0wY&fE-u13 zBOb7*Akm*cpAg*IH_`wCeT>_R9Xm|pW;Er zDVF^Ob*Yl=0-DRBx2F7!=Qzrb-fEW_PH%PCHDvkihV3$gYFzzdA2M$a7n6aYq$fgq0Ia7}uMIEak=frc za4^v5m*=GD&FRG|p|b(3_2u25`O}Tjx%Pk&u_Iix>!JGoATYd~`F_89RN*BqL1bLs z7B4XH<5uaAGBks8o&6ziGFeuX>*C9mG%c?`R8)6ZW0E(B$XNqc%J#2hbtv#8N>(YV3+ZaO;Ivby zTUxa9#}?gyE7!shawyj_PzY$diGh&q&E`_~jbPgC-{oA6v$-|``6r^)R5sq(|i zpuBAC^EgcXD3F;k)F#`)l`N`MJH*^tJ6Y!V`ohrgc)QWiaCaNj;TNBReStd1G{6tIeKdJTHzbbHybOOx)p|(_ZU@w_2GlPG)Nckte9cTdIoy9a=)tzt z*8uLyV%Ti!L!2F7?UyqDwAJ96f#!jo;Ekwai=ubcQUyXI!GMamKyP z%Dl~JIyh8gd?hIC6;OAhe!^09W6xba(+)0VV540akipsohCIFFPamj^ki=cUL7N|N zducRa;@Z*_q0oz8rU~kJG9`@1qWH&rNq&k*nqRWBj*8Q%S`YiQ_u1LjCq{XbnDg9; z_lO0}seO5aVm55f%@rIo!FC>`_x7W2Xa-^7Go#?SQCCGaWuDGz+{$W#_20idxz7!h z{O0kDkfcddS*O!{Zye_iJHf5+QvXsdD`{puc`HvHbbs#^da9|4 zOYgZQBBYl>5h~wKq>)k~1%1JxL+bs4Bk{iAW8B06YEop#YP9BOtFaJGMXS{iO(m=2 z5KU!mcpevKfe8}H!B528+wDDnejlUQwUmPYNwt0#^08DGZ_j?doAx-r;`g4jzY@q^ ziz}JGyz6AFR0gBHj;y8c74nP)-=F4&tWJjVD+ZEs2S-+{wIC^d>S=_1~s|C)Xt4Xue!#?IRYe}JX!KsDkc zrM2HEu@;!TVTsz)rh`Z$=+J$a{x#zb5W6#AG+T`$%_uU@p15t4{X z3qNuXO5lTMxnS0%d3niU0vxb3esyTUi=?GeY^k=?7>!VrRE-=yLBv-m6M^**O^kLS zN7OH0Ga6}MF&qE%(hDjT6bZ)30Mk`^PpYWW?bxqSDt-l}FSf8RT^BB_r=znY99h+d z`~(bx%tN)2E)_q>%0CSDZEJ^+mD7j@KjCD4iXA@Qg55Z~Jo2^rXRe5__RPOJPMm;P zwPQnHt#)kjzysCm;U(2 zCBMDB)!9+&S+Eu(cPgWQDs!+F!_`T;@574@c`HW~FQX0^q7nkQL=XU^QIpXS5rN-; z)DRUhA~Ebz)c?QUCu&bJ@Bee_Q>+^Yj|#&WSsSnjAroWmLAL@8fffMw0Kp!h6T%XL z{zT?#ZSK`R{MM}Ad{HL{l*jkON=Pqv?%iS?k3;$IOgqABUk z-NndMt+~a|fWsjz^nE`yZ|6MsjWYkazCa`rm#IGJ9Pk}UtOP|&_CL{M4s_D!iB(Np z<^*L<-xPo0)#m~g5~=Kpu_@28&$f{@O3;*65&;n#kz+=x@Gbth{z=?Tr5+DejVL#j zF!&wP7K!FwBSRXu<;#1rn!1nAjj@f0AEizKD~olmyNj2~#nbbmPLzHK76r zTM>Qk#~}%uVvi2Pw<;|$ktWg1H!&v(>o0(WVxW*ylx{T41K>-h#KRON(FCP3mig(8)2|$-xoBpw41olN{^C$nh$tqP(waNGgx- z#qxjeok0!6+wSSxi+kZe&`6$8%2|T<%ef1U4Z@`F?A9Hp&ok@d&A;zbugI zps&LZiLLtfBB{gAoQ?8io3_IbpN(>VyP(6bA*K3YyWIQ!z))mhbpGsuyVI{CrFwhY z*7sh(P~`9E{Po3kho3*&de12D54s1K2(=Re^1*HyJX#R#g3b95` zg8+(R>P}!iLNliQi%MCr(UG&vKS6cO*X)2(EBR=5^DCGN{g>vCQ#E{uCSuHgRV*uQ zsZkmM0yJZ&7y+6o)R+Lx1gb({qD1jJCLtrr-`vQw(PB(P!T-9Hr8>YwXZ=)ds?Q&A z7zj|+cu)Ah-@!K@-dFY8>@U4np1+B9J4W1UvG``oNMximugu={tt!vhkKRDG_s-ad z<>dWvvGa=2H{VY+6rTq>>C3IjZj+6PR}yd(M~yp}S%RHcT}+cl4q*Mr7Xx7-sc>KeUJYtu5~gwTrB?%x*K2A1W)ho-Ni}+L+6oBx3mIsGH<< zXelYKhlkw-utlGKhd&eE!Z+6VR=NwHWY8Z)(5t5FxOWNR1Vr;i&N**RwEEx*Eh+Dk z<$RND@Yb^d`ho-QX8NK7jb{2<{f(R}>mk1BPS|)Sq&?S%fxEZATL$CdN+h?ko7qe& zw6p24_+-8e0u~FW>CR+u!S34M!s;!aJ30PlNVn+!L?l*d!ymVta>!1gW{+4-1|6i( z8|F=2%9G$d=|bV~7FpnY)MB%x(>U=7Z@<4~%PwXKVQk>aIOH&vugjSnu0c~-w52Op zRlnsUSk<`oQb;j>>a3{hcPsRdLT&{k(wFJwq`5xu_EGuf z(P8`4aURE-jqzmI^pVdQ8+fv;{=7qWoBHorYb7|y_peho)=6+5yy1xVqDA!WZA=Zk z+%As(B&tXFO$+5!4xII_`*C^o#lCwHuGKxO5n%U809Z-8N&NCzTK(l;b*dAe^tF$0 zMC{xn90~aVbS?GL9QXt9z<4jU6e*qPrHS>qw3aK%$`2aebmTc(GBSLJ@AGTSsGalfY{%=84AN z(Q#QVP&cJnXVObY9^u`$%Rr_M?Q8cv7&9uSuQsr`Dx3bvP=$M~dTGr~%<<)}#;r72 zi1e&C5-4$B@>2bxXruZt7~w5+Uw?A$9BhKnX>E&7h6xhsZ3>v`Km-rlww{cJQ^9n{ z@VllU8%BaxeqTYUPX*oGTHWIB@#V{Wjz$P0$cTB{H~ZPhg!b#V8{egWA|v*8Fzo#X z71CMtP0Q0N4SRCglCw&vXLkAbw*dt8Y=o8frd?*Qz#_cIgGvUbnJ;W+ImBl%++a!) zUG4IPv3?xes<#EMTwkYK{fk*0oqoaZN4~2~^Yxt$t+?FhhWT}N=WqQbz<;N=RxY_o z&GOF2ev#Cd_vg?8#0cZoK_ek6J?hl)0d!ZiAMG%$NAj=3;Jzp;GnR+|Zo1r{AX(*$ z5?B~FiR<6plQ7CH^NvZt&MWf?1hyT2vY6B87$8WXBI2(A<8qbUrW>|k8WB}^O(~c z(r!&jO(c=DYfZh}rSzD?^{h(zS%OM!WBXqT-I^9!ogB)F=r(kFj!9O-?W+D<;v}ee z9o)*?Q$O0`L25#vQUi8;$$uCZw>#w6UA(e4seq~_{JEGzJr+0gqoFY;)8^x#^m-Xeg=jXNe5w~CNlJ=w46nP1p&hm3g|uP>ybEe=}K2X5HNJ z61v^V$4Rws+Bf-Eh@WEpROOL`q^`^DPbtWljGm{%IskFR<)KH3_vj5-+Q)|dUrG$w zQijzJ0dd4OoP4KEWu6GiH>7Qa3_-YT{LeK%kP&Zwwb=(QVSDH|H zdS8cQ1hdV82bSlWb=8C&-Y&k|OYhiZ%T$xARF0BlhYUvxNsMjg}%`Ms3Na^>X(6g=Xfdj_s_= z>aeom@cG6@nlYiYCD?A3eJ|?q#+~x3B));nmmp8T1MNKxD`60E2WMKEH zycM_;BxLoXOX(=(M#a7 z;~eNy$`zjpaGpIu{s-ubU8{zHEthp8q{_xPv6E4~nhp_)uZODvR~ZwXpUz}UJN!dAO&xjb^Jx(IBI2-8A^Et&uCeD=ewK?#z@QeCrD6D{Z*Bfb=?g+_$zSJHM z4sOkSmA=&^=4L#QgWk7n-jwTdc3q6cz@X`L26nAA^5n69W{}oD*p@JG8>%ok=^ z-j`j9vp+J^p6ZSRe-7u6-_^JG5NXb~6uh8C^F;}t!10YrhoG6aWiiatKdvD5@(B=3 zV!d+T!q5MP=Sn(vVoNdMhFd?~C4<5~3tk__!tAm;;LC|C6_#WiylCaZSeX4yX{w~< ze>aD-P&5>T9bF1XTHboTyBe%aw&wK(m}CgI8P9Kt#rheZj=e)EKmR(oVpxD-Z*vh4 zN{^D6>lrr7s&`J?5nh(yi}}p|S(Yb<$jzL+;Zn|Hj{PG}Osb;r8Xvo%v|-i9C-LEN zA?Gk{(^#RoA5qH8Nm%z##_8hDgyyfJvTauO88X}ZL%rlVYb?n8L@u{atoGj`-3~5E zi65fUrNGr394J>q3zV#%$w{gin#))oTzvVIPr|q}HW2%5m$vQ#3dRqHGIj?mI!=(_ zH5dA~xE&=Mu;8_?v=MVnP8X3h>`gwGf!vkq_WSQ+aEqocIZmCj-Qb~<_f-_Y;I zgmH(^$6u^_Uz$*ke#Qlk9iQRGAITKAL}smPKwtav*H|dNqi6kzMe7ZlLq_!NJ^_9i zw1GnCOS5pgh4*lVDhgy6bKbx|qjf;*IlSFFO|a+rdk#+5$Dg#y0X|K3e}Y;`rmO>J z%6fhIe>`l5OzlvR?~S!O%EW3yt40wRPU@UO)NTW697&WZSeCWE2&>B;Sd5cjKJf4D zk_y{``O6q$-%naQuD*Jz^Ts^OyXzNlfeU$h+%Q}nu*dTw&!;RFw<$2ov zKFST25PFD>U-?gd`exKN(PKKkvM~^Kp_6Dp?NdK_y`Z&Ruxf^=D4Iv#*)aG)3klcn z=1O7uTETm7oumE}ZR6J#pRVHmji^3TVMN0*CE#gV7~m zLXTiqHaW^ygYFl*Yi5Cafe*pA01B?6+|%8&E^zNP zm#;&RzhG(Kw`NQM&KvcF=x&)kxo?eATVt2pFr{CE^z}9fyvld)E-#(PU;jr*8`nb% zX%VSWmUqZ!0I#>lzPIaCNxv zRo)2wOu-Q@z+2Ec{yUdyU)`5byCl#)NH!I3)+7uj@T;Qf=KV-^)EL8P-~4htCSQYb zSkg7)eCn%>Oh&Z=Z}5$rHueQZ*uL)**0V9c6$o`GAYd^SBb` zh%|ws^V{$7OAYTFw*AIaI_>DVVL+!0HaYQA`0>iPsPotPT92wZS&d)Mi`FSvayylS z8UqFg`ShY9x2?(6*Pvt>~|in z3(v*ZNpdT(K7W!leBSvNeMGPOgS4}C{g#EhaH+1Z4EsaKpO13_9s9~Pq`2>2-Iph! zji-Sx&$HglE)V;mzKatthtC~PRy+&dPlDYq*Vf?J8TkGFk>22pR6=%rgQfwvRrja8 zj-ffry-s`exnE65wSj<*cOVY+VV6($%{;#M4=2W5|1luAcD2=e_m`TLrIO#er-)j@ z-;9Ds77Cz{&A@kVJqO$2Ic&Rbo^}k6^SxU$3;gn8a zrOT_au#74ge0fD4cmA49w3lG{J*8_(E>)?ZbO>omH~GnaN>4f~JV9C2OkoLmKzDO> z>;_+abhs#1Cipo#22D~H5vigsv&cH^0$nWed&{?r&Ra)VUnSL>k-8N6#CXk$#GCn| zW1Rl^=tOrQHEl5*&d+&&Gh_bSn6e|uSFn!eEc(=UA6JvEY-qrO*q$-4fEvHdz~INE zt^)EAf1m*7)#*{1R>EV$M0~G;jiTN(+1+UT=6lX`XV}*5YV+Bk+H?lE;x0>9q1dqX z;-EI!rmkVIelK*H&n5rY`2wek!kmczVM>Eqt`-{>$&}$znCljy)yWH7g+(Vr6#GZIWVWp_b3;>uP$M+!r?JqhIsR-lR@Lf1WCBvw|`-t z8Smowkaol)diqDxy2#ze>Jl@jr3I`_hIL|V`bdqHgqyB*=~b(W4PPe<9y|@LrH8O1 zFtjYH=AcH7h;#fG=*m+S29IqSZ#bpvJs!amk7eO()dD(T*y->dMgJGnVHf`QDlKac-*hy?Z&&aWgly{ zdQODyIq&>*5dX|millPc;71?kWFFIGbPPa~PCEs2@fQ?KWW$PoH%viaT|fV>21+VM zHibG^@k%Qkz`6WqoZ<_@6(Ua>L!MV^oHVQtO%;*;b=ysJaBu+2~QxJ*cy{1^OiDxV$>8m z5h}dve zru$vA-T%X>c^q``Q@iX1aZff=Jl4P&SQgyPmdXA563TAQ-%EMAUM+UK zyBt>)+F#9GD|Z#|_A6#{_PY%Z)30*zb)#F*3p`$T)Oi5Q2I%7M?it8RGS5eq-R=Qj zC(ZAFOIq5ZM(`b$Hh)sAibeNpYm7;!vzXBkweC$=U(bBJ>NjWpQZ)AK(0~{Bi<_TZ zkpp?-*Zal{LfW9+Cyv0qyF3DLE3Bh?GbNkE6z;UcZuwzBL#;lXhxiXv;W9n&#+YQG zq<|KCAWH+P99lRbE7==sgeO>xO~?A&0hc6Ih!b>4!z)%yp&GCBnfEm$1tD`M4M#S;YaYeUXh(K(^aq z-R5(!Z0 zR(+OG<4ETh6e7LZxx4aj_%E->{nMdU?4D^(z|Gxz@-)%ig;PqmZ&%PqGXnzdw5rNU zxbQ?uV{)jw=tEuPpC>o5s}gMQK5nm8!)_9Rsr`0aNzV^R{%wLjn?|SJoEU$)6`?Z9uv{I#fl$1nSymHuN*SPjoxpR3jook>*Io55P*EWiHd)XM zg=^kuQhi1c_ODEAK};8yt*MBt6q%n9*2*ShlFJQrh3$g7^N&4(>0CCc;G4Po#*g5r z@UriM4l401m|oXf1#`z8fZLq8gXNUozfaG7%IhK^-wm%c%ja5ZcI%T$$ojxJQ4Q8*VWwHjGK9=JoTx){3EK@>IPKw z7T_pJ7x9@pbrIbj~VYZ(js5N%~Zd6ZJnOWJ8PM%D9PP-#qiIN_4KOfhRFfi{cW z=!O`}Lv6CTn$6p>EnU6B+5BOTk*xK;U&T>mq1#CNg)i~GHtZ|0=-oSlwU;3ILo+C2l^)m;+X3~XY<(4PRF$DlBU-C=4n!()ljx2_g!Z%6=TkR zfEFc6tJyt6gBbYor{!X*sO1H$$^BGRv-i?qm6*$V$zI|CytL%OyD*vX=~gP+5%^Tp zpPHfQXfwS1G-N5s<677kk+<4zq?SN}s614acG_U#8?>n@ z>F)_0i_7FL&;DRvhy_qQ?x>X1+~#$~3MPw+{G=yb>sYF#{S&VCm^ELm56Z5MJA_|4 zfq%-*#dbc>E3Xt!{Z!sL7*%j_=x6tPX_AvS$2Sj2{_{yjN}fJxiZeeEheVTv zj>bF*8lUDTqbi}9sQQ+g_zfC8Z9!h%=}C3$>Cj#0xa<17d-gh1Ytj5!hon-e1cW<`YEW%Kg7asm|bwoO6ahmXd!b~MN1 zpI{dTLdBtb;-nufbMW+gdN;ao9s=k=8ocn}BLgN>m8G`_k=gfUsq!NMvi<*%*8Z$N zzGZ5#8PUV8o#-@EajfFU&{9C0k&2i6o1-~H8=DX>LUBjc68der*IZAOc2|e>*;WLj z`;{9Z_VULDVdukmgSLMvD9YL{x0VUgI1uW1lDaO$ND6vFC0Ov=Oc~tLxpU^fm?~#n zP_7~@Yd>`{&d@m)W{AyZNzng8lHVQu%qXaw`b`Tt#Z~u$h)FFbA`(o3cpRwuKEyDmY57l)b+iJ+Nzy!bmmx` zbbo|O7FBm}_VKrLUx}BR2m%?e9F9aGk8O-gM_zHb7`==KG6y2mvfV38Y+Szq%72M3 zx4T+iIxXlw-s>FIXM<1ih)})ZDZ~V3xnuV9o!dY((>L?&G4yDq_ld>XvoYv=?=ti% z2O)PQw1!~|<$dFsS7xi$s3XfGMMiyor>&-WlX5*_-Z_r@HhA$XmJV$jh-j0cX8nT2 z_3Ve*b7?LxjR6j!t2T0%mb)X%xUZ<}Mr}2hf;Uf_8CvEmcl<=)rvlZtkj%V{uO-3} zu2~v$nOCXE?ZSR_DQ9Z|^XMYp1F}8OcP__cLrBZJHUaX{=Dn9qv&fs=bQKgBav%Ei zJtOtiI&ASVyon2-GQ>4piNVLFAT74o2INmjjnJ5j;LB z4)0G(c%8TD>co~gx>t+$in#V3ztQ{D z8!cXz zQ^uIaAgfYUIb}C~BFz!hskCt*y7+m3;d_sOPta$!M7bLHw{QHP=dPyXcCP!*cI2W@ zY9Qxau_u8_&DXu-jORmuc>On}=LRHg0lVSHYNHIP?q33gbxvEM{}?njj3sP}`UFsL zyI`CiM|?f@Yy#RZ!r)D#rFO%;0?}3N{KWb+MBjQ!qm<7m+J~Y|+jBt3;y9iQIft^* zY%JXYifNV7YTNP|`o=I+$$XO8_A?1{= zA_;Gzo_prR0jl$=4w#{;e9u&;BSCc0N2jH6 zsTwq2m-Zhk&LV8+6&>{&;aG2S=>uHvSMkV>)=5fF=R^2uTpG>B6>t%_o5@>4`6UeN zDffR2be`%_w-qIYrpe+Jc@VKXiLSHF!AUS2qCCN7F2;@ny4(wz(6I_M|2=8MdV}HJ!H2mGruX~E0sruz2y)uf+0%w4&mT#Rm3QV zC=_ku@YhPF;V<#g^_(KL%gOpK5`X3uso=sd;~5f1oupHdMg}#Mo}+fx!y*#j0`ng6 zCO;4$8a3;BRSi;n{a*l6K&-#C-1+hQQet_3e`nDkAv~8}xsm}JcW#=B5(!0jRrU0@ zlkm}WmG6^#B={{^l2?Pi8X-8fZHonR$di%T9hLyYyrvROM{Qhcv0jPVN_O}7g$!=?-Y=C?JSw=i2hjpY^lUP5(ZY?8?UAbh;m4C+Rf0&7d`%Vn2U7>3SLTQ5wPio zvgjpq5*+zVHJS@pIMGx+NB$iNL&?1FOMU?ix(oUZU1i|Tv+{>`_ES*ms}z{hNkB=V z^ow1{BR_mLJ{BEh;I&e2#BJnjvx>uG*H&>rYCKExN4{@gB%L(lCV<76vg282kh3@X zcB=m+;dI5wlVNQJ{wR>2Zv7--r}yk-RX30a78Fakpug4`7-}3`NC2;ZprL^h1H0=V ze}8e8fuXDJ^ZLz_YY zx6CgG*lTn%P0^f!iO5f8kM#){Ij?$|5A*l?Df7`;ZI}~Nk3}<#88~*gf9PZj3Eq?D zTf0dTw&ZFTkpq~oYW_Jc{uFe4TC@Gizc=9$iWgal`#Vi_mEIwo&%06@+rm2;c=vu_ z-a>pGr%(Ph+|y3MPtL&RWIqXI(pw@Yib*K8YN#%_PlJ3>TFys1>?boJfeS2hUcI%7 zel`s?F+1je+DSv*iPFlbE!dAOPp;c+1kn8XCTVsS2Z!$l>8^Gr;Pe8=hgEqrM756i zCY0bk>=HO$^pl3wQ!CwX{UYG@a+eJg7Z~`kMR{Q(&S_cRD4Djj3hZ!M%q*l1k zY{k8nZ!3D5fakORXz|gF-ZbQ!Cx<@C<3RVJT87X`2JEJN67SJv!AObse%fv1RC<9^ zeHHHa8onRS?HssAS8H#oVd1R0cjiY88V)(lFWytZf!2nIMwz)Zh>OMP+wURZ^3bM` zbI9vKn&N&9-5fl+8*jEr2jG>H?vi=vW4j%c`0Ow@-p+dymx8=6F}OgC+=4uN-f_EJ zI0emmsl1#H2R^epY7W^Gp!ZqXbMsmf?yFr$Nf$-V&wN`NevpQkXmhdU_*^U2`SLA{ z#$N6?{$J@X3jTAq9B|_USbiZc(Zhy=G1Uhr?_;V5DFX4N+m{;O+Sd{hX%<(rcIg%ASHZaR8o*H;>(db7VzFdRgg7A@-D zPC<$1H?a*jX^2>}?D*Wh9Pm%yGjb#v{j1J-WVI0k@6R~aUl`>;%0)8h#AXKMiFuJB z`1eT%)5dg==Spu2W_&_ks?b%YYca0}ZHe?lK{OaU-N`RbBw_K<<@Oq?93*|Z@*mwo zz$d*Y)tk<;;I{j;aiKi}!O~}}6~jpQ^=6r%SSAC9dbT;HbOO9tqE!9LlmW?)^Ri8k z0BGOyJU7RhhOScWQ=-^Uxl3&K{zi_es=m^b9tm(k{p$1{yx%_;Hk^NeJT?DqRetCT zv1^a`ZFXU~3{ZOL1soRZ# zrEiRiOJ1^YOL^UgG93!mroH-WAWcD6;j%>mn+TY8FL<-!zn|OINVUqaC1HL^iRu`O zpZ~cgm3qK{n(nPH2NH0;|3nO@&P3mGTx_IrgRW6vbJJ$ugxUvZYbl2{l4fo0>Z(U~NCj#gPMggu~fXRnQhGXsLnjM1&16f9$( zcN*T~;J0)ByK^f@kk58&9zM#z-IRN8S52c}-hg`ywTFW>auO3)(VKnr_0>-IP;kSu zA@7eB3m+t##a+?=0<1_yO=S+YEiS6nn#RIr>f(VCAym(|@z6H6v4IT)$g_96}%`fj0 z?pfvdlG@GKr+%N4W<6cN!efQ9=vUZ_U37kBJ9#@q$mdNMngXMG2&*$L7cG;A{Z2$wd zeFt3C@#~L;nWkbMj2w127negpbX(D$+em!p-~9r2 zjK^?~{p=$Zzuv{YzW8|KKJ>Y+bt@x2M{pqf@aeW;J{EL*9HtmA5+amR#eSKfKe0Z8 zKRFh5?$EBb_(;Q_i%f{=EdamYVPQ@B4EX!)P`&kt1M}q%W+q-FVamz5b(swX_xn0B zJ^!#EG_JVVK!F1}*CmZ=-*8T$Gh?s@U}f5X?krsjuGfsOlbBBeI|X0*@%fB-yd$qH zWS~LjZGtU&%N?BxUiVTC2$MH2&!La5Ia&U(A&r6F@9k6@gZz_~cf(JLfM@pCa-^4F ze)rj>6yrSVbV)GRX9Ac+-sK9fVebBH8hqG~z546GTuVQ6kxI@^x@Frdb};|T$KAKOdnI059g&b+xh5xr;doQ^EsKc8{#rZ(h1!DA1f zY`a9krlgvp+a@H4Ijhi|91SxrP3{TeBcT@Bt*Td1FkJ7lyB@!uSg+J|olzDREq_^J z?~DCGtw{A!A>h$?s+WK^4YbPHWfIny%U{~>MF_L-wOKi4g(e3j=S*G&{l~(@YLRDQ z=)t7G^MKwK2Cn+-y-1pA*=ewvH!U zoCD#Os@_@lH1HeOlmwy|n$PFm%sknG&~HC}vH7%D20&%n+{Q!s=vNXS$5M8X z!2j{^Hd*xZ6sxPDGjF3O%pCu|Ss#Ga97^MVN`S`2V&^i<4b%G9kGH*G;gLtb-GMv? z5<2Xqif3{#xj@}S;xPK2@JaqF=pTQbejfTT$U^*qDwKVFV7L{tU?D~BzFtWKjBzBTGzkDR$Hxnya&0Rgf~ zi^g=uu&=i&#dSUbuuT4)oFYNOpAUX;LxF(phI2GFFU5Vgi*O6DMIX4O?kF6Dy>jh= zh1mz>4|R!gKPB|6!guHIBd<$pTgK`k*BjZLn>v8!CvJVyf8-4fpBkC93ZBRZ_c{b) z4g%ai{UU@GVd2b%$P1z46l8b{m)N2|g?H`PE_xcz>#0Ia1$xAC8;`gzpE&SaJ|^`c ziG@H1AB~_a5|lHmEIl;P8!h8jlfmd)E|-c%_hN2N*PU^LMy|@5s(*uC^F_JdXX+yb zny*GaUflvPL$Z%OgL!5*=X~%gAAH>nSuc0!v9QNFI^e)^60RpYGX--g7+sXc(>NcqPJqD{lkDJQ1T6XK<825GxO7TC-MfMV z54U{IC$llf*rV4^P}5bBYGma`Vp(9{8Nw_oY|Ja4_B& z>#kyo`PaUBtPOJ@MW#BL&P`UDh_jY23!kgYjkNzme(2`rCSebKR#E%f zQ$Rumdq?gKazfJFm({zl|7R@b<&3Wopu4KYZ&L#S3tl$;-RMd~Y@APMI4=w7!rtSr zkvnFT2_AjAjD`nU&%|Z_`NiyThd?bnkAuPIh{rTMoivHtg?%~T=)Zx;p+RS{jD-=- zQ=vzpzo-NS=|%T9tiZj|^}O)%&UXOCI#9YCYw}{7Kns+PY+I&w=r3l*?`*AJb8z{UldwlA0PiO|;b^KO4RRNGk1pbnuRXqe zT9XQ}w9-mHtON76Wn0!hmVn9Lr*T7z7}#QafNhWhu=P}TUK+`P&1B;q^>_|)x4tU6 zbd7}F{0rYLT8|u}_i4-I8xrycoy&lqg)c4R6cb3n%h5wYqUa?<&nw=lqMt83bNE(B z6$!4n3u1l5Sa2|blKhh~4zUn)<*NaUFNPz((t;)ms&k0M|0Z1-W;5m3Lcw#=yz zbN6uQ>uO~NR2=RYdE6zSQRZ07r!D{7Op9l&rZ!0Z1h@4=&l#LUt0{A1iUstx50Ie{&Kk-&H5fC6?W}r%lAbNr1 zDTTOmm?HsA!V8ZNAjdebH@%IoXYg69**1F?F74>?8#%>+MfhX=)>r?2YX6P;76k_O z#CWXSs7ZiVeW1DBFZ6~zJcd1&8JK<7Gtvk>`>pP&y#upp*dO%v(px?T3cBy_+h9(D zf&8G{)DP?z<1g)f&3G@$^L}5UDDXVE=9!lh3$wc4Ir^+(fPa~%8d*ZZkEZH9!*5v# zvl8EDgR-0*&a7X3qsCgU*bzdq*C;LcHT=Q*PauNk+ z1+7(nAU8~8U%0D|`;mTjOhMTn&m|#o^(xG{@1jzn>-zxYHmJ8KVIGb`;`IhkfXY+v zp~MDreC5N@kI0+RYt&n-MJR}B6i;6q$3ljE)qP)O3Ro$^c*2b7^Pv$4c4HH`MkKpt*Fr6Aa;3&?%jIyDLL6ox_=qa34b{IBl1R}=KS(QwjA6!^`hOd zi-4F@*Jd`O-%jU!o^KROLTr10@N^y)lu{)|&Y%x@dBwW&Pmn;}*57j0l7fhR$yl z+{lRMYup16l9f9UhCbi0^`8BFnu2SuZI3KkN`tZIy1f@aqBnB~T~}jn_YN>k?i+EB zc;1S2#E`JD+q88Ob8V{3CbRE83+8coC#Ky)e>C4BeN~GALmlttLYjbsqvx9~qtW}- z>Ib6I&=c+^TR!svaFQi{xBq0}#J!n`V>AgzhDL3b{s3ria_{Fy4|k(;+-o*da6wYR zF){&aWil-6f;P(N94-kI0pto9jELAPXN68{d#EO zMSzzA6ZBT>EBpBR$3yxQ95ZV1+xY>x_TcIog{?H?iz^%37y)ehY3-A~5Ph!og@GQP z=fR7XnNgVYQ+zdnqf>Z~WxupjTG3N;i#vqM7^q86DSYq>d%`4Ti$S-!r&pl;H(92m@J}kx6HOAML4)V zb==4PDsous_l^2q6nL$3pE``auGI13>9%^%#{geHNBEgy$JJvQWQk!IU!2=|O9A zi*;rsyexRDnsuK>PM#}v^b8Htb<7>RFn6-Gf3!+HrQo*B^;PT9R~em34I zQ_aFnVq~GoH-MAh8=jn6@XtqhuI8gBY}aM#-23pnPbtVLt)$_VS;i)d`FLJkf1YRK zy@sq&w5zhl-2bb2bn9~p{`l=~T86o=_@vXb=sAtN>b^GEor76F?oX4y&Ol804>#pV z298WsXiN$Ko_zr>9!g08# zXZ_x*F#!Y&UXYxfKMA1haDLuX^n}nkChR5bwF|kLhYaxRe7N4kJN-T8Kz8Br)i@vb z4sWa8W{&r?V5P|^Ar=Jh2YM&j14z02cXRA63mdP#BnN^4HX08ek2Ik{{&{AgQUM1Q zLS1)7(ch%69?13_rs3$J-T48?A@u{DwyqtRhtA1-$EMLRCsZX~cs+&w?cvhTt^cH_JDs^lUEN2&|Lc&iC8+S>HMwvq*rryd(z+A!Y) zN|d$;QD8X|wKQV^0j6PYcg6Oh@Ay2u=!c#${et^x$vS|e{SEyx+wi>Zbd2O`V2?E_ zKWw^1K=O~B2gGzac;J#PqSZ^n_@;$BPiRw6@igJ}8|0mt=HHf-SJCifkIbd$K?06d z#}+8ZGLVrbWx-{za9m~9!Z1Dx93xiWZaq)J?0n%Mb21WI~O;I=pAzC$4_;s3pXbt4IQs6Ond&Gm zW+$KrjlETS=)9YNJtwGPTI|)?3o?HcpJXAZxJH1oW}w;kz1swGUA2SqVCWDHf08H{lV%oP`>VS) z%Chis&q}9@gB<8Js~YNELl6GXLO-zypv$TMvU5EL-BnK>8ep%twk9XKx>8W3kSM+u zJ=FeWqPo`*3oe>ly3SdX@T%Y7cE|t=1vgYq+(j=)49d(X#T?`U1FSEBXV+J%IEKH&#%LP_Yp^X|y=R}3UZjahl!A>pj`@wlyu z1Wd)ewQHDz`{|T0@8~FUwBr|fk$wh#|NX)Z8B)+u{^tGF4K%#u4*sZYA>rgS-voQi z?I!(;_uhU&PqbFqX_tl`tK27dsEvWR4WidA)F@bWc_82OBA)lnI)9rf0;KHQ8=hse zu=&B?%Klmwo=atP`d8AhgZlf#1W{B(!*KKA%_%5I1YyXc_YS;PZR?#|RE) zmzA7p#r#dWcIH>)OB&|M&)Vmsj{6bXb~;3!gI&{@q7>}iAqT4uKkfmz4~+DJeY z@%Cf{^32bfKBM<0kkj7&TKcSu1hsu$H}$5Whk2jhwGn-K*_woTMwbDyCiyO_x&n;o zXV(;b0vImIHh+B+`9bNs!Pai%*%@*M7D)sIy>s6wg&f{k7pmKIn1alOyMF~Yk>K_& zLGnEE?Rxgn-x<3YP)du^u^Yrb8^6C*0DX1Nm7*~{VH&DVUdp@|fqP|JXf}L{1l3Ez zW{0H+xK?4m^|TNT!@)0q*x=mmj;at7KyLhfE2U7Ql?LIpt0LV!2^h+ozDniaFZGf8 z<|-dc!6gUr7yo|mkc;c7Sz=!}2=Pux;a4MJ$D;Rnf!JrY&E9^)p$z1E3_Q4pAHR-g z^-L*}gp96rk6iJ5`rf^-bP^^Ye}x$zuo~kKq%ziPV6PQ^LYZRQaEt&Zb+4g=HRtkW%3iu(*aek@kSZuV{LSe z6y}@|N$ujrLHIT6mgp$*c8L9~`k&~Ts~>9QiR0_55cD?r_wO7#=SIoxYhhtB=Ec1{ z9S*i$9}5>0d{2v~REc>o^)j$hXAD?yJ<%^D26pUFVsm~Q)q zKNP6H%q{dsUTpdjC%*hFa=g4qq@e);=~W+&Tt;r~xx0C15pwC}UP~nc=hVe>_ti^$ zB-m-McRlVvUIcXW!%72& z1WXg!GusV)Aj(4hfCujN&V*gno5pY+ZgVC&=P8JJIOfoYd2SMub$DG1dZNk4v!6dO zFkg7jm5DzL{Ktd`DWg|JKNY`t0q2a977vIRq<~Tw`+IPl1of=<>Z#aUvW5NwUu9{i zb~)c4rilJ4+M=V5zB}8`+gM>X0av%5?Tp0dYYw?&))|GI{dxb+d|Tw`sffTs4cMRe z#54UpXn1qFFUdz4K)qmkNtHMa^fZCP4%yfzl{E{uBm#K*eHi(aNW*dKixGkr0Nd6t zlc_G`AWuz0fJJXmSvbDwoHU2rdtYpwEC*K>tu`OK!ofbnnC16}N!S*-!fg@uxpb+9 zkHH%j4BxyKYI(sz)XK7F|9*j+ZR|#+AoT9s_X6j9Pcm>~&VNE5H3;~zIkWMQC=Gw_ zyDr2!faN?d3xv*5P@wI*dxbv%6$|P!;u<)Z+n>AFuAhYOX;$G1ogB<}50CA}Ue#Pb z-?~6Jf!1Nhu8R5^~G;A)zDXWeEN z>U^V(9S3MgaQ`^G+lhou{w40urjv00KWD|Q$bl=}b}9uG;rCmvmt7hMurl(9+kZ3z z*&~-sUoWE|+-ikIus#jzsSUBAwhYLLWpWR`cbo}R#sOLt&p=Xn(p;u_A zkCwP9H^Ko?oEp?S9f03!@tkYR35clutoLXb!2L(KNykMBPBf^Ask|d##`jHs29f9V zY$L^a*AZa6cU9vJH{`$r!8;9v0M?JX?|ONjf#XZ&OqZI5_op^TdJ?%j`*2jUq9XE< zEZ?W4`Xtnyz24I3$$|XBU1hs|5s)}(DlYn$f$h`g2E+>Byc#TDUc88e0gZmI$7dL* zS#ZKd0QX6GH1$NTItjKR%QI9k=Tw}N6(^T+kU1l9BEuNDQl}vCyA96ao>bY2g#ZDy z>%QzAAwagn)v!DpUx#(J*08GpKW6GG-C;JfSdcF`0s%EdTnNHHw_`PMq~|;&wdDg4SZcpK|X!| zOJECuf##uUs#{#dQ~X->>-25wX& zdj))CAA>x%@I8d!o}ul6T+)2~PFeBOcx?NMuYsE_hEv;lX))i-##VyTrviI~upssbJ`#_98(VymfM>)pIb46RAcI0%kncXfk4qii&q2P?w2O zDT$c?c8f%$FCqUP*n80Xa|{EAs;;??Um;*XRO8ir$A7=4c7L4pT?%?X`p1vbBq-b& zj`=-SQH@aP59Bri3I4+-<@59-kjt0b0zneg%q=ZhIs6iQld8Q zVGzL8@#Ue_4;Yv+RvvhMfPuSrp9}RaBtZV%YMnsL%VcYF%_1cL37#!8r&>8!Hg;sr zYi$~)y^@$-+eg6Y$F?EygA9-j&qsMTFko&|9~=4rU`v_E*`rT6NKPyasaC>sTE|@L z+CqXa-}s+h2^>sbYU-`5Bw*j$&9l2Vlc4OR?7UZpfC-tCm%|JhV9VG)Uqb-?=t+>- z=`6(0dh@eLnS@p6Y0a_}3Xa9gYi>o(P!JnFn5vBPY*GAo^H1DYgGaa3upd9heiFCH z<)Ds#@Yusp%>6@c^+B5eN*t%H|Bd{eE(a%}NQu^W-&45Lq=>#%8tt$i=QWdGVD7>I4r->tb-K~N4^2DW9G;0eQUm9-+W{7B zoa|OyiRY?gZx{ZGhOXQn%co`|hlkEe?lJrKyMISbQY~0GcyQW#?ceAfM~)S&#C*)& z!`TfAFfdJ$e0OOG=Tai9Y%AuerAOVUwG}{t#nkf-845&{H*~!g0SHk5JryqjkDm z@GT0o!gk+xw4lN1RJ+AZ^t}s@eBHN?kg$Ele#KveANGifd9(Pd-UFNAQGu9XNIraI56kkv!xV>wZHx` zag~N((joE^-p97(E#ANM0iLv;^|v{U`*6>W*L5)q4%#wRecm*<{odf*6A4he=Lm5E zd%LxVOREs!Kx4K2m1oFlZ)+_Uq4vYxXsOjUku2~(nUs$D&cUNgUv0lEpuyzDJW<}q z1T2bM7~_O0>4O<|Y5$B#F;K*G;Tr;q=Y;ebd@# zFml&g)rB4*y7=t=gk}nwQ#Nc%6{Df8v0TC>l!d2~N77fXB*CXUqdxUGfRcXq!X0Wf zc&p1VS=kM+GeWK9j0*`N{Mn~pd?LX`*tPCvB?o+uE>-`xoPz^x<{1-t0LfAGvNr6~ z1BA$bN49d1>NuyW=*hpIANojtegiNS?<6shLc;+EJ+p&&zk#os?o0&X{w`5YP5a71 ziGFL+=OzMr-;U3nDS&h9zH)}M7z@%`>0we~*uNiV*cRYDR8;D&m<|M-I)3uqElGg> zvx4VO%%MOcU;3;Z_Q2^m3S#fC(-6(y+u?heh6yc0)$e8KVYYsU+C6C)3<{>tq8HdF zmpHsZaF<%-E+LFww%|;g>@hDE3Zkp#i6c+jZojpD;Vpnht;a|6od~$zI(^V#7YDrC z^Qh?0$V)Y$9j|y9Fu44{YxjQyxb3iAtC9s^J?)`Nq$mrsY-G>OMW25@dhLgf0Sgzu z{o+YpgS_@Vf3*biq)fH2r5gIyes5KikEa2GKQwftV;=ObUr;h2f!zH zbG0)G7<2k`?}`i!?@w%>HD8H_MMq2LH$9_ao3X%e0fvEC8Ap|hnKX1S|D}F-H3jaT zua5sf52C%!*?U-WkTA1!PnjH^Tfz3Mr@HA7iK@_;H*%r z!G$3j5x2*g8wE}ZSOFp?Mv5W$8MPZd9`gVC>?!18{0<6qB5AH^Pdp_!_VUGQIB&f}# zxfH#4Nws$PMFu>bQ-kF#81NRq_QX;h`O|84#fJSfeCRoUxx)zON3>G6_9Y3SMj6&N zYiMXKPR&Rep+QP=JnL~8`pbD^rBa+P`MHg5*SN`-A zCLy@p<7d%4fbPR9KW}cKfR^$W4@2I6sjQ)Ov4e#4mP^;HkT-mO?T^wwM}SnVY^Cy6 z>;dob6}l0~3o>HO|Ng&%-lN`hUsDbglJrgf`$j?GteLhx5iI;C{mol#3qVSZ%btJl zgE#J(9m*e1K+c%eXO(FzG#D-Uv{sY>?TKwE;jNgL{%Z=Hc2O`x!}{h9H3Fu56y6sJ zA+KoXD-v21WZbQPv?Lth^1Q~a-=sOg?upcC(-%I}xCE)#83&-Z| zG_3yNeB%i)kYsk-d;d)iX1NEyU6?_`PvUD@KsX7T?Clx^T4>mt6T#nwJvYwmefH@) z4O2IEsA*vist@lwd-f&;gQ_c{w}-LdUMFLy^MQp;b2besTqi(5(R+8%W)8LtdKaoG zaiF=;YTgmd#l>%G=WM|HUELR!c}^J6)0yy^?~aQ;MhVO~ZTbe5n3*7kaSV}l7RcmW zA>qUkO^KcRNl4^9&Lv6ydw;r9Rocf1sQLN&1oNE*@4r%8$1OXwB$1viU)BUxDjz8L#fZ8c!v`NkwNDT0I<$yKhu(IXT#?!U3*26ERD;Hs{2 zpetN|{opkQgh#~WlYh~0nm6ZYNIna0?Nzx~ev%+?XGv5a_Q*Uju^-`a6#Q~HG=9CG z0d?2QidqJkGg8jhx_CZ|>W>D>;Qq_m*$5TYAb%Zs-~6`#=O=ZIK)D79Oxzwy)`JD} zy{3k$$`+ZfNSmOzmHQa8Y0p6GWHx0 z@Vv^v(P6%ZA1^4dAZ+rs@B%zLb4EqkmID6Uv1NmK9NheQCUraZTrU4}=ZP2+TBCRz z`5w|x)%Ul9HfO-B)T(j_If?(M*-_0^=p}XhJE|YkP;mX!=DX;@i=+~RHl@>`Cw$<| zK4}`hx&(0iH2|BZn`kSAkuWVqCbSU!ZFJT?oBC}OuAbo1-4qK=rRyWt zu0~#X@>DLXgod?i?R1m(aqu#0wbdf@wItswPuCnmFUwbxyK)Kdr)5G~4LLG)IB8a+XFXI567I?_KnTgZSAePG-i^ z;3Cl2+qs^D_wDB`QnE3>9fgCV9x$R1xv9L}!$ zmda~)bOH9&y3F0Y|9-Cd51SX=OTdd#$0nZH0Ndr4UyjmZVY)-*T8DTJei=L}d-RNe zAj+je4gG$0OvUlyFX(|Eg<3ae;5|_6fVHmc{kZ0D~5vdR#m<;Dwszr zb9Q8|rQll;48vJld}R3uiI29KlXOj}J@IJPKy-Ao4pltQ&b^sck8vSYGx6ZW=A*n2_40TRp` zY_$(PAff+}?8JZ)audIEQh^Hrzc&5{mS;E+rk?w{Vea3!R+W>s58#K*x2iGh*;`YG z+XQF=&Ne@Id%=i=mBiTe;bki9WtF9V{U!M^(FZIAaY-lR3bKvTvJ zvsCnB0e&ejecWRc!O6J>_mE2;^*!|}XTj0c>V8rtda{O@VDAd-p^QyNEwc#dJd;w# zi+rh<^({lbn1Qn)rorj>I>ug$2xez+(EdPrVHa|t%?h*9RX34C`qIySz}zhRwxNIh zAPIgOs!V3=LEc?^gigSGZd^HAR24l}$8UT=a|{7#^Bz2{JOJRdGIjcQGY%vOhtWGC z1Qbw--0nmgzMYr-)_H>lO*bBIUq{U4nvRO+I6o7&csA(= zO9IET&jC=IfA71d36M2UWp2IL>j63uBIuhpPIX>fGDJYxLE|?=?hI&6 zUuszCiXMHfROk0i06vukqaXbla4a`E>WDsZMLr~V$sh$auavW&i(Gw9zPeX1Sr?sDLtr53nTkptD9=7i1IvwVs@ zsj}THRQ&p#^*8{)$Hr$t!Y~OJFP)W3^krZ;|4vQPb_V=f-Y;GEkb`9nhq5!IA`}iMw=gF$;nRqyKP||9<{(RaGPM-sFsMcWWCCV(a_{ zl1`$h$NWlv-^IeP%E0O+GguIr8ec5KN5NXF%^OPtaXzaTyxv=f-e%C*9FKdvInPOZ zjTr;4jr}cWn^16Z-W6V<2@1|Q%yu$cLBN*IRf+9W0Q%{rFN-`ln0C}?ZCV!2zsR2Z zG5rL1dv|?pF(KfOLVKZ#FZ#oE!HCE?1nBt*t#mw%TpioEBi)^b(Vp+aPtZeUVL^H0 z-+%90!aL^KPeJF2i@!@nXtwWUz?CFJ4e?;q!7ZbgrpP%@5nVL;Dg!q9sa zz+&U2JsP)IXjYv&bzu+AwUg%c#ke1H_wPL6tV%;g|C^WgQWU%k7}_!$`QB3JrQQ$p z#%3Q67jsq2&sx=WU!_QR(0*BKlRE*mORo%NyJmM#QypEwN|Lgas%hox?o7^VRnDUhIRWg!H*qG`P0qJ~0vH z!25Kgiqs(%inCq4Y@U(uS3t4PJB5K$D?eWPfqxe0-HvaRNAA2f{ZPw%fMqk^Jimv& zSEBEoX5>e~eDmd5tF&pDrIp}ZM+^ z&;xk= z_SRyrkMP~g>qRbd2_A8YXk|cc=N8qTO*E{aH?At%#o+y|bJ4A*fjjD__&^f*E9RF@ zkRk=k;$12;Fb9V9QVI(KNQg6$SJ<0Q!J3vA=@*Pxkn1^77F30NFqtR0V>t_&r@ya} zZDQbFR)AE;S`r2e&V4_GbF@rnwNC^3hLrm1l*Psb9J^L860?{F!^^rpx}mtYS&~&E zOBgs7dEwFaY!Y@>ZSEX-%)r+-Ig8$+e+Vlp8Jh^W4H8X zLp<_LZ%3{BA%LG9^B5!auC1O==~?HI>zB>iRWX}@=XsOVb`uVGB%{TLbO^XxuAw5e z2;flsM3i0+p0DU;?P-`xf>ianb1wGfL zytQk7qi<+k9X!$L#lZ==2^b&Xpl6Gk|Mv9^WYlPFdj6J%n$M;6a~dhwv$WCfcP9Zw z{n0$@eMvAYi~khdLxAV1_gxzn65xA>*L3~~7N7q}sr+Z;lkn{$VkLMkxy9Bi##k^^ z+>vh^NkQ?SU)9%DXlS@vN;DjyA+7(kY4HRLK6%2|s`s&=9PnqM##sQNqsl!0euu%u zr2YG1Mo7ps9yt{qid?p1j#X0}10&N0=t2DNi=RNQriKE|M_-@n&FA37skootjxq4g zwC+QA407VMN{#>7`^#{rl4gAzu3ox$tiIOdRlquC2iTq9IQJ0>AHW#auZuJ$3j^F%NCr3q{M7OYR)*XWK%oeZf zJkcrL;#Aqgs>swXUfv~R+9I#HRJ3spC-RvC&WbaW%arbO?-h&l2@-wh70xk3(84wQ zy^2q!-rKynM&?xMNxv8KDj|M{90BXVMMVC?;WdX2QEBz;{Z~tU5|m}jYi(6oLAAQg zy+|982afKVJY`;)Zs(ZPc3v#(Kc|!Xi*LZWI(E-ubxR`uO_;0Bo}ySEyZrg3Jo0h* zX8h4j-YXRT?ZP@s9-T@bm{zt@Bb{EqfA{TXGC?=r)oF7RdGPr+4f8~dmMOf3)APS zb{6@=>Xn;sXW1iY$?TN9dyPO|+Ni>#X}Di6UTX9%%d^P;cPG!|tcTFw#(9n(LH@O9 zdYcbpN9ojK&g(x?mqPveckd)|2%2qa^zuMeox-2anbfYfLYo5#?>Beg{Pb(8t7{r7 z^mOgTTU>yW7A-JS=Fj1%NuKIKv$l-GszN!_m^QT>+pe_s-5k) zPR(oU=i?`d{QHXpO{@1&X=jm-+pf&l>HNs_2Xg*_e6dZ75`~%|E>mYtn1^%2<(#>a z=EM5CMO2}o)irXYcv!ebZH?wFqYh%VMr!i-nRi^1DPsH7j0+Io9>q$3a=g;$(IZRO zJ1(5BOfMF{{49}W$k_D*pQ%*oK;x4ia4u`GzUi%DT~u1`j9s(lEt%fmmTiMJ3aaF* zn6nMWd0e~FC60ZG{M_|hhPIhhdb@t=t(;g0CY3#0XW>wdJUfo$)>3G6qub~AJ2J_% zVahk{`UaU6$K953QB0X;cf7niC)#W6y0N#=I+a?bzw#|_1%*0wNYqi_+_w5n!Z_Yf z3T3v<7`yqSOizCs`laPVjrvx1J*$CpYyI_WPc&$PdPNo#k5x#an1g>WpyRyX=+d0+ zvu5ekz1zHTtNRPeeOYSy4(Ej5kGeAVi>p$;7>zcyN{aPzl_O%%HbHwc zmOX4ftx}6MnQkn_`EtoIeM^G&GX42A>D(Ed#i`yaQ{dcMvvH@? zw-p+-sawIp6(kxq^3Ko-btTIGsibu#67-|3r}eHJ&?(oJQ<;*r7L@@JQ>Qtgc z>E{5fKvTafBL82A8!h(i#QxPk(cTBMFG}<}YvOJ3PG~f(=gd8$aUQzV_4vAv1y!U_NmO_jTU#k2V(|O4MnYIc`y$ zFUmIA)Tx9A5H4hnGOS3r`%=*o^s*xahgj zs!X3I-5v3!nSyocB5KhDa+PZX-IbO>u2M1I+buti>hY@<*K$V^P_=8DQyQ+bebs0 zb28DR(N%C>-%YZPLX<> zOl$UU9XYBF&L#6+kJ^p<-ScqV>667VuG8!FpInQ`on%qwuVrOwkfe{Razdhv+xmVw zK0~L5OAi*V-A$+Uv*R}pdno9F=aRkw`&rwn6OX1Fr&8?}_qUhDzTRSs*qq_Dpz7)8 zPdJ=cqa4@AoqgO+B6an(kB2*IG(5kjM1QQm13s=_+aBk5k$ggl${7TG*g3S!Ye}c; z;|`{(Rs?xv-lv7KmdCmLY?tb5ZX(~DG;!dEJ(v%BI!+mhyzbDP4~+@?*zug}x=vVu zb*9(jrq?#`(MI^sHV^25Ul4gO>H<&DP`>haKg zuzfw58qd2DZd#^O)5;BtUBbHY^7j2Rj($2h3-qjJ))ZQC=1E|GUyZhQi=S%pc8RQ6 z>Bx~8I3JY1mAmz6nMU2NbaV49%+ua)FNbGU$lm*4sz;+#n(*S<#u$ZFI{u<;y|hcQ zAMDDpefCz3nl&tcVKLSrU)qPQCJ#~Q+UJATHOX?pMNvA>Hg+=cFDek0MgaZ|oVVBZ*bY|`$=UnLrO z|Jdlu86=uN;M0O3Vc2(1mF(@DOw_2W<3qP=$RAdg$#f%|Lfd=h`cgHuMrErfO*}Rh zk>zop^Gn)DRBrg)4BsXp&z-Ke`I=6p)SLR<>Cr%=B*)J$_BIw&f6m19F{%@lYk&CI ztT&O*K__f-oEH>!^7iL#Z*=P5ibz-+{T#X0+4Ohh$AiuO+qG6m)FghQ872EGvF(x_#x@X^UIe;15fKX76nnUV#r^ywL{)5Ec&Y>w9| zT{)R!NEvJ#-8z{Dp2#bx!R-e(K4HE!xT`fzl0%}Cl?FY?Ur?v9ulxRzu!~HVX6p9DOW^{jG6|QB_8tLObpKi3XHcX==-Ca^?}(H_A`)WooR^Ufaoa z?fXj9F2$&q@0RG)I!;`D#24gI*_J09(OII5)*i`LoDkGB!^XP16qTwEzI$O7&M_69 z&uljq`^@~#sW-RAdNFwGz8c?A9bl8vmx^k_jyGsaa zUh;YCO;Q!yffpG7j0?Rz0%lz5}8s<#Yep-GS^wXerhe`ZRx5F zNSITn;y1THD9~3>ZFi2z4bIC{e8R3N5wnRpeoC6I$_kkl^qEln4C1)|!PYAuv455~ zea%uG=f&mqpAU)gRHN4$NAyg4hUkr&JM-=Wf~L2;n644lrLT_Ee^kPLocVO!6v`$+ zzqPru$HLWN~RRlVoZ|)_3{yYYTxtHBaAC$=#cwuS)Ja@?0;Zv zCzX0v-IZw9dYrd%ydN`hw@h(6$8^-nuF{3a&u0(7zL9)dhm~bsN%Zi3^G=y+X!J+g z(VLr^RT5=3u9|`TsMS;5IbnlN*A~>cmMNJ+!yk=k@}RFwB^oa~a~t{Mk`6~cj@TvW zMA%nLK0o~N4qe&s6Y}$6gL@UL7WuhI`kV#oC{#2}r;Q7-zgHdFIsdjaf@(yv`j#!N~m5;g9kc2SYvIO5bu;|xm34;KaV zL_T+3*z88dAo9CSzi)UNP)YWET2yf#^0HM8t5m`MzOF=`*ZGeN+PeDWkrz1MKB@o5 zs^nN->vZViAA^0kWRl6<;$vT*`l9Q*vI}LZn(I)*`n7e6TQS{vPjaI8BTBUPCd9e& zX{|B+ao&*&m}k2VYIL~W%e68v8weyESe- zqUVpYS9`TeP}|IH3r>f>x;iA!g0>o+lQq-C-!)p-BKe%>wN%<3`^K9gI2UG^K4nz; zN7(01$RnDc#5sJ}sNM;4BJWu+^VG@p8inobR(P9Jr*p$^zUvWzd_BvkBgxVu|2Y0_ z*2*dpB`OgA`s*GVrC5Kt@TIppr9ZcO@v2vX9Mh5~ul`Y@6CX03bYVR_n{{NYGI2Ee zJUCH%xw=9J2KDq#$wstJo%kWSTcr(88h2{rmZ)+<=ZzUfaew&krry_BreAEY^WQEc zXlb_MgT|3Ug-qjeWVC5?x$)hH9er5O+IIYA!+G%I!2B^rCX*>^<!amXP zV|Yufs|hB*SeCE?=6}5VosWbQZ7FtdXhxjlCpZ?IJ=zWDt?k!FeOjhb{sEWH=N*G_ z&-%Q3zD^pAzgA;axl%G^j=T72xda%OH%A8l+FYUP7nZb{m``rm>YS=r3_W7 zepRSM&B;YEE$Qs=dS41YRO!YnQ7$$1v30Zk_gn49{wJTA(%_>;H5cSrXvaCDLf@KQ z%iz4)D}M4pbsOrGEYb1&2hS>WJ5T@a)}lBsFa5Y};!Naor`m3OdO;^o)~kyrW1U-K zZrr`z8kKsKr($ogpZ8wVKkJfbGF8bq>0bOp3N2dS>9=H9PZEsl-}Yo?je2-9|FWf( zM6(;V^5jFF)X!{rkO=$o(S=zX4RRCBpAqr+MRB4ex9vekL5bofJkmB_ew8v7&tZ9x zS0Y!UP1)yFR!DZHXyCSLbox>EGlxGb)O%%{ft|44)u|qP=E?}7#lH`%9r-&`gNu~R z+;;=cq5FTk`{ziFMvQM=ztS;5_jS*LW;pkB%J#NE?35Y}E_ZzV$jBdNP5yF77Th;) zvtK+D=c+^#lor8{u0tdQ+*7{|{?rRdAzdz0V1o!c?(UZe|9j;Kyw_{7?KZ88cnaVK?dNuyMw{1rwiLzzL z6L%BZ9qsD4;7{bsFL(X1cRlP^ANXW$zY!XB+;sEt)?G4Gl#0k-IgU^irpM zv&&TL^F^oo6|z;UY}cvDfx-ixj==JKsDfB~U8f%13yd#^b7Qg5tuIZRg!5LuB~mv{;td^G7+)9N^Xww%=?Lkptv?c%O(DJit$ z?lvVY&V_$WIlU!TPojqx2D}+LOHjduN|N0;e`T#&(K_~mhJ7Z- zxMFXoYLsnaFmYBP(Jylj*S}j!Ceh_d#D)hlxlPskMgBd|&F8&lrF^Z?wmU;Nzrnay z%6fd%nF!4P9WQR#k&nNgn4{pN6GZ*nPi|+*k8{J9`YDPaKg*py)yS(e6w(g#Z1J*+ zOw%H|pV3b%wEjz{o%@k5&2Jv}&pzl^yY-)+UV5*RyWsAt5ANaI{c3ccs?(4c&VEs) z>phKb#ozIKu}h&;X-<~BQ(4gQgz1;MMhWWGYD@V0u0)-xUrv=9`{LmVJv7(mU(Lyi=zm4#%kHg5MO{_dbE|X-b7E4tt()>?xf- zOe$OV_9%&###mpccSWMhzn&hp5$D~Tsn_53-%-f6>dfs_WpEB%{;c=gRv6c>t%nao zzCE_`y?xovC={pAIZxw^I=wsoXaD$vG%7H2^R~#(+uAJ|RXTNMi56$cJv+x+qJm8d zUl`RA`*!gseLg(WY1!7U2_8I#A8ng98kh|JH(w||Vn4FCn=~eN7NXQAHq|Y>MI%>+ z`)4l>kf~Ybc~|m#h<2P=o1`Z8v2R5ur`e=vR5xk&w=)h4`jqSCtt1l8j~|v!T-`*b zqibJ9VpZY%2vh5o6e5<64{QrK+ z%zs(pFwXy73JhK~V3DAtbvx`^whaAWUNWLbE#y4~zq(uPKt8mievbH$CXQ7dW1wfYvykE zD6OFLH6O3{@L>tqcd={wB$HW}wNs#~mCwUUwF`wSeI zcMkdIv8Saz#?a|#<}c01V1H`*<;E{34k)CqDgW4)U8a7UKTMc3UeF-Zsku@dowj<- zjaOj*_%(g4A%PScjoo{@f8EDK=ePHtw;g%U&hDA(AFZNL?qW}~uh(VT`=E$-ba#z* zn77Of?}vH%Fx`R{$onJqExCApGf}DYQ~KsAuh6zdw+FSt{%@+X?~e%Nd)9Hk#P0l3 zq3eg@l}Xc8qR%~+mmIxarss*;Z#e%_qr1+WX_jx&>E7ns#kQu1&~zgwmrSI1|*{ajFi zc&qO>o~Ti-2mV15*C_NpMUuE5zW(3+{_p?Q@9_R#{T}ZB)$h{gXufR8=++vwTz_$7 z{zoeHDLLc91+3qrf0a&u{!!rU+)6UFaLthiAaCh7Aa~fJ>l$f;o5kDnn@Zi3 z+bxfk)9A<_sTQlV6-u_IjC=P3g)Rij6uKRuQk<{9Hz-m=ry~&q`qmnvQkXZjB~cNb zu4i_Q9!l6}?(SF12C&|>o|eq-Q|b2tCFhMq-gvQW`qlL_DpbDI@Cp*nOIb2F%{P%} zpQ!(-=IZ>2?}|eH)5yOY&XJ~X94+YKD94urS!GgZO>St~){sc*d3tc~3`D&bX783{K2e9vshtkKLH z?&Fi^X;l96tX`*cDm2Y=x%0MZnCFj9#g~t3G-J`&q45{ue2_Lt$r{IyPk*_7ed-jM zuE!fVAq~#+u}+OCJN$%9gM3NT)EI_ye#|PB${x{3RuT{0)+a55tMR??PNP>i6acMj`*Kw<|gZnNMfqa`7!G zhdr*gWp44P-vQ6msYiznzl45wUBgRd-kLL4?mT((;^d5e~fUHq*RN}@ zyOwHKXLWxuYtNpxv+fKpx9j=AIeX$9pKnUnIqSo0D~5Kjcy+nA;pLGZdRDJiYeLthf7BtiZu?6PBK@*8Hf}`j=lKhQ-{Ru|Vo%**|y8`fL2~0yS!Oi`jB} z%yvDqAK5!-Sk|r+=RerJpi--(%bdSg+Pkn?heJsoOgx=Vn{i=S-q^35o_6Mhdln@L z>ySTXt*jQ2aWN{g#oH2Lhn=IRW$NDmU#V!7t%_?R!JA|!V*sLMQA*e3qeA}jivm@y3a(^hC(mvZlYyU=)LVlI4z48dSA3&g8vwa ze2)_wnP@CU{WQIoND9C{QbENj<2@9xe5puT$*Pj!F5<%>#jszqe35lEufr%r3i-&|0XB=-* z60T$B4-AY zgBf5iBkX1*VZm9~nuQb}FpvehIpCue^{ph?fetHtu%RC|^tA-qDM6~aH2f%!cFL1* zSAcFs=u{%|R7Txj5z}8u^8HGxvkGiiCCOD4zE>yVsY&FnMT)f!+HXv%r3uQd(2v%r z(}pBB0`F^s7`BCeJCd#KNpiMFxdTbm5%xQxzs|`2!;oZOSJ>@FilsaH&=2wNkMaQY z$%VFEqy${(?_g3KgHav~|AwLuL!pbau>E(8;W&~kZrE~@WsJQe+&4%;&jADo??b4l3dBfd-F=TcH_D@gXPK;4yy<4WkSBoUfFUu2AP z4<#9f5d^ zB*ill@gIf$OoTu1(+dA=Qz4rHyYSC7pJc}Z^lKqxi!j!UFs~M4o-HQf$2r)t65|LT z1Mt61TxKD>S%m7(C4@bXpkN*$dGZO_o?i$ba%EcqA$tl5G8YlTViRi6CS-RBLB0|~ zvXq3)QbKZ+5{kbRmX6Xw4U|T`azeJ2hwbu0aRV(CgygR%R99u8n14l`s<2xXdR2vD z$2rJjM|)VxP)pGuPb7D@Qhpqv(V(nFq6 zqF{DOvgbfMIV9O`1zIJ=l@srCNop{cq&Ra+swIyk+47=ZUg+kNWXdl|?);K$E+h$S zVM+ECMmt5IUlelWMV{i4WVK1cQbLjfC4r?;E+eV7vglJ;ye}uIj`HY7c}enCkYrm$ z^r0g3Doe8KSM=joN!V}>b=Huu4@gR|rX<{TCCS_XW#qAeM(A%-_|yz>Xn{6bpwBJg zOIu0uwS{hb1g?Xmx;i2bPV}XVB<#H;)!9!Hfe~nbBy>i>mr*EBk`&uyNp?&@{i(1s zRTAczh}BHQW~QXlZ1iciqgUW@Tx3tu)!s&@l) zH=^y0@OKm1+$72F&1hpYVzCACEf|BXlH%AZk!ve#??nH1qV8Tv@$N;R_Mw0Kfd|me z0ZFnPM86KA?qSS_Bd~oG^^c*CCnVW(LK3c%7>ARH)oF?B=V9khv~>wKFH4H~3gUeQ zV|)d3=pp9hBaGE!Nd%vuuTQ~!ktF9ga3l-AD$8D7mV9Bd?207|XC|3EnPkirS@jo| zg{7D*JB!J})?6lMb6K{xkX3tInfz^K;pz(AuCk!sGI@K;O0c(#Jx^BL{bb=CE~~*& zvf>zx_hV(%Jq|hRcv*E#MBPa;_G55UVPm#TzIn17^uX={S#d2#d9|!q*T`gFBP-;U zCEHr?Yh?w19gWh zl4T_HMk}iKcj(MkWZxXvzN@IdyC^?Y6!$}vRh8VTDtvKp3^c2<->iyYd{uQNR7D`E zO3vh}WKRK|6sqL0s5tMbvNww=9C=jo5pTqpI${ z;0CKWi>SgsOeOya)L8+$E1VRx-6S=Xs#TL<}iRkm(W z71sv%xLH-~TOi+}A^}vTz;4*{sfuNfO0GRY zhNhTnYO1pq%C$AcQU`VFYKp5Ke5jB24K?yK(j;oEVV%?Dpi>jB?wV@pp~-<>XuA)5 z9;m6VL3lq1z7K}|!J6zDqDj7?n(P{;N%rBIOd~WA90{AF;PY6tJ5E!*6VaE6Xnzvg zo}?-6$&k%NjAm()YZm%E8$QR^$(dM}T}hGaC)4FXGM#)Wbm7d7j4QXU26F2Z%&kk7 zJh~jrs|$NEom|Cq;rLZo%~kNeiZ0yMbn;fyg|E7<1Zu!eOBD+3REt!k<^@%PU>jzQN`<^j8m49kIg%#SK%f3Bt&pAWXP(gpoH# znCiBM32SA%uN)>kjxfpDG>n2x!>}fX$-x$3vnYZzI& zLVqyc4-XT;+0dJVy7QnjKTOz{g-PyZsJ9CI+A#DvjC||EB-e&8a&Lg`U11XK4pY6m z!TI3Z9<;v)_V$KJ!M$N*-3NXjeB2*KfumuvPTnzG-!~60v z6mNwX!nZtz6xJ#g)w@JF}a}mBS=^ttQEm(?qtMCgCq&QoRLXtDs3x zVc0Ef61HL{#a0}?*i4eWgh_RlG?BTKNq9=bb{Y6n)}&H76Ishax4cR6l{cw@3MQh8 z(64Bct(DMDC6jEo!-n0Y27y$~B%IYvilsJe)inuoeb{JdB6mZR9B7C>H!|VeZc-dg z;6oFWWOJY`2YhRazBDrlM|0S0Zju5(YGD%gVJ4inO~Nu9ehh~nZumSA+(h^}6|tI% zI@3&wXFA?bHwo`-$mgK#xhB~%4>4P2QoPIH({htcD@!N%CKX z|JO|vylzsgH%vI_A@;XWzJ-|G#uz?8A0NQ>L-_j$Z9M|_#3VbOz%J1D6gHk=uDmb_ z*E_WH9)0=@z0a5@Uon5aqE8e~UJ4gM5iUFBaB|Dxic<-f14_8$Rl`-67A|b2aB`Z$ zRSFN6UE$$MAUs_4#10qcxTqU9Tv!r^tG>kH*fYb)oDA=iK`(i@YDY5K+YRMO4PF;Pb%;hl6`5AGp7Zg7CCxGqbP@M!?ADGfhbKJCMJ(BH7jkI&Deuwj=VlLyq5`$le|{yMXUPs-+u|ryJVr z1%LXW-9E_u`@p~7NU`)I$m*VflSnwH z!`5{8JqrocY$D5S*qwtqb4hY9C7G6yvm0b46zYc=6~NRn+0 zeD{(P@S-1U;qN;5w2oxQdW^>g60SX@y7r>4`$+NaNBICrfdk+U!rmd+KSXNaFvjW# z{5*!(9fSNB{5}r*$6@~jsdN(lokT28Lw*|l1o}^t;yQykpU3+Puz!)rbrG?;3SX}w z2GI#1yH6O8{;#PccYJ%caL;lp!`ZxCbs z0(0plbY5Z{Uy)*e4V$k?cD;cwZ(;Kt+WY|S1I8`#^XQLg>nqA%F_*r@pfybqX#?23z=J09}nctWN4LJq`7{+>Xn_Jl(A zB}BbMLiQ&{yGc+liIBWW;2Yk1k_y?GOi0e;$mx?Kw@)R=l}aeyRPZS^^8M68^`(Jd zX@%-ei(EW2>|}#4IfUf0!hTNpkPEWh$m{dKmpsVf3!smM(4WG1UskBzazYB0LrlsG z;j94r6@==lh<;QO!e1SEejWJQ1oEcvyO$7=&o?ZQ&omYxUW%!TW_0@IwZ)oQChGh2lCbRLdE(eGc(E57`BwS}&q}5%Npu<0bU@vXJaoVEYQj z>MCM#75-flvg10&>IV25LUr6iJ9nUWN09rOknPVA^Or()zZ7JDEoAR&#P1Da^cHQr zgZw?l@+127QQ&+d6f;HUddM#&udchCj&lU8f~sK97E$ zN4${tn=fG;kmozDqHotwzJdPTkd)vJv>!yggYf?a>b-z(FVOx=*#00Xt`CrXfc|IT zXN;ea6{jSVOOhp$fwC+dCgj*AS-9iNQZRnxcU@%F6M=j>LKdF%*a9&w>}-ej+sVS)9=>*zWosu{HFuU3PiN>k;g?euwr;T7O;+4MZx2~9_e4K?g6}O$ zzTUF%_mO4WZ)m@-EIIljUj5MCIM^SDK92|IM!+W^UQ=Lmnygx;%Ve95Hh|tpoPl`F zmSu0`vy$2HWudIv7s;}3F??Q(dP`7mDf+w=ek_$G*HVnZTKKzGR)T9~DX_T}r{M?OL0=+)e z-GjP&FuuSb(7YG1+>7|_i~O-KjLUx1-49=npg%`t;X5WvmgDg21o)G{Q?PLg=!ZQ& zd_5=2o^!H7=MnSsGUhg7b^(3AfU)>fRvdrIasX(%h&Wt?-X&RfU4rjG`(@}|mT@0o zE?kl2AkcOd@wqB1mTQ=k*PsV3< z3_igaKE-%GMW0_HUN13juQ8r);QL#Q<2&3J?-7&tnA`6$Z$3cpEA+l%+^{s+B}MU~ z+g3%9?Ybg@x*~hR6k&^@V2@UWJEkJrVkxRO7IeZD#SyNM%dDurxC+_gVUv%qsE!DQ zd=ZLbNvI&NL*2x{#ER@o0{x^4Ig=`iFDY!LRFq&!1$zR@sT9>?LHnr{#g$qia~egs z(kkRnt0>lV3f2Ln?CH^V2Gq}>2zy3gMnyPsgU_u<_B^ncAN2|-va5h10ygNDQiQoQ z{3;DuCHPQDk%E;_=U0USzd~LGzSUJ^Z(T)r>njwjugH!D@Vz1W(FXl$r>M5}sM`^3 zbX0_;n?jZzig5N(RO@f3GXQ-Z2wMXc**sX0oP$vw3Li$mpOFf=M=EL%XdR_Uw$ZRN z4!YwN$&CZNW1^xsC&8b|kWWz*_Z0X%9rkCS?OBNZEcAUA>MTaw7sKWf*jj?}YV>0b z>a9Uc*23Oe*joqrdeq+tTbmT&+o7oDov^hNb$23;`xPayALRpz^=!Rg>g8eNanMM!C7$U@%{qZ`x9+nL|-o<7B|q=4b%ntZYYv9 z0RE;Txo#q!w-DpoNMY_GCXvq?A0yV!;O8^&&lS}YM4VqBJ}(f^h`dRT5T&1Ac(3ah576hjr>gsS9BtcqX~Rkb8jWm^hW*s`dS zC7&wXRaH4yRTWM*WYbY+o=Ucbs%CR&W^1x_ zj;4C%XmW6lhC`I5+U9GrYrdv<=WCL45#BG>RQD2%f=e{4ff@ytX|iv*CY-Bbd$p$6 z*J$KeqY2+ejl7#Q#j_bYn>AtHrjd1+KgS#}@xm%NbyV34m*xReA=Kavw zugUI1niM>$k>!|%d_f~RuBnz2nruE9`M05(>N=w-zB8I|U(#gHWwd=2ZCuqP^EKGN zsR{3G)W4%C*1OQZhxYCRAHb)Fn(Td~N$w{a`JQNs`6>GO3_df?`W|h**Cgu)O$mI0{m=0GGi0CP*H^^!D|{wh zu?Sr_B%QpHF8O6$CRLX#nl8M$t_EZ1vNM*h`b;_n!gbXXM_2rDblGaw$sbo&tnqZL zE4u6gy5s4R4;YN6tJVa%L`igVB-Mq*qGP?#g)@^bdw$W$ni=)8=)#;$SKZll5zL{> zj+}U(3w3gX&##jwKd>;`E`s+(bm1-vUy8yf8``z$!dC+Rmehrx#RwF1dcy z$@i-+IjiWhzY6rL>d1G|zZ&qdhOT&lff~AGt*I-Hn!2zzgzbjVYp5&KSSM#=UGV?| zjp1JtT?sZpJqP;a0N)h0Tj-LbCHR)Q^iSUt`u|`5zj1l1gryUV=G1{<8O>Z8OJeBXI#Lzig6R8kMStuImYXZ4;WuEerD7hAv7~4WlYDIov{F8 zNyf^IwHY0Z?HIc=4qzO~IFWHS;}S+M<2J_qj3*f{GTvf*!uXbvnuahYV+3PL#!QU4 z7>h8LWwbNaXKcaPiLp21V8$_wQyD#sD;PI0?qWR5c!u#R<2}Y8<3~oN8Q=emi5b%{ zy6VNI@2{%NH)o{q5VSGyR_0w?9?9rnehimqGCG-G!{u#^9_IaAzRVbm6XLHM>{$8v z`(s199_GJtS#8eqf%#Zmj$m}f4e?32oHk0I`OI7{%ot?86qjo=+T(@z2F;_#$Hjan z1Mg$LkD=WF^Fs~1C4Pu^8|*lkpUwO-MmO`T4SxEV-^ArZjGlAP;H}Ir;qqEWC-a-Ryob@p{NWbS$I+84#Gf_re&z#)`26Qf z)VLnEh(6D)%)c_|JDC4u;N8r}ZW$^kVe~O?<+6>@og&27Z5chD{*)oUWlKZ5sX}}& z<_AaFVSYH5XE1u0_j37Q6mJRf*A0I9nSae?trg$bsY84!F6U$PGhdX;wHYZ*2nKL@ z9HWK##ZmfE^EqvZ-_Xi1ADG`~)K4GcPct82v@w4_%8%A;Cu67>qqSlEVLpQSbc{~s zv$T#rF1Ab|zA*FU82!vwZXJF82bpisI(nRKzl8W!2K%1OA-IYYeEJ~~giLhzaSunruDA|c+y<*5C_$9!_;qxP2o^VvE?w_`3E z;$0;|=q?pPXBmoiHS8SHZ^~${9^zYfj(#85YK8a?oukipH}l5zz+OAV8}EA;^Tu_; z%Y2{C(eKv)^CJzsrEZ9ix<5Qoa^{zEc{`(n`8}Nt?J{p%=N$Dyyr1=NF#4E}x?htz z*-z$kbJ@m7^+Qn4z`K}l?To(eS{sD;zRu|PrGxqL27Nd4OAPux=64$OgUp{c=-V2G z_`3$)$^0t=?_pl*5}o%mpTNLVqY&gU@K)xF8h8iuRSmqG`8EdL$9#7KA7p;8fwwdc z@ooceXWnDrUCggF@LuK*82AA5R}H+mNr-=G;BCz7T@C9Z^C_6m$>?*0c;o)*Xd2>; z`=^`vsP(u&lpW@ab&bAG2bize)i93DLQu17bUPO2<93g}Zdp2n_-qE=%6uUMZ)3ik zfwwbX)4)5JZ)V_~%sUOdi}?Wt-p%|N1Mgveo`Lr=zsA7(nBQyQ{mh>-@B!wp82BLb zZw!3o-yua_5j~>wX67>+cnkA|47`>3DhA%hd;YYe=d`4a}-!TcQq?_@qkFGKv9&t~A=%-1mR9_F0}-pl+L1Mg#gk%9L!zumwG zm_K9SgUmlL@YFd3F?$>0&wLgGZ(;sd18-%%vw^oU?=tXq<~;`9!Tc@*?_~bAfp;;l z_c6qud5eMfFrUZ3dzmk5;C;+DHSm7sdmH!w^WzMBkokoMo}3}rW8lrqUpDX-=06#D zEAt6|GsK_y+y>sxd~E~oVBTrqoy<=)@Gj<88hAJJ#|*rO`DX^+%Y3}PhWIm|#lZWS zuV~-{%r`giLFW4zcV8f%h{Xr@tZo%;z`oLFQ{1ckNE=`I81d$oxG6Pu)WB)xevXPdLyJf9BI0cq{XD28GJ}UjR@iE=PSjHjK+U zmuGP~jDCEbnh=s-obW$(WWn|=jFGQrEdRt9^EA_U*Ouk|7{@?QVV&>o&0zT|#yLP{~_Z!QnGFrdW``+FTmY-+L%X-Fn^`7N%r-m@!cY5FVHxJA0jLVMy)$aG> z){o^g8J8dbtH0mt9cK9*##P7vYWI7+VdF#cZ-$HbN;2)Q_2NhJl+*rNKYk=n{}1xi z6aHSFmFp`%k>{NDm;c}U7$wj5om~2XJo|*dmlyr6zVZWk$$#iy`G1i2?(|oDf8_sV z_h0@0QTtV=h5kRRsLOKV z`x4|*bh+oKL9Q6&#|(1SVE?#5t{LhZ8F>ydFTV`hKy|(1D8SuJ(t<=l@K}@ z-HblQAS2!2dB#Wqmfz%gbBpDSKE_}ae}{QS^WDGy-kPl*cdmX2EsWL%Axvb@cQNm0 zbTtfNfKhylu5WF`_8W)L?O<#g!k7m8mS!Q|-aLd3Mn7XLgMJ`NuSE!hjJB2`Ol#C< z-rtJrwGN?^(QMEUGVg8^UEkh5gsM@Wc~=Mahtb|KgyBYg=3Sja=<5`{zqt(hj?U5b zqx^}|@18rlzSqg)+J*JI@pv)DGuU@^XM2pk9z1?MxgR}3MIV;~jJ94}X0-JVVbr&e z++6lDdit>bZ)~?;2o3I2u;1VJH=9A^6OJkE?k#w-SX`>?vF zRK6b=U5su<52KgS$LMDaFa{ZE8poH>!f0i*G1?g&jLvD%7u_v=Ltmt++cRl2-dQWO z@H^`AeqV<7cRQa0%JVtG!RL&|e7<0PM>F1Ui?DuC-rpJhB|;c1$@ZsnKN(#!e(a~2 zG3p#>^k4c3FMh&DUA9)C{l&@XVT`)&EeV~wVlU-oX2!T)}vZ@dIOul_4z4*qL!E<9^25jQT1*hpvi#u5^D0t*C*I zA+&sojxNUkS*LyUw^!8tHSZWdUpOAZ#EcfkOpLi1>$AR%%jFpBF*ajt&DfQ(AL9td z35>HC7c;JAT*v5R^fO*&e8~8U@e^aL6Cq5(n2|9rV;RP(C!)t!`C*j)_jS!s@85n~ z{<)!^@jURe>$9QW-yG4RU8a-K>zV##z2t^^@&9GLtcH3K|FT{QL%oDQ(KB9lLp|gA z|Ff%!p`P*F_p__3p`LMn`q?$mP%r6E^o-YJL%rnxvfffdy_Em5-VQ@O<9Y9A*D*sq z?I(K1>%5`fXT$S=pItW%^^E7(pIwi*UR1vl75#CZ9d$*G?|VZ%d#1l{C#s%)DtbNb zmw#6;{;BBw`+G0{dY+Q@|FK?_-1`4mFX}3G>i=>50E@elTc|6o6=zTgf_=P!o)CWZE2=j7;_k7Y`z>|qQ}4WVsXbUlS9 z$`ZEcn$G+E^yqrF8C;&pc4vjqH!HfH#Qt4odzQH&xnpj0y&#vZ^FnA|z~_nu(e+gJ zukz`r^A-EI)R^h9pUaqE&i0oZ;$kwJBj=h|jsABVRxNtj#$~60{~tSv|D7F*6DkMe z{Mb(9^>=eWXy$rxe_TKEYi^NKTxA@JSIO@tp*^ZzgrRQFQ(vXOK6{D0 z!j#bG7g3irc}UNZ{Kx*MGpCETWm{5&^jwTyMr+z1*LVIB;`5NfPS$^C$HjV4cC2Qr z<$JmRU)uR!{uugY&Gz5fDQ)PN`(N5I_RD7ISMa~|$8NS+zvu1Q|2sQb!;(kqm-oN4 zlZT?)G3WU2;uAp;(d;sT$Z2Peo7KL=}JjCzm0k>%krqtL+-KOe8Y3y zsLw;5vV4Kzx$eYSA^B^Tca8F&pR4bB!g=nK=y8bh{{j0Qy#HhQa;~@TC-q;bKR(Gv z#cvhIZNu#!*FVDTo-wo^6~BGluaillw-Xh=`pma9$fM%dpZSr-`YiV_zml=hm5?7@ z7zZ)B89j`DFdksM#293Zb(QmM#`KIf#=4B%8OJd$VO-DXV?4!po$)!Nel3KF7}GQ6 zWGv2Dg|Q)HJI20@V;E;$i#`tXE_1%lc$V=o<3~pG3*L7ayW^F4c=?kBB7goukNg}f z#G60!`uK&{14hr+==Zt*Tc{lP7R87Dz|zVXV04HOMt!E@MwnCuUumC?cIV)QZw8UNR3HnINRZ?i$} z`UidE71>^vmi#3|ybo}FuZc07{bO{*iEiIxW#3zV<0{En~^g7!#Eh^ zQT;S$<#~`by1YB1jd=&7D{J(5?&h+W(a)He*V&YY_11W0VtEcm!M>TtM*sgE%CWri z?Z5J^$nU}>H0ZTuc}K>`=^A;yO<~Y8?xzYYoqIf!@BhbhEu`FzY&DjzAU z6qQrSA<1kxALme1hEGZ4v>Zx>sgOC&!W20l7KX8$*_=10?RWV;et*5U`*2;?eP7r8 zem~!@>pslCH=J3&7s44vPm*7T4jv4Ry`}K|o<=rtmj#`f|XF#^Esoc}UgpW!YpJ*2jz5T}~ zko2=y?eXP1GHZh0YPIe^8t}fKC7@Q?ARWIxB)j&otk(Gk`c`Sm*;1u8Q(cmKod&;| zR9n0j>3$uZzwRR=;I~`d&*`@>{fg4g`f9Z0YpooTF(yR_hbHW`lXEd?_)zSq%p1#l z=LTf&cddRsEhzqVR3diN7%Xxf$n@?CBd(mh{3Fy_IJ9Skw_EGfe28%9f!ko;&f7|~ zZ~gDmvL-pPk9?C+?60Q{@BhU*S&lYZ3Wq+u?fgPpc|Kcb#;H>_gOPem^T>R*?o9vP z5T1)d^G-T5`ko;Iq`oy;kwBY()`=WIR>qUO?&PKSo7-mcHl zo~t5SSkw&Pu20r(gfGb7-@Hx2n0MFwSQ*BchaHqzH!E)`mfHNItJ9%*kZtC3Xl-OVx%=nF`mK5H_NuoG^gqk%oR!8YIM;AwdN(1+udsb1doCh6>pcl)_s zig?UsU*)^-^PAd*);RZTU&Ib^spcNof9Zi>p7bF;;$c3CVzKT7;k=J`&stu%z_p3h zoeoL+-R4m?Ux#IG@9562EOo4gd3G54b-dLrLwRu%j;vkEB5y^#Y>OJ>89Lb!BG3T! z!K{AxLKA)pS5X|DWFfT>df`1MsGqgy9;|{ZM5k7~>3E7WA6<~79t9sd=%`CFJBVF- zh+LGeguuM~kWSK4aquB-M;DU(6Zp_!$|2G&FGtVGGo`xFQwJxQqv~zpkl7a?+lCie z3bPoFTGj3lLUWgwe4_fu?9{wloInbVn{)dp#)rwV184M6Mr_y*pXS(&QG{+xsyoOrdmxG|)uM;73Ukefbwr~UG9G>|EZX!Z zzGyrsZiAnBiWb+XD745!S0!DEHy0g}qKIO`B-plBlR=vW!h0f5R3w=-CozxhFyF0^ zZGWzhDPp$8*os|T)D9h7lOcZ0LLf%B-IK}3QkeM%X-C?mj&aE!?C8M9kNd_s3u&aH zN{MJG6JAFvlKfMStp7>84n~!uPOs^jZ%DoGD9YLe<(~gJ@9+|3K~0Vl(LGX|7-udP zY9Xf~$}Z^|N8xWTP4{eGc^Ki6lFERwtlv2!({vE}JHlPRjiw_6*)}sjjVR8n?RU3N z^mnZP2@!2TE1C#8@{l3~v3jzO7NjEq+k!2|Jo2$LeUFx+IA-noyOpuL)KxEAXV_`< ztE9%|J6gB{Nfx0>_yLSJ7uNqYJ`gj%pI~@{oxwW@AHmc}MxOA`1Lx!P&ZjOb6dtm~ zh?{B}mn1^hbHhh&Z?eJyHVK)n>(*$}GLu>LA*OQwPIB%F+V?ft=cMVVHf!wMDsprj zvn^l0&O0Q~(#B5MEtN>J6%H-4sw=8`DPMYY$D_;ZL~)}uSuzqjvt~5C$cO%C?Tryd ziF+SW$x{%NsD>Flz`|Ig^Xt6bslsRUQ4&91Xu%h@++M;1R%3(ZZ>&o2L zXbWC`DagVzX)+`dJ>x{p@yL>5 zCxVZ*tta_N5i}dnhfJg>&Z8PjF3}x4Oc7dA1Ltt4Azl>XxqEi=Q&(7nyt^Y0D>0Xu z(N~L#LDf-3V@5r%?@rp+Yeb`k)*_JvT?E(8KWkh_Pid(b0*}d9!BngtCX9y-?b+#QWvvPz3>bE9~%3Phtb8}J2+sxtn3_)Lhs7;N@Z z%3NWYbkM2ro_OL&8(wuh1?GBvu{IsGNKL_UGxyU%8==B6!wvYeOIG&z;SP`>v+y2$ zYM?gacz93634NEvdyXPIvt0qyQ8x-tUwzj)RYZqw%nEy#oFOvv$Zt2vc|` zrK^xwahGDuQjWB%7N+Q9iVhM?)*!aTOz}vInX(h>enq&2V4v}zCkVq&fpij{xDeFZo=m; zSusIfQxH=26L__hV>-$Tsw%T~6X~m^ArG7Nro!%0Cc!q*+ZfcycnV#BDNiH)fJVqu zd@-3q*fn9~o2QSQxUp+jkng1{#bL7!U~JIkSp~kaMeX?eOIBlCll?xT!ZI zMOD$kDTnuH5{)5-`(U&7RNUPxQP`|DvK>+h{-cl(%rH1$Z)BacR5W}`81D^Xs_%sJ z8NNa(!$AQ-)Dt>5uJ9gl>IZEcD4RhX`GsFzvO@1;?xWFT;Gd%%fU~8hu=gY6Y2S&^ z-2)U`OqdXCwv!sDLq8eblS19lp-Y7KxHMdlPTL5(5X%;%-zW8h2dicArP5@6*z75! zxeiV$yl0G+zJ0chc@YY`GeK7h@`n zEtkUwK|IZ!y5CI_BYxuNmaJ5xz|gxgo?^$#)Ta#*p;rW$N;LjPXbCSph&0&%Ejidx z`((tEnqtS5){4KmWK|TS+J?UZY;@5N{35WiMa_8oam6PC`^Zr!%kh)}Uh)%^On8qq z7?R++EU*DRsv7bk@2q$9dRf2h`^>uo;cefbN|Zyz`P(W*4yC!1e(9cFC$VZ!;MM zddwhyGug$3Rbbx>mZyRLgDbiDFibvFZ8@_JI z$}~#$2i^kMo>~%Kb37$5FC-CF2$<#rut|PLgomC^>67{aTcrY91ydK(Ed*Bi$}GS9nN=Cu=O0k{e}R>0gvy@>b_m$Zz-~1{>)uRNynj?54>x%NN*WQw z`k!p~#yIT5u9@5+Ya_rR&wDxj@4SiR2guVzen9J@2LdpvbAfU#N9E7Xn+MjqjHiTp z?Y7__Px;4s^lT-ggvV2o3f2{A)eX@0JYeJA_6nQ4n6e^G5~1xMLSCVC!3KMl3EE+? zdYm>ugvwh3n-c>!Ndnu|?Z>WN%z#3e8atiuXkIyP6!;P+%H|!|MKWPU^$c{PzTzbq zX@)U2+?A=Q5~>SNJ6B}_%4|G2YO7)K9*C{FxhcfDX8h)oV%${Bb`;8i8lP+1Ab$^h z7?<$nD(5-{McNg?`m5j-AweRr*(+4`-K_S{J3U>Q#M^1Wr7c+%zU`k5vev-?KQZgE z?&m5%a|DkGjk?-lbxVJO;av8vGNZrqtEx8Q;LdW_FtWB4(-w^ts2eGc9Cb=X^^T7n zsN-1&PhD(}Yd-1NXP3D~EhhQ|qR?>8+_!qQBco&EljcvT4V=+4Q00Y`C? zd@S5VmeP;e7Ga;7al+S^NPCu@B1rg)Y~2hay>Be#&i1x5$I@Hz{7X(PaI13nwTMwj2dJ*VYl1y&?9KqJy~ zc%+%y;XRf!#!)C2u*U_kB7e-%LgxH-1j(q(2~bzRzoMIC0Bb$hdj|Fsk6q@xH{hv! zLyyJyH8R`_10?COZHbga02k+pscu=KY@aSm;!QV-@oQfNd`m;W8^x7%c7`9BrcFP< zGJPjYtdg!2h~(We!rNg~yB#;NQNz4A9$HeP`GE*-M>y&2t5lc6TgUOydwg-!Uw0}| zR+`MkzB^+(WA9A55@jwR!)+Lc@*X!A7~w`7-ea`tBlE95VYWd`>;2er1+%>$HfxX- zWI_{Zghuew+B@-@)D&1kU``8ujGBTF0MS?3wnw;{hhl*#;=-;OBfTMHu(J)N6MNTD zbq`C-+VEUUR#*Wk56XT(B2i)4DB5^R{eJpCv=8949iZGJjqqdj4$_Ti9snXB6gWl* zEa#&@NPGY;_&w1-f0}{OENUb`b1vA3qVw6OqI^MfjyTGLYy`d~gTIK`=7Y_`km?X7 zG`we*YNCa6{JheeTN*pxjF$ntvCl9agp10k`U4$p_ztjuJ4lHZ3TlycRG`IyS_LTb z7+F579ydaPhIj(s(!f8(w8g?h_W`W90z#z%-V$O6Naif^C`lazT^K$aiTcPp zY>f4NM`2b$Jn`mTN2fg?>34JV-UE_#2pOQSCpF`L0lkx0;ZC^=))NEnQT}BWxJ0dL zAq4ke1o*DoOI8GK!xyLoP&bd^bCfqVMM1zY5oHRh6h=tWMq~D9WLugEaT1uRq$;>k zO2HmHQf^rl=c~`cqr$RKspBb;q6k?A`hPfb40r-7edKm92hfxfo(CfGxalX8z$fQ~Baz%ypIEa3N@!cWm&UHTxqb`Z~xDLQ~%Q=6N2b+iTB#MHb%T^Jdf zFp|$*nJfr5AIy^XJ-&0`>LvsCYxTPuMI8Y6ora44fvp{K5mG4w`y}khPs@oS#16?i zLP>Qz*gg@QDh>7op7H1!gB+?wm)@h22x%rj$s5RBh^J~QqDv1M3$d5n!xFMKP9?nO z6*W$m4q*9xL!z6b1F*zI4@Wqwz`bGQCt%U*0(dJ-CjTClIFk6NDQ|d!#AyQ!P<2td zB*}~iTh5I)m1cr|c|rBiri1>R1x|Q- z7k~uuh4(z9D!4kDvI_c%WN*XvCjPT01`m* zau~i6Lgs~e@gN02<_Mde+=F?L8I{wha>tBkTrOj z0xY4cvkDf8{r4$itO9aWO%}?W2n~q!Zo-#;o8@ocM}Cdc2LQ5HDG2ily2H^%fFYq!&c07U?C}VQ z^h^L4dx5ZRuOE7yIkcTyYU{ZdDsYD?*b@LvWFz#|JF;?~7=jGppS>!K!+VJ3>^;Ppt=Ow-c*!b$uYeXRhhH8v3f9qfe^;O`3Do9C zgx2K%+ZF{k!2mlC@+p2$2A!XE|6Su=`yfkJmU}y|{y&-MFInmCEeHA3|G)&qr2l~l zEDmn+0+e9Nh1J_T4pSbizaD6n{pWMlXIu74Sf{2W?H$h^F{isN5lGWU&%)(z&3Hm16RxaR$? zx8bHM#eh|MwlKDch!-743)S&C`jBq-q`Gj0ssbn73Z1-T#`KX zju!nQt3Y?pBY-?O5=quVfYJ4qYNCSxDcfy)y%U7VyW{yT=uY1>Unux4I;)^eodq1m zXe{UGnnuE>hQ<9mitqd;kkeHAq9>saebXlqcc*>02(tb;7wS3{I`Egl$q$MJieMO3 z^C|dfdH6ilH+obyK1_@)+qFFbUagKkP-iZ}6yNdd5|}_LtEv(N{UhaY3Ol5(GjcfP zor5zmMnERBcqc?d=8sRGm1-XMAQEJ&|KyEIO@EvJE&-c8jI@PRf&+_{+~G`PhWDdQ z<)vu%K`Z0YazH0>B<*sWLpX@R$H-NPy+WkVjK#;6>!gb4F@b9Cmau22oiPW8=gT#5 z;D=L~H01%~aOCkfsm@e#;%->zS>O?pI5nX~#JWZ!Hea-K(n*EdIQ{S*;TZvQU4&M6PvS?0 z7fBcdNF$?R6(vK^BEj!NbPa(YKWs5sU2)70m5|{a3K2~ z{YWK+^MWJXI5XNgf4Lbupv+c+l((0k5=uj5k0;K4h{M4#*2q zM6*^r2$A)?z>$LZFMPjlLygA&KkU%>-*E`ywf0=bh%iEv_864J#WbNIK%G()Q%oiZ z_&1RP5UHp==n&mx0bCLV_)-GaD^jkJd;l1j;Axm)07Ak@C!aZ^T%Mj3AOJFAREcDJ)F}r~D`~QJM@+c|5oz92HpwiqYP`@JE-d%=yVl zC?ycxxq%-%(E#0jog)hg65o?RLjRZG?Z-!A9Du+oh|r=L01@RNWgRmN9F8B-nG|tu zk;_a_BNJr~R&Y}iG1f&d5nY>~J#xEybuD-pWjr<*#@) zH(i*d?3B0TKU2D=?6@yEoI~?4wQ1Fj1cgwQj`j5YX#O>(5rv!Oor`W7&xz*9toh5) zEr6ig*)-nqLz8wR)Ih{uDAINY`yeg9*vaIF;(jv$x&!d%gaax9@$4NEzG{hlrrl0I z$wn89Ov`tYNVHCeqxI%wt{rl!|03?=aH^4w?Sof0>yj&Wn`uV^yrWmEa!)-V=mn0_ zm$ofziK+J~>r0krfsb!_O7C|zo2>>J)7W13`x3zs?{?i6jH5iN3O=yOhu zgD(9FOYv-5)I7)&##2m?qt2Kzmlteqbs@w*w1+@6@w~bESBq5f6n;m6B25wo(VuJm zr~6+1{sM?(djZxTS+)nQUdpPGF2{i)0b~(`HZ2Ei$+rl!WKFO@5TU+D{da_czW^+K zN1oEV$I@QH$Qo%+TsERBgBqyY0mk!TApX%0ujJYw8QM6p@SZzVn8zNnq)luoxI6l? z3Mz;ZEqGaK$^fx;Lx@HK8jSbN`J!G-Cg2f`y6kRocIaCYmbqs(qsX4MVD$r|6Pg^3--VQSpk`jH@lySEb09~bMO-bzBD5KB4C;ywcV*G9so5ww&$0g9F>urW0+QPA;}@7#uOQ8(w_%a-foy)fRQ*m7Y!WKZcD zlBCE1Dcx$gJG>JwMNRo$xS6v@_jUXfLDFqaYr~JS6d6)~JUoL70TPQw=;A(&raeha zp6KIAxy~xcZm9c-Py64zPZ|jODS=$f6SUSoxZwkPBx4x`_(~R^ENuwfVh|FmO*a56 z(jf0ixdSY5w*g;CP1)dPs?s6>aLH0`!w7o+8a_}&ey_NQ1Atvmy`)SzDLSf?%y`je{`WZdjXyJAy(>344?nzz2tRM@4*tWYA zJJxV6nN-zzS`@A5zkfh%WNWodGRPrjX3O+6or?h4urx38u8%jO-bTSYr{_iB`G?y z70&}WCK?EbjZ0Q##F17!P+F4sg9K>TWj@IIAH=`i%P0NS@Pn8d;Npyseh@Oyo)f9` z|B0K+djd8uYg168Ved4H6V+WQ2U!Iv4Q4;VwQA+ppzaBnq5y_eg;W9s#0^=*ss$V* z1$wCtg370+j0)2ENLTV7pY`RD^2h?2(4yTs%+(MKuCz0ZXMQfh``P?62Psmc z9xcQCoFgXYc(gX%2T%&}0)zHp+O(_i0ChpV`z87v4HkkxD`Ki5WB0o|wtFfUbIy$^fY( zGa4YWC1{=~{UK@hpn28T9)aSyXiG3Y6aP~~?poebPyjXVx6G68 zI$E;|+EBIKn0bEe+EJu2MwS~3CT_g`Hw2VH4}xrI?H-b;g-`&pNCW61orj_`Dtmnb zF#ixGl(ZNHua=>BkQPA%<);Lab?Qow@YYFuGQ=JN zz~?@-{;s1Et3b9P`|chBUMvdLTvw&70A)gievnWPCS#(@iVYukf|$weC`ii#3~}D< zlCDOTSM^y#T5#(BqNfC2W-qtZB{RQ^R&@5E|U7Kj+rd>`p^}z!CloqT~%;>>t9_&BsRZc<->pGhdIton9qEV zUyl%;ekT07S#k!avu1*9^Y}y?X)}WQ^BSR3Vv?t2+|?@Irb2JIOb#BZk^P8=P29QR z{OEIHvE=RL>Q^o6M7CQMA@|v*>^i^dw`C;;Wy&{qZEm+Xk#=W4%>`BQ|6#Y;_A{;q z{QfH2VgBi6Xk>`HlM3O}kQrn-!1u;)=49u~4^e3&S0bMfA&qUUsdU;e4#)lAj$Zhf zWQwNsM4WF6vs1gJ=y?6Eb}a_psCoZ7InZFot%=#Z^z9&5lk%a`^N$3|BvC38MPnxD zowN0=q6n>x4{{A(+A=J6A~ zqv>0}jO1)ND-+u#=__q47wwkPf9jRACsUSNmaLDmO-{TqTdz#tc)5(#p4NoUbEY;| zeg&Jk8Y3p9zf{7|kr3ZFP1?6_@^@M`6w5y8O-MAQx`vmrK1#mwD7W8vw^X${;u_|#4c#A!@JHi*x34g5s+%Bt(4*5 z`rzS{l9mRa0-BUuZm8*X#xeX{RZnt~d8gs82}|QYIyHyKuK&w!Jh0-9tN3pxSax?? zYVPXjUK1B) z#}x@8uoK^zh7TB8HIZ8x*K$PHpN}YJD%Q3BLC5LYF6%XUi}$to-HYHt z9bnYo{JQ!CWu96@Ep*Tx9Sql3CI;j$W{Nybwylo56CcxZ$_H2WG`aVxP+)|^=X4w+ zhj&t0qdT1OWXR+QEV;LKSa>yAd~&>#p~oPFMHX(c*lM-x(KnCGMlLn$!Lb?>>nfI$ zKI`VX9OtWaa6?AR@{>Do6S2ZTwOY!BMV9~*Ntmd~9jW^I;y3T)%9rQmOHW4-`F8oz zpy}^U+e>^sZ9()Z5z_2@mc)5GKl{*@wObGn^LRQ?59c|N$}}K3Azi#}gSQj!Pd;Pr zIQFf(H=MbBu&3#)e91kVVsZ)r7m(U0nD>QJ={XBGsE6kDd} zLx)ahW9IX>-@?}+zm4i~1uK+&^gc|C@f=pw1Tt-Fq6xlGBkA2DG&SPg#}4jPm;Vo) zZ8b7%Qzhgg&^%mjGH1d&_s06n=S8~Fws?Q$P$tQ@Et-UC8Y?WJ-(ZF55Neh-*PQ=~ zo~wOCb@5h&^Ym+s^FOWZq;k4KfU33eS1 z9B$g!*ZgRcW$Zqw6PlVId}^-wA@CH7t;f%h4-3$)7x>l8=5oSxUMNG`F&jlqve9Ge zmGcdW^RD3?UKESwlp5*juMUSg7ap_CzEh@-Id@I&Z2MNJTh<T5|07>!w3@K_8wv@BJ;8~I^l_TeLf;IsJv{qHV9P1!Xv!=>gE#W>PN z+u744MPGM!o+(CMC)|n-zEA!!9I$T<{n;rt`$o;hFjbd%r}AZM5NkA^CKJZ45pr+j z@P@yvgniD+g^^ZBVI75!@imf*JBC5Jqm`QxI-Kba>MIcf3Iyy2I#`DICw*8L1cHyQo#B}kJ-0sie zqNU}pgc_3$>83kOwC`!}Ms0?hh*Q;^ge@rB%cJ1RQwHcT4{EzX3BT z7}}!DnJ{Vdr+)QL=I3kfELC}<8ED_EU`twv&!chLlYzAMFln;O;vweTJx8=teDEVt z+LpJY5Wv|T0VW4~_jm}_bkJJ!6rV@dcywwM|vZC|Tp8u~O` zsp&ORT-mIarD+8Fk_k&IukTL2Z80)|qbHE3CxS!bmJ?aD zDp7T3W!r`7c^QRs$FQS*;Kfir^$uTbYTpO1+A|5Wr_I5wHck>oZ)IMiArh<-eOmq_1Y+05UNG${rA9SG(>Xb-$A@IrPVLlRUj0x6 zAMd=@k^6LP>~h2l1K%wqc$P_#F`Xa2D#!9(Jk#0E>7Cx}SRr@N(l^N+3kKp*~CsbEAGekyxJN$ zexFhNZf0j8^W*$XeQ7Y+PfRu_Jo^&O@3tas`)s{UI~x8IF9hS56fa>Fr;cGcl=Sby zH=N6TYq3FeaFf!XF`p@y?GUc9W1^Cjj+ATvZ5_z5U`^e*v$RY>ZD?K~JWpbE?ob+H zSMhI-lybt!WFtaL*jPk$DQfrk_EWF(<1}~QvAjBTWiVzp+4n|{N>k#=;2r+Mucyys zt^8&B`HpV}pxAu>`iF8RI=y4@h$0~ZVmZg$Ar!n)y3aY{^wZ?|s~+R)Bbc4&a`RR5 zUf*nQAzV=HR?l?R1BGYyW5d#xvL(zp&m+!k+DArZ-#<#6iD~|U@hu$3n>lNnlUG8x zw5A#?H{Cg}Bt)oJ3>9=iUkiI+WP@HPb!=L{gcOB|ew!|>Xyh|T%iRo)kle7J%=Wz= zAPB{eJ~!y}KFO?~NJTqv^4dK+*EPc<*H3(8nC&KPte$6YHU^u+!^K*J5_P_3ZlL&_ zVzDRnBR=;DB%aa#IE&=7sA7uf)O&6;1YJno%DwiHrb}CPeD;arE37jxeC2fCu5qhS z|4G;jTEsQt)f~p)a`;lVOJ4X@;knJ$TQFg5p8?`ru#e}*K=T%Z2^eLwXAq^%WY&DU zlwilH-c8yFh(CBoZb_qhvXH53`6s#XQO(9*(_x)a%4*ZulGWN>#=zYABSfXt2Ub3| zqisK%b*1yyBw4MB_C1jy`WZFc?v6DYUaU7rC|VTObXofi#s2XP{d>MWutwqG60t6A zXRDj7#BkX9puonBcV(1iY}sec7;ZGoW(1hSzI`iZ{G7j8yf{?1bAC4@qRl*&E{~3& z5lfxWjpLg+uNmJ@lc~$m(3B0u+bc_J_Jk@ALTRIAGY9rl3YVc}nwC)kDi!$;`2KFXuYvfywruH`P07 zpWTz<2_ek4Y$wf-k8W?pQ?|2N5C3i#dT7Y3J$~y`m7kNd5=_l~>+8^ILwP*7=pqn@ zrvIF;2@bkma!R&s(EfE`z3D{dZk+;syJfzvGp7diSEb}~W}<$i47p+L2RS!>x^CdU z--8BmZwWJQ^8E7`Jk5svq|6}>q#dbpuK3r{jcOr;qs@`*E*tbv%>aR2&eJWfr`xCFS~W#&a+yN8Z17r8K=9)>kJZe zD)e{N)|)<7B7Vz!%coK2ogFJ5E)d5yhLdX7C-bh}4@=!JH5ft7{rMWfGw>&fun{(C zup3xvWH@#s6p{Khab)}}`t@-5qV7-+xv(Gc&Fgnb(y~P+PHys1$gk1AgUJiC^^-rJ zJJ1K^cl-2bD6cw_BRM3g^_(y&UGlbo`XY|8?M@|AY?Bz7fB5Z8wju`}aehWS(KC?L z$&H)NEQ(8Dhqsw0=56o#nYPL)&fAgi3l5?hjNZ**#zVDw-+k@A!#ShK_p^%S-L=7A zojHaEpUEB5m3UXC3r3TK_S}I(>^ZMPmlh}1Zw{NkSK#!`H&&kCeFZzS0sll$+=71+ z9n%Fk&!0%45#XazrEvyM+HPr0;?D7Xr$8sEviGFHs?|V7hQi$Cy0b3wXu*!FB zheCqp^Xd!(cN~Mmx+F;q#!BKz7SlSo_95qs5dn)C&3}-8#FAq^u12~IvE-n~FOvqP zgG_wUZhPzDev?MKV5^gYr7@u)2Dcj6>*mH zddsP}g+pJ&Uf0~(O(+lfF*p5+IX>NBy$zYR--<{mztB+1&*Y_*V3@VVEKg!xNAOm} ztIZYx4i{b%f^7=!RDHqW%6hwd+P0(SD;DxRN>dojcueldZm;T-5zQ`wI5C>-pXCv1 zW{h_y>qhTpl>>rnX@n@8u)6w(u)SI$Rhcl{yf_hvs=+%^G!Xy)cY`x*_x;WNyHYMM zg6fSM!!C!TJ=&MvKPzqBHI-zvx#WK%ci-DzzoV?OJr|}LDEB9~p%il#68i_{8+ws* z(D(Lo{Fcmr<0UV(KPHd|n_UK#U-S~~A?ObWe@g#3%5ez)#m!`VqmIXQ}=k}XwBMd;tozS$UW(}S(jYM0bShVP|2?{{fC3t1^}#h(1HQ@bF) z^JofxfCLo2SsEGCgq^NXTE(DMN53YvY%^PL)s~SiPCMDZEE8;fdA^dfb!5A<@(=pN z>IAcy!P#W6lvrqT(6zQr^m`bGfOe_10XKf{OciCu*wt_rb=aIGr@la&aHqe3{#;7o zJ1d+@>m_vMljhl@PixQ|JPb_|nojFj`5JhpaHkNt-gGLbb=M%CVD%zcFmVuVk;Z2A zY_5v7+MJ)b_GkLc<;aD>mE?{*8|~BX%@<8h<@B3wUtj4=ow)zLE!p%hw(-{YKRVfC zy)RR>HjGjSWd$o*`Uyox=L+g0)Ww2cmbBL&?>$>;!L=>mKcG93CvS4(>fEhV$B2IC zxsj-@H(QVp*&7|cGB$9q&+wJ*9H=DPw zvNH{j8T!7mTFCsskPWd8L?j*D^)Qq4{P`<&`ScO^?<^gdecxHS;9dKTCbcpD+gw$C zx7i)121CkqKL+NM&d&Tsi5Gm0#!!lXhnx6EPH)oaS4|6TenV0X=_*yw;th(P`zy^L zN=f+F03C*U-BdqYv>+ zl-IW@<(x9h_)qsjY4zB#yldh*mWkIB^W$wJLW`)3E9(RQo!uy3D9&8-_?f&G`{<|W zdeeV`H@^n`yz4*kCo6M3LEP?Wb|h91Iysvt-JQRX)g3B)^Gurh=lFc+1_9tEVa53o7rngnhxgqW1lUns`cTbH+1I%ZY=lJs@1)cm=zG7yM_pyI$4+W zY|Sh@J;^`HKxUCa7DIfj37_(2Q`U4Ztew;hi*aUHg!=o@@uzoZu0fK1 z$w5tRB6pMLP0LnYJ3roY**L!YIV%r2XH@qyGV?{tN2J2jD%Y0%yYHfRoKke4pPkyi zSTFqiu*&#x5_A8be!r?{BVw9vNv zNh8@Qgm&VEhK`C~^{CyS=PHWi#^;BxdG`HHTBhAF#l9jwL8Yt-O~7A8XAwJkSJxu^ ze{t(%q#K_-6WTVDo32Efd-UIr=b?pTXL8}1_WQ@1Rc6^FG(xjMUOp%GL{pOH&o?gg55aBgavX*NYzs@fL% z5>nmp`*&MgqIyqK%mEcU{9OmfxZ2sz{Tt1PjZ?neU?z;b4J%W#S+Ej4a<>=U>mr!Dtu+b41ey-&PE112NN8`E?zjEp~sFZTWLu-&2@enX7M z@KVh)<0Sdpa~gR^FF_R?ki**q#baS`qX<@>^Mew_*w+=pL1Fp*6;d6_> zo8A1Lozpizf9gg`JL<`1Z;WJ(#TB2zFW%hr7oSwJo+x#+CHEiYk+`q@<)HSFlh1DL z6jpNosFmA2{A8u^Q)sOP>2XGXk!I{_njiL{yY;xu;qLp@QVOlff7LtO6uR@_OtA>) z+e_(cb&t<8iVgntkI1#pR8u#W%^P4DX-qej~E)t4hmK6KsceQG1xd#31A#{F*@ zJ1NbI(ZAMKDaa#fjmqqVKNOf{)57C0Cq&`!;^CsnkAt%-!HycGh7S+CZH|QZE%a1A zGk$a+-ARPdtbDX^Y@(~!W*MvNDw3X^^HKSc$aBr0p9?EO9gBuxJa66&&LWMoY(9y6 z9rF$3O_Ap+YZSBEFmg(IaH&DE;-XsjD_Hx2;GLZ}UzFXLPbRGzB@26%1jqfino;hM z(;D>`njnrEg}t+aca-efrz(YdmRbF}D?r@NWR$tMb2J2F1TU^~o^4D_VH|#051NhxNtiZ6j9V$Q6ll z-d8&wA-Uo84QtP9aKQJ-VB4@02r+&q3KLDvX17vYiLj zG(sLElW8b^ALqf6a4x^4o->#5#SV@}$qsAc?GqcpZUNaMFJ~$*`aE#6|1)=JS7)lt zx1VX6b=3Q(T87fYln860hULGn|2g&odaU$>iqeg~KAA&gOGloxN5R)zyw5oDr#-q< zsCxVspP2kK|e)N9YBHibvs#S{NXwo6J@y*%D6%0~Nd5+?9;@)KyA-e+0F8A3L39 z%eW77Iq$4E+ooh3k+)sb1hL5bhCun8U#ConhTN@Fs&g0~m3y7ad#*O_)}7e&E!yl{X}{Pma8Z`+PaH~(QJqHg@~%IUW+l(tX1 zs>NO6FBZ%EnC*P-3gzApE3qQNE9q}<{=L(ulB`;7t(s% zPG1FT?NDBk{%yzWu^z82_ot2r8PbdNk0T|iBFaw}>%XK26_pA_B#*z4*z@LS4bmt%A~SaIj*+W>=*9up~QtsKA&%#XH}+FYx@T!VAt-HNr%03 z8QTuyjXE<2L0LL z_y*@?DPv!g`v%6A>%~mmzsFuG+^3RK&EcpB85{7xjxxORhd&NXwf}sCg!>6^JU5%&bjzs9nht3$am^ zQGR{+KOtz%E%dKb{gX!wv;O^T?XvPfseF5~-B!-B_|@N`Q?9~q`WA>i6BC~}e~(#V z)Y4|uJeh@cgKt?Z^$R68<3D)o8yfA6S$)$zVH06X7xi5 zCm3Dql{@MvZ$r!Z7x!>!`p2i4?;<;ehuoQBtNF^WPe}5xU*E0vAoo=GcUwfjQ07Zd zn<)R-AN{lQ*zB`mj0-kN?q%PD&$Zxd)D@}&e28SnG~=u_b(`a9PKZ)Ac5EWhYvW9Y zOL;8b*K1Pg*Z6Vm&D;4YJXiF*KVB-oaa`R|_mzT8|Med?&rK>#a@!HopEdQgp4dA3 zV(jphfa<2Jnn&q6cIb@Z#eQgrzD4W%TY6V6YrjDnmz2MIwRTVGVgX)K5^L8{m?|Lz z)3$56eBy-YSBkiO-bnhS2-+CMxYWPkO8?4|zh?DXqNw<8$h~J77Z)SLjEo;h@i_k; zny|YkUg?;Sl{9&&b5VF`H}7BPuY8N$J$d`BTqACtx}}jNaU}ouiA(Bs5yXeb-hZEw z)!)%?3^*_RFoAs%msU2Q(p@ZB)%sszrkwgImQLh6!>jbU_>1AXk9XdFbPxNL=&FN! zUVZ$+1?HjqmcOsNRxy^h{>mS7O{rjdd3_taV6U|;hqz{2bK-+@h^geN&pS*l>SqYd zYD7x@G4;aw_9wjMi_~Z#Z#mTJo3v~GW+>H-^VcV}^+U)NZL(46t>FsBec^eVBM*0v zc|87FzHXwFTH)3%VRpMeFStOfVQQl03-xVo%mV5Yn@Z@5Do{NR{r?95aXK~g|0pdcEX<5F1 z{U7-y9I%aie(Exoulcpg6!-F^;ES98_Q~0-L)*A!kgTR3eaebf7iiBqj?vGFM0}M? zZ99d^7mr^JRx(%)SFZk+_0&wPO5swk{nUiN4o=>E zV@>*!#dG{`|#cRKSv@AvEVd_AAf*K5waO*bvpuOGIg_e;~H zCnYzsU%vRbc3SBR$(2KUd^gPb-L`1g>sj|cO>up=SUUK@;0vzaNZ}lr-*({<`P+pN zfuH?l>0Vc7-%A$;ZateaA|=u#X2Og3(@Qr7iJysjCR9$!4s0=AJ{&wOV)63e74Mg9 z$xeOOzVCbNo4-Q(UH|l9K$n020lKsyBGqpN4eFFv4M#H%cD?@YzJANe%&FOZsGrg2 zE>Et^E_n9u&0`CvTXrlsl$3tF#Cl?5BQx`zeE-%QN4_n-98 z+r@A8zTNv&(cbg2JtVmE)k|LR8>{?fv+w(6rPnBnY(emaic^y{eb!FQeZ2N@%eSAc zFS?4nR~L@5P{((y*|UG_+0d)ka}OU%&PhHWUq0riL$4OQRD`T~jlb|)`S|#SsbvM% z&mS)HcUtmt>C~skKUsR-rtbPNiI3bq{O8?}7qyF~NJYUzk~98Ula z8{uyz-1b>4vPS<|Jw>8_(X}M_&`U3_rS**x*wB)tX1SeTURPz2 zPX3fPxx1}-USjJQ;L#bpL$OJ>|!Ik#M`XL$~$Cenr~c#);hDR%y=O znN%~)DJ%W)3wurOp33pV4}W(0Vs7%=#D8Dx@|HwZHE!=T{`^bu=cjSapE|rBoiFXw z7)R1c_4JzlTj3A1}D0#S9PqhXC{EyG52<*XWU?SkU+kwq^)tw_#UxVw4Imis@ATIy`rb%!&@_pj2Y zv%VVjJa{>O!{66Fzpo#6ea)ufrT=_2rd`HWN6C7 ze1Ba1^pyOJlzCB`w)}Q`%il|4O6?U>h+BVbJxBlY>EPkx*3dsvvUxAA9js6iS9bTC zJ@+$n*rv;l|O#O+!;^fPa)P(KHiYfff?NR;h zy*r$^oSwr0bGb#J{#d3!#ne6c6` z?y~F4W=%Rb@~7cqkC#`zal6vK;$faC{FRTd)IXQ&>+^_4SJjHEoT_xW#9W_?U?>56@Iri?W=sbMq>FbBl+6T|4!AJnpaQUHu81Ivn_p{PY$>llIQy2=>(#3-Dme_%~#YR z1F}yy4timqH7#A`Ns`EFY$UXJ7dY^M51oj*l(KyE@muuOE@z7 zlIhttMaFXUZv}Sc&7p;vKkA9NJ#}L4yqjAZ!j{D2>B|rP-pF0Tdz=5Md`vk1hI#+B z?!`+Y3m&Ck9+Wlv$I`2lzwK%XwJys0&$j2@Cr#ktTgei9L;H=ub9?6ebD$|{4*lb; z8+S^#jXhMgbLW)iR~PG-Jx^G?=YGc2rRc>Cgl>rULC?Dx9WH4nLO)Z_3^#Iq{V{3Q zjIB3k{1&zD)vV>?a|@G~+n2P9wx3Q;+7SNg%`cx$X;-bcpIB*)$t-zGYq-B1Y5RL+ z&cPn?*3wI>orBgsd4SRzl^5zN!{UD2v5GmlBfU#{%&>6$TRd`&N&SU=y!$3+s`e^#*cT0AS*kT-`PIz+XH-EvemLMI=MFxKe%jm zS=_?;>pqwAE+GEP%I>RYkHAR#E95I`p6|WS+CK!=YsuerN6aq!4(U(#?^#en_6nK%uR_XL@*9}eu9k|RoM zjhaTpNs~DfCC!vIe2snjG%oYi#%sT|5P~XUzOlUQh`pGzNG53wTq`@<>Qt=w5k1}N zv<5xeTBzrkg3?6EDVMo0N!?R2ji~H-K8={mFFjcMTX5;3YI8jIxCp7t4P1yaejdgw zm*3`#Yw)6U-8PLlZL7?kimoueN6hBV+e`OOS`{!|@zok^VX5@Bh3nf^$ZlKQ{M#!HVHLS^u=)3>uPQn5K5LzgeJXhe(0Q9&T0%4V z*ad|qdWeYE5;YBZZP{>ZRp$b@^*dO9YDr7T8uW^mJAnDF=jt?O-ywU1T3$x{y|Rql ze&e`u;<1OXxS?Z>uh*$9S+}Ws_hjSyATE{(7bM%MC(2u70K34GZJ}K9zpcN?#I3Pw zWJ#?_#mYr!daKt>Y^2DmB_w4nT7GqE&BEZ)b3}S;s3@gLdvfkuf11KW)~vw}|CQ>C zK2Gi6Nbb{3Wftwcy+LbY*3VuoNneAH7her#gW>t@VA%5PBIyG3pL9#!@~QL)%g9hV z_>n9n%_R!=el=EoVXMm?q;^{?zGjs1{Wi75wOUgjI+Y%08R_niJY7`UyZ@6oJk^4( z#*lF0h#}=Zd{tEQm72(4{hk&Z#7P!XR3oU*YktkXUtA~LDT?Z~pQvesqQ^ds-wzvR z8C%JF5ZtT#W#9go+dScDqiZm6Om_@UiuVTX-Y|9JAEIQVP5d^U>!DR^Cb{7!M9EW- zs#oyOE+_hbTQBM?>l3^BdrQe}`x^U(+q5j~H<1S$-~pAweCprvZi2aL5XNyQRquE&6aUg?s|tm;7f*xuomPAz!_HjTMeDUVT|cy<3B2 zmnF3Q0QaJaR7#pNZr(CH%l|_i6Lv7-S;3J^i&Qf2r-HHRjP;(O;+3n1)|^l8EdPnk zM(W=$9;#G!(cU*)c1O-;5igaY z>6^~_6r0TV4KM0_X4fesC+QS6eqM`OJa3RE*&kzmw?G^y1urL+9kx&EuzGG*Dl)DO z{jcJrF>;JTw-s>6Wb-9e6&!az;^dO~M*;Naq3fF@-^*^LTlVOs6N%~`*N>d9n|Kod zTsQp8ZVwJ$qDvM(+=~>xPM3U&dC?D@-4iqospyG^pl819@xP$^-Lm3>ZnTt)uE>3S zUBfwsH?K^^4W)sA9tSJLBmMaO5D&{b-%4)!Uu5nX-o9U@{E<{{g&06n+efjmRCzN^ zi<~(x3F4IkOnZ4ZvNuL3tWt@IPgExp36M8h!8TKXhq0vg&8Z0XUZd_>%gYk0v8^4W z)jJP5XERsk@b3<-mt+we>JLN2Nbt6)v^#q^zp??uWsD<{TzHu6{0Q`lrLqt0))Ke| zopnA7d7Cbs{>!sFY7j%)9nnlTicy^~~c~k5Jo=PAw|*<7OZld+?dIb;jkAk_1~3B*!(sHtvlo=g||1DBR9W|`4qKcN3njt3_bz!n3 zH-rU1*~Qk_wP=6JULn3qB_PuN+=b{lje8FoD!~707KYjeqy(VjZn-fNS%`7fBKPiP z;It!_&;l`})I?TS#M2#gi zr=dvtRGROL6o*O~7ZyR03yN~VXi8kc5%_GhTy5$~^Ti-X9xO1Nhge&Dl)OedqaVpIDj<=z{JzVIb z#M3pxuHUFhfTF*meix67f5(5JIre@=#It{|)kXhG$L!3+m$*J~@)n;x?=`zl6RKZ+ zn%iO-)F?P138T8$)KZpMjyD$odVVc-(sL&gp;aQ}@Y2u++)5r9)9KD0)H2H;_=cX0 z)7)*AlZ~7s8&^`k^O&Wb*7N>tJ7Y3`m^a&(rNs> zxi{vBIX4+O#b#f12b{OYiT`j+jiSdDZR`(^R8ImTpgA^^{Gz7>c-m2nYk`NQZV%o+ zezoYyEnjq~yVU-yN%}oRor7fAiiIISxS+~^LCtJ&O1{MHuUi1D%WIt?@XF5e=j<(Y zY5FkVvUu{^K4_KII68bUc$D3dSg=w$|iEx{WxAM!!lCuAC;VmM#O#`p9qt>VP% zj#7I8Q3hbsft4UCjk1`xSXQP`SoyndoAIkV%RjQ0ZxtTn$BW!6&G|;#d&>g;b{b9+9PR+*0ql0-aT~#VJTrmR-yuDr>m*#a-_lcvlLMLm0B^_px#6Pr&D+ z=QvoY&bG1Z`oGRy<~T!7JaXN0Ig6miiR4isoHua7THel(p^i*5US%R5Mhx$nGGiw* z=qQm)eeY|GZz8VsdT&DqQo0I`n;$Y*wmRwqe=T~&mj9*96VRKV+{yweSNU*unL6B^ z678CIV<C#C-NK*$(PzR@Vbtk^2*D@Os$_VL$O zvg8wiy%rIIWNbX(4|>J)LyNGKd(F!1se%}-Kt?jaW?*@yV?N4Jtv=giTsTv|QQV ze4ZGB&gl6_QQIQmqmBx{o#iVWOF|o)=jI-cioNZl7-MvOF4^DgH1iI7`hd%)fr4r# zAXPoC8!tkyd{gZCA4 zYLSVPj2(XYg}S&BK^k|CeH1l^x?pfoyO#-8RFgYwx#N4I^D(DrfqnTzD)Z_Ztah+1 ze+)l*){>1nLHA^-S?poL%%d0^p$K_tTxp&ev+>9uALgv@acb#RnX+?7t&5z4#d<#B z9x466IbXZPCRmqbpLs_Rw5WCv6CkSYa4O@Za!Yvl3}vV~sf?2*DfUT{jKO=*k?2zD zzjmiu7u;Lc_7wG?f2ST9ZgnR)lZAShT;kFXBv0?JGH)@F>#vLrH|a0gnSW7u-7LKlvWp>`on zl(5dH>0*gbCK~aNo+ww|Mz>S{k`t&L@^8bfU`6({uPBtf93{m__?WFaDF@x&;qKzh zxO(CPvdn6|vgu+Or%Knaf1A0_<&=Afe$;MA^NM-C@1Yr7>Q(VwX|?HYZhT2m&q#x-vlqbiUqKOk6@y?j%55GD#k_s*7W zGx^oh#UH|v%0bu!xVp_Nkq5gd$>`PO*t1$&orGFyx-J}f?<{AA^P?GRCc25*vPdO) zYv8}`MAYSF;?<+*gQDt#W}Xur*M%DGm1S1B%TJm4Vt3qn5 zzEmlBua>70$az^^IytXMH&3h4i`AMHcmi`P4~w*UL^(4T@^JqwEeE~4d<|BnNR^C1 zlDNkt9>{m-?^NA7x%33ARZ=8~myAYcLCB1wf7#8ftFgo*Zul(pt&#uuksgsN*DHBV zKDBh9y)S))k~=~^gm9-1LBfSSN0luu>TadU-AWCn2T@hzDzmujS2>;vm*y8?SJVNo z3h3hp6C_(hv#~nETb!e7m7@DC{d=!Li^x8sOs}lQf*k8ORNba4eVhv zU(h?V5OtUg>&FCAy-#JmA$qJ#oN00;l?K;bNilXqTmGINhsvtgAS>{r{66^J8h%8O zDgbh@0Ch!S0Q88=IvuVWuC3d*EJ>=W)|B1Im2TR zb0UvO6KSx4h-uV(Y>s^>eCoy%@(<%6A1jp1VAPeWYIoPiHo1FP>I8$Z$!Pm+;=>v; z0~UIRTZANYcXi(4%JDp-fV!#FJ=Oc|fr-%1@yiqVBl-E(Cx&PMwy5Td8FR|9P?h-w zgn^(~Tq0PDIjecTodp4|bR?Woovw&TA@;$xYRQuT6}$pB_JSJo_GncBjn|`oHXuOBoP@7|79~or(w374`JO+UelYm z$3c+bdFp~RDU-HaRiLuOh7;Bu#XiYIeO(yd5C0C04CgbhWHp2nkw`e<+FjTnY#h3S zdZol4sV%&6EKouuFm7mVyOUnr?9^Z>^GRTw&`+Nyk~bc0W#-vOqu-#NpF>GE09JsF zH0~}R%Ibis+0-Gjt1HgvUaY&Nj->V`Z^mYtt$4h?d^;o5Ln3&Q8(@9>6iHW;2R?$H zP5rkPzjsw{Rz>9%qJ2rtK{^W^$OJpE{BJ`9*~i4BO!5W6|2nU;w-c;_BGlJvE!l-h4912!;<{S!B9`; zQbzI#luv>!>4T7uGdeNYu%(U~Xk89pI}(tEe!;|1MePMm1;viH`W_1Y-+&+G`ttw9 zp$C4iK(A>hbwQn8WxP~L5kF1R+4TX_bDDBvZB~AfJDKPxYaH`%8ktAi=12*>OZO{F zBz4oK?2GP(WIAT(rw-_KLkB~xL%)E;xGRmem$;MD)rzg6YL7)}WqV=w7F2ornhN{5~q6Wv_XHp?-W}=$ys*8You}q2dswoR~M}<_L z;g(F26FW^2*}>HYVz1N*cprW=1`KU9g8V=tkh|e{%%6H46!hnGljdWJN%*IL&R;{W zGWg%(X7rZl*i}>3BYJFC>DKA}Z}Ba0mAQkMXxNQNwp6FO(qV81z4+X0 z3rpyB(%C<;zs==x>=Y!cEl}fvls9$~#yBG(`<^@xmq+B}<%0Tfna@nMtuJSiQ|wQ& zO#(}B(F5RhjrYuZ(Y=BvM2sa0sBR+?P1~a7GI{0QTj))Jskz05zFKIQ@26 z*RNy!WXh$Ut`sT!CQ%~W{*7Y*V0Th2`&9+y>^=>k;F({w0(+`0NK|b%)`%owdt{1R z`>yHvO;M}yj+lS?;S>tqTEFOxL)se24bzcNO;O!SN&Oqik#~aiK2#9g-YLEZdVzbN zpM^T(W8m^9-Xm~2#8=d$4q{WU%yBpS{b}G{q1#mVo zV;nqa96E^l@Sbr5{Cm2rOD+Hg*Grb@#KZw>A4;t6rw;34hEgBr(JPR6Y&fK6bzlOs z(1s-msBnG$_zF`ABsrc)90Jrc=>vsx$~kBN6yE^dNcuM*a^)O)zbBGQoWM7fE82xk z%s)^L@(hJ#obC}c8>ZcoQB;-FYb=*5_c;dN;{jw59}0kWQyJk(4n+1ng8DWHwx5N# zG-9BH-QXTDaF$T;5iVd1I(a1dks!;EBy2n)graF9uNiJ_lrHVQz+EPsX%rU3py^xQ z6%bCN32%x6sFlU^N5k$2fLGiwgu6awU>}9{#rj-Th^t5qsC~sQY3f9%{=j@A08Rwk zv9_o5#)^YJF&q?zbI{A;L)4f@^Gqgrtjr-;QldX@v{uoDja z4*+h!Q!%L^4Q5bs9g4f7d#Ib*RcfVVxykBm>_y3q>(796rIQap%>f`qeg&i48;T58As zdAr~;O*?f@Bh)9|Mr z0|g?)OZEAgMKN8UO`MIV$^Z}FY6Ds;s}NZm9TU;PX={uzChK)7HZjlm_nYZWoiQ(8>8p#6fFu#(XgpiUwvN`ZfAF2TCe-y$d9~DiXP5 zihv|PV`vZz!UqB+(Q;8Y5WE-lf!l5H7CtHtTafG};u2ma+6M~))f6spVTQ2XCj972LW0zu1_%s6xi^?^LEXwW(1cpau(%SCV31vr@V z$CYe}y#<^+0Hw0^_=+u!?5m zfflN`avj>N#mF0aDZm>7aK){9PGgfr(bMcqRT!SYABs~@*J4%B@o(d{vs#D09au`%t4h;v2H9-zok$T{37im=mA+sq`mCod?vHZ%KRs^4^g?l z;NYL%3QCwS@UGjEBHeMIc)9urL#s{dkr(jwrfO@!)mkgom+%j>_C{R1qc2h*X zfG_P{Hjye;&%tvHX;{QM6%Wd(nM!Vwz7Xe_s`=k#0YL-qfHe_Cyj96ZttQWfO+`|O{| zt>yVx3u8g z`86#^X66fmW^Efz4}$d@((6;p-;|)yhg4yKDaOOSoAH?@pmAtldLi|n!x#{@Z3==z zdCqO(m-FuM_hq+6I}`IV43g+~(j(bJ+eXk%_u2(>a0kLk z18Rxl4<5P!35W+UnN?%|rIapuC#a_vov>HEQ7p69ZISz&)>C-8?%9qRGoVPZ2x`A^6miG* zT0c%oz{bPL+b@{}xOC#WWSf}lhb^^~;R(o4Fi={F=#A^hNH$FlGPUa#lap#9=nIBB zc=vs|1pE=4t|rfKqyB46zy?7Qm|P1Im|n|#yx@er>_G{=soUc1g8gOn=Gcx|?3&^a zQOsDm5Hb|B1}5BXDL&vn6OGES_^Fk?YWLJF4BUelE7b`Cu&=04cBM}@F4jlW2`13v zM!$yB+UXF-3qoPzyDu(w!B}9oc@aO6AwWj88MZsHhj?{6lbA}h$+Z0U+h~@gN-kkG zTSN7o1SC`|XpN;iija8Xf8O0Bs?+^73h~WLlkA#KU50}-Mv7w5kcNSHprbejxRPfd z$Nu%#FKMU`6lbn zQC=!;5t|?ZEuQU#2gelF@;rljj?I`O5W@vJyo?!6R! z`2Sd_*XAGS{tnvD0I}4S*)LiPE!~pi9}~Izrf3C1!nh@tqcRsOKk)G!S)6^sab^h1 zylm2OXinuCtOlsb_Wv`zVx5_ZU}==&aPIhr+5)uXc>=$4e~t9HNGa}%BJjFZ3cjTg zh%_*C=4xC5GY}5eqAim4`sLv^vQ3(R^ua%r1B}VQvx73Wtrvq|u~-e^{SL9&Ixgyw zA|JCk4HF&Js}26jkJu#2bsYf}VWPT#GZO7SM{_>`Gdo1K9;>0b-%wlOpdA)SypKLV zUeqNX+Gf#eZZq-KG&oa$p&H34J$&y2RQciwa%L<)8fUiIPxM;sGrJ`Z>`9hhu+4hU zXCg^pZ3Ji9yx2VVqm|wKCO^obRT}=G{xtuJ?B=0SiO!{nIOJrf5~Pv?L7UBv-1V55%WTo!zd zA^Vf{Mt&tF+0v0=NU0|M%bg+e+Magtngn&$h&U|{LLNV0?W#@SYv#2vvk20Eg*!c; z^`&M|ek{q^fS2rBk4PgKu?IY>)xC^UGb|X2mQs-FPOpX6X`pmkzs%$R&pw?WN=^j& z+|8_&BS+befRbA?3a#;oL4zBFXbTVkO{A@!2nr#!kUR|nO8T4`!uX=wAfn83hNI8< z-(e@EApB>bTd4gvSrF{te}z0=rRC4;M7Fl^j8wo&+uy#rq`aSlvz`Ubogn#1c542a z50iZM<+migO#0KMXHpC2Wp#LqZByk$Z5+3=Krp7})`#!f2~FyZb2Jg_qCD%8r8ujv z6~<&A0LR0b^@*E=WxyAl!Aj})T&Ty(b*x422O34Ak7|4s(*#yX1T;eUa>g09D=j5K z(ltBJ$=yOthDckU%Po?mIQlsg$>|_c6ts697)e9$ehybj)5N+;81Resa=jmQ*{%b- z<_YW2g4gR{{OcTyUyv-BYafS3QXjyVw5i>}NkIm%ZzplaeE(;^Y@>2&IZfOboJ`b( zUPN7}<*0}fP8RBcK5clO>~q(aLWbkT*W_YoI!Fbab+LA8;m9joHQD|J|@22YMkC>{hk2dlenV{WPXZXB3pNto!Rlw~GHY_p`*Bcn-B#_2VQ60A zC1%hzCS4&0m2Q{Evl+hDE`Q^*%-+nAv)!y%hWeExnt_z<9t;-~;?~o8DKKp&BN%(A z4=Pb+JY0TiS#W8~EuG5P7+a|wl+x(^WP$9@%K_?pSDJP%~@{TF>OI=0SDbJ&@t#_>N{M6+WZL1YjQmq&q`#x4NJh; z!VHGV1aP<6f{HCuT|9EszKeSEhPT|V$K~7<*sYsJB>hKtT0a<6N{OM%EcVYGK`)2G zhV2wpX9QQ6ja?HpFORict3W16f0A9CpG?h;@$2b}oqpyT6WSwdaeWzbTKZLaOGrhe z7L$mE$J8K8WPd)JYFXTKxMz2ZM+M%H+Zbbc$4xS(V40s|-e)c@6H1dt>vo(d4_vqR z@G~G`&({1#Ryw$b{U?>A94vO$!g`jjUy4-4KV3I(}$S^RNsRD z#0#0r!*`4Pz^xHxi$c)51EJo#g0ZJVd8i&7p@R*Q5~&!p#CPV}GfyOY?tD#+hJf#d z5RaJ|4;FEF>0C_j)eL~k`uA&Ak;Vg5n7=*1?AmMWG>nz=f#mN2kQxsbL}0BWifOGg z6Y0quOaj>LR~N~&lXnF50xS12?Q}H}+fXieumf+$?eo|7WrhO0?s6FJ-Cnat(FD{L zt%Bso25oayMT$~sL35xae|e(B$w3GGs;d+Dz~MidYESSz@!ix-Q^Us}NiGno0wPsn zpZ|M=x1-G{71V?MY|hsYM}LpYN4p4V^&=JauYo8lpaUCRu>Mecw0PlJedH&yt7ROX zm547a7eqqOp@g67=(CjE-;lthog!w)Y#AfyU#b7Bz&e6_{zw45+D21sYwlTwUI6vF z5WD&@k@-zYbeEtz@^Hvv+E&5I!7>eyF?6>~u5g5&j#n}CGK_ub@0Cue*Fo94di zNQLgxS1P3(Cjp0i#N2y7+IZHdCPn^Zz@JcagImZPnWEO%q|fhQSn2=wGL zh5GVO)^^rX;<63wolL+`6~$g{8`!C1D4&Bg(6b{8(Qh>uA2Gl1nWZGsVHH}dOA-_Z zX^t_A7*0h$#fBR1NCwmXV19AO%>Y*lPJu{eHg#7sXY|+>T0F}b*#X)`NWTC7Jv#gg z5aFwIN&Q1y7|=`|0af+|B_ELfP}v_@%P*Tk4Yq$(2Dz9Omi7W(6TQ@i9e=|3s?84- zu96B%u2I;07x;E&<_k+ASXOv5?LB`HdG;ZbOZM{mVtI9}+cfF_yl4VZLmuSxuAn##QxHjiyksou*mcVJ=mGDR9OKZNvmRF1fB zt;gS-n1`N}LPw_#Ru^hS5A19dpYm~axbI!M^%JOI@Cx&Y)l;dz*}njjGO!E=mAqOd zmw;)B(W^w+|3s(hmD?GquaZb6LOM5kQ(5(NRJBz~#!>n&RopsLZ2VfJ%>)mGr`+u%%8Aaa^C@+9??%>n+z$1Zgc?W=q$Qs>AfaT&0kF?a)>W2sdb8FV^XfJi1g))zo<@b%Gf0Nb2YT!E;V^wro*V4wPq9b#d;=w&98aF(pD*)QGf`;{2;BpnW)}$QQ8^KN>cpxO+ zPOc?H;{#?(%Dbw(sgQM8<7Panj<1wq{wvA%*;kl}N%orxSAX|WsDp4y2O!S)R)~~D zLx!lB=w64+R2Q5cgD0M_|GrxVs^J5HZKK1t(fQE29t61Sjf*W#a^eq1CBh5;W;=+h zp%QMV_CL_`)L;SPW0>L(h3KwtM1T_nlhckxzteSoL%reUqr&fuk@eh$Uh#05<9{aG z%yoHkR;6popRy|SExwX``;a~_vxudm(4~-*VIT4H`q-Uu>c|{)TZa&;<2?vSYnC-n#a%Oiyo4|)V0=*2+{pGMzAi3-4llH5$q{x!go zb8@qlm}&QE9EC=hh~5qORbe6l6Dc2!K9=vve!wI(#AhZ7#|LB6fZx6%#7DVoRn4yb z3}0u92W5?`2>V2x(bkDoI=T3cpbTEioBdpm)tUnBce&L_Coo4#emZN7<^?h@o8=1q zU}M2!gLgJ|xw3*Y%HhxbNVe)1Xfu?Y@HSIP%`3S;%!ZPpB;=Me?g4w$j}7GvjO@pH zYj+s<^ou8^4*Q5~y~Wyi(Co22UIzmqV<9x$bI{El0m|VtXC9+Xx8^Xt^xP9n$|(uB zqy(fR-(;Ja25}4RR+`E%!KjlN@fe6`^FJ`VHuXw(Kyff}(LOL4#7cqBTDmA+i=BsO}r zHCSJ4j-yd*0Jj-DQWaez}qoBpJfNj~${Y(~YjfPZ7Q{lQL*E$$7w(TeFdM z)r=>9*~2d+R) zAmfokFc}i#=>AB{peDfB5VXG7X(~bXG%B8j4No%i0L248U@*-k-@mLeahSwBESF!T zuzhXDLHJXvV{hx1vY6%vZli$WsV#26+t}tbw7-j>{XLuNu18_824tWbPX3-_5{2mx zK4>(_Z_I{EYd36aH|HaoF8Bg ztDtNUvjO(YKAN_XhBo|4SfCPnU}L14iwMt(nRBC46j zx4D9E%?_nt&N$4&DVgK>iP&ld$#7{#kmZo3&gHEC34x+g{%^X>80P`)? zxQ-FmX!z{ZjH45X97Dp(6%OPm;Etg81GZf(^GV|dPoqq5w!O%4e5q1w@a7|l*l0)A z^mW8~EZZhtMZ4u8HHONj$QO$75ZO(`RT@WIb09ttCI4VgKeLxui^TJn+jO_N^QTfL z>Y0!%dlIqE#$3;5^szgbER!F(-~l`pyEsBFxnjqv-eY9+Lqg7RnL2|Z$($Sn12IMUpS$AhFd5zpGmhxLI!Iu>Uqr<%XrI~ z&|0PizTBmh<>GADkcyj1xzEES)?OYtMk!G~q0u%Ndtzp@9`H`hqynMlk6 zbV4D}Nj{N^+wBe8m#BoVdwFv^b93zRPYN(l{NJ!ROjp{%8z1t4H2i zGG4;C8N-hL@Km}YUK-(RRKl;!4P+AJY!Kd;3}fdE?DU?wNqVlce1DAnZrZc)lxA|GP1zMq=6Q~NTNjwv@aIOuBcsbh`$3zn<#BfVl zc#U}u!Wtq{LzhmFQ|ZDSe}u*4V`-BC2cCCAHp{g>~U13B>rH?%C~0W ztgy{?_CA0YEn_NIB@(QOZoT85K(O5wW)(#rh}x~hIAsBqpO85dIUl^n^J6g@gkY_S|h}r?rxet%&k^gAH%uYD1gLnUSw6OzW+-{lH17XLA zXM@~fhdi+{xOx{hm=SnlP)rp)lWpZN0+*u7U4gscV^Jl!O2Yqk_HZ-;FxI-wF6z=< z+Vjddk@Z0%{Zxv7dj;F-1$o@*(e`g24Wsw(hJj;K?Yosw_Wz&JH4s?LC#co|2K96p z@}-wK*E^H{a$*_6Jq_)-H(l zfH7joh&{c6J!aidIpUa~^cRk#SKYy{3YZ5?NQqz!G8;59nD@*UJB-wsY+3Bo$Zhn= z2R6)=lKccdmmf_KMWn}In<`W;00NEERTLx{9lMTo)Xe30vn?ziABfF|eRc`O@0uiT z^s+n5?e~luju3Gh$)yMB>;RJoTT8O|u8E94D=l*^$MqawX|d=4y-%|f)C+s(q8b5g zn*SVF%yo(zX2Z^-MD!jpEf7lQ0yU7m7XIjKqbbcOBg>B%W|aVjBXO z_CUGt0UcOl)kki(3o}iG8$I?8lb0V%1+8N?MA}!9C)eShvxvEr6FAumVEhC=wD8=A zdZ4>YHwv>Ec4D#7Y{D(a^xExqPNvNr9gOOa38;GLAHxI^W&43U2zyw^z$9*2xtur! zEa_b*bSWDceg$W|V}fb}Db%xs&wlOg>44wqTSi{DQcBWyw@+4S;x1~<%gO@O0zUH7 zwo&L57Fj+b3HbwAM$#cng$|rMI^8jihWV<8+(~n^iUv@FEZ4LVf#Fe{!-fr8JyZ4Fn=!PadU~w;bh1|`U5*= z$xDk#kgx*tl|om9|6>dgOYPM$Fgcwki2?Ph$=x2TLjE&G1lOYK1~Xnc{Mp~@@6#cJ z;fdPJY(E`w2*mAp$Z{qBOT0h&=&pcjI0mJM?cUF&lF67R4`$P^ss$_#;PBNUl2gQ% z8bdtsD^%cB5@PCw5%e&)?~85p^I2w{WxrhVx4PiNUG8Pp7`MW3^B4#e4IjQI6h!Y$ zoO!+G2eKTzP|0N+JH!6YE2Y6^HZQlodBa(tLjLVds_B(_Q>nVw^ucN}!vs?=CZ#Lt z3!B!PVLDrH@>@r)f{=|1Mf^?bv_t_cwGgd+q<3P~9GQ}&6>Buu1()c6LvHBt8YVA?VJ9#NxUHYX!)b8`S6ztChR->eA{_?9QWE<_jd9>s z+dVm!V-lN!@U8m%xFXsQ25|l(KdKWXkk=TKVJT2ZN)2+Kcu+lbdu9;}RWdg%MOvvx_r)RsCm#d zu}~c5WFA)?P;JWHk%zoLtO^lTSUq)tcyj(>l$G_nI?8j<&k?GC7wjDwd3(>W4o$a? z>xc4V?L+AA;j=uh(M(plP1n3%rFc#Ay91deq>G`lGEOUvNz~)eL56N~6P7}JhrT5g z-qMkDpWV#)8l!otDtb4wN@j3r^6H@Yt(h1ZaDbNfJcMdJ1}$F;c-l3w8{1`1wkYyO zp+B&o!-^0c&5TuDEMf2L1{5^D!9}Gkfz`Fza+SdB0pFsH*62MOu{|HZrZ%$W8tadx zL%+1uSaL@$W(PVQ{Db^BnNl2}_Q0ISQG7Uf3~x1F@ePRz5SzO(^-vW9b86j`P{pts zb6XqYXC$74kvulr@j=uwu90emex6Y+WKHy0=pcv0K1lmd9*8(Q$TL0($aCU*h#V!o zjEs9|ztoNcjV|bRr+1H z`a0PCe*jKwi&fF+hPv)%3fpMm307KkGl*L-8?EozDPRE|d5&pRnn+K!+1~ zYlw!~d3+SBX0|54@FzBgu+*+!O*$Q*F8XT(rT-sI?*i9U{r`_wX5^KU3Kd~0w{MXy zL`CFsDpOL_WoD?TGfmM*r;s8z&e6P(a+!dN$Yq~2B|=?ZCn7rv3QUBhT&92?<|Y#+ zY;26}Z0GWSo%#De9(FU_<(&8X^?tpcx8rov*bZWFqn-`PL1-rOf&uVcZ zy)tMN^0L$(RA>dJ(c_eg(49TV<8;sesCb7BqwvpNI}jk*?Q!;9*XeCiUgn3Wm$)$v z|B0&oS9*t+DIJgZT4X)bU22!@boZrdDeT`%yc%H!&)D0^nv!35HaYsfwFmzHUBx@> zm26B`PCQv+SHZDi!=GY9O!@V2wMuo9>EC_{>87?6T6~{J+lRv{?8Dw;*FwqK|CEJa z_nB%L6oMhjdopzcE{gRBp*4H2{h0+d=pGK1hfCiFGVYTsh;ao%V*bkX6xCK0p znRL-O7@j^7JJ*J^tl}}-LDtIE(8}}??7OfLgAi;uI~RD1e+c2C>~2zzJY*krldZ5A z!vjqeY_RQYT;NFj9r)Sub;z_Xc`e|}R;nX%Apl;pY)%nW$`Sl6Mde!0LPI)N$Ta+B z$7K=gX~J7z9OS^4{SPr z(mZ_v=E=Mb9RPPJiHHMwkbCUv;N9`Og{n+|-eA#n#?f4B2rN!fpGp%?s#1)a z8iM8wAYJ4_+I=y7R4q8o8^K=7W}_8)n`xtl;A(tW4cA86urcJ9&KCA1BY}3b7Lm(% zf@yI1U58rg?2D#yu8G~^qM_O0xCb1UoLI7^=OE$(h)@{^bC)AkCVc|klTL4F<*JJV zS!6H2#-0WAT*3YK5RqL?aH2EzJU!$TRR>oi7ieC{>*RP2vki;TN&| zRUp%8SFtzHAWF+!fLqLUG?0qvWt(2}l9&_S)Tn^>;o!taC@iv@0y}wrJ=5^iUA{_7-7B0uT7GpY9tpKfqBE~9me7B z>zRkw`$k~Y}zwpu3rsVSe6CcqfUyCA>%$xFOL^Ut!&&?0i=P&e`H@)7O z)7;=Ml|_a5MsX9=eB;%a zs2|w9N#91zvlQiG3$m3UmX*esLmWn(V35$CjM$LM6yJ^r^A{~{s^3fTv0Z)B+{Zs{ zh|cwLTeP%(iFu4}AN5wne8wxx5Bqw1p8nKViQlX{**Qh@Ww!FeuqnKlrAp6ZmAXRZ z1k$D}qGA)jLZxeR<4bZt)RppKV&3id!X}Giw&%D{N!MBGx0z-2ACfBFSkgCfamn@o zhrs8nrOIiE`M2K=^A!o2l+L_S=3rfJM0V$#mF7db0CIKWmx_$esmS87N)P2WbBFGa zh~|fb(%bK+v}Ws0)Hlxa6D=59@63zSh15?WC&EQ_OHs`8C!(j-t#gv7$z zQ^KYg=Qk-!|3reIQPQc8T{7wgPQU7Au27b0CL zJw^ni$1f_m@@ZIrDyFHP@C|Lsz5QXBpYfk4@_nnaG%vGCvQZV$QNv-==YB(cjoPbfpN6E>#T=a=gH^2}v;GE&?y?8QGxiO7;R z=hJ`GAwAh8tC<)FPHTs=Efzbbs z1xU(UE80fgji?|Sx+gVBn!hBA5Mnn9vP<^YB|lY`cNWIby^-jl#yP8y?M-hKKv zXH0M77xEVV61o{+iiiq_r7^O|vtaLIt2Vv>=>={NYb88V*V*@ShqYuCHbnoo;S9(6 zT{7hJ-)GNf6QgRF{>{{T7G16AXv$&BV$0%G>Wt^P=ff?g)b@4US-DuZ6hi!X;{AWLqlGumcy~ehRwqOgZcujRwRks&gYpCXN+>*11fG?tFqY~yi? zC#SX1aOBT4#IQsoYG&}Qy@l3|Y{{ZQzJ_-X6hio779UC(YmdK910@g<;ccYjKLZRe!l>}dz|Nc*r=P{2~JO0-{e z6QgAjzhEL!jils4;8l2Ta68_H>)7h$ zBe6)hQnb4U<6Ka^^h5Dy8XgeyAYx}D;0WdJ#(}a!EO2QUl?5{0gY4JP_55RNo@>hZ z{2sZ9T9qkFIdEuC8tRi$m1a1kv(|*}p?ApPUvi~w;;Ns4o3{KpDy;@Rn|q9sApY*>1i!b2(VAS03D zM!C;{^PHWa-{=P70~@JvzvFtvS;cGZ9@pus?oO?1|B;njax_KLXY`N>jZ>^AS05iz z{Zxy;u=R8#jdHb+!eH99f!j02>K|n+y+vzoe{bB{o6zO}275yG2Qaq-EV4p|%NLAf zbFztV3CFe?!S(SV?R!Af9RxDZt0x>gJ!(fJta>2})g$$T&;O752U-BoqXUcYQzNP6 ze==yrt3GhX)`_MNwD2A5Em#dn-~dQ^)7Y(E37%5mf{6Y2d-z5$GmA9X0hbYLA!u== z)i~JNfj^1{`3Q@pF~unZnh4_y!UXz2R58b?Y7stdQHvX5O}Qay*M&=q@E`a3%N3po|2X0-X^n>Oa#4_m|^mW4Lw7S&~pRud14qM;60 zJ=Y}8;a~r&MfKWwH)By%$KNp1raXrsw=%=gbf9}fH3IF}c1R0p0As1#)rJ`(RxY)J zQH|47b~TNR_!G^rY#B~iXRcnzZE%}vo&{aK<+&gxR*#MYY=r&T7=JQhBt9GZ7>{Q5 z-XJpe5HS{7Lf&|xWgYP%1-6iA$Uj|0iGU!XE=?| z-ZV|VYiGQRzXudRb!Ple-M*8iL68Or3d&v>r+rg60#9!v9+J0jgO&x>i$n_Cvi!ye zE-^uD2>7Q0+Xv>+2MXegt_Ba8UIBUe=di!0>rGGZY zd=x$cKi~*2u(Y(W?auIxHxUk6Xthl{7(NP{0T&~Fwo~+lU^|0ext5|>=kR*Ydp5s! zTGePhDA4HU{k%gi>;8~kIh`X2++7qcItf)jWhOBjK&YE<3Vo5^Gz35v!{K*yfH<4z zY*jO#fFN=jSQ^>I0R7S*QolAN(n&z&`kTU=h615a59DC(Fz$s$gK>@%8rQwhHw}Tl zo%2eCjAK9=$xR;Yd&mxmXb3dLAyffTnG&ptv4V_7B zqQg3*)%-WBf1-VDyfX=o^CZwc8;^1?^kl7l9_$-yA)vU0mEmeWr832V&>8bYJDxJ{ zGoMoR;NK)_jdMDrcLY}_8Pq_Hd)_n7?U3G;9dxOi0Ze_$j6;RM@`u_!U_N^DEro|& z^66sx#mFLqC+PleGy{%$kN|7vVfHN-aX2^)Y>xC=d&)p1=ZYK$K!iVtvCcJ%`ag9g z3cyvm2$p`3UAuwoIvIE$;~cQoe^iWd=JD_J1YG0LSHNeFW7cuM*BN6k-No1vn-D4i z_-%X)n*-A}uusm&ZlWg;COYmO!HNPBNu$22Dn>`(-dwiC-ccAtvb0;=kxyK?Bl(a! z0{+W@Voa+7s>VYRN~K)KPf^KvNJvWC%XG_`0NGFG%>l;E=0^dNP;yZow$Z#i0N-wQ z3lLnd-m(4k?Z1^l1-BK-@Ke;hJot$H2i-p5(bV4#n&P2y zs6+eYQgwrd-^%bbajZYpXn*-3WXO0cX&10l3oKs0TlTmpK(2-O#{_SpPaXUpKr}Gn zyNvJugE`_r;PLFALDv2gyB-F(h?`hqn%;p>Lo#@{h);`zx0t09hL&@z@sP2sTF5ygKoh+m=W;5BL+cBq1i}YVO871| z(|S=l9fTjxE(7nrR7<&XML>?x2iU0LsY&z-wyL)s(lEmyTH|o*Djb9vydy3pO1Mk? zBI6vC6R2z#(+49ngvVgxT;VaJfF3306w}ZM_za`bUl{$OnUDA8pVb1ve&W6Lv z;QF2ApmL<)_o!d8J<#@9H_>bh84L+3mjRB-zXv=&N2oeSYhqn~li!KAT&foDf$(^H zH*yH%3XwH(Gz4fZFxeHejRV!v|5=ddofv1Hd>iq>#~Twc?cx$4M?!F6B_r8UK)2*~ zTk6k|5sA6LR6zNMfEnaG7c2n6muL4vp*I?xuh0>9FXAb>mt*zQsQ^k7t&|aO)70V zR|YC)s8{f2X+*CjE~UT>tRF@`X2s&9O$M_ zp6h86b9QSU^$o|Vl=Hk`e~v^Ty^H$CxO^J_BaQ-1BUd*wjv#=pS3yFY9Wq1Ni?zCG{fD^w|TwxFxs?16P>{FeTF!qSF~m-brL!&K_faI+z3f!rf@*q z(n{_a_e&>W)3gauUCt}K$ETf|Q$}~a-I<5dw{}iYNvCLcKHaf*;^qkMSN_$KR=iWr zYd5`(Iui->kFNatlIh8p;vw>Z3&fJ8L$fs+6bxJ(kIVKl5Te;wj7v%CEv61KHaJHa6LtqOSfj-J_a$F38)EU_8qby9qPtWmf&{0=$T{7_&GX)ls=vQ&0AYH6D+nr)s zG^dFD7_BkZeb4@}tW$PssB>$rwn?-HX>UU|wd{yWXbBZ#s#)KQns%L%BpfzizciiY zCdpBQwZ5vH_ZEo|$phv{1d>I-!s>r`dA z5(N6iekM^@V5p;Q<49tq#E>@-8_KNrnS^(z)@XNSD?KlU_6!95?Ri0!daW-MG~LD; z)?dwOING$jb|e+L=~l;MM*rG&_nDt{-Cv5$7lovwtFan-d0<$JN-tL^?yW5WGlWE_iZtt^FyS?`zp-;Vy_DtZ>wOfcbTPN{xdK&k*0sA%sm= zi4piDI05y+U~0VVvhce1f5JIeTZPS;((Y-Y$BY7{UOH$i&3w9k56wK)9r3D9MV~8t zc1ccjXk^Ep)6(1&}*c9%e4NJgu)D_k%NQkh1* z$z^l1jlck~ovU9Zi=t{)6FsH2QS4|~30!W4s!P~PLZNi7#r4b^E35NU%CjBS8sh<4 zxQT6kvSSZk^pp9J*V0sWChu5kM%H=SeY6^T2$O4GIbQurbG!Z3LN)*IWIjj1_tMq< z2dWW}we<v6H!YAE*AO`!++M`zSUF-c>r=3d3owtm4s*lK8VQL+O{5CHE|dAR>Ch zK^L50YK~3=@%JgtLHlO1n-0_Q|5NxyO{X@(qAlTtQAt3fmoth*r;<7 z80soykR?(FM@0E0-u=94)#YF&Zz=m}e{>7>pa+>o1@t1@Y6HHaCq~6_JgSV?tzYWVKwoWHuRC4 zy#=sH9Q+8~lu^wtt255mlWqtXAJt0o6n<3YT$E3?VJqK9v;p=*MkaXADvs)alO{qi zniDgu1A!Dym*i#2PV@U$sU%pqlMxcdYCopKXAL8kS6bA-@)J1 z$;Qw_((TKKh(ZNa=GXT9&7!+12}eolUw&`D&%sLIh7sP8n=(SvoHFvH$*b31_0o&| zgxcNC%EeY+d!*^CTuciz;ha!zAM#b5>M7HWgEzO%C~UIunC~HQD7i~?$4E7(+364& z1_A?dWbZamEV!bIco0?ODu$kliV1yKhn)RAnaleEGR7SbKpX5UWa;zIajt3zL>LCD z(>n0Gxo6qHs0S>S)uWpI2obAdk#zh+IOw=J`Nke4J;Sd9!VhMC5dV!I>LWZe zL2t`>pz=8pbt4g7t-2LvvE_(p2=9BLe`BX4Pg#Imen!XLF&+|%;jjasK)@XUTdraY zuo~xqA?D?o#&nA+R60n?CaA9)z z4K^v>$<|nxcd$HyPBtAf#h0;%j1Ki-{3Dz`#6?`;_XJr;I{;NX=w}_XW%6yoMe3&; z9vftTJ@<{!5n(4=G>VI~E>-Y)+ihz7I7T+NBnfo-YHe+2HGYxlRM60P$g`!8 znBB*08{K)TVbUk{0c_$O#TU{Y&chF9r9;as}*_i7&!xb?U!z}IAs^_bB3|24vovF|BOwG zv(Iv&73dHDUISGt$KM*4yw@-iSwL?p*1BVdvq4#0;+6!e^rU#YE%_i@oXsYS(RUzt z=mi7@_UzsO9q&-`Ar`pqTOjV)2xFNDq;+q@3WtU`V<5bisI{LKbsTXjLv*P_b5exn zblvonq2OF(3O{CSUCi^N;`b`%H7WJS^x@0RR^1z6lp-2}6h44P>7la3m#J~@8OTTd>`!4tlnnrUHtc$YihcF00WyPGUU(uyJM+0A#iMtl{%#pLt5$N%F{PHBmTXIA-T-E#3J zt6+xB?ZL)*DT>Qsv8p6@y|w2oi1RsnZG75Xm{4#I-P;e~ngq7XU6eSGq}8S_f9mZ81vK0oj@x z^e*!#t-2WfGU^U^st-`o0k5TqA07_M!9tzjj%Y6Z2Q>@&WKj+;i%4_?lXB?$1iy%1 z@{=dA#ya-KMwB+<%S+s!ma+8zme9oR4Uq*y-<*%E>MWMT4N#loyR|>2Ai)4_Wl}!? z?VeBzXY;LDYB&3XUbwM2QXbOTE-#7GrEBGx!oif1_SqMr;WCVpvcCqB7XjAlvL*W)mw&nv;R%PVCRYYCzHOW6EU)Aa~+piIG_2|8|;9V(k09cQ2S&kgcJJx$iiLB z>z^+P7Srqc``vUN)Sf`l&6~Zcd-h>;WX^B#WYxu=_YoaeoJ#lbLR;YOehgVy#_OxL z?C9Wyl*|e#Ik9$534Bi8rnZ`8!^8^Z4fgR`v-sKcVQYAO0oG556XeZ889mUS%R3@l zvl|(0TOX+nDko-wrpDV`sZGvA{UmKowtf78;g|UtV#py5(*&aMviUczAxy3!|kh7);deF6wH25~}xC1LZW(D2fQifq{h^bz^#oVDf# zZBYJNYzjGxa~%y_)$3o4Zj!-9waFe7I13)v*MUiHJLDXuu?8}U+|?Y9r*$BN6%9{cuyG)T%sc?P@mSsuZl6^yLR+F7~!?6@9M-)|KuZ zC6>v3Y~!e1({9eVy+Ps`!F!|jES-`0ks zQgPMrp;=NXk-U#k-^z5_(k8GC4_W6XKqb3m?qgfa{SkLX;!~(pF|Z3=Z3jJ330j>syXkwn&>Be= zRyDAfGgx)b#6!2*3pcYC`Ob?i;aiYPdf6+-OU*+2#Il zY9tbNPpagr#8RE~9<1RF-6DQ6UFpaE=}0ECu|0Afb}zQyvV?xpBI^ty%ZB2qe(Oc{ zAXNlWTiimFJ*%@uz=vZ3b+N4u8`&n@f$Lb;Rky}i-@a5gaH3WEFcUZ5bv9UX9cySD z-`mNMR!gB~#MoC@WUG&kq28*sd_`DOzGNZ*j#zQk=bz1yGks11@&p?!N;;YS03G~S zMhBDX0d0Dk<QZu5W- zHk7WD*Fj%C%bC*pTHp96m*5F2UQ*6#CE(zhb316mp#jQ4Fb-g0v6RzP0?@Jn@fa3+xAP_A~P1f%01ZE$LWk z9#sq5!3pQf$4F{ZC_By^{UQ*4xMa^db7>iSy8xLlwWTdI2XpU%Ny5@rnQdJ``Oom` zSU>h)Hh9UIbnhX|8m=Il{n(4JVWV~If8dIX)5Xs{<$Sx8mj}q`ZpSur`3uM{{S8^> z>!SbN{uKBik;hv}S4iY3^U?K2WHEVfjbZ-(G52fo;RP)Sg4Drro#ZCWPmQs@3 zVn^_+kN>7vcAw_a+4pWf|5@A?c3iuyw)~a&*<=WV)J`MBW0b}}fU?|7ogVNtr*pa2 z)6DT2d6Bh4C#n{O8v=Cz!xw3#IoNO{gn3bGeEFQTFn^Ndm7bqco@@SnyIa-qy|J?9 z6e5De(?CR{StA;czY;IQqRq|aY_<&j5*_b?u=c|n1Zkp|B2(b4H_*S+SIn1mwtKPD z!_gBqKkS0T7$)T%L*hC_lqE|hZq!^Mvge*Lk^oH)%Oz$!1e{E#D#j(*CDT);z%eS0T*< zyl)2j2z?U3eg-IisSc?YAcNj`CY20_Yetk{;$OL}H>|&bnEF`B&&>HI(wU>v*Thb* zl(s~jq#pNTuRGl5nG>|>JZG(J2@2kiI>kUaHkC{pIMI+T%%Fq=aZOF*P zf;2JmjFv5vvBBiK2-BiwAH?TkUc!oB*=S?? zJlLeh3W2uVW+-&*FO#NCBjGq`Xp>>SWY5$gR%AvW=?$L|3(^1M_N^Nc1IqbL*3i&C z3He^Cs=@%Ftey3fx^*>eT!bmCJ-3u&1|)V(UePbrPh_Ts#W*^ zoSSvA)8p_ZkiZMN=BrF%NxJ8*v3Y*px_ z@H+%TFX^3@GD$BZ%+}@0p>wtvg>mM ziq%Gm7wh!e9Z4mwDG9>*u*jKZShcV~a6WSXRslcR=4;lwDoy0Fg73_+nxOn*_gUs- zjrz@D^fe^M|NJ!}-m=t>44#X6klh;>GrrWGsY2(0BmV3#D09XMLQ;Y|x|>_-3Z3Oz zw>|jAab;NJz$D4ShvFH|7=E(4?yp?Xl~%lFT25zLx$pdlyf~vmM^r=g;<9UxQ!A<8 z5{1t4<%^hPSQi;X>WSo6fnP97@ulb&Xxbt|JXYy`-@SD3qW%x+iaqBlS_odNZ2-Kv zXG(K`)d^v~bXwL0+S{6c3e=GYJsd*8iLbuLZg~_)4v6cnE<)E)y_1R5Kk8*51jgdD zL87!XqKKi$VAZ1|!^FIOj@G=D!t;^yZCGE~cz5^+L7UTt64lbvWEs=aM9zm>{t(uq zq6RG;Iex>(V0TEw}}ivL(#;J{-mv=@IDfD%?(A~u=J90NVmAo)Jwf_Ii;JOpYbeQw z1DW>F8m6;v(V1MZ zyjk`lx`~n3W4{xLZK6D0Isw7L=DZRm1#GcD@-D39`xo$|tFWFIYh-0OB*vnuFX%iS z`xj&*SkDi5)ynuv%UiIzuvv;k?P6x#qg6Ck(#kyV-yPqbVheG5;(jv6fvJZ+*H*G+ z{08gtQP|Vl(jUwyKUu}s=|Pjk<4*pxFr~ZN(Li=D{}NpSebs!GM|(pIngFldfcnJ$ z&_I>^sgcIPu(Zth_tl$GtDh%RxgYznvDR2qU!CTCC4`b(U_RR%WlI#XvN=HnaW}pY z_e{Al$c}J2{|#DLm7v;8#m-R2ciXPcv)Fe1@NmhUp z%R(^x?9Riwbhy*4p<3ZAEw5x6=?^PJR%rR^S&{o{{bimu$abB`#Q4d2g3J{{hbLnH z>E;GYOuY~tC4Q-Jz|L?|ngFFS8{RfHQ>b+AGS~Hj8R3yr@*$aF^3@VyyPn4?$VnC0 zXnDS~&e_#HM6=KDCg^px16I{6B5zBA`Z>!>|n7(F9}nM(J&knu2NJ6ZKS>fJ$!SY;yh$~(@w&n>x#OdQAYPs2~ ztMpBB)1KAlju%|}dW=%cMFq|_YymTet71-}JyXYTaj53lWbyu>Fuvlr>V@a;RU0Gq zdbQz~EcWxP|Qx`DM8sunUkxq=rM0P-}dD2dL&+yG4ZlvIKJ zJ`kK#JVZsGyTPUq*7*w+>lZoY)QQ2Sw$Y$9PZQ=pA!(QmY4hWM~OATVQHe>RVd z<&tfFqdoF4feUEDn&F8fBgiS9m$A9@STcpfdHul>?l>%BRy;pW&2V)%#Sn5~4TTiU zdQm3GM&2X6O)k&|fp(+^n$L&x6Or~8=-{t6FUw3>JLvz@)OGdtRP)#QsS<+X8!YXi z{9Gb=ggq@^)}@Rd!M>f`)qCIUtCLhgK=+9~OOmFA$&ZIX_0K^( z@Rt{)$ME-%oo&Wr*t@uP65AV?i;aT*oSsNZO$kzaZ@SKHtP^U`>wNMHN{Kb;d^n&X!OR>iqwo3}8fnEl!lrVc|g+y7aF2 zrU4liKx(*cD`H~> zQ+aa0%PN;P&dYG>#@^a@LA4++L?H8o7dK$4QGt&^7X1q29_>wpHn7R zS>ZudW~(n(YAy)@AgF`NC>@Vb3(ci)G61dEV4px{KxOo#=|8(XmBeALCD!*Pc+_W` z{>|ofs+r$aS8oL7QeNDQmF9DOZkZ0b4{h9R4g_%M>b9n7q z+R*|PbHEqjgjL+u$SU*0Kj1K-Sz6D60mEWn@+Ew$CbsrkF0ag5+USfPkLTCP(a@Vy zRE5NUEc5Ko=h5xxayAjh6Tr8N%2=b9gFsHNC`*2kRm7H)jzfCP9IB~XK~S=XlgYhs zPJ?Y|U4N6%Eb4MwicDp_;ps3Q<^$U|uyW?zTIpnC?B-nRwySn7M&AVx{y>309tO=8 z;*Y%#iKk~*`C+p;s6!btY?a|Ida^*I^M^@z98KsEqP`NpDtcEY-c9ptj{n zhc-9ynxJ0)#I4v(?>j?bg*#Tj*%zB`uSF&k5L0UP(y8j8@I>p3wUQ!hoQ}9B8Uw-J z=@j3!48?t)wBc9LZ9ghy0BJF-mD!peXTEKlHcVV1O=bT=pSAEEiuH5Sb>;FasD20B z$x)inEa5fwoGupdQ#d`I6)iUx{z8XY0As&rw?cdE3EyQeoTYW}ARomnRcU8CJCIQp zs@+z^E~d{|)l(ouFMhh73Ixe%?*-w-g=iYLNE3AP07&6<1nx@SAH2B+u<{%xeuEwp zNYQN+YjB8&llt`S28{7*iKBNn31<-l4fUcdX;__X#&JX7o02KGD|5HDy4MSj@J3s` z@jqZhhk}~{s|c$5SW;#RpT~+U@xSjz6_d8xZ|-I-f#;qdPyUx#LT~w93^(!;p5liD za`yu;qLNq+4lc*b2TrJO&5II5+UJrH)|00H3=UR&-CMf0zuxk8@@SLz7j&zNcTG?s zLQf4nCCdNdPbOFO|BUjF*|>;x+4Oqm4Zu8WDx@pu21CpkClurw?AAkmzzAbg2oh3B z4-u>CV-a8M`VC?gUiF2On=LsJVXS8kmDkSBe`w7LW2!omjI$Ce|-GlekG)4~PjW&jMIArl)GqCdesL2pIq-M6vk6lAt4dZR@=bM-QF zAq+jjU~ABM^jfaYeFf&9l(uRI*0FBBIUlYzac+CzQ+pphLw~%hH8$(VP4iOyAVJo0nXx>WtP8C7XdC0kGAVc*TK_`Tq#|#3l>E+ zGm^o^x(B_lzp!a!{T=Qe=4L=F-Ivh+wa6|it=#V0Pp18Qhl*NxvqYB397JHLU7InZ$rxFfe(i4wU8YWiwJ1ro!o*v3kW)k!*}Rx|-zv4;ni!%0pB zqmSs7H_CJ@$0=^(=m2WARxC!e@`A1G@$>ZZZu;}%Tt9N94tchYTcEa9zN4zC*bjdj zY5)hSdcxyL!Z))$r|7^Z;3I<;cr8B090<)0b|j2>pnt|!xqb42ymP{|!ydKA-#&lO z68n-p8|^WB66{h_lj)wj$$5Sa($cm5E6q!v@vJed!;q2XQ*UW=AH=E(`aabTki#Cz zA{&|6J5lj#{ikOF48s8Nnefp1kGae39P9j#ohW|)_T))-8=~s%pFd%(tzZ~&hPRtI zd8*>5`^lm0whs)J^?GLaA3FDH0O1>{D;PUv&JX5NSOdJm0ZF61GzS(-`Gu_T1sB|=~16W9u; zf%a3?vyeIjm`la({kqb<2L~$efJ~e`WQqfbGwpQORHY1Nc=Ttu+BQcn0Nh7hrAW5s zbD%~}v;tK$5z6)FXzFF7eZ}+~>dU%?#>P(5 zDGGgf1^!xY`r~a>&@Ga@tc%y)6Eo8$bdh8G_98zi3Wq-F^UR8=@BvJN{mLRfo4>b1 zgsCihn;GQ2E!eDX^wrL`nAhRY$CzWap_%yMffG`6O}0A!tOM~8MQ%sklYc9zc{J>{ zbSZuLYyoyY5<;4X5|f#=%A={jlgZ~q?3hcr)UVv~ekS}GsL}X0maARYb!x!c@mH2R z*A0F+H^VS>ItfSV4LU8YUQbxqhC|SWETx9VD6A_hmzf2>u%d@fu7X9w*(2FL*s7|> z&ecJ2+X;|>3(=!D?YOEdP01zqCOv1{?NyKjnI5&3@sR`UIH}9BSiXZE{K!@+w`aq# zH4Y;5h}$d7L9pigcpRBKABO*vu6t!XCW5DsSVPQlFXDlXV5*jw!vWleGthQV)Y)Cg zjv#9W@;by8e0QZz@OX$Qu?4{vd3lLA33gL?u|Z0`v@h#WvzgSzRCpX`!BG5px~KRJ zYDblI-e&+5y1`B%^Wp1wu4QK{@pA^_bb`uIVt??6m*LF7vt0ITy3&HyylF$EJp+{; zp-k)EdZdvwmfU795A015s}D3s73;qb|5KA^^|30U7yPb*A(fvKZ`a(=6iB zO;Pw`81n;vaY~0;MY)oJLV>ee^m2ldfZ`pB4ktw0Blu+WHCm;oKW((U|57hJX3Llh zLH6@Scb+bxF@^e)WN^5}rWTIxO3`OLE;+DHm#Vohb}rxuWNkV2A|oYAfB)_0vebpW zjpR6s;hfny&t|_pm3#?38u6dq>*TL&0RLr5N4`UZo%OXU{^H8;37|rdK6_hyTBfcv zt-x?7crzAataIz8?cJ5RQg0}i4_}K4o)XnRm@OJu&w|YqTsBtXhx>8uhdN^dnFSI} z0*1kh!jm>(!YCz)68SGFJWgq0ck=b*wS1?P*C#m_8KS7Rw$v#Yf1PJFAP^kx(&yoV z>cjEVy4ZW7iNM?Z3?hirfJVf1(}O2TDt_jVW&nY0jUujx&_<&|Z~QX2uaCi0ryX2w zj(t(O6!De5f=qx{PBKeW!Q;-OR}SB)?4Q{3NY;a@aE2y z71oNxIx0=Rlr@B$iDu*Fi9R*#RH&6NH61f zMkxDW39;F8h2x;*+aD!VJQ^QXa=@LR9FaUJPhV9#+C*;Xm7I+{vQ?!4l}Cia;ensl zZ(Y}iqhBoTU2FPGC8{Ma#m+N_f|wk7ad@ch;6}4dQwP);#Dj03&d~o135Gbtj6UCG z>CY1SBXq*+Wc+01I^p1~3ZK9EpILV5611%NbTJQ0X}pa%+uu@G;vrGj1gkpKGqoMS zY&%3smRph=no5s^Np}a7TxwW;y9s^U=FQ_SrMfWhTl5Q90`!c@Adg=Upbf}$;hZi7 z2@G#V;0fxM3rx!tHf9RbK2wUyY3XOi*oSCG!&>YYrLmN0NkP6a_x4!u^0)K(4#Vxc;-V;`BxiF~Hq=et4Q99K zl(ACr2&3B{esA>Zg>JEQkpdOQ^sjYAufaoErjH<9Z z!^Y!_PEMS z(}i5&MCd%JgvJ1J4D1kBKe}VL(wN1e+jl4BB5!m58yfoskbXL!i2t_xIG`_p&j{z_ z>j-%G?xv>n;p!mecL3-+<~5S{xN^#Tg+7o!MKX!mz=~b0D8RTs01uVl2PnudwQ$+% zMC+80^aCZ@+w7K*rZsM%*7*>OI<-U0bZ>jgmi+4^#NUe)$)|8jm@K+p_PZ1mCZsS4 zRff8T!Yyw@P6o_Stv}@=V>ty@9?sFw*5NK80rXXq;VBXzRME?hffQNuyFx2m^{rV7 zbP+%(NRNG*&VYTbaD)!=pxI_G9(sznCg{cJaOz2qTq(*V);$SDv*!EZ92hZ>0M`Mp zfVjn7%ihe|*dj$1i1CLuY}9p!+)(=m%6o_Uj1JqzVajMUZs%5F=Cajfa~(Nx}dNU+tHTbFv-IX{*I0QocE zr)2ci+xhwm!mEq?)Ve)ILRKCZ(eXMdr_b87Rmo8iZL(jOv^E*^H_~Bv_n<)+-KEYF z4%xz94|F)_2&02}%Z9Z$_dD)BR#1OwO3HJz&Z_yCY6G!WkA?{ju4ZBg#lejcu<))3L0IiJ&f;x~RsEgpl zLuDTI$j?Ton5Ax?;1CEbdU&;};)C7>kB5R(SS#a*M3M@n;#x;>Z}r z9acOS<{GFgb^ZYfg=cIbGXRB&Uy9$jt;>hg3b+G- zD+#X4(a=0H@ES8xK`!^2LaMFytR4NV4!;AHuK-TRM740487arl$mpDIq#N`_oG+|< zQ7$i2)|pH_v~=7`LA~5^c^p54CVZ@9L5B32T3By{hhsx^K4We5!zR6<lYBCcxO(B=K7vBI*}y#yIDmc2 zB~ldDaUw;^olDAnML9E_?ywewlRzyyfqV}v+d*E7&`|CzPKXZ*cfsW6sfEd}n0f|s zGTt$41T3pDW~nxmTO?;xtH55kPytv1+@cR%YC>WXSO+(nm$>$&^dL}JBgk?t&~^$k z(RKD=+|646#;1Y-H;0(SQp76v&D2y?Qd;mGeciL6ulg9BXJB9T9dv=9$M8E^AE#RfWz&V+#4r#~u&;^Sh_ZC3g25=@>$jfElz z%_B(Rx#m$-!)*&@vuj)MY9yfgWJECW3+mZ};88m+-u1at-Yoky2s+9WF+9v%4y_az z{7=Xy1reo7wx2XM_s{-OA-NK#G@gCU&Fk6TU6xJP;;+aGgiXvLAm7{p{K03-^T2@v zQ_&KRf>KRUM{~166yc9^&jocE*vwOa(&nBppU@G5A{=x+jP$TVE3pZ>^>7{k4jc;G zN<{D!nyf8>N})_E&$Bp=Q+@V}#gOzlh5Z8>z4?S7W_OND+1F=0#(r%G2b{t<7b`*+ zpoc|s@V*y?c-C5fgaf`vwI4nnO^PKP8A@3O&kOsXD*rW;^*VjYP|dO(@3K}eQt{hW zo$Z`}t)+Ynp|ml7TYIeYq+yDF&#rntM4q}_hmFtt@F-}$xWK`9U2?T*x24bXQGbL0 zG$WPkJj~PJQJAqqT_~XYqN-EY?W)k$NPmn&yq0hkmJUP~fN_j_ZfQ-G9~;T-qB>nE zm2JI}FXEcBJAxteMRlflAH*uh+D=kG!u_M?lEVfJz~rX>vxSyg*Ou_^r#$UWoI%H8S~F^iz@8SdeqIwL<7HZCqLlF z*7yZzXdm0tL}G_^?sqvlZO=wy_loOR^zCDFnV0!s{4T zrB9iDjGETYjw;d%%qjy_0SLiCpo5gdfgqRyCPhd{)s-?iq~SI8^6N{OOWTv{ZsNWU zL^YpomgPb9=pP&gV>f`99IfPPKrawe*-%*Z84!3%31H$s`zgm|JF$<+or4fr?ex;= z?})}@|IeH@ zWXELWBYbCu#hxSl*!5?_Uxa2!;c2ue{Udk8tVw0^@ONAV+?|Uhdqu6N%rcCCmoXrd z*@7DcKzo5yfi}5N#N5-x-{-&w^h@V%CEv9Gn1WnTy;nU;re>IdC0dFli&XI~v^?H)@jR2gO-N83A&;HIe5@VeHsv4|^O_hD+^dS#f zH?+rH6V~d4HIfbmS#pg11U@WBE?f`yLwaNv)Zt?E8+1B+_d_aPJv@8%nZ|%wmvWH$ zn%#=FdeNWs|3}ozvm~=L(}BbiD=HN#S605kT#1TM zso^`7D_6d-t#7$zie@QI6cy0Slr=L*P2hvc@(qy5JAD2R?)=XGoSoU}n0$oix$o<~ z?(4emG{ePNTZvEsge<$j>fvhxDK|L_IADs}SC;e;N*d_r&`iSr8biN`MT&~K$+Z0x zPjJ;?rb1k6imnpslP=FiwhAALS-sCJKwAJN;QW*E@M4fQ6NW7288%*Lc^h(*F|Yl! ze{MUESOD4hJjL1(>mJi~0zI_V&qH9&Qy-Hp_}X{g`eWUSQ1Kk@faMg3S$l-h)5i4C z%hr3O?j6d$DR^Q=7$)M!stuK9AVf1#b=SvSK-JGVJ%ryjjX>Tu-JQTp1zT=~Z{XO2 z%Vni`Xk&nZrt8KdmA{Yq0Uni_VPyz3Od}wuoHz(8Af1HEHJFKL>VJ3%SqgZJKp>uh zpZyNYO~HaPb2OMXV-DFe(FfZKBl|VgV`bLC5AV12Fo7ic4f6)De~*J()&kiAvjF(l z*1TQ{pclaFOE8p|&6e!1W`oYdPI7&rDGWr2`KCPDS6}wjB1@;{sBU3&VKXU1g}vsN z2W5%SlzY~_#_D=enX_QsIr%b@O?gJ_s7|o_gnb5(o>017Jk;1^~ec z$bq~8j)flpaEwnsFpK|3MSYQO>VY4DnU1z&)fLx)8X#=8(Q4VX-tmnbt6ye*f==~D zPE!+nUThI>0&^6=$Eg+Eir+xBmZ_HTu}D6>N;QYsNWEVo_HVNURow!HUaQ8plVz`W4U@{dcg8?c~a zav11i1GNdz!@kwMbpz89*qil}^F`oppuHYDB-R}f-!vV}FpbX60oO=~=}mN-C4P+a zN|o6wgq2A>@HK1wnlzie>5PRlU4zdI2U=D@s$$l2&;VM&lVbm4DSw;POwQN9Z^3k! z3;m7KZLL;XM?xrLRBJ?^1AX?>UyO^z6>`7XR0(j`WCPlCnKg4S2F{S?)%f)Co-zH| zm+HQ-O}!W}W>Gjvy(wf?zSNr3mY&&e%$-ND44+vx3HRLLD0V8~o|G#<=yBat1#C)m zARz#F{b#a0sEh$<1}b94O^z)<=0J!He`bxedo8in5^f#b3Qj5VCLBxWfPaR#dlj6` zxY;mJoiVN>uUYXZ@W2CMI)lUZ)6ZG6Y06Ep12DKnkCj`ew``(*0nX`s_X~3!Ofv=- zsS?lx7iBJ^oKor$IRo^)Ee9*9fWS8$E+?Nk&?Nu`pDnoVexRf71(=t!-6^O2DNb^; z#Hj}?;{oKuve`*%e%da^B;HjZH!=28+@|4pVB!!*1Kf56jG270`Ms%%K;iuG!lXAx z;8J`&nHYP<_847kDSTTPg@c3G`kS3m%U;N7OK%E?*=_CA^gj-~@AS&WUY2?FW{v*X zx=TM<+_!icp;h&)NI=U*iODl)@pae{BsEWc&jW}o12Vcssr6dpyXjVY)J1S)tH+wH zp3g$nE$y8GJ)gXT@`s81=XGK9!Jf&quiv-evM&5#GNq^gVUb6^w?D5q8431Lt_f{KerU^T^~(H)mHUFJ(37 zbe%Bc*(Cg_-uxPVg~e7%hF&K-Fyq`OKuy<08 zRWs?5x#sjNFvGZw-8S{btl`EHv0m}|g2hBkehRC@3oZSjYqil2;9GXHX2ttDOo_hQ z!eVo7xpHtGcq8^_U$G_N!6e7*I=Az+pbPjw=Dd5TIo8}#3Py0e`w;Xza(jqc{2{1E zkRyK5wu555T|h!+vX(h!^Dkqd7+q{Fr?oBA+olU3{%8fx`ejyRWgSgtu^0F}vG}C} z3bP6DenzO8qnill{9v9p;g~^R`{6X)hXtC}A58rzZH8AI7UeGu+?!_dpN*8p&qGI! zrDaSkw7^x`_@K7gvVm023yZ;z_4twUwf~rQc*oi6M&Y}XEotzd&zI`W2|ca`F=x#a z?XRxKU{0_Q19}wYR$rPISW^1`MVf-=rc_Wt13g+b-W((-2ayE;lY{@oh%Nuc2u~0r z3PFre!(*_~EScCkYnH^G{Yj7f0;XT~>9u3*h@*tWJWR)NQum7IdgA=Yh28%C8cyhG zLu1F+G5FwaZ%m`c^H$cp`4&403s6uJKTF742rSy|@r*EVR( zy^w=~mu4=q&9Z1r*a6VxJ^-h!%#OC3ZY^QI0%f}Qm|s^UAQ-@8y4N2jgCs!q{hNNy z@RE&xX)RwW=}({rGt386Ucr-Xt6xSI~b zaKL|+!A-mysBd1W?{CuAD$4<7#t8yfm(e|wEUKS=bco}I-oGI@^$$CSaQGl7*Nx=4PS)U z84;xit#*aun7dca)+7I$Rxvz=9WYo?NIY16SxDjhZ^wOrF98pRUA*5y0-&{URB-+} z=Od03{7|5j0e3mwl-fw-JbC=68xijY&%H)8z8A`Hjq>!@Q%)LKd;Xl@t5gEH(p>ZP zMsPZB8ON*jA+K;Vp?-IcTJxV#6K1Wh1wMa(atl0^@;E@(ykyDa6Ih7){UGXZak!h(gWqV!gf6?f+zRp!m%M zni62?ibU$g#XWwx|Ai8J$~&FERbLbYJN)rp|IZ1Kv3Irh(0(u-GXEwn zJaG0Wu-f1Sj28-?|C&BZI0sKFzEP~5NIMIJVtXyti7eQ7$YRyDT7;m|7y+9H!vU^i z++YAd5#R@3>OQ=@iT}tNh|Iv+2lj9S2>dlui{Ar^p^7TM1-6d7I77#mn=XOz9^L>uMUIdRFHo&2I&b1VtmlwJEB|bZ{^AqMZ{uWt;wWI#=AacZ zDf4o9%DIW~g9pIEKx-wF+x6{~%TCKdeVuUqtQ?5O!mVw#%+}VWLL+FyO#A%_$KYlW zp!tN~Cc|!%30*z)@q@_2GZ`Ox^;wgiipMER>2<2dycC%J1W=BTzW|$rUhFZ(UOZ+Y^{22xi{v|MZnfa?@yDuy z*1myNb+6hGvt`pgY3M%9LI*dSG{Sdch(+d!hMGTnCv-Q)QsP#o^EI5|i;M zyVO~C7TJnm?60u>WBAeOlC%zaZh6gqi!HczV6q_iDP@BjF>iN2DjV#Zd;C!R*a+Xf z$pAW_MDw{MYZe}9#`&bf)KKPL(3N=s#MUae&~GGQ~?0ZB@oQ^ zXoJ^+rdJw>+Q8=p*7S~Pf#wNwUv-Oq{Fp|_!1|?y?nsrQ@oWNYJb%XKAx5#j0uKCK z*!Unsqi#%^rFH@~G~kT&vn7x21_n5vvMRwCeAOCn$xgu9z73RpRy4_3Gykr)D5I+J zZ@-<5{6{7JTP}RgxrR?QEYL^Jf%d|6lfW83zoPuHX9obMn?Nyd1zwxygB`XT)R$*` ztbOJfgRWHUIuDa<=V1~RJ^%a3CqPNIOjuVIfBiEWN$E6O@t+0{MBuE|$-XAQ^4=MqS* zf8J(O80qHoIC{D{No|m`LlcwI%~(`Ub##TME_TL*7C_@#k&6O0Rcx!$`u5LVbrlz+ zj~?D82!1@hk?}jH_1WlFT}U3md)x)|N#^#1^5-+B_k8B8e$?0oK7_Vwptc+CF=3jexy$Ist}4oFedO>V$zpbS53*x235OYAQ+64Kn|##uC9xuA;Uu}+qzA%L&fk%A=UMf9zdwf%F#YkybS(u`FIOb}YY zMEunjIQ{n0{&C}ielzrCKi29r;x9}WQyA#beLz4CRBkCwvqI^7N&YENa{AaF1Vd%! z8(dnD>JdL;KJWWhum7AO{Zo46Hl*bQZVBQvAKxNq@g0Os4+lmQH&-EojpB7MFu((} z8QHj9E%?yJgb#)1RDjkmO!nUQm)>WgB=*M4M$f}Q- zwr;(X57W)IIEB)%p?4h5NI%|2or_P;dUYmC=;z~CE`NG^8a565k5hT(IQ#&<@d4Oa zg#QTw7>VTfj7swz@-uWKCk8g>JCqs9KI=Ba(sBKf4tdgVUeDm8@IJ8_D?bN-8)!Kp zZ@}}imps$#RQQcl&}>`?C|9?!vi1}d;8*76VJ`*HtpN679IWFy$C&Lv;8E>%&U&Q($|J+?gkNp*% zi@lGDDmh6hn-WVL^_so}QoduG`P@^-arIYThLn_IO>0DE!h_a5g1DVPp{#J1l_NIm zk*a|>vby1iM`PYGH}pR~Qdj~#LDv4^5fk{kQo-B}&<%Q}Ls`8sBUtLkQGfv#TYLm+ zE50#VAQf6Lpb8#MqvA2UK=agU0&6py682fnnUo8~x}YBX6|t2e{|k(IXr2V7nM4E7 zP6S0lmMz*KW!9&TcpQ*`B;LLKC;67^m({1zB~{2{paC6wFHb5%rR}KA!c1LU1_6l6JKD%O_fvlUgm8E zesVUWP^iWA1;G)qTLp6FCbeQ?Fnlg0X@gqugrhW~u>Upf3+tGF*K82~Wx(!`!-FFr zKD!jd+r0Y3oX^R|ef^kI&(ytYuReue(G+Zy1q?M0PCN=480|Ffm^I4M>NLgJy_WB- zXpwotN|S)B%|UP|hn~siUQFm^>N`^LQF;1==6#9q>x8jqO?#p9 zjnA1$S)!8nCGyP)s#eEEp^f&r)N&sLw#x(%NE5^C$(aOdy|0 z*=yMWMxp*Dzwz{|S%o$6x{Xl>!F9FjXx7g9*Byi_DKcUYnB5xlXQ=+-)3$+>qO-8j zaIbQ%2U#;0ZhRAIw(G%zkM)vy;TW?6QKhV9IO~IfZq@3i;HWkjIVIxKHoeOY*a4IA zw-9#LpMP`D42mv8{HwV!ZIYX*ZTjqFkHYPeE{i)YES0OhZ;?Ye-5KejgjNf)?DUW} zc)FPJ@3f1Z-c=|k$9MNuDE`o>mqc1CzsCmKyM`CNb&5>fwjZ+8#)L_PGJQ1uf0p1U z;m3!geuQJvkIpizsopwA^nfMP^zU)8>UXrao_gLaCj+U+O1FAyFt|Qo4G*Zk1%>n1 z2b#Y=e)Re4GXc_lWKT*C1ITf-tJeq)Qo=!>66o)3XMY<*%v(OJg@zfv34ARewpf*H z&Y)lae{aEKU>CA*DzpqBMN0*0&r;!14rhTFBigSKr}Rw%DR)!UGjsamWqrmyZZ9bR zHe^+43!cE8%Z}5TKO1iVk_dFxGA8y}Hd#-G!uC16D+N3Xx#~J~E#?DCXrpI>>85yB zFa7>CQ?xkp!1_ zmns19O=jF|v$dxU_Q%xxtj1?nj3JLwD-;=Rs?ssWpiCR3SH~;n6h9aDnYEkJM>CwB zS1P`p044p&igGUue6Zl5SO-$B{G5t0Ok*Z-DP}F^3W|FZ3|0rx`;} zIM$DB*m8aY5Ta7oM2U59s`)Fx4m<~Tzn5Cce&6gkI zmE?wJzl>E|9SNr=uwR?U2|a>pi;|7JI|u#~+il=1ZTo?dV$Hp3>Z%?lpdK!no3Xd7 z0~xZ^@#p3Xp}TIhBs7 zvQEzd)}0%ske!t7F|&0KAR_X%X<#QcGO(K?F@)@h1&T^5(AQnAT0Fz1ucG*HJ8Z*o6rQ4sg`C=lwHcl z@yZ*p^t`1!zV+Q=p)~SjEWH%o!?Hur;z&NI@}s(Ato}vPt#@hS?BR?K)-wXn_Qg1D zSeZdsj9eTv_QP*Klp2)clcQ~qzGmfq-YmYAD_Jv{paL!B6*>5wK-gV4xJXQTt!%l`6bkJ%}@yDr$$yrP{eS&6gjm-I5`0)uFl~%#H%*;_-cDEMXya zv9K9@w2>AnDT=kIS+bC_pkI=&`3)0o&-^Gz+A3XLB6To1Kf*`?21cKNS8C7UAq)a-2>kQLW zhmt>vp&@f^NCh_MkfwN4P8`dQY7?$}Z2JXHiI;+9BgtH~`&U${4S3Sk6&H*SXW5cz z>5I{SZR=K@pk$Yr7X9$2cDZZhp~@9=Ry082}yt1WIL*IafKS-9by zYe`EhI#ZDmuyje?l$XgPCbtZ8)xve@xTp(5U50<$V5UXV!gSyPn3R=f8WWtBU+#Ut zuoAeUgJLdhDm>6ih*u(4Hk6Kotx?!n6Y1SgA4 ziDViE4`7t8u9_to5Gi)v;PbGpGA#suUKX#Mt`BW@`*FRQmUcNoDI=_{n%iJNAQuKn zf~3oFQ{RIhxGY>5Pq4>LuQVX=HqFxgk@sO`Ai*0=!Bx;S5G9mu^A-`y=?5 zAUa^BA63Ci4G4IZgEUgQiQRSQUt}A&e^hYKb`@Hp6G1)N~mZJsNem}vc%^|r{@By}s zpP;X((yW+3Zb2g}+IzWcNwKKhw9DDt94}Ok;BaV+F0?i+r+yUK75)jQcM=@ZT0~^( zX7d$V`>lJqlZ1sScD6T^c7~ngjU(5P*P?XZ?@M#a*h?A6lfzEQ73;}??iE{1S7$$T zM&U~ke#^R*bp-QLX(0%Xwsn=8(SxgAdXspo*p%DKQNj|&!L~(W?&OC>o#OUe_PO4L zcR&mzkZ&jp2<8>iKIy;TN^K-dlRr$B*^3>7b;~I>5t0|To_IS~IQC24OANO3G8@VgCY(}f{>4;+gie_P2j@rgXfT=!_Ec z^u%S-8$nVoty6l}L^bR*Epvag5v}NZThYIBYL$+J;FXlG@M4@U4mGli^defiHn)uCL_)B-%C~!i z>$($l8BYsVu4vwd(JXh6E+&UmbSOFpLvp=aDlP2?xoc55{<^POPs@}xgim0HAdR!f zt1fY*j@MMB$YoqEJ?xG^VstT8w8a)Nvb z{v&VYG(jCBb$LzWT)C&tv3G=hCm0+UGK^4ix72vxs(TRkFc``AM_y zs?n9nxJ~XIYjy6ZTo{Q>^U#LPg;L1hZ1{faynoYoKiTam*z5Rs`+53>Pu_O!EBvbX zan4xncPrLx_$t8e*UvQp#ZMDmH@vM*d^@2v|JYiGG0ch8#LCoIdBqO-OCzNPnbsIT zs}EcW9~g1>OWs*HSMeo$yeNLXtn$9Xizs+I#pr>EqPy!ycb03l-w``fj&lS%#G>*| z9)GAlJmPe5R>&)}InF|!C&fd4Hjm1C8-!*)dzp5d)fp4YlsWQB-$EZy+?odC)=2>9 z7gy{4PDjH#poE58w^J@@mE8giX+1Huj4;zNAdZbduB9omAfREm`8)aKC{z_ zeJ;p$-$0Hx(xc$rBIp^a%R8X$o00TT^?Sts_Mk)?LIW7alxp3di(Yd*N|Y#fSu3jJ z&F*n^p{^-+M5}Xz7R(f->H^#$di|Y(_|q=TR~j9*LaG+M-P(peOO>5K(PG_Y8Wf)? zTT0FAYklO`r24|$yk+#^v{uco&+Yx}H%wjj+L5I#y3JFkkVaiAb?xc}C3)eb!S$N% z1*%if0^g>e^~jZ2{`e_k+?4w%u-o%fffKD;gh5GNsyN&(1{8}z`C8oM%E5!rX>QUb zX|e0o#j2APQo*^DisgPh=h58KQpj}BRj^srisHsLP*0#&c*paiZXd`}JR;d?RDP)T zt}loV_`)#L(h}t`sw=fjrFT+vb?qQ_ydUXiIb0&ROflr_Q(C0TGf`#R=EIa2kz`B+ z0bcz$sq+$QzelI@-w_?CSBl9{t%&EsJWXafG!fGhaT{<21BHD4Q8igzsOZEk zP^F^U71Gs$Hx`mzldq$K>)Uo=ncTj)gR)CyD`I`eUJ;(lD|__bw8~ld9exw#6Qj@;xyuBYwCoCQMB&!RQ6Y~F{YM3|EIe;?sLd|ag6)^O#_m-Jg8&G zNU3HU1yM~}A6U1NTD;bj{ifvr`6HUHXuRb_j57QU!lnD{o;S!$-7a@&vgnV`DE7m; z*lBt-^pPLW6Gw|17nVHS7L27ZW-~2yKtqrWZ(q!K-6-?RIlk4NEagb&xRo8 zc8tX_UbilAB2S%o(xkhP80VJB+U%fgL~;EfmhWU?6xzeBITd+6i@keNmL%fSenji8 zme{%sKUecs5pzET#q!i`-5%FVHrRb3^Jcy2#gtqh6Cqn;I z>(&QQ0^w?cY-vz8Yz&TujZCjg1mu)VOU(0fm#sqa!=HNiHr>kT;|(_CmY=pw<2ri$ zK3$%QBV5B?usDz4MP2F54VJC^RrPf6#4}gDMv$4>d`#yiIQ#B#Q5QzCoY8LQ+gfR; z5(g2NDf@_;7luBf4%uRHXd%^(bAzA4ap*{Q>26Lv zjN1v{Inah z+aZE@Ir6+Eeg~4X4MSe*Kn$xmjR;=LGuP9-&tdBs%_pXWrr%FHHIFm+Sr7H9 zBh(+L(g&jMNV=}&GbHJ`RH8gz1b|2OSxl4Qm zgiuwi=-3N$-d6O=VbZpe+xcLKTrdB=;9G>49%k}w!WP6;a9t|nIHzsBYxtGK`n~QD zsg}fS!h5%A(CSzLxuM~T`EqM9^pQM}$dM#Ak^-BkEBrbfM{{HhfG@;bd^4`hwFlu( zVx*y{kxiBrZo~YRdfGutUwRipvYOuRI$Bg5`!7rry8vkL?Sw2bNE_6o{hZ>(__zBV zu}CgiD(?$3;`tW|MJ;iuNHZ*etElV+)w<;tvHUOCjQyO6>J*-g!+?3eu8e2@}ekYLmfSA>U^f@<4X>O;*K5e z(v@)DG>BrTc9$n1z84+UJGQhr+Un{G$I>_H(uCHgJ^Ull;z>joC zE2QJ7v=ZYdM)~o`#MtGC&IKL}{KOv8R@G{SXO;BoqSde%Iah zk?n*o)r?}GuCnWb>lo!L=W5;HK^9`Rpe(^i1`pHaeO?i}f%0(OcH8;YUlrj z;KI<@FUd=3tbgwR%lXQB2wfwHw#6bMKNuGx~Deec2}7l)cczbzg5r~E^&n+s@=@={e$$%H@S=A(EU zm5V6xv(OVa%_TlRon{|GQAOvrROOUrexUamf5+a{w?G;Z`SM(QXy~J9;+-!lnr-=W z_1fz?VzDgf1tC-!f989*n&d>3ZzDeJ<#)-(xXzY^!?pa)4g~#UHV53$b6!=~qR6X< zBlBuvcbUEUq5bJoescqmlPF^^S$L}gZh~%zxE$$1?4JcHNxvu|JfuR(X&AtbC@&af z&4k%Us1BShu_u8SN?93C+TMTh`>clHTr;*r!&>W54wQfUyYe#xrr||)Y z^m)Iu55>je!_5nzI??LqSy#QtujXD-ud@Q0Mpp>hyH2Y3K}vRgVWsDYt!T}QEZTH4 zzxl{}JrqRT{x#d9n-vo<@t(wyxKpBKUdJ(0S?k)=d;48xifqs;BXFQ_K@_7koaABM6-Jb6Rz!PgQqoym2mqsN-oX$ZDcve5vNMVDFueG!mP-Qe*} zsY=GI_dmg2W@Me*+68Y@*i4nUaIOy6K|N1NZKb@HfY3Wy?$+0gPBQWHBO{LCj7E#- zZofg?X;SWjq*%y1`HTtQ=z%ig$O~xpyyAqYlwVnXfw%MJ(jTAcmh>O3Z@VZ;@!{7J zo`4)qGxB?hR|`k3A&3k}QcwD{iJpfQcct%mk%cYObn`qkbhYI#-To@S{z_+jdW(g< znXr%}q!P`FDY0ezNW#btuBpd` z@UW8A*p41G6iv7`8P`~?^O%wzy^F{cW$YS>o#bqw#Hyv9B0e$!#z7w@Mxm630h;-~ z+4z`pLJ{!BUXF7?4p>t11PSi=A6$&ca#4Czj9oX&Sz|18J4YRHr`ho6r&E4mZFL}| zB`#gT$uIRB!3f9r{fxD_>1-ad({tt*eVe0}-?dwH(p?fsXu7MlDG?Gz>suq(Ur$Pr z_4<1xpMI%00KdL};JRUwkA;>GqlVXABCzw#S8BIH@c^U6!SO_766cmafm}{n*_2km z;m!DJYy!1kP^TnwJ4fT7kO_ate#pXRo%7OowK+#GdXM;B$PcA{)4HRd5M5m6s2#Lm z3&P_XD|9}Ivae*fb<6t)UF0pax#SJLh3~qc3Dkx4$(I`+K;Fa*rY?P|*_rcTdN@zq zp^1VvuVJh!Au1n|vb$7@omB#!OVESSmBbreCiJylN5pAjlKWQYph4ZY!~}O)6)L)i z{CbkdZ7|*=q2Iu}2A6ilC9Px*8p`>Y1b8b{ifX>4-!9cBIpJy9)2D4|tovPQdBc*` zX}lia@qrvhycR_7sWgqt^t)`R2gOf4&+u?$@r4OfLT0;X z-L=+!D419feX6mL<#kE9++Er*A?KwM{yxIqt4?!K45)Mv=LyF5?+OcRy(S}X@+2~- z+cTNwZ~Ej?l?&&dKBEA`ceETeayH-!v&;AZlri2ISjX&Vr7uuj&C-QI-OsSR$OL#4 za;bk8j$U2PEEIP;R~4-_t<@+OPszlD%jzfJqjg){p&?W!0$QzYxAVAG{3UvAi!O6Y zS}f|eC0;Hzu15NngvB_9vo-}9^W9D(F*$J~i@Zr6MS$XcT97_yvf{TwBuCXkrdO5pCziHE_o?b5dHQAZ6~8Ds zZ&iKp?nF$xwpURwnPALEBN8K4V0IbZXLh5_M@O*cN)k%Lk*7;PAE$I~8@=qrKB$&@ z?F=P5yA_hC=;tI=fT$x#`cY8(Hp;$XKW9uKASW^pk{G|2SeUuh(Y5C2Sk;{ozk4Iybtb{o|mDRrMI2^*R8>_>R)8#&RZ-Z&#^7(zw@ zilj+{_rg6gtM*RQ)6P>X&cjb3^$^k}{bvhniu-SO(DG)=a-rvpgmD@cYK(w0C?ei@)wmKk-IHE@)vxe3!jq5PzhCJ=u(K$hPdq3P_PXB-y2 zo_bB5UD*gB8fX}N~^qApv`sV1Vs)MYD~9gY#iB6>pWx6pdJ>~H835kF0`TwK`tMUhK*S)pxBZ-MtY zqYr;uMBR#%4$UBhd4jvB!t4w!pG)jBjDD6}z+D3{_IFXHO_vh#!zBwN+s`Jc+$#nh z{Zjfk+IPOAVIy^Z{R7uBN?4zjsESb?#YAg;%DIxdjHi$+rwhqx`5dn19dzq1BIR#Y zz)JX9_FfS7U=QL$?_-KhPqW@(mMszR&!_f#^`FR#Ja4=W z?T-B0LLr_>e6m8Ezt;4r1E}|DHtrCDki+4*;__>4MM!I`h$Fv`nrfy29>Vamj#7;A zoyki=e-uvoSaYI~(=qdn>f5Qr<%)Lc-Wh-Q6leIpioshH5{gK9Pf;FYCPC2+&nk@z zqIHKWrcbs$HRIm6*>du$A!Q!PaW2kO*hPPg++K1JeaQ4PYhr*$bSjRAs)<+e?c)qc zD(bq?MK8q0*~9sOH*Zj@=8;{w z9y)&)A>cP`#+5{|7sT^GVhA3BAZB2~lS(mMB5HWm8)`sGR5ls(CpE+#GkC*)8KNB`}$T{V; zin4LEmPlUX4Ys3QTrXZqJYE3~V8B6gP~h2A&Uxy{0;4^T3GnX0VHpv{#hH{tZ%AJAie@KOh-i2NWw%iliVA%};oEua zv8)tKK2oHqb`Ax{^D>`W?k|Gp-u4?TE6e`rRCfq3dWr*}6TH$Npg&Phz`Jw=H}Z*b z6)*Av`zFh+NmX+f$+>U#Da7y=ROO_S9+bc*0JA=_4xlXZvF>caIZ6e*V^;btio2@e z@R1ms@4Q(UY7wMDqZdXn7c}Z&k5Ph2}46iX&WHtVKl%WHM;6 z2)&uII=GrAh#O}8i$@GtT=J_WC=O{=glbnG_RETDZ`DB_pD^-ej9m-8Fd?qGm=)5$ zvmjIUzZMdz3AZwmi=(O#s5QQ!*UUGdft_uKA+}$=cvJqNDayt+oGg{s9G6+e$~6S5 z8bv2Q^~f9811&E<$oC=SP@|YD8Bxp?Wt1CgPDIS2aF#~|(cV49u{rC8_MHQG-R#Pu zGFHPu(+vYw_1!dGeu2t9Fe$4Qz4Z?LMaGY>pDSlc4)Q?4o|31ZoJ>KBBtleKI3w&cd~fB2kqSubmvKshqE7ltun$Ly*c!^nRYcPckZ80 zeXpd(@(1yrR76+g$UmEtOKM$sNw2eb0;wc`h@WEnQfvXdkJ}!)54|jtRP_ewIr={t za;xRgF!i(kfpP|>E1x;hr(W$>SDd{cI{E}rofqtw_TVu!&vt-^bPRgXNZk&*Kmnox z9_Me2`2-mk5*9OiqzVo?Z9Fp3x3YPi$IBJX+|4tEm2V-(ArERq+e$xf7g|&JCYq=F zC*{#e>c7>2NTq!J-qJDNAj2i`?rRImN}rXhkTR!*E4jYErWs3gPwzZ7jcOZDH?e zKPO@P8zRx)z2f^2VuA6b%#Rq6wHN)&Z}M=xl7pLt)4d#e^XMjOD6R4ZL}qTYB=E7V zU53|=W!=PBRVC`&mgGaQ1i8 zWi|Ew`^b}%hV*Yopo=q{X5KU*56|E1akaPuP^r1qr5K+0R9qfu)pYhL2%lH2$dCLz zd=~xj41F8a`|-GcM9dYI!5K>RE>Q5hzE?#(K@cOMZIrN(TY9N7wF=>IHZLCf*KItU zG8X}z8PBIr>g0v?8d+A!!ML~u>L|YzSVcGKcHZQvxLcrHe3OT)rusUjb$iax2j*U4 zyBiOqZKBKh3H5aCxlHnk;Aq~WQHjwZW{oW-zXtlgM%3{kP~|m5ZDF|qyo7uTC#&Lz zJechk4^?t4T_7bfS6a-G)9pZMG%ivNQ3LMZW|YP>-J9pYVv&ONTiIK%8@RQ>~_Rm zjbk60l*XZSG`as}b5RYnyt)0#;W2!(kmNyREHI&e!lLt7+1-wjv-#Cdk*ZIiH+!Ik z!McSS;@c~L^rwZzxOMA1Dj-?|ahv)TS+^)r?t*CdsLC^c44R(Yn)>74m>S2q#-|>k zwR*o*awJa;%Hdyr*KKl#x8IS%UTOF|a{w*|?cQ>qcdsF8FZxjIQB$ji^DX;? zQMMM<72c43ONU&m7(A`sT7~eXzf0f+5FLP?Be{6`2S{E|w{e#ozKfuigqcE0lzZJ} z??m-H>OVSSTgM}8I&pJF&APfgFOVNr@t4`ARw=o>>(dHRH13~}3e^&Z zM%b2+?vJNDS9?`)sms&c@{2vEurcH%@~4~Sj+tBeX@8G$B1bA%Ssa~bG3dS0jL?|p zI4#Qbu!;J!`gcl>9)Q|n!{*Rm0hwUG^i0-p9&^nLHFgcJgRr3scE3tqkTN=q59{|KbvGAmAwTMssKx)uC$hbo3@}PD{%17QH zQY(8!q`l~c<+`Xu*&-1i69<}jp}yNIR8Rq`vstzpljk(DEMB)Yv1wIgyK@yiiS5xs z0?+K~F6sTD$OT{Q6|>A+cd0^pR+Q*XaUW(mT>{+?S@V-nk9%{(|IL9UNpHSj)E7iZx zDSfBuh=0g7GfsUpxN8@Nc;iNqVj0nOioLn+aw=<01Tl6>6Hk>b!CWq000j8KB=XrL93B`yMAM+y5kJtTFt4rUv$ZdQrm{&;8O@~ho?tT-!6zw zyj*=D3vJtyl8-%4nWu){aH*2jZpd%Iwsw~YRRHoIEf$*0r?DLR-?I?gOM+!f0PC_u z^^Lpa6*$xP^~x{$XwBfI0K*?H&n~1s&p2)GrYK11+|*_ht-~eC0!1C~>I;ph5S*I1 zQ78IfhR&yt<^qo50$Y-wgX$SVZ*h<#?}NEeW?HS5kCiOPcD}WR+#1xsvNWGg0D7KJ zKb`rL&UH$XEaJBIASIT2|B9*4z`DP12vNW7gekuSj++5(%e;)P)vAx@j*WVY_|ZR- zNeBn!;Yv)u+R=z8>jdz{O~z}c@(i*r5x16bjnS_0hDt;g>nJ$2wC%N8wLCHMwE1-@ zpK;n&umOKLt^(wxJDnM);{HGls-Unj1mV@I*NiSbcSj6hSL`AhXpJ8r50qiIX;Wa^ zN=|n6K9gg_+0Us-m_l{6{;b+EsG|7b3Mv&O_yUD(6D>}KPp7o$7M2i4>30ymS4tN_ zvlF?1b}Mi@m2d-8Lo#n7>u0S`3DGan*AelThLqD)R?wY5K^x;NAsMcd*^Td^z9)j( z707PO8r7m&ntjo!R^hCL?^u`9hvWrDD5pjHk0IS9w^97c!AXAhYc~;cq8M@&ne&Yc z<3Xnreo1M!991n?3kRz3J8@*yv03T=Bk9YdlH9)cH3u>i9J6xZqgH07R90dVR8}_J zG_$hQB=eS;BUy@x;85X^(rwamp!AlNIW;&GG9*ZrNuyetOo^2P5Gf!CzjMF8W%ngjA%c_zwUf-LCrHH{{naR`Y)Atd! zD}?)8MyG5+!n|#1T2RqooW_Snc{+Po?l7>yod|>Aue5Oar!)%D@n7acV8?;a&|mN& z{KWT)F`~Mr_{izTI^hsUv<%6f-~x~0+f<1~8T#W*rsXb13hkWMMKc zJ4`qiG|=fZ*{X%pjrOBAx6&Qh_nt2v%P|_7;DBXT8f*$Y|?)Jd)i_0 z-&tJpTMGHzaNBOw;!Ks#S1#TZr$~~oNu-coS`F_ey>QWl47Pq(2S<0L;KtiU4r@!s zga)llzwtfM#5Ug&*>XoQC?a__f7wOL(F)x}v+O)Bn-rH7`S*$m!0!+a;x3g7=eM%@ z#GG+*`$B0S~Q`**hcQl{4Z}ZQ1T3<=!LROPR?~lq)ft}C0 z+R7n68(!6Cl-c&mzDEkO`3dvw`T0(ffINeqphU!L9C;3dZtO;pE#;jm6Tm>n?uQi$kUDJS~cT1W;+|_H{N44I8fo4lwF^3%vYpXN4(gk=};|o z6bO-Q>PfmlM|LL1q?##BO62oF zCU3SRk>=pw*$xAB*)@6chGY63PzxHfq~KzvPW57@NtPPq5qIJqh5pq{osHZU#6m&? zcO;qA=|azTi_|()jh>U_6&H&l=+TS>kv2x`E)M|tyW%55Ab>D{$`)31K1n&>FI<@1 zxkz_pOurJ%Qzb;=m+{g)6`Ik5WB+R--)rmVj z(&Ql(g54+nA(ghRfu?vl3SZy~k5mY2o30j8K{@Mu{Wnp_uZ=X)Srk5}k2}`fQ97HS zs+u2^BK-1Qe|_{qK%~8crVN$_ijU;XI1aBruMK1R1$uMaLr<`@z! zYuh~2f{XYdVYG6Zw~9DHDLpp2QWNW~eD5bmCsf%}*}W!JJQNYlU7DiuRj|4-z?^f^ zk9IC!q3B#c`R@dX)<&#gpJ!BVj>7I+m_|IbMYhXP>~#x4RryxJ(CdOp*l@!8^zSbY zMJ~=%bNYpmhPcFpNLRXjt;#GRYQ_>-BKj!#Y< z=(HlD`=ogsqm)8^XqY|a0DooG#ffl{G)58;K|6eetQO73zEdTNjuj11RN;=F?f4gB zhPGzCP(_b_b4jY39V^0io(Py0yY|(YDLSh>bz8U}cR9w?UB$b#KvHx+rF1u4sG{Qy z0XLb2yTiF8HJWVYj5B!b1Nr6DR9$m1{7zOjh?yn%nF3p7*JR|R|F}^3Y#0UNDmFHL zR*g!ohk<(hJc)6nBzV3U%RcHNo6G^ObjJ7vl=}@5gCXNP_;A#+nR@TJ9C@prCQzUf3O;+=0I*8GLo_{qVPyQMGDW zn#cuY{^cWY@NmIAB{*hyGLf<5B%jDH&< zzZ~RPDdygzG|}%j8N+SPa!Y8L3x`K4gz5xL0!RT*V^!e$JsVC zGQKd_rWKn1_Ut43hTK$DFCd847u2@-r#xnM6J%cT<}7$kWpBSVH-6U1%jddT7vRNMV5`s;hmF!oSHY_*Qt9? z$mk&*SsA_YcY#-e#C~uw*TSZ+=CW$DXc2M2iwhit==&PnZF#lG1%RvO7zb)Sq* zHBxY{`=-!wS)$eS&*pnp+kNxkc4|nlY)j`I^$qU*GX|JAR^C~ zFC+z^I>+)IMmR4D22}_VbiMBrUE9|aeQa=sweZPo@z){J)AVE; z%7^K*ehB1!u5MqyJxJ_)+7zjZ{mtiOfK;sms2U6&HD}~Ip&T<+HWU2IQwM#IVdAp1 zd6MLn!+M|5`gYa)l5m;s%j?nh8zP6+-zcelf2UPaVdX)PE}>94!$v*93L){UZu6vP zMW;)UycC(agSJDlab?g#lB58{r&SBE%rt+TE>Vm6ILXj-|6EDG*j7}RSVOeOxjCvp z&0BRvWirm_QM*tTh4?u^n$y(x!;4o$<#oW=nQ>IXM`UMdQmvj0VtjHkr#Db^0@>yC zUqrVeNItiiw6f~_9u#nmfBP#qq@(usgMg(`??j+8-{y1M`~fx@aOxz!Jn;%FD$1U$ z3Ve%+B3Sy{urtzs)8yBYz9ErD5!8YRk!3nlV%tS_(^gGWU!M~h@ZQhhEz&UdQg-iJ z+{KBXG-*mMLU&_aFduXN2SO*@P!ht z53Vg={SM6NZxu2hUlEMk}r5m2?o?r{+n|`K*ep11LZXnaEb>l^S?7FH1I&Y z@k9*D+25W0q=^jroTO|Q;M0V27?1hEMU5{f4+ADcl%+36!Ur7P(;>~IZ-#^RMVZGS zur3ehdiC!j#yi~wPV@78yv=k`p^u}T&FB{{zOZO_Ox@GAc|ka*@f%jH%-#`E+G1{2 z%EiEAj7NCu5?N#-1zhvhV5u)m4J%aWx)iisgr#W>3-}k^hURhm^Rp(fB~yL)47ocX zJK!2}@jgMa)%9UJBTr&*k&FDVb1YH{kBnnICkyjdhioB5q zbMVa%K7~%u^_~L^ z2=36;*`nF-|75wcDYd8#6C^jT5Sk3M9iZ8jtCqB+bmyVqk>>twe$y$Lj)MM_e6C<2 zsUVbQ?C@+6yd~w{6mB@+6ZbAI8{|9vkzEwAZ903qliYLQWnNPr=EDsBc^}sk2L-eU z0ge0x@gM26KDA zcubKt3^olb*0JmT+3;hK;5!|m!P;lwwar*8&7NGMs;)s%OaU$ZQv5MZl00i5{?+X5 zSwLZL3)Fh7m!lGdQvgNN7z${QDX9-Bwixl8u1{kW-2}ABJ&46FE zAc>qfWbdy6ct5#`Y%?zIL^4aY-P&dV&;i>v2cxDHO=+r($CPA`K3?M@Vin=WZc+l~ zwfci922kQb(5h(zc^k3$1|orj>!QRzj7WtGj6q2(>vQ6veC0K}ClLxDF}#KEJmi^&Y>3bf9pNu{uPd352uO}k?h)anDxOv&XIQIM-Brfcx`i0W|f3C z9i3G`ZY;udS?L@{=g1=yrfeDkM*@;IYzeQ9T9d7jBP2e1Bb^=c*i%gx;GNLPf+MK; znJV*f@mVAj+-=&Meo-x2f$Z9olIt(BRf6bhp+mA<$5}Rf-m4-SQXYGgcT{eHWKQ7< z;mk#o`SmWMO@m@3?=+fR;4YdEsEGcey916cx{yJNZ8E1_*^ap%OIS`Q^)Bs-_42@hmx`uh5(|nhGK&0#(ijeoX7Gz=q?)%kWqc*INP3$=hTmpv z3>mu6Hszr-Op_zWgC*$O4F3yCIElgRtb4j~nBr5RoFjiggW>Yy#kQFIw>&@4?cg^q zXlyF#KL&%pn*q$A!^x3m?rkjoI3-Th$A0@$6b0C7(J)>odbn@`!LDt}+pP^i6BL+F z`YWS5hgcMl|75y27uGbgKX#Xj<S(t#`$Rh+X}#ZOhVRHTqE!x9Ek?l5qhTwFvAJ zG?H1Dam$1?*tP4cVSwoJ2sznu|$y><}+F&g*%eR!k zXjm}Eu!*Hb8Qv&WCQVWx1nalx*E&a=2eDPW+3Jj=_*=$H#C!18|pya97^ z*R&^8PIQv4ko&zQBRrW3u3giUTca0Z2=JD^6gZ&s5vqrttbuMTj+V(SDlFk=z)y|#9#X5@no7$_CRxR>{<08abcoYc7A#YnZ(*g#? zc}Py|)8Xy72f5?+pg>a2_QGYq5E#Q=>Q2Z{RtOQzFSgOq8ekbUlLJ9&eDj#N@fkYm{*?U2mfvdbHxFH98-*uu^o}YUz0k;& zAV4yyI6`djpF}-U%TzDdR4pXs_@K=FGZ%ppLGo7;RdsEed$+KO&R9(x7n~dDREiBS zpyO=Ce+CkiJI3uvHENkP$jS9hGm0ob@VMbO3lVex+qQ#_y_EcN;X+JF=+Ghu2)hv= zx(ivTgJZ%*#MP-FxwWJ7_l#(?7LpUKsVgT=bHIWDLACqnpL5pb5IS1Q4cndQa~Ct7|ld ziU$B>HPN9KzjnRODwqRmHBL0=OgtPaI}G|JY-`aiDTVPO{R=<+*_Z?&xhZe(7|=&U zXi7dgJ_{8zCPq9ZI?D<5l(3Uow5T!hL^xazeB!++`O>BLOM@flMYJuRPRSaNv4?<~ zVdN4blx$z55iMjCg=#a*Cd;*=nS4PkV352>r}=QHXiii+G40MoRHaujZlcsrYy=qO$Q+6hKmI@< zF83G?1|22^u}IRQ-=)^2Y;?mVAA)(%?o{|mM@-a3W!R!=U-?|hlBySxs3q$rW%k7DBlV3~Ea!lTc{Mr3tTmOLuF5&fK{TQ%q3t>zT=e%P7 z4D`Uux|D1ZE6J!AHZ}X=OLEw%s_o;gp7;4>io(2mqUTR}vW@JInSKmpeAjH=TL}%@ zR9q{vBsI?Baw`{zu+3!N4^_$?&u(%f?aFLUawn*Ih7Q@qBAMHF_ku>3lfV2Zax~L9 zs>p~I%q8nttG&-Aa0?yLGN>c6V>Ut?8CpcmlAcjecDL6L#-rHk`DG0O&5 z$Y6oiE9(S6++;CMWf8WIiQhEbK{w>b5FCO);sJkf$Sk4_o+kSdkJpO|ko_U3%IT0; zNx+cwWmZXN!#K9$tX`UBKXYt0!}GVEp&;|#V_3)E$WgeAgegH*WQnG(wr!DoZYxJ; zKBRvN!`d>)F(`{#yxIDwV~k%6ccBwIk+1`BfR4fKpQ=zfZ$xv$IA#sx|9Td&3ovZA}L81Tf+od zAA&1tU8!Ly?+YtpBTMkP$uB(`0yD)HlMuiqsKDp?i)DLy%}YQM{8nI`&o6$P%n26F)%4>?y*5hF{j)=}kiwcI$os?LjW5x|c}9cGv0_ zadD1f2V9{;OiV}Ho^`#j$?Zemm>Q7nOUJc@piIc+fSvs0g2o_FmF_rhiG8Q_=Wvq0 z>jpVGP_RGirm^GiME%84S}D_tB5Ycxo3PI|_VKYrI1I>=DYwrmSIhCD_e8nxEq%KD z*tp;=vg^S#W;yzvV&jtDR2wx^ucSXK$)6g9IgIg?Q2Os>pcRYb+{ahG7Iv`%wEydM zI!+S*C{txjnN7Nb5+5pGJ{nXIXrl~P5SL6=*dbi1?3T+L`{Q-FppV*n$#9**V&dPgT`f&3_pIHOGUqtz&2`E4Cqq3QZitn*FV<;KAUjxhH-#?$^zWuSE7#dRH>|(2 zGS2+d(|7l-*+;fqtk2@5R`LCm6{%9E?$d+a@MLs*AAveLKDbeHoPNUp>l2L+JP)nf zCJFPL<@7Kt(0@`S=^hHoQC*v8wFHsP`)jqnXr6wp!YR;-@vX;AkZ2)E6#V1XQ?rY7 zEhFkDWi^80nnlX{Vgx^Dm7WYF+gM$|zc3XuwoJc};H8?cdd*v-cTu9V6B|pP zOH&?{1m;TRdnDW(uD-H;zRq>Ph#~gx852N=6!nO}N6?x_nv1p|DwJ06ls`riL|!W9 zn9fD@+5TJD?~U9tj46Gg)=B2Zg5fO{F-&mE(;RI@(M^AWyUtqS!>nyx%{M?JSz4&< z_|+b`A!8EPLp8*Llt#RO+q;?s(O$`r3?UVn3(5kS;6l2&Z7r`*P#wM?Bgs*nwMYa( zTO{QKHXbEH!Izt+Om_G9=J2ApTC^%%a!!Ida85RQhkDLX5q3?Fh}QYaT1waOO`**6 zo7tHHOFJiH$f}HC9pc#H>Ni3xU+WY2!E`MVc~4ZKMwg|t~HP?$G`Z%aF#AqBQW=pTdrxU-ZH z_#EsUD3fAHH2_n}^%R5(oK6ag$lyt_&zT`h%Bjo@?O=iC*~Mf_Mg6m_mpT?}3b%T1M@(CwD}#DH6_ zfIKlg6w+v&8Qwuv^+Etwu>>D7Z;~t=vz0l)G)lW6dWsBWlSiYfKfl&O-sx5-_4(?x z`0sf=SOcmtY4-(bKu?Y%H;?oZ#>j@*(r`nsagnJ{2*4CHP<@_`)EiLFY#Tjpl;5bi`E@}N`5`dTdn^b^B%65qX@eLV`7Y{27C=} z2O`B8HMNbVXD*d}WRjU#oYqxTu+a=V!W?b%2was4R)S-l*#0q?o2nq zHkmEngfN#7`9+^8=|?+@;WBAnFwtxTQlVL+=w}+LYQbJ;Fou&kfvon%+Z~3bObgkM z!;;t>N*)ktb0Px&y{?B;Kc1It5}YHIZ9Nas{!jJ^vnDt_`7) zhQL5;C2MK);m<)sb@bD4ts-;>g>-^wta5BaUYH)i?$zDK;5<`}XLrhqn6=Dg6RIJ} zn6Q9gj#M`G@J%S$QomwV4tzuLZwulam9feLd=whUi=E(!fNw-K(l5e)cR1O|90bb( z5&ZM*ogVuy9B)f=K$#HdfdC0Cj5{+Hymyq&(FVQ<(J`s}C5*RP*gL8TDWY93?Mc67 zNFu9UW!=&>nE8Zx+J!Pd84CYh(*t3JVvbHT4cjnzN)Bli2HT$QpeVnViP@qtcAj0W zs6DFx6kNtI&YHyBWZZdat_zmM$q+rRSxgv&YJ#yMKnZ^MLDC(a$41qL6pwUp*zIYU z7AVUjNSq<)o$`(^=w; zwSKbovQ}n|AaimvUfNRufuu(hUGcw0%iah+3Z8%9SGGgY#>Dk1r0Tf9M!+|v8uMw8 zk+I<(c!=-?Oi|0Kb!KYMj_8eUl8n*jbY&EKbaM|Zl(1S6!@3LcLFDjI=hdfVKM+0` zC&KTrGVDBO^#O^mr$zwFsdav>T{Oa9+JhY|nz7I|e9$jeR8HTaqi(6`8cU3EBFs2vD&GnbOe79R1?RIV2;;Y*Mw>h#w1;q_4s_5(w^! z7+dMFVHjG0mtZ0Ab8~+6uPfu52B!W%WK;5ZxmZ(8yZai35n&!-F>1|)84Ksi6X;sJ zuVQ1BaaA;Zt9+%x25c~dWvE&Vd;<>boCB-f^)(DAsyWd1!Pi-`Bgz^=E@BS#3uVRv zy^)79LE{XBCR7u+3B`LA!xHuEm3D*7MHZo~q2=;xhKxva2vF?-S&Ov-glnoQ#Vdp? zfH}&H5xjp>rp(?gI4;9AL3+Jb!%ZQAF;*s~Qvp2>hKk zoy#LR6BcOqf?35d+da!hS?;W`+(2s25D;>B*feYnW9t_PW2dZRa;w$0*@N0e-(mPc z)hxKJkZ|sby1~3>wCa`iXv%{g?ZT84KLrr~Hbb44Y=gXV8v(kS1O;x9@0x0mloOnT z0W3vT$WjSr4g^>~{(POeES>3gKmv*E0!wE$d@zm`mt8H78YNkCXTcJX*xGcUg22N-_F0m2pX8Gn|=b}w?C>4UqIPE8b z5Uo|2f{(fIzvL0oItUO=r-q|+W8iW^Qsnd4_F8v=0ZQ?=1Y8Bp4yIvN`!~*Y~{^CkI3|Istev$Ur2wL?_z&BJu`g zDz7e0%gb)eCjMLS8`^8qBi(mSMrPFrNT#Js{z1ZdHMfwm2BYSfz!BF3#}e1|1m`4!XQ>A!|8wd_!_)h}>wU_o24K$nWy@6y@$kFijvdnxpWl-weL^#E`}=}a!R zYY0L`=ln4kjmnwcB_9L8MhgMPOzdzO4Xlqg||4*0U2PB!fMlv^GLn(%( z&wENg@1Ko{GNI7U`^^dNu#+2N|Eve{flX8WBXD5F2+CQ(2EZaCR6c^GGE5G2o(?)- zq_=M~qQ_UNR{^bb)LP1Jw)$YK2~d!#S5%e2VQTK#e+@Y z|2vfFu1Lm1AMz)u~KD;Uqmo!n};@n zveGl5uYG`|E(YTbvTB)dWBxqdTKr(@O}!z(Qd_g^H;jOR%Z)Iv12g6X2aw(?%1Un| za;TE2Y8Wow*YwhZkNxL5EK0e~TN%W?(X0pl+%i)5e4flD1kRP>at9V;T#*sO?C2g=-FF|Pl zb+DUY^XIBSxfz~~fzEo!yaYo_6pWX^ioxz+jCE@79_caNb?=@$F8I@X3&|XBHuDd6 zRX_L(dK+T~#s@dO1z=_lNkZT8jdctE#Ii_6vp>XTye;r-hEdj}Sx>vclm#JJreQG_ z{tqokd`Ch3lR^HFr_2ewS5_LWL$;aBCJMxY_i>Uj7Ar~Oo0~j`|F4$1S<50bvDxbG z!;;1=V1eQ9sLR*z)r+6<1)9h_?YB+(e=qketkxS8arb^XE1=Fgpd zg_fiTkmeE=&s-INkA@N!%(OE0m)*`DU{&GHPycHe((jB&_0_5-~{o>JBVZV~BeEN_*RKw%A(X229pR zpgE(gLGMn@oXI0F^gOf)(W+ zAwa*PorDl9twei&jXhJt)Ew2>wwY!%bkU$y#tchIa4yLlY^!n;X=kPS3G~b2X2Zst^#PFj>%22O&|hl)&>!bXbE2O9=znR%tLkn zV9o^~ybcG!)oy@}k_|?m_as9o*_gS*raD`^t@b#;K@q^5LwTFDESMogU?%&UiGnz{ zn@(7OVZy;M+rcA7;1LANe(D_^z&-Gj!+_}&m(`QJzk&JKE2|fvRF0|wfimzUhE?lC zzr!OeMIH)NLxP=85DoG32yUvRz_Mghs-eDM9zykI1x=8g6YO-HKKR5EE#nA=R$s%U zm;s#h<7W>zPFVFoU_diP>(01utjd|5JL6E*t#y^{loh|8jr}qm*m#@>`IGM7t2G1W ze9?&hiJWMq6=W8|d7HD0v2%1Tcv}J`{=#hLdxEV3k)zCk8+qr9G9j#BQ}~?5+;YL$ z6a%_5!B~3^KnZDHBSdcp)~prDtn zyBPeV0GBT$c!Jp0QDuPt-hMKfwG^N<8$oL}FB@1Z3`Onv8m;1j-|E;-G=2{1H{n`x ziW$I@28=^rAo_;KB&U}(<9Uqg@jTM)@jR)*aez^!4iQ-6;XrpyfV*iIBqEp^>mg&( ze|Qi|a8S{C+cHQl3O67p!C+EwDk-5aKFcB3)XIGttRQK8NN7?cmP0=mn;VcJEqB|vbvIV8!3e<3Ot@bKysGn&JY zG1d^uv;(JdniCvpP_WEiU~RX{EClH}{JFa2DlY*K*o9;hy`KsLV7c;z^Ms8`2qJ47 zP{+9cd``M8kN1(t*jdn?nG?vp2{%L=&IZZ8RQ9|PwdZN6`uGjBN!c?fRw}D=BtGe^ zb<~x0QqxtBI{7Io4OHLblt6FowD;GZBy*jJk2-yzJU`lZ*>x!VblcCALE$@L3=}gc zwBJB@DRgi||CC9Q+8{E2Oh$InwRKX@sGdOiB3YOd{j)IHUu#4riDbNQ?7Vk)`=}b;nWkE?Vg}e<~J2K;^OzEf#CQE00 z*7H9KHAvc9A=#hyO;`!Vycc#rOC8ZaWG>_^@(ZZaiFmJ*hf|qA8SjPVPC9ckV_e4Y zC$##1(HkAb;b&ETkZsf`6wijI;G|aK8|-=JY{fn~ z?Jtx&TX9-W+k$Ge<<+qjr{uKls75=Ug(#yEABCv$%^%$^*WSYqF9$bv{rkkibx4%) z1W$}WC0p?#U>}APLol4wt9W76!s$Z zZaHBeifYZPUepD6)%>pI4su|e zsHaE`kyqI9!Z-?wNbN2s>_K7ed6=S6EIYLhuRDO^Tk;qMqvawsL!`cer+ZbUiqz?H zLLiE7$5XNuS#s?rl-{LkkgW)iYj>b93wXh7MW|d`i`VT&wR>(mQ8Zd2QZJGZ?L?(7 z;2mM7_IrYUq?2N{B30D>0zc_em036%B~oX|FXfNgvs2;np)i!vhBwPm6otJO`7Esx zOB4yhQFD&MRZcsELRj&rPS;LQ_!@8{-ms38F&LgPc%u##7v1o9;HLDTgG9QETU1?h z{r>QS?Yif+`%BZqFK)hf{{8Dmt@|cgtM3xI<$Gn@0*Hf`TH4mV%?!nmQaZ5XZokc zB+N@*RL3dg7a!VK>x<%7@EYn^wEGWr+o8{ zgTu7^p;l8ERj~{H8^YC40a%S6}Ty%KG91pUL+w$uar%zbgzkhS!INkc}$} zIzKOSY5(}IsNag-sJ89C!MCzX4*BQ6vha$96Q_3Pr#?QqK2zK3^nLH%5q?_f>8o!Z zs$R5+LZ5z(kN&>%>bHxpEP{jv-S(=SChIfZ(*K>>H~dNfiTZ|>k2R;fwYeLrt#;h| zI1Tf=WdGu-SH+VdVdLrdSAALIy|iWJFQ+{=N`H?Jr);OzXs7jvV^-w}Z-r40yNXw^8-Bj)xe>qRi$Tf}YyIgjKQbZ>0vbm? z*N`QF?%U12KJ>UvtFq%JY9Bc8`5B&S%s)r~2N;M7Fc}QE!r; zbT2=@t@sTmot9p6m{ISm^%PXs0ik&yKHX$R1R##oEcfEdg zX7|#vi0+Dz+0Tvh^hw=q+d4|t42MYkAFFLJPm4q(wePPMS!*z+OIb_+FPpBxTz~hgVHp(OVqwFuS-{#QIlJ?)kGb+yQ*hF z;n?Aav!=tpt(NCUB>~{N~5BfLt!I4wm*F41|x4+(uUr@BOX!A4s zb6;`M%~9sv)lJtCbCb)fKj5oF@(V`@oVg!@TD*@TzP?-as9|x{m2dA)XXp%UFFxDZ z(kCxjGQsKUTK1|eKY00Tn*+wtAL7~ay0s5Oo*nv1Z8|h?d`aMuGVIifvzU|n=5I}H zx_PFX=glkiEGk7vk5ohqMELDpSz4=H;C>>!nsM|l?NZ^%mFH=$GahwELX7uUyu7yK zL0hQ#&A)v9nOrAtMIDS=<`cJVz^we{jpZ!&dzl@-Lj1i~TxI)XqJ2AG6X=`n9UwfF zhhJ2`{Ck{}{U_*N?znXUY4-DpYsV7MR9Hul+aPlQ`QoF^MBFo*LesOUH&=Cm<8Ep{ zl+0Az`O2%T4jtHqsJMIL!8h*-`|isjTM9Ya-e-$$8vCp3*?=M*4J} z?FGVJ8mc3*0{3-s=yl%ycCTv>9w$ufTc=A2PrE7_H~i;*=F6o08)nl!*eurUby98% z*=Oo78+DyOzG~~9!*4$3Sbedbu0DEy#Mtn%$+oq_*KdyGt?^#cn7Nxy}K>s>5b6^ z_CH3e+SRVL@_!c^zR$0ImU{3^H=F18esSTs1K(x;p1;7ZGh#oEb&qnqH(XNjYX8HD zEID*ZKy$F!h5U44Lb2=Kea_wXn|O8UnZ+)W3cI;4ZoRqk77?&MdVQ=!)A-5Te#MBq zFyg^hQ}}~d+e0qhIQPjupa1;NjlkJ|_y5jK>Ns3rePn)%^B!bOTk*trJ2PQ$8poTVvA?*@6IM+dA{0#Z5J=~25vhOW?gCjM6rGG3LLKO+MYT= z+0yeh(P<^ix2An6>OOkw?zIX*Qqz5Tl)LT3nH^Pk+=s3u&uW+W9Y(c9MlKJ1^Neu( z!J!0AgMal6m1m&m+I9OLWOgiHhfJEIq$!>)o2_1!bYOn$HfDcaJ@MS90PG z;1=W+T7vVv8dYJGHTt;o+mp9n1oozTBn9a%9Lh`~F*a29XQq$2jI;g0g0QsraOJkQ zbf-^h$Low!qQX;;T!!B*jX7h5mG(VGiLpOqZ^_4#1k zveCvGw|~kp;~z65l=!$OY9Ei87uCU`HfAYLPoItPqBc5C*DRZg{2gPoUB0cOZ(jLgI4(4BX`@(pI!Zq-vgreoJnmwI;J=3rPRN@HA#*MsQOS~bd(`^ zVCdjaHDnNumn3G;7y)0=R3S0?+Fz-o53SFBlc%9}KIS`52k+%x;>xPizuXQ<_hCQ% z>#;*~C^Ga`)YqFTVd=-T1jAa{IjMp7#4!{wa?U zM^tdGJczJZe%z)@Jb%xdg|@d9RnV;JdQh=9PFC$@!XgDiay)?xrt5d*-4kC`9WUL70Rfs>Ze72KUmfkLqG8OzWJTR zlX>)2>CgO@E-d}>Z_V_oXP;~>?)l9wjQIB9OfoT2dA!c+i~L~p8OWNL>lYCp{H<$t z5B^=E+aLFSC$0QqSxnmUae{;W$m!?Ejs9*k>#GZkt~eL;yry(}TUxJNd=yP0N-D#J zrxbZl)lNRAN6y^gpTa3Z9vv^gC_EPZDFTMnSzu|sCm+{a-!=Yv%J+@Gr-<}(M)!yL z*R^xs?ng*l-hEkU`}-vla_^IM#?W}n;;7`{*7PqUsXm+^DC@?(%QLNLZOywGodym^eD`~d_;l0m7mDGd;uo)HuiNS8F7kXfIFXYhxcou-b|DXb-}&%< z#jdw^) z?_(V`IX}+#ZOyT{zT`(*<+eM`gqNQSk8>j#e+6wFO&@T6)%U;SeHoFx8?YU?YE_@Z< z9?wLq{VSlQ*zTW?ua~|b^t^P>&prIhvkOU2zh9VLZh>~oaSy+@LU zRaUQeZjw)K*?d)Td9Cxu%K53wUCEq7n|4(WHg@~r<^&umD4e)AHc$CoQ0JKZ<=f(0 zbq+~YuEqyTN_e)VH{w2^I$UrT65qIYlh2NJ?Fl;gdL40Q(Qk+If4L_NcHGKs`St_y zpk=Fap&1{x)w$kwclUJn*}ao>`xTwzf(yQ9PLw~r>aN*#J`DM%y6ix16@xBu+GBpy zgIZe>w#k3^kzi7^W$~`ClMXj3K7IffTi)bc*qEB>BN4v!5S=&`8kOH~8+)T+r5EC0 z8kQa#jPux7MZ2`zam9YgBLl=WclIEubbo1}=gdTN_YLgDt9g!p;@1>>dYt1!6S^n_ zH>FS0w~p^=lR!RMlUcWt9|h&_Ew_Pwz_)t6T#~osT*d}S-0R}JlS^|xPp~enCSRSG zn;2rhfooB-!qyS8SoaWqAmZtszt&n*U$|cGraASCv**#vwvc)Fq_h*^y>D*qCM}9v zSycYm7V4d7Qt45GD1W{1->XN$&Ugf;zTAcQhhX&ZTu%99$yE8ma(C8a>cxa<1@_?f=KD`O?te*qwXn={ENI8Ihm>FSqIE~2UOibK zQB!%1^Y8>4*pmJ?CuzB+VOs*o1=CgTi~qUYVv@f_lyKJa!l!_Zck35b7so;DhKCQe zpCTrlX}WVozG3+X0ehfr74x`{@wan35P7RCXN?gpRjwC{Lu%OZik7{+;l%pW9};v# z>5q{mM=HwSaykcGl=Js5tT5|{TXz&%-{0=1z2_IR4!bGF!&`iMI(u0WJ9G8wD9WiM zw{0a0uRhT|eDx{EXm-x$aA^2WnzW-TuqHXgxS}NRUbFVzrDEP$?edO@CePvgfTbrJ zo+is2e5zJHIQ&GP*AewKx#QZJJ<>zJPudW1d#~UMu76x1aXqs7S?H#6#bcD`J?<0c z#=P07@Za@sh7aW3KKt~;SSwBR`49HH=Gm2m^X^BtoM~td=p$V36+5v`teQo9Gjj{y z|0vdPP1J+-4%eE2V|PiLuI#y$BMsb>u`jU1ICNOIoqy-(fK6ia=hUlEi{EDxO7GlW z&}UzN{S>a-Q#zN&gPvI;-MYZ7bk#|*Q9)Ee2Y3&B$K90qvho>fXJUnzF=$7k``X%38x30_hkdt@0DZnAT z`Sz_(2Pl94Qq4c_V!LN+t;_a@JBtf<-J-Sb3l96FUH%uX`@dc0mz-X4jFtwS zc~6~)c0KR!a3^}pw9tB)qx+@Xo7diQE!u5pc5?HwZ^s?gn(VsJz z_IptQlCosS4?k~U?XEvsj;(!KnbL*`J-;bgcMW19ZdjdMVY}{BztyhwhyU4nWaX>( zPp=*Q-{v%rz*lp+yk@_Ct{z?dy4k#g_#rb>-uKssmDg)`4{>Za!U9Vl?idSS_Uv%) zirIUWlHSDA{&9K1`w9*JTIihgqru}r7^E&V@hW3uICPM?sIy{u`I^uL_h7c)s&9Vk zUgLdz|J?7McRVhZbb1w?SXGfYW?p@N-SqVH+5bIlr51g+YCpIv;?&ONUpGi3rl~;vGd6T$Ljg!!%T6ebXj3@`QVrxoDa6) z&E$h^W7GLyTia^OJJ`_+%G|=5`Q{;G%U7hdR<PmTp%Y%Cq~|hy;ZI5s>q*u&x;J3`|CVCYJ1h_R_JmGz1cUmkl2kyX4~Pa zVN{#U+Oo=PBhitx80ANma^q+f#?3xEWqx;Mh3<#)vp8xZ*EX&W*E*ABd`_`f zT@j-Vq+MtyyRn672S#0@-u0@zqbSHK_;T;DuqS;mLnoI{xmp+#_^zS+CJgD>OG_x< zLMov;-eI1P!G?DpPt&oTdrp-`d(RQn>m=PyM~uO_6^+H% z%4|zVE4`ZEHSn;B6%BIM$D-KaHGNTV?Sovrjk_Z6oX|#Oo89u(y_O;5Jqd<1=yliq zhR51?G%Q*G?L12mP0T{b$Y~ek?5lKvZt^f~>I1F9L0e~>a*LBX-cn93_w+1bATOSg zr$j@wox^Vjt&;Vo!EG4yQ!cg0<`}8l>_-iGnUJiwVTBGgs)VPrR#jT6k$X?$?2s{% zC*XrFuSHke#tB%t95%p=sylqhg3`9)b?EKhu=VmxNoBn%I;9=M_Y85CYvMHeJ&~?usw~skvtBq)XAbv4X>PfO)f8|s4{G1m^K~aRtY!9*tW7%Q9t%n zbZltF*@{Ur6RAkjJqtCj=LqGCjry^bP>houRZ`wM4Z?Zr z-HG5l&;7-XlPz#e*_oCijVBX*w^?tCAw<(?&ka_w<^Gh}sA4&-!&8@rAErrTwJceB z5{gV=R;P~hiB=cmWgJ^p;Y-XcZ~I}?ZKDk_JWh(q2EV$}cWxk(n=oRyBsXygUC`2% zIa|Z+@dmp_q@j#0BPeA}>d_cWU3M8Xl1q)GHZoW3ttwAwM-m?y3+R-D5vEvJ(ao4s z_F^MprP^4h#T7ao=dOtuF2O_BM70%k;E>U^b%mB`$0ZqLL*WLME{r=Y-n45S(LGCd zI^N!AEErZ>tBKTPLU%>Sx@mQN`Q2-6K4XiH!ki8; z7uVHmpy%~NGxXYQoYP6AcSH=(Z51{{P#bFn3f*`zVT}M(1DvMIvNOnh(qvKse`8At zfi*!BTTBrh(Jcb;BHOm%F8AqoUC-!`UV@>UeXB{a|jiWWbAR9zizbGxVBXkT6x{967y_O zQZAeSc;Bji0~k#)7p)%6xEf-$3)m5>TMXhiY~S^E_PGm z{LJH89_wAs*VM_;0o{ERuF+hgX~`%G+&K{~Dc$`+SItrPBz_mm2^wnSS9?`wQlTOe zuiNv|&1^4e02 zK_%Wxxk_zTQT!U!=59&u>!E`eq(US}%_8k#{@LX5)p+ zTcthgrS)QYqEf6(Rz$F^U6Q1mhJrRw`Oxy>ZZmpiDPimj?*sALBh}v1D znH65^u=XwPN-@MM&Nv-5+*Ri|5cX|bave!Dgfz=SrASRMd5z>vb##0vao%b0X*J^* z484o8t}<%3v+iq|-z7?0Kak0-*KzY>UT3}1Zm0!GYsa>=S}4~oaL$X={1#5XB*t;a zzU^_$jZVXpM}e|qlo6rv@kGLKiic@$>s05h18`%3<%VHXuZP;n6L)5<-JWJy)<-D= z=LD3qH9plL=fslfLsI4(VoC{BrYgm1mc3L@+Bm{9d> zuqoavANNU9jL?^9GU5VJ?V1FysA4LqGB(X-qo?t zzIe0bcT^&E5`spP<7;J=2h++j?Rdg{zdKXZ;el!p$mzUItDtd`QmM`8wA;&kLm!NbbLMjMk5-Rfa%_kvki9e7rFU`knu?1yTxl};|vM)3V4j5`kW)D<6F2#nq}CK{jhfmxbTKC2%&KB*XfUB}-bsWeO%acl)AC6+RV{zTlT|yOezlx6=dR3f@LZXkjLOPZ zQ`@et(l#kfi0OAhWbFM2opq1AEiuEMoO?H8Y)!R0*6fLST4B?k7~|5OJVdO6NoOSO zMm+6Zs~oj08~sRA5YJ^8rAu!HaE0a$1Dd05G`*xDzSjp>T({3pS;uCN2T+3$qXT^oFhgL@8X_So1dI~#RO{*;HBY2`2i+kL<_{yF@ ztJ_@@&Hn%BtJ3P4%E;u6$n0M)?Z<>bVOLrVWKWlpt+eNAVY-NT6ehEx%VvF!G8tDk zQk{>B{Ce4)n{pc>**sZSVe_>&cIS-m!p6Q4IlD5jv({`E^OI}K(yDQPquJ6-I5S{8 zu95pH<$>nm4sEP(BtJH>tkpBgk3<;C`~YRj=Oql~Ae($IM7DmC^!r2+8+=iwZ7DxW zlbw9OnthDPHn(e54nq>xT-v5~u&DuoEbO>k?$+JY?d_u*`t+_z=#>~id zulZ`tYyI~)SeC1I>>#NzFb#)W`HWE5-x$Cwu|10B%N7s&J<;$+iF=>aK?^nG84taw z0i!DgkAM3Dm)^F#lx+K2!T}x=P~iaoi#Fn&Drk7 zZ&~YiW_Kj%l)fJ_{f6W_;+dEy`_6F)M-~W;9Q=75QbR)Gl?#926h`$a3XUzbw$MacsF`I7Q+L#DP zYP(~)KI*#O44r(b%4L=G7SQSnC&DS4!owR`97{%!Snk${3u4AP=REF=b8Oo&P!(yg zLy44-Z-c%dzD>nnky{!z{Fqc)VZmTurbhY%?tCFp$dxfpgA8lqi7KzHWu$@akZRf8 z$+HvkNn`dWThg2XFLS0l+ zHKEib3US_)xn^8FCIVqb`eP~bSqeHM)~hD8scvPugL|O0CUMfx^~5?&m}+lTM_6fS zz16sdSH40G16FR?B_qT4q!*2XZ}k%8xm`z?xG$bfm5K*&Q@P)@jZ?TM1FxN`Cf7@) zb^2^U%Wx#&z_@l`pk-WJo)&~%RB198i^nQ*VNjNYOJCI?S#G+fGdX8~rwtJLV z-cKsd??UU0z1!tHZmtwQD2w`Xuk8D)tro)xuCCX~XcR`_knE~d@G6umZ>ga!mC}Oc zG-N5>TNg6pG_@uBZ%sSZ)@__zPPkH1QC~$Yr}7(JN3IvB*W+|nvxzc&%FByOMd_&D zIN~&#W=AyH^5?VkgX|8nCy$u(B!%-HQWwZ&&ZKATTFj4Gg<)&^6nU=PzQ)?xyk&cW za;xp&b87TE>y4LkcV4`luwfii3jCHLT1fP`c}l356IteU>cbmiYyYhQ_N4swa?lgQ z!~MH332AISY|`4pSVc}~@{hW_wM1TA5^n?HudB;JiQEV#uiZhVN&mQ$bCc7+4?UUV zfIQA+;6Bxs*I5-kHe;g%eZNQipLy48$Lpz0bJ7WO9|b-VZ{6&Z`8m(@c$pJ|(mCxT z7R1$UI;mq%C)%p5vEs{O?})q(LTG4pr7b+5E5|GyZ>dO_Fg^!(fdQmNmjlvLS=w=D ziZ9%Q!j3ihp;IxgO!ufzNYB;cjx{zUO8rnT)zbdkJDYh_Z+t;m9vg~xLS=1=8T2je z`LR3-u)H0sBr&Wq*J>?OP)oILY;3fRk{L z-Y{1?ceL?DY>Lyo`DKH!CRR(vCX{pfO5M%I+qoq%nF`r%ZbKsO=r&|Y-u0QQFmkw+S_{J`!(&m>t-u%tEIlC z6gOHKOM}VR(0qP^E%);~DpkH`Qhz4NYMRYbtX~{0Z?#%Kntn(n%KmDCdSQ0EyorBP zCtS>TP@y1QP-8K@$tl(1pqDq^O2#&+W)&BnrXAo(yghl&V}k%BOZGVD8$Q(#9|tAR z@dd&S+;l5iqigN;(ldu;DD<*xkqyI<>WCGU^KaYXfY%N$EG@`^3r>sJkqbx*6_u@|MS&dXZM|5G!~TV&gzd)G z62gqf7SdsChb&Zjhl%WMZ@GYsO{Thb0ckYQ#jT~-Xrn7S>{~52GCUsLi|NCat)$ba z<;sPmPnt_xOP6HKwvs+OjLX`#TDK+*H!i4t;ZBANNfkFfTS*gH&*flmt0}TWyKuqn zjSk0_Ve*?Lrb{ml==?NRzRG z=@2gkY%yB$72UajLO-D#xZvJK?8pW5HaZtjZ@BAL-+E$BCD*@-p8oCfRtxr3*>1EZ4T!HBOD?R^HOIXlu-* zu88C>maev&EnF!z=zylNDwN+z6YjCTB5!(XwySke<_3?^wOQVh%b)AwBV*1-SS=HF z2SVOqlX{O$yf0y^)BeRmL3v&B=p(T4y~XjKhx41qFVZJvW$w?-9-yPK7vj{8wI#aF zKwgKU9no^`=XT_wL*m{{9WUGtXWTN{h2oSAu=Un|DD6mnD? zdE3+wH*T0=aLy#j)zeP58>F8|!ikVg97)3WMq`(<&mv)IT^r3lgv4JNi%o8D`l#Tz zyV+w$`2qF@oHugPZVRW4GF;f)S)-j9-fwu)Xlz%y&Hs*bMq^IiZuyjvlPBTk&KMQk zsGZLVqtT?@Eu1fMQy|>x=_0pqZ02myj&wVNPZo`5SlZONqUHIrd1c>vVN4^bhbs|;l2oX2OVMvn>q=djVW#`&hrQ5MCG3JZ3 zQ$E4QP3;(NP@&7Jt#4AM?Tw(4Mqdk@E>6#?llp`9M_9cW?HR7X{vY(M;8a1i;h0|P z`O&zj6?d*3a8r@4&6qE%W5fpxn}xBAl4v<|Reszqm+9Jky0&1J*;q%fUfz)~+}FPj zgS~fCW+HY>sH@HKEjZDvTU#>>lXRJ*e}p#hrtZPRC2;Z^aiWK|fpNL@kv}BL&c{1P zo23QoL!#wE#kwq(dk1R2kUTA5SkBtlC;gMbOZwMtH|_6%&dc3EY+Q6CBii(vutwo3 zR2-Z6(M1ubCCBW-z%QfjDhq$uN=YW#yqRY0bdzOKqz9NBjZG%6l0qvlXcejEn-LX4fP(+3&jU8Wd8`JJGjx%R>s!au|lR}`tIfP-L~a8slNMg{(<9tTx0rM zM1x{!*qThWH7GhpF*hs^47mBOiLJz+z>L4&n>1;El(T4jl`}I@#Y-u9M~|6O^v=o` zX4OXxmgLtUk|xFmnt0Oe{(L^o#Q%0_FHWw_s0$2->3d~#$o}Wi{&AOY*e#$Ah5hy~ zm|C-7Z>pZU)&`s}%5Jxe%XamK-%V7j`JHQ5dA+X36D-2L;T}gbf?%ic7^~!WbmZC> zjivgn^i*=bM83o1GiqKe%*OmpU>}Xt~FNp3u28*i_Rd)!w&J;IntX8C8?~$& z1$T4S3zqe2z=x9LyvU{;;8QeX`r7BTl-b}Qo)d+ho+BM}5-s_bRBuSp&#Pxn!q$=@ zhquP9qRAPejBdATHPR3JrS?p9pOk;lX-)fhD!a&|Qd+*HwHB|J5h1H&qY#;vnvq$u z%cGSxJ?&&=R@ZB6;Gvr?$>uC&A%w@U`7wFmZ&s#u!=BzU^gP*Kq#CfyWyx7Th>c1^ zMVa*W3dGo0!ohv2UZgl`8RzwkfyO5iu209+W+Myi{$LAvGeOcKXrFg(wkF_HKq2j zU!pn`sXZ0Ushw|6#kIDno{C5=(Y$tB^pdI5-N}Q_`qnZ}xmv}7h8oEpPsYI_qy06y zJLdA%m>h_A)I#atflo0%u~zIf7FT3jFiTfj|F32ArI&8!YXr8h_(9n2QV(Q%u2yef z`oCD(5CT!)_?IYiG6b@uO&80mL&AVHVumpOJOkeXuY_tPSql&MszFTsj?ML`moH4l z9$_ub70bZXYqp?XtJ{L^N-fPXS1j$O8+FCI7icgn|Mu(cfPaMj8vl^0T1Wn%K~EU* z-SV(qo3tYD8lfayN9nh#t;%7)L049>FVc(zKdc-Rg2%e71mnC#miG ziXYLZDotNuGt`cDyj59xq}mA(N%}1(;3P2|yr$>q>fLs&I%(s$Hfv}#j?2Kv&n!@^ zJ3gm4e^NpG;LDjbHu{_abn{27m%rAlt-e?3c08M6H|hI*HA)qod`c3-QTg2jK|1l3 zgOpzV9pgIx;)u&zNaBR`LpxoKQXra6|CnbL1o>&t*s|z56>1&%v*oE@H6io$&la68 zXf(Wrp))0`W@%Yfp{Asw!ds(p1)U=m?Y`k}RrI-hBW380aI-9`R4~nuxS@Xr-NOa> z+q}v+!?=;^c~@uHprq2pf{a+8ONPuSAc94Qf$boP z>Y|F2sG3yOv_ndT-{NnFl?N6VmQ@*ZoQt~B*lM@!^^TgU>x$)tN{&ps%jFeeR$f;^ z(OkB;v9&E5nxt=rGLEBD68#Qs_w?usaBa`>%!$4k+okgKRbnhv7}qAM>qj`LsU1vD z3GI-TWkOoWNM1TIEc+_`P#aglRn0-qDKXF|qL(a;N>gi5Hyp0fNI>3`U7>H3q$N>8 z%H2|>q=vB0O@#RC)DHQPn3im>MK3jWu2pZl^{C%+BpgxYTU-^duGYIXKaqI0K)=JZ zL>6`#6UtfIN+4VnjkAbt1YJOyuMfnA?f|Ho0oARqGUc+hpjS{&eRaOBpN-4do zjRGwa!7Sb)q!c#lemjUQ%RX&OHEJk#x=UbMC5bW(?0NNmFO;uVVSjIN zVP>~(BADu`n&>-&xDrv(!JA6*N>%veg^?;#T^fb-+chy_pch+h-S7588n$%XK{xax z_L1yXTl9_fNPZ$wc18LfO5($^&w&0!t;59@B}0~8+kb?|>ojQ^QZy2Nt0g~tEnnwJ zZ+fy5u_Q(=MlU4@y}lUk^i&7K$A$7nb7IsU2xB*R^XHdw>E#%$EtI#KS~;x ze#0-k8RC8gci(F0fwil0dMJup4w0vkde4a;#Vbk`+8-fjW%=qaL)eio@`RX{W#biD zOB_{K`9>`*r7FRFJ4+-hrzPQTqb1_EfF+{%b*N98u*+$y=PMl(v{IZRQmg5XiuM7d zh$^C&*-RL&+R325>a*?$t+bh$EA1;TIT1I6I(-b86?a1(CzEMdHg=9s^oKp=m)uD_ zC#Us~ByC5%R??qVl|JCus89PzzyjSQsX0y;gSwQO_21S8BFExMv7<6&yE6V7Va#wi`lVxlx$dVY?v( zrW+)%+#rD+4w_7Ii6GF)#_M@-+2L+0s_pU!xI;T%bHjsaYC!yB`m zP?DsHMwvnWFj#q487L$^)ekq{NUQq3c4i?MMV2)W-V8U#s(wQ@mhT%L@$C{9Y zg)v8>(!rG$l2X+~_B-+ng36u)v-pfRz-Zb8#m%Gux8*$Gc5evlf|by>X?)c`?T_Q(*bqZdjN|sf+}S z)1Tf14a41bV_g3*(z)H3&_4`u*sSK`U$&x97&m`AO8OvZ{M%NJO^%ztZRLb&u=uwf zMk7Fql!2MqeR4R|$3?8PY(e76{^FH%KN5e8qi<){nLHZGG^j0M`C8SJY6Zz?M3m`h+?UkLT7Sl zaHFd~DPhJX=+|1j5kZEw=C9HWv1l@lDDY33bufp{>(w5QVYyVK+i|C$Qc?*)kUE-> zJz}Jg5qR3MWu!TOySFDlj~ILB%>Kj*Iia~&DlX8%7_AA>2S9OgYNi4uLb$@Wr6Tna z0yzsu<|=#lpfv2kER)*hug$8A0`hlE<-on$lk(Ey+`_W*CZ=-N%7N+SIZ)g9E9#B8=ranE1J1>#OK zpQ2D7c?VMY3aZ!iJ<}%j({#Tp_9rRjMnSwGbSf#cNhu#RHG{$u>rHFNGOGZx*YUOK zvY_i8N&^WqV-?WRs!aS0dkdl9?t_ z4Y{(0@X&H&Omc}i?JXRqY+8WR!ii9Lt490Otw#mfAQ&>qg>K!Hae?Qn4Cv%!N+V3+ zqgPl!eOa&*%lJEgP{Yv8YHYE^nz_ z$Lvm}TU=#6vXKV$L#zPVy&0$9Xcn(TiI$yDzq8tk4a!86rWDnx;VQJ#hGpIws6{F}+ts>jI}rljOqDmBU8(i- zwlRF5#WuXrRxrY`LOh8|muA>;#JsA=Qd;$}g;iP|u|=+TtLbsEO}nA6?4&EzTgjG2 z^g@KFv^am|71Q%}Y?P3XVsV;<39WW~d8snHw0o~|Lm`!Lnm(d==ti%$3g-+H>h*7k zu)7Lgg-1hU6%uQL7^zE<+#`iP#)>c}%MS0x*}YKNJwLZFJGZb`uY_xX-VUTC+_{Ww zq3RniiE%I16y)SfY5z<)FFUH0yshQlQ?}l*UD&{OA!)lKcSfbVxj0=E-RspA;VBCa z?APjsqsU<=!n&4F-;j~1N~ARVEWB%J^hmXW!aU~ z#A97kU)MKcA*`0AFkvZ7vxM%gu9zgVfaW|zZ&pN(?s->{iKWsJ^z1E*x>B3=w9vF> z3pqfROnrSV848s<#=rsTL~oX$%htV-Lurp(M1G z$)1S%tXz(N$t9+ywZTqw_Ix_gsJ8F)YgET0XUnh@8QXdwHYCP^DCe`U)`qH>iR8jj z#WIyzC^yzkQ|yt{HQ8OMrI;kCYm%1Dk|xzUV;mm2yk^+t$NwLtD}FxG5SoW|m9 zh9*7WSk6t+U|ly$lj;vvdNVZX{xlD(%urzm!0sY z8OIuGyjpCDjBGQc_O2aEWo^;1H0HW+ieiA9=DKc*!nWD4s)t&|RjJu!`Uz2{XsG-t?HZWlrm{10cl_&XA8Icg1iPyydhB-pRq+TQgm?j0r zCgT+|fMJp_HW@FU0SuFbMr4inXf&g3#>*T?4_GD(ZIhW}eLZ=yDYvv~b2dr~)r?fq z-O!p&(7NQ@nWHl2&$KFQ4$ZM^%um@;)NP3MCnhXzbVaYug9t}4FqLKbCd$TnB%Cmv zOgJG-CY*9NE#b7E<~)jPBAMYU9=l~m*ov!HH)gov_&5qgz>pyHhQtnfs=^4kp*Lo& z9L=CfEyGwc+R`uCPnMf*`D4kJox`HwZ%rGggq!J6Y!Z1#M^f-mFFK_{hAm7 z^0`R*O=H5czseUUNzrb3B2aXHMF+WjO)pwL+)bCx)8^2W&hn*e)5Y?BYWP!?rB>C) zWhYsdXLir+}e<8iZVyt z&yp4FrQLgrov6o$3$jd<#5j+NjDM`kyOT3bPR|ZdF7_iig}6~jGIKx}S&|b|16`6^ z9Yn;JHqe)yqkj$l>h7g$iyd#?Z`WjvpsFl>M2EYRqd7jiTQ>+w~J@_pf~QOLxSCbc1} zA=Sn&U0f)-EkDoi_amBUOsb8R{nE7=y7yBAuT1Uk9&UD#M>=Qa1cPh~7gD9gROxK0 zw3I5nEZv0z-A3tv-|2@kEiz18HqgXnb3BNe)Zo&V#PLq-0ZxcBq$$>85e+9z znlMNXr!1tF&lXU5F$FZXvjh`XGU-toi?Z5a96wf4d_j;`x>l(VkICIiy;Tjv)RlSo znZER$qKsFo6Ut)=rP2B2KuLk|lZ~y)w#*5u+fQbrDO0U_)hBMOs9RoL&1kQZN~4M= zlx8hYD9ws)E1kA0sm%1#X~7c8Oh4UrEUC=$({0TX%1l3vHZ74jX2X-Pgmx?uyFr#z zYnBLJs-@J{CS9wdIM^;`%m%tJ$c)|MZS0O?Mzwnhp_L?w)-O$ytw?vrl2S|P2qtWS zY`9#BocM5MBDSuK8b(((;ijo>9|R{ zbH;jhrN=jwQs>rO<#yXd?O;l?sB}ZPWQexkZO>KZdT!Jk@*P8QF!-XO@ zUZ&w4AL;fBbytzA)C+!6m|~sJ9J`9uEQMVD6N%Fh&aVVRPaGADIC1>QYfEv>@4BLu zfI{BC9R$0d66QXI&g=AteXW%WL>#{2!sQ2wOVi86SuSv{P+dh6bzN1`tT3~CK}1x% z@uK0!6rqq&mD_H4X0hcT^BOdvO2MW3&}A(478gV$=I5>|&Mwa#D55T;>!@alYAa9k zW*NxPR2x@|^zjRg+d~CY!Eg#Ii(Ls-IbntF3LbW_DGYI#=N6WWdy7lFE8ddG!wD{Z4(589C%&Ls5+`eHH_ZDy@*Nwz{=y3kkt#E^?UO;zQ404us5I z-Imb1-e`>&^-rS}k!Z9bPWM6g84NUgCQ_gl`s1FBlDME~2SuG@ z-(-TUOx_OCg90IwI4SVDsv%|- zc0!vCp3$o^hx(=4>^o+Fzf;mPm8H}B@<~Gh$ON&B_#st8iY1z$%7ZB5$Yfa0qvyn`EFz> z#=d)b5uI(j)1}KTV+ZQp7N2BQp_MU_-Mp49Y-=ch~J9Eo3n4@q#(Z zHfJ#`adJFv%tn}EE)tEzG?#WaeNrlRO-USw%&A}P)}Z;|*hNF=IBG6oXh#RFYs8Kk zLdQ|+PbM5Sg^s{*)DSw3>IsH~%Lz@I)bPXAI36wU$S1roWHy_FNiB0r!1P4IK|^M< zSq`+!aS+qm2}d1yQf4@A%M%$m2Uqj78Gfl{(U#xk4D>ZoOogA13>&l_$r z>Ys)S65YxmM)YPqkQk5kjnYq;Rb|sEYhvCorGgp`&d!xf^V3%@7Y{D8YnH$OcE=Q$ zjw}^tmKT?s`H+mi6q&_^+0N}a7>kSZ#pwkjxe}5+35k`R3$w*NDFNf)(!rq&Rn96X3A0Md zeXV}fx;^x@WH8JX;7AO%x_;fmQ75{hPHMtTP0w55+wJ9@sU6Q1i(#ZZa3i-V^W8T* z4(aGal?sinK4Dm0H?im8%#IE!Up^K@DkzO^rP`?F;U)`+FrV=jimc}iK49zg|*YAqV^}EB)Sqg#7 z*4=Z9xcEidVUs<`!|-UaIK^08R2rLru63ihg=sgvz+kolrJ<28YdVztu2;2!RnkZ6uOk;E5%E zw-N?k-LJNE?AR1J7s~Tl6m-6|)9}(X)^KHp$FX~cZFVjp#!f|^?Uv@~wyX|)zuO47 z@ij-Co?eHq`B%woOAcEkUplRB2tV-Ul{gv@P#hw!qw-T_Vuz39QO^1uGpuy4ioA+$ zX^EG~D_*ZB@0wwfyXRC%l5Z9_)vSDwUuKeps@7jw5#iw#4V9$EO)e_s;xaBNF31vP ztZ}inCDgZOTT&wjwJc5|EsEwXmLgA5J1I#b`zR{KYzH9_ zEeI8^eorJDF|w6@a&FQu&P3_4%)7RlCT#18Qu>AanpTFrxno6b>!5^PX{0o5@?&AD zQHp<2BO15i^7o2vBv(}XDSmK*h##CDD-BK#@mHk=r3ASgP6e{esPN80_IiKHc;!A$wukpcNF=UiK#a|_@hWe*$5aM6l?n`^^fIk~4 z{z}8bRo?P9jIz{U5yhxK?B-O4fV1H8J(UB+1L$J|(8UJPz)fD?tpuyR?qQpF>t-zD zw}N|6jUPy(M2OG!^kYU5k_v7+n&RIiFQyOb``QhBJ*%& z_io%(EB7G)epTphpa!$v*BoaQe)Aj({p3Lq{gPFJe$d@Z{YkZVj_)Z| zPD)*V#{gwvJX^_w^w^L1H?AI5DASTrw*oS>z8+r~p|N~7KwjG{ZH4t}PmDgAB5Cc= z=;JSzP@Xo>`&BJft=nz6ru_zArNXxh=>(3a6_kDU1fJ*tv%=HoYKA%L?G zz*)%9s)n{bU1-WYv1KrOkyaA5mjjgPWf-kP=cq2nwnXQtY+BMarX6aUGPJh)2{VD` ztzpxUym8z#7#Cd zDucRy(LQskUHy(vbMZs2GCCBp+Nj*7P2n-oXUHjawYDA>M3!5|$CRX|9pEZP64`{o zNv7CIXk#a}advVDotApDD*oS|iypbFAYOBs~fejqWh-X@mQcT%lf}fY_uR&)e za8}}al}^7k?5!0l5W1Mq_5~8<3dp3rXANkZ+RfhmnBiA5X9{W&dhu6rMpVQwJgn(lhKWu&X? zd@_?Z(x-c$o{IxaARA2+8TquKf%FhrIk{1)ZZg3KChdS!+hl?cOv{1EI4FTOn|NvI zmXl?>HuO_hDipfCgQU{)+Fj8L)!htJCrAA9vt_o8I$bu7uQg*!pZe6gG6XG~#Ya!# zEBpB?K(WyH3XT%$PYl>*WnLyt>yHQ6rb!%|9>&n4+gaI8>odb%72qJq++UbkT-d{l z0wl$ill(mzLm&_TN~2dy&tHx`?`)UX5ZD<1X1Uz*;??Z2$r;%N)Q(Y#ioY6jr>jM+rWrALpG2c8_raxT1|);I}5+A=4Pa7&#$WR1HxhEW)E^`QS~x;A~-qXHuw)<$3<_Zu{OEBXO2 zJmi2P5PSx&OXvLMXQizV4yFqIkV_lwuk%H-B)y2|NvdI`8q&9^%JW|JXa!$}7AYb6 zJd_ezzy=bEiu{HLT%+q0yJ`!$N=r*vT-ZCeP(%g82dBNxs^5`cKW1gmTNPD}=1Zwq zcj;5I##GxgGy-_rN7bKo@+FRJ?GOzVVd2CvpW0G zY>6MM`rCg3LKCDWxMnrhD2TtgL3h@R9P5ShD{rPsxuQNqs=biAGbtqfHojXZK~{+^b8ACKFzA{wa)5qvCbObVx6_Vz^aMcP3uFenzSynQ7unfZKCrfO_r4B_Qv5N zh{Gm&5d0ZZ`a*^L3@P_@lu~_$6v%B;7=OWLR)5EI*7#`Ytp56F!ssziN8N3@-A}0+ z#$thprf4^k4LODX_Ru`)VCWJ-2`(#QjHr7oKm*F(r`36;TTn-y(%hP+9XU*!UtqP2 zn+n-Ja{$piD}bgXgCf5h1+^hjX4pl`kRgI8>m{IZ2u+Ar7wrZM2b)S_Y1B>1+SMaG(Ho0U z(W#tkA(!(Qhmw17*4;kx&CWFW@w}AA%GMZ-8x*MxnWK}REU0iNMyN`Qa|_G-i9^~8 z?D%0C=1g>3G+P2QM#x~skaXFM*?M+u0e{nM51p7EO6lnJM8W6{dmpC^0T+m+C4Tes zrhNUAC-_3gK&2(Wf0>k_+aQk;(%vaJXR~435xUmpPBhnzSm-+BXbji*;DO;1X1t@O zJT^oLX~ah!+*Iz!c^%7=VHyV-j)w9As@Qq9tmZsjNu|Rw;dqKkKE9fwpee=_v3X`w z3`bIgv{l{`&?jQ$)TuPux6+vQH?b#i^4YQ#7RsY~)!rIh=>~Efqy=)Wh)WuRi&7+y zCtDI}RGzCzNTo*!sr;U&B`!~2QLL1grc@9^OJMkNJfq^P4eP0LXVU>6_T}-CA z2bE&pRvaiU?^~SZAvb>aJ4D0Cp7lUYE>WL!*n?9eAbMhHRto8(w0wsK&j^Tav6`sY zX0DK3sOtVQjnKp0TFbpTeW$wj0#^_t?m7pFSsdR%dClYwrg?>FSQnN%| zNiOq>Q0f(`*&r?O{~vSj*49RjJPzl>yw|(fKn|A#oC&`W@M4mk*_(dac4PNoyFJ}) zLgv%|RR>8`lG-+ed10T~U{#e$CAAKvQmF)fCHY?)NrW@LPr;$L(ta&wSvC!<5J@EO zd~A};>Au*hNKpI*pR;ItFbV}IZ_qlF0tx^cv^32u=7sFfQtZ#dtOqeD%!=GdiH{jZ zaO9bSCU{W!pQGYG>(ay4g9F&)2RXg@ouV>WqnI<)I`k|;+haDoc(Pb{L#;ULig9?< z7iT7yw-FU&Ut8iQ$G>Y%?PWIw!iXk+PwQ+-lD#$wRAHxZyk5v{kq* z^t(9_*Gx~ZpwUnYJvBie!|4g|YiKT*N%e%qGrU@(&b2(8Xx4?aN70h}C7#pGm0 zr(a|1SVKbR*%}rMoHj2WDYiC2WfPc3jH3GFSlj6PKpZWK(w7 z*tG3@jvWq9^8*2{O09HV$zt2lhm@ZR1gnFB$sq$i--j{S^ZQkYV zw%46EjYU5P$9j>Ei?DrXDT7VzKgYLyhmIFYO%8GGOxzznqSav*E`tXP^4n=dUNH z2~alreA0XK8eVslZ#%kfOy?cZRLx*JNr!OulCYnjV-C6R&FQH|Pg7{%WC30^(g2Z% zAu`vrvIRqGyb|!KhfgbaoJs&V*<7_>DWKRJcG-F9t?`32{b3z(+oW`OV8&Nc%fs}v=p%1p$@a<29`%K1a-L^ z&GWjLez90v1)%uX-PNkVzy&YSXBKcl=ihJe&kOe&mI1rXf@2#mY*b?aY?k;KLW3y? z{#q9Ig-THS8$h{gu!oqR(}J#Iz|RUT&oy0QYT6pFZM}TFU>_^Enc~HhhwvtfPY2hu zV1jN0pe~r+-fk(DsES!U)EGh**|ZBaCje7<>!hs~)eqQu2!cu86pAZwaX|D(SaBFq z3bF2zFVMrQ@t^;GWBz&JZ;TSqeXWoGN49AALCt7>%-mz1n6z(&!f%tckAC##htDt> zHw1-5v!dTL+ksD*CtNUYQGo-ELbVUA=K%@y%!h=5lFOM*C`_~RzZVb<2*Zrqmso$4 z?Hq$4L0}|bg%)Cuq2wHt9IAt!A5H|fV>ZK6zV(v0N*JO2T|y*8o&dui2%^>n82v(^ zZ!tp2G!nRGBfLo7U7nvc=c}vpv#T1z_c34uEWr0ggD*@@3BAJBLT?P4@Xj(A_0tA7 z7fv#8qag_ma*lu6TSNg=)-S|~x`ikx0lwEmUA(js=EZQs31cQUfL7)6Cc#w#$rk?L|J zsAEcg)F5IW!|BygpC?7%MXUH+4*uI&hQW1}6Xr|j(T#F?eUWM_15UM-rj98Cc^!ES zPu3xzk7=EdV3v3n2#&Bo>X0s=@p1{m02oRdQ3g~_o@;$0!{HKeUu1yP@+f2|S7*+S zTuM&wA+4HJqJoy~$d%5atDv)`ajDv+@}=|03S`4?AJN++tQjNXdc`?X)JkrZC51`U zCy^&b??gV^)^ESwoT_r(OUtMo^bMJe2n`WC_)c%pS=SELlWgMS}pr{}}ZgKKO9=38ZRLkXKi$CzV^ z>ooD=0Ije|!>t>McI#eE3%aaGda1CFI{Ef{s?iKMWzW&nF=fzLjy#4Z?KuIxqJ0Sy=G~-u z_wP6O=cT_R$-w@2<~J*rPIE$=68dxHKe9TFg7~lT}&#iJ>9OhEF^K7~U2Nq3k{haaTjceX*$4F~Ytcmx7+Grh_ma zd3H8;_8|7-w@;spmXAkO=xak~RfYfubr&3rw*l^j#{GL5_b;w|lMHU$_xgB~irs24 zy&f0AO&fvW-I9mDN3iprC1r@_64y)gJu4!y4j~;xqDXz2=k6ngfgFAuP=mgf+2y^Q zU08gx=+rrcV{Nj0GAY&^P2VZ{p;GDg6(J&}W4d7pPsD z%M3v6)qp_k#2}ylt>--^<)0k4hg0_Ef~bc*Uti{TG)-qsep`q)rtv+qe3s>HTbJVv z&+Tx*ZX7iR{L%gW)6~xI|?{Fx<}1(!IxgzpQQ3I5b8j_dD&I^)Zar`c_U)GYm%q@aM3K(6!LvFc zbi-i-9N@OB8#5NjN5ere-Nf1X&BGV@4AlZYLG|Lk2q=PVa#odwS63IbMrb(2S`XAJ z47VObO$dv&yBdpn;+};*7-glaDg2|>l=&^pu zESAt`6~L@QEh2@1M}Sp{lGjUo!O79T-@N>5Nd7IP>(e9r2k$h1LT!fDq}mwu3ue-n z7E{Y2=PrjTSCbex1&46cm*Dkeejs<2DQ++F@qM(+S90wvxO8@%3_XibmsYVbuREgB z23dTdOi_Z|#39%Ny35#>$(I7E|8*3c_BskqdmRU-4M5sS2s~!BkJ1ah3vR%ZyMMpI zKQG*cSO)T{zoSK;{_+cYkbq|(%wHOb<_E@X#9wCG!N0W7n;#ISkMBsSG$!nI1M|~f zH*i0*bz2jw5*~gB>LzRW(H|&jj@MCuCBAu0yotev{sOg?O87Zr5yhIW9dkg&HT`|* z2a^I}1J!T(@xDIQ*S0Mh9!;JIPJvKHP*FUQ=?@7Z3@#AF5vMJ z^9~rU{{1FrWA6)helG@gd-PziR%lGXB_<1@5CAVOu?V#1MI;lpPD2bP5W204mp;jJ zQlQvqo4P#tI2yYV8qm2#8Zr7FyD?hrqKEM1;^Yw`o z39^z1qXT@q`Sc#MLNrtOTQn0a<_)~U*k93O^Z+0iBO5N@#ar2m5%dm@RpH{ey|kQO z4ipr9&+IFE&~e49ArdDecVS$CS@Gh-(TU-k+}(0rQM5b_ygz)S6UYLrxun#Z4mQK> zrcoI%=gl9(L1f9LC+;yD<~NBxJfhsLVCo7$?Q;DFE*X_C9g5ou;b(!1tE(^HhJ5F% zga?t0(uY5=viQsV!|_q<2`XNW+Tb;;HyJTCJMy;v>D9cCe`IIp*V*9m=K7Dg^(YG& zYJwb}Pez}+4d*D3%#PNqRBomFVO`TV;JAa=W|6M$K-mG~LW|4`UgS-A5Xt)GKORXD zeLV`ZjSvcm4J{P`>qVN-Be(?q*tXfw<{y3#Y&9h2W`m8_`lb7~6#`obPlWtFkb~{K zg5CPs-T3DXY&IN?vf;_~$=8wF!77+e(*`Yo(J;c#Lh+0qU?DX{2_VE8GC)vE2uN$N zMOg=~fU`X#3czL!XkWu|Y%7mqfA_Uw%|by`Qe*kI57Np|S_00k?!#o&q*<3!MI+T3 z27S9Zl{pdi+u(9=eR6UBzrk6=)vH)HNDozUD!l#4l8}zLrVP8Qn2%LT1 z(-|!Up;CYu0)j6+-Iz6m{O{YB-Vk{)Tx;houDfsmouTnF2Ahfd5{`HiaI@$|a0g(k zi-MV@w$}ks{IZi9)8jvyy}G^99+alCIq{O&r0^;m-h=PV=CD*ZiV4suq;EO_6`=$I z491_XSjS1>lDUvyM=r$Seg(g>Q9)iqXG1ak&@P*115dNUeQ=}h=L^!X{QTsTF)1B4 zN~DjWK!c8y#t@92TPn58+lgfb-Bz$m6d=k)89U@ag|ULyaW$!cVW4gyqy%FfFE`6^ zQKzaS3XR~R1Yb|QbZ>fV)Q$n(Y1&Sd^=6F5BnvoT2yaVlN%Z2Vw6`-nFIV;qTwxlklWAVp*wz`dd} z7T`uvLg=7GLYn!eomM|q8cJpX35gab59dWqEodHVmbO_@%e{}s<%=@c(>B%ttH@E4 z{2EFG)}2UzsG9BsOJLx0TU(C+!}pHr!zLpaGp&n&1W+B_7dFB3&Llsn<}eP8EZ+Y# z3o@qxv5vqL%ggTI~*(+(ii4f}!mibx) zI%~)9l!Fa4Si;J}Dc)MP^-CW&crkphy}AB2x{(`YWiH?)5&{SEPyB=Ll;^Et!5Wk6 z77lt4k;%T`%2#Q;7zDirRJ*or$@SRlszjwHQqNy12VM+m!SS@JTD_{A=BuAF>E}%@ zGkuyUkYdWUdt_IL`9iB{UQb=6sB7FGzJ{IwQn-$Bf0){^38)-b8tg`*2J7va$GLR4 zB{Lws85{h5Gq@a`UtOk*#YwE66g(FCy0Xb?RIrQGNMYsmh*>wo*|t2cBx?N^cUbXa z`02swrMPJ^`?I2keT7nQVK7WKbq(j_6rbFIMG}6R&-tgd94rHXP5N^=w3?K+w{P%F z+OKYMm!y=ZAhdr#phEnoGk4?Yt?CrVB*x`Q>_{x$HksHD1_WSeO>c@iQ3sA~VqXx^ zgle*X*)J!2n{LLW$jvs952CH8%6O380CsKO0LO6ivFD~Urp1Xjcj;?X=G}DTL4L3}L!J z-mp%0zkX@&Juilzw7(Co&p-V^t{K8y#@~oud=Wifk&8~R;Ch@X6j@E+!ARJ;(W`ny8HkgmJYp+*@o)>Bl3ma|ZVl*2R5S*B}a zw4wt&QsS1FR}HP$?AwDM9hd?lSryzDQMR-=fZNte2l)2smQMhmG|QVwS1WfmMyS5W zt-;|HsG8m&92FtA9#Ih~xUvl^3~fr>dSK=a`=|pu|0HIX6a_vh=_sP;{E4Ya_ulk| zB&HFJ%)nc>v{!Rm6xo1&YcesP?G?;$B!)iA<7KDuCP)j7-2*Jf%-pSguU;YGF@P9R zxz!Gm>Rf@3?+(%|sl%ptCQS_XEHE%Jux-JeKphC>;motO6!<{jHUcU;c3hAQG+G)t zKu9oh!1sWreFfcu#0X4+&*W1M2gRzK7N96J z2M2L9P(8)P0u+Vj;vms2a+gy>UG|tbhAPhP_PJ#2fQqjojewG?4TvsyT9D)@$pVn$ zc9yLDS-SQyaSWB5Lpje9a-PL|^I*g;kg`2J1E3fpJNm)GxmzqYS#0dH{=L19rAe9u{9VpPl@Qo|UN6_osLN1)7-1lZjRN9H&72zgTyX`CO#BF}TnN0ct zN`$a4+nu>JP%QCzX@UGYxcPi_76{J3Kz>veO@6t@uzPL+2lH^bxGgf(^Ln`xaQX;b&E34vG;PHK^2$8RXI#N;VgQ?D#^w z_wZ&&g+VOPh7KSFSMcV|gM9G@g$!mQKMOr%ufb-N35${Ys@~clzN!RMQ_9;k7C-?yu8FLh91fGoiI1TEI=e= zy?9^X#~DNXC}O~@Xyw*I?jyqecDS()_qpMQHrQka+sQ~Sebnv6b8y~ux|4X9RSW`# zc?G~er`Ey>&m4Gte+ACA!qZlna012HgH%$f-Tz4{#^|yu`3QQQTPf`Sp((kxI->*o zhIJ(ds@;D=oy~ztNfhP6Iio3OFaAkT2K`A!=f8C?&XJ$bOwn+nU*+$mf1mQV-fcyOA zciSj~?q=Ul{sYRoBg_43@QIJ-Qs?2b8)If)|;*70RaU@iO|)+wAcXdE$5+ z^(r^-Vk?L(EhP1p7a(qSRVbg0R=~j1>s16^c~F5@d@tePE3&6JkV?uPV?m-b;6+R6 zU1fBDaw-8tAfG_70V~{m)`jbIp~!da>zsw_W}h%4OkOF{0SP;cBeMHx;Q`mBRe~}If*_E z=cQr9QQ3$-89}U*rMHyL*pp2Pg*xh*%XtM&Ok&6u#s*Lgj1U`DndJPgwHQG9(&_ znS|?d7xu4>nH6u#qHa3#qc1&vt7ZPuF@BsB@g#sY@_fCv(Jpx;emxlu6V8lluH!4H z4~ZcwL|@!Y95~8s>J4=ooX5ZkoW4IM>H8A^{G2E)xv3G15Wa>IOk9^&XMp3e=a)C!FbVK94TcVz07oF|v@w{vZ48HPpp`d;=Z*~| z5M%8Xv@D6?XDQGcKc3(4O#_~#Lg1>QduRA4E8hSx%vKGZ!@~r&Y4TYS5;z330EE?k z`34{<+|u^_5Y9_&%x}DGyejH);_s1CBzd@@Pc&gJ?}E=zJpJN4F1i)tP-(zi_8MM@ zP*5ArE5_Jh3L1mPkG4kDOoGh3>{9?A&nOwMV0vOPco5Y%HVo)ZR-3`&5-=5kb>aNe`RU2c05zEN z0Z&FrL;}cf=_L^2?Nhg1=JO_z7F%h04A!SZ26l5PnRr;R0Ts*~%rpEp8eCsq-JGBD zDYi`e-^4_ug}EC6*8?_eilqC=CT^FMr*lGCsU8#ZQ@KJeo&n799$}}45EuaL z6Pp_7nVyeE-(WW$N|%DiFyfwoQvs;lPS@1F_(PDOE8d!Ju;A~qf!o?J`aG~M@&+}g z8NeDkP=HyVNp|R~d$^YD9Nj|X$Ww2v8$OMh66nQz_;_m9>cz^ zAs9~-LSWejP5M`um)@uf<>Bci@!>3s1d6MH-;wIdx0nsESZ!YV-nx^5SKEfP)4}Dp zucm|C49>EflaChza>R7sioOe{u_h1g-Ajkt^kO*m8}*|zQwi{@_%=HEjW>8H__PC# zejo{W45M+KfIHu*ZHFj{eB5_H)4ki^Pk7=09@P@Q#9C3eUo5P5ui&Fzy7EYvn~YYsM@&=6%gk_B)wVhi?&mphd?h??QBUD z5U6yOXvS8skVwqBAsBjinIFG=2S*|F?0J52Y`6A}-R}z+ZtWo-#is)vFYiO_q)1p; zQAs^HJspfjp`BHs(3pw8v$MhI^!j{w7Tu0g3W~xw0aA-f3C@QS% z=Sit|>OKkaL#lMSR#!0J0e>{~E5XlMbYoQ6cC&YDgVDTRLw~cTG8iTJJR8|*NxQ(u zZ7Cid9A1OghneQ(xNNh0^>Yn1dHI7^D@17Y!r+Opc4LqznzJDZgI?Q#fEo-lXzIdH zfA;>4N2+v`8^dRpO&bA4+-`=d$XVYhwx5lM7zGdIJTBZiTfAf%ol2xjI_KdlKo?FC*TJGNZzsYnWdj*z_bzJvn&d?IuC2ww>+$)IPsQFN9~o2lcHvc zqUcsa8lhr*Hk3{$Z>C;&kYP9h;~g>r>ZuV! zAk|0BOcyaNQ-XVH0%LmPJzNCQ7}M9gk@B>~us-h2Go8y7Qa<%fslsjRzmxpL-$!bZ?)DcoXM!#PUZmf%=OYjd` zEp5W<-Uty{m`0E@G5rREh8gA<;8sE4Hu!IY>j1i0RNx*%Fq(=&N`?xK3{iP`ea0I5 zYS`u2rc3)`rsb9+_jJ2`!6o0O*ht$ zHZ~K=W(iOESnnloTeqLaTh=WkDn#l6@Wm1md9Wm=dw66};}v)Ovn}wU_IdH(faW)V z5a)owGrBVoVH?*25o{ZAFvY&fj5=qQ2{;uH*NkUjbZ-1@HAolWTP#%+IC@&~tvs5< z1zs}a^==&`sh@+(Yq`dS9e$n&S%@VrMx-Q@) zCK%slUl!!`#AEC&q#QTJ0omd-e0La4%$qzx|nVz1stmq$fyA89G(#l0CosM z^(~4tG@Eh(!2(8{USnuWV#AN4SsXU_qM{c+?$JBh0F0zrxm+*k1N~KVqFI!1e+Lwq zzv_0`gVo$J`q$n6in^jtS~ja{1rH(oi(W&xLyDno)Q?7uz~1T~zOmglgTefNm*?5l z>Y~c0`_O&cgK}I}7sYDU&Oe%N_mMAR@0H=QTAi%n4Gn%-WLqW|q<9aak#*wTLJCix zl=-6kyVy==)TXb+^5gl{b}}>iFY=}x(91geQrd{+I|ML)cJkb#WYg~dt;=>tg$>oH zXdm${PvO?{Bj`UZHqG9B#muEcv(oSUoIjvqH`UITs*8O3$qdzfnsnrz5Z)@`zJ#!J z?kv`{&t>sr-_`+^JG-WR33;C?pO))+QGcrHeTcvBGbGFAe07e`3~i&DR-er_w3+93 zJ9?)n_R{QThjrDOhMHAo4CC`mJMx??)~gXNB6|=Ii=sX?v-K%G47De9c?!{o{jEWVg=vTNc z-!)|zqU)krn?b&NZ0=8dTdo@z?xSMIn1I2yDxv$5MSK@+IXhX*b`RjMMx86_onIK{ z4{XzGGmNX{9`gcNtM??n$?=J}W`T;WKRPh_MD}NXpQF`7lt$G!3#KsZ4V>zx zFsAel%%lZ3v?PY(Zl`e5qG{_2-m{&oNLFh#vQeG@aSK``RG4!GB-9eOm3YYztSi@l z3~#QokKaCh8eGS>06UtPG_k#QPKtEb=81{T7PvM`z)rdb_bO0#2xdj5zEudIw7<#7 z(V!d@+9=EGI{z7B@G}&v`))ZlZHc>gFum=ds*70(gCZlKDaN-R=d>9P-niykB(XH* znopR|%xqfRmJi*9UzFoIuYZc&-InFd66fpsYyM|d@6=~ShVT3BG)0cjd2=&|@x^V9 zi!{EUxoO&J>C6m(6OXLZize`KV$`%iGg1c@>;d>d6QQ?EAg9#|ADG(HHlWG(FcH`c zyiAjVdFlq{{yDFUzcvLtCHj=iRy%lPR-2;ykEqGzdWF05y+1DO%^#P8cTA^XwcfeU6W#Pl`GWHH*+q90`IxLh zsiR$X%8nxU&eu?;Jq(gT4wGD=ZT`_&eyjGJ5_PugVrA_qju!rFief%GB{{&FU3a6?Rxe_8goq;Y8-X|9q?t zpNG}W#pu5GjI5rM*E6*i@_Mf3!h!NUyiU!ww5I9o*BQdRh>Oh?+&g+UZXF-jwBw%YU`Qqqd5IHUl|qmITjzs?wLBF;|J#>7yP>2 z^R(Via5UPY>phAa`14#B?WTUNegbuVw#mkEe6~3!e;6*{W|&dCc3C+d!PAVns-8Xz z!v{Q1D<*fJ7WwRC@gx7)U~;tK(ErcJ{frdi|CWpCXtX^y^?r3~@*Lq)-BiC7?Ky_fJw-RZ=jMG{|LVjNGCiZS z0T6fOUnTiRil`U=U5-Q_-FWo z^Xp5^MTHrkxd=Ks zx+wD7=bpKp-{ehIT~ynS#vjf0+CJX>EF8~Q_b@M=ZaEo$6u)8K&Pfp+JWPsW`rLvJ z?VaXlYtL^hya4=s6Fz>UKX>@pK5n>?@=e3xYuRvgo@e2Ga#v1A=KucyTatH#ZgkJg z-snHy;8yfyW%ZYTL>Z+TJR9qIzM3w~oN4y>KtB*y%%8bYNb~xq1UKxTTXr7amvCtI zTvJh9J^cK~<|E(OXXUJH^JiE7QGDTh(16yoSX4irdrJL>KIITYjyyN>%gSiI@LpIu zXyU9_ecLVk`MdGZ=b0M=S%y%dhr0l z5Ii%_^Ysb2=07vvvwUT=r?1bzQ62txjcJ!>=%aaiPv)z2Q9s9G0LQPUgIzcI@NUwK z=J_*tP-up~-en_}W;8rE%j4&8_W+iI$@IC~5?PLl$=Wn!-Fwa^b(~F$BIjpoS;zUf z-$TyN*1+)1n#TNFJa^}ET%WanIj+yw#ikt9;u(iP@9XrKdTeX&#Go@;aB0*&yXL6y zmRr7F)5Fx|^ueRxGLyyhJ-A7OCG}yN`Y=s>l%_sPQy-_P1$dn|&5x>{iYsdtHJwwy zzU3NjL*+|~jWnH$R^vQKbN0CtkzsG>LiP^&N8Q-|!O{C}?9su8w;p?X^E*6pt$ixV zSiX}4ZUrXD@a|-Gdg9S|Uu<10(Sy@L2j7PjF?F-f>f)Aw7#T?d;3LqLD-6h0&Js0S zfs^Q(uO?6o8UagW*oP)xtXGy$(A>=H>Iax=*0j1h*yU!i7*;sgQfTq>WOywox&|pO z>U;U3N=oZ&n*HzcXl#aXuQy+2w{<>)NJ(e?!F?C1CsF-_X}f58S+y=!+1%b{9}M53 zt8F`64(FGn>}a&kSLgG=AkIjk#w>G_aWZZoqsi6ln_}S>KU|e;2cuplM0!WrVbm;* zLYL1(p^*2$e4?+65ZT^SHvFDpK2LUd3}mK0O}1HaOz>jr>xTncm zR@mte{22CWm)_aU#VEV_cJtd6#OV4u7(tleV~ok7S{|!Hm#?yF)6Ocm9LjZS5|y%b z$B8y-Gt=&4^LKQZ&Et=BYw|Sn&P>qY$0u+FRoBISB#1~C=IzLG(dz@Xt&bxgchL9W z`y#fqciake+xT`-&gN}`?4s9eARb*O!Qm0{sf)JGSIyG2>0TewvcMwT?RKMMebuB@NP<62rLeKh1d~JH+JL0M07Ubg|Ukj zWNzwq4YT`gxq#a?+5ay3J-nRIKHv>chZ}yGnDNjaW?8RyuvwLVZ49oh&ky?zzJt=V z+35Oa2m$HMx5OuF%+F8f7Xzx)dpm8vJ1gL_Z+&jY9^IVNSdwy1!59QQ0T#owL=qL-jhH142Q-=8-6$fka6DmyL+{>Wo zVqL*&S<&i2+1Gf3JX;j2M=)UdkwGl~aycJ4=a-E zqM-Melj4L8To;echGoGZoxxj@1%+ZwW1`?#tZ(hE ztcUMKR#rUleJi-LW?H6Upy`HGBcrS>e)_G0Sv}fNdb*(YNa@2&dr6NU=I}8JAx;H{ zPu2(DV(!PNgZ%@brWqO{!UX!5SE9dYweT2bchd3J6m2k@H>|eD)8jiQ*M2%A(eF?p zxwenop&&B9-S;d^YOUoF(xE#OmgY5UG4jMePLQ< zK8GVL4c(t|PvaXkW~!C3;oOX9xqtFmpR`}?9l1p$U*L*oWehclL%27*#;4XarDSGW z{A6%)#&?0uH*8It^bT>BWx2M|_Z~+deD4t3+lKAEJ;jh$V9lZHwu5muH4_i|2gUt5 z%Z;{cTBD{g;aQsyT8*F4yUhHD+`puSIkxwq`OtQoA z+D9~Azdv#jL(gwTGql?TJ)|2xTFegGx7I=CvRX}wi()(ZhtcNNYy^PJDDcazLXf0< zcWEbi?`!^GBT8=V67PO1+LP(Dgs++}&Ttsc=PmCg=zE&okRCsp$I;cgAce{eHe_&V zHuC3Y8)Z5}KieWgZ)CPKqeM;54A!kj@YLQ6tK> zHaiBN=kJ4|$tRbo-aF7mMuQRR-nh_tlOL+*$3TF#U3wgIFzojzw0dUa*1^*W%2spO zi@0Vgk2EXBJM-RKT>uWg#5Vj0x8Z;87Kqejjw=DG83isgOR1eL8I#yNUG$G-W$fz; z*{vC0TNsqIXAiTbq1kO!bEpJTwO92f(gIk^SS5YUa?*eL079|o&tQx~WCdazuk z2;QHSQ@kQNNTT0?r_W~Df$bfil?}W^z*K5j_K#^ZX69J_Li7faslQuur0Rr zohyMaN5^JS_<-oqLFQ(;J!dGClSE!`#@IyGS`sLiv^Uzm;ZMzFzL@5&T;qc_>$_4I zWsKBvKZ4}d(#TZUXBg}k=LlO8Lsm#0{$_A<@4WvKUQ zrd+9GU~7c#ut->YGo2R0C%=Php}r&asS|}HVj>xSNsRAhC^)SRGN>l`Az%%pNj#9G z>v$VpU?OZc#<9e1H5Rp%0DRcb84j(dyuh)Er?wF@p_2q>6nKw~*!tJ$=ozQ%e;0=b z+lhu^IT(P>Doz}>aF!C_b9uFk@ocjvz?|bi^ms?FfqpxFVz(HLTIuA51z(ur9TsSx z^!ahW1-n?YWJEihb{vjhU?R*K<;>J~>(;0b9hyY*(UvYu>jZ0ZCs}VVvLkqQA-!UL z@P|v#xaY&q+ZPkRy&NmM+4iEZzo_}Tlltv-v8?V<;q7cPnrU$(DlMmjZ2RBRWbdW$n|7Wp^X3kf`NYB~meszA@ov4g?1xXs}SIP{2| zI-Lj5v+d|)ZYZ|~EB6jIg3i9B=pR~)pXNT!4h;{a zq=bRJK%ow_omS%BVf&q*b!I-kat6EcmiGCCCy;}E+ePO6F!Bbu?WOd3XGNokgHP7a ze!IB(cyf_tm!Pl=PCHtsn=WXoPQqlnby9aXVrQ1ewrD4#?akYn?U8MLM9(Ph)!rfP z);14__h|PZ$G&YJWc#)D512jX8w6P&`yEaPr^EmB_xlb3?e_1WiTC*qI>+AML1%mF z%)6cGBR}rZ>BoJiaG*qwil|6X&P5>jk ztAS1HE7P-4x{UfQ%!N;#|Z?h%DtQ|@|YHw^UN1EI^g9j!Eav`qsTcd??+cmn`#ZmfO z*2Wj&VWBtfWo-0?GWoCsDIK2I(6A5|!;q-A2a&T0DIiXqwxKX|J?wJXw~KZu;2s&D z021M`1mb2mq{HhI%39dga$+crG-wB~SdgRVyYb6}0|#s)-tpu2a%S03%}TWIXqAvaFEsYq`pOf?>S z#t-2Y_cC9J(mTLi+{`lTQ3MjC^ua7GFa91~7k|2MJF|L0V6C@rHd!EFCJMo`kT%Mk z$QWg^r7rPR3SFdz%@mc+6VlHjjyMR#!KbGKuSGH{2Qwj0l3+))uW`7Oj}9Ck$)u5Z z)=r}O(r>3?B9Mt6Hqi*GCPb(#)?Yj5xiOd>(sg?oIhRJOr-&!I*>M==b2my;FxzY) zJOx|Zr@r0y4$oz|#nNDD)lt`SYWW;b0MLcDRf~5#EIRJmF!7>EE4yN}AHB^I zObCz0hX%qPnXCNdqyYhy{%8rPw||x4w7y*99c?$Ci?prn#J(u)dOROetjO9X%8-=e z3?lA55uy{=rPR@PvM=AHV0a-YC|18?=?ad)kb{=4^ieTaat{*02~Uj;ZoP1Wxf?_C z@s49^Bsda%2ctW9 z>xZ^-!HEkDDe$_wy#w!5+YqRz15PNiK!l>^5W>i;w0A3`o~2oPe&ml@KBkNzT&~X)0fHy1ZN>yR*lda1*?o~wcS&BQDO*d~ngx{Xf#c1 zAka+vGn<#ImUpa4T+-QO!k2s^pOOiClu&^kg}L#xAIjGXp!OzVvOPKpERsh|ckZnO z(mdFWk?UD1PdB~gW(G6ZIOC#b+mKD6jgW3JGBYR|?eyLUPNDNfgZCh*?cAtf zv$=aoTa2%^-b~QJ6BJ-SJ2(7rm{y2sISgJGlq6HLO2eARH+(|0yD5zGB|UO!TTkv} z^uDdU7xOjlv>I>*#Fzann5;oD9H)Qp#hUvFhHfgRu2qA@Re-JAVF0$bD)kBgV!^#v zQ?JEOn6vuFZ?o#wbz9gMhb{3qdcOXw)^dsEm39?U4fJnGn4)>wTCDW zpq}`(7!5Qq$It3KFe+rG@v6E=)9izX-YG16Hb{|0|2e(u^|BvTeV3a~y5hPCBZAwk zDZ5<=&1O|K?LbAlW=A2de*ti3u2hNhQhzuQVw{35`T1G}Y~2gBngebyZ}AmcK$vZA z=f-|jfVN~7itEt{Q%8_sfr{3pH`(C&np}Qpfv!!D=Q0p#&)Ni>WSp0YHL#e3dq)p( zbli<$aLW~4?k$XBuxZ40ZMFp$=X2YiuFL^JxU(Uma6;2qob$8){V!NZ;cS^!QZxH@ z-hsFu9DTyuv*5yc`E4=$bU(ZwjE=KG_C7NcJG>czHzJR-Jqmf}3xS6$ybgM@m_4#2 zdArEW6<~;fhPF1CVj2g{0GIyg4#M^uB*HZvrt7pjK3vM{x?+9t!F-U-8ctvQQD^V4 zSlwsKsXen(uA=Hd4Nd!e_X5zgP^Is62Yhstm&33+Q+i#hyG9)qIy#CPhFZKyEoe#h z>Qc(#TJaD$-I1Yg9%;H%3W_-2e%RepWGy);1+=_A%+DP3ldUwq7qrV@+C7h zew!$+P6TRZ2Od?4_-kc>IVwS6$Jur&a?0uaH>CcUg1zFWc08!c=j$|X@Bab7-};66{|pcR*6L?Nl7{XGv|ek1R8 z$nqyK=Y;7ov>4dh=b1lU8FV?ysJpUw zQ!zaDF2fngn<>ViNRmT?8~2IBbvf<(%&VEMjst$U^R&#IbIuk9M*#!e?7`OM3Bow{A~OW&} zHTQt?wmY@M@#rM$j#)(RD*->c(JaPgG{B=Ay~pP8zB3`hd2-mlBRIiwt9RN>&^>CY;$%X|j@G;BF>Ye6fV1e(A=-qR2%du*hYHC8s+Ae*+}!cmMX z#A~!4YP`ee#DptKdJ<2qwU(*QEVL;J@SENs;AS+o!lP&~W=`jcP@NQ$i^*6`dFKLt zR&ck&Y)&J|hxf7^J}igU)%#pNC{H5Dn1!`YX6nMBQL6}#={Qr2NFE*7HtRs4?bM%x zg|}}4lxbSEALrXL)%vHq5%UQ3YjAn^|78b$JKYf_qW^9&S6$Uz(J`}>S1Vob#D#u= zufhMbE1}Xnu`)eTnJ@RlFTCsRe@N8;6Wqf8SMIqAjV)KyGs++4~-S zKazSkw89N|u;GQg(>R`}K=!sE6Ol*-a$8QiFx-uQ5BSffb+*Tx(wTY(V6UMNF#7U< zru%S)3UU-NN>jZhsx3nH-}39YRs9*FPhJ;o{qveG;BUzWVW6;TLmm7mD4uwC?rnmq zAD@~L2wXYB(jc_ybZ0#>@_gdsk*_DGC|0a#3M0@Vo1*k4g`xrdC~MR4+bH0Jb$GUB z6~D8AVF{soFy!&o>D7f*@uGkv;-)AZILx}VnBN4Y-jOe;IE|b>M zXR|i@2He2v`E#9ZC7Z9)rsA+1vE6Z6=eKSD{UG!(D|uTP+_e=BXbJR|oLh3KM0CAZ zY8#!1@N>l)OibaF6W(&tIs%U{vHxHg45iD&EUketD;F~<9ruVfg;5TQ_?XOal+2(5 zD-CsMg$urRAC-do4IVN~MJ%4EOb$%DH9JIp>B#*q%&mR=D!RHHFZZ(6ojj6?LRkNP zRDUM|eVQ#rAN9KO+V??3B%?@>g1M; z+MpxbQ8eUTwvIN?jO?TrZrc+W`$CRFzyW52b@p@nD!4PaOMu%k!8_mLgwH3PSy=dn zg=z2F75Ih$eV%^RoXnF?{`I`p_!7ISTZX}`KX_^IYZlEzcvC?x zx##U%Sl#jHmr-U{oCg;p7z5^uMH@An1iK9w#)D|9I`qE{;6bzn+TG$j8DMlIr+3uH z<@K(Rkrgrmx~`JMG>C~haZkcRnh?(>2&Tbu6r$IKfNKnO9$}NbI{9#HM2Cqk!-R#X z3oprhGgo1Y0sk`*nGTmwL7={t?in4W$3;y0rBa~qYFPk`Vto?TaPXCbWtNpD2f5$F zYg{Qba32ouA;RlIFo_r(IR;xZbL7mF2p@n$5)T!EJvgk-o=L>u*fZ$J3}B;{Sk2K7 zX01{3@mh>{T#&~O_rg(+Lspp*Y(^KCq&udgnlS=#-MEgc(LZ2DsH$YV6r#;PDfMue z5-XU<62+?^g?!;p!Jf74>|D3IxdOQot?0t@p%HOf1#ovXcJJGnjjPl?qnoSi0W!tc z4ArD5714O+?{(p~<=vJKZ$Z-jJy?fq?3+03&34C{-l6IYPjaPUGoB9ZsGKKsA>40W z9)5uUes`^SiLgHjLuOYU6CM6Z6@5=n-PLBh_2mN_ZGY;7#Ge zvydmE{A8C3=Fs_ANxELr_QcNsa*2GjWdkTA_i{nC?bucIC2;Y8H-HTWpLH_@KwuPw zqr=gNLK}YM3pVJj?9bJ^j##%nb!7H`;@vZ|ZY{_OIrZ#OXw*}pu!=#Mw!Bee!@#I> zd-wh#UTY(ROUha&8^h({>kODR{h4H0$MhCl!T0hS8Azp-Hu_RO1{7rTA#wvhm~f0? zyZYEzsamyRm+s?=!m}&;(})qnCP5Xtg0k_|SN6{~+2ms_qL?Et1`ZylPkyEyf7|X5 zxOTy;Dsqh6%WGJ!LH#~*u==$-GJnE5g-!wDL-%>kp}|s4(b4M3Z{lqiAnw^mhx}YE z#@+R|P@r&r5o8C?aG*^qpkX_*UFXF190);O{1t}AzUUv~eTY>VPHKC< zop5X!Jbev}nCz4z$wXD<@pC&fD3;BywFNNv<+kulexWUFRi4roePB>Oy;~e{{$FVc z&EsheA?xvkb~t>Coyr}kqqhCz4s!TD$?X@~Xe-C3wb;S5G79@s`whxdKsc_Fxjdzv z4iuMPX`d9Qr?pF5v!}GrTM;AXsblNC;PFf2i?4$IN~?tRc}kn$ILFQ2Q(NS%gfw+vMqud2AW{x)wV5wPiw8%f} zEM$n?;c0UaWP0kt+Yk8s%F>(W^|WD^s^3!@m%dKjuJ>wIngNmgm*#Wi@XPbL;PMOe zxvR}n#_5OHi1vg=Il`U+mm}oEJm`({933|!RMDpR>bT1>i3k5 zQm_%Rp2xp7a3)oCS3bR;^?*k*PVSRg9(J?*g)h3B?UO$4t_FC@cScEde7C0#iQ|~n zuZ)ULc264^9Tj~_oAu)uf=_I+qlnY5v{)CnC#uHDT0Ui!hC^FnNxe1AKN$Y0r!5ZC zVfUU8C&`}EV@MHwD%5)i26vCoFnY#(nNK*?&_4r#<-DgL)4aetz}HbL#fp z=VMDp+U>MY4@Y>)9p3u=fbYI;`n@grw)p1x^&e%4dc;UT*Ed1289plQVmBaogFmGMKx02H8Naeir2H9bB0@ zT?e$6TVG%LFf!#(mLWFUg>18BnsQ`D`xeRrPZ`HE14jpRYP>WR5Uzm~p5N5DLTxKo z$Hqs~s-yjq4RQF6fI+A^X=T4YrXp?E*!9wT z?7ATD1@}9dxj1o;b%zL8+Ayw<7moZR4qa&RZMBrHImukxsv});eCDl!hwdMg#lL9d zpW})t^&2|_@yO23MzzkKKJ#8}5oX)spWEFxMA#|kthA5lhr|YB`l3M$lmK|6JMCYj zX66qfuXD_O6o)hus|35Oj*)`1XrneN$+$iw*xLBYEq<8v;4Di(I=r~(O&OL{G zn~z>H!>C!-zngr_d`K;P?V@w zm+ZxWTx$(&IITMVOLRf=UOdF8#J_A!xlUm2)8hBSiSnK@1OfqM zvj_wSh+z0v9pKEiZk)f@$sQe^a0?4mB-_+ z`+Q+GXw|CFDbEukd5@u0T=Q9oPyEEdcqZi<-!g>u+Ppn--_KVZFMIihEgRjgq8FF} zUH)CDj(|b_eP%Or^~WpxxhU+fv4NHD=K@@Qrqg07a1t_;y=7*lhNgnfJBC#qT)XG_ zN|c+M8Lt#A15u)6U$m_tX#hNRKiAcdKK?e_+I%59qk&Pd^dIiG`=01PORpzz3LS6x zcd;V&)AF90q?wnugs%P1)oNMU_ML7f@YN?pn1-RSUgV5eVi2LR?7h`A=+UcMUdJN;E>cySW zV#W(l;g}n2b1+g@20%S+Yo&2q!c}n7_g3z$l;#oUe`UAI%WxcR76F7R zYD@D&@y~oxjmxOYXV;uEYOClnVe1iR|jqt8keo=ix^_MA2g=qiD5O1 zX)yVV9(j!@)j>org|%9AXwwL?dmBkXHqdcEYmW2~pob@SU1C+$%Y0!cHSTv#C6HMG zVyQVBYB3T#sv?^eH1QAA8Cahfuc+H{&%oZR%V&9t)f|if zW|Alu3w#0xm-D!o8kLA^qn`^cW!)5;X~j6`%N){9OebRWFwpa$;|sHBm9SSet2PB~ zRbwtJ$3lJY7iICv;Pnbp-^Q$gEpyzQwSML>-?X*socS^MKFq+Zjy4gx5iWo$T*xCZ z7*D<3RDs)kMi;Fe6ov%pb_#@pQ$a{=R3KUf6bS0K0-ZFo zEmde(s}8X{atEKAp>cl^eZ_UZsE0(|g8-=NlShh~sq?ra;+tSpWNxNf(}it?ovC3q zS%Rs-nxUDoAqMDVJP818tpU~TSOW|{0r=1lcmP20FaXWN02B{+DNXS}faF1Msf{&2 z%K^reNl=OcKo$Xj$GzhUX{eZ`94aVTLsJfrO7l& zTwG8T5uCA6633TKfU>vFYkV5>%o{lmRjN-Zwq{zKv_J~t9;H^wAjugH1Bh*h5ZaHS zc8c7r&~?pzp08*8S`jD2)MiqCnly=U=KLC}H(vt{h|@-Ibt}`?gtGHB)QK2in!l~9 zpE@I0$n26PU`aBv_a|OafH7%c)96E!EM0zWre(m5=;( z+IcxWc0UBTndd$KOC$RIBl$Z(!ix|7w?-bmKa#&S^4LyT^jpg)1hOehaC-I`w?i9_ zx^!E@gX)9R?91Q}3Kz(a&l}Kh*H_<$WO8)3E^+H<_Ge(QbwE@tZg0yKm;=nd!F(%- z!AD!1(dQFv1Ne&%Tzo#knwh^;iPN*uNj5tFjk~W)a<_D8kWQu>k)BpAa;L0|f1VDd z9hWn(5W|z_z6$gctPXPPe4p2}hoxST&J&uT60O7{APx+&52A&H@Zi`X+MFI|si%c1 zF0lT11R6B2hJ#Bu{uyt|1-$UUv5Yzq+#?JZ3Sh*o&k~>&4(PJqK5=jO#8xKj9Oh{4 zhvFnXRDm1!sz&(m>iqJ?6+kJTJ8UxG0mYAhvIO+>o5SJxff4(*SQJY#Rbv_iY`{!W zP@mPUsU>m<+sX5k{{jAjjLXSWN9YC%M=@vT|=WRP_>RARA5p9L{8fh?4OJbruhWIzT(A6=wIvnuT3Fyf}ZB|E?-UzrAfi85U!Oh zZ49OR2~re-sFhaNva179(LP2Zb`*MG8w_b^s}y}CB###N;3+6sW&7BZCRVVZ^KqZY zC4ocW*##w<8Wl$5AZM9$o&YCuHGZN^WsK37mzE10iZK|E<0RORQ6xy^ftayKrI9#a zNB~Y5ZoM+!jf~7-NEL9Kjpc#kMxVkX=F}?eTQ((dN=}xdn2GtRnzVhK5Cja<{c0j% z)ZyuN5jfl@CSi~dmasMGx>8e!;eJ8$NTUNMGdXyi>FNTxVz3AAhqvY!Z4nn2*2?J*~$xW+y1@@-IN_ zUByK;KO1oh$!CJRQ%4Z`NU@G_Yv{1=kT=f9%k1OD`Q_R93 zahKFtR+XuVGBv48tlqkT2{4ran}^Y6P30i;qOFXFA`{l^RYo_cWj+U#*))&@QZO6j zK9<6M9kFrX_6f9TZUC9~#cRtG@u261MaV9kfO9hkJ>e4LxW@VRamCRq-j z^mbs<<;;KB<6jK~K-i=Ji#ddpz>&_6gNmlJ8*)Bf4&+Ud^BJl?Emk1$*#uO3?I@IT z(NP$AE}TGY;VI8`69}Dis)mO>#kH+D4O&;7oBAy7v@)QJ$rLw#_65JJt_i_BJEgOrKgqA>Iw$&RSY&|snSf}ctpb)>bT(yFm5=@H`Q>iOle06DdG=)TM#c< zVO7^+A}n0KJ^_L&ARsKgfN0T^2qCz@3#(Ot*=E!dWO22zEYy?H>6YH4Qh9zD-4h-l zA$iF+7q5HLre{Vz&30?T<5as4ZSFhqn(F3X`zg9!Nq>;x;38B@EI53-5|T zX-4-0{9%AU4Dd$*{wTn|4e)OR{QCg^UajO|9VV0I+aqn`X#%9RlY~fYv5uS%7*$^v z1u)rU0IQ@(GWAv`k94x97&xXf^;Rd3bkZ|WZafT&E=F>TMz<&hy@Efyf18<>Q+G0B z7#ipD375O7WET_Eu6i*n!9le!LbgeltdYy5+Q4B{i;Wx9{9M=5X*5WfS*5TD-zO;z zTR>+&B2a>Tx~nk`t|7c3opuTP$0PJ$$21OmpGGYR9q1&z1YiaMz#RGggQWLKq{bxZ z1Y}Fjo29ZRtBDOyV-y4?PU~HgK*2Q%M0I9*h*37vLpWo*a?0(w_?mWKb;#pt0c zM<|iKQhD3~f2S)4SEM@gDCE0P0$6U^Dv}uXxp3~2LL{p<{+ac|cG|LNi={mccT0Lo z+_)WC+!P{$S#g6NBBf2DOxayWNORxqIeH+xX>o` zH{F7R9zr+ZJPfZi2e7)B_TidQgaBY4p4|!&yu&(<5VYGYQ;Y$V^z14P1`|>Q-=A+(|yT)|K0rOwl`6sU%;iV1IcA~VX-vc;>3KxHLn9n zIq%nELN~K5Xln%XYmkT8*eGcr?ERZ19#aIEy4pZ*37~LMD~%DB%K)>?=|q&l;riym zYH?Gw3@T`oYGESKx-RBL?kQj`^5!yfWuR{MhG0hz8cHA=jSO?Hjp}PK8DFD&AD}~! zhv5xsy1ztxnl|Jeo^eai`v;csAtB_0@KQb`#C%A}`H&X$A(Ye(Te=9!g2;x| z4f@8lg235|yva^Rm%Z%c`As&u0edoCLoEA~5b-9P^vf8S$WHg^YQWxx%kkoX;J=NOkw&4 z97IUNEWZsdO^%E6{|(M=&f&FOQHKx*Ez-0#-W=*8=)mODKV_ZaNpjr^0P#(Aet9$a zZE!8x#zN4E(0x>fiZALQQw#a#K#Gxigbq^@JKKs4^O~MLi7AXKba{TJ;Uo=y!xk>oh&m_=RNxHbtwhe| z+_c=BCP#t{-<)2q`N`>S8u9kF`?xS*ME9Vi`zbyvCdEe$#$Yh;qT!7IYnk)YL%>Xr zc!a_`v^M?6klWTLn<&|nX>3`Xmp1JrnvcHSoJN9LSR{==1Q1_s$IyHYD(#JdeR>a4 zPx$A0)7s92UgQ``nE?hD1B%Mh6N-G)vlH@35W0i!QK-? zOk&~^2D@I+``Cg62AKhX`LLvbKnUi;X1sl^bFV6h=-?Sls>QV>cnlLoXn5Epm*?LH*P}$MN0<~;HdTNLN95OGJRm(xr{!>T zuz=y^w~GtGi<0iA;DE_+Fc`f`jsr}3jZ5@?NOld`)L@OY`osVg4A0I-!;6zYZU(>K z5H|%73F0FGlF7B%n_ON0AzL^C1^~-Y1B0rL2LJgsxI7(XCzpSS+G?<%zKIAK!rN+i zwxTL01)HSdjv->2PE1|gQg|Y~^u8)W@U)oVi75s`qRR`k;;)=nj7^G_9+FLdFOzk_ zhptXkx;?gTttivVXvt%Lqja_;?O&%!lMZ0C0pPG4)wGyi=vXcWq#}2ibP%$|Kt7VI zxd+Hw;YGxjKq~%+U<)C)=oAw!Mf|m~5qF3x;`|ZG6@;Yucz(s+(B=nwL7QLeYE3_^ zA1?g9_jugKQJgE2sxjaX{4(v{&@T+WD}H)vv!7rZd;o)fk_Q^1!7a|E@SdMb@g)iP zSyU{Qh;jl+>_{Tr%&U#*23|x42>BdIi(%&1?k`l$qJvR_JWJRyRJQp6~(@>&8#6mR0;?3qH4Gd4oBvz z!2qa3TdeH&v>y?q;V)aRq^#gJqMKF~l|#@5%;rs1zG7=Kg2_>)S*pI93H z^53jn<9d#<+~JORMNfOLFf?sVGz?tfum=Fmc?`y>!4ujv1Odd?<;aw_f;$HspL1#8 zb9jA*CJu~A46nlJD8+LQcxe8LxW;A$#h>P{h31X>iOuDI(I>`y%QZ#UG=GWs+o$a7 z;44owoZyeZDOkPmVy^|v>gtD1k-gQ!sE<(EOhpuYntrcT0xdb5SWGair%k{~9wf}z z!YrpBKk{8IGQ6CA98sIX%x*pzdU!wpdYr_^HpLDt1b{pD9mtpA zfAb|Xn{3t?$QhFEtD>}}n{$LTs&tXS3woHVndCJ*p;s;fY1{#g0#=i@(9P{o9Li}P=4H%?Hyae~^75{tai`P`Qpk%}bJR7C3YyCSzB z(;bJ$!Rn{$JR0H~5(ZHkBlEx@?7(XN5zV?N?%2$T7dj=O??nP=y9$A-HHTo;?8Ego9c^?zL-xl zo5`%Bi=Vy2gJXevNFdXh;tO}>_6dOac=~ZGOTGm>1Rg=J{lmTF#$h(J_ft2m`1(m`ZS242LP3D{x`(`2y!1O z-bYF#6_ycLqx)66Xm+6evTOtIrbxlLD&}QxXLS@6qPG+!NOu+m7h5!ey8e?8CW-Hm z2EVoEv8CPopn;?wCidw_;m5NRp_{M*fQP0?`u*hM8(Xz0 zFYQr?jD8aw?Fu$@xB&4YO^`(e`$9MmhF@gI@fb>DlD2LO3=f9@ym)QCI?~aeTER&a zYlo#LhJAWfInZJ*D~3)c(+Z7^2@E>7llw^m>m68Jv-&1zz%GeC4o;}`ri1|dEzAzn z%G7R*5z z+=EOeVU6EspD8p1Ik1orK^z4V#z#%?L@rG}7wgT~OF=$A<`)eQFE|3NSwn*Hyy&xy zRDlOYE(MbnS`u-h8vk@5%q6fI|49lk#N%Weo#RuzXft1}z( z@ca2Be=-7a&k`{S0qVL| z08jKoNgcqMTE@pv~vb>4uD~Hv^#}7JTPPwU<5a^eG5*kzP5A!_PnnT9{5>4s9C5?f|_mWKO1yV z^Xw_JRpUZPwj!8eo}i>-&jbRrrUY1?^<@~XKLmn7T>;>}J!*qUx>0Qm1{NyBH=l;x z5)cog^7@XsRBK%A900Wyetv+&s7*&|GG74!+znM2BpaCHKkc%uy;8_|B)HLHH=rw|)^Ws|0 zzc(uiRqKM=kIaNhehaj+!qc;Umqs+-aV8C;=|j#qqCU~aaG;JN>l1MV9?>KpkP&3) zqXXNNe^g+o!tq*w49II?^kG~IHkxWuwwPtvK!D3M(ntBnB&njl*r`bP`2Rx=aNC2A zD?oXJ+Dr;40H}aznpw;X*`KA@pM_ZuVo;bBxzXAZGmPNKGX+iXpz=RQ#edeNhph)& zzvKt859v51&S}jKJ&VwGKZZvzX2ayPX0a)XakwXpGm|l}i4mlDv#{!Ej9BR-2wY6} z6;COFBDc~SBzdUTg-ia)rmaH7K(6Pq?Qqz^)8=cuG~Yf11JOSEScZ)rj8J^|W3!IG z%mH!^;Qlo)9(1#nn4ug@Bsqj#Ky)9@BKB5hK<5m=4r1?V287BuEDHv!lb%NtSj7k* zWRKYMGwm&CB{97Km_{?3sE`3cX{Q7`S@J*g>0+hRI45Glu-DW8u;h4o8#iBnGnyto z_?4P20Uk}T2n>d?WlX&V`&ePk9yUcr9hb7>$O3GYZJIiUP1zn`)3zNQJM3xjaL!NF zU>;)Iz(e@W5=_cw6Cj<*E=8zDO}G^?sx}8nPPHM|Jc1=!u$ZU@oc528dmmK0F$f3J zl3uY7P~1};(!y1v2$-v(fKkI(fNB~E46|~3aEcp+Q^7O>Q;jGQLkFdiG{12rAq=JH zR5f=`nN-~hAgW*0YwYX@1(Fic%}(dodkvXAou`tlBUXwb$V&;#@f6Pl(ai<8jGeW$9j>Ei?DrXDT+Fn zIBnYYS)0bLUWXH>14wc?`fE3d)@~BDaO&z1qxU!)e*ALwsn1c+BM`?Qb+w@lS~MOQ zY)3)}XS0i~5xY{kDsXzL(bE(fI9Y(7CN)6hVTjBHPsbAH|5x0#HMfl`Y5yXZCEH2n z(YBOCCyuR1O3uu@uqcU^c_fh z&*pPH{vKvc-Kj^69qH1#W#gY&etdO7cohA)d{FGx*EK?VKNTU<1oHz)AeSGeo5QgH zLpywl8h^S?G>m83vWs`DG%cl(QkG(c7A2cxU&eqyBo}j#jM=CxX9ALqRZ3WO2%x{lr ziaqvuONV5@&dG+Ik)Hri;kDte-hulH#jwiTLP-TF4#aJZN?L@z_~7I1zeu5mA0)&4 z5SL&fXz$pvLk-FE;%YXo$VL$+$ zoWgBN8&H(C^1pXr(t{@6_69S)jtu5QU`phR(1AS>=9~?}p*Xnf=Ca`Ca?>E_q9r>Q zh))Oo7clyTpsz6^$uKgwY9qP`LXOdr)94pqY_)@ZsQ_HQF+N2z zRXVxNe!sr>Et`$Mna;-6+X|*j!Y)RFJLY7o{YZF>lWnSxlj85=RdOE2|L#r0(E8E| z5bVFOXeqCx1+wg zO~y!ZtE~6b?WnhI({WP#HUr6i%8A%TL0Ck4ejk=F;!p6ZQpgAiLZWnWj0CsKt3Ke= z$!nUZ7~2Y&APke_?LV}d{C z^|M>5-jbCTarReNx~|d{74wFCIA2d5KHA&0F-F#Pv@&`%6#1K1cXVnGJ3ns!Ka}Kh zN(6W=gO3OxxGuKv6e<%@oAud269^TQw0>~1psJf2~KEVs^{4JRyDoM65; zqr)VNjzB<7C7qVC_T;X+2A@I%eXfEc1bj|K`5^inD&;1thwX>MHg2Q=J0EleGBBAG zp(;5Kc!cs#V?7=Er2$Q=R#9Pk$?0rZiApW}kWdvrBveJo7w`mim{Ju#q*O)48E^&p zna(CZa4UxzT_VSdBbEPN87Je1#YhM_$&xI0zm$^_e2IHU+T?7VX78Wa-RSG*?ijKw zmedj{ULJ)@qn>3^a!Hg}4n<3$o+aGP_uo_bGjM7Nr@3R!AYYC>#*<69fM4^%dtC2% zTz6wBuAXq?I{%riil!@{$`w5kLHx^OwcZtyQ*(LTtQA!l^47--G|5k?oXCJ8p=FSY zNV0TXd-o?SP7b5*#ZMmn@hAF|D_h=f2{1fj4^cjFfeQj5;icHs zhnQe5E2q$#YCR6)(OFQt^~?6qdm=u^HpCNsK1RauVOa!+(1O6zsK-Ab>`selETJal z%Bau#iN)^I_ArSg^*J{aOk-|xu&Qsfe6=bL=z&tqK;dZTCcKc48vI;y2nq72&rpX` zWjHeA`GsdHT_x^>4PQ7MVw;DWhimQ7l3Q0aepL=Gm@w~ij_tQC{yG-;5T(Z1^Mci~ zh&H`t_nfv$217^FZ5%vYOClXX{{)S-4EU8Tk5`9n?Du8 za|B+K&No@!bxpZEa-RpQj%8FC5OWC^0U;6!!w{WBnRYV_8nGbg7R9^g!HvfdwXYmG z;Hhj{_xP(rgNIE8w|X}9_WQ?=Mw^{JKC+)Gjne^v#_0Y{9GyO%!Ydq@5Zs&5WlLPFi5dLHH(E#*6?&b0s+%qNa=`Y0zFhFE zd1D>B5Sx8B5xwmvqgY0`VtIxC;Mp9GM%G!sbvad5s~Al9p_(PF=fzu`(6*)h_pHaT zK8WIt!$k_=8C8`iPUn7-gBlQ3R^;6nrdhts2Pr+{ zu_TB@4-+Jt1aRho+vZBUG+r0Qf$ciD`F#Dp82_%v9D;VmuhIF;Hr+Lv4(~8S=^F3)s; z90D)xZc>+THaJ{cci-ghI#5)}i33K%YP^4Z{PKOuXPu=f5R4M|10&%t^AB4r*%b=x z{j^7ROKpAwhfzDgN17cMZvOyxh}rmIG5aIV1x2B0Dv0sKUlV4d4bk_k9x|IM)Mqsl`GkU`EP5Oee>7(8omJId3VB-hRO6K!ydNaHEI(LJ= z!gSa_>@6^=S!{D;boL63`Ovlk1cNXGVn0NX`hg|cZG+!EiMxnGR8Uj-x3|bE3G;t(HqfFQ=-u`gmG^{h4y*3vZB!Mgcp}cK%|R#?8yg?oHbl>*Q^4=YZM}r z62>wiI2B-dtdaXkifH|4mj7)2iSZ3|p5!tiSBw!1)cG{tn9Vm zqnX8*@-xIR@!ORfLi5GrY^Z}r1BS}E;ojxF}jBZq}+_x4~f$&hMK`R6|cpZu(6u`JI zD`62mRgz>Wk|pTRz0P@*>vSf3{u6nhjek6Tq1Tt99EUWO^90eCAVCaD5^w})R5DA@ zB@C1CB^CT(iCGszjYcVFsLn`)bZ3cp*Q*jGHB2JWVB&Eegwz1!sb(nJ6php+JT2dq zx$gN`22`uq%+Zf|#E=+EsF-dsDJbajtgg2KOm6mq^02P!%xe1>Cp$@bm zVPLnCBe>aVly4UdxaC9>1G|lmri(-O9HEC}b3lL30giPOq61RT<08c{$gm64py?!F z&+QkAVT6#vK9*CJu<0j}t9qDdk+_cH^%8x|rg-7GwI_BDgUxZ+GB-WAPcaoEO=<0d zITZ)>T1%7!c~D)v^5* zD|G<&>2E+Yb4O8;o27Cc2vW3G5ZhBk6*DBnD<(-15Kn|GCCHp6rjpRpyiQwA-<;JmEquV$v1#D6nJ$MpSOonW zD70eO2Y?OwJvdw~;R7kK9q9)-sj#VWJh)32FsM6q>B#*w@)eVkn51|QL9udV>9)#* zY8o)W8a=I_^^z`}*o2A_@q{zBcWsapy-QbPGRy6$EMG)gv5I(+)2xw?;*xWeyatgwu04yXupIti?VrcfHi| z&`0WxBBtz+sM>-esz-=oa;PChf_3+e$g+nxXm}R;&W^%sJtthUE=eniYj1P+N zQ-e?-B#VNoBSlMR2WY2i=>i?2ocII-eBq>B_0@83WrXTjTpMf#N5%B}Hc=8%m5h=| z;c5#IT|f4+nY@(QiNZEY7U*s9SGZ!HjvrRUYUbJgl{TwQbS( zG=-qrLgQ`Wi?m4k=qZ`Wm5kkE4puU}gmF8Y7h-Z6d9jv* z-lICfKxxNkDpG(Z%e53_(yyg(osEl%Xg@e2d4<}MPzOxV=oCNDfL$y+`IjJ>0?3XL zrx0Y!E*L^lxN1OgeYr}l5OmaDc|jgRc>!5P^$X!-)Fe4ttpv2NZhft*Sat(L_cDtn z&g?A7?yG=_0TOQRKRxoJ!!qoD3$2gB-rUuL^H9QEMOnMCj>4@U775sd0Q!quE69(( zY7qq`Qvewmxt~H8$4c;28x$Nkws;O3B#wviVt#is9cT0DmkG0J%;v*YjkgSSn0mY#uIK=Tl@stz`C=M;fA(8o^i>Xl;J!XzkCE1xRO2H0PausU?N~!kJ zo{(uFDN(Wokdrn_mVT5jeasxAQgWEjQ6itCIG-U~=46b{DZyrqLi&cDzLyu=>kIE89QtX2)Hq@vt?$C% zq9Cwz946Lw z9uzZ3U}mJoGt`#Uc*cm*rhKXv5ox;P?PECL`l~Miq=G`agMuj&9f3$Wf?vlL`NCe} zYJtreNblShp-QUlht%}wR9!eFl?U9CAa*OP9&QaotlwcrWT(SA;|vD*rzc6fXQ}2` z4O`%_*-%zbxF-U?;jdYNuz4Sj2J>k3fwzwhvt3%*ur0h@!ni8ZD;)Q6+@`MhL;}-c zQEHC`U)vGpDT(tay8p;Qb4oQOuOtfL5)4&IFHV5sGYM4p03ZZu$S8r_P@*ps2K`Bk z)_;Sg!Ua8j`3e}Df{XnoiwGN63!#yTLeRFvwgIWH2hGSwTICgJ)@1IUh{-O^`pD{r z#C&mc_a%G$_{-#x?%iMeVDR0oy8`z;EFc0>1P}fGt1#m0oPkZz$=O3r6`@KM>Q135 z6Y4C19ua5^QNHxnq|ctiS+THJP@U=!jME-cK{X3}OLi_imX>k^zmBcuk8jl48#{zK zdG0X@AF{vX-nx4V1tQ7?2DpZV=pm-V{FIc&(gG*3yh*sGiKI=!YWi^_r!x%MU!qtL z3g2e~(p!+Fjl!<#5%SQI`n8$OAkKub08__aUiX39 z;a=;pA-hfEw`h`EO^IzL9E?;GvR3P|+8m^Uvid9}Y6cC}Er$6?vHGA zv$%=xXj5FYLbta@h(CETqgSNY8>B>)V?^w|I}-5i`^|ka9crQeq$k@IFU3yno=Gb0 zZURdm(Vv#2=|`ja^)OM=RlP1!=@XSUQPJP6Bw{3SYSl-DCx*+XvS`}g_;`#OfBkHq z9}f106MvPjH`8fqV_XVFhmPl6qBN(2S~RbmKI zt|57)qEq5_Gctz*NC2#bKZL_LXDVc`=0L23WF0X?&>SL<*zjD0PY8oGMY2L~Z|0Lb zc$aX#nB7bs7MykhUZ-KG)&v|u{7!uY`^v>boZHEQ&w2RUh9ESPAD6J6uy+X2Hm5Sn zpp~};p(~8wLzQ^KSpJxUQPiI~j&eH--OXZ*oqaOqMwBEZKz3J9v724US!I!|4l+fSm?C~%*L!yilC*v1yo#Fvo_qgySoH;g1bWq8eD@m!QGwU z?j9fz0tC09!Civ8ySqDIXXegKX5M*o|9{=xtJmqKyPjuP)!wJ}scQDQ?`tSV=--UN zRd<`nPLHrJeg9Nr?riZCu$8>noH2;=pp>phC8ZBISq}!=LD-Sc4K4I)|JO z6J-GuS0Ny|XUE?@@D0`&94Wa*#9lxS*(|kRlnW|8uA#KOPgjKD9dBO-&X0Duro!wB zu%DN3`hLwl5gRs2DoMb#A|s+}kd?dED=Dy_8Drt|zImD)#122D(Oy_siZ;{e9@RHy z(a7@7TJ~J)qX{hYu+@R1KEYj|b~Rk^$?55OH!h^ZhbeL$pL_E$nx9xMkE^PSM6g-X zIOLohjro;ABk4}3c=UVBIBfQ4%J0MPBimjjSv9{FEJeILh+g`xo2akIx{?Wb->VoP zM*?OvadIHQ(ZY?5P7i(cq?!$%Og?NZg} zxO&B{>>5#VgtPvne3@IqAtUj+Jv)h3T8*ebz8+dRncIJq-?}m^6Jr61sjTcnc96g1 z`-g@|xOVzktzdw4%ETKv(IWH4^`lTa!mjRg9UuWv_$`@{il+P%DVuqGhj|9qDG_+o zdg*LM4Mj(tU@`${fzEX}ULyoY{O0q>S-C1qgH61!6?2MDi|$~z$aXEMkSDSD79PL2 zM~d5_O3A_9;7(Z#<=j#rHBZ{N&T#W2N`I+TXQ)j=LNw#sy_re<*GQ4jv4tk}QBq{V zs?u6SPk`eRDDF2JiOvDSm!Lh0?@5Xs%g=IA@eI49$wg7baJgK|lOgkQsC%0zQs0Ih z;81)>H?>P>HJ993W|7(spc{Alp4yzwnlt55;o zyf29*8rqu_sd6^9#~?HOB^R)9X<2qQ1}(TrHl@5rb>iITq{RNsN&pAC5B zOp&D;vAz|6U%{FPW9ZCxo$-NPrgGFYh^_|S1jVJa)HS0E!I%)b zt?%vhOnhvvZvB2Olx5@$qEai3D37cp=AGuz6#b{1IhN7>jsZR$V9QuzBmtZL!~;=g zr;S1o!bKF2Ix%*rljcO zm?FwTZdEV@so0ssY#7d9Tq@b8xQ8!8_EVEJ7}+^$x{wtHND(?Ln-)H=b64vE92!{ygs(zBUd+~BdZ)`I-EbBfb%=(AP?)5Tg|VH!IZsyJVq&dXL6ynYO@WGeR)Gz zed8b{J_wL&y7zrXUdqVWW-b$JuykHBlvGD%Yf zAIV=}fN9jVpm~?nk?^zvPILi>M{19~-OOp!gZB_Gl6keF4Xa5MP_e9=%{Kwrg*gbO zaMH=k&#@cRb|OYMTEjx==VJ^H26zLpQJXD=h}RYKDIXU3Q!`%`0@>M4`lP~zea;t6 zsY+q!fy#_g`h_*PhNf7lCBN_b(q#*F8dRC4BpP@p8N&BW~I+nfZDK{dE*tK>nokEC*?K+*<=sM1&KB0y= zEwb3(i4h!Tgh$ssulf437n=@XB?Fsy2Q#Km)i=eBdxM7wqEi+Y%usR}z}heC`>iyS zixs0m2)T|h=xNfiOc*vfnwU>*)4~K>>)yFe$_Nrh)HVbYjet?1S~x%pN#xUY3xy!2 z2nMZcf&gvZ!cnZsRcOd<&3h znhVp+_#ZRC4x$+iO2Zgzx%=Vi_VP(Zo|vp^nbPu-)#z!{)YQmm85tOhB`)zy=Q1TdTm=#^39ry4c;Z*DO1$hiaMo`3 z%8X_yVDqBf%_225)_B=ocX5K>l384OwbSG@sHlG3w4fUof3+DwBQQ&4NVoh!iFbW> zP$2u=gZNfR%|DRHG5wBo_W2d-TY8)L?JUEA2@KeyfWMmt$>U;h{B7r3I2&+uR=i(7fa}+W(qpb0g43%8Z z;B+Ojkm18l-?f~@Ng6~k-U&p@}AU~!rrJf2dFk?t@78r!EG!X z`&gzJnGQ61o0`bi47R4o!ZhC3wq_=wM0-h!wGfkdqk>jmx+>SoOq~dFz0M+3 zz>QB0?xGL#n=7N*5E*l<0AW`_6y0v&k!@8eP6Cr4iG7*fUTnSeMev*#=5*kf6Fu-HepjthM@BRtF>{=V3#}Nj%fSP^ z;^=(4bqU15L3u6eR>CdaHbx?A1p1HdWRhv3nN>xp?h&V${MsVEFI%DHhhr~G*{umXdauimo(sP9TV3{NiSZ_ zKAe9)xzpvevp@=GkS)f(BH*wjf{;FVVOe5fs#C(j(ysl=cZJM7W_d@5(p2hgGR~YgkLZ)W|7XniRPVLwP^sxtn<&1vqL}8TQ=6VB36pnw zM=HFdx(R2}6MzY)TP|2*&r+(VRF8Q;6cZ0T@=Nc5!(XxuTa{BTS|y?0uni5<^Orx_ zQzbwJ_iMD*%iA0K(!WYLxq^z4_yQoLxr!uw^Qe@(e{k&H!eA9m8mGa|rl0p-RhjrC z$DbrBL=v7{tS(mp4OYuNF+SFalI4s|>+9#`Igx@wpD@K!_uhsU2>q&KaAO?sw{keK z6R#VLYYX5zH$+G58X{iH!_+k9#1KTO2sySM*jI{vznd-2zM;1qd^f6HL(qWL7bm+? zzclU_EIR8_0E0OhKKS%-zG^;O(|F&e|5a!0=1%8ZRQ@sPg!4G_@ck)_f4K|qIRa>p z^5|Zdwc+dA)rhw)K3&w|zBann8*(1b{a#E+6uFZD;gyB-RSzVY$L6l2mdfox30+Yz z;~PZ%bP&XjAEsFxLJgaG)+xA$d#@zPUd{#45^6Czjc@OI3q78(3ME3Y!&o^#=o5OUuaXoqf@TMdSe$(lQfjw@`*~N^m>T>7jP=ZO;A33?* zGBB1rWR{$}yi&MOp@sCMv2!pnqdRB3A9m-%O@W*7{BAvs%+xc^+(EW6Nds);+Wv%D ziV_JkJ@2#0e5;|c+^3}8mB)#X&c7(RC;w6^4_ZE#Wg+eLt;pyzN~fN+l-X^^^%hCx zh^E2je15pL^5fC|rFZ%&)Ld+e0xWj!)QJiIw;92S#bVQF57T;waa=`8$G*1)@_ilj z^58SgXBl-ghsE22$*cmoU6zQodSSK4zJl*{)w|cAI}XMl*AKsYB?%ZqbU-Ct?<=rS zR~zX!lDxIrR=bGwsz#~PB5zTMnJ>FiGT$OtrGh9Tt&5pb`ZAc~qFma2mahAD)n&!S zsS%w+(b8O}{zNX0EKgv9mWX92;%8kzP$h#mJsP&ve|M^la4W`q}EoA(F zU0Z+pN_fh;c3U(@w}rw)ut*&8-~7yH*|4($VQWI zvujO`DmuN8m03)q8R0D04AD@_IR5#l zyCYl9T?;c3hsomnJ#afQHM}(kZOH2$nq_>etji;d7&Q=2_>Av`hp0qYqp2@N>LE*5FpE@e zLka|{lsXdO9^B=boz;f!qOt7iHJ&crbp`bVMC6P__K?)LVBS2-2{hjL$or7u?|7)_ z!3)rNe;f|;@9c$jXYG%+oM?gLb^650F3OfCByftkDFrRH2?Nd$G#}W9uF@QDUzgWIii=AifLqaaOHG`DZ>~jyYsEk*@m^7Oh+F@4W371Li_4Lsur%>v1!1TxsDz z>BC~dDYnRWESk+wQaq8w2pQ1wwW!|Negq|MIwd4owY^w{O%fs0!|$&xk&HEeGu;H;vp4wV^*W!qUY1c*NoXI% zq`P=^l==Ki)or!^*N)Ui-E#gUK3jNDKASmj+}$94^unDRpt&`ikd^%;HQLHBCC=va zq&-?bYq>h?!#szo%;sasUUkPt##`7<5AZY~n@{KQR6Y*hLZV0DAyTg3H8#mTc+atXc_)ZtvVc?|yR6nEZZ6IJIuOP?yn>5fFV_Sd^aB5_?~HGsP7Zh$8Ly zL0n4OF-^SXGo@$A^mrN2$+CjUox3?>+Zg)beD7pD7qL%9G5}5(W=Vn~ z0aXRgn{fjsP$c(>TidjqP~p3&TKjc>w!MabK$4+%8adJReeLiK!@2X*2d*ysd~6o+ zmJMwJsv-FFO-7;)OA=X0>cqYyBmZ=G#KAUbLQV;GPd#;lP{Eva>P!Y4Ik%bmFJ}S~ zrCoFKxxOLGBwNEKkqzC1OYNHcb`fTuIwcy}G^t)%0!0fv!Mg@0?yroDE;=udB6a93 zkK1IZxjBga%I8l#Lws5y1lyyB^=yuwtfm~<>s#|n^_O~zFpz`as=(hK3iJblP(E-}-Wc^~i(@5jFdV8fVy1-SIaAq===#3x|iScP0bE^4yQwKA50<;vv zwu*%$hd^iJ^Y;EjiMM-uJmwkeL8jhmv>@+s$4BE5h~@)@8TzzgY~@dj@3ZyjFVw0$ zMJdI~5=wEKz_Jj&g=sykY+klHvW%OQSDA(_Nq)m!e#EGuzbkkSxwQ7=dgN3oe*1hh zAHG7#XO%jJMzKVftEIF_Z23s_HNpeTuBbs!ob{Pdr`4JN#c{3Ptr%pHd}HYiogp$a z8P$$m*;jjv1*LPJ z9u3ZnEHRf}ogGe_!3ub`v|N+jZ&V^J^FORYoS)VnuLn2U$wMN!9vvF-#yQ+;>N19~;nf)v;rtTnL9e*a`~lp8 zdObkL?@R<&(*PE7zG>$*?s^%VXcr8`JIyM56Lp+f^ZK0ShxM=*+maH2jjqr^Z`=U{ zvDB|z*)a5C>Gj4eS%xg|d@n*!qy^TX=A}<1!*Rqlrg8SyUM9N6CB4cpd5wd!<;12j z27$5EwR6?1!|8V3-6M4GTdKHmOQB0KRw$Lim4$15q`7hrH1NBPJxyD>*W~Zz z=i8#jTUsb@7__YM zVz~|?3{igF@uhf^qWlhp>-r1sdn(;iApvPB>Y>{(oLl2Tbux&xcyAd`hTyvpWTiKs zG@hU)`>5rh00n;Z&I1^&WZM*d+^4&KVor%mD5YF2(4&k9s8Z?_dZr}HcZkz38Q?un zK2|xkqBlU+REREg`zP8S>?eI0z``f#bfo`kzMBB4KEDGZANn4m`BbKJS9LF1*Y%?+UBGAuRZLl&E>0&Fsx?t>f1ACFI@d20j;q2|VVoL(1LmMeHF2ZZIQEVJ! z?b>b!ZrwQqk#DrytzC=$ng%{@t`FA`oZikK-IAP&==|fM>RyA@>ICJ%ma%%K29=GS zaJ21jhwrzsKy(amw?Pq#x0bDTO~TwcIi1e)-Cf?@@eYa(aCpyBXdj*>E%f=3rSVeA zv(N_Z`x+2_z@5apVTcQt-OmyH==1aP~z5VCK*|AY!~;n0&h^9d=Hb3 z2Th-&Z=5^~&8}mnkD)>M1@u25U3jV z4rMzJyxf2RJ?R+4av&f}=0)=Ya>YD6+JZk-j)!SWvr@JNTDun$OfMqPr zik2z{1!?CMkvaNgwyS#1W_km`%%q|&ehICh7yk;`=f$vX_D}#!oGbF20z1_1izgzq z(UXqBd)>BFQQ-%Q8lUFreB@$p{L87G9%!>hTB`Zs%vvuJv*!^NQ&- zNMS0m1EvdnInrd(_WI4x%pVzaPFC%d;RAEcsEH23!{Mj6a!$H18kX&5H)Yx{l!yqY zxYfRiyXL}^S?f4-=#tz=koO*+nC2_;ZwLjXa_g1MY>@>Q>8?jBGJ>M(ga+~j<7@9c zn%alLbRHpz6T-4G4qyMq_IfC5Z@_J7jdkJ6Udp*01-QAPIGdC{_|_GaRPE|zw83cd14KEz2^@V00Ce zplx=7o(pB6e!LhM9^;4!X(2P|CIxeyoz3iGP}JmCd9)KB3hZ;4&?O8fspuBJyi#im ztxS$cP!S$e(e50#lWyc;T*Zu5d+*Iiy;4!A2w6xMikXA?IV&dVlcHi2_=*-p!9V~k z4L+L5YXZZM^no)@8u{`@_NG~58@S+2v|osC*~jpe(vB`aVm=;Y>K44-aeL3{66L9I z2S@z5^B$aK(~Vs%{zGd(-i9gTae34`w&n^MHJWW#3_UZR0WR+nR`I?v3f)i?vz3L0 z;E5un)59VPR;92LJWcjD(qrJNw7kgsI|3pWrQn>t&f~Mq#B1M{<5$Dacb%=-i;9D^ z^EPBe>lVJC;2nsV=R!2G>c1b4ZT+r_U57*N@-nrt2@!dsddY}>>h6wvyfA)EVi|nK zWN6Um8A?$G26o^98PBH~1wgY)2~d=QfW!pAf*$^*;qri=8YSqpoP>%9qqMvvvzZN( zHPF$~jma5k?_g$Q?Z9MfZ(|FzcQgY!L`I1ybdsV<9zSAHc&O0HX3txV`HvR32wc7V z>?k9e4c1`y+*AOAAuus-9Cy9s0=iKLjI29FMMGpR%zqMXxG|F}fb0qXUUBhe0% zgPW9Or!2XC$nO}w*F1~Q)z=LD%;C~49N%`e3+`%ofk^kkR6~Nf!OYVL-Sx^jYKa?- zA1;p`a3G2|RXVu8NbbEz&uzsIhJGV2_5wFV*Jh;1=leD`ACRMA4dRE%cFiymRzL>n z1q1wMGWg#N74i?0q5NzzD+6bsHS-Vq{`;44|MSa#Sos%H+`o|e%|nciM%D(7X3juH zLmPWD>vs+gmW&RLpuZUXhWfj&Me_gdYoY%Gs;!L`ldF~GA)&6rJm*JyTktF})DX z?A(|PPu7cT3y38SU#F}O>Ezq$hRp@kx`zzwV!xu2=3kAAs2hI8(5pm20mqLm!uFyh zrs1ST8Oc4D1#hjF{vOILOqgR_*W>CdzTjsW^pF&axJ}!tz=8IZ6W=;R{JRg(Ov*P~ z{Fw!Pi1_IbmN6j)vS69*A*jugZZx9n)3a-^shxH=b9P(j^>SF>YeKr^8^pJ(=Z`M# zh!9MNpqr39sVGBdj^UzHq=IBb#YjPSNGF}ncOWMh?LqSw)JH^p8SRQ_&rPlrvBne0 z{z#l`*BLC0XiGoz*|`1e9MH;zAEQ;I(C~=wHooU_FSSv~cXRY`C~VMZuKTve-QPN2 zVTf7!{+r7Jk>jw(iIb>SQ&u0!;?gOaK`UEI2xB|!G$6o!gZA8V>|L*uoYw2kSEf0} zy*h$xOvOyN)Q*F_8ot~R8klgJ?3?J6^nT(f6Gr_;`0KUpU%#>wq%k5mva-WNr4jT> zeh)Ex{f?m*VSVjk@V4FC55I&|kC|fl;~U%aVXTEB9Ok#YLKzuX1}h0#{4uP1@DXtj zUsWRW+f@?|S@CH76_KV6!uN5enGXUl%;hu`yy*D1o(nj>eL}3)^-$%G!)FSMno!H4qF6-WUf_saBD0i@)>u4>uKj}fFt99-8io4as($itADFcoLBTYD-mJgV3+Io4?mT0cfk zSsuC(rcOzK588k0-gcLK zGIgy}KB{(`u?=cT<`>i65^Nb{Jx={V#M&=4I6-*zL0A(iLkrCpjeh&o#1M#;j`F1j z!t}L0pqn#ocE}miM%E&Gdg{)sN)E7KcX<&zyud!=P8Z6^F}0ozsHhnZ*0V_1dlv>K zbf#f!t+3%pYBiY3P(qzPMZ1V-NPP3&eQ)M%sswFZFEDkkjBNI$2mzH`!#EKYl4J#x z1I8w=C0i9iA{WVl=!=nIw3=%01_UhGI+WtF+8K3wC&9 z>xTv6;{?)z%0=4F*V>oP49Y;3IPH&tuo9@kT!UO1LB+cjc;|0~<}@2$*1DyTvqP;f zNqS=h+`A91Pw%iv+oru#I;=o7VyO-eEYvR=uNNDddnop-dP<-8eY<0clL zWp|SB(qyJFXuvNkTTOI0iyES#$qAwrJxVZqcI__RbR#f97UQ&`x~E)sqO$}(Hx_bssYWwSR6LNBLerVmNA$guYO=s@+pmR!7{RMTJWx3m zek@3zEk|(mh3riI2_=OauVH4$PL-U1PFCJbi3D}WSd|LzSN;{&e8FdbBFk7`dK81O z8FI`w3-D{2?dQ$obOf3 z@eNLvC6T@qV11Iyu#V)W#N@>b?V75#baP?N zlYlS0x-bFc8=8QNeBu|zyEJRmO9-rYKYHG+u24!ogF#4vLsxct(MN->{7?0IypqphKpqqX(oGL^%Li|AN5w{ zdntI`uL+nTf>FJRlCH)SA4_{aRQWY@_HlaK`bW?yuOb4?uVLdB3hI^JJGAWG*TyRb zi+7R7WJao|V*!CMYnlv4VPCX%F(K*Re)T;jHPm;hoM~!UB!3Hc>(+`R1(^dgwwD!) z_7bp&5-kIStAV1h-F(gDiMT43I z2wuXMU}=YkLNUZXWW%c54vTHoX?TyosMrmwByPtluC% z#5}B}*+WQ?U5j!QH~4bnLp?Lf=1x+B8bgYP`p8huEAdELUF6h6NP3jv7gKh9?Uhkq zfrnGAi}&rbO{~tihqAILAG=UKcaX^W3uyqIwVH{j3zaFgH*vRe4R?xS(OgN%N5Tpp zcMFxV9?5;W&s^!eY&keI#w>4p4y&e6GWNsIn+vV#j;^BkaENYD4t%d}?ebacJm+V- zLcs%UmYfx$%V31`9PyLqz6t^0OwTU+%=YKK*Zb}{9*gYT`(wUx6z&fUndMI&=hDmv zzj)ZqM^BLKI|{g%Cq`LH?BSn(aX_cXf@E@(_9Nq^D-2BjQGi<=dR!^NKlV^ekq={|nR`C}J$X%+g0!#0x?85wWy80C7%LHldVm zT;kg=k!=|YL3{9WlZM>;9DK;hFAc`rZug4p8;aideH8B0JbvW)mR?Za&Zr`th8QXh z+FPOz-q1zzYkH74nj-KOyN|*xDw{W*nD-Q3b+PqvCu^~Kk@p#~jK31O`)>Ar4p-NFflHgJDihoz7VE$4G7!eCqRj@G@PwBRisJpRXuG$B;an3OD|2)mHuEV!|;ZD(3;$c zwP_@CRenNKwaSh{MiNJEqo|=C@i7Um`*oWEM_4RCv^)sQ6aw>B0k?Tdo0osymG*rx znMB2V8c{=~w*Dp(*j29lv4Sx*g;CpudyRVm%#U1m){KvIr%-IVh9#ytYE&`7W?I-= z@r)t)_yM!4}Rpz9(PvY5LVZO_dd#WQD)Cx8wK1 zVDJ*N=@;KjQFt5PM~1J$#b!(OZgO*sN@S+Iw-z_Rea2BYTS6sa z)Q~=8?sT==xJQGn#Nts$@}HG`(GYS_49s3$RGsu*qR)|X+0 zkTsCfeq3%cQ&$fxb^y_I{A&?)?6NF|r08qdCqD$#`hsAKI0?3+mpwT8JRkT&k+%$L zFz>KStPwpH{eav1mHR_iqI*MkY2#a7ZwU|7>!1h{M2d0i27(cBl^B<{!9>!Y_`boj zXB~GaxyxcjM#V<(U2auZZ8cHtmCPs$qo)HrcKYB_Ex|jX6R93=cEI>;=Vkl z0I}k$&Zr^G5BBuU;OomXzbSADxtA6-kI7O9(kbqq*KN4@fzOv43!9mU*kqKrab?RW z@izvZU&m#=9`#0t!ybH~>F_r;HC$C7?`p3{i8HFFJ@S(&$XBthbE`cU;9i;Q0^WHE zaK6XvzNt&!sQl#fPR6S|q$s0bKq4a3KWYGYe(SJZKSW5S zxOC9le}O0yf5}iIMR3aOWAu8tz>bxZ)_uI5Mjz6IJDdgfo{CO1mag9}+6!3Dw(rZT zs4aGjszDXY$`_?C!xK&P9J`7U@q0bwG zwA@NUAp%|BX7wXdFZ?ynzm(kz9%9s>xaHZ%+PTN-=xSra$);&`Kl03&mr;56<|j#q z45y^Ld*a6yq-6GT)dUnRCbegZHqCo96Dh!9y^<1Ws~psw-#h+Ru#a-WtnPEN>9#lv zVCue+EQ)!$=eJr-Az60MNd-Am&%d5wCRU{5| zKBt67%0}8_cXZK;(p$2R6h z@6@$eYF~ks?{QiCZoPZ{Ern#9zAQw91^}2F{+>emJrfiBj}(&R|1lHuS6Yq;G@Sie zPyr(XdUh)x^hJs2xD&>bkTdpBKDd7ctMA=Jytz(nK2=3AL{~}EpIHqIoKbmn@gJaD z5rACg%)X?qTUrtbn9GtQ>tnUU(K!@dKZ>5>s78AiIH!Qm<;mI=0%wSE~!+AkA~(nB%~eGD>_a{bKxgEGuaL!*pLU||&d_`7&J zc&U5!pdBc;c7T-FcDD^d++62gU_axj-w)KkyWcbb1IL8;_dFgP==0wjD&SD$$E!cH zriwB@#Qt(L)qXbf7B~RF2@L>X{*Ta)L#`k7&th^S^3swL$|_89lD{MTPW(^fLHvpE zgIEFd*31T!-{gD#RsN5p zfuw+-wCxWQ|Iu>&m_X7XbN#OUpKAQs z4N)f(6QDhlk)?ry!_Qv)0m^jQPY=qM15}U!0GdA#LC0J_Kd4`V%9NL_XQN?~_;Z4kp3><-KKzq=u|JLt9+;05}kRgge9{7>){iEgj z1rhy{L0MT=(ca7%L?8onlLI=M+8B!ijqKfQ9aRh*EdE`cV!|&dsUR|A&~X2|k#b4? zH?seZN|!T`a0Nn?0!^U58!4Xbe?$GJ3jf~kU>B=OR1m`o$VfkO?SHgfzkQ1TzYPCf zkNpbH;wg}Iwm>ZWe^7vqxqb$qVfu@4`7f?eHhX9N-o+X? zODbic!;i|FikXoG&=E8d|6P-MI+QUEkS2s6ioeg0GqGR7|D?&U6_7e>pn(UWPyfdZ z2?Y(XUyR*-&wd1VaRSkCv8K}Xrlkq6+iq= z`}&tGpm``}0}6hwj{oU~A7S^MPKR*Qdo;jOF!evY?5_BqVaw9Y@V7vu_ecEEvoy1I za@BL?=G5b4XL7LlIRyQHzeDq?qHO?mdgR4F;FHGD{#t+hPgMK|wu_myv5kwvALz!G zmj8tRxj_8)8B82S@*nX3*?Sv@Kl^TOZ3g=MXM!Jf=AX-{lz+4PlPvRJ1b@^+|E=o$ zJLJDF$$r#fe{VllMy&EbLH<#d{cDx<&m!)Ry6j)VZ~s&Hk6!(4t@2NlzhcLaE+qT{ z^UE-1=Cfg&6LKn{9jB7kCoAv5UR{{x4YrcD3< literal 0 HcmV?d00001 diff --git a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java index d26abfdb2529..c390ba4287e2 100644 --- a/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java +++ b/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java @@ -132,6 +132,8 @@ private static ChannelFuture bind(Initializer initializer) throws ConfigurationE ServerBootstrap bootstrap = initializer.settings.socketFactory .newServerBootstrap() .option(ChannelOption.SO_BACKLOG, 1 << 9) + .option(ChannelOption.ALLOCATOR, GlobalBufferPoolAllocator.instance) + .option(ChannelOption.SO_REUSEADDR, true) .childHandler(initializer); int socketReceiveBufferSizeInBytes = initializer.settings.socketReceiveBufferSizeInBytes; diff --git a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java index a63ccf92858b..fdfb2dfa74e1 100644 --- a/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java +++ b/src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java @@ -171,13 +171,15 @@ else if (future.isCancelled()) */ private Bootstrap createBootstrap(EventLoop eventLoop) { - Bootstrap bootstrap = newBootstrap(eventLoop, settings.tcpUserTimeoutInMS) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, settings.tcpConnectTimeoutInMS) - .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.SO_REUSEADDR, true) - .option(ChannelOption.TCP_NODELAY, settings.tcpNoDelay) - .option(ChannelOption.MESSAGE_SIZE_ESTIMATOR, NoSizeEstimator.instance) - .handler(new Initializer()); + Bootstrap bootstrap = settings.socketFactory + .newClientBootstrap(eventLoop, settings.tcpUserTimeoutInMS) + .option(ChannelOption.ALLOCATOR, GlobalBufferPoolAllocator.instance) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, settings.tcpConnectTimeoutInMS) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.SO_REUSEADDR, true) + .option(ChannelOption.TCP_NODELAY, settings.tcpNoDelay) + .option(ChannelOption.MESSAGE_SIZE_ESTIMATOR, NoSizeEstimator.instance) + .handler(new Initializer()); if (settings.socketSendBufferSizeInBytes > 0) bootstrap.option(ChannelOption.SO_SNDBUF, settings.socketSendBufferSizeInBytes); diff --git a/src/java/org/apache/cassandra/net/SocketFactory.java b/src/java/org/apache/cassandra/net/SocketFactory.java index 18bb0d5c70f0..062c44ba63d7 100644 --- a/src/java/org/apache/cassandra/net/SocketFactory.java +++ b/src/java/org/apache/cassandra/net/SocketFactory.java @@ -18,13 +18,14 @@ package org.apache.cassandra.net; import java.io.IOException; -import java.lang.reflect.Field; import java.net.ConnectException; import java.net.InetSocketAddress; import java.nio.channels.ClosedChannelException; +import java.nio.channels.spi.SelectorProvider; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeoutException; import javax.annotation.Nullable; import javax.net.ssl.SSLEngine; @@ -37,11 +38,11 @@ import io.netty.bootstrap.Bootstrap; import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.Channel; -import io.netty.channel.ChannelOption; +import io.netty.channel.ChannelFactory; +import io.netty.channel.DefaultSelectStrategyFactory; import io.netty.channel.EventLoop; import io.netty.channel.EventLoopGroup; -import io.netty.channel.MultithreadEventLoopGroup; -import io.netty.channel.SingleThreadEventLoop; +import io.netty.channel.ServerChannel; import io.netty.channel.epoll.EpollChannelOption; import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.epoll.EpollServerSocketChannel; @@ -53,10 +54,10 @@ import io.netty.handler.ssl.OpenSsl; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; +import io.netty.util.concurrent.DefaultEventExecutorChooserFactory; import io.netty.util.concurrent.DefaultThreadFactory; -import io.netty.util.concurrent.EventExecutor; -import io.netty.util.concurrent.MultithreadEventExecutorGroup; -import io.netty.util.concurrent.SingleThreadEventExecutor; +import io.netty.util.concurrent.RejectedExecutionHandlers; +import io.netty.util.concurrent.ThreadPerTaskExecutor; import io.netty.util.internal.logging.InternalLoggerFactory; import io.netty.util.internal.logging.Slf4JLoggerFactory; import org.apache.cassandra.concurrent.NamedThreadFactory; @@ -82,8 +83,85 @@ public final class SocketFactory private static final int EVENT_THREADS = Integer.getInteger(Config.PROPERTY_PREFIX + "internode-event-threads", FBUtilities.getAvailableProcessors()); - public enum Provider { EPOLL, NIO } - private static final Provider DEFAULT_PROVIDER = NativeTransportService.useEpoll() ? Provider.EPOLL : Provider.NIO; + /** + * The default task queue used by {@code NioEventLoop} and {@code EpollEventLoop} is {@code MpscUnboundedArrayQueue}, + * provided by JCTools. While efficient, it has an undesirable quality for a queue backing an event loop: it is + * not non-blocking, and can cause the event loop to busy-spin while waiting for a partially completed task + * offer, if the producer thread has been suspended mid-offer. + * + * As it happens, however, we have an MPSC queue implementation that is perfectly fit for this purpose - + * {@link ManyToOneConcurrentLinkedQueue}, that is non-blocking, and already used throughout the codebase, + * that we can and do use here as well. + */ + enum Provider + { + NIO + { + @Override + NioEventLoopGroup makeEventLoopGroup(int threadCount, ThreadFactory threadFactory) + { + return new NioEventLoopGroup(threadCount, + new ThreadPerTaskExecutor(threadFactory), + DefaultEventExecutorChooserFactory.INSTANCE, + SelectorProvider.provider(), + DefaultSelectStrategyFactory.INSTANCE, + RejectedExecutionHandlers.reject(), + capacity -> new ManyToOneConcurrentLinkedQueue<>()); + } + + @Override + ChannelFactory clientChannelFactory() + { + return NioSocketChannel::new; + } + + @Override + ChannelFactory serverChannelFactory() + { + return NioServerSocketChannel::new; + } + }, + EPOLL + { + @Override + EpollEventLoopGroup makeEventLoopGroup(int threadCount, ThreadFactory threadFactory) + { + return new EpollEventLoopGroup(threadCount, + new ThreadPerTaskExecutor(threadFactory), + DefaultEventExecutorChooserFactory.INSTANCE, + DefaultSelectStrategyFactory.INSTANCE, + RejectedExecutionHandlers.reject(), + capacity -> new ManyToOneConcurrentLinkedQueue<>()); + } + + @Override + ChannelFactory clientChannelFactory() + { + return EpollSocketChannel::new; + } + + @Override + ChannelFactory serverChannelFactory() + { + return EpollServerSocketChannel::new; + } + }; + + EventLoopGroup makeEventLoopGroup(int threadCount, String threadNamePrefix) + { + logger.debug("using netty {} event loop for pool prefix {}", name(), threadNamePrefix); + return makeEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix, true)); + } + + abstract EventLoopGroup makeEventLoopGroup(int threadCount, ThreadFactory threadFactory); + abstract ChannelFactory clientChannelFactory(); + abstract ChannelFactory serverChannelFactory(); + + static Provider optimalProvider() + { + return NativeTransportService.useEpoll() ? EPOLL : NIO; + } + } /** a useful addition for debugging; simply set to true to get more data in your logs */ static final boolean WIRETRACE = false; @@ -93,94 +171,42 @@ public enum Provider { EPOLL, NIO } InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE); } + private final Provider provider; private final EventLoopGroup acceptGroup; private final EventLoopGroup defaultGroup; // we need a separate EventLoopGroup for outbound streaming because sendFile is blocking private final EventLoopGroup outboundStreamingGroup; final ExecutorService synchronousWorkExecutor = Executors.newCachedThreadPool(new NamedThreadFactory("Messaging-SynchronousWork")); - SocketFactory() { this(DEFAULT_PROVIDER); } - SocketFactory(Provider provider) - { - this.acceptGroup = getEventLoopGroup(provider, 1, "Messaging-AcceptLoop"); - this.defaultGroup = getEventLoopGroup(provider, EVENT_THREADS, NamedThreadFactory.globalPrefix() + "Messaging-EventLoop"); - this.outboundStreamingGroup = getEventLoopGroup(provider, EVENT_THREADS, "Streaming-EventLoop"); - assert provider == providerOf(acceptGroup) - && provider == providerOf(defaultGroup) - && provider == providerOf(outboundStreamingGroup); - } - - private static EventLoopGroup getEventLoopGroup(Provider provider, int threadCount, String threadNamePrefix) + SocketFactory() { - switch (provider) - { - case EPOLL: - logger.debug("using netty epoll event loop for pool prefix {}", threadNamePrefix); - return overwriteMPSCQueues(new EpollEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix, true))); - case NIO: - logger.debug("using netty nio event loop for pool prefix {}", threadNamePrefix); - return overwriteMPSCQueues(new NioEventLoopGroup(threadCount, new DefaultThreadFactory(threadNamePrefix, true))); - default: - throw new IllegalStateException(); - } + this(Provider.optimalProvider()); } - private static Provider providerOf(EventLoopGroup eventLoopGroup) + SocketFactory(Provider provider) { - while (eventLoopGroup instanceof SingleThreadEventLoop) - eventLoopGroup = ((SingleThreadEventLoop) eventLoopGroup).parent(); - - if (eventLoopGroup instanceof EpollEventLoopGroup) - return Provider.EPOLL; - if (eventLoopGroup instanceof NioEventLoopGroup) - return Provider.NIO; - throw new IllegalStateException(); + this.provider = provider; + this.acceptGroup = provider.makeEventLoopGroup(1, "Messaging-AcceptLoop"); + this.defaultGroup = provider.makeEventLoopGroup(EVENT_THREADS, NamedThreadFactory.globalPrefix() + "Messaging-EventLoop"); + this.outboundStreamingGroup = provider.makeEventLoopGroup(EVENT_THREADS, "Streaming-EventLoop"); } - static Bootstrap newBootstrap(EventLoop eventLoop, int tcpUserTimeoutInMS) + Bootstrap newClientBootstrap(EventLoop eventLoop, int tcpUserTimeoutInMS) { if (eventLoop == null) throw new IllegalArgumentException("must provide eventLoop"); - Bootstrap bootstrap = new Bootstrap() - .group(eventLoop) - .option(ChannelOption.ALLOCATOR, GlobalBufferPoolAllocator.instance) - .option(ChannelOption.SO_KEEPALIVE, true); + Bootstrap bootstrap = new Bootstrap().group(eventLoop).channelFactory(provider.clientChannelFactory()); + + if (provider == Provider.EPOLL) + bootstrap.option(EpollChannelOption.TCP_USER_TIMEOUT, tcpUserTimeoutInMS); - switch (providerOf(eventLoop)) - { - case EPOLL: - bootstrap.channel(EpollSocketChannel.class); - bootstrap.option(EpollChannelOption.TCP_USER_TIMEOUT, tcpUserTimeoutInMS); - break; - case NIO: - bootstrap.channel(NioSocketChannel.class); - } return bootstrap; } ServerBootstrap newServerBootstrap() { - return newServerBootstrap(acceptGroup, defaultGroup); - } - - private static ServerBootstrap newServerBootstrap(EventLoopGroup acceptGroup, EventLoopGroup defaultGroup) - { - ServerBootstrap bootstrap = new ServerBootstrap() - .group(acceptGroup, defaultGroup) - .option(ChannelOption.ALLOCATOR, GlobalBufferPoolAllocator.instance) - .option(ChannelOption.SO_REUSEADDR, true); - - switch (providerOf(defaultGroup)) - { - case EPOLL: - bootstrap.channel(EpollServerSocketChannel.class); - break; - case NIO: - bootstrap.channel(NioServerSocketChannel.class); - } - - return bootstrap; + return new ServerBootstrap().group(acceptGroup, defaultGroup).channelFactory(provider.serverChannelFactory()); } /** @@ -270,58 +296,4 @@ static String channelId(InetAddressAndPort from, InetAddressAndPort to, Connecti { return from + "->" + to + '-' + type + '-' + id; } - - /** - * The default task queue used by {@code NioEventLoop} and {@code EpollEventLoop} is {@code MpscUnboundedArrayQueue}, - * provided by JCTools. While efficient, it has an undesirable quality for a queue backing an event loop: it is - * not non-blocking, and can cause the event loop to busy-spin while waiting for a partially completed task - * offer, if the producer thread has been suspended mid-offer. Sadly, there is currently no way to work around - * this behaviour in application-logic. - * - * As it happens, however, we have an MPSC queue implementation that is perfectly fit for this purpose - - * {@link ManyToOneConcurrentLinkedQueue}, that is non-blocking, and already used throughout the codebase. - * - * Unfortunately, there is no Netty API or to override the default queue, so we have to resort to reflection, - * for now. - * - * We filed a Netty issue asking for this capability to be provided cleanly: - * https://github.com/netty/netty/issues/9105, and hopefully Netty will implement it some day. When and if - * that happens, this reflection-based workaround should be removed. - */ - private static EventLoopGroup overwriteMPSCQueues(MultithreadEventLoopGroup eventLoopGroup) - { - try - { - for (EventExecutor eventExecutor : (EventExecutor[]) childrenField.get(eventLoopGroup)) - { - SingleThreadEventLoop eventLoop = (SingleThreadEventLoop) eventExecutor; - taskQueueField.set(eventLoop, new ManyToOneConcurrentLinkedQueue<>()); - tailTasksField.set(eventLoop, new ManyToOneConcurrentLinkedQueue<>()); - } - return eventLoopGroup; - } - catch (IllegalAccessException e) - { - throw new IllegalStateException(e); - } - } - - private static final Field childrenField, taskQueueField, tailTasksField; - static - { - try - { - childrenField = MultithreadEventExecutorGroup.class.getDeclaredField("children"); - taskQueueField = SingleThreadEventExecutor.class.getDeclaredField("taskQueue"); - tailTasksField = SingleThreadEventLoop.class.getDeclaredField("tailTasks"); - - childrenField.setAccessible(true); - taskQueueField.setAccessible(true); - tailTasksField.setAccessible(true); - } - catch (NoSuchFieldException e) - { - throw new IllegalStateException(e); - } - } } diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java index 667496388c1b..2ccb1268e62e 100644 --- a/src/java/org/apache/cassandra/security/SSLFactory.java +++ b/src/java/org/apache/cassandra/security/SSLFactory.java @@ -240,10 +240,12 @@ public static SslContext getOrCreateSslContext(EncryptionOptions options, boolea * Get a netty {@link SslContext} instance. */ @VisibleForTesting - static SslContext getOrCreateSslContext(EncryptionOptions options, boolean buildTruststore, - SocketType socketType, boolean useOpenSsl) throws IOException + static SslContext getOrCreateSslContext(EncryptionOptions options, + boolean buildTruststore, + SocketType socketType, + boolean useOpenSsl) throws IOException { - CacheKey key = new CacheKey(options, socketType); + CacheKey key = new CacheKey(options, socketType, useOpenSsl); SslContext sslContext; sslContext = cachedSslContexts.get(key); @@ -413,11 +415,13 @@ static class CacheKey { private final EncryptionOptions encryptionOptions; private final SocketType socketType; + private final boolean useOpenSSL; - public CacheKey(EncryptionOptions encryptionOptions, SocketType socketType) + public CacheKey(EncryptionOptions encryptionOptions, SocketType socketType, boolean useOpenSSL) { this.encryptionOptions = encryptionOptions; this.socketType = socketType; + this.useOpenSSL = useOpenSSL; } public boolean equals(Object o) @@ -426,6 +430,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) return false; CacheKey cacheKey = (CacheKey) o; return (socketType == cacheKey.socketType && + useOpenSSL == cacheKey.useOpenSSL && Objects.equals(encryptionOptions, cacheKey.encryptionOptions)); } @@ -434,6 +439,7 @@ public int hashCode() int result = 0; result += 31 * socketType.hashCode(); result += 31 * encryptionOptions.hashCode(); + result += 31 * Boolean.hashCode(useOpenSSL); return result; } } diff --git a/src/java/org/apache/cassandra/service/NativeTransportService.java b/src/java/org/apache/cassandra/service/NativeTransportService.java index 79acab1b02a0..79caafcc7ffd 100644 --- a/src/java/org/apache/cassandra/service/NativeTransportService.java +++ b/src/java/org/apache/cassandra/service/NativeTransportService.java @@ -153,7 +153,7 @@ public static boolean useEpoll() final boolean enableEpoll = Boolean.parseBoolean(System.getProperty("cassandra.native.epoll.enabled", "true")); if (enableEpoll && !Epoll.isAvailable() && NativeLibrary.osType == NativeLibrary.OSType.LINUX) - logger.warn("epoll not available {}", Epoll.unavailabilityCause()); + logger.warn("epoll not available", Epoll.unavailabilityCause()); return enableEpoll && Epoll.isAvailable(); } From b2f6953addfb1fce111d1b49627285d1a57a7f40 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Per=20Otterstr=C3=B6m?= Date: Tue, 28 May 2019 21:29:25 +0200 Subject: [PATCH 25/78] Handle exceptions during authentication/authorization MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Patch by Per Otterström; reviewed by Sam Tunnicliffe for CASSANDRA-15041 --- CHANGES.txt | 1 + .../cassandra/auth/CassandraAuthorizer.java | 23 +++++----- .../cassandra/auth/CassandraRoleManager.java | 45 +++++++++++-------- .../cassandra/auth/PasswordAuthenticator.java | 2 +- .../cassandra/auth/PermissionsCache.java | 17 +++++-- src/java/org/apache/cassandra/auth/Roles.java | 25 ++++++++--- .../org/apache/cassandra/auth/RolesCache.java | 19 +++++--- .../exceptions/AuthenticationException.java | 5 +++ .../exceptions/UnauthorizedException.java | 5 +++ .../apache/cassandra/service/ClientState.java | 13 +++++- 10 files changed, 106 insertions(+), 49 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 48bf14f5d411..d8da35464ab2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.2.15 + * Handle exceptions during authentication/authorization (CASSANDRA-15041) * Fix JDK7 compatibility broken in cassandra-2.2 (CASSANDRA-15050) * Support cross version messaging in in-jvm upgrade dtests (CASSANDRA-15078) * Fix index summary redistribution cancellation (CASSANDRA-15045) diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java index 360d59a28e47..68d4303f9793 100644 --- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java +++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java @@ -73,26 +73,23 @@ public CassandraAuthorizer() // or indirectly via roles granted to the user. public Set authorize(AuthenticatedUser user, IResource resource) { - if (user.isSuper()) - return resource.applicablePermissions(); - - Set permissions = EnumSet.noneOf(Permission.class); try { + if (user.isSuper()) + return resource.applicablePermissions(); + + Set permissions = EnumSet.noneOf(Permission.class); + for (RoleResource role: user.getRoles()) addPermissionsForRole(permissions, resource, role); + + return permissions; } - catch (RequestValidationException e) - { - throw new AssertionError(e); // not supposed to happen - } - catch (RequestExecutionException e) + catch (RequestExecutionException | RequestValidationException e) { - logger.warn("CassandraAuthorizer failed to authorize {} for {}", user, resource); - throw new RuntimeException(e); + logger.debug("Failed to authorize {} for {}", user, resource); + throw new UnauthorizedException("Unable to perform authorization of permissions: " + e.getMessage(), e); } - - return permissions; } public void grant(AuthenticatedUser performer, Set permissions, IResource resource, RoleResource grantee) diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java index bfd048393cfd..1e5ea8a74655 100644 --- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java +++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java @@ -310,12 +310,28 @@ public RoleResource apply(UntypedResultSet.Row row) public boolean isSuper(RoleResource role) { - return getRole(role.getRoleName()).isSuper; + try + { + return getRole(role.getRoleName()).isSuper; + } + catch (RequestExecutionException e) + { + logger.debug("Failed to authorize {} for super-user permission", role.getRoleName()); + throw new UnauthorizedException("Unable to perform authorization of super-user permission: " + e.getMessage(), e); + } } public boolean canLogin(RoleResource role) { - return getRole(role.getRoleName()).canLogin; + try + { + return getRole(role.getRoleName()).canLogin; + } + catch (RequestExecutionException e) + { + logger.debug("Failed to authorize {} for login permission", role.getRoleName()); + throw new UnauthorizedException("Unable to perform authorization of login permission: " + e.getMessage(), e); + } } public Map getCustomOptions(RoleResource role) @@ -505,23 +521,16 @@ private void collectRoles(Role role, Set collected, boolean includ */ private Role getRole(String name) { - try - { - // If it exists, try the legacy users table in case the cluster - // is in the process of being upgraded and so is running with mixed - // versions of the authn schema. - if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, "users") == null) - return getRoleFromTable(name, loadRoleStatement, ROW_TO_ROLE); - else - { - if (legacySelectUserStatement == null) - legacySelectUserStatement = prepareLegacySelectUserStatement(); - return getRoleFromTable(name, legacySelectUserStatement, LEGACY_ROW_TO_ROLE); - } - } - catch (RequestExecutionException | RequestValidationException e) + // If it exists, try the legacy users table in case the cluster + // is in the process of being upgraded and so is running with mixed + // versions of the authn schema. + if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, "users") == null) + return getRoleFromTable(name, loadRoleStatement, ROW_TO_ROLE); + else { - throw new RuntimeException(e); + if (legacySelectUserStatement == null) + legacySelectUserStatement = prepareLegacySelectUserStatement(); + return getRoleFromTable(name, legacySelectUserStatement, LEGACY_ROW_TO_ROLE); } } diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java index 20f8790d4c36..b7250a87f132 100644 --- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java +++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java @@ -83,7 +83,7 @@ private AuthenticatedUser authenticate(String username, String password) throws catch (RequestExecutionException e) { logger.trace("Error performing internal authentication", e); - throw new AuthenticationException(e.toString()); + throw new AuthenticationException("Unable to perform authentication: " + e.getMessage(), e); } } diff --git a/src/java/org/apache/cassandra/auth/PermissionsCache.java b/src/java/org/apache/cassandra/auth/PermissionsCache.java index c8d777e12fb9..ddd6348410d9 100644 --- a/src/java/org/apache/cassandra/auth/PermissionsCache.java +++ b/src/java/org/apache/cassandra/auth/PermissionsCache.java @@ -17,16 +17,18 @@ */ package org.apache.cassandra.auth; -import java.lang.management.ManagementFactory; import java.util.Set; import java.util.concurrent.*; import org.apache.cassandra.config.DatabaseDescriptor; + +import com.google.common.base.Throwables; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFutureTask; +import com.google.common.util.concurrent.UncheckedExecutionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +43,13 @@ public class PermissionsCache implements PermissionsCacheMBean private final String MBEAN_NAME = "org.apache.cassandra.auth:type=PermissionsCache"; private final ThreadPoolExecutor cacheRefreshExecutor = new DebuggableThreadPoolExecutor("PermissionsCacheRefresh", - Thread.NORM_PRIORITY); + Thread.NORM_PRIORITY) + { + protected void afterExecute(Runnable r, Throwable t) + { + // empty to avoid logging on background updates + } + }; private final IAuthorizer authorizer; private volatile LoadingCache, Set> cache; @@ -61,9 +69,10 @@ public Set getPermissions(AuthenticatedUser user, IResource resource { return cache.get(Pair.create(user, resource)); } - catch (ExecutionException e) + catch (ExecutionException | UncheckedExecutionException e) { - throw new RuntimeException(e); + Throwables.propagateIfInstanceOf(e.getCause(), RuntimeException.class); + throw Throwables.propagate(e); } } diff --git a/src/java/org/apache/cassandra/auth/Roles.java b/src/java/org/apache/cassandra/auth/Roles.java index da6804b70772..4b145315560c 100644 --- a/src/java/org/apache/cassandra/auth/Roles.java +++ b/src/java/org/apache/cassandra/auth/Roles.java @@ -19,10 +19,17 @@ import java.util.Set; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.exceptions.UnauthorizedException; public class Roles { + private static final Logger logger = LoggerFactory.getLogger(Roles.class); + private static final RolesCache cache = new RolesCache(DatabaseDescriptor.getRoleManager()); /** @@ -47,10 +54,18 @@ public static Set getRoles(RoleResource primaryRole) */ public static boolean hasSuperuserStatus(RoleResource role) { - IRoleManager roleManager = DatabaseDescriptor.getRoleManager(); - for (RoleResource r : cache.getRoles(role)) - if (roleManager.isSuper(r)) - return true; - return false; + try + { + IRoleManager roleManager = DatabaseDescriptor.getRoleManager(); + for (RoleResource r : cache.getRoles(role)) + if (roleManager.isSuper(r)) + return true; + return false; + } + catch (RequestExecutionException e) + { + logger.debug("Failed to authorize {} for super-user permission", role.getRoleName()); + throw new UnauthorizedException("Unable to perform authorization of super-user permission: " + e.getMessage(), e); + } } } diff --git a/src/java/org/apache/cassandra/auth/RolesCache.java b/src/java/org/apache/cassandra/auth/RolesCache.java index 75ac89da28a6..c781ee0733b5 100644 --- a/src/java/org/apache/cassandra/auth/RolesCache.java +++ b/src/java/org/apache/cassandra/auth/RolesCache.java @@ -17,15 +17,16 @@ */ package org.apache.cassandra.auth; -import java.lang.management.ManagementFactory; import java.util.Set; import java.util.concurrent.*; +import com.google.common.base.Throwables; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFutureTask; +import com.google.common.util.concurrent.UncheckedExecutionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,16 +34,19 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.utils.MBeanWrapper; -import javax.management.MBeanServer; -import javax.management.ObjectName; - public class RolesCache implements RolesCacheMBean { private static final Logger logger = LoggerFactory.getLogger(RolesCache.class); private final String MBEAN_NAME = "org.apache.cassandra.auth:type=RolesCache"; private final ThreadPoolExecutor cacheRefreshExecutor = new DebuggableThreadPoolExecutor("RolesCacheRefresh", - Thread.NORM_PRIORITY); + Thread.NORM_PRIORITY) + { + protected void afterExecute(Runnable r, Throwable t) + { + // empty to avoid logging on background updates + } + }; private final IRoleManager roleManager; private volatile LoadingCache> cache; @@ -62,9 +66,10 @@ public Set getRoles(RoleResource role) { return cache.get(role); } - catch (ExecutionException e) + catch (ExecutionException | UncheckedExecutionException e) { - throw new RuntimeException(e); + Throwables.propagateIfInstanceOf(e.getCause(), RuntimeException.class); + throw Throwables.propagate(e); } } diff --git a/src/java/org/apache/cassandra/exceptions/AuthenticationException.java b/src/java/org/apache/cassandra/exceptions/AuthenticationException.java index ce6cb2c7602b..067f3ae2f43e 100644 --- a/src/java/org/apache/cassandra/exceptions/AuthenticationException.java +++ b/src/java/org/apache/cassandra/exceptions/AuthenticationException.java @@ -23,4 +23,9 @@ public AuthenticationException(String msg) { super(ExceptionCode.BAD_CREDENTIALS, msg); } + + public AuthenticationException(String msg, Throwable e) + { + super(ExceptionCode.BAD_CREDENTIALS, msg, e); + } } diff --git a/src/java/org/apache/cassandra/exceptions/UnauthorizedException.java b/src/java/org/apache/cassandra/exceptions/UnauthorizedException.java index 12a3f8af6767..008d793545ea 100644 --- a/src/java/org/apache/cassandra/exceptions/UnauthorizedException.java +++ b/src/java/org/apache/cassandra/exceptions/UnauthorizedException.java @@ -23,4 +23,9 @@ public UnauthorizedException(String msg) { super(ExceptionCode.UNAUTHORIZED, msg); } + + public UnauthorizedException(String msg, Throwable e) + { + super(ExceptionCode.UNAUTHORIZED, msg, e); + } } diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java index 12189282f241..9593802b7310 100644 --- a/src/java/org/apache/cassandra/service/ClientState.java +++ b/src/java/org/apache/cassandra/service/ClientState.java @@ -38,6 +38,7 @@ import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.exceptions.AuthenticationException; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestExecutionException; import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.schema.LegacySchemaTables; import org.apache.cassandra.thrift.ThriftValidation; @@ -266,12 +267,22 @@ public void login(AuthenticatedUser user) throws AuthenticationException // Login privilege is not inherited via granted roles, so just // verify that the role with the credentials that were actually // supplied has it - if (user.isAnonymous() || DatabaseDescriptor.getRoleManager().canLogin(user.getPrimaryRole())) + if (user.isAnonymous() || canLogin(user)) this.user = user; else throw new AuthenticationException(String.format("%s is not permitted to log in", user.getName())); } + private boolean canLogin(AuthenticatedUser user) + { + try + { + return DatabaseDescriptor.getRoleManager().canLogin(user.getPrimaryRole()); + } catch (RequestExecutionException e) { + throw new AuthenticationException("Unable to perform authentication: " + e.getMessage(), e); + } + } + public void hasAllKeyspacesAccess(Permission perm) throws UnauthorizedException { if (isInternal) From 177a8e91e3f0ef85e2bc3f64b0e566ace6330071 Mon Sep 17 00:00:00 2001 From: Zephyr Guo Date: Wed, 3 Jul 2019 12:18:57 -0700 Subject: [PATCH 26/78] Prevent RuntimeException when username or password is empty/null Patch by Zephyr Guo; Reviewed by Blake Eggleston for CASSANDRA-15198 --- CHANGES.txt | 1 + .../cassandra/auth/PasswordAuthenticator.java | 11 ++- .../auth/PasswordAuthenticatorTest.java | 81 ++++++++++++++++++- 3 files changed, 88 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 364720dbc5f0..ef6f49c6f689 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Prevent RuntimeException when username or password is empty/null (CASSANDRA-15198) * Multiget thrift query returns null records after digest mismatch (CASSANDRA-14812) * Skipping illegal legacy cells can break reverse iteration of indexed partitions (CASSANDRA-15178) * Handle paging states serialized with a different version than the session's (CASSANDRA-15176) diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java index 2b65783efb09..602fea4e2168 100644 --- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java +++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java @@ -62,7 +62,7 @@ public class PasswordAuthenticator implements IAuthenticator public static final String USERNAME_KEY = "username"; public static final String PASSWORD_KEY = "password"; - private static final byte NUL = 0; + static final byte NUL = 0; private SelectStatement authenticateStatement; public static final String LEGACY_CREDENTIALS_TABLE = "credentials"; @@ -231,7 +231,7 @@ private void decodeCredentials(byte[] bytes) throws AuthenticationException byte[] user = null; byte[] pass = null; int end = bytes.length; - for (int i = bytes.length - 1 ; i >= 0; i--) + for (int i = bytes.length - 1; i >= 0; i--) { if (bytes[i] == NUL) { @@ -239,13 +239,16 @@ private void decodeCredentials(byte[] bytes) throws AuthenticationException pass = Arrays.copyOfRange(bytes, i + 1, end); else if (user == null) user = Arrays.copyOfRange(bytes, i + 1, end); + else + throw new AuthenticationException("Credential format error: username or password is empty or contains NUL(\\0) character"); + end = i; } } - if (user == null) + if (user == null || user.length == 0) throw new AuthenticationException("Authentication ID must not be null"); - if (pass == null) + if (pass == null || pass.length == 0) throw new AuthenticationException("Password must not be null"); username = new String(user, StandardCharsets.UTF_8); diff --git a/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java b/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java index 37763d74de98..02850499cfe9 100644 --- a/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java +++ b/test/unit/org/apache/cassandra/auth/PasswordAuthenticatorTest.java @@ -18,8 +18,21 @@ package org.apache.cassandra.auth; +import java.nio.charset.StandardCharsets; + +import com.google.common.collect.Iterables; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; +import com.datastax.driver.core.Authenticator; +import com.datastax.driver.core.PlainTextAuthProvider; +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.exceptions.AuthenticationException; +import org.apache.cassandra.schema.KeyspaceParams; + import static org.apache.cassandra.auth.CassandraRoleManager.*; import static org.apache.cassandra.auth.PasswordAuthenticator.*; import static org.junit.Assert.assertFalse; @@ -27,8 +40,11 @@ import static org.mindrot.jbcrypt.BCrypt.hashpw; import static org.mindrot.jbcrypt.BCrypt.gensalt; -public class PasswordAuthenticatorTest +public class PasswordAuthenticatorTest extends CQLTester { + + private static PasswordAuthenticator authenticator = new PasswordAuthenticator(); + @Test public void testCheckpw() throws Exception { @@ -61,4 +77,67 @@ public void testCheckpw() throws Exception assertFalse(checkpw(DEFAULT_SUPERUSER_PASSWORD, "$2$6$abcdefghijklmnopqrstuvABCDEFGHIJKLMNOPQRSTUVWXYZ01234")); assertFalse(checkpw(DEFAULT_SUPERUSER_PASSWORD, "$2a$6$abcdefghijklmnopqrstuvABCDEFGHIJKLMNOPQRSTUVWXYZ01234")); } + + @Test(expected = AuthenticationException.class) + public void testEmptyUsername() + { + testDecodeIllegalUserAndPwd("", "pwd"); + } + + @Test(expected = AuthenticationException.class) + public void testEmptyPassword() + { + testDecodeIllegalUserAndPwd("user", ""); + } + + @Test(expected = AuthenticationException.class) + public void testNULUsername0() + { + byte[] user = {'u', 's', PasswordAuthenticator.NUL, 'e', 'r'}; + testDecodeIllegalUserAndPwd(new String(user, StandardCharsets.UTF_8), "pwd"); + } + + @Test(expected = AuthenticationException.class) + public void testNULUsername1() + { + testDecodeIllegalUserAndPwd(new String(new byte[4]), "pwd"); + } + + @Test(expected = AuthenticationException.class) + public void testNULPassword0() + { + byte[] pwd = {'p', 'w', PasswordAuthenticator.NUL, 'd'}; + testDecodeIllegalUserAndPwd("user", new String(pwd, StandardCharsets.UTF_8)); + } + + @Test(expected = AuthenticationException.class) + public void testNULPassword1() + { + testDecodeIllegalUserAndPwd("user", new String(new byte[4])); + } + + private void testDecodeIllegalUserAndPwd(String username, String password) + { + SaslNegotiator negotiator = authenticator.newSaslNegotiator(null); + Authenticator clientAuthenticator = (new PlainTextAuthProvider(username, password)) + .newAuthenticator(null, null); + + negotiator.evaluateResponse(clientAuthenticator.initialResponse()); + negotiator.getAuthenticatedUser(); + } + + @BeforeClass + public static void setUp() + { + SchemaLoader.createKeyspace(AuthKeyspace.NAME, + KeyspaceParams.simple(1), + Iterables.toArray(AuthKeyspace.metadata().tables, CFMetaData.class)); + authenticator.setup(); + } + + @AfterClass + public static void tearDown() + { + schemaChange("DROP KEYSPACE " + AuthKeyspace.NAME); + } } \ No newline at end of file From d50ec52d6e6c9140080c86fcff08b3cd37a6ca3d Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Thu, 4 Jul 2019 12:45:45 +0100 Subject: [PATCH 27/78] LegacyLayout should handle paging states that cross a collection column --- CHANGES.txt | 1 + .../org/apache/cassandra/db/LegacyLayout.java | 98 ++++++++++++++----- .../apache/cassandra/db/LegacyLayoutTest.java | 54 ++++++++++ 3 files changed, 130 insertions(+), 23 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index ef6f49c6f689..878877ada3e6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * LegacyLayout should handle paging states that cross a collection column (CASSANDRA-15201) * Prevent RuntimeException when username or password is empty/null (CASSANDRA-15198) * Multiget thrift query returns null records after digest mismatch (CASSANDRA-14812) * Skipping illegal legacy cells can break reverse iteration of indexed partitions (CASSANDRA-15178) diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java index b03f56eb0820..a46aa91f8fbb 100644 --- a/src/java/org/apache/cassandra/db/LegacyLayout.java +++ b/src/java/org/apache/cassandra/db/LegacyLayout.java @@ -24,6 +24,7 @@ import java.security.MessageDigest; import java.util.*; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.SuperColumnCompatibility; @@ -235,36 +236,45 @@ private static LegacyBound decodeBound(CFMetaData metadata, ByteBuffer bound, bo assert !isStatic || (components.size() >= clusteringSize && all(components.subList(0, clusteringSize), ByteBufferUtil.EMPTY_BYTE_BUFFER::equals)); + ColumnDefinition collectionName = null; if (components.size() > clusteringSize) { // For a deletion, there can be more components than the clustering size only in the case this is the // bound of a collection range tombstone. In such a case, there is exactly one more component, and that - // component is the name of the collection being selected/deleted. - // If the bound is not part of a deletion, it is from slice query filter. In this scnario, the column name - // may be a valid, non-collection column or it may be an empty buffer, representing a row marker. In either - // case, this needn't be included in the returned bound, so we pop the last element from the components - // list but ensure that the collection name remains null. - - assert clusteringSize + 1 == components.size() && !metadata.isCompactTable(); - // pop the final element from the back of the list of clusterings - ByteBuffer columnNameBytes = components.remove(clusteringSize); - if (isDeletion) + // component is the name of the collection being deleted, since we do not support collection range deletions. + // If the bound is not part of a deletion, it is from slice query filter. The column name may be: + // - a valid, non-collection column; in this case we expect a single extra component + // - an empty buffer, representing a row marker; in this case we also expect a single extra empty component + // - a valid collection column and the first part of a cell path; in this case we expect exactly two extra components + // In any of these slice cases, these items are unnecessary for the bound we construct, + // so we can simply remove them, after corroborating we have encountered one of these scenario. + assert !metadata.isCompactTable() : toDebugHex(components); + + // In all cases, the element straight after the clusterings should contain the name of a column. + if (components.size() > clusteringSize + 1) { - collectionName = metadata.getColumnDefinition(columnNameBytes); - if (collectionName == null || !collectionName.isComplex()) - { - collectionName = metadata.getDroppedColumnDefinition(columnNameBytes, isStatic); - // if no record of the column having ever existed is found, something is badly wrong - if (collectionName == null) - throw new RuntimeException("Unknown collection column " + UTF8Type.instance.getString(columnNameBytes) + " during deserialization"); - - // if we do have a record of dropping this column but it wasn't previously complex, use a fake - // column definition for safety (see the comment on the constant declaration for details) - if (!collectionName.isComplex()) - collectionName = INVALID_DROPPED_COMPLEX_SUBSTITUTE_COLUMN; - } + // we accept bounds from paging state that occur inside a complex column - in this case, we expect + // two excess components, the first of which is a column name, the second a key into the collection + if (isDeletion) + throw new IllegalArgumentException("Invalid bound " + toDebugHex(components) + ": deletion can have at most one extra component"); + + if (clusteringSize + 2 != components.size()) + throw new IllegalArgumentException("Invalid bound " + toDebugHex(components) + ": complex slices require exactly two extra components"); + + // decode simply to verify that we have (or may have had) a complex column; we assume the collection key is valid + decodeBoundLookupComplexColumn(metadata, components, clusteringSize, isStatic); + components.remove(clusteringSize + 1); + } + else if (isDeletion) + { + collectionName = decodeBoundLookupComplexColumn(metadata, components, clusteringSize, isStatic); + } + else if (components.get(clusteringSize).hasRemaining()) + { + decodeBoundVerifySimpleColumn(metadata, components, clusteringSize, isStatic); } + components.remove(clusteringSize); } boolean isInclusive; @@ -292,6 +302,48 @@ private static LegacyBound decodeBound(CFMetaData metadata, ByteBuffer bound, bo return new LegacyBound(sb, isStatic, collectionName); } + // finds the simple column definition associated with components.get(clusteringSize) + // if no such columns exists, or ever existed, we throw an exception; if we do not know, we return a dummy column definition + private static ColumnDefinition decodeBoundLookupComplexColumn(CFMetaData metadata, List components, int clusteringSize, boolean isStatic) + { + ByteBuffer columnNameBytes = components.get(clusteringSize); + ColumnDefinition columnName = metadata.getColumnDefinition(columnNameBytes); + if (columnName == null || !columnName.isComplex()) + { + columnName = metadata.getDroppedColumnDefinition(columnNameBytes, isStatic); + // if no record of the column having ever existed is found, something is badly wrong + if (columnName == null) + throw new IllegalArgumentException("Invalid bound " + toDebugHex(components) + ": expected complex column at position " + clusteringSize); + + // if we do have a record of dropping this column but it wasn't previously complex, use a fake + // column definition for safety (see the comment on the constant declaration for details) + if (!columnName.isComplex()) + columnName = INVALID_DROPPED_COMPLEX_SUBSTITUTE_COLUMN; + } + + return columnName; + } + + // finds the simple column definition associated with components.get(clusteringSize) + // if no such columns exists, and definitely never existed, we throw an exception + private static void decodeBoundVerifySimpleColumn(CFMetaData metadata, List components, int clusteringSize, boolean isStatic) + { + ByteBuffer columnNameBytes = components.get(clusteringSize); + ColumnDefinition columnName = metadata.getColumnDefinition(columnNameBytes); + if (columnName == null || !columnName.isSimple()) + { + columnName = metadata.getDroppedColumnDefinition(columnNameBytes, isStatic); + // if no record of the column having ever existed is found, something is badly wrong + if (columnName == null) + throw new IllegalArgumentException("Invalid bound " + toDebugHex(components) + ": expected simple column at position " + clusteringSize); + } + } + + private static String toDebugHex(Collection buffers) + { + return buffers.stream().map(ByteBufferUtil::bytesToHex).collect(Collectors.joining()); + } + public static ByteBuffer encodeBound(CFMetaData metadata, Slice.Bound bound, boolean isStart) { if (bound == Slice.Bound.BOTTOM || bound == Slice.Bound.TOP || metadata.comparator.size() == 0) diff --git a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java index e34f58eaf251..580a0e70d667 100644 --- a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java +++ b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java @@ -38,6 +38,7 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.serializers.Int32Serializer; import org.apache.cassandra.serializers.UTF8Serializer; +import org.apache.cassandra.service.MigrationManager; import org.apache.cassandra.utils.FBUtilities; import org.junit.BeforeClass; import org.junit.Test; @@ -57,6 +58,7 @@ import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Hex; import static org.junit.Assert.*; @@ -291,4 +293,56 @@ public void testCollectionDeletionRoundTripForDroppedColumn() throws Throwable } } + @Test + public void testDecodeLegacyPagedRangeCommandSerializer() throws IOException + { + /* + Run on 2.1 + public static void main(String[] args) throws IOException, ConfigurationException + { + Gossiper.instance.start((int) (System.currentTimeMillis() / 1000)); + Keyspace.setInitialized(); + CFMetaData cfMetaData = CFMetaData.sparseCFMetaData("ks", "cf", UTF8Type.instance) + .addColumnDefinition(new ColumnDefinition("ks", "cf", new ColumnIdentifier("v", true), SetType.getInstance(Int32Type.instance, false), null, null, null, null, ColumnDefinition.Kind.REGULAR)); + KSMetaData ksMetaData = KSMetaData.testMetadata("ks", SimpleStrategy.class, KSMetaData.optsWithRF(3), cfMetaData); + MigrationManager.announceNewKeyspace(ksMetaData); + RowPosition position = RowPosition.ForKey.get(ByteBufferUtil.EMPTY_BYTE_BUFFER, new Murmur3Partitioner()); + SliceQueryFilter filter = new IdentityQueryFilter(); + Composite cellName = CellNames.compositeSparseWithCollection(new ByteBuffer[0], Int32Type.instance.decompose(1), new ColumnIdentifier("v", true), false); + try (DataOutputBuffer buffer = new DataOutputBuffer(1024)) + { + PagedRangeCommand command = new PagedRangeCommand("ks", "cf", 1, AbstractBounds.bounds(position, true, position, true), filter, cellName, filter.finish(), Collections.emptyList(), 1, true); + PagedRangeCommand.serializer.serialize(command, buffer, MessagingService.current_version); + System.out.println(Hex.bytesToHex(buffer.toByteArray())); + } + } + */ + + DatabaseDescriptor.setDaemonInitialized(); + Keyspace.setInitialized(); + CFMetaData table = CFMetaData.Builder.create("ks", "cf") + .addPartitionKey("k", Int32Type.instance) + .addRegularColumn("v", SetType.getInstance(Int32Type.instance, true)) + .build(); + SchemaLoader.createKeyspace("ks", KeyspaceParams.simple(1)); + MigrationManager.announceNewColumnFamily(table); + + byte[] bytes = Hex.hexToBytes("00026b73000263660000000000000001fffffffe01000000088000000000000000010000000880000000000000000000000100000000007fffffffffffffff000b00017600000400000001000000000000000000000101"); + ReadCommand.legacyPagedRangeCommandSerializer.deserialize(new DataInputBuffer(bytes), MessagingService.VERSION_21); + } + + @Test + public void testDecodeCollectionPageBoundary() + { + CFMetaData table = CFMetaData.Builder.create("ks", "cf") + .addPartitionKey("k", Int32Type.instance) + .addRegularColumn("v", SetType.getInstance(Int32Type.instance, true)) + .build(); + + ColumnDefinition v = table.getColumnDefinition(new ColumnIdentifier("v", false)); + ByteBuffer bound = LegacyLayout.encodeCellName(table, Clustering.EMPTY, v.name.bytes, Int32Type.instance.decompose(1)); + + LegacyLayout.decodeSliceBound(table, bound, true); + } + } \ No newline at end of file From 08b2192da0eb6deddcd8f79cd180d069442223ae Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Tue, 9 Jul 2019 16:05:45 +0100 Subject: [PATCH 28/78] Fix AlterTableStatement dropped type validation order patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for CASSANDRA-15203 --- CHANGES.txt | 1 + .../cassandra/cql3/statements/schema/AlterTableStatement.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index e234df11dd77..dfdc017e26e4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Fix AlterTableStatement dropped type validation order (CASSANDRA-15203) * Update Netty dependencies to latest, clean up SocketFactory (CASSANDRA-15195) * Native Transport - Apply noSpamLogger to ConnectionLimitHandler (CASSANDRA-15167) * Reduce heap pressure during compactions (CASSANDRA-14654) diff --git a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java index c348cc4f8ace..6410e67a70e6 100644 --- a/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java @@ -166,7 +166,7 @@ private void addColumn(KeyspaceMetadata keyspace, { // After #8099, not safe to re-add columns of incompatible types - until *maybe* deser logic with dropped // columns is pushed deeper down the line. The latter would still be problematic in cases of schema races. - if (!droppedColumn.type.isValueCompatibleWith(type)) + if (!type.isValueCompatibleWith(droppedColumn.type)) { throw ire("Cannot re-add previously dropped column '%s' of type %s, incompatible with previous type %s", name, From 855fc91596b10a89a504b3491952d06ffe93618d Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Tue, 9 Jul 2019 15:59:40 +0100 Subject: [PATCH 29/78] Toughen up column drop/recreate type validations patch by Aleksey Yeschenko; reviewed by Sam Tunnicliffe for CASSANDRA-15204 --- CHANGES.txt | 1 + .../cql3/statements/AlterTableStatement.java | 28 ++++---- .../validation/entities/CollectionsTest.java | 4 +- .../cql3/validation/operations/AlterTest.java | 67 ++++++++++++++++--- 4 files changed, 75 insertions(+), 25 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 878877ada3e6..c8bd30d5b911 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Toughen up column drop/recreate type validations (CASSANDRA-15204) * LegacyLayout should handle paging states that cross a collection column (CASSANDRA-15201) * Prevent RuntimeException when username or password is empty/null (CASSANDRA-15198) * Multiget thrift query returns null records after digest mismatch (CASSANDRA-14812) diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java index 59c4e08ce9a1..9474fb07f3cb 100644 --- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java @@ -30,7 +30,6 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.marshal.AbstractType; -import org.apache.cassandra.db.marshal.CollectionType; import org.apache.cassandra.db.view.View; import org.apache.cassandra.exceptions.*; import org.apache.cassandra.schema.IndexMetadata; @@ -138,10 +137,6 @@ public Event.SchemaChange announceMigration(QueryState queryState, boolean isLoc } } - // Cannot re-add a dropped counter column. See #7831. - if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes)) - throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName)); - AbstractType type = validator.getType(); if (type.isCollection() && type.isMultiCell()) { @@ -149,21 +144,26 @@ public Event.SchemaChange announceMigration(QueryState queryState, boolean isLoc throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables"); if (cfm.isSuper()) throw new InvalidRequestException("Cannot use non-frozen collections with super column families"); + } - // If there used to be a non-frozen collection column with the same name (that has been dropped), - // we could still have some data using the old type, and so we can't allow adding a collection - // with the same name unless the types are compatible (see #6276). - CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes); - if (dropped != null && dropped.type instanceof CollectionType - && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type)) + CFMetaData.DroppedColumn droppedColumn = meta.getDroppedColumns().get(columnName.bytes); + if (null != droppedColumn) + { + // After #8099, not safe to re-add columns of incompatible types - until *maybe* deser logic with dropped + // columns is pushed deeper down the line. The latter would still be problematic in cases of schema races. + if (!type.isValueCompatibleWith(droppedColumn.type)) { String message = - String.format("Cannot add a collection with the name %s because a collection with the same name" - + " and a different type (%s) has already been used in the past", + String.format("Cannot re-add previously dropped column '%s' of type %s, incompatible with previous type %s", columnName, - dropped.type.asCQL3Type()); + type.asCQL3Type(), + droppedColumn.type.asCQL3Type()); throw new InvalidRequestException(message); } + + // Cannot re-add a dropped counter column. See #7831. + if (meta.isCounter()) + throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName)); } cfm.addColumnDefinition(isStatic diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java index c00d7077a516..918033eca30a 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java @@ -593,13 +593,13 @@ public void testDropAndReaddCollection() throws Throwable } @Test - public void testDropAndReaddDroppedCollection() throws Throwable + public void testDropAndReaddFrozenCollection() throws Throwable { createTable("create table %s (k int primary key, v frozen>, x int)"); execute("insert into %s (k, v) VALUES (0, {'fffffffff'})"); flush(); execute("alter table %s drop v"); - execute("alter table %s add v set"); + assertInvalid("alter table %s add v frozen>"); } @Test diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java index b37462f1aedf..5e0c69c70ce6 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java @@ -250,18 +250,67 @@ public void testAlterStatementWithAdd() throws Throwable row(1, null, null, "111")); } - /** - * Test for 7744, - * migrated from cql_tests.py:TestCQL.downgrade_to_compact_bug_test() - */ - @Test - public void testDowngradeToCompact() throws Throwable + @Test(expected = InvalidRequestException.class) + public void testDropComplexAddSimpleColumn() throws Throwable { createTable("create table %s (k int primary key, v set)"); - execute("insert into %s (k, v) VALUES (0, {'f'})"); - flush(); execute("alter table %s drop v"); - execute("alter table %s add v int"); + execute("alter table %s add v text"); + } + + @Test(expected = InvalidRequestException.class) + public void testDropSimpleAddComplexColumn() throws Throwable + { + createTable("create table %s (k int primary key, v text)"); + execute("alter table %s drop v"); + execute("alter table %s add v set"); + } + + @Test(expected = InvalidRequestException.class) + public void testDropMultiCellAddFrozenColumn() throws Throwable + { + createTable("create table %s (k int primary key, v set)"); + execute("alter table %s drop v"); + execute("alter table %s add v frozen>"); + } + + @Test(expected = InvalidRequestException.class) + public void testDropFrozenAddMultiCellColumn() throws Throwable + { + createTable("create table %s (k int primary key, v frozen>)"); + execute("alter table %s drop v"); + execute("alter table %s add v set"); + } + + @Test + public void testDropTimeUUIDAddUUIDColumn() throws Throwable + { + createTable("create table %s (k int primary key, v timeuuid)"); + execute("alter table %s drop v"); + execute("alter table %s add v uuid"); + } + + @Test(expected = InvalidRequestException.class) + public void testDropUUIDAddTimeUUIDColumn() throws Throwable + { + createTable("create table %s (k int primary key, v uuid)"); + execute("alter table %s drop v"); + execute("alter table %s add v timeuuid"); + } + + @Test + public void testDropAddSameType() throws Throwable + { + createTable("create table %s (k int primary key, v1 timeuuid, v2 set, v3 frozen>)"); + + execute("alter table %s drop v1"); + execute("alter table %s add v1 timeuuid"); + + execute("alter table %s drop v2"); + execute("alter table %s add v2 set"); + + execute("alter table %s drop v3"); + execute("alter table %s add v3 frozen>"); } @Test(expected = SyntaxException.class) From 73a65d6ae045f6a1fd3ef06d23044d945f121639 Mon Sep 17 00:00:00 2001 From: Aleksey Yeshchenko Date: Wed, 10 Jul 2019 15:38:21 +0100 Subject: [PATCH 30/78] Use correct sentinel timestamp value for in-JVM dtests --- src/java/org/apache/cassandra/cql3/QueryOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/cql3/QueryOptions.java b/src/java/org/apache/cassandra/cql3/QueryOptions.java index d32a707140c3..a062567742f9 100644 --- a/src/java/org/apache/cassandra/cql3/QueryOptions.java +++ b/src/java/org/apache/cassandra/cql3/QueryOptions.java @@ -78,7 +78,7 @@ public static QueryOptions create(ConsistencyLevel consistency, List public static QueryOptions create(ConsistencyLevel consistency, List values, boolean skipMetadata, int pageSize, PagingState pagingState, ConsistencyLevel serialConsistency, int protocolVersion) { - return new DefaultQueryOptions(consistency, values, skipMetadata, new SpecificOptions(pageSize, pagingState, serialConsistency, -1L), protocolVersion); + return new DefaultQueryOptions(consistency, values, skipMetadata, new SpecificOptions(pageSize, pagingState, serialConsistency, Long.MIN_VALUE), protocolVersion); } public static QueryOptions addColumnSpecifications(QueryOptions options, List columnSpecs) From 0240a4659d761f06f94f8cd97097f2d0ad2d220c Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Wed, 26 Jun 2019 14:44:15 +0200 Subject: [PATCH 31/78] Introduce optional timeouts for idle client sessions Patch by Alex Petrov, reviewed by Aleksey Yeshchenko for CASSANDRA-11097 --- conf/cassandra.yaml | 10 +++ .../org/apache/cassandra/config/Config.java | 2 + .../cassandra/config/DatabaseDescriptor.java | 10 +++ .../apache/cassandra/transport/Server.java | 18 +++++ .../transport/IdleDisconnectTest.java | 78 +++++++++++++++++++ 5 files changed, 118 insertions(+) create mode 100644 test/unit/org/apache/cassandra/transport/IdleDisconnectTest.java diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 300285710946..28d86fd10ae3 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -683,6 +683,16 @@ native_transport_port: 9042 # The default is true, which means all supported protocols will be honored. native_transport_allow_older_protocols: true +# Controls when idle client connections are closed. Idle connections are ones that had neither reads +# nor writes for a time period. +# +# Clients may implement heartbeats by sending OPTIONS native protocol message after a timeout, which +# will reset idle timeout timer on the server side. To close idle client connections, corresponding +# values for heartbeat intervals have to be set on the client side. +# +# Idle connection timeouts are disabled by default. +# native_transport_idle_timeout_in_ms: 60000 + # The address or interface to bind the native transport server to. # # Set rpc_address OR rpc_interface, not both. diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 9713ea2109ff..6b487fea01b5 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -87,6 +87,8 @@ public class Config /** Triggers automatic allocation of tokens if set, using the replication strategy of the referenced keyspace */ public String allocate_tokens_for_keyspace = null; + public long native_transport_idle_timeout_in_ms = 0L; + public volatile long request_timeout_in_ms = 10000L; public volatile long read_request_timeout_in_ms = 5000L; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 3b7009b113b2..bb92716116c7 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -1473,6 +1473,16 @@ public static int getSSLStoragePort() return Integer.parseInt(System.getProperty(Config.PROPERTY_PREFIX + "ssl_storage_port", Integer.toString(conf.ssl_storage_port))); } + public static long nativeTransportIdleTimeout() + { + return conf.native_transport_idle_timeout_in_ms; + } + + public static void setNativeTransportIdleTimeout(long nativeTransportTimeout) + { + conf.native_transport_idle_timeout_in_ms = nativeTransportTimeout; + } + public static long getRpcTimeout(TimeUnit unit) { return unit.convert(conf.request_timeout_in_ms, MILLISECONDS); diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java index 33cd0fb3ae2c..f16aa881a054 100644 --- a/src/java/org/apache/cassandra/transport/Server.java +++ b/src/java/org/apache/cassandra/transport/Server.java @@ -23,6 +23,7 @@ import java.net.UnknownHostException; import java.util.*; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.slf4j.Logger; @@ -41,6 +42,9 @@ import io.netty.handler.codec.ByteToMessageDecoder; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslHandler; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; +import io.netty.handler.timeout.IdleStateHandler; import io.netty.util.Version; import io.netty.util.concurrent.EventExecutor; import io.netty.util.concurrent.GlobalEventExecutor; @@ -368,6 +372,20 @@ protected void initChannel(Channel channel) throws Exception pipeline.addFirst("connectionLimitHandler", connectionLimitHandler); } + long idleTimeout = DatabaseDescriptor.nativeTransportIdleTimeout(); + if (idleTimeout > 0) + { + pipeline.addLast("idleStateHandler", new IdleStateHandler(false, 0, 0, idleTimeout, TimeUnit.MILLISECONDS) + { + @Override + protected void channelIdle(ChannelHandlerContext ctx, IdleStateEvent evt) + { + logger.info("Closing client connection {} after timeout of {}ms", channel.remoteAddress(), idleTimeout); + ctx.close(); + } + }); + } + //pipeline.addLast("debug", new LoggingHandler()); pipeline.addLast("frameDecoder", new Frame.Decoder(server.connectionFactory)); diff --git a/test/unit/org/apache/cassandra/transport/IdleDisconnectTest.java b/test/unit/org/apache/cassandra/transport/IdleDisconnectTest.java new file mode 100644 index 000000000000..2c8adeabae0f --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/IdleDisconnectTest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.transport; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.ConsistencyLevel; + +public class IdleDisconnectTest extends CQLTester +{ + private static final long TIMEOUT = 2000L; + + @BeforeClass + public static void setUp() + { + requireNetwork(); + DatabaseDescriptor.setNativeTransportIdleTimeout(TIMEOUT); + } + + @Test + public void testIdleDisconnect() throws Throwable + { + DatabaseDescriptor.setNativeTransportIdleTimeout(TIMEOUT); + try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort)) + { + client.connect(false, false); + Assert.assertTrue(client.channel.isOpen()); + long start = System.currentTimeMillis(); + CompletableFuture.runAsync(() -> { + while (!Thread.currentThread().isInterrupted() && client.channel.isOpen()); + }).get(30, TimeUnit.SECONDS); + Assert.assertFalse(client.channel.isOpen()); + Assert.assertTrue(System.currentTimeMillis() - start >= TIMEOUT); + } + } + + @Test + public void testIdleDisconnectProlonged() throws Throwable + { + long sleepTime = 1000; + try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort)) + { + client.connect(false, false); + Assert.assertTrue(client.channel.isOpen()); + long start = System.currentTimeMillis(); + Thread.sleep(sleepTime); + client.execute("SELECT * FROM system.peers", ConsistencyLevel.ONE); + CompletableFuture.runAsync(() -> { + while (!Thread.currentThread().isInterrupted() && client.channel.isOpen()); + }).get(30, TimeUnit.SECONDS); + Assert.assertFalse(client.channel.isOpen()); + Assert.assertTrue(System.currentTimeMillis() - start >= TIMEOUT + sleepTime); + } + } +} From 149caf01e08f58f306ff51379ab189c7a4b1ca6d Mon Sep 17 00:00:00 2001 From: Sam Tunnicliffe Date: Tue, 16 Apr 2019 16:33:59 +0100 Subject: [PATCH 32/78] CassandraNetworkAuthorizer gets login privilege from RolesCache Patch by Sam Tunnicliffe; reviewed by Blake Eggleston for CASSANDRA-15089 --- CHANGES.txt | 2 ++ .../auth/CassandraNetworkAuthorizer.java | 2 +- .../auth/CassandraNetworkAuthorizerTest.java | 17 +++++++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 3248cfe7cc14..cbdd91f17746 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 4.0 + * CassandraNetworkAuthorizer uses cached roles info (CASSANDRA-15089) + * Introduce optional timeouts for idle client sessions (CASSANDRA-11097) * Fix AlterTableStatement dropped type validation order (CASSANDRA-15203) * Update Netty dependencies to latest, clean up SocketFactory (CASSANDRA-15195) * Native Transport - Apply noSpamLogger to ConnectionLimitHandler (CASSANDRA-15167) diff --git a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java index 34a01402683b..6fdcd6959c9d 100644 --- a/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java +++ b/src/java/org/apache/cassandra/auth/CassandraNetworkAuthorizer.java @@ -78,7 +78,7 @@ private Set getAuthorizedDcs(String name) public DCPermissions authorize(RoleResource role) { - if (!DatabaseDescriptor.getRoleManager().canLogin(role)) + if (!Roles.canLogin(role)) { return DCPermissions.none(); } diff --git a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java index c24a769aed45..2e57173bb5cd 100644 --- a/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java +++ b/test/unit/org/apache/cassandra/auth/CassandraNetworkAuthorizerTest.java @@ -51,6 +51,7 @@ import static org.apache.cassandra.auth.AuthKeyspace.NETWORK_PERMISSIONS; import static org.apache.cassandra.auth.RoleTestUtils.LocalCassandraRoleManager; import static org.apache.cassandra.schema.SchemaConstants.AUTH_KEYSPACE_NAME; +import static org.apache.cassandra.auth.RoleTestUtils.getReadCount; public class CassandraNetworkAuthorizerTest { @@ -105,6 +106,8 @@ public static void defineSchema() throws ConfigurationException new LocalCassandraAuthorizer(), new LocalCassandraNetworkAuthorizer()); setupSuperUser(); + // not strictly necessary to init the cache here, but better to be explicit + Roles.initRolesCache(DatabaseDescriptor.getRoleManager(), () -> true); } @Before @@ -227,6 +230,8 @@ public void superUser() Assert.assertEquals(DCPermissions.subset("dc1"), dcPerms(username)); assertDcPermRow(username, "dc1"); + // clear the roles cache to lose the (non-)superuser status for the user + Roles.clearCache(); auth("ALTER ROLE %s WITH superuser = true", username); Assert.assertEquals(DCPermissions.all(), dcPerms(username)); } @@ -238,4 +243,16 @@ public void cantLogin() auth("CREATE ROLE %s", username); Assert.assertEquals(DCPermissions.none(), dcPerms(username)); } + + @Test + public void getLoginPrivilegeFromRolesCache() throws Exception + { + String username = createName(); + auth("CREATE ROLE %s", username); + long readCount = getReadCount(); + dcPerms(username); + Assert.assertEquals(++readCount, getReadCount()); + dcPerms(username); + Assert.assertEquals(readCount, getReadCount()); + } } From 5a03898c680ed6ada63901e8a4b278ccc8070717 Mon Sep 17 00:00:00 2001 From: sumanthpasupuleti Date: Mon, 25 Mar 2019 08:06:13 -0700 Subject: [PATCH 33/78] Prevent client requests from blocking on executor task queue patch by Sumanth Pasupuleti, reviewed by Benedict for CASSANDRA-15013 --- CHANGES.txt | 1 + doc/native_protocol_v4.spec | 4 + .../org/apache/cassandra/config/Config.java | 3 + .../cassandra/config/DatabaseDescriptor.java | 31 +++ .../cassandra/metrics/ClientMetrics.java | 56 +++- .../apache/cassandra/net/ResourceLimits.java | 245 +++++++++++++++++ .../service/NativeTransportService.java | 26 +- .../cassandra/transport/Connection.java | 11 + .../org/apache/cassandra/transport/Frame.java | 12 +- .../apache/cassandra/transport/Message.java | 146 ++++++++++- .../transport/RequestThreadPoolExecutor.java | 96 ------- .../apache/cassandra/transport/Server.java | 64 +++-- .../cassandra/transport/SimpleClient.java | 10 + .../transport/messages/StartupMessage.java | 3 + .../org/apache/cassandra/cql3/CQLTester.java | 2 + .../service/NativeTransportServiceTest.java | 3 +- .../InflightRequestPayloadTrackerTest.java | 248 ++++++++++++++++++ 17 files changed, 811 insertions(+), 150 deletions(-) create mode 100644 src/java/org/apache/cassandra/net/ResourceLimits.java delete mode 100644 src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java create mode 100644 test/unit/org/apache/cassandra/transport/InflightRequestPayloadTrackerTest.java diff --git a/CHANGES.txt b/CHANGES.txt index c8bd30d5b911..68d309c69b9e 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Prevent client requests from blocking on executor task queue (CASSANDRA-15013) * Toughen up column drop/recreate type validations (CASSANDRA-15204) * LegacyLayout should handle paging states that cross a collection column (CASSANDRA-15201) * Prevent RuntimeException when username or password is empty/null (CASSANDRA-15198) diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec index 02802a7a889e..8beb77b6b180 100644 --- a/doc/native_protocol_v4.spec +++ b/doc/native_protocol_v4.spec @@ -275,6 +275,9 @@ Table of Contents mode. This mode will make all Thrift and Compact Tables to be exposed as if they were CQL Tables. This is optional; if not specified, the option will not be used. + - "THROW_ON_OVERLOAD": In case of server overloaded with too many requests, by default the server puts + back pressure on the client connection. Instead, the server can send an OverloadedException error message back to + the client if this option is set to true. 4.1.2. AUTH_RESPONSE @@ -1175,3 +1178,4 @@ Table of Contents * The returned in the v4 protocol is not compatible with the v3 protocol. In other words, a returned by a node using protocol v4 should not be used to query a node using protocol v3 (and vice-versa). + * Added THROW_ON_OVERLOAD startup option (Section 4.1.1). diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index de158bdeec27..830d3e11a4f1 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -154,6 +154,9 @@ public class Config public volatile Long native_transport_max_concurrent_connections = -1L; public volatile Long native_transport_max_concurrent_connections_per_ip = -1L; public boolean native_transport_flush_in_batches_legacy = true; + public volatile long native_transport_max_concurrent_requests_in_bytes_per_ip = -1L; + public volatile long native_transport_max_concurrent_requests_in_bytes = -1L; + @Deprecated public Integer thrift_max_message_length_in_mb = 16; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index db55c20998e2..8417c39bbca2 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -470,6 +470,17 @@ else if (conf.native_transport_max_frame_size_in_mb >= 2048) { throw new ConfigurationException("Missing endpoint_snitch directive", false); } + + if (conf.native_transport_max_concurrent_requests_in_bytes <= 0) + { + conf.native_transport_max_concurrent_requests_in_bytes = Runtime.getRuntime().maxMemory() / 10; + } + + if (conf.native_transport_max_concurrent_requests_in_bytes_per_ip <= 0) + { + conf.native_transport_max_concurrent_requests_in_bytes_per_ip = Runtime.getRuntime().maxMemory() / 40; + } + snitch = createEndpointSnitch(conf.endpoint_snitch); EndpointSnitchInfo.create(); @@ -1524,6 +1535,26 @@ public static void setCommitLogSyncBatchWindow(double windowMillis) conf.commitlog_sync_batch_window_in_ms = windowMillis; } + public static long getNativeTransportMaxConcurrentRequestsInBytesPerIp() + { + return conf.native_transport_max_concurrent_requests_in_bytes_per_ip; + } + + public static void setNativeTransportMaxConcurrentRequestsInBytesPerIp(long maxConcurrentRequestsInBytes) + { + conf.native_transport_max_concurrent_requests_in_bytes_per_ip = maxConcurrentRequestsInBytes; + } + + public static long getNativeTransportMaxConcurrentRequestsInBytes() + { + return conf.native_transport_max_concurrent_requests_in_bytes; + } + + public static void setNativeTransportMaxConcurrentRequestsInBytes(long maxConcurrentRequestsInBytes) + { + conf.native_transport_max_concurrent_requests_in_bytes = maxConcurrentRequestsInBytes; + } + public static int getCommitLogSyncPeriod() { return conf.commitlog_sync_period_in_ms; diff --git a/src/java/org/apache/cassandra/metrics/ClientMetrics.java b/src/java/org/apache/cassandra/metrics/ClientMetrics.java index 4a384ebc2f21..08f053159ce2 100644 --- a/src/java/org/apache/cassandra/metrics/ClientMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ClientMetrics.java @@ -18,9 +18,14 @@ */ package org.apache.cassandra.metrics; +import java.util.Collection; +import java.util.Collections; import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; import com.codahale.metrics.Gauge; +import com.codahale.metrics.Meter; +import org.apache.cassandra.transport.Server; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; @@ -28,13 +33,40 @@ public class ClientMetrics { private static final MetricNameFactory factory = new DefaultNameFactory("Client"); - public static final ClientMetrics instance = new ClientMetrics(); - + + private volatile boolean initialized = false; + + private Collection servers = Collections.emptyList(); + + private AtomicInteger pausedConnections; + private Gauge pausedConnectionsGauge; + private Meter requestDiscarded; + private ClientMetrics() { } + public void pauseConnection() { pausedConnections.incrementAndGet(); } + public void unpauseConnection() { pausedConnections.decrementAndGet(); } + public void markRequestDiscarded() { requestDiscarded.mark(); } + + public synchronized void init(Collection servers) + { + if (initialized) + return; + + this.servers = servers; + + registerGauge("connectedNativeClients", this::countConnectedClients); + + pausedConnections = new AtomicInteger(); + pausedConnectionsGauge = registerGauge("PausedConnections", pausedConnections::get); + requestDiscarded = registerMeter("RequestDiscarded"); + + initialized = true; + } + public void addCounter(String name, final Callable provider) { Metrics.register(factory.createMetricName(name), new Gauge() @@ -51,4 +83,24 @@ public Integer getValue() } }); } + + private int countConnectedClients() + { + int count = 0; + + for (Server server : servers) + count += server.getConnectedClients(); + + return count; + } + + private Gauge registerGauge(String name, Gauge gauge) + { + return Metrics.register(factory.createMetricName(name), gauge); + } + + private Meter registerMeter(String name) + { + return Metrics.meter(factory.createMetricName(name)); + } } diff --git a/src/java/org/apache/cassandra/net/ResourceLimits.java b/src/java/org/apache/cassandra/net/ResourceLimits.java new file mode 100644 index 000000000000..f8d24d778bdc --- /dev/null +++ b/src/java/org/apache/cassandra/net/ResourceLimits.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.net; + +import java.util.concurrent.atomic.AtomicLongFieldUpdater; + +public abstract class ResourceLimits +{ + /** + * Represents permits to utilise a resource and ways to allocate and release them. + * + * Two implementations are currently provided: + * 1. {@link Concurrent}, for shared limits, which is thread-safe; + * 2. {@link Basic}, for limits that are not shared between threads, is not thread-safe. + */ + public interface Limit + { + /** + * @return total amount of permits represented by this {@link Limit} - the capacity + */ + long limit(); + + /** + * @return remaining, unallocated permit amount + */ + long remaining(); + + /** + * @return amount of permits currently in use + */ + long using(); + + /** + * Attempts to allocate an amount of permits from this limit. If allocated, MUST eventually + * be released back with {@link #release(long)}. + * + * @return {@code true} if the allocation was successful, {@code false} otherwise + */ + boolean tryAllocate(long amount); + + /** + * Allocates an amount independent of permits available from this limit. MUST eventually + * be released back with {@link #release(long)}. + * + */ + void allocate(long amount); + + /** + * @param amount return the amount of permits back to this limit + * @return {@code ABOVE_LIMIT} if there aren't enough permits available even after the release, or + * {@code BELOW_LIMIT} if there are enough permits available after the releaese. + */ + Outcome release(long amount); + } + + /** + * A thread-safe permit container. + */ + public static class Concurrent implements Limit + { + private final long limit; + + private volatile long using; + private static final AtomicLongFieldUpdater usingUpdater = + AtomicLongFieldUpdater.newUpdater(Concurrent.class, "using"); + + public Concurrent(long limit) + { + this.limit = limit; + } + + public long limit() + { + return limit; + } + + public long remaining() + { + return limit - using; + } + + public long using() + { + return using; + } + + public boolean tryAllocate(long amount) + { + long current, next; + do + { + current = using; + next = current + amount; + + if (next > limit) + return false; + } + while (!usingUpdater.compareAndSet(this, current, next)); + + return true; + } + + public void allocate(long amount) + { + long current, next; + do + { + current = using; + next = current + amount; + } while (!usingUpdater.compareAndSet(this, current, next)); + } + + public Outcome release(long amount) + { + assert amount >= 0; + long using = usingUpdater.addAndGet(this, -amount); + assert using >= 0; + return using >= limit ? Outcome.ABOVE_LIMIT : Outcome.BELOW_LIMIT; + } + } + + /** + * A cheaper, thread-unsafe permit container to be used for unshared limits. + */ + static class Basic implements Limit + { + private final long limit; + private long using; + + Basic(long limit) + { + this.limit = limit; + } + + public long limit() + { + return limit; + } + + public long remaining() + { + return limit - using; + } + + public long using() + { + return using; + } + + public boolean tryAllocate(long amount) + { + if (using + amount > limit) + return false; + + using += amount; + return true; + } + + public void allocate(long amount) + { + using += amount; + } + + public Outcome release(long amount) + { + assert amount >= 0 && amount <= using; + using -= amount; + return using >= limit ? Outcome.ABOVE_LIMIT : Outcome.BELOW_LIMIT; + } + } + + /** + * A convenience class that groups a per-endpoint limit with the global one + * to allow allocating/releasing permits from/to both limits as one logical operation. + */ + public static class EndpointAndGlobal + { + final Limit endpoint; + final Limit global; + + public EndpointAndGlobal(Limit endpoint, Limit global) + { + this.endpoint = endpoint; + this.global = global; + } + + public Limit endpoint() + { + return endpoint; + } + + public Limit global() + { + return global; + } + + /** + * @return {@code INSUFFICIENT_GLOBAL} if there weren't enough permits in the global limit, or + * {@code INSUFFICIENT_ENDPOINT} if there weren't enough permits in the per-endpoint limit, or + * {@code SUCCESS} if there were enough permits to take from both. + */ + public Outcome tryAllocate(long amount) + { + if (!global.tryAllocate(amount)) + return Outcome.INSUFFICIENT_GLOBAL; + + if (endpoint.tryAllocate(amount)) + return Outcome.SUCCESS; + + global.release(amount); + return Outcome.INSUFFICIENT_ENDPOINT; + } + + public void allocate(long amount) + { + global.allocate(amount); + endpoint.allocate(amount); + } + + public Outcome release(long amount) + { + Outcome endpointReleaseOutcome = endpoint.release(amount); + Outcome globalReleaseOutcome = global.release(amount); + return (endpointReleaseOutcome == Outcome.ABOVE_LIMIT || globalReleaseOutcome == Outcome.ABOVE_LIMIT) + ? Outcome.ABOVE_LIMIT : Outcome.BELOW_LIMIT; + } + } + + public enum Outcome { SUCCESS, INSUFFICIENT_ENDPOINT, INSUFFICIENT_GLOBAL, BELOW_LIMIT, ABOVE_LIMIT } +} diff --git a/src/java/org/apache/cassandra/service/NativeTransportService.java b/src/java/org/apache/cassandra/service/NativeTransportService.java index 48839f1cbb0c..22808184ce94 100644 --- a/src/java/org/apache/cassandra/service/NativeTransportService.java +++ b/src/java/org/apache/cassandra/service/NativeTransportService.java @@ -31,11 +31,9 @@ import io.netty.channel.epoll.Epoll; import io.netty.channel.epoll.EpollEventLoopGroup; import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.util.concurrent.EventExecutor; -import io.netty.util.concurrent.Future; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.metrics.ClientMetrics; -import org.apache.cassandra.transport.RequestThreadPoolExecutor; +import org.apache.cassandra.transport.Message; import org.apache.cassandra.transport.Server; /** @@ -50,7 +48,6 @@ public class NativeTransportService private boolean initialized = false; private EventLoopGroup workerGroup; - private EventExecutor eventExecutorGroup; /** * Creates netty thread pools and event loops. @@ -61,9 +58,6 @@ synchronized void initialize() if (initialized) return; - // prepare netty resources - eventExecutorGroup = new RequestThreadPoolExecutor(); - if (useEpoll()) { workerGroup = new EpollEventLoopGroup(); @@ -80,7 +74,6 @@ synchronized void initialize() InetAddress nativeAddr = DatabaseDescriptor.getRpcAddress(); org.apache.cassandra.transport.Server.Builder builder = new org.apache.cassandra.transport.Server.Builder() - .withEventExecutor(eventExecutorGroup) .withEventLoopGroup(workerGroup) .withHost(nativeAddr); @@ -108,13 +101,7 @@ synchronized void initialize() } // register metrics - ClientMetrics.instance.addCounter("connectedNativeClients", () -> - { - int ret = 0; - for (Server server : servers) - ret += server.getConnectedClients(); - return ret; - }); + ClientMetrics.instance.init(servers); initialized = true; } @@ -147,8 +134,7 @@ public void destroy() // shutdown executors used by netty for native transport server workerGroup.shutdownGracefully(3, 5, TimeUnit.SECONDS).awaitUninterruptibly(); - // shutdownGracefully not implemented yet in RequestThreadPoolExecutor - eventExecutorGroup.shutdown(); + Message.Dispatcher.shutdown(); } /** @@ -176,12 +162,6 @@ EventLoopGroup getWorkerGroup() return workerGroup; } - @VisibleForTesting - EventExecutor getEventExecutor() - { - return eventExecutorGroup; - } - @VisibleForTesting Collection getServers() { diff --git a/src/java/org/apache/cassandra/transport/Connection.java b/src/java/org/apache/cassandra/transport/Connection.java index af26557325aa..2966d9be4da1 100644 --- a/src/java/org/apache/cassandra/transport/Connection.java +++ b/src/java/org/apache/cassandra/transport/Connection.java @@ -29,6 +29,7 @@ public class Connection private final Tracker tracker; private volatile FrameCompressor frameCompressor; + private boolean throwOnOverload; public Connection(Channel channel, int version, Tracker tracker) { @@ -49,6 +50,16 @@ public FrameCompressor getCompressor() return frameCompressor; } + public void setThrowOnOverload(boolean throwOnOverload) + { + this.throwOnOverload = throwOnOverload; + } + + public boolean isThrowOnOverload() + { + return throwOnOverload; + } + public Tracker getTracker() { return tracker; diff --git a/src/java/org/apache/cassandra/transport/Frame.java b/src/java/org/apache/cassandra/transport/Frame.java index 3940b47470ca..c28be9f5d212 100644 --- a/src/java/org/apache/cassandra/transport/Frame.java +++ b/src/java/org/apache/cassandra/transport/Frame.java @@ -68,7 +68,7 @@ public boolean release() public static Frame create(Message.Type type, int streamId, int version, EnumSet flags, ByteBuf body) { - Header header = new Header(version, flags, streamId, type); + Header header = new Header(version, flags, streamId, type, body.readableBytes()); return new Frame(header, body); } @@ -83,18 +83,20 @@ public static class Header public final EnumSet flags; public final int streamId; public final Message.Type type; + public final long bodySizeInBytes; - private Header(int version, int flags, int streamId, Message.Type type) + private Header(int version, int flags, int streamId, Message.Type type, long bodySizeInBytes) { - this(version, Flag.deserialize(flags), streamId, type); + this(version, Flag.deserialize(flags), streamId, type, bodySizeInBytes); } - private Header(int version, EnumSet flags, int streamId, Message.Type type) + private Header(int version, EnumSet flags, int streamId, Message.Type type, long bodySizeInBytes) { this.version = version; this.flags = flags; this.streamId = streamId; this.type = type; + this.bodySizeInBytes = bodySizeInBytes; } public static enum Flag @@ -240,7 +242,7 @@ else if (connection.getVersion() != version) streamId); } - results.add(new Frame(new Header(version, flags, streamId, type), body)); + results.add(new Frame(new Header(version, flags, streamId, type, bodyLength), body)); } private void fail() diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java index 0851b194438f..08a8600df0e8 100644 --- a/src/java/org/apache/cassandra/transport/Message.java +++ b/src/java/org/apache/cassandra/transport/Message.java @@ -42,11 +42,18 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.concurrent.LocalAwareExecutorService; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.exceptions.OverloadedException; +import org.apache.cassandra.metrics.ClientMetrics; +import org.apache.cassandra.net.ResourceLimits; import org.apache.cassandra.service.ClientWarn; import org.apache.cassandra.transport.messages.*; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.utils.JVMStabilityInspector; +import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED; + /** * A message from the CQL binary protocol. */ @@ -404,19 +411,42 @@ public void encode(ChannelHandlerContext ctx, Message message, List results) } } - @ChannelHandler.Sharable public static class Dispatcher extends SimpleChannelInboundHandler { + private static final LocalAwareExecutorService requestExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), + Integer.MAX_VALUE, + "transport", + "Native-Transport-Requests"); + + /** + * Current count of *request* bytes that are live on the channel. + * + * Note: should only be accessed while on the netty event loop. + */ + private long channelPayloadBytesInFlight; + + private final Server.EndpointPayloadTracker endpointPayloadTracker; + + private boolean paused; + private static class FlushItem { final ChannelHandlerContext ctx; final Object response; final Frame sourceFrame; - private FlushItem(ChannelHandlerContext ctx, Object response, Frame sourceFrame) + final Dispatcher dispatcher; + + private FlushItem(ChannelHandlerContext ctx, Object response, Frame sourceFrame, Dispatcher dispatcher) { this.ctx = ctx; this.sourceFrame = sourceFrame; this.response = response; + this.dispatcher = dispatcher; + } + + public void release() + { + dispatcher.releaseItem(this); } } @@ -472,7 +502,7 @@ public void run() for (ChannelHandlerContext channel : channels) channel.flush(); for (FlushItem item : flushed) - item.sourceFrame.release(); + item.release(); channels.clear(); flushed.clear(); @@ -524,7 +554,7 @@ public void run() for (ChannelHandlerContext channel : channels) channel.flush(); for (FlushItem item : flushed) - item.sourceFrame.release(); + item.release(); channels.clear(); flushed.clear(); @@ -536,16 +566,98 @@ public void run() private final boolean useLegacyFlusher; - public Dispatcher(boolean useLegacyFlusher) + public Dispatcher(boolean useLegacyFlusher, Server.EndpointPayloadTracker endpointPayloadTracker) { super(false); this.useLegacyFlusher = useLegacyFlusher; + this.endpointPayloadTracker = endpointPayloadTracker; } @Override public void channelRead0(ChannelHandlerContext ctx, Request request) { + // if we decide to handle this message, process it outside of the netty event loop + if (shouldHandleRequest(ctx, request)) + requestExecutor.submit(() -> processRequest(ctx, request)); + } + + /** This check for inflight payload to potentially discard the request should have been ideally in one of the + * first handlers in the pipeline (Frame::decode()). However, incase of any exception thrown between that + * handler (where inflight payload is incremented) and this handler (Dispatcher::channelRead0) (where inflight + * payload in decremented), inflight payload becomes erroneous. ExceptionHandler is not sufficient for this + * purpose since it does not have the frame associated with the exception. + * + * Note: this method should execute on the netty event loop. + */ + private boolean shouldHandleRequest(ChannelHandlerContext ctx, Request request) + { + long frameSize = request.getSourceFrame().header.bodySizeInBytes; + + ResourceLimits.EndpointAndGlobal endpointAndGlobalPayloadsInFlight = endpointPayloadTracker.endpointAndGlobalPayloadsInFlight; + + // check for overloaded state by trying to allocate framesize to inflight payload trackers + if (endpointAndGlobalPayloadsInFlight.tryAllocate(frameSize) != ResourceLimits.Outcome.SUCCESS) + { + if (request.connection.isThrowOnOverload()) + { + // discard the request and throw an exception + ClientMetrics.instance.markRequestDiscarded(); + logger.trace("Discarded request of size: {}. InflightChannelRequestPayload: {}, InflightEndpointRequestPayload: {}, InflightOverallRequestPayload: {}, Request: {}", + frameSize, + channelPayloadBytesInFlight, + endpointAndGlobalPayloadsInFlight.endpoint().using(), + endpointAndGlobalPayloadsInFlight.global().using(), + request); + throw ErrorMessage.wrap(new OverloadedException("Server is in overloaded state. Cannot accept more requests at this point"), + request.getSourceFrame().header.streamId); + } + else + { + // set backpressure on the channel, and handle the request + endpointAndGlobalPayloadsInFlight.allocate(frameSize); + ctx.channel().config().setAutoRead(false); + ClientMetrics.instance.pauseConnection(); + paused = true; + } + } + + channelPayloadBytesInFlight += frameSize; + return true; + } + + /** + * Note: this method will be used in the {@link Flusher#run()}, which executes on the netty event loop + * ({@link Dispatcher#flusherLookup}). Thus, we assume the semantics and visibility of variables + * of being on the event loop. + */ + private void releaseItem(FlushItem item) + { + long itemSize = item.sourceFrame.header.bodySizeInBytes; + item.sourceFrame.release(); + + // since the request has been processed, decrement inflight payload at channel, endpoint and global levels + channelPayloadBytesInFlight -= itemSize; + ResourceLimits.Outcome endpointGlobalReleaseOutcome = endpointPayloadTracker.endpointAndGlobalPayloadsInFlight.release(itemSize); + + // now check to see if we need to reenable the channel's autoRead. + // If the current payload side is zero, we must reenable autoread as + // 1) we allow no other thread/channel to do it, and + // 2) there's no other events following this one (becuase we're at zero bytes in flight), + // so no successive to trigger the other clause in this if-block + ChannelConfig config = item.ctx.channel().config(); + if (paused && (channelPayloadBytesInFlight == 0 || endpointGlobalReleaseOutcome == ResourceLimits.Outcome.BELOW_LIMIT)) + { + paused = false; + ClientMetrics.instance.unpauseConnection(); + config.setAutoRead(true); + } + } + /** + * Note: this method is not expected to execute on the netty event loop. + */ + void processRequest(ChannelHandlerContext ctx, Request request) + { final Response response; final ServerConnection connection; @@ -569,7 +681,7 @@ public void channelRead0(ChannelHandlerContext ctx, Request request) { JVMStabilityInspector.inspectThrowable(t); UnexpectedChannelExceptionHandler handler = new UnexpectedChannelExceptionHandler(ctx.channel(), true); - flush(new FlushItem(ctx, ErrorMessage.fromException(t, handler).setStreamId(request.getStreamId()), request.getSourceFrame())); + flush(new FlushItem(ctx, ErrorMessage.fromException(t, handler).setStreamId(request.getStreamId()), request.getSourceFrame(), this)); return; } finally @@ -578,7 +690,19 @@ public void channelRead0(ChannelHandlerContext ctx, Request request) } logger.trace("Responding: {}, v={}", response, connection.getVersion()); - flush(new FlushItem(ctx, response, request.getSourceFrame())); + flush(new FlushItem(ctx, response, request.getSourceFrame(), this)); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) + { + endpointPayloadTracker.release(); + if (paused) + { + paused = false; + ClientMetrics.instance.unpauseConnection(); + } + ctx.fireChannelInactive(); } private void flush(FlushItem item) @@ -596,6 +720,14 @@ private void flush(FlushItem item) flusher.queued.add(item); flusher.start(); } + + public static void shutdown() + { + if (requestExecutor != null) + { + requestExecutor.shutdown(); + } + } } @ChannelHandler.Sharable diff --git a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java deleted file mode 100644 index 75dd05ddf878..000000000000 --- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.transport; - -import java.util.List; -import java.util.concurrent.TimeUnit; - -import io.netty.util.concurrent.AbstractEventExecutor; -import io.netty.util.concurrent.EventExecutorGroup; -import io.netty.util.concurrent.Future; -import org.apache.cassandra.concurrent.LocalAwareExecutorService; -import org.apache.cassandra.config.DatabaseDescriptor; - -import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED; - -public class RequestThreadPoolExecutor extends AbstractEventExecutor -{ - private final static int MAX_QUEUED_REQUESTS = Integer.getInteger("cassandra.max_queued_native_transport_requests", 128); - private final static String THREAD_FACTORY_ID = "Native-Transport-Requests"; - private final LocalAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), - MAX_QUEUED_REQUESTS, - "transport", - THREAD_FACTORY_ID); - - public boolean isShuttingDown() - { - return wrapped.isShutdown(); - } - - public Future shutdownGracefully(long l, long l2, TimeUnit timeUnit) - { - throw new IllegalStateException(); - } - - public Future terminationFuture() - { - throw new IllegalStateException(); - } - - @Override - public void shutdown() - { - wrapped.shutdown(); - } - - @Override - public List shutdownNow() - { - return wrapped.shutdownNow(); - } - - public boolean isShutdown() - { - return wrapped.isShutdown(); - } - - public boolean isTerminated() - { - return wrapped.isTerminated(); - } - - public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException - { - return wrapped.awaitTermination(timeout, unit); - } - - public EventExecutorGroup parent() - { - return null; - } - - public boolean inEventLoop(Thread thread) - { - return false; - } - - public void execute(Runnable command) - { - wrapped.execute(command); - } -} diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java index 8c781dbd2980..83a676c18be4 100644 --- a/src/java/org/apache/cassandra/transport/Server.java +++ b/src/java/org/apache/cassandra/transport/Server.java @@ -26,6 +26,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,6 +51,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.net.ResourceLimits; import org.apache.cassandra.security.SSLFactory; import org.apache.cassandra.service.*; import org.apache.cassandra.transport.messages.EventMessage; @@ -84,7 +87,6 @@ public Connection newConnection(Channel channel, int version) private final AtomicBoolean isRunning = new AtomicBoolean(false); private EventLoopGroup workerGroup; - private EventExecutor eventExecutorGroup; private Server (Builder builder) { @@ -101,8 +103,6 @@ private Server (Builder builder) else workerGroup = new NioEventLoopGroup(); } - if (builder.eventExecutorGroup != null) - eventExecutorGroup = builder.eventExecutorGroup; EventNotifier notifier = new EventNotifier(this); StorageService.instance.register(notifier); MigrationManager.instance.register(notifier); @@ -201,12 +201,6 @@ public Builder withEventLoopGroup(EventLoopGroup eventLoopGroup) return this; } - public Builder withEventExecutor(EventExecutor eventExecutor) - { - this.eventExecutorGroup = eventExecutor; - return this; - } - public Builder withHost(InetAddress host) { this.hostAddr = host; @@ -286,6 +280,49 @@ public int getConnectedClients() } } + // global inflight payload across all channels across all endpoints + private static final ResourceLimits.Concurrent globalRequestPayloadInFlight = new ResourceLimits.Concurrent(DatabaseDescriptor.getNativeTransportMaxConcurrentRequestsInBytes()); + + public static class EndpointPayloadTracker + { + // inflight payload per endpoint across corresponding channels + private static final ConcurrentMap requestPayloadInFlightPerEndpoint = new ConcurrentHashMap<>(); + + private final AtomicInteger refCount = new AtomicInteger(0); + private final InetAddress endpoint; + + final ResourceLimits.EndpointAndGlobal endpointAndGlobalPayloadsInFlight = new ResourceLimits.EndpointAndGlobal(new ResourceLimits.Concurrent(DatabaseDescriptor.getNativeTransportMaxConcurrentRequestsInBytesPerIp()), + globalRequestPayloadInFlight); + + private EndpointPayloadTracker(InetAddress endpoint) + { + this.endpoint = endpoint; + } + + public static EndpointPayloadTracker get(InetAddress endpoint) + { + while (true) + { + EndpointPayloadTracker result = requestPayloadInFlightPerEndpoint.computeIfAbsent(endpoint, EndpointPayloadTracker::new); + if (result.acquire()) + return result; + + requestPayloadInFlightPerEndpoint.remove(endpoint, result); + } + } + + private boolean acquire() + { + return 0 < refCount.updateAndGet(i -> i < 0 ? i : i + 1); + } + + public void release() + { + if (-1 == refCount.updateAndGet(i -> i == 1 ? -1 : i - 1)) + requestPayloadInFlightPerEndpoint.remove(endpoint, this); + } + } + private static class Initializer extends ChannelInitializer { // Stateless handlers @@ -295,7 +332,6 @@ private static class Initializer extends ChannelInitializer private static final Frame.Compressor frameCompressor = new Frame.Compressor(); private static final Frame.Encoder frameEncoder = new Frame.Encoder(); private static final Message.ExceptionHandler exceptionHandler = new Message.ExceptionHandler(); - private static final Message.Dispatcher dispatcher = new Message.Dispatcher(DatabaseDescriptor.useNativeTransportLegacyFlusher()); private static final ConnectionLimitHandler connectionLimitHandler = new ConnectionLimitHandler(); private final Server server; @@ -328,6 +364,9 @@ protected void initChannel(Channel channel) throws Exception pipeline.addLast("messageDecoder", messageDecoder); pipeline.addLast("messageEncoder", messageEncoder); + pipeline.addLast("executor", new Message.Dispatcher(DatabaseDescriptor.useNativeTransportLegacyFlusher(), + EndpointPayloadTracker.get(((InetSocketAddress) channel.remoteAddress()).getAddress()))); + // The exceptionHandler will take care of handling exceptionCaught(...) events while still running // on the same EventLoop as all previous added handlers in the pipeline. This is important as the used // eventExecutorGroup may not enforce strict ordering for channel events. @@ -335,11 +374,6 @@ protected void initChannel(Channel channel) throws Exception // correctly handled before the handler itself is removed. // See https://issues.apache.org/jira/browse/CASSANDRA-13649 pipeline.addLast("exceptionHandler", exceptionHandler); - - if (server.eventExecutorGroup != null) - pipeline.addLast(server.eventExecutorGroup, "executor", dispatcher); - else - pipeline.addLast("executor", dispatcher); } } diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java index 4759c2ae3409..7916debee787 100644 --- a/src/java/org/apache/cassandra/transport/SimpleClient.java +++ b/src/java/org/apache/cassandra/transport/SimpleClient.java @@ -114,11 +114,21 @@ public SimpleClient(String host, int port) } public void connect(boolean useCompression) throws IOException + { + connect(useCompression, false); + } + + public void connect(boolean useCompression, boolean throwOnOverload) throws IOException { establishConnection(); Map options = new HashMap<>(); options.put(StartupMessage.CQL_VERSION, "3.0.0"); + + if (throwOnOverload) + options.put(StartupMessage.THROW_ON_OVERLOAD, "1"); + connection.setThrowOnOverload(throwOnOverload); + if (useCompression) { options.put(StartupMessage.COMPRESSION, "snappy"); diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java index 774be6a38890..92278fa9e69c 100644 --- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java +++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java @@ -36,6 +36,7 @@ public class StartupMessage extends Message.Request public static final String CQL_VERSION = "CQL_VERSION"; public static final String COMPRESSION = "COMPRESSION"; public static final String NO_COMPACT = "NO_COMPACT"; + public static final String THROW_ON_OVERLOAD = "THROW_ON_OVERLOAD"; public static final Message.Codec codec = new Message.Codec() { @@ -101,6 +102,8 @@ else if (compression.equals("lz4")) if (options.containsKey(NO_COMPACT) && Boolean.parseBoolean(options.get(NO_COMPACT))) state.getClientState().setNoCompactMode(); + connection.setThrowOnOverload("1".equals(options.get(THROW_ON_OVERLOAD))); + if (DatabaseDescriptor.getAuthenticator().requireAuthentication()) return new AuthenticateMessage(DatabaseDescriptor.getAuthenticator().getClass().getName()); else diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 3c0cefcb58f5..999404e1b4c3 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -44,6 +44,7 @@ import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.Schema; import org.apache.cassandra.cql3.functions.FunctionName; @@ -334,6 +335,7 @@ protected static void requireNetwork() throws ConfigurationException SchemaLoader.startGossiper(); server = new Server.Builder().withHost(nativeAddr).withPort(nativePort).build(); + ClientMetrics.instance.init(Collections.singleton(server)); server.start(); for (int version : PROTOCOL_VERSIONS) diff --git a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java index d0e291a51fa4..8f2689a5d364 100644 --- a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java +++ b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java @@ -79,8 +79,7 @@ public void testDestroy() { withService((NativeTransportService service) -> { Supplier allTerminated = () -> - service.getWorkerGroup().isShutdown() && service.getWorkerGroup().isTerminated() && - service.getEventExecutor().isShutdown() && service.getEventExecutor().isTerminated(); + service.getWorkerGroup().isShutdown() && service.getWorkerGroup().isTerminated(); assertFalse(allTerminated.get()); service.destroy(); assertTrue(allTerminated.get()); diff --git a/test/unit/org/apache/cassandra/transport/InflightRequestPayloadTrackerTest.java b/test/unit/org/apache/cassandra/transport/InflightRequestPayloadTrackerTest.java new file mode 100644 index 000000000000..e4d335ba3ac2 --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/InflightRequestPayloadTrackerTest.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.transport; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; + +import org.apache.cassandra.OrderedJUnit4ClassRunner; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.exceptions.OverloadedException; +import org.apache.cassandra.transport.messages.QueryMessage; + +@RunWith(OrderedJUnit4ClassRunner.class) +public class InflightRequestPayloadTrackerTest extends CQLTester +{ + @BeforeClass + public static void setUp() + { + DatabaseDescriptor.setNativeTransportMaxConcurrentRequestsInBytesPerIp(600); + DatabaseDescriptor.setNativeTransportMaxConcurrentRequestsInBytes(600); + requireNetwork(); + } + + @AfterClass + public static void tearDown() + { + DatabaseDescriptor.setNativeTransportMaxConcurrentRequestsInBytesPerIp(3000000000L); + DatabaseDescriptor.setNativeTransportMaxConcurrentRequestsInBytes(5000000000L); + } + + @After + public void dropCreatedTable() + { + try + { + QueryProcessor.executeOnceInternal("DROP TABLE " + KEYSPACE + ".atable"); + } + catch (Throwable t) + { + // ignore + } + } + + @Test + public void testQueryExecutionWithThrowOnOverload() throws Throwable + { + SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), + nativePort, + Server.CURRENT_VERSION, + new EncryptionOptions.ClientEncryptionOptions()); + + try + { + client.connect(false, true); + QueryOptions queryOptions = QueryOptions.create( + QueryOptions.DEFAULT.getConsistency(), + QueryOptions.DEFAULT.getValues(), + QueryOptions.DEFAULT.skipMetadata(), + QueryOptions.DEFAULT.getPageSize(), + QueryOptions.DEFAULT.getPagingState(), + QueryOptions.DEFAULT.getSerialConsistency(), + Server.CURRENT_VERSION); + + QueryMessage queryMessage = new QueryMessage(String.format("CREATE TABLE %s.atable (pk1 int PRIMARY KEY, v text)", KEYSPACE), + queryOptions); + client.execute(queryMessage); + } + finally + { + client.close(); + } + } + + @Test + public void testQueryExecutionWithoutThrowOnOverload() throws Throwable + { + SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), + nativePort, + Server.CURRENT_VERSION, + new EncryptionOptions.ClientEncryptionOptions()); + + try + { + client.connect(false, false); + QueryOptions queryOptions = QueryOptions.create( + QueryOptions.DEFAULT.getConsistency(), + QueryOptions.DEFAULT.getValues(), + QueryOptions.DEFAULT.skipMetadata(), + QueryOptions.DEFAULT.getPageSize(), + QueryOptions.DEFAULT.getPagingState(), + QueryOptions.DEFAULT.getSerialConsistency(), + Server.CURRENT_VERSION); + + QueryMessage queryMessage = new QueryMessage(String.format("CREATE TABLE %s.atable (pk int PRIMARY KEY, v text)", KEYSPACE), + queryOptions); + client.execute(queryMessage); + queryMessage = new QueryMessage(String.format("SELECT * FROM %s.atable", KEYSPACE), + queryOptions); + client.execute(queryMessage); + } + finally + { + client.close(); + } + } + + @Test + public void testQueryExecutionWithoutThrowOnOverloadAndInflightLimitedExceeded() throws Throwable + { + SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), + nativePort, + Server.CURRENT_VERSION, + new EncryptionOptions.ClientEncryptionOptions()); + + try + { + client.connect(false, false); + QueryOptions queryOptions = QueryOptions.create( + QueryOptions.DEFAULT.getConsistency(), + QueryOptions.DEFAULT.getValues(), + QueryOptions.DEFAULT.skipMetadata(), + QueryOptions.DEFAULT.getPageSize(), + QueryOptions.DEFAULT.getPagingState(), + QueryOptions.DEFAULT.getSerialConsistency(), + Server.CURRENT_VERSION); + + QueryMessage queryMessage = new QueryMessage(String.format("CREATE TABLE %s.atable (pk int PRIMARY KEY, v text)", KEYSPACE), + queryOptions); + client.execute(queryMessage); + + queryMessage = new QueryMessage(String.format("INSERT INTO %s.atable (pk, v) VALUES (1, 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa')", KEYSPACE), + queryOptions); + client.execute(queryMessage); + } + finally + { + client.close(); + } + } + + @Test + public void testOverloadedExceptionForEndpointInflightLimit() throws Throwable + { + SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), + nativePort, + Server.CURRENT_VERSION, + new EncryptionOptions.ClientEncryptionOptions()); + + try + { + client.connect(false, true); + QueryOptions queryOptions = QueryOptions.create( + QueryOptions.DEFAULT.getConsistency(), + QueryOptions.DEFAULT.getValues(), + QueryOptions.DEFAULT.skipMetadata(), + QueryOptions.DEFAULT.getPageSize(), + QueryOptions.DEFAULT.getPagingState(), + QueryOptions.DEFAULT.getSerialConsistency(), + Server.CURRENT_VERSION); + + QueryMessage queryMessage = new QueryMessage(String.format("CREATE TABLE %s.atable (pk int PRIMARY KEY, v text)", KEYSPACE), + queryOptions); + client.execute(queryMessage); + + queryMessage = new QueryMessage(String.format("INSERT INTO %s.atable (pk, v) VALUES (1, 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa')", KEYSPACE), + queryOptions); + try + { + client.execute(queryMessage); + Assert.fail(); + } + catch (RuntimeException e) + { + Assert.assertTrue(e.getCause() instanceof OverloadedException); + } + } + finally + { + client.close(); + } + } + + @Test + public void testOverloadedExceptionForOverallInflightLimit() throws Throwable + { + SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), + nativePort, + Server.CURRENT_VERSION, + new EncryptionOptions.ClientEncryptionOptions()); + + try + { + client.connect(false, true); + QueryOptions queryOptions = QueryOptions.create( + QueryOptions.DEFAULT.getConsistency(), + QueryOptions.DEFAULT.getValues(), + QueryOptions.DEFAULT.skipMetadata(), + QueryOptions.DEFAULT.getPageSize(), + QueryOptions.DEFAULT.getPagingState(), + QueryOptions.DEFAULT.getSerialConsistency(), + Server.CURRENT_VERSION); + + QueryMessage queryMessage = new QueryMessage(String.format("CREATE TABLE %s.atable (pk int PRIMARY KEY, v text)", KEYSPACE), + queryOptions); + client.execute(queryMessage); + + queryMessage = new QueryMessage(String.format("INSERT INTO %s.atable (pk, v) VALUES (1, 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa')", KEYSPACE), + queryOptions); + try + { + client.execute(queryMessage); + Assert.fail(); + } + catch (RuntimeException e) + { + Assert.assertTrue(e.getCause() instanceof OverloadedException); + } + } + finally + { + client.close(); + } + } +} \ No newline at end of file From fc862e207b04ed92f15b2129ae7738186ebc6d69 Mon Sep 17 00:00:00 2001 From: Vincent White Date: Tue, 16 Jul 2019 13:13:11 +0100 Subject: [PATCH 34/78] Prevent recreation of previously dropped columns with a different kind patch by Vincent White; reviewed by Aleksey Yeschenko for CASSANDRA-14948 Co-authored-by: Aleksey Yeschenko Co-authored-by: Vincent White --- CHANGES.txt | 1 + .../apache/cassandra/config/CFMetaData.java | 24 +++++++++++++++---- .../cql3/statements/AlterTableStatement.java | 21 ++++++++++++---- .../schema/LegacySchemaMigrator.java | 2 +- .../cassandra/schema/SchemaKeyspace.java | 10 ++++++-- .../utils/NativeSSTableLoaderClient.java | 4 +++- .../cql3/validation/operations/AlterTest.java | 16 +++++++++++++ .../schema/LegacySchemaMigratorTest.java | 2 +- 8 files changed, 66 insertions(+), 14 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 68d309c69b9e..e0494f5c9ce8 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Prevent recreation of previously dropped columns with a different kind (CASSANDRA-14948) * Prevent client requests from blocking on executor task queue (CASSANDRA-15013) * Toughen up column drop/recreate type validations (CASSANDRA-15204) * LegacyLayout should handle paging states that cross a collection column (CASSANDRA-15201) diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java index c1b217113cfd..ecb0fbf6c76f 100644 --- a/src/java/org/apache/cassandra/config/CFMetaData.java +++ b/src/java/org/apache/cassandra/config/CFMetaData.java @@ -26,6 +26,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import javax.annotation.Nullable; + import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; @@ -1089,7 +1091,13 @@ public boolean removeColumnDefinition(ColumnDefinition def) */ public void recordColumnDrop(ColumnDefinition def, long deleteTimestamp) { - droppedColumns.put(def.name.bytes, new DroppedColumn(def.name.toString(), def.type, deleteTimestamp)); + recordColumnDrop(def, deleteTimestamp, true); + } + + @VisibleForTesting + public void recordColumnDrop(ColumnDefinition def, long deleteTimestamp, boolean preserveKind) + { + droppedColumns.put(def.name.bytes, new DroppedColumn(def.name.toString(), preserveKind ? def.kind : null, def.type, deleteTimestamp)); } public void renameColumn(ColumnIdentifier from, ColumnIdentifier to) throws InvalidRequestException @@ -1513,9 +1521,13 @@ public static class DroppedColumn // drop timestamp, in microseconds, yet with millisecond granularity public final long droppedTime; - public DroppedColumn(String name, AbstractType type, long droppedTime) + @Nullable + public final ColumnDefinition.Kind kind; + + public DroppedColumn(String name, ColumnDefinition.Kind kind, AbstractType type, long droppedTime) { this.name = name; + this.kind = kind; this.type = type; this.droppedTime = droppedTime; } @@ -1531,13 +1543,16 @@ public boolean equals(Object o) DroppedColumn dc = (DroppedColumn) o; - return name.equals(dc.name) && type.equals(dc.type) && droppedTime == dc.droppedTime; + return name.equals(dc.name) + && kind == dc.kind + && type.equals(dc.type) + && droppedTime == dc.droppedTime; } @Override public int hashCode() { - return Objects.hashCode(name, type, droppedTime); + return Objects.hashCode(name, kind, type, droppedTime); } @Override @@ -1545,6 +1560,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("name", name) + .add("kind", kind) .add("type", type) .add("droppedTime", droppedTime) .toString(); diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java index 9474fb07f3cb..5bff24ee98b3 100644 --- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java @@ -146,9 +146,23 @@ public Event.SchemaChange announceMigration(QueryState queryState, boolean isLoc throw new InvalidRequestException("Cannot use non-frozen collections with super column families"); } + ColumnDefinition toAdd = isStatic + ? ColumnDefinition.staticDef(cfm, columnName.bytes, type) + : ColumnDefinition.regularDef(cfm, columnName.bytes, type); + CFMetaData.DroppedColumn droppedColumn = meta.getDroppedColumns().get(columnName.bytes); if (null != droppedColumn) { + if (droppedColumn.kind != toAdd.kind) + { + String message = + String.format("Cannot re-add previously dropped column '%s' of kind %s, incompatible with previous kind %s", + columnName, + toAdd.kind, + droppedColumn.kind == null ? "UNKNOWN" : droppedColumn.kind); + throw new InvalidRequestException(message); + } + // After #8099, not safe to re-add columns of incompatible types - until *maybe* deser logic with dropped // columns is pushed deeper down the line. The latter would still be problematic in cases of schema races. if (!type.isValueCompatibleWith(droppedColumn.type)) @@ -166,12 +180,9 @@ public Event.SchemaChange announceMigration(QueryState queryState, boolean isLoc throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName)); } - cfm.addColumnDefinition(isStatic - ? ColumnDefinition.staticDef(cfm, columnName.bytes, type) - : ColumnDefinition.regularDef(cfm, columnName.bytes, type)); + cfm.addColumnDefinition(toAdd); - // Adding a column to a table which has an include all view requires the column to be added to the view - // as well + // Adding a column to a table which has an include all view requires the column to be added to the view as well if (!isStatic) { for (ViewDefinition view : views) diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java index 184effc106a5..59df65b1627f 100644 --- a/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java +++ b/src/java/org/apache/cassandra/schema/LegacySchemaMigrator.java @@ -673,7 +673,7 @@ private static void addDroppedColumns(CFMetaData cfm, AbstractType comparator ? collections.get(nameBytes) : BytesType.instance; - cfm.getDroppedColumns().put(nameBytes, new CFMetaData.DroppedColumn(name, type, time)); + cfm.getDroppedColumns().put(nameBytes, new CFMetaData.DroppedColumn(name, null, type, time)); } } diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index 94832ed5af78..8dbfe9b658e1 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -162,6 +162,7 @@ private SchemaKeyspace() + "table_name text," + "column_name text," + "dropped_time timestamp," + + "kind text," + "type text," + "PRIMARY KEY ((keyspace_name), table_name, column_name))"); @@ -672,6 +673,7 @@ private static void addDroppedColumnToSchemaMutation(CFMetaData table, CFMetaDat RowUpdateBuilder adder = new RowUpdateBuilder(DroppedColumns, timestamp, mutation).clustering(table.cfName, column.name); adder.add("dropped_time", new Date(TimeUnit.MICROSECONDS.toMillis(column.droppedTime))) + .add("kind", null != column.kind ? column.kind.toString().toLowerCase() : null) .add("type", expandUserTypes(column.type).asCQL3Type().toString()) .build(); } @@ -765,7 +767,7 @@ public static Mutation makeUpdateViewMutation(Mutation mutation, // dropped columns MapDifference droppedColumnDiff = - Maps.difference(oldView.metadata.getDroppedColumns(), oldView.metadata.getDroppedColumns()); + Maps.difference(oldView.metadata.getDroppedColumns(), oldView.metadata.getDroppedColumns()); // newly dropped columns for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values()) @@ -1096,6 +1098,10 @@ private static CFMetaData.DroppedColumn createDroppedColumnFromRow(UntypedResult { String keyspace = row.getString("keyspace_name"); String name = row.getString("column_name"); + + ColumnDefinition.Kind kind = + row.has("kind") ? ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase()) + : null; /* * we never store actual UDT names in dropped column types (so that we can safely drop types if nothing refers to * them anymore), so before storing dropped columns in schema we expand UDTs to tuples. See expandUserTypes method. @@ -1103,7 +1109,7 @@ private static CFMetaData.DroppedColumn createDroppedColumnFromRow(UntypedResult */ AbstractType type = parse(keyspace, row.getString("type"), org.apache.cassandra.schema.Types.none()); long droppedTime = TimeUnit.MILLISECONDS.toMicros(row.getLong("dropped_time")); - return new CFMetaData.DroppedColumn(name, type, droppedTime); + return new CFMetaData.DroppedColumn(name, kind, type, droppedTime); } private static Indexes fetchIndexes(String keyspace, String table) diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java index 1d65aa0736de..9ab4538828ac 100644 --- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java +++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java @@ -228,8 +228,10 @@ private static ColumnDefinition createDefinitionFromRow(Row row, String keyspace private static CFMetaData.DroppedColumn createDroppedColumnFromRow(Row row, String keyspace) { String name = row.getString("column_name"); + ColumnDefinition.Kind kind = + row.isNull("kind") ? null : ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase()); AbstractType type = CQLTypeParser.parse(keyspace, row.getString("type"), Types.none()); long droppedTime = TimeUnit.MILLISECONDS.toMicros(row.getTimestamp("dropped_time").getTime()); - return new CFMetaData.DroppedColumn(name, type, droppedTime); + return new CFMetaData.DroppedColumn(name, kind, type, droppedTime); } } diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java index 5e0c69c70ce6..126225861e41 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java @@ -313,6 +313,22 @@ public void testDropAddSameType() throws Throwable execute("alter table %s add v3 frozen>"); } + @Test(expected = InvalidRequestException.class) + public void testDropRegularAddStatic() throws Throwable + { + createTable("create table %s (k int, c int, v uuid, PRIMARY KEY (k, c))"); + execute("alter table %s drop v"); + execute("alter table %s add v uuid static"); + } + + @Test(expected = InvalidRequestException.class) + public void testDropStaticAddRegular() throws Throwable + { + createTable("create table %s (k int, c int, v uuid static, PRIMARY KEY (k, c))"); + execute("alter table %s drop v"); + execute("alter table %s add v uuid"); + } + @Test(expected = SyntaxException.class) public void renameToEmptyTest() throws Throwable { diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java index f7b0e47f1d98..7643456ea00c 100644 --- a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java +++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java @@ -300,7 +300,7 @@ private static KeyspaceMetadata keyspaceWithDroppedCollections() for (String name : collectionColumnNames) { ColumnDefinition column = table.getColumnDefinition(bytes(name)); - table.recordColumnDrop(column, FBUtilities.timestampMicros()); + table.recordColumnDrop(column, FBUtilities.timestampMicros(), false); table.removeColumnDefinition(column); } From 3f70e7c72c703bc323b169a28e8754ce67d4e479 Mon Sep 17 00:00:00 2001 From: Jay Zhuang Date: Thu, 18 Apr 2019 14:59:39 -0700 Subject: [PATCH 35/78] Avoid updating unchanged gossip states patch by Jay Zhuang; reviewed by Sam Tunnicliffe for CASSANDRA-15097 --- CHANGES.txt | 1 + .../org/apache/cassandra/gms/Gossiper.java | 21 ++- .../apache/cassandra/gms/GossiperTest.java | 128 ++++++++++++++---- 3 files changed, 124 insertions(+), 26 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index e0494f5c9ce8..f04b489d92db 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Avoid updating unchanged gossip states (CASSANDRA-15097) * Prevent recreation of previously dropped columns with a different kind (CASSANDRA-14948) * Prevent client requests from blocking on executor task queue (CASSANDRA-15013) * Toughen up column drop/recreate type validations (CASSANDRA-15204) diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 4ea0a4a69f00..c39f45aeb677 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -23,6 +23,7 @@ import java.util.Map.Entry; import java.util.concurrent.*; import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Throwables; @@ -1252,10 +1253,24 @@ private void applyNewStates(InetAddress addr, EndpointState localState, Endpoint Set> remoteStates = remoteState.states(); assert remoteState.getHeartBeatState().getGeneration() == localState.getHeartBeatState().getGeneration(); - localState.addApplicationStates(remoteStates); - for (Entry remoteEntry : remoteStates) - doOnChangeNotifications(addr, remoteEntry.getKey(), remoteEntry.getValue()); + // filter out the states that are already up to date (has the same or higher version) + Set> updatedStates = remoteStates.stream().filter(entry -> { + VersionedValue local = localState.getApplicationState(entry.getKey()); + return (local == null || local.version < entry.getValue().version); + }).collect(Collectors.toSet()); + + if (logger.isTraceEnabled() && updatedStates.size() > 0) + { + for (Entry entry : updatedStates) + { + logger.trace("Updating {} state version to {} for {}", entry.getKey().toString(), entry.getValue().version, addr); + } + } + localState.addApplicationStates(updatedStates); + + for (Entry updatedEntry : updatedStates) + doOnChangeNotifications(addr, updatedEntry.getKey(), updatedEntry.getValue()); } // notify that a local application state is going to change (doesn't get triggered for remote changes) diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java index f23c01609a13..42e4483fa746 100644 --- a/test/unit/org/apache/cassandra/gms/GossiperTest.java +++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java @@ -21,23 +21,19 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.UUID; import com.google.common.collect.ImmutableMap; -import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.Util; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.RandomPartitioner; import org.apache.cassandra.dht.Token; -import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.TokenMetadata; import org.apache.cassandra.service.StorageService; @@ -68,32 +64,118 @@ public void setup() public void testLargeGenerationJump() throws UnknownHostException, InterruptedException { Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 2); - InetAddress remoteHostAddress = hosts.get(1); + try + { + InetAddress remoteHostAddress = hosts.get(1); - EndpointState initialRemoteState = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress); - HeartBeatState initialRemoteHeartBeat = initialRemoteState.getHeartBeatState(); + EndpointState initialRemoteState = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress); + HeartBeatState initialRemoteHeartBeat = initialRemoteState.getHeartBeatState(); - //Util.createInitialRing should have initialized remoteHost's HeartBeatState's generation to 1 - assertEquals(initialRemoteHeartBeat.getGeneration(), 1); + //Util.createInitialRing should have initialized remoteHost's HeartBeatState's generation to 1 + assertEquals(initialRemoteHeartBeat.getGeneration(), 1); - HeartBeatState proposedRemoteHeartBeat = new HeartBeatState(initialRemoteHeartBeat.getGeneration() + Gossiper.MAX_GENERATION_DIFFERENCE + 1); - EndpointState proposedRemoteState = new EndpointState(proposedRemoteHeartBeat); + HeartBeatState proposedRemoteHeartBeat = new HeartBeatState(initialRemoteHeartBeat.getGeneration() + Gossiper.MAX_GENERATION_DIFFERENCE + 1); + EndpointState proposedRemoteState = new EndpointState(proposedRemoteHeartBeat); - Gossiper.instance.applyStateLocally(ImmutableMap.of(remoteHostAddress, proposedRemoteState)); + Gossiper.instance.applyStateLocally(ImmutableMap.of(remoteHostAddress, proposedRemoteState)); - //The generation should have been updated because it isn't over Gossiper.MAX_GENERATION_DIFFERENCE in the future - HeartBeatState actualRemoteHeartBeat = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress).getHeartBeatState(); - assertEquals(proposedRemoteHeartBeat.getGeneration(), actualRemoteHeartBeat.getGeneration()); + //The generation should have been updated because it isn't over Gossiper.MAX_GENERATION_DIFFERENCE in the future + HeartBeatState actualRemoteHeartBeat = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress).getHeartBeatState(); + assertEquals(proposedRemoteHeartBeat.getGeneration(), actualRemoteHeartBeat.getGeneration()); - //Propose a generation 10 years in the future - this should be rejected. - HeartBeatState badProposedRemoteHeartBeat = new HeartBeatState((int) (System.currentTimeMillis()/1000) + Gossiper.MAX_GENERATION_DIFFERENCE * 10); - EndpointState badProposedRemoteState = new EndpointState(badProposedRemoteHeartBeat); + //Propose a generation 10 years in the future - this should be rejected. + HeartBeatState badProposedRemoteHeartBeat = new HeartBeatState((int) (System.currentTimeMillis() / 1000) + Gossiper.MAX_GENERATION_DIFFERENCE * 10); + EndpointState badProposedRemoteState = new EndpointState(badProposedRemoteHeartBeat); - Gossiper.instance.applyStateLocally(ImmutableMap.of(remoteHostAddress, badProposedRemoteState)); + Gossiper.instance.applyStateLocally(ImmutableMap.of(remoteHostAddress, badProposedRemoteState)); - actualRemoteHeartBeat = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress).getHeartBeatState(); + actualRemoteHeartBeat = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress).getHeartBeatState(); - //The generation should not have been updated because it is over Gossiper.MAX_GENERATION_DIFFERENCE in the future - assertEquals(proposedRemoteHeartBeat.getGeneration(), actualRemoteHeartBeat.getGeneration()); + //The generation should not have been updated because it is over Gossiper.MAX_GENERATION_DIFFERENCE in the future + assertEquals(proposedRemoteHeartBeat.getGeneration(), actualRemoteHeartBeat.getGeneration()); + } + finally + { + // clean up the gossip states + Gossiper.instance.endpointStateMap.clear(); + } + } + + int stateChangedNum = 0; + + @Test + public void testDuplicatedStateUpdate() throws Exception + { + VersionedValue.VersionedValueFactory valueFactory = + new VersionedValue.VersionedValueFactory(DatabaseDescriptor.getPartitioner()); + + Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 2); + try + { + InetAddress remoteHostAddress = hosts.get(1); + + EndpointState initialRemoteState = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress); + HeartBeatState initialRemoteHeartBeat = initialRemoteState.getHeartBeatState(); + + //Util.createInitialRing should have initialized remoteHost's HeartBeatState's generation to 1 + assertEquals(initialRemoteHeartBeat.getGeneration(), 1); + + HeartBeatState proposedRemoteHeartBeat = new HeartBeatState(initialRemoteHeartBeat.getGeneration()); + EndpointState proposedRemoteState = new EndpointState(proposedRemoteHeartBeat); + + final Token token = DatabaseDescriptor.getPartitioner().getRandomToken(); + VersionedValue tokensValue = valueFactory.tokens(Collections.singletonList(token)); + proposedRemoteState.addApplicationState(ApplicationState.TOKENS, tokensValue); + + Gossiper.instance.register( + new IEndpointStateChangeSubscriber() + { + public void onJoin(InetAddress endpoint, EndpointState epState) { } + + public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) { } + + public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) + { + assertEquals(ApplicationState.TOKENS, state); + stateChangedNum++; + } + + public void onAlive(InetAddress endpoint, EndpointState state) { } + + public void onDead(InetAddress endpoint, EndpointState state) { } + + public void onRemove(InetAddress endpoint) { } + + public void onRestart(InetAddress endpoint, EndpointState state) { } + } + ); + + stateChangedNum = 0; + Gossiper.instance.applyStateLocally(ImmutableMap.of(remoteHostAddress, proposedRemoteState)); + assertEquals(1, stateChangedNum); + + HeartBeatState actualRemoteHeartBeat = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress).getHeartBeatState(); + assertEquals(proposedRemoteHeartBeat.getGeneration(), actualRemoteHeartBeat.getGeneration()); + + // Clone a new HeartBeatState + proposedRemoteHeartBeat = new HeartBeatState(initialRemoteHeartBeat.getGeneration(), proposedRemoteHeartBeat.getHeartBeatVersion()); + proposedRemoteState = new EndpointState(proposedRemoteHeartBeat); + + // Bump the heartbeat version and use the same TOKENS state + proposedRemoteHeartBeat.updateHeartBeat(); + proposedRemoteState.addApplicationState(ApplicationState.TOKENS, tokensValue); + + // The following state change should only update heartbeat without updating the TOKENS state + Gossiper.instance.applyStateLocally(ImmutableMap.of(remoteHostAddress, proposedRemoteState)); + assertEquals(1, stateChangedNum); + + actualRemoteHeartBeat = Gossiper.instance.getEndpointStateForEndpoint(remoteHostAddress).getHeartBeatState(); + assertEquals(proposedRemoteHeartBeat.getGeneration(), actualRemoteHeartBeat.getGeneration()); + } + finally + { + // clean up the gossip states + Gossiper.instance.endpointStateMap.clear(); + } } } From 893908e2dc1fe011e38c847395cb39be4ca53d89 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Mon, 22 Jul 2019 18:15:37 -0500 Subject: [PATCH 36/78] Nodetool status Load columns has wrong width Patch by Kirk True; Reviewed by Chris Lohfink for CASSANDRA-14787 --- CHANGES.txt | 1 + .../cassandra/tools/nodetool/Status.java | 96 ++++++------------- .../nodetool/formatter/TableBuilder.java | 9 +- 3 files changed, 35 insertions(+), 71 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index bc781d6747b1..f9c0306b0015 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Align load column in nodetool status output (CASSANDRA-14787) * CassandraNetworkAuthorizer uses cached roles info (CASSANDRA-15089) * Introduce optional timeouts for idle client sessions (CASSANDRA-11097) * Fix AlterTableStatement dropped type validation order (CASSANDRA-15203) diff --git a/src/java/org/apache/cassandra/tools/nodetool/Status.java b/src/java/org/apache/cassandra/tools/nodetool/Status.java index 21868e722220..8c37022f5173 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Status.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Status.java @@ -25,18 +25,16 @@ import java.net.UnknownHostException; import java.text.DecimalFormat; import java.util.Collection; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.SortedMap; -import java.util.function.ToIntFunction; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; import com.google.common.collect.ArrayListMultimap; @@ -51,7 +49,6 @@ public class Status extends NodeToolCmd private boolean resolveIp = false; private boolean isTokenPerNode = true; - private String format = null; private Collection joiningNodes, leavingNodes, movingNodes, liveNodes, unreachableNodes; private Map loadMap, hostIDMap; private EndpointSnitchInfoMBean epSnitchInfo; @@ -70,6 +67,7 @@ public void execute(NodeProbe probe) epSnitchInfo = probe.getEndpointSnitchInfoProxy(); StringBuilder errors = new StringBuilder(); + TableBuilder tableBuilder = new TableBuilder(" "); if (printPort) { @@ -97,8 +95,6 @@ public void execute(NodeProbe probe) if (dcs.size() < tokensToEndpoints.size()) isTokenPerNode = false; - int maxAddressLength = findMaxAddressLength(dcs, s -> s.ipOrDns().length()); - // Datacenters for (Map.Entry dc : dcs.entrySet()) { @@ -111,7 +107,7 @@ public void execute(NodeProbe probe) System.out.println("Status=Up/Down"); System.out.println("|/ State=Normal/Leaving/Joining/Moving"); - printNodesHeader(hasEffectiveOwns, isTokenPerNode, maxAddressLength); + addNodesHeader(hasEffectiveOwns, tableBuilder); ArrayListMultimap hostToTokens = ArrayListMultimap.create(); for (HostStatWithPort stat : dc.getValue()) @@ -121,10 +117,11 @@ public void execute(NodeProbe probe) { Float owns = ownerships.get(endpoint.toString()); List tokens = hostToTokens.get(endpoint); - printNodeWithPort(endpoint.toString(), owns, tokens, hasEffectiveOwns, isTokenPerNode, maxAddressLength); + addNodeWithPort(endpoint.toString(), owns, tokens, hasEffectiveOwns, tableBuilder); } } + tableBuilder.printTo(System.out); System.out.printf("%n" + errors); } else @@ -153,8 +150,6 @@ public void execute(NodeProbe probe) if (dcs.values().size() < tokensToEndpoints.keySet().size()) isTokenPerNode = false; - int maxAddressLength = findMaxAddressLength(dcs, s -> s.ipOrDns().length()); - // Datacenters for (Map.Entry dc : dcs.entrySet()) { @@ -167,7 +162,7 @@ public void execute(NodeProbe probe) System.out.println("Status=Up/Down"); System.out.println("|/ State=Normal/Leaving/Joining/Moving"); - printNodesHeader(hasEffectiveOwns, isTokenPerNode, maxAddressLength); + addNodesHeader(hasEffectiveOwns, tableBuilder); ArrayListMultimap hostToTokens = ArrayListMultimap.create(); for (HostStat stat : dc.getValue()) @@ -177,43 +172,29 @@ public void execute(NodeProbe probe) { Float owns = ownerships.get(endpoint); List tokens = hostToTokens.get(endpoint); - printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode, maxAddressLength); + addNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, tableBuilder); } } + tableBuilder.printTo(System.out); System.out.printf("%n" + errors); } } - private , U> int findMaxAddressLength(Map dcs, ToIntFunction computeLength) + private void addNodesHeader(boolean hasEffectiveOwns, TableBuilder tableBuilder) { - int maxAddressLength = 0; - - Set seenHosts = new HashSet<>(); - for (T stats : dcs.values()) - for (U stat : stats) - if (seenHosts.add(stat)) - maxAddressLength = Math.max(maxAddressLength, computeLength.applyAsInt(stat)); - - return maxAddressLength; - } - - private void printNodesHeader(boolean hasEffectiveOwns, boolean isTokenPerNode, int maxAddressLength) - { - String fmt = getFormat(hasEffectiveOwns, isTokenPerNode, maxAddressLength); String owns = hasEffectiveOwns ? "Owns (effective)" : "Owns"; if (isTokenPerNode) - System.out.printf(fmt, "-", "-", "Address", "Load", owns, "Host ID", "Token", "Rack"); + tableBuilder.add("--", "Address", "Load", owns, "Host ID", "Token", "Rack"); else - System.out.printf(fmt, "-", "-", "Address", "Load", "Tokens", owns, "Host ID", "Rack"); + tableBuilder.add("--", "Address", "Load", "Tokens", owns, "Host ID", "Rack"); } - private void printNode(String endpoint, Float owns, String epDns, String token, int size, boolean hasEffectiveOwns, - boolean isTokenPerNode, int maxAddressLength) + private void addNode(String endpoint, Float owns, String epDns, String token, int size, boolean hasEffectiveOwns, + TableBuilder tableBuilder) { - String status, state, load, strOwns, hostID, rack, fmt; - fmt = getFormat(hasEffectiveOwns, isTokenPerNode, maxAddressLength); + String status, state, load, strOwns, hostID, rack; if (liveNodes.contains(endpoint)) status = "U"; else if (unreachableNodes.contains(endpoint)) status = "D"; else status = "?"; @@ -222,6 +203,7 @@ private void printNode(String endpoint, Float owns, String epDns, String token, else if (movingNodes.contains(endpoint)) state = "M"; else state = "N"; + String statusAndState = status.concat(state); load = loadMap.getOrDefault(endpoint, "?"); strOwns = owns != null && hasEffectiveOwns ? new DecimalFormat("##0.0%").format(owns) : "?"; hostID = hostIDMap.get(endpoint); @@ -235,48 +217,24 @@ private void printNode(String endpoint, Float owns, String epDns, String token, } if (isTokenPerNode) - System.out.printf(fmt, status, state, epDns, load, strOwns, hostID, token, rack); + { + tableBuilder.add(statusAndState, epDns, load, strOwns, hostID, token, rack); + } else - System.out.printf(fmt, status, state, epDns, load, size, strOwns, hostID, rack); - } - - private void printNode(String endpoint, Float owns, List tokens, boolean hasEffectiveOwns, - boolean isTokenPerNode, int maxAddressLength) - { - printNode(endpoint, owns, tokens.get(0).ipOrDns(), tokens.get(0).token, tokens.size(), hasEffectiveOwns, - isTokenPerNode, maxAddressLength); + { + tableBuilder.add(statusAndState, epDns, load, String.valueOf(size), strOwns, hostID, rack); + } } - private void printNodeWithPort(String endpoint, Float owns, List tokens, boolean hasEffectiveOwns, - boolean isTokenPerNode, int maxAddressLength) + private void addNode(String endpoint, Float owns, List tokens, boolean hasEffectiveOwns, + TableBuilder tableBuilder) { - printNode(endpoint, owns, tokens.get(0).ipOrDns(), tokens.get(0).token, tokens.size(), hasEffectiveOwns, - isTokenPerNode, maxAddressLength); + addNode(endpoint, owns, tokens.get(0).ipOrDns(), tokens.get(0).token, tokens.size(), hasEffectiveOwns, tableBuilder); } - private String getFormat(boolean hasEffectiveOwns, boolean isTokenPerNode, int maxAddressLength) + private void addNodeWithPort(String endpoint, Float owns, List tokens, boolean hasEffectiveOwns, + TableBuilder tableBuilder) { - if (format == null) - { - StringBuilder buf = new StringBuilder(); - String addressPlaceholder = String.format("%%-%ds ", maxAddressLength); - buf.append("%s%s "); // status - buf.append(addressPlaceholder); // address - buf.append("%-9s "); // load - if (!isTokenPerNode) - buf.append("%-11s "); // "Tokens" - if (hasEffectiveOwns) - buf.append("%-16s "); // "Owns (effective)" - else - buf.append("%-6s "); // "Owns - buf.append("%-36s "); // Host ID - if (isTokenPerNode) - buf.append("%-39s "); // token - buf.append("%s%n"); // "Rack" - - format = buf.toString(); - } - - return format; + addNode(endpoint, owns, tokens.get(0).ipOrDns(), tokens.get(0).token, tokens.size(), hasEffectiveOwns, tableBuilder); } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java b/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java index a56e52eb35c5..bf06d99ce562 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java +++ b/src/java/org/apache/cassandra/tools/nodetool/formatter/TableBuilder.java @@ -41,8 +41,8 @@ */ public class TableBuilder { - // column delimiter char - private final char columnDelimiter; + // column delimiter + private final String columnDelimiter; private int[] maximumColumnWidth; private final List rows = new ArrayList<>(); @@ -53,6 +53,11 @@ public TableBuilder() } public TableBuilder(char columnDelimiter) + { + this(String.valueOf(columnDelimiter)); + } + + public TableBuilder(String columnDelimiter) { this.columnDelimiter = columnDelimiter; } From 2117e2af00603f5fb2181e53dbcba190b2eab861 Mon Sep 17 00:00:00 2001 From: Jeff Jirsa Date: Sat, 16 Mar 2019 17:30:54 +0000 Subject: [PATCH 37/78] Make repair coordination less expensive by moving MerkleTrees off heap patch by Aleksey Yeschenko and Jeff Jirsa; reviewed by Benedict Elliott Smith and Marcus Eriksson for CASSANDRA-15202 Co-authored-by: Aleksey Yeschenko Co-authored-by: Jeff Jirsa --- .../org/apache/cassandra/config/Config.java | 2 + .../cassandra/config/DatabaseDescriptor.java | 11 + .../cassandra/dht/ByteOrderedPartitioner.java | 6 + .../apache/cassandra/dht/IPartitioner.java | 5 + .../cassandra/dht/Murmur3Partitioner.java | 33 + .../cassandra/dht/RandomPartitioner.java | 32 + src/java/org/apache/cassandra/dht/Token.java | 40 +- .../apache/cassandra/repair/RepairJob.java | 3 + .../apache/cassandra/repair/Validator.java | 73 +- .../repair/asymmetric/DifferenceHolder.java | 2 + .../repair/messages/ValidationComplete.java | 9 + .../service/ActiveRepairService.java | 11 +- .../service/ActiveRepairServiceMBean.java | 3 + .../cassandra/utils/ByteBufferUtil.java | 30 +- .../apache/cassandra/utils/FBUtilities.java | 24 - .../cassandra/utils/FastByteOperations.java | 46 +- .../apache/cassandra/utils/MerkleTree.java | 1573 +++++++++++------ .../apache/cassandra/utils/MerkleTrees.java | 66 +- .../cassandra/repair/LocalSyncTaskTest.java | 4 - .../cassandra/repair/RepairJobTest.java | 4 - .../cassandra/repair/ValidatorTest.java | 18 +- .../asymmetric/DifferenceHolderTest.java | 16 +- .../cassandra/utils/MerkleTreeTest.java | 400 ++++- .../cassandra/utils/MerkleTreesTest.java | 108 +- 24 files changed, 1730 insertions(+), 789 deletions(-) diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 34a5ce8745d7..a1fdfdc786a4 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -130,6 +130,8 @@ public class Config public volatile Integer repair_session_max_tree_depth = null; public volatile Integer repair_session_space_in_mb = null; + public volatile boolean use_offheap_merkle_trees = true; + public int storage_port = 7000; public int ssl_storage_port = 7001; public String listen_address; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 0166c5fd092e..43203e50c431 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -2907,4 +2907,15 @@ public static boolean strictRuntimeChecks() { return strictRuntimeChecks; } + + public static boolean useOffheapMerkleTrees() + { + return conf.use_offheap_merkle_trees; + } + + public static void useOffheapMerkleTrees(boolean value) + { + logger.info("Setting use_offheap_merkle_trees to {}", value); + conf.use_offheap_merkle_trees = value; + } } diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java index 08088f730e2e..a6314dcccc8e 100644 --- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java +++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java @@ -234,6 +234,12 @@ public Token fromByteArray(ByteBuffer bytes) return new BytesToken(bytes); } + @Override + public int byteSize(Token token) + { + return ((BytesToken) token).token.length; + } + public String toString(Token token) { BytesToken bytesToken = (BytesToken) token; diff --git a/src/java/org/apache/cassandra/dht/IPartitioner.java b/src/java/org/apache/cassandra/dht/IPartitioner.java index f433f209e73d..ef8ced25b146 100644 --- a/src/java/org/apache/cassandra/dht/IPartitioner.java +++ b/src/java/org/apache/cassandra/dht/IPartitioner.java @@ -135,4 +135,9 @@ default Optional splitter() { return Optional.empty(); } + + default public int getMaxTokenSize() + { + return Integer.MIN_VALUE; + } } diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java index 0f922e340388..52d0efbb5837 100644 --- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java +++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.dht; +import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -25,10 +26,12 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.PreHashedDecoratedKey; +import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.PartitionerDefinedOrder; import org.apache.cassandra.db.marshal.LongType; import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.MurmurHash; import org.apache.cassandra.utils.ObjectSizes; @@ -42,6 +45,7 @@ public class Murmur3Partitioner implements IPartitioner { public static final LongToken MINIMUM = new LongToken(Long.MIN_VALUE); public static final long MAXIMUM = Long.MAX_VALUE; + private static final int MAXIMUM_TOKEN_SIZE = TypeSizes.sizeof(MAXIMUM); private static final int HEAP_SIZE = (int) ObjectSizes.measureDeep(MINIMUM); @@ -224,6 +228,11 @@ private LongToken getToken(ByteBuffer key, long[] hash) return new LongToken(normalize(hash[0])); } + public int getMaxTokenSize() + { + return MAXIMUM_TOKEN_SIZE; + } + private long[] getHash(ByteBuffer key) { long[] hash = new long[2]; @@ -300,11 +309,35 @@ public ByteBuffer toByteArray(Token token) return ByteBufferUtil.bytes(longToken.token); } + @Override + public void serialize(Token token, DataOutputPlus out) throws IOException + { + out.writeLong(((LongToken) token).token); + } + + @Override + public void serialize(Token token, ByteBuffer out) + { + out.putLong(((LongToken) token).token); + } + + @Override + public int byteSize(Token token) + { + return 8; + } + public Token fromByteArray(ByteBuffer bytes) { return new LongToken(ByteBufferUtil.toLong(bytes)); } + @Override + public Token fromByteBuffer(ByteBuffer bytes, int position, int length) + { + return new LongToken(bytes.getLong(position)); + } + public String toString(Token token) { return token.toString(); diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java index 4e63475bbe47..0457a893f0c5 100644 --- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java +++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java @@ -17,6 +17,7 @@ */ package org.apache.cassandra.dht; +import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -31,6 +32,7 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.IntegerType; import org.apache.cassandra.db.marshal.PartitionerDefinedOrder; +import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.GuidGenerator; @@ -46,6 +48,7 @@ public class RandomPartitioner implements IPartitioner public static final BigInteger ZERO = new BigInteger("0"); public static final BigIntegerToken MINIMUM = new BigIntegerToken("-1"); public static final BigInteger MAXIMUM = new BigInteger("2").pow(127); + public static final int MAXIMUM_TOKEN_SIZE = MAXIMUM.bitLength() / 8 + 1; /** * Maintain a separate threadlocal message digest, exclusively for token hashing. This is necessary because @@ -162,11 +165,35 @@ public ByteBuffer toByteArray(Token token) return ByteBuffer.wrap(bigIntegerToken.token.toByteArray()); } + @Override + public void serialize(Token token, DataOutputPlus out) throws IOException + { + out.write(((BigIntegerToken) token).token.toByteArray()); + } + + @Override + public void serialize(Token token, ByteBuffer out) + { + out.put(((BigIntegerToken) token).token.toByteArray()); + } + + @Override + public int byteSize(Token token) + { + return ((BigIntegerToken) token).token.bitLength() / 8 + 1; + } + public Token fromByteArray(ByteBuffer bytes) { return new BigIntegerToken(new BigInteger(ByteBufferUtil.getArray(bytes))); } + @Override + public Token fromByteBuffer(ByteBuffer bytes, int position, int length) + { + return new BigIntegerToken(new BigInteger(ByteBufferUtil.getArray(bytes, position, length))); + } + public String toString(Token token) { BigIntegerToken bigIntegerToken = (BigIntegerToken) token; @@ -252,6 +279,11 @@ public BigIntegerToken getToken(ByteBuffer key) return new BigIntegerToken(hashToBigInteger(key)); } + public int getMaxTokenSize() + { + return MAXIMUM_TOKEN_SIZE; + } + public Map describeOwnership(List sortedTokens) { Map ownerships = new HashMap(); diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java index 20b45ef0f237..ccb66fd4bb3e 100644 --- a/src/java/org/apache/cassandra/dht/Token.java +++ b/src/java/org/apache/cassandra/dht/Token.java @@ -26,7 +26,6 @@ import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.utils.ByteBufferUtil; public abstract class Token implements RingPosition, Serializable { @@ -40,8 +39,30 @@ public static abstract class TokenFactory public abstract Token fromByteArray(ByteBuffer bytes); public abstract String toString(Token token); // serialize as string, not necessarily human-readable public abstract Token fromString(String string); // deserialize - public abstract void validate(String token) throws ConfigurationException; + + public void serialize(Token token, DataOutputPlus out) throws IOException + { + out.write(toByteArray(token)); + } + + public void serialize(Token token, ByteBuffer out) throws IOException + { + out.put(toByteArray(token)); + } + + public Token fromByteBuffer(ByteBuffer bytes, int position, int length) + { + bytes = bytes.duplicate(); + bytes.position(position) + .limit(position + length); + return fromByteArray(bytes); + } + + public int byteSize(Token token) + { + return toByteArray(token).remaining(); + } } public static class TokenSerializer implements IPartitionerDependentSerializer @@ -49,23 +70,28 @@ public static class TokenSerializer implements IPartitionerDependentSerializer= version 2.0 - MessagingService.instance().send(Message.out(REPAIR_REQ, new ValidationComplete(desc)), initiator); + respond(new ValidationComplete(desc)); } /** @@ -413,12 +399,51 @@ public void fail() */ public void run() { - // respond to the request that triggered this validation - if (!initiator.equals(FBUtilities.getBroadcastAddressAndPort())) + if (initiatorIsRemote()) { logger.info("{} Sending completed merkle tree to {} for {}.{}", previewKind.logPrefix(desc.sessionId), initiator, desc.keyspace, desc.columnFamily); Tracing.traceRepair("Sending completed merkle tree to {} for {}.{}", initiator, desc.keyspace, desc.columnFamily); } - MessagingService.instance().send(Message.out(REPAIR_REQ, new ValidationComplete(desc, trees)), initiator); + else + { + logger.info("{} Local completed merkle tree for {} for {}.{}", previewKind.logPrefix(desc.sessionId), initiator, desc.keyspace, desc.columnFamily); + Tracing.traceRepair("Local completed merkle tree for {} for {}.{}", initiator, desc.keyspace, desc.columnFamily); + + } + respond(new ValidationComplete(desc, trees)); + } + + private boolean initiatorIsRemote() + { + return !FBUtilities.getBroadcastAddressAndPort().equals(initiator); + } + + private void respond(ValidationComplete response) + { + if (initiatorIsRemote()) + { + MessagingService.instance().send(Message.out(REPAIR_REQ, response), initiator); + return; + } + + /* + * For local initiators, DO NOT send the message to self over loopback. This is a wasted ser/de loop + * and a ton of garbage. Instead, move the trees off heap and invoke message handler. We could do it + * directly, since this method will only be called from {@code Stage.ENTI_ENTROPY}, but we do instead + * execute a {@code Runnable} on the stage - in case that assumption ever changes by accident. + */ + StageManager.getStage(Stage.ANTI_ENTROPY).execute(() -> + { + ValidationComplete movedResponse = response; + try + { + movedResponse = response.tryMoveOffHeap(); + } + catch (IOException e) + { + logger.error("Failed to move local merkle tree for {} off heap", desc, e); + } + ActiveRepairService.instance.handleMessage(initiator, movedResponse); + }); } } diff --git a/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java b/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java index c9b7ed7ccf7a..f85c2ebb05d9 100644 --- a/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java +++ b/src/java/org/apache/cassandra/repair/asymmetric/DifferenceHolder.java @@ -51,9 +51,11 @@ public DifferenceHolder(List trees) TreeResponse r2 = trees.get(j); hd.add(r2.endpoint, MerkleTrees.difference(r1.trees, r2.trees)); } + r1.trees.release(); // and add them to the diff map diffBuilder.put(r1.endpoint, hd); } + trees.get(trees.size() - 1).trees.release(); differences = diffBuilder.build(); } diff --git a/src/java/org/apache/cassandra/repair/messages/ValidationComplete.java b/src/java/org/apache/cassandra/repair/messages/ValidationComplete.java index 704bffb09353..b8aa736b8804 100644 --- a/src/java/org/apache/cassandra/repair/messages/ValidationComplete.java +++ b/src/java/org/apache/cassandra/repair/messages/ValidationComplete.java @@ -56,6 +56,15 @@ public boolean success() return trees != null; } + /** + * @return a new {@link ValidationComplete} instance with all trees moved off heap, or {@code this} + * if it's a failure response. + */ + public ValidationComplete tryMoveOffHeap() throws IOException + { + return trees == null ? this : new ValidationComplete(desc, trees.tryMoveOffHeap()); + } + @Override public boolean equals(Object o) { diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index abfd6d9c7927..409d7991a5df 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -71,7 +71,6 @@ import org.apache.cassandra.repair.messages.*; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.utils.CassandraVersion; -import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.Pair; @@ -248,6 +247,16 @@ public void run() return session; } + public boolean getUseOffheapMerkleTrees() + { + return DatabaseDescriptor.useOffheapMerkleTrees(); + } + + public void setUseOffheapMerkleTrees(boolean value) + { + DatabaseDescriptor.useOffheapMerkleTrees(value); + } + private void registerOnFdAndGossip(final T task) diff --git a/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java b/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java index 283d466b9c7d..d967280e83bc 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairServiceMBean.java @@ -30,4 +30,7 @@ public interface ActiveRepairServiceMBean public void setRepairSessionSpaceInMegabytes(int sizeInMegabytes); public int getRepairSessionSpaceInMegabytes(); + + public boolean getUseOffheapMerkleTrees(); + public void setUseOffheapMerkleTrees(boolean value); } diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java index 788300c7461e..518436ea2527 100644 --- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java +++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java @@ -101,6 +101,16 @@ public static int compare(ByteBuffer o1, byte[] o2) return FastByteOperations.compareUnsigned(o1, o2, 0, o2.length); } + public static int compare(ByteBuffer o1, int s1, int l1, byte[] o2) + { + return FastByteOperations.compareUnsigned(o1, s1, l1, o2, 0, o2.length); + } + + public static int compare(byte[] o1, ByteBuffer o2, int s2, int l2) + { + return FastByteOperations.compareUnsigned(o1, 0, o1.length, o2, s2, l2); + } + /** * Decode a String representation. * This method assumes that the encoding charset is UTF_8. @@ -161,16 +171,25 @@ public static String string(ByteBuffer buffer, Charset charset) throws Character */ public static byte[] getArray(ByteBuffer buffer) { - int length = buffer.remaining(); + return getArray(buffer, buffer.position(), buffer.remaining()); + } + + /** + * You should almost never use this. Instead, use the write* methods to avoid copies. + */ + public static byte[] getArray(ByteBuffer buffer, int position, int length) + { if (buffer.hasArray()) { - int boff = buffer.arrayOffset() + buffer.position(); + int boff = buffer.arrayOffset() + position; return Arrays.copyOfRange(buffer.array(), boff, boff + length); } + // else, DirectByteBuffer.get() is the fastest route byte[] bytes = new byte[length]; - buffer.duplicate().get(bytes); - + ByteBuffer dup = buffer.duplicate(); + dup.position(position).limit(position + length); + dup.get(bytes); return bytes; } @@ -631,6 +650,7 @@ public static int compareSubArrays(ByteBuffer bytes1, int offset1, ByteBuffer by assert bytes1.limit() >= offset1 + length : "The first byte array isn't long enough for the specified offset and length."; assert bytes2.limit() >= offset2 + length : "The second byte array isn't long enough for the specified offset and length."; + for (int i = 0; i < length; i++) { byte byte1 = bytes1.get(offset1 + i); @@ -669,7 +689,7 @@ public static ByteBuffer minimalBufferFor(ByteBuffer buf) return buf.capacity() > buf.remaining() || !buf.hasArray() ? ByteBuffer.wrap(getArray(buf)) : buf; } - // Doesn't change bb position + // doesn't change bb position public static int getShortLength(ByteBuffer bb, int position) { int length = (bb.get(position) & 0xFF) << 8; diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index c37dccacdc97..f0d913246b57 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -265,30 +265,6 @@ public static int compareUnsigned(byte[] bytes1, byte[] bytes2) return compareUnsigned(bytes1, bytes2, 0, 0, bytes1.length, bytes2.length); } - /** - * @return The bitwise XOR of the inputs. The output will be the same length as the - * longer input, but if either input is null, the output will be null. - */ - public static byte[] xor(byte[] left, byte[] right) - { - if (left == null || right == null) - return null; - if (left.length > right.length) - { - byte[] swap = left; - left = right; - right = swap; - } - - // left.length is now <= right.length - byte[] out = Arrays.copyOf(right, right.length); - for (int i = 0; i < left.length; i++) - { - out[i] = (byte)((left[i] & 0xFF) ^ (right[i] & 0xFF)); - } - return out; - } - public static void sortSampledKeys(List keys, Range range) { if (range.left.compareTo(range.right) >= 0) diff --git a/src/java/org/apache/cassandra/utils/FastByteOperations.java b/src/java/org/apache/cassandra/utils/FastByteOperations.java index 6581736c5d0b..060dee59ad4a 100644 --- a/src/java/org/apache/cassandra/utils/FastByteOperations.java +++ b/src/java/org/apache/cassandra/utils/FastByteOperations.java @@ -55,6 +55,16 @@ public static int compareUnsigned(byte[] b1, int s1, int l1, ByteBuffer b2) return -BestHolder.BEST.compare(b2, b1, s1, l1); } + public static int compareUnsigned(ByteBuffer b1, int s1, int l1, byte[] b2, int s2, int l2) + { + return BestHolder.BEST.compare(b1, s1, l1, b2, s2, l2); + } + + public static int compareUnsigned(byte[] b1, int s1, int l1, ByteBuffer b2, int s2, int l2) + { + return -BestHolder.BEST.compare(b2, s2, l2, b1, s1, l1); + } + public static int compareUnsigned(ByteBuffer b1, ByteBuffer b2) { return BestHolder.BEST.compare(b1, b2); @@ -77,6 +87,8 @@ abstract public int compare(byte[] buffer1, int offset1, int length1, abstract public int compare(ByteBuffer buffer1, byte[] buffer2, int offset2, int length2); + abstract public int compare(ByteBuffer buffer1, int offset1, int length1, byte[] buffer2, int offset2, int length2); + abstract public int compare(ByteBuffer buffer1, ByteBuffer buffer2); abstract public void copy(ByteBuffer src, int srcPosition, byte[] trg, int trgPosition, int length); @@ -186,26 +198,25 @@ public int compare(byte[] buffer1, int offset1, int length1, byte[] buffer2, int } public int compare(ByteBuffer buffer1, byte[] buffer2, int offset2, int length2) + { + return compare(buffer1, buffer1.position(), buffer1.remaining(), buffer2, offset2, length2); + } + + public int compare(ByteBuffer buffer1, int position1, int length1, byte[] buffer2, int offset2, int length2) { Object obj1; long offset1; if (buffer1.hasArray()) { obj1 = buffer1.array(); - offset1 = BYTE_ARRAY_BASE_OFFSET + buffer1.arrayOffset(); + offset1 = BYTE_ARRAY_BASE_OFFSET + buffer1.arrayOffset() + position1; } else { obj1 = null; - offset1 = theUnsafe.getLong(buffer1, DIRECT_BUFFER_ADDRESS_OFFSET); - } - int length1; - { - int position = buffer1.position(); - int limit = buffer1.limit(); - length1 = limit - position; - offset1 += position; + offset1 = theUnsafe.getLong(buffer1, DIRECT_BUFFER_ADDRESS_OFFSET) + position1; } + return compareTo(obj1, offset1, length1, buffer2, BYTE_ARRAY_BASE_OFFSET + offset2, length2); } @@ -397,11 +408,28 @@ public int compare(byte[] buffer1, int offset1, int length1, return length1 - length2; } + public int compare(ByteBuffer buffer1, int position1, int length1, byte[] buffer2, int offset2, int length2) + { + if (buffer1.hasArray()) + return compare(buffer1.array(), buffer1.arrayOffset() + position1, length1, buffer2, offset2, length2); + + if (position1 != buffer1.position()) + { + buffer1 = buffer1.duplicate(); + buffer1.position(position1); + } + + return compare(buffer1, ByteBuffer.wrap(buffer2, offset2, length2)); + } + public int compare(ByteBuffer buffer1, byte[] buffer2, int offset2, int length2) { if (buffer1.hasArray()) + { return compare(buffer1.array(), buffer1.arrayOffset() + buffer1.position(), buffer1.remaining(), buffer2, offset2, length2); + } + return compare(buffer1, ByteBuffer.wrap(buffer2, offset2, length2)); } diff --git a/src/java/org/apache/cassandra/utils/MerkleTree.java b/src/java/org/apache/cassandra/utils/MerkleTree.java index d131ff58f539..1b9255511d08 100644 --- a/src/java/org/apache/cassandra/utils/MerkleTree.java +++ b/src/java/org/apache/cassandra/utils/MerkleTree.java @@ -19,25 +19,35 @@ import java.io.DataInput; import java.io.IOException; -import java.io.Serializable; +import java.nio.ByteBuffer; import java.util.*; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.PeekingIterator; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Shorts; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.IPartitioner; -import org.apache.cassandra.dht.IPartitionerDependentSerializer; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.RandomPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.ConfigurationException; -import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.concurrent.Ref; +import org.apache.cassandra.utils.memory.MemoryUtil; + +import static java.lang.String.format; +import static org.apache.cassandra.db.TypeSizes.sizeof; +import static org.apache.cassandra.utils.ByteBufferUtil.compare; +import static org.apache.cassandra.utils.MerkleTree.Difference.*; /** * A MerkleTree implemented as a binary tree. @@ -59,84 +69,45 @@ * If two MerkleTrees have the same hashdepth, they represent a perfect tree * of the same depth, and can always be compared, regardless of size or splits. */ -public class MerkleTree implements Serializable +public class MerkleTree { - private static Logger logger = LoggerFactory.getLogger(MerkleTree.class); + private static final Logger logger = LoggerFactory.getLogger(MerkleTree.class); - public static final MerkleTreeSerializer serializer = new MerkleTreeSerializer(); - private static final long serialVersionUID = 2L; + private static final int HASH_SIZE = 32; // 2xMM3_128 = 32 bytes. + private static final byte[] EMPTY_HASH = new byte[HASH_SIZE]; - public static final byte RECOMMENDED_DEPTH = Byte.MAX_VALUE - 1; + /* + * Thread-local byte array, large enough to host 32B of digest or MM3/Random partitoners' tokens + */ + private static final ThreadLocal byteArray = ThreadLocal.withInitial(() -> new byte[HASH_SIZE]); + + private static byte[] getTempArray(int minimumSize) + { + return minimumSize <= HASH_SIZE ? byteArray.get() : new byte[minimumSize]; + } - public static final int CONSISTENT = 0; - public static final int FULLY_INCONSISTENT = 1; - public static final int PARTIALLY_INCONSISTENT = 2; - private static final byte[] EMPTY_HASH = new byte[0]; + public static final byte RECOMMENDED_DEPTH = Byte.MAX_VALUE - 1; - public final byte hashdepth; + private final int hashdepth; /** The top level range that this MerkleTree covers. */ - public final Range fullRange; + final Range fullRange; private final IPartitioner partitioner; private long maxsize; private long size; - private Hashable root; + private Node root; - public static class MerkleTreeSerializer implements IVersionedSerializer + /** + * @param partitioner The partitioner in use. + * @param range the range this tree covers + * @param hashdepth The maximum depth of the tree. 100/(2^depth) is the % + * of the key space covered by each subrange of a fully populated tree. + * @param maxsize The maximum number of subranges in the tree. + */ + public MerkleTree(IPartitioner partitioner, Range range, int hashdepth, long maxsize) { - public void serialize(MerkleTree mt, DataOutputPlus out, int version) throws IOException - { - out.writeByte(mt.hashdepth); - out.writeLong(mt.maxsize); - out.writeLong(mt.size); - out.writeUTF(mt.partitioner.getClass().getCanonicalName()); - // full range - Token.serializer.serialize(mt.fullRange.left, out, version); - Token.serializer.serialize(mt.fullRange.right, out, version); - Hashable.serializer.serialize(mt.root, out, version); - } - - public MerkleTree deserialize(DataInputPlus in, int version) throws IOException - { - byte hashdepth = in.readByte(); - long maxsize = in.readLong(); - long size = in.readLong(); - IPartitioner partitioner; - try - { - partitioner = FBUtilities.newPartitioner(in.readUTF()); - } - catch (ConfigurationException e) - { - throw new IOException(e); - } - - // full range - Token left = Token.serializer.deserialize(in, partitioner, version); - Token right = Token.serializer.deserialize(in, partitioner, version); - Range fullRange = new Range<>(left, right); - - MerkleTree mt = new MerkleTree(partitioner, fullRange, hashdepth, maxsize); - mt.size = size; - mt.root = Hashable.serializer.deserialize(in, partitioner, version); - return mt; - } - - public long serializedSize(MerkleTree mt, int version) - { - long size = 1 // mt.hashdepth - + TypeSizes.sizeof(mt.maxsize) - + TypeSizes.sizeof(mt.size) - + TypeSizes.sizeof(mt.partitioner.getClass().getCanonicalName()); - - // full range - size += Token.serializer.serializedSize(mt.fullRange.left, version); - size += Token.serializer.serializedSize(mt.fullRange.right, version); - - size += Hashable.serializer.serializedSize(mt.root, version); - return size; - } + this(new OnHeapLeaf(), partitioner, range, hashdepth, maxsize, 1); } /** @@ -145,60 +116,56 @@ public long serializedSize(MerkleTree mt, int version) * @param hashdepth The maximum depth of the tree. 100/(2^depth) is the % * of the key space covered by each subrange of a fully populated tree. * @param maxsize The maximum number of subranges in the tree. + * @param size The size of the tree. Typically 1, unless deserilized from an existing tree */ - public MerkleTree(IPartitioner partitioner, Range range, byte hashdepth, long maxsize) + private MerkleTree(Node root, IPartitioner partitioner, Range range, int hashdepth, long maxsize, long size) { assert hashdepth < Byte.MAX_VALUE; + + this.root = root; this.fullRange = Preconditions.checkNotNull(range); this.partitioner = Preconditions.checkNotNull(partitioner); this.hashdepth = hashdepth; this.maxsize = maxsize; - - size = 1; - root = new Leaf(null); - } - - - static byte inc(byte in) - { - assert in < Byte.MAX_VALUE; - return (byte)(in + 1); + this.size = size; } /** * Initializes this tree by splitting it until hashdepth is reached, * or until an additional level of splits would violate maxsize. * - * NB: Replaces all nodes in the tree. + * NB: Replaces all nodes in the tree, and always builds on the heap */ public void init() { // determine the depth to which we can safely split the tree - byte sizedepth = (byte)(Math.log10(maxsize) / Math.log10(2)); - byte depth = (byte)Math.min(sizedepth, hashdepth); + int sizedepth = (int) (Math.log10(maxsize) / Math.log10(2)); + int depth = Math.min(sizedepth, hashdepth); - root = initHelper(fullRange.left, fullRange.right, (byte)0, depth); - size = (long)Math.pow(2, depth); + root = initHelper(fullRange.left, fullRange.right, 0, depth); + size = (long) Math.pow(2, depth); } - private Hashable initHelper(Token left, Token right, byte depth, byte max) + private OnHeapNode initHelper(Token left, Token right, int depth, int max) { if (depth == max) // we've reached the leaves - return new Leaf(); + return new OnHeapLeaf(); Token midpoint = partitioner.midpoint(left, right); if (midpoint.equals(left) || midpoint.equals(right)) - return new Leaf(); + return new OnHeapLeaf(); - Hashable lchild = initHelper(left, midpoint, inc(depth), max); - Hashable rchild = initHelper(midpoint, right, inc(depth), max); - return new Inner(midpoint, lchild, rchild); + OnHeapNode leftChild = initHelper(left, midpoint, depth + 1, max); + OnHeapNode rightChild = initHelper(midpoint, right, depth + 1, max); + return new OnHeapInner(midpoint, leftChild, rightChild); } - Hashable root() + public void release() { - return root; + if (root instanceof OffHeapNode) + ((OffHeapNode) root).release(); + root = null; } public IPartitioner partitioner() @@ -233,20 +200,21 @@ public void maxsize(long maxsize) public static List difference(MerkleTree ltree, MerkleTree rtree) { if (!ltree.fullRange.equals(rtree.fullRange)) - throw new IllegalArgumentException("Difference only make sense on tree covering the same range (but " + ltree.fullRange + " != " + rtree.fullRange + ")"); + throw new IllegalArgumentException("Difference only make sense on tree covering the same range (but " + ltree.fullRange + " != " + rtree.fullRange + ')'); + + // ensure on-heap trees' inner node hashes have been computed + ltree.fillInnerHashes(); + rtree.fillInnerHashes(); List diff = new ArrayList<>(); - TreeDifference active = new TreeDifference(ltree.fullRange.left, ltree.fullRange.right, (byte)0); + TreeRange active = new TreeRange(ltree.fullRange.left, ltree.fullRange.right, 0); - Hashable lnode = ltree.find(active); - Hashable rnode = rtree.find(active); - byte[] lhash = lnode.hash(); - byte[] rhash = rnode.hash(); - active.setSize(lnode.sizeOfRange(), rnode.sizeOfRange()); + Node lnode = ltree.root; + Node rnode = rtree.root; - if (lhash != null && rhash != null && !Arrays.equals(lhash, rhash)) + if (lnode.hashesDiffer(rnode)) { - if(lnode instanceof Leaf || rnode instanceof Leaf) + if (lnode instanceof Leaf || rnode instanceof Leaf) { logger.debug("Digest mismatch detected among leaf nodes {}, {}", lnode, rnode); diff.add(active); @@ -261,20 +229,20 @@ public static List difference(MerkleTree ltree, MerkleTree rtree) } } } - else if (lhash == null || rhash == null) - diff.add(active); + return diff; } + enum Difference { CONSISTENT, FULLY_INCONSISTENT, PARTIALLY_INCONSISTENT } + /** - * TODO: This function could be optimized into a depth first traversal of - * the two trees in parallel. + * TODO: This function could be optimized into a depth first traversal of the two trees in parallel. * * Takes two trees and a range for which they have hashes, but are inconsistent. * @return FULLY_INCONSISTENT if active is inconsistent, PARTIALLY_INCONSISTENT if only a subrange is inconsistent. */ @VisibleForTesting - static int differenceHelper(MerkleTree ltree, MerkleTree rtree, List diff, TreeRange active) + static Difference differenceHelper(MerkleTree ltree, MerkleTree rtree, List diff, TreeRange active) { if (active.depth == Byte.MAX_VALUE) return CONSISTENT; @@ -289,29 +257,26 @@ static int differenceHelper(MerkleTree ltree, MerkleTree rtree, List return FULLY_INCONSISTENT; } - TreeDifference left = new TreeDifference(active.left, midpoint, inc(active.depth)); - TreeDifference right = new TreeDifference(midpoint, active.right, inc(active.depth)); + TreeRange left = new TreeRange(active.left, midpoint, active.depth + 1); + TreeRange right = new TreeRange(midpoint, active.right, active.depth + 1); logger.debug("({}) Hashing sub-ranges [{}, {}] for {} divided by midpoint {}", active.depth, left, right, active, midpoint); - byte[] lhash, rhash; - Hashable lnode, rnode; + Node lnode, rnode; // see if we should recurse left lnode = ltree.find(left); rnode = rtree.find(left); - lhash = lnode.hash(); - rhash = rnode.hash(); - left.setSize(lnode.sizeOfRange(), rnode.sizeOfRange()); - left.setRows(lnode.rowsInRange(), rnode.rowsInRange()); - int ldiff = CONSISTENT; - boolean lreso = lhash != null && rhash != null; - if (lreso && !Arrays.equals(lhash, rhash)) + Difference ldiff = CONSISTENT; + if (null != lnode && null != rnode && lnode.hashesDiffer(rnode)) { logger.debug("({}) Inconsistent digest on left sub-range {}: [{}, {}]", active.depth, left, lnode, rnode); - if (lnode instanceof Leaf) ldiff = FULLY_INCONSISTENT; - else ldiff = differenceHelper(ltree, rtree, diff, left); + + if (lnode instanceof Leaf) + ldiff = FULLY_INCONSISTENT; + else + ldiff = differenceHelper(ltree, rtree, diff, left); } - else if (!lreso) + else if (null == lnode || null == rnode) { logger.debug("({}) Left sub-range fully inconsistent {}", active.depth, left); ldiff = FULLY_INCONSISTENT; @@ -320,20 +285,18 @@ else if (!lreso) // see if we should recurse right lnode = ltree.find(right); rnode = rtree.find(right); - lhash = lnode.hash(); - rhash = rnode.hash(); - right.setSize(lnode.sizeOfRange(), rnode.sizeOfRange()); - right.setRows(lnode.rowsInRange(), rnode.rowsInRange()); - int rdiff = CONSISTENT; - boolean rreso = lhash != null && rhash != null; - if (rreso && !Arrays.equals(lhash, rhash)) + Difference rdiff = CONSISTENT; + if (null != lnode && null != rnode && lnode.hashesDiffer(rnode)) { logger.debug("({}) Inconsistent digest on right sub-range {}: [{}, {}]", active.depth, right, lnode, rnode); - if (rnode instanceof Leaf) rdiff = FULLY_INCONSISTENT; - else rdiff = differenceHelper(ltree, rtree, diff, right); + + if (rnode instanceof Leaf) + rdiff = FULLY_INCONSISTENT; + else + rdiff = differenceHelper(ltree, rtree, diff, right); } - else if (!rreso) + else if (null == lnode || null == rnode) { logger.debug("({}) Right sub-range fully inconsistent {}", active.depth, right); rdiff = FULLY_INCONSISTENT; @@ -362,133 +325,70 @@ else if (rdiff == FULLY_INCONSISTENT) } /** - * For testing purposes. - * Gets the smallest range containing the token. - */ - public TreeRange get(Token t) - { - return getHelper(root, fullRange.left, fullRange.right, (byte)0, t); - } - - TreeRange getHelper(Hashable hashable, Token pleft, Token pright, byte depth, Token t) - { - while (true) - { - if (hashable instanceof Leaf) - { - // we've reached a hash: wrap it up and deliver it - return new TreeRange(this, pleft, pright, depth, hashable); - } - // else: node. - - Inner node = (Inner) hashable; - depth = inc(depth); - if (Range.contains(pleft, node.token, t)) - { // left child contains token - hashable = node.lchild; - pright = node.token; - } - else - { // else: right child contains token - hashable = node.rchild; - pleft = node.token; - } - } - } - - /** - * Invalidates the ranges containing the given token. - * Useful for testing. - */ - public void invalidate(Token t) - { - invalidateHelper(root, fullRange.left, t); - } - - private void invalidateHelper(Hashable hashable, Token pleft, Token t) - { - hashable.hash(null); - if (hashable instanceof Leaf) - return; - // else: node. - - Inner node = (Inner)hashable; - if (Range.contains(pleft, node.token, t)) - // left child contains token - invalidateHelper(node.lchild, pleft, t); - else - // right child contains token - invalidateHelper(node.rchild, node.token, t); - } - - /** - * Hash the given range in the tree. The range must have been generated - * with recursive applications of partitioner.midpoint(). - * - * NB: Currently does not support wrapping ranges that do not end with - * partitioner.getMinimumToken(). - * - * @return Null if any subrange of the range is invalid, or if the exact - * range cannot be calculated using this tree. + * Exceptions that stop recursion early when we are sure that no answer + * can be found. */ - public byte[] hash(Range range) + static abstract class StopRecursion extends Exception { - return find(range).hash(); + static class TooDeep extends StopRecursion {} + static class BadRange extends StopRecursion {} } /** - * Find the {@link Hashable} node that matches the given {@code range}. + * Find the {@link Node} node that matches the given {@code range}. * * @param range Range to find - * @return {@link Hashable} found. If nothing found, return {@link Leaf} with null hash. + * @return {@link Node} found. If nothing found, return {@code null} */ - private Hashable find(Range range) + @VisibleForTesting + private Node find(Range range) { try { - return findHelper(root, new Range(fullRange.left, fullRange.right), range); + return findHelper(root, fullRange, range); } catch (StopRecursion e) { - return new Leaf(); + return null; } } /** * @throws StopRecursion If no match could be found for the range. */ - private Hashable findHelper(Hashable current, Range activeRange, Range find) throws StopRecursion + private Node findHelper(Node current, Range activeRange, Range find) throws StopRecursion { while (true) { if (current instanceof Leaf) { if (!find.contains(activeRange)) - // we are not fully contained in this range! - throw new StopRecursion.BadRange(); + throw new StopRecursion.BadRange(); // we are not fully contained in this range! + return current; } - // else: node. - Inner node = (Inner) current; - Range leftRange = new Range<>(activeRange.left, node.token); - Range rightRange = new Range<>(node.token, activeRange.right); + assert current instanceof Inner; + Inner inner = (Inner) current; - if (find.contains(activeRange)) - // this node is fully contained in the range - return node.calc(); + if (find.contains(activeRange)) // this node is fully contained in the range + return inner.fillInnerHashes(); + + Token midpoint = inner.token(); + Range leftRange = new Range<>(activeRange.left, midpoint); + Range rightRange = new Range<>(midpoint, activeRange.right); // else: one of our children contains the range - if (leftRange.contains(find)) - { // left child contains/matches the range - current = node.lchild; + if (leftRange.contains(find)) // left child contains/matches the range + { activeRange = leftRange; + current = inner.left(); } - else if (rightRange.contains(find)) - { // right child contains/matches the range - current = node.rchild; + else if (rightRange.contains(find)) // right child contains/matches the range + { activeRange = rightRange; + current = inner.right(); } else { @@ -506,12 +406,12 @@ else if (rightRange.contains(find)) */ public boolean split(Token t) { - if (!(size < maxsize)) + if (size >= maxsize) return false; try { - root = splitHelper(root, fullRange.left, fullRange.right, (byte)0, t); + root = splitHelper(root, fullRange.left, fullRange.right, 0, t); } catch (StopRecursion.TooDeep e) { @@ -520,12 +420,12 @@ public boolean split(Token t) return true; } - private Hashable splitHelper(Hashable hashable, Token pleft, Token pright, byte depth, Token t) throws StopRecursion.TooDeep + private OnHeapNode splitHelper(Node node, Token pleft, Token pright, int depth, Token t) throws StopRecursion.TooDeep { if (depth >= hashdepth) throw new StopRecursion.TooDeep(); - if (hashable instanceof Leaf) + if (node instanceof Leaf) { Token midpoint = partitioner.midpoint(pleft, pright); @@ -536,47 +436,47 @@ private Hashable splitHelper(Hashable hashable, Token pleft, Token pright, byte // split size++; - return new Inner(midpoint, new Leaf(), new Leaf()); + return new OnHeapInner(midpoint, new OnHeapLeaf(), new OnHeapLeaf()); } // else: node. // recurse on the matching child - Inner node = (Inner)hashable; + assert node instanceof OnHeapInner; + OnHeapInner inner = (OnHeapInner) node; - if (Range.contains(pleft, node.token, t)) - // left child contains token - node.lchild(splitHelper(node.lchild, pleft, node.token, inc(depth), t)); - else - // else: right child contains token - node.rchild(splitHelper(node.rchild, node.token, pright, inc(depth), t)); - return node; + if (Range.contains(pleft, inner.token(), t)) // left child contains token + inner.left(splitHelper(inner.left(), pleft, inner.token(), depth + 1, t)); + else // else: right child contains token + inner.right(splitHelper(inner.right(), inner.token(), pright, depth + 1, t)); + + return inner; } /** * Returns a lazy iterator of invalid TreeRanges that need to be filled * in order to make the given Range valid. */ - public TreeRangeIterator invalids() + TreeRangeIterator rangeIterator() { return new TreeRangeIterator(this); } - public EstimatedHistogram histogramOfRowSizePerLeaf() + EstimatedHistogram histogramOfRowSizePerLeaf() { HistogramBuilder histbuild = new HistogramBuilder(); for (TreeRange range : new TreeRangeIterator(this)) { - histbuild.add(range.hashable.sizeOfRange); + histbuild.add(range.node.sizeOfRange()); } return histbuild.buildWithStdevRangesAroundMean(); } - public EstimatedHistogram histogramOfRowCountPerLeaf() + EstimatedHistogram histogramOfRowCountPerLeaf() { HistogramBuilder histbuild = new HistogramBuilder(); for (TreeRange range : new TreeRangeIterator(this)) { - histbuild.add(range.hashable.rowsInRange); + histbuild.add(range.node.partitionsInRange()); } return histbuild.buildWithStdevRangesAroundMean(); } @@ -586,7 +486,7 @@ public long rowCount() long count = 0; for (TreeRange range : new TreeRangeIterator(this)) { - count += range.hashable.rowsInRange; + count += range.node.partitionsInRange(); } return count; } @@ -597,61 +497,23 @@ public String toString() StringBuilder buff = new StringBuilder(); buff.append("#"); + buff.append('>'); return buff.toString(); } - public static class TreeDifference extends TreeRange + @Override + public boolean equals(Object other) { - private static final long serialVersionUID = 6363654174549968183L; - - private long sizeOnLeft; - private long sizeOnRight; - private long rowsOnLeft; - private long rowsOnRight; - - void setSize(long sizeOnLeft, long sizeOnRight) - { - this.sizeOnLeft = sizeOnLeft; - this.sizeOnRight = sizeOnRight; - } - - void setRows(long rowsOnLeft, long rowsOnRight) - { - this.rowsOnLeft = rowsOnLeft; - this.rowsOnRight = rowsOnRight; - } - - public long sizeOnLeft() - { - return sizeOnLeft; - } - - public long sizeOnRight() - { - return sizeOnRight; - } - - public long rowsOnLeft() - { - return rowsOnLeft; - } - - public long rowsOnRight() - { - return rowsOnRight; - } - - public TreeDifference(Token left, Token right, byte depth) - { - super(null, left, right, depth, null); - } - - public long totalRows() - { - return rowsOnLeft + rowsOnRight; - } - + if (!(other instanceof MerkleTree)) + return false; + MerkleTree that = (MerkleTree) other; + + return this.root.equals(that.root) + && this.fullRange.equals(that.fullRange) + && this.partitioner == that.partitioner + && this.hashdepth == that.hashdepth + && this.maxsize == that.maxsize + && this.size == that.size; } /** @@ -664,28 +526,27 @@ public long totalRows() */ public static class TreeRange extends Range { - public static final long serialVersionUID = 1L; private final MerkleTree tree; - public final byte depth; - private final Hashable hashable; + public final int depth; + private final Node node; - TreeRange(MerkleTree tree, Token left, Token right, byte depth, Hashable hashable) + TreeRange(MerkleTree tree, Token left, Token right, int depth, Node node) { super(left, right); this.tree = tree; this.depth = depth; - this.hashable = hashable; + this.node = node; } - public void hash(byte[] hash) + TreeRange(Token left, Token right, int depth) { - assert tree != null : "Not intended for modification!"; - hashable.hash(hash); + this(null, left, right, depth, null); } - public byte[] hash() + public void hash(byte[] hash) { - return hashable.hash(); + assert tree != null : "Not intended for modification!"; + node.hash(hash); } /** @@ -693,33 +554,26 @@ public byte[] hash() */ public void addHash(RowHash entry) { - assert tree != null : "Not intended for modification!"; - assert hashable instanceof Leaf; - - hashable.addHash(entry.hash, entry.size); + addHash(entry.hash, entry.size); } - public void ensureHashInitialised() + void addHash(byte[] hash, long partitionSize) { assert tree != null : "Not intended for modification!"; - assert hashable instanceof Leaf; - if (hashable.hash == null) - hashable.hash = EMPTY_HASH; + assert node instanceof OnHeapLeaf; + ((OnHeapLeaf) node).addHash(hash, partitionSize); } public void addAll(Iterator entries) { - while (entries.hasNext()) - addHash(entries.next()); + while (entries.hasNext()) addHash(entries.next()); } @Override public String toString() { - StringBuilder buff = new StringBuilder("#").toString(); + return "#"; } + } + + public void serialize(DataOutputPlus out, int version) throws IOException + { + out.writeByte(hashdepth); + out.writeLong(maxsize); + out.writeLong(size); + out.writeUTF(partitioner.getClass().getCanonicalName()); + Token.serializer.serialize(fullRange.left, out, version); + Token.serializer.serialize(fullRange.right, out, version); + root.serialize(out, version); + } + + public long serializedSize(int version) + { + long size = 1 // mt.hashdepth + + sizeof(maxsize) + + sizeof(this.size) + + sizeof(partitioner.getClass().getCanonicalName()); + size += Token.serializer.serializedSize(fullRange.left, version); + size += Token.serializer.serializedSize(fullRange.right, version); + size += root.serializedSize(version); + return size; + } + + public static MerkleTree deserialize(DataInputPlus in, int version) throws IOException + { + return deserialize(in, DatabaseDescriptor.useOffheapMerkleTrees(), version); + } + + public static MerkleTree deserialize(DataInputPlus in, boolean offHeapRequested, int version) throws IOException + { + int hashDepth = in.readByte(); + long maxSize = in.readLong(); + int innerNodeCount = Ints.checkedCast(in.readLong()); - public Hashable rchild() + IPartitioner partitioner; + try { - return rchild; + partitioner = FBUtilities.newPartitioner(in.readUTF()); } - - public void lchild(Hashable child) + catch (ConfigurationException e) { - lchild = child; + throw new IOException(e); } - public void rchild(Hashable child) + Token left = Token.serializer.deserialize(in, partitioner, version); + Token right = Token.serializer.deserialize(in, partitioner, version); + Range fullRange = new Range<>(left, right); + Node root = deserializeTree(in, partitioner, innerNodeCount, offHeapRequested, version); + return new MerkleTree(root, partitioner, fullRange, hashDepth, maxSize, innerNodeCount); + } + + private static boolean shouldUseOffHeapTrees(IPartitioner partitioner, boolean offHeapRequested) + { + boolean offHeapSupported = partitioner instanceof Murmur3Partitioner || partitioner instanceof RandomPartitioner; + + if (offHeapRequested && !offHeapSupported && !warnedOnce) { - rchild = child; + logger.warn("Configuration requests off-heap merkle trees, but partitioner does not support it. Ignoring."); + warnedOnce = true; } - Hashable calc() + return offHeapRequested && offHeapSupported; + } + private static boolean warnedOnce; + + private static ByteBuffer allocate(int innerNodeCount, IPartitioner partitioner) + { + int size = offHeapBufferSize(innerNodeCount, partitioner); + logger.debug("Allocating direct buffer of size {} for an off-heap merkle tree", size); + ByteBuffer buffer = ByteBuffer.allocateDirect(size); + if (Ref.DEBUG_ENABLED) + MemoryUtil.setAttachment(buffer, new Ref<>(null, null)); + return buffer; + } + + private static Node deserializeTree(DataInputPlus in, IPartitioner partitioner, int innerNodeCount, boolean offHeapRequested, int version) throws IOException + { + return shouldUseOffHeapTrees(partitioner, offHeapRequested) + ? deserializeOffHeap(in, partitioner, innerNodeCount, version) + : OnHeapNode.deserialize(in, partitioner, version); + } + + /* + * Coordinating multiple trees from multiple replicas can get expensive. + * On the deserialization path, we know in advance what the tree looks like, + * So we can pre-size an offheap buffer and deserialize into that. + */ + MerkleTree tryMoveOffHeap() throws IOException + { + return root instanceof OnHeapNode && shouldUseOffHeapTrees(partitioner, DatabaseDescriptor.useOffheapMerkleTrees()) + ? moveOffHeap() + : this; + } + + @VisibleForTesting + MerkleTree moveOffHeap() throws IOException + { + assert root instanceof OnHeapNode; + root.fillInnerHashes(); // ensure on-heap trees' inner node hashes have been computed + ByteBuffer buffer = allocate(Ints.checkedCast(size), partitioner); + int pointer = ((OnHeapNode) root).serializeOffHeap(buffer, partitioner); + OffHeapNode newRoot = fromPointer(pointer, buffer, partitioner); + return new MerkleTree(newRoot, partitioner, fullRange, hashdepth, maxsize, size); + } + + private static OffHeapNode deserializeOffHeap(DataInputPlus in, IPartitioner partitioner, int innerNodeCount, int version) throws IOException + { + ByteBuffer buffer = allocate(innerNodeCount, partitioner); + int pointer = OffHeapNode.deserialize(in, buffer, partitioner, version); + return fromPointer(pointer, buffer, partitioner); + } + + private static OffHeapNode fromPointer(int pointer, ByteBuffer buffer, IPartitioner partitioner) + { + return pointer >= 0 ? new OffHeapInner(buffer, pointer, partitioner) : new OffHeapLeaf(buffer, ~pointer); + } + + private static int offHeapBufferSize(int innerNodeCount, IPartitioner partitioner) + { + return innerNodeCount * OffHeapInner.maxOffHeapSize(partitioner) + (innerNodeCount + 1) * OffHeapLeaf.maxOffHeapSize(); + } + + interface Node + { + byte[] hash(); + + boolean hasEmptyHash(); + + void hash(byte[] hash); + + boolean hashesDiffer(Node other); + + default Node fillInnerHashes() { - if (hash == null) - { - // hash and size haven't been calculated; calc children then compute - Hashable lnode = lchild.calc(); - Hashable rnode = rchild.calc(); - // cache the computed value - hash(lnode.hash, rnode.hash); - sizeOfRange = lnode.sizeOfRange + rnode.sizeOfRange; - rowsInRange = lnode.rowsInRange + rnode.rowsInRange; - } return this; } - /** - * Recursive toString. - */ - public void toString(StringBuilder buff, int maxdepth) + default long sizeOfRange() { - buff.append("#<").append(getClass().getSimpleName()); - buff.append(" ").append(token); - buff.append(" hash=").append(Hashable.toString(hash())); - buff.append(" children=["); - if (maxdepth < 1) - { - buff.append("#"); - } - else - { - if (lchild == null) - buff.append("null"); - else - lchild.toString(buff, maxdepth-1); - buff.append(" "); - if (rchild == null) - buff.append("null"); - else - rchild.toString(buff, maxdepth-1); - } - buff.append("]>"); + return 0; } - @Override - public String toString() + default long partitionsInRange() { - StringBuilder buff = new StringBuilder(); - toString(buff, 1); - return buff.toString(); + return 0; } - private static class InnerSerializer implements IPartitionerDependentSerializer - { - public void serialize(Inner inner, DataOutputPlus out, int version) throws IOException - { - Token.serializer.serialize(inner.token, out, version); - Hashable.serializer.serialize(inner.lchild, out, version); - Hashable.serializer.serialize(inner.rchild, out, version); - } + void serialize(DataOutputPlus out, int version) throws IOException; + int serializedSize(int version); - public Inner deserialize(DataInput in, IPartitioner p, int version) throws IOException - { - Token token = Token.serializer.deserialize(in, p, version); - Hashable lchild = Hashable.serializer.deserialize(in, p, version); - Hashable rchild = Hashable.serializer.deserialize(in, p, version); - return new Inner(token, lchild, rchild); - } + void toString(StringBuilder buff, int maxdepth); - public long serializedSize(Inner inner, int version) - { - return Token.serializer.serializedSize(inner.token, version) - + Hashable.serializer.serializedSize(inner.lchild, version) - + Hashable.serializer.serializedSize(inner.rchild, version); - } + static String toString(byte[] hash) + { + return hash == null + ? "null" + : '[' + Hex.bytesToHex(hash) + ']'; } + + boolean equals(Node node); } - /** - * A leaf node in the MerkleTree. Because the MerkleTree represents a much - * larger perfect binary tree of depth hashdepth, a Leaf object contains - * the value that would be contained in the perfect tree at its position. - * - * When rows are added to the MerkleTree using TreeRange.validate(), the - * tree extending below the Leaf is generated in memory, but only the root - * is stored in the Leaf. - */ - static class Leaf extends Hashable + static abstract class OnHeapNode implements Node { - public static final long serialVersionUID = 1L; - static final byte IDENT = 1; - private static final LeafSerializer serializer = new LeafSerializer(); + long sizeOfRange; + long partitionsInRange; - /** - * Constructs a null hash. - */ - public Leaf() + protected byte[] hash; + + OnHeapNode(byte[] hash) { - super(null); + if (hash == null) + throw new IllegalArgumentException(); + + this.hash = hash; } - public Leaf(byte[] hash) + public byte[] hash() { - super(hash); + return hash; } - public void toString(StringBuilder buff, int maxdepth) + public boolean hasEmptyHash() { - buff.append(toString()); + //noinspection ArrayEquality + return hash == EMPTY_HASH; } - @Override - public String toString() + public void hash(byte[] hash) { - return "#"; + if (hash == null) + throw new IllegalArgumentException(); + + this.hash = hash; } - private static class LeafSerializer implements IPartitionerDependentSerializer + public boolean hashesDiffer(Node other) { - public void serialize(Leaf leaf, DataOutputPlus out, int version) throws IOException - { - if (leaf.hash == null) - { - out.writeByte(-1); - } - else - { - out.writeByte(leaf.hash.length); - out.write(leaf.hash); - } - } + return other instanceof OnHeapNode + ? hashesDiffer( (OnHeapNode) other) + : hashesDiffer((OffHeapNode) other); + } + + private boolean hashesDiffer(OnHeapNode other) + { + return !Arrays.equals(hash(), other.hash()); + } + + private boolean hashesDiffer(OffHeapNode other) + { + return compare(hash(), other.buffer(), other.hashBytesOffset(), HASH_SIZE) != 0; + } + + @Override + public long sizeOfRange() + { + return sizeOfRange; + } + + @Override + public long partitionsInRange() + { + return partitionsInRange; + } + + static OnHeapNode deserialize(DataInputPlus in, IPartitioner p, int version) throws IOException + { + byte ident = in.readByte(); - public Leaf deserialize(DataInput in, IPartitioner p, int version) throws IOException + switch (ident) { - int hashLen = in.readByte(); - byte[] hash = hashLen < 0 ? null : new byte[hashLen]; - if (hash != null) - in.readFully(hash); - return new Leaf(hash); + case Inner.IDENT: + return OnHeapInner.deserializeWithoutIdent(in, p, version); + case Leaf.IDENT: + return OnHeapLeaf.deserializeWithoutIdent(in); + default: + throw new IOException("Unexpected node type: " + ident); } + } + + abstract int serializeOffHeap(ByteBuffer buffer, IPartitioner p) throws IOException; + } + + static abstract class OffHeapNode implements Node + { + protected final ByteBuffer buffer; + protected final int offset; + + OffHeapNode(ByteBuffer buffer, int offset) + { + this.buffer = buffer; + this.offset = offset; + } + + ByteBuffer buffer() + { + return buffer; + } + + public byte[] hash() + { + final int position = buffer.position(); + buffer.position(hashBytesOffset()); + byte[] array = new byte[HASH_SIZE]; + buffer.get(array); + buffer.position(position); + return array; + } + + public boolean hasEmptyHash() + { + return compare(buffer(), hashBytesOffset(), HASH_SIZE, EMPTY_HASH) == 0; + } + + public void hash(byte[] hash) + { + throw new UnsupportedOperationException(); + } + + public boolean hashesDiffer(Node other) + { + return other instanceof OnHeapNode + ? hashesDiffer((OnHeapNode) other) + : hashesDiffer((OffHeapNode) other); + } - public long serializedSize(Leaf leaf, int version) + private boolean hashesDiffer(OnHeapNode other) + { + return compare(buffer(), hashBytesOffset(), HASH_SIZE, other.hash()) != 0; + } + + private boolean hashesDiffer(OffHeapNode other) + { + int thisOffset = hashBytesOffset(); + int otherOffset = other.hashBytesOffset(); + + for (int i = 0; i < HASH_SIZE; i += 8) + if (buffer().getLong(thisOffset + i) != other.buffer().getLong(otherOffset + i)) + return true; + + return false; + } + + void release() + { + Object attachment = MemoryUtil.getAttachment(buffer); + if (attachment instanceof Ref) + ((Ref) attachment).release(); + FileUtils.clean(buffer); + } + + abstract int hashBytesOffset(); + + static int deserialize(DataInputPlus in, ByteBuffer buffer, IPartitioner p, int version) throws IOException + { + byte ident = in.readByte(); + + switch (ident) { - long size = 1; - if (leaf.hash != null) - size += leaf.hash().length; - return size; + case Inner.IDENT: + return OffHeapInner.deserializeWithoutIdent(in, buffer, p, version); + case Leaf.IDENT: + return OffHeapLeaf.deserializeWithoutIdent(in, buffer); + default: + throw new IOException("Unexpected node type: " + ident); } } } /** - * Hash value representing a row, to be used to pass hashes to the MerkleTree. - * The byte[] hash value should contain a digest of the key and value of the row - * created using a very strong hash function. + * A leaf node in the MerkleTree. Because the MerkleTree represents a much + * larger perfect binary tree of depth hashdepth, a Leaf object contains + * the value that would be contained in the perfect tree at its position. + * + * When rows are added to the MerkleTree using TreeRange.validate(), the + * tree extending below the Leaf is generated in memory, but only the root + * is stored in the Leaf. */ - public static class RowHash + interface Leaf extends Node { - public final Token token; - public final byte[] hash; - public final long size; - public RowHash(Token token, byte[] hash, long size) + static final byte IDENT = 1; + + default void serialize(DataOutputPlus out, int version) throws IOException { - this.token = token; - this.hash = hash; - this.size = size; + byte[] hash = hash(); + assert hash.length == HASH_SIZE; + + out.writeByte(Leaf.IDENT); + + if (!hasEmptyHash()) + { + out.writeByte(HASH_SIZE); + out.write(hash); + } + else + { + out.writeByte(0); + } + } + + default int serializedSize(int version) + { + return 2 + (hasEmptyHash() ? 0 : HASH_SIZE); + } + + default void toString(StringBuilder buff, int maxdepth) + { + buff.append(toString()); + } + + default boolean equals(Node other) + { + return other instanceof Leaf && !hashesDiffer(other); + } + } + + static class OnHeapLeaf extends OnHeapNode implements Leaf + { + OnHeapLeaf() + { + super(EMPTY_HASH); + } + + OnHeapLeaf(byte[] hash) + { + super(hash); + } + + /** + * Mixes the given value into our hash. If our hash is null, + * our hash will become the given value. + */ + void addHash(byte[] partitionHash, long partitionSize) + { + if (hasEmptyHash()) + hash(partitionHash); + else + xorOntoLeft(hash, partitionHash); + + sizeOfRange += partitionSize; + partitionsInRange += 1; + } + + static OnHeapLeaf deserializeWithoutIdent(DataInputPlus in) throws IOException + { + int size = in.readByte(); + switch (size) + { + case HASH_SIZE: + byte[] hash = new byte[HASH_SIZE]; + in.readFully(hash); + return new OnHeapLeaf(hash); + case 0: + return new OnHeapLeaf(); + default: + throw new IllegalStateException(format("Hash of size %d encountered, expecting %d or %d", size, HASH_SIZE, 0)); + } + } + + int serializeOffHeap(ByteBuffer buffer, IPartitioner p) + { + if (buffer.remaining() < OffHeapLeaf.maxOffHeapSize()) + throw new IllegalStateException("Insufficient remaining bytes to deserialize a Leaf node off-heap"); + + if (hash.length != HASH_SIZE) + throw new IllegalArgumentException("Hash of unexpected size when serializing a Leaf off-heap: " + hash.length); + + final int position = buffer.position(); + buffer.put(hash); + return ~position; + } + + @Override + public String toString() + { + return "#'; + } + } + + static class OffHeapLeaf extends OffHeapNode implements Leaf + { + static final int HASH_BYTES_OFFSET = 0; + + OffHeapLeaf(ByteBuffer buffer, int offset) + { + super(buffer, offset); + } + + public int hashBytesOffset() + { + return offset + HASH_BYTES_OFFSET; + } + + static int deserializeWithoutIdent(DataInput in, ByteBuffer buffer) throws IOException + { + if (buffer.remaining() < maxOffHeapSize()) + throw new IllegalStateException("Insufficient remaining bytes to deserialize a Leaf node off-heap"); + + final int position = buffer.position(); + + int hashLength = in.readByte(); + if (hashLength > 0) + { + if (hashLength != HASH_SIZE) + throw new IllegalStateException("Hash of unexpected size when deserializing an off-heap Leaf node: " + hashLength); + + byte[] hashBytes = getTempArray(HASH_SIZE); + in.readFully(hashBytes, 0, HASH_SIZE); + buffer.put(hashBytes, 0, HASH_SIZE); + } + else + { + buffer.put(EMPTY_HASH, 0, HASH_SIZE); + } + + return ~position; + } + + static int maxOffHeapSize() + { + return HASH_SIZE; } @Override public String toString() { - return "#"; + return "#'; } } /** - * Abstract class containing hashing logic, and containing a single hash field. + * An inner node in the MerkleTree. Inners can contain cached hash values, which + * are the binary hash of their two children. */ - static abstract class Hashable implements Serializable + interface Inner extends Node { - private static final long serialVersionUID = 1L; - private static final IPartitionerDependentSerializer serializer = new HashableSerializer(); + static final byte IDENT = 2; - protected byte[] hash; - protected long sizeOfRange; - protected long rowsInRange; + public Token token(); + + public Node left(); + public Node right(); - protected Hashable(byte[] hash) + default void serialize(DataOutputPlus out, int version) throws IOException { - this.hash = hash; + out.writeByte(Inner.IDENT); + Token.serializer.serialize(token(), out, version); + left().serialize(out, version); + right().serialize(out, version); } - public byte[] hash() + default int serializedSize(int version) { - return hash; + return 1 + + (int) Token.serializer.serializedSize(token(), version) + + left().serializedSize(version) + + right().serializedSize(version); } - public long sizeOfRange() + default void toString(StringBuilder buff, int maxdepth) { - return sizeOfRange; + buff.append("#<").append(getClass().getSimpleName()) + .append(' ').append(token()) + .append(" hash=").append(Node.toString(hash())) + .append(" children=["); + + if (maxdepth < 1) + { + buff.append('#'); + } + else + { + Node left = left(); + if (left == null) + buff.append("null"); + else + left.toString(buff, maxdepth - 1); + + buff.append(' '); + + Node right = right(); + if (right == null) + buff.append("null"); + else + right.toString(buff, maxdepth - 1); + } + + buff.append("]>"); } - public long rowsInRange() + default boolean equals(Node other) { - return rowsInRange; + if (!(other instanceof Inner)) + return false; + Inner that = (Inner) other; + return !hashesDiffer(other) && this.left().equals(that.left()) && this.right().equals(that.right()); } - void hash(byte[] hash) + default void unsafeInvalidate() { - this.hash = hash; } + } - Hashable calc() + static class OnHeapInner extends OnHeapNode implements Inner + { + private final Token token; + + private OnHeapNode left; + private OnHeapNode right; + + private boolean computed; + + OnHeapInner(Token token, OnHeapNode left, OnHeapNode right) { - return this; + super(EMPTY_HASH); + + this.token = token; + this.left = left; + this.right = right; } - /** - * Sets the value of this hash to binaryHash of its children. - * @param lefthash Hash of left child. - * @param righthash Hash of right child. - */ - void hash(byte[] lefthash, byte[] righthash) + public Token token() { - hash = binaryHash(lefthash, righthash); + return token; } - /** - * Mixes the given value into our hash. If our hash is null, - * our hash will become the given value. - */ - void addHash(byte[] righthash, long sizeOfRow) + public OnHeapNode left() { - if (hash == null) - hash = righthash; - else - hash = binaryHash(hash, righthash); - this.sizeOfRange += sizeOfRow; - this.rowsInRange += 1; + return left; } - /** - * The primitive with which all hashing should be accomplished: hashes - * a left and right value together. - */ - static byte[] binaryHash(final byte[] left, final byte[] right) + public OnHeapNode right() { - return FBUtilities.xor(left, right); + return right; } - public abstract void toString(StringBuilder buff, int maxdepth); + void left(OnHeapNode child) + { + left = child; + } - public static String toString(byte[] hash) + void right(OnHeapNode child) { - if (hash == null) - return "null"; - return "[" + Hex.bytesToHex(hash) + "]"; + right = child; } - private static class HashableSerializer implements IPartitionerDependentSerializer + @Override + public Node fillInnerHashes() { - public void serialize(Hashable h, DataOutputPlus out, int version) throws IOException + if (!computed) // hash and size haven't been calculated; compute children then compute this { - if (h instanceof Inner) - { - out.writeByte(Inner.IDENT); - Inner.serializer.serialize((Inner)h, out, version); - } - else if (h instanceof Leaf) - { - out.writeByte(Leaf.IDENT); - Leaf.serializer.serialize((Leaf) h, out, version); - } - else - throw new IOException("Unexpected Hashable: " + h.getClass().getCanonicalName()); - } + left.fillInnerHashes(); + right.fillInnerHashes(); - public Hashable deserialize(DataInput in, IPartitioner p, int version) throws IOException - { - byte ident = in.readByte(); - if (Inner.IDENT == ident) - return Inner.serializer.deserialize(in, p, version); - else if (Leaf.IDENT == ident) - return Leaf.serializer.deserialize(in, p, version); - else - throw new IOException("Unexpected Hashable: " + ident); + if (!left.hasEmptyHash() && !right.hasEmptyHash()) + hash = xor(left.hash(), right.hash()); + else if (left.hasEmptyHash()) + hash = right.hash(); + else if (right.hasEmptyHash()) + hash = left.hash(); + + sizeOfRange = left.sizeOfRange() + right.sizeOfRange(); + partitionsInRange = left.partitionsInRange() + right.partitionsInRange(); + + computed = true; } - public long serializedSize(Hashable h, int version) + return this; + } + + static OnHeapInner deserializeWithoutIdent(DataInputPlus in, IPartitioner p, int version) throws IOException + { + Token token = Token.serializer.deserialize(in, p, version); + OnHeapNode left = OnHeapNode.deserialize(in, p, version); + OnHeapNode right = OnHeapNode.deserialize(in, p, version); + return new OnHeapInner(token, left, right); + } + + int serializeOffHeap(ByteBuffer buffer, IPartitioner partitioner) throws IOException + { + if (buffer.remaining() < OffHeapInner.maxOffHeapSize(partitioner)) + throw new IllegalStateException("Insufficient remaining bytes to deserialize Inner node off-heap"); + + final int offset = buffer.position(); + + int tokenSize = partitioner.getTokenFactory().byteSize(token); + buffer.putShort(offset + OffHeapInner.TOKEN_LENGTH_OFFSET, Shorts.checkedCast(tokenSize)); + buffer.position(offset + OffHeapInner.TOKEN_BYTES_OFFSET); + partitioner.getTokenFactory().serialize(token, buffer); + + int leftPointer = left.serializeOffHeap(buffer, partitioner); + int rightPointer = right.serializeOffHeap(buffer, partitioner); + + buffer.putInt(offset + OffHeapInner.LEFT_CHILD_POINTER_OFFSET, leftPointer); + buffer.putInt(offset + OffHeapInner.RIGHT_CHILD_POINTER_OFFSET, rightPointer); + + int leftHashOffset = OffHeapInner.hashBytesOffset(leftPointer); + int rightHashOffset = OffHeapInner.hashBytesOffset(rightPointer); + + for (int i = 0; i < HASH_SIZE; i += 8) { - if (h instanceof Inner) - return 1 + Inner.serializer.serializedSize((Inner) h, version); - else if (h instanceof Leaf) - return 1 + Leaf.serializer.serializedSize((Leaf) h, version); - throw new AssertionError(h.getClass()); + buffer.putLong(offset + OffHeapInner.HASH_BYTES_OFFSET + i, + buffer.getLong(leftHashOffset + i) ^ buffer.getLong(rightHashOffset + i)); } + + return offset; + } + + @Override + public String toString() + { + StringBuilder buff = new StringBuilder(); + toString(buff, 1); + return buff.toString(); + } + + @Override + public void unsafeInvalidate() + { + computed = false; } } - /** - * Exceptions that stop recursion early when we are sure that no answer - * can be found. - */ - static abstract class StopRecursion extends Exception + static class OffHeapInner extends OffHeapNode implements Inner { - static class BadRange extends StopRecursion + /** + * All we want to keep here is just a pointer to the start of the Inner leaf in the + * direct buffer. From there, we'll be able to deserialize the following, in this order: + * + * 1. pointer to left child (int) + * 2. pointer to right child (int) + * 3. hash bytes (space allocated as HASH_MAX_SIZE) + * 4. token length (short) + * 5. token bytes (variable length) + */ + static final int LEFT_CHILD_POINTER_OFFSET = 0; + static final int RIGHT_CHILD_POINTER_OFFSET = 4; + static final int HASH_BYTES_OFFSET = 8; + static final int TOKEN_LENGTH_OFFSET = 8 + HASH_SIZE; + static final int TOKEN_BYTES_OFFSET = TOKEN_LENGTH_OFFSET + 2; + + private final IPartitioner partitioner; + + OffHeapInner(ByteBuffer buffer, int offset, IPartitioner partitioner) + { + super(buffer, offset); + this.partitioner = partitioner; + } + + public Token token() + { + int length = buffer.getShort(offset + TOKEN_LENGTH_OFFSET); + return partitioner.getTokenFactory().fromByteBuffer(buffer, offset + TOKEN_BYTES_OFFSET, length); + } + + public Node left() + { + return child(LEFT_CHILD_POINTER_OFFSET); + } + + public Node right() { - public BadRange(){ super(); } + return child(RIGHT_CHILD_POINTER_OFFSET); } - static class InvalidHash extends StopRecursion + private Node child(int childOffset) { - public InvalidHash(){ super(); } + int pointer = buffer.getInt(offset + childOffset); + return pointer >= 0 ? new OffHeapInner(buffer, pointer, partitioner) : new OffHeapLeaf(buffer, ~pointer); } - static class TooDeep extends StopRecursion + public int hashBytesOffset() { - public TooDeep(){ super(); } + return offset + HASH_BYTES_OFFSET; } + + static int deserializeWithoutIdent(DataInputPlus in, ByteBuffer buffer, IPartitioner partitioner, int version) throws IOException + { + if (buffer.remaining() < maxOffHeapSize(partitioner)) + throw new IllegalStateException("Insufficient remaining bytes to deserialize Inner node off-heap"); + + final int offset = buffer.position(); + + int tokenSize = Token.serializer.deserializeSize(in); + byte[] tokenBytes = getTempArray(tokenSize); + in.readFully(tokenBytes, 0, tokenSize); + + buffer.putShort(offset + OffHeapInner.TOKEN_LENGTH_OFFSET, Shorts.checkedCast(tokenSize)); + buffer.position(offset + OffHeapInner.TOKEN_BYTES_OFFSET); + buffer.put(tokenBytes, 0, tokenSize); + + int leftPointer = OffHeapNode.deserialize(in, buffer, partitioner, version); + int rightPointer = OffHeapNode.deserialize(in, buffer, partitioner, version); + + buffer.putInt(offset + OffHeapInner.LEFT_CHILD_POINTER_OFFSET, leftPointer); + buffer.putInt(offset + OffHeapInner.RIGHT_CHILD_POINTER_OFFSET, rightPointer); + + int leftHashOffset = hashBytesOffset(leftPointer); + int rightHashOffset = hashBytesOffset(rightPointer); + + for (int i = 0; i < HASH_SIZE; i += 8) + { + buffer.putLong(offset + OffHeapInner.HASH_BYTES_OFFSET + i, + buffer.getLong(leftHashOffset + i) ^ buffer.getLong(rightHashOffset + i)); + } + + return offset; + } + + static int maxOffHeapSize(IPartitioner partitioner) + { + return 4 // left pointer + + 4 // right pointer + + HASH_SIZE + + 2 + partitioner.getMaxTokenSize(); + } + + static int hashBytesOffset(int pointer) + { + return pointer >= 0 ? pointer + OffHeapInner.HASH_BYTES_OFFSET : ~pointer + OffHeapLeaf.HASH_BYTES_OFFSET; + } + + @Override + public String toString() + { + StringBuilder buff = new StringBuilder(); + toString(buff, 1); + return buff.toString(); + } + } + + /** + * @return The bitwise XOR of the inputs. + */ + static byte[] xor(byte[] left, byte[] right) + { + assert left.length == right.length; + + byte[] out = Arrays.copyOf(right, right.length); + for (int i = 0; i < left.length; i++) + out[i] = (byte)((left[i] & 0xFF) ^ (right[i] & 0xFF)); + return out; + } + + /** + * Bitwise XOR of the inputs, in place on the left array. + */ + private static void xorOntoLeft(byte[] left, byte[] right) + { + assert left.length == right.length; + + for (int i = 0; i < left.length; i++) + left[i] = (byte) ((left[i] & 0xFF) ^ (right[i] & 0xFF)); } /** @@ -1183,10 +1512,10 @@ public static int estimatedMaxDepthForBytes(IPartitioner partitioner, long numBy { byte[] hashLeft = new byte[bytesPerHash]; byte[] hashRigth = new byte[bytesPerHash]; - Leaf left = new Leaf(hashLeft); - Leaf right = new Leaf(hashRigth); - Inner inner = new Inner(partitioner.getMinimumToken(), left, right); - inner.calc(); + OnHeapLeaf left = new OnHeapLeaf(hashLeft); + OnHeapLeaf right = new OnHeapLeaf(hashRigth); + Inner inner = new OnHeapInner(partitioner.getMinimumToken(), left, right); + inner.fillInnerHashes(); // Some partioners have variable token sizes, try to estimate as close as we can by using the same // heap estimate as the memtables use. @@ -1201,4 +1530,124 @@ public static int estimatedMaxDepthForBytes(IPartitioner partitioner, long numBy long adjustedBytes = Math.max(1, (numBytes + sizeOfInner) / (sizeOfLeaf + sizeOfInner)); return Math.max(1, (int) Math.floor(Math.log(adjustedBytes) / Math.log(2))); } + + /* + * Test-only methods. + */ + + /** + * Invalidates the ranges containing the given token. + * Useful for testing. + */ + @VisibleForTesting + void unsafeInvalidate(Token t) + { + unsafeInvalidateHelper(root, fullRange.left, t); + } + + private void unsafeInvalidateHelper(Node node, Token pleft, Token t) + { + node.hash(EMPTY_HASH); + + if (node instanceof Leaf) + return; + + assert node instanceof Inner; + Inner inner = (Inner) node; + inner.unsafeInvalidate(); + + if (Range.contains(pleft, inner.token(), t)) + unsafeInvalidateHelper(inner.left(), pleft, t); // left child contains token + else + unsafeInvalidateHelper(inner.right(), inner.token(), t); // right child contains token + } + + /** + * Hash the given range in the tree. The range must have been generated + * with recursive applications of partitioner.midpoint(). + * + * NB: Currently does not support wrapping ranges that do not end with + * partitioner.getMinimumToken(). + * + * @return {@link #EMPTY_HASH} if any subrange of the range is invalid, or if the exact + * range cannot be calculated using this tree. + */ + @VisibleForTesting + byte[] hash(Range range) + { + return find(range).hash(); + } + + interface Consumer + { + void accept(Node node) throws E; + } + + @VisibleForTesting + boolean ifHashesRange(Range range, Consumer consumer) throws E + { + try + { + Node node = findHelper(root, new Range<>(fullRange.left, fullRange.right), range); + boolean hasHash = !node.hasEmptyHash(); + if (hasHash) + consumer.accept(node); + return hasHash; + } + catch (StopRecursion e) + { + return false; + } + } + + @VisibleForTesting + boolean hashesRange(Range range) + { + return ifHashesRange(range, n -> {}); + } + + /** + * For testing purposes. + * Gets the smallest range containing the token. + */ + @VisibleForTesting + public TreeRange get(Token t) + { + return getHelper(root, fullRange.left, fullRange.right, t); + } + + private TreeRange getHelper(Node node, Token pleft, Token pright, Token t) + { + int depth = 0; + + while (true) + { + if (node instanceof Leaf) + { + // we've reached a hash: wrap it up and deliver it + return new TreeRange(this, pleft, pright, depth, node); + } + + assert node instanceof Inner; + Inner inner = (Inner) node; + + if (Range.contains(pleft, inner.token(), t)) // left child contains token + { + pright = inner.token(); + node = inner.left(); + } + else // right child contains token + { + pleft = inner.token(); + node = inner.right(); + } + + depth++; + } + } + + private void fillInnerHashes() + { + root.fillInnerHashes(); + } } diff --git a/src/java/org/apache/cassandra/utils/MerkleTrees.java b/src/java/org/apache/cassandra/utils/MerkleTrees.java index d2a80583af9a..0043fe07ad5a 100644 --- a/src/java/org/apache/cassandra/utils/MerkleTrees.java +++ b/src/java/org/apache/cassandra/utils/MerkleTrees.java @@ -44,9 +44,9 @@ public class MerkleTrees implements Iterable, MerkleTree> { public static final MerkleTreesSerializer serializer = new MerkleTreesSerializer(); - private Map, MerkleTree> merkleTrees = new TreeMap<>(new TokenRangeComparator()); + private final Map, MerkleTree> merkleTrees = new TreeMap<>(new TokenRangeComparator()); - private IPartitioner partitioner; + private final IPartitioner partitioner; /** * Creates empty MerkleTrees object. @@ -142,6 +142,15 @@ public void init() } } + /** + * Dereference all merkle trees and release direct memory for all off-heap trees. + */ + public void release() + { + merkleTrees.values().forEach(MerkleTree::release); + merkleTrees.clear(); + } + /** * Init a selected MerkleTree with an even tree distribution. * @@ -171,7 +180,7 @@ public boolean split(Token t) @VisibleForTesting public void invalidate(Token t) { - getMerkleTree(t).invalidate(t); + getMerkleTree(t).unsafeInvalidate(t); } /** @@ -247,11 +256,11 @@ private boolean validateNonOverlapping(MerkleTree tree) } /** - * Get an iterator for all the invalids generated by the MerkleTrees. + * Get an iterator for all the iterator generated by the MerkleTrees. * * @return */ - public TreeRangeIterator invalids() + public TreeRangeIterator rangeIterator() { return new TreeRangeIterator(); } @@ -285,30 +294,20 @@ public void logRowSizePerLeaf(Logger logger) @VisibleForTesting public byte[] hash(Range range) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - boolean hashed = false; - - try + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - for (Range rt : merkleTrees.keySet()) - { - if (rt.intersects(range)) - { - byte[] bytes = merkleTrees.get(rt).hash(range); - if (bytes != null) - { - baos.write(bytes); - hashed = true; - } - } - } + boolean hashed = false; + + for (Map.Entry, MerkleTree> entry : merkleTrees.entrySet()) + if (entry.getKey().intersects(range)) + hashed |= entry.getValue().ifHashesRange(range, n -> baos.write(n.hash())); + + return hashed ? baos.toByteArray() : null; } catch (IOException e) { throw new RuntimeException("Unable to append merkle tree hash to result"); } - - return hashed ? baos.toByteArray() : null; } /** @@ -354,7 +353,7 @@ private MerkleTree.TreeRange nextIterator() { if (it.hasNext()) { - current = it.next().invalids(); + current = it.next().rangeIterator(); return current.next(); } @@ -368,6 +367,17 @@ public Iterator iterator() } } + /** + * @return a new {@link MerkleTrees} instance with all trees moved off heap. + */ + public MerkleTrees tryMoveOffHeap() throws IOException + { + Map, MerkleTree> movedTrees = new TreeMap<>(new TokenRangeComparator()); + for (Map.Entry, MerkleTree> entry : merkleTrees.entrySet()) + movedTrees.put(entry.getKey(), entry.getValue().tryMoveOffHeap()); + return new MerkleTrees(partitioner, movedTrees.values()); + } + /** * Get the differences between the two sets of MerkleTrees. * @@ -379,9 +389,7 @@ public static List> difference(MerkleTrees ltree, MerkleTrees rtree { List> differences = new ArrayList<>(); for (MerkleTree tree : ltree.merkleTrees.values()) - { differences.addAll(MerkleTree.difference(tree, rtree.getMerkleTree(tree.fullRange))); - } return differences; } @@ -392,7 +400,7 @@ public void serialize(MerkleTrees trees, DataOutputPlus out, int version) throws out.writeInt(trees.merkleTrees.size()); for (MerkleTree tree : trees.merkleTrees.values()) { - MerkleTree.serializer.serialize(tree, out, version); + tree.serialize(out, version); } } @@ -405,7 +413,7 @@ public MerkleTrees deserialize(DataInputPlus in, int version) throws IOException { for (int i = 0; i < nTrees; i++) { - MerkleTree tree = MerkleTree.serializer.deserialize(in, version); + MerkleTree tree = MerkleTree.deserialize(in, version); trees.add(tree); if (partitioner == null) @@ -425,7 +433,7 @@ public long serializedSize(MerkleTrees trees, int version) long size = TypeSizes.sizeof(trees.merkleTrees.size()); for (MerkleTree tree : trees.merkleTrees.values()) { - size += MerkleTree.serializer.serializedSize(tree, version); + size += tree.serializedSize(version); } return size; } diff --git a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java index e787595bab6d..443d59ecaca2 100644 --- a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java +++ b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java @@ -238,10 +238,6 @@ private MerkleTrees createInitialTree(RepairJobDesc desc, IPartitioner partition MerkleTrees tree = new MerkleTrees(partitioner); tree.addMerkleTrees((int) Math.pow(2, 15), desc.ranges); tree.init(); - for (MerkleTree.TreeRange r : tree.invalids()) - { - r.ensureHashInitialised(); - } return tree; } diff --git a/test/unit/org/apache/cassandra/repair/RepairJobTest.java b/test/unit/org/apache/cassandra/repair/RepairJobTest.java index 78fa588b19db..b84adaa06617 100644 --- a/test/unit/org/apache/cassandra/repair/RepairJobTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairJobTest.java @@ -774,10 +774,6 @@ private MerkleTrees createInitialTree(boolean invalidate) MerkleTrees tree = new MerkleTrees(MURMUR3_PARTITIONER); tree.addMerkleTrees((int) Math.pow(2, 15), fullRange); tree.init(); - for (MerkleTree.TreeRange r : tree.invalids()) - { - r.ensureHashInitialised(); - } if (invalidate) { diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java index aec2612b712a..9e848a95e392 100644 --- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java +++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java @@ -203,12 +203,14 @@ public void simpleValidationTest(int n) throws Exception cfs.getTableName(), Collections.singletonList(new Range<>(sstable.first.getToken(), sstable.last.getToken()))); - ActiveRepairService.instance.registerParentRepairSession(repairSessionId, FBUtilities.getBroadcastAddressAndPort(), + InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.2"); + + ActiveRepairService.instance.registerParentRepairSession(repairSessionId, host, Collections.singletonList(cfs), desc.ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, false, PreviewKind.NONE); final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); - Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), 0, true, false, PreviewKind.NONE); + Validator validator = new Validator(desc, host, 0, true, false, PreviewKind.NONE); ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); @@ -260,12 +262,14 @@ public void testSizeLimiting() throws Exception cfs.getTableName(), Collections.singletonList(new Range<>(sstable.first.getToken(), sstable.last.getToken()))); - ActiveRepairService.instance.registerParentRepairSession(repairSessionId, FBUtilities.getBroadcastAddressAndPort(), + InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.2"); + + ActiveRepairService.instance.registerParentRepairSession(repairSessionId, host, Collections.singletonList(cfs), desc.ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, false, PreviewKind.NONE); final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); - Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), 0, true, false, PreviewKind.NONE); + Validator validator = new Validator(desc, host, 0, true, false, PreviewKind.NONE); ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); @@ -320,12 +324,14 @@ public void testRangeSplittingTreeSizeLimit() throws Exception final RepairJobDesc desc = new RepairJobDesc(repairSessionId, UUIDGen.getTimeUUID(), cfs.keyspace.getName(), cfs.getTableName(), ranges); - ActiveRepairService.instance.registerParentRepairSession(repairSessionId, FBUtilities.getBroadcastAddressAndPort(), + InetAddressAndPort host = InetAddressAndPort.getByName("127.0.0.2"); + + ActiveRepairService.instance.registerParentRepairSession(repairSessionId, host, Collections.singletonList(cfs), desc.ranges, false, ActiveRepairService.UNREPAIRED_SSTABLE, false, PreviewKind.NONE); final CompletableFuture outgoingMessageSink = registerOutgoingMessageSink(); - Validator validator = new Validator(desc, FBUtilities.getBroadcastAddressAndPort(), 0, true, false, PreviewKind.NONE); + Validator validator = new Validator(desc, host, 0, true, false, PreviewKind.NONE); ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); diff --git a/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java b/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java index 96930108d634..88810180eb96 100644 --- a/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java +++ b/test/unit/org/apache/cassandra/repair/asymmetric/DifferenceHolderTest.java @@ -31,6 +31,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.TreeResponse; +import org.apache.cassandra.utils.HashingUtils; import org.apache.cassandra.utils.MerkleTree; import org.apache.cassandra.utils.MerkleTrees; import org.apache.cassandra.utils.MerkleTreesTest; @@ -40,6 +41,11 @@ public class DifferenceHolderTest { + private static byte[] digest(String string) + { + return HashingUtils.newMessageDigest("SHA-256").digest(string.getBytes()); + } + @Test public void testFromEmptyMerkleTrees() throws UnknownHostException { @@ -74,9 +80,9 @@ public void testFromMismatchedMerkleTrees() throws UnknownHostException mt1.init(); mt2.init(); // add dummy hashes to both trees - for (MerkleTree.TreeRange range : mt1.invalids()) + for (MerkleTree.TreeRange range : mt1.rangeIterator()) range.addAll(new MerkleTreesTest.HIterator(range.right)); - for (MerkleTree.TreeRange range : mt2.invalids()) + for (MerkleTree.TreeRange range : mt2.rangeIterator()) range.addAll(new MerkleTreesTest.HIterator(range.right)); MerkleTree.TreeRange leftmost = null; @@ -85,14 +91,14 @@ public void testFromMismatchedMerkleTrees() throws UnknownHostException mt1.maxsize(fullRange, maxsize + 2); // give some room for splitting // split the leftmost - Iterator ranges = mt1.invalids(); + Iterator ranges = mt1.rangeIterator(); leftmost = ranges.next(); mt1.split(leftmost.right); // set the hashes for the leaf of the created split middle = mt1.get(leftmost.right); - middle.hash("arbitrary!".getBytes()); - mt1.get(partitioner.midpoint(leftmost.left, leftmost.right)).hash("even more arbitrary!".getBytes()); + middle.hash(digest("arbitrary!")); + mt1.get(partitioner.midpoint(leftmost.left, leftmost.right)).hash(digest("even more arbitrary!")); TreeResponse tr1 = new TreeResponse(a1, mt1); TreeResponse tr2 = new TreeResponse(a2, mt2); diff --git a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java index c213271c0f88..36ae4a0479fb 100644 --- a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java +++ b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java @@ -1,29 +1,27 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one -* or more contributor license agreements. See the NOTICE file -* distributed with this work for additional information -* regarding copyten ownership. The ASF licenses this file -* to you under the Apache License, Version 2.0 (the -* "License"); you may not use this file except in compliance -* with the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, -* software distributed under the License is distributed on an -* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -* KIND, either express or implied. See the License for the -* specific language governing permissions and limitations -* under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.cassandra.utils; +import java.io.IOException; import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.*; -import com.google.common.collect.Lists; - import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -36,20 +34,24 @@ import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.utils.MerkleTree.Hashable; import org.apache.cassandra.utils.MerkleTree.RowHash; import org.apache.cassandra.utils.MerkleTree.TreeRange; import org.apache.cassandra.utils.MerkleTree.TreeRangeIterator; +import static com.google.common.collect.Lists.newArrayList; import static org.apache.cassandra.utils.MerkleTree.RECOMMENDED_DEPTH; import static org.junit.Assert.*; public class MerkleTreeTest { - public static byte[] DUMMY = "blah".getBytes(); + private static final byte[] DUMMY = digest("dummy"); + + private static byte[] digest(String string) + { + return HashingUtils.newMessageDigest("SHA-256").digest(string.getBytes()); + } /** * If a test assumes that the tree is 8 units wide, then it should set this value @@ -68,6 +70,9 @@ private Range fullRange() @Before public void setup() { + DatabaseDescriptor.clientInitialization(); + DatabaseDescriptor.useOffheapMerkleTrees(false); + TOKEN_SCALE = new BigInteger("8"); partitioner = RandomPartitioner.instance; // TODO need to trickle TokenSerializer @@ -171,7 +176,7 @@ public void testInvalids() Iterator ranges; // (zero, zero] - ranges = mt.invalids(); + ranges = mt.rangeIterator(); assertEquals(new Range<>(tok(-1), tok(-1)), ranges.next()); assertFalse(ranges.hasNext()); @@ -181,7 +186,7 @@ public void testInvalids() mt.split(tok(6)); mt.split(tok(3)); mt.split(tok(5)); - ranges = mt.invalids(); + ranges = mt.rangeIterator(); assertEquals(new Range<>(tok(6), tok(-1)), ranges.next()); assertEquals(new Range<>(tok(-1), tok(2)), ranges.next()); assertEquals(new Range<>(tok(2), tok(3)), ranges.next()); @@ -200,7 +205,7 @@ public void testHashFull() Range range = new Range<>(tok(-1), tok(-1)); // (zero, zero] - assertNull(mt.hash(range)); + assertFalse(mt.hashesRange(range)); // validate the range mt.get(tok(-1)).hash(val); @@ -223,11 +228,12 @@ public void testHashPartial() // (zero,two] (two,four] (four, zero] mt.split(tok(4)); mt.split(tok(2)); - assertNull(mt.hash(left)); - assertNull(mt.hash(partial)); - assertNull(mt.hash(right)); - assertNull(mt.hash(linvalid)); - assertNull(mt.hash(rinvalid)); + + assertFalse(mt.hashesRange(left)); + assertFalse(mt.hashesRange(partial)); + assertFalse(mt.hashesRange(right)); + assertFalse(mt.hashesRange(linvalid)); + assertFalse(mt.hashesRange(rinvalid)); // validate the range mt.get(tok(2)).hash(val); @@ -237,8 +243,8 @@ public void testHashPartial() assertHashEquals(leftval, mt.hash(left)); assertHashEquals(partialval, mt.hash(partial)); assertHashEquals(val, mt.hash(right)); - assertNull(mt.hash(linvalid)); - assertNull(mt.hash(rinvalid)); + assertFalse(mt.hashesRange(linvalid)); + assertFalse(mt.hashesRange(rinvalid)); } @Test @@ -258,10 +264,6 @@ public void testHashInner() mt.split(tok(2)); mt.split(tok(6)); mt.split(tok(1)); - assertNull(mt.hash(full)); - assertNull(mt.hash(lchild)); - assertNull(mt.hash(rchild)); - assertNull(mt.hash(invalid)); // validate the range mt.get(tok(1)).hash(val); @@ -270,10 +272,14 @@ public void testHashInner() mt.get(tok(6)).hash(val); mt.get(tok(-1)).hash(val); + assertTrue(mt.hashesRange(full)); + assertTrue(mt.hashesRange(lchild)); + assertTrue(mt.hashesRange(rchild)); + assertFalse(mt.hashesRange(invalid)); + assertHashEquals(fullval, mt.hash(full)); assertHashEquals(lchildval, mt.hash(lchild)); assertHashEquals(rchildval, mt.hash(rchild)); - assertNull(mt.hash(invalid)); } @Test @@ -294,9 +300,6 @@ public void testHashDegenerate() mt.split(tok(4)); mt.split(tok(2)); mt.split(tok(1)); - assertNull(mt.hash(full)); - assertNull(mt.hash(childfull)); - assertNull(mt.hash(invalid)); // validate the range mt.get(tok(1)).hash(val); @@ -306,9 +309,12 @@ public void testHashDegenerate() mt.get(tok(16)).hash(val); mt.get(tok(-1)).hash(val); + assertTrue(mt.hashesRange(full)); + assertTrue(mt.hashesRange(childfull)); + assertFalse(mt.hashesRange(invalid)); + assertHashEquals(fullval, mt.hash(full)); assertHashEquals(childfullval, mt.hash(childfull)); - assertNull(mt.hash(invalid)); } @Test @@ -326,7 +332,7 @@ public void testHashRandom() } // validate the tree - TreeRangeIterator ranges = mt.invalids(); + TreeRangeIterator ranges = mt.rangeIterator(); for (TreeRange range : ranges) range.addHash(new RowHash(range.right, new byte[0], 0)); @@ -355,7 +361,7 @@ public void testValidateTree() mt.split(tok(6)); mt.split(tok(10)); - ranges = mt.invalids(); + ranges = mt.rangeIterator(); ranges.next().addAll(new HIterator(2, 4)); // (-1,4]: depth 2 ranges.next().addAll(new HIterator(6)); // (4,6] ranges.next().addAll(new HIterator(8)); // (6,8] @@ -372,7 +378,7 @@ public void testValidateTree() mt2.split(tok(9)); mt2.split(tok(11)); - ranges = mt2.invalids(); + ranges = mt2.rangeIterator(); ranges.next().addAll(new HIterator(2)); // (-1,2] ranges.next().addAll(new HIterator(4)); // (2,4] ranges.next().addAll(new HIterator(6, 8)); // (4,8]: depth 2 @@ -395,19 +401,33 @@ public void testSerialization() throws Exception // populate and validate the tree mt.maxsize(256); mt.init(); - for (TreeRange range : mt.invalids()) + for (TreeRange range : mt.rangeIterator()) range.addAll(new HIterator(range.right)); byte[] initialhash = mt.hash(full); DataOutputBuffer out = new DataOutputBuffer(); - MerkleTree.serializer.serialize(mt, out, MessagingService.current_version); + mt.serialize(out, MessagingService.current_version); byte[] serialized = out.toByteArray(); - DataInputPlus in = new DataInputBuffer(serialized); - MerkleTree restored = MerkleTree.serializer.deserialize(in, MessagingService.current_version); + MerkleTree restoredOnHeap = + MerkleTree.deserialize(new DataInputBuffer(serialized), false, MessagingService.current_version); + MerkleTree restoredOffHeap = + MerkleTree.deserialize(new DataInputBuffer(serialized), true, MessagingService.current_version); + MerkleTree movedOffHeap = mt.moveOffHeap(); + + assertHashEquals(initialhash, restoredOnHeap.hash(full)); + assertHashEquals(initialhash, restoredOffHeap.hash(full)); + assertHashEquals(initialhash, movedOffHeap.hash(full)); + + assertEquals(mt, restoredOnHeap); + assertEquals(mt, restoredOffHeap); + assertEquals(mt, movedOffHeap); - assertHashEquals(initialhash, restored.hash(full)); + assertEquals(restoredOnHeap, restoredOffHeap); + assertEquals(restoredOnHeap, movedOffHeap); + + assertEquals(restoredOffHeap, movedOffHeap); } @Test @@ -420,9 +440,9 @@ public void testDifference() mt2.init(); // add dummy hashes to both trees - for (TreeRange range : mt.invalids()) + for (TreeRange range : mt.rangeIterator()) range.addAll(new HIterator(range.right)); - for (TreeRange range : mt2.invalids()) + for (TreeRange range : mt2.rangeIterator()) range.addAll(new HIterator(range.right)); TreeRange leftmost = null; @@ -431,14 +451,14 @@ public void testDifference() mt.maxsize(maxsize + 2); // give some room for splitting // split the leftmost - Iterator ranges = mt.invalids(); + Iterator ranges = mt.rangeIterator(); leftmost = ranges.next(); mt.split(leftmost.right); // set the hashes for the leaf of the created split middle = mt.get(leftmost.right); - middle.hash("arbitrary!".getBytes()); - mt.get(partitioner.midpoint(leftmost.left, leftmost.right)).hash("even more arbitrary!".getBytes()); + middle.hash(digest("arbitrary!")); + mt.get(partitioner.midpoint(leftmost.left, leftmost.right)).hash(digest("even more arbitrary!")); // trees should disagree for (leftmost.left, middle.right] List diffs = MerkleTree.difference(mt, mt2); @@ -461,22 +481,23 @@ public void differenceSmallRange() MerkleTree rtree = new MerkleTree(partitioner, range, RECOMMENDED_DEPTH, 16); rtree.init(); - byte[] h1 = "asdf".getBytes(); - byte[] h2 = "hjkl".getBytes(); + byte[] h1 = digest("asdf"); + byte[] h2 = digest("hjkl"); // add dummy hashes to both trees - for (TreeRange tree : ltree.invalids()) + for (TreeRange tree : ltree.rangeIterator()) { tree.addHash(new RowHash(range.right, h1, h1.length)); } - for (TreeRange tree : rtree.invalids()) + for (TreeRange tree : rtree.rangeIterator()) { tree.addHash(new RowHash(range.right, h2, h2.length)); } List diffs = MerkleTree.difference(ltree, rtree); - assertEquals(Lists.newArrayList(range), diffs); - assertEquals(MerkleTree.FULLY_INCONSISTENT, MerkleTree.differenceHelper(ltree, rtree, new ArrayList<>(), new MerkleTree.TreeDifference(ltree.fullRange.left, ltree.fullRange.right, (byte) 0))); + assertEquals(newArrayList(range), diffs); + assertEquals(MerkleTree.Difference.FULLY_INCONSISTENT, + MerkleTree.differenceHelper(ltree, rtree, new ArrayList<>(), new MerkleTree.TreeRange(ltree.fullRange.left, ltree.fullRange.right, (byte)0))); } /** @@ -494,22 +515,22 @@ public void matchingSmallRange() MerkleTree rtree = new MerkleTree(partitioner, range, RECOMMENDED_DEPTH, 16); rtree.init(); - byte[] h1 = "asdf".getBytes(); - byte[] h2 = "asdf".getBytes(); + byte[] h1 = digest("asdf"); + byte[] h2 = digest("asdf"); // add dummy hashes to both trees - for (TreeRange tree : ltree.invalids()) + for (TreeRange tree : ltree.rangeIterator()) { tree.addHash(new RowHash(range.right, h1, h1.length)); } - for (TreeRange tree : rtree.invalids()) + for (TreeRange tree : rtree.rangeIterator()) { tree.addHash(new RowHash(range.right, h2, h2.length)); } // top level difference() should show no differences - assertEquals(MerkleTree.difference(ltree, rtree), Lists.newArrayList()); + assertEquals(MerkleTree.difference(ltree, rtree), newArrayList()); } /** @@ -533,7 +554,7 @@ byte[] hashed(byte[] val, Integer... depths) while (depth.equals(dstack.peek())) { // consume the stack - hash = Hashable.binaryHash(hstack.pop(), hash); + hash = MerkleTree.xor(hstack.pop(), hash); depth = dstack.pop() - 1; } dstack.push(depth); @@ -643,4 +664,251 @@ private long measureTree(MerkleTree tree, Range fullRange, int depth, Ran tree.hash(fullRange); return ObjectSizes.measureDeep(tree); } + + @Test + public void testEqualTreesSameDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 3, 3); + testDifferences(trees, Collections.emptyList()); + } + + @Test + public void testEqualTreesDifferentDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 2, 3); + testDifferences(trees, Collections.emptyList()); + } + + @Test + public void testEntirelyDifferentTrees() throws IOException + { + int seed1 = makeSeed(); + int seed2 = seed1 * 32; + Trees trees = Trees.make(seed1, seed2, 3, 3); + testDifferences(trees, newArrayList(makeTreeRange(0, 16, 0))); + } + + @Test + public void testDifferentTrees1SameDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 3, 3); + trees.tree1.get(longToken(1)).addHash(digest("diff_1"), 1); + testDifferences(trees, newArrayList(makeTreeRange(0, 2, 3))); + } + + @Test + public void testDifferentTrees1DifferentDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 2, 3); + trees.tree1.get(longToken(1)).addHash(digest("diff_1"), 1); + testDifferences(trees, newArrayList(makeTreeRange(0, 4, 2))); + } + + @Test + public void testDifferentTrees2SameDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 3, 3); + trees.tree1.get(longToken(1)).addHash(digest("diff_1"), 1); + trees.tree2.get(longToken(16)).addHash(digest("diff_16"), 1); + testDifferences(trees, newArrayList(makeTreeRange(0, 2, 3), + makeTreeRange(14, 16, 3))); + } + + @Test + public void testDifferentTrees2DifferentDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 2, 3); + trees.tree1.get(longToken(1)).addHash(digest("diff_1"), 1); + trees.tree2.get(longToken(16)).addHash(digest("diff_16"), 1); + testDifferences(trees, newArrayList(makeTreeRange(0, 4, 2), + makeTreeRange(12, 16, 2))); + } + + @Test + public void testDifferentTrees3SameDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 3, 3); + trees.tree1.get(longToken(1)).addHash(digest("diff_1"), 1); + trees.tree1.get(longToken(3)).addHash(digest("diff_3"), 1); + testDifferences(trees, newArrayList(makeTreeRange(0, 4, 2))); + } + + @Test + public void testDifferentTrees3Differentepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 2, 3); + trees.tree1.get(longToken(1)).addHash(digest("diff_1"), 1); + trees.tree1.get(longToken(3)).addHash(digest("diff_3"), 1); + testDifferences(trees, newArrayList(makeTreeRange(0, 4, 2))); + } + + @Test + public void testDifferentTrees4SameDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 3, 3); + trees.tree1.get(longToken(4)).addHash(digest("diff_4"), 1); + trees.tree1.get(longToken(8)).addHash(digest("diff_8"), 1); + trees.tree1.get(longToken(12)).addHash(digest("diff_12"), 1); + trees.tree1.get(longToken(16)).addHash(digest("diff_16"), 1); + testDifferences(trees, newArrayList(makeTreeRange(2, 4, 3), + makeTreeRange(6, 8, 3), + makeTreeRange(10, 12, 3), + makeTreeRange(14, 16, 3))); + } + + @Test + public void testDifferentTrees4DifferentDepth() throws IOException + { + int seed = makeSeed(); + Trees trees = Trees.make(seed, seed, 2, 3); + trees.tree1.get(longToken(4)).addHash(digest("diff_4"), 1); + trees.tree1.get(longToken(8)).addHash(digest("diff_8"), 1); + trees.tree1.get(longToken(12)).addHash(digest("diff_12"), 1); + trees.tree1.get(longToken(16)).addHash(digest("diff_16"), 1); + testDifferences(trees, newArrayList(makeTreeRange(0, 16, 0))); + } + + private static void testDifferences(Trees trees, List expectedDifference) throws IOException + { + MerkleTree mt1 = trees.tree1; + MerkleTree mt2 = trees.tree2; + + assertDiffer(mt1, mt2, expectedDifference); + assertDiffer(mt1, mt2.moveOffHeap(), expectedDifference); + assertDiffer(mt1, cycle(mt2, true), expectedDifference); + assertDiffer(mt1, cycle(mt2, false), expectedDifference); + assertDiffer(mt1, cycle(mt2.moveOffHeap(), true), expectedDifference); + assertDiffer(mt1, cycle(mt2.moveOffHeap(), false), expectedDifference); + + assertDiffer(mt1.moveOffHeap(), mt2, expectedDifference); + assertDiffer(mt1.moveOffHeap(), mt2.moveOffHeap(), expectedDifference); + assertDiffer(mt1.moveOffHeap(), cycle(mt2, true), expectedDifference); + assertDiffer(mt1.moveOffHeap(), cycle(mt2, false), expectedDifference); + assertDiffer(mt1.moveOffHeap(), cycle(mt2.moveOffHeap(), true), expectedDifference); + assertDiffer(mt1.moveOffHeap(), cycle(mt2.moveOffHeap(), false), expectedDifference); + + assertDiffer(cycle(mt1, true), mt2, expectedDifference); + assertDiffer(cycle(mt1, true), mt2.moveOffHeap(), expectedDifference); + assertDiffer(cycle(mt1, true), cycle(mt2, true), expectedDifference); + assertDiffer(cycle(mt1, true), cycle(mt2, false), expectedDifference); + assertDiffer(cycle(mt1, true), cycle(mt2.moveOffHeap(), true), expectedDifference); + assertDiffer(cycle(mt1, true), cycle(mt2.moveOffHeap(), false), expectedDifference); + + assertDiffer(cycle(mt1, false), mt2, expectedDifference); + assertDiffer(cycle(mt1, false), mt2.moveOffHeap(), expectedDifference); + assertDiffer(cycle(mt1, false), cycle(mt2, true), expectedDifference); + assertDiffer(cycle(mt1, false), cycle(mt2, false), expectedDifference); + assertDiffer(cycle(mt1, false), cycle(mt2.moveOffHeap(), true), expectedDifference); + assertDiffer(cycle(mt1, false), cycle(mt2.moveOffHeap(), false), expectedDifference); + + assertDiffer(cycle(mt1.moveOffHeap(), true), mt2, expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), true), mt2.moveOffHeap(), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), true), cycle(mt2, true), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), true), cycle(mt2, false), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), true), cycle(mt2.moveOffHeap(), true), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), true), cycle(mt2.moveOffHeap(), false), expectedDifference); + + assertDiffer(cycle(mt1.moveOffHeap(), false), mt2, expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), false), mt2.moveOffHeap(), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), false), cycle(mt2, true), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), false), cycle(mt2, false), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), false), cycle(mt2.moveOffHeap(), true), expectedDifference); + assertDiffer(cycle(mt1.moveOffHeap(), false), cycle(mt2.moveOffHeap(), false), expectedDifference); + } + + private static void assertDiffer(MerkleTree mt1, MerkleTree mt2, List expectedDifference) + { + assertEquals(expectedDifference, MerkleTree.difference(mt1, mt2)); + assertEquals(expectedDifference, MerkleTree.difference(mt2, mt1)); + } + + private static Range longTokenRange(long start, long end) + { + return new Range<>(longToken(start), longToken(end)); + } + + private static Murmur3Partitioner.LongToken longToken(long value) + { + return new Murmur3Partitioner.LongToken(value); + } + + private static MerkleTree cycle(MerkleTree mt, boolean offHeapRequested) throws IOException + { + try (DataOutputBuffer output = new DataOutputBuffer()) + { + mt.serialize(output, MessagingService.current_version); + + try (DataInputBuffer input = new DataInputBuffer(output.buffer(false), false)) + { + return MerkleTree.deserialize(input, offHeapRequested, MessagingService.current_version); + } + } + } + + private static MerkleTree makeTree(long start, long end, int depth) + { + MerkleTree mt = new MerkleTree(Murmur3Partitioner.instance, longTokenRange(start, end), depth, Long.MAX_VALUE); + mt.init(); + return mt; + } + + private static TreeRange makeTreeRange(long start, long end, int depth) + { + return new TreeRange(longToken(start), longToken(end), depth); + } + + private static byte[][] makeHashes(int count, int seed) + { + Random random = new Random(seed); + + byte[][] hashes = new byte[count][32]; + for (int i = 0; i < count; i++) + random.nextBytes(hashes[i]); + return hashes; + } + + private static int makeSeed() + { + int seed = (int) System.currentTimeMillis(); + System.out.println("Using seed " + seed); + return seed; + } + + private static class Trees + { + MerkleTree tree1; + MerkleTree tree2; + + Trees(MerkleTree tree1, MerkleTree tree2) + { + this.tree1 = tree1; + this.tree2 = tree2; + } + + static Trees make(int hashes1seed, int hashes2seed, int tree1depth, int tree2depth) + { + byte[][] hashes1 = makeHashes(16, hashes1seed); + byte[][] hashes2 = makeHashes(16, hashes2seed); + + MerkleTree tree1 = makeTree(0, 16, tree1depth); + MerkleTree tree2 = makeTree(0, 16, tree2depth); + + for (int tok = 1; tok <= 16; tok++) + { + tree1.get(longToken(tok)).addHash(hashes1[tok - 1], 1); + tree2.get(longToken(tok)).addHash(hashes2[tok - 1], 1); + } + + return new Trees(tree1, tree2); + } + } } diff --git a/test/unit/org/apache/cassandra/utils/MerkleTreesTest.java b/test/unit/org/apache/cassandra/utils/MerkleTreesTest.java index b40f6c437c64..9e70c2048455 100644 --- a/test/unit/org/apache/cassandra/utils/MerkleTreesTest.java +++ b/test/unit/org/apache/cassandra/utils/MerkleTreesTest.java @@ -34,7 +34,6 @@ import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.utils.MerkleTree.Hashable; import org.apache.cassandra.utils.MerkleTree.RowHash; import org.apache.cassandra.utils.MerkleTree.TreeRange; import org.apache.cassandra.utils.MerkleTrees.TreeRangeIterator; @@ -43,7 +42,12 @@ public class MerkleTreesTest { - public static byte[] DUMMY = "blah".getBytes(); + private static final byte[] DUMMY = digest("dummy"); + + private static byte[] digest(String string) + { + return HashingUtils.newMessageDigest("SHA-256").digest(string.getBytes()); + } /** * If a test assumes that the tree is 8 units wide, then it should set this value @@ -193,7 +197,7 @@ public void testInvalids() Iterator ranges; // (zero, zero] - ranges = mts.invalids(); + ranges = mts.rangeIterator(); assertEquals(new Range<>(tok(-1), tok(-1)), ranges.next()); assertFalse(ranges.hasNext()); @@ -203,7 +207,7 @@ public void testInvalids() mts.split(tok(6)); mts.split(tok(3)); mts.split(tok(5)); - ranges = mts.invalids(); + ranges = mts.rangeIterator(); assertEquals(new Range<>(tok(6), tok(-1)), ranges.next()); assertEquals(new Range<>(tok(-1), tok(2)), ranges.next()); assertEquals(new Range<>(tok(2), tok(3)), ranges.next()); @@ -245,11 +249,6 @@ public void testHashPartial() // (zero,two] (two,four] (four, zero] mts.split(tok(4)); mts.split(tok(2)); - assertNull(mts.hash(left)); - assertNull(mts.hash(partial)); - assertNull(mts.hash(right)); - assertNull(mts.hash(linvalid)); - assertNull(mts.hash(rinvalid)); // validate the range mts.get(tok(2)).hash(val); @@ -280,10 +279,6 @@ public void testHashInner() mts.split(tok(2)); mts.split(tok(6)); mts.split(tok(1)); - assertNull(mts.hash(full)); - assertNull(mts.hash(lchild)); - assertNull(mts.hash(rchild)); - assertNull(mts.hash(invalid)); // validate the range mts.get(tok(1)).hash(val); @@ -315,9 +310,6 @@ public void testHashDegenerate() mts.split(tok(4)); mts.split(tok(2)); mts.split(tok(1)); - assertNull(mts.hash(full)); - assertNull(mts.hash(childfull)); - assertNull(mts.hash(invalid)); // validate the range mts.get(tok(1)).hash(val); @@ -349,7 +341,7 @@ public void testHashRandom() } // validate the tree - TreeRangeIterator ranges = mts.invalids(); + TreeRangeIterator ranges = mts.rangeIterator(); for (TreeRange range : ranges) range.addHash(new RowHash(range.right, new byte[0], 0)); @@ -378,13 +370,16 @@ public void testValidateTree() mts.split(tok(6)); mts.split(tok(10)); - ranges = mts.invalids(); - ranges.next().addAll(new HIterator(2, 4)); // (-1,4]: depth 2 - ranges.next().addAll(new HIterator(6)); // (4,6] - ranges.next().addAll(new HIterator(8)); // (6,8] + int seed = 123456789; + + Random random1 = new Random(seed); + ranges = mts.rangeIterator(); + ranges.next().addAll(new HIterator(random1, 2, 4)); // (-1,4]: depth 2 + ranges.next().addAll(new HIterator(random1, 6)); // (4,6] + ranges.next().addAll(new HIterator(random1, 8)); // (6,8] ranges.next().addAll(new HIterator(/*empty*/ new int[0])); // (8,10] - ranges.next().addAll(new HIterator(12)); // (10,12] - ranges.next().addAll(new HIterator(14, -1)); // (12,-1]: depth 2 + ranges.next().addAll(new HIterator(random1, 12)); // (10,12] + ranges.next().addAll(new HIterator(random1, 14, -1)); // (12,-1]: depth 2 mts2.split(tok(8)); @@ -395,15 +390,16 @@ public void testValidateTree() mts2.split(tok(9)); mts2.split(tok(11)); - ranges = mts2.invalids(); - ranges.next().addAll(new HIterator(2)); // (-1,2] - ranges.next().addAll(new HIterator(4)); // (2,4] - ranges.next().addAll(new HIterator(6, 8)); // (4,8]: depth 2 + Random random2 = new Random(seed); + ranges = mts2.rangeIterator(); + ranges.next().addAll(new HIterator(random2, 2)); // (-1,2] + ranges.next().addAll(new HIterator(random2, 4)); // (2,4] + ranges.next().addAll(new HIterator(random2, 6, 8)); // (4,8]: depth 2 ranges.next().addAll(new HIterator(/*empty*/ new int[0])); // (8,9] ranges.next().addAll(new HIterator(/*empty*/ new int[0])); // (9,10] ranges.next().addAll(new HIterator(/*empty*/ new int[0])); // (10,11]: depth 4 - ranges.next().addAll(new HIterator(12)); // (11,12]: depth 4 - ranges.next().addAll(new HIterator(14, -1)); // (12,-1]: depth 2 + ranges.next().addAll(new HIterator(random2, 12)); // (11,12]: depth 4 + ranges.next().addAll(new HIterator(random2, 14, -1)); // (12,-1]: depth 2 byte[] mthash = mts.hash(full); byte[] mt2hash = mts2.hash(full); @@ -425,7 +421,7 @@ public void testSerialization() throws Exception // populate and validate the tree mts.init(); - for (TreeRange range : mts.invalids()) + for (TreeRange range : mts.rangeIterator()) range.addAll(new HIterator(range.right)); byte[] initialhash = mts.hash(first); @@ -456,11 +452,15 @@ public void testDifference() mts.init(); mts2.init(); + int seed = 123456789; // add dummy hashes to both trees - for (TreeRange range : mts.invalids()) - range.addAll(new HIterator(range.right)); - for (TreeRange range : mts2.invalids()) - range.addAll(new HIterator(range.right)); + Random random1 = new Random(seed); + for (TreeRange range : mts.rangeIterator()) + range.addAll(new HIterator(random1, range.right)); + + Random random2 = new Random(seed); + for (TreeRange range : mts2.rangeIterator()) + range.addAll(new HIterator(random2, range.right)); TreeRange leftmost = null; TreeRange middle = null; @@ -468,14 +468,14 @@ public void testDifference() mts.maxsize(fullRange(), maxsize + 2); // give some room for splitting // split the leftmost - Iterator ranges = mts.invalids(); + Iterator ranges = mts.rangeIterator(); leftmost = ranges.next(); mts.split(leftmost.right); // set the hashes for the leaf of the created split middle = mts.get(leftmost.right); - middle.hash("arbitrary!".getBytes()); - mts.get(partitioner.midpoint(leftmost.left, leftmost.right)).hash("even more arbitrary!".getBytes()); + middle.hash(digest("arbitrary!")); + mts.get(partitioner.midpoint(leftmost.left, leftmost.right)).hash(digest("even more arbitrary!")); // trees should disagree for (leftmost.left, middle.right] List> diffs = MerkleTrees.difference(mts, mts2); @@ -504,7 +504,7 @@ byte[] hashed(byte[] val, Integer... depths) while (depth.equals(dstack.peek())) { // consume the stack - hash = Hashable.binaryHash(hstack.pop(), hash); + hash = MerkleTree.xor(hstack.pop(), hash); depth = dstack.pop()-1; } dstack.push(depth); @@ -516,25 +516,47 @@ byte[] hashed(byte[] val, Integer... depths) public static class HIterator extends AbstractIterator { - private Iterator tokens; + private final Random random; + private final Iterator tokens; - public HIterator(int... tokens) + HIterator(int... tokens) { - List tlist = new LinkedList(); + this(new Random(), tokens); + } + + HIterator(Random random, int... tokens) + { + List tlist = new ArrayList<>(tokens.length); for (int token : tokens) tlist.add(tok(token)); this.tokens = tlist.iterator(); + this.random = random; } public HIterator(Token... tokens) { - this.tokens = Arrays.asList(tokens).iterator(); + this(new Random(), tokens); + } + + HIterator(Random random, Token... tokens) + { + this(random, Arrays.asList(tokens).iterator()); + } + + private HIterator(Random random, Iterator tokens) + { + this.random = random; + this.tokens = tokens; } public RowHash computeNext() { if (tokens.hasNext()) - return new RowHash(tokens.next(), DUMMY, DUMMY.length); + { + byte[] digest = new byte[32]; + random.nextBytes(digest); + return new RowHash(tokens.next(), digest, 12345L); + } return endOfData(); } } From da8d41f497efedf57e335ec2664680da583a3aba Mon Sep 17 00:00:00 2001 From: Jordan West Date: Mon, 5 Aug 2019 09:44:14 -0700 Subject: [PATCH 38/78] Use mean row count instead of mean column count for index selectivity calculation patch by Jordan West; reviewed by Blake Eggleston for CASSANDRA-15259 --- CHANGES.txt | 1 + .../index/internal/CassandraIndex.java | 21 +++++++++++- .../org/apache/cassandra/SchemaLoader.java | 33 +++++++++++++++++++ .../cassandra/db/SecondaryIndexTest.java | 26 +++++++++++++++ 4 files changed, 80 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index f04b489d92db..c2bed922e040 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Use mean row count instead of mean column count for index selectivity calculation (CASSANDRA-15259) * Avoid updating unchanged gossip states (CASSANDRA-15097) * Prevent recreation of previously dropped columns with a different kind (CASSANDRA-14948) * Prevent client requests from blocking on executor task queue (CASSANDRA-15013) diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java index 3211fe9cc727..ad5dd4bb4027 100644 --- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java +++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java @@ -278,7 +278,26 @@ public AbstractType customExpressionValueType() public long getEstimatedResultRows() { - return indexCfs.getMeanColumns(); + long totalRows = 0; + long totalPartitions = 0; + for (SSTableReader sstable : indexCfs.getSSTables(SSTableSet.CANONICAL)) + { + if (sstable.descriptor.version.storeRows()) + { + totalPartitions += sstable.getEstimatedPartitionSize().count(); + totalRows += sstable.getTotalRows(); + } else + { + // for legacy sstables we don't have a total row count so we approximate it + // using estimated column count (which is the same logic as pre-3.0 + // see CASSANDRA-15259 + long colCount = sstable.getEstimatedColumnCount().count(); + totalPartitions += colCount; + totalRows += sstable.getEstimatedColumnCount().mean() * colCount; + } + } + + return totalPartitions > 0 ? (int) (totalRows / totalPartitions) : 0; } /** diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java index 16869739a47c..8d61f393a747 100644 --- a/test/unit/org/apache/cassandra/SchemaLoader.java +++ b/test/unit/org/apache/cassandra/SchemaLoader.java @@ -425,6 +425,39 @@ public static CFMetaData compositeIndexCFMD(String ksName, String cfName, boolea return cfm.compression(getCompressionParameters()); } + + public static CFMetaData compositeMultipleIndexCFMD(String ksName, String cfName) throws ConfigurationException + { + CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName) + .addPartitionKey("key", AsciiType.instance) + .addClusteringColumn("c1", AsciiType.instance) + .addRegularColumn("birthdate", LongType.instance) + .addRegularColumn("notbirthdate", LongType.instance) + .build(); + + cfm.indexes( + cfm.getIndexes() + .with(IndexMetadata.fromIndexTargets(cfm, + Collections.singletonList( + new IndexTarget(new ColumnIdentifier("birthdate", true), + IndexTarget.Type.VALUES)), + "birthdate_key_index", + IndexMetadata.Kind.COMPOSITES, + Collections.EMPTY_MAP)) + .with(IndexMetadata.fromIndexTargets(cfm, + Collections.singletonList( + new IndexTarget(new ColumnIdentifier("notbirthdate", true), + IndexTarget.Type.VALUES)), + "notbirthdate_key_index", + IndexMetadata.Kind.COMPOSITES, + Collections.EMPTY_MAP)) + ); + + + return cfm.compression(getCompressionParameters()); + } + + public static CFMetaData keysIndexCFMD(String ksName, String cfName, boolean withIndex) throws ConfigurationException { CFMetaData cfm = CFMetaData.Builder.createDense(ksName, cfName, false, false) diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java index adcb68b653f8..9fb0463845c4 100644 --- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java +++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java @@ -57,6 +57,7 @@ public class SecondaryIndexTest { public static final String KEYSPACE1 = "SecondaryIndexTest1"; public static final String WITH_COMPOSITE_INDEX = "WithCompositeIndex"; + public static final String WITH_MULTIPLE_COMPOSITE_INDEX = "WithMultipleCompositeIndex"; public static final String WITH_KEYS_INDEX = "WithKeysIndex"; public static final String COMPOSITE_INDEX_TO_BE_ADDED = "CompositeIndexToBeAdded"; @@ -68,6 +69,7 @@ public static void defineSchema() throws ConfigurationException KeyspaceParams.simple(1), SchemaLoader.compositeIndexCFMD(KEYSPACE1, WITH_COMPOSITE_INDEX, true).gcGraceSeconds(0), SchemaLoader.compositeIndexCFMD(KEYSPACE1, COMPOSITE_INDEX_TO_BE_ADDED, false).gcGraceSeconds(0), + SchemaLoader.compositeMultipleIndexCFMD(KEYSPACE1, WITH_MULTIPLE_COMPOSITE_INDEX).gcGraceSeconds(0), SchemaLoader.keysIndexCFMD(KEYSPACE1, WITH_KEYS_INDEX, true).gcGraceSeconds(0)); } @@ -76,6 +78,7 @@ public void truncateCFS() { Keyspace.open(KEYSPACE1).getColumnFamilyStore(WITH_COMPOSITE_INDEX).truncateBlocking(); Keyspace.open(KEYSPACE1).getColumnFamilyStore(COMPOSITE_INDEX_TO_BE_ADDED).truncateBlocking(); + Keyspace.open(KEYSPACE1).getColumnFamilyStore(WITH_MULTIPLE_COMPOSITE_INDEX).truncateBlocking(); Keyspace.open(KEYSPACE1).getColumnFamilyStore(WITH_KEYS_INDEX).truncateBlocking(); } @@ -491,6 +494,29 @@ public void testKeysSearcherSimple() throws Exception assertIndexedCount(cfs, ByteBufferUtil.bytes("birthdate"), 1l, 10); } + @Test + public void testSelectivityWithMultipleIndexes() + { + ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(WITH_MULTIPLE_COMPOSITE_INDEX); + + // creates rows such that birthday_index has 1 partition (key = 1L) with 4 rows -- mean row count = 4, and notbirthdate_index has 2 partitions with 2 rows each -- mean row count = 2 + new RowUpdateBuilder(cfs.metadata, 0, "k1").clustering("c").add("birthdate", 1L).add("notbirthdate", 2L).build().applyUnsafe(); + new RowUpdateBuilder(cfs.metadata, 0, "k2").clustering("c").add("birthdate", 1L).add("notbirthdate", 2L).build().applyUnsafe(); + new RowUpdateBuilder(cfs.metadata, 0, "k3").clustering("c").add("birthdate", 1L).add("notbirthdate", 3L).build().applyUnsafe(); + new RowUpdateBuilder(cfs.metadata, 0, "k4").clustering("c").add("birthdate", 1L).add("notbirthdate", 3L).build().applyUnsafe(); + + cfs.forceBlockingFlush(); + ReadCommand rc = Util.cmd(cfs) + .fromKeyIncl("k1") + .toKeyIncl("k3") + .columns("birthdate") + .filterOn("birthdate", Operator.EQ, 1L) + .filterOn("notbirthdate", Operator.EQ, 0L) + .build(); + + assertEquals("notbirthdate_key_index", rc.indexMetadata().name); + } + private void assertIndexedNone(ColumnFamilyStore cfs, ByteBuffer col, Object val) { assertIndexedCount(cfs, col, val, 0); From b773bc7ac51fedc07145017edaefa919fac25696 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson Date: Thu, 9 May 2019 10:29:44 +0200 Subject: [PATCH 39/78] Make sure user defined compaction transactions are always closed Patch by marcuse; reviewed by Blake Eggleston for CASSANDRA-15123 --- CHANGES.txt | 1 + .../db/compaction/CompactionManager.java | 58 ++++++----- .../db/compaction/CompactionsBytemanTest.java | 99 +++++++++++++++++++ 3 files changed, 134 insertions(+), 24 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 43dbda330645..dc8baf2c265b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.11.5 + * Make sure user defined compaction transactions are always closed (CASSANDRA-15123) * Fix cassandra-env.sh to use $CASSANDRA_CONF to find cassandra-jaas.config (CASSANDRA-14305) * Fixed nodetool cfstats printing index name twice (CASSANDRA-14903) * Add flag to disable SASI indexes, and warnings on creation (CASSANDRA-14866) diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index e0ec179f0596..7086d7729d4e 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -784,36 +784,39 @@ protected void runMayThrow() public void forceCompactionForTokenRange(ColumnFamilyStore cfStore, Collection> ranges) { - final Collection tasks = cfStore.runWithCompactionsDisabled(() -> - { - Collection sstables = sstablesInBounds(cfStore, ranges); - if (sstables == null || sstables.isEmpty()) - { - logger.debug("No sstables found for the provided token range"); - return null; - } - return cfStore.getCompactionStrategyManager().getUserDefinedTasks(sstables, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds())); - }, false, false); + Callable> taskCreator = () -> { + Collection sstables = sstablesInBounds(cfStore, ranges); + if (sstables == null || sstables.isEmpty()) + { + logger.debug("No sstables found for the provided token range"); + return null; + } + return cfStore.getCompactionStrategyManager().getUserDefinedTasks(sstables, getDefaultGcBefore(cfStore, FBUtilities.nowInSeconds())); + }; + + final Collection tasks = cfStore.runWithCompactionsDisabled(taskCreator, false, false); if (tasks == null) return; Runnable runnable = new WrappedRunnable() { - protected void runMayThrow() + protected void runMayThrow() throws Exception { - for (AbstractCompactionTask task : tasks) - if (task != null) - task.execute(metrics); + try + { + for (AbstractCompactionTask task : tasks) + if (task != null) + task.execute(metrics); + } + finally + { + FBUtilities.closeAll(tasks.stream().map(task -> task.transaction).collect(Collectors.toList())); + } } }; - if (executor.isShutdown()) - { - logger.info("Compaction executor has shut down, not submitting task"); - return; - } - FBUtilities.waitOnFuture(executor.submit(runnable)); + FBUtilities.waitOnFuture(executor.submitIfRunning(runnable, "force compaction for token range")); } private static Collection sstablesInBounds(ColumnFamilyStore cfs, Collection> tokenRangeCollection) @@ -915,7 +918,7 @@ public Future submitUserDefined(final ColumnFamilyStore cfs, final Collection { Runnable runnable = new WrappedRunnable() { - protected void runMayThrow() + protected void runMayThrow() throws Exception { // look up the sstables now that we're on the compaction executor, so we don't try to re-compact // something that was already being compacted earlier. @@ -941,10 +944,17 @@ protected void runMayThrow() else { List tasks = cfs.getCompactionStrategyManager().getUserDefinedTasks(sstables, gcBefore); - for (AbstractCompactionTask task : tasks) + try { - if (task != null) - task.execute(metrics); + for (AbstractCompactionTask task : tasks) + { + if (task != null) + task.execute(metrics); + } + } + finally + { + FBUtilities.closeAll(tasks.stream().map(task -> task.transaction).collect(Collectors.toList())); } } } diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java index 0b391a5006cc..d5f28007f1ab 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java @@ -18,17 +18,32 @@ package org.apache.cassandra.db.compaction; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; + import org.junit.Test; import org.junit.runner.RunWith; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.metrics.CompactionMetrics; import org.apache.cassandra.utils.FBUtilities; import org.jboss.byteman.contrib.bmunit.BMRule; import org.jboss.byteman.contrib.bmunit.BMUnitRunner; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; @RunWith(BMUnitRunner.class) public class CompactionsBytemanTest extends CQLTester @@ -53,4 +68,88 @@ public void testCompactingCFCounting() throws Throwable FBUtilities.waitOnFutures(CompactionManager.instance.submitBackground(cfs)); assertEquals(0, CompactionManager.instance.compactingCF.count(cfs)); } + + @Test + @BMRule(name = "Stop all compactions", + targetClass = "CompactionTask", + targetMethod = "runMayThrow", + targetLocation = "AT INVOKE getCompactionAwareWriter", + action = "$ci.stop()") + public void testStopUserDefinedCompactionRepaired() throws Throwable + { + testStopCompactionRepaired((cfs) -> { + Collection files = cfs.getLiveSSTables().stream().map(s -> s.descriptor).collect(Collectors.toList()); + FBUtilities.waitOnFuture(CompactionManager.instance.submitUserDefined(cfs, files, CompactionManager.NO_GC)); + }); + } + + @Test + @BMRule(name = "Stop all compactions", + targetClass = "CompactionTask", + targetMethod = "runMayThrow", + targetLocation = "AT INVOKE getCompactionAwareWriter", + action = "$ci.stop()") + public void testStopSubRangeCompactionRepaired() throws Throwable + { + testStopCompactionRepaired((cfs) -> { + Collection> ranges = Collections.singleton(new Range<>(cfs.getPartitioner().getMinimumToken(), + cfs.getPartitioner().getMaximumToken())); + CompactionManager.instance.forceCompactionForTokenRange(cfs, ranges); + }); + } + + public void testStopCompactionRepaired(Consumer compactionRunner) throws Throwable + { + String table = createTable("CREATE TABLE %s (k INT, c INT, v INT, PRIMARY KEY (k, c))"); + ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(table); + cfs.disableAutoCompaction(); + for (int i = 0; i < 5; i++) + { + for (int j = 0; j < 10; j++) + { + execute("insert into %s (k, c, v) values (?, ?, ?)", i, j, i*j); + } + cfs.forceBlockingFlush(); + } + setRepaired(cfs, cfs.getLiveSSTables()); + for (int i = 0; i < 5; i++) + { + for (int j = 0; j < 10; j++) + { + execute("insert into %s (k, c, v) values (?, ?, ?)", i, j, i*j); + } + cfs.forceBlockingFlush(); + } + + assertTrue(cfs.getTracker().getCompacting().isEmpty()); + assertTrue(CompactionMetrics.getCompactions().stream().noneMatch(h -> h.getCompactionInfo().getCFMetaData().equals(cfs.metadata))); + + try + { + compactionRunner.accept(cfs); + fail("compaction should fail"); + } + catch (RuntimeException t) + { + if (!(t.getCause().getCause() instanceof CompactionInterruptedException)) + throw t; + //expected + } + + assertTrue(cfs.getTracker().getCompacting().isEmpty()); + assertTrue(CompactionMetrics.getCompactions().stream().noneMatch(h -> h.getCompactionInfo().getCFMetaData().equals(cfs.metadata))); + + } + + private void setRepaired(ColumnFamilyStore cfs, Iterable sstables) throws IOException + { + Set changed = new HashSet<>(); + for (SSTableReader sstable: sstables) + { + sstable.descriptor.getMetadataSerializer().mutateRepairedAt(sstable.descriptor, System.currentTimeMillis()); + sstable.reloadSSTableMetadata(); + changed.add(sstable); + } + cfs.getTracker().notifySSTableRepairedStatusChanged(changed); + } } From 9af57a508da637f85b32ada0f54e91c72aca0104 Mon Sep 17 00:00:00 2001 From: Marcus Eriksson Date: Thu, 25 Apr 2019 12:31:25 +0200 Subject: [PATCH 40/78] Filter sstables earlier when running cleanup Patch by marcuse; reviewed by Jordan West for CASSANDRA-15100 --- CHANGES.txt | 1 + .../db/compaction/CompactionManager.java | 30 ++++++++--- .../org/apache/cassandra/db/CleanupTest.java | 53 +++++++++++++++---- 3 files changed, 68 insertions(+), 16 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index c2bed922e040..a46a3272ade1 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Filter sstables earlier when running cleanup (CASSANDRA-15100) * Use mean row count instead of mean column count for index selectivity calculation (CASSANDRA-15259) * Avoid updating unchanged gossip states (CASSANDRA-15097) * Prevent recreation of previously dropped columns with a different kind (CASSANDRA-14948) diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java index 1bd8ff3729cc..694ad6211e44 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java @@ -285,6 +285,7 @@ public void run() @SuppressWarnings("resource") private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException { + logger.info("Starting {} for {}.{}", operationType, cfs.keyspace.getName(), cfs.getTableName()); List transactions = new ArrayList<>(); List> futures = new ArrayList<>(); try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType)) @@ -326,6 +327,7 @@ public Object call() throws Exception } FBUtilities.waitOnFutures(futures); assert compacting.originals().isEmpty(); + logger.info("Finished {} for {}.{} successfully", operationType, cfs.keyspace.getName(), cfs.getTableName()); return AllSSTableOpStatus.SUCCESSFUL; } finally @@ -341,7 +343,7 @@ public Object call() throws Exception } Throwable fail = Throwables.close(null, transactions); if (fail != null) - logger.error("Failed to cleanup lifecycle transactions", fail); + logger.error("Failed to cleanup lifecycle transactions ({} for {}.{})", operationType, cfs.keyspace.getName(), cfs.getTableName(), fail); } } @@ -463,7 +465,25 @@ public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jo public Iterable filterSSTables(LifecycleTransaction transaction) { List sortedSSTables = Lists.newArrayList(transaction.originals()); - Collections.sort(sortedSSTables, new SSTableReader.SizeComparator()); + Iterator sstableIter = sortedSSTables.iterator(); + int totalSSTables = 0; + int skippedSStables = 0; + while (sstableIter.hasNext()) + { + SSTableReader sstable = sstableIter.next(); + totalSSTables++; + if (!needsCleanup(sstable, ranges)) + { + logger.debug("Not cleaning up {} ([{}, {}]) - no tokens outside owned ranges {}", + sstable, sstable.first.getToken(), sstable.last.getToken(), ranges); + sstableIter.remove(); + transaction.cancel(sstable); + skippedSStables++; + } + } + logger.info("Skipping cleanup for {}/{} sstables for {}.{} since they are fully contained in owned ranges ({})", + skippedSStables, totalSSTables, cfStore.keyspace.getName(), cfStore.getTableName(), ranges); + sortedSSTables.sort(new SSTableReader.SizeComparator()); return sortedSSTables; } @@ -886,11 +906,7 @@ private void doCleanupOne(final ColumnFamilyStore cfs, LifecycleTransaction txn, { txn.obsoleteOriginals(); txn.finish(); - return; - } - if (!needsCleanup(sstable, ranges)) - { - logger.trace("Skipping {} for cleanup; all rows should be kept", sstable); + logger.info("SSTable {} ([{}, {}]) does not intersect the owned ranges ({}), dropping it", sstable, sstable.first.getToken(), sstable.last.getToken(), ranges); return; } diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java index 99030c5c4b94..d4c613d237fc 100644 --- a/test/unit/org/apache/cassandra/db/CleanupTest.java +++ b/test/unit/org/apache/cassandra/db/CleanupTest.java @@ -28,10 +28,12 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import com.google.common.collect.Sets; import org.junit.BeforeClass; import org.junit.Test; @@ -68,6 +70,9 @@ public class CleanupTest public static final String CF_INDEXED2 = "Indexed2"; public static final String CF_STANDARD2 = "Standard2"; + public static final String KEYSPACE3 = "CleanupSkipSSTables"; + public static final String CF_STANDARD3 = "Standard3"; + public static final ByteBuffer COLUMN = ByteBufferUtil.bytes("birthdate"); public static final ByteBuffer VALUE = ByteBuffer.allocate(8); static @@ -105,9 +110,11 @@ public String getDatacenter(InetAddress endpoint) KeyspaceParams.nts("DC1", 1), SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD2), SchemaLoader.compositeIndexCFMD(KEYSPACE2, CF_INDEXED2, true)); + SchemaLoader.createKeyspace(KEYSPACE3, + KeyspaceParams.nts("DC1", 1), + SchemaLoader.standardCFMD(KEYSPACE3, CF_STANDARD3)); } - /* @Test public void testCleanup() throws ExecutionException, InterruptedException { @@ -116,17 +123,13 @@ public void testCleanup() throws ExecutionException, InterruptedException Keyspace keyspace = Keyspace.open(KEYSPACE1); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1); - UnfilteredPartitionIterator iter; - // insert data and verify we get it back w/ range query fillCF(cfs, "val", LOOPS); // record max timestamps of the sstables pre-cleanup List expectedMaxTimestamps = getMaxTimestampList(cfs); - iter = Util.getRangeSlice(cfs); - assertEquals(LOOPS, Iterators.size(iter)); - + assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size()); // with one token in the ring, owned by the local node, cleanup should be a no-op CompactionManager.instance.performCleanup(cfs, 2); @@ -134,10 +137,8 @@ public void testCleanup() throws ExecutionException, InterruptedException assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs)); // check data is still there - iter = Util.getRangeSlice(cfs); - assertEquals(LOOPS, Iterators.size(iter)); + assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size()); } - */ @Test public void testCleanupWithIndexes() throws IOException, ExecutionException, InterruptedException @@ -234,6 +235,40 @@ public void testCleanupWithNoTokenRange() throws Exception assertTrue(cfs.getLiveSSTables().isEmpty()); } + @Test + public void testCleanupSkippingSSTables() throws UnknownHostException, ExecutionException, InterruptedException + { + Keyspace keyspace = Keyspace.open(KEYSPACE3); + ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD3); + cfs.disableAutoCompaction(); + for (byte i = 0; i < 100; i++) + { + new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), ByteBuffer.wrap(new byte[] {i})) + .clustering(COLUMN) + .add("val", VALUE) + .build() + .applyUnsafe(); + cfs.forceBlockingFlush(); + } + TokenMetadata tmd = StorageService.instance.getTokenMetadata(); + tmd.clearUnsafe(); + tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.1")); + tmd.updateNormalToken(token(new byte[] {50}), InetAddress.getByName("127.0.0.1")); + Set beforeFirstCleanup = Sets.newHashSet(cfs.getLiveSSTables()); + // single token - 127.0.0.1 owns everything, cleanup should be noop + cfs.forceCleanup(2); + assertEquals(beforeFirstCleanup, cfs.getLiveSSTables()); + tmd.updateNormalToken(token(new byte[] {120}), InetAddress.getByName("127.0.0.2")); + cfs.forceCleanup(2); + for (SSTableReader sstable : cfs.getLiveSSTables()) + { + assertEquals(sstable.first, sstable.last); // single-token sstables + assertTrue(sstable.first.getToken().compareTo(token(new byte[]{50})) <= 0); + // with single-token sstables they should all either be skipped or dropped: + assertTrue(beforeFirstCleanup.contains(sstable)); + } + } + @Test public void testNeedsCleanup() throws Exception From 6d037b678188ef06ec183fbcd8439d24a004fc50 Mon Sep 17 00:00:00 2001 From: Liudmila Kornilova Date: Thu, 18 Jul 2019 11:25:13 +0300 Subject: [PATCH 41/78] Catch non-IOException in FileUtils.close so all resources are closed patch by Liudmila Kornilova; reviewed by Benedict Elliott Smith for CASSANDRA-15225 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/io/util/FileUtils.java | 10 +++++----- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index bec65cf9b518..c9adf599e6b6 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Catch non-IOException in FileUtils.close to make sure that all resources are closed (CASSANDRA-15225) * Align load column in nodetool status output (CASSANDRA-14787) * CassandraNetworkAuthorizer uses cached roles info (CASSANDRA-15089) * Introduce optional timeouts for idle client sessions (CASSANDRA-11097) diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java index ed03715e5a22..6e5c00af6ecf 100644 --- a/src/java/org/apache/cassandra/io/util/FileUtils.java +++ b/src/java/org/apache/cassandra/io/util/FileUtils.java @@ -312,7 +312,7 @@ public static void close(Closeable... cs) throws IOException public static void close(Iterable cs) throws IOException { - IOException e = null; + Throwable e = null; for (Closeable c : cs) { try @@ -320,14 +320,14 @@ public static void close(Iterable cs) throws IOException if (c != null) c.close(); } - catch (IOException ex) + catch (Throwable ex) { - e = ex; + if (e == null) e = ex; + else e.addSuppressed(ex); logger.warn("Failed closing stream {}", c, ex); } } - if (e != null) - throw e; + maybeFail(e, IOException.class); } public static void closeQuietly(Iterable cs) From f21106fcd0e5d870cf9d85b2d396eab9fe4515cd Mon Sep 17 00:00:00 2001 From: Liudmila Kornilova Date: Thu, 18 Jul 2019 11:25:13 +0300 Subject: [PATCH 42/78] Catch non-IOException in FileUtils.close so all resources are closed patch by Liudmila Kornilova; reviewed by Benedict Elliott Smith for CASSANDRA-15225 --- CHANGES.txt | 1 + .../apache/cassandra/io/util/FileUtils.java | 10 +++---- .../apache/cassandra/utils/Throwables.java | 27 +++++++++++++++++-- 3 files changed, 31 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d8da35464ab2..eabdcaa04216 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.2.15 + * Catch non-IOException in FileUtils.close to make sure that all resources are closed (CASSANDRA-15225) * Handle exceptions during authentication/authorization (CASSANDRA-15041) * Fix JDK7 compatibility broken in cassandra-2.2 (CASSANDRA-15050) * Support cross version messaging in in-jvm upgrade dtests (CASSANDRA-15078) diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java index bf0fae5ec662..3252ec8b9fc0 100644 --- a/src/java/org/apache/cassandra/io/util/FileUtils.java +++ b/src/java/org/apache/cassandra/io/util/FileUtils.java @@ -235,7 +235,7 @@ public static void close(Closeable... cs) throws IOException public static void close(Iterable cs) throws IOException { - IOException e = null; + Throwable e = null; for (Closeable c : cs) { try @@ -243,14 +243,14 @@ public static void close(Iterable cs) throws IOException if (c != null) c.close(); } - catch (IOException ex) + catch (Throwable ex) { - e = ex; + if (e == null) e = ex; + else e.addSuppressed(ex); logger.warn("Failed closing stream {}", c, ex); } } - if (e != null) - throw e; + maybeFail(e, IOException.class); } public static String getCanonicalPath(String filename) diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java index 877f388d2a60..82703c8b0cce 100644 --- a/src/java/org/apache/cassandra/utils/Throwables.java +++ b/src/java/org/apache/cassandra/utils/Throwables.java @@ -35,8 +35,31 @@ public static Throwable merge(Throwable existingFail, Throwable newFail) public static void maybeFail(Throwable fail) { - if (fail != null) - com.google.common.base.Throwables.propagate(fail); + if (failIfCanCast(fail, null)) + throw new RuntimeException(fail); + } + + public static void maybeFail(Throwable fail, Class checked) throws T + { + if (failIfCanCast(fail, checked)) + throw new RuntimeException(fail); + } + + public static boolean failIfCanCast(Throwable fail, Class checked) throws T + { + if (fail == null) + return false; + + if (fail instanceof Error) + throw (Error) fail; + + if (fail instanceof RuntimeException) + throw (RuntimeException) fail; + + if (checked != null && checked.isInstance(fail)) + throw checked.cast(fail); + + return true; } public static Throwable close(Throwable accumulate, Iterable closeables) From 905ed484932688b14f5bfe83e99b14d63094d62b Mon Sep 17 00:00:00 2001 From: Sumanth Pasupuleti Date: Thu, 8 Aug 2019 08:50:42 -0700 Subject: [PATCH 43/78] Add more information around commit message format expected for a patch --- doc/source/development/patches.rst | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/doc/source/development/patches.rst b/doc/source/development/patches.rst index 43bdb8c68091..f3a2cca0f1cf 100644 --- a/doc/source/development/patches.rst +++ b/doc/source/development/patches.rst @@ -108,7 +108,14 @@ So you've finished coding and the great moment arrives: it's time to submit your a. Attach a patch to JIRA with a single squashed commit in it (per branch), or b. Squash the commits in-place in your branches into one - 6. Include a CHANGES.txt entry (put it at the top of the list), and format the commit message appropriately in your patch ending with the following statement on the last line: ``patch by X; reviewed by Y for CASSANDRA-ZZZZZ`` + 6. Include a CHANGES.txt entry (put it at the top of the list), and format the commit message appropriately in your patch as below. + + :: + + + + patch by ; reviewed by for CASSANDRA-##### + 7. When you're happy with the result, create a patch: :: From d6c049f0835f137fc07711ec5cf9adc323347c65 Mon Sep 17 00:00:00 2001 From: Hannu Kroger Date: Thu, 8 Aug 2019 11:02:15 -0700 Subject: [PATCH 44/78] Fix error with non-existent table for nodetool tablehistograms Patch by Hannu Kroger; reviewed by Chris Lohfink for CASSANDRA-14410 --- CHANGES.txt | 1 + .../tools/nodetool/TableHistograms.java | 50 +++++++++++-------- 2 files changed, 30 insertions(+), 21 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index c9adf599e6b6..1d0e11d16438 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Fix error with non-existent table for nodetool tablehistograms (CASSANDRA-14410) * Catch non-IOException in FileUtils.close to make sure that all resources are closed (CASSANDRA-15225) * Align load column in nodetool status output (CASSANDRA-14787) * CassandraNetworkAuthorizer uses cached roles info (CASSANDRA-15089) diff --git a/src/java/org/apache/cassandra/tools/nodetool/TableHistograms.java b/src/java/org/apache/cassandra/tools/nodetool/TableHistograms.java index f24c8a369e64..cb3b9463df91 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/TableHistograms.java +++ b/src/java/org/apache/cassandra/tools/nodetool/TableHistograms.java @@ -23,17 +23,19 @@ import io.airlift.airline.Command; import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; + import org.apache.cassandra.db.ColumnFamilyStoreMBean; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.NodeTool.NodeToolCmd; import org.apache.cassandra.utils.EstimatedHistogram; + import org.apache.commons.lang3.ArrayUtils; @Command(name = "tablehistograms", description = "Print statistic histograms for a given table") @@ -45,40 +47,46 @@ public class TableHistograms extends NodeToolCmd @Override public void execute(NodeProbe probe) { - Map> tablesList = new HashMap<>(); + Multimap tablesList = HashMultimap.create(); + + // a > mapping for verification or as reference if none provided + Multimap allTables = HashMultimap.create(); + Iterator> tableMBeans = probe.getColumnFamilyStoreMBeanProxies(); + while (tableMBeans.hasNext()) + { + Map.Entry entry = tableMBeans.next(); + allTables.put(entry.getKey(), entry.getValue().getTableName()); + } + if (args.size() == 2) { - tablesList.put(args.get(0), new ArrayList(Arrays.asList(args.get(1)))); + tablesList.put(args.get(0), args.get(1)); } else if (args.size() == 1) { String[] input = args.get(0).split("\\."); checkArgument(input.length == 2, "tablehistograms requires keyspace and table name arguments"); - tablesList.put(input[0], new ArrayList(Arrays.asList(input[1]))); + tablesList.put(input[0], input[1]); } else { - // get a list of table stores - Iterator> tableMBeans = probe.getColumnFamilyStoreMBeanProxies(); - while (tableMBeans.hasNext()) + // use all tables + tablesList = allTables; + } + + // verify that all tables to list exist + for (String keyspace : tablesList.keys()) + { + for (String table : tablesList.get(keyspace)) { - Map.Entry entry = tableMBeans.next(); - String keyspaceName = entry.getKey(); - ColumnFamilyStoreMBean tableProxy = entry.getValue(); - if (!tablesList.containsKey(keyspaceName)) - { - tablesList.put(keyspaceName, new ArrayList()); - } - tablesList.get(keyspaceName).add(tableProxy.getTableName()); + if (!allTables.containsEntry(keyspace, table)) + throw new IllegalArgumentException("Unknown table " + keyspace + '.' + table); } } - Iterator>> iter = tablesList.entrySet().iterator(); - while(iter.hasNext()) + for (String keyspace : tablesList.keys()) { - Map.Entry> entry = iter.next(); - String keyspace = entry.getKey(); - for (String table : entry.getValue()) + for (String table : tablesList.get(keyspace)) { // calculate percentile of row size and column count long[] estimatedPartitionSize = (long[]) probe.getColumnFamilyMetric(keyspace, table, "EstimatedPartitionSizeHistogram"); From 2374a74eba6a4df84f9bda3fd311916c820e9cd6 Mon Sep 17 00:00:00 2001 From: Mick Semb Wever Date: Sun, 4 Aug 2019 20:31:05 +0200 Subject: [PATCH 45/78] Fix NPE when using allocate_tokens_for_keyspace on new DC/rack patch by Jaydeepkumar Chovatia; reviewed by Mick Semb Wever for CASSANDRA-14592 --- CHANGES.txt | 1 + .../cassandra/dht/tokenallocator/TokenAllocation.java | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index c07457b66763..e4f4d22d5873 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Fix NPE when using allocate_tokens_for_keyspace on new DC/rack (CASSANDRA-14592) * Filter sstables earlier when running cleanup (CASSANDRA-15100) * Use mean row count instead of mean column count for index selectivity calculation (CASSANDRA-15259) * Avoid updating unchanged gossip states (CASSANDRA-15097) diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java index 971a1207922d..5501378c60e0 100644 --- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java +++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java @@ -199,7 +199,11 @@ static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final Netw final int replicas = rs.getReplicationFactor(dc); Topology topology = tokenMetadata.getTopology(); - int racks = topology.getDatacenterRacks().get(dc).asMap().size(); + + // if topology hasn't been setup yet for this endpoint+rack then treat it as a separate unit + int racks = topology.getDatacenterRacks().get(dc) != null && topology.getDatacenterRacks().get(dc).containsKey(snitch.getRack(endpoint)) + ? topology.getDatacenterRacks().get(dc).asMap().size() + : 1; if (racks >= replicas) { From 9a175a1697b1107fb63480fb86ffe37b02122267 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Thu, 8 Aug 2019 12:43:18 -0700 Subject: [PATCH 46/78] Improve readability of Table metrics Virtual tables units Patch by Chris Lohfink; reviewed by Jon Haddad and Benedict Elliott Smith for CASSANDRA-15194 --- CHANGES.txt | 1 + .../db/virtual/AbstractVirtualTable.java | 2 +- .../cassandra/db/virtual/SimpleDataSet.java | 7 + .../db/virtual/TableMetricTables.java | 246 ++++++++++++------ 4 files changed, 180 insertions(+), 76 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index fb246ff8c443..389569b1954a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Improve readability of Table metrics Virtual tables units (CASSANDRA-15194) * Fix error with non-existent table for nodetool tablehistograms (CASSANDRA-14410) * Catch non-IOException in FileUtils.close to make sure that all resources are closed (CASSANDRA-15225) * Align load column in nodetool status output (CASSANDRA-14787) diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java index 2998b779e263..6c49b9a1b00a 100644 --- a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java @@ -42,7 +42,7 @@ */ public abstract class AbstractVirtualTable implements VirtualTable { - private final TableMetadata metadata; + protected final TableMetadata metadata; protected AbstractVirtualTable(TableMetadata metadata) { diff --git a/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java b/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java index bf401401d28f..6cead975c164 100644 --- a/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java +++ b/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java @@ -73,6 +73,8 @@ public SimpleDataSet column(String columnName, Object value) { if (null == currentRow) throw new IllegalStateException(); + if (null == value || columnName == null) + throw new IllegalStateException(String.format("Invalid column: %s=%s for %s", columnName, value, currentRow)); currentRow.add(columnName, value); return this; } @@ -181,6 +183,11 @@ private org.apache.cassandra.db.rows.Row toTableRow(RegularAndStaticColumns colu return builder.build(); } + + public String toString() + { + return "Row[...:" + clustering.toString(metadata)+']'; + } } @SuppressWarnings("unchecked") diff --git a/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java b/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java index acae2d0e0ccb..4a043adf0765 100644 --- a/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java +++ b/src/java/org/apache/cassandra/db/virtual/TableMetricTables.java @@ -18,29 +18,25 @@ package org.apache.cassandra.db.virtual; +import java.math.BigDecimal; import java.util.Collection; import java.util.function.Function; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import org.apache.commons.math3.util.Precision; -import com.codahale.metrics.Counter; import com.codahale.metrics.Counting; import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; import com.codahale.metrics.Metered; import com.codahale.metrics.Metric; import com.codahale.metrics.Sampling; import com.codahale.metrics.Snapshot; -import com.codahale.metrics.Timer; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.CompositeType; import org.apache.cassandra.db.marshal.DoubleType; -import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.LongType; -import org.apache.cassandra.db.marshal.ReversedType; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.LocalPartitioner; @@ -55,13 +51,14 @@ public class TableMetricTables { private final static String KEYSPACE_NAME = "keyspace_name"; private final static String TABLE_NAME = "table_name"; - private final static String MEDIAN = "median"; + private final static String P50 = "50th"; private final static String P99 = "99th"; private final static String MAX = "max"; private final static String RATE = "per_second"; + private final static double BYTES_TO_MIB = 1.0 / (1024 * 1024); + private final static double NS_TO_MS = 0.000001; - private final static AbstractType TYPE = CompositeType.getInstance(ReversedType.getInstance(LongType.instance), - UTF8Type.instance, + private final static AbstractType TYPE = CompositeType.getInstance(UTF8Type.instance, UTF8Type.instance); private final static IPartitioner PARTITIONER = new LocalPartitioner(TYPE); @@ -71,48 +68,188 @@ public class TableMetricTables public static Collection getAll(String name) { return ImmutableList.of( - getMetricTable(name, "local_reads", t -> t.readLatency.latency), - getMetricTable(name, "local_scans", t -> t.rangeLatency.latency), - getMetricTable(name, "coordinator_reads", t -> t.coordinatorReadLatency), - getMetricTable(name, "coordinator_scans", t -> t.coordinatorScanLatency), - getMetricTable(name, "local_writes", t -> t.writeLatency.latency), - getMetricTable(name, "coordinator_writes", t -> t.coordinatorWriteLatency), - getMetricTable(name, "tombstones_scanned", t -> t.tombstoneScannedHistogram.cf), - getMetricTable(name, "live_scanned", t -> t.liveScannedHistogram.cf), - getMetricTable(name, "disk_usage", t -> t.totalDiskSpaceUsed, "disk_space"), - getMetricTable(name, "max_partition_size", t -> t.maxPartitionSize, "max_partition_size")); + new LatencyTableMetric(name, "local_read_latency", t -> t.readLatency.latency), + new LatencyTableMetric(name, "local_scan_latency", t -> t.rangeLatency.latency), + new LatencyTableMetric(name, "coordinator_read_latency", t -> t.coordinatorReadLatency), + new LatencyTableMetric(name, "coordinator_scan_latency", t -> t.coordinatorScanLatency), + new LatencyTableMetric(name, "local_write_latency", t -> t.writeLatency.latency), + new LatencyTableMetric(name, "coordinator_write_latency", t -> t.coordinatorWriteLatency), + new HistogramTableMetric(name, "tombstones_per_read", t -> t.tombstoneScannedHistogram.cf), + new HistogramTableMetric(name, "rows_per_read", t -> t.liveScannedHistogram.cf), + new StorageTableMetric(name, "disk_usage", (TableMetrics t) -> t.totalDiskSpaceUsed), + new StorageTableMetric(name, "max_partition_size", (TableMetrics t) -> t.maxPartitionSize)); } - public static VirtualTable getMetricTable(String keyspace, String table, Function func) + /** + * A table that describes a some amount of disk on space in a Counter or Gauge + */ + private static class StorageTableMetric extends TableMetricTable + { + interface GaugeFunction extends Function> {} + interface CountingFunction extends Function {} + + StorageTableMetric(String keyspace, String table, CountingFunction func) + { + super(keyspace, table, func, "mebibytes", LongType.instance, ""); + } + + StorageTableMetric(String keyspace, String table, GaugeFunction func) + { + super(keyspace, table, func, "mebibytes", LongType.instance, ""); + } + + /** + * Convert bytes to mebibytes, always round up to nearest MiB + */ + public void add(SimpleDataSet result, String column, long value) + { + result.column(column, (long) Math.ceil(value * BYTES_TO_MIB)); + } + } + + /** + * A table that describes a Latency metric, specifically a Timer + */ + private static class HistogramTableMetric extends TableMetricTable + { + HistogramTableMetric(String keyspace, String table, Function func) + { + this(keyspace, table, func, ""); + } + + HistogramTableMetric(String keyspace, String table, Function func, String suffix) + { + super(keyspace, table, func, "count", LongType.instance, suffix); + } + + /** + * When displaying in cqlsh if we allow doubles to be too precise we get scientific notation which is hard to + * read so round off at 0.000. + */ + public void add(SimpleDataSet result, String column, double value) + { + result.column(column, Precision.round(value, 3, BigDecimal.ROUND_HALF_UP)); + } + } + + /** + * A table that describes a Latency metric, specifically a Timer + */ + private static class LatencyTableMetric extends HistogramTableMetric + { + LatencyTableMetric(String keyspace, String table, Function func) + { + super(keyspace, table, func, "_ms"); + } + + /** + * For the metrics that are time based, convert to to milliseconds + */ + public void add(SimpleDataSet result, String column, double value) + { + if (column.endsWith(suffix)) + value *= NS_TO_MS; + + super.add(result, column, value); + } + } + + /** + * Abstraction over the Metrics Gauge, Counter, and Timer that will turn it into a (keyspace_name, table_name) + * table. + */ + private static class TableMetricTable extends AbstractVirtualTable { - return getMetricTable(keyspace, table, func, "count"); + final Function func; + final String columnName; + final String suffix; + + TableMetricTable(String keyspace, String table, Function func, + String colName, AbstractType colType, String suffix) + { + super(buildMetadata(keyspace, table, func, colName, colType, suffix)); + this.func = func; + this.columnName = colName; + this.suffix = suffix; + } + + public void add(SimpleDataSet result, String column, double value) + { + result.column(column, value); + } + + public void add(SimpleDataSet result, String column, long value) + { + result.column(column, value); + } + + public DataSet data() + { + SimpleDataSet result = new SimpleDataSet(metadata()); + + // Iterate over all tables and get metric by function + for (ColumnFamilyStore cfs : ColumnFamilyStore.all()) + { + Metric metric = func.apply(cfs.metric); + + // set new partition for this table + result.row(cfs.keyspace.getName(), cfs.name); + + // extract information by metric type and put it in row based on implementation of `add` + if (metric instanceof Counting) + { + add(result, columnName, ((Counting) metric).getCount()); + if (metric instanceof Sampling) + { + Sampling histo = (Sampling) metric; + Snapshot snapshot = histo.getSnapshot(); + // EstimatedHistogram keeping them in ns is hard to parse as a human so convert to ms + add(result, P50 + suffix, snapshot.getMedian()); + add(result, P99 + suffix, snapshot.get99thPercentile()); + add(result, MAX + suffix, (double) snapshot.getMax()); + } + if (metric instanceof Metered) + { + Metered timer = (Metered) metric; + add(result, RATE, timer.getFiveMinuteRate()); + } + } + else if (metric instanceof Gauge) + { + add(result, columnName, (long) ((Gauge) metric).getValue()); + } + } + return result; + } } /** - * Abstraction over the Metrics Gauge, Counter, and Timer that will turn it into a ([pk], keyspace_name, table_name) - * table. The primary key (default 'count') is in descending orde in order to visually sort the rows when selecting - * the entire table in CQLSH. + * Identify the type of Metric it is (gauge, counter etc) abd create the TableMetadata. The column name + * and type for a counter/gauge is formatted differently based on the units (bytes/time) so allowed to + * be set. */ - public static VirtualTable getMetricTable(String keyspace, String table, Function func, String pk) + private static TableMetadata buildMetadata(String keyspace, String table, Function func, + String colName, AbstractType colType, String suffix) { TableMetadata.Builder metadata = TableMetadata.builder(keyspace, table) .kind(TableMetadata.Kind.VIRTUAL) - .addPartitionKeyColumn(pk, ReversedType.getInstance(LongType.instance)) .addPartitionKeyColumn(KEYSPACE_NAME, UTF8Type.instance) .addPartitionKeyColumn(TABLE_NAME, UTF8Type.instance) .partitioner(PARTITIONER); + // get a table from system keyspace and get metric from it for determining type of metric Keyspace system = Keyspace.system().iterator().next(); - - // Identify the type of Metric it is (gauge, counter etc) and verify the types work Metric test = func.apply(system.getColumnFamilyStores().iterator().next().metric); - if(test instanceof Counting) + + if (test instanceof Counting) { + metadata.addRegularColumn(colName, colType); + // if it has a Histogram include some information about distribution if (test instanceof Sampling) { - metadata.addRegularColumn(MEDIAN, LongType.instance) - .addRegularColumn(P99, LongType.instance) - .addRegularColumn(MAX, LongType.instance); + metadata.addRegularColumn(P50 + suffix, DoubleType.instance) + .addRegularColumn(P99 + suffix, DoubleType.instance) + .addRegularColumn(MAX + suffix, DoubleType.instance); } if (test instanceof Metered) { @@ -121,49 +258,8 @@ public static VirtualTable getMetricTable(String keyspace, String table, Functio } else if (test instanceof Gauge) { - Preconditions.checkArgument(((Gauge) test).getValue().getClass().isAssignableFrom(Long.class)); + metadata.addRegularColumn(colName, colType); } - - // Create the VirtualTable that will walk through all tables and get the Metric for each to build the tables - // SimpleDataSet - return new AbstractVirtualTable(metadata.build()) - { - public DataSet data() - { - SimpleDataSet result = new SimpleDataSet(metadata()); - for (ColumnFamilyStore cfs : ColumnFamilyStore.all()) - { - Metric metric = func.apply(cfs.metric); - - if(metric instanceof Counting) - { - Counting counting = (Counting) metric; - result.row(counting.getCount(), cfs.keyspace.getName(), cfs.name); - if (metric instanceof Sampling) - { - Sampling histo = (Sampling) metric; - Snapshot snapshot = histo.getSnapshot(); - result.column(MEDIAN, (long) snapshot.getMedian()) - .column(P99, (long) snapshot.get99thPercentile()) - .column(MAX, (long) snapshot.getMax()); - } - if (metric instanceof Metered) - { - Metered timer = (Metered) metric; - result.column(RATE, timer.getFiveMinuteRate()); - } - } - else if (metric instanceof Gauge) - { - result.row(((Gauge) metric).getValue(), cfs.keyspace.getName(), cfs.name); - } - else if (metric instanceof Counter) - { - result.row(((Counter) metric).getCount(), cfs.keyspace.getName(), cfs.name); - } - } - return result; - } - }; + return metadata.build(); } } From 54aeb507593dd4e3d5b8db34bc9fa6164ba504bc Mon Sep 17 00:00:00 2001 From: Mick Semb Wever Date: Mon, 12 Aug 2019 18:10:04 +0200 Subject: [PATCH 47/78] ninja fix CHANGES.txt for #14952 --- CHANGES.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index e4f4d22d5873..41ddef6dd619 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,5 @@ 3.0.19 - * Fix NPE when using allocate_tokens_for_keyspace on new DC/rack (CASSANDRA-14592) + * Fix NPE when using allocate_tokens_for_keyspace on new DC/rack (CASSANDRA-14952) * Filter sstables earlier when running cleanup (CASSANDRA-15100) * Use mean row count instead of mean column count for index selectivity calculation (CASSANDRA-15259) * Avoid updating unchanged gossip states (CASSANDRA-15097) From fcb4d52403a3de893eb2813468a788b0c8fa6fc7 Mon Sep 17 00:00:00 2001 From: Chris Lohfink Date: Wed, 14 Aug 2019 14:07:40 -0500 Subject: [PATCH 48/78] Remove bad virtual table result check Patch by Chris Lohfink; Reviewed by Blake Eggleston for CASSANDRA-15279 --- src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java b/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java index 6cead975c164..00acaedc02f0 100644 --- a/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java +++ b/src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java @@ -73,7 +73,7 @@ public SimpleDataSet column(String columnName, Object value) { if (null == currentRow) throw new IllegalStateException(); - if (null == value || columnName == null) + if (null == columnName) throw new IllegalStateException(String.format("Invalid column: %s=%s for %s", columnName, value, currentRow)); currentRow.add(columnName, value); return this; From d60e7988736ed4358595e9c781b110a5bbb5f812 Mon Sep 17 00:00:00 2001 From: Liudmila Kornilova Date: Thu, 8 Aug 2019 17:36:19 +0800 Subject: [PATCH 49/78] Avoid result truncate in decimal operations add (+), subtract (-) and multiply (*) operations: * before: * precision of result used to be always 34 (see MathContext.DECIMAL128) * after: * precision (number of significant digits) of result is at most 10000. If result exceeds given precision it will be rounded using HALF_UP mode division (/) operation: * before: * precision used to be always 34 (see MathContext.DECIMAL128) * after: * expected scale is set to minimum precision (32) minus estimated position of first digit in quotient * scale should be at least as big as maximum scale of operands * scale should not be less than 32 * scale should not be bigger than 1000 * if actual quotient scale is bigger than calculated scale then result is rounded using HALF_UP mode * trailing zeros are stripped modulo (%) operation: * before: * ArithmeticException used to occur when implicit division produces number with precision bigger than 34 (see MathContext.DECIMAL128) * after: * No exception patch by Liudmila Kornilova; reviewed by Benedict Elliott Smith for CASSANDRA-15232 --- CHANGES.txt | 1 + .../cassandra/db/marshal/DecimalType.java | 29 ++++++++-- .../cql3/functions/OperationFctsTest.java | 56 +++++++++++++------ 3 files changed, 64 insertions(+), 22 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 99c021ecd4d1..a5d437849084 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,6 +1,7 @@ 4.0 * Improve readability of Table metrics Virtual tables units (CASSANDRA-15194) * Fix error with non-existent table for nodetool tablehistograms (CASSANDRA-14410) + * Avoid result truncation in decimal operations (CASSANDRA-15232) * Catch non-IOException in FileUtils.close to make sure that all resources are closed (CASSANDRA-15225) * Align load column in nodetool status output (CASSANDRA-14787) * CassandraNetworkAuthorizer uses cached roles info (CASSANDRA-15089) diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java index b98bf009cdfc..110dc0e924e6 100644 --- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java +++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java @@ -20,6 +20,7 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.math.MathContext; +import java.math.RoundingMode; import java.nio.ByteBuffer; import org.apache.cassandra.cql3.CQL3Type; @@ -34,6 +35,10 @@ public class DecimalType extends NumberType { public static final DecimalType instance = new DecimalType(); + private static final int MIN_SCALE = 32; + private static final int MIN_SIGNIFICANT_DIGITS = MIN_SCALE; + private static final int MAX_SCALE = 1000; + private static final MathContext MAX_PRECISION = new MathContext(10000); DecimalType() {super(ComparisonType.CUSTOM);} // singleton @@ -142,27 +147,41 @@ protected BigDecimal toBigDecimal(ByteBuffer value) public ByteBuffer add(NumberType leftType, ByteBuffer left, NumberType rightType, ByteBuffer right) { - return decompose(leftType.toBigDecimal(left).add(rightType.toBigDecimal(right), MathContext.DECIMAL128)); + return decompose(leftType.toBigDecimal(left).add(rightType.toBigDecimal(right), MAX_PRECISION)); } public ByteBuffer substract(NumberType leftType, ByteBuffer left, NumberType rightType, ByteBuffer right) { - return decompose(leftType.toBigDecimal(left).subtract(rightType.toBigDecimal(right), MathContext.DECIMAL128)); + return decompose(leftType.toBigDecimal(left).subtract(rightType.toBigDecimal(right), MAX_PRECISION)); } public ByteBuffer multiply(NumberType leftType, ByteBuffer left, NumberType rightType, ByteBuffer right) { - return decompose(leftType.toBigDecimal(left).multiply(rightType.toBigDecimal(right), MathContext.DECIMAL128)); + return decompose(leftType.toBigDecimal(left).multiply(rightType.toBigDecimal(right), MAX_PRECISION)); } public ByteBuffer divide(NumberType leftType, ByteBuffer left, NumberType rightType, ByteBuffer right) { - return decompose(leftType.toBigDecimal(left).divide(rightType.toBigDecimal(right), MathContext.DECIMAL128)); + BigDecimal leftOperand = leftType.toBigDecimal(left); + BigDecimal rightOperand = rightType.toBigDecimal(right); + + // Predict position of first significant digit in the quotient. + // Note: it is possible to improve prediction accuracy by comparing first significant digits in operands + // but it requires additional computations so this step is omitted + int quotientFirstDigitPos = (leftOperand.precision() - leftOperand.scale()) - (rightOperand.precision() - rightOperand.scale()); + + int scale = MIN_SIGNIFICANT_DIGITS - quotientFirstDigitPos; + scale = Math.max(scale, leftOperand.scale()); + scale = Math.max(scale, rightOperand.scale()); + scale = Math.max(scale, MIN_SCALE); + scale = Math.min(scale, MAX_SCALE); + + return decompose(leftOperand.divide(rightOperand, scale, RoundingMode.HALF_UP).stripTrailingZeros()); } public ByteBuffer mod(NumberType leftType, ByteBuffer left, NumberType rightType, ByteBuffer right) { - return decompose(leftType.toBigDecimal(left).remainder(rightType.toBigDecimal(right), MathContext.DECIMAL128)); + return decompose(leftType.toBigDecimal(left).remainder(rightType.toBigDecimal(right))); } public ByteBuffer negate(ByteBuffer input) diff --git a/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java b/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java index d27b746c1702..c8ee9352e944 100644 --- a/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java +++ b/test/unit/org/apache/cassandra/cql3/functions/OperationFctsTest.java @@ -177,35 +177,35 @@ public void testSingleOperations() throws Throwable row((short) 0, (short) 1, 1, 2L, 2.75F, 3.25, BigInteger.valueOf(3), new BigDecimal("4.25"))); assertRows(execute("SELECT a / c, b / c, c / c, d / c, e / c, f / c, g / c, h / c FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"), - row(0, 0, 1, 1L, 1.8333334F, 2.1666666666666665, BigInteger.valueOf(2), new BigDecimal("2.833333333333333333333333333333333"))); + row(0, 0, 1, 1L, 1.8333334F, 2.1666666666666665, BigInteger.valueOf(2), new BigDecimal("2.83333333333333333333333333333333"))); assertRows(execute("SELECT a / d, b / d, c / d, d / d, e / d, f / d, g / d, h / d FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"), row(0L, 0L, 0L, 1L, 1.375, 1.625, BigInteger.valueOf(1), new BigDecimal("2.125"))); assertRows(execute("SELECT a / e, b / e, c / e, d / e, e / e, f / e, g / e, h / e FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"), - row(0.18181819F, 0.36363637F, 0.54545456F, 0.7272727272727273, 1.0F, 1.1818181818181819, new BigDecimal("1.272727272727272727272727272727273"), new BigDecimal("1.545454545454545454545454545454545"))); + row(0.18181819F, 0.36363637F, 0.54545456F, 0.7272727272727273, 1.0F, 1.1818181818181819, new BigDecimal("1.27272727272727272727272727272727"), new BigDecimal("1.54545454545454545454545454545455"))); assertRows(execute("SELECT a / f, b / f, c / f, d / f, e / f, f / f, g / f, h / f FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"), - row(0.15384615384615385, 0.3076923076923077, 0.46153846153846156, 0.6153846153846154, 0.8461538461538461, 1.0, new BigDecimal("1.076923076923076923076923076923077"), new BigDecimal("1.307692307692307692307692307692308"))); + row(0.15384615384615385, 0.3076923076923077, 0.46153846153846156, 0.6153846153846154, 0.8461538461538461, 1.0, new BigDecimal("1.07692307692307692307692307692308"), new BigDecimal("1.30769230769230769230769230769231"))); assertRows(execute("SELECT a / g, b / g, c / g, d / g, e / g, f / g, g / g, h / g FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"), row(BigInteger.valueOf(0), BigInteger.valueOf(0), BigInteger.valueOf(0), BigInteger.valueOf(0), - new BigDecimal("0.7857142857142857142857142857142857"), - new BigDecimal("0.9285714285714285714285714285714286"), + new BigDecimal("0.78571428571428571428571428571429"), + new BigDecimal("0.92857142857142857142857142857143"), BigInteger.valueOf(1), - new BigDecimal("1.214285714285714285714285714285714"))); + new BigDecimal("1.21428571428571428571428571428571"))); assertRows(execute("SELECT a / h, b / h, c / h, d / h, e / h, f / h, g / h, h / h FROM %s WHERE a = 1 AND b = 2 AND c = 3 / 1"), - row(new BigDecimal("0.1176470588235294117647058823529412"), - new BigDecimal("0.2352941176470588235294117647058824"), - new BigDecimal("0.3529411764705882352941176470588235"), - new BigDecimal("0.4705882352941176470588235294117647"), - new BigDecimal("0.6470588235294117647058823529411765"), - new BigDecimal("0.7647058823529411764705882352941176"), - new BigDecimal("0.8235294117647058823529411764705882"), + row(new BigDecimal("0.11764705882352941176470588235294"), + new BigDecimal("0.23529411764705882352941176470588"), + new BigDecimal("0.35294117647058823529411764705882"), + new BigDecimal("0.47058823529411764705882352941176"), + new BigDecimal("0.64705882352941176470588235294118"), + new BigDecimal("0.76470588235294117647058823529412"), + new BigDecimal("0.82352941176470588235294117647059"), new BigDecimal("1"))); // Test modulo operations @@ -265,6 +265,16 @@ public void testSingleOperations() throws Throwable row(null, null, null, null, null, null, null, null)); } + @Test + public void testModuloWithDecimals() throws Throwable + { + createTable("CREATE TABLE %s (numerator decimal, dec_mod decimal, int_mod int, bigint_mod bigint, PRIMARY KEY((numerator, dec_mod)))"); + execute("INSERT INTO %s (numerator, dec_mod, int_mod, bigint_mod) VALUES (123456789112345678921234567893123456, 2, 2, 2)"); + + assertRows(execute("SELECT numerator %% dec_mod, numerator %% int_mod, numerator %% bigint_mod from %s"), + row(new BigDecimal("0"), new BigDecimal("0.0"), new BigDecimal("0.0"))); + } + @Test public void testSingleOperationsWithLiterals() throws Throwable { @@ -438,7 +448,7 @@ public void testSingleOperationsWithLiterals() throws Throwable row(0, 1, 1, 2L, 2.75F, 3.25, BigInteger.valueOf(3), new BigDecimal("4.25"))); assertRows(execute("SELECT a / 3, b / 3, c / 3, d / 3, e / 3, f / 3, g / 3, h / 3 FROM %s WHERE a = 1 AND b = 2"), - row(0, 0, 1, 1L, 1.8333334F, 2.1666666666666665, BigInteger.valueOf(2), new BigDecimal("2.833333333333333333333333333333333"))); + row(0, 0, 1, 1L, 1.8333334F, 2.1666666666666665, BigInteger.valueOf(2), new BigDecimal("2.83333333333333333333333333333333"))); assertRows(execute("SELECT a / " + bigInt + "," + " b / " + bigInt + "," @@ -456,10 +466,10 @@ public void testSingleOperationsWithLiterals() throws Throwable BigInteger.valueOf(7).divide(BigInteger.valueOf(bigInt)))); assertRows(execute("SELECT a / 5.5, b / 5.5, c / 5.5, d / 5.5, e / 5.5, f / 5.5, g / 5.5, h / 5.5 FROM %s WHERE a = 1 AND b = 2"), - row(0.18181818181818182, 0.36363636363636365, 0.5454545454545454, 0.7272727272727273, 1.0, 1.1818181818181819, new BigDecimal("1.272727272727272727272727272727273"), new BigDecimal("1.545454545454545454545454545454545"))); + row(0.18181818181818182, 0.36363636363636365, 0.5454545454545454, 0.7272727272727273, 1.0, 1.1818181818181819, new BigDecimal("1.27272727272727272727272727272727"), new BigDecimal("1.54545454545454545454545454545455"))); assertRows(execute("SELECT a / 6.5, b / 6.5, c / 6.5, d / 6.5, e / 6.5, f / 6.5, g / 6.5, h / 6.5 FROM %s WHERE a = 1 AND b = 2"), - row(0.15384615384615385, 0.3076923076923077, 0.46153846153846156, 0.6153846153846154, 0.8461538461538461, 1.0, new BigDecimal("1.076923076923076923076923076923077"), new BigDecimal("1.307692307692307692307692307692308"))); + row(0.15384615384615385, 0.3076923076923077, 0.46153846153846156, 0.6153846153846154, 0.8461538461538461, 1.0, new BigDecimal("1.07692307692307692307692307692308"), new BigDecimal("1.30769230769230769230769230769231"))); // Test modulo operations @@ -502,6 +512,18 @@ public void testSingleOperationsWithLiterals() throws Throwable row((byte) 1, (short) 2, 2, 1, 4, 2, 0, -1)); } + @Test + public void testDivisionWithDecimals() throws Throwable + { + createTable("CREATE TABLE %s (numerator decimal, denominator decimal, PRIMARY KEY((numerator, denominator)))"); + execute("INSERT INTO %s (numerator, denominator) VALUES (8.5, 200000000000000000000000000000000000)"); + execute("INSERT INTO %s (numerator, denominator) VALUES (10000, 3)"); + + assertRows(execute("SELECT numerator / denominator from %s"), + row(new BigDecimal("0.0000000000000000000000000000000000425")), + row(new BigDecimal("3333.33333333333333333333333333333333"))); + } + @Test public void testWithCounters() throws Throwable { @@ -663,7 +685,7 @@ public void testWithDivisionByZero() throws Throwable OperationExecutionException.class, "SELECT g / a FROM %s WHERE a = 0 AND b = 2"); - assertInvalidThrowMessage("the operation 'decimal / tinyint' failed: Division by zero", + assertInvalidThrowMessage("the operation 'decimal / tinyint' failed: BigInteger divide by zero", OperationExecutionException.class, "SELECT h / a FROM %s WHERE a = 0 AND b = 2"); } From 8dcaa12baa97ce870f23ff9045f968f2fa28b2cc Mon Sep 17 00:00:00 2001 From: Jon Meredith Date: Thu, 15 Aug 2019 10:12:06 -0600 Subject: [PATCH 50/78] Allow instance class loaders to be garbage collected for inJVM dtest Backport support optional network/gossip test features, instance generations and subnet support from trunk. Backport MessageFilter and cluster builder API changes to match trunk (except for the change in namespace for Verbs). Add a test for repeatedly creating/tearing down in-JVM dtest clusters to help find resource leaks. Change IsolatedExecutor to clean up on an executor with zero core threads so that it exits sooner enabling GC on the InstanceClassLoader. Moved classloader close after shutdown to prevent improve logging when the isolated executor is shutdown. Update the logback config for dtests to make it obvious which log threads are for instances vs the main logger. Disable native library loading until it can be reinstated (tracked in CASSANDRA-15170). Shutdown various executors and threads that were preventing the instance classloader from being unloaded. Add test-jvm-dtest-some ant target Patch by Jon Meredith; Reviewed by Alex Petrov and Benedict Elliott Smith for CASSANDRA-15170 --- CHANGES.txt | 1 + build.xml | 22 +- .../concurrent/InfiniteLoopExecutor.java | 2 +- .../concurrent/ScheduledExecutors.java | 11 +- .../concurrent/SharedExecutorPool.java | 4 +- .../cassandra/concurrent/StageManager.java | 8 +- .../apache/cassandra/db/BatchlogManager.java | 11 +- .../cassandra/db/ColumnFamilyStore.java | 14 +- .../cassandra/db/HintedHandOffManager.java | 9 + .../cassandra/db/commitlog/CommitLog.java | 1 + .../org/apache/cassandra/gms/Gossiper.java | 9 + .../io/sstable/IndexSummaryManager.java | 11 + .../io/sstable/format/SSTableReader.java | 5 +- .../cassandra/net/MessagingService.java | 7 +- .../cassandra/net/OutboundTcpConnection.java | 2 +- .../cassandra/service/CassandraDaemon.java | 3 +- .../PendingRangeCalculatorService.java | 11 +- .../cassandra/service/StorageService.java | 29 ++- .../streaming/StreamCoordinator.java | 13 ++ .../utils/BackgroundActivityMonitor.java | 12 +- .../org/apache/cassandra/utils/CLibrary.java | 9 +- .../apache/cassandra/utils/ExecutorUtils.java | 151 +++++++++++++ .../utils/NanoTimeToCurrentTimeMillis.java | 14 +- .../cassandra/utils/concurrent/Ref.java | 8 +- .../cassandra/utils/memory/MemtablePool.java | 8 +- test/conf/logback-dtest.xml | 20 +- .../apache/cassandra/distributed/Cluster.java | 20 +- .../distributed/UpgradeableCluster.java | 22 +- .../cassandra/distributed/api/Feature.java | 24 +++ .../cassandra/distributed/api/IInstance.java | 2 + .../distributed/api/IInstanceConfig.java | 1 + .../distributed/api/IMessageFilters.java | 6 +- .../distributed/impl/AbstractCluster.java | 138 ++++++++---- .../cassandra/distributed/impl/Instance.java | 165 +++++++++++--- .../distributed/impl/InstanceClassLoader.java | 9 +- .../distributed/impl/InstanceConfig.java | 28 ++- .../distributed/impl/IsolatedExecutor.java | 47 +++- .../distributed/impl/MessageFilters.java | 31 +-- .../distributed/test/DistributedTestBase.java | 28 +++ .../distributed/test/ResourceLeakTest.java | 201 ++++++++++++++++++ .../cassandra/concurrent/SEPExecutorTest.java | 3 +- 41 files changed, 915 insertions(+), 205 deletions(-) create mode 100644 src/java/org/apache/cassandra/utils/ExecutorUtils.java create mode 100644 test/distributed/org/apache/cassandra/distributed/api/Feature.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java diff --git a/CHANGES.txt b/CHANGES.txt index eabdcaa04216..caea0f4ae746 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -8,6 +8,7 @@ * Fixing invalid CQL in security documentation (CASSANDRA-15020) * Make tools/bin/token-generator py2/3 compatible (CASSANDRA-15012) * Multi-version in-JVM dtests (CASSANDRA-14937) + * Allow instance class loaders to be garbage collected for inJVM dtest (CASSANDRA-15170) 2.2.14 diff --git a/build.xml b/build.xml index d522b595a451..b42c3ed3ca83 100644 --- a/build.xml +++ b/build.xml @@ -1818,6 +1818,10 @@ + + + + @@ -1851,10 +1855,24 @@ - + - + + + + + + + + + + + + + diff --git a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java index 473edb7b183d..af415139084d 100644 --- a/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java +++ b/src/java/org/apache/cassandra/concurrent/InfiniteLoopExecutor.java @@ -76,7 +76,7 @@ public InfiniteLoopExecutor start() return this; } - public void shutdown() + public void shutdownNow() { isShutdown = true; thread.interrupt(); diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java index 489f58e32d94..13d27a8616d6 100644 --- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java +++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java @@ -19,9 +19,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import com.google.common.annotations.VisibleForTesting; +import org.apache.cassandra.utils.ExecutorUtils; + /** * Centralized location for shared executors */ @@ -43,12 +46,8 @@ public class ScheduledExecutors public static final DebuggableScheduledThreadPoolExecutor optionalTasks = new DebuggableScheduledThreadPoolExecutor("OptionalTasks"); @VisibleForTesting - public static void shutdownAndWait() throws InterruptedException + public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - ExecutorService[] executors = new ExecutorService[] { scheduledTasks, nonPeriodicTasks, optionalTasks }; - for (ExecutorService executor : executors) - executor.shutdownNow(); - for (ExecutorService executor : executors) - executor.awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.shutdownNowAndWait(timeout, unit, scheduledTasks, nonPeriodicTasks, optionalTasks); } } diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java index d355d77b4b7a..50cc5a3628b8 100644 --- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java +++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java @@ -114,7 +114,7 @@ public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTa return executor; } - public void shutdown() throws InterruptedException + public void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException { shuttingDown = true; for (SEPExecutor executor : executors) @@ -122,7 +122,7 @@ public void shutdown() throws InterruptedException terminateWorkers(); - long until = System.nanoTime() + TimeUnit.MINUTES.toNanos(1L); + long until = System.nanoTime() + unit.toNanos(timeout); for (SEPExecutor executor : executors) executor.shutdown.await(until - System.nanoTime(), TimeUnit.NANOSECONDS); } diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java index 5e0a66770cb6..01bfb7cf12bc 100644 --- a/src/java/org/apache/cassandra/concurrent/StageManager.java +++ b/src/java/org/apache/cassandra/concurrent/StageManager.java @@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; import static org.apache.cassandra.config.DatabaseDescriptor.*; @@ -121,12 +122,9 @@ public void run() }; @VisibleForTesting - public static void shutdownAndWait() throws InterruptedException + public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - for (Stage stage : Stage.values()) - StageManager.stages.get(stage).shutdown(); - for (Stage stage : Stage.values()) - StageManager.stages.get(stage).awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.shutdownNowAndWait(timeout, unit, StageManager.stages.values()); } /** diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java index 9a2d1f69a99e..40f8ce01ec28 100644 --- a/src/java/org/apache/cassandra/db/BatchlogManager.java +++ b/src/java/org/apache/cassandra/db/BatchlogManager.java @@ -19,13 +19,11 @@ import java.io.DataInputStream; import java.io.IOException; -import java.lang.management.ManagementFactory; import java.net.InetAddress; import java.nio.ByteBuffer; import java.util.*; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicLong; -import javax.management.MBeanServer; import javax.management.ObjectName; import com.google.common.annotations.VisibleForTesting; @@ -54,7 +52,9 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.WriteResponseHandler; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.WrappedRunnable; import static org.apache.cassandra.cql3.QueryProcessor.executeInternal; @@ -74,7 +74,7 @@ public class BatchlogManager implements BatchlogManagerMBean public void start() { - MBeanServer mbs = ManagementFactory.getPlatformMBeanServer(); + MBeanWrapper mbs = MBeanWrapper.instance; try { mbs.registerMBean(this, new ObjectName(MBEAN_NAME)); @@ -95,10 +95,9 @@ public void runMayThrow() throws ExecutionException, InterruptedException batchlogTasks.scheduleWithFixedDelay(runnable, StorageService.RING_DELAY, REPLAY_INTERVAL, TimeUnit.MILLISECONDS); } - public static void shutdown() throws InterruptedException + public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - batchlogTasks.shutdown(); - batchlogTasks.awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.shutdownAndWait(timeout, unit, batchlogTasks); } public int countAllBatches() diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index d26cd610bf0f..01330a6ff040 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -88,6 +88,8 @@ import com.clearspring.analytics.stream.Counter; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; +import static org.apache.cassandra.utils.ExecutorUtils.shutdown; import static org.apache.cassandra.utils.Throwables.maybeFail; public class ColumnFamilyStore implements ColumnFamilyStoreMBean @@ -192,24 +194,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean public volatile long sampleLatencyNanos; private final ScheduledFuture latencyCalculator; - public static void shutdownFlushExecutor() throws InterruptedException - { - flushExecutor.shutdown(); - flushExecutor.awaitTermination(60, TimeUnit.SECONDS); - } - public static void shutdownPostFlushExecutor() throws InterruptedException { postFlushExecutor.shutdown(); postFlushExecutor.awaitTermination(60, TimeUnit.SECONDS); } - public static void shutdownReclaimExecutor() throws InterruptedException + public static void shutdownExecutorsAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - reclaimExecutor.shutdown(); - reclaimExecutor.awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.shutdownAndWait(timeout, unit, reclaimExecutor, postFlushExecutor, flushExecutor); } + public void reload() { // metadata object has been mutated directly. make all the members jibe with new settings. diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java index 95af9baa55f8..7a570d2c9341 100644 --- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java +++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java @@ -68,6 +68,9 @@ import org.apache.cassandra.utils.MBeanWrapper; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; +import static org.apache.cassandra.utils.ExecutorUtils.shutdown; + /** * The hint schema looks like this: * @@ -619,4 +622,10 @@ private List getHintsSlice(int columnCount) } } + @VisibleForTesting + public void shutdownAndWait(long timeout, TimeUnit units) throws InterruptedException, TimeoutException + { + shutdown(executor, hintDeliveryExecutor); + awaitTermination(timeout, units, executor, hintDeliveryExecutor); + } } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java index 2f0179d1ab56..6dd519a01587 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java @@ -382,6 +382,7 @@ public Map getActiveSegmentCompressionRatios() /** * Shuts down the threads used by the commit log, blocking until completion. + * TODO this should accept a timeout, and throw TimeoutException */ public void shutdownBlocking() throws InterruptedException { diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java index 831c252254c0..bd4fe1369652 100644 --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@ -29,6 +29,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.Uninterruptibles; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.Pair; import org.slf4j.Logger; @@ -47,6 +48,8 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; +import static org.apache.cassandra.utils.ExecutorUtils.shutdown; /** * This module is responsible for Gossiping information for the local endpoint. This abstraction @@ -1547,4 +1550,10 @@ public static long computeExpireTime() return System.currentTimeMillis() + Gossiper.aVeryLongTime; } + @VisibleForTesting + public void stopShutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException + { + stop(); + ExecutorUtils.shutdownAndWait(timeout, unit, executor); + } } diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java index 3ebbb6edc362..931713226076 100644 --- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java +++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java @@ -26,6 +26,7 @@ import java.util.UUID; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; @@ -43,11 +44,15 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.db.lifecycle.View; import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.MBeanWrapper; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.WrappedRunnable; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; +import static org.apache.cassandra.utils.ExecutorUtils.shutdown; + /** * Manages the fixed-size memory pool for index summaries, periodically resizing them * in order to give more memory to hot sstables and less memory to cold sstables. @@ -257,4 +262,10 @@ public static List redistributeSummaries(IndexSummaryRedistributi { return CompactionManager.instance.runIndexSummaryRedistribution(redistribution); } + + @VisibleForTesting + public void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException + { + ExecutorUtils.shutdownAndWait(timeout, unit, executor); + } } diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 81af9f00d008..16fa6c9deba5 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -2326,10 +2326,9 @@ public abstract SSTableReader open(final Descriptor descriptor, } - public static void shutdownBlocking() throws InterruptedException + public static void shutdownBlocking(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - syncExecutor.shutdownNow(); - syncExecutor.awaitTermination(0, TimeUnit.SECONDS); + ExecutorUtils.shutdownNowAndWait(timeout, unit, syncExecutor); resetTidying(); } } diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java index e7ce964d5fa1..e42b91b6a369 100644 --- a/src/java/org/apache/cassandra/net/MessagingService.java +++ b/src/java/org/apache/cassandra/net/MessagingService.java @@ -775,6 +775,8 @@ public void shutdown() // see https://issues.apache.org/jira/browse/CASSANDRA-10545 handleIOException(e); } + + connectionManagers.values().forEach(OutboundTcpConnectionPool::close); } catch (IOException e) { @@ -1063,7 +1065,10 @@ private static void handleIOException(IOException e) throws IOException { // dirty hack for clean shutdown on OSX w/ Java >= 1.8.0_20 // see https://bugs.openjdk.java.net/browse/JDK-8050499 - if (!"Unknown error: 316".equals(e.getMessage()) || !"Mac OS X".equals(System.getProperty("os.name"))) + if ((!"Unknown error: 316".equals(e.getMessage()) || !"Mac OS X".equals(System.getProperty("os.name"))) && + !"Thread signal failed".equals(e.getMessage()) && // handle shutdown for in-JVM dtests + !"Bad file descriptor".equals(e.getMessage()) && + !"No such file or directory".equals(e.getMessage())) throw e; } diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java index 4cfe0196ce80..e8346d86fd1b 100644 --- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java +++ b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java @@ -392,7 +392,7 @@ private boolean connect() long start = System.nanoTime(); long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getRpcTimeout()); - while (System.nanoTime() - start < timeout) + while (System.nanoTime() - start < timeout && !isStopped) { targetVersion = MessagingService.instance().getVersion(poolReference.endPoint()); try diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 8f6c9c21f902..1380f432dba0 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -601,7 +601,8 @@ public void deactivate() } } - private void waitForGossipToSettle() + @VisibleForTesting + public static void waitForGossipToSettle() { int forceAfter = Integer.getInteger("cassandra.skip_wait_for_gossip_to_settle", -1); if (forceAfter == 0) diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java index e82b0bbd1529..a7ee333a9407 100644 --- a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java +++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java @@ -23,15 +23,21 @@ import org.apache.cassandra.config.Schema; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.utils.ExecutorUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Arrays; import java.util.List; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import com.google.common.annotations.VisibleForTesting; +import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; +import static org.apache.cassandra.utils.ExecutorUtils.shutdownNow; + public class PendingRangeCalculatorService { public static final PendingRangeCalculatorService instance = new PendingRangeCalculatorService(); @@ -112,9 +118,8 @@ public static void calculatePendingRanges(AbstractReplicationStrategy strategy, } @VisibleForTesting - public void shutdownExecutor() throws InterruptedException + public void shutdownExecutor(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - executor.shutdown(); - executor.awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.shutdownNowAndWait(timeout, unit, executor); } } diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index c5c1ca6054a2..0a9a8da98629 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -22,6 +22,7 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.nio.ByteBuffer; +import java.sql.Time; import java.util.*; import java.util.Map.Entry; import java.util.concurrent.*; @@ -79,6 +80,8 @@ import org.apache.cassandra.utils.progress.jmx.JMXProgressSupport; import org.apache.cassandra.utils.progress.jmx.LegacyJMXProgressSupport; +import static java.util.concurrent.TimeUnit.MINUTES; + /** * This abstraction contains the token/identifier of this node * on the identifier space. This token gets gossiped around. @@ -659,7 +662,7 @@ public void runMayThrow() throws InterruptedException // wait for miscellaneous tasks like sstable and commitlog segment deletion ScheduledExecutors.nonPeriodicTasks.shutdown(); - if (!ScheduledExecutors.nonPeriodicTasks.awaitTermination(1, TimeUnit.MINUTES)) + if (!ScheduledExecutors.nonPeriodicTasks.awaitTermination(1, MINUTES)) logger.warn("Miscellaneous task executor still busy after one minute; proceeding with shutdown"); } }, "StorageServiceShutdownHook"); @@ -1365,9 +1368,9 @@ public double getSeverity(InetAddress endpoint) return bgMonitor.getSeverity(endpoint); } - public void shutdownBGMonitor() + public void shutdownBGMonitorAndWait(long timeout, TimeUnit units) throws TimeoutException, InterruptedException { - bgMonitor.shutdown(); + bgMonitor.shutdownAndWait(timeout, units); } /** @@ -4067,7 +4070,15 @@ public synchronized void drain() throws IOException, InterruptedException, Execu remainingCFs--; } - BatchlogManager.shutdown(); + try + { + /* not clear this is reasonable time, but propagated from prior embedded behaviour */ + BatchlogManager.shutdownAndWait(1L, MINUTES); + } + catch (TimeoutException t) + { + logger.error("Batchlog manager timed out shutting down", t); + } // Interrupt on going compaction and shutdown to prevent further compaction CompactionManager.instance.forceShutdown(); @@ -4093,7 +4104,7 @@ public synchronized void drain() throws IOException, InterruptedException, Execu // wait for miscellaneous tasks like sstable and commitlog segment deletion ScheduledExecutors.nonPeriodicTasks.shutdown(); - if (!ScheduledExecutors.nonPeriodicTasks.awaitTermination(1, TimeUnit.MINUTES)) + if (!ScheduledExecutors.nonPeriodicTasks.awaitTermination(1, MINUTES)) logger.warn("Failed to wait for non periodic tasks to shutdown"); ColumnFamilyStore.shutdownPostFlushExecutor(); @@ -4551,4 +4562,12 @@ public void setHintedHandoffThrottleInKB(int throttleInKB) logger.info(String.format("Updated hinted_handoff_throttle_in_kb to %d", throttleInKB)); } + @VisibleForTesting + public void shutdownServer() + { + if (drainOnShutdown != null) + { + Runtime.getRuntime().removeShutdownHook(drainOnShutdown); + } + } } diff --git a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java index 603366d14a8f..e0948c996475 100644 --- a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java +++ b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java @@ -19,11 +19,17 @@ import java.net.InetAddress; import java.util.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; /** @@ -290,4 +296,11 @@ public Collection getAllSessionInfo() return sessionInfos.values(); } } + + @VisibleForTesting + public static void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException + { + ExecutorUtils.shutdownAndWait(timeout, unit, streamExecutor); + } + } diff --git a/src/java/org/apache/cassandra/utils/BackgroundActivityMonitor.java b/src/java/org/apache/cassandra/utils/BackgroundActivityMonitor.java index ab81c200d583..711c5dd0ca1d 100644 --- a/src/java/org/apache/cassandra/utils/BackgroundActivityMonitor.java +++ b/src/java/org/apache/cassandra/utils/BackgroundActivityMonitor.java @@ -28,6 +28,7 @@ import java.util.StringTokenizer; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor; import org.apache.cassandra.gms.ApplicationState; @@ -143,16 +144,9 @@ public double getSeverity(InetAddress endpoint) return 0.0; } - public void shutdown() + public void shutdownAndWait(long timeout, TimeUnit unit) throws TimeoutException, InterruptedException { - reportThread.shutdown(); - try - { - reportThread.awaitTermination(1L, TimeUnit.MINUTES); - } catch (InterruptedException e) - { - throw new IllegalStateException(); - } + ExecutorUtils.shutdownAndWait(timeout, unit, reportThread); } public class BackgroundActivityReporter implements Runnable diff --git a/src/java/org/apache/cassandra/utils/CLibrary.java b/src/java/org/apache/cassandra/utils/CLibrary.java index b6598ec4199e..e3bec4f68df9 100644 --- a/src/java/org/apache/cassandra/utils/CLibrary.java +++ b/src/java/org/apache/cassandra/utils/CLibrary.java @@ -60,7 +60,14 @@ public final class CLibrary { try { - Native.register("c"); + if (Boolean.getBoolean("cassandra.disable_clibrary")) + { + jnaAvailable = false; + } + else + { + Native.register("c"); + } } catch (NoClassDefFoundError e) { diff --git a/src/java/org/apache/cassandra/utils/ExecutorUtils.java b/src/java/org/apache/cassandra/utils/ExecutorUtils.java new file mode 100644 index 000000000000..21933a312360 --- /dev/null +++ b/src/java/org/apache/cassandra/utils/ExecutorUtils.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.utils; + +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import org.apache.cassandra.concurrent.InfiniteLoopExecutor; + +import static java.util.concurrent.TimeUnit.NANOSECONDS; + +public class ExecutorUtils +{ + + public static Runnable runWithThreadName(Runnable runnable, String threadName) + { + return () -> { + String oldThreadName = Thread.currentThread().getName(); + try + { + Thread.currentThread().setName(threadName); + runnable.run(); + } + finally + { + Thread.currentThread().setName(oldThreadName); + } + }; + } + + public static void shutdownNow(Iterable executors) + { + shutdown(true, executors); + } + + public static void shutdown(Iterable executors) + { + shutdown(false, executors); + } + + public static void shutdown(boolean interrupt, Iterable executors) + { + for (Object executor : executors) + { + if (executor instanceof ExecutorService) + { + if (interrupt) ((ExecutorService) executor).shutdownNow(); + else ((ExecutorService) executor).shutdown(); + } + else if (executor instanceof InfiniteLoopExecutor) + ((InfiniteLoopExecutor) executor).shutdownNow(); + else if (executor instanceof Thread) + ((Thread) executor).interrupt(); + else if (executor != null) + throw new IllegalArgumentException(executor.toString()); + } + } + + public static void shutdown(ExecutorService ... executors) + { + shutdown(Arrays.asList(executors)); + } + + public static void shutdownNow(ExecutorService ... executors) + { + shutdownNow(Arrays.asList(executors)); + } + + public static void awaitTermination(long timeout, TimeUnit unit, ExecutorService ... executors) throws InterruptedException, TimeoutException + { + awaitTermination(timeout, unit, Arrays.asList(executors)); + } + + public static void awaitTermination(long timeout, TimeUnit unit, Collection executors) throws InterruptedException, TimeoutException + { + long deadline = System.nanoTime() + unit.toNanos(timeout); + awaitTerminationUntil(deadline, executors); + } + + public static void awaitTerminationUntil(long deadline, Collection executors) throws InterruptedException, TimeoutException + { + for (Object executor : executors) + { + long wait = deadline - System.nanoTime(); + if (executor instanceof ExecutorService) + { + if (wait <= 0 || !((ExecutorService)executor).awaitTermination(wait, NANOSECONDS)) + throw new TimeoutException(executor + " did not terminate on time"); + } + else if (executor instanceof InfiniteLoopExecutor) + { + if (wait <= 0 || !((InfiniteLoopExecutor)executor).awaitTermination(wait, NANOSECONDS)) + throw new TimeoutException(executor + " did not terminate on time"); + } + else if (executor instanceof Thread) + { + Thread t = (Thread) executor; + if (wait <= 0) + throw new TimeoutException(executor + " did not terminate on time"); + t.join((wait + 999999) / 1000000L, (int) (wait % 1000000L)); + if (t.isAlive()) + throw new TimeoutException(executor + " did not terminate on time"); + } + else if (executor != null) + { + throw new IllegalArgumentException(executor.toString()); + } + } + } + + public static void shutdownAndWait(long timeout, TimeUnit unit, Collection executors) throws TimeoutException, InterruptedException + { + shutdown(executors); + awaitTermination(timeout, unit, executors); + } + + public static void shutdownNowAndWait(long timeout, TimeUnit unit, Collection executors) throws TimeoutException, InterruptedException + { + shutdownNow(executors); + awaitTermination(timeout, unit, executors); + } + + public static void shutdownAndWait(long timeout, TimeUnit unit, Object ... executors) throws TimeoutException, InterruptedException + { + shutdownAndWait(timeout, unit, Arrays.asList(executors)); + } + + public static void shutdownNowAndWait(long timeout, TimeUnit unit, Object ... executors) throws TimeoutException, InterruptedException + { + shutdownNowAndWait(timeout, unit, Arrays.asList(executors)); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java b/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java index f124383cdd21..9d42acbd1a74 100644 --- a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java +++ b/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java @@ -39,6 +39,8 @@ public class NanoTimeToCurrentTimeMillis @VisibleForTesting public static final Object TIMESTAMP_UPDATE = new Object(); + private static final Thread updater; + /* * System.currentTimeMillis() is 25 nanoseconds. This is 2 nanoseconds (maybe) according to JMH. * Faster than calling both currentTimeMillis() and nanoTime(). @@ -57,7 +59,7 @@ public static final long convert(long nanoTime) static { //Pick up updates from NTP periodically - Thread t = new Thread("NanoTimeToCurrentTimeMillis updater") + updater = new Thread("NanoTimeToCurrentTimeMillis updater") { @Override public void run() @@ -82,7 +84,13 @@ public void run() } } }; - t.setDaemon(true); - t.start(); + updater.setDaemon(true); + updater.start(); + } + + public static void shutdown(long millis) throws InterruptedException + { + updater.interrupt(); + updater.join(millis); } } diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java index e1cc7ff6a01e..c0090328ad4c 100644 --- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java +++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java @@ -22,6 +22,7 @@ import java.lang.ref.PhantomReference; import java.lang.ref.ReferenceQueue; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Set; @@ -35,7 +36,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.concurrent.InfiniteLoopExecutor.InterruptibleRunnable; -import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.utils.ExecutorUtils; import static org.apache.cassandra.utils.Throwables.maybeFail; import static org.apache.cassandra.utils.Throwables.merge; @@ -343,9 +344,8 @@ private static void reapOneReference() throws InterruptedException } @VisibleForTesting - public static void shutdownReferenceReaper() throws InterruptedException + public static void shutdownReferenceReaper(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - EXEC.shutdown(); - EXEC.awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.shutdownNowAndWait(timeout, unit, EXEC); } } diff --git a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java index b4efaa64cf77..9c4824a08853 100644 --- a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java +++ b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java @@ -19,10 +19,12 @@ package org.apache.cassandra.utils.memory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import com.google.common.annotations.VisibleForTesting; +import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.concurrent.WaitQueue; @@ -62,12 +64,12 @@ MemtableCleanerThread getCleaner(Runnable cleaner) public abstract boolean needToCopyOnHeap(); @VisibleForTesting - public void shutdown() throws InterruptedException + public void shutdownAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { - cleaner.shutdown(); - cleaner.awaitTermination(60, TimeUnit.SECONDS); + ExecutorUtils.shutdownNowAndWait(timeout, unit, cleaner); } + public abstract MemtableAllocator newAllocator(); /** diff --git a/test/conf/logback-dtest.xml b/test/conf/logback-dtest.xml index b8899a30a5bf..4282feec4bd9 100644 --- a/test/conf/logback-dtest.xml +++ b/test/conf/logback-dtest.xml @@ -23,7 +23,7 @@ - + ./build/test/logs/${cassandra.testtag}/TEST-${suitename}.log @@ -38,18 +38,18 @@ %-5level [%thread] ${instance_id} %date{ISO8601} %msg%n - false + false - + 0 0 1024 - + - + %-5level %date{HH:mm:ss,SSS} %msg%n @@ -58,7 +58,7 @@ - + %-5level %date{HH:mm:ss,SSS} %msg%n @@ -67,7 +67,7 @@ - + %-5level [%thread] ${instance_id} %date{ISO8601} %F:%L - %msg%n @@ -79,8 +79,8 @@ - - - + + + diff --git a/test/distributed/org/apache/cassandra/distributed/Cluster.java b/test/distributed/org/apache/cassandra/distributed/Cluster.java index c7f7675cb847..95862b6a627a 100644 --- a/test/distributed/org/apache/cassandra/distributed/Cluster.java +++ b/test/distributed/org/apache/cassandra/distributed/Cluster.java @@ -20,8 +20,8 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.List; +import java.util.function.Consumer; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.impl.AbstractCluster; @@ -40,18 +40,24 @@ private Cluster(File root, Versions.Version version, List config super(root, version, configs, sharedClassLoader); } - protected IInvokableInstance newInstanceWrapper(Versions.Version version, InstanceConfig config) + protected IInvokableInstance newInstanceWrapper(int generation, Versions.Version version, InstanceConfig config) { - return new Wrapper(version, config); + return new Wrapper(generation, version, config); } - public static Cluster create(int nodeCount) throws Throwable + public static Builder build(int nodeCount) + { + return new Builder<>(nodeCount, Cluster::new); + } + + public static Cluster create(int nodeCount, Consumer configUpdater) throws IOException { - return create(nodeCount, Cluster::new); + return build(nodeCount).withConfig(configUpdater).start(); } - public static Cluster create(int nodeCount, File root) + + public static Cluster create(int nodeCount) throws Throwable { - return create(nodeCount, Versions.CURRENT, root, Cluster::new); + return build(nodeCount).start(); } } diff --git a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java index 0c8e63ae9101..232ef0b7b333 100644 --- a/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/UpgradeableCluster.java @@ -43,28 +43,24 @@ private UpgradeableCluster(File root, Versions.Version version, List build(int nodeCount) { - return create(nodeCount, Versions.CURRENT, root, UpgradeableCluster::new); + return new Builder<>(nodeCount, UpgradeableCluster::new); } - public static UpgradeableCluster create(int nodeCount, Versions.Version version) throws IOException + public static UpgradeableCluster create(int nodeCount) throws Throwable { - return create(nodeCount, version, Files.createTempDirectory("dtests").toFile(), UpgradeableCluster::new); + return build(nodeCount).start(); } - public static UpgradeableCluster create(int nodeCount, Versions.Version version, File root) + + public static UpgradeableCluster create(int nodeCount, Versions.Version version) throws Throwable { - return create(nodeCount, version, root, UpgradeableCluster::new); + return build(nodeCount).withVersion(version).start(); } - } diff --git a/test/distributed/org/apache/cassandra/distributed/api/Feature.java b/test/distributed/org/apache/cassandra/distributed/api/Feature.java new file mode 100644 index 000000000000..a5c9316930e3 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/api/Feature.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.api; + +public enum Feature +{ + NETWORK, GOSSIP +} diff --git a/test/distributed/org/apache/cassandra/distributed/api/IInstance.java b/test/distributed/org/apache/cassandra/distributed/api/IInstance.java index 3834093fbbec..d5382b4970f0 100644 --- a/test/distributed/org/apache/cassandra/distributed/api/IInstance.java +++ b/test/distributed/org/apache/cassandra/distributed/api/IInstance.java @@ -37,7 +37,9 @@ public interface IInstance extends IIsolatedExecutor UUID schemaVersion(); void startup(); + boolean isShutdown(); Future shutdown(); + Future shutdown(boolean graceful); // these methods are not for external use, but for simplicity we leave them public and on the normal IInstance interface void startup(ICluster cluster); diff --git a/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java index 6741b3fdd59b..3e5a18fe7c8c 100644 --- a/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java +++ b/test/distributed/org/apache/cassandra/distributed/api/IInstanceConfig.java @@ -38,4 +38,5 @@ public interface IInstanceConfig Object get(String fieldName); String getString(String fieldName); int getInt(String fieldName); + boolean has(Feature featureFlag); } diff --git a/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java b/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java index b5fde840e748..f7c80940a800 100644 --- a/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java +++ b/test/distributed/org/apache/cassandra/distributed/api/IMessageFilters.java @@ -18,11 +18,8 @@ package org.apache.cassandra.distributed.api; -import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.MessagingService; -import java.util.function.BiConsumer; - public interface IMessageFilters { public interface Filter @@ -44,5 +41,6 @@ public interface Builder void reset(); // internal - BiConsumer filter(BiConsumer applyIfNotFiltered); + boolean permit(IInstance from, IInstance to, int verb); + } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java index c27d9bf2d892..19fb7e57fc29 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/AbstractCluster.java @@ -30,11 +30,9 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -91,6 +89,7 @@ public abstract class AbstractCluster implements ICluster, // to ensure we have instantiated the main classloader's LoggerFactory (and any LogbackStatusListener) // before we instantiate any for a new instance private static final Logger logger = LoggerFactory.getLogger(AbstractCluster.class); + private static final AtomicInteger generation = new AtomicInteger(); private final File root; private final ClassLoader sharedClassLoader; @@ -104,6 +103,7 @@ public abstract class AbstractCluster implements ICluster, protected class Wrapper extends DelegatingInvokableInstance implements IUpgradeableInstance { + private final int generation; private final InstanceConfig config; private volatile IInvokableInstance delegate; private volatile Versions.Version version; @@ -112,21 +112,22 @@ protected class Wrapper extends DelegatingInvokableInstance implements IUpgradea protected IInvokableInstance delegate() { if (delegate == null) - delegate = newInstance(); + delegate = newInstance(generation); return delegate; } - public Wrapper(Versions.Version version, InstanceConfig config) + public Wrapper(int generation, Versions.Version version, InstanceConfig config) { + this.generation = generation; this.config = config; this.version = version; // we ensure there is always a non-null delegate, so that the executor may be used while the node is offline - this.delegate = newInstance(); + this.delegate = newInstance(generation); } - private IInvokableInstance newInstance() + private IInvokableInstance newInstance(int generation) { - ClassLoader classLoader = new InstanceClassLoader(config.num(), version.classpath, sharedClassLoader); + ClassLoader classLoader = new InstanceClassLoader(generation, version.classpath, sharedClassLoader); return Instance.transferAdhoc((SerializableBiFunction)Instance::new, classLoader) .apply(config, classLoader); } @@ -136,6 +137,11 @@ public IInstanceConfig config() return config; } + public boolean isShutdown() + { + return isShutdown; + } + @Override public synchronized void startup() { @@ -148,11 +154,17 @@ public synchronized void startup() @Override public synchronized Future shutdown() + { + return shutdown(true); + } + + @Override + public synchronized Future shutdown(boolean graceful) { if (isShutdown) throw new IllegalStateException(); isShutdown = true; - Future future = delegate.shutdown(); + Future future = delegate.shutdown(graceful); delegate = null; return future; } @@ -187,19 +199,20 @@ protected AbstractCluster(File root, Versions.Version version, List(); this.instanceMap = new HashMap<>(); + int generation = AbstractCluster.generation.incrementAndGet(); for (InstanceConfig config : configs) { - I instance = newInstanceWrapper(version, config); + I instance = newInstanceWrapper(generation, version, config); instances.add(instance); // we use the config().broadcastAddressAndPort() here because we have not initialised the Instance I prev = instanceMap.put(instance.broadcastAddressAndPort(), instance); if (null != prev) throw new IllegalStateException("Cluster cannot have multiple nodes with same InetAddressAndPort: " + instance.broadcastAddressAndPort() + " vs " + prev.broadcastAddressAndPort()); } - this.filters = new MessageFilters(this); + this.filters = new MessageFilters(); } - protected abstract I newInstanceWrapper(Versions.Version version, InstanceConfig config); + protected abstract I newInstanceWrapper(int generation, Versions.Version version, InstanceConfig config); /** * WARNING: we index from 1 here, for consistency with inet address! @@ -257,9 +270,12 @@ private void updateMessagingVersions() { for (IInstance reportTo: instances) { + if (reportTo.isShutdown()) + continue; + for (IInstance reportFrom: instances) { - if (reportFrom == reportTo) + if (reportFrom == reportTo || reportFrom.isShutdown()) continue; int minVersion = Math.min(reportFrom.getMessagingVersion(), reportTo.getMessagingVersion()); @@ -335,46 +351,83 @@ protected interface Factory> C newCluster(File root, Versions.Version version, List configs, ClassLoader sharedClassLoader); } - protected static > C - create(int nodeCount, Factory factory) throws Throwable + public static class Builder> { - return create(nodeCount, Files.createTempDirectory("dtests").toFile(), factory); - } + private final int nodeCount; + private final Factory factory; + private int subnet; + private File root; + private Versions.Version version; + private Consumer configUpdater; + public Builder(int nodeCount, Factory factory) + { + this.nodeCount = nodeCount; + this.factory = factory; + } - protected static > C - create(int nodeCount, File root, Factory factory) - { - return create(nodeCount, Versions.CURRENT, root, factory); - } + public Builder withSubnet(int subnet) + { + this.subnet = subnet; + return this; + } - protected static > C - create(int nodeCount, Versions.Version version, Factory factory) throws IOException - { - return create(nodeCount, version, Files.createTempDirectory("dtests").toFile(), factory); - } + public Builder withRoot(File root) + { + this.root = root; + return this; + } - protected static > C - create(int nodeCount, Versions.Version version, File root, Factory factory) - { - root.mkdirs(); - setupLogging(root); + public Builder withVersion(Versions.Version version) + { + this.version = version; + return this; + } - ClassLoader sharedClassLoader = Thread.currentThread().getContextClassLoader(); + public Builder withConfig(Consumer updater) + { + this.configUpdater = updater; + return this; + } - List configs = new ArrayList<>(); - long token = Long.MIN_VALUE + 1, increment = 2 * (Long.MAX_VALUE / nodeCount); - for (int i = 0 ; i < nodeCount ; ++i) + public C createWithoutStarting() throws IOException { - InstanceConfig config = InstanceConfig.generate(i + 1, root, String.valueOf(token)); - configs.add(config); - token += increment; + File root = this.root; + Versions.Version version = this.version; + + if (root == null) + root = Files.createTempDirectory("dtests").toFile(); + if (version == null) + version = Versions.CURRENT; + + root.mkdirs(); + setupLogging(root); + + ClassLoader sharedClassLoader = Thread.currentThread().getContextClassLoader(); + + List configs = new ArrayList<>(); + long token = Long.MIN_VALUE + 1, increment = 2 * (Long.MAX_VALUE / nodeCount); + for (int i = 0; i < nodeCount; ++i) + { + InstanceConfig config = InstanceConfig.generate(i + 1, subnet, root, String.valueOf(token)); + if (configUpdater != null) + configUpdater.accept(config); + configs.add(config); + token += increment; + } + + C cluster = factory.newCluster(root, version, configs, sharedClassLoader); + return cluster; } - C cluster = factory.newCluster(root, version, configs, sharedClassLoader); - cluster.startup(); - return cluster; + public C start() throws IOException + { + C cluster = createWithoutStarting(); + cluster.startup(); + return cluster; + } } + private static void setupLogging(File root) { try @@ -398,6 +451,7 @@ private static void setupLogging(File root) public void close() { FBUtilities.waitOnFutures(instances.stream() + .filter(i -> !i.isShutdown()) .map(IInstance::shutdown) .collect(Collectors.toList()), 1L, TimeUnit.MINUTES); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index dce03cab30a8..29426cbe89b5 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -35,9 +35,6 @@ import java.util.function.BiConsumer; import java.util.function.Function; -import org.slf4j.LoggerFactory; - -import ch.qos.logback.classic.LoggerContext; import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.concurrent.SharedExecutorPool; import org.apache.cassandra.concurrent.StageManager; @@ -48,7 +45,9 @@ import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.statements.ParsedStatement; +import org.apache.cassandra.db.BatchlogManager; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.HintedHandOffManager; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.Memtable; import org.apache.cassandra.db.SystemKeyspace; @@ -58,12 +57,14 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.api.ICoordinator; +import org.apache.cassandra.distributed.api.IInstance; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IListen; import org.apache.cassandra.distributed.api.IMessage; import org.apache.cassandra.gms.ApplicationState; import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.io.sstable.IndexSummaryManager; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.locator.InetAddressAndPort; @@ -76,11 +77,17 @@ import org.apache.cassandra.service.PendingRangeCalculatorService; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.streaming.StreamCoordinator; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.NanoTimeToCurrentTimeMillis; import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.Ref; +import static java.util.concurrent.TimeUnit.MINUTES; +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + public class Instance extends IsolatedExecutor implements IInvokableInstance { public final IInstanceConfig config; @@ -93,6 +100,10 @@ public class Instance extends IsolatedExecutor implements IInvokableInstance this.config = config; InstanceIDDefiner.setInstanceId(config.num()); FBUtilities.setBroadcastInetAddress(config.broadcastAddressAndPort().address); + // Set the config at instance creation, possibly before startup() has run on all other instances. + // setMessagingVersions below will call runOnInstance which will instantiate + // the MessagingService and dependencies preventing later changes to network parameters. + Config.setOverrideLoadConfig(() -> loadConfig(config)); } public IInstanceConfig config() @@ -140,6 +151,11 @@ public void startup() throw new UnsupportedOperationException(); } + public boolean isShutdown() + { + throw new UnsupportedOperationException(); + } + @Override public void schemaChangeInternal(String query) { @@ -166,7 +182,10 @@ public void schemaChangeInternal(String query) private void registerMockMessaging(ICluster cluster) { BiConsumer deliverToInstance = (to, message) -> cluster.get(to).receiveMessage(message); - BiConsumer deliverToInstanceIfNotFiltered = cluster.filters().filter(deliverToInstance); + BiConsumer deliverToInstanceIfNotFiltered = (to, message) -> { + if (cluster.filters().permit(this, cluster.get(to), message.verb())) + deliverToInstance.accept(to, message); + }; Map addressAndPortMap = new HashMap<>(); cluster.stream().forEach(instance -> { @@ -182,6 +201,26 @@ private void registerMockMessaging(ICluster cluster) new MessageDeliverySink(deliverToInstanceIfNotFiltered, addressAndPortMap::get)); } + // unnecessary if registerMockMessaging used + private void registerFilter(ICluster cluster) + { + IInstance instance = this; + MessagingService.instance().addMessageSink(new IMessageSink() + { + public boolean allowOutgoingMessage(MessageOut message, int id, InetAddress toAddress) + { + // Port is not passed in, so take a best guess at the destination port from this instance + IInstance to = cluster.get(InetAddressAndPort.getByAddressOverrideDefaults(toAddress, instance.config().broadcastAddressAndPort().port)); + return cluster.filters().permit(instance, to, message.verb.ordinal()); + } + + public boolean allowIncomingMessage(MessageIn message, int id) + { + return true; + } + }); + } + private class MessageDeliverySink implements IMessageSink { private final BiConsumer deliver; @@ -200,6 +239,11 @@ public boolean allowOutgoingMessage(MessageOut messageOut, int id, InetAddress t assert from.equals(lookupAddressAndPort.apply(messageOut.from)); InetAddressAndPort toFull = lookupAddressAndPort.apply(to); int version = MessagingService.instance().getVersion(to); + + out.writeInt(MessagingService.PROTOCOL_MAGIC); + out.writeInt(id); + long timestamp = System.currentTimeMillis(); + out.writeInt((int) timestamp); messageOut.serialize(out, version); deliver.accept(toFull, new Message(messageOut.verb.ordinal(), out.toByteArray(), id, version, from)); } @@ -217,14 +261,45 @@ public boolean allowIncomingMessage(MessageIn message, int id) } } - public void receiveMessage(IMessage message) + public void receiveMessage(IMessage imessage) { sync(() -> { - try (DataInputStream in = new DataInputStream(new ByteArrayInputStream(message.bytes()))) + // Based on org.apache.cassandra.net.IncomingTcpConnection.receiveMessage + try (DataInputStream input = new DataInputStream(new ByteArrayInputStream(imessage.bytes()))) { - MessageIn messageIn = MessageIn.read(in, message.version(), message.id()); - Runnable deliver = new MessageDeliveryTask(messageIn, message.id(), System.currentTimeMillis(), false); - deliver.run(); + int version = imessage.version(); + + MessagingService.validateMagic(input.readInt()); + int id; + if (version < MessagingService.VERSION_20) + id = Integer.parseInt(input.readUTF()); + else + id = input.readInt(); + assert imessage.id() == id; + + long timestamp = System.currentTimeMillis(); + boolean isCrossNodeTimestamp = false; + + // make sure to readInt, even if cross_node_to is not enabled + int partial = input.readInt(); + if (DatabaseDescriptor.hasCrossNodeTimeout()) + { + long crossNodeTimestamp = (timestamp & 0xFFFFFFFF00000000L) | (((partial & 0xFFFFFFFFL) << 2) >> 2); + isCrossNodeTimestamp = (timestamp != crossNodeTimestamp); + timestamp = crossNodeTimestamp; + } + + MessageIn message = MessageIn.read(input, version, id); + if (message == null) + { + // callback expired; nothing to do + return; + } + if (version <= MessagingService.current_version) + { + MessagingService.instance().receive(message, id, timestamp, isCrossNodeTimestamp); + } + // else ignore message } catch (Throwable t) { @@ -251,7 +326,6 @@ public void startup(ICluster cluster) { mkdirs(); - Config.setOverrideLoadConfig(() -> loadConfig(config)); DatabaseDescriptor.setDaemonInitialized(); DatabaseDescriptor.createAllDirectories(); @@ -281,8 +355,28 @@ public void startup(ICluster cluster) throw new RuntimeException(e); } - initializeRing(cluster); - registerMockMessaging(cluster); + if (config.has(NETWORK)) + { + registerFilter(cluster); + MessagingService.instance().listen(); + } + else + { + // Even though we don't use MessagingService, access the static SocketFactory + // instance here so that we start the static event loop state +// -- not sure what that means? SocketFactory.instance.getClass(); + registerMockMessaging(cluster); + } + + // TODO: this is more than just gossip + if (config.has(GOSSIP)) + { + StorageService.instance.initServer(); + } + else + { + initializeRing(cluster); + } SystemKeyspace.finishStartup(); @@ -365,33 +459,48 @@ private void initializeRing(ICluster cluster) } public Future shutdown() + { + return shutdown(true); + } + + public Future shutdown(boolean graceful) { Future future = async((ExecutorService executor) -> { Throwable error = null; + + if (config.has(GOSSIP) || config.has(NETWORK)) + { + StorageService.instance.shutdownServer(); + + error = parallelRun(error, executor, + () -> NanoTimeToCurrentTimeMillis.shutdown(MINUTES.toMillis(1L)) + ); + } + error = parallelRun(error, executor, - Gossiper.instance::stop, - CompactionManager.instance::forceShutdown, - CommitLog.instance::shutdownBlocking, - ColumnFamilyStore::shutdownFlushExecutor, - ColumnFamilyStore::shutdownPostFlushExecutor, - ColumnFamilyStore::shutdownReclaimExecutor, - PendingRangeCalculatorService.instance::shutdownExecutor, - StorageService.instance::shutdownBGMonitor, - Ref::shutdownReferenceReaper, - Memtable.MEMORY_POOL::shutdown, - ScheduledExecutors::shutdownAndWait, - SSTableReader::shutdownBlocking + () -> Gossiper.instance.stopShutdownAndWait(1L, MINUTES), + CompactionManager.instance::forceShutdown, + () -> BatchlogManager.shutdownAndWait(1L, MINUTES), + () -> HintedHandOffManager.instance.shutdownAndWait(1L, MINUTES), + () -> StreamCoordinator.shutdownAndWait(1L, MINUTES), + () -> IndexSummaryManager.instance.shutdownAndWait(1L, MINUTES), + () -> ColumnFamilyStore.shutdownExecutorsAndWait(1L, MINUTES), + () -> PendingRangeCalculatorService.instance.shutdownExecutor(1L, MINUTES), + () -> StorageService.instance.shutdownBGMonitorAndWait(1L, MINUTES), + () -> Ref.shutdownReferenceReaper(1L, MINUTES), + () -> Memtable.MEMORY_POOL.shutdownAndWait(1L, MINUTES), + () -> SSTableReader.shutdownBlocking(1L, MINUTES), + () -> ScheduledExecutors.shutdownAndWait(1L, MINUTES) ); error = parallelRun(error, executor, + CommitLog.instance::shutdownBlocking, MessagingService.instance()::shutdown ); error = parallelRun(error, executor, - StageManager::shutdownAndWait, - SharedExecutorPool.SHARED::shutdown + () -> StageManager.shutdownAndWait(1L, MINUTES), + () -> SharedExecutorPool.SHARED.shutdownAndWait(1L, MINUTES) ); - LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); - loggerContext.stop(); Throwables.maybeFail(error); }).apply(isolatedExecutor); diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java index 6fd5c7e5b085..363a1df501db 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceClassLoader.java @@ -48,6 +48,7 @@ public class InstanceClassLoader extends URLClassLoader name.startsWith("org.apache.cassandra.distributed.api.") || name.startsWith("sun.") || name.startsWith("oracle.") + || name.startsWith("com.intellij.") || name.startsWith("com.sun.") || name.startsWith("com.oracle.") || name.startsWith("java.") @@ -63,16 +64,16 @@ public static interface Factory InstanceClassLoader create(int id, URL[] urls, ClassLoader sharedClassLoader); } - private final int id; private final URL[] urls; + private final int generation; // used to help debug class loader leaks, by helping determine which classloaders should have been collected private final ClassLoader sharedClassLoader; - InstanceClassLoader(int id, URL[] urls, ClassLoader sharedClassLoader) + InstanceClassLoader(int generation, URL[] urls, ClassLoader sharedClassLoader) { super(urls, null); - this.id = id; this.urls = urls; this.sharedClassLoader = sharedClassLoader; + this.generation = generation; } @Override @@ -109,7 +110,7 @@ public static boolean wasLoadedByAnInstanceClassLoader(Class clazz) public String toString() { return "InstanceClassLoader{" + - "id=" + id + + "generation=" + generation + ", urls=" + Arrays.toString(urls) + '}'; } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java index ab15fed6222d..efe9a0f765d0 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/InstanceConfig.java @@ -20,6 +20,7 @@ import org.apache.cassandra.config.Config; import org.apache.cassandra.config.ParameterizedClass; +import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.SimpleSeedProvider; @@ -30,6 +31,7 @@ import java.net.UnknownHostException; import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; import java.util.Map; import java.util.TreeMap; import java.util.UUID; @@ -45,6 +47,8 @@ public class InstanceConfig implements IInstanceConfig public UUID hostId() { return hostId; } private final Map params = new TreeMap<>(); + private EnumSet featureFlags; + private volatile InetAddressAndPort broadcastAddressAndPort; @Override @@ -103,6 +107,7 @@ private InstanceConfig(int num, Collections.singletonMap("seeds", "127.0.0.1"))) // legacy parameters .forceSet("commitlog_sync_batch_window_in_ms", 1.0); + this.featureFlags = EnumSet.noneOf(Feature.class); } private InstanceConfig(InstanceConfig copy) @@ -110,6 +115,18 @@ private InstanceConfig(InstanceConfig copy) this.num = copy.num; this.params.putAll(copy.params); this.hostId = copy.hostId; + this.featureFlags = copy.featureFlags; + } + + public InstanceConfig with(Feature featureFlag) + { + featureFlags.add(featureFlag); + return this; + } + + public boolean has(Feature featureFlag) + { + return featureFlags.contains(featureFlag); } public InstanceConfig set(String fieldName, Object value) @@ -200,13 +217,14 @@ public String getString(String name) return (String)params.get(name); } - public static InstanceConfig generate(int nodeNum, File root, String token) + public static InstanceConfig generate(int nodeNum, int subnet, File root, String token) { + String ipPrefix = "127.0." + subnet + "."; return new InstanceConfig(nodeNum, - "127.0.0." + nodeNum, - "127.0.0." + nodeNum, - "127.0.0." + nodeNum, - "127.0.0." + nodeNum, + ipPrefix + nodeNum, + ipPrefix + nodeNum, + ipPrefix + nodeNum, + ipPrefix + nodeNum, String.format("%s/node%d/saved_caches", root, nodeNum), new String[] { String.format("%s/node%d/data", root, nodeNum) }, String.format("%s/node%d/commitlog", root, nodeNum), diff --git a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java index d82c9e49ab28..1d26c5dec36f 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/IsolatedExecutor.java @@ -27,28 +27,36 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.URLClassLoader; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; +import org.slf4j.LoggerFactory; + +import ch.qos.logback.classic.LoggerContext; import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.distributed.api.IIsolatedExecutor; +import org.apache.cassandra.utils.ExecutorUtils; public class IsolatedExecutor implements IIsolatedExecutor { final ExecutorService isolatedExecutor; + private final String name; private final ClassLoader classLoader; private final Method deserializeOnInstance; IsolatedExecutor(String name, ClassLoader classLoader) { + this.name = name; this.isolatedExecutor = Executors.newCachedThreadPool(new NamedThreadFactory("isolatedExecutor", Thread.NORM_PRIORITY, classLoader, new ThreadGroup(name))); this.classLoader = classLoader; this.deserializeOnInstance = lookupDeserializeOneObject(classLoader); @@ -57,9 +65,40 @@ public class IsolatedExecutor implements IIsolatedExecutor public Future shutdown() { isolatedExecutor.shutdown(); - ThrowingRunnable.toRunnable(((URLClassLoader) classLoader)::close).run(); - return CompletableFuture.runAsync(ThrowingRunnable.toRunnable(() -> isolatedExecutor.awaitTermination(60, TimeUnit.SECONDS)), - Executors.newSingleThreadExecutor()); + + /* Use a thread pool with a core pool size of zero to terminate the thread as soon as possible + ** so the instance class loader can be garbage collected. Uses a custom thread factory + ** rather than NamedThreadFactory to avoid calling FastThreadLocal.removeAll() in 3.0 and up + ** as it was observed crashing during test failures and made it harder to find the real cause. + */ + ThreadFactory threadFactory = (Runnable r) -> { + Thread t = new Thread(r, name + "_shutdown"); + t.setDaemon(true); + return t; + }; + ExecutorService shutdownExecutor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 0, TimeUnit.SECONDS, + new LinkedBlockingQueue(), threadFactory); + return shutdownExecutor.submit(() -> { + try + { + ExecutorUtils.awaitTermination(60, TimeUnit.SECONDS, isolatedExecutor); + + // Shutdown logging last - this is not ideal as the logging subsystem is initialized + // outsize of this class, however doing it this way provides access to the full + // logging system while termination is taking place. + LoggerContext loggerContext = (LoggerContext) LoggerFactory.getILoggerFactory(); + loggerContext.stop(); + + // Close the instance class loader after shutting down the isolatedExecutor and logging + // in case error handling triggers loading additional classes + ((URLClassLoader) classLoader).close(); + } + finally + { + shutdownExecutor.shutdownNow(); + } + return null; + }); } public CallableNoExcept> async(CallableNoExcept call) { return () -> isolatedExecutor.submit(call); } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java b/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java index a72c7a56b48b..c1607f8ba061 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/MessageFilters.java @@ -32,33 +32,20 @@ public class MessageFilters implements IMessageFilters { - private final ICluster cluster; private final Set filters = new CopyOnWriteArraySet<>(); - public MessageFilters(AbstractCluster cluster) + public boolean permit(IInstance from, IInstance to, int verb) { - this.cluster = cluster; - } + if (from == null || to == null) + return false; // cannot deliver + int fromNum = from.config().num(); + int toNum = to.config().num(); - public BiConsumer filter(BiConsumer applyIfNotFiltered) - { - return (toAddress, message) -> - { - IInstance from = cluster.get(message.from()); - IInstance to = cluster.get(toAddress); - if (from == null || to == null) - return; // cannot deliver - int fromNum = from.config().num(); - int toNum = to.config().num(); - int verb = message.verb(); - for (Filter filter : filters) - { - if (filter.matches(fromNum, toNum, verb)) - return; - } + for (Filter filter : filters) + if (filter.matches(fromNum, toNum, verb)) + return false; - applyIfNotFiltered.accept(toAddress, message); - }; + return true; } public class Filter implements IMessageFilters.Filter diff --git a/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java index 18ca17f04d8b..757c17f32984 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/DistributedTestBase.java @@ -29,6 +29,7 @@ import org.junit.BeforeClass; import org.apache.cassandra.distributed.impl.AbstractCluster; +import org.apache.cassandra.distributed.impl.IsolatedExecutor; public class DistributedTestBase { @@ -41,10 +42,37 @@ public void afterEach() public static String KEYSPACE = "distributed_test_keyspace"; + public static void nativeLibraryWorkaround() + { + // Disable the C library for in-JVM dtests otherwise it holds a gcroot against the InstanceClassLoader + System.setProperty("cassandra.disable_clibrary", "true"); + + // Disable the Netty tcnative library otherwise the io.netty.internal.tcnative.CertificateCallbackTask, + // CertificateVerifierTask, SSLPrivateKeyMethodDecryptTask, SSLPrivateKeyMethodSignTask, + // SSLPrivateKeyMethodTask, and SSLTask hold a gcroot against the InstanceClassLoader. + System.setProperty("cassandra.disable_tcactive_openssl", "true"); + System.setProperty("io.netty.transport.noNative", "true"); + } + + public static void processReaperWorkaround() + { + // Make sure the 'process reaper' thread is initially created under the main classloader, + // otherwise it gets created with the contextClassLoader pointing to an InstanceClassLoader + // which prevents it from being garbage collected. + IsolatedExecutor.ThrowingRunnable.toRunnable(() -> new ProcessBuilder().command("true").start().waitFor()).run(); + } + @BeforeClass public static void setup() { System.setProperty("org.apache.cassandra.disable_mbean_registration", "true"); + nativeLibraryWorkaround(); + processReaperWorkaround(); + } + + static String withKeyspace(String replaceIn) + { + return String.format(replaceIn, KEYSPACE); } protected static > C init(C cluster) diff --git a/test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java b/test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java new file mode 100644 index 000000000000..55c700c33995 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/ResourceLeakTest.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.File; +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.nio.file.FileSystems; +import java.nio.file.Path; +import java.sql.Date; +import java.text.SimpleDateFormat; +import java.time.Instant; +import java.util.List; +import java.util.function.Consumer; +import javax.management.MBeanServer; + +import org.junit.Ignore; +import org.junit.Test; + +import com.sun.management.HotSpotDiagnosticMXBean; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.impl.InstanceConfig; +import org.apache.cassandra.service.CassandraDaemon; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.SigarLibrary; + +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + +/* Resource Leak Test - useful when tracking down issues with in-JVM framework cleanup. + * All objects referencing the InstanceClassLoader need to be garbage collected or + * the JVM runs out of metaspace. This test also calls out to lsof to check which + * file handles are still opened. + * + * This is intended to be a burn type test where it is run outside of the test suites + * when a problem is detected (like OutOfMetaspace exceptions). + * + * Currently this test demonstrates that the InstanceClassLoader is cleaned up (load up + * the final hprof and check that the class loaders are not reachable from a GC root), + * but it shows that the file handles for Data/Index files are being leaked. + */ +@Ignore +public class ResourceLeakTest extends DistributedTestBase +{ + // Parameters to adjust while hunting for leaks + final int numTestLoops = 1; // Set this value high to crash on leaks, or low when tracking down an issue. + final boolean dumpEveryLoop = false; // Dump heap & possibly files every loop + final boolean dumpFileHandles = false; // Call lsof whenever dumping resources + final boolean forceCollection = false; // Whether to explicitly force finalization/gc for smaller heap dumps + final long finalWaitMillis = 0l; // Number of millis to wait before final resource dump to give gc a chance + + static final SimpleDateFormat format = new SimpleDateFormat("yyyyMMddHHmmss"); + static final String when = format.format(Date.from(Instant.now())); + + static String outputFilename(String base, String description, String extension) + { + Path p = FileSystems.getDefault().getPath("build", "test", + String.join("-", when, base, description) + extension); + return p.toString(); + } + + /** + * Retrieves the process ID or null if the process ID cannot be retrieved. + * @return the process ID or null if the process ID cannot be retrieved. + * + * (Duplicated from HeapUtils to avoid refactoring older releases where this test is useful). + */ + private static Long getProcessId() + { + // Once Java 9 is ready the process API should provide a better way to get the process ID. + long pid = SigarLibrary.instance.getPid(); + + if (pid >= 0) + return Long.valueOf(pid); + + return getProcessIdFromJvmName(); + } + + /** + * Retrieves the process ID from the JVM name. + * @return the process ID or null if the process ID cannot be retrieved. + */ + private static Long getProcessIdFromJvmName() + { + // the JVM name in Oracle JVMs is: '@' but this might not be the case on all JVMs + String jvmName = ManagementFactory.getRuntimeMXBean().getName(); + try + { + return Long.parseLong(jvmName.split("@")[0]); + } + catch (NumberFormatException e) + { + // ignore + } + return null; + } + + static void dumpHeap(String description, boolean live) throws IOException + { + MBeanServer server = ManagementFactory.getPlatformMBeanServer(); + HotSpotDiagnosticMXBean mxBean = ManagementFactory.newPlatformMXBeanProxy( + server, "com.sun.management:type=HotSpotDiagnostic", HotSpotDiagnosticMXBean.class); + mxBean.dumpHeap(outputFilename("heap", description, ".hprof"), live); + } + + static void dumpOpenFiles(String description) throws IOException, InterruptedException + { + long pid = getProcessId(); + ProcessBuilder map = new ProcessBuilder("/usr/sbin/lsof", "-p", Long.toString(pid)); + File output = new File(outputFilename("lsof", description, ".txt")); + map.redirectOutput(output); + map.redirectErrorStream(true); + map.start().waitFor(); + } + + void dumpResources(String description) throws IOException, InterruptedException + { + dumpHeap(description, false); + if (dumpFileHandles) + { + dumpOpenFiles(description); + } + } + + void doTest(int numClusterNodes, Consumer updater) throws Throwable + { + for (int loop = 0; loop < numTestLoops; loop++) + { + try (Cluster cluster = Cluster.build(numClusterNodes).withConfig(updater).start()) + { + if (cluster.get(1).config().has(GOSSIP)) // Wait for gossip to settle on the seed node + cluster.get(1).runOnInstance(() -> CassandraDaemon.waitForGossipToSettle()); + + init(cluster); + String tableName = "tbl" + loop; + cluster.schemaChange("CREATE TABLE " + KEYSPACE + "." + tableName + " (pk int, ck int, v int, PRIMARY KEY (pk, ck))"); + cluster.coordinator(1).execute("INSERT INTO " + KEYSPACE + "." + tableName + "(pk,ck,v) VALUES (0,0,0)", ConsistencyLevel.ALL); + cluster.get(1).callOnInstance(() -> FBUtilities.waitOnFutures(Keyspace.open(KEYSPACE).flush())); + if (dumpEveryLoop) + { + dumpResources(String.format("loop%03d", loop)); + } + } + catch (Throwable tr) + { + System.out.println("Dumping resources for exception: " + tr.getMessage()); + tr.printStackTrace(); + dumpResources("exception"); + } + if (forceCollection) + { + System.runFinalization(); + System.gc(); + } + } + } + + @Test + public void looperTest() throws Throwable + { + doTest(1, config -> {}); + if (forceCollection) + { + System.runFinalization(); + System.gc(); + Thread.sleep(finalWaitMillis); + } + dumpResources("final"); + } + + @Test + public void looperGossipNetworkTest() throws Throwable + { + doTest(2, config -> config.with(GOSSIP).with(NETWORK)); + if (forceCollection) + { + System.runFinalization(); + System.gc(); + Thread.sleep(finalWaitMillis); + } + dumpResources("final-gossip-network"); + } +} diff --git a/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java b/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java index 7bb4a5125147..0d61ad894993 100644 --- a/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java +++ b/test/unit/org/apache/cassandra/concurrent/SEPExecutorTest.java @@ -22,6 +22,7 @@ import java.io.PrintStream; import java.util.Arrays; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import org.junit.Assert; import org.junit.Test; @@ -62,7 +63,7 @@ public void run() } // shutdown does not guarantee that threads are actually dead once it exits, only that they will stop promptly afterwards - sharedPool.shutdown(); + sharedPool.shutdownAndWait(1L, TimeUnit.MINUTES); for (Thread thread : Thread.getAllStackTraces().keySet()) { if (thread.getName().contains(MAGIC)) From db2ad0f7c59d9deb1f8755858cd630d640c5baa9 Mon Sep 17 00:00:00 2001 From: Jordan West Date: Mon, 19 Aug 2019 10:10:37 -0700 Subject: [PATCH 51/78] Rename StatsMetadata estimatedColumnCount to estimatedCellPerPartitionCount patch by Jordan West; reviewed by Benedict for CASSANDRA-15285 --- .../cassandra/db/ColumnFamilyStore.java | 8 ++++---- .../SSTableReversedIterator.java | 2 +- .../AbstractCompactionStrategy.java | 6 ++---- .../cassandra/db/filter/DataLimits.java | 2 +- .../index/SecondaryIndexManager.java | 4 +--- .../io/sstable/format/SSTableReader.java | 4 ++-- .../io/sstable/metadata/StatsMetadata.java | 20 +++++++++---------- .../cassandra/metrics/TableMetrics.java | 2 +- .../tools/SSTableMetadataViewer.java | 2 +- .../index/internal/CustomCassandraIndex.java | 3 +-- 10 files changed, 24 insertions(+), 29 deletions(-) diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 2ef5a76ad761..075ebac486e9 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -2380,14 +2380,14 @@ private void validateCompactionThresholds(int minThreshold, int maxThreshold) // End JMX get/set. - public int getMeanColumns() + public int getMeanEstimatedCellPerPartitionCount() { long sum = 0; long count = 0; for (SSTableReader sstable : getSSTables(SSTableSet.CANONICAL)) { - long n = sstable.getEstimatedColumnCount().count(); - sum += sstable.getEstimatedColumnCount().mean() * n; + long n = sstable.getEstimatedCellPerPartitionCount().count(); + sum += sstable.getEstimatedCellPerPartitionCount().mean() * n; count += n; } return count > 0 ? (int) (sum / count) : 0; @@ -2559,7 +2559,7 @@ public double getDroppableTombstoneRatio() for (SSTableReader sstable : getSSTables(SSTableSet.LIVE)) { allDroppable += sstable.getDroppableTombstonesBefore(localTime - metadata().params.gcGraceSeconds); - allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count(); + allColumns += sstable.getEstimatedCellPerPartitionCount().mean() * sstable.getEstimatedCellPerPartitionCount().count(); } return allColumns > 0 ? allDroppable / allColumns : 0; } diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java index 60a6f70534e4..1e1030cf3dfe 100644 --- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java +++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java @@ -108,7 +108,7 @@ protected ReusablePartitionData createBuffer(int blocksCount) // FIXME: so far we only keep stats on cells, so to get a rough estimate on the number of rows, // we divide by the number of regular columns the table has. We should fix once we collect the // stats on rows - int estimatedRowsPerPartition = (int)(sstable.getEstimatedColumnCount().percentile(0.75) / columnCount); + int estimatedRowsPerPartition = (int)(sstable.getEstimatedCellPerPartitionCount().percentile(0.75) / columnCount); estimatedRowCount = Math.max(estimatedRowsPerPartition / blocksCount, 1); } catch (IllegalStateException e) diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java index 74c154d51905..ad494b15da06 100644 --- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java +++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java @@ -21,8 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import org.apache.cassandra.db.Directories; import org.apache.cassandra.db.SerializationHeader; @@ -412,8 +410,8 @@ else if (CompactionController.getFullyExpiredSSTables(cfs, Collections.singleton ranges.add(new Range<>(overlap.first.getToken(), overlap.last.getToken())); long remainingKeys = keys - sstable.estimatedKeysForRanges(ranges); // next, calculate what percentage of columns we have within those keys - long columns = sstable.getEstimatedColumnCount().mean() * remainingKeys; - double remainingColumnsRatio = ((double) columns) / (sstable.getEstimatedColumnCount().count() * sstable.getEstimatedColumnCount().mean()); + long columns = sstable.getEstimatedCellPerPartitionCount().mean() * remainingKeys; + double remainingColumnsRatio = ((double) columns) / (sstable.getEstimatedCellPerPartitionCount().count() * sstable.getEstimatedCellPerPartitionCount().mean()); // return if we still expect to have droppable tombstones in rest of columns return remainingColumnsRatio * droppableRatio > tombstoneThreshold; diff --git a/src/java/org/apache/cassandra/db/filter/DataLimits.java b/src/java/org/apache/cassandra/db/filter/DataLimits.java index 246cc08d850a..3a766e0c2aa8 100644 --- a/src/java/org/apache/cassandra/db/filter/DataLimits.java +++ b/src/java/org/apache/cassandra/db/filter/DataLimits.java @@ -446,7 +446,7 @@ public float estimateTotalResults(ColumnFamilyStore cfs) { // TODO: we should start storing stats on the number of rows (instead of the number of cells, which // is what getMeanColumns returns) - float rowsPerPartition = ((float) cfs.getMeanColumns()) / cfs.metadata().regularColumns().size(); + float rowsPerPartition = ((float) cfs.getMeanEstimatedCellPerPartitionCount()) / cfs.metadata().regularColumns().size(); return rowsPerPartition * (cfs.estimateKeys()); } diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java index b37251af76bc..60fc3ba314e3 100644 --- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java +++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java @@ -71,10 +71,8 @@ import org.apache.cassandra.service.pager.SinglePartitionPager; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.ProtocolVersion; -import org.apache.cassandra.utils.ExecutorUtils; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; -import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.concurrent.Refs; import static org.apache.cassandra.utils.ExecutorUtils.awaitTermination; @@ -929,7 +927,7 @@ public int calculateIndexingPageSize() if (meanPartitionSize <= 0) return DEFAULT_PAGE_SIZE; - int meanCellsPerPartition = baseCfs.getMeanColumns(); + int meanCellsPerPartition = baseCfs.getMeanEstimatedCellPerPartitionCount(); if (meanCellsPerPartition <= 0) return DEFAULT_PAGE_SIZE; diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index aa4ca55beef5..36a1e633080d 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -1943,9 +1943,9 @@ public EstimatedHistogram getEstimatedPartitionSize() return sstableMetadata.estimatedPartitionSize; } - public EstimatedHistogram getEstimatedColumnCount() + public EstimatedHistogram getEstimatedCellPerPartitionCount() { - return sstableMetadata.estimatedColumnCount; + return sstableMetadata.estimatedCellPerPartitionCount; } public double getEstimatedDroppableTombstoneRatio(int gcBefore) diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java index 5d464fef4cca..f4e5beba24b6 100755 --- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java +++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java @@ -47,7 +47,7 @@ public class StatsMetadata extends MetadataComponent public static final ISerializer> commitLogPositionSetSerializer = IntervalSet.serializer(CommitLogPosition.serializer); public final EstimatedHistogram estimatedPartitionSize; - public final EstimatedHistogram estimatedColumnCount; + public final EstimatedHistogram estimatedCellPerPartitionCount; public final IntervalSet commitLogIntervals; public final long minTimestamp; public final long maxTimestamp; @@ -70,7 +70,7 @@ public class StatsMetadata extends MetadataComponent public final EncodingStats encodingStats; public StatsMetadata(EstimatedHistogram estimatedPartitionSize, - EstimatedHistogram estimatedColumnCount, + EstimatedHistogram estimatedCellPerPartitionCount, IntervalSet commitLogIntervals, long minTimestamp, long maxTimestamp, @@ -91,7 +91,7 @@ public StatsMetadata(EstimatedHistogram estimatedPartitionSize, boolean isTransient) { this.estimatedPartitionSize = estimatedPartitionSize; - this.estimatedColumnCount = estimatedColumnCount; + this.estimatedCellPerPartitionCount = estimatedCellPerPartitionCount; this.commitLogIntervals = commitLogIntervals; this.minTimestamp = minTimestamp; this.maxTimestamp = maxTimestamp; @@ -124,7 +124,7 @@ public MetadataType getType() */ public double getEstimatedDroppableTombstoneRatio(int gcBefore) { - long estimatedColumnCount = this.estimatedColumnCount.mean() * this.estimatedColumnCount.count(); + long estimatedColumnCount = this.estimatedCellPerPartitionCount.mean() * this.estimatedCellPerPartitionCount.count(); if (estimatedColumnCount > 0) { double droppable = getDroppableTombstonesBefore(gcBefore); @@ -145,7 +145,7 @@ public double getDroppableTombstonesBefore(int gcBefore) public StatsMetadata mutateLevel(int newLevel) { return new StatsMetadata(estimatedPartitionSize, - estimatedColumnCount, + estimatedCellPerPartitionCount, commitLogIntervals, minTimestamp, maxTimestamp, @@ -169,7 +169,7 @@ public StatsMetadata mutateLevel(int newLevel) public StatsMetadata mutateRepairedMetadata(long newRepairedAt, UUID newPendingRepair, boolean newIsTransient) { return new StatsMetadata(estimatedPartitionSize, - estimatedColumnCount, + estimatedCellPerPartitionCount, commitLogIntervals, minTimestamp, maxTimestamp, @@ -199,7 +199,7 @@ public boolean equals(Object o) StatsMetadata that = (StatsMetadata) o; return new EqualsBuilder() .append(estimatedPartitionSize, that.estimatedPartitionSize) - .append(estimatedColumnCount, that.estimatedColumnCount) + .append(estimatedCellPerPartitionCount, that.estimatedCellPerPartitionCount) .append(commitLogIntervals, that.commitLogIntervals) .append(minTimestamp, that.minTimestamp) .append(maxTimestamp, that.maxTimestamp) @@ -225,7 +225,7 @@ public int hashCode() { return new HashCodeBuilder() .append(estimatedPartitionSize) - .append(estimatedColumnCount) + .append(estimatedCellPerPartitionCount) .append(commitLogIntervals) .append(minTimestamp) .append(maxTimestamp) @@ -252,7 +252,7 @@ public int serializedSize(Version version, StatsMetadata component) throws IOExc { int size = 0; size += EstimatedHistogram.serializer.serializedSize(component.estimatedPartitionSize); - size += EstimatedHistogram.serializer.serializedSize(component.estimatedColumnCount); + size += EstimatedHistogram.serializer.serializedSize(component.estimatedCellPerPartitionCount); size += CommitLogPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE)); size += 8 + 8 + 4 + 4 + 4 + 4 + 8 + 8; // mix/max timestamp(long), min/maxLocalDeletionTime(int), min/max TTL, compressionRatio(double), repairedAt (long) size += TombstoneHistogram.serializer.serializedSize(component.estimatedTombstoneDropTime); @@ -290,7 +290,7 @@ public int serializedSize(Version version, StatsMetadata component) throws IOExc public void serialize(Version version, StatsMetadata component, DataOutputPlus out) throws IOException { EstimatedHistogram.serializer.serialize(component.estimatedPartitionSize, out); - EstimatedHistogram.serializer.serialize(component.estimatedColumnCount, out); + EstimatedHistogram.serializer.serialize(component.estimatedCellPerPartitionCount, out); CommitLogPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE), out); out.writeLong(component.minTimestamp); out.writeLong(component.maxTimestamp); diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index beb6d2e6a6f9..d8330cc2f03c 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -523,7 +523,7 @@ public long[] getValue() { public EstimatedHistogram getHistogram(SSTableReader reader) { - return reader.getEstimatedColumnCount(); + return reader.getEstimatedCellPerPartitionCount(); } }); } diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java index 8ff964f0d57a..8f7e8a5dbd8d 100755 --- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java +++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java @@ -398,7 +398,7 @@ private void printSStableMetadata(String fname, boolean scan) throws IOException String::valueOf); rowSize.printHistogram(out, color, unicode); field("Column Count", ""); - TermHistogram cellCount = new TermHistogram(stats.estimatedColumnCount, + TermHistogram cellCount = new TermHistogram(stats.estimatedCellPerPartitionCount, "Columns", String::valueOf, String::valueOf); diff --git a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java index 1c07760d966f..04db7f6f90bd 100644 --- a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java +++ b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java @@ -59,7 +59,6 @@ import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Pair; -import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.concurrent.Refs; import static org.apache.cassandra.index.internal.CassandraIndex.getFunctions; @@ -207,7 +206,7 @@ private boolean supportsExpression(RowFilter.Expression expression) public long getEstimatedResultRows() { - return indexCfs.getMeanColumns(); + return indexCfs.getMeanEstimatedCellPerPartitionCount(); } /** From 2b10a5f2b5e62f2900119a37e91637916e8b23df Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Mon, 12 Aug 2019 14:41:40 +0100 Subject: [PATCH 52/78] Fix LegacyLayout RangeTombstoneList IndexOutOfBoundsException when upgrading and RangeTombstone bounds are asymmetric. patch by Benedict Elliott Smith; reviewed by Mick Semb Wever for CASSANDRA-15172 --- CHANGES.txt | 1 + .../org/apache/cassandra/db/LegacyLayout.java | 5 ++- .../apache/cassandra/db/LegacyLayoutTest.java | 36 ++++++++++++++++--- 3 files changed, 34 insertions(+), 8 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index e9567963472d..3f2b4d0a0d31 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.19 + * Fix LegacyLayout RangeTombstoneList IndexOutOfBoundsException when upgrading and RangeTombstone bounds are asymmetric (CASSANDRA-15172) * Fix NPE when using allocate_tokens_for_keyspace on new DC/rack (CASSANDRA-14952) * Filter sstables earlier when running cleanup (CASSANDRA-15100) * Use mean row count instead of mean column count for index selectivity calculation (CASSANDRA-15259) diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java index a46aa91f8fbb..1a03c91371f7 100644 --- a/src/java/org/apache/cassandra/db/LegacyLayout.java +++ b/src/java/org/apache/cassandra/db/LegacyLayout.java @@ -2598,11 +2598,10 @@ private long serializedSizeCompound(boolean isDense) CompositeType.Builder startBuilder = type.builder(); CompositeType.Builder endBuilder = type.builder(); - for (int j = 0; j < start.bound.clustering().size(); j++) - { + for (int j = 0; j < start.bound.size(); j++) startBuilder.add(start.bound.get(j)); + for (int j = 0; j < end.bound.size(); j++) endBuilder.add(end.bound.get(j)); - } if (start.collectionName != null) startBuilder.add(start.collectionName.name.bytes); diff --git a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java index 580a0e70d667..77197d3b47b4 100644 --- a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java +++ b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java @@ -25,12 +25,15 @@ import java.nio.file.Paths; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.partitions.ImmutableBTreePartition; import org.apache.cassandra.db.rows.BufferCell; import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.RowIterator; +import org.apache.cassandra.db.rows.Rows; import org.apache.cassandra.db.rows.SerializationHelper; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer; +import org.apache.cassandra.db.rows.UnfilteredRowIterators; import org.apache.cassandra.db.transform.FilteredRows; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.util.DataInputBuffer; @@ -60,6 +63,7 @@ import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.Hex; +import static org.apache.cassandra.net.MessagingService.VERSION_21; import static org.junit.Assert.*; public class LegacyLayoutTest @@ -181,10 +185,10 @@ private static UnfilteredRowIterator roundTripVia21(UnfilteredRowIterator partit { try (DataOutputBuffer out = new DataOutputBuffer()) { - LegacyLayout.serializeAsLegacyPartition(null, partition, out, MessagingService.VERSION_21); + LegacyLayout.serializeAsLegacyPartition(null, partition, out, VERSION_21); try (DataInputBuffer in = new DataInputBuffer(out.buffer(), false)) { - return LegacyLayout.deserializeLegacyPartition(in, MessagingService.VERSION_21, SerializationHelper.Flag.LOCAL, partition.partitionKey().getKey()); + return LegacyLayout.deserializeLegacyPartition(in, VERSION_21, SerializationHelper.Flag.LOCAL, partition.partitionKey().getKey()); } } } @@ -278,11 +282,11 @@ public void testCollectionDeletionRoundTripForDroppedColumn() throws Throwable try (RowIterator before = FilteredRows.filter(upd.unfilteredIterator(), FBUtilities.nowInSeconds()); DataOutputBuffer serialized21 = new DataOutputBuffer()) { - LegacyLayout.serializeAsLegacyPartition(null, upd.unfilteredIterator(), serialized21, MessagingService.VERSION_21); + LegacyLayout.serializeAsLegacyPartition(null, upd.unfilteredIterator(), serialized21, VERSION_21); QueryProcessor.executeInternal(String.format("ALTER TABLE \"%s\".legacy_rt_rt_dc DROP s", KEYSPACE)); try (DataInputBuffer in = new DataInputBuffer(serialized21.buffer(), false)) { - try (UnfilteredRowIterator deser21 = LegacyLayout.deserializeLegacyPartition(in, MessagingService.VERSION_21, SerializationHelper.Flag.LOCAL, upd.partitionKey().getKey()); + try (UnfilteredRowIterator deser21 = LegacyLayout.deserializeLegacyPartition(in, VERSION_21, SerializationHelper.Flag.LOCAL, upd.partitionKey().getKey()); RowIterator after = FilteredRows.filter(deser21, FBUtilities.nowInSeconds());) { while (before.hasNext() || after.hasNext()) @@ -328,7 +332,7 @@ public static void main(String[] args) throws IOException, ConfigurationExceptio MigrationManager.announceNewColumnFamily(table); byte[] bytes = Hex.hexToBytes("00026b73000263660000000000000001fffffffe01000000088000000000000000010000000880000000000000000000000100000000007fffffffffffffff000b00017600000400000001000000000000000000000101"); - ReadCommand.legacyPagedRangeCommandSerializer.deserialize(new DataInputBuffer(bytes), MessagingService.VERSION_21); + ReadCommand.legacyPagedRangeCommandSerializer.deserialize(new DataInputBuffer(bytes), VERSION_21); } @Test @@ -345,4 +349,26 @@ public void testDecodeCollectionPageBoundary() LegacyLayout.decodeSliceBound(table, bound, true); } + @Test + public void testAsymmetricRTBoundSerializedSize() + { + CFMetaData table = CFMetaData.Builder.create("ks", "cf") + .addPartitionKey("k", Int32Type.instance) + .addClusteringColumn("c1", Int32Type.instance) + .addClusteringColumn("c2", Int32Type.instance) + .addRegularColumn("v", Int32Type.instance) + .build(); + + ByteBuffer one = Int32Type.instance.decompose(1); + ByteBuffer two = Int32Type.instance.decompose(2); + PartitionUpdate p = new PartitionUpdate(table, table.decorateKey(one), table.partitionColumns(), 0); + p.add(new RangeTombstone(Slice.make(new Slice.Bound(ClusteringPrefix.Kind.EXCL_START_BOUND, new ByteBuffer[] { one, one }), + new Slice.Bound(ClusteringPrefix.Kind.INCL_END_BOUND, new ByteBuffer[] { two })), + new DeletionTime(1, 1) + )); + + LegacyLayout.fromUnfilteredRowIterator(null, p.unfilteredIterator()); + LegacyLayout.serializedSizeAsLegacyPartition(null, p.unfilteredIterator(), VERSION_21); + } + } \ No newline at end of file From eb2c549940baf0e30a8c07ea51fe4a1d828ca3ac Mon Sep 17 00:00:00 2001 From: Paulo Motta Date: Fri, 23 Aug 2019 09:06:45 -0300 Subject: [PATCH 53/78] ninja: fix build error on LegacyLayoutTest --- test/unit/org/apache/cassandra/db/LegacyLayoutTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java index 4058b0363f38..5a08e3e346ff 100644 --- a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java +++ b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java @@ -374,8 +374,8 @@ public void testAsymmetricRTBoundSerializedSize() ByteBuffer one = Int32Type.instance.decompose(1); ByteBuffer two = Int32Type.instance.decompose(2); PartitionUpdate p = new PartitionUpdate(table, table.decorateKey(one), table.partitionColumns(), 0); - p.add(new RangeTombstone(Slice.make(new Slice.Bound(ClusteringPrefix.Kind.EXCL_START_BOUND, new ByteBuffer[] { one, one }), - new Slice.Bound(ClusteringPrefix.Kind.INCL_END_BOUND, new ByteBuffer[] { two })), + p.add(new RangeTombstone(Slice.make(new ClusteringBound(ClusteringPrefix.Kind.EXCL_START_BOUND, new ByteBuffer[] { one, one }), + new ClusteringBound(ClusteringPrefix.Kind.INCL_END_BOUND, new ByteBuffer[] { two })), new DeletionTime(1, 1) )); From 3a4e00615b14275e5dac535a304add9e1cf4e4eb Mon Sep 17 00:00:00 2001 From: "Dinesh A. Joshi" Date: Thu, 8 Aug 2019 17:08:02 -0700 Subject: [PATCH 54/78] Enhance & reenable RepairTest with compression=off and compression=on patch by Dinesh Joshi; reviewed by Jon Meredith for CASSANDRA-15272 --- CHANGES.txt | 1 + .../distributed/test/RepairTest.java | 71 ++++++++++++------- 2 files changed, 45 insertions(+), 27 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 8ac7fbf457c5..25f52c0bd0be 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Enhance & reenable RepairTest with compression=off and compression=on (CASSANDRA-15272) * Improve readability of Table metrics Virtual tables units (CASSANDRA-15194) * Fix error with non-existent table for nodetool tablehistograms (CASSANDRA-14410) * Avoid result truncation in decimal operations (CASSANDRA-15232) diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java index 143b5cd73ab3..1c488aaf5cfa 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/RepairTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; @@ -38,11 +39,13 @@ import static org.apache.cassandra.distributed.impl.ExecUtil.rethrow; import static org.apache.cassandra.distributed.api.Feature.GOSSIP; import static org.apache.cassandra.distributed.api.Feature.NETWORK; -@Ignore + public class RepairTest extends DistributedTestBase { private static final String insert = withKeyspace("INSERT INTO %s.test (k, c1, c2) VALUES (?, 'value1', 'value2');"); private static final String query = withKeyspace("SELECT k, c1, c2 FROM %s.test WHERE k = ?;"); + private static Cluster cluster; + private static void insert(Cluster cluster, int start, int end, int ... nodes) { for (int i = start ; i < end ; ++i) @@ -71,19 +74,38 @@ private static void flush(Cluster cluster, int ... nodes) cluster.get(node).runOnInstance(rethrow(() -> StorageService.instance.forceKeyspaceFlush(KEYSPACE))); } - private Cluster create(Consumer configModifier) throws IOException + private static Cluster create(Consumer configModifier) throws IOException { configModifier = configModifier.andThen( - config -> config.set("hinted_handoff_enabled", false) - .set("commitlog_sync_batch_window_in_ms", 5) - .with(NETWORK) - .with(GOSSIP) + config -> config.set("hinted_handoff_enabled", false) + .set("commitlog_sync_batch_window_in_ms", 5) + .with(NETWORK) + .with(GOSSIP) ); Cluster cluster = init(Cluster.build(3).withConfig(configModifier).start()); + return cluster; + } + + private void repair(Cluster cluster, Map options) + { + cluster.get(1).runOnInstance(rethrow(() -> { + SimpleCondition await = new SimpleCondition(); + StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> { + if (event.getType() == ProgressEventType.COMPLETE) + await.signalAll(); + })).right.get(); + await.await(1L, MINUTES); + })); + } + + void populate(Cluster cluster, boolean compression) + { try { - cluster.schemaChange(withKeyspace("CREATE TABLE %s.test (k text, c1 text, c2 text, PRIMARY KEY (k));")); + cluster.schemaChange(withKeyspace("DROP TABLE IF EXISTS %s.test;")); + cluster.schemaChange(withKeyspace("CREATE TABLE %s.test (k text, c1 text, c2 text, PRIMARY KEY (k))") + + (compression == false ? " WITH compression = {'enabled' : false};" : ";")); insert(cluster, 0, 1000, 1, 2, 3); flush(cluster, 1); @@ -94,42 +116,37 @@ private Cluster create(Consumer configModifier) throws IOExcepti verify(cluster, 0, 1000, 1, 2, 3); verify(cluster, 1000, 1001, 1, 2); verify(cluster, 1001, 2001, 1, 2, 3); - return cluster; } catch (Throwable t) { cluster.close(); throw t; } - } - private void repair(Cluster cluster, Map options) - { - cluster.get(1).runOnInstance(rethrow(() -> { - SimpleCondition await = new SimpleCondition(); - StorageService.instance.repair(KEYSPACE, options, ImmutableList.of((tag, event) -> { - if (event.getType() == ProgressEventType.COMPLETE) - await.signalAll(); - })).right.get(); - await.await(1L, MINUTES); - })); } - void simpleRepair(boolean orderPreservingPartitioner, boolean sequential) throws IOException + void simpleRepair(Cluster cluster, boolean sequential, boolean compression) throws IOException { - Cluster cluster = create(config -> { - if (orderPreservingPartitioner) - config.set("partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner"); - }); + populate(cluster, compression); repair(cluster, ImmutableMap.of("parallelism", sequential ? "sequential" : "parallel")); verify(cluster, 0, 2001, 1, 2, 3); } - @Test - public void testSimpleSequentialRepair() throws IOException + @BeforeClass + public static void setupCluster() throws IOException { - simpleRepair(false, true); + cluster = create(config -> {}); } + @Ignore("Test requires CASSANDRA-13938 to be merged") + public void testSimpleSequentialRepairDefaultCompression() throws IOException + { + simpleRepair(cluster, true, true); + } + @Test + public void testSimpleSequentialRepairCompressionOff() throws IOException + { + simpleRepair(cluster, true, false); + } } From 0622288c9b2252e31bba0da73a8608ba2aa13ff3 Mon Sep 17 00:00:00 2001 From: Josh Turner Date: Wed, 21 Aug 2019 13:40:03 -0700 Subject: [PATCH 55/78] Fix InternodeOutboundMetrics overloaded bytes/count mixup Patch by Josh Turner; Reviewed by Jeff Jirsa for CASSANDRA-15186 --- CHANGES.txt | 1 + .../cassandra/metrics/InternodeOutboundMetrics.java | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 25f52c0bd0be..0ff429f4f035 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Fix InternodeOutboundMetrics overloaded bytes/count mixup (CASSANDRA-15186) * Enhance & reenable RepairTest with compression=off and compression=on (CASSANDRA-15272) * Improve readability of Table metrics Virtual tables units (CASSANDRA-15194) * Fix error with non-existent table for nodetool tablehistograms (CASSANDRA-14410) diff --git a/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java index 4e64f663b661..f04b42877ba4 100644 --- a/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java +++ b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java @@ -126,8 +126,8 @@ public InternodeOutboundMetrics(InetAddressAndPort ip, final OutboundConnections largeMessageCompletedTasks = Metrics.register(factory.createMetricName("LargeMessageCompletedTasks"),messagingPool.large::sentCount); largeMessageCompletedBytes = Metrics.register(factory.createMetricName("LargeMessageCompletedBytes"),messagingPool.large::sentBytes); largeMessageDropped = Metrics.register(factory.createMetricName("LargeMessageDroppedTasks"), messagingPool.large::dropped); - largeMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToOverload"), messagingPool.large::overloadedBytes); - largeMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToOverload"), messagingPool.large::overloadedCount); + largeMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToOverload"), messagingPool.large::overloadedCount); + largeMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToOverload"), messagingPool.large::overloadedBytes); largeMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToTimeout"), messagingPool.large::expiredCount); largeMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("LargeMessageDroppedBytesDueToTimeout"), messagingPool.large::expiredBytes); largeMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("LargeMessageDroppedTasksDueToError"), messagingPool.large::errorCount); @@ -148,8 +148,8 @@ public InternodeOutboundMetrics(InetAddressAndPort ip, final OutboundConnections urgentMessageCompletedTasks = Metrics.register(factory.createMetricName("UrgentMessageCompletedTasks"), messagingPool.urgent::sentCount); urgentMessageCompletedBytes = Metrics.register(factory.createMetricName("UrgentMessageCompletedBytes"),messagingPool.urgent::sentBytes); urgentMessageDroppedTasks = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasks"), messagingPool.urgent::dropped); - urgentMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToOverload"), messagingPool.urgent::overloadedBytes); - urgentMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToOverload"), messagingPool.urgent::overloadedCount); + urgentMessageDroppedTasksDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToOverload"), messagingPool.urgent::overloadedCount); + urgentMessageDroppedBytesDueToOverload = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToOverload"), messagingPool.urgent::overloadedBytes); urgentMessageDroppedTasksDueToTimeout = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToTimeout"), messagingPool.urgent::expiredCount); urgentMessageDroppedBytesDueToTimeout = Metrics.register(factory.createMetricName("UrgentMessageDroppedBytesDueToTimeout"), messagingPool.urgent::expiredBytes); urgentMessageDroppedTasksDueToError = Metrics.register(factory.createMetricName("UrgentMessageDroppedTasksDueToError"), messagingPool.urgent::errorCount); From a145a485206f6a32b07979dfb1b983addf49dd10 Mon Sep 17 00:00:00 2001 From: Josh Turner Date: Thu, 22 Aug 2019 08:24:38 -0700 Subject: [PATCH 56/78] Fixed exception message in nodetool snapshot when specifying kt-list and keyspace Patch by Josh Turner; Reviewed by Jeff Jirsa for CASSANDRA-15287 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/tools/nodetool/Snapshot.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index 0ff429f4f035..d6bb70140eb4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0 + * Inaccurate exception message with nodetool snapshot (CASSANDRA-15287) * Fix InternodeOutboundMetrics overloaded bytes/count mixup (CASSANDRA-15186) * Enhance & reenable RepairTest with compression=off and compression=on (CASSANDRA-15272) * Improve readability of Table metrics Virtual tables units (CASSANDRA-15194) diff --git a/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java b/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java index 8d01d3a2b96b..495ee9dae539 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java @@ -71,7 +71,7 @@ public void execute(NodeProbe probe) else { throw new IOException( - "When specifying the Keyspace columfamily list for a snapshot, you should not specify columnfamily"); + "When specifying the Keyspace table list (using -kt,--kt-list,-kc,--kc.list), you must not also specify keyspaces to snapshot"); } if (!snapshotName.isEmpty()) sb.append(" with snapshot name [").append(snapshotName).append("]"); From 96ac94dabc63fe98fef22198041e8acba8916267 Mon Sep 17 00:00:00 2001 From: Blake Eggleston Date: Sat, 31 Aug 2019 11:27:19 -0700 Subject: [PATCH 57/78] ninja: fix ant artifacts on java 8 --- build.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/build.xml b/build.xml index c032e1ce4e1e..40d2bf584d84 100644 --- a/build.xml +++ b/build.xml @@ -1107,7 +1107,7 @@ - + @@ -1169,7 +1169,6 @@ From fa167135f86548c7face6a0790ce4fc0a2a2f86c Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Tue, 3 Sep 2019 11:52:24 -0500 Subject: [PATCH 58/78] Set debian/changelog version to 4.0~alpha1 --- debian/changelog | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/debian/changelog b/debian/changelog index 273c6bc2ed9e..b54826ab8299 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,8 +1,8 @@ -cassandra (4.0) UNRELEASED; urgency=medium +cassandra (4.0~alpha1) unstable; urgency=medium * New release - -- Michael Shuler Wed, 15 Feb 2017 18:20:09 -0600 + -- Michael Shuler Tue, 03 Sep 2019 11:51:18 -0500 cassandra (3.10) unstable; urgency=medium From d4054e0cf88bdf85cbde33b6416a6eb20da876e2 Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Wed, 4 Sep 2019 10:31:22 -0700 Subject: [PATCH 59/78] ninja: Fix "No newline at end of file" in c*.yaml After building artifacts, a diff is left over, so let's fix that: diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 28d86fd10a..e776a5a620 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1341,4 +1341,4 @@ enable_sasi_indexes: false # Enables creation of transiently replicated keyspaces on this node. # Transient replication is experimental and is not recommended for production use. -enable_transient_replication: false \ No newline at end of file +enable_transient_replication: false --- conf/cassandra.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 28d86fd10ae3..e776a5a620bc 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1341,4 +1341,4 @@ enable_sasi_indexes: false # Enables creation of transiently replicated keyspaces on this node. # Transient replication is experimental and is not recommended for production use. -enable_transient_replication: false \ No newline at end of file +enable_transient_replication: false From 145f6d6dc445276467ebe456668930f41fbf9127 Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Thu, 5 Sep 2019 12:06:43 -0700 Subject: [PATCH 60/78] Set base.version and changelog to 4.0-alpha1 --- CHANGES.txt | 2 +- build.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d6bb70140eb4..6425b1f15ded 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,4 @@ -4.0 +4.0-alpha1 * Inaccurate exception message with nodetool snapshot (CASSANDRA-15287) * Fix InternodeOutboundMetrics overloaded bytes/count mixup (CASSANDRA-15186) * Enhance & reenable RepairTest with compression=off and compression=on (CASSANDRA-15272) diff --git a/build.xml b/build.xml index 40d2bf584d84..0478338ec224 100644 --- a/build.xml +++ b/build.xml @@ -25,7 +25,7 @@ - + From 4cb90dbbd081a0b6bd5c504404badd6e0d4b3a1c Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Thu, 5 Sep 2019 13:01:53 -0700 Subject: [PATCH 61/78] Use `rm -f` in rpm spec to prevent failure on missing file --- redhat/cassandra.spec | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/redhat/cassandra.spec b/redhat/cassandra.spec index eaf7565922f5..d843f4d633a9 100644 --- a/redhat/cassandra.spec +++ b/redhat/cassandra.spec @@ -74,14 +74,14 @@ patch -p1 < debian/patches/cassandra_logdir_fix.diff sed -i 's/^# hints_directory:/hints_directory:/' conf/cassandra.yaml # remove batch, powershell, and other files not being installed -rm conf/*.ps1 -rm bin/*.bat -rm bin/*.orig -rm bin/*.ps1 -rm bin/cassandra.in.sh -rm lib/sigar-bin/*winnt* # strip segfaults on dll.. -rm tools/bin/*.bat -rm tools/bin/cassandra.in.sh +rm -f conf/*.ps1 +rm -f bin/*.bat +rm -f bin/*.orig +rm -f bin/*.ps1 +rm -f bin/cassandra.in.sh +rm -f lib/sigar-bin/*winnt* # strip segfaults on dll.. +rm -f tools/bin/*.bat +rm -f tools/bin/cassandra.in.sh # copy default configs cp -pr conf/* %{buildroot}/%{_sysconfdir}/%{username}/default.conf/ From 13e43a03086dd623b32a733c72f74993c823ce75 Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Thu, 5 Sep 2019 14:49:16 -0700 Subject: [PATCH 62/78] Define upstream_version in rpm spec to deal with s/~/-/ --- redhat/cassandra.spec | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/redhat/cassandra.spec b/redhat/cassandra.spec index d843f4d633a9..5bc77e521e87 100644 --- a/redhat/cassandra.spec +++ b/redhat/cassandra.spec @@ -8,7 +8,9 @@ %global username cassandra -%define relname apache-cassandra-%{version} +# input of ~alphaN, ~betaN, ~rcN package versions need to retain upstream '-alphaN, etc' version for sources +%define upstream_version %(echo %{version} | sed -r 's/~/-/g') +%define relname apache-cassandra-%{upstream_version} Name: cassandra Version: %{version} From fc4381ca89ab39a82c9018e5171975285cc3bfe7 Mon Sep 17 00:00:00 2001 From: Michael Shuler Date: Thu, 5 Sep 2019 15:07:01 -0700 Subject: [PATCH 63/78] Add auditlogviewer and fqltool to rpm spec --- redhat/cassandra.spec | 2 ++ 1 file changed, 2 insertions(+) diff --git a/redhat/cassandra.spec b/redhat/cassandra.spec index 5bc77e521e87..ca5d38e3e7d5 100644 --- a/redhat/cassandra.spec +++ b/redhat/cassandra.spec @@ -120,10 +120,12 @@ exit 0 %files %defattr(0644,root,root,0755) %doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt +%attr(755,root,root) %{_bindir}/auditlogviewer %attr(755,root,root) %{_bindir}/cassandra-stress %attr(755,root,root) %{_bindir}/cqlsh %attr(755,root,root) %{_bindir}/cqlsh.py %attr(755,root,root) %{_bindir}/debug-cql +%attr(755,root,root) %{_bindir}/fqltool %attr(755,root,root) %{_bindir}/nodetool %attr(755,root,root) %{_bindir}/sstableloader %attr(755,root,root) %{_bindir}/sstablescrub From 068d2d37c6fbdb60546821c4d408a84161fd1cb6 Mon Sep 17 00:00:00 2001 From: Mick Semb Wever Date: Mon, 5 Aug 2019 00:06:30 +0200 Subject: [PATCH 64/78] Add `allocate_tokens_for_local_rf` yaml option for token allocation that doesn't require keyspace knowledge/existence patch by Mick Semb Wever; reviewed by Branimir Lambov for CASSANDRA-15260 --- CHANGES.txt | 3 +++ conf/cassandra.yaml | 10 ++++++-- .../org/apache/cassandra/config/Config.java | 2 ++ .../cassandra/config/DatabaseDescriptor.java | 5 ++++ .../apache/cassandra/dht/BootStrapper.java | 22 ++++++++++++++++ .../cassandra/dht/BootstrapDiagnostics.java | 16 ++++++++++++ .../apache/cassandra/dht/BootstrapEvent.java | 6 ++++- .../dht/tokenallocator/TokenAllocation.java | 25 +++++++++++++++++++ .../cassandra/dht/BootStrapperTest.java | 19 ++++++++++++++ 9 files changed, 105 insertions(+), 3 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 6425b1f15ded..709e4365f965 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +4.0-alpha2 + * Add `allocate_tokens_for_local_replication_factor` option for token allocation (CASSANDRA-15260) + 4.0-alpha1 * Inaccurate exception message with nodetool snapshot (CASSANDRA-15287) * Fix InternodeOutboundMetrics overloaded bytes/count mixup (CASSANDRA-15186) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index e776a5a620bc..f3e5c7507c08 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -26,15 +26,21 @@ num_tokens: 256 # Triggers automatic allocation of num_tokens tokens for this node. The allocation # algorithm attempts to choose tokens in a way that optimizes replicated load over -# the nodes in the datacenter for the replication strategy used by the specified -# keyspace. +# the nodes in the datacenter for the replica factor. # # The load assigned to each node will be close to proportional to its number of # vnodes. # # Only supported with the Murmur3Partitioner. + +# Replica factor is determined via the replication strategy used by the specified +# keyspace. # allocate_tokens_for_keyspace: KEYSPACE +# Replica factor is explicitly set, regardless of keyspace or datacenter. +# This is the replica factor within the datacenter, like NTS. +# allocate_tokens_for_local_replication_factor: 3 + # initial_token allows you to specify tokens manually. While you can use it with # vnodes (num_tokens > 1, above) -- in which case you should provide a # comma-separated list -- it's primarily used when adding nodes to legacy clusters diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index a1fdfdc786a4..b86b7c57e690 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -86,6 +86,8 @@ public class Config public int num_tokens = 1; /** Triggers automatic allocation of tokens if set, using the replication strategy of the referenced keyspace */ public String allocate_tokens_for_keyspace = null; + /** Triggers automatic allocation of tokens if set, based on the provided replica count for a datacenter */ + public Integer allocate_tokens_for_local_replication_factor = null; public long native_transport_idle_timeout_in_ms = 0L; diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 43203e50c431..e4ea611b734f 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -1422,6 +1422,11 @@ public static String getAllocateTokensForKeyspace() return System.getProperty(Config.PROPERTY_PREFIX + "allocate_tokens_for_keyspace", conf.allocate_tokens_for_keyspace); } + public static Integer getAllocateTokensForLocalRf() + { + return conf.allocate_tokens_for_local_replication_factor; + } + public static Collection tokensFromString(String tokenString) { List tokens = new ArrayList(); diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java index f5c455ccc14a..94bf283e435b 100644 --- a/src/java/org/apache/cassandra/dht/BootStrapper.java +++ b/src/java/org/apache/cassandra/dht/BootStrapper.java @@ -24,6 +24,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.db.Keyspace; @@ -32,6 +33,7 @@ import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.locator.AbstractReplicationStrategy; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.NetworkTopologyStrategy; import org.apache.cassandra.locator.TokenMetadata; import org.apache.cassandra.service.StorageService; import org.apache.cassandra.streaming.*; @@ -153,6 +155,7 @@ public void onFailure(Throwable throwable) public static Collection getBootstrapTokens(final TokenMetadata metadata, InetAddressAndPort address, int schemaWaitDelay) throws ConfigurationException { String allocationKeyspace = DatabaseDescriptor.getAllocateTokensForKeyspace(); + Integer allocationLocalRf = DatabaseDescriptor.getAllocateTokensForLocalRf(); Collection initialTokens = DatabaseDescriptor.getInitialTokens(); if (initialTokens.size() > 0 && allocationKeyspace != null) logger.warn("manually specified tokens override automatic allocation"); @@ -172,6 +175,9 @@ public static Collection getBootstrapTokens(final TokenMetadata metadata, if (allocationKeyspace != null) return allocateTokens(metadata, address, allocationKeyspace, numTokens, schemaWaitDelay); + if (allocationLocalRf != null) + return allocateTokens(metadata, address, allocationLocalRf, numTokens, schemaWaitDelay); + if (numTokens == 1) logger.warn("Picking random token for a single vnode. You should probably add more vnodes and/or use the automatic token allocation mechanism."); @@ -215,6 +221,22 @@ static Collection allocateTokens(final TokenMetadata metadata, return tokens; } + + static Collection allocateTokens(final TokenMetadata metadata, + InetAddressAndPort address, + int rf, + int numTokens, + int schemaWaitDelay) + { + StorageService.instance.waitForSchema(schemaWaitDelay); + if (!FBUtilities.getBroadcastAddressAndPort().equals(InetAddressAndPort.getLoopbackAddress())) + Gossiper.waitToSettle(); + + Collection tokens = TokenAllocation.allocateTokens(metadata, rf, address, numTokens); + BootstrapDiagnostics.tokensAllocated(address, metadata, rf, numTokens, tokens); + return tokens; + } + public static Collection getRandomTokens(TokenMetadata metadata, int numTokens) { Set tokens = new HashSet<>(numTokens); diff --git a/src/java/org/apache/cassandra/dht/BootstrapDiagnostics.java b/src/java/org/apache/cassandra/dht/BootstrapDiagnostics.java index 56955326de3f..5c2b46a03087 100644 --- a/src/java/org/apache/cassandra/dht/BootstrapDiagnostics.java +++ b/src/java/org/apache/cassandra/dht/BootstrapDiagnostics.java @@ -45,6 +45,7 @@ static void useSpecifiedTokens(InetAddressAndPort address, String allocationKeys address, null, allocationKeyspace, + null, numTokens, ImmutableList.copyOf(initialTokens))); } @@ -56,6 +57,7 @@ static void useRandomTokens(InetAddressAndPort address, TokenMetadata metadata, address, metadata.cloneOnlyTokenMap(), null, + null, numTokens, ImmutableList.copyOf(tokens))); } @@ -68,6 +70,20 @@ static void tokensAllocated(InetAddressAndPort address, TokenMetadata metadata, address, metadata.cloneOnlyTokenMap(), allocationKeyspace, + null, + numTokens, + ImmutableList.copyOf(tokens))); + } + + static void tokensAllocated(InetAddressAndPort address, TokenMetadata metadata, + int rf, int numTokens, Collection tokens) + { + if (isEnabled(BootstrapEventType.TOKENS_ALLOCATED)) + service.publish(new BootstrapEvent(BootstrapEventType.TOKENS_ALLOCATED, + address, + metadata.cloneOnlyTokenMap(), + null, + rf, numTokens, ImmutableList.copyOf(tokens))); } diff --git a/src/java/org/apache/cassandra/dht/BootstrapEvent.java b/src/java/org/apache/cassandra/dht/BootstrapEvent.java index 5bad09a19fa1..4936c2942ad9 100644 --- a/src/java/org/apache/cassandra/dht/BootstrapEvent.java +++ b/src/java/org/apache/cassandra/dht/BootstrapEvent.java @@ -42,16 +42,19 @@ final class BootstrapEvent extends DiagnosticEvent private final InetAddressAndPort address; @Nullable private final String allocationKeyspace; + @Nullable + private final Integer rf; private final Integer numTokens; private final Collection tokens; BootstrapEvent(BootstrapEventType type, InetAddressAndPort address, @Nullable TokenMetadata tokenMetadata, - @Nullable String allocationKeyspace, int numTokens, ImmutableCollection tokens) + @Nullable String allocationKeyspace, @Nullable Integer rf, int numTokens, ImmutableCollection tokens) { this.type = type; this.address = address; this.tokenMetadata = tokenMetadata; this.allocationKeyspace = allocationKeyspace; + this.rf = rf; this.numTokens = numTokens; this.tokens = tokens; } @@ -75,6 +78,7 @@ public Map toMap() HashMap ret = new HashMap<>(); ret.put("tokenMetadata", String.valueOf(tokenMetadata)); ret.put("allocationKeyspace", allocationKeyspace); + ret.put("rf", rf); ret.put("numTokens", numTokens); ret.put("tokens", String.valueOf(tokens)); return ret; diff --git a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java index ba93eb8b9688..bfa281e0c270 100644 --- a/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java +++ b/src/java/org/apache/cassandra/dht/tokenallocator/TokenAllocation.java @@ -32,6 +32,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.locator.AbstractReplicationStrategy; @@ -72,6 +73,20 @@ public static Collection allocateTokens(final TokenMetadata tokenMetadata return tokens; } + public static Collection allocateTokens(final TokenMetadata tokenMetadata, + final int replicas, + final InetAddressAndPort endpoint, + int numTokens) + { + TokenMetadata tokenMetadataCopy = tokenMetadata.cloneOnlyTokenMap(); + StrategyAdapter strategy = getStrategy(tokenMetadataCopy, replicas, endpoint); + Collection tokens = create(tokenMetadata, strategy).addUnit(endpoint, numTokens); + tokens = adjustForCrossDatacenterClashes(tokenMetadata, strategy, tokens); + logger.warn("Selected tokens {}", tokens); + // SummaryStatistics is not implemented for `allocate_tokens_for_local_replication_factor` + return tokens; + } + private static Collection adjustForCrossDatacenterClashes(final TokenMetadata tokenMetadata, StrategyAdapter strategy, Collection tokens) { @@ -197,7 +212,17 @@ static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final Netw { final String dc = snitch.getDatacenter(endpoint); final int replicas = rs.getReplicationFactor(dc).allReplicas; + return getStrategy(tokenMetadata, replicas, snitch, endpoint); + } + + static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final int replicas, final InetAddressAndPort endpoint) + { + return getStrategy(tokenMetadata, replicas, DatabaseDescriptor.getEndpointSnitch(), endpoint); + } + static StrategyAdapter getStrategy(final TokenMetadata tokenMetadata, final int replicas, final IEndpointSnitch snitch, final InetAddressAndPort endpoint) + { + final String dc = snitch.getDatacenter(endpoint); if (replicas == 0 || replicas == 1) { // No replication, each node is treated as separate. diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java index 2f412ad63793..c0b6d5cb70ea 100644 --- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java +++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java @@ -174,6 +174,17 @@ public void testAllocateTokens() throws UnknownHostException allocateTokensForNode(vn, ks, tm, addr); } + @Test + public void testAllocateTokensLocalRf() throws UnknownHostException + { + int vn = 16; + int allocateTokensForLocalRf = 3; + TokenMetadata tm = new TokenMetadata(); + generateFakeEndpoints(tm, 10, vn); + InetAddressAndPort addr = FBUtilities.getBroadcastAddressAndPort(); + allocateTokensForNode(vn, allocateTokensForLocalRf, tm, addr); + } + public void testAllocateTokensNetworkStrategy(int rackCount, int replicas) throws UnknownHostException { IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch(); @@ -243,6 +254,14 @@ private void allocateTokensForNode(int vn, String ks, TokenMetadata tm, InetAddr verifyImprovement(os, ns); } + private void allocateTokensForNode(int vn, int rf, TokenMetadata tm, InetAddressAndPort addr) + { + Collection tokens = BootStrapper.allocateTokens(tm, addr, rf, vn, 0); + assertEquals(vn, tokens.size()); + tm.updateNormalTokens(tokens, addr); + // SummaryStatistics is not implemented for `allocate_tokens_for_local_replication_factor` so can't be verified + } + private void verifyImprovement(SummaryStatistics os, SummaryStatistics ns) { if (ns.getStandardDeviation() > os.getStandardDeviation()) From 21ebba85ac4a78dc39eaaa8816e3cd2f93467552 Mon Sep 17 00:00:00 2001 From: Mick Semb Wever Date: Tue, 10 Sep 2019 07:17:51 +0200 Subject: [PATCH 65/78] Update version to 4.0-alpha2 --- build.xml | 2 +- debian/changelog | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/build.xml b/build.xml index 0478338ec224..4828389893ea 100644 --- a/build.xml +++ b/build.xml @@ -25,7 +25,7 @@ - + diff --git a/debian/changelog b/debian/changelog index b54826ab8299..edd48a8aa0e1 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (4.0~alpha2) UNRELEASED; urgency=medium + + * New release + + -- + cassandra (4.0~alpha1) unstable; urgency=medium * New release From a7a0e7fe40cd375c647d2f5e56d04fd98963cff2 Mon Sep 17 00:00:00 2001 From: "xuanling.gc" Date: Tue, 3 Sep 2019 11:13:42 +0800 Subject: [PATCH 66/78] Add a new snitch for Alibaba cloud platform Closes #350 Patch by xuanling.gc, reviewed by Brandon Williams for CASSANDRA-15092 --- CHANGES.txt | 2 + .../cassandra/locator/AlibabaCloudSnitch.java | 146 ++++++++++++++++++ .../locator/AlibabaCloudSnitchTest.java | 103 ++++++++++++ 3 files changed, 251 insertions(+) create mode 100644 src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java create mode 100644 test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java diff --git a/CHANGES.txt b/CHANGES.txt index 709e4365f965..2929a980da09 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,7 @@ 4.0-alpha2 * Add `allocate_tokens_for_local_replication_factor` option for token allocation (CASSANDRA-15260) + * Add Alibaba Cloud Platform snitch (CASSANDRA-15092) + 4.0-alpha1 * Inaccurate exception message with nodetool snapshot (CASSANDRA-15287) diff --git a/src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java b/src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java new file mode 100644 index 000000000000..729e1b376393 --- /dev/null +++ b/src/java/org/apache/cassandra/locator/AlibabaCloudSnitch.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.locator; + +import java.io.DataInputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.MalformedURLException; +import java.net.SocketTimeoutException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.EndpointState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.FBUtilities; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A snitch that assumes an ECS region is a DC and an ECS availability_zone + * is a rack. This information is available in the config for the node. the + * format of the zone-id is like :cn-hangzhou-a where cn means china, hangzhou + * means the hangzhou region, a means the az id. We use cn-hangzhou as the dc, + * and f as the zone-id. + */ +public class AlibabaCloudSnitch extends AbstractNetworkTopologySnitch +{ + protected static final Logger logger = LoggerFactory.getLogger(AlibabaCloudSnitch.class); + protected static final String ZONE_NAME_QUERY_URL = "http://100.100.100.200/latest/meta-data/zone-id"; + private static final String DEFAULT_DC = "UNKNOWN-DC"; + private static final String DEFAULT_RACK = "UNKNOWN-RACK"; + private Map> savedEndpoints; + protected String ecsZone; + protected String ecsRegion; + + private static final int HTTP_CONNECT_TIMEOUT = 30000; + + + public AlibabaCloudSnitch() throws MalformedURLException, IOException + { + String response = alibabaApiCall(ZONE_NAME_QUERY_URL); + String[] splits = response.split("/"); + String az = splits[splits.length - 1]; + + // Split "us-central1-a" or "asia-east1-a" into "us-central1"/"a" and "asia-east1"/"a". + splits = az.split("-"); + ecsZone = splits[splits.length - 1]; + + int lastRegionIndex = az.lastIndexOf("-"); + ecsRegion = az.substring(0, lastRegionIndex); + + String datacenterSuffix = (new SnitchProperties()).get("dc_suffix", ""); + ecsRegion = ecsRegion.concat(datacenterSuffix); + logger.info("AlibabaSnitch using region: {}, zone: {}.", ecsRegion, ecsZone); + + } + + String alibabaApiCall(String url) throws ConfigurationException, IOException, SocketTimeoutException + { + // Populate the region and zone by introspection, fail if 404 on metadata + HttpURLConnection conn = (HttpURLConnection) new URL(url).openConnection(); + DataInputStream d = null; + try + { + conn.setConnectTimeout(HTTP_CONNECT_TIMEOUT); + conn.setRequestMethod("GET"); + + int code = conn.getResponseCode(); + if (code != HttpURLConnection.HTTP_OK) + throw new ConfigurationException("AlibabaSnitch was unable to execute the API call. Not an ecs node? and the returun code is " + code); + + // Read the information. I wish I could say (String) conn.getContent() here... + int cl = conn.getContentLength(); + byte[] b = new byte[cl]; + d = new DataInputStream((FilterInputStream) conn.getContent()); + d.readFully(b); + return new String(b, StandardCharsets.UTF_8); + } + catch (SocketTimeoutException e) + { + throw new SocketTimeoutException("Timeout occurred reading a response from the Alibaba ECS metadata"); + } + finally + { + FileUtils.close(d); + conn.disconnect(); + } + } + + @Override + public String getRack(InetAddressAndPort endpoint) + { + if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) + return ecsZone; + EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint); + if (state == null || state.getApplicationState(ApplicationState.RACK) == null) + { + if (savedEndpoints == null) + savedEndpoints = SystemKeyspace.loadDcRackInfo(); + if (savedEndpoints.containsKey(endpoint)) + return savedEndpoints.get(endpoint).get("rack"); + return DEFAULT_RACK; + } + return state.getApplicationState(ApplicationState.RACK).value; + + } + + @Override + public String getDatacenter(InetAddressAndPort endpoint) + { + if (endpoint.equals(FBUtilities.getBroadcastAddressAndPort())) + return ecsRegion; + EndpointState state = Gossiper.instance.getEndpointStateForEndpoint(endpoint); + if (state == null || state.getApplicationState(ApplicationState.DC) == null) + { + if (savedEndpoints == null) + savedEndpoints = SystemKeyspace.loadDcRackInfo(); + if (savedEndpoints.containsKey(endpoint)) + return savedEndpoints.get(endpoint).get("data_center"); + return DEFAULT_DC; + } + return state.getApplicationState(ApplicationState.DC).value; + + } + +} diff --git a/test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java new file mode 100644 index 000000000000..4e8ab164634c --- /dev/null +++ b/test/unit/org/apache/cassandra/locator/AlibabaCloudSnitchTest.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.locator; + +import static org.junit.Assert.assertEquals; +import java.io.IOException; +import java.net.InetAddress; +import java.util.EnumMap; +import java.util.Map; +import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.exceptions.ConfigurationException; +import org.apache.cassandra.gms.ApplicationState; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.VersionedValue; +import org.apache.cassandra.service.StorageService; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +public class AlibabaCloudSnitchTest +{ + private static String az; + + @BeforeClass + public static void setup() throws Exception + { + System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true"); + DatabaseDescriptor.daemonInitialization(); + SchemaLoader.mkdirs(); + SchemaLoader.cleanup(); + Keyspace.setInitialized(); + StorageService.instance.initServer(0); + } + + private class TestAlibabaCloudSnitch extends AlibabaCloudSnitch + { + public TestAlibabaCloudSnitch() throws IOException, ConfigurationException + { + super(); + } + + @Override + String alibabaApiCall(String url) throws IOException, ConfigurationException + { + return az; + } + } + + @Test + public void testRac() throws IOException, ConfigurationException + { + az = "cn-hangzhou-f"; + AlibabaCloudSnitch snitch = new TestAlibabaCloudSnitch(); + InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); + InetAddressAndPort nonlocal = InetAddressAndPort.getByName("127.0.0.7"); + + Gossiper.instance.addSavedEndpoint(nonlocal); + Map stateMap = new EnumMap<>(ApplicationState.class); + stateMap.put(ApplicationState.DC, StorageService.instance.valueFactory.datacenter("cn-shanghai")); + stateMap.put(ApplicationState.RACK, StorageService.instance.valueFactory.datacenter("a")); + Gossiper.instance.getEndpointStateForEndpoint(nonlocal).addApplicationStates(stateMap); + + assertEquals("cn-shanghai", snitch.getDatacenter(nonlocal)); + assertEquals("a", snitch.getRack(nonlocal)); + + assertEquals("cn-hangzhou", snitch.getDatacenter(local)); + assertEquals("f", snitch.getRack(local)); + } + + @Test + public void testNewRegions() throws IOException, ConfigurationException + { + az = "us-east-1a"; + AlibabaCloudSnitch snitch = new TestAlibabaCloudSnitch(); + InetAddressAndPort local = InetAddressAndPort.getByName("127.0.0.1"); + assertEquals("us-east", snitch.getDatacenter(local)); + assertEquals("1a", snitch.getRack(local)); + } + + @AfterClass + public static void tearDown() + { + StorageService.instance.stopClient(); + } + +} From f9ff88437742675db5c53f5834884b43f8937e00 Mon Sep 17 00:00:00 2001 From: Aleksey Yeschenko Date: Fri, 14 Jun 2019 14:49:46 +0100 Subject: [PATCH 67/78] Untangle RepairMessage sub-hierarchy of messages, use new messaging (more) correctly patch by Aleksey Yeschenko; reviewed by Blake Eggleston for CASSANDRA-15163 --- CHANGES.txt | 1 + src/java/org/apache/cassandra/net/Verb.java | 152 +++++++++++------- .../repair/AsymmetricRemoteSyncTask.java | 4 +- .../repair/RepairMessageVerbHandler.java | 36 ++--- .../apache/cassandra/repair/SnapshotTask.java | 4 +- .../cassandra/repair/StreamingRepairTask.java | 10 +- .../repair/SymmetricRemoteSyncTask.java | 6 +- .../cassandra/repair/ValidationTask.java | 4 +- .../apache/cassandra/repair/Validator.java | 16 +- .../repair/consistent/CoordinatorSession.java | 16 +- .../repair/consistent/LocalSessions.java | 46 +++--- .../messages/AsymmetricSyncRequest.java | 14 +- .../repair/messages/CleanupMessage.java | 13 +- .../repair/messages/FailSession.java | 5 +- .../repair/messages/FinalizeCommit.java | 5 +- .../repair/messages/FinalizePromise.java | 5 +- .../repair/messages/FinalizePropose.java | 5 +- .../messages/PrepareConsistentRequest.java | 6 +- .../messages/PrepareConsistentResponse.java | 5 +- .../repair/messages/PrepareMessage.java | 24 ++- .../repair/messages/RepairMessage.java | 85 +--------- .../repair/messages/SnapshotMessage.java | 13 +- .../repair/messages/StatusRequest.java | 5 +- .../repair/messages/StatusResponse.java | 5 +- .../repair/messages/SyncRequest.java | 14 +- .../{SyncComplete.java => SyncResponse.java} | 34 ++-- .../repair/messages/ValidationRequest.java | 9 +- ...nComplete.java => ValidationResponse.java} | 42 +++-- .../service/ActiveRepairService.java | 22 ++- .../4.0/service.SyncComplete.bin | Bin 258 -> 256 bytes .../serialization/4.0/service.SyncRequest.bin | Bin 111 -> 110 bytes .../4.0/service.ValidationComplete.bin | Bin 600 -> 597 bytes .../4.0/service.ValidationRequest.bin | Bin 75 -> 74 bytes .../cassandra/repair/RepairJobTest.java | 22 +-- .../repair/SymmetricRemoteSyncTaskTest.java | 2 +- .../cassandra/repair/ValidatorTest.java | 37 ++--- .../consistent/CoordinatorMessagingTest.java | 44 ++--- .../consistent/CoordinatorSessionTest.java | 5 +- .../repair/consistent/LocalSessionTest.java | 6 +- .../RepairMessageSerializationsTest.java | 18 +-- .../messages/RepairMessageSerializerTest.java | 41 +++-- .../cassandra/service/SerializationsTest.java | 81 +++++----- 42 files changed, 400 insertions(+), 462 deletions(-) rename src/java/org/apache/cassandra/repair/messages/{SyncComplete.java => SyncResponse.java} (79%) rename src/java/org/apache/cassandra/repair/messages/{ValidationComplete.java => ValidationResponse.java} (70%) diff --git a/CHANGES.txt b/CHANGES.txt index 2929a980da09..1a3df81bcb2a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0-alpha2 + * Untangle RepairMessage sub-hierarchy of messages, use new messaging (more) correctly (CASSANDRA-15163) * Add `allocate_tokens_for_local_replication_factor` option for token allocation (CASSANDRA-15260) * Add Alibaba Cloud Platform snitch (CASSANDRA-15092) diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index 6463a5a1ed1b..67d847e939eb 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -56,7 +56,22 @@ import org.apache.cassandra.hints.HintVerbHandler; import org.apache.cassandra.io.IVersionedAsymmetricSerializer; import org.apache.cassandra.repair.RepairMessageVerbHandler; -import org.apache.cassandra.repair.messages.RepairMessage; +import org.apache.cassandra.repair.messages.AsymmetricSyncRequest; +import org.apache.cassandra.repair.messages.CleanupMessage; +import org.apache.cassandra.repair.messages.FailSession; +import org.apache.cassandra.repair.messages.FinalizeCommit; +import org.apache.cassandra.repair.messages.FinalizePromise; +import org.apache.cassandra.repair.messages.FinalizePropose; +import org.apache.cassandra.repair.messages.PrepareConsistentRequest; +import org.apache.cassandra.repair.messages.PrepareConsistentResponse; +import org.apache.cassandra.repair.messages.PrepareMessage; +import org.apache.cassandra.repair.messages.SnapshotMessage; +import org.apache.cassandra.repair.messages.StatusRequest; +import org.apache.cassandra.repair.messages.StatusResponse; +import org.apache.cassandra.repair.messages.SyncResponse; +import org.apache.cassandra.repair.messages.SyncRequest; +import org.apache.cassandra.repair.messages.ValidationResponse; +import org.apache.cassandra.repair.messages.ValidationRequest; import org.apache.cassandra.schema.SchemaPullVerbHandler; import org.apache.cassandra.schema.SchemaPushVerbHandler; import org.apache.cassandra.schema.SchemaVersionVerbHandler; @@ -84,75 +99,92 @@ */ public enum Verb { - MUTATION_RSP (60, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - MUTATION_REQ (0, P3, writeTimeout, MUTATION, () -> Mutation.serializer, () -> MutationVerbHandler.instance, MUTATION_RSP ), - HINT_RSP (61, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - HINT_REQ (1, P4, writeTimeout, MUTATION, () -> HintMessage.serializer, () -> HintVerbHandler.instance, HINT_RSP ), - READ_REPAIR_RSP (62, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - READ_REPAIR_REQ (2, P1, writeTimeout, MUTATION, () -> Mutation.serializer, () -> ReadRepairVerbHandler.instance, READ_REPAIR_RSP ), - BATCH_STORE_RSP (65, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - BATCH_STORE_REQ (5, P3, writeTimeout, MUTATION, () -> Batch.serializer, () -> BatchStoreVerbHandler.instance, BATCH_STORE_RSP ), - BATCH_REMOVE_RSP (66, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - BATCH_REMOVE_REQ (6, P3, writeTimeout, MUTATION, () -> UUIDSerializer.serializer, () -> BatchRemoveVerbHandler.instance, BATCH_REMOVE_RSP ), - - PAXOS_PREPARE_RSP (93, P2, writeTimeout, REQUEST_RESPONSE, () -> PrepareResponse.serializer, () -> ResponseVerbHandler.instance ), - PAXOS_PREPARE_REQ (33, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> PrepareVerbHandler.instance, PAXOS_PREPARE_RSP ), - PAXOS_PROPOSE_RSP (94, P2, writeTimeout, REQUEST_RESPONSE, () -> BooleanSerializer.serializer, () -> ResponseVerbHandler.instance ), - PAXOS_PROPOSE_REQ (34, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> ProposeVerbHandler.instance, PAXOS_PROPOSE_RSP ), - PAXOS_COMMIT_RSP (95, P2, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - PAXOS_COMMIT_REQ (35, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> CommitVerbHandler.instance, PAXOS_COMMIT_RSP ), - - TRUNCATE_RSP (79, P0, truncateTimeout, REQUEST_RESPONSE, () -> TruncateResponse.serializer, () -> ResponseVerbHandler.instance ), - TRUNCATE_REQ (19, P0, truncateTimeout, MUTATION, () -> TruncateRequest.serializer, () -> TruncateVerbHandler.instance, TRUNCATE_RSP ), - - COUNTER_MUTATION_RSP (84, P1, counterTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - COUNTER_MUTATION_REQ (24, P2, counterTimeout, COUNTER_MUTATION, () -> CounterMutation.serializer, () -> CounterMutationVerbHandler.instance, COUNTER_MUTATION_RSP), - - READ_RSP (63, P2, readTimeout, REQUEST_RESPONSE, () -> ReadResponse.serializer, () -> ResponseVerbHandler.instance ), - READ_REQ (3, P3, readTimeout, READ, () -> ReadCommand.serializer, () -> ReadCommandVerbHandler.instance, READ_RSP ), - RANGE_RSP (69, P2, rangeTimeout, REQUEST_RESPONSE, () -> ReadResponse.serializer, () -> ResponseVerbHandler.instance ), - RANGE_REQ (9, P3, rangeTimeout, READ, () -> ReadCommand.serializer, () -> ReadCommandVerbHandler.instance, RANGE_RSP ), - - GOSSIP_DIGEST_SYN (14, P0, longTimeout, GOSSIP, () -> GossipDigestSyn.serializer, () -> GossipDigestSynVerbHandler.instance ), - GOSSIP_DIGEST_ACK (15, P0, longTimeout, GOSSIP, () -> GossipDigestAck.serializer, () -> GossipDigestAckVerbHandler.instance ), - GOSSIP_DIGEST_ACK2 (16, P0, longTimeout, GOSSIP, () -> GossipDigestAck2.serializer, () -> GossipDigestAck2VerbHandler.instance ), - GOSSIP_SHUTDOWN (29, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> GossipShutdownVerbHandler.instance ), - - ECHO_RSP (91, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - ECHO_REQ (31, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> EchoVerbHandler.instance, ECHO_RSP ), - PING_RSP (97, P1, pingTimeout, GOSSIP, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - PING_REQ (37, P1, pingTimeout, GOSSIP, () -> PingRequest.serializer, () -> PingVerbHandler.instance, PING_RSP ), + MUTATION_RSP (60, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + MUTATION_REQ (0, P3, writeTimeout, MUTATION, () -> Mutation.serializer, () -> MutationVerbHandler.instance, MUTATION_RSP ), + HINT_RSP (61, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + HINT_REQ (1, P4, writeTimeout, MUTATION, () -> HintMessage.serializer, () -> HintVerbHandler.instance, HINT_RSP ), + READ_REPAIR_RSP (62, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + READ_REPAIR_REQ (2, P1, writeTimeout, MUTATION, () -> Mutation.serializer, () -> ReadRepairVerbHandler.instance, READ_REPAIR_RSP ), + BATCH_STORE_RSP (65, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + BATCH_STORE_REQ (5, P3, writeTimeout, MUTATION, () -> Batch.serializer, () -> BatchStoreVerbHandler.instance, BATCH_STORE_RSP ), + BATCH_REMOVE_RSP (66, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + BATCH_REMOVE_REQ (6, P3, writeTimeout, MUTATION, () -> UUIDSerializer.serializer, () -> BatchRemoveVerbHandler.instance, BATCH_REMOVE_RSP ), + + PAXOS_PREPARE_RSP (93, P2, writeTimeout, REQUEST_RESPONSE, () -> PrepareResponse.serializer, () -> ResponseVerbHandler.instance ), + PAXOS_PREPARE_REQ (33, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> PrepareVerbHandler.instance, PAXOS_PREPARE_RSP ), + PAXOS_PROPOSE_RSP (94, P2, writeTimeout, REQUEST_RESPONSE, () -> BooleanSerializer.serializer, () -> ResponseVerbHandler.instance ), + PAXOS_PROPOSE_REQ (34, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> ProposeVerbHandler.instance, PAXOS_PROPOSE_RSP ), + PAXOS_COMMIT_RSP (95, P2, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + PAXOS_COMMIT_REQ (35, P2, writeTimeout, MUTATION, () -> Commit.serializer, () -> CommitVerbHandler.instance, PAXOS_COMMIT_RSP ), + + TRUNCATE_RSP (79, P0, truncateTimeout, REQUEST_RESPONSE, () -> TruncateResponse.serializer, () -> ResponseVerbHandler.instance ), + TRUNCATE_REQ (19, P0, truncateTimeout, MUTATION, () -> TruncateRequest.serializer, () -> TruncateVerbHandler.instance, TRUNCATE_RSP ), + + COUNTER_MUTATION_RSP (84, P1, counterTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + COUNTER_MUTATION_REQ (24, P2, counterTimeout, COUNTER_MUTATION, () -> CounterMutation.serializer, () -> CounterMutationVerbHandler.instance, COUNTER_MUTATION_RSP), + + READ_RSP (63, P2, readTimeout, REQUEST_RESPONSE, () -> ReadResponse.serializer, () -> ResponseVerbHandler.instance ), + READ_REQ (3, P3, readTimeout, READ, () -> ReadCommand.serializer, () -> ReadCommandVerbHandler.instance, READ_RSP ), + RANGE_RSP (69, P2, rangeTimeout, REQUEST_RESPONSE, () -> ReadResponse.serializer, () -> ResponseVerbHandler.instance ), + RANGE_REQ (9, P3, rangeTimeout, READ, () -> ReadCommand.serializer, () -> ReadCommandVerbHandler.instance, RANGE_RSP ), + + GOSSIP_DIGEST_SYN (14, P0, longTimeout, GOSSIP, () -> GossipDigestSyn.serializer, () -> GossipDigestSynVerbHandler.instance ), + GOSSIP_DIGEST_ACK (15, P0, longTimeout, GOSSIP, () -> GossipDigestAck.serializer, () -> GossipDigestAckVerbHandler.instance ), + GOSSIP_DIGEST_ACK2 (16, P0, longTimeout, GOSSIP, () -> GossipDigestAck2.serializer, () -> GossipDigestAck2VerbHandler.instance ), + GOSSIP_SHUTDOWN (29, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> GossipShutdownVerbHandler.instance ), + + ECHO_RSP (91, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + ECHO_REQ (31, P0, rpcTimeout, GOSSIP, () -> NoPayload.serializer, () -> EchoVerbHandler.instance, ECHO_RSP ), + PING_RSP (97, P1, pingTimeout, GOSSIP, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + PING_REQ (37, P1, pingTimeout, GOSSIP, () -> PingRequest.serializer, () -> PingVerbHandler.instance, PING_RSP ), // P1 because messages can be arbitrarily large or aren't crucial - SCHEMA_PUSH_RSP (98, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - SCHEMA_PUSH_REQ (18, P1, rpcTimeout, MIGRATION, () -> MigrationsSerializer.instance, () -> SchemaPushVerbHandler.instance, SCHEMA_PUSH_RSP ), - SCHEMA_PULL_RSP (88, P1, rpcTimeout, MIGRATION, () -> MigrationsSerializer.instance, () -> ResponseVerbHandler.instance ), - SCHEMA_PULL_REQ (28, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> SchemaPullVerbHandler.instance, SCHEMA_PULL_RSP ), - SCHEMA_VERSION_RSP (80, P1, rpcTimeout, MIGRATION, () -> UUIDSerializer.serializer, () -> ResponseVerbHandler.instance ), - SCHEMA_VERSION_REQ (20, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> SchemaVersionVerbHandler.instance, SCHEMA_VERSION_RSP ), - REPAIR_RSP (92, P1, rpcTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - REPAIR_REQ (32, P1, rpcTimeout, ANTI_ENTROPY, () -> RepairMessage.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), - - REPLICATION_DONE_RSP (82, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - REPLICATION_DONE_REQ (22, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ReplicationDoneVerbHandler.instance, REPLICATION_DONE_RSP), - SNAPSHOT_RSP (87, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), - SNAPSHOT_REQ (27, P0, rpcTimeout, MISC, () -> SnapshotCommand.serializer, () -> SnapshotVerbHandler.instance, SNAPSHOT_RSP ), + SCHEMA_PUSH_RSP (98, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + SCHEMA_PUSH_REQ (18, P1, rpcTimeout, MIGRATION, () -> MigrationsSerializer.instance, () -> SchemaPushVerbHandler.instance, SCHEMA_PUSH_RSP ), + SCHEMA_PULL_RSP (88, P1, rpcTimeout, MIGRATION, () -> MigrationsSerializer.instance, () -> ResponseVerbHandler.instance ), + SCHEMA_PULL_REQ (28, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> SchemaPullVerbHandler.instance, SCHEMA_PULL_RSP ), + SCHEMA_VERSION_RSP (80, P1, rpcTimeout, MIGRATION, () -> UUIDSerializer.serializer, () -> ResponseVerbHandler.instance ), + SCHEMA_VERSION_REQ (20, P1, rpcTimeout, MIGRATION, () -> NoPayload.serializer, () -> SchemaVersionVerbHandler.instance, SCHEMA_VERSION_RSP ), + + // repair; mostly doesn't use callbacks and sends responses as their own request messages, with matching sessions by uuid; should eventually harmonize and make idiomatic + REPAIR_RSP (100, P1, rpcTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + VALIDATION_RSP (102, P1, rpcTimeout, ANTI_ENTROPY, () -> ValidationResponse.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + VALIDATION_REQ (101, P1, rpcTimeout, ANTI_ENTROPY, () -> ValidationRequest.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + SYNC_RSP (104, P1, rpcTimeout, ANTI_ENTROPY, () -> SyncResponse.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + SYNC_REQ (103, P1, rpcTimeout, ANTI_ENTROPY, () -> SyncRequest.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + PREPARE_MSG (105, P1, rpcTimeout, ANTI_ENTROPY, () -> PrepareMessage.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + SNAPSHOT_MSG (106, P1, rpcTimeout, ANTI_ENTROPY, () -> SnapshotMessage.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + CLEANUP_MSG (107, P1, rpcTimeout, ANTI_ENTROPY, () -> CleanupMessage.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + PREPARE_CONSISTENT_RSP (109, P1, rpcTimeout, ANTI_ENTROPY, () -> PrepareConsistentResponse.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + PREPARE_CONSISTENT_REQ (108, P1, rpcTimeout, ANTI_ENTROPY, () -> PrepareConsistentRequest.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + FINALIZE_PROPOSE_MSG (110, P1, rpcTimeout, ANTI_ENTROPY, () -> FinalizePropose.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + FINALIZE_PROMISE_MSG (111, P1, rpcTimeout, ANTI_ENTROPY, () -> FinalizePromise.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + FINALIZE_COMMIT_MSG (112, P1, rpcTimeout, ANTI_ENTROPY, () -> FinalizeCommit.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + FAILED_SESSION_MSG (113, P1, rpcTimeout, ANTI_ENTROPY, () -> FailSession.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + STATUS_RSP (115, P1, rpcTimeout, ANTI_ENTROPY, () -> StatusResponse.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + STATUS_REQ (114, P1, rpcTimeout, ANTI_ENTROPY, () -> StatusRequest.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + ASYMMETRIC_SYNC_REQ (116, P1, rpcTimeout, ANTI_ENTROPY, () -> AsymmetricSyncRequest.serializer, () -> RepairMessageVerbHandler.instance, REPAIR_RSP ), + + REPLICATION_DONE_RSP (82, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + REPLICATION_DONE_REQ (22, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ReplicationDoneVerbHandler.instance, REPLICATION_DONE_RSP), + SNAPSHOT_RSP (87, P0, rpcTimeout, MISC, () -> NoPayload.serializer, () -> ResponseVerbHandler.instance ), + SNAPSHOT_REQ (27, P0, rpcTimeout, MISC, () -> SnapshotCommand.serializer, () -> SnapshotVerbHandler.instance, SNAPSHOT_RSP ), // generic failure response - FAILURE_RSP (99, P0, noTimeout, REQUEST_RESPONSE, () -> RequestFailureReason.serializer, () -> ResponseVerbHandler.instance ), + FAILURE_RSP (99, P0, noTimeout, REQUEST_RESPONSE, () -> RequestFailureReason.serializer, () -> ResponseVerbHandler.instance ), // dummy verbs - _TRACE (30, P1, rpcTimeout, TRACING, () -> NoPayload.serializer, () -> null ), - _SAMPLE (42, P1, rpcTimeout, INTERNAL_RESPONSE, () -> NoPayload.serializer, () -> null ), - _TEST_1 (10, P0, writeTimeout, IMMEDIATE, () -> NoPayload.serializer, () -> null ), - _TEST_2 (11, P1, rpcTimeout, IMMEDIATE, () -> NoPayload.serializer, () -> null ), + _TRACE (30, P1, rpcTimeout, TRACING, () -> NoPayload.serializer, () -> null ), + _SAMPLE (42, P1, rpcTimeout, INTERNAL_RESPONSE, () -> NoPayload.serializer, () -> null ), + _TEST_1 (10, P0, writeTimeout, IMMEDIATE, () -> NoPayload.serializer, () -> null ), + _TEST_2 (11, P1, rpcTimeout, IMMEDIATE, () -> NoPayload.serializer, () -> null ), @Deprecated - REQUEST_RSP (4, P1, rpcTimeout, REQUEST_RESPONSE, () -> null, () -> ResponseVerbHandler.instance ), + REQUEST_RSP (4, P1, rpcTimeout, REQUEST_RESPONSE, () -> null, () -> ResponseVerbHandler.instance ), @Deprecated - INTERNAL_RSP (23, P1, rpcTimeout, INTERNAL_RESPONSE, () -> null, () -> ResponseVerbHandler.instance ), + INTERNAL_RSP (23, P1, rpcTimeout, INTERNAL_RESPONSE, () -> null, () -> ResponseVerbHandler.instance ), - // largest used ID: 99 + // largest used ID: 116 ; public static final List VERBS = ImmutableList.copyOf(Verb.values()); diff --git a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java index 68a58242b940..cf6d84b5af20 100644 --- a/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java +++ b/src/java/org/apache/cassandra/repair/AsymmetricRemoteSyncTask.java @@ -32,7 +32,7 @@ import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.ASYMMETRIC_SYNC_REQ; /** * AsymmetricRemoteSyncTask sends {@link AsymmetricSyncRequest} to target node to repair(stream) @@ -53,7 +53,7 @@ public void startSync() AsymmetricSyncRequest request = new AsymmetricSyncRequest(desc, local, nodePair.coordinator, nodePair.peer, rangesToSync, previewKind); String message = String.format("Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", request.ranges.size(), request.fetchingNode, request.fetchFrom); Tracing.traceRepair(message); - MessagingService.instance().send(Message.out(REPAIR_REQ, request), request.fetchingNode); + MessagingService.instance().send(Message.out(ASYMMETRIC_SYNC_REQ, request), request.fetchingNode); } public void syncComplete(boolean success, List summaries) diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java index 2a87fa2ec612..27ffd05926e7 100644 --- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java +++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java @@ -32,7 +32,7 @@ import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.streaming.PreviewKind; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.VALIDATION_RSP; /** * Handles all repair related message. @@ -62,9 +62,9 @@ public void doVerb(final Message message) RepairJobDesc desc = message.payload.desc; try { - switch (message.payload.messageType) + switch (message.verb()) { - case PREPARE_MESSAGE: + case PREPARE_MSG: PrepareMessage prepareMessage = (PrepareMessage) message.payload; logger.debug("Preparing, {}", prepareMessage); List columnFamilyStores = new ArrayList<>(prepareMessage.tableIds.size()); @@ -90,7 +90,7 @@ public void doVerb(final Message message) MessagingService.instance().send(message.emptyResponse(), message.from()); break; - case SNAPSHOT: + case SNAPSHOT_MSG: logger.debug("Snapshotting {}", desc); final ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(desc.keyspace, desc.columnFamily); if (cfs == null) @@ -114,7 +114,7 @@ public void doVerb(final Message message) MessagingService.instance().send(message.emptyResponse(), message.from()); break; - case VALIDATION_REQUEST: + case VALIDATION_REQ: ValidationRequest validationRequest = (ValidationRequest) message.payload; logger.debug("Validating {}", validationRequest); // trigger read-only compaction @@ -122,7 +122,7 @@ public void doVerb(final Message message) if (store == null) { logger.error("Table {}.{} was dropped during snapshot phase of repair", desc.keyspace, desc.columnFamily); - MessagingService.instance().send(Message.out(REPAIR_REQ, new ValidationComplete(desc)), message.from()); + MessagingService.instance().send(Message.out(VALIDATION_RSP, new ValidationResponse(desc)), message.from()); return; } @@ -132,7 +132,7 @@ public void doVerb(final Message message) ValidationManager.instance.submitValidation(store, validator); break; - case SYNC_REQUEST: + case SYNC_REQ: // forwarded sync request SyncRequest request = (SyncRequest) message.payload; logger.debug("Syncing {}", request); @@ -147,7 +147,7 @@ public void doVerb(final Message message) task.run(); break; - case ASYMMETRIC_SYNC_REQUEST: + case ASYMMETRIC_SYNC_REQ: // forwarded sync request AsymmetricSyncRequest asymmetricSyncRequest = (AsymmetricSyncRequest) message.payload; logger.debug("Syncing {}", asymmetricSyncRequest); @@ -162,49 +162,49 @@ public void doVerb(final Message message) asymmetricTask.run(); break; - case CLEANUP: + case CLEANUP_MSG: logger.debug("cleaning up repair"); CleanupMessage cleanup = (CleanupMessage) message.payload; ActiveRepairService.instance.removeParentRepairSession(cleanup.parentRepairSession); MessagingService.instance().send(message.emptyResponse(), message.from()); break; - case CONSISTENT_REQUEST: + case PREPARE_CONSISTENT_REQ: ActiveRepairService.instance.consistent.local.handlePrepareMessage(message.from(), (PrepareConsistentRequest) message.payload); break; - case CONSISTENT_RESPONSE: + case PREPARE_CONSISTENT_RSP: ActiveRepairService.instance.consistent.coordinated.handlePrepareResponse((PrepareConsistentResponse) message.payload); break; - case FINALIZE_PROPOSE: + case FINALIZE_PROPOSE_MSG: ActiveRepairService.instance.consistent.local.handleFinalizeProposeMessage(message.from(), (FinalizePropose) message.payload); break; - case FINALIZE_PROMISE: + case FINALIZE_PROMISE_MSG: ActiveRepairService.instance.consistent.coordinated.handleFinalizePromiseMessage((FinalizePromise) message.payload); break; - case FINALIZE_COMMIT: + case FINALIZE_COMMIT_MSG: ActiveRepairService.instance.consistent.local.handleFinalizeCommitMessage(message.from(), (FinalizeCommit) message.payload); break; - case FAILED_SESSION: + case FAILED_SESSION_MSG: FailSession failure = (FailSession) message.payload; ActiveRepairService.instance.consistent.coordinated.handleFailSessionMessage(failure); ActiveRepairService.instance.consistent.local.handleFailSessionMessage(message.from(), failure); break; - case STATUS_REQUEST: + case STATUS_REQ: ActiveRepairService.instance.consistent.local.handleStatusRequest(message.from(), (StatusRequest) message.payload); break; - case STATUS_RESPONSE: + case STATUS_RSP: ActiveRepairService.instance.consistent.local.handleStatusResponse(message.from(), (StatusResponse) message.payload); break; default: - ActiveRepairService.instance.handleMessage(message.from(), message.payload); + ActiveRepairService.instance.handleMessage(message); break; } } diff --git a/src/java/org/apache/cassandra/repair/SnapshotTask.java b/src/java/org/apache/cassandra/repair/SnapshotTask.java index fab4b28e5cbb..40e4b3d09377 100644 --- a/src/java/org/apache/cassandra/repair/SnapshotTask.java +++ b/src/java/org/apache/cassandra/repair/SnapshotTask.java @@ -28,7 +28,7 @@ import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.messages.SnapshotMessage; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.SNAPSHOT_MSG; /** * SnapshotTask is a task that sends snapshot request. @@ -46,7 +46,7 @@ public class SnapshotTask extends AbstractFuture implements public void run() { - MessagingService.instance().sendWithCallback(Message.out(REPAIR_REQ, new SnapshotMessage(desc)), + MessagingService.instance().sendWithCallback(Message.out(SNAPSHOT_MSG, new SnapshotMessage(desc)), endpoint, new SnapshotCallback(this)); } diff --git a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java index 53407c9ba1fc..827dce3256ed 100644 --- a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java +++ b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java @@ -31,7 +31,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.repair.messages.SyncComplete; +import org.apache.cassandra.repair.messages.SyncResponse; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamEvent; import org.apache.cassandra.streaming.StreamEventHandler; @@ -39,11 +39,11 @@ import org.apache.cassandra.streaming.StreamState; import org.apache.cassandra.streaming.StreamOperation; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.SYNC_RSP; /** * StreamingRepairTask performs data streaming between two remote replicas, neither of which is repair coordinator. - * Task will send {@link SyncComplete} message back to coordinator upon streaming completion. + * Task will send {@link SyncResponse} message back to coordinator upon streaming completion. */ public class StreamingRepairTask implements Runnable, StreamEventHandler { @@ -103,7 +103,7 @@ public void handleStreamEvent(StreamEvent event) public void onSuccess(StreamState state) { logger.info("[repair #{}] streaming task succeed, returning response to {}", desc.sessionId, initiator); - MessagingService.instance().send(Message.out(REPAIR_REQ, new SyncComplete(desc, src, dst, true, state.createSummaries())), initiator); + MessagingService.instance().send(Message.out(SYNC_RSP, new SyncResponse(desc, src, dst, true, state.createSummaries())), initiator); } /** @@ -111,6 +111,6 @@ public void onSuccess(StreamState state) */ public void onFailure(Throwable t) { - MessagingService.instance().send(Message.out(REPAIR_REQ, new SyncComplete(desc, src, dst, false, Collections.emptyList())), initiator); + MessagingService.instance().send(Message.out(SYNC_RSP, new SyncResponse(desc, src, dst, false, Collections.emptyList())), initiator); } } diff --git a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java index b608d679091e..181554a0b5e7 100644 --- a/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java +++ b/src/java/org/apache/cassandra/repair/SymmetricRemoteSyncTask.java @@ -36,7 +36,7 @@ import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.SYNC_REQ; /** * SymmetricRemoteSyncTask sends {@link SyncRequest} to remote(non-coordinator) node @@ -53,9 +53,9 @@ public SymmetricRemoteSyncTask(RepairJobDesc desc, InetAddressAndPort r1, InetAd super(desc, r1, r2, differences, previewKind); } - void sendRequest(RepairMessage request, InetAddressAndPort to) + void sendRequest(SyncRequest request, InetAddressAndPort to) { - MessagingService.instance().send(Message.out(REPAIR_REQ, request), to); + MessagingService.instance().send(Message.out(SYNC_REQ, request), to); } @Override diff --git a/src/java/org/apache/cassandra/repair/ValidationTask.java b/src/java/org/apache/cassandra/repair/ValidationTask.java index 1892a597d531..0161acf8d867 100644 --- a/src/java/org/apache/cassandra/repair/ValidationTask.java +++ b/src/java/org/apache/cassandra/repair/ValidationTask.java @@ -27,7 +27,7 @@ import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.utils.MerkleTrees; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.VALIDATION_REQ; /** * ValidationTask sends {@link ValidationRequest} to a replica. @@ -54,7 +54,7 @@ public ValidationTask(RepairJobDesc desc, InetAddressAndPort endpoint, int nowIn public void run() { ValidationRequest request = new ValidationRequest(desc, nowInSec); - MessagingService.instance().send(Message.out(REPAIR_REQ, request), endpoint); + MessagingService.instance().send(Message.out(VALIDATION_REQ, request), endpoint); } /** diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java index 9a89fa616983..0d6fdb4c4ddc 100644 --- a/src/java/org/apache/cassandra/repair/Validator.java +++ b/src/java/org/apache/cassandra/repair/Validator.java @@ -44,7 +44,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.repair.messages.ValidationComplete; +import org.apache.cassandra.repair.messages.ValidationResponse; import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.tracing.Tracing; @@ -53,7 +53,7 @@ import org.apache.cassandra.utils.MerkleTree.RowHash; import org.apache.cassandra.utils.MerkleTrees; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.VALIDATION_RSP; /** * Handles the building of a merkle tree for a column family. @@ -391,7 +391,7 @@ public void complete() public void fail() { logger.error("Failed creating a merkle tree for {}, {} (see log for details)", desc, initiator); - respond(new ValidationComplete(desc)); + respond(new ValidationResponse(desc)); } /** @@ -410,7 +410,7 @@ public void run() Tracing.traceRepair("Local completed merkle tree for {} for {}.{}", initiator, desc.keyspace, desc.columnFamily); } - respond(new ValidationComplete(desc, trees)); + respond(new ValidationResponse(desc, trees)); } private boolean initiatorIsRemote() @@ -418,11 +418,11 @@ private boolean initiatorIsRemote() return !FBUtilities.getBroadcastAddressAndPort().equals(initiator); } - private void respond(ValidationComplete response) + private void respond(ValidationResponse response) { if (initiatorIsRemote()) { - MessagingService.instance().send(Message.out(REPAIR_REQ, response), initiator); + MessagingService.instance().send(Message.out(VALIDATION_RSP, response), initiator); return; } @@ -434,7 +434,7 @@ private void respond(ValidationComplete response) */ StageManager.getStage(Stage.ANTI_ENTROPY).execute(() -> { - ValidationComplete movedResponse = response; + ValidationResponse movedResponse = response; try { movedResponse = response.tryMoveOffHeap(); @@ -443,7 +443,7 @@ private void respond(ValidationComplete response) { logger.error("Failed to move local merkle tree for {} off heap", desc, e); } - ActiveRepairService.instance.handleMessage(initiator, movedResponse); + ActiveRepairService.instance.handleMessage(Message.out(VALIDATION_RSP, movedResponse)); }); } } diff --git a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java index 28f5d0809281..8f1759afd125 100644 --- a/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java +++ b/src/java/org/apache/cassandra/repair/consistent/CoordinatorSession.java @@ -139,11 +139,10 @@ synchronized boolean hasFailed() return getState() == State.FAILED || Iterables.any(participantStates.values(), v -> v == State.FAILED); } - protected void sendMessage(InetAddressAndPort destination, RepairMessage message) + protected void sendMessage(InetAddressAndPort destination, Message message) { - logger.trace("Sending {} to {}", message, destination); - Message messageOut = Message.out(Verb.REPAIR_REQ, message); - MessagingService.instance().send(messageOut, destination); + logger.trace("Sending {} to {}", message.payload, destination); + MessagingService.instance().send(message, destination); } public ListenableFuture prepare() @@ -151,7 +150,8 @@ public ListenableFuture prepare() Preconditions.checkArgument(allStates(State.PREPARING)); logger.debug("Beginning prepare phase of incremental repair session {}", sessionID); - PrepareConsistentRequest message = new PrepareConsistentRequest(sessionID, coordinator, participants); + Message message = + Message.out(Verb.PREPARE_CONSISTENT_REQ, new PrepareConsistentRequest(sessionID, coordinator, participants)); for (final InetAddressAndPort participant : participants) { sendMessage(participant, message); @@ -198,7 +198,7 @@ public synchronized ListenableFuture finalizePropose() { Preconditions.checkArgument(allStates(State.REPAIRING)); logger.debug("Proposing finalization of repair session {}", sessionID); - FinalizePropose message = new FinalizePropose(sessionID); + Message message = Message.out(Verb.FINALIZE_PROPOSE_MSG, new FinalizePropose(sessionID)); for (final InetAddressAndPort participant : participants) { sendMessage(participant, message); @@ -234,7 +234,7 @@ public synchronized void finalizeCommit() { Preconditions.checkArgument(allStates(State.FINALIZE_PROMISED)); logger.debug("Committing finalization of repair session {}", sessionID); - FinalizeCommit message = new FinalizeCommit(sessionID); + Message message = Message.out(Verb.FINALIZE_COMMIT_MSG, new FinalizeCommit(sessionID)); for (final InetAddressAndPort participant : participants) { sendMessage(participant, message); @@ -245,7 +245,7 @@ public synchronized void finalizeCommit() private void sendFailureMessageToParticipants() { - FailSession message = new FailSession(sessionID); + Message message = Message.out(Verb.FAILED_SESSION_MSG, new FailSession(sessionID)); for (final InetAddressAndPort participant : participants) { if (participantStates.get(participant) != State.FAILED) diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java index e93ccb08449c..935bba8dfc9e 100644 --- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java +++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java @@ -88,7 +88,11 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.FAILED_SESSION_MSG; +import static org.apache.cassandra.net.Verb.FINALIZE_PROMISE_MSG; +import static org.apache.cassandra.net.Verb.PREPARE_CONSISTENT_RSP; +import static org.apache.cassandra.net.Verb.STATUS_REQ; +import static org.apache.cassandra.net.Verb.STATUS_RSP; import static org.apache.cassandra.repair.consistent.ConsistentSession.State.*; /** @@ -190,10 +194,11 @@ public void cancelSession(UUID sessionID, boolean force) sessionID, session.coordinator); setStateAndSave(session, FAILED); + Message message = Message.out(FAILED_SESSION_MSG, new FailSession(sessionID)); for (InetAddressAndPort participant : session.participants) { if (!participant.equals(getBroadcastAddressAndPort())) - sendMessage(participant, new FailSession(sessionID)); + sendMessage(participant, message); } } @@ -491,11 +496,10 @@ protected ActiveRepairService.ParentRepairSession getParentRepairSession(UUID se return ActiveRepairService.instance.getParentRepairSession(sessionID); } - protected void sendMessage(InetAddressAndPort destination, RepairMessage message) + protected void sendMessage(InetAddressAndPort destination, Message message) { - logger.trace("sending {} to {}", message, destination); - Message messageOut = Message.out(REPAIR_REQ, message); - MessagingService.instance().send(messageOut, destination); + logger.trace("sending {} to {}", message.payload, destination); + MessagingService.instance().send(message, destination); } private void setStateAndSave(LocalSession session, ConsistentSession.State state) @@ -538,7 +542,7 @@ public void failSession(UUID sessionID, boolean sendMessage) } if (sendMessage) { - sendMessage(session.coordinator, new FailSession(sessionID)); + sendMessage(session.coordinator, Message.out(FAILED_SESSION_MSG, new FailSession(sessionID))); } } } @@ -609,7 +613,7 @@ public void handlePrepareMessage(InetAddressAndPort from, PrepareConsistentReque catch (Throwable e) { logger.error("Error retrieving ParentRepairSession for session {}, responding with failure", sessionID); - sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), false)); + sendMessage(coordinator, Message.out(PREPARE_CONSISTENT_RSP, new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), false))); return; } @@ -632,15 +636,14 @@ public void onSuccess(@Nullable Object result) { logger.info("Prepare phase for incremental repair session {} completed", sessionID); if (session.getState() != FAILED) - { setStateAndSave(session, PREPARED); - sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), true)); - } else - { logger.info("Session {} failed before anticompaction completed", sessionID); - sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), false)); - } + + Message message = + Message.out(PREPARE_CONSISTENT_RSP, + new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), session.getState() != FAILED)); + sendMessage(coordinator, message); } finally { @@ -653,7 +656,9 @@ public void onFailure(Throwable t) try { logger.error("Prepare phase for incremental repair session {} failed", sessionID, t); - sendMessage(coordinator, new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), false)); + sendMessage(coordinator, + Message.out(PREPARE_CONSISTENT_RSP, + new PrepareConsistentResponse(sessionID, getBroadcastAddressAndPort(), false))); failSession(sessionID, false); } finally @@ -682,7 +687,7 @@ public void handleFinalizeProposeMessage(InetAddressAndPort from, FinalizePropos if (session == null) { logger.debug("Received FinalizePropose message for unknown repair session {}, responding with failure", sessionID); - sendMessage(from, new FailSession(sessionID)); + sendMessage(from, Message.out(FAILED_SESSION_MSG, new FailSession(sessionID))); return; } @@ -699,7 +704,7 @@ public void handleFinalizeProposeMessage(InetAddressAndPort from, FinalizePropos */ syncTable(); - sendMessage(from, new FinalizePromise(sessionID, getBroadcastAddressAndPort(), true)); + sendMessage(from, Message.out(FINALIZE_PROMISE_MSG, new FinalizePromise(sessionID, getBroadcastAddressAndPort(), true))); logger.debug("Received FinalizePropose message for incremental repair session {}, responded with FinalizePromise", sessionID); } catch (IllegalArgumentException e) @@ -753,7 +758,8 @@ public void handleFailSessionMessage(InetAddressAndPort from, FailSession msg) public void sendStatusRequest(LocalSession session) { logger.debug("Attempting to learn the outcome of unfinished local incremental repair session {}", session.sessionID); - StatusRequest request = new StatusRequest(session.sessionID); + Message request = Message.out(STATUS_REQ, new StatusRequest(session.sessionID)); + for (InetAddressAndPort participant : session.participants) { if (!getBroadcastAddressAndPort().equals(participant) && isAlive(participant)) @@ -771,11 +777,11 @@ public void handleStatusRequest(InetAddressAndPort from, StatusRequest request) if (session == null) { logger.warn("Received status response message for unknown session {}", sessionID); - sendMessage(from, new StatusResponse(sessionID, FAILED)); + sendMessage(from, Message.out(STATUS_RSP, new StatusResponse(sessionID, FAILED))); } else { - sendMessage(from, new StatusResponse(sessionID, session.getState())); + sendMessage(from, Message.out(STATUS_RSP, new StatusResponse(sessionID, session.getState()))); logger.debug("Responding to status response message for incremental repair session {} with local state {}", sessionID, session.getState()); } } diff --git a/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java b/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java index 0a6d257bbf0b..eacc285aa8f5 100644 --- a/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/AsymmetricSyncRequest.java @@ -28,6 +28,7 @@ import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; @@ -38,8 +39,6 @@ public class AsymmetricSyncRequest extends RepairMessage { - public static MessageSerializer serializer = new SyncRequestSerializer(); - public final InetAddressAndPort initiator; public final InetAddressAndPort fetchingNode; public final InetAddressAndPort fetchFrom; @@ -48,7 +47,7 @@ public class AsymmetricSyncRequest extends RepairMessage public AsymmetricSyncRequest(RepairJobDesc desc, InetAddressAndPort initiator, InetAddressAndPort fetchingNode, InetAddressAndPort fetchFrom, Collection> ranges, PreviewKind previewKind) { - super(Type.ASYMMETRIC_SYNC_REQUEST, desc); + super(desc); this.initiator = initiator; this.fetchingNode = fetchingNode; this.fetchFrom = fetchFrom; @@ -62,8 +61,7 @@ public boolean equals(Object o) if (!(o instanceof AsymmetricSyncRequest)) return false; AsymmetricSyncRequest req = (AsymmetricSyncRequest)o; - return messageType == req.messageType && - desc.equals(req.desc) && + return desc.equals(req.desc) && initiator.equals(req.initiator) && fetchingNode.equals(req.fetchingNode) && fetchFrom.equals(req.fetchFrom) && @@ -73,10 +71,10 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(messageType, desc, initiator, fetchingNode, fetchFrom, ranges); + return Objects.hash(desc, initiator, fetchingNode, fetchFrom, ranges); } - public static class SyncRequestSerializer implements MessageSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(AsymmetricSyncRequest message, DataOutputPlus out, int version) throws IOException { @@ -119,7 +117,7 @@ public long serializedSize(AsymmetricSyncRequest message, int version) size += TypeSizes.sizeof(message.previewKind.getSerializationVal()); return size; } - } + }; public String toString() { diff --git a/src/java/org/apache/cassandra/repair/messages/CleanupMessage.java b/src/java/org/apache/cassandra/repair/messages/CleanupMessage.java index 69d147a29db0..5ec7fc65d395 100644 --- a/src/java/org/apache/cassandra/repair/messages/CleanupMessage.java +++ b/src/java/org/apache/cassandra/repair/messages/CleanupMessage.java @@ -21,6 +21,7 @@ import java.util.Objects; import java.util.UUID; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.UUIDSerializer; @@ -32,12 +33,11 @@ */ public class CleanupMessage extends RepairMessage { - public static MessageSerializer serializer = new CleanupMessageSerializer(); public final UUID parentRepairSession; public CleanupMessage(UUID parentRepairSession) { - super(Type.CLEANUP, null); + super(null); this.parentRepairSession = parentRepairSession; } @@ -47,17 +47,16 @@ public boolean equals(Object o) if (!(o instanceof CleanupMessage)) return false; CleanupMessage other = (CleanupMessage) o; - return messageType == other.messageType && - parentRepairSession.equals(other.parentRepairSession); + return parentRepairSession.equals(other.parentRepairSession); } @Override public int hashCode() { - return Objects.hash(messageType, parentRepairSession); + return Objects.hash(parentRepairSession); } - public static class CleanupMessageSerializer implements MessageSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(CleanupMessage message, DataOutputPlus out, int version) throws IOException { @@ -74,5 +73,5 @@ public long serializedSize(CleanupMessage message, int version) { return UUIDSerializer.serializer.serializedSize(message.parentRepairSession, version); } - } + }; } diff --git a/src/java/org/apache/cassandra/repair/messages/FailSession.java b/src/java/org/apache/cassandra/repair/messages/FailSession.java index 1227cc395e5d..b8c7ad34f798 100644 --- a/src/java/org/apache/cassandra/repair/messages/FailSession.java +++ b/src/java/org/apache/cassandra/repair/messages/FailSession.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.UUID; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.UUIDSerializer; @@ -31,7 +32,7 @@ public class FailSession extends RepairMessage public FailSession(UUID sessionID) { - super(Type.FAILED_SESSION, null); + super(null); assert sessionID != null; this.sessionID = sessionID; } @@ -51,7 +52,7 @@ public int hashCode() return sessionID.hashCode(); } - public static final MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(FailSession msg, DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/repair/messages/FinalizeCommit.java b/src/java/org/apache/cassandra/repair/messages/FinalizeCommit.java index a4eb111f7304..bb5cca72b012 100644 --- a/src/java/org/apache/cassandra/repair/messages/FinalizeCommit.java +++ b/src/java/org/apache/cassandra/repair/messages/FinalizeCommit.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.UUID; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.UUIDSerializer; @@ -31,7 +32,7 @@ public class FinalizeCommit extends RepairMessage public FinalizeCommit(UUID sessionID) { - super(Type.FINALIZE_COMMIT, null); + super(null); assert sessionID != null; this.sessionID = sessionID; } @@ -58,7 +59,7 @@ public String toString() '}'; } - public static MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(FinalizeCommit msg, DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java b/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java index 07e7e0d2f956..cfdc07c3825e 100644 --- a/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java +++ b/src/java/org/apache/cassandra/repair/messages/FinalizePromise.java @@ -22,6 +22,7 @@ import java.util.UUID; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; @@ -37,7 +38,7 @@ public class FinalizePromise extends RepairMessage public FinalizePromise(UUID sessionID, InetAddressAndPort participant, boolean promised) { - super(Type.FINALIZE_PROMISE, null); + super(null); assert sessionID != null; assert participant != null; this.sessionID = sessionID; @@ -65,7 +66,7 @@ public int hashCode() return result; } - public static MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(FinalizePromise msg, DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/repair/messages/FinalizePropose.java b/src/java/org/apache/cassandra/repair/messages/FinalizePropose.java index c0c49df72177..c21dd78b9fe4 100644 --- a/src/java/org/apache/cassandra/repair/messages/FinalizePropose.java +++ b/src/java/org/apache/cassandra/repair/messages/FinalizePropose.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.UUID; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.UUIDSerializer; @@ -31,7 +32,7 @@ public class FinalizePropose extends RepairMessage public FinalizePropose(UUID sessionID) { - super(Type.FINALIZE_PROPOSE, null); + super(null); assert sessionID != null; this.sessionID = sessionID; } @@ -58,7 +59,7 @@ public String toString() '}'; } - public static MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(FinalizePropose msg, DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java index b1e9b047d3c7..c1be082464c4 100644 --- a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentRequest.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; @@ -41,7 +42,7 @@ public class PrepareConsistentRequest extends RepairMessage public PrepareConsistentRequest(UUID parentSession, InetAddressAndPort coordinator, Set participants) { - super(Type.CONSISTENT_REQUEST, null); + super(null); assert parentSession != null; assert coordinator != null; assert participants != null && !participants.isEmpty(); @@ -79,9 +80,8 @@ public String toString() '}'; } - public static MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { - public void serialize(PrepareConsistentRequest request, DataOutputPlus out, int version) throws IOException { UUIDSerializer.serializer.serialize(request.parentSession, out, version); diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java index 3362a40826f7..00de77dab385 100644 --- a/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java +++ b/src/java/org/apache/cassandra/repair/messages/PrepareConsistentResponse.java @@ -22,6 +22,7 @@ import java.util.UUID; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; @@ -37,7 +38,7 @@ public class PrepareConsistentResponse extends RepairMessage public PrepareConsistentResponse(UUID parentSession, InetAddressAndPort participant, boolean success) { - super(Type.CONSISTENT_RESPONSE, null); + super(null); assert parentSession != null; assert participant != null; this.parentSession = parentSession; @@ -65,7 +66,7 @@ public int hashCode() return result; } - public static MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(PrepareConsistentResponse response, DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java index 5a0701c6d390..9c485bc2f13d 100644 --- a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java +++ b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java @@ -24,10 +24,13 @@ import java.util.Objects; import java.util.UUID; +import com.google.common.base.Preconditions; + import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.net.MessagingService; @@ -38,7 +41,6 @@ public class PrepareMessage extends RepairMessage { - public final static MessageSerializer serializer = new PrepareMessageSerializer(); public final List tableIds; public final Collection> ranges; @@ -50,7 +52,7 @@ public class PrepareMessage extends RepairMessage public PrepareMessage(UUID parentRepairSession, List tableIds, Collection> ranges, boolean isIncremental, long timestamp, boolean isGlobal, PreviewKind previewKind) { - super(Type.PREPARE_MESSAGE, null); + super(null); this.parentRepairSession = parentRepairSession; this.tableIds = tableIds; this.ranges = ranges; @@ -66,8 +68,7 @@ public boolean equals(Object o) if (!(o instanceof PrepareMessage)) return false; PrepareMessage other = (PrepareMessage) o; - return messageType == other.messageType && - parentRepairSession.equals(other.parentRepairSession) && + return parentRepairSession.equals(other.parentRepairSession) && isIncremental == other.isIncremental && isGlobal == other.isGlobal && previewKind == other.previewKind && @@ -79,13 +80,18 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(messageType, parentRepairSession, isGlobal, previewKind, isIncremental, timestamp, tableIds, ranges); + return Objects.hash(parentRepairSession, isGlobal, previewKind, isIncremental, timestamp, tableIds, ranges); } - public static class PrepareMessageSerializer implements MessageSerializer + private static final String MIXED_MODE_ERROR = "Some nodes involved in repair are on an incompatible major version. " + + "Repair is not supported in mixed major version clusters."; + + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(PrepareMessage message, DataOutputPlus out, int version) throws IOException { + Preconditions.checkArgument(version == MessagingService.current_version, MIXED_MODE_ERROR); + out.writeInt(message.tableIds.size()); for (TableId tableId : message.tableIds) tableId.serialize(out); @@ -104,6 +110,8 @@ public void serialize(PrepareMessage message, DataOutputPlus out, int version) t public PrepareMessage deserialize(DataInputPlus in, int version) throws IOException { + Preconditions.checkArgument(version == MessagingService.current_version, MIXED_MODE_ERROR); + int tableIdCount = in.readInt(); List tableIds = new ArrayList<>(tableIdCount); for (int i = 0; i < tableIdCount; i++) @@ -122,6 +130,8 @@ public PrepareMessage deserialize(DataInputPlus in, int version) throws IOExcept public long serializedSize(PrepareMessage message, int version) { + Preconditions.checkArgument(version == MessagingService.current_version, MIXED_MODE_ERROR); + long size; size = TypeSizes.sizeof(message.tableIds.size()); for (TableId tableId : message.tableIds) @@ -136,7 +146,7 @@ public long serializedSize(PrepareMessage message, int version) size += TypeSizes.sizeof(message.previewKind.getSerializationVal()); return size; } - } + }; @Override public String toString() diff --git a/src/java/org/apache/cassandra/repair/messages/RepairMessage.java b/src/java/org/apache/cassandra/repair/messages/RepairMessage.java index db1a134bb1d2..3137b4e474ae 100644 --- a/src/java/org/apache/cassandra/repair/messages/RepairMessage.java +++ b/src/java/org/apache/cassandra/repair/messages/RepairMessage.java @@ -17,14 +17,6 @@ */ package org.apache.cassandra.repair.messages; -import java.io.IOException; - -import com.google.common.base.Preconditions; - -import org.apache.cassandra.io.IVersionedSerializer; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairJobDesc; /** @@ -34,85 +26,10 @@ */ public abstract class RepairMessage { - public static final IVersionedSerializer serializer = new RepairMessageSerializer(); - - public static interface MessageSerializer extends IVersionedSerializer {} - - public static final int MIN_MESSAGING_VERSION = MessagingService.VERSION_40; - private static final String MIXED_MODE_ERROR = "Some nodes involved in repair are on an incompatible major version. " + - "Repair is not supported in mixed major version clusters."; - - public enum Type - { - VALIDATION_REQUEST(0, ValidationRequest.serializer), - VALIDATION_COMPLETE(1, ValidationComplete.serializer), - SYNC_REQUEST(2, SyncRequest.serializer), - SYNC_COMPLETE(3, SyncComplete.serializer), - PREPARE_MESSAGE(5, PrepareMessage.serializer), - SNAPSHOT(6, SnapshotMessage.serializer), - CLEANUP(7, CleanupMessage.serializer), - - CONSISTENT_REQUEST(8, PrepareConsistentRequest.serializer), - CONSISTENT_RESPONSE(9, PrepareConsistentResponse.serializer), - FINALIZE_PROPOSE(10, FinalizePropose.serializer), - FINALIZE_PROMISE(11, FinalizePromise.serializer), - FINALIZE_COMMIT(12, FinalizeCommit.serializer), - FAILED_SESSION(13, FailSession.serializer), - STATUS_REQUEST(14, StatusRequest.serializer), - STATUS_RESPONSE(15, StatusResponse.serializer), - ASYMMETRIC_SYNC_REQUEST(16, AsymmetricSyncRequest.serializer); - - private final byte type; - private final MessageSerializer serializer; - - Type(int type, MessageSerializer serializer) - { - this.type = (byte) type; - this.serializer = serializer; - } - - public static Type fromByte(byte b) - { - for (Type t : values()) - { - if (t.type == b) - return t; - } - throw new IllegalArgumentException("Unknown RepairMessage.Type: " + b); - } - } - - public final Type messageType; public final RepairJobDesc desc; - protected RepairMessage(Type messageType, RepairJobDesc desc) + protected RepairMessage(RepairJobDesc desc) { - this.messageType = messageType; this.desc = desc; } - - public static class RepairMessageSerializer implements MessageSerializer - { - public void serialize(RepairMessage message, DataOutputPlus out, int version) throws IOException - { - Preconditions.checkArgument(version >= MIN_MESSAGING_VERSION, MIXED_MODE_ERROR); - out.write(message.messageType.type); - message.messageType.serializer.serialize(message, out, version); - } - - public RepairMessage deserialize(DataInputPlus in, int version) throws IOException - { - Preconditions.checkArgument(version >= MIN_MESSAGING_VERSION, MIXED_MODE_ERROR); - RepairMessage.Type messageType = RepairMessage.Type.fromByte(in.readByte()); - return messageType.serializer.deserialize(in, version); - } - - public long serializedSize(RepairMessage message, int version) - { - Preconditions.checkArgument(version >= MIN_MESSAGING_VERSION, MIXED_MODE_ERROR); - long size = 1; // for messageType byte - size += message.messageType.serializer.serializedSize(message, version); - return size; - } - } } diff --git a/src/java/org/apache/cassandra/repair/messages/SnapshotMessage.java b/src/java/org/apache/cassandra/repair/messages/SnapshotMessage.java index d4737d3e96ce..c18950a097b8 100644 --- a/src/java/org/apache/cassandra/repair/messages/SnapshotMessage.java +++ b/src/java/org/apache/cassandra/repair/messages/SnapshotMessage.java @@ -20,17 +20,16 @@ import java.io.IOException; import java.util.Objects; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.repair.RepairJobDesc; public class SnapshotMessage extends RepairMessage { - public final static MessageSerializer serializer = new SnapshotMessageSerializer(); - public SnapshotMessage(RepairJobDesc desc) { - super(Type.SNAPSHOT, desc); + super(desc); } @Override @@ -39,16 +38,16 @@ public boolean equals(Object o) if (!(o instanceof SnapshotMessage)) return false; SnapshotMessage other = (SnapshotMessage) o; - return messageType == other.messageType; + return desc.equals(other.desc); } @Override public int hashCode() { - return Objects.hash(messageType); + return Objects.hash(desc); } - public static class SnapshotMessageSerializer implements MessageSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(SnapshotMessage message, DataOutputPlus out, int version) throws IOException { @@ -65,5 +64,5 @@ public long serializedSize(SnapshotMessage message, int version) { return RepairJobDesc.serializer.serializedSize(message.desc, version); } - } + }; } diff --git a/src/java/org/apache/cassandra/repair/messages/StatusRequest.java b/src/java/org/apache/cassandra/repair/messages/StatusRequest.java index f6a2b827fee0..09354e63503b 100644 --- a/src/java/org/apache/cassandra/repair/messages/StatusRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/StatusRequest.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.UUID; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.utils.UUIDSerializer; @@ -31,7 +32,7 @@ public class StatusRequest extends RepairMessage public StatusRequest(UUID sessionID) { - super(Type.STATUS_REQUEST, null); + super(null); this.sessionID = sessionID; } @@ -57,7 +58,7 @@ public String toString() '}'; } - public static MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(StatusRequest msg, DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/repair/messages/StatusResponse.java b/src/java/org/apache/cassandra/repair/messages/StatusResponse.java index 99eb76ba664d..e62d337df052 100644 --- a/src/java/org/apache/cassandra/repair/messages/StatusResponse.java +++ b/src/java/org/apache/cassandra/repair/messages/StatusResponse.java @@ -22,6 +22,7 @@ import java.util.UUID; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.repair.consistent.ConsistentSession; @@ -34,7 +35,7 @@ public class StatusResponse extends RepairMessage public StatusResponse(UUID sessionID, ConsistentSession.State state) { - super(Type.STATUS_RESPONSE, null); + super(null); assert sessionID != null; assert state != null; this.sessionID = sessionID; @@ -67,7 +68,7 @@ public String toString() '}'; } - public static final MessageSerializer serializer = new MessageSerializer() + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(StatusResponse msg, DataOutputPlus out, int version) throws IOException { diff --git a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java index 71fcdb0832c7..341455f7cdf6 100644 --- a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java @@ -28,6 +28,7 @@ import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; @@ -44,8 +45,6 @@ */ public class SyncRequest extends RepairMessage { - public static MessageSerializer serializer = new SyncRequestSerializer(); - public final InetAddressAndPort initiator; public final InetAddressAndPort src; public final InetAddressAndPort dst; @@ -54,7 +53,7 @@ public class SyncRequest extends RepairMessage public SyncRequest(RepairJobDesc desc, InetAddressAndPort initiator, InetAddressAndPort src, InetAddressAndPort dst, Collection> ranges, PreviewKind previewKind) { - super(Type.SYNC_REQUEST, desc); + super(desc); this.initiator = initiator; this.src = src; this.dst = dst; @@ -68,8 +67,7 @@ public boolean equals(Object o) if (!(o instanceof SyncRequest)) return false; SyncRequest req = (SyncRequest)o; - return messageType == req.messageType && - desc.equals(req.desc) && + return desc.equals(req.desc) && initiator.equals(req.initiator) && src.equals(req.src) && dst.equals(req.dst) && @@ -80,10 +78,10 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(messageType, desc, initiator, src, dst, ranges, previewKind); + return Objects.hash(desc, initiator, src, dst, ranges, previewKind); } - public static class SyncRequestSerializer implements MessageSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(SyncRequest message, DataOutputPlus out, int version) throws IOException { @@ -124,7 +122,7 @@ public long serializedSize(SyncRequest message, int version) size += TypeSizes.sizeof(message.previewKind.getSerializationVal()); return size; } - } + }; @Override public String toString() diff --git a/src/java/org/apache/cassandra/repair/messages/SyncComplete.java b/src/java/org/apache/cassandra/repair/messages/SyncResponse.java similarity index 79% rename from src/java/org/apache/cassandra/repair/messages/SyncComplete.java rename to src/java/org/apache/cassandra/repair/messages/SyncResponse.java index c51d1fd0a564..e7e7985fff34 100644 --- a/src/java/org/apache/cassandra/repair/messages/SyncComplete.java +++ b/src/java/org/apache/cassandra/repair/messages/SyncResponse.java @@ -23,6 +23,7 @@ import java.util.Objects; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; @@ -34,10 +35,8 @@ * * @since 2.0 */ -public class SyncComplete extends RepairMessage +public class SyncResponse extends RepairMessage { - public static final MessageSerializer serializer = new SyncCompleteSerializer(); - /** nodes that involved in this sync */ public final SyncNodePair nodes; /** true if sync success, false otherwise */ @@ -45,17 +44,17 @@ public class SyncComplete extends RepairMessage public final List summaries; - public SyncComplete(RepairJobDesc desc, SyncNodePair nodes, boolean success, List summaries) + public SyncResponse(RepairJobDesc desc, SyncNodePair nodes, boolean success, List summaries) { - super(Type.SYNC_COMPLETE, desc); + super(desc); this.nodes = nodes; this.success = success; this.summaries = summaries; } - public SyncComplete(RepairJobDesc desc, InetAddressAndPort endpoint1, InetAddressAndPort endpoint2, boolean success, List summaries) + public SyncResponse(RepairJobDesc desc, InetAddressAndPort endpoint1, InetAddressAndPort endpoint2, boolean success, List summaries) { - super(Type.SYNC_COMPLETE, desc); + super(desc); this.summaries = summaries; this.nodes = new SyncNodePair(endpoint1, endpoint2); this.success = success; @@ -64,11 +63,10 @@ public SyncComplete(RepairJobDesc desc, InetAddressAndPort endpoint1, InetAddres @Override public boolean equals(Object o) { - if (!(o instanceof SyncComplete)) + if (!(o instanceof SyncResponse)) return false; - SyncComplete other = (SyncComplete)o; - return messageType == other.messageType && - desc.equals(other.desc) && + SyncResponse other = (SyncResponse)o; + return desc.equals(other.desc) && success == other.success && nodes.equals(other.nodes) && summaries.equals(other.summaries); @@ -77,12 +75,12 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(messageType, desc, success, nodes, summaries); + return Objects.hash(desc, success, nodes, summaries); } - private static class SyncCompleteSerializer implements MessageSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { - public void serialize(SyncComplete message, DataOutputPlus out, int version) throws IOException + public void serialize(SyncResponse message, DataOutputPlus out, int version) throws IOException { RepairJobDesc.serializer.serialize(message.desc, out, version); SyncNodePair.serializer.serialize(message.nodes, out, version); @@ -95,7 +93,7 @@ public void serialize(SyncComplete message, DataOutputPlus out, int version) thr } } - public SyncComplete deserialize(DataInputPlus in, int version) throws IOException + public SyncResponse deserialize(DataInputPlus in, int version) throws IOException { RepairJobDesc desc = RepairJobDesc.serializer.deserialize(in, version); SyncNodePair nodes = SyncNodePair.serializer.deserialize(in, version); @@ -108,10 +106,10 @@ public SyncComplete deserialize(DataInputPlus in, int version) throws IOExceptio summaries.add(SessionSummary.serializer.deserialize(in, version)); } - return new SyncComplete(desc, nodes, success, summaries); + return new SyncResponse(desc, nodes, success, summaries); } - public long serializedSize(SyncComplete message, int version) + public long serializedSize(SyncResponse message, int version) { long size = RepairJobDesc.serializer.serializedSize(message.desc, version); size += SyncNodePair.serializer.serializedSize(message.nodes, version); @@ -125,5 +123,5 @@ public long serializedSize(SyncComplete message, int version) return size; } - } + }; } diff --git a/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java b/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java index 646624462ad5..f9a1f4e2be76 100644 --- a/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java +++ b/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.repair.RepairJobDesc; @@ -31,13 +32,11 @@ */ public class ValidationRequest extends RepairMessage { - public static MessageSerializer serializer = new ValidationRequestSerializer(); - public final int nowInSec; public ValidationRequest(RepairJobDesc desc, int nowInSec) { - super(Type.VALIDATION_REQUEST, desc); + super(desc); this.nowInSec = nowInSec; } @@ -65,7 +64,7 @@ public int hashCode() return nowInSec; } - public static class ValidationRequestSerializer implements MessageSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { public void serialize(ValidationRequest message, DataOutputPlus out, int version) throws IOException { @@ -85,5 +84,5 @@ public long serializedSize(ValidationRequest message, int version) size += TypeSizes.sizeof(message.nowInSec); return size; } - } + }; } diff --git a/src/java/org/apache/cassandra/repair/messages/ValidationComplete.java b/src/java/org/apache/cassandra/repair/messages/ValidationResponse.java similarity index 70% rename from src/java/org/apache/cassandra/repair/messages/ValidationComplete.java rename to src/java/org/apache/cassandra/repair/messages/ValidationResponse.java index b8aa736b8804..d9f44677bcdc 100644 --- a/src/java/org/apache/cassandra/repair/messages/ValidationComplete.java +++ b/src/java/org/apache/cassandra/repair/messages/ValidationResponse.java @@ -21,6 +21,7 @@ import java.util.Objects; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.repair.RepairJobDesc; @@ -31,22 +32,20 @@ * * @since 2.0 */ -public class ValidationComplete extends RepairMessage +public class ValidationResponse extends RepairMessage { - public static MessageSerializer serializer = new ValidationCompleteSerializer(); - /** Merkle hash tree response. Null if validation failed. */ public final MerkleTrees trees; - public ValidationComplete(RepairJobDesc desc) + public ValidationResponse(RepairJobDesc desc) { - super(Type.VALIDATION_COMPLETE, desc); + super(desc); trees = null; } - public ValidationComplete(RepairJobDesc desc, MerkleTrees trees) + public ValidationResponse(RepairJobDesc desc, MerkleTrees trees) { - super(Type.VALIDATION_COMPLETE, desc); + super(desc); assert trees != null; this.trees = trees; } @@ -57,34 +56,33 @@ public boolean success() } /** - * @return a new {@link ValidationComplete} instance with all trees moved off heap, or {@code this} + * @return a new {@link ValidationResponse} instance with all trees moved off heap, or {@code this} * if it's a failure response. */ - public ValidationComplete tryMoveOffHeap() throws IOException + public ValidationResponse tryMoveOffHeap() throws IOException { - return trees == null ? this : new ValidationComplete(desc, trees.tryMoveOffHeap()); + return trees == null ? this : new ValidationResponse(desc, trees.tryMoveOffHeap()); } @Override public boolean equals(Object o) { - if (!(o instanceof ValidationComplete)) + if (!(o instanceof ValidationResponse)) return false; - ValidationComplete other = (ValidationComplete)o; - return messageType == other.messageType && - desc.equals(other.desc); + ValidationResponse other = (ValidationResponse)o; + return desc.equals(other.desc); } @Override public int hashCode() { - return Objects.hash(messageType, desc); + return Objects.hash(desc); } - private static class ValidationCompleteSerializer implements MessageSerializer + public static final IVersionedSerializer serializer = new IVersionedSerializer() { - public void serialize(ValidationComplete message, DataOutputPlus out, int version) throws IOException + public void serialize(ValidationResponse message, DataOutputPlus out, int version) throws IOException { RepairJobDesc.serializer.serialize(message.desc, out, version); out.writeBoolean(message.success()); @@ -92,7 +90,7 @@ public void serialize(ValidationComplete message, DataOutputPlus out, int versio MerkleTrees.serializer.serialize(message.trees, out, version); } - public ValidationComplete deserialize(DataInputPlus in, int version) throws IOException + public ValidationResponse deserialize(DataInputPlus in, int version) throws IOException { RepairJobDesc desc = RepairJobDesc.serializer.deserialize(in, version); boolean success = in.readBoolean(); @@ -100,13 +98,13 @@ public ValidationComplete deserialize(DataInputPlus in, int version) throws IOEx if (success) { MerkleTrees trees = MerkleTrees.serializer.deserialize(in, version); - return new ValidationComplete(desc, trees); + return new ValidationResponse(desc, trees); } - return new ValidationComplete(desc); + return new ValidationResponse(desc); } - public long serializedSize(ValidationComplete message, int version) + public long serializedSize(ValidationResponse message, int version) { long size = RepairJobDesc.serializer.serializedSize(message.desc, version); size += TypeSizes.sizeof(message.success()); @@ -114,5 +112,5 @@ public long serializedSize(ValidationComplete message, int version) size += MerkleTrees.serializer.serializedSize(message.trees, version); return size; } - } + }; } diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index 409d7991a5df..6f4c474fba5e 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -78,7 +78,7 @@ import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.transform; -import static org.apache.cassandra.net.Verb.REPAIR_REQ; +import static org.apache.cassandra.net.Verb.PREPARE_MSG; /** * ActiveRepairService is the starting point for manual "active" repairs. @@ -112,8 +112,6 @@ public static class ConsistentSessions private boolean registeredForEndpointChanges = false; - public static CassandraVersion SUPPORTS_GLOBAL_PREPARE_FLAG_VERSION = new CassandraVersion("2.2.1"); - private static final Logger logger = LoggerFactory.getLogger(ActiveRepairService.class); // singleton enforcement public static final ActiveRepairService instance = new ActiveRepairService(FailureDetector.instance, Gossiper.instance); @@ -437,7 +435,7 @@ public boolean invokeOnFailure() if (FailureDetector.instance.isAlive(neighbour)) { PrepareMessage message = new PrepareMessage(parentRepairSession, tableIds, options.getRanges(), options.isIncremental(), repairedAt, options.isGlobal(), options.getPreviewKind()); - Message msg = Message.out(REPAIR_REQ, message); + Message msg = Message.out(PREPARE_MSG, message); MessagingService.instance().sendWithCallback(msg, neighbour, callback); } else @@ -527,21 +525,21 @@ public synchronized ParentRepairSession removeParentRepairSession(UUID parentSes return parentRepairSessions.remove(parentSessionId); } - public void handleMessage(InetAddressAndPort endpoint, RepairMessage message) + public void handleMessage(Message message) { - RepairJobDesc desc = message.desc; + RepairJobDesc desc = message.payload.desc; RepairSession session = sessions.get(desc.sessionId); if (session == null) return; - switch (message.messageType) + switch (message.verb()) { - case VALIDATION_COMPLETE: - ValidationComplete validation = (ValidationComplete) message; - session.validationComplete(desc, endpoint, validation.trees); + case VALIDATION_RSP: + ValidationResponse validation = (ValidationResponse) message.payload; + session.validationComplete(desc, message.from(), validation.trees); break; - case SYNC_COMPLETE: + case SYNC_RSP: // one of replica is synced. - SyncComplete sync = (SyncComplete) message; + SyncResponse sync = (SyncResponse) message.payload; session.syncComplete(desc, sync.nodes, sync.success, sync.summaries); break; default: diff --git a/test/data/serialization/4.0/service.SyncComplete.bin b/test/data/serialization/4.0/service.SyncComplete.bin index 849faf219b9a1365fdfb53f738feed76f3e6296a..4e8caa6c37437500958e0d4914598c7166286dfc 100644 GIT binary patch delta 80 zcmZo-YG9hUGGt6@u>n<{@e5d% Mfq`oxV+ceo09+0ifdBvi delta 78 zcmZo*YGPt!o{(+ZRR5qsqf_wZ1bM?7Q~BdoFfcH%0x<}r05Ky_z7bWP@e5d%fq{#8 ILI^}H0D5y1bpQYW diff --git a/test/data/serialization/4.0/service.SyncRequest.bin b/test/data/serialization/4.0/service.SyncRequest.bin index f5f2e3d81fe0d675c2c388c0aa0df1c651e0a6cf..b0cc44eee16c356df30539a482fa42ebda25649d 100644 GIT binary patch delta 5 Mcmd1Lo5+|400s*I-T(jq delta 6 Ncmd1HXJnd?2LJ|L0p9=s diff --git a/test/data/serialization/4.0/service.ValidationComplete.bin b/test/data/serialization/4.0/service.ValidationComplete.bin index 3f5b31897499937ceecdab6a955edb6bb59de694..7402c9e848d5707d9ade3a2427ec509eb9031e5f 100644 GIT binary patch delta 117 zcmcb?a+PJ`w1A0$t`nL3Cq8SO+`z~x#le6I82>Z=X95ZdNK7td6fqS+m1O`)Llqcc Tkpk;X0O_3km{D)C5R)4Ka6=VT delta 100 zcmcc0a)X7Dal+II0TZ7zaxwmA{LjR|z#w2S*^yC1N(0D525@N$khF{mvK){Nl7{L} P04bRKkWp`f8zTb%H5e0u diff --git a/test/data/serialization/4.0/service.ValidationRequest.bin b/test/data/serialization/4.0/service.ValidationRequest.bin index 72d4a1fa000cc77eff8e4de2ffbc864f9a14b5fc..fa4a9138426b958aae159c00bd9b7233cd003a5d 100644 GIT binary patch delta 5 McmebFn#kw{00g4}asU7T delta 6 NcmebBW@MP)1pox60dW8T diff --git a/test/unit/org/apache/cassandra/repair/RepairJobTest.java b/test/unit/org/apache/cassandra/repair/RepairJobTest.java index b84adaa06617..068544d897bf 100644 --- a/test/unit/org/apache/cassandra/repair/RepairJobTest.java +++ b/test/unit/org/apache/cassandra/repair/RepairJobTest.java @@ -54,6 +54,7 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.messages.RepairMessage; import org.apache.cassandra.repair.messages.SyncRequest; import org.apache.cassandra.schema.KeyspaceParams; @@ -181,14 +182,14 @@ public void testEndToEndNoDifferences() throws InterruptedException, ExecutionEx assertEquals(0, result.stats.size()); // RepairJob should send out SNAPSHOTS -> VALIDATIONS -> done - List expectedTypes = new ArrayList<>(); + List expectedTypes = new ArrayList<>(); for (int i = 0; i < 3; i++) - expectedTypes.add(RepairMessage.Type.SNAPSHOT); + expectedTypes.add(Verb.SNAPSHOT_MSG); for (int i = 0; i < 3; i++) - expectedTypes.add(RepairMessage.Type.VALIDATION_REQUEST); + expectedTypes.add(Verb.VALIDATION_REQ); assertEquals(expectedTypes, observedMessages.stream() - .map(k -> ((RepairMessage) k.payload).messageType) + .map(Message::verb) .collect(Collectors.toList())); } @@ -251,7 +252,7 @@ public void testNoTreesRetainedAfterDifference() throws Throwable assertTrue(results.stream().allMatch(s -> s.numberOfDifferences == 1)); assertEquals(2, messages.size()); - assertTrue(messages.stream().allMatch(m -> ((RepairMessage) m.payload).messageType == RepairMessage.Type.SYNC_REQUEST)); + assertTrue(messages.stream().allMatch(m -> m.verb() == Verb.SYNC_REQ)); } @Test @@ -800,17 +801,16 @@ private void interceptRepairMessages(Map mockTr messageCapture.add(message); } - RepairMessage rm = (RepairMessage) message.payload; - switch (rm.messageType) + switch (message.verb()) { - case SNAPSHOT: + case SNAPSHOT_MSG: MessagingService.instance().callbacks.removeAndRespond(message.id(), to, message.emptyResponse()); break; - case VALIDATION_REQUEST: + case VALIDATION_REQ: session.validationComplete(sessionJobDesc, to, mockTrees.get(to)); break; - case SYNC_REQUEST: - SyncRequest syncRequest = (SyncRequest) rm; + case SYNC_REQ: + SyncRequest syncRequest = (SyncRequest) message.payload; session.syncComplete(sessionJobDesc, new SyncNodePair(syncRequest.src, syncRequest.dst), true, Collections.emptyList()); break; diff --git a/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java index 7f48788bc6fa..cba64ae17f5f 100644 --- a/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java +++ b/test/unit/org/apache/cassandra/repair/SymmetricRemoteSyncTaskTest.java @@ -48,7 +48,7 @@ public InstrumentedSymmetricRemoteSyncTask(InetAddressAndPort e1, InetAddressAnd InetAddressAndPort sentTo = null; @Override - void sendRequest(RepairMessage request, InetAddressAndPort to) + void sendRequest(SyncRequest request, InetAddressAndPort to) { Assert.assertNull(sentMessage); Assert.assertNotNull(request); diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java index 9e848a95e392..a288edb1de61 100644 --- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java +++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java @@ -51,8 +51,7 @@ import org.apache.cassandra.dht.Token; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; -import org.apache.cassandra.repair.messages.RepairMessage; -import org.apache.cassandra.repair.messages.ValidationComplete; +import org.apache.cassandra.repair.messages.ValidationResponse; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.service.ActiveRepairService; import org.apache.cassandra.streaming.PreviewKind; @@ -131,12 +130,11 @@ public void testValidatorComplete() throws Throwable assertNotNull(tree.hash(new Range<>(min, min))); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - assertEquals(Verb.REPAIR_REQ, message.verb()); - RepairMessage m = (RepairMessage) message.payload; - assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType); + assertEquals(Verb.VALIDATION_RSP, message.verb()); + ValidationResponse m = (ValidationResponse) message.payload; assertEquals(desc, m.desc); - assertTrue(((ValidationComplete) m).success()); - assertNotNull(((ValidationComplete) m).trees); + assertTrue(m.success()); + assertNotNull(m.trees); } @@ -154,12 +152,11 @@ public void testValidatorFailed() throws Throwable validator.fail(); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - assertEquals(Verb.REPAIR_REQ, message.verb()); - RepairMessage m = (RepairMessage) message.payload; - assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType); + assertEquals(Verb.VALIDATION_RSP, message.verb()); + ValidationResponse m = (ValidationResponse) message.payload; assertEquals(desc, m.desc); - assertFalse(((ValidationComplete) m).success()); - assertNull(((ValidationComplete) m).trees); + assertFalse(m.success()); + assertNull(m.trees); } @Test @@ -214,19 +211,17 @@ public void simpleValidationTest(int n) throws Exception ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - assertEquals(Verb.REPAIR_REQ, message.verb()); - RepairMessage m = (RepairMessage) message.payload; - assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType); + assertEquals(Verb.VALIDATION_RSP, message.verb()); + ValidationResponse m = (ValidationResponse) message.payload; assertEquals(desc, m.desc); - assertTrue(((ValidationComplete) m).success()); - MerkleTrees trees = ((ValidationComplete) m).trees; + assertTrue(m.success()); - Iterator, MerkleTree>> iterator = trees.iterator(); + Iterator, MerkleTree>> iterator = m.trees.iterator(); while (iterator.hasNext()) { assertEquals(Math.pow(2, Math.ceil(Math.log(n) / Math.log(2))), iterator.next().getValue().size(), 0.0); } - assertEquals(trees.rowCount(), n); + assertEquals(m.trees.rowCount(), n); } /* @@ -273,7 +268,7 @@ public void testSizeLimiting() throws Exception ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - MerkleTrees trees = ((ValidationComplete) message.payload).trees; + MerkleTrees trees = ((ValidationResponse) message.payload).trees; Iterator, MerkleTree>> iterator = trees.iterator(); int numTrees = 0; @@ -335,7 +330,7 @@ public void testRangeSplittingTreeSizeLimit() throws Exception ValidationManager.instance.submitValidation(cfs, validator); Message message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS); - MerkleTrees trees = ((ValidationComplete) message.payload).trees; + MerkleTrees trees = ((ValidationResponse) message.payload).trees; // Should have 4 trees each with a depth of on average 10 (since each range should have gotten 0.25 megabytes) Iterator, MerkleTree>> iterator = trees.iterator(); diff --git a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java index 237e9a86df71..c9fd9132c832 100644 --- a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java +++ b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorMessagingTest.java @@ -61,7 +61,6 @@ import org.apache.cassandra.service.ActiveRepairService; import static org.apache.cassandra.net.MockMessagingService.all; -import static org.apache.cassandra.net.MockMessagingService.payload; import static org.apache.cassandra.net.MockMessagingService.to; import static org.apache.cassandra.net.MockMessagingService.verb; import static org.junit.Assert.fail; @@ -275,45 +274,36 @@ private MockMessagingSpy createPrepareSpy(Collection failed, Collection timeout, Function sessionIdFunc) { - return MockMessagingService.when( - all(verb(Verb.REPAIR_REQ), - payload((p) -> p instanceof PrepareConsistentRequest)) - ).respond((msgOut, to) -> - { - if(timeout.contains(to)) return null; - else return Message.out(Verb.REPAIR_REQ, new PrepareConsistentResponse(sessionIdFunc.apply((PrepareConsistentRequest) msgOut.payload), to, !failed.contains(to))); - }); + return MockMessagingService.when(verb(Verb.PREPARE_CONSISTENT_REQ)).respond((msgOut, to) -> + { + if (timeout.contains(to)) + return null; + + return Message.out(Verb.PREPARE_CONSISTENT_RSP, + new PrepareConsistentResponse(sessionIdFunc.apply((PrepareConsistentRequest) msgOut.payload), to, !failed.contains(to))); + }); } private MockMessagingSpy createFinalizeSpy(Collection failed, Collection timeout) { - return MockMessagingService.when( - all(verb(Verb.REPAIR_REQ), - payload((p) -> p instanceof FinalizePropose)) - ).respond((msgOut, to) -> - { - if(timeout.contains(to)) return null; - else return Message.out(Verb.REPAIR_REQ, - new FinalizePromise(((FinalizePropose) msgOut.payload).sessionID, to, !failed.contains(to))); - }); + return MockMessagingService.when(verb(Verb.FINALIZE_PROPOSE_MSG)).respond((msgOut, to) -> + { + if (timeout.contains(to)) + return null; + + return Message.out(Verb.FINALIZE_PROMISE_MSG, new FinalizePromise(((FinalizePropose) msgOut.payload).sessionID, to, !failed.contains(to))); + }); } private MockMessagingSpy createCommitSpy() { - return MockMessagingService.when( - all(verb(Verb.REPAIR_REQ), - payload((p) -> p instanceof FinalizeCommit)) - ).dontReply(); + return MockMessagingService.when(verb(Verb.FINALIZE_COMMIT_MSG)).dontReply(); } private MockMessagingSpy createFailSessionSpy(Collection participants) { - return MockMessagingService.when( - all(verb(Verb.REPAIR_REQ), - payload((p) -> p instanceof FailSession), - to(participants::contains)) - ).dontReply(); + return MockMessagingService.when(all(verb(Verb.FAILED_SESSION_MSG), to(participants::contains))).dontReply(); } private static RepairSessionResult createResult(CoordinatorSession coordinator) diff --git a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java index c6980fe00862..1cee312a0e2f 100644 --- a/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java +++ b/test/unit/org/apache/cassandra/repair/consistent/CoordinatorSessionTest.java @@ -35,6 +35,7 @@ import org.junit.Test; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; import org.apache.cassandra.repair.AbstractRepairTest; import org.apache.cassandra.repair.RepairSessionResult; import org.apache.cassandra.repair.messages.FailSession; @@ -93,13 +94,13 @@ public InstrumentedCoordinatorSession(Builder builder) Map> sentMessages = new HashMap<>(); - protected void sendMessage(InetAddressAndPort destination, RepairMessage message) + protected void sendMessage(InetAddressAndPort destination, Message message) { if (!sentMessages.containsKey(destination)) { sentMessages.put(destination, new ArrayList<>()); } - sentMessages.get(destination).add(message); + sentMessages.get(destination).add(message.payload); } Runnable onSetRepairing = null; diff --git a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java index a6b4fe26837d..15fd1fc096e9 100644 --- a/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java +++ b/test/unit/org/apache/cassandra/repair/consistent/LocalSessionTest.java @@ -44,6 +44,7 @@ import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; import org.apache.cassandra.locator.RangesAtEndpoint; +import org.apache.cassandra.net.Message; import org.apache.cassandra.repair.AbstractRepairTest; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.repair.KeyspaceRepairManager; @@ -123,13 +124,14 @@ private static void assertMessagesSent(InstrumentedLocalSessions sessions, InetA static class InstrumentedLocalSessions extends LocalSessions { Map> sentMessages = new HashMap<>(); - protected void sendMessage(InetAddressAndPort destination, RepairMessage message) + + protected void sendMessage(InetAddressAndPort destination, Message message) { if (!sentMessages.containsKey(destination)) { sentMessages.put(destination, new ArrayList<>()); } - sentMessages.get(destination).add(message); + sentMessages.get(destination).add(message.payload); } SettableFuture prepareSessionFuture = null; diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java index d583d851fa9d..fa037a09af4d 100644 --- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java +++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java @@ -118,7 +118,7 @@ private T serializeRoundTrip(T msg, IVersionedSerializ @Test public void validationCompleteMessage_NoMerkleTree() throws IOException { - ValidationComplete deserialized = validationCompleteMessage(null); + ValidationResponse deserialized = validationCompleteMessage(null); Assert.assertNull(deserialized.trees); } @@ -127,19 +127,19 @@ public void validationCompleteMessage_WithMerkleTree() throws IOException { MerkleTrees trees = new MerkleTrees(Murmur3Partitioner.instance); trees.addMerkleTree(256, new Range<>(new LongToken(1000), new LongToken(1001))); - ValidationComplete deserialized = validationCompleteMessage(trees); + ValidationResponse deserialized = validationCompleteMessage(trees); // a simple check to make sure we got some merkle trees back. Assert.assertEquals(trees.size(), deserialized.trees.size()); } - private ValidationComplete validationCompleteMessage(MerkleTrees trees) throws IOException + private ValidationResponse validationCompleteMessage(MerkleTrees trees) throws IOException { RepairJobDesc jobDesc = buildRepairJobDesc(); - ValidationComplete msg = trees == null ? - new ValidationComplete(jobDesc) : - new ValidationComplete(jobDesc, trees); - ValidationComplete deserialized = serializeRoundTrip(msg, ValidationComplete.serializer); + ValidationResponse msg = trees == null ? + new ValidationResponse(jobDesc) : + new ValidationResponse(jobDesc, trees); + ValidationResponse deserialized = serializeRoundTrip(msg, ValidationResponse.serializer); return deserialized; } @@ -164,8 +164,8 @@ public void syncCompleteMessage() throws IOException Lists.newArrayList(new StreamSummary(TableId.fromUUID(UUIDGen.getTimeUUID()), 5, 100)), Lists.newArrayList(new StreamSummary(TableId.fromUUID(UUIDGen.getTimeUUID()), 500, 10)) )); - SyncComplete msg = new SyncComplete(buildRepairJobDesc(), new SyncNodePair(src, dst), true, summaries); - serializeRoundTrip(msg, SyncComplete.serializer); + SyncResponse msg = new SyncResponse(buildRepairJobDesc(), new SyncNodePair(src, dst), true, summaries); + serializeRoundTrip(msg, SyncResponse.serializer); } @Test diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java index d876139a3fe3..fedf498aeb99 100644 --- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java +++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializerTest.java @@ -24,6 +24,7 @@ import org.junit.Assert; import org.junit.Test; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.locator.InetAddressAndPort; @@ -37,16 +38,16 @@ public class RepairMessageSerializerTest { private static int MS_VERSION = MessagingService.current_version; - static RepairMessage serdes(RepairMessage message) + private static T serdes(IVersionedSerializer serializer, T message) { - int expectedSize = (int) RepairMessage.serializer.serializedSize(message, MS_VERSION); + int expectedSize = (int) serializer.serializedSize(message, MS_VERSION); try (DataOutputBuffer out = new DataOutputBuffer(expectedSize)) { - RepairMessage.serializer.serialize(message, out, MS_VERSION); + serializer.serialize(message, out, MS_VERSION); Assert.assertEquals(expectedSize, out.buffer().limit()); try (DataInputBuffer in = new DataInputBuffer(out.buffer(), false)) { - return RepairMessage.serializer.deserialize(in, MS_VERSION); + return serializer.deserialize(in, MS_VERSION); } } catch (IOException e) @@ -62,54 +63,50 @@ public void prepareConsistentRequest() throws Exception InetAddressAndPort peer1 = InetAddressAndPort.getByName("10.0.0.2"); InetAddressAndPort peer2 = InetAddressAndPort.getByName("10.0.0.3"); InetAddressAndPort peer3 = InetAddressAndPort.getByName("10.0.0.4"); - RepairMessage expected = new PrepareConsistentRequest(UUIDGen.getTimeUUID(), - coordinator, - Sets.newHashSet(peer1, peer2, peer3)); - RepairMessage actual = serdes(expected); + PrepareConsistentRequest expected = + new PrepareConsistentRequest(UUIDGen.getTimeUUID(), coordinator, Sets.newHashSet(peer1, peer2, peer3)); + PrepareConsistentRequest actual = serdes(PrepareConsistentRequest.serializer, expected); Assert.assertEquals(expected, actual); } @Test public void prepareConsistentResponse() throws Exception { - RepairMessage expected = new PrepareConsistentResponse(UUIDGen.getTimeUUID(), - InetAddressAndPort.getByName("10.0.0.2"), - true); - RepairMessage actual = serdes(expected); + PrepareConsistentResponse expected = + new PrepareConsistentResponse(UUIDGen.getTimeUUID(), InetAddressAndPort.getByName("10.0.0.2"), true); + PrepareConsistentResponse actual = serdes(PrepareConsistentResponse.serializer, expected); Assert.assertEquals(expected, actual); } @Test public void failSession() throws Exception { - RepairMessage expected = new FailSession(UUIDGen.getTimeUUID()); - RepairMessage actual = serdes(expected); + FailSession expected = new FailSession(UUIDGen.getTimeUUID()); + FailSession actual = serdes(FailSession.serializer, expected); Assert.assertEquals(expected, actual);; } @Test public void finalizeCommit() throws Exception { - RepairMessage expected = new FinalizeCommit(UUIDGen.getTimeUUID()); - RepairMessage actual = serdes(expected); + FinalizeCommit expected = new FinalizeCommit(UUIDGen.getTimeUUID()); + FinalizeCommit actual = serdes(FinalizeCommit.serializer, expected); Assert.assertEquals(expected, actual);; } @Test public void finalizePromise() throws Exception { - RepairMessage expected = new FinalizePromise(UUIDGen.getTimeUUID(), - InetAddressAndPort.getByName("10.0.0.2"), - true); - RepairMessage actual = serdes(expected); + FinalizePromise expected = new FinalizePromise(UUIDGen.getTimeUUID(), InetAddressAndPort.getByName("10.0.0.2"), true); + FinalizePromise actual = serdes(FinalizePromise.serializer, expected); Assert.assertEquals(expected, actual); } @Test public void finalizePropose() throws Exception { - RepairMessage expected = new FinalizePropose(UUIDGen.getTimeUUID()); - RepairMessage actual = serdes(expected); + FinalizePropose expected = new FinalizePropose(UUIDGen.getTimeUUID()); + FinalizePropose actual = serdes(FinalizePropose.serializer, expected); Assert.assertEquals(expected, actual);; } } diff --git a/test/unit/org/apache/cassandra/service/SerializationsTest.java b/test/unit/org/apache/cassandra/service/SerializationsTest.java index 273d23302e8b..0a5a023a48e0 100644 --- a/test/unit/org/apache/cassandra/service/SerializationsTest.java +++ b/test/unit/org/apache/cassandra/service/SerializationsTest.java @@ -38,10 +38,10 @@ import org.apache.cassandra.dht.RandomPartitioner; import org.apache.cassandra.dht.Range; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus; import org.apache.cassandra.io.util.DataOutputStreamPlus; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.net.Message; import org.apache.cassandra.repair.SyncNodePair; import org.apache.cassandra.repair.RepairJobDesc; import org.apache.cassandra.repair.Validator; @@ -79,14 +79,14 @@ public static void tearDown() partitionerSwitcher.close(); } - private void testRepairMessageWrite(String fileName, RepairMessage... messages) throws IOException + private void testRepairMessageWrite(String fileName, IVersionedSerializer serializer, T... messages) throws IOException { try (DataOutputStreamPlus out = getOutput(fileName)) { - for (RepairMessage message : messages) + for (T message : messages) { - testSerializedSize(message, RepairMessage.serializer); - RepairMessage.serializer.serialize(message, out, getVersion()); + testSerializedSize(message, serializer); + serializer.serialize(message, out, getVersion()); } } } @@ -94,7 +94,7 @@ private void testRepairMessageWrite(String fileName, RepairMessage... messages) private void testValidationRequestWrite() throws IOException { ValidationRequest message = new ValidationRequest(DESC, 1234); - testRepairMessageWrite("service.ValidationRequest.bin", message); + testRepairMessageWrite("service.ValidationRequest.bin", ValidationRequest.serializer, message); } @Test @@ -105,10 +105,9 @@ public void testValidationRequestRead() throws IOException try (DataInputStreamPlus in = getInput("service.ValidationRequest.bin")) { - RepairMessage message = RepairMessage.serializer.deserialize(in, getVersion()); - assert message.messageType == RepairMessage.Type.VALIDATION_REQUEST; + ValidationRequest message = ValidationRequest.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); - assert ((ValidationRequest) message).nowInSec == 1234; + assert message.nowInSec == 1234; } } @@ -121,7 +120,7 @@ private void testValidationCompleteWrite() throws IOException // empty validation mt.addMerkleTree((int) Math.pow(2, 15), FULL_RANGE); Validator v0 = new Validator(DESC, FBUtilities.getBroadcastAddressAndPort(), -1, PreviewKind.NONE); - ValidationComplete c0 = new ValidationComplete(DESC, mt); + ValidationResponse c0 = new ValidationResponse(DESC, mt); // validation with a tree mt = new MerkleTrees(p); @@ -129,12 +128,12 @@ private void testValidationCompleteWrite() throws IOException for (int i = 0; i < 10; i++) mt.split(p.getRandomToken()); Validator v1 = new Validator(DESC, FBUtilities.getBroadcastAddressAndPort(), -1, PreviewKind.NONE); - ValidationComplete c1 = new ValidationComplete(DESC, mt); + ValidationResponse c1 = new ValidationResponse(DESC, mt); // validation failed - ValidationComplete c3 = new ValidationComplete(DESC); + ValidationResponse c3 = new ValidationResponse(DESC); - testRepairMessageWrite("service.ValidationComplete.bin", c0, c1, c3); + testRepairMessageWrite("service.ValidationComplete.bin", ValidationResponse.serializer, c0, c1, c3); } @Test @@ -146,28 +145,25 @@ public void testValidationCompleteRead() throws IOException try (DataInputStreamPlus in = getInput("service.ValidationComplete.bin")) { // empty validation - RepairMessage message = RepairMessage.serializer.deserialize(in, getVersion()); - assert message.messageType == RepairMessage.Type.VALIDATION_COMPLETE; + ValidationResponse message = ValidationResponse.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); - assert ((ValidationComplete) message).success(); - assert ((ValidationComplete) message).trees != null; + assert message.success(); + assert message.trees != null; // validation with a tree - message = RepairMessage.serializer.deserialize(in, getVersion()); - assert message.messageType == RepairMessage.Type.VALIDATION_COMPLETE; + message = ValidationResponse.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); - assert ((ValidationComplete) message).success(); - assert ((ValidationComplete) message).trees != null; + assert message.success(); + assert message.trees != null; // failed validation - message = RepairMessage.serializer.deserialize(in, getVersion()); - assert message.messageType == RepairMessage.Type.VALIDATION_COMPLETE; + message = ValidationResponse.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); - assert !((ValidationComplete) message).success(); - assert ((ValidationComplete) message).trees == null; + assert !message.success(); + assert message.trees == null; } } @@ -178,7 +174,7 @@ private void testSyncRequestWrite() throws IOException InetAddressAndPort dest = InetAddressAndPort.getByNameOverrideDefaults("127.0.0.3", PORT); SyncRequest message = new SyncRequest(DESC, local, src, dest, Collections.singleton(FULL_RANGE), PreviewKind.NONE); - testRepairMessageWrite("service.SyncRequest.bin", message); + testRepairMessageWrite("service.SyncRequest.bin", SyncRequest.serializer, message); } @Test @@ -193,13 +189,12 @@ public void testSyncRequestRead() throws IOException try (DataInputStreamPlus in = getInput("service.SyncRequest.bin")) { - RepairMessage message = RepairMessage.serializer.deserialize(in, getVersion()); - assert message.messageType == RepairMessage.Type.SYNC_REQUEST; + SyncRequest message = SyncRequest.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); - assert local.equals(((SyncRequest) message).initiator); - assert src.equals(((SyncRequest) message).src); - assert dest.equals(((SyncRequest) message).dst); - assert ((SyncRequest) message).ranges.size() == 1 && ((SyncRequest) message).ranges.contains(FULL_RANGE); + assert local.equals(message.initiator); + assert src.equals(message.src); + assert dest.equals(message.dst); + assert message.ranges.size() == 1 && message.ranges.contains(FULL_RANGE); } } @@ -213,11 +208,11 @@ private void testSyncCompleteWrite() throws IOException Lists.newArrayList(new StreamSummary(TableId.fromUUID(UUIDGen.getTimeUUID()), 5, 100)), Lists.newArrayList(new StreamSummary(TableId.fromUUID(UUIDGen.getTimeUUID()), 500, 10)) )); - SyncComplete success = new SyncComplete(DESC, src, dest, true, summaries); + SyncResponse success = new SyncResponse(DESC, src, dest, true, summaries); // sync fail - SyncComplete fail = new SyncComplete(DESC, src, dest, false, Collections.emptyList()); + SyncResponse fail = new SyncResponse(DESC, src, dest, false, Collections.emptyList()); - testRepairMessageWrite("service.SyncComplete.bin", success, fail); + testRepairMessageWrite("service.SyncComplete.bin", SyncResponse.serializer, success, fail); } @Test @@ -233,22 +228,20 @@ public void testSyncCompleteRead() throws IOException try (DataInputStreamPlus in = getInput("service.SyncComplete.bin")) { // success - RepairMessage message = RepairMessage.serializer.deserialize(in, getVersion()); - assert message.messageType == RepairMessage.Type.SYNC_COMPLETE; + SyncResponse message = SyncResponse.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); System.out.println(nodes); - System.out.println(((SyncComplete) message).nodes); - assert nodes.equals(((SyncComplete) message).nodes); - assert ((SyncComplete) message).success; + System.out.println(message.nodes); + assert nodes.equals(message.nodes); + assert message.success; // fail - message = RepairMessage.serializer.deserialize(in, getVersion()); - assert message.messageType == RepairMessage.Type.SYNC_COMPLETE; + message = SyncResponse.serializer.deserialize(in, getVersion()); assert DESC.equals(message.desc); - assert nodes.equals(((SyncComplete) message).nodes); - assert !((SyncComplete) message).success; + assert nodes.equals(message.nodes); + assert !message.success; } } } From b0f9d72840ec13030ad97ad77bf7478a079c2f6f Mon Sep 17 00:00:00 2001 From: Mick Semb Wever Date: Sat, 21 Sep 2019 18:53:14 +0200 Subject: [PATCH 68/78] Add the how and when to perform the step of incrementing the codebase's version into the existing release process documentation patch by Mick Semb Wever; reviewed by Michael Shuler for CASSANDRA-15333 --- doc/source/development/release_process.rst | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/doc/source/development/release_process.rst b/doc/source/development/release_process.rst index b3c403215fd7..0ab6dff1a40c 100644 --- a/doc/source/development/release_process.rst +++ b/doc/source/development/release_process.rst @@ -108,6 +108,15 @@ The next step is to copy and commit these binaries to staging svnpubsub:: svn add cassandra-dist-dev/ svn ci cassandra-dist-dev/ +After committing the binaries to staging, increment the version number in Cassandra on the `cassandra-` + + cd ~/git/cassandra/ + git checkout cassandra- + edit build.xml # update ` ` + edit debian/changelog # add entry for new version + edit CHANGES.txt # add entry for new version + git commit -m "Update version to " build.xml debian/changelog CHANGES.txt + git push Call for a Vote =============== From bc5fc8bc2dc517e2749edd73f6f28be3ce2fdb95 Mon Sep 17 00:00:00 2001 From: Marvin Froeder Date: Tue, 10 Sep 2019 05:56:46 +1200 Subject: [PATCH 69/78] Move chronicle-core version from SNAPSHOT to STABLE, and include carrotsearch on pom.xml dependency list Closes #358 patch by Marvin Froeder; reviewed by Mick Semb Wever for CASSANDRA-15321 --- CHANGES.txt | 1 + build.xml | 3 ++- ...SNAPSHOT.jar => chronicle-core-1.16.4.jar} | Bin 218156 -> 218599 bytes 3 files changed, 3 insertions(+), 1 deletion(-) rename lib/{chronicle-core-1.16.3-SNAPSHOT.jar => chronicle-core-1.16.4.jar} (63%) diff --git a/CHANGES.txt b/CHANGES.txt index 1a3df81bcb2a..96be055a2ded 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0-alpha2 + * Move chronicle-core version from snapshot to stable, and include carrotsearch in generated pom.xml (CASSANDRA-15321) * Untangle RepairMessage sub-hierarchy of messages, use new messaging (more) correctly (CASSANDRA-15163) * Add `allocate_tokens_for_local_replication_factor` option for token allocation (CASSANDRA-15260) * Add Alibaba Cloud Platform snitch (CASSANDRA-15092) diff --git a/build.xml b/build.xml index 4828389893ea..3eb074fa226a 100644 --- a/build.xml +++ b/build.xml @@ -118,7 +118,7 @@ - + @@ -772,6 +772,7 @@ + q5cC+ED#HC*m$aO zU1P3h-G<3mt79TGo!8@+N522N0$RSBJn0F8^U-@og8&c`lODo8TPU9$IM}4OV9!}A zzB^sP36EQ!C=xhz+%sRl(}#m3`2mk0g^Qe>waWt@EA^bmhl5cy+Trs^56=FS;XU(9 z9MC`T3c9XI&(blYs0THYRWnyozN*?q+;V0tS!=$q#X-Br^l%En7I zITHfxdg(e#naE4Qc1%!48Tsn2`2m3U;TC9n4BI$NtBFOwQh77M4eB;$HPo4*fisZ( zq=GHmHoG4C=GPFTB(n&?#ukK`aEmmLJPJI}6zbA8*L=CkHLc=J=HqWSwU(}nOqj3p4Mg=ksdto;kh0oS4Nr| z4TA)EsPq@kJBO~!uNMdr8METJlw_e{zNEHq(!fAKcws<5{@2lsngYUubAr$RgW}-) ze=!7H>MzcMYl8oCw1LzRvVZ}&L1qMyIp03g`lb;pW!#J35*Yla$ZqzXm@+CRbZ^gZ z@s8~%z&At4`;SzstO{})a?A~! z&YfN{m88ek?AA7|;C?J`FRyJT^Otiy+TuNdtHBI_13cHleD(vxM=A2igrspQf(cBK zA{^hFA6MOyc{431(}E&Q82!$j?tkujMjOmw;cl@c8u_;}XXP zx%0X2V02VF7O+dK?BWg*FkrJ}5pHl4hHsjwMM>#h2bX`!OnU8dpFggTW?E(4mZ0Mp z?QO;am})5(>EL00Ciq8#kwW}F*nvIlfAWiQLP+-ZLVAGmio^NtH354eT}o}}N-Hv5 zr>yKTUMA|ot7sr!NU*f`lUX3vPL zD?Paa)!&q79BaDWg2=6%x4MIVTk&#v#Y05_5a@|>vS{*y6sW8``IA25_>HjOi)naaSw?G>Xq_K1xMbmMUm^;?G6*7qXyc zsRsAiH8NeQj7ECoPPVbTA#~l2jqE^`^(^SVLKjG(SMYW-5_m5*9w&{Rm} zLYi{`rjqnK8&+sL)bPNc*GtTkVy4W0%q&~^gfU)I0@1!(p89o`ihS_kCM&N;bqFQzGYBW^G}n$mY;ZbjL;Y; z-cj%cVa_O#-n22x*1+=wG;RZWEAhkDwLCm6etYwVjF-X6*1{rMtNA-xjSjv!nqcr% zq1Z=~#m`W1%{z!cQ~H*z!u~A2fPZl5vYPHa1ABCVfJ}DWdAwh}OTo)i`}u z`3pppxttS)518ImjXTJSsjn939aRFN%Psv_FoR^a1@GxyTRp3Trfs(Xxvk+@u}^se zKW;?n6plbKs{-`A>2>DJ2OI3B5uq$3x|!&Iq$vseCf(M&uY47cqL?qSkic3s!hT8w zhI!pCgx5WH5|AGmSBoYsMD-HaxEXe{Ge~Pw0-{4ITCp&OkmGOxzB2U&4(SF0zo*1? zD;|A8yj?(yv8omF)ZWTL2uqvvMAqvHpb&w=Ds)xiL@bVsk`v)xQL_UK!^a>q?|9*v z9%ox~1&D*w1E~nmgUdt?lZ@p`0vQRMXxRen1fu(=rwJq~d6&irkXmuRlW12zpht_T zp)BWcLoHw?H$x}^>5>YPzC#UHzO=VBsi2j&KRbFd*2C(5${;F5 zOu}U?Gv1b!THO>}r)9D{YdlmGyibvc63j!Y*=hbbXbI?m@~_(Z7-;v+1Wwu#;+?yv zm~@66S(*cdq^(iRM%d1V!DGbPW*NTdrNxDbA*^L4XJn25x)v|Anp&jD->nqnf)RId z>PVJU6-Y=P&v^21@w2pJGx<(ZsSS)d-l}qP20;C>F+<*JN8@>l?Z;GKvm^C!=9bcQ z!0<_>b_(X#1<=g1XY zXdy0uIFWjqIOmcqwAYdJj1WYaOt0G2WolKEw3PKC%$UgtxzBjeS;cI(j}1he%@4Em zn|}4Wui<{@bd**QA*E~ahxa}=gnbNYckyW4=>B3}c-*5b`Cs@nA$Lvfd4kJ(Nvjdr zzIB7AzeWb|9GBHcMxu9h83{{sJh2(znL|DS!{QJoHe3vkd=U!jC1PdOzD+R3zRKTp z&1K!ilxj|d@YD=5P_9uhFYC?QhV*Spg5Yv1-!CI1~07Rm zQ^WjsNIP@Uk7>nENiy$G0Q1-6JZf5;R5ADSGjWgBthieS`n1N3eW-m>q&Gm<7c~3V-!oK8bha6=tBx@0wXhEpyeTNU z>y^YQB6?4o($01eWm;-jLnBW{FL~PG$~>w4sVTuZhPz&%HX#ex_crWhyQ}}O%4Vf} z>}RgaP*hNO7E@G8!d%UN{nMsF!xYUsK?TEtfXLDP@3!sVR#OlK4=9LC0se0bXMw!@ zx7`#(5&DaPD58IH1V!>MBB3h$MMG4rzu1YY4?Y(()r5y85B`_1!DRW1xtKhEaUE0a zFA`%({cD(pdIUxf)J4N+s=~?x|NE&t-VNBlFF;|!`@bfF6=3vf!l>X7O%~r8|JtPW z6#YlxrQM)_LpNEG_W%9l|MDbFUKEd@f89AzY5(VUnwFpJufM+`xfF)~CkTC79~wA( z6NjP`>R-lhoxW(v2Bl?9BZ!30<6DCkDw^@Ko6dr_FI8ye6A$$Cg&gFQB&zI0+V zk@oQ3C08Ue?pkrj-zu4eGv${VTgkDRy~C)S`4NJ?Dr;r-r*^Z%SQ`9d>HWY$nj_*NNg}+VC}%MC zV+^$J21c15>D8E6CY%viT}AlX5Hv>j3u{mYTmus)vbGnuBAG5`cop zSA1S|tipC7pPnJHM&mGEJ};GxQix`8(8#JT(A;Wv@-?>LKu%&MGRgqJjN9qgFhOsC zpEreh+1)_kWn@pdTMv$EB!4iTlVb%|crl z;w9Nim(NL^u@+Nl_LI@ckt1B5DS}j|h}Gf=KS%slPsg1AD@KCxb$@b)*|R=M%UU1H z(a4B+5x{`S`8LHWXTSiIaKy(u3zXTDUZjGNebTHAx)w7atbz{Od*U62q6PXmlAwx< zsA!cR(eA~v%{b}LEF<^>s~zd+kf_g>U*0&3aFA9~Z}tQZP07G~cHyd@Wjx&xsg*#5 zA4OA*zn9SXGc2|N!JPO+z;Typ`k#`k1;pJKg$W_Ch33Y~4Tu?#>AGpHTd5q5lt>*8 zN<8I;HE+3xo;dcz$!|DNX*NlB&q;IpxysG>^ML_Q^x)Xg8^9VTax(0ZeS70-ac7a? zXXhW+@41|F#41}1*3s^48C~g=#z1y5bGa?zE&vx(^%KOOm}dF&H$Rdwrw%n; z=zay_R*`!a-A0dvNBV=EZB0&VM{QsGLKhDym1X5luD2<+On*;h4&c6kI-O3{k-caG znR%0X?Yu9J$D1fZ#|)WDK)i`ke_w@H|LOFk@LJ^F$1 zQdiLS9{|%G{VFL~8j4@IESbVjNm~c#393Z#{-hmwM?t((7>cxcCs-0jqMPyOIo0Np zxfuP2-i!c5gdSv9Wm-X$=Ki-6!s52Xi?z%|QK7n6J8a zHshh-)k#x$Yb%CRdwrLH#-N4PUu z(YIRTG`c+esso>eLmy9ny)nXCWV9HRF`1{Q>Od2x71s4BSY+)`U~9f?`v}G6Z9eZm`>iNnIV1XL};PI?+ z;+HvWmOpz2eTP{aZzh6|Wc0i=_KRRzhT4Ffkp~st9%i`fkkBQ!EU7oBE@hgN{ZG=? zrv=BA7#L5zX9$S9T!%BJ-*3ka41xFN4@jxpDuXU~S~Z1a6x9gm^*Y+)%^uNaqoalJC zYqssKYiQ2vE6cHqD-SSqN@#8wZ2hj?@SShD{pva1&X5~Kx%u+{d4CG@y%Km%ZtLrN z-kf*r{#t4hXP+vcpg{%qaLuWNZGf$@4OHo z&uEx7#dUBe4>WvG1??k;_m{mgc)F!q=snwae1G8!cw+}>GVTEPcDP=l1Mq07K9q6+ z=CXOzsRn{&0PEC4BuD`#Jy|-u1l-HQrs$_I&s8a@H3d5kGAFM!y(cGCUujp$peCvZ0@U;2 zP9Qr z`jhdP?kUr43iEW9$upF4!3iht$Zh3jor$D8&+X^+_f%6FFER2m&&o3X1iNWWl;|)n z9%qdAtucQ<% zo=pXCXiRkJ2z_YPq;_#M8sv1Br}aV!>6iXGx>IODT732I;QP*aZtg_Jd6;H)1Ep&J zP$4haqMbX7>C8D{9y3?3UhJA%G5fr@bq3a92b>Bkg?l>{446>c_^8xwc{vR*e^;-v z5o=o~@g!OMBE@~Iv};#oG$v^vgsLTtAC3g*{GqMnm2+=sX6YCPAEu@0y1uK}A`M*= z+|GeVyq=Ia{+3?24$r4XbXO(2R1)Bx@m8;>&$y-evXmm|p2FNi&2Tf5UIthaxa;ua-x-b01r>r$Ej8C&T(a>pG& z)?01X@%5!%aqDW|SN>63b<4PtFT)R?>~bI;iEq?wS(4PlPMaGiVZ)iyRP2MdRCr5b zK!4!OiFrX|D?Kud)E+*)8{g(vUXMx0w1Zwlz{cJBqnw5ROsDn6nI?;TWz1^SnN>cw0-7XdcnAn2)wATr4>(iR@>W zPgZtJ;R$+qvE-tc9`wc3*Yup&)|2@22_!|54UuuFgx`{CtY|7`9q#y+2j z5QDU|E_u?>>a448KQ!tp0_JcIuubLR%Cg1JPHWE)M5J7(ZL7tOu{Nd^LOePvTsL!B zSKO%Q%oN30 zJan|Hcahl*rDh*(b(!Uf@={deXD@X};+Jk@V6 zhL~|wc+J#@)?^nm-jgG$GQk?7+}xNO)>YX0_OTbS;-25lgflxJ0f9WTc=6Y4^3+6g zr3)m5!oLI-y_b}eE0{w(wkXzZfv)Hhe6 zpFN7{y&b*;ur+d_USkHH>mb`Jy88XU*U!|#l9>u49#&D+NuneB!^AXgwnFdo73ktF zmHQfZMji^9ad{?j0oK>?uwRe4uB9YBeo+lYDMg@ImW(!&`J6^+%^{4SE{e^v6?Xpg z4f3H^b546}6m>6aCLp%i?99Y3yx4ZgxQhX& zHmZdptYz9b_gA9V9x_On2mypqr7w9DG~bzfsP4Xx8H6s!weVT}kaN#5>#^}bf8AIY zJcCvZX3EO?d*#ZiVr#RNDmu$4JD2gKNN>D|rdUq$4;C0x;!pj~ebOSkEptvTZ)}YLbq{2$DvF4W8dM>ZVL;fCVfj-&Jcc^Uz77(GF8@x_3%8;Fixb z4N9UvX8}Qe5JD9-Q%b~{XRg$6@{$~GOd1NLy0+X&A!gJfzG{X?X&bjkTIFP*UE2C| zjqIYze3v~`R#GjZ!$XR#U@GF=tGzuG+y>cx{L`dweY*tU>2C+0tw?j(xYy~9Dw)g`6J>a3DT^B9o$sF<1ffi?i zrKcI=FfcxQG)8U76`!M4tiSB|q?YtHv?ye?vYDXmLg*j8-G%6#`pxMXRgksx2FVf$ zSOJ#^^ieFDpzWDwfhFG#To6=|Cl29L-$)&smxlO%sX#B2$ql{nRh`>6-r>Xqh&gJV zDZ9gnYw!{8s3hEKQOg{ubHoT%66j6eGb~qh(m%HQ461A)II-IZ_XpFcr}9>RYdHD2 z8vaR|wzGg=de*Q6Jfn%RoKPr&5+8%HF$N6e@Xf(?#eCOA?+Rh*qGhO~)s5{U!-_6K zFRaqpf{$i!7Z*UZ<)Of9Szhxm%~9XO*NB8XsA|Zy%tikw|5zw-J^zli;u9zr@gc%(6G*2iTA_b*eT0|&Q2B_q)~Uh)YK)2wjM({s#4~)vqipYSOPd6_#0ot z%9%DDfZpmrMtm@EWuAGazV*C?lzf)wrU{^&g+Je^0(lYVQMF4y)_z~YAZC+oK%2k5 zx-5l6DnY4-(3xd6YA?=at7;JG=wL;sQIJXd$E#q)C3i?nC$oz1n8!?pHuVuVOlif` z?I&n;r`j?YP^4}7@!avdT>?s*ro}ZjwozM`43~FypW}&=rKM6olt$7ZTPS3`naIBc zN|t)-VPo?6U(?TsQs2tnuov@trN>pijwH1og|I$yiQBlAbxYrvs%3j};6$mWxM4tL zi{{1oQNdRiK5hpL9usO~rOSZ8^UU(F}1ON`Vg1s8k?enbT zMpH_D4dys5x4VNO5)>=<^8Tc8=rwzS4ANf1OF7OlLZe)|lJR>j2#Yt`l8ie>xB(6q zMK8VSaDVW;Q;l(7vBqLWAJfM3)CACM5ii9<2A(9(o16hVy zqhkwFm17Z15YU4@kpKlb3^R;7mbgRSvDV32qcEoAizh+Ok><&GMf9fZ3Ah&=qr!=j zxGH&|5l=&cQ#v8cxmZ&;V=)Z-2=3FYqxk4@riJEmtdv=jVdiRhsvyRVxQmR_m>~_Z z6(-l~#Fe7+mEyh;f~|N^&p;=G7X4xhSWM(3#CEy6;}A(qDL^ITu0nxLrnwxKg%-c2 ziG1yBe{i9^X{q@%tA*}^=3CZCmRZq6nLoYVEto;%CVt%c2)!v1r$s4OVEUkos1dH~ zGJ0pE<0xhc@^IsZ^C&A%bB>o!L^HEfRL{+K@>$wA4Y6`ev>CpO^mcM3cMs}~u{el7 zMpK~1Q!m?&NC4d44Sbb2@A6iKszT|JKDj1^AHZ5h6*O*&WZ)e!LEP%cPwt#hq+!l? zaROeaOR@#6xfS!r`5&&l)X-9q{l1DSk@iXV6TQotPs#?qiqyFv`ymZhGf!UZ*>bDr z?8J*>N+Qbb;PCL1iriZd(X^!0>ZQ5ANbDRbj%3AEK>^fM+yDG78l~4#$9*);hEi{5 zND5f!?sbU^mHfOXx8ClKZd8(?!dlvwiYpy>;ugmetJnz$a(}n)2-8sL7G0fQgsRr( z@=`uR@5fDC&dlCegp!5jv%y&l*MKT@bRL7E_e6O(`~S*HC$w zx7^Nyw*#)u^y~GI>)plujQhb)tjKM~eEru2ng`z&g>J|XxNc`4%l7@&5|nnyfKaO* zrsUu3T2!gdd)LE95jha2G>D~;hmDWuoyp#Rn=unyTh7u*Pl2)qJBkVpv&3BtO~xBl zqXI9bnrfi&-pGOq5-J4_-%J%iJ5wRa8vU6sumG_qF0ou>oh{I01Dv|Te(wHn9_DlI zB-$e%;wpeE6+WCF$V?fMi7Y)o+adiY!+>$3Gue-(Y8qcZBR3p!7bBh<+V$xyZ?sKC zFDG1Z)#9Y;asqlJS5Ok(6%@uOEx+Wv2kBv9QGFFfRTEVWs(yOO6a;|=?+!1@z-EQ! z8v(@clvBw#x*sQc6|flb%h4p1B0gTsA39GQbOmKk7e|(@bT2NYr}OxUCH#9xt7;t3 zN;Q#(-Ez^bG%kY7Cn6R*f#9!86;={hVY$iG)Pzsy|!Kt)wo`8#+K)C zOP46SH*Vezfvdtu6Xy=Q0a8t9s4;$Hf z;GUkhm3QjO2@%9B8*MJa=Xf=~(uJ^0oLBqGjp4@KZ*#4u&NqriR zJ0@>hJp3UJ`3khwQf!g;q#XrK7vKO+E`m&Bm6Two0c6|^yI>G~Isrx0>wSzUn$MBG zIjwPZ{O3aB_X(HaHf|@ex_GutezFZJw(MgF?#J9p1#(V|?gh799fg1ELOKj(l9AbV z|Lyg`j=Z>XO#0};Z6%vqQX9TiCnuUrDEVeJJ(1H(pw9I@evR=50KkwEj^2T1 z)poj+!_jH}nxk$&szc>%tF08fRoA!PY2IHBT9jd(-zqevWpW$xCvha1;}s+twzX7h#`dS5&n-p}hrI}RW%W<-*NUO9ohiYFKjf}Wt6Pmm+?$a&jYbo0 zcXLcp&I|zj6Gh+{TMH58i4?;mqY_Vzv{dg&)B%D)K?*sDL?kU4Ns9*`pw|b;aJo3P;FuX z1%EI?&#D`%Q~-*VhFa=3%4<{bYO^~jMt99gb^gjgFE>ygxZfD9 zL8N+}*5&nY?{f`G1ZE&j>}gG@ahn^8Z- zDCnisi?V`2*(m62_r=JM1BP3I&e2`)F@RU7B&Rw-=|YipXUt=!@!XCA8?|VYb6+6u z0((OK4}M&myYX4Q0!&JFWYT?qTIP-CRAlTs+^%EU9<|rEkRi=tim^1XDI$_Zr2eg&d z7AS4?IiNrrRME)e5Xe3zhRKw;ChfYSMi4AQn+bAU@#_+!J#kUNa|73b4;_l!?I-l1 zcw5mk(wsBZ;e!6$udT?%XQNGUty5Qa+)Iqkgwll%96%t}Vn}oHHyK?~LBwN;Zk$b| zGC#qXG^3V^_N+0E6V>MAG*T>dCRg*{v4&UcP=FVqc>C1@Oc(8r>4T>Gci+2(OVR-emhL~`FH`&xI(3I;3}kYZuGGgl(@ zMJ?UiRcZG1Ydp)3!_hCT_l7=B-e@IjXYxmsxd-gM;dx z3IOIgrI2XHuERSj&DD#|)yI-)smeaw2F4No(C?`Cst8;C_=@Dd&VWWk=$}~&WRC9l zi^2WcFa>CjA3+o_863#asZZBhM;;hz5UHMv(H~|$NFaQI*+{JWyEqV1yO0Do;3ybi za0uWSIIuX(NFe#ypSd0+OfyF_r>oXKjsg0#?5~@_zbX3tHiHD<=lxm7D`t?Tz`sL4>)eBKK%sR7X6H5kLOAH{p3qx_1x( zv=$fkP7-`2)|jcVF^c-4$ory9LYO`#NL9lbbWN{(jR71VPVv}U&1_yn(2#0XJ>5U3 zwGMXusr!0f3Wi|Mw_;+W+$v=PI~p(m*`z(68lW+}&+Ovc`a}0UAv^NN%$j8G3cw6f zvbBGcFv4dljN-SLCD_QM~C- zC)v`vns@qo{O^JYkA9sPYPWiz$5q)TDI5{8BdP?FNsKZ34@wctg;^08_ay)`cp(|N zFcX>%XwnXIt~}5SDdk=-Gs;LP;&@)KIFO<{&<1tv43$*%H&YpZs5HLZ45aLRJEkWj zd)&stiv@YcSOX2|zIexl6*JERXP}P5wu&MiH@nH5#O76Sa2GKJs2ZJ5xv7vm-BV!H z##N#8cPk6k$!W&$3`}3RXhy(ksTT1SNaIj4w~q3f*hJ&Eb@hwWf-QmOW21Fjx9I&A zcAQ%a$~+*_*J>mg!SbZe0HI%l#wJ|6bo-Jgny5e~zSoJId<%BnSwpONy(secdKc(V zlNa4f(ICGB{P7&XpSzCrAXhywt{I7)&gegP z-S9Jkzn2hZS~^9q>BAn?sf)zeNwh=fU4D0EgwLVG$2^(Z4mq%vw-UBBvB2WQgZw4| zNJTXn03-J5#|SY8A+#Y{=^AKk{ce?LarABkk%tx(tcDX zUuY21Iy}`S(iZ8d81D6~0=Em$is>XzVPk{5+I&BFMy)mW$KoV5$y<}w9BQjQE%hHe zS%2L|a~v6OG)dUtmsml4ocn$R2GldWI4#AWW#II6%|hyBegF(?LUBp=uL5;!Vp?Wq z<+2v!7_fhN(x_&hrlS?}8OScc|Ezz3d&u~}@wB)3+0U6WrFp0BF!2C#z%cHKRtY18-5QdJ@^x}|P>jty#eXDj+<$^_ z9q~Y8olyhJ^RonhA%ou_u6hxTyK=W}4fC}A=E;m-K&pQV-`^)5(DDkD^jx6Y{Eqw$+H;|1)>HvWa@&Is@tZgFVbnWOsp2=nwX8G3z7O_a zUkET;$RnFbBYs#i9U-(Y3TlI%Buv{mL5-;=D3oPzF@*3-^anPp10i>Cs(x-(KMawz zc~!FaW#$(EEzApw$o`cpXM`O>$t1)vj?>kxka9P(D*CI%msO5f>Q8% zR~%5PG6+*5yG+wNjT$3Cw> ztWJMG)10LVs;UbFW$aA061xyKy2~R(-WA;~p~T~3{gOtO6`ejvM4q%ehUw{ChjPdKZi{j!#0jj~4=Iyhtt1Wfm%c}KxRS%ukGnBeti#O0?0}Gd ztDSy`$DK-n1J^Z~tB?;Ft^iQ7>%(_F-#+wBxIQwz(;v|+MV+Vf5foS7FlEbEBG?=N zm9t^6%jX_WyhwZvrxM(AvQCYTB8f~xu7=WMhVc7>KcBm9xiHQ9=S>BcC~${Y#1!3B zJtay&(#lTiXHx~1$jacLMs**&h}t#*L1j6KiVzo})+kN}sGh|P=7|b~hI20_CU}N& za{Dn+)_yE3aZHq@wku7BB>=4Aby zLn3!khf%+$G(F0Y4|Md>+>jG|R1|#_{vN?qW=TK}%B3H5D;NIdkH9h^jXA~+2o-Oy zg&`2(ueM|5J++1yR`7vBRj%E7s79js?t6k0Jn0(2K8NgAp)zO8bh1(<2bGjly20bL zF&$E$0pir81e_g>QN`eHl)-wKFKGxFT# zA9L8RrQc{&u^E;qBADt?B~+;xE(#@rdV=x}+uyuLmyy|{rS%k%cG(t0pi%UZSYxx| zExZub2|vfn|0$^>BkyGcD8Yy7Y=yKWv;1V|-vV%!30z8X5J_973b>rS3U) z=oDyldPY5L3A$f?f&Z(i$W~r){!I)5A}#U1H5LCzDw-NH^CAAK02?y-fC+gp0L^m% zh9+(R<(e5MP`Z&mbRk$C4wS3FxP^hi-iq9=B9sv{DH*SMdE#QiV*Fy7mkqz)=5}G- zr>~&Js>imwS!cZwwD8gs5b*GI;|us4KAxU6NTPoY3K(B=^=A6{-hJ&mdvN{Lar-SW z;wCrTI9vHv8j;~e4Vs;wF5>tAaAO@eWciO|__vCMMy)(14m}PN+ zHzL;T&Ri_<>!WX{jYfyFuSK6Fr-h4VsoUj?LNb-l=aSG>2qDd6$`OSjrf zFuz{`I`-`dcW*NJO_GM$0$}GJQCD0H1wly!IXc z9)vZS6MSmMH+Vo}Es#igZjTo@h!w8b!hp%K;dFO8k+hG~)S5305MksWvZR0^L#=lKAjrVB8-dSCkA;_t@Uad;?JaWD>kfY z-QM86YO>&FU5wd3dWoD-qYZMpiHwTHRGN_V{G=C|a0sq@?aBx<^%!b?x+1^ZVUD&o zrH9zyf`i$smto8yGLG`E=bk-mpfp5c`C?KX=LggG+k;RWON=ys0Uqy8n@H0i5vFbz zB|n%2_Mp%yHcI1)$!6yt&JbElp`;tmU|ssC2RUBVuwFf5e-Q9EMw=W^?7w zrt+)uTTkdc)l%5z0n~mugRplUKlupD0KSh27Q6dF5G~&_f1AH`CZPD_O>VI9{&vg1 zqFVoq#7Zr|UujfMlPhOGd8d!kQLpUA9|*mMDZd^TSD%_cL@_lh&9yV~tmbDphU_Fg zhI~uL(_Q&{jMreuoKOf9*j2LgAfDbI(E3t+24tsd8EW3L5ekhlxyc#p zj6o=zq$h_yEbAw1w2tF*c9EjA@EVs^w$$yXyF*vQs80!Cy|fvGRudp|X=P_YiVncBjYGKRa27WlxQ%ef6}mbaI@7ESEAqO0LK zznpA7ZhhjYebqL9Oxn3+YpEj+=10_)yH%bqU#hN^lulY??Lt!Mvk&Nb&Tk;FtetAR z{s5AbrRkCx7I=#r!0MOkQsaHQuGmbzxNlu*xT#zf0W@+s?VP2&Xc!D^wAK6R2xeW^ zGpWiGp+z-sF`F|qs#U3lmLJQ+)4;2>W*O3kZZGky@R_9F!@a5EN6U0=xkubBB!Xmx zvZ|EOZceTGEn(Qr!0z0M*LsAF=hD2=ZZA7yuDEX|BGxz6TJpqP1xs1R4cg5b?cuwI zYj<$)0pK4~qWf4ktIrveqv0N$$50JUl;@6x@NlCq)OAkE_YUm#kkv50|B7&;8##rS zxa}j<&~vEfYyV_7U42T%9IuPcB`tv5gXh1*rOeyuqgnOWMWt=`H09ZZe%*r_PE4)? zv6OyAxoMLy7*cxqL4Ev=zHzY8?@guIy}%Hc#Zkyh`I}Ygch!6n)6TAR>QG)EPiy~BB+|I@}PI=2j+Md8u!SxNJPke{tEKl zJSDMLFoA`W2&6}MJs(17(IYP8M~3}vD8QNR>xu2Ep2(NnU9b6tpJC?7`C=@qM^E@= z!A&bktr*#s`fEv6-7nSli0%mh`R`T5U1T=zgnCXSX4r(=IQ+dY@S^&RW*eD(<{J#~ zAA1;uPG&M)QV)F%bgd=(;1>=9xAo-SJj!8AP8jEqa1&Z%SAiFxNdD#*y^w18{%pZjq9k*mUTv<`l&LELp-f*4z*@33mn~rcS z*(IG=L%@$3Xj0u<>Iui%4hakVhkXwJ07c1*i-ZS0vJczUt|!t-uJn^wIreQqXZ|5G zja_~W8PKdG9o%Qba6Dbn>PNz6Lv7QCE9Q{yyhpJ=-Nb2kUQjE8yO_xh!Sv>E{ycYo zFekes({7f(!2Ye?T093zLC_!|-%$R)>iu`Hph>S(6Z`MDfiNL`8WR>cJn*uI;ICeC z;Lkr31phK=@mT+eCe?b$|Bef!^_={pqx_d|YV5Q7Z-RhiK>5ElPzdn~14Y2G;hxLX<#`^y8JsQxF)HV!yOlld6tf9A}t{}Uxm5*Hk&X?oo7FW(?Qg;5P*S1_a7&%KlybI~-xS2h%jWY4W%G zNh7HH$KG)7@b7ZBH5h%9>+kyib|;TVpN2&QjshsrhV{cgOac5%5@=<2COCG>7Q=xy zTWdfgJ|HLeK>MA~O`<@MEo@95cPVWaM;DjPE!-hfpcIcKBh5XzCM>%Fp|e3QeBvSN z*|pWQ(Q|#d{ry?6Gx_3cCidV5eU}ASU*2)nYwvT<`$yk1&CdJHW|z1dCZH<%&jTIk zn>yf)BMc4h!za+^l1s8191^Bzh-l%yH`Y8sgOR^x3B2Rq^0b(KhOo>EM#d zwj%aZN9vPO7$6IvO6W_o|3b+J6ca(8YxiwDmfwjd%wER@Uw(Xq#uHL!)M|9o9!sp&35Vxe>OB1Hff#=?^Zof-F4MwLz z2!6{3Z}z9rZsP1>`epd(G2(?{IwWO84$iQ)Qg=5d>6$w>CuvSBoyqi)3a{L!-0?fB zQ`9NH*sS$oeg1Y&={6WeY;0HFlLze$=c4nor*MS5({NC2?@4NdJi*$oy+NWZY4EkZ z=zn4?z1ag29La%_0j-bt!E}^>Wk+nJrW+6BZtE7s$auf{k;rI1a}6hjBKj$eiyzy~ zOTQC4yx#)<9FalAWt@JVpaLn583}fvTA7otx^aAb5yVHZJC#uW(X2Z8f1rn&ZvtHH3LIv$D%B_Wx zzukctsDBjSkPo2cq&21{AB>ppEV}66t|T20R3*c*M`hD(SG^;kzA)!O`6~m=jGj6o;sx zx4_bjY?hmEDfR}`VO;o*$9Y2WtDhZ2Y%<~5qbg+m)ZK+)&o0Ba1@Mx&Y)kf9WcOSx+51m(y$9sx@~(F}QTdUKLy(_sH7 zi2Q@$IRLmfHeQ*{=`W@ zE0Ers^uE+P6lv)~UaPewxU=eN=m8_i!3+Vl;|g7qyGQD$K+?0PkXc|%=?xKo^@Va$ zg16YVaFXf`G9HjnJiEen)-fsMVQ2XJ&q(_*9O>LREtX2B^th1D#XqPtx|wD=$Bpn% z3y7>4=~6^ONuzzYm@C58O=YNwN8j;Kgfr=f5unLjTM}&Hp7zlU1ln(;>w; za!^2^OdbbdD!#JwZtHa~eSzsz#ab`ATKA3_y}zPV?~DNo7G3@JMwOYZs84~Ks~Z{h zg!y&^-VJH=Xq7}*XjI&Sintb|$3+2^JBoB^q2`-PsQjqhsjv5(+MA>P=={?BZ~+q} zSaX%|0uhv>VwE_ECQx=o-N|2~*SRrWg}=V@bY+#4vu&a78{Jhp%gqxbM@P0uCsxU6 z)6$sptnvu-^abGW%WVtqVLldR8U+=FuT;nN7_eIPi=8G!r5o z4W1^N_)w$f8`V{LrTG!TMo;>#WsB;L=VcJ)a*Z4qG|n?oZTkZfTXEl4eXr*7?cOVI zczP<$TEN?JmWhv}MJARSXh0k^RqCC*IUKYw8DEtanT>(HHfMDdv)`0KEt@V7!`g2# zJbx_0W}gLfo#$wk4v||=aH=D6^}aT4GWl9mXUOBO-XB>X$VGAT%1K0^qd>~&^5A2 zr9uHZiq$*mq{c+%54`3_yQ@1IEB^WO>~NzYo2igI$qOfbEFO|^iAF)3r%^=paF#x0 zL0M}0=O>Gq;(|gRsVX_QX?voU=3H-8}PKqnw>ci!6gIJnki zDHpSlFE!vY6cn~Je1@l0!eaBsQsg(lovXDVDtH9i>8|?-Zn*x~_%0rgeF>Js^X-2x%aAmDJef<5-^Zo7fY0v z#d;R9UC)+_%XCj9?lF%IpUgeiOVgiz5DzlrVPq#zb5xueZBFJnOyn5OTxQImC~`w6 zupUMI>d<5u{bI$sESL6`BHcj>=w@9eHhVXC9Mdk1c&j!um8g1)rTQSg?U=Or)4ech zQ2qq-5L5k$`-wxf?_)INT)C#8ZkKP7>3Hj!&t^Sl=%&n?aD}sqpgc{PM`1}AC z71Yi9^200VGv{dPGzp8Hn{Z6qAA&4}ubUY7adYQ(>Ful4z4 z-1$ToU5$jCuVJA!7-Lm94%cPjF2|(BzTWJg6M_Dmp^{}M{8w2h);Qhuy#_WPJA@@m z$BVmQ3&tU>f5$n}0ApT63^jy+Xo?GY`D6&ja@#eT)SfA^Kf@ISGFGf31tMh%Jx8$A zGqZvdL&bgS=hxefPd=ll8s6|BBT4%w!gA_jAGpKQS-+!m*bU`FGXmD@K>ti zd{5%@7ZfzNA1mVx*mjGH^5R`54%pNDs5wfrgo2Lp$^{Yr2=aK)oX&F5r2gZsun**1 z-?`d^r5$CO9Mx7f6d9+{9R~|Ret|sIn#}h)O3-3NgRTkd_R`N&2bxtYNuKyrFAw#| ziqvti$n>n!Zx%Ui{-WP0%m&x|8c{x0`Uh_!c`2R631h$6a9Noa_Q@6kQ>i73VXq-L zAXJ)EUsQ7%QFl9*wD&fmof|eK@TaPapqj^r`lNnBC8az?;73t{!jNupcvy#_5|4wDxd`b-j&+ia+!l@zbB^qund>B;$A%>>PCdtQ|7p` zKPo}|P=e6+@fLsbV40z39ksb7>DV1Sf?6gH>fGaFzaIyF7ygi5g;%h+j3?Nk-@-e~ zA~TacF*Q@Uk3sn9%feBFpoBp6**C@-El=}6JW|d1N5 z#nUMV2H$vrT26qN8ac%A>!yX8In6AsTGJAeP60(tST4iH3|`&O;TZ)B2UhZmS-{BZ zjjfvNzSTxosxepk^SN8v&63B9;nukZvp`^xoCBA_UH332SG!xYIkp+4Xv^mC+fL=Ag~o1E+h%}&++MK!!@!J%KWMRc+YtAz`Bg#} zKx%^N@qAy5a{Snsc5T7#eGlt{;GoXc&=n10;q9AjrksFTt5@K6Rb2>~&z;uxK%R>D zQfrvUxutJ0>1OvCiePE`oP_0j{%3B*pKVlr$CaMQ+&M|{7bWYJnMvTUdgh(dn_qtI zQAg$j?dVkIcw}%n7X&3|z7`%7kM@r>^okFL*7zn?16ZQx_LW;XR2&LQwszC-#C?@1 zB9ITLEqy-WFHoCqvl8@&wGgOS^7_mPtP*_rat5w5)>6o$`VlCeBq(EIKh?;v||O}Mh=+=h2`d+LsQ7FCFX@O%BjTKs1m|2AosjpYMZ?H5xN!{ zsI41v>>%XE(WhH=gCia%uSq)evvdcI=23WDFf!8jqo)sS57t0WdfzD(Vms&v|G5@v z=Tv{K!sL*bq51H;U!r(}Vcxw!ZpFn5jQ(QrG^jTr54=i7Ug(U^o+-!8XA?HU9{S{3VQANk16e=ffTPl0*wdlU8W(RS zx^5#|adaiuJHvN_qcEw{#NEJHD%QIV?Q55Q-Z{hA(MuN240Z~;=0QV+^PKT(M8gT+ zdOXrZl~TisvP)zG+(g80n1(W_DU5{?o!kW;4>$1kmJwA-dAA+OyNuG^d6?CEYww&d z4)Rf@fvj6zT$)+DvcnzqY9x&>4xe(0e&R-Kc3{1H^4TN+rfoJ=1U%w(jUB|{cAZN+ zAzj>u+(_OG-L2)!Uz9-u)`?LZ^R( z-g7d5-jJ^8|FS)7m%NSg)+Y=I3>pyp?_14Zhs$45N;f5`a?PeUNCBP<~f z%?nY&8?Jw-4E&DkrFN6g7;X`ZUh)STQtf0wAPMcR!UKcbh>gk=N)3KHSII{q6ETRj z%y9EH`b>p6@*t=OPUK!3f{~hQTxmz!EP}PL4t0=OwB}1&Zbr3<2H%uZv;7y4WyV(> zd+a58t>F3~;_+*@mZHEs`d?al?y|`VggdQi_j*Gda6pWf3p$^Crue1jx-|Y?7`X&;-9Y2Z)gy!V6f>y5onOV z3Z{x{Qh>xF?pMMi)zr>?1tK&))r-mcsnjubBfCNsCL<&!|pieb?wVjD9{ak~RI{OAWN1(k(JOn+9IHG<>%(+K?7%9l#MORn&>94v&|w0%ebRYBZuBX zRS0t`PH`Y4**&Oiwo0kEinqBGEK~bwaSR5eA!@>CRdEbLs<#o^%eLqdN-mm$=K_2n zDMXimJ;=gR`vy#zjy_6*!?jn0fbQupF=<}yun!!-Dq>TLU~x?w)}tTQeMi~Z8~z(T zqQIY;Y@1<`6V4bV3=F`}T<{T=dgH25VO`J2nvD-IQA8%G#p(?Ah{ z=by}_((p)>n1vk*g;~#ntE7%~Lk!ODKN1F{o;1uY@92KUXaH5(%4{1D_7N=nR^KAW z2jaaUKD`}#+Gz>4K<8N871Y>Cb@S(_1L}Mc6w#kQ!fdg*C(k;=e?&&zfeIby{|F>i z2=!o_w)Dv?wU-8$KP}URZrGHl{SEaE5yGKhAES4vC`A40mYn{MA^(KG0lCT_hPHu3 z*ekvhCp?Y8$Wk&~!)a1LA{E}hf-_f+w72e*6D3vRhGHJhvh-8LaU=D-!cN`^8%D_v zu1G&(NPudnzL36pkv;6K2$mk9Jv?v;0@xD3K*C8SFZ1JR%%m^PIc48_5oTnCnvQ$9 zuuw5%r7?~7vR*!~UUu5j*U4(XjwdRlW(By2I+X>` zZK13F!}<+09>?zGs-=$%U?&VW(+;8IO8o$cU(%4ppZnnRl`M{q37PjgZL7t_piC(X z>P}QGJG=Ils2_3RbO@>-(ZO~6E$1o|>}TO;3G5ZuF)*A7dV2$eNGo^LqgE38typ{F zM&>YV$s;~ANspa+1l)b)p)dN>IMb<&)lUL^_pj${m<6*#eE8rB`+rS5!zX||fGEZL z695-@53y!#(^#c0EH10wT9*4MRIOpr+Gbs+tyI4;V|7)>u1iDg@y|gAr!LF4&s~gy z`yFlBr`flY7bin81SI)o$LyFL-OQf-&X>z%F%wtIBr#Q2&0Hyr+p=aOW(=*Nj7tx& zm@SV^X>BSy6?%zNnz9Et+V%1~Hu}5Uietk-i`~l=z0T;Gr3Yt)KkX60*9fz(0TB+n z8HKmvQC++F$`tL4Hs19S?dAux1SFm76YdvTIOk|v?jyl%>o4jXMVCv3&M|QP>$EY? zg>AYTKKeSF!^DrYse((k$kdp?gf8+J2j2aO4Y#65ZRaLVvj<;J!EPjq$3RNg`(aI z%d47>pz~b)i!#D8Vl?jow;Pzo#c8A4ri_&^@Un<78J7*Q=ZoMOD*_GZ~%&MqO!BoOqMYfNGxU zqwZP(7DMPjLF*SlJ&m8Kno%{43V|m)cuDzkm5f>$U$BkduhcbrdzbM>qUOV~@^Qz3 zHgiR{L{Tv7Z>m`zs~4ow2=~+|1HIUF%-gx7T*K+`cGOCE)Pq!`T+1C5*f+{(vdU>v z4Ad5;wbXaiP&3OH%P4r%vx+IWfeWUg+VYO&b9yeJ4#=NXA(oFsy!aa=<>%&Ij>f-S z!!G2Or4cw$eZpQpQmbcYmsE95yBSm(Ce%j05ij^drMf*^{3wfIcNJnR>zy~5E7+@6 zTj^>=gmEpI-j^pt7m1=dR>uyPQkA*cP?}`5d|ZssDc{+e%Q&wNV!|`T1>Q`VUOn*e zf`1p{Y9K?73Th+iwEw`!tT|}n|5@LL>(jM6$Me@(E)e@P`qjWs6hT$pCiePSrG`Ng zf2F1I;#ffq%(Kgo#hQo(*UA{brs&+RYErz*jEsa(i%H0EBVogF!sBJ0O~SR4`;b-d znJA*r6qXjwk4x2ti=++4g+Mo^=DIqCkEZlcKN2+hMZXJyx|kcNkrDjvJ@b^f_Um)A zR?(te7VIvD7A3s>l^P~f(kpFXhID@R5Pfd;>Q;vLe)L(FCWh#B9aHVoN+P_J-04{c zJBe~1Yl)Oem~Ztc742RmFaEw=*|Tqf5H^JlDI@Cl$p1$QYMuo7CJ<;glgM@C$hsr%7vi0!lNT)}-zHtIW`-n?iJ zGBE#zN8{}?A<1>V9d@7~9^#V^ajxU28}MM1;P8ruVyN+10n3=3_jH=YTII zh6-fJSD^m3r6*E5fkRfg7V00{aM?Q%9}682R!x3^b9Fh2ihzs2W&HkryFIxlAO+?v zQ47lNIwGTFwY5AA`yRjW6|xb{sFN*WsEQv(;9AF|GdfC1tiS=tHQ||_UQ-4k$N5`O z)h&R7#+%E?dW>|tG1uE*PiU@;ry3FTAhNBILshWs&;Jb^1p(_r7&Rr%wYafdlZ^1m zA>zltd=!pU#6)I4Mbg8te$lFZ9?1#5@7FDSgfYVQl z_KYAK!g!=PaB6-&ZFqpK@gq0C>cZtCCs<})M6e?$2;sFyjq{65K%zerE`;APz~n9z zyAxm=wi&V@?N<hp-7RVD=fmojFcPJb2t4-P1?(`{ix;QgwCp?I;jMrJYfLtUnLUTUeZ zvsrPG8rDwW^P))x)Ye@pVjCH)Rv`k0GR%P*U@tpE-Y$5uEVJ0IfN*@*mc`Z3RixAt zqZs*WHhg6Ippw{BXf=h13B`+sf_dUmY=-hT9oFLMm8;7`!A$r_!Uw0l>E2UhL=x5@ zOYW-t;DimZ_yiQb`*1|2B!`ZsE-L<=t?}|k+DdOpw!}PCX}r6% zNiKOP5iQn=QY@T|ww@gY7oIH%bL5E|Py8+ZvRYp|@JA+qLS9JW5fikWEC0dFuw*W&IHXep@Ln z2CkC$b`eJk&ShL&6R}`KiF@;#&l1poDRqWjXn{!Qsfk@WEZ47^owwG{su6%_sX(;d!&uo2V?a7!(SJPhk*}IlPjOisBkeXz zU(I!nbKwy2@`J$czAzEvC^aDQw7obk2kLHH{y`)crLC+Wa7Bst(HBQkSR!pH;M~|b zt>y5bbwx)XZAB;1j^q5Wz*K?ZK!>+s=@%ips#H`V?}W8C=5tB)#mnbc?Yzx?^@i(q zC}u@#0Z#|zjr~4>zFpqEYa=>OZ%h6R0~hea?av}-a`3Hz<9HEQD}RX!xFWB#R-m4y zW+|9ZRo|XKQIqrY*lObf&530Dv+KiQdjTSd) zIP8Td(G%6>je50D3_XD(h3XrGy&(9pISFGe`hSI*MFzNhv+0YQ6oPKuSRo2%Lj-fU z7H@rFc7T^bKiAnrjbSIEn?LfVUTD5T&~ zx*m=#+JE54oF!3>^&*!on5w>xo68+4o;fY;ifv`zyB%qVUeTRH!XlVj{b4rEEqCfEV9 z(~c=hu`<}bQBi4-lY7bQg z-?v39=trr!Ddj^XfwR9D&8hAzAei$cm|8kAdn&6&EE`g8j?CgS`gsH@ln>Er$Q)`4wLr;4sU`(Nl@Vre zk)?Ji%(_D*I{9-21MGm6kTwzjh~HVkoEL-J2OSbP$M}rNulyhIYn)l}xH_OXhbNAy z^?eN4NxKob#p6kVI+{~b-$tg!weqY5sZ7Nf4wfx@vuT5k5#@G*P1zHmkivhfPK(7( z%8{li;OtsceIJT-!v$_RaMYCe4y(P6R70|`u`6$Qs*tha^w8stNKwa|$?Tt1l{WPF z(c?ia*N3E{AfA3QtMk;?;%!bgN)=f}{D^qKo!U}ULJ1wlzythFL#C>Xp&ONkeUS7b zA|7sL%6ysH<+$G7sN9wK3RkDZO?5~|m#cU0s)vRo_98#1Z?JeyrGt_h0N5EM%ZR2d z&Jzm#BjrI`PeM}{EuxUW((3I}q{6f%q5mlwcZnOWcVyRC+nor9U4MpXR=>4O7uLDZ z#tEvOgJAt@H6svNms$b!57~?ZqKnw>7Hvqm*K~3y;bWIjqq@dO6hGoxGjkQcTUFL= zwG~;SU96P_ou_01b=NL=r1_5#2;xwYYu?a7ww0(#TRgnS-kxIq;H`b>c$*Cn2dI|^ zEVyMi;yK2JLWa7f{dx%F>1ExC?2f88fH+-(jCE*!2p-5AKW0%T3GzkwaUW_l%Z-%gtU!i= z)vj}L(wBjz=a?q$rM!nMCnwz+c5K^5;^r@YC9*?Th+v5|u{4(;u&s0yL$wka zGV+#PMBl)Fbe8(k&^ba2Bjk-O|wq5<4~p8h*4@w&S0w%yG)n z9FKya_0uL0%&_}r!xty~wT8ZaM@_3nHlS50GZd$}RjoiNW@BxHzJuEyx=Eq7u!C~= z@R_w;H=2pq{)17^gMbYDDLGk_sHUX+Z!Jwhpt9iHBbY(Kb&Gd)wQn>}{M0^2)>KM7 zWt9ZnH;y3Ar(fxjZO`VwanK~CIl!o;O_g811I%y^4bdUdK|V+JU>fZ621+{S#~D~> zvt-8H=tkvEn^)5WiqW2X4TMbnEjUQsyxC76qF?ndTbJs^-$jkzzOZfmIz;>G&p^^$ z0QANoUf@FU+>OUJ>53%;pQSzQlBvPo=_)iddRe-{WoZYRM7|v#I6t4bekE;ry+3^Q z{}Y0bCU#svkIDaeHAB1GI#+9pNc5&B4$_%eu>Hh*682n6@muE#q2(^DU8rV2*w;<y(~O=0qdg8RAt`POwbPR#|o66qqGRkuyHDz(1Cu2c$F%sI4IHqA2H`~9q4 zI#>MMG8XqI`*eEQ8ph%$UH4h6N)GqdUGir=U+rU-vAQzm`t%wkkbb%;BF`$NdzYc8{)pvpb8lW)QwjOWjk;kUr!`$b zcI^=IiEUjhCy#f!Ev6b+5OU<~;wPX8#eB`SE+v@H*yC^KzwZq(Nr9;8h&vACy*T$% z-axdr@4K|O$a4Dvvp%xdFsBrFZFH8Rg3`7vrRR6j18XH;48;}mmatYlr1^shcP=Of zZf3SL(sxiiwJ}0Mizcy;{G^BO3Y==6iJ_^%2k(eb6+5xx+nycp;8M~u|2I?RTF^o5 ztI`H+lx#%aL0`+bngs!(I(mY~2T8Rbvd(Tw>dX^Cl{4;mg^x^U5ZOT;4bZ;_I$X>d z>5qyA*Ch=Vaty5}cS5I*izpOXTa}t;d@W^Vn^Tg#-VjSIb9zn2yjjVJx$H2XgFaaP z>Uz;hAezY&&>3j}a(ezm9f>;7Bp%Yg;4v(ac44Fsbm6@`%U_=ksu88_^he*!e?D5X zf_xO3B_gaXq-L4_06G&rmewj6x`5k@?J&x^!P+07ZPeOTyPx3>AD?Vl8wu3N>qwbQ zaA#~3g?F~Am(=cE5jR`p&z@QyAHVQNC^tW7DN z@zsfRN5zZ2*dEA+*=28#oU9%AS(;8W%jJ|{_d_z#xkuTP^%PAk z1m}008fG6C0a3G{1m-UeIiK#`09rp3ZQ>=r7(Jmx5`_?BVd4i^5{AGCz^MC%E0mn$ znx@;gY>=Fl0>8%$1tiAtjn0cSLaTv6pMQkZWqhO)bUU4JS2_dev^^iLAr~$Q%JXen zI?GEEy3+7)>PNgNWh9(nhC^1#F6Hh9FhnM=Cg`*X{K^+n!Tj~rs;~PoYUD94fSO9b zUeH+3wcr3IjTRmWI^psg>xIYHlb9LtQx3&`Z(TaVT%hGIKzPrmpOC8i7ZlE_284DB zcrpZtW)XO!i)=m#5%{v;%Cbi#lx+xvlyoF zTsd#)hxrQi9fqhO#vB$MY!4X}qThUKKaDxkc!GJ^8G}w2403kZd2;ZDLWE8(+|R;#dK^&rTh^AkVmMG##8o4zeiuEiq- zfBVz3ZJ)U_I;N4GKZ?2}czxLkiV_0{%OcMz{;q(O=C)9c?~lm(Pvl6m?5BrzYYC>v8cU0yFTukjGma{dB*I?|(ro3xk!}{VDojXDK zRO3RMj2?MQ+u?S+-VlG`ihIJk#yJP3JuuYmhh)p$*L!=MWW3hj0#2V{UJmd}Z*QGC zkc2TV-SZmUWRhJpMGeBdcmd5`_cNR_dV8M05Vx7stM*SzyV#Frk7jPBOgh?s;CWZ0 z*Up$$qt_*o<}v2h&F)wvmn1wf=BmF)K73d=;YmlsT+_Q)k;iaHVE9b!2nVMRE|s1j z9WOi->NKtTGn-C<3dWlgJ*h4HXk)&6W@uEuk^D?xwQ3k|bsuu|Ef|@zxnhi-TG*%y z$Uv!lrVzTWqkM*@g1B(7a(pd&C41$j+^okqlEYV!TbaAHh`)Q~U65zV8AVU~VH4NF zW0a56z;M6kTl6B}Y2V%{*%NUZgcD8|yC?3{y~%;poLAaDtZ{f!^o$!WB{&z96ikPr228I%7Fip5(&1_%@Zsv1>tv*G+P=DXU48P1__W|#=Rr4#C$kCd}Q zKTyVe5#pqKHr|y;>FKmTFmmC18C#0ASPzNZ2kh!RBbO2GBpkN!#ruXnT=B&er))uN zY%|{wiyjp>C*5047xQhEugEVxa7RC3!)5!U#KM%SI(5G=zK$MEjE(@RQafFw%tER`DSs z9IPHZZp{xYUq78fDm{JQ6P0TQ%5?9g<<4luB#&KS2^Mcl_GncrJ5S6$z`xY5u(d`A zt8eQG1b~%pL0jQW35Xa(66T>aAr1Sa&O@Vg7`A68an1?)RjD>p`o1ncaq(E1eYdx8e9V_mHIkj7}y^$~!Rvco-5YTdGB zXyP<*R8ukug17Ej;QF+xZuybBaauAO9_DGqwmR7!{&Yh9$ec9r<9abjev6pLPim3#GNbsz zJ~ot`L}ZqqBECRAOvw%_eoorM_#I#C5bDN`)!L4B*O2pLY>sxr7)CAjNNQf#%K+!p|bsTa7M|?z4IZQ1yC>5uV^+o=o(=pvlaTwp${i=}d0zv1IAy zpzP^%*yGXE1Ql4iJdXcx`mX*_TE#HnPL(&(3vBK1m)Ftr0q|@>>w90bU3a+E!ky2J znuNJES#Qxw{k(Caf#SJ*U^<#(hhs?{jDQ_$R&bcRG(-WSPcB%eTse6@hE&|(#?BdGfK zO`z|Rn76)88n94TYnA~EPTwiWt|!9^-A-HrY$XIM(pGy;xnQE>eoXdst~P8v=0wB!2-)PMxWyd4*xB#(46L9WV%aF>9y%tK zD^AoS&Dg+B$k#iYkD;5>E2StZ_ujPeSNc7t=J296=)ewenvfYh1H1lOv1IG$7Z}py zAQ0xK8X7dCtf`zjtPGLS4?7L-JYZVLNO%TEGHkJ9lwxa<>BN$9mIkh0 z?~$zxZ^O-+r5evN}%||Fg|_SK=^{e zI2tTMsV~Y%q^u$ZXDIYk@u1ciN{iPL>dpD3f>c)%{nGWgr^W2p>=y1;l}6H^j&llU z6E+oe>9%3i;M+fSd9Crw?oY(b*Ob?ubM0RO5?`$w&}hvX&C6K-VEzsMNrNOBgCqFx zAz0-9ei=?F|4&#>$|?ys9%zRNQ2VcB%J=mDgzJp40yN%(dEN`QfZpPDT;Bmjpf_w% z`+q}$-ss3F5oF*b|90sG4Uy)h6z{wB%tqkCdWA^)?5l>VWfq zI{igc{gVyV1jPQEm;aT`(F34?jC24n|HgWcZ+h?djj{TV_O55OA;9s!e^cK6Vt}lT z0p|Z`@5OQ6n&GD8h5{f#L}q|mi1&wjUzorBrEt-L6N1QW044vTtlm160%3ds{Ht*T zdi&e*zuilL*xUeZ|6k9!wExqypeLXK;(Z3*W4YcKP$}*7;P4=JAVB_K3n8bscj12+ zB%oD)z|FrUm8-=Q}mMFmcb_$Q2gE~zMn4T zFB$Yk?ninTMJWX^r#Qp{AVAthfX08LztQI2nSWtxe;Yug_%Xdf{*K>%Kn!JoU+?n* zdRr65e-rV(lJ9c4%;29uarFSezd}HNm*_7&nFSmlw9^7u{I{C_EB{7^2R*g{?*8*y z|3wjQ4gsK2CRo5xL0o-+lYb4~->WwU9LQh*pz^Ne+p~b)D3kBVzLeStfZV&vZ*Mhy ziU1op9xzG6=5H9AuR?;$nweKo5DC2&ggmNUYy?pd*j|1WeHDhTZlha9W)ob-dh^N` z6h;r>f#-)F>}ki$OYli0K`v7IWb7G6o*l-Eq0)y;`Se01VakMipQ)95-?iVr&j)fo z*bbHO{R$QFo; zsLHwv!RBD)sF(_jGy;nXAPPpO9q}nY+eq1bacL+U(`c?x3W)ye>?u{n3vymK#@*v+0kcRs3Zsz@ zEr>~+OqX+ua+3-C1DU}EJ2dJ^8g5Kdo>Z$ng}*eK`aemp+G{|@3}bWeX$vR@bONEZ zZ~8g1(!^%4_-9JOeIZVrlB*=G&^Iw;XjHRr*xN-GjW`d;i>wKQJ4JkXlVHGgFG7We zfM=xSU;2gckgPlwx@|o4KTnKySm%N>BOJirY~}ig%vf#3=6^aw)5@6O4w9ij)XtWB zdTfS2kx20uhPd1D^Y0to*!g@@p$U`mKIIV<*-Tt&5SBVs%$6t_^C7YZ7$as#iWFTlJ zZmxy{#Ea6(5-ICfJ`X-&$qr8FJUU$m1Ztd)G>q((41PaFOvA$4zdr1u-|gHGO4<2p zHP>XI!OMuF$BvRjpZLNe%nRHkQ|`>{wN%>?E>}iCb&N@m{Xhw(Hy{|2z?fK7s6ZkH zdl)%@No2gF%-rNRt^UpAa&VYk?GOAZO}X)d4-dJZj(aN0H}-ZkwV!d;+c)byli!&E2+pRk5Crp|0v$xmy!bg1we1*n?1l(Vit^5gMA0WuPT;77TLLJ( zz!aOYVY6@0`O2cI424YSn*0{Vz@{=fa{L`c5JVZ$s_4_J5L6iKIg_Cy9eq$}Okj-j z2nKxp1T-6zvGv?OwnhQ@M1?_0xDF6TI~v=Q=s_*du(z9%mDvkS-4y6%tYh9biED?y z-Dp;niz|zBE73BP)*50v48xLk&DAT}y-Z9ZD{kQ|x5uDE3<|;)Tl~pwkk+Bms!bTH z=;zrs)!Sg&mL6sKoyr)O}{A|ZhhBb$+w5$u~`)AE%P zQleeK0AIK`WKMNB`gJOi3z|=SJLunn3qYD@BSh?hvi{)?RC(U|k*~q(OEbMk+U|it z&>_9o%?Bhjj62&lT0oSjr5PSKd^^+STE8exH4DSmN67($0X)Pw&j#0M7xfp5`QM>l zkLVmRch?l#K1;wauN#X`U7ZrLz63ZT>*QrkEsHkGKi2Qiaem4a+Bp#*MH&Rp+>t}2 z6jXxvaTvaHh8(y_$_%;2;lfXU_Rz!{Flq(}MAU$JLu^8D875ILf}LbCUE4FE(jQu! zL2>CzwkSiU?GYa{ucV63vqehdiQ*K@Tm6K;)b=IH>EkjtOr6?h<%^kmG<$)dupH|L z^=pQvl=Dje78Q1#a*W{@muZ|c{@xm4hT4ImE@7uSm$OX#ryobzH6f?pOx^owGuVf# zbniRxz3g9AIpz^R63X^Vi^e|zNn zUVrh%0O_p(y5Fn({e`P5z3*CYov@`)a=h)G&6|M!e_Q?ACi$Q3^6#FOd;ov~MeYE^ z{uL?ui$({%?gQHX$-cGE{x{6~_&_dj#1uNNx5HchDZn54?+|~t{x>o<$mR}!_20AN zUrA8+BVhP_OHcTFOnf7~g@L~UrrtGsKZ(4PYPrFQ8V|q@{*Ci?NVEo;F*<{cc3p3h zT#E4}vaDXuZ`fU*ei%pz8B?;rpV()wA3S^do&5UpVV95$`pzU~S~f+FP~a zy$p8ko{beog&g-F;>jc{gquyyVH@$C6eT#$u}&9+syKXdyl*z|3rozgiw6bdeZ3R% z90a~V^a(8_7&mHfVr)ZF!Uy+QZ+!tKgc!8j`z)7O6F4S$&8*}CYejm;YM|BOeqi$L zy5@;V9mo<&taA{+y^73_K4j|LY(Tq5(HeV9t0rdXeeIlzNiMHz$CI=TGudnN(s6)4 z_}T9D14!rH*(K6#n?^i`oQK(b$>Kab^~I{Pb?mXv&1S!IhtPZNBBDQv)?IjS^u%XMe0BHz-LDZ!{@FaDZ*@MQ>bY zb1+m*fSn4uz=BiTmlVz=LBEnvIk(IujPT^k+b#t7$U2=Y$%`T(*G`abbs2Q*t(q{kMDtG^95DzTcAzf?0IV&NBg1?XPaN?&z ze96KYz0Z>@x5q3#wQ9vQoUfXVeo*!gVlHq&Ev>#7jLu%HzS~I*#WRSSApHf*xYqP8 zl4GJD7Q+j?Kw5A?WxmAnC3%@eK}mKU#CW(kB8ehXi#@W@Aw>HV&iZ}}sq6?Y(To-Ofu#h;uAiWf*!ZopIb zz-zDi38j8ZRG>*5o^!8)>y@Au4CjB7ZNO1*+hO%a1RiSVE9E|7x1O@;6&nUN&SQ0U zCnYC~wmnGsh1fuJFW>(->7;MR;)1~rc?SvsC3>+t^Qrv&5b$-(!X>P;h_2Ie(X}*iRl~-q7 z=FUd`v)&Sy_^VD5QiUUXj-Wvl8$|;sLroXi8OMTMGT59OKGqCBV>#EO)9NvYVhrAD z@&)Setu+u5x*g`@hYx#il~4a$tp1)}ZRSmc83a!P=JWrl)y?;&RtM=n zwLm{e!NOtQALjj@@s|lorw3zwrzQNoe!t7Z3xcD7K5>Ep|6L1D&B2&cYy`p4fsN{G z3aG+Ze~N`PbEh$o#n3*&f{A1B*(`|RD#zYuO16Hpj}QJ-wg+ z@iTx=K!&$Z2!_VeFNODB^Gr<<&ywby2w zRIig<@D-P%uY&Ncm!6}v)mWf^^)imN^!th1`Z;aC!`ojtEI}|5)4Y zw0TIbRbA2>b=$-9Z+**EFv@htk+@nF!M@@Go}!c^Q^vN$kj5gHlGtT;4usafY@r3u z042F;A2}U*eoK1B-N{V?`kC<=)Tnt@FU5Hg$zx_tcl76wp%oCjLivAR>L5{*tz(FpNcjmZC(&~rPAgIcVDF)=u; z9dVAxr-*$c=4Fe@wAeN3c3|}hbV&GwME)=zhI0Au&IjbyPAXa}i9RMTj{_4{oiWmU zf*;-SE$ke9L(Y-Aubc32W7L&f3b2^W<4g+-JxTJ7v?nSJy7TS|qBEnLCWloZZn*nH zbtM&RPFtqJ#X>aSnH^!GQaeFJ_p%^N#S8M&a>@X`6AzF7*Vk8oMfp6B!<}@eASK=1 zU4nFqfP{jCNQZ==rxKEq@`99IUZ1f zIdS6!C8n${0bx2J@}ZY~`gk@%X*JpudyE9TuxmTQ7rq@|`x)``)U9K?iyx_YSwPa? zy;CHe9a`*E(_5^%@3F6@lCBFGCS6MCVcmV#9hIinDu6AQ_xKxrQ7EEkqjkpIk0T*I zz8CN2PxeiAojuWahIvy%_E6~5(pfL{xGBn@dlHRQubjl_Ubjz2#ra~JDV&%--(kVu z$^Qv2UQja``ZQ@i$J2e*q8|n6bpIfKz|7~b+k@GJ9@ddha??E) zq=(=_y~=R%x??IZsXs1>avfaA^AwB<;ciH!N`AezFIy$o{0zA&IgbU8&vtoXNe8WO-pU9{ zVp=HUGYKpG1dXf7rPDQGZf-7}?#Fb%Q6xrloYtu71^K(z1Rv@|7@HoOYv>t^(lc{6 z>9FojwwJs-sPVYxvyvsQ>zlcKTr!3Hy14PiX^P&L22}++v}UhFcIA>rT%&0k7Q&fT zmF}9jlq>o;rj?gm82w1m$n0epL8?50@H>0gHZCORF(J1osr)uk+Lv{wYstq)$^-;N z$5tMvuZoJwiJrAvv@tcFigxVXU`331df)!$9g!$i5jS~8%2-_KPVZc86{Gf0TEh#r zs14cxjgPH00?}#gBQ%A6#qbH84)!ib5zn?amOPG$Ew_R!h>5b!s69TtbAB)&Oc$Xs zchPhuBQ0`4<>bU^+mmPSN{lLWhD}s7==_7+@ zD!~_4T&)v2nTerHvs>V{H|y1G+V$n`ge8TtIiXn&*G^dP?5!T*dY;s-JtxnW3DWz!l^G)FDpUMj)1KAT4= zgY-VaVJ^jS=GN;EMAU8*D?UXiL>>!mo9j5It`!mR@*#DHnIJ1eM#tufbJ~+cVV`*g z$6jsUv~uyXF&2xxrv~X}HBhBTkKB*__)u6w@bX}|?d;|a0d=#v)eBj!lgsWG>s+?m zeJck96ho5v9O#0Xv}dJXMvrZ(=bX&ZDj}9uY7o;izdPYm7hdA}^=;qWZA6BLe)>7; zV_{}8cf9*r&NusDdk#tm=8nHekR-PoMg)xoY%8tnq<2JSbc-msky%^ZzxDlX zSBvU_uVNvKkKZ0#X=V~|^a5KYGVyj1CF=7{V`lQLWgVt!&*%NfHJ214_UCiwr-a7G z9-7D-$FNqh753e|fj0h;-^LF^89zmxS^l*KvwhR45rnPouTiI@wTw6MzG+zhC&jgI0wd{5-MBAwUcp!;&hu&lZ2X|gM^m$gsWZlSV zu>uj5-lo7Ry-Q>wTK!%#u>wA#`&ernO!pUDDk~-=(tG-^-ns5)WzPvcQNK|x^M#>x zDNj;f8nKUK!`sFEwkWhcboGaN-hSk#PVY5-?N_R6Zt@LxHN9@K3C2xOuQf1g`em^`e#eItbbK%-=h0G?p$6|h%WKy}&Z4eg3AR~m zxl%XrCGk7|FaBNqohdSreXiLRo3>I)SNn{Ko~|B+^{$nb&7Zw{o>(5Us>0>=ZV8`C zlnRU=i@4iGu(sKlS4*g0t+gHl;el}6XmdzmNzq&iser z;qrgNgX4Mc=Xnlvk{U*Q{)>iwXU}B$%=2Yae<-%3YLTnhaiD+11N~%B_#QhU_25by z`vZecm&P3~n%5$`T*&ZX*6=x;+3rtxcvyVl?MK>1T2y!#^Fut;F=wCTe$=WR8_LBD zuevtz=tEHMcsFnmpQ63k_~>^fm164O?nT4QQ=Zo=b}}BX+Yxf6a}gii)_0$!nVsFX z_K=;^YV7FRGt;h?Zs6CUEq4};VWU7iNFlw_Z1p-etbusm%|e!F-O|CUO+g?c{LW?8 zF?CqQ<p6KrW#_)xT-ej@J z+4GqtTlHjy5wclAFNl~-e(z4v#2z6!tbLgx~3~*Q$0UV zc>W#YZfnSs5G5@Ia!+Np<#_Gul-w^;!x%YwYmymU=pdf{zC~?T&r!PC)yJ-RMCs4 zKZrVj*h9hUz?P6N9!eJQ#OrWvge(RXYk-9p&Q(|eitg{i2_T8;7DHf39LQ40H6F66 zy7F8YF`6S-7oz1tr7}Rt;aLJ0DxU_sQ{IJ#i$hFgUGoDNGrC#eao0KYFR;mW7X1r6 zv`hJK+s;4wJIV+u-c+*!vrr12S212ZO%Iw*MS)pp2OAh=1U9A`t+sw>ZPV=2>^%ti zjz@<`_Wn^mZ5Ab2sqg9R{Pdbc3&S7zQPL%^e$0#H{xU44jEa#z?o%ILmCrvKa73Fs~ zo~zVn_qWEyvebWQ?03?BEpi#(Cfj2mXHlSpZc(2jZjzkeXNc@tYY1PV*x2@`HzKyiSKp4)K6ua|tPZhw zG$KmhJknHEu_#QY!_!y5J&PZhU)!Qv=8;NEM?3n(Ipj>NL7W+V<3qmwY>OYWKNx*8 zC+DAL+C6^hwkKT^cji$%gIB|x(CPjvC+0_5dCy-f8vA=G-saV#T6NY*@4Gai=fu(F zENXfqIY1;somP6K%`VNgl(too5yBIRBhC0WI<6RZ+V77LMKIATv`B!|}4$4B_m4;V!%n1fQ(J$%=Wty+|(>vX!C0@|_d||O5yqtQa`IFZ5 zJJi>1W`6|BG$*I6vL8{zyc>O0{1XCHBkCJT?{j3>oSQOy3{T%eh-WU zs6~vl`jhW@fn^%q?D-C(m|3r1lXh<%>cgK}aKuye`+_K(PuI4H$|s_gE`R%fjxv8d zur9O$7EXn9zGUPR*}fhaew}g~3_Q#SAoM{103(4)Ucx95Hogp!)QV4-JZEIu>m@jG zM*17y`ww_rYBOi*mwLd&IF{1FCEVqwIl&W^RP*WM2i)EEUc10h=|%W6@n2jKrQ`ZD z7pTrviI0>%?^l#tm?+(wtvLPy+oC03EWOZDZ#p$}6sO+8|9exYJTVTlR1IN%6Fyy2 zdF@@1a%TLRI>be5PvJhDt4&^tO!(4b{Q*xu{XF|{{7Mk@U~$dX4y$2m0NWgW$zA#C zo&?ur_ZE@KR2?_Xy=?r(XA8+iV}s0Fb%7_p1T1GN!R##3!^qlNU4|Dz*%yCOO_lju z#eLPxA9a63;kOhe5`6WiF7x*$5jK2!CpjN;Zz>$W=J--XS+KTTNV)%b(erE5=^Y(( z=L3e1=UnEM8sy?_5qWt~tl9WMrzC`im;Gm6A*mzJ()WwTWOw@`MzxXx%F04*0?wz| zHWw1wTz|6Yak+|qN9pEV!$Q5%l4C`mpgcRvxDi(;v&!Y_KPmUE#bydb?(G%_V-?u6|fl8w9K)Nb-JAQeoP>GE3mKy>Q#fvI@g zTB(#icFt7x-sz+iUFTz-j6dizZ*WkOMLhjb=~zq>bb4f<Xn5l`1h2^)K$ZGebbDhA?i=Q<+ax+ z&L%a3J&3d$Vu+FLI+dSzS#F}TYb(Pei=>zBD0-k$9oiwtTOft8vDeukbp_-C^6JOwNx_RexZ|!McYY-}&ih zA=zyc7x_Nw@*cA%a%TO0%wazSj)hf>|DT{?Y7XB3YYI$%kZvThuBw}Ur zdnT8xHp9op5$|fBGVm-lG(EPV!4JN#IO&nRVs7Yv+sUA&p_44|(S&x`kH~CREA!D? zxx*U;!;BkpY?s#W7$${3@M$=>hnKx$7$z=%JM1jiblc;=_g^yh@rFKAZCklZXa0y7 znD7~KcaD1o+2TJhtv$bbV=IhS^_9rzT4T~@RTa_wbM6X=SpFH`k2UB0e}t80#Zg-Z zpP|K(638k`DRWKE>=K>Nktv$?NOlNvJy|nEMG`a>&iqbJZ!0QA$M>RY)Oq)Z_g%jc zj1$yzt7nki{qdkb13t;^Jn(FCYf?m^IM!K8sN_xz-I^i7S3@~UZcba=7=D~z^fTYP z{qQFaGKkD=3V4(AHHi8xOvtuB*tMj_vqW_&+Bv&HZK30{*XXc_0QX%)mdREO`*r0s z?-#5kIsWyo{X%8@h3ZV-uZKKuwI|)uvEZ=aof2^Bs($6g6+J{6>`?5}`$ruJMM;2&VZQWoN(6eEh0k0PA_d7Qod{xT7^DxH&)|AWd+yy? z$)Sj$A4$@-$YDG7kfY??L0sguIKB0heH`o9YNwYRWL4){3tyd1ykyNr2wjAc5Sg>x zS*NGg;S_jxH9g|>Wg#l6*ruhhU9%MSXAz!B)a6u4d5J}v@qxD_qm8ur%sH7Q?ahCZ zA$?)|X1Sva@P(>>d|_-DCJU+Pf=QxvH%tRP-9y1{^uzEGLKofae0{Ud*qBmjQ!6s5 z;YRRK9J3oDJDZ<@B^TM)cuZBh>}e(JI>*b|vsWT;ak<7px{LRdQ)H`Gw)C$JU#t72 zpIYavhM!a@W$>XZtEukWl5g*G@tmfRUwF+nkKz*3nOaWH3v`XvM_JJ_5)lzgk-bH2 z9uf-^n(FPRCrR0`J}*;F7)IsN8?gK-cFKbs)NHVY$uSXbWS3dMr89gcJk>veyQA!C(cPYA)U8%3xD$@f%sWP4pHnfMaZLXzCC zb8sRcE%RZ`N5@z5=QT}&^^FtpY!}}pGL6I+p1GI7!8diz3`f2vyD#oWu$bSraoV}-uZ>6sld-~suq?SJ5=pdd|@|3C#+`sf@O~UU5 zqBC<7p_EO3S@DV#+@CFcT2J3M0FhNU({R7yrd>_Nk4;#OD<7ftYP)!{)T-d76<4TT z<2ck^z~6b#yO3y8`)TD^h1TT}sQ`V^21)ehrjsaR_mn?rvIA{ZT602>Uec7JmQ{Un z;frR)i16!FZ{Z}f8}))yaN~B(_gH!;LjKV&&t-Y~v@g zjd&&W4n6O(TIo9UhgEX$`n$SBr)^I<-wT1?rZ>%sUCMOFWpl#2v__mZyfTJYU=nzH zdUIz>>)>a=PwR|Uf?jPtVx9!gJA)cC2zlMbbsAsslZAVX-!SKdiL!c|?fSgU=2u{q6(71f>=o5(H4>hPW{g zd0DGD(i@c?3aM)XZ)^as5BUE-6Hvp#sx$A0p--PeD4z_}gOef(&sdobsL&}|>A(wh za8#d7KCBHSm=IQd=I2msYUI+n`a0d+;$*^C|0Ka>vbFKko2rvch~(KiZ5|vx_L1~Y zx=o*Vda~y`D=H?j#?*qsy5?9nW~0MPEv-Y{?cTIri4Y)j9!15difG`2W#`n&gY7Kj$sYPmPQE^&^%mi0RURpLyxH_r`_;P6fL| z(PQ7$;&B6Lud)^Pj};7DUMS_Ws2mFOI5w7?+g*3-$E|)M_sJrsy=NcRltruszS>q* zk80Ql|CpdDbI^Q~uiisLU=`K;yik(v&bP2Sk`Kkb8F8Gp!%1#-(bL_^kw1i2#kvt9 zgX{INA;Hv)#V%jVd)Lnv$n92$zTdZ^S4@9Y;y)E_*?TG2ZQ=Tm9!}f4u1 zSE^!v(rD{g%$Hk=9Gu~0TE0E^>EwNUt83`9Qd^1QuHqzk$t`wnAtIGnI!1A4MQJJQ{v81(A~GMZ?>x6>Wm>#E z_&+2wT(%khpmnPqpZi`lq1L{7cbPPiA)T06;ozvCNq~k9Manhih+8rTA4zzYj{C}Y z&Ls)As#y;U_TD3nWLz)|pC{V#Yzh1{QByenPHb!MUJ;JyX3BKy`7i24wE?yfO`NX~ z&A)a<@QL7B$vAKBE3;a}rIMOsachv!4CJDWO*CKED&Y zv30B&>!p1&Uj8H9PrSz9^?<+h-siJ$+j&Pfrey}=tV`S+ct5MBaHXg9BmVmfgbg9g zR2vE>UT8)Vj~!c*&5U=u`6`OyJcqoxd1`jiDjk_ihAx4&hMEzZMByaT3)j>GT$F$p z^8fFcI0k&Ju4M>D_a`Q5BWz4ZHK`SeI$#%@Id4XUByh>Zk&v*l=u@52GUY1pBi?&_ zyUyQ9kd{3mK(RkMb+S)6xijMSC&lE+l-D-ebPU#9cbv}-H3olcaPECxy2TPd2@4E6 zp6u`@pRV;;csOHIn<7yRty=^mS*da9Trk^(?dN9_)kYt3ewrw+9e<8c(eB+0|ME~} zIPpiwQ_ZFqiXUaPecs%?cKuay_qbN$h-tsaH!+97_jfL&6$hHQX>>Q@Uw-yA+4)JH z^;ggJX7O(&Rrh@HY`|&XXQTC0wtFOn}K z6gJf2b!uH+B$&8(BrRh3($Gk(T+988z2$IeeLz20cd6{v7J=JiR*AMblbv5Xmm+NU zXQR(XOPlaf-Fy+r-BwdY-N<)-EvTxgEvAUd!rI>AV2yso>1E{iGlom_9f*hbp5kRZ zqcOx8Cx}?MCRtHLY(|4qZn4#ssh`E!a7m2L$wa$07~iJ~enrupe@mL9#tnKJ5ire- z4u4#tnXl|iZn%hTt*5m<4$@1m2H`I-p;KMK-eTGO0$<3FXR;yjo%1nk@U|74z!!{C z^p8~9x9&~bl@w)Xk?0|CF88KfoYrPGTos5ft?%&qDe*q}_OioFL zQihWP3eK<2rpQH}Biy8=PuVZ}QE(8qKe=}DE(T|1xYsg?wvFq_pM|w$VqIHILd;wy1ggUJSd|dF!;IGQc zirE{CidB_QV~wxQCzC(9=+OC8fy#lXI-;1~p(#)DhPJiXDDJLUf}PVHf-P4cr&rQx z5jHJ)=c4imWyL8(6{if>NI;fHL@0K|(B1HZgrMWcyG=f}qqoopCbP1qu~2Qm$${PCI3I<@RZ$JfXCooI2~~c~OYdg5#+g|LFs$lJIu%q;~jr zs>PTFL4Y#q;dfY4{Vg8gV)_3Kbal34u!6tmQZuk&^obPkCYV*U*WoO}>=5P* z8g$nqLIlDJ^A_243TiyqIBWw(=l< z{yp%+>}q`A0@?SC$Zu}1@KSyRUhR_om>#5@9rPo8D{F5y=lpdX5mNqJM0D{};?}0| zz3lk2SC5^aiijYnNjnoStX_Q3!Isvlk02R%vunofafLm!!ff81L_DiLe=tDOL2_)s zD~DCVg=jRx9-kw^obDEV-7ET@dAF8@%T_o-Q1>nM)QL>j^qy#5_ZlTJ+~-ehI49Z8 zFC-Dg^y&`He0a!DzwwCN+KI91+#GdsH^EnWkMNkaMrRAhCBjjkdb@@s2FYIkSTlr& zhty3mQfc06i)Nn)^7veDsa7qE5)VxfT48_DipkZLQXt-IAn_fQU^Go37Gvn$1FT?-Dx%2`RT1zq3Un}Du}6&=3|?!rB79HfdZ|H)S9hA`L}f)_1Tso z(R+;PdC#&rd#k0KyQ5jkSx-Bvb{gJBbdy{9xc1M8*S0Px>xZVa4z7~!@|boC%=;-c zKf8k?aM_~2GS-u>V^Y(WiGOjRGWzz3K|h7oixIk>w1%BjA#c5&cOReRrTSUVoimz8 zy){<nZx>RzV%5vOri0ad}lGOIHy~w0XoO4!7(Z5Dt`9%ebF>hLG zo=A#85xq&AUe43GXm$toflQMF&$St~xS=ykcYbYMYNX~XpUo^VM_Aq>+_4#D^E0D} zW{FWeJ4Zqo>bYj>tKBr`QR=Tf7A%MuJmoqnT2q#{_$JBt%3!B-Y^D|8PMz9DL7c_$ zpxm=;td#D=_b?eRX*gi+JYR3GHf4vA1kko7k^d?z2lb#y68$D5`j|?W+1NJ z)be2&`8s_bZ!tF4m1W%3SJ>+-VUXvspP^0PddJ%)oyqzJg1L6YdvH}m-YHJ@=|-$$ zuQ2b|pMKOR8kNzm5w|XL`+l!)gj4gybBe5n*%s;T@$BJOb|daC`8t~ZGJP}FF2Oj~ zNjolni`=^LGGEGO%89z=7YjC*Qm9(-5Zejgg=WN*@+|Q*UE*0o^uEc(@mMOej>5k9 z(-WNU`jsda-6o=`mGO@K!*aUzn`dgN*q4MU-X{>gIHec5Dvj&mqSfnKe>LJpN64!3 zOU@JQDUA8*fn?Xv@6jqJz0*Gk#)!oKjuF2f@FWLvbD&R;l^MXP50|8c=6Kc&L-YsA zqNGt(iF42Gsi@2;hu5OBXlITQ*=adErpWK(_`0(}E;e%~+rWRtuOPVBPnKyZ+r?GA zOGN+PSE(yo6Tt<+-yZBtEOF%C)LV*Y3HbK>=lsFmi!CL^jaP+uSSuH43}n2Y24+&k zR|&1WWU}9ISPeu(-ww-O%6nJEpy%Y;>EaQNcS_Srq55Ud4GYp<4=ryP3tqKK=8jeTK1t?MIs2?p4* z(@*#jx>ivNi3VQX7Ll%hu$CWDzUE*eP4~$_e^yf@mm^u>wGq=-r4X6->aBL;kvK_F z)7QA+c8avKn&RpFC(Joxzv46{b0Ur%RM=)i?^9GY zyHrHvi92hInY&Ms@3FL7wVsfM@5&5mMs(khvtK-Ct?$(@9yrp9Wv*G}jYnn8y75Q` zntLqUe>G($d*Mfo-yl0MB8>oWa&(RQhF7i+IBtX_~QEOD|Pf+fD+4rDXXJoeDzy1cpR z0}qF@^iq(Sy+v~TR{AROJlfu9&eYp1;aX5nVKyc4!|%J1n`VP|EXA=yOqizMow-%_SyN;$ zS1C6(E0T|dNMr1AOv`7%?z}PkZ;>r`dBR(GoX$2+uAGMQ2va3iMl>`7gpReVoetIv zuIrmv-n3Z~d-DqMjbg1-tErvId?1f&<)U4{&F9Z3Ev6cMG&WLl*M8KW?AbNZs4`uA zJsb}gD{s5#SUq)>UizYeCbvhDHyu;8px=oM?E#~X0i5hdr5$+V^*Od=^{UG(t)XJf z@v0HmbW5zKMFeJX$9SgO#FTEu>ujgR#?MlTk+SlvAWS-nz^iBpKo-xvGx3ftonC>w z-V)R~$JwgWVtc2aFmUru_P*I>R1%&h5^IVS*4};+TQOC7%$PNcBSgPrXq80T{)$0} zDY%C3ZnyFz_S@7`u_ipBd=gfI$>Sw^Dy?MU#5=nb*9B&|w|NtJ7`E>WP=CGSMzUfg z-n>Vw46PC=2q2~*vFqXI#ohQ$%i|u*$aX(w;~vZ9IK45HPLm)}V@ z7=>7LuzWv#ne2^7x}E2aRo*GVon4!4s{PFJJT{wmya(Y1X>>)6lS$?u3Jun1Gz3X( zGM_8vy|A{-8_$KyRV1>Vp5^uNv)K_?xl1{H&Uu*7mt7O@_!;1Px!;f*EaV7c9DAO#-mBb>b9VB zgZ93H`w=e&%ISC!_G)ioF5ombKaP^*WgUqV^xEmhd)0%vpYz#zW-?wauOzKjrF7Kj zE?W}$Pdc4-E}Dujk`d9lGs73j*=A%?BDI}%J(?&_AgR?PDZ@tRD$D$}_%z(awl}@n zlsj;|Lpj@WW>zrq>2mKU!AJRCruc&qRv3uz8bB{4!@`l5?4=ow{&=$w6crs$I!KX zkQMg+lA`nmZdZCb@2w?$kIGs3)C}#|D3*ctI_kNgpzC9`H#eS7Z2rACDlfhK{`1}xX8R9fikmLbFXk2D-QCzQL3oi^;Iee@Ek3wg=3>V9 zQndNR=1H~kms&Dc?n500c!Ez|TJ&Xq~X%U9KyqaH&nJnm7Bp&=$~6~BEqbihI~<^w7I1-=CwKf#Zq@n+SoN@~wXTZf-S zvMSu1sQXklrt@udG3VLtqxy$*WsP6<a}ZkTfLZ4X<;LNQ}FuD{z2b_y{+76EO+uXL4g1fe;Z(V1$zQAWCYql62>tRGIx_bK<*>TA^LX8KC??rM*) z{yJArac)}Eu#LJL;ZiVFCrz95`Q%Ms_`7+{k>0a!*X~tIt=6>N@}Q@rWB2?q7}q#m zeiL`R=R$IvGpD0h6V{t69I|(~X>{duv`=W!>ONu9zSYt-6EJuyB6@r_v*jXJ%ls7Q z#4gu#T4;`DTyk-m`oI+7m~0%A_^@o$i0m<3{s)z=ly4OcdcPv*SJK;4i7b^zd0j3% zzakM7>7B9YDKC{Dwfqh1gRQuC@!g5LQsJgDj?WEE?eHzYh- zGxT^9h6g!rqL2S0&!YgqC)DF{Cx8t!=}o(JMUE zJ!T}cI)z;r6$*yBLWKf_|Afh+Ky~0&f&AYiF-Rm7VFIq_s~*5;&~W6vD^#IaZ1~|V z{6D-Q8aR9jS$iF*{XU8#5V@`o2d)Muc?Zs7A81W{hrlT#If#V-E{nPeZbblRkQoOs zeT_4!g4za};Gh6Dj2ep?3m+dJ>$g?_9`zwR{EPlW3zZYVnNe(d0UMYZJb4_>1Q8Iz ziBJBwBp*h}`$$Rw0gS4Ed?%k+z6=J0x4~Biqrfg4iX((;vVd#=din3XaBqPQ)Z@p( z62+hzg<=WeqKH6uHw?~)UHiKclo(~6ksfKgucME@%SA_D3?(4z{F0YUYzfNcg0C?~ z`Tc*&Cx?@%V>a#o6kw1Hq9BH!WcjbgXfUdwgN%sb9RH}59)rk+9GWHp$pb2g;l!wJ zX(9&Q@&E;8hxUI%NdOIO3k4Nq`oA-q=u$|8vDQt3)C=b zenNq)*~sF*jQ}@+j(lLCP2l_*hIap3g7^qM7M9du2?(1IPK|2PeNrI24-J$nf=ve= z`eE2Agfhc=8VH*V&WmzGK{7xM4#fd#rerHhN+^g77$>qV(#LQb?}4`HfD?DXkdA!j z=+WdsC&-~_I-p5F4oxE&4Kt!JkO~F-Sh5!r9!4)CnUZ~r@wyIHc?E!}>=Bk&SUpTf ziaIf$fF?j52NdT_a=^KY^$+?4^qm||g_7}o(o@n=Y%HuOz!G^j_V1^S;}5+cii;No z;DXQn2TBjMQNSrtTmRwyqP78eGuR4A7cE{Wpu5y?g60-84!6`T$Q6`}%APbW0*DK}im zg9=WK60rDZfRGsI3M*hOfbWq{o68^aWSh(QP&d_K@AWvBde(p)?Lr1DEbb!(z^LH% zy9hC~#CND;QEE{4*(+#?T&dwqsCqMv!5cbHVqs}(U}3RiFyRV9lfI+|R)_5VF(d=a zduY}-a>yV~n!`Ri`hslC(dbw9Z*p)bljcy8do+O2*aI}i`T-?|3H{d>Fg0x7gcb_C zJo`9GC~aE6tNt$(^{Q2!cyN9P|D z5i~&uKZz0*n;xKeUmrn%w|S$EhC=R0)X1B^sAR`C$Dw7N{}FmHyVk16g!@^phL6Ii=&(J~L%!kHx^o?kkqjvu%G-rAi zI6ca{XJcdpUjvQ;fV&U|XX+nl&N-XpD6>T=D3Jw-C%c7I5F6CTa%dQnNNCg{l=&o_ z6U91+z(VQ(bl)PFg~(yB{=SQ507wa-J160X-gpvlF+cc=O8&A>2HoQTN<~^k@ppVL zbfH5=-xmgh9w-0Pm{9 z90SEA(4-(C6$z9kbJ*Z`9q%SN&@EAbJ!V{Mfo`zFbx{rOVFwL%RDf=>!v&E^st6zw z1#F6cY*v*6ka4K|AwwHm2@<-lii8S4DIAAQdKi#UAq(n_1NCAKO$F*`Qjp;41gds- zPQXB211&wsZbgfNzD7dZHIYy@2%ifsgfbn}ENfE>qTq(JLqS}Jz0&x}5yu8pR0N`$ z5{CXXwb7&?8x|949)MD!`@NUD{eb=o(6=-OJ((VoUIQh8B{z^j(+~*|fbzIO+(stI zYD7XKP0?&Yjw}ku?f3NJ1P?$}nW0f2R~8S-pF{IQ=F@_R_zs63eUK6hMriW~g`VV~SlwW{vLgrTi}^J-7bGeO=YUxJv1uS;P$7!m z`r~}A5uhLWN5F31BlLd+R1yFJM){*r2mv?`sy81Zq47a~3Q$8{Jiu0wD~{5X+1osz zf(@X2%&BBD0!e=w#UNM^Fkrid1c*Sd1mSEb=oci^BMjX{K_R#(3Th_=pxcpXD98my zkDh}fqZVrLi%Ec3MUi$beAo%Ly#;0n&^{_K(UQkd-{>JJNC(F7k4N4R2KBVvMKgOL z3_pod7|5MT5)azb4it#NKAYi*BBQ{V<{?d!NXC;b z1E^vPn5G1VZjD|ci>duH0bs$P((i#PCE?7%7rli4J4Zj7fd+Gw%S0DKW<5jOpNj;L zQ~3jm6$Db#&_mB94kKR|616(`2SowVNglRYP!gbCjiOPOqrh*$Hs8PV19l97pF-GD zaISy!a5`sb@F5suazSrNViX{qLRP?uOm@SJvY^yTEAt>gdL3Ai7zQcB44O1r3eJEc zz3=~WiUx4}2{eqUf{j@;DM*k*106^mHkMQxw8(P-jWUu3<2vXg=q3_sxrBzcNyC{@ zwzZXC`@|V}m=I`If;zCP zf*H2ib|GvTI6nl^?NFqGD36kcOcqGPjRjGh274^OQ|}N%w!iW2NW2jC5j@DlBY>*u zXi^T$sgOJF5qgly=NQB#ci7AFa)4hyAsYWXZ3pMyxsn($8imZl@lUrB;Vq;Gg0_u; zEJ&CU{??IV5J>L92pPy9w$Mf%)aXF@hb=9Xt$2ufhD7C2{XyYCAnylyk-)3~P!DO3 zpg`siR&)mWlH zU|Li#+Yx$@J%kP|^`Bu?fCGtVgI0dG<)TwE22`Q@_28yG%dok(Bz^n+_cyP0w8Pz4_DxgAnT{INj@gTaeI$y)1&vZv zKh&|EI)DaVMnmtb!`V&`r&Mk_Y^3qTt#r zjYPfm{DY!`#!ekd_!)`%t=LaD|f;RUYNMbK#`+8=!m^D33WY$ts4Xe=`u& z0t|Lbph_*cB!tj9Y(^r7X$Nw9Vi$Z>0FS>P(^4d<0jh?hH{~!5a`_QbB6jG)qoW=W zeNRvK92M9}273gUHb?syO&?YPG(i7*ljLQ}$#J9vDL|Q+Q|sO~Bt09X&;|o>6}g{* z9IlSu^1(Dxi4GKfsvaC4c`J|nuU8(u4S`vP_RZfi;BF8)<gl9xb|n8}q<#j1a07N6(^S@_6zdMG1N7!P!uxp~@cN z`=D=pfs!yu&kX-&iVuzH!B3#bj~>CpB(MGWms~~vaFS=E52RBYN7orY1|$3@fbEB0 zAuR`;qYaj|m|HJPGw5RAARjquhY=+}QAB1Z(wq-Kd=UI=Q{dO;JeuCb;BaH&=usej z4AP}VG$}X%B#YvD^jQ_AqsV>vLrM)P7y_*$1CFsFXxG#aB#H_3nCPFWbS{!YZ0ipN z{XZ>-%QYA`0kQBvYcNe&WBX4r)MGEwf69KnbSM20C_4eZm}TvE&}G2`GTMKHYB2&r zx$pkrjt9Yv4`W%(gx04ez|9*-gBe=tejcF*Pu0l(k>(VGZ9WAkR|dYAeTcPpq->k< zp;b&7!x^wWAv+T|FSPv�cbu1*#u3p-O|FfJ+{*2+VesVL>e>VCoMZ)1jxH?caHq z_yOtHe=5a;BT3Oq>-tx-0xv+bNB|XPvl5P>8G>hd|F($fG`6rBC?X2140Bkjz(E(2 oI0pZBvGFs6;Q@s0kjr*boj#1VDQRPLA6`2nezMAIK=dH2?qr delta 52126 zcmZ6yb8u!$v_2f$wylY6+xEn^^~S~|Z*1GPjfp1{+nU(Pcg~!9>$~;)W7o6Sv$|Jz z@9NrXRoBzURfyH)h=|H^;1C!fFfcG6J(XjLi1dH~mo;XT9^cWgo!vQDw$^?mieZF^ z&;=P*(uq*{Zvt)@dQhl3E|j>Uy}k5VXjb+lJ*&4@zK+|nx(O_0uqtbBip7P&5%CFY zt?nbH*GTjNSS$)OQQtf8?|CnqYPZjGA4o{b7#BgvzBGldJ(4X?>40bK4QZ*GQ5E) zkXIpr(9==XwMb5rV91(n;UaWJm~Mf0jx-8v;YT;Nj(JU8!-02sKdX68Rjwg_i)weT}Dd% zE)Bj<@IPa;x0{NKZiU~X&=|f^T?%8MW3ypH(Wo+lr?h{JjJ&8aYj&abBPzv%)%o_VZMpeMBgr;4Met4#RNXI2}>R9!rYNzsyaen7wnrn2YUw{3&% zsQE?SQes=80sF5^w!5?~p6{p-=E#B0kQe~y^UIsrtNA~UsqUe3U-OVLZP`p(XgN9A zy&_xDO_4sJ4Ul@{97p0J*5kHkiP9*PFVKA*kz=Ov}$s5GkMTy||1NX+strQMriIbBWI1;zj$m{@2 zEd8eak-PRP;o{myu~m5~$v6GZzs_6YI2Jkg<*1qa2Rn$2^c70Au$Xg;{G-9hAodRW zU=K+i{bHODvc0{K-l4qWaHPB@U@v4!xs6=uL}tFJ=y;BoiMsGAnMt_g=8*t!L6RfL z-S{&LZ|-q=h!w^b(=sRP_E*Q&mDm9H=O>>5>=|)&r6-R>1{;b@V@)@g5L<%z^9P>y zQ(i7FXsF5r1mc`5TC5;N>a;1(k|AoMbn*=8-(`n~i{Q_K+~!#a2LM^GQ>*&ss9A&S<7w65_*IERlzSF#K$2$72Ecyk1iY&zpy3_G@E0=Le*Fo*S>3AB!^Fy( zwMdGCE^^oY9M8;6Y{A1Qf}sE&5;wj_VvgA%meYt5M>MSs2$(_P2lR?IC74Fu!{Qal?^`MI+o{s77E&fkOVFC3 z<=_tRz=SIhMz)Y2a#Gbk3Vf(TkH@}D2AJk?a2p}fkU*kiyCCf6qugm~R4taST$ea? zYtKba+PfNaM0%~!nXm!wN;EDdq`LCWXV-5g!3xwMp(6q5Ct{`hu+X&!u)|89YZRN0 zB$}k|Yh>Ep0Vf0H;Z08RdX3v^-ho(yrHK?G!f*9K+dT-H?Mti^>Nzk-WeA$AFl&s1^ z$>G-0XR`9_yNNEi#)$us4pf_GZ4jE>W_D|7R_qfoY2qJIdc`A9%&GuYZwB2t^T7tY zYD6dtiSBOj)^Dpnwf@8~oA=8sugfo0s9kdBNMbWAO+5cZ z(NeINHR9@rlnMj18$plQ*i_XSOk+eeI}MW14>*XVplh%NVdFzY$u;^xCegaFG2VoY zrYUdluKfXZP$pA2Y&o>BoBOi_FLcun>5iPLUfK5*)K_KZAofYn>~0Tf0yox3tPBzseQ#G&aDDnGuT@&vn+x@5D=Z|iMq}R<)tD5JE6}!kqu3TQ+#RbStI9bK zUCemhbbREXx;$ZoW_%U^AHExL6f3}}UB@h+Ky+hX*t7J?7|6>L$PRB<#*!l75s1{J z<3K{VG8YmqLsF6AUWlB|#Z_b`)-ArWgLUK@c;t^LHq&!)AlTcQ+9U&99K7=ba)PiMl9!L=h>D&rgA1_&_ zc+P1RdW9_`)Q9tdL-=E1hP>96#`7fGkEuOmN0K30UD0cS;j>E13#W*y(_CVuhsZO) z0E&qKu&1M`u_FHZkIhZH^4gG!`ER7oVfE)Y(4ZZxB;_!2g}FNv;E zpKEF&vs?v3xVb8C;I#>|I{|DE^%12x=YF5J%dweR?^;{ykB}>MBN;6iU&pa8+H*s% zJW|;}=wa-sGa6lvb}2{HWQ|{$o2cR;6%FJAUcgcpucX+rtXAsmc?xFjH7BeM?3Z$; zWgSsBNmVnaS4YO!I5^sjx|_%>EDA$h&5w{E9|c^vk>2i7{ln>4pg#C_S;_|gJjbIC zfDzh;+}XiAHv&&$kZ50-p>--2HJ*%^9;s$Z@vH(Y&U{VdkstoRCkrS3X}+y6No^qk zoJ|olcd{$zz3GnEXeldVCRKWY{FIT+evZRYz|O0b-X=4$M0@{@;osav(O6)}+t21` z^GQnMcAMf+r@fV}YpQH|V6eTrfz6uR~mi3TcuD6}4OIOvtC zPza*b6sK&ul3aCi4uYzy>Bslp)q|cyfD4V#)Z3{ZAybrpd50%mj)MU~^^GQ>hT(>a z-&DcS{n6&g6xc?-5F2Vx_S%fx$8Lel1uRv3kfcxh@wm)Ik|0z`$os zFr?}A?fL}s)odR!YCGc20OKiaQu?T2TSxJ0C|prU0Mp3;QM<1|JndWMZ7Pqgp2)=R ztI_wTBwni@9cj8#`mWy$HT(^Bcw}w5)$m;;UN_)pH^dMP|G_$TVN72Z7Y6SBkK=8l zuY1L>5dUqmPHK!}x|vgv?7JNRV1Ao}$IY6PI_7Tvq2Z<0tWtPqasB zDqWtqS8TD9JnSFmd76;37fp*lr(EKlrEwv!drA{owI*6r`lLlMi+er+i=ynsgSKVS zj}oi7s=$e-9=I9BMMH+PtS5~6joiaJZ!zlb`lTt#hybVNv@?K!x~|0HP9({rjGl4*+FIqeBBHYN8}c|J%H&pA_-GmpKa2za7$8gh3gA7o@mp z0qEdRO~%wn|M{Ki_{slwz%*hEa5UhRLgW9A&@`*)i~6t8q+rYd^_QKqW^1amAo`!+ z*5UunM!}d77;cA$9(l<j!=pn_`7)@1ypinefO_c!|aunji4U5kp|W6bLIVgl6Z<`hh$U+d^1@N%-V zGmg~E#v|w|N?K433uyK0kxgPAYN3YDtY8?IYG#u)B*lAUSKy+ojc^z3WXa-X%vuaD zwt7i!=FQ-(%#wnyl*MUqL0G}(H`a0@!b1-irTILbKcx3*iPJDM#oC+I{35hN1@&JCdTxi>PXsAJOf`vduUd%q$~( z09736=^yFOm!DrbjB${bQ~&G=9Ga4Y`Ru?|KgoK!Bhn~?3O|UZe)lb*^=Dk{1C}}R ziG<)T)AoC$RSQVFFB%gB&iZu`K5Ae4W zE%Asg=CuY{5&+$>R{y+ap|bSX&t>xw%>9{X%zznYeISW59Vi`8&Y2Hz7n#aM$tb46 zW`If*@1oW+SQ;ruEOoy+W=)vdT&-L4b6r!R+s#edss7?=t;)67P^ou+LSIeY@9WrHQD2ki+pcS7^*) zy?IhvljzJrttBVZJ4HMIj)vNou+P}mS-&1$MB{f|Dq4{ObA;_ej!aw4E{o4~#`}8e z>=w2fAJ@bWFY1dci`*O$k}cWat_p3a#(XO^n>A~Vc;rZ!iVuopQ&7&ZAoA${Hj8hF#4+s#lr%+YAD-KM+Djl5wR-8>0Fl{%^bH6$U>tq zLq#4W(Jk>929$IG&wBRXFGdJcDc4wpvon_QhO1x$#?1#x4EO7;U{#?h5YXsZJJ8<< zoCgE2y*XIsO%p&PPaG!3YnvkVl7!)Ra7m<@eA6x zr_FM&+5*KD#U9d1e8ofJnT$m>+2_izEA>+x7cPkV6p{k^LrhwNT$&RP#ZwPgukG=C z`ot7ir12>i=LVo7J3q~v)6j`pVW{mz(wt)C+DAyI_0YK`Y$Nm`_4Lmek=*rck}NvV zE=i@L%1ed;p;hAon9*)w*@~x&nwFC)$EcmU&$R**5^kYPg|4jW1U-R>K>8CfpNZM7 zG-rj50M8Tx>4YjHWK^FXj%`zGOq%)1vE^vFo`ICid!jh4BHt?FnMx+d;Vc3J#rdp# zhnus;`CsO+S$^yp3>{`|+?fd8lF{=r*w2P(8R`Q7d1DW1yj{$2AXBbKHf3UW;0#c% zz10&{@AahZP9l`A))x%STM79A+xg?}H(vh>s}Hm+0qu7l#A?m?L^RECSnV1HGhIHh zO97bfSOpl^bWCQ;4|*8wp2csjvbjG$5Zm+2e(r)svvfUZ zwmo>1Jur3SUpMFy4@TT1eu4g*!^e7x1nU1eT#Cp4KRF!a-)y-H3Chrv;9l@Ig#rDi z8PWo=z+nI*e$Za(8f#x~xqK5j@(v^jq$n^bKhV&mPe>F*k&9($pwtPHvY1&jqSR=r zYB3F_YOl_)^w-bmI_aaM!b#}bCE89Z*Q@k4+Lw1)U0%ZVd#!I{1SgNv%nBSQHm3{&B-XTg-tE5u#-#F&t|t$DS>NBNzWSw# z%vy*i?-I<`5)NQ% zdNKIcM=joaeTZ%-VL0NY3g?)zTa=apeW#JSVeXp(JDH-z#8Z0vqF&qRqLDj!Qz^n~ zTd#CjMOu3Mwxq0KjDkL~$y`Sls=wrZvg%Q(rEoXT5Fo-! zf1pa6cIuMxy)r|vQ*q|Da<%c5Ky8L&xm%yZD964MQm|8%@m{4ym*S1#ah|c za;s8HC8t{zp$m5M$G23Ky2)_9Lo@&y+WPk$x(n&p@{uyK$|3bMY5WwqX{Bnk#y6(A zHAd0&{liV3xj(rCZdXCHl|A)h52SgW>IO$@X;sL+%a{xPC0YiD8d~t=E8|QnuQCCX4z4+Z{l6jpnn+(l<>OO)Nm3qb|@e~dZl?m|2c|TU#yq7O;|H#z6CTjuw*Qx|f zD?fhF_a)1fyr-u96O_=fcUC=*V>zd^GiIWTWrOJCq`pXA)1BK+DlG!6A)U~am2-)c z`NftA=>kqY9Bj9Ba@30*$3bsAxu?M9<17W=%wvD@VO>+b&XU!(3a`|$%3z4SgUW7< zo}AFbOUkyp(OZfg4T&@2ViIs9y;Qq2wIZGgPFf9I#Ib{i-is=qX+D~^V3aMQ#@WJL zwbuHzc0-t$$`RxwH*VN2o@$`B-TqzAK;u$&!xk>bx2QCr#R(&-Rby1Nrs-R4byRc9 zs>&|RlB1r{?C`R@_Z0cAHa|+dBU~t2>#kQ6n@JOY!`o85OfH63odY17&sT>7Vo)#L z5ApmwtYMYbM~VGwwJDV&j)4qo)I>)(X!zXEsw_`C1FZe zi34`^8pDp71|RtQAJl_Yl@*2tT3wAuqIdSyVhOearbx}%W#(UJm1wcW=2zX z^3BeL(W)GWy9>~V8;f*7=z|+T4j_pdJ)@FHYi~-_6#K$0y^8^mqrB}@IIZVsm+O}& zP&&}bC+25*85SwI7p!WEUG+wKPo=`_H0K>LK7&CX>@V>kcwj30qn9+1Glm!l(ED=f zaBLTv0_NyPqn;3^WIq=#FS3jy&@a)mYi_e6q?;*mkV1D@OBz$_IB*a}n-{;V9 zA-`&}O{-seR z;A`r1;s;ZmmV61k(X5CvMFyTxkC{atBuJu?r9H80HKbiUSFgm9h;qKGa`+Z)UFw*R zEqgtLGIF^qfLVz{OW{wnKOeTB4rdNd+o)DeRifiK%3`N$w6^oZ!Nh}m3~iS|zZFk3 zp*YSA)0qG!9oVZgImv|9_Hj|~cbT3PgM7B=1zz=HkxfaaYp6Z<;tW<$rje&2$G9q+ z=YnG2dw*|fZ#XzcN(O>*^1cp>H4(;WkQbJdFnOQ)r<=M-*HK$Uppfya@Pnd*y$979 z+p!_AHu9BnRUopjaOIXgN&S$ya9-L%v8zD5kXZ#p*=9xVRgpzG$#pVO70Yz;Vl=LU z6;xX3iE7IcA6X+tV}F7u;|z;{ly+(*dF2;rO0~P!N`*IQ+#wYs@nJ0bGI@ga}^yHg$_jG&gx(J!i)%oN|Q}L4IDiOvIGa1C$ zWTnH%(j^3LNb-S zI38`SzZTgkZCdCZ!A6q9=fL@*RR1I~LyE0rX`7dmTWNoP|A2%7chz{VLTgKq0X4Jq zACJwxY+MNtL7hO=BKlX)3n9=y;LqUFcVPr{SX%@MMio6VqdgTy4QFvihMRK(Fa z*m;fWOURK_-pbtYmTasz-80;W>M6HOO5L+`-m5*km0xW}2&OE{uBop{q8Vxnj*?KM zu>x@wtSNa2Qz}P+w%(T*Sk^_0t>HHE3 zg@8;-wI1V&x{A82GyK%oQlI(4rxX7e+m4?m-#&)+2=q_-?m6K$kEDGioP2#pY9FI7 z2iimImNr&XePsOMMFdh|v>*y+_~>-OZiK_Skti7v**pqfmk;pQ%_gVLFk`j~JQ*J< z?OHIgAM76ppEXO0qhx~s>#46y({lbw7gko`()N|tdS}pQ5v8xF)y@?-e-acP(B(eV zKdS32+j+@f9BU|J@@QGQ=%oA3#s2wg4$20Zoy9g zl;~Q%*$wWMDGHXpMwEP3e2%b}JEAo%^>wIdek4frQIB6LJfv29!k?=;Qise{up)`} zH%e0w#Ylm+x&iE*T79UEKHz7%Uf$E~UUElR;`Op;N=vo?*dou|T3!O*Zv=Z4r`zXQ z$Bn0y{i4iqTyAy-LnJ7<{>=E3#i7^i3Npfa4L9eo#0ZVHX-lRi+T)h)w!{+j&a?V$ z{Fb`$XTf~taA6!`y=92U3rUA#{Z3t4n0y_|hvwYf1b&I+2OirPJ0RIx%?_bF$@)I3 zJVnDCt|LhbV2xzahj`@+^=M`9^6N8?-DxOQF~cQ~Q;|1?8YN9tv5ReuW#RX5YGuI+ zW|#f(ioo5B{HU{l7kJaGJ?dp>f@@i^pUN_jZiNSAFl}TRVV`6u+DM0k0ZycQe6Pic zHJ3;CU4c?Tja46B=euuMc52THy^FpiW0WcP`eD}(piB}XW~CEWH5c@R(^zCQ4QE;0 z^G36Eaf8Cbu*Yj+Fl}WSE9_g?X4s;@SK!$<{T_pTOLnj&E&t5|*R_F#tp{`%BMBXB zD!nXsNy3k$}tlWl$xX*s7od z#SXL{00~Bq>OR0#`fhXUht0H9i@PMI0_$fIQDTNzJ zD-ZSlT#zks;^<&6Q&mEI_M!-nuqecqZ?vc#Fk9)drk6Egvh@kV)R|cjCq~mgK|(aD zeKQ|4LoggJa*$9v-N61 zyH0?C$Hv+RNd`BnNwHb;%@OGvB(1V`oSa7ES?VIUKSGD(&;Y7T{2OAjY9q&9?oUO2 z#PbPLRU?vL>A$xwvWpaP+UaC|IxE^!-ZC@J)X)9`)#5ot@EESx5&>r@rl_2&ucd!ryfA`B1W*G_yglZ$vvu9$~ML5tA+N#l|%#3xAncKex|*^Knk*Y z2ikLmn+9p}c{g6w)Y@(+Yqa@M2Hy)4l+L1YHGY$U5N!xJ%1&-3AR&JqJrpBYEElZ4 zfDy-MSw-Sn_sK61Lc@72UsJ0TIyxw_1$dJ%tV4@B3`8=Gc`X*|`H~P=H9-z*{VH|P zg4D-O(%4XZAdhsn*?{}F#_cG1WG_H`O`TbXO`{fajzZY@b>aq(| z9t)Lv9$lKqa^(Hm>58oz;!XSYft#J|(QGpHOwtdO)~8_(NS%`9*Vr-~JRA}jMF4M6 za=i=;Tqq^6Ik)PNUfspH*=|x0oeG5e?*t`DE^&|bTOHXvrdo-06GtWE&ty>AvvW*b zdv_pnj|WMqYmRV}3o?V8VD_(Uym4wJi78f^`(!hPuYAky9U>iu=C(XRsAyN7@q7DM z@Vgws5FmxH1A*4NHVs`T>S?e1P5@V)ZY`d%NklwNSA zmZxz`mngeeZr%ogqrym0=MMP+GAU@NF@Z)gk9GxPvG1cHmgRORkbvr@syd-G4dIR+1p`Yn{V< z8&@mNg5Kq#;^l=X97pZ3vAqf*&dH~Yq&Hh9{FyeVa4WM2YcsI|`H>Q8h-${HV-OWr zP?;{^ge8;`g>Z>ayaBF18xAOiI`M*`XdQ%-4kk*Pqd;D*rqgJLU)2k}U;)Q#_d7?H zAmc;dVe1<0$x!vlEZY7JVRvq)W&Hl8vh7^Q?-qTlNEr3EIvd>+ZQWKXX2V zoWGX#Kf{gzWddK&vyP<#ePPF63+p~xeD}2u+eJ?|pAc6~hgvQR)YO=r8b7spc9v?! z?3*??vlKRc8X}r`Iv*xrGaMML@1!Q5P@BmQ#1TG9l80XSApl)3H|$$4=JYSTQC%{G z)cZd^d$yD#$n?uh{6LY(OU7oQh!@W9d?$Tq&bhsjPL1wM;*Lf287(o8u?@XK-@Q`3 zpHoVM2{r&qMk&1 z;4UkR%s5|q1OUmd>3-*vxn(yCDo3*d@|p_XT!)v`-!eraCOfVFbb?a{I49p`Auaw; zy!BjE27oecvdjNwV^<*;!J7BL6P;0)XLQ3_MLF`wRJ6~W6TQ%t$4WSE5_TcV89{fF z;#Xn&-Yw@%rG)pwnc=^bGX(6w7(|`C4rM>UZlZq=%Ll9pb!SN~B@1=^)@JQ|7R*!? z^f>}kN`zj*x-9!XD3=JG?Y=nvk!*0l<5n{Ot@!?jCeDVAPkv-f)eZZMLoB~9$3ivz zXeWTwz1WIKrpOC!=^!zuL!MsVib#C)E!ok9=PtBOcvt=2yMML3n0IQfxM6UyIaj~e zVF#|3A^~u84vUbuT-7WoX(q0u_}-_F$pG)0 z0KvRVI-;pMPH)guqwyGmvfwY@`wh(QpF$N8RBv)!Di>*QQ?ZsXitH^Obo)q?Rosdt z=K!R+`b=>4;?f9r5wTkr$C(^Q<-Bz|u4=e=f z6GM>6s4L;?ocw*J$%3we8ef#M>ydWohJ|A*xU~v5A%!An=1&mDf(R>If$YXme!PJg zZ`gK15pO@FZzDE~PF#6ab7fYz-^3WG3;|b@!?=BeEr@>rVC(f0^oNZ$9TQb|ufDg9 z#qA3ZoPB*;MAuxlh+17ul?Fn%3u2bOx{n%hw6Mt-bHTQ?-09J=g7$q%Vj;7|1UTqk zjCJT;p+9Y$%92LAG_RGX+^N?W3#DG?A2edCm5N4GxCeLlMogohApJaUdLQ@v2LPYV z-N8Y1j|C+ni(pZA9ea*4x_Io2=N7R%(H<{x~iU4$>@pO~A?E7;e)$d_%yI5XEj-HJ?XCqdiFw~ zru*Vq82NR+Nc&|0hXQ<8g^$_wRe(LbqiBVr3R9pMX;04DOlW$Wbf*&UMVC5^8_l+B zjz0(*!AHvtFQNMi*;%K~H|&{ChsvNVADi72NC9=v(6fwpL`kk3+i8q~0{}@+9iX^1 zg467U%VV*0WG!tI<8(XrKzl>sm{f$QfGQ-KIB0_r#sXn^!B1B_refZF=w|cZAKrvk zdO<%&k?b6}?eD`UT!} zuk!kexV`V{^*jv>!LDxxU}C-8IBo(vY5o(!Q}b5Rig!QL|m zPUX;-MMYeqqaecu=z_4&Jel~`-keqVnGQaVr1q|@tm%=P!{3wG?yDfTQ^S| ze{RjZH^U&vh56Zl8o^VT(vrmYj8@a0LXhHAaZM5G!_M_*PABusuDf!C6>T8gi#s-j z?+^CKwk|f@f%k^+)%!khb{Q#qpRC;)=@oBsGGTFR&pAv7uz6n)0^eOI^Hn0THRQJ1 z0J%kW?Rvlap{{L)<(;q?kuZlD9lw~e>9#E5tpWcq8M9_8RnaO#W#BK5ZbvFMvvf(b zLrkrtnDl`Xe*4T1lMfrv8YX$CnF({9F^etSfxST2D-CZh5WH^eb>OGCm!~mhGHS#ivURLlN)J2 zDv>YLjA^CG@(gi6z0PI7xhkRP|EkAwS7Eg*h2Q9Sn!G00A9`i56CM_<0@emO8mue4 z#V@+UH^wwhNv2@adPQ0o-la%}9zqa|?H<(%jEUB?#2Y0PFp{d7X}`P@roikvgELTnI&PTGR*M ztk8@Abs5=l3XRq{Mn$VQU`4%H@)qP+X13(0j+xcCWd3cMVn`}2GH0SR?Lv9wziZu| z!P!=6A(($jfD{HhylM*}^mCc)`;*M>Zg`*{w>KVJLCJEJKc>2yuZeV<|49(rBZkS> zdcuP3yxr9!dDJTiOdx;o1Ixc^ZEkeQJ{pSxe21CUbU>?~E)y&#hVl>q496Wjh1ik$ zRp>6sv<1f}gg}_igW@g9MS} zwZGU%+=MV(p)u}sXbu(W_Pdl}x8u6Kl<+2XE5*{Y=@^_Dl$(AOeTPP#BuqLe5{r2U zK#eaX?&)hqr`Cog2Kbz)^=Mv?f9$e@>iDVoKmydhz{aVC8k|=GmS&uU4GBHxx-TqG z78Sz3*y0q3zCK4w86r)TO_4WM)EttJkKGn6yH0-+b)M&|QF2EinO>z5q#{}w2F;x} z$DLww6Nh37h93YT>7Jb{4-|F#l}cRz%i8T1QX;HH?c?qP0@AWm8ujzcbH1|6_8;Dz zDgo#Re}=z>P!qor@yb3$c3;)dMs)c*u#qhJw@94^Sl!;ai}ByP{Q?Utg3c`USPY3c z?GI*hN!xw=_XjbAh0WmCIAECxJ$HxreB`qQ4~)dw7Uk(#*1=H-L%Ali;_l7IvlZAX|$Gl_fH=iX-A`B@ZEW`UmleqDtU3+_((*12q zGCq>-OkX`K4vD>=Fw{eV&kunO$SSG1A?nQ$f`!D$|Mi~yo9oZQ%sZ}jd(fmG@ir*F zr>X<~i(;Vi;(-hBn*D_dc+L4j1iY5u2GteBR$!`Jm;ya#5)u)krM{DXF~yB>QBcwB z6ESXfG!A}sTqL^fkf(ea2q?kt3ru{v`KayLhI9zqR>`@yTKzr0dHzhu8I z{z?}z9E)7Gu|>F#o+KzB`v~`)gfp4hk6xu$NioboFjxFnT4+?~r!uI(mY6}3NVmYf z8JBxn3cQndG}5`u^cd4gYEl*Ibm;~%?i*Ff zpOIpW-t^=_Y@q71i7K3t{NA@)a;D6{w_oXg^?+zpgv7kr@9&obMi>Cwo2L=K-3Jv2 zgoqU$A2=C|TKik#ASogDYNO{a3dZ-tTT;910UM-KcVekO&Mt$_fH+Llw5fL~)aS{p zSVbSa#5!QE$O+JvOQ6bhtbvF({^VwZH16cXhz=X)cg)oTof?Qv*|`VPJKM*wE!K}U zJB&P6W)r3q@E@}hJsb%?6+t)Wg#lgOw2_P}p!M{S>*$#!CGQa5oW)3|4^4o1`p|Bv zW*1R@G9H`!znr1X0lcx!8N|(i7o;}HtHSj-%khww4-b!*))uSI*)n3uQry- ztlmY5?aSip*S-~n?d!t3!$;`yg)XbAjZft#hs1T_g7oWu6bVwNW7Sv}p0O z=q|sDim4R7{#=u(;LaP)Abm2i4wRiBJPJh;Nm4kZ!8kTuI>8o4jAmVT@>=Ucs|EejR(ZCqen6bf;({`}Hp_|Gx4I%#338wuoj|C{v zJoQA^!1z*Vlc83_12dKl9C*YCN%|AZTN*KHT{6>`Evu&1HY+WufYrWll$kg}N_0-#K^ws%^9?0h;GoX1t zd7`Eq9B-w04||XPB0_F|Pe{mXn%cm!fQ+@v*VPAZA0~WCMU=1I`Dm zu&+iA$Q1uPD#}K%;LaX_IqiH&hOy^M#gG zK#2S>v2-8JEX%T8V4{bEg<4?&kPw;)8exsz!*JcnZR@qw_GX3$>S{w<8rOF~N7xKotRoUUDUp77(P&=`Uz7aFvK=@4(j8Yp|Pkg)kw=nh|j^)dytnnn9{ zlkO{EI>PRSMV6_Nh-xfa>nM&k={BKkWeo;}j?!@Z zCa=VsIyeQD@{=Si3_pH+Ra>*Cq40i}Pk_SQ(224tOVd zz>w`C#LP_q1C)KKGw!zR@6KWvN;_?RB>dB$=9FBfLztNJ42$p1VvB9^`h`E7vbY?B zydLKMIZE4CT*DQmoC(yJVf&G3slr)3o&OF;m)T|{a%iq}>@lkg8HaCOEd`UM9KraT z^1gBHw?YDs7u&AJAW^2)f?k;5ayz;wD)X)bmp^3#S4`!17uv5Mcglx^;`~)Mn&S!i z{l7MQcLZ|RdaO!1>V3lpQRtlENSXOV&ghjh0igian4Lci>T4AR=7;R51`?O-q6Mke z41Ljm*fr3NDfbws%xLY3s~LeBq5`gfTGkUf`hbs z$|M)q4 zyVo$&j3^{-`tOBJCR}A^e<%PfNfCM{A-klfeF~#dqvg`^oN{iuU@f0-txp~(MSb1DDR_0O=%e@uheKk`GT zQRDxtzpDIyL&J^J{$+r3)BnY?pZK5X{Tlxhf@n$~^zXdI*CoW?rvC`~|MkEC``00W_Uia z|BV2-Dfch&_nV-QfWxHq5QD=tC2nf}P421M|AhUA#sD(x&HvjjEwSPs#-0Qmze#F8 z<-c`oAIJeB2;yB~_91jUsv%%_V54$ExoH$5LqgDLVoMqXqN<%s649Y)lw}K)Oz)&ep{E_yzPKK@RJUljilEx(ZPxi=`pji~q1 zgsDp4xi^d39wn&ZY&7v3?=T|=x zIqhez)}#>h0Oj}Mhj#N)DH4ab24M3sIaEx>>F3FJ;N~$CU>&LjI=vk{E9T%gdvxv~ z(s#Bu<%VV*gH$Y`Mm_w-q1w3*7<0~?jqjo@;Ym8?N-_wJ8`wY^5njJ?9Um|jC;wlU zUe*07cLLK?kAXuehAfvdjKEcH@Q%ZqVvl!jem-k9HGk(5I1C*j8N=x%G#?)-)sP!l zpHfIQW>LFcz>uGv>pJFE6zh*AOI;!+ecMUsQgV*u780WssDbinkelwLm}rP@v^v-* z^HTzA-X$vj;wy^-b5dlushW&dR5{gBCVka*g;Tj`T0u5j7RD*y#AwZFGiM{Ut+&Ge z$JSd$#T7K|q8Z#FXmBS;aCe8`5S-vH!CeP;x53>B1RsJX5Zv9}-Q5l$@B5v5zIEr< zo@%Szy;oONcRf`!C(c~Sgb-)z%oufy!-nu!?`kkR!vPveboXz@!5@Nf-rGj?T*;cIXR%xcf{x?Apxg7TPm20c z)c;-%oaVx{#8U^oD?j$fT-?Cw^yd0xKPx}rnDkD$$G9jdeO>BiR^tS|*M;P%2$u*W z9ooNX@EgzZcIWkT?;p>o!3w-}(bW{ExTW*g*e18-9ihOt^%36*vQlcdPSpisdnH_2)W@@ZZZE{SU{zd(`YXl@4lg* zcmjE6n|-oH*bEDS3h&@IOESNAsW+SZ6LwibMHr~q?4U=rFh-UR=$*Ed9`5~5?SJR9B{w^h0;e7WY(cfyIG&j z4khk)sy?BD73ZCgq5U!EgzNJ?n9q^@*;_qbAqv!w+)3G+BXp06()nyfr|?M%=Y-r_ z=U2ftN4-QdAcgWrB*F4?nrDCMon1bb>R63hwCd&WThl(rGFv;h?oKJJL9j`Bt2mCq zSjn`e%@Q!4Lj}P1sJYwGf%oIv`k!Sz;7RamWLw$?^~9KiB4tZBt%=5z*Xt%|+dD>B zDy+fy>9LSDr1Xg=86uKOJET@Zs3z~gq=2>9WIzx#zDNKYWXqR3)#vzrJ#4n`_LOrw z{`s?3r}!WL*n+TG4%nJ&?D38HQXjCA+WYl7!#KU`!5@J`k;j)!jxOfV2|o*&X6#b&%h89v$0bkk^A>3gc8m+razaNKNC zo^)eKp`HasRyeU0d!~lr%URuy8q@2XC2(3Jo#H=Uk(oH8qx5$j<2wMxSh-e24H(ng zzR@xqM<2@_z>8{q`B_Z*^OMD?L&dsEfifLi5W|v#CSed1J%c^AHIH;zz+}HM>`}1D z9_z$H{jD#Wg3P`hu_&Y3ycp^T!pD)L=850_P5?|Ggr*&HsM&;tHi5%Rkj2Jec+yi7 zrHSu!>Cyt;3|1@-M^9}8wjx7ESF^WnU+b=%aHGP=><)hwt&xQV9eETCIzz(IP~%Sl z#*?WIlpiAaBJdq)Nv!+uOBhc1bqS`G(FlZbW`wcp!-c2*rz%e^5Re%Xh3MgpWsDhl zJ;^M=PJhqmpnBhiI>PZ2YfkFq7_08;Zc^0 zp6bIId=GYp95>PF(TdH_!Y6v8E(1R4j$lYS+@CpCVPfhoa5p?T#)ch59T!u}cw%A| z(JCT>RP8#U9ivqcPoC{5ta?Z^cjWui734e*SNNobN3DpjZ8f#u;R-4=H{MqU>Q{HFi6O4I0cHs~7x;=ro=1;01+l zK?TlsadK8WKBV3yQFIn2ueDykiJ_BTRzks&ldoeRMx~CeRmaGP{l%WN?(4zW@?Llv z*oNDemPa?w6_jF0o?vA!Hb7&m0~^uberiXP%E|skE@+UvFcqXm{{`ap6O7T6FwI`h zmb0{grKbHf;a+-{6Yd$V)?GgdD6SLU{^PU9MpJL~K~*_x)U)=zJ2l zjj{}7d;_N%a97Tuq&VECD5I<=rGpUx7*qa8)lOf*W@Fg?xxHWUZ-^bD>0k*Ki7wa-HtU_e`{L+g(N~_AYqx2;Pa~48>(E{gY^#w*&KE zgLBuFcwm%+4yIOVzD`~gT#9v<039;Qbmp8CRnX%Bi)(@+YwR~rt$M>kZPGu-A4 zT;(+Hall@Z$^PY$E952zwS0h3LF*XAV|e1&C4i!f9$z1)gm7$g$B^`(3$()hcty}S zp01)JakXRK&@!DESIgA%)Df+vG-^i2mEC{jL{rR;DJF}zhu7rqGK&yG@V-)Sb4aO? z05iMbA-4lErGmmI+aJMuU-ZK{2|pBrVJ{x1z;F9BUTF}C<_?yb@(Rr+(p+GOwtB2XA0>Z91%R^Dy6#zo+I zy}o<_>1FHZ9xi%_ak*Qs>I%#}Da2h1518@uz536&W*7ZB*%5aM4J^gGh?PCIfATc= z@HL&>kw{TaB=$Mw{s4c@>kbR~L)V7z%cmV}t8#CPBi^0NI(VRHCCVkIXCT29@4Bmp zEmVnq{~jjf7fOwGtgR^$v2nU{)VsrLO;0wQyj0sd{yab}?#YpzrwJUQ7ob=zo4^4^dTB4g z>*-dX55jOuRF+)mRgu@@DU#S#EfZzVR{HWme45T%$AO^2^Hrty4VwyG$BVRL)`g7i z)oe!ke1*R2HCvsbTf``G5R}3%_hXi&VRkw`6ViF}fregXS2d`O?S@RdBOcYw%iX~) ztCk)Z)wcd8fD$N*b#~+4^yS@s+0x=1gySM#g@A&U$Nr;{(-jOblNkF)g{VcPm3vI3 z-W}0UfckW15k3ARory=Mh1SMk9fI@6$Ld^y0vW2$R-7u|zn51@VWpPRwcAa@%v{yM z1BJDuGLL7i&14|7Dti{?uE84vaI%;rh!B6u0Q!h%>o7oLi4V?==TM#)OGYvT z%h#Ggd*yEB9lxL0TaxB9UApbOQsP5L!`AcpMw7AbC$4 zQGhscQCQFe?&vGx09CW0fd0F|!~(%1)9vqzfY^ay&fNo>A-*7`x+rl=ElG=wqI+e` z3dwMrGFf1Y*no~xV?7?zzeTGaKt2mm5%2Phu_O;G|gZB)Vby$nk*8%gquL!f}S&&Z` z(5*9E!&_b_>^7`?k~^6lTSPPLMsYXh0Imk+?FQ1I@d?BeFxnZ zJAM5^}8G&88E$kDYkK@x9^q`y=Xuqq~9(HAE314n21W*$!Z^L8Pbk%@}C(M1^XbVBi-&&{JP zz1c$L9;@G!GQUG>RRoxE{z^~ha~?P51RA%8Ny}&E1dZGv+)Ph~@^DqRhty8}1P3l^ zddg6K8F+y?)1+;yzg-&m%{Xy$bf$hw-%Oh_a>naL^z#hXrr-q)2owX#0u9@rxSyb( zdc7a4?#KRNFl8tW_|MSZy*s1)UuP|~q}OC`2Mr)Wq}QX9*ca#o*PFK^h__eksziO{7 z*+Em}kl4TqwFAe$cMwYyLsSFU3fzQXfqY`qPN?g13^LDO}7U&1G0ZNLkrGBx-v zoI+UQ?;n5NrYMP(_mV^guZx#EnXNKpr=Sl%ABFJWzgkFC1uy(`0hkO@zz88?~-!d{@fo?}O`E ztw0s@#DYT769vsmGN!{4e9Fh^=6BFWy+#Qy`Js%UlbtVE z#YLO9veabcQ%RHco=Fe$Cxt1#X7hbZtyf23h7W1=Nu-Bca^TXaTuLk4EK3j)o#1|{ z$1<>*UYYU3W(V`+S%;nO(U9r2U4(U!i3jh8cz7UHpK!2ZF~1?xr@pl0F+Bzp{9#`X zZ4!Cw>FN58ob8m1ER$e2#e!DO87q&hwaYl85tk2%L<1}t!~WY%Ff`vzv4kcdZKy*u zve&kb;=I%Hv5@0qf8;1Pn9GHl^VfbhsL}=A%ieo@Aag3_B$-vMEarx)%ZrM3pr)b@ z%10_cK{S;wD#D>28=B%)n25Qypo}XE1c~3V!FM7(cth+n9=HI!R*muX=5ZgnKeEDb zn}GiU=J6G#Fn zF)=ZP(>SD&8Fp5AiBL>ZC9n8=u+R#M(=+S@gg%GGt!~NCm zr8}%1`xE_Va%s7uI4N36SCN*#*d^DF2%GKRGV+VjQ#df1Pp9yZ7#eF_#UTGubj?on z&Kp0af0h=_O#QH~S}|PU*c@HZ+zj-*_k=;Y&>QojWGMo%@|CTJhXi+4g?4xGsb=3c zqxz^OWRI4Zu<3AWgG*bZ?vHX-`c`GEZTncQ2MQyqf84NK{>=cKK(gb5JojDH+R#&ZswxS@kP>IEc=jIXN1TR z>VwouKT=e*@MB!Rf+go~i3K6u+WXRP2I_Rlf9PU&rN=jQ?$Fv>d-^n0rRbVP+Cc$y z+%R$k!*E{$CKfd%MwrorB4dEWB-k4mkzAzfV?y~?@Oe#8$ZT~*Hsqrou)n8?(Rc`2 z0?HQPR?vvlB4$8!v%H}-u47_#5Zx7{kx)lc_hr0&8JepZ~Ma^xK~ z6=1X!iA(HFsY%sP@%nzuvf^qk-YCkw+itY#&fHZ@!-SI3l1bUzWJF*9e#g~X+A5>^ znDC}Vsc<+x(EU5F>)Lt72p$8sU{GNQUcu!YQ?r00r%xNHlMl`QRSe^Zc3kmU?^S9a z{n+G4>=$LXI_7(fpv6c9ySNVI@JLjBn!-~+$Dt^~6U6?Uq(&D*UaEIE`yo0<^+=on zXl$rbU+@7U;E-0<1sRBOD3d|9jO#*pQv1U$&O1ux5MSG11!2<#jXJ>mHy$5!*z?+4 zWW8b-sK6{I$_)@7nV%_7in8>nE%hFYRMT9#39k-sBP$A z4hjGs|1%7g!3I)Ztl3^aF>_a&SEZ~NtQdPf#61n0fW|dG)Gxy!Xpw!Ca^7~sN}y^M}2~rW%%T}Dm8DlR~JcQnd^pCyZ_IG?48iEy|xTxO7AupbNZ zD(waVfA$W6`WLtG^w#pc+P?6p_oWfy=^c^1!ptND$YVyf&)K~yy0ab6cXZe?ek45O zXnXOmkW?^QK2qXDajbc3Zs6-|vh(Vlp`BUVv}K-*-pT`fK;Ie{hV-^{@PKkWH;qDz zt>=vjP0c1#Glf@C1o7?u&=-R+aSmskcVEwew+%+m8HAvL6m~OD(clx;3&=-(L-**2 zGPjtjni;%t-N{>09Qsw6%*U!0;`g*SQswGI4?14H4;haotAfG9mD%Rkp2K13w|4>ni0@7SaCT+dH* z9$}yJBVPUhv1?Y)m~6wp`^f_cwHUeXe#rZLHGQUfYWM8zKKc0?5rC2O+!pU8CZZB8 z=oTDNx&C7Q*cMT_jT$(Ky5toocpj?*j0w?r$+%~oj9zn8~Yq78PS0W0sx zCl!}^g;2V=XkXvtcM~&}LwuIN+Abs8itAyrD+^yL6HDqhYV59$c0byd=;1^3k7)be zPXKB)x;B>NrpS@tn)+;4A<_3}Ud#i(SkBC{(Is4#V6vuB%+pm$n_-!u9OYUDgqK&{ zQ&IhS>0cJyU4~bjwyRPv&)!$M&&EDiS|l`RS*D!1pV#S?`7WL6mpsP&K`y^#Q<3Rs zv!ZnMG8nm`^H|Zi-d&X9wEWVaX=yzLoWUZ`y&XA>cvq6C2q1QPjV8BRLCmlFh|Vxi z?~DAmDh6KR@Pgw6DA0xmk&Rl(z#lqwUQ_->T@|vO(`?q4*4I@n5{0UBhI(=)Tnk-U%n;+VLQzCQyf=JS1okvQKhI*f z3r=oF*gy!kME12ZH`;Wb6Xjx&*9ZJgN^&M6vEp8h*2YEcf>?U)9^DpVd$+t3BC?h; z;_o7@%r&kG9wYvWa|SIk#-nFhQ%Ol+6$TsNyLJqaWIo-xasC*D2{%*m8QDnH=MQld ztPx^0=Q8~c3y<8FgNml8X2&Cb4yOB7$G!@nk=q7{)oKq8Z(l7EI*dJPFhQ-;ynrdte8@3S9r^6ur@c*#o!` zp7l65D7Br4pk_!y=ZmNre><+-n;_qAnuVv#a*v#W@>GIaNPPp8iOqhD@r|qV9M&gx^QCt+DM`S^5=Wvn-vy z$j-sfmz=HzmR5uY&iI|*B|*v+?>F)+t|JD0_0EelguO;X5o6PTM^ThvK8O>>JVy&% zozIJg6fQBV&!gD=P9Q4hBRXn*+QDM?JqLkX#HUWEZZYo-e&95MC(~lJRE8-}T-()w zcrkct2#m*7DpwmDXki&=B5=lbQLWB^;UFJh2s7)`?HygcoP*b<2U|TbGlB{uXS>PP zL>E+((JuiROB6uWUx$Xmel27E4I#uce$C!=E9Z;c&(!BQ)eOrJ_sQh1LMhT*L>p*B z26FU%1>-O&roaqeQ|sS?0&R(4|cIzAV8}Rf%RTG;PJ*vHPy5)D_R|Jzz#9Y zgBeOnP5xpKiF*7s#+t}?!HDND{=0r0hlUtshB9=br!Gf5E>E#TT7#SqQ$p?Br1unj z?zU!%NUscCkx+JRa4f(=&;OU$VQI=B+YN?z_GK+lUjsZaGqx5ZL02RT)2k|q2o~iE z9mLeN&dlgwQ)S8&FqWNDzJjV!hSoTEfu4gbNDyL{Q&d65>Z+OSo}enW*I~>|BNbtW z%%A5GCsI}xD#}KqiT^%$45xKhR?bcka-xHdmN1-zl^|Bf6Jrb-&UdSm2Nr^zJ%V(n z_H2RU_w-Nl#d_mE*y?4~&7^m-FW(a#8kP<3D`QioSHR)qHCM4@`bY?xrCm!QK zm6NL%fAkUk04(BM-4Qnagcot^F<8K_wW?@_Q|5@k=4?ayV@kDItD=;>iATX3!O{_e zrf_(RPegsEygsc-NL?Vv*W=v2mY>j1P*0t3wx7PH9Ycp`*re%v!ZS_lM>)gZw^&J_ z?*Lo9(hk|)H~;H*h%CNqJ)z$eTgA?2SEp3L9b=D#(!pvP>iqj66c0_|)ESkM4wAMl zJqrfbr)vAOw297iVl`y8@5&6+=$AFPTKeAMfa-Fk)zi0_`qgjOl~%9sr;SQy26b8w zI^Q#@IZHZwsP8Uy@Hfp%pX^!UI|VvgiRbIsJ-^%UlG%}l`4utQ6=dn@Ct-z9=o!|e zq7Et@a2C+i*Cvgja}O-49G^nj)2qA#?xEKb-CQdp$WwCVl4z-$b*6{Xtn# zgPQGDzbZXRO>Vzu%TUifBxZx!ef9BG8aohB$ySWu-P1fB=0l#5cJQPdt(Rp`Z#u|zvW86?V){Whb zu1D_h`}kyi(|yK0-#LqpPH|gK`N0%&(|ukO+Wm8(h`>xE;36&A*2W;*=L_>A4zTII zhF~*lA4nIpU*u0PuUlf19MCSV4DO#hF;0s$bi-mv$s*c-uy&6r1X<{A)+wmegaVIn z{lSCDd!Z3ca_Zo6UN-0A%HuDtIGbfZu*-DT7205M92H2x7W=w7OT{i$`v*<;!S_@S z@sR~^xI3}-#nL3gPxOh0WEJUXMA08DY!AvzfhE9ZF#1oSDWIb%E{}QgyQcewCOvi3 z68^OL^74>LA|3uuF_M!4D$q^)s=_))>IY(UsG$%06Yk9UFoSdG5@F=JdrIyEV%UqT zNrF1wl7!>#0*s@3l=r2TDCIqhjgjdr&f}9;Ob8)!{?3sJ1+=Om1_%pLU$k$io+EAI zfOAAU?8jnaW=>a)_m!1dia3zbDrg)k@>PCq* zPMJ{_G%kBFR)@q*OQSyFqd~`uTE&hwF2L+#p>(|AP?^{Ng&hrfs3|%K7vd7e($!M_ zUE7)>HcSPMgTLUY2D{Hus$m`Kl&|#jMDlk9O#jJpn{q1XS6ff6PO@F%D5CS^2&%2%^c`$&5iQ3C| zMiE}mHr(i*m>`E&z~SP&sB^s+2IOXEE7pe=T*5t&u5 zh4I95jxua?yXBhe2CD~xt{?11xfdXwbG$Afx8$4OW8ElGf>Js3%*+IFlJuSHKl@zi z$B!t;u?KCPvXfUaNFEX=mofoaqL5Ke=Z+%!*sp~sHCY=Mq|4$v6)9I!yhVgye7bNE z)2ZtJkVrqSIihL&A;uD1Wzz7=>8G6-u{HS6C($uuf`vb9Hl7n6x8?Fp)R5|F4sS(0MoAa08RqXpGNO2g;+T3y zze~bYBavhxdO@PurnPq?*Faz~@4BtDArnL}n={*(_1MvIi%#BtHQ0@11!AlGX8kon z1mDGpOg#lFwfVPL=f@QwGA9LOtl_|Uz3Bk%=OUj%-2S-F@Lf7Kr{Z59zi!F2#GcXi zMWXes1SJ~7Bo{2o)CARPLatKuIOd-@E7AQ*QsXcUQ>%@f9BjN439KC44z7tbun<2; z7W!@!FdHVT4)+7Abn*vT(tRmAoN`*K7Suz~snPvFVr@eJ=esIg;4-~r3L2+xmA)4- zW$-l_`a*8`Ydoy)<=G|KdtL%RrtSy?)YSzg3H!|K(}#DY`V!Rgp+(X=(;uDXx|h6b z1-HMRVAT3nEiRR273l{5us{gm>;o5#6gip$m`>n^5dF=AezFYUDE7=#uhau;q%gKt zJB@;UTHR_pb~ z0QO8S^>Wpt&R$gmre~`=WZdO;=58OEc6_F5yw)yAz-30fcS-3V{<^bSWGI2%}^(?rjXPo~+C2R|rV;0>kt8@>|XvucC z!Lxd?epHk<9WKze%zMdEwX7_-*@FaCk^^vIRQiP@cEdqK*h8<9PIHX3B~k8p zG0h1Hvh7XcQSo_C88b`AlnnV9k+buSK#kFjxjwfAag^{x#CG3<`S&U_&e^X*L2g+Q zx`bGxpYX(sC#2XyXw4a-irv}X)0ryrN~q`82M_05RNX7r#SIs8RTK{NnS~b48-Jon zru$U=S!PlUmtF}4i0>WQU^CPTLs>nhR3{S}gFqUa(fz%8d^Xr1&MIpt_Q3Yc9hq3v zVAiJh=da|%3){hL2?rEjU-@BY8X?5>^?ga|A4T^{JV7NfwBIhdq81YP3_^tayXz+C zeIMTk{dfJ`d&v-?MsE{;_mIQ*Wz zmPI=RIbjp*U%hRlD!G-KBaZ3&mZZp~4|z)qJO+Kz61BkT>)T~bUo9L;gBzAOb?By; z_85!$mb%!c1ea9z`&-ZC9g>?+M%hw%tvQF{3TLpDpN3q=HNJDWHf>TqX!~gHvm|e) z2e3AwDhBvX0{>`em0gW}!awTWT9Q;(11AcybtNNE~1?mNie6YJ0dfm?_&CSCP2B&r)ZT*G$(sw5N;}klxHMqA__ z%XuFiQ1=d|ZM3!ogq`1B02((cMvbz7zZpju z(fq*dnl_-eEL~g#X)Xt$LGy0@p0RzN;`3!m3{Z{vNG|K1!eQf1wbvepZ)C=tWl3g$ zbV3OS)IXrVq)vd)HCgQ@e8|Rya^Ej9=PFz-&gu$2!)C4oiu+xG^M491Ca} z8ZO%8H3#k~aY^O;929~yY2$O=H0$g{fAS3;LoFrH9g`XP9n&gAwu#i8^5_Y*?*P%B;=rTX?!2wP)yLAkFj~7@NrYu2eA-wIGQEn| zUS`e1Tvg72s<#w%hrujc>pCuhGKY>x=|ig8f%VW5#)OcAMKzK_KWBTn4$NKSA4qik zAY0FwS*N87_&%%G=Yx$D;>W&hw3k0ZHa#&o^xo*lzlcd+{9xl`876b$+5#5i60&JO zEA;e98Sz31`2%rJL44v7sN+*dMPuGgeRWMEOo30D?;qF$d+GywFMbN#w}>3JS86s8 z7BJS$$S%HK0goYvf8m~yqHzE7;MOr93YQ<-H%USy7cz>cN$K6iRJK!H#TWeX?9`21 z6Gj{xGVD`-$_9K2hJ-_?U;$8~>Pw%h>9hq9mW(lYY*-E)&M4AjC3e@IsNzDW{Eemv z->d`MCT;I?$`zog@VDkMpLVuYxoc9Rw1ewdC*{v*&(F0dRLvF3)(kCIr1S2?Tk{7? z%@!ICyA2P$ew3%#;QZy8fhz$h%|EN+<4wveh0S>9(u-hp-)2{0?^fU+)ht&xoGAJc zI(xa)RJ~+9Q%UUnq^*&<{G-M#m;9qy?@J+T<|Q3dMn@;>cr)fB1lK^+QU0QQDf{Y9 z=9#YSVsn90S z(3~d}pdEZ<@n#EcO~CCk+l2D=&*ym-R4S)Blu=ICgxWzz-?k}5W4}*~@sQ6A?!S=5 zx5OJn%ovwe6rr^GBjP9ds>q2GmtzY`7q8IxXC*x`Dfwd89KQhXA28BjjD~WVPjN6} zF%-m5*n=D`P`;ayo)?-LOP6!UG_Arm8O-63ayI?x{SiF2bjqkFn&nK)k_H=VbtGCK z@P~Y_nJ?TY^!A(}CKqQ7dU?D0k|A5<1*Rn?X9XqeSCHbqly=vZ_RtVt+>9N4c1#@F zwjo~-<65Z~eRe;P^UT5vNA2ni$tT0?e1b1RZ*}gGYGf^+Pkrm0Pqq7MpF+1penZ?6 zg^}aiF>Xqq04?MtUg96{n96F2xUnAcQ3{mVx&+&bt(Yy&ef{>Q+hya(pLbF<_Jqbt ze;7yMvXH2Wd=mD*R|{w%;#n!0{Kmvs{xSqT^hLinMUz$m_|CdGFGyP$dBbAXpqPf1 zm~V2^ig;}=K176r)g8vA@fO_!;s{3ZzIsbkwh<`(YfCd{Od}?F;PiuF z(VFCMjY=igq3K(v$J#lzrU+rxH9i4bDkBrTX5m#}BGj&ts3xmxm} zN)>lsRJD8=n>yWTNhDbO4BZkbeMZzJ=Y+ti%ntt01Y}2Eb+kyyrHPN3rl_66+xRnU zP9MlXonPPE`+mr+G7G!%0SFyJ?Qom*+u1VYZCu;FNd4gQGXuU!@U+0pEyW@e_bbHneP8g$)eCXM6!cPus0Hl}!)l9>AQs`Um}^mRxQl5n7?s-DzPvn;3$Q9ZjuFt&IH<(z6`ou* z=dZ(EOHSDF#B&iUcB?CcKghw%Zd1(OeAnLT-*>@`V#rSY%1|9nBDF6w>f=ysuda{p z_!>Rh`z50lm(B%`6Gel$b-pboYp_=~O01F1Gg&Kg{GhbeLNS4?lJi)VqqEpLH^H55 z(gJ^dunXQTy>S?m@~wJ|kgidgSZ z`(B1DVeGk==RI@0Q8HcV<~*pVa)eUFKFXia7rcfnZa)&m>b8|OFVJ>t9m=m{Wsd$t zexcJe?TOZeU*X9nSsI?sSvXjYOIZMvC& zb!rm}ISwC&w%8?*WQutks~L9sEp$0;g|6G}0wB zQC#7~z8xC_lZT>nt#wCqG&y#VtS?}IC={IKo2Cy>qVp}xbfycVOt$_jtelL&X`y-M zQ{=4=Qb$8gOT9{~M?Bv}M;jNu6OAEAUBSf~9ODYb*w?#hd*o2!P|5K8U=CnjAiY2D zyv#Y=se4L@#S@33YT1o&4e1}BPK;k??trtV_kXDu+nam>|Ccury^SW zbs0)vcalm$lZ=xJr3BD5ivm_&toq9x;B)!>D{`OY>{>0`Grh=Z-bYlQ8Lb=Uapn9% z4VrTHubI#7e=mjk?wj+Fh~B*m7x~|p!usX^xfFh(fJ6jkP(WgVqnH6z|1y%i!KPk) ze?YH=@4^0T0I7c|Mt+gLxdeehX+S10HC1$ricOk zAbv$gzje~147mO``YV=;|E*Pn2B70#zpo?~&=M^q0T|;e0Pv6Y>caG&K3@wkK`+#= zoEYf~b>c1}z3mS0;V1i%!{_r-{|BFuQ1aSC&hWl3^B+mc|2i|f4)crqW!h&Br8;1Tn zDd5*sHIZ+HU`Bs{)W0XsKtRak_0O=d!Z_|Us z3`qiJivy&-H3h!{?e0uNWzC0;f+t zk;EAQf^J(b(Wli&>*~Q!pE2AQh?fub&&O00G?=4(n^}%0S&lwC?yfJlW@l#acuxzX zsk0`S%JLzMrrh90)N@69m?N#+gpXnlfNt+UV&25BQ(>CzYg0`+h;$aP#Hpkwt`}%g zO|ldwV_2q6Gv)stAM z^gP5+%OjbKO#s&!S0)NJmeL+Ngsl*{eT#`g7vUp&-J!Nr3dkv`B8vC9>hUZaUn@m7 zu4Eq**)uPU6_|_Hb{nNt3zG;Dz7U1GFzq!Z(T9n}H>4;1CYP2Nc1s?~%mFfrVDneY zneiw=<$t_49{Z7_GhBUDQn9`uAqTxBrPpbNNoen=`=#vKV3%$atStOtHdqph=h=~% zCa4o;-A$wO0A~4DI)s(lu_`2Cs0Ej84lFFvH}PV>fN+@{WqoUo9ZcF(E^~L5y`2q~ zDtsN7;K_4}YkaPP(=3R5TWugI4FtY)aRB!Mkqdo@2_wP>H$+=2x1L|kTy_&M2A&~m z4P)hXt=iq7QWpG$(=5N6Gk~(qs;HOos*zX08phUcah(>n>xx)Z^KJhrx7);T$AFXhbg0CjmvVNV?3;D0gFUYX*}ILaE>%Cu&p{rT z5AW~4z0YUiI)wZ7>LcgpZ=Gw?f|VHcHHlYUWqV1z=Jwb8Mfnrg z(cWPeBX($`!r8zkImEe#C>0AEr7o>nPM1jgdR>3)Hh;TaWRbdIltnSmTM2zTLQf0p z+@ib6`Jws<@4Ao&n`sx~i6`S#k;vGj!=~a>HbNa#!dBtma6k>&XKVtR=|}s({`2M{F|kWEI3EKBdT2B?`B0ayaAwu=?@m*tFbqzi_%q#Wsd+U{BeB`P$|8=HDWL zqHFGpu7D)Viil|X}u<>{vOc5xXWqJ7aN%8G`$X{*`sB^TT)`@(S!<`~$X%3tOW!2ttG z(+fn1pg-xIwaw&(M@_xRFy>4x<&o=Xi@sT%TfVwoH-I5^lW=wj-j9PV_=94I_o)v6 z`56-1L{d}t6c<^JZ6^9CeF(|8?hn>!TrwWf#JxMr!)7?!h_?tmXYB2fjB8NUvR;xN zNlt)-E(8TMBd(HD_hpljSY`Tpjqj(1b1JrCNlFWompCrr?Nt&g^k=n=-{VJ?EmkG{ zJ&2xg{%UD+(y@F3ugLcW#Q&|Oy~!zE4oEz3{1m|ApPYK022B1F%5T}_$T%U1z{HDy zqJP=zUbE&T=|jMQ9#{S;OrXDA{H1^uR{uX@ksc(#^{OjbtpgJN_4+E?-2UlqUORxD zf8z7MO+h}~km%s=dw_o+bI>aX^dN@KSLw-n42Xk!lZ}7j^smFfgQ0E!)c-{Fs}lG| z0u}PS4&`wV0KaKBf1&TMHD}+Nz3Li&|Db^Ecp>q@u>gpjw^vJe)e&FG$e>yn2xee4 z=;s2{InTW`&jd>TaNUdReHxFq&BDhym07tWF6izhxom(G=!wg-$~6dbY|w>&Wr z-N>_4HI1sjploguuH*r(_9MSsxhG{e?N)w4LQcRCv1lNE3AdZxQKBFVjwqKGZKT(0 zbZizWE7opmgTp;1)13~;>0{CbiOd0om3zDCOn9T1VzV|77vMz<$|a~9@#h?2AFPlH zlaf`IqcviibvLc=4JG=8?6;ufDMAi}8%=Bx>v0_vBsfk!9Bm2FgT1o7E|+f#ii>cm zBn0GqoHBFGIP9QB^_CJ0f7aL1bg;{RKzh3Rwf4+|+Ul0+Gn-TgYU3?DrJ5J8nxwqj?elYLRYP!}T%^h65E1P=HAQouwl+;@(MzZx-!niO^*MgLP0fB2nH@^1EbeQOa{ zCUe~5OKpBnMeX8^IzOc)jE`yKm@1!SM?%e&8-WExhd>%ETLxYE1_Xt*8r$9M+7U2*Zt1(*A(dvd> zw8}i$%8fa8h{4^tDv2oJVN(mpUZ{H}0IjhyiFpZvO@U&2tkz-cq@M?la`FW+bUIn2 zC*Jq(tSXiPNuO_Ikb!E_zYR0IilQ93HH)alS=-=gdf963jjQiDaeX zSL+Gfj092pwzYs?Eezg&d)MFn2)uv+G51fU|GSleH?Sc-{M$MIYVu$@dp810}tdegfX9s5-Wvr=}AJV;V=t)Ke^46g>mFjU`3HUXW76u zRb`eo*HZ9>p5(qH5)1gArAC^$Sd}`2#m*La9+tr><+Ne?$QuEQ|A9n zW@|9tfg~Ev8~vIfB6sc_Wo4tGXRF~!mPWx&^V2sLV?B5sg&9VWm|mZ2K<6jTb1aMM?15OG>n1HZC6Js`Gy9DGEW3#1cpHPOEi&2KHb)Y^3qOnSl3pXwL7P3s-v<5LI|mtg(yHg*GX4>4XvWP?cT-@ex7!&tK=a%E-02`7>Im^S|HR z_;tPXb*7CD90B|YZF9)^=PuH|f5He+vV!v1{h*TLz%AS0K6~Hb1Md=}_>T`x9bP=~ zgP#~bFvL#wjfw{{;0(?egaM`Ie>h741@{!d%i0Z;Y&{kb>Ud+$y5&L%6HtSBpG zhwRNw*(9@%Eiyu(Y{`sL*(4*o$WDc*{GWTTitq2=>vdn>uXEnddDc1SInVqg)5Ks1 zU8@Or$TygmP#QPKwJ}c>5TBCGHQuIwg^auv^+DdehS-%iFbs?w5WbwSTGYh89$1YI z+Tu1O{g>U0)(?AC?&3@JtJ{ID3mz~cuE{@J;MagKbfA$}(_no^CSCFh^Mjfg!mL<(; zWNZ?f&f931+8L;}H`zR&WM5Oru(9qwBjIUIJ0<~bTdYcGDb5t((Q;Ww89wuQ!jlEm zei5zqMv*^~w|Cb<^3htqQDBL<*u)UXWMij))mvr{-)v&xyH=XRCPP!M{_NG)qdl3 zu?&eXPbS%lM9H0j*NV0@WqzFayUUCBQe|-H33_ySBCNmpli;kp+g<2&TB!A1m)6bU zzUA^p1o75I`=Da#&0&wnK^21ax9bziZI_v;&!cR+%>t%bq|NE4Tl!WGVVmu5sOOvkkLtynDdo#xJW6Hj^OtUgV)86jyBAIkyyeo-Pe zy?oE`k=mWI0s3aTMEj}&IVKm+b@Rr@I+;!S0fIjCoL%(~l_LkolEgz*9=(=V{D!+7 zllt0yA%&;YSnac`pL%-PrxpUs(1zs-9z-=27N!m5-D@a)SD3xye204SF~6YL)bi7e zFDQMtfAy$bBj{>ZeaIPxTJwpyplqAMoUu) zkMi4#s5>c|ybeGA)&9C%hXvi&04Zr~{7Z#T92JQ=uS%wotl9j8Mj7V}3?dL_l#_MF zsY^GhHrcJ?Bsg8`Y3=R#UdlUl>pc{0h~$)A7wn7@MCp6GC)JpTY}{9xtLbuYr=TEb z(}pyEt%qaTXFJY3?;jYG0()ddDdq;wlFp*GrbQG@3|qn!Bab)Lvxd8`uTOY^1IXcB z_pW`89&Z|KnG*cf<8iH(2Z7Re@0rGwayzC|t1VC1{qNfBSFYGJ1UFe`y5SXG$;;)a zhT}(yaTL)6#sGcKxyS#dKk+{EKt%L`Wj>LmgKi^Y-bmAcP9 z&-}8zvc^KD-D`dJ9(^~~c8$|Ft_xs3t8-Q}{>4pdkqJF#{@-`2d$b?#U%b#q>buw*#oey|Pv4O-7%D!h5736KpyzMMrFT@w1YLkp{xO5pid1EMUS!Gv8~HHKB^Oay@GtP8uh4oFPD+4+$5mR6$*g4f}at6^XB zg60!C31>F5C5b%o+XR6LI2?Mn^gG6h7S#IIM zRd44^dc)P}l#l&hk#-^^qy-B;y?_Keeuo z?+jXJjOO}Nhw_R1#m19_^V(f7FHUz@5YbKv6UW)KJg~yqt-5Ji%Q$Cf4u8Vyi_kMR zc0{l!s9LdHOcRGYr1MB~vpWsyl|D^>d22i%YwlXf?ce=DZZ;xOSq&+jY6K!WOjMga z_a-Ro$T#tNvxaRPDc3~4oh3s0d^}mL_4B%sQ?aRRW;+2icy#aYKKSaiz{$OP%eSA} zMg;c}gNONW7gwdPXi0l_e0l(_T#cqnb4&3*b$ z!)f;RPVMdqg>I2dY~DpzsJPUBdKE6?f9nmw43dYfaQLx7!(y z^Gp#mt;FybjQ5F2L^fv$O8dy4pHplSd2&G`gY&+B%8tOGV8M^^c~kEqVJPuxKX%N) zwXf{FzeK+1&o7Q8C*EHC;p^E+xIm_sDH>=+?}t6-b4z?=*txkfNprsA&Mo2nJN(q>!;4 zY#l{c>ITDukSd&}{*FD2>+e#FNd1xnO#IlN`c5!0^q*n{Z3FQ0=ZS;NxFamU5fBXQ%QVb)25;fo?^}~fQ zT6B4lt3l`lfQx=n5&3|?k()rKsE;XuF`>bzeI!)Vp+vpbgcH~AJcJRUxu|uof83mM z73FcbJa2@tExT@eZ4F_Bl!cqCafGeEqG@VJ zLz$qh&t!v@3E$_SFn9T)2zpvY_cta*x=KC5Ww<$cn*({Tc^(S9&fwWMOf9&qwoGV+ zt087=%tU;)ap&=rF_-RWiPndNVehZ5&dD}wx*T_`$yYsRh(kJpd5c7+_7-0YTN($x z{6JY;&?uw=)!zZMB3{xTuT!s%m&l0?#-7h9s=K#*mfAR z*+v5jsF&3xxu~AvK08HmW4MKLLQT9{_&22`t{wl`S4$V?!`}al)d(O2->=VC(=gfdeN(qgs{K4 zY<}-a0SPwttCD1m*c6GFAF=T%W}0}kUHOSy5oN}A-dzh?8;y7NdzVPA7&U)d@)hO= zMQcofgP-dgjdazA&h_f@!<7dcQn1HaSXw3FjWx|H3{;U?RV5*5dZ%`DaMCUMC| zrCfL==H}e0mNxFfO3Q_3lZfu`zVOF|OJEWo=FZL@Qm@i-qiEt%N|eoa=j(Tww`_xC z4Rez&H^L7H#v`Z62)qPJUz{IVjlJLbOd&u^VBG_ef<5$GKgTGFh2YJ?NT)D@Y{%}9 zWWSxppD)M$d9~H1_e%LG%V4gz71R64*v;JWATsz|H)SxTipja>QG^Ikbc*U9O*m+N z#w}-L%{;eCus{E(?#2F!6gv(77scWvrds3I)@rX`C*0{Zzg5(F zS1oMd4ih#Q=fAlt`GL^&EZYos6$1#$B?%H(IGr} zb_z7q!4tX9&Z0Gk^*7IRzOMJ7@SiWOfo6XScgdcOv|)8!v@eJv7i|!~c_2PYxtkTc z__SAE`$0z+b&oh)9Fxdd8L z*XsqoO`f}V=?nSB4;o38a=-cX&jeJjETm$C>eK6=-cNhH9wd{F;50A$aQP=?2JX+B zpYKezvK$=rd+hV@y0HoP*@V`su1g=$#9AH3(8@S$wWYbcz$sqrFOhmjFlR{q8ME5z7yZPso{hzV~XBA6$HPuk5{lU}A%Gl+)BxcUfY3KmOqBbAl>iQLEEm!@O%tI65oSiovo-2&Bxx20!y_b~5{%HlfSU8a5NQ!79dtE(T|t5nH;>+I%UIKOz> zn_rOJ)>{OuZ;gTMaz;3D=e$9E#!USm!43uM9 z8=4e78rS2h80Zt~T(z-{nU;*8`L{4W1n!6ulu~FCbbT?)Q6=}2#VPBT480Nl_&ov1 zB?&BQ$((e<-ZUI7_eCPGvd=m%>}#{U(cv~~t3G9q`$m`Cadban|W5vW@vR6e^%tdimjX1)1Sta&cNuN|EIpq$u-i)(J zs!LzAxv1Sh@#75H9c_v=Po4WVb6PJv6mBb*hI3vP8cx?sw^EZ%9^2CG*punu7T>N3 zjW01{k4h`NV1j4PE?SE~sXA6OU%)k>=>`9z=^(z9sps#58KUDm@TVqX(vV!6!je(> zbmSRQtoCOl^0@aG)|aJEeM#OXiYs*U3OYHtdllnF}ErjADI>Q13`F&hAJ|#7$;Q;Ar<>>jhlStCB93O!~8e9_yPGW{(S%N`N5D3 zgIMXN(@r!L34>`JGznQx!xS(JGrow-lU_Bv!u z>yRZ7>PM*V9}=&Jxbkw#aW7vv7v|~p&5EO?I$VMIYss%RcVUW1Rs6V>PB z@Mcg(hi8G<$tzB&rAx6ek3gZcnU{8B`DxAV+Iks{8w}W>LLI(U^&Mf+$#VtB#rz)G0 z{cWu=cuCfJdtq!0OquSBYJHj)7HzPd^CU98VxC^-p2qsJvxM=pv%Y4eX;CH^dt6yf z@71hVocc5NY&iBK>ggAQmU=$k;V7{%|F|lrV4;=myPL{tX%KhIrtp)d$c&SO+Q6)C zp4clT~CsIs10^duy@?jSw zH!Fa$ztQJ*JwDv`x0xNvcCWGp3qtGEy(V+c4*7R02Uihsfpj&(;xF zq^u{T?E^~UUhFJB=bO;Ho2ste2jQ4!v^_5pp-FI!s>d2t%B#-Gy1D|+_VR~O!H_k|tNDFCl{YM2 zH!CR4dkipS4Mp7)7n3QDjOxkYnrS=Vttz>Ilfb3RlbpzPaLlhd?^RZ5}I zYq=l)ZTD0gOHS9#U3CNMS4PFnr8s+~juQlWvm7QOW+Gp3J~_csj2DcjzGp-XlRnI+ z*p7g4Ela1Ilf7M?BtS+l^5Uy4c;RdCD*@BILVIgsm0`g>J7??PgvbDI@$&C3Szv$+ z{~h(}ZQjGIj~Uy~aTp20+-Ud$$#0^O(^^Tfw|0X@sZwVxac{kjo;Pl;>CEcOj#R-d zT=GsPADNlW`*LJ&4cswBSL`0;Xx5EaAFS8UxNZMhf+{iGX4}tXY?JKRbadDfK^|Ql zsaRRS-ke9Af8^rOSl#0_bTqFGx3C_rd&YF;b7M=DB_$3*b}0A8*sI-lDR*Wx^%|eH z*ZX@mP-&V5X-%vfMZUdtnUa8=Qa|?N!%Ngkcg8}t<1ePZ`9;a(|4G`1>YHG2)E$n6q}y`|t* zM2NtGx1^?j~M-Uukc2nIn`oPjv?CcdqiPn5c#EQOQ?3 zTzjUemq8Hupp=8p@|5(4u%`5cv;*zr^L5>pozJ!|@G!32tWVY@P=A^(cqRMos9d-3 z6}JTrSLz|EijN+Qk~GCZl<9bz8Gi_5upKG+cq%IAeAGIBjbL^*`J zgjIxy6Nq9juXLUv%De^-!FeTI{W3f#J)3s>dx?#@yZB>OgNW>LoNc2F^8YW3i) zyTZSx%+Na!mHJ>EQgQ+I3VmK4`7s&P`Uv3#IAQ&{C0I4u26vWWHV7kn6Pi$>aQ^Tj zUCex(cMl&1OJuRVt7GZv9x6Wl8e(xZx!22j>HUVi-r0fQl!qA8p0}4H7EDTp73Xsl z+dfD4yJ^5ueX|oXdVcsT=6=oIZQgt)^!vxUAI3GaxP0=wu7-U%nu>0cE@}~rP9H+a zTfP3mI^y};^mpd8b(|Xzv>GLuUrhMiSQ-68AJ6%U6-!JFx|^z;XOXdaR;S1_ASK+u zPofzl&Eu|c{h4RoPk+lYPOlI7h4QPHimP3CW#2R7vWK<{8q~8$7A9A4E~Ps37=mNl zf!#huTC(<~98M`cO+%8+1>X4dEjE;?OB46l3EvnEojItx_DI949C13;w@`EUw`tN# zO_MZB%-*v1I2Iv%YZ#8|mtX>AWu(hJZkdzCbLno=_uoI19=v=ZPpY`Tp~bk&;+x&Z z4!Jf}$fTFkPM=5$r{_5CRkeT-p)*dOZe7R~z8^nPk@6+v@}usCO`mwKNUqOgb<0`r z-o;-c=#`kdc7E&Qk3QHFgj(ezerZ>1DS3x37TPPT%gU1SKEbyiCD6WIX}Bv*_*I6g z>WuKt*}2gt;ckpGS1)zOCTv#mx_|C%AO3Cz}b zUaL;8`HH(v-^Z|%opELXn^CdA{Qg3`9|=t<<1U`>ljLhY%Y|(@hdc;Zta%@q-7(3c zk7DF21rbkIFXD9TJ76=9^YV547!6r!o4q7GVh<%m2=af%ehVAMJM*o?kSEM`RBX{L zV1x8&QiZ^!eREMs>w6zru1EcTCUKxN=#@SEjThoHHZrU7Q8tAs$B@@?lQnDQZ|VlV zOH01{eEq)R>5b=9*PBDv5RUFH=LU@e9xS@VyV6|}$|I&%{#g_6Hz7i)NleIa+sc-x z>Bko_eEH=90pIVmiOSePlFwiVp729{i48$dJ9nz=tjN%?_xPjo3=i_%NjE)5_)af( zarVJBO>@*v-@%#qHGxU@wDZ96CoUg$^xA{Z#HywaxvXcO61(nhRl*VCeg$^SkISq3 z-RDGWCE4VYF6=IC%ze#(*VN|G|Dar=S4!WrI^$T0<4cT~W-q^}Gt1R@>MMgF3)yz| zCp@!HZV{PLsq)IO>lz)4QQaLri;kNEUT~vV)%6tbOa|_?i{F`S(8?SRPhA zB#s`7(BEP;C8YU8%bcI*;FqJ!z-%;qZsuZz0^#jOUy2Nz!PCO8-V-pYJ)W1==_l0+ z%zv(`+WB!=+uK>=_nm2TI)b`}5owqu?rp``U8$GLQ|;=@on7b&=cNyrSG$3mK>XKD z9NT#CUkB?eQP%WhjE4#$L^0J#c4BfJ(J}0}F5?L~g%YgoPun=IYQ`(43@m)Zp!HDK z=Vo}u_d!h8Z`FGkzp_O^e*xdK42dAko zZ=L_*Z^StrU$5a`9$w#AMSa(#<92;e+m;^nbc7%NygP&8=k2ZIsjrv$$X8 zZj&i2D$6Tagx!taRqtB3CK$|+m$;=#o%tm_Tp!mn5HW!zq_EK8@I@LokGiS#LRezr zz8bE+s6QJbIGNtUD4;dJTrOjT@N}d3a098%nH~;((bTAerY2=lHlo#TW0s6`IUiP% zcO}IQ9+COHWGe(}%Fo*e_S!Buo_TSBx#j*_zOe8DT`U3x+E)*0G`>)9Q#pt$6!F-< zo+aRpBdd4an~?MfEB}f5qSGs zQHC*lf0qxp-XETurh3;vJFwJnR^I9Kh?C0Lyz?#cy!K(T^9`aYPh`_)@$wzU8~NgN z_0<)}pDVfE6;-FWmZk6e@H$+}!N-!}@(Qv2=&t(yt$fECAyFpAwP=FKK9L%V?sFpF z#~bLDtE^TLLua`7@8fRuxvNumT25Zsc&DQ$GDN!+^M1fWN93{ppkMb7#FTP|bk2}H z&5r9$p5!5MJ2uvveCeQvaudCvLu$er&- zk>;_YA|o!{2SZabpVqr*Ch?+dUtP@8`v~6J6-zCsYfC8DnXxEvuj+f%?)FPoC1uQy zoRr{M^;#f8OI0OiOr6xiKL5u0sc21~%$J6^)1~z7-=d1cMP*5iGV6z?gR|RL1sF)j zJKLRZ^NEPwPHN=mrV?YejOBG}{;}sl7U;rPQpY>II>mj;@4=W~b@x*}gyFy@`#$Vf zS%I$Zd!A4>_32{w>4NVK1%a#noagOG}mwWVXwd(%KKlsFh6)Ps|6 zbe_aG_=EAK-a@A{UY~7sFQzTLag$T4EcxKC(QU&2isl7LXK9XY{sn)lZrU16v(zsH z6`E$rPvR+Q*BeYJ}euYxj+Z zk6oU9#_L|P`$^Rb?pkqNpxroQHGp3jd*gk?%mVRejvmAUC47y!^rnu7)@r1#ruoC= zjh-xphc5}wzc|xS-)5^^>plEdCH9kH;qu13)q8HNx3yXyXktvhk4tyWGG6BA?oqWlHkT?vd+09#I)??{>E>*Sg2GY;3HRp>2C9WU#QH&qCYq zfro2sQ(9MMFQUH6PnOqhh^t^K{cd@|*?U|IP*2aWY-WpR+mo=(4XUlcZKVhFB8CN6 zJR-32CXDhc{E~Z*6#B6;Cz~C3dU~zsoIP#N>)s}*xm=cVrjLii^u>KY9fr|QXEEtz z*oraqnMJ1R4DNp)UeambF<4orQZBC+KK)(wmbro2*rhZ?b79f-0XwGZTS3G3*&{=( z`xNAJ>~4k%&m_VsHr#*S+j5{qw9nYdoym;HYsxjR9$J`aR_KJ=Tvxc?U8qx`Vj`XO zL36KU&*0&z7|-@*EcdN{_zg{jJ#*BrR zFn5QCZyN?q>vc1B3;H{4cEY`f((Tn~R0bC)`&2kq9( z)oS6e^dZ92Ic$i4k|%dbz-t}yS8QA~UkIOH{#BsM{ootZ1H{lu^&0Or@0GgE)viT` zB6J`6`HvjYos^$-ioxdIyzv&A!yJr1-9icD5sAu~k>J=j&c<*iA zx+9WS6GsZW0Bo%(%7%?Pi?2iea#vz3%1{ji0O%tA!9>UZhoNQmbu%ORw8`T_Rg^lfxX(jE}W_a5ud|=P`L(z{ z@5oe@zhOAX4vZjiwd-&W6bYHK{u~^B`JY$_34jJp!9SnOd=m@qibkQ1m!MFsCgAQ- zAvT-`PWZg+rB0jElk5ht^W6(g;K8M3{#+La6x2U(dEnT;-+-*EW~?NjsdrxJf;J|M z0)qkr2L}fO*@n9KaC%Hh$O#|Lgu2O`M*^pZ`NC5?;EYfs5uAYKzp{CcNyflvM9xaC*U|(|KJk@ zaB@`Pwm>#8YGR0x5YB?S!!Aq+2$zTA~+4-f9j69 zVgIMeXiwzcfJ|@(B_MzZWPwQv>4X2?BL<%u)ggfs{Jo(6CpjM^L=2aM5&;jDij^3o zdi*yPS;|cpEXbW0aHYXreFTzANcKl2iknPE2F)S?s8Ry}wIxt^H#_hD=`vQ(MVLvD zJopa^Y>w=*Udj|qI+}k4K=mUz6(BOA@?}BZA#s*bza0gp0!3gdAaJ6>R!AfV#6t?F zMj<6nVKrrf%4vZkUZ8J|{X0hokOI&hQczSN6Urt9xr;HPGI|G+qljBbA`cUa$V&$2 zMaj$+zj(SB$UFr1TjY*O#K0h8Ln2ewSXrSTyhn1|$$$tZP88!O2P?!te#GD-2aIdn zM=5BbP;xjKO1{lyoyY5J7#R2l7#JKUWmj|?Edw_~_ZWRwL}QNYill7B`f ze{GLWUTRPHuV&{cku`HhbACAyLNffo&XDDL`TT)=6>xP4^#2uH;(~%hLzG7?yoMC= z$OqL5cYR19TJ9qmy)r%WV=NdLqG}iz>?aBr9Dr(tn^bTLRBp|ERT z&mmSC_!<7=mWgbLJ>}H{JWLFXD)5s$QHeuT3N=&{JJA3UZmlR0PzCq{MSY8;QnaI~ zq_juv#YYRMjvZ*KJ1v|Rg^fp2zjmUiDO)dypc&dDE!#kHLc7qMpl-4g&I4J*&^hmC z2cT&ApolUjibx2#)4?@S@|)>^6!%wXDXu*<5H0;tH{o_WUgiTy55Q0UL~_>GXgJdw zY7`vRTKbYeQ2iSSTxaF{pQ2n759zf8`X~c#&MKUcg?Enuu8R_IfdL2@9z+R%+8E(j zC))hKPX1)C@eTb;2CXx|(FZ#*8G*#;<17tExELxkl=ovE{hJEi*i*;^h<{Wr;_ z{XYZDNsGEMfs%s!3^eqZ>8Nv>nSiA3zZ^D*g&A;=S|!8`IEqs!eRN`mqhH(zLK5$w ziGRKpO@-3wN1OPv^`OO;!B6N!%W=)2WB@n|3aL}Jsuu+M>KkxiXHOty=21xSb!l!C z^7vP!PP$IoMI|04;?m!LFc3w)FscI`Wgp}Rv4$+I5Sm-gc>64YRsJ9;2vmXC!o;hPBe6cFgjTp2N;@>jnT#diWnn?rouUoY?_%9P^o2* zRDK9o7L)8>^TZ%o&t%c8$8wnT&^qT)%l$;M0u}$V@F0CII2p>Fxl_dH8-cDL0;Li@ zQ6f#~CKp^0RogPZ#7b!Zx&uHb1D}0mw9qvbOd^O_`^f&ysm^n70{kKP37!y|p^C(F zLD#wARH*!KEF-$w0gMRvYULA9I_N$(+yJHc$S?&{8zMOa7e*y&TC~Hm1Z61!iB9^2 zY$S*S0xt(KLif)cRlo8KsJ@*xB*Ft4ZSM@67FDTolX}%T5HBQyctPw$&e?iMs1gf! zGe_srIabRzpnS+1aH1zr(}ob~88|xx9vCA1AFuTwKYe)=BpU=7ob-UW&=(#!`mvdF zyr5i~rf4xYdEs=Zj?Y0-i7un5;0dBLsM1>4dc+{(3ri5Q$et+26^p-6)N4g#C@CI% zK*|kkB#Q&8vcsf6zEy++_3<6G$p<8#^6D|ZDF2ahXz~MAx9wjR=m^k~s8UoCJh^lS zH2NpJ|LSkIGrAO&*D;y@wLGf;5Y>JI&FTw4znFBa6nBt(cCTanzoqzrWX<>-V}S>b z&|~}i@PnI~z=F02{u^t3^#6;eN0$Qm7*xL?5cN3}%_0_pbD-)%cjIf(1n8o5&_${z zTGT)6FO(2+yM;;huK~!01LCMa1H1-t{rMA#Uff0}dK`|)`L6*OghAGN(P$QUSBVp) z&nWjdcNfYRh6|z;+N!_M1Odd7gJ|Vs?}|Y8g@KXt zq=G+Cc`=9qs!}zYYAy!mS4f`~jHE_4BB|`ihnmo;i{~at`wpnB7V&>gsPfrgC_ZG! zbQJ2Et_zF8BHvZUw&ak%=Xwu&jmWhL0>AKJ?@TSvgXj2@{h$;}V1Yi}v zM6>iHj;2i>Na}-LNa-w`6{?gt((U6J(jXXx9fNoCm*?CA>F+ zl&XS4y|Xlk@C9|y1G;k3!DzfgqJ$tB$)hS8Xi_-G13laa{V#o@%FL7KVsc4>@eUaT zpn@f`86=en^+E^nf8tB%J~EuLJ3@drKnS2E$s?b4?}@o-04Q|m|9CUcIkZH8M~`s# zeY<8UL6HuCp`C0F?FBS6SL$e}>X8B&c7OU;B&cc$jOzl@M=B;K4fs{d$N1p25wynS zAz23N$5`OK5t9E_>7EQg(e4~W!FPcUBj+rrmi{KZPmlx3Sr0Vq-T zcpZTi5>PlQxqzD#+YFG+24tU{%hZ#irR0#osQyi=JgmMIB8L`xP+Rd6HT_P8#)C%| zXrb5gN8LFt53>G6iDr>29K~UB3V^Cghom(PlUmouiiXN5f^{OKA)uTnf*na^`=3DTkwzm03$Qz0Dhv$86LsoG zr+}I`VCaDg)!SLDr4S6@i%I`GC$kVh*Uk@+6Kk9@Krg;oFJ$^OM* z$M18V#9vfLOj|^~lXIr~zuF6PjwJ24_a)eiKQhxsIlG04hoq z)O|5pq1 zf}wo%qv*Y0lKAU1u&E26LMQFpJs;ij;C>$kgsXcbHR9F7{6kQBWDI3u+-V^p}bafuq7?(1$;2ar~6yNl|^n=<!hgu_t2moTov8fmRiC3z!Kge{rCwVZZL(E$_|<7!zodg?5+>;+M7f7 z3pfpiR?I#m_0%Uc6`Tb_&-;!a8PPo^zkxvVf6}xjR71+i>2iJ^-fK{@4j0JXb(5zr2 z>*?Mx)^o!nE&PCF{rvHlMGXlW9Th{~2(Z)+{<5$ku){+H#sA@FA^9~wkMY5759Qx= zoO{NAz5V+iJ8d1TonA0Ll5P#ya2-r|N*FH#1a^Tap^87O=SbEl49)tpBSZobnH(i$ kGy(i?@MC Date: Tue, 24 Sep 2019 11:20:54 -0700 Subject: [PATCH 70/78] Fix Zstd compression level documentation Patch by Romain Hardouin; Reviewed by Dinesh Joshi for CASSANDRA-15296 --- doc/source/operating/compression.rst | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/doc/source/operating/compression.rst b/doc/source/operating/compression.rst index 42a057b242d6..b4308b31a3eb 100644 --- a/doc/source/operating/compression.rst +++ b/doc/source/operating/compression.rst @@ -38,7 +38,9 @@ default, three options are relevant: - ``chunk_length_in_kb`` specifies the number of kilobytes of data per compression chunk. The default is 64KB. - ``crc_check_chance`` determines how likely Cassandra is to verify the checksum on each compression chunk during reads. The default is 1.0. -- ``compression_level`` is only applicable for ``ZstdCompressor`` and accepts values between ``-131072`` and ``2``. +- ``compression_level`` is only applicable for ``ZstdCompressor`` and accepts values between ``-131072`` and ``22``. + The lower the level, the faster the speed (at the cost of compression). Values from 20 to 22 are called + "ultra levels" and should be used with caution, as they require more memory. The default is 3. Users can set compression using the following syntax: From aaa4a602ef1f658bd3898ceeedc3dc1f0fdc4fcd Mon Sep 17 00:00:00 2001 From: James Berragan Date: Tue, 20 Aug 2019 14:47:26 -0700 Subject: [PATCH 71/78] Extract an AbstractCompactionController to allow for custom implementations Patch by James Berragan; reviewed by marcuse for CASSANDRA-15286 --- CHANGES.txt | 1 + .../db/AbstractCompactionController.java | 61 +++++++++++++++++++ .../db/compaction/CompactionController.java | 37 +++-------- .../db/compaction/CompactionIterator.java | 14 ++--- 4 files changed, 77 insertions(+), 36 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/AbstractCompactionController.java diff --git a/CHANGES.txt b/CHANGES.txt index 96be055a2ded..a90144186ae0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 4.0-alpha2 + * Extract an AbstractCompactionController to allow for custom implementations (CASSANDRA-15286) * Move chronicle-core version from snapshot to stable, and include carrotsearch in generated pom.xml (CASSANDRA-15321) * Untangle RepairMessage sub-hierarchy of messages, use new messaging (more) correctly (CASSANDRA-15163) * Add `allocate_tokens_for_local_replication_factor` option for token allocation (CASSANDRA-15260) diff --git a/src/java/org/apache/cassandra/db/AbstractCompactionController.java b/src/java/org/apache/cassandra/db/AbstractCompactionController.java new file mode 100644 index 000000000000..99193f8626a5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/AbstractCompactionController.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db; + +import java.util.function.LongPredicate; + +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.schema.CompactionParams; + +/** + * AbstractCompactionController allows custom implementations of the CompactionController for use in tooling, without being tied to the SSTableReader and local filesystem + */ +public abstract class AbstractCompactionController implements AutoCloseable +{ + public final ColumnFamilyStore cfs; + public final int gcBefore; + public final CompactionParams.TombstoneOption tombstoneOption; + + public AbstractCompactionController(final ColumnFamilyStore cfs, final int gcBefore, CompactionParams.TombstoneOption tombstoneOption) + { + assert cfs != null; + this.cfs = cfs; + this.gcBefore = gcBefore; + this.tombstoneOption = tombstoneOption; + } + + public abstract boolean compactingRepaired(); + + public String getKeyspace() + { + return cfs.keyspace.getName(); + } + + public String getColumnFamily() + { + return cfs.name; + } + + public Iterable shadowSources(DecoratedKey key, boolean tombstoneOnly) + { + return null; + } + + public abstract LongPredicate getPurgeEvaluator(DecoratedKey key); +} diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java index 59bba0a5cc8b..e1b0f3258359 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java @@ -19,26 +19,21 @@ import java.util.*; import java.util.function.LongPredicate; -import java.util.function.Predicate; - -import org.apache.cassandra.config.Config; -import org.apache.cassandra.db.Memtable; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; import com.google.common.base.Predicates; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.RateLimiter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.db.*; import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.schema.CompactionParams.TombstoneOption; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.db.*; import org.apache.cassandra.utils.AlwaysPresentFilter; import org.apache.cassandra.utils.OverlapIterator; import org.apache.cassandra.utils.concurrent.Refs; @@ -48,13 +43,12 @@ /** * Manage compaction options. */ -public class CompactionController implements AutoCloseable +public class CompactionController extends AbstractCompactionController { private static final Logger logger = LoggerFactory.getLogger(CompactionController.class); private static final String NEVER_PURGE_TOMBSTONES_PROPERTY = Config.PROPERTY_PREFIX + "never_purge_tombstones"; static final boolean NEVER_PURGE_TOMBSTONES = Boolean.getBoolean(NEVER_PURGE_TOMBSTONES_PROPERTY); - public final ColumnFamilyStore cfs; private final boolean compactingRepaired; // note that overlapIterator and overlappingSSTables will be null if NEVER_PURGE_TOMBSTONES is set - this is a // good thing so that noone starts using them and thinks that if overlappingSSTables is empty, there @@ -64,11 +58,8 @@ public class CompactionController implements AutoCloseable private final Iterable compacting; private final RateLimiter limiter; private final long minTimestamp; - final TombstoneOption tombstoneOption; final Map openDataFiles = new HashMap<>(); - public final int gcBefore; - protected CompactionController(ColumnFamilyStore cfs, int maxValue) { this(cfs, null, maxValue); @@ -82,13 +73,10 @@ public CompactionController(ColumnFamilyStore cfs, Set compacting public CompactionController(ColumnFamilyStore cfs, Set compacting, int gcBefore, RateLimiter limiter, TombstoneOption tombstoneOption) { - assert cfs != null; - this.cfs = cfs; - this.gcBefore = gcBefore; + super(cfs, gcBefore, tombstoneOption); this.compacting = compacting; this.limiter = limiter; compactingRepaired = compacting != null && compacting.stream().allMatch(SSTableReader::isRepaired); - this.tombstoneOption = tombstoneOption; this.minTimestamp = compacting != null && !compacting.isEmpty() // check needed for test ? compacting.stream().mapToLong(SSTableReader::getMinTimestamp).min().getAsLong() : 0; @@ -246,16 +234,6 @@ public static Set getFullyExpiredSSTables(ColumnFamilyStore cfSto return getFullyExpiredSSTables(cfStore, compacting, overlapping, gcBefore, false); } - public String getKeyspace() - { - return cfs.keyspace.getName(); - } - - public String getColumnFamily() - { - return cfs.name; - } - /** * @param key * @return a predicate for whether tombstones marked for deletion at the given time for the given partition are @@ -263,6 +241,7 @@ public String getColumnFamily() * containing his partition and not participating in the compaction. This means there isn't any data in those * sstables that might still need to be suppressed by a tombstone at this timestamp. */ + @Override public LongPredicate getPurgeEvaluator(DecoratedKey key) { if (NEVER_PURGE_TOMBSTONES || !compactingRepaired() || cfs.getNeverPurgeTombstones()) diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java index 1c56a87bc82d..789d1eeeb5c8 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java @@ -57,7 +57,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte private static final long UNFILTERED_TO_UPDATE_PROGRESS = 100; private final OperationType type; - private final CompactionController controller; + private final AbstractCompactionController controller; private final List scanners; private final ImmutableSet sstables; private final int nowInSec; @@ -77,13 +77,13 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte private final UnfilteredPartitionIterator compacted; private final ActiveCompactionsTracker activeCompactions; - public CompactionIterator(OperationType type, List scanners, CompactionController controller, int nowInSec, UUID compactionId) + public CompactionIterator(OperationType type, List scanners, AbstractCompactionController controller, int nowInSec, UUID compactionId) { this(type, scanners, controller, nowInSec, compactionId, ActiveCompactionsTracker.NOOP); } @SuppressWarnings("resource") // We make sure to close mergedIterator in close() and CompactionIterator is itself an AutoCloseable - public CompactionIterator(OperationType type, List scanners, CompactionController controller, int nowInSec, UUID compactionId, ActiveCompactionsTracker activeCompactions) + public CompactionIterator(OperationType type, List scanners, AbstractCompactionController controller, int nowInSec, UUID compactionId, ActiveCompactionsTracker activeCompactions) { this.controller = controller; this.type = type; @@ -259,14 +259,14 @@ public String toString() private class Purger extends PurgeFunction { - private final CompactionController controller; + private final AbstractCompactionController controller; private DecoratedKey currentKey; private LongPredicate purgeEvaluator; private long compactedUnfiltered; - private Purger(CompactionController controller, int nowInSec) + private Purger(AbstractCompactionController controller, int nowInSec) { super(nowInSec, controller.gcBefore, controller.compactingRepaired() ? Integer.MAX_VALUE : Integer.MIN_VALUE, controller.cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones(), @@ -510,10 +510,10 @@ private DeletionTime updateOpenDeletionTime(DeletionTime openDeletionTime, Unfil */ private static class GarbageSkipper extends Transformation { - final CompactionController controller; + final AbstractCompactionController controller; final boolean cellLevelGC; - private GarbageSkipper(CompactionController controller) + private GarbageSkipper(AbstractCompactionController controller) { this.controller = controller; cellLevelGC = controller.tombstoneOption == TombstoneOption.CELL; From 2511785c2a8b66edac2d9959912edb7d4b7062ed Mon Sep 17 00:00:00 2001 From: Josh McKenzie Date: Sun, 13 Oct 2019 14:00:00 -0400 Subject: [PATCH 72/78] Remove inheritance for CLSM, break compose through Allocation The original CDC implementation leaned on inheritance on multiple CLSM with an abstract base. This refactor changes that approach to isntead use composition with the segment allocation being the axis of composability. Also tidies up some scope to package private while in the area. --- .../cassandra/db/commitlog/CommitLog.java | 16 +- .../db/commitlog/CommitLogReplayer.java | 4 +- .../db/commitlog/CommitLogSegment.java | 44 +++--- .../commitlog/CommitLogSegmentAllocator.java | 55 +++++++ ...java => CommitLogSegmentAllocatorCDC.java} | 81 ++++++---- .../CommitLogSegmentAllocatorStandard.java | 90 +++++++++++ ...ager.java => CommitLogSegmentManager.java} | 148 +++++++++--------- .../CommitLogSegmentManagerStandard.java | 68 -------- .../db/commitlog/CompressedSegment.java | 4 +- .../db/commitlog/EncryptedSegment.java | 4 +- .../db/commitlog/FileDirectSegment.java | 2 +- .../db/commitlog/MemoryMappedSegment.java | 2 +- .../cassandra/metrics/CommitLogMetrics.java | 4 +- .../db/commitlog/CommitLogStressTest.java | 2 +- .../apache/cassandra/cql3/OutOfSpaceTest.java | 2 +- .../db/commitlog/CommitLogCQLTest.java | 4 +- ... => CommitLogSegmentAllocatorCDCTest.java} | 33 ++-- .../CommitLogSegmentBackpressureTest.java | 10 +- .../cassandra/db/commitlog/CommitLogTest.java | 16 +- 19 files changed, 340 insertions(+), 249 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocator.java rename src/java/org/apache/cassandra/db/commitlog/{CommitLogSegmentManagerCDC.java => CommitLogSegmentAllocatorCDC.java} (79%) create mode 100644 src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorStandard.java rename src/java/org/apache/cassandra/db/commitlog/{AbstractCommitLogSegmentManager.java => CommitLogSegmentManager.java} (80%) delete mode 100644 src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java rename test/unit/org/apache/cassandra/db/commitlog/{CommitLogSegmentManagerCDCTest.java => CommitLogSegmentAllocatorCDCTest.java} (95%) diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java index 71fc1aaf2688..34f2f12bc614 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java @@ -67,7 +67,7 @@ public class CommitLog implements CommitLogMBean // empty segments when writing large records final long MAX_MUTATION_SIZE = DatabaseDescriptor.getMaxMutationSize(); - final public AbstractCommitLogSegmentManager segmentManager; + final public CommitLogSegmentManager segmentManager; public final CommitLogArchiver archiver; final CommitLogMetrics metrics; @@ -108,9 +108,7 @@ private static CommitLog construct() throw new IllegalArgumentException("Unknown commitlog service type: " + DatabaseDescriptor.getCommitLogSync()); } - segmentManager = DatabaseDescriptor.isCDCEnabled() - ? new CommitLogSegmentManagerCDC(this, DatabaseDescriptor.getCommitLogLocation()) - : new CommitLogSegmentManagerStandard(this, DatabaseDescriptor.getCommitLogLocation()); + segmentManager = new CommitLogSegmentManager(this, DatabaseDescriptor.getCommitLogLocation()); // register metrics metrics.attach(executor, segmentManager); @@ -288,7 +286,7 @@ public CommitLogPosition add(Mutation mutation) throws CDCWriteException } catch (IOException e) { - throw new FSWriteError(e, segmentManager.allocatingFrom().getPath()); + throw new FSWriteError(e, segmentManager.getActiveSegment().getPath()); } } @@ -308,7 +306,7 @@ public void discardCompletedSegments(final TableId id, final CommitLogPosition l // flushed CF as clean, until we reach the segment file containing the CommitLogPosition passed // in the arguments. Any segments that become unused after they are marked clean will be // recycled or discarded. - for (Iterator iter = segmentManager.getActiveSegments().iterator(); iter.hasNext();) + for (Iterator iter = segmentManager.getUnflushedSegments().iterator(); iter.hasNext();) { CommitLogSegment segment = iter.next(); segment.markClean(id, lowerBound, upperBound); @@ -364,7 +362,7 @@ public String getRestorePrecision() public List getActiveSegmentNames() { - Collection segments = segmentManager.getActiveSegments(); + Collection segments = segmentManager.getUnflushedSegments(); List segmentNames = new ArrayList<>(segments.size()); for (CommitLogSegment seg : segments) segmentNames.add(seg.getName()); @@ -380,7 +378,7 @@ public List getArchivingSegmentNames() public long getActiveContentSize() { long size = 0; - for (CommitLogSegment seg : segmentManager.getActiveSegments()) + for (CommitLogSegment seg : segmentManager.getUnflushedSegments()) size += seg.contentSize(); return size; } @@ -395,7 +393,7 @@ public long getActiveOnDiskSize() public Map getActiveSegmentCompressionRatios() { Map segmentRatios = new TreeMap<>(); - for (CommitLogSegment seg : segmentManager.getActiveSegments()) + for (CommitLogSegment seg : segmentManager.getUnflushedSegments()) segmentRatios.put(seg.getName(), 1.0 * seg.onDiskSize() / seg.contentSize()); return segmentRatios; } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java index 2947222e4d93..ca8eb4f123d4 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java @@ -167,8 +167,8 @@ public void replayFiles(File[] clogs) throws IOException */ private void handleCDCReplayCompletion(File f) throws IOException { - // Can only reach this point if CDC is enabled, thus we have a CDCSegmentManager - ((CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager).addCDCSize(f.length()); + // Can only reach this point if CDC is enabled, thus we have a CDC Allocator + ((CommitLogSegmentAllocatorCDC)CommitLog.instance.segmentManager.segmentAllocator).addCDCSize(f.length()); File dest = new File(DatabaseDescriptor.getCDCLogLocation(), f.getName()); diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java index 5303de9da40e..aa771a1fcfc3 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java @@ -65,7 +65,7 @@ public enum CDCState FORBIDDEN, CONTAINS } - Object cdcStateLock = new Object(); + final Object cdcStateLock = new Object(); private final static AtomicInteger nextId = new AtomicInteger(1); private static long replayLimitId; @@ -81,7 +81,7 @@ public enum CDCState } // The commit log entry overhead in bytes (int: length + int: head checksum + int: tail checksum) - public static final int ENTRY_OVERHEAD_SIZE = 4 + 4 + 4; + static final int ENTRY_OVERHEAD_SIZE = 4 + 4 + 4; // The commit log (chained) sync marker/header size in bytes (int: length + int: checksum [segmentId, position]) static final int SYNC_MARKER_SIZE = 4 + 4; @@ -123,14 +123,14 @@ public enum CDCState final FileChannel channel; final int fd; - protected final AbstractCommitLogSegmentManager manager; + protected final CommitLogSegmentManager manager; ByteBuffer buffer; private volatile boolean headerWritten; public final CommitLogDescriptor descriptor; - static CommitLogSegment createSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + static CommitLogSegment createSegment(CommitLog commitLog, CommitLogSegmentManager manager) { Configuration config = commitLog.configuration; CommitLogSegment segment = config.useEncryption() ? new EncryptedSegment(commitLog, manager) @@ -160,7 +160,7 @@ static long getNextId() /** * Constructs a new segment file. */ - CommitLogSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + CommitLogSegment(CommitLog commitLog, CommitLogSegmentManager manager) { this.manager = manager; @@ -265,10 +265,10 @@ private int allocate(int size) } } - // ensures no more of this segment is writeable, by allocating any unused section at the end and marking it discarded + // Ensures no more of this segment is writeable, by allocating any unused section at the end and marking it discarded. void discardUnusedTail() { - // We guard this with the OpOrdering instead of synchronised due to potential dead-lock with ACLSM.advanceAllocatingFrom() + // We guard this with the OpOrdering instead of synchronised due to potential dead-lock with CLSM.switchToNewSegment() // Ensures endOfBuffer update is reflected in the buffer end position picked up by sync(). // This actually isn't strictly necessary, as currently all calls to discardUnusedTail are executed either by the thread // running sync or within a mutation already protected by this OpOrdering, but to prevent future potential mistakes, @@ -383,7 +383,7 @@ synchronized void sync(boolean flush) * in shared / memory-mapped buffers reflects un-synced data so we need an external sentinel for clients to read to * determine actual durable data persisted. */ - public static void writeCDCIndexFile(CommitLogDescriptor desc, int offset, boolean complete) + static void writeCDCIndexFile(CommitLogDescriptor desc, int offset, boolean complete) { try(FileWriter writer = new FileWriter(new File(DatabaseDescriptor.getCDCLogLocation(), desc.cdcIndexFileName()))) { @@ -425,7 +425,7 @@ protected static void writeSyncMarker(long id, ByteBuffer buffer, int offset, in abstract void flush(int startMarker, int nextMarker); - public boolean isStillAllocating() + boolean hasRoom() { return allocatePosition.get() < endOfBuffer; } @@ -445,7 +445,7 @@ void discard(boolean deleteFile) /** * @return the current CommitLogPosition for this log segment */ - public CommitLogPosition getCurrentCommitLogPosition() + CommitLogPosition getCurrentCommitLogPosition() { return new CommitLogPosition(id, allocatePosition.get()); } @@ -469,7 +469,7 @@ public String getName() /** * @return a File object representing the CDC directory and this file name for hard-linking */ - public File getCDCFile() + File getCDCFile() { return new File(DatabaseDescriptor.getCDCLogLocation(), logFile.getName()); } @@ -477,7 +477,7 @@ public File getCDCFile() /** * @return a File object representing the CDC Index file holding the offset and completion status of this segment */ - public File getCDCIndexFile() + File getCDCIndexFile() { return new File(DatabaseDescriptor.getCDCLogLocation(), descriptor.cdcIndexFileName()); } @@ -539,7 +539,7 @@ protected void internalClose() } } - public static void coverInMap(ConcurrentMap map, K key, int value) + private static void coverInMap(ConcurrentMap map, K key, int value) { IntegerInterval i = map.get(key); if (i == null) @@ -561,7 +561,7 @@ public static void coverInMap(ConcurrentMap map, K key, i * @param startPosition the start of the range that is clean * @param endPosition the end of the range that is clean */ - public synchronized void markClean(TableId tableId, CommitLogPosition startPosition, CommitLogPosition endPosition) + synchronized void markClean(TableId tableId, CommitLogPosition startPosition, CommitLogPosition endPosition) { if (startPosition.segmentId > id || endPosition.segmentId < id) return; @@ -576,7 +576,7 @@ public synchronized void markClean(TableId tableId, CommitLogPosition startPosit private void removeCleanFromDirty() { // if we're still allocating from this segment, don't touch anything since it can't be done thread-safely - if (isStillAllocating()) + if (hasRoom()) return; Iterator> iter = tableClean.entrySet().iterator(); @@ -619,9 +619,9 @@ public synchronized Collection getDirtyTableIds() */ public synchronized boolean isUnused() { - // if room to allocate, we're still in use as the active allocatingFrom, + // if room to allocate, we're still in use as the active segment, // so we don't want to race with updates to tableClean with removeCleanFromDirty - if (isStillAllocating()) + if (hasRoom()) return false; removeCleanFromDirty(); @@ -640,7 +640,7 @@ public boolean contains(CommitLogPosition context) } // For debugging, not fast - public String dirtyString() + String dirtyString() { StringBuilder sb = new StringBuilder(); for (TableId tableId : getDirtyTableIds()) @@ -656,7 +656,7 @@ public String dirtyString() abstract public long onDiskSize(); - public long contentSize() + long contentSize() { return lastSyncedOffset; } @@ -677,7 +677,7 @@ public int compare(File f, File f2) } } - public CDCState getCDCState() + CDCState getCDCState() { return cdcState; } @@ -686,7 +686,7 @@ public CDCState getCDCState() * Change the current cdcState on this CommitLogSegment. There are some restrictions on state transitions and this * method is idempotent. */ - public void setCDCState(CDCState newState) + void setCDCState(CDCState newState) { if (newState == cdcState) return; @@ -748,7 +748,7 @@ void awaitDiskSync(Timer waitingOnCommit) /** * Returns the position in the CommitLogSegment at the end of this allocation. */ - public CommitLogPosition getCommitLogPosition() + CommitLogPosition getCommitLogPosition() { return new CommitLogPosition(segment.id, buffer.limit()); } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocator.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocator.java new file mode 100644 index 000000000000..883b825be13e --- /dev/null +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.commitlog; + +import java.io.File; + +import org.apache.cassandra.db.Mutation; + +public interface CommitLogSegmentAllocator +{ + void start(); + void shutdown(); + + /** + * Indicates that a segment file has been flushed and is no longer needed. This can perform blocking disk + * operations so use with caution in critical path. + * + * @param segment segment to be discarded + * @param delete whether or not the segment is safe to be deleted. + */ + void discard(CommitLogSegment segment, boolean delete); + + /** + * Allocate a segment. This is always expected to succeed so should throw some form of exception on failure to + * allocate; if you can't allocate a CLS, you can no longer write and the node is in a bad state. + */ + CommitLogSegment.Allocation allocate(Mutation mutation, int size); + + /** + * Hook to allow segment managers to track state surrounding creation of new segments. This method is called + * on a separate segment management thread instead of the critical path so longer-running operations are acceptable. + */ + CommitLogSegment createSegment(); + + /** + * When segments complete replay, the allocator has a hook to take action at that time. + */ + void handleReplayedSegment(final File file); +} diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java similarity index 79% rename from src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java rename to src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java index bdd4f74ae1e6..4f696c5d4175 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java @@ -39,28 +39,27 @@ import org.apache.cassandra.utils.DirectorySizeCalculator; import org.apache.cassandra.utils.NoSpamLogger; -public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager +public class CommitLogSegmentAllocatorCDC implements CommitLogSegmentAllocator { - static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManagerCDC.class); + static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentAllocatorCDC.class); private final CDCSizeTracker cdcSizeTracker; + private final CommitLogSegmentManager segmentManager; - public CommitLogSegmentManagerCDC(final CommitLog commitLog, String storageDirectory) + CommitLogSegmentAllocatorCDC(CommitLogSegmentManager segmentManager) { - super(commitLog, storageDirectory); - cdcSizeTracker = new CDCSizeTracker(this, new File(DatabaseDescriptor.getCDCLogLocation())); + this.segmentManager = segmentManager; + cdcSizeTracker = new CDCSizeTracker(segmentManager, new File(DatabaseDescriptor.getCDCLogLocation())); } - @Override - void start() + public void start() { cdcSizeTracker.start(); - super.start(); } public void discard(CommitLogSegment segment, boolean delete) { segment.close(); - addSize(-segment.onDiskSize()); + segmentManager.addSize(-segment.onDiskSize()); cdcSizeTracker.processDiscardedSegment(segment); @@ -82,12 +81,11 @@ public void discard(CommitLogSegment segment, boolean delete) } /** - * Initiates the shutdown process for the management thread. Also stops the cdc on-disk size calculator executor. + * Stops the thread pool for CDC on disk size tracking. */ public void shutdown() { cdcSizeTracker.shutdown(); - super.shutdown(); } /** @@ -99,20 +97,23 @@ public void shutdown() * @return the created Allocation object * @throws CDCWriteException If segment disallows CDC mutations, we throw */ - @Override public CommitLogSegment.Allocation allocate(Mutation mutation, int size) throws CDCWriteException { - CommitLogSegment segment = allocatingFrom(); - CommitLogSegment.Allocation alloc; - + CommitLogSegment segment = segmentManager.getActiveSegment(); throwIfForbidden(mutation, segment); - while ( null == (alloc = segment.allocate(mutation, size)) ) + + CommitLogSegment.Allocation alloc = segment.allocate(mutation, size); + // If we failed to allocate in the segment, prompt for a switch to a new segment and loop on re-attempt. This + // is expected to succeed or throw, since CommitLog allocation working is central to how a node operates. + while (alloc == null) { // Failed to allocate, so move to a new segment with enough room if possible. - advanceAllocatingFrom(segment); - segment = allocatingFrom(); + segmentManager.switchToNewSegment(segment); + segment = segmentManager.getActiveSegment(); + // New segment, so confirm whether or not CDC mutations are allowed on this. throwIfForbidden(mutation, segment); + alloc = segment.allocate(mutation, size); } if (mutation.trackedByCDC()) @@ -143,7 +144,7 @@ private void throwIfForbidden(Mutation mutation, CommitLogSegment segment) throw */ public CommitLogSegment createSegment() { - CommitLogSegment segment = CommitLogSegment.createSegment(commitLog, this); + CommitLogSegment segment = CommitLogSegment.createSegment(segmentManager.commitLog, segmentManager); // Hard link file in cdc folder for realtime tracking FileUtils.createHardLink(segment.logFile, segment.getCDCFile()); @@ -157,11 +158,8 @@ public CommitLogSegment createSegment() * * @param file segment file that is no longer in use. */ - @Override - void handleReplayedSegment(final File file) + public void handleReplayedSegment(final File file) { - super.handleReplayedSegment(file); - // delete untracked cdc segment hard link files if their index files do not exist File cdcFile = new File(DatabaseDescriptor.getCDCLogLocation(), file.getName()); File cdcIndexFile = new File(DatabaseDescriptor.getCDCLogLocation(), CommitLogDescriptor.fromFileName(file.getName()).cdcIndexFileName()); @@ -175,7 +173,7 @@ void handleReplayedSegment(final File file) /** * For use after replay when replayer hard-links / adds tracking of replayed segments */ - public void addCDCSize(long size) + void addCDCSize(long size) { cdcSizeTracker.addSize(size); } @@ -191,12 +189,12 @@ private static class CDCSizeTracker extends DirectorySizeCalculator { private final RateLimiter rateLimiter = RateLimiter.create(1000.0 / DatabaseDescriptor.getCDCDiskCheckInterval()); private ExecutorService cdcSizeCalculationExecutor; - private CommitLogSegmentManagerCDC segmentManager; + private CommitLogSegmentManager segmentManager; // Used instead of size during walk to remove chance of over-allocation private volatile long sizeInProgress = 0; - CDCSizeTracker(CommitLogSegmentManagerCDC segmentManager, File path) + CDCSizeTracker(CommitLogSegmentManager segmentManager, File path) { super(path); this.segmentManager = segmentManager; @@ -215,9 +213,9 @@ public void start() * Synchronous size recalculation on each segment creation/deletion call could lead to very long delays in new * segment allocation, thus long delays in thread signaling to wake waiting allocation / writer threads. * - * This can be reached either from the segment management thread in ABstractCommitLogSegmentManager or from the + * This can be reached either from the segment management thread in CommitLogSegmentManager or from the * size recalculation executor, so we synchronize on this object to reduce the race overlap window available for - * size to get off. + * size to drift. * * Reference DirectorySizerBench for more information about performance of the directory size recalc. */ @@ -237,6 +235,10 @@ void processNewSegment(CommitLogSegment segment) submitOverflowSizeRecalculation(); } + /** + * Upon segment discard, we need to adjust our known CDC consumption on disk based on whether or not this segment + * was flagged to be allowable for CDC. + */ void processDiscardedSegment(CommitLogSegment segment) { // See synchronization in CommitLogSegment.setCDCState @@ -258,7 +260,13 @@ private long allowableCDCBytes() return (long)DatabaseDescriptor.getCDCSpaceInMB() * 1024 * 1024; } - public void submitOverflowSizeRecalculation() + /** + * The overflow size calculation requires walking the flie tree and checking file size for all linked CDC + * files. As such, we do this async on the executor in the CDCSizeTracker instead of the context of the calling + * thread. While this can obviously introduce some delay / raciness in the calculation of CDC size consumed, + * the alternative of significantly long blocks for critical path CL allocation is unacceptable. + */ + void submitOverflowSizeRecalculation() { try { @@ -274,9 +282,12 @@ private void recalculateOverflowSize() { rateLimiter.acquire(); calculateSize(); - CommitLogSegment allocatingFrom = segmentManager.allocatingFrom(); - if (allocatingFrom.getCDCState() == CDCState.FORBIDDEN) - processNewSegment(allocatingFrom); + CommitLogSegment activeCommitLogSegment = segmentManager.getActiveSegment(); + // In the event that the current segment is disallowed for CDC, re-check it as our size on disk may have + // reduced, thus allowing the segment to accept CDC writes. It's worth noting: this would spin on recalc + // endlessly if not for the rate limiter dropping looping calls on the floor. + if (activeCommitLogSegment.getCDCState() == CDCState.FORBIDDEN) + processNewSegment(activeCommitLogSegment); } private int defaultSegmentSize() @@ -327,7 +338,7 @@ private long totalCDCSizeOnDisk() * Only use for testing / validation that size tracker is working. Not for production use. */ @VisibleForTesting - public long updateCDCTotalSize() + long updateCDCTotalSize() { cdcSizeTracker.submitOverflowSizeRecalculation(); @@ -336,7 +347,9 @@ public long updateCDCTotalSize() { Thread.sleep(DatabaseDescriptor.getCDCDiskCheckInterval() + 10); } - catch (InterruptedException e) {} + catch (InterruptedException e) { + // Expected in test context. no-op. + } return cdcSizeTracker.totalCDCSizeOnDisk(); } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorStandard.java new file mode 100644 index 000000000000..f3d64bfbe922 --- /dev/null +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorStandard.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.commitlog; + +import java.io.File; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.io.util.FileUtils; + +/** + * This is a fairly simple form of a CommitLogSegmentAllocator. + */ +public class CommitLogSegmentAllocatorStandard implements CommitLogSegmentAllocator +{ + static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentAllocatorStandard.class); + private final CommitLogSegmentManager segmentManager; + + public void start() {} + public void shutdown() {} + + CommitLogSegmentAllocatorStandard(CommitLogSegmentManager segmentManager) { + this.segmentManager = segmentManager; + } + + /** + * No extra processing required beyond deletion of the file once we have replayed it. + */ + public void handleReplayedSegment(final File file) { + // (don't decrease managed size, since this was never a "live" segment) + logger.trace("(Unopened) segment {} is no longer needed and will be deleted now", file); + FileUtils.deleteWithConfirm(file); + } + + public void discard(CommitLogSegment segment, boolean delete) + { + segment.close(); + if (delete) + FileUtils.deleteWithConfirm(segment.logFile); + segmentManager.addSize(-segment.onDiskSize()); + } + + /** + * Reserve space in the current segment for the provided mutation or, if there isn't space available, + * create a new segment. allocate() is blocking until allocation succeeds as it waits on a signal in switchToNewSegment + * + * @param mutation mutation to allocate space for + * @param size total size of mutation (overhead + serialized size) + * @return the provided Allocation object + */ + public CommitLogSegment.Allocation allocate(Mutation mutation, int size) + { + CommitLogSegment segment = segmentManager.getActiveSegment(); + + CommitLogSegment.Allocation alloc = segment.allocate(mutation, size); + // If we failed to allocate in the segment, prompt for a switch to a new segment and loop on re-attempt. This + // is expected to succeed or throw, since CommitLog allocation working is central to how a node operates. + while (alloc == null) + { + // Failed to allocate, so move to a new segment with enough room if possible. + segmentManager.switchToNewSegment(segment); + segment = segmentManager.getActiveSegment(); + alloc = segment.allocate(mutation, size); + } + + return alloc; + } + + public CommitLogSegment createSegment() + { + return CommitLogSegment.createSegment(segmentManager.commitLog, segmentManager); + } +} diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java similarity index 80% rename from src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java rename to src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java index fdbf7f661ff3..3aad2f323417 100755 --- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java @@ -32,45 +32,45 @@ import net.nicoulaj.compilecommand.annotations.DontInline; import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.schema.Schema; import org.apache.cassandra.db.*; +import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.utils.*; +import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.WrappedRunnable; import org.apache.cassandra.utils.concurrent.WaitQueue; -import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation; - /** * Performs eager-creation of commit log segments in a background thread. All the * public methods are thread safe. */ -public abstract class AbstractCommitLogSegmentManager +public class CommitLogSegmentManager { - static final Logger logger = LoggerFactory.getLogger(AbstractCommitLogSegmentManager.class); + static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManager.class); /** * Segment that is ready to be used. The management thread fills this and blocks until consumed. * * A single management thread produces this, and consumers are already synchronizing to make sure other work is * performed atomically with consuming this. Volatile to make sure writes by the management thread become - * visible (ordered/lazySet would suffice). Consumers (advanceAllocatingFrom and discardAvailableSegment) must + * visible (ordered/lazySet would suffice). Consumers (switchToNewSegment and discardAvailableSegment) must * synchronize on 'this'. */ private volatile CommitLogSegment availableSegment = null; private final WaitQueue segmentPrepared = new WaitQueue(); - /** Active segments, containing unflushed data. The tail of this queue is the one we allocate writes to */ - private final ConcurrentLinkedQueue activeSegments = new ConcurrentLinkedQueue<>(); + /** Segments that are still in memtables and not yet flushed to sstables. + * The tail of this queue is the one we allocate writes to. + */ + private final ConcurrentLinkedQueue unflushedSegments = new ConcurrentLinkedQueue<>(); /** * The segment we are currently allocating commit log records to. * - * Written by advanceAllocatingFrom which synchronizes on 'this'. Volatile to ensure reads get current value. + * Written by switchToNewSegment which synchronizes on 'this'. Volatile to ensure reads get current value. */ - private volatile CommitLogSegment allocatingFrom = null; + private volatile CommitLogSegment activeSegment = null; final String storageDirectory; @@ -83,7 +83,7 @@ public abstract class AbstractCommitLogSegmentManager private final AtomicLong size = new AtomicLong(); private Thread managerThread; - protected final CommitLog commitLog; + final CommitLog commitLog; private volatile boolean shutdown; private final BooleanSupplier managerThreadWaitCondition = () -> (availableSegment == null && !atSegmentBufferLimit()) || shutdown; private final WaitQueue managerThreadWaitQueue = new WaitQueue(); @@ -91,14 +91,20 @@ public abstract class AbstractCommitLogSegmentManager private static final SimpleCachedBufferPool bufferPool = new SimpleCachedBufferPool(DatabaseDescriptor.getCommitLogMaxCompressionBuffersInPool(), DatabaseDescriptor.getCommitLogSegmentSize()); - AbstractCommitLogSegmentManager(final CommitLog commitLog, String storageDirectory) + final CommitLogSegmentAllocator segmentAllocator; + + CommitLogSegmentManager(final CommitLog commitLog, String storageDirectory) { this.commitLog = commitLog; this.storageDirectory = storageDirectory; + this.segmentAllocator = DatabaseDescriptor.isCDCEnabled() ? + new CommitLogSegmentAllocatorCDC(this) : + new CommitLogSegmentAllocatorStandard(this); } void start() { + segmentAllocator.start(); // The run loop for the manager thread Runnable runnable = new WrappedRunnable() { @@ -110,7 +116,7 @@ public void runMayThrow() throws Exception { assert availableSegment == null; logger.trace("No segments in reserve; creating a fresh one"); - availableSegment = createSegment(); + availableSegment = segmentAllocator.createSegment(); if (shutdown) { // If shutdown() started and finished during segment creation, we are now left with a @@ -153,7 +159,7 @@ public void runMayThrow() throws Exception managerThread.start(); // for simplicity, ensure the first segment is allocated before continuing - advanceAllocatingFrom(null); + switchToNewSegment(null); } private boolean atSegmentBufferLimit() @@ -168,9 +174,9 @@ private void maybeFlushToReclaim() { long flushingSize = 0; List segmentsToRecycle = new ArrayList<>(); - for (CommitLogSegment segment : activeSegments) + for (CommitLogSegment segment : unflushedSegments) { - if (segment == allocatingFrom) + if (segment == activeSegment) break; flushingSize += segment.onDiskSize(); segmentsToRecycle.add(segment); @@ -182,54 +188,34 @@ private void maybeFlushToReclaim() } /** - * Allocate a segment within this CLSM. Should either succeed or throw. - */ - public abstract Allocation allocate(Mutation mutation, int size); - - /** - * Hook to allow segment managers to track state surrounding creation of new segments. Onl perform as task submit - * to segment manager so it's performed on segment management thread. - */ - abstract CommitLogSegment createSegment(); - - /** - * Indicates that a segment file has been flushed and is no longer needed. Only perform as task submit to segment - * manager so it's performend on segment management thread, or perform while segment management thread is shutdown - * during testing resets. - * - * @param segment segment to be discarded - * @param delete whether or not the segment is safe to be deleted. - */ - abstract void discard(CommitLogSegment segment, boolean delete); - - /** - * Advances the allocatingFrom pointer to the next prepared segment, but only if it is currently the segment provided. + * Advances the activeSegment pointer to the next prepared segment, but only if it is currently the segment provided. * * WARNING: Assumes segment management thread always succeeds in allocating a new segment or kills the JVM. */ @DontInline - void advanceAllocatingFrom(CommitLogSegment old) + void switchToNewSegment(CommitLogSegment old) { while (true) { synchronized (this) { - // do this in a critical section so we can maintain the order of segment construction when moving to allocatingFrom/activeSegments - if (allocatingFrom != old) + // do this in a critical section so we can maintain the order of segment construction when moving to activeSegment/unflushedSegments + if (activeSegment != old) return; // If a segment is ready, take it now, otherwise wait for the management thread to construct it. if (availableSegment != null) { - // Success! Change allocatingFrom and activeSegments (which must be kept in order) before leaving + // Success! Change activeSegment and unflushedSegments (which must be kept in order) before leaving // the critical section. - activeSegments.add(allocatingFrom = availableSegment); + activeSegment = availableSegment; + unflushedSegments.add(activeSegment); availableSegment = null; break; } } - awaitAvailableSegment(old); + awaitSegmentAllocation(old); } // Signal the management thread to prepare a new segment. @@ -249,17 +235,22 @@ void advanceAllocatingFrom(CommitLogSegment old) commitLog.requestExtraSync(); } - void awaitAvailableSegment(CommitLogSegment currentAllocatingFrom) + /** + * Spins while waiting on next available segment's allocation, putting caller to sleep until the new segment is created. + * @param oldActiveSegment + */ + void awaitSegmentAllocation(CommitLogSegment oldActiveSegment) { do { WaitQueue.Signal prepared = segmentPrepared.register(commitLog.metrics.waitingOnSegmentAllocation.time()); - if (availableSegment == null && allocatingFrom == currentAllocatingFrom) + // No new segment created, and the active segment is the one we already know about. Time to sleep... + if (availableSegment == null && activeSegment == oldActiveSegment) prepared.awaitUninterruptibly(); else prepared.cancel(); } - while (availableSegment == null && allocatingFrom == currentAllocatingFrom); + while (availableSegment == null && activeSegment == oldActiveSegment); } /** @@ -269,9 +260,9 @@ void awaitAvailableSegment(CommitLogSegment currentAllocatingFrom) */ void forceRecycleAll(Iterable droppedTables) { - List segmentsToRecycle = new ArrayList<>(activeSegments); + List segmentsToRecycle = new ArrayList<>(unflushedSegments); CommitLogSegment last = segmentsToRecycle.get(segmentsToRecycle.size() - 1); - advanceAllocatingFrom(last); + switchToNewSegment(last); // wait for the commit log modifications last.waitForModifications(); @@ -286,21 +277,21 @@ void forceRecycleAll(Iterable droppedTables) { future.get(); - for (CommitLogSegment segment : activeSegments) + for (CommitLogSegment segment : unflushedSegments) for (TableId tableId : droppedTables) segment.markClean(tableId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition()); // now recycle segments that are unused, as we may not have triggered a discardCompletedSegments() // if the previous active segment was the only one to recycle (since an active segment isn't // necessarily dirty, and we only call dCS after a flush). - for (CommitLogSegment segment : activeSegments) + for (CommitLogSegment segment : unflushedSegments) { if (segment.isUnused()) archiveAndDiscard(segment); } - CommitLogSegment first; - if ((first = activeSegments.peek()) != null && first.id <= last.id) + CommitLogSegment first = unflushedSegments.peek(); + if (first != null && first.id <= last.id) logger.error("Failed to force-recycle all segments; at least one segment is still in use with dirty CFs."); } catch (Throwable t) @@ -318,11 +309,11 @@ void forceRecycleAll(Iterable droppedTables) void archiveAndDiscard(final CommitLogSegment segment) { boolean archiveSuccess = commitLog.archiver.maybeWaitForArchiving(segment.getName()); - if (!activeSegments.remove(segment)) + if (!unflushedSegments.remove(segment)) return; // already discarded // if archiving (command) was not successful then leave the file alone. don't delete or recycle. logger.debug("Segment {} is no longer active and will be deleted {}", segment, archiveSuccess ? "now" : "by the archive script"); - discard(segment, archiveSuccess); + segmentAllocator.discard(segment, archiveSuccess); } /** @@ -332,9 +323,7 @@ void archiveAndDiscard(final CommitLogSegment segment) */ void handleReplayedSegment(final File file) { - // (don't decrease managed size, since this was never a "live" segment) - logger.trace("(Unopened) segment {} is no longer needed and will be deleted now", file); - FileUtils.deleteWithConfirm(file); + segmentAllocator.handleReplayedSegment(file); } /** @@ -419,9 +408,9 @@ public void stopUnsafe(boolean deleteSegments) throw new RuntimeException(e); } - for (CommitLogSegment segment : activeSegments) + for (CommitLogSegment segment : unflushedSegments) closeAndDeleteSegmentUnsafe(segment, deleteSegments); - activeSegments.clear(); + unflushedSegments.clear(); size.set(0L); @@ -435,7 +424,7 @@ void awaitManagementTasksCompletion() { if (availableSegment == null && !atSegmentBufferLimit()) { - awaitAvailableSegment(allocatingFrom); + awaitSegmentAllocation(activeSegment); } } @@ -446,7 +435,7 @@ private void closeAndDeleteSegmentUnsafe(CommitLogSegment segment, boolean delet { try { - discard(segment, delete); + segmentAllocator.discard(segment, delete); } catch (AssertionError ignored) { @@ -455,13 +444,15 @@ private void closeAndDeleteSegmentUnsafe(CommitLogSegment segment, boolean delet } /** - * Initiates the shutdown process for the management thread. + * Initiates the shutdown process for the management thread and segment allocator. */ public void shutdown() { assert !shutdown; shutdown = true; + segmentAllocator.shutdown(); + // Release the management thread and delete prepared segment. // Do not block as another thread may claim the segment (this can happen during unit test initialization). discardAvailableSegment(); @@ -488,19 +479,19 @@ public void awaitTermination() throws InterruptedException managerThread.join(); managerThread = null; - for (CommitLogSegment segment : activeSegments) + for (CommitLogSegment segment : unflushedSegments) segment.close(); bufferPool.shutdown(); } /** - * @return a read-only collection of the active commit log segments + * @return a read-only collection of all active and unflushed segments in the system */ @VisibleForTesting - public Collection getActiveSegments() + public Collection getUnflushedSegments() { - return Collections.unmodifiableCollection(activeSegments); + return Collections.unmodifiableCollection(unflushedSegments); } /** @@ -508,7 +499,7 @@ public Collection getActiveSegments() */ CommitLogPosition getCurrentPosition() { - return allocatingFrom.getCurrentCommitLogPosition(); + return activeSegment.getCurrentCommitLogPosition(); } /** @@ -518,8 +509,8 @@ CommitLogPosition getCurrentPosition() */ public void sync(boolean flush) throws IOException { - CommitLogSegment current = allocatingFrom; - for (CommitLogSegment segment : getActiveSegments()) + CommitLogSegment current = activeSegment; + for (CommitLogSegment segment : getUnflushedSegments()) { // Do not sync segments that became active after sync started. if (segment.id > current.id) @@ -550,10 +541,17 @@ void notifyBufferFreed() wakeManager(); } - /** Read-only access to current segment for subclasses. */ - CommitLogSegment allocatingFrom() + /** + * Pass-through call to allocator. Allocates a mutation in the active CommitLogSegment. + */ + CommitLogSegment.Allocation allocate(Mutation mutation, int size) { + return segmentAllocator.allocate(mutation, size); + } + + /** Read-only access to current segment for package usage. */ + CommitLogSegment getActiveSegment() { - return allocatingFrom; + return activeSegment; } } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java deleted file mode 100644 index b9bd744da1a5..000000000000 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.db.commitlog; - -import java.io.File; - -import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.io.util.FileUtils; - -public class CommitLogSegmentManagerStandard extends AbstractCommitLogSegmentManager -{ - public CommitLogSegmentManagerStandard(final CommitLog commitLog, String storageDirectory) - { - super(commitLog, storageDirectory); - } - - public void discard(CommitLogSegment segment, boolean delete) - { - segment.close(); - if (delete) - FileUtils.deleteWithConfirm(segment.logFile); - addSize(-segment.onDiskSize()); - } - - /** - * Reserve space in the current segment for the provided mutation or, if there isn't space available, - * create a new segment. allocate() is blocking until allocation succeeds as it waits on a signal in advanceAllocatingFrom - * - * @param mutation mutation to allocate space for - * @param size total size of mutation (overhead + serialized size) - * @return the provided Allocation object - */ - public CommitLogSegment.Allocation allocate(Mutation mutation, int size) - { - CommitLogSegment segment = allocatingFrom(); - - CommitLogSegment.Allocation alloc; - while ( null == (alloc = segment.allocate(mutation, size)) ) - { - // failed to allocate, so move to a new segment with enough room - advanceAllocatingFrom(segment); - segment = allocatingFrom(); - } - - return alloc; - } - - public CommitLogSegment createSegment() - { - return CommitLogSegment.createSegment(commitLog, this); - } -} diff --git a/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java index d5e61137d842..c74e8430b218 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java @@ -41,7 +41,7 @@ public class CompressedSegment extends FileDirectSegment /** * Constructs a new segment file. */ - CompressedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + CompressedSegment(CommitLog commitLog, CommitLogSegmentManager manager) { super(commitLog, manager); this.compressor = commitLog.configuration.getCompressor(); @@ -59,7 +59,7 @@ void write(int startMarker, int nextMarker) int contentStart = startMarker + SYNC_MARKER_SIZE; int length = nextMarker - contentStart; // The length may be 0 when the segment is being closed. - assert length > 0 || length == 0 && !isStillAllocating(); + assert length > 0 || length == 0 && !hasRoom(); try { diff --git a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java index 21b7c11fb052..b699da438290 100644 --- a/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/EncryptedSegment.java @@ -64,7 +64,7 @@ public class EncryptedSegment extends FileDirectSegment private final EncryptionContext encryptionContext; private final Cipher cipher; - public EncryptedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + public EncryptedSegment(CommitLog commitLog, CommitLogSegmentManager manager) { super(commitLog, manager); this.encryptionContext = commitLog.configuration.getEncryptionContext(); @@ -101,7 +101,7 @@ void write(int startMarker, int nextMarker) int contentStart = startMarker + SYNC_MARKER_SIZE; final int length = nextMarker - contentStart; // The length may be 0 when the segment is being closed. - assert length > 0 || length == 0 && !isStillAllocating(); + assert length > 0 || length == 0 && !hasRoom(); final ICompressor compressor = encryptionContext.getCompressor(); final int blockSize = encryptionContext.getChunkLength(); diff --git a/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java b/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java index d5431f875b5f..d82fd10d40a6 100644 --- a/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/FileDirectSegment.java @@ -31,7 +31,7 @@ public abstract class FileDirectSegment extends CommitLogSegment { volatile long lastWrittenPos = 0; - FileDirectSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + FileDirectSegment(CommitLog commitLog, CommitLogSegmentManager manager) { super(commitLog, manager); } diff --git a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java index 6ecdbd3c7764..50c09dd813d8 100644 --- a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java @@ -40,7 +40,7 @@ public class MemoryMappedSegment extends CommitLogSegment * * @param commitLog the commit log it will be used with. */ - MemoryMappedSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + MemoryMappedSegment(CommitLog commitLog, CommitLogSegmentManager manager) { super(commitLog, manager); // mark the initial sync marker as uninitialised diff --git a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java index 08c1c8e46d28..4473760d0e5f 100644 --- a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java +++ b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java @@ -20,7 +20,7 @@ import com.codahale.metrics.Gauge; import com.codahale.metrics.Timer; import org.apache.cassandra.db.commitlog.AbstractCommitLogService; -import org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager; +import org.apache.cassandra.db.commitlog.CommitLogSegmentManager; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; @@ -48,7 +48,7 @@ public CommitLogMetrics() waitingOnCommit = Metrics.timer(factory.createMetricName("WaitingOnCommit")); } - public void attach(final AbstractCommitLogService service, final AbstractCommitLogSegmentManager segmentManager) + public void attach(final AbstractCommitLogService service, final CommitLogSegmentManager segmentManager) { completedTasks = Metrics.register(factory.createMetricName("CompletedTasks"), new Gauge() { diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java index e2c6e33789bc..3749e11031b4 100644 --- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java +++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java @@ -287,7 +287,7 @@ private void verifySizes(CommitLog commitLog) List logFileNames = commitLog.getActiveSegmentNames(); Map ratios = commitLog.getActiveSegmentCompressionRatios(); - Collection segments = commitLog.segmentManager.getActiveSegments(); + Collection segments = commitLog.segmentManager.getUnflushedSegments(); for (CommitLogSegment segment : segments) { diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java index b4fe0f5fd2f3..3224e7f940fd 100644 --- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java +++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java @@ -126,7 +126,7 @@ public void flushAndExpectError() throws InterruptedException, ExecutionExceptio // Make sure commit log wasn't discarded. TableId tableId = currentTableMetadata().id; - for (CommitLogSegment segment : CommitLog.instance.segmentManager.getActiveSegments()) + for (CommitLogSegment segment : CommitLog.instance.segmentManager.getUnflushedSegments()) if (segment.getDirtyTableIds().contains(tableId)) return; fail("Expected commit log to remain dirty for the affected table."); diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java index 72356003719b..c113b971a576 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java @@ -31,11 +31,11 @@ public void testTruncateSegmentDiscard() throws Throwable execute("INSERT INTO %s (idx, data) VALUES (?, ?)", 15, Integer.toString(17)); - Collection active = new ArrayList<>(CommitLog.instance.segmentManager.getActiveSegments()); + Collection active = new ArrayList<>(CommitLog.instance.segmentManager.getUnflushedSegments()); CommitLog.instance.forceRecycleAllSegments(); // If one of the previous segments remains, it wasn't clean. - active.retainAll(CommitLog.instance.segmentManager.getActiveSegments()); + active.retainAll(CommitLog.instance.segmentManager.getUnflushedSegments()); assert active.isEmpty(); } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java similarity index 95% rename from test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java rename to test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java index 8c0647c83917..0d74ef629297 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java @@ -38,7 +38,7 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.schema.TableMetadata; -public class CommitLogSegmentManagerCDCTest extends CQLTester +public class CommitLogSegmentAllocatorCDCTest extends CQLTester { private static final Random random = new Random(); @@ -56,14 +56,14 @@ public void beforeTest() throws Throwable // Need to clean out any files from previous test runs. Prevents flaky test failures. CommitLog.instance.stopUnsafe(true); CommitLog.instance.start(); - ((CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager).updateCDCTotalSize(); + updateCDCTotalSize(CommitLog.instance.segmentManager); } @Test public void testCDCWriteFailure() throws Throwable { createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=true;"); - CommitLogSegmentManagerCDC cdcMgr = (CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager; + CommitLogSegmentManager cdcMgr = CommitLog.instance.segmentManager; TableMetadata cfm = currentTableMetadata(); // Confirm that logic to check for whether or not we can allocate new CDC segments works @@ -103,8 +103,8 @@ public void testCDCWriteFailure() throws Throwable for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).listFiles()) FileUtils.deleteWithConfirm(f); - // Update size tracker to reflect deleted files. Should flip flag on current allocatingFrom to allow. - cdcMgr.updateCDCTotalSize(); + // Update size tracker to reflect deleted files. Should flip flag on current active segment to allow. + updateCDCTotalSize(cdcMgr); expectCurrentCDCState(CDCState.PERMITTED); } finally @@ -116,7 +116,7 @@ public void testCDCWriteFailure() throws Throwable @Test public void testSegmentFlaggingOnCreation() throws Throwable { - CommitLogSegmentManagerCDC cdcMgr = (CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager; + CommitLogSegmentManager cdcMgr = CommitLog.instance.segmentManager; String ct = createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=true;"); int origSize = DatabaseDescriptor.getCDCSpaceInMB(); @@ -144,7 +144,7 @@ public void testSegmentFlaggingOnCreation() throws Throwable // Delete all files in cdc_raw for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).listFiles()) f.delete(); - cdcMgr.updateCDCTotalSize(); + updateCDCTotalSize(cdcMgr); // Confirm cdc update process changes flag on active segment expectCurrentCDCState(CDCState.PERMITTED); @@ -168,7 +168,7 @@ public void testCDCIndexFileWriteOnSync() throws IOException .build().apply(); CommitLog.instance.sync(true); - CommitLogSegment currentSegment = CommitLog.instance.segmentManager.allocatingFrom(); + CommitLogSegment currentSegment = CommitLog.instance.segmentManager.getActiveSegment(); int syncOffset = currentSegment.lastSyncedOffset; // Confirm index file is written @@ -187,7 +187,7 @@ public void testCDCIndexFileWriteOnSync() throws IOException public void testCompletedFlag() throws IOException { createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=true;"); - CommitLogSegment initialSegment = CommitLog.instance.segmentManager.allocatingFrom(); + CommitLogSegment initialSegment = CommitLog.instance.segmentManager.getActiveSegment(); Integer originalCDCSize = DatabaseDescriptor.getCDCSpaceInMB(); DatabaseDescriptor.setCDCSpaceInMB(8); @@ -230,7 +230,7 @@ public void testDeleteLinkOnDiscardNoCDC() throws Throwable new RowUpdateBuilder(currentTableMetadata(), 0, 1) .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); - CommitLogSegment currentSegment = CommitLog.instance.segmentManager.allocatingFrom(); + CommitLogSegment currentSegment = CommitLog.instance.segmentManager.getActiveSegment(); // Confirm that, with no CDC data present, we've hard-linked but have no index file Path linked = new File(DatabaseDescriptor.getCDCLogLocation(), currentSegment.logFile.getName()).toPath(); @@ -253,7 +253,7 @@ public void testDeleteLinkOnDiscardNoCDC() throws Throwable public void testRetainLinkOnDiscardCDC() throws Throwable { createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=true;"); - CommitLogSegment currentSegment = CommitLog.instance.segmentManager.allocatingFrom(); + CommitLogSegment currentSegment = CommitLog.instance.segmentManager.getActiveSegment(); File cdcIndexFile = currentSegment.getCDCIndexFile(); Assert.assertFalse("Expected no index file before flush but found: " + cdcIndexFile, cdcIndexFile.exists()); @@ -437,13 +437,18 @@ private int getCDCRawCount() private void expectCurrentCDCState(CDCState expectedState) { - CDCState currentState = CommitLog.instance.segmentManager.allocatingFrom().getCDCState(); + CDCState currentState = CommitLog.instance.segmentManager.getActiveSegment().getCDCState(); if (currentState != expectedState) { logger.error("expectCurrentCDCState violation! Expected state: {}. Found state: {}. Current CDC allocation: {}", - expectedState, currentState, ((CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager).updateCDCTotalSize()); - Assert.fail(String.format("Received unexpected CDCState on current allocatingFrom segment. Expected: %s. Received: %s", + expectedState, currentState, updateCDCTotalSize(CommitLog.instance.segmentManager)); + Assert.fail(String.format("Received unexpected CDCState on current active segment. Expected: %s. Received: %s", expectedState, currentState)); } } + + private long updateCDCTotalSize(CommitLogSegmentManager segmentManager) + { + return ((CommitLogSegmentAllocatorCDC)segmentManager.segmentAllocator).updateCDCTotalSize(); + } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java index 6b167b2500ee..70af3cbfbadf 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java @@ -111,21 +111,21 @@ public void testCompressedCommitLogBackpressure() throws Throwable dummyThread.start(); - AbstractCommitLogSegmentManager clsm = CommitLog.instance.segmentManager; + CommitLogSegmentManager clsm = CommitLog.instance.segmentManager; - Util.spinAssertEquals(3, () -> clsm.getActiveSegments().size(), 5); + Util.spinAssertEquals(3, () -> clsm.getUnflushedSegments().size(), 5); Thread.sleep(1000); // Should only be able to create 3 segments not 7 because it blocks waiting for truncation that never comes - Assert.assertEquals(3, clsm.getActiveSegments().size()); + Assert.assertEquals(3, clsm.getUnflushedSegments().size()); // Discard the currently active segments so allocation can continue. // Take snapshot of the list, otherwise this will also discard newly allocated segments. - new ArrayList<>(clsm.getActiveSegments()).forEach( clsm::archiveAndDiscard ); + new ArrayList<>(clsm.getUnflushedSegments()).forEach( clsm::archiveAndDiscard ); // The allocated count should reach the limit again. - Util.spinAssertEquals(3, () -> clsm.getActiveSegments().size(), 5); + Util.spinAssertEquals(3, () -> clsm.getUnflushedSegments().size(), 5); } finally { diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java index 25e2f306254f..50a4bbc15127 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java @@ -314,13 +314,13 @@ public void testDontDeleteIfDirty() throws Exception .build(); CommitLog.instance.add(m2); - assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size()); + assertEquals(2, CommitLog.instance.segmentManager.getUnflushedSegments().size()); TableId id2 = m2.getTableIds().iterator().next(); CommitLog.instance.discardCompletedSegments(id2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); // Assert we still have both our segments - assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size()); + assertEquals(2, CommitLog.instance.segmentManager.getUnflushedSegments().size()); } @Test @@ -340,14 +340,14 @@ public void testDeleteIfNotDirty() throws Exception CommitLog.instance.add(rm); CommitLog.instance.add(rm); - assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size()); + assertEquals(1, CommitLog.instance.segmentManager.getUnflushedSegments().size()); // "Flush": this won't delete anything TableId id1 = rm.getTableIds().iterator().next(); CommitLog.instance.sync(true); CommitLog.instance.discardCompletedSegments(id1, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); - assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size()); + assertEquals(1, CommitLog.instance.segmentManager.getUnflushedSegments().size()); // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created Mutation rm2 = new RowUpdateBuilder(cfs2.metadata(), 0, "k") @@ -359,7 +359,7 @@ public void testDeleteIfNotDirty() throws Exception CommitLog.instance.add(rm2); CommitLog.instance.add(rm2); - Collection segments = CommitLog.instance.segmentManager.getActiveSegments(); + Collection segments = CommitLog.instance.segmentManager.getUnflushedSegments(); assertEquals(String.format("Expected 3 segments but got %d (%s)", segments.size(), getDirtyCFIds(segments)), 3, @@ -371,7 +371,7 @@ public void testDeleteIfNotDirty() throws Exception TableId id2 = rm2.getTableIds().iterator().next(); CommitLog.instance.discardCompletedSegments(id2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); - segments = CommitLog.instance.segmentManager.getActiveSegments(); + segments = CommitLog.instance.segmentManager.getUnflushedSegments(); // Assert we still have both our segment assertEquals(String.format("Expected 1 segment but got %d (%s)", segments.size(), getDirtyCFIds(segments)), @@ -617,13 +617,13 @@ public void testTruncateWithoutSnapshot() throws ExecutionException, Interrupted for (int i = 0 ; i < 5 ; i++) CommitLog.instance.add(m2); - assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size()); + assertEquals(2, CommitLog.instance.segmentManager.getUnflushedSegments().size()); CommitLogPosition position = CommitLog.instance.getCurrentPosition(); for (Keyspace keyspace : Keyspace.system()) for (ColumnFamilyStore syscfs : keyspace.getColumnFamilyStores()) CommitLog.instance.discardCompletedSegments(syscfs.metadata().id, CommitLogPosition.NONE, position); CommitLog.instance.discardCompletedSegments(cfs2.metadata().id, CommitLogPosition.NONE, position); - assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size()); + assertEquals(1, CommitLog.instance.segmentManager.getUnflushedSegments().size()); } finally { From 9cf0813da1897627eeb713ccc58c1994612d8738 Mon Sep 17 00:00:00 2001 From: Josh McKenzie Date: Sun, 13 Oct 2019 14:00:00 -0400 Subject: [PATCH 73/78] Minor refactor to CommitLogReader Adds some javadocs and tightens up package private scope on a couple things. --- .../db/commitlog/CommitLogReader.java | 44 ++++++++++++------- 1 file changed, 28 insertions(+), 16 deletions(-) diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java index 078bb5304d41..cf7fa48e5f58 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java @@ -46,6 +46,10 @@ import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt; +/** + * A CommitLogReader is used to traverse a flat CommitLog file and provide deserialized Mutation objects to an accepting + * CommitLogReadHandler. + */ public class CommitLogReader { private static final Logger logger = LoggerFactory.getLogger(CommitLogReader.class); @@ -53,7 +57,7 @@ public class CommitLogReader private static final int LEGACY_END_OF_SEGMENT_MARKER = 0; @VisibleForTesting - public static final int ALL_MUTATIONS = -1; + static final int ALL_MUTATIONS = -1; private final CRC32 checksum; private final Map invalidMutations; @@ -66,7 +70,7 @@ public CommitLogReader() buffer = new byte[4096]; } - public Set> getInvalidMutations() + Set> getInvalidMutations() { return invalidMutations.entrySet(); } @@ -74,7 +78,7 @@ public Set> getInvalidMutations() /** * Reads all passed in files with no minimum, no start, and no mutation limit. */ - public void readAllFiles(CommitLogReadHandler handler, File[] files) throws IOException + void readAllFiles(CommitLogReadHandler handler, File[] files) throws IOException { readAllFiles(handler, files, CommitLogPosition.NONE); } @@ -119,7 +123,7 @@ static List filterCommitLogFiles(File[] toFilter) /** * Reads all passed in files with minPosition, no start, and no mutation limit. */ - public void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPosition minPosition) throws IOException + private void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPosition minPosition) throws IOException { List filteredLogs = filterCommitLogFiles(files); int i = 0; @@ -141,7 +145,7 @@ public void readCommitLogSegment(CommitLogReadHandler handler, File file, boolea /** * Reads all mutations from passed in file from minPosition */ - public void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException + void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException { readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, tolerateTruncation); } @@ -150,7 +154,7 @@ public void readCommitLogSegment(CommitLogReadHandler handler, File file, Commit * Reads passed in file fully, up to mutationLimit count */ @VisibleForTesting - public void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException + void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException { readCommitLogSegment(handler, file, CommitLogPosition.NONE, mutationLimit, tolerateTruncation); } @@ -165,7 +169,7 @@ public void readCommitLogSegment(CommitLogReadHandler handler, File file, int mu * * @throws IOException */ - public void readCommitLogSegment(CommitLogReadHandler handler, + void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, int mutationLimit, @@ -361,6 +365,9 @@ private void readSection(CommitLogReadHandler handler, return; } + // Overallocate a touch relative to the required size for this mutation, presumably to allow a little + // breathing room and not thrash on a need to re-allocate repeatedly for other mutations of slightly larger size. + // TODO: Why 1.2x? Why not double? Why not 1.1? if (serializedSize > buffer.length) buffer = new byte[(int) (1.2 * serializedSize)]; reader.readFully(buffer, 0, serializedSize); @@ -479,51 +486,56 @@ protected void readMutation(CommitLogReadHandler handler, /** * Helper methods to deal with changing formats of internals of the CommitLog without polluting deserialization code. + * Note: while this has a currently unused variable (commitLogVersion) in the 4.0 line, the idea is to future proof + * and allow support for multiple disparate CommitLogFormats at a single time. */ private static class CommitLogFormat { - public static long calculateClaimedChecksum(FileDataInput input, int commitLogVersion) throws IOException + static long calculateClaimedChecksum(FileDataInput input, int commitLogVersion) throws IOException { return input.readInt() & 0xffffffffL; } - public static void updateChecksum(CRC32 checksum, int serializedSize, int commitLogVersion) + static void updateChecksum(CRC32 checksum, int serializedSize, int commitLogVersion) { updateChecksumInt(checksum, serializedSize); } - public static long calculateClaimedCRC32(FileDataInput input, int commitLogVersion) throws IOException + static long calculateClaimedCRC32(FileDataInput input, int commitLogVersion) throws IOException { return input.readInt() & 0xffffffffL; } } + /** + * Caches the state needed for decision-making on multiple CommitLog Read operations + */ private static class ReadStatusTracker { private int mutationsLeft; - public String errorContext = ""; - public boolean tolerateErrorsInSection; + String errorContext = ""; + boolean tolerateErrorsInSection; private boolean error; - public ReadStatusTracker(int mutationLimit, boolean tolerateErrorsInSection) + ReadStatusTracker(int mutationLimit, boolean tolerateErrorsInSection) { this.mutationsLeft = mutationLimit; this.tolerateErrorsInSection = tolerateErrorsInSection; } - public void addProcessedMutation() + void addProcessedMutation() { if (mutationsLeft == ALL_MUTATIONS) return; --mutationsLeft; } - public boolean shouldContinue() + boolean shouldContinue() { return !error && (mutationsLeft != 0 || mutationsLeft == ALL_MUTATIONS); } - public void requestTermination() + void requestTermination() { error = true; } From cb993a8eac3a02cc0a03d65e712b496e3f6ba138 Mon Sep 17 00:00:00 2001 From: Josh McKenzie Date: Sun, 13 Oct 2019 14:00:00 -0400 Subject: [PATCH 74/78] Interim commit working on resumable reader Adds a CommitLogTestUtils concept; initial stubbing in of ResumableCommitLogReader and ResumableCommitLogReaderTest. Squash this before merge. --- .../db/commitlog/CommitLogReader.java | 87 +++++----- .../commitlog/ResumableCommitLogReader.java | 42 +++++ .../CommitLogSegmentAllocatorCDCTest.java | 60 ++----- .../db/commitlog/CommitLogTestUtils.java | 111 ++++++++++++ .../ResumableCommitLogReaderTest.java | 161 ++++++++++++++++++ 5 files changed, 375 insertions(+), 86 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java create mode 100644 test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java create mode 100644 test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java index cf7fa48e5f58..dafe36fbf16d 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java @@ -48,7 +48,8 @@ /** * A CommitLogReader is used to traverse a flat CommitLog file and provide deserialized Mutation objects to an accepting - * CommitLogReadHandler. + * CommitLogReadHandler. None of the actual state about traversal or progress is actually stored inside this class; it's + * largely a stateless container w/logic to allow reading files which are assumed to be CommitLog Segments. */ public class CommitLogReader { @@ -78,11 +79,53 @@ Set> getInvalidMutations() /** * Reads all passed in files with no minimum, no start, and no mutation limit. */ - void readAllFiles(CommitLogReadHandler handler, File[] files) throws IOException + public void readAllFiles(CommitLogReadHandler handler, File[] files) throws IOException { readAllFiles(handler, files, CommitLogPosition.NONE); } + /** + * Reads all passed in files with minPosition, no start, and no mutation limit. + */ + public void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPosition minPosition) throws IOException + { + List filteredLogs = filterCommitLogFiles(files); + int i = 0; + for (File file: filteredLogs) + { + i++; + readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, i == filteredLogs.size()); + } + } + + /** + * Reads passed in file fully + */ + public void readCommitLogSegment(CommitLogReadHandler handler, File file, boolean tolerateTruncation) throws IOException + { + readCommitLogSegment(handler, file, CommitLogPosition.NONE, ALL_MUTATIONS, tolerateTruncation); + } + + /** + * Reads all mutations from passed in file from minPosition + */ + public void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException + { + readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, tolerateTruncation); + } + + /** + * Reads passed in file fully, up to mutationLimit count + */ + @VisibleForTesting + public void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException + { + readCommitLogSegment(handler, file, CommitLogPosition.NONE, mutationLimit, tolerateTruncation); + } + + /** + * Confirms whether the passed in file is one we should read or skip based on whether it's empty and passes crc + */ private static boolean shouldSkip(File file) throws IOException, ConfigurationException { try(RandomAccessReader reader = RandomAccessReader.open(file)) @@ -120,44 +163,6 @@ static List filterCommitLogFiles(File[] toFilter) return filtered; } - /** - * Reads all passed in files with minPosition, no start, and no mutation limit. - */ - private void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPosition minPosition) throws IOException - { - List filteredLogs = filterCommitLogFiles(files); - int i = 0; - for (File file: filteredLogs) - { - i++; - readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, i == filteredLogs.size()); - } - } - - /** - * Reads passed in file fully - */ - public void readCommitLogSegment(CommitLogReadHandler handler, File file, boolean tolerateTruncation) throws IOException - { - readCommitLogSegment(handler, file, CommitLogPosition.NONE, ALL_MUTATIONS, tolerateTruncation); - } - - /** - * Reads all mutations from passed in file from minPosition - */ - void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException - { - readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, tolerateTruncation); - } - - /** - * Reads passed in file fully, up to mutationLimit count - */ - @VisibleForTesting - void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException - { - readCommitLogSegment(handler, file, CommitLogPosition.NONE, mutationLimit, tolerateTruncation); - } /** * Reads mutations from file, handing them off to handler @@ -508,7 +513,7 @@ static long calculateClaimedCRC32(FileDataInput input, int commitLogVersion) thr } /** - * Caches the state needed for decision-making on multiple CommitLog Read operations + * Caches the state needed for decision-making on multiple CommitLog Read operations. Used internally in the CommitLogReader */ private static class ReadStatusTracker { diff --git a/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java new file mode 100644 index 000000000000..23a8c25db3db --- /dev/null +++ b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.commitlog; + +import org.apache.cassandra.config.DatabaseDescriptor; + +/** + * A CommitLogReader that allows both a) indicating how much of the CommitLog the user would like to read, and b) the ability + * to resume reading from a given offset to prevent the need to re-read CommitLog files in a CDC / consumption context. + * As this is purely a class for use in CDC-enabled contexts, we don't test its operation in non-cdc and assert that CDC + * is enabled if this class is in use. + * + * Note: As the plan for usage of this class is by external users writing CDC consumers, treat the public members of this + * class as an external API contract. i.e. don't make changes to it lightly, and consider backwards-compatibility options + * on changes. In the event that this interface proves to be more fluid over time than the previous stability of the CommitLog + * in general would imply, we can break out a proper interface for it at that time. + */ +public class ResumableCommitLogReader +{ + public ResumableCommitLogReader() + { + if (!DatabaseDescriptor.isCDCEnabled()) + { + throw new RuntimeException("Cannot use a ResumableCommitLogReader if CDC is not enabled."); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java index 0d74ef629297..c50d9973ed51 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java @@ -19,7 +19,6 @@ package org.apache.cassandra.db.commitlog; import java.io.*; -import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Path; import java.util.*; @@ -40,8 +39,6 @@ public class CommitLogSegmentAllocatorCDCTest extends CQLTester { - private static final Random random = new Random(); - @BeforeClass public static void setUpClass() { @@ -56,7 +53,7 @@ public void beforeTest() throws Throwable // Need to clean out any files from previous test runs. Prevents flaky test failures. CommitLog.instance.stopUnsafe(true); CommitLog.instance.start(); - updateCDCTotalSize(CommitLog.instance.segmentManager); + CommitLogTestUtils.updateCDCTotalSize(CommitLog.instance.segmentManager); } @Test @@ -78,7 +75,7 @@ public void testCDCWriteFailure() throws Throwable for (int i = 0; i < 100; i++) { new RowUpdateBuilder(cfm, 0, i) - .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) + .add("data", CommitLogTestUtils.randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); } Assert.fail("Expected CDCWriteException from full CDC but did not receive it."); @@ -87,7 +84,7 @@ public void testCDCWriteFailure() throws Throwable { // expected, do nothing } - expectCurrentCDCState(CDCState.FORBIDDEN); + CommitLogTestUtils.expectCurrentCDCState(CDCState.FORBIDDEN); // Confirm we can create a non-cdc table and write to it even while at cdc capacity createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=false;"); @@ -97,15 +94,15 @@ public void testCDCWriteFailure() throws Throwable Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush(); CommitLog.instance.forceRecycleAllSegments(); cdcMgr.awaitManagementTasksCompletion(); - Assert.assertTrue("Expected files to be moved to overflow.", getCDCRawCount() > 0); + Assert.assertTrue("Expected files to be moved to overflow.", CommitLogTestUtils.getCDCRawCount() > 0); // Simulate a CDC consumer reading files then deleting them for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).listFiles()) FileUtils.deleteWithConfirm(f); // Update size tracker to reflect deleted files. Should flip flag on current active segment to allow. - updateCDCTotalSize(cdcMgr); - expectCurrentCDCState(CDCState.PERMITTED); + CommitLogTestUtils.updateCDCTotalSize(cdcMgr); + CommitLogTestUtils.expectCurrentCDCState(CDCState.PERMITTED); } finally { @@ -130,23 +127,23 @@ public void testSegmentFlaggingOnCreation() throws Throwable for (int i = 0; i < 1000; i++) { new RowUpdateBuilder(ccfm, 0, i) - .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) + .add("data", CommitLogTestUtils.randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); } Assert.fail("Expected CDCWriteException from full CDC but did not receive it."); } catch (CDCWriteException e) { } - expectCurrentCDCState(CDCState.FORBIDDEN); + CommitLogTestUtils.expectCurrentCDCState(CDCState.FORBIDDEN); CommitLog.instance.forceRecycleAllSegments(); cdcMgr.awaitManagementTasksCompletion(); // Delete all files in cdc_raw for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).listFiles()) f.delete(); - updateCDCTotalSize(cdcMgr); + CommitLogTestUtils.updateCDCTotalSize(cdcMgr); // Confirm cdc update process changes flag on active segment - expectCurrentCDCState(CDCState.PERMITTED); + CommitLogTestUtils.expectCurrentCDCState(CDCState.PERMITTED); // Clear out archived CDC files for (File f : new File(DatabaseDescriptor.getCDCLogLocation()).listFiles()) { @@ -164,7 +161,7 @@ public void testCDCIndexFileWriteOnSync() throws IOException { createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=true;"); new RowUpdateBuilder(currentTableMetadata(), 0, 1) - .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) + .add("data", CommitLogTestUtils.randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); CommitLog.instance.sync(true); @@ -196,7 +193,7 @@ public void testCompletedFlag() throws IOException for (int i = 0; i < 1000; i++) { new RowUpdateBuilder(currentTableMetadata(), 0, 1) - .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) + .add("data", CommitLogTestUtils.randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); } } @@ -228,7 +225,7 @@ public void testDeleteLinkOnDiscardNoCDC() throws Throwable { createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=false;"); new RowUpdateBuilder(currentTableMetadata(), 0, 1) - .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) + .add("data", CommitLogTestUtils.randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); CommitLogSegment currentSegment = CommitLog.instance.segmentManager.getActiveSegment(); @@ -258,7 +255,7 @@ public void testRetainLinkOnDiscardCDC() throws Throwable Assert.assertFalse("Expected no index file before flush but found: " + cdcIndexFile, cdcIndexFile.exists()); new RowUpdateBuilder(currentTableMetadata(), 0, 1) - .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) + .add("data", CommitLogTestUtils.randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); Path linked = new File(DatabaseDescriptor.getCDCLogLocation(), currentSegment.logFile.getName()).toPath(); @@ -290,7 +287,7 @@ public void testReplayLogic() throws IOException for (int i = 0; i < 1000; i++) { new RowUpdateBuilder(ccfm, 0, i) - .add("data", randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) + .add("data", CommitLogTestUtils.randomizeBuffer(DatabaseDescriptor.getCommitLogSegmentSize() / 3)) .build().apply(); } Assert.fail("Expected CDCWriteException from full CDC but did not receive it."); @@ -423,32 +420,5 @@ public boolean equals(Object other) } } - private ByteBuffer randomizeBuffer(int size) - { - byte[] toWrap = new byte[size]; - random.nextBytes(toWrap); - return ByteBuffer.wrap(toWrap); - } - private int getCDCRawCount() - { - return new File(DatabaseDescriptor.getCDCLogLocation()).listFiles().length; - } - - private void expectCurrentCDCState(CDCState expectedState) - { - CDCState currentState = CommitLog.instance.segmentManager.getActiveSegment().getCDCState(); - if (currentState != expectedState) - { - logger.error("expectCurrentCDCState violation! Expected state: {}. Found state: {}. Current CDC allocation: {}", - expectedState, currentState, updateCDCTotalSize(CommitLog.instance.segmentManager)); - Assert.fail(String.format("Received unexpected CDCState on current active segment. Expected: %s. Received: %s", - expectedState, currentState)); - } - } - - private long updateCDCTotalSize(CommitLogSegmentManager segmentManager) - { - return ((CommitLogSegmentAllocatorCDC)segmentManager.segmentAllocator).updateCDCTotalSize(); - } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java new file mode 100644 index 000000000000..517ff17c491b --- /dev/null +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.commitlog; + +import java.io.File; +import java.nio.ByteBuffer; +import java.util.Random; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.junit.Assert; + +import org.apache.cassandra.config.DatabaseDescriptor; + +class CommitLogTestUtils +{ + private static final Logger logger = LoggerFactory.getLogger(CommitLogTestUtils.class); + + static ByteBuffer randomizeBuffer(int size) + { + byte[] toWrap = new byte[size]; + new Random().nextBytes(toWrap); + return ByteBuffer.wrap(toWrap); + } + + static int getCDCRawCount() + { + return new File(DatabaseDescriptor.getCDCLogLocation()).listFiles().length; + } + + static void expectCurrentCDCState(CommitLogSegment.CDCState expectedState) + { + CommitLogSegment.CDCState currentState = CommitLog.instance.segmentManager.getActiveSegment().getCDCState(); + if (currentState != expectedState) + { + logger.error("expectCurrentCDCState violation! Expected state: {}. Found state: {}. Current CDC allocation: {}", + expectedState, currentState, updateCDCTotalSize(CommitLog.instance.segmentManager)); + Assert.fail(String.format("Received unexpected CDCState on current active segment. Expected: %s. Received: %s", + expectedState, currentState)); + } + } + + static long updateCDCTotalSize(CommitLogSegmentManager segmentManager) + { + return ((CommitLogSegmentAllocatorCDC)segmentManager.segmentAllocator).updateCDCTotalSize(); + } + + static File getFilledCommitLogFile() + { + File result = new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()[0]; + Assert.assertNotEquals(result.toString(), CommitLog.instance.segmentManager.getActiveSegment().logFile); + return result; + } + + /** + * There's some possible raciness here, but for purposes of unit tests this should be deterministic enough not to cause + * test failures, assuming we're doing deterministic writes when using this method. + */ + static int getCommitLogCountOnDisk() + { + return CommitLog.instance.segmentManager.getUnflushedSegments().size(); + } + + /** + * For a given input file, writes randomized garbage from offset to end of file to "corrupt" it + */ + static void corruptFileAtOffset(File f, int offset) + { + } + + static void listCommitLogFiles(String message) + { + StringBuilder result = new StringBuilder(); + result.append(String.format("%s\n", message)); + result.append("List of files in CommitLog directory:\n"); + for (File f: new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()) + { + result.append(String.format("\t%s\n", f.getAbsolutePath())); + } + debugLog(result.toString()); + } + + /** + * Used during test debug to differentiate output visually + */ + static void debugLog(String input) + { + logger.debug("\n\n**************** [TEST DEBUG] *****************\n" + + input + + System.lineSeparator() + + "***************************************************\n\n" + + System.lineSeparator()); + } +} diff --git a/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java new file mode 100644 index 000000000000..070488856a58 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.commitlog; + +import java.io.File; +import java.nio.ByteBuffer; +import java.util.Random; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLTester; + +/* Forces CDC enabled on the run of these tests as they aren't used in non-CDC context and ResumableCommitLogReader + * class asserts cdc enabled on creation. + */ +public class ResumableCommitLogReaderTest extends CQLTester +{ + private Random random = new Random(); + + @BeforeClass + public static void setUpClass() + { + DatabaseDescriptor.setCDCEnabled(true); + CQLTester.setUpClass(); + } + + @Before + public void beforeTest() throws Throwable + { + super.beforeTest(); + + // Need to clean out any files from previous test runs. Prevents flaky test failures. + CommitLog.instance.stopUnsafe(true); + CommitLog.instance.start(); + + // For each test, we start with the assumption of a populated set of a few files we can pull from. + createTable("CREATE TABLE %s (a int, b int, c double, d decimal, e smallint, f tinyint, g blob, primary key (a, b))"); + + byte[] bBlob = new byte[1024 * 1024]; + CommitLog.instance.sync(true); + + // Populate some CommitLog segments on disk + for (int i = 0; i < 20; i++) + { + random.nextBytes(bBlob); + + logger.debug(String.format("Executing insert for index: [%d]", i)); + execute("INSERT INTO %s (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?)", + random.nextInt(), + random.nextInt(), + random.nextDouble(), + random.nextLong(), + (short)random.nextInt(), + (byte)random.nextInt(), + ByteBuffer.wrap(bBlob)); + } + CommitLog.instance.sync(true); + + // Should have well more than 3 segments to work with on subsequent tests. + Assert.assertTrue(CommitLog.instance.segmentManager.getUnflushedSegments().size() > 3); + } + + /** + * Expect operation as though non-resumable, read file to end and complete. + */ + @Test + public void testNonResumedGeneralCase() + { + File writtenFile = CommitLogTestUtils.getFilledCommitLogFile(); + CommitLogTestUtils.debugLog(String.format("Got written CL segment: %s", writtenFile.toString())); + CommitLogTestUtils.debugLog(String.format("Active segment writing to is: %s", CommitLog.instance.segmentManager.getActiveSegment().logFile.toString())); + CommitLogTestUtils.listCommitLogFiles("Checking total CL files:"); + } + + /** + * Expect operation to pick up and feed total mutation # consistent w/mutations in CL + */ + @Test + public void testSingleResumeCase() + { + } + + /** + * Expect operation to pick up and feed total mutation # consistent w/mutations in CL, no errors. + */ + @Test + public void testResumeAtFileEndOffset() + { + } + + /** + * If the offset provided by the user is past the end of the file itself, we expect an RTE + */ + @Test + public void testOffsetPastEnd() throws RuntimeException + { + } + + /** + * If the offset provided by the user is < 0, we expect an RTE + */ + @Test + public void testNegativeOffset() throws RuntimeException + { + } + + /** + * If the offset is at Zero, more or less expect no-op since the user didn't ask us to read anything. Worth allowing + * this behavior in case offset / etc is written by someone to periodically poll and read to the CDC synced offset, so + * 0 is a valid / expected behavior in that case. + */ + @Test + public void testOffsetAtZero() + { + } + + /** + * Expect RTE if it's a corrupt input file. + */ + @Test + public void testCorruptInputFile() + { + } + + /** + * Expect RTE if the input file is completely missing. + */ + @Test + public void testMissingInputFile() + { + } + + /** + * Expect RTE if the input file exists and isn't a CLR. Overlaps w/"Corrupt", but want to make sure failure on header + * parsing is handled as well as failure once header is validated but something blows up mid-file. + */ + @Test + public void testWrongFile() + { + } +} From d82c3b51c4a36a342ac05fd9f6e3aa8d657ed859 Mon Sep 17 00:00:00 2001 From: Josh McKenzie Date: Sun, 13 Oct 2019 14:00:00 -0400 Subject: [PATCH 75/78] Disambiguate some names in CommitLogSegmentManager Mostly consists of method renames to clarify what we're flushing and on what conditions in either naming, javadoc, or a combination of both. A touch of cleaning up unused imports too, but no functional changes. --- .../cassandra/db/ColumnFamilyStore.java | 14 ++--- .../org/apache/cassandra/db/Keyspace.java | 4 +- .../apache/cassandra/db/SystemKeyspace.java | 2 +- .../cassandra/db/commitlog/CommitLog.java | 8 +-- .../db/commitlog/CommitLogReplayer.java | 2 +- .../db/commitlog/CommitLogSegmentManager.java | 41 +++++++++------ .../db/repair/PendingAntiCompaction.java | 2 +- .../db/streaming/CassandraStreamReceiver.java | 4 +- .../apache/cassandra/db/view/TableViews.java | 3 +- .../apache/cassandra/db/view/ViewBuilder.java | 2 +- .../index/SecondaryIndexManager.java | 2 +- .../index/internal/CassandraIndex.java | 9 ++-- .../repair/SystemDistributedKeyspace.java | 2 +- .../repair/consistent/LocalSessions.java | 2 +- .../cassandra/schema/SchemaKeyspace.java | 2 +- .../cassandra/service/StorageService.java | 6 +-- .../cassandra/streaming/StreamSession.java | 2 +- .../test/RepairDigestTrackingTest.java | 9 ++-- .../db/commitlog/CommitLogStressTest.java | 5 +- .../db/compaction/LongCompactionsTest.java | 2 +- .../LongLeveledCompactionStrategyTest.java | 6 +-- .../test/microbench/CompactionBench.java | 4 +- .../ZeroCopyStreamingBenchmark.java | 2 +- test/unit/org/apache/cassandra/Util.java | 2 +- .../batchlog/BatchlogManagerTest.java | 6 +-- .../cassandra/cache/AutoSavingCacheTest.java | 2 +- .../org/apache/cassandra/cql3/CQLTester.java | 2 +- .../cassandra/cql3/GcCompactionTest.java | 4 +- .../cassandra/cql3/KeyCacheCqlTest.java | 4 +- .../apache/cassandra/cql3/OutOfSpaceTest.java | 4 +- .../cassandra/cql3/ViewComplexTest.java | 4 +- .../cassandra/cql3/ViewFilteringTest.java | 2 +- .../org/apache/cassandra/cql3/ViewTest.java | 14 ++--- .../miscellaneous/CrcCheckChanceTest.java | 10 ++-- .../SSTableMetadataTrackingTest.java | 14 ++--- .../org/apache/cassandra/db/CleanupTest.java | 4 +- .../cassandra/db/CleanupTransientTest.java | 2 +- .../cassandra/db/ColumnFamilyMetricTest.java | 2 +- .../cassandra/db/ColumnFamilyStoreTest.java | 19 ++++--- .../cassandra/db/DeletePartitionTest.java | 4 +- .../org/apache/cassandra/db/ImportTest.java | 46 +++++++--------- .../org/apache/cassandra/db/KeyCacheTest.java | 8 +-- .../org/apache/cassandra/db/KeyspaceTest.java | 22 ++++---- .../org/apache/cassandra/db/NameSortTest.java | 2 +- .../cassandra/db/PartitionRangeReadTest.java | 6 +-- .../cassandra/db/RangeTombstoneTest.java | 46 ++++++++-------- .../apache/cassandra/db/ReadCommandTest.java | 24 ++++----- .../db/RecoveryManagerFlushedTest.java | 2 +- .../apache/cassandra/db/RemoveCellTest.java | 2 +- .../org/apache/cassandra/db/RowCacheTest.java | 2 +- .../apache/cassandra/db/RowIterationTest.java | 8 +-- .../org/apache/cassandra/db/ScrubTest.java | 10 ++-- .../cassandra/db/SecondaryIndexTest.java | 8 +-- .../db/SinglePartitionReadCommandCQLTest.java | 4 +- .../db/SinglePartitionSliceCommandTest.java | 10 ++-- .../org/apache/cassandra/db/TimeSortTest.java | 4 +- .../org/apache/cassandra/db/VerifyTest.java | 7 +-- .../SSTableReverseIteratorTest.java | 2 +- .../db/commitlog/CommitLogCQLTest.java | 4 +- .../db/commitlog/CommitLogReaderTest.java | 2 +- .../CommitLogSegmentAllocatorCDCTest.java | 2 +- .../CommitLogSegmentBackpressureTest.java | 8 +-- .../cassandra/db/commitlog/CommitLogTest.java | 22 ++++---- .../db/commitlog/CommitLogTestUtils.java | 2 +- .../ResumableCommitLogReaderTest.java | 2 +- .../db/commitlog/SnapshotDeletingTest.java | 2 +- .../AbstractCompactionStrategyTest.java | 2 +- .../compaction/AbstractPendingRepairTest.java | 2 +- .../db/compaction/ActiveCompactionsTest.java | 11 ++-- .../compaction/AntiCompactionBytemanTest.java | 2 +- .../db/compaction/AntiCompactionTest.java | 5 +- .../BlacklistingCompactionsTest.java | 2 +- .../compaction/CompactionAwareWriterTest.java | 2 +- .../compaction/CompactionControllerTest.java | 13 +++-- .../CompactionStrategyManagerTest.java | 2 +- .../db/compaction/CompactionTaskTest.java | 12 ++--- .../db/compaction/CompactionsBytemanTest.java | 8 +-- .../db/compaction/CompactionsCQLTest.java | 26 +++++----- .../db/compaction/CompactionsPurgeTest.java | 52 +++++++++---------- .../db/compaction/CompactionsTest.java | 20 +++---- .../DateTieredCompactionStrategyTest.java | 16 +++--- .../LeveledCompactionStrategyTest.java | 18 +++---- .../db/compaction/NeverPurgeTest.java | 6 +-- .../db/compaction/OneCompactionTest.java | 2 +- .../compaction/SingleSSTableLCSTaskTest.java | 6 +-- .../SizeTieredCompactionStrategyTest.java | 4 +- .../db/compaction/TTLExpiryTest.java | 28 +++++----- .../TimeWindowCompactionStrategyTest.java | 16 +++--- .../AbstractPendingAntiCompactionTest.java | 2 +- ...onManagerGetSSTablesForValidationTest.java | 2 +- .../db/repair/PendingAntiCompactionTest.java | 4 +- .../rows/ThrottledUnfilteredIteratorTest.java | 8 +-- ...assandraEntireSSTableStreamWriterTest.java | 2 +- .../streaming/CassandraOutgoingFileTest.java | 2 +- .../streaming/CassandraStreamManagerTest.java | 2 +- .../db/view/ViewBuilderTaskTest.java | 2 +- .../cassandra/index/CustomIndexTest.java | 10 ++-- .../index/internal/CustomCassandraIndex.java | 8 +-- .../cassandra/index/sasi/SASIIndexTest.java | 40 +++++++------- .../CompressedSequentialWriterReopenTest.java | 4 +- .../io/sstable/IndexSummaryManagerTest.java | 9 ++-- .../IndexSummaryRedistributionTest.java | 2 +- .../SSTableCorruptionDetectionTest.java | 2 +- .../io/sstable/SSTableLoaderTest.java | 8 +-- .../io/sstable/SSTableMetadataTest.java | 12 ++--- .../io/sstable/SSTableReaderTest.java | 29 +++++------ .../io/sstable/SSTableRewriterTest.java | 10 +--- .../io/sstable/SSTableScannerTest.java | 6 +-- .../big/BigTableZeroCopyWriterTest.java | 2 +- .../cassandra/repair/ValidatorTest.java | 7 ++- .../schema/MigrationManagerTest.java | 12 ++--- .../service/ActiveRepairServiceTest.java | 2 +- .../cassandra/service/ClientWarningsTest.java | 4 +- .../streaming/StreamTransferTaskTest.java | 4 +- .../streaming/StreamingTransferTest.java | 4 +- 115 files changed, 460 insertions(+), 485 deletions(-) diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 075ebac486e9..f20d74c1d702 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -285,7 +285,7 @@ protected void runMayThrow() else { // we'll be rescheduled by the constructor of the Memtable. - forceFlush(); + forceFlushToSSTable(); } } } @@ -858,7 +858,7 @@ private void logFlush() * @return a Future yielding the commit log position that can be guaranteed to have been successfully written * to sstables for this table once the future completes */ - public ListenableFuture forceFlush() + public ListenableFuture forceFlushToSSTable() { synchronized (data) { @@ -877,7 +877,7 @@ public ListenableFuture forceFlush() * @return a Future yielding the commit log position that can be guaranteed to have been successfully written * to sstables for this table once the future completes */ - public ListenableFuture forceFlush(CommitLogPosition flushIfDirtyBefore) + public ListenableFuture forceFlushToSSTable(CommitLogPosition flushIfDirtyBefore) { // we don't loop through the remaining memtables since here we only care about commit log dirtiness // and this does not vary between a table and its table-backed indexes @@ -904,9 +904,9 @@ private ListenableFuture waitForFlushes() return task; } - public CommitLogPosition forceBlockingFlush() + public CommitLogPosition forceBlockingFlushToSSTable() { - return FBUtilities.waitOnFuture(forceFlush()); + return FBUtilities.waitOnFuture(forceFlushToSSTable()); } /** @@ -1886,7 +1886,7 @@ public Set snapshot(String snapshotName, Predicate { if (!skipFlush) { - forceBlockingFlush(); + forceBlockingFlushToSSTable(); } return snapshotWithoutFlush(snapshotName, predicate, ephemeral); } @@ -2110,7 +2110,7 @@ public void truncateBlocking() if (keyspace.getMetadata().params.durableWrites || DatabaseDescriptor.isAutoSnapshot()) { - replayAfter = forceBlockingFlush(); + replayAfter = forceBlockingFlushToSSTable(); viewManager.forceBlockingFlush(); } else diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index 4c1c2cf5448d..fa1f7c87debb 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -398,7 +398,7 @@ public void dropCf(TableId tableId) // disassociate a cfs from this keyspace instance. private void unloadCf(ColumnFamilyStore cfs) { - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.invalidate(); } @@ -674,7 +674,7 @@ public List> flush() { List> futures = new ArrayList<>(columnFamilyStores.size()); for (ColumnFamilyStore cfs : columnFamilyStores.values()) - futures.add(cfs.forceFlush()); + futures.add(cfs.forceFlushToSSTable()); return futures; } diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java index 0d79ae95e22d..4c586f407452 100644 --- a/src/java/org/apache/cassandra/db/SystemKeyspace.java +++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java @@ -800,7 +800,7 @@ public static void forceBlockingFlush(String ...cfnames) for (String cfname : cfnames) { - futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(cfname).forceFlush()); + futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(cfname).forceFlushToSSTable()); } FBUtilities.waitOnFutures(futures); } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java index 34f2f12bc614..1b3fe2e3ad97 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java @@ -306,7 +306,7 @@ public void discardCompletedSegments(final TableId id, final CommitLogPosition l // flushed CF as clean, until we reach the segment file containing the CommitLogPosition passed // in the arguments. Any segments that become unused after they are marked clean will be // recycled or discarded. - for (Iterator iter = segmentManager.getUnflushedSegments().iterator(); iter.hasNext();) + for (Iterator iter = segmentManager.getSegmentsForUnflushedTables().iterator(); iter.hasNext();) { CommitLogSegment segment = iter.next(); segment.markClean(id, lowerBound, upperBound); @@ -362,7 +362,7 @@ public String getRestorePrecision() public List getActiveSegmentNames() { - Collection segments = segmentManager.getUnflushedSegments(); + Collection segments = segmentManager.getSegmentsForUnflushedTables(); List segmentNames = new ArrayList<>(segments.size()); for (CommitLogSegment seg : segments) segmentNames.add(seg.getName()); @@ -378,7 +378,7 @@ public List getArchivingSegmentNames() public long getActiveContentSize() { long size = 0; - for (CommitLogSegment seg : segmentManager.getUnflushedSegments()) + for (CommitLogSegment seg : segmentManager.getSegmentsForUnflushedTables()) size += seg.contentSize(); return size; } @@ -393,7 +393,7 @@ public long getActiveOnDiskSize() public Map getActiveSegmentCompressionRatios() { Map segmentRatios = new TreeMap<>(); - for (CommitLogSegment seg : segmentManager.getUnflushedSegments()) + for (CommitLogSegment seg : segmentManager.getSegmentsForUnflushedTables()) segmentRatios.put(seg.getName(), 1.0 * seg.onDiskSize() / seg.contentSize()); return segmentRatios; } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java index ca8eb4f123d4..f63499ab23c0 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java @@ -217,7 +217,7 @@ public int blockForWrites() // also flush batchlog incase of any MV updates if (!flushingSystem) - futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlush()); + futures.add(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlushToSSTable()); FBUtilities.waitOnFutures(futures); diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java index 3aad2f323417..50d80862f65d 100755 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java @@ -133,8 +133,8 @@ public void runMayThrow() throws Exception continue; // Writing threads are not waiting for new segments, we can spend time on other tasks. - // flush old Cfs if we're full - maybeFlushToReclaim(); + // Flush old Cfs if we're full. + flushIfOverLimit(); } catch (Throwable t) { @@ -167,10 +167,13 @@ private boolean atSegmentBufferLimit() return CommitLogSegment.usesBufferPool(commitLog) && bufferPool.atLimit(); } - private void maybeFlushToReclaim() + /** + * In the event we've overallocated (i.e. size on disk > limit in config), we want to trigger a flush of the number + * of required memtables to sstables in order to be able to reclaim some CL space on disk. + */ + private void flushIfOverLimit() { - long unused = unusedCapacity(); - if (unused < 0) + if (overConfigDiskCapacity(0)) { long flushingSize = 0; List segmentsToRecycle = new ArrayList<>(); @@ -180,10 +183,10 @@ private void maybeFlushToReclaim() break; flushingSize += segment.onDiskSize(); segmentsToRecycle.add(segment); - if (flushingSize + unused >= 0) + if (!overConfigDiskCapacity((flushingSize))) break; } - flushDataFrom(segmentsToRecycle, false); + flushTablesForSegments(segmentsToRecycle, false); } } @@ -272,7 +275,7 @@ void forceRecycleAll(Iterable droppedTables) Keyspace.writeOrder.awaitNewBarrier(); // flush and wait for all CFs that are dirty in segments up-to and including 'last' - Future future = flushDataFrom(segmentsToRecycle, true); + Future future = flushTablesForSegments(segmentsToRecycle, true); try { future.get(); @@ -343,12 +346,16 @@ public long onDiskSize() return size.get(); } - private long unusedCapacity() + /** + * We offset by the amount we've planned to flush with to allow for selective calculation up front of how much to flush + */ + private boolean overConfigDiskCapacity(long toBeFlushed) { long total = DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024; - long currentSize = size.get(); + long currentSize = size.get() + toBeFlushed; logger.trace("Total active commitlog segment space used is {} out of {}", currentSize, total); - return total - currentSize; + // TODO: Consider whether to do >=. Original logic strictly equated with > from CASSANDRA-9095 + return currentSize > total; } /** @@ -356,7 +363,7 @@ private long unusedCapacity() * * @return a Future that will finish when all the flushes are complete. */ - private Future flushDataFrom(List segments, boolean force) + private Future flushTablesForSegments(List segments, boolean force) { if (segments.isEmpty()) return Futures.immediateFuture(null); @@ -382,7 +389,7 @@ else if (!flushes.containsKey(dirtyTableId)) final ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(dirtyTableId); // can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush, // no deadlock possibility since switchLock removal - flushes.put(dirtyTableId, force ? cfs.forceFlush() : cfs.forceFlush(maxCommitLogPosition)); + flushes.put(dirtyTableId, force ? cfs.forceFlushToSSTable() : cfs.forceFlushToSSTable(maxCommitLogPosition)); } } } @@ -486,10 +493,12 @@ public void awaitTermination() throws InterruptedException } /** - * @return a read-only collection of all active and unflushed segments in the system + * @return a read-only collection of all active and unflushed segments in the system. In this context, "Flushed" is + * referring to "memtable / CF flushed to sstables", not whether or not the CommitLogSegment itself is flushed via + * fsync. */ @VisibleForTesting - public Collection getUnflushedSegments() + public Collection getSegmentsForUnflushedTables() { return Collections.unmodifiableCollection(unflushedSegments); } @@ -510,7 +519,7 @@ CommitLogPosition getCurrentPosition() public void sync(boolean flush) throws IOException { CommitLogSegment current = activeSegment; - for (CommitLogSegment segment : getUnflushedSegments()) + for (CommitLogSegment segment : getSegmentsForUnflushedTables()) { // Do not sync segments that became active after sync started. if (segment.id > current.id) diff --git a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java index fac164d006a5..85d262566290 100644 --- a/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java +++ b/src/java/org/apache/cassandra/db/repair/PendingAntiCompaction.java @@ -349,7 +349,7 @@ public ListenableFuture run() List> tasks = new ArrayList<>(tables.size()); for (ColumnFamilyStore cfs : tables) { - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ListenableFutureTask task = ListenableFutureTask.create(getAcquisitionCallable(cfs, tokenRanges.ranges(), prsId, acquireRetrySeconds, acquireSleepMillis)); executor.submit(task); tasks.add(task); diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index b2b2ce5cf093..8338a178361a 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -28,7 +28,7 @@ import org.apache.cassandra.db.lifecycle.LifecycleNewTracker; import org.apache.cassandra.io.sstable.SSTable; -import org.apache.cassandra.streaming.StreamReceiveTask; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -273,7 +273,7 @@ public void cleanup() // the streamed sstables. if (requiresWritePath) { - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); abort(); } } diff --git a/src/java/org/apache/cassandra/db/view/TableViews.java b/src/java/org/apache/cassandra/db/view/TableViews.java index 09490e8bd7ec..9d4d997a774b 100644 --- a/src/java/org/apache/cassandra/db/view/TableViews.java +++ b/src/java/org/apache/cassandra/db/view/TableViews.java @@ -23,7 +23,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; @@ -96,7 +95,7 @@ public Iterable allViewsCfs() public void forceBlockingFlush() { for (ColumnFamilyStore viewCfs : allViewsCfs()) - viewCfs.forceBlockingFlush(); + viewCfs.forceBlockingFlushToSSTable(); } public void dumpMemtables() diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java index 67172973ee82..6d92c73cd739 100644 --- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java +++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java @@ -96,7 +96,7 @@ public void start() logger.debug("Starting build of view({}.{}). Flushing base table {}.{}", ksName, view.name, ksName, baseCfs.name); - baseCfs.forceBlockingFlush(); + baseCfs.forceBlockingFlushToSSTable(); loadStatusAndBuild(); } diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java index 60fc3ba314e3..779ea7c68857 100644 --- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java +++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java @@ -788,7 +788,7 @@ private void flushIndexesBlocking(Set indexes, FutureCallback cal { indexes.forEach(index -> index.getBackingTable() - .map(cfs -> wait.add(cfs.forceFlush())) + .map(cfs -> wait.add(cfs.forceFlushToSSTable())) .orElseGet(() -> nonCfsIndexes.add(index))); } diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java index 58056b9d866b..d1feb7917d1a 100644 --- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java +++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java @@ -61,7 +61,6 @@ import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Pair; -import org.apache.cassandra.utils.concurrent.OpOrder; import org.apache.cassandra.utils.concurrent.Refs; import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse; @@ -186,7 +185,7 @@ public Optional getBackingTable() public Callable getBlockingFlushTask() { return () -> { - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlushToSSTable(); return null; }; } @@ -663,7 +662,7 @@ private void invalidate() CompactionManager.instance.interruptCompactionForCFs(cfss, (sstable) -> true, true); CompactionManager.instance.waitForCessation(cfss, (sstable) -> true); Keyspace.writeOrder.awaitNewBarrier(); - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlushToSSTable(); indexCfs.readOrdering.awaitNewBarrier(); indexCfs.invalidate(); } @@ -689,7 +688,7 @@ private Callable getBuildIndexTask() @SuppressWarnings("resource") private void buildBlocking() { - baseCfs.forceBlockingFlush(); + baseCfs.forceBlockingFlushToSSTable(); try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL)); Refs sstables = viewFragment.refs) @@ -713,7 +712,7 @@ private void buildBlocking() ImmutableSet.copyOf(sstables)); Future future = CompactionManager.instance.submitIndexBuild(builder); FBUtilities.waitOnFuture(future); - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlushToSSTable(); } logger.info("Index build of {} complete", metadata.name); } diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java index 645694e8cc0c..650a80a3ee53 100644 --- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java +++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java @@ -352,7 +352,7 @@ private static void processSilent(String fmtQry, String... values) public static void forceBlockingFlush(String table) { if (!DatabaseDescriptor.isUnsafeSystem()) - FBUtilities.waitOnFuture(Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(table).forceFlush()); + FBUtilities.waitOnFuture(Keyspace.open(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME).getColumnFamilyStore(table).forceFlushToSSTable()); } private enum RepairState diff --git a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java index 935bba8dfc9e..b6103c4ea985 100644 --- a/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java +++ b/src/java/org/apache/cassandra/repair/consistent/LocalSessions.java @@ -416,7 +416,7 @@ private void syncTable() { TableId tid = Schema.instance.getTableMetadata(keyspace, table).id; ColumnFamilyStore cfm = Schema.instance.getColumnFamilyStoreInstance(tid); - cfm.forceBlockingFlush(); + cfm.forceBlockingFlushToSSTable(); } /** diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index 3c3e06313284..5498d6eafd7b 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -345,7 +345,7 @@ public static void truncate() private static void flush() { if (!DatabaseDescriptor.isUnsafeSystem()) - ALL.forEach(table -> FBUtilities.waitOnFuture(getSchemaCFS(table).forceFlush())); + ALL.forEach(table -> FBUtilities.waitOnFuture(getSchemaCFS(table).forceFlushToSSTable())); } /** diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index ae83e9229107..a8ae42c2f0c7 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -3734,7 +3734,7 @@ public void forceKeyspaceFlush(String keyspaceName, String... tableNames) throws for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tableNames)) { logger.debug("Forcing flush on keyspace {}, CF {}", keyspaceName, cfStore.name); - cfStore.forceBlockingFlush(); + cfStore.forceBlockingFlushToSSTable(); } } @@ -4651,7 +4651,7 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I for (Keyspace keyspace : Keyspace.nonSystem()) { for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) - flushes.add(cfs.forceFlush()); + flushes.add(cfs.forceFlushToSSTable()); } // wait for the flushes. // TODO this is a godawful way to track progress, since they flush in parallel. a long one could @@ -4683,7 +4683,7 @@ protected synchronized void drain(boolean isFinalShutdown) throws IOException, I for (Keyspace keyspace : Keyspace.system()) { for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) - flushes.add(cfs.forceFlush()); + flushes.add(cfs.forceFlushToSSTable()); } FBUtilities.waitOnFutures(flushes); diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java index 95d3755d6c43..cad1ba708db4 100644 --- a/src/java/org/apache/cassandra/streaming/StreamSession.java +++ b/src/java/org/apache/cassandra/streaming/StreamSession.java @@ -780,7 +780,7 @@ private void flushSSTables(Iterable stores) { List> flushes = new ArrayList<>(); for (ColumnFamilyStore cfs : stores) - flushes.add(cfs.forceFlush()); + flushes.add(cfs.forceFlushToSSTable()); FBUtilities.waitOnFutures(flushes); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java b/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java index a987ea30cd81..0c394550f775 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/RepairDigestTrackingTest.java @@ -22,7 +22,6 @@ import java.io.Serializable; import java.util.EnumSet; import java.util.Map; -import java.util.Set; import org.junit.Assert; import org.junit.Test; @@ -60,10 +59,10 @@ public void testInconsistenciesFound() throws Throwable } cluster.get(1).runOnInstance(() -> - Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlush() + Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlushToSSTable() ); cluster.get(2).runOnInstance(() -> - Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlush() + Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlushToSSTable() ); for (int i = 10; i < 20; i++) @@ -74,10 +73,10 @@ public void testInconsistenciesFound() throws Throwable } cluster.get(1).runOnInstance(() -> - Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlush() + Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlushToSSTable() ); cluster.get(2).runOnInstance(() -> - Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlush() + Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl").forceBlockingFlushToSSTable() ); cluster.get(1).runOnInstance(() -> diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java index 3749e11031b4..8f217439d3c1 100644 --- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java +++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java @@ -35,15 +35,12 @@ import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; import io.netty.util.concurrent.FastThreadLocalThread; import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.Util; import org.apache.cassandra.UpdateBuilder; -import org.apache.cassandra.config.Config.CommitLogSync; import org.apache.cassandra.config.*; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.marshal.UTF8Type; @@ -287,7 +284,7 @@ private void verifySizes(CommitLog commitLog) List logFileNames = commitLog.getActiveSegmentNames(); Map ratios = commitLog.getActiveSegmentCompressionRatios(); - Collection segments = commitLog.segmentManager.getUnflushedSegments(); + Collection segments = commitLog.segmentManager.getSegmentsForUnflushedTables(); for (CommitLogSegment segment : segments) { diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java index fe8cdc2fd92b..6e247936a9ae 100644 --- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java +++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java @@ -165,7 +165,7 @@ public void testStandardColumnCompactions() inserted.add(key); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); CompactionsTest.assertMaxTimestamp(cfs, maxTimestampExpected); assertEquals(inserted.toString(), inserted.size(), Util.getAll(Util.cmd(cfs).build()).size()); diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java index f8f94a0aadfa..62c4099d1803 100644 --- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java +++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java @@ -29,7 +29,6 @@ import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.apache.cassandra.SchemaLoader; @@ -40,7 +39,6 @@ import org.apache.cassandra.schema.CompactionParams; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.service.ActiveRepairService; -import org.apache.cassandra.service.StorageService; import org.apache.cassandra.utils.FBUtilities; import static org.junit.Assert.assertFalse; @@ -164,7 +162,7 @@ public void testLeveledScanner() throws Exception } //Flush sstable - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); store.runWithCompactionsDisabled(new Callable() { @@ -263,7 +261,7 @@ private void populateSSTables(ColumnFamilyStore store) Mutation rm = new Mutation(builder.build()); rm.apply(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); } } } diff --git a/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java b/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java index 41220a2a655c..4b817305439e 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java +++ b/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java @@ -70,13 +70,13 @@ public void setup() throws Throwable execute(writeStatement, i, i, i ); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); System.err.println("Writing 50k again..."); for (long i = 0; i < 50000; i++) execute(writeStatement, i, i, i ); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.snapshot("originals"); diff --git a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java index 2b642a83dd81..72571e8dc98c 100644 --- a/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java +++ b/test/microbench/org/apache/cassandra/test/microbench/ZeroCopyStreamingBenchmark.java @@ -200,7 +200,7 @@ private void generateData() .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); } diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java index 3dcaff7abb2a..0e4611a48aed 100644 --- a/test/unit/org/apache/cassandra/Util.java +++ b/test/unit/org/apache/cassandra/Util.java @@ -188,7 +188,7 @@ public static ColumnFamilyStore writeColumnFamily(List mutations) rm.applyUnsafe(); ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(tableId); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); return store; } diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java index 361759f11b9a..5ab7597e4a99 100644 --- a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java +++ b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java @@ -156,7 +156,7 @@ public void testReplay() throws Exception } // Flush the batchlog to disk (see CASSANDRA-6822). - Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceBlockingFlush(); + Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceBlockingFlushToSSTable(); assertEquals(100, BatchlogManager.instance.countAllBatches() - initialAllBatches); assertEquals(0, BatchlogManager.instance.getTotalBatchesReplayed() - initialReplayedBatches); @@ -240,7 +240,7 @@ public void testTruncatedReplay() throws InterruptedException, ExecutionExceptio } // Flush the batchlog to disk (see CASSANDRA-6822). - Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceBlockingFlush(); + Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceBlockingFlushToSSTable(); // Force batchlog replay and wait for it to complete. BatchlogManager.instance.startBatchlogReplay().get(); @@ -368,7 +368,7 @@ public void testReplayWithNoPeers() throws Exception assertEquals(1, BatchlogManager.instance.countAllBatches() - initialAllBatches); // Flush the batchlog to disk (see CASSANDRA-6822). - Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceBlockingFlush(); + Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceBlockingFlushToSSTable(); assertEquals(1, BatchlogManager.instance.countAllBatches() - initialAllBatches); assertEquals(0, BatchlogManager.instance.getTotalBatchesReplayed() - initialReplayedBatches); diff --git a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java index bb5129af9a46..7dd26b1478c0 100644 --- a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java +++ b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java @@ -74,7 +74,7 @@ private static void doTestSerializeAndLoadKeyCache() throws Exception RowUpdateBuilder rowBuilder = new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), "key1"); rowBuilder.add(colDef, "val1"); rowBuilder.build().apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } Assert.assertEquals(2, cfs.getLiveSSTables().size()); diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 9c4f22e51ee0..85f8bdfc6f3b 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -474,7 +474,7 @@ public void flush(String keyspace) { ColumnFamilyStore store = getCurrentColumnFamilyStore(keyspace); if (store != null) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); } public void disableCompaction(String keyspace) diff --git a/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java b/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java index 2fc07eb89db5..375ffcaa7a3e 100644 --- a/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java +++ b/test/unit/org/apache/cassandra/cql3/GcCompactionTest.java @@ -346,11 +346,11 @@ public void testLocalDeletionTime() throws Throwable createTable("create table %s (k int, c1 int, primary key (k, c1)) with compaction = {'class': 'SizeTieredCompactionStrategy', 'provide_overlapping_tombstones':'row'}"); execute("delete from %s where k = 1"); Set readers = new HashSet<>(getCurrentColumnFamilyStore().getLiveSSTables()); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); SSTableReader oldSSTable = getNewTable(readers); Thread.sleep(2000); execute("delete from %s where k = 1"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); SSTableReader newTable = getNewTable(readers); CompactionManager.instance.forceUserDefinedCompaction(oldSSTable.getFilename()); diff --git a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java index b76cc784396c..1268bb002eb8 100644 --- a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java +++ b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java @@ -543,7 +543,7 @@ private void insertData(String table, String index, boolean withClustering) thro if (i % 10 == 9) { - Keyspace.open(KEYSPACE_PER_TEST).getColumnFamilyStore(table).forceFlush().get(); + Keyspace.open(KEYSPACE_PER_TEST).getColumnFamilyStore(table).forceFlushToSSTable().get(); if (index != null) triggerBlockingFlush(Keyspace.open(KEYSPACE_PER_TEST).getColumnFamilyStore(table).indexManager.getIndexByName(index)); } @@ -553,7 +553,7 @@ private void insertData(String table, String index, boolean withClustering) thro private static void prepareTable(String table) throws IOException, InterruptedException, java.util.concurrent.ExecutionException { StorageService.instance.disableAutoCompaction(KEYSPACE_PER_TEST, table); - Keyspace.open(KEYSPACE_PER_TEST).getColumnFamilyStore(table).forceFlush().get(); + Keyspace.open(KEYSPACE_PER_TEST).getColumnFamilyStore(table).forceFlushToSSTable().get(); Keyspace.open(KEYSPACE_PER_TEST).getColumnFamilyStore(table).truncateBlocking(); } diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java index 3224e7f940fd..cd86ce313da6 100644 --- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java +++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java @@ -115,7 +115,7 @@ public void flushAndExpectError() throws InterruptedException, ExecutionExceptio { try { - Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get(); + Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlushToSSTable().get(); fail("FSWriteError expected."); } catch (ExecutionException e) @@ -126,7 +126,7 @@ public void flushAndExpectError() throws InterruptedException, ExecutionExceptio // Make sure commit log wasn't discarded. TableId tableId = currentTableMetadata().id; - for (CommitLogSegment segment : CommitLog.instance.segmentManager.getUnflushedSegments()) + for (CommitLogSegment segment : CommitLog.instance.segmentManager.getSegmentsForUnflushedTables()) if (segment.getDirtyTableIds().contains(tableId)) return; fail("Expected commit log to remain dirty for the affected table."); diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java index d24ab526385a..44c20d010b63 100644 --- a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java +++ b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java @@ -844,8 +844,8 @@ private void testExpiredLivenessLimit(boolean flush) throws Throwable } if (flush) { - ks.getColumnFamilyStore("mv1").forceBlockingFlush(); - ks.getColumnFamilyStore("mv2").forceBlockingFlush(); + ks.getColumnFamilyStore("mv1").forceBlockingFlushToSSTable(); + ks.getColumnFamilyStore("mv2").forceBlockingFlushToSSTable(); } for (String view : Arrays.asList("mv1", "mv2")) diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java index 8b4a556b722a..73b50744299b 100644 --- a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java +++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java @@ -2121,7 +2121,7 @@ public void testOldTimestampsWithRestrictions() throws Throwable for (int i = 0; i < 100; i++) updateView("INSERT into %s (k,c,val)VALUES(?,?,?)", 0, i % 2, "baz"); - Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush(); + Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlushToSSTable(); Assert.assertEquals(2, execute("select * from %s").size()); Assert.assertEquals(2, execute("select * from mv_tstest").size()); diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java index 02fa19effa09..8647f4670982 100644 --- a/test/unit/org/apache/cassandra/cql3/ViewTest.java +++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java @@ -128,7 +128,7 @@ public void testExistingRangeTombstone(boolean flush) throws Throwable updateView("DELETE FROM %s USING TIMESTAMP 10 WHERE k1 = 1 and c1=1"); if (flush) - Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush(); + Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlushToSSTable(); String table = KEYSPACE + "." + currentTable(); updateView("BEGIN BATCH " + @@ -338,7 +338,7 @@ public void testOldTimestamps() throws Throwable for (int i = 0; i < 100; i++) updateView("INSERT into %s (k,c,val)VALUES(?,?,?)", 0, i % 2, "baz"); - Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush(); + Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlushToSSTable(); Assert.assertEquals(2, execute("select * from %s").size()); Assert.assertEquals(2, execute("select * from mv_tstest").size()); @@ -926,7 +926,7 @@ public void testIgnoreUpdate() throws Throwable assertRows(execute("SELECT a, b, c from mv WHERE b = ?", 1), row(0, 1, null)); ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore("mv"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Assert.assertEquals(1, cfs.getLiveSSTables().size()); } @@ -1350,22 +1350,22 @@ private void testViewBuilderResume(int concurrentViewBuilders) throws Throwable for (int i = 0; i < 1024; i++) execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); for (int i = 0; i < 1024; i++) execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); for (int i = 0; i < 1024; i++) execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); for (int i = 0; i < 1024; i++) execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); String viewName1 = "mv_test_" + concurrentViewBuilders; createView(viewName1, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)"); diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java index 246f512f66b5..dc7e6c0a955e 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java @@ -68,7 +68,7 @@ public void testChangingCrcCheckChance(boolean newFormat) throws Throwable ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(currentTable()); ColumnFamilyStore indexCfs = cfs.indexManager.getAllIndexColumnFamilyStores().iterator().next(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Assert.assertEquals(0.99, cfs.getCrcCheckChance(), 0.0); Assert.assertEquals(0.99, cfs.getLiveSSTables().iterator().next().getCrcCheckChance(), 0.0); @@ -96,19 +96,19 @@ public void testChangingCrcCheckChance(boolean newFormat) throws Throwable execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2"); execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1"); execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2"); execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); execute("INSERT INTO %s(p, c, v, s) values (?, ?, ?, ?)", "p1", "k1", "v1", "sv1"); execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2"); execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.forceMajorCompaction(); //Now let's change via JMX @@ -182,7 +182,7 @@ public void testDropDuringCompaction() throws Throwable execute("INSERT INTO %s(p, c, v) values (?, ?, ?)", "p1", "k2", "v2"); execute("INSERT INTO %s(p, s) values (?, ?)", "p2", "sv2"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } DatabaseDescriptor.setCompactionThroughputMbPerSec(1); diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java index 288cbe1a042e..0b9e71b138fb 100644 --- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java +++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/SSTableMetadataTrackingTest.java @@ -33,7 +33,7 @@ public void baseCheck() throws Throwable createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b))"); ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 9999"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); StatsMetadata metadata = cfs.getLiveSSTables().iterator().next().getSSTableMetadata(); assertEquals(9999, metadata.minTimestamp); assertEquals(Integer.MAX_VALUE, metadata.maxLocalDeletionTime); @@ -50,7 +50,7 @@ public void testMinMaxtimestampRange() throws Throwable ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000"); execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); StatsMetadata metadata = cfs.getLiveSSTables().iterator().next().getSSTableMetadata(); assertEquals(9999, metadata.minTimestamp); assertEquals(10000, metadata.maxTimestamp); @@ -69,7 +69,7 @@ public void testMinMaxtimestampRow() throws Throwable ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); execute("INSERT INTO %s (a,b,c) VALUES (1,1,'1') using timestamp 10000"); execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); StatsMetadata metadata = cfs.getLiveSSTables().iterator().next().getSSTableMetadata(); assertEquals(9999, metadata.minTimestamp); assertEquals(10000, metadata.maxTimestamp); @@ -88,7 +88,7 @@ public void testTrackMetadata_rangeTombstone() throws Throwable createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY (a, b)) WITH gc_grace_seconds = 10000"); ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1 and b = 1"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(1, cfs.getLiveSSTables().size()); StatsMetadata metadata = cfs.getLiveSSTables().iterator().next().getSSTableMetadata(); assertEquals(9999, metadata.minTimestamp); @@ -108,7 +108,7 @@ public void testTrackMetadata_rowTombstone() throws Throwable ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a = 1"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(1, cfs.getLiveSSTables().size()); StatsMetadata metadata = cfs.getLiveSSTables().iterator().next().getSSTableMetadata(); assertEquals(9999, metadata.minTimestamp); @@ -128,7 +128,7 @@ public void testTrackMetadata_rowMarker() throws Throwable ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); execute("INSERT INTO %s (a) VALUES (1) USING TIMESTAMP 9999"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(1, cfs.getLiveSSTables().size()); StatsMetadata metadata = cfs.getLiveSSTables().iterator().next().getSSTableMetadata(); assertEquals(9999, metadata.minTimestamp); @@ -147,7 +147,7 @@ public void testTrackMetadata_rowMarkerDelete() throws Throwable createTable("CREATE TABLE %s (a int, PRIMARY KEY (a))"); ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); execute("DELETE FROM %s USING TIMESTAMP 9999 WHERE a=1"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(1, cfs.getLiveSSTables().size()); StatsMetadata metadata = cfs.getLiveSSTables().iterator().next().getSSTableMetadata(); assertEquals(9999, metadata.minTimestamp); diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java index 996536126bbf..a91a723dd24e 100644 --- a/test/unit/org/apache/cassandra/db/CleanupTest.java +++ b/test/unit/org/apache/cassandra/db/CleanupTest.java @@ -272,7 +272,7 @@ public void testCleanupSkippingSSTables() throws UnknownHostException, Execution .add("val", VALUE) .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } Set beforeFirstCleanup = Sets.newHashSet(cfs.getLiveSSTables()); @@ -407,7 +407,7 @@ protected void fillCF(ColumnFamilyStore cfs, String colName, int rowsPerSSTable) .applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } protected List getMaxTimestampList(ColumnFamilyStore cfs) diff --git a/test/unit/org/apache/cassandra/db/CleanupTransientTest.java b/test/unit/org/apache/cassandra/db/CleanupTransientTest.java index 9789183dc14b..dbce4e1be61e 100644 --- a/test/unit/org/apache/cassandra/db/CleanupTransientTest.java +++ b/test/unit/org/apache/cassandra/db/CleanupTransientTest.java @@ -182,7 +182,7 @@ protected void fillCF(ColumnFamilyStore cfs, String colName, int rowsPerSSTable) .applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } protected List getMaxTimestampList(ColumnFamilyStore cfs) diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java index c016f9ba6446..aeaad46d2d79 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java @@ -61,7 +61,7 @@ public void testSizeMetric() .build() .applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Collection sstables = cfs.getLiveSSTables(); long size = 0; for (SSTableReader reader : sstables) diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java index 888cdc6c8e75..1b8f98646e37 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java @@ -38,7 +38,6 @@ import org.apache.cassandra.db.lifecycle.SSTableSet; import org.apache.cassandra.db.rows.*; import org.apache.cassandra.db.partitions.*; -import org.apache.cassandra.db.marshal.*; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.Descriptor; @@ -101,14 +100,14 @@ public void testTimeSortedQuery() .add("val", "asdf") .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, "key1") .clustering("Column1") .add("val", "asdf") .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ((ClearableHistogram)cfs.metric.sstablesPerReadHistogram.cf).clear(); // resets counts Util.getAll(Util.cmd(cfs, "key1").includeRow("c1").build()); @@ -177,7 +176,7 @@ public void testDeleteStandardRowSticksAfterFlush() throws Throwable assertRangeCount(cfs, col, val, 2); // flush. - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // insert, don't flush new RowUpdateBuilder(cfs.metadata(), 1, "key3").clustering("Column1").add("val", "val1").build().applyUnsafe(); @@ -192,7 +191,7 @@ public void testDeleteStandardRowSticksAfterFlush() throws Throwable assertRangeCount(cfs, col, val, 2); // flush - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // re-verify delete. // first breakage is right here because of CASSANDRA-1837. assertRangeCount(cfs, col, val, 2); @@ -210,7 +209,7 @@ public void testDeleteStandardRowSticksAfterFlush() throws Throwable assertRangeCount(cfs, col, val, 4); // and it remains so after flush. (this wasn't failing before, but it's good to check.) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertRangeCount(cfs, col, val, 4); } @@ -259,9 +258,9 @@ public void testBackupAfterFlush() throws Throwable { ColumnFamilyStore cfs = Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1); new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key1")).clustering("Column1").add("val", "asdf").build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key2")).clustering("Column1").add("val", "asdf").build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); for (int version = 1; version <= 2; ++version) { @@ -400,7 +399,7 @@ public void testBackupAfterFlush() throws Throwable public void reTest(ColumnFamilyStore cfs, Runnable verify) throws Exception { verify.run(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); verify.run(); } @@ -435,7 +434,7 @@ public void testScrubDataDirectories() throws Throwable ColumnFamilyStore.scrubDataDirectories(cfs.metadata()); new RowUpdateBuilder(cfs.metadata(), 2, "key").clustering("name").add("val", "2").build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // Nuke the metadata and reload that sstable Collection ssTables = cfs.getLiveSSTables(); diff --git a/test/unit/org/apache/cassandra/db/DeletePartitionTest.java b/test/unit/org/apache/cassandra/db/DeletePartitionTest.java index 6ed43f726250..f8b31470b6eb 100644 --- a/test/unit/org/apache/cassandra/db/DeletePartitionTest.java +++ b/test/unit/org/apache/cassandra/db/DeletePartitionTest.java @@ -75,7 +75,7 @@ public void testDeletePartition(DecoratedKey key, boolean flushBeforeRemove, boo assertTrue(r.getCell(column).value().equals(ByteBufferUtil.bytes("asdf"))); if (flushBeforeRemove) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); // delete the partition new Mutation.PartitionUpdateCollector(KEYSPACE1, key) @@ -84,7 +84,7 @@ public void testDeletePartition(DecoratedKey key, boolean flushBeforeRemove, boo .applyUnsafe(); if (flushAfterRemove) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); // validate removal ImmutableBTreePartition partitionUnfiltered = Util.getOnlyPartitionUnfiltered(Util.cmd(store, key).build()); diff --git a/test/unit/org/apache/cassandra/db/ImportTest.java b/test/unit/org/apache/cassandra/db/ImportTest.java index 5ceb233a56e4..a1d04ab30a8b 100644 --- a/test/unit/org/apache/cassandra/db/ImportTest.java +++ b/test/unit/org/apache/cassandra/db/ImportTest.java @@ -23,18 +23,14 @@ import java.io.RandomAccessFile; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Random; import java.util.Set; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; import org.junit.Test; @@ -43,8 +39,6 @@ import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.dht.BootStrapper; -import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.dht.Token; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.locator.InetAddressAndPort; @@ -66,7 +60,7 @@ public void basicImportTest() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); @@ -87,14 +81,14 @@ public void basicImportMultiDirTest() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); File backupdir = moveToBackupDir(sstables); for (int i = 10; i < 20; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); @@ -118,7 +112,7 @@ public void refreshTest() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); sstables.forEach(s -> s.selfRef().release()); @@ -133,7 +127,7 @@ public void importResetLevelTest() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); for (SSTableReader sstable : sstables) @@ -170,7 +164,7 @@ public void importClearRepairedTest() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); for (SSTableReader sstable : sstables) @@ -252,7 +246,7 @@ public void testGetCorrectDirectory() throws Throwable for (int i = 0; i < 10; i++) { execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); } Set toMove = getCurrentColumnFamilyStore().getLiveSSTables(); @@ -281,11 +275,11 @@ private void testCorruptHelper(boolean verify) throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); SSTableReader sstableToCorrupt = getCurrentColumnFamilyStore().getLiveSSTables().iterator().next(); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i + 10, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); @@ -302,7 +296,7 @@ private void testCorruptHelper(boolean verify) throws Throwable // now move a correct sstable to another directory to make sure that directory gets properly imported for (int i = 100; i < 130; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set correctSSTables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); @@ -359,7 +353,7 @@ public void testImportOutOfRange() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 1000; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); @@ -404,7 +398,7 @@ public void testImportOutOfRangeExtendedVerify() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 1000; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); @@ -440,7 +434,7 @@ public void testImportInvalidateCache() throws Throwable createTable("create table %s (id int primary key, d int) WITH caching = { 'keys': 'NONE', 'rows_per_partition': 'ALL' }"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); CacheService.instance.setRowCacheCapacityInMB(1); Set keysToInvalidate = new HashSet<>(); @@ -461,7 +455,7 @@ public void testImportInvalidateCache() throws Throwable for (int i = 10; i < 20; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set allCachedKeys = new HashSet<>(); @@ -508,7 +502,7 @@ public void testImportCacheEnabledWithoutSrcDir() throws Throwable createTable("create table %s (id int primary key, d int) WITH caching = { 'keys': 'NONE', 'rows_per_partition': 'ALL' }"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); CacheService.instance.setRowCacheCapacityInMB(1); getCurrentColumnFamilyStore().clearUnsafe(); @@ -525,7 +519,7 @@ public void testRefreshCorrupt() throws Throwable createTable("create table %s (id int primary key, d int) WITH caching = { 'keys': 'NONE', 'rows_per_partition': 'ALL' }"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); sstables.forEach(s -> s.selfRef().release()); @@ -540,10 +534,10 @@ public void testRefreshCorrupt() throws Throwable for (int i = 10; i < 20; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); for (int i = 20; i < 30; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set expectedFiles = new HashSet<>(getCurrentColumnFamilyStore().getLiveSSTables()); @@ -589,14 +583,14 @@ public void importBadDirectoryTest() throws Throwable createTable("create table %s (id int primary key, d int)"); for (int i = 0; i < 10; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); File backupdir = moveToBackupDir(sstables); for (int i = 10; i < 20; i++) execute("insert into %s (id, d) values (?, ?)", i, i); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); sstables = getCurrentColumnFamilyStore().getLiveSSTables(); getCurrentColumnFamilyStore().clearUnsafe(); diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java index 1819b1811844..c1dab8c2ed74 100644 --- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java +++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java @@ -102,7 +102,7 @@ private void testKeyCacheLoad(String cf) throws Exception // insert data and force to disk SchemaLoader.insertData(KEYSPACE1, cf, 0, 100); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); // populate the cache readData(KEYSPACE1, cf, 0, 100); @@ -202,7 +202,7 @@ private void testKeyCacheLoadWithLostTable(String cf) throws Exception // insert data and force to disk SchemaLoader.insertData(KEYSPACE1, cf, 0, 100); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); Collection firstFlushTables = ImmutableList.copyOf(store.getLiveSSTables()); @@ -212,7 +212,7 @@ private void testKeyCacheLoadWithLostTable(String cf) throws Exception // insert some new data and force to disk SchemaLoader.insertData(KEYSPACE1, cf, 100, 50); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); // check that it's fine readData(KEYSPACE1, cf, 100, 50); @@ -273,7 +273,7 @@ private void testKeyCache(String cf) throws ExecutionException, InterruptedExcep new RowUpdateBuilder(cfs.metadata(), 0, "key2").clustering("2").build().applyUnsafe(); // to make sure we have SSTable - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // reads to cache key position Util.getAll(Util.cmd(cfs, "key1").build()); diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java index 3e088fbf6d36..df13edfe6ffb 100644 --- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java @@ -83,7 +83,7 @@ public void testGetRowNoColumns() throws Throwable Util.assertEmpty(Util.cmd(cfs, "0").columns("c").includeRow(1).build()); if (round == 0) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } @@ -118,7 +118,7 @@ public void testGetRowSingleColumn() throws Throwable } if (round == 0) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } @@ -135,7 +135,7 @@ public void testGetSliceBloomFilterFalsePositive() throws Throwable for (String key : new String[]{"0", "2"}) Util.assertEmpty(Util.cmd(cfs, key).build()); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); for (String key : new String[]{"0", "2"}) Util.assertEmpty(Util.cmd(cfs, key).build()); @@ -207,7 +207,7 @@ public void testGetSliceWithCutoff() throws Throwable assertRowsInSlice(cfs, "0", 288, 299, 12, true, prefix); if (round == 0) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } @@ -220,7 +220,7 @@ public void testReversedWithFlushing() throws Throwable for (int i = 0; i < 10; i++) execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "0", i, i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); for (int i = 10; i < 20; i++) { @@ -334,7 +334,7 @@ public void testGetSliceFromBasic() throws Throwable assertRowsInResult(cfs, command); if (round == 0) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } @@ -357,7 +357,7 @@ public void testGetSliceWithExpiration() throws Throwable assertRowsInResult(cfs, command, 1); if (round == 0) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } @@ -370,7 +370,7 @@ public void testGetSliceFromAdvanced() throws Throwable for (int i = 1; i < 7; i++) execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "0", i, i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // overwrite three rows with -1 for (int i = 1; i < 4; i++) @@ -382,7 +382,7 @@ public void testGetSliceFromAdvanced() throws Throwable assertRowsInResult(cfs, command, -1, -1, 4); if (round == 0) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } @@ -395,7 +395,7 @@ public void testGetSliceFromLarge() throws Throwable for (int i = 1000; i < 2000; i++) execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "0", i, i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); validateSliceLarge(cfs); @@ -423,7 +423,7 @@ public void testLimitSSTables() throws Throwable for (int i = 1000 + (j*100); i < 1000 + ((j+1)*100); i++) execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP ?", "0", i, i, (long)i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } ((ClearableHistogram)cfs.metric.sstablesPerReadHistogram.cf).clear(); diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java index 0b00f40ea2b1..517489c411d3 100644 --- a/test/unit/org/apache/cassandra/db/NameSortTest.java +++ b/test/unit/org/apache/cassandra/db/NameSortTest.java @@ -84,7 +84,7 @@ private void testNameSort(int N) throws IOException rub.build().applyUnsafe(); } validateNameSort(cfs); - keyspace.getColumnFamilyStore("Standard1").forceBlockingFlush(); + keyspace.getColumnFamilyStore("Standard1").forceBlockingFlushToSSTable(); validateNameSort(cfs); } diff --git a/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java b/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java index 9ae6c757b018..5d56ab4bd693 100644 --- a/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java +++ b/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java @@ -99,14 +99,14 @@ public void testCassandra6778() throws CharacterCodingException .add("val", "val1") .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, "k1") .clustering(new BigInteger(new byte[]{0, 0, 1})) .add("val", "val2") .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // fetch by the first column name; we should get the second version of the column value Row row = Util.getOnlyRow(Util.cmd(cfs, "k1").includeRow(new BigInteger(new byte[]{1})).build()); @@ -158,7 +158,7 @@ public void testRangeSliceInclusionExclusion() throws Throwable builder.build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ColumnMetadata cDef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val")); diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java index 3d1d00322d78..31eb28b57d5b 100644 --- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java +++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java @@ -85,7 +85,7 @@ public void simpleQueryWithRangeTombstoneTest() throws Exception for (int i = 0; i < 40; i += 2) builder.newRow(i).add("val", i); builder.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(10, 22).build().applyUnsafe(); @@ -235,7 +235,7 @@ public void testTrackTimesPartitionTombstone() throws ExecutionException, Interr int nowInSec = FBUtilities.nowInSeconds(); new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1000, nowInSec)).apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); assertTimes(sstable.getSSTableMetadata(), 1000, 1000, nowInSec); @@ -257,7 +257,7 @@ public void testTrackTimesPartitionTombstoneWithData() throws ExecutionException key = "rt_times2"; int nowInSec = FBUtilities.nowInSeconds(); new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1000, nowInSec)).apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); assertTimes(sstable.getSSTableMetadata(), 999, 1000, Integer.MAX_VALUE); @@ -276,7 +276,7 @@ public void testTrackTimesRangeTombstone() throws ExecutionException, Interrupte int nowInSec = FBUtilities.nowInSeconds(); new RowUpdateBuilder(cfs.metadata(), nowInSec, 1000L, key).addRangeTombstone(1, 2).build().apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); assertTimes(sstable.getSSTableMetadata(), 1000, 1000, nowInSec); @@ -298,9 +298,9 @@ public void testTrackTimesRangeTombstoneWithData() throws ExecutionException, In key = "rt_times2"; int nowInSec = FBUtilities.nowInSeconds(); new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1000, nowInSec)).apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); assertTimes(sstable.getSSTableMetadata(), 999, 1000, Integer.MAX_VALUE); cfs.forceMajorCompaction(); @@ -328,10 +328,10 @@ public void test7810() throws ExecutionException, InterruptedException for (int i = 10; i < 20; i ++) builder.newRow(i).add("val", i); builder.apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(10, 11).build().apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Thread.sleep(5); cfs.forceMajorCompaction(); @@ -350,10 +350,10 @@ public void test7808_1() throws ExecutionException, InterruptedException for (int i = 0; i < 40; i += 2) builder.newRow(i).add("val", i); builder.apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1, 1)).apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Thread.sleep(5); cfs.forceMajorCompaction(); } @@ -370,13 +370,13 @@ public void test7808_2() throws ExecutionException, InterruptedException for (int i = 10; i < 20; i ++) builder.newRow(i).add("val", i); builder.apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 0, 0)).apply(); UpdateBuilder.create(cfs.metadata(), key).withTimestamp(1).newRow(5).add("val", 5).apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Thread.sleep(5); cfs.forceMajorCompaction(); assertEquals(1, Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build()).rowCount()); @@ -396,16 +396,16 @@ public void overlappingRangeTest() throws Exception for (int i = 0; i < 20; i++) builder.newRow(i).add("val", i); builder.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(5, 15).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(5, 10).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 2, key).addRangeTombstone(5, 8).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Partition partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build()); int nowInSec = FBUtilities.nowInSeconds(); @@ -447,11 +447,11 @@ public void reverseQueryTest() throws Exception String key = "k3"; UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0).newRow(2).add("val", 2).applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(0, 10).build().applyUnsafe(); UpdateBuilder.create(cfs.metadata(), key).withTimestamp(2).newRow(1).add("val", 1).applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // Get the last value of the row FilteredPartition partition = Util.getOnlyPartition(Util.cmd(cfs, key).build()); @@ -508,10 +508,10 @@ public void testRowWithRangeTombstonesUpdatesSecondaryIndex() throws Exception for (int i = 0; i < 10; i++) builder.newRow(i).add("val", i); builder.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 0, key).addRangeTombstone(0, 7).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(10, index.rowsInserted.size()); @@ -538,10 +538,10 @@ public void testRangeTombstoneCompaction() throws Exception for (int i = 0; i < 10; i += 2) builder.newRow(i).add("val", i); builder.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 0, key).addRangeTombstone(0, 7).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // there should be 2 sstables assertEquals(2, cfs.getLiveSSTables().size()); @@ -614,7 +614,7 @@ public void testOverwritesToDeletedColumns() throws Exception // now re-insert that column UpdateBuilder.create(cfs.metadata(), key).withTimestamp(2).newRow(1).add("val", 1).applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // We should have 1 insert and 1 update to the indexed "1" column // CASSANDRA-6640 changed index update to just update, not insert then delete diff --git a/test/unit/org/apache/cassandra/db/ReadCommandTest.java b/test/unit/org/apache/cassandra/db/ReadCommandTest.java index 8b735026f8c7..fd31b9b1c243 100644 --- a/test/unit/org/apache/cassandra/db/ReadCommandTest.java +++ b/test/unit/org/apache/cassandra/db/ReadCommandTest.java @@ -175,7 +175,7 @@ public void testPartitionRangeAbort() throws Exception .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key2")) .clustering("Column1") @@ -203,7 +203,7 @@ public void testSinglePartitionSliceAbort() throws Exception .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key")) .clustering("dd") @@ -234,7 +234,7 @@ public void testSinglePartitionNamesAbort() throws Exception .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key")) .clustering("dd") @@ -313,7 +313,7 @@ public void testSinglePartitionGroupMerge() throws Exception commands.add(SinglePartitionReadCommand.create(cfs.metadata(), nowInSeconds, columnFilter, rowFilter, DataLimits.NONE, Util.dk(data[1]), sliceFilter)); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ReadQuery query = new SinglePartitionReadCommand.Group(commands, DataLimits.NONE); @@ -483,7 +483,7 @@ public void testCountDeletedRows() throws Exception DataLimits.NONE, Util.dk(data[1]), sliceFilter)); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ReadQuery query = new SinglePartitionReadCommand.Group(commands, DataLimits.NONE); @@ -559,7 +559,7 @@ public void testCountWithNoDeletedRow() throws Exception DataLimits.NONE, Util.dk(data[1]), sliceFilter)); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ReadQuery query = new SinglePartitionReadCommand.Group(commands, DataLimits.NONE); @@ -618,7 +618,7 @@ public void testSinglePartitionNamesSkipsOptimisationsIfTrackingRepairedData() .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, ByteBufferUtil.bytes("key")) .clustering("dd") @@ -626,7 +626,7 @@ public void testSinglePartitionNamesSkipsOptimisationsIfTrackingRepairedData() .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); List sstables = new ArrayList<>(cfs.getLiveSSTables()); assertEquals(2, sstables.size()); Collections.sort(sstables, SSTableReader.maxTimestampDescending); @@ -666,7 +666,7 @@ public void skipRowCacheIfTrackingRepairedData() .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ReadCommand readCommand = Util.cmd(cfs, Util.dk("key")).build(); assertTrue(cfs.isRowCacheEnabled()); @@ -739,7 +739,7 @@ private void testRepairedDataTracking(ColumnFamilyStore cfs, ReadCommand readCom .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 1, ByteBufferUtil.bytes("key")) .clustering("dd") @@ -747,7 +747,7 @@ private void testRepairedDataTracking(ColumnFamilyStore cfs, ReadCommand readCom .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); List sstables = new ArrayList<>(cfs.getLiveSSTables()); assertEquals(2, sstables.size()); sstables.forEach(sstable -> assertFalse(sstable.isRepaired() || sstable.isPendingRepair())); @@ -804,7 +804,7 @@ private void testRepairedDataTracking(ColumnFamilyStore cfs, ReadCommand readCom assertEquals(ByteBufferUtil.EMPTY_BYTE_BUFFER, digest); // now flush so we have an unrepaired table with the deletion and repeat the check - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); digest = performReadAndVerifyRepairedInfo(readCommand, 0, rowsPerPartition, false); assertEquals(ByteBufferUtil.EMPTY_BYTE_BUFFER, digest); } diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java index fc3494234021..f373a00e0ef0 100644 --- a/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java +++ b/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java @@ -115,7 +115,7 @@ public void testWithFlush() throws Exception Keyspace keyspace1 = Keyspace.open(KEYSPACE1); ColumnFamilyStore cfs = keyspace1.getColumnFamilyStore("Standard1"); logger.debug("forcing flush"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); logger.debug("begin manual replay"); // replay the commit log (nothing on Standard1 should be replayed since everything was flushed, so only the row on Standard2 diff --git a/test/unit/org/apache/cassandra/db/RemoveCellTest.java b/test/unit/org/apache/cassandra/db/RemoveCellTest.java index 01fe2551f4b9..03381774e1cf 100644 --- a/test/unit/org/apache/cassandra/db/RemoveCellTest.java +++ b/test/unit/org/apache/cassandra/db/RemoveCellTest.java @@ -30,7 +30,7 @@ public void testDeleteCell() throws Throwable String tableName = createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))"); ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP ?", 0, 0, 0, 0L); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); execute("DELETE c FROM %s USING TIMESTAMP ? WHERE a = ? AND b = ?", 1L, 0, 0); assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ?", 0, 0), row(0, 0, null)); assertRows(execute("SELECT c FROM %s WHERE a = ? AND b = ?", 0, 0), row(new Object[]{null})); diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java index 5ca1eefb854e..b5440c2ac0a9 100644 --- a/test/unit/org/apache/cassandra/db/RowCacheTest.java +++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java @@ -490,7 +490,7 @@ public void testSSTablesPerReadHistogramWhenRowCache() SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, 0, 100); //force flush for confidence that SSTables exists - cachedStore.forceBlockingFlush(); + cachedStore.forceBlockingFlushToSSTable(); ((ClearableHistogram)cachedStore.metric.sstablesPerReadHistogram.cf).clear(); diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java index b0cd4fc1ca40..0e229aa2ed69 100644 --- a/test/unit/org/apache/cassandra/db/RowIterationTest.java +++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java @@ -36,7 +36,7 @@ public void testRowIteration() throws Throwable ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); for (int i = 0; i < 10; i++) execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TIMESTAMP ?", i, 0, i, i, (long)i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(10, execute("SELECT * FROM %s").size()); } @@ -49,7 +49,7 @@ public void testRowIterationDeletionTime() throws Throwable execute("INSERT INTO %s (a, b) VALUES (?, ?) USING TIMESTAMP ?", 0, 0, 0L); execute("DELETE FROM %s USING TIMESTAMP ? WHERE a = ?", 0L, 0); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // Delete row in second sstable with higher timestamp execute("INSERT INTO %s (a, b) VALUES (?, ?) USING TIMESTAMP ?", 0, 0, 1L); @@ -57,7 +57,7 @@ public void testRowIterationDeletionTime() throws Throwable int localDeletionTime = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs).build()).partitionLevelDeletion().localDeletionTime(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); DeletionTime dt = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs).build()).partitionLevelDeletion(); assertEquals(1L, dt.markedForDeleteAt()); @@ -72,7 +72,7 @@ public void testRowIterationDeletion() throws Throwable // Delete a row in first sstable execute("DELETE FROM %s USING TIMESTAMP ? WHERE a = ?", 0L, 0); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertFalse(Util.getOnlyPartitionUnfiltered(Util.cmd(cfs).build()).isEmpty()); } diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java index 28962dba3053..ffc1bbd17c43 100644 --- a/test/unit/org/apache/cassandra/db/ScrubTest.java +++ b/test/unit/org/apache/cassandra/db/ScrubTest.java @@ -460,7 +460,7 @@ protected void fillCF(ColumnFamilyStore cfs, int partitionsPerSSTable) new Mutation(update).applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } public static void fillIndexCF(ColumnFamilyStore cfs, boolean composite, long ... values) @@ -484,7 +484,7 @@ public static void fillIndexCF(ColumnFamilyStore cfs, boolean composite, long .. new Mutation(builder.build()).applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } protected void fillCounterCF(ColumnFamilyStore cfs, int partitionsPerSSTable) throws WriteTimeoutException @@ -497,7 +497,7 @@ protected void fillCounterCF(ColumnFamilyStore cfs, int partitionsPerSSTable) th new CounterMutation(new Mutation(update), ConsistencyLevel.ONE).apply(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } @Test @@ -509,14 +509,14 @@ public void testScrubColumnValidation() throws InterruptedException, RequestExec ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_static_columns"); QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')", KEYSPACE)); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); CompactionManager.instance.performScrub(cfs, false, true, 2); QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE); ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation"); new Mutation(UpdateBuilder.create(cfs2.metadata(), "key").newRow().add("b", LongType.instance.decompose(1L)).build()).apply(); - cfs2.forceBlockingFlush(); + cfs2.forceBlockingFlushToSSTable(); CompactionManager.instance.performScrub(cfs2, false, false, 2); } diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java index c09b16c094f7..62aac898f7b8 100644 --- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java +++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java @@ -300,7 +300,7 @@ public void testDeleteOfInconsistentValuesInKeysIndex() throws Exception new RowUpdateBuilder(cfs.metadata(), 1, "k1").noRowMarker().add("birthdate", 1L).build().applyUnsafe(); // force a flush, so our index isn't being read from a memtable - keyspace.getColumnFamilyStore(WITH_KEYS_INDEX).forceBlockingFlush(); + keyspace.getColumnFamilyStore(WITH_KEYS_INDEX).forceBlockingFlushToSSTable(); // now apply another update, but force the index update to be skipped keyspace.apply(new RowUpdateBuilder(cfs.metadata(), 2, "k1").noRowMarker().add("birthdate", 2L).build(), @@ -356,7 +356,7 @@ private void runDeleteOfInconsistentValuesFromCompositeIndexTest(boolean isStati assertIndexedOne(cfs, col, 10l); // force a flush and retry the query, so our index isn't being read from a memtable - keyspace.getColumnFamilyStore(cfName).forceBlockingFlush(); + keyspace.getColumnFamilyStore(cfName).forceBlockingFlushToSSTable(); assertIndexedOne(cfs, col, 10l); // now apply another update, but force the index update to be skipped @@ -522,7 +522,7 @@ public void testKeysSearcherSimple() throws Exception new RowUpdateBuilder(cfs.metadata(), 0, "k" + i).noRowMarker().add("birthdate", 1l).build().applyUnsafe(); assertIndexedCount(cfs, ByteBufferUtil.bytes("birthdate"), 1l, 10); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertIndexedCount(cfs, ByteBufferUtil.bytes("birthdate"), 1l, 10); } @@ -537,7 +537,7 @@ public void testSelectivityWithMultipleIndexes() new RowUpdateBuilder(cfs.metadata(), 0, "k3").clustering("c").add("birthdate", 1L).add("notbirthdate", 3L).build().applyUnsafe(); new RowUpdateBuilder(cfs.metadata(), 0, "k4").clustering("c").add("birthdate", 1L).add("notbirthdate", 3L).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ReadCommand rc = Util.cmd(cfs) .fromKeyIncl("k1") .toKeyIncl("k3") diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionReadCommandCQLTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionReadCommandCQLTest.java index 1c891ec2b2c6..2bbe43b8c799 100644 --- a/test/unit/org/apache/cassandra/db/SinglePartitionReadCommandCQLTest.java +++ b/test/unit/org/apache/cassandra/db/SinglePartitionReadCommandCQLTest.java @@ -31,10 +31,10 @@ public void partitionLevelDeletionTest() throws Throwable { createTable("CREATE TABLE %s (bucket_id TEXT,name TEXT,data TEXT,PRIMARY KEY (bucket_id, name))"); execute("insert into %s (bucket_id, name, data) values ('8772618c9009cf8f5a5e0c18', 'test', 'hello')"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); execute("insert into %s (bucket_id, name, data) values ('8772618c9009cf8f5a5e0c19', 'test2', 'hello');"); execute("delete from %s where bucket_id = '8772618c9009cf8f5a5e0c18'"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); UntypedResultSet res = execute("select * from %s where bucket_id = '8772618c9009cf8f5a5e0c18' and name = 'test'"); assertTrue(res.isEmpty()); } diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java index 67fd314a066e..8e50ad584534 100644 --- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java +++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.TimeUnit; @@ -48,7 +47,6 @@ import org.apache.cassandra.Util; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.QueryProcessor; -import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.cql3.statements.SelectStatement; import org.apache.cassandra.db.filter.AbstractClusteringIndexFilter; import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter; @@ -182,7 +180,7 @@ private void testMultiNamesOrSlicesCommand(boolean flush, boolean isSlice) ck1)); if (flush) - Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE_SCLICES).forceBlockingFlush(); + Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE_SCLICES).forceBlockingFlushToSSTable(); AbstractClusteringIndexFilter clusteringFilter = createClusteringFilter(uniqueCk1, uniqueCk2, isSlice); ReadCommand cmd = SinglePartitionReadCommand.create(CFM_SLICES, @@ -301,7 +299,7 @@ public void staticColumnsAreReturned() throws IOException } // check (de)serialized iterator for sstable static cell - Schema.instance.getColumnFamilyStoreInstance(metadata.id).forceBlockingFlush(); + Schema.instance.getColumnFamilyStoreInstance(metadata.id).forceBlockingFlushToSSTable(); try (ReadExecutionController executionController = cmd.executionController(); UnfilteredPartitionIterator pi = cmd.executeLocally(executionController)) { response = ReadResponse.createDataResponse(pi, cmd); @@ -391,7 +389,7 @@ public void sstableFiltering() QueryProcessor.executeOnceInternal("INSERT INTO ks.legacy_mc_inaccurate_min_max (k, c1, c2, c3, v) VALUES (100, 2, 2, 2, 2)"); QueryProcessor.executeOnceInternal("DELETE FROM ks.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=1"); assertQueryReturnsSingleRT("SELECT * FROM ks.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=1 AND c2=1"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertQueryReturnsSingleRT("SELECT * FROM ks.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=1 AND c2=1"); assertQueryReturnsSingleRT("SELECT * FROM ks.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=1 AND c2=1 AND c3=1"); // clustering names @@ -407,7 +405,7 @@ public void sstableFiltering() new Mutation(builder.build()).apply(); assertQueryReturnsSingleRT("SELECT * FROM ks.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=3 AND c2=2"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertQueryReturnsSingleRT("SELECT * FROM ks.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=3 AND c2=2"); assertQueryReturnsSingleRT("SELECT * FROM ks.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=3 AND c2=2 AND c3=2"); // clustering names diff --git a/test/unit/org/apache/cassandra/db/TimeSortTest.java b/test/unit/org/apache/cassandra/db/TimeSortTest.java index 8ae05ea9578f..c16916f51e87 100644 --- a/test/unit/org/apache/cassandra/db/TimeSortTest.java +++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java @@ -36,7 +36,7 @@ public void testMixedSources() throws Throwable ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName); execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP ?", 0, 100, 0, 100L); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP ?", 0, 0, 1, 0L); assertRows(execute("SELECT * FROM %s WHERE a = ? AND b >= ? LIMIT 1000", 0, 10), row(0, 100, 0)); @@ -53,7 +53,7 @@ public void testTimeSort() throws Throwable execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP ?", i, j * 2, 0, (long)j * 2); validateTimeSort(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); validateTimeSort(); // interleave some new data to test memtable + sstable diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java index df2acb4fb179..6969567678dc 100644 --- a/test/unit/org/apache/cassandra/db/VerifyTest.java +++ b/test/unit/org/apache/cassandra/db/VerifyTest.java @@ -24,7 +24,6 @@ import org.apache.cassandra.Util; import org.apache.cassandra.cache.ChunkCache; import org.apache.cassandra.UpdateBuilder; -import org.apache.cassandra.db.compaction.AbstractCompactionStrategy; import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.compaction.Verifier; import org.apache.cassandra.db.marshal.UUIDType; @@ -35,7 +34,6 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.WriteTimeoutException; import org.apache.cassandra.io.FSWriteError; -import org.apache.cassandra.io.compress.CorruptBlockException; import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.format.SSTableReader; @@ -52,7 +50,6 @@ import org.junit.runner.RunWith; import java.io.*; -import java.net.UnknownHostException; import java.nio.file.Files; import java.util.ArrayList; import java.util.Collections; @@ -702,7 +699,7 @@ protected void fillCF(ColumnFamilyStore cfs, int partitionsPerSSTable) .apply(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } protected void fillCounterCF(ColumnFamilyStore cfs, int partitionsPerSSTable) throws WriteTimeoutException @@ -714,7 +711,7 @@ protected void fillCounterCF(ColumnFamilyStore cfs, int partitionsPerSSTable) th .apply(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } protected long simpleFullChecksum(String filename) throws IOException diff --git a/test/unit/org/apache/cassandra/db/columniterator/SSTableReverseIteratorTest.java b/test/unit/org/apache/cassandra/db/columniterator/SSTableReverseIteratorTest.java index 9040f1197cfa..0a423f48b790 100644 --- a/test/unit/org/apache/cassandra/db/columniterator/SSTableReverseIteratorTest.java +++ b/test/unit/org/apache/cassandra/db/columniterator/SSTableReverseIteratorTest.java @@ -81,7 +81,7 @@ public void emptyBlockTolerance() QueryProcessor.executeInternal(String.format("UPDATE %s.%s SET v1=? WHERE k=? AND c=?", KEYSPACE, table), bytes(0x20000), key, 2); QueryProcessor.executeInternal(String.format("UPDATE %s.%s SET v1=? WHERE k=? AND c=?", KEYSPACE, table), bytes(0x20000), key, 3); - tbl.forceBlockingFlush(); + tbl.forceBlockingFlushToSSTable(); SSTableReader sstable = Iterables.getOnlyElement(tbl.getLiveSSTables()); DecoratedKey dk = tbl.getPartitioner().decorateKey(Int32Type.instance.decompose(key)); RowIndexEntry indexEntry = sstable.getPosition(dk, SSTableReader.Operator.EQ); diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java index c113b971a576..29f7a02842c6 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogCQLTest.java @@ -31,11 +31,11 @@ public void testTruncateSegmentDiscard() throws Throwable execute("INSERT INTO %s (idx, data) VALUES (?, ?)", 15, Integer.toString(17)); - Collection active = new ArrayList<>(CommitLog.instance.segmentManager.getUnflushedSegments()); + Collection active = new ArrayList<>(CommitLog.instance.segmentManager.getSegmentsForUnflushedTables()); CommitLog.instance.forceRecycleAllSegments(); // If one of the previous segments remains, it wasn't clean. - active.retainAll(CommitLog.instance.segmentManager.getUnflushedSegments()); + active.retainAll(CommitLog.instance.segmentManager.getSegmentsForUnflushedTables()); assert active.isEmpty(); } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java index ca76e45ac8ad..8d5430f60d3d 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java @@ -261,7 +261,7 @@ CommitLogPosition populateData(int entryCount) throws Throwable for (int i = midpoint; i < entryCount; i++) execute("INSERT INTO %s (idx, data) VALUES (?, ?)", i, Integer.toString(i)); - Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush(); + Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlushToSSTable(); return result; } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java index c50d9973ed51..9f0949d8eec5 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java @@ -91,7 +91,7 @@ public void testCDCWriteFailure() throws Throwable execute("INSERT INTO %s (idx, data) VALUES (1, '1');"); // Confirm that, on flush+recyle, we see files show up in cdc_raw - Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush(); + Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlushToSSTable(); CommitLog.instance.forceRecycleAllSegments(); cdcMgr.awaitManagementTasksCompletion(); Assert.assertTrue("Expected files to be moved to overflow.", CommitLogTestUtils.getCDCRawCount() > 0); diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java index 70af3cbfbadf..f4f703837054 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentBackpressureTest.java @@ -113,19 +113,19 @@ public void testCompressedCommitLogBackpressure() throws Throwable CommitLogSegmentManager clsm = CommitLog.instance.segmentManager; - Util.spinAssertEquals(3, () -> clsm.getUnflushedSegments().size(), 5); + Util.spinAssertEquals(3, () -> clsm.getSegmentsForUnflushedTables().size(), 5); Thread.sleep(1000); // Should only be able to create 3 segments not 7 because it blocks waiting for truncation that never comes - Assert.assertEquals(3, clsm.getUnflushedSegments().size()); + Assert.assertEquals(3, clsm.getSegmentsForUnflushedTables().size()); // Discard the currently active segments so allocation can continue. // Take snapshot of the list, otherwise this will also discard newly allocated segments. - new ArrayList<>(clsm.getUnflushedSegments()).forEach( clsm::archiveAndDiscard ); + new ArrayList<>(clsm.getSegmentsForUnflushedTables()).forEach(clsm::archiveAndDiscard ); // The allocated count should reach the limit again. - Util.spinAssertEquals(3, () -> clsm.getUnflushedSegments().size(), 5); + Util.spinAssertEquals(3, () -> clsm.getSegmentsForUnflushedTables().size(), 5); } finally { diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java index 50a4bbc15127..004945badfa4 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java @@ -314,13 +314,13 @@ public void testDontDeleteIfDirty() throws Exception .build(); CommitLog.instance.add(m2); - assertEquals(2, CommitLog.instance.segmentManager.getUnflushedSegments().size()); + assertEquals(2, CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size()); TableId id2 = m2.getTableIds().iterator().next(); CommitLog.instance.discardCompletedSegments(id2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); // Assert we still have both our segments - assertEquals(2, CommitLog.instance.segmentManager.getUnflushedSegments().size()); + assertEquals(2, CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size()); } @Test @@ -340,14 +340,14 @@ public void testDeleteIfNotDirty() throws Exception CommitLog.instance.add(rm); CommitLog.instance.add(rm); - assertEquals(1, CommitLog.instance.segmentManager.getUnflushedSegments().size()); + assertEquals(1, CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size()); // "Flush": this won't delete anything TableId id1 = rm.getTableIds().iterator().next(); CommitLog.instance.sync(true); CommitLog.instance.discardCompletedSegments(id1, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); - assertEquals(1, CommitLog.instance.segmentManager.getUnflushedSegments().size()); + assertEquals(1, CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size()); // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created Mutation rm2 = new RowUpdateBuilder(cfs2.metadata(), 0, "k") @@ -359,7 +359,7 @@ public void testDeleteIfNotDirty() throws Exception CommitLog.instance.add(rm2); CommitLog.instance.add(rm2); - Collection segments = CommitLog.instance.segmentManager.getUnflushedSegments(); + Collection segments = CommitLog.instance.segmentManager.getSegmentsForUnflushedTables(); assertEquals(String.format("Expected 3 segments but got %d (%s)", segments.size(), getDirtyCFIds(segments)), 3, @@ -371,7 +371,7 @@ public void testDeleteIfNotDirty() throws Exception TableId id2 = rm2.getTableIds().iterator().next(); CommitLog.instance.discardCompletedSegments(id2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition()); - segments = CommitLog.instance.segmentManager.getUnflushedSegments(); + segments = CommitLog.instance.segmentManager.getSegmentsForUnflushedTables(); // Assert we still have both our segment assertEquals(String.format("Expected 1 segment but got %d (%s)", segments.size(), getDirtyCFIds(segments)), @@ -617,13 +617,13 @@ public void testTruncateWithoutSnapshot() throws ExecutionException, Interrupted for (int i = 0 ; i < 5 ; i++) CommitLog.instance.add(m2); - assertEquals(2, CommitLog.instance.segmentManager.getUnflushedSegments().size()); + assertEquals(2, CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size()); CommitLogPosition position = CommitLog.instance.getCurrentPosition(); for (Keyspace keyspace : Keyspace.system()) for (ColumnFamilyStore syscfs : keyspace.getColumnFamilyStores()) CommitLog.instance.discardCompletedSegments(syscfs.metadata().id, CommitLogPosition.NONE, position); CommitLog.instance.discardCompletedSegments(cfs2.metadata().id, CommitLogPosition.NONE, position); - assertEquals(1, CommitLog.instance.segmentManager.getUnflushedSegments().size()); + assertEquals(1, CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size()); } finally { @@ -791,7 +791,7 @@ public void testUnwriteableFlushRecovery() throws ExecutionException, Interrupte { try (Closeable c = Util.markDirectoriesUnwriteable(cfs)) { - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } catch (Throwable t) { @@ -801,7 +801,7 @@ public void testUnwriteableFlushRecovery() throws ExecutionException, Interrupte } } else - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } finally @@ -854,7 +854,7 @@ public void testOutOfOrderFlushRecovery(BiConsumer { try { - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } catch (Throwable t) { diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java index 517ff17c491b..2a05d41db529 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java @@ -75,7 +75,7 @@ static File getFilledCommitLogFile() */ static int getCommitLogCountOnDisk() { - return CommitLog.instance.segmentManager.getUnflushedSegments().size(); + return CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size(); } /** diff --git a/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java index 070488856a58..76094e6df09d 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java @@ -77,7 +77,7 @@ public void beforeTest() throws Throwable CommitLog.instance.sync(true); // Should have well more than 3 segments to work with on subsequent tests. - Assert.assertTrue(CommitLog.instance.segmentManager.getUnflushedSegments().size() > 3); + Assert.assertTrue(CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size() > 3); } /** diff --git a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java index b3dc07010146..18dd42868684 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java @@ -76,7 +76,7 @@ public void testCompactionHook() throws Exception // Compact the cf and confirm that the executor's after hook calls rescheduleDeletion populate(20000); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); store.forceMajorCompaction(); long start = System.currentTimeMillis(); diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java index 4092f541f4b1..f2387a66f09c 100644 --- a/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/AbstractCompactionStrategyTest.java @@ -139,6 +139,6 @@ private static void insertKeyAndFlush(String table, int key) .add("val", "val") .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } diff --git a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java index de7ddfcb1f38..ba927d989353 100644 --- a/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/AbstractPendingRepairTest.java @@ -86,7 +86,7 @@ SSTableReader makeSSTable(boolean orphan) int pk = nextSSTableKey++; Set pre = cfs.getLiveSSTables(); QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, v) VALUES(?, ?)", ks, tbl), pk, pk); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Set post = cfs.getLiveSSTables(); Set diff = new HashSet<>(post); diff.removeAll(pre); diff --git a/test/unit/org/apache/cassandra/db/compaction/ActiveCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/ActiveCompactionsTest.java index be5e7df7a78f..445ff573e6f3 100644 --- a/test/unit/org/apache/cassandra/db/compaction/ActiveCompactionsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/ActiveCompactionsTest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.db.compaction; -import java.util.ArrayList; import java.util.Collections; import java.util.Map; import java.util.Set; @@ -60,7 +59,7 @@ public void testSecondaryIndexTracking() throws Throwable for (int i = 0; i < 5; i++) { execute("INSERT INTO %s (pk, ck, a, b) VALUES ("+i+", 2, 3, 4)"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); } Index idx = getCurrentColumnFamilyStore().indexManager.getIndexByName(idxName); @@ -83,7 +82,7 @@ public void testIndexSummaryRedistributionTracking() throws Throwable for (int i = 0; i < 5; i++) { execute("INSERT INTO %s (pk, ck, a, b) VALUES ("+i+", 2, 3, 4)"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); } Set sstables = getCurrentColumnFamilyStore().getLiveSSTables(); try (LifecycleTransaction txn = getCurrentColumnFamilyStore().getTracker().tryModify(sstables, OperationType.INDEX_SUMMARY)) @@ -108,7 +107,7 @@ public void testViewBuildTracking() throws Throwable for (int i = 0; i < 5; i++) { execute("INSERT INTO %s (k1, c1, val) VALUES ("+i+", 2, 3)"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); } execute(String.format("CREATE MATERIALIZED VIEW %s.view1 AS SELECT k1, c1, val FROM %s.%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)", keyspace(), keyspace(), currentTable())); View view = Iterables.getOnlyElement(getCurrentColumnFamilyStore().viewManager); @@ -132,7 +131,7 @@ public void testScrubOne() throws Throwable for (int i = 0; i < 5; i++) { execute("INSERT INTO %s (pk, ck, a, b) VALUES (" + i + ", 2, 3, 4)"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); } SSTableReader sstable = Iterables.getFirst(getCurrentColumnFamilyStore().getLiveSSTables(), null); @@ -157,7 +156,7 @@ public void testVerifyOne() throws Throwable for (int i = 0; i < 5; i++) { execute("INSERT INTO %s (pk, ck, a, b) VALUES (" + i + ", 2, 3, 4)"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); } SSTableReader sstable = Iterables.getFirst(getCurrentColumnFamilyStore().getLiveSSTables(), null); diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionBytemanTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionBytemanTest.java index 38d2607d2f7d..499002c9dcb5 100644 --- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionBytemanTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionBytemanTest.java @@ -67,7 +67,7 @@ public void testRedundantTransitions() throws Throwable execute("insert into %s (id, i) values (1, 1)"); execute("insert into %s (id, i) values (2, 1)"); execute("insert into %s (id, i) values (3, 1)"); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); UntypedResultSet res = execute("select token(id) as tok from %s"); Iterator it = res.iterator(); List tokens = new ArrayList<>(); diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java index b2618e54f7ff..8f5b1608765c 100644 --- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java @@ -28,7 +28,6 @@ import java.util.Set; import java.util.UUID; import java.util.function.Predicate; -import java.util.stream.Collectors; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; @@ -299,7 +298,7 @@ public void generateSStable(ColumnFamilyStore store, String Suffix) .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); } @Test @@ -442,7 +441,7 @@ private ColumnFamilyStore prepareColumnFamilyStore() .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); return store; } diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java index e0f24f28d68b..2d3c886a31cc 100644 --- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java @@ -158,7 +158,7 @@ private void testBlacklisting(String tableName) throws Exception maxTimestampExpected = Math.max(timestamp, maxTimestampExpected); inserted.add(key); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); CompactionsTest.assertMaxTimestamp(cfs, maxTimestampExpected); assertEquals(inserted.toString(), inserted.size(), Util.getAll(Util.cmd(cfs).build()).size()); } diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java index 68936f55427b..766eb4e8850a 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java @@ -195,7 +195,7 @@ private void populate(int count) throws Throwable execute(String.format("INSERT INTO %s.%s(k, t, v) VALUES (?, ?, ?)", KEYSPACE, TABLE), i, j, b); ColumnFamilyStore cfs = getColumnFamilyStore(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); if (cfs.getLiveSSTables().size() > 1) { // we want just one big sstable to avoid doing actual compaction in compact() above diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java index 0ab714acfc34..0d39a625d91e 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java @@ -21,7 +21,6 @@ import java.nio.ByteBuffer; import java.util.Set; import java.util.function.LongPredicate; -import java.util.function.Predicate; import com.google.common.collect.Sets; import org.junit.BeforeClass; @@ -95,7 +94,7 @@ public void testMaxPurgeableTimestamp() { assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp1); //memtable only - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertTrue(controller.getPurgeEvaluator(key).test(Long.MAX_VALUE)); //no memtables and no sstables } @@ -103,7 +102,7 @@ public void testMaxPurgeableTimestamp() // create another sstable applyMutation(cfs.metadata(), key, timestamp2); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // check max purgeable timestamp when compacting the first sstable with and without a memtable try (CompactionController controller = new CompactionController(cfs, compacting, 0)) @@ -116,7 +115,7 @@ public void testMaxPurgeableTimestamp() } // check max purgeable timestamp again without any sstables but with different insertion orders on the memtable - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); //newest to oldest try (CompactionController controller = new CompactionController(cfs, null, 0)) @@ -128,7 +127,7 @@ public void testMaxPurgeableTimestamp() assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp3); //memtable only } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); //oldest to newest try (CompactionController controller = new CompactionController(cfs, null, 0)) @@ -156,14 +155,14 @@ public void testGetFullyExpiredSSTables() // create sstable with tombstone that should be expired in no older timestamps applyDeleteMutation(cfs.metadata(), key, timestamp2); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // first sstable with tombstone is compacting Set compacting = Sets.newHashSet(cfs.getLiveSSTables()); // create another sstable with more recent timestamp applyMutation(cfs.metadata(), key, timestamp1); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // second sstable is overlapping Set overlapping = Sets.difference(Sets.newHashSet(cfs.getLiveSSTables()), compacting); diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java index 73e6852eb178..ddabcaf82e22 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionStrategyManagerTest.java @@ -511,7 +511,7 @@ private static SSTableReader createSSTableWithKey(String keyspace, String table, .build() .applyUnsafe(); Set before = cfs.getLiveSSTables(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Set after = cfs.getLiveSSTables(); return Iterables.getOnlyElement(Sets.difference(after, before)); } diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java index af74603fd205..a9c178276e18 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionTaskTest.java @@ -71,10 +71,10 @@ public void compactionInterruption() throws Exception cfs.getCompactionStrategyManager().disable(); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (1, 1);"); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (2, 2);"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (3, 3);"); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (4, 4);"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Set sstables = cfs.getLiveSSTables(); Assert.assertEquals(2, sstables.size()); @@ -111,13 +111,13 @@ public void mixedSSTableFailure() throws Exception { cfs.getCompactionStrategyManager().disable(); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (1, 1);"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (2, 2);"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (3, 3);"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, v) VALUES (4, 4);"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); List sstables = new ArrayList<>(cfs.getLiveSSTables()); Assert.assertEquals(4, sstables.size()); diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java index 95069f17a770..75268082accb 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java @@ -128,7 +128,7 @@ public void testCompactingCFCounting() throws Throwable execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 1, 1); assertEquals(0, CompactionManager.instance.compactingCF.count(cfs)); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); FBUtilities.waitOnFutures(CompactionManager.instance.submitBackground(cfs)); assertEquals(0, CompactionManager.instance.compactingCF.count(cfs)); @@ -145,7 +145,7 @@ private void createPossiblyExpiredSSTable(final ColumnFamilyStore cfs, final boo { execute("INSERT INTO %s (id, val) values (2, 'immortal')"); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } private void createLowGCGraceTable(){ @@ -192,7 +192,7 @@ public void testStopCompactionRepaired(Consumer compactionRun { execute("insert into %s (k, c, v) values (?, ?, ?)", i, j, i*j); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } cfs.getCompactionStrategyManager().mutateRepaired(cfs.getLiveSSTables(), System.currentTimeMillis(), null, false); for (int i = 0; i < 5; i++) @@ -201,7 +201,7 @@ public void testStopCompactionRepaired(Consumer compactionRun { execute("insert into %s (k, c, v) values (?, ?, ?)", i, j, i*j); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } assertTrue(cfs.getTracker().getCompacting().isEmpty()); diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java index b003721c06ec..3cdb3a0aff4e 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java @@ -290,7 +290,7 @@ public void testCompactionInvalidRTs() throws Throwable RangeTombstone rt = new RangeTombstone(Slice.ALL, new DeletionTime(System.currentTimeMillis(), -1)); RowUpdateBuilder rub = new RowUpdateBuilder(getCurrentColumnFamilyStore().metadata(), System.currentTimeMillis() * 1000, 22).clustering(33).addRangeTombstone(rt); rub.build().apply(); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); compactAndValidate(); readAndValidate(true); readAndValidate(false); @@ -304,7 +304,7 @@ public void testCompactionInvalidTombstone() throws Throwable // write a standard tombstone with negative local deletion time (LDTs are not set by user and should not be negative): RowUpdateBuilder rub = new RowUpdateBuilder(getCurrentColumnFamilyStore().metadata(), -1, System.currentTimeMillis() * 1000, 22).clustering(33).delete("b"); rub.build().apply(); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); compactAndValidate(); readAndValidate(true); readAndValidate(false); @@ -318,7 +318,7 @@ public void testCompactionInvalidPartitionDeletion() throws Throwable // write a partition deletion with negative local deletion time (LDTs are not set by user and should not be negative):: PartitionUpdate pu = PartitionUpdate.simpleBuilder(getCurrentColumnFamilyStore().metadata(), 22).nowInSec(-1).delete().build(); new Mutation(pu).apply(); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); compactAndValidate(); readAndValidate(true); readAndValidate(false); @@ -331,7 +331,7 @@ public void testCompactionInvalidRowDeletion() throws Throwable prepare(); // write a row deletion with negative local deletion time (LDTs are not set by user and should not be negative): RowUpdateBuilder.deleteRowAt(getCurrentColumnFamilyStore().metadata(), System.currentTimeMillis() * 1000, -1, 22, 33).apply(); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); compactAndValidate(); readAndValidate(true); readAndValidate(false); @@ -353,7 +353,7 @@ public void testIndexedReaderRowDeletion() throws Throwable DatabaseDescriptor.setColumnIndexSize(1024); prepareWide(); RowUpdateBuilder.deleteRowAt(getCurrentColumnFamilyStore().metadata(), System.currentTimeMillis() * 1000, -1, 22, 33).apply(); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); readAndValidate(true); readAndValidate(false); DatabaseDescriptor.setColumnIndexSize(maxSizePre); @@ -369,7 +369,7 @@ public void testIndexedReaderTombstone() throws Throwable prepareWide(); RowUpdateBuilder rub = new RowUpdateBuilder(getCurrentColumnFamilyStore().metadata(), -1, System.currentTimeMillis() * 1000, 22).clustering(33).delete("b"); rub.build().apply(); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); readAndValidate(true); readAndValidate(false); DatabaseDescriptor.setColumnIndexSize(maxSizePre); @@ -386,7 +386,7 @@ public void testIndexedReaderRT() throws Throwable RangeTombstone rt = new RangeTombstone(Slice.ALL, new DeletionTime(System.currentTimeMillis(), -1)); RowUpdateBuilder rub = new RowUpdateBuilder(getCurrentColumnFamilyStore().metadata(), System.currentTimeMillis() * 1000, 22).clustering(33).addRangeTombstone(rt); rub.build().apply(); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); readAndValidate(true); readAndValidate(false); DatabaseDescriptor.setColumnIndexSize(maxSizePre); @@ -408,7 +408,7 @@ public void testLCSThresholdParams() throws Throwable { execute("insert into %s (id, id2, t) values (?, ?, ?)", i, j, value); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } assertEquals(50, cfs.getLiveSSTables().size()); LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first(); @@ -425,7 +425,7 @@ public void testSTCSinL0() throws Throwable ColumnFamilyStore cfs = getCurrentColumnFamilyStore(); cfs.disableAutoCompaction(); execute("insert into %s (id, id2, t) values (?, ?, ?)", 1,1,"L1"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.forceMajorCompaction(); SSTableReader l1sstable = cfs.getLiveSSTables().iterator().next(); assertEquals(1, l1sstable.getSSTableLevel()); @@ -439,7 +439,7 @@ public void testSTCSinL0() throws Throwable { execute("insert into %s (id, id2, t) values (?, ?, ?)", i, j, value); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } assertEquals(51, cfs.getLiveSSTables().size()); @@ -533,7 +533,7 @@ public void testPerCFSNeverPurgeTombstonesHelper(boolean deletedCell) throws Thr { execute("INSERT INTO %s (id, b) VALUES (?, ?)", i, String.valueOf(i)); } - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); assertTombstones(getCurrentColumnFamilyStore().getLiveSSTables().iterator().next(), false); if (deletedCell) @@ -541,7 +541,7 @@ public void testPerCFSNeverPurgeTombstonesHelper(boolean deletedCell) throws Thr else execute("DELETE FROM %s WHERE id = ?", 50); getCurrentColumnFamilyStore().setNeverPurgeTombstones(false); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Thread.sleep(2000); // wait for gcgs to pass getCurrentColumnFamilyStore().forceMajorCompaction(); assertTombstones(getCurrentColumnFamilyStore().getLiveSSTables().iterator().next(), false); @@ -550,7 +550,7 @@ public void testPerCFSNeverPurgeTombstonesHelper(boolean deletedCell) throws Thr else execute("DELETE FROM %s WHERE id = ?", 44); getCurrentColumnFamilyStore().setNeverPurgeTombstones(true); - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); Thread.sleep(1100); getCurrentColumnFamilyStore().forceMajorCompaction(); assertTombstones(getCurrentColumnFamilyStore().getLiveSSTables().iterator().next(), true); diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java index a0d52aa6cccd..203c94de8777 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java @@ -102,14 +102,14 @@ public void testMajorCompactionPurge() .build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // deletes for (int i = 0; i < 10; i++) { RowUpdateBuilder.deleteRow(cfs.metadata(), 1, key, String.valueOf(i)).applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // resurrect one column RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key); @@ -117,7 +117,7 @@ public void testMajorCompactionPurge() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // major compact and test that all columns but the resurrected one is completely gone FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); @@ -146,14 +146,14 @@ public void testMajorCompactionPurgeTombstonesWithMaxTimestamp() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // deletes for (int i = 0; i < 10; i++) { RowUpdateBuilder.deleteRow(cfs.metadata(), Long.MAX_VALUE, key, String.valueOf(i)).applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // major compact - tombstones should be purged FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); @@ -164,7 +164,7 @@ public void testMajorCompactionPurgeTombstonesWithMaxTimestamp() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.invalidateCachedPartition(dk(key)); @@ -191,13 +191,13 @@ public void testMajorCompactionPurgeTopLevelTombstoneWithMaxTimestamp() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new Mutation.PartitionUpdateCollector(KEYSPACE1, dk(key)) .add(PartitionUpdate.fullPartitionDelete(cfs.metadata(), dk(key), Long.MAX_VALUE, FBUtilities.nowInSeconds())) .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // major compact - tombstones should be purged FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); @@ -208,7 +208,7 @@ public void testMajorCompactionPurgeTopLevelTombstoneWithMaxTimestamp() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.invalidateCachedPartition(dk(key)); @@ -235,11 +235,11 @@ public void testMajorCompactionPurgeRangeTombstoneWithMaxTimestamp() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), Long.MAX_VALUE, dk(key)) .addRangeTombstone(String.valueOf(0), String.valueOf(9)).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // major compact - tombstones should be purged FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false)); @@ -250,7 +250,7 @@ public void testMajorCompactionPurgeRangeTombstoneWithMaxTimestamp() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.invalidateCachedPartition(dk(key)); @@ -278,7 +278,7 @@ public void testMinorCompactionPurge() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // deletes for (int i = 0; i < 10; i++) @@ -286,7 +286,7 @@ public void testMinorCompactionPurge() RowUpdateBuilder.deleteRow(cfs.metadata(), 1, key, String.valueOf(i)).applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } DecoratedKey key1 = Util.dk("key1"); @@ -294,7 +294,7 @@ public void testMinorCompactionPurge() // flush, remember the current sstable and then resurrect one column // for first key. Then submit minor compaction on remembered sstables. - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Collection sstablesIncomplete = cfs.getLiveSSTables(); RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, "key1"); @@ -302,7 +302,7 @@ public void testMinorCompactionPurge() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); try (CompactionTasks tasks = cfs.getCompactionStrategyManager().getUserDefinedTasks(sstablesIncomplete, Integer.MAX_VALUE)) { Iterables.getOnlyElement(tasks).execute(ActiveCompactionsTracker.NOOP); @@ -343,16 +343,16 @@ public void testMinTimestampPurge() .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // delete c1 RowUpdateBuilder.deleteRow(cfs.metadata(), 10, key3, "c1").applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Collection sstablesIncomplete = cfs.getLiveSSTables(); // delete c2 so we have new delete in a diffrent SSTable RowUpdateBuilder.deleteRow(cfs.metadata(), 9, key3, "c2").applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // compact the sstables with the c1/c2 data and the c1 tombstone try (CompactionTasks tasks = cfs.getCompactionStrategyManager().getUserDefinedTasks(sstablesIncomplete, Integer.MAX_VALUE)) @@ -393,7 +393,7 @@ public void testCompactionPurgeOneFile() throws ExecutionException, InterruptedE { RowUpdateBuilder.deleteRow(cfs.metadata(), 1, key, String.valueOf(i)).applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(String.valueOf(cfs.getLiveSSTables()), 1, cfs.getLiveSSTables().size()); // inserts & deletes were in the same memtable -> only deletes in sstable // compact and test that the row is completely gone @@ -438,7 +438,7 @@ public void testCompactionPurgeCachedRow() throws ExecutionException, Interrupte assertFalse(Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build()).isEmpty()); // flush and major compact - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Util.compactAll(cfs, Integer.MAX_VALUE).get(); // Since we've force purging (by passing MAX_VALUE for gc_before), the row should have been invalidated and we should have no deletion info anymore @@ -474,7 +474,7 @@ public void testCompactionPurgeTombstonedRow() throws ExecutionException, Interr assertFalse(partition.partitionLevelDeletion().isLive()); // flush and major compact (with tombstone purging) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Util.compactAll(cfs, Integer.MAX_VALUE).get(); assertFalse(Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build()).isEmpty()); @@ -504,14 +504,14 @@ public void testRowTombstoneObservedBeforePurging() // write a row out to one sstable QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, v1, v2) VALUES (%d, '%s', %d)", keyspace, table, 1, "foo", 1)); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s WHERE k = %d", keyspace, table, 1)); assertEquals(1, result.size()); // write a row tombstone out to a second sstable QueryProcessor.executeInternal(String.format("DELETE FROM %s.%s WHERE k = %d", keyspace, table, 1)); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // basic check that the row is considered deleted assertEquals(2, cfs.getLiveSSTables().size()); @@ -529,14 +529,14 @@ public void testRowTombstoneObservedBeforePurging() // write a row out to one sstable QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, v1, v2) VALUES (%d, '%s', %d)", keyspace, table, 1, "foo", 1)); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(2, cfs.getLiveSSTables().size()); result = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s WHERE k = %d", keyspace, table, 1)); assertEquals(1, result.size()); // write a row tombstone out to a different sstable QueryProcessor.executeInternal(String.format("DELETE FROM %s.%s WHERE k = %d", keyspace, table, 1)); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // compact the two sstables with a gcBefore that *does* allow the row tombstone to be purged FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) + 10000, false)); diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java index 941ef13eb250..a83b11e219ef 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java @@ -156,7 +156,7 @@ public void testSingleSSTableCompaction() throws Exception long timestamp = populate(KEYSPACE1, CF_DENSE1, 0, 9, 3); //ttl=3s - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1, store.getLiveSSTables().size()); long originalSize = store.getLiveSSTables().iterator().next().uncompressedLength(); @@ -196,11 +196,11 @@ public void testSuperColumnTombstones() .clustering(ByteBufferUtil.bytes("cols")) .add("val", "val1") .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // shadow the subcolumn with a supercolumn tombstone RowUpdateBuilder.deleteRow(table, FBUtilities.timestampMicros(), key.getKey(), ByteBufferUtil.bytes("cols")).applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(cfs, false); assertEquals(1, cfs.getLiveSSTables().size()); @@ -236,11 +236,11 @@ public void testUncheckedTombstoneSizeTieredCompaction() throws Exception //Populate sstable1 with with keys [0..9] populate(KEYSPACE1, CF_STANDARD1, 0, 9, 3); //ttl=3s - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); //Populate sstable2 with with keys [10..19] (keys do not overlap with SSTable1) long timestamp2 = populate(KEYSPACE1, CF_STANDARD1, 10, 19, 3); //ttl=3s - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(2, store.getLiveSSTables().size()); @@ -330,7 +330,7 @@ public void testUserDefinedCompaction() throws Exception .add("val", "val1") .build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Collection sstables = cfs.getLiveSSTables(); assertEquals(1, sstables.size()); @@ -365,7 +365,7 @@ public static void writeSSTableWithRangeTombstoneMaskingOneColumn(ColumnFamilySt notYetDeletedRowUpdateBuilder.clustering("02").add("val", "a"); //Range tombstone doesn't cover this (timestamp 3 > 2) notYetDeletedRowUpdateBuilder.build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } @Test @@ -450,7 +450,7 @@ private void testDontPurgeAccidentally(String k, String cfname) throws Interrupt rowUpdateBuilder.clustering("c").add("val", "a"); rowUpdateBuilder.build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Collection sstablesBefore = cfs.getLiveSSTables(); @@ -468,7 +468,7 @@ private void testDontPurgeAccidentally(String k, String cfname) throws Interrupt // Sleep one second so that the removal is indeed purgeable even with gcgrace == 0 Thread.sleep(1000); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Collection sstablesAfter = cfs.getLiveSSTables(); Collection toCompact = new ArrayList(); @@ -550,7 +550,7 @@ public void testNeedsCleanup() insertRowWithKey(i + 100); insertRowWithKey(i + 200); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1, store.getLiveSSTables().size()); SSTableReader sstable = store.getLiveSSTables().iterator().next(); diff --git a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java index f75842d05a85..6d04ccf71648 100644 --- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java @@ -231,9 +231,9 @@ public void testPrepBucket() .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); List sstrs = new ArrayList<>(cfs.getLiveSSTables()); @@ -267,9 +267,9 @@ public void testFilterOldSSTables() .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Iterable filtered; List sstrs = new ArrayList<>(cfs.getLiveSSTables()); @@ -304,7 +304,7 @@ public void testDropExpiredSSTables() throws InterruptedException .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader expiredSSTable = cfs.getLiveSSTables().iterator().next(); Thread.sleep(10); @@ -313,7 +313,7 @@ public void testDropExpiredSSTables() throws InterruptedException .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(cfs.getLiveSSTables().size(), 2); Map options = new HashMap<>(); @@ -357,7 +357,7 @@ public void testSTCSBigWindow() .clustering("column") .add("val", bigValue).build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } // and small ones: for (int r = 0; r < numSSTables / 2; r++) @@ -366,7 +366,7 @@ public void testSTCSBigWindow() new RowUpdateBuilder(cfs.metadata(), timestamp, key.getKey()) .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } Map options = new HashMap<>(); options.put(SizeTieredCompactionStrategyOptions.MIN_SSTABLE_SIZE_KEY, "1"); diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java index 6c75e7bc63cf..7488a592a982 100644 --- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java @@ -127,7 +127,7 @@ public void testGrouperLevels() throws Exception{ for (int c = 0; c < columns; c++) update.newRow("column" + c).add("val", value); update.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } waitForLeveling(cfs); @@ -183,7 +183,7 @@ public void testValidationMultipleSSTablePerLevel() throws Exception for (int c = 0; c < columns; c++) update.newRow("column" + c).add("val", value); update.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } waitForLeveling(cfs); @@ -257,7 +257,7 @@ public void testCompactionProgress() throws Exception for (int c = 0; c < columns; c++) update.newRow("column" + c).add("val", value); update.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } waitForLeveling(cfs); @@ -294,9 +294,9 @@ public void testMutateLevel() throws Exception for (int c = 0; c < columns; c++) update.newRow("column" + c).add("val", value); update.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); LeveledCompactionStrategy strategy = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getStrategies().get(1).get(0); cfs.forceMajorCompaction(); @@ -335,7 +335,7 @@ public void testNewRepairedSSTable() throws Exception for (int c = 0; c < columns; c++) update.newRow("column" + c).add("val", value); update.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } waitForLeveling(cfs); cfs.disableAutoCompaction(); @@ -414,7 +414,7 @@ public void testTokenRangeCompaction() throws Exception update.newRow("column" + c).add("val", value); update.applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } // create 20 more sstables with 10 containing data for key1 and other 10 containing data for key2 @@ -424,7 +424,7 @@ public void testTokenRangeCompaction() throws Exception for (int c = 0; c < columns; c++) update.newRow("column" + c).add("val", value); update.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } @@ -475,7 +475,7 @@ public void testCompactionCandidateOrdering() throws Exception for (int c = 0; c < columns; c++) update.newRow("column" + c).add("val", value); update.applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } LeveledCompactionStrategy strategy = (LeveledCompactionStrategy) (cfs.getCompactionStrategyManager()).getStrategies().get(1).get(0); // get readers for level 0 sstables diff --git a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java index 0d5bc81b024d..905f415e35b5 100644 --- a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java @@ -72,13 +72,13 @@ public void minorNeverPurgeTombstonesTest() throws Throwable { execute("INSERT INTO %s (a, b, c) VALUES (" + j + ", 2, '3')"); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } execute("UPDATE %s SET c = null WHERE a=1 AND b=2"); execute("DELETE FROM %s WHERE a=2 AND b=2"); execute("DELETE FROM %s WHERE a=3"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.enableAutoCompaction(); while (cfs.getLiveSSTables().size() > 1 || !cfs.getTracker().getCompacting().isEmpty()) Thread.sleep(100); @@ -92,7 +92,7 @@ private void testHelper(String deletionStatement) throws Throwable execute("INSERT INTO %s (a, b, c) VALUES (1, 2, '3')"); execute(deletionStatement); Thread.sleep(1000); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.forceMajorCompaction(); verifyContainsTombstones(cfs.getLiveSSTables(), 1); } diff --git a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java index 0c469dc534b3..41fdadec8cdc 100644 --- a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java @@ -71,7 +71,7 @@ private void testCompaction(String columnFamilyName, int insertsPerTable) .applyUnsafe(); inserted.add(key); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(inserted.size(), Util.getAll(Util.cmd(store).build()).size()); } CompactionManager.instance.performMaximal(store, false); diff --git a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java index 61cf302d6c51..38499cde415c 100644 --- a/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/SingleSSTableLCSTaskTest.java @@ -42,7 +42,7 @@ public void basicTest() throws Throwable createTable("create table %s (id int primary key, t text) with compaction = {'class':'LeveledCompactionStrategy','single_sstable_uplevel':true}"); ColumnFamilyStore cfs = getCurrentColumnFamilyStore(); execute("insert into %s (id, t) values (1, 'meep')"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstable, OperationType.COMPACTION)) @@ -95,7 +95,7 @@ private void compactionTestHelper(boolean singleSSTUplevel) throws Throwable execute("insert into %s (id, id2, t) values (?, ?, ?)", i, j, value); } if (i % 100 == 0) - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } // now we have a bunch of data in L0, first compaction will be a normal one, containing all sstables: LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) cfs.getCompactionStrategyManager().getUnrepairedUnsafe().first(); @@ -123,7 +123,7 @@ public void corruptMetadataTest() throws Throwable createTable("create table %s (id int primary key, t text) with compaction = {'class':'LeveledCompactionStrategy','single_sstable_uplevel':true}"); ColumnFamilyStore cfs = getCurrentColumnFamilyStore(); execute("insert into %s (id, t) values (1, 'meep')"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); String filenameToCorrupt = sstable.descriptor.filenameFor(Component.STATS); diff --git a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java index 00c4a86e0dd0..a7313a5d4945 100644 --- a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java @@ -165,9 +165,9 @@ public void testPrepBucket() throws Exception new RowUpdateBuilder(cfs.metadata(), 0, key) .clustering("column").add("val", value) .build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); List sstrs = new ArrayList<>(cfs.getLiveSSTables()); Pair, Double> bucket; diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java index a2352fcf02aa..31dbb09441ae 100644 --- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java @@ -94,7 +94,7 @@ public void testAggressiveFullyExpired() .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 2L, 1, key) .add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build() @@ -105,7 +105,7 @@ public void testAggressiveFullyExpired() .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 4L, 1, key) .add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER) @@ -117,7 +117,7 @@ public void testAggressiveFullyExpired() .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), 6L, 3, key) @@ -130,7 +130,7 @@ public void testAggressiveFullyExpired() .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Set sstables = Sets.newHashSet(cfs.getLiveSSTables()); int now = (int)(System.currentTimeMillis() / 1000); @@ -173,7 +173,7 @@ public void testSimpleExpire(boolean force10944Bug) throws InterruptedException .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key) .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER) @@ -183,7 +183,7 @@ public void testSimpleExpire(boolean force10944Bug) throws InterruptedException .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // To reproduce #10944, we need to avoid the optimization that get rid of full sstable because everything // is known to be gcAble, so keep some data non-expiring in that case. new RowUpdateBuilder(cfs.metadata(), timestamp, force10944Bug ? 0 : 1, key) @@ -192,14 +192,14 @@ public void testSimpleExpire(boolean force10944Bug) throws InterruptedException .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key) .add("col311", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Thread.sleep(2000); // wait for ttl to expire assertEquals(4, cfs.getLiveSSTables().size()); cfs.enableAutoCompaction(true); @@ -221,24 +221,24 @@ public void testNoExpire() throws InterruptedException, IOException .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key) .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key) .add("col3", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); String noTTLKey = "nottl"; new RowUpdateBuilder(cfs.metadata(), timestamp, noTTLKey) .add("col311", ByteBufferUtil.EMPTY_BYTE_BUFFER) .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Thread.sleep(2000); // wait for ttl to expire assertEquals(4, cfs.getLiveSSTables().size()); cfs.enableAutoCompaction(true); @@ -270,7 +270,7 @@ public void testCheckForExpiredSSTableBlockers() throws InterruptedException .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader blockingSSTable = cfs.getSSTables(SSTableSet.LIVE).iterator().next(); for (int i = 0; i < 10; i++) { @@ -279,7 +279,7 @@ public void testCheckForExpiredSSTableBlockers() throws InterruptedException .delete("col1") .build() .applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } Multimap blockers = SSTableExpiredBlockers.checkForExpiredSSTableBlockers(cfs.getSSTables(SSTableSet.LIVE), (int) (System.currentTimeMillis() / 1000) + 100); assertEquals(1, blockers.keySet().size()); diff --git a/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java index 89dd2f59b162..69a43fc94791 100644 --- a/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java @@ -168,7 +168,7 @@ public void testPrepBucket() .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } // Decrement the timestamp to simulate a timestamp in the past hour for (int r = 3; r < 5; r++) @@ -178,10 +178,10 @@ public void testPrepBucket() new RowUpdateBuilder(cfs.metadata(), r, key.getKey()) .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); HashMultimap buckets = HashMultimap.create(); List sstrs = new ArrayList<>(cfs.getLiveSSTables()); @@ -220,7 +220,7 @@ public void testPrepBucket() .clustering("column") .add("val", value).build().applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } // Reset the buckets, overfill it now @@ -252,7 +252,7 @@ public void testDropExpiredSSTables() throws InterruptedException .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader expiredSSTable = cfs.getLiveSSTables().iterator().next(); Thread.sleep(10); @@ -261,7 +261,7 @@ public void testDropExpiredSSTables() throws InterruptedException .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(cfs.getLiveSSTables().size(), 2); Map options = new HashMap<>(); @@ -300,7 +300,7 @@ public void testDropOverlappingExpiredSSTables() throws InterruptedException .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader expiredSSTable = cfs.getLiveSSTables().iterator().next(); Thread.sleep(10); @@ -312,7 +312,7 @@ public void testDropOverlappingExpiredSSTables() throws InterruptedException .clustering("column") .add("val", value).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(cfs.getLiveSSTables().size(), 2); Map options = new HashMap<>(); diff --git a/test/unit/org/apache/cassandra/db/repair/AbstractPendingAntiCompactionTest.java b/test/unit/org/apache/cassandra/db/repair/AbstractPendingAntiCompactionTest.java index 62b7db148465..5a4d7c171ce8 100644 --- a/test/unit/org/apache/cassandra/db/repair/AbstractPendingAntiCompactionTest.java +++ b/test/unit/org/apache/cassandra/db/repair/AbstractPendingAntiCompactionTest.java @@ -109,7 +109,7 @@ void makeSSTables(int num, ColumnFamilyStore cfs, int rowsPerSSTable) int val = i * rowsPerSSTable; // multiplied to prevent ranges from overlapping for (int j = 0; j < rowsPerSSTable; j++) QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, v) VALUES (?, ?)", ks, cfs.getTableName()), val + j, val + j); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } Assert.assertEquals(num, cfs.getLiveSSTables().size()); } diff --git a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java index 3b29cc5b50d7..08044e9c7346 100644 --- a/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java +++ b/test/unit/org/apache/cassandra/db/repair/CompactionManagerGetSSTablesForValidationTest.java @@ -93,7 +93,7 @@ private void makeSSTables() for (int i=0; i<3; i++) { QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, v) VALUES(?, ?)", ks, tbl), i, i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } Assert.assertEquals(3, cfs.getLiveSSTables().size()); diff --git a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java index b140813f48e3..b91c315a0db1 100644 --- a/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java +++ b/test/unit/org/apache/cassandra/db/repair/PendingAntiCompactionTest.java @@ -122,12 +122,12 @@ public void successCase() throws Exception { QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, v) VALUES (?, ?)", ks, tbl), i, i); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); for (int i = 8; i < 12; i++) { QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (k, v) VALUES (?, ?)", ks, tbl), i, i); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(2, cfs.getLiveSSTables().size()); Token left = ByteOrderedPartitioner.instance.getToken(ByteBufferUtil.bytes((int) 6)); diff --git a/test/unit/org/apache/cassandra/db/rows/ThrottledUnfilteredIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/ThrottledUnfilteredIteratorTest.java index cc886f1c24f8..bba7334dee4a 100644 --- a/test/unit/org/apache/cassandra/db/rows/ThrottledUnfilteredIteratorTest.java +++ b/test/unit/org/apache/cassandra/db/rows/ThrottledUnfilteredIteratorTest.java @@ -111,7 +111,7 @@ public void emptyPartitionDeletionTest() throws Throwable // flush and generate 1 sstable ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.disableAutoCompaction(); cfs.forceMajorCompaction(); @@ -146,7 +146,7 @@ public void emptyStaticTest() throws Throwable // flush and generate 1 sstable ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.disableAutoCompaction(); cfs.forceMajorCompaction(); @@ -204,7 +204,7 @@ else if (ck1 == ck2 - 1) // cell tombstone // flush and generate 1 sstable ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.disableAutoCompaction(); cfs.forceMajorCompaction(); @@ -623,7 +623,7 @@ public void testThrottledIteratorWithRangeDeletions() throws Exception new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(10, 22).build().applyUnsafe(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(2); for (int i = 1; i < 40; i += 2) diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java index c722738a272e..baee189af016 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraEntireSSTableStreamWriterTest.java @@ -100,7 +100,7 @@ public static void defineSchemaAndPrepareSSTable() .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); sstable = store.getLiveSSTables().iterator().next(); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java index 5e443463d522..bf25f1a1931f 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraOutgoingFileTest.java @@ -78,7 +78,7 @@ public static void defineSchemaAndPrepareSSTable() .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); sstable = store.getLiveSSTables().iterator().next(); diff --git a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java index eb15e9aa7339..93f769b66479 100644 --- a/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java +++ b/test/unit/org/apache/cassandra/db/streaming/CassandraStreamManagerTest.java @@ -113,7 +113,7 @@ private SSTableReader createSSTable(Runnable queryable) { Set before = cfs.getLiveSSTables(); queryable.run(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Set after = cfs.getLiveSSTables(); Set diff = Sets.difference(after, before); diff --git a/test/unit/org/apache/cassandra/db/view/ViewBuilderTaskTest.java b/test/unit/org/apache/cassandra/db/view/ViewBuilderTaskTest.java index 2341c730a423..bc306791a1c4 100644 --- a/test/unit/org/apache/cassandra/db/view/ViewBuilderTaskTest.java +++ b/test/unit/org/apache/cassandra/db/view/ViewBuilderTaskTest.java @@ -85,7 +85,7 @@ private void test(int indexOfStartToken, { // Truncate the materialized view (not the base table) cfs.viewManager.forceBlockingFlush(); - cfs.viewManager.truncateBlocking(cfs.forceBlockingFlush(), System.currentTimeMillis()); + cfs.viewManager.truncateBlocking(cfs.forceBlockingFlushToSSTable(), System.currentTimeMillis()); assertRowCount(execute("SELECT * FROM " + viewName), 0); // Get the tokens from the referenced inserted rows diff --git a/test/unit/org/apache/cassandra/index/CustomIndexTest.java b/test/unit/org/apache/cassandra/index/CustomIndexTest.java index 35e0353f82d3..dfab35df400b 100644 --- a/test/unit/org/apache/cassandra/index/CustomIndexTest.java +++ b/test/unit/org/apache/cassandra/index/CustomIndexTest.java @@ -639,7 +639,7 @@ public void testFailing2iFlush() throws Throwable try { - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); fail("Exception should have been propagated"); } catch (Throwable t) @@ -661,7 +661,7 @@ public void indexBuildingPagesLargePartitions() throws Throwable // Insert a single wide partition to be indexed for (int i = 0; i < totalRows; i++) execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // Create the index, which won't automatically start building String indexName = "build_single_partition_idx"; @@ -714,7 +714,7 @@ public void partitionIndexTest() throws Throwable execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 5, 3, 3); execute("DELETE FROM %s WHERE k = ?", 5); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); String indexName = "partition_index_test_idx"; createIndex(String.format("CREATE CUSTOM INDEX %s ON %%s(v) USING '%s'", @@ -776,7 +776,7 @@ public void partitionIsNotOverIndexed() throws Throwable // Insert a single row partition to be indexed for (int i = 0; i < totalRows; i++) execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // Create the index, which won't automatically start building String indexName = "partition_overindex_test_idx"; @@ -802,7 +802,7 @@ public void rangeTombstoneTest() throws Throwable // Insert a single range tombstone execute("DELETE FROM %s WHERE k=1 and c > 2"); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // Create the index, which won't automatically start building String indexName = "range_tombstone_idx"; diff --git a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java index 04db7f6f90bd..3e43f11ef46a 100644 --- a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java +++ b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java @@ -135,7 +135,7 @@ public Optional getBackingTable() public Callable getBlockingFlushTask() { return () -> { - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlushToSSTable(); return null; }; } @@ -597,7 +597,7 @@ private void invalidate() CompactionManager.instance.interruptCompactionForCFs(cfss, (sstable) -> true, true); CompactionManager.instance.waitForCessation(cfss, (sstable) -> true); indexCfs.keyspace.writeOrder.awaitNewBarrier(); - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlushToSSTable(); indexCfs.readOrdering.awaitNewBarrier(); indexCfs.invalidate(); } @@ -622,7 +622,7 @@ private Callable getBuildIndexTask() private void buildBlocking() { - baseCfs.forceBlockingFlush(); + baseCfs.forceBlockingFlushToSSTable(); try (ColumnFamilyStore.RefViewFragment viewFragment = baseCfs.selectAndReference(View.selectFunction(SSTableSet.CANONICAL)); Refs sstables = viewFragment.refs) @@ -646,7 +646,7 @@ private void buildBlocking() ImmutableSet.copyOf(sstables)); Future future = CompactionManager.instance.submitIndexBuild(builder); FBUtilities.waitOnFuture(future); - indexCfs.forceBlockingFlush(); + indexCfs.forceBlockingFlushToSSTable(); } logger.info("Index build of {} complete", metadata.name); } diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java index e3ba45b44c1e..ed249f624e7b 100644 --- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java +++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java @@ -453,7 +453,7 @@ private void testPrefixSearchWithContainsMode(boolean forceFlush) throws Excepti if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); final UntypedResultSet results = executeCQL(FTS_CF_NAME, "SELECT * FROM %s.%s WHERE artist LIKE 'lady%%'"); Assert.assertNotNull(results); @@ -807,7 +807,7 @@ private void testColumnNamesWithSlashes(boolean forceFlush) throws Exception rm3.build().apply(); if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); final ByteBuffer dataOutputId = UTF8Type.instance.decompose("/data/output/id"); @@ -969,7 +969,7 @@ private void redistributeSummaries(int expected, ColumnFamilyStore store, ByteBu { setMinIndexInterval(minIndexInterval); IndexSummaryManager.instance.redistributeSummaries(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); Set rows = getIndexed(store, 100, buildExpression(firstName, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a"))); Assert.assertEquals(rows.toString(), expected, rows.size()); @@ -1189,7 +1189,7 @@ public void testInsertingIncorrectValuesIntoAgeIndex() }}); rm.build().apply(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); Set rows = getIndexed(store, 10, buildExpression(firstName, Operator.EQ, UTF8Type.instance.decompose("a")), buildExpression(age, Operator.GTE, Int32Type.instance.decompose(26))); @@ -1234,7 +1234,7 @@ private void testUnicodeSupport(boolean forceFlush) rm.build().apply(); if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); Set rows; @@ -1306,7 +1306,7 @@ private void testUnicodeSuffixModeNoSplits(boolean forceFlush) rm.build().apply(); if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); Set rows; @@ -1367,7 +1367,7 @@ public void testThatTooBigValueIsRejected() rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate())); Assert.assertEquals(0, rows.size()); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); rows = getIndexed(store, 10, buildExpression(comment, Operator.LIKE_MATCHES, bigValue.duplicate())); Assert.assertEquals(0, rows.size()); @@ -1470,7 +1470,7 @@ public void testChinesePrefixSearch() update(rm, fullName, UTF8Type.instance.decompose("利久 寺地"), System.currentTimeMillis()); rm.build().apply(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); Set rows; @@ -1507,7 +1507,7 @@ public void testLowerCaseAnalyzer(boolean forceFlush) rm.build().apply(); if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); Set rows; @@ -1592,7 +1592,7 @@ public void testPrefixSSTableLookup() rm.build().apply(); // first flush would make interval for name - 'johnny' -> 'pavel' - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); rm = new Mutation.PartitionUpdateCollector(KS_NAME, decoratedKey("key6")); update(rm, name, UTF8Type.instance.decompose("Jason"), System.currentTimeMillis()); @@ -1607,7 +1607,7 @@ public void testPrefixSSTableLookup() rm.build().apply(); // this flush is going to produce range - 'jason' -> 'vijay' - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); // make sure that overlap of the prefixes is properly handled across sstables // since simple interval tree lookup is not going to cover it, prefix lookup actually required. @@ -1771,7 +1771,7 @@ public void testClusteringIndexes(boolean forceFlush) throws Exception executeCQL(CLUSTERING_CF_NAME_1 ,"INSERT INTO %s.%s (name, nickname, location, age, height, score) VALUES (?, ?, ?, ?, ?, ?)", "Jordan", "jrwest", "US", 27, 182, 1.0); if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); UntypedResultSet results; @@ -1858,7 +1858,7 @@ public void testClusteringIndexes(boolean forceFlush) throws Exception executeCQL(CLUSTERING_CF_NAME_2 ,"INSERT INTO %s.%s (name, nickname, location, age, height, score) VALUES (?, ?, ?, ?, ?, ?)", "Christopher", "chis", "US", 27, 180, 1.0); if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); results = executeCQL(CLUSTERING_CF_NAME_2 ,"SELECT * FROM %s.%s WHERE location LIKE 'US' AND age = 43 ALLOW FILTERING"); Assert.assertNotNull(results); @@ -1884,7 +1884,7 @@ public void testStaticIndex(boolean shouldFlush) throws Exception executeCQL(STATIC_CF_NAME, "INSERT INTO %s.%s (sensor_id,date,value,variance) VALUES(?, ?, ?, ?)", 1, 20160403L, 24.96, 4); if (shouldFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); executeCQL(STATIC_CF_NAME, "INSERT INTO %s.%s (sensor_id,sensor_type) VALUES(?, ?)", 2, "PRESSURE"); executeCQL(STATIC_CF_NAME, "INSERT INTO %s.%s (sensor_id,date,value,variance) VALUES(?, ?, ?, ?)", 2, 20160401L, 1.03, 9); @@ -1892,7 +1892,7 @@ public void testStaticIndex(boolean shouldFlush) throws Exception executeCQL(STATIC_CF_NAME, "INSERT INTO %s.%s (sensor_id,date,value,variance) VALUES(?, ?, ?, ?)", 2, 20160403L, 1.01, 4); if (shouldFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); UntypedResultSet results; @@ -1973,7 +1973,7 @@ public void testTableRebuild() throws Exception executeCQL(CLUSTERING_CF_NAME_1, "INSERT INTO %s.%s (name, location, age, height, score) VALUES (?, ?, ?, ?, ?)", "Pavel", "BY", 28, 182, 2.0); executeCQL(CLUSTERING_CF_NAME_1, "INSERT INTO %s.%s (name, nickname, location, age, height, score) VALUES (?, ?, ?, ?, ?, ?)", "Jordan", "jrwest", "US", 27, 182, 1.0); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); SSTable ssTable = store.getSSTables(SSTableSet.LIVE).iterator().next(); Path path = FileSystems.getDefault().getPath(ssTable.getFilename().replace("-Data", "-SI_" + CLUSTERING_CF_NAME_1 + "_age")); @@ -2010,7 +2010,7 @@ public void testIndexRebuild() throws Exception executeCQL(CLUSTERING_CF_NAME_1, "INSERT INTO %s.%s (name, nickname) VALUES (?, ?)", "Alex", "ifesdjeen"); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); for (Index index : store.indexManager.listIndexes()) { @@ -2136,7 +2136,7 @@ private void testLIKEAndEQSemanticsWithDifferenceKindsOfIndexes(String containsT { Keyspace keyspace = Keyspace.open(KS_NAME); for (String table : Arrays.asList(containsTable, prefixTable, analyzedPrefixTable)) - keyspace.getColumnFamilyStore(table).forceBlockingFlush(); + keyspace.getColumnFamilyStore(table).forceBlockingFlushToSSTable(); } UntypedResultSet results; @@ -2370,7 +2370,7 @@ public void testIndexMemtableSwitching() Assert.assertTrue(beforeFlushMemtable.search(expression).getCount() > 0); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); IndexMemtable afterFlushMemtable = index.getCurrentMemtable(); @@ -2501,7 +2501,7 @@ private static ColumnFamilyStore loadData(Map> dat ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME); if (forceFlush) - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); return store; } diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java index 461c13cd09ee..f02820b7badc 100644 --- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java +++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterReopenTest.java @@ -78,12 +78,12 @@ public void compressionEnabled() throws Throwable { execute("insert into %s (id, t) values (?, ?)", i, ByteBuffer.wrap(blob)); } - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); for (int i = 0; i < 10000; i++) { execute("insert into %s (id, t) values (?, ?)", i, ByteBuffer.wrap(blob)); } - getCurrentColumnFamilyStore().forceBlockingFlush(); + getCurrentColumnFamilyStore().forceBlockingFlushToSSTable(); DatabaseDescriptor.setSSTablePreemptiveOpenIntervalInMB(1); getCurrentColumnFamilyStore().forceMajorCompaction(); } diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java index 68ee3e1931bc..eb98643cadcb 100644 --- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java @@ -23,7 +23,6 @@ import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import java.util.stream.Collectors; import com.google.common.base.Joiner; import com.google.common.collect.Sets; @@ -42,7 +41,6 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.RowUpdateBuilder; -import org.apache.cassandra.db.compaction.AntiCompactionTest; import org.apache.cassandra.db.compaction.CompactionInfo; import org.apache.cassandra.db.compaction.CompactionInterruptedException; import org.apache.cassandra.db.compaction.CompactionManager; @@ -51,7 +49,6 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.metrics.CompactionMetrics; import org.apache.cassandra.metrics.RestorableMeter; import org.apache.cassandra.schema.CachingParams; import org.apache.cassandra.schema.KeyspaceParams; @@ -194,7 +191,7 @@ private void createSSTables(String ksname, String cfname, int numSSTables, int n .build() .applyUnsafe(); } - futures.add(cfs.forceFlush()); + futures.add(cfs.forceFlushToSSTable()); } for (Future future : futures) { @@ -519,7 +516,7 @@ public void testRebuildAtSamplingLevel() throws IOException .applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); List sstables = new ArrayList<>(cfs.getLiveSSTables()); assertEquals(1, sstables.size()); @@ -584,7 +581,7 @@ public void testJMXFunctions() throws IOException .build() .applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } assertTrue(manager.getAverageIndexInterval() >= cfs.metadata().params.minIndexInterval); diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryRedistributionTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryRedistributionTest.java index 07a2212e8f9d..919e873a9009 100644 --- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryRedistributionTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryRedistributionTest.java @@ -125,7 +125,7 @@ private void createSSTables(String ksname, String cfname, int numSSTables, int n .build() .applyUnsafe(); } - futures.add(cfs.forceFlush()); + futures.add(cfs.forceFlushToSSTable()); } for (Future future : futures) { diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java index 2510c5e8a487..a44f6921056c 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java @@ -117,7 +117,7 @@ public static void setUp() .add("reg2", ByteBuffer.wrap(reg2)); writer.append(builder.build().unfilteredIterator()); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ssTableReader = writer.finish(true); txn.update(ssTableReader, false); diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java index 5d40f8cc6218..13cdfd38fd23 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java @@ -138,7 +138,7 @@ public void testLoadingSSTable() throws Exception } ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1); - cfs.forceBlockingFlush(); // wait for sstables to be on disk else we won't be able to stream them + cfs.forceBlockingFlushToSSTable(); // wait for sstables to be on disk else we won't be able to stream them final CountDownLatch latch = new CountDownLatch(1); SSTableLoader loader = new SSTableLoader(dataDir, new TestClient(), new OutputHandler.SystemOutput(false, false)); @@ -185,7 +185,7 @@ public void testLoadingIncompleteSSTable() throws Exception } ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD2); - cfs.forceBlockingFlush(); // wait for sstables to be on disk else we won't be able to stream them + cfs.forceBlockingFlushToSSTable(); // wait for sstables to be on disk else we won't be able to stream them //make sure we have some tables... assertTrue(dataDir.listFiles().length > 0); @@ -233,14 +233,14 @@ public void testLoadingSSTableToDifferentKeyspace() throws Exception } ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1); - cfs.forceBlockingFlush(); // wait for sstables to be on disk else we won't be able to stream them + cfs.forceBlockingFlushToSSTable(); // wait for sstables to be on disk else we won't be able to stream them final CountDownLatch latch = new CountDownLatch(1); SSTableLoader loader = new SSTableLoader(dataDir, new TestClient(), new OutputHandler.SystemOutput(false, false), 1, KEYSPACE2); loader.stream(Collections.emptySet(), completionStreamListener(latch)).get(); cfs = Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); List partitions = Util.getAll(Util.cmd(cfs).build()); diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java index 98b356a0ae75..3f103167f778 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java @@ -93,7 +93,7 @@ public void testTrackMaxDeletionTime() .applyUnsafe(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1, store.getLiveSSTables().size()); int ttltimestamp = (int)(System.currentTimeMillis()/1000); int firstDelTime = 0; @@ -112,7 +112,7 @@ public void testTrackMaxDeletionTime() ttltimestamp = (int) (System.currentTimeMillis()/1000); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(2, store.getLiveSSTables().size()); List sstables = new ArrayList<>(store.getLiveSSTables()); if(sstables.get(0).getSSTableMetadata().maxLocalDeletionTime < sstables.get(1).getSSTableMetadata().maxLocalDeletionTime) @@ -166,7 +166,7 @@ public void testWithDeletes() throws ExecutionException, InterruptedException .build() .applyUnsafe(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1,store.getLiveSSTables().size()); int ttltimestamp = (int) (System.currentTimeMillis()/1000); int firstMaxDelTime = 0; @@ -178,7 +178,7 @@ public void testWithDeletes() throws ExecutionException, InterruptedException RowUpdateBuilder.deleteRow(store.metadata(), timestamp + 1, "deletetest", "todelete").applyUnsafe(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(2,store.getLiveSSTables().size()); boolean foundDelete = false; for(SSTableReader sstable : store.getLiveSSTables()) @@ -215,7 +215,7 @@ public void trackMaxMinColNames() throws CharacterCodingException, ExecutionExce .applyUnsafe(); } } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1, store.getLiveSSTables().size()); for (SSTableReader sstable : store.getLiveSSTables()) { @@ -233,7 +233,7 @@ public void trackMaxMinColNames() throws CharacterCodingException, ExecutionExce .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); store.forceMajorCompaction(); assertEquals(1, store.getLiveSSTables().size()); for (SSTableReader sstable : store.getLiveSSTables()) diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java index 580b099bda98..16b78ba12807 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java @@ -22,7 +22,6 @@ import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.attribute.FileAttribute; import java.util.*; import java.util.concurrent.*; @@ -111,7 +110,7 @@ public void testGetPositionsForRanges() .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); List> ranges = new ArrayList>(); @@ -157,7 +156,7 @@ public void testSpannedIndexPositions() throws IOException .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); // check that all our keys are found correctly @@ -199,7 +198,7 @@ public void testPersistentStatistics() .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); clearAndLoad(store); assert store.metric.maxPartitionSize.getValue() != 0; @@ -228,7 +227,7 @@ public void testReadRateTracking() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); SSTableReader sstable = store.getLiveSSTables().iterator().next(); assertEquals(0, sstable.getReadMeter().count()); @@ -261,7 +260,7 @@ public void testGetPositionsForRangesWithKeyCache() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); SSTableReader sstable = store.getLiveSSTables().iterator().next(); @@ -293,7 +292,7 @@ public void testPersistentStatisticsWithSecondaryIndex() .build() .applyUnsafe(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); // check if opening and querying works assertIndexQueryWorks(store); @@ -315,7 +314,7 @@ public void testGetPositionsKeyCacheStats() .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); SSTableReader sstable = store.getLiveSSTables().iterator().next(); @@ -363,7 +362,7 @@ public void testOpeningSSTable() throws Exception .build() .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); SSTableReader sstable = store.getLiveSSTables().iterator().next(); Descriptor desc = sstable.descriptor; @@ -465,7 +464,7 @@ public void testLoadingSummaryUsesCorrectPartitioner() throws Exception .build() .applyUnsafe(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); for(ColumnFamilyStore indexCfs : store.indexManager.getAllIndexColumnFamilyStores()) { @@ -494,7 +493,7 @@ public void testGetScannerForNoIntersectingRanges() throws Exception .build() .applyUnsafe(); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); boolean foundScanner = false; for (SSTableReader s : store.getLiveSSTables()) { @@ -528,7 +527,7 @@ public void testGetPositionsForRangesFromTableOpenedForBulkLoading() throws IOEx .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); // construct a range which is present in the sstable, but whose @@ -567,7 +566,7 @@ public void testIndexSummaryReplacement() throws IOException, ExecutionException .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); Collection sstables = store.getLiveSSTables(); @@ -646,7 +645,7 @@ private void testIndexSummaryUpsampleAndReload0() throws Exception .applyUnsafe(); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); CompactionManager.instance.performMaximal(store, false); Collection sstables = store.getLiveSSTables(); @@ -759,7 +758,7 @@ private SSTableReader getNewSSTable(ColumnFamilyStore cfs) .build() .applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); return Sets.difference(cfs.getLiveSSTables(), before).iterator().next(); } diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java index 7c47c8b9beb0..5e562616beab 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java @@ -22,19 +22,15 @@ import java.io.IOException; import java.util.*; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import com.google.common.util.concurrent.Uninterruptibles; import org.junit.Test; import org.apache.cassandra.Util; import org.apache.cassandra.UpdateBuilder; import org.apache.cassandra.concurrent.NamedThreadFactory; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Keyspace; @@ -56,8 +52,6 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.db.lifecycle.LifecycleTransaction; import org.apache.cassandra.metrics.StorageMetrics; -import org.apache.cassandra.streaming.PreviewKind; -import org.apache.cassandra.streaming.StreamSession; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.UUIDGen; @@ -81,7 +75,7 @@ public void basicTest() throws InterruptedException .build() .apply(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Set sstables = new HashSet<>(cfs.getLiveSSTables()); assertEquals(1, sstables.size()); assertEquals(sstables.iterator().next().bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount()); @@ -698,7 +692,7 @@ public void testAllKeysReadable() throws Exception .build() .apply(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); cfs.forceMajorCompaction(); validateKeys(keyspace); diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java index eff95fccbb1c..46df324cc1e0 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java @@ -215,7 +215,7 @@ public void testSingleDataRange() throws IOException for (int i = 2; i < 10; i++) insertRowWithKey(store.metadata(), i); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1, store.getLiveSSTables().size()); SSTableReader sstable = store.getLiveSSTables().iterator().next(); @@ -321,7 +321,7 @@ public void testMultipleRanges() throws IOException for (int i = 0; i < 3; i++) for (int j = 2; j < 10; j++) insertRowWithKey(store.metadata(), i * 100 + j); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1, store.getLiveSSTables().size()); SSTableReader sstable = store.getLiveSSTables().iterator().next(); @@ -441,7 +441,7 @@ public void testSingleKeyMultipleRanges() throws IOException store.disableAutoCompaction(); insertRowWithKey(store.metadata(), 205); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertEquals(1, store.getLiveSSTables().size()); SSTableReader sstable = store.getLiveSSTables().iterator().next(); diff --git a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java index 3cf96f2698f8..1aef85103745 100644 --- a/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/format/big/BigTableZeroCopyWriterTest.java @@ -116,7 +116,7 @@ public static void defineSchema() throws Exception .applyUnsafe(); expectedRowCount++; } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); sstable = store.getLiveSSTables().iterator().next(); } diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java index a288edb1de61..20f50ed3fce6 100644 --- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java +++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java @@ -58,7 +58,6 @@ import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.MerkleTree; import org.apache.cassandra.utils.MerkleTrees; -import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.UUIDGen; import static org.junit.Assert.assertEquals; @@ -188,7 +187,7 @@ public void simpleValidationTest(int n) throws Exception CompactionsTest.populate(keyspace, columnFamily, 0, n, 0); //ttl=3s - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(1, cfs.getLiveSSTables().size()); // wait enough to force single compaction @@ -245,7 +244,7 @@ public void testSizeLimiting() throws Exception // 2 ** 14 rows would normally use 2^14 leaves, but with only 1 meg we should only use 2^12 CompactionsTest.populate(keyspace, columnFamily, 0, 1 << 14, 0); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(1, cfs.getLiveSSTables().size()); // wait enough to force single compaction @@ -304,7 +303,7 @@ public void testRangeSplittingTreeSizeLimit() throws Exception // 2 ** 14 rows would normally use 2^14 leaves, but with only 1 meg we should only use 2^12 CompactionsTest.populate(keyspace, columnFamily, 0, 1 << 14, 0); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertEquals(1, cfs.getLiveSSTables().size()); // wait enough to force single compaction diff --git a/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java index 5c709036c00d..f7dedea702d6 100644 --- a/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java +++ b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java @@ -179,7 +179,7 @@ public void addNewTable() throws ConfigurationException // flush to exercise more than just hitting the memtable ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(tableName); assertNotNull(cfs); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); // and make sure we get out what we put in UntypedResultSet rows = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s", ksName, tableName)); @@ -202,7 +202,7 @@ public void dropCf() throws ConfigurationException "dropCf", "col" + i, "anyvalue"); ColumnFamilyStore store = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name); assertNotNull(store); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); assertTrue(store.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().size() > 0); MigrationManager.announceTableDrop(ks.name, cfm.name, false); @@ -251,7 +251,7 @@ public void addNewKS() throws ConfigurationException "key0", "col0", "val0"); ColumnFamilyStore store = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name); assertNotNull(store); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); UntypedResultSet rows = QueryProcessor.executeInternal("SELECT * FROM newkeyspace1.newstandard1"); assertRows(rows, row("key0", "col0", "val0")); @@ -273,7 +273,7 @@ public void dropKS() throws ConfigurationException "dropKs", "col" + i, "anyvalue"); ColumnFamilyStore cfs = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name); assertNotNull(cfs); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); assertTrue(!cfs.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().isEmpty()); MigrationManager.announceKeyspaceDrop(ks.name); @@ -354,7 +354,7 @@ public void createEmptyKsAddNewCf() throws ConfigurationException ColumnFamilyStore cfs = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.name); assertNotNull(cfs); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); UntypedResultSet rows = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s", EMPTY_KEYSPACE, tableName)); assertRows(rows, row("key0", "col0", "val0")); @@ -509,7 +509,7 @@ public void testDropIndex() throws ConfigurationException TABLE1i), "key0", "col0", 1L, 1L); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); ColumnFamilyStore indexCfs = cfs.indexManager.getIndexByName(indexName) .getBackingTable() .orElseThrow(throwAssert("Cannot access index cfs")); diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java index 4f7cde035acb..935dd19ec15c 100644 --- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java +++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java @@ -304,7 +304,7 @@ private void createSSTables(ColumnFamilyStore cfs, int count) .build() .applyUnsafe(); } - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } } diff --git a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java index 3ae49edf96a2..ffa12d19766d 100644 --- a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java +++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java @@ -101,7 +101,7 @@ public void testTombstoneWarning() throws Exception client.execute(query); } ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()); - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); for (int i = 0; i < iterations; i++) { @@ -111,7 +111,7 @@ public void testTombstoneWarning() throws Exception i), QueryOptions.DEFAULT); client.execute(query); } - store.forceBlockingFlush(); + store.forceBlockingFlushToSSTable(); { QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1", diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java index 80607c56ea9a..ae4e766ff764 100644 --- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java +++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java @@ -83,7 +83,7 @@ public void testScheduleTimeout() throws Exception for (int i = 0; i < 2; i++) { SchemaLoader.insertData(KEYSPACE1, CF_STANDARD, i, 1); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } // create streaming task that streams those two sstables @@ -132,7 +132,7 @@ public void testFailSessionDuringTransferShouldNotReleaseReferences() throws Exc for (int i = 0; i < 2; i++) { SchemaLoader.insertData(KEYSPACE1, CF_STANDARD, i, 1); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); } // create streaming task that streams those two sstables diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java index 909e221ae28b..9746fd08449c 100644 --- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java +++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java @@ -174,7 +174,7 @@ private List createAndTransfer(ColumnFamilyStore cfs, Mutator mutator, b long timestamp = 1234; for (int i = 1; i <= 3; i++) mutator.mutate("key" + i, "col" + i, timestamp); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); Util.compactAll(cfs, Integer.MAX_VALUE).get(); assertEquals(1, cfs.getLiveSSTables().size()); @@ -362,7 +362,7 @@ public void testTransferRangeTombstones() throws Exception .build() .apply(); - cfs.forceBlockingFlush(); + cfs.forceBlockingFlushToSSTable(); SSTableReader sstable = cfs.getLiveSSTables().iterator().next(); cfs.clearUnsafe(); From c9fc09828cafd81761db029a2c91616facc7127d Mon Sep 17 00:00:00 2001 From: Josh McKenzie Date: Sun, 13 Oct 2019 14:00:00 -0400 Subject: [PATCH 76/78] Refactor commit log testing and cdc testing to support resumable testing The original CDC CommitLogReadHandler for test only handled a boolean state of whether or not a cdc mutation was seen or not. This opens that up to allow for a tracking of a MutationIdentifier to determine if the specific mutation we're seeing on replay is a duplicate and if so, how many times we've seen it. This will play into resumable clreader testing in subsequent commits (some risk of yagni here, but have a general framework of how I want to test this in my head. We'll see.) --- resumable_todo.txt | 10 ++ .../db/commitlog/CommitLogSegmentManager.java | 4 +- .../db/commitlog/CommitLogSegmentReader.java | 18 +-- .../commitlog/ResumableCommitLogReader.java | 8 +- .../org/apache/cassandra/cql3/CQLTester.java | 36 +++++ .../db/commitlog/CDCTestReplayer.java | 76 --------- .../CommitLogSegmentAllocatorCDCTest.java | 16 +- .../db/commitlog/CommitLogTestUtils.java | 145 +++++++++++++++++- .../ResumableCommitLogReaderTest.java | 75 +++++---- 9 files changed, 260 insertions(+), 128 deletions(-) create mode 100644 resumable_todo.txt delete mode 100644 test/unit/org/apache/cassandra/db/commitlog/CDCTestReplayer.java diff --git a/resumable_todo.txt b/resumable_todo.txt new file mode 100644 index 000000000000..6aa782aa0214 --- /dev/null +++ b/resumable_todo.txt @@ -0,0 +1,10 @@ +ToDo: + +Get the unit test prep up and working + + Finish stubbing in CDCMockInitiator + + processMutation + + Get back to initial test + + testNonResumedGeneralCase -> test out the infra / piping of the logic on the test pieces + Think about resumability design and doc here: + Per SyncSegment? That abstraction bubbles out? + What do we hand out? What kind of class can we wrap that in? + diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java index 50d80862f65d..68d2fb369a47 100755 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java @@ -242,7 +242,7 @@ void switchToNewSegment(CommitLogSegment old) * Spins while waiting on next available segment's allocation, putting caller to sleep until the new segment is created. * @param oldActiveSegment */ - void awaitSegmentAllocation(CommitLogSegment oldActiveSegment) + private void awaitSegmentAllocation(CommitLogSegment oldActiveSegment) { do { @@ -401,7 +401,7 @@ else if (!flushes.containsKey(dirtyTableId)) * Stops CL, for testing purposes. DO NOT USE THIS OUTSIDE OF TESTS. * Only call this after the AbstractCommitLogService is shut down. */ - public void stopUnsafe(boolean deleteSegments) + void stopUnsafe(boolean deleteSegments) { logger.debug("CLSM closing and clearing existing commit log segments..."); diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java index e23a915ba355..1dfaaa6ef343 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java @@ -186,17 +186,17 @@ public static class SyncSegment public final FileDataInput input; /** offset in file where this section begins. */ - public final int fileStartPosition; + final int fileStartPosition; /** offset in file where this section ends. */ - public final int fileEndPosition; + final int fileEndPosition; /** the logical ending position of the buffer */ - public final int endPosition; + final int endPosition; - public final boolean toleratesErrorsInSection; + final boolean toleratesErrorsInSection; - public SyncSegment(FileDataInput input, int fileStartPosition, int fileEndPosition, int endPosition, boolean toleratesErrorsInSection) + SyncSegment(FileDataInput input, int fileStartPosition, int fileEndPosition, int endPosition, boolean toleratesErrorsInSection) { this.input = input; this.fileStartPosition = fileStartPosition; @@ -234,7 +234,7 @@ static class NoOpSegmenter implements Segmenter { private final RandomAccessReader reader; - public NoOpSegmenter(RandomAccessReader reader) + NoOpSegmenter(RandomAccessReader reader) { this.reader = reader; } @@ -259,12 +259,12 @@ static class CompressedSegmenter implements Segmenter private byte[] uncompressedBuffer; private long nextLogicalStart; - public CompressedSegmenter(CommitLogDescriptor desc, RandomAccessReader reader) + CompressedSegmenter(CommitLogDescriptor desc, RandomAccessReader reader) { this(CompressionParams.createCompressor(desc.compression), reader); } - public CompressedSegmenter(ICompressor compressor, RandomAccessReader reader) + CompressedSegmenter(ICompressor compressor, RandomAccessReader reader) { this.compressor = compressor; this.reader = reader; @@ -315,7 +315,7 @@ static class EncryptedSegmenter implements Segmenter private long currentSegmentEndPosition; private long nextLogicalStart; - public EncryptedSegmenter(CommitLogDescriptor descriptor, RandomAccessReader reader) + EncryptedSegmenter(CommitLogDescriptor descriptor, RandomAccessReader reader) { this(reader, descriptor.getEncryptionContext()); } diff --git a/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java index 23a8c25db3db..a5fce7b618df 100644 --- a/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java @@ -18,6 +18,7 @@ package org.apache.cassandra.db.commitlog; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.util.RandomAccessReader; /** * A CommitLogReader that allows both a) indicating how much of the CommitLog the user would like to read, and b) the ability @@ -32,11 +33,16 @@ */ public class ResumableCommitLogReader { - public ResumableCommitLogReader() + public final CommitLogDescriptor descriptor; + public final RandomAccessReader reader; + + public ResumableCommitLogReader(CommitLogDescriptor descriptor, RandomAccessReader reader) { if (!DatabaseDescriptor.isCDCEnabled()) { throw new RuntimeException("Cannot use a ResumableCommitLogReader if CDC is not enabled."); } + this.descriptor = descriptor; + this.reader = reader; } } diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 85f8bdfc6f3b..9f383df14c67 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -1697,6 +1697,42 @@ protected com.datastax.driver.core.TupleType tupleTypeOf(ProtocolVersion protoco return clusters.get(protocolVersion).getMetadata().newTupleType(types); } + /** + * Creates a default reference table with some pre-populated data, generating enough data to create a few commit log + * files. + */ + protected void populateReferenceData(boolean withCDC) throws Throwable + { + Random random = new Random(); + + String createString = "CREATE TABLE %s (a int, b int, c double, d decimal, e smallint, f tinyint, g blob, primary key (a, b))"; + if (withCDC) + createString += " WITH cdc=true"; + + // For each test, we start with the assumption of a populated set of a few files we can pull from. + createTable(createString); + + byte[] bBlob = new byte[1024 * 1024]; + CommitLog.instance.sync(true); + + // Populate some CommitLog segments on disk + for (int i = 0; i < 20; i++) + { + random.nextBytes(bBlob); + + logger.debug(String.format("Executing insert for index: [%d]", i)); + execute("INSERT INTO %s (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?)", + random.nextInt(), + random.nextInt(), + random.nextDouble(), + random.nextLong(), + (short)random.nextInt(), + (byte)random.nextInt(), + ByteBuffer.wrap(bBlob)); + } + CommitLog.instance.sync(true); + } + // Attempt to find an AbstracType from a value (for serialization/printing sake). // Will work as long as we use types we know of, which is good enough for testing private static AbstractType typeFor(Object value) diff --git a/test/unit/org/apache/cassandra/db/commitlog/CDCTestReplayer.java b/test/unit/org/apache/cassandra/db/commitlog/CDCTestReplayer.java deleted file mode 100644 index 18bc6e0a957f..000000000000 --- a/test/unit/org/apache/cassandra/db/commitlog/CDCTestReplayer.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.db.commitlog; - -import java.io.File; -import java.io.IOException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.db.rows.SerializationHelper; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.RebufferingInputStream; - -/** - * Utility class that flags the replayer as having seen a CDC mutation and calculates offset but doesn't apply mutations - */ -public class CDCTestReplayer extends CommitLogReplayer -{ - private static final Logger logger = LoggerFactory.getLogger(CDCTestReplayer.class); - - public CDCTestReplayer() throws IOException - { - super(CommitLog.instance, CommitLogPosition.NONE, null, ReplayFilter.create()); - CommitLog.instance.sync(true); - commitLogReader = new CommitLogTestReader(); - } - - public void examineCommitLog() throws IOException - { - replayFiles(new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()); - } - - private class CommitLogTestReader extends CommitLogReader - { - @Override - protected void readMutation(CommitLogReadHandler handler, - byte[] inputBuffer, - int size, - CommitLogPosition minPosition, - final int entryLocation, - final CommitLogDescriptor desc) throws IOException - { - RebufferingInputStream bufIn = new DataInputBuffer(inputBuffer, 0, size); - Mutation mutation; - try - { - mutation = Mutation.serializer.deserialize(bufIn, desc.getMessagingVersion(), SerializationHelper.Flag.LOCAL); - if (mutation.trackedByCDC()) - sawCDCMutation = true; - } - catch (IOException e) - { - // Test fails. - throw new AssertionError(e); - } - } - } -} diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java index 9f0949d8eec5..253ea43d5874 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java @@ -18,10 +18,16 @@ package org.apache.cassandra.db.commitlog; -import java.io.*; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.InputStreamReader; import java.nio.file.Files; import java.nio.file.Path; -import java.util.*; +import java.util.ArrayList; +import java.util.List; import org.junit.Assert; import org.junit.Before; @@ -322,8 +328,8 @@ public void testReplayLogic() throws IOException CommitLog.instance.start(); CommitLog.instance.segmentManager.awaitManagementTasksCompletion(); } - CDCTestReplayer replayer = new CDCTestReplayer(); - replayer.examineCommitLog(); + CommitLogTestUtils.CDCMutationCountingReplayer replayer = new CommitLogTestUtils.CDCMutationCountingReplayer(); + replayer.replayExistingCommitLog(); // Rough sanity check -> should be files there now. Assert.assertTrue("Expected non-zero number of files in CDC folder after restart.", @@ -419,6 +425,4 @@ public boolean equals(Object other) return fileName.equals(cid.fileName) && offset == cid.offset; } } - - } diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java index 2a05d41db529..62de4561f6f0 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java @@ -19,16 +19,28 @@ package org.apache.cassandra.db.commitlog; import java.io.File; +import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.junit.Assert; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.rows.SerializationHelper; +import org.apache.cassandra.io.util.DataInputBuffer; +import org.apache.cassandra.io.util.RebufferingInputStream; +/** + * Collection of some helper methods and classes for use in our various CommitLog Unit Tests + */ class CommitLogTestUtils { private static final Logger logger = LoggerFactory.getLogger(CommitLogTestUtils.class); @@ -62,7 +74,11 @@ static long updateCDCTotalSize(CommitLogSegmentManager segmentManager) return ((CommitLogSegmentAllocatorCDC)segmentManager.segmentAllocator).updateCDCTotalSize(); } - static File getFilledCommitLogFile() + /** + * Pulls the back of the commit log files list and bails out if that is == our current allocating. Goal is to get a filled + * and usable commit log segment for testing work. + */ + static File getFilledCommitLogFile() throws NullPointerException { File result = new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()[0]; Assert.assertNotEquals(result.toString(), CommitLog.instance.segmentManager.getActiveSegment().logFile); @@ -85,6 +101,9 @@ static void corruptFileAtOffset(File f, int offset) { } + /** + * Debug method to allow printing of a message when sanity checking commit log state while writing tests + */ static void listCommitLogFiles(String message) { StringBuilder result = new StringBuilder(); @@ -108,4 +127,128 @@ static void debugLog(String input) "***************************************************\n\n" + System.lineSeparator()); } + + /** + * Utility class that flags the replayer as having seen a CDC mutation and calculates offset but doesn't apply mutations + */ + static class CDCMutationCountingReplayer extends CommitLogReplayer + { + final Set seenMutations = Sets.newConcurrentHashSet(); + final ConcurrentHashMap duplicateMutations = new ConcurrentHashMap<>(); + + CDCMutationCountingReplayer() throws IOException + { + super(CommitLog.instance, CommitLogPosition.NONE, null, ReplayFilter.create()); + CommitLog.instance.sync(true); + commitLogReader = new CDCCountingReader(); + } + + /** + * Takes existing files in the commit log location and forces a replay on them. Only really meaningful if you're + * intercepting either the mutation read handler on replay or the mutation initiation object. + */ + void replayExistingCommitLog() throws IOException + { + replayFiles(new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()); + } + + void replaySingleFile(File f) throws IOException + { + replayFiles(new File[] { f }); + } + + boolean hasSeenMutation(MutationIdentifier id) + { + return seenMutations.contains(id); + } + + int duplicateMutationCount(MutationIdentifier id) + { + Integer result = duplicateMutations.get(id); + return result == null ? 0 : result; + } + + boolean hasSeenDuplicateMutations() + { + return duplicateMutations.size() > 0; + } + + private class CDCCountingReader extends CommitLogReader + { + @Override + protected void readMutation(CommitLogReadHandler handler, + byte[] inputBuffer, + int size, + CommitLogPosition minPosition, + final int entryLocation, + final CommitLogDescriptor desc) + { + MutationIdentifier id = new MutationIdentifier(minPosition.segmentId, size, entryLocation); + + RebufferingInputStream bufIn = new DataInputBuffer(inputBuffer, 0, size); + Mutation mutation; + try + { + mutation = Mutation.serializer.deserialize(bufIn, desc.getMessagingVersion(), SerializationHelper.Flag.LOCAL); + + if (mutation.trackedByCDC()) + { + sawCDCMutation = true; + if (seenMutations.contains(id)) + { + Integer pv = duplicateMutations.get(id); + if (pv == null) + pv = 0; + duplicateMutations.put(id, pv + 1); + } + seenMutations.add(id); + } + } + catch (IOException e) + { + // Test fails. + throw new AssertionError(e); + } + } + } + + /** + * Helper class that allows us to uniquely identify a mutation at least within a single instance of a running node. + */ + private static class MutationIdentifier + { + final long segmentId; + final int size; + final int location; + + MutationIdentifier(long segmentId, int size, int location) + { + this.segmentId = segmentId; + this.size = size; + this.location = location; + } + + @Override + public boolean equals(Object o) + { + if (o == this) + return true; + + if (!(o instanceof MutationIdentifier)) + return false; + + MutationIdentifier other = (MutationIdentifier) o; + + return other.size == this.size && + other.location == this.location && + other.segmentId == this.segmentId; + } + + @Override + public int hashCode() + { + return Objects.hash(size, location, segmentId); + } + } + } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java index 76094e6df09d..1c8484c1eb0e 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java @@ -19,8 +19,7 @@ package org.apache.cassandra.db.commitlog; import java.io.File; -import java.nio.ByteBuffer; -import java.util.Random; +import java.io.IOException; import org.junit.Assert; import org.junit.Before; @@ -32,10 +31,14 @@ /* Forces CDC enabled on the run of these tests as they aren't used in non-CDC context and ResumableCommitLogReader * class asserts cdc enabled on creation. + * + * Note: Many of these tests depend on having a CommitLog segment they're working on, and we don't force a stop of the + * commit log segment allocator or flushing mechanisms while running these tests. As such, it's _possible_ we could end + * up with files getting yanked out from under us and the test failing, but with 4+ segments created by the default ref + * data population, the risk of that should be very low. Keep it in mind as time goes by. */ public class ResumableCommitLogReaderTest extends CQLTester { - private Random random = new Random(); @BeforeClass public static void setUpClass() @@ -49,32 +52,11 @@ public void beforeTest() throws Throwable { super.beforeTest(); - // Need to clean out any files from previous test runs. Prevents flaky test failures. - CommitLog.instance.stopUnsafe(true); - CommitLog.instance.start(); - - // For each test, we start with the assumption of a populated set of a few files we can pull from. - createTable("CREATE TABLE %s (a int, b int, c double, d decimal, e smallint, f tinyint, g blob, primary key (a, b))"); - - byte[] bBlob = new byte[1024 * 1024]; - CommitLog.instance.sync(true); - - // Populate some CommitLog segments on disk - for (int i = 0; i < 20; i++) - { - random.nextBytes(bBlob); - - logger.debug(String.format("Executing insert for index: [%d]", i)); - execute("INSERT INTO %s (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?)", - random.nextInt(), - random.nextInt(), - random.nextDouble(), - random.nextLong(), - (short)random.nextInt(), - (byte)random.nextInt(), - ByteBuffer.wrap(bBlob)); - } - CommitLog.instance.sync(true); + // Start with a clean slate each test. Arguably could pre-populate and just use populated data; keep an eye + // on runtime for this test suite and group if that becomes a worthy time and complexity tradeoff. + CommitLog.instance.resetUnsafe(true); + + populateReferenceData(true); // Should have well more than 3 segments to work with on subsequent tests. Assert.assertTrue(CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size() > 3); @@ -84,12 +66,37 @@ public void beforeTest() throws Throwable * Expect operation as though non-resumable, read file to end and complete. */ @Test - public void testNonResumedGeneralCase() + public void testNonResumedGeneralCase() throws IOException + { + File writtenFile = CommitLogTestUtils.getFilledCommitLogFile(); + Assert.assertTrue(writtenFile.length() > 0); + + CommitLogTestUtils.CDCMutationCountingReplayer testReplayer = new CommitLogTestUtils.CDCMutationCountingReplayer(); + testReplayer.replaySingleFile(writtenFile); + + Assert.assertTrue("Did not see any CDC enabled mutations.", testReplayer.sawCDCMutation); + Assert.assertFalse("Saw a duplicate mutation while replaying a single file. This... shouldn't happen.", + testReplayer.hasSeenDuplicateMutations()); + } + + /** + * Confirm our duplicate mutation testing infrastructure is working. + */ + @Test + public void testDuplicateCheckLogic() throws IOException { File writtenFile = CommitLogTestUtils.getFilledCommitLogFile(); - CommitLogTestUtils.debugLog(String.format("Got written CL segment: %s", writtenFile.toString())); - CommitLogTestUtils.debugLog(String.format("Active segment writing to is: %s", CommitLog.instance.segmentManager.getActiveSegment().logFile.toString())); - CommitLogTestUtils.listCommitLogFiles("Checking total CL files:"); + Assert.assertTrue(writtenFile.length() > 0); + + CommitLogTestUtils.CDCMutationCountingReplayer testReplayer = new CommitLogTestUtils.CDCMutationCountingReplayer(); + testReplayer.replaySingleFile(writtenFile); + + Assert.assertTrue("Did not see any CDC enabled mutations.", testReplayer.sawCDCMutation); + Assert.assertFalse("Saw a duplicate mutation while replaying a single file. This... shouldn't happen.", + testReplayer.hasSeenDuplicateMutations()); + + testReplayer.replaySingleFile(writtenFile); + Assert.assertTrue("Expected to see duplicate mutations on 2nd replay of file.", testReplayer.hasSeenDuplicateMutations()); } /** @@ -158,4 +165,6 @@ public void testMissingInputFile() public void testWrongFile() { } + + } From f253f1cb4038a508664ff4a8b7741dd348c7f943 Mon Sep 17 00:00:00 2001 From: Josh McKenzie Date: Sun, 13 Oct 2019 14:00:00 -0400 Subject: [PATCH 77/78] Clean up javadoc and scoping in CommitLogSegment and CDC Allocator Had a handful of variables // when /** */ would give us IDE hinting. Also added a class comment to header of CDC Allocator to give high level view of why it exists. --- .../db/commitlog/CommitLogDescriptor.java | 2 +- .../db/commitlog/CommitLogReader.java | 12 +++++--- .../db/commitlog/CommitLogSegment.java | 28 ++++++++++--------- .../CommitLogSegmentAllocatorCDC.java | 12 ++++++++ .../db/commitlog/CommitLogSegmentReader.java | 11 ++++++-- 5 files changed, 45 insertions(+), 20 deletions(-) diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java index 700f12a242ca..2d04126ff3d5 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java @@ -62,7 +62,7 @@ public class CommitLogDescriptor public static final int VERSION_40 = 7; /** - * Increment this number if there is a changes in the commit log disc layout or MessagingVersion changes. + * Increment this number if there is a changes in the commit log disk layout or MessagingVersion changes. * Note: make sure to handle {@link #getMessagingVersion()} */ @VisibleForTesting diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java index dafe36fbf16d..684c3a7ac4fe 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java @@ -137,6 +137,10 @@ private static boolean shouldSkip(File file) throws IOException, ConfigurationEx } } + /** + * Filters list of passed in CommitLogSegments based on shouldSkip logic, specifically whether files are empty and + * pass crc. + */ static List filterCommitLogFiles(File[] toFilter) { List filtered = new ArrayList<>(toFilter.length); @@ -175,10 +179,10 @@ static List filterCommitLogFiles(File[] toFilter) * @throws IOException */ void readCommitLogSegment(CommitLogReadHandler handler, - File file, - CommitLogPosition minPosition, - int mutationLimit, - boolean tolerateTruncation) throws IOException + File file, + CommitLogPosition minPosition, + int mutationLimit, + boolean tolerateTruncation) throws IOException { // just transform from the file name (no reading of headers) to determine version CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName()); diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java index aa771a1fcfc3..dcbe7af348c7 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java @@ -49,7 +49,7 @@ import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt; -/* +/** * A single commit log file on disk. Manages creation of the file and writing mutations to disk, * as well as tracking the last mutation position of any "dirty" CFs covered by the segment file. Segment * files are initially allocated to a fixed size and can grow to accomidate a larger value if necessary. @@ -80,20 +80,21 @@ public enum CDCState replayLimitId = idBase = Math.max(System.currentTimeMillis(), maxId + 1); } - // The commit log entry overhead in bytes (int: length + int: head checksum + int: tail checksum) + /** The commit log entry overhead in bytes (int: length + int: head checksum + int: tail checksum) */ static final int ENTRY_OVERHEAD_SIZE = 4 + 4 + 4; - // The commit log (chained) sync marker/header size in bytes (int: length + int: checksum [segmentId, position]) + /** The commit log (chained) sync marker/header size in bytes (int: length + int: checksum [segmentId, position]) */ static final int SYNC_MARKER_SIZE = 4 + 4; - // The OpOrder used to order appends wrt sync + /** The OpOrder used to order appends wrt sync */ private final OpOrder appendOrder = new OpOrder(); private final AtomicInteger allocatePosition = new AtomicInteger(); - // Everything before this offset has been synced and written. The SYNC_MARKER_SIZE bytes after - // each sync are reserved, and point forwards to the next such offset. The final - // sync marker in a segment will be zeroed out, or point to a position too close to the EOF to fit a marker. + /** Everything before this offset has been synced and written. The SYNC_MARKER_SIZE bytes after + * each sync are reserved, and point forwards to the next such offset. The final + * sync marker in a segment will be zeroed out, or point to a position too close to the EOF to fit a marker. + */ @VisibleForTesting volatile int lastSyncedOffset; @@ -103,18 +104,19 @@ public enum CDCState */ private volatile int lastMarkerOffset; - // The end position of the buffer. Initially set to its capacity and updated to point to the last written position - // as the segment is being closed. - // No need to be volatile as writes are protected by appendOrder barrier. + /** The end position of the buffer. Initially set to its capacity and updated to point to the last written position + * as the segment is being closed. + * No need to be volatile as writes are protected by appendOrder barrier. + */ private int endOfBuffer; - // a signal for writers to wait on to confirm the log message they provided has been written to disk + /** a signal for writers to wait on to confirm the log message they provided has been written to disk */ private final WaitQueue syncComplete = new WaitQueue(); - // a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data + /** a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data */ private final NonBlockingHashMap tableDirty = new NonBlockingHashMap<>(1024); - // a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use + /** a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use */ private final ConcurrentHashMap tableClean = new ConcurrentHashMap<>(); public final long id; diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java index 4f696c5d4175..0a7fbd21cc40 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java @@ -39,6 +39,18 @@ import org.apache.cassandra.utils.DirectorySizeCalculator; import org.apache.cassandra.utils.NoSpamLogger; +/** + * A CommitLogSegmentAllocator that respects the configured total allowable CDC space on disk. On allocation of a mutation + * checks if it's on a table tracked by CDC and, if so, either throws an exception if at CDC limit or flags that segment + * as containing a CDC mutation if it's a new one. + * + * This code path is only exercised if cdc is enabled on a node. We pay the duplication cost of having both CDC and non + * allocators in order to keep the old allocator code clean and separate from this allocator, as well as to not introduce + * unnecessary operations on the critical path for nodes / users where they have no interest in CDC. May be worth considering + * unifying in the future should the perf implications of this be shown to be negligible, though the hard linking and + * size tracking is somewhat distasteful to have floating around on nodes where cdc is not in use (which we assume to be + * the majority). + */ public class CommitLogSegmentAllocatorCDC implements CommitLogSegmentAllocator { static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentAllocatorCDC.class); diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java index 1dfaaa6ef343..2840d82d92ee 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java @@ -53,11 +53,15 @@ public class CommitLogSegmentReader implements Iterable Date: Sun, 13 Oct 2019 14:00:00 -0400 Subject: [PATCH 78/78] Implement a ResumableCommitLogReader Adds a new Single Source of Reading Truth for CommitLogSegments. This resumable reader has a simpler API than our existing CommitLogReader and is primarily intended to be externally user-facing. Primary tradeoffs of this implementation are 1) Requiring init of a new ResumableCommitLogReader object on each CLS read, 2) taking on ownership of some complexity in SegmentReader (iterator resumability and RAR reinit) in order to provide a very simple API to external CDC users, and 3) splitting out the "Reading a CommitLog" API into both part in CommitLogReader and part in ResumableCommitLogReader. For now, for API backwards compatibility, the CommitLogReader APIs are unchanged and ResumableCommitLogReader's API is purely simplified and additive. --- resumable_todo.txt | 10 - .../db/commitlog/CommitLogReader.java | 238 +++++----- .../db/commitlog/CommitLogSegment.java | 9 +- .../CommitLogSegmentAllocatorCDC.java | 10 +- .../db/commitlog/CommitLogSegmentManager.java | 6 +- .../db/commitlog/CommitLogSegmentReader.java | 285 ++++++++---- .../db/commitlog/MemoryMappedSegment.java | 5 + .../commitlog/ResumableCommitLogReader.java | 258 ++++++++++- .../org/apache/cassandra/cql3/CQLTester.java | 39 +- .../CommitLogSegmentAllocatorCDCTest.java | 2 +- .../db/commitlog/CommitLogTestUtils.java | 164 +++---- .../ResumableCommitLogReaderTest.java | 407 +++++++++++++++--- .../db/commitlog/SegmentReaderTest.java | 10 +- 13 files changed, 1053 insertions(+), 390 deletions(-) delete mode 100644 resumable_todo.txt diff --git a/resumable_todo.txt b/resumable_todo.txt deleted file mode 100644 index 6aa782aa0214..000000000000 --- a/resumable_todo.txt +++ /dev/null @@ -1,10 +0,0 @@ -ToDo: - +Get the unit test prep up and working - + Finish stubbing in CDCMockInitiator - + processMutation - + Get back to initial test - + testNonResumedGeneralCase -> test out the infra / piping of the logic on the test pieces - Think about resumability design and doc here: - Per SyncSegment? That abstraction bubbles out? - What do we hand out? What kind of class can we wrap that in? - diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java index 684c3a7ac4fe..7fb63bb27724 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java @@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.zip.CRC32; +import javax.annotation.Nonnull; + import com.google.common.annotations.VisibleForTesting; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -43,13 +45,14 @@ import org.apache.cassandra.io.util.RebufferingInputStream; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.utils.JVMStabilityInspector; +import org.apache.cassandra.utils.Pair; import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt; /** - * A CommitLogReader is used to traverse a flat CommitLog file and provide deserialized Mutation objects to an accepting - * CommitLogReadHandler. None of the actual state about traversal or progress is actually stored inside this class; it's - * largely a stateless container w/logic to allow reading files which are assumed to be CommitLog Segments. + * The CommitLogReader presents an idempotentinterface for legacy CommitLogSegment reads. The logic to read and + * parse a CommitLogSegment is housed here, however this depends upon {@link ResumableCommitLogReader} for any non-trivial + * or resumable coordination of reads. */ public class CommitLogReader { @@ -57,14 +60,18 @@ public class CommitLogReader private static final int LEGACY_END_OF_SEGMENT_MARKER = 0; + /** Used to indicate we want to read to the end of a commit log segment during logic for resumable reading */ + static final int READ_TO_END_OF_FILE = Integer.MAX_VALUE; + @VisibleForTesting static final int ALL_MUTATIONS = -1; private final CRC32 checksum; private final Map invalidMutations; + @Nonnull private byte[] buffer; - public CommitLogReader() + CommitLogReader() { checksum = new CRC32(); invalidMutations = new HashMap<>(); @@ -76,17 +83,13 @@ Set> getInvalidMutations() return invalidMutations.entrySet(); } - /** - * Reads all passed in files with no minimum, no start, and no mutation limit. - */ + /** Reads all passed in files with no minimum, no start, and no mutation limit. */ public void readAllFiles(CommitLogReadHandler handler, File[] files) throws IOException { readAllFiles(handler, files, CommitLogPosition.NONE); } - /** - * Reads all passed in files with minPosition, no start, and no mutation limit. - */ + /** Reads all passed in files with minPosition, no start, and no mutation limit. */ public void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPosition minPosition) throws IOException { List filteredLogs = filterCommitLogFiles(files); @@ -98,34 +101,40 @@ public void readAllFiles(CommitLogReadHandler handler, File[] files, CommitLogPo } } - /** - * Reads passed in file fully - */ - public void readCommitLogSegment(CommitLogReadHandler handler, File file, boolean tolerateTruncation) throws IOException + /** Read a CommitLogSegment fully, no restrictions */ + void readCommitLogSegment(CommitLogReadHandler handler, File file, boolean tolerateTruncation) throws IOException { readCommitLogSegment(handler, file, CommitLogPosition.NONE, ALL_MUTATIONS, tolerateTruncation); } - /** - * Reads all mutations from passed in file from minPosition - */ - public void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException + /** Read passed in file fully, up to mutationLimit count */ + @VisibleForTesting + void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException + { + readCommitLogSegment(handler, file, CommitLogPosition.NONE, mutationLimit, tolerateTruncation); + } + + /** Read all mutations from passed in file from minPosition in the logical CommitLog */ + void readCommitLogSegment(CommitLogReadHandler handler, File file, CommitLogPosition minPosition, boolean tolerateTruncation) throws IOException { readCommitLogSegment(handler, file, minPosition, ALL_MUTATIONS, tolerateTruncation); } - /** - * Reads passed in file fully, up to mutationLimit count - */ - @VisibleForTesting - public void readCommitLogSegment(CommitLogReadHandler handler, File file, int mutationLimit, boolean tolerateTruncation) throws IOException + void readCommitLogSegment(CommitLogReadHandler handler, + File file, + CommitLogPosition minPosition, + int mutationLimit, + boolean tolerateTruncation) throws IOException { - readCommitLogSegment(handler, file, CommitLogPosition.NONE, mutationLimit, tolerateTruncation); + // TODO: Consider removing the need to create a resumable reader here and instead simply build and use the needed components. + // This would require a change to internalRead since that assumes a ResumableReader for convenience. + try(ResumableCommitLogReader resumableReader = new ResumableCommitLogReader(file, handler, minPosition, mutationLimit, tolerateTruncation)) + { + resumableReader.readToCompletion(); + } } - /** - * Confirms whether the passed in file is one we should read or skip based on whether it's empty and passes crc - */ + /** Confirms whether the passed in file is one we should read or skip based on whether it's empty and passes crc */ private static boolean shouldSkip(File file) throws IOException, ConfigurationException { try(RandomAccessReader reader = RandomAccessReader.open(file)) @@ -137,10 +146,7 @@ private static boolean shouldSkip(File file) throws IOException, ConfigurationEx } } - /** - * Filters list of passed in CommitLogSegments based on shouldSkip logic, specifically whether files are empty and - * pass crc. - */ + /** Filters list of passed in CommitLogSegments based on shouldSkip logic, specifically whether files are empty and pass crc. */ static List filterCommitLogFiles(File[] toFilter) { List filtered = new ArrayList<>(toFilter.length); @@ -167,126 +173,127 @@ static List filterCommitLogFiles(File[] toFilter) return filtered; } - /** - * Reads mutations from file, handing them off to handler - * @param handler Handler that will take action based on deserialized Mutations - * @param file CommitLogSegment file to read - * @param minPosition Optional minimum CommitLogPosition - all segments with id larger or matching w/greater position will be read - * @param mutationLimit Optional limit on # of mutations to replay. Local ALL_MUTATIONS serves as marker to play all. - * @param tolerateTruncation Whether or not we should allow truncation of this file or throw if EOF found + * Reads and constructs the {@link CommitLogDescriptor} portion of a File. * - * @throws IOException + * @return Pair, Integer> An optional descriptor and serialized header size */ - void readCommitLogSegment(CommitLogReadHandler handler, - File file, - CommitLogPosition minPosition, - int mutationLimit, - boolean tolerateTruncation) throws IOException + static Pair, Integer> readCommitLogDescriptor(CommitLogReadHandler handler, + File file, + boolean tolerateTruncation) throws IOException { // just transform from the file name (no reading of headers) to determine version CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName()); + long segmentIdFromFilename = desc.id; + int descriptorSize = -1; - try(RandomAccessReader reader = RandomAccessReader.open(file)) + try(RandomAccessReader rawSegmentReader = RandomAccessReader.open(file)) { - final long segmentIdFromFilename = desc.id; try { // The following call can either throw or legitimately return null. For either case, we need to check // desc outside this block and set it to null in the exception case. - desc = CommitLogDescriptor.readHeader(reader, DatabaseDescriptor.getEncryptionContext()); + desc = CommitLogDescriptor.readHeader(rawSegmentReader, DatabaseDescriptor.getEncryptionContext()); } catch (Exception e) { desc = null; } + if (desc == null) { - // don't care about whether or not the handler thinks we can continue. We can't w/out descriptor. - // whether or not we can continue depends on whether this is the last segment - handler.handleUnrecoverableError(new CommitLogReadException( + // Don't care about whether or not the handler thinks we can continue. We can't w/out descriptor. + // Whether or not we can continue depends on whether this is the last segment + handler.handleUnrecoverableError(new CommitLogReadHandler.CommitLogReadException( String.format("Could not read commit log descriptor in file %s", file), - CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR, + CommitLogReadHandler.CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR, tolerateTruncation)); - return; + return Pair.create(Optional.empty(), -1); } + // Continuing if our file name and descriptor mismatch is optional. if (segmentIdFromFilename != desc.id) { - if (handler.shouldSkipSegmentOnError(new CommitLogReadException(String.format( - "Segment id mismatch (filename %d, descriptor %d) in file %s", segmentIdFromFilename, desc.id, file), - CommitLogReadErrorReason.RECOVERABLE_DESCRIPTOR_ERROR, - false))) + if (handler.shouldSkipSegmentOnError(new CommitLogReadHandler.CommitLogReadException( + String.format("Segment id mismatch (filename %d, descriptor %d) in file %s", segmentIdFromFilename, desc.id, file), + CommitLogReadHandler.CommitLogReadErrorReason.RECOVERABLE_DESCRIPTOR_ERROR, + false))) { - return; + return Pair.create(Optional.empty(), -1); } } - if (shouldSkipSegmentId(file, desc, minPosition)) - return; - - CommitLogSegmentReader segmentReader; - try - { - segmentReader = new CommitLogSegmentReader(handler, desc, reader, tolerateTruncation); - } - catch(Exception e) - { - handler.handleUnrecoverableError(new CommitLogReadException( - String.format("Unable to create segment reader for commit log file: %s", e), - CommitLogReadErrorReason.UNRECOVERABLE_UNKNOWN_ERROR, - tolerateTruncation)); - return; - } - - try - { - ReadStatusTracker statusTracker = new ReadStatusTracker(mutationLimit, tolerateTruncation); - for (CommitLogSegmentReader.SyncSegment syncSegment : segmentReader) - { - // Only tolerate truncation if we allow in both global and segment - statusTracker.tolerateErrorsInSection = tolerateTruncation & syncSegment.toleratesErrorsInSection; - - // Skip segments that are completely behind the desired minPosition - if (desc.id == minPosition.segmentId && syncSegment.endPosition < minPosition.position) - continue; - - statusTracker.errorContext = String.format("Next section at %d in %s", syncSegment.fileStartPosition, desc.fileName()); + descriptorSize = (int)rawSegmentReader.getPosition(); + } + return Pair.create(Optional.of(desc), descriptorSize); + } - readSection(handler, syncSegment.input, minPosition, syncSegment.endPosition, statusTracker, desc); - if (!statusTracker.shouldContinue()) - break; - } - } - // Unfortunately AbstractIterator cannot throw a checked exception, so we check to see if a RuntimeException - // is wrapping an IOException. - catch (RuntimeException re) - { - if (re.getCause() instanceof IOException) - throw (IOException) re.getCause(); - throw re; - } - logger.debug("Finished reading {}", file); + /** + * Opens a RandomAccessReader to a CommitLogSegment _and does not close it_. Closed out in {@link ResumableCommitLogReader#close} + */ + static CommitLogSegmentReader getCommitLogSegmentReader(ResumableCommitLogReader parent) throws IOException + { + CommitLogSegmentReader result; + try + { + result = new CommitLogSegmentReader(parent); } + catch(Exception e) + { + parent.readHandler.handleUnrecoverableError(new CommitLogReadHandler.CommitLogReadException( + String.format("Unable to create segment reader for commit log file: %s", e), + CommitLogReadHandler.CommitLogReadErrorReason.UNRECOVERABLE_UNKNOWN_ERROR, + parent.tolerateTruncation)); + // Regardless of whether this is in the node context and we allow the node to continue to run, this reader is + // dead. + parent.close(); + return null; + } + return result; } /** - * Any segment with id >= minPosition.segmentId is a candidate for read. + * Iterates over {@link CommitLogSegmentReader.SyncSegment} until it hits offset limit or end of iterator, based + * on the resumable reader's sentinel. */ - private boolean shouldSkipSegmentId(File file, CommitLogDescriptor desc, CommitLogPosition minPosition) + void internalReadCommitLogSegment(ResumableCommitLogReader rr) throws IOException { - logger.debug("Reading {} (CL version {}, messaging version {}, compression {})", - file.getPath(), - desc.version, - desc.getMessagingVersion(), - desc.compression); + try + { + ReadStatusTracker statusTracker = new ReadStatusTracker(rr.mutationLimit, rr.tolerateTruncation); - if (minPosition.segmentId > desc.id) + int lastSegmentEnd = -1; + + while (lastSegmentEnd < rr.offsetLimit && rr.activeIterator.hasNext()) + { + CommitLogSegmentReader.SyncSegment syncSegment = rr.activeIterator.next(); + // Back out if we're at the end of our current partially written CL segment. + if (syncSegment == CommitLogSegmentReader.RESUMABLE_SENTINEL) + break; + + lastSegmentEnd = syncSegment.endPosition; + + statusTracker.tolerateErrorsInSection = rr.tolerateTruncation & syncSegment.toleratesErrorsInSection; + + // Skip segments that are completely behind the desired minPosition + if (rr.descriptor.id == rr.minPosition.segmentId && syncSegment.endPosition < rr.minPosition.position) + continue; + + statusTracker.errorContext = String.format("Next section at %d in %s", syncSegment.fileStartPosition, rr.descriptor.fileName()); + + readSection(rr.readHandler, syncSegment.input, rr.minPosition, syncSegment.endPosition, statusTracker, rr.descriptor); + if (!statusTracker.shouldContinue()) + break; + } + } + // Unfortunately AbstractIterator cannot throw a checked exception, so we check to see if a RuntimeException + // is wrapping an IOException. + catch (RuntimeException | IOException re) { - logger.trace("Skipping read of fully-flushed {}", file); - return true; + if (re.getCause() instanceof IOException) + throw (IOException) re.getCause(); + throw re; } - return false; } /** @@ -374,9 +381,6 @@ private void readSection(CommitLogReadHandler handler, return; } - // Overallocate a touch relative to the required size for this mutation, presumably to allow a little - // breathing room and not thrash on a need to re-allocate repeatedly for other mutations of slightly larger size. - // TODO: Why 1.2x? Why not double? Why not 1.1? if (serializedSize > buffer.length) buffer = new byte[(int) (1.2 * serializedSize)]; reader.readFully(buffer, 0, serializedSize); @@ -419,7 +423,7 @@ private void readSection(CommitLogReadHandler handler, } /** - * Deserializes and passes a Mutation to the ICommitLogReadHandler requested + * Deserializes and passes a Mutation to the CommitLogReadHandler requested * * @param handler Handler that will take action based on deserialized Mutations * @param inputBuffer raw byte array w/Mutation data @@ -495,8 +499,6 @@ protected void readMutation(CommitLogReadHandler handler, /** * Helper methods to deal with changing formats of internals of the CommitLog without polluting deserialization code. - * Note: while this has a currently unused variable (commitLogVersion) in the 4.0 line, the idea is to future proof - * and allow support for multiple disparate CommitLogFormats at a single time. */ private static class CommitLogFormat { diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java index dcbe7af348c7..793605b860a5 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java @@ -32,6 +32,9 @@ import com.google.common.annotations.VisibleForTesting; import org.cliffc.high_scale_lib.NonBlockingHashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.codahale.metrics.Timer; import org.apache.cassandra.config.*; import org.apache.cassandra.db.Mutation; @@ -56,6 +59,8 @@ */ public abstract class CommitLogSegment { + static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManager.class); + private final static long idBase; private CDCState cdcState = CDCState.PERMITTED; @@ -121,6 +126,7 @@ public enum CDCState public final long id; + /** The CommitLogSegment log file on disk */ final File logFile; final FileChannel channel; final int fd; @@ -312,6 +318,8 @@ void waitForModifications() * Update the chained markers in the commit log buffer and possibly force a disk flush for this segment file. * * @param flush true if the segment should flush to disk; else, false for just updating the chained markers. + * Named such to disambiguate whether we're looking to flush associated memtables to disk + * or just this CL segment. */ synchronized void sync(boolean flush) { @@ -365,7 +373,6 @@ synchronized void sync(boolean flush) sectionEnd = nextMarker; } - if (flush || close) { flush(startMarker, sectionEnd); diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java index 0a7fbd21cc40..bd96a23d47b2 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDC.java @@ -195,15 +195,15 @@ void addCDCSize(long size) * data in them and all segments archived into cdc_raw. * * Allows atomic increment/decrement of unflushed size, however only allows increment on flushed and requires a full - * directory walk to determine any potential deletions by CDC consumer. + * directory walk to determine any potential deletions by an external CDC consumer. */ private static class CDCSizeTracker extends DirectorySizeCalculator { private final RateLimiter rateLimiter = RateLimiter.create(1000.0 / DatabaseDescriptor.getCDCDiskCheckInterval()); private ExecutorService cdcSizeCalculationExecutor; - private CommitLogSegmentManager segmentManager; + private final CommitLogSegmentManager segmentManager; - // Used instead of size during walk to remove chance of over-allocation + /** Used only in context of file tree walking thread; not read nor mutated outside this context */ private volatile long sizeInProgress = 0; CDCSizeTracker(CommitLogSegmentManager segmentManager, File path) @@ -311,14 +311,14 @@ private void calculateSize() { try { - // The Arrays.stream approach is considerably slower on Windows than linux + // The Arrays.stream approach is considerably slower sizeInProgress = 0; Files.walkFileTree(path.toPath(), this); size = sizeInProgress; } catch (IOException ie) { - CommitLog.instance.handleCommitError("Failed CDC Size Calculation", ie); + CommitLog.handleCommitError("Failed CDC Size Calculation", ie); } } diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java index 68d2fb369a47..2c3878ac1917 100755 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java @@ -65,11 +65,7 @@ public class CommitLogSegmentManager */ private final ConcurrentLinkedQueue unflushedSegments = new ConcurrentLinkedQueue<>(); - /** - * The segment we are currently allocating commit log records to. - * - * Written by switchToNewSegment which synchronizes on 'this'. Volatile to ensure reads get current value. - */ + /** The segment we are currently allocating commit log records to. */ private volatile CommitLogSegment activeSegment = null; final String storageDirectory; diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java index 2840d82d92ee..fd8ff3078ddb 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentReader.java @@ -21,63 +21,64 @@ import java.nio.ByteBuffer; import java.util.Iterator; import java.util.zip.CRC32; +import javax.annotation.concurrent.NotThreadSafe; import javax.crypto.Cipher; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.AbstractIterator; +import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadErrorReason; +import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadException; import org.apache.cassandra.db.commitlog.EncryptedFileSegmentInputStream.ChunkProvider; -import org.apache.cassandra.db.commitlog.CommitLogReadHandler.*; import org.apache.cassandra.io.FSReadError; import org.apache.cassandra.io.compress.ICompressor; import org.apache.cassandra.io.util.FileDataInput; import org.apache.cassandra.io.util.FileSegmentInputStream; import org.apache.cassandra.io.util.RandomAccessReader; import org.apache.cassandra.schema.CompressionParams; -import org.apache.cassandra.security.EncryptionUtils; import org.apache.cassandra.security.EncryptionContext; +import org.apache.cassandra.security.EncryptionUtils; import org.apache.cassandra.utils.ByteBufferUtil; import static org.apache.cassandra.db.commitlog.CommitLogSegment.SYNC_MARKER_SIZE; import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt; /** - * Read each sync section of a commit log, iteratively. + * Read each sync section of a commit log, iteratively. Can be run in either one-shot or resumable mode. In resumable, + * we snapshot the start position of any successful SyncSegment deserialization with the expectation that some reads will + * land in partially written segments and need to be rolled back to the start of that segment and repeated on further + * mutation serialization (specifically in encrypted or compressed contexts). */ +@NotThreadSafe public class CommitLogSegmentReader implements Iterable { - private final CommitLogReadHandler handler; - private final CommitLogDescriptor descriptor; - private final RandomAccessReader reader; + private final ResumableCommitLogReader parent; private final Segmenter segmenter; - private final boolean tolerateTruncation; - /** - * Ending position of the current sync section. - */ + /** A special SyncSegment we use to indicate / keep our iterators open on a read we intend to resume */ + static final SyncSegment RESUMABLE_SENTINEL = new SyncSegment(null, -1, -1, -1, false); + + /** Ending position of the current sync section. */ protected int end; /** * Rather than relying on a formal Builder, this constructs the appropriate type of segment reader (memmap, encrypted, * compressed) based on the type stored in the descriptor. + * + * Note: If ever using this object directly in a test, ensure you set the {@link ResumableCommitLogReader#offsetLimit} + * before attempting to use this reader or iteration will never advance. */ - CommitLogSegmentReader(CommitLogReadHandler handler, - CommitLogDescriptor descriptor, - RandomAccessReader reader, - boolean tolerateTruncation) + CommitLogSegmentReader(ResumableCommitLogReader parent) { - this.handler = handler; - this.descriptor = descriptor; - this.reader = reader; - this.tolerateTruncation = tolerateTruncation; - - end = (int) reader.getFilePointer(); - if (descriptor.getEncryptionContext().isEnabled()) - segmenter = new EncryptedSegmenter(descriptor, reader); - else if (descriptor.compression != null) - segmenter = new CompressedSegmenter(descriptor, reader); + this.parent = parent; + + end = (int) parent.rawReader.getFilePointer(); + if (parent.descriptor.getEncryptionContext().isEnabled()) + segmenter = new EncryptedSegmenter(parent.descriptor, parent); + else if (parent.descriptor.compression != null) + segmenter = new CompressedSegmenter(parent.descriptor, parent); else - segmenter = new NoOpSegmenter(reader); + segmenter = new NoOpSegmenter(parent.rawReader); } public Iterator iterator() @@ -85,36 +86,75 @@ public Iterator iterator() return new SegmentIterator(); } + /** Will return endOfData() or our resumable sentinel depending on what mode the iterator is being used in */ protected class SegmentIterator extends AbstractIterator { protected SyncSegment computeNext() { + // A couple sanity checks that we're in a good state + if (parent.offsetLimit == Integer.MIN_VALUE) + throw new RuntimeException("Attempted to use a CommitLogSegmentReader with an uninitialized ResumableCommitLogReader parent."); + + // Since this could be mis-used by client app parsing code, keep it RTE instead of assertion. + if (parent.isClosed) + throw new RuntimeException("Attempted to use a closed ResumableCommitLogReader."); + while (true) { try { final int currentStart = end; - end = readSyncMarker(descriptor, currentStart, reader); - if (end == -1) + + // Segmenters need to know our original state to appropriately roll back on snapshot restore + segmenter.stageSnapshot(); + end = readSyncMarker(parent.descriptor, currentStart, parent.rawReader); + + if (parent.isPartial()) { - return endOfData(); + // Revert our SegmentIterator's state to beginning of last completed SyncSegment read on a partial read. + if (end == -1 || end > parent.offsetLimit) + { + segmenter.revertToSnapshot(); + end = (int)parent.rawReader.getFilePointer(); + return RESUMABLE_SENTINEL; + } + // Flag our RR's data as exhausted if we've hit the end of our reader but think this is partial. + else if (end >= parent.rawReader.length()) + { + parent.readToExhaustion = true; + } } - if (end > reader.length()) + // Iterate on a non-resumable read. + else { - // the CRC was good (meaning it was good when it was written and still looks legit), but the file is truncated now. - // try to grab and use as much of the file as possible, which might be nothing if the end of the file truly is corrupt - end = (int) reader.length(); + if (end == -1) + { + // We only transition to endOfData if we're doing a non-resumable (i.e. read to end) read, + // since it leaves this iterator in a non-reusable state. + return endOfData(); + } + else if (end > parent.rawReader.length()) + { + // the CRC was good (meaning it was good when it was written and still looks legit), but the file is truncated now. + // try to grab and use as much of the file as possible, which might be nothing if the end of the file truly is corrupt + end = (int) parent.rawReader.length(); + } } + + // Retain the starting point of this SyncSegment in case we need to roll back a future read to this point. + segmenter.takeSnapshot(); + + // Passed the gauntlet. The next segment is cleanly ready for read. return segmenter.nextSegment(currentStart + SYNC_MARKER_SIZE, end); } catch(CommitLogSegmentReader.SegmentReadException e) { try { - handler.handleUnrecoverableError(new CommitLogReadException( - e.getMessage(), - CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR, - !e.invalidCrc && tolerateTruncation)); + parent.readHandler.handleUnrecoverableError(new CommitLogReadException( + e.getMessage(), + CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR, + !e.invalidCrc && parent.tolerateTruncation)); } catch (IOException ioe) { @@ -125,12 +165,12 @@ protected SyncSegment computeNext() { try { - boolean tolerateErrorsInSection = tolerateTruncation & segmenter.tolerateSegmentErrors(end, reader.length()); + boolean tolerateErrorsInSection = parent.tolerateTruncation & segmenter.tolerateSegmentErrors(end, parent.rawReader.length()); // if no exception is thrown, the while loop will continue - handler.handleUnrecoverableError(new CommitLogReadException( - e.getMessage(), - CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR, - tolerateErrorsInSection)); + parent.readHandler.handleUnrecoverableError(new CommitLogReadException( + e.getMessage(), + CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR, + tolerateErrorsInSection)); } catch (IOException ioe) { @@ -141,13 +181,13 @@ protected SyncSegment computeNext() } } + /** + * @return length of this sync segment, -1 if at or beyond the end of file. + */ private int readSyncMarker(CommitLogDescriptor descriptor, int offset, RandomAccessReader reader) throws IOException { if (offset > reader.length() - SYNC_MARKER_SIZE) - { - // There was no room in the segment to write a final header. No data could be present here. return -1; - } reader.seek(offset); CRC32 crc = new CRC32(); updateChecksumInt(crc, (int) (descriptor.id & 0xFFFFFFFFL)); @@ -184,9 +224,7 @@ public SegmentReadException(String msg, boolean invalidCrc) } } - /** - * The logical unit of data we sync across and read across in CommitLogs. - */ + /** The logical unit of data we sync across and read across in CommitLogs. */ public static class SyncSegment { /** the 'buffer' to replay commit log data from */ @@ -215,6 +253,8 @@ public static class SyncSegment /** * Derives the next section of the commit log to be replayed. Section boundaries are derived from the commit log sync markers. + * Allows snapshot and resume from snapshot functionality to revert to a "last known good segment" in the event of + * a partial read on an a file being actively written. */ interface Segmenter { @@ -235,11 +275,27 @@ default boolean tolerateSegmentErrors(int segmentEndPosition, long fileLength) { return segmentEndPosition >= fileLength || segmentEndPosition < 0; } + + /** Holds snapshot data in temporary variables to be finalized when we determine a SyncSegment is fully written */ + void stageSnapshot(); + + /** Finalizes snapshot staged in stageSnapshot */ + void takeSnapshot(); + + /** Reverts the segmenter to the previously held position. Allows for resumable reads to rollback when they occur + * in the middle of a SyncSegment. This can be called repeatedly if we have multiple attempts to partially read + * on an incomplete SyncSegment. */ + void revertToSnapshot(); + + /** Visible for debugging only */ + long getSnapshot(); } static class NoOpSegmenter implements Segmenter { private final RandomAccessReader reader; + private long snapshotPosition = Long.MIN_VALUE; + private long stagedSnapshot = Long.MIN_VALUE; NoOpSegmenter(RandomAccessReader reader) { @@ -256,54 +312,114 @@ public boolean tolerateSegmentErrors(int end, long length) { return true; } + + public void stageSnapshot() + { + stagedSnapshot = reader.getFilePointer(); + // Deal with edge case of initial read attempt being before SyncSegment completion + if (snapshotPosition == Long.MIN_VALUE) + takeSnapshot(); + } + + public void takeSnapshot() + { + snapshotPosition = stagedSnapshot; + } + + public void revertToSnapshot() + { + reader.seek(snapshotPosition); + } + + public long getSnapshot() + { + return snapshotPosition; + } } static class CompressedSegmenter implements Segmenter { private final ICompressor compressor; - private final RandomAccessReader reader; + /** We store a reference to a ResumableReader in the event it needs to re-init and swap out the underlying reader */ + private final ResumableCommitLogReader parent; private byte[] compressedBuffer; private byte[] uncompressedBuffer; private long nextLogicalStart; - CompressedSegmenter(CommitLogDescriptor desc, RandomAccessReader reader) + private long stagedLogicalStart = Long.MIN_VALUE; + private long stagedReaderLocation = Long.MIN_VALUE; + private long snapshotLogicalStart = Long.MIN_VALUE; + private long snapshotReaderLocation = Long.MIN_VALUE; + + CompressedSegmenter(CommitLogDescriptor desc, ResumableCommitLogReader parent) { - this(CompressionParams.createCompressor(desc.compression), reader); + this(CompressionParams.createCompressor(desc.compression), parent); } - CompressedSegmenter(ICompressor compressor, RandomAccessReader reader) + CompressedSegmenter(ICompressor compressor, ResumableCommitLogReader parent) { this.compressor = compressor; - this.reader = reader; + this.parent = parent; compressedBuffer = new byte[0]; uncompressedBuffer = new byte[0]; - nextLogicalStart = reader.getFilePointer(); + nextLogicalStart = parent.rawReader.getFilePointer(); } @SuppressWarnings("resource") public SyncSegment nextSegment(final int startPosition, final int nextSectionStartPosition) throws IOException { - reader.seek(startPosition); - int uncompressedLength = reader.readInt(); + parent.rawReader.seek(startPosition); + int uncompressedLength = parent.rawReader.readInt(); - int compressedLength = nextSectionStartPosition - (int)reader.getPosition(); + int compressedLength = nextSectionStartPosition - (int)parent.rawReader.getPosition(); if (compressedLength > compressedBuffer.length) compressedBuffer = new byte[(int) (1.2 * compressedLength)]; - reader.readFully(compressedBuffer, 0, compressedLength); + parent.rawReader.readFully(compressedBuffer, 0, compressedLength); if (uncompressedLength > uncompressedBuffer.length) uncompressedBuffer = new byte[(int) (1.2 * uncompressedLength)]; int count = compressor.uncompress(compressedBuffer, 0, compressedLength, uncompressedBuffer, 0); nextLogicalStart += SYNC_MARKER_SIZE; - FileDataInput input = new FileSegmentInputStream(ByteBuffer.wrap(uncompressedBuffer, 0, count), reader.getPath(), nextLogicalStart); + FileDataInput input = new FileSegmentInputStream(ByteBuffer.wrap(uncompressedBuffer, 0, count), parent.rawReader.getPath(), nextLogicalStart); nextLogicalStart += uncompressedLength; - return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, reader.length())); + return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, parent.rawReader.length())); + } + + public void stageSnapshot() + { + stagedLogicalStart = nextLogicalStart; + stagedReaderLocation = parent.rawReader.getFilePointer(); + + // In our default 0 case on a segment w/out anything yet to read, we want to stage the first valid location + // we've seen, else a resume will kick us to a bad value + if (snapshotLogicalStart == Long.MIN_VALUE) + takeSnapshot(); + } + + /** Since {@link #nextLogicalStart} is mutated during decompression but relied upon for decompression, we need + * to both snapshot and revert that along with the reader's position. */ + public void takeSnapshot() + { + snapshotLogicalStart = stagedLogicalStart; + snapshotReaderLocation = stagedReaderLocation; + } + + public void revertToSnapshot() + { + nextLogicalStart = snapshotLogicalStart; + parent.rawReader.seek(snapshotReaderLocation); + } + + public long getSnapshot() + { + return snapshotReaderLocation; } } static class EncryptedSegmenter implements Segmenter { - private final RandomAccessReader reader; + /** We store a reference to a ResumableReader in the event it needs to re-init and swap out the underlying reader */ + private final ResumableCommitLogReader parent; private final ICompressor compressor; private final Cipher cipher; @@ -322,18 +438,21 @@ static class EncryptedSegmenter implements Segmenter private long currentSegmentEndPosition; private long nextLogicalStart; - EncryptedSegmenter(CommitLogDescriptor descriptor, RandomAccessReader reader) + private long stagedSnapshotPosition; + private long snapshotPosition; + + EncryptedSegmenter(CommitLogDescriptor descriptor, ResumableCommitLogReader parent) { - this(reader, descriptor.getEncryptionContext()); + this(parent, descriptor.getEncryptionContext()); } @VisibleForTesting - EncryptedSegmenter(final RandomAccessReader reader, EncryptionContext encryptionContext) + EncryptedSegmenter(final ResumableCommitLogReader parent, EncryptionContext encryptionContext) { - this.reader = reader; + this.parent = parent; decryptedBuffer = ByteBuffer.allocate(0); compressor = encryptionContext.getCompressor(); - nextLogicalStart = reader.getFilePointer(); + nextLogicalStart = parent.rawReader.getFilePointer(); try { @@ -341,21 +460,21 @@ static class EncryptedSegmenter implements Segmenter } catch (IOException ioe) { - throw new FSReadError(ioe, reader.getPath()); + throw new FSReadError(ioe, parent.rawReader.getPath()); } chunkProvider = () -> { - if (reader.getFilePointer() >= currentSegmentEndPosition) + if (parent.rawReader.getFilePointer() >= currentSegmentEndPosition) return ByteBufferUtil.EMPTY_BYTE_BUFFER; try { - decryptedBuffer = EncryptionUtils.decrypt(reader, decryptedBuffer, true, cipher); + decryptedBuffer = EncryptionUtils.decrypt(parent.rawReader, decryptedBuffer, true, cipher); uncompressedBuffer = EncryptionUtils.uncompress(decryptedBuffer, uncompressedBuffer, true, compressor); return uncompressedBuffer; } catch (IOException e) { - throw new FSReadError(e, reader.getPath()); + throw new FSReadError(e, parent.rawReader.getPath()); } }; } @@ -363,13 +482,35 @@ static class EncryptedSegmenter implements Segmenter @SuppressWarnings("resource") public SyncSegment nextSegment(int startPosition, int nextSectionStartPosition) throws IOException { - int totalPlainTextLength = reader.readInt(); + int totalPlainTextLength = parent.rawReader.readInt(); currentSegmentEndPosition = nextSectionStartPosition - 1; nextLogicalStart += SYNC_MARKER_SIZE; - FileDataInput input = new EncryptedFileSegmentInputStream(reader.getPath(), nextLogicalStart, 0, totalPlainTextLength, chunkProvider); + FileDataInput input = new EncryptedFileSegmentInputStream(parent.rawReader.getPath(), nextLogicalStart, 0, totalPlainTextLength, chunkProvider); nextLogicalStart += totalPlainTextLength; - return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, reader.length())); + return new SyncSegment(input, startPosition, nextSectionStartPosition, (int)nextLogicalStart, tolerateSegmentErrors(nextSectionStartPosition, parent.rawReader.length())); + } + + public void stageSnapshot() + { + stagedSnapshotPosition = parent.rawReader.getFilePointer(); + if (snapshotPosition == Long.MIN_VALUE) + takeSnapshot(); + } + + public void takeSnapshot() + { + snapshotPosition = stagedSnapshotPosition; + } + + public void revertToSnapshot() + { + parent.rawReader.seek(snapshotPosition); + } + + public long getSnapshot() + { + return snapshotPosition; } } } diff --git a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java index 50c09dd813d8..9009e4cd2250 100644 --- a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java @@ -22,6 +22,9 @@ import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.io.FSWriteError; import org.apache.cassandra.io.util.FileUtils; @@ -35,6 +38,8 @@ */ public class MemoryMappedSegment extends CommitLogSegment { + private static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentReader.class); + /** * Constructs a new segment file. * diff --git a/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java index a5fce7b618df..ac925bb5f36c 100644 --- a/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java +++ b/src/java/org/apache/cassandra/db/commitlog/ResumableCommitLogReader.java @@ -17,32 +17,256 @@ */ package org.apache.cassandra.db.commitlog; -import org.apache.cassandra.config.DatabaseDescriptor; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.text.MessageFormat; +import java.util.Iterator; +import java.util.Optional; +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.cassandra.io.util.RandomAccessReader; +import org.apache.cassandra.utils.Pair; /** - * A CommitLogReader that allows both a) indicating how much of the CommitLog the user would like to read, and b) the ability - * to resume reading from a given offset to prevent the need to re-read CommitLog files in a CDC / consumption context. - * As this is purely a class for use in CDC-enabled contexts, we don't test its operation in non-cdc and assert that CDC - * is enabled if this class is in use. + * A state holder for a potentially resumable read. As we want to resume our reading with existing file pointers, buffers, + * and sentinels without re-opening a file and re-decompressing or decrypting, we store references to a {@link CommitLogSegmentReader}, + * to a {@link RandomAccessReader}, and to an iterator of {@link CommitLogSegmentReader.SyncSegment} here for re-use. + * + * This serves dual purpose as an API endpoint and logical state holder to pop out our handles across multiple reads + * while minimizing pollution to core CommitLogReader code implementation. * - * Note: As the plan for usage of this class is by external users writing CDC consumers, treat the public members of this - * class as an external API contract. i.e. don't make changes to it lightly, and consider backwards-compatibility options - * on changes. In the event that this interface proves to be more fluid over time than the previous stability of the CommitLog - * in general would imply, we can break out a proper interface for it at that time. + * _Mandatory_ usage of this API is as follows: + * 0-N) {@link #readPartial(int limit)} + * 1) {@link #readToCompletion()} + * NOTE: neither of these callse will {@link #close} this reader. try-with-resources is the correct usage. + * to correctly close out. + * + * As this is intended to be used both in an internal C* state as well as by external users looking to read CommitLogSegments, + * we allow construction to fail gracefully and indicate usability through {@link #isClosed()}. */ -public class ResumableCommitLogReader +@NotThreadSafe +public class ResumableCommitLogReader implements AutoCloseable { - public final CommitLogDescriptor descriptor; - public final RandomAccessReader reader; + private static final Logger logger = LoggerFactory.getLogger(CommitLogReader.class); + + /** We hold a reference to these so we can re-use them for subsequent descriptor parsing on resumed reads */ + final File segmentFile; + final CommitLogDescriptor descriptor; + private final CommitLogReader commitLogReader; + final CommitLogReadHandler readHandler; + final boolean tolerateTruncation; + + /** Can be re-initialized if re-reading a reader w/compression enabled and we're at our known limit. */ + RandomAccessReader rawReader; + + /** We allow the users to determine whether or not the system should continue on various forms of read failure. As + * such, we allow resumable readers to be constructed even if they are unusable to the end-user. */ + boolean isClosed = false; + + /** Separate sentinel to indicate whether we have read to completion on our underlying file. Flagged by SegmentReader + * We use this during {@link #reBufferData()} to determine whether or not to recreate our underlying RAR in the compressed + * case. + */ + boolean readToExhaustion = false; + + /** Minimum position before which we completely skip CommitLogSegments */ + final CommitLogPosition minPosition; + + /** Sentinel used to limit reads */ + final int mutationLimit; + + /** We cache, snapshot, and revert position inside our {@link CommitLogSegmentReader.SegmentIterator#computeNext} calls + * to keep the user-facing API simple */ + @Nullable + Iterator activeIterator; + + @Nullable + private CommitLogSegmentReader segmentReader; + + /** Raw file offset at which we will stop iterating and processing mutations on a read */ + int offsetLimit = Integer.MIN_VALUE; + + public ResumableCommitLogReader(File commitLogSegment, CommitLogReadHandler readHandler) throws IOException, NullPointerException + { + this(commitLogSegment, readHandler, CommitLogPosition.NONE, CommitLogReader.ALL_MUTATIONS, true); + } + + public ResumableCommitLogReader(File commitLogSegment, + CommitLogReadHandler readHandler, + CommitLogPosition minPosition, + int mutationLimit, + boolean tolerateTruncation) throws IOException, NullPointerException + { + this.segmentFile = commitLogSegment; + this.commitLogReader = new CommitLogReader(); + this.readHandler = readHandler; + this.mutationLimit = mutationLimit; + this.minPosition = minPosition; + this.tolerateTruncation = tolerateTruncation; + + Pair, Integer> header = CommitLogReader.readCommitLogDescriptor(readHandler, + commitLogSegment, + tolerateTruncation); + // Totally fail out if we fail to parse this CommitLogSegment descriptor + if (!header.left.isPresent()) + throw new RuntimeException(MessageFormat.format("Failed to parse the CommitLogDescriptor from {0}", commitLogSegment)); + descriptor = header.left.get(); + + if (shouldSkipSegmentId(new File(descriptor.fileName()), descriptor, minPosition)) + { + close(); + } + else + { + try + { + this.rawReader = RandomAccessReader.open(commitLogSegment); + rawReader.seek(header.right); + + // This is where we grab and old open our handles if we succeed + segmentReader = CommitLogReader.getCommitLogSegmentReader(this); + if (segmentReader != null) + this.activeIterator = segmentReader.iterator(); + } + finally + { + if (segmentReader == null) + close(); + } + } + } + + /** + * Performs a partial CommitLogSegment read. Closes down this resumable reader on read error. + * + * @param readLimit How far to read into the file before stopping. + */ + public void readPartial(int readLimit) throws IOException + { + if (isClosed) + { + logger.warn("Attempted to use invalid ResumableCommitLogReader for file {}. Ignoring.", descriptor.fileName()); + return; + } + + if (readLimit <= offsetLimit) + { + logger.warn("Attempted to resume reading a commit log but used already read offset: {}", readLimit); + return; + } + offsetLimit = readLimit; + rebufferAndRead(); + } - public ResumableCommitLogReader(CommitLogDescriptor descriptor, RandomAccessReader reader) + /** Reads to end of file from current cached offset. */ + public void readToCompletion() throws IOException { - if (!DatabaseDescriptor.isCDCEnabled()) + if (isClosed) { - throw new RuntimeException("Cannot use a ResumableCommitLogReader if CDC is not enabled."); + logger.warn("Attempted to use invalid ResumableCommitLogReader for file {}. Ignoring.", descriptor.fileName()); + return; } - this.descriptor = descriptor; - this.reader = reader; + offsetLimit = CommitLogReader.READ_TO_END_OF_FILE; + rebufferAndRead(); + } + + public void close() + { + isClosed = true; + if (rawReader != null) + rawReader.close(); + segmentReader = null; + activeIterator = null; + } + + public boolean isClosed() + { + return isClosed; + } + + /** + * When we have compression enabled, RandomAccessReader's have CompressionMetadata to indicate the end of their file + * length. For our purposes, this means we have some difficulty in re-using previously constructed underlying buffers + * for decompression and reading, so if the underlying file length has changed because a file is actively being written + * and we've exhausted the current data we know about, we close out our RAR and construct a new one with the new + * {@link org.apache.cassandra.io.compress.CompressionMetadata}. While it would arguably be better to extend the + * hierarchy to have a rebuffering compressed segment, YAGNI for now. The added gc pressure from this + overhead + * on closing and re-opening RAR's should be restricted to non-node partial/resumed CL reading cases which we expect + * to have very different properties than critical path log replay on a running node, for example. + */ + private void reBufferData() throws FileNotFoundException + { + if (readToExhaustion) + { + long toSeek = rawReader.getPosition(); + this.rawReader.close(); + if (!segmentFile.exists()) + throw new FileNotFoundException(String.format("Attempting to reBufferData but underlying file cannot be found: {}", + segmentFile.getAbsolutePath())); + this.rawReader = RandomAccessReader.open(segmentFile); + this.rawReader.seek(toSeek); + } + else + { + rawReader.reBuffer(); + } + } + + /** Performs the read operation and closes down this reader on exception. */ + private void rebufferAndRead() throws RuntimeException, IOException + { + reBufferData(); + + try + { + commitLogReader.internalReadCommitLogSegment(this); + } + catch (RuntimeException | IOException e) + { + close(); + throw e; + } + } + + /** Any segment with id >= minPosition.segmentId is a candidate for read. */ + private boolean shouldSkipSegmentId(File file, CommitLogDescriptor desc, CommitLogPosition minPosition) + { + logger.debug("Reading {} (CL version {}, messaging version {}, compression {})", + file.getPath(), + desc.version, + desc.getMessagingVersion(), + desc.compression); + + if (minPosition.segmentId > desc.id) + { + logger.trace("Skipping read of fully-flushed {}", file); + return true; + } + return false; + } + + /** Flag to indicate how the {@link CommitLogSegmentReader.SegmentIterator} should behave on failure to compute next + * segments. + */ + boolean isPartial() + { + return offsetLimit != CommitLogReader.READ_TO_END_OF_FILE; + } + + @Override + public String toString() + { + return new StringBuilder() + .append("File: ").append(descriptor.fileName()).append(", ") + .append("minPos: ").append(minPosition).append(", ") + .append("offsetLimit: ").append(offsetLimit).append(", ") + .append("readerPos: ").append(rawReader.getPosition()).append(", ") + .append("activeIter: ").append(activeIterator) + .toString(); } } diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 9f383df14c67..b09fa0cfa74f 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -1703,7 +1703,6 @@ protected com.datastax.driver.core.TupleType tupleTypeOf(ProtocolVersion protoco */ protected void populateReferenceData(boolean withCDC) throws Throwable { - Random random = new Random(); String createString = "CREATE TABLE %s (a int, b int, c double, d decimal, e smallint, f tinyint, g blob, primary key (a, b))"; if (withCDC) @@ -1712,27 +1711,35 @@ protected void populateReferenceData(boolean withCDC) throws Throwable // For each test, we start with the assumption of a populated set of a few files we can pull from. createTable(createString); - byte[] bBlob = new byte[1024 * 1024]; + byte[] buffer = new byte[1024 * 256]; CommitLog.instance.sync(true); // Populate some CommitLog segments on disk - for (int i = 0; i < 20; i++) - { - random.nextBytes(bBlob); - - logger.debug(String.format("Executing insert for index: [%d]", i)); - execute("INSERT INTO %s (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?)", - random.nextInt(), - random.nextInt(), - random.nextDouble(), - random.nextLong(), - (short)random.nextInt(), - (byte)random.nextInt(), - ByteBuffer.wrap(bBlob)); - } + writeReferenceLines(80, buffer); CommitLog.instance.sync(true); } + protected void writeReferenceLines(int num, byte[] buffer) throws Throwable + { + for (int i = 0; i < num; i++) + writeReferenceDataLine(buffer); + } + + /** Broken out for access from tests that need to write incrementally more ref. data */ + protected void writeReferenceDataLine(byte[] buffer) throws Throwable + { + Random random = new Random(); + random.nextBytes(buffer); + execute("INSERT INTO %s (a, b, c, d, e, f, g) VALUES (?, ?, ?, ?, ?, ?, ?)", + random.nextInt(), + random.nextInt(), + random.nextDouble(), + random.nextLong(), + (short)random.nextInt(), + (byte)random.nextInt(), + ByteBuffer.wrap(buffer)); + } + // Attempt to find an AbstracType from a value (for serialization/printing sake). // Will work as long as we use types we know of, which is good enough for testing private static AbstractType typeFor(Object value) diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java index 253ea43d5874..19ddb8095363 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentAllocatorCDCTest.java @@ -328,7 +328,7 @@ public void testReplayLogic() throws IOException CommitLog.instance.start(); CommitLog.instance.segmentManager.awaitManagementTasksCompletion(); } - CommitLogTestUtils.CDCMutationCountingReplayer replayer = new CommitLogTestUtils.CDCMutationCountingReplayer(); + CommitLogTestUtils.MutationCountingReplayer replayer = new CommitLogTestUtils.MutationCountingReplayer(); replayer.replayExistingCommitLog(); // Rough sanity check -> should be files there now. diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java index 62de4561f6f0..5c886315aad5 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java +++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestUtils.java @@ -21,22 +21,20 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.text.MessageFormat; import java.util.Objects; import java.util.Random; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; -import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.junit.Assert; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.db.rows.SerializationHelper; -import org.apache.cassandra.io.util.DataInputBuffer; -import org.apache.cassandra.io.util.RebufferingInputStream; /** * Collection of some helper methods and classes for use in our various CommitLog Unit Tests @@ -74,36 +72,7 @@ static long updateCDCTotalSize(CommitLogSegmentManager segmentManager) return ((CommitLogSegmentAllocatorCDC)segmentManager.segmentAllocator).updateCDCTotalSize(); } - /** - * Pulls the back of the commit log files list and bails out if that is == our current allocating. Goal is to get a filled - * and usable commit log segment for testing work. - */ - static File getFilledCommitLogFile() throws NullPointerException - { - File result = new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()[0]; - Assert.assertNotEquals(result.toString(), CommitLog.instance.segmentManager.getActiveSegment().logFile); - return result; - } - - /** - * There's some possible raciness here, but for purposes of unit tests this should be deterministic enough not to cause - * test failures, assuming we're doing deterministic writes when using this method. - */ - static int getCommitLogCountOnDisk() - { - return CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size(); - } - - /** - * For a given input file, writes randomized garbage from offset to end of file to "corrupt" it - */ - static void corruptFileAtOffset(File f, int offset) - { - } - - /** - * Debug method to allow printing of a message when sanity checking commit log state while writing tests - */ + /** Debug method to show written files; useful when debugging specific tests. */ static void listCommitLogFiles(String message) { StringBuilder result = new StringBuilder(); @@ -116,9 +85,7 @@ static void listCommitLogFiles(String message) debugLog(result.toString()); } - /** - * Used during test debug to differentiate output visually - */ + /** Used during test debug to differentiate output visually */ static void debugLog(String input) { logger.debug("\n\n**************** [TEST DEBUG] *****************\n" + @@ -129,32 +96,32 @@ static void debugLog(String input) } /** - * Utility class that flags the replayer as having seen a CDC mutation and calculates offset but doesn't apply mutations + * Utility class that flags the replayer as having seen a CDC mutation and calculates offset but doesn't apply mutations. */ - static class CDCMutationCountingReplayer extends CommitLogReplayer + static class MutationCountingReplayer extends CommitLogReplayer { - final Set seenMutations = Sets.newConcurrentHashSet(); + final ConcurrentLinkedQueue seenMutations = new ConcurrentLinkedQueue<>(); + final ConcurrentHashMap duplicateMutations = new ConcurrentHashMap<>(); + final MutationCountingHandler mutationHandler = new MutationCountingHandler(); - CDCMutationCountingReplayer() throws IOException + MutationCountingReplayer() throws IOException { super(CommitLog.instance, CommitLogPosition.NONE, null, ReplayFilter.create()); CommitLog.instance.sync(true); - commitLogReader = new CDCCountingReader(); } - /** - * Takes existing files in the commit log location and forces a replay on them. Only really meaningful if you're - * intercepting either the mutation read handler on replay or the mutation initiation object. - */ void replayExistingCommitLog() throws IOException { - replayFiles(new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()); + for (File f: new File(DatabaseDescriptor.getCommitLogLocation()).listFiles()) + { + commitLogReader.readCommitLogSegment(mutationHandler, f, true); + } } void replaySingleFile(File f) throws IOException { - replayFiles(new File[] { f }); + commitLogReader.readCommitLogSegment(mutationHandler, f, true); } boolean hasSeenMutation(MutationIdentifier id) @@ -173,56 +140,61 @@ boolean hasSeenDuplicateMutations() return duplicateMutations.size() > 0; } - private class CDCCountingReader extends CommitLogReader + private class MutationCountingHandler implements CommitLogReadHandler { - @Override - protected void readMutation(CommitLogReadHandler handler, - byte[] inputBuffer, - int size, - CommitLogPosition minPosition, - final int entryLocation, - final CommitLogDescriptor desc) + public boolean shouldSkipSegmentOnError(CommitLogReadException exception) + { + return false; + } + + public void handleUnrecoverableError(CommitLogReadException exception) + { + Assert.fail(MessageFormat.format("Got unrecoverable error during test: {0}", exception.getMessage())); + } + + public void handleMutation(Mutation m, int size, int entryLocation, CommitLogDescriptor desc) { - MutationIdentifier id = new MutationIdentifier(minPosition.segmentId, size, entryLocation); + MutationIdentifier id = new MutationIdentifier(m.getKeyspaceName(), m.key(), desc.id, size, entryLocation); + + if (m.trackedByCDC()) + sawCDCMutation = true; - RebufferingInputStream bufIn = new DataInputBuffer(inputBuffer, 0, size); - Mutation mutation; - try + if (seenMutations.contains(id)) { - mutation = Mutation.serializer.deserialize(bufIn, desc.getMessagingVersion(), SerializationHelper.Flag.LOCAL); - - if (mutation.trackedByCDC()) - { - sawCDCMutation = true; - if (seenMutations.contains(id)) - { - Integer pv = duplicateMutations.get(id); - if (pv == null) - pv = 0; - duplicateMutations.put(id, pv + 1); - } - seenMutations.add(id); - } + Integer pv = duplicateMutations.get(id); + if (pv == null) + pv = 0; + duplicateMutations.put(id, pv + 1); } - catch (IOException e) + else { - // Test fails. - throw new AssertionError(e); + seenMutations.add(id); } } } + public void reset() + { + seenMutations.clear(); + duplicateMutations.clear(); + sawCDCMutation = false; + } + /** * Helper class that allows us to uniquely identify a mutation at least within a single instance of a running node. */ - private static class MutationIdentifier + static class MutationIdentifier { final long segmentId; final int size; final int location; + final String keyspaceName; + final DecoratedKey decoratedKey; - MutationIdentifier(long segmentId, int size, int location) + MutationIdentifier(String keyspaceName, DecoratedKey key, long segmentId, int size, int location) { + this.keyspaceName = keyspaceName; + this.decoratedKey = key; this.segmentId = segmentId; this.size = size; this.location = location; @@ -241,14 +213,42 @@ public boolean equals(Object o) return other.size == this.size && other.location == this.location && - other.segmentId == this.segmentId; + other.segmentId == this.segmentId && + other.keyspaceName.equals(this.keyspaceName) && + other.decoratedKey.equals(this.decoratedKey); } @Override public int hashCode() { - return Objects.hash(size, location, segmentId); + return Objects.hash(size, location, segmentId, keyspaceName, decoratedKey); + } + + @Override + public String toString() + { + return new StringBuilder() + .append("sId: ").append(segmentId).append(", ") + .append(" size: ").append(size).append(", ") + .append(" loc: ").append(location).append(", ") + .append(" ks: ").append(keyspaceName).append(", ") + .append(" dk: ").append(decoratedKey) + .toString(); } } } + + static class NoopMutationHandler implements CommitLogReadHandler + { + public boolean shouldSkipSegmentOnError(CommitLogReadException exception) throws IOException + { + return false; + } + + public void handleUnrecoverableError(CommitLogReadException exception) throws IOException + { } + + public void handleMutation(Mutation m, int size, int entryLocation, CommitLogDescriptor desc) + { } + } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java index 1c8484c1eb0e..fae154ec3653 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/ResumableCommitLogReaderTest.java @@ -18,61 +18,60 @@ package org.apache.cassandra.db.commitlog; +import java.io.BufferedReader; import java.io.File; +import java.io.FileReader; +import java.io.FileWriter; import java.io.IOException; +import java.util.ArrayList; import org.junit.Assert; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLTester; -/* Forces CDC enabled on the run of these tests as they aren't used in non-CDC context and ResumableCommitLogReader - * class asserts cdc enabled on creation. - * - * Note: Many of these tests depend on having a CommitLog segment they're working on, and we don't force a stop of the - * commit log segment allocator or flushing mechanisms while running these tests. As such, it's _possible_ we could end - * up with files getting yanked out from under us and the test failing, but with 4+ segments created by the default ref - * data population, the risk of that should be very low. Keep it in mind as time goes by. +/** + * Tests various alignments, offsets, and operations of the {@link ResumableCommitLogReader} */ public class ResumableCommitLogReaderTest extends CQLTester { - - @BeforeClass - public static void setUpClass() - { - DatabaseDescriptor.setCDCEnabled(true); - CQLTester.setUpClass(); - } + private CommitLogSegment populatedSegment; + private CommitLogTestUtils.MutationCountingReplayer testReplayer; @Before - public void beforeTest() throws Throwable + public void setUpTest() throws Throwable { - super.beforeTest(); - - // Start with a clean slate each test. Arguably could pre-populate and just use populated data; keep an eye - // on runtime for this test suite and group if that becomes a worthy time and complexity tradeoff. CommitLog.instance.resetUnsafe(true); + testReplayer = new CommitLogTestUtils.MutationCountingReplayer(); populateReferenceData(true); // Should have well more than 3 segments to work with on subsequent tests. Assert.assertTrue(CommitLog.instance.segmentManager.getSegmentsForUnflushedTables().size() > 3); + + // And always reset which file we're using as including CDC Mutations, since things may change between tests + CommitLogTestUtils.MutationCountingReplayer testReplayer = new CommitLogTestUtils.MutationCountingReplayer(); + for (CommitLogSegment cls : CommitLog.instance.segmentManager.getSegmentsForUnflushedTables()) + { + testReplayer.replaySingleFile(cls.logFile); + if (testReplayer.sawCDCMutation) + { + populatedSegment = cls; + return; + } + } + throw new RuntimeException("No mutations seen in passed in collection."); } /** * Expect operation as though non-resumable, read file to end and complete. */ @Test - public void testNonResumedGeneralCase() throws IOException + public void testNonResumedGeneralCase() throws Throwable { - File writtenFile = CommitLogTestUtils.getFilledCommitLogFile(); - Assert.assertTrue(writtenFile.length() > 0); - - CommitLogTestUtils.CDCMutationCountingReplayer testReplayer = new CommitLogTestUtils.CDCMutationCountingReplayer(); - testReplayer.replaySingleFile(writtenFile); + testReplayer.replaySingleFile(populatedSegment.logFile); Assert.assertTrue("Did not see any CDC enabled mutations.", testReplayer.sawCDCMutation); Assert.assertFalse("Saw a duplicate mutation while replaying a single file. This... shouldn't happen.", @@ -83,88 +82,380 @@ public void testNonResumedGeneralCase() throws IOException * Confirm our duplicate mutation testing infrastructure is working. */ @Test - public void testDuplicateCheckLogic() throws IOException + public void testDuplicateCheckLogic() throws Throwable { - File writtenFile = CommitLogTestUtils.getFilledCommitLogFile(); - Assert.assertTrue(writtenFile.length() > 0); - - CommitLogTestUtils.CDCMutationCountingReplayer testReplayer = new CommitLogTestUtils.CDCMutationCountingReplayer(); - testReplayer.replaySingleFile(writtenFile); + testReplayer.replaySingleFile(populatedSegment.logFile); Assert.assertTrue("Did not see any CDC enabled mutations.", testReplayer.sawCDCMutation); Assert.assertFalse("Saw a duplicate mutation while replaying a single file. This... shouldn't happen.", testReplayer.hasSeenDuplicateMutations()); - testReplayer.replaySingleFile(writtenFile); + testReplayer.replaySingleFile(populatedSegment.logFile); Assert.assertTrue("Expected to see duplicate mutations on 2nd replay of file.", testReplayer.hasSeenDuplicateMutations()); } /** - * Expect operation to pick up and feed total mutation # consistent w/mutations in CL + * Expect operation to pick up where left off and feed total mutation # consistent w/mutations in CL, so we do a + * single normal full replay, then a 2 step replay to ensure the total mutation count is as expected. */ @Test - public void testSingleResumeCase() + public void testSingleResumeCase() throws Throwable { + int expectedCount = getExpectedMutationCount(populatedSegment.logFile); + Assert.assertTrue("Failed to successfully perform a start to finish CL read.", expectedCount != 0); + + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + // This runs the risk of being flaky, since if we don't have CDC mutations in the first or second half, + // this test will fail out. This has been 100% stable at the .5 barrier both non and compressed, but keep in mind. + rr.readPartial((int)(rr.rawReader.length() * .5)); + Assert.assertFalse("Resumable Reader got constructed badly somehow.", rr.isClosed); + Assert.assertNotNull("Resumable Reader doesn't have a RAR cached in it as expected.", rr.rawReader); + + // Confirm we didn't just parse everything in the first part + Assert.assertNotEquals(expectedCount, testReplayer.seenMutations.size()); + + int interimCount = testReplayer.seenMutations.size(); + Assert.assertTrue("Failed on initial partial replay", interimCount != 0); + + Assert.assertFalse("Expected reader to be open still.", rr.isClosed()); + Assert.assertTrue("Interim replay should have played back less than a full replay. Check logs.", interimCount < expectedCount); + + rr.readPartial(CommitLogReader.READ_TO_END_OF_FILE); + Assert.assertEquals("Expected resumable read to give same # mutations as non but did not.", + testReplayer.seenMutations.size(), + expectedCount); + } } - /** - * Expect operation to pick up and feed total mutation # consistent w/mutations in CL, no errors. - */ + /** Test multiple resumes w/end not matching SyncSegment offsets. */ @Test - public void testResumeAtFileEndOffset() + public void testMultipleResumeNonAligned() throws Throwable { + int expectedCount = getExpectedMutationCount(populatedSegment.logFile); + + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + rr.readPartial(1024); + + // Sentinel to keep from locking the test. Since we are misusing the API by not sending the end sentinel, + // this checks to see if that kind of "bad offset overflow" gives us both a) a stable API, and b) the right + // parsed results from our end file. + int limit = 50; + + // throw some strange offsets at this and make sure it's robust to them, non-multiple of SyncSegment end + int offset = 1024 * 512; + while (limit > 0 && !rr.isClosed() && !rr.readToExhaustion) + { + rr.readPartial(offset); + offset += 1024 * 512; + --limit; + } + rr.readToCompletion(); + } + Assert.assertEquals("Expected non-aligned resumable read to give same # mutations as non but did not.", + expectedCount, + testReplayer.seenMutations.size()); } - /** - * If the offset provided by the user is past the end of the file itself, we expect an RTE - */ + /** Ensure that resumable readers w/offsets at sync segment boundaries don't blow up logic */ @Test - public void testOffsetPastEnd() throws RuntimeException + public void testResumingAtAlignedOffsets() throws Throwable { + int expectedCount = getExpectedMutationCount(populatedSegment.logFile); + + ArrayList segmentBoundaries = new ArrayList<>(); + // First, we want to get a list of all the segment boundaries out of the SegmentIterator + try(ResumableCommitLogReader fr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + fr.offsetLimit = CommitLogReader.READ_TO_END_OF_FILE; + CommitLogSegmentReader lsr = new CommitLogSegmentReader(fr); + for (CommitLogSegmentReader.SyncSegment ss : lsr) + { + segmentBoundaries.add(ss.endPosition); + } + Assert.assertTrue(segmentBoundaries.size() > 0); + } + Assert.assertNotEquals(-1, expectedCount); + testReplayer.reset(); + + // And now we iterate through the file using those boundaries, ensuring it works in our resumable reader. + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + // Confirm first read doesn't exhaust so the test is actually testing something. + rr.readPartial(segmentBoundaries.get(0)); + Assert.assertFalse(rr.readToExhaustion); + Assert.assertFalse(rr.isClosed); + + for (Integer offset : segmentBoundaries) + { + rr.readPartial(offset); + } + rr.readToCompletion(); + + Assert.assertEquals("Reading based on segment offsets produced unexpected results.", + expectedCount, + testReplayer.seenMutations.size()); + } } - /** - * If the offset provided by the user is < 0, we expect an RTE - */ + /** Expect operation to pick up and feed total mutation # consistent w/mutations in CL, no errors. */ @Test - public void testNegativeOffset() throws RuntimeException + public void testResumeAtFileEndOffset() throws Throwable { + int expectedCount = getExpectedMutationCount(populatedSegment.logFile); + + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + rr.readPartial((int) populatedSegment.logFile.length()); + Assert.assertEquals(expectedCount, testReplayer.seenMutations.size()); + } } /** - * If the offset is at Zero, more or less expect no-op since the user didn't ask us to read anything. Worth allowing - * this behavior in case offset / etc is written by someone to periodically poll and read to the CDC synced offset, so - * 0 is a valid / expected behavior in that case. + * If the offset provided by the user is past the end of the file itself, we expect a graceful read when using the logic + * on a resumable reader as we read to end of file. */ @Test - public void testOffsetAtZero() + public void testOffsetPastEnd() throws Throwable + { + int expectedCount = getExpectedMutationCount(populatedSegment.logFile); + + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + rr.readPartial(Integer.MAX_VALUE); + Assert.assertEquals(expectedCount, testReplayer.seenMutations.size()); + } + } + + /** If the offset provided by the user is < 0, we expect a no-op on read followed by the ability to resume and read */ + @Test + public void testNegativeOffset() throws Throwable { + int expectedCount = getExpectedMutationCount(populatedSegment.logFile); + + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + rr.readPartial(-1); + // Expect nothing to have been read so at beginning of file still + Assert.assertFalse(rr.readToExhaustion); + Assert.assertEquals(0, testReplayer.seenMutations.size()); + + // Then re-use the infra to read to end + rr.readPartial(CommitLogReader.READ_TO_END_OF_FILE); + Assert.assertEquals(expectedCount, testReplayer.seenMutations.size()); + Assert.assertEquals(0, testReplayer.duplicateMutations.size()); + } } /** - * Expect RTE if it's a corrupt input file. + * Confirm we gracefully handle cases where people may put in an offset that regresses what we're reading. Since the + * RAR and iteration should be unidirectional, we should still not see duplicates. */ @Test - public void testCorruptInputFile() + public void testRepeatOffsets() throws Throwable { + int expectedCount = getExpectedMutationCount(populatedSegment.logFile); + Assert.assertTrue(testReplayer.duplicateMutations.isEmpty()); + Assert.assertNotEquals(0, expectedCount); + + // Cache what we saw on first replay vs. newest, confirm >= all original seen + ArrayList ids = new ArrayList<>(testReplayer.seenMutations); + testReplayer.reset(); + + ArrayList newIds = null; + + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(populatedSegment.logFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + rr.readPartial(1024 * 512); + int countBeforeRegression = testReplayer.seenMutations.size(); + + // Confirm no reads if we regress our offset + rr.readPartial(1024 * 256); + Assert.assertEquals(0, testReplayer.duplicateMutations.size()); + Assert.assertEquals(countBeforeRegression, testReplayer.seenMutations.size()); + + // Confirm can resume w/correct offset + rr.readPartial(CommitLogReader.READ_TO_END_OF_FILE); + + newIds = new ArrayList<>(testReplayer.seenMutations); + + // Confirm all mutation id's seen in straight read are seen by partial. + for (Object id : ids) + if (!newIds.contains(id)) + Assert.fail(String.format("Missing id in resumable replay: %s", id)); + Assert.assertTrue(testReplayer.duplicateMutations.size() > 0); + } } /** - * Expect RTE if the input file is completely missing. + * Expect RTE if the input file is completely missing. Since these exceptions are user-facing in CL consumption in + * a CDC context, this unit test serves to calcify that UI interaction a bit and confirm we're deliberate about changing + * the type of exception we're throwing. */ @Test - public void testMissingInputFile() + public void testMissingInputFile() throws Throwable + { + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(new File("This_should_fail.txt"), + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { } + catch (RuntimeException e) + { + if (!e.getMessage().contains("version")) + Assert.fail(); + return; + } + Assert.fail("Expected RuntimeException on creation. Did not get it."); + } + + @Test + public void testWrongFile() throws Throwable { + File tempFile = File.createTempFile("test_file", ".tmp"); + try(FileWriter writer = new FileWriter(tempFile)) + { + writer.write("This is really not a commit log header. This will go badly."); + } + + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(tempFile, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { } + catch (RuntimeException rte) + { + if (rte.getMessage().contains("version")) + return; + } + Assert.fail("Expected RuntimeException complaining about inability to parse version from CommitLogHeader"); } /** - * Expect RTE if the input file exists and isn't a CLR. Overlaps w/"Corrupt", but want to make sure failure on header - * parsing is handled as well as failure once header is validated but something blows up mid-file. + * Uses CDC pipeline to read CDC index file + hard linked CDC File w/multiple mutations to confirm user use-case is functional. + * This test requires CDC to be enabled to run as we need the CDC Allocator to be hardlinking files, etc. + * + * This test is unique (and has surfaced multiple pain points) in that it is the only test that is processing + * a file being actively written. */ @Test - public void testWrongFile() + public void testUsingCDCOffsets() throws Throwable { + if (!(CommitLog.instance.segmentManager.segmentAllocator instanceof CommitLogSegmentAllocatorCDC)) + return; + + CommitLogSegment activeSegment = CommitLog.instance.segmentManager.getActiveSegment(); + Assert.assertSame(activeSegment.getCDCState(), CommitLogSegment.CDCState.CONTAINS); + + File cdcSegment = activeSegment.getCDCFile(); + Assert.assertTrue(cdcSegment.exists()); + + // Confirm we have a reasonable offset now and can scan to it + int cdcOffset = parseCDCOffset(activeSegment.getCDCIndexFile()); + Assert.assertNotEquals(0, cdcOffset); + + int totalReadThroughCDC = -1; + // Start a resumable reader wrapped around our active segment + try(ResumableCommitLogReader rr = new ResumableCommitLogReader(cdcSegment, + testReplayer.mutationHandler, + CommitLogPosition.NONE, + CommitLogReader.ALL_MUTATIONS, + true)) + { + rr.readPartial(cdcOffset); + int lastMutationsRead = testReplayer.seenMutations.size(); + Assert.assertNotEquals(0, lastMutationsRead); + + byte[] buffer = new byte[1024 * 128]; + + int tries = 250; + int newCDCOffset; + while (tries > 0) + { + cdcOffset = parseCDCOffset(activeSegment.getCDCIndexFile()); + writeReferenceLines(50, buffer); + CommitLog.instance.sync(true); + + newCDCOffset = parseCDCOffset(activeSegment.getCDCIndexFile()); + // Look for 1 cdcOffset change and read it. We stop here since we can't really deterministically get + // a repeatable test on # of writes leading to # of SyncSegment's written w/compression, etc. + if (cdcOffset != newCDCOffset) + { + rr.readPartial(newCDCOffset); + break; + } + tries--; + } + Assert.assertNotEquals(0, tries); + // Confirm we read far enough to exhaust the underlying buffer in compression context. This will exercise the RAR re-alloc code. + if (DatabaseDescriptor.getCommitLogCompression() != null) + Assert.assertTrue(rr.readToExhaustion); + + // Write a chunk more data; hopefully cycles it so we have a full file, though not particularly relevant to our needs. + buffer = new byte[1024 * 512]; + writeReferenceLines(150, buffer); + CommitLog.instance.sync(true); + + rr.readToCompletion(); + totalReadThroughCDC = testReplayer.seenMutations.size(); + } + + // Get final count with one straight through read, confirm matches CDC staged + testReplayer.reset(); + int expectedSeen = getExpectedMutationCount(cdcSegment); + Assert.assertEquals(expectedSeen, totalReadThroughCDC); } + /** Straightforward single non-resumable replay to count mutations expected. Assumes you're looking for > 0 */ + private int getExpectedMutationCount(File file) throws IOException + { + testReplayer.replaySingleFile(file); + int expectedCount = testReplayer.seenMutations.size(); + testReplayer.reset(); + Assert.assertTrue("Failed to successfully perform a start to finish CL read.", expectedCount != 0); + return expectedCount; + } + private int parseCDCOffset(File cdcIndexFile) throws IOException + { + try(BufferedReader br = new BufferedReader(new FileReader(cdcIndexFile))) + { + return Integer.parseInt(br.readLine()); + } + } } diff --git a/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java index ce209351fa62..7fdbf6f4f09f 100644 --- a/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java +++ b/test/unit/org/apache/cassandra/db/commitlog/SegmentReaderTest.java @@ -107,9 +107,9 @@ private void compressedSegmenter(ICompressor compressor) throws IOException fos.getChannel().write(compBuffer); fos.close(); - try (RandomAccessReader reader = RandomAccessReader.open(compressedFile)) + try (ResumableCommitLogReader rr = new ResumableCommitLogReader(compressedFile, new CommitLogTestUtils.NoopMutationHandler())) { - CompressedSegmenter segmenter = new CompressedSegmenter(compressor, reader); + CompressedSegmenter segmenter = new CompressedSegmenter(compressor, rr); int fileLength = (int) compressedFile.length(); SyncSegment syncSegment = segmenter.nextSegment(0, fileLength); FileDataInput fileDataInput = syncSegment.input; @@ -195,11 +195,11 @@ public void underlyingEncryptedSegmenterTest(BiFunction